diff --git a/sdks/go/cmd/beamctl/cmd/artifact.go b/sdks/go/cmd/beamctl/cmd/artifact.go index 1b5b7c9df8f9..b1ca18e96436 100644 --- a/sdks/go/cmd/beamctl/cmd/artifact.go +++ b/sdks/go/cmd/beamctl/cmd/artifact.go @@ -95,7 +95,7 @@ func listFn(cmd *cobra.Command, args []string) error { } for _, a := range md.GetManifest().GetArtifact() { - cmd.Println(a.Name) + cmd.Println(a.GetName()) } return nil } diff --git a/sdks/go/container/boot.go b/sdks/go/container/boot.go index 15f9ecc101cb..4170f8063014 100644 --- a/sdks/go/container/boot.go +++ b/sdks/go/container/boot.go @@ -151,7 +151,7 @@ func main() { enableGoogleCloudProfiler := strings.Contains(options, enableGoogleCloudProfilerOption) if enableGoogleCloudProfiler { - err := configureGoogleCloudProfilerEnvVars(ctx, logger, info.Metadata) + err := configureGoogleCloudProfilerEnvVars(ctx, logger, info.GetMetadata()) if err != nil { logger.Printf(ctx, "could not configure Google Cloud Profiler variables, got %v", err) } diff --git a/sdks/go/container/boot_test.go b/sdks/go/container/boot_test.go index 49c78047249e..97c0e4172dda 100644 --- a/sdks/go/container/boot_test.go +++ b/sdks/go/container/boot_test.go @@ -29,11 +29,11 @@ import ( ) func TestEnsureEndpointsSet_AllSet(t *testing.T) { - provisionInfo := &fnpb.ProvisionInfo{ - LoggingEndpoint: &pipepb.ApiServiceDescriptor{Url: "testLoggingEndpointUrl"}, - ArtifactEndpoint: &pipepb.ApiServiceDescriptor{Url: "testArtifactEndpointUrl"}, - ControlEndpoint: &pipepb.ApiServiceDescriptor{Url: "testControlEndpointUrl"}, - } + provisionInfo := fnpb.ProvisionInfo_builder{ + LoggingEndpoint: pipepb.ApiServiceDescriptor_builder{Url: "testLoggingEndpointUrl"}.Build(), + ArtifactEndpoint: pipepb.ApiServiceDescriptor_builder{Url: "testArtifactEndpointUrl"}.Build(), + ControlEndpoint: pipepb.ApiServiceDescriptor_builder{Url: "testControlEndpointUrl"}.Build(), + }.Build() *loggingEndpoint = "" *artifactEndpoint = "" *controlEndpoint = "" @@ -53,11 +53,11 @@ func TestEnsureEndpointsSet_AllSet(t *testing.T) { } func TestEnsureEndpointsSet_OneMissing(t *testing.T) { - provisionInfo := &fnpb.ProvisionInfo{ - LoggingEndpoint: &pipepb.ApiServiceDescriptor{Url: "testLoggingEndpointUrl"}, - ArtifactEndpoint: &pipepb.ApiServiceDescriptor{Url: "testArtifactEndpointUrl"}, - ControlEndpoint: &pipepb.ApiServiceDescriptor{Url: ""}, - } + provisionInfo := fnpb.ProvisionInfo_builder{ + LoggingEndpoint: pipepb.ApiServiceDescriptor_builder{Url: "testLoggingEndpointUrl"}.Build(), + ArtifactEndpoint: pipepb.ApiServiceDescriptor_builder{Url: "testArtifactEndpointUrl"}.Build(), + ControlEndpoint: pipepb.ApiServiceDescriptor_builder{Url: ""}.Build(), + }.Build() *loggingEndpoint = "" *artifactEndpoint = "" *controlEndpoint = "" @@ -85,7 +85,7 @@ func TestGetGoWorkerArtifactName_NoArtifacts(t *testing.T) { func TestGetGoWorkerArtifactName_OneArtifact(t *testing.T) { artifact := constructArtifactInformation(t, artifact.URNGoWorkerBinaryRole, "test/path", "sha") - artifacts := []*pipepb.ArtifactInformation{&artifact} + artifacts := []*pipepb.ArtifactInformation{artifact} val, err := getGoWorkerArtifactName(context.Background(), &tools.Logger{}, artifacts) if err != nil { @@ -99,7 +99,7 @@ func TestGetGoWorkerArtifactName_OneArtifact(t *testing.T) { func TestGetGoWorkerArtifactName_MultipleArtifactsFirstIsWorker(t *testing.T) { artifact1 := constructArtifactInformation(t, artifact.URNGoWorkerBinaryRole, "test/path", "sha") artifact2 := constructArtifactInformation(t, "other role", "test/path2", "sha") - artifacts := []*pipepb.ArtifactInformation{&artifact1, &artifact2} + artifacts := []*pipepb.ArtifactInformation{artifact1, artifact2} val, err := getGoWorkerArtifactName(context.Background(), &tools.Logger{}, artifacts) if err != nil { @@ -113,7 +113,7 @@ func TestGetGoWorkerArtifactName_MultipleArtifactsFirstIsWorker(t *testing.T) { func TestGetGoWorkerArtifactName_MultipleArtifactsSecondIsWorker(t *testing.T) { artifact1 := constructArtifactInformation(t, "other role", "test/path", "sha") artifact2 := constructArtifactInformation(t, artifact.URNGoWorkerBinaryRole, "test/path2", "sha") - artifacts := []*pipepb.ArtifactInformation{&artifact1, &artifact2} + artifacts := []*pipepb.ArtifactInformation{artifact1, artifact2} val, err := getGoWorkerArtifactName(context.Background(), &tools.Logger{}, artifacts) if err != nil { @@ -127,7 +127,7 @@ func TestGetGoWorkerArtifactName_MultipleArtifactsSecondIsWorker(t *testing.T) { func TestGetGoWorkerArtifactName_MultipleArtifactsLegacyWay(t *testing.T) { artifact1 := constructArtifactInformation(t, "other role", "test/path", "sha") artifact2 := constructArtifactInformation(t, "other role", "worker", "sha") - artifacts := []*pipepb.ArtifactInformation{&artifact1, &artifact2} + artifacts := []*pipepb.ArtifactInformation{artifact1, artifact2} val, err := getGoWorkerArtifactName(context.Background(), &tools.Logger{}, artifacts) if err != nil { @@ -141,7 +141,7 @@ func TestGetGoWorkerArtifactName_MultipleArtifactsLegacyWay(t *testing.T) { func TestGetGoWorkerArtifactName_MultipleArtifactsNoneMatch(t *testing.T) { artifact1 := constructArtifactInformation(t, "other role", "test/path", "sha") artifact2 := constructArtifactInformation(t, "other role", "test/path2", "sha") - artifacts := []*pipepb.ArtifactInformation{&artifact1, &artifact2} + artifacts := []*pipepb.ArtifactInformation{artifact1, artifact2} _, err := getGoWorkerArtifactName(context.Background(), &tools.Logger{}, artifacts) if err == nil { @@ -193,16 +193,16 @@ func TestCopyExe(t *testing.T) { } } -func constructArtifactInformation(t *testing.T, roleUrn string, path string, sha string) pipepb.ArtifactInformation { +func constructArtifactInformation(t *testing.T, roleUrn string, path string, sha string) *pipepb.ArtifactInformation { t.Helper() - typePayload, _ := proto.Marshal(&pipepb.ArtifactFilePayload{Path: path, Sha256: sha}) + typePayload, _ := proto.Marshal(pipepb.ArtifactFilePayload_builder{Path: path, Sha256: sha}.Build()) - return pipepb.ArtifactInformation{ + return pipepb.ArtifactInformation_builder{ RoleUrn: roleUrn, TypeUrn: artifact.URNFileArtifact, TypePayload: typePayload, - } + }.Build() } func TestConfigureGoogleCloudProfilerEnvVars(t *testing.T) { diff --git a/sdks/go/container/tools/buffered_logging_test.go b/sdks/go/container/tools/buffered_logging_test.go index 9f542d2d5ab6..9f11cf319c4d 100644 --- a/sdks/go/container/tools/buffered_logging_test.go +++ b/sdks/go/container/tools/buffered_logging_test.go @@ -63,11 +63,11 @@ func TestBufferedLogger(t *testing.T) { received := catcher.msgs[0].GetLogEntries()[0] - if got, want := received.Message, "test message"; got != want { + if got, want := received.GetMessage(), "test message"; got != want { t.Errorf("got message %q, want %q", got, want) } - if got, want := received.Severity, fnpb.LogEntry_Severity_DEBUG; got != want { + if got, want := received.GetSeverity(), fnpb.LogEntry_Severity_DEBUG; got != want { t.Errorf("got severity %v, want %v", got, want) } }) @@ -96,11 +96,11 @@ func TestBufferedLogger(t *testing.T) { received := catcher.msgs[0].GetLogEntries() for i, message := range received { - if got, want := message.Message, messages[i]; got != want { + if got, want := message.GetMessage(), messages[i]; got != want { t.Errorf("got message %q, want %q", got, want) } - if got, want := message.Severity, fnpb.LogEntry_Severity_DEBUG; got != want { + if got, want := message.GetSeverity(), fnpb.LogEntry_Severity_DEBUG; got != want { t.Errorf("got severity %v, want %v", got, want) } } @@ -125,11 +125,11 @@ func TestBufferedLogger(t *testing.T) { received := catcher.msgs[0].GetLogEntries()[0] - if got, want := received.Message, "test error"; got != want { + if got, want := received.GetMessage(), "test error"; got != want { t.Errorf("got message %q, want %q", got, want) } - if got, want := received.Severity, fnpb.LogEntry_Severity_ERROR; got != want { + if got, want := received.GetSeverity(), fnpb.LogEntry_Severity_ERROR; got != want { t.Errorf("got severity %v, want %v", got, want) } }) @@ -158,11 +158,11 @@ func TestBufferedLogger(t *testing.T) { received := catcher.msgs[0].GetLogEntries() for i, message := range received { - if got, want := message.Message, messages[i]; got != want { + if got, want := message.GetMessage(), messages[i]; got != want { t.Errorf("got message %q, want %q", got, want) } - if got, want := message.Severity, fnpb.LogEntry_Severity_ERROR; got != want { + if got, want := message.GetSeverity(), fnpb.LogEntry_Severity_ERROR; got != want { t.Errorf("got severity %v, want %v", got, want) } } @@ -177,11 +177,11 @@ func TestBufferedLogger(t *testing.T) { received := catcher.msgs[0].GetLogEntries()[0] - if got, want := received.Message, "foo bar"; got != want { + if got, want := received.GetMessage(), "foo bar"; got != want { t.Errorf("l.Printf(\"foo %%v\", \"bar\"): got message %q, want %q", got, want) } - if got, want := received.Severity, fnpb.LogEntry_Severity_DEBUG; got != want { + if got, want := received.GetSeverity(), fnpb.LogEntry_Severity_DEBUG; got != want { t.Errorf("l.Printf(\"foo %%v\", \"bar\"): got severity %v, want %v", got, want) } }) @@ -229,11 +229,11 @@ func TestBufferedLogger(t *testing.T) { messages = append(messages, lastMessage) for i, message := range received { - if got, want := message.Message, messages[i]; got != want { + if got, want := message.GetMessage(), messages[i]; got != want { t.Errorf("got message %q, want %q", got, want) } - if got, want := message.Severity, fnpb.LogEntry_Severity_DEBUG; got != want { + if got, want := message.GetSeverity(), fnpb.LogEntry_Severity_DEBUG; got != want { t.Errorf("got severity %v, want %v", got, want) } } diff --git a/sdks/go/container/tools/logging.go b/sdks/go/container/tools/logging.go index ced13b744d45..ed0f7b82c9f9 100644 --- a/sdks/go/container/tools/logging.go +++ b/sdks/go/container/tools/logging.go @@ -86,15 +86,15 @@ func (l *Logger) Log(ctx context.Context, sev fnpb.LogEntry_Severity_Enum, messa } } - err := l.client.Send(&fnpb.LogEntry_List{ + err := l.client.Send(fnpb.LogEntry_List_builder{ LogEntries: []*fnpb.LogEntry{ - { + fnpb.LogEntry_builder{ Severity: sev, Timestamp: timestamppb.Now(), Message: message, - }, + }.Build(), }, - }) + }.Build()) if err != nil { exitErr = err return diff --git a/sdks/go/container/tools/logging_test.go b/sdks/go/container/tools/logging_test.go index 8730a0fe9c19..b837b40fa24e 100644 --- a/sdks/go/container/tools/logging_test.go +++ b/sdks/go/container/tools/logging_test.go @@ -50,11 +50,11 @@ func TestLogger(t *testing.T) { received := catcher.msgs[0].GetLogEntries()[0] - if got, want := received.Message, "foo bar"; got != want { + if got, want := received.GetMessage(), "foo bar"; got != want { t.Errorf("l.Printf(\"foo %%v\", \"bar\"): got message %q, want %q", got, want) } - if got, want := received.Severity, fnpb.LogEntry_Severity_DEBUG; got != want { + if got, want := received.GetSeverity(), fnpb.LogEntry_Severity_DEBUG; got != want { t.Errorf("l.Printf(\"foo %%v\", \"bar\"): got severity %v, want %v", got, want) } }) @@ -66,11 +66,11 @@ func TestLogger(t *testing.T) { received := catcher.msgs[0].GetLogEntries()[0] - if got, want := received.Message, "failed to install dependency bar"; got != want { + if got, want := received.GetMessage(), "failed to install dependency bar"; got != want { t.Errorf("l.Printf(\"foo %%v\", \"bar\"): got message %q, want %q", got, want) } - if got, want := received.Severity, fnpb.LogEntry_Severity_ERROR; got != want { + if got, want := received.GetSeverity(), fnpb.LogEntry_Severity_ERROR; got != want { t.Errorf("l.Errorf(\"failed to install dependency %%v\", \"bar\"): got severity %v, want %v", got, want) } }) diff --git a/sdks/go/container/tools/provision_test.go b/sdks/go/container/tools/provision_test.go index dbb78040632f..1820092c0cfc 100644 --- a/sdks/go/container/tools/provision_test.go +++ b/sdks/go/container/tools/provision_test.go @@ -66,7 +66,7 @@ type ProvisionServiceServicer struct { } func (p ProvisionServiceServicer) GetProvisionInfo(ctx context.Context, req *fnpb.GetProvisionInfoRequest) (*fnpb.GetProvisionInfoResponse, error) { - return &fnpb.GetProvisionInfoResponse{Info: &fnpb.ProvisionInfo{RetrievalToken: "token"}}, nil + return fnpb.GetProvisionInfoResponse_builder{Info: fnpb.ProvisionInfo_builder{RetrievalToken: "token"}.Build()}.Build(), nil } func setup(addr *string, wg *sync.WaitGroup) { @@ -103,7 +103,7 @@ func TestProvisionInfo(t *testing.T) { if err != nil { t.Errorf("error in response: %v", err) } - want := &fnpb.ProvisionInfo{RetrievalToken: "token"} + want := fnpb.ProvisionInfo_builder{RetrievalToken: "token"}.Build() if got.GetRetrievalToken() != want.GetRetrievalToken() { t.Errorf("provision.Info() = %v, want %v", got, want) } diff --git a/sdks/go/examples/snippets/06schemas_test.go b/sdks/go/examples/snippets/06schemas_test.go index 325a96274c2e..df8799cc1413 100644 --- a/sdks/go/examples/snippets/06schemas_test.go +++ b/sdks/go/examples/snippets/06schemas_test.go @@ -31,55 +31,49 @@ import ( ) func atomicSchemaField(name string, typ pipepb.AtomicType) *pipepb.Field { - return &pipepb.Field{ + return pipepb.Field_builder{ Name: name, - Type: &pipepb.FieldType{ - TypeInfo: &pipepb.FieldType_AtomicType{ - AtomicType: typ, - }, - }, - } + Type: pipepb.FieldType_builder{ + AtomicType: typ.Enum(), + }.Build(), + }.Build() } func rowSchemaField(name string, typ *pipepb.Schema) *pipepb.Field { - return &pipepb.Field{ + return pipepb.Field_builder{ Name: name, - Type: &pipepb.FieldType{ - TypeInfo: &pipepb.FieldType_RowType{ - RowType: &pipepb.RowType{ - Schema: typ, - }, - }, - }, - } + Type: pipepb.FieldType_builder{ + RowType: pipepb.RowType_builder{ + Schema: typ, + }.Build(), + }.Build(), + }.Build() } func listSchemaField(name string, typ *pipepb.Field) *pipepb.Field { - return &pipepb.Field{ + return pipepb.Field_builder{ Name: name, - Type: &pipepb.FieldType{ - TypeInfo: &pipepb.FieldType_ArrayType{ - ArrayType: &pipepb.ArrayType{ - ElementType: typ.GetType(), - }, - }, - }, - } + Type: pipepb.FieldType_builder{ + ArrayType: pipepb.ArrayType_builder{ + ElementType: typ.GetType(), + }.Build(), + }.Build(), + }.Build() } func nillable(f *pipepb.Field) *pipepb.Field { - f.Type.Nullable = true + f.GetType().SetNullable(true) return f } func TestSchemaTypes(t *testing.T) { - transactionSchema := &pipepb.Schema{ + transactionSchema := pipepb.Schema_builder{ Fields: []*pipepb.Field{ atomicSchemaField("bank", pipepb.AtomicType_STRING), atomicSchemaField("purchaseAmount", pipepb.AtomicType_DOUBLE), }, - } - shippingAddressSchema := &pipepb.Schema{ + }.Build() + shippingAddressSchema := pipepb.Schema_builder{ Fields: []*pipepb.Field{ atomicSchemaField("streetAddress", pipepb.AtomicType_STRING), atomicSchemaField("city", pipepb.AtomicType_STRING), @@ -87,7 +81,7 @@ func TestSchemaTypes(t *testing.T) { atomicSchemaField("country", pipepb.AtomicType_STRING), atomicSchemaField("postCode", pipepb.AtomicType_STRING), }, - } + }.Build() tests := []struct { rt reflect.Type @@ -101,7 +95,7 @@ func TestSchemaTypes(t *testing.T) { st: shippingAddressSchema, }, { rt: reflect.TypeOf(Purchase{}), - st: &pipepb.Schema{ + st: pipepb.Schema_builder{ Fields: []*pipepb.Field{ atomicSchemaField("userId", pipepb.AtomicType_STRING), atomicSchemaField("itemId", pipepb.AtomicType_INT64), @@ -110,15 +104,15 @@ func TestSchemaTypes(t *testing.T) { listSchemaField("transactions", rowSchemaField("n/a", transactionSchema)), }, - }, + }.Build(), }, { rt: tnType, - st: &pipepb.Schema{ + st: pipepb.Schema_builder{ Fields: []*pipepb.Field{ atomicSchemaField("seconds", pipepb.AtomicType_INT64), atomicSchemaField("nanos", pipepb.AtomicType_INT32), }, - }, + }.Build(), preReg: func(reg *schema.Registry) { reg.RegisterLogicalType(schema.ToLogicalType(tnType.Name(), tnType, tnStorageType)) }, diff --git a/sdks/go/pkg/beam/artifact/gcsproxy/retrieval.go b/sdks/go/pkg/beam/artifact/gcsproxy/retrieval.go index ceb8a319be98..602af701b222 100644 --- a/sdks/go/pkg/beam/artifact/gcsproxy/retrieval.go +++ b/sdks/go/pkg/beam/artifact/gcsproxy/retrieval.go @@ -75,7 +75,7 @@ func NewRetrievalServer(md *jobpb.ProxyManifest) (*RetrievalServer, error) { // GetManifest returns the manifest for all artifacts. func (s *RetrievalServer) GetManifest(ctx context.Context, req *jobpb.GetManifestRequest) (*jobpb.GetManifestResponse, error) { - return &jobpb.GetManifestResponse{Manifest: s.md}, nil + return jobpb.GetManifestResponse_builder{Manifest: s.md}.Build(), nil } // GetArtifact returns a given artifact. @@ -105,7 +105,7 @@ func (s *RetrievalServer) GetArtifact(req *jobpb.LegacyGetArtifactRequest, strea for { n, err := r.Read(data) if n > 0 { - if err := stream.Send(&jobpb.ArtifactChunk{Data: data[:n]}); err != nil { + if err := stream.Send(jobpb.ArtifactChunk_builder{Data: data[:n]}.Build()); err != nil { return errors.Wrap(err, "chunk send failed") } } @@ -122,20 +122,20 @@ func (s *RetrievalServer) GetArtifact(req *jobpb.LegacyGetArtifactRequest, strea func validate(md *jobpb.ProxyManifest) error { keys := make(map[string]bool) for _, a := range md.GetManifest().GetArtifact() { - if _, seen := keys[a.Name]; seen { - return errors.Errorf("multiple artifact with name %v", a.Name) + if _, seen := keys[a.GetName()]; seen { + return errors.Errorf("multiple artifact with name %v", a.GetName()) } - keys[a.Name] = true + keys[a.GetName()] = true } for _, l := range md.GetLocation() { - fresh, seen := keys[l.Name] + fresh, seen := keys[l.GetName()] if !seen { - return errors.Errorf("no artifact named %v for location %v", l.Name, l.Uri) + return errors.Errorf("no artifact named %v for location %v", l.GetName(), l.GetUri()) } if !fresh { - return errors.Errorf("multiple locations for %v:%v", l.Name, l.Uri) + return errors.Errorf("multiple locations for %v:%v", l.GetName(), l.GetUri()) } - keys[l.Name] = false + keys[l.GetName()] = false } for key, fresh := range keys { diff --git a/sdks/go/pkg/beam/artifact/gcsproxy/staging.go b/sdks/go/pkg/beam/artifact/gcsproxy/staging.go index 9113e780f339..4031d94c4df1 100644 --- a/sdks/go/pkg/beam/artifact/gcsproxy/staging.go +++ b/sdks/go/pkg/beam/artifact/gcsproxy/staging.go @@ -75,7 +75,7 @@ func (s *StagingServer) CommitManifest(ctx context.Context, req *jobpb.CommitMan } s.mu.Unlock() - data, err := proto.Marshal(&jobpb.ProxyManifest{Manifest: manifest, Location: loc}) + data, err := proto.Marshal(jobpb.ProxyManifest_builder{Manifest: manifest, Location: loc}.Build()) if err != nil { return nil, errors.Wrap(err, "failed to marshal proxy manifest") } @@ -93,7 +93,7 @@ func (s *StagingServer) CommitManifest(ctx context.Context, req *jobpb.CommitMan // now, but would be needed for a staging server that serves multiple // jobs. Such a server would also use the ID sent with each request. - return &jobpb.CommitManifestResponse{RetrievalToken: gcsx.MakeObject(s.bucket, s.manifest)}, nil + return jobpb.CommitManifestResponse_builder{RetrievalToken: gcsx.MakeObject(s.bucket, s.manifest)}.Build(), nil } // matchLocations ensures that all artifacts have been staged and have valid @@ -101,18 +101,18 @@ func (s *StagingServer) CommitManifest(ctx context.Context, req *jobpb.CommitMan func matchLocations(artifacts []*jobpb.ArtifactMetadata, blobs map[string]staged) ([]*jobpb.ProxyManifest_Location, error) { var loc []*jobpb.ProxyManifest_Location for _, a := range artifacts { - info, ok := blobs[a.Name] + info, ok := blobs[a.GetName()] if !ok { - return nil, errors.Errorf("artifact %v not staged", a.Name) + return nil, errors.Errorf("artifact %v not staged", a.GetName()) } - if a.Sha256 == "" { - a.Sha256 = info.hash + if a.GetSha256() == "" { + a.SetSha256(info.hash) } - if info.hash != a.Sha256 { - return nil, errors.Errorf("staged artifact for %v has invalid SHA256: %v, want %v", a.Name, info.hash, a.Sha256) + if info.hash != a.GetSha256() { + return nil, errors.Errorf("staged artifact for %v has invalid SHA256: %v, want %v", a.GetName(), info.hash, a.GetSha256()) } - loc = append(loc, &jobpb.ProxyManifest_Location{Name: a.Name, Uri: info.object}) + loc = append(loc, jobpb.ProxyManifest_Location_builder{Name: a.GetName(), Uri: info.object}.Build()) } return loc, nil } @@ -129,7 +129,7 @@ func (s *StagingServer) PutArtifact(ps jobpb.LegacyArtifactStagingService_PutArt if md == nil { return errors.Errorf("expected header as first message: %v", header) } - object := path.Join(s.root, md.Name) + object := path.Join(s.root, md.GetName()) // Stream content to GCS. We don't have to worry about partial // or abandoned writes, because object writes are atomic. @@ -142,15 +142,15 @@ func (s *StagingServer) PutArtifact(ps jobpb.LegacyArtifactStagingService_PutArt r := &reader{sha256W: sha256.New(), stream: ps} if err := gcsx.WriteObject(ctx, cl, s.bucket, object, r); err != nil { - return errors.Wrapf(err, "failed to stage artifact %v", md.Name) + return errors.Wrapf(err, "failed to stage artifact %v", md.GetName()) } hash := r.SHA256() - if md.Sha256 != "" && md.Sha256 != hash { - return errors.Errorf("invalid SHA256 for artifact %v: %v want %v", md.Name, hash, md.Sha256) + if md.GetSha256() != "" && md.GetSha256() != hash { + return errors.Errorf("invalid SHA256 for artifact %v: %v want %v", md.GetName(), hash, md.GetSha256()) } s.mu.Lock() - s.blobs[md.Name] = staged{object: gcsx.MakeObject(s.bucket, object), hash: hash} + s.blobs[md.GetName()] = staged{object: gcsx.MakeObject(s.bucket, object), hash: hash} s.mu.Unlock() return ps.SendAndClose(&jobpb.PutArtifactResponse{}) diff --git a/sdks/go/pkg/beam/artifact/materialize.go b/sdks/go/pkg/beam/artifact/materialize.go index 624e30efcd2b..e955941061da 100644 --- a/sdks/go/pkg/beam/artifact/materialize.go +++ b/sdks/go/pkg/beam/artifact/materialize.go @@ -78,59 +78,59 @@ func newMaterialize(ctx context.Context, endpoint string, dependencies []*pipepb } func newMaterializeWithClient(ctx context.Context, client jobpb.ArtifactRetrievalServiceClient, dependencies []*pipepb.ArtifactInformation, dest string) ([]*pipepb.ArtifactInformation, error) { - resolution, err := client.ResolveArtifacts(ctx, &jobpb.ResolveArtifactsRequest{Artifacts: dependencies}) + resolution, err := client.ResolveArtifacts(ctx, jobpb.ResolveArtifactsRequest_builder{Artifacts: dependencies}.Build()) if err != nil { return nil, err } var artifacts []*pipepb.ArtifactInformation var list []retrievable - for _, dep := range resolution.Replacements { + for _, dep := range resolution.GetReplacements() { path, err := extractStagingToPath(dep) if err != nil { return nil, err } - filePayload := pipepb.ArtifactFilePayload{ + filePayload := pipepb.ArtifactFilePayload_builder{ Path: path, - } - if dep.TypeUrn == URNFileArtifact { - typePayload := pipepb.ArtifactFilePayload{} - if err := proto.Unmarshal(dep.TypePayload, &typePayload); err != nil { + }.Build() + if dep.GetTypeUrn() == URNFileArtifact { + typePayload := &pipepb.ArtifactFilePayload{} + if err := proto.Unmarshal(dep.GetTypePayload(), typePayload); err != nil { return nil, errors.Wrap(err, "failed to parse artifact file payload") } - filePayload.Sha256 = typePayload.Sha256 - } else if dep.TypeUrn == URNUrlArtifact { - typePayload := pipepb.ArtifactUrlPayload{} - if err := proto.Unmarshal(dep.TypePayload, &typePayload); err != nil { + filePayload.SetSha256(typePayload.GetSha256()) + } else if dep.GetTypeUrn() == URNUrlArtifact { + typePayload := &pipepb.ArtifactUrlPayload{} + if err := proto.Unmarshal(dep.GetTypePayload(), typePayload); err != nil { return nil, errors.Wrap(err, "failed to parse artifact url payload") } - filePayload.Sha256 = typePayload.Sha256 + filePayload.SetSha256(typePayload.GetSha256()) } - newTypePayload, err := proto.Marshal(&filePayload) + newTypePayload, err := proto.Marshal(filePayload) if err != nil { return nil, errors.Wrap(err, "failed to create artifact type payload") } - artifacts = append(artifacts, &pipepb.ArtifactInformation{ + artifacts = append(artifacts, pipepb.ArtifactInformation_builder{ TypeUrn: URNFileArtifact, TypePayload: newTypePayload, - RoleUrn: dep.RoleUrn, - RolePayload: dep.RolePayload, - }) + RoleUrn: dep.GetRoleUrn(), + RolePayload: dep.GetRolePayload(), + }.Build()) - rolePayload, err := proto.Marshal(&pipepb.ArtifactStagingToRolePayload{ + rolePayload, err := proto.Marshal(pipepb.ArtifactStagingToRolePayload_builder{ StagedName: path, - }) + }.Build()) if err != nil { return nil, errors.Wrap(err, "failed to create artifact role payload") } list = append(list, &artifact{ client: client, - dep: &pipepb.ArtifactInformation{ - TypeUrn: dep.TypeUrn, - TypePayload: dep.TypePayload, + dep: pipepb.ArtifactInformation_builder{ + TypeUrn: dep.GetTypeUrn(), + TypePayload: dep.GetTypePayload(), RoleUrn: URNStagingTo, RolePayload: rolePayload, - }, + }.Build(), }) } @@ -147,39 +147,39 @@ func generateId() string { func extractStagingToPath(artifact *pipepb.ArtifactInformation) (string, error) { var stagedName string - if artifact.RoleUrn == URNStagingTo { - role := pipepb.ArtifactStagingToRolePayload{} - if err := proto.Unmarshal(artifact.RolePayload, &role); err != nil { + if artifact.GetRoleUrn() == URNStagingTo { + role := &pipepb.ArtifactStagingToRolePayload{} + if err := proto.Unmarshal(artifact.GetRolePayload(), role); err != nil { return "", err } - stagedName = role.StagedName - } else if artifact.TypeUrn == URNFileArtifact { - ty := pipepb.ArtifactFilePayload{} - if err := proto.Unmarshal(artifact.TypePayload, &ty); err != nil { + stagedName = role.GetStagedName() + } else if artifact.GetTypeUrn() == URNFileArtifact { + ty := &pipepb.ArtifactFilePayload{} + if err := proto.Unmarshal(artifact.GetTypePayload(), ty); err != nil { return "", err } - stagedName = generateId() + "-" + filepath.Base(ty.Path) - } else if artifact.TypeUrn == URNUrlArtifact { - ty := pipepb.ArtifactUrlPayload{} - if err := proto.Unmarshal(artifact.TypePayload, &ty); err != nil { + stagedName = generateId() + "-" + filepath.Base(ty.GetPath()) + } else if artifact.GetTypeUrn() == URNUrlArtifact { + ty := &pipepb.ArtifactUrlPayload{} + if err := proto.Unmarshal(artifact.GetTypePayload(), ty); err != nil { return "", err } - stagedName = generateId() + "-" + path.Base(ty.Url) + stagedName = generateId() + "-" + path.Base(ty.GetUrl()) } else { - return "", errors.Errorf("failed to extract staging path for artifact type %v role %v", artifact.TypeUrn, artifact.RoleUrn) + return "", errors.Errorf("failed to extract staging path for artifact type %v role %v", artifact.GetTypeUrn(), artifact.GetRoleUrn()) } return stagedName, nil } func MustExtractFilePayload(artifact *pipepb.ArtifactInformation) (string, string) { - if artifact.TypeUrn != URNFileArtifact { - log.Fatalf("Unsupported artifact type %v", artifact.TypeUrn) + if artifact.GetTypeUrn() != URNFileArtifact { + log.Fatalf("Unsupported artifact type %v", artifact.GetTypeUrn()) } - ty := pipepb.ArtifactFilePayload{} - if err := proto.Unmarshal(artifact.TypePayload, &ty); err != nil { + ty := &pipepb.ArtifactFilePayload{} + if err := proto.Unmarshal(artifact.GetTypePayload(), ty); err != nil { log.Fatalf("failed to parse artifact file payload: %v", err) } - return ty.Path, ty.Sha256 + return ty.GetPath(), ty.GetSha256() } type artifact struct { @@ -208,7 +208,7 @@ func (a artifact) retrieve(ctx context.Context, dest string) error { return err } - stream, err := a.client.GetArtifact(ctx, &jobpb.GetArtifactRequest{Artifact: a.dep}) + stream, err := a.client.GetArtifact(ctx, jobpb.GetArtifactRequest_builder{Artifact: a.dep}.Build()) if err != nil { return err } @@ -244,10 +244,10 @@ func writeChunks(stream jobpb.ArtifactRetrievalService_GetArtifactClient, w io.W if err != nil { return "", err } - if _, err := sha256W.Write(chunk.Data); err != nil { + if _, err := sha256W.Write(chunk.GetData()); err != nil { panic(err) // cannot fail } - if _, err := w.Write(chunk.Data); err != nil { + if _, err := w.Write(chunk.GetData()); err != nil { return "", errors.Wrapf(err, "chunk write failed") } } @@ -263,7 +263,7 @@ func legacyMaterialize(ctx context.Context, endpoint string, rt string, dest str client := jobpb.NewLegacyArtifactRetrievalServiceClient(cc) - m, err := client.GetManifest(ctx, &jobpb.GetManifestRequest{RetrievalToken: rt}) + m, err := client.GetManifest(ctx, jobpb.GetManifestRequest_builder{RetrievalToken: rt}.Build()) if err != nil { return nil, errors.Wrap(err, "failed to get manifest") } @@ -272,25 +272,25 @@ func legacyMaterialize(ctx context.Context, endpoint string, rt string, dest str var artifacts []*pipepb.ArtifactInformation var list []retrievable for _, md := range mds { - typePayload, err := proto.Marshal(&pipepb.ArtifactFilePayload{ - Path: md.Name, - Sha256: md.Sha256, - }) + typePayload, err := proto.Marshal(pipepb.ArtifactFilePayload_builder{ + Path: md.GetName(), + Sha256: md.GetSha256(), + }.Build()) if err != nil { return nil, errors.Wrap(err, "failed to create artifact type payload") } - rolePayload, err := proto.Marshal(&pipepb.ArtifactStagingToRolePayload{ - StagedName: md.Name, - }) + rolePayload, err := proto.Marshal(pipepb.ArtifactStagingToRolePayload_builder{ + StagedName: md.GetName(), + }.Build()) if err != nil { return nil, errors.Wrap(err, "failed to create artifact role payload") } - artifacts = append(artifacts, &pipepb.ArtifactInformation{ + artifacts = append(artifacts, pipepb.ArtifactInformation_builder{ TypeUrn: URNFileArtifact, TypePayload: typePayload, RoleUrn: URNStagingTo, RolePayload: rolePayload, - }) + }.Build()) list = append(list, &legacyArtifact{ client: client, rt: rt, @@ -382,7 +382,7 @@ func (a legacyArtifact) retrieve(ctx context.Context, dest string) error { // previous retrieval attempt and may leave a corrupt/partial local file on // failure. func Retrieve(ctx context.Context, client jobpb.LegacyArtifactRetrievalServiceClient, a *jobpb.ArtifactMetadata, rt string, dest string) error { - filename := filepath.Join(dest, filepath.FromSlash(a.Name)) + filename := filepath.Join(dest, filepath.FromSlash(a.GetName())) _, err := os.Stat(filename) if err != nil && !os.IsNotExist(err) { @@ -392,7 +392,7 @@ func Retrieve(ctx context.Context, client jobpb.LegacyArtifactRetrievalServiceCl // File already exists. Validate or delete. hash, err := computeSHA256(filename) - if err == nil && a.Sha256 == hash { + if err == nil && a.GetSha256() == hash { // NOTE(herohde) 10/5/2017: We ignore permissions here, because // they may differ from the requested permissions due to umask // settings on unix systems (which we in turn want to respect). @@ -417,12 +417,12 @@ func Retrieve(ctx context.Context, client jobpb.LegacyArtifactRetrievalServiceCl // It expects the file to not exist, but does not clean up on failure and // may leave a corrupt file. func retrieve(ctx context.Context, client jobpb.LegacyArtifactRetrievalServiceClient, a *jobpb.ArtifactMetadata, rt string, filename string) error { - stream, err := client.GetArtifact(ctx, &jobpb.LegacyGetArtifactRequest{Name: a.Name, RetrievalToken: rt}) + stream, err := client.GetArtifact(ctx, jobpb.LegacyGetArtifactRequest_builder{Name: a.GetName(), RetrievalToken: rt}.Build()) if err != nil { return err } - fd, err := os.OpenFile(filename, os.O_CREATE|os.O_TRUNC|os.O_WRONLY, os.FileMode(a.Permissions)) + fd, err := os.OpenFile(filename, os.O_CREATE|os.O_TRUNC|os.O_WRONLY, os.FileMode(a.GetPermissions())) if err != nil { return err } @@ -442,8 +442,8 @@ func retrieve(ctx context.Context, client jobpb.LegacyArtifactRetrievalServiceCl } // Artifact Sha256 hash is an optional field in metadata so we should only validate when its present. - if a.Sha256 != "" && sha256Hash != a.Sha256 { - return errors.Errorf("bad SHA256 for %v: %v, want %v", filename, sha256Hash, a.Sha256) + if a.GetSha256() != "" && sha256Hash != a.GetSha256() { + return errors.Errorf("bad SHA256 for %v: %v, want %v", filename, sha256Hash, a.GetSha256()) } return nil } @@ -459,10 +459,10 @@ func retrieveChunks(stream jobpb.LegacyArtifactRetrievalService_GetArtifactClien return "", err } - if _, err := sha256W.Write(chunk.Data); err != nil { + if _, err := sha256W.Write(chunk.GetData()); err != nil { panic(err) // cannot fail } - if _, err := w.Write(chunk.Data); err != nil { + if _, err := w.Write(chunk.GetData()); err != nil { return "", errors.Wrapf(err, "chunk write failed") } } diff --git a/sdks/go/pkg/beam/artifact/materialize_test.go b/sdks/go/pkg/beam/artifact/materialize_test.go index 31890ed045cc..2eb3fb6c7a70 100644 --- a/sdks/go/pkg/beam/artifact/materialize_test.go +++ b/sdks/go/pkg/beam/artifact/materialize_test.go @@ -49,12 +49,12 @@ func TestRetrieve(t *testing.T) { client := jobpb.NewLegacyArtifactRetrievalServiceClient(cc) for _, a := range artifacts { - filename := makeFilename(dst, a.Name) + filename := makeFilename(dst, a.GetName()) if err := Retrieve(ctx, client, a, rt, dst); err != nil { - t.Errorf("failed to retrieve %v: %v", a.Name, err) + t.Errorf("failed to retrieve %v: %v", a.GetName(), err) continue } - verifySHA256(t, filename, a.Sha256) + verifySHA256(t, filename, a.GetSha256()) } } @@ -78,7 +78,7 @@ func TestMultiRetrieve(t *testing.T) { } for _, a := range artifacts { - verifySHA256(t, makeFilename(dst, a.Name), a.Sha256) + verifySHA256(t, makeFilename(dst, a.GetName()), a.GetSha256()) } } @@ -112,20 +112,18 @@ func stage(ctx context.Context, scl jobpb.LegacyArtifactStagingServiceClient, t sha256W.Write(data) hash := hex.EncodeToString(sha256W.Sum(nil)) md := makeArtifact(key, hash) - pmd := &jobpb.PutArtifactMetadata{ + pmd := jobpb.PutArtifactMetadata_builder{ Metadata: md, StagingSessionToken: st, - } + }.Build() stream, err := scl.PutArtifact(ctx) if err != nil { t.Fatalf("put failed: %v", err) } - header := &jobpb.PutArtifactRequest{ - Content: &jobpb.PutArtifactRequest_Metadata{ - Metadata: pmd, - }, - } + header := jobpb.PutArtifactRequest_builder{ + Metadata: proto.ValueOrDefault(pmd), + }.Build() if err := stream.Send(header); err != nil { t.Fatalf("send header failed: %v", err) } @@ -136,13 +134,11 @@ func stage(ctx context.Context, scl jobpb.LegacyArtifactStagingServiceClient, t end = size } - chunk := &jobpb.PutArtifactRequest{ - Content: &jobpb.PutArtifactRequest_Data{ - Data: &jobpb.ArtifactChunk{ - Data: data[i:end], - }, - }, - } + chunk := jobpb.PutArtifactRequest_builder{ + Data: jobpb.ArtifactChunk_builder{ + Data: data[i:end], + }.Build(), + }.Build() if err := stream.Send(chunk); err != nil { t.Fatalf("send chunk[%v:%v] failed: %v", i, end, err) } @@ -193,8 +189,8 @@ func TestNewRetrieveWithFileGeneratedStageName(t *testing.T) { generated := make(map[string]string) for _, md := range mds { name, _ := MustExtractFilePayload(md) - payload, _ := proto.Marshal(&pipepb.ArtifactStagingToRolePayload{ - StagedName: name}) + payload, _ := proto.Marshal(pipepb.ArtifactStagingToRolePayload_builder{ + StagedName: name}.Build()) generated[name] = string(payload) } @@ -220,8 +216,8 @@ func TestNewRetrieveWithUrlGeneratedStageName(t *testing.T) { generated := make(map[string]string) for _, md := range mds { name, _ := MustExtractFilePayload(md) - payload, _ := proto.Marshal(&pipepb.ArtifactStagingToRolePayload{ - StagedName: name}) + payload, _ := proto.Marshal(pipepb.ArtifactStagingToRolePayload_builder{ + StagedName: name}.Build()) generated[name] = string(payload) } @@ -298,14 +294,14 @@ type fakeRetrievalService struct { func (fake *fakeRetrievalService) resolvedArtifactsWithStagingTo() []*pipepb.ArtifactInformation { var artifacts []*pipepb.ArtifactInformation for name, contents := range fake.artifacts { - payload, _ := proto.Marshal(&pipepb.ArtifactStagingToRolePayload{ - StagedName: name}) - artifacts = append(artifacts, &pipepb.ArtifactInformation{ + payload, _ := proto.Marshal(pipepb.ArtifactStagingToRolePayload_builder{ + StagedName: name}.Build()) + artifacts = append(artifacts, pipepb.ArtifactInformation_builder{ TypeUrn: "resolved", TypePayload: []byte(contents), RoleUrn: URNStagingTo, RolePayload: payload, - }) + }.Build()) } return artifacts } @@ -313,12 +309,12 @@ func (fake *fakeRetrievalService) resolvedArtifactsWithStagingTo() []*pipepb.Art func (fake *fakeRetrievalService) fileArtifactsWithoutStagingTo() []*pipepb.ArtifactInformation { var artifacts []*pipepb.ArtifactInformation for name := range fake.artifacts { - payload, _ := proto.Marshal(&pipepb.ArtifactFilePayload{ - Path: filepath.Join("/tmp", name)}) - artifacts = append(artifacts, &pipepb.ArtifactInformation{ + payload, _ := proto.Marshal(pipepb.ArtifactFilePayload_builder{ + Path: filepath.Join("/tmp", name)}.Build()) + artifacts = append(artifacts, pipepb.ArtifactInformation_builder{ TypeUrn: URNFileArtifact, TypePayload: payload, - }) + }.Build()) } return artifacts } @@ -326,44 +322,44 @@ func (fake *fakeRetrievalService) fileArtifactsWithoutStagingTo() []*pipepb.Arti func (fake *fakeRetrievalService) urlArtifactsWithoutStagingTo() []*pipepb.ArtifactInformation { var artifacts []*pipepb.ArtifactInformation for name := range fake.artifacts { - payload, _ := proto.Marshal(&pipepb.ArtifactUrlPayload{ - Url: path.Join("gs://tmp", name)}) - artifacts = append(artifacts, &pipepb.ArtifactInformation{ + payload, _ := proto.Marshal(pipepb.ArtifactUrlPayload_builder{ + Url: path.Join("gs://tmp", name)}.Build()) + artifacts = append(artifacts, pipepb.ArtifactInformation_builder{ TypeUrn: URNUrlArtifact, TypePayload: payload, - }) + }.Build()) } return artifacts } func (fake *fakeRetrievalService) unresolvedArtifacts() []*pipepb.ArtifactInformation { return []*pipepb.ArtifactInformation{ - &pipepb.ArtifactInformation{ + pipepb.ArtifactInformation_builder{ TypeUrn: "unresolved", - }, + }.Build(), } } func (fake *fakeRetrievalService) ResolveArtifacts(ctx context.Context, request *jobpb.ResolveArtifactsRequest, opts ...grpc.CallOption) (*jobpb.ResolveArtifactsResponse, error) { - response := jobpb.ResolveArtifactsResponse{} - for _, dep := range request.Artifacts { - if dep.TypeUrn == "unresolved" { - response.Replacements = append(response.Replacements, fake.resolvedArtifactsWithStagingTo()...) + response := &jobpb.ResolveArtifactsResponse{} + for _, dep := range request.GetArtifacts() { + if dep.GetTypeUrn() == "unresolved" { + response.SetReplacements(append(response.GetReplacements(), fake.resolvedArtifactsWithStagingTo()...)) } else { - response.Replacements = append(response.Replacements, dep) + response.SetReplacements(append(response.GetReplacements(), dep)) } } - return &response, nil + return response, nil } func (fake *fakeRetrievalService) GetArtifact(ctx context.Context, request *jobpb.GetArtifactRequest, opts ...grpc.CallOption) (jobpb.ArtifactRetrievalService_GetArtifactClient, error) { - switch request.Artifact.TypeUrn { + switch request.GetArtifact().GetTypeUrn() { case "resolved": - return &fakeGetArtifactResponseStream{data: request.Artifact.TypePayload}, nil + return &fakeGetArtifactResponseStream{data: request.GetArtifact().GetTypePayload()}, nil case URNFileArtifact, URNUrlArtifact: - return &fakeGetArtifactResponseStream{data: request.Artifact.RolePayload}, nil + return &fakeGetArtifactResponseStream{data: request.GetArtifact().GetRolePayload()}, nil default: - return nil, errors.Errorf("Unsupported artifact %v", request.Artifact) + return nil, errors.Errorf("Unsupported artifact %v", request.GetArtifact()) } } @@ -375,7 +371,7 @@ type fakeGetArtifactResponseStream struct { func (fake *fakeGetArtifactResponseStream) Recv() (*jobpb.GetArtifactResponse, error) { if fake.index < len(fake.data) { fake.index++ - return &jobpb.GetArtifactResponse{Data: fake.data[fake.index-1 : fake.index]}, nil + return jobpb.GetArtifactResponse_builder{Data: fake.data[fake.index-1 : fake.index]}.Build(), nil } return nil, io.EOF } @@ -451,11 +447,11 @@ func makeTempFile(t *testing.T, filename string, size int) string { } func makeArtifact(key, hash string) *jobpb.ArtifactMetadata { - return &jobpb.ArtifactMetadata{ + return jobpb.ArtifactMetadata_builder{ Name: key, Sha256: hash, Permissions: 0644, - } + }.Build() } func makeFilename(dir, key string) string { diff --git a/sdks/go/pkg/beam/artifact/server_test.go b/sdks/go/pkg/beam/artifact/server_test.go index 3fdc8fa5cac8..f0ce3a59f05c 100644 --- a/sdks/go/pkg/beam/artifact/server_test.go +++ b/sdks/go/pkg/beam/artifact/server_test.go @@ -85,7 +85,7 @@ func (s *server) PutArtifact(ps jobpb.LegacyArtifactStagingService_PutArtifactSe if header.GetMetadata() == nil { return errors.Errorf("expected header as first message: %v", header) } - key := header.GetMetadata().GetMetadata().Name + key := header.GetMetadata().GetMetadata().GetName() if header.GetMetadata().GetStagingSessionToken() == "" { return errors.New("missing staging session token") } @@ -138,19 +138,19 @@ func (s *server) CommitManifest(ctx context.Context, req *jobpb.CommitManifestRe artifacts := req.GetManifest().GetArtifact() for _, md := range artifacts { - if _, ok := m.m[md.Name]; !ok { - return nil, errors.Errorf("artifact %v not staged", md.Name) + if _, ok := m.m[md.GetName()]; !ok { + return nil, errors.Errorf("artifact %v not staged", md.GetName()) } } // Update commit. Only one manifest can exist for each staging id. for _, md := range artifacts { - m.m[md.Name].md = md + m.m[md.GetName()].md = md } m.md = req.GetManifest() - return &jobpb.CommitManifestResponse{RetrievalToken: token}, nil + return jobpb.CommitManifestResponse_builder{RetrievalToken: token}.Build(), nil } func (s *server) GetManifest(ctx context.Context, req *jobpb.GetManifestRequest) (*jobpb.GetManifestResponse, error) { @@ -166,7 +166,7 @@ func (s *server) GetManifest(ctx context.Context, req *jobpb.GetManifestRequest) m.mu.Lock() defer m.mu.Unlock() - return &jobpb.GetManifestResponse{Manifest: m.md}, nil + return jobpb.GetManifestResponse_builder{Manifest: m.md}.Build(), nil } func (s *server) GetArtifact(req *jobpb.LegacyGetArtifactRequest, stream jobpb.LegacyArtifactRetrievalService_GetArtifactServer) error { @@ -195,7 +195,7 @@ func (s *server) GetArtifact(req *jobpb.LegacyGetArtifactRequest, stream jobpb.L // Send chunks exactly as we received them. for _, chunk := range chunks { - if err := stream.Send(&jobpb.ArtifactChunk{Data: chunk}); err != nil { + if err := stream.Send(jobpb.ArtifactChunk_builder{Data: chunk}.Build()); err != nil { return err } } diff --git a/sdks/go/pkg/beam/artifact/stage.go b/sdks/go/pkg/beam/artifact/stage.go index de017b4c3c07..d248380c8366 100644 --- a/sdks/go/pkg/beam/artifact/stage.go +++ b/sdks/go/pkg/beam/artifact/stage.go @@ -31,17 +31,18 @@ import ( "github.com/apache/beam/sdks/v2/go/pkg/beam/internal/errors" jobpb "github.com/apache/beam/sdks/v2/go/pkg/beam/model/jobmanagement_v1" "github.com/apache/beam/sdks/v2/go/pkg/beam/util/errorx" + "google.golang.org/protobuf/proto" ) // Commit commits a manifest with the given staged artifacts. It returns the // staging token, if successful. func Commit(ctx context.Context, client jobpb.LegacyArtifactStagingServiceClient, artifacts []*jobpb.ArtifactMetadata, st string) (string, error) { - req := &jobpb.CommitManifestRequest{ - Manifest: &jobpb.Manifest{ + req := jobpb.CommitManifestRequest_builder{ + Manifest: jobpb.Manifest_builder{ Artifact: artifacts, - }, + }.Build(), StagingSessionToken: st, - } + }.Build() resp, err := client.CommitManifest(ctx, req) if err != nil { return "", err @@ -127,15 +128,15 @@ func Stage(ctx context.Context, client jobpb.LegacyArtifactStagingServiceClient, if err != nil { return nil, err } - md := &jobpb.ArtifactMetadata{ + md := jobpb.ArtifactMetadata_builder{ Name: key, Permissions: uint32(stat.Mode()), Sha256: hash, - } - pmd := &jobpb.PutArtifactMetadata{ + }.Build() + pmd := jobpb.PutArtifactMetadata_builder{ Metadata: md, StagingSessionToken: st, - } + }.Build() fd, err := os.Open(filename) if err != nil { @@ -148,11 +149,9 @@ func Stage(ctx context.Context, client jobpb.LegacyArtifactStagingServiceClient, return nil, err } - header := &jobpb.PutArtifactRequest{ - Content: &jobpb.PutArtifactRequest_Metadata{ - Metadata: pmd, - }, - } + header := jobpb.PutArtifactRequest_builder{ + Metadata: proto.ValueOrDefault(pmd), + }.Build() if err := stream.Send(header); err != nil { stream.CloseAndRecv() // ignore error return nil, errors.Wrapf(err, "failed to send header for %v", filename) @@ -181,13 +180,11 @@ func stageChunks(stream jobpb.LegacyArtifactStagingService_PutArtifactClient, r panic(err) // cannot fail } - chunk := &jobpb.PutArtifactRequest{ - Content: &jobpb.PutArtifactRequest_Data{ - Data: &jobpb.ArtifactChunk{ - Data: data[:n], - }, - }, - } + chunk := jobpb.PutArtifactRequest_builder{ + Data: jobpb.ArtifactChunk_builder{ + Data: data[:n], + }.Build(), + }.Build() err := stream.Send(chunk) if err == io.EOF { return "", err diff --git a/sdks/go/pkg/beam/artifact/stage_test.go b/sdks/go/pkg/beam/artifact/stage_test.go index 2355f0a0176d..af2af73ad757 100644 --- a/sdks/go/pkg/beam/artifact/stage_test.go +++ b/sdks/go/pkg/beam/artifact/stage_test.go @@ -84,7 +84,7 @@ func validate(ctx context.Context, cc *grpc.ClientConn, t *testing.T, keys, sha2 rcl := jobpb.NewLegacyArtifactRetrievalServiceClient(cc) for i, key := range keys { - stream, err := rcl.GetArtifact(ctx, &jobpb.LegacyGetArtifactRequest{Name: key, RetrievalToken: rt}) + stream, err := rcl.GetArtifact(ctx, jobpb.LegacyGetArtifactRequest_builder{Name: key, RetrievalToken: rt}.Build()) if err != nil { t.Fatalf("failed to get artifact for %v: %v", key, err) } diff --git a/sdks/go/pkg/beam/core/runtime/exec/translate.go b/sdks/go/pkg/beam/core/runtime/exec/translate.go index b74ede228fd9..37b379784ced 100644 --- a/sdks/go/pkg/beam/core/runtime/exec/translate.go +++ b/sdks/go/pkg/beam/core/runtime/exec/translate.go @@ -349,7 +349,7 @@ func (b *builder) makeCoderForPCollection(id string) (*coder.Coder, *coder.Windo if !ok { return nil, nil, errors.Errorf("pcollection %v not found", id) } - c, err := b.coders.Coder(col.CoderId) + c, err := b.coders.Coder(col.GetCoderId()) if err != nil { return nil, nil, err } @@ -539,11 +539,11 @@ func (b *builder) makeLink(from string, id linkID) (Node, error) { var cID string var kcID string if rmw := spec.GetReadModifyWriteSpec(); rmw != nil { - cID = rmw.CoderId + cID = rmw.GetCoderId() } else if bs := spec.GetBagSpec(); bs != nil { - cID = bs.ElementCoderId + cID = bs.GetElementCoderId() } else if cs := spec.GetCombiningSpec(); cs != nil { - cID = cs.AccumulatorCoderId + cID = cs.GetAccumulatorCoderId() cmbData := string(cs.GetCombineFn().GetPayload()) var cmbTp v1pb.TransformPayload if err := protox.DecodeBase64(cmbData, &cmbTp); err != nil { @@ -559,10 +559,10 @@ func (b *builder) makeLink(from string, id linkID) (Node, error) { } stateIDToCombineFn[key] = cfn } else if ms := spec.GetMapSpec(); ms != nil { - cID = ms.ValueCoderId - kcID = ms.KeyCoderId + cID = ms.GetValueCoderId() + kcID = ms.GetKeyCoderId() } else if ss := spec.GetSetSpec(); ss != nil { - kcID = ss.ElementCoderId + kcID = ss.GetElementCoderId() } else { return nil, errors.Errorf("Unrecognized state type %v", spec) } @@ -801,10 +801,10 @@ func (b *builder) makeLink(from string, id linkID) (Node, error) { u = &MapWindows{UID: b.idgen.New(), Fn: mapper, Out: out[0], FnUrn: fn.GetUrn()} case graphx.URNFlatten: - u = &Flatten{UID: b.idgen.New(), N: len(transform.Inputs), Out: out[0]} + u = &Flatten{UID: b.idgen.New(), N: len(transform.GetInputs()), Out: out[0]} // Use the same flatten instance for all the inputs links to this transform. - for i := 0; i < len(transform.Inputs); i++ { + for i := 0; i < len(transform.GetInputs()); i++ { b.links[linkID{id.to, i}] = u } @@ -918,5 +918,5 @@ func unmarshalPort(data []byte) (Port, string, error) { } return Port{ URL: port.GetApiServiceDescriptor().GetUrl(), - }, port.CoderId, nil + }, port.GetCoderId(), nil } diff --git a/sdks/go/pkg/beam/core/runtime/exec/translate_test.go b/sdks/go/pkg/beam/core/runtime/exec/translate_test.go index a9917ec456fe..4203bc5c3962 100644 --- a/sdks/go/pkg/beam/core/runtime/exec/translate_test.go +++ b/sdks/go/pkg/beam/core/runtime/exec/translate_test.go @@ -17,12 +17,13 @@ package exec import ( "fmt" - fnpb "github.com/apache/beam/sdks/v2/go/pkg/beam/model/fnexecution_v1" "reflect" "strings" "testing" "time" + fnpb "github.com/apache/beam/sdks/v2/go/pkg/beam/model/fnexecution_v1" + "github.com/apache/beam/sdks/v2/go/pkg/beam/core/graph" "github.com/apache/beam/sdks/v2/go/pkg/beam/core/graph/coder" "github.com/apache/beam/sdks/v2/go/pkg/beam/core/graph/window" @@ -71,12 +72,12 @@ func TestUnmarshalKeyedValues(t *testing.T) { func TestUnmarshalReshuffleCoders(t *testing.T) { payloads := map[string][]byte{} encode := func(id, urn string, comps ...string) { - payloads[id] = protox.MustEncode(&pipepb.Coder{ - Spec: &pipepb.FunctionSpec{ + payloads[id] = protox.MustEncode(pipepb.Coder_builder{ + Spec: pipepb.FunctionSpec_builder{ Urn: urn, - }, + }.Build(), ComponentCoderIds: comps, - }) + }.Build()) } encode("a", "beam:coder:bytes:v1") encode("b", "beam:coder:string_utf8:v1") @@ -267,37 +268,37 @@ func TestUnmarshalWindowMapper(t *testing.T) { func makeWindowFn(w *window.Fn) (*pipepb.FunctionSpec, error) { switch w.Kind { case window.GlobalWindows: - return &pipepb.FunctionSpec{ + return pipepb.FunctionSpec_builder{ Urn: graphx.URNGlobalWindowsWindowFn, - }, nil + }.Build(), nil case window.FixedWindows: - return &pipepb.FunctionSpec{ + return pipepb.FunctionSpec_builder{ Urn: graphx.URNFixedWindowsWindowFn, Payload: protox.MustEncode( - &pipepb.FixedWindowsPayload{ + pipepb.FixedWindowsPayload_builder{ Size: durationpb.New(w.Size), - }, + }.Build(), ), - }, nil + }.Build(), nil case window.SlidingWindows: - return &pipepb.FunctionSpec{ + return pipepb.FunctionSpec_builder{ Urn: graphx.URNSlidingWindowsWindowFn, Payload: protox.MustEncode( - &pipepb.SlidingWindowsPayload{ + pipepb.SlidingWindowsPayload_builder{ Size: durationpb.New(w.Size), Period: durationpb.New(w.Period), - }, + }.Build(), ), - }, nil + }.Build(), nil case window.Sessions: - return &pipepb.FunctionSpec{ + return pipepb.FunctionSpec_builder{ Urn: graphx.URNSessionsWindowFn, Payload: protox.MustEncode( - &pipepb.SessionWindowsPayload{ + pipepb.SessionWindowsPayload_builder{ GapSize: durationpb.New(w.Gap), - }, + }.Build(), ), - }, nil + }.Build(), nil default: return nil, errors.Errorf("unexpected windowing strategy: %v", w) } @@ -310,11 +311,11 @@ func makeWindowMappingFn(w *window.Fn) (*pipepb.FunctionSpec, error) { } switch w.Kind { case window.GlobalWindows: - wFn.Urn = graphx.URNWindowMappingGlobal + wFn.SetUrn(graphx.URNWindowMappingGlobal) case window.FixedWindows: - wFn.Urn = graphx.URNWindowMappingFixed + wFn.SetUrn(graphx.URNWindowMappingFixed) case window.SlidingWindows: - wFn.Urn = graphx.URNWindowMappingSliding + wFn.SetUrn(graphx.URNWindowMappingSliding) default: return nil, fmt.Errorf("unknown window fn type %v", w.Kind) } @@ -393,7 +394,7 @@ func TestUnmarshalPort(t *testing.T) { { inputData: []byte{}, outputPort: Port{URL: port.GetApiServiceDescriptor().GetUrl()}, - outputStr: fnpb.RemoteGrpcPort{}.CoderId, + outputStr: (&fnpb.RemoteGrpcPort{}).GetCoderId(), outputError: nil, }, } @@ -413,12 +414,12 @@ func TestUnmarshalPort(t *testing.T) { } func TestUnmarshalPlan(t *testing.T) { - transform := pipepb.PTransform{ - Spec: &pipepb.FunctionSpec{ + transform := pipepb.PTransform_builder{ + Spec: pipepb.FunctionSpec_builder{ Urn: urnDataSource, - }, + }.Build(), Outputs: map[string]string{}, - } + }.Build() tests := []struct { name string inputDesc *fnpb.ProcessBundleDescriptor @@ -427,32 +428,32 @@ func TestUnmarshalPlan(t *testing.T) { }{ { name: "test_no_root_units", - inputDesc: &fnpb.ProcessBundleDescriptor{ + inputDesc: fnpb.ProcessBundleDescriptor_builder{ Id: "", Transforms: map[string]*pipepb.PTransform{}, - }, + }.Build(), outputPlan: nil, outputError: errors.New("no root units"), }, { name: "test_zero_transform", - inputDesc: &fnpb.ProcessBundleDescriptor{ + inputDesc: fnpb.ProcessBundleDescriptor_builder{ Id: "", Transforms: map[string]*pipepb.PTransform{ "": {}, }, - }, + }.Build(), outputPlan: nil, outputError: errors.New("no root units"), }, { name: "test_transform_outputs_length_not_one", - inputDesc: &fnpb.ProcessBundleDescriptor{ + inputDesc: fnpb.ProcessBundleDescriptor_builder{ Id: "", Transforms: map[string]*pipepb.PTransform{ - "": &transform, + "": transform, }, - }, + }.Build(), outputPlan: nil, outputError: errors.Errorf("expected one output from DataSource, got %v", transform.GetOutputs()), }, @@ -473,10 +474,10 @@ func TestUnmarshalPlan(t *testing.T) { } func TestNewBuilder(t *testing.T) { - descriptor := fnpb.ProcessBundleDescriptor{ + descriptor := fnpb.ProcessBundleDescriptor_builder{ Id: "", Transforms: map[string]*pipepb.PTransform{}, - } + }.Build() tests := []struct { name string inputDesc *fnpb.ProcessBundleDescriptor @@ -485,9 +486,9 @@ func TestNewBuilder(t *testing.T) { }{ { name: "test_1", - inputDesc: &descriptor, + inputDesc: descriptor, outputBuilder: &builder{ - desc: &descriptor, + desc: descriptor, coders: graphx.NewCoderUnmarshaller(descriptor.GetCoders()), prev: make(map[string]int), succ: make(map[string][]linkID), diff --git a/sdks/go/pkg/beam/core/runtime/graphx/coder.go b/sdks/go/pkg/beam/core/runtime/graphx/coder.go index 99ca5517d3d3..4ea77dee183d 100644 --- a/sdks/go/pkg/beam/core/runtime/graphx/coder.go +++ b/sdks/go/pkg/beam/core/runtime/graphx/coder.go @@ -454,12 +454,12 @@ func (b *CoderMarshaller) Add(c *coder.Coder) (string, error) { if err != nil { return "", errors.Wrapf(err, "failed to marshal custom coder %v", c) } - inner := b.internCoder(&pipepb.Coder{ - Spec: &pipepb.FunctionSpec{ + inner := b.internCoder(pipepb.Coder_builder{ + Spec: pipepb.FunctionSpec_builder{ Urn: urnCustomCoder, Payload: []byte(data), - }, - }) + }.Build(), + }.Build()) return b.internBuiltInCoder(urnLengthPrefixCoder, inner), nil case coder.KV: @@ -596,22 +596,22 @@ func (b *CoderMarshaller) Build() map[string]*pipepb.Coder { } func (b *CoderMarshaller) internBuiltInCoder(urn string, components ...string) string { - return b.internCoder(&pipepb.Coder{ - Spec: &pipepb.FunctionSpec{ + return b.internCoder(pipepb.Coder_builder{ + Spec: pipepb.FunctionSpec_builder{ Urn: urn, - }, + }.Build(), ComponentCoderIds: components, - }) + }.Build()) } func (b *CoderMarshaller) internRowCoder(schema *pipepb.Schema) string { payload := protox.MustEncode(schema) - return b.internCoder(&pipepb.Coder{ - Spec: &pipepb.FunctionSpec{ + return b.internCoder(pipepb.Coder_builder{ + Spec: pipepb.FunctionSpec_builder{ Urn: urnRowCoder, Payload: payload, - }, - }) + }.Build(), + }.Build()) } func (b *CoderMarshaller) internCoder(coder *pipepb.Coder) string { diff --git a/sdks/go/pkg/beam/core/runtime/graphx/schema/schema.go b/sdks/go/pkg/beam/core/runtime/graphx/schema/schema.go index 0d44e68285b5..2a23e3642280 100644 --- a/sdks/go/pkg/beam/core/runtime/graphx/schema/schema.go +++ b/sdks/go/pkg/beam/core/runtime/graphx/schema/schema.go @@ -375,12 +375,12 @@ func (r *Registry) fromType(ot reflect.Type) (*pipepb.Schema, error) { schm := ftype.GetRowType().GetSchema() schm = proto.Clone(schm).(*pipepb.Schema) if ot.Kind() == reflect.Ptr { - schm.Options = append(schm.Options, optGoNillable()) + schm.SetOptions(append(schm.GetOptions(), optGoNillable())) } if lID != "" { - schm.Options = append(schm.Options, logicalOption(lID)) + schm.SetOptions(append(schm.GetOptions(), logicalOption(lID))) } - schm.Id = getUUID(ot) + schm.SetId(getUUID(ot)) r.typeToSchema[ot] = schm r.idToType[schm.GetId()] = ot return schm, nil @@ -395,8 +395,8 @@ func (r *Registry) fromType(ot reflect.Type) (*pipepb.Schema, error) { // Cache the pointer type here with it's own id. pt := reflect.PtrTo(t) schm = proto.Clone(schm).(*pipepb.Schema) - schm.Id = getUUID(pt) - schm.Options = append(schm.Options, optGoNillable()) + schm.SetId(getUUID(pt)) + schm.SetOptions(append(schm.GetOptions(), optGoNillable())) r.idToType[schm.GetId()] = pt r.typeToSchema[pt] = schm @@ -428,23 +428,17 @@ func optGoEmbedded() *pipepb.Option { // is not set if the toggle isn't true, so the value is always // true. func newToggleOption(urn string) *pipepb.Option { - return &pipepb.Option{ + return pipepb.Option_builder{ Name: urn, - Type: &pipepb.FieldType{ - TypeInfo: &pipepb.FieldType_AtomicType{ - AtomicType: pipepb.AtomicType_BOOLEAN, - }, - }, - Value: &pipepb.FieldValue{ - FieldValue: &pipepb.FieldValue_AtomicValue{ - AtomicValue: &pipepb.AtomicTypeValue{ - Value: &pipepb.AtomicTypeValue_Boolean{ - Boolean: true, - }, - }, - }, - }, - } + Type: pipepb.FieldType_builder{ + AtomicType: pipepb.AtomicType_BOOLEAN.Enum(), + }.Build(), + Value: pipepb.FieldValue_builder{ + AtomicValue: pipepb.AtomicTypeValue_builder{ + Boolean: proto.Bool(true), + }.Build(), + }.Build(), + }.Build() } func checkOptions(opts []*pipepb.Option, urn string) *pipepb.Option { @@ -465,26 +459,20 @@ func nillableFromOptions(opts []*pipepb.Option, t reflect.Type) reflect.Type { return nil } -var optGoLogicalType = &pipepb.FieldType{ - TypeInfo: &pipepb.FieldType_AtomicType{ - AtomicType: pipepb.AtomicType_STRING, - }, -} +var optGoLogicalType = pipepb.FieldType_builder{ + AtomicType: pipepb.AtomicType_STRING.Enum(), +}.Build() func logicalOption(lID string) *pipepb.Option { - return &pipepb.Option{ + return pipepb.Option_builder{ Name: optGoLogicalUrn, Type: optGoLogicalType, - Value: &pipepb.FieldValue{ - FieldValue: &pipepb.FieldValue_AtomicValue{ - AtomicValue: &pipepb.AtomicTypeValue{ - Value: &pipepb.AtomicTypeValue_String_{ - String_: lID, - }, - }, - }, - }, - } + Value: pipepb.FieldValue_builder{ + AtomicValue: pipepb.AtomicTypeValue_builder{ + String: proto.String(lID), + }.Build(), + }.Build(), + }.Build() } // fromLogicalOption returns the logical type id of this top @@ -513,8 +501,8 @@ func (r *Registry) structToSchema(t reflect.Type) (*pipepb.Schema, error) { if ftype != nil { schm := ftype.GetRowType().GetSchema() schm = proto.Clone(schm).(*pipepb.Schema) - schm.Options = append(schm.Options, logicalOption(lID)) - schm.Id = getUUID(t) + schm.SetOptions(append(schm.GetOptions(), logicalOption(lID))) + schm.SetId(getUUID(t)) r.typeToSchema[t] = schm r.idToType[schm.GetId()] = t return schm, nil @@ -536,15 +524,15 @@ func (r *Registry) structToSchema(t reflect.Type) (*pipepb.Schema, error) { } if isAnon { f = proto.Clone(f).(*pipepb.Field) - f.Options = append(f.Options, optGoEmbedded()) + f.SetOptions(append(f.GetOptions(), optGoEmbedded())) } fields = append(fields, f) } - schm := &pipepb.Schema{ + schm := pipepb.Schema_builder{ Fields: fields, Id: getUUID(t), - } + }.Build() r.idToType[schm.GetId()] = t r.typeToSchema[t] = schm return schm, nil @@ -559,10 +547,10 @@ func (r *Registry) structFieldToField(sf reflect.StructField) (*pipepb.Field, er if err != nil { return nil, err } - return &pipepb.Field{ + return pipepb.Field_builder{ Name: name, Type: ftype, - }, nil + }.Build(), nil } func (r *Registry) reflectTypeToFieldType(ot reflect.Type) (*pipepb.FieldType, error) { @@ -571,15 +559,13 @@ func (r *Registry) reflectTypeToFieldType(ot reflect.Type) (*pipepb.FieldType, e return nil, err } if ftype != nil { - return &pipepb.FieldType{ - TypeInfo: &pipepb.FieldType_LogicalType{ - LogicalType: &pipepb.LogicalType{ - Urn: lID, - Representation: ftype, - // TODO(BEAM-9615): Handle type Arguments. - }, - }, - }, nil + return pipepb.FieldType_builder{ + LogicalType: pipepb.LogicalType_builder{ + Urn: lID, + Representation: ftype, + // TODO(BEAM-9615): Handle type Arguments. + }.Build(), + }.Build(), nil } t := ot @@ -589,7 +575,7 @@ func (r *Registry) reflectTypeToFieldType(ot reflect.Type) (*pipepb.FieldType, e if err != nil { return nil, errors.Wrapf(err, "unable to convert key of %v to schema field", ot) } - vt.Nullable = true + vt.SetNullable(true) return vt, nil case reflect.Map: kt, err := r.reflectTypeToFieldType(t.Key()) @@ -600,55 +586,45 @@ func (r *Registry) reflectTypeToFieldType(ot reflect.Type) (*pipepb.FieldType, e if err != nil { return nil, errors.Wrapf(err, "unable to convert value of %v to schema field", ot) } - return &pipepb.FieldType{ - TypeInfo: &pipepb.FieldType_MapType{ - MapType: &pipepb.MapType{ - KeyType: kt, - ValueType: vt, - }, - }, - }, nil + return pipepb.FieldType_builder{ + MapType: pipepb.MapType_builder{ + KeyType: kt, + ValueType: vt, + }.Build(), + }.Build(), nil case reflect.Struct: sch, err := r.structToSchema(t) if err != nil { return nil, errors.Wrapf(err, "unable to convert %v to schema field", ot) } - return &pipepb.FieldType{ - TypeInfo: &pipepb.FieldType_RowType{ - RowType: &pipepb.RowType{ - Schema: sch, - }, - }, - }, nil + return pipepb.FieldType_builder{ + RowType: pipepb.RowType_builder{ + Schema: sch, + }.Build(), + }.Build(), nil case reflect.Slice, reflect.Array: // Special handling for []byte if t == reflectx.ByteSlice { - return &pipepb.FieldType{ - TypeInfo: &pipepb.FieldType_AtomicType{ - AtomicType: pipepb.AtomicType_BYTES, - }, - }, nil + return pipepb.FieldType_builder{ + AtomicType: pipepb.AtomicType_BYTES.Enum(), + }.Build(), nil } vt, err := r.reflectTypeToFieldType(t.Elem()) if err != nil { return nil, errors.Wrapf(err, "unable to convert element type of %v to schema field", ot) } - return &pipepb.FieldType{ - TypeInfo: &pipepb.FieldType_ArrayType{ - ArrayType: &pipepb.ArrayType{ - ElementType: vt, - }, - }, - }, nil + return pipepb.FieldType_builder{ + ArrayType: pipepb.ArrayType_builder{ + ElementType: vt, + }.Build(), + }.Build(), nil case reflect.Interface, reflect.Func, reflect.Chan, reflect.UnsafePointer, reflect.Complex128, reflect.Complex64, reflect.Invalid: return nil, errors.Errorf("unable to convert unsupported type %v to schema", ot) default: // must be an atomic type if enum, ok := reflectTypeToAtomicTypeMap[t.Kind()]; ok { - return &pipepb.FieldType{ - TypeInfo: &pipepb.FieldType_AtomicType{ - AtomicType: enum, - }, - }, nil + return pipepb.FieldType_builder{ + AtomicType: enum.Enum(), + }.Build(), nil } return nil, errors.Errorf("unable to map %v to pipepb.AtomicType", t) } @@ -713,7 +689,7 @@ func (r *Registry) toType(s *pipepb.Schema) (reflect.Type, error) { if err != nil { return nil, errors.Wrapf(err, "cannot convert schema field %v to field", sf.GetName()) } - if checkOptions(sf.Options, optGoEmbeddedUrn) != nil { + if checkOptions(sf.GetOptions(), optGoEmbeddedUrn) != nil { rf.Anonymous = true } fields = append(fields, rf) @@ -730,7 +706,7 @@ func (r *Registry) toType(s *pipepb.Schema) (reflect.Type, error) { func (r *Registry) fieldToStructField(sf *pipepb.Field) (reflect.StructField, error) { name := sf.GetName() - rt, err := r.fieldTypeToReflectType(sf.GetType(), sf.Options) + rt, err := r.fieldTypeToReflectType(sf.GetType(), sf.GetOptions()) if err != nil { return reflect.StructField{}, err } @@ -760,19 +736,19 @@ var atomicTypeToReflectType = map[pipepb.AtomicType]reflect.Type{ func (r *Registry) fieldTypeToReflectType(sft *pipepb.FieldType, opts []*pipepb.Option) (reflect.Type, error) { var t reflect.Type - switch sft.GetTypeInfo().(type) { - case *pipepb.FieldType_AtomicType: + switch sft.WhichTypeInfo() { + case pipepb.FieldType_AtomicType_case: var ok bool if t, ok = atomicTypeToReflectType[sft.GetAtomicType()]; !ok { return nil, errors.Errorf("unknown atomic type: %v", sft.GetAtomicType()) } - case *pipepb.FieldType_ArrayType: + case pipepb.FieldType_ArrayType_case: rt, err := r.fieldTypeToReflectType(sft.GetArrayType().GetElementType(), nil) if err != nil { return nil, errors.Wrap(err, "unable to convert array element type") } t = reflect.SliceOf(rt) - case *pipepb.FieldType_MapType: + case pipepb.FieldType_MapType_case: kt, err := r.fieldTypeToReflectType(sft.GetMapType().GetKeyType(), nil) if err != nil { return nil, errors.Wrap(err, "unable to convert map key type") @@ -782,7 +758,7 @@ func (r *Registry) fieldTypeToReflectType(sft *pipepb.FieldType, opts []*pipepb. return nil, errors.Wrap(err, "unable to convert map value type") } t = reflect.MapOf(kt, vt) // Panics for invalid map keys (slices/iterables) - case *pipepb.FieldType_RowType: + case pipepb.FieldType_RowType_case: rt, err := r.toType(sft.GetRowType().GetSchema()) if err != nil { return nil, errors.Wrapf(err, "unable to convert row type: %v", sft.GetRowType().GetSchema().GetId()) @@ -791,7 +767,7 @@ func (r *Registry) fieldTypeToReflectType(sft *pipepb.FieldType, opts []*pipepb. // case *pipepb.FieldType_IterableType: // TODO(BEAM-9615): handle IterableTypes (eg. CoGBK values) - case *pipepb.FieldType_LogicalType: + case pipepb.FieldType_LogicalType_case: lst := sft.GetLogicalType() identifier := lst.GetUrn() lt, ok := r.logicalTypes[identifier] diff --git a/sdks/go/pkg/beam/core/runtime/graphx/schema/schema_test.go b/sdks/go/pkg/beam/core/runtime/graphx/schema/schema_test.go index 367d70e81d17..ebe3d3898aba 100644 --- a/sdks/go/pkg/beam/core/runtime/graphx/schema/schema_test.go +++ b/sdks/go/pkg/beam/core/runtime/graphx/schema/schema_test.go @@ -107,650 +107,532 @@ func TestSchemaConversion(t *testing.T) { rt reflect.Type }{ { - st: &pipepb.Schema{ + st: pipepb.Schema_builder{ Fields: []*pipepb.Field{ - &pipepb.Field{ + pipepb.Field_builder{ Name: "firstField", - Type: &pipepb.FieldType{ - TypeInfo: &pipepb.FieldType_AtomicType{ - AtomicType: pipepb.AtomicType_INT32, - }, - }, - }, + Type: pipepb.FieldType_builder{ + AtomicType: pipepb.AtomicType_INT32.Enum(), + }.Build(), + }.Build(), }, - }, + }.Build(), rt: reflect.TypeOf(struct { FirstField int32 `beam:"firstField"` }{}), }, { - st: &pipepb.Schema{ + st: pipepb.Schema_builder{ Fields: []*pipepb.Field{ - &pipepb.Field{ + pipepb.Field_builder{ Name: "stringField", - Type: &pipepb.FieldType{ - TypeInfo: &pipepb.FieldType_AtomicType{ - AtomicType: pipepb.AtomicType_STRING, - }, - }, - }, - &pipepb.Field{ + Type: pipepb.FieldType_builder{ + AtomicType: pipepb.AtomicType_STRING.Enum(), + }.Build(), + }.Build(), + pipepb.Field_builder{ Name: "intPtrField", - Type: &pipepb.FieldType{ - Nullable: true, - TypeInfo: &pipepb.FieldType_AtomicType{ - AtomicType: pipepb.AtomicType_INT32, - }, - }, - }, + Type: pipepb.FieldType_builder{ + Nullable: true, + AtomicType: pipepb.AtomicType_INT32.Enum(), + }.Build(), + }.Build(), }, - }, + }.Build(), rt: reflect.TypeOf(struct { StringField string `beam:"stringField"` IntPtrField *int32 `beam:"intPtrField"` }{}), }, { - st: &pipepb.Schema{ + st: pipepb.Schema_builder{ Fields: []*pipepb.Field{ - &pipepb.Field{ + pipepb.Field_builder{ Name: "cypher", - Type: &pipepb.FieldType{ - TypeInfo: &pipepb.FieldType_MapType{ - MapType: &pipepb.MapType{ - KeyType: &pipepb.FieldType{ - TypeInfo: &pipepb.FieldType_AtomicType{ - AtomicType: pipepb.AtomicType_BOOLEAN, - }, - }, - ValueType: &pipepb.FieldType{ - TypeInfo: &pipepb.FieldType_AtomicType{ - AtomicType: pipepb.AtomicType_FLOAT, - }, - }, - }, - }, - }, - }, + Type: pipepb.FieldType_builder{ + MapType: pipepb.MapType_builder{ + KeyType: pipepb.FieldType_builder{ + AtomicType: pipepb.AtomicType_BOOLEAN.Enum(), + }.Build(), + ValueType: pipepb.FieldType_builder{ + AtomicType: pipepb.AtomicType_FLOAT.Enum(), + }.Build(), + }.Build(), + }.Build(), + }.Build(), }, - }, + }.Build(), rt: hasMapType, }, { - st: &pipepb.Schema{ + st: pipepb.Schema_builder{ Fields: []*pipepb.Field{ - &pipepb.Field{ + pipepb.Field_builder{ Name: "wrapper", - Type: &pipepb.FieldType{ - TypeInfo: &pipepb.FieldType_RowType{ - RowType: &pipepb.RowType{ - Schema: &pipepb.Schema{ - Fields: []*pipepb.Field{{ - Name: "threshold", - Type: &pipepb.FieldType{ - TypeInfo: &pipepb.FieldType_AtomicType{ - AtomicType: pipepb.AtomicType_DOUBLE, - }, - }, - }, - }, + Type: pipepb.FieldType_builder{ + RowType: pipepb.RowType_builder{ + Schema: pipepb.Schema_builder{ + Fields: []*pipepb.Field{pipepb.Field_builder{ + Name: "threshold", + Type: pipepb.FieldType_builder{ + AtomicType: pipepb.AtomicType_DOUBLE.Enum(), + }.Build(), + }.Build(), }, - }, - }, - }, - }, + }.Build(), + }.Build(), + }.Build(), + }.Build(), }, - }, + }.Build(), rt: reflect.TypeOf(struct { Wrapper struct { Threshold float64 `beam:"threshold"` } `beam:"wrapper"` }{}), }, { - st: &pipepb.Schema{ + st: pipepb.Schema_builder{ Fields: []*pipepb.Field{ - &pipepb.Field{ + pipepb.Field_builder{ Name: "payloads", - Type: &pipepb.FieldType{ - TypeInfo: &pipepb.FieldType_ArrayType{ - ArrayType: &pipepb.ArrayType{ - ElementType: &pipepb.FieldType{ - TypeInfo: &pipepb.FieldType_AtomicType{ - AtomicType: pipepb.AtomicType_BYTES, - }, - }, - }, - }, - }, - }, + Type: pipepb.FieldType_builder{ + ArrayType: pipepb.ArrayType_builder{ + ElementType: pipepb.FieldType_builder{ + AtomicType: pipepb.AtomicType_BYTES.Enum(), + }.Build(), + }.Build(), + }.Build(), + }.Build(), }, - }, + }.Build(), rt: reflect.TypeOf(struct { Payloads [][]byte `beam:"payloads"` }{}), }, { - st: &pipepb.Schema{ + st: pipepb.Schema_builder{ Fields: []*pipepb.Field{ - &pipepb.Field{ + pipepb.Field_builder{ Name: "AString", - Type: &pipepb.FieldType{ - TypeInfo: &pipepb.FieldType_AtomicType{ - AtomicType: pipepb.AtomicType_STRING, - }, - }, - }, - &pipepb.Field{ + Type: pipepb.FieldType_builder{ + AtomicType: pipepb.AtomicType_STRING.Enum(), + }.Build(), + }.Build(), + pipepb.Field_builder{ Name: "AnIntPtr", - Type: &pipepb.FieldType{ - Nullable: true, - TypeInfo: &pipepb.FieldType_AtomicType{ - AtomicType: pipepb.AtomicType_INT32, - }, - }, - }, + Type: pipepb.FieldType_builder{ + Nullable: true, + AtomicType: pipepb.AtomicType_INT32.Enum(), + }.Build(), + }.Build(), }, - }, + }.Build(), rt: reflect.TypeOf(struct { AString string AnIntPtr *int32 }{}), }, { - st: &pipepb.Schema{ + st: pipepb.Schema_builder{ Fields: []*pipepb.Field{ - &pipepb.Field{ + pipepb.Field_builder{ Name: "A", - Type: &pipepb.FieldType{ - TypeInfo: &pipepb.FieldType_AtomicType{ - AtomicType: pipepb.AtomicType_STRING, - }, - }, - }, - &pipepb.Field{ + Type: pipepb.FieldType_builder{ + AtomicType: pipepb.AtomicType_STRING.Enum(), + }.Build(), + }.Build(), + pipepb.Field_builder{ Name: "B", - Type: &pipepb.FieldType{ - TypeInfo: &pipepb.FieldType_AtomicType{ - AtomicType: pipepb.AtomicType_STRING, - }, - }, - }, - &pipepb.Field{ + Type: pipepb.FieldType_builder{ + AtomicType: pipepb.AtomicType_STRING.Enum(), + }.Build(), + }.Build(), + pipepb.Field_builder{ Name: "C", - Type: &pipepb.FieldType{ - TypeInfo: &pipepb.FieldType_AtomicType{ - AtomicType: pipepb.AtomicType_BOOLEAN, - }, - }, - }, + Type: pipepb.FieldType_builder{ + AtomicType: pipepb.AtomicType_BOOLEAN.Enum(), + }.Build(), + }.Build(), }, - }, + }.Build(), rt: reflect.TypeOf(registeredType{}), }, { - st: &pipepb.Schema{ + st: pipepb.Schema_builder{ Fields: []*pipepb.Field{ - &pipepb.Field{ + pipepb.Field_builder{ Name: "D", - Type: &pipepb.FieldType{ - TypeInfo: &pipepb.FieldType_AtomicType{ - AtomicType: pipepb.AtomicType_INT32, - }, - }, - }, + Type: pipepb.FieldType_builder{ + AtomicType: pipepb.AtomicType_INT32.Enum(), + }.Build(), + }.Build(), }, - }, + }.Build(), rt: reflect.TypeOf(sRegisteredType{}), }, { - st: &pipepb.Schema{ + st: pipepb.Schema_builder{ Fields: []*pipepb.Field{ - &pipepb.Field{ + pipepb.Field_builder{ Name: "A", - Type: &pipepb.FieldType{ - TypeInfo: &pipepb.FieldType_AtomicType{ - AtomicType: pipepb.AtomicType_STRING, - }, - }, - }, - &pipepb.Field{ + Type: pipepb.FieldType_builder{ + AtomicType: pipepb.AtomicType_STRING.Enum(), + }.Build(), + }.Build(), + pipepb.Field_builder{ Name: "B", - Type: &pipepb.FieldType{ - TypeInfo: &pipepb.FieldType_AtomicType{ - AtomicType: pipepb.AtomicType_STRING, - }, - }, - }, - &pipepb.Field{ + Type: pipepb.FieldType_builder{ + AtomicType: pipepb.AtomicType_STRING.Enum(), + }.Build(), + }.Build(), + pipepb.Field_builder{ Name: "C", - Type: &pipepb.FieldType{ - TypeInfo: &pipepb.FieldType_LogicalType{ - LogicalType: &pipepb.LogicalType{ - Urn: "int", - Representation: &pipepb.FieldType{ - TypeInfo: &pipepb.FieldType_AtomicType{ - AtomicType: pipepb.AtomicType_INT64, - }, - }, - }, - }, - }, - }, + Type: pipepb.FieldType_builder{ + LogicalType: pipepb.LogicalType_builder{ + Urn: "int", + Representation: pipepb.FieldType_builder{ + AtomicType: pipepb.AtomicType_INT64.Enum(), + }.Build(), + }.Build(), + }.Build(), + }.Build(), }, - }, + }.Build(), rt: reflect.TypeOf(justAType{}), }, { - st: &pipepb.Schema{ + st: pipepb.Schema_builder{ Fields: []*pipepb.Field{ - { + pipepb.Field_builder{ Name: "Q", - Type: &pipepb.FieldType{ - TypeInfo: &pipepb.FieldType_MapType{ - MapType: &pipepb.MapType{ - KeyType: &pipepb.FieldType{ - TypeInfo: &pipepb.FieldType_LogicalType{ - LogicalType: &pipepb.LogicalType{ - Urn: "int", - Representation: &pipepb.FieldType{ - TypeInfo: &pipepb.FieldType_AtomicType{ - AtomicType: pipepb.AtomicType_INT64, - }, - }, - }, - }, - }, - ValueType: &pipepb.FieldType{ - TypeInfo: &pipepb.FieldType_LogicalType{ - LogicalType: &pipepb.LogicalType{ - Urn: "int", - Representation: &pipepb.FieldType{ - TypeInfo: &pipepb.FieldType_AtomicType{ - AtomicType: pipepb.AtomicType_INT64, - }, - }, - }, - }, - }, - }, - }, - }, - }, { + Type: pipepb.FieldType_builder{ + MapType: pipepb.MapType_builder{ + KeyType: pipepb.FieldType_builder{ + LogicalType: pipepb.LogicalType_builder{ + Urn: "int", + Representation: pipepb.FieldType_builder{ + AtomicType: pipepb.AtomicType_INT64.Enum(), + }.Build(), + }.Build(), + }.Build(), + ValueType: pipepb.FieldType_builder{ + LogicalType: pipepb.LogicalType_builder{ + Urn: "int", + Representation: pipepb.FieldType_builder{ + AtomicType: pipepb.AtomicType_INT64.Enum(), + }.Build(), + }.Build(), + }.Build(), + }.Build(), + }.Build(), + }.Build(), pipepb.Field_builder{ Name: "T", - Type: &pipepb.FieldType{ - TypeInfo: &pipepb.FieldType_ArrayType{ - ArrayType: &pipepb.ArrayType{ - ElementType: &pipepb.FieldType{ - TypeInfo: &pipepb.FieldType_LogicalType{ - LogicalType: &pipepb.LogicalType{ - Urn: "int", - Representation: &pipepb.FieldType{ - TypeInfo: &pipepb.FieldType_AtomicType{ - AtomicType: pipepb.AtomicType_INT64, - }, - }, - }, - }, - }, - }, - }, - }, - }, + Type: pipepb.FieldType_builder{ + ArrayType: pipepb.ArrayType_builder{ + ElementType: pipepb.FieldType_builder{ + LogicalType: pipepb.LogicalType_builder{ + Urn: "int", + Representation: pipepb.FieldType_builder{ + AtomicType: pipepb.AtomicType_INT64.Enum(), + }.Build(), + }.Build(), + }.Build(), + }.Build(), + }.Build(), + }.Build(), }, - }, + }.Build(), rt: reflect.TypeOf(struct { Q map[int]int T []int }{}), }, { - st: &pipepb.Schema{ + st: pipepb.Schema_builder{ Fields: []*pipepb.Field{ - { + pipepb.Field_builder{ Name: "SuperNES", - Type: &pipepb.FieldType{ - TypeInfo: &pipepb.FieldType_AtomicType{ - AtomicType: pipepb.AtomicType_INT16, - }, - }, - }, + Type: pipepb.FieldType_builder{ + AtomicType: pipepb.AtomicType_INT16.Enum(), + }.Build(), + }.Build(), }, Options: []*pipepb.Option{optGoNillable()}, - }, + }.Build(), rt: reflect.TypeOf(&struct { SuperNES int16 }{}), }, { - st: &pipepb.Schema{ + st: pipepb.Schema_builder{ Options: []*pipepb.Option{ logicalOption("schema.unexportedFields"), }, Fields: []*pipepb.Field{ - { + pipepb.Field_builder{ Name: "D", - Type: &pipepb.FieldType{ - TypeInfo: &pipepb.FieldType_LogicalType{ - LogicalType: &pipepb.LogicalType{ - Urn: "uint64", - Representation: &pipepb.FieldType{ - TypeInfo: &pipepb.FieldType_AtomicType{ - AtomicType: pipepb.AtomicType_INT64, - }, - }, - }, - }, - }, - }, + Type: pipepb.FieldType_builder{ + LogicalType: pipepb.LogicalType_builder{ + Urn: "uint64", + Representation: pipepb.FieldType_builder{ + AtomicType: pipepb.AtomicType_INT64.Enum(), + }.Build(), + }.Build(), + }.Build(), + }.Build(), }, - }, + }.Build(), rt: unexportedFieldsType, }, { - st: &pipepb.Schema{ + st: pipepb.Schema_builder{ Fields: []*pipepb.Field{ - { + pipepb.Field_builder{ Name: "G", - Type: &pipepb.FieldType{ - TypeInfo: &pipepb.FieldType_LogicalType{ - LogicalType: &pipepb.LogicalType{ - Urn: "schema.unexportedFields", - Representation: &pipepb.FieldType{ - TypeInfo: &pipepb.FieldType_RowType{ - RowType: &pipepb.RowType{ - Schema: &pipepb.Schema{ - Fields: []*pipepb.Field{ - { - Name: "D", - Type: &pipepb.FieldType{ - TypeInfo: &pipepb.FieldType_LogicalType{ - LogicalType: &pipepb.LogicalType{ - Urn: "uint64", - Representation: &pipepb.FieldType{ - TypeInfo: &pipepb.FieldType_AtomicType{ - AtomicType: pipepb.AtomicType_INT64, - }, - }, - }, - }, - }, - }, - }, - }, + Type: pipepb.FieldType_builder{ + LogicalType: pipepb.LogicalType_builder{ + Urn: "schema.unexportedFields", + Representation: pipepb.FieldType_builder{ + RowType: pipepb.RowType_builder{ + Schema: pipepb.Schema_builder{ + Fields: []*pipepb.Field{ + pipepb.Field_builder{ + Name: "D", + Type: pipepb.FieldType_builder{ + LogicalType: pipepb.LogicalType_builder{ + Urn: "uint64", + Representation: pipepb.FieldType_builder{ + AtomicType: pipepb.AtomicType_INT64.Enum(), + }.Build(), + }.Build(), + }.Build(), + }.Build(), }, - }, - }, - }, - }, - }, - }, + }.Build(), + }.Build(), + }.Build(), + }.Build(), + }.Build(), + }.Build(), }, - }, + }.Build(), rt: reflect.TypeOf(struct{ G unexportedFields }{}), }, { - st: &pipepb.Schema{ + st: pipepb.Schema_builder{ Fields: []*pipepb.Field{ - { + pipepb.Field_builder{ Name: "H", - Type: &pipepb.FieldType{ + Type: pipepb.FieldType_builder{ Nullable: true, - TypeInfo: &pipepb.FieldType_LogicalType{ - LogicalType: &pipepb.LogicalType{ - Urn: "schema.unexportedFields", - Representation: &pipepb.FieldType{ - TypeInfo: &pipepb.FieldType_RowType{ - RowType: &pipepb.RowType{ - Schema: &pipepb.Schema{ - Fields: []*pipepb.Field{ - { - Name: "D", - Type: &pipepb.FieldType{ - TypeInfo: &pipepb.FieldType_LogicalType{ - LogicalType: &pipepb.LogicalType{ - Urn: "uint64", - Representation: &pipepb.FieldType{ - TypeInfo: &pipepb.FieldType_AtomicType{ - AtomicType: pipepb.AtomicType_INT64, - }, - }, - }, - }, - }, - }, - }, - }, + LogicalType: pipepb.LogicalType_builder{ + Urn: "schema.unexportedFields", + Representation: pipepb.FieldType_builder{ + RowType: pipepb.RowType_builder{ + Schema: pipepb.Schema_builder{ + Fields: []*pipepb.Field{ + pipepb.Field_builder{ + Name: "D", + Type: pipepb.FieldType_builder{ + LogicalType: pipepb.LogicalType_builder{ + Urn: "uint64", + Representation: pipepb.FieldType_builder{ + AtomicType: pipepb.AtomicType_INT64.Enum(), + }.Build(), + }.Build(), + }.Build(), + }.Build(), }, - }, - }, - }, - }, - }, - }, + }.Build(), + }.Build(), + }.Build(), + }.Build(), + }.Build(), + }.Build(), }, - }, + }.Build(), rt: reflect.TypeOf(struct{ H *unexportedFields }{}), }, { - st: &pipepb.Schema{ + st: pipepb.Schema_builder{ Fields: []*pipepb.Field{ - { + pipepb.Field_builder{ Name: "E", - Type: &pipepb.FieldType{ - TypeInfo: &pipepb.FieldType_AtomicType{ - AtomicType: pipepb.AtomicType_INT16, - }, - }, - }, + Type: pipepb.FieldType_builder{ + AtomicType: pipepb.AtomicType_INT16.Enum(), + }.Build(), + }.Build(), }, Options: []*pipepb.Option{optGoNillable(), logicalOption("*schema.exportedFunc")}, - }, + }.Build(), rt: exportedFuncType, }, { - st: &pipepb.Schema{ + st: pipepb.Schema_builder{ Fields: []*pipepb.Field{ - { + pipepb.Field_builder{ Name: "Q", - Type: &pipepb.FieldType{ - TypeInfo: &pipepb.FieldType_LogicalType{ - LogicalType: &pipepb.LogicalType{ - Urn: "schema.myInt", - Representation: &pipepb.FieldType{ - TypeInfo: &pipepb.FieldType_LogicalType{ - LogicalType: &pipepb.LogicalType{ - Urn: "int", - Representation: &pipepb.FieldType{ - TypeInfo: &pipepb.FieldType_AtomicType{ - AtomicType: pipepb.AtomicType_INT64, - }, - }, - }, - }, - }, - }, - }, - }, - }, + Type: pipepb.FieldType_builder{ + LogicalType: pipepb.LogicalType_builder{ + Urn: "schema.myInt", + Representation: pipepb.FieldType_builder{ + LogicalType: pipepb.LogicalType_builder{ + Urn: "int", + Representation: pipepb.FieldType_builder{ + AtomicType: pipepb.AtomicType_INT64.Enum(), + }.Build(), + }.Build(), + }.Build(), + }.Build(), + }.Build(), + }.Build(), }, - }, + }.Build(), rt: anotherType, }, { - st: &pipepb.Schema{ + st: pipepb.Schema_builder{ Fields: []*pipepb.Field{ - { + pipepb.Field_builder{ Name: "Exported", Options: []*pipepb.Option{optGoEmbedded()}, - Type: &pipepb.FieldType{ - TypeInfo: &pipepb.FieldType_RowType{ - RowType: &pipepb.RowType{ - Schema: &pipepb.Schema{ - Fields: []*pipepb.Field{ - { - Name: "G", - Type: &pipepb.FieldType{ - TypeInfo: &pipepb.FieldType_LogicalType{ - LogicalType: &pipepb.LogicalType{ - Urn: "schema.myInt", - Representation: &pipepb.FieldType{ - TypeInfo: &pipepb.FieldType_LogicalType{ - LogicalType: &pipepb.LogicalType{ - Urn: "int", - Representation: &pipepb.FieldType{ - TypeInfo: &pipepb.FieldType_AtomicType{ - AtomicType: pipepb.AtomicType_INT64, - }, - }, - }, - }, - }, - }, - }, - }, - }, - }, + Type: pipepb.FieldType_builder{ + RowType: pipepb.RowType_builder{ + Schema: pipepb.Schema_builder{ + Fields: []*pipepb.Field{ + pipepb.Field_builder{ + Name: "G", + Type: pipepb.FieldType_builder{ + LogicalType: pipepb.LogicalType_builder{ + Urn: "schema.myInt", + Representation: pipepb.FieldType_builder{ + LogicalType: pipepb.LogicalType_builder{ + Urn: "int", + Representation: pipepb.FieldType_builder{ + AtomicType: pipepb.AtomicType_INT64.Enum(), + }.Build(), + }.Build(), + }.Build(), + }.Build(), + }.Build(), + }.Build(), }, - }, - }, - }, - }, + }.Build(), + }.Build(), + }.Build(), + }.Build(), }, - }, + }.Build(), rt: hasEmbeddedType, }, { - st: &pipepb.Schema{ + st: pipepb.Schema_builder{ Fields: []*pipepb.Field{ - { + pipepb.Field_builder{ Name: "Exported", Options: []*pipepb.Option{optGoEmbedded()}, - Type: &pipepb.FieldType{ + Type: pipepb.FieldType_builder{ Nullable: true, - TypeInfo: &pipepb.FieldType_RowType{ - RowType: &pipepb.RowType{ - Schema: &pipepb.Schema{ - Fields: []*pipepb.Field{ - { - Name: "G", - Type: &pipepb.FieldType{ - TypeInfo: &pipepb.FieldType_LogicalType{ - LogicalType: &pipepb.LogicalType{ - Urn: "schema.myInt", - Representation: &pipepb.FieldType{ - TypeInfo: &pipepb.FieldType_LogicalType{ - LogicalType: &pipepb.LogicalType{ - Urn: "int", - Representation: &pipepb.FieldType{ - TypeInfo: &pipepb.FieldType_AtomicType{ - AtomicType: pipepb.AtomicType_INT64, - }, - }, - }, - }, - }, - }, - }, - }, - }, - }, + RowType: pipepb.RowType_builder{ + Schema: pipepb.Schema_builder{ + Fields: []*pipepb.Field{ + pipepb.Field_builder{ + Name: "G", + Type: pipepb.FieldType_builder{ + LogicalType: pipepb.LogicalType_builder{ + Urn: "schema.myInt", + Representation: pipepb.FieldType_builder{ + LogicalType: pipepb.LogicalType_builder{ + Urn: "int", + Representation: pipepb.FieldType_builder{ + AtomicType: pipepb.AtomicType_INT64.Enum(), + }.Build(), + }.Build(), + }.Build(), + }.Build(), + }.Build(), + }.Build(), }, - }, - }, - }, - }, + }.Build(), + }.Build(), + }.Build(), + }.Build(), }, - }, + }.Build(), rt: hasEmbeddedPtrType, }, { - st: &pipepb.Schema{ + st: pipepb.Schema_builder{ Fields: []*pipepb.Field{ - { + pipepb.Field_builder{ Name: "T", - Type: &pipepb.FieldType{ - TypeInfo: &pipepb.FieldType_AtomicType{ - AtomicType: pipepb.AtomicType_STRING, - }, - }, - }, + Type: pipepb.FieldType_builder{ + AtomicType: pipepb.AtomicType_STRING.Enum(), + }.Build(), + }.Build(), }, Options: []*pipepb.Option{optGoNillable()}, - }, + }.Build(), rt: reflect.TypeOf(&struct { myInt T string i int }{}), }, { - st: &pipepb.Schema{ + st: pipepb.Schema_builder{ Options: []*pipepb.Option{ logicalOption("schema.exportedFunc"), }, Fields: []*pipepb.Field{ - { + pipepb.Field_builder{ Name: "V", - Type: &pipepb.FieldType{ - TypeInfo: &pipepb.FieldType_AtomicType{ - AtomicType: pipepb.AtomicType_INT16, - }, - }, - }, + Type: pipepb.FieldType_builder{ + AtomicType: pipepb.AtomicType_INT16.Enum(), + }.Build(), + }.Build(), }, - }, + }.Build(), rt: reflect.TypeOf(exportedFunc{}), }, { - st: &pipepb.Schema{ + st: pipepb.Schema_builder{ Fields: []*pipepb.Field{ - { + pipepb.Field_builder{ Name: "U", - Type: &pipepb.FieldType{ - TypeInfo: &pipepb.FieldType_LogicalType{ - LogicalType: &pipepb.LogicalType{ - Urn: "schema.exportedFunc", - Representation: &pipepb.FieldType{ - TypeInfo: &pipepb.FieldType_RowType{ - RowType: &pipepb.RowType{ - Schema: &pipepb.Schema{ - Fields: []*pipepb.Field{ - { - Name: "V", - Type: &pipepb.FieldType{ - TypeInfo: &pipepb.FieldType_AtomicType{ - AtomicType: pipepb.AtomicType_INT16, - }, - }, - }, - }, - }, + Type: pipepb.FieldType_builder{ + LogicalType: pipepb.LogicalType_builder{ + Urn: "schema.exportedFunc", + Representation: pipepb.FieldType_builder{ + RowType: pipepb.RowType_builder{ + Schema: pipepb.Schema_builder{ + Fields: []*pipepb.Field{ + pipepb.Field_builder{ + Name: "V", + Type: pipepb.FieldType_builder{ + AtomicType: pipepb.AtomicType_INT16.Enum(), + }.Build(), + }.Build(), }, - }, - }, - }, - }, - }, - }, + }.Build(), + }.Build(), + }.Build(), + }.Build(), + }.Build(), + }.Build(), }, - }, + }.Build(), rt: reflect.TypeOf(struct { U exportedFunc }{}), }, { - st: &pipepb.Schema{ + st: pipepb.Schema_builder{ Fields: []*pipepb.Field{ - { + pipepb.Field_builder{ Name: "U", - Type: &pipepb.FieldType{ - TypeInfo: &pipepb.FieldType_LogicalType{ - LogicalType: &pipepb.LogicalType{ - Urn: "schema.nonRegisteredLogical", - Representation: &pipepb.FieldType{ - TypeInfo: &pipepb.FieldType_RowType{ - RowType: &pipepb.RowType{ - Schema: &pipepb.Schema{ - Fields: []*pipepb.Field{ - { - Name: "K", - Type: &pipepb.FieldType{ - TypeInfo: &pipepb.FieldType_AtomicType{ - AtomicType: pipepb.AtomicType_INT32, - }, - }, - }, - }, - }, + Type: pipepb.FieldType_builder{ + LogicalType: pipepb.LogicalType_builder{ + Urn: "schema.nonRegisteredLogical", + Representation: pipepb.FieldType_builder{ + RowType: pipepb.RowType_builder{ + Schema: pipepb.Schema_builder{ + Fields: []*pipepb.Field{ + pipepb.Field_builder{ + Name: "K", + Type: pipepb.FieldType_builder{ + AtomicType: pipepb.AtomicType_INT32.Enum(), + }.Build(), + }.Build(), }, - }, - }, - }, - }, - }, - }, + }.Build(), + }.Build(), + }.Build(), + }.Build(), + }.Build(), + }.Build(), }, - }, + }.Build(), rt: reflect.TypeOf(struct { U nonRegisteredLogical }{}), diff --git a/sdks/go/pkg/beam/core/runtime/graphx/translate.go b/sdks/go/pkg/beam/core/runtime/graphx/translate.go index 1e30d4258507..41dd4f0b351e 100644 --- a/sdks/go/pkg/beam/core/runtime/graphx/translate.go +++ b/sdks/go/pkg/beam/core/runtime/graphx/translate.go @@ -126,27 +126,27 @@ func CreateEnvironment(ctx context.Context, urn string, extractEnvironmentConfig return nil, errors.Errorf("unsupported environment %v", urn) case URNEnvExternal: config := extractEnvironmentConfig(ctx) - payload := &pipepb.ExternalPayload{Endpoint: &pipepb.ApiServiceDescriptor{Url: config}} + payload := pipepb.ExternalPayload_builder{Endpoint: pipepb.ApiServiceDescriptor_builder{Url: config}.Build()}.Build() serializedPayload = protox.MustEncode(payload) case URNEnvDocker: fallthrough default: config := extractEnvironmentConfig(ctx) - payload := &pipepb.DockerPayload{ContainerImage: config} + payload := pipepb.DockerPayload_builder{ContainerImage: config}.Build() serializedPayload = protox.MustEncode(payload) } - return &pipepb.Environment{ + return pipepb.Environment_builder{ Urn: urn, Payload: serializedPayload, Capabilities: goCapabilities(), Dependencies: []*pipepb.ArtifactInformation{ - { + pipepb.ArtifactInformation_builder{ TypeUrn: URNArtifactFileType, TypePayload: protox.MustEncode(&pipepb.ArtifactFilePayload{}), RoleUrn: URNArtifactGoWorkerRole, - }, + }.Build(), }, - }, nil + }.Build(), nil } // TODO(https://github.com/apache/beam/issues/23893): Along with scoped resource hints, @@ -195,10 +195,10 @@ func Marshal(edges []*graph.MultiEdge, opt *Options) (*pipepb.Pipeline, error) { } } - p := &pipepb.Pipeline{ + p := pipepb.Pipeline_builder{ Components: m.build(), Requirements: m.getRequirements(), - } + }.Build() p, err := pipelinex.Normalize(p) if err != nil { @@ -248,13 +248,13 @@ func newMarshaller(opt *Options) *marshaller { } func (m *marshaller) build() *pipepb.Components { - return &pipepb.Components{ + return pipepb.Components_builder{ Transforms: m.transforms, Pcollections: m.pcollections, WindowingStrategies: m.windowing, Environments: m.environments, Coders: m.coders.Build(), - } + }.Build() } func (m *marshaller) getRequirements() []string { @@ -292,13 +292,13 @@ func (m *marshaller) addScopeTree(s *ScopeTree) (string, error) { metadata := m.opt.GetContextReg().ExtractTransformMetadata(s.Scope.Scope.Context) - transform := &pipepb.PTransform{ + transform := pipepb.PTransform_builder{ UniqueName: s.Scope.Name, Subtransforms: subtransforms, EnvironmentId: m.addDefaultEnv(), Annotations: metadata.Annotations, // DisplayData: metadata.DisplayData, - } + }.Build() if err := m.updateIfCombineComposite(s, transform); err != nil { return "", errors.Wrapf(err, "failed to add scope tree: %v", s) @@ -330,14 +330,14 @@ func (m *marshaller) updateIfCombineComposite(s *ScopeTree, transform *pipepb.PT if err != nil { return errors.Wrapf(err, "failed to update PTransform spec: %v", transform) } - payload := &pipepb.CombinePayload{ - CombineFn: &pipepb.FunctionSpec{ + payload := pipepb.CombinePayload_builder{ + CombineFn: pipepb.FunctionSpec_builder{ Urn: URNDoFn, Payload: []byte(mustEncodeMultiEdge), - }, + }.Build(), AccumulatorCoderId: acID, - } - transform.Spec = &pipepb.FunctionSpec{Urn: URNCombinePerKey, Payload: protox.MustEncode(payload)} + }.Build() + transform.SetSpec(pipepb.FunctionSpec_builder{Urn: URNCombinePerKey, Payload: protox.MustEncode(payload)}.Build()) return nil } @@ -414,7 +414,7 @@ func (m *marshaller) addMultiEdge(edge NamedEdge) ([]string, error) { var spec *pipepb.FunctionSpec switch edge.Edge.Op { case graph.Impulse: - spec = &pipepb.FunctionSpec{Urn: URNImpulse} + spec = pipepb.FunctionSpec_builder{Urn: URNImpulse}.Build() case graph.ParDo: si := make(map[string]*pipepb.SideInput) @@ -432,18 +432,18 @@ func (m *marshaller) addMultiEdge(edge NamedEdge) ([]string, error) { return nil, err } - si[fmt.Sprintf("i%v", i)] = &pipepb.SideInput{ - AccessPattern: &pipepb.FunctionSpec{ + si[fmt.Sprintf("i%v", i)] = pipepb.SideInput_builder{ + AccessPattern: pipepb.FunctionSpec_builder{ Urn: URNIterableSideInput, - }, - ViewFn: &pipepb.FunctionSpec{ + }.Build(), + ViewFn: pipepb.FunctionSpec_builder{ Urn: "foo", - }, - WindowMappingFn: &pipepb.FunctionSpec{ + }.Build(), + WindowMappingFn: pipepb.FunctionSpec_builder{ Urn: mappingUrn, - Payload: siWSpec.Payload, - }, - } + Payload: siWSpec.GetPayload(), + }.Build(), + }.Build() case graph.Map, graph.MultiMap: // Already in a MultiMap form, don't need to add a fixed key. @@ -456,18 +456,18 @@ func (m *marshaller) addMultiEdge(edge NamedEdge) ([]string, error) { return nil, err } - si[fmt.Sprintf("i%v", i)] = &pipepb.SideInput{ - AccessPattern: &pipepb.FunctionSpec{ + si[fmt.Sprintf("i%v", i)] = pipepb.SideInput_builder{ + AccessPattern: pipepb.FunctionSpec_builder{ Urn: URNMultimapSideInput, - }, - ViewFn: &pipepb.FunctionSpec{ + }.Build(), + ViewFn: pipepb.FunctionSpec_builder{ Urn: "foo", - }, - WindowMappingFn: &pipepb.FunctionSpec{ + }.Build(), + WindowMappingFn: pipepb.FunctionSpec_builder{ Urn: mappingUrn, - Payload: siWSpec.Payload, - }, - } + Payload: siWSpec.GetPayload(), + }.Build(), + }.Build() default: return nil, errors.Errorf("unexpected input kind: %v", edge) } @@ -478,23 +478,23 @@ func (m *marshaller) addMultiEdge(edge NamedEdge) ([]string, error) { return handleErr(err) } - payload := &pipepb.ParDoPayload{ - DoFn: &pipepb.FunctionSpec{ + payload := pipepb.ParDoPayload_builder{ + DoFn: pipepb.FunctionSpec_builder{ Urn: URNDoFn, Payload: []byte(mustEncodeMultiEdge), - }, + }.Build(), SideInputs: si, - } + }.Build() if edge.Edge.DoFn.IsSplittable() { coderID, err := m.coders.Add(edge.Edge.RestrictionCoder) if err != nil { return handleErr(err) } - payload.RestrictionCoderId = coderID + payload.SetRestrictionCoderId(coderID) m.requirements[URNRequiresSplittableDoFn] = true } if _, ok := edge.Edge.DoFn.ProcessElementFn().BundleFinalization(); ok { - payload.RequestsFinalization = true + payload.SetRequestsFinalization(true) m.requirements[URNRequiresBundleFinalization] = true } if _, ok := edge.Edge.DoFn.ProcessElementFn().StateProvider(); ok { @@ -520,27 +520,23 @@ func (m *marshaller) addMultiEdge(edge NamedEdge) ([]string, error) { } switch ps.StateType() { case state.TypeValue: - stateSpecs[ps.StateKey()] = &pipepb.StateSpec{ - Spec: &pipepb.StateSpec_ReadModifyWriteSpec{ - ReadModifyWriteSpec: &pipepb.ReadModifyWriteStateSpec{ - CoderId: coderID, - }, - }, - Protocol: &pipepb.FunctionSpec{ + stateSpecs[ps.StateKey()] = pipepb.StateSpec_builder{ + ReadModifyWriteSpec: pipepb.ReadModifyWriteStateSpec_builder{ + CoderId: coderID, + }.Build(), + Protocol: pipepb.FunctionSpec_builder{ Urn: URNBagUserState, - }, - } + }.Build(), + }.Build() case state.TypeBag: - stateSpecs[ps.StateKey()] = &pipepb.StateSpec{ - Spec: &pipepb.StateSpec_BagSpec{ - BagSpec: &pipepb.BagStateSpec{ - ElementCoderId: coderID, - }, - }, - Protocol: &pipepb.FunctionSpec{ + stateSpecs[ps.StateKey()] = pipepb.StateSpec_builder{ + BagSpec: pipepb.BagStateSpec_builder{ + ElementCoderId: coderID, + }.Build(), + Protocol: pipepb.FunctionSpec_builder{ Urn: URNBagUserState, - }, - } + }.Build(), + }.Build() case state.TypeCombining: cps := ps.(state.CombiningPipelineState).GetCombineFn() f, err := graph.NewFn(cps) @@ -559,48 +555,42 @@ func (m *marshaller) addMultiEdge(edge NamedEdge) ([]string, error) { if err != nil { return handleErr(err) } - stateSpecs[ps.StateKey()] = &pipepb.StateSpec{ - Spec: &pipepb.StateSpec_CombiningSpec{ - CombiningSpec: &pipepb.CombiningStateSpec{ - AccumulatorCoderId: coderID, - CombineFn: &pipepb.FunctionSpec{ - Urn: "beam:combinefn:gosdk:v1", - Payload: []byte(mustEncodeMultiEdge), - }, - }, - }, - Protocol: &pipepb.FunctionSpec{ + stateSpecs[ps.StateKey()] = pipepb.StateSpec_builder{ + CombiningSpec: pipepb.CombiningStateSpec_builder{ + AccumulatorCoderId: coderID, + CombineFn: pipepb.FunctionSpec_builder{ + Urn: "beam:combinefn:gosdk:v1", + Payload: []byte(mustEncodeMultiEdge), + }.Build(), + }.Build(), + Protocol: pipepb.FunctionSpec_builder{ Urn: URNBagUserState, - }, - } + }.Build(), + }.Build() case state.TypeMap: - stateSpecs[ps.StateKey()] = &pipepb.StateSpec{ - Spec: &pipepb.StateSpec_MapSpec{ - MapSpec: &pipepb.MapStateSpec{ - KeyCoderId: keyCoderID, - ValueCoderId: coderID, - }, - }, - Protocol: &pipepb.FunctionSpec{ + stateSpecs[ps.StateKey()] = pipepb.StateSpec_builder{ + MapSpec: pipepb.MapStateSpec_builder{ + KeyCoderId: keyCoderID, + ValueCoderId: coderID, + }.Build(), + Protocol: pipepb.FunctionSpec_builder{ Urn: URNMultiMapUserState, - }, - } + }.Build(), + }.Build() case state.TypeSet: - stateSpecs[ps.StateKey()] = &pipepb.StateSpec{ - Spec: &pipepb.StateSpec_SetSpec{ - SetSpec: &pipepb.SetStateSpec{ - ElementCoderId: keyCoderID, - }, - }, - Protocol: &pipepb.FunctionSpec{ + stateSpecs[ps.StateKey()] = pipepb.StateSpec_builder{ + SetSpec: pipepb.SetStateSpec_builder{ + ElementCoderId: keyCoderID, + }.Build(), + Protocol: pipepb.FunctionSpec_builder{ Urn: URNMultiMapUserState, - }, - } + }.Build(), + }.Build() default: return nil, errors.Errorf("State type %v not recognized for state %v", ps.StateKey(), ps) } } - payload.StateSpecs = stateSpecs + payload.SetStateSpecs(stateSpecs) } if _, ok := edge.Edge.DoFn.ProcessElementFn().TimerProvider(); ok { m.requirements[URNRequiresStatefulProcessing] = true @@ -610,7 +600,7 @@ func (m *marshaller) addMultiEdge(edge NamedEdge) ([]string, error) { // All timers for a single DoFn have the same key and window coders, that match the input PCollection. mainInputID := inputs["i0"] pCol := m.pcollections[mainInputID] - kvCoder := m.coders.coders[pCol.CoderId] + kvCoder := m.coders.coders[pCol.GetCoderId()] if kvCoder.GetSpec().GetUrn() != urnKVCoder { return nil, errors.Errorf("timer using DoFn %v doesn't use a KV as PCollection input. Unable to extract key coder for timers, got %v", edge.Name, kvCoder.GetSpec().GetUrn()) } @@ -624,15 +614,15 @@ func (m *marshaller) addMultiEdge(edge NamedEdge) ([]string, error) { for _, pt := range pipelineTimers { for timerFamilyID, timeDomain := range pt.Timers() { - timerSpecs[timerFamilyID] = &pipepb.TimerFamilySpec{ + timerSpecs[timerFamilyID] = pipepb.TimerFamilySpec_builder{ TimeDomain: pipepb.TimeDomain_Enum(timeDomain), TimerFamilyCoderId: timerCoderID, - } + }.Build() } } - payload.TimerFamilySpecs = timerSpecs + payload.SetTimerFamilySpecs(timerSpecs) } - spec = &pipepb.FunctionSpec{Urn: URNParDo, Payload: protox.MustEncode(payload)} + spec = pipepb.FunctionSpec_builder{Urn: URNParDo, Payload: protox.MustEncode(payload)}.Build() annotations = edge.Edge.DoFn.Annotations() case graph.Combine: @@ -640,33 +630,33 @@ func (m *marshaller) addMultiEdge(edge NamedEdge) ([]string, error) { if err != nil { return handleErr(err) } - payload := &pipepb.ParDoPayload{ - DoFn: &pipepb.FunctionSpec{ + payload := pipepb.ParDoPayload_builder{ + DoFn: pipepb.FunctionSpec_builder{ Urn: URNDoFn, Payload: []byte(mustEncodeMultiEdge), - }, - } - spec = &pipepb.FunctionSpec{Urn: URNParDo, Payload: protox.MustEncode(payload)} + }.Build(), + }.Build() + spec = pipepb.FunctionSpec_builder{Urn: URNParDo, Payload: protox.MustEncode(payload)}.Build() case graph.Flatten: - spec = &pipepb.FunctionSpec{Urn: URNFlatten} + spec = pipepb.FunctionSpec_builder{Urn: URNFlatten}.Build() case graph.CoGBK: - spec = &pipepb.FunctionSpec{Urn: URNGBK} + spec = pipepb.FunctionSpec_builder{Urn: URNGBK}.Build() case graph.WindowInto: windowFn, err := makeWindowFn(edge.Edge.WindowFn) if err != nil { return handleErr(err) } - payload := &pipepb.WindowIntoPayload{ + payload := pipepb.WindowIntoPayload_builder{ WindowFn: windowFn, - } - spec = &pipepb.FunctionSpec{Urn: URNWindow, Payload: protox.MustEncode(payload)} + }.Build() + spec = pipepb.FunctionSpec_builder{Urn: URNWindow, Payload: protox.MustEncode(payload)}.Build() case graph.External: pyld := edge.Edge.Payload - spec = &pipepb.FunctionSpec{Urn: pyld.URN, Payload: pyld.Data} + spec = pipepb.FunctionSpec_builder{Urn: pyld.URN, Payload: pyld.Data}.Build() if len(pyld.InputsMap) != 0 { if got, want := len(pyld.InputsMap), len(edge.Edge.Input); got != want { @@ -700,18 +690,18 @@ func (m *marshaller) addMultiEdge(edge NamedEdge) ([]string, error) { } var transformEnvID = "" - if !(spec.Urn == URNGBK || spec.Urn == URNImpulse) { + if !(spec.GetUrn() == URNGBK || spec.GetUrn() == URNImpulse) { transformEnvID = m.addDefaultEnv() } - transform := &pipepb.PTransform{ + transform := pipepb.PTransform_builder{ UniqueName: edge.Name, Spec: spec, Inputs: inputs, Outputs: outputs, EnvironmentId: transformEnvID, Annotations: annotations, - } + }.Build() m.transforms[id] = transform allPIds = append(allPIds, id) return allPIds, nil @@ -734,22 +724,22 @@ func (m *marshaller) expandCrossLanguage(namedEdge NamedEdge) (string, error) { inputs[tag] = nodeID(n) } - spec := &pipepb.FunctionSpec{ + spec := pipepb.FunctionSpec_builder{ Urn: edge.External.Urn, Payload: edge.External.Payload, - } + }.Build() - transform := &pipepb.PTransform{ + transform := pipepb.PTransform_builder{ UniqueName: namedEdge.Name, Spec: spec, Inputs: inputs, EnvironmentId: m.addDefaultEnv(), - } + }.Build() // Add the coders for output in the marshaller even if expanded is nil // for output coder field in expansion request. // We need this specifically for Python External Transforms. - names := strings.Split(spec.Urn, ":") + names := strings.Split(spec.GetUrn(), ":") if len(names) > 2 && names[2] == "python" { for _, out := range edge.Output { id, err := m.coders.Add(out.To.Coder) @@ -771,12 +761,12 @@ func (m *marshaller) expandCrossLanguage(namedEdge NamedEdge) (string, error) { } outputs[fmt.Sprintf("i%v", i)] = nodeID(out.To) } - transform.Outputs = outputs + transform.SetOutputs(outputs) environment, err := ExpandedTransform(edge.External.Expanded) if err != nil { return "", errors.Wrapf(err, "failed to expand cross language transform for edge: %v", namedEdge) } - transform.EnvironmentId = environment.EnvironmentId + transform.SetEnvironmentId(environment.GetEnvironmentId()) } m.transforms[id] = transform @@ -825,25 +815,25 @@ func (m *marshaller) expandCoGBK(edge NamedEdge) (string, error) { // Inject(i) injectID := fmt.Sprintf("%v_inject%v", id, i) - payload := &pipepb.ParDoPayload{ - DoFn: &pipepb.FunctionSpec{ + payload := pipepb.ParDoPayload_builder{ + DoFn: pipepb.FunctionSpec_builder{ Urn: URNInject, Payload: []byte(protox.MustEncodeBase64(&v1pb.TransformPayload{ Urn: URNInject, Inject: &v1pb.InjectPayload{N: (int32)(i)}, })), - }, - } - inject := &pipepb.PTransform{ + }.Build(), + }.Build() + inject := pipepb.PTransform_builder{ UniqueName: injectID, - Spec: &pipepb.FunctionSpec{ + Spec: pipepb.FunctionSpec_builder{ Urn: URNParDo, Payload: protox.MustEncode(payload), - }, + }.Build(), Inputs: map[string]string{"i0": nodeID(in.From)}, Outputs: map[string]string{"i0": out}, EnvironmentId: m.addDefaultEnv(), - } + }.Build() m.transforms[injectID] = inject subtransforms = append(subtransforms, injectID) @@ -860,13 +850,13 @@ func (m *marshaller) expandCoGBK(edge NamedEdge) (string, error) { } flattenID := fmt.Sprintf("%v_flatten", id) - flatten := &pipepb.PTransform{ + flatten := pipepb.PTransform_builder{ UniqueName: flattenID, - Spec: &pipepb.FunctionSpec{Urn: URNFlatten}, + Spec: pipepb.FunctionSpec_builder{Urn: URNFlatten}.Build(), Inputs: inputs, Outputs: map[string]string{"i0": out}, EnvironmentId: m.addDefaultEnv(), - } + }.Build() m.transforms[flattenID] = flatten subtransforms = append(subtransforms, flattenID) @@ -878,12 +868,12 @@ func (m *marshaller) expandCoGBK(edge NamedEdge) (string, error) { } gbkID := fmt.Sprintf("%v_gbk", id) - gbk := &pipepb.PTransform{ + gbk := pipepb.PTransform_builder{ UniqueName: gbkID, - Spec: &pipepb.FunctionSpec{Urn: URNGBK}, + Spec: pipepb.FunctionSpec_builder{Urn: URNGBK}.Build(), Inputs: map[string]string{"i0": out}, Outputs: map[string]string{"i0": gbkOut}, - } + }.Build() m.transforms[gbkID] = gbk subtransforms = append(subtransforms, gbkID) @@ -894,35 +884,35 @@ func (m *marshaller) expandCoGBK(edge NamedEdge) (string, error) { } expandID := fmt.Sprintf("%v_expand", id) - payload := &pipepb.ParDoPayload{ - DoFn: &pipepb.FunctionSpec{ + payload := pipepb.ParDoPayload_builder{ + DoFn: pipepb.FunctionSpec_builder{ Urn: URNExpand, Payload: []byte(protox.MustEncodeBase64(&v1pb.TransformPayload{ Urn: URNExpand, })), - }, - } - expand := &pipepb.PTransform{ + }.Build(), + }.Build() + expand := pipepb.PTransform_builder{ UniqueName: expandID, - Spec: &pipepb.FunctionSpec{ + Spec: pipepb.FunctionSpec_builder{ Urn: URNParDo, Payload: protox.MustEncode(payload), - }, + }.Build(), Inputs: map[string]string{"i0": gbkOut}, Outputs: map[string]string{"i0": nodeID(outNode)}, EnvironmentId: m.addDefaultEnv(), - } + }.Build() m.transforms[id] = expand subtransforms = append(subtransforms, id) // Add composite for visualization cogbkID := fmt.Sprintf("%v_cogbk", id) - m.transforms[cogbkID] = &pipepb.PTransform{ + m.transforms[cogbkID] = pipepb.PTransform_builder{ UniqueName: edge.Name, Subtransforms: subtransforms, EnvironmentId: m.addDefaultEnv(), - } + }.Build() return cogbkID, nil } @@ -1002,16 +992,14 @@ func (m *marshaller) expandReshuffle(edge NamedEdge) (string, error) { if err != nil { return handleErr(err) } - m.pcollections[postReify].WindowingStrategyId = - m.internWindowingStrategy(&pipepb.WindowingStrategy{ + m.pcollections[postReify].SetWindowingStrategyId( + m.internWindowingStrategy(pipepb.WindowingStrategy_builder{ // Not segregated by time... WindowFn: windowFn, // ...output after every element is received... - Trigger: &pipepb.Trigger{ - Trigger: &pipepb.Trigger_Always_{ - Always: &pipepb.Trigger_Always{}, - }, - }, + Trigger: pipepb.Trigger_builder{ + Always: &pipepb.Trigger_Always{}, + }.Build(), // ...and after outputing, discard the output elements... AccumulationMode: pipepb.AccumulationMode_DISCARDING, // ...and since every pane should have 1 element, @@ -1025,14 +1013,14 @@ func (m *marshaller) expandReshuffle(edge NamedEdge) (string, error) { ClosingBehavior: pipepb.ClosingBehavior_EMIT_IF_NONEMPTY, AllowedLateness: int64(in.From.WindowingStrategy().AllowedLateness), OnTimeBehavior: pipepb.OnTimeBehavior_FIRE_ALWAYS, - }) + }.Build())) } // Inputs (i) inputID := fmt.Sprintf("%v_reifyts", id) - payload := &pipepb.ParDoPayload{ - DoFn: &pipepb.FunctionSpec{ + payload := pipepb.ParDoPayload_builder{ + DoFn: pipepb.FunctionSpec_builder{ Urn: URNReshuffleInput, Payload: []byte(protox.MustEncodeBase64(&v1pb.TransformPayload{ Urn: URNReshuffleInput, @@ -1041,18 +1029,18 @@ func (m *marshaller) expandReshuffle(edge NamedEdge) (string, error) { CoderPayloads: coderPayloads, }, })), - }, - } - input := &pipepb.PTransform{ + }.Build(), + }.Build() + input := pipepb.PTransform_builder{ UniqueName: inputID, - Spec: &pipepb.FunctionSpec{ + Spec: pipepb.FunctionSpec_builder{ Urn: URNParDo, Payload: protox.MustEncode(payload), - }, + }.Build(), Inputs: map[string]string{"i0": nodeID(in.From)}, Outputs: map[string]string{"i0": postReify}, EnvironmentId: m.addDefaultEnv(), - } + }.Build() m.transforms[inputID] = input subtransforms = append(subtransforms, inputID) @@ -1066,12 +1054,12 @@ func (m *marshaller) expandReshuffle(edge NamedEdge) (string, error) { } gbkID := fmt.Sprintf("%v_gbk", id) - gbk := &pipepb.PTransform{ + gbk := pipepb.PTransform_builder{ UniqueName: gbkID, - Spec: &pipepb.FunctionSpec{Urn: URNGBK}, + Spec: pipepb.FunctionSpec_builder{Urn: URNGBK}.Build(), Inputs: map[string]string{"i0": postReify}, Outputs: map[string]string{"i0": gbkOut}, - } + }.Build() m.transforms[gbkID] = gbk subtransforms = append(subtransforms, gbkID) @@ -1081,11 +1069,11 @@ func (m *marshaller) expandReshuffle(edge NamedEdge) (string, error) { if err != nil { return handleErr(err) } - m.pcollections[outPCol].WindowingStrategyId = preservedWSID + m.pcollections[outPCol].SetWindowingStrategyId(preservedWSID) outputID := fmt.Sprintf("%v_unreify", id) - outputPayload := &pipepb.ParDoPayload{ - DoFn: &pipepb.FunctionSpec{ + outputPayload := pipepb.ParDoPayload_builder{ + DoFn: pipepb.FunctionSpec_builder{ Urn: URNReshuffleOutput, Payload: []byte(protox.MustEncodeBase64(&v1pb.TransformPayload{ Urn: URNReshuffleOutput, @@ -1094,31 +1082,31 @@ func (m *marshaller) expandReshuffle(edge NamedEdge) (string, error) { CoderPayloads: coderPayloads, }, })), - }, - } - output := &pipepb.PTransform{ + }.Build(), + }.Build() + output := pipepb.PTransform_builder{ UniqueName: outputID, - Spec: &pipepb.FunctionSpec{ + Spec: pipepb.FunctionSpec_builder{ Urn: URNParDo, Payload: protox.MustEncode(outputPayload), - }, + }.Build(), Inputs: map[string]string{"i0": gbkOut}, Outputs: map[string]string{"i0": nodeID(outNode)}, EnvironmentId: m.addDefaultEnv(), - } + }.Build() m.transforms[id] = output subtransforms = append(subtransforms, id) // Add composite for visualization, or runner optimization reshuffleID := fmt.Sprintf("%v_reshuffle", id) - m.transforms[reshuffleID] = &pipepb.PTransform{ + m.transforms[reshuffleID] = pipepb.PTransform_builder{ UniqueName: edge.Name, Subtransforms: subtransforms, - Spec: &pipepb.FunctionSpec{ + Spec: pipepb.FunctionSpec_builder{ Urn: URNReshuffle, - }, + }.Build(), EnvironmentId: m.addDefaultEnv(), - } + }.Build() return reshuffleID, nil } @@ -1164,12 +1152,12 @@ func (m *marshaller) makeNode(id, cid string, n *graph.Node) (string, error) { return "", errors.Wrapf(err, "failed to make node %v with node id %v", n, id) } - col := &pipepb.PCollection{ + col := pipepb.PCollection_builder{ UniqueName: id, CoderId: cid, IsBounded: boolToBounded(n.Bounded()), WindowingStrategyId: windowingStrategyId, - } + }.Build() m.pcollections[id] = col return id, nil } @@ -1194,7 +1182,7 @@ func (m *marshaller) addDefaultEnv() string { // Add the pipeline level resource hints here for now. // TODO(https://github.com/apache/beam/issues/23893) move to a better place for // scoped hints in next pass, which affect number of environments set by Go pipelines. - env.ResourceHints = m.opt.PipelineResourceHints.Payloads() + env.SetResourceHints(m.opt.PipelineResourceHints.Payloads()) m.environments[defaultEnvId] = env } return defaultEnvId @@ -1205,7 +1193,7 @@ func (m *marshaller) addWindowingStrategy(w *window.WindowingStrategy) (string, if err != nil { return "", errors.Wrapf(err, "failed to add window strategy %v", w) } - ws.EnvironmentId = m.addDefaultEnv() + ws.SetEnvironmentId(m.addDefaultEnv()) return m.internWindowingStrategy(ws), nil } @@ -1243,7 +1231,7 @@ func MarshalWindowingStrategy(c *CoderMarshaller, w *window.WindowingStrategy) ( mergeStat = pipepb.MergeStatus_NON_MERGING } - ws := &pipepb.WindowingStrategy{ + ws := pipepb.WindowingStrategy_builder{ WindowFn: windowFn, MergeStatus: mergeStat, WindowCoderId: windowCoderID, @@ -1253,7 +1241,7 @@ func MarshalWindowingStrategy(c *CoderMarshaller, w *window.WindowingStrategy) ( ClosingBehavior: pipepb.ClosingBehavior_EMIT_IF_NONEMPTY, AllowedLateness: int64(w.AllowedLateness), OnTimeBehavior: pipepb.OnTimeBehavior_FIRE_IF_NONEMPTY, - } + }.Build() return ws, nil } @@ -1275,33 +1263,25 @@ func makeAccumulationMode(m window.AccumulationMode) pipepb.AccumulationMode_Enu func makeTrigger(t trigger.Trigger) *pipepb.Trigger { switch t := t.(type) { case *trigger.DefaultTrigger: - return &pipepb.Trigger{ - Trigger: &pipepb.Trigger_Default_{ - Default: &pipepb.Trigger_Default{}, - }, - } + return pipepb.Trigger_builder{ + Default: &pipepb.Trigger_Default{}, + }.Build() case *trigger.AlwaysTrigger: - return &pipepb.Trigger{ - Trigger: &pipepb.Trigger_Always_{ - Always: &pipepb.Trigger_Always{}, - }, - } + return pipepb.Trigger_builder{ + Always: &pipepb.Trigger_Always{}, + }.Build() case *trigger.AfterAnyTrigger: - return &pipepb.Trigger{ - Trigger: &pipepb.Trigger_AfterAny_{ - AfterAny: &pipepb.Trigger_AfterAny{ - Subtriggers: extractSubtriggers(t.SubTriggers()), - }, - }, - } + return pipepb.Trigger_builder{ + AfterAny: pipepb.Trigger_AfterAny_builder{ + Subtriggers: extractSubtriggers(t.SubTriggers()), + }.Build(), + }.Build() case *trigger.AfterAllTrigger: - return &pipepb.Trigger{ - Trigger: &pipepb.Trigger_AfterAll_{ - AfterAll: &pipepb.Trigger_AfterAll{ - Subtriggers: extractSubtriggers(t.SubTriggers()), - }, - }, - } + return pipepb.Trigger_builder{ + AfterAll: pipepb.Trigger_AfterAll_builder{ + Subtriggers: extractSubtriggers(t.SubTriggers()), + }.Build(), + }.Build() case *trigger.AfterProcessingTimeTrigger: if len(t.TimestampTransforms()) == 0 { panic("AfterProcessingTime trigger set without a delay or alignment.") @@ -1311,88 +1291,66 @@ func makeTrigger(t trigger.Trigger) *pipepb.Trigger { var ttp *pipepb.TimestampTransform switch tt := tt.(type) { case trigger.DelayTransform: - ttp = &pipepb.TimestampTransform{ - TimestampTransform: &pipepb.TimestampTransform_Delay_{ - Delay: &pipepb.TimestampTransform_Delay{DelayMillis: tt.Delay}, - }} + ttp = pipepb.TimestampTransform_builder{ + Delay: pipepb.TimestampTransform_Delay_builder{DelayMillis: tt.Delay}.Build()}.Build() case trigger.AlignToTransform: - ttp = &pipepb.TimestampTransform{ - TimestampTransform: &pipepb.TimestampTransform_AlignTo_{ - AlignTo: &pipepb.TimestampTransform_AlignTo{ - Period: tt.Period, - Offset: tt.Offset, - }, - }} + ttp = pipepb.TimestampTransform_builder{ + AlignTo: pipepb.TimestampTransform_AlignTo_builder{ + Period: tt.Period, + Offset: tt.Offset, + }.Build()}.Build() } tts = append(tts, ttp) } - return &pipepb.Trigger{ - Trigger: &pipepb.Trigger_AfterProcessingTime_{ - AfterProcessingTime: &pipepb.Trigger_AfterProcessingTime{ - TimestampTransforms: tts, - }, - }, - } + return pipepb.Trigger_builder{ + AfterProcessingTime: pipepb.Trigger_AfterProcessingTime_builder{ + TimestampTransforms: tts, + }.Build(), + }.Build() case *trigger.AfterCountTrigger: - return &pipepb.Trigger{ - Trigger: &pipepb.Trigger_ElementCount_{ - ElementCount: &pipepb.Trigger_ElementCount{ElementCount: t.ElementCount()}, - }, - } + return pipepb.Trigger_builder{ + ElementCount: pipepb.Trigger_ElementCount_builder{ElementCount: t.ElementCount()}.Build(), + }.Build() case *trigger.AfterEndOfWindowTrigger: var lateTrigger *pipepb.Trigger if t.Late() != nil { lateTrigger = makeTrigger(t.Late()) } - return &pipepb.Trigger{ - Trigger: &pipepb.Trigger_AfterEndOfWindow_{ - AfterEndOfWindow: &pipepb.Trigger_AfterEndOfWindow{ - EarlyFirings: makeTrigger(t.Early()), - LateFirings: lateTrigger, - }, - }, - } + return pipepb.Trigger_builder{ + AfterEndOfWindow: pipepb.Trigger_AfterEndOfWindow_builder{ + EarlyFirings: makeTrigger(t.Early()), + LateFirings: lateTrigger, + }.Build(), + }.Build() case *trigger.RepeatTrigger: - return &pipepb.Trigger{ - Trigger: &pipepb.Trigger_Repeat_{ - Repeat: &pipepb.Trigger_Repeat{Subtrigger: makeTrigger(t.SubTrigger())}, - }, - } + return pipepb.Trigger_builder{ + Repeat: pipepb.Trigger_Repeat_builder{Subtrigger: makeTrigger(t.SubTrigger())}.Build(), + }.Build() case *trigger.NeverTrigger: - return &pipepb.Trigger{ - Trigger: &pipepb.Trigger_Never_{ - Never: &pipepb.Trigger_Never{}, - }, - } + return pipepb.Trigger_builder{ + Never: &pipepb.Trigger_Never{}, + }.Build() case *trigger.AfterSynchronizedProcessingTimeTrigger: - return &pipepb.Trigger{ - Trigger: &pipepb.Trigger_AfterSynchronizedProcessingTime_{ - AfterSynchronizedProcessingTime: &pipepb.Trigger_AfterSynchronizedProcessingTime{}, - }, - } + return pipepb.Trigger_builder{ + AfterSynchronizedProcessingTime: &pipepb.Trigger_AfterSynchronizedProcessingTime{}, + }.Build() case *trigger.OrFinallyTrigger: - return &pipepb.Trigger{ - Trigger: &pipepb.Trigger_OrFinally_{ - OrFinally: &pipepb.Trigger_OrFinally{ - Main: makeTrigger(t.Main()), - Finally: makeTrigger(t.Finally()), - }, - }, - } + return pipepb.Trigger_builder{ + OrFinally: pipepb.Trigger_OrFinally_builder{ + Main: makeTrigger(t.Main()), + Finally: makeTrigger(t.Finally()), + }.Build(), + }.Build() case *trigger.AfterEachTrigger: - return &pipepb.Trigger{ - Trigger: &pipepb.Trigger_AfterEach_{ - AfterEach: &pipepb.Trigger_AfterEach{ - Subtriggers: extractSubtriggers(t.Subtriggers()), - }, - }, - } + return pipepb.Trigger_builder{ + AfterEach: pipepb.Trigger_AfterEach_builder{ + Subtriggers: extractSubtriggers(t.Subtriggers()), + }.Build(), + }.Build() default: - return &pipepb.Trigger{ - Trigger: &pipepb.Trigger_Default_{ - Default: &pipepb.Trigger_Default{}, - }, - } + return pipepb.Trigger_builder{ + Default: &pipepb.Trigger_Default{}, + }.Build() } } @@ -1411,37 +1369,37 @@ func extractSubtriggers(t []trigger.Trigger) []*pipepb.Trigger { func makeWindowFn(w *window.Fn) (*pipepb.FunctionSpec, error) { switch w.Kind { case window.GlobalWindows: - return &pipepb.FunctionSpec{ + return pipepb.FunctionSpec_builder{ Urn: URNGlobalWindowsWindowFn, - }, nil + }.Build(), nil case window.FixedWindows: - return &pipepb.FunctionSpec{ + return pipepb.FunctionSpec_builder{ Urn: URNFixedWindowsWindowFn, Payload: protox.MustEncode( - &pipepb.FixedWindowsPayload{ + pipepb.FixedWindowsPayload_builder{ Size: durationpb.New(w.Size), - }, + }.Build(), ), - }, nil + }.Build(), nil case window.SlidingWindows: - return &pipepb.FunctionSpec{ + return pipepb.FunctionSpec_builder{ Urn: URNSlidingWindowsWindowFn, Payload: protox.MustEncode( - &pipepb.SlidingWindowsPayload{ + pipepb.SlidingWindowsPayload_builder{ Size: durationpb.New(w.Size), Period: durationpb.New(w.Period), - }, + }.Build(), ), - }, nil + }.Build(), nil case window.Sessions: - return &pipepb.FunctionSpec{ + return pipepb.FunctionSpec_builder{ Urn: URNSessionsWindowFn, Payload: protox.MustEncode( - &pipepb.SessionWindowsPayload{ + pipepb.SessionWindowsPayload_builder{ GapSize: durationpb.New(w.Gap), - }, + }.Build(), ), - }, nil + }.Build(), nil default: return nil, errors.Errorf("unexpected windowing strategy: %v", w) } @@ -1491,11 +1449,11 @@ func UpdateDefaultEnvWorkerType(typeUrn string, pyld []byte, p *pipepb.Pipeline) return errors.Errorf("unable to find default Go environment with ID %q", defaultEnvId) } for _, dep := range env.GetDependencies() { - if dep.RoleUrn != URNArtifactGoWorkerRole { + if dep.GetRoleUrn() != URNArtifactGoWorkerRole { continue } - dep.TypeUrn = typeUrn - dep.TypePayload = pyld + dep.SetTypeUrn(typeUrn) + dep.SetTypePayload(pyld) return nil } return errors.Errorf("unable to find dependency with %q role in environment with ID %q,", URNArtifactGoWorkerRole, defaultEnvId) diff --git a/sdks/go/pkg/beam/core/runtime/graphx/translate_test.go b/sdks/go/pkg/beam/core/runtime/graphx/translate_test.go index e18a5f97796b..2f52818200d7 100644 --- a/sdks/go/pkg/beam/core/runtime/graphx/translate_test.go +++ b/sdks/go/pkg/beam/core/runtime/graphx/translate_test.go @@ -170,12 +170,12 @@ func TestMarshal(t *testing.T) { t.Fatalf("got %v edges, want %v", got, want) } - payload, err := proto.Marshal(&pipepb.DockerPayload{ContainerImage: "foo"}) + payload, err := proto.Marshal(pipepb.DockerPayload_builder{ContainerImage: "foo"}.Build()) if err != nil { t.Fatal(err) } p, err := graphx.Marshal(edges, - &graphx.Options{Environment: &pipepb.Environment{Urn: "beam:env:docker:v1", Payload: payload}}) + &graphx.Options{Environment: pipepb.Environment_builder{Urn: "beam:env:docker:v1", Payload: payload}.Build()}) if err != nil { t.Fatal(err) } @@ -236,12 +236,12 @@ func TestMarshal_PTransformAnnotations(t *testing.T) { t.Fatal(err) } - payload, err := proto.Marshal(&pipepb.DockerPayload{ContainerImage: "foo"}) + payload, err := proto.Marshal(pipepb.DockerPayload_builder{ContainerImage: "foo"}.Build()) if err != nil { t.Fatal(err) } p, err := graphx.Marshal(edges, - &graphx.Options{Environment: &pipepb.Environment{Urn: "beam:env:docker:v1", Payload: payload}, ContextReg: &creg}) + &graphx.Options{Environment: pipepb.Environment_builder{Urn: "beam:env:docker:v1", Payload: payload}.Build(), ContextReg: &creg}) if err != nil { t.Fatal(err) } @@ -317,19 +317,19 @@ func TestCreateEnvironment(t *testing.T) { name: "external", urn: graphx.URNEnvExternal, payload: func(name string) []byte { - return protox.MustEncode(&pipepb.ExternalPayload{ - Endpoint: &pipepb.ApiServiceDescriptor{ + return protox.MustEncode(pipepb.ExternalPayload_builder{ + Endpoint: pipepb.ApiServiceDescriptor_builder{ Url: name, - }, - }) + }.Build(), + }.Build()) }, }, { name: "docker", urn: graphx.URNEnvDocker, payload: func(name string) []byte { - return protox.MustEncode(&pipepb.DockerPayload{ + return protox.MustEncode(pipepb.DockerPayload_builder{ ContainerImage: name, - }) + }.Build()) }, }, } @@ -340,17 +340,17 @@ func TestCreateEnvironment(t *testing.T) { if err != nil { t.Errorf("CreateEnvironment(%v) = %v error, want nil", test.urn, err) } - want := &pipepb.Environment{ + want := pipepb.Environment_builder{ Urn: test.urn, Payload: test.payload(test.name), Dependencies: []*pipepb.ArtifactInformation{ - { + pipepb.ArtifactInformation_builder{ TypeUrn: graphx.URNArtifactFileType, TypePayload: protox.MustEncode(&pipepb.ArtifactFilePayload{}), RoleUrn: graphx.URNArtifactGoWorkerRole, - }, + }.Build(), }, - } + }.Build() opts := []cmp.Option{ protocmp.Transform(), // Ignore the capabilities field, since we can't access that method here. @@ -365,87 +365,87 @@ func TestCreateEnvironment(t *testing.T) { func TestUpdateDefaultEnvWorkerType(t *testing.T) { t.Run("noEnvs", func(t *testing.T) { - if err := graphx.UpdateDefaultEnvWorkerType("unused", nil, &pipepb.Pipeline{ + if err := graphx.UpdateDefaultEnvWorkerType("unused", nil, pipepb.Pipeline_builder{ Components: &pipepb.Components{}, - }); err == nil { + }.Build()); err == nil { t.Error("UpdateDefaultEnvWorkerType() no error, want err") } }) t.Run("noGoEnvs", func(t *testing.T) { - if err := graphx.UpdateDefaultEnvWorkerType("unused", nil, &pipepb.Pipeline{ - Components: &pipepb.Components{ + if err := graphx.UpdateDefaultEnvWorkerType("unused", nil, pipepb.Pipeline_builder{ + Components: pipepb.Components_builder{ Environments: map[string]*pipepb.Environment{ - "java": {Urn: "java"}, - "python": {Urn: "python"}, - "typescript": {Urn: "typescript"}, + "java": pipepb.Environment_builder{Urn: "java"}.Build(), + "python": pipepb.Environment_builder{Urn: "python"}.Build(), + "typescript": pipepb.Environment_builder{Urn: "typescript"}.Build(), }, - }, - }); err == nil { + }.Build(), + }.Build()); err == nil { t.Error("UpdateDefaultEnvWorkerType() no error, want err") } }) t.Run("badGoEnv", func(t *testing.T) { - if err := graphx.UpdateDefaultEnvWorkerType("unused", nil, &pipepb.Pipeline{ - Components: &pipepb.Components{ + if err := graphx.UpdateDefaultEnvWorkerType("unused", nil, pipepb.Pipeline_builder{ + Components: pipepb.Components_builder{ Environments: map[string]*pipepb.Environment{ - "java": {Urn: "java"}, - "python": {Urn: "python"}, - "typescript": {Urn: "typescript"}, - "go": { + "java": pipepb.Environment_builder{Urn: "java"}.Build(), + "python": pipepb.Environment_builder{Urn: "python"}.Build(), + "typescript": pipepb.Environment_builder{Urn: "typescript"}.Build(), + "go": pipepb.Environment_builder{ Urn: "test", Payload: []byte("test"), Dependencies: []*pipepb.ArtifactInformation{ - { + pipepb.ArtifactInformation_builder{ RoleUrn: "unset", - }, + }.Build(), }, - }, + }.Build(), }, - }, - }); err == nil { + }.Build(), + }.Build()); err == nil { t.Error("UpdateDefaultEnvWorkerType() no error, want err") } }) t.Run("goEnv", func(t *testing.T) { wantUrn := graphx.URNArtifactFileType - wantPyld := protox.MustEncode(&pipepb.ArtifactFilePayload{ + wantPyld := protox.MustEncode(pipepb.ArtifactFilePayload_builder{ Path: "good", - }) - p := &pipepb.Pipeline{ - Components: &pipepb.Components{ + }.Build()) + p := pipepb.Pipeline_builder{ + Components: pipepb.Components_builder{ Environments: map[string]*pipepb.Environment{ - "java": {Urn: "java"}, - "python": {Urn: "python"}, - "typescript": {Urn: "typescript"}, - "go": { + "java": pipepb.Environment_builder{Urn: "java"}.Build(), + "python": pipepb.Environment_builder{Urn: "python"}.Build(), + "typescript": pipepb.Environment_builder{Urn: "typescript"}.Build(), + "go": pipepb.Environment_builder{ Urn: "test", Payload: []byte("test"), Dependencies: []*pipepb.ArtifactInformation{ - { + pipepb.ArtifactInformation_builder{ TypeUrn: "to be removed", TypePayload: nil, RoleUrn: graphx.URNArtifactGoWorkerRole, - }, + }.Build(), }, - }, + }.Build(), }, - }, - } + }.Build(), + }.Build() if err := graphx.UpdateDefaultEnvWorkerType(wantUrn, wantPyld, p); err != nil { t.Errorf("UpdateDefaultEnvWorkerType() = %v, want nil", err) } got := p.GetComponents().GetEnvironments()["go"] - want := &pipepb.Environment{ + want := pipepb.Environment_builder{ Urn: "test", Payload: []byte("test"), Dependencies: []*pipepb.ArtifactInformation{ - { + pipepb.ArtifactInformation_builder{ TypeUrn: wantUrn, TypePayload: wantPyld, RoleUrn: graphx.URNArtifactGoWorkerRole, - }, + }.Build(), }, - } + }.Build() opts := []cmp.Option{ protocmp.Transform(), } diff --git a/sdks/go/pkg/beam/core/runtime/graphx/xlang.go b/sdks/go/pkg/beam/core/runtime/graphx/xlang.go index baee3c00093f..b02af765c372 100644 --- a/sdks/go/pkg/beam/core/runtime/graphx/xlang.go +++ b/sdks/go/pkg/beam/core/runtime/graphx/xlang.go @@ -34,7 +34,7 @@ func mergeExpandedWithPipeline(edges []*graph.MultiEdge, p *pipepb.Pipeline) { exp := e.External.Expanded id := fmt.Sprintf("e%v", e.ID()) - p.Requirements = append(p.Requirements, exp.Requirements...) + p.SetRequirements(append(p.GetRequirements(), exp.Requirements...)) // Adding components of the Expanded Transforms to the current Pipeline components, err := ExpandedComponents(exp) @@ -42,16 +42,16 @@ func mergeExpandedWithPipeline(edges []*graph.MultiEdge, p *pipepb.Pipeline) { panic(err) } for k, v := range components.GetTransforms() { - p.Components.Transforms[k] = v + p.GetComponents().GetTransforms()[k] = v } for k, v := range components.GetPcollections() { - p.Components.Pcollections[k] = v + p.GetComponents().GetPcollections()[k] = v } for k, v := range components.GetWindowingStrategies() { - p.Components.WindowingStrategies[k] = v + p.GetComponents().GetWindowingStrategies()[k] = v } for k, v := range components.GetCoders() { - p.Components.Coders[k] = v + p.GetComponents().GetCoders()[k] = v } for k, v := range components.GetEnvironments() { if k == defaultEnvId { @@ -61,14 +61,14 @@ func mergeExpandedWithPipeline(edges []*graph.MultiEdge, p *pipepb.Pipeline) { // using unique namespace prevents collision. continue } - p.Components.Environments[k] = v + p.GetComponents().GetEnvironments()[k] = v } transform, err := ExpandedTransform(exp) if err != nil { panic(err) } - p.Components.Transforms[id] = transform + p.GetComponents().GetTransforms()[id] = transform } } @@ -102,7 +102,7 @@ func purgeOutputInput(edges []*graph.MultiEdge, p *pipepb.Pipeline) { } idxMap[nodeID] = pcolID - delete(components.Pcollections, nodeID) + delete(components.GetPcollections(), nodeID) } } @@ -200,13 +200,13 @@ func AddFakeImpulses(p *pipepb.Pipeline) { key := fmt.Sprintf("%s_%s", "impulse", tag) output := map[string]string{"out": id} - impulse := &pipepb.PTransform{ + impulse := pipepb.PTransform_builder{ UniqueName: key, - Spec: &pipepb.FunctionSpec{ + Spec: pipepb.FunctionSpec_builder{ Urn: URNImpulse, - }, + }.Build(), Outputs: output, - } + }.Build() transforms[key] = impulse } diff --git a/sdks/go/pkg/beam/core/runtime/graphx/xlang_test.go b/sdks/go/pkg/beam/core/runtime/graphx/xlang_test.go index 7b68a7707682..283e49e691c4 100644 --- a/sdks/go/pkg/beam/core/runtime/graphx/xlang_test.go +++ b/sdks/go/pkg/beam/core/runtime/graphx/xlang_test.go @@ -129,17 +129,17 @@ func TestExternalOutputs(t *testing.T) { } func newTransform(name string) *pipepb.PTransform { - return &pipepb.PTransform{ + return pipepb.PTransform_builder{ UniqueName: name, - } + }.Build() } func newComponents(ts []string) *pipepb.Components { components := &pipepb.Components{} - components.Transforms = make(map[string]*pipepb.PTransform) + components.SetTransforms(make(map[string]*pipepb.PTransform)) for id, t := range ts { - components.Transforms[fmt.Sprint(id)] = newTransform(t) + components.GetTransforms()[fmt.Sprint(id)] = newTransform(t) } return components @@ -205,23 +205,23 @@ var testExternal = graph.ExternalTransform{ Expanded: &testExpanded, } -var testComponents = pipepb.Components{ - Transforms: map[string]*pipepb.PTransform{"transform_id1": {UniqueName: "test_components_transform"}}, - Pcollections: map[string]*pipepb.PCollection{"pcollection_id1": {UniqueName: "test_components_pcollection"}}, - WindowingStrategies: map[string]*pipepb.WindowingStrategy{"windowing_id1": {WindowCoderId: "test_components_windowing"}}, - Coders: map[string]*pipepb.Coder{"coder_id1": {Spec: &pipepb.FunctionSpec{Urn: "test_components_coder"}}}, - Environments: map[string]*pipepb.Environment{"environment_id1": {Urn: "test_components_environment"}}, -} +var testComponents = pipepb.Components_builder{ + Transforms: map[string]*pipepb.PTransform{"transform_id1": pipepb.PTransform_builder{UniqueName: "test_components_transform"}.Build()}, + Pcollections: map[string]*pipepb.PCollection{"pcollection_id1": pipepb.PCollection_builder{UniqueName: "test_components_pcollection"}.Build()}, + WindowingStrategies: map[string]*pipepb.WindowingStrategy{"windowing_id1": pipepb.WindowingStrategy_builder{WindowCoderId: "test_components_windowing"}.Build()}, + Coders: map[string]*pipepb.Coder{"coder_id1": pipepb.Coder_builder{Spec: pipepb.FunctionSpec_builder{Urn: "test_components_coder"}.Build()}.Build()}, + Environments: map[string]*pipepb.Environment{"environment_id1": pipepb.Environment_builder{Urn: "test_components_environment"}.Build()}, +}.Build() var testRequirements = []string{"test_requirement1", "test_requirement2"} -var testTransform = pipepb.PTransform{ +var testTransform = pipepb.PTransform_builder{ UniqueName: "test_transform", -} +}.Build() var testExpanded = graph.ExpandedTransform{ - Components: &testComponents, - Transform: &testTransform, + Components: testComponents, + Transform: testTransform, Requirements: testRequirements, } @@ -232,23 +232,23 @@ var testExternal2 = graph.ExternalTransform{ Expanded: &testExpanded2, } -var testComponents2 = pipepb.Components{ - Transforms: map[string]*pipepb.PTransform{"transform_id2": {UniqueName: "test_components2_transform"}}, - Pcollections: map[string]*pipepb.PCollection{"pcollection_id2": {UniqueName: "test_components2_pcollection"}}, - WindowingStrategies: map[string]*pipepb.WindowingStrategy{"windowing_id2": {WindowCoderId: "test_components2_windowing"}}, - Coders: map[string]*pipepb.Coder{"coder_id2": {Spec: &pipepb.FunctionSpec{Urn: "test_components2_coder"}}}, - Environments: map[string]*pipepb.Environment{"environment_id2": {Urn: "test_components2_environment"}}, -} +var testComponents2 = pipepb.Components_builder{ + Transforms: map[string]*pipepb.PTransform{"transform_id2": pipepb.PTransform_builder{UniqueName: "test_components2_transform"}.Build()}, + Pcollections: map[string]*pipepb.PCollection{"pcollection_id2": pipepb.PCollection_builder{UniqueName: "test_components2_pcollection"}.Build()}, + WindowingStrategies: map[string]*pipepb.WindowingStrategy{"windowing_id2": pipepb.WindowingStrategy_builder{WindowCoderId: "test_components2_windowing"}.Build()}, + Coders: map[string]*pipepb.Coder{"coder_id2": pipepb.Coder_builder{Spec: pipepb.FunctionSpec_builder{Urn: "test_components2_coder"}.Build()}.Build()}, + Environments: map[string]*pipepb.Environment{"environment_id2": pipepb.Environment_builder{Urn: "test_components2_environment"}.Build()}, +}.Build() var testRequirements2 = []string{"test_requirement2", "test_requirement3"} -var testTransform2 = pipepb.PTransform{ +var testTransform2 = pipepb.PTransform_builder{ UniqueName: "test_transform2", -} +}.Build() var testExpanded2 = graph.ExpandedTransform{ - Components: &testComponents2, - Transform: &testTransform2, + Components: testComponents2, + Transform: testTransform2, Requirements: testRequirements2, } @@ -314,13 +314,13 @@ func TestMergeExpandedWithPipeline(t *testing.T) { g := graph.New() edges := test.makeEdges(g) var p pipepb.Pipeline - p.Components = &pipepb.Components{ + p.SetComponents(pipepb.Components_builder{ Transforms: make(map[string]*pipepb.PTransform), Pcollections: make(map[string]*pipepb.PCollection), WindowingStrategies: make(map[string]*pipepb.WindowingStrategy), Coders: make(map[string]*pipepb.Coder), Environments: make(map[string]*pipepb.Environment), - } + }.Build()) mergeExpandedWithPipeline(edges, &p) // Check that all wanted expanded components have been added to @@ -371,7 +371,7 @@ func validateComponents(t *testing.T, wantComps, gotComps *pipepb.Components) { // Transforms for k, wantVal := range wantComps.GetTransforms() { - gotVal, ok := gotComps.Transforms[k] + gotVal, ok := gotComps.GetTransforms()[k] if !ok { t.Errorf("Pipeline components missing expected transform with key \"%v\": %v", k, wantVal) } else if d := cmp.Diff(wantVal, gotVal, protocmp.Transform()); d != "" { @@ -380,7 +380,7 @@ func validateComponents(t *testing.T, wantComps, gotComps *pipepb.Components) { } // PCollections for k, wantVal := range wantComps.GetPcollections() { - gotVal, ok := gotComps.Pcollections[k] + gotVal, ok := gotComps.GetPcollections()[k] if !ok { t.Errorf("Pipeline components missing expected PCollection with key \"%v\": %v", k, wantVal) } else if d := cmp.Diff(wantVal, gotVal, protocmp.Transform()); d != "" { @@ -389,7 +389,7 @@ func validateComponents(t *testing.T, wantComps, gotComps *pipepb.Components) { } // Windowing Strategies for k, wantVal := range wantComps.GetWindowingStrategies() { - gotVal, ok := gotComps.WindowingStrategies[k] + gotVal, ok := gotComps.GetWindowingStrategies()[k] if !ok { t.Errorf("Pipeline components missing expected windowing strategy with key \"%v\": %v", k, wantVal) } else if d := cmp.Diff(wantVal, gotVal, protocmp.Transform()); d != "" { @@ -398,7 +398,7 @@ func validateComponents(t *testing.T, wantComps, gotComps *pipepb.Components) { } // Coders for k, wantVal := range wantComps.GetCoders() { - gotVal, ok := gotComps.Coders[k] + gotVal, ok := gotComps.GetCoders()[k] if !ok { t.Errorf("Pipeline components missing expected coder with key \"%v\": %v", k, wantVal) } else if d := cmp.Diff(wantVal, gotVal, protocmp.Transform()); d != "" { @@ -407,7 +407,7 @@ func validateComponents(t *testing.T, wantComps, gotComps *pipepb.Components) { } // Environments for k, wantVal := range wantComps.GetEnvironments() { - gotVal, ok := gotComps.Environments[k] + gotVal, ok := gotComps.GetEnvironments()[k] if !ok { t.Errorf("Pipeline components missing expected environment with key \"%v\": %v", k, wantVal) } else if d := cmp.Diff(wantVal, gotVal, protocmp.Transform()); d != "" { diff --git a/sdks/go/pkg/beam/core/runtime/harness/datamgr.go b/sdks/go/pkg/beam/core/runtime/harness/datamgr.go index 6385098d013c..dc7733a31118 100644 --- a/sdks/go/pkg/beam/core/runtime/harness/datamgr.go +++ b/sdks/go/pkg/beam/core/runtime/harness/datamgr.go @@ -594,17 +594,17 @@ func (w *dataWriter) Close() error { w.ch.mu.Lock() defer w.ch.mu.Unlock() delete(w.ch.writers[w.id.instID], w.id.ptransformID) - msg := &fnpb.Elements{ + msg := fnpb.Elements_builder{ Data: []*fnpb.Elements_Data{ - { + fnpb.Elements_Data_builder{ InstructionId: string(w.id.instID), TransformId: w.id.ptransformID, // TODO(https://github.com/apache/beam/issues/21164): Set IsLast true on final flush instead of w/empty sentinel? // Empty data == sentinel IsLast: true, - }, + }.Build(), }, - } + }.Build() return w.send(msg) } @@ -617,15 +617,15 @@ func (w *dataWriter) Flush() error { w.ch.mu.Lock() defer w.ch.mu.Unlock() - msg := &fnpb.Elements{ + msg := fnpb.Elements_builder{ Data: []*fnpb.Elements_Data{ - { + fnpb.Elements_Data_builder{ InstructionId: string(w.id.instID), TransformId: w.id.ptransformID, Data: w.buf, - }, + }.Build(), }, - } + }.Build() if l := len(w.buf); l > largeBufferNotificationThreshold { log.Infof(context.TODO(), "dataWriter[%v;%v].Flush flushed large buffer of length %d", w.id, w.ch.id, l) } @@ -707,16 +707,16 @@ func (w *timerWriter) Close() error { w.ch.mu.Lock() defer w.ch.mu.Unlock() delete(w.ch.timerWriters[w.id.instID], timerKey{w.id.ptransformID, w.timerFamilyID}) - msg := &fnpb.Elements{ + msg := fnpb.Elements_builder{ Timers: []*fnpb.Elements_Timers{ - { + fnpb.Elements_Timers_builder{ InstructionId: string(w.id.instID), TransformId: w.id.ptransformID, TimerFamilyId: w.timerFamilyID, IsLast: true, - }, + }.Build(), }, - } + }.Build() return w.send(msg) } @@ -724,16 +724,16 @@ func (w *timerWriter) writeTimers(p []byte) error { w.ch.mu.Lock() defer w.ch.mu.Unlock() - msg := &fnpb.Elements{ + msg := fnpb.Elements_builder{ Timers: []*fnpb.Elements_Timers{ - { + fnpb.Elements_Timers_builder{ InstructionId: string(w.id.instID), TransformId: w.id.ptransformID, TimerFamilyId: w.timerFamilyID, Timers: p, - }, + }.Build(), }, - } + }.Build() return w.send(msg) } diff --git a/sdks/go/pkg/beam/core/runtime/harness/datamgr_test.go b/sdks/go/pkg/beam/core/runtime/harness/datamgr_test.go index 9f6f8a986a3f..dc197e00fd0a 100644 --- a/sdks/go/pkg/beam/core/runtime/harness/datamgr_test.go +++ b/sdks/go/pkg/beam/core/runtime/harness/datamgr_test.go @@ -52,20 +52,20 @@ func (f *fakeDataClient) Recv() (*fnpb.Elements, error) { f.calls++ data := []byte{1, 2, 3, 4, 1, 2, 3, 4} - elemData := fnpb.Elements_Data{ + elemData := fnpb.Elements_Data_builder{ InstructionId: "inst_ref", Data: data, TransformId: "ptr", - } + }.Build() if f.isLastCall == f.calls { - elemData.IsLast = true + elemData.SetIsLast(true) } - msg := fnpb.Elements{} + msg := &fnpb.Elements{} // Send extraData more than the number of elements buffered in the channel. for i := 0; i < bufElements+extraData; i++ { - msg.Data = append(msg.Data, &elemData) + msg.SetData(append(msg.GetData(), elemData)) } // The first two calls fill up the buffer completely to stimulate the deadlock @@ -73,16 +73,16 @@ func (f *fakeDataClient) Recv() (*fnpb.Elements, error) { // Subsequent calls return no data. switch f.calls { case 1: - return &msg, f.err + return msg, f.err case 2: - return &msg, f.err + return msg, f.err case 3: - elemData.Data = []byte{} - msg.Data = []*fnpb.Elements_Data{&elemData} + elemData.SetData([]byte{}) + msg.SetData([]*fnpb.Elements_Data{elemData}) // Broadcasting done here means that this code providing messages // has not been blocked by the bug blocking the dataReader // from getting more messages. - return &msg, f.err + return msg, f.err default: f.done <- true return nil, io.EOF @@ -165,16 +165,16 @@ func TestElementChan(t *testing.T) { } timerElm := func(val byte, isLast bool) *fnpb.Elements_Timers { - return &fnpb.Elements_Timers{InstructionId: instID, TransformId: timerID, Timers: []byte{val}, IsLast: isLast, TimerFamilyId: timerFamily} + return fnpb.Elements_Timers_builder{InstructionId: instID, TransformId: timerID, Timers: []byte{val}, IsLast: isLast, TimerFamilyId: timerFamily}.Build() } dataElm := func(val byte, isLast bool) *fnpb.Elements_Data { - return &fnpb.Elements_Data{InstructionId: instID, TransformId: dataID, Data: []byte{val}, IsLast: isLast} + return fnpb.Elements_Data_builder{InstructionId: instID, TransformId: dataID, Data: []byte{val}, IsLast: isLast}.Build() } noTimerElm := func() *fnpb.Elements_Timers { - return &fnpb.Elements_Timers{InstructionId: instID, TransformId: timerID, Timers: []byte{}, IsLast: true} + return fnpb.Elements_Timers_builder{InstructionId: instID, TransformId: timerID, Timers: []byte{}, IsLast: true}.Build() } noDataElm := func() *fnpb.Elements_Data { - return &fnpb.Elements_Data{InstructionId: instID, TransformId: dataID, Data: []byte{}, IsLast: true} + return fnpb.Elements_Data_builder{InstructionId: instID, TransformId: dataID, Data: []byte{}, IsLast: true}.Build() } openChan := func(ctx context.Context, t *testing.T, c *DataChannel, timers ...string) <-chan exec.Elements { t.Helper() @@ -196,13 +196,13 @@ func TestElementChan(t *testing.T) { name: "ReadThenData_singleRecv", sequenceFn: func(ctx context.Context, t *testing.T, client *fakeChanClient, c *DataChannel) <-chan exec.Elements { elms := openChan(ctx, t, c) - client.Send(&fnpb.Elements{ + client.Send(fnpb.Elements_builder{ Data: []*fnpb.Elements_Data{ dataElm(1, false), dataElm(2, false), dataElm(3, true), }, - }) + }.Build()) return elms }, wantSum: 6, wantCount: 3, @@ -210,9 +210,9 @@ func TestElementChan(t *testing.T) { name: "ReadThenData_multipleRecv", sequenceFn: func(ctx context.Context, t *testing.T, client *fakeChanClient, c *DataChannel) <-chan exec.Elements { elms := openChan(ctx, t, c) - client.Send(&fnpb.Elements{Data: []*fnpb.Elements_Data{dataElm(1, false)}}) - client.Send(&fnpb.Elements{Data: []*fnpb.Elements_Data{dataElm(2, false)}}) - client.Send(&fnpb.Elements{Data: []*fnpb.Elements_Data{dataElm(3, true)}}) + client.Send(fnpb.Elements_builder{Data: []*fnpb.Elements_Data{dataElm(1, false)}}.Build()) + client.Send(fnpb.Elements_builder{Data: []*fnpb.Elements_Data{dataElm(2, false)}}.Build()) + client.Send(fnpb.Elements_builder{Data: []*fnpb.Elements_Data{dataElm(3, true)}}.Build()) return elms }, wantSum: 6, wantCount: 3, @@ -220,14 +220,14 @@ func TestElementChan(t *testing.T) { name: "ReadThenNoData", sequenceFn: func(ctx context.Context, t *testing.T, client *fakeChanClient, c *DataChannel) <-chan exec.Elements { elms := openChan(ctx, t, c) - client.Send(&fnpb.Elements{Data: []*fnpb.Elements_Data{noDataElm()}}) + client.Send(fnpb.Elements_builder{Data: []*fnpb.Elements_Data{noDataElm()}}.Build()) return elms }, wantSum: 0, wantCount: 0, }, { name: "NoDataThenRead", sequenceFn: func(ctx context.Context, t *testing.T, client *fakeChanClient, c *DataChannel) <-chan exec.Elements { - client.Send(&fnpb.Elements{Data: []*fnpb.Elements_Data{noDataElm()}}) + client.Send(fnpb.Elements_builder{Data: []*fnpb.Elements_Data{noDataElm()}}.Build()) elms := openChan(ctx, t, c) return elms }, @@ -235,7 +235,7 @@ func TestElementChan(t *testing.T) { }, { name: "NoDataInstEndsThenRead", sequenceFn: func(ctx context.Context, t *testing.T, client *fakeChanClient, c *DataChannel) <-chan exec.Elements { - client.Send(&fnpb.Elements{Data: []*fnpb.Elements_Data{noDataElm()}}) + client.Send(fnpb.Elements_builder{Data: []*fnpb.Elements_Data{noDataElm()}}.Build()) c.removeInstruction(instID) elms := openChan(ctx, t, c) return elms @@ -245,18 +245,18 @@ func TestElementChan(t *testing.T) { name: "ReadThenDataAndTimers", sequenceFn: func(ctx context.Context, t *testing.T, client *fakeChanClient, c *DataChannel) <-chan exec.Elements { elms := openChan(ctx, t, c, timerID) - client.Send(&fnpb.Elements{Data: []*fnpb.Elements_Data{dataElm(1, false)}}) - client.Send(&fnpb.Elements{Timers: []*fnpb.Elements_Timers{timerElm(2, true)}}) - client.Send(&fnpb.Elements{Data: []*fnpb.Elements_Data{dataElm(3, true)}}) + client.Send(fnpb.Elements_builder{Data: []*fnpb.Elements_Data{dataElm(1, false)}}.Build()) + client.Send(fnpb.Elements_builder{Timers: []*fnpb.Elements_Timers{timerElm(2, true)}}.Build()) + client.Send(fnpb.Elements_builder{Data: []*fnpb.Elements_Data{dataElm(3, true)}}.Build()) return elms }, wantSum: 6, wantCount: 3, }, { name: "AllDataAndTimersThenRead", sequenceFn: func(ctx context.Context, t *testing.T, client *fakeChanClient, c *DataChannel) <-chan exec.Elements { - client.Send(&fnpb.Elements{Data: []*fnpb.Elements_Data{dataElm(1, false)}}) - client.Send(&fnpb.Elements{Timers: []*fnpb.Elements_Timers{timerElm(2, true)}}) - client.Send(&fnpb.Elements{Data: []*fnpb.Elements_Data{dataElm(3, true)}}) + client.Send(fnpb.Elements_builder{Data: []*fnpb.Elements_Data{dataElm(1, false)}}.Build()) + client.Send(fnpb.Elements_builder{Timers: []*fnpb.Elements_Timers{timerElm(2, true)}}.Build()) + client.Send(fnpb.Elements_builder{Data: []*fnpb.Elements_Data{dataElm(3, true)}}.Build()) elms := openChan(ctx, t, c, timerID) return elms }, @@ -264,30 +264,30 @@ func TestElementChan(t *testing.T) { }, { name: "DataThenReaderThenLast", sequenceFn: func(ctx context.Context, t *testing.T, client *fakeChanClient, c *DataChannel) <-chan exec.Elements { - client.Send(&fnpb.Elements{ + client.Send(fnpb.Elements_builder{ Data: []*fnpb.Elements_Data{ dataElm(1, false), dataElm(2, false), }, - }) + }.Build()) elms := openChan(ctx, t, c) - client.Send(&fnpb.Elements{Data: []*fnpb.Elements_Data{dataElm(3, true)}}) + client.Send(fnpb.Elements_builder{Data: []*fnpb.Elements_Data{dataElm(3, true)}}.Build()) return elms }, wantSum: 6, wantCount: 3, }, { name: "PartialTimersAllDataReadThenLastTimer", sequenceFn: func(ctx context.Context, t *testing.T, client *fakeChanClient, c *DataChannel) <-chan exec.Elements { - client.Send(&fnpb.Elements{ + client.Send(fnpb.Elements_builder{ Timers: []*fnpb.Elements_Timers{ timerElm(1, false), timerElm(2, false), }, - }) - client.Send(&fnpb.Elements{Data: []*fnpb.Elements_Data{noDataElm()}}) + }.Build()) + client.Send(fnpb.Elements_builder{Data: []*fnpb.Elements_Data{noDataElm()}}.Build()) elms := openChan(ctx, t, c, timerID) - client.Send(&fnpb.Elements{Timers: []*fnpb.Elements_Timers{timerElm(3, true)}}) + client.Send(fnpb.Elements_builder{Timers: []*fnpb.Elements_Timers{timerElm(3, true)}}.Build()) return elms }, @@ -295,16 +295,16 @@ func TestElementChan(t *testing.T) { }, { name: "AllTimerThenReaderThenDataClose", sequenceFn: func(ctx context.Context, t *testing.T, client *fakeChanClient, c *DataChannel) <-chan exec.Elements { - client.Send(&fnpb.Elements{ + client.Send(fnpb.Elements_builder{ Timers: []*fnpb.Elements_Timers{ timerElm(1, false), timerElm(2, false), timerElm(3, true), }, - }) + }.Build()) elms := openChan(ctx, t, c, timerID) - client.Send(&fnpb.Elements{Data: []*fnpb.Elements_Data{noDataElm()}}) + client.Send(fnpb.Elements_builder{Data: []*fnpb.Elements_Data{noDataElm()}}.Build()) return elms }, @@ -312,28 +312,28 @@ func TestElementChan(t *testing.T) { }, { name: "NoTimersThenReaderThenNoData", sequenceFn: func(ctx context.Context, t *testing.T, client *fakeChanClient, c *DataChannel) <-chan exec.Elements { - client.Send(&fnpb.Elements{Timers: []*fnpb.Elements_Timers{noTimerElm()}}) + client.Send(fnpb.Elements_builder{Timers: []*fnpb.Elements_Timers{noTimerElm()}}.Build()) elms := openChan(ctx, t, c, timerID) - client.Send(&fnpb.Elements{Data: []*fnpb.Elements_Data{noDataElm()}}) + client.Send(fnpb.Elements_builder{Data: []*fnpb.Elements_Data{noDataElm()}}.Build()) return elms }, wantSum: 0, wantCount: 0, }, { name: "SomeTimersThenReaderThenAData", sequenceFn: func(ctx context.Context, t *testing.T, client *fakeChanClient, c *DataChannel) <-chan exec.Elements { - client.Send(&fnpb.Elements{Timers: []*fnpb.Elements_Timers{timerElm(1, false), timerElm(2, true)}}) + client.Send(fnpb.Elements_builder{Timers: []*fnpb.Elements_Timers{timerElm(1, false), timerElm(2, true)}}.Build()) elms := openChan(ctx, t, c, timerID) - client.Send(&fnpb.Elements{Data: []*fnpb.Elements_Data{dataElm(3, true)}}) + client.Send(fnpb.Elements_builder{Data: []*fnpb.Elements_Data{dataElm(3, true)}}.Build()) return elms }, wantSum: 6, wantCount: 3, }, { name: "SomeTimersAndADataThenReader", sequenceFn: func(ctx context.Context, t *testing.T, client *fakeChanClient, c *DataChannel) <-chan exec.Elements { - client.Send(&fnpb.Elements{ + client.Send(fnpb.Elements_builder{ Timers: []*fnpb.Elements_Timers{timerElm(1, false), timerElm(2, true)}, Data: []*fnpb.Elements_Data{dataElm(3, true)}, - }) + }.Build()) elms := openChan(ctx, t, c, timerID) return elms }, @@ -341,12 +341,12 @@ func TestElementChan(t *testing.T) { }, { name: "PartialReadThenEndInstruction", sequenceFn: func(ctx context.Context, t *testing.T, client *fakeChanClient, c *DataChannel) <-chan exec.Elements { - client.Send(&fnpb.Elements{ + client.Send(fnpb.Elements_builder{ Data: []*fnpb.Elements_Data{ dataElm(1, false), dataElm(2, false), }, - }) + }.Build()) elms := openChan(ctx, t, c) var sum int e := <-elms @@ -362,12 +362,12 @@ func TestElementChan(t *testing.T) { c.removeInstruction(instID) // Instruction is ended, so further data for this instruction is ignored. - client.Send(&fnpb.Elements{ + client.Send(fnpb.Elements_builder{ Data: []*fnpb.Elements_Data{ dataElm(3, false), dataElm(4, true), }, - }) + }.Build()) elms = openChan(ctx, t, c) return elms @@ -410,12 +410,12 @@ func BenchmarkElementChan(b *testing.B) { if err != nil { b.Errorf("Unexpected error from OpenElementChan(%v, %v, nil): %v", dataID, instID, err) } - e := &fnpb.Elements_Data{InstructionId: instID, TransformId: dataID, Data: []byte{1}, IsLast: false} + e := fnpb.Elements_Data_builder{InstructionId: instID, TransformId: dataID, Data: []byte{1}, IsLast: false}.Build() es := make([]*fnpb.Elements_Data, 0, bench.size) for i := 0; i < bench.size; i++ { es = append(es, e) } - batch := &fnpb.Elements{Data: es} + batch := fnpb.Elements_builder{Data: es}.Build() var wg sync.WaitGroup wg.Add(1) go func() { diff --git a/sdks/go/pkg/beam/core/runtime/harness/harness.go b/sdks/go/pkg/beam/core/runtime/harness/harness.go index 20aad81c1237..2b32efc02eb2 100644 --- a/sdks/go/pkg/beam/core/runtime/harness/harness.go +++ b/sdks/go/pkg/beam/core/runtime/harness/harness.go @@ -40,6 +40,7 @@ import ( "github.com/apache/beam/sdks/v2/go/pkg/beam/util/grpcx" "golang.org/x/sync/singleflight" "google.golang.org/grpc" + "google.golang.org/protobuf/proto" "google.golang.org/protobuf/types/known/durationpb" "google.golang.org/protobuf/types/known/timestamppb" ) @@ -109,7 +110,7 @@ func MainWithOptions(ctx context.Context, loggingEndpoint, controlEndpoint strin client := fnpb.NewBeamFnControlClient(conn) lookupDesc := func(id bundleDescriptorID) (*fnpb.ProcessBundleDescriptor, error) { - return client.GetProcessBundleDescriptor(ctx, &fnpb.GetProcessBundleDescriptorRequest{ProcessBundleDescriptorId: string(id)}) + return client.GetProcessBundleDescriptor(ctx, fnpb.GetProcessBundleDescriptorRequest_builder{ProcessBundleDescriptorId: string(id)}.Build()) } stub, err := client.Control(ctx) @@ -374,12 +375,10 @@ func (c *control) handleInstruction(ctx context.Context, req *fnpb.InstructionRe } c.mu.Unlock() - return &fnpb.InstructionResponse{ + return fnpb.InstructionResponse_builder{ InstructionId: string(instID), - Response: &fnpb.InstructionResponse_Register{ - Register: &fnpb.RegisterResponse{}, - }, - } + Register: &fnpb.RegisterResponse{}, + }.Build() case req.GetProcessBundle() != nil: msg := req.GetProcessBundle() @@ -467,15 +466,15 @@ func (c *control) handleInstruction(ctx context.Context, req *fnpb.InstructionRe if len(checkpoints) > 0 { for _, cp := range checkpoints { for _, r := range cp.SR.RS { - rRoots = append(rRoots, &fnpb.DelayedBundleApplication{ - Application: &fnpb.BundleApplication{ + rRoots = append(rRoots, fnpb.DelayedBundleApplication_builder{ + Application: fnpb.BundleApplication_builder{ TransformId: cp.SR.TId, InputId: cp.SR.InId, Element: r, OutputWatermarks: cp.SR.OW, - }, + }.Build(), RequestedTimeDelay: durationpb.New(cp.Reapply), - }) + }.Build()) } } } @@ -491,17 +490,15 @@ func (c *control) handleInstruction(ctx context.Context, req *fnpb.InstructionRe if err != nil { return fail(ctx, instID, "process bundle failed for instruction %v using plan %v : %v", instID, bdID, err) } - return &fnpb.InstructionResponse{ + return fnpb.InstructionResponse_builder{ InstructionId: string(instID), - Response: &fnpb.InstructionResponse_ProcessBundle{ - ProcessBundle: &fnpb.ProcessBundleResponse{ - ResidualRoots: rRoots, - MonitoringData: pylds, - MonitoringInfos: mons, - RequiresFinalization: requiresFinalization, - }, - }, - } + ProcessBundle: fnpb.ProcessBundleResponse_builder{ + ResidualRoots: rRoots, + MonitoringData: pylds, + MonitoringInfos: mons, + RequiresFinalization: requiresFinalization, + }.Build(), + }.Build() case req.GetFinalizeBundle() != nil: msg := req.GetFinalizeBundle() @@ -521,12 +518,10 @@ func (c *control) handleInstruction(ctx context.Context, req *fnpb.InstructionRe c.plans[af.bdID] = append(c.plans[af.bdID], af.plan) delete(c.awaitingFinalization, ref) - return &fnpb.InstructionResponse{ - InstructionId: string(instID), - Response: &fnpb.InstructionResponse_FinalizeBundle{ - FinalizeBundle: &fnpb.FinalizeBundleResponse{}, - }, - } + return fnpb.InstructionResponse_builder{ + InstructionId: string(instID), + FinalizeBundle: &fnpb.FinalizeBundleResponse{}, + }.Build() case req.GetProcessBundleProgress() != nil: msg := req.GetProcessBundleProgress() @@ -538,26 +533,22 @@ func (c *control) handleInstruction(ctx context.Context, req *fnpb.InstructionRe return resp } if plan == nil && resp == nil { - return &fnpb.InstructionResponse{ - InstructionId: string(instID), - Response: &fnpb.InstructionResponse_ProcessBundleProgress{ - ProcessBundleProgress: &fnpb.ProcessBundleProgressResponse{}, - }, - } + return fnpb.InstructionResponse_builder{ + InstructionId: string(instID), + ProcessBundleProgress: &fnpb.ProcessBundleProgressResponse{}, + }.Build() } mons, pylds, consumingReceivedData := monitoring(plan, store, c.runnerCapabilities[URNMonitoringInfoShortID]) - return &fnpb.InstructionResponse{ + return fnpb.InstructionResponse_builder{ InstructionId: string(instID), - Response: &fnpb.InstructionResponse_ProcessBundleProgress{ - ProcessBundleProgress: &fnpb.ProcessBundleProgressResponse{ - MonitoringData: pylds, - MonitoringInfos: mons, - ConsumingReceivedData: &consumingReceivedData, - }, - }, - } + ProcessBundleProgress: fnpb.ProcessBundleProgressResponse_builder{ + MonitoringData: pylds, + MonitoringInfos: mons, + ConsumingReceivedData: &consumingReceivedData, + }.Build(), + }.Build() case req.GetProcessBundleSplit() != nil: msg := req.GetProcessBundleSplit() @@ -571,12 +562,10 @@ func (c *control) handleInstruction(ctx context.Context, req *fnpb.InstructionRe return resp } if plan == nil { - return &fnpb.InstructionResponse{ - InstructionId: string(instID), - Response: &fnpb.InstructionResponse_ProcessBundleSplit{ - ProcessBundleSplit: &fnpb.ProcessBundleSplitResponse{}, - }, - } + return fnpb.InstructionResponse_builder{ + InstructionId: string(instID), + ProcessBundleSplit: &fnpb.ProcessBundleSplitResponse{}, + }.Build() } // Get the desired splits for the root FnAPI read operation. @@ -597,12 +586,10 @@ func (c *control) handleInstruction(ctx context.Context, req *fnpb.InstructionRe // Unsuccessful splits without errors indicate we should return an empty response, // as processing can continue. if sr.Unsuccessful { - return &fnpb.InstructionResponse{ - InstructionId: string(instID), - Response: &fnpb.InstructionResponse_ProcessBundleSplit{ - ProcessBundleSplit: &fnpb.ProcessBundleSplitResponse{}, - }, - } + return fnpb.InstructionResponse_builder{ + InstructionId: string(instID), + ProcessBundleSplit: &fnpb.ProcessBundleSplitResponse{}, + }.Build() } var pRoots []*fnpb.BundleApplication @@ -610,59 +597,53 @@ func (c *control) handleInstruction(ctx context.Context, req *fnpb.InstructionRe if len(sr.PS) > 0 && len(sr.RS) > 0 { pRoots = make([]*fnpb.BundleApplication, len(sr.PS)) for i, p := range sr.PS { - pRoots[i] = &fnpb.BundleApplication{ + pRoots[i] = fnpb.BundleApplication_builder{ TransformId: sr.TId, InputId: sr.InId, Element: p, - } + }.Build() } rRoots = make([]*fnpb.DelayedBundleApplication, len(sr.RS)) for i, r := range sr.RS { - rRoots[i] = &fnpb.DelayedBundleApplication{ - Application: &fnpb.BundleApplication{ + rRoots[i] = fnpb.DelayedBundleApplication_builder{ + Application: fnpb.BundleApplication_builder{ TransformId: sr.TId, InputId: sr.InId, Element: r, OutputWatermarks: sr.OW, - }, - } + }.Build(), + }.Build() } } - return &fnpb.InstructionResponse{ + return fnpb.InstructionResponse_builder{ InstructionId: string(instID), - Response: &fnpb.InstructionResponse_ProcessBundleSplit{ - ProcessBundleSplit: &fnpb.ProcessBundleSplitResponse{ - ChannelSplits: []*fnpb.ProcessBundleSplitResponse_ChannelSplit{{ - TransformId: plan.SourcePTransformID(), - LastPrimaryElement: sr.PI, - FirstResidualElement: sr.RI, - }}, - PrimaryRoots: pRoots, - ResidualRoots: rRoots, - }, - }, - } + ProcessBundleSplit: fnpb.ProcessBundleSplitResponse_builder{ + ChannelSplits: []*fnpb.ProcessBundleSplitResponse_ChannelSplit{fnpb.ProcessBundleSplitResponse_ChannelSplit_builder{ + TransformId: plan.SourcePTransformID(), + LastPrimaryElement: sr.PI, + FirstResidualElement: sr.RI, + }.Build()}, + PrimaryRoots: pRoots, + ResidualRoots: rRoots, + }.Build(), + }.Build() case req.GetMonitoringInfos() != nil: msg := req.GetMonitoringInfos() - return &fnpb.InstructionResponse{ + return fnpb.InstructionResponse_builder{ InstructionId: string(instID), - Response: &fnpb.InstructionResponse_MonitoringInfos{ - MonitoringInfos: &fnpb.MonitoringInfosMetadataResponse{ - MonitoringInfo: shortIdsToInfos(msg.GetMonitoringInfoId()), - }, - }, - } + MonitoringInfos: fnpb.MonitoringInfosMetadataResponse_builder{ + MonitoringInfo: shortIdsToInfos(msg.GetMonitoringInfoId()), + }.Build(), + }.Build() case req.GetHarnessMonitoringInfos() != nil: - return &fnpb.InstructionResponse{ + return fnpb.InstructionResponse_builder{ InstructionId: string(instID), - Response: &fnpb.InstructionResponse_HarnessMonitoringInfos{ - HarnessMonitoringInfos: &fnpb.HarnessMonitoringInfosResponse{ - // TODO(BEAM-11092): Populate with non-bundle metrics data. - MonitoringData: map[string][]byte{}, - }, - }, - } + HarnessMonitoringInfos: fnpb.HarnessMonitoringInfosResponse_builder{ + // TODO(BEAM-11092): Populate with non-bundle metrics data. + MonitoringData: map[string][]byte{}, + }.Build(), + }.Build() case req.GetSampleData() != nil: msg := req.GetSampleData() var samples = make(map[string]*fnpb.SampleDataResponse_ElementList) @@ -671,22 +652,20 @@ func (c *control) handleInstruction(ctx context.Context, req *fnpb.InstructionRe for pid, elements := range elementsMap { var elementList fnpb.SampleDataResponse_ElementList for i := range elements { - var sampledElement = &fnpb.SampledElement{ + var sampledElement = fnpb.SampledElement_builder{ Element: elements[i].Element, SampleTimestamp: timestamppb.New(elements[i].Timestamp), - } - elementList.Elements = append(elementList.Elements, sampledElement) + }.Build() + elementList.SetElements(append(elementList.GetElements(), sampledElement)) } samples[pid] = &elementList } } - return &fnpb.InstructionResponse{ + return fnpb.InstructionResponse_builder{ InstructionId: string(instID), - Response: &fnpb.InstructionResponse_SampleData{ - SampleData: &fnpb.SampleDataResponse{ElementSamples: samples}, - }, - } + SampleData: fnpb.SampleDataResponse_builder{ElementSamples: samples}.Build(), + }.Build() default: return fail(ctx, instID, "Unexpected request: %v", req) } @@ -730,11 +709,11 @@ func fail(ctx context.Context, id instructionID, format string, args ...any) *fn log.Output(ctx, log.SevError, 1, fmt.Sprintf(format, args...)) dummy := &fnpb.InstructionResponse_Register{Register: &fnpb.RegisterResponse{}} - return &fnpb.InstructionResponse{ + return fnpb.InstructionResponse_builder{ InstructionId: string(id), Error: fmt.Sprintf(format, args...), - Response: dummy, - } + Register: proto.ValueOrDefault(dummy.Register), + }.Build() } // dial to the specified endpoint. if timeout <=0, call blocks until diff --git a/sdks/go/pkg/beam/core/runtime/harness/harness_test.go b/sdks/go/pkg/beam/core/runtime/harness/harness_test.go index 8c25db613eba..e52ca9b1c11f 100644 --- a/sdks/go/pkg/beam/core/runtime/harness/harness_test.go +++ b/sdks/go/pkg/beam/core/runtime/harness/harness_test.go @@ -29,62 +29,62 @@ import ( // validDescriptor describes a valid pipeline with a source and a sink, but doesn't do anything else. func validDescriptor(t *testing.T) *fnpb.ProcessBundleDescriptor { t.Helper() - port := &fnpb.RemoteGrpcPort{ + port := fnpb.RemoteGrpcPort_builder{ CoderId: "c1", - ApiServiceDescriptor: &pipepb.ApiServiceDescriptor{ + ApiServiceDescriptor: pipepb.ApiServiceDescriptor_builder{ Url: "hostname:port", - }, - } + }.Build(), + }.Build() portBytes, err := proto.Marshal(port) if err != nil { t.Fatalf("bad port: %v", err) } - return &fnpb.ProcessBundleDescriptor{ + return fnpb.ProcessBundleDescriptor_builder{ Id: "test", Transforms: map[string]*pipepb.PTransform{ - "source": &pipepb.PTransform{ - Spec: &pipepb.FunctionSpec{ + "source": pipepb.PTransform_builder{ + Spec: pipepb.FunctionSpec_builder{ Urn: "beam:runner:source:v1", Payload: portBytes, - }, + }.Build(), Outputs: map[string]string{ "o1": "p1", }, - }, - "sink": &pipepb.PTransform{ - Spec: &pipepb.FunctionSpec{ + }.Build(), + "sink": pipepb.PTransform_builder{ + Spec: pipepb.FunctionSpec_builder{ Urn: "beam:runner:sink:v1", Payload: portBytes, - }, + }.Build(), Inputs: map[string]string{ "i1": "p1", }, - }, + }.Build(), }, Pcollections: map[string]*pipepb.PCollection{ - "p1": &pipepb.PCollection{ + "p1": pipepb.PCollection_builder{ CoderId: "c1", - }, + }.Build(), }, Coders: map[string]*pipepb.Coder{ - "c1": &pipepb.Coder{ - Spec: &pipepb.FunctionSpec{ + "c1": pipepb.Coder_builder{ + Spec: pipepb.FunctionSpec_builder{ Urn: "beam:coder:windowed_value:v1", - }, + }.Build(), ComponentCoderIds: []string{"c2", "c3"}, - }, - "c2": &pipepb.Coder{ - Spec: &pipepb.FunctionSpec{ + }.Build(), + "c2": pipepb.Coder_builder{ + Spec: pipepb.FunctionSpec_builder{ Urn: "beam:coder:varint:v1", - }, - }, - "c3": &pipepb.Coder{ - Spec: &pipepb.FunctionSpec{ + }.Build(), + }.Build(), + "c3": pipepb.Coder_builder{ + Spec: pipepb.FunctionSpec_builder{ Urn: "beam:coder:global_window:v1", - }, - }, + }.Build(), + }.Build(), }, - } + }.Build() } diff --git a/sdks/go/pkg/beam/core/runtime/harness/logging.go b/sdks/go/pkg/beam/core/runtime/harness/logging.go index 20fe76608280..f5d6d18b6225 100644 --- a/sdks/go/pkg/beam/core/runtime/harness/logging.go +++ b/sdks/go/pkg/beam/core/runtime/harness/logging.go @@ -38,16 +38,16 @@ type logger struct { func (l *logger) Log(ctx context.Context, sev log.Severity, calldepth int, msg string) { now := timestamppb.New(time.Now()) - entry := &fnpb.LogEntry{ + entry := fnpb.LogEntry_builder{ Timestamp: now, Severity: convertSeverity(sev), Message: msg, - } + }.Build() if _, file, line, ok := runtime.Caller(calldepth + 1); ok { - entry.LogLocation = fmt.Sprintf("%v:%v", file, line) + entry.SetLogLocation(fmt.Sprintf("%v:%v", file, line)) } - entry.InstructionId = metrics.GetBundleID(ctx) - entry.TransformId = metrics.GetTransformID(ctx) + entry.SetInstructionId(metrics.GetBundleID(ctx)) + entry.SetTransformId(metrics.GetTransformID(ctx)) select { case l.out <- entry: @@ -174,7 +174,7 @@ func (w *remoteWriter) connect(ctx context.Context, makeClient func(ctx context. return errBuffClosed } msgs = append(msgs, newMsg) - flush = newMsg.Severity >= fnpb.LogEntry_Severity_CRITICAL + flush = newMsg.GetSeverity() >= fnpb.LogEntry_Severity_CRITICAL } // If there are still messages in the buffer, drain them out to batch them to a cap. @@ -185,12 +185,12 @@ func (w *remoteWriter) connect(ctx context.Context, makeClient func(ctx context. return errBuffClosed } msgs = append(msgs, newMsg) - flush = newMsg.Severity >= fnpb.LogEntry_Severity_CRITICAL + flush = newMsg.GetSeverity() >= fnpb.LogEntry_Severity_CRITICAL } - list := &fnpb.LogEntry_List{ + list := fnpb.LogEntry_List_builder{ LogEntries: msgs, - } + }.Build() if err := client.Send(list); err != nil { if err == io.EOF { diff --git a/sdks/go/pkg/beam/core/runtime/harness/logging_test.go b/sdks/go/pkg/beam/core/runtime/harness/logging_test.go index b2383111a4de..0087bb6cbf0d 100644 --- a/sdks/go/pkg/beam/core/runtime/harness/logging_test.go +++ b/sdks/go/pkg/beam/core/runtime/harness/logging_test.go @@ -103,7 +103,7 @@ func TestLogger_connect(t *testing.T) { var count int for _, batch := range cacher.logs { - count += len(batch.LogEntries) + count += len(batch.GetLogEntries()) } if got, want := count, 10; got != want { diff --git a/sdks/go/pkg/beam/core/runtime/harness/monitoring.go b/sdks/go/pkg/beam/core/runtime/harness/monitoring.go index 08b875147325..ab5c091c3dbe 100644 --- a/sdks/go/pkg/beam/core/runtime/harness/monitoring.go +++ b/sdks/go/pkg/beam/core/runtime/harness/monitoring.go @@ -69,11 +69,11 @@ func (c *shortIDCache) getShortID(l metrics.Labels, urn metricsx.Urn) string { } s = c.getNextShortID() c.labels2ShortIds[k] = s - c.shortIds2Infos[s] = &pipepb.MonitoringInfo{ + c.shortIds2Infos[s] = pipepb.MonitoringInfo_builder{ Urn: metricsx.UrnToString(urn), Type: metricsx.UrnToType(urn), Labels: l.Map(), - } + }.Build() return s } @@ -120,12 +120,12 @@ func monitoring(p *exec.Plan, store *metrics.Store, supportShortID bool) ([]*pip payloads[getShortID(l, metricsx.UrnUserSumInt64)] = payload if !supportShortID { monitoringInfo = append(monitoringInfo, - &pipepb.MonitoringInfo{ + pipepb.MonitoringInfo_builder{ Urn: metricsx.UrnToString(metricsx.UrnUserSumInt64), Type: metricsx.UrnToType(metricsx.UrnUserSumInt64), Labels: l.Map(), Payload: payload, - }) + }.Build()) } }, DistributionInt64: func(l metrics.Labels, count, sum, min, max int64) { @@ -136,12 +136,12 @@ func monitoring(p *exec.Plan, store *metrics.Store, supportShortID bool) ([]*pip payloads[getShortID(l, metricsx.UrnUserDistInt64)] = payload if !supportShortID { monitoringInfo = append(monitoringInfo, - &pipepb.MonitoringInfo{ + pipepb.MonitoringInfo_builder{ Urn: metricsx.UrnToString(metricsx.UrnUserDistInt64), Type: metricsx.UrnToType(metricsx.UrnUserDistInt64), Labels: l.Map(), Payload: payload, - }) + }.Build()) } }, GaugeInt64: func(l metrics.Labels, v int64, t time.Time) { @@ -152,12 +152,12 @@ func monitoring(p *exec.Plan, store *metrics.Store, supportShortID bool) ([]*pip payloads[getShortID(l, metricsx.UrnUserLatestMsInt64)] = payload if !supportShortID { monitoringInfo = append(monitoringInfo, - &pipepb.MonitoringInfo{ + pipepb.MonitoringInfo_builder{ Urn: metricsx.UrnToString(metricsx.UrnUserLatestMsInt64), Type: metricsx.UrnToType(metricsx.UrnUserLatestMsInt64), Labels: l.Map(), Payload: payload, - }) + }.Build()) } }, MsecsInt64: func(l string, states *[4]metrics.ExecutionState) { @@ -171,12 +171,12 @@ func monitoring(p *exec.Plan, store *metrics.Store, supportShortID bool) ([]*pip payloads[getShortID(metrics.PTransformLabels(l), ul)] = payload if !supportShortID { monitoringInfo = append(monitoringInfo, - &pipepb.MonitoringInfo{ + pipepb.MonitoringInfo_builder{ Urn: metricsx.UrnToString(ul), Type: metricsx.UrnToType(ul), Labels: label, Payload: payload, - }) + }.Build()) } } }, @@ -199,14 +199,14 @@ func monitoring(p *exec.Plan, store *metrics.Store, supportShortID bool) ([]*pip if !supportShortID { monitoringInfo = append(monitoringInfo, - &pipepb.MonitoringInfo{ + pipepb.MonitoringInfo_builder{ Urn: metricsx.UrnToString(metricsx.UrnElementCount), Type: metricsx.UrnToType(metricsx.UrnElementCount), Labels: map[string]string{ "PCOLLECTION": pcol.ID, }, Payload: payload, - }) + }.Build()) } // Skip pcollections without size if pcol.SizeCount != 0 { @@ -218,14 +218,14 @@ func monitoring(p *exec.Plan, store *metrics.Store, supportShortID bool) ([]*pip if !supportShortID { monitoringInfo = append(monitoringInfo, - &pipepb.MonitoringInfo{ + pipepb.MonitoringInfo_builder{ Urn: metricsx.UrnToString(metricsx.UrnSampledByteSize), Type: metricsx.UrnToType(metricsx.UrnSampledByteSize), Labels: map[string]string{ "PCOLLECTION": pcol.ID, }, Payload: payload, - }) + }.Build()) } } } @@ -238,14 +238,14 @@ func monitoring(p *exec.Plan, store *metrics.Store, supportShortID bool) ([]*pip payloads[getShortID(metrics.PTransformLabels(snapshot.Source.ID), metricsx.UrnDataChannelReadIndex)] = payload if !supportShortID { monitoringInfo = append(monitoringInfo, - &pipepb.MonitoringInfo{ + pipepb.MonitoringInfo_builder{ Urn: metricsx.UrnToString(metricsx.UrnDataChannelReadIndex), Type: metricsx.UrnToType(metricsx.UrnDataChannelReadIndex), Labels: map[string]string{ "PTRANSFORM": snapshot.Source.ID, }, Payload: payload, - }) + }.Build()) } return monitoringInfo, payloads, snapshot.Source.ConsumingReceivedData } diff --git a/sdks/go/pkg/beam/core/runtime/harness/statecache/statecache_test.go b/sdks/go/pkg/beam/core/runtime/harness/statecache/statecache_test.go index 23f6fbf2461d..7535ad4d5c31 100644 --- a/sdks/go/pkg/beam/core/runtime/harness/statecache/statecache_test.go +++ b/sdks/go/pkg/beam/core/runtime/harness/statecache/statecache_test.go @@ -122,15 +122,13 @@ func TestSetCache_CacheableCase(t *testing.T) { } func makeRequest(transformID, sideInputID string, t token) *fnpb.ProcessBundleRequest_CacheToken { - return &fnpb.ProcessBundleRequest_CacheToken{ + return fnpb.ProcessBundleRequest_CacheToken_builder{ Token: []byte(t), - Type: &fnpb.ProcessBundleRequest_CacheToken_SideInput_{ - SideInput: &fnpb.ProcessBundleRequest_CacheToken_SideInput{ - TransformId: transformID, - SideInputId: sideInputID, - }, - }, - } + SideInput: fnpb.ProcessBundleRequest_CacheToken_SideInput_builder{ + TransformId: transformID, + SideInputId: sideInputID, + }.Build(), + }.Build() } func makeCacheToken(transformID, sideInputID string, tok token) cacheToken { diff --git a/sdks/go/pkg/beam/core/runtime/harness/statemgr.go b/sdks/go/pkg/beam/core/runtime/harness/statemgr.go index 4c1bc0b55fe3..e082d9806de7 100644 --- a/sdks/go/pkg/beam/core/runtime/harness/statemgr.go +++ b/sdks/go/pkg/beam/core/runtime/harness/statemgr.go @@ -228,16 +228,14 @@ type stateKeyWriter struct { } func newSideInputReader(ch *StateChannel, id exec.StreamID, sideInputID string, instID instructionID, k, w []byte) *stateKeyReader { - key := &fnpb.StateKey{ - Type: &fnpb.StateKey_MultimapSideInput_{ - MultimapSideInput: &fnpb.StateKey_MultimapSideInput{ - TransformId: id.PtransformID, - SideInputId: sideInputID, - Window: w, - Key: k, - }, - }, - } + key := fnpb.StateKey_builder{ + MultimapSideInput: fnpb.StateKey_MultimapSideInput_builder{ + TransformId: id.PtransformID, + SideInputId: sideInputID, + Window: w, + Key: k, + }.Build(), + }.Build() return &stateKeyReader{ instID: instID, key: key, @@ -246,15 +244,13 @@ func newSideInputReader(ch *StateChannel, id exec.StreamID, sideInputID string, } func newIterableSideInputReader(ch *StateChannel, id exec.StreamID, sideInputID string, instID instructionID, w []byte) *stateKeyReader { - key := &fnpb.StateKey{ - Type: &fnpb.StateKey_IterableSideInput_{ - IterableSideInput: &fnpb.StateKey_IterableSideInput{ - TransformId: id.PtransformID, - SideInputId: sideInputID, - Window: w, - }, - }, - } + key := fnpb.StateKey_builder{ + IterableSideInput: fnpb.StateKey_IterableSideInput_builder{ + TransformId: id.PtransformID, + SideInputId: sideInputID, + Window: w, + }.Build(), + }.Build() return &stateKeyReader{ instID: instID, @@ -264,13 +260,11 @@ func newIterableSideInputReader(ch *StateChannel, id exec.StreamID, sideInputID } func newRunnerReader(ch *StateChannel, instID instructionID, k []byte) *stateKeyReader { - key := &fnpb.StateKey{ - Type: &fnpb.StateKey_Runner_{ - Runner: &fnpb.StateKey_Runner{ - Key: k, - }, - }, - } + key := fnpb.StateKey_builder{ + Runner: fnpb.StateKey_Runner_builder{ + Key: k, + }.Build(), + }.Build() return &stateKeyReader{ instID: instID, key: key, @@ -279,16 +273,14 @@ func newRunnerReader(ch *StateChannel, instID instructionID, k []byte) *stateKey } func newBagUserStateReader(ch *StateChannel, id exec.StreamID, instID instructionID, userStateID string, k []byte, w []byte) *stateKeyReader { - key := &fnpb.StateKey{ - Type: &fnpb.StateKey_BagUserState_{ - BagUserState: &fnpb.StateKey_BagUserState{ - TransformId: id.PtransformID, - UserStateId: userStateID, - Window: w, - Key: k, - }, - }, - } + key := fnpb.StateKey_builder{ + BagUserState: fnpb.StateKey_BagUserState_builder{ + TransformId: id.PtransformID, + UserStateId: userStateID, + Window: w, + Key: k, + }.Build(), + }.Build() return &stateKeyReader{ instID: instID, key: key, @@ -297,16 +289,14 @@ func newBagUserStateReader(ch *StateChannel, id exec.StreamID, instID instructio } func newBagUserStateWriter(ch *StateChannel, id exec.StreamID, instID instructionID, userStateID string, k []byte, w []byte, wt writeTypeEnum) *stateKeyWriter { - key := &fnpb.StateKey{ - Type: &fnpb.StateKey_BagUserState_{ - BagUserState: &fnpb.StateKey_BagUserState{ - TransformId: id.PtransformID, - UserStateId: userStateID, - Window: w, - Key: k, - }, - }, - } + key := fnpb.StateKey_builder{ + BagUserState: fnpb.StateKey_BagUserState_builder{ + TransformId: id.PtransformID, + UserStateId: userStateID, + Window: w, + Key: k, + }.Build(), + }.Build() return &stateKeyWriter{ instID: instID, key: key, @@ -316,17 +306,15 @@ func newBagUserStateWriter(ch *StateChannel, id exec.StreamID, instID instructio } func newMultimapUserStateReader(ch *StateChannel, id exec.StreamID, instID instructionID, userStateID string, k []byte, w []byte, mk []byte) *stateKeyReader { - key := &fnpb.StateKey{ - Type: &fnpb.StateKey_MultimapUserState_{ - MultimapUserState: &fnpb.StateKey_MultimapUserState{ - TransformId: id.PtransformID, - UserStateId: userStateID, - Window: w, - Key: k, - MapKey: mk, - }, - }, - } + key := fnpb.StateKey_builder{ + MultimapUserState: fnpb.StateKey_MultimapUserState_builder{ + TransformId: id.PtransformID, + UserStateId: userStateID, + Window: w, + Key: k, + MapKey: mk, + }.Build(), + }.Build() return &stateKeyReader{ instID: instID, key: key, @@ -335,17 +323,15 @@ func newMultimapUserStateReader(ch *StateChannel, id exec.StreamID, instID instr } func newMultimapUserStateWriter(ch *StateChannel, id exec.StreamID, instID instructionID, userStateID string, k []byte, w []byte, mk []byte, wt writeTypeEnum) *stateKeyWriter { - key := &fnpb.StateKey{ - Type: &fnpb.StateKey_MultimapUserState_{ - MultimapUserState: &fnpb.StateKey_MultimapUserState{ - TransformId: id.PtransformID, - UserStateId: userStateID, - Window: w, - Key: k, - MapKey: mk, - }, - }, - } + key := fnpb.StateKey_builder{ + MultimapUserState: fnpb.StateKey_MultimapUserState_builder{ + TransformId: id.PtransformID, + UserStateId: userStateID, + Window: w, + Key: k, + MapKey: mk, + }.Build(), + }.Build() return &stateKeyWriter{ instID: instID, key: key, @@ -355,16 +341,14 @@ func newMultimapUserStateWriter(ch *StateChannel, id exec.StreamID, instID instr } func newMultimapKeysUserStateReader(ch *StateChannel, id exec.StreamID, instID instructionID, userStateID string, k []byte, w []byte) *stateKeyReader { - key := &fnpb.StateKey{ - Type: &fnpb.StateKey_MultimapKeysUserState_{ - MultimapKeysUserState: &fnpb.StateKey_MultimapKeysUserState{ - TransformId: id.PtransformID, - UserStateId: userStateID, - Window: w, - Key: k, - }, - }, - } + key := fnpb.StateKey_builder{ + MultimapKeysUserState: fnpb.StateKey_MultimapKeysUserState_builder{ + TransformId: id.PtransformID, + UserStateId: userStateID, + Window: w, + Key: k, + }.Build(), + }.Build() return &stateKeyReader{ instID: instID, key: key, @@ -373,16 +357,14 @@ func newMultimapKeysUserStateReader(ch *StateChannel, id exec.StreamID, instID i } func newMultimapKeysUserStateWriter(ch *StateChannel, id exec.StreamID, instID instructionID, userStateID string, k []byte, w []byte, wt writeTypeEnum) *stateKeyWriter { - key := &fnpb.StateKey{ - Type: &fnpb.StateKey_MultimapKeysUserState_{ - MultimapKeysUserState: &fnpb.StateKey_MultimapKeysUserState{ - TransformId: id.PtransformID, - UserStateId: userStateID, - Window: w, - Key: k, - }, - }, - } + key := fnpb.StateKey_builder{ + MultimapKeysUserState: fnpb.StateKey_MultimapKeysUserState_builder{ + TransformId: id.PtransformID, + UserStateId: userStateID, + Window: w, + Key: k, + }.Build(), + }.Build() return &stateKeyWriter{ instID: instID, key: key, @@ -407,16 +389,14 @@ func (r *stateKeyReader) Read(buf []byte) (int, error) { localChannel := r.ch r.mu.Unlock() - req := &fnpb.StateRequest{ + req := fnpb.StateRequest_builder{ // Id: set by StateChannel InstructionId: string(r.instID), StateKey: r.key, - Request: &fnpb.StateRequest_Get{ - Get: &fnpb.StateGetRequest{ - ContinuationToken: r.token, - }, - }, - } + Get: fnpb.StateGetRequest_builder{ + ContinuationToken: r.token, + }.Build(), + }.Build() resp, err := localChannel.Send(req) if err != nil { r.Close() @@ -477,25 +457,21 @@ func (r *stateKeyWriter) Write(buf []byte) (int, error) { var req *fnpb.StateRequest switch r.writeType { case writeTypeAppend: - req = &fnpb.StateRequest{ + req = fnpb.StateRequest_builder{ // Id: set by StateChannel InstructionId: string(r.instID), StateKey: r.key, - Request: &fnpb.StateRequest_Append{ - Append: &fnpb.StateAppendRequest{ - Data: toSend, - }, - }, - } + Append: fnpb.StateAppendRequest_builder{ + Data: toSend, + }.Build(), + }.Build() case writeTypeClear: - req = &fnpb.StateRequest{ + req = fnpb.StateRequest_builder{ // ID: set by StateChannel InstructionId: string(r.instID), StateKey: r.key, - Request: &fnpb.StateRequest_Clear{ - Clear: &fnpb.StateClearRequest{}, - }, - } + Clear: &fnpb.StateClearRequest{}, + }.Build() default: return 0, errors.Errorf("Unknown write type %v", r.writeType) } @@ -631,8 +607,8 @@ func (c *StateChannel) read(ctx context.Context) { } c.mu.Lock() - ch, ok := c.responses[msg.Id] - delete(c.responses, msg.Id) + ch, ok := c.responses[msg.GetId()] + delete(c.responses, msg.GetId()) c.mu.Unlock() if !ok { // This can happen if Send returns an error that write handles, but @@ -662,7 +638,7 @@ func (c *StateChannel) write(ctx context.Context) { } err = c.client.Send(req) if err != nil { - id = req.Id + id = req.GetId() break // non-nil errors mean the stream is broken and can't be re-used. } } @@ -688,14 +664,14 @@ func (c *StateChannel) write(ctx context.Context) { c.terminateStreamOnError(err) if ok { - ch <- &fnpb.StateResponse{Id: id, Error: fmt.Sprintf("StateChannel[%v].write failed to send: %v", c.id, err)} + ch <- fnpb.StateResponse_builder{Id: id, Error: fmt.Sprintf("StateChannel[%v].write failed to send: %v", c.id, err)}.Build() } } // Send sends a state request and returns the response. func (c *StateChannel) Send(req *fnpb.StateRequest) (*fnpb.StateResponse, error) { id := fmt.Sprintf("r%v", atomic.AddInt32(&c.nextRequestNo, 1)) - req.Id = id + req.SetId(id) ch := make(chan *fnpb.StateResponse, 1) c.mu.Lock() @@ -716,8 +692,8 @@ func (c *StateChannel) Send(req *fnpb.StateRequest) (*fnpb.StateResponse, error) defer c.mu.Unlock() return nil, errors.Wrapf(c.closedErr, "StateChannel[%v].Send(%v): context canceled", c.id, id) } - if resp.Error != "" { - return nil, errors.New(resp.Error) + if resp.GetError() != "" { + return nil, errors.New(resp.GetError()) } return resp, nil } diff --git a/sdks/go/pkg/beam/core/runtime/harness/statemgr_test.go b/sdks/go/pkg/beam/core/runtime/harness/statemgr_test.go index 53c4ca05ee12..81b1f2ca11e4 100644 --- a/sdks/go/pkg/beam/core/runtime/harness/statemgr_test.go +++ b/sdks/go/pkg/beam/core/runtime/harness/statemgr_test.go @@ -93,25 +93,21 @@ func TestStateChannel(t *testing.T) { for i := 0; i < count; i++ { go func() { req := <-client.send - client.recv <- &fnpb.StateResponse{ - Id: req.Id, // Ids need to match up to ensure routing can occur properly. - Response: &fnpb.StateResponse_Get{ - Get: &fnpb.StateGetResponse{ - ContinuationToken: req.GetGet().GetContinuationToken(), - }, - }, - } + client.recv <- fnpb.StateResponse_builder{ + Id: req.GetId(), // Ids need to match up to ensure routing can occur properly. + Get: fnpb.StateGetResponse_builder{ + ContinuationToken: req.GetGet().GetContinuationToken(), + }.Build(), + }.Build() }() } for i := 0; i < count; i++ { token := []byte(fmt.Sprintf("%d", i)) - resp, err := c.Send(&fnpb.StateRequest{ - Request: &fnpb.StateRequest_Get{ - Get: &fnpb.StateGetRequest{ - ContinuationToken: token, - }, - }, - }) + resp, err := c.Send(fnpb.StateRequest_builder{ + Get: fnpb.StateGetRequest_builder{ + ContinuationToken: token, + }.Build(), + }.Build()) if err != nil { t.Fatalf("unexpected error from Send: %v", err) } @@ -128,9 +124,9 @@ func TestStateChannel(t *testing.T) { req := <-client.send // Send should succeed. client.setRecvErr(io.EOF) - client.recv <- &fnpb.StateResponse{ - Id: req.Id, - } + client.recv <- fnpb.StateResponse_builder{ + Id: req.GetId(), + }.Build() }() _, err := c.Send(&fnpb.StateRequest{}) return err @@ -144,9 +140,9 @@ func TestStateChannel(t *testing.T) { req := <-client.send // Send should succeed. client.setRecvErr(expectedError) - client.recv <- &fnpb.StateResponse{ - Id: req.Id, - } + client.recv <- fnpb.StateResponse_builder{ + Id: req.GetId(), + }.Build() }() _, err := c.Send(&fnpb.StateRequest{}) return err @@ -160,13 +156,13 @@ func TestStateChannel(t *testing.T) { req := <-client.send // Send should succeed. c.mu.Lock() - ch := c.responses[req.Id] - delete(c.responses, req.Id) + ch := c.responses[req.GetId()] + delete(c.responses, req.GetId()) c.mu.Unlock() - resp := &fnpb.StateResponse{ - Id: req.Id, - } + resp := fnpb.StateResponse_builder{ + Id: req.GetId(), + }.Build() client.recv <- resp // unblock Send. ch <- resp @@ -183,9 +179,9 @@ func TestStateChannel(t *testing.T) { // This can be plumbed through on either side, write or read, // the important part is that we get it. client.setRecvErr(expectedError) - client.recv <- &fnpb.StateResponse{ - Id: req.Id, - } + client.recv <- fnpb.StateResponse_builder{ + Id: req.GetId(), + }.Build() }() _, err := c.Send(&fnpb.StateRequest{}) return err @@ -235,7 +231,7 @@ func TestStateChannel(t *testing.T) { client.setRecvErr(nil) // Drain the next send, and ensure the response is unblocked. req := <-client.send - client.recv <- &fnpb.StateResponse{Id: req.Id} // Ids need to match up to ensure routing can occur properly. + client.recv <- fnpb.StateResponse_builder{Id: req.GetId()}.Build() // Ids need to match up to ensure routing can occur properly. }() if _, err := c.Send(&fnpb.StateRequest{}); !contains(err, test.expectedErr) { t.Errorf("Unexpected error from Send: got %v, want %v", err, test.expectedErr) @@ -337,9 +333,9 @@ func TestStateKeyReader(t *testing.T) { go func() { if test.noGet { req := <-ch.requests - ch.responses[req.Id] <- &fnpb.StateResponse{ - Id: req.Id, - } + ch.responses[req.GetId()] <- fnpb.StateResponse_builder{ + Id: req.GetId(), + }.Build() return } for i, buflen := range test.buflens { @@ -354,15 +350,13 @@ func TestStateKeyReader(t *testing.T) { } req := <-ch.requests - ch.responses[req.Id] <- &fnpb.StateResponse{ - Id: req.Id, - Response: &fnpb.StateResponse_Get{ - Get: &fnpb.StateGetResponse{ - ContinuationToken: token, - Data: buf, - }, - }, - } + ch.responses[req.GetId()] <- fnpb.StateResponse_builder{ + Id: req.GetId(), + Get: fnpb.StateGetResponse_builder{ + ContinuationToken: token, + Data: buf, + }.Build(), + }.Build() } }() @@ -463,30 +457,30 @@ func TestStateKeyWriter(t *testing.T) { if !ok { t.Errorf("Append write: got %v, want data of type StateRequest_Append", req.Request) } else { - data := sra.Append.Data + data := sra.Append.GetData() if !bytes.Equal(data, test.data) { t.Errorf("Expected request data: got %v, want %v", data, test.data) } } - ch.responses[req.Id] <- &fnpb.StateResponse{ - Id: req.Id, - Response: &fnpb.StateResponse_Append{}, - } + ch.responses[req.GetId()] <- fnpb.StateResponse_builder{ + Id: req.GetId(), + Append: &fnpb.StateAppendResponse{}, + }.Build() case writeTypeClear: _, ok := req.Request.(*fnpb.StateRequest_Clear) if !ok { t.Errorf("Append write: got %v, want data of type StateRequest_Append", req.Request) } - ch.responses[req.Id] <- &fnpb.StateResponse{ - Id: req.Id, - Response: &fnpb.StateResponse_Clear{}, - } + ch.responses[req.GetId()] <- fnpb.StateResponse_builder{ + Id: req.GetId(), + Clear: &fnpb.StateClearResponse{}, + }.Build() default: // Still return response so that write doesn't hang. - ch.responses[req.Id] <- &fnpb.StateResponse{ - Id: req.Id, - Response: &fnpb.StateResponse_Append{}, - } + ch.responses[req.GetId()] <- fnpb.StateResponse_builder{ + Id: req.GetId(), + Append: &fnpb.StateAppendResponse{}, + }.Build() } }() diff --git a/sdks/go/pkg/beam/core/runtime/harness/worker_status.go b/sdks/go/pkg/beam/core/runtime/harness/worker_status.go index 0ec7e1f805ce..9b53f3cf065a 100644 --- a/sdks/go/pkg/beam/core/runtime/harness/worker_status.go +++ b/sdks/go/pkg/beam/core/runtime/harness/worker_status.go @@ -129,7 +129,7 @@ func (w *workerStatusHandler) reader(ctx context.Context, stub fnpb.BeamFnWorker goroutineDump(statusInfo) buildInfo(statusInfo) - response := &fnpb.WorkerStatusResponse{Id: req.GetId(), StatusInfo: statusInfo.String()} + response := fnpb.WorkerStatusResponse_builder{Id: req.GetId(), StatusInfo: statusInfo.String()}.Build() if err := stub.Send(response); err != nil && err != io.EOF { log.Errorf(ctx, "workerStatus.Writer: Failed to respond: %v", err) } diff --git a/sdks/go/pkg/beam/core/runtime/harness/worker_status_test.go b/sdks/go/pkg/beam/core/runtime/harness/worker_status_test.go index 9cc2e5330797..dffe0e26fb24 100644 --- a/sdks/go/pkg/beam/core/runtime/harness/worker_status_test.go +++ b/sdks/go/pkg/beam/core/runtime/harness/worker_status_test.go @@ -37,7 +37,7 @@ type BeamFnWorkerStatusServicer struct { } func (w *BeamFnWorkerStatusServicer) WorkerStatus(b fnpb.BeamFnWorkerStatus_WorkerStatusServer) error { - b.Send(&fnpb.WorkerStatusRequest{Id: "1"}) + b.Send(fnpb.WorkerStatusRequest_builder{Id: "1"}.Build()) resp, err := b.Recv() if err != nil { return fmt.Errorf("error receiving response b.recv: %v", err) diff --git a/sdks/go/pkg/beam/core/runtime/metricsx/metricsx_test.go b/sdks/go/pkg/beam/core/runtime/metricsx/metricsx_test.go index 95b92f929c08..06d58290346f 100644 --- a/sdks/go/pkg/beam/core/runtime/metricsx/metricsx_test.go +++ b/sdks/go/pkg/beam/core/runtime/metricsx/metricsx_test.go @@ -46,12 +46,12 @@ func TestFromMonitoringInfos_Counters(t *testing.T) { "NAME": "customCounter", } - mInfo := &pipepb.MonitoringInfo{ + mInfo := pipepb.MonitoringInfo_builder{ Urn: UrnToString(UrnUserSumInt64), Type: UrnToType(UrnUserSumInt64), Labels: labels, Payload: payload, - } + }.Build() attempted := []*pipepb.MonitoringInfo{mInfo} committed := []*pipepb.MonitoringInfo{} @@ -111,30 +111,30 @@ func TestFromMonitoringInfos_Msec(t *testing.T) { if err != nil { t.Fatalf("Failed to encode Int64Counter: %v", err) } - mStartBundleInfo := &pipepb.MonitoringInfo{ + mStartBundleInfo := pipepb.MonitoringInfo_builder{ Urn: UrnToString(ExecutionMsecUrn(0)), Type: UrnToType(ExecutionMsecUrn(0)), Labels: labels, Payload: startValue, - } - mProcessBundleInfo := &pipepb.MonitoringInfo{ + }.Build() + mProcessBundleInfo := pipepb.MonitoringInfo_builder{ Urn: UrnToString(ExecutionMsecUrn(1)), Type: UrnToType(ExecutionMsecUrn(1)), Labels: labels, Payload: processValue, - } - mFinishBundleInfo := &pipepb.MonitoringInfo{ + }.Build() + mFinishBundleInfo := pipepb.MonitoringInfo_builder{ Urn: UrnToString(ExecutionMsecUrn(2)), Type: UrnToType(ExecutionMsecUrn(2)), Labels: labels, Payload: finishValue, - } - mTotalTimeInfo := &pipepb.MonitoringInfo{ + }.Build() + mTotalTimeInfo := pipepb.MonitoringInfo_builder{ Urn: UrnToString(ExecutionMsecUrn(3)), Type: UrnToType(ExecutionMsecUrn(3)), Labels: labels, Payload: totalValue, - } + }.Build() attempted := []*pipepb.MonitoringInfo{mStartBundleInfo, mProcessBundleInfo, mFinishBundleInfo, mTotalTimeInfo} committed := []*pipepb.MonitoringInfo{} @@ -174,12 +174,12 @@ func TestFromMonitoringInfos_PColCounters(t *testing.T) { "NAME": "customCounter", } - mInfo := &pipepb.MonitoringInfo{ + mInfo := pipepb.MonitoringInfo_builder{ Urn: UrnToString(UrnElementCount), Type: UrnToType(UrnElementCount), Labels: labels, Payload: payload, - } + }.Build() attempted := []*pipepb.MonitoringInfo{mInfo} committed := []*pipepb.MonitoringInfo{} @@ -224,12 +224,12 @@ func TestFromMonitoringInfos_SampledByteSize(t *testing.T) { "NAME": "customCounter", } - mInfo := &pipepb.MonitoringInfo{ + mInfo := pipepb.MonitoringInfo_builder{ Urn: UrnToString(UrnSampledByteSize), Type: UrnToType(UrnSampledByteSize), Labels: labels, Payload: payload, - } + }.Build() attempted := []*pipepb.MonitoringInfo{mInfo} committed := []*pipepb.MonitoringInfo{} @@ -274,12 +274,12 @@ func TestFromMonitoringInfos_Distributions(t *testing.T) { "NAME": "customDist", } - mInfo := &pipepb.MonitoringInfo{ + mInfo := pipepb.MonitoringInfo_builder{ Urn: UrnToString(UrnUserDistInt64), Type: UrnToType(UrnUserDistInt64), Labels: labels, Payload: payload, - } + }.Build() attempted := []*pipepb.MonitoringInfo{mInfo} committed := []*pipepb.MonitoringInfo{} @@ -324,12 +324,12 @@ func TestFromMonitoringInfos_Gauges(t *testing.T) { "NAME": "customGauge", } - mInfo := &pipepb.MonitoringInfo{ + mInfo := pipepb.MonitoringInfo_builder{ Urn: UrnToString(UrnUserLatestMsInt64), Type: UrnToType(UrnUserLatestMsInt64), Labels: labels, Payload: payload, - } + }.Build() attempted := []*pipepb.MonitoringInfo{mInfo} committed := []*pipepb.MonitoringInfo{} diff --git a/sdks/go/pkg/beam/core/runtime/pipelinex/clone.go b/sdks/go/pkg/beam/core/runtime/pipelinex/clone.go index 7745cec942ec..443853b07699 100644 --- a/sdks/go/pkg/beam/core/runtime/pipelinex/clone.go +++ b/sdks/go/pkg/beam/core/runtime/pipelinex/clone.go @@ -21,10 +21,10 @@ import ( ) func shallowClonePipeline(p *pipepb.Pipeline) *pipepb.Pipeline { - ret := &pipepb.Pipeline{ + ret := pipepb.Pipeline_builder{ Components: shallowCloneComponents(p.GetComponents()), Requirements: reflectx.ShallowClone(p.GetRequirements()).([]string), - } + }.Build() ret.RootTransformIds, _ = reflectx.ShallowClone(p.GetRootTransformIds()).([]string) return ret } @@ -45,15 +45,15 @@ func ShallowClonePTransform(t *pipepb.PTransform) *pipepb.PTransform { return nil } - ret := &pipepb.PTransform{ - UniqueName: t.UniqueName, - Spec: t.Spec, - DisplayData: t.DisplayData, - Annotations: t.Annotations, - } - ret.Subtransforms, _ = reflectx.ShallowClone(t.Subtransforms).([]string) - ret.Inputs, _ = reflectx.ShallowClone(t.Inputs).(map[string]string) - ret.Outputs, _ = reflectx.ShallowClone(t.Outputs).(map[string]string) - ret.EnvironmentId = t.EnvironmentId + ret := pipepb.PTransform_builder{ + UniqueName: t.GetUniqueName(), + Spec: t.GetSpec(), + DisplayData: t.GetDisplayData(), + Annotations: t.GetAnnotations(), + }.Build() + ret.Subtransforms, _ = reflectx.ShallowClone(t.GetSubtransforms()).([]string) + ret.Inputs, _ = reflectx.ShallowClone(t.GetInputs()).(map[string]string) + ret.Outputs, _ = reflectx.ShallowClone(t.GetOutputs()).(map[string]string) + ret.SetEnvironmentId(t.GetEnvironmentId()) return ret } diff --git a/sdks/go/pkg/beam/core/runtime/pipelinex/clone_test.go b/sdks/go/pkg/beam/core/runtime/pipelinex/clone_test.go index b58a30983797..cc674b969790 100644 --- a/sdks/go/pkg/beam/core/runtime/pipelinex/clone_test.go +++ b/sdks/go/pkg/beam/core/runtime/pipelinex/clone_test.go @@ -26,11 +26,11 @@ import ( func TestShallowClonePTransform(t *testing.T) { tests := []*pipepb.PTransform{ {}, - {UniqueName: "a"}, - {Spec: &pipepb.FunctionSpec{Urn: "foo"}}, - {Subtransforms: []string{"a", "b"}}, - {Inputs: map[string]string{"a": "b"}}, - {Outputs: map[string]string{"a": "b"}}, + pipepb.PTransform_builder{UniqueName: "a"}.Build(), + pipepb.PTransform_builder{Spec: pipepb.FunctionSpec_builder{Urn: "foo"}.Build()}.Build(), + pipepb.PTransform_builder{Subtransforms: []string{"a", "b"}}.Build(), + pipepb.PTransform_builder{Inputs: map[string]string{"a": "b"}}.Build(), + pipepb.PTransform_builder{Outputs: map[string]string{"a": "b"}}.Build(), } for _, test := range tests { diff --git a/sdks/go/pkg/beam/core/runtime/pipelinex/replace.go b/sdks/go/pkg/beam/core/runtime/pipelinex/replace.go index 9e527f2fd322..89e671f37adc 100644 --- a/sdks/go/pkg/beam/core/runtime/pipelinex/replace.go +++ b/sdks/go/pkg/beam/core/runtime/pipelinex/replace.go @@ -36,11 +36,11 @@ import ( // modified. func Update(p *pipepb.Pipeline, values *pipepb.Components) (*pipepb.Pipeline, error) { ret := shallowClonePipeline(p) - reflectx.UpdateMap(ret.Components.Transforms, values.Transforms) - reflectx.UpdateMap(ret.Components.Pcollections, values.Pcollections) - reflectx.UpdateMap(ret.Components.WindowingStrategies, values.WindowingStrategies) - reflectx.UpdateMap(ret.Components.Coders, values.Coders) - reflectx.UpdateMap(ret.Components.Environments, values.Environments) + reflectx.UpdateMap(ret.GetComponents().GetTransforms(), values.GetTransforms()) + reflectx.UpdateMap(ret.GetComponents().GetPcollections(), values.GetPcollections()) + reflectx.UpdateMap(ret.GetComponents().GetWindowingStrategies(), values.GetWindowingStrategies()) + reflectx.UpdateMap(ret.GetComponents().GetCoders(), values.GetCoders()) + reflectx.UpdateMap(ret.GetComponents().GetEnvironments(), values.GetEnvironments()) return Normalize(ret) } @@ -60,12 +60,12 @@ func Normalize(p *pipepb.Pipeline) (*pipepb.Pipeline, error) { ret := shallowClonePipeline(p) if IdempotentNormalize { - ret.Components.Transforms = ensureUniqueNames(ret.Components.Transforms) + ret.GetComponents().SetTransforms(ensureUniqueNames(ret.GetComponents().GetTransforms())) } else { - ret.Components.Transforms = ensureUniqueNamesLegacy(ret.Components.Transforms) + ret.GetComponents().SetTransforms(ensureUniqueNamesLegacy(ret.GetComponents().GetTransforms())) } - ret.Components.Transforms = computeCompositeInputOutput(ret.Components.Transforms) - ret.RootTransformIds = computeRoots(ret.Components.Transforms) + ret.GetComponents().SetTransforms(computeCompositeInputOutput(ret.GetComponents().GetTransforms())) + ret.SetRootTransformIds(computeRoots(ret.GetComponents().GetTransforms())) return ret, nil } @@ -85,7 +85,7 @@ func walkCoders(coders, accum map[string]*pipepb.Coder, id string) { c := coders[id] accum[id] = c - for _, sub := range c.ComponentCoderIds { + for _, sub := range c.GetComponentCoderIds() { walkCoders(coders, accum, sub) } } @@ -106,7 +106,7 @@ func computeRoots(xforms map[string]*pipepb.PTransform) []string { func makeParentMap(xforms map[string]*pipepb.PTransform) map[string]string { parent := make(map[string]string) for id, t := range xforms { - for _, key := range t.Subtransforms { + for _, key := range t.GetSubtransforms() { parent[key] = id } } @@ -138,7 +138,7 @@ func computeCompositeInputOutput(xforms map[string]*pipepb.PTransform) map[strin // maps of composite transforms. Update the transform map. func walk(id string, ret map[string]*pipepb.PTransform, seen map[string]bool, primitiveXformsForInput map[string][]string) { t := ret[id] - if seen[id] || len(t.Subtransforms) == 0 { + if seen[id] || len(t.GetSubtransforms()) == 0 { return } @@ -150,7 +150,7 @@ func walk(id string, ret map[string]*pipepb.PTransform, seen map[string]bool, pr in := make(map[string]bool) out := make(map[string]bool) local := map[string]bool{id: true} - for _, sid := range t.Subtransforms { + for _, sid := range t.GetSubtransforms() { walk(sid, ret, seen, primitiveXformsForInput) inout(ret[sid], in, out) local[sid] = true @@ -164,9 +164,9 @@ func walk(id string, ret map[string]*pipepb.PTransform, seen map[string]bool, pr externalIns(local, primitiveXformsForInput, extIn, out) upd := ShallowClonePTransform(t) - upd.Inputs = diff(in, out) - upd.Outputs = diffAndMerge(out, in, extIn) - upd.Subtransforms = TopologicalSort(ret, upd.Subtransforms) + upd.SetInputs(diff(in, out)) + upd.SetOutputs(diffAndMerge(out, in, extIn)) + upd.SetSubtransforms(TopologicalSort(ret, upd.GetSubtransforms())) ret[id] = upd seen[id] = true @@ -312,9 +312,9 @@ func ensureUniqueNames(xforms map[string]*pipepb.PTransform) map[string]*pipepb. name := findFreeName(seen, base) seen[name] = true - if name != t.UniqueName { + if name != t.GetUniqueName() { upd := ShallowClonePTransform(t) - upd.UniqueName = name + upd.SetUniqueName(name) ret[id] = upd } return name @@ -373,12 +373,12 @@ func ensureUniqueNamesLegacy(xforms map[string]*pipepb.PTransform) map[string]*p seen := make(map[string]bool) for _, id := range ordering { t := xforms[id] - name := findFreeName(seen, t.UniqueName) + name := findFreeName(seen, t.GetUniqueName()) seen[name] = true - if name != t.UniqueName { + if name != t.GetUniqueName() { upd := ShallowClonePTransform(t) - upd.UniqueName = name + upd.SetUniqueName(name) ret[id] = upd } } @@ -426,12 +426,12 @@ func ApplySdkImageOverrides(p *pipepb.Pipeline, patterns map[string]string) erro for re, replacement := range regexes { newImg := re.ReplaceAllLiteralString(oldImg, replacement) if newImg != oldImg { - payload.ContainerImage = newImg + payload.SetContainerImage(newImg) pl, err := proto.Marshal(&payload) if err != nil { return err } - env.Payload = pl + env.SetPayload(pl) break // Apply at most one override to each environment. } } diff --git a/sdks/go/pkg/beam/core/runtime/pipelinex/replace_test.go b/sdks/go/pkg/beam/core/runtime/pipelinex/replace_test.go index 3024787e6163..02f4d660f5f2 100644 --- a/sdks/go/pkg/beam/core/runtime/pipelinex/replace_test.go +++ b/sdks/go/pkg/beam/core/runtime/pipelinex/replace_test.go @@ -33,130 +33,130 @@ func TestEnsureUniqueName(t *testing.T) { { name: "AlreadyUnique", in: map[string]*pipepb.PTransform{ - "1": {UniqueName: "a"}, - "2": {UniqueName: "b"}, - "3": {UniqueName: "c"}, + "1": pipepb.PTransform_builder{UniqueName: "a"}.Build(), + "2": pipepb.PTransform_builder{UniqueName: "b"}.Build(), + "3": pipepb.PTransform_builder{UniqueName: "c"}.Build(), }, exp: map[string]*pipepb.PTransform{ - "1": {UniqueName: "a"}, - "2": {UniqueName: "b"}, - "3": {UniqueName: "c"}, + "1": pipepb.PTransform_builder{UniqueName: "a"}.Build(), + "2": pipepb.PTransform_builder{UniqueName: "b"}.Build(), + "3": pipepb.PTransform_builder{UniqueName: "c"}.Build(), }, }, { name: "NeedsUniqueLeaves", in: map[string]*pipepb.PTransform{ - "2": {UniqueName: "a"}, - "1": {UniqueName: "a"}, - "3": {UniqueName: "a"}, + "2": pipepb.PTransform_builder{UniqueName: "a"}.Build(), + "1": pipepb.PTransform_builder{UniqueName: "a"}.Build(), + "3": pipepb.PTransform_builder{UniqueName: "a"}.Build(), }, exp: map[string]*pipepb.PTransform{ - "1": {UniqueName: "a"}, - "2": {UniqueName: "a'1"}, - "3": {UniqueName: "a'2"}, + "1": pipepb.PTransform_builder{UniqueName: "a"}.Build(), + "2": pipepb.PTransform_builder{UniqueName: "a'1"}.Build(), + "3": pipepb.PTransform_builder{UniqueName: "a'2"}.Build(), }, }, { name: "StripUniqueLeaves", in: map[string]*pipepb.PTransform{ - "1": {UniqueName: "a"}, - "2": {UniqueName: "a'1"}, - "3": {UniqueName: "a'2"}, + "1": pipepb.PTransform_builder{UniqueName: "a"}.Build(), + "2": pipepb.PTransform_builder{UniqueName: "a'1"}.Build(), + "3": pipepb.PTransform_builder{UniqueName: "a'2"}.Build(), }, exp: map[string]*pipepb.PTransform{ - "1": {UniqueName: "a"}, - "2": {UniqueName: "a'1"}, - "3": {UniqueName: "a'2"}, + "1": pipepb.PTransform_builder{UniqueName: "a"}.Build(), + "2": pipepb.PTransform_builder{UniqueName: "a'1"}.Build(), + "3": pipepb.PTransform_builder{UniqueName: "a'2"}.Build(), }, }, { name: "NonTopologicalIdOrder", in: map[string]*pipepb.PTransform{ - "e1": {UniqueName: "a"}, - "s1": {UniqueName: "a", Subtransforms: []string{"e1"}}, - "s2": {UniqueName: "a", Subtransforms: []string{"s1"}}, - "s3": {UniqueName: "a", Subtransforms: []string{"s2"}}, // root + "e1": pipepb.PTransform_builder{UniqueName: "a"}.Build(), + "s1": pipepb.PTransform_builder{UniqueName: "a", Subtransforms: []string{"e1"}}.Build(), + "s2": pipepb.PTransform_builder{UniqueName: "a", Subtransforms: []string{"s1"}}.Build(), + "s3": pipepb.PTransform_builder{UniqueName: "a", Subtransforms: []string{"s2"}}.Build(), // root }, exp: map[string]*pipepb.PTransform{ - "e1": {UniqueName: "a/a/a/a"}, - "s1": {UniqueName: "a/a/a", Subtransforms: []string{"e1"}}, - "s2": {UniqueName: "a/a", Subtransforms: []string{"s1"}}, - "s3": {UniqueName: "a", Subtransforms: []string{"s2"}}, // root + "e1": pipepb.PTransform_builder{UniqueName: "a/a/a/a"}.Build(), + "s1": pipepb.PTransform_builder{UniqueName: "a/a/a", Subtransforms: []string{"e1"}}.Build(), + "s2": pipepb.PTransform_builder{UniqueName: "a/a", Subtransforms: []string{"s1"}}.Build(), + "s3": pipepb.PTransform_builder{UniqueName: "a", Subtransforms: []string{"s2"}}.Build(), // root }, }, { name: "UniqueComps", in: map[string]*pipepb.PTransform{ - "e1": {UniqueName: "a"}, - "e2": {UniqueName: "a"}, - "s1": {UniqueName: "a", Subtransforms: []string{"e1"}}, - "s2": {UniqueName: "a", Subtransforms: []string{"e2"}}, + "e1": pipepb.PTransform_builder{UniqueName: "a"}.Build(), + "e2": pipepb.PTransform_builder{UniqueName: "a"}.Build(), + "s1": pipepb.PTransform_builder{UniqueName: "a", Subtransforms: []string{"e1"}}.Build(), + "s2": pipepb.PTransform_builder{UniqueName: "a", Subtransforms: []string{"e2"}}.Build(), }, exp: map[string]*pipepb.PTransform{ - "e1": {UniqueName: "a/a"}, - "e2": {UniqueName: "a'1/a"}, - "s1": {UniqueName: "a", Subtransforms: []string{"e1"}}, - "s2": {UniqueName: "a'1", Subtransforms: []string{"e2"}}, + "e1": pipepb.PTransform_builder{UniqueName: "a/a"}.Build(), + "e2": pipepb.PTransform_builder{UniqueName: "a'1/a"}.Build(), + "s1": pipepb.PTransform_builder{UniqueName: "a", Subtransforms: []string{"e1"}}.Build(), + "s2": pipepb.PTransform_builder{UniqueName: "a'1", Subtransforms: []string{"e2"}}.Build(), }, }, { name: "StripComps", in: map[string]*pipepb.PTransform{ - "e1": {UniqueName: "a/a"}, - "e2": {UniqueName: "a'1/a"}, - "s1": {UniqueName: "a", Subtransforms: []string{"e1"}}, - "s2": {UniqueName: "a'1", Subtransforms: []string{"e2"}}, + "e1": pipepb.PTransform_builder{UniqueName: "a/a"}.Build(), + "e2": pipepb.PTransform_builder{UniqueName: "a'1/a"}.Build(), + "s1": pipepb.PTransform_builder{UniqueName: "a", Subtransforms: []string{"e1"}}.Build(), + "s2": pipepb.PTransform_builder{UniqueName: "a'1", Subtransforms: []string{"e2"}}.Build(), }, exp: map[string]*pipepb.PTransform{ - "e1": {UniqueName: "a/a"}, - "e2": {UniqueName: "a'1/a"}, - "s1": {UniqueName: "a", Subtransforms: []string{"e1"}}, - "s2": {UniqueName: "a'1", Subtransforms: []string{"e2"}}, + "e1": pipepb.PTransform_builder{UniqueName: "a/a"}.Build(), + "e2": pipepb.PTransform_builder{UniqueName: "a'1/a"}.Build(), + "s1": pipepb.PTransform_builder{UniqueName: "a", Subtransforms: []string{"e1"}}.Build(), + "s2": pipepb.PTransform_builder{UniqueName: "a'1", Subtransforms: []string{"e2"}}.Build(), }, }, { name: "large", in: map[string]*pipepb.PTransform{ - "e1": {UniqueName: "a"}, - "e2": {UniqueName: "a"}, - "e3": {UniqueName: "a"}, - "e4": {UniqueName: "a"}, - "e5": {UniqueName: "a"}, - "e6": {UniqueName: "a"}, - "e7": {UniqueName: "a"}, - "e8": {UniqueName: "a"}, - "e9": {UniqueName: "a"}, - "e10": {UniqueName: "a"}, - "e11": {UniqueName: "a"}, - "e12": {UniqueName: "a"}, - "s1": {UniqueName: "a", Subtransforms: []string{"s2", "s3"}}, - "s2": {UniqueName: "a", Subtransforms: []string{"s4", "s5"}}, - "s3": {UniqueName: "a", Subtransforms: []string{"s6", "s7"}}, - "s4": {UniqueName: "a", Subtransforms: []string{"e1"}}, - "s5": {UniqueName: "a", Subtransforms: []string{"e2", "e3"}}, - "s6": {UniqueName: "a", Subtransforms: []string{"e4", "e5", "e6"}}, - "s7": {UniqueName: "a", Subtransforms: []string{"e7", "e8", "e9", "e10", "e11", "e12"}}, + "e1": pipepb.PTransform_builder{UniqueName: "a"}.Build(), + "e2": pipepb.PTransform_builder{UniqueName: "a"}.Build(), + "e3": pipepb.PTransform_builder{UniqueName: "a"}.Build(), + "e4": pipepb.PTransform_builder{UniqueName: "a"}.Build(), + "e5": pipepb.PTransform_builder{UniqueName: "a"}.Build(), + "e6": pipepb.PTransform_builder{UniqueName: "a"}.Build(), + "e7": pipepb.PTransform_builder{UniqueName: "a"}.Build(), + "e8": pipepb.PTransform_builder{UniqueName: "a"}.Build(), + "e9": pipepb.PTransform_builder{UniqueName: "a"}.Build(), + "e10": pipepb.PTransform_builder{UniqueName: "a"}.Build(), + "e11": pipepb.PTransform_builder{UniqueName: "a"}.Build(), + "e12": pipepb.PTransform_builder{UniqueName: "a"}.Build(), + "s1": pipepb.PTransform_builder{UniqueName: "a", Subtransforms: []string{"s2", "s3"}}.Build(), + "s2": pipepb.PTransform_builder{UniqueName: "a", Subtransforms: []string{"s4", "s5"}}.Build(), + "s3": pipepb.PTransform_builder{UniqueName: "a", Subtransforms: []string{"s6", "s7"}}.Build(), + "s4": pipepb.PTransform_builder{UniqueName: "a", Subtransforms: []string{"e1"}}.Build(), + "s5": pipepb.PTransform_builder{UniqueName: "a", Subtransforms: []string{"e2", "e3"}}.Build(), + "s6": pipepb.PTransform_builder{UniqueName: "a", Subtransforms: []string{"e4", "e5", "e6"}}.Build(), + "s7": pipepb.PTransform_builder{UniqueName: "a", Subtransforms: []string{"e7", "e8", "e9", "e10", "e11", "e12"}}.Build(), }, exp: map[string]*pipepb.PTransform{ - "e1": {UniqueName: "a/a/a/a"}, - "e2": {UniqueName: "a/a/a'1/a"}, - "e3": {UniqueName: "a/a/a'1/a'1"}, - "e4": {UniqueName: "a/a'1/a/a"}, - "e5": {UniqueName: "a/a'1/a/a'1"}, - "e6": {UniqueName: "a/a'1/a/a'2"}, - "e7": {UniqueName: "a/a'1/a'1/a"}, - "e8": {UniqueName: "a/a'1/a'1/a'1"}, - "e9": {UniqueName: "a/a'1/a'1/a'2"}, - "e10": {UniqueName: "a/a'1/a'1/a'3"}, - "e11": {UniqueName: "a/a'1/a'1/a'4"}, - "e12": {UniqueName: "a/a'1/a'1/a'5"}, - "s1": {UniqueName: "a", Subtransforms: []string{"s2", "s3"}}, - "s2": {UniqueName: "a/a", Subtransforms: []string{"s4", "s5"}}, - "s3": {UniqueName: "a/a'1", Subtransforms: []string{"s6", "s7"}}, - "s4": {UniqueName: "a/a/a", Subtransforms: []string{"e1"}}, - "s5": {UniqueName: "a/a/a'1", Subtransforms: []string{"e2", "e3"}}, - "s6": {UniqueName: "a/a'1/a", Subtransforms: []string{"e4", "e5", "e6"}}, - "s7": {UniqueName: "a/a'1/a'1", Subtransforms: []string{"e7", "e8", "e9", "e10", "e11", "e12"}}, + "e1": pipepb.PTransform_builder{UniqueName: "a/a/a/a"}.Build(), + "e2": pipepb.PTransform_builder{UniqueName: "a/a/a'1/a"}.Build(), + "e3": pipepb.PTransform_builder{UniqueName: "a/a/a'1/a'1"}.Build(), + "e4": pipepb.PTransform_builder{UniqueName: "a/a'1/a/a"}.Build(), + "e5": pipepb.PTransform_builder{UniqueName: "a/a'1/a/a'1"}.Build(), + "e6": pipepb.PTransform_builder{UniqueName: "a/a'1/a/a'2"}.Build(), + "e7": pipepb.PTransform_builder{UniqueName: "a/a'1/a'1/a"}.Build(), + "e8": pipepb.PTransform_builder{UniqueName: "a/a'1/a'1/a'1"}.Build(), + "e9": pipepb.PTransform_builder{UniqueName: "a/a'1/a'1/a'2"}.Build(), + "e10": pipepb.PTransform_builder{UniqueName: "a/a'1/a'1/a'3"}.Build(), + "e11": pipepb.PTransform_builder{UniqueName: "a/a'1/a'1/a'4"}.Build(), + "e12": pipepb.PTransform_builder{UniqueName: "a/a'1/a'1/a'5"}.Build(), + "s1": pipepb.PTransform_builder{UniqueName: "a", Subtransforms: []string{"s2", "s3"}}.Build(), + "s2": pipepb.PTransform_builder{UniqueName: "a/a", Subtransforms: []string{"s4", "s5"}}.Build(), + "s3": pipepb.PTransform_builder{UniqueName: "a/a'1", Subtransforms: []string{"s6", "s7"}}.Build(), + "s4": pipepb.PTransform_builder{UniqueName: "a/a/a", Subtransforms: []string{"e1"}}.Build(), + "s5": pipepb.PTransform_builder{UniqueName: "a/a/a'1", Subtransforms: []string{"e2", "e3"}}.Build(), + "s6": pipepb.PTransform_builder{UniqueName: "a/a'1/a", Subtransforms: []string{"e4", "e5", "e6"}}.Build(), + "s7": pipepb.PTransform_builder{UniqueName: "a/a'1/a'1", Subtransforms: []string{"e7", "e8", "e9", "e10", "e11", "e12"}}.Build(), }, }, } @@ -179,124 +179,124 @@ func TestComputeInputOutput(t *testing.T) { { name: "singleton composite", in: map[string]*pipepb.PTransform{ - "1": { + "1": pipepb.PTransform_builder{ UniqueName: "a", Subtransforms: []string{"2"}, - }, - "2": { + }.Build(), + "2": pipepb.PTransform_builder{ UniqueName: "b", Inputs: map[string]string{"i0": "p1"}, Outputs: map[string]string{"i0": "p2"}, - }, + }.Build(), }, exp: map[string]*pipepb.PTransform{ - "1": { + "1": pipepb.PTransform_builder{ UniqueName: "a", Subtransforms: []string{"2"}, Inputs: map[string]string{"p1": "p1"}, Outputs: map[string]string{"p2": "p2"}, - }, - "2": { + }.Build(), + "2": pipepb.PTransform_builder{ UniqueName: "b", Inputs: map[string]string{"i0": "p1"}, Outputs: map[string]string{"i0": "p2"}, - }, + }.Build(), }, }, { name: "closed composite", in: map[string]*pipepb.PTransform{ - "1": { + "1": pipepb.PTransform_builder{ UniqueName: "a", Subtransforms: []string{"2", "3"}, - }, - "2": {UniqueName: "b", Outputs: map[string]string{"i0": "p1"}}, - "3": {UniqueName: "c", Inputs: map[string]string{"i0": "p1"}}, + }.Build(), + "2": pipepb.PTransform_builder{UniqueName: "b", Outputs: map[string]string{"i0": "p1"}}.Build(), + "3": pipepb.PTransform_builder{UniqueName: "c", Inputs: map[string]string{"i0": "p1"}}.Build(), }, exp: map[string]*pipepb.PTransform{ - "1": { + "1": pipepb.PTransform_builder{ UniqueName: "a", Subtransforms: []string{"2", "3"}, - }, - "2": {UniqueName: "b", Outputs: map[string]string{"i0": "p1"}}, - "3": {UniqueName: "c", Inputs: map[string]string{"i0": "p1"}}, + }.Build(), + "2": pipepb.PTransform_builder{UniqueName: "b", Outputs: map[string]string{"i0": "p1"}}.Build(), + "3": pipepb.PTransform_builder{UniqueName: "c", Inputs: map[string]string{"i0": "p1"}}.Build(), }, }, { name: "nested composites", in: map[string]*pipepb.PTransform{ - "1": { + "1": pipepb.PTransform_builder{ UniqueName: "a", Subtransforms: []string{"2"}, - }, - "2": { + }.Build(), + "2": pipepb.PTransform_builder{ UniqueName: "b", Subtransforms: []string{"3", "7", "8"}, - }, - "3": { + }.Build(), + "3": pipepb.PTransform_builder{ UniqueName: "c", Subtransforms: []string{"4", "5", "6"}, - }, - "4": {UniqueName: "d", Inputs: map[string]string{"i0": "p1"}, Outputs: map[string]string{"i0": "p2"}}, - "5": {UniqueName: "e", Inputs: map[string]string{"i0": "p2"}, Outputs: map[string]string{"i0": "p3", "i1": "p4"}}, - "6": {UniqueName: "f", Inputs: map[string]string{"i0": "p2", "i1": "p5"}, Outputs: map[string]string{"i0": "p6"}}, - "7": {UniqueName: "g", Inputs: map[string]string{"i0": "p4", "i1": "p6", "i2": "p8"}, Outputs: map[string]string{"i0": "p7"}}, - "8": {UniqueName: "h", Inputs: map[string]string{"i0": "p7"}}, + }.Build(), + "4": pipepb.PTransform_builder{UniqueName: "d", Inputs: map[string]string{"i0": "p1"}, Outputs: map[string]string{"i0": "p2"}}.Build(), + "5": pipepb.PTransform_builder{UniqueName: "e", Inputs: map[string]string{"i0": "p2"}, Outputs: map[string]string{"i0": "p3", "i1": "p4"}}.Build(), + "6": pipepb.PTransform_builder{UniqueName: "f", Inputs: map[string]string{"i0": "p2", "i1": "p5"}, Outputs: map[string]string{"i0": "p6"}}.Build(), + "7": pipepb.PTransform_builder{UniqueName: "g", Inputs: map[string]string{"i0": "p4", "i1": "p6", "i2": "p8"}, Outputs: map[string]string{"i0": "p7"}}.Build(), + "8": pipepb.PTransform_builder{UniqueName: "h", Inputs: map[string]string{"i0": "p7"}}.Build(), }, exp: map[string]*pipepb.PTransform{ - "1": { + "1": pipepb.PTransform_builder{ UniqueName: "a", Subtransforms: []string{"2"}, Inputs: map[string]string{"p1": "p1", "p5": "p5", "p8": "p8"}, Outputs: map[string]string{"p3": "p3"}, - }, - "2": { + }.Build(), + "2": pipepb.PTransform_builder{ UniqueName: "b", Subtransforms: []string{"3", "7", "8"}, Inputs: map[string]string{"p1": "p1", "p5": "p5", "p8": "p8"}, Outputs: map[string]string{"p3": "p3"}, - }, - "3": { + }.Build(), + "3": pipepb.PTransform_builder{ UniqueName: "c", Subtransforms: []string{"4", "6", "5"}, // topologically sorted. Inputs: map[string]string{"p1": "p1", "p5": "p5"}, Outputs: map[string]string{"p4": "p4", "p6": "p6", "p3": "p3"}, - }, - "4": {UniqueName: "d", Inputs: map[string]string{"i0": "p1"}, Outputs: map[string]string{"i0": "p2"}}, - "5": {UniqueName: "e", Inputs: map[string]string{"i0": "p2"}, Outputs: map[string]string{"i0": "p3", "i1": "p4"}}, - "6": {UniqueName: "f", Inputs: map[string]string{"i0": "p2", "i1": "p5"}, Outputs: map[string]string{"i0": "p6"}}, - "7": {UniqueName: "g", Inputs: map[string]string{"i0": "p4", "i1": "p6", "i2": "p8"}, Outputs: map[string]string{"i0": "p7"}}, - "8": {UniqueName: "h", Inputs: map[string]string{"i0": "p7"}}, + }.Build(), + "4": pipepb.PTransform_builder{UniqueName: "d", Inputs: map[string]string{"i0": "p1"}, Outputs: map[string]string{"i0": "p2"}}.Build(), + "5": pipepb.PTransform_builder{UniqueName: "e", Inputs: map[string]string{"i0": "p2"}, Outputs: map[string]string{"i0": "p3", "i1": "p4"}}.Build(), + "6": pipepb.PTransform_builder{UniqueName: "f", Inputs: map[string]string{"i0": "p2", "i1": "p5"}, Outputs: map[string]string{"i0": "p6"}}.Build(), + "7": pipepb.PTransform_builder{UniqueName: "g", Inputs: map[string]string{"i0": "p4", "i1": "p6", "i2": "p8"}, Outputs: map[string]string{"i0": "p7"}}.Build(), + "8": pipepb.PTransform_builder{UniqueName: "h", Inputs: map[string]string{"i0": "p7"}}.Build(), }, }, { name: "sibling composite", in: map[string]*pipepb.PTransform{ - "1": { + "1": pipepb.PTransform_builder{ UniqueName: "a", Subtransforms: []string{"3", "4"}, - }, - "2": { + }.Build(), + "2": pipepb.PTransform_builder{ UniqueName: "b", Subtransforms: []string{"5"}, - }, - "3": {UniqueName: "c", Outputs: map[string]string{"i0": "p1"}}, - "4": {UniqueName: "d", Inputs: map[string]string{"i0": "p1"}}, - "5": {UniqueName: "e", Inputs: map[string]string{"i0": "p1"}}, + }.Build(), + "3": pipepb.PTransform_builder{UniqueName: "c", Outputs: map[string]string{"i0": "p1"}}.Build(), + "4": pipepb.PTransform_builder{UniqueName: "d", Inputs: map[string]string{"i0": "p1"}}.Build(), + "5": pipepb.PTransform_builder{UniqueName: "e", Inputs: map[string]string{"i0": "p1"}}.Build(), }, exp: map[string]*pipepb.PTransform{ - "1": { + "1": pipepb.PTransform_builder{ UniqueName: "a", Subtransforms: []string{"3", "4"}, Outputs: map[string]string{"p1": "p1"}, - }, - "2": { + }.Build(), + "2": pipepb.PTransform_builder{ UniqueName: "b", Subtransforms: []string{"5"}, Inputs: map[string]string{"p1": "p1"}, - }, - "3": {UniqueName: "c", Outputs: map[string]string{"i0": "p1"}}, - "4": {UniqueName: "d", Inputs: map[string]string{"i0": "p1"}}, - "5": {UniqueName: "e", Inputs: map[string]string{"i0": "p1"}}, + }.Build(), + "3": pipepb.PTransform_builder{UniqueName: "c", Outputs: map[string]string{"i0": "p1"}}.Build(), + "4": pipepb.PTransform_builder{UniqueName: "d", Inputs: map[string]string{"i0": "p1"}}.Build(), + "5": pipepb.PTransform_builder{UniqueName: "e", Inputs: map[string]string{"i0": "p1"}}.Build(), }, }, } @@ -317,15 +317,15 @@ func BenchmarkComputeInputOutput(b *testing.B) { for i := 0; i < 3000; i++ { compositeID := fmt.Sprintf("x%d", i) primitiveID := fmt.Sprintf("y%d", i) - in[compositeID] = &pipepb.PTransform{ + in[compositeID] = pipepb.PTransform_builder{ UniqueName: compositeID, Subtransforms: []string{primitiveID}, - } - in[primitiveID] = &pipepb.PTransform{ + }.Build() + in[primitiveID] = pipepb.PTransform_builder{ UniqueName: primitiveID, Inputs: map[string]string{"i0": fmt.Sprintf("p%d", i)}, Outputs: map[string]string{"i0": fmt.Sprintf("p%d", i+1)}, - } + }.Build() } b.ResetTimer() b.ReportAllocs() @@ -395,11 +395,11 @@ func TestApplySdkImageOverrides(t *testing.T) { wantEnvs[id] = env } - p := &pipepb.Pipeline{ - Components: &pipepb.Components{ + p := pipepb.Pipeline_builder{ + Components: pipepb.Components_builder{ Environments: envs, - }, - } + }.Build(), + }.Build() if err := ApplySdkImageOverrides(p, test.patterns); err != nil { t.Fatalf("ApplySdkImageOverrides failed: %v", err) } @@ -412,16 +412,16 @@ func TestApplySdkImageOverrides(t *testing.T) { func buildEnvironment(t *testing.T, containerImg string) *pipepb.Environment { t.Helper() - env := &pipepb.Environment{ + env := pipepb.Environment_builder{ Urn: "alpha", - DisplayData: []*pipepb.DisplayData{{Urn: "beta"}}, + DisplayData: []*pipepb.DisplayData{pipepb.DisplayData_builder{Urn: "beta"}.Build()}, Capabilities: []string{"delta", "gamma"}, - } - pl := pipepb.DockerPayload{ContainerImage: containerImg} - plb, err := proto.Marshal(&pl) + }.Build() + pl := pipepb.DockerPayload_builder{ContainerImage: containerImg}.Build() + plb, err := proto.Marshal(pl) if err != nil { t.Fatalf("Failed to marshal DockerPayload with container image %v: %v", containerImg, err) } - env.Payload = plb + env.SetPayload(plb) return env } diff --git a/sdks/go/pkg/beam/core/runtime/pipelinex/util.go b/sdks/go/pkg/beam/core/runtime/pipelinex/util.go index 4735e7b77d20..8ba87ff1e650 100644 --- a/sdks/go/pkg/beam/core/runtime/pipelinex/util.go +++ b/sdks/go/pkg/beam/core/runtime/pipelinex/util.go @@ -25,7 +25,7 @@ import ( // Bounded returns true iff all PCollections are bounded. func Bounded(p *pipepb.Pipeline) bool { for _, col := range p.GetComponents().GetPcollections() { - if col.IsBounded == pipepb.IsBounded_UNBOUNDED { + if col.GetIsBounded() == pipepb.IsBounded_UNBOUNDED { return false } } @@ -41,7 +41,7 @@ func ContainerImages(p *pipepb.Pipeline) []string { for _, t := range p.GetComponents().GetEnvironments() { var payload pipepb.DockerPayload proto.Unmarshal(t.GetPayload(), &payload) - ret = append(ret, payload.ContainerImage) + ret = append(ret, payload.GetContainerImage()) } return ret } @@ -75,7 +75,7 @@ func newVisiter(xforms map[string]*pipepb.PTransform, ids []string) *visiter { next: make(map[string][]string), } for _, id := range ids { - for _, in := range xforms[id].Inputs { + for _, in := range xforms[id].GetInputs() { ret.next[in] = append(ret.next[in], id) } } @@ -91,8 +91,8 @@ func (v *visiter) visit(xforms map[string]*pipepb.PTransform, id string) { } v.seen[id] = true // Deterministically iterate through the output keys. - outputKeys := make([]string, 0, len(xforms[id].Outputs)) - for _, k := range xforms[id].Outputs { + outputKeys := make([]string, 0, len(xforms[id].GetOutputs())) + for _, k := range xforms[id].GetOutputs() { outputKeys = append(outputKeys, k) } sort.Strings(outputKeys) diff --git a/sdks/go/pkg/beam/core/runtime/pipelinex/util_test.go b/sdks/go/pkg/beam/core/runtime/pipelinex/util_test.go index bdadd669629f..795e0fb76792 100644 --- a/sdks/go/pkg/beam/core/runtime/pipelinex/util_test.go +++ b/sdks/go/pkg/beam/core/runtime/pipelinex/util_test.go @@ -145,23 +145,23 @@ func ptSink(input string, sides ...string) *pipepb.PTransform { for i, s := range sides { ins[fmt.Sprintf("i%d", i+1)] = s } - return &pipepb.PTransform{ + return pipepb.PTransform_builder{ Inputs: ins, - } + }.Build() } // ptComp generates a composite PCollection. Unlike the other helpers, it takes in // the *subtransform ids* instead of the input/output pcollections. func ptComp(subs ...string) *pipepb.PTransform { - return &pipepb.PTransform{ + return pipepb.PTransform_builder{ Subtransforms: subs, - } + }.Build() } func ptImpulse(output string) *pipepb.PTransform { - return &pipepb.PTransform{ + return pipepb.PTransform_builder{ Outputs: map[string]string{"o0": output}, - } + }.Build() } func ptNoSide(input string, outputs ...string) *pipepb.PTransform { @@ -169,10 +169,10 @@ func ptNoSide(input string, outputs ...string) *pipepb.PTransform { for i, o := range outputs { outs[fmt.Sprintf("o%d", i)] = o } - return &pipepb.PTransform{ + return pipepb.PTransform_builder{ Inputs: map[string]string{"i0": input}, Outputs: outs, - } + }.Build() } // validateSortForTest ensures that in the sorted ids, outputs are declared before @@ -186,7 +186,7 @@ func validateSortForTest(t *testing.T, xforms map[string]*pipepb.PTransform, sor for _, id := range sorted { pt := xforms[id] - for _, in := range pt.Inputs { + for _, in := range pt.GetInputs() { if !seenPCollections[in] { t.Errorf("out of order pcollection %v required by %v", in, id) missingPCollections[in] = true diff --git a/sdks/go/pkg/beam/core/runtime/xlangx/expand.go b/sdks/go/pkg/beam/core/runtime/xlangx/expand.go index 94dda75e8051..4aae6062bbce 100644 --- a/sdks/go/pkg/beam/core/runtime/xlangx/expand.go +++ b/sdks/go/pkg/beam/core/runtime/xlangx/expand.go @@ -62,7 +62,7 @@ func Expand(edge *graph.MultiEdge, ext *graph.ExternalTransform) error { // should be removed from proto before submitting expansion request. extTransformID := p.GetRootTransformIds()[0] extTransform := transforms[extTransformID] - for extTransform.UniqueName != "External" { + for extTransform.GetUniqueName() != "External" { delete(transforms, extTransformID) p, err = pipelinex.Normalize(p) // Update root transform IDs. if err != nil { @@ -136,12 +136,12 @@ func expand( } return h(ctx, &HandlerParams{ Config: config, - Req: &jobpb.ExpansionRequest{ + Req: jobpb.ExpansionRequest_builder{ Components: comps, Transform: transform, Namespace: ext.Namespace, OutputCoderRequests: outputCoderID, - }, + }.Build(), edge: edge, ext: ext, }) diff --git a/sdks/go/pkg/beam/core/runtime/xlangx/namespace.go b/sdks/go/pkg/beam/core/runtime/xlangx/namespace.go index 723d2e25cce7..20d2acc7eb5f 100644 --- a/sdks/go/pkg/beam/core/runtime/xlangx/namespace.go +++ b/sdks/go/pkg/beam/core/runtime/xlangx/namespace.go @@ -27,27 +27,27 @@ func addCoderID(c *pipepb.Components, idMap map[string]string, cid string, newID return idMap[cid] } - coder, exists := c.Coders[cid] + coder, exists := c.GetCoders()[cid] if !exists { - panic(errors.Errorf("attempted to add namespace to missing coder id: %v not in %v", cid, c.Coders)) + panic(errors.Errorf("attempted to add namespace to missing coder id: %v not in %v", cid, c.GetCoders())) } // Updating ComponentCoderIDs of Coder if coder.GetComponentCoderIds() != nil { var updatedComponentCoderIDs []string - updatedComponentCoderIDs = append(updatedComponentCoderIDs, coder.ComponentCoderIds...) + updatedComponentCoderIDs = append(updatedComponentCoderIDs, coder.GetComponentCoderIds()...) - for i, ccid := range coder.ComponentCoderIds { + for i, ccid := range coder.GetComponentCoderIds() { updatedComponentCoderIDs[i] = addCoderID(c, idMap, ccid, newID) } - coder.ComponentCoderIds = updatedComponentCoderIDs + coder.SetComponentCoderIds(updatedComponentCoderIDs) } idMap[cid] = newID(cid) // Updating Coders map - c.Coders[idMap[cid]] = coder - delete(c.Coders, cid) + c.GetCoders()[idMap[cid]] = coder + delete(c.GetCoders(), cid) return idMap[cid] } @@ -57,21 +57,21 @@ func addWindowingStrategyID(c *pipepb.Components, idMap map[string]string, wid s return idMap[wid] } - windowingStrategy, exists := c.WindowingStrategies[wid] + windowingStrategy, exists := c.GetWindowingStrategies()[wid] if !exists { - panic(errors.Errorf("attempted to add namespace to missing windowing strategy id: %v not in %v", wid, c.WindowingStrategies)) + panic(errors.Errorf("attempted to add namespace to missing windowing strategy id: %v not in %v", wid, c.GetWindowingStrategies())) } // Updating WindowCoderID of WindowingStrategy - if windowingStrategy.WindowCoderId != "" { - windowingStrategy.WindowCoderId = addCoderID(c, idMap, windowingStrategy.WindowCoderId, newID) + if windowingStrategy.GetWindowCoderId() != "" { + windowingStrategy.SetWindowCoderId(addCoderID(c, idMap, windowingStrategy.GetWindowCoderId(), newID)) } idMap[wid] = newID(wid) // Updating WindowingStrategies map - c.WindowingStrategies[idMap[wid]] = windowingStrategy - delete(c.WindowingStrategies, wid) + c.GetWindowingStrategies()[idMap[wid]] = windowingStrategy + delete(c.GetWindowingStrategies(), wid) return idMap[wid] } @@ -88,14 +88,14 @@ func addNamespace(t *pipepb.PTransform, c *pipepb.Components, namespace string) // environment. If multiple Go SDK environments become possible, then // namespacing of non-default environments should happen here. - for _, pcolsMap := range []map[string]string{t.Inputs, t.Outputs} { + for _, pcolsMap := range []map[string]string{t.GetInputs(), t.GetOutputs()} { for _, pid := range pcolsMap { - if pcol, exists := c.Pcollections[pid]; exists { + if pcol, exists := c.GetPcollections()[pid]; exists { // Update Coder ID of PCollection - pcol.CoderId = addCoderID(c, idMap, pcol.CoderId, newID) + pcol.SetCoderId(addCoderID(c, idMap, pcol.GetCoderId(), newID)) // Update WindowingStrategyID of PCollection - pcol.WindowingStrategyId = addWindowingStrategyID(c, idMap, pcol.WindowingStrategyId, newID) + pcol.SetWindowingStrategyId(addWindowingStrategyID(c, idMap, pcol.GetWindowingStrategyId(), newID)) } } } @@ -106,32 +106,32 @@ func addNamespace(t *pipepb.PTransform, c *pipepb.Components, namespace string) continue } var updatedComponentCoderIDs []string - updatedComponentCoderIDs = append(updatedComponentCoderIDs, coder.ComponentCoderIds...) + updatedComponentCoderIDs = append(updatedComponentCoderIDs, coder.GetComponentCoderIds()...) for i, ccid := range coder.GetComponentCoderIds() { if _, exists := idMap[ccid]; exists { updatedComponentCoderIDs[i] = idMap[ccid] } } - coder.ComponentCoderIds = updatedComponentCoderIDs + coder.SetComponentCoderIds(updatedComponentCoderIDs) } - sourceName := t.UniqueName - for _, t := range c.Transforms { - if t.UniqueName != sourceName { - if id, exists := idMap[t.EnvironmentId]; exists { - t.EnvironmentId = id + sourceName := t.GetUniqueName() + for _, t := range c.GetTransforms() { + if t.GetUniqueName() != sourceName { + if id, exists := idMap[t.GetEnvironmentId()]; exists { + t.SetEnvironmentId(id) } - for _, pcolsMap := range []map[string]string{t.Inputs, t.Outputs} { + for _, pcolsMap := range []map[string]string{t.GetInputs(), t.GetOutputs()} { for _, pid := range pcolsMap { - if pcol, exists := c.Pcollections[pid]; exists { + if pcol, exists := c.GetPcollections()[pid]; exists { // Update Coder ID of PCollection - if id, exists := idMap[pcol.CoderId]; exists { - pcol.CoderId = id + if id, exists := idMap[pcol.GetCoderId()]; exists { + pcol.SetCoderId(id) } // Update WindowingStrategyID of PCollection - if id, exists := idMap[pcol.WindowingStrategyId]; exists { - pcol.WindowingStrategyId = id + if id, exists := idMap[pcol.GetWindowingStrategyId()]; exists { + pcol.SetWindowingStrategyId(id) } } } diff --git a/sdks/go/pkg/beam/core/runtime/xlangx/namespace_test.go b/sdks/go/pkg/beam/core/runtime/xlangx/namespace_test.go index e2ee43780bc2..eb98d0d7e7de 100644 --- a/sdks/go/pkg/beam/core/runtime/xlangx/namespace_test.go +++ b/sdks/go/pkg/beam/core/runtime/xlangx/namespace_test.go @@ -41,515 +41,515 @@ func TestAddNamespace(t *testing.T) { }{ { name: "[Correctness] Single Input Multi Output", - init: &pipepb.Components{ + init: pipepb.Components_builder{ Transforms: map[string]*pipepb.PTransform{ - "t0": { + "t0": pipepb.PTransform_builder{ UniqueName: "t0", Inputs: map[string]string{"t0i0": "p0"}, Outputs: map[string]string{"t0o0": "p1", "t0o1": "p2"}, EnvironmentId: "e0", - }, - "t1": { + }.Build(), + "t1": pipepb.PTransform_builder{ UniqueName: "t1", Inputs: map[string]string{"t1i0": "p1"}, Outputs: map[string]string{"t1o0": "p3"}, EnvironmentId: "e1", - }, - "t2": { + }.Build(), + "t2": pipepb.PTransform_builder{ UniqueName: "t2", Inputs: map[string]string{"t2i0": "p2"}, Outputs: map[string]string{"t2o0": "p4"}, EnvironmentId: "e0", - }, - "t3": { + }.Build(), + "t3": pipepb.PTransform_builder{ UniqueName: "t3", Inputs: map[string]string{"t3i0": "p3", "t3i1": "p4"}, Outputs: map[string]string{"t3o0": "p5"}, EnvironmentId: "e1", - }, + }.Build(), }, Pcollections: map[string]*pipepb.PCollection{ - "p0": {CoderId: "c0", WindowingStrategyId: "w1"}, - "p1": {CoderId: "c1", WindowingStrategyId: "w1"}, - "p2": {CoderId: "c0", WindowingStrategyId: "w1"}, - "p3": {CoderId: "c3", WindowingStrategyId: "w1"}, - "p4": {CoderId: "c2", WindowingStrategyId: "w0"}, - "p5": {CoderId: "c2", WindowingStrategyId: "w1"}, + "p0": pipepb.PCollection_builder{CoderId: "c0", WindowingStrategyId: "w1"}.Build(), + "p1": pipepb.PCollection_builder{CoderId: "c1", WindowingStrategyId: "w1"}.Build(), + "p2": pipepb.PCollection_builder{CoderId: "c0", WindowingStrategyId: "w1"}.Build(), + "p3": pipepb.PCollection_builder{CoderId: "c3", WindowingStrategyId: "w1"}.Build(), + "p4": pipepb.PCollection_builder{CoderId: "c2", WindowingStrategyId: "w0"}.Build(), + "p5": pipepb.PCollection_builder{CoderId: "c2", WindowingStrategyId: "w1"}.Build(), }, WindowingStrategies: map[string]*pipepb.WindowingStrategy{ - "w0": {WindowCoderId: "c3", EnvironmentId: "e0"}, - "w1": {WindowCoderId: "c4", EnvironmentId: "e1"}, + "w0": pipepb.WindowingStrategy_builder{WindowCoderId: "c3", EnvironmentId: "e0"}.Build(), + "w1": pipepb.WindowingStrategy_builder{WindowCoderId: "c4", EnvironmentId: "e1"}.Build(), }, Coders: map[string]*pipepb.Coder{ - "c0": {Spec: &pipepb.FunctionSpec{Urn: "c0"}}, - "c1": {Spec: &pipepb.FunctionSpec{Urn: "c1"}}, - "c2": {Spec: &pipepb.FunctionSpec{Urn: "c2"}}, - "c3": {Spec: &pipepb.FunctionSpec{Urn: "c3"}}, - "c4": {Spec: &pipepb.FunctionSpec{Urn: "c4"}}, + "c0": pipepb.Coder_builder{Spec: pipepb.FunctionSpec_builder{Urn: "c0"}.Build()}.Build(), + "c1": pipepb.Coder_builder{Spec: pipepb.FunctionSpec_builder{Urn: "c1"}.Build()}.Build(), + "c2": pipepb.Coder_builder{Spec: pipepb.FunctionSpec_builder{Urn: "c2"}.Build()}.Build(), + "c3": pipepb.Coder_builder{Spec: pipepb.FunctionSpec_builder{Urn: "c3"}.Build()}.Build(), + "c4": pipepb.Coder_builder{Spec: pipepb.FunctionSpec_builder{Urn: "c4"}.Build()}.Build(), }, Environments: map[string]*pipepb.Environment{ - "e0": {Urn: "e0"}, - "e1": {Urn: "e1"}, + "e0": pipepb.Environment_builder{Urn: "e0"}.Build(), + "e1": pipepb.Environment_builder{Urn: "e1"}.Build(), }, - }, + }.Build(), namespace: "daASxQwenJ", transformID: "t0", - want: &pipepb.Components{ + want: pipepb.Components_builder{ Transforms: map[string]*pipepb.PTransform{ - "t0": { + "t0": pipepb.PTransform_builder{ UniqueName: "t0", Inputs: map[string]string{"t0i0": "p0"}, Outputs: map[string]string{"t0o0": "p1", "t0o1": "p2"}, EnvironmentId: "e0", - }, - "t1": { + }.Build(), + "t1": pipepb.PTransform_builder{ UniqueName: "t1", Inputs: map[string]string{"t1i0": "p1"}, Outputs: map[string]string{"t1o0": "p3"}, EnvironmentId: "e1", - }, - "t2": { + }.Build(), + "t2": pipepb.PTransform_builder{ UniqueName: "t2", Inputs: map[string]string{"t2i0": "p2"}, Outputs: map[string]string{"t2o0": "p4"}, EnvironmentId: "e0", - }, - "t3": { + }.Build(), + "t3": pipepb.PTransform_builder{ UniqueName: "t3", Inputs: map[string]string{"t3i0": "p3", "t3i1": "p4"}, Outputs: map[string]string{"t3o0": "p5"}, EnvironmentId: "e1", - }, + }.Build(), }, Pcollections: map[string]*pipepb.PCollection{ - "p0": {CoderId: "c0@daASxQwenJ", WindowingStrategyId: "w1@daASxQwenJ"}, - "p1": {CoderId: "c1@daASxQwenJ", WindowingStrategyId: "w1@daASxQwenJ"}, - "p2": {CoderId: "c0@daASxQwenJ", WindowingStrategyId: "w1@daASxQwenJ"}, - "p3": {CoderId: "c3", WindowingStrategyId: "w1@daASxQwenJ"}, - "p4": {CoderId: "c2", WindowingStrategyId: "w0"}, - "p5": {CoderId: "c2", WindowingStrategyId: "w1@daASxQwenJ"}, + "p0": pipepb.PCollection_builder{CoderId: "c0@daASxQwenJ", WindowingStrategyId: "w1@daASxQwenJ"}.Build(), + "p1": pipepb.PCollection_builder{CoderId: "c1@daASxQwenJ", WindowingStrategyId: "w1@daASxQwenJ"}.Build(), + "p2": pipepb.PCollection_builder{CoderId: "c0@daASxQwenJ", WindowingStrategyId: "w1@daASxQwenJ"}.Build(), + "p3": pipepb.PCollection_builder{CoderId: "c3", WindowingStrategyId: "w1@daASxQwenJ"}.Build(), + "p4": pipepb.PCollection_builder{CoderId: "c2", WindowingStrategyId: "w0"}.Build(), + "p5": pipepb.PCollection_builder{CoderId: "c2", WindowingStrategyId: "w1@daASxQwenJ"}.Build(), }, WindowingStrategies: map[string]*pipepb.WindowingStrategy{ - "w0": {WindowCoderId: "c3", EnvironmentId: "e0"}, - "w1@daASxQwenJ": {WindowCoderId: "c4@daASxQwenJ", EnvironmentId: "e1"}, + "w0": pipepb.WindowingStrategy_builder{WindowCoderId: "c3", EnvironmentId: "e0"}.Build(), + "w1@daASxQwenJ": pipepb.WindowingStrategy_builder{WindowCoderId: "c4@daASxQwenJ", EnvironmentId: "e1"}.Build(), }, Coders: map[string]*pipepb.Coder{ - "c0@daASxQwenJ": {Spec: &pipepb.FunctionSpec{Urn: "c0"}}, - "c1@daASxQwenJ": {Spec: &pipepb.FunctionSpec{Urn: "c1"}}, - "c2": {Spec: &pipepb.FunctionSpec{Urn: "c2"}}, - "c3": {Spec: &pipepb.FunctionSpec{Urn: "c3"}}, - "c4@daASxQwenJ": {Spec: &pipepb.FunctionSpec{Urn: "c4"}}, + "c0@daASxQwenJ": pipepb.Coder_builder{Spec: pipepb.FunctionSpec_builder{Urn: "c0"}.Build()}.Build(), + "c1@daASxQwenJ": pipepb.Coder_builder{Spec: pipepb.FunctionSpec_builder{Urn: "c1"}.Build()}.Build(), + "c2": pipepb.Coder_builder{Spec: pipepb.FunctionSpec_builder{Urn: "c2"}.Build()}.Build(), + "c3": pipepb.Coder_builder{Spec: pipepb.FunctionSpec_builder{Urn: "c3"}.Build()}.Build(), + "c4@daASxQwenJ": pipepb.Coder_builder{Spec: pipepb.FunctionSpec_builder{Urn: "c4"}.Build()}.Build(), }, Environments: map[string]*pipepb.Environment{ - "e0": {Urn: "e0"}, - "e1": {Urn: "e1"}, + "e0": pipepb.Environment_builder{Urn: "e0"}.Build(), + "e1": pipepb.Environment_builder{Urn: "e1"}.Build(), }, - }, + }.Build(), }, { name: "[Correctness] Single Input Single Output", - init: &pipepb.Components{ + init: pipepb.Components_builder{ Transforms: map[string]*pipepb.PTransform{ - "t0": { + "t0": pipepb.PTransform_builder{ UniqueName: "t0", Inputs: map[string]string{"t0i0": "p0"}, Outputs: map[string]string{"t0o0": "p1", "t0o1": "p2"}, EnvironmentId: "e0", - }, - "t1": { + }.Build(), + "t1": pipepb.PTransform_builder{ UniqueName: "t1", Inputs: map[string]string{"t1i0": "p1"}, Outputs: map[string]string{"t1o0": "p3"}, EnvironmentId: "e1", - }, - "t2": { + }.Build(), + "t2": pipepb.PTransform_builder{ UniqueName: "t2", Inputs: map[string]string{"t2i0": "p2"}, Outputs: map[string]string{"t2o0": "p4"}, EnvironmentId: "e0", - }, - "t3": { + }.Build(), + "t3": pipepb.PTransform_builder{ UniqueName: "t3", Inputs: map[string]string{"t3i0": "p3", "t3i1": "p4"}, Outputs: map[string]string{"t3o0": "p5"}, EnvironmentId: "e1", - }, + }.Build(), }, Pcollections: map[string]*pipepb.PCollection{ - "p0": {CoderId: "c0", WindowingStrategyId: "w1"}, - "p1": {CoderId: "c1", WindowingStrategyId: "w1"}, - "p2": {CoderId: "c0", WindowingStrategyId: "w1"}, - "p3": {CoderId: "c3", WindowingStrategyId: "w1"}, - "p4": {CoderId: "c2", WindowingStrategyId: "w0"}, - "p5": {CoderId: "c2", WindowingStrategyId: "w1"}, + "p0": pipepb.PCollection_builder{CoderId: "c0", WindowingStrategyId: "w1"}.Build(), + "p1": pipepb.PCollection_builder{CoderId: "c1", WindowingStrategyId: "w1"}.Build(), + "p2": pipepb.PCollection_builder{CoderId: "c0", WindowingStrategyId: "w1"}.Build(), + "p3": pipepb.PCollection_builder{CoderId: "c3", WindowingStrategyId: "w1"}.Build(), + "p4": pipepb.PCollection_builder{CoderId: "c2", WindowingStrategyId: "w0"}.Build(), + "p5": pipepb.PCollection_builder{CoderId: "c2", WindowingStrategyId: "w1"}.Build(), }, WindowingStrategies: map[string]*pipepb.WindowingStrategy{ - "w0": {WindowCoderId: "c3", EnvironmentId: "e0"}, - "w1": {WindowCoderId: "c4", EnvironmentId: "e1"}, + "w0": pipepb.WindowingStrategy_builder{WindowCoderId: "c3", EnvironmentId: "e0"}.Build(), + "w1": pipepb.WindowingStrategy_builder{WindowCoderId: "c4", EnvironmentId: "e1"}.Build(), }, Coders: map[string]*pipepb.Coder{ - "c0": {Spec: &pipepb.FunctionSpec{Urn: "c0"}}, - "c1": {Spec: &pipepb.FunctionSpec{Urn: "c1"}}, - "c2": {Spec: &pipepb.FunctionSpec{Urn: "c2"}}, - "c3": {Spec: &pipepb.FunctionSpec{Urn: "c3"}}, - "c4": {Spec: &pipepb.FunctionSpec{Urn: "c4"}}, + "c0": pipepb.Coder_builder{Spec: pipepb.FunctionSpec_builder{Urn: "c0"}.Build()}.Build(), + "c1": pipepb.Coder_builder{Spec: pipepb.FunctionSpec_builder{Urn: "c1"}.Build()}.Build(), + "c2": pipepb.Coder_builder{Spec: pipepb.FunctionSpec_builder{Urn: "c2"}.Build()}.Build(), + "c3": pipepb.Coder_builder{Spec: pipepb.FunctionSpec_builder{Urn: "c3"}.Build()}.Build(), + "c4": pipepb.Coder_builder{Spec: pipepb.FunctionSpec_builder{Urn: "c4"}.Build()}.Build(), }, Environments: map[string]*pipepb.Environment{ - "e0": {Urn: "e0"}, - "e1": {Urn: "e1"}, + "e0": pipepb.Environment_builder{Urn: "e0"}.Build(), + "e1": pipepb.Environment_builder{Urn: "e1"}.Build(), }, - }, + }.Build(), namespace: "daASxQwenJ", transformID: "t1", - want: &pipepb.Components{ + want: pipepb.Components_builder{ Transforms: map[string]*pipepb.PTransform{ - "t0": { + "t0": pipepb.PTransform_builder{ UniqueName: "t0", Inputs: map[string]string{"t0i0": "p0"}, Outputs: map[string]string{"t0o0": "p1", "t0o1": "p2"}, EnvironmentId: "e0", - }, - "t1": { + }.Build(), + "t1": pipepb.PTransform_builder{ UniqueName: "t1", Inputs: map[string]string{"t1i0": "p1"}, Outputs: map[string]string{"t1o0": "p3"}, EnvironmentId: "e1", - }, - "t2": { + }.Build(), + "t2": pipepb.PTransform_builder{ UniqueName: "t2", Inputs: map[string]string{"t2i0": "p2"}, Outputs: map[string]string{"t2o0": "p4"}, EnvironmentId: "e0", - }, - "t3": { + }.Build(), + "t3": pipepb.PTransform_builder{ UniqueName: "t3", Inputs: map[string]string{"t3i0": "p3", "t3i1": "p4"}, Outputs: map[string]string{"t3o0": "p5"}, EnvironmentId: "e1", - }, + }.Build(), }, Pcollections: map[string]*pipepb.PCollection{ - "p0": {CoderId: "c0", WindowingStrategyId: "w1@daASxQwenJ"}, - "p1": {CoderId: "c1@daASxQwenJ", WindowingStrategyId: "w1@daASxQwenJ"}, - "p2": {CoderId: "c0", WindowingStrategyId: "w1@daASxQwenJ"}, - "p3": {CoderId: "c3@daASxQwenJ", WindowingStrategyId: "w1@daASxQwenJ"}, - "p4": {CoderId: "c2", WindowingStrategyId: "w0"}, - "p5": {CoderId: "c2", WindowingStrategyId: "w1@daASxQwenJ"}, + "p0": pipepb.PCollection_builder{CoderId: "c0", WindowingStrategyId: "w1@daASxQwenJ"}.Build(), + "p1": pipepb.PCollection_builder{CoderId: "c1@daASxQwenJ", WindowingStrategyId: "w1@daASxQwenJ"}.Build(), + "p2": pipepb.PCollection_builder{CoderId: "c0", WindowingStrategyId: "w1@daASxQwenJ"}.Build(), + "p3": pipepb.PCollection_builder{CoderId: "c3@daASxQwenJ", WindowingStrategyId: "w1@daASxQwenJ"}.Build(), + "p4": pipepb.PCollection_builder{CoderId: "c2", WindowingStrategyId: "w0"}.Build(), + "p5": pipepb.PCollection_builder{CoderId: "c2", WindowingStrategyId: "w1@daASxQwenJ"}.Build(), }, WindowingStrategies: map[string]*pipepb.WindowingStrategy{ - "w0": {WindowCoderId: "c3", EnvironmentId: "e0"}, - "w1@daASxQwenJ": {WindowCoderId: "c4@daASxQwenJ", EnvironmentId: "e1"}, + "w0": pipepb.WindowingStrategy_builder{WindowCoderId: "c3", EnvironmentId: "e0"}.Build(), + "w1@daASxQwenJ": pipepb.WindowingStrategy_builder{WindowCoderId: "c4@daASxQwenJ", EnvironmentId: "e1"}.Build(), }, Coders: map[string]*pipepb.Coder{ - "c0": {Spec: &pipepb.FunctionSpec{Urn: "c0"}}, - "c1@daASxQwenJ": {Spec: &pipepb.FunctionSpec{Urn: "c1"}}, - "c2": {Spec: &pipepb.FunctionSpec{Urn: "c2"}}, - "c3@daASxQwenJ": {Spec: &pipepb.FunctionSpec{Urn: "c3"}}, - "c4@daASxQwenJ": {Spec: &pipepb.FunctionSpec{Urn: "c4"}}, + "c0": pipepb.Coder_builder{Spec: pipepb.FunctionSpec_builder{Urn: "c0"}.Build()}.Build(), + "c1@daASxQwenJ": pipepb.Coder_builder{Spec: pipepb.FunctionSpec_builder{Urn: "c1"}.Build()}.Build(), + "c2": pipepb.Coder_builder{Spec: pipepb.FunctionSpec_builder{Urn: "c2"}.Build()}.Build(), + "c3@daASxQwenJ": pipepb.Coder_builder{Spec: pipepb.FunctionSpec_builder{Urn: "c3"}.Build()}.Build(), + "c4@daASxQwenJ": pipepb.Coder_builder{Spec: pipepb.FunctionSpec_builder{Urn: "c4"}.Build()}.Build(), }, Environments: map[string]*pipepb.Environment{ - "e0": {Urn: "e0"}, - "e1": {Urn: "e1"}, + "e0": pipepb.Environment_builder{Urn: "e0"}.Build(), + "e1": pipepb.Environment_builder{Urn: "e1"}.Build(), }, - }, + }.Build(), }, { name: "[Correctness] Multi Input Single Output", - init: &pipepb.Components{ + init: pipepb.Components_builder{ Transforms: map[string]*pipepb.PTransform{ - "t0": { + "t0": pipepb.PTransform_builder{ UniqueName: "t0", Inputs: map[string]string{"t0i0": "p0"}, Outputs: map[string]string{"t0o0": "p1", "t0o1": "p2"}, EnvironmentId: "e0", - }, - "t1": { + }.Build(), + "t1": pipepb.PTransform_builder{ UniqueName: "t1", Inputs: map[string]string{"t1i0": "p1"}, Outputs: map[string]string{"t1o0": "p3"}, EnvironmentId: "e1", - }, - "t2": { + }.Build(), + "t2": pipepb.PTransform_builder{ UniqueName: "t2", Inputs: map[string]string{"t2i0": "p2"}, Outputs: map[string]string{"t2o0": "p4"}, EnvironmentId: "e0", - }, - "t3": { + }.Build(), + "t3": pipepb.PTransform_builder{ UniqueName: "t3", Inputs: map[string]string{"t3i0": "p3", "t3i1": "p4"}, Outputs: map[string]string{"t3o0": "p5"}, EnvironmentId: "e1", - }, + }.Build(), }, Pcollections: map[string]*pipepb.PCollection{ - "p0": {CoderId: "c0", WindowingStrategyId: "w1"}, - "p1": {CoderId: "c1", WindowingStrategyId: "w1"}, - "p2": {CoderId: "c0", WindowingStrategyId: "w1"}, - "p3": {CoderId: "c3", WindowingStrategyId: "w1"}, - "p4": {CoderId: "c2", WindowingStrategyId: "w0"}, - "p5": {CoderId: "c2", WindowingStrategyId: "w1"}, + "p0": pipepb.PCollection_builder{CoderId: "c0", WindowingStrategyId: "w1"}.Build(), + "p1": pipepb.PCollection_builder{CoderId: "c1", WindowingStrategyId: "w1"}.Build(), + "p2": pipepb.PCollection_builder{CoderId: "c0", WindowingStrategyId: "w1"}.Build(), + "p3": pipepb.PCollection_builder{CoderId: "c3", WindowingStrategyId: "w1"}.Build(), + "p4": pipepb.PCollection_builder{CoderId: "c2", WindowingStrategyId: "w0"}.Build(), + "p5": pipepb.PCollection_builder{CoderId: "c2", WindowingStrategyId: "w1"}.Build(), }, WindowingStrategies: map[string]*pipepb.WindowingStrategy{ - "w0": {WindowCoderId: "c3", EnvironmentId: "e0"}, - "w1": {WindowCoderId: "c4", EnvironmentId: "e1"}, + "w0": pipepb.WindowingStrategy_builder{WindowCoderId: "c3", EnvironmentId: "e0"}.Build(), + "w1": pipepb.WindowingStrategy_builder{WindowCoderId: "c4", EnvironmentId: "e1"}.Build(), }, Coders: map[string]*pipepb.Coder{ - "c0": {Spec: &pipepb.FunctionSpec{Urn: "c0"}}, - "c1": {Spec: &pipepb.FunctionSpec{Urn: "c1"}}, - "c2": {Spec: &pipepb.FunctionSpec{Urn: "c2"}}, - "c3": {Spec: &pipepb.FunctionSpec{Urn: "c3"}}, - "c4": {Spec: &pipepb.FunctionSpec{Urn: "c4"}}, + "c0": pipepb.Coder_builder{Spec: pipepb.FunctionSpec_builder{Urn: "c0"}.Build()}.Build(), + "c1": pipepb.Coder_builder{Spec: pipepb.FunctionSpec_builder{Urn: "c1"}.Build()}.Build(), + "c2": pipepb.Coder_builder{Spec: pipepb.FunctionSpec_builder{Urn: "c2"}.Build()}.Build(), + "c3": pipepb.Coder_builder{Spec: pipepb.FunctionSpec_builder{Urn: "c3"}.Build()}.Build(), + "c4": pipepb.Coder_builder{Spec: pipepb.FunctionSpec_builder{Urn: "c4"}.Build()}.Build(), }, Environments: map[string]*pipepb.Environment{ - "e0": {Urn: "e0"}, - "e1": {Urn: "e1"}, + "e0": pipepb.Environment_builder{Urn: "e0"}.Build(), + "e1": pipepb.Environment_builder{Urn: "e1"}.Build(), }, - }, + }.Build(), namespace: "daASxQwenJ", transformID: "t3", - want: &pipepb.Components{ + want: pipepb.Components_builder{ Transforms: map[string]*pipepb.PTransform{ - "t0": { + "t0": pipepb.PTransform_builder{ UniqueName: "t0", Inputs: map[string]string{"t0i0": "p0"}, Outputs: map[string]string{"t0o0": "p1", "t0o1": "p2"}, EnvironmentId: "e0", - }, - "t1": { + }.Build(), + "t1": pipepb.PTransform_builder{ UniqueName: "t1", Inputs: map[string]string{"t1i0": "p1"}, Outputs: map[string]string{"t1o0": "p3"}, EnvironmentId: "e1", - }, - "t2": { + }.Build(), + "t2": pipepb.PTransform_builder{ UniqueName: "t2", Inputs: map[string]string{"t2i0": "p2"}, Outputs: map[string]string{"t2o0": "p4"}, EnvironmentId: "e0", - }, - "t3": { + }.Build(), + "t3": pipepb.PTransform_builder{ UniqueName: "t3", Inputs: map[string]string{"t3i0": "p3", "t3i1": "p4"}, Outputs: map[string]string{"t3o0": "p5"}, EnvironmentId: "e1", - }, + }.Build(), }, Pcollections: map[string]*pipepb.PCollection{ - "p0": {CoderId: "c0", WindowingStrategyId: "w1@daASxQwenJ"}, - "p1": {CoderId: "c1", WindowingStrategyId: "w1@daASxQwenJ"}, - "p2": {CoderId: "c0", WindowingStrategyId: "w1@daASxQwenJ"}, - "p3": {CoderId: "c3@daASxQwenJ", WindowingStrategyId: "w1@daASxQwenJ"}, - "p4": {CoderId: "c2@daASxQwenJ", WindowingStrategyId: "w0@daASxQwenJ"}, - "p5": {CoderId: "c2@daASxQwenJ", WindowingStrategyId: "w1@daASxQwenJ"}, + "p0": pipepb.PCollection_builder{CoderId: "c0", WindowingStrategyId: "w1@daASxQwenJ"}.Build(), + "p1": pipepb.PCollection_builder{CoderId: "c1", WindowingStrategyId: "w1@daASxQwenJ"}.Build(), + "p2": pipepb.PCollection_builder{CoderId: "c0", WindowingStrategyId: "w1@daASxQwenJ"}.Build(), + "p3": pipepb.PCollection_builder{CoderId: "c3@daASxQwenJ", WindowingStrategyId: "w1@daASxQwenJ"}.Build(), + "p4": pipepb.PCollection_builder{CoderId: "c2@daASxQwenJ", WindowingStrategyId: "w0@daASxQwenJ"}.Build(), + "p5": pipepb.PCollection_builder{CoderId: "c2@daASxQwenJ", WindowingStrategyId: "w1@daASxQwenJ"}.Build(), }, WindowingStrategies: map[string]*pipepb.WindowingStrategy{ - "w0@daASxQwenJ": {WindowCoderId: "c3@daASxQwenJ", EnvironmentId: "e0"}, - "w1@daASxQwenJ": {WindowCoderId: "c4@daASxQwenJ", EnvironmentId: "e1"}, + "w0@daASxQwenJ": pipepb.WindowingStrategy_builder{WindowCoderId: "c3@daASxQwenJ", EnvironmentId: "e0"}.Build(), + "w1@daASxQwenJ": pipepb.WindowingStrategy_builder{WindowCoderId: "c4@daASxQwenJ", EnvironmentId: "e1"}.Build(), }, Coders: map[string]*pipepb.Coder{ - "c0": {Spec: &pipepb.FunctionSpec{Urn: "c0"}}, - "c1": {Spec: &pipepb.FunctionSpec{Urn: "c1"}}, - "c2@daASxQwenJ": {Spec: &pipepb.FunctionSpec{Urn: "c2"}}, - "c3@daASxQwenJ": {Spec: &pipepb.FunctionSpec{Urn: "c3"}}, - "c4@daASxQwenJ": {Spec: &pipepb.FunctionSpec{Urn: "c4"}}, + "c0": pipepb.Coder_builder{Spec: pipepb.FunctionSpec_builder{Urn: "c0"}.Build()}.Build(), + "c1": pipepb.Coder_builder{Spec: pipepb.FunctionSpec_builder{Urn: "c1"}.Build()}.Build(), + "c2@daASxQwenJ": pipepb.Coder_builder{Spec: pipepb.FunctionSpec_builder{Urn: "c2"}.Build()}.Build(), + "c3@daASxQwenJ": pipepb.Coder_builder{Spec: pipepb.FunctionSpec_builder{Urn: "c3"}.Build()}.Build(), + "c4@daASxQwenJ": pipepb.Coder_builder{Spec: pipepb.FunctionSpec_builder{Urn: "c4"}.Build()}.Build(), }, Environments: map[string]*pipepb.Environment{ - "e0": {Urn: "e0"}, - "e1": {Urn: "e1"}, + "e0": pipepb.Environment_builder{Urn: "e0"}.Build(), + "e1": pipepb.Environment_builder{Urn: "e1"}.Build(), }, - }, + }.Build(), }, { name: "[Correctness] Component Coders", - init: &pipepb.Components{ + init: pipepb.Components_builder{ Transforms: map[string]*pipepb.PTransform{ - "t0": { + "t0": pipepb.PTransform_builder{ UniqueName: "t0", Inputs: map[string]string{"t0i0": "p0"}, Outputs: map[string]string{"t0o0": "p1"}, EnvironmentId: "e0", - }, - "t1": { + }.Build(), + "t1": pipepb.PTransform_builder{ UniqueName: "t1", Inputs: map[string]string{"t1i0": "p1"}, Outputs: map[string]string{"t1o0": "p2"}, EnvironmentId: "e1", - }, + }.Build(), }, Pcollections: map[string]*pipepb.PCollection{ - "p0": {CoderId: "c0", WindowingStrategyId: "w0"}, - "p1": {CoderId: "c1", WindowingStrategyId: "w1"}, - "p2": {CoderId: "c0", WindowingStrategyId: "w1"}, + "p0": pipepb.PCollection_builder{CoderId: "c0", WindowingStrategyId: "w0"}.Build(), + "p1": pipepb.PCollection_builder{CoderId: "c1", WindowingStrategyId: "w1"}.Build(), + "p2": pipepb.PCollection_builder{CoderId: "c0", WindowingStrategyId: "w1"}.Build(), }, WindowingStrategies: map[string]*pipepb.WindowingStrategy{ - "w0": {WindowCoderId: "c3", EnvironmentId: "e0"}, - "w1": {WindowCoderId: "c4", EnvironmentId: "e1"}, + "w0": pipepb.WindowingStrategy_builder{WindowCoderId: "c3", EnvironmentId: "e0"}.Build(), + "w1": pipepb.WindowingStrategy_builder{WindowCoderId: "c4", EnvironmentId: "e1"}.Build(), }, Coders: map[string]*pipepb.Coder{ - "c0": {Spec: &pipepb.FunctionSpec{Urn: "c0"}, ComponentCoderIds: []string{"c2"}}, - "c1": {Spec: &pipepb.FunctionSpec{Urn: "c1"}}, - "c2": {Spec: &pipepb.FunctionSpec{Urn: "c2"}}, - "c3": {Spec: &pipepb.FunctionSpec{Urn: "c3"}}, - "c4": {Spec: &pipepb.FunctionSpec{Urn: "c4"}}, + "c0": pipepb.Coder_builder{Spec: pipepb.FunctionSpec_builder{Urn: "c0"}.Build(), ComponentCoderIds: []string{"c2"}}.Build(), + "c1": pipepb.Coder_builder{Spec: pipepb.FunctionSpec_builder{Urn: "c1"}.Build()}.Build(), + "c2": pipepb.Coder_builder{Spec: pipepb.FunctionSpec_builder{Urn: "c2"}.Build()}.Build(), + "c3": pipepb.Coder_builder{Spec: pipepb.FunctionSpec_builder{Urn: "c3"}.Build()}.Build(), + "c4": pipepb.Coder_builder{Spec: pipepb.FunctionSpec_builder{Urn: "c4"}.Build()}.Build(), }, Environments: map[string]*pipepb.Environment{ - "e0": {Urn: "e0"}, - "e1": {Urn: "e1"}, + "e0": pipepb.Environment_builder{Urn: "e0"}.Build(), + "e1": pipepb.Environment_builder{Urn: "e1"}.Build(), }, - }, + }.Build(), namespace: "daASxQwenJ", transformID: "t0", - want: &pipepb.Components{ + want: pipepb.Components_builder{ Transforms: map[string]*pipepb.PTransform{ - "t0": { + "t0": pipepb.PTransform_builder{ UniqueName: "t0", Inputs: map[string]string{"t0i0": "p0"}, Outputs: map[string]string{"t0o0": "p1"}, EnvironmentId: "e0", - }, - "t1": { + }.Build(), + "t1": pipepb.PTransform_builder{ UniqueName: "t1", Inputs: map[string]string{"t1i0": "p1"}, Outputs: map[string]string{"t1o0": "p2"}, EnvironmentId: "e1", - }, + }.Build(), }, Pcollections: map[string]*pipepb.PCollection{ - "p0": {CoderId: "c0@daASxQwenJ", WindowingStrategyId: "w0@daASxQwenJ"}, - "p1": {CoderId: "c1@daASxQwenJ", WindowingStrategyId: "w1@daASxQwenJ"}, - "p2": {CoderId: "c0@daASxQwenJ", WindowingStrategyId: "w1@daASxQwenJ"}, + "p0": pipepb.PCollection_builder{CoderId: "c0@daASxQwenJ", WindowingStrategyId: "w0@daASxQwenJ"}.Build(), + "p1": pipepb.PCollection_builder{CoderId: "c1@daASxQwenJ", WindowingStrategyId: "w1@daASxQwenJ"}.Build(), + "p2": pipepb.PCollection_builder{CoderId: "c0@daASxQwenJ", WindowingStrategyId: "w1@daASxQwenJ"}.Build(), }, WindowingStrategies: map[string]*pipepb.WindowingStrategy{ - "w0@daASxQwenJ": {WindowCoderId: "c3@daASxQwenJ", EnvironmentId: "e0"}, - "w1@daASxQwenJ": {WindowCoderId: "c4@daASxQwenJ", EnvironmentId: "e1"}, + "w0@daASxQwenJ": pipepb.WindowingStrategy_builder{WindowCoderId: "c3@daASxQwenJ", EnvironmentId: "e0"}.Build(), + "w1@daASxQwenJ": pipepb.WindowingStrategy_builder{WindowCoderId: "c4@daASxQwenJ", EnvironmentId: "e1"}.Build(), }, Coders: map[string]*pipepb.Coder{ - "c0@daASxQwenJ": {Spec: &pipepb.FunctionSpec{Urn: "c0"}, ComponentCoderIds: []string{"c2@daASxQwenJ"}}, - "c1@daASxQwenJ": {Spec: &pipepb.FunctionSpec{Urn: "c1"}}, - "c2@daASxQwenJ": {Spec: &pipepb.FunctionSpec{Urn: "c2"}}, - "c3@daASxQwenJ": {Spec: &pipepb.FunctionSpec{Urn: "c3"}}, - "c4@daASxQwenJ": {Spec: &pipepb.FunctionSpec{Urn: "c4"}}, + "c0@daASxQwenJ": pipepb.Coder_builder{Spec: pipepb.FunctionSpec_builder{Urn: "c0"}.Build(), ComponentCoderIds: []string{"c2@daASxQwenJ"}}.Build(), + "c1@daASxQwenJ": pipepb.Coder_builder{Spec: pipepb.FunctionSpec_builder{Urn: "c1"}.Build()}.Build(), + "c2@daASxQwenJ": pipepb.Coder_builder{Spec: pipepb.FunctionSpec_builder{Urn: "c2"}.Build()}.Build(), + "c3@daASxQwenJ": pipepb.Coder_builder{Spec: pipepb.FunctionSpec_builder{Urn: "c3"}.Build()}.Build(), + "c4@daASxQwenJ": pipepb.Coder_builder{Spec: pipepb.FunctionSpec_builder{Urn: "c4"}.Build()}.Build(), }, Environments: map[string]*pipepb.Environment{ - "e0": {Urn: "e0"}, - "e1": {Urn: "e1"}, + "e0": pipepb.Environment_builder{Urn: "e0"}.Build(), + "e1": pipepb.Environment_builder{Urn: "e1"}.Build(), }, - }, + }.Build(), }, { name: "[Consistency] Missing EnvironmentID", - init: &pipepb.Components{ + init: pipepb.Components_builder{ Transforms: map[string]*pipepb.PTransform{ - "t0": { + "t0": pipepb.PTransform_builder{ UniqueName: "t0", Inputs: map[string]string{"t0i0": "p0"}, Outputs: map[string]string{"t0o0": "p1"}, EnvironmentId: "e0", - }, - "t1": { + }.Build(), + "t1": pipepb.PTransform_builder{ UniqueName: "t1", Inputs: map[string]string{"t1i0": "p1"}, Outputs: map[string]string{"t1o0": "p2"}, EnvironmentId: "e1", - }, + }.Build(), }, Pcollections: map[string]*pipepb.PCollection{ - "p0": {CoderId: "c0", WindowingStrategyId: "w0"}, - "p1": {CoderId: "c1", WindowingStrategyId: "w1"}, - "p2": {CoderId: "c0", WindowingStrategyId: "w1"}, + "p0": pipepb.PCollection_builder{CoderId: "c0", WindowingStrategyId: "w0"}.Build(), + "p1": pipepb.PCollection_builder{CoderId: "c1", WindowingStrategyId: "w1"}.Build(), + "p2": pipepb.PCollection_builder{CoderId: "c0", WindowingStrategyId: "w1"}.Build(), }, WindowingStrategies: map[string]*pipepb.WindowingStrategy{ - "w0": {WindowCoderId: "c3", EnvironmentId: "e0"}, - "w1": {WindowCoderId: "c4", EnvironmentId: "e1"}, + "w0": pipepb.WindowingStrategy_builder{WindowCoderId: "c3", EnvironmentId: "e0"}.Build(), + "w1": pipepb.WindowingStrategy_builder{WindowCoderId: "c4", EnvironmentId: "e1"}.Build(), }, Coders: map[string]*pipepb.Coder{ - "c0": {Spec: &pipepb.FunctionSpec{Urn: "c0"}}, - "c1": {Spec: &pipepb.FunctionSpec{Urn: "c1"}}, + "c0": pipepb.Coder_builder{Spec: pipepb.FunctionSpec_builder{Urn: "c0"}.Build()}.Build(), + "c1": pipepb.Coder_builder{Spec: pipepb.FunctionSpec_builder{Urn: "c1"}.Build()}.Build(), }, Environments: map[string]*pipepb.Environment{ // Missing "e0" - "e1": {Urn: "e1"}, + "e1": pipepb.Environment_builder{Urn: "e1"}.Build(), }, - }, + }.Build(), namespace: "daASxQwenJ", transformID: "t0", err: "trying to add an Environment whose key is absent should panic", }, { name: "[Consistency] Missing WindowingStrategyID", - init: &pipepb.Components{ + init: pipepb.Components_builder{ Transforms: map[string]*pipepb.PTransform{ - "t0": { + "t0": pipepb.PTransform_builder{ UniqueName: "t0", Inputs: map[string]string{"t0i0": "p0"}, Outputs: map[string]string{"t0o0": "p1"}, EnvironmentId: "e0", - }, - "t1": { + }.Build(), + "t1": pipepb.PTransform_builder{ UniqueName: "t1", Inputs: map[string]string{"t1i0": "p1"}, Outputs: map[string]string{"t1o0": "p2"}, EnvironmentId: "e1", - }, + }.Build(), }, Pcollections: map[string]*pipepb.PCollection{ - "p0": {CoderId: "c0", WindowingStrategyId: "w0"}, - "p1": {CoderId: "c1", WindowingStrategyId: "w1"}, - "p2": {CoderId: "c0", WindowingStrategyId: "w1"}, + "p0": pipepb.PCollection_builder{CoderId: "c0", WindowingStrategyId: "w0"}.Build(), + "p1": pipepb.PCollection_builder{CoderId: "c1", WindowingStrategyId: "w1"}.Build(), + "p2": pipepb.PCollection_builder{CoderId: "c0", WindowingStrategyId: "w1"}.Build(), }, WindowingStrategies: map[string]*pipepb.WindowingStrategy{ // Missing w0 - "w1": {WindowCoderId: "c4", EnvironmentId: "e1"}, + "w1": pipepb.WindowingStrategy_builder{WindowCoderId: "c4", EnvironmentId: "e1"}.Build(), }, Coders: map[string]*pipepb.Coder{ - "c0": {Spec: &pipepb.FunctionSpec{Urn: "c0"}}, - "c1": {Spec: &pipepb.FunctionSpec{Urn: "c1"}}, + "c0": pipepb.Coder_builder{Spec: pipepb.FunctionSpec_builder{Urn: "c0"}.Build()}.Build(), + "c1": pipepb.Coder_builder{Spec: pipepb.FunctionSpec_builder{Urn: "c1"}.Build()}.Build(), }, Environments: map[string]*pipepb.Environment{ - "e0": {Urn: "e0"}, - "e1": {Urn: "e1"}, + "e0": pipepb.Environment_builder{Urn: "e0"}.Build(), + "e1": pipepb.Environment_builder{Urn: "e1"}.Build(), }, - }, + }.Build(), namespace: "daASxQwenJ", transformID: "t0", err: "trying to add an WindowingStrategy whose key is absent should panic", }, { name: "[Consistency] Missing CoderID", - init: &pipepb.Components{ + init: pipepb.Components_builder{ Transforms: map[string]*pipepb.PTransform{ - "t0": { + "t0": pipepb.PTransform_builder{ UniqueName: "t0", Inputs: map[string]string{"t0i0": "p0"}, Outputs: map[string]string{"t0o0": "p1"}, EnvironmentId: "e0", - }, - "t1": { + }.Build(), + "t1": pipepb.PTransform_builder{ UniqueName: "t1", Inputs: map[string]string{"t1i0": "p1"}, Outputs: map[string]string{"t1o0": "p2"}, EnvironmentId: "e1", - }, + }.Build(), }, Pcollections: map[string]*pipepb.PCollection{ - "p0": {CoderId: "c0", WindowingStrategyId: "w0"}, - "p1": {CoderId: "c1", WindowingStrategyId: "w1"}, - "p2": {CoderId: "c0", WindowingStrategyId: "w1"}, + "p0": pipepb.PCollection_builder{CoderId: "c0", WindowingStrategyId: "w0"}.Build(), + "p1": pipepb.PCollection_builder{CoderId: "c1", WindowingStrategyId: "w1"}.Build(), + "p2": pipepb.PCollection_builder{CoderId: "c0", WindowingStrategyId: "w1"}.Build(), }, WindowingStrategies: map[string]*pipepb.WindowingStrategy{ - "w0": {WindowCoderId: "c3", EnvironmentId: "e0"}, - "w1": {WindowCoderId: "c4", EnvironmentId: "e1"}, + "w0": pipepb.WindowingStrategy_builder{WindowCoderId: "c3", EnvironmentId: "e0"}.Build(), + "w1": pipepb.WindowingStrategy_builder{WindowCoderId: "c4", EnvironmentId: "e1"}.Build(), }, Coders: map[string]*pipepb.Coder{ // Missing c0 - "c1": {Spec: &pipepb.FunctionSpec{Urn: "c1"}}, + "c1": pipepb.Coder_builder{Spec: pipepb.FunctionSpec_builder{Urn: "c1"}.Build()}.Build(), }, Environments: map[string]*pipepb.Environment{ - "e0": {Urn: "e0"}, - "e1": {Urn: "e1"}, + "e0": pipepb.Environment_builder{Urn: "e0"}.Build(), + "e1": pipepb.Environment_builder{Urn: "e1"}.Build(), }, - }, + }.Build(), namespace: "daASxQwenJ", transformID: "t0", err: "trying to add an WindowingStrategy whose key is absent should panic", @@ -559,7 +559,7 @@ func TestAddNamespace(t *testing.T) { for _, tt := range tests { t.Run(tt.name, func(t *testing.T) { if strings.Contains(tt.name, "Correctness") { - transform := tt.init.Transforms[tt.transformID] + transform := tt.init.GetTransforms()[tt.transformID] addNamespace(transform, tt.init, tt.namespace) @@ -570,7 +570,7 @@ func TestAddNamespace(t *testing.T) { if strings.Contains(tt.name, "Consistency") { defer expectPanic(t, tt.err) - transform := tt.init.Transforms[tt.transformID] + transform := tt.init.GetTransforms()[tt.transformID] addNamespace(transform, tt.init, tt.namespace) } diff --git a/sdks/go/pkg/beam/core/runtime/xlangx/payload.go b/sdks/go/pkg/beam/core/runtime/xlangx/payload.go index 8c01ffbd34ff..fcab9cd769af 100644 --- a/sdks/go/pkg/beam/core/runtime/xlangx/payload.go +++ b/sdks/go/pkg/beam/core/runtime/xlangx/payload.go @@ -52,10 +52,10 @@ func CreateExternalConfigurationPayload(pl any) (*pipepb.ExternalConfigurationPa } // Put schema and row into payload proto, and marshal it. - ecp := &pipepb.ExternalConfigurationPayload{ + ecp := pipepb.ExternalConfigurationPayload_builder{ Schema: scm, Payload: buf.Bytes(), - } + }.Build() return ecp, nil } @@ -102,7 +102,7 @@ func DecodeStructPayload(plBytes []byte) (any, error) { err = errors.WithContextf(err, "creating Row decoder for type %v", rt) return nil, errors.WithContext(err, "decoding external payload") } - buf := bytes.NewBuffer(ecp.Payload) + buf := bytes.NewBuffer(ecp.GetPayload()) val, err := dec(buf) if err != nil { err = errors.WithContext(err, "decoding Row to payload") diff --git a/sdks/go/pkg/beam/core/runtime/xlangx/registry.go b/sdks/go/pkg/beam/core/runtime/xlangx/registry.go index 1957cc61bb95..d796dee5a152 100644 --- a/sdks/go/pkg/beam/core/runtime/xlangx/registry.go +++ b/sdks/go/pkg/beam/core/runtime/xlangx/registry.go @@ -78,7 +78,7 @@ type HandlerParams struct { // CoderMarshaller returns a coder marshaller initialized with the request's namespace. func (p *HandlerParams) CoderMarshaller() *graphx.CoderMarshaller { cm := graphx.NewCoderMarshaller() - cm.Namespace = p.Req.Namespace + cm.Namespace = p.Req.GetNamespace() return cm } @@ -134,7 +134,7 @@ func makePCol(node *graph.Node, index int, local, namespace string) PCol { func (p *HandlerParams) Outputs() []PCol { out := make([]PCol, 0, len(p.ext.OutputsMap)) for local, i := range p.ext.OutputsMap { - out = append(out, makePCol(p.edge.Output[i].To, i, local, p.Req.Namespace)) + out = append(out, makePCol(p.edge.Output[i].To, i, local, p.Req.GetNamespace())) } return out } @@ -144,7 +144,7 @@ func (p *HandlerParams) Outputs() []PCol { func (p *HandlerParams) Inputs() []PCol { out := make([]PCol, 0, len(p.ext.InputsMap)) for local, i := range p.ext.InputsMap { - out = append(out, makePCol(p.edge.Input[i].From, i, local, p.Req.Namespace)) + out = append(out, makePCol(p.edge.Input[i].From, i, local, p.Req.GetNamespace())) } return out } diff --git a/sdks/go/pkg/beam/core/runtime/xlangx/resolve.go b/sdks/go/pkg/beam/core/runtime/xlangx/resolve.go index aabe6eface45..a62025039ac6 100644 --- a/sdks/go/pkg/beam/core/runtime/xlangx/resolve.go +++ b/sdks/go/pkg/beam/core/runtime/xlangx/resolve.go @@ -87,7 +87,7 @@ func ResolveArtifactsWithConfig(ctx context.Context, edges []*graph.MultiEdge, c return nil, errors.WithContextf(err, "resolving remote artifacts for edge %v", e.Name()) } - envs := components.Environments + envs := components.GetEnvironments() for eid, env := range envs { if strings.HasPrefix(eid, "go") { continue @@ -111,21 +111,21 @@ func ResolveArtifactsWithConfig(ctx context.Context, edges []*graph.MultiEdge, c fullSdkPath = cfg.JoinFn(cfg.SdkPath, name) } resolvedDeps = append(resolvedDeps, - &pipepb.ArtifactInformation{ + pipepb.ArtifactInformation_builder{ TypeUrn: "beam:artifact:type:file:v1", TypePayload: protox.MustEncode( - &pipepb.ArtifactFilePayload{ + pipepb.ArtifactFilePayload_builder{ Path: fullSdkPath, Sha256: sha256, - }, + }.Build(), ), - RoleUrn: a.RoleUrn, - RolePayload: a.RolePayload, - }, + RoleUrn: a.GetRoleUrn(), + RolePayload: a.GetRolePayload(), + }.Build(), ) paths[fullTmpPath] = fullSdkPath } - env.Dependencies = resolvedDeps + env.SetDependencies(resolvedDeps) } } } @@ -142,24 +142,24 @@ func UpdateArtifactTypeFromFileToURL(edges []*graph.MultiEdge) { panic(errors.WithContextf(err, "updating URL artifacts type for edge %v", e.Name())) } - envs := components.Environments + envs := components.GetEnvironments() for _, env := range envs { deps := env.GetDependencies() var resolvedDeps []*pipepb.ArtifactInformation for _, a := range deps { path, sha256 := artifact.MustExtractFilePayload(a) if strings.Contains(path, "://") { - a.TypeUrn = "beam:artifact:type:url:v1" - a.TypePayload = protox.MustEncode( - &pipepb.ArtifactUrlPayload{ + a.SetTypeUrn("beam:artifact:type:url:v1") + a.SetTypePayload(protox.MustEncode( + pipepb.ArtifactUrlPayload_builder{ Url: path, Sha256: sha256, - }, - ) + }.Build(), + )) } resolvedDeps = append(resolvedDeps, a) } - env.Dependencies = resolvedDeps + env.SetDependencies(resolvedDeps) } } } diff --git a/sdks/go/pkg/beam/core/runtime/xlangx/resolve_test.go b/sdks/go/pkg/beam/core/runtime/xlangx/resolve_test.go index eec13c451a13..1693ad14b22d 100644 --- a/sdks/go/pkg/beam/core/runtime/xlangx/resolve_test.go +++ b/sdks/go/pkg/beam/core/runtime/xlangx/resolve_test.go @@ -25,30 +25,30 @@ import ( func createExternalEdge(typeUrn string, typePayload []byte) *graph.MultiEdge { env := map[string]*pipepb.Environment{ - "env_java": { + "env_java": pipepb.Environment_builder{ Dependencies: []*pipepb.ArtifactInformation{ - { + pipepb.ArtifactInformation_builder{ TypeUrn: typeUrn, TypePayload: typePayload, - }, + }.Build(), }, - }, + }.Build(), } return &graph.MultiEdge{ Op: graph.External, External: &graph.ExternalTransform{ Expanded: &graph.ExpandedTransform{ - Components: &pipepb.Components{ + Components: pipepb.Components_builder{ Environments: env, - }, + }.Build(), }, }, } } func TestUpdateFileArtifactWithUrlPath(t *testing.T) { - payload, _ := proto.Marshal(&pipepb.ArtifactFilePayload{ - Path: "gs://dummy"}) + payload, _ := proto.Marshal(pipepb.ArtifactFilePayload_builder{ + Path: "gs://dummy"}.Build()) e := createExternalEdge("beam:artifact:type:file:v1", payload) UpdateArtifactTypeFromFileToURL([]*graph.MultiEdge{e}) expected := createExternalEdge("beam:artifact:type:url:v1", payload) @@ -63,8 +63,8 @@ func TestUpdateFileArtifactWithUrlPath(t *testing.T) { } func TestUpdateFileArtifactWithLocalPath(t *testing.T) { - payload, _ := proto.Marshal(&pipepb.ArtifactFilePayload{ - Path: "/tmp/artifact/dummy"}) + payload, _ := proto.Marshal(pipepb.ArtifactFilePayload_builder{ + Path: "/tmp/artifact/dummy"}.Build()) e := createExternalEdge("beam:artifact:type:file:v1", payload) UpdateArtifactTypeFromFileToURL([]*graph.MultiEdge{e}) expected := createExternalEdge("beam:artifact:type:file:v1", payload) diff --git a/sdks/go/pkg/beam/io/pubsubio/pubsubio.go b/sdks/go/pkg/beam/io/pubsubio/pubsubio.go index bcf20e89c893..8eef70b7f04d 100644 --- a/sdks/go/pkg/beam/io/pubsubio/pubsubio.go +++ b/sdks/go/pkg/beam/io/pubsubio/pubsubio.go @@ -64,16 +64,16 @@ type ReadOptions struct { func Read(s beam.Scope, project, topic string, opts *ReadOptions) beam.PCollection { s = s.Scope("pubsubio.Read") - payload := &pipepb.PubSubReadPayload{ + payload := pipepb.PubSubReadPayload_builder{ Topic: pubsubx.MakeQualifiedTopicName(project, topic), - } + }.Build() if opts != nil { - payload.IdAttribute = opts.IDAttribute - payload.TimestampAttribute = opts.TimestampAttribute + payload.SetIdAttribute(opts.IDAttribute) + payload.SetTimestampAttribute(opts.TimestampAttribute) if opts.Subscription != "" { - payload.Subscription = pubsubx.MakeQualifiedSubscriptionName(project, opts.Subscription) + payload.SetSubscription(pubsubx.MakeQualifiedSubscriptionName(project, opts.Subscription)) } - payload.WithAttributes = opts.WithAttributes + payload.SetWithAttributes(opts.WithAttributes) } out := beam.External(s, readURN, protox.MustEncode(payload), nil, []beam.FullType{typex.New(reflectx.ByteSlice)}, false) @@ -118,9 +118,9 @@ var pubSubMessageT = reflect.TypeOf((*pb.PubsubMessage)(nil)) func Write(s beam.Scope, project, topic string, col beam.PCollection) { s = s.Scope("pubsubio.Write") - payload := &pipepb.PubSubWritePayload{ + payload := pipepb.PubSubWritePayload_builder{ Topic: pubsubx.MakeQualifiedTopicName(project, topic), - } + }.Build() out := col if col.Type().Type() == reflectx.ByteSlice { diff --git a/sdks/go/pkg/beam/model/fnexecution_v1/beam_fn_api.pb.go b/sdks/go/pkg/beam/model/fnexecution_v1/beam_fn_api.pb.go index 55ec131df067..3eb826fae936 100644 --- a/sdks/go/pkg/beam/model/fnexecution_v1/beam_fn_api.pb.go +++ b/sdks/go/pkg/beam/model/fnexecution_v1/beam_fn_api.pb.go @@ -26,13 +26,15 @@ // Code generated by protoc-gen-go. DO NOT EDIT. // versions: -// protoc-gen-go v1.27.1 +// protoc-gen-go v1.36.0 // protoc v5.27.3 // source: org/apache/beam/model/fn_execution/v1/beam_fn_api.proto // TODO: Consider consolidating common components in another package // and language namespaces for re-use with Runner Api. +//go:build !protoopaque + package fnexecution_v1 import ( @@ -44,7 +46,6 @@ import ( structpb "google.golang.org/protobuf/types/known/structpb" timestamppb "google.golang.org/protobuf/types/known/timestamppb" reflect "reflect" - sync "sync" ) const ( @@ -130,11 +131,6 @@ func (x FnApiTransforms_Runner) Number() protoreflect.EnumNumber { return protoreflect.EnumNumber(x) } -// Deprecated: Use FnApiTransforms_Runner.Descriptor instead. -func (FnApiTransforms_Runner) EnumDescriptor() ([]byte, []int) { - return file_org_apache_beam_model_fn_execution_v1_beam_fn_api_proto_rawDescGZIP(), []int{0, 0} -} - type LogEntry_Severity_Enum int32 const ( @@ -203,26 +199,19 @@ func (x LogEntry_Severity_Enum) Number() protoreflect.EnumNumber { return protoreflect.EnumNumber(x) } -// Deprecated: Use LogEntry_Severity_Enum.Descriptor instead. -func (LogEntry_Severity_Enum) EnumDescriptor() ([]byte, []int) { - return file_org_apache_beam_model_fn_execution_v1_beam_fn_api_proto_rawDescGZIP(), []int{36, 1, 0} -} - // Describes transforms necessary to execute Beam over the FnAPI but are // implementation details rather than part of the core model. type FnApiTransforms struct { - state protoimpl.MessageState - sizeCache protoimpl.SizeCache + state protoimpl.MessageState `protogen:"hybrid.v1"` unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache } func (x *FnApiTransforms) Reset() { *x = FnApiTransforms{} - if protoimpl.UnsafeEnabled { - mi := &file_org_apache_beam_model_fn_execution_v1_beam_fn_api_proto_msgTypes[0] - ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) - ms.StoreMessageInfo(mi) - } + mi := &file_org_apache_beam_model_fn_execution_v1_beam_fn_api_proto_msgTypes[0] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) } func (x *FnApiTransforms) String() string { @@ -233,7 +222,7 @@ func (*FnApiTransforms) ProtoMessage() {} func (x *FnApiTransforms) ProtoReflect() protoreflect.Message { mi := &file_org_apache_beam_model_fn_execution_v1_beam_fn_api_proto_msgTypes[0] - if protoimpl.UnsafeEnabled && x != nil { + if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { ms.StoreMessageInfo(mi) @@ -243,9 +232,16 @@ func (x *FnApiTransforms) ProtoReflect() protoreflect.Message { return mi.MessageOf(x) } -// Deprecated: Use FnApiTransforms.ProtoReflect.Descriptor instead. -func (*FnApiTransforms) Descriptor() ([]byte, []int) { - return file_org_apache_beam_model_fn_execution_v1_beam_fn_api_proto_rawDescGZIP(), []int{0} +type FnApiTransforms_builder struct { + _ [0]func() // Prevents comparability and use of unkeyed literals for the builder. + +} + +func (b0 FnApiTransforms_builder) Build() *FnApiTransforms { + m0 := &FnApiTransforms{} + b, x := &b0, m0 + _, _ = b, x + return m0 } // A descriptor for connecting to a remote port using the Beam Fn Data API. @@ -253,25 +249,22 @@ func (*FnApiTransforms) Descriptor() ([]byte, []int) { // runner and the SDK). // Stable type RemoteGrpcPort struct { - state protoimpl.MessageState - sizeCache protoimpl.SizeCache - unknownFields protoimpl.UnknownFields - + state protoimpl.MessageState `protogen:"hybrid.v1"` // (Required) An API descriptor which describes where to // connect to including any authentication that is required. ApiServiceDescriptor *pipeline_v1.ApiServiceDescriptor `protobuf:"bytes,1,opt,name=api_service_descriptor,json=apiServiceDescriptor,proto3" json:"api_service_descriptor,omitempty"` // (Required) The ID of the Coder that will be used to encode and decode data // sent over this port. - CoderId string `protobuf:"bytes,2,opt,name=coder_id,json=coderId,proto3" json:"coder_id,omitempty"` + CoderId string `protobuf:"bytes,2,opt,name=coder_id,json=coderId,proto3" json:"coder_id,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache } func (x *RemoteGrpcPort) Reset() { *x = RemoteGrpcPort{} - if protoimpl.UnsafeEnabled { - mi := &file_org_apache_beam_model_fn_execution_v1_beam_fn_api_proto_msgTypes[1] - ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) - ms.StoreMessageInfo(mi) - } + mi := &file_org_apache_beam_model_fn_execution_v1_beam_fn_api_proto_msgTypes[1] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) } func (x *RemoteGrpcPort) String() string { @@ -282,7 +275,7 @@ func (*RemoteGrpcPort) ProtoMessage() {} func (x *RemoteGrpcPort) ProtoReflect() protoreflect.Message { mi := &file_org_apache_beam_model_fn_execution_v1_beam_fn_api_proto_msgTypes[1] - if protoimpl.UnsafeEnabled && x != nil { + if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { ms.StoreMessageInfo(mi) @@ -292,11 +285,6 @@ func (x *RemoteGrpcPort) ProtoReflect() protoreflect.Message { return mi.MessageOf(x) } -// Deprecated: Use RemoteGrpcPort.ProtoReflect.Descriptor instead. -func (*RemoteGrpcPort) Descriptor() ([]byte, []int) { - return file_org_apache_beam_model_fn_execution_v1_beam_fn_api_proto_rawDescGZIP(), []int{1} -} - func (x *RemoteGrpcPort) GetApiServiceDescriptor() *pipeline_v1.ApiServiceDescriptor { if x != nil { return x.ApiServiceDescriptor @@ -311,22 +299,58 @@ func (x *RemoteGrpcPort) GetCoderId() string { return "" } +func (x *RemoteGrpcPort) SetApiServiceDescriptor(v *pipeline_v1.ApiServiceDescriptor) { + x.ApiServiceDescriptor = v +} + +func (x *RemoteGrpcPort) SetCoderId(v string) { + x.CoderId = v +} + +func (x *RemoteGrpcPort) HasApiServiceDescriptor() bool { + if x == nil { + return false + } + return x.ApiServiceDescriptor != nil +} + +func (x *RemoteGrpcPort) ClearApiServiceDescriptor() { + x.ApiServiceDescriptor = nil +} + +type RemoteGrpcPort_builder struct { + _ [0]func() // Prevents comparability and use of unkeyed literals for the builder. + + // (Required) An API descriptor which describes where to + // connect to including any authentication that is required. + ApiServiceDescriptor *pipeline_v1.ApiServiceDescriptor + // (Required) The ID of the Coder that will be used to encode and decode data + // sent over this port. + CoderId string +} + +func (b0 RemoteGrpcPort_builder) Build() *RemoteGrpcPort { + m0 := &RemoteGrpcPort{} + b, x := &b0, m0 + _, _ = b, x + x.ApiServiceDescriptor = b.ApiServiceDescriptor + x.CoderId = b.CoderId + return m0 +} + // Requests the ProcessBundleDescriptor with the given id. type GetProcessBundleDescriptorRequest struct { - state protoimpl.MessageState - sizeCache protoimpl.SizeCache - unknownFields protoimpl.UnknownFields - - ProcessBundleDescriptorId string `protobuf:"bytes,1,opt,name=process_bundle_descriptor_id,json=processBundleDescriptorId,proto3" json:"process_bundle_descriptor_id,omitempty"` + state protoimpl.MessageState `protogen:"hybrid.v1"` + ProcessBundleDescriptorId string `protobuf:"bytes,1,opt,name=process_bundle_descriptor_id,json=processBundleDescriptorId,proto3" json:"process_bundle_descriptor_id,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache } func (x *GetProcessBundleDescriptorRequest) Reset() { *x = GetProcessBundleDescriptorRequest{} - if protoimpl.UnsafeEnabled { - mi := &file_org_apache_beam_model_fn_execution_v1_beam_fn_api_proto_msgTypes[2] - ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) - ms.StoreMessageInfo(mi) - } + mi := &file_org_apache_beam_model_fn_execution_v1_beam_fn_api_proto_msgTypes[2] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) } func (x *GetProcessBundleDescriptorRequest) String() string { @@ -337,7 +361,7 @@ func (*GetProcessBundleDescriptorRequest) ProtoMessage() {} func (x *GetProcessBundleDescriptorRequest) ProtoReflect() protoreflect.Message { mi := &file_org_apache_beam_model_fn_execution_v1_beam_fn_api_proto_msgTypes[2] - if protoimpl.UnsafeEnabled && x != nil { + if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { ms.StoreMessageInfo(mi) @@ -347,11 +371,6 @@ func (x *GetProcessBundleDescriptorRequest) ProtoReflect() protoreflect.Message return mi.MessageOf(x) } -// Deprecated: Use GetProcessBundleDescriptorRequest.ProtoReflect.Descriptor instead. -func (*GetProcessBundleDescriptorRequest) Descriptor() ([]byte, []int) { - return file_org_apache_beam_model_fn_execution_v1_beam_fn_api_proto_rawDescGZIP(), []int{2} -} - func (x *GetProcessBundleDescriptorRequest) GetProcessBundleDescriptorId() string { if x != nil { return x.ProcessBundleDescriptorId @@ -359,21 +378,36 @@ func (x *GetProcessBundleDescriptorRequest) GetProcessBundleDescriptorId() strin return "" } +func (x *GetProcessBundleDescriptorRequest) SetProcessBundleDescriptorId(v string) { + x.ProcessBundleDescriptorId = v +} + +type GetProcessBundleDescriptorRequest_builder struct { + _ [0]func() // Prevents comparability and use of unkeyed literals for the builder. + + ProcessBundleDescriptorId string +} + +func (b0 GetProcessBundleDescriptorRequest_builder) Build() *GetProcessBundleDescriptorRequest { + m0 := &GetProcessBundleDescriptorRequest{} + b, x := &b0, m0 + _, _ = b, x + x.ProcessBundleDescriptorId = b.ProcessBundleDescriptorId + return m0 +} + // A request sent by a runner which the SDK is asked to fulfill. // For any unsupported request type, an error should be returned with a // matching instruction id. // Stable type InstructionRequest struct { - state protoimpl.MessageState - sizeCache protoimpl.SizeCache - unknownFields protoimpl.UnknownFields - + state protoimpl.MessageState `protogen:"hybrid.v1"` // (Required) A unique identifier provided by the runner which represents // this requests execution. The InstructionResponse MUST have the matching id. InstructionId string `protobuf:"bytes,1,opt,name=instruction_id,json=instructionId,proto3" json:"instruction_id,omitempty"` // (Required) A request that the SDK Harness needs to interpret. // - // Types that are assignable to Request: + // Types that are valid to be assigned to Request: // // *InstructionRequest_ProcessBundle // *InstructionRequest_ProcessBundleProgress @@ -383,16 +417,16 @@ type InstructionRequest struct { // *InstructionRequest_HarnessMonitoringInfos // *InstructionRequest_SampleData // *InstructionRequest_Register - Request isInstructionRequest_Request `protobuf_oneof:"request"` + Request isInstructionRequest_Request `protobuf_oneof:"request"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache } func (x *InstructionRequest) Reset() { *x = InstructionRequest{} - if protoimpl.UnsafeEnabled { - mi := &file_org_apache_beam_model_fn_execution_v1_beam_fn_api_proto_msgTypes[3] - ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) - ms.StoreMessageInfo(mi) - } + mi := &file_org_apache_beam_model_fn_execution_v1_beam_fn_api_proto_msgTypes[3] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) } func (x *InstructionRequest) String() string { @@ -403,7 +437,7 @@ func (*InstructionRequest) ProtoMessage() {} func (x *InstructionRequest) ProtoReflect() protoreflect.Message { mi := &file_org_apache_beam_model_fn_execution_v1_beam_fn_api_proto_msgTypes[3] - if protoimpl.UnsafeEnabled && x != nil { + if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { ms.StoreMessageInfo(mi) @@ -413,11 +447,6 @@ func (x *InstructionRequest) ProtoReflect() protoreflect.Message { return mi.MessageOf(x) } -// Deprecated: Use InstructionRequest.ProtoReflect.Descriptor instead. -func (*InstructionRequest) Descriptor() ([]byte, []int) { - return file_org_apache_beam_model_fn_execution_v1_beam_fn_api_proto_rawDescGZIP(), []int{3} -} - func (x *InstructionRequest) GetInstructionId() string { if x != nil { return x.InstructionId @@ -425,69 +454,375 @@ func (x *InstructionRequest) GetInstructionId() string { return "" } -func (m *InstructionRequest) GetRequest() isInstructionRequest_Request { - if m != nil { - return m.Request +func (x *InstructionRequest) GetRequest() isInstructionRequest_Request { + if x != nil { + return x.Request } return nil } func (x *InstructionRequest) GetProcessBundle() *ProcessBundleRequest { - if x, ok := x.GetRequest().(*InstructionRequest_ProcessBundle); ok { - return x.ProcessBundle + if x != nil { + if x, ok := x.Request.(*InstructionRequest_ProcessBundle); ok { + return x.ProcessBundle + } } return nil } func (x *InstructionRequest) GetProcessBundleProgress() *ProcessBundleProgressRequest { - if x, ok := x.GetRequest().(*InstructionRequest_ProcessBundleProgress); ok { - return x.ProcessBundleProgress + if x != nil { + if x, ok := x.Request.(*InstructionRequest_ProcessBundleProgress); ok { + return x.ProcessBundleProgress + } } return nil } func (x *InstructionRequest) GetProcessBundleSplit() *ProcessBundleSplitRequest { - if x, ok := x.GetRequest().(*InstructionRequest_ProcessBundleSplit); ok { - return x.ProcessBundleSplit + if x != nil { + if x, ok := x.Request.(*InstructionRequest_ProcessBundleSplit); ok { + return x.ProcessBundleSplit + } } return nil } func (x *InstructionRequest) GetFinalizeBundle() *FinalizeBundleRequest { - if x, ok := x.GetRequest().(*InstructionRequest_FinalizeBundle); ok { - return x.FinalizeBundle + if x != nil { + if x, ok := x.Request.(*InstructionRequest_FinalizeBundle); ok { + return x.FinalizeBundle + } } return nil } func (x *InstructionRequest) GetMonitoringInfos() *MonitoringInfosMetadataRequest { - if x, ok := x.GetRequest().(*InstructionRequest_MonitoringInfos); ok { - return x.MonitoringInfos + if x != nil { + if x, ok := x.Request.(*InstructionRequest_MonitoringInfos); ok { + return x.MonitoringInfos + } } return nil } func (x *InstructionRequest) GetHarnessMonitoringInfos() *HarnessMonitoringInfosRequest { - if x, ok := x.GetRequest().(*InstructionRequest_HarnessMonitoringInfos); ok { - return x.HarnessMonitoringInfos + if x != nil { + if x, ok := x.Request.(*InstructionRequest_HarnessMonitoringInfos); ok { + return x.HarnessMonitoringInfos + } } return nil } func (x *InstructionRequest) GetSampleData() *SampleDataRequest { - if x, ok := x.GetRequest().(*InstructionRequest_SampleData); ok { - return x.SampleData + if x != nil { + if x, ok := x.Request.(*InstructionRequest_SampleData); ok { + return x.SampleData + } } return nil } func (x *InstructionRequest) GetRegister() *RegisterRequest { - if x, ok := x.GetRequest().(*InstructionRequest_Register); ok { - return x.Register + if x != nil { + if x, ok := x.Request.(*InstructionRequest_Register); ok { + return x.Register + } } return nil } +func (x *InstructionRequest) SetInstructionId(v string) { + x.InstructionId = v +} + +func (x *InstructionRequest) SetProcessBundle(v *ProcessBundleRequest) { + if v == nil { + x.Request = nil + return + } + x.Request = &InstructionRequest_ProcessBundle{v} +} + +func (x *InstructionRequest) SetProcessBundleProgress(v *ProcessBundleProgressRequest) { + if v == nil { + x.Request = nil + return + } + x.Request = &InstructionRequest_ProcessBundleProgress{v} +} + +func (x *InstructionRequest) SetProcessBundleSplit(v *ProcessBundleSplitRequest) { + if v == nil { + x.Request = nil + return + } + x.Request = &InstructionRequest_ProcessBundleSplit{v} +} + +func (x *InstructionRequest) SetFinalizeBundle(v *FinalizeBundleRequest) { + if v == nil { + x.Request = nil + return + } + x.Request = &InstructionRequest_FinalizeBundle{v} +} + +func (x *InstructionRequest) SetMonitoringInfos(v *MonitoringInfosMetadataRequest) { + if v == nil { + x.Request = nil + return + } + x.Request = &InstructionRequest_MonitoringInfos{v} +} + +func (x *InstructionRequest) SetHarnessMonitoringInfos(v *HarnessMonitoringInfosRequest) { + if v == nil { + x.Request = nil + return + } + x.Request = &InstructionRequest_HarnessMonitoringInfos{v} +} + +func (x *InstructionRequest) SetSampleData(v *SampleDataRequest) { + if v == nil { + x.Request = nil + return + } + x.Request = &InstructionRequest_SampleData{v} +} + +func (x *InstructionRequest) SetRegister(v *RegisterRequest) { + if v == nil { + x.Request = nil + return + } + x.Request = &InstructionRequest_Register{v} +} + +func (x *InstructionRequest) HasRequest() bool { + if x == nil { + return false + } + return x.Request != nil +} + +func (x *InstructionRequest) HasProcessBundle() bool { + if x == nil { + return false + } + _, ok := x.Request.(*InstructionRequest_ProcessBundle) + return ok +} + +func (x *InstructionRequest) HasProcessBundleProgress() bool { + if x == nil { + return false + } + _, ok := x.Request.(*InstructionRequest_ProcessBundleProgress) + return ok +} + +func (x *InstructionRequest) HasProcessBundleSplit() bool { + if x == nil { + return false + } + _, ok := x.Request.(*InstructionRequest_ProcessBundleSplit) + return ok +} + +func (x *InstructionRequest) HasFinalizeBundle() bool { + if x == nil { + return false + } + _, ok := x.Request.(*InstructionRequest_FinalizeBundle) + return ok +} + +func (x *InstructionRequest) HasMonitoringInfos() bool { + if x == nil { + return false + } + _, ok := x.Request.(*InstructionRequest_MonitoringInfos) + return ok +} + +func (x *InstructionRequest) HasHarnessMonitoringInfos() bool { + if x == nil { + return false + } + _, ok := x.Request.(*InstructionRequest_HarnessMonitoringInfos) + return ok +} + +func (x *InstructionRequest) HasSampleData() bool { + if x == nil { + return false + } + _, ok := x.Request.(*InstructionRequest_SampleData) + return ok +} + +func (x *InstructionRequest) HasRegister() bool { + if x == nil { + return false + } + _, ok := x.Request.(*InstructionRequest_Register) + return ok +} + +func (x *InstructionRequest) ClearRequest() { + x.Request = nil +} + +func (x *InstructionRequest) ClearProcessBundle() { + if _, ok := x.Request.(*InstructionRequest_ProcessBundle); ok { + x.Request = nil + } +} + +func (x *InstructionRequest) ClearProcessBundleProgress() { + if _, ok := x.Request.(*InstructionRequest_ProcessBundleProgress); ok { + x.Request = nil + } +} + +func (x *InstructionRequest) ClearProcessBundleSplit() { + if _, ok := x.Request.(*InstructionRequest_ProcessBundleSplit); ok { + x.Request = nil + } +} + +func (x *InstructionRequest) ClearFinalizeBundle() { + if _, ok := x.Request.(*InstructionRequest_FinalizeBundle); ok { + x.Request = nil + } +} + +func (x *InstructionRequest) ClearMonitoringInfos() { + if _, ok := x.Request.(*InstructionRequest_MonitoringInfos); ok { + x.Request = nil + } +} + +func (x *InstructionRequest) ClearHarnessMonitoringInfos() { + if _, ok := x.Request.(*InstructionRequest_HarnessMonitoringInfos); ok { + x.Request = nil + } +} + +func (x *InstructionRequest) ClearSampleData() { + if _, ok := x.Request.(*InstructionRequest_SampleData); ok { + x.Request = nil + } +} + +func (x *InstructionRequest) ClearRegister() { + if _, ok := x.Request.(*InstructionRequest_Register); ok { + x.Request = nil + } +} + +const InstructionRequest_Request_not_set_case case_InstructionRequest_Request = 0 +const InstructionRequest_ProcessBundle_case case_InstructionRequest_Request = 1001 +const InstructionRequest_ProcessBundleProgress_case case_InstructionRequest_Request = 1002 +const InstructionRequest_ProcessBundleSplit_case case_InstructionRequest_Request = 1003 +const InstructionRequest_FinalizeBundle_case case_InstructionRequest_Request = 1004 +const InstructionRequest_MonitoringInfos_case case_InstructionRequest_Request = 1005 +const InstructionRequest_HarnessMonitoringInfos_case case_InstructionRequest_Request = 1006 +const InstructionRequest_SampleData_case case_InstructionRequest_Request = 1007 +const InstructionRequest_Register_case case_InstructionRequest_Request = 1000 + +func (x *InstructionRequest) WhichRequest() case_InstructionRequest_Request { + if x == nil { + return InstructionRequest_Request_not_set_case + } + switch x.Request.(type) { + case *InstructionRequest_ProcessBundle: + return InstructionRequest_ProcessBundle_case + case *InstructionRequest_ProcessBundleProgress: + return InstructionRequest_ProcessBundleProgress_case + case *InstructionRequest_ProcessBundleSplit: + return InstructionRequest_ProcessBundleSplit_case + case *InstructionRequest_FinalizeBundle: + return InstructionRequest_FinalizeBundle_case + case *InstructionRequest_MonitoringInfos: + return InstructionRequest_MonitoringInfos_case + case *InstructionRequest_HarnessMonitoringInfos: + return InstructionRequest_HarnessMonitoringInfos_case + case *InstructionRequest_SampleData: + return InstructionRequest_SampleData_case + case *InstructionRequest_Register: + return InstructionRequest_Register_case + default: + return InstructionRequest_Request_not_set_case + } +} + +type InstructionRequest_builder struct { + _ [0]func() // Prevents comparability and use of unkeyed literals for the builder. + + // (Required) A unique identifier provided by the runner which represents + // this requests execution. The InstructionResponse MUST have the matching id. + InstructionId string + // (Required) A request that the SDK Harness needs to interpret. + + // Fields of oneof Request: + ProcessBundle *ProcessBundleRequest + ProcessBundleProgress *ProcessBundleProgressRequest + ProcessBundleSplit *ProcessBundleSplitRequest + FinalizeBundle *FinalizeBundleRequest + MonitoringInfos *MonitoringInfosMetadataRequest + HarnessMonitoringInfos *HarnessMonitoringInfosRequest + SampleData *SampleDataRequest + // DEPRECATED + Register *RegisterRequest + // -- end of Request +} + +func (b0 InstructionRequest_builder) Build() *InstructionRequest { + m0 := &InstructionRequest{} + b, x := &b0, m0 + _, _ = b, x + x.InstructionId = b.InstructionId + if b.ProcessBundle != nil { + x.Request = &InstructionRequest_ProcessBundle{b.ProcessBundle} + } + if b.ProcessBundleProgress != nil { + x.Request = &InstructionRequest_ProcessBundleProgress{b.ProcessBundleProgress} + } + if b.ProcessBundleSplit != nil { + x.Request = &InstructionRequest_ProcessBundleSplit{b.ProcessBundleSplit} + } + if b.FinalizeBundle != nil { + x.Request = &InstructionRequest_FinalizeBundle{b.FinalizeBundle} + } + if b.MonitoringInfos != nil { + x.Request = &InstructionRequest_MonitoringInfos{b.MonitoringInfos} + } + if b.HarnessMonitoringInfos != nil { + x.Request = &InstructionRequest_HarnessMonitoringInfos{b.HarnessMonitoringInfos} + } + if b.SampleData != nil { + x.Request = &InstructionRequest_SampleData{b.SampleData} + } + if b.Register != nil { + x.Request = &InstructionRequest_Register{b.Register} + } + return m0 +} + +type case_InstructionRequest_Request protoreflect.FieldNumber + +func (x case_InstructionRequest_Request) String() string { + md := file_org_apache_beam_model_fn_execution_v1_beam_fn_api_proto_msgTypes[3].Descriptor() + if x == 0 { + return "not set" + } + return protoimpl.X.MessageFieldStringOf(md, protoreflect.FieldNumber(x)) +} + type isInstructionRequest_Request interface { isInstructionRequest_Request() } @@ -544,10 +879,7 @@ func (*InstructionRequest_Register) isInstructionRequest_Request() {} // The response for an associated request the SDK had been asked to fulfill. // Stable type InstructionResponse struct { - state protoimpl.MessageState - sizeCache protoimpl.SizeCache - unknownFields protoimpl.UnknownFields - + state protoimpl.MessageState `protogen:"hybrid.v1"` // (Required) A reference provided by the runner which represents a requests // execution. The InstructionResponse MUST have the matching id when // responding to the runner. @@ -559,7 +891,7 @@ type InstructionResponse struct { // If the instruction did not fail, it is required to return an equivalent // response type depending on the request this matches. // - // Types that are assignable to Response: + // Types that are valid to be assigned to Response: // // *InstructionResponse_ProcessBundle // *InstructionResponse_ProcessBundleProgress @@ -569,16 +901,16 @@ type InstructionResponse struct { // *InstructionResponse_HarnessMonitoringInfos // *InstructionResponse_SampleData // *InstructionResponse_Register - Response isInstructionResponse_Response `protobuf_oneof:"response"` + Response isInstructionResponse_Response `protobuf_oneof:"response"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache } func (x *InstructionResponse) Reset() { *x = InstructionResponse{} - if protoimpl.UnsafeEnabled { - mi := &file_org_apache_beam_model_fn_execution_v1_beam_fn_api_proto_msgTypes[4] - ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) - ms.StoreMessageInfo(mi) - } + mi := &file_org_apache_beam_model_fn_execution_v1_beam_fn_api_proto_msgTypes[4] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) } func (x *InstructionResponse) String() string { @@ -589,7 +921,7 @@ func (*InstructionResponse) ProtoMessage() {} func (x *InstructionResponse) ProtoReflect() protoreflect.Message { mi := &file_org_apache_beam_model_fn_execution_v1_beam_fn_api_proto_msgTypes[4] - if protoimpl.UnsafeEnabled && x != nil { + if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { ms.StoreMessageInfo(mi) @@ -599,11 +931,6 @@ func (x *InstructionResponse) ProtoReflect() protoreflect.Message { return mi.MessageOf(x) } -// Deprecated: Use InstructionResponse.ProtoReflect.Descriptor instead. -func (*InstructionResponse) Descriptor() ([]byte, []int) { - return file_org_apache_beam_model_fn_execution_v1_beam_fn_api_proto_rawDescGZIP(), []int{4} -} - func (x *InstructionResponse) GetInstructionId() string { if x != nil { return x.InstructionId @@ -618,95 +945,412 @@ func (x *InstructionResponse) GetError() string { return "" } -func (m *InstructionResponse) GetResponse() isInstructionResponse_Response { - if m != nil { - return m.Response +func (x *InstructionResponse) GetResponse() isInstructionResponse_Response { + if x != nil { + return x.Response } return nil } func (x *InstructionResponse) GetProcessBundle() *ProcessBundleResponse { - if x, ok := x.GetResponse().(*InstructionResponse_ProcessBundle); ok { - return x.ProcessBundle + if x != nil { + if x, ok := x.Response.(*InstructionResponse_ProcessBundle); ok { + return x.ProcessBundle + } } return nil } func (x *InstructionResponse) GetProcessBundleProgress() *ProcessBundleProgressResponse { - if x, ok := x.GetResponse().(*InstructionResponse_ProcessBundleProgress); ok { - return x.ProcessBundleProgress + if x != nil { + if x, ok := x.Response.(*InstructionResponse_ProcessBundleProgress); ok { + return x.ProcessBundleProgress + } } return nil } func (x *InstructionResponse) GetProcessBundleSplit() *ProcessBundleSplitResponse { - if x, ok := x.GetResponse().(*InstructionResponse_ProcessBundleSplit); ok { - return x.ProcessBundleSplit + if x != nil { + if x, ok := x.Response.(*InstructionResponse_ProcessBundleSplit); ok { + return x.ProcessBundleSplit + } } return nil } func (x *InstructionResponse) GetFinalizeBundle() *FinalizeBundleResponse { - if x, ok := x.GetResponse().(*InstructionResponse_FinalizeBundle); ok { - return x.FinalizeBundle + if x != nil { + if x, ok := x.Response.(*InstructionResponse_FinalizeBundle); ok { + return x.FinalizeBundle + } } return nil } func (x *InstructionResponse) GetMonitoringInfos() *MonitoringInfosMetadataResponse { - if x, ok := x.GetResponse().(*InstructionResponse_MonitoringInfos); ok { - return x.MonitoringInfos + if x != nil { + if x, ok := x.Response.(*InstructionResponse_MonitoringInfos); ok { + return x.MonitoringInfos + } } return nil } func (x *InstructionResponse) GetHarnessMonitoringInfos() *HarnessMonitoringInfosResponse { - if x, ok := x.GetResponse().(*InstructionResponse_HarnessMonitoringInfos); ok { - return x.HarnessMonitoringInfos + if x != nil { + if x, ok := x.Response.(*InstructionResponse_HarnessMonitoringInfos); ok { + return x.HarnessMonitoringInfos + } } return nil } func (x *InstructionResponse) GetSampleData() *SampleDataResponse { - if x, ok := x.GetResponse().(*InstructionResponse_SampleData); ok { - return x.SampleData + if x != nil { + if x, ok := x.Response.(*InstructionResponse_SampleData); ok { + return x.SampleData + } } return nil } func (x *InstructionResponse) GetRegister() *RegisterResponse { - if x, ok := x.GetResponse().(*InstructionResponse_Register); ok { - return x.Register + if x != nil { + if x, ok := x.Response.(*InstructionResponse_Register); ok { + return x.Register + } } return nil } -type isInstructionResponse_Response interface { - isInstructionResponse_Response() +func (x *InstructionResponse) SetInstructionId(v string) { + x.InstructionId = v } -type InstructionResponse_ProcessBundle struct { - ProcessBundle *ProcessBundleResponse `protobuf:"bytes,1001,opt,name=process_bundle,json=processBundle,proto3,oneof"` +func (x *InstructionResponse) SetError(v string) { + x.Error = v } -type InstructionResponse_ProcessBundleProgress struct { - ProcessBundleProgress *ProcessBundleProgressResponse `protobuf:"bytes,1002,opt,name=process_bundle_progress,json=processBundleProgress,proto3,oneof"` +func (x *InstructionResponse) SetProcessBundle(v *ProcessBundleResponse) { + if v == nil { + x.Response = nil + return + } + x.Response = &InstructionResponse_ProcessBundle{v} } -type InstructionResponse_ProcessBundleSplit struct { - ProcessBundleSplit *ProcessBundleSplitResponse `protobuf:"bytes,1003,opt,name=process_bundle_split,json=processBundleSplit,proto3,oneof"` +func (x *InstructionResponse) SetProcessBundleProgress(v *ProcessBundleProgressResponse) { + if v == nil { + x.Response = nil + return + } + x.Response = &InstructionResponse_ProcessBundleProgress{v} } -type InstructionResponse_FinalizeBundle struct { - FinalizeBundle *FinalizeBundleResponse `protobuf:"bytes,1004,opt,name=finalize_bundle,json=finalizeBundle,proto3,oneof"` +func (x *InstructionResponse) SetProcessBundleSplit(v *ProcessBundleSplitResponse) { + if v == nil { + x.Response = nil + return + } + x.Response = &InstructionResponse_ProcessBundleSplit{v} } -type InstructionResponse_MonitoringInfos struct { - MonitoringInfos *MonitoringInfosMetadataResponse `protobuf:"bytes,1005,opt,name=monitoring_infos,json=monitoringInfos,proto3,oneof"` +func (x *InstructionResponse) SetFinalizeBundle(v *FinalizeBundleResponse) { + if v == nil { + x.Response = nil + return + } + x.Response = &InstructionResponse_FinalizeBundle{v} } -type InstructionResponse_HarnessMonitoringInfos struct { - HarnessMonitoringInfos *HarnessMonitoringInfosResponse `protobuf:"bytes,1006,opt,name=harness_monitoring_infos,json=harnessMonitoringInfos,proto3,oneof"` +func (x *InstructionResponse) SetMonitoringInfos(v *MonitoringInfosMetadataResponse) { + if v == nil { + x.Response = nil + return + } + x.Response = &InstructionResponse_MonitoringInfos{v} +} + +func (x *InstructionResponse) SetHarnessMonitoringInfos(v *HarnessMonitoringInfosResponse) { + if v == nil { + x.Response = nil + return + } + x.Response = &InstructionResponse_HarnessMonitoringInfos{v} +} + +func (x *InstructionResponse) SetSampleData(v *SampleDataResponse) { + if v == nil { + x.Response = nil + return + } + x.Response = &InstructionResponse_SampleData{v} +} + +func (x *InstructionResponse) SetRegister(v *RegisterResponse) { + if v == nil { + x.Response = nil + return + } + x.Response = &InstructionResponse_Register{v} +} + +func (x *InstructionResponse) HasResponse() bool { + if x == nil { + return false + } + return x.Response != nil +} + +func (x *InstructionResponse) HasProcessBundle() bool { + if x == nil { + return false + } + _, ok := x.Response.(*InstructionResponse_ProcessBundle) + return ok +} + +func (x *InstructionResponse) HasProcessBundleProgress() bool { + if x == nil { + return false + } + _, ok := x.Response.(*InstructionResponse_ProcessBundleProgress) + return ok +} + +func (x *InstructionResponse) HasProcessBundleSplit() bool { + if x == nil { + return false + } + _, ok := x.Response.(*InstructionResponse_ProcessBundleSplit) + return ok +} + +func (x *InstructionResponse) HasFinalizeBundle() bool { + if x == nil { + return false + } + _, ok := x.Response.(*InstructionResponse_FinalizeBundle) + return ok +} + +func (x *InstructionResponse) HasMonitoringInfos() bool { + if x == nil { + return false + } + _, ok := x.Response.(*InstructionResponse_MonitoringInfos) + return ok +} + +func (x *InstructionResponse) HasHarnessMonitoringInfos() bool { + if x == nil { + return false + } + _, ok := x.Response.(*InstructionResponse_HarnessMonitoringInfos) + return ok +} + +func (x *InstructionResponse) HasSampleData() bool { + if x == nil { + return false + } + _, ok := x.Response.(*InstructionResponse_SampleData) + return ok +} + +func (x *InstructionResponse) HasRegister() bool { + if x == nil { + return false + } + _, ok := x.Response.(*InstructionResponse_Register) + return ok +} + +func (x *InstructionResponse) ClearResponse() { + x.Response = nil +} + +func (x *InstructionResponse) ClearProcessBundle() { + if _, ok := x.Response.(*InstructionResponse_ProcessBundle); ok { + x.Response = nil + } +} + +func (x *InstructionResponse) ClearProcessBundleProgress() { + if _, ok := x.Response.(*InstructionResponse_ProcessBundleProgress); ok { + x.Response = nil + } +} + +func (x *InstructionResponse) ClearProcessBundleSplit() { + if _, ok := x.Response.(*InstructionResponse_ProcessBundleSplit); ok { + x.Response = nil + } +} + +func (x *InstructionResponse) ClearFinalizeBundle() { + if _, ok := x.Response.(*InstructionResponse_FinalizeBundle); ok { + x.Response = nil + } +} + +func (x *InstructionResponse) ClearMonitoringInfos() { + if _, ok := x.Response.(*InstructionResponse_MonitoringInfos); ok { + x.Response = nil + } +} + +func (x *InstructionResponse) ClearHarnessMonitoringInfos() { + if _, ok := x.Response.(*InstructionResponse_HarnessMonitoringInfos); ok { + x.Response = nil + } +} + +func (x *InstructionResponse) ClearSampleData() { + if _, ok := x.Response.(*InstructionResponse_SampleData); ok { + x.Response = nil + } +} + +func (x *InstructionResponse) ClearRegister() { + if _, ok := x.Response.(*InstructionResponse_Register); ok { + x.Response = nil + } +} + +const InstructionResponse_Response_not_set_case case_InstructionResponse_Response = 0 +const InstructionResponse_ProcessBundle_case case_InstructionResponse_Response = 1001 +const InstructionResponse_ProcessBundleProgress_case case_InstructionResponse_Response = 1002 +const InstructionResponse_ProcessBundleSplit_case case_InstructionResponse_Response = 1003 +const InstructionResponse_FinalizeBundle_case case_InstructionResponse_Response = 1004 +const InstructionResponse_MonitoringInfos_case case_InstructionResponse_Response = 1005 +const InstructionResponse_HarnessMonitoringInfos_case case_InstructionResponse_Response = 1006 +const InstructionResponse_SampleData_case case_InstructionResponse_Response = 1007 +const InstructionResponse_Register_case case_InstructionResponse_Response = 1000 + +func (x *InstructionResponse) WhichResponse() case_InstructionResponse_Response { + if x == nil { + return InstructionResponse_Response_not_set_case + } + switch x.Response.(type) { + case *InstructionResponse_ProcessBundle: + return InstructionResponse_ProcessBundle_case + case *InstructionResponse_ProcessBundleProgress: + return InstructionResponse_ProcessBundleProgress_case + case *InstructionResponse_ProcessBundleSplit: + return InstructionResponse_ProcessBundleSplit_case + case *InstructionResponse_FinalizeBundle: + return InstructionResponse_FinalizeBundle_case + case *InstructionResponse_MonitoringInfos: + return InstructionResponse_MonitoringInfos_case + case *InstructionResponse_HarnessMonitoringInfos: + return InstructionResponse_HarnessMonitoringInfos_case + case *InstructionResponse_SampleData: + return InstructionResponse_SampleData_case + case *InstructionResponse_Register: + return InstructionResponse_Register_case + default: + return InstructionResponse_Response_not_set_case + } +} + +type InstructionResponse_builder struct { + _ [0]func() // Prevents comparability and use of unkeyed literals for the builder. + + // (Required) A reference provided by the runner which represents a requests + // execution. The InstructionResponse MUST have the matching id when + // responding to the runner. + InstructionId string + // If this is specified, then this instruction has failed. + // A human readable string representing the reason as to why processing has + // failed. + Error string + // If the instruction did not fail, it is required to return an equivalent + // response type depending on the request this matches. + + // Fields of oneof Response: + ProcessBundle *ProcessBundleResponse + ProcessBundleProgress *ProcessBundleProgressResponse + ProcessBundleSplit *ProcessBundleSplitResponse + FinalizeBundle *FinalizeBundleResponse + MonitoringInfos *MonitoringInfosMetadataResponse + HarnessMonitoringInfos *HarnessMonitoringInfosResponse + SampleData *SampleDataResponse + // DEPRECATED + Register *RegisterResponse + // -- end of Response +} + +func (b0 InstructionResponse_builder) Build() *InstructionResponse { + m0 := &InstructionResponse{} + b, x := &b0, m0 + _, _ = b, x + x.InstructionId = b.InstructionId + x.Error = b.Error + if b.ProcessBundle != nil { + x.Response = &InstructionResponse_ProcessBundle{b.ProcessBundle} + } + if b.ProcessBundleProgress != nil { + x.Response = &InstructionResponse_ProcessBundleProgress{b.ProcessBundleProgress} + } + if b.ProcessBundleSplit != nil { + x.Response = &InstructionResponse_ProcessBundleSplit{b.ProcessBundleSplit} + } + if b.FinalizeBundle != nil { + x.Response = &InstructionResponse_FinalizeBundle{b.FinalizeBundle} + } + if b.MonitoringInfos != nil { + x.Response = &InstructionResponse_MonitoringInfos{b.MonitoringInfos} + } + if b.HarnessMonitoringInfos != nil { + x.Response = &InstructionResponse_HarnessMonitoringInfos{b.HarnessMonitoringInfos} + } + if b.SampleData != nil { + x.Response = &InstructionResponse_SampleData{b.SampleData} + } + if b.Register != nil { + x.Response = &InstructionResponse_Register{b.Register} + } + return m0 +} + +type case_InstructionResponse_Response protoreflect.FieldNumber + +func (x case_InstructionResponse_Response) String() string { + md := file_org_apache_beam_model_fn_execution_v1_beam_fn_api_proto_msgTypes[4].Descriptor() + if x == 0 { + return "not set" + } + return protoimpl.X.MessageFieldStringOf(md, protoreflect.FieldNumber(x)) +} + +type isInstructionResponse_Response interface { + isInstructionResponse_Response() +} + +type InstructionResponse_ProcessBundle struct { + ProcessBundle *ProcessBundleResponse `protobuf:"bytes,1001,opt,name=process_bundle,json=processBundle,proto3,oneof"` +} + +type InstructionResponse_ProcessBundleProgress struct { + ProcessBundleProgress *ProcessBundleProgressResponse `protobuf:"bytes,1002,opt,name=process_bundle_progress,json=processBundleProgress,proto3,oneof"` +} + +type InstructionResponse_ProcessBundleSplit struct { + ProcessBundleSplit *ProcessBundleSplitResponse `protobuf:"bytes,1003,opt,name=process_bundle_split,json=processBundleSplit,proto3,oneof"` +} + +type InstructionResponse_FinalizeBundle struct { + FinalizeBundle *FinalizeBundleResponse `protobuf:"bytes,1004,opt,name=finalize_bundle,json=finalizeBundle,proto3,oneof"` +} + +type InstructionResponse_MonitoringInfos struct { + MonitoringInfos *MonitoringInfosMetadataResponse `protobuf:"bytes,1005,opt,name=monitoring_infos,json=monitoringInfos,proto3,oneof"` +} + +type InstructionResponse_HarnessMonitoringInfos struct { + HarnessMonitoringInfos *HarnessMonitoringInfosResponse `protobuf:"bytes,1006,opt,name=harness_monitoring_infos,json=harnessMonitoringInfos,proto3,oneof"` } type InstructionResponse_SampleData struct { @@ -739,21 +1383,18 @@ func (*InstructionResponse_Register) isInstructionResponse_Response() {} // "beam:protocol:data_sampling:v1" capability. Samples are taken only from the // specified PCollection ids. An empty list will return everything. type SampleDataRequest struct { - state protoimpl.MessageState - sizeCache protoimpl.SizeCache - unknownFields protoimpl.UnknownFields - + state protoimpl.MessageState `protogen:"hybrid.v1"` // (Optional) The PCollection ids to filter for. PcollectionIds []string `protobuf:"bytes,1,rep,name=pcollection_ids,json=pcollectionIds,proto3" json:"pcollection_ids,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache } func (x *SampleDataRequest) Reset() { *x = SampleDataRequest{} - if protoimpl.UnsafeEnabled { - mi := &file_org_apache_beam_model_fn_execution_v1_beam_fn_api_proto_msgTypes[5] - ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) - ms.StoreMessageInfo(mi) - } + mi := &file_org_apache_beam_model_fn_execution_v1_beam_fn_api_proto_msgTypes[5] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) } func (x *SampleDataRequest) String() string { @@ -764,7 +1405,7 @@ func (*SampleDataRequest) ProtoMessage() {} func (x *SampleDataRequest) ProtoReflect() protoreflect.Message { mi := &file_org_apache_beam_model_fn_execution_v1_beam_fn_api_proto_msgTypes[5] - if protoimpl.UnsafeEnabled && x != nil { + if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { ms.StoreMessageInfo(mi) @@ -774,11 +1415,6 @@ func (x *SampleDataRequest) ProtoReflect() protoreflect.Message { return mi.MessageOf(x) } -// Deprecated: Use SampleDataRequest.ProtoReflect.Descriptor instead. -func (*SampleDataRequest) Descriptor() ([]byte, []int) { - return file_org_apache_beam_model_fn_execution_v1_beam_fn_api_proto_rawDescGZIP(), []int{5} -} - func (x *SampleDataRequest) GetPcollectionIds() []string { if x != nil { return x.PcollectionIds @@ -786,13 +1422,29 @@ func (x *SampleDataRequest) GetPcollectionIds() []string { return nil } +func (x *SampleDataRequest) SetPcollectionIds(v []string) { + x.PcollectionIds = v +} + +type SampleDataRequest_builder struct { + _ [0]func() // Prevents comparability and use of unkeyed literals for the builder. + + // (Optional) The PCollection ids to filter for. + PcollectionIds []string +} + +func (b0 SampleDataRequest_builder) Build() *SampleDataRequest { + m0 := &SampleDataRequest{} + b, x := &b0, m0 + _, _ = b, x + x.PcollectionIds = b.PcollectionIds + return m0 +} + // An element sampled when the SDK is processing a bundle. This is a proto // message to allow for additional per-element metadata. type SampledElement struct { - state protoimpl.MessageState - sizeCache protoimpl.SizeCache - unknownFields protoimpl.UnknownFields - + state protoimpl.MessageState `protogen:"hybrid.v1"` // (Required) Sampled raw bytes for an element. This is a // single encoded element in the nested context. Element []byte `protobuf:"bytes,1,opt,name=element,proto3" json:"element,omitempty"` @@ -800,16 +1452,16 @@ type SampledElement struct { SampleTimestamp *timestamppb.Timestamp `protobuf:"bytes,2,opt,name=sample_timestamp,json=sampleTimestamp,proto3" json:"sample_timestamp,omitempty"` // (Optional) This will be set if this element was sampled because of a user // exception. - Exception *SampledElement_Exception `protobuf:"bytes,3,opt,name=exception,proto3" json:"exception,omitempty"` + Exception *SampledElement_Exception `protobuf:"bytes,3,opt,name=exception,proto3" json:"exception,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache } func (x *SampledElement) Reset() { *x = SampledElement{} - if protoimpl.UnsafeEnabled { - mi := &file_org_apache_beam_model_fn_execution_v1_beam_fn_api_proto_msgTypes[6] - ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) - ms.StoreMessageInfo(mi) - } + mi := &file_org_apache_beam_model_fn_execution_v1_beam_fn_api_proto_msgTypes[6] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) } func (x *SampledElement) String() string { @@ -820,7 +1472,7 @@ func (*SampledElement) ProtoMessage() {} func (x *SampledElement) ProtoReflect() protoreflect.Message { mi := &file_org_apache_beam_model_fn_execution_v1_beam_fn_api_proto_msgTypes[6] - if protoimpl.UnsafeEnabled && x != nil { + if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { ms.StoreMessageInfo(mi) @@ -830,11 +1482,6 @@ func (x *SampledElement) ProtoReflect() protoreflect.Message { return mi.MessageOf(x) } -// Deprecated: Use SampledElement.ProtoReflect.Descriptor instead. -func (*SampledElement) Descriptor() ([]byte, []int) { - return file_org_apache_beam_model_fn_execution_v1_beam_fn_api_proto_rawDescGZIP(), []int{6} -} - func (x *SampledElement) GetElement() []byte { if x != nil { return x.Element @@ -856,24 +1503,81 @@ func (x *SampledElement) GetException() *SampledElement_Exception { return nil } +func (x *SampledElement) SetElement(v []byte) { + if v == nil { + v = []byte{} + } + x.Element = v +} + +func (x *SampledElement) SetSampleTimestamp(v *timestamppb.Timestamp) { + x.SampleTimestamp = v +} + +func (x *SampledElement) SetException(v *SampledElement_Exception) { + x.Exception = v +} + +func (x *SampledElement) HasSampleTimestamp() bool { + if x == nil { + return false + } + return x.SampleTimestamp != nil +} + +func (x *SampledElement) HasException() bool { + if x == nil { + return false + } + return x.Exception != nil +} + +func (x *SampledElement) ClearSampleTimestamp() { + x.SampleTimestamp = nil +} + +func (x *SampledElement) ClearException() { + x.Exception = nil +} + +type SampledElement_builder struct { + _ [0]func() // Prevents comparability and use of unkeyed literals for the builder. + + // (Required) Sampled raw bytes for an element. This is a + // single encoded element in the nested context. + Element []byte + // (Required) Timestamp of when the sample was taken. + SampleTimestamp *timestamppb.Timestamp + // (Optional) This will be set if this element was sampled because of a user + // exception. + Exception *SampledElement_Exception +} + +func (b0 SampledElement_builder) Build() *SampledElement { + m0 := &SampledElement{} + b, x := &b0, m0 + _, _ = b, x + x.Element = b.Element + x.SampleTimestamp = b.SampleTimestamp + x.Exception = b.Exception + return m0 +} + // If supported, the `SampleDataResponse` will contain samples from PCollections // based upon the filters specified in the request. type SampleDataResponse struct { - state protoimpl.MessageState - sizeCache protoimpl.SizeCache - unknownFields protoimpl.UnknownFields - + state protoimpl.MessageState `protogen:"hybrid.v1"` // Map from PCollection id to sampled elements. - ElementSamples map[string]*SampleDataResponse_ElementList `protobuf:"bytes,1,rep,name=element_samples,json=elementSamples,proto3" json:"element_samples,omitempty" protobuf_key:"bytes,1,opt,name=key,proto3" protobuf_val:"bytes,2,opt,name=value,proto3"` + ElementSamples map[string]*SampleDataResponse_ElementList `protobuf:"bytes,1,rep,name=element_samples,json=elementSamples,proto3" json:"element_samples,omitempty" protobuf_key:"bytes,1,opt,name=key" protobuf_val:"bytes,2,opt,name=value"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache } func (x *SampleDataResponse) Reset() { *x = SampleDataResponse{} - if protoimpl.UnsafeEnabled { - mi := &file_org_apache_beam_model_fn_execution_v1_beam_fn_api_proto_msgTypes[7] - ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) - ms.StoreMessageInfo(mi) - } + mi := &file_org_apache_beam_model_fn_execution_v1_beam_fn_api_proto_msgTypes[7] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) } func (x *SampleDataResponse) String() string { @@ -884,7 +1588,7 @@ func (*SampleDataResponse) ProtoMessage() {} func (x *SampleDataResponse) ProtoReflect() protoreflect.Message { mi := &file_org_apache_beam_model_fn_execution_v1_beam_fn_api_proto_msgTypes[7] - if protoimpl.UnsafeEnabled && x != nil { + if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { ms.StoreMessageInfo(mi) @@ -894,11 +1598,6 @@ func (x *SampleDataResponse) ProtoReflect() protoreflect.Message { return mi.MessageOf(x) } -// Deprecated: Use SampleDataResponse.ProtoReflect.Descriptor instead. -func (*SampleDataResponse) Descriptor() ([]byte, []int) { - return file_org_apache_beam_model_fn_execution_v1_beam_fn_api_proto_rawDescGZIP(), []int{7} -} - func (x *SampleDataResponse) GetElementSamples() map[string]*SampleDataResponse_ElementList { if x != nil { return x.ElementSamples @@ -906,6 +1605,25 @@ func (x *SampleDataResponse) GetElementSamples() map[string]*SampleDataResponse_ return nil } +func (x *SampleDataResponse) SetElementSamples(v map[string]*SampleDataResponse_ElementList) { + x.ElementSamples = v +} + +type SampleDataResponse_builder struct { + _ [0]func() // Prevents comparability and use of unkeyed literals for the builder. + + // Map from PCollection id to sampled elements. + ElementSamples map[string]*SampleDataResponse_ElementList +} + +func (b0 SampleDataResponse_builder) Build() *SampleDataResponse { + m0 := &SampleDataResponse{} + b, x := &b0, m0 + _, _ = b, x + x.ElementSamples = b.ElementSamples + return m0 +} + // A request to provide full MonitoringInfo associated with the entire SDK // harness process, not specific to a bundle. // @@ -919,18 +1637,16 @@ func (x *SampleDataResponse) GetElementSamples() map[string]*SampleDataResponse_ // as the MonitoringInfo could be reconstructed fully by overwriting its // payload field with the bytes specified here. type HarnessMonitoringInfosRequest struct { - state protoimpl.MessageState - sizeCache protoimpl.SizeCache + state protoimpl.MessageState `protogen:"hybrid.v1"` unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache } func (x *HarnessMonitoringInfosRequest) Reset() { *x = HarnessMonitoringInfosRequest{} - if protoimpl.UnsafeEnabled { - mi := &file_org_apache_beam_model_fn_execution_v1_beam_fn_api_proto_msgTypes[8] - ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) - ms.StoreMessageInfo(mi) - } + mi := &file_org_apache_beam_model_fn_execution_v1_beam_fn_api_proto_msgTypes[8] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) } func (x *HarnessMonitoringInfosRequest) String() string { @@ -941,7 +1657,7 @@ func (*HarnessMonitoringInfosRequest) ProtoMessage() {} func (x *HarnessMonitoringInfosRequest) ProtoReflect() protoreflect.Message { mi := &file_org_apache_beam_model_fn_execution_v1_beam_fn_api_proto_msgTypes[8] - if protoimpl.UnsafeEnabled && x != nil { + if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { ms.StoreMessageInfo(mi) @@ -951,16 +1667,20 @@ func (x *HarnessMonitoringInfosRequest) ProtoReflect() protoreflect.Message { return mi.MessageOf(x) } -// Deprecated: Use HarnessMonitoringInfosRequest.ProtoReflect.Descriptor instead. -func (*HarnessMonitoringInfosRequest) Descriptor() ([]byte, []int) { - return file_org_apache_beam_model_fn_execution_v1_beam_fn_api_proto_rawDescGZIP(), []int{8} +type HarnessMonitoringInfosRequest_builder struct { + _ [0]func() // Prevents comparability and use of unkeyed literals for the builder. + } -type HarnessMonitoringInfosResponse struct { - state protoimpl.MessageState - sizeCache protoimpl.SizeCache - unknownFields protoimpl.UnknownFields +func (b0 HarnessMonitoringInfosRequest_builder) Build() *HarnessMonitoringInfosRequest { + m0 := &HarnessMonitoringInfosRequest{} + b, x := &b0, m0 + _, _ = b, x + return m0 +} +type HarnessMonitoringInfosResponse struct { + state protoimpl.MessageState `protogen:"hybrid.v1"` // An identifier to MonitoringInfo.payload mapping containing // Metrics associated with the SDK harness, not a specific bundle. // @@ -973,16 +1693,16 @@ type HarnessMonitoringInfosResponse struct { // for the lifetime of the associated control connection as long // as the MonitoringInfo could be reconstructed fully by overwriting its // payload field with the bytes specified here. - MonitoringData map[string][]byte `protobuf:"bytes,1,rep,name=monitoring_data,json=monitoringData,proto3" json:"monitoring_data,omitempty" protobuf_key:"bytes,1,opt,name=key,proto3" protobuf_val:"bytes,2,opt,name=value,proto3"` + MonitoringData map[string][]byte `protobuf:"bytes,1,rep,name=monitoring_data,json=monitoringData,proto3" json:"monitoring_data,omitempty" protobuf_key:"bytes,1,opt,name=key" protobuf_val:"bytes,2,opt,name=value"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache } func (x *HarnessMonitoringInfosResponse) Reset() { *x = HarnessMonitoringInfosResponse{} - if protoimpl.UnsafeEnabled { - mi := &file_org_apache_beam_model_fn_execution_v1_beam_fn_api_proto_msgTypes[9] - ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) - ms.StoreMessageInfo(mi) - } + mi := &file_org_apache_beam_model_fn_execution_v1_beam_fn_api_proto_msgTypes[9] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) } func (x *HarnessMonitoringInfosResponse) String() string { @@ -993,7 +1713,7 @@ func (*HarnessMonitoringInfosResponse) ProtoMessage() {} func (x *HarnessMonitoringInfosResponse) ProtoReflect() protoreflect.Message { mi := &file_org_apache_beam_model_fn_execution_v1_beam_fn_api_proto_msgTypes[9] - if protoimpl.UnsafeEnabled && x != nil { + if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { ms.StoreMessageInfo(mi) @@ -1003,11 +1723,6 @@ func (x *HarnessMonitoringInfosResponse) ProtoReflect() protoreflect.Message { return mi.MessageOf(x) } -// Deprecated: Use HarnessMonitoringInfosResponse.ProtoReflect.Descriptor instead. -func (*HarnessMonitoringInfosResponse) Descriptor() ([]byte, []int) { - return file_org_apache_beam_model_fn_execution_v1_beam_fn_api_proto_rawDescGZIP(), []int{9} -} - func (x *HarnessMonitoringInfosResponse) GetMonitoringData() map[string][]byte { if x != nil { return x.MonitoringData @@ -1015,25 +1730,52 @@ func (x *HarnessMonitoringInfosResponse) GetMonitoringData() map[string][]byte { return nil } +func (x *HarnessMonitoringInfosResponse) SetMonitoringData(v map[string][]byte) { + x.MonitoringData = v +} + +type HarnessMonitoringInfosResponse_builder struct { + _ [0]func() // Prevents comparability and use of unkeyed literals for the builder. + + // An identifier to MonitoringInfo.payload mapping containing + // Metrics associated with the SDK harness, not a specific bundle. + // + // An SDK can report metrics using an identifier that only contains the + // associated payload. A runner who wants to receive the full metrics + // information can request all the monitoring metadata via a + // MonitoringInfosMetadataRequest providing a list of ids as necessary. + // + // The SDK is allowed to reuse the identifiers + // for the lifetime of the associated control connection as long + // as the MonitoringInfo could be reconstructed fully by overwriting its + // payload field with the bytes specified here. + MonitoringData map[string][]byte +} + +func (b0 HarnessMonitoringInfosResponse_builder) Build() *HarnessMonitoringInfosResponse { + m0 := &HarnessMonitoringInfosResponse{} + b, x := &b0, m0 + _, _ = b, x + x.MonitoringData = b.MonitoringData + return m0 +} + // A list of objects which can be referred to by the runner in // future requests. // Stable type RegisterRequest struct { - state protoimpl.MessageState - sizeCache protoimpl.SizeCache - unknownFields protoimpl.UnknownFields - + state protoimpl.MessageState `protogen:"hybrid.v1"` // (Optional) The set of descriptors used to process bundles. ProcessBundleDescriptor []*ProcessBundleDescriptor `protobuf:"bytes,1,rep,name=process_bundle_descriptor,json=processBundleDescriptor,proto3" json:"process_bundle_descriptor,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache } func (x *RegisterRequest) Reset() { *x = RegisterRequest{} - if protoimpl.UnsafeEnabled { - mi := &file_org_apache_beam_model_fn_execution_v1_beam_fn_api_proto_msgTypes[10] - ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) - ms.StoreMessageInfo(mi) - } + mi := &file_org_apache_beam_model_fn_execution_v1_beam_fn_api_proto_msgTypes[10] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) } func (x *RegisterRequest) String() string { @@ -1044,7 +1786,7 @@ func (*RegisterRequest) ProtoMessage() {} func (x *RegisterRequest) ProtoReflect() protoreflect.Message { mi := &file_org_apache_beam_model_fn_execution_v1_beam_fn_api_proto_msgTypes[10] - if protoimpl.UnsafeEnabled && x != nil { + if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { ms.StoreMessageInfo(mi) @@ -1054,11 +1796,6 @@ func (x *RegisterRequest) ProtoReflect() protoreflect.Message { return mi.MessageOf(x) } -// Deprecated: Use RegisterRequest.ProtoReflect.Descriptor instead. -func (*RegisterRequest) Descriptor() ([]byte, []int) { - return file_org_apache_beam_model_fn_execution_v1_beam_fn_api_proto_rawDescGZIP(), []int{10} -} - func (x *RegisterRequest) GetProcessBundleDescriptor() []*ProcessBundleDescriptor { if x != nil { return x.ProcessBundleDescriptor @@ -1066,20 +1803,37 @@ func (x *RegisterRequest) GetProcessBundleDescriptor() []*ProcessBundleDescripto return nil } +func (x *RegisterRequest) SetProcessBundleDescriptor(v []*ProcessBundleDescriptor) { + x.ProcessBundleDescriptor = v +} + +type RegisterRequest_builder struct { + _ [0]func() // Prevents comparability and use of unkeyed literals for the builder. + + // (Optional) The set of descriptors used to process bundles. + ProcessBundleDescriptor []*ProcessBundleDescriptor +} + +func (b0 RegisterRequest_builder) Build() *RegisterRequest { + m0 := &RegisterRequest{} + b, x := &b0, m0 + _, _ = b, x + x.ProcessBundleDescriptor = b.ProcessBundleDescriptor + return m0 +} + // Stable type RegisterResponse struct { - state protoimpl.MessageState - sizeCache protoimpl.SizeCache + state protoimpl.MessageState `protogen:"hybrid.v1"` unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache } func (x *RegisterResponse) Reset() { *x = RegisterResponse{} - if protoimpl.UnsafeEnabled { - mi := &file_org_apache_beam_model_fn_execution_v1_beam_fn_api_proto_msgTypes[11] - ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) - ms.StoreMessageInfo(mi) - } + mi := &file_org_apache_beam_model_fn_execution_v1_beam_fn_api_proto_msgTypes[11] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) } func (x *RegisterResponse) String() string { @@ -1090,7 +1844,7 @@ func (*RegisterResponse) ProtoMessage() {} func (x *RegisterResponse) ProtoReflect() protoreflect.Message { mi := &file_org_apache_beam_model_fn_execution_v1_beam_fn_api_proto_msgTypes[11] - if protoimpl.UnsafeEnabled && x != nil { + if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { ms.StoreMessageInfo(mi) @@ -1100,30 +1854,34 @@ func (x *RegisterResponse) ProtoReflect() protoreflect.Message { return mi.MessageOf(x) } -// Deprecated: Use RegisterResponse.ProtoReflect.Descriptor instead. -func (*RegisterResponse) Descriptor() ([]byte, []int) { - return file_org_apache_beam_model_fn_execution_v1_beam_fn_api_proto_rawDescGZIP(), []int{11} +type RegisterResponse_builder struct { + _ [0]func() // Prevents comparability and use of unkeyed literals for the builder. + +} + +func (b0 RegisterResponse_builder) Build() *RegisterResponse { + m0 := &RegisterResponse{} + b, x := &b0, m0 + _, _ = b, x + return m0 } // Definitions that should be used to construct the bundle processing graph. type ProcessBundleDescriptor struct { - state protoimpl.MessageState - sizeCache protoimpl.SizeCache - unknownFields protoimpl.UnknownFields - + state protoimpl.MessageState `protogen:"hybrid.v1"` // (Required) A pipeline level unique id which can be used as a reference to // refer to this. Id string `protobuf:"bytes,1,opt,name=id,proto3" json:"id,omitempty"` // (Required) A map from pipeline-scoped id to PTransform. - Transforms map[string]*pipeline_v1.PTransform `protobuf:"bytes,2,rep,name=transforms,proto3" json:"transforms,omitempty" protobuf_key:"bytes,1,opt,name=key,proto3" protobuf_val:"bytes,2,opt,name=value,proto3"` + Transforms map[string]*pipeline_v1.PTransform `protobuf:"bytes,2,rep,name=transforms,proto3" json:"transforms,omitempty" protobuf_key:"bytes,1,opt,name=key" protobuf_val:"bytes,2,opt,name=value"` // (Required) A map from pipeline-scoped id to PCollection. - Pcollections map[string]*pipeline_v1.PCollection `protobuf:"bytes,3,rep,name=pcollections,proto3" json:"pcollections,omitempty" protobuf_key:"bytes,1,opt,name=key,proto3" protobuf_val:"bytes,2,opt,name=value,proto3"` + Pcollections map[string]*pipeline_v1.PCollection `protobuf:"bytes,3,rep,name=pcollections,proto3" json:"pcollections,omitempty" protobuf_key:"bytes,1,opt,name=key" protobuf_val:"bytes,2,opt,name=value"` // (Required) A map from pipeline-scoped id to WindowingStrategy. - WindowingStrategies map[string]*pipeline_v1.WindowingStrategy `protobuf:"bytes,4,rep,name=windowing_strategies,json=windowingStrategies,proto3" json:"windowing_strategies,omitempty" protobuf_key:"bytes,1,opt,name=key,proto3" protobuf_val:"bytes,2,opt,name=value,proto3"` + WindowingStrategies map[string]*pipeline_v1.WindowingStrategy `protobuf:"bytes,4,rep,name=windowing_strategies,json=windowingStrategies,proto3" json:"windowing_strategies,omitempty" protobuf_key:"bytes,1,opt,name=key" protobuf_val:"bytes,2,opt,name=value"` // (Required) A map from pipeline-scoped id to Coder. - Coders map[string]*pipeline_v1.Coder `protobuf:"bytes,5,rep,name=coders,proto3" json:"coders,omitempty" protobuf_key:"bytes,1,opt,name=key,proto3" protobuf_val:"bytes,2,opt,name=value,proto3"` + Coders map[string]*pipeline_v1.Coder `protobuf:"bytes,5,rep,name=coders,proto3" json:"coders,omitempty" protobuf_key:"bytes,1,opt,name=key" protobuf_val:"bytes,2,opt,name=value"` // (Required) A map from pipeline-scoped id to Environment. - Environments map[string]*pipeline_v1.Environment `protobuf:"bytes,6,rep,name=environments,proto3" json:"environments,omitempty" protobuf_key:"bytes,1,opt,name=key,proto3" protobuf_val:"bytes,2,opt,name=value,proto3"` + Environments map[string]*pipeline_v1.Environment `protobuf:"bytes,6,rep,name=environments,proto3" json:"environments,omitempty" protobuf_key:"bytes,1,opt,name=key" protobuf_val:"bytes,2,opt,name=value"` // A descriptor describing the end point to use for State API // calls. Required if the Runner intends to send remote references over the // data plane or if any of the transforms rely on user state or side inputs. @@ -1131,15 +1889,15 @@ type ProcessBundleDescriptor struct { // A descriptor describing the end point to use for Data API for user timers. // Required if the ProcessBundleDescriptor contains any transforms that have user timers. TimerApiServiceDescriptor *pipeline_v1.ApiServiceDescriptor `protobuf:"bytes,8,opt,name=timer_api_service_descriptor,json=timerApiServiceDescriptor,proto3" json:"timer_api_service_descriptor,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache } func (x *ProcessBundleDescriptor) Reset() { *x = ProcessBundleDescriptor{} - if protoimpl.UnsafeEnabled { - mi := &file_org_apache_beam_model_fn_execution_v1_beam_fn_api_proto_msgTypes[12] - ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) - ms.StoreMessageInfo(mi) - } + mi := &file_org_apache_beam_model_fn_execution_v1_beam_fn_api_proto_msgTypes[12] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) } func (x *ProcessBundleDescriptor) String() string { @@ -1150,7 +1908,7 @@ func (*ProcessBundleDescriptor) ProtoMessage() {} func (x *ProcessBundleDescriptor) ProtoReflect() protoreflect.Message { mi := &file_org_apache_beam_model_fn_execution_v1_beam_fn_api_proto_msgTypes[12] - if protoimpl.UnsafeEnabled && x != nil { + if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { ms.StoreMessageInfo(mi) @@ -1160,11 +1918,6 @@ func (x *ProcessBundleDescriptor) ProtoReflect() protoreflect.Message { return mi.MessageOf(x) } -// Deprecated: Use ProcessBundleDescriptor.ProtoReflect.Descriptor instead. -func (*ProcessBundleDescriptor) Descriptor() ([]byte, []int) { - return file_org_apache_beam_model_fn_execution_v1_beam_fn_api_proto_rawDescGZIP(), []int{12} -} - func (x *ProcessBundleDescriptor) GetId() string { if x != nil { return x.Id @@ -1221,15 +1974,106 @@ func (x *ProcessBundleDescriptor) GetTimerApiServiceDescriptor() *pipeline_v1.Ap return nil } +func (x *ProcessBundleDescriptor) SetId(v string) { + x.Id = v +} + +func (x *ProcessBundleDescriptor) SetTransforms(v map[string]*pipeline_v1.PTransform) { + x.Transforms = v +} + +func (x *ProcessBundleDescriptor) SetPcollections(v map[string]*pipeline_v1.PCollection) { + x.Pcollections = v +} + +func (x *ProcessBundleDescriptor) SetWindowingStrategies(v map[string]*pipeline_v1.WindowingStrategy) { + x.WindowingStrategies = v +} + +func (x *ProcessBundleDescriptor) SetCoders(v map[string]*pipeline_v1.Coder) { + x.Coders = v +} + +func (x *ProcessBundleDescriptor) SetEnvironments(v map[string]*pipeline_v1.Environment) { + x.Environments = v +} + +func (x *ProcessBundleDescriptor) SetStateApiServiceDescriptor(v *pipeline_v1.ApiServiceDescriptor) { + x.StateApiServiceDescriptor = v +} + +func (x *ProcessBundleDescriptor) SetTimerApiServiceDescriptor(v *pipeline_v1.ApiServiceDescriptor) { + x.TimerApiServiceDescriptor = v +} + +func (x *ProcessBundleDescriptor) HasStateApiServiceDescriptor() bool { + if x == nil { + return false + } + return x.StateApiServiceDescriptor != nil +} + +func (x *ProcessBundleDescriptor) HasTimerApiServiceDescriptor() bool { + if x == nil { + return false + } + return x.TimerApiServiceDescriptor != nil +} + +func (x *ProcessBundleDescriptor) ClearStateApiServiceDescriptor() { + x.StateApiServiceDescriptor = nil +} + +func (x *ProcessBundleDescriptor) ClearTimerApiServiceDescriptor() { + x.TimerApiServiceDescriptor = nil +} + +type ProcessBundleDescriptor_builder struct { + _ [0]func() // Prevents comparability and use of unkeyed literals for the builder. + + // (Required) A pipeline level unique id which can be used as a reference to + // refer to this. + Id string + // (Required) A map from pipeline-scoped id to PTransform. + Transforms map[string]*pipeline_v1.PTransform + // (Required) A map from pipeline-scoped id to PCollection. + Pcollections map[string]*pipeline_v1.PCollection + // (Required) A map from pipeline-scoped id to WindowingStrategy. + WindowingStrategies map[string]*pipeline_v1.WindowingStrategy + // (Required) A map from pipeline-scoped id to Coder. + Coders map[string]*pipeline_v1.Coder + // (Required) A map from pipeline-scoped id to Environment. + Environments map[string]*pipeline_v1.Environment + // A descriptor describing the end point to use for State API + // calls. Required if the Runner intends to send remote references over the + // data plane or if any of the transforms rely on user state or side inputs. + StateApiServiceDescriptor *pipeline_v1.ApiServiceDescriptor + // A descriptor describing the end point to use for Data API for user timers. + // Required if the ProcessBundleDescriptor contains any transforms that have user timers. + TimerApiServiceDescriptor *pipeline_v1.ApiServiceDescriptor +} + +func (b0 ProcessBundleDescriptor_builder) Build() *ProcessBundleDescriptor { + m0 := &ProcessBundleDescriptor{} + b, x := &b0, m0 + _, _ = b, x + x.Id = b.Id + x.Transforms = b.Transforms + x.Pcollections = b.Pcollections + x.WindowingStrategies = b.WindowingStrategies + x.Coders = b.Coders + x.Environments = b.Environments + x.StateApiServiceDescriptor = b.StateApiServiceDescriptor + x.TimerApiServiceDescriptor = b.TimerApiServiceDescriptor + return m0 +} + // One of the applications specifying the scope of work for a bundle. // See // https://docs.google.com/document/d/1tUDb45sStdR8u7-jBkGdw3OGFK7aa2-V7eo86zYSE_4/edit#heading=h.9g3g5weg2u9 // for further details. type BundleApplication struct { - state protoimpl.MessageState - sizeCache protoimpl.SizeCache - unknownFields protoimpl.UnknownFields - + state protoimpl.MessageState `protogen:"hybrid.v1"` // (Required) The transform to which to pass the element TransformId string `protobuf:"bytes,1,opt,name=transform_id,json=transformId,proto3" json:"transform_id,omitempty"` // (Required) Name of the transform's input to which to pass the element. @@ -1243,20 +2087,20 @@ type BundleApplication struct { // // If there is no watermark reported from RestrictionTracker, the runner will // use MIN_TIMESTAMP by default. - OutputWatermarks map[string]*timestamppb.Timestamp `protobuf:"bytes,4,rep,name=output_watermarks,json=outputWatermarks,proto3" json:"output_watermarks,omitempty" protobuf_key:"bytes,1,opt,name=key,proto3" protobuf_val:"bytes,2,opt,name=value,proto3"` + OutputWatermarks map[string]*timestamppb.Timestamp `protobuf:"bytes,4,rep,name=output_watermarks,json=outputWatermarks,proto3" json:"output_watermarks,omitempty" protobuf_key:"bytes,1,opt,name=key" protobuf_val:"bytes,2,opt,name=value"` // Whether this application potentially produces an unbounded // amount of data. Note that this should only be set to BOUNDED if and // only if the application is known to produce a finite amount of output. - IsBounded pipeline_v1.IsBounded_Enum `protobuf:"varint,5,opt,name=is_bounded,json=isBounded,proto3,enum=org.apache.beam.model.pipeline.v1.IsBounded_Enum" json:"is_bounded,omitempty"` + IsBounded pipeline_v1.IsBounded_Enum `protobuf:"varint,5,opt,name=is_bounded,json=isBounded,proto3,enum=org.apache.beam.model.pipeline.v1.IsBounded_Enum" json:"is_bounded,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache } func (x *BundleApplication) Reset() { *x = BundleApplication{} - if protoimpl.UnsafeEnabled { - mi := &file_org_apache_beam_model_fn_execution_v1_beam_fn_api_proto_msgTypes[13] - ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) - ms.StoreMessageInfo(mi) - } + mi := &file_org_apache_beam_model_fn_execution_v1_beam_fn_api_proto_msgTypes[13] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) } func (x *BundleApplication) String() string { @@ -1267,7 +2111,7 @@ func (*BundleApplication) ProtoMessage() {} func (x *BundleApplication) ProtoReflect() protoreflect.Message { mi := &file_org_apache_beam_model_fn_execution_v1_beam_fn_api_proto_msgTypes[13] - if protoimpl.UnsafeEnabled && x != nil { + if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { ms.StoreMessageInfo(mi) @@ -1277,11 +2121,6 @@ func (x *BundleApplication) ProtoReflect() protoreflect.Message { return mi.MessageOf(x) } -// Deprecated: Use BundleApplication.ProtoReflect.Descriptor instead. -func (*BundleApplication) Descriptor() ([]byte, []int) { - return file_org_apache_beam_model_fn_execution_v1_beam_fn_api_proto_rawDescGZIP(), []int{13} -} - func (x *BundleApplication) GetTransformId() string { if x != nil { return x.TransformId @@ -1317,29 +2156,84 @@ func (x *BundleApplication) GetIsBounded() pipeline_v1.IsBounded_Enum { return pipeline_v1.IsBounded_Enum(0) } +func (x *BundleApplication) SetTransformId(v string) { + x.TransformId = v +} + +func (x *BundleApplication) SetInputId(v string) { + x.InputId = v +} + +func (x *BundleApplication) SetElement(v []byte) { + if v == nil { + v = []byte{} + } + x.Element = v +} + +func (x *BundleApplication) SetOutputWatermarks(v map[string]*timestamppb.Timestamp) { + x.OutputWatermarks = v +} + +func (x *BundleApplication) SetIsBounded(v pipeline_v1.IsBounded_Enum) { + x.IsBounded = v +} + +type BundleApplication_builder struct { + _ [0]func() // Prevents comparability and use of unkeyed literals for the builder. + + // (Required) The transform to which to pass the element + TransformId string + // (Required) Name of the transform's input to which to pass the element. + InputId string + // (Required) The encoded element to pass to the transform. + Element []byte + // The map is keyed by the local output name of the PTransform. Each + // value represents a lower bound on the timestamps of elements that + // are produced by this PTransform into each of its output PCollections + // when invoked with this application. + // + // If there is no watermark reported from RestrictionTracker, the runner will + // use MIN_TIMESTAMP by default. + OutputWatermarks map[string]*timestamppb.Timestamp + // Whether this application potentially produces an unbounded + // amount of data. Note that this should only be set to BOUNDED if and + // only if the application is known to produce a finite amount of output. + IsBounded pipeline_v1.IsBounded_Enum +} + +func (b0 BundleApplication_builder) Build() *BundleApplication { + m0 := &BundleApplication{} + b, x := &b0, m0 + _, _ = b, x + x.TransformId = b.TransformId + x.InputId = b.InputId + x.Element = b.Element + x.OutputWatermarks = b.OutputWatermarks + x.IsBounded = b.IsBounded + return m0 +} + // An Application should be scheduled for execution after a delay. // Either an absolute timestamp or a relative timestamp can represent a // scheduled execution time. type DelayedBundleApplication struct { - state protoimpl.MessageState - sizeCache protoimpl.SizeCache - unknownFields protoimpl.UnknownFields - + state protoimpl.MessageState `protogen:"hybrid.v1"` // (Required) The application that should be scheduled. Application *BundleApplication `protobuf:"bytes,1,opt,name=application,proto3" json:"application,omitempty"` // Recommended time delay at which the application should be scheduled to // execute by the runner. Time delay that equals 0 may be scheduled to execute // immediately. The unit of time delay should be microsecond. RequestedTimeDelay *durationpb.Duration `protobuf:"bytes,2,opt,name=requested_time_delay,json=requestedTimeDelay,proto3" json:"requested_time_delay,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache } func (x *DelayedBundleApplication) Reset() { *x = DelayedBundleApplication{} - if protoimpl.UnsafeEnabled { - mi := &file_org_apache_beam_model_fn_execution_v1_beam_fn_api_proto_msgTypes[14] - ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) - ms.StoreMessageInfo(mi) - } + mi := &file_org_apache_beam_model_fn_execution_v1_beam_fn_api_proto_msgTypes[14] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) } func (x *DelayedBundleApplication) String() string { @@ -1350,7 +2244,7 @@ func (*DelayedBundleApplication) ProtoMessage() {} func (x *DelayedBundleApplication) ProtoReflect() protoreflect.Message { mi := &file_org_apache_beam_model_fn_execution_v1_beam_fn_api_proto_msgTypes[14] - if protoimpl.UnsafeEnabled && x != nil { + if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { ms.StoreMessageInfo(mi) @@ -1360,11 +2254,6 @@ func (x *DelayedBundleApplication) ProtoReflect() protoreflect.Message { return mi.MessageOf(x) } -// Deprecated: Use DelayedBundleApplication.ProtoReflect.Descriptor instead. -func (*DelayedBundleApplication) Descriptor() ([]byte, []int) { - return file_org_apache_beam_model_fn_execution_v1_beam_fn_api_proto_rawDescGZIP(), []int{14} -} - func (x *DelayedBundleApplication) GetApplication() *BundleApplication { if x != nil { return x.Application @@ -1379,13 +2268,60 @@ func (x *DelayedBundleApplication) GetRequestedTimeDelay() *durationpb.Duration return nil } +func (x *DelayedBundleApplication) SetApplication(v *BundleApplication) { + x.Application = v +} + +func (x *DelayedBundleApplication) SetRequestedTimeDelay(v *durationpb.Duration) { + x.RequestedTimeDelay = v +} + +func (x *DelayedBundleApplication) HasApplication() bool { + if x == nil { + return false + } + return x.Application != nil +} + +func (x *DelayedBundleApplication) HasRequestedTimeDelay() bool { + if x == nil { + return false + } + return x.RequestedTimeDelay != nil +} + +func (x *DelayedBundleApplication) ClearApplication() { + x.Application = nil +} + +func (x *DelayedBundleApplication) ClearRequestedTimeDelay() { + x.RequestedTimeDelay = nil +} + +type DelayedBundleApplication_builder struct { + _ [0]func() // Prevents comparability and use of unkeyed literals for the builder. + + // (Required) The application that should be scheduled. + Application *BundleApplication + // Recommended time delay at which the application should be scheduled to + // execute by the runner. Time delay that equals 0 may be scheduled to execute + // immediately. The unit of time delay should be microsecond. + RequestedTimeDelay *durationpb.Duration +} + +func (b0 DelayedBundleApplication_builder) Build() *DelayedBundleApplication { + m0 := &DelayedBundleApplication{} + b, x := &b0, m0 + _, _ = b, x + x.Application = b.Application + x.RequestedTimeDelay = b.RequestedTimeDelay + return m0 +} + // A request to process a given bundle. // Stable type ProcessBundleRequest struct { - state protoimpl.MessageState - sizeCache protoimpl.SizeCache - unknownFields protoimpl.UnknownFields - + state protoimpl.MessageState `protogen:"hybrid.v1"` // (Required) A reference to the process bundle descriptor that must be // instantiated and executed by the SDK harness. ProcessBundleDescriptorId string `protobuf:"bytes,1,opt,name=process_bundle_descriptor_id,json=processBundleDescriptorId,proto3" json:"process_bundle_descriptor_id,omitempty"` @@ -1407,16 +2343,16 @@ type ProcessBundleRequest struct { // side. This field can be set only if the SDK declares that it supports the // beam:protocol:control_request_elements_embedding:v1 capability. See more // at https://s.apache.org/beam-fn-api-control-data-embedding. - Elements *Elements `protobuf:"bytes,3,opt,name=elements,proto3" json:"elements,omitempty"` + Elements *Elements `protobuf:"bytes,3,opt,name=elements,proto3" json:"elements,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache } func (x *ProcessBundleRequest) Reset() { *x = ProcessBundleRequest{} - if protoimpl.UnsafeEnabled { - mi := &file_org_apache_beam_model_fn_execution_v1_beam_fn_api_proto_msgTypes[15] - ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) - ms.StoreMessageInfo(mi) - } + mi := &file_org_apache_beam_model_fn_execution_v1_beam_fn_api_proto_msgTypes[15] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) } func (x *ProcessBundleRequest) String() string { @@ -1427,7 +2363,7 @@ func (*ProcessBundleRequest) ProtoMessage() {} func (x *ProcessBundleRequest) ProtoReflect() protoreflect.Message { mi := &file_org_apache_beam_model_fn_execution_v1_beam_fn_api_proto_msgTypes[15] - if protoimpl.UnsafeEnabled && x != nil { + if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { ms.StoreMessageInfo(mi) @@ -1437,11 +2373,6 @@ func (x *ProcessBundleRequest) ProtoReflect() protoreflect.Message { return mi.MessageOf(x) } -// Deprecated: Use ProcessBundleRequest.ProtoReflect.Descriptor instead. -func (*ProcessBundleRequest) Descriptor() ([]byte, []int) { - return file_org_apache_beam_model_fn_execution_v1_beam_fn_api_proto_rawDescGZIP(), []int{15} -} - func (x *ProcessBundleRequest) GetProcessBundleDescriptorId() string { if x != nil { return x.ProcessBundleDescriptorId @@ -1463,11 +2394,68 @@ func (x *ProcessBundleRequest) GetElements() *Elements { return nil } -type ProcessBundleResponse struct { - state protoimpl.MessageState - sizeCache protoimpl.SizeCache - unknownFields protoimpl.UnknownFields +func (x *ProcessBundleRequest) SetProcessBundleDescriptorId(v string) { + x.ProcessBundleDescriptorId = v +} + +func (x *ProcessBundleRequest) SetCacheTokens(v []*ProcessBundleRequest_CacheToken) { + x.CacheTokens = v +} + +func (x *ProcessBundleRequest) SetElements(v *Elements) { + x.Elements = v +} + +func (x *ProcessBundleRequest) HasElements() bool { + if x == nil { + return false + } + return x.Elements != nil +} + +func (x *ProcessBundleRequest) ClearElements() { + x.Elements = nil +} + +type ProcessBundleRequest_builder struct { + _ [0]func() // Prevents comparability and use of unkeyed literals for the builder. + + // (Required) A reference to the process bundle descriptor that must be + // instantiated and executed by the SDK harness. + ProcessBundleDescriptorId string + // (Optional) A list of cache tokens that can be used by an SDK to reuse + // cached data returned by the State API across multiple bundles. + // + // Note that SDKs that can efficiently consume this field should declare + // the beam:protocol:state_caching:v1 capability enabling runners to reduce + // the amount of memory used. + // + // See https://s.apache.org/beam-fn-state-api-and-bundle-processing#heading=h.7ghoih5aig5m + // for additional details on how to use the cache token with the State API + // to cache data across bundle boundaries. + CacheTokens []*ProcessBundleRequest_CacheToken + // (Optional) Elements to be processed with the bundle. Either all or + // none of the bundle elements should be included in the ProcessBundleRequest. + // This embedding is to achieve better efficiency for bundles that contain + // only small amounts of data and are cheap to be processed on the SDK harness + // side. This field can be set only if the SDK declares that it supports the + // beam:protocol:control_request_elements_embedding:v1 capability. See more + // at https://s.apache.org/beam-fn-api-control-data-embedding. + Elements *Elements +} + +func (b0 ProcessBundleRequest_builder) Build() *ProcessBundleRequest { + m0 := &ProcessBundleRequest{} + b, x := &b0, m0 + _, _ = b, x + x.ProcessBundleDescriptorId = b.ProcessBundleDescriptorId + x.CacheTokens = b.CacheTokens + x.Elements = b.Elements + return m0 +} +type ProcessBundleResponse struct { + state protoimpl.MessageState `protogen:"hybrid.v1"` // (Optional) Specifies that the bundle has not been completed and the // following applications need to be scheduled and executed in the future. // A runner that does not yet support residual roots MUST still check that @@ -1494,7 +2482,7 @@ type ProcessBundleResponse struct { // The SDK is allowed to reuse the identifiers across multiple bundles as long // as the MonitoringInfo could be reconstructed fully by overwriting its // payload field with the bytes specified here. - MonitoringData map[string][]byte `protobuf:"bytes,5,rep,name=monitoring_data,json=monitoringData,proto3" json:"monitoring_data,omitempty" protobuf_key:"bytes,1,opt,name=key,proto3" protobuf_val:"bytes,2,opt,name=value,proto3"` + MonitoringData map[string][]byte `protobuf:"bytes,5,rep,name=monitoring_data,json=monitoringData,proto3" json:"monitoring_data,omitempty" protobuf_key:"bytes,1,opt,name=key" protobuf_val:"bytes,2,opt,name=value"` // (Optional) Output elements of the processed bundle. Either all or // none of the bundle elements should be included in the ProcessBundleResponse. // This embedding is to achieve better efficiency for bundles that only @@ -1502,16 +2490,16 @@ type ProcessBundleResponse struct { // declares that it supports the // beam:protocol:control_request_elements_embedding:v1 capability. See more at // https://s.apache.org/beam-fn-api-control-data-embedding. - Elements *Elements `protobuf:"bytes,6,opt,name=elements,proto3" json:"elements,omitempty"` + Elements *Elements `protobuf:"bytes,6,opt,name=elements,proto3" json:"elements,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache } func (x *ProcessBundleResponse) Reset() { *x = ProcessBundleResponse{} - if protoimpl.UnsafeEnabled { - mi := &file_org_apache_beam_model_fn_execution_v1_beam_fn_api_proto_msgTypes[16] - ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) - ms.StoreMessageInfo(mi) - } + mi := &file_org_apache_beam_model_fn_execution_v1_beam_fn_api_proto_msgTypes[16] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) } func (x *ProcessBundleResponse) String() string { @@ -1522,7 +2510,7 @@ func (*ProcessBundleResponse) ProtoMessage() {} func (x *ProcessBundleResponse) ProtoReflect() protoreflect.Message { mi := &file_org_apache_beam_model_fn_execution_v1_beam_fn_api_proto_msgTypes[16] - if protoimpl.UnsafeEnabled && x != nil { + if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { ms.StoreMessageInfo(mi) @@ -1532,11 +2520,6 @@ func (x *ProcessBundleResponse) ProtoReflect() protoreflect.Message { return mi.MessageOf(x) } -// Deprecated: Use ProcessBundleResponse.ProtoReflect.Descriptor instead. -func (*ProcessBundleResponse) Descriptor() ([]byte, []int) { - return file_org_apache_beam_model_fn_execution_v1_beam_fn_api_proto_rawDescGZIP(), []int{16} -} - func (x *ProcessBundleResponse) GetResidualRoots() []*DelayedBundleApplication { if x != nil { return x.ResidualRoots @@ -1572,26 +2555,106 @@ func (x *ProcessBundleResponse) GetElements() *Elements { return nil } +func (x *ProcessBundleResponse) SetResidualRoots(v []*DelayedBundleApplication) { + x.ResidualRoots = v +} + +func (x *ProcessBundleResponse) SetMonitoringInfos(v []*pipeline_v1.MonitoringInfo) { + x.MonitoringInfos = v +} + +func (x *ProcessBundleResponse) SetRequiresFinalization(v bool) { + x.RequiresFinalization = v +} + +func (x *ProcessBundleResponse) SetMonitoringData(v map[string][]byte) { + x.MonitoringData = v +} + +func (x *ProcessBundleResponse) SetElements(v *Elements) { + x.Elements = v +} + +func (x *ProcessBundleResponse) HasElements() bool { + if x == nil { + return false + } + return x.Elements != nil +} + +func (x *ProcessBundleResponse) ClearElements() { + x.Elements = nil +} + +type ProcessBundleResponse_builder struct { + _ [0]func() // Prevents comparability and use of unkeyed literals for the builder. + + // (Optional) Specifies that the bundle has not been completed and the + // following applications need to be scheduled and executed in the future. + // A runner that does not yet support residual roots MUST still check that + // this is empty for correctness. + // + // Note that these residual roots must not have been returned as part of a + // prior split for this bundle. + ResidualRoots []*DelayedBundleApplication + // DEPRECATED (Required) The list of metrics or other MonitoredState + // collected while processing this bundle. + MonitoringInfos []*pipeline_v1.MonitoringInfo + // (Optional) Specifies that the runner must callback to this worker + // once the output of the bundle is committed. The Runner must send a + // FinalizeBundleRequest with the instruction id of the ProcessBundleRequest + // that is related to this ProcessBundleResponse. + RequiresFinalization bool + // An identifier to MonitoringInfo.payload mapping. + // + // An SDK can report metrics using an identifier that only contains the + // associated payload. A runner who wants to receive the full metrics + // information can request all the monitoring metadata via a + // MonitoringInfosMetadataRequest providing a list of ids as necessary. + // + // The SDK is allowed to reuse the identifiers across multiple bundles as long + // as the MonitoringInfo could be reconstructed fully by overwriting its + // payload field with the bytes specified here. + MonitoringData map[string][]byte + // (Optional) Output elements of the processed bundle. Either all or + // none of the bundle elements should be included in the ProcessBundleResponse. + // This embedding is to achieve better efficiency for bundles that only + // contain small amounts of data. his field can be set only if the runner + // declares that it supports the + // beam:protocol:control_request_elements_embedding:v1 capability. See more at + // https://s.apache.org/beam-fn-api-control-data-embedding. + Elements *Elements +} + +func (b0 ProcessBundleResponse_builder) Build() *ProcessBundleResponse { + m0 := &ProcessBundleResponse{} + b, x := &b0, m0 + _, _ = b, x + x.ResidualRoots = b.ResidualRoots + x.MonitoringInfos = b.MonitoringInfos + x.RequiresFinalization = b.RequiresFinalization + x.MonitoringData = b.MonitoringData + x.Elements = b.Elements + return m0 +} + // A request to report progress information for a given bundle. // This is an optional request to be handled and is used to support advanced // SDK features such as SplittableDoFn, user level metrics etc. type ProcessBundleProgressRequest struct { - state protoimpl.MessageState - sizeCache protoimpl.SizeCache - unknownFields protoimpl.UnknownFields - + state protoimpl.MessageState `protogen:"hybrid.v1"` // (Required) A reference to an active process bundle request with the given // instruction id. InstructionId string `protobuf:"bytes,1,opt,name=instruction_id,json=instructionId,proto3" json:"instruction_id,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache } func (x *ProcessBundleProgressRequest) Reset() { *x = ProcessBundleProgressRequest{} - if protoimpl.UnsafeEnabled { - mi := &file_org_apache_beam_model_fn_execution_v1_beam_fn_api_proto_msgTypes[17] - ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) - ms.StoreMessageInfo(mi) - } + mi := &file_org_apache_beam_model_fn_execution_v1_beam_fn_api_proto_msgTypes[17] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) } func (x *ProcessBundleProgressRequest) String() string { @@ -1602,7 +2665,7 @@ func (*ProcessBundleProgressRequest) ProtoMessage() {} func (x *ProcessBundleProgressRequest) ProtoReflect() protoreflect.Message { mi := &file_org_apache_beam_model_fn_execution_v1_beam_fn_api_proto_msgTypes[17] - if protoimpl.UnsafeEnabled && x != nil { + if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { ms.StoreMessageInfo(mi) @@ -1612,11 +2675,6 @@ func (x *ProcessBundleProgressRequest) ProtoReflect() protoreflect.Message { return mi.MessageOf(x) } -// Deprecated: Use ProcessBundleProgressRequest.ProtoReflect.Descriptor instead. -func (*ProcessBundleProgressRequest) Descriptor() ([]byte, []int) { - return file_org_apache_beam_model_fn_execution_v1_beam_fn_api_proto_rawDescGZIP(), []int{17} -} - func (x *ProcessBundleProgressRequest) GetInstructionId() string { if x != nil { return x.InstructionId @@ -1624,6 +2682,26 @@ func (x *ProcessBundleProgressRequest) GetInstructionId() string { return "" } +func (x *ProcessBundleProgressRequest) SetInstructionId(v string) { + x.InstructionId = v +} + +type ProcessBundleProgressRequest_builder struct { + _ [0]func() // Prevents comparability and use of unkeyed literals for the builder. + + // (Required) A reference to an active process bundle request with the given + // instruction id. + InstructionId string +} + +func (b0 ProcessBundleProgressRequest_builder) Build() *ProcessBundleProgressRequest { + m0 := &ProcessBundleProgressRequest{} + b, x := &b0, m0 + _, _ = b, x + x.InstructionId = b.InstructionId + return m0 +} + // A request to provide full MonitoringInfo for a set of provided ids. // // An SDK can report metrics using an identifier that only contains the @@ -1636,21 +2714,18 @@ func (x *ProcessBundleProgressRequest) GetInstructionId() string { // reconstructed fully by overwriting its payload field with the bytes specified // here. type MonitoringInfosMetadataRequest struct { - state protoimpl.MessageState - sizeCache protoimpl.SizeCache - unknownFields protoimpl.UnknownFields - + state protoimpl.MessageState `protogen:"hybrid.v1"` // A list of ids for which the full MonitoringInfo is requested for. MonitoringInfoId []string `protobuf:"bytes,1,rep,name=monitoring_info_id,json=monitoringInfoId,proto3" json:"monitoring_info_id,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache } func (x *MonitoringInfosMetadataRequest) Reset() { *x = MonitoringInfosMetadataRequest{} - if protoimpl.UnsafeEnabled { - mi := &file_org_apache_beam_model_fn_execution_v1_beam_fn_api_proto_msgTypes[18] - ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) - ms.StoreMessageInfo(mi) - } + mi := &file_org_apache_beam_model_fn_execution_v1_beam_fn_api_proto_msgTypes[18] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) } func (x *MonitoringInfosMetadataRequest) String() string { @@ -1661,7 +2736,7 @@ func (*MonitoringInfosMetadataRequest) ProtoMessage() {} func (x *MonitoringInfosMetadataRequest) ProtoReflect() protoreflect.Message { mi := &file_org_apache_beam_model_fn_execution_v1_beam_fn_api_proto_msgTypes[18] - if protoimpl.UnsafeEnabled && x != nil { + if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { ms.StoreMessageInfo(mi) @@ -1671,11 +2746,6 @@ func (x *MonitoringInfosMetadataRequest) ProtoReflect() protoreflect.Message { return mi.MessageOf(x) } -// Deprecated: Use MonitoringInfosMetadataRequest.ProtoReflect.Descriptor instead. -func (*MonitoringInfosMetadataRequest) Descriptor() ([]byte, []int) { - return file_org_apache_beam_model_fn_execution_v1_beam_fn_api_proto_rawDescGZIP(), []int{18} -} - func (x *MonitoringInfosMetadataRequest) GetMonitoringInfoId() []string { if x != nil { return x.MonitoringInfoId @@ -1683,11 +2753,27 @@ func (x *MonitoringInfosMetadataRequest) GetMonitoringInfoId() []string { return nil } -type ProcessBundleProgressResponse struct { - state protoimpl.MessageState - sizeCache protoimpl.SizeCache - unknownFields protoimpl.UnknownFields +func (x *MonitoringInfosMetadataRequest) SetMonitoringInfoId(v []string) { + x.MonitoringInfoId = v +} + +type MonitoringInfosMetadataRequest_builder struct { + _ [0]func() // Prevents comparability and use of unkeyed literals for the builder. + + // A list of ids for which the full MonitoringInfo is requested for. + MonitoringInfoId []string +} +func (b0 MonitoringInfosMetadataRequest_builder) Build() *MonitoringInfosMetadataRequest { + m0 := &MonitoringInfosMetadataRequest{} + b, x := &b0, m0 + _, _ = b, x + x.MonitoringInfoId = b.MonitoringInfoId + return m0 +} + +type ProcessBundleProgressResponse struct { + state protoimpl.MessageState `protogen:"hybrid.v1"` // DEPRECATED (Required) The list of metrics or other MonitoredState // collected while processing this bundle. MonitoringInfos []*pipeline_v1.MonitoringInfo `protobuf:"bytes,3,rep,name=monitoring_infos,json=monitoringInfos,proto3" json:"monitoring_infos,omitempty"` @@ -1702,21 +2788,21 @@ type ProcessBundleProgressResponse struct { // for the lifetime of the associated control connection as long // as the MonitoringInfo could be reconstructed fully by overwriting its // payload field with the bytes specified here. - MonitoringData map[string][]byte `protobuf:"bytes,5,rep,name=monitoring_data,json=monitoringData,proto3" json:"monitoring_data,omitempty" protobuf_key:"bytes,1,opt,name=key,proto3" protobuf_val:"bytes,2,opt,name=value,proto3"` + MonitoringData map[string][]byte `protobuf:"bytes,5,rep,name=monitoring_data,json=monitoringData,proto3" json:"monitoring_data,omitempty" protobuf_key:"bytes,1,opt,name=key" protobuf_val:"bytes,2,opt,name=value"` // Indicates that the SDK is still busy consuming the data that as already // been received on the data channel. If this is set, a runner may abstain // from sending further data on the data channel until this field becomes // unset. This field is currently used during shuffle reads on large elements. ConsumingReceivedData *bool `protobuf:"varint,6,opt,name=consuming_received_data,json=consumingReceivedData,proto3,oneof" json:"consuming_received_data,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache } func (x *ProcessBundleProgressResponse) Reset() { *x = ProcessBundleProgressResponse{} - if protoimpl.UnsafeEnabled { - mi := &file_org_apache_beam_model_fn_execution_v1_beam_fn_api_proto_msgTypes[19] - ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) - ms.StoreMessageInfo(mi) - } + mi := &file_org_apache_beam_model_fn_execution_v1_beam_fn_api_proto_msgTypes[19] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) } func (x *ProcessBundleProgressResponse) String() string { @@ -1727,7 +2813,7 @@ func (*ProcessBundleProgressResponse) ProtoMessage() {} func (x *ProcessBundleProgressResponse) ProtoReflect() protoreflect.Message { mi := &file_org_apache_beam_model_fn_execution_v1_beam_fn_api_proto_msgTypes[19] - if protoimpl.UnsafeEnabled && x != nil { + if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { ms.StoreMessageInfo(mi) @@ -1737,11 +2823,6 @@ func (x *ProcessBundleProgressResponse) ProtoReflect() protoreflect.Message { return mi.MessageOf(x) } -// Deprecated: Use ProcessBundleProgressResponse.ProtoReflect.Descriptor instead. -func (*ProcessBundleProgressResponse) Descriptor() ([]byte, []int) { - return file_org_apache_beam_model_fn_execution_v1_beam_fn_api_proto_rawDescGZIP(), []int{19} -} - func (x *ProcessBundleProgressResponse) GetMonitoringInfos() []*pipeline_v1.MonitoringInfo { if x != nil { return x.MonitoringInfos @@ -1763,6 +2844,64 @@ func (x *ProcessBundleProgressResponse) GetConsumingReceivedData() bool { return false } +func (x *ProcessBundleProgressResponse) SetMonitoringInfos(v []*pipeline_v1.MonitoringInfo) { + x.MonitoringInfos = v +} + +func (x *ProcessBundleProgressResponse) SetMonitoringData(v map[string][]byte) { + x.MonitoringData = v +} + +func (x *ProcessBundleProgressResponse) SetConsumingReceivedData(v bool) { + x.ConsumingReceivedData = &v +} + +func (x *ProcessBundleProgressResponse) HasConsumingReceivedData() bool { + if x == nil { + return false + } + return x.ConsumingReceivedData != nil +} + +func (x *ProcessBundleProgressResponse) ClearConsumingReceivedData() { + x.ConsumingReceivedData = nil +} + +type ProcessBundleProgressResponse_builder struct { + _ [0]func() // Prevents comparability and use of unkeyed literals for the builder. + + // DEPRECATED (Required) The list of metrics or other MonitoredState + // collected while processing this bundle. + MonitoringInfos []*pipeline_v1.MonitoringInfo + // An identifier to MonitoringInfo.payload mapping. + // + // An SDK can report metrics using an identifier that only contains the + // associated payload. A runner who wants to receive the full metrics + // information can request all the monitoring metadata via a + // MonitoringInfosMetadataRequest providing a list of ids as necessary. + // + // The SDK is allowed to reuse the identifiers + // for the lifetime of the associated control connection as long + // as the MonitoringInfo could be reconstructed fully by overwriting its + // payload field with the bytes specified here. + MonitoringData map[string][]byte + // Indicates that the SDK is still busy consuming the data that as already + // been received on the data channel. If this is set, a runner may abstain + // from sending further data on the data channel until this field becomes + // unset. This field is currently used during shuffle reads on large elements. + ConsumingReceivedData *bool +} + +func (b0 ProcessBundleProgressResponse_builder) Build() *ProcessBundleProgressResponse { + m0 := &ProcessBundleProgressResponse{} + b, x := &b0, m0 + _, _ = b, x + x.MonitoringInfos = b.MonitoringInfos + x.MonitoringData = b.MonitoringData + x.ConsumingReceivedData = b.ConsumingReceivedData + return m0 +} + // A response that contains the full mapping information associated with // a specified set of identifiers. // @@ -1776,21 +2915,18 @@ func (x *ProcessBundleProgressResponse) GetConsumingReceivedData() bool { // as the MonitoringInfo could be reconstructed fully by overwriting its // payload field with the bytes specified here. type MonitoringInfosMetadataResponse struct { - state protoimpl.MessageState - sizeCache protoimpl.SizeCache - unknownFields protoimpl.UnknownFields - + state protoimpl.MessageState `protogen:"hybrid.v1"` // A mapping from an identifier to the full metrics information. - MonitoringInfo map[string]*pipeline_v1.MonitoringInfo `protobuf:"bytes,1,rep,name=monitoring_info,json=monitoringInfo,proto3" json:"monitoring_info,omitempty" protobuf_key:"bytes,1,opt,name=key,proto3" protobuf_val:"bytes,2,opt,name=value,proto3"` + MonitoringInfo map[string]*pipeline_v1.MonitoringInfo `protobuf:"bytes,1,rep,name=monitoring_info,json=monitoringInfo,proto3" json:"monitoring_info,omitempty" protobuf_key:"bytes,1,opt,name=key" protobuf_val:"bytes,2,opt,name=value"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache } func (x *MonitoringInfosMetadataResponse) Reset() { *x = MonitoringInfosMetadataResponse{} - if protoimpl.UnsafeEnabled { - mi := &file_org_apache_beam_model_fn_execution_v1_beam_fn_api_proto_msgTypes[20] - ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) - ms.StoreMessageInfo(mi) - } + mi := &file_org_apache_beam_model_fn_execution_v1_beam_fn_api_proto_msgTypes[20] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) } func (x *MonitoringInfosMetadataResponse) String() string { @@ -1801,7 +2937,7 @@ func (*MonitoringInfosMetadataResponse) ProtoMessage() {} func (x *MonitoringInfosMetadataResponse) ProtoReflect() protoreflect.Message { mi := &file_org_apache_beam_model_fn_execution_v1_beam_fn_api_proto_msgTypes[20] - if protoimpl.UnsafeEnabled && x != nil { + if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { ms.StoreMessageInfo(mi) @@ -1811,11 +2947,6 @@ func (x *MonitoringInfosMetadataResponse) ProtoReflect() protoreflect.Message { return mi.MessageOf(x) } -// Deprecated: Use MonitoringInfosMetadataResponse.ProtoReflect.Descriptor instead. -func (*MonitoringInfosMetadataResponse) Descriptor() ([]byte, []int) { - return file_org_apache_beam_model_fn_execution_v1_beam_fn_api_proto_rawDescGZIP(), []int{20} -} - func (x *MonitoringInfosMetadataResponse) GetMonitoringInfo() map[string]*pipeline_v1.MonitoringInfo { if x != nil { return x.MonitoringInfo @@ -1823,12 +2954,28 @@ func (x *MonitoringInfosMetadataResponse) GetMonitoringInfo() map[string]*pipeli return nil } +func (x *MonitoringInfosMetadataResponse) SetMonitoringInfo(v map[string]*pipeline_v1.MonitoringInfo) { + x.MonitoringInfo = v +} + +type MonitoringInfosMetadataResponse_builder struct { + _ [0]func() // Prevents comparability and use of unkeyed literals for the builder. + + // A mapping from an identifier to the full metrics information. + MonitoringInfo map[string]*pipeline_v1.MonitoringInfo +} + +func (b0 MonitoringInfosMetadataResponse_builder) Build() *MonitoringInfosMetadataResponse { + m0 := &MonitoringInfosMetadataResponse{} + b, x := &b0, m0 + _, _ = b, x + x.MonitoringInfo = b.MonitoringInfo + return m0 +} + // Represents a request to the SDK to split a currently active bundle. type ProcessBundleSplitRequest struct { - state protoimpl.MessageState - sizeCache protoimpl.SizeCache - unknownFields protoimpl.UnknownFields - + state protoimpl.MessageState `protogen:"hybrid.v1"` // (Required) A reference to an active process bundle request with the given // instruction id. InstructionId string `protobuf:"bytes,1,opt,name=instruction_id,json=instructionId,proto3" json:"instruction_id,omitempty"` @@ -1837,16 +2984,16 @@ type ProcessBundleSplitRequest struct { // Currently only splits at gRPC read operations are supported. // This may, of course, limit the amount of work downstream operations // receive. - DesiredSplits map[string]*ProcessBundleSplitRequest_DesiredSplit `protobuf:"bytes,3,rep,name=desired_splits,json=desiredSplits,proto3" json:"desired_splits,omitempty" protobuf_key:"bytes,1,opt,name=key,proto3" protobuf_val:"bytes,2,opt,name=value,proto3"` + DesiredSplits map[string]*ProcessBundleSplitRequest_DesiredSplit `protobuf:"bytes,3,rep,name=desired_splits,json=desiredSplits,proto3" json:"desired_splits,omitempty" protobuf_key:"bytes,1,opt,name=key" protobuf_val:"bytes,2,opt,name=value"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache } func (x *ProcessBundleSplitRequest) Reset() { *x = ProcessBundleSplitRequest{} - if protoimpl.UnsafeEnabled { - mi := &file_org_apache_beam_model_fn_execution_v1_beam_fn_api_proto_msgTypes[21] - ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) - ms.StoreMessageInfo(mi) - } + mi := &file_org_apache_beam_model_fn_execution_v1_beam_fn_api_proto_msgTypes[21] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) } func (x *ProcessBundleSplitRequest) String() string { @@ -1857,7 +3004,7 @@ func (*ProcessBundleSplitRequest) ProtoMessage() {} func (x *ProcessBundleSplitRequest) ProtoReflect() protoreflect.Message { mi := &file_org_apache_beam_model_fn_execution_v1_beam_fn_api_proto_msgTypes[21] - if protoimpl.UnsafeEnabled && x != nil { + if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { ms.StoreMessageInfo(mi) @@ -1867,11 +3014,6 @@ func (x *ProcessBundleSplitRequest) ProtoReflect() protoreflect.Message { return mi.MessageOf(x) } -// Deprecated: Use ProcessBundleSplitRequest.ProtoReflect.Descriptor instead. -func (*ProcessBundleSplitRequest) Descriptor() ([]byte, []int) { - return file_org_apache_beam_model_fn_execution_v1_beam_fn_api_proto_rawDescGZIP(), []int{21} -} - func (x *ProcessBundleSplitRequest) GetInstructionId() string { if x != nil { return x.InstructionId @@ -1886,6 +3028,37 @@ func (x *ProcessBundleSplitRequest) GetDesiredSplits() map[string]*ProcessBundle return nil } +func (x *ProcessBundleSplitRequest) SetInstructionId(v string) { + x.InstructionId = v +} + +func (x *ProcessBundleSplitRequest) SetDesiredSplits(v map[string]*ProcessBundleSplitRequest_DesiredSplit) { + x.DesiredSplits = v +} + +type ProcessBundleSplitRequest_builder struct { + _ [0]func() // Prevents comparability and use of unkeyed literals for the builder. + + // (Required) A reference to an active process bundle request with the given + // instruction id. + InstructionId string + // (Required) Specifies the desired split for each transform. + // + // Currently only splits at gRPC read operations are supported. + // This may, of course, limit the amount of work downstream operations + // receive. + DesiredSplits map[string]*ProcessBundleSplitRequest_DesiredSplit +} + +func (b0 ProcessBundleSplitRequest_builder) Build() *ProcessBundleSplitRequest { + m0 := &ProcessBundleSplitRequest{} + b, x := &b0, m0 + _, _ = b, x + x.InstructionId = b.InstructionId + x.DesiredSplits = b.DesiredSplits + return m0 +} + // Represents a partition of the bundle: a "primary" and a "residual", with the // following properties: // - The work in primary and residual doesn't overlap, and combined, adds up @@ -1949,10 +3122,7 @@ func (x *ProcessBundleSplitRequest) GetDesiredSplits() map[string]*ProcessBundle // // For more rigorous definitions see https://s.apache.org/beam-breaking-fusion type ProcessBundleSplitResponse struct { - state protoimpl.MessageState - sizeCache protoimpl.SizeCache - unknownFields protoimpl.UnknownFields - + state protoimpl.MessageState `protogen:"hybrid.v1"` // (Optional) Root applications that should replace the current bundle. // // Note that primary roots can only be specified if a channel split's @@ -1980,15 +3150,15 @@ type ProcessBundleSplitResponse struct { // applications roots above of the current split or any prior split of the // same bundle. ChannelSplits []*ProcessBundleSplitResponse_ChannelSplit `protobuf:"bytes,3,rep,name=channel_splits,json=channelSplits,proto3" json:"channel_splits,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache } func (x *ProcessBundleSplitResponse) Reset() { *x = ProcessBundleSplitResponse{} - if protoimpl.UnsafeEnabled { - mi := &file_org_apache_beam_model_fn_execution_v1_beam_fn_api_proto_msgTypes[22] - ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) - ms.StoreMessageInfo(mi) - } + mi := &file_org_apache_beam_model_fn_execution_v1_beam_fn_api_proto_msgTypes[22] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) } func (x *ProcessBundleSplitResponse) String() string { @@ -1999,7 +3169,7 @@ func (*ProcessBundleSplitResponse) ProtoMessage() {} func (x *ProcessBundleSplitResponse) ProtoReflect() protoreflect.Message { mi := &file_org_apache_beam_model_fn_execution_v1_beam_fn_api_proto_msgTypes[22] - if protoimpl.UnsafeEnabled && x != nil { + if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { ms.StoreMessageInfo(mi) @@ -2009,11 +3179,6 @@ func (x *ProcessBundleSplitResponse) ProtoReflect() protoreflect.Message { return mi.MessageOf(x) } -// Deprecated: Use ProcessBundleSplitResponse.ProtoReflect.Descriptor instead. -func (*ProcessBundleSplitResponse) Descriptor() ([]byte, []int) { - return file_org_apache_beam_model_fn_execution_v1_beam_fn_api_proto_rawDescGZIP(), []int{22} -} - func (x *ProcessBundleSplitResponse) GetPrimaryRoots() []*BundleApplication { if x != nil { return x.PrimaryRoots @@ -2035,23 +3200,74 @@ func (x *ProcessBundleSplitResponse) GetChannelSplits() []*ProcessBundleSplitRes return nil } -type FinalizeBundleRequest struct { - state protoimpl.MessageState - sizeCache protoimpl.SizeCache - unknownFields protoimpl.UnknownFields +func (x *ProcessBundleSplitResponse) SetPrimaryRoots(v []*BundleApplication) { + x.PrimaryRoots = v +} + +func (x *ProcessBundleSplitResponse) SetResidualRoots(v []*DelayedBundleApplication) { + x.ResidualRoots = v +} + +func (x *ProcessBundleSplitResponse) SetChannelSplits(v []*ProcessBundleSplitResponse_ChannelSplit) { + x.ChannelSplits = v +} + +type ProcessBundleSplitResponse_builder struct { + _ [0]func() // Prevents comparability and use of unkeyed literals for the builder. + + // (Optional) Root applications that should replace the current bundle. + // + // Note that primary roots can only be specified if a channel split's + // last_primary_element + 1 < first_residual_element + // + // Note that there must be a corresponding residual root contained within + // residual_roots representing the remainder of processing for the original + // element this this primary root represents a fraction of. + PrimaryRoots []*BundleApplication + // (Optional) Root applications that have been removed from the current bundle and + // have to be executed in a separate bundle (e.g. in parallel on a different + // worker, or after the current bundle completes, etc.) + // + // Note that residual roots can only be specified if a channel split's + // last_primary_element + 1 < first_residual_element + // + // Note that there must be a corresponding primary root contained within + // primary_roots representing the remainder of processing for the original + // element this this residual root represents a fraction of. + // + // Note that subsequent splits must not return prior residual roots. + ResidualRoots []*DelayedBundleApplication + // (Required) Partitions of input data channels into primary and residual + // elements, if any. Must not include any elements represented in the bundle + // applications roots above of the current split or any prior split of the + // same bundle. + ChannelSplits []*ProcessBundleSplitResponse_ChannelSplit +} +func (b0 ProcessBundleSplitResponse_builder) Build() *ProcessBundleSplitResponse { + m0 := &ProcessBundleSplitResponse{} + b, x := &b0, m0 + _, _ = b, x + x.PrimaryRoots = b.PrimaryRoots + x.ResidualRoots = b.ResidualRoots + x.ChannelSplits = b.ChannelSplits + return m0 +} + +type FinalizeBundleRequest struct { + state protoimpl.MessageState `protogen:"hybrid.v1"` // (Required) A reference to a completed process bundle request with the given // instruction id. InstructionId string `protobuf:"bytes,1,opt,name=instruction_id,json=instructionId,proto3" json:"instruction_id,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache } func (x *FinalizeBundleRequest) Reset() { *x = FinalizeBundleRequest{} - if protoimpl.UnsafeEnabled { - mi := &file_org_apache_beam_model_fn_execution_v1_beam_fn_api_proto_msgTypes[23] - ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) - ms.StoreMessageInfo(mi) - } + mi := &file_org_apache_beam_model_fn_execution_v1_beam_fn_api_proto_msgTypes[23] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) } func (x *FinalizeBundleRequest) String() string { @@ -2062,7 +3278,7 @@ func (*FinalizeBundleRequest) ProtoMessage() {} func (x *FinalizeBundleRequest) ProtoReflect() protoreflect.Message { mi := &file_org_apache_beam_model_fn_execution_v1_beam_fn_api_proto_msgTypes[23] - if protoimpl.UnsafeEnabled && x != nil { + if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { ms.StoreMessageInfo(mi) @@ -2072,11 +3288,6 @@ func (x *FinalizeBundleRequest) ProtoReflect() protoreflect.Message { return mi.MessageOf(x) } -// Deprecated: Use FinalizeBundleRequest.ProtoReflect.Descriptor instead. -func (*FinalizeBundleRequest) Descriptor() ([]byte, []int) { - return file_org_apache_beam_model_fn_execution_v1_beam_fn_api_proto_rawDescGZIP(), []int{23} -} - func (x *FinalizeBundleRequest) GetInstructionId() string { if x != nil { return x.InstructionId @@ -2084,19 +3295,37 @@ func (x *FinalizeBundleRequest) GetInstructionId() string { return "" } +func (x *FinalizeBundleRequest) SetInstructionId(v string) { + x.InstructionId = v +} + +type FinalizeBundleRequest_builder struct { + _ [0]func() // Prevents comparability and use of unkeyed literals for the builder. + + // (Required) A reference to a completed process bundle request with the given + // instruction id. + InstructionId string +} + +func (b0 FinalizeBundleRequest_builder) Build() *FinalizeBundleRequest { + m0 := &FinalizeBundleRequest{} + b, x := &b0, m0 + _, _ = b, x + x.InstructionId = b.InstructionId + return m0 +} + type FinalizeBundleResponse struct { - state protoimpl.MessageState - sizeCache protoimpl.SizeCache + state protoimpl.MessageState `protogen:"hybrid.v1"` unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache } func (x *FinalizeBundleResponse) Reset() { *x = FinalizeBundleResponse{} - if protoimpl.UnsafeEnabled { - mi := &file_org_apache_beam_model_fn_execution_v1_beam_fn_api_proto_msgTypes[24] - ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) - ms.StoreMessageInfo(mi) - } + mi := &file_org_apache_beam_model_fn_execution_v1_beam_fn_api_proto_msgTypes[24] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) } func (x *FinalizeBundleResponse) String() string { @@ -2107,7 +3336,7 @@ func (*FinalizeBundleResponse) ProtoMessage() {} func (x *FinalizeBundleResponse) ProtoReflect() protoreflect.Message { mi := &file_org_apache_beam_model_fn_execution_v1_beam_fn_api_proto_msgTypes[24] - if protoimpl.UnsafeEnabled && x != nil { + if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { ms.StoreMessageInfo(mi) @@ -2117,31 +3346,35 @@ func (x *FinalizeBundleResponse) ProtoReflect() protoreflect.Message { return mi.MessageOf(x) } -// Deprecated: Use FinalizeBundleResponse.ProtoReflect.Descriptor instead. -func (*FinalizeBundleResponse) Descriptor() ([]byte, []int) { - return file_org_apache_beam_model_fn_execution_v1_beam_fn_api_proto_rawDescGZIP(), []int{24} +type FinalizeBundleResponse_builder struct { + _ [0]func() // Prevents comparability and use of unkeyed literals for the builder. + +} + +func (b0 FinalizeBundleResponse_builder) Build() *FinalizeBundleResponse { + m0 := &FinalizeBundleResponse{} + b, x := &b0, m0 + _, _ = b, x + return m0 } // Messages used to represent logical byte streams. // Stable type Elements struct { - state protoimpl.MessageState - sizeCache protoimpl.SizeCache - unknownFields protoimpl.UnknownFields - + state protoimpl.MessageState `protogen:"hybrid.v1"` // (Optional) A list containing parts of logical byte streams. Data []*Elements_Data `protobuf:"bytes,1,rep,name=data,proto3" json:"data,omitempty"` // (Optional) A list of timer byte streams. - Timers []*Elements_Timers `protobuf:"bytes,2,rep,name=timers,proto3" json:"timers,omitempty"` + Timers []*Elements_Timers `protobuf:"bytes,2,rep,name=timers,proto3" json:"timers,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache } func (x *Elements) Reset() { *x = Elements{} - if protoimpl.UnsafeEnabled { - mi := &file_org_apache_beam_model_fn_execution_v1_beam_fn_api_proto_msgTypes[25] - ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) - ms.StoreMessageInfo(mi) - } + mi := &file_org_apache_beam_model_fn_execution_v1_beam_fn_api_proto_msgTypes[25] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) } func (x *Elements) String() string { @@ -2152,7 +3385,7 @@ func (*Elements) ProtoMessage() {} func (x *Elements) ProtoReflect() protoreflect.Message { mi := &file_org_apache_beam_model_fn_execution_v1_beam_fn_api_proto_msgTypes[25] - if protoimpl.UnsafeEnabled && x != nil { + if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { ms.StoreMessageInfo(mi) @@ -2162,11 +3395,6 @@ func (x *Elements) ProtoReflect() protoreflect.Message { return mi.MessageOf(x) } -// Deprecated: Use Elements.ProtoReflect.Descriptor instead. -func (*Elements) Descriptor() ([]byte, []int) { - return file_org_apache_beam_model_fn_execution_v1_beam_fn_api_proto_rawDescGZIP(), []int{25} -} - func (x *Elements) GetData() []*Elements_Data { if x != nil { return x.Data @@ -2181,11 +3409,34 @@ func (x *Elements) GetTimers() []*Elements_Timers { return nil } -type StateRequest struct { - state protoimpl.MessageState - sizeCache protoimpl.SizeCache - unknownFields protoimpl.UnknownFields +func (x *Elements) SetData(v []*Elements_Data) { + x.Data = v +} +func (x *Elements) SetTimers(v []*Elements_Timers) { + x.Timers = v +} + +type Elements_builder struct { + _ [0]func() // Prevents comparability and use of unkeyed literals for the builder. + + // (Optional) A list containing parts of logical byte streams. + Data []*Elements_Data + // (Optional) A list of timer byte streams. + Timers []*Elements_Timers +} + +func (b0 Elements_builder) Build() *Elements { + m0 := &Elements{} + b, x := &b0, m0 + _, _ = b, x + x.Data = b.Data + x.Timers = b.Timers + return m0 +} + +type StateRequest struct { + state protoimpl.MessageState `protogen:"hybrid.v1"` // (Required) A unique identifier provided by the SDK which represents this // requests execution. The StateResponse corresponding with this request // will have the matching id. @@ -2198,21 +3449,21 @@ type StateRequest struct { StateKey *StateKey `protobuf:"bytes,3,opt,name=state_key,json=stateKey,proto3" json:"state_key,omitempty"` // (Required) The action to take on this request. // - // Types that are assignable to Request: + // Types that are valid to be assigned to Request: // // *StateRequest_Get // *StateRequest_Append // *StateRequest_Clear - Request isStateRequest_Request `protobuf_oneof:"request"` + Request isStateRequest_Request `protobuf_oneof:"request"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache } func (x *StateRequest) Reset() { *x = StateRequest{} - if protoimpl.UnsafeEnabled { - mi := &file_org_apache_beam_model_fn_execution_v1_beam_fn_api_proto_msgTypes[26] - ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) - ms.StoreMessageInfo(mi) - } + mi := &file_org_apache_beam_model_fn_execution_v1_beam_fn_api_proto_msgTypes[26] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) } func (x *StateRequest) String() string { @@ -2223,7 +3474,7 @@ func (*StateRequest) ProtoMessage() {} func (x *StateRequest) ProtoReflect() protoreflect.Message { mi := &file_org_apache_beam_model_fn_execution_v1_beam_fn_api_proto_msgTypes[26] - if protoimpl.UnsafeEnabled && x != nil { + if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { ms.StoreMessageInfo(mi) @@ -2233,11 +3484,6 @@ func (x *StateRequest) ProtoReflect() protoreflect.Message { return mi.MessageOf(x) } -// Deprecated: Use StateRequest.ProtoReflect.Descriptor instead. -func (*StateRequest) Descriptor() ([]byte, []int) { - return file_org_apache_beam_model_fn_execution_v1_beam_fn_api_proto_rawDescGZIP(), []int{26} -} - func (x *StateRequest) GetId() string { if x != nil { return x.Id @@ -2259,89 +3505,267 @@ func (x *StateRequest) GetStateKey() *StateKey { return nil } -func (m *StateRequest) GetRequest() isStateRequest_Request { - if m != nil { - return m.Request +func (x *StateRequest) GetRequest() isStateRequest_Request { + if x != nil { + return x.Request } return nil } func (x *StateRequest) GetGet() *StateGetRequest { - if x, ok := x.GetRequest().(*StateRequest_Get); ok { - return x.Get + if x != nil { + if x, ok := x.Request.(*StateRequest_Get); ok { + return x.Get + } } return nil } func (x *StateRequest) GetAppend() *StateAppendRequest { - if x, ok := x.GetRequest().(*StateRequest_Append); ok { - return x.Append + if x != nil { + if x, ok := x.Request.(*StateRequest_Append); ok { + return x.Append + } } return nil } func (x *StateRequest) GetClear() *StateClearRequest { - if x, ok := x.GetRequest().(*StateRequest_Clear); ok { - return x.Clear + if x != nil { + if x, ok := x.Request.(*StateRequest_Clear); ok { + return x.Clear + } } return nil } -type isStateRequest_Request interface { - isStateRequest_Request() +func (x *StateRequest) SetId(v string) { + x.Id = v } -type StateRequest_Get struct { - // A request to get state. - Get *StateGetRequest `protobuf:"bytes,1000,opt,name=get,proto3,oneof"` +func (x *StateRequest) SetInstructionId(v string) { + x.InstructionId = v } -type StateRequest_Append struct { - // A request to append to state. - Append *StateAppendRequest `protobuf:"bytes,1001,opt,name=append,proto3,oneof"` +func (x *StateRequest) SetStateKey(v *StateKey) { + x.StateKey = v } -type StateRequest_Clear struct { - // A request to clear state. - Clear *StateClearRequest `protobuf:"bytes,1002,opt,name=clear,proto3,oneof"` +func (x *StateRequest) SetGet(v *StateGetRequest) { + if v == nil { + x.Request = nil + return + } + x.Request = &StateRequest_Get{v} } -func (*StateRequest_Get) isStateRequest_Request() {} +func (x *StateRequest) SetAppend(v *StateAppendRequest) { + if v == nil { + x.Request = nil + return + } + x.Request = &StateRequest_Append{v} +} -func (*StateRequest_Append) isStateRequest_Request() {} +func (x *StateRequest) SetClear(v *StateClearRequest) { + if v == nil { + x.Request = nil + return + } + x.Request = &StateRequest_Clear{v} +} -func (*StateRequest_Clear) isStateRequest_Request() {} +func (x *StateRequest) HasStateKey() bool { + if x == nil { + return false + } + return x.StateKey != nil +} -type StateResponse struct { - state protoimpl.MessageState - sizeCache protoimpl.SizeCache - unknownFields protoimpl.UnknownFields +func (x *StateRequest) HasRequest() bool { + if x == nil { + return false + } + return x.Request != nil +} - // (Required) A reference provided by the SDK which represents a requests - // execution. The StateResponse must have the matching id when responding - // to the SDK. - Id string `protobuf:"bytes,1,opt,name=id,proto3" json:"id,omitempty"` - // (Optional) If this is specified, then the state request has failed. - // A human readable string representing the reason as to why the request - // failed. - Error string `protobuf:"bytes,2,opt,name=error,proto3" json:"error,omitempty"` - // A corresponding response matching the request will be populated. - // - // Types that are assignable to Response: +func (x *StateRequest) HasGet() bool { + if x == nil { + return false + } + _, ok := x.Request.(*StateRequest_Get) + return ok +} + +func (x *StateRequest) HasAppend() bool { + if x == nil { + return false + } + _, ok := x.Request.(*StateRequest_Append) + return ok +} + +func (x *StateRequest) HasClear() bool { + if x == nil { + return false + } + _, ok := x.Request.(*StateRequest_Clear) + return ok +} + +func (x *StateRequest) ClearStateKey() { + x.StateKey = nil +} + +func (x *StateRequest) ClearRequest() { + x.Request = nil +} + +func (x *StateRequest) ClearGet() { + if _, ok := x.Request.(*StateRequest_Get); ok { + x.Request = nil + } +} + +func (x *StateRequest) ClearAppend() { + if _, ok := x.Request.(*StateRequest_Append); ok { + x.Request = nil + } +} + +func (x *StateRequest) ClearClear() { + if _, ok := x.Request.(*StateRequest_Clear); ok { + x.Request = nil + } +} + +const StateRequest_Request_not_set_case case_StateRequest_Request = 0 +const StateRequest_Get_case case_StateRequest_Request = 1000 +const StateRequest_Append_case case_StateRequest_Request = 1001 +const StateRequest_Clear_case case_StateRequest_Request = 1002 + +func (x *StateRequest) WhichRequest() case_StateRequest_Request { + if x == nil { + return StateRequest_Request_not_set_case + } + switch x.Request.(type) { + case *StateRequest_Get: + return StateRequest_Get_case + case *StateRequest_Append: + return StateRequest_Append_case + case *StateRequest_Clear: + return StateRequest_Clear_case + default: + return StateRequest_Request_not_set_case + } +} + +type StateRequest_builder struct { + _ [0]func() // Prevents comparability and use of unkeyed literals for the builder. + + // (Required) A unique identifier provided by the SDK which represents this + // requests execution. The StateResponse corresponding with this request + // will have the matching id. + Id string + // (Required) The associated instruction id of the work that is currently + // being processed. This allows for the runner to associate any modifications + // to state to be committed with the appropriate work execution. + InstructionId string + // (Required) The state key this request is for. + StateKey *StateKey + // (Required) The action to take on this request. + + // Fields of oneof Request: + // A request to get state. + Get *StateGetRequest + // A request to append to state. + Append *StateAppendRequest + // A request to clear state. + Clear *StateClearRequest + // -- end of Request +} + +func (b0 StateRequest_builder) Build() *StateRequest { + m0 := &StateRequest{} + b, x := &b0, m0 + _, _ = b, x + x.Id = b.Id + x.InstructionId = b.InstructionId + x.StateKey = b.StateKey + if b.Get != nil { + x.Request = &StateRequest_Get{b.Get} + } + if b.Append != nil { + x.Request = &StateRequest_Append{b.Append} + } + if b.Clear != nil { + x.Request = &StateRequest_Clear{b.Clear} + } + return m0 +} + +type case_StateRequest_Request protoreflect.FieldNumber + +func (x case_StateRequest_Request) String() string { + md := file_org_apache_beam_model_fn_execution_v1_beam_fn_api_proto_msgTypes[26].Descriptor() + if x == 0 { + return "not set" + } + return protoimpl.X.MessageFieldStringOf(md, protoreflect.FieldNumber(x)) +} + +type isStateRequest_Request interface { + isStateRequest_Request() +} + +type StateRequest_Get struct { + // A request to get state. + Get *StateGetRequest `protobuf:"bytes,1000,opt,name=get,proto3,oneof"` +} + +type StateRequest_Append struct { + // A request to append to state. + Append *StateAppendRequest `protobuf:"bytes,1001,opt,name=append,proto3,oneof"` +} + +type StateRequest_Clear struct { + // A request to clear state. + Clear *StateClearRequest `protobuf:"bytes,1002,opt,name=clear,proto3,oneof"` +} + +func (*StateRequest_Get) isStateRequest_Request() {} + +func (*StateRequest_Append) isStateRequest_Request() {} + +func (*StateRequest_Clear) isStateRequest_Request() {} + +type StateResponse struct { + state protoimpl.MessageState `protogen:"hybrid.v1"` + // (Required) A reference provided by the SDK which represents a requests + // execution. The StateResponse must have the matching id when responding + // to the SDK. + Id string `protobuf:"bytes,1,opt,name=id,proto3" json:"id,omitempty"` + // (Optional) If this is specified, then the state request has failed. + // A human readable string representing the reason as to why the request + // failed. + Error string `protobuf:"bytes,2,opt,name=error,proto3" json:"error,omitempty"` + // A corresponding response matching the request will be populated. + // + // Types that are valid to be assigned to Response: // // *StateResponse_Get // *StateResponse_Append // *StateResponse_Clear - Response isStateResponse_Response `protobuf_oneof:"response"` + Response isStateResponse_Response `protobuf_oneof:"response"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache } func (x *StateResponse) Reset() { *x = StateResponse{} - if protoimpl.UnsafeEnabled { - mi := &file_org_apache_beam_model_fn_execution_v1_beam_fn_api_proto_msgTypes[27] - ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) - ms.StoreMessageInfo(mi) - } + mi := &file_org_apache_beam_model_fn_execution_v1_beam_fn_api_proto_msgTypes[27] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) } func (x *StateResponse) String() string { @@ -2352,7 +3776,7 @@ func (*StateResponse) ProtoMessage() {} func (x *StateResponse) ProtoReflect() protoreflect.Message { mi := &file_org_apache_beam_model_fn_execution_v1_beam_fn_api_proto_msgTypes[27] - if protoimpl.UnsafeEnabled && x != nil { + if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { ms.StoreMessageInfo(mi) @@ -2362,11 +3786,6 @@ func (x *StateResponse) ProtoReflect() protoreflect.Message { return mi.MessageOf(x) } -// Deprecated: Use StateResponse.ProtoReflect.Descriptor instead. -func (*StateResponse) Descriptor() ([]byte, []int) { - return file_org_apache_beam_model_fn_execution_v1_beam_fn_api_proto_rawDescGZIP(), []int{27} -} - func (x *StateResponse) GetId() string { if x != nil { return x.Id @@ -2381,34 +3800,197 @@ func (x *StateResponse) GetError() string { return "" } -func (m *StateResponse) GetResponse() isStateResponse_Response { - if m != nil { - return m.Response +func (x *StateResponse) GetResponse() isStateResponse_Response { + if x != nil { + return x.Response } return nil } func (x *StateResponse) GetGet() *StateGetResponse { - if x, ok := x.GetResponse().(*StateResponse_Get); ok { - return x.Get + if x != nil { + if x, ok := x.Response.(*StateResponse_Get); ok { + return x.Get + } } return nil } func (x *StateResponse) GetAppend() *StateAppendResponse { - if x, ok := x.GetResponse().(*StateResponse_Append); ok { - return x.Append + if x != nil { + if x, ok := x.Response.(*StateResponse_Append); ok { + return x.Append + } } return nil } func (x *StateResponse) GetClear() *StateClearResponse { - if x, ok := x.GetResponse().(*StateResponse_Clear); ok { - return x.Clear + if x != nil { + if x, ok := x.Response.(*StateResponse_Clear); ok { + return x.Clear + } } return nil } +func (x *StateResponse) SetId(v string) { + x.Id = v +} + +func (x *StateResponse) SetError(v string) { + x.Error = v +} + +func (x *StateResponse) SetGet(v *StateGetResponse) { + if v == nil { + x.Response = nil + return + } + x.Response = &StateResponse_Get{v} +} + +func (x *StateResponse) SetAppend(v *StateAppendResponse) { + if v == nil { + x.Response = nil + return + } + x.Response = &StateResponse_Append{v} +} + +func (x *StateResponse) SetClear(v *StateClearResponse) { + if v == nil { + x.Response = nil + return + } + x.Response = &StateResponse_Clear{v} +} + +func (x *StateResponse) HasResponse() bool { + if x == nil { + return false + } + return x.Response != nil +} + +func (x *StateResponse) HasGet() bool { + if x == nil { + return false + } + _, ok := x.Response.(*StateResponse_Get) + return ok +} + +func (x *StateResponse) HasAppend() bool { + if x == nil { + return false + } + _, ok := x.Response.(*StateResponse_Append) + return ok +} + +func (x *StateResponse) HasClear() bool { + if x == nil { + return false + } + _, ok := x.Response.(*StateResponse_Clear) + return ok +} + +func (x *StateResponse) ClearResponse() { + x.Response = nil +} + +func (x *StateResponse) ClearGet() { + if _, ok := x.Response.(*StateResponse_Get); ok { + x.Response = nil + } +} + +func (x *StateResponse) ClearAppend() { + if _, ok := x.Response.(*StateResponse_Append); ok { + x.Response = nil + } +} + +func (x *StateResponse) ClearClear() { + if _, ok := x.Response.(*StateResponse_Clear); ok { + x.Response = nil + } +} + +const StateResponse_Response_not_set_case case_StateResponse_Response = 0 +const StateResponse_Get_case case_StateResponse_Response = 1000 +const StateResponse_Append_case case_StateResponse_Response = 1001 +const StateResponse_Clear_case case_StateResponse_Response = 1002 + +func (x *StateResponse) WhichResponse() case_StateResponse_Response { + if x == nil { + return StateResponse_Response_not_set_case + } + switch x.Response.(type) { + case *StateResponse_Get: + return StateResponse_Get_case + case *StateResponse_Append: + return StateResponse_Append_case + case *StateResponse_Clear: + return StateResponse_Clear_case + default: + return StateResponse_Response_not_set_case + } +} + +type StateResponse_builder struct { + _ [0]func() // Prevents comparability and use of unkeyed literals for the builder. + + // (Required) A reference provided by the SDK which represents a requests + // execution. The StateResponse must have the matching id when responding + // to the SDK. + Id string + // (Optional) If this is specified, then the state request has failed. + // A human readable string representing the reason as to why the request + // failed. + Error string + // A corresponding response matching the request will be populated. + + // Fields of oneof Response: + // A response to getting state. + Get *StateGetResponse + // A response to appending to state. + Append *StateAppendResponse + // A response to clearing state. + Clear *StateClearResponse + // -- end of Response +} + +func (b0 StateResponse_builder) Build() *StateResponse { + m0 := &StateResponse{} + b, x := &b0, m0 + _, _ = b, x + x.Id = b.Id + x.Error = b.Error + if b.Get != nil { + x.Response = &StateResponse_Get{b.Get} + } + if b.Append != nil { + x.Response = &StateResponse_Append{b.Append} + } + if b.Clear != nil { + x.Response = &StateResponse_Clear{b.Clear} + } + return m0 +} + +type case_StateResponse_Response protoreflect.FieldNumber + +func (x case_StateResponse_Response) String() string { + md := file_org_apache_beam_model_fn_execution_v1_beam_fn_api_proto_msgTypes[27].Descriptor() + if x == 0 { + return "not set" + } + return protoimpl.X.MessageFieldStringOf(md, protoreflect.FieldNumber(x)) +} + type isStateResponse_Response interface { isStateResponse_Response() } @@ -2435,13 +4017,10 @@ func (*StateResponse_Append) isStateResponse_Response() {} func (*StateResponse_Clear) isStateResponse_Response() {} type StateKey struct { - state protoimpl.MessageState - sizeCache protoimpl.SizeCache - unknownFields protoimpl.UnknownFields - + state protoimpl.MessageState `protogen:"hybrid.v1"` // (Required) One of the following state keys must be set. // - // Types that are assignable to Type: + // Types that are valid to be assigned to Type: // // *StateKey_Runner_ // *StateKey_MultimapSideInput_ @@ -2452,16 +4031,16 @@ type StateKey struct { // *StateKey_MultimapKeysUserState_ // *StateKey_MultimapUserState_ // *StateKey_OrderedListUserState_ - Type isStateKey_Type `protobuf_oneof:"type"` + Type isStateKey_Type `protobuf_oneof:"type"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache } func (x *StateKey) Reset() { *x = StateKey{} - if protoimpl.UnsafeEnabled { - mi := &file_org_apache_beam_model_fn_execution_v1_beam_fn_api_proto_msgTypes[28] - ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) - ms.StoreMessageInfo(mi) - } + mi := &file_org_apache_beam_model_fn_execution_v1_beam_fn_api_proto_msgTypes[28] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) } func (x *StateKey) String() string { @@ -2472,7 +4051,7 @@ func (*StateKey) ProtoMessage() {} func (x *StateKey) ProtoReflect() protoreflect.Message { mi := &file_org_apache_beam_model_fn_execution_v1_beam_fn_api_proto_msgTypes[28] - if protoimpl.UnsafeEnabled && x != nil { + if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { ms.StoreMessageInfo(mi) @@ -2482,79 +4061,402 @@ func (x *StateKey) ProtoReflect() protoreflect.Message { return mi.MessageOf(x) } -// Deprecated: Use StateKey.ProtoReflect.Descriptor instead. -func (*StateKey) Descriptor() ([]byte, []int) { - return file_org_apache_beam_model_fn_execution_v1_beam_fn_api_proto_rawDescGZIP(), []int{28} -} - -func (m *StateKey) GetType() isStateKey_Type { - if m != nil { - return m.Type +func (x *StateKey) GetType() isStateKey_Type { + if x != nil { + return x.Type } return nil } func (x *StateKey) GetRunner() *StateKey_Runner { - if x, ok := x.GetType().(*StateKey_Runner_); ok { - return x.Runner + if x != nil { + if x, ok := x.Type.(*StateKey_Runner_); ok { + return x.Runner + } } return nil } func (x *StateKey) GetMultimapSideInput() *StateKey_MultimapSideInput { - if x, ok := x.GetType().(*StateKey_MultimapSideInput_); ok { - return x.MultimapSideInput + if x != nil { + if x, ok := x.Type.(*StateKey_MultimapSideInput_); ok { + return x.MultimapSideInput + } } return nil } func (x *StateKey) GetBagUserState() *StateKey_BagUserState { - if x, ok := x.GetType().(*StateKey_BagUserState_); ok { - return x.BagUserState + if x != nil { + if x, ok := x.Type.(*StateKey_BagUserState_); ok { + return x.BagUserState + } } return nil } func (x *StateKey) GetIterableSideInput() *StateKey_IterableSideInput { - if x, ok := x.GetType().(*StateKey_IterableSideInput_); ok { - return x.IterableSideInput + if x != nil { + if x, ok := x.Type.(*StateKey_IterableSideInput_); ok { + return x.IterableSideInput + } } return nil } func (x *StateKey) GetMultimapKeysSideInput() *StateKey_MultimapKeysSideInput { - if x, ok := x.GetType().(*StateKey_MultimapKeysSideInput_); ok { - return x.MultimapKeysSideInput + if x != nil { + if x, ok := x.Type.(*StateKey_MultimapKeysSideInput_); ok { + return x.MultimapKeysSideInput + } } return nil } func (x *StateKey) GetMultimapKeysValuesSideInput() *StateKey_MultimapKeysValuesSideInput { - if x, ok := x.GetType().(*StateKey_MultimapKeysValuesSideInput_); ok { - return x.MultimapKeysValuesSideInput + if x != nil { + if x, ok := x.Type.(*StateKey_MultimapKeysValuesSideInput_); ok { + return x.MultimapKeysValuesSideInput + } } return nil } func (x *StateKey) GetMultimapKeysUserState() *StateKey_MultimapKeysUserState { - if x, ok := x.GetType().(*StateKey_MultimapKeysUserState_); ok { - return x.MultimapKeysUserState + if x != nil { + if x, ok := x.Type.(*StateKey_MultimapKeysUserState_); ok { + return x.MultimapKeysUserState + } } return nil } func (x *StateKey) GetMultimapUserState() *StateKey_MultimapUserState { - if x, ok := x.GetType().(*StateKey_MultimapUserState_); ok { - return x.MultimapUserState + if x != nil { + if x, ok := x.Type.(*StateKey_MultimapUserState_); ok { + return x.MultimapUserState + } + } + return nil +} + +func (x *StateKey) GetOrderedListUserState() *StateKey_OrderedListUserState { + if x != nil { + if x, ok := x.Type.(*StateKey_OrderedListUserState_); ok { + return x.OrderedListUserState + } + } + return nil +} + +func (x *StateKey) SetRunner(v *StateKey_Runner) { + if v == nil { + x.Type = nil + return + } + x.Type = &StateKey_Runner_{v} +} + +func (x *StateKey) SetMultimapSideInput(v *StateKey_MultimapSideInput) { + if v == nil { + x.Type = nil + return + } + x.Type = &StateKey_MultimapSideInput_{v} +} + +func (x *StateKey) SetBagUserState(v *StateKey_BagUserState) { + if v == nil { + x.Type = nil + return + } + x.Type = &StateKey_BagUserState_{v} +} + +func (x *StateKey) SetIterableSideInput(v *StateKey_IterableSideInput) { + if v == nil { + x.Type = nil + return + } + x.Type = &StateKey_IterableSideInput_{v} +} + +func (x *StateKey) SetMultimapKeysSideInput(v *StateKey_MultimapKeysSideInput) { + if v == nil { + x.Type = nil + return + } + x.Type = &StateKey_MultimapKeysSideInput_{v} +} + +func (x *StateKey) SetMultimapKeysValuesSideInput(v *StateKey_MultimapKeysValuesSideInput) { + if v == nil { + x.Type = nil + return + } + x.Type = &StateKey_MultimapKeysValuesSideInput_{v} +} + +func (x *StateKey) SetMultimapKeysUserState(v *StateKey_MultimapKeysUserState) { + if v == nil { + x.Type = nil + return + } + x.Type = &StateKey_MultimapKeysUserState_{v} +} + +func (x *StateKey) SetMultimapUserState(v *StateKey_MultimapUserState) { + if v == nil { + x.Type = nil + return + } + x.Type = &StateKey_MultimapUserState_{v} +} + +func (x *StateKey) SetOrderedListUserState(v *StateKey_OrderedListUserState) { + if v == nil { + x.Type = nil + return + } + x.Type = &StateKey_OrderedListUserState_{v} +} + +func (x *StateKey) HasType() bool { + if x == nil { + return false + } + return x.Type != nil +} + +func (x *StateKey) HasRunner() bool { + if x == nil { + return false + } + _, ok := x.Type.(*StateKey_Runner_) + return ok +} + +func (x *StateKey) HasMultimapSideInput() bool { + if x == nil { + return false + } + _, ok := x.Type.(*StateKey_MultimapSideInput_) + return ok +} + +func (x *StateKey) HasBagUserState() bool { + if x == nil { + return false + } + _, ok := x.Type.(*StateKey_BagUserState_) + return ok +} + +func (x *StateKey) HasIterableSideInput() bool { + if x == nil { + return false + } + _, ok := x.Type.(*StateKey_IterableSideInput_) + return ok +} + +func (x *StateKey) HasMultimapKeysSideInput() bool { + if x == nil { + return false + } + _, ok := x.Type.(*StateKey_MultimapKeysSideInput_) + return ok +} + +func (x *StateKey) HasMultimapKeysValuesSideInput() bool { + if x == nil { + return false + } + _, ok := x.Type.(*StateKey_MultimapKeysValuesSideInput_) + return ok +} + +func (x *StateKey) HasMultimapKeysUserState() bool { + if x == nil { + return false + } + _, ok := x.Type.(*StateKey_MultimapKeysUserState_) + return ok +} + +func (x *StateKey) HasMultimapUserState() bool { + if x == nil { + return false + } + _, ok := x.Type.(*StateKey_MultimapUserState_) + return ok +} + +func (x *StateKey) HasOrderedListUserState() bool { + if x == nil { + return false + } + _, ok := x.Type.(*StateKey_OrderedListUserState_) + return ok +} + +func (x *StateKey) ClearType() { + x.Type = nil +} + +func (x *StateKey) ClearRunner() { + if _, ok := x.Type.(*StateKey_Runner_); ok { + x.Type = nil + } +} + +func (x *StateKey) ClearMultimapSideInput() { + if _, ok := x.Type.(*StateKey_MultimapSideInput_); ok { + x.Type = nil + } +} + +func (x *StateKey) ClearBagUserState() { + if _, ok := x.Type.(*StateKey_BagUserState_); ok { + x.Type = nil + } +} + +func (x *StateKey) ClearIterableSideInput() { + if _, ok := x.Type.(*StateKey_IterableSideInput_); ok { + x.Type = nil + } +} + +func (x *StateKey) ClearMultimapKeysSideInput() { + if _, ok := x.Type.(*StateKey_MultimapKeysSideInput_); ok { + x.Type = nil + } +} + +func (x *StateKey) ClearMultimapKeysValuesSideInput() { + if _, ok := x.Type.(*StateKey_MultimapKeysValuesSideInput_); ok { + x.Type = nil + } +} + +func (x *StateKey) ClearMultimapKeysUserState() { + if _, ok := x.Type.(*StateKey_MultimapKeysUserState_); ok { + x.Type = nil + } +} + +func (x *StateKey) ClearMultimapUserState() { + if _, ok := x.Type.(*StateKey_MultimapUserState_); ok { + x.Type = nil + } +} + +func (x *StateKey) ClearOrderedListUserState() { + if _, ok := x.Type.(*StateKey_OrderedListUserState_); ok { + x.Type = nil + } +} + +const StateKey_Type_not_set_case case_StateKey_Type = 0 +const StateKey_Runner_case case_StateKey_Type = 1 +const StateKey_MultimapSideInput_case case_StateKey_Type = 2 +const StateKey_BagUserState_case case_StateKey_Type = 3 +const StateKey_IterableSideInput_case case_StateKey_Type = 4 +const StateKey_MultimapKeysSideInput_case case_StateKey_Type = 5 +const StateKey_MultimapKeysValuesSideInput_case case_StateKey_Type = 8 +const StateKey_MultimapKeysUserState_case case_StateKey_Type = 6 +const StateKey_MultimapUserState_case case_StateKey_Type = 7 +const StateKey_OrderedListUserState_case case_StateKey_Type = 9 + +func (x *StateKey) WhichType() case_StateKey_Type { + if x == nil { + return StateKey_Type_not_set_case + } + switch x.Type.(type) { + case *StateKey_Runner_: + return StateKey_Runner_case + case *StateKey_MultimapSideInput_: + return StateKey_MultimapSideInput_case + case *StateKey_BagUserState_: + return StateKey_BagUserState_case + case *StateKey_IterableSideInput_: + return StateKey_IterableSideInput_case + case *StateKey_MultimapKeysSideInput_: + return StateKey_MultimapKeysSideInput_case + case *StateKey_MultimapKeysValuesSideInput_: + return StateKey_MultimapKeysValuesSideInput_case + case *StateKey_MultimapKeysUserState_: + return StateKey_MultimapKeysUserState_case + case *StateKey_MultimapUserState_: + return StateKey_MultimapUserState_case + case *StateKey_OrderedListUserState_: + return StateKey_OrderedListUserState_case + default: + return StateKey_Type_not_set_case + } +} + +type StateKey_builder struct { + _ [0]func() // Prevents comparability and use of unkeyed literals for the builder. + + // (Required) One of the following state keys must be set. + + // Fields of oneof Type: + Runner *StateKey_Runner + MultimapSideInput *StateKey_MultimapSideInput + BagUserState *StateKey_BagUserState + IterableSideInput *StateKey_IterableSideInput + MultimapKeysSideInput *StateKey_MultimapKeysSideInput + MultimapKeysValuesSideInput *StateKey_MultimapKeysValuesSideInput + MultimapKeysUserState *StateKey_MultimapKeysUserState + MultimapUserState *StateKey_MultimapUserState + OrderedListUserState *StateKey_OrderedListUserState + // -- end of Type +} + +func (b0 StateKey_builder) Build() *StateKey { + m0 := &StateKey{} + b, x := &b0, m0 + _, _ = b, x + if b.Runner != nil { + x.Type = &StateKey_Runner_{b.Runner} + } + if b.MultimapSideInput != nil { + x.Type = &StateKey_MultimapSideInput_{b.MultimapSideInput} + } + if b.BagUserState != nil { + x.Type = &StateKey_BagUserState_{b.BagUserState} + } + if b.IterableSideInput != nil { + x.Type = &StateKey_IterableSideInput_{b.IterableSideInput} + } + if b.MultimapKeysSideInput != nil { + x.Type = &StateKey_MultimapKeysSideInput_{b.MultimapKeysSideInput} + } + if b.MultimapKeysValuesSideInput != nil { + x.Type = &StateKey_MultimapKeysValuesSideInput_{b.MultimapKeysValuesSideInput} + } + if b.MultimapKeysUserState != nil { + x.Type = &StateKey_MultimapKeysUserState_{b.MultimapKeysUserState} } - return nil + if b.MultimapUserState != nil { + x.Type = &StateKey_MultimapUserState_{b.MultimapUserState} + } + if b.OrderedListUserState != nil { + x.Type = &StateKey_OrderedListUserState_{b.OrderedListUserState} + } + return m0 } -func (x *StateKey) GetOrderedListUserState() *StateKey_OrderedListUserState { - if x, ok := x.GetType().(*StateKey_OrderedListUserState_); ok { - return x.OrderedListUserState +type case_StateKey_Type protoreflect.FieldNumber + +func (x case_StateKey_Type) String() string { + md := file_org_apache_beam_model_fn_execution_v1_beam_fn_api_proto_msgTypes[28].Descriptor() + if x == 0 { + return "not set" } - return nil + return protoimpl.X.MessageFieldStringOf(md, protoreflect.FieldNumber(x)) } type isStateKey_Type interface { @@ -2617,25 +4519,22 @@ func (*StateKey_OrderedListUserState_) isStateKey_Type() {} // A request to get state. type StateGetRequest struct { - state protoimpl.MessageState - sizeCache protoimpl.SizeCache - unknownFields protoimpl.UnknownFields - + state protoimpl.MessageState `protogen:"hybrid.v1"` // (Optional) If specified, signals to the runner that the response // should resume from the following continuation token. // // If unspecified, signals to the runner that the response should start // from the beginning of the logical continuable stream. ContinuationToken []byte `protobuf:"bytes,1,opt,name=continuation_token,json=continuationToken,proto3" json:"continuation_token,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache } func (x *StateGetRequest) Reset() { *x = StateGetRequest{} - if protoimpl.UnsafeEnabled { - mi := &file_org_apache_beam_model_fn_execution_v1_beam_fn_api_proto_msgTypes[29] - ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) - ms.StoreMessageInfo(mi) - } + mi := &file_org_apache_beam_model_fn_execution_v1_beam_fn_api_proto_msgTypes[29] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) } func (x *StateGetRequest) String() string { @@ -2646,7 +4545,7 @@ func (*StateGetRequest) ProtoMessage() {} func (x *StateGetRequest) ProtoReflect() protoreflect.Message { mi := &file_org_apache_beam_model_fn_execution_v1_beam_fn_api_proto_msgTypes[29] - if protoimpl.UnsafeEnabled && x != nil { + if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { ms.StoreMessageInfo(mi) @@ -2656,11 +4555,6 @@ func (x *StateGetRequest) ProtoReflect() protoreflect.Message { return mi.MessageOf(x) } -// Deprecated: Use StateGetRequest.ProtoReflect.Descriptor instead. -func (*StateGetRequest) Descriptor() ([]byte, []int) { - return file_org_apache_beam_model_fn_execution_v1_beam_fn_api_proto_rawDescGZIP(), []int{29} -} - func (x *StateGetRequest) GetContinuationToken() []byte { if x != nil { return x.ContinuationToken @@ -2668,13 +4562,36 @@ func (x *StateGetRequest) GetContinuationToken() []byte { return nil } +func (x *StateGetRequest) SetContinuationToken(v []byte) { + if v == nil { + v = []byte{} + } + x.ContinuationToken = v +} + +type StateGetRequest_builder struct { + _ [0]func() // Prevents comparability and use of unkeyed literals for the builder. + + // (Optional) If specified, signals to the runner that the response + // should resume from the following continuation token. + // + // If unspecified, signals to the runner that the response should start + // from the beginning of the logical continuable stream. + ContinuationToken []byte +} + +func (b0 StateGetRequest_builder) Build() *StateGetRequest { + m0 := &StateGetRequest{} + b, x := &b0, m0 + _, _ = b, x + x.ContinuationToken = b.ContinuationToken + return m0 +} + // A response to get state representing a logical byte stream which can be // continued using the state API. type StateGetResponse struct { - state protoimpl.MessageState - sizeCache protoimpl.SizeCache - unknownFields protoimpl.UnknownFields - + state protoimpl.MessageState `protogen:"hybrid.v1"` // (Optional) If specified, represents a token which can be used with the // state API to get the next chunk of this logical byte stream. The end of // the logical byte stream is signalled by this field being unset. @@ -2684,16 +4601,16 @@ type StateGetResponse struct { // concatenated together. // // See also the note about OrderedListState in StateAppendRequest. - Data []byte `protobuf:"bytes,2,opt,name=data,proto3" json:"data,omitempty"` + Data []byte `protobuf:"bytes,2,opt,name=data,proto3" json:"data,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache } func (x *StateGetResponse) Reset() { *x = StateGetResponse{} - if protoimpl.UnsafeEnabled { - mi := &file_org_apache_beam_model_fn_execution_v1_beam_fn_api_proto_msgTypes[30] - ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) - ms.StoreMessageInfo(mi) - } + mi := &file_org_apache_beam_model_fn_execution_v1_beam_fn_api_proto_msgTypes[30] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) } func (x *StateGetResponse) String() string { @@ -2704,7 +4621,7 @@ func (*StateGetResponse) ProtoMessage() {} func (x *StateGetResponse) ProtoReflect() protoreflect.Message { mi := &file_org_apache_beam_model_fn_execution_v1_beam_fn_api_proto_msgTypes[30] - if protoimpl.UnsafeEnabled && x != nil { + if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { ms.StoreMessageInfo(mi) @@ -2714,11 +4631,6 @@ func (x *StateGetResponse) ProtoReflect() protoreflect.Message { return mi.MessageOf(x) } -// Deprecated: Use StateGetResponse.ProtoReflect.Descriptor instead. -func (*StateGetResponse) Descriptor() ([]byte, []int) { - return file_org_apache_beam_model_fn_execution_v1_beam_fn_api_proto_rawDescGZIP(), []int{30} -} - func (x *StateGetResponse) GetContinuationToken() []byte { if x != nil { return x.ContinuationToken @@ -2733,12 +4645,47 @@ func (x *StateGetResponse) GetData() []byte { return nil } +func (x *StateGetResponse) SetContinuationToken(v []byte) { + if v == nil { + v = []byte{} + } + x.ContinuationToken = v +} + +func (x *StateGetResponse) SetData(v []byte) { + if v == nil { + v = []byte{} + } + x.Data = v +} + +type StateGetResponse_builder struct { + _ [0]func() // Prevents comparability and use of unkeyed literals for the builder. + + // (Optional) If specified, represents a token which can be used with the + // state API to get the next chunk of this logical byte stream. The end of + // the logical byte stream is signalled by this field being unset. + ContinuationToken []byte + // Represents a part of a logical byte stream. Elements within + // the logical byte stream are encoded in the nested context and + // concatenated together. + // + // See also the note about OrderedListState in StateAppendRequest. + Data []byte +} + +func (b0 StateGetResponse_builder) Build() *StateGetResponse { + m0 := &StateGetResponse{} + b, x := &b0, m0 + _, _ = b, x + x.ContinuationToken = b.ContinuationToken + x.Data = b.Data + return m0 +} + // A request to append state. type StateAppendRequest struct { - state protoimpl.MessageState - sizeCache protoimpl.SizeCache - unknownFields protoimpl.UnknownFields - + state protoimpl.MessageState `protogen:"hybrid.v1"` // Represents a part of a logical byte stream. Elements within // the logical byte stream are encoded in the nested context and // multiple append requests are concatenated together. @@ -2747,16 +4694,16 @@ type StateAppendRequest struct { // beam:coder:kv:v1 coder, where the first (key) component must be a // beam:coder:varint:v1 and the second (value) component must be encoded // with a beam:coder:length_prefix:v1 coder. - Data []byte `protobuf:"bytes,1,opt,name=data,proto3" json:"data,omitempty"` + Data []byte `protobuf:"bytes,1,opt,name=data,proto3" json:"data,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache } func (x *StateAppendRequest) Reset() { *x = StateAppendRequest{} - if protoimpl.UnsafeEnabled { - mi := &file_org_apache_beam_model_fn_execution_v1_beam_fn_api_proto_msgTypes[31] - ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) - ms.StoreMessageInfo(mi) - } + mi := &file_org_apache_beam_model_fn_execution_v1_beam_fn_api_proto_msgTypes[31] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) } func (x *StateAppendRequest) String() string { @@ -2767,7 +4714,7 @@ func (*StateAppendRequest) ProtoMessage() {} func (x *StateAppendRequest) ProtoReflect() protoreflect.Message { mi := &file_org_apache_beam_model_fn_execution_v1_beam_fn_api_proto_msgTypes[31] - if protoimpl.UnsafeEnabled && x != nil { + if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { ms.StoreMessageInfo(mi) @@ -2777,11 +4724,6 @@ func (x *StateAppendRequest) ProtoReflect() protoreflect.Message { return mi.MessageOf(x) } -// Deprecated: Use StateAppendRequest.ProtoReflect.Descriptor instead. -func (*StateAppendRequest) Descriptor() ([]byte, []int) { - return file_org_apache_beam_model_fn_execution_v1_beam_fn_api_proto_rawDescGZIP(), []int{31} -} - func (x *StateAppendRequest) GetData() []byte { if x != nil { return x.Data @@ -2789,20 +4731,47 @@ func (x *StateAppendRequest) GetData() []byte { return nil } +func (x *StateAppendRequest) SetData(v []byte) { + if v == nil { + v = []byte{} + } + x.Data = v +} + +type StateAppendRequest_builder struct { + _ [0]func() // Prevents comparability and use of unkeyed literals for the builder. + + // Represents a part of a logical byte stream. Elements within + // the logical byte stream are encoded in the nested context and + // multiple append requests are concatenated together. + // + // For OrderedListState, elements of should be encoded with the + // beam:coder:kv:v1 coder, where the first (key) component must be a + // beam:coder:varint:v1 and the second (value) component must be encoded + // with a beam:coder:length_prefix:v1 coder. + Data []byte +} + +func (b0 StateAppendRequest_builder) Build() *StateAppendRequest { + m0 := &StateAppendRequest{} + b, x := &b0, m0 + _, _ = b, x + x.Data = b.Data + return m0 +} + // A response to append state. type StateAppendResponse struct { - state protoimpl.MessageState - sizeCache protoimpl.SizeCache + state protoimpl.MessageState `protogen:"hybrid.v1"` unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache } func (x *StateAppendResponse) Reset() { *x = StateAppendResponse{} - if protoimpl.UnsafeEnabled { - mi := &file_org_apache_beam_model_fn_execution_v1_beam_fn_api_proto_msgTypes[32] - ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) - ms.StoreMessageInfo(mi) - } + mi := &file_org_apache_beam_model_fn_execution_v1_beam_fn_api_proto_msgTypes[32] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) } func (x *StateAppendResponse) String() string { @@ -2813,7 +4782,7 @@ func (*StateAppendResponse) ProtoMessage() {} func (x *StateAppendResponse) ProtoReflect() protoreflect.Message { mi := &file_org_apache_beam_model_fn_execution_v1_beam_fn_api_proto_msgTypes[32] - if protoimpl.UnsafeEnabled && x != nil { + if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { ms.StoreMessageInfo(mi) @@ -2823,25 +4792,30 @@ func (x *StateAppendResponse) ProtoReflect() protoreflect.Message { return mi.MessageOf(x) } -// Deprecated: Use StateAppendResponse.ProtoReflect.Descriptor instead. -func (*StateAppendResponse) Descriptor() ([]byte, []int) { - return file_org_apache_beam_model_fn_execution_v1_beam_fn_api_proto_rawDescGZIP(), []int{32} +type StateAppendResponse_builder struct { + _ [0]func() // Prevents comparability and use of unkeyed literals for the builder. + +} + +func (b0 StateAppendResponse_builder) Build() *StateAppendResponse { + m0 := &StateAppendResponse{} + b, x := &b0, m0 + _, _ = b, x + return m0 } // A request to clear state. type StateClearRequest struct { - state protoimpl.MessageState - sizeCache protoimpl.SizeCache + state protoimpl.MessageState `protogen:"hybrid.v1"` unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache } func (x *StateClearRequest) Reset() { *x = StateClearRequest{} - if protoimpl.UnsafeEnabled { - mi := &file_org_apache_beam_model_fn_execution_v1_beam_fn_api_proto_msgTypes[33] - ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) - ms.StoreMessageInfo(mi) - } + mi := &file_org_apache_beam_model_fn_execution_v1_beam_fn_api_proto_msgTypes[33] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) } func (x *StateClearRequest) String() string { @@ -2852,7 +4826,7 @@ func (*StateClearRequest) ProtoMessage() {} func (x *StateClearRequest) ProtoReflect() protoreflect.Message { mi := &file_org_apache_beam_model_fn_execution_v1_beam_fn_api_proto_msgTypes[33] - if protoimpl.UnsafeEnabled && x != nil { + if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { ms.StoreMessageInfo(mi) @@ -2862,25 +4836,30 @@ func (x *StateClearRequest) ProtoReflect() protoreflect.Message { return mi.MessageOf(x) } -// Deprecated: Use StateClearRequest.ProtoReflect.Descriptor instead. -func (*StateClearRequest) Descriptor() ([]byte, []int) { - return file_org_apache_beam_model_fn_execution_v1_beam_fn_api_proto_rawDescGZIP(), []int{33} +type StateClearRequest_builder struct { + _ [0]func() // Prevents comparability and use of unkeyed literals for the builder. + +} + +func (b0 StateClearRequest_builder) Build() *StateClearRequest { + m0 := &StateClearRequest{} + b, x := &b0, m0 + _, _ = b, x + return m0 } // A response to clear state. type StateClearResponse struct { - state protoimpl.MessageState - sizeCache protoimpl.SizeCache + state protoimpl.MessageState `protogen:"hybrid.v1"` unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache } func (x *StateClearResponse) Reset() { *x = StateClearResponse{} - if protoimpl.UnsafeEnabled { - mi := &file_org_apache_beam_model_fn_execution_v1_beam_fn_api_proto_msgTypes[34] - ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) - ms.StoreMessageInfo(mi) - } + mi := &file_org_apache_beam_model_fn_execution_v1_beam_fn_api_proto_msgTypes[34] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) } func (x *StateClearResponse) String() string { @@ -2891,7 +4870,7 @@ func (*StateClearResponse) ProtoMessage() {} func (x *StateClearResponse) ProtoReflect() protoreflect.Message { mi := &file_org_apache_beam_model_fn_execution_v1_beam_fn_api_proto_msgTypes[34] - if protoimpl.UnsafeEnabled && x != nil { + if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { ms.StoreMessageInfo(mi) @@ -2901,28 +4880,32 @@ func (x *StateClearResponse) ProtoReflect() protoreflect.Message { return mi.MessageOf(x) } -// Deprecated: Use StateClearResponse.ProtoReflect.Descriptor instead. -func (*StateClearResponse) Descriptor() ([]byte, []int) { - return file_org_apache_beam_model_fn_execution_v1_beam_fn_api_proto_rawDescGZIP(), []int{34} +type StateClearResponse_builder struct { + _ [0]func() // Prevents comparability and use of unkeyed literals for the builder. + +} + +func (b0 StateClearResponse_builder) Build() *StateClearResponse { + m0 := &StateClearResponse{} + b, x := &b0, m0 + _, _ = b, x + return m0 } // A message describes a sort key range [start, end). type OrderedListRange struct { - state protoimpl.MessageState - sizeCache protoimpl.SizeCache + state protoimpl.MessageState `protogen:"hybrid.v1"` + Start int64 `protobuf:"varint,1,opt,name=start,proto3" json:"start,omitempty"` + End int64 `protobuf:"varint,2,opt,name=end,proto3" json:"end,omitempty"` unknownFields protoimpl.UnknownFields - - Start int64 `protobuf:"varint,1,opt,name=start,proto3" json:"start,omitempty"` - End int64 `protobuf:"varint,2,opt,name=end,proto3" json:"end,omitempty"` + sizeCache protoimpl.SizeCache } func (x *OrderedListRange) Reset() { *x = OrderedListRange{} - if protoimpl.UnsafeEnabled { - mi := &file_org_apache_beam_model_fn_execution_v1_beam_fn_api_proto_msgTypes[35] - ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) - ms.StoreMessageInfo(mi) - } + mi := &file_org_apache_beam_model_fn_execution_v1_beam_fn_api_proto_msgTypes[35] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) } func (x *OrderedListRange) String() string { @@ -2933,7 +4916,7 @@ func (*OrderedListRange) ProtoMessage() {} func (x *OrderedListRange) ProtoReflect() protoreflect.Message { mi := &file_org_apache_beam_model_fn_execution_v1_beam_fn_api_proto_msgTypes[35] - if protoimpl.UnsafeEnabled && x != nil { + if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { ms.StoreMessageInfo(mi) @@ -2943,11 +4926,6 @@ func (x *OrderedListRange) ProtoReflect() protoreflect.Message { return mi.MessageOf(x) } -// Deprecated: Use OrderedListRange.ProtoReflect.Descriptor instead. -func (*OrderedListRange) Descriptor() ([]byte, []int) { - return file_org_apache_beam_model_fn_execution_v1_beam_fn_api_proto_rawDescGZIP(), []int{35} -} - func (x *OrderedListRange) GetStart() int64 { if x != nil { return x.Start @@ -2962,12 +4940,33 @@ func (x *OrderedListRange) GetEnd() int64 { return 0 } +func (x *OrderedListRange) SetStart(v int64) { + x.Start = v +} + +func (x *OrderedListRange) SetEnd(v int64) { + x.End = v +} + +type OrderedListRange_builder struct { + _ [0]func() // Prevents comparability and use of unkeyed literals for the builder. + + Start int64 + End int64 +} + +func (b0 OrderedListRange_builder) Build() *OrderedListRange { + m0 := &OrderedListRange{} + b, x := &b0, m0 + _, _ = b, x + x.Start = b.Start + x.End = b.End + return m0 +} + // A log entry type LogEntry struct { - state protoimpl.MessageState - sizeCache protoimpl.SizeCache - unknownFields protoimpl.UnknownFields - + state protoimpl.MessageState `protogen:"hybrid.v1"` // (Required) The severity of the log statement. Severity LogEntry_Severity_Enum `protobuf:"varint,1,opt,name=severity,proto3,enum=org.apache.beam.model.fn_execution.v1.LogEntry_Severity_Enum" json:"severity,omitempty"` // (Required) The time at which this log statement occurred. @@ -2996,16 +4995,16 @@ type LogEntry struct { Thread string `protobuf:"bytes,8,opt,name=thread,proto3" json:"thread,omitempty"` // (Optional) Additional structured data to log. // Keys are limited to these characters: [a-zA-Z_-] - CustomData *structpb.Struct `protobuf:"bytes,9,opt,name=custom_data,json=customData,proto3" json:"custom_data,omitempty"` + CustomData *structpb.Struct `protobuf:"bytes,9,opt,name=custom_data,json=customData,proto3" json:"custom_data,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache } func (x *LogEntry) Reset() { *x = LogEntry{} - if protoimpl.UnsafeEnabled { - mi := &file_org_apache_beam_model_fn_execution_v1_beam_fn_api_proto_msgTypes[36] - ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) - ms.StoreMessageInfo(mi) - } + mi := &file_org_apache_beam_model_fn_execution_v1_beam_fn_api_proto_msgTypes[36] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) } func (x *LogEntry) String() string { @@ -3016,7 +5015,7 @@ func (*LogEntry) ProtoMessage() {} func (x *LogEntry) ProtoReflect() protoreflect.Message { mi := &file_org_apache_beam_model_fn_execution_v1_beam_fn_api_proto_msgTypes[36] - if protoimpl.UnsafeEnabled && x != nil { + if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { ms.StoreMessageInfo(mi) @@ -3026,11 +5025,6 @@ func (x *LogEntry) ProtoReflect() protoreflect.Message { return mi.MessageOf(x) } -// Deprecated: Use LogEntry.ProtoReflect.Descriptor instead. -func (*LogEntry) Descriptor() ([]byte, []int) { - return file_org_apache_beam_model_fn_execution_v1_beam_fn_api_proto_rawDescGZIP(), []int{36} -} - func (x *LogEntry) GetSeverity() LogEntry_Severity_Enum { if x != nil { return x.Severity @@ -3094,19 +5088,125 @@ func (x *LogEntry) GetCustomData() *structpb.Struct { return nil } +func (x *LogEntry) SetSeverity(v LogEntry_Severity_Enum) { + x.Severity = v +} + +func (x *LogEntry) SetTimestamp(v *timestamppb.Timestamp) { + x.Timestamp = v +} + +func (x *LogEntry) SetMessage(v string) { + x.Message = v +} + +func (x *LogEntry) SetTrace(v string) { + x.Trace = v +} + +func (x *LogEntry) SetInstructionId(v string) { + x.InstructionId = v +} + +func (x *LogEntry) SetTransformId(v string) { + x.TransformId = v +} + +func (x *LogEntry) SetLogLocation(v string) { + x.LogLocation = v +} + +func (x *LogEntry) SetThread(v string) { + x.Thread = v +} + +func (x *LogEntry) SetCustomData(v *structpb.Struct) { + x.CustomData = v +} + +func (x *LogEntry) HasTimestamp() bool { + if x == nil { + return false + } + return x.Timestamp != nil +} + +func (x *LogEntry) HasCustomData() bool { + if x == nil { + return false + } + return x.CustomData != nil +} + +func (x *LogEntry) ClearTimestamp() { + x.Timestamp = nil +} + +func (x *LogEntry) ClearCustomData() { + x.CustomData = nil +} + +type LogEntry_builder struct { + _ [0]func() // Prevents comparability and use of unkeyed literals for the builder. + + // (Required) The severity of the log statement. + Severity LogEntry_Severity_Enum + // (Required) The time at which this log statement occurred. + Timestamp *timestamppb.Timestamp + // (Required) A human readable message. + Message string + // (Optional) An optional trace of the functions involved. For example, in + // Java this can include multiple causes and multiple suppressed exceptions. + Trace string + // (Optional) A reference to the instruction this log statement is associated + // with. + InstructionId string + // (Optional) A reference to the transform this log statement is + // associated with. + TransformId string + // (Optional) Human-readable name of the function or method being invoked, + // with optional context such as the class or package name. The format can + // vary by language. For example: + // + // qual.if.ied.Class.method (Java) + // dir/package.func (Go) + // module.function (Python) + // file.cc:382 (C++) + LogLocation string + // (Optional) The name of the thread this log statement is associated with. + Thread string + // (Optional) Additional structured data to log. + // Keys are limited to these characters: [a-zA-Z_-] + CustomData *structpb.Struct +} + +func (b0 LogEntry_builder) Build() *LogEntry { + m0 := &LogEntry{} + b, x := &b0, m0 + _, _ = b, x + x.Severity = b.Severity + x.Timestamp = b.Timestamp + x.Message = b.Message + x.Trace = b.Trace + x.InstructionId = b.InstructionId + x.TransformId = b.TransformId + x.LogLocation = b.LogLocation + x.Thread = b.Thread + x.CustomData = b.CustomData + return m0 +} + type LogControl struct { - state protoimpl.MessageState - sizeCache protoimpl.SizeCache + state protoimpl.MessageState `protogen:"hybrid.v1"` unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache } func (x *LogControl) Reset() { *x = LogControl{} - if protoimpl.UnsafeEnabled { - mi := &file_org_apache_beam_model_fn_execution_v1_beam_fn_api_proto_msgTypes[37] - ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) - ms.StoreMessageInfo(mi) - } + mi := &file_org_apache_beam_model_fn_execution_v1_beam_fn_api_proto_msgTypes[37] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) } func (x *LogControl) String() string { @@ -3117,7 +5217,7 @@ func (*LogControl) ProtoMessage() {} func (x *LogControl) ProtoReflect() protoreflect.Message { mi := &file_org_apache_beam_model_fn_execution_v1_beam_fn_api_proto_msgTypes[37] - if protoimpl.UnsafeEnabled && x != nil { + if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { ms.StoreMessageInfo(mi) @@ -3127,31 +5227,35 @@ func (x *LogControl) ProtoReflect() protoreflect.Message { return mi.MessageOf(x) } -// Deprecated: Use LogControl.ProtoReflect.Descriptor instead. -func (*LogControl) Descriptor() ([]byte, []int) { - return file_org_apache_beam_model_fn_execution_v1_beam_fn_api_proto_rawDescGZIP(), []int{37} +type LogControl_builder struct { + _ [0]func() // Prevents comparability and use of unkeyed literals for the builder. + } -type StartWorkerRequest struct { - state protoimpl.MessageState - sizeCache protoimpl.SizeCache - unknownFields protoimpl.UnknownFields +func (b0 LogControl_builder) Build() *LogControl { + m0 := &LogControl{} + b, x := &b0, m0 + _, _ = b, x + return m0 +} +type StartWorkerRequest struct { + state protoimpl.MessageState `protogen:"hybrid.v1"` WorkerId string `protobuf:"bytes,1,opt,name=worker_id,json=workerId,proto3" json:"worker_id,omitempty"` ControlEndpoint *pipeline_v1.ApiServiceDescriptor `protobuf:"bytes,2,opt,name=control_endpoint,json=controlEndpoint,proto3" json:"control_endpoint,omitempty"` LoggingEndpoint *pipeline_v1.ApiServiceDescriptor `protobuf:"bytes,3,opt,name=logging_endpoint,json=loggingEndpoint,proto3" json:"logging_endpoint,omitempty"` ArtifactEndpoint *pipeline_v1.ApiServiceDescriptor `protobuf:"bytes,4,opt,name=artifact_endpoint,json=artifactEndpoint,proto3" json:"artifact_endpoint,omitempty"` ProvisionEndpoint *pipeline_v1.ApiServiceDescriptor `protobuf:"bytes,5,opt,name=provision_endpoint,json=provisionEndpoint,proto3" json:"provision_endpoint,omitempty"` - Params map[string]string `protobuf:"bytes,10,rep,name=params,proto3" json:"params,omitempty" protobuf_key:"bytes,1,opt,name=key,proto3" protobuf_val:"bytes,2,opt,name=value,proto3"` + Params map[string]string `protobuf:"bytes,10,rep,name=params,proto3" json:"params,omitempty" protobuf_key:"bytes,1,opt,name=key" protobuf_val:"bytes,2,opt,name=value"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache } func (x *StartWorkerRequest) Reset() { *x = StartWorkerRequest{} - if protoimpl.UnsafeEnabled { - mi := &file_org_apache_beam_model_fn_execution_v1_beam_fn_api_proto_msgTypes[38] - ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) - ms.StoreMessageInfo(mi) - } + mi := &file_org_apache_beam_model_fn_execution_v1_beam_fn_api_proto_msgTypes[38] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) } func (x *StartWorkerRequest) String() string { @@ -3162,7 +5266,7 @@ func (*StartWorkerRequest) ProtoMessage() {} func (x *StartWorkerRequest) ProtoReflect() protoreflect.Message { mi := &file_org_apache_beam_model_fn_execution_v1_beam_fn_api_proto_msgTypes[38] - if protoimpl.UnsafeEnabled && x != nil { + if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { ms.StoreMessageInfo(mi) @@ -3172,11 +5276,6 @@ func (x *StartWorkerRequest) ProtoReflect() protoreflect.Message { return mi.MessageOf(x) } -// Deprecated: Use StartWorkerRequest.ProtoReflect.Descriptor instead. -func (*StartWorkerRequest) Descriptor() ([]byte, []int) { - return file_org_apache_beam_model_fn_execution_v1_beam_fn_api_proto_rawDescGZIP(), []int{38} -} - func (x *StartWorkerRequest) GetWorkerId() string { if x != nil { return x.WorkerId @@ -3205,35 +5304,124 @@ func (x *StartWorkerRequest) GetArtifactEndpoint() *pipeline_v1.ApiServiceDescri return nil } -func (x *StartWorkerRequest) GetProvisionEndpoint() *pipeline_v1.ApiServiceDescriptor { - if x != nil { - return x.ProvisionEndpoint - } - return nil +func (x *StartWorkerRequest) GetProvisionEndpoint() *pipeline_v1.ApiServiceDescriptor { + if x != nil { + return x.ProvisionEndpoint + } + return nil +} + +func (x *StartWorkerRequest) GetParams() map[string]string { + if x != nil { + return x.Params + } + return nil +} + +func (x *StartWorkerRequest) SetWorkerId(v string) { + x.WorkerId = v +} + +func (x *StartWorkerRequest) SetControlEndpoint(v *pipeline_v1.ApiServiceDescriptor) { + x.ControlEndpoint = v +} + +func (x *StartWorkerRequest) SetLoggingEndpoint(v *pipeline_v1.ApiServiceDescriptor) { + x.LoggingEndpoint = v +} + +func (x *StartWorkerRequest) SetArtifactEndpoint(v *pipeline_v1.ApiServiceDescriptor) { + x.ArtifactEndpoint = v +} + +func (x *StartWorkerRequest) SetProvisionEndpoint(v *pipeline_v1.ApiServiceDescriptor) { + x.ProvisionEndpoint = v +} + +func (x *StartWorkerRequest) SetParams(v map[string]string) { + x.Params = v +} + +func (x *StartWorkerRequest) HasControlEndpoint() bool { + if x == nil { + return false + } + return x.ControlEndpoint != nil +} + +func (x *StartWorkerRequest) HasLoggingEndpoint() bool { + if x == nil { + return false + } + return x.LoggingEndpoint != nil +} + +func (x *StartWorkerRequest) HasArtifactEndpoint() bool { + if x == nil { + return false + } + return x.ArtifactEndpoint != nil +} + +func (x *StartWorkerRequest) HasProvisionEndpoint() bool { + if x == nil { + return false + } + return x.ProvisionEndpoint != nil +} + +func (x *StartWorkerRequest) ClearControlEndpoint() { + x.ControlEndpoint = nil +} + +func (x *StartWorkerRequest) ClearLoggingEndpoint() { + x.LoggingEndpoint = nil +} + +func (x *StartWorkerRequest) ClearArtifactEndpoint() { + x.ArtifactEndpoint = nil +} + +func (x *StartWorkerRequest) ClearProvisionEndpoint() { + x.ProvisionEndpoint = nil } -func (x *StartWorkerRequest) GetParams() map[string]string { - if x != nil { - return x.Params - } - return nil +type StartWorkerRequest_builder struct { + _ [0]func() // Prevents comparability and use of unkeyed literals for the builder. + + WorkerId string + ControlEndpoint *pipeline_v1.ApiServiceDescriptor + LoggingEndpoint *pipeline_v1.ApiServiceDescriptor + ArtifactEndpoint *pipeline_v1.ApiServiceDescriptor + ProvisionEndpoint *pipeline_v1.ApiServiceDescriptor + Params map[string]string +} + +func (b0 StartWorkerRequest_builder) Build() *StartWorkerRequest { + m0 := &StartWorkerRequest{} + b, x := &b0, m0 + _, _ = b, x + x.WorkerId = b.WorkerId + x.ControlEndpoint = b.ControlEndpoint + x.LoggingEndpoint = b.LoggingEndpoint + x.ArtifactEndpoint = b.ArtifactEndpoint + x.ProvisionEndpoint = b.ProvisionEndpoint + x.Params = b.Params + return m0 } type StartWorkerResponse struct { - state protoimpl.MessageState - sizeCache protoimpl.SizeCache + state protoimpl.MessageState `protogen:"hybrid.v1"` + Error string `protobuf:"bytes,1,opt,name=error,proto3" json:"error,omitempty"` unknownFields protoimpl.UnknownFields - - Error string `protobuf:"bytes,1,opt,name=error,proto3" json:"error,omitempty"` + sizeCache protoimpl.SizeCache } func (x *StartWorkerResponse) Reset() { *x = StartWorkerResponse{} - if protoimpl.UnsafeEnabled { - mi := &file_org_apache_beam_model_fn_execution_v1_beam_fn_api_proto_msgTypes[39] - ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) - ms.StoreMessageInfo(mi) - } + mi := &file_org_apache_beam_model_fn_execution_v1_beam_fn_api_proto_msgTypes[39] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) } func (x *StartWorkerResponse) String() string { @@ -3244,7 +5432,7 @@ func (*StartWorkerResponse) ProtoMessage() {} func (x *StartWorkerResponse) ProtoReflect() protoreflect.Message { mi := &file_org_apache_beam_model_fn_execution_v1_beam_fn_api_proto_msgTypes[39] - if protoimpl.UnsafeEnabled && x != nil { + if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { ms.StoreMessageInfo(mi) @@ -3254,11 +5442,6 @@ func (x *StartWorkerResponse) ProtoReflect() protoreflect.Message { return mi.MessageOf(x) } -// Deprecated: Use StartWorkerResponse.ProtoReflect.Descriptor instead. -func (*StartWorkerResponse) Descriptor() ([]byte, []int) { - return file_org_apache_beam_model_fn_execution_v1_beam_fn_api_proto_rawDescGZIP(), []int{39} -} - func (x *StartWorkerResponse) GetError() string { if x != nil { return x.Error @@ -3266,21 +5449,36 @@ func (x *StartWorkerResponse) GetError() string { return "" } +func (x *StartWorkerResponse) SetError(v string) { + x.Error = v +} + +type StartWorkerResponse_builder struct { + _ [0]func() // Prevents comparability and use of unkeyed literals for the builder. + + Error string +} + +func (b0 StartWorkerResponse_builder) Build() *StartWorkerResponse { + m0 := &StartWorkerResponse{} + b, x := &b0, m0 + _, _ = b, x + x.Error = b.Error + return m0 +} + type StopWorkerRequest struct { - state protoimpl.MessageState - sizeCache protoimpl.SizeCache + state protoimpl.MessageState `protogen:"hybrid.v1"` + WorkerId string `protobuf:"bytes,1,opt,name=worker_id,json=workerId,proto3" json:"worker_id,omitempty"` unknownFields protoimpl.UnknownFields - - WorkerId string `protobuf:"bytes,1,opt,name=worker_id,json=workerId,proto3" json:"worker_id,omitempty"` + sizeCache protoimpl.SizeCache } func (x *StopWorkerRequest) Reset() { *x = StopWorkerRequest{} - if protoimpl.UnsafeEnabled { - mi := &file_org_apache_beam_model_fn_execution_v1_beam_fn_api_proto_msgTypes[40] - ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) - ms.StoreMessageInfo(mi) - } + mi := &file_org_apache_beam_model_fn_execution_v1_beam_fn_api_proto_msgTypes[40] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) } func (x *StopWorkerRequest) String() string { @@ -3291,7 +5489,7 @@ func (*StopWorkerRequest) ProtoMessage() {} func (x *StopWorkerRequest) ProtoReflect() protoreflect.Message { mi := &file_org_apache_beam_model_fn_execution_v1_beam_fn_api_proto_msgTypes[40] - if protoimpl.UnsafeEnabled && x != nil { + if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { ms.StoreMessageInfo(mi) @@ -3301,11 +5499,6 @@ func (x *StopWorkerRequest) ProtoReflect() protoreflect.Message { return mi.MessageOf(x) } -// Deprecated: Use StopWorkerRequest.ProtoReflect.Descriptor instead. -func (*StopWorkerRequest) Descriptor() ([]byte, []int) { - return file_org_apache_beam_model_fn_execution_v1_beam_fn_api_proto_rawDescGZIP(), []int{40} -} - func (x *StopWorkerRequest) GetWorkerId() string { if x != nil { return x.WorkerId @@ -3313,21 +5506,36 @@ func (x *StopWorkerRequest) GetWorkerId() string { return "" } +func (x *StopWorkerRequest) SetWorkerId(v string) { + x.WorkerId = v +} + +type StopWorkerRequest_builder struct { + _ [0]func() // Prevents comparability and use of unkeyed literals for the builder. + + WorkerId string +} + +func (b0 StopWorkerRequest_builder) Build() *StopWorkerRequest { + m0 := &StopWorkerRequest{} + b, x := &b0, m0 + _, _ = b, x + x.WorkerId = b.WorkerId + return m0 +} + type StopWorkerResponse struct { - state protoimpl.MessageState - sizeCache protoimpl.SizeCache + state protoimpl.MessageState `protogen:"hybrid.v1"` + Error string `protobuf:"bytes,1,opt,name=error,proto3" json:"error,omitempty"` unknownFields protoimpl.UnknownFields - - Error string `protobuf:"bytes,1,opt,name=error,proto3" json:"error,omitempty"` + sizeCache protoimpl.SizeCache } func (x *StopWorkerResponse) Reset() { *x = StopWorkerResponse{} - if protoimpl.UnsafeEnabled { - mi := &file_org_apache_beam_model_fn_execution_v1_beam_fn_api_proto_msgTypes[41] - ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) - ms.StoreMessageInfo(mi) - } + mi := &file_org_apache_beam_model_fn_execution_v1_beam_fn_api_proto_msgTypes[41] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) } func (x *StopWorkerResponse) String() string { @@ -3338,7 +5546,7 @@ func (*StopWorkerResponse) ProtoMessage() {} func (x *StopWorkerResponse) ProtoReflect() protoreflect.Message { mi := &file_org_apache_beam_model_fn_execution_v1_beam_fn_api_proto_msgTypes[41] - if protoimpl.UnsafeEnabled && x != nil { + if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { ms.StoreMessageInfo(mi) @@ -3348,11 +5556,6 @@ func (x *StopWorkerResponse) ProtoReflect() protoreflect.Message { return mi.MessageOf(x) } -// Deprecated: Use StopWorkerResponse.ProtoReflect.Descriptor instead. -func (*StopWorkerResponse) Descriptor() ([]byte, []int) { - return file_org_apache_beam_model_fn_execution_v1_beam_fn_api_proto_rawDescGZIP(), []int{41} -} - func (x *StopWorkerResponse) GetError() string { if x != nil { return x.Error @@ -3360,24 +5563,39 @@ func (x *StopWorkerResponse) GetError() string { return "" } +func (x *StopWorkerResponse) SetError(v string) { + x.Error = v +} + +type StopWorkerResponse_builder struct { + _ [0]func() // Prevents comparability and use of unkeyed literals for the builder. + + Error string +} + +func (b0 StopWorkerResponse_builder) Build() *StopWorkerResponse { + m0 := &StopWorkerResponse{} + b, x := &b0, m0 + _, _ = b, x + x.Error = b.Error + return m0 +} + // Request from runner to SDK Harness asking for its status. For more details see // https://s.apache.org/beam-fn-api-harness-status type WorkerStatusRequest struct { - state protoimpl.MessageState - sizeCache protoimpl.SizeCache - unknownFields protoimpl.UnknownFields - + state protoimpl.MessageState `protogen:"hybrid.v1"` // (Required) Unique ID identifying this request. - Id string `protobuf:"bytes,1,opt,name=id,proto3" json:"id,omitempty"` + Id string `protobuf:"bytes,1,opt,name=id,proto3" json:"id,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache } func (x *WorkerStatusRequest) Reset() { *x = WorkerStatusRequest{} - if protoimpl.UnsafeEnabled { - mi := &file_org_apache_beam_model_fn_execution_v1_beam_fn_api_proto_msgTypes[42] - ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) - ms.StoreMessageInfo(mi) - } + mi := &file_org_apache_beam_model_fn_execution_v1_beam_fn_api_proto_msgTypes[42] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) } func (x *WorkerStatusRequest) String() string { @@ -3388,7 +5606,7 @@ func (*WorkerStatusRequest) ProtoMessage() {} func (x *WorkerStatusRequest) ProtoReflect() protoreflect.Message { mi := &file_org_apache_beam_model_fn_execution_v1_beam_fn_api_proto_msgTypes[42] - if protoimpl.UnsafeEnabled && x != nil { + if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { ms.StoreMessageInfo(mi) @@ -3398,11 +5616,6 @@ func (x *WorkerStatusRequest) ProtoReflect() protoreflect.Message { return mi.MessageOf(x) } -// Deprecated: Use WorkerStatusRequest.ProtoReflect.Descriptor instead. -func (*WorkerStatusRequest) Descriptor() ([]byte, []int) { - return file_org_apache_beam_model_fn_execution_v1_beam_fn_api_proto_rawDescGZIP(), []int{42} -} - func (x *WorkerStatusRequest) GetId() string { if x != nil { return x.Id @@ -3410,12 +5623,28 @@ func (x *WorkerStatusRequest) GetId() string { return "" } +func (x *WorkerStatusRequest) SetId(v string) { + x.Id = v +} + +type WorkerStatusRequest_builder struct { + _ [0]func() // Prevents comparability and use of unkeyed literals for the builder. + + // (Required) Unique ID identifying this request. + Id string +} + +func (b0 WorkerStatusRequest_builder) Build() *WorkerStatusRequest { + m0 := &WorkerStatusRequest{} + b, x := &b0, m0 + _, _ = b, x + x.Id = b.Id + return m0 +} + // Response from SDK Harness to runner containing the debug related status info. type WorkerStatusResponse struct { - state protoimpl.MessageState - sizeCache protoimpl.SizeCache - unknownFields protoimpl.UnknownFields - + state protoimpl.MessageState `protogen:"hybrid.v1"` // (Required) Unique ID from the original request. Id string `protobuf:"bytes,1,opt,name=id,proto3" json:"id,omitempty"` // (Optional) Error message if exception encountered generating the status response. @@ -3425,16 +5654,16 @@ type WorkerStatusResponse struct { // appropriate as an HTTP response body for end user. For details of the preferred // info to include in the message see // https://s.apache.org/beam-fn-api-harness-status - StatusInfo string `protobuf:"bytes,3,opt,name=status_info,json=statusInfo,proto3" json:"status_info,omitempty"` + StatusInfo string `protobuf:"bytes,3,opt,name=status_info,json=statusInfo,proto3" json:"status_info,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache } func (x *WorkerStatusResponse) Reset() { *x = WorkerStatusResponse{} - if protoimpl.UnsafeEnabled { - mi := &file_org_apache_beam_model_fn_execution_v1_beam_fn_api_proto_msgTypes[43] - ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) - ms.StoreMessageInfo(mi) - } + mi := &file_org_apache_beam_model_fn_execution_v1_beam_fn_api_proto_msgTypes[43] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) } func (x *WorkerStatusResponse) String() string { @@ -3445,7 +5674,7 @@ func (*WorkerStatusResponse) ProtoMessage() {} func (x *WorkerStatusResponse) ProtoReflect() protoreflect.Message { mi := &file_org_apache_beam_model_fn_execution_v1_beam_fn_api_proto_msgTypes[43] - if protoimpl.UnsafeEnabled && x != nil { + if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { ms.StoreMessageInfo(mi) @@ -3455,11 +5684,6 @@ func (x *WorkerStatusResponse) ProtoReflect() protoreflect.Message { return mi.MessageOf(x) } -// Deprecated: Use WorkerStatusResponse.ProtoReflect.Descriptor instead. -func (*WorkerStatusResponse) Descriptor() ([]byte, []int) { - return file_org_apache_beam_model_fn_execution_v1_beam_fn_api_proto_rawDescGZIP(), []int{43} -} - func (x *WorkerStatusResponse) GetId() string { if x != nil { return x.Id @@ -3481,11 +5705,45 @@ func (x *WorkerStatusResponse) GetStatusInfo() string { return "" } -type SampledElement_Exception struct { - state protoimpl.MessageState - sizeCache protoimpl.SizeCache - unknownFields protoimpl.UnknownFields +func (x *WorkerStatusResponse) SetId(v string) { + x.Id = v +} +func (x *WorkerStatusResponse) SetError(v string) { + x.Error = v +} + +func (x *WorkerStatusResponse) SetStatusInfo(v string) { + x.StatusInfo = v +} + +type WorkerStatusResponse_builder struct { + _ [0]func() // Prevents comparability and use of unkeyed literals for the builder. + + // (Required) Unique ID from the original request. + Id string + // (Optional) Error message if exception encountered generating the status response. + Error string + // (Optional) Status debugging info reported by SDK harness worker. Content and + // format is not strongly enforced but should be print-friendly and + // appropriate as an HTTP response body for end user. For details of the preferred + // info to include in the message see + // https://s.apache.org/beam-fn-api-harness-status + StatusInfo string +} + +func (b0 WorkerStatusResponse_builder) Build() *WorkerStatusResponse { + m0 := &WorkerStatusResponse{} + b, x := &b0, m0 + _, _ = b, x + x.Id = b.Id + x.Error = b.Error + x.StatusInfo = b.StatusInfo + return m0 +} + +type SampledElement_Exception struct { + state protoimpl.MessageState `protogen:"hybrid.v1"` // (Required) The instruction ID of the associated ProcessBundleRequest. InstructionId string `protobuf:"bytes,1,opt,name=instruction_id,json=instructionId,proto3" json:"instruction_id,omitempty"` // (Required) The transform ID of the executing PTransform during the @@ -3493,16 +5751,16 @@ type SampledElement_Exception struct { TransformId string `protobuf:"bytes,2,opt,name=transform_id,json=transformId,proto3" json:"transform_id,omitempty"` // (Required) The error message to be displayed to the user. Can use the // other fields to query for contextual logs. - Error string `protobuf:"bytes,3,opt,name=error,proto3" json:"error,omitempty"` + Error string `protobuf:"bytes,3,opt,name=error,proto3" json:"error,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache } func (x *SampledElement_Exception) Reset() { *x = SampledElement_Exception{} - if protoimpl.UnsafeEnabled { - mi := &file_org_apache_beam_model_fn_execution_v1_beam_fn_api_proto_msgTypes[44] - ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) - ms.StoreMessageInfo(mi) - } + mi := &file_org_apache_beam_model_fn_execution_v1_beam_fn_api_proto_msgTypes[44] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) } func (x *SampledElement_Exception) String() string { @@ -3513,7 +5771,7 @@ func (*SampledElement_Exception) ProtoMessage() {} func (x *SampledElement_Exception) ProtoReflect() protoreflect.Message { mi := &file_org_apache_beam_model_fn_execution_v1_beam_fn_api_proto_msgTypes[44] - if protoimpl.UnsafeEnabled && x != nil { + if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { ms.StoreMessageInfo(mi) @@ -3523,11 +5781,6 @@ func (x *SampledElement_Exception) ProtoReflect() protoreflect.Message { return mi.MessageOf(x) } -// Deprecated: Use SampledElement_Exception.ProtoReflect.Descriptor instead. -func (*SampledElement_Exception) Descriptor() ([]byte, []int) { - return file_org_apache_beam_model_fn_execution_v1_beam_fn_api_proto_rawDescGZIP(), []int{6, 0} -} - func (x *SampledElement_Exception) GetInstructionId() string { if x != nil { return x.InstructionId @@ -3549,22 +5802,54 @@ func (x *SampledElement_Exception) GetError() string { return "" } -type SampleDataResponse_ElementList struct { - state protoimpl.MessageState - sizeCache protoimpl.SizeCache - unknownFields protoimpl.UnknownFields +func (x *SampledElement_Exception) SetInstructionId(v string) { + x.InstructionId = v +} + +func (x *SampledElement_Exception) SetTransformId(v string) { + x.TransformId = v +} + +func (x *SampledElement_Exception) SetError(v string) { + x.Error = v +} + +type SampledElement_Exception_builder struct { + _ [0]func() // Prevents comparability and use of unkeyed literals for the builder. + + // (Required) The instruction ID of the associated ProcessBundleRequest. + InstructionId string + // (Required) The transform ID of the executing PTransform during the + // exception. + TransformId string + // (Required) The error message to be displayed to the user. Can use the + // other fields to query for contextual logs. + Error string +} + +func (b0 SampledElement_Exception_builder) Build() *SampledElement_Exception { + m0 := &SampledElement_Exception{} + b, x := &b0, m0 + _, _ = b, x + x.InstructionId = b.InstructionId + x.TransformId = b.TransformId + x.Error = b.Error + return m0 +} +type SampleDataResponse_ElementList struct { + state protoimpl.MessageState `protogen:"hybrid.v1"` // Required. The individual elements sampled from a PCollection. - Elements []*SampledElement `protobuf:"bytes,1,rep,name=elements,proto3" json:"elements,omitempty"` + Elements []*SampledElement `protobuf:"bytes,1,rep,name=elements,proto3" json:"elements,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache } func (x *SampleDataResponse_ElementList) Reset() { *x = SampleDataResponse_ElementList{} - if protoimpl.UnsafeEnabled { - mi := &file_org_apache_beam_model_fn_execution_v1_beam_fn_api_proto_msgTypes[45] - ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) - ms.StoreMessageInfo(mi) - } + mi := &file_org_apache_beam_model_fn_execution_v1_beam_fn_api_proto_msgTypes[45] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) } func (x *SampleDataResponse_ElementList) String() string { @@ -3575,7 +5860,7 @@ func (*SampleDataResponse_ElementList) ProtoMessage() {} func (x *SampleDataResponse_ElementList) ProtoReflect() protoreflect.Message { mi := &file_org_apache_beam_model_fn_execution_v1_beam_fn_api_proto_msgTypes[45] - if protoimpl.UnsafeEnabled && x != nil { + if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { ms.StoreMessageInfo(mi) @@ -3585,11 +5870,6 @@ func (x *SampleDataResponse_ElementList) ProtoReflect() protoreflect.Message { return mi.MessageOf(x) } -// Deprecated: Use SampleDataResponse_ElementList.ProtoReflect.Descriptor instead. -func (*SampleDataResponse_ElementList) Descriptor() ([]byte, []int) { - return file_org_apache_beam_model_fn_execution_v1_beam_fn_api_proto_rawDescGZIP(), []int{7, 0} -} - func (x *SampleDataResponse_ElementList) GetElements() []*SampledElement { if x != nil { return x.Elements @@ -3597,35 +5877,51 @@ func (x *SampleDataResponse_ElementList) GetElements() []*SampledElement { return nil } +func (x *SampleDataResponse_ElementList) SetElements(v []*SampledElement) { + x.Elements = v +} + +type SampleDataResponse_ElementList_builder struct { + _ [0]func() // Prevents comparability and use of unkeyed literals for the builder. + + // Required. The individual elements sampled from a PCollection. + Elements []*SampledElement +} + +func (b0 SampleDataResponse_ElementList_builder) Build() *SampleDataResponse_ElementList { + m0 := &SampleDataResponse_ElementList{} + b, x := &b0, m0 + _, _ = b, x + x.Elements = b.Elements + return m0 +} + // Contains the cache token and also defines the scope of what the token applies to. // // See https://s.apache.org/beam-fn-state-api-and-bundle-processing#heading=h.7ghoih5aig5m // for additional details on how to use the cache token with the State API // to cache data across bundle boundaries. type ProcessBundleRequest_CacheToken struct { - state protoimpl.MessageState - sizeCache protoimpl.SizeCache - unknownFields protoimpl.UnknownFields - + state protoimpl.MessageState `protogen:"hybrid.v1"` // The scope of a cache token. // - // Types that are assignable to Type: + // Types that are valid to be assigned to Type: // // *ProcessBundleRequest_CacheToken_UserState_ // *ProcessBundleRequest_CacheToken_SideInput_ Type isProcessBundleRequest_CacheToken_Type `protobuf_oneof:"type"` // An opaque token used with the StateKey to create a globally unique // identifier. - Token []byte `protobuf:"bytes,10,opt,name=token,proto3" json:"token,omitempty"` + Token []byte `protobuf:"bytes,10,opt,name=token,proto3" json:"token,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache } func (x *ProcessBundleRequest_CacheToken) Reset() { *x = ProcessBundleRequest_CacheToken{} - if protoimpl.UnsafeEnabled { - mi := &file_org_apache_beam_model_fn_execution_v1_beam_fn_api_proto_msgTypes[54] - ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) - ms.StoreMessageInfo(mi) - } + mi := &file_org_apache_beam_model_fn_execution_v1_beam_fn_api_proto_msgTypes[54] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) } func (x *ProcessBundleRequest_CacheToken) String() string { @@ -3636,7 +5932,7 @@ func (*ProcessBundleRequest_CacheToken) ProtoMessage() {} func (x *ProcessBundleRequest_CacheToken) ProtoReflect() protoreflect.Message { mi := &file_org_apache_beam_model_fn_execution_v1_beam_fn_api_proto_msgTypes[54] - if protoimpl.UnsafeEnabled && x != nil { + if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { ms.StoreMessageInfo(mi) @@ -3646,28 +5942,27 @@ func (x *ProcessBundleRequest_CacheToken) ProtoReflect() protoreflect.Message { return mi.MessageOf(x) } -// Deprecated: Use ProcessBundleRequest_CacheToken.ProtoReflect.Descriptor instead. -func (*ProcessBundleRequest_CacheToken) Descriptor() ([]byte, []int) { - return file_org_apache_beam_model_fn_execution_v1_beam_fn_api_proto_rawDescGZIP(), []int{15, 0} -} - -func (m *ProcessBundleRequest_CacheToken) GetType() isProcessBundleRequest_CacheToken_Type { - if m != nil { - return m.Type +func (x *ProcessBundleRequest_CacheToken) GetType() isProcessBundleRequest_CacheToken_Type { + if x != nil { + return x.Type } return nil } func (x *ProcessBundleRequest_CacheToken) GetUserState() *ProcessBundleRequest_CacheToken_UserState { - if x, ok := x.GetType().(*ProcessBundleRequest_CacheToken_UserState_); ok { - return x.UserState + if x != nil { + if x, ok := x.Type.(*ProcessBundleRequest_CacheToken_UserState_); ok { + return x.UserState + } } return nil } func (x *ProcessBundleRequest_CacheToken) GetSideInput() *ProcessBundleRequest_CacheToken_SideInput { - if x, ok := x.GetType().(*ProcessBundleRequest_CacheToken_SideInput_); ok { - return x.SideInput + if x != nil { + if x, ok := x.Type.(*ProcessBundleRequest_CacheToken_SideInput_); ok { + return x.SideInput + } } return nil } @@ -3679,6 +5974,124 @@ func (x *ProcessBundleRequest_CacheToken) GetToken() []byte { return nil } +func (x *ProcessBundleRequest_CacheToken) SetUserState(v *ProcessBundleRequest_CacheToken_UserState) { + if v == nil { + x.Type = nil + return + } + x.Type = &ProcessBundleRequest_CacheToken_UserState_{v} +} + +func (x *ProcessBundleRequest_CacheToken) SetSideInput(v *ProcessBundleRequest_CacheToken_SideInput) { + if v == nil { + x.Type = nil + return + } + x.Type = &ProcessBundleRequest_CacheToken_SideInput_{v} +} + +func (x *ProcessBundleRequest_CacheToken) SetToken(v []byte) { + if v == nil { + v = []byte{} + } + x.Token = v +} + +func (x *ProcessBundleRequest_CacheToken) HasType() bool { + if x == nil { + return false + } + return x.Type != nil +} + +func (x *ProcessBundleRequest_CacheToken) HasUserState() bool { + if x == nil { + return false + } + _, ok := x.Type.(*ProcessBundleRequest_CacheToken_UserState_) + return ok +} + +func (x *ProcessBundleRequest_CacheToken) HasSideInput() bool { + if x == nil { + return false + } + _, ok := x.Type.(*ProcessBundleRequest_CacheToken_SideInput_) + return ok +} + +func (x *ProcessBundleRequest_CacheToken) ClearType() { + x.Type = nil +} + +func (x *ProcessBundleRequest_CacheToken) ClearUserState() { + if _, ok := x.Type.(*ProcessBundleRequest_CacheToken_UserState_); ok { + x.Type = nil + } +} + +func (x *ProcessBundleRequest_CacheToken) ClearSideInput() { + if _, ok := x.Type.(*ProcessBundleRequest_CacheToken_SideInput_); ok { + x.Type = nil + } +} + +const ProcessBundleRequest_CacheToken_Type_not_set_case case_ProcessBundleRequest_CacheToken_Type = 0 +const ProcessBundleRequest_CacheToken_UserState_case case_ProcessBundleRequest_CacheToken_Type = 1 +const ProcessBundleRequest_CacheToken_SideInput_case case_ProcessBundleRequest_CacheToken_Type = 2 + +func (x *ProcessBundleRequest_CacheToken) WhichType() case_ProcessBundleRequest_CacheToken_Type { + if x == nil { + return ProcessBundleRequest_CacheToken_Type_not_set_case + } + switch x.Type.(type) { + case *ProcessBundleRequest_CacheToken_UserState_: + return ProcessBundleRequest_CacheToken_UserState_case + case *ProcessBundleRequest_CacheToken_SideInput_: + return ProcessBundleRequest_CacheToken_SideInput_case + default: + return ProcessBundleRequest_CacheToken_Type_not_set_case + } +} + +type ProcessBundleRequest_CacheToken_builder struct { + _ [0]func() // Prevents comparability and use of unkeyed literals for the builder. + + // The scope of a cache token. + + // Fields of oneof Type: + UserState *ProcessBundleRequest_CacheToken_UserState + SideInput *ProcessBundleRequest_CacheToken_SideInput + // -- end of Type + // An opaque token used with the StateKey to create a globally unique + // identifier. + Token []byte +} + +func (b0 ProcessBundleRequest_CacheToken_builder) Build() *ProcessBundleRequest_CacheToken { + m0 := &ProcessBundleRequest_CacheToken{} + b, x := &b0, m0 + _, _ = b, x + if b.UserState != nil { + x.Type = &ProcessBundleRequest_CacheToken_UserState_{b.UserState} + } + if b.SideInput != nil { + x.Type = &ProcessBundleRequest_CacheToken_SideInput_{b.SideInput} + } + x.Token = b.Token + return m0 +} + +type case_ProcessBundleRequest_CacheToken_Type protoreflect.FieldNumber + +func (x case_ProcessBundleRequest_CacheToken_Type) String() string { + md := file_org_apache_beam_model_fn_execution_v1_beam_fn_api_proto_msgTypes[54].Descriptor() + if x == 0 { + return "not set" + } + return protoimpl.X.MessageFieldStringOf(md, protoreflect.FieldNumber(x)) +} + type isProcessBundleRequest_CacheToken_Type interface { isProcessBundleRequest_CacheToken_Type() } @@ -3697,18 +6110,16 @@ func (*ProcessBundleRequest_CacheToken_SideInput_) isProcessBundleRequest_CacheT // A flag to indicate a cache token is valid for all user state. type ProcessBundleRequest_CacheToken_UserState struct { - state protoimpl.MessageState - sizeCache protoimpl.SizeCache + state protoimpl.MessageState `protogen:"hybrid.v1"` unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache } func (x *ProcessBundleRequest_CacheToken_UserState) Reset() { *x = ProcessBundleRequest_CacheToken_UserState{} - if protoimpl.UnsafeEnabled { - mi := &file_org_apache_beam_model_fn_execution_v1_beam_fn_api_proto_msgTypes[55] - ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) - ms.StoreMessageInfo(mi) - } + mi := &file_org_apache_beam_model_fn_execution_v1_beam_fn_api_proto_msgTypes[55] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) } func (x *ProcessBundleRequest_CacheToken_UserState) String() string { @@ -3719,7 +6130,7 @@ func (*ProcessBundleRequest_CacheToken_UserState) ProtoMessage() {} func (x *ProcessBundleRequest_CacheToken_UserState) ProtoReflect() protoreflect.Message { mi := &file_org_apache_beam_model_fn_execution_v1_beam_fn_api_proto_msgTypes[55] - if protoimpl.UnsafeEnabled && x != nil { + if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { ms.StoreMessageInfo(mi) @@ -3729,30 +6140,34 @@ func (x *ProcessBundleRequest_CacheToken_UserState) ProtoReflect() protoreflect. return mi.MessageOf(x) } -// Deprecated: Use ProcessBundleRequest_CacheToken_UserState.ProtoReflect.Descriptor instead. -func (*ProcessBundleRequest_CacheToken_UserState) Descriptor() ([]byte, []int) { - return file_org_apache_beam_model_fn_execution_v1_beam_fn_api_proto_rawDescGZIP(), []int{15, 0, 0} +type ProcessBundleRequest_CacheToken_UserState_builder struct { + _ [0]func() // Prevents comparability and use of unkeyed literals for the builder. + +} + +func (b0 ProcessBundleRequest_CacheToken_UserState_builder) Build() *ProcessBundleRequest_CacheToken_UserState { + m0 := &ProcessBundleRequest_CacheToken_UserState{} + b, x := &b0, m0 + _, _ = b, x + return m0 } // A flag to indicate a cache token is valid for a side input. type ProcessBundleRequest_CacheToken_SideInput struct { - state protoimpl.MessageState - sizeCache protoimpl.SizeCache - unknownFields protoimpl.UnknownFields - + state protoimpl.MessageState `protogen:"hybrid.v1"` // (Required) The id of the PTransform containing a side input. TransformId string `protobuf:"bytes,1,opt,name=transform_id,json=transformId,proto3" json:"transform_id,omitempty"` // (Required) The id of the side input. - SideInputId string `protobuf:"bytes,2,opt,name=side_input_id,json=sideInputId,proto3" json:"side_input_id,omitempty"` + SideInputId string `protobuf:"bytes,2,opt,name=side_input_id,json=sideInputId,proto3" json:"side_input_id,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache } func (x *ProcessBundleRequest_CacheToken_SideInput) Reset() { *x = ProcessBundleRequest_CacheToken_SideInput{} - if protoimpl.UnsafeEnabled { - mi := &file_org_apache_beam_model_fn_execution_v1_beam_fn_api_proto_msgTypes[56] - ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) - ms.StoreMessageInfo(mi) - } + mi := &file_org_apache_beam_model_fn_execution_v1_beam_fn_api_proto_msgTypes[56] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) } func (x *ProcessBundleRequest_CacheToken_SideInput) String() string { @@ -3763,7 +6178,7 @@ func (*ProcessBundleRequest_CacheToken_SideInput) ProtoMessage() {} func (x *ProcessBundleRequest_CacheToken_SideInput) ProtoReflect() protoreflect.Message { mi := &file_org_apache_beam_model_fn_execution_v1_beam_fn_api_proto_msgTypes[56] - if protoimpl.UnsafeEnabled && x != nil { + if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { ms.StoreMessageInfo(mi) @@ -3773,11 +6188,6 @@ func (x *ProcessBundleRequest_CacheToken_SideInput) ProtoReflect() protoreflect. return mi.MessageOf(x) } -// Deprecated: Use ProcessBundleRequest_CacheToken_SideInput.ProtoReflect.Descriptor instead. -func (*ProcessBundleRequest_CacheToken_SideInput) Descriptor() ([]byte, []int) { - return file_org_apache_beam_model_fn_execution_v1_beam_fn_api_proto_rawDescGZIP(), []int{15, 0, 1} -} - func (x *ProcessBundleRequest_CacheToken_SideInput) GetTransformId() string { if x != nil { return x.TransformId @@ -3792,12 +6202,35 @@ func (x *ProcessBundleRequest_CacheToken_SideInput) GetSideInputId() string { return "" } -// A message specifying the desired split for a single transform. -type ProcessBundleSplitRequest_DesiredSplit struct { - state protoimpl.MessageState - sizeCache protoimpl.SizeCache - unknownFields protoimpl.UnknownFields +func (x *ProcessBundleRequest_CacheToken_SideInput) SetTransformId(v string) { + x.TransformId = v +} + +func (x *ProcessBundleRequest_CacheToken_SideInput) SetSideInputId(v string) { + x.SideInputId = v +} +type ProcessBundleRequest_CacheToken_SideInput_builder struct { + _ [0]func() // Prevents comparability and use of unkeyed literals for the builder. + + // (Required) The id of the PTransform containing a side input. + TransformId string + // (Required) The id of the side input. + SideInputId string +} + +func (b0 ProcessBundleRequest_CacheToken_SideInput_builder) Build() *ProcessBundleRequest_CacheToken_SideInput { + m0 := &ProcessBundleRequest_CacheToken_SideInput{} + b, x := &b0, m0 + _, _ = b, x + x.TransformId = b.TransformId + x.SideInputId = b.SideInputId + return m0 +} + +// A message specifying the desired split for a single transform. +type ProcessBundleSplitRequest_DesiredSplit struct { + state protoimpl.MessageState `protogen:"hybrid.v1"` // (Required) The fraction of known work remaining in this bundle // for this transform that should be kept by the SDK after this split. // @@ -3811,15 +6244,15 @@ type ProcessBundleSplitRequest_DesiredSplit struct { // expected to be sent to this GrpcRead operation, required to correctly // account for unreceived data when determining where to split. EstimatedInputElements int64 `protobuf:"varint,2,opt,name=estimated_input_elements,json=estimatedInputElements,proto3" json:"estimated_input_elements,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache } func (x *ProcessBundleSplitRequest_DesiredSplit) Reset() { *x = ProcessBundleSplitRequest_DesiredSplit{} - if protoimpl.UnsafeEnabled { - mi := &file_org_apache_beam_model_fn_execution_v1_beam_fn_api_proto_msgTypes[60] - ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) - ms.StoreMessageInfo(mi) - } + mi := &file_org_apache_beam_model_fn_execution_v1_beam_fn_api_proto_msgTypes[60] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) } func (x *ProcessBundleSplitRequest_DesiredSplit) String() string { @@ -3830,7 +6263,7 @@ func (*ProcessBundleSplitRequest_DesiredSplit) ProtoMessage() {} func (x *ProcessBundleSplitRequest_DesiredSplit) ProtoReflect() protoreflect.Message { mi := &file_org_apache_beam_model_fn_execution_v1_beam_fn_api_proto_msgTypes[60] - if protoimpl.UnsafeEnabled && x != nil { + if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { ms.StoreMessageInfo(mi) @@ -3840,11 +6273,6 @@ func (x *ProcessBundleSplitRequest_DesiredSplit) ProtoReflect() protoreflect.Mes return mi.MessageOf(x) } -// Deprecated: Use ProcessBundleSplitRequest_DesiredSplit.ProtoReflect.Descriptor instead. -func (*ProcessBundleSplitRequest_DesiredSplit) Descriptor() ([]byte, []int) { - return file_org_apache_beam_model_fn_execution_v1_beam_fn_api_proto_rawDescGZIP(), []int{21, 0} -} - func (x *ProcessBundleSplitRequest_DesiredSplit) GetFractionOfRemainder() float64 { if x != nil { return x.FractionOfRemainder @@ -3866,6 +6294,46 @@ func (x *ProcessBundleSplitRequest_DesiredSplit) GetEstimatedInputElements() int return 0 } +func (x *ProcessBundleSplitRequest_DesiredSplit) SetFractionOfRemainder(v float64) { + x.FractionOfRemainder = v +} + +func (x *ProcessBundleSplitRequest_DesiredSplit) SetAllowedSplitPoints(v []int64) { + x.AllowedSplitPoints = v +} + +func (x *ProcessBundleSplitRequest_DesiredSplit) SetEstimatedInputElements(v int64) { + x.EstimatedInputElements = v +} + +type ProcessBundleSplitRequest_DesiredSplit_builder struct { + _ [0]func() // Prevents comparability and use of unkeyed literals for the builder. + + // (Required) The fraction of known work remaining in this bundle + // for this transform that should be kept by the SDK after this split. + // + // Set to 0 to "checkpoint" as soon as possible (keeping as little work as + // possible and returning the remainder). + FractionOfRemainder float64 + // (Optional) A set of allowed element indices where the SDK may split. When + // this is empty, there are no constraints on where to split. + AllowedSplitPoints []int64 + // (Required for gRPC Read operation transforms) Number of total elements + // expected to be sent to this GrpcRead operation, required to correctly + // account for unreceived data when determining where to split. + EstimatedInputElements int64 +} + +func (b0 ProcessBundleSplitRequest_DesiredSplit_builder) Build() *ProcessBundleSplitRequest_DesiredSplit { + m0 := &ProcessBundleSplitRequest_DesiredSplit{} + b, x := &b0, m0 + _, _ = b, x + x.FractionOfRemainder = b.FractionOfRemainder + x.AllowedSplitPoints = b.AllowedSplitPoints + x.EstimatedInputElements = b.EstimatedInputElements + return m0 +} + // Represents contiguous portions of the data channel that are either // entirely processed or entirely unprocessed and belong to the primary // or residual respectively. @@ -3890,10 +6358,7 @@ func (x *ProcessBundleSplitRequest_DesiredSplit) GetEstimatedInputElements() int // - the first_residual_element does not decrease if there were residual // or primary roots returned in a prior split. type ProcessBundleSplitResponse_ChannelSplit struct { - state protoimpl.MessageState - sizeCache protoimpl.SizeCache - unknownFields protoimpl.UnknownFields - + state protoimpl.MessageState `protogen:"hybrid.v1"` // (Required) The grpc read transform reading this channel. TransformId string `protobuf:"bytes,1,opt,name=transform_id,json=transformId,proto3" json:"transform_id,omitempty"` // (Required) The last element of the input channel that should be entirely @@ -3904,15 +6369,15 @@ type ProcessBundleSplitResponse_ChannelSplit struct { // considered part of the residual, identified by its absolute zero-based // index in the (ordered) channel. FirstResidualElement int64 `protobuf:"varint,3,opt,name=first_residual_element,json=firstResidualElement,proto3" json:"first_residual_element,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache } func (x *ProcessBundleSplitResponse_ChannelSplit) Reset() { *x = ProcessBundleSplitResponse_ChannelSplit{} - if protoimpl.UnsafeEnabled { - mi := &file_org_apache_beam_model_fn_execution_v1_beam_fn_api_proto_msgTypes[62] - ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) - ms.StoreMessageInfo(mi) - } + mi := &file_org_apache_beam_model_fn_execution_v1_beam_fn_api_proto_msgTypes[62] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) } func (x *ProcessBundleSplitResponse_ChannelSplit) String() string { @@ -3923,7 +6388,7 @@ func (*ProcessBundleSplitResponse_ChannelSplit) ProtoMessage() {} func (x *ProcessBundleSplitResponse_ChannelSplit) ProtoReflect() protoreflect.Message { mi := &file_org_apache_beam_model_fn_execution_v1_beam_fn_api_proto_msgTypes[62] - if protoimpl.UnsafeEnabled && x != nil { + if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { ms.StoreMessageInfo(mi) @@ -3933,11 +6398,6 @@ func (x *ProcessBundleSplitResponse_ChannelSplit) ProtoReflect() protoreflect.Me return mi.MessageOf(x) } -// Deprecated: Use ProcessBundleSplitResponse_ChannelSplit.ProtoReflect.Descriptor instead. -func (*ProcessBundleSplitResponse_ChannelSplit) Descriptor() ([]byte, []int) { - return file_org_apache_beam_model_fn_execution_v1_beam_fn_api_proto_rawDescGZIP(), []int{22, 0} -} - func (x *ProcessBundleSplitResponse_ChannelSplit) GetTransformId() string { if x != nil { return x.TransformId @@ -3959,13 +6419,47 @@ func (x *ProcessBundleSplitResponse_ChannelSplit) GetFirstResidualElement() int6 return 0 } +func (x *ProcessBundleSplitResponse_ChannelSplit) SetTransformId(v string) { + x.TransformId = v +} + +func (x *ProcessBundleSplitResponse_ChannelSplit) SetLastPrimaryElement(v int64) { + x.LastPrimaryElement = v +} + +func (x *ProcessBundleSplitResponse_ChannelSplit) SetFirstResidualElement(v int64) { + x.FirstResidualElement = v +} + +type ProcessBundleSplitResponse_ChannelSplit_builder struct { + _ [0]func() // Prevents comparability and use of unkeyed literals for the builder. + + // (Required) The grpc read transform reading this channel. + TransformId string + // (Required) The last element of the input channel that should be entirely + // considered part of the primary, identified by its absolute zero-based + // index in the (ordered) channel. + LastPrimaryElement int64 + // (Required) The first element of the input channel that should be entirely + // considered part of the residual, identified by its absolute zero-based + // index in the (ordered) channel. + FirstResidualElement int64 +} + +func (b0 ProcessBundleSplitResponse_ChannelSplit_builder) Build() *ProcessBundleSplitResponse_ChannelSplit { + m0 := &ProcessBundleSplitResponse_ChannelSplit{} + b, x := &b0, m0 + _, _ = b, x + x.TransformId = b.TransformId + x.LastPrimaryElement = b.LastPrimaryElement + x.FirstResidualElement = b.FirstResidualElement + return m0 +} + // Represents multiple encoded elements in nested context for a given named // instruction and transform. type Elements_Data struct { - state protoimpl.MessageState - sizeCache protoimpl.SizeCache - unknownFields protoimpl.UnknownFields - + state protoimpl.MessageState `protogen:"hybrid.v1"` // (Required) A reference to an active instruction request with the given // instruction id. InstructionId string `protobuf:"bytes,1,opt,name=instruction_id,json=instructionId,proto3" json:"instruction_id,omitempty"` @@ -3984,16 +6478,16 @@ type Elements_Data struct { Data []byte `protobuf:"bytes,3,opt,name=data,proto3" json:"data,omitempty"` // (Optional) Set this bit to indicate the this is the last data block // for the given instruction and transform, ending the stream. - IsLast bool `protobuf:"varint,4,opt,name=is_last,json=isLast,proto3" json:"is_last,omitempty"` + IsLast bool `protobuf:"varint,4,opt,name=is_last,json=isLast,proto3" json:"is_last,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache } func (x *Elements_Data) Reset() { *x = Elements_Data{} - if protoimpl.UnsafeEnabled { - mi := &file_org_apache_beam_model_fn_execution_v1_beam_fn_api_proto_msgTypes[63] - ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) - ms.StoreMessageInfo(mi) - } + mi := &file_org_apache_beam_model_fn_execution_v1_beam_fn_api_proto_msgTypes[63] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) } func (x *Elements_Data) String() string { @@ -4004,7 +6498,7 @@ func (*Elements_Data) ProtoMessage() {} func (x *Elements_Data) ProtoReflect() protoreflect.Message { mi := &file_org_apache_beam_model_fn_execution_v1_beam_fn_api_proto_msgTypes[63] - if protoimpl.UnsafeEnabled && x != nil { + if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { ms.StoreMessageInfo(mi) @@ -4014,11 +6508,6 @@ func (x *Elements_Data) ProtoReflect() protoreflect.Message { return mi.MessageOf(x) } -// Deprecated: Use Elements_Data.ProtoReflect.Descriptor instead. -func (*Elements_Data) Descriptor() ([]byte, []int) { - return file_org_apache_beam_model_fn_execution_v1_beam_fn_api_proto_rawDescGZIP(), []int{25, 0} -} - func (x *Elements_Data) GetInstructionId() string { if x != nil { return x.InstructionId @@ -4047,13 +6536,64 @@ func (x *Elements_Data) GetIsLast() bool { return false } +func (x *Elements_Data) SetInstructionId(v string) { + x.InstructionId = v +} + +func (x *Elements_Data) SetTransformId(v string) { + x.TransformId = v +} + +func (x *Elements_Data) SetData(v []byte) { + if v == nil { + v = []byte{} + } + x.Data = v +} + +func (x *Elements_Data) SetIsLast(v bool) { + x.IsLast = v +} + +type Elements_Data_builder struct { + _ [0]func() // Prevents comparability and use of unkeyed literals for the builder. + + // (Required) A reference to an active instruction request with the given + // instruction id. + InstructionId string + // (Required) A definition representing a consumer or producer of this data. + // If received by a harness, this represents the consumer within that + // harness that should consume these bytes. If sent by a harness, this + // represents the producer of these bytes. + // + // Note that a single element may span multiple Data messages. + // + // Note that a sending/receiving pair should share the same identifier. + TransformId string + // (Optional) Represents a part of a logical byte stream. Elements within + // the logical byte stream are encoded in the nested context and + // concatenated together. + Data []byte + // (Optional) Set this bit to indicate the this is the last data block + // for the given instruction and transform, ending the stream. + IsLast bool +} + +func (b0 Elements_Data_builder) Build() *Elements_Data { + m0 := &Elements_Data{} + b, x := &b0, m0 + _, _ = b, x + x.InstructionId = b.InstructionId + x.TransformId = b.TransformId + x.Data = b.Data + x.IsLast = b.IsLast + return m0 +} + // Represent the encoded user timer for a given instruction, transform and // timer id. type Elements_Timers struct { - state protoimpl.MessageState - sizeCache protoimpl.SizeCache - unknownFields protoimpl.UnknownFields - + state protoimpl.MessageState `protogen:"hybrid.v1"` // (Required) A reference to an active instruction request with the given // instruction id. InstructionId string `protobuf:"bytes,1,opt,name=instruction_id,json=instructionId,proto3" json:"instruction_id,omitempty"` @@ -4070,16 +6610,16 @@ type Elements_Timers struct { Timers []byte `protobuf:"bytes,4,opt,name=timers,proto3" json:"timers,omitempty"` // (Optional) Set this bit to indicate the this is the last data block // for the given instruction and transform, ending the stream. - IsLast bool `protobuf:"varint,5,opt,name=is_last,json=isLast,proto3" json:"is_last,omitempty"` + IsLast bool `protobuf:"varint,5,opt,name=is_last,json=isLast,proto3" json:"is_last,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache } func (x *Elements_Timers) Reset() { *x = Elements_Timers{} - if protoimpl.UnsafeEnabled { - mi := &file_org_apache_beam_model_fn_execution_v1_beam_fn_api_proto_msgTypes[64] - ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) - ms.StoreMessageInfo(mi) - } + mi := &file_org_apache_beam_model_fn_execution_v1_beam_fn_api_proto_msgTypes[64] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) } func (x *Elements_Timers) String() string { @@ -4090,7 +6630,7 @@ func (*Elements_Timers) ProtoMessage() {} func (x *Elements_Timers) ProtoReflect() protoreflect.Message { mi := &file_org_apache_beam_model_fn_execution_v1_beam_fn_api_proto_msgTypes[64] - if protoimpl.UnsafeEnabled && x != nil { + if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { ms.StoreMessageInfo(mi) @@ -4100,11 +6640,6 @@ func (x *Elements_Timers) ProtoReflect() protoreflect.Message { return mi.MessageOf(x) } -// Deprecated: Use Elements_Timers.ProtoReflect.Descriptor instead. -func (*Elements_Timers) Descriptor() ([]byte, []int) { - return file_org_apache_beam_model_fn_execution_v1_beam_fn_api_proto_rawDescGZIP(), []int{25, 1} -} - func (x *Elements_Timers) GetInstructionId() string { if x != nil { return x.InstructionId @@ -4140,11 +6675,65 @@ func (x *Elements_Timers) GetIsLast() bool { return false } -type StateKey_Runner struct { - state protoimpl.MessageState - sizeCache protoimpl.SizeCache - unknownFields protoimpl.UnknownFields +func (x *Elements_Timers) SetInstructionId(v string) { + x.InstructionId = v +} + +func (x *Elements_Timers) SetTransformId(v string) { + x.TransformId = v +} + +func (x *Elements_Timers) SetTimerFamilyId(v string) { + x.TimerFamilyId = v +} + +func (x *Elements_Timers) SetTimers(v []byte) { + if v == nil { + v = []byte{} + } + x.Timers = v +} + +func (x *Elements_Timers) SetIsLast(v bool) { + x.IsLast = v +} + +type Elements_Timers_builder struct { + _ [0]func() // Prevents comparability and use of unkeyed literals for the builder. + + // (Required) A reference to an active instruction request with the given + // instruction id. + InstructionId string + // (Required) A definition representing a consumer or producer of this data. + // If received by a harness, this represents the consumer within that + // harness that should consume these timers. If sent by a harness, this + // represents the producer of these timers. + TransformId string + // (Required) The local timer family name used to identify the associated + // timer family specification + TimerFamilyId string + // (Optional) Represents a logical byte stream of timers. Encoded according + // to the coder in the timer spec. + Timers []byte + // (Optional) Set this bit to indicate the this is the last data block + // for the given instruction and transform, ending the stream. + IsLast bool +} + +func (b0 Elements_Timers_builder) Build() *Elements_Timers { + m0 := &Elements_Timers{} + b, x := &b0, m0 + _, _ = b, x + x.InstructionId = b.InstructionId + x.TransformId = b.TransformId + x.TimerFamilyId = b.TimerFamilyId + x.Timers = b.Timers + x.IsLast = b.IsLast + return m0 +} +type StateKey_Runner struct { + state protoimpl.MessageState `protogen:"hybrid.v1"` // (Required) Opaque information supplied by the runner. Used to support // remote references. // https://s.apache.org/beam-fn-api-send-and-receive-data @@ -4152,16 +6741,16 @@ type StateKey_Runner struct { // Used by state backed iterable. And in this use case, request type can // only be of type get. Details see: // https://s.apache.org/beam-fn-api-state-backed-iterables - Key []byte `protobuf:"bytes,1,opt,name=key,proto3" json:"key,omitempty"` + Key []byte `protobuf:"bytes,1,opt,name=key,proto3" json:"key,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache } func (x *StateKey_Runner) Reset() { *x = StateKey_Runner{} - if protoimpl.UnsafeEnabled { - mi := &file_org_apache_beam_model_fn_execution_v1_beam_fn_api_proto_msgTypes[65] - ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) - ms.StoreMessageInfo(mi) - } + mi := &file_org_apache_beam_model_fn_execution_v1_beam_fn_api_proto_msgTypes[65] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) } func (x *StateKey_Runner) String() string { @@ -4172,7 +6761,7 @@ func (*StateKey_Runner) ProtoMessage() {} func (x *StateKey_Runner) ProtoReflect() protoreflect.Message { mi := &file_org_apache_beam_model_fn_execution_v1_beam_fn_api_proto_msgTypes[65] - if protoimpl.UnsafeEnabled && x != nil { + if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { ms.StoreMessageInfo(mi) @@ -4182,11 +6771,6 @@ func (x *StateKey_Runner) ProtoReflect() protoreflect.Message { return mi.MessageOf(x) } -// Deprecated: Use StateKey_Runner.ProtoReflect.Descriptor instead. -func (*StateKey_Runner) Descriptor() ([]byte, []int) { - return file_org_apache_beam_model_fn_execution_v1_beam_fn_api_proto_rawDescGZIP(), []int{28, 0} -} - func (x *StateKey_Runner) GetKey() []byte { if x != nil { return x.Key @@ -4194,6 +6778,34 @@ func (x *StateKey_Runner) GetKey() []byte { return nil } +func (x *StateKey_Runner) SetKey(v []byte) { + if v == nil { + v = []byte{} + } + x.Key = v +} + +type StateKey_Runner_builder struct { + _ [0]func() // Prevents comparability and use of unkeyed literals for the builder. + + // (Required) Opaque information supplied by the runner. Used to support + // remote references. + // https://s.apache.org/beam-fn-api-send-and-receive-data + // + // Used by state backed iterable. And in this use case, request type can + // only be of type get. Details see: + // https://s.apache.org/beam-fn-api-state-backed-iterables + Key []byte +} + +func (b0 StateKey_Runner_builder) Build() *StateKey_Runner { + m0 := &StateKey_Runner{} + b, x := &b0, m0 + _, _ = b, x + x.Key = b.Key + return m0 +} + // Represents a request for the values associated with a specified window // in a PCollection. See // https://s.apache.org/beam-fn-state-api-and-bundle-processing for further @@ -4206,26 +6818,23 @@ func (x *StateKey_Runner) GetKey() []byte { // of all V's. See https://s.apache.org/beam-fn-api-send-and-receive-data // for further details. type StateKey_IterableSideInput struct { - state protoimpl.MessageState - sizeCache protoimpl.SizeCache - unknownFields protoimpl.UnknownFields - + state protoimpl.MessageState `protogen:"hybrid.v1"` // (Required) The id of the PTransform containing a side input. TransformId string `protobuf:"bytes,1,opt,name=transform_id,json=transformId,proto3" json:"transform_id,omitempty"` // (Required) The id of the side input. SideInputId string `protobuf:"bytes,2,opt,name=side_input_id,json=sideInputId,proto3" json:"side_input_id,omitempty"` // (Required) The window (after mapping the currently executing elements // window into the side input windows domain) encoded in a nested context. - Window []byte `protobuf:"bytes,3,opt,name=window,proto3" json:"window,omitempty"` + Window []byte `protobuf:"bytes,3,opt,name=window,proto3" json:"window,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache } func (x *StateKey_IterableSideInput) Reset() { *x = StateKey_IterableSideInput{} - if protoimpl.UnsafeEnabled { - mi := &file_org_apache_beam_model_fn_execution_v1_beam_fn_api_proto_msgTypes[66] - ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) - ms.StoreMessageInfo(mi) - } + mi := &file_org_apache_beam_model_fn_execution_v1_beam_fn_api_proto_msgTypes[66] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) } func (x *StateKey_IterableSideInput) String() string { @@ -4236,7 +6845,7 @@ func (*StateKey_IterableSideInput) ProtoMessage() {} func (x *StateKey_IterableSideInput) ProtoReflect() protoreflect.Message { mi := &file_org_apache_beam_model_fn_execution_v1_beam_fn_api_proto_msgTypes[66] - if protoimpl.UnsafeEnabled && x != nil { + if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { ms.StoreMessageInfo(mi) @@ -4246,11 +6855,6 @@ func (x *StateKey_IterableSideInput) ProtoReflect() protoreflect.Message { return mi.MessageOf(x) } -// Deprecated: Use StateKey_IterableSideInput.ProtoReflect.Descriptor instead. -func (*StateKey_IterableSideInput) Descriptor() ([]byte, []int) { - return file_org_apache_beam_model_fn_execution_v1_beam_fn_api_proto_rawDescGZIP(), []int{28, 1} -} - func (x *StateKey_IterableSideInput) GetTransformId() string { if x != nil { return x.TransformId @@ -4272,6 +6876,43 @@ func (x *StateKey_IterableSideInput) GetWindow() []byte { return nil } +func (x *StateKey_IterableSideInput) SetTransformId(v string) { + x.TransformId = v +} + +func (x *StateKey_IterableSideInput) SetSideInputId(v string) { + x.SideInputId = v +} + +func (x *StateKey_IterableSideInput) SetWindow(v []byte) { + if v == nil { + v = []byte{} + } + x.Window = v +} + +type StateKey_IterableSideInput_builder struct { + _ [0]func() // Prevents comparability and use of unkeyed literals for the builder. + + // (Required) The id of the PTransform containing a side input. + TransformId string + // (Required) The id of the side input. + SideInputId string + // (Required) The window (after mapping the currently executing elements + // window into the side input windows domain) encoded in a nested context. + Window []byte +} + +func (b0 StateKey_IterableSideInput_builder) Build() *StateKey_IterableSideInput { + m0 := &StateKey_IterableSideInput{} + b, x := &b0, m0 + _, _ = b, x + x.TransformId = b.TransformId + x.SideInputId = b.SideInputId + x.Window = b.Window + return m0 +} + // Represents a request for the values associated with a specified user key // and window in a PCollection. See // https://s.apache.org/beam-fn-state-api-and-bundle-processing for further @@ -4285,10 +6926,7 @@ func (x *StateKey_IterableSideInput) GetWindow() []byte { // https://s.apache.org/beam-fn-api-send-and-receive-data for further // details. type StateKey_MultimapSideInput struct { - state protoimpl.MessageState - sizeCache protoimpl.SizeCache - unknownFields protoimpl.UnknownFields - + state protoimpl.MessageState `protogen:"hybrid.v1"` // (Required) The id of the PTransform containing a side input. TransformId string `protobuf:"bytes,1,opt,name=transform_id,json=transformId,proto3" json:"transform_id,omitempty"` // (Required) The id of the side input. @@ -4297,16 +6935,16 @@ type StateKey_MultimapSideInput struct { // window into the side input windows domain) encoded in a nested context. Window []byte `protobuf:"bytes,3,opt,name=window,proto3" json:"window,omitempty"` // (Required) The key encoded in a nested context. - Key []byte `protobuf:"bytes,4,opt,name=key,proto3" json:"key,omitempty"` + Key []byte `protobuf:"bytes,4,opt,name=key,proto3" json:"key,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache } func (x *StateKey_MultimapSideInput) Reset() { *x = StateKey_MultimapSideInput{} - if protoimpl.UnsafeEnabled { - mi := &file_org_apache_beam_model_fn_execution_v1_beam_fn_api_proto_msgTypes[67] - ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) - ms.StoreMessageInfo(mi) - } + mi := &file_org_apache_beam_model_fn_execution_v1_beam_fn_api_proto_msgTypes[67] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) } func (x *StateKey_MultimapSideInput) String() string { @@ -4317,7 +6955,7 @@ func (*StateKey_MultimapSideInput) ProtoMessage() {} func (x *StateKey_MultimapSideInput) ProtoReflect() protoreflect.Message { mi := &file_org_apache_beam_model_fn_execution_v1_beam_fn_api_proto_msgTypes[67] - if protoimpl.UnsafeEnabled && x != nil { + if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { ms.StoreMessageInfo(mi) @@ -4327,11 +6965,6 @@ func (x *StateKey_MultimapSideInput) ProtoReflect() protoreflect.Message { return mi.MessageOf(x) } -// Deprecated: Use StateKey_MultimapSideInput.ProtoReflect.Descriptor instead. -func (*StateKey_MultimapSideInput) Descriptor() ([]byte, []int) { - return file_org_apache_beam_model_fn_execution_v1_beam_fn_api_proto_rawDescGZIP(), []int{28, 2} -} - func (x *StateKey_MultimapSideInput) GetTransformId() string { if x != nil { return x.TransformId @@ -4360,6 +6993,53 @@ func (x *StateKey_MultimapSideInput) GetKey() []byte { return nil } +func (x *StateKey_MultimapSideInput) SetTransformId(v string) { + x.TransformId = v +} + +func (x *StateKey_MultimapSideInput) SetSideInputId(v string) { + x.SideInputId = v +} + +func (x *StateKey_MultimapSideInput) SetWindow(v []byte) { + if v == nil { + v = []byte{} + } + x.Window = v +} + +func (x *StateKey_MultimapSideInput) SetKey(v []byte) { + if v == nil { + v = []byte{} + } + x.Key = v +} + +type StateKey_MultimapSideInput_builder struct { + _ [0]func() // Prevents comparability and use of unkeyed literals for the builder. + + // (Required) The id of the PTransform containing a side input. + TransformId string + // (Required) The id of the side input. + SideInputId string + // (Required) The window (after mapping the currently executing elements + // window into the side input windows domain) encoded in a nested context. + Window []byte + // (Required) The key encoded in a nested context. + Key []byte +} + +func (b0 StateKey_MultimapSideInput_builder) Build() *StateKey_MultimapSideInput { + m0 := &StateKey_MultimapSideInput{} + b, x := &b0, m0 + _, _ = b, x + x.TransformId = b.TransformId + x.SideInputId = b.SideInputId + x.Window = b.Window + x.Key = b.Key + return m0 +} + // Represents a request for the keys associated with a specified window in a PCollection. See // https://s.apache.org/beam-fn-state-api-and-bundle-processing for further // details. @@ -4372,26 +7052,23 @@ func (x *StateKey_MultimapSideInput) GetKey() []byte { // https://s.apache.org/beam-fn-api-send-and-receive-data for further // details. type StateKey_MultimapKeysSideInput struct { - state protoimpl.MessageState - sizeCache protoimpl.SizeCache - unknownFields protoimpl.UnknownFields - + state protoimpl.MessageState `protogen:"hybrid.v1"` // (Required) The id of the PTransform containing a side input. TransformId string `protobuf:"bytes,1,opt,name=transform_id,json=transformId,proto3" json:"transform_id,omitempty"` // (Required) The id of the side input. SideInputId string `protobuf:"bytes,2,opt,name=side_input_id,json=sideInputId,proto3" json:"side_input_id,omitempty"` // (Required) The window (after mapping the currently executing elements // window into the side input windows domain) encoded in a nested context. - Window []byte `protobuf:"bytes,3,opt,name=window,proto3" json:"window,omitempty"` + Window []byte `protobuf:"bytes,3,opt,name=window,proto3" json:"window,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache } func (x *StateKey_MultimapKeysSideInput) Reset() { *x = StateKey_MultimapKeysSideInput{} - if protoimpl.UnsafeEnabled { - mi := &file_org_apache_beam_model_fn_execution_v1_beam_fn_api_proto_msgTypes[68] - ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) - ms.StoreMessageInfo(mi) - } + mi := &file_org_apache_beam_model_fn_execution_v1_beam_fn_api_proto_msgTypes[68] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) } func (x *StateKey_MultimapKeysSideInput) String() string { @@ -4402,7 +7079,7 @@ func (*StateKey_MultimapKeysSideInput) ProtoMessage() {} func (x *StateKey_MultimapKeysSideInput) ProtoReflect() protoreflect.Message { mi := &file_org_apache_beam_model_fn_execution_v1_beam_fn_api_proto_msgTypes[68] - if protoimpl.UnsafeEnabled && x != nil { + if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { ms.StoreMessageInfo(mi) @@ -4412,11 +7089,6 @@ func (x *StateKey_MultimapKeysSideInput) ProtoReflect() protoreflect.Message { return mi.MessageOf(x) } -// Deprecated: Use StateKey_MultimapKeysSideInput.ProtoReflect.Descriptor instead. -func (*StateKey_MultimapKeysSideInput) Descriptor() ([]byte, []int) { - return file_org_apache_beam_model_fn_execution_v1_beam_fn_api_proto_rawDescGZIP(), []int{28, 3} -} - func (x *StateKey_MultimapKeysSideInput) GetTransformId() string { if x != nil { return x.TransformId @@ -4438,6 +7110,43 @@ func (x *StateKey_MultimapKeysSideInput) GetWindow() []byte { return nil } +func (x *StateKey_MultimapKeysSideInput) SetTransformId(v string) { + x.TransformId = v +} + +func (x *StateKey_MultimapKeysSideInput) SetSideInputId(v string) { + x.SideInputId = v +} + +func (x *StateKey_MultimapKeysSideInput) SetWindow(v []byte) { + if v == nil { + v = []byte{} + } + x.Window = v +} + +type StateKey_MultimapKeysSideInput_builder struct { + _ [0]func() // Prevents comparability and use of unkeyed literals for the builder. + + // (Required) The id of the PTransform containing a side input. + TransformId string + // (Required) The id of the side input. + SideInputId string + // (Required) The window (after mapping the currently executing elements + // window into the side input windows domain) encoded in a nested context. + Window []byte +} + +func (b0 StateKey_MultimapKeysSideInput_builder) Build() *StateKey_MultimapKeysSideInput { + m0 := &StateKey_MultimapKeysSideInput{} + b, x := &b0, m0 + _, _ = b, x + x.TransformId = b.TransformId + x.SideInputId = b.SideInputId + x.Window = b.Window + return m0 +} + // Represents a request for the keys and values associated with a specified window in a PCollection. See // https://s.apache.org/beam-fn-state-api-and-bundle-processing for further // details. @@ -4458,26 +7167,23 @@ func (x *StateKey_MultimapKeysSideInput) GetWindow() []byte { // https://s.apache.org/beam-fn-api-send-and-receive-data for further // details. type StateKey_MultimapKeysValuesSideInput struct { - state protoimpl.MessageState - sizeCache protoimpl.SizeCache - unknownFields protoimpl.UnknownFields - + state protoimpl.MessageState `protogen:"hybrid.v1"` // (Required) The id of the PTransform containing a side input. TransformId string `protobuf:"bytes,1,opt,name=transform_id,json=transformId,proto3" json:"transform_id,omitempty"` // (Required) The id of the side input. SideInputId string `protobuf:"bytes,2,opt,name=side_input_id,json=sideInputId,proto3" json:"side_input_id,omitempty"` // (Required) The window (after mapping the currently executing elements // window into the side input windows domain) encoded in a nested context. - Window []byte `protobuf:"bytes,3,opt,name=window,proto3" json:"window,omitempty"` + Window []byte `protobuf:"bytes,3,opt,name=window,proto3" json:"window,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache } func (x *StateKey_MultimapKeysValuesSideInput) Reset() { *x = StateKey_MultimapKeysValuesSideInput{} - if protoimpl.UnsafeEnabled { - mi := &file_org_apache_beam_model_fn_execution_v1_beam_fn_api_proto_msgTypes[69] - ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) - ms.StoreMessageInfo(mi) - } + mi := &file_org_apache_beam_model_fn_execution_v1_beam_fn_api_proto_msgTypes[69] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) } func (x *StateKey_MultimapKeysValuesSideInput) String() string { @@ -4488,7 +7194,7 @@ func (*StateKey_MultimapKeysValuesSideInput) ProtoMessage() {} func (x *StateKey_MultimapKeysValuesSideInput) ProtoReflect() protoreflect.Message { mi := &file_org_apache_beam_model_fn_execution_v1_beam_fn_api_proto_msgTypes[69] - if protoimpl.UnsafeEnabled && x != nil { + if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { ms.StoreMessageInfo(mi) @@ -4498,11 +7204,6 @@ func (x *StateKey_MultimapKeysValuesSideInput) ProtoReflect() protoreflect.Messa return mi.MessageOf(x) } -// Deprecated: Use StateKey_MultimapKeysValuesSideInput.ProtoReflect.Descriptor instead. -func (*StateKey_MultimapKeysValuesSideInput) Descriptor() ([]byte, []int) { - return file_org_apache_beam_model_fn_execution_v1_beam_fn_api_proto_rawDescGZIP(), []int{28, 4} -} - func (x *StateKey_MultimapKeysValuesSideInput) GetTransformId() string { if x != nil { return x.TransformId @@ -4524,6 +7225,43 @@ func (x *StateKey_MultimapKeysValuesSideInput) GetWindow() []byte { return nil } +func (x *StateKey_MultimapKeysValuesSideInput) SetTransformId(v string) { + x.TransformId = v +} + +func (x *StateKey_MultimapKeysValuesSideInput) SetSideInputId(v string) { + x.SideInputId = v +} + +func (x *StateKey_MultimapKeysValuesSideInput) SetWindow(v []byte) { + if v == nil { + v = []byte{} + } + x.Window = v +} + +type StateKey_MultimapKeysValuesSideInput_builder struct { + _ [0]func() // Prevents comparability and use of unkeyed literals for the builder. + + // (Required) The id of the PTransform containing a side input. + TransformId string + // (Required) The id of the side input. + SideInputId string + // (Required) The window (after mapping the currently executing elements + // window into the side input windows domain) encoded in a nested context. + Window []byte +} + +func (b0 StateKey_MultimapKeysValuesSideInput_builder) Build() *StateKey_MultimapKeysValuesSideInput { + m0 := &StateKey_MultimapKeysValuesSideInput{} + b, x := &b0, m0 + _, _ = b, x + x.TransformId = b.TransformId + x.SideInputId = b.SideInputId + x.Window = b.Window + return m0 +} + // Represents a request for an unordered set of values associated with a // specified user key and window for a PTransform. See // https://s.apache.org/beam-fn-state-api-and-bundle-processing for further @@ -4534,10 +7272,7 @@ func (x *StateKey_MultimapKeysValuesSideInput) GetWindow() []byte { // See https://s.apache.org/beam-fn-api-send-and-receive-data for further // details. type StateKey_BagUserState struct { - state protoimpl.MessageState - sizeCache protoimpl.SizeCache - unknownFields protoimpl.UnknownFields - + state protoimpl.MessageState `protogen:"hybrid.v1"` // (Required) The id of the PTransform containing user state. TransformId string `protobuf:"bytes,1,opt,name=transform_id,json=transformId,proto3" json:"transform_id,omitempty"` // (Required) The id of the user state. @@ -4546,16 +7281,16 @@ type StateKey_BagUserState struct { Window []byte `protobuf:"bytes,3,opt,name=window,proto3" json:"window,omitempty"` // (Required) The key of the currently executing element encoded in a // nested context. - Key []byte `protobuf:"bytes,4,opt,name=key,proto3" json:"key,omitempty"` + Key []byte `protobuf:"bytes,4,opt,name=key,proto3" json:"key,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache } func (x *StateKey_BagUserState) Reset() { *x = StateKey_BagUserState{} - if protoimpl.UnsafeEnabled { - mi := &file_org_apache_beam_model_fn_execution_v1_beam_fn_api_proto_msgTypes[70] - ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) - ms.StoreMessageInfo(mi) - } + mi := &file_org_apache_beam_model_fn_execution_v1_beam_fn_api_proto_msgTypes[70] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) } func (x *StateKey_BagUserState) String() string { @@ -4566,7 +7301,7 @@ func (*StateKey_BagUserState) ProtoMessage() {} func (x *StateKey_BagUserState) ProtoReflect() protoreflect.Message { mi := &file_org_apache_beam_model_fn_execution_v1_beam_fn_api_proto_msgTypes[70] - if protoimpl.UnsafeEnabled && x != nil { + if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { ms.StoreMessageInfo(mi) @@ -4576,11 +7311,6 @@ func (x *StateKey_BagUserState) ProtoReflect() protoreflect.Message { return mi.MessageOf(x) } -// Deprecated: Use StateKey_BagUserState.ProtoReflect.Descriptor instead. -func (*StateKey_BagUserState) Descriptor() ([]byte, []int) { - return file_org_apache_beam_model_fn_execution_v1_beam_fn_api_proto_rawDescGZIP(), []int{28, 5} -} - func (x *StateKey_BagUserState) GetTransformId() string { if x != nil { return x.TransformId @@ -4609,6 +7339,53 @@ func (x *StateKey_BagUserState) GetKey() []byte { return nil } +func (x *StateKey_BagUserState) SetTransformId(v string) { + x.TransformId = v +} + +func (x *StateKey_BagUserState) SetUserStateId(v string) { + x.UserStateId = v +} + +func (x *StateKey_BagUserState) SetWindow(v []byte) { + if v == nil { + v = []byte{} + } + x.Window = v +} + +func (x *StateKey_BagUserState) SetKey(v []byte) { + if v == nil { + v = []byte{} + } + x.Key = v +} + +type StateKey_BagUserState_builder struct { + _ [0]func() // Prevents comparability and use of unkeyed literals for the builder. + + // (Required) The id of the PTransform containing user state. + TransformId string + // (Required) The id of the user state. + UserStateId string + // (Required) The window encoded in a nested context. + Window []byte + // (Required) The key of the currently executing element encoded in a + // nested context. + Key []byte +} + +func (b0 StateKey_BagUserState_builder) Build() *StateKey_BagUserState { + m0 := &StateKey_BagUserState{} + b, x := &b0, m0 + _, _ = b, x + x.TransformId = b.TransformId + x.UserStateId = b.UserStateId + x.Window = b.Window + x.Key = b.Key + return m0 +} + // Represents a request for the keys of a multimap associated with a specified // user key and window for a PTransform. See // https://s.apache.org/beam-fn-state-api-and-bundle-processing for further @@ -4622,10 +7399,7 @@ func (x *StateKey_BagUserState) GetKey() []byte { // See https://s.apache.org/beam-fn-api-send-and-receive-data for further // details. type StateKey_MultimapKeysUserState struct { - state protoimpl.MessageState - sizeCache protoimpl.SizeCache - unknownFields protoimpl.UnknownFields - + state protoimpl.MessageState `protogen:"hybrid.v1"` // (Required) The id of the PTransform containing user state. TransformId string `protobuf:"bytes,1,opt,name=transform_id,json=transformId,proto3" json:"transform_id,omitempty"` // (Required) The id of the user state. @@ -4634,16 +7408,16 @@ type StateKey_MultimapKeysUserState struct { Window []byte `protobuf:"bytes,3,opt,name=window,proto3" json:"window,omitempty"` // (Required) The key of the currently executing element encoded in a // nested context. - Key []byte `protobuf:"bytes,4,opt,name=key,proto3" json:"key,omitempty"` + Key []byte `protobuf:"bytes,4,opt,name=key,proto3" json:"key,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache } func (x *StateKey_MultimapKeysUserState) Reset() { *x = StateKey_MultimapKeysUserState{} - if protoimpl.UnsafeEnabled { - mi := &file_org_apache_beam_model_fn_execution_v1_beam_fn_api_proto_msgTypes[71] - ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) - ms.StoreMessageInfo(mi) - } + mi := &file_org_apache_beam_model_fn_execution_v1_beam_fn_api_proto_msgTypes[71] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) } func (x *StateKey_MultimapKeysUserState) String() string { @@ -4654,7 +7428,7 @@ func (*StateKey_MultimapKeysUserState) ProtoMessage() {} func (x *StateKey_MultimapKeysUserState) ProtoReflect() protoreflect.Message { mi := &file_org_apache_beam_model_fn_execution_v1_beam_fn_api_proto_msgTypes[71] - if protoimpl.UnsafeEnabled && x != nil { + if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { ms.StoreMessageInfo(mi) @@ -4664,11 +7438,6 @@ func (x *StateKey_MultimapKeysUserState) ProtoReflect() protoreflect.Message { return mi.MessageOf(x) } -// Deprecated: Use StateKey_MultimapKeysUserState.ProtoReflect.Descriptor instead. -func (*StateKey_MultimapKeysUserState) Descriptor() ([]byte, []int) { - return file_org_apache_beam_model_fn_execution_v1_beam_fn_api_proto_rawDescGZIP(), []int{28, 6} -} - func (x *StateKey_MultimapKeysUserState) GetTransformId() string { if x != nil { return x.TransformId @@ -4697,6 +7466,53 @@ func (x *StateKey_MultimapKeysUserState) GetKey() []byte { return nil } +func (x *StateKey_MultimapKeysUserState) SetTransformId(v string) { + x.TransformId = v +} + +func (x *StateKey_MultimapKeysUserState) SetUserStateId(v string) { + x.UserStateId = v +} + +func (x *StateKey_MultimapKeysUserState) SetWindow(v []byte) { + if v == nil { + v = []byte{} + } + x.Window = v +} + +func (x *StateKey_MultimapKeysUserState) SetKey(v []byte) { + if v == nil { + v = []byte{} + } + x.Key = v +} + +type StateKey_MultimapKeysUserState_builder struct { + _ [0]func() // Prevents comparability and use of unkeyed literals for the builder. + + // (Required) The id of the PTransform containing user state. + TransformId string + // (Required) The id of the user state. + UserStateId string + // (Required) The window encoded in a nested context. + Window []byte + // (Required) The key of the currently executing element encoded in a + // nested context. + Key []byte +} + +func (b0 StateKey_MultimapKeysUserState_builder) Build() *StateKey_MultimapKeysUserState { + m0 := &StateKey_MultimapKeysUserState{} + b, x := &b0, m0 + _, _ = b, x + x.TransformId = b.TransformId + x.UserStateId = b.UserStateId + x.Window = b.Window + x.Key = b.Key + return m0 +} + // Represents a request for the values of the map key associated with a // specified user key and window for a PTransform. See // https://s.apache.org/beam-fn-state-api-and-bundle-processing for further @@ -4707,10 +7523,7 @@ func (x *StateKey_MultimapKeysUserState) GetKey() []byte { // See https://s.apache.org/beam-fn-api-send-and-receive-data for further // details. type StateKey_MultimapUserState struct { - state protoimpl.MessageState - sizeCache protoimpl.SizeCache - unknownFields protoimpl.UnknownFields - + state protoimpl.MessageState `protogen:"hybrid.v1"` // (Required) The id of the PTransform containing user state. TransformId string `protobuf:"bytes,1,opt,name=transform_id,json=transformId,proto3" json:"transform_id,omitempty"` // (Required) The id of the user state. @@ -4721,16 +7534,16 @@ type StateKey_MultimapUserState struct { // nested context. Key []byte `protobuf:"bytes,4,opt,name=key,proto3" json:"key,omitempty"` // (Required) The map key encoded in a nested context. - MapKey []byte `protobuf:"bytes,5,opt,name=map_key,json=mapKey,proto3" json:"map_key,omitempty"` + MapKey []byte `protobuf:"bytes,5,opt,name=map_key,json=mapKey,proto3" json:"map_key,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache } func (x *StateKey_MultimapUserState) Reset() { *x = StateKey_MultimapUserState{} - if protoimpl.UnsafeEnabled { - mi := &file_org_apache_beam_model_fn_execution_v1_beam_fn_api_proto_msgTypes[72] - ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) - ms.StoreMessageInfo(mi) - } + mi := &file_org_apache_beam_model_fn_execution_v1_beam_fn_api_proto_msgTypes[72] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) } func (x *StateKey_MultimapUserState) String() string { @@ -4741,7 +7554,7 @@ func (*StateKey_MultimapUserState) ProtoMessage() {} func (x *StateKey_MultimapUserState) ProtoReflect() protoreflect.Message { mi := &file_org_apache_beam_model_fn_execution_v1_beam_fn_api_proto_msgTypes[72] - if protoimpl.UnsafeEnabled && x != nil { + if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { ms.StoreMessageInfo(mi) @@ -4751,11 +7564,6 @@ func (x *StateKey_MultimapUserState) ProtoReflect() protoreflect.Message { return mi.MessageOf(x) } -// Deprecated: Use StateKey_MultimapUserState.ProtoReflect.Descriptor instead. -func (*StateKey_MultimapUserState) Descriptor() ([]byte, []int) { - return file_org_apache_beam_model_fn_execution_v1_beam_fn_api_proto_rawDescGZIP(), []int{28, 7} -} - func (x *StateKey_MultimapUserState) GetTransformId() string { if x != nil { return x.TransformId @@ -4791,6 +7599,63 @@ func (x *StateKey_MultimapUserState) GetMapKey() []byte { return nil } +func (x *StateKey_MultimapUserState) SetTransformId(v string) { + x.TransformId = v +} + +func (x *StateKey_MultimapUserState) SetUserStateId(v string) { + x.UserStateId = v +} + +func (x *StateKey_MultimapUserState) SetWindow(v []byte) { + if v == nil { + v = []byte{} + } + x.Window = v +} + +func (x *StateKey_MultimapUserState) SetKey(v []byte) { + if v == nil { + v = []byte{} + } + x.Key = v +} + +func (x *StateKey_MultimapUserState) SetMapKey(v []byte) { + if v == nil { + v = []byte{} + } + x.MapKey = v +} + +type StateKey_MultimapUserState_builder struct { + _ [0]func() // Prevents comparability and use of unkeyed literals for the builder. + + // (Required) The id of the PTransform containing user state. + TransformId string + // (Required) The id of the user state. + UserStateId string + // (Required) The window encoded in a nested context. + Window []byte + // (Required) The key of the currently executing element encoded in a + // nested context. + Key []byte + // (Required) The map key encoded in a nested context. + MapKey []byte +} + +func (b0 StateKey_MultimapUserState_builder) Build() *StateKey_MultimapUserState { + m0 := &StateKey_MultimapUserState{} + b, x := &b0, m0 + _, _ = b, x + x.TransformId = b.TransformId + x.UserStateId = b.UserStateId + x.Window = b.Window + x.Key = b.Key + x.MapKey = b.MapKey + return m0 +} + // Represents a request for an ordered list of values associated with a // specified user key and window for a PTransform. See // https://s.apache.org/beam-fn-state-api-and-bundle-processing for further @@ -4801,10 +7666,7 @@ func (x *StateKey_MultimapUserState) GetMapKey() []byte { // See https://s.apache.org/beam-fn-api-send-and-receive-data for further // details. type StateKey_OrderedListUserState struct { - state protoimpl.MessageState - sizeCache protoimpl.SizeCache - unknownFields protoimpl.UnknownFields - + state protoimpl.MessageState `protogen:"hybrid.v1"` // (Required) The id of the PTransform containing user state. TransformId string `protobuf:"bytes,1,opt,name=transform_id,json=transformId,proto3" json:"transform_id,omitempty"` // (Required) The id of the user state. @@ -4815,16 +7677,16 @@ type StateKey_OrderedListUserState struct { // nested context. Key []byte `protobuf:"bytes,4,opt,name=key,proto3" json:"key,omitempty"` // (Required) The sort range encoded in a nested context. - Range *OrderedListRange `protobuf:"bytes,5,opt,name=range,proto3" json:"range,omitempty"` + Range *OrderedListRange `protobuf:"bytes,5,opt,name=range,proto3" json:"range,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache } func (x *StateKey_OrderedListUserState) Reset() { *x = StateKey_OrderedListUserState{} - if protoimpl.UnsafeEnabled { - mi := &file_org_apache_beam_model_fn_execution_v1_beam_fn_api_proto_msgTypes[73] - ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) - ms.StoreMessageInfo(mi) - } + mi := &file_org_apache_beam_model_fn_execution_v1_beam_fn_api_proto_msgTypes[73] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) } func (x *StateKey_OrderedListUserState) String() string { @@ -4835,7 +7697,7 @@ func (*StateKey_OrderedListUserState) ProtoMessage() {} func (x *StateKey_OrderedListUserState) ProtoReflect() protoreflect.Message { mi := &file_org_apache_beam_model_fn_execution_v1_beam_fn_api_proto_msgTypes[73] - if protoimpl.UnsafeEnabled && x != nil { + if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { ms.StoreMessageInfo(mi) @@ -4845,11 +7707,6 @@ func (x *StateKey_OrderedListUserState) ProtoReflect() protoreflect.Message { return mi.MessageOf(x) } -// Deprecated: Use StateKey_OrderedListUserState.ProtoReflect.Descriptor instead. -func (*StateKey_OrderedListUserState) Descriptor() ([]byte, []int) { - return file_org_apache_beam_model_fn_execution_v1_beam_fn_api_proto_rawDescGZIP(), []int{28, 8} -} - func (x *StateKey_OrderedListUserState) GetTransformId() string { if x != nil { return x.TransformId @@ -4885,24 +7742,86 @@ func (x *StateKey_OrderedListUserState) GetRange() *OrderedListRange { return nil } +func (x *StateKey_OrderedListUserState) SetTransformId(v string) { + x.TransformId = v +} + +func (x *StateKey_OrderedListUserState) SetUserStateId(v string) { + x.UserStateId = v +} + +func (x *StateKey_OrderedListUserState) SetWindow(v []byte) { + if v == nil { + v = []byte{} + } + x.Window = v +} + +func (x *StateKey_OrderedListUserState) SetKey(v []byte) { + if v == nil { + v = []byte{} + } + x.Key = v +} + +func (x *StateKey_OrderedListUserState) SetRange(v *OrderedListRange) { + x.Range = v +} + +func (x *StateKey_OrderedListUserState) HasRange() bool { + if x == nil { + return false + } + return x.Range != nil +} + +func (x *StateKey_OrderedListUserState) ClearRange() { + x.Range = nil +} + +type StateKey_OrderedListUserState_builder struct { + _ [0]func() // Prevents comparability and use of unkeyed literals for the builder. + + // (Required) The id of the PTransform containing user state. + TransformId string + // (Required) The id of the user state. + UserStateId string + // (Required) The window encoded in a nested context. + Window []byte + // (Required) The key of the currently executing element encoded in a + // nested context. + Key []byte + // (Required) The sort range encoded in a nested context. + Range *OrderedListRange +} + +func (b0 StateKey_OrderedListUserState_builder) Build() *StateKey_OrderedListUserState { + m0 := &StateKey_OrderedListUserState{} + b, x := &b0, m0 + _, _ = b, x + x.TransformId = b.TransformId + x.UserStateId = b.UserStateId + x.Window = b.Window + x.Key = b.Key + x.Range = b.Range + return m0 +} + // A list of log entries, enables buffering and batching of multiple // log messages using the logging API. type LogEntry_List struct { - state protoimpl.MessageState - sizeCache protoimpl.SizeCache - unknownFields protoimpl.UnknownFields - + state protoimpl.MessageState `protogen:"hybrid.v1"` // (Required) One or or more log messages. - LogEntries []*LogEntry `protobuf:"bytes,1,rep,name=log_entries,json=logEntries,proto3" json:"log_entries,omitempty"` + LogEntries []*LogEntry `protobuf:"bytes,1,rep,name=log_entries,json=logEntries,proto3" json:"log_entries,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache } func (x *LogEntry_List) Reset() { *x = LogEntry_List{} - if protoimpl.UnsafeEnabled { - mi := &file_org_apache_beam_model_fn_execution_v1_beam_fn_api_proto_msgTypes[74] - ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) - ms.StoreMessageInfo(mi) - } + mi := &file_org_apache_beam_model_fn_execution_v1_beam_fn_api_proto_msgTypes[74] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) } func (x *LogEntry_List) String() string { @@ -4913,7 +7832,7 @@ func (*LogEntry_List) ProtoMessage() {} func (x *LogEntry_List) ProtoReflect() protoreflect.Message { mi := &file_org_apache_beam_model_fn_execution_v1_beam_fn_api_proto_msgTypes[74] - if protoimpl.UnsafeEnabled && x != nil { + if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { ms.StoreMessageInfo(mi) @@ -4923,11 +7842,6 @@ func (x *LogEntry_List) ProtoReflect() protoreflect.Message { return mi.MessageOf(x) } -// Deprecated: Use LogEntry_List.ProtoReflect.Descriptor instead. -func (*LogEntry_List) Descriptor() ([]byte, []int) { - return file_org_apache_beam_model_fn_execution_v1_beam_fn_api_proto_rawDescGZIP(), []int{36, 0} -} - func (x *LogEntry_List) GetLogEntries() []*LogEntry { if x != nil { return x.LogEntries @@ -4935,6 +7849,25 @@ func (x *LogEntry_List) GetLogEntries() []*LogEntry { return nil } +func (x *LogEntry_List) SetLogEntries(v []*LogEntry) { + x.LogEntries = v +} + +type LogEntry_List_builder struct { + _ [0]func() // Prevents comparability and use of unkeyed literals for the builder. + + // (Required) One or or more log messages. + LogEntries []*LogEntry +} + +func (b0 LogEntry_List_builder) Build() *LogEntry_List { + m0 := &LogEntry_List{} + b, x := &b0, m0 + _, _ = b, x + x.LogEntries = b.LogEntries + return m0 +} + // The severity of the event described in a log entry, expressed as one of the // severity levels listed below. For your reference, the levels are // assigned the listed numeric values. The effect of using numeric values @@ -4949,18 +7882,16 @@ func (x *LogEntry_List) GetLogEntries() []*LogEntry { // can provide filtering and searching across log types. Users of the API are // free not to use all severity levels in their log messages. type LogEntry_Severity struct { - state protoimpl.MessageState - sizeCache protoimpl.SizeCache + state protoimpl.MessageState `protogen:"hybrid.v1"` unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache } func (x *LogEntry_Severity) Reset() { *x = LogEntry_Severity{} - if protoimpl.UnsafeEnabled { - mi := &file_org_apache_beam_model_fn_execution_v1_beam_fn_api_proto_msgTypes[75] - ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) - ms.StoreMessageInfo(mi) - } + mi := &file_org_apache_beam_model_fn_execution_v1_beam_fn_api_proto_msgTypes[75] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) } func (x *LogEntry_Severity) String() string { @@ -4971,7 +7902,7 @@ func (*LogEntry_Severity) ProtoMessage() {} func (x *LogEntry_Severity) ProtoReflect() protoreflect.Message { mi := &file_org_apache_beam_model_fn_execution_v1_beam_fn_api_proto_msgTypes[75] - if protoimpl.UnsafeEnabled && x != nil { + if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { ms.StoreMessageInfo(mi) @@ -4981,9 +7912,16 @@ func (x *LogEntry_Severity) ProtoReflect() protoreflect.Message { return mi.MessageOf(x) } -// Deprecated: Use LogEntry_Severity.ProtoReflect.Descriptor instead. -func (*LogEntry_Severity) Descriptor() ([]byte, []int) { - return file_org_apache_beam_model_fn_execution_v1_beam_fn_api_proto_rawDescGZIP(), []int{36, 1} +type LogEntry_Severity_builder struct { + _ [0]func() // Prevents comparability and use of unkeyed literals for the builder. + +} + +func (b0 LogEntry_Severity_builder) Build() *LogEntry_Severity { + m0 := &LogEntry_Severity{} + b, x := &b0, m0 + _, _ = b, x + return m0 } var File_org_apache_beam_model_fn_execution_v1_beam_fn_api_proto protoreflect.FileDescriptor @@ -5976,21 +8914,9 @@ var file_org_apache_beam_model_fn_execution_v1_beam_fn_api_proto_rawDesc = []byt 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x76, 0x31, 0x62, 0x06, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x33, } -var ( - file_org_apache_beam_model_fn_execution_v1_beam_fn_api_proto_rawDescOnce sync.Once - file_org_apache_beam_model_fn_execution_v1_beam_fn_api_proto_rawDescData = file_org_apache_beam_model_fn_execution_v1_beam_fn_api_proto_rawDesc -) - -func file_org_apache_beam_model_fn_execution_v1_beam_fn_api_proto_rawDescGZIP() []byte { - file_org_apache_beam_model_fn_execution_v1_beam_fn_api_proto_rawDescOnce.Do(func() { - file_org_apache_beam_model_fn_execution_v1_beam_fn_api_proto_rawDescData = protoimpl.X.CompressGZIP(file_org_apache_beam_model_fn_execution_v1_beam_fn_api_proto_rawDescData) - }) - return file_org_apache_beam_model_fn_execution_v1_beam_fn_api_proto_rawDescData -} - var file_org_apache_beam_model_fn_execution_v1_beam_fn_api_proto_enumTypes = make([]protoimpl.EnumInfo, 2) var file_org_apache_beam_model_fn_execution_v1_beam_fn_api_proto_msgTypes = make([]protoimpl.MessageInfo, 77) -var file_org_apache_beam_model_fn_execution_v1_beam_fn_api_proto_goTypes = []interface{}{ +var file_org_apache_beam_model_fn_execution_v1_beam_fn_api_proto_goTypes = []any{ (FnApiTransforms_Runner)(0), // 0: org.apache.beam.model.fn_execution.v1.FnApiTransforms.Runner (LogEntry_Severity_Enum)(0), // 1: org.apache.beam.model.fn_execution.v1.LogEntry.Severity.Enum (*FnApiTransforms)(nil), // 2: org.apache.beam.model.fn_execution.v1.FnApiTransforms @@ -6197,777 +9123,7 @@ func file_org_apache_beam_model_fn_execution_v1_beam_fn_api_proto_init() { if File_org_apache_beam_model_fn_execution_v1_beam_fn_api_proto != nil { return } - if !protoimpl.UnsafeEnabled { - file_org_apache_beam_model_fn_execution_v1_beam_fn_api_proto_msgTypes[0].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*FnApiTransforms); i { - case 0: - return &v.state - case 1: - return &v.sizeCache - case 2: - return &v.unknownFields - default: - return nil - } - } - file_org_apache_beam_model_fn_execution_v1_beam_fn_api_proto_msgTypes[1].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*RemoteGrpcPort); i { - case 0: - return &v.state - case 1: - return &v.sizeCache - case 2: - return &v.unknownFields - default: - return nil - } - } - file_org_apache_beam_model_fn_execution_v1_beam_fn_api_proto_msgTypes[2].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*GetProcessBundleDescriptorRequest); i { - case 0: - return &v.state - case 1: - return &v.sizeCache - case 2: - return &v.unknownFields - default: - return nil - } - } - file_org_apache_beam_model_fn_execution_v1_beam_fn_api_proto_msgTypes[3].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*InstructionRequest); i { - case 0: - return &v.state - case 1: - return &v.sizeCache - case 2: - return &v.unknownFields - default: - return nil - } - } - file_org_apache_beam_model_fn_execution_v1_beam_fn_api_proto_msgTypes[4].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*InstructionResponse); i { - case 0: - return &v.state - case 1: - return &v.sizeCache - case 2: - return &v.unknownFields - default: - return nil - } - } - file_org_apache_beam_model_fn_execution_v1_beam_fn_api_proto_msgTypes[5].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*SampleDataRequest); i { - case 0: - return &v.state - case 1: - return &v.sizeCache - case 2: - return &v.unknownFields - default: - return nil - } - } - file_org_apache_beam_model_fn_execution_v1_beam_fn_api_proto_msgTypes[6].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*SampledElement); i { - case 0: - return &v.state - case 1: - return &v.sizeCache - case 2: - return &v.unknownFields - default: - return nil - } - } - file_org_apache_beam_model_fn_execution_v1_beam_fn_api_proto_msgTypes[7].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*SampleDataResponse); i { - case 0: - return &v.state - case 1: - return &v.sizeCache - case 2: - return &v.unknownFields - default: - return nil - } - } - file_org_apache_beam_model_fn_execution_v1_beam_fn_api_proto_msgTypes[8].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*HarnessMonitoringInfosRequest); i { - case 0: - return &v.state - case 1: - return &v.sizeCache - case 2: - return &v.unknownFields - default: - return nil - } - } - file_org_apache_beam_model_fn_execution_v1_beam_fn_api_proto_msgTypes[9].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*HarnessMonitoringInfosResponse); i { - case 0: - return &v.state - case 1: - return &v.sizeCache - case 2: - return &v.unknownFields - default: - return nil - } - } - file_org_apache_beam_model_fn_execution_v1_beam_fn_api_proto_msgTypes[10].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*RegisterRequest); i { - case 0: - return &v.state - case 1: - return &v.sizeCache - case 2: - return &v.unknownFields - default: - return nil - } - } - file_org_apache_beam_model_fn_execution_v1_beam_fn_api_proto_msgTypes[11].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*RegisterResponse); i { - case 0: - return &v.state - case 1: - return &v.sizeCache - case 2: - return &v.unknownFields - default: - return nil - } - } - file_org_apache_beam_model_fn_execution_v1_beam_fn_api_proto_msgTypes[12].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*ProcessBundleDescriptor); i { - case 0: - return &v.state - case 1: - return &v.sizeCache - case 2: - return &v.unknownFields - default: - return nil - } - } - file_org_apache_beam_model_fn_execution_v1_beam_fn_api_proto_msgTypes[13].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*BundleApplication); i { - case 0: - return &v.state - case 1: - return &v.sizeCache - case 2: - return &v.unknownFields - default: - return nil - } - } - file_org_apache_beam_model_fn_execution_v1_beam_fn_api_proto_msgTypes[14].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*DelayedBundleApplication); i { - case 0: - return &v.state - case 1: - return &v.sizeCache - case 2: - return &v.unknownFields - default: - return nil - } - } - file_org_apache_beam_model_fn_execution_v1_beam_fn_api_proto_msgTypes[15].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*ProcessBundleRequest); i { - case 0: - return &v.state - case 1: - return &v.sizeCache - case 2: - return &v.unknownFields - default: - return nil - } - } - file_org_apache_beam_model_fn_execution_v1_beam_fn_api_proto_msgTypes[16].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*ProcessBundleResponse); i { - case 0: - return &v.state - case 1: - return &v.sizeCache - case 2: - return &v.unknownFields - default: - return nil - } - } - file_org_apache_beam_model_fn_execution_v1_beam_fn_api_proto_msgTypes[17].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*ProcessBundleProgressRequest); i { - case 0: - return &v.state - case 1: - return &v.sizeCache - case 2: - return &v.unknownFields - default: - return nil - } - } - file_org_apache_beam_model_fn_execution_v1_beam_fn_api_proto_msgTypes[18].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*MonitoringInfosMetadataRequest); i { - case 0: - return &v.state - case 1: - return &v.sizeCache - case 2: - return &v.unknownFields - default: - return nil - } - } - file_org_apache_beam_model_fn_execution_v1_beam_fn_api_proto_msgTypes[19].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*ProcessBundleProgressResponse); i { - case 0: - return &v.state - case 1: - return &v.sizeCache - case 2: - return &v.unknownFields - default: - return nil - } - } - file_org_apache_beam_model_fn_execution_v1_beam_fn_api_proto_msgTypes[20].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*MonitoringInfosMetadataResponse); i { - case 0: - return &v.state - case 1: - return &v.sizeCache - case 2: - return &v.unknownFields - default: - return nil - } - } - file_org_apache_beam_model_fn_execution_v1_beam_fn_api_proto_msgTypes[21].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*ProcessBundleSplitRequest); i { - case 0: - return &v.state - case 1: - return &v.sizeCache - case 2: - return &v.unknownFields - default: - return nil - } - } - file_org_apache_beam_model_fn_execution_v1_beam_fn_api_proto_msgTypes[22].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*ProcessBundleSplitResponse); i { - case 0: - return &v.state - case 1: - return &v.sizeCache - case 2: - return &v.unknownFields - default: - return nil - } - } - file_org_apache_beam_model_fn_execution_v1_beam_fn_api_proto_msgTypes[23].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*FinalizeBundleRequest); i { - case 0: - return &v.state - case 1: - return &v.sizeCache - case 2: - return &v.unknownFields - default: - return nil - } - } - file_org_apache_beam_model_fn_execution_v1_beam_fn_api_proto_msgTypes[24].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*FinalizeBundleResponse); i { - case 0: - return &v.state - case 1: - return &v.sizeCache - case 2: - return &v.unknownFields - default: - return nil - } - } - file_org_apache_beam_model_fn_execution_v1_beam_fn_api_proto_msgTypes[25].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*Elements); i { - case 0: - return &v.state - case 1: - return &v.sizeCache - case 2: - return &v.unknownFields - default: - return nil - } - } - file_org_apache_beam_model_fn_execution_v1_beam_fn_api_proto_msgTypes[26].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*StateRequest); i { - case 0: - return &v.state - case 1: - return &v.sizeCache - case 2: - return &v.unknownFields - default: - return nil - } - } - file_org_apache_beam_model_fn_execution_v1_beam_fn_api_proto_msgTypes[27].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*StateResponse); i { - case 0: - return &v.state - case 1: - return &v.sizeCache - case 2: - return &v.unknownFields - default: - return nil - } - } - file_org_apache_beam_model_fn_execution_v1_beam_fn_api_proto_msgTypes[28].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*StateKey); i { - case 0: - return &v.state - case 1: - return &v.sizeCache - case 2: - return &v.unknownFields - default: - return nil - } - } - file_org_apache_beam_model_fn_execution_v1_beam_fn_api_proto_msgTypes[29].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*StateGetRequest); i { - case 0: - return &v.state - case 1: - return &v.sizeCache - case 2: - return &v.unknownFields - default: - return nil - } - } - file_org_apache_beam_model_fn_execution_v1_beam_fn_api_proto_msgTypes[30].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*StateGetResponse); i { - case 0: - return &v.state - case 1: - return &v.sizeCache - case 2: - return &v.unknownFields - default: - return nil - } - } - file_org_apache_beam_model_fn_execution_v1_beam_fn_api_proto_msgTypes[31].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*StateAppendRequest); i { - case 0: - return &v.state - case 1: - return &v.sizeCache - case 2: - return &v.unknownFields - default: - return nil - } - } - file_org_apache_beam_model_fn_execution_v1_beam_fn_api_proto_msgTypes[32].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*StateAppendResponse); i { - case 0: - return &v.state - case 1: - return &v.sizeCache - case 2: - return &v.unknownFields - default: - return nil - } - } - file_org_apache_beam_model_fn_execution_v1_beam_fn_api_proto_msgTypes[33].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*StateClearRequest); i { - case 0: - return &v.state - case 1: - return &v.sizeCache - case 2: - return &v.unknownFields - default: - return nil - } - } - file_org_apache_beam_model_fn_execution_v1_beam_fn_api_proto_msgTypes[34].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*StateClearResponse); i { - case 0: - return &v.state - case 1: - return &v.sizeCache - case 2: - return &v.unknownFields - default: - return nil - } - } - file_org_apache_beam_model_fn_execution_v1_beam_fn_api_proto_msgTypes[35].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*OrderedListRange); i { - case 0: - return &v.state - case 1: - return &v.sizeCache - case 2: - return &v.unknownFields - default: - return nil - } - } - file_org_apache_beam_model_fn_execution_v1_beam_fn_api_proto_msgTypes[36].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*LogEntry); i { - case 0: - return &v.state - case 1: - return &v.sizeCache - case 2: - return &v.unknownFields - default: - return nil - } - } - file_org_apache_beam_model_fn_execution_v1_beam_fn_api_proto_msgTypes[37].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*LogControl); i { - case 0: - return &v.state - case 1: - return &v.sizeCache - case 2: - return &v.unknownFields - default: - return nil - } - } - file_org_apache_beam_model_fn_execution_v1_beam_fn_api_proto_msgTypes[38].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*StartWorkerRequest); i { - case 0: - return &v.state - case 1: - return &v.sizeCache - case 2: - return &v.unknownFields - default: - return nil - } - } - file_org_apache_beam_model_fn_execution_v1_beam_fn_api_proto_msgTypes[39].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*StartWorkerResponse); i { - case 0: - return &v.state - case 1: - return &v.sizeCache - case 2: - return &v.unknownFields - default: - return nil - } - } - file_org_apache_beam_model_fn_execution_v1_beam_fn_api_proto_msgTypes[40].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*StopWorkerRequest); i { - case 0: - return &v.state - case 1: - return &v.sizeCache - case 2: - return &v.unknownFields - default: - return nil - } - } - file_org_apache_beam_model_fn_execution_v1_beam_fn_api_proto_msgTypes[41].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*StopWorkerResponse); i { - case 0: - return &v.state - case 1: - return &v.sizeCache - case 2: - return &v.unknownFields - default: - return nil - } - } - file_org_apache_beam_model_fn_execution_v1_beam_fn_api_proto_msgTypes[42].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*WorkerStatusRequest); i { - case 0: - return &v.state - case 1: - return &v.sizeCache - case 2: - return &v.unknownFields - default: - return nil - } - } - file_org_apache_beam_model_fn_execution_v1_beam_fn_api_proto_msgTypes[43].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*WorkerStatusResponse); i { - case 0: - return &v.state - case 1: - return &v.sizeCache - case 2: - return &v.unknownFields - default: - return nil - } - } - file_org_apache_beam_model_fn_execution_v1_beam_fn_api_proto_msgTypes[44].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*SampledElement_Exception); i { - case 0: - return &v.state - case 1: - return &v.sizeCache - case 2: - return &v.unknownFields - default: - return nil - } - } - file_org_apache_beam_model_fn_execution_v1_beam_fn_api_proto_msgTypes[45].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*SampleDataResponse_ElementList); i { - case 0: - return &v.state - case 1: - return &v.sizeCache - case 2: - return &v.unknownFields - default: - return nil - } - } - file_org_apache_beam_model_fn_execution_v1_beam_fn_api_proto_msgTypes[54].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*ProcessBundleRequest_CacheToken); i { - case 0: - return &v.state - case 1: - return &v.sizeCache - case 2: - return &v.unknownFields - default: - return nil - } - } - file_org_apache_beam_model_fn_execution_v1_beam_fn_api_proto_msgTypes[55].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*ProcessBundleRequest_CacheToken_UserState); i { - case 0: - return &v.state - case 1: - return &v.sizeCache - case 2: - return &v.unknownFields - default: - return nil - } - } - file_org_apache_beam_model_fn_execution_v1_beam_fn_api_proto_msgTypes[56].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*ProcessBundleRequest_CacheToken_SideInput); i { - case 0: - return &v.state - case 1: - return &v.sizeCache - case 2: - return &v.unknownFields - default: - return nil - } - } - file_org_apache_beam_model_fn_execution_v1_beam_fn_api_proto_msgTypes[60].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*ProcessBundleSplitRequest_DesiredSplit); i { - case 0: - return &v.state - case 1: - return &v.sizeCache - case 2: - return &v.unknownFields - default: - return nil - } - } - file_org_apache_beam_model_fn_execution_v1_beam_fn_api_proto_msgTypes[62].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*ProcessBundleSplitResponse_ChannelSplit); i { - case 0: - return &v.state - case 1: - return &v.sizeCache - case 2: - return &v.unknownFields - default: - return nil - } - } - file_org_apache_beam_model_fn_execution_v1_beam_fn_api_proto_msgTypes[63].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*Elements_Data); i { - case 0: - return &v.state - case 1: - return &v.sizeCache - case 2: - return &v.unknownFields - default: - return nil - } - } - file_org_apache_beam_model_fn_execution_v1_beam_fn_api_proto_msgTypes[64].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*Elements_Timers); i { - case 0: - return &v.state - case 1: - return &v.sizeCache - case 2: - return &v.unknownFields - default: - return nil - } - } - file_org_apache_beam_model_fn_execution_v1_beam_fn_api_proto_msgTypes[65].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*StateKey_Runner); i { - case 0: - return &v.state - case 1: - return &v.sizeCache - case 2: - return &v.unknownFields - default: - return nil - } - } - file_org_apache_beam_model_fn_execution_v1_beam_fn_api_proto_msgTypes[66].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*StateKey_IterableSideInput); i { - case 0: - return &v.state - case 1: - return &v.sizeCache - case 2: - return &v.unknownFields - default: - return nil - } - } - file_org_apache_beam_model_fn_execution_v1_beam_fn_api_proto_msgTypes[67].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*StateKey_MultimapSideInput); i { - case 0: - return &v.state - case 1: - return &v.sizeCache - case 2: - return &v.unknownFields - default: - return nil - } - } - file_org_apache_beam_model_fn_execution_v1_beam_fn_api_proto_msgTypes[68].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*StateKey_MultimapKeysSideInput); i { - case 0: - return &v.state - case 1: - return &v.sizeCache - case 2: - return &v.unknownFields - default: - return nil - } - } - file_org_apache_beam_model_fn_execution_v1_beam_fn_api_proto_msgTypes[69].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*StateKey_MultimapKeysValuesSideInput); i { - case 0: - return &v.state - case 1: - return &v.sizeCache - case 2: - return &v.unknownFields - default: - return nil - } - } - file_org_apache_beam_model_fn_execution_v1_beam_fn_api_proto_msgTypes[70].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*StateKey_BagUserState); i { - case 0: - return &v.state - case 1: - return &v.sizeCache - case 2: - return &v.unknownFields - default: - return nil - } - } - file_org_apache_beam_model_fn_execution_v1_beam_fn_api_proto_msgTypes[71].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*StateKey_MultimapKeysUserState); i { - case 0: - return &v.state - case 1: - return &v.sizeCache - case 2: - return &v.unknownFields - default: - return nil - } - } - file_org_apache_beam_model_fn_execution_v1_beam_fn_api_proto_msgTypes[72].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*StateKey_MultimapUserState); i { - case 0: - return &v.state - case 1: - return &v.sizeCache - case 2: - return &v.unknownFields - default: - return nil - } - } - file_org_apache_beam_model_fn_execution_v1_beam_fn_api_proto_msgTypes[73].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*StateKey_OrderedListUserState); i { - case 0: - return &v.state - case 1: - return &v.sizeCache - case 2: - return &v.unknownFields - default: - return nil - } - } - file_org_apache_beam_model_fn_execution_v1_beam_fn_api_proto_msgTypes[74].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*LogEntry_List); i { - case 0: - return &v.state - case 1: - return &v.sizeCache - case 2: - return &v.unknownFields - default: - return nil - } - } - file_org_apache_beam_model_fn_execution_v1_beam_fn_api_proto_msgTypes[75].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*LogEntry_Severity); i { - case 0: - return &v.state - case 1: - return &v.sizeCache - case 2: - return &v.unknownFields - default: - return nil - } - } - } - file_org_apache_beam_model_fn_execution_v1_beam_fn_api_proto_msgTypes[3].OneofWrappers = []interface{}{ + file_org_apache_beam_model_fn_execution_v1_beam_fn_api_proto_msgTypes[3].OneofWrappers = []any{ (*InstructionRequest_ProcessBundle)(nil), (*InstructionRequest_ProcessBundleProgress)(nil), (*InstructionRequest_ProcessBundleSplit)(nil), @@ -6977,7 +9133,7 @@ func file_org_apache_beam_model_fn_execution_v1_beam_fn_api_proto_init() { (*InstructionRequest_SampleData)(nil), (*InstructionRequest_Register)(nil), } - file_org_apache_beam_model_fn_execution_v1_beam_fn_api_proto_msgTypes[4].OneofWrappers = []interface{}{ + file_org_apache_beam_model_fn_execution_v1_beam_fn_api_proto_msgTypes[4].OneofWrappers = []any{ (*InstructionResponse_ProcessBundle)(nil), (*InstructionResponse_ProcessBundleProgress)(nil), (*InstructionResponse_ProcessBundleSplit)(nil), @@ -6987,18 +9143,18 @@ func file_org_apache_beam_model_fn_execution_v1_beam_fn_api_proto_init() { (*InstructionResponse_SampleData)(nil), (*InstructionResponse_Register)(nil), } - file_org_apache_beam_model_fn_execution_v1_beam_fn_api_proto_msgTypes[19].OneofWrappers = []interface{}{} - file_org_apache_beam_model_fn_execution_v1_beam_fn_api_proto_msgTypes[26].OneofWrappers = []interface{}{ + file_org_apache_beam_model_fn_execution_v1_beam_fn_api_proto_msgTypes[19].OneofWrappers = []any{} + file_org_apache_beam_model_fn_execution_v1_beam_fn_api_proto_msgTypes[26].OneofWrappers = []any{ (*StateRequest_Get)(nil), (*StateRequest_Append)(nil), (*StateRequest_Clear)(nil), } - file_org_apache_beam_model_fn_execution_v1_beam_fn_api_proto_msgTypes[27].OneofWrappers = []interface{}{ + file_org_apache_beam_model_fn_execution_v1_beam_fn_api_proto_msgTypes[27].OneofWrappers = []any{ (*StateResponse_Get)(nil), (*StateResponse_Append)(nil), (*StateResponse_Clear)(nil), } - file_org_apache_beam_model_fn_execution_v1_beam_fn_api_proto_msgTypes[28].OneofWrappers = []interface{}{ + file_org_apache_beam_model_fn_execution_v1_beam_fn_api_proto_msgTypes[28].OneofWrappers = []any{ (*StateKey_Runner_)(nil), (*StateKey_MultimapSideInput_)(nil), (*StateKey_BagUserState_)(nil), @@ -7009,7 +9165,7 @@ func file_org_apache_beam_model_fn_execution_v1_beam_fn_api_proto_init() { (*StateKey_MultimapUserState_)(nil), (*StateKey_OrderedListUserState_)(nil), } - file_org_apache_beam_model_fn_execution_v1_beam_fn_api_proto_msgTypes[54].OneofWrappers = []interface{}{ + file_org_apache_beam_model_fn_execution_v1_beam_fn_api_proto_msgTypes[54].OneofWrappers = []any{ (*ProcessBundleRequest_CacheToken_UserState_)(nil), (*ProcessBundleRequest_CacheToken_SideInput_)(nil), } diff --git a/sdks/go/pkg/beam/model/fnexecution_v1/beam_fn_api_grpc.pb.go b/sdks/go/pkg/beam/model/fnexecution_v1/beam_fn_api_grpc.pb.go index 60bdeccc31e4..90ad06cd04c8 100644 --- a/sdks/go/pkg/beam/model/fnexecution_v1/beam_fn_api_grpc.pb.go +++ b/sdks/go/pkg/beam/model/fnexecution_v1/beam_fn_api_grpc.pb.go @@ -14,12 +14,25 @@ // WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. // See the License for the specific language governing permissions and // limitations under the License. + +// +// Protocol Buffers describing the Fn API and boostrapping. +// +// TODO: Usage of plural names in lists looks awkward in Java +// e.g. getOutputsMap, addCodersBuilder +// +// TODO: gRPC / proto field names conflict with generated code +// e.g. "class" in java, "output" in python + // Code generated by protoc-gen-go-grpc. DO NOT EDIT. // versions: -// - protoc-gen-go-grpc v1.1.0 +// - protoc-gen-go-grpc v1.5.1 // - protoc v5.27.3 // source: org/apache/beam/model/fn_execution/v1/beam_fn_api.proto +// TODO: Consider consolidating common components in another package +// and language namespaces for re-use with Runner Api. + package fnexecution_v1 import ( @@ -31,16 +44,24 @@ import ( // This is a compile-time assertion to ensure that this generated file // is compatible with the grpc package it is being compiled against. -// Requires gRPC-Go v1.32.0 or later. -const _ = grpc.SupportPackageIsVersion7 +// Requires gRPC-Go v1.64.0 or later. +const _ = grpc.SupportPackageIsVersion9 + +const ( + BeamFnControl_Control_FullMethodName = "/org.apache.beam.model.fn_execution.v1.BeamFnControl/Control" + BeamFnControl_GetProcessBundleDescriptor_FullMethodName = "/org.apache.beam.model.fn_execution.v1.BeamFnControl/GetProcessBundleDescriptor" +) // BeamFnControlClient is the client API for BeamFnControl service. // // For semantics around ctx use and closing/ending streaming RPCs, please refer to https://pkg.go.dev/google.golang.org/grpc/?tab=doc#ClientConn.NewStream. +// +// An API that describes the work that a SDK harness is meant to do. +// Stable type BeamFnControlClient interface { // Instructions sent by the runner to the SDK requesting different types // of work. - Control(ctx context.Context, opts ...grpc.CallOption) (BeamFnControl_ControlClient, error) + Control(ctx context.Context, opts ...grpc.CallOption) (grpc.BidiStreamingClient[InstructionResponse, InstructionRequest], error) // Used to get the full process bundle descriptors for bundles one // is asked to process. GetProcessBundleDescriptor(ctx context.Context, in *GetProcessBundleDescriptorRequest, opts ...grpc.CallOption) (*ProcessBundleDescriptor, error) @@ -54,40 +75,23 @@ func NewBeamFnControlClient(cc grpc.ClientConnInterface) BeamFnControlClient { return &beamFnControlClient{cc} } -func (c *beamFnControlClient) Control(ctx context.Context, opts ...grpc.CallOption) (BeamFnControl_ControlClient, error) { - stream, err := c.cc.NewStream(ctx, &BeamFnControl_ServiceDesc.Streams[0], "/org.apache.beam.model.fn_execution.v1.BeamFnControl/Control", opts...) +func (c *beamFnControlClient) Control(ctx context.Context, opts ...grpc.CallOption) (grpc.BidiStreamingClient[InstructionResponse, InstructionRequest], error) { + cOpts := append([]grpc.CallOption{grpc.StaticMethod()}, opts...) + stream, err := c.cc.NewStream(ctx, &BeamFnControl_ServiceDesc.Streams[0], BeamFnControl_Control_FullMethodName, cOpts...) if err != nil { return nil, err } - x := &beamFnControlControlClient{stream} + x := &grpc.GenericClientStream[InstructionResponse, InstructionRequest]{ClientStream: stream} return x, nil } -type BeamFnControl_ControlClient interface { - Send(*InstructionResponse) error - Recv() (*InstructionRequest, error) - grpc.ClientStream -} - -type beamFnControlControlClient struct { - grpc.ClientStream -} - -func (x *beamFnControlControlClient) Send(m *InstructionResponse) error { - return x.ClientStream.SendMsg(m) -} - -func (x *beamFnControlControlClient) Recv() (*InstructionRequest, error) { - m := new(InstructionRequest) - if err := x.ClientStream.RecvMsg(m); err != nil { - return nil, err - } - return m, nil -} +// This type alias is provided for backwards compatibility with existing code that references the prior non-generic stream type by name. +type BeamFnControl_ControlClient = grpc.BidiStreamingClient[InstructionResponse, InstructionRequest] func (c *beamFnControlClient) GetProcessBundleDescriptor(ctx context.Context, in *GetProcessBundleDescriptorRequest, opts ...grpc.CallOption) (*ProcessBundleDescriptor, error) { + cOpts := append([]grpc.CallOption{grpc.StaticMethod()}, opts...) out := new(ProcessBundleDescriptor) - err := c.cc.Invoke(ctx, "/org.apache.beam.model.fn_execution.v1.BeamFnControl/GetProcessBundleDescriptor", in, out, opts...) + err := c.cc.Invoke(ctx, BeamFnControl_GetProcessBundleDescriptor_FullMethodName, in, out, cOpts...) if err != nil { return nil, err } @@ -96,28 +100,35 @@ func (c *beamFnControlClient) GetProcessBundleDescriptor(ctx context.Context, in // BeamFnControlServer is the server API for BeamFnControl service. // All implementations must embed UnimplementedBeamFnControlServer -// for forward compatibility +// for forward compatibility. +// +// An API that describes the work that a SDK harness is meant to do. +// Stable type BeamFnControlServer interface { // Instructions sent by the runner to the SDK requesting different types // of work. - Control(BeamFnControl_ControlServer) error + Control(grpc.BidiStreamingServer[InstructionResponse, InstructionRequest]) error // Used to get the full process bundle descriptors for bundles one // is asked to process. GetProcessBundleDescriptor(context.Context, *GetProcessBundleDescriptorRequest) (*ProcessBundleDescriptor, error) mustEmbedUnimplementedBeamFnControlServer() } -// UnimplementedBeamFnControlServer must be embedded to have forward compatible implementations. -type UnimplementedBeamFnControlServer struct { -} +// UnimplementedBeamFnControlServer must be embedded to have +// forward compatible implementations. +// +// NOTE: this should be embedded by value instead of pointer to avoid a nil +// pointer dereference when methods are called. +type UnimplementedBeamFnControlServer struct{} -func (UnimplementedBeamFnControlServer) Control(BeamFnControl_ControlServer) error { +func (UnimplementedBeamFnControlServer) Control(grpc.BidiStreamingServer[InstructionResponse, InstructionRequest]) error { return status.Errorf(codes.Unimplemented, "method Control not implemented") } func (UnimplementedBeamFnControlServer) GetProcessBundleDescriptor(context.Context, *GetProcessBundleDescriptorRequest) (*ProcessBundleDescriptor, error) { return nil, status.Errorf(codes.Unimplemented, "method GetProcessBundleDescriptor not implemented") } func (UnimplementedBeamFnControlServer) mustEmbedUnimplementedBeamFnControlServer() {} +func (UnimplementedBeamFnControlServer) testEmbeddedByValue() {} // UnsafeBeamFnControlServer may be embedded to opt out of forward compatibility for this service. // Use of this interface is not recommended, as added methods to BeamFnControlServer will @@ -127,34 +138,22 @@ type UnsafeBeamFnControlServer interface { } func RegisterBeamFnControlServer(s grpc.ServiceRegistrar, srv BeamFnControlServer) { + // If the following call pancis, it indicates UnimplementedBeamFnControlServer was + // embedded by pointer and is nil. This will cause panics if an + // unimplemented method is ever invoked, so we test this at initialization + // time to prevent it from happening at runtime later due to I/O. + if t, ok := srv.(interface{ testEmbeddedByValue() }); ok { + t.testEmbeddedByValue() + } s.RegisterService(&BeamFnControl_ServiceDesc, srv) } func _BeamFnControl_Control_Handler(srv interface{}, stream grpc.ServerStream) error { - return srv.(BeamFnControlServer).Control(&beamFnControlControlServer{stream}) + return srv.(BeamFnControlServer).Control(&grpc.GenericServerStream[InstructionResponse, InstructionRequest]{ServerStream: stream}) } -type BeamFnControl_ControlServer interface { - Send(*InstructionRequest) error - Recv() (*InstructionResponse, error) - grpc.ServerStream -} - -type beamFnControlControlServer struct { - grpc.ServerStream -} - -func (x *beamFnControlControlServer) Send(m *InstructionRequest) error { - return x.ServerStream.SendMsg(m) -} - -func (x *beamFnControlControlServer) Recv() (*InstructionResponse, error) { - m := new(InstructionResponse) - if err := x.ServerStream.RecvMsg(m); err != nil { - return nil, err - } - return m, nil -} +// This type alias is provided for backwards compatibility with existing code that references the prior non-generic stream type by name. +type BeamFnControl_ControlServer = grpc.BidiStreamingServer[InstructionResponse, InstructionRequest] func _BeamFnControl_GetProcessBundleDescriptor_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { in := new(GetProcessBundleDescriptorRequest) @@ -166,7 +165,7 @@ func _BeamFnControl_GetProcessBundleDescriptor_Handler(srv interface{}, ctx cont } info := &grpc.UnaryServerInfo{ Server: srv, - FullMethod: "/org.apache.beam.model.fn_execution.v1.BeamFnControl/GetProcessBundleDescriptor", + FullMethod: BeamFnControl_GetProcessBundleDescriptor_FullMethodName, } handler := func(ctx context.Context, req interface{}) (interface{}, error) { return srv.(BeamFnControlServer).GetProcessBundleDescriptor(ctx, req.(*GetProcessBundleDescriptorRequest)) @@ -197,12 +196,18 @@ var BeamFnControl_ServiceDesc = grpc.ServiceDesc{ Metadata: "org/apache/beam/model/fn_execution/v1/beam_fn_api.proto", } +const ( + BeamFnData_Data_FullMethodName = "/org.apache.beam.model.fn_execution.v1.BeamFnData/Data" +) + // BeamFnDataClient is the client API for BeamFnData service. // // For semantics around ctx use and closing/ending streaming RPCs, please refer to https://pkg.go.dev/google.golang.org/grpc/?tab=doc#ClientConn.NewStream. +// +// Stable type BeamFnDataClient interface { // Used to send data between harnesses. - Data(ctx context.Context, opts ...grpc.CallOption) (BeamFnData_DataClient, error) + Data(ctx context.Context, opts ...grpc.CallOption) (grpc.BidiStreamingClient[Elements, Elements], error) } type beamFnDataClient struct { @@ -213,54 +218,42 @@ func NewBeamFnDataClient(cc grpc.ClientConnInterface) BeamFnDataClient { return &beamFnDataClient{cc} } -func (c *beamFnDataClient) Data(ctx context.Context, opts ...grpc.CallOption) (BeamFnData_DataClient, error) { - stream, err := c.cc.NewStream(ctx, &BeamFnData_ServiceDesc.Streams[0], "/org.apache.beam.model.fn_execution.v1.BeamFnData/Data", opts...) +func (c *beamFnDataClient) Data(ctx context.Context, opts ...grpc.CallOption) (grpc.BidiStreamingClient[Elements, Elements], error) { + cOpts := append([]grpc.CallOption{grpc.StaticMethod()}, opts...) + stream, err := c.cc.NewStream(ctx, &BeamFnData_ServiceDesc.Streams[0], BeamFnData_Data_FullMethodName, cOpts...) if err != nil { return nil, err } - x := &beamFnDataDataClient{stream} + x := &grpc.GenericClientStream[Elements, Elements]{ClientStream: stream} return x, nil } -type BeamFnData_DataClient interface { - Send(*Elements) error - Recv() (*Elements, error) - grpc.ClientStream -} - -type beamFnDataDataClient struct { - grpc.ClientStream -} - -func (x *beamFnDataDataClient) Send(m *Elements) error { - return x.ClientStream.SendMsg(m) -} - -func (x *beamFnDataDataClient) Recv() (*Elements, error) { - m := new(Elements) - if err := x.ClientStream.RecvMsg(m); err != nil { - return nil, err - } - return m, nil -} +// This type alias is provided for backwards compatibility with existing code that references the prior non-generic stream type by name. +type BeamFnData_DataClient = grpc.BidiStreamingClient[Elements, Elements] // BeamFnDataServer is the server API for BeamFnData service. // All implementations must embed UnimplementedBeamFnDataServer -// for forward compatibility +// for forward compatibility. +// +// Stable type BeamFnDataServer interface { // Used to send data between harnesses. - Data(BeamFnData_DataServer) error + Data(grpc.BidiStreamingServer[Elements, Elements]) error mustEmbedUnimplementedBeamFnDataServer() } -// UnimplementedBeamFnDataServer must be embedded to have forward compatible implementations. -type UnimplementedBeamFnDataServer struct { -} +// UnimplementedBeamFnDataServer must be embedded to have +// forward compatible implementations. +// +// NOTE: this should be embedded by value instead of pointer to avoid a nil +// pointer dereference when methods are called. +type UnimplementedBeamFnDataServer struct{} -func (UnimplementedBeamFnDataServer) Data(BeamFnData_DataServer) error { +func (UnimplementedBeamFnDataServer) Data(grpc.BidiStreamingServer[Elements, Elements]) error { return status.Errorf(codes.Unimplemented, "method Data not implemented") } func (UnimplementedBeamFnDataServer) mustEmbedUnimplementedBeamFnDataServer() {} +func (UnimplementedBeamFnDataServer) testEmbeddedByValue() {} // UnsafeBeamFnDataServer may be embedded to opt out of forward compatibility for this service. // Use of this interface is not recommended, as added methods to BeamFnDataServer will @@ -270,34 +263,22 @@ type UnsafeBeamFnDataServer interface { } func RegisterBeamFnDataServer(s grpc.ServiceRegistrar, srv BeamFnDataServer) { + // If the following call pancis, it indicates UnimplementedBeamFnDataServer was + // embedded by pointer and is nil. This will cause panics if an + // unimplemented method is ever invoked, so we test this at initialization + // time to prevent it from happening at runtime later due to I/O. + if t, ok := srv.(interface{ testEmbeddedByValue() }); ok { + t.testEmbeddedByValue() + } s.RegisterService(&BeamFnData_ServiceDesc, srv) } func _BeamFnData_Data_Handler(srv interface{}, stream grpc.ServerStream) error { - return srv.(BeamFnDataServer).Data(&beamFnDataDataServer{stream}) + return srv.(BeamFnDataServer).Data(&grpc.GenericServerStream[Elements, Elements]{ServerStream: stream}) } -type BeamFnData_DataServer interface { - Send(*Elements) error - Recv() (*Elements, error) - grpc.ServerStream -} - -type beamFnDataDataServer struct { - grpc.ServerStream -} - -func (x *beamFnDataDataServer) Send(m *Elements) error { - return x.ServerStream.SendMsg(m) -} - -func (x *beamFnDataDataServer) Recv() (*Elements, error) { - m := new(Elements) - if err := x.ServerStream.RecvMsg(m); err != nil { - return nil, err - } - return m, nil -} +// This type alias is provided for backwards compatibility with existing code that references the prior non-generic stream type by name. +type BeamFnData_DataServer = grpc.BidiStreamingServer[Elements, Elements] // BeamFnData_ServiceDesc is the grpc.ServiceDesc for BeamFnData service. // It's only intended for direct use with grpc.RegisterService, @@ -317,12 +298,16 @@ var BeamFnData_ServiceDesc = grpc.ServiceDesc{ Metadata: "org/apache/beam/model/fn_execution/v1/beam_fn_api.proto", } +const ( + BeamFnState_State_FullMethodName = "/org.apache.beam.model.fn_execution.v1.BeamFnState/State" +) + // BeamFnStateClient is the client API for BeamFnState service. // // For semantics around ctx use and closing/ending streaming RPCs, please refer to https://pkg.go.dev/google.golang.org/grpc/?tab=doc#ClientConn.NewStream. type BeamFnStateClient interface { // Used to get/append/clear state stored by the runner on behalf of the SDK. - State(ctx context.Context, opts ...grpc.CallOption) (BeamFnState_StateClient, error) + State(ctx context.Context, opts ...grpc.CallOption) (grpc.BidiStreamingClient[StateRequest, StateResponse], error) } type beamFnStateClient struct { @@ -333,54 +318,40 @@ func NewBeamFnStateClient(cc grpc.ClientConnInterface) BeamFnStateClient { return &beamFnStateClient{cc} } -func (c *beamFnStateClient) State(ctx context.Context, opts ...grpc.CallOption) (BeamFnState_StateClient, error) { - stream, err := c.cc.NewStream(ctx, &BeamFnState_ServiceDesc.Streams[0], "/org.apache.beam.model.fn_execution.v1.BeamFnState/State", opts...) +func (c *beamFnStateClient) State(ctx context.Context, opts ...grpc.CallOption) (grpc.BidiStreamingClient[StateRequest, StateResponse], error) { + cOpts := append([]grpc.CallOption{grpc.StaticMethod()}, opts...) + stream, err := c.cc.NewStream(ctx, &BeamFnState_ServiceDesc.Streams[0], BeamFnState_State_FullMethodName, cOpts...) if err != nil { return nil, err } - x := &beamFnStateStateClient{stream} + x := &grpc.GenericClientStream[StateRequest, StateResponse]{ClientStream: stream} return x, nil } -type BeamFnState_StateClient interface { - Send(*StateRequest) error - Recv() (*StateResponse, error) - grpc.ClientStream -} - -type beamFnStateStateClient struct { - grpc.ClientStream -} - -func (x *beamFnStateStateClient) Send(m *StateRequest) error { - return x.ClientStream.SendMsg(m) -} - -func (x *beamFnStateStateClient) Recv() (*StateResponse, error) { - m := new(StateResponse) - if err := x.ClientStream.RecvMsg(m); err != nil { - return nil, err - } - return m, nil -} +// This type alias is provided for backwards compatibility with existing code that references the prior non-generic stream type by name. +type BeamFnState_StateClient = grpc.BidiStreamingClient[StateRequest, StateResponse] // BeamFnStateServer is the server API for BeamFnState service. // All implementations must embed UnimplementedBeamFnStateServer -// for forward compatibility +// for forward compatibility. type BeamFnStateServer interface { // Used to get/append/clear state stored by the runner on behalf of the SDK. - State(BeamFnState_StateServer) error + State(grpc.BidiStreamingServer[StateRequest, StateResponse]) error mustEmbedUnimplementedBeamFnStateServer() } -// UnimplementedBeamFnStateServer must be embedded to have forward compatible implementations. -type UnimplementedBeamFnStateServer struct { -} +// UnimplementedBeamFnStateServer must be embedded to have +// forward compatible implementations. +// +// NOTE: this should be embedded by value instead of pointer to avoid a nil +// pointer dereference when methods are called. +type UnimplementedBeamFnStateServer struct{} -func (UnimplementedBeamFnStateServer) State(BeamFnState_StateServer) error { +func (UnimplementedBeamFnStateServer) State(grpc.BidiStreamingServer[StateRequest, StateResponse]) error { return status.Errorf(codes.Unimplemented, "method State not implemented") } func (UnimplementedBeamFnStateServer) mustEmbedUnimplementedBeamFnStateServer() {} +func (UnimplementedBeamFnStateServer) testEmbeddedByValue() {} // UnsafeBeamFnStateServer may be embedded to opt out of forward compatibility for this service. // Use of this interface is not recommended, as added methods to BeamFnStateServer will @@ -390,34 +361,22 @@ type UnsafeBeamFnStateServer interface { } func RegisterBeamFnStateServer(s grpc.ServiceRegistrar, srv BeamFnStateServer) { + // If the following call pancis, it indicates UnimplementedBeamFnStateServer was + // embedded by pointer and is nil. This will cause panics if an + // unimplemented method is ever invoked, so we test this at initialization + // time to prevent it from happening at runtime later due to I/O. + if t, ok := srv.(interface{ testEmbeddedByValue() }); ok { + t.testEmbeddedByValue() + } s.RegisterService(&BeamFnState_ServiceDesc, srv) } func _BeamFnState_State_Handler(srv interface{}, stream grpc.ServerStream) error { - return srv.(BeamFnStateServer).State(&beamFnStateStateServer{stream}) + return srv.(BeamFnStateServer).State(&grpc.GenericServerStream[StateRequest, StateResponse]{ServerStream: stream}) } -type BeamFnState_StateServer interface { - Send(*StateResponse) error - Recv() (*StateRequest, error) - grpc.ServerStream -} - -type beamFnStateStateServer struct { - grpc.ServerStream -} - -func (x *beamFnStateStateServer) Send(m *StateResponse) error { - return x.ServerStream.SendMsg(m) -} - -func (x *beamFnStateStateServer) Recv() (*StateRequest, error) { - m := new(StateRequest) - if err := x.ServerStream.RecvMsg(m); err != nil { - return nil, err - } - return m, nil -} +// This type alias is provided for backwards compatibility with existing code that references the prior non-generic stream type by name. +type BeamFnState_StateServer = grpc.BidiStreamingServer[StateRequest, StateResponse] // BeamFnState_ServiceDesc is the grpc.ServiceDesc for BeamFnState service. // It's only intended for direct use with grpc.RegisterService, @@ -437,13 +396,19 @@ var BeamFnState_ServiceDesc = grpc.ServiceDesc{ Metadata: "org/apache/beam/model/fn_execution/v1/beam_fn_api.proto", } +const ( + BeamFnLogging_Logging_FullMethodName = "/org.apache.beam.model.fn_execution.v1.BeamFnLogging/Logging" +) + // BeamFnLoggingClient is the client API for BeamFnLogging service. // // For semantics around ctx use and closing/ending streaming RPCs, please refer to https://pkg.go.dev/google.golang.org/grpc/?tab=doc#ClientConn.NewStream. +// +// Stable type BeamFnLoggingClient interface { // Allows for the SDK to emit log entries which the runner can // associate with the active job. - Logging(ctx context.Context, opts ...grpc.CallOption) (BeamFnLogging_LoggingClient, error) + Logging(ctx context.Context, opts ...grpc.CallOption) (grpc.BidiStreamingClient[LogEntry_List, LogControl], error) } type beamFnLoggingClient struct { @@ -454,55 +419,43 @@ func NewBeamFnLoggingClient(cc grpc.ClientConnInterface) BeamFnLoggingClient { return &beamFnLoggingClient{cc} } -func (c *beamFnLoggingClient) Logging(ctx context.Context, opts ...grpc.CallOption) (BeamFnLogging_LoggingClient, error) { - stream, err := c.cc.NewStream(ctx, &BeamFnLogging_ServiceDesc.Streams[0], "/org.apache.beam.model.fn_execution.v1.BeamFnLogging/Logging", opts...) +func (c *beamFnLoggingClient) Logging(ctx context.Context, opts ...grpc.CallOption) (grpc.BidiStreamingClient[LogEntry_List, LogControl], error) { + cOpts := append([]grpc.CallOption{grpc.StaticMethod()}, opts...) + stream, err := c.cc.NewStream(ctx, &BeamFnLogging_ServiceDesc.Streams[0], BeamFnLogging_Logging_FullMethodName, cOpts...) if err != nil { return nil, err } - x := &beamFnLoggingLoggingClient{stream} + x := &grpc.GenericClientStream[LogEntry_List, LogControl]{ClientStream: stream} return x, nil } -type BeamFnLogging_LoggingClient interface { - Send(*LogEntry_List) error - Recv() (*LogControl, error) - grpc.ClientStream -} - -type beamFnLoggingLoggingClient struct { - grpc.ClientStream -} - -func (x *beamFnLoggingLoggingClient) Send(m *LogEntry_List) error { - return x.ClientStream.SendMsg(m) -} - -func (x *beamFnLoggingLoggingClient) Recv() (*LogControl, error) { - m := new(LogControl) - if err := x.ClientStream.RecvMsg(m); err != nil { - return nil, err - } - return m, nil -} +// This type alias is provided for backwards compatibility with existing code that references the prior non-generic stream type by name. +type BeamFnLogging_LoggingClient = grpc.BidiStreamingClient[LogEntry_List, LogControl] // BeamFnLoggingServer is the server API for BeamFnLogging service. // All implementations must embed UnimplementedBeamFnLoggingServer -// for forward compatibility +// for forward compatibility. +// +// Stable type BeamFnLoggingServer interface { // Allows for the SDK to emit log entries which the runner can // associate with the active job. - Logging(BeamFnLogging_LoggingServer) error + Logging(grpc.BidiStreamingServer[LogEntry_List, LogControl]) error mustEmbedUnimplementedBeamFnLoggingServer() } -// UnimplementedBeamFnLoggingServer must be embedded to have forward compatible implementations. -type UnimplementedBeamFnLoggingServer struct { -} +// UnimplementedBeamFnLoggingServer must be embedded to have +// forward compatible implementations. +// +// NOTE: this should be embedded by value instead of pointer to avoid a nil +// pointer dereference when methods are called. +type UnimplementedBeamFnLoggingServer struct{} -func (UnimplementedBeamFnLoggingServer) Logging(BeamFnLogging_LoggingServer) error { +func (UnimplementedBeamFnLoggingServer) Logging(grpc.BidiStreamingServer[LogEntry_List, LogControl]) error { return status.Errorf(codes.Unimplemented, "method Logging not implemented") } func (UnimplementedBeamFnLoggingServer) mustEmbedUnimplementedBeamFnLoggingServer() {} +func (UnimplementedBeamFnLoggingServer) testEmbeddedByValue() {} // UnsafeBeamFnLoggingServer may be embedded to opt out of forward compatibility for this service. // Use of this interface is not recommended, as added methods to BeamFnLoggingServer will @@ -512,34 +465,22 @@ type UnsafeBeamFnLoggingServer interface { } func RegisterBeamFnLoggingServer(s grpc.ServiceRegistrar, srv BeamFnLoggingServer) { + // If the following call pancis, it indicates UnimplementedBeamFnLoggingServer was + // embedded by pointer and is nil. This will cause panics if an + // unimplemented method is ever invoked, so we test this at initialization + // time to prevent it from happening at runtime later due to I/O. + if t, ok := srv.(interface{ testEmbeddedByValue() }); ok { + t.testEmbeddedByValue() + } s.RegisterService(&BeamFnLogging_ServiceDesc, srv) } func _BeamFnLogging_Logging_Handler(srv interface{}, stream grpc.ServerStream) error { - return srv.(BeamFnLoggingServer).Logging(&beamFnLoggingLoggingServer{stream}) -} - -type BeamFnLogging_LoggingServer interface { - Send(*LogControl) error - Recv() (*LogEntry_List, error) - grpc.ServerStream + return srv.(BeamFnLoggingServer).Logging(&grpc.GenericServerStream[LogEntry_List, LogControl]{ServerStream: stream}) } -type beamFnLoggingLoggingServer struct { - grpc.ServerStream -} - -func (x *beamFnLoggingLoggingServer) Send(m *LogControl) error { - return x.ServerStream.SendMsg(m) -} - -func (x *beamFnLoggingLoggingServer) Recv() (*LogEntry_List, error) { - m := new(LogEntry_List) - if err := x.ServerStream.RecvMsg(m); err != nil { - return nil, err - } - return m, nil -} +// This type alias is provided for backwards compatibility with existing code that references the prior non-generic stream type by name. +type BeamFnLogging_LoggingServer = grpc.BidiStreamingServer[LogEntry_List, LogControl] // BeamFnLogging_ServiceDesc is the grpc.ServiceDesc for BeamFnLogging service. // It's only intended for direct use with grpc.RegisterService, @@ -559,6 +500,11 @@ var BeamFnLogging_ServiceDesc = grpc.ServiceDesc{ Metadata: "org/apache/beam/model/fn_execution/v1/beam_fn_api.proto", } +const ( + BeamFnExternalWorkerPool_StartWorker_FullMethodName = "/org.apache.beam.model.fn_execution.v1.BeamFnExternalWorkerPool/StartWorker" + BeamFnExternalWorkerPool_StopWorker_FullMethodName = "/org.apache.beam.model.fn_execution.v1.BeamFnExternalWorkerPool/StopWorker" +) + // BeamFnExternalWorkerPoolClient is the client API for BeamFnExternalWorkerPool service. // // For semantics around ctx use and closing/ending streaming RPCs, please refer to https://pkg.go.dev/google.golang.org/grpc/?tab=doc#ClientConn.NewStream. @@ -578,8 +524,9 @@ func NewBeamFnExternalWorkerPoolClient(cc grpc.ClientConnInterface) BeamFnExtern } func (c *beamFnExternalWorkerPoolClient) StartWorker(ctx context.Context, in *StartWorkerRequest, opts ...grpc.CallOption) (*StartWorkerResponse, error) { + cOpts := append([]grpc.CallOption{grpc.StaticMethod()}, opts...) out := new(StartWorkerResponse) - err := c.cc.Invoke(ctx, "/org.apache.beam.model.fn_execution.v1.BeamFnExternalWorkerPool/StartWorker", in, out, opts...) + err := c.cc.Invoke(ctx, BeamFnExternalWorkerPool_StartWorker_FullMethodName, in, out, cOpts...) if err != nil { return nil, err } @@ -587,8 +534,9 @@ func (c *beamFnExternalWorkerPoolClient) StartWorker(ctx context.Context, in *St } func (c *beamFnExternalWorkerPoolClient) StopWorker(ctx context.Context, in *StopWorkerRequest, opts ...grpc.CallOption) (*StopWorkerResponse, error) { + cOpts := append([]grpc.CallOption{grpc.StaticMethod()}, opts...) out := new(StopWorkerResponse) - err := c.cc.Invoke(ctx, "/org.apache.beam.model.fn_execution.v1.BeamFnExternalWorkerPool/StopWorker", in, out, opts...) + err := c.cc.Invoke(ctx, BeamFnExternalWorkerPool_StopWorker_FullMethodName, in, out, cOpts...) if err != nil { return nil, err } @@ -597,7 +545,7 @@ func (c *beamFnExternalWorkerPoolClient) StopWorker(ctx context.Context, in *Sto // BeamFnExternalWorkerPoolServer is the server API for BeamFnExternalWorkerPool service. // All implementations must embed UnimplementedBeamFnExternalWorkerPoolServer -// for forward compatibility +// for forward compatibility. type BeamFnExternalWorkerPoolServer interface { // Start the SDK worker with the given ID. StartWorker(context.Context, *StartWorkerRequest) (*StartWorkerResponse, error) @@ -606,9 +554,12 @@ type BeamFnExternalWorkerPoolServer interface { mustEmbedUnimplementedBeamFnExternalWorkerPoolServer() } -// UnimplementedBeamFnExternalWorkerPoolServer must be embedded to have forward compatible implementations. -type UnimplementedBeamFnExternalWorkerPoolServer struct { -} +// UnimplementedBeamFnExternalWorkerPoolServer must be embedded to have +// forward compatible implementations. +// +// NOTE: this should be embedded by value instead of pointer to avoid a nil +// pointer dereference when methods are called. +type UnimplementedBeamFnExternalWorkerPoolServer struct{} func (UnimplementedBeamFnExternalWorkerPoolServer) StartWorker(context.Context, *StartWorkerRequest) (*StartWorkerResponse, error) { return nil, status.Errorf(codes.Unimplemented, "method StartWorker not implemented") @@ -618,6 +569,7 @@ func (UnimplementedBeamFnExternalWorkerPoolServer) StopWorker(context.Context, * } func (UnimplementedBeamFnExternalWorkerPoolServer) mustEmbedUnimplementedBeamFnExternalWorkerPoolServer() { } +func (UnimplementedBeamFnExternalWorkerPoolServer) testEmbeddedByValue() {} // UnsafeBeamFnExternalWorkerPoolServer may be embedded to opt out of forward compatibility for this service. // Use of this interface is not recommended, as added methods to BeamFnExternalWorkerPoolServer will @@ -627,6 +579,13 @@ type UnsafeBeamFnExternalWorkerPoolServer interface { } func RegisterBeamFnExternalWorkerPoolServer(s grpc.ServiceRegistrar, srv BeamFnExternalWorkerPoolServer) { + // If the following call pancis, it indicates UnimplementedBeamFnExternalWorkerPoolServer was + // embedded by pointer and is nil. This will cause panics if an + // unimplemented method is ever invoked, so we test this at initialization + // time to prevent it from happening at runtime later due to I/O. + if t, ok := srv.(interface{ testEmbeddedByValue() }); ok { + t.testEmbeddedByValue() + } s.RegisterService(&BeamFnExternalWorkerPool_ServiceDesc, srv) } @@ -640,7 +599,7 @@ func _BeamFnExternalWorkerPool_StartWorker_Handler(srv interface{}, ctx context. } info := &grpc.UnaryServerInfo{ Server: srv, - FullMethod: "/org.apache.beam.model.fn_execution.v1.BeamFnExternalWorkerPool/StartWorker", + FullMethod: BeamFnExternalWorkerPool_StartWorker_FullMethodName, } handler := func(ctx context.Context, req interface{}) (interface{}, error) { return srv.(BeamFnExternalWorkerPoolServer).StartWorker(ctx, req.(*StartWorkerRequest)) @@ -658,7 +617,7 @@ func _BeamFnExternalWorkerPool_StopWorker_Handler(srv interface{}, ctx context.C } info := &grpc.UnaryServerInfo{ Server: srv, - FullMethod: "/org.apache.beam.model.fn_execution.v1.BeamFnExternalWorkerPool/StopWorker", + FullMethod: BeamFnExternalWorkerPool_StopWorker_FullMethodName, } handler := func(ctx context.Context, req interface{}) (interface{}, error) { return srv.(BeamFnExternalWorkerPoolServer).StopWorker(ctx, req.(*StopWorkerRequest)) @@ -686,11 +645,17 @@ var BeamFnExternalWorkerPool_ServiceDesc = grpc.ServiceDesc{ Metadata: "org/apache/beam/model/fn_execution/v1/beam_fn_api.proto", } +const ( + BeamFnWorkerStatus_WorkerStatus_FullMethodName = "/org.apache.beam.model.fn_execution.v1.BeamFnWorkerStatus/WorkerStatus" +) + // BeamFnWorkerStatusClient is the client API for BeamFnWorkerStatus service. // // For semantics around ctx use and closing/ending streaming RPCs, please refer to https://pkg.go.dev/google.golang.org/grpc/?tab=doc#ClientConn.NewStream. +// +// API for SDKs to report debug-related statuses to runner during pipeline execution. type BeamFnWorkerStatusClient interface { - WorkerStatus(ctx context.Context, opts ...grpc.CallOption) (BeamFnWorkerStatus_WorkerStatusClient, error) + WorkerStatus(ctx context.Context, opts ...grpc.CallOption) (grpc.BidiStreamingClient[WorkerStatusResponse, WorkerStatusRequest], error) } type beamFnWorkerStatusClient struct { @@ -701,53 +666,41 @@ func NewBeamFnWorkerStatusClient(cc grpc.ClientConnInterface) BeamFnWorkerStatus return &beamFnWorkerStatusClient{cc} } -func (c *beamFnWorkerStatusClient) WorkerStatus(ctx context.Context, opts ...grpc.CallOption) (BeamFnWorkerStatus_WorkerStatusClient, error) { - stream, err := c.cc.NewStream(ctx, &BeamFnWorkerStatus_ServiceDesc.Streams[0], "/org.apache.beam.model.fn_execution.v1.BeamFnWorkerStatus/WorkerStatus", opts...) +func (c *beamFnWorkerStatusClient) WorkerStatus(ctx context.Context, opts ...grpc.CallOption) (grpc.BidiStreamingClient[WorkerStatusResponse, WorkerStatusRequest], error) { + cOpts := append([]grpc.CallOption{grpc.StaticMethod()}, opts...) + stream, err := c.cc.NewStream(ctx, &BeamFnWorkerStatus_ServiceDesc.Streams[0], BeamFnWorkerStatus_WorkerStatus_FullMethodName, cOpts...) if err != nil { return nil, err } - x := &beamFnWorkerStatusWorkerStatusClient{stream} + x := &grpc.GenericClientStream[WorkerStatusResponse, WorkerStatusRequest]{ClientStream: stream} return x, nil } -type BeamFnWorkerStatus_WorkerStatusClient interface { - Send(*WorkerStatusResponse) error - Recv() (*WorkerStatusRequest, error) - grpc.ClientStream -} - -type beamFnWorkerStatusWorkerStatusClient struct { - grpc.ClientStream -} - -func (x *beamFnWorkerStatusWorkerStatusClient) Send(m *WorkerStatusResponse) error { - return x.ClientStream.SendMsg(m) -} - -func (x *beamFnWorkerStatusWorkerStatusClient) Recv() (*WorkerStatusRequest, error) { - m := new(WorkerStatusRequest) - if err := x.ClientStream.RecvMsg(m); err != nil { - return nil, err - } - return m, nil -} +// This type alias is provided for backwards compatibility with existing code that references the prior non-generic stream type by name. +type BeamFnWorkerStatus_WorkerStatusClient = grpc.BidiStreamingClient[WorkerStatusResponse, WorkerStatusRequest] // BeamFnWorkerStatusServer is the server API for BeamFnWorkerStatus service. // All implementations must embed UnimplementedBeamFnWorkerStatusServer -// for forward compatibility +// for forward compatibility. +// +// API for SDKs to report debug-related statuses to runner during pipeline execution. type BeamFnWorkerStatusServer interface { - WorkerStatus(BeamFnWorkerStatus_WorkerStatusServer) error + WorkerStatus(grpc.BidiStreamingServer[WorkerStatusResponse, WorkerStatusRequest]) error mustEmbedUnimplementedBeamFnWorkerStatusServer() } -// UnimplementedBeamFnWorkerStatusServer must be embedded to have forward compatible implementations. -type UnimplementedBeamFnWorkerStatusServer struct { -} +// UnimplementedBeamFnWorkerStatusServer must be embedded to have +// forward compatible implementations. +// +// NOTE: this should be embedded by value instead of pointer to avoid a nil +// pointer dereference when methods are called. +type UnimplementedBeamFnWorkerStatusServer struct{} -func (UnimplementedBeamFnWorkerStatusServer) WorkerStatus(BeamFnWorkerStatus_WorkerStatusServer) error { +func (UnimplementedBeamFnWorkerStatusServer) WorkerStatus(grpc.BidiStreamingServer[WorkerStatusResponse, WorkerStatusRequest]) error { return status.Errorf(codes.Unimplemented, "method WorkerStatus not implemented") } func (UnimplementedBeamFnWorkerStatusServer) mustEmbedUnimplementedBeamFnWorkerStatusServer() {} +func (UnimplementedBeamFnWorkerStatusServer) testEmbeddedByValue() {} // UnsafeBeamFnWorkerStatusServer may be embedded to opt out of forward compatibility for this service. // Use of this interface is not recommended, as added methods to BeamFnWorkerStatusServer will @@ -757,34 +710,22 @@ type UnsafeBeamFnWorkerStatusServer interface { } func RegisterBeamFnWorkerStatusServer(s grpc.ServiceRegistrar, srv BeamFnWorkerStatusServer) { + // If the following call pancis, it indicates UnimplementedBeamFnWorkerStatusServer was + // embedded by pointer and is nil. This will cause panics if an + // unimplemented method is ever invoked, so we test this at initialization + // time to prevent it from happening at runtime later due to I/O. + if t, ok := srv.(interface{ testEmbeddedByValue() }); ok { + t.testEmbeddedByValue() + } s.RegisterService(&BeamFnWorkerStatus_ServiceDesc, srv) } func _BeamFnWorkerStatus_WorkerStatus_Handler(srv interface{}, stream grpc.ServerStream) error { - return srv.(BeamFnWorkerStatusServer).WorkerStatus(&beamFnWorkerStatusWorkerStatusServer{stream}) -} - -type BeamFnWorkerStatus_WorkerStatusServer interface { - Send(*WorkerStatusRequest) error - Recv() (*WorkerStatusResponse, error) - grpc.ServerStream -} - -type beamFnWorkerStatusWorkerStatusServer struct { - grpc.ServerStream -} - -func (x *beamFnWorkerStatusWorkerStatusServer) Send(m *WorkerStatusRequest) error { - return x.ServerStream.SendMsg(m) + return srv.(BeamFnWorkerStatusServer).WorkerStatus(&grpc.GenericServerStream[WorkerStatusResponse, WorkerStatusRequest]{ServerStream: stream}) } -func (x *beamFnWorkerStatusWorkerStatusServer) Recv() (*WorkerStatusResponse, error) { - m := new(WorkerStatusResponse) - if err := x.ServerStream.RecvMsg(m); err != nil { - return nil, err - } - return m, nil -} +// This type alias is provided for backwards compatibility with existing code that references the prior non-generic stream type by name. +type BeamFnWorkerStatus_WorkerStatusServer = grpc.BidiStreamingServer[WorkerStatusResponse, WorkerStatusRequest] // BeamFnWorkerStatus_ServiceDesc is the grpc.ServiceDesc for BeamFnWorkerStatus service. // It's only intended for direct use with grpc.RegisterService, diff --git a/sdks/go/pkg/beam/model/fnexecution_v1/beam_fn_api_protoopaque.pb.go b/sdks/go/pkg/beam/model/fnexecution_v1/beam_fn_api_protoopaque.pb.go new file mode 100644 index 000000000000..cd819e3aedd7 --- /dev/null +++ b/sdks/go/pkg/beam/model/fnexecution_v1/beam_fn_api_protoopaque.pb.go @@ -0,0 +1,8793 @@ +// +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +// +// Protocol Buffers describing the Fn API and boostrapping. +// +// TODO: Usage of plural names in lists looks awkward in Java +// e.g. getOutputsMap, addCodersBuilder +// +// TODO: gRPC / proto field names conflict with generated code +// e.g. "class" in java, "output" in python + +// Code generated by protoc-gen-go. DO NOT EDIT. +// versions: +// protoc-gen-go v1.36.0 +// protoc v5.27.3 +// source: org/apache/beam/model/fn_execution/v1/beam_fn_api.proto + +// TODO: Consider consolidating common components in another package +// and language namespaces for re-use with Runner Api. + +//go:build protoopaque + +package fnexecution_v1 + +import ( + pipeline_v1 "github.com/apache/beam/sdks/v2/go/pkg/beam/model/pipeline_v1" + protoreflect "google.golang.org/protobuf/reflect/protoreflect" + protoimpl "google.golang.org/protobuf/runtime/protoimpl" + _ "google.golang.org/protobuf/types/descriptorpb" + durationpb "google.golang.org/protobuf/types/known/durationpb" + structpb "google.golang.org/protobuf/types/known/structpb" + timestamppb "google.golang.org/protobuf/types/known/timestamppb" + reflect "reflect" +) + +const ( + // Verify that this generated code is sufficiently up-to-date. + _ = protoimpl.EnforceVersion(20 - protoimpl.MinVersion) + // Verify that runtime/protoimpl is sufficiently up-to-date. + _ = protoimpl.EnforceVersion(protoimpl.MaxVersion - 20) +) + +type FnApiTransforms_Runner int32 + +const ( + // DataSource is a Root Transform, and a source of data for downstream + // transforms in the same ProcessBundleDescriptor. + // It represents a stream of values coming in from an external source/over + // a data channel, typically from the runner. It's not the PCollection itself + // but a description of how to get the portion of the PCollection for a given + // bundle. + // + // The DataSource transform is implemented in each SDK and not explicitly + // provided during pipeline construction. A runner inserts the transform + // in ProcessBundleDescriptors to indicate where the bundle + // can retrieve data for an associated ProcessBundleRequest. + // Data for the same request will be retrieved with the matching instruction ID, + // and transform ID determined by the runner. + // + // The DataSource transform will take a stream of bytes from the remote + // source for the matching instruction ID and decode them as windowed + // values using the provided coder ID, which must be a windowed value coder. + // + // Payload: RemoteGrpcPort + FnApiTransforms_DATA_SOURCE FnApiTransforms_Runner = 0 + // DataSink is a transform that sends PCollection elements to a remote + // port using the Data API. + // + // The DataSink transform is implemented in each SDK and not explicitly + // provided during pipeline construction. A runner inserts the transform in + // ProcessBundleDescriptors to indicate where the bundle can send + // data for each associated ProcessBundleRequest. Data for the same + // request will be sent with the matching instruction ID and transform ID. + // Each PCollection that exits the ProcessBundleDescriptor subgraph will have + // it's own DataSink, keyed by a transform ID determined by the runner. + // + // The DataSink will take in a stream of elements for a given instruction ID + // and encode them for transmission to the remote sink. The coder ID must be + // for a windowed value coder. + // + // Payload: RemoteGrpcPort + FnApiTransforms_DATA_SINK FnApiTransforms_Runner = 1 +) + +// Enum value maps for FnApiTransforms_Runner. +var ( + FnApiTransforms_Runner_name = map[int32]string{ + 0: "DATA_SOURCE", + 1: "DATA_SINK", + } + FnApiTransforms_Runner_value = map[string]int32{ + "DATA_SOURCE": 0, + "DATA_SINK": 1, + } +) + +func (x FnApiTransforms_Runner) Enum() *FnApiTransforms_Runner { + p := new(FnApiTransforms_Runner) + *p = x + return p +} + +func (x FnApiTransforms_Runner) String() string { + return protoimpl.X.EnumStringOf(x.Descriptor(), protoreflect.EnumNumber(x)) +} + +func (FnApiTransforms_Runner) Descriptor() protoreflect.EnumDescriptor { + return file_org_apache_beam_model_fn_execution_v1_beam_fn_api_proto_enumTypes[0].Descriptor() +} + +func (FnApiTransforms_Runner) Type() protoreflect.EnumType { + return &file_org_apache_beam_model_fn_execution_v1_beam_fn_api_proto_enumTypes[0] +} + +func (x FnApiTransforms_Runner) Number() protoreflect.EnumNumber { + return protoreflect.EnumNumber(x) +} + +type LogEntry_Severity_Enum int32 + +const ( + // Unspecified level information. Will be logged at the TRACE level. + LogEntry_Severity_UNSPECIFIED LogEntry_Severity_Enum = 0 + LogEntry_Severity_TRACE LogEntry_Severity_Enum = 1 + // Debugging information. + LogEntry_Severity_DEBUG LogEntry_Severity_Enum = 2 + // Normal events. + LogEntry_Severity_INFO LogEntry_Severity_Enum = 3 + // Normal but significant events, such as start up, shut down, or + // configuration. + LogEntry_Severity_NOTICE LogEntry_Severity_Enum = 4 + // Warning events might cause problems. + LogEntry_Severity_WARN LogEntry_Severity_Enum = 5 + // Error events are likely to cause problems. + LogEntry_Severity_ERROR LogEntry_Severity_Enum = 6 + // Critical events cause severe problems or brief outages and may + // indicate that a person must take action. + LogEntry_Severity_CRITICAL LogEntry_Severity_Enum = 7 +) + +// Enum value maps for LogEntry_Severity_Enum. +var ( + LogEntry_Severity_Enum_name = map[int32]string{ + 0: "UNSPECIFIED", + 1: "TRACE", + 2: "DEBUG", + 3: "INFO", + 4: "NOTICE", + 5: "WARN", + 6: "ERROR", + 7: "CRITICAL", + } + LogEntry_Severity_Enum_value = map[string]int32{ + "UNSPECIFIED": 0, + "TRACE": 1, + "DEBUG": 2, + "INFO": 3, + "NOTICE": 4, + "WARN": 5, + "ERROR": 6, + "CRITICAL": 7, + } +) + +func (x LogEntry_Severity_Enum) Enum() *LogEntry_Severity_Enum { + p := new(LogEntry_Severity_Enum) + *p = x + return p +} + +func (x LogEntry_Severity_Enum) String() string { + return protoimpl.X.EnumStringOf(x.Descriptor(), protoreflect.EnumNumber(x)) +} + +func (LogEntry_Severity_Enum) Descriptor() protoreflect.EnumDescriptor { + return file_org_apache_beam_model_fn_execution_v1_beam_fn_api_proto_enumTypes[1].Descriptor() +} + +func (LogEntry_Severity_Enum) Type() protoreflect.EnumType { + return &file_org_apache_beam_model_fn_execution_v1_beam_fn_api_proto_enumTypes[1] +} + +func (x LogEntry_Severity_Enum) Number() protoreflect.EnumNumber { + return protoreflect.EnumNumber(x) +} + +// Describes transforms necessary to execute Beam over the FnAPI but are +// implementation details rather than part of the core model. +type FnApiTransforms struct { + state protoimpl.MessageState `protogen:"opaque.v1"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *FnApiTransforms) Reset() { + *x = FnApiTransforms{} + mi := &file_org_apache_beam_model_fn_execution_v1_beam_fn_api_proto_msgTypes[0] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *FnApiTransforms) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*FnApiTransforms) ProtoMessage() {} + +func (x *FnApiTransforms) ProtoReflect() protoreflect.Message { + mi := &file_org_apache_beam_model_fn_execution_v1_beam_fn_api_proto_msgTypes[0] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +type FnApiTransforms_builder struct { + _ [0]func() // Prevents comparability and use of unkeyed literals for the builder. + +} + +func (b0 FnApiTransforms_builder) Build() *FnApiTransforms { + m0 := &FnApiTransforms{} + b, x := &b0, m0 + _, _ = b, x + return m0 +} + +// A descriptor for connecting to a remote port using the Beam Fn Data API. +// Allows for communication between two environments (for example between the +// runner and the SDK). +// Stable +type RemoteGrpcPort struct { + state protoimpl.MessageState `protogen:"opaque.v1"` + xxx_hidden_ApiServiceDescriptor *pipeline_v1.ApiServiceDescriptor `protobuf:"bytes,1,opt,name=api_service_descriptor,json=apiServiceDescriptor,proto3" json:"api_service_descriptor,omitempty"` + xxx_hidden_CoderId string `protobuf:"bytes,2,opt,name=coder_id,json=coderId,proto3" json:"coder_id,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *RemoteGrpcPort) Reset() { + *x = RemoteGrpcPort{} + mi := &file_org_apache_beam_model_fn_execution_v1_beam_fn_api_proto_msgTypes[1] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *RemoteGrpcPort) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*RemoteGrpcPort) ProtoMessage() {} + +func (x *RemoteGrpcPort) ProtoReflect() protoreflect.Message { + mi := &file_org_apache_beam_model_fn_execution_v1_beam_fn_api_proto_msgTypes[1] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +func (x *RemoteGrpcPort) GetApiServiceDescriptor() *pipeline_v1.ApiServiceDescriptor { + if x != nil { + return x.xxx_hidden_ApiServiceDescriptor + } + return nil +} + +func (x *RemoteGrpcPort) GetCoderId() string { + if x != nil { + return x.xxx_hidden_CoderId + } + return "" +} + +func (x *RemoteGrpcPort) SetApiServiceDescriptor(v *pipeline_v1.ApiServiceDescriptor) { + x.xxx_hidden_ApiServiceDescriptor = v +} + +func (x *RemoteGrpcPort) SetCoderId(v string) { + x.xxx_hidden_CoderId = v +} + +func (x *RemoteGrpcPort) HasApiServiceDescriptor() bool { + if x == nil { + return false + } + return x.xxx_hidden_ApiServiceDescriptor != nil +} + +func (x *RemoteGrpcPort) ClearApiServiceDescriptor() { + x.xxx_hidden_ApiServiceDescriptor = nil +} + +type RemoteGrpcPort_builder struct { + _ [0]func() // Prevents comparability and use of unkeyed literals for the builder. + + // (Required) An API descriptor which describes where to + // connect to including any authentication that is required. + ApiServiceDescriptor *pipeline_v1.ApiServiceDescriptor + // (Required) The ID of the Coder that will be used to encode and decode data + // sent over this port. + CoderId string +} + +func (b0 RemoteGrpcPort_builder) Build() *RemoteGrpcPort { + m0 := &RemoteGrpcPort{} + b, x := &b0, m0 + _, _ = b, x + x.xxx_hidden_ApiServiceDescriptor = b.ApiServiceDescriptor + x.xxx_hidden_CoderId = b.CoderId + return m0 +} + +// Requests the ProcessBundleDescriptor with the given id. +type GetProcessBundleDescriptorRequest struct { + state protoimpl.MessageState `protogen:"opaque.v1"` + xxx_hidden_ProcessBundleDescriptorId string `protobuf:"bytes,1,opt,name=process_bundle_descriptor_id,json=processBundleDescriptorId,proto3" json:"process_bundle_descriptor_id,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *GetProcessBundleDescriptorRequest) Reset() { + *x = GetProcessBundleDescriptorRequest{} + mi := &file_org_apache_beam_model_fn_execution_v1_beam_fn_api_proto_msgTypes[2] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *GetProcessBundleDescriptorRequest) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*GetProcessBundleDescriptorRequest) ProtoMessage() {} + +func (x *GetProcessBundleDescriptorRequest) ProtoReflect() protoreflect.Message { + mi := &file_org_apache_beam_model_fn_execution_v1_beam_fn_api_proto_msgTypes[2] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +func (x *GetProcessBundleDescriptorRequest) GetProcessBundleDescriptorId() string { + if x != nil { + return x.xxx_hidden_ProcessBundleDescriptorId + } + return "" +} + +func (x *GetProcessBundleDescriptorRequest) SetProcessBundleDescriptorId(v string) { + x.xxx_hidden_ProcessBundleDescriptorId = v +} + +type GetProcessBundleDescriptorRequest_builder struct { + _ [0]func() // Prevents comparability and use of unkeyed literals for the builder. + + ProcessBundleDescriptorId string +} + +func (b0 GetProcessBundleDescriptorRequest_builder) Build() *GetProcessBundleDescriptorRequest { + m0 := &GetProcessBundleDescriptorRequest{} + b, x := &b0, m0 + _, _ = b, x + x.xxx_hidden_ProcessBundleDescriptorId = b.ProcessBundleDescriptorId + return m0 +} + +// A request sent by a runner which the SDK is asked to fulfill. +// For any unsupported request type, an error should be returned with a +// matching instruction id. +// Stable +type InstructionRequest struct { + state protoimpl.MessageState `protogen:"opaque.v1"` + xxx_hidden_InstructionId string `protobuf:"bytes,1,opt,name=instruction_id,json=instructionId,proto3" json:"instruction_id,omitempty"` + xxx_hidden_Request isInstructionRequest_Request `protobuf_oneof:"request"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *InstructionRequest) Reset() { + *x = InstructionRequest{} + mi := &file_org_apache_beam_model_fn_execution_v1_beam_fn_api_proto_msgTypes[3] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *InstructionRequest) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*InstructionRequest) ProtoMessage() {} + +func (x *InstructionRequest) ProtoReflect() protoreflect.Message { + mi := &file_org_apache_beam_model_fn_execution_v1_beam_fn_api_proto_msgTypes[3] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +func (x *InstructionRequest) GetInstructionId() string { + if x != nil { + return x.xxx_hidden_InstructionId + } + return "" +} + +func (x *InstructionRequest) GetProcessBundle() *ProcessBundleRequest { + if x != nil { + if x, ok := x.xxx_hidden_Request.(*instructionRequest_ProcessBundle); ok { + return x.ProcessBundle + } + } + return nil +} + +func (x *InstructionRequest) GetProcessBundleProgress() *ProcessBundleProgressRequest { + if x != nil { + if x, ok := x.xxx_hidden_Request.(*instructionRequest_ProcessBundleProgress); ok { + return x.ProcessBundleProgress + } + } + return nil +} + +func (x *InstructionRequest) GetProcessBundleSplit() *ProcessBundleSplitRequest { + if x != nil { + if x, ok := x.xxx_hidden_Request.(*instructionRequest_ProcessBundleSplit); ok { + return x.ProcessBundleSplit + } + } + return nil +} + +func (x *InstructionRequest) GetFinalizeBundle() *FinalizeBundleRequest { + if x != nil { + if x, ok := x.xxx_hidden_Request.(*instructionRequest_FinalizeBundle); ok { + return x.FinalizeBundle + } + } + return nil +} + +func (x *InstructionRequest) GetMonitoringInfos() *MonitoringInfosMetadataRequest { + if x != nil { + if x, ok := x.xxx_hidden_Request.(*instructionRequest_MonitoringInfos); ok { + return x.MonitoringInfos + } + } + return nil +} + +func (x *InstructionRequest) GetHarnessMonitoringInfos() *HarnessMonitoringInfosRequest { + if x != nil { + if x, ok := x.xxx_hidden_Request.(*instructionRequest_HarnessMonitoringInfos); ok { + return x.HarnessMonitoringInfos + } + } + return nil +} + +func (x *InstructionRequest) GetSampleData() *SampleDataRequest { + if x != nil { + if x, ok := x.xxx_hidden_Request.(*instructionRequest_SampleData); ok { + return x.SampleData + } + } + return nil +} + +func (x *InstructionRequest) GetRegister() *RegisterRequest { + if x != nil { + if x, ok := x.xxx_hidden_Request.(*instructionRequest_Register); ok { + return x.Register + } + } + return nil +} + +func (x *InstructionRequest) SetInstructionId(v string) { + x.xxx_hidden_InstructionId = v +} + +func (x *InstructionRequest) SetProcessBundle(v *ProcessBundleRequest) { + if v == nil { + x.xxx_hidden_Request = nil + return + } + x.xxx_hidden_Request = &instructionRequest_ProcessBundle{v} +} + +func (x *InstructionRequest) SetProcessBundleProgress(v *ProcessBundleProgressRequest) { + if v == nil { + x.xxx_hidden_Request = nil + return + } + x.xxx_hidden_Request = &instructionRequest_ProcessBundleProgress{v} +} + +func (x *InstructionRequest) SetProcessBundleSplit(v *ProcessBundleSplitRequest) { + if v == nil { + x.xxx_hidden_Request = nil + return + } + x.xxx_hidden_Request = &instructionRequest_ProcessBundleSplit{v} +} + +func (x *InstructionRequest) SetFinalizeBundle(v *FinalizeBundleRequest) { + if v == nil { + x.xxx_hidden_Request = nil + return + } + x.xxx_hidden_Request = &instructionRequest_FinalizeBundle{v} +} + +func (x *InstructionRequest) SetMonitoringInfos(v *MonitoringInfosMetadataRequest) { + if v == nil { + x.xxx_hidden_Request = nil + return + } + x.xxx_hidden_Request = &instructionRequest_MonitoringInfos{v} +} + +func (x *InstructionRequest) SetHarnessMonitoringInfos(v *HarnessMonitoringInfosRequest) { + if v == nil { + x.xxx_hidden_Request = nil + return + } + x.xxx_hidden_Request = &instructionRequest_HarnessMonitoringInfos{v} +} + +func (x *InstructionRequest) SetSampleData(v *SampleDataRequest) { + if v == nil { + x.xxx_hidden_Request = nil + return + } + x.xxx_hidden_Request = &instructionRequest_SampleData{v} +} + +func (x *InstructionRequest) SetRegister(v *RegisterRequest) { + if v == nil { + x.xxx_hidden_Request = nil + return + } + x.xxx_hidden_Request = &instructionRequest_Register{v} +} + +func (x *InstructionRequest) HasRequest() bool { + if x == nil { + return false + } + return x.xxx_hidden_Request != nil +} + +func (x *InstructionRequest) HasProcessBundle() bool { + if x == nil { + return false + } + _, ok := x.xxx_hidden_Request.(*instructionRequest_ProcessBundle) + return ok +} + +func (x *InstructionRequest) HasProcessBundleProgress() bool { + if x == nil { + return false + } + _, ok := x.xxx_hidden_Request.(*instructionRequest_ProcessBundleProgress) + return ok +} + +func (x *InstructionRequest) HasProcessBundleSplit() bool { + if x == nil { + return false + } + _, ok := x.xxx_hidden_Request.(*instructionRequest_ProcessBundleSplit) + return ok +} + +func (x *InstructionRequest) HasFinalizeBundle() bool { + if x == nil { + return false + } + _, ok := x.xxx_hidden_Request.(*instructionRequest_FinalizeBundle) + return ok +} + +func (x *InstructionRequest) HasMonitoringInfos() bool { + if x == nil { + return false + } + _, ok := x.xxx_hidden_Request.(*instructionRequest_MonitoringInfos) + return ok +} + +func (x *InstructionRequest) HasHarnessMonitoringInfos() bool { + if x == nil { + return false + } + _, ok := x.xxx_hidden_Request.(*instructionRequest_HarnessMonitoringInfos) + return ok +} + +func (x *InstructionRequest) HasSampleData() bool { + if x == nil { + return false + } + _, ok := x.xxx_hidden_Request.(*instructionRequest_SampleData) + return ok +} + +func (x *InstructionRequest) HasRegister() bool { + if x == nil { + return false + } + _, ok := x.xxx_hidden_Request.(*instructionRequest_Register) + return ok +} + +func (x *InstructionRequest) ClearRequest() { + x.xxx_hidden_Request = nil +} + +func (x *InstructionRequest) ClearProcessBundle() { + if _, ok := x.xxx_hidden_Request.(*instructionRequest_ProcessBundle); ok { + x.xxx_hidden_Request = nil + } +} + +func (x *InstructionRequest) ClearProcessBundleProgress() { + if _, ok := x.xxx_hidden_Request.(*instructionRequest_ProcessBundleProgress); ok { + x.xxx_hidden_Request = nil + } +} + +func (x *InstructionRequest) ClearProcessBundleSplit() { + if _, ok := x.xxx_hidden_Request.(*instructionRequest_ProcessBundleSplit); ok { + x.xxx_hidden_Request = nil + } +} + +func (x *InstructionRequest) ClearFinalizeBundle() { + if _, ok := x.xxx_hidden_Request.(*instructionRequest_FinalizeBundle); ok { + x.xxx_hidden_Request = nil + } +} + +func (x *InstructionRequest) ClearMonitoringInfos() { + if _, ok := x.xxx_hidden_Request.(*instructionRequest_MonitoringInfos); ok { + x.xxx_hidden_Request = nil + } +} + +func (x *InstructionRequest) ClearHarnessMonitoringInfos() { + if _, ok := x.xxx_hidden_Request.(*instructionRequest_HarnessMonitoringInfos); ok { + x.xxx_hidden_Request = nil + } +} + +func (x *InstructionRequest) ClearSampleData() { + if _, ok := x.xxx_hidden_Request.(*instructionRequest_SampleData); ok { + x.xxx_hidden_Request = nil + } +} + +func (x *InstructionRequest) ClearRegister() { + if _, ok := x.xxx_hidden_Request.(*instructionRequest_Register); ok { + x.xxx_hidden_Request = nil + } +} + +const InstructionRequest_Request_not_set_case case_InstructionRequest_Request = 0 +const InstructionRequest_ProcessBundle_case case_InstructionRequest_Request = 1001 +const InstructionRequest_ProcessBundleProgress_case case_InstructionRequest_Request = 1002 +const InstructionRequest_ProcessBundleSplit_case case_InstructionRequest_Request = 1003 +const InstructionRequest_FinalizeBundle_case case_InstructionRequest_Request = 1004 +const InstructionRequest_MonitoringInfos_case case_InstructionRequest_Request = 1005 +const InstructionRequest_HarnessMonitoringInfos_case case_InstructionRequest_Request = 1006 +const InstructionRequest_SampleData_case case_InstructionRequest_Request = 1007 +const InstructionRequest_Register_case case_InstructionRequest_Request = 1000 + +func (x *InstructionRequest) WhichRequest() case_InstructionRequest_Request { + if x == nil { + return InstructionRequest_Request_not_set_case + } + switch x.xxx_hidden_Request.(type) { + case *instructionRequest_ProcessBundle: + return InstructionRequest_ProcessBundle_case + case *instructionRequest_ProcessBundleProgress: + return InstructionRequest_ProcessBundleProgress_case + case *instructionRequest_ProcessBundleSplit: + return InstructionRequest_ProcessBundleSplit_case + case *instructionRequest_FinalizeBundle: + return InstructionRequest_FinalizeBundle_case + case *instructionRequest_MonitoringInfos: + return InstructionRequest_MonitoringInfos_case + case *instructionRequest_HarnessMonitoringInfos: + return InstructionRequest_HarnessMonitoringInfos_case + case *instructionRequest_SampleData: + return InstructionRequest_SampleData_case + case *instructionRequest_Register: + return InstructionRequest_Register_case + default: + return InstructionRequest_Request_not_set_case + } +} + +type InstructionRequest_builder struct { + _ [0]func() // Prevents comparability and use of unkeyed literals for the builder. + + // (Required) A unique identifier provided by the runner which represents + // this requests execution. The InstructionResponse MUST have the matching id. + InstructionId string + // (Required) A request that the SDK Harness needs to interpret. + + // Fields of oneof xxx_hidden_Request: + ProcessBundle *ProcessBundleRequest + ProcessBundleProgress *ProcessBundleProgressRequest + ProcessBundleSplit *ProcessBundleSplitRequest + FinalizeBundle *FinalizeBundleRequest + MonitoringInfos *MonitoringInfosMetadataRequest + HarnessMonitoringInfos *HarnessMonitoringInfosRequest + SampleData *SampleDataRequest + // DEPRECATED + Register *RegisterRequest + // -- end of xxx_hidden_Request +} + +func (b0 InstructionRequest_builder) Build() *InstructionRequest { + m0 := &InstructionRequest{} + b, x := &b0, m0 + _, _ = b, x + x.xxx_hidden_InstructionId = b.InstructionId + if b.ProcessBundle != nil { + x.xxx_hidden_Request = &instructionRequest_ProcessBundle{b.ProcessBundle} + } + if b.ProcessBundleProgress != nil { + x.xxx_hidden_Request = &instructionRequest_ProcessBundleProgress{b.ProcessBundleProgress} + } + if b.ProcessBundleSplit != nil { + x.xxx_hidden_Request = &instructionRequest_ProcessBundleSplit{b.ProcessBundleSplit} + } + if b.FinalizeBundle != nil { + x.xxx_hidden_Request = &instructionRequest_FinalizeBundle{b.FinalizeBundle} + } + if b.MonitoringInfos != nil { + x.xxx_hidden_Request = &instructionRequest_MonitoringInfos{b.MonitoringInfos} + } + if b.HarnessMonitoringInfos != nil { + x.xxx_hidden_Request = &instructionRequest_HarnessMonitoringInfos{b.HarnessMonitoringInfos} + } + if b.SampleData != nil { + x.xxx_hidden_Request = &instructionRequest_SampleData{b.SampleData} + } + if b.Register != nil { + x.xxx_hidden_Request = &instructionRequest_Register{b.Register} + } + return m0 +} + +type case_InstructionRequest_Request protoreflect.FieldNumber + +func (x case_InstructionRequest_Request) String() string { + md := file_org_apache_beam_model_fn_execution_v1_beam_fn_api_proto_msgTypes[3].Descriptor() + if x == 0 { + return "not set" + } + return protoimpl.X.MessageFieldStringOf(md, protoreflect.FieldNumber(x)) +} + +type isInstructionRequest_Request interface { + isInstructionRequest_Request() +} + +type instructionRequest_ProcessBundle struct { + ProcessBundle *ProcessBundleRequest `protobuf:"bytes,1001,opt,name=process_bundle,json=processBundle,proto3,oneof"` +} + +type instructionRequest_ProcessBundleProgress struct { + ProcessBundleProgress *ProcessBundleProgressRequest `protobuf:"bytes,1002,opt,name=process_bundle_progress,json=processBundleProgress,proto3,oneof"` +} + +type instructionRequest_ProcessBundleSplit struct { + ProcessBundleSplit *ProcessBundleSplitRequest `protobuf:"bytes,1003,opt,name=process_bundle_split,json=processBundleSplit,proto3,oneof"` +} + +type instructionRequest_FinalizeBundle struct { + FinalizeBundle *FinalizeBundleRequest `protobuf:"bytes,1004,opt,name=finalize_bundle,json=finalizeBundle,proto3,oneof"` +} + +type instructionRequest_MonitoringInfos struct { + MonitoringInfos *MonitoringInfosMetadataRequest `protobuf:"bytes,1005,opt,name=monitoring_infos,json=monitoringInfos,proto3,oneof"` +} + +type instructionRequest_HarnessMonitoringInfos struct { + HarnessMonitoringInfos *HarnessMonitoringInfosRequest `protobuf:"bytes,1006,opt,name=harness_monitoring_infos,json=harnessMonitoringInfos,proto3,oneof"` +} + +type instructionRequest_SampleData struct { + SampleData *SampleDataRequest `protobuf:"bytes,1007,opt,name=sample_data,json=sampleData,proto3,oneof"` +} + +type instructionRequest_Register struct { + // DEPRECATED + Register *RegisterRequest `protobuf:"bytes,1000,opt,name=register,proto3,oneof"` +} + +func (*instructionRequest_ProcessBundle) isInstructionRequest_Request() {} + +func (*instructionRequest_ProcessBundleProgress) isInstructionRequest_Request() {} + +func (*instructionRequest_ProcessBundleSplit) isInstructionRequest_Request() {} + +func (*instructionRequest_FinalizeBundle) isInstructionRequest_Request() {} + +func (*instructionRequest_MonitoringInfos) isInstructionRequest_Request() {} + +func (*instructionRequest_HarnessMonitoringInfos) isInstructionRequest_Request() {} + +func (*instructionRequest_SampleData) isInstructionRequest_Request() {} + +func (*instructionRequest_Register) isInstructionRequest_Request() {} + +// The response for an associated request the SDK had been asked to fulfill. +// Stable +type InstructionResponse struct { + state protoimpl.MessageState `protogen:"opaque.v1"` + xxx_hidden_InstructionId string `protobuf:"bytes,1,opt,name=instruction_id,json=instructionId,proto3" json:"instruction_id,omitempty"` + xxx_hidden_Error string `protobuf:"bytes,2,opt,name=error,proto3" json:"error,omitempty"` + xxx_hidden_Response isInstructionResponse_Response `protobuf_oneof:"response"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *InstructionResponse) Reset() { + *x = InstructionResponse{} + mi := &file_org_apache_beam_model_fn_execution_v1_beam_fn_api_proto_msgTypes[4] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *InstructionResponse) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*InstructionResponse) ProtoMessage() {} + +func (x *InstructionResponse) ProtoReflect() protoreflect.Message { + mi := &file_org_apache_beam_model_fn_execution_v1_beam_fn_api_proto_msgTypes[4] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +func (x *InstructionResponse) GetInstructionId() string { + if x != nil { + return x.xxx_hidden_InstructionId + } + return "" +} + +func (x *InstructionResponse) GetError() string { + if x != nil { + return x.xxx_hidden_Error + } + return "" +} + +func (x *InstructionResponse) GetProcessBundle() *ProcessBundleResponse { + if x != nil { + if x, ok := x.xxx_hidden_Response.(*instructionResponse_ProcessBundle); ok { + return x.ProcessBundle + } + } + return nil +} + +func (x *InstructionResponse) GetProcessBundleProgress() *ProcessBundleProgressResponse { + if x != nil { + if x, ok := x.xxx_hidden_Response.(*instructionResponse_ProcessBundleProgress); ok { + return x.ProcessBundleProgress + } + } + return nil +} + +func (x *InstructionResponse) GetProcessBundleSplit() *ProcessBundleSplitResponse { + if x != nil { + if x, ok := x.xxx_hidden_Response.(*instructionResponse_ProcessBundleSplit); ok { + return x.ProcessBundleSplit + } + } + return nil +} + +func (x *InstructionResponse) GetFinalizeBundle() *FinalizeBundleResponse { + if x != nil { + if x, ok := x.xxx_hidden_Response.(*instructionResponse_FinalizeBundle); ok { + return x.FinalizeBundle + } + } + return nil +} + +func (x *InstructionResponse) GetMonitoringInfos() *MonitoringInfosMetadataResponse { + if x != nil { + if x, ok := x.xxx_hidden_Response.(*instructionResponse_MonitoringInfos); ok { + return x.MonitoringInfos + } + } + return nil +} + +func (x *InstructionResponse) GetHarnessMonitoringInfos() *HarnessMonitoringInfosResponse { + if x != nil { + if x, ok := x.xxx_hidden_Response.(*instructionResponse_HarnessMonitoringInfos); ok { + return x.HarnessMonitoringInfos + } + } + return nil +} + +func (x *InstructionResponse) GetSampleData() *SampleDataResponse { + if x != nil { + if x, ok := x.xxx_hidden_Response.(*instructionResponse_SampleData); ok { + return x.SampleData + } + } + return nil +} + +func (x *InstructionResponse) GetRegister() *RegisterResponse { + if x != nil { + if x, ok := x.xxx_hidden_Response.(*instructionResponse_Register); ok { + return x.Register + } + } + return nil +} + +func (x *InstructionResponse) SetInstructionId(v string) { + x.xxx_hidden_InstructionId = v +} + +func (x *InstructionResponse) SetError(v string) { + x.xxx_hidden_Error = v +} + +func (x *InstructionResponse) SetProcessBundle(v *ProcessBundleResponse) { + if v == nil { + x.xxx_hidden_Response = nil + return + } + x.xxx_hidden_Response = &instructionResponse_ProcessBundle{v} +} + +func (x *InstructionResponse) SetProcessBundleProgress(v *ProcessBundleProgressResponse) { + if v == nil { + x.xxx_hidden_Response = nil + return + } + x.xxx_hidden_Response = &instructionResponse_ProcessBundleProgress{v} +} + +func (x *InstructionResponse) SetProcessBundleSplit(v *ProcessBundleSplitResponse) { + if v == nil { + x.xxx_hidden_Response = nil + return + } + x.xxx_hidden_Response = &instructionResponse_ProcessBundleSplit{v} +} + +func (x *InstructionResponse) SetFinalizeBundle(v *FinalizeBundleResponse) { + if v == nil { + x.xxx_hidden_Response = nil + return + } + x.xxx_hidden_Response = &instructionResponse_FinalizeBundle{v} +} + +func (x *InstructionResponse) SetMonitoringInfos(v *MonitoringInfosMetadataResponse) { + if v == nil { + x.xxx_hidden_Response = nil + return + } + x.xxx_hidden_Response = &instructionResponse_MonitoringInfos{v} +} + +func (x *InstructionResponse) SetHarnessMonitoringInfos(v *HarnessMonitoringInfosResponse) { + if v == nil { + x.xxx_hidden_Response = nil + return + } + x.xxx_hidden_Response = &instructionResponse_HarnessMonitoringInfos{v} +} + +func (x *InstructionResponse) SetSampleData(v *SampleDataResponse) { + if v == nil { + x.xxx_hidden_Response = nil + return + } + x.xxx_hidden_Response = &instructionResponse_SampleData{v} +} + +func (x *InstructionResponse) SetRegister(v *RegisterResponse) { + if v == nil { + x.xxx_hidden_Response = nil + return + } + x.xxx_hidden_Response = &instructionResponse_Register{v} +} + +func (x *InstructionResponse) HasResponse() bool { + if x == nil { + return false + } + return x.xxx_hidden_Response != nil +} + +func (x *InstructionResponse) HasProcessBundle() bool { + if x == nil { + return false + } + _, ok := x.xxx_hidden_Response.(*instructionResponse_ProcessBundle) + return ok +} + +func (x *InstructionResponse) HasProcessBundleProgress() bool { + if x == nil { + return false + } + _, ok := x.xxx_hidden_Response.(*instructionResponse_ProcessBundleProgress) + return ok +} + +func (x *InstructionResponse) HasProcessBundleSplit() bool { + if x == nil { + return false + } + _, ok := x.xxx_hidden_Response.(*instructionResponse_ProcessBundleSplit) + return ok +} + +func (x *InstructionResponse) HasFinalizeBundle() bool { + if x == nil { + return false + } + _, ok := x.xxx_hidden_Response.(*instructionResponse_FinalizeBundle) + return ok +} + +func (x *InstructionResponse) HasMonitoringInfos() bool { + if x == nil { + return false + } + _, ok := x.xxx_hidden_Response.(*instructionResponse_MonitoringInfos) + return ok +} + +func (x *InstructionResponse) HasHarnessMonitoringInfos() bool { + if x == nil { + return false + } + _, ok := x.xxx_hidden_Response.(*instructionResponse_HarnessMonitoringInfos) + return ok +} + +func (x *InstructionResponse) HasSampleData() bool { + if x == nil { + return false + } + _, ok := x.xxx_hidden_Response.(*instructionResponse_SampleData) + return ok +} + +func (x *InstructionResponse) HasRegister() bool { + if x == nil { + return false + } + _, ok := x.xxx_hidden_Response.(*instructionResponse_Register) + return ok +} + +func (x *InstructionResponse) ClearResponse() { + x.xxx_hidden_Response = nil +} + +func (x *InstructionResponse) ClearProcessBundle() { + if _, ok := x.xxx_hidden_Response.(*instructionResponse_ProcessBundle); ok { + x.xxx_hidden_Response = nil + } +} + +func (x *InstructionResponse) ClearProcessBundleProgress() { + if _, ok := x.xxx_hidden_Response.(*instructionResponse_ProcessBundleProgress); ok { + x.xxx_hidden_Response = nil + } +} + +func (x *InstructionResponse) ClearProcessBundleSplit() { + if _, ok := x.xxx_hidden_Response.(*instructionResponse_ProcessBundleSplit); ok { + x.xxx_hidden_Response = nil + } +} + +func (x *InstructionResponse) ClearFinalizeBundle() { + if _, ok := x.xxx_hidden_Response.(*instructionResponse_FinalizeBundle); ok { + x.xxx_hidden_Response = nil + } +} + +func (x *InstructionResponse) ClearMonitoringInfos() { + if _, ok := x.xxx_hidden_Response.(*instructionResponse_MonitoringInfos); ok { + x.xxx_hidden_Response = nil + } +} + +func (x *InstructionResponse) ClearHarnessMonitoringInfos() { + if _, ok := x.xxx_hidden_Response.(*instructionResponse_HarnessMonitoringInfos); ok { + x.xxx_hidden_Response = nil + } +} + +func (x *InstructionResponse) ClearSampleData() { + if _, ok := x.xxx_hidden_Response.(*instructionResponse_SampleData); ok { + x.xxx_hidden_Response = nil + } +} + +func (x *InstructionResponse) ClearRegister() { + if _, ok := x.xxx_hidden_Response.(*instructionResponse_Register); ok { + x.xxx_hidden_Response = nil + } +} + +const InstructionResponse_Response_not_set_case case_InstructionResponse_Response = 0 +const InstructionResponse_ProcessBundle_case case_InstructionResponse_Response = 1001 +const InstructionResponse_ProcessBundleProgress_case case_InstructionResponse_Response = 1002 +const InstructionResponse_ProcessBundleSplit_case case_InstructionResponse_Response = 1003 +const InstructionResponse_FinalizeBundle_case case_InstructionResponse_Response = 1004 +const InstructionResponse_MonitoringInfos_case case_InstructionResponse_Response = 1005 +const InstructionResponse_HarnessMonitoringInfos_case case_InstructionResponse_Response = 1006 +const InstructionResponse_SampleData_case case_InstructionResponse_Response = 1007 +const InstructionResponse_Register_case case_InstructionResponse_Response = 1000 + +func (x *InstructionResponse) WhichResponse() case_InstructionResponse_Response { + if x == nil { + return InstructionResponse_Response_not_set_case + } + switch x.xxx_hidden_Response.(type) { + case *instructionResponse_ProcessBundle: + return InstructionResponse_ProcessBundle_case + case *instructionResponse_ProcessBundleProgress: + return InstructionResponse_ProcessBundleProgress_case + case *instructionResponse_ProcessBundleSplit: + return InstructionResponse_ProcessBundleSplit_case + case *instructionResponse_FinalizeBundle: + return InstructionResponse_FinalizeBundle_case + case *instructionResponse_MonitoringInfos: + return InstructionResponse_MonitoringInfos_case + case *instructionResponse_HarnessMonitoringInfos: + return InstructionResponse_HarnessMonitoringInfos_case + case *instructionResponse_SampleData: + return InstructionResponse_SampleData_case + case *instructionResponse_Register: + return InstructionResponse_Register_case + default: + return InstructionResponse_Response_not_set_case + } +} + +type InstructionResponse_builder struct { + _ [0]func() // Prevents comparability and use of unkeyed literals for the builder. + + // (Required) A reference provided by the runner which represents a requests + // execution. The InstructionResponse MUST have the matching id when + // responding to the runner. + InstructionId string + // If this is specified, then this instruction has failed. + // A human readable string representing the reason as to why processing has + // failed. + Error string + // If the instruction did not fail, it is required to return an equivalent + // response type depending on the request this matches. + + // Fields of oneof xxx_hidden_Response: + ProcessBundle *ProcessBundleResponse + ProcessBundleProgress *ProcessBundleProgressResponse + ProcessBundleSplit *ProcessBundleSplitResponse + FinalizeBundle *FinalizeBundleResponse + MonitoringInfos *MonitoringInfosMetadataResponse + HarnessMonitoringInfos *HarnessMonitoringInfosResponse + SampleData *SampleDataResponse + // DEPRECATED + Register *RegisterResponse + // -- end of xxx_hidden_Response +} + +func (b0 InstructionResponse_builder) Build() *InstructionResponse { + m0 := &InstructionResponse{} + b, x := &b0, m0 + _, _ = b, x + x.xxx_hidden_InstructionId = b.InstructionId + x.xxx_hidden_Error = b.Error + if b.ProcessBundle != nil { + x.xxx_hidden_Response = &instructionResponse_ProcessBundle{b.ProcessBundle} + } + if b.ProcessBundleProgress != nil { + x.xxx_hidden_Response = &instructionResponse_ProcessBundleProgress{b.ProcessBundleProgress} + } + if b.ProcessBundleSplit != nil { + x.xxx_hidden_Response = &instructionResponse_ProcessBundleSplit{b.ProcessBundleSplit} + } + if b.FinalizeBundle != nil { + x.xxx_hidden_Response = &instructionResponse_FinalizeBundle{b.FinalizeBundle} + } + if b.MonitoringInfos != nil { + x.xxx_hidden_Response = &instructionResponse_MonitoringInfos{b.MonitoringInfos} + } + if b.HarnessMonitoringInfos != nil { + x.xxx_hidden_Response = &instructionResponse_HarnessMonitoringInfos{b.HarnessMonitoringInfos} + } + if b.SampleData != nil { + x.xxx_hidden_Response = &instructionResponse_SampleData{b.SampleData} + } + if b.Register != nil { + x.xxx_hidden_Response = &instructionResponse_Register{b.Register} + } + return m0 +} + +type case_InstructionResponse_Response protoreflect.FieldNumber + +func (x case_InstructionResponse_Response) String() string { + md := file_org_apache_beam_model_fn_execution_v1_beam_fn_api_proto_msgTypes[4].Descriptor() + if x == 0 { + return "not set" + } + return protoimpl.X.MessageFieldStringOf(md, protoreflect.FieldNumber(x)) +} + +type isInstructionResponse_Response interface { + isInstructionResponse_Response() +} + +type instructionResponse_ProcessBundle struct { + ProcessBundle *ProcessBundleResponse `protobuf:"bytes,1001,opt,name=process_bundle,json=processBundle,proto3,oneof"` +} + +type instructionResponse_ProcessBundleProgress struct { + ProcessBundleProgress *ProcessBundleProgressResponse `protobuf:"bytes,1002,opt,name=process_bundle_progress,json=processBundleProgress,proto3,oneof"` +} + +type instructionResponse_ProcessBundleSplit struct { + ProcessBundleSplit *ProcessBundleSplitResponse `protobuf:"bytes,1003,opt,name=process_bundle_split,json=processBundleSplit,proto3,oneof"` +} + +type instructionResponse_FinalizeBundle struct { + FinalizeBundle *FinalizeBundleResponse `protobuf:"bytes,1004,opt,name=finalize_bundle,json=finalizeBundle,proto3,oneof"` +} + +type instructionResponse_MonitoringInfos struct { + MonitoringInfos *MonitoringInfosMetadataResponse `protobuf:"bytes,1005,opt,name=monitoring_infos,json=monitoringInfos,proto3,oneof"` +} + +type instructionResponse_HarnessMonitoringInfos struct { + HarnessMonitoringInfos *HarnessMonitoringInfosResponse `protobuf:"bytes,1006,opt,name=harness_monitoring_infos,json=harnessMonitoringInfos,proto3,oneof"` +} + +type instructionResponse_SampleData struct { + SampleData *SampleDataResponse `protobuf:"bytes,1007,opt,name=sample_data,json=sampleData,proto3,oneof"` +} + +type instructionResponse_Register struct { + // DEPRECATED + Register *RegisterResponse `protobuf:"bytes,1000,opt,name=register,proto3,oneof"` +} + +func (*instructionResponse_ProcessBundle) isInstructionResponse_Response() {} + +func (*instructionResponse_ProcessBundleProgress) isInstructionResponse_Response() {} + +func (*instructionResponse_ProcessBundleSplit) isInstructionResponse_Response() {} + +func (*instructionResponse_FinalizeBundle) isInstructionResponse_Response() {} + +func (*instructionResponse_MonitoringInfos) isInstructionResponse_Response() {} + +func (*instructionResponse_HarnessMonitoringInfos) isInstructionResponse_Response() {} + +func (*instructionResponse_SampleData) isInstructionResponse_Response() {} + +func (*instructionResponse_Register) isInstructionResponse_Response() {} + +// If supported, the `SampleDataRequest` will respond with a +// `SampleDataResponse`. The SDK being queried must have the +// "beam:protocol:data_sampling:v1" capability. Samples are taken only from the +// specified PCollection ids. An empty list will return everything. +type SampleDataRequest struct { + state protoimpl.MessageState `protogen:"opaque.v1"` + xxx_hidden_PcollectionIds []string `protobuf:"bytes,1,rep,name=pcollection_ids,json=pcollectionIds,proto3" json:"pcollection_ids,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *SampleDataRequest) Reset() { + *x = SampleDataRequest{} + mi := &file_org_apache_beam_model_fn_execution_v1_beam_fn_api_proto_msgTypes[5] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *SampleDataRequest) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*SampleDataRequest) ProtoMessage() {} + +func (x *SampleDataRequest) ProtoReflect() protoreflect.Message { + mi := &file_org_apache_beam_model_fn_execution_v1_beam_fn_api_proto_msgTypes[5] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +func (x *SampleDataRequest) GetPcollectionIds() []string { + if x != nil { + return x.xxx_hidden_PcollectionIds + } + return nil +} + +func (x *SampleDataRequest) SetPcollectionIds(v []string) { + x.xxx_hidden_PcollectionIds = v +} + +type SampleDataRequest_builder struct { + _ [0]func() // Prevents comparability and use of unkeyed literals for the builder. + + // (Optional) The PCollection ids to filter for. + PcollectionIds []string +} + +func (b0 SampleDataRequest_builder) Build() *SampleDataRequest { + m0 := &SampleDataRequest{} + b, x := &b0, m0 + _, _ = b, x + x.xxx_hidden_PcollectionIds = b.PcollectionIds + return m0 +} + +// An element sampled when the SDK is processing a bundle. This is a proto +// message to allow for additional per-element metadata. +type SampledElement struct { + state protoimpl.MessageState `protogen:"opaque.v1"` + xxx_hidden_Element []byte `protobuf:"bytes,1,opt,name=element,proto3" json:"element,omitempty"` + xxx_hidden_SampleTimestamp *timestamppb.Timestamp `protobuf:"bytes,2,opt,name=sample_timestamp,json=sampleTimestamp,proto3" json:"sample_timestamp,omitempty"` + xxx_hidden_Exception *SampledElement_Exception `protobuf:"bytes,3,opt,name=exception,proto3" json:"exception,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *SampledElement) Reset() { + *x = SampledElement{} + mi := &file_org_apache_beam_model_fn_execution_v1_beam_fn_api_proto_msgTypes[6] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *SampledElement) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*SampledElement) ProtoMessage() {} + +func (x *SampledElement) ProtoReflect() protoreflect.Message { + mi := &file_org_apache_beam_model_fn_execution_v1_beam_fn_api_proto_msgTypes[6] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +func (x *SampledElement) GetElement() []byte { + if x != nil { + return x.xxx_hidden_Element + } + return nil +} + +func (x *SampledElement) GetSampleTimestamp() *timestamppb.Timestamp { + if x != nil { + return x.xxx_hidden_SampleTimestamp + } + return nil +} + +func (x *SampledElement) GetException() *SampledElement_Exception { + if x != nil { + return x.xxx_hidden_Exception + } + return nil +} + +func (x *SampledElement) SetElement(v []byte) { + if v == nil { + v = []byte{} + } + x.xxx_hidden_Element = v +} + +func (x *SampledElement) SetSampleTimestamp(v *timestamppb.Timestamp) { + x.xxx_hidden_SampleTimestamp = v +} + +func (x *SampledElement) SetException(v *SampledElement_Exception) { + x.xxx_hidden_Exception = v +} + +func (x *SampledElement) HasSampleTimestamp() bool { + if x == nil { + return false + } + return x.xxx_hidden_SampleTimestamp != nil +} + +func (x *SampledElement) HasException() bool { + if x == nil { + return false + } + return x.xxx_hidden_Exception != nil +} + +func (x *SampledElement) ClearSampleTimestamp() { + x.xxx_hidden_SampleTimestamp = nil +} + +func (x *SampledElement) ClearException() { + x.xxx_hidden_Exception = nil +} + +type SampledElement_builder struct { + _ [0]func() // Prevents comparability and use of unkeyed literals for the builder. + + // (Required) Sampled raw bytes for an element. This is a + // single encoded element in the nested context. + Element []byte + // (Required) Timestamp of when the sample was taken. + SampleTimestamp *timestamppb.Timestamp + // (Optional) This will be set if this element was sampled because of a user + // exception. + Exception *SampledElement_Exception +} + +func (b0 SampledElement_builder) Build() *SampledElement { + m0 := &SampledElement{} + b, x := &b0, m0 + _, _ = b, x + x.xxx_hidden_Element = b.Element + x.xxx_hidden_SampleTimestamp = b.SampleTimestamp + x.xxx_hidden_Exception = b.Exception + return m0 +} + +// If supported, the `SampleDataResponse` will contain samples from PCollections +// based upon the filters specified in the request. +type SampleDataResponse struct { + state protoimpl.MessageState `protogen:"opaque.v1"` + xxx_hidden_ElementSamples map[string]*SampleDataResponse_ElementList `protobuf:"bytes,1,rep,name=element_samples,json=elementSamples,proto3" json:"element_samples,omitempty" protobuf_key:"bytes,1,opt,name=key" protobuf_val:"bytes,2,opt,name=value"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *SampleDataResponse) Reset() { + *x = SampleDataResponse{} + mi := &file_org_apache_beam_model_fn_execution_v1_beam_fn_api_proto_msgTypes[7] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *SampleDataResponse) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*SampleDataResponse) ProtoMessage() {} + +func (x *SampleDataResponse) ProtoReflect() protoreflect.Message { + mi := &file_org_apache_beam_model_fn_execution_v1_beam_fn_api_proto_msgTypes[7] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +func (x *SampleDataResponse) GetElementSamples() map[string]*SampleDataResponse_ElementList { + if x != nil { + return x.xxx_hidden_ElementSamples + } + return nil +} + +func (x *SampleDataResponse) SetElementSamples(v map[string]*SampleDataResponse_ElementList) { + x.xxx_hidden_ElementSamples = v +} + +type SampleDataResponse_builder struct { + _ [0]func() // Prevents comparability and use of unkeyed literals for the builder. + + // Map from PCollection id to sampled elements. + ElementSamples map[string]*SampleDataResponse_ElementList +} + +func (b0 SampleDataResponse_builder) Build() *SampleDataResponse { + m0 := &SampleDataResponse{} + b, x := &b0, m0 + _, _ = b, x + x.xxx_hidden_ElementSamples = b.ElementSamples + return m0 +} + +// A request to provide full MonitoringInfo associated with the entire SDK +// harness process, not specific to a bundle. +// +// An SDK can report metrics using an identifier that only contains the +// associated payload. A runner who wants to receive the full metrics +// information can request all the monitoring metadata via a +// MonitoringInfosMetadataRequest providing a list of ids as necessary. +// +// The SDK is allowed to reuse the identifiers +// for the lifetime of the associated control connection as long +// as the MonitoringInfo could be reconstructed fully by overwriting its +// payload field with the bytes specified here. +type HarnessMonitoringInfosRequest struct { + state protoimpl.MessageState `protogen:"opaque.v1"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *HarnessMonitoringInfosRequest) Reset() { + *x = HarnessMonitoringInfosRequest{} + mi := &file_org_apache_beam_model_fn_execution_v1_beam_fn_api_proto_msgTypes[8] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *HarnessMonitoringInfosRequest) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*HarnessMonitoringInfosRequest) ProtoMessage() {} + +func (x *HarnessMonitoringInfosRequest) ProtoReflect() protoreflect.Message { + mi := &file_org_apache_beam_model_fn_execution_v1_beam_fn_api_proto_msgTypes[8] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +type HarnessMonitoringInfosRequest_builder struct { + _ [0]func() // Prevents comparability and use of unkeyed literals for the builder. + +} + +func (b0 HarnessMonitoringInfosRequest_builder) Build() *HarnessMonitoringInfosRequest { + m0 := &HarnessMonitoringInfosRequest{} + b, x := &b0, m0 + _, _ = b, x + return m0 +} + +type HarnessMonitoringInfosResponse struct { + state protoimpl.MessageState `protogen:"opaque.v1"` + xxx_hidden_MonitoringData map[string][]byte `protobuf:"bytes,1,rep,name=monitoring_data,json=monitoringData,proto3" json:"monitoring_data,omitempty" protobuf_key:"bytes,1,opt,name=key" protobuf_val:"bytes,2,opt,name=value"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *HarnessMonitoringInfosResponse) Reset() { + *x = HarnessMonitoringInfosResponse{} + mi := &file_org_apache_beam_model_fn_execution_v1_beam_fn_api_proto_msgTypes[9] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *HarnessMonitoringInfosResponse) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*HarnessMonitoringInfosResponse) ProtoMessage() {} + +func (x *HarnessMonitoringInfosResponse) ProtoReflect() protoreflect.Message { + mi := &file_org_apache_beam_model_fn_execution_v1_beam_fn_api_proto_msgTypes[9] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +func (x *HarnessMonitoringInfosResponse) GetMonitoringData() map[string][]byte { + if x != nil { + return x.xxx_hidden_MonitoringData + } + return nil +} + +func (x *HarnessMonitoringInfosResponse) SetMonitoringData(v map[string][]byte) { + x.xxx_hidden_MonitoringData = v +} + +type HarnessMonitoringInfosResponse_builder struct { + _ [0]func() // Prevents comparability and use of unkeyed literals for the builder. + + // An identifier to MonitoringInfo.payload mapping containing + // Metrics associated with the SDK harness, not a specific bundle. + // + // An SDK can report metrics using an identifier that only contains the + // associated payload. A runner who wants to receive the full metrics + // information can request all the monitoring metadata via a + // MonitoringInfosMetadataRequest providing a list of ids as necessary. + // + // The SDK is allowed to reuse the identifiers + // for the lifetime of the associated control connection as long + // as the MonitoringInfo could be reconstructed fully by overwriting its + // payload field with the bytes specified here. + MonitoringData map[string][]byte +} + +func (b0 HarnessMonitoringInfosResponse_builder) Build() *HarnessMonitoringInfosResponse { + m0 := &HarnessMonitoringInfosResponse{} + b, x := &b0, m0 + _, _ = b, x + x.xxx_hidden_MonitoringData = b.MonitoringData + return m0 +} + +// A list of objects which can be referred to by the runner in +// future requests. +// Stable +type RegisterRequest struct { + state protoimpl.MessageState `protogen:"opaque.v1"` + xxx_hidden_ProcessBundleDescriptor *[]*ProcessBundleDescriptor `protobuf:"bytes,1,rep,name=process_bundle_descriptor,json=processBundleDescriptor,proto3" json:"process_bundle_descriptor,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *RegisterRequest) Reset() { + *x = RegisterRequest{} + mi := &file_org_apache_beam_model_fn_execution_v1_beam_fn_api_proto_msgTypes[10] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *RegisterRequest) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*RegisterRequest) ProtoMessage() {} + +func (x *RegisterRequest) ProtoReflect() protoreflect.Message { + mi := &file_org_apache_beam_model_fn_execution_v1_beam_fn_api_proto_msgTypes[10] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +func (x *RegisterRequest) GetProcessBundleDescriptor() []*ProcessBundleDescriptor { + if x != nil { + if x.xxx_hidden_ProcessBundleDescriptor != nil { + return *x.xxx_hidden_ProcessBundleDescriptor + } + } + return nil +} + +func (x *RegisterRequest) SetProcessBundleDescriptor(v []*ProcessBundleDescriptor) { + x.xxx_hidden_ProcessBundleDescriptor = &v +} + +type RegisterRequest_builder struct { + _ [0]func() // Prevents comparability and use of unkeyed literals for the builder. + + // (Optional) The set of descriptors used to process bundles. + ProcessBundleDescriptor []*ProcessBundleDescriptor +} + +func (b0 RegisterRequest_builder) Build() *RegisterRequest { + m0 := &RegisterRequest{} + b, x := &b0, m0 + _, _ = b, x + x.xxx_hidden_ProcessBundleDescriptor = &b.ProcessBundleDescriptor + return m0 +} + +// Stable +type RegisterResponse struct { + state protoimpl.MessageState `protogen:"opaque.v1"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *RegisterResponse) Reset() { + *x = RegisterResponse{} + mi := &file_org_apache_beam_model_fn_execution_v1_beam_fn_api_proto_msgTypes[11] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *RegisterResponse) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*RegisterResponse) ProtoMessage() {} + +func (x *RegisterResponse) ProtoReflect() protoreflect.Message { + mi := &file_org_apache_beam_model_fn_execution_v1_beam_fn_api_proto_msgTypes[11] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +type RegisterResponse_builder struct { + _ [0]func() // Prevents comparability and use of unkeyed literals for the builder. + +} + +func (b0 RegisterResponse_builder) Build() *RegisterResponse { + m0 := &RegisterResponse{} + b, x := &b0, m0 + _, _ = b, x + return m0 +} + +// Definitions that should be used to construct the bundle processing graph. +type ProcessBundleDescriptor struct { + state protoimpl.MessageState `protogen:"opaque.v1"` + xxx_hidden_Id string `protobuf:"bytes,1,opt,name=id,proto3" json:"id,omitempty"` + xxx_hidden_Transforms map[string]*pipeline_v1.PTransform `protobuf:"bytes,2,rep,name=transforms,proto3" json:"transforms,omitempty" protobuf_key:"bytes,1,opt,name=key" protobuf_val:"bytes,2,opt,name=value"` + xxx_hidden_Pcollections map[string]*pipeline_v1.PCollection `protobuf:"bytes,3,rep,name=pcollections,proto3" json:"pcollections,omitempty" protobuf_key:"bytes,1,opt,name=key" protobuf_val:"bytes,2,opt,name=value"` + xxx_hidden_WindowingStrategies map[string]*pipeline_v1.WindowingStrategy `protobuf:"bytes,4,rep,name=windowing_strategies,json=windowingStrategies,proto3" json:"windowing_strategies,omitempty" protobuf_key:"bytes,1,opt,name=key" protobuf_val:"bytes,2,opt,name=value"` + xxx_hidden_Coders map[string]*pipeline_v1.Coder `protobuf:"bytes,5,rep,name=coders,proto3" json:"coders,omitempty" protobuf_key:"bytes,1,opt,name=key" protobuf_val:"bytes,2,opt,name=value"` + xxx_hidden_Environments map[string]*pipeline_v1.Environment `protobuf:"bytes,6,rep,name=environments,proto3" json:"environments,omitempty" protobuf_key:"bytes,1,opt,name=key" protobuf_val:"bytes,2,opt,name=value"` + xxx_hidden_StateApiServiceDescriptor *pipeline_v1.ApiServiceDescriptor `protobuf:"bytes,7,opt,name=state_api_service_descriptor,json=stateApiServiceDescriptor,proto3" json:"state_api_service_descriptor,omitempty"` + xxx_hidden_TimerApiServiceDescriptor *pipeline_v1.ApiServiceDescriptor `protobuf:"bytes,8,opt,name=timer_api_service_descriptor,json=timerApiServiceDescriptor,proto3" json:"timer_api_service_descriptor,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *ProcessBundleDescriptor) Reset() { + *x = ProcessBundleDescriptor{} + mi := &file_org_apache_beam_model_fn_execution_v1_beam_fn_api_proto_msgTypes[12] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *ProcessBundleDescriptor) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*ProcessBundleDescriptor) ProtoMessage() {} + +func (x *ProcessBundleDescriptor) ProtoReflect() protoreflect.Message { + mi := &file_org_apache_beam_model_fn_execution_v1_beam_fn_api_proto_msgTypes[12] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +func (x *ProcessBundleDescriptor) GetId() string { + if x != nil { + return x.xxx_hidden_Id + } + return "" +} + +func (x *ProcessBundleDescriptor) GetTransforms() map[string]*pipeline_v1.PTransform { + if x != nil { + return x.xxx_hidden_Transforms + } + return nil +} + +func (x *ProcessBundleDescriptor) GetPcollections() map[string]*pipeline_v1.PCollection { + if x != nil { + return x.xxx_hidden_Pcollections + } + return nil +} + +func (x *ProcessBundleDescriptor) GetWindowingStrategies() map[string]*pipeline_v1.WindowingStrategy { + if x != nil { + return x.xxx_hidden_WindowingStrategies + } + return nil +} + +func (x *ProcessBundleDescriptor) GetCoders() map[string]*pipeline_v1.Coder { + if x != nil { + return x.xxx_hidden_Coders + } + return nil +} + +func (x *ProcessBundleDescriptor) GetEnvironments() map[string]*pipeline_v1.Environment { + if x != nil { + return x.xxx_hidden_Environments + } + return nil +} + +func (x *ProcessBundleDescriptor) GetStateApiServiceDescriptor() *pipeline_v1.ApiServiceDescriptor { + if x != nil { + return x.xxx_hidden_StateApiServiceDescriptor + } + return nil +} + +func (x *ProcessBundleDescriptor) GetTimerApiServiceDescriptor() *pipeline_v1.ApiServiceDescriptor { + if x != nil { + return x.xxx_hidden_TimerApiServiceDescriptor + } + return nil +} + +func (x *ProcessBundleDescriptor) SetId(v string) { + x.xxx_hidden_Id = v +} + +func (x *ProcessBundleDescriptor) SetTransforms(v map[string]*pipeline_v1.PTransform) { + x.xxx_hidden_Transforms = v +} + +func (x *ProcessBundleDescriptor) SetPcollections(v map[string]*pipeline_v1.PCollection) { + x.xxx_hidden_Pcollections = v +} + +func (x *ProcessBundleDescriptor) SetWindowingStrategies(v map[string]*pipeline_v1.WindowingStrategy) { + x.xxx_hidden_WindowingStrategies = v +} + +func (x *ProcessBundleDescriptor) SetCoders(v map[string]*pipeline_v1.Coder) { + x.xxx_hidden_Coders = v +} + +func (x *ProcessBundleDescriptor) SetEnvironments(v map[string]*pipeline_v1.Environment) { + x.xxx_hidden_Environments = v +} + +func (x *ProcessBundleDescriptor) SetStateApiServiceDescriptor(v *pipeline_v1.ApiServiceDescriptor) { + x.xxx_hidden_StateApiServiceDescriptor = v +} + +func (x *ProcessBundleDescriptor) SetTimerApiServiceDescriptor(v *pipeline_v1.ApiServiceDescriptor) { + x.xxx_hidden_TimerApiServiceDescriptor = v +} + +func (x *ProcessBundleDescriptor) HasStateApiServiceDescriptor() bool { + if x == nil { + return false + } + return x.xxx_hidden_StateApiServiceDescriptor != nil +} + +func (x *ProcessBundleDescriptor) HasTimerApiServiceDescriptor() bool { + if x == nil { + return false + } + return x.xxx_hidden_TimerApiServiceDescriptor != nil +} + +func (x *ProcessBundleDescriptor) ClearStateApiServiceDescriptor() { + x.xxx_hidden_StateApiServiceDescriptor = nil +} + +func (x *ProcessBundleDescriptor) ClearTimerApiServiceDescriptor() { + x.xxx_hidden_TimerApiServiceDescriptor = nil +} + +type ProcessBundleDescriptor_builder struct { + _ [0]func() // Prevents comparability and use of unkeyed literals for the builder. + + // (Required) A pipeline level unique id which can be used as a reference to + // refer to this. + Id string + // (Required) A map from pipeline-scoped id to PTransform. + Transforms map[string]*pipeline_v1.PTransform + // (Required) A map from pipeline-scoped id to PCollection. + Pcollections map[string]*pipeline_v1.PCollection + // (Required) A map from pipeline-scoped id to WindowingStrategy. + WindowingStrategies map[string]*pipeline_v1.WindowingStrategy + // (Required) A map from pipeline-scoped id to Coder. + Coders map[string]*pipeline_v1.Coder + // (Required) A map from pipeline-scoped id to Environment. + Environments map[string]*pipeline_v1.Environment + // A descriptor describing the end point to use for State API + // calls. Required if the Runner intends to send remote references over the + // data plane or if any of the transforms rely on user state or side inputs. + StateApiServiceDescriptor *pipeline_v1.ApiServiceDescriptor + // A descriptor describing the end point to use for Data API for user timers. + // Required if the ProcessBundleDescriptor contains any transforms that have user timers. + TimerApiServiceDescriptor *pipeline_v1.ApiServiceDescriptor +} + +func (b0 ProcessBundleDescriptor_builder) Build() *ProcessBundleDescriptor { + m0 := &ProcessBundleDescriptor{} + b, x := &b0, m0 + _, _ = b, x + x.xxx_hidden_Id = b.Id + x.xxx_hidden_Transforms = b.Transforms + x.xxx_hidden_Pcollections = b.Pcollections + x.xxx_hidden_WindowingStrategies = b.WindowingStrategies + x.xxx_hidden_Coders = b.Coders + x.xxx_hidden_Environments = b.Environments + x.xxx_hidden_StateApiServiceDescriptor = b.StateApiServiceDescriptor + x.xxx_hidden_TimerApiServiceDescriptor = b.TimerApiServiceDescriptor + return m0 +} + +// One of the applications specifying the scope of work for a bundle. +// See +// https://docs.google.com/document/d/1tUDb45sStdR8u7-jBkGdw3OGFK7aa2-V7eo86zYSE_4/edit#heading=h.9g3g5weg2u9 +// for further details. +type BundleApplication struct { + state protoimpl.MessageState `protogen:"opaque.v1"` + xxx_hidden_TransformId string `protobuf:"bytes,1,opt,name=transform_id,json=transformId,proto3" json:"transform_id,omitempty"` + xxx_hidden_InputId string `protobuf:"bytes,2,opt,name=input_id,json=inputId,proto3" json:"input_id,omitempty"` + xxx_hidden_Element []byte `protobuf:"bytes,3,opt,name=element,proto3" json:"element,omitempty"` + xxx_hidden_OutputWatermarks map[string]*timestamppb.Timestamp `protobuf:"bytes,4,rep,name=output_watermarks,json=outputWatermarks,proto3" json:"output_watermarks,omitempty" protobuf_key:"bytes,1,opt,name=key" protobuf_val:"bytes,2,opt,name=value"` + xxx_hidden_IsBounded pipeline_v1.IsBounded_Enum `protobuf:"varint,5,opt,name=is_bounded,json=isBounded,proto3,enum=org.apache.beam.model.pipeline.v1.IsBounded_Enum" json:"is_bounded,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *BundleApplication) Reset() { + *x = BundleApplication{} + mi := &file_org_apache_beam_model_fn_execution_v1_beam_fn_api_proto_msgTypes[13] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *BundleApplication) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*BundleApplication) ProtoMessage() {} + +func (x *BundleApplication) ProtoReflect() protoreflect.Message { + mi := &file_org_apache_beam_model_fn_execution_v1_beam_fn_api_proto_msgTypes[13] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +func (x *BundleApplication) GetTransformId() string { + if x != nil { + return x.xxx_hidden_TransformId + } + return "" +} + +func (x *BundleApplication) GetInputId() string { + if x != nil { + return x.xxx_hidden_InputId + } + return "" +} + +func (x *BundleApplication) GetElement() []byte { + if x != nil { + return x.xxx_hidden_Element + } + return nil +} + +func (x *BundleApplication) GetOutputWatermarks() map[string]*timestamppb.Timestamp { + if x != nil { + return x.xxx_hidden_OutputWatermarks + } + return nil +} + +func (x *BundleApplication) GetIsBounded() pipeline_v1.IsBounded_Enum { + if x != nil { + return x.xxx_hidden_IsBounded + } + return pipeline_v1.IsBounded_Enum(0) +} + +func (x *BundleApplication) SetTransformId(v string) { + x.xxx_hidden_TransformId = v +} + +func (x *BundleApplication) SetInputId(v string) { + x.xxx_hidden_InputId = v +} + +func (x *BundleApplication) SetElement(v []byte) { + if v == nil { + v = []byte{} + } + x.xxx_hidden_Element = v +} + +func (x *BundleApplication) SetOutputWatermarks(v map[string]*timestamppb.Timestamp) { + x.xxx_hidden_OutputWatermarks = v +} + +func (x *BundleApplication) SetIsBounded(v pipeline_v1.IsBounded_Enum) { + x.xxx_hidden_IsBounded = v +} + +type BundleApplication_builder struct { + _ [0]func() // Prevents comparability and use of unkeyed literals for the builder. + + // (Required) The transform to which to pass the element + TransformId string + // (Required) Name of the transform's input to which to pass the element. + InputId string + // (Required) The encoded element to pass to the transform. + Element []byte + // The map is keyed by the local output name of the PTransform. Each + // value represents a lower bound on the timestamps of elements that + // are produced by this PTransform into each of its output PCollections + // when invoked with this application. + // + // If there is no watermark reported from RestrictionTracker, the runner will + // use MIN_TIMESTAMP by default. + OutputWatermarks map[string]*timestamppb.Timestamp + // Whether this application potentially produces an unbounded + // amount of data. Note that this should only be set to BOUNDED if and + // only if the application is known to produce a finite amount of output. + IsBounded pipeline_v1.IsBounded_Enum +} + +func (b0 BundleApplication_builder) Build() *BundleApplication { + m0 := &BundleApplication{} + b, x := &b0, m0 + _, _ = b, x + x.xxx_hidden_TransformId = b.TransformId + x.xxx_hidden_InputId = b.InputId + x.xxx_hidden_Element = b.Element + x.xxx_hidden_OutputWatermarks = b.OutputWatermarks + x.xxx_hidden_IsBounded = b.IsBounded + return m0 +} + +// An Application should be scheduled for execution after a delay. +// Either an absolute timestamp or a relative timestamp can represent a +// scheduled execution time. +type DelayedBundleApplication struct { + state protoimpl.MessageState `protogen:"opaque.v1"` + xxx_hidden_Application *BundleApplication `protobuf:"bytes,1,opt,name=application,proto3" json:"application,omitempty"` + xxx_hidden_RequestedTimeDelay *durationpb.Duration `protobuf:"bytes,2,opt,name=requested_time_delay,json=requestedTimeDelay,proto3" json:"requested_time_delay,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *DelayedBundleApplication) Reset() { + *x = DelayedBundleApplication{} + mi := &file_org_apache_beam_model_fn_execution_v1_beam_fn_api_proto_msgTypes[14] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *DelayedBundleApplication) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*DelayedBundleApplication) ProtoMessage() {} + +func (x *DelayedBundleApplication) ProtoReflect() protoreflect.Message { + mi := &file_org_apache_beam_model_fn_execution_v1_beam_fn_api_proto_msgTypes[14] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +func (x *DelayedBundleApplication) GetApplication() *BundleApplication { + if x != nil { + return x.xxx_hidden_Application + } + return nil +} + +func (x *DelayedBundleApplication) GetRequestedTimeDelay() *durationpb.Duration { + if x != nil { + return x.xxx_hidden_RequestedTimeDelay + } + return nil +} + +func (x *DelayedBundleApplication) SetApplication(v *BundleApplication) { + x.xxx_hidden_Application = v +} + +func (x *DelayedBundleApplication) SetRequestedTimeDelay(v *durationpb.Duration) { + x.xxx_hidden_RequestedTimeDelay = v +} + +func (x *DelayedBundleApplication) HasApplication() bool { + if x == nil { + return false + } + return x.xxx_hidden_Application != nil +} + +func (x *DelayedBundleApplication) HasRequestedTimeDelay() bool { + if x == nil { + return false + } + return x.xxx_hidden_RequestedTimeDelay != nil +} + +func (x *DelayedBundleApplication) ClearApplication() { + x.xxx_hidden_Application = nil +} + +func (x *DelayedBundleApplication) ClearRequestedTimeDelay() { + x.xxx_hidden_RequestedTimeDelay = nil +} + +type DelayedBundleApplication_builder struct { + _ [0]func() // Prevents comparability and use of unkeyed literals for the builder. + + // (Required) The application that should be scheduled. + Application *BundleApplication + // Recommended time delay at which the application should be scheduled to + // execute by the runner. Time delay that equals 0 may be scheduled to execute + // immediately. The unit of time delay should be microsecond. + RequestedTimeDelay *durationpb.Duration +} + +func (b0 DelayedBundleApplication_builder) Build() *DelayedBundleApplication { + m0 := &DelayedBundleApplication{} + b, x := &b0, m0 + _, _ = b, x + x.xxx_hidden_Application = b.Application + x.xxx_hidden_RequestedTimeDelay = b.RequestedTimeDelay + return m0 +} + +// A request to process a given bundle. +// Stable +type ProcessBundleRequest struct { + state protoimpl.MessageState `protogen:"opaque.v1"` + xxx_hidden_ProcessBundleDescriptorId string `protobuf:"bytes,1,opt,name=process_bundle_descriptor_id,json=processBundleDescriptorId,proto3" json:"process_bundle_descriptor_id,omitempty"` + xxx_hidden_CacheTokens *[]*ProcessBundleRequest_CacheToken `protobuf:"bytes,2,rep,name=cache_tokens,json=cacheTokens,proto3" json:"cache_tokens,omitempty"` + xxx_hidden_Elements *Elements `protobuf:"bytes,3,opt,name=elements,proto3" json:"elements,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *ProcessBundleRequest) Reset() { + *x = ProcessBundleRequest{} + mi := &file_org_apache_beam_model_fn_execution_v1_beam_fn_api_proto_msgTypes[15] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *ProcessBundleRequest) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*ProcessBundleRequest) ProtoMessage() {} + +func (x *ProcessBundleRequest) ProtoReflect() protoreflect.Message { + mi := &file_org_apache_beam_model_fn_execution_v1_beam_fn_api_proto_msgTypes[15] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +func (x *ProcessBundleRequest) GetProcessBundleDescriptorId() string { + if x != nil { + return x.xxx_hidden_ProcessBundleDescriptorId + } + return "" +} + +func (x *ProcessBundleRequest) GetCacheTokens() []*ProcessBundleRequest_CacheToken { + if x != nil { + if x.xxx_hidden_CacheTokens != nil { + return *x.xxx_hidden_CacheTokens + } + } + return nil +} + +func (x *ProcessBundleRequest) GetElements() *Elements { + if x != nil { + return x.xxx_hidden_Elements + } + return nil +} + +func (x *ProcessBundleRequest) SetProcessBundleDescriptorId(v string) { + x.xxx_hidden_ProcessBundleDescriptorId = v +} + +func (x *ProcessBundleRequest) SetCacheTokens(v []*ProcessBundleRequest_CacheToken) { + x.xxx_hidden_CacheTokens = &v +} + +func (x *ProcessBundleRequest) SetElements(v *Elements) { + x.xxx_hidden_Elements = v +} + +func (x *ProcessBundleRequest) HasElements() bool { + if x == nil { + return false + } + return x.xxx_hidden_Elements != nil +} + +func (x *ProcessBundleRequest) ClearElements() { + x.xxx_hidden_Elements = nil +} + +type ProcessBundleRequest_builder struct { + _ [0]func() // Prevents comparability and use of unkeyed literals for the builder. + + // (Required) A reference to the process bundle descriptor that must be + // instantiated and executed by the SDK harness. + ProcessBundleDescriptorId string + // (Optional) A list of cache tokens that can be used by an SDK to reuse + // cached data returned by the State API across multiple bundles. + // + // Note that SDKs that can efficiently consume this field should declare + // the beam:protocol:state_caching:v1 capability enabling runners to reduce + // the amount of memory used. + // + // See https://s.apache.org/beam-fn-state-api-and-bundle-processing#heading=h.7ghoih5aig5m + // for additional details on how to use the cache token with the State API + // to cache data across bundle boundaries. + CacheTokens []*ProcessBundleRequest_CacheToken + // (Optional) Elements to be processed with the bundle. Either all or + // none of the bundle elements should be included in the ProcessBundleRequest. + // This embedding is to achieve better efficiency for bundles that contain + // only small amounts of data and are cheap to be processed on the SDK harness + // side. This field can be set only if the SDK declares that it supports the + // beam:protocol:control_request_elements_embedding:v1 capability. See more + // at https://s.apache.org/beam-fn-api-control-data-embedding. + Elements *Elements +} + +func (b0 ProcessBundleRequest_builder) Build() *ProcessBundleRequest { + m0 := &ProcessBundleRequest{} + b, x := &b0, m0 + _, _ = b, x + x.xxx_hidden_ProcessBundleDescriptorId = b.ProcessBundleDescriptorId + x.xxx_hidden_CacheTokens = &b.CacheTokens + x.xxx_hidden_Elements = b.Elements + return m0 +} + +type ProcessBundleResponse struct { + state protoimpl.MessageState `protogen:"opaque.v1"` + xxx_hidden_ResidualRoots *[]*DelayedBundleApplication `protobuf:"bytes,2,rep,name=residual_roots,json=residualRoots,proto3" json:"residual_roots,omitempty"` + xxx_hidden_MonitoringInfos *[]*pipeline_v1.MonitoringInfo `protobuf:"bytes,3,rep,name=monitoring_infos,json=monitoringInfos,proto3" json:"monitoring_infos,omitempty"` + xxx_hidden_RequiresFinalization bool `protobuf:"varint,4,opt,name=requires_finalization,json=requiresFinalization,proto3" json:"requires_finalization,omitempty"` + xxx_hidden_MonitoringData map[string][]byte `protobuf:"bytes,5,rep,name=monitoring_data,json=monitoringData,proto3" json:"monitoring_data,omitempty" protobuf_key:"bytes,1,opt,name=key" protobuf_val:"bytes,2,opt,name=value"` + xxx_hidden_Elements *Elements `protobuf:"bytes,6,opt,name=elements,proto3" json:"elements,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *ProcessBundleResponse) Reset() { + *x = ProcessBundleResponse{} + mi := &file_org_apache_beam_model_fn_execution_v1_beam_fn_api_proto_msgTypes[16] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *ProcessBundleResponse) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*ProcessBundleResponse) ProtoMessage() {} + +func (x *ProcessBundleResponse) ProtoReflect() protoreflect.Message { + mi := &file_org_apache_beam_model_fn_execution_v1_beam_fn_api_proto_msgTypes[16] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +func (x *ProcessBundleResponse) GetResidualRoots() []*DelayedBundleApplication { + if x != nil { + if x.xxx_hidden_ResidualRoots != nil { + return *x.xxx_hidden_ResidualRoots + } + } + return nil +} + +func (x *ProcessBundleResponse) GetMonitoringInfos() []*pipeline_v1.MonitoringInfo { + if x != nil { + if x.xxx_hidden_MonitoringInfos != nil { + return *x.xxx_hidden_MonitoringInfos + } + } + return nil +} + +func (x *ProcessBundleResponse) GetRequiresFinalization() bool { + if x != nil { + return x.xxx_hidden_RequiresFinalization + } + return false +} + +func (x *ProcessBundleResponse) GetMonitoringData() map[string][]byte { + if x != nil { + return x.xxx_hidden_MonitoringData + } + return nil +} + +func (x *ProcessBundleResponse) GetElements() *Elements { + if x != nil { + return x.xxx_hidden_Elements + } + return nil +} + +func (x *ProcessBundleResponse) SetResidualRoots(v []*DelayedBundleApplication) { + x.xxx_hidden_ResidualRoots = &v +} + +func (x *ProcessBundleResponse) SetMonitoringInfos(v []*pipeline_v1.MonitoringInfo) { + x.xxx_hidden_MonitoringInfos = &v +} + +func (x *ProcessBundleResponse) SetRequiresFinalization(v bool) { + x.xxx_hidden_RequiresFinalization = v +} + +func (x *ProcessBundleResponse) SetMonitoringData(v map[string][]byte) { + x.xxx_hidden_MonitoringData = v +} + +func (x *ProcessBundleResponse) SetElements(v *Elements) { + x.xxx_hidden_Elements = v +} + +func (x *ProcessBundleResponse) HasElements() bool { + if x == nil { + return false + } + return x.xxx_hidden_Elements != nil +} + +func (x *ProcessBundleResponse) ClearElements() { + x.xxx_hidden_Elements = nil +} + +type ProcessBundleResponse_builder struct { + _ [0]func() // Prevents comparability and use of unkeyed literals for the builder. + + // (Optional) Specifies that the bundle has not been completed and the + // following applications need to be scheduled and executed in the future. + // A runner that does not yet support residual roots MUST still check that + // this is empty for correctness. + // + // Note that these residual roots must not have been returned as part of a + // prior split for this bundle. + ResidualRoots []*DelayedBundleApplication + // DEPRECATED (Required) The list of metrics or other MonitoredState + // collected while processing this bundle. + MonitoringInfos []*pipeline_v1.MonitoringInfo + // (Optional) Specifies that the runner must callback to this worker + // once the output of the bundle is committed. The Runner must send a + // FinalizeBundleRequest with the instruction id of the ProcessBundleRequest + // that is related to this ProcessBundleResponse. + RequiresFinalization bool + // An identifier to MonitoringInfo.payload mapping. + // + // An SDK can report metrics using an identifier that only contains the + // associated payload. A runner who wants to receive the full metrics + // information can request all the monitoring metadata via a + // MonitoringInfosMetadataRequest providing a list of ids as necessary. + // + // The SDK is allowed to reuse the identifiers across multiple bundles as long + // as the MonitoringInfo could be reconstructed fully by overwriting its + // payload field with the bytes specified here. + MonitoringData map[string][]byte + // (Optional) Output elements of the processed bundle. Either all or + // none of the bundle elements should be included in the ProcessBundleResponse. + // This embedding is to achieve better efficiency for bundles that only + // contain small amounts of data. his field can be set only if the runner + // declares that it supports the + // beam:protocol:control_request_elements_embedding:v1 capability. See more at + // https://s.apache.org/beam-fn-api-control-data-embedding. + Elements *Elements +} + +func (b0 ProcessBundleResponse_builder) Build() *ProcessBundleResponse { + m0 := &ProcessBundleResponse{} + b, x := &b0, m0 + _, _ = b, x + x.xxx_hidden_ResidualRoots = &b.ResidualRoots + x.xxx_hidden_MonitoringInfos = &b.MonitoringInfos + x.xxx_hidden_RequiresFinalization = b.RequiresFinalization + x.xxx_hidden_MonitoringData = b.MonitoringData + x.xxx_hidden_Elements = b.Elements + return m0 +} + +// A request to report progress information for a given bundle. +// This is an optional request to be handled and is used to support advanced +// SDK features such as SplittableDoFn, user level metrics etc. +type ProcessBundleProgressRequest struct { + state protoimpl.MessageState `protogen:"opaque.v1"` + xxx_hidden_InstructionId string `protobuf:"bytes,1,opt,name=instruction_id,json=instructionId,proto3" json:"instruction_id,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *ProcessBundleProgressRequest) Reset() { + *x = ProcessBundleProgressRequest{} + mi := &file_org_apache_beam_model_fn_execution_v1_beam_fn_api_proto_msgTypes[17] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *ProcessBundleProgressRequest) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*ProcessBundleProgressRequest) ProtoMessage() {} + +func (x *ProcessBundleProgressRequest) ProtoReflect() protoreflect.Message { + mi := &file_org_apache_beam_model_fn_execution_v1_beam_fn_api_proto_msgTypes[17] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +func (x *ProcessBundleProgressRequest) GetInstructionId() string { + if x != nil { + return x.xxx_hidden_InstructionId + } + return "" +} + +func (x *ProcessBundleProgressRequest) SetInstructionId(v string) { + x.xxx_hidden_InstructionId = v +} + +type ProcessBundleProgressRequest_builder struct { + _ [0]func() // Prevents comparability and use of unkeyed literals for the builder. + + // (Required) A reference to an active process bundle request with the given + // instruction id. + InstructionId string +} + +func (b0 ProcessBundleProgressRequest_builder) Build() *ProcessBundleProgressRequest { + m0 := &ProcessBundleProgressRequest{} + b, x := &b0, m0 + _, _ = b, x + x.xxx_hidden_InstructionId = b.InstructionId + return m0 +} + +// A request to provide full MonitoringInfo for a set of provided ids. +// +// An SDK can report metrics using an identifier that only contains the +// associated payload. A runner who wants to receive the full metrics +// information can request all the monitoring metadata via a +// MonitoringInfosMetadataRequest providing a list of ids as necessary. +// +// The SDK is allowed to reuse the identifiers for the lifetime of the +// associated control connection as long as the MonitoringInfo could be +// reconstructed fully by overwriting its payload field with the bytes specified +// here. +type MonitoringInfosMetadataRequest struct { + state protoimpl.MessageState `protogen:"opaque.v1"` + xxx_hidden_MonitoringInfoId []string `protobuf:"bytes,1,rep,name=monitoring_info_id,json=monitoringInfoId,proto3" json:"monitoring_info_id,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *MonitoringInfosMetadataRequest) Reset() { + *x = MonitoringInfosMetadataRequest{} + mi := &file_org_apache_beam_model_fn_execution_v1_beam_fn_api_proto_msgTypes[18] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *MonitoringInfosMetadataRequest) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*MonitoringInfosMetadataRequest) ProtoMessage() {} + +func (x *MonitoringInfosMetadataRequest) ProtoReflect() protoreflect.Message { + mi := &file_org_apache_beam_model_fn_execution_v1_beam_fn_api_proto_msgTypes[18] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +func (x *MonitoringInfosMetadataRequest) GetMonitoringInfoId() []string { + if x != nil { + return x.xxx_hidden_MonitoringInfoId + } + return nil +} + +func (x *MonitoringInfosMetadataRequest) SetMonitoringInfoId(v []string) { + x.xxx_hidden_MonitoringInfoId = v +} + +type MonitoringInfosMetadataRequest_builder struct { + _ [0]func() // Prevents comparability and use of unkeyed literals for the builder. + + // A list of ids for which the full MonitoringInfo is requested for. + MonitoringInfoId []string +} + +func (b0 MonitoringInfosMetadataRequest_builder) Build() *MonitoringInfosMetadataRequest { + m0 := &MonitoringInfosMetadataRequest{} + b, x := &b0, m0 + _, _ = b, x + x.xxx_hidden_MonitoringInfoId = b.MonitoringInfoId + return m0 +} + +type ProcessBundleProgressResponse struct { + state protoimpl.MessageState `protogen:"opaque.v1"` + xxx_hidden_MonitoringInfos *[]*pipeline_v1.MonitoringInfo `protobuf:"bytes,3,rep,name=monitoring_infos,json=monitoringInfos,proto3" json:"monitoring_infos,omitempty"` + xxx_hidden_MonitoringData map[string][]byte `protobuf:"bytes,5,rep,name=monitoring_data,json=monitoringData,proto3" json:"monitoring_data,omitempty" protobuf_key:"bytes,1,opt,name=key" protobuf_val:"bytes,2,opt,name=value"` + xxx_hidden_ConsumingReceivedData bool `protobuf:"varint,6,opt,name=consuming_received_data,json=consumingReceivedData,proto3,oneof" json:"consuming_received_data,omitempty"` + XXX_raceDetectHookData protoimpl.RaceDetectHookData + XXX_presence [1]uint32 + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *ProcessBundleProgressResponse) Reset() { + *x = ProcessBundleProgressResponse{} + mi := &file_org_apache_beam_model_fn_execution_v1_beam_fn_api_proto_msgTypes[19] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *ProcessBundleProgressResponse) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*ProcessBundleProgressResponse) ProtoMessage() {} + +func (x *ProcessBundleProgressResponse) ProtoReflect() protoreflect.Message { + mi := &file_org_apache_beam_model_fn_execution_v1_beam_fn_api_proto_msgTypes[19] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +func (x *ProcessBundleProgressResponse) GetMonitoringInfos() []*pipeline_v1.MonitoringInfo { + if x != nil { + if x.xxx_hidden_MonitoringInfos != nil { + return *x.xxx_hidden_MonitoringInfos + } + } + return nil +} + +func (x *ProcessBundleProgressResponse) GetMonitoringData() map[string][]byte { + if x != nil { + return x.xxx_hidden_MonitoringData + } + return nil +} + +func (x *ProcessBundleProgressResponse) GetConsumingReceivedData() bool { + if x != nil { + return x.xxx_hidden_ConsumingReceivedData + } + return false +} + +func (x *ProcessBundleProgressResponse) SetMonitoringInfos(v []*pipeline_v1.MonitoringInfo) { + x.xxx_hidden_MonitoringInfos = &v +} + +func (x *ProcessBundleProgressResponse) SetMonitoringData(v map[string][]byte) { + x.xxx_hidden_MonitoringData = v +} + +func (x *ProcessBundleProgressResponse) SetConsumingReceivedData(v bool) { + x.xxx_hidden_ConsumingReceivedData = v + protoimpl.X.SetPresent(&(x.XXX_presence[0]), 2, 3) +} + +func (x *ProcessBundleProgressResponse) HasConsumingReceivedData() bool { + if x == nil { + return false + } + return protoimpl.X.Present(&(x.XXX_presence[0]), 2) +} + +func (x *ProcessBundleProgressResponse) ClearConsumingReceivedData() { + protoimpl.X.ClearPresent(&(x.XXX_presence[0]), 2) + x.xxx_hidden_ConsumingReceivedData = false +} + +type ProcessBundleProgressResponse_builder struct { + _ [0]func() // Prevents comparability and use of unkeyed literals for the builder. + + // DEPRECATED (Required) The list of metrics or other MonitoredState + // collected while processing this bundle. + MonitoringInfos []*pipeline_v1.MonitoringInfo + // An identifier to MonitoringInfo.payload mapping. + // + // An SDK can report metrics using an identifier that only contains the + // associated payload. A runner who wants to receive the full metrics + // information can request all the monitoring metadata via a + // MonitoringInfosMetadataRequest providing a list of ids as necessary. + // + // The SDK is allowed to reuse the identifiers + // for the lifetime of the associated control connection as long + // as the MonitoringInfo could be reconstructed fully by overwriting its + // payload field with the bytes specified here. + MonitoringData map[string][]byte + // Indicates that the SDK is still busy consuming the data that as already + // been received on the data channel. If this is set, a runner may abstain + // from sending further data on the data channel until this field becomes + // unset. This field is currently used during shuffle reads on large elements. + ConsumingReceivedData *bool +} + +func (b0 ProcessBundleProgressResponse_builder) Build() *ProcessBundleProgressResponse { + m0 := &ProcessBundleProgressResponse{} + b, x := &b0, m0 + _, _ = b, x + x.xxx_hidden_MonitoringInfos = &b.MonitoringInfos + x.xxx_hidden_MonitoringData = b.MonitoringData + if b.ConsumingReceivedData != nil { + protoimpl.X.SetPresentNonAtomic(&(x.XXX_presence[0]), 2, 3) + x.xxx_hidden_ConsumingReceivedData = *b.ConsumingReceivedData + } + return m0 +} + +// A response that contains the full mapping information associated with +// a specified set of identifiers. +// +// An SDK can report metrics using an identifier that only contains the +// associated payload. A runner who wants to receive the full metrics +// information can request all the monitoring metadata via a +// MonitoringInfosMetadataRequest providing a list of ids as necessary. +// +// The SDK is allowed to reuse the identifiers +// for the lifetime of the associated control connection as long +// as the MonitoringInfo could be reconstructed fully by overwriting its +// payload field with the bytes specified here. +type MonitoringInfosMetadataResponse struct { + state protoimpl.MessageState `protogen:"opaque.v1"` + xxx_hidden_MonitoringInfo map[string]*pipeline_v1.MonitoringInfo `protobuf:"bytes,1,rep,name=monitoring_info,json=monitoringInfo,proto3" json:"monitoring_info,omitempty" protobuf_key:"bytes,1,opt,name=key" protobuf_val:"bytes,2,opt,name=value"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *MonitoringInfosMetadataResponse) Reset() { + *x = MonitoringInfosMetadataResponse{} + mi := &file_org_apache_beam_model_fn_execution_v1_beam_fn_api_proto_msgTypes[20] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *MonitoringInfosMetadataResponse) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*MonitoringInfosMetadataResponse) ProtoMessage() {} + +func (x *MonitoringInfosMetadataResponse) ProtoReflect() protoreflect.Message { + mi := &file_org_apache_beam_model_fn_execution_v1_beam_fn_api_proto_msgTypes[20] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +func (x *MonitoringInfosMetadataResponse) GetMonitoringInfo() map[string]*pipeline_v1.MonitoringInfo { + if x != nil { + return x.xxx_hidden_MonitoringInfo + } + return nil +} + +func (x *MonitoringInfosMetadataResponse) SetMonitoringInfo(v map[string]*pipeline_v1.MonitoringInfo) { + x.xxx_hidden_MonitoringInfo = v +} + +type MonitoringInfosMetadataResponse_builder struct { + _ [0]func() // Prevents comparability and use of unkeyed literals for the builder. + + // A mapping from an identifier to the full metrics information. + MonitoringInfo map[string]*pipeline_v1.MonitoringInfo +} + +func (b0 MonitoringInfosMetadataResponse_builder) Build() *MonitoringInfosMetadataResponse { + m0 := &MonitoringInfosMetadataResponse{} + b, x := &b0, m0 + _, _ = b, x + x.xxx_hidden_MonitoringInfo = b.MonitoringInfo + return m0 +} + +// Represents a request to the SDK to split a currently active bundle. +type ProcessBundleSplitRequest struct { + state protoimpl.MessageState `protogen:"opaque.v1"` + xxx_hidden_InstructionId string `protobuf:"bytes,1,opt,name=instruction_id,json=instructionId,proto3" json:"instruction_id,omitempty"` + xxx_hidden_DesiredSplits map[string]*ProcessBundleSplitRequest_DesiredSplit `protobuf:"bytes,3,rep,name=desired_splits,json=desiredSplits,proto3" json:"desired_splits,omitempty" protobuf_key:"bytes,1,opt,name=key" protobuf_val:"bytes,2,opt,name=value"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *ProcessBundleSplitRequest) Reset() { + *x = ProcessBundleSplitRequest{} + mi := &file_org_apache_beam_model_fn_execution_v1_beam_fn_api_proto_msgTypes[21] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *ProcessBundleSplitRequest) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*ProcessBundleSplitRequest) ProtoMessage() {} + +func (x *ProcessBundleSplitRequest) ProtoReflect() protoreflect.Message { + mi := &file_org_apache_beam_model_fn_execution_v1_beam_fn_api_proto_msgTypes[21] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +func (x *ProcessBundleSplitRequest) GetInstructionId() string { + if x != nil { + return x.xxx_hidden_InstructionId + } + return "" +} + +func (x *ProcessBundleSplitRequest) GetDesiredSplits() map[string]*ProcessBundleSplitRequest_DesiredSplit { + if x != nil { + return x.xxx_hidden_DesiredSplits + } + return nil +} + +func (x *ProcessBundleSplitRequest) SetInstructionId(v string) { + x.xxx_hidden_InstructionId = v +} + +func (x *ProcessBundleSplitRequest) SetDesiredSplits(v map[string]*ProcessBundleSplitRequest_DesiredSplit) { + x.xxx_hidden_DesiredSplits = v +} + +type ProcessBundleSplitRequest_builder struct { + _ [0]func() // Prevents comparability and use of unkeyed literals for the builder. + + // (Required) A reference to an active process bundle request with the given + // instruction id. + InstructionId string + // (Required) Specifies the desired split for each transform. + // + // Currently only splits at gRPC read operations are supported. + // This may, of course, limit the amount of work downstream operations + // receive. + DesiredSplits map[string]*ProcessBundleSplitRequest_DesiredSplit +} + +func (b0 ProcessBundleSplitRequest_builder) Build() *ProcessBundleSplitRequest { + m0 := &ProcessBundleSplitRequest{} + b, x := &b0, m0 + _, _ = b, x + x.xxx_hidden_InstructionId = b.InstructionId + x.xxx_hidden_DesiredSplits = b.DesiredSplits + return m0 +} + +// Represents a partition of the bundle: a "primary" and a "residual", with the +// following properties: +// - The work in primary and residual doesn't overlap, and combined, adds up +// to the work in the current bundle if the split hadn't happened. +// - The current bundle, if it keeps executing, will have done exactly none of +// the work under residual_roots and none of the elements at and beyond the +// first_residual_element. +// - The current bundle, if no further splits happen, will have done exactly +// the work under primary_roots and all elements up to and including the +// channel split's last_primary_element. +// +// This allows the SDK to relinquish ownership of and commit to not process some +// of the elements that it may have been sent (the residual) while retaining +// ownership and commitment to finish the other portion (the primary). +// +// Example with three splits of a single bundle: +// Let's say the SDK is processing elements [A B C D E]. These elements make +// up the 0-indexed channel. +// +// ** First Split ** +// Channel Split = [ A B C D <> E ] +// Primary Roots = [] (No elements were split) +// Residual Roots = [] +// +// Say a split request comes in. The SDK could return a response with a channel +// split representing a last_primary_element of 3 (D) and +// first_residual_element of 4 (E). The SDK is now responsible for processing A +// B C D and the runner must process E in the future. +// +// (A B C D) | (E) +// +// ** Second Split ** +// Channel Split = [ A < B C > D E ] +// Primary Roots = [B1 C1] +// Residual Roots = [B2 C2] +// +// A future split request could have the SDK split the elements B into B1 and +// B2 and C into C1 and C2 representing their primary and residual roots. The +// +// (A B1 C1) | (B2 C2 D) +// +// SDK would return a response with a channel split representing a +// last_primary_element of 0 (A) and first_residual_element of 3 (D) with +// primary_roots (B1, C1) and residual_roots (B2, C2). The SDK is now +// responsible for processing A B1 C1 and the runner must process B2 C2 D (and +// E from the prior split) in the future. +// +// ** Third Split ** +// Channel Split = [ A < B C > D E ] +// Primary Roots = [B1a] +// Residual Roots [B1b C1] +// Yet another future split request could have the SDK could split B1 further +// into B1a and B1b primary and residuals and return C1 as a residual (assuming +// C1 was left unprocessed). The SDK would return a response with a channel +// split representing a last_primary_element of 0 (A) and +// first_residual_element of 3 (E) with primary_roots (B1a) and residual_roots +// (B1b, C1). The SDK is now responsible for processing A B1a the runner must +// process B1b C1 (in addition to C2, D, E from prior splits) in the future. +// +// (A B1a) | (B1b C1) +// +// For more rigorous definitions see https://s.apache.org/beam-breaking-fusion +type ProcessBundleSplitResponse struct { + state protoimpl.MessageState `protogen:"opaque.v1"` + xxx_hidden_PrimaryRoots *[]*BundleApplication `protobuf:"bytes,1,rep,name=primary_roots,json=primaryRoots,proto3" json:"primary_roots,omitempty"` + xxx_hidden_ResidualRoots *[]*DelayedBundleApplication `protobuf:"bytes,2,rep,name=residual_roots,json=residualRoots,proto3" json:"residual_roots,omitempty"` + xxx_hidden_ChannelSplits *[]*ProcessBundleSplitResponse_ChannelSplit `protobuf:"bytes,3,rep,name=channel_splits,json=channelSplits,proto3" json:"channel_splits,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *ProcessBundleSplitResponse) Reset() { + *x = ProcessBundleSplitResponse{} + mi := &file_org_apache_beam_model_fn_execution_v1_beam_fn_api_proto_msgTypes[22] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *ProcessBundleSplitResponse) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*ProcessBundleSplitResponse) ProtoMessage() {} + +func (x *ProcessBundleSplitResponse) ProtoReflect() protoreflect.Message { + mi := &file_org_apache_beam_model_fn_execution_v1_beam_fn_api_proto_msgTypes[22] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +func (x *ProcessBundleSplitResponse) GetPrimaryRoots() []*BundleApplication { + if x != nil { + if x.xxx_hidden_PrimaryRoots != nil { + return *x.xxx_hidden_PrimaryRoots + } + } + return nil +} + +func (x *ProcessBundleSplitResponse) GetResidualRoots() []*DelayedBundleApplication { + if x != nil { + if x.xxx_hidden_ResidualRoots != nil { + return *x.xxx_hidden_ResidualRoots + } + } + return nil +} + +func (x *ProcessBundleSplitResponse) GetChannelSplits() []*ProcessBundleSplitResponse_ChannelSplit { + if x != nil { + if x.xxx_hidden_ChannelSplits != nil { + return *x.xxx_hidden_ChannelSplits + } + } + return nil +} + +func (x *ProcessBundleSplitResponse) SetPrimaryRoots(v []*BundleApplication) { + x.xxx_hidden_PrimaryRoots = &v +} + +func (x *ProcessBundleSplitResponse) SetResidualRoots(v []*DelayedBundleApplication) { + x.xxx_hidden_ResidualRoots = &v +} + +func (x *ProcessBundleSplitResponse) SetChannelSplits(v []*ProcessBundleSplitResponse_ChannelSplit) { + x.xxx_hidden_ChannelSplits = &v +} + +type ProcessBundleSplitResponse_builder struct { + _ [0]func() // Prevents comparability and use of unkeyed literals for the builder. + + // (Optional) Root applications that should replace the current bundle. + // + // Note that primary roots can only be specified if a channel split's + // last_primary_element + 1 < first_residual_element + // + // Note that there must be a corresponding residual root contained within + // residual_roots representing the remainder of processing for the original + // element this this primary root represents a fraction of. + PrimaryRoots []*BundleApplication + // (Optional) Root applications that have been removed from the current bundle and + // have to be executed in a separate bundle (e.g. in parallel on a different + // worker, or after the current bundle completes, etc.) + // + // Note that residual roots can only be specified if a channel split's + // last_primary_element + 1 < first_residual_element + // + // Note that there must be a corresponding primary root contained within + // primary_roots representing the remainder of processing for the original + // element this this residual root represents a fraction of. + // + // Note that subsequent splits must not return prior residual roots. + ResidualRoots []*DelayedBundleApplication + // (Required) Partitions of input data channels into primary and residual + // elements, if any. Must not include any elements represented in the bundle + // applications roots above of the current split or any prior split of the + // same bundle. + ChannelSplits []*ProcessBundleSplitResponse_ChannelSplit +} + +func (b0 ProcessBundleSplitResponse_builder) Build() *ProcessBundleSplitResponse { + m0 := &ProcessBundleSplitResponse{} + b, x := &b0, m0 + _, _ = b, x + x.xxx_hidden_PrimaryRoots = &b.PrimaryRoots + x.xxx_hidden_ResidualRoots = &b.ResidualRoots + x.xxx_hidden_ChannelSplits = &b.ChannelSplits + return m0 +} + +type FinalizeBundleRequest struct { + state protoimpl.MessageState `protogen:"opaque.v1"` + xxx_hidden_InstructionId string `protobuf:"bytes,1,opt,name=instruction_id,json=instructionId,proto3" json:"instruction_id,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *FinalizeBundleRequest) Reset() { + *x = FinalizeBundleRequest{} + mi := &file_org_apache_beam_model_fn_execution_v1_beam_fn_api_proto_msgTypes[23] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *FinalizeBundleRequest) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*FinalizeBundleRequest) ProtoMessage() {} + +func (x *FinalizeBundleRequest) ProtoReflect() protoreflect.Message { + mi := &file_org_apache_beam_model_fn_execution_v1_beam_fn_api_proto_msgTypes[23] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +func (x *FinalizeBundleRequest) GetInstructionId() string { + if x != nil { + return x.xxx_hidden_InstructionId + } + return "" +} + +func (x *FinalizeBundleRequest) SetInstructionId(v string) { + x.xxx_hidden_InstructionId = v +} + +type FinalizeBundleRequest_builder struct { + _ [0]func() // Prevents comparability and use of unkeyed literals for the builder. + + // (Required) A reference to a completed process bundle request with the given + // instruction id. + InstructionId string +} + +func (b0 FinalizeBundleRequest_builder) Build() *FinalizeBundleRequest { + m0 := &FinalizeBundleRequest{} + b, x := &b0, m0 + _, _ = b, x + x.xxx_hidden_InstructionId = b.InstructionId + return m0 +} + +type FinalizeBundleResponse struct { + state protoimpl.MessageState `protogen:"opaque.v1"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *FinalizeBundleResponse) Reset() { + *x = FinalizeBundleResponse{} + mi := &file_org_apache_beam_model_fn_execution_v1_beam_fn_api_proto_msgTypes[24] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *FinalizeBundleResponse) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*FinalizeBundleResponse) ProtoMessage() {} + +func (x *FinalizeBundleResponse) ProtoReflect() protoreflect.Message { + mi := &file_org_apache_beam_model_fn_execution_v1_beam_fn_api_proto_msgTypes[24] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +type FinalizeBundleResponse_builder struct { + _ [0]func() // Prevents comparability and use of unkeyed literals for the builder. + +} + +func (b0 FinalizeBundleResponse_builder) Build() *FinalizeBundleResponse { + m0 := &FinalizeBundleResponse{} + b, x := &b0, m0 + _, _ = b, x + return m0 +} + +// Messages used to represent logical byte streams. +// Stable +type Elements struct { + state protoimpl.MessageState `protogen:"opaque.v1"` + xxx_hidden_Data *[]*Elements_Data `protobuf:"bytes,1,rep,name=data,proto3" json:"data,omitempty"` + xxx_hidden_Timers *[]*Elements_Timers `protobuf:"bytes,2,rep,name=timers,proto3" json:"timers,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *Elements) Reset() { + *x = Elements{} + mi := &file_org_apache_beam_model_fn_execution_v1_beam_fn_api_proto_msgTypes[25] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *Elements) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*Elements) ProtoMessage() {} + +func (x *Elements) ProtoReflect() protoreflect.Message { + mi := &file_org_apache_beam_model_fn_execution_v1_beam_fn_api_proto_msgTypes[25] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +func (x *Elements) GetData() []*Elements_Data { + if x != nil { + if x.xxx_hidden_Data != nil { + return *x.xxx_hidden_Data + } + } + return nil +} + +func (x *Elements) GetTimers() []*Elements_Timers { + if x != nil { + if x.xxx_hidden_Timers != nil { + return *x.xxx_hidden_Timers + } + } + return nil +} + +func (x *Elements) SetData(v []*Elements_Data) { + x.xxx_hidden_Data = &v +} + +func (x *Elements) SetTimers(v []*Elements_Timers) { + x.xxx_hidden_Timers = &v +} + +type Elements_builder struct { + _ [0]func() // Prevents comparability and use of unkeyed literals for the builder. + + // (Optional) A list containing parts of logical byte streams. + Data []*Elements_Data + // (Optional) A list of timer byte streams. + Timers []*Elements_Timers +} + +func (b0 Elements_builder) Build() *Elements { + m0 := &Elements{} + b, x := &b0, m0 + _, _ = b, x + x.xxx_hidden_Data = &b.Data + x.xxx_hidden_Timers = &b.Timers + return m0 +} + +type StateRequest struct { + state protoimpl.MessageState `protogen:"opaque.v1"` + xxx_hidden_Id string `protobuf:"bytes,1,opt,name=id,proto3" json:"id,omitempty"` + xxx_hidden_InstructionId string `protobuf:"bytes,2,opt,name=instruction_id,json=instructionId,proto3" json:"instruction_id,omitempty"` + xxx_hidden_StateKey *StateKey `protobuf:"bytes,3,opt,name=state_key,json=stateKey,proto3" json:"state_key,omitempty"` + xxx_hidden_Request isStateRequest_Request `protobuf_oneof:"request"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *StateRequest) Reset() { + *x = StateRequest{} + mi := &file_org_apache_beam_model_fn_execution_v1_beam_fn_api_proto_msgTypes[26] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *StateRequest) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*StateRequest) ProtoMessage() {} + +func (x *StateRequest) ProtoReflect() protoreflect.Message { + mi := &file_org_apache_beam_model_fn_execution_v1_beam_fn_api_proto_msgTypes[26] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +func (x *StateRequest) GetId() string { + if x != nil { + return x.xxx_hidden_Id + } + return "" +} + +func (x *StateRequest) GetInstructionId() string { + if x != nil { + return x.xxx_hidden_InstructionId + } + return "" +} + +func (x *StateRequest) GetStateKey() *StateKey { + if x != nil { + return x.xxx_hidden_StateKey + } + return nil +} + +func (x *StateRequest) GetGet() *StateGetRequest { + if x != nil { + if x, ok := x.xxx_hidden_Request.(*stateRequest_Get); ok { + return x.Get + } + } + return nil +} + +func (x *StateRequest) GetAppend() *StateAppendRequest { + if x != nil { + if x, ok := x.xxx_hidden_Request.(*stateRequest_Append); ok { + return x.Append + } + } + return nil +} + +func (x *StateRequest) GetClear() *StateClearRequest { + if x != nil { + if x, ok := x.xxx_hidden_Request.(*stateRequest_Clear); ok { + return x.Clear + } + } + return nil +} + +func (x *StateRequest) SetId(v string) { + x.xxx_hidden_Id = v +} + +func (x *StateRequest) SetInstructionId(v string) { + x.xxx_hidden_InstructionId = v +} + +func (x *StateRequest) SetStateKey(v *StateKey) { + x.xxx_hidden_StateKey = v +} + +func (x *StateRequest) SetGet(v *StateGetRequest) { + if v == nil { + x.xxx_hidden_Request = nil + return + } + x.xxx_hidden_Request = &stateRequest_Get{v} +} + +func (x *StateRequest) SetAppend(v *StateAppendRequest) { + if v == nil { + x.xxx_hidden_Request = nil + return + } + x.xxx_hidden_Request = &stateRequest_Append{v} +} + +func (x *StateRequest) SetClear(v *StateClearRequest) { + if v == nil { + x.xxx_hidden_Request = nil + return + } + x.xxx_hidden_Request = &stateRequest_Clear{v} +} + +func (x *StateRequest) HasStateKey() bool { + if x == nil { + return false + } + return x.xxx_hidden_StateKey != nil +} + +func (x *StateRequest) HasRequest() bool { + if x == nil { + return false + } + return x.xxx_hidden_Request != nil +} + +func (x *StateRequest) HasGet() bool { + if x == nil { + return false + } + _, ok := x.xxx_hidden_Request.(*stateRequest_Get) + return ok +} + +func (x *StateRequest) HasAppend() bool { + if x == nil { + return false + } + _, ok := x.xxx_hidden_Request.(*stateRequest_Append) + return ok +} + +func (x *StateRequest) HasClear() bool { + if x == nil { + return false + } + _, ok := x.xxx_hidden_Request.(*stateRequest_Clear) + return ok +} + +func (x *StateRequest) ClearStateKey() { + x.xxx_hidden_StateKey = nil +} + +func (x *StateRequest) ClearRequest() { + x.xxx_hidden_Request = nil +} + +func (x *StateRequest) ClearGet() { + if _, ok := x.xxx_hidden_Request.(*stateRequest_Get); ok { + x.xxx_hidden_Request = nil + } +} + +func (x *StateRequest) ClearAppend() { + if _, ok := x.xxx_hidden_Request.(*stateRequest_Append); ok { + x.xxx_hidden_Request = nil + } +} + +func (x *StateRequest) ClearClear() { + if _, ok := x.xxx_hidden_Request.(*stateRequest_Clear); ok { + x.xxx_hidden_Request = nil + } +} + +const StateRequest_Request_not_set_case case_StateRequest_Request = 0 +const StateRequest_Get_case case_StateRequest_Request = 1000 +const StateRequest_Append_case case_StateRequest_Request = 1001 +const StateRequest_Clear_case case_StateRequest_Request = 1002 + +func (x *StateRequest) WhichRequest() case_StateRequest_Request { + if x == nil { + return StateRequest_Request_not_set_case + } + switch x.xxx_hidden_Request.(type) { + case *stateRequest_Get: + return StateRequest_Get_case + case *stateRequest_Append: + return StateRequest_Append_case + case *stateRequest_Clear: + return StateRequest_Clear_case + default: + return StateRequest_Request_not_set_case + } +} + +type StateRequest_builder struct { + _ [0]func() // Prevents comparability and use of unkeyed literals for the builder. + + // (Required) A unique identifier provided by the SDK which represents this + // requests execution. The StateResponse corresponding with this request + // will have the matching id. + Id string + // (Required) The associated instruction id of the work that is currently + // being processed. This allows for the runner to associate any modifications + // to state to be committed with the appropriate work execution. + InstructionId string + // (Required) The state key this request is for. + StateKey *StateKey + // (Required) The action to take on this request. + + // Fields of oneof xxx_hidden_Request: + // A request to get state. + Get *StateGetRequest + // A request to append to state. + Append *StateAppendRequest + // A request to clear state. + Clear *StateClearRequest + // -- end of xxx_hidden_Request +} + +func (b0 StateRequest_builder) Build() *StateRequest { + m0 := &StateRequest{} + b, x := &b0, m0 + _, _ = b, x + x.xxx_hidden_Id = b.Id + x.xxx_hidden_InstructionId = b.InstructionId + x.xxx_hidden_StateKey = b.StateKey + if b.Get != nil { + x.xxx_hidden_Request = &stateRequest_Get{b.Get} + } + if b.Append != nil { + x.xxx_hidden_Request = &stateRequest_Append{b.Append} + } + if b.Clear != nil { + x.xxx_hidden_Request = &stateRequest_Clear{b.Clear} + } + return m0 +} + +type case_StateRequest_Request protoreflect.FieldNumber + +func (x case_StateRequest_Request) String() string { + md := file_org_apache_beam_model_fn_execution_v1_beam_fn_api_proto_msgTypes[26].Descriptor() + if x == 0 { + return "not set" + } + return protoimpl.X.MessageFieldStringOf(md, protoreflect.FieldNumber(x)) +} + +type isStateRequest_Request interface { + isStateRequest_Request() +} + +type stateRequest_Get struct { + // A request to get state. + Get *StateGetRequest `protobuf:"bytes,1000,opt,name=get,proto3,oneof"` +} + +type stateRequest_Append struct { + // A request to append to state. + Append *StateAppendRequest `protobuf:"bytes,1001,opt,name=append,proto3,oneof"` +} + +type stateRequest_Clear struct { + // A request to clear state. + Clear *StateClearRequest `protobuf:"bytes,1002,opt,name=clear,proto3,oneof"` +} + +func (*stateRequest_Get) isStateRequest_Request() {} + +func (*stateRequest_Append) isStateRequest_Request() {} + +func (*stateRequest_Clear) isStateRequest_Request() {} + +type StateResponse struct { + state protoimpl.MessageState `protogen:"opaque.v1"` + xxx_hidden_Id string `protobuf:"bytes,1,opt,name=id,proto3" json:"id,omitempty"` + xxx_hidden_Error string `protobuf:"bytes,2,opt,name=error,proto3" json:"error,omitempty"` + xxx_hidden_Response isStateResponse_Response `protobuf_oneof:"response"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *StateResponse) Reset() { + *x = StateResponse{} + mi := &file_org_apache_beam_model_fn_execution_v1_beam_fn_api_proto_msgTypes[27] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *StateResponse) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*StateResponse) ProtoMessage() {} + +func (x *StateResponse) ProtoReflect() protoreflect.Message { + mi := &file_org_apache_beam_model_fn_execution_v1_beam_fn_api_proto_msgTypes[27] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +func (x *StateResponse) GetId() string { + if x != nil { + return x.xxx_hidden_Id + } + return "" +} + +func (x *StateResponse) GetError() string { + if x != nil { + return x.xxx_hidden_Error + } + return "" +} + +func (x *StateResponse) GetGet() *StateGetResponse { + if x != nil { + if x, ok := x.xxx_hidden_Response.(*stateResponse_Get); ok { + return x.Get + } + } + return nil +} + +func (x *StateResponse) GetAppend() *StateAppendResponse { + if x != nil { + if x, ok := x.xxx_hidden_Response.(*stateResponse_Append); ok { + return x.Append + } + } + return nil +} + +func (x *StateResponse) GetClear() *StateClearResponse { + if x != nil { + if x, ok := x.xxx_hidden_Response.(*stateResponse_Clear); ok { + return x.Clear + } + } + return nil +} + +func (x *StateResponse) SetId(v string) { + x.xxx_hidden_Id = v +} + +func (x *StateResponse) SetError(v string) { + x.xxx_hidden_Error = v +} + +func (x *StateResponse) SetGet(v *StateGetResponse) { + if v == nil { + x.xxx_hidden_Response = nil + return + } + x.xxx_hidden_Response = &stateResponse_Get{v} +} + +func (x *StateResponse) SetAppend(v *StateAppendResponse) { + if v == nil { + x.xxx_hidden_Response = nil + return + } + x.xxx_hidden_Response = &stateResponse_Append{v} +} + +func (x *StateResponse) SetClear(v *StateClearResponse) { + if v == nil { + x.xxx_hidden_Response = nil + return + } + x.xxx_hidden_Response = &stateResponse_Clear{v} +} + +func (x *StateResponse) HasResponse() bool { + if x == nil { + return false + } + return x.xxx_hidden_Response != nil +} + +func (x *StateResponse) HasGet() bool { + if x == nil { + return false + } + _, ok := x.xxx_hidden_Response.(*stateResponse_Get) + return ok +} + +func (x *StateResponse) HasAppend() bool { + if x == nil { + return false + } + _, ok := x.xxx_hidden_Response.(*stateResponse_Append) + return ok +} + +func (x *StateResponse) HasClear() bool { + if x == nil { + return false + } + _, ok := x.xxx_hidden_Response.(*stateResponse_Clear) + return ok +} + +func (x *StateResponse) ClearResponse() { + x.xxx_hidden_Response = nil +} + +func (x *StateResponse) ClearGet() { + if _, ok := x.xxx_hidden_Response.(*stateResponse_Get); ok { + x.xxx_hidden_Response = nil + } +} + +func (x *StateResponse) ClearAppend() { + if _, ok := x.xxx_hidden_Response.(*stateResponse_Append); ok { + x.xxx_hidden_Response = nil + } +} + +func (x *StateResponse) ClearClear() { + if _, ok := x.xxx_hidden_Response.(*stateResponse_Clear); ok { + x.xxx_hidden_Response = nil + } +} + +const StateResponse_Response_not_set_case case_StateResponse_Response = 0 +const StateResponse_Get_case case_StateResponse_Response = 1000 +const StateResponse_Append_case case_StateResponse_Response = 1001 +const StateResponse_Clear_case case_StateResponse_Response = 1002 + +func (x *StateResponse) WhichResponse() case_StateResponse_Response { + if x == nil { + return StateResponse_Response_not_set_case + } + switch x.xxx_hidden_Response.(type) { + case *stateResponse_Get: + return StateResponse_Get_case + case *stateResponse_Append: + return StateResponse_Append_case + case *stateResponse_Clear: + return StateResponse_Clear_case + default: + return StateResponse_Response_not_set_case + } +} + +type StateResponse_builder struct { + _ [0]func() // Prevents comparability and use of unkeyed literals for the builder. + + // (Required) A reference provided by the SDK which represents a requests + // execution. The StateResponse must have the matching id when responding + // to the SDK. + Id string + // (Optional) If this is specified, then the state request has failed. + // A human readable string representing the reason as to why the request + // failed. + Error string + // A corresponding response matching the request will be populated. + + // Fields of oneof xxx_hidden_Response: + // A response to getting state. + Get *StateGetResponse + // A response to appending to state. + Append *StateAppendResponse + // A response to clearing state. + Clear *StateClearResponse + // -- end of xxx_hidden_Response +} + +func (b0 StateResponse_builder) Build() *StateResponse { + m0 := &StateResponse{} + b, x := &b0, m0 + _, _ = b, x + x.xxx_hidden_Id = b.Id + x.xxx_hidden_Error = b.Error + if b.Get != nil { + x.xxx_hidden_Response = &stateResponse_Get{b.Get} + } + if b.Append != nil { + x.xxx_hidden_Response = &stateResponse_Append{b.Append} + } + if b.Clear != nil { + x.xxx_hidden_Response = &stateResponse_Clear{b.Clear} + } + return m0 +} + +type case_StateResponse_Response protoreflect.FieldNumber + +func (x case_StateResponse_Response) String() string { + md := file_org_apache_beam_model_fn_execution_v1_beam_fn_api_proto_msgTypes[27].Descriptor() + if x == 0 { + return "not set" + } + return protoimpl.X.MessageFieldStringOf(md, protoreflect.FieldNumber(x)) +} + +type isStateResponse_Response interface { + isStateResponse_Response() +} + +type stateResponse_Get struct { + // A response to getting state. + Get *StateGetResponse `protobuf:"bytes,1000,opt,name=get,proto3,oneof"` +} + +type stateResponse_Append struct { + // A response to appending to state. + Append *StateAppendResponse `protobuf:"bytes,1001,opt,name=append,proto3,oneof"` +} + +type stateResponse_Clear struct { + // A response to clearing state. + Clear *StateClearResponse `protobuf:"bytes,1002,opt,name=clear,proto3,oneof"` +} + +func (*stateResponse_Get) isStateResponse_Response() {} + +func (*stateResponse_Append) isStateResponse_Response() {} + +func (*stateResponse_Clear) isStateResponse_Response() {} + +type StateKey struct { + state protoimpl.MessageState `protogen:"opaque.v1"` + xxx_hidden_Type isStateKey_Type `protobuf_oneof:"type"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *StateKey) Reset() { + *x = StateKey{} + mi := &file_org_apache_beam_model_fn_execution_v1_beam_fn_api_proto_msgTypes[28] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *StateKey) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*StateKey) ProtoMessage() {} + +func (x *StateKey) ProtoReflect() protoreflect.Message { + mi := &file_org_apache_beam_model_fn_execution_v1_beam_fn_api_proto_msgTypes[28] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +func (x *StateKey) GetRunner() *StateKey_Runner { + if x != nil { + if x, ok := x.xxx_hidden_Type.(*stateKey_Runner_); ok { + return x.Runner + } + } + return nil +} + +func (x *StateKey) GetMultimapSideInput() *StateKey_MultimapSideInput { + if x != nil { + if x, ok := x.xxx_hidden_Type.(*stateKey_MultimapSideInput_); ok { + return x.MultimapSideInput + } + } + return nil +} + +func (x *StateKey) GetBagUserState() *StateKey_BagUserState { + if x != nil { + if x, ok := x.xxx_hidden_Type.(*stateKey_BagUserState_); ok { + return x.BagUserState + } + } + return nil +} + +func (x *StateKey) GetIterableSideInput() *StateKey_IterableSideInput { + if x != nil { + if x, ok := x.xxx_hidden_Type.(*stateKey_IterableSideInput_); ok { + return x.IterableSideInput + } + } + return nil +} + +func (x *StateKey) GetMultimapKeysSideInput() *StateKey_MultimapKeysSideInput { + if x != nil { + if x, ok := x.xxx_hidden_Type.(*stateKey_MultimapKeysSideInput_); ok { + return x.MultimapKeysSideInput + } + } + return nil +} + +func (x *StateKey) GetMultimapKeysValuesSideInput() *StateKey_MultimapKeysValuesSideInput { + if x != nil { + if x, ok := x.xxx_hidden_Type.(*stateKey_MultimapKeysValuesSideInput_); ok { + return x.MultimapKeysValuesSideInput + } + } + return nil +} + +func (x *StateKey) GetMultimapKeysUserState() *StateKey_MultimapKeysUserState { + if x != nil { + if x, ok := x.xxx_hidden_Type.(*stateKey_MultimapKeysUserState_); ok { + return x.MultimapKeysUserState + } + } + return nil +} + +func (x *StateKey) GetMultimapUserState() *StateKey_MultimapUserState { + if x != nil { + if x, ok := x.xxx_hidden_Type.(*stateKey_MultimapUserState_); ok { + return x.MultimapUserState + } + } + return nil +} + +func (x *StateKey) GetOrderedListUserState() *StateKey_OrderedListUserState { + if x != nil { + if x, ok := x.xxx_hidden_Type.(*stateKey_OrderedListUserState_); ok { + return x.OrderedListUserState + } + } + return nil +} + +func (x *StateKey) SetRunner(v *StateKey_Runner) { + if v == nil { + x.xxx_hidden_Type = nil + return + } + x.xxx_hidden_Type = &stateKey_Runner_{v} +} + +func (x *StateKey) SetMultimapSideInput(v *StateKey_MultimapSideInput) { + if v == nil { + x.xxx_hidden_Type = nil + return + } + x.xxx_hidden_Type = &stateKey_MultimapSideInput_{v} +} + +func (x *StateKey) SetBagUserState(v *StateKey_BagUserState) { + if v == nil { + x.xxx_hidden_Type = nil + return + } + x.xxx_hidden_Type = &stateKey_BagUserState_{v} +} + +func (x *StateKey) SetIterableSideInput(v *StateKey_IterableSideInput) { + if v == nil { + x.xxx_hidden_Type = nil + return + } + x.xxx_hidden_Type = &stateKey_IterableSideInput_{v} +} + +func (x *StateKey) SetMultimapKeysSideInput(v *StateKey_MultimapKeysSideInput) { + if v == nil { + x.xxx_hidden_Type = nil + return + } + x.xxx_hidden_Type = &stateKey_MultimapKeysSideInput_{v} +} + +func (x *StateKey) SetMultimapKeysValuesSideInput(v *StateKey_MultimapKeysValuesSideInput) { + if v == nil { + x.xxx_hidden_Type = nil + return + } + x.xxx_hidden_Type = &stateKey_MultimapKeysValuesSideInput_{v} +} + +func (x *StateKey) SetMultimapKeysUserState(v *StateKey_MultimapKeysUserState) { + if v == nil { + x.xxx_hidden_Type = nil + return + } + x.xxx_hidden_Type = &stateKey_MultimapKeysUserState_{v} +} + +func (x *StateKey) SetMultimapUserState(v *StateKey_MultimapUserState) { + if v == nil { + x.xxx_hidden_Type = nil + return + } + x.xxx_hidden_Type = &stateKey_MultimapUserState_{v} +} + +func (x *StateKey) SetOrderedListUserState(v *StateKey_OrderedListUserState) { + if v == nil { + x.xxx_hidden_Type = nil + return + } + x.xxx_hidden_Type = &stateKey_OrderedListUserState_{v} +} + +func (x *StateKey) HasType() bool { + if x == nil { + return false + } + return x.xxx_hidden_Type != nil +} + +func (x *StateKey) HasRunner() bool { + if x == nil { + return false + } + _, ok := x.xxx_hidden_Type.(*stateKey_Runner_) + return ok +} + +func (x *StateKey) HasMultimapSideInput() bool { + if x == nil { + return false + } + _, ok := x.xxx_hidden_Type.(*stateKey_MultimapSideInput_) + return ok +} + +func (x *StateKey) HasBagUserState() bool { + if x == nil { + return false + } + _, ok := x.xxx_hidden_Type.(*stateKey_BagUserState_) + return ok +} + +func (x *StateKey) HasIterableSideInput() bool { + if x == nil { + return false + } + _, ok := x.xxx_hidden_Type.(*stateKey_IterableSideInput_) + return ok +} + +func (x *StateKey) HasMultimapKeysSideInput() bool { + if x == nil { + return false + } + _, ok := x.xxx_hidden_Type.(*stateKey_MultimapKeysSideInput_) + return ok +} + +func (x *StateKey) HasMultimapKeysValuesSideInput() bool { + if x == nil { + return false + } + _, ok := x.xxx_hidden_Type.(*stateKey_MultimapKeysValuesSideInput_) + return ok +} + +func (x *StateKey) HasMultimapKeysUserState() bool { + if x == nil { + return false + } + _, ok := x.xxx_hidden_Type.(*stateKey_MultimapKeysUserState_) + return ok +} + +func (x *StateKey) HasMultimapUserState() bool { + if x == nil { + return false + } + _, ok := x.xxx_hidden_Type.(*stateKey_MultimapUserState_) + return ok +} + +func (x *StateKey) HasOrderedListUserState() bool { + if x == nil { + return false + } + _, ok := x.xxx_hidden_Type.(*stateKey_OrderedListUserState_) + return ok +} + +func (x *StateKey) ClearType() { + x.xxx_hidden_Type = nil +} + +func (x *StateKey) ClearRunner() { + if _, ok := x.xxx_hidden_Type.(*stateKey_Runner_); ok { + x.xxx_hidden_Type = nil + } +} + +func (x *StateKey) ClearMultimapSideInput() { + if _, ok := x.xxx_hidden_Type.(*stateKey_MultimapSideInput_); ok { + x.xxx_hidden_Type = nil + } +} + +func (x *StateKey) ClearBagUserState() { + if _, ok := x.xxx_hidden_Type.(*stateKey_BagUserState_); ok { + x.xxx_hidden_Type = nil + } +} + +func (x *StateKey) ClearIterableSideInput() { + if _, ok := x.xxx_hidden_Type.(*stateKey_IterableSideInput_); ok { + x.xxx_hidden_Type = nil + } +} + +func (x *StateKey) ClearMultimapKeysSideInput() { + if _, ok := x.xxx_hidden_Type.(*stateKey_MultimapKeysSideInput_); ok { + x.xxx_hidden_Type = nil + } +} + +func (x *StateKey) ClearMultimapKeysValuesSideInput() { + if _, ok := x.xxx_hidden_Type.(*stateKey_MultimapKeysValuesSideInput_); ok { + x.xxx_hidden_Type = nil + } +} + +func (x *StateKey) ClearMultimapKeysUserState() { + if _, ok := x.xxx_hidden_Type.(*stateKey_MultimapKeysUserState_); ok { + x.xxx_hidden_Type = nil + } +} + +func (x *StateKey) ClearMultimapUserState() { + if _, ok := x.xxx_hidden_Type.(*stateKey_MultimapUserState_); ok { + x.xxx_hidden_Type = nil + } +} + +func (x *StateKey) ClearOrderedListUserState() { + if _, ok := x.xxx_hidden_Type.(*stateKey_OrderedListUserState_); ok { + x.xxx_hidden_Type = nil + } +} + +const StateKey_Type_not_set_case case_StateKey_Type = 0 +const StateKey_Runner_case case_StateKey_Type = 1 +const StateKey_MultimapSideInput_case case_StateKey_Type = 2 +const StateKey_BagUserState_case case_StateKey_Type = 3 +const StateKey_IterableSideInput_case case_StateKey_Type = 4 +const StateKey_MultimapKeysSideInput_case case_StateKey_Type = 5 +const StateKey_MultimapKeysValuesSideInput_case case_StateKey_Type = 8 +const StateKey_MultimapKeysUserState_case case_StateKey_Type = 6 +const StateKey_MultimapUserState_case case_StateKey_Type = 7 +const StateKey_OrderedListUserState_case case_StateKey_Type = 9 + +func (x *StateKey) WhichType() case_StateKey_Type { + if x == nil { + return StateKey_Type_not_set_case + } + switch x.xxx_hidden_Type.(type) { + case *stateKey_Runner_: + return StateKey_Runner_case + case *stateKey_MultimapSideInput_: + return StateKey_MultimapSideInput_case + case *stateKey_BagUserState_: + return StateKey_BagUserState_case + case *stateKey_IterableSideInput_: + return StateKey_IterableSideInput_case + case *stateKey_MultimapKeysSideInput_: + return StateKey_MultimapKeysSideInput_case + case *stateKey_MultimapKeysValuesSideInput_: + return StateKey_MultimapKeysValuesSideInput_case + case *stateKey_MultimapKeysUserState_: + return StateKey_MultimapKeysUserState_case + case *stateKey_MultimapUserState_: + return StateKey_MultimapUserState_case + case *stateKey_OrderedListUserState_: + return StateKey_OrderedListUserState_case + default: + return StateKey_Type_not_set_case + } +} + +type StateKey_builder struct { + _ [0]func() // Prevents comparability and use of unkeyed literals for the builder. + + // (Required) One of the following state keys must be set. + + // Fields of oneof xxx_hidden_Type: + Runner *StateKey_Runner + MultimapSideInput *StateKey_MultimapSideInput + BagUserState *StateKey_BagUserState + IterableSideInput *StateKey_IterableSideInput + MultimapKeysSideInput *StateKey_MultimapKeysSideInput + MultimapKeysValuesSideInput *StateKey_MultimapKeysValuesSideInput + MultimapKeysUserState *StateKey_MultimapKeysUserState + MultimapUserState *StateKey_MultimapUserState + OrderedListUserState *StateKey_OrderedListUserState + // -- end of xxx_hidden_Type +} + +func (b0 StateKey_builder) Build() *StateKey { + m0 := &StateKey{} + b, x := &b0, m0 + _, _ = b, x + if b.Runner != nil { + x.xxx_hidden_Type = &stateKey_Runner_{b.Runner} + } + if b.MultimapSideInput != nil { + x.xxx_hidden_Type = &stateKey_MultimapSideInput_{b.MultimapSideInput} + } + if b.BagUserState != nil { + x.xxx_hidden_Type = &stateKey_BagUserState_{b.BagUserState} + } + if b.IterableSideInput != nil { + x.xxx_hidden_Type = &stateKey_IterableSideInput_{b.IterableSideInput} + } + if b.MultimapKeysSideInput != nil { + x.xxx_hidden_Type = &stateKey_MultimapKeysSideInput_{b.MultimapKeysSideInput} + } + if b.MultimapKeysValuesSideInput != nil { + x.xxx_hidden_Type = &stateKey_MultimapKeysValuesSideInput_{b.MultimapKeysValuesSideInput} + } + if b.MultimapKeysUserState != nil { + x.xxx_hidden_Type = &stateKey_MultimapKeysUserState_{b.MultimapKeysUserState} + } + if b.MultimapUserState != nil { + x.xxx_hidden_Type = &stateKey_MultimapUserState_{b.MultimapUserState} + } + if b.OrderedListUserState != nil { + x.xxx_hidden_Type = &stateKey_OrderedListUserState_{b.OrderedListUserState} + } + return m0 +} + +type case_StateKey_Type protoreflect.FieldNumber + +func (x case_StateKey_Type) String() string { + md := file_org_apache_beam_model_fn_execution_v1_beam_fn_api_proto_msgTypes[28].Descriptor() + if x == 0 { + return "not set" + } + return protoimpl.X.MessageFieldStringOf(md, protoreflect.FieldNumber(x)) +} + +type isStateKey_Type interface { + isStateKey_Type() +} + +type stateKey_Runner_ struct { + Runner *StateKey_Runner `protobuf:"bytes,1,opt,name=runner,proto3,oneof"` +} + +type stateKey_MultimapSideInput_ struct { + MultimapSideInput *StateKey_MultimapSideInput `protobuf:"bytes,2,opt,name=multimap_side_input,json=multimapSideInput,proto3,oneof"` +} + +type stateKey_BagUserState_ struct { + BagUserState *StateKey_BagUserState `protobuf:"bytes,3,opt,name=bag_user_state,json=bagUserState,proto3,oneof"` +} + +type stateKey_IterableSideInput_ struct { + IterableSideInput *StateKey_IterableSideInput `protobuf:"bytes,4,opt,name=iterable_side_input,json=iterableSideInput,proto3,oneof"` +} + +type stateKey_MultimapKeysSideInput_ struct { + MultimapKeysSideInput *StateKey_MultimapKeysSideInput `protobuf:"bytes,5,opt,name=multimap_keys_side_input,json=multimapKeysSideInput,proto3,oneof"` +} + +type stateKey_MultimapKeysValuesSideInput_ struct { + MultimapKeysValuesSideInput *StateKey_MultimapKeysValuesSideInput `protobuf:"bytes,8,opt,name=multimap_keys_values_side_input,json=multimapKeysValuesSideInput,proto3,oneof"` +} + +type stateKey_MultimapKeysUserState_ struct { + MultimapKeysUserState *StateKey_MultimapKeysUserState `protobuf:"bytes,6,opt,name=multimap_keys_user_state,json=multimapKeysUserState,proto3,oneof"` +} + +type stateKey_MultimapUserState_ struct { + MultimapUserState *StateKey_MultimapUserState `protobuf:"bytes,7,opt,name=multimap_user_state,json=multimapUserState,proto3,oneof"` +} + +type stateKey_OrderedListUserState_ struct { + OrderedListUserState *StateKey_OrderedListUserState `protobuf:"bytes,9,opt,name=ordered_list_user_state,json=orderedListUserState,proto3,oneof"` +} + +func (*stateKey_Runner_) isStateKey_Type() {} + +func (*stateKey_MultimapSideInput_) isStateKey_Type() {} + +func (*stateKey_BagUserState_) isStateKey_Type() {} + +func (*stateKey_IterableSideInput_) isStateKey_Type() {} + +func (*stateKey_MultimapKeysSideInput_) isStateKey_Type() {} + +func (*stateKey_MultimapKeysValuesSideInput_) isStateKey_Type() {} + +func (*stateKey_MultimapKeysUserState_) isStateKey_Type() {} + +func (*stateKey_MultimapUserState_) isStateKey_Type() {} + +func (*stateKey_OrderedListUserState_) isStateKey_Type() {} + +// A request to get state. +type StateGetRequest struct { + state protoimpl.MessageState `protogen:"opaque.v1"` + xxx_hidden_ContinuationToken []byte `protobuf:"bytes,1,opt,name=continuation_token,json=continuationToken,proto3" json:"continuation_token,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *StateGetRequest) Reset() { + *x = StateGetRequest{} + mi := &file_org_apache_beam_model_fn_execution_v1_beam_fn_api_proto_msgTypes[29] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *StateGetRequest) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*StateGetRequest) ProtoMessage() {} + +func (x *StateGetRequest) ProtoReflect() protoreflect.Message { + mi := &file_org_apache_beam_model_fn_execution_v1_beam_fn_api_proto_msgTypes[29] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +func (x *StateGetRequest) GetContinuationToken() []byte { + if x != nil { + return x.xxx_hidden_ContinuationToken + } + return nil +} + +func (x *StateGetRequest) SetContinuationToken(v []byte) { + if v == nil { + v = []byte{} + } + x.xxx_hidden_ContinuationToken = v +} + +type StateGetRequest_builder struct { + _ [0]func() // Prevents comparability and use of unkeyed literals for the builder. + + // (Optional) If specified, signals to the runner that the response + // should resume from the following continuation token. + // + // If unspecified, signals to the runner that the response should start + // from the beginning of the logical continuable stream. + ContinuationToken []byte +} + +func (b0 StateGetRequest_builder) Build() *StateGetRequest { + m0 := &StateGetRequest{} + b, x := &b0, m0 + _, _ = b, x + x.xxx_hidden_ContinuationToken = b.ContinuationToken + return m0 +} + +// A response to get state representing a logical byte stream which can be +// continued using the state API. +type StateGetResponse struct { + state protoimpl.MessageState `protogen:"opaque.v1"` + xxx_hidden_ContinuationToken []byte `protobuf:"bytes,1,opt,name=continuation_token,json=continuationToken,proto3" json:"continuation_token,omitempty"` + xxx_hidden_Data []byte `protobuf:"bytes,2,opt,name=data,proto3" json:"data,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *StateGetResponse) Reset() { + *x = StateGetResponse{} + mi := &file_org_apache_beam_model_fn_execution_v1_beam_fn_api_proto_msgTypes[30] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *StateGetResponse) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*StateGetResponse) ProtoMessage() {} + +func (x *StateGetResponse) ProtoReflect() protoreflect.Message { + mi := &file_org_apache_beam_model_fn_execution_v1_beam_fn_api_proto_msgTypes[30] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +func (x *StateGetResponse) GetContinuationToken() []byte { + if x != nil { + return x.xxx_hidden_ContinuationToken + } + return nil +} + +func (x *StateGetResponse) GetData() []byte { + if x != nil { + return x.xxx_hidden_Data + } + return nil +} + +func (x *StateGetResponse) SetContinuationToken(v []byte) { + if v == nil { + v = []byte{} + } + x.xxx_hidden_ContinuationToken = v +} + +func (x *StateGetResponse) SetData(v []byte) { + if v == nil { + v = []byte{} + } + x.xxx_hidden_Data = v +} + +type StateGetResponse_builder struct { + _ [0]func() // Prevents comparability and use of unkeyed literals for the builder. + + // (Optional) If specified, represents a token which can be used with the + // state API to get the next chunk of this logical byte stream. The end of + // the logical byte stream is signalled by this field being unset. + ContinuationToken []byte + // Represents a part of a logical byte stream. Elements within + // the logical byte stream are encoded in the nested context and + // concatenated together. + // + // See also the note about OrderedListState in StateAppendRequest. + Data []byte +} + +func (b0 StateGetResponse_builder) Build() *StateGetResponse { + m0 := &StateGetResponse{} + b, x := &b0, m0 + _, _ = b, x + x.xxx_hidden_ContinuationToken = b.ContinuationToken + x.xxx_hidden_Data = b.Data + return m0 +} + +// A request to append state. +type StateAppendRequest struct { + state protoimpl.MessageState `protogen:"opaque.v1"` + xxx_hidden_Data []byte `protobuf:"bytes,1,opt,name=data,proto3" json:"data,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *StateAppendRequest) Reset() { + *x = StateAppendRequest{} + mi := &file_org_apache_beam_model_fn_execution_v1_beam_fn_api_proto_msgTypes[31] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *StateAppendRequest) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*StateAppendRequest) ProtoMessage() {} + +func (x *StateAppendRequest) ProtoReflect() protoreflect.Message { + mi := &file_org_apache_beam_model_fn_execution_v1_beam_fn_api_proto_msgTypes[31] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +func (x *StateAppendRequest) GetData() []byte { + if x != nil { + return x.xxx_hidden_Data + } + return nil +} + +func (x *StateAppendRequest) SetData(v []byte) { + if v == nil { + v = []byte{} + } + x.xxx_hidden_Data = v +} + +type StateAppendRequest_builder struct { + _ [0]func() // Prevents comparability and use of unkeyed literals for the builder. + + // Represents a part of a logical byte stream. Elements within + // the logical byte stream are encoded in the nested context and + // multiple append requests are concatenated together. + // + // For OrderedListState, elements of should be encoded with the + // beam:coder:kv:v1 coder, where the first (key) component must be a + // beam:coder:varint:v1 and the second (value) component must be encoded + // with a beam:coder:length_prefix:v1 coder. + Data []byte +} + +func (b0 StateAppendRequest_builder) Build() *StateAppendRequest { + m0 := &StateAppendRequest{} + b, x := &b0, m0 + _, _ = b, x + x.xxx_hidden_Data = b.Data + return m0 +} + +// A response to append state. +type StateAppendResponse struct { + state protoimpl.MessageState `protogen:"opaque.v1"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *StateAppendResponse) Reset() { + *x = StateAppendResponse{} + mi := &file_org_apache_beam_model_fn_execution_v1_beam_fn_api_proto_msgTypes[32] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *StateAppendResponse) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*StateAppendResponse) ProtoMessage() {} + +func (x *StateAppendResponse) ProtoReflect() protoreflect.Message { + mi := &file_org_apache_beam_model_fn_execution_v1_beam_fn_api_proto_msgTypes[32] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +type StateAppendResponse_builder struct { + _ [0]func() // Prevents comparability and use of unkeyed literals for the builder. + +} + +func (b0 StateAppendResponse_builder) Build() *StateAppendResponse { + m0 := &StateAppendResponse{} + b, x := &b0, m0 + _, _ = b, x + return m0 +} + +// A request to clear state. +type StateClearRequest struct { + state protoimpl.MessageState `protogen:"opaque.v1"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *StateClearRequest) Reset() { + *x = StateClearRequest{} + mi := &file_org_apache_beam_model_fn_execution_v1_beam_fn_api_proto_msgTypes[33] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *StateClearRequest) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*StateClearRequest) ProtoMessage() {} + +func (x *StateClearRequest) ProtoReflect() protoreflect.Message { + mi := &file_org_apache_beam_model_fn_execution_v1_beam_fn_api_proto_msgTypes[33] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +type StateClearRequest_builder struct { + _ [0]func() // Prevents comparability and use of unkeyed literals for the builder. + +} + +func (b0 StateClearRequest_builder) Build() *StateClearRequest { + m0 := &StateClearRequest{} + b, x := &b0, m0 + _, _ = b, x + return m0 +} + +// A response to clear state. +type StateClearResponse struct { + state protoimpl.MessageState `protogen:"opaque.v1"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *StateClearResponse) Reset() { + *x = StateClearResponse{} + mi := &file_org_apache_beam_model_fn_execution_v1_beam_fn_api_proto_msgTypes[34] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *StateClearResponse) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*StateClearResponse) ProtoMessage() {} + +func (x *StateClearResponse) ProtoReflect() protoreflect.Message { + mi := &file_org_apache_beam_model_fn_execution_v1_beam_fn_api_proto_msgTypes[34] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +type StateClearResponse_builder struct { + _ [0]func() // Prevents comparability and use of unkeyed literals for the builder. + +} + +func (b0 StateClearResponse_builder) Build() *StateClearResponse { + m0 := &StateClearResponse{} + b, x := &b0, m0 + _, _ = b, x + return m0 +} + +// A message describes a sort key range [start, end). +type OrderedListRange struct { + state protoimpl.MessageState `protogen:"opaque.v1"` + xxx_hidden_Start int64 `protobuf:"varint,1,opt,name=start,proto3" json:"start,omitempty"` + xxx_hidden_End int64 `protobuf:"varint,2,opt,name=end,proto3" json:"end,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *OrderedListRange) Reset() { + *x = OrderedListRange{} + mi := &file_org_apache_beam_model_fn_execution_v1_beam_fn_api_proto_msgTypes[35] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *OrderedListRange) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*OrderedListRange) ProtoMessage() {} + +func (x *OrderedListRange) ProtoReflect() protoreflect.Message { + mi := &file_org_apache_beam_model_fn_execution_v1_beam_fn_api_proto_msgTypes[35] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +func (x *OrderedListRange) GetStart() int64 { + if x != nil { + return x.xxx_hidden_Start + } + return 0 +} + +func (x *OrderedListRange) GetEnd() int64 { + if x != nil { + return x.xxx_hidden_End + } + return 0 +} + +func (x *OrderedListRange) SetStart(v int64) { + x.xxx_hidden_Start = v +} + +func (x *OrderedListRange) SetEnd(v int64) { + x.xxx_hidden_End = v +} + +type OrderedListRange_builder struct { + _ [0]func() // Prevents comparability and use of unkeyed literals for the builder. + + Start int64 + End int64 +} + +func (b0 OrderedListRange_builder) Build() *OrderedListRange { + m0 := &OrderedListRange{} + b, x := &b0, m0 + _, _ = b, x + x.xxx_hidden_Start = b.Start + x.xxx_hidden_End = b.End + return m0 +} + +// A log entry +type LogEntry struct { + state protoimpl.MessageState `protogen:"opaque.v1"` + xxx_hidden_Severity LogEntry_Severity_Enum `protobuf:"varint,1,opt,name=severity,proto3,enum=org.apache.beam.model.fn_execution.v1.LogEntry_Severity_Enum" json:"severity,omitempty"` + xxx_hidden_Timestamp *timestamppb.Timestamp `protobuf:"bytes,2,opt,name=timestamp,proto3" json:"timestamp,omitempty"` + xxx_hidden_Message string `protobuf:"bytes,3,opt,name=message,proto3" json:"message,omitempty"` + xxx_hidden_Trace string `protobuf:"bytes,4,opt,name=trace,proto3" json:"trace,omitempty"` + xxx_hidden_InstructionId string `protobuf:"bytes,5,opt,name=instruction_id,json=instructionId,proto3" json:"instruction_id,omitempty"` + xxx_hidden_TransformId string `protobuf:"bytes,6,opt,name=transform_id,json=transformId,proto3" json:"transform_id,omitempty"` + xxx_hidden_LogLocation string `protobuf:"bytes,7,opt,name=log_location,json=logLocation,proto3" json:"log_location,omitempty"` + xxx_hidden_Thread string `protobuf:"bytes,8,opt,name=thread,proto3" json:"thread,omitempty"` + xxx_hidden_CustomData *structpb.Struct `protobuf:"bytes,9,opt,name=custom_data,json=customData,proto3" json:"custom_data,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *LogEntry) Reset() { + *x = LogEntry{} + mi := &file_org_apache_beam_model_fn_execution_v1_beam_fn_api_proto_msgTypes[36] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *LogEntry) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*LogEntry) ProtoMessage() {} + +func (x *LogEntry) ProtoReflect() protoreflect.Message { + mi := &file_org_apache_beam_model_fn_execution_v1_beam_fn_api_proto_msgTypes[36] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +func (x *LogEntry) GetSeverity() LogEntry_Severity_Enum { + if x != nil { + return x.xxx_hidden_Severity + } + return LogEntry_Severity_UNSPECIFIED +} + +func (x *LogEntry) GetTimestamp() *timestamppb.Timestamp { + if x != nil { + return x.xxx_hidden_Timestamp + } + return nil +} + +func (x *LogEntry) GetMessage() string { + if x != nil { + return x.xxx_hidden_Message + } + return "" +} + +func (x *LogEntry) GetTrace() string { + if x != nil { + return x.xxx_hidden_Trace + } + return "" +} + +func (x *LogEntry) GetInstructionId() string { + if x != nil { + return x.xxx_hidden_InstructionId + } + return "" +} + +func (x *LogEntry) GetTransformId() string { + if x != nil { + return x.xxx_hidden_TransformId + } + return "" +} + +func (x *LogEntry) GetLogLocation() string { + if x != nil { + return x.xxx_hidden_LogLocation + } + return "" +} + +func (x *LogEntry) GetThread() string { + if x != nil { + return x.xxx_hidden_Thread + } + return "" +} + +func (x *LogEntry) GetCustomData() *structpb.Struct { + if x != nil { + return x.xxx_hidden_CustomData + } + return nil +} + +func (x *LogEntry) SetSeverity(v LogEntry_Severity_Enum) { + x.xxx_hidden_Severity = v +} + +func (x *LogEntry) SetTimestamp(v *timestamppb.Timestamp) { + x.xxx_hidden_Timestamp = v +} + +func (x *LogEntry) SetMessage(v string) { + x.xxx_hidden_Message = v +} + +func (x *LogEntry) SetTrace(v string) { + x.xxx_hidden_Trace = v +} + +func (x *LogEntry) SetInstructionId(v string) { + x.xxx_hidden_InstructionId = v +} + +func (x *LogEntry) SetTransformId(v string) { + x.xxx_hidden_TransformId = v +} + +func (x *LogEntry) SetLogLocation(v string) { + x.xxx_hidden_LogLocation = v +} + +func (x *LogEntry) SetThread(v string) { + x.xxx_hidden_Thread = v +} + +func (x *LogEntry) SetCustomData(v *structpb.Struct) { + x.xxx_hidden_CustomData = v +} + +func (x *LogEntry) HasTimestamp() bool { + if x == nil { + return false + } + return x.xxx_hidden_Timestamp != nil +} + +func (x *LogEntry) HasCustomData() bool { + if x == nil { + return false + } + return x.xxx_hidden_CustomData != nil +} + +func (x *LogEntry) ClearTimestamp() { + x.xxx_hidden_Timestamp = nil +} + +func (x *LogEntry) ClearCustomData() { + x.xxx_hidden_CustomData = nil +} + +type LogEntry_builder struct { + _ [0]func() // Prevents comparability and use of unkeyed literals for the builder. + + // (Required) The severity of the log statement. + Severity LogEntry_Severity_Enum + // (Required) The time at which this log statement occurred. + Timestamp *timestamppb.Timestamp + // (Required) A human readable message. + Message string + // (Optional) An optional trace of the functions involved. For example, in + // Java this can include multiple causes and multiple suppressed exceptions. + Trace string + // (Optional) A reference to the instruction this log statement is associated + // with. + InstructionId string + // (Optional) A reference to the transform this log statement is + // associated with. + TransformId string + // (Optional) Human-readable name of the function or method being invoked, + // with optional context such as the class or package name. The format can + // vary by language. For example: + // + // qual.if.ied.Class.method (Java) + // dir/package.func (Go) + // module.function (Python) + // file.cc:382 (C++) + LogLocation string + // (Optional) The name of the thread this log statement is associated with. + Thread string + // (Optional) Additional structured data to log. + // Keys are limited to these characters: [a-zA-Z_-] + CustomData *structpb.Struct +} + +func (b0 LogEntry_builder) Build() *LogEntry { + m0 := &LogEntry{} + b, x := &b0, m0 + _, _ = b, x + x.xxx_hidden_Severity = b.Severity + x.xxx_hidden_Timestamp = b.Timestamp + x.xxx_hidden_Message = b.Message + x.xxx_hidden_Trace = b.Trace + x.xxx_hidden_InstructionId = b.InstructionId + x.xxx_hidden_TransformId = b.TransformId + x.xxx_hidden_LogLocation = b.LogLocation + x.xxx_hidden_Thread = b.Thread + x.xxx_hidden_CustomData = b.CustomData + return m0 +} + +type LogControl struct { + state protoimpl.MessageState `protogen:"opaque.v1"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *LogControl) Reset() { + *x = LogControl{} + mi := &file_org_apache_beam_model_fn_execution_v1_beam_fn_api_proto_msgTypes[37] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *LogControl) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*LogControl) ProtoMessage() {} + +func (x *LogControl) ProtoReflect() protoreflect.Message { + mi := &file_org_apache_beam_model_fn_execution_v1_beam_fn_api_proto_msgTypes[37] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +type LogControl_builder struct { + _ [0]func() // Prevents comparability and use of unkeyed literals for the builder. + +} + +func (b0 LogControl_builder) Build() *LogControl { + m0 := &LogControl{} + b, x := &b0, m0 + _, _ = b, x + return m0 +} + +type StartWorkerRequest struct { + state protoimpl.MessageState `protogen:"opaque.v1"` + xxx_hidden_WorkerId string `protobuf:"bytes,1,opt,name=worker_id,json=workerId,proto3" json:"worker_id,omitempty"` + xxx_hidden_ControlEndpoint *pipeline_v1.ApiServiceDescriptor `protobuf:"bytes,2,opt,name=control_endpoint,json=controlEndpoint,proto3" json:"control_endpoint,omitempty"` + xxx_hidden_LoggingEndpoint *pipeline_v1.ApiServiceDescriptor `protobuf:"bytes,3,opt,name=logging_endpoint,json=loggingEndpoint,proto3" json:"logging_endpoint,omitempty"` + xxx_hidden_ArtifactEndpoint *pipeline_v1.ApiServiceDescriptor `protobuf:"bytes,4,opt,name=artifact_endpoint,json=artifactEndpoint,proto3" json:"artifact_endpoint,omitempty"` + xxx_hidden_ProvisionEndpoint *pipeline_v1.ApiServiceDescriptor `protobuf:"bytes,5,opt,name=provision_endpoint,json=provisionEndpoint,proto3" json:"provision_endpoint,omitempty"` + xxx_hidden_Params map[string]string `protobuf:"bytes,10,rep,name=params,proto3" json:"params,omitempty" protobuf_key:"bytes,1,opt,name=key" protobuf_val:"bytes,2,opt,name=value"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *StartWorkerRequest) Reset() { + *x = StartWorkerRequest{} + mi := &file_org_apache_beam_model_fn_execution_v1_beam_fn_api_proto_msgTypes[38] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *StartWorkerRequest) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*StartWorkerRequest) ProtoMessage() {} + +func (x *StartWorkerRequest) ProtoReflect() protoreflect.Message { + mi := &file_org_apache_beam_model_fn_execution_v1_beam_fn_api_proto_msgTypes[38] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +func (x *StartWorkerRequest) GetWorkerId() string { + if x != nil { + return x.xxx_hidden_WorkerId + } + return "" +} + +func (x *StartWorkerRequest) GetControlEndpoint() *pipeline_v1.ApiServiceDescriptor { + if x != nil { + return x.xxx_hidden_ControlEndpoint + } + return nil +} + +func (x *StartWorkerRequest) GetLoggingEndpoint() *pipeline_v1.ApiServiceDescriptor { + if x != nil { + return x.xxx_hidden_LoggingEndpoint + } + return nil +} + +func (x *StartWorkerRequest) GetArtifactEndpoint() *pipeline_v1.ApiServiceDescriptor { + if x != nil { + return x.xxx_hidden_ArtifactEndpoint + } + return nil +} + +func (x *StartWorkerRequest) GetProvisionEndpoint() *pipeline_v1.ApiServiceDescriptor { + if x != nil { + return x.xxx_hidden_ProvisionEndpoint + } + return nil +} + +func (x *StartWorkerRequest) GetParams() map[string]string { + if x != nil { + return x.xxx_hidden_Params + } + return nil +} + +func (x *StartWorkerRequest) SetWorkerId(v string) { + x.xxx_hidden_WorkerId = v +} + +func (x *StartWorkerRequest) SetControlEndpoint(v *pipeline_v1.ApiServiceDescriptor) { + x.xxx_hidden_ControlEndpoint = v +} + +func (x *StartWorkerRequest) SetLoggingEndpoint(v *pipeline_v1.ApiServiceDescriptor) { + x.xxx_hidden_LoggingEndpoint = v +} + +func (x *StartWorkerRequest) SetArtifactEndpoint(v *pipeline_v1.ApiServiceDescriptor) { + x.xxx_hidden_ArtifactEndpoint = v +} + +func (x *StartWorkerRequest) SetProvisionEndpoint(v *pipeline_v1.ApiServiceDescriptor) { + x.xxx_hidden_ProvisionEndpoint = v +} + +func (x *StartWorkerRequest) SetParams(v map[string]string) { + x.xxx_hidden_Params = v +} + +func (x *StartWorkerRequest) HasControlEndpoint() bool { + if x == nil { + return false + } + return x.xxx_hidden_ControlEndpoint != nil +} + +func (x *StartWorkerRequest) HasLoggingEndpoint() bool { + if x == nil { + return false + } + return x.xxx_hidden_LoggingEndpoint != nil +} + +func (x *StartWorkerRequest) HasArtifactEndpoint() bool { + if x == nil { + return false + } + return x.xxx_hidden_ArtifactEndpoint != nil +} + +func (x *StartWorkerRequest) HasProvisionEndpoint() bool { + if x == nil { + return false + } + return x.xxx_hidden_ProvisionEndpoint != nil +} + +func (x *StartWorkerRequest) ClearControlEndpoint() { + x.xxx_hidden_ControlEndpoint = nil +} + +func (x *StartWorkerRequest) ClearLoggingEndpoint() { + x.xxx_hidden_LoggingEndpoint = nil +} + +func (x *StartWorkerRequest) ClearArtifactEndpoint() { + x.xxx_hidden_ArtifactEndpoint = nil +} + +func (x *StartWorkerRequest) ClearProvisionEndpoint() { + x.xxx_hidden_ProvisionEndpoint = nil +} + +type StartWorkerRequest_builder struct { + _ [0]func() // Prevents comparability and use of unkeyed literals for the builder. + + WorkerId string + ControlEndpoint *pipeline_v1.ApiServiceDescriptor + LoggingEndpoint *pipeline_v1.ApiServiceDescriptor + ArtifactEndpoint *pipeline_v1.ApiServiceDescriptor + ProvisionEndpoint *pipeline_v1.ApiServiceDescriptor + Params map[string]string +} + +func (b0 StartWorkerRequest_builder) Build() *StartWorkerRequest { + m0 := &StartWorkerRequest{} + b, x := &b0, m0 + _, _ = b, x + x.xxx_hidden_WorkerId = b.WorkerId + x.xxx_hidden_ControlEndpoint = b.ControlEndpoint + x.xxx_hidden_LoggingEndpoint = b.LoggingEndpoint + x.xxx_hidden_ArtifactEndpoint = b.ArtifactEndpoint + x.xxx_hidden_ProvisionEndpoint = b.ProvisionEndpoint + x.xxx_hidden_Params = b.Params + return m0 +} + +type StartWorkerResponse struct { + state protoimpl.MessageState `protogen:"opaque.v1"` + xxx_hidden_Error string `protobuf:"bytes,1,opt,name=error,proto3" json:"error,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *StartWorkerResponse) Reset() { + *x = StartWorkerResponse{} + mi := &file_org_apache_beam_model_fn_execution_v1_beam_fn_api_proto_msgTypes[39] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *StartWorkerResponse) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*StartWorkerResponse) ProtoMessage() {} + +func (x *StartWorkerResponse) ProtoReflect() protoreflect.Message { + mi := &file_org_apache_beam_model_fn_execution_v1_beam_fn_api_proto_msgTypes[39] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +func (x *StartWorkerResponse) GetError() string { + if x != nil { + return x.xxx_hidden_Error + } + return "" +} + +func (x *StartWorkerResponse) SetError(v string) { + x.xxx_hidden_Error = v +} + +type StartWorkerResponse_builder struct { + _ [0]func() // Prevents comparability and use of unkeyed literals for the builder. + + Error string +} + +func (b0 StartWorkerResponse_builder) Build() *StartWorkerResponse { + m0 := &StartWorkerResponse{} + b, x := &b0, m0 + _, _ = b, x + x.xxx_hidden_Error = b.Error + return m0 +} + +type StopWorkerRequest struct { + state protoimpl.MessageState `protogen:"opaque.v1"` + xxx_hidden_WorkerId string `protobuf:"bytes,1,opt,name=worker_id,json=workerId,proto3" json:"worker_id,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *StopWorkerRequest) Reset() { + *x = StopWorkerRequest{} + mi := &file_org_apache_beam_model_fn_execution_v1_beam_fn_api_proto_msgTypes[40] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *StopWorkerRequest) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*StopWorkerRequest) ProtoMessage() {} + +func (x *StopWorkerRequest) ProtoReflect() protoreflect.Message { + mi := &file_org_apache_beam_model_fn_execution_v1_beam_fn_api_proto_msgTypes[40] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +func (x *StopWorkerRequest) GetWorkerId() string { + if x != nil { + return x.xxx_hidden_WorkerId + } + return "" +} + +func (x *StopWorkerRequest) SetWorkerId(v string) { + x.xxx_hidden_WorkerId = v +} + +type StopWorkerRequest_builder struct { + _ [0]func() // Prevents comparability and use of unkeyed literals for the builder. + + WorkerId string +} + +func (b0 StopWorkerRequest_builder) Build() *StopWorkerRequest { + m0 := &StopWorkerRequest{} + b, x := &b0, m0 + _, _ = b, x + x.xxx_hidden_WorkerId = b.WorkerId + return m0 +} + +type StopWorkerResponse struct { + state protoimpl.MessageState `protogen:"opaque.v1"` + xxx_hidden_Error string `protobuf:"bytes,1,opt,name=error,proto3" json:"error,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *StopWorkerResponse) Reset() { + *x = StopWorkerResponse{} + mi := &file_org_apache_beam_model_fn_execution_v1_beam_fn_api_proto_msgTypes[41] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *StopWorkerResponse) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*StopWorkerResponse) ProtoMessage() {} + +func (x *StopWorkerResponse) ProtoReflect() protoreflect.Message { + mi := &file_org_apache_beam_model_fn_execution_v1_beam_fn_api_proto_msgTypes[41] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +func (x *StopWorkerResponse) GetError() string { + if x != nil { + return x.xxx_hidden_Error + } + return "" +} + +func (x *StopWorkerResponse) SetError(v string) { + x.xxx_hidden_Error = v +} + +type StopWorkerResponse_builder struct { + _ [0]func() // Prevents comparability and use of unkeyed literals for the builder. + + Error string +} + +func (b0 StopWorkerResponse_builder) Build() *StopWorkerResponse { + m0 := &StopWorkerResponse{} + b, x := &b0, m0 + _, _ = b, x + x.xxx_hidden_Error = b.Error + return m0 +} + +// Request from runner to SDK Harness asking for its status. For more details see +// https://s.apache.org/beam-fn-api-harness-status +type WorkerStatusRequest struct { + state protoimpl.MessageState `protogen:"opaque.v1"` + xxx_hidden_Id string `protobuf:"bytes,1,opt,name=id,proto3" json:"id,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *WorkerStatusRequest) Reset() { + *x = WorkerStatusRequest{} + mi := &file_org_apache_beam_model_fn_execution_v1_beam_fn_api_proto_msgTypes[42] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *WorkerStatusRequest) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*WorkerStatusRequest) ProtoMessage() {} + +func (x *WorkerStatusRequest) ProtoReflect() protoreflect.Message { + mi := &file_org_apache_beam_model_fn_execution_v1_beam_fn_api_proto_msgTypes[42] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +func (x *WorkerStatusRequest) GetId() string { + if x != nil { + return x.xxx_hidden_Id + } + return "" +} + +func (x *WorkerStatusRequest) SetId(v string) { + x.xxx_hidden_Id = v +} + +type WorkerStatusRequest_builder struct { + _ [0]func() // Prevents comparability and use of unkeyed literals for the builder. + + // (Required) Unique ID identifying this request. + Id string +} + +func (b0 WorkerStatusRequest_builder) Build() *WorkerStatusRequest { + m0 := &WorkerStatusRequest{} + b, x := &b0, m0 + _, _ = b, x + x.xxx_hidden_Id = b.Id + return m0 +} + +// Response from SDK Harness to runner containing the debug related status info. +type WorkerStatusResponse struct { + state protoimpl.MessageState `protogen:"opaque.v1"` + xxx_hidden_Id string `protobuf:"bytes,1,opt,name=id,proto3" json:"id,omitempty"` + xxx_hidden_Error string `protobuf:"bytes,2,opt,name=error,proto3" json:"error,omitempty"` + xxx_hidden_StatusInfo string `protobuf:"bytes,3,opt,name=status_info,json=statusInfo,proto3" json:"status_info,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *WorkerStatusResponse) Reset() { + *x = WorkerStatusResponse{} + mi := &file_org_apache_beam_model_fn_execution_v1_beam_fn_api_proto_msgTypes[43] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *WorkerStatusResponse) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*WorkerStatusResponse) ProtoMessage() {} + +func (x *WorkerStatusResponse) ProtoReflect() protoreflect.Message { + mi := &file_org_apache_beam_model_fn_execution_v1_beam_fn_api_proto_msgTypes[43] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +func (x *WorkerStatusResponse) GetId() string { + if x != nil { + return x.xxx_hidden_Id + } + return "" +} + +func (x *WorkerStatusResponse) GetError() string { + if x != nil { + return x.xxx_hidden_Error + } + return "" +} + +func (x *WorkerStatusResponse) GetStatusInfo() string { + if x != nil { + return x.xxx_hidden_StatusInfo + } + return "" +} + +func (x *WorkerStatusResponse) SetId(v string) { + x.xxx_hidden_Id = v +} + +func (x *WorkerStatusResponse) SetError(v string) { + x.xxx_hidden_Error = v +} + +func (x *WorkerStatusResponse) SetStatusInfo(v string) { + x.xxx_hidden_StatusInfo = v +} + +type WorkerStatusResponse_builder struct { + _ [0]func() // Prevents comparability and use of unkeyed literals for the builder. + + // (Required) Unique ID from the original request. + Id string + // (Optional) Error message if exception encountered generating the status response. + Error string + // (Optional) Status debugging info reported by SDK harness worker. Content and + // format is not strongly enforced but should be print-friendly and + // appropriate as an HTTP response body for end user. For details of the preferred + // info to include in the message see + // https://s.apache.org/beam-fn-api-harness-status + StatusInfo string +} + +func (b0 WorkerStatusResponse_builder) Build() *WorkerStatusResponse { + m0 := &WorkerStatusResponse{} + b, x := &b0, m0 + _, _ = b, x + x.xxx_hidden_Id = b.Id + x.xxx_hidden_Error = b.Error + x.xxx_hidden_StatusInfo = b.StatusInfo + return m0 +} + +type SampledElement_Exception struct { + state protoimpl.MessageState `protogen:"opaque.v1"` + xxx_hidden_InstructionId string `protobuf:"bytes,1,opt,name=instruction_id,json=instructionId,proto3" json:"instruction_id,omitempty"` + xxx_hidden_TransformId string `protobuf:"bytes,2,opt,name=transform_id,json=transformId,proto3" json:"transform_id,omitempty"` + xxx_hidden_Error string `protobuf:"bytes,3,opt,name=error,proto3" json:"error,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *SampledElement_Exception) Reset() { + *x = SampledElement_Exception{} + mi := &file_org_apache_beam_model_fn_execution_v1_beam_fn_api_proto_msgTypes[44] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *SampledElement_Exception) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*SampledElement_Exception) ProtoMessage() {} + +func (x *SampledElement_Exception) ProtoReflect() protoreflect.Message { + mi := &file_org_apache_beam_model_fn_execution_v1_beam_fn_api_proto_msgTypes[44] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +func (x *SampledElement_Exception) GetInstructionId() string { + if x != nil { + return x.xxx_hidden_InstructionId + } + return "" +} + +func (x *SampledElement_Exception) GetTransformId() string { + if x != nil { + return x.xxx_hidden_TransformId + } + return "" +} + +func (x *SampledElement_Exception) GetError() string { + if x != nil { + return x.xxx_hidden_Error + } + return "" +} + +func (x *SampledElement_Exception) SetInstructionId(v string) { + x.xxx_hidden_InstructionId = v +} + +func (x *SampledElement_Exception) SetTransformId(v string) { + x.xxx_hidden_TransformId = v +} + +func (x *SampledElement_Exception) SetError(v string) { + x.xxx_hidden_Error = v +} + +type SampledElement_Exception_builder struct { + _ [0]func() // Prevents comparability and use of unkeyed literals for the builder. + + // (Required) The instruction ID of the associated ProcessBundleRequest. + InstructionId string + // (Required) The transform ID of the executing PTransform during the + // exception. + TransformId string + // (Required) The error message to be displayed to the user. Can use the + // other fields to query for contextual logs. + Error string +} + +func (b0 SampledElement_Exception_builder) Build() *SampledElement_Exception { + m0 := &SampledElement_Exception{} + b, x := &b0, m0 + _, _ = b, x + x.xxx_hidden_InstructionId = b.InstructionId + x.xxx_hidden_TransformId = b.TransformId + x.xxx_hidden_Error = b.Error + return m0 +} + +type SampleDataResponse_ElementList struct { + state protoimpl.MessageState `protogen:"opaque.v1"` + xxx_hidden_Elements *[]*SampledElement `protobuf:"bytes,1,rep,name=elements,proto3" json:"elements,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *SampleDataResponse_ElementList) Reset() { + *x = SampleDataResponse_ElementList{} + mi := &file_org_apache_beam_model_fn_execution_v1_beam_fn_api_proto_msgTypes[45] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *SampleDataResponse_ElementList) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*SampleDataResponse_ElementList) ProtoMessage() {} + +func (x *SampleDataResponse_ElementList) ProtoReflect() protoreflect.Message { + mi := &file_org_apache_beam_model_fn_execution_v1_beam_fn_api_proto_msgTypes[45] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +func (x *SampleDataResponse_ElementList) GetElements() []*SampledElement { + if x != nil { + if x.xxx_hidden_Elements != nil { + return *x.xxx_hidden_Elements + } + } + return nil +} + +func (x *SampleDataResponse_ElementList) SetElements(v []*SampledElement) { + x.xxx_hidden_Elements = &v +} + +type SampleDataResponse_ElementList_builder struct { + _ [0]func() // Prevents comparability and use of unkeyed literals for the builder. + + // Required. The individual elements sampled from a PCollection. + Elements []*SampledElement +} + +func (b0 SampleDataResponse_ElementList_builder) Build() *SampleDataResponse_ElementList { + m0 := &SampleDataResponse_ElementList{} + b, x := &b0, m0 + _, _ = b, x + x.xxx_hidden_Elements = &b.Elements + return m0 +} + +// Contains the cache token and also defines the scope of what the token applies to. +// +// See https://s.apache.org/beam-fn-state-api-and-bundle-processing#heading=h.7ghoih5aig5m +// for additional details on how to use the cache token with the State API +// to cache data across bundle boundaries. +type ProcessBundleRequest_CacheToken struct { + state protoimpl.MessageState `protogen:"opaque.v1"` + xxx_hidden_Type isProcessBundleRequest_CacheToken_Type `protobuf_oneof:"type"` + xxx_hidden_Token []byte `protobuf:"bytes,10,opt,name=token,proto3" json:"token,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *ProcessBundleRequest_CacheToken) Reset() { + *x = ProcessBundleRequest_CacheToken{} + mi := &file_org_apache_beam_model_fn_execution_v1_beam_fn_api_proto_msgTypes[54] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *ProcessBundleRequest_CacheToken) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*ProcessBundleRequest_CacheToken) ProtoMessage() {} + +func (x *ProcessBundleRequest_CacheToken) ProtoReflect() protoreflect.Message { + mi := &file_org_apache_beam_model_fn_execution_v1_beam_fn_api_proto_msgTypes[54] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +func (x *ProcessBundleRequest_CacheToken) GetUserState() *ProcessBundleRequest_CacheToken_UserState { + if x != nil { + if x, ok := x.xxx_hidden_Type.(*processBundleRequest_CacheToken_UserState_); ok { + return x.UserState + } + } + return nil +} + +func (x *ProcessBundleRequest_CacheToken) GetSideInput() *ProcessBundleRequest_CacheToken_SideInput { + if x != nil { + if x, ok := x.xxx_hidden_Type.(*processBundleRequest_CacheToken_SideInput_); ok { + return x.SideInput + } + } + return nil +} + +func (x *ProcessBundleRequest_CacheToken) GetToken() []byte { + if x != nil { + return x.xxx_hidden_Token + } + return nil +} + +func (x *ProcessBundleRequest_CacheToken) SetUserState(v *ProcessBundleRequest_CacheToken_UserState) { + if v == nil { + x.xxx_hidden_Type = nil + return + } + x.xxx_hidden_Type = &processBundleRequest_CacheToken_UserState_{v} +} + +func (x *ProcessBundleRequest_CacheToken) SetSideInput(v *ProcessBundleRequest_CacheToken_SideInput) { + if v == nil { + x.xxx_hidden_Type = nil + return + } + x.xxx_hidden_Type = &processBundleRequest_CacheToken_SideInput_{v} +} + +func (x *ProcessBundleRequest_CacheToken) SetToken(v []byte) { + if v == nil { + v = []byte{} + } + x.xxx_hidden_Token = v +} + +func (x *ProcessBundleRequest_CacheToken) HasType() bool { + if x == nil { + return false + } + return x.xxx_hidden_Type != nil +} + +func (x *ProcessBundleRequest_CacheToken) HasUserState() bool { + if x == nil { + return false + } + _, ok := x.xxx_hidden_Type.(*processBundleRequest_CacheToken_UserState_) + return ok +} + +func (x *ProcessBundleRequest_CacheToken) HasSideInput() bool { + if x == nil { + return false + } + _, ok := x.xxx_hidden_Type.(*processBundleRequest_CacheToken_SideInput_) + return ok +} + +func (x *ProcessBundleRequest_CacheToken) ClearType() { + x.xxx_hidden_Type = nil +} + +func (x *ProcessBundleRequest_CacheToken) ClearUserState() { + if _, ok := x.xxx_hidden_Type.(*processBundleRequest_CacheToken_UserState_); ok { + x.xxx_hidden_Type = nil + } +} + +func (x *ProcessBundleRequest_CacheToken) ClearSideInput() { + if _, ok := x.xxx_hidden_Type.(*processBundleRequest_CacheToken_SideInput_); ok { + x.xxx_hidden_Type = nil + } +} + +const ProcessBundleRequest_CacheToken_Type_not_set_case case_ProcessBundleRequest_CacheToken_Type = 0 +const ProcessBundleRequest_CacheToken_UserState_case case_ProcessBundleRequest_CacheToken_Type = 1 +const ProcessBundleRequest_CacheToken_SideInput_case case_ProcessBundleRequest_CacheToken_Type = 2 + +func (x *ProcessBundleRequest_CacheToken) WhichType() case_ProcessBundleRequest_CacheToken_Type { + if x == nil { + return ProcessBundleRequest_CacheToken_Type_not_set_case + } + switch x.xxx_hidden_Type.(type) { + case *processBundleRequest_CacheToken_UserState_: + return ProcessBundleRequest_CacheToken_UserState_case + case *processBundleRequest_CacheToken_SideInput_: + return ProcessBundleRequest_CacheToken_SideInput_case + default: + return ProcessBundleRequest_CacheToken_Type_not_set_case + } +} + +type ProcessBundleRequest_CacheToken_builder struct { + _ [0]func() // Prevents comparability and use of unkeyed literals for the builder. + + // The scope of a cache token. + + // Fields of oneof xxx_hidden_Type: + UserState *ProcessBundleRequest_CacheToken_UserState + SideInput *ProcessBundleRequest_CacheToken_SideInput + // -- end of xxx_hidden_Type + // An opaque token used with the StateKey to create a globally unique + // identifier. + Token []byte +} + +func (b0 ProcessBundleRequest_CacheToken_builder) Build() *ProcessBundleRequest_CacheToken { + m0 := &ProcessBundleRequest_CacheToken{} + b, x := &b0, m0 + _, _ = b, x + if b.UserState != nil { + x.xxx_hidden_Type = &processBundleRequest_CacheToken_UserState_{b.UserState} + } + if b.SideInput != nil { + x.xxx_hidden_Type = &processBundleRequest_CacheToken_SideInput_{b.SideInput} + } + x.xxx_hidden_Token = b.Token + return m0 +} + +type case_ProcessBundleRequest_CacheToken_Type protoreflect.FieldNumber + +func (x case_ProcessBundleRequest_CacheToken_Type) String() string { + md := file_org_apache_beam_model_fn_execution_v1_beam_fn_api_proto_msgTypes[54].Descriptor() + if x == 0 { + return "not set" + } + return protoimpl.X.MessageFieldStringOf(md, protoreflect.FieldNumber(x)) +} + +type isProcessBundleRequest_CacheToken_Type interface { + isProcessBundleRequest_CacheToken_Type() +} + +type processBundleRequest_CacheToken_UserState_ struct { + UserState *ProcessBundleRequest_CacheToken_UserState `protobuf:"bytes,1,opt,name=user_state,json=userState,proto3,oneof"` +} + +type processBundleRequest_CacheToken_SideInput_ struct { + SideInput *ProcessBundleRequest_CacheToken_SideInput `protobuf:"bytes,2,opt,name=side_input,json=sideInput,proto3,oneof"` +} + +func (*processBundleRequest_CacheToken_UserState_) isProcessBundleRequest_CacheToken_Type() {} + +func (*processBundleRequest_CacheToken_SideInput_) isProcessBundleRequest_CacheToken_Type() {} + +// A flag to indicate a cache token is valid for all user state. +type ProcessBundleRequest_CacheToken_UserState struct { + state protoimpl.MessageState `protogen:"opaque.v1"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *ProcessBundleRequest_CacheToken_UserState) Reset() { + *x = ProcessBundleRequest_CacheToken_UserState{} + mi := &file_org_apache_beam_model_fn_execution_v1_beam_fn_api_proto_msgTypes[55] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *ProcessBundleRequest_CacheToken_UserState) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*ProcessBundleRequest_CacheToken_UserState) ProtoMessage() {} + +func (x *ProcessBundleRequest_CacheToken_UserState) ProtoReflect() protoreflect.Message { + mi := &file_org_apache_beam_model_fn_execution_v1_beam_fn_api_proto_msgTypes[55] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +type ProcessBundleRequest_CacheToken_UserState_builder struct { + _ [0]func() // Prevents comparability and use of unkeyed literals for the builder. + +} + +func (b0 ProcessBundleRequest_CacheToken_UserState_builder) Build() *ProcessBundleRequest_CacheToken_UserState { + m0 := &ProcessBundleRequest_CacheToken_UserState{} + b, x := &b0, m0 + _, _ = b, x + return m0 +} + +// A flag to indicate a cache token is valid for a side input. +type ProcessBundleRequest_CacheToken_SideInput struct { + state protoimpl.MessageState `protogen:"opaque.v1"` + xxx_hidden_TransformId string `protobuf:"bytes,1,opt,name=transform_id,json=transformId,proto3" json:"transform_id,omitempty"` + xxx_hidden_SideInputId string `protobuf:"bytes,2,opt,name=side_input_id,json=sideInputId,proto3" json:"side_input_id,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *ProcessBundleRequest_CacheToken_SideInput) Reset() { + *x = ProcessBundleRequest_CacheToken_SideInput{} + mi := &file_org_apache_beam_model_fn_execution_v1_beam_fn_api_proto_msgTypes[56] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *ProcessBundleRequest_CacheToken_SideInput) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*ProcessBundleRequest_CacheToken_SideInput) ProtoMessage() {} + +func (x *ProcessBundleRequest_CacheToken_SideInput) ProtoReflect() protoreflect.Message { + mi := &file_org_apache_beam_model_fn_execution_v1_beam_fn_api_proto_msgTypes[56] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +func (x *ProcessBundleRequest_CacheToken_SideInput) GetTransformId() string { + if x != nil { + return x.xxx_hidden_TransformId + } + return "" +} + +func (x *ProcessBundleRequest_CacheToken_SideInput) GetSideInputId() string { + if x != nil { + return x.xxx_hidden_SideInputId + } + return "" +} + +func (x *ProcessBundleRequest_CacheToken_SideInput) SetTransformId(v string) { + x.xxx_hidden_TransformId = v +} + +func (x *ProcessBundleRequest_CacheToken_SideInput) SetSideInputId(v string) { + x.xxx_hidden_SideInputId = v +} + +type ProcessBundleRequest_CacheToken_SideInput_builder struct { + _ [0]func() // Prevents comparability and use of unkeyed literals for the builder. + + // (Required) The id of the PTransform containing a side input. + TransformId string + // (Required) The id of the side input. + SideInputId string +} + +func (b0 ProcessBundleRequest_CacheToken_SideInput_builder) Build() *ProcessBundleRequest_CacheToken_SideInput { + m0 := &ProcessBundleRequest_CacheToken_SideInput{} + b, x := &b0, m0 + _, _ = b, x + x.xxx_hidden_TransformId = b.TransformId + x.xxx_hidden_SideInputId = b.SideInputId + return m0 +} + +// A message specifying the desired split for a single transform. +type ProcessBundleSplitRequest_DesiredSplit struct { + state protoimpl.MessageState `protogen:"opaque.v1"` + xxx_hidden_FractionOfRemainder float64 `protobuf:"fixed64,1,opt,name=fraction_of_remainder,json=fractionOfRemainder,proto3" json:"fraction_of_remainder,omitempty"` + xxx_hidden_AllowedSplitPoints []int64 `protobuf:"varint,3,rep,packed,name=allowed_split_points,json=allowedSplitPoints,proto3" json:"allowed_split_points,omitempty"` + xxx_hidden_EstimatedInputElements int64 `protobuf:"varint,2,opt,name=estimated_input_elements,json=estimatedInputElements,proto3" json:"estimated_input_elements,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *ProcessBundleSplitRequest_DesiredSplit) Reset() { + *x = ProcessBundleSplitRequest_DesiredSplit{} + mi := &file_org_apache_beam_model_fn_execution_v1_beam_fn_api_proto_msgTypes[60] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *ProcessBundleSplitRequest_DesiredSplit) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*ProcessBundleSplitRequest_DesiredSplit) ProtoMessage() {} + +func (x *ProcessBundleSplitRequest_DesiredSplit) ProtoReflect() protoreflect.Message { + mi := &file_org_apache_beam_model_fn_execution_v1_beam_fn_api_proto_msgTypes[60] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +func (x *ProcessBundleSplitRequest_DesiredSplit) GetFractionOfRemainder() float64 { + if x != nil { + return x.xxx_hidden_FractionOfRemainder + } + return 0 +} + +func (x *ProcessBundleSplitRequest_DesiredSplit) GetAllowedSplitPoints() []int64 { + if x != nil { + return x.xxx_hidden_AllowedSplitPoints + } + return nil +} + +func (x *ProcessBundleSplitRequest_DesiredSplit) GetEstimatedInputElements() int64 { + if x != nil { + return x.xxx_hidden_EstimatedInputElements + } + return 0 +} + +func (x *ProcessBundleSplitRequest_DesiredSplit) SetFractionOfRemainder(v float64) { + x.xxx_hidden_FractionOfRemainder = v +} + +func (x *ProcessBundleSplitRequest_DesiredSplit) SetAllowedSplitPoints(v []int64) { + x.xxx_hidden_AllowedSplitPoints = v +} + +func (x *ProcessBundleSplitRequest_DesiredSplit) SetEstimatedInputElements(v int64) { + x.xxx_hidden_EstimatedInputElements = v +} + +type ProcessBundleSplitRequest_DesiredSplit_builder struct { + _ [0]func() // Prevents comparability and use of unkeyed literals for the builder. + + // (Required) The fraction of known work remaining in this bundle + // for this transform that should be kept by the SDK after this split. + // + // Set to 0 to "checkpoint" as soon as possible (keeping as little work as + // possible and returning the remainder). + FractionOfRemainder float64 + // (Optional) A set of allowed element indices where the SDK may split. When + // this is empty, there are no constraints on where to split. + AllowedSplitPoints []int64 + // (Required for gRPC Read operation transforms) Number of total elements + // expected to be sent to this GrpcRead operation, required to correctly + // account for unreceived data when determining where to split. + EstimatedInputElements int64 +} + +func (b0 ProcessBundleSplitRequest_DesiredSplit_builder) Build() *ProcessBundleSplitRequest_DesiredSplit { + m0 := &ProcessBundleSplitRequest_DesiredSplit{} + b, x := &b0, m0 + _, _ = b, x + x.xxx_hidden_FractionOfRemainder = b.FractionOfRemainder + x.xxx_hidden_AllowedSplitPoints = b.AllowedSplitPoints + x.xxx_hidden_EstimatedInputElements = b.EstimatedInputElements + return m0 +} + +// Represents contiguous portions of the data channel that are either +// entirely processed or entirely unprocessed and belong to the primary +// or residual respectively. +// +// This affords both a more efficient representation over the FnAPI +// (if the bundle is large) and often a more efficient representation +// on the runner side (e.g. if the set of elements can be represented +// as some range in an underlying dataset). +// +// Note that for a split the following properties must hold: +// - last_primary_element < first_residual_element +// - primary roots and residual roots can only be specified if the +// last_primary_element + 1 < first_residual_element +// (typically there is one primary and residual root per element in the +// range (last_primary_element, first_residual_element)) +// - primary roots and residual roots must represent a disjoint but full +// coverage of work represented by the elements between last_primary_element +// and first_residual_element +// +// Note that subsequent splits of the same bundle must ensure that: +// - the first_residual_element does not increase +// - the first_residual_element does not decrease if there were residual +// or primary roots returned in a prior split. +type ProcessBundleSplitResponse_ChannelSplit struct { + state protoimpl.MessageState `protogen:"opaque.v1"` + xxx_hidden_TransformId string `protobuf:"bytes,1,opt,name=transform_id,json=transformId,proto3" json:"transform_id,omitempty"` + xxx_hidden_LastPrimaryElement int64 `protobuf:"varint,2,opt,name=last_primary_element,json=lastPrimaryElement,proto3" json:"last_primary_element,omitempty"` + xxx_hidden_FirstResidualElement int64 `protobuf:"varint,3,opt,name=first_residual_element,json=firstResidualElement,proto3" json:"first_residual_element,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *ProcessBundleSplitResponse_ChannelSplit) Reset() { + *x = ProcessBundleSplitResponse_ChannelSplit{} + mi := &file_org_apache_beam_model_fn_execution_v1_beam_fn_api_proto_msgTypes[62] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *ProcessBundleSplitResponse_ChannelSplit) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*ProcessBundleSplitResponse_ChannelSplit) ProtoMessage() {} + +func (x *ProcessBundleSplitResponse_ChannelSplit) ProtoReflect() protoreflect.Message { + mi := &file_org_apache_beam_model_fn_execution_v1_beam_fn_api_proto_msgTypes[62] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +func (x *ProcessBundleSplitResponse_ChannelSplit) GetTransformId() string { + if x != nil { + return x.xxx_hidden_TransformId + } + return "" +} + +func (x *ProcessBundleSplitResponse_ChannelSplit) GetLastPrimaryElement() int64 { + if x != nil { + return x.xxx_hidden_LastPrimaryElement + } + return 0 +} + +func (x *ProcessBundleSplitResponse_ChannelSplit) GetFirstResidualElement() int64 { + if x != nil { + return x.xxx_hidden_FirstResidualElement + } + return 0 +} + +func (x *ProcessBundleSplitResponse_ChannelSplit) SetTransformId(v string) { + x.xxx_hidden_TransformId = v +} + +func (x *ProcessBundleSplitResponse_ChannelSplit) SetLastPrimaryElement(v int64) { + x.xxx_hidden_LastPrimaryElement = v +} + +func (x *ProcessBundleSplitResponse_ChannelSplit) SetFirstResidualElement(v int64) { + x.xxx_hidden_FirstResidualElement = v +} + +type ProcessBundleSplitResponse_ChannelSplit_builder struct { + _ [0]func() // Prevents comparability and use of unkeyed literals for the builder. + + // (Required) The grpc read transform reading this channel. + TransformId string + // (Required) The last element of the input channel that should be entirely + // considered part of the primary, identified by its absolute zero-based + // index in the (ordered) channel. + LastPrimaryElement int64 + // (Required) The first element of the input channel that should be entirely + // considered part of the residual, identified by its absolute zero-based + // index in the (ordered) channel. + FirstResidualElement int64 +} + +func (b0 ProcessBundleSplitResponse_ChannelSplit_builder) Build() *ProcessBundleSplitResponse_ChannelSplit { + m0 := &ProcessBundleSplitResponse_ChannelSplit{} + b, x := &b0, m0 + _, _ = b, x + x.xxx_hidden_TransformId = b.TransformId + x.xxx_hidden_LastPrimaryElement = b.LastPrimaryElement + x.xxx_hidden_FirstResidualElement = b.FirstResidualElement + return m0 +} + +// Represents multiple encoded elements in nested context for a given named +// instruction and transform. +type Elements_Data struct { + state protoimpl.MessageState `protogen:"opaque.v1"` + xxx_hidden_InstructionId string `protobuf:"bytes,1,opt,name=instruction_id,json=instructionId,proto3" json:"instruction_id,omitempty"` + xxx_hidden_TransformId string `protobuf:"bytes,2,opt,name=transform_id,json=transformId,proto3" json:"transform_id,omitempty"` + xxx_hidden_Data []byte `protobuf:"bytes,3,opt,name=data,proto3" json:"data,omitempty"` + xxx_hidden_IsLast bool `protobuf:"varint,4,opt,name=is_last,json=isLast,proto3" json:"is_last,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *Elements_Data) Reset() { + *x = Elements_Data{} + mi := &file_org_apache_beam_model_fn_execution_v1_beam_fn_api_proto_msgTypes[63] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *Elements_Data) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*Elements_Data) ProtoMessage() {} + +func (x *Elements_Data) ProtoReflect() protoreflect.Message { + mi := &file_org_apache_beam_model_fn_execution_v1_beam_fn_api_proto_msgTypes[63] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +func (x *Elements_Data) GetInstructionId() string { + if x != nil { + return x.xxx_hidden_InstructionId + } + return "" +} + +func (x *Elements_Data) GetTransformId() string { + if x != nil { + return x.xxx_hidden_TransformId + } + return "" +} + +func (x *Elements_Data) GetData() []byte { + if x != nil { + return x.xxx_hidden_Data + } + return nil +} + +func (x *Elements_Data) GetIsLast() bool { + if x != nil { + return x.xxx_hidden_IsLast + } + return false +} + +func (x *Elements_Data) SetInstructionId(v string) { + x.xxx_hidden_InstructionId = v +} + +func (x *Elements_Data) SetTransformId(v string) { + x.xxx_hidden_TransformId = v +} + +func (x *Elements_Data) SetData(v []byte) { + if v == nil { + v = []byte{} + } + x.xxx_hidden_Data = v +} + +func (x *Elements_Data) SetIsLast(v bool) { + x.xxx_hidden_IsLast = v +} + +type Elements_Data_builder struct { + _ [0]func() // Prevents comparability and use of unkeyed literals for the builder. + + // (Required) A reference to an active instruction request with the given + // instruction id. + InstructionId string + // (Required) A definition representing a consumer or producer of this data. + // If received by a harness, this represents the consumer within that + // harness that should consume these bytes. If sent by a harness, this + // represents the producer of these bytes. + // + // Note that a single element may span multiple Data messages. + // + // Note that a sending/receiving pair should share the same identifier. + TransformId string + // (Optional) Represents a part of a logical byte stream. Elements within + // the logical byte stream are encoded in the nested context and + // concatenated together. + Data []byte + // (Optional) Set this bit to indicate the this is the last data block + // for the given instruction and transform, ending the stream. + IsLast bool +} + +func (b0 Elements_Data_builder) Build() *Elements_Data { + m0 := &Elements_Data{} + b, x := &b0, m0 + _, _ = b, x + x.xxx_hidden_InstructionId = b.InstructionId + x.xxx_hidden_TransformId = b.TransformId + x.xxx_hidden_Data = b.Data + x.xxx_hidden_IsLast = b.IsLast + return m0 +} + +// Represent the encoded user timer for a given instruction, transform and +// timer id. +type Elements_Timers struct { + state protoimpl.MessageState `protogen:"opaque.v1"` + xxx_hidden_InstructionId string `protobuf:"bytes,1,opt,name=instruction_id,json=instructionId,proto3" json:"instruction_id,omitempty"` + xxx_hidden_TransformId string `protobuf:"bytes,2,opt,name=transform_id,json=transformId,proto3" json:"transform_id,omitempty"` + xxx_hidden_TimerFamilyId string `protobuf:"bytes,3,opt,name=timer_family_id,json=timerFamilyId,proto3" json:"timer_family_id,omitempty"` + xxx_hidden_Timers []byte `protobuf:"bytes,4,opt,name=timers,proto3" json:"timers,omitempty"` + xxx_hidden_IsLast bool `protobuf:"varint,5,opt,name=is_last,json=isLast,proto3" json:"is_last,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *Elements_Timers) Reset() { + *x = Elements_Timers{} + mi := &file_org_apache_beam_model_fn_execution_v1_beam_fn_api_proto_msgTypes[64] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *Elements_Timers) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*Elements_Timers) ProtoMessage() {} + +func (x *Elements_Timers) ProtoReflect() protoreflect.Message { + mi := &file_org_apache_beam_model_fn_execution_v1_beam_fn_api_proto_msgTypes[64] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +func (x *Elements_Timers) GetInstructionId() string { + if x != nil { + return x.xxx_hidden_InstructionId + } + return "" +} + +func (x *Elements_Timers) GetTransformId() string { + if x != nil { + return x.xxx_hidden_TransformId + } + return "" +} + +func (x *Elements_Timers) GetTimerFamilyId() string { + if x != nil { + return x.xxx_hidden_TimerFamilyId + } + return "" +} + +func (x *Elements_Timers) GetTimers() []byte { + if x != nil { + return x.xxx_hidden_Timers + } + return nil +} + +func (x *Elements_Timers) GetIsLast() bool { + if x != nil { + return x.xxx_hidden_IsLast + } + return false +} + +func (x *Elements_Timers) SetInstructionId(v string) { + x.xxx_hidden_InstructionId = v +} + +func (x *Elements_Timers) SetTransformId(v string) { + x.xxx_hidden_TransformId = v +} + +func (x *Elements_Timers) SetTimerFamilyId(v string) { + x.xxx_hidden_TimerFamilyId = v +} + +func (x *Elements_Timers) SetTimers(v []byte) { + if v == nil { + v = []byte{} + } + x.xxx_hidden_Timers = v +} + +func (x *Elements_Timers) SetIsLast(v bool) { + x.xxx_hidden_IsLast = v +} + +type Elements_Timers_builder struct { + _ [0]func() // Prevents comparability and use of unkeyed literals for the builder. + + // (Required) A reference to an active instruction request with the given + // instruction id. + InstructionId string + // (Required) A definition representing a consumer or producer of this data. + // If received by a harness, this represents the consumer within that + // harness that should consume these timers. If sent by a harness, this + // represents the producer of these timers. + TransformId string + // (Required) The local timer family name used to identify the associated + // timer family specification + TimerFamilyId string + // (Optional) Represents a logical byte stream of timers. Encoded according + // to the coder in the timer spec. + Timers []byte + // (Optional) Set this bit to indicate the this is the last data block + // for the given instruction and transform, ending the stream. + IsLast bool +} + +func (b0 Elements_Timers_builder) Build() *Elements_Timers { + m0 := &Elements_Timers{} + b, x := &b0, m0 + _, _ = b, x + x.xxx_hidden_InstructionId = b.InstructionId + x.xxx_hidden_TransformId = b.TransformId + x.xxx_hidden_TimerFamilyId = b.TimerFamilyId + x.xxx_hidden_Timers = b.Timers + x.xxx_hidden_IsLast = b.IsLast + return m0 +} + +type StateKey_Runner struct { + state protoimpl.MessageState `protogen:"opaque.v1"` + xxx_hidden_Key []byte `protobuf:"bytes,1,opt,name=key,proto3" json:"key,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *StateKey_Runner) Reset() { + *x = StateKey_Runner{} + mi := &file_org_apache_beam_model_fn_execution_v1_beam_fn_api_proto_msgTypes[65] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *StateKey_Runner) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*StateKey_Runner) ProtoMessage() {} + +func (x *StateKey_Runner) ProtoReflect() protoreflect.Message { + mi := &file_org_apache_beam_model_fn_execution_v1_beam_fn_api_proto_msgTypes[65] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +func (x *StateKey_Runner) GetKey() []byte { + if x != nil { + return x.xxx_hidden_Key + } + return nil +} + +func (x *StateKey_Runner) SetKey(v []byte) { + if v == nil { + v = []byte{} + } + x.xxx_hidden_Key = v +} + +type StateKey_Runner_builder struct { + _ [0]func() // Prevents comparability and use of unkeyed literals for the builder. + + // (Required) Opaque information supplied by the runner. Used to support + // remote references. + // https://s.apache.org/beam-fn-api-send-and-receive-data + // + // Used by state backed iterable. And in this use case, request type can + // only be of type get. Details see: + // https://s.apache.org/beam-fn-api-state-backed-iterables + Key []byte +} + +func (b0 StateKey_Runner_builder) Build() *StateKey_Runner { + m0 := &StateKey_Runner{} + b, x := &b0, m0 + _, _ = b, x + x.xxx_hidden_Key = b.Key + return m0 +} + +// Represents a request for the values associated with a specified window +// in a PCollection. See +// https://s.apache.org/beam-fn-state-api-and-bundle-processing for further +// details. +// +// Can only be used to perform StateGetRequests on side inputs of the URN +// beam:side_input:iterable:v1. +// +// For a PCollection, the response data stream will be a concatenation +// of all V's. See https://s.apache.org/beam-fn-api-send-and-receive-data +// for further details. +type StateKey_IterableSideInput struct { + state protoimpl.MessageState `protogen:"opaque.v1"` + xxx_hidden_TransformId string `protobuf:"bytes,1,opt,name=transform_id,json=transformId,proto3" json:"transform_id,omitempty"` + xxx_hidden_SideInputId string `protobuf:"bytes,2,opt,name=side_input_id,json=sideInputId,proto3" json:"side_input_id,omitempty"` + xxx_hidden_Window []byte `protobuf:"bytes,3,opt,name=window,proto3" json:"window,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *StateKey_IterableSideInput) Reset() { + *x = StateKey_IterableSideInput{} + mi := &file_org_apache_beam_model_fn_execution_v1_beam_fn_api_proto_msgTypes[66] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *StateKey_IterableSideInput) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*StateKey_IterableSideInput) ProtoMessage() {} + +func (x *StateKey_IterableSideInput) ProtoReflect() protoreflect.Message { + mi := &file_org_apache_beam_model_fn_execution_v1_beam_fn_api_proto_msgTypes[66] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +func (x *StateKey_IterableSideInput) GetTransformId() string { + if x != nil { + return x.xxx_hidden_TransformId + } + return "" +} + +func (x *StateKey_IterableSideInput) GetSideInputId() string { + if x != nil { + return x.xxx_hidden_SideInputId + } + return "" +} + +func (x *StateKey_IterableSideInput) GetWindow() []byte { + if x != nil { + return x.xxx_hidden_Window + } + return nil +} + +func (x *StateKey_IterableSideInput) SetTransformId(v string) { + x.xxx_hidden_TransformId = v +} + +func (x *StateKey_IterableSideInput) SetSideInputId(v string) { + x.xxx_hidden_SideInputId = v +} + +func (x *StateKey_IterableSideInput) SetWindow(v []byte) { + if v == nil { + v = []byte{} + } + x.xxx_hidden_Window = v +} + +type StateKey_IterableSideInput_builder struct { + _ [0]func() // Prevents comparability and use of unkeyed literals for the builder. + + // (Required) The id of the PTransform containing a side input. + TransformId string + // (Required) The id of the side input. + SideInputId string + // (Required) The window (after mapping the currently executing elements + // window into the side input windows domain) encoded in a nested context. + Window []byte +} + +func (b0 StateKey_IterableSideInput_builder) Build() *StateKey_IterableSideInput { + m0 := &StateKey_IterableSideInput{} + b, x := &b0, m0 + _, _ = b, x + x.xxx_hidden_TransformId = b.TransformId + x.xxx_hidden_SideInputId = b.SideInputId + x.xxx_hidden_Window = b.Window + return m0 +} + +// Represents a request for the values associated with a specified user key +// and window in a PCollection. See +// https://s.apache.org/beam-fn-state-api-and-bundle-processing for further +// details. +// +// Can only be used to perform StateGetRequests on side inputs of the URN +// beam:side_input:multimap:v1. +// +// For a PCollection>, the response data stream will be a +// concatenation of all V's associated with the specified key K. See +// https://s.apache.org/beam-fn-api-send-and-receive-data for further +// details. +type StateKey_MultimapSideInput struct { + state protoimpl.MessageState `protogen:"opaque.v1"` + xxx_hidden_TransformId string `protobuf:"bytes,1,opt,name=transform_id,json=transformId,proto3" json:"transform_id,omitempty"` + xxx_hidden_SideInputId string `protobuf:"bytes,2,opt,name=side_input_id,json=sideInputId,proto3" json:"side_input_id,omitempty"` + xxx_hidden_Window []byte `protobuf:"bytes,3,opt,name=window,proto3" json:"window,omitempty"` + xxx_hidden_Key []byte `protobuf:"bytes,4,opt,name=key,proto3" json:"key,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *StateKey_MultimapSideInput) Reset() { + *x = StateKey_MultimapSideInput{} + mi := &file_org_apache_beam_model_fn_execution_v1_beam_fn_api_proto_msgTypes[67] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *StateKey_MultimapSideInput) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*StateKey_MultimapSideInput) ProtoMessage() {} + +func (x *StateKey_MultimapSideInput) ProtoReflect() protoreflect.Message { + mi := &file_org_apache_beam_model_fn_execution_v1_beam_fn_api_proto_msgTypes[67] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +func (x *StateKey_MultimapSideInput) GetTransformId() string { + if x != nil { + return x.xxx_hidden_TransformId + } + return "" +} + +func (x *StateKey_MultimapSideInput) GetSideInputId() string { + if x != nil { + return x.xxx_hidden_SideInputId + } + return "" +} + +func (x *StateKey_MultimapSideInput) GetWindow() []byte { + if x != nil { + return x.xxx_hidden_Window + } + return nil +} + +func (x *StateKey_MultimapSideInput) GetKey() []byte { + if x != nil { + return x.xxx_hidden_Key + } + return nil +} + +func (x *StateKey_MultimapSideInput) SetTransformId(v string) { + x.xxx_hidden_TransformId = v +} + +func (x *StateKey_MultimapSideInput) SetSideInputId(v string) { + x.xxx_hidden_SideInputId = v +} + +func (x *StateKey_MultimapSideInput) SetWindow(v []byte) { + if v == nil { + v = []byte{} + } + x.xxx_hidden_Window = v +} + +func (x *StateKey_MultimapSideInput) SetKey(v []byte) { + if v == nil { + v = []byte{} + } + x.xxx_hidden_Key = v +} + +type StateKey_MultimapSideInput_builder struct { + _ [0]func() // Prevents comparability and use of unkeyed literals for the builder. + + // (Required) The id of the PTransform containing a side input. + TransformId string + // (Required) The id of the side input. + SideInputId string + // (Required) The window (after mapping the currently executing elements + // window into the side input windows domain) encoded in a nested context. + Window []byte + // (Required) The key encoded in a nested context. + Key []byte +} + +func (b0 StateKey_MultimapSideInput_builder) Build() *StateKey_MultimapSideInput { + m0 := &StateKey_MultimapSideInput{} + b, x := &b0, m0 + _, _ = b, x + x.xxx_hidden_TransformId = b.TransformId + x.xxx_hidden_SideInputId = b.SideInputId + x.xxx_hidden_Window = b.Window + x.xxx_hidden_Key = b.Key + return m0 +} + +// Represents a request for the keys associated with a specified window in a PCollection. See +// https://s.apache.org/beam-fn-state-api-and-bundle-processing for further +// details. +// +// Can only be used to perform StateGetRequests on side inputs of the URN +// beam:side_input:multimap:v1. +// +// For a PCollection>, the response data stream will be a +// concatenation of all K's associated with the specified window. See +// https://s.apache.org/beam-fn-api-send-and-receive-data for further +// details. +type StateKey_MultimapKeysSideInput struct { + state protoimpl.MessageState `protogen:"opaque.v1"` + xxx_hidden_TransformId string `protobuf:"bytes,1,opt,name=transform_id,json=transformId,proto3" json:"transform_id,omitempty"` + xxx_hidden_SideInputId string `protobuf:"bytes,2,opt,name=side_input_id,json=sideInputId,proto3" json:"side_input_id,omitempty"` + xxx_hidden_Window []byte `protobuf:"bytes,3,opt,name=window,proto3" json:"window,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *StateKey_MultimapKeysSideInput) Reset() { + *x = StateKey_MultimapKeysSideInput{} + mi := &file_org_apache_beam_model_fn_execution_v1_beam_fn_api_proto_msgTypes[68] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *StateKey_MultimapKeysSideInput) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*StateKey_MultimapKeysSideInput) ProtoMessage() {} + +func (x *StateKey_MultimapKeysSideInput) ProtoReflect() protoreflect.Message { + mi := &file_org_apache_beam_model_fn_execution_v1_beam_fn_api_proto_msgTypes[68] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +func (x *StateKey_MultimapKeysSideInput) GetTransformId() string { + if x != nil { + return x.xxx_hidden_TransformId + } + return "" +} + +func (x *StateKey_MultimapKeysSideInput) GetSideInputId() string { + if x != nil { + return x.xxx_hidden_SideInputId + } + return "" +} + +func (x *StateKey_MultimapKeysSideInput) GetWindow() []byte { + if x != nil { + return x.xxx_hidden_Window + } + return nil +} + +func (x *StateKey_MultimapKeysSideInput) SetTransformId(v string) { + x.xxx_hidden_TransformId = v +} + +func (x *StateKey_MultimapKeysSideInput) SetSideInputId(v string) { + x.xxx_hidden_SideInputId = v +} + +func (x *StateKey_MultimapKeysSideInput) SetWindow(v []byte) { + if v == nil { + v = []byte{} + } + x.xxx_hidden_Window = v +} + +type StateKey_MultimapKeysSideInput_builder struct { + _ [0]func() // Prevents comparability and use of unkeyed literals for the builder. + + // (Required) The id of the PTransform containing a side input. + TransformId string + // (Required) The id of the side input. + SideInputId string + // (Required) The window (after mapping the currently executing elements + // window into the side input windows domain) encoded in a nested context. + Window []byte +} + +func (b0 StateKey_MultimapKeysSideInput_builder) Build() *StateKey_MultimapKeysSideInput { + m0 := &StateKey_MultimapKeysSideInput{} + b, x := &b0, m0 + _, _ = b, x + x.xxx_hidden_TransformId = b.TransformId + x.xxx_hidden_SideInputId = b.SideInputId + x.xxx_hidden_Window = b.Window + return m0 +} + +// Represents a request for the keys and values associated with a specified window in a PCollection. See +// https://s.apache.org/beam-fn-state-api-and-bundle-processing for further +// details. +// +// This is expected to be more efficient than iterating over they keys and +// looking up the values one at a time. If a runner chooses not to implement +// this protocol, or a key has too many values to fit into a single response, +// the runner is free to fail the request and a fallback to point lookups +// will be performed by the SDK. +// +// Can only be used to perform StateGetRequests on side inputs of the URN +// beam:side_input:multimap:v1. +// +// For a PCollection>, the response data stream will be a +// concatenation of all KVs associated with the specified window, +// encoded with the the KV> coder. +// See +// https://s.apache.org/beam-fn-api-send-and-receive-data for further +// details. +type StateKey_MultimapKeysValuesSideInput struct { + state protoimpl.MessageState `protogen:"opaque.v1"` + xxx_hidden_TransformId string `protobuf:"bytes,1,opt,name=transform_id,json=transformId,proto3" json:"transform_id,omitempty"` + xxx_hidden_SideInputId string `protobuf:"bytes,2,opt,name=side_input_id,json=sideInputId,proto3" json:"side_input_id,omitempty"` + xxx_hidden_Window []byte `protobuf:"bytes,3,opt,name=window,proto3" json:"window,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *StateKey_MultimapKeysValuesSideInput) Reset() { + *x = StateKey_MultimapKeysValuesSideInput{} + mi := &file_org_apache_beam_model_fn_execution_v1_beam_fn_api_proto_msgTypes[69] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *StateKey_MultimapKeysValuesSideInput) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*StateKey_MultimapKeysValuesSideInput) ProtoMessage() {} + +func (x *StateKey_MultimapKeysValuesSideInput) ProtoReflect() protoreflect.Message { + mi := &file_org_apache_beam_model_fn_execution_v1_beam_fn_api_proto_msgTypes[69] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +func (x *StateKey_MultimapKeysValuesSideInput) GetTransformId() string { + if x != nil { + return x.xxx_hidden_TransformId + } + return "" +} + +func (x *StateKey_MultimapKeysValuesSideInput) GetSideInputId() string { + if x != nil { + return x.xxx_hidden_SideInputId + } + return "" +} + +func (x *StateKey_MultimapKeysValuesSideInput) GetWindow() []byte { + if x != nil { + return x.xxx_hidden_Window + } + return nil +} + +func (x *StateKey_MultimapKeysValuesSideInput) SetTransformId(v string) { + x.xxx_hidden_TransformId = v +} + +func (x *StateKey_MultimapKeysValuesSideInput) SetSideInputId(v string) { + x.xxx_hidden_SideInputId = v +} + +func (x *StateKey_MultimapKeysValuesSideInput) SetWindow(v []byte) { + if v == nil { + v = []byte{} + } + x.xxx_hidden_Window = v +} + +type StateKey_MultimapKeysValuesSideInput_builder struct { + _ [0]func() // Prevents comparability and use of unkeyed literals for the builder. + + // (Required) The id of the PTransform containing a side input. + TransformId string + // (Required) The id of the side input. + SideInputId string + // (Required) The window (after mapping the currently executing elements + // window into the side input windows domain) encoded in a nested context. + Window []byte +} + +func (b0 StateKey_MultimapKeysValuesSideInput_builder) Build() *StateKey_MultimapKeysValuesSideInput { + m0 := &StateKey_MultimapKeysValuesSideInput{} + b, x := &b0, m0 + _, _ = b, x + x.xxx_hidden_TransformId = b.TransformId + x.xxx_hidden_SideInputId = b.SideInputId + x.xxx_hidden_Window = b.Window + return m0 +} + +// Represents a request for an unordered set of values associated with a +// specified user key and window for a PTransform. See +// https://s.apache.org/beam-fn-state-api-and-bundle-processing for further +// details. +// +// The response data stream will be a concatenation of all V's associated +// with the specified user key and window. +// See https://s.apache.org/beam-fn-api-send-and-receive-data for further +// details. +type StateKey_BagUserState struct { + state protoimpl.MessageState `protogen:"opaque.v1"` + xxx_hidden_TransformId string `protobuf:"bytes,1,opt,name=transform_id,json=transformId,proto3" json:"transform_id,omitempty"` + xxx_hidden_UserStateId string `protobuf:"bytes,2,opt,name=user_state_id,json=userStateId,proto3" json:"user_state_id,omitempty"` + xxx_hidden_Window []byte `protobuf:"bytes,3,opt,name=window,proto3" json:"window,omitempty"` + xxx_hidden_Key []byte `protobuf:"bytes,4,opt,name=key,proto3" json:"key,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *StateKey_BagUserState) Reset() { + *x = StateKey_BagUserState{} + mi := &file_org_apache_beam_model_fn_execution_v1_beam_fn_api_proto_msgTypes[70] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *StateKey_BagUserState) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*StateKey_BagUserState) ProtoMessage() {} + +func (x *StateKey_BagUserState) ProtoReflect() protoreflect.Message { + mi := &file_org_apache_beam_model_fn_execution_v1_beam_fn_api_proto_msgTypes[70] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +func (x *StateKey_BagUserState) GetTransformId() string { + if x != nil { + return x.xxx_hidden_TransformId + } + return "" +} + +func (x *StateKey_BagUserState) GetUserStateId() string { + if x != nil { + return x.xxx_hidden_UserStateId + } + return "" +} + +func (x *StateKey_BagUserState) GetWindow() []byte { + if x != nil { + return x.xxx_hidden_Window + } + return nil +} + +func (x *StateKey_BagUserState) GetKey() []byte { + if x != nil { + return x.xxx_hidden_Key + } + return nil +} + +func (x *StateKey_BagUserState) SetTransformId(v string) { + x.xxx_hidden_TransformId = v +} + +func (x *StateKey_BagUserState) SetUserStateId(v string) { + x.xxx_hidden_UserStateId = v +} + +func (x *StateKey_BagUserState) SetWindow(v []byte) { + if v == nil { + v = []byte{} + } + x.xxx_hidden_Window = v +} + +func (x *StateKey_BagUserState) SetKey(v []byte) { + if v == nil { + v = []byte{} + } + x.xxx_hidden_Key = v +} + +type StateKey_BagUserState_builder struct { + _ [0]func() // Prevents comparability and use of unkeyed literals for the builder. + + // (Required) The id of the PTransform containing user state. + TransformId string + // (Required) The id of the user state. + UserStateId string + // (Required) The window encoded in a nested context. + Window []byte + // (Required) The key of the currently executing element encoded in a + // nested context. + Key []byte +} + +func (b0 StateKey_BagUserState_builder) Build() *StateKey_BagUserState { + m0 := &StateKey_BagUserState{} + b, x := &b0, m0 + _, _ = b, x + x.xxx_hidden_TransformId = b.TransformId + x.xxx_hidden_UserStateId = b.UserStateId + x.xxx_hidden_Window = b.Window + x.xxx_hidden_Key = b.Key + return m0 +} + +// Represents a request for the keys of a multimap associated with a specified +// user key and window for a PTransform. See +// https://s.apache.org/beam-fn-state-api-and-bundle-processing for further +// details. +// +// Can only be used to perform StateGetRequests and StateClearRequests on the +// user state. +// +// The response data stream will be a concatenation of all K's associated +// with the specified user key and window. +// See https://s.apache.org/beam-fn-api-send-and-receive-data for further +// details. +type StateKey_MultimapKeysUserState struct { + state protoimpl.MessageState `protogen:"opaque.v1"` + xxx_hidden_TransformId string `protobuf:"bytes,1,opt,name=transform_id,json=transformId,proto3" json:"transform_id,omitempty"` + xxx_hidden_UserStateId string `protobuf:"bytes,2,opt,name=user_state_id,json=userStateId,proto3" json:"user_state_id,omitempty"` + xxx_hidden_Window []byte `protobuf:"bytes,3,opt,name=window,proto3" json:"window,omitempty"` + xxx_hidden_Key []byte `protobuf:"bytes,4,opt,name=key,proto3" json:"key,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *StateKey_MultimapKeysUserState) Reset() { + *x = StateKey_MultimapKeysUserState{} + mi := &file_org_apache_beam_model_fn_execution_v1_beam_fn_api_proto_msgTypes[71] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *StateKey_MultimapKeysUserState) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*StateKey_MultimapKeysUserState) ProtoMessage() {} + +func (x *StateKey_MultimapKeysUserState) ProtoReflect() protoreflect.Message { + mi := &file_org_apache_beam_model_fn_execution_v1_beam_fn_api_proto_msgTypes[71] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +func (x *StateKey_MultimapKeysUserState) GetTransformId() string { + if x != nil { + return x.xxx_hidden_TransformId + } + return "" +} + +func (x *StateKey_MultimapKeysUserState) GetUserStateId() string { + if x != nil { + return x.xxx_hidden_UserStateId + } + return "" +} + +func (x *StateKey_MultimapKeysUserState) GetWindow() []byte { + if x != nil { + return x.xxx_hidden_Window + } + return nil +} + +func (x *StateKey_MultimapKeysUserState) GetKey() []byte { + if x != nil { + return x.xxx_hidden_Key + } + return nil +} + +func (x *StateKey_MultimapKeysUserState) SetTransformId(v string) { + x.xxx_hidden_TransformId = v +} + +func (x *StateKey_MultimapKeysUserState) SetUserStateId(v string) { + x.xxx_hidden_UserStateId = v +} + +func (x *StateKey_MultimapKeysUserState) SetWindow(v []byte) { + if v == nil { + v = []byte{} + } + x.xxx_hidden_Window = v +} + +func (x *StateKey_MultimapKeysUserState) SetKey(v []byte) { + if v == nil { + v = []byte{} + } + x.xxx_hidden_Key = v +} + +type StateKey_MultimapKeysUserState_builder struct { + _ [0]func() // Prevents comparability and use of unkeyed literals for the builder. + + // (Required) The id of the PTransform containing user state. + TransformId string + // (Required) The id of the user state. + UserStateId string + // (Required) The window encoded in a nested context. + Window []byte + // (Required) The key of the currently executing element encoded in a + // nested context. + Key []byte +} + +func (b0 StateKey_MultimapKeysUserState_builder) Build() *StateKey_MultimapKeysUserState { + m0 := &StateKey_MultimapKeysUserState{} + b, x := &b0, m0 + _, _ = b, x + x.xxx_hidden_TransformId = b.TransformId + x.xxx_hidden_UserStateId = b.UserStateId + x.xxx_hidden_Window = b.Window + x.xxx_hidden_Key = b.Key + return m0 +} + +// Represents a request for the values of the map key associated with a +// specified user key and window for a PTransform. See +// https://s.apache.org/beam-fn-state-api-and-bundle-processing for further +// details. +// +// The response data stream will be a concatenation of all V's associated +// with the specified map key, user key, and window. +// See https://s.apache.org/beam-fn-api-send-and-receive-data for further +// details. +type StateKey_MultimapUserState struct { + state protoimpl.MessageState `protogen:"opaque.v1"` + xxx_hidden_TransformId string `protobuf:"bytes,1,opt,name=transform_id,json=transformId,proto3" json:"transform_id,omitempty"` + xxx_hidden_UserStateId string `protobuf:"bytes,2,opt,name=user_state_id,json=userStateId,proto3" json:"user_state_id,omitempty"` + xxx_hidden_Window []byte `protobuf:"bytes,3,opt,name=window,proto3" json:"window,omitempty"` + xxx_hidden_Key []byte `protobuf:"bytes,4,opt,name=key,proto3" json:"key,omitempty"` + xxx_hidden_MapKey []byte `protobuf:"bytes,5,opt,name=map_key,json=mapKey,proto3" json:"map_key,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *StateKey_MultimapUserState) Reset() { + *x = StateKey_MultimapUserState{} + mi := &file_org_apache_beam_model_fn_execution_v1_beam_fn_api_proto_msgTypes[72] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *StateKey_MultimapUserState) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*StateKey_MultimapUserState) ProtoMessage() {} + +func (x *StateKey_MultimapUserState) ProtoReflect() protoreflect.Message { + mi := &file_org_apache_beam_model_fn_execution_v1_beam_fn_api_proto_msgTypes[72] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +func (x *StateKey_MultimapUserState) GetTransformId() string { + if x != nil { + return x.xxx_hidden_TransformId + } + return "" +} + +func (x *StateKey_MultimapUserState) GetUserStateId() string { + if x != nil { + return x.xxx_hidden_UserStateId + } + return "" +} + +func (x *StateKey_MultimapUserState) GetWindow() []byte { + if x != nil { + return x.xxx_hidden_Window + } + return nil +} + +func (x *StateKey_MultimapUserState) GetKey() []byte { + if x != nil { + return x.xxx_hidden_Key + } + return nil +} + +func (x *StateKey_MultimapUserState) GetMapKey() []byte { + if x != nil { + return x.xxx_hidden_MapKey + } + return nil +} + +func (x *StateKey_MultimapUserState) SetTransformId(v string) { + x.xxx_hidden_TransformId = v +} + +func (x *StateKey_MultimapUserState) SetUserStateId(v string) { + x.xxx_hidden_UserStateId = v +} + +func (x *StateKey_MultimapUserState) SetWindow(v []byte) { + if v == nil { + v = []byte{} + } + x.xxx_hidden_Window = v +} + +func (x *StateKey_MultimapUserState) SetKey(v []byte) { + if v == nil { + v = []byte{} + } + x.xxx_hidden_Key = v +} + +func (x *StateKey_MultimapUserState) SetMapKey(v []byte) { + if v == nil { + v = []byte{} + } + x.xxx_hidden_MapKey = v +} + +type StateKey_MultimapUserState_builder struct { + _ [0]func() // Prevents comparability and use of unkeyed literals for the builder. + + // (Required) The id of the PTransform containing user state. + TransformId string + // (Required) The id of the user state. + UserStateId string + // (Required) The window encoded in a nested context. + Window []byte + // (Required) The key of the currently executing element encoded in a + // nested context. + Key []byte + // (Required) The map key encoded in a nested context. + MapKey []byte +} + +func (b0 StateKey_MultimapUserState_builder) Build() *StateKey_MultimapUserState { + m0 := &StateKey_MultimapUserState{} + b, x := &b0, m0 + _, _ = b, x + x.xxx_hidden_TransformId = b.TransformId + x.xxx_hidden_UserStateId = b.UserStateId + x.xxx_hidden_Window = b.Window + x.xxx_hidden_Key = b.Key + x.xxx_hidden_MapKey = b.MapKey + return m0 +} + +// Represents a request for an ordered list of values associated with a +// specified user key and window for a PTransform. See +// https://s.apache.org/beam-fn-state-api-and-bundle-processing for further +// details. +// +// The response data stream will be a concatenation of all entries of sort key +// and V's associated with the specified user key and window. +// See https://s.apache.org/beam-fn-api-send-and-receive-data for further +// details. +type StateKey_OrderedListUserState struct { + state protoimpl.MessageState `protogen:"opaque.v1"` + xxx_hidden_TransformId string `protobuf:"bytes,1,opt,name=transform_id,json=transformId,proto3" json:"transform_id,omitempty"` + xxx_hidden_UserStateId string `protobuf:"bytes,2,opt,name=user_state_id,json=userStateId,proto3" json:"user_state_id,omitempty"` + xxx_hidden_Window []byte `protobuf:"bytes,3,opt,name=window,proto3" json:"window,omitempty"` + xxx_hidden_Key []byte `protobuf:"bytes,4,opt,name=key,proto3" json:"key,omitempty"` + xxx_hidden_Range *OrderedListRange `protobuf:"bytes,5,opt,name=range,proto3" json:"range,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *StateKey_OrderedListUserState) Reset() { + *x = StateKey_OrderedListUserState{} + mi := &file_org_apache_beam_model_fn_execution_v1_beam_fn_api_proto_msgTypes[73] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *StateKey_OrderedListUserState) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*StateKey_OrderedListUserState) ProtoMessage() {} + +func (x *StateKey_OrderedListUserState) ProtoReflect() protoreflect.Message { + mi := &file_org_apache_beam_model_fn_execution_v1_beam_fn_api_proto_msgTypes[73] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +func (x *StateKey_OrderedListUserState) GetTransformId() string { + if x != nil { + return x.xxx_hidden_TransformId + } + return "" +} + +func (x *StateKey_OrderedListUserState) GetUserStateId() string { + if x != nil { + return x.xxx_hidden_UserStateId + } + return "" +} + +func (x *StateKey_OrderedListUserState) GetWindow() []byte { + if x != nil { + return x.xxx_hidden_Window + } + return nil +} + +func (x *StateKey_OrderedListUserState) GetKey() []byte { + if x != nil { + return x.xxx_hidden_Key + } + return nil +} + +func (x *StateKey_OrderedListUserState) GetRange() *OrderedListRange { + if x != nil { + return x.xxx_hidden_Range + } + return nil +} + +func (x *StateKey_OrderedListUserState) SetTransformId(v string) { + x.xxx_hidden_TransformId = v +} + +func (x *StateKey_OrderedListUserState) SetUserStateId(v string) { + x.xxx_hidden_UserStateId = v +} + +func (x *StateKey_OrderedListUserState) SetWindow(v []byte) { + if v == nil { + v = []byte{} + } + x.xxx_hidden_Window = v +} + +func (x *StateKey_OrderedListUserState) SetKey(v []byte) { + if v == nil { + v = []byte{} + } + x.xxx_hidden_Key = v +} + +func (x *StateKey_OrderedListUserState) SetRange(v *OrderedListRange) { + x.xxx_hidden_Range = v +} + +func (x *StateKey_OrderedListUserState) HasRange() bool { + if x == nil { + return false + } + return x.xxx_hidden_Range != nil +} + +func (x *StateKey_OrderedListUserState) ClearRange() { + x.xxx_hidden_Range = nil +} + +type StateKey_OrderedListUserState_builder struct { + _ [0]func() // Prevents comparability and use of unkeyed literals for the builder. + + // (Required) The id of the PTransform containing user state. + TransformId string + // (Required) The id of the user state. + UserStateId string + // (Required) The window encoded in a nested context. + Window []byte + // (Required) The key of the currently executing element encoded in a + // nested context. + Key []byte + // (Required) The sort range encoded in a nested context. + Range *OrderedListRange +} + +func (b0 StateKey_OrderedListUserState_builder) Build() *StateKey_OrderedListUserState { + m0 := &StateKey_OrderedListUserState{} + b, x := &b0, m0 + _, _ = b, x + x.xxx_hidden_TransformId = b.TransformId + x.xxx_hidden_UserStateId = b.UserStateId + x.xxx_hidden_Window = b.Window + x.xxx_hidden_Key = b.Key + x.xxx_hidden_Range = b.Range + return m0 +} + +// A list of log entries, enables buffering and batching of multiple +// log messages using the logging API. +type LogEntry_List struct { + state protoimpl.MessageState `protogen:"opaque.v1"` + xxx_hidden_LogEntries *[]*LogEntry `protobuf:"bytes,1,rep,name=log_entries,json=logEntries,proto3" json:"log_entries,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *LogEntry_List) Reset() { + *x = LogEntry_List{} + mi := &file_org_apache_beam_model_fn_execution_v1_beam_fn_api_proto_msgTypes[74] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *LogEntry_List) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*LogEntry_List) ProtoMessage() {} + +func (x *LogEntry_List) ProtoReflect() protoreflect.Message { + mi := &file_org_apache_beam_model_fn_execution_v1_beam_fn_api_proto_msgTypes[74] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +func (x *LogEntry_List) GetLogEntries() []*LogEntry { + if x != nil { + if x.xxx_hidden_LogEntries != nil { + return *x.xxx_hidden_LogEntries + } + } + return nil +} + +func (x *LogEntry_List) SetLogEntries(v []*LogEntry) { + x.xxx_hidden_LogEntries = &v +} + +type LogEntry_List_builder struct { + _ [0]func() // Prevents comparability and use of unkeyed literals for the builder. + + // (Required) One or or more log messages. + LogEntries []*LogEntry +} + +func (b0 LogEntry_List_builder) Build() *LogEntry_List { + m0 := &LogEntry_List{} + b, x := &b0, m0 + _, _ = b, x + x.xxx_hidden_LogEntries = &b.LogEntries + return m0 +} + +// The severity of the event described in a log entry, expressed as one of the +// severity levels listed below. For your reference, the levels are +// assigned the listed numeric values. The effect of using numeric values +// other than those listed is undefined. +// +// If you are writing log entries, you should map other severity encodings to +// one of these standard levels. For example, you might map all of +// Java's FINE, FINER, and FINEST levels to `Severity.DEBUG`. +// +// This list is intentionally not comprehensive; the intent is to provide a +// common set of "good enough" severity levels so that logging front ends +// can provide filtering and searching across log types. Users of the API are +// free not to use all severity levels in their log messages. +type LogEntry_Severity struct { + state protoimpl.MessageState `protogen:"opaque.v1"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *LogEntry_Severity) Reset() { + *x = LogEntry_Severity{} + mi := &file_org_apache_beam_model_fn_execution_v1_beam_fn_api_proto_msgTypes[75] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *LogEntry_Severity) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*LogEntry_Severity) ProtoMessage() {} + +func (x *LogEntry_Severity) ProtoReflect() protoreflect.Message { + mi := &file_org_apache_beam_model_fn_execution_v1_beam_fn_api_proto_msgTypes[75] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +type LogEntry_Severity_builder struct { + _ [0]func() // Prevents comparability and use of unkeyed literals for the builder. + +} + +func (b0 LogEntry_Severity_builder) Build() *LogEntry_Severity { + m0 := &LogEntry_Severity{} + b, x := &b0, m0 + _, _ = b, x + return m0 +} + +var File_org_apache_beam_model_fn_execution_v1_beam_fn_api_proto protoreflect.FileDescriptor + +var file_org_apache_beam_model_fn_execution_v1_beam_fn_api_proto_rawDesc = []byte{ + 0x0a, 0x37, 0x6f, 0x72, 0x67, 0x2f, 0x61, 0x70, 0x61, 0x63, 0x68, 0x65, 0x2f, 0x62, 0x65, 0x61, + 0x6d, 0x2f, 0x6d, 0x6f, 0x64, 0x65, 0x6c, 0x2f, 0x66, 0x6e, 0x5f, 0x65, 0x78, 0x65, 0x63, 0x75, + 0x74, 0x69, 0x6f, 0x6e, 0x2f, 0x76, 0x31, 0x2f, 0x62, 0x65, 0x61, 0x6d, 0x5f, 0x66, 0x6e, 0x5f, + 0x61, 0x70, 0x69, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x12, 0x25, 0x6f, 0x72, 0x67, 0x2e, 0x61, + 0x70, 0x61, 0x63, 0x68, 0x65, 0x2e, 0x62, 0x65, 0x61, 0x6d, 0x2e, 0x6d, 0x6f, 0x64, 0x65, 0x6c, + 0x2e, 0x66, 0x6e, 0x5f, 0x65, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x2e, 0x76, 0x31, + 0x1a, 0x37, 0x6f, 0x72, 0x67, 0x2f, 0x61, 0x70, 0x61, 0x63, 0x68, 0x65, 0x2f, 0x62, 0x65, 0x61, + 0x6d, 0x2f, 0x6d, 0x6f, 0x64, 0x65, 0x6c, 0x2f, 0x70, 0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e, 0x65, + 0x2f, 0x76, 0x31, 0x2f, 0x62, 0x65, 0x61, 0x6d, 0x5f, 0x72, 0x75, 0x6e, 0x6e, 0x65, 0x72, 0x5f, + 0x61, 0x70, 0x69, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x1a, 0x31, 0x6f, 0x72, 0x67, 0x2f, 0x61, + 0x70, 0x61, 0x63, 0x68, 0x65, 0x2f, 0x62, 0x65, 0x61, 0x6d, 0x2f, 0x6d, 0x6f, 0x64, 0x65, 0x6c, + 0x2f, 0x70, 0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e, 0x65, 0x2f, 0x76, 0x31, 0x2f, 0x65, 0x6e, 0x64, + 0x70, 0x6f, 0x69, 0x6e, 0x74, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x1a, 0x2f, 0x6f, 0x72, + 0x67, 0x2f, 0x61, 0x70, 0x61, 0x63, 0x68, 0x65, 0x2f, 0x62, 0x65, 0x61, 0x6d, 0x2f, 0x6d, 0x6f, + 0x64, 0x65, 0x6c, 0x2f, 0x70, 0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e, 0x65, 0x2f, 0x76, 0x31, 0x2f, + 0x6d, 0x65, 0x74, 0x72, 0x69, 0x63, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x1a, 0x20, 0x67, + 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2f, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66, 0x2f, 0x64, + 0x65, 0x73, 0x63, 0x72, 0x69, 0x70, 0x74, 0x6f, 0x72, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x1a, + 0x1c, 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2f, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66, + 0x2f, 0x73, 0x74, 0x72, 0x75, 0x63, 0x74, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x1a, 0x1f, 0x67, + 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2f, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66, 0x2f, 0x74, + 0x69, 0x6d, 0x65, 0x73, 0x74, 0x61, 0x6d, 0x70, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x1a, 0x1e, + 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2f, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66, 0x2f, + 0x64, 0x75, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x22, 0x73, + 0x0a, 0x0f, 0x46, 0x6e, 0x41, 0x70, 0x69, 0x54, 0x72, 0x61, 0x6e, 0x73, 0x66, 0x6f, 0x72, 0x6d, + 0x73, 0x22, 0x60, 0x0a, 0x06, 0x52, 0x75, 0x6e, 0x6e, 0x65, 0x72, 0x12, 0x2c, 0x0a, 0x0b, 0x44, + 0x41, 0x54, 0x41, 0x5f, 0x53, 0x4f, 0x55, 0x52, 0x43, 0x45, 0x10, 0x00, 0x1a, 0x1b, 0xa2, 0xb4, + 0xfa, 0xc2, 0x05, 0x15, 0x62, 0x65, 0x61, 0x6d, 0x3a, 0x72, 0x75, 0x6e, 0x6e, 0x65, 0x72, 0x3a, + 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x3a, 0x76, 0x31, 0x12, 0x28, 0x0a, 0x09, 0x44, 0x41, 0x54, + 0x41, 0x5f, 0x53, 0x49, 0x4e, 0x4b, 0x10, 0x01, 0x1a, 0x19, 0xa2, 0xb4, 0xfa, 0xc2, 0x05, 0x13, + 0x62, 0x65, 0x61, 0x6d, 0x3a, 0x72, 0x75, 0x6e, 0x6e, 0x65, 0x72, 0x3a, 0x73, 0x69, 0x6e, 0x6b, + 0x3a, 0x76, 0x31, 0x22, 0x9a, 0x01, 0x0a, 0x0e, 0x52, 0x65, 0x6d, 0x6f, 0x74, 0x65, 0x47, 0x72, + 0x70, 0x63, 0x50, 0x6f, 0x72, 0x74, 0x12, 0x6d, 0x0a, 0x16, 0x61, 0x70, 0x69, 0x5f, 0x73, 0x65, + 0x72, 0x76, 0x69, 0x63, 0x65, 0x5f, 0x64, 0x65, 0x73, 0x63, 0x72, 0x69, 0x70, 0x74, 0x6f, 0x72, + 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x37, 0x2e, 0x6f, 0x72, 0x67, 0x2e, 0x61, 0x70, 0x61, + 0x63, 0x68, 0x65, 0x2e, 0x62, 0x65, 0x61, 0x6d, 0x2e, 0x6d, 0x6f, 0x64, 0x65, 0x6c, 0x2e, 0x70, + 0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x41, 0x70, 0x69, 0x53, 0x65, + 0x72, 0x76, 0x69, 0x63, 0x65, 0x44, 0x65, 0x73, 0x63, 0x72, 0x69, 0x70, 0x74, 0x6f, 0x72, 0x52, + 0x14, 0x61, 0x70, 0x69, 0x53, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, 0x44, 0x65, 0x73, 0x63, 0x72, + 0x69, 0x70, 0x74, 0x6f, 0x72, 0x12, 0x19, 0x0a, 0x08, 0x63, 0x6f, 0x64, 0x65, 0x72, 0x5f, 0x69, + 0x64, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x07, 0x63, 0x6f, 0x64, 0x65, 0x72, 0x49, 0x64, + 0x22, 0x64, 0x0a, 0x21, 0x47, 0x65, 0x74, 0x50, 0x72, 0x6f, 0x63, 0x65, 0x73, 0x73, 0x42, 0x75, + 0x6e, 0x64, 0x6c, 0x65, 0x44, 0x65, 0x73, 0x63, 0x72, 0x69, 0x70, 0x74, 0x6f, 0x72, 0x52, 0x65, + 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x3f, 0x0a, 0x1c, 0x70, 0x72, 0x6f, 0x63, 0x65, 0x73, 0x73, + 0x5f, 0x62, 0x75, 0x6e, 0x64, 0x6c, 0x65, 0x5f, 0x64, 0x65, 0x73, 0x63, 0x72, 0x69, 0x70, 0x74, + 0x6f, 0x72, 0x5f, 0x69, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x19, 0x70, 0x72, 0x6f, + 0x63, 0x65, 0x73, 0x73, 0x42, 0x75, 0x6e, 0x64, 0x6c, 0x65, 0x44, 0x65, 0x73, 0x63, 0x72, 0x69, + 0x70, 0x74, 0x6f, 0x72, 0x49, 0x64, 0x22, 0xbc, 0x07, 0x0a, 0x12, 0x49, 0x6e, 0x73, 0x74, 0x72, + 0x75, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x25, 0x0a, + 0x0e, 0x69, 0x6e, 0x73, 0x74, 0x72, 0x75, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x69, 0x64, 0x18, + 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0d, 0x69, 0x6e, 0x73, 0x74, 0x72, 0x75, 0x63, 0x74, 0x69, + 0x6f, 0x6e, 0x49, 0x64, 0x12, 0x65, 0x0a, 0x0e, 0x70, 0x72, 0x6f, 0x63, 0x65, 0x73, 0x73, 0x5f, + 0x62, 0x75, 0x6e, 0x64, 0x6c, 0x65, 0x18, 0xe9, 0x07, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x3b, 0x2e, + 0x6f, 0x72, 0x67, 0x2e, 0x61, 0x70, 0x61, 0x63, 0x68, 0x65, 0x2e, 0x62, 0x65, 0x61, 0x6d, 0x2e, + 0x6d, 0x6f, 0x64, 0x65, 0x6c, 0x2e, 0x66, 0x6e, 0x5f, 0x65, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, + 0x6f, 0x6e, 0x2e, 0x76, 0x31, 0x2e, 0x50, 0x72, 0x6f, 0x63, 0x65, 0x73, 0x73, 0x42, 0x75, 0x6e, + 0x64, 0x6c, 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x48, 0x00, 0x52, 0x0d, 0x70, 0x72, + 0x6f, 0x63, 0x65, 0x73, 0x73, 0x42, 0x75, 0x6e, 0x64, 0x6c, 0x65, 0x12, 0x7e, 0x0a, 0x17, 0x70, + 0x72, 0x6f, 0x63, 0x65, 0x73, 0x73, 0x5f, 0x62, 0x75, 0x6e, 0x64, 0x6c, 0x65, 0x5f, 0x70, 0x72, + 0x6f, 0x67, 0x72, 0x65, 0x73, 0x73, 0x18, 0xea, 0x07, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x43, 0x2e, + 0x6f, 0x72, 0x67, 0x2e, 0x61, 0x70, 0x61, 0x63, 0x68, 0x65, 0x2e, 0x62, 0x65, 0x61, 0x6d, 0x2e, + 0x6d, 0x6f, 0x64, 0x65, 0x6c, 0x2e, 0x66, 0x6e, 0x5f, 0x65, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, + 0x6f, 0x6e, 0x2e, 0x76, 0x31, 0x2e, 0x50, 0x72, 0x6f, 0x63, 0x65, 0x73, 0x73, 0x42, 0x75, 0x6e, + 0x64, 0x6c, 0x65, 0x50, 0x72, 0x6f, 0x67, 0x72, 0x65, 0x73, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, + 0x73, 0x74, 0x48, 0x00, 0x52, 0x15, 0x70, 0x72, 0x6f, 0x63, 0x65, 0x73, 0x73, 0x42, 0x75, 0x6e, + 0x64, 0x6c, 0x65, 0x50, 0x72, 0x6f, 0x67, 0x72, 0x65, 0x73, 0x73, 0x12, 0x75, 0x0a, 0x14, 0x70, + 0x72, 0x6f, 0x63, 0x65, 0x73, 0x73, 0x5f, 0x62, 0x75, 0x6e, 0x64, 0x6c, 0x65, 0x5f, 0x73, 0x70, + 0x6c, 0x69, 0x74, 0x18, 0xeb, 0x07, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x40, 0x2e, 0x6f, 0x72, 0x67, + 0x2e, 0x61, 0x70, 0x61, 0x63, 0x68, 0x65, 0x2e, 0x62, 0x65, 0x61, 0x6d, 0x2e, 0x6d, 0x6f, 0x64, + 0x65, 0x6c, 0x2e, 0x66, 0x6e, 0x5f, 0x65, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x2e, + 0x76, 0x31, 0x2e, 0x50, 0x72, 0x6f, 0x63, 0x65, 0x73, 0x73, 0x42, 0x75, 0x6e, 0x64, 0x6c, 0x65, + 0x53, 0x70, 0x6c, 0x69, 0x74, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x48, 0x00, 0x52, 0x12, + 0x70, 0x72, 0x6f, 0x63, 0x65, 0x73, 0x73, 0x42, 0x75, 0x6e, 0x64, 0x6c, 0x65, 0x53, 0x70, 0x6c, + 0x69, 0x74, 0x12, 0x68, 0x0a, 0x0f, 0x66, 0x69, 0x6e, 0x61, 0x6c, 0x69, 0x7a, 0x65, 0x5f, 0x62, + 0x75, 0x6e, 0x64, 0x6c, 0x65, 0x18, 0xec, 0x07, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x3c, 0x2e, 0x6f, + 0x72, 0x67, 0x2e, 0x61, 0x70, 0x61, 0x63, 0x68, 0x65, 0x2e, 0x62, 0x65, 0x61, 0x6d, 0x2e, 0x6d, + 0x6f, 0x64, 0x65, 0x6c, 0x2e, 0x66, 0x6e, 0x5f, 0x65, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, + 0x6e, 0x2e, 0x76, 0x31, 0x2e, 0x46, 0x69, 0x6e, 0x61, 0x6c, 0x69, 0x7a, 0x65, 0x42, 0x75, 0x6e, + 0x64, 0x6c, 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x48, 0x00, 0x52, 0x0e, 0x66, 0x69, + 0x6e, 0x61, 0x6c, 0x69, 0x7a, 0x65, 0x42, 0x75, 0x6e, 0x64, 0x6c, 0x65, 0x12, 0x73, 0x0a, 0x10, + 0x6d, 0x6f, 0x6e, 0x69, 0x74, 0x6f, 0x72, 0x69, 0x6e, 0x67, 0x5f, 0x69, 0x6e, 0x66, 0x6f, 0x73, + 0x18, 0xed, 0x07, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x45, 0x2e, 0x6f, 0x72, 0x67, 0x2e, 0x61, 0x70, + 0x61, 0x63, 0x68, 0x65, 0x2e, 0x62, 0x65, 0x61, 0x6d, 0x2e, 0x6d, 0x6f, 0x64, 0x65, 0x6c, 0x2e, + 0x66, 0x6e, 0x5f, 0x65, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x2e, 0x76, 0x31, 0x2e, + 0x4d, 0x6f, 0x6e, 0x69, 0x74, 0x6f, 0x72, 0x69, 0x6e, 0x67, 0x49, 0x6e, 0x66, 0x6f, 0x73, 0x4d, + 0x65, 0x74, 0x61, 0x64, 0x61, 0x74, 0x61, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x48, 0x00, + 0x52, 0x0f, 0x6d, 0x6f, 0x6e, 0x69, 0x74, 0x6f, 0x72, 0x69, 0x6e, 0x67, 0x49, 0x6e, 0x66, 0x6f, + 0x73, 0x12, 0x81, 0x01, 0x0a, 0x18, 0x68, 0x61, 0x72, 0x6e, 0x65, 0x73, 0x73, 0x5f, 0x6d, 0x6f, + 0x6e, 0x69, 0x74, 0x6f, 0x72, 0x69, 0x6e, 0x67, 0x5f, 0x69, 0x6e, 0x66, 0x6f, 0x73, 0x18, 0xee, + 0x07, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x44, 0x2e, 0x6f, 0x72, 0x67, 0x2e, 0x61, 0x70, 0x61, 0x63, + 0x68, 0x65, 0x2e, 0x62, 0x65, 0x61, 0x6d, 0x2e, 0x6d, 0x6f, 0x64, 0x65, 0x6c, 0x2e, 0x66, 0x6e, + 0x5f, 0x65, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x2e, 0x76, 0x31, 0x2e, 0x48, 0x61, + 0x72, 0x6e, 0x65, 0x73, 0x73, 0x4d, 0x6f, 0x6e, 0x69, 0x74, 0x6f, 0x72, 0x69, 0x6e, 0x67, 0x49, + 0x6e, 0x66, 0x6f, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x48, 0x00, 0x52, 0x16, 0x68, + 0x61, 0x72, 0x6e, 0x65, 0x73, 0x73, 0x4d, 0x6f, 0x6e, 0x69, 0x74, 0x6f, 0x72, 0x69, 0x6e, 0x67, + 0x49, 0x6e, 0x66, 0x6f, 0x73, 0x12, 0x5c, 0x0a, 0x0b, 0x73, 0x61, 0x6d, 0x70, 0x6c, 0x65, 0x5f, + 0x64, 0x61, 0x74, 0x61, 0x18, 0xef, 0x07, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x38, 0x2e, 0x6f, 0x72, + 0x67, 0x2e, 0x61, 0x70, 0x61, 0x63, 0x68, 0x65, 0x2e, 0x62, 0x65, 0x61, 0x6d, 0x2e, 0x6d, 0x6f, + 0x64, 0x65, 0x6c, 0x2e, 0x66, 0x6e, 0x5f, 0x65, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, + 0x2e, 0x76, 0x31, 0x2e, 0x53, 0x61, 0x6d, 0x70, 0x6c, 0x65, 0x44, 0x61, 0x74, 0x61, 0x52, 0x65, + 0x71, 0x75, 0x65, 0x73, 0x74, 0x48, 0x00, 0x52, 0x0a, 0x73, 0x61, 0x6d, 0x70, 0x6c, 0x65, 0x44, + 0x61, 0x74, 0x61, 0x12, 0x55, 0x0a, 0x08, 0x72, 0x65, 0x67, 0x69, 0x73, 0x74, 0x65, 0x72, 0x18, + 0xe8, 0x07, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x36, 0x2e, 0x6f, 0x72, 0x67, 0x2e, 0x61, 0x70, 0x61, + 0x63, 0x68, 0x65, 0x2e, 0x62, 0x65, 0x61, 0x6d, 0x2e, 0x6d, 0x6f, 0x64, 0x65, 0x6c, 0x2e, 0x66, + 0x6e, 0x5f, 0x65, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x2e, 0x76, 0x31, 0x2e, 0x52, + 0x65, 0x67, 0x69, 0x73, 0x74, 0x65, 0x72, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x48, 0x00, + 0x52, 0x08, 0x72, 0x65, 0x67, 0x69, 0x73, 0x74, 0x65, 0x72, 0x42, 0x09, 0x0a, 0x07, 0x72, 0x65, + 0x71, 0x75, 0x65, 0x73, 0x74, 0x22, 0xdc, 0x07, 0x0a, 0x13, 0x49, 0x6e, 0x73, 0x74, 0x72, 0x75, + 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x25, 0x0a, + 0x0e, 0x69, 0x6e, 0x73, 0x74, 0x72, 0x75, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x69, 0x64, 0x18, + 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0d, 0x69, 0x6e, 0x73, 0x74, 0x72, 0x75, 0x63, 0x74, 0x69, + 0x6f, 0x6e, 0x49, 0x64, 0x12, 0x14, 0x0a, 0x05, 0x65, 0x72, 0x72, 0x6f, 0x72, 0x18, 0x02, 0x20, + 0x01, 0x28, 0x09, 0x52, 0x05, 0x65, 0x72, 0x72, 0x6f, 0x72, 0x12, 0x66, 0x0a, 0x0e, 0x70, 0x72, + 0x6f, 0x63, 0x65, 0x73, 0x73, 0x5f, 0x62, 0x75, 0x6e, 0x64, 0x6c, 0x65, 0x18, 0xe9, 0x07, 0x20, + 0x01, 0x28, 0x0b, 0x32, 0x3c, 0x2e, 0x6f, 0x72, 0x67, 0x2e, 0x61, 0x70, 0x61, 0x63, 0x68, 0x65, + 0x2e, 0x62, 0x65, 0x61, 0x6d, 0x2e, 0x6d, 0x6f, 0x64, 0x65, 0x6c, 0x2e, 0x66, 0x6e, 0x5f, 0x65, + 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x2e, 0x76, 0x31, 0x2e, 0x50, 0x72, 0x6f, 0x63, + 0x65, 0x73, 0x73, 0x42, 0x75, 0x6e, 0x64, 0x6c, 0x65, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, + 0x65, 0x48, 0x00, 0x52, 0x0d, 0x70, 0x72, 0x6f, 0x63, 0x65, 0x73, 0x73, 0x42, 0x75, 0x6e, 0x64, + 0x6c, 0x65, 0x12, 0x7f, 0x0a, 0x17, 0x70, 0x72, 0x6f, 0x63, 0x65, 0x73, 0x73, 0x5f, 0x62, 0x75, + 0x6e, 0x64, 0x6c, 0x65, 0x5f, 0x70, 0x72, 0x6f, 0x67, 0x72, 0x65, 0x73, 0x73, 0x18, 0xea, 0x07, + 0x20, 0x01, 0x28, 0x0b, 0x32, 0x44, 0x2e, 0x6f, 0x72, 0x67, 0x2e, 0x61, 0x70, 0x61, 0x63, 0x68, + 0x65, 0x2e, 0x62, 0x65, 0x61, 0x6d, 0x2e, 0x6d, 0x6f, 0x64, 0x65, 0x6c, 0x2e, 0x66, 0x6e, 0x5f, + 0x65, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x2e, 0x76, 0x31, 0x2e, 0x50, 0x72, 0x6f, + 0x63, 0x65, 0x73, 0x73, 0x42, 0x75, 0x6e, 0x64, 0x6c, 0x65, 0x50, 0x72, 0x6f, 0x67, 0x72, 0x65, + 0x73, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x48, 0x00, 0x52, 0x15, 0x70, 0x72, + 0x6f, 0x63, 0x65, 0x73, 0x73, 0x42, 0x75, 0x6e, 0x64, 0x6c, 0x65, 0x50, 0x72, 0x6f, 0x67, 0x72, + 0x65, 0x73, 0x73, 0x12, 0x76, 0x0a, 0x14, 0x70, 0x72, 0x6f, 0x63, 0x65, 0x73, 0x73, 0x5f, 0x62, + 0x75, 0x6e, 0x64, 0x6c, 0x65, 0x5f, 0x73, 0x70, 0x6c, 0x69, 0x74, 0x18, 0xeb, 0x07, 0x20, 0x01, + 0x28, 0x0b, 0x32, 0x41, 0x2e, 0x6f, 0x72, 0x67, 0x2e, 0x61, 0x70, 0x61, 0x63, 0x68, 0x65, 0x2e, + 0x62, 0x65, 0x61, 0x6d, 0x2e, 0x6d, 0x6f, 0x64, 0x65, 0x6c, 0x2e, 0x66, 0x6e, 0x5f, 0x65, 0x78, + 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x2e, 0x76, 0x31, 0x2e, 0x50, 0x72, 0x6f, 0x63, 0x65, + 0x73, 0x73, 0x42, 0x75, 0x6e, 0x64, 0x6c, 0x65, 0x53, 0x70, 0x6c, 0x69, 0x74, 0x52, 0x65, 0x73, + 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x48, 0x00, 0x52, 0x12, 0x70, 0x72, 0x6f, 0x63, 0x65, 0x73, 0x73, + 0x42, 0x75, 0x6e, 0x64, 0x6c, 0x65, 0x53, 0x70, 0x6c, 0x69, 0x74, 0x12, 0x69, 0x0a, 0x0f, 0x66, + 0x69, 0x6e, 0x61, 0x6c, 0x69, 0x7a, 0x65, 0x5f, 0x62, 0x75, 0x6e, 0x64, 0x6c, 0x65, 0x18, 0xec, + 0x07, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x3d, 0x2e, 0x6f, 0x72, 0x67, 0x2e, 0x61, 0x70, 0x61, 0x63, + 0x68, 0x65, 0x2e, 0x62, 0x65, 0x61, 0x6d, 0x2e, 0x6d, 0x6f, 0x64, 0x65, 0x6c, 0x2e, 0x66, 0x6e, + 0x5f, 0x65, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x2e, 0x76, 0x31, 0x2e, 0x46, 0x69, + 0x6e, 0x61, 0x6c, 0x69, 0x7a, 0x65, 0x42, 0x75, 0x6e, 0x64, 0x6c, 0x65, 0x52, 0x65, 0x73, 0x70, + 0x6f, 0x6e, 0x73, 0x65, 0x48, 0x00, 0x52, 0x0e, 0x66, 0x69, 0x6e, 0x61, 0x6c, 0x69, 0x7a, 0x65, + 0x42, 0x75, 0x6e, 0x64, 0x6c, 0x65, 0x12, 0x74, 0x0a, 0x10, 0x6d, 0x6f, 0x6e, 0x69, 0x74, 0x6f, + 0x72, 0x69, 0x6e, 0x67, 0x5f, 0x69, 0x6e, 0x66, 0x6f, 0x73, 0x18, 0xed, 0x07, 0x20, 0x01, 0x28, + 0x0b, 0x32, 0x46, 0x2e, 0x6f, 0x72, 0x67, 0x2e, 0x61, 0x70, 0x61, 0x63, 0x68, 0x65, 0x2e, 0x62, + 0x65, 0x61, 0x6d, 0x2e, 0x6d, 0x6f, 0x64, 0x65, 0x6c, 0x2e, 0x66, 0x6e, 0x5f, 0x65, 0x78, 0x65, + 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x2e, 0x76, 0x31, 0x2e, 0x4d, 0x6f, 0x6e, 0x69, 0x74, 0x6f, + 0x72, 0x69, 0x6e, 0x67, 0x49, 0x6e, 0x66, 0x6f, 0x73, 0x4d, 0x65, 0x74, 0x61, 0x64, 0x61, 0x74, + 0x61, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x48, 0x00, 0x52, 0x0f, 0x6d, 0x6f, 0x6e, + 0x69, 0x74, 0x6f, 0x72, 0x69, 0x6e, 0x67, 0x49, 0x6e, 0x66, 0x6f, 0x73, 0x12, 0x82, 0x01, 0x0a, + 0x18, 0x68, 0x61, 0x72, 0x6e, 0x65, 0x73, 0x73, 0x5f, 0x6d, 0x6f, 0x6e, 0x69, 0x74, 0x6f, 0x72, + 0x69, 0x6e, 0x67, 0x5f, 0x69, 0x6e, 0x66, 0x6f, 0x73, 0x18, 0xee, 0x07, 0x20, 0x01, 0x28, 0x0b, + 0x32, 0x45, 0x2e, 0x6f, 0x72, 0x67, 0x2e, 0x61, 0x70, 0x61, 0x63, 0x68, 0x65, 0x2e, 0x62, 0x65, + 0x61, 0x6d, 0x2e, 0x6d, 0x6f, 0x64, 0x65, 0x6c, 0x2e, 0x66, 0x6e, 0x5f, 0x65, 0x78, 0x65, 0x63, + 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x2e, 0x76, 0x31, 0x2e, 0x48, 0x61, 0x72, 0x6e, 0x65, 0x73, 0x73, + 0x4d, 0x6f, 0x6e, 0x69, 0x74, 0x6f, 0x72, 0x69, 0x6e, 0x67, 0x49, 0x6e, 0x66, 0x6f, 0x73, 0x52, + 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x48, 0x00, 0x52, 0x16, 0x68, 0x61, 0x72, 0x6e, 0x65, + 0x73, 0x73, 0x4d, 0x6f, 0x6e, 0x69, 0x74, 0x6f, 0x72, 0x69, 0x6e, 0x67, 0x49, 0x6e, 0x66, 0x6f, + 0x73, 0x12, 0x5d, 0x0a, 0x0b, 0x73, 0x61, 0x6d, 0x70, 0x6c, 0x65, 0x5f, 0x64, 0x61, 0x74, 0x61, + 0x18, 0xef, 0x07, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x39, 0x2e, 0x6f, 0x72, 0x67, 0x2e, 0x61, 0x70, + 0x61, 0x63, 0x68, 0x65, 0x2e, 0x62, 0x65, 0x61, 0x6d, 0x2e, 0x6d, 0x6f, 0x64, 0x65, 0x6c, 0x2e, + 0x66, 0x6e, 0x5f, 0x65, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x2e, 0x76, 0x31, 0x2e, + 0x53, 0x61, 0x6d, 0x70, 0x6c, 0x65, 0x44, 0x61, 0x74, 0x61, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, + 0x73, 0x65, 0x48, 0x00, 0x52, 0x0a, 0x73, 0x61, 0x6d, 0x70, 0x6c, 0x65, 0x44, 0x61, 0x74, 0x61, + 0x12, 0x56, 0x0a, 0x08, 0x72, 0x65, 0x67, 0x69, 0x73, 0x74, 0x65, 0x72, 0x18, 0xe8, 0x07, 0x20, + 0x01, 0x28, 0x0b, 0x32, 0x37, 0x2e, 0x6f, 0x72, 0x67, 0x2e, 0x61, 0x70, 0x61, 0x63, 0x68, 0x65, + 0x2e, 0x62, 0x65, 0x61, 0x6d, 0x2e, 0x6d, 0x6f, 0x64, 0x65, 0x6c, 0x2e, 0x66, 0x6e, 0x5f, 0x65, + 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x2e, 0x76, 0x31, 0x2e, 0x52, 0x65, 0x67, 0x69, + 0x73, 0x74, 0x65, 0x72, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x48, 0x00, 0x52, 0x08, + 0x72, 0x65, 0x67, 0x69, 0x73, 0x74, 0x65, 0x72, 0x42, 0x0a, 0x0a, 0x08, 0x72, 0x65, 0x73, 0x70, + 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x3c, 0x0a, 0x11, 0x53, 0x61, 0x6d, 0x70, 0x6c, 0x65, 0x44, 0x61, + 0x74, 0x61, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x27, 0x0a, 0x0f, 0x70, 0x63, 0x6f, + 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x69, 0x64, 0x73, 0x18, 0x01, 0x20, 0x03, + 0x28, 0x09, 0x52, 0x0e, 0x70, 0x63, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x49, + 0x64, 0x73, 0x22, 0xbd, 0x02, 0x0a, 0x0e, 0x53, 0x61, 0x6d, 0x70, 0x6c, 0x65, 0x64, 0x45, 0x6c, + 0x65, 0x6d, 0x65, 0x6e, 0x74, 0x12, 0x18, 0x0a, 0x07, 0x65, 0x6c, 0x65, 0x6d, 0x65, 0x6e, 0x74, + 0x18, 0x01, 0x20, 0x01, 0x28, 0x0c, 0x52, 0x07, 0x65, 0x6c, 0x65, 0x6d, 0x65, 0x6e, 0x74, 0x12, + 0x45, 0x0a, 0x10, 0x73, 0x61, 0x6d, 0x70, 0x6c, 0x65, 0x5f, 0x74, 0x69, 0x6d, 0x65, 0x73, 0x74, + 0x61, 0x6d, 0x70, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1a, 0x2e, 0x67, 0x6f, 0x6f, 0x67, + 0x6c, 0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66, 0x2e, 0x54, 0x69, 0x6d, 0x65, + 0x73, 0x74, 0x61, 0x6d, 0x70, 0x52, 0x0f, 0x73, 0x61, 0x6d, 0x70, 0x6c, 0x65, 0x54, 0x69, 0x6d, + 0x65, 0x73, 0x74, 0x61, 0x6d, 0x70, 0x12, 0x5d, 0x0a, 0x09, 0x65, 0x78, 0x63, 0x65, 0x70, 0x74, + 0x69, 0x6f, 0x6e, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x3f, 0x2e, 0x6f, 0x72, 0x67, 0x2e, + 0x61, 0x70, 0x61, 0x63, 0x68, 0x65, 0x2e, 0x62, 0x65, 0x61, 0x6d, 0x2e, 0x6d, 0x6f, 0x64, 0x65, + 0x6c, 0x2e, 0x66, 0x6e, 0x5f, 0x65, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x2e, 0x76, + 0x31, 0x2e, 0x53, 0x61, 0x6d, 0x70, 0x6c, 0x65, 0x64, 0x45, 0x6c, 0x65, 0x6d, 0x65, 0x6e, 0x74, + 0x2e, 0x45, 0x78, 0x63, 0x65, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x09, 0x65, 0x78, 0x63, 0x65, + 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x1a, 0x6b, 0x0a, 0x09, 0x45, 0x78, 0x63, 0x65, 0x70, 0x74, 0x69, + 0x6f, 0x6e, 0x12, 0x25, 0x0a, 0x0e, 0x69, 0x6e, 0x73, 0x74, 0x72, 0x75, 0x63, 0x74, 0x69, 0x6f, + 0x6e, 0x5f, 0x69, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0d, 0x69, 0x6e, 0x73, 0x74, + 0x72, 0x75, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x64, 0x12, 0x21, 0x0a, 0x0c, 0x74, 0x72, 0x61, + 0x6e, 0x73, 0x66, 0x6f, 0x72, 0x6d, 0x5f, 0x69, 0x64, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, + 0x0b, 0x74, 0x72, 0x61, 0x6e, 0x73, 0x66, 0x6f, 0x72, 0x6d, 0x49, 0x64, 0x12, 0x14, 0x0a, 0x05, + 0x65, 0x72, 0x72, 0x6f, 0x72, 0x18, 0x03, 0x20, 0x01, 0x28, 0x09, 0x52, 0x05, 0x65, 0x72, 0x72, + 0x6f, 0x72, 0x22, 0xf9, 0x02, 0x0a, 0x12, 0x53, 0x61, 0x6d, 0x70, 0x6c, 0x65, 0x44, 0x61, 0x74, + 0x61, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x76, 0x0a, 0x0f, 0x65, 0x6c, 0x65, + 0x6d, 0x65, 0x6e, 0x74, 0x5f, 0x73, 0x61, 0x6d, 0x70, 0x6c, 0x65, 0x73, 0x18, 0x01, 0x20, 0x03, + 0x28, 0x0b, 0x32, 0x4d, 0x2e, 0x6f, 0x72, 0x67, 0x2e, 0x61, 0x70, 0x61, 0x63, 0x68, 0x65, 0x2e, + 0x62, 0x65, 0x61, 0x6d, 0x2e, 0x6d, 0x6f, 0x64, 0x65, 0x6c, 0x2e, 0x66, 0x6e, 0x5f, 0x65, 0x78, + 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x2e, 0x76, 0x31, 0x2e, 0x53, 0x61, 0x6d, 0x70, 0x6c, + 0x65, 0x44, 0x61, 0x74, 0x61, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x2e, 0x45, 0x6c, + 0x65, 0x6d, 0x65, 0x6e, 0x74, 0x53, 0x61, 0x6d, 0x70, 0x6c, 0x65, 0x73, 0x45, 0x6e, 0x74, 0x72, + 0x79, 0x52, 0x0e, 0x65, 0x6c, 0x65, 0x6d, 0x65, 0x6e, 0x74, 0x53, 0x61, 0x6d, 0x70, 0x6c, 0x65, + 0x73, 0x1a, 0x60, 0x0a, 0x0b, 0x45, 0x6c, 0x65, 0x6d, 0x65, 0x6e, 0x74, 0x4c, 0x69, 0x73, 0x74, + 0x12, 0x51, 0x0a, 0x08, 0x65, 0x6c, 0x65, 0x6d, 0x65, 0x6e, 0x74, 0x73, 0x18, 0x01, 0x20, 0x03, + 0x28, 0x0b, 0x32, 0x35, 0x2e, 0x6f, 0x72, 0x67, 0x2e, 0x61, 0x70, 0x61, 0x63, 0x68, 0x65, 0x2e, + 0x62, 0x65, 0x61, 0x6d, 0x2e, 0x6d, 0x6f, 0x64, 0x65, 0x6c, 0x2e, 0x66, 0x6e, 0x5f, 0x65, 0x78, + 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x2e, 0x76, 0x31, 0x2e, 0x53, 0x61, 0x6d, 0x70, 0x6c, + 0x65, 0x64, 0x45, 0x6c, 0x65, 0x6d, 0x65, 0x6e, 0x74, 0x52, 0x08, 0x65, 0x6c, 0x65, 0x6d, 0x65, + 0x6e, 0x74, 0x73, 0x1a, 0x88, 0x01, 0x0a, 0x13, 0x45, 0x6c, 0x65, 0x6d, 0x65, 0x6e, 0x74, 0x53, + 0x61, 0x6d, 0x70, 0x6c, 0x65, 0x73, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x12, 0x10, 0x0a, 0x03, 0x6b, + 0x65, 0x79, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x03, 0x6b, 0x65, 0x79, 0x12, 0x5b, 0x0a, + 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x45, 0x2e, 0x6f, + 0x72, 0x67, 0x2e, 0x61, 0x70, 0x61, 0x63, 0x68, 0x65, 0x2e, 0x62, 0x65, 0x61, 0x6d, 0x2e, 0x6d, + 0x6f, 0x64, 0x65, 0x6c, 0x2e, 0x66, 0x6e, 0x5f, 0x65, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, + 0x6e, 0x2e, 0x76, 0x31, 0x2e, 0x53, 0x61, 0x6d, 0x70, 0x6c, 0x65, 0x44, 0x61, 0x74, 0x61, 0x52, + 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x2e, 0x45, 0x6c, 0x65, 0x6d, 0x65, 0x6e, 0x74, 0x4c, + 0x69, 0x73, 0x74, 0x52, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x3a, 0x02, 0x38, 0x01, 0x22, 0x1f, + 0x0a, 0x1d, 0x48, 0x61, 0x72, 0x6e, 0x65, 0x73, 0x73, 0x4d, 0x6f, 0x6e, 0x69, 0x74, 0x6f, 0x72, + 0x69, 0x6e, 0x67, 0x49, 0x6e, 0x66, 0x6f, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x22, + 0xe8, 0x01, 0x0a, 0x1e, 0x48, 0x61, 0x72, 0x6e, 0x65, 0x73, 0x73, 0x4d, 0x6f, 0x6e, 0x69, 0x74, + 0x6f, 0x72, 0x69, 0x6e, 0x67, 0x49, 0x6e, 0x66, 0x6f, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, + 0x73, 0x65, 0x12, 0x82, 0x01, 0x0a, 0x0f, 0x6d, 0x6f, 0x6e, 0x69, 0x74, 0x6f, 0x72, 0x69, 0x6e, + 0x67, 0x5f, 0x64, 0x61, 0x74, 0x61, 0x18, 0x01, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x59, 0x2e, 0x6f, + 0x72, 0x67, 0x2e, 0x61, 0x70, 0x61, 0x63, 0x68, 0x65, 0x2e, 0x62, 0x65, 0x61, 0x6d, 0x2e, 0x6d, + 0x6f, 0x64, 0x65, 0x6c, 0x2e, 0x66, 0x6e, 0x5f, 0x65, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, + 0x6e, 0x2e, 0x76, 0x31, 0x2e, 0x48, 0x61, 0x72, 0x6e, 0x65, 0x73, 0x73, 0x4d, 0x6f, 0x6e, 0x69, + 0x74, 0x6f, 0x72, 0x69, 0x6e, 0x67, 0x49, 0x6e, 0x66, 0x6f, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, + 0x6e, 0x73, 0x65, 0x2e, 0x4d, 0x6f, 0x6e, 0x69, 0x74, 0x6f, 0x72, 0x69, 0x6e, 0x67, 0x44, 0x61, + 0x74, 0x61, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x52, 0x0e, 0x6d, 0x6f, 0x6e, 0x69, 0x74, 0x6f, 0x72, + 0x69, 0x6e, 0x67, 0x44, 0x61, 0x74, 0x61, 0x1a, 0x41, 0x0a, 0x13, 0x4d, 0x6f, 0x6e, 0x69, 0x74, + 0x6f, 0x72, 0x69, 0x6e, 0x67, 0x44, 0x61, 0x74, 0x61, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x12, 0x10, + 0x0a, 0x03, 0x6b, 0x65, 0x79, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x03, 0x6b, 0x65, 0x79, + 0x12, 0x14, 0x0a, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0c, 0x52, + 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x3a, 0x02, 0x38, 0x01, 0x22, 0x8d, 0x01, 0x0a, 0x0f, 0x52, + 0x65, 0x67, 0x69, 0x73, 0x74, 0x65, 0x72, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x7a, + 0x0a, 0x19, 0x70, 0x72, 0x6f, 0x63, 0x65, 0x73, 0x73, 0x5f, 0x62, 0x75, 0x6e, 0x64, 0x6c, 0x65, + 0x5f, 0x64, 0x65, 0x73, 0x63, 0x72, 0x69, 0x70, 0x74, 0x6f, 0x72, 0x18, 0x01, 0x20, 0x03, 0x28, + 0x0b, 0x32, 0x3e, 0x2e, 0x6f, 0x72, 0x67, 0x2e, 0x61, 0x70, 0x61, 0x63, 0x68, 0x65, 0x2e, 0x62, + 0x65, 0x61, 0x6d, 0x2e, 0x6d, 0x6f, 0x64, 0x65, 0x6c, 0x2e, 0x66, 0x6e, 0x5f, 0x65, 0x78, 0x65, + 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x2e, 0x76, 0x31, 0x2e, 0x50, 0x72, 0x6f, 0x63, 0x65, 0x73, + 0x73, 0x42, 0x75, 0x6e, 0x64, 0x6c, 0x65, 0x44, 0x65, 0x73, 0x63, 0x72, 0x69, 0x70, 0x74, 0x6f, + 0x72, 0x52, 0x17, 0x70, 0x72, 0x6f, 0x63, 0x65, 0x73, 0x73, 0x42, 0x75, 0x6e, 0x64, 0x6c, 0x65, + 0x44, 0x65, 0x73, 0x63, 0x72, 0x69, 0x70, 0x74, 0x6f, 0x72, 0x22, 0x12, 0x0a, 0x10, 0x52, 0x65, + 0x67, 0x69, 0x73, 0x74, 0x65, 0x72, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x9d, + 0x0b, 0x0a, 0x17, 0x50, 0x72, 0x6f, 0x63, 0x65, 0x73, 0x73, 0x42, 0x75, 0x6e, 0x64, 0x6c, 0x65, + 0x44, 0x65, 0x73, 0x63, 0x72, 0x69, 0x70, 0x74, 0x6f, 0x72, 0x12, 0x0e, 0x0a, 0x02, 0x69, 0x64, + 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x02, 0x69, 0x64, 0x12, 0x6e, 0x0a, 0x0a, 0x74, 0x72, + 0x61, 0x6e, 0x73, 0x66, 0x6f, 0x72, 0x6d, 0x73, 0x18, 0x02, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x4e, + 0x2e, 0x6f, 0x72, 0x67, 0x2e, 0x61, 0x70, 0x61, 0x63, 0x68, 0x65, 0x2e, 0x62, 0x65, 0x61, 0x6d, + 0x2e, 0x6d, 0x6f, 0x64, 0x65, 0x6c, 0x2e, 0x66, 0x6e, 0x5f, 0x65, 0x78, 0x65, 0x63, 0x75, 0x74, + 0x69, 0x6f, 0x6e, 0x2e, 0x76, 0x31, 0x2e, 0x50, 0x72, 0x6f, 0x63, 0x65, 0x73, 0x73, 0x42, 0x75, + 0x6e, 0x64, 0x6c, 0x65, 0x44, 0x65, 0x73, 0x63, 0x72, 0x69, 0x70, 0x74, 0x6f, 0x72, 0x2e, 0x54, + 0x72, 0x61, 0x6e, 0x73, 0x66, 0x6f, 0x72, 0x6d, 0x73, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x52, 0x0a, + 0x74, 0x72, 0x61, 0x6e, 0x73, 0x66, 0x6f, 0x72, 0x6d, 0x73, 0x12, 0x74, 0x0a, 0x0c, 0x70, 0x63, + 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x18, 0x03, 0x20, 0x03, 0x28, 0x0b, + 0x32, 0x50, 0x2e, 0x6f, 0x72, 0x67, 0x2e, 0x61, 0x70, 0x61, 0x63, 0x68, 0x65, 0x2e, 0x62, 0x65, + 0x61, 0x6d, 0x2e, 0x6d, 0x6f, 0x64, 0x65, 0x6c, 0x2e, 0x66, 0x6e, 0x5f, 0x65, 0x78, 0x65, 0x63, + 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x2e, 0x76, 0x31, 0x2e, 0x50, 0x72, 0x6f, 0x63, 0x65, 0x73, 0x73, + 0x42, 0x75, 0x6e, 0x64, 0x6c, 0x65, 0x44, 0x65, 0x73, 0x63, 0x72, 0x69, 0x70, 0x74, 0x6f, 0x72, + 0x2e, 0x50, 0x63, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x45, 0x6e, 0x74, + 0x72, 0x79, 0x52, 0x0c, 0x70, 0x63, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x73, + 0x12, 0x8a, 0x01, 0x0a, 0x14, 0x77, 0x69, 0x6e, 0x64, 0x6f, 0x77, 0x69, 0x6e, 0x67, 0x5f, 0x73, + 0x74, 0x72, 0x61, 0x74, 0x65, 0x67, 0x69, 0x65, 0x73, 0x18, 0x04, 0x20, 0x03, 0x28, 0x0b, 0x32, + 0x57, 0x2e, 0x6f, 0x72, 0x67, 0x2e, 0x61, 0x70, 0x61, 0x63, 0x68, 0x65, 0x2e, 0x62, 0x65, 0x61, + 0x6d, 0x2e, 0x6d, 0x6f, 0x64, 0x65, 0x6c, 0x2e, 0x66, 0x6e, 0x5f, 0x65, 0x78, 0x65, 0x63, 0x75, + 0x74, 0x69, 0x6f, 0x6e, 0x2e, 0x76, 0x31, 0x2e, 0x50, 0x72, 0x6f, 0x63, 0x65, 0x73, 0x73, 0x42, + 0x75, 0x6e, 0x64, 0x6c, 0x65, 0x44, 0x65, 0x73, 0x63, 0x72, 0x69, 0x70, 0x74, 0x6f, 0x72, 0x2e, + 0x57, 0x69, 0x6e, 0x64, 0x6f, 0x77, 0x69, 0x6e, 0x67, 0x53, 0x74, 0x72, 0x61, 0x74, 0x65, 0x67, + 0x69, 0x65, 0x73, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x52, 0x13, 0x77, 0x69, 0x6e, 0x64, 0x6f, 0x77, + 0x69, 0x6e, 0x67, 0x53, 0x74, 0x72, 0x61, 0x74, 0x65, 0x67, 0x69, 0x65, 0x73, 0x12, 0x62, 0x0a, + 0x06, 0x63, 0x6f, 0x64, 0x65, 0x72, 0x73, 0x18, 0x05, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x4a, 0x2e, + 0x6f, 0x72, 0x67, 0x2e, 0x61, 0x70, 0x61, 0x63, 0x68, 0x65, 0x2e, 0x62, 0x65, 0x61, 0x6d, 0x2e, + 0x6d, 0x6f, 0x64, 0x65, 0x6c, 0x2e, 0x66, 0x6e, 0x5f, 0x65, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, + 0x6f, 0x6e, 0x2e, 0x76, 0x31, 0x2e, 0x50, 0x72, 0x6f, 0x63, 0x65, 0x73, 0x73, 0x42, 0x75, 0x6e, + 0x64, 0x6c, 0x65, 0x44, 0x65, 0x73, 0x63, 0x72, 0x69, 0x70, 0x74, 0x6f, 0x72, 0x2e, 0x43, 0x6f, + 0x64, 0x65, 0x72, 0x73, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x52, 0x06, 0x63, 0x6f, 0x64, 0x65, 0x72, + 0x73, 0x12, 0x74, 0x0a, 0x0c, 0x65, 0x6e, 0x76, 0x69, 0x72, 0x6f, 0x6e, 0x6d, 0x65, 0x6e, 0x74, + 0x73, 0x18, 0x06, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x50, 0x2e, 0x6f, 0x72, 0x67, 0x2e, 0x61, 0x70, + 0x61, 0x63, 0x68, 0x65, 0x2e, 0x62, 0x65, 0x61, 0x6d, 0x2e, 0x6d, 0x6f, 0x64, 0x65, 0x6c, 0x2e, + 0x66, 0x6e, 0x5f, 0x65, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x2e, 0x76, 0x31, 0x2e, + 0x50, 0x72, 0x6f, 0x63, 0x65, 0x73, 0x73, 0x42, 0x75, 0x6e, 0x64, 0x6c, 0x65, 0x44, 0x65, 0x73, + 0x63, 0x72, 0x69, 0x70, 0x74, 0x6f, 0x72, 0x2e, 0x45, 0x6e, 0x76, 0x69, 0x72, 0x6f, 0x6e, 0x6d, + 0x65, 0x6e, 0x74, 0x73, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x52, 0x0c, 0x65, 0x6e, 0x76, 0x69, 0x72, + 0x6f, 0x6e, 0x6d, 0x65, 0x6e, 0x74, 0x73, 0x12, 0x78, 0x0a, 0x1c, 0x73, 0x74, 0x61, 0x74, 0x65, + 0x5f, 0x61, 0x70, 0x69, 0x5f, 0x73, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, 0x5f, 0x64, 0x65, 0x73, + 0x63, 0x72, 0x69, 0x70, 0x74, 0x6f, 0x72, 0x18, 0x07, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x37, 0x2e, + 0x6f, 0x72, 0x67, 0x2e, 0x61, 0x70, 0x61, 0x63, 0x68, 0x65, 0x2e, 0x62, 0x65, 0x61, 0x6d, 0x2e, + 0x6d, 0x6f, 0x64, 0x65, 0x6c, 0x2e, 0x70, 0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e, 0x65, 0x2e, 0x76, + 0x31, 0x2e, 0x41, 0x70, 0x69, 0x53, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, 0x44, 0x65, 0x73, 0x63, + 0x72, 0x69, 0x70, 0x74, 0x6f, 0x72, 0x52, 0x19, 0x73, 0x74, 0x61, 0x74, 0x65, 0x41, 0x70, 0x69, + 0x53, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, 0x44, 0x65, 0x73, 0x63, 0x72, 0x69, 0x70, 0x74, 0x6f, + 0x72, 0x12, 0x78, 0x0a, 0x1c, 0x74, 0x69, 0x6d, 0x65, 0x72, 0x5f, 0x61, 0x70, 0x69, 0x5f, 0x73, + 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, 0x5f, 0x64, 0x65, 0x73, 0x63, 0x72, 0x69, 0x70, 0x74, 0x6f, + 0x72, 0x18, 0x08, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x37, 0x2e, 0x6f, 0x72, 0x67, 0x2e, 0x61, 0x70, + 0x61, 0x63, 0x68, 0x65, 0x2e, 0x62, 0x65, 0x61, 0x6d, 0x2e, 0x6d, 0x6f, 0x64, 0x65, 0x6c, 0x2e, + 0x70, 0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x41, 0x70, 0x69, 0x53, + 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, 0x44, 0x65, 0x73, 0x63, 0x72, 0x69, 0x70, 0x74, 0x6f, 0x72, + 0x52, 0x19, 0x74, 0x69, 0x6d, 0x65, 0x72, 0x41, 0x70, 0x69, 0x53, 0x65, 0x72, 0x76, 0x69, 0x63, + 0x65, 0x44, 0x65, 0x73, 0x63, 0x72, 0x69, 0x70, 0x74, 0x6f, 0x72, 0x1a, 0x6c, 0x0a, 0x0f, 0x54, + 0x72, 0x61, 0x6e, 0x73, 0x66, 0x6f, 0x72, 0x6d, 0x73, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x12, 0x10, + 0x0a, 0x03, 0x6b, 0x65, 0x79, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x03, 0x6b, 0x65, 0x79, + 0x12, 0x43, 0x0a, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, + 0x2d, 0x2e, 0x6f, 0x72, 0x67, 0x2e, 0x61, 0x70, 0x61, 0x63, 0x68, 0x65, 0x2e, 0x62, 0x65, 0x61, + 0x6d, 0x2e, 0x6d, 0x6f, 0x64, 0x65, 0x6c, 0x2e, 0x70, 0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e, 0x65, + 0x2e, 0x76, 0x31, 0x2e, 0x50, 0x54, 0x72, 0x61, 0x6e, 0x73, 0x66, 0x6f, 0x72, 0x6d, 0x52, 0x05, + 0x76, 0x61, 0x6c, 0x75, 0x65, 0x3a, 0x02, 0x38, 0x01, 0x1a, 0x6f, 0x0a, 0x11, 0x50, 0x63, 0x6f, + 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x12, 0x10, + 0x0a, 0x03, 0x6b, 0x65, 0x79, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x03, 0x6b, 0x65, 0x79, + 0x12, 0x44, 0x0a, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, + 0x2e, 0x2e, 0x6f, 0x72, 0x67, 0x2e, 0x61, 0x70, 0x61, 0x63, 0x68, 0x65, 0x2e, 0x62, 0x65, 0x61, + 0x6d, 0x2e, 0x6d, 0x6f, 0x64, 0x65, 0x6c, 0x2e, 0x70, 0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e, 0x65, + 0x2e, 0x76, 0x31, 0x2e, 0x50, 0x43, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x52, + 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x3a, 0x02, 0x38, 0x01, 0x1a, 0x7c, 0x0a, 0x18, 0x57, 0x69, + 0x6e, 0x64, 0x6f, 0x77, 0x69, 0x6e, 0x67, 0x53, 0x74, 0x72, 0x61, 0x74, 0x65, 0x67, 0x69, 0x65, + 0x73, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x12, 0x10, 0x0a, 0x03, 0x6b, 0x65, 0x79, 0x18, 0x01, 0x20, + 0x01, 0x28, 0x09, 0x52, 0x03, 0x6b, 0x65, 0x79, 0x12, 0x4a, 0x0a, 0x05, 0x76, 0x61, 0x6c, 0x75, + 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x34, 0x2e, 0x6f, 0x72, 0x67, 0x2e, 0x61, 0x70, + 0x61, 0x63, 0x68, 0x65, 0x2e, 0x62, 0x65, 0x61, 0x6d, 0x2e, 0x6d, 0x6f, 0x64, 0x65, 0x6c, 0x2e, + 0x70, 0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x57, 0x69, 0x6e, 0x64, + 0x6f, 0x77, 0x69, 0x6e, 0x67, 0x53, 0x74, 0x72, 0x61, 0x74, 0x65, 0x67, 0x79, 0x52, 0x05, 0x76, + 0x61, 0x6c, 0x75, 0x65, 0x3a, 0x02, 0x38, 0x01, 0x1a, 0x63, 0x0a, 0x0b, 0x43, 0x6f, 0x64, 0x65, + 0x72, 0x73, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x12, 0x10, 0x0a, 0x03, 0x6b, 0x65, 0x79, 0x18, 0x01, + 0x20, 0x01, 0x28, 0x09, 0x52, 0x03, 0x6b, 0x65, 0x79, 0x12, 0x3e, 0x0a, 0x05, 0x76, 0x61, 0x6c, + 0x75, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x28, 0x2e, 0x6f, 0x72, 0x67, 0x2e, 0x61, + 0x70, 0x61, 0x63, 0x68, 0x65, 0x2e, 0x62, 0x65, 0x61, 0x6d, 0x2e, 0x6d, 0x6f, 0x64, 0x65, 0x6c, + 0x2e, 0x70, 0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x43, 0x6f, 0x64, + 0x65, 0x72, 0x52, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x3a, 0x02, 0x38, 0x01, 0x1a, 0x6f, 0x0a, + 0x11, 0x45, 0x6e, 0x76, 0x69, 0x72, 0x6f, 0x6e, 0x6d, 0x65, 0x6e, 0x74, 0x73, 0x45, 0x6e, 0x74, + 0x72, 0x79, 0x12, 0x10, 0x0a, 0x03, 0x6b, 0x65, 0x79, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, + 0x03, 0x6b, 0x65, 0x79, 0x12, 0x44, 0x0a, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x18, 0x02, 0x20, + 0x01, 0x28, 0x0b, 0x32, 0x2e, 0x2e, 0x6f, 0x72, 0x67, 0x2e, 0x61, 0x70, 0x61, 0x63, 0x68, 0x65, + 0x2e, 0x62, 0x65, 0x61, 0x6d, 0x2e, 0x6d, 0x6f, 0x64, 0x65, 0x6c, 0x2e, 0x70, 0x69, 0x70, 0x65, + 0x6c, 0x69, 0x6e, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x45, 0x6e, 0x76, 0x69, 0x72, 0x6f, 0x6e, 0x6d, + 0x65, 0x6e, 0x74, 0x52, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x3a, 0x02, 0x38, 0x01, 0x22, 0x9b, + 0x03, 0x0a, 0x11, 0x42, 0x75, 0x6e, 0x64, 0x6c, 0x65, 0x41, 0x70, 0x70, 0x6c, 0x69, 0x63, 0x61, + 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x21, 0x0a, 0x0c, 0x74, 0x72, 0x61, 0x6e, 0x73, 0x66, 0x6f, 0x72, + 0x6d, 0x5f, 0x69, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0b, 0x74, 0x72, 0x61, 0x6e, + 0x73, 0x66, 0x6f, 0x72, 0x6d, 0x49, 0x64, 0x12, 0x19, 0x0a, 0x08, 0x69, 0x6e, 0x70, 0x75, 0x74, + 0x5f, 0x69, 0x64, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x07, 0x69, 0x6e, 0x70, 0x75, 0x74, + 0x49, 0x64, 0x12, 0x18, 0x0a, 0x07, 0x65, 0x6c, 0x65, 0x6d, 0x65, 0x6e, 0x74, 0x18, 0x03, 0x20, + 0x01, 0x28, 0x0c, 0x52, 0x07, 0x65, 0x6c, 0x65, 0x6d, 0x65, 0x6e, 0x74, 0x12, 0x7b, 0x0a, 0x11, + 0x6f, 0x75, 0x74, 0x70, 0x75, 0x74, 0x5f, 0x77, 0x61, 0x74, 0x65, 0x72, 0x6d, 0x61, 0x72, 0x6b, + 0x73, 0x18, 0x04, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x4e, 0x2e, 0x6f, 0x72, 0x67, 0x2e, 0x61, 0x70, + 0x61, 0x63, 0x68, 0x65, 0x2e, 0x62, 0x65, 0x61, 0x6d, 0x2e, 0x6d, 0x6f, 0x64, 0x65, 0x6c, 0x2e, + 0x66, 0x6e, 0x5f, 0x65, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x2e, 0x76, 0x31, 0x2e, + 0x42, 0x75, 0x6e, 0x64, 0x6c, 0x65, 0x41, 0x70, 0x70, 0x6c, 0x69, 0x63, 0x61, 0x74, 0x69, 0x6f, + 0x6e, 0x2e, 0x4f, 0x75, 0x74, 0x70, 0x75, 0x74, 0x57, 0x61, 0x74, 0x65, 0x72, 0x6d, 0x61, 0x72, + 0x6b, 0x73, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x52, 0x10, 0x6f, 0x75, 0x74, 0x70, 0x75, 0x74, 0x57, + 0x61, 0x74, 0x65, 0x72, 0x6d, 0x61, 0x72, 0x6b, 0x73, 0x12, 0x50, 0x0a, 0x0a, 0x69, 0x73, 0x5f, + 0x62, 0x6f, 0x75, 0x6e, 0x64, 0x65, 0x64, 0x18, 0x05, 0x20, 0x01, 0x28, 0x0e, 0x32, 0x31, 0x2e, + 0x6f, 0x72, 0x67, 0x2e, 0x61, 0x70, 0x61, 0x63, 0x68, 0x65, 0x2e, 0x62, 0x65, 0x61, 0x6d, 0x2e, + 0x6d, 0x6f, 0x64, 0x65, 0x6c, 0x2e, 0x70, 0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e, 0x65, 0x2e, 0x76, + 0x31, 0x2e, 0x49, 0x73, 0x42, 0x6f, 0x75, 0x6e, 0x64, 0x65, 0x64, 0x2e, 0x45, 0x6e, 0x75, 0x6d, + 0x52, 0x09, 0x69, 0x73, 0x42, 0x6f, 0x75, 0x6e, 0x64, 0x65, 0x64, 0x1a, 0x5f, 0x0a, 0x15, 0x4f, + 0x75, 0x74, 0x70, 0x75, 0x74, 0x57, 0x61, 0x74, 0x65, 0x72, 0x6d, 0x61, 0x72, 0x6b, 0x73, 0x45, + 0x6e, 0x74, 0x72, 0x79, 0x12, 0x10, 0x0a, 0x03, 0x6b, 0x65, 0x79, 0x18, 0x01, 0x20, 0x01, 0x28, + 0x09, 0x52, 0x03, 0x6b, 0x65, 0x79, 0x12, 0x30, 0x0a, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x18, + 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1a, 0x2e, 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2e, 0x70, + 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66, 0x2e, 0x54, 0x69, 0x6d, 0x65, 0x73, 0x74, 0x61, 0x6d, + 0x70, 0x52, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x3a, 0x02, 0x38, 0x01, 0x22, 0xc3, 0x01, 0x0a, + 0x18, 0x44, 0x65, 0x6c, 0x61, 0x79, 0x65, 0x64, 0x42, 0x75, 0x6e, 0x64, 0x6c, 0x65, 0x41, 0x70, + 0x70, 0x6c, 0x69, 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x5a, 0x0a, 0x0b, 0x61, 0x70, 0x70, + 0x6c, 0x69, 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x38, + 0x2e, 0x6f, 0x72, 0x67, 0x2e, 0x61, 0x70, 0x61, 0x63, 0x68, 0x65, 0x2e, 0x62, 0x65, 0x61, 0x6d, + 0x2e, 0x6d, 0x6f, 0x64, 0x65, 0x6c, 0x2e, 0x66, 0x6e, 0x5f, 0x65, 0x78, 0x65, 0x63, 0x75, 0x74, + 0x69, 0x6f, 0x6e, 0x2e, 0x76, 0x31, 0x2e, 0x42, 0x75, 0x6e, 0x64, 0x6c, 0x65, 0x41, 0x70, 0x70, + 0x6c, 0x69, 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x0b, 0x61, 0x70, 0x70, 0x6c, 0x69, 0x63, + 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x4b, 0x0a, 0x14, 0x72, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, + 0x65, 0x64, 0x5f, 0x74, 0x69, 0x6d, 0x65, 0x5f, 0x64, 0x65, 0x6c, 0x61, 0x79, 0x18, 0x02, 0x20, + 0x01, 0x28, 0x0b, 0x32, 0x19, 0x2e, 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2e, 0x70, 0x72, 0x6f, + 0x74, 0x6f, 0x62, 0x75, 0x66, 0x2e, 0x44, 0x75, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x12, + 0x72, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x65, 0x64, 0x54, 0x69, 0x6d, 0x65, 0x44, 0x65, 0x6c, + 0x61, 0x79, 0x22, 0x83, 0x05, 0x0a, 0x14, 0x50, 0x72, 0x6f, 0x63, 0x65, 0x73, 0x73, 0x42, 0x75, + 0x6e, 0x64, 0x6c, 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x3f, 0x0a, 0x1c, 0x70, + 0x72, 0x6f, 0x63, 0x65, 0x73, 0x73, 0x5f, 0x62, 0x75, 0x6e, 0x64, 0x6c, 0x65, 0x5f, 0x64, 0x65, + 0x73, 0x63, 0x72, 0x69, 0x70, 0x74, 0x6f, 0x72, 0x5f, 0x69, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, + 0x09, 0x52, 0x19, 0x70, 0x72, 0x6f, 0x63, 0x65, 0x73, 0x73, 0x42, 0x75, 0x6e, 0x64, 0x6c, 0x65, + 0x44, 0x65, 0x73, 0x63, 0x72, 0x69, 0x70, 0x74, 0x6f, 0x72, 0x49, 0x64, 0x12, 0x69, 0x0a, 0x0c, + 0x63, 0x61, 0x63, 0x68, 0x65, 0x5f, 0x74, 0x6f, 0x6b, 0x65, 0x6e, 0x73, 0x18, 0x02, 0x20, 0x03, + 0x28, 0x0b, 0x32, 0x46, 0x2e, 0x6f, 0x72, 0x67, 0x2e, 0x61, 0x70, 0x61, 0x63, 0x68, 0x65, 0x2e, + 0x62, 0x65, 0x61, 0x6d, 0x2e, 0x6d, 0x6f, 0x64, 0x65, 0x6c, 0x2e, 0x66, 0x6e, 0x5f, 0x65, 0x78, + 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x2e, 0x76, 0x31, 0x2e, 0x50, 0x72, 0x6f, 0x63, 0x65, + 0x73, 0x73, 0x42, 0x75, 0x6e, 0x64, 0x6c, 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x2e, + 0x43, 0x61, 0x63, 0x68, 0x65, 0x54, 0x6f, 0x6b, 0x65, 0x6e, 0x52, 0x0b, 0x63, 0x61, 0x63, 0x68, + 0x65, 0x54, 0x6f, 0x6b, 0x65, 0x6e, 0x73, 0x12, 0x4b, 0x0a, 0x08, 0x65, 0x6c, 0x65, 0x6d, 0x65, + 0x6e, 0x74, 0x73, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x2f, 0x2e, 0x6f, 0x72, 0x67, 0x2e, + 0x61, 0x70, 0x61, 0x63, 0x68, 0x65, 0x2e, 0x62, 0x65, 0x61, 0x6d, 0x2e, 0x6d, 0x6f, 0x64, 0x65, + 0x6c, 0x2e, 0x66, 0x6e, 0x5f, 0x65, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x2e, 0x76, + 0x31, 0x2e, 0x45, 0x6c, 0x65, 0x6d, 0x65, 0x6e, 0x74, 0x73, 0x52, 0x08, 0x65, 0x6c, 0x65, 0x6d, + 0x65, 0x6e, 0x74, 0x73, 0x1a, 0xf1, 0x02, 0x0a, 0x0a, 0x43, 0x61, 0x63, 0x68, 0x65, 0x54, 0x6f, + 0x6b, 0x65, 0x6e, 0x12, 0x71, 0x0a, 0x0a, 0x75, 0x73, 0x65, 0x72, 0x5f, 0x73, 0x74, 0x61, 0x74, + 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x50, 0x2e, 0x6f, 0x72, 0x67, 0x2e, 0x61, 0x70, + 0x61, 0x63, 0x68, 0x65, 0x2e, 0x62, 0x65, 0x61, 0x6d, 0x2e, 0x6d, 0x6f, 0x64, 0x65, 0x6c, 0x2e, + 0x66, 0x6e, 0x5f, 0x65, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x2e, 0x76, 0x31, 0x2e, + 0x50, 0x72, 0x6f, 0x63, 0x65, 0x73, 0x73, 0x42, 0x75, 0x6e, 0x64, 0x6c, 0x65, 0x52, 0x65, 0x71, + 0x75, 0x65, 0x73, 0x74, 0x2e, 0x43, 0x61, 0x63, 0x68, 0x65, 0x54, 0x6f, 0x6b, 0x65, 0x6e, 0x2e, + 0x55, 0x73, 0x65, 0x72, 0x53, 0x74, 0x61, 0x74, 0x65, 0x48, 0x00, 0x52, 0x09, 0x75, 0x73, 0x65, + 0x72, 0x53, 0x74, 0x61, 0x74, 0x65, 0x12, 0x71, 0x0a, 0x0a, 0x73, 0x69, 0x64, 0x65, 0x5f, 0x69, + 0x6e, 0x70, 0x75, 0x74, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x50, 0x2e, 0x6f, 0x72, 0x67, + 0x2e, 0x61, 0x70, 0x61, 0x63, 0x68, 0x65, 0x2e, 0x62, 0x65, 0x61, 0x6d, 0x2e, 0x6d, 0x6f, 0x64, + 0x65, 0x6c, 0x2e, 0x66, 0x6e, 0x5f, 0x65, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x2e, + 0x76, 0x31, 0x2e, 0x50, 0x72, 0x6f, 0x63, 0x65, 0x73, 0x73, 0x42, 0x75, 0x6e, 0x64, 0x6c, 0x65, + 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x2e, 0x43, 0x61, 0x63, 0x68, 0x65, 0x54, 0x6f, 0x6b, + 0x65, 0x6e, 0x2e, 0x53, 0x69, 0x64, 0x65, 0x49, 0x6e, 0x70, 0x75, 0x74, 0x48, 0x00, 0x52, 0x09, + 0x73, 0x69, 0x64, 0x65, 0x49, 0x6e, 0x70, 0x75, 0x74, 0x12, 0x14, 0x0a, 0x05, 0x74, 0x6f, 0x6b, + 0x65, 0x6e, 0x18, 0x0a, 0x20, 0x01, 0x28, 0x0c, 0x52, 0x05, 0x74, 0x6f, 0x6b, 0x65, 0x6e, 0x1a, + 0x0b, 0x0a, 0x09, 0x55, 0x73, 0x65, 0x72, 0x53, 0x74, 0x61, 0x74, 0x65, 0x1a, 0x52, 0x0a, 0x09, + 0x53, 0x69, 0x64, 0x65, 0x49, 0x6e, 0x70, 0x75, 0x74, 0x12, 0x21, 0x0a, 0x0c, 0x74, 0x72, 0x61, + 0x6e, 0x73, 0x66, 0x6f, 0x72, 0x6d, 0x5f, 0x69, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, + 0x0b, 0x74, 0x72, 0x61, 0x6e, 0x73, 0x66, 0x6f, 0x72, 0x6d, 0x49, 0x64, 0x12, 0x22, 0x0a, 0x0d, + 0x73, 0x69, 0x64, 0x65, 0x5f, 0x69, 0x6e, 0x70, 0x75, 0x74, 0x5f, 0x69, 0x64, 0x18, 0x02, 0x20, + 0x01, 0x28, 0x09, 0x52, 0x0b, 0x73, 0x69, 0x64, 0x65, 0x49, 0x6e, 0x70, 0x75, 0x74, 0x49, 0x64, + 0x42, 0x06, 0x0a, 0x04, 0x74, 0x79, 0x70, 0x65, 0x22, 0xa3, 0x04, 0x0a, 0x15, 0x50, 0x72, 0x6f, + 0x63, 0x65, 0x73, 0x73, 0x42, 0x75, 0x6e, 0x64, 0x6c, 0x65, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, + 0x73, 0x65, 0x12, 0x66, 0x0a, 0x0e, 0x72, 0x65, 0x73, 0x69, 0x64, 0x75, 0x61, 0x6c, 0x5f, 0x72, + 0x6f, 0x6f, 0x74, 0x73, 0x18, 0x02, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x3f, 0x2e, 0x6f, 0x72, 0x67, + 0x2e, 0x61, 0x70, 0x61, 0x63, 0x68, 0x65, 0x2e, 0x62, 0x65, 0x61, 0x6d, 0x2e, 0x6d, 0x6f, 0x64, + 0x65, 0x6c, 0x2e, 0x66, 0x6e, 0x5f, 0x65, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x2e, + 0x76, 0x31, 0x2e, 0x44, 0x65, 0x6c, 0x61, 0x79, 0x65, 0x64, 0x42, 0x75, 0x6e, 0x64, 0x6c, 0x65, + 0x41, 0x70, 0x70, 0x6c, 0x69, 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x0d, 0x72, 0x65, 0x73, + 0x69, 0x64, 0x75, 0x61, 0x6c, 0x52, 0x6f, 0x6f, 0x74, 0x73, 0x12, 0x5c, 0x0a, 0x10, 0x6d, 0x6f, + 0x6e, 0x69, 0x74, 0x6f, 0x72, 0x69, 0x6e, 0x67, 0x5f, 0x69, 0x6e, 0x66, 0x6f, 0x73, 0x18, 0x03, + 0x20, 0x03, 0x28, 0x0b, 0x32, 0x31, 0x2e, 0x6f, 0x72, 0x67, 0x2e, 0x61, 0x70, 0x61, 0x63, 0x68, + 0x65, 0x2e, 0x62, 0x65, 0x61, 0x6d, 0x2e, 0x6d, 0x6f, 0x64, 0x65, 0x6c, 0x2e, 0x70, 0x69, 0x70, + 0x65, 0x6c, 0x69, 0x6e, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x4d, 0x6f, 0x6e, 0x69, 0x74, 0x6f, 0x72, + 0x69, 0x6e, 0x67, 0x49, 0x6e, 0x66, 0x6f, 0x52, 0x0f, 0x6d, 0x6f, 0x6e, 0x69, 0x74, 0x6f, 0x72, + 0x69, 0x6e, 0x67, 0x49, 0x6e, 0x66, 0x6f, 0x73, 0x12, 0x33, 0x0a, 0x15, 0x72, 0x65, 0x71, 0x75, + 0x69, 0x72, 0x65, 0x73, 0x5f, 0x66, 0x69, 0x6e, 0x61, 0x6c, 0x69, 0x7a, 0x61, 0x74, 0x69, 0x6f, + 0x6e, 0x18, 0x04, 0x20, 0x01, 0x28, 0x08, 0x52, 0x14, 0x72, 0x65, 0x71, 0x75, 0x69, 0x72, 0x65, + 0x73, 0x46, 0x69, 0x6e, 0x61, 0x6c, 0x69, 0x7a, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x79, 0x0a, + 0x0f, 0x6d, 0x6f, 0x6e, 0x69, 0x74, 0x6f, 0x72, 0x69, 0x6e, 0x67, 0x5f, 0x64, 0x61, 0x74, 0x61, + 0x18, 0x05, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x50, 0x2e, 0x6f, 0x72, 0x67, 0x2e, 0x61, 0x70, 0x61, + 0x63, 0x68, 0x65, 0x2e, 0x62, 0x65, 0x61, 0x6d, 0x2e, 0x6d, 0x6f, 0x64, 0x65, 0x6c, 0x2e, 0x66, + 0x6e, 0x5f, 0x65, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x2e, 0x76, 0x31, 0x2e, 0x50, + 0x72, 0x6f, 0x63, 0x65, 0x73, 0x73, 0x42, 0x75, 0x6e, 0x64, 0x6c, 0x65, 0x52, 0x65, 0x73, 0x70, + 0x6f, 0x6e, 0x73, 0x65, 0x2e, 0x4d, 0x6f, 0x6e, 0x69, 0x74, 0x6f, 0x72, 0x69, 0x6e, 0x67, 0x44, + 0x61, 0x74, 0x61, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x52, 0x0e, 0x6d, 0x6f, 0x6e, 0x69, 0x74, 0x6f, + 0x72, 0x69, 0x6e, 0x67, 0x44, 0x61, 0x74, 0x61, 0x12, 0x4b, 0x0a, 0x08, 0x65, 0x6c, 0x65, 0x6d, + 0x65, 0x6e, 0x74, 0x73, 0x18, 0x06, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x2f, 0x2e, 0x6f, 0x72, 0x67, + 0x2e, 0x61, 0x70, 0x61, 0x63, 0x68, 0x65, 0x2e, 0x62, 0x65, 0x61, 0x6d, 0x2e, 0x6d, 0x6f, 0x64, + 0x65, 0x6c, 0x2e, 0x66, 0x6e, 0x5f, 0x65, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x2e, + 0x76, 0x31, 0x2e, 0x45, 0x6c, 0x65, 0x6d, 0x65, 0x6e, 0x74, 0x73, 0x52, 0x08, 0x65, 0x6c, 0x65, + 0x6d, 0x65, 0x6e, 0x74, 0x73, 0x1a, 0x41, 0x0a, 0x13, 0x4d, 0x6f, 0x6e, 0x69, 0x74, 0x6f, 0x72, + 0x69, 0x6e, 0x67, 0x44, 0x61, 0x74, 0x61, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x12, 0x10, 0x0a, 0x03, + 0x6b, 0x65, 0x79, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x03, 0x6b, 0x65, 0x79, 0x12, 0x14, + 0x0a, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0c, 0x52, 0x05, 0x76, + 0x61, 0x6c, 0x75, 0x65, 0x3a, 0x02, 0x38, 0x01, 0x4a, 0x04, 0x08, 0x01, 0x10, 0x02, 0x22, 0x45, + 0x0a, 0x1c, 0x50, 0x72, 0x6f, 0x63, 0x65, 0x73, 0x73, 0x42, 0x75, 0x6e, 0x64, 0x6c, 0x65, 0x50, + 0x72, 0x6f, 0x67, 0x72, 0x65, 0x73, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x25, + 0x0a, 0x0e, 0x69, 0x6e, 0x73, 0x74, 0x72, 0x75, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x69, 0x64, + 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0d, 0x69, 0x6e, 0x73, 0x74, 0x72, 0x75, 0x63, 0x74, + 0x69, 0x6f, 0x6e, 0x49, 0x64, 0x22, 0x4e, 0x0a, 0x1e, 0x4d, 0x6f, 0x6e, 0x69, 0x74, 0x6f, 0x72, + 0x69, 0x6e, 0x67, 0x49, 0x6e, 0x66, 0x6f, 0x73, 0x4d, 0x65, 0x74, 0x61, 0x64, 0x61, 0x74, 0x61, + 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x2c, 0x0a, 0x12, 0x6d, 0x6f, 0x6e, 0x69, 0x74, + 0x6f, 0x72, 0x69, 0x6e, 0x67, 0x5f, 0x69, 0x6e, 0x66, 0x6f, 0x5f, 0x69, 0x64, 0x18, 0x01, 0x20, + 0x03, 0x28, 0x09, 0x52, 0x10, 0x6d, 0x6f, 0x6e, 0x69, 0x74, 0x6f, 0x72, 0x69, 0x6e, 0x67, 0x49, + 0x6e, 0x66, 0x6f, 0x49, 0x64, 0x22, 0xaf, 0x03, 0x0a, 0x1d, 0x50, 0x72, 0x6f, 0x63, 0x65, 0x73, + 0x73, 0x42, 0x75, 0x6e, 0x64, 0x6c, 0x65, 0x50, 0x72, 0x6f, 0x67, 0x72, 0x65, 0x73, 0x73, 0x52, + 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x5c, 0x0a, 0x10, 0x6d, 0x6f, 0x6e, 0x69, 0x74, + 0x6f, 0x72, 0x69, 0x6e, 0x67, 0x5f, 0x69, 0x6e, 0x66, 0x6f, 0x73, 0x18, 0x03, 0x20, 0x03, 0x28, + 0x0b, 0x32, 0x31, 0x2e, 0x6f, 0x72, 0x67, 0x2e, 0x61, 0x70, 0x61, 0x63, 0x68, 0x65, 0x2e, 0x62, + 0x65, 0x61, 0x6d, 0x2e, 0x6d, 0x6f, 0x64, 0x65, 0x6c, 0x2e, 0x70, 0x69, 0x70, 0x65, 0x6c, 0x69, + 0x6e, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x4d, 0x6f, 0x6e, 0x69, 0x74, 0x6f, 0x72, 0x69, 0x6e, 0x67, + 0x49, 0x6e, 0x66, 0x6f, 0x52, 0x0f, 0x6d, 0x6f, 0x6e, 0x69, 0x74, 0x6f, 0x72, 0x69, 0x6e, 0x67, + 0x49, 0x6e, 0x66, 0x6f, 0x73, 0x12, 0x81, 0x01, 0x0a, 0x0f, 0x6d, 0x6f, 0x6e, 0x69, 0x74, 0x6f, + 0x72, 0x69, 0x6e, 0x67, 0x5f, 0x64, 0x61, 0x74, 0x61, 0x18, 0x05, 0x20, 0x03, 0x28, 0x0b, 0x32, + 0x58, 0x2e, 0x6f, 0x72, 0x67, 0x2e, 0x61, 0x70, 0x61, 0x63, 0x68, 0x65, 0x2e, 0x62, 0x65, 0x61, + 0x6d, 0x2e, 0x6d, 0x6f, 0x64, 0x65, 0x6c, 0x2e, 0x66, 0x6e, 0x5f, 0x65, 0x78, 0x65, 0x63, 0x75, + 0x74, 0x69, 0x6f, 0x6e, 0x2e, 0x76, 0x31, 0x2e, 0x50, 0x72, 0x6f, 0x63, 0x65, 0x73, 0x73, 0x42, + 0x75, 0x6e, 0x64, 0x6c, 0x65, 0x50, 0x72, 0x6f, 0x67, 0x72, 0x65, 0x73, 0x73, 0x52, 0x65, 0x73, + 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x2e, 0x4d, 0x6f, 0x6e, 0x69, 0x74, 0x6f, 0x72, 0x69, 0x6e, 0x67, + 0x44, 0x61, 0x74, 0x61, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x52, 0x0e, 0x6d, 0x6f, 0x6e, 0x69, 0x74, + 0x6f, 0x72, 0x69, 0x6e, 0x67, 0x44, 0x61, 0x74, 0x61, 0x12, 0x3b, 0x0a, 0x17, 0x63, 0x6f, 0x6e, + 0x73, 0x75, 0x6d, 0x69, 0x6e, 0x67, 0x5f, 0x72, 0x65, 0x63, 0x65, 0x69, 0x76, 0x65, 0x64, 0x5f, + 0x64, 0x61, 0x74, 0x61, 0x18, 0x06, 0x20, 0x01, 0x28, 0x08, 0x48, 0x00, 0x52, 0x15, 0x63, 0x6f, + 0x6e, 0x73, 0x75, 0x6d, 0x69, 0x6e, 0x67, 0x52, 0x65, 0x63, 0x65, 0x69, 0x76, 0x65, 0x64, 0x44, + 0x61, 0x74, 0x61, 0x88, 0x01, 0x01, 0x1a, 0x41, 0x0a, 0x13, 0x4d, 0x6f, 0x6e, 0x69, 0x74, 0x6f, + 0x72, 0x69, 0x6e, 0x67, 0x44, 0x61, 0x74, 0x61, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x12, 0x10, 0x0a, + 0x03, 0x6b, 0x65, 0x79, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x03, 0x6b, 0x65, 0x79, 0x12, + 0x14, 0x0a, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0c, 0x52, 0x05, + 0x76, 0x61, 0x6c, 0x75, 0x65, 0x3a, 0x02, 0x38, 0x01, 0x42, 0x1a, 0x0a, 0x18, 0x5f, 0x63, 0x6f, + 0x6e, 0x73, 0x75, 0x6d, 0x69, 0x6e, 0x67, 0x5f, 0x72, 0x65, 0x63, 0x65, 0x69, 0x76, 0x65, 0x64, + 0x5f, 0x64, 0x61, 0x74, 0x61, 0x4a, 0x04, 0x08, 0x01, 0x10, 0x02, 0x4a, 0x04, 0x08, 0x02, 0x10, + 0x03, 0x4a, 0x04, 0x08, 0x04, 0x10, 0x05, 0x22, 0x9d, 0x02, 0x0a, 0x1f, 0x4d, 0x6f, 0x6e, 0x69, + 0x74, 0x6f, 0x72, 0x69, 0x6e, 0x67, 0x49, 0x6e, 0x66, 0x6f, 0x73, 0x4d, 0x65, 0x74, 0x61, 0x64, + 0x61, 0x74, 0x61, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x83, 0x01, 0x0a, 0x0f, + 0x6d, 0x6f, 0x6e, 0x69, 0x74, 0x6f, 0x72, 0x69, 0x6e, 0x67, 0x5f, 0x69, 0x6e, 0x66, 0x6f, 0x18, + 0x01, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x5a, 0x2e, 0x6f, 0x72, 0x67, 0x2e, 0x61, 0x70, 0x61, 0x63, + 0x68, 0x65, 0x2e, 0x62, 0x65, 0x61, 0x6d, 0x2e, 0x6d, 0x6f, 0x64, 0x65, 0x6c, 0x2e, 0x66, 0x6e, + 0x5f, 0x65, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x2e, 0x76, 0x31, 0x2e, 0x4d, 0x6f, + 0x6e, 0x69, 0x74, 0x6f, 0x72, 0x69, 0x6e, 0x67, 0x49, 0x6e, 0x66, 0x6f, 0x73, 0x4d, 0x65, 0x74, + 0x61, 0x64, 0x61, 0x74, 0x61, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x2e, 0x4d, 0x6f, + 0x6e, 0x69, 0x74, 0x6f, 0x72, 0x69, 0x6e, 0x67, 0x49, 0x6e, 0x66, 0x6f, 0x45, 0x6e, 0x74, 0x72, + 0x79, 0x52, 0x0e, 0x6d, 0x6f, 0x6e, 0x69, 0x74, 0x6f, 0x72, 0x69, 0x6e, 0x67, 0x49, 0x6e, 0x66, + 0x6f, 0x1a, 0x74, 0x0a, 0x13, 0x4d, 0x6f, 0x6e, 0x69, 0x74, 0x6f, 0x72, 0x69, 0x6e, 0x67, 0x49, + 0x6e, 0x66, 0x6f, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x12, 0x10, 0x0a, 0x03, 0x6b, 0x65, 0x79, 0x18, + 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x03, 0x6b, 0x65, 0x79, 0x12, 0x47, 0x0a, 0x05, 0x76, 0x61, + 0x6c, 0x75, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x31, 0x2e, 0x6f, 0x72, 0x67, 0x2e, + 0x61, 0x70, 0x61, 0x63, 0x68, 0x65, 0x2e, 0x62, 0x65, 0x61, 0x6d, 0x2e, 0x6d, 0x6f, 0x64, 0x65, + 0x6c, 0x2e, 0x70, 0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x4d, 0x6f, + 0x6e, 0x69, 0x74, 0x6f, 0x72, 0x69, 0x6e, 0x67, 0x49, 0x6e, 0x66, 0x6f, 0x52, 0x05, 0x76, 0x61, + 0x6c, 0x75, 0x65, 0x3a, 0x02, 0x38, 0x01, 0x22, 0x81, 0x04, 0x0a, 0x19, 0x50, 0x72, 0x6f, 0x63, + 0x65, 0x73, 0x73, 0x42, 0x75, 0x6e, 0x64, 0x6c, 0x65, 0x53, 0x70, 0x6c, 0x69, 0x74, 0x52, 0x65, + 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x25, 0x0a, 0x0e, 0x69, 0x6e, 0x73, 0x74, 0x72, 0x75, 0x63, + 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x69, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0d, 0x69, + 0x6e, 0x73, 0x74, 0x72, 0x75, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x64, 0x12, 0x7a, 0x0a, 0x0e, + 0x64, 0x65, 0x73, 0x69, 0x72, 0x65, 0x64, 0x5f, 0x73, 0x70, 0x6c, 0x69, 0x74, 0x73, 0x18, 0x03, + 0x20, 0x03, 0x28, 0x0b, 0x32, 0x53, 0x2e, 0x6f, 0x72, 0x67, 0x2e, 0x61, 0x70, 0x61, 0x63, 0x68, + 0x65, 0x2e, 0x62, 0x65, 0x61, 0x6d, 0x2e, 0x6d, 0x6f, 0x64, 0x65, 0x6c, 0x2e, 0x66, 0x6e, 0x5f, + 0x65, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x2e, 0x76, 0x31, 0x2e, 0x50, 0x72, 0x6f, + 0x63, 0x65, 0x73, 0x73, 0x42, 0x75, 0x6e, 0x64, 0x6c, 0x65, 0x53, 0x70, 0x6c, 0x69, 0x74, 0x52, + 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x2e, 0x44, 0x65, 0x73, 0x69, 0x72, 0x65, 0x64, 0x53, 0x70, + 0x6c, 0x69, 0x74, 0x73, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x52, 0x0d, 0x64, 0x65, 0x73, 0x69, 0x72, + 0x65, 0x64, 0x53, 0x70, 0x6c, 0x69, 0x74, 0x73, 0x1a, 0xae, 0x01, 0x0a, 0x0c, 0x44, 0x65, 0x73, + 0x69, 0x72, 0x65, 0x64, 0x53, 0x70, 0x6c, 0x69, 0x74, 0x12, 0x32, 0x0a, 0x15, 0x66, 0x72, 0x61, + 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x6f, 0x66, 0x5f, 0x72, 0x65, 0x6d, 0x61, 0x69, 0x6e, 0x64, + 0x65, 0x72, 0x18, 0x01, 0x20, 0x01, 0x28, 0x01, 0x52, 0x13, 0x66, 0x72, 0x61, 0x63, 0x74, 0x69, + 0x6f, 0x6e, 0x4f, 0x66, 0x52, 0x65, 0x6d, 0x61, 0x69, 0x6e, 0x64, 0x65, 0x72, 0x12, 0x30, 0x0a, + 0x14, 0x61, 0x6c, 0x6c, 0x6f, 0x77, 0x65, 0x64, 0x5f, 0x73, 0x70, 0x6c, 0x69, 0x74, 0x5f, 0x70, + 0x6f, 0x69, 0x6e, 0x74, 0x73, 0x18, 0x03, 0x20, 0x03, 0x28, 0x03, 0x52, 0x12, 0x61, 0x6c, 0x6c, + 0x6f, 0x77, 0x65, 0x64, 0x53, 0x70, 0x6c, 0x69, 0x74, 0x50, 0x6f, 0x69, 0x6e, 0x74, 0x73, 0x12, + 0x38, 0x0a, 0x18, 0x65, 0x73, 0x74, 0x69, 0x6d, 0x61, 0x74, 0x65, 0x64, 0x5f, 0x69, 0x6e, 0x70, + 0x75, 0x74, 0x5f, 0x65, 0x6c, 0x65, 0x6d, 0x65, 0x6e, 0x74, 0x73, 0x18, 0x02, 0x20, 0x01, 0x28, + 0x03, 0x52, 0x16, 0x65, 0x73, 0x74, 0x69, 0x6d, 0x61, 0x74, 0x65, 0x64, 0x49, 0x6e, 0x70, 0x75, + 0x74, 0x45, 0x6c, 0x65, 0x6d, 0x65, 0x6e, 0x74, 0x73, 0x1a, 0x8f, 0x01, 0x0a, 0x12, 0x44, 0x65, + 0x73, 0x69, 0x72, 0x65, 0x64, 0x53, 0x70, 0x6c, 0x69, 0x74, 0x73, 0x45, 0x6e, 0x74, 0x72, 0x79, + 0x12, 0x10, 0x0a, 0x03, 0x6b, 0x65, 0x79, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x03, 0x6b, + 0x65, 0x79, 0x12, 0x63, 0x0a, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, + 0x0b, 0x32, 0x4d, 0x2e, 0x6f, 0x72, 0x67, 0x2e, 0x61, 0x70, 0x61, 0x63, 0x68, 0x65, 0x2e, 0x62, + 0x65, 0x61, 0x6d, 0x2e, 0x6d, 0x6f, 0x64, 0x65, 0x6c, 0x2e, 0x66, 0x6e, 0x5f, 0x65, 0x78, 0x65, + 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x2e, 0x76, 0x31, 0x2e, 0x50, 0x72, 0x6f, 0x63, 0x65, 0x73, + 0x73, 0x42, 0x75, 0x6e, 0x64, 0x6c, 0x65, 0x53, 0x70, 0x6c, 0x69, 0x74, 0x52, 0x65, 0x71, 0x75, + 0x65, 0x73, 0x74, 0x2e, 0x44, 0x65, 0x73, 0x69, 0x72, 0x65, 0x64, 0x53, 0x70, 0x6c, 0x69, 0x74, + 0x52, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x3a, 0x02, 0x38, 0x01, 0x22, 0xf6, 0x03, 0x0a, 0x1a, + 0x50, 0x72, 0x6f, 0x63, 0x65, 0x73, 0x73, 0x42, 0x75, 0x6e, 0x64, 0x6c, 0x65, 0x53, 0x70, 0x6c, + 0x69, 0x74, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x5d, 0x0a, 0x0d, 0x70, 0x72, + 0x69, 0x6d, 0x61, 0x72, 0x79, 0x5f, 0x72, 0x6f, 0x6f, 0x74, 0x73, 0x18, 0x01, 0x20, 0x03, 0x28, + 0x0b, 0x32, 0x38, 0x2e, 0x6f, 0x72, 0x67, 0x2e, 0x61, 0x70, 0x61, 0x63, 0x68, 0x65, 0x2e, 0x62, + 0x65, 0x61, 0x6d, 0x2e, 0x6d, 0x6f, 0x64, 0x65, 0x6c, 0x2e, 0x66, 0x6e, 0x5f, 0x65, 0x78, 0x65, + 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x2e, 0x76, 0x31, 0x2e, 0x42, 0x75, 0x6e, 0x64, 0x6c, 0x65, + 0x41, 0x70, 0x70, 0x6c, 0x69, 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x0c, 0x70, 0x72, 0x69, + 0x6d, 0x61, 0x72, 0x79, 0x52, 0x6f, 0x6f, 0x74, 0x73, 0x12, 0x66, 0x0a, 0x0e, 0x72, 0x65, 0x73, + 0x69, 0x64, 0x75, 0x61, 0x6c, 0x5f, 0x72, 0x6f, 0x6f, 0x74, 0x73, 0x18, 0x02, 0x20, 0x03, 0x28, + 0x0b, 0x32, 0x3f, 0x2e, 0x6f, 0x72, 0x67, 0x2e, 0x61, 0x70, 0x61, 0x63, 0x68, 0x65, 0x2e, 0x62, + 0x65, 0x61, 0x6d, 0x2e, 0x6d, 0x6f, 0x64, 0x65, 0x6c, 0x2e, 0x66, 0x6e, 0x5f, 0x65, 0x78, 0x65, + 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x2e, 0x76, 0x31, 0x2e, 0x44, 0x65, 0x6c, 0x61, 0x79, 0x65, + 0x64, 0x42, 0x75, 0x6e, 0x64, 0x6c, 0x65, 0x41, 0x70, 0x70, 0x6c, 0x69, 0x63, 0x61, 0x74, 0x69, + 0x6f, 0x6e, 0x52, 0x0d, 0x72, 0x65, 0x73, 0x69, 0x64, 0x75, 0x61, 0x6c, 0x52, 0x6f, 0x6f, 0x74, + 0x73, 0x12, 0x75, 0x0a, 0x0e, 0x63, 0x68, 0x61, 0x6e, 0x6e, 0x65, 0x6c, 0x5f, 0x73, 0x70, 0x6c, + 0x69, 0x74, 0x73, 0x18, 0x03, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x4e, 0x2e, 0x6f, 0x72, 0x67, 0x2e, + 0x61, 0x70, 0x61, 0x63, 0x68, 0x65, 0x2e, 0x62, 0x65, 0x61, 0x6d, 0x2e, 0x6d, 0x6f, 0x64, 0x65, + 0x6c, 0x2e, 0x66, 0x6e, 0x5f, 0x65, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x2e, 0x76, + 0x31, 0x2e, 0x50, 0x72, 0x6f, 0x63, 0x65, 0x73, 0x73, 0x42, 0x75, 0x6e, 0x64, 0x6c, 0x65, 0x53, + 0x70, 0x6c, 0x69, 0x74, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x2e, 0x43, 0x68, 0x61, + 0x6e, 0x6e, 0x65, 0x6c, 0x53, 0x70, 0x6c, 0x69, 0x74, 0x52, 0x0d, 0x63, 0x68, 0x61, 0x6e, 0x6e, + 0x65, 0x6c, 0x53, 0x70, 0x6c, 0x69, 0x74, 0x73, 0x1a, 0x99, 0x01, 0x0a, 0x0c, 0x43, 0x68, 0x61, + 0x6e, 0x6e, 0x65, 0x6c, 0x53, 0x70, 0x6c, 0x69, 0x74, 0x12, 0x21, 0x0a, 0x0c, 0x74, 0x72, 0x61, + 0x6e, 0x73, 0x66, 0x6f, 0x72, 0x6d, 0x5f, 0x69, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, + 0x0b, 0x74, 0x72, 0x61, 0x6e, 0x73, 0x66, 0x6f, 0x72, 0x6d, 0x49, 0x64, 0x12, 0x30, 0x0a, 0x14, + 0x6c, 0x61, 0x73, 0x74, 0x5f, 0x70, 0x72, 0x69, 0x6d, 0x61, 0x72, 0x79, 0x5f, 0x65, 0x6c, 0x65, + 0x6d, 0x65, 0x6e, 0x74, 0x18, 0x02, 0x20, 0x01, 0x28, 0x03, 0x52, 0x12, 0x6c, 0x61, 0x73, 0x74, + 0x50, 0x72, 0x69, 0x6d, 0x61, 0x72, 0x79, 0x45, 0x6c, 0x65, 0x6d, 0x65, 0x6e, 0x74, 0x12, 0x34, + 0x0a, 0x16, 0x66, 0x69, 0x72, 0x73, 0x74, 0x5f, 0x72, 0x65, 0x73, 0x69, 0x64, 0x75, 0x61, 0x6c, + 0x5f, 0x65, 0x6c, 0x65, 0x6d, 0x65, 0x6e, 0x74, 0x18, 0x03, 0x20, 0x01, 0x28, 0x03, 0x52, 0x14, + 0x66, 0x69, 0x72, 0x73, 0x74, 0x52, 0x65, 0x73, 0x69, 0x64, 0x75, 0x61, 0x6c, 0x45, 0x6c, 0x65, + 0x6d, 0x65, 0x6e, 0x74, 0x22, 0x3e, 0x0a, 0x15, 0x46, 0x69, 0x6e, 0x61, 0x6c, 0x69, 0x7a, 0x65, + 0x42, 0x75, 0x6e, 0x64, 0x6c, 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x25, 0x0a, + 0x0e, 0x69, 0x6e, 0x73, 0x74, 0x72, 0x75, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x69, 0x64, 0x18, + 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0d, 0x69, 0x6e, 0x73, 0x74, 0x72, 0x75, 0x63, 0x74, 0x69, + 0x6f, 0x6e, 0x49, 0x64, 0x22, 0x18, 0x0a, 0x16, 0x46, 0x69, 0x6e, 0x61, 0x6c, 0x69, 0x7a, 0x65, + 0x42, 0x75, 0x6e, 0x64, 0x6c, 0x65, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0xd6, + 0x03, 0x0a, 0x08, 0x45, 0x6c, 0x65, 0x6d, 0x65, 0x6e, 0x74, 0x73, 0x12, 0x48, 0x0a, 0x04, 0x64, + 0x61, 0x74, 0x61, 0x18, 0x01, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x34, 0x2e, 0x6f, 0x72, 0x67, 0x2e, + 0x61, 0x70, 0x61, 0x63, 0x68, 0x65, 0x2e, 0x62, 0x65, 0x61, 0x6d, 0x2e, 0x6d, 0x6f, 0x64, 0x65, + 0x6c, 0x2e, 0x66, 0x6e, 0x5f, 0x65, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x2e, 0x76, + 0x31, 0x2e, 0x45, 0x6c, 0x65, 0x6d, 0x65, 0x6e, 0x74, 0x73, 0x2e, 0x44, 0x61, 0x74, 0x61, 0x52, + 0x04, 0x64, 0x61, 0x74, 0x61, 0x12, 0x4e, 0x0a, 0x06, 0x74, 0x69, 0x6d, 0x65, 0x72, 0x73, 0x18, + 0x02, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x36, 0x2e, 0x6f, 0x72, 0x67, 0x2e, 0x61, 0x70, 0x61, 0x63, + 0x68, 0x65, 0x2e, 0x62, 0x65, 0x61, 0x6d, 0x2e, 0x6d, 0x6f, 0x64, 0x65, 0x6c, 0x2e, 0x66, 0x6e, + 0x5f, 0x65, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x2e, 0x76, 0x31, 0x2e, 0x45, 0x6c, + 0x65, 0x6d, 0x65, 0x6e, 0x74, 0x73, 0x2e, 0x54, 0x69, 0x6d, 0x65, 0x72, 0x73, 0x52, 0x06, 0x74, + 0x69, 0x6d, 0x65, 0x72, 0x73, 0x1a, 0x81, 0x01, 0x0a, 0x04, 0x44, 0x61, 0x74, 0x61, 0x12, 0x25, + 0x0a, 0x0e, 0x69, 0x6e, 0x73, 0x74, 0x72, 0x75, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x69, 0x64, + 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0d, 0x69, 0x6e, 0x73, 0x74, 0x72, 0x75, 0x63, 0x74, + 0x69, 0x6f, 0x6e, 0x49, 0x64, 0x12, 0x21, 0x0a, 0x0c, 0x74, 0x72, 0x61, 0x6e, 0x73, 0x66, 0x6f, + 0x72, 0x6d, 0x5f, 0x69, 0x64, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0b, 0x74, 0x72, 0x61, + 0x6e, 0x73, 0x66, 0x6f, 0x72, 0x6d, 0x49, 0x64, 0x12, 0x16, 0x0a, 0x04, 0x64, 0x61, 0x74, 0x61, + 0x18, 0x03, 0x20, 0x01, 0x28, 0x0c, 0x42, 0x02, 0x08, 0x01, 0x52, 0x04, 0x64, 0x61, 0x74, 0x61, + 0x12, 0x17, 0x0a, 0x07, 0x69, 0x73, 0x5f, 0x6c, 0x61, 0x73, 0x74, 0x18, 0x04, 0x20, 0x01, 0x28, + 0x08, 0x52, 0x06, 0x69, 0x73, 0x4c, 0x61, 0x73, 0x74, 0x1a, 0xab, 0x01, 0x0a, 0x06, 0x54, 0x69, + 0x6d, 0x65, 0x72, 0x73, 0x12, 0x25, 0x0a, 0x0e, 0x69, 0x6e, 0x73, 0x74, 0x72, 0x75, 0x63, 0x74, + 0x69, 0x6f, 0x6e, 0x5f, 0x69, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0d, 0x69, 0x6e, + 0x73, 0x74, 0x72, 0x75, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x64, 0x12, 0x21, 0x0a, 0x0c, 0x74, + 0x72, 0x61, 0x6e, 0x73, 0x66, 0x6f, 0x72, 0x6d, 0x5f, 0x69, 0x64, 0x18, 0x02, 0x20, 0x01, 0x28, + 0x09, 0x52, 0x0b, 0x74, 0x72, 0x61, 0x6e, 0x73, 0x66, 0x6f, 0x72, 0x6d, 0x49, 0x64, 0x12, 0x26, + 0x0a, 0x0f, 0x74, 0x69, 0x6d, 0x65, 0x72, 0x5f, 0x66, 0x61, 0x6d, 0x69, 0x6c, 0x79, 0x5f, 0x69, + 0x64, 0x18, 0x03, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0d, 0x74, 0x69, 0x6d, 0x65, 0x72, 0x46, 0x61, + 0x6d, 0x69, 0x6c, 0x79, 0x49, 0x64, 0x12, 0x16, 0x0a, 0x06, 0x74, 0x69, 0x6d, 0x65, 0x72, 0x73, + 0x18, 0x04, 0x20, 0x01, 0x28, 0x0c, 0x52, 0x06, 0x74, 0x69, 0x6d, 0x65, 0x72, 0x73, 0x12, 0x17, + 0x0a, 0x07, 0x69, 0x73, 0x5f, 0x6c, 0x61, 0x73, 0x74, 0x18, 0x05, 0x20, 0x01, 0x28, 0x08, 0x52, + 0x06, 0x69, 0x73, 0x4c, 0x61, 0x73, 0x74, 0x22, 0x94, 0x03, 0x0a, 0x0c, 0x53, 0x74, 0x61, 0x74, + 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x0e, 0x0a, 0x02, 0x69, 0x64, 0x18, 0x01, + 0x20, 0x01, 0x28, 0x09, 0x52, 0x02, 0x69, 0x64, 0x12, 0x25, 0x0a, 0x0e, 0x69, 0x6e, 0x73, 0x74, + 0x72, 0x75, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x69, 0x64, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, + 0x52, 0x0d, 0x69, 0x6e, 0x73, 0x74, 0x72, 0x75, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x64, 0x12, + 0x4c, 0x0a, 0x09, 0x73, 0x74, 0x61, 0x74, 0x65, 0x5f, 0x6b, 0x65, 0x79, 0x18, 0x03, 0x20, 0x01, + 0x28, 0x0b, 0x32, 0x2f, 0x2e, 0x6f, 0x72, 0x67, 0x2e, 0x61, 0x70, 0x61, 0x63, 0x68, 0x65, 0x2e, + 0x62, 0x65, 0x61, 0x6d, 0x2e, 0x6d, 0x6f, 0x64, 0x65, 0x6c, 0x2e, 0x66, 0x6e, 0x5f, 0x65, 0x78, + 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x2e, 0x76, 0x31, 0x2e, 0x53, 0x74, 0x61, 0x74, 0x65, + 0x4b, 0x65, 0x79, 0x52, 0x08, 0x73, 0x74, 0x61, 0x74, 0x65, 0x4b, 0x65, 0x79, 0x12, 0x4b, 0x0a, + 0x03, 0x67, 0x65, 0x74, 0x18, 0xe8, 0x07, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x36, 0x2e, 0x6f, 0x72, + 0x67, 0x2e, 0x61, 0x70, 0x61, 0x63, 0x68, 0x65, 0x2e, 0x62, 0x65, 0x61, 0x6d, 0x2e, 0x6d, 0x6f, + 0x64, 0x65, 0x6c, 0x2e, 0x66, 0x6e, 0x5f, 0x65, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, + 0x2e, 0x76, 0x31, 0x2e, 0x53, 0x74, 0x61, 0x74, 0x65, 0x47, 0x65, 0x74, 0x52, 0x65, 0x71, 0x75, + 0x65, 0x73, 0x74, 0x48, 0x00, 0x52, 0x03, 0x67, 0x65, 0x74, 0x12, 0x54, 0x0a, 0x06, 0x61, 0x70, + 0x70, 0x65, 0x6e, 0x64, 0x18, 0xe9, 0x07, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x39, 0x2e, 0x6f, 0x72, + 0x67, 0x2e, 0x61, 0x70, 0x61, 0x63, 0x68, 0x65, 0x2e, 0x62, 0x65, 0x61, 0x6d, 0x2e, 0x6d, 0x6f, + 0x64, 0x65, 0x6c, 0x2e, 0x66, 0x6e, 0x5f, 0x65, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, + 0x2e, 0x76, 0x31, 0x2e, 0x53, 0x74, 0x61, 0x74, 0x65, 0x41, 0x70, 0x70, 0x65, 0x6e, 0x64, 0x52, + 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x48, 0x00, 0x52, 0x06, 0x61, 0x70, 0x70, 0x65, 0x6e, 0x64, + 0x12, 0x51, 0x0a, 0x05, 0x63, 0x6c, 0x65, 0x61, 0x72, 0x18, 0xea, 0x07, 0x20, 0x01, 0x28, 0x0b, + 0x32, 0x38, 0x2e, 0x6f, 0x72, 0x67, 0x2e, 0x61, 0x70, 0x61, 0x63, 0x68, 0x65, 0x2e, 0x62, 0x65, + 0x61, 0x6d, 0x2e, 0x6d, 0x6f, 0x64, 0x65, 0x6c, 0x2e, 0x66, 0x6e, 0x5f, 0x65, 0x78, 0x65, 0x63, + 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x2e, 0x76, 0x31, 0x2e, 0x53, 0x74, 0x61, 0x74, 0x65, 0x43, 0x6c, + 0x65, 0x61, 0x72, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x48, 0x00, 0x52, 0x05, 0x63, 0x6c, + 0x65, 0x61, 0x72, 0x42, 0x09, 0x0a, 0x07, 0x72, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x22, 0xba, + 0x02, 0x0a, 0x0d, 0x53, 0x74, 0x61, 0x74, 0x65, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, + 0x12, 0x0e, 0x0a, 0x02, 0x69, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x02, 0x69, 0x64, + 0x12, 0x14, 0x0a, 0x05, 0x65, 0x72, 0x72, 0x6f, 0x72, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, + 0x05, 0x65, 0x72, 0x72, 0x6f, 0x72, 0x12, 0x4c, 0x0a, 0x03, 0x67, 0x65, 0x74, 0x18, 0xe8, 0x07, + 0x20, 0x01, 0x28, 0x0b, 0x32, 0x37, 0x2e, 0x6f, 0x72, 0x67, 0x2e, 0x61, 0x70, 0x61, 0x63, 0x68, + 0x65, 0x2e, 0x62, 0x65, 0x61, 0x6d, 0x2e, 0x6d, 0x6f, 0x64, 0x65, 0x6c, 0x2e, 0x66, 0x6e, 0x5f, + 0x65, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x2e, 0x76, 0x31, 0x2e, 0x53, 0x74, 0x61, + 0x74, 0x65, 0x47, 0x65, 0x74, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x48, 0x00, 0x52, + 0x03, 0x67, 0x65, 0x74, 0x12, 0x55, 0x0a, 0x06, 0x61, 0x70, 0x70, 0x65, 0x6e, 0x64, 0x18, 0xe9, + 0x07, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x3a, 0x2e, 0x6f, 0x72, 0x67, 0x2e, 0x61, 0x70, 0x61, 0x63, + 0x68, 0x65, 0x2e, 0x62, 0x65, 0x61, 0x6d, 0x2e, 0x6d, 0x6f, 0x64, 0x65, 0x6c, 0x2e, 0x66, 0x6e, + 0x5f, 0x65, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x2e, 0x76, 0x31, 0x2e, 0x53, 0x74, + 0x61, 0x74, 0x65, 0x41, 0x70, 0x70, 0x65, 0x6e, 0x64, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, + 0x65, 0x48, 0x00, 0x52, 0x06, 0x61, 0x70, 0x70, 0x65, 0x6e, 0x64, 0x12, 0x52, 0x0a, 0x05, 0x63, + 0x6c, 0x65, 0x61, 0x72, 0x18, 0xea, 0x07, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x39, 0x2e, 0x6f, 0x72, + 0x67, 0x2e, 0x61, 0x70, 0x61, 0x63, 0x68, 0x65, 0x2e, 0x62, 0x65, 0x61, 0x6d, 0x2e, 0x6d, 0x6f, + 0x64, 0x65, 0x6c, 0x2e, 0x66, 0x6e, 0x5f, 0x65, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, + 0x2e, 0x76, 0x31, 0x2e, 0x53, 0x74, 0x61, 0x74, 0x65, 0x43, 0x6c, 0x65, 0x61, 0x72, 0x52, 0x65, + 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x48, 0x00, 0x52, 0x05, 0x63, 0x6c, 0x65, 0x61, 0x72, 0x42, + 0x0a, 0x0a, 0x08, 0x72, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0xd6, 0x11, 0x0a, 0x08, + 0x53, 0x74, 0x61, 0x74, 0x65, 0x4b, 0x65, 0x79, 0x12, 0x50, 0x0a, 0x06, 0x72, 0x75, 0x6e, 0x6e, + 0x65, 0x72, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x36, 0x2e, 0x6f, 0x72, 0x67, 0x2e, 0x61, + 0x70, 0x61, 0x63, 0x68, 0x65, 0x2e, 0x62, 0x65, 0x61, 0x6d, 0x2e, 0x6d, 0x6f, 0x64, 0x65, 0x6c, + 0x2e, 0x66, 0x6e, 0x5f, 0x65, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x2e, 0x76, 0x31, + 0x2e, 0x53, 0x74, 0x61, 0x74, 0x65, 0x4b, 0x65, 0x79, 0x2e, 0x52, 0x75, 0x6e, 0x6e, 0x65, 0x72, + 0x48, 0x00, 0x52, 0x06, 0x72, 0x75, 0x6e, 0x6e, 0x65, 0x72, 0x12, 0x73, 0x0a, 0x13, 0x6d, 0x75, + 0x6c, 0x74, 0x69, 0x6d, 0x61, 0x70, 0x5f, 0x73, 0x69, 0x64, 0x65, 0x5f, 0x69, 0x6e, 0x70, 0x75, + 0x74, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x41, 0x2e, 0x6f, 0x72, 0x67, 0x2e, 0x61, 0x70, + 0x61, 0x63, 0x68, 0x65, 0x2e, 0x62, 0x65, 0x61, 0x6d, 0x2e, 0x6d, 0x6f, 0x64, 0x65, 0x6c, 0x2e, + 0x66, 0x6e, 0x5f, 0x65, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x2e, 0x76, 0x31, 0x2e, + 0x53, 0x74, 0x61, 0x74, 0x65, 0x4b, 0x65, 0x79, 0x2e, 0x4d, 0x75, 0x6c, 0x74, 0x69, 0x6d, 0x61, + 0x70, 0x53, 0x69, 0x64, 0x65, 0x49, 0x6e, 0x70, 0x75, 0x74, 0x48, 0x00, 0x52, 0x11, 0x6d, 0x75, + 0x6c, 0x74, 0x69, 0x6d, 0x61, 0x70, 0x53, 0x69, 0x64, 0x65, 0x49, 0x6e, 0x70, 0x75, 0x74, 0x12, + 0x64, 0x0a, 0x0e, 0x62, 0x61, 0x67, 0x5f, 0x75, 0x73, 0x65, 0x72, 0x5f, 0x73, 0x74, 0x61, 0x74, + 0x65, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x3c, 0x2e, 0x6f, 0x72, 0x67, 0x2e, 0x61, 0x70, + 0x61, 0x63, 0x68, 0x65, 0x2e, 0x62, 0x65, 0x61, 0x6d, 0x2e, 0x6d, 0x6f, 0x64, 0x65, 0x6c, 0x2e, + 0x66, 0x6e, 0x5f, 0x65, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x2e, 0x76, 0x31, 0x2e, + 0x53, 0x74, 0x61, 0x74, 0x65, 0x4b, 0x65, 0x79, 0x2e, 0x42, 0x61, 0x67, 0x55, 0x73, 0x65, 0x72, + 0x53, 0x74, 0x61, 0x74, 0x65, 0x48, 0x00, 0x52, 0x0c, 0x62, 0x61, 0x67, 0x55, 0x73, 0x65, 0x72, + 0x53, 0x74, 0x61, 0x74, 0x65, 0x12, 0x73, 0x0a, 0x13, 0x69, 0x74, 0x65, 0x72, 0x61, 0x62, 0x6c, + 0x65, 0x5f, 0x73, 0x69, 0x64, 0x65, 0x5f, 0x69, 0x6e, 0x70, 0x75, 0x74, 0x18, 0x04, 0x20, 0x01, + 0x28, 0x0b, 0x32, 0x41, 0x2e, 0x6f, 0x72, 0x67, 0x2e, 0x61, 0x70, 0x61, 0x63, 0x68, 0x65, 0x2e, + 0x62, 0x65, 0x61, 0x6d, 0x2e, 0x6d, 0x6f, 0x64, 0x65, 0x6c, 0x2e, 0x66, 0x6e, 0x5f, 0x65, 0x78, + 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x2e, 0x76, 0x31, 0x2e, 0x53, 0x74, 0x61, 0x74, 0x65, + 0x4b, 0x65, 0x79, 0x2e, 0x49, 0x74, 0x65, 0x72, 0x61, 0x62, 0x6c, 0x65, 0x53, 0x69, 0x64, 0x65, + 0x49, 0x6e, 0x70, 0x75, 0x74, 0x48, 0x00, 0x52, 0x11, 0x69, 0x74, 0x65, 0x72, 0x61, 0x62, 0x6c, + 0x65, 0x53, 0x69, 0x64, 0x65, 0x49, 0x6e, 0x70, 0x75, 0x74, 0x12, 0x80, 0x01, 0x0a, 0x18, 0x6d, + 0x75, 0x6c, 0x74, 0x69, 0x6d, 0x61, 0x70, 0x5f, 0x6b, 0x65, 0x79, 0x73, 0x5f, 0x73, 0x69, 0x64, + 0x65, 0x5f, 0x69, 0x6e, 0x70, 0x75, 0x74, 0x18, 0x05, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x45, 0x2e, + 0x6f, 0x72, 0x67, 0x2e, 0x61, 0x70, 0x61, 0x63, 0x68, 0x65, 0x2e, 0x62, 0x65, 0x61, 0x6d, 0x2e, + 0x6d, 0x6f, 0x64, 0x65, 0x6c, 0x2e, 0x66, 0x6e, 0x5f, 0x65, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, + 0x6f, 0x6e, 0x2e, 0x76, 0x31, 0x2e, 0x53, 0x74, 0x61, 0x74, 0x65, 0x4b, 0x65, 0x79, 0x2e, 0x4d, + 0x75, 0x6c, 0x74, 0x69, 0x6d, 0x61, 0x70, 0x4b, 0x65, 0x79, 0x73, 0x53, 0x69, 0x64, 0x65, 0x49, + 0x6e, 0x70, 0x75, 0x74, 0x48, 0x00, 0x52, 0x15, 0x6d, 0x75, 0x6c, 0x74, 0x69, 0x6d, 0x61, 0x70, + 0x4b, 0x65, 0x79, 0x73, 0x53, 0x69, 0x64, 0x65, 0x49, 0x6e, 0x70, 0x75, 0x74, 0x12, 0x93, 0x01, + 0x0a, 0x1f, 0x6d, 0x75, 0x6c, 0x74, 0x69, 0x6d, 0x61, 0x70, 0x5f, 0x6b, 0x65, 0x79, 0x73, 0x5f, + 0x76, 0x61, 0x6c, 0x75, 0x65, 0x73, 0x5f, 0x73, 0x69, 0x64, 0x65, 0x5f, 0x69, 0x6e, 0x70, 0x75, + 0x74, 0x18, 0x08, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x4b, 0x2e, 0x6f, 0x72, 0x67, 0x2e, 0x61, 0x70, + 0x61, 0x63, 0x68, 0x65, 0x2e, 0x62, 0x65, 0x61, 0x6d, 0x2e, 0x6d, 0x6f, 0x64, 0x65, 0x6c, 0x2e, + 0x66, 0x6e, 0x5f, 0x65, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x2e, 0x76, 0x31, 0x2e, + 0x53, 0x74, 0x61, 0x74, 0x65, 0x4b, 0x65, 0x79, 0x2e, 0x4d, 0x75, 0x6c, 0x74, 0x69, 0x6d, 0x61, + 0x70, 0x4b, 0x65, 0x79, 0x73, 0x56, 0x61, 0x6c, 0x75, 0x65, 0x73, 0x53, 0x69, 0x64, 0x65, 0x49, + 0x6e, 0x70, 0x75, 0x74, 0x48, 0x00, 0x52, 0x1b, 0x6d, 0x75, 0x6c, 0x74, 0x69, 0x6d, 0x61, 0x70, + 0x4b, 0x65, 0x79, 0x73, 0x56, 0x61, 0x6c, 0x75, 0x65, 0x73, 0x53, 0x69, 0x64, 0x65, 0x49, 0x6e, + 0x70, 0x75, 0x74, 0x12, 0x80, 0x01, 0x0a, 0x18, 0x6d, 0x75, 0x6c, 0x74, 0x69, 0x6d, 0x61, 0x70, + 0x5f, 0x6b, 0x65, 0x79, 0x73, 0x5f, 0x75, 0x73, 0x65, 0x72, 0x5f, 0x73, 0x74, 0x61, 0x74, 0x65, + 0x18, 0x06, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x45, 0x2e, 0x6f, 0x72, 0x67, 0x2e, 0x61, 0x70, 0x61, + 0x63, 0x68, 0x65, 0x2e, 0x62, 0x65, 0x61, 0x6d, 0x2e, 0x6d, 0x6f, 0x64, 0x65, 0x6c, 0x2e, 0x66, + 0x6e, 0x5f, 0x65, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x2e, 0x76, 0x31, 0x2e, 0x53, + 0x74, 0x61, 0x74, 0x65, 0x4b, 0x65, 0x79, 0x2e, 0x4d, 0x75, 0x6c, 0x74, 0x69, 0x6d, 0x61, 0x70, + 0x4b, 0x65, 0x79, 0x73, 0x55, 0x73, 0x65, 0x72, 0x53, 0x74, 0x61, 0x74, 0x65, 0x48, 0x00, 0x52, + 0x15, 0x6d, 0x75, 0x6c, 0x74, 0x69, 0x6d, 0x61, 0x70, 0x4b, 0x65, 0x79, 0x73, 0x55, 0x73, 0x65, + 0x72, 0x53, 0x74, 0x61, 0x74, 0x65, 0x12, 0x73, 0x0a, 0x13, 0x6d, 0x75, 0x6c, 0x74, 0x69, 0x6d, + 0x61, 0x70, 0x5f, 0x75, 0x73, 0x65, 0x72, 0x5f, 0x73, 0x74, 0x61, 0x74, 0x65, 0x18, 0x07, 0x20, + 0x01, 0x28, 0x0b, 0x32, 0x41, 0x2e, 0x6f, 0x72, 0x67, 0x2e, 0x61, 0x70, 0x61, 0x63, 0x68, 0x65, + 0x2e, 0x62, 0x65, 0x61, 0x6d, 0x2e, 0x6d, 0x6f, 0x64, 0x65, 0x6c, 0x2e, 0x66, 0x6e, 0x5f, 0x65, + 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x2e, 0x76, 0x31, 0x2e, 0x53, 0x74, 0x61, 0x74, + 0x65, 0x4b, 0x65, 0x79, 0x2e, 0x4d, 0x75, 0x6c, 0x74, 0x69, 0x6d, 0x61, 0x70, 0x55, 0x73, 0x65, + 0x72, 0x53, 0x74, 0x61, 0x74, 0x65, 0x48, 0x00, 0x52, 0x11, 0x6d, 0x75, 0x6c, 0x74, 0x69, 0x6d, + 0x61, 0x70, 0x55, 0x73, 0x65, 0x72, 0x53, 0x74, 0x61, 0x74, 0x65, 0x12, 0x7d, 0x0a, 0x17, 0x6f, + 0x72, 0x64, 0x65, 0x72, 0x65, 0x64, 0x5f, 0x6c, 0x69, 0x73, 0x74, 0x5f, 0x75, 0x73, 0x65, 0x72, + 0x5f, 0x73, 0x74, 0x61, 0x74, 0x65, 0x18, 0x09, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x44, 0x2e, 0x6f, + 0x72, 0x67, 0x2e, 0x61, 0x70, 0x61, 0x63, 0x68, 0x65, 0x2e, 0x62, 0x65, 0x61, 0x6d, 0x2e, 0x6d, + 0x6f, 0x64, 0x65, 0x6c, 0x2e, 0x66, 0x6e, 0x5f, 0x65, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, + 0x6e, 0x2e, 0x76, 0x31, 0x2e, 0x53, 0x74, 0x61, 0x74, 0x65, 0x4b, 0x65, 0x79, 0x2e, 0x4f, 0x72, + 0x64, 0x65, 0x72, 0x65, 0x64, 0x4c, 0x69, 0x73, 0x74, 0x55, 0x73, 0x65, 0x72, 0x53, 0x74, 0x61, + 0x74, 0x65, 0x48, 0x00, 0x52, 0x14, 0x6f, 0x72, 0x64, 0x65, 0x72, 0x65, 0x64, 0x4c, 0x69, 0x73, + 0x74, 0x55, 0x73, 0x65, 0x72, 0x53, 0x74, 0x61, 0x74, 0x65, 0x1a, 0x1a, 0x0a, 0x06, 0x52, 0x75, + 0x6e, 0x6e, 0x65, 0x72, 0x12, 0x10, 0x0a, 0x03, 0x6b, 0x65, 0x79, 0x18, 0x01, 0x20, 0x01, 0x28, + 0x0c, 0x52, 0x03, 0x6b, 0x65, 0x79, 0x1a, 0x72, 0x0a, 0x11, 0x49, 0x74, 0x65, 0x72, 0x61, 0x62, + 0x6c, 0x65, 0x53, 0x69, 0x64, 0x65, 0x49, 0x6e, 0x70, 0x75, 0x74, 0x12, 0x21, 0x0a, 0x0c, 0x74, + 0x72, 0x61, 0x6e, 0x73, 0x66, 0x6f, 0x72, 0x6d, 0x5f, 0x69, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, + 0x09, 0x52, 0x0b, 0x74, 0x72, 0x61, 0x6e, 0x73, 0x66, 0x6f, 0x72, 0x6d, 0x49, 0x64, 0x12, 0x22, + 0x0a, 0x0d, 0x73, 0x69, 0x64, 0x65, 0x5f, 0x69, 0x6e, 0x70, 0x75, 0x74, 0x5f, 0x69, 0x64, 0x18, + 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0b, 0x73, 0x69, 0x64, 0x65, 0x49, 0x6e, 0x70, 0x75, 0x74, + 0x49, 0x64, 0x12, 0x16, 0x0a, 0x06, 0x77, 0x69, 0x6e, 0x64, 0x6f, 0x77, 0x18, 0x03, 0x20, 0x01, + 0x28, 0x0c, 0x52, 0x06, 0x77, 0x69, 0x6e, 0x64, 0x6f, 0x77, 0x1a, 0x84, 0x01, 0x0a, 0x11, 0x4d, + 0x75, 0x6c, 0x74, 0x69, 0x6d, 0x61, 0x70, 0x53, 0x69, 0x64, 0x65, 0x49, 0x6e, 0x70, 0x75, 0x74, + 0x12, 0x21, 0x0a, 0x0c, 0x74, 0x72, 0x61, 0x6e, 0x73, 0x66, 0x6f, 0x72, 0x6d, 0x5f, 0x69, 0x64, + 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0b, 0x74, 0x72, 0x61, 0x6e, 0x73, 0x66, 0x6f, 0x72, + 0x6d, 0x49, 0x64, 0x12, 0x22, 0x0a, 0x0d, 0x73, 0x69, 0x64, 0x65, 0x5f, 0x69, 0x6e, 0x70, 0x75, + 0x74, 0x5f, 0x69, 0x64, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0b, 0x73, 0x69, 0x64, 0x65, + 0x49, 0x6e, 0x70, 0x75, 0x74, 0x49, 0x64, 0x12, 0x16, 0x0a, 0x06, 0x77, 0x69, 0x6e, 0x64, 0x6f, + 0x77, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0c, 0x52, 0x06, 0x77, 0x69, 0x6e, 0x64, 0x6f, 0x77, 0x12, + 0x10, 0x0a, 0x03, 0x6b, 0x65, 0x79, 0x18, 0x04, 0x20, 0x01, 0x28, 0x0c, 0x52, 0x03, 0x6b, 0x65, + 0x79, 0x1a, 0x76, 0x0a, 0x15, 0x4d, 0x75, 0x6c, 0x74, 0x69, 0x6d, 0x61, 0x70, 0x4b, 0x65, 0x79, + 0x73, 0x53, 0x69, 0x64, 0x65, 0x49, 0x6e, 0x70, 0x75, 0x74, 0x12, 0x21, 0x0a, 0x0c, 0x74, 0x72, + 0x61, 0x6e, 0x73, 0x66, 0x6f, 0x72, 0x6d, 0x5f, 0x69, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, + 0x52, 0x0b, 0x74, 0x72, 0x61, 0x6e, 0x73, 0x66, 0x6f, 0x72, 0x6d, 0x49, 0x64, 0x12, 0x22, 0x0a, + 0x0d, 0x73, 0x69, 0x64, 0x65, 0x5f, 0x69, 0x6e, 0x70, 0x75, 0x74, 0x5f, 0x69, 0x64, 0x18, 0x02, + 0x20, 0x01, 0x28, 0x09, 0x52, 0x0b, 0x73, 0x69, 0x64, 0x65, 0x49, 0x6e, 0x70, 0x75, 0x74, 0x49, + 0x64, 0x12, 0x16, 0x0a, 0x06, 0x77, 0x69, 0x6e, 0x64, 0x6f, 0x77, 0x18, 0x03, 0x20, 0x01, 0x28, + 0x0c, 0x52, 0x06, 0x77, 0x69, 0x6e, 0x64, 0x6f, 0x77, 0x1a, 0x7c, 0x0a, 0x1b, 0x4d, 0x75, 0x6c, + 0x74, 0x69, 0x6d, 0x61, 0x70, 0x4b, 0x65, 0x79, 0x73, 0x56, 0x61, 0x6c, 0x75, 0x65, 0x73, 0x53, + 0x69, 0x64, 0x65, 0x49, 0x6e, 0x70, 0x75, 0x74, 0x12, 0x21, 0x0a, 0x0c, 0x74, 0x72, 0x61, 0x6e, + 0x73, 0x66, 0x6f, 0x72, 0x6d, 0x5f, 0x69, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0b, + 0x74, 0x72, 0x61, 0x6e, 0x73, 0x66, 0x6f, 0x72, 0x6d, 0x49, 0x64, 0x12, 0x22, 0x0a, 0x0d, 0x73, + 0x69, 0x64, 0x65, 0x5f, 0x69, 0x6e, 0x70, 0x75, 0x74, 0x5f, 0x69, 0x64, 0x18, 0x02, 0x20, 0x01, + 0x28, 0x09, 0x52, 0x0b, 0x73, 0x69, 0x64, 0x65, 0x49, 0x6e, 0x70, 0x75, 0x74, 0x49, 0x64, 0x12, + 0x16, 0x0a, 0x06, 0x77, 0x69, 0x6e, 0x64, 0x6f, 0x77, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0c, 0x52, + 0x06, 0x77, 0x69, 0x6e, 0x64, 0x6f, 0x77, 0x1a, 0x7f, 0x0a, 0x0c, 0x42, 0x61, 0x67, 0x55, 0x73, + 0x65, 0x72, 0x53, 0x74, 0x61, 0x74, 0x65, 0x12, 0x21, 0x0a, 0x0c, 0x74, 0x72, 0x61, 0x6e, 0x73, + 0x66, 0x6f, 0x72, 0x6d, 0x5f, 0x69, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0b, 0x74, + 0x72, 0x61, 0x6e, 0x73, 0x66, 0x6f, 0x72, 0x6d, 0x49, 0x64, 0x12, 0x22, 0x0a, 0x0d, 0x75, 0x73, + 0x65, 0x72, 0x5f, 0x73, 0x74, 0x61, 0x74, 0x65, 0x5f, 0x69, 0x64, 0x18, 0x02, 0x20, 0x01, 0x28, + 0x09, 0x52, 0x0b, 0x75, 0x73, 0x65, 0x72, 0x53, 0x74, 0x61, 0x74, 0x65, 0x49, 0x64, 0x12, 0x16, + 0x0a, 0x06, 0x77, 0x69, 0x6e, 0x64, 0x6f, 0x77, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0c, 0x52, 0x06, + 0x77, 0x69, 0x6e, 0x64, 0x6f, 0x77, 0x12, 0x10, 0x0a, 0x03, 0x6b, 0x65, 0x79, 0x18, 0x04, 0x20, + 0x01, 0x28, 0x0c, 0x52, 0x03, 0x6b, 0x65, 0x79, 0x1a, 0x88, 0x01, 0x0a, 0x15, 0x4d, 0x75, 0x6c, + 0x74, 0x69, 0x6d, 0x61, 0x70, 0x4b, 0x65, 0x79, 0x73, 0x55, 0x73, 0x65, 0x72, 0x53, 0x74, 0x61, + 0x74, 0x65, 0x12, 0x21, 0x0a, 0x0c, 0x74, 0x72, 0x61, 0x6e, 0x73, 0x66, 0x6f, 0x72, 0x6d, 0x5f, + 0x69, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0b, 0x74, 0x72, 0x61, 0x6e, 0x73, 0x66, + 0x6f, 0x72, 0x6d, 0x49, 0x64, 0x12, 0x22, 0x0a, 0x0d, 0x75, 0x73, 0x65, 0x72, 0x5f, 0x73, 0x74, + 0x61, 0x74, 0x65, 0x5f, 0x69, 0x64, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0b, 0x75, 0x73, + 0x65, 0x72, 0x53, 0x74, 0x61, 0x74, 0x65, 0x49, 0x64, 0x12, 0x16, 0x0a, 0x06, 0x77, 0x69, 0x6e, + 0x64, 0x6f, 0x77, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0c, 0x52, 0x06, 0x77, 0x69, 0x6e, 0x64, 0x6f, + 0x77, 0x12, 0x10, 0x0a, 0x03, 0x6b, 0x65, 0x79, 0x18, 0x04, 0x20, 0x01, 0x28, 0x0c, 0x52, 0x03, + 0x6b, 0x65, 0x79, 0x1a, 0x9d, 0x01, 0x0a, 0x11, 0x4d, 0x75, 0x6c, 0x74, 0x69, 0x6d, 0x61, 0x70, + 0x55, 0x73, 0x65, 0x72, 0x53, 0x74, 0x61, 0x74, 0x65, 0x12, 0x21, 0x0a, 0x0c, 0x74, 0x72, 0x61, + 0x6e, 0x73, 0x66, 0x6f, 0x72, 0x6d, 0x5f, 0x69, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, + 0x0b, 0x74, 0x72, 0x61, 0x6e, 0x73, 0x66, 0x6f, 0x72, 0x6d, 0x49, 0x64, 0x12, 0x22, 0x0a, 0x0d, + 0x75, 0x73, 0x65, 0x72, 0x5f, 0x73, 0x74, 0x61, 0x74, 0x65, 0x5f, 0x69, 0x64, 0x18, 0x02, 0x20, + 0x01, 0x28, 0x09, 0x52, 0x0b, 0x75, 0x73, 0x65, 0x72, 0x53, 0x74, 0x61, 0x74, 0x65, 0x49, 0x64, + 0x12, 0x16, 0x0a, 0x06, 0x77, 0x69, 0x6e, 0x64, 0x6f, 0x77, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0c, + 0x52, 0x06, 0x77, 0x69, 0x6e, 0x64, 0x6f, 0x77, 0x12, 0x10, 0x0a, 0x03, 0x6b, 0x65, 0x79, 0x18, + 0x04, 0x20, 0x01, 0x28, 0x0c, 0x52, 0x03, 0x6b, 0x65, 0x79, 0x12, 0x17, 0x0a, 0x07, 0x6d, 0x61, + 0x70, 0x5f, 0x6b, 0x65, 0x79, 0x18, 0x05, 0x20, 0x01, 0x28, 0x0c, 0x52, 0x06, 0x6d, 0x61, 0x70, + 0x4b, 0x65, 0x79, 0x1a, 0xd6, 0x01, 0x0a, 0x14, 0x4f, 0x72, 0x64, 0x65, 0x72, 0x65, 0x64, 0x4c, + 0x69, 0x73, 0x74, 0x55, 0x73, 0x65, 0x72, 0x53, 0x74, 0x61, 0x74, 0x65, 0x12, 0x21, 0x0a, 0x0c, + 0x74, 0x72, 0x61, 0x6e, 0x73, 0x66, 0x6f, 0x72, 0x6d, 0x5f, 0x69, 0x64, 0x18, 0x01, 0x20, 0x01, + 0x28, 0x09, 0x52, 0x0b, 0x74, 0x72, 0x61, 0x6e, 0x73, 0x66, 0x6f, 0x72, 0x6d, 0x49, 0x64, 0x12, + 0x22, 0x0a, 0x0d, 0x75, 0x73, 0x65, 0x72, 0x5f, 0x73, 0x74, 0x61, 0x74, 0x65, 0x5f, 0x69, 0x64, + 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0b, 0x75, 0x73, 0x65, 0x72, 0x53, 0x74, 0x61, 0x74, + 0x65, 0x49, 0x64, 0x12, 0x16, 0x0a, 0x06, 0x77, 0x69, 0x6e, 0x64, 0x6f, 0x77, 0x18, 0x03, 0x20, + 0x01, 0x28, 0x0c, 0x52, 0x06, 0x77, 0x69, 0x6e, 0x64, 0x6f, 0x77, 0x12, 0x10, 0x0a, 0x03, 0x6b, + 0x65, 0x79, 0x18, 0x04, 0x20, 0x01, 0x28, 0x0c, 0x52, 0x03, 0x6b, 0x65, 0x79, 0x12, 0x4d, 0x0a, + 0x05, 0x72, 0x61, 0x6e, 0x67, 0x65, 0x18, 0x05, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x37, 0x2e, 0x6f, + 0x72, 0x67, 0x2e, 0x61, 0x70, 0x61, 0x63, 0x68, 0x65, 0x2e, 0x62, 0x65, 0x61, 0x6d, 0x2e, 0x6d, + 0x6f, 0x64, 0x65, 0x6c, 0x2e, 0x66, 0x6e, 0x5f, 0x65, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, + 0x6e, 0x2e, 0x76, 0x31, 0x2e, 0x4f, 0x72, 0x64, 0x65, 0x72, 0x65, 0x64, 0x4c, 0x69, 0x73, 0x74, + 0x52, 0x61, 0x6e, 0x67, 0x65, 0x52, 0x05, 0x72, 0x61, 0x6e, 0x67, 0x65, 0x42, 0x06, 0x0a, 0x04, + 0x74, 0x79, 0x70, 0x65, 0x22, 0x40, 0x0a, 0x0f, 0x53, 0x74, 0x61, 0x74, 0x65, 0x47, 0x65, 0x74, + 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x2d, 0x0a, 0x12, 0x63, 0x6f, 0x6e, 0x74, 0x69, + 0x6e, 0x75, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x74, 0x6f, 0x6b, 0x65, 0x6e, 0x18, 0x01, 0x20, + 0x01, 0x28, 0x0c, 0x52, 0x11, 0x63, 0x6f, 0x6e, 0x74, 0x69, 0x6e, 0x75, 0x61, 0x74, 0x69, 0x6f, + 0x6e, 0x54, 0x6f, 0x6b, 0x65, 0x6e, 0x22, 0x55, 0x0a, 0x10, 0x53, 0x74, 0x61, 0x74, 0x65, 0x47, + 0x65, 0x74, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x2d, 0x0a, 0x12, 0x63, 0x6f, + 0x6e, 0x74, 0x69, 0x6e, 0x75, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x74, 0x6f, 0x6b, 0x65, 0x6e, + 0x18, 0x01, 0x20, 0x01, 0x28, 0x0c, 0x52, 0x11, 0x63, 0x6f, 0x6e, 0x74, 0x69, 0x6e, 0x75, 0x61, + 0x74, 0x69, 0x6f, 0x6e, 0x54, 0x6f, 0x6b, 0x65, 0x6e, 0x12, 0x12, 0x0a, 0x04, 0x64, 0x61, 0x74, + 0x61, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0c, 0x52, 0x04, 0x64, 0x61, 0x74, 0x61, 0x22, 0x28, 0x0a, + 0x12, 0x53, 0x74, 0x61, 0x74, 0x65, 0x41, 0x70, 0x70, 0x65, 0x6e, 0x64, 0x52, 0x65, 0x71, 0x75, + 0x65, 0x73, 0x74, 0x12, 0x12, 0x0a, 0x04, 0x64, 0x61, 0x74, 0x61, 0x18, 0x01, 0x20, 0x01, 0x28, + 0x0c, 0x52, 0x04, 0x64, 0x61, 0x74, 0x61, 0x22, 0x15, 0x0a, 0x13, 0x53, 0x74, 0x61, 0x74, 0x65, + 0x41, 0x70, 0x70, 0x65, 0x6e, 0x64, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x13, + 0x0a, 0x11, 0x53, 0x74, 0x61, 0x74, 0x65, 0x43, 0x6c, 0x65, 0x61, 0x72, 0x52, 0x65, 0x71, 0x75, + 0x65, 0x73, 0x74, 0x22, 0x14, 0x0a, 0x12, 0x53, 0x74, 0x61, 0x74, 0x65, 0x43, 0x6c, 0x65, 0x61, + 0x72, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x3a, 0x0a, 0x10, 0x4f, 0x72, 0x64, + 0x65, 0x72, 0x65, 0x64, 0x4c, 0x69, 0x73, 0x74, 0x52, 0x61, 0x6e, 0x67, 0x65, 0x12, 0x14, 0x0a, + 0x05, 0x73, 0x74, 0x61, 0x72, 0x74, 0x18, 0x01, 0x20, 0x01, 0x28, 0x03, 0x52, 0x05, 0x73, 0x74, + 0x61, 0x72, 0x74, 0x12, 0x10, 0x0a, 0x03, 0x65, 0x6e, 0x64, 0x18, 0x02, 0x20, 0x01, 0x28, 0x03, + 0x52, 0x03, 0x65, 0x6e, 0x64, 0x22, 0xdc, 0x04, 0x0a, 0x08, 0x4c, 0x6f, 0x67, 0x45, 0x6e, 0x74, + 0x72, 0x79, 0x12, 0x59, 0x0a, 0x08, 0x73, 0x65, 0x76, 0x65, 0x72, 0x69, 0x74, 0x79, 0x18, 0x01, + 0x20, 0x01, 0x28, 0x0e, 0x32, 0x3d, 0x2e, 0x6f, 0x72, 0x67, 0x2e, 0x61, 0x70, 0x61, 0x63, 0x68, + 0x65, 0x2e, 0x62, 0x65, 0x61, 0x6d, 0x2e, 0x6d, 0x6f, 0x64, 0x65, 0x6c, 0x2e, 0x66, 0x6e, 0x5f, + 0x65, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x2e, 0x76, 0x31, 0x2e, 0x4c, 0x6f, 0x67, + 0x45, 0x6e, 0x74, 0x72, 0x79, 0x2e, 0x53, 0x65, 0x76, 0x65, 0x72, 0x69, 0x74, 0x79, 0x2e, 0x45, + 0x6e, 0x75, 0x6d, 0x52, 0x08, 0x73, 0x65, 0x76, 0x65, 0x72, 0x69, 0x74, 0x79, 0x12, 0x38, 0x0a, + 0x09, 0x74, 0x69, 0x6d, 0x65, 0x73, 0x74, 0x61, 0x6d, 0x70, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, + 0x32, 0x1a, 0x2e, 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, + 0x75, 0x66, 0x2e, 0x54, 0x69, 0x6d, 0x65, 0x73, 0x74, 0x61, 0x6d, 0x70, 0x52, 0x09, 0x74, 0x69, + 0x6d, 0x65, 0x73, 0x74, 0x61, 0x6d, 0x70, 0x12, 0x18, 0x0a, 0x07, 0x6d, 0x65, 0x73, 0x73, 0x61, + 0x67, 0x65, 0x18, 0x03, 0x20, 0x01, 0x28, 0x09, 0x52, 0x07, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, + 0x65, 0x12, 0x14, 0x0a, 0x05, 0x74, 0x72, 0x61, 0x63, 0x65, 0x18, 0x04, 0x20, 0x01, 0x28, 0x09, + 0x52, 0x05, 0x74, 0x72, 0x61, 0x63, 0x65, 0x12, 0x25, 0x0a, 0x0e, 0x69, 0x6e, 0x73, 0x74, 0x72, + 0x75, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x69, 0x64, 0x18, 0x05, 0x20, 0x01, 0x28, 0x09, 0x52, + 0x0d, 0x69, 0x6e, 0x73, 0x74, 0x72, 0x75, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x64, 0x12, 0x21, + 0x0a, 0x0c, 0x74, 0x72, 0x61, 0x6e, 0x73, 0x66, 0x6f, 0x72, 0x6d, 0x5f, 0x69, 0x64, 0x18, 0x06, + 0x20, 0x01, 0x28, 0x09, 0x52, 0x0b, 0x74, 0x72, 0x61, 0x6e, 0x73, 0x66, 0x6f, 0x72, 0x6d, 0x49, + 0x64, 0x12, 0x21, 0x0a, 0x0c, 0x6c, 0x6f, 0x67, 0x5f, 0x6c, 0x6f, 0x63, 0x61, 0x74, 0x69, 0x6f, + 0x6e, 0x18, 0x07, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0b, 0x6c, 0x6f, 0x67, 0x4c, 0x6f, 0x63, 0x61, + 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x16, 0x0a, 0x06, 0x74, 0x68, 0x72, 0x65, 0x61, 0x64, 0x18, 0x08, + 0x20, 0x01, 0x28, 0x09, 0x52, 0x06, 0x74, 0x68, 0x72, 0x65, 0x61, 0x64, 0x12, 0x38, 0x0a, 0x0b, + 0x63, 0x75, 0x73, 0x74, 0x6f, 0x6d, 0x5f, 0x64, 0x61, 0x74, 0x61, 0x18, 0x09, 0x20, 0x01, 0x28, + 0x0b, 0x32, 0x17, 0x2e, 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, + 0x62, 0x75, 0x66, 0x2e, 0x53, 0x74, 0x72, 0x75, 0x63, 0x74, 0x52, 0x0a, 0x63, 0x75, 0x73, 0x74, + 0x6f, 0x6d, 0x44, 0x61, 0x74, 0x61, 0x1a, 0x58, 0x0a, 0x04, 0x4c, 0x69, 0x73, 0x74, 0x12, 0x50, + 0x0a, 0x0b, 0x6c, 0x6f, 0x67, 0x5f, 0x65, 0x6e, 0x74, 0x72, 0x69, 0x65, 0x73, 0x18, 0x01, 0x20, + 0x03, 0x28, 0x0b, 0x32, 0x2f, 0x2e, 0x6f, 0x72, 0x67, 0x2e, 0x61, 0x70, 0x61, 0x63, 0x68, 0x65, + 0x2e, 0x62, 0x65, 0x61, 0x6d, 0x2e, 0x6d, 0x6f, 0x64, 0x65, 0x6c, 0x2e, 0x66, 0x6e, 0x5f, 0x65, + 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x2e, 0x76, 0x31, 0x2e, 0x4c, 0x6f, 0x67, 0x45, + 0x6e, 0x74, 0x72, 0x79, 0x52, 0x0a, 0x6c, 0x6f, 0x67, 0x45, 0x6e, 0x74, 0x72, 0x69, 0x65, 0x73, + 0x1a, 0x72, 0x0a, 0x08, 0x53, 0x65, 0x76, 0x65, 0x72, 0x69, 0x74, 0x79, 0x22, 0x66, 0x0a, 0x04, + 0x45, 0x6e, 0x75, 0x6d, 0x12, 0x0f, 0x0a, 0x0b, 0x55, 0x4e, 0x53, 0x50, 0x45, 0x43, 0x49, 0x46, + 0x49, 0x45, 0x44, 0x10, 0x00, 0x12, 0x09, 0x0a, 0x05, 0x54, 0x52, 0x41, 0x43, 0x45, 0x10, 0x01, + 0x12, 0x09, 0x0a, 0x05, 0x44, 0x45, 0x42, 0x55, 0x47, 0x10, 0x02, 0x12, 0x08, 0x0a, 0x04, 0x49, + 0x4e, 0x46, 0x4f, 0x10, 0x03, 0x12, 0x0a, 0x0a, 0x06, 0x4e, 0x4f, 0x54, 0x49, 0x43, 0x45, 0x10, + 0x04, 0x12, 0x08, 0x0a, 0x04, 0x57, 0x41, 0x52, 0x4e, 0x10, 0x05, 0x12, 0x09, 0x0a, 0x05, 0x45, + 0x52, 0x52, 0x4f, 0x52, 0x10, 0x06, 0x12, 0x0c, 0x0a, 0x08, 0x43, 0x52, 0x49, 0x54, 0x49, 0x43, + 0x41, 0x4c, 0x10, 0x07, 0x22, 0x0c, 0x0a, 0x0a, 0x4c, 0x6f, 0x67, 0x43, 0x6f, 0x6e, 0x74, 0x72, + 0x6f, 0x6c, 0x22, 0xe1, 0x04, 0x0a, 0x12, 0x53, 0x74, 0x61, 0x72, 0x74, 0x57, 0x6f, 0x72, 0x6b, + 0x65, 0x72, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x1b, 0x0a, 0x09, 0x77, 0x6f, 0x72, + 0x6b, 0x65, 0x72, 0x5f, 0x69, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x08, 0x77, 0x6f, + 0x72, 0x6b, 0x65, 0x72, 0x49, 0x64, 0x12, 0x62, 0x0a, 0x10, 0x63, 0x6f, 0x6e, 0x74, 0x72, 0x6f, + 0x6c, 0x5f, 0x65, 0x6e, 0x64, 0x70, 0x6f, 0x69, 0x6e, 0x74, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, + 0x32, 0x37, 0x2e, 0x6f, 0x72, 0x67, 0x2e, 0x61, 0x70, 0x61, 0x63, 0x68, 0x65, 0x2e, 0x62, 0x65, + 0x61, 0x6d, 0x2e, 0x6d, 0x6f, 0x64, 0x65, 0x6c, 0x2e, 0x70, 0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e, + 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x41, 0x70, 0x69, 0x53, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, 0x44, + 0x65, 0x73, 0x63, 0x72, 0x69, 0x70, 0x74, 0x6f, 0x72, 0x52, 0x0f, 0x63, 0x6f, 0x6e, 0x74, 0x72, + 0x6f, 0x6c, 0x45, 0x6e, 0x64, 0x70, 0x6f, 0x69, 0x6e, 0x74, 0x12, 0x62, 0x0a, 0x10, 0x6c, 0x6f, + 0x67, 0x67, 0x69, 0x6e, 0x67, 0x5f, 0x65, 0x6e, 0x64, 0x70, 0x6f, 0x69, 0x6e, 0x74, 0x18, 0x03, + 0x20, 0x01, 0x28, 0x0b, 0x32, 0x37, 0x2e, 0x6f, 0x72, 0x67, 0x2e, 0x61, 0x70, 0x61, 0x63, 0x68, + 0x65, 0x2e, 0x62, 0x65, 0x61, 0x6d, 0x2e, 0x6d, 0x6f, 0x64, 0x65, 0x6c, 0x2e, 0x70, 0x69, 0x70, + 0x65, 0x6c, 0x69, 0x6e, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x41, 0x70, 0x69, 0x53, 0x65, 0x72, 0x76, + 0x69, 0x63, 0x65, 0x44, 0x65, 0x73, 0x63, 0x72, 0x69, 0x70, 0x74, 0x6f, 0x72, 0x52, 0x0f, 0x6c, + 0x6f, 0x67, 0x67, 0x69, 0x6e, 0x67, 0x45, 0x6e, 0x64, 0x70, 0x6f, 0x69, 0x6e, 0x74, 0x12, 0x64, + 0x0a, 0x11, 0x61, 0x72, 0x74, 0x69, 0x66, 0x61, 0x63, 0x74, 0x5f, 0x65, 0x6e, 0x64, 0x70, 0x6f, + 0x69, 0x6e, 0x74, 0x18, 0x04, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x37, 0x2e, 0x6f, 0x72, 0x67, 0x2e, + 0x61, 0x70, 0x61, 0x63, 0x68, 0x65, 0x2e, 0x62, 0x65, 0x61, 0x6d, 0x2e, 0x6d, 0x6f, 0x64, 0x65, + 0x6c, 0x2e, 0x70, 0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x41, 0x70, + 0x69, 0x53, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, 0x44, 0x65, 0x73, 0x63, 0x72, 0x69, 0x70, 0x74, + 0x6f, 0x72, 0x52, 0x10, 0x61, 0x72, 0x74, 0x69, 0x66, 0x61, 0x63, 0x74, 0x45, 0x6e, 0x64, 0x70, + 0x6f, 0x69, 0x6e, 0x74, 0x12, 0x66, 0x0a, 0x12, 0x70, 0x72, 0x6f, 0x76, 0x69, 0x73, 0x69, 0x6f, + 0x6e, 0x5f, 0x65, 0x6e, 0x64, 0x70, 0x6f, 0x69, 0x6e, 0x74, 0x18, 0x05, 0x20, 0x01, 0x28, 0x0b, + 0x32, 0x37, 0x2e, 0x6f, 0x72, 0x67, 0x2e, 0x61, 0x70, 0x61, 0x63, 0x68, 0x65, 0x2e, 0x62, 0x65, + 0x61, 0x6d, 0x2e, 0x6d, 0x6f, 0x64, 0x65, 0x6c, 0x2e, 0x70, 0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e, + 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x41, 0x70, 0x69, 0x53, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, 0x44, + 0x65, 0x73, 0x63, 0x72, 0x69, 0x70, 0x74, 0x6f, 0x72, 0x52, 0x11, 0x70, 0x72, 0x6f, 0x76, 0x69, + 0x73, 0x69, 0x6f, 0x6e, 0x45, 0x6e, 0x64, 0x70, 0x6f, 0x69, 0x6e, 0x74, 0x12, 0x5d, 0x0a, 0x06, + 0x70, 0x61, 0x72, 0x61, 0x6d, 0x73, 0x18, 0x0a, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x45, 0x2e, 0x6f, + 0x72, 0x67, 0x2e, 0x61, 0x70, 0x61, 0x63, 0x68, 0x65, 0x2e, 0x62, 0x65, 0x61, 0x6d, 0x2e, 0x6d, + 0x6f, 0x64, 0x65, 0x6c, 0x2e, 0x66, 0x6e, 0x5f, 0x65, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, + 0x6e, 0x2e, 0x76, 0x31, 0x2e, 0x53, 0x74, 0x61, 0x72, 0x74, 0x57, 0x6f, 0x72, 0x6b, 0x65, 0x72, + 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x2e, 0x50, 0x61, 0x72, 0x61, 0x6d, 0x73, 0x45, 0x6e, + 0x74, 0x72, 0x79, 0x52, 0x06, 0x70, 0x61, 0x72, 0x61, 0x6d, 0x73, 0x1a, 0x39, 0x0a, 0x0b, 0x50, + 0x61, 0x72, 0x61, 0x6d, 0x73, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x12, 0x10, 0x0a, 0x03, 0x6b, 0x65, + 0x79, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x03, 0x6b, 0x65, 0x79, 0x12, 0x14, 0x0a, 0x05, + 0x76, 0x61, 0x6c, 0x75, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x05, 0x76, 0x61, 0x6c, + 0x75, 0x65, 0x3a, 0x02, 0x38, 0x01, 0x22, 0x2b, 0x0a, 0x13, 0x53, 0x74, 0x61, 0x72, 0x74, 0x57, + 0x6f, 0x72, 0x6b, 0x65, 0x72, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x14, 0x0a, + 0x05, 0x65, 0x72, 0x72, 0x6f, 0x72, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x05, 0x65, 0x72, + 0x72, 0x6f, 0x72, 0x22, 0x30, 0x0a, 0x11, 0x53, 0x74, 0x6f, 0x70, 0x57, 0x6f, 0x72, 0x6b, 0x65, + 0x72, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x1b, 0x0a, 0x09, 0x77, 0x6f, 0x72, 0x6b, + 0x65, 0x72, 0x5f, 0x69, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x08, 0x77, 0x6f, 0x72, + 0x6b, 0x65, 0x72, 0x49, 0x64, 0x22, 0x2a, 0x0a, 0x12, 0x53, 0x74, 0x6f, 0x70, 0x57, 0x6f, 0x72, + 0x6b, 0x65, 0x72, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x14, 0x0a, 0x05, 0x65, + 0x72, 0x72, 0x6f, 0x72, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x05, 0x65, 0x72, 0x72, 0x6f, + 0x72, 0x22, 0x25, 0x0a, 0x13, 0x57, 0x6f, 0x72, 0x6b, 0x65, 0x72, 0x53, 0x74, 0x61, 0x74, 0x75, + 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x0e, 0x0a, 0x02, 0x69, 0x64, 0x18, 0x01, + 0x20, 0x01, 0x28, 0x09, 0x52, 0x02, 0x69, 0x64, 0x22, 0x5d, 0x0a, 0x14, 0x57, 0x6f, 0x72, 0x6b, + 0x65, 0x72, 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, + 0x12, 0x0e, 0x0a, 0x02, 0x69, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x02, 0x69, 0x64, + 0x12, 0x14, 0x0a, 0x05, 0x65, 0x72, 0x72, 0x6f, 0x72, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, + 0x05, 0x65, 0x72, 0x72, 0x6f, 0x72, 0x12, 0x1f, 0x0a, 0x0b, 0x73, 0x74, 0x61, 0x74, 0x75, 0x73, + 0x5f, 0x69, 0x6e, 0x66, 0x6f, 0x18, 0x03, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0a, 0x73, 0x74, 0x61, + 0x74, 0x75, 0x73, 0x49, 0x6e, 0x66, 0x6f, 0x32, 0xc3, 0x02, 0x0a, 0x0d, 0x42, 0x65, 0x61, 0x6d, + 0x46, 0x6e, 0x43, 0x6f, 0x6e, 0x74, 0x72, 0x6f, 0x6c, 0x12, 0x86, 0x01, 0x0a, 0x07, 0x43, 0x6f, + 0x6e, 0x74, 0x72, 0x6f, 0x6c, 0x12, 0x3a, 0x2e, 0x6f, 0x72, 0x67, 0x2e, 0x61, 0x70, 0x61, 0x63, + 0x68, 0x65, 0x2e, 0x62, 0x65, 0x61, 0x6d, 0x2e, 0x6d, 0x6f, 0x64, 0x65, 0x6c, 0x2e, 0x66, 0x6e, + 0x5f, 0x65, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x2e, 0x76, 0x31, 0x2e, 0x49, 0x6e, + 0x73, 0x74, 0x72, 0x75, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, + 0x65, 0x1a, 0x39, 0x2e, 0x6f, 0x72, 0x67, 0x2e, 0x61, 0x70, 0x61, 0x63, 0x68, 0x65, 0x2e, 0x62, + 0x65, 0x61, 0x6d, 0x2e, 0x6d, 0x6f, 0x64, 0x65, 0x6c, 0x2e, 0x66, 0x6e, 0x5f, 0x65, 0x78, 0x65, + 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x2e, 0x76, 0x31, 0x2e, 0x49, 0x6e, 0x73, 0x74, 0x72, 0x75, + 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x22, 0x00, 0x28, 0x01, + 0x30, 0x01, 0x12, 0xa8, 0x01, 0x0a, 0x1a, 0x47, 0x65, 0x74, 0x50, 0x72, 0x6f, 0x63, 0x65, 0x73, + 0x73, 0x42, 0x75, 0x6e, 0x64, 0x6c, 0x65, 0x44, 0x65, 0x73, 0x63, 0x72, 0x69, 0x70, 0x74, 0x6f, + 0x72, 0x12, 0x48, 0x2e, 0x6f, 0x72, 0x67, 0x2e, 0x61, 0x70, 0x61, 0x63, 0x68, 0x65, 0x2e, 0x62, + 0x65, 0x61, 0x6d, 0x2e, 0x6d, 0x6f, 0x64, 0x65, 0x6c, 0x2e, 0x66, 0x6e, 0x5f, 0x65, 0x78, 0x65, + 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x2e, 0x76, 0x31, 0x2e, 0x47, 0x65, 0x74, 0x50, 0x72, 0x6f, + 0x63, 0x65, 0x73, 0x73, 0x42, 0x75, 0x6e, 0x64, 0x6c, 0x65, 0x44, 0x65, 0x73, 0x63, 0x72, 0x69, + 0x70, 0x74, 0x6f, 0x72, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x3e, 0x2e, 0x6f, 0x72, + 0x67, 0x2e, 0x61, 0x70, 0x61, 0x63, 0x68, 0x65, 0x2e, 0x62, 0x65, 0x61, 0x6d, 0x2e, 0x6d, 0x6f, + 0x64, 0x65, 0x6c, 0x2e, 0x66, 0x6e, 0x5f, 0x65, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, + 0x2e, 0x76, 0x31, 0x2e, 0x50, 0x72, 0x6f, 0x63, 0x65, 0x73, 0x73, 0x42, 0x75, 0x6e, 0x64, 0x6c, + 0x65, 0x44, 0x65, 0x73, 0x63, 0x72, 0x69, 0x70, 0x74, 0x6f, 0x72, 0x22, 0x00, 0x32, 0x7c, 0x0a, + 0x0a, 0x42, 0x65, 0x61, 0x6d, 0x46, 0x6e, 0x44, 0x61, 0x74, 0x61, 0x12, 0x6e, 0x0a, 0x04, 0x44, + 0x61, 0x74, 0x61, 0x12, 0x2f, 0x2e, 0x6f, 0x72, 0x67, 0x2e, 0x61, 0x70, 0x61, 0x63, 0x68, 0x65, + 0x2e, 0x62, 0x65, 0x61, 0x6d, 0x2e, 0x6d, 0x6f, 0x64, 0x65, 0x6c, 0x2e, 0x66, 0x6e, 0x5f, 0x65, + 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x2e, 0x76, 0x31, 0x2e, 0x45, 0x6c, 0x65, 0x6d, + 0x65, 0x6e, 0x74, 0x73, 0x1a, 0x2f, 0x2e, 0x6f, 0x72, 0x67, 0x2e, 0x61, 0x70, 0x61, 0x63, 0x68, + 0x65, 0x2e, 0x62, 0x65, 0x61, 0x6d, 0x2e, 0x6d, 0x6f, 0x64, 0x65, 0x6c, 0x2e, 0x66, 0x6e, 0x5f, + 0x65, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x2e, 0x76, 0x31, 0x2e, 0x45, 0x6c, 0x65, + 0x6d, 0x65, 0x6e, 0x74, 0x73, 0x22, 0x00, 0x28, 0x01, 0x30, 0x01, 0x32, 0x87, 0x01, 0x0a, 0x0b, + 0x42, 0x65, 0x61, 0x6d, 0x46, 0x6e, 0x53, 0x74, 0x61, 0x74, 0x65, 0x12, 0x78, 0x0a, 0x05, 0x53, + 0x74, 0x61, 0x74, 0x65, 0x12, 0x33, 0x2e, 0x6f, 0x72, 0x67, 0x2e, 0x61, 0x70, 0x61, 0x63, 0x68, + 0x65, 0x2e, 0x62, 0x65, 0x61, 0x6d, 0x2e, 0x6d, 0x6f, 0x64, 0x65, 0x6c, 0x2e, 0x66, 0x6e, 0x5f, + 0x65, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x2e, 0x76, 0x31, 0x2e, 0x53, 0x74, 0x61, + 0x74, 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x34, 0x2e, 0x6f, 0x72, 0x67, 0x2e, + 0x61, 0x70, 0x61, 0x63, 0x68, 0x65, 0x2e, 0x62, 0x65, 0x61, 0x6d, 0x2e, 0x6d, 0x6f, 0x64, 0x65, + 0x6c, 0x2e, 0x66, 0x6e, 0x5f, 0x65, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x2e, 0x76, + 0x31, 0x2e, 0x53, 0x74, 0x61, 0x74, 0x65, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, + 0x00, 0x28, 0x01, 0x30, 0x01, 0x32, 0x89, 0x01, 0x0a, 0x0d, 0x42, 0x65, 0x61, 0x6d, 0x46, 0x6e, + 0x4c, 0x6f, 0x67, 0x67, 0x69, 0x6e, 0x67, 0x12, 0x78, 0x0a, 0x07, 0x4c, 0x6f, 0x67, 0x67, 0x69, + 0x6e, 0x67, 0x12, 0x34, 0x2e, 0x6f, 0x72, 0x67, 0x2e, 0x61, 0x70, 0x61, 0x63, 0x68, 0x65, 0x2e, + 0x62, 0x65, 0x61, 0x6d, 0x2e, 0x6d, 0x6f, 0x64, 0x65, 0x6c, 0x2e, 0x66, 0x6e, 0x5f, 0x65, 0x78, + 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x2e, 0x76, 0x31, 0x2e, 0x4c, 0x6f, 0x67, 0x45, 0x6e, + 0x74, 0x72, 0x79, 0x2e, 0x4c, 0x69, 0x73, 0x74, 0x1a, 0x31, 0x2e, 0x6f, 0x72, 0x67, 0x2e, 0x61, + 0x70, 0x61, 0x63, 0x68, 0x65, 0x2e, 0x62, 0x65, 0x61, 0x6d, 0x2e, 0x6d, 0x6f, 0x64, 0x65, 0x6c, + 0x2e, 0x66, 0x6e, 0x5f, 0x65, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x2e, 0x76, 0x31, + 0x2e, 0x4c, 0x6f, 0x67, 0x43, 0x6f, 0x6e, 0x74, 0x72, 0x6f, 0x6c, 0x22, 0x00, 0x28, 0x01, 0x30, + 0x01, 0x32, 0xa9, 0x02, 0x0a, 0x18, 0x42, 0x65, 0x61, 0x6d, 0x46, 0x6e, 0x45, 0x78, 0x74, 0x65, + 0x72, 0x6e, 0x61, 0x6c, 0x57, 0x6f, 0x72, 0x6b, 0x65, 0x72, 0x50, 0x6f, 0x6f, 0x6c, 0x12, 0x86, + 0x01, 0x0a, 0x0b, 0x53, 0x74, 0x61, 0x72, 0x74, 0x57, 0x6f, 0x72, 0x6b, 0x65, 0x72, 0x12, 0x39, + 0x2e, 0x6f, 0x72, 0x67, 0x2e, 0x61, 0x70, 0x61, 0x63, 0x68, 0x65, 0x2e, 0x62, 0x65, 0x61, 0x6d, + 0x2e, 0x6d, 0x6f, 0x64, 0x65, 0x6c, 0x2e, 0x66, 0x6e, 0x5f, 0x65, 0x78, 0x65, 0x63, 0x75, 0x74, + 0x69, 0x6f, 0x6e, 0x2e, 0x76, 0x31, 0x2e, 0x53, 0x74, 0x61, 0x72, 0x74, 0x57, 0x6f, 0x72, 0x6b, + 0x65, 0x72, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x3a, 0x2e, 0x6f, 0x72, 0x67, 0x2e, + 0x61, 0x70, 0x61, 0x63, 0x68, 0x65, 0x2e, 0x62, 0x65, 0x61, 0x6d, 0x2e, 0x6d, 0x6f, 0x64, 0x65, + 0x6c, 0x2e, 0x66, 0x6e, 0x5f, 0x65, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x2e, 0x76, + 0x31, 0x2e, 0x53, 0x74, 0x61, 0x72, 0x74, 0x57, 0x6f, 0x72, 0x6b, 0x65, 0x72, 0x52, 0x65, 0x73, + 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x83, 0x01, 0x0a, 0x0a, 0x53, 0x74, 0x6f, 0x70, + 0x57, 0x6f, 0x72, 0x6b, 0x65, 0x72, 0x12, 0x38, 0x2e, 0x6f, 0x72, 0x67, 0x2e, 0x61, 0x70, 0x61, + 0x63, 0x68, 0x65, 0x2e, 0x62, 0x65, 0x61, 0x6d, 0x2e, 0x6d, 0x6f, 0x64, 0x65, 0x6c, 0x2e, 0x66, + 0x6e, 0x5f, 0x65, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x2e, 0x76, 0x31, 0x2e, 0x53, + 0x74, 0x6f, 0x70, 0x57, 0x6f, 0x72, 0x6b, 0x65, 0x72, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, + 0x1a, 0x39, 0x2e, 0x6f, 0x72, 0x67, 0x2e, 0x61, 0x70, 0x61, 0x63, 0x68, 0x65, 0x2e, 0x62, 0x65, + 0x61, 0x6d, 0x2e, 0x6d, 0x6f, 0x64, 0x65, 0x6c, 0x2e, 0x66, 0x6e, 0x5f, 0x65, 0x78, 0x65, 0x63, + 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x2e, 0x76, 0x31, 0x2e, 0x53, 0x74, 0x6f, 0x70, 0x57, 0x6f, 0x72, + 0x6b, 0x65, 0x72, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x32, 0xa4, 0x01, + 0x0a, 0x12, 0x42, 0x65, 0x61, 0x6d, 0x46, 0x6e, 0x57, 0x6f, 0x72, 0x6b, 0x65, 0x72, 0x53, 0x74, + 0x61, 0x74, 0x75, 0x73, 0x12, 0x8d, 0x01, 0x0a, 0x0c, 0x57, 0x6f, 0x72, 0x6b, 0x65, 0x72, 0x53, + 0x74, 0x61, 0x74, 0x75, 0x73, 0x12, 0x3b, 0x2e, 0x6f, 0x72, 0x67, 0x2e, 0x61, 0x70, 0x61, 0x63, + 0x68, 0x65, 0x2e, 0x62, 0x65, 0x61, 0x6d, 0x2e, 0x6d, 0x6f, 0x64, 0x65, 0x6c, 0x2e, 0x66, 0x6e, + 0x5f, 0x65, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x2e, 0x76, 0x31, 0x2e, 0x57, 0x6f, + 0x72, 0x6b, 0x65, 0x72, 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, + 0x73, 0x65, 0x1a, 0x3a, 0x2e, 0x6f, 0x72, 0x67, 0x2e, 0x61, 0x70, 0x61, 0x63, 0x68, 0x65, 0x2e, + 0x62, 0x65, 0x61, 0x6d, 0x2e, 0x6d, 0x6f, 0x64, 0x65, 0x6c, 0x2e, 0x66, 0x6e, 0x5f, 0x65, 0x78, + 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x2e, 0x76, 0x31, 0x2e, 0x57, 0x6f, 0x72, 0x6b, 0x65, + 0x72, 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x22, 0x00, + 0x28, 0x01, 0x30, 0x01, 0x42, 0x81, 0x01, 0x0a, 0x24, 0x6f, 0x72, 0x67, 0x2e, 0x61, 0x70, 0x61, + 0x63, 0x68, 0x65, 0x2e, 0x62, 0x65, 0x61, 0x6d, 0x2e, 0x6d, 0x6f, 0x64, 0x65, 0x6c, 0x2e, 0x66, + 0x6e, 0x65, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x2e, 0x76, 0x31, 0x42, 0x09, 0x42, + 0x65, 0x61, 0x6d, 0x46, 0x6e, 0x41, 0x70, 0x69, 0x5a, 0x4e, 0x67, 0x69, 0x74, 0x68, 0x75, 0x62, + 0x2e, 0x63, 0x6f, 0x6d, 0x2f, 0x61, 0x70, 0x61, 0x63, 0x68, 0x65, 0x2f, 0x62, 0x65, 0x61, 0x6d, + 0x2f, 0x73, 0x64, 0x6b, 0x73, 0x2f, 0x76, 0x32, 0x2f, 0x67, 0x6f, 0x2f, 0x70, 0x6b, 0x67, 0x2f, + 0x62, 0x65, 0x61, 0x6d, 0x2f, 0x6d, 0x6f, 0x64, 0x65, 0x6c, 0x2f, 0x66, 0x6e, 0x65, 0x78, 0x65, + 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x76, 0x31, 0x3b, 0x66, 0x6e, 0x65, 0x78, 0x65, 0x63, + 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x76, 0x31, 0x62, 0x06, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x33, +} + +var file_org_apache_beam_model_fn_execution_v1_beam_fn_api_proto_enumTypes = make([]protoimpl.EnumInfo, 2) +var file_org_apache_beam_model_fn_execution_v1_beam_fn_api_proto_msgTypes = make([]protoimpl.MessageInfo, 77) +var file_org_apache_beam_model_fn_execution_v1_beam_fn_api_proto_goTypes = []any{ + (FnApiTransforms_Runner)(0), // 0: org.apache.beam.model.fn_execution.v1.FnApiTransforms.Runner + (LogEntry_Severity_Enum)(0), // 1: org.apache.beam.model.fn_execution.v1.LogEntry.Severity.Enum + (*FnApiTransforms)(nil), // 2: org.apache.beam.model.fn_execution.v1.FnApiTransforms + (*RemoteGrpcPort)(nil), // 3: org.apache.beam.model.fn_execution.v1.RemoteGrpcPort + (*GetProcessBundleDescriptorRequest)(nil), // 4: org.apache.beam.model.fn_execution.v1.GetProcessBundleDescriptorRequest + (*InstructionRequest)(nil), // 5: org.apache.beam.model.fn_execution.v1.InstructionRequest + (*InstructionResponse)(nil), // 6: org.apache.beam.model.fn_execution.v1.InstructionResponse + (*SampleDataRequest)(nil), // 7: org.apache.beam.model.fn_execution.v1.SampleDataRequest + (*SampledElement)(nil), // 8: org.apache.beam.model.fn_execution.v1.SampledElement + (*SampleDataResponse)(nil), // 9: org.apache.beam.model.fn_execution.v1.SampleDataResponse + (*HarnessMonitoringInfosRequest)(nil), // 10: org.apache.beam.model.fn_execution.v1.HarnessMonitoringInfosRequest + (*HarnessMonitoringInfosResponse)(nil), // 11: org.apache.beam.model.fn_execution.v1.HarnessMonitoringInfosResponse + (*RegisterRequest)(nil), // 12: org.apache.beam.model.fn_execution.v1.RegisterRequest + (*RegisterResponse)(nil), // 13: org.apache.beam.model.fn_execution.v1.RegisterResponse + (*ProcessBundleDescriptor)(nil), // 14: org.apache.beam.model.fn_execution.v1.ProcessBundleDescriptor + (*BundleApplication)(nil), // 15: org.apache.beam.model.fn_execution.v1.BundleApplication + (*DelayedBundleApplication)(nil), // 16: org.apache.beam.model.fn_execution.v1.DelayedBundleApplication + (*ProcessBundleRequest)(nil), // 17: org.apache.beam.model.fn_execution.v1.ProcessBundleRequest + (*ProcessBundleResponse)(nil), // 18: org.apache.beam.model.fn_execution.v1.ProcessBundleResponse + (*ProcessBundleProgressRequest)(nil), // 19: org.apache.beam.model.fn_execution.v1.ProcessBundleProgressRequest + (*MonitoringInfosMetadataRequest)(nil), // 20: org.apache.beam.model.fn_execution.v1.MonitoringInfosMetadataRequest + (*ProcessBundleProgressResponse)(nil), // 21: org.apache.beam.model.fn_execution.v1.ProcessBundleProgressResponse + (*MonitoringInfosMetadataResponse)(nil), // 22: org.apache.beam.model.fn_execution.v1.MonitoringInfosMetadataResponse + (*ProcessBundleSplitRequest)(nil), // 23: org.apache.beam.model.fn_execution.v1.ProcessBundleSplitRequest + (*ProcessBundleSplitResponse)(nil), // 24: org.apache.beam.model.fn_execution.v1.ProcessBundleSplitResponse + (*FinalizeBundleRequest)(nil), // 25: org.apache.beam.model.fn_execution.v1.FinalizeBundleRequest + (*FinalizeBundleResponse)(nil), // 26: org.apache.beam.model.fn_execution.v1.FinalizeBundleResponse + (*Elements)(nil), // 27: org.apache.beam.model.fn_execution.v1.Elements + (*StateRequest)(nil), // 28: org.apache.beam.model.fn_execution.v1.StateRequest + (*StateResponse)(nil), // 29: org.apache.beam.model.fn_execution.v1.StateResponse + (*StateKey)(nil), // 30: org.apache.beam.model.fn_execution.v1.StateKey + (*StateGetRequest)(nil), // 31: org.apache.beam.model.fn_execution.v1.StateGetRequest + (*StateGetResponse)(nil), // 32: org.apache.beam.model.fn_execution.v1.StateGetResponse + (*StateAppendRequest)(nil), // 33: org.apache.beam.model.fn_execution.v1.StateAppendRequest + (*StateAppendResponse)(nil), // 34: org.apache.beam.model.fn_execution.v1.StateAppendResponse + (*StateClearRequest)(nil), // 35: org.apache.beam.model.fn_execution.v1.StateClearRequest + (*StateClearResponse)(nil), // 36: org.apache.beam.model.fn_execution.v1.StateClearResponse + (*OrderedListRange)(nil), // 37: org.apache.beam.model.fn_execution.v1.OrderedListRange + (*LogEntry)(nil), // 38: org.apache.beam.model.fn_execution.v1.LogEntry + (*LogControl)(nil), // 39: org.apache.beam.model.fn_execution.v1.LogControl + (*StartWorkerRequest)(nil), // 40: org.apache.beam.model.fn_execution.v1.StartWorkerRequest + (*StartWorkerResponse)(nil), // 41: org.apache.beam.model.fn_execution.v1.StartWorkerResponse + (*StopWorkerRequest)(nil), // 42: org.apache.beam.model.fn_execution.v1.StopWorkerRequest + (*StopWorkerResponse)(nil), // 43: org.apache.beam.model.fn_execution.v1.StopWorkerResponse + (*WorkerStatusRequest)(nil), // 44: org.apache.beam.model.fn_execution.v1.WorkerStatusRequest + (*WorkerStatusResponse)(nil), // 45: org.apache.beam.model.fn_execution.v1.WorkerStatusResponse + (*SampledElement_Exception)(nil), // 46: org.apache.beam.model.fn_execution.v1.SampledElement.Exception + (*SampleDataResponse_ElementList)(nil), // 47: org.apache.beam.model.fn_execution.v1.SampleDataResponse.ElementList + nil, // 48: org.apache.beam.model.fn_execution.v1.SampleDataResponse.ElementSamplesEntry + nil, // 49: org.apache.beam.model.fn_execution.v1.HarnessMonitoringInfosResponse.MonitoringDataEntry + nil, // 50: org.apache.beam.model.fn_execution.v1.ProcessBundleDescriptor.TransformsEntry + nil, // 51: org.apache.beam.model.fn_execution.v1.ProcessBundleDescriptor.PcollectionsEntry + nil, // 52: org.apache.beam.model.fn_execution.v1.ProcessBundleDescriptor.WindowingStrategiesEntry + nil, // 53: org.apache.beam.model.fn_execution.v1.ProcessBundleDescriptor.CodersEntry + nil, // 54: org.apache.beam.model.fn_execution.v1.ProcessBundleDescriptor.EnvironmentsEntry + nil, // 55: org.apache.beam.model.fn_execution.v1.BundleApplication.OutputWatermarksEntry + (*ProcessBundleRequest_CacheToken)(nil), // 56: org.apache.beam.model.fn_execution.v1.ProcessBundleRequest.CacheToken + (*ProcessBundleRequest_CacheToken_UserState)(nil), // 57: org.apache.beam.model.fn_execution.v1.ProcessBundleRequest.CacheToken.UserState + (*ProcessBundleRequest_CacheToken_SideInput)(nil), // 58: org.apache.beam.model.fn_execution.v1.ProcessBundleRequest.CacheToken.SideInput + nil, // 59: org.apache.beam.model.fn_execution.v1.ProcessBundleResponse.MonitoringDataEntry + nil, // 60: org.apache.beam.model.fn_execution.v1.ProcessBundleProgressResponse.MonitoringDataEntry + nil, // 61: org.apache.beam.model.fn_execution.v1.MonitoringInfosMetadataResponse.MonitoringInfoEntry + (*ProcessBundleSplitRequest_DesiredSplit)(nil), // 62: org.apache.beam.model.fn_execution.v1.ProcessBundleSplitRequest.DesiredSplit + nil, // 63: org.apache.beam.model.fn_execution.v1.ProcessBundleSplitRequest.DesiredSplitsEntry + (*ProcessBundleSplitResponse_ChannelSplit)(nil), // 64: org.apache.beam.model.fn_execution.v1.ProcessBundleSplitResponse.ChannelSplit + (*Elements_Data)(nil), // 65: org.apache.beam.model.fn_execution.v1.Elements.Data + (*Elements_Timers)(nil), // 66: org.apache.beam.model.fn_execution.v1.Elements.Timers + (*StateKey_Runner)(nil), // 67: org.apache.beam.model.fn_execution.v1.StateKey.Runner + (*StateKey_IterableSideInput)(nil), // 68: org.apache.beam.model.fn_execution.v1.StateKey.IterableSideInput + (*StateKey_MultimapSideInput)(nil), // 69: org.apache.beam.model.fn_execution.v1.StateKey.MultimapSideInput + (*StateKey_MultimapKeysSideInput)(nil), // 70: org.apache.beam.model.fn_execution.v1.StateKey.MultimapKeysSideInput + (*StateKey_MultimapKeysValuesSideInput)(nil), // 71: org.apache.beam.model.fn_execution.v1.StateKey.MultimapKeysValuesSideInput + (*StateKey_BagUserState)(nil), // 72: org.apache.beam.model.fn_execution.v1.StateKey.BagUserState + (*StateKey_MultimapKeysUserState)(nil), // 73: org.apache.beam.model.fn_execution.v1.StateKey.MultimapKeysUserState + (*StateKey_MultimapUserState)(nil), // 74: org.apache.beam.model.fn_execution.v1.StateKey.MultimapUserState + (*StateKey_OrderedListUserState)(nil), // 75: org.apache.beam.model.fn_execution.v1.StateKey.OrderedListUserState + (*LogEntry_List)(nil), // 76: org.apache.beam.model.fn_execution.v1.LogEntry.List + (*LogEntry_Severity)(nil), // 77: org.apache.beam.model.fn_execution.v1.LogEntry.Severity + nil, // 78: org.apache.beam.model.fn_execution.v1.StartWorkerRequest.ParamsEntry + (*pipeline_v1.ApiServiceDescriptor)(nil), // 79: org.apache.beam.model.pipeline.v1.ApiServiceDescriptor + (*timestamppb.Timestamp)(nil), // 80: google.protobuf.Timestamp + (pipeline_v1.IsBounded_Enum)(0), // 81: org.apache.beam.model.pipeline.v1.IsBounded.Enum + (*durationpb.Duration)(nil), // 82: google.protobuf.Duration + (*pipeline_v1.MonitoringInfo)(nil), // 83: org.apache.beam.model.pipeline.v1.MonitoringInfo + (*structpb.Struct)(nil), // 84: google.protobuf.Struct + (*pipeline_v1.PTransform)(nil), // 85: org.apache.beam.model.pipeline.v1.PTransform + (*pipeline_v1.PCollection)(nil), // 86: org.apache.beam.model.pipeline.v1.PCollection + (*pipeline_v1.WindowingStrategy)(nil), // 87: org.apache.beam.model.pipeline.v1.WindowingStrategy + (*pipeline_v1.Coder)(nil), // 88: org.apache.beam.model.pipeline.v1.Coder + (*pipeline_v1.Environment)(nil), // 89: org.apache.beam.model.pipeline.v1.Environment +} +var file_org_apache_beam_model_fn_execution_v1_beam_fn_api_proto_depIdxs = []int32{ + 79, // 0: org.apache.beam.model.fn_execution.v1.RemoteGrpcPort.api_service_descriptor:type_name -> org.apache.beam.model.pipeline.v1.ApiServiceDescriptor + 17, // 1: org.apache.beam.model.fn_execution.v1.InstructionRequest.process_bundle:type_name -> org.apache.beam.model.fn_execution.v1.ProcessBundleRequest + 19, // 2: org.apache.beam.model.fn_execution.v1.InstructionRequest.process_bundle_progress:type_name -> org.apache.beam.model.fn_execution.v1.ProcessBundleProgressRequest + 23, // 3: org.apache.beam.model.fn_execution.v1.InstructionRequest.process_bundle_split:type_name -> org.apache.beam.model.fn_execution.v1.ProcessBundleSplitRequest + 25, // 4: org.apache.beam.model.fn_execution.v1.InstructionRequest.finalize_bundle:type_name -> org.apache.beam.model.fn_execution.v1.FinalizeBundleRequest + 20, // 5: org.apache.beam.model.fn_execution.v1.InstructionRequest.monitoring_infos:type_name -> org.apache.beam.model.fn_execution.v1.MonitoringInfosMetadataRequest + 10, // 6: org.apache.beam.model.fn_execution.v1.InstructionRequest.harness_monitoring_infos:type_name -> org.apache.beam.model.fn_execution.v1.HarnessMonitoringInfosRequest + 7, // 7: org.apache.beam.model.fn_execution.v1.InstructionRequest.sample_data:type_name -> org.apache.beam.model.fn_execution.v1.SampleDataRequest + 12, // 8: org.apache.beam.model.fn_execution.v1.InstructionRequest.register:type_name -> org.apache.beam.model.fn_execution.v1.RegisterRequest + 18, // 9: org.apache.beam.model.fn_execution.v1.InstructionResponse.process_bundle:type_name -> org.apache.beam.model.fn_execution.v1.ProcessBundleResponse + 21, // 10: org.apache.beam.model.fn_execution.v1.InstructionResponse.process_bundle_progress:type_name -> org.apache.beam.model.fn_execution.v1.ProcessBundleProgressResponse + 24, // 11: org.apache.beam.model.fn_execution.v1.InstructionResponse.process_bundle_split:type_name -> org.apache.beam.model.fn_execution.v1.ProcessBundleSplitResponse + 26, // 12: org.apache.beam.model.fn_execution.v1.InstructionResponse.finalize_bundle:type_name -> org.apache.beam.model.fn_execution.v1.FinalizeBundleResponse + 22, // 13: org.apache.beam.model.fn_execution.v1.InstructionResponse.monitoring_infos:type_name -> org.apache.beam.model.fn_execution.v1.MonitoringInfosMetadataResponse + 11, // 14: org.apache.beam.model.fn_execution.v1.InstructionResponse.harness_monitoring_infos:type_name -> org.apache.beam.model.fn_execution.v1.HarnessMonitoringInfosResponse + 9, // 15: org.apache.beam.model.fn_execution.v1.InstructionResponse.sample_data:type_name -> org.apache.beam.model.fn_execution.v1.SampleDataResponse + 13, // 16: org.apache.beam.model.fn_execution.v1.InstructionResponse.register:type_name -> org.apache.beam.model.fn_execution.v1.RegisterResponse + 80, // 17: org.apache.beam.model.fn_execution.v1.SampledElement.sample_timestamp:type_name -> google.protobuf.Timestamp + 46, // 18: org.apache.beam.model.fn_execution.v1.SampledElement.exception:type_name -> org.apache.beam.model.fn_execution.v1.SampledElement.Exception + 48, // 19: org.apache.beam.model.fn_execution.v1.SampleDataResponse.element_samples:type_name -> org.apache.beam.model.fn_execution.v1.SampleDataResponse.ElementSamplesEntry + 49, // 20: org.apache.beam.model.fn_execution.v1.HarnessMonitoringInfosResponse.monitoring_data:type_name -> org.apache.beam.model.fn_execution.v1.HarnessMonitoringInfosResponse.MonitoringDataEntry + 14, // 21: org.apache.beam.model.fn_execution.v1.RegisterRequest.process_bundle_descriptor:type_name -> org.apache.beam.model.fn_execution.v1.ProcessBundleDescriptor + 50, // 22: org.apache.beam.model.fn_execution.v1.ProcessBundleDescriptor.transforms:type_name -> org.apache.beam.model.fn_execution.v1.ProcessBundleDescriptor.TransformsEntry + 51, // 23: org.apache.beam.model.fn_execution.v1.ProcessBundleDescriptor.pcollections:type_name -> org.apache.beam.model.fn_execution.v1.ProcessBundleDescriptor.PcollectionsEntry + 52, // 24: org.apache.beam.model.fn_execution.v1.ProcessBundleDescriptor.windowing_strategies:type_name -> org.apache.beam.model.fn_execution.v1.ProcessBundleDescriptor.WindowingStrategiesEntry + 53, // 25: org.apache.beam.model.fn_execution.v1.ProcessBundleDescriptor.coders:type_name -> org.apache.beam.model.fn_execution.v1.ProcessBundleDescriptor.CodersEntry + 54, // 26: org.apache.beam.model.fn_execution.v1.ProcessBundleDescriptor.environments:type_name -> org.apache.beam.model.fn_execution.v1.ProcessBundleDescriptor.EnvironmentsEntry + 79, // 27: org.apache.beam.model.fn_execution.v1.ProcessBundleDescriptor.state_api_service_descriptor:type_name -> org.apache.beam.model.pipeline.v1.ApiServiceDescriptor + 79, // 28: org.apache.beam.model.fn_execution.v1.ProcessBundleDescriptor.timer_api_service_descriptor:type_name -> org.apache.beam.model.pipeline.v1.ApiServiceDescriptor + 55, // 29: org.apache.beam.model.fn_execution.v1.BundleApplication.output_watermarks:type_name -> org.apache.beam.model.fn_execution.v1.BundleApplication.OutputWatermarksEntry + 81, // 30: org.apache.beam.model.fn_execution.v1.BundleApplication.is_bounded:type_name -> org.apache.beam.model.pipeline.v1.IsBounded.Enum + 15, // 31: org.apache.beam.model.fn_execution.v1.DelayedBundleApplication.application:type_name -> org.apache.beam.model.fn_execution.v1.BundleApplication + 82, // 32: org.apache.beam.model.fn_execution.v1.DelayedBundleApplication.requested_time_delay:type_name -> google.protobuf.Duration + 56, // 33: org.apache.beam.model.fn_execution.v1.ProcessBundleRequest.cache_tokens:type_name -> org.apache.beam.model.fn_execution.v1.ProcessBundleRequest.CacheToken + 27, // 34: org.apache.beam.model.fn_execution.v1.ProcessBundleRequest.elements:type_name -> org.apache.beam.model.fn_execution.v1.Elements + 16, // 35: org.apache.beam.model.fn_execution.v1.ProcessBundleResponse.residual_roots:type_name -> org.apache.beam.model.fn_execution.v1.DelayedBundleApplication + 83, // 36: org.apache.beam.model.fn_execution.v1.ProcessBundleResponse.monitoring_infos:type_name -> org.apache.beam.model.pipeline.v1.MonitoringInfo + 59, // 37: org.apache.beam.model.fn_execution.v1.ProcessBundleResponse.monitoring_data:type_name -> org.apache.beam.model.fn_execution.v1.ProcessBundleResponse.MonitoringDataEntry + 27, // 38: org.apache.beam.model.fn_execution.v1.ProcessBundleResponse.elements:type_name -> org.apache.beam.model.fn_execution.v1.Elements + 83, // 39: org.apache.beam.model.fn_execution.v1.ProcessBundleProgressResponse.monitoring_infos:type_name -> org.apache.beam.model.pipeline.v1.MonitoringInfo + 60, // 40: org.apache.beam.model.fn_execution.v1.ProcessBundleProgressResponse.monitoring_data:type_name -> org.apache.beam.model.fn_execution.v1.ProcessBundleProgressResponse.MonitoringDataEntry + 61, // 41: org.apache.beam.model.fn_execution.v1.MonitoringInfosMetadataResponse.monitoring_info:type_name -> org.apache.beam.model.fn_execution.v1.MonitoringInfosMetadataResponse.MonitoringInfoEntry + 63, // 42: org.apache.beam.model.fn_execution.v1.ProcessBundleSplitRequest.desired_splits:type_name -> org.apache.beam.model.fn_execution.v1.ProcessBundleSplitRequest.DesiredSplitsEntry + 15, // 43: org.apache.beam.model.fn_execution.v1.ProcessBundleSplitResponse.primary_roots:type_name -> org.apache.beam.model.fn_execution.v1.BundleApplication + 16, // 44: org.apache.beam.model.fn_execution.v1.ProcessBundleSplitResponse.residual_roots:type_name -> org.apache.beam.model.fn_execution.v1.DelayedBundleApplication + 64, // 45: org.apache.beam.model.fn_execution.v1.ProcessBundleSplitResponse.channel_splits:type_name -> org.apache.beam.model.fn_execution.v1.ProcessBundleSplitResponse.ChannelSplit + 65, // 46: org.apache.beam.model.fn_execution.v1.Elements.data:type_name -> org.apache.beam.model.fn_execution.v1.Elements.Data + 66, // 47: org.apache.beam.model.fn_execution.v1.Elements.timers:type_name -> org.apache.beam.model.fn_execution.v1.Elements.Timers + 30, // 48: org.apache.beam.model.fn_execution.v1.StateRequest.state_key:type_name -> org.apache.beam.model.fn_execution.v1.StateKey + 31, // 49: org.apache.beam.model.fn_execution.v1.StateRequest.get:type_name -> org.apache.beam.model.fn_execution.v1.StateGetRequest + 33, // 50: org.apache.beam.model.fn_execution.v1.StateRequest.append:type_name -> org.apache.beam.model.fn_execution.v1.StateAppendRequest + 35, // 51: org.apache.beam.model.fn_execution.v1.StateRequest.clear:type_name -> org.apache.beam.model.fn_execution.v1.StateClearRequest + 32, // 52: org.apache.beam.model.fn_execution.v1.StateResponse.get:type_name -> org.apache.beam.model.fn_execution.v1.StateGetResponse + 34, // 53: org.apache.beam.model.fn_execution.v1.StateResponse.append:type_name -> org.apache.beam.model.fn_execution.v1.StateAppendResponse + 36, // 54: org.apache.beam.model.fn_execution.v1.StateResponse.clear:type_name -> org.apache.beam.model.fn_execution.v1.StateClearResponse + 67, // 55: org.apache.beam.model.fn_execution.v1.StateKey.runner:type_name -> org.apache.beam.model.fn_execution.v1.StateKey.Runner + 69, // 56: org.apache.beam.model.fn_execution.v1.StateKey.multimap_side_input:type_name -> org.apache.beam.model.fn_execution.v1.StateKey.MultimapSideInput + 72, // 57: org.apache.beam.model.fn_execution.v1.StateKey.bag_user_state:type_name -> org.apache.beam.model.fn_execution.v1.StateKey.BagUserState + 68, // 58: org.apache.beam.model.fn_execution.v1.StateKey.iterable_side_input:type_name -> org.apache.beam.model.fn_execution.v1.StateKey.IterableSideInput + 70, // 59: org.apache.beam.model.fn_execution.v1.StateKey.multimap_keys_side_input:type_name -> org.apache.beam.model.fn_execution.v1.StateKey.MultimapKeysSideInput + 71, // 60: org.apache.beam.model.fn_execution.v1.StateKey.multimap_keys_values_side_input:type_name -> org.apache.beam.model.fn_execution.v1.StateKey.MultimapKeysValuesSideInput + 73, // 61: org.apache.beam.model.fn_execution.v1.StateKey.multimap_keys_user_state:type_name -> org.apache.beam.model.fn_execution.v1.StateKey.MultimapKeysUserState + 74, // 62: org.apache.beam.model.fn_execution.v1.StateKey.multimap_user_state:type_name -> org.apache.beam.model.fn_execution.v1.StateKey.MultimapUserState + 75, // 63: org.apache.beam.model.fn_execution.v1.StateKey.ordered_list_user_state:type_name -> org.apache.beam.model.fn_execution.v1.StateKey.OrderedListUserState + 1, // 64: org.apache.beam.model.fn_execution.v1.LogEntry.severity:type_name -> org.apache.beam.model.fn_execution.v1.LogEntry.Severity.Enum + 80, // 65: org.apache.beam.model.fn_execution.v1.LogEntry.timestamp:type_name -> google.protobuf.Timestamp + 84, // 66: org.apache.beam.model.fn_execution.v1.LogEntry.custom_data:type_name -> google.protobuf.Struct + 79, // 67: org.apache.beam.model.fn_execution.v1.StartWorkerRequest.control_endpoint:type_name -> org.apache.beam.model.pipeline.v1.ApiServiceDescriptor + 79, // 68: org.apache.beam.model.fn_execution.v1.StartWorkerRequest.logging_endpoint:type_name -> org.apache.beam.model.pipeline.v1.ApiServiceDescriptor + 79, // 69: org.apache.beam.model.fn_execution.v1.StartWorkerRequest.artifact_endpoint:type_name -> org.apache.beam.model.pipeline.v1.ApiServiceDescriptor + 79, // 70: org.apache.beam.model.fn_execution.v1.StartWorkerRequest.provision_endpoint:type_name -> org.apache.beam.model.pipeline.v1.ApiServiceDescriptor + 78, // 71: org.apache.beam.model.fn_execution.v1.StartWorkerRequest.params:type_name -> org.apache.beam.model.fn_execution.v1.StartWorkerRequest.ParamsEntry + 8, // 72: org.apache.beam.model.fn_execution.v1.SampleDataResponse.ElementList.elements:type_name -> org.apache.beam.model.fn_execution.v1.SampledElement + 47, // 73: org.apache.beam.model.fn_execution.v1.SampleDataResponse.ElementSamplesEntry.value:type_name -> org.apache.beam.model.fn_execution.v1.SampleDataResponse.ElementList + 85, // 74: org.apache.beam.model.fn_execution.v1.ProcessBundleDescriptor.TransformsEntry.value:type_name -> org.apache.beam.model.pipeline.v1.PTransform + 86, // 75: org.apache.beam.model.fn_execution.v1.ProcessBundleDescriptor.PcollectionsEntry.value:type_name -> org.apache.beam.model.pipeline.v1.PCollection + 87, // 76: org.apache.beam.model.fn_execution.v1.ProcessBundleDescriptor.WindowingStrategiesEntry.value:type_name -> org.apache.beam.model.pipeline.v1.WindowingStrategy + 88, // 77: org.apache.beam.model.fn_execution.v1.ProcessBundleDescriptor.CodersEntry.value:type_name -> org.apache.beam.model.pipeline.v1.Coder + 89, // 78: org.apache.beam.model.fn_execution.v1.ProcessBundleDescriptor.EnvironmentsEntry.value:type_name -> org.apache.beam.model.pipeline.v1.Environment + 80, // 79: org.apache.beam.model.fn_execution.v1.BundleApplication.OutputWatermarksEntry.value:type_name -> google.protobuf.Timestamp + 57, // 80: org.apache.beam.model.fn_execution.v1.ProcessBundleRequest.CacheToken.user_state:type_name -> org.apache.beam.model.fn_execution.v1.ProcessBundleRequest.CacheToken.UserState + 58, // 81: org.apache.beam.model.fn_execution.v1.ProcessBundleRequest.CacheToken.side_input:type_name -> org.apache.beam.model.fn_execution.v1.ProcessBundleRequest.CacheToken.SideInput + 83, // 82: org.apache.beam.model.fn_execution.v1.MonitoringInfosMetadataResponse.MonitoringInfoEntry.value:type_name -> org.apache.beam.model.pipeline.v1.MonitoringInfo + 62, // 83: org.apache.beam.model.fn_execution.v1.ProcessBundleSplitRequest.DesiredSplitsEntry.value:type_name -> org.apache.beam.model.fn_execution.v1.ProcessBundleSplitRequest.DesiredSplit + 37, // 84: org.apache.beam.model.fn_execution.v1.StateKey.OrderedListUserState.range:type_name -> org.apache.beam.model.fn_execution.v1.OrderedListRange + 38, // 85: org.apache.beam.model.fn_execution.v1.LogEntry.List.log_entries:type_name -> org.apache.beam.model.fn_execution.v1.LogEntry + 6, // 86: org.apache.beam.model.fn_execution.v1.BeamFnControl.Control:input_type -> org.apache.beam.model.fn_execution.v1.InstructionResponse + 4, // 87: org.apache.beam.model.fn_execution.v1.BeamFnControl.GetProcessBundleDescriptor:input_type -> org.apache.beam.model.fn_execution.v1.GetProcessBundleDescriptorRequest + 27, // 88: org.apache.beam.model.fn_execution.v1.BeamFnData.Data:input_type -> org.apache.beam.model.fn_execution.v1.Elements + 28, // 89: org.apache.beam.model.fn_execution.v1.BeamFnState.State:input_type -> org.apache.beam.model.fn_execution.v1.StateRequest + 76, // 90: org.apache.beam.model.fn_execution.v1.BeamFnLogging.Logging:input_type -> org.apache.beam.model.fn_execution.v1.LogEntry.List + 40, // 91: org.apache.beam.model.fn_execution.v1.BeamFnExternalWorkerPool.StartWorker:input_type -> org.apache.beam.model.fn_execution.v1.StartWorkerRequest + 42, // 92: org.apache.beam.model.fn_execution.v1.BeamFnExternalWorkerPool.StopWorker:input_type -> org.apache.beam.model.fn_execution.v1.StopWorkerRequest + 45, // 93: org.apache.beam.model.fn_execution.v1.BeamFnWorkerStatus.WorkerStatus:input_type -> org.apache.beam.model.fn_execution.v1.WorkerStatusResponse + 5, // 94: org.apache.beam.model.fn_execution.v1.BeamFnControl.Control:output_type -> org.apache.beam.model.fn_execution.v1.InstructionRequest + 14, // 95: org.apache.beam.model.fn_execution.v1.BeamFnControl.GetProcessBundleDescriptor:output_type -> org.apache.beam.model.fn_execution.v1.ProcessBundleDescriptor + 27, // 96: org.apache.beam.model.fn_execution.v1.BeamFnData.Data:output_type -> org.apache.beam.model.fn_execution.v1.Elements + 29, // 97: org.apache.beam.model.fn_execution.v1.BeamFnState.State:output_type -> org.apache.beam.model.fn_execution.v1.StateResponse + 39, // 98: org.apache.beam.model.fn_execution.v1.BeamFnLogging.Logging:output_type -> org.apache.beam.model.fn_execution.v1.LogControl + 41, // 99: org.apache.beam.model.fn_execution.v1.BeamFnExternalWorkerPool.StartWorker:output_type -> org.apache.beam.model.fn_execution.v1.StartWorkerResponse + 43, // 100: org.apache.beam.model.fn_execution.v1.BeamFnExternalWorkerPool.StopWorker:output_type -> org.apache.beam.model.fn_execution.v1.StopWorkerResponse + 44, // 101: org.apache.beam.model.fn_execution.v1.BeamFnWorkerStatus.WorkerStatus:output_type -> org.apache.beam.model.fn_execution.v1.WorkerStatusRequest + 94, // [94:102] is the sub-list for method output_type + 86, // [86:94] is the sub-list for method input_type + 86, // [86:86] is the sub-list for extension type_name + 86, // [86:86] is the sub-list for extension extendee + 0, // [0:86] is the sub-list for field type_name +} + +func init() { file_org_apache_beam_model_fn_execution_v1_beam_fn_api_proto_init() } +func file_org_apache_beam_model_fn_execution_v1_beam_fn_api_proto_init() { + if File_org_apache_beam_model_fn_execution_v1_beam_fn_api_proto != nil { + return + } + file_org_apache_beam_model_fn_execution_v1_beam_fn_api_proto_msgTypes[3].OneofWrappers = []any{ + (*instructionRequest_ProcessBundle)(nil), + (*instructionRequest_ProcessBundleProgress)(nil), + (*instructionRequest_ProcessBundleSplit)(nil), + (*instructionRequest_FinalizeBundle)(nil), + (*instructionRequest_MonitoringInfos)(nil), + (*instructionRequest_HarnessMonitoringInfos)(nil), + (*instructionRequest_SampleData)(nil), + (*instructionRequest_Register)(nil), + } + file_org_apache_beam_model_fn_execution_v1_beam_fn_api_proto_msgTypes[4].OneofWrappers = []any{ + (*instructionResponse_ProcessBundle)(nil), + (*instructionResponse_ProcessBundleProgress)(nil), + (*instructionResponse_ProcessBundleSplit)(nil), + (*instructionResponse_FinalizeBundle)(nil), + (*instructionResponse_MonitoringInfos)(nil), + (*instructionResponse_HarnessMonitoringInfos)(nil), + (*instructionResponse_SampleData)(nil), + (*instructionResponse_Register)(nil), + } + file_org_apache_beam_model_fn_execution_v1_beam_fn_api_proto_msgTypes[19].OneofWrappers = []any{} + file_org_apache_beam_model_fn_execution_v1_beam_fn_api_proto_msgTypes[26].OneofWrappers = []any{ + (*stateRequest_Get)(nil), + (*stateRequest_Append)(nil), + (*stateRequest_Clear)(nil), + } + file_org_apache_beam_model_fn_execution_v1_beam_fn_api_proto_msgTypes[27].OneofWrappers = []any{ + (*stateResponse_Get)(nil), + (*stateResponse_Append)(nil), + (*stateResponse_Clear)(nil), + } + file_org_apache_beam_model_fn_execution_v1_beam_fn_api_proto_msgTypes[28].OneofWrappers = []any{ + (*stateKey_Runner_)(nil), + (*stateKey_MultimapSideInput_)(nil), + (*stateKey_BagUserState_)(nil), + (*stateKey_IterableSideInput_)(nil), + (*stateKey_MultimapKeysSideInput_)(nil), + (*stateKey_MultimapKeysValuesSideInput_)(nil), + (*stateKey_MultimapKeysUserState_)(nil), + (*stateKey_MultimapUserState_)(nil), + (*stateKey_OrderedListUserState_)(nil), + } + file_org_apache_beam_model_fn_execution_v1_beam_fn_api_proto_msgTypes[54].OneofWrappers = []any{ + (*processBundleRequest_CacheToken_UserState_)(nil), + (*processBundleRequest_CacheToken_SideInput_)(nil), + } + type x struct{} + out := protoimpl.TypeBuilder{ + File: protoimpl.DescBuilder{ + GoPackagePath: reflect.TypeOf(x{}).PkgPath(), + RawDescriptor: file_org_apache_beam_model_fn_execution_v1_beam_fn_api_proto_rawDesc, + NumEnums: 2, + NumMessages: 77, + NumExtensions: 0, + NumServices: 6, + }, + GoTypes: file_org_apache_beam_model_fn_execution_v1_beam_fn_api_proto_goTypes, + DependencyIndexes: file_org_apache_beam_model_fn_execution_v1_beam_fn_api_proto_depIdxs, + EnumInfos: file_org_apache_beam_model_fn_execution_v1_beam_fn_api_proto_enumTypes, + MessageInfos: file_org_apache_beam_model_fn_execution_v1_beam_fn_api_proto_msgTypes, + }.Build() + File_org_apache_beam_model_fn_execution_v1_beam_fn_api_proto = out.File + file_org_apache_beam_model_fn_execution_v1_beam_fn_api_proto_rawDesc = nil + file_org_apache_beam_model_fn_execution_v1_beam_fn_api_proto_goTypes = nil + file_org_apache_beam_model_fn_execution_v1_beam_fn_api_proto_depIdxs = nil +} diff --git a/sdks/go/pkg/beam/model/fnexecution_v1/beam_provision_api.pb.go b/sdks/go/pkg/beam/model/fnexecution_v1/beam_provision_api.pb.go index 1611f92a414b..3427dd3dcb30 100644 --- a/sdks/go/pkg/beam/model/fnexecution_v1/beam_provision_api.pb.go +++ b/sdks/go/pkg/beam/model/fnexecution_v1/beam_provision_api.pb.go @@ -21,10 +21,12 @@ // Code generated by protoc-gen-go. DO NOT EDIT. // versions: -// protoc-gen-go v1.27.1 +// protoc-gen-go v1.36.0 // protoc v5.27.3 // source: org/apache/beam/model/fn_execution/v1/beam_provision_api.proto +//go:build !protoopaque + package fnexecution_v1 import ( @@ -33,7 +35,6 @@ import ( protoimpl "google.golang.org/protobuf/runtime/protoimpl" structpb "google.golang.org/protobuf/types/known/structpb" reflect "reflect" - sync "sync" ) const ( @@ -45,18 +46,16 @@ const ( // A request to get the provision info of a SDK harness worker instance. type GetProvisionInfoRequest struct { - state protoimpl.MessageState - sizeCache protoimpl.SizeCache + state protoimpl.MessageState `protogen:"hybrid.v1"` unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache } func (x *GetProvisionInfoRequest) Reset() { *x = GetProvisionInfoRequest{} - if protoimpl.UnsafeEnabled { - mi := &file_org_apache_beam_model_fn_execution_v1_beam_provision_api_proto_msgTypes[0] - ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) - ms.StoreMessageInfo(mi) - } + mi := &file_org_apache_beam_model_fn_execution_v1_beam_provision_api_proto_msgTypes[0] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) } func (x *GetProvisionInfoRequest) String() string { @@ -67,7 +66,7 @@ func (*GetProvisionInfoRequest) ProtoMessage() {} func (x *GetProvisionInfoRequest) ProtoReflect() protoreflect.Message { mi := &file_org_apache_beam_model_fn_execution_v1_beam_provision_api_proto_msgTypes[0] - if protoimpl.UnsafeEnabled && x != nil { + if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { ms.StoreMessageInfo(mi) @@ -77,27 +76,31 @@ func (x *GetProvisionInfoRequest) ProtoReflect() protoreflect.Message { return mi.MessageOf(x) } -// Deprecated: Use GetProvisionInfoRequest.ProtoReflect.Descriptor instead. -func (*GetProvisionInfoRequest) Descriptor() ([]byte, []int) { - return file_org_apache_beam_model_fn_execution_v1_beam_provision_api_proto_rawDescGZIP(), []int{0} +type GetProvisionInfoRequest_builder struct { + _ [0]func() // Prevents comparability and use of unkeyed literals for the builder. + +} + +func (b0 GetProvisionInfoRequest_builder) Build() *GetProvisionInfoRequest { + m0 := &GetProvisionInfoRequest{} + b, x := &b0, m0 + _, _ = b, x + return m0 } // A response containing the provision info of a SDK harness worker instance. type GetProvisionInfoResponse struct { - state protoimpl.MessageState - sizeCache protoimpl.SizeCache + state protoimpl.MessageState `protogen:"hybrid.v1"` + Info *ProvisionInfo `protobuf:"bytes,1,opt,name=info,proto3" json:"info,omitempty"` unknownFields protoimpl.UnknownFields - - Info *ProvisionInfo `protobuf:"bytes,1,opt,name=info,proto3" json:"info,omitempty"` + sizeCache protoimpl.SizeCache } func (x *GetProvisionInfoResponse) Reset() { *x = GetProvisionInfoResponse{} - if protoimpl.UnsafeEnabled { - mi := &file_org_apache_beam_model_fn_execution_v1_beam_provision_api_proto_msgTypes[1] - ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) - ms.StoreMessageInfo(mi) - } + mi := &file_org_apache_beam_model_fn_execution_v1_beam_provision_api_proto_msgTypes[1] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) } func (x *GetProvisionInfoResponse) String() string { @@ -108,7 +111,7 @@ func (*GetProvisionInfoResponse) ProtoMessage() {} func (x *GetProvisionInfoResponse) ProtoReflect() protoreflect.Message { mi := &file_org_apache_beam_model_fn_execution_v1_beam_provision_api_proto_msgTypes[1] - if protoimpl.UnsafeEnabled && x != nil { + if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { ms.StoreMessageInfo(mi) @@ -118,11 +121,6 @@ func (x *GetProvisionInfoResponse) ProtoReflect() protoreflect.Message { return mi.MessageOf(x) } -// Deprecated: Use GetProvisionInfoResponse.ProtoReflect.Descriptor instead. -func (*GetProvisionInfoResponse) Descriptor() ([]byte, []int) { - return file_org_apache_beam_model_fn_execution_v1_beam_provision_api_proto_rawDescGZIP(), []int{1} -} - func (x *GetProvisionInfoResponse) GetInfo() *ProvisionInfo { if x != nil { return x.Info @@ -130,13 +128,39 @@ func (x *GetProvisionInfoResponse) GetInfo() *ProvisionInfo { return nil } +func (x *GetProvisionInfoResponse) SetInfo(v *ProvisionInfo) { + x.Info = v +} + +func (x *GetProvisionInfoResponse) HasInfo() bool { + if x == nil { + return false + } + return x.Info != nil +} + +func (x *GetProvisionInfoResponse) ClearInfo() { + x.Info = nil +} + +type GetProvisionInfoResponse_builder struct { + _ [0]func() // Prevents comparability and use of unkeyed literals for the builder. + + Info *ProvisionInfo +} + +func (b0 GetProvisionInfoResponse_builder) Build() *GetProvisionInfoResponse { + m0 := &GetProvisionInfoResponse{} + b, x := &b0, m0 + _, _ = b, x + x.Info = b.Info + return m0 +} + // Runtime provisioning information for a SDK harness worker instance, // such as pipeline options, resource constraints and other job metadata type ProvisionInfo struct { - state protoimpl.MessageState - sizeCache protoimpl.SizeCache - unknownFields protoimpl.UnknownFields - + state protoimpl.MessageState `protogen:"hybrid.v1"` // (required) Pipeline options. For non-template jobs, the options are // identical to what is passed to job submission. PipelineOptions *structpb.Struct `protobuf:"bytes,3,opt,name=pipeline_options,json=pipelineOptions,proto3" json:"pipeline_options,omitempty"` @@ -161,20 +185,20 @@ type ProvisionInfo struct { RunnerCapabilities []string `protobuf:"bytes,12,rep,name=runner_capabilities,json=runnerCapabilities,proto3" json:"runner_capabilities,omitempty"` // (optional) Runtime environment metadata that are static throughout the // pipeline execution. - Metadata map[string]string `protobuf:"bytes,13,rep,name=metadata,proto3" json:"metadata,omitempty" protobuf_key:"bytes,1,opt,name=key,proto3" protobuf_val:"bytes,2,opt,name=value,proto3"` + Metadata map[string]string `protobuf:"bytes,13,rep,name=metadata,proto3" json:"metadata,omitempty" protobuf_key:"bytes,1,opt,name=key" protobuf_val:"bytes,2,opt,name=value"` // (optional) If this environment supports SIBLING_WORKERS, used to indicate // the ids of sibling workers, if any, that should be started in addition // to this worker (which already has its own worker id). SiblingWorkerIds []string `protobuf:"bytes,14,rep,name=sibling_worker_ids,json=siblingWorkerIds,proto3" json:"sibling_worker_ids,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache } func (x *ProvisionInfo) Reset() { *x = ProvisionInfo{} - if protoimpl.UnsafeEnabled { - mi := &file_org_apache_beam_model_fn_execution_v1_beam_provision_api_proto_msgTypes[2] - ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) - ms.StoreMessageInfo(mi) - } + mi := &file_org_apache_beam_model_fn_execution_v1_beam_provision_api_proto_msgTypes[2] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) } func (x *ProvisionInfo) String() string { @@ -185,7 +209,7 @@ func (*ProvisionInfo) ProtoMessage() {} func (x *ProvisionInfo) ProtoReflect() protoreflect.Message { mi := &file_org_apache_beam_model_fn_execution_v1_beam_provision_api_proto_msgTypes[2] - if protoimpl.UnsafeEnabled && x != nil { + if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { ms.StoreMessageInfo(mi) @@ -195,11 +219,6 @@ func (x *ProvisionInfo) ProtoReflect() protoreflect.Message { return mi.MessageOf(x) } -// Deprecated: Use ProvisionInfo.ProtoReflect.Descriptor instead. -func (*ProvisionInfo) Descriptor() ([]byte, []int) { - return file_org_apache_beam_model_fn_execution_v1_beam_provision_api_proto_rawDescGZIP(), []int{2} -} - func (x *ProvisionInfo) GetPipelineOptions() *structpb.Struct { if x != nil { return x.PipelineOptions @@ -270,6 +289,152 @@ func (x *ProvisionInfo) GetSiblingWorkerIds() []string { return nil } +func (x *ProvisionInfo) SetPipelineOptions(v *structpb.Struct) { + x.PipelineOptions = v +} + +func (x *ProvisionInfo) SetRetrievalToken(v string) { + x.RetrievalToken = v +} + +func (x *ProvisionInfo) SetStatusEndpoint(v *pipeline_v1.ApiServiceDescriptor) { + x.StatusEndpoint = v +} + +func (x *ProvisionInfo) SetLoggingEndpoint(v *pipeline_v1.ApiServiceDescriptor) { + x.LoggingEndpoint = v +} + +func (x *ProvisionInfo) SetArtifactEndpoint(v *pipeline_v1.ApiServiceDescriptor) { + x.ArtifactEndpoint = v +} + +func (x *ProvisionInfo) SetControlEndpoint(v *pipeline_v1.ApiServiceDescriptor) { + x.ControlEndpoint = v +} + +func (x *ProvisionInfo) SetDependencies(v []*pipeline_v1.ArtifactInformation) { + x.Dependencies = v +} + +func (x *ProvisionInfo) SetRunnerCapabilities(v []string) { + x.RunnerCapabilities = v +} + +func (x *ProvisionInfo) SetMetadata(v map[string]string) { + x.Metadata = v +} + +func (x *ProvisionInfo) SetSiblingWorkerIds(v []string) { + x.SiblingWorkerIds = v +} + +func (x *ProvisionInfo) HasPipelineOptions() bool { + if x == nil { + return false + } + return x.PipelineOptions != nil +} + +func (x *ProvisionInfo) HasStatusEndpoint() bool { + if x == nil { + return false + } + return x.StatusEndpoint != nil +} + +func (x *ProvisionInfo) HasLoggingEndpoint() bool { + if x == nil { + return false + } + return x.LoggingEndpoint != nil +} + +func (x *ProvisionInfo) HasArtifactEndpoint() bool { + if x == nil { + return false + } + return x.ArtifactEndpoint != nil +} + +func (x *ProvisionInfo) HasControlEndpoint() bool { + if x == nil { + return false + } + return x.ControlEndpoint != nil +} + +func (x *ProvisionInfo) ClearPipelineOptions() { + x.PipelineOptions = nil +} + +func (x *ProvisionInfo) ClearStatusEndpoint() { + x.StatusEndpoint = nil +} + +func (x *ProvisionInfo) ClearLoggingEndpoint() { + x.LoggingEndpoint = nil +} + +func (x *ProvisionInfo) ClearArtifactEndpoint() { + x.ArtifactEndpoint = nil +} + +func (x *ProvisionInfo) ClearControlEndpoint() { + x.ControlEndpoint = nil +} + +type ProvisionInfo_builder struct { + _ [0]func() // Prevents comparability and use of unkeyed literals for the builder. + + // (required) Pipeline options. For non-template jobs, the options are + // identical to what is passed to job submission. + PipelineOptions *structpb.Struct + // (required) The artifact retrieval token produced by + // LegacyArtifactStagingService.CommitManifestResponse. + RetrievalToken string + // (optional) The endpoint that the runner is hosting for the SDK to submit + // status reports to during pipeline execution. This field will only be + // populated if the runner supports SDK status reports. For more details see + // https://s.apache.org/beam-fn-api-harness-status + StatusEndpoint *pipeline_v1.ApiServiceDescriptor + // (optional) The logging endpoint this SDK should use. + LoggingEndpoint *pipeline_v1.ApiServiceDescriptor + // (optional) The artifact retrieval endpoint this SDK should use. + ArtifactEndpoint *pipeline_v1.ApiServiceDescriptor + // (optional) The control endpoint this SDK should use. + ControlEndpoint *pipeline_v1.ApiServiceDescriptor + // The set of dependencies that should be staged into this environment. + Dependencies []*pipeline_v1.ArtifactInformation + // (optional) A set of capabilities that this SDK is allowed to use in its + // interactions with this runner. + RunnerCapabilities []string + // (optional) Runtime environment metadata that are static throughout the + // pipeline execution. + Metadata map[string]string + // (optional) If this environment supports SIBLING_WORKERS, used to indicate + // the ids of sibling workers, if any, that should be started in addition + // to this worker (which already has its own worker id). + SiblingWorkerIds []string +} + +func (b0 ProvisionInfo_builder) Build() *ProvisionInfo { + m0 := &ProvisionInfo{} + b, x := &b0, m0 + _, _ = b, x + x.PipelineOptions = b.PipelineOptions + x.RetrievalToken = b.RetrievalToken + x.StatusEndpoint = b.StatusEndpoint + x.LoggingEndpoint = b.LoggingEndpoint + x.ArtifactEndpoint = b.ArtifactEndpoint + x.ControlEndpoint = b.ControlEndpoint + x.Dependencies = b.Dependencies + x.RunnerCapabilities = b.RunnerCapabilities + x.Metadata = b.Metadata + x.SiblingWorkerIds = b.SiblingWorkerIds + return m0 +} + var File_org_apache_beam_model_fn_execution_v1_beam_provision_api_proto protoreflect.FileDescriptor var file_org_apache_beam_model_fn_execution_v1_beam_provision_api_proto_rawDesc = []byte{ @@ -373,20 +538,8 @@ var file_org_apache_beam_model_fn_execution_v1_beam_provision_api_proto_rawDesc 0x74, 0x6f, 0x33, } -var ( - file_org_apache_beam_model_fn_execution_v1_beam_provision_api_proto_rawDescOnce sync.Once - file_org_apache_beam_model_fn_execution_v1_beam_provision_api_proto_rawDescData = file_org_apache_beam_model_fn_execution_v1_beam_provision_api_proto_rawDesc -) - -func file_org_apache_beam_model_fn_execution_v1_beam_provision_api_proto_rawDescGZIP() []byte { - file_org_apache_beam_model_fn_execution_v1_beam_provision_api_proto_rawDescOnce.Do(func() { - file_org_apache_beam_model_fn_execution_v1_beam_provision_api_proto_rawDescData = protoimpl.X.CompressGZIP(file_org_apache_beam_model_fn_execution_v1_beam_provision_api_proto_rawDescData) - }) - return file_org_apache_beam_model_fn_execution_v1_beam_provision_api_proto_rawDescData -} - var file_org_apache_beam_model_fn_execution_v1_beam_provision_api_proto_msgTypes = make([]protoimpl.MessageInfo, 4) -var file_org_apache_beam_model_fn_execution_v1_beam_provision_api_proto_goTypes = []interface{}{ +var file_org_apache_beam_model_fn_execution_v1_beam_provision_api_proto_goTypes = []any{ (*GetProvisionInfoRequest)(nil), // 0: org.apache.beam.model.fn_execution.v1.GetProvisionInfoRequest (*GetProvisionInfoResponse)(nil), // 1: org.apache.beam.model.fn_execution.v1.GetProvisionInfoResponse (*ProvisionInfo)(nil), // 2: org.apache.beam.model.fn_execution.v1.ProvisionInfo @@ -418,44 +571,6 @@ func file_org_apache_beam_model_fn_execution_v1_beam_provision_api_proto_init() if File_org_apache_beam_model_fn_execution_v1_beam_provision_api_proto != nil { return } - if !protoimpl.UnsafeEnabled { - file_org_apache_beam_model_fn_execution_v1_beam_provision_api_proto_msgTypes[0].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*GetProvisionInfoRequest); i { - case 0: - return &v.state - case 1: - return &v.sizeCache - case 2: - return &v.unknownFields - default: - return nil - } - } - file_org_apache_beam_model_fn_execution_v1_beam_provision_api_proto_msgTypes[1].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*GetProvisionInfoResponse); i { - case 0: - return &v.state - case 1: - return &v.sizeCache - case 2: - return &v.unknownFields - default: - return nil - } - } - file_org_apache_beam_model_fn_execution_v1_beam_provision_api_proto_msgTypes[2].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*ProvisionInfo); i { - case 0: - return &v.state - case 1: - return &v.sizeCache - case 2: - return &v.unknownFields - default: - return nil - } - } - } type x struct{} out := protoimpl.TypeBuilder{ File: protoimpl.DescBuilder{ diff --git a/sdks/go/pkg/beam/model/fnexecution_v1/beam_provision_api_grpc.pb.go b/sdks/go/pkg/beam/model/fnexecution_v1/beam_provision_api_grpc.pb.go index d9c247cda7b3..15909bcaffaa 100644 --- a/sdks/go/pkg/beam/model/fnexecution_v1/beam_provision_api_grpc.pb.go +++ b/sdks/go/pkg/beam/model/fnexecution_v1/beam_provision_api_grpc.pb.go @@ -14,9 +14,14 @@ // WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. // See the License for the specific language governing permissions and // limitations under the License. + +// +// Protocol Buffers describing the Provision API, for communicating with a runner +// for job and environment provisioning information over GRPC. + // Code generated by protoc-gen-go-grpc. DO NOT EDIT. // versions: -// - protoc-gen-go-grpc v1.1.0 +// - protoc-gen-go-grpc v1.5.1 // - protoc v5.27.3 // source: org/apache/beam/model/fn_execution/v1/beam_provision_api.proto @@ -31,12 +36,20 @@ import ( // This is a compile-time assertion to ensure that this generated file // is compatible with the grpc package it is being compiled against. -// Requires gRPC-Go v1.32.0 or later. -const _ = grpc.SupportPackageIsVersion7 +// Requires gRPC-Go v1.64.0 or later. +const _ = grpc.SupportPackageIsVersion9 + +const ( + ProvisionService_GetProvisionInfo_FullMethodName = "/org.apache.beam.model.fn_execution.v1.ProvisionService/GetProvisionInfo" +) // ProvisionServiceClient is the client API for ProvisionService service. // // For semantics around ctx use and closing/ending streaming RPCs, please refer to https://pkg.go.dev/google.golang.org/grpc/?tab=doc#ClientConn.NewStream. +// +// A service to provide runtime provisioning information to the SDK harness +// worker instances -- such as pipeline options, resource constraints and +// other job metadata -- needed by an SDK harness instance to initialize. type ProvisionServiceClient interface { // Get provision information for the SDK harness worker instance. GetProvisionInfo(ctx context.Context, in *GetProvisionInfoRequest, opts ...grpc.CallOption) (*GetProvisionInfoResponse, error) @@ -51,8 +64,9 @@ func NewProvisionServiceClient(cc grpc.ClientConnInterface) ProvisionServiceClie } func (c *provisionServiceClient) GetProvisionInfo(ctx context.Context, in *GetProvisionInfoRequest, opts ...grpc.CallOption) (*GetProvisionInfoResponse, error) { + cOpts := append([]grpc.CallOption{grpc.StaticMethod()}, opts...) out := new(GetProvisionInfoResponse) - err := c.cc.Invoke(ctx, "/org.apache.beam.model.fn_execution.v1.ProvisionService/GetProvisionInfo", in, out, opts...) + err := c.cc.Invoke(ctx, ProvisionService_GetProvisionInfo_FullMethodName, in, out, cOpts...) if err != nil { return nil, err } @@ -61,21 +75,29 @@ func (c *provisionServiceClient) GetProvisionInfo(ctx context.Context, in *GetPr // ProvisionServiceServer is the server API for ProvisionService service. // All implementations must embed UnimplementedProvisionServiceServer -// for forward compatibility +// for forward compatibility. +// +// A service to provide runtime provisioning information to the SDK harness +// worker instances -- such as pipeline options, resource constraints and +// other job metadata -- needed by an SDK harness instance to initialize. type ProvisionServiceServer interface { // Get provision information for the SDK harness worker instance. GetProvisionInfo(context.Context, *GetProvisionInfoRequest) (*GetProvisionInfoResponse, error) mustEmbedUnimplementedProvisionServiceServer() } -// UnimplementedProvisionServiceServer must be embedded to have forward compatible implementations. -type UnimplementedProvisionServiceServer struct { -} +// UnimplementedProvisionServiceServer must be embedded to have +// forward compatible implementations. +// +// NOTE: this should be embedded by value instead of pointer to avoid a nil +// pointer dereference when methods are called. +type UnimplementedProvisionServiceServer struct{} func (UnimplementedProvisionServiceServer) GetProvisionInfo(context.Context, *GetProvisionInfoRequest) (*GetProvisionInfoResponse, error) { return nil, status.Errorf(codes.Unimplemented, "method GetProvisionInfo not implemented") } func (UnimplementedProvisionServiceServer) mustEmbedUnimplementedProvisionServiceServer() {} +func (UnimplementedProvisionServiceServer) testEmbeddedByValue() {} // UnsafeProvisionServiceServer may be embedded to opt out of forward compatibility for this service. // Use of this interface is not recommended, as added methods to ProvisionServiceServer will @@ -85,6 +107,13 @@ type UnsafeProvisionServiceServer interface { } func RegisterProvisionServiceServer(s grpc.ServiceRegistrar, srv ProvisionServiceServer) { + // If the following call pancis, it indicates UnimplementedProvisionServiceServer was + // embedded by pointer and is nil. This will cause panics if an + // unimplemented method is ever invoked, so we test this at initialization + // time to prevent it from happening at runtime later due to I/O. + if t, ok := srv.(interface{ testEmbeddedByValue() }); ok { + t.testEmbeddedByValue() + } s.RegisterService(&ProvisionService_ServiceDesc, srv) } @@ -98,7 +127,7 @@ func _ProvisionService_GetProvisionInfo_Handler(srv interface{}, ctx context.Con } info := &grpc.UnaryServerInfo{ Server: srv, - FullMethod: "/org.apache.beam.model.fn_execution.v1.ProvisionService/GetProvisionInfo", + FullMethod: ProvisionService_GetProvisionInfo_FullMethodName, } handler := func(ctx context.Context, req interface{}) (interface{}, error) { return srv.(ProvisionServiceServer).GetProvisionInfo(ctx, req.(*GetProvisionInfoRequest)) diff --git a/sdks/go/pkg/beam/model/fnexecution_v1/beam_provision_api_protoopaque.pb.go b/sdks/go/pkg/beam/model/fnexecution_v1/beam_provision_api_protoopaque.pb.go new file mode 100644 index 000000000000..b9f5838620ab --- /dev/null +++ b/sdks/go/pkg/beam/model/fnexecution_v1/beam_provision_api_protoopaque.pb.go @@ -0,0 +1,575 @@ +// +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +// +// Protocol Buffers describing the Provision API, for communicating with a runner +// for job and environment provisioning information over GRPC. + +// Code generated by protoc-gen-go. DO NOT EDIT. +// versions: +// protoc-gen-go v1.36.0 +// protoc v5.27.3 +// source: org/apache/beam/model/fn_execution/v1/beam_provision_api.proto + +//go:build protoopaque + +package fnexecution_v1 + +import ( + pipeline_v1 "github.com/apache/beam/sdks/v2/go/pkg/beam/model/pipeline_v1" + protoreflect "google.golang.org/protobuf/reflect/protoreflect" + protoimpl "google.golang.org/protobuf/runtime/protoimpl" + structpb "google.golang.org/protobuf/types/known/structpb" + reflect "reflect" +) + +const ( + // Verify that this generated code is sufficiently up-to-date. + _ = protoimpl.EnforceVersion(20 - protoimpl.MinVersion) + // Verify that runtime/protoimpl is sufficiently up-to-date. + _ = protoimpl.EnforceVersion(protoimpl.MaxVersion - 20) +) + +// A request to get the provision info of a SDK harness worker instance. +type GetProvisionInfoRequest struct { + state protoimpl.MessageState `protogen:"opaque.v1"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *GetProvisionInfoRequest) Reset() { + *x = GetProvisionInfoRequest{} + mi := &file_org_apache_beam_model_fn_execution_v1_beam_provision_api_proto_msgTypes[0] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *GetProvisionInfoRequest) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*GetProvisionInfoRequest) ProtoMessage() {} + +func (x *GetProvisionInfoRequest) ProtoReflect() protoreflect.Message { + mi := &file_org_apache_beam_model_fn_execution_v1_beam_provision_api_proto_msgTypes[0] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +type GetProvisionInfoRequest_builder struct { + _ [0]func() // Prevents comparability and use of unkeyed literals for the builder. + +} + +func (b0 GetProvisionInfoRequest_builder) Build() *GetProvisionInfoRequest { + m0 := &GetProvisionInfoRequest{} + b, x := &b0, m0 + _, _ = b, x + return m0 +} + +// A response containing the provision info of a SDK harness worker instance. +type GetProvisionInfoResponse struct { + state protoimpl.MessageState `protogen:"opaque.v1"` + xxx_hidden_Info *ProvisionInfo `protobuf:"bytes,1,opt,name=info,proto3" json:"info,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *GetProvisionInfoResponse) Reset() { + *x = GetProvisionInfoResponse{} + mi := &file_org_apache_beam_model_fn_execution_v1_beam_provision_api_proto_msgTypes[1] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *GetProvisionInfoResponse) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*GetProvisionInfoResponse) ProtoMessage() {} + +func (x *GetProvisionInfoResponse) ProtoReflect() protoreflect.Message { + mi := &file_org_apache_beam_model_fn_execution_v1_beam_provision_api_proto_msgTypes[1] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +func (x *GetProvisionInfoResponse) GetInfo() *ProvisionInfo { + if x != nil { + return x.xxx_hidden_Info + } + return nil +} + +func (x *GetProvisionInfoResponse) SetInfo(v *ProvisionInfo) { + x.xxx_hidden_Info = v +} + +func (x *GetProvisionInfoResponse) HasInfo() bool { + if x == nil { + return false + } + return x.xxx_hidden_Info != nil +} + +func (x *GetProvisionInfoResponse) ClearInfo() { + x.xxx_hidden_Info = nil +} + +type GetProvisionInfoResponse_builder struct { + _ [0]func() // Prevents comparability and use of unkeyed literals for the builder. + + Info *ProvisionInfo +} + +func (b0 GetProvisionInfoResponse_builder) Build() *GetProvisionInfoResponse { + m0 := &GetProvisionInfoResponse{} + b, x := &b0, m0 + _, _ = b, x + x.xxx_hidden_Info = b.Info + return m0 +} + +// Runtime provisioning information for a SDK harness worker instance, +// such as pipeline options, resource constraints and other job metadata +type ProvisionInfo struct { + state protoimpl.MessageState `protogen:"opaque.v1"` + xxx_hidden_PipelineOptions *structpb.Struct `protobuf:"bytes,3,opt,name=pipeline_options,json=pipelineOptions,proto3" json:"pipeline_options,omitempty"` + xxx_hidden_RetrievalToken string `protobuf:"bytes,6,opt,name=retrieval_token,json=retrievalToken,proto3" json:"retrieval_token,omitempty"` + xxx_hidden_StatusEndpoint *pipeline_v1.ApiServiceDescriptor `protobuf:"bytes,7,opt,name=status_endpoint,json=statusEndpoint,proto3" json:"status_endpoint,omitempty"` + xxx_hidden_LoggingEndpoint *pipeline_v1.ApiServiceDescriptor `protobuf:"bytes,8,opt,name=logging_endpoint,json=loggingEndpoint,proto3" json:"logging_endpoint,omitempty"` + xxx_hidden_ArtifactEndpoint *pipeline_v1.ApiServiceDescriptor `protobuf:"bytes,9,opt,name=artifact_endpoint,json=artifactEndpoint,proto3" json:"artifact_endpoint,omitempty"` + xxx_hidden_ControlEndpoint *pipeline_v1.ApiServiceDescriptor `protobuf:"bytes,10,opt,name=control_endpoint,json=controlEndpoint,proto3" json:"control_endpoint,omitempty"` + xxx_hidden_Dependencies *[]*pipeline_v1.ArtifactInformation `protobuf:"bytes,11,rep,name=dependencies,proto3" json:"dependencies,omitempty"` + xxx_hidden_RunnerCapabilities []string `protobuf:"bytes,12,rep,name=runner_capabilities,json=runnerCapabilities,proto3" json:"runner_capabilities,omitempty"` + xxx_hidden_Metadata map[string]string `protobuf:"bytes,13,rep,name=metadata,proto3" json:"metadata,omitempty" protobuf_key:"bytes,1,opt,name=key" protobuf_val:"bytes,2,opt,name=value"` + xxx_hidden_SiblingWorkerIds []string `protobuf:"bytes,14,rep,name=sibling_worker_ids,json=siblingWorkerIds,proto3" json:"sibling_worker_ids,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *ProvisionInfo) Reset() { + *x = ProvisionInfo{} + mi := &file_org_apache_beam_model_fn_execution_v1_beam_provision_api_proto_msgTypes[2] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *ProvisionInfo) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*ProvisionInfo) ProtoMessage() {} + +func (x *ProvisionInfo) ProtoReflect() protoreflect.Message { + mi := &file_org_apache_beam_model_fn_execution_v1_beam_provision_api_proto_msgTypes[2] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +func (x *ProvisionInfo) GetPipelineOptions() *structpb.Struct { + if x != nil { + return x.xxx_hidden_PipelineOptions + } + return nil +} + +func (x *ProvisionInfo) GetRetrievalToken() string { + if x != nil { + return x.xxx_hidden_RetrievalToken + } + return "" +} + +func (x *ProvisionInfo) GetStatusEndpoint() *pipeline_v1.ApiServiceDescriptor { + if x != nil { + return x.xxx_hidden_StatusEndpoint + } + return nil +} + +func (x *ProvisionInfo) GetLoggingEndpoint() *pipeline_v1.ApiServiceDescriptor { + if x != nil { + return x.xxx_hidden_LoggingEndpoint + } + return nil +} + +func (x *ProvisionInfo) GetArtifactEndpoint() *pipeline_v1.ApiServiceDescriptor { + if x != nil { + return x.xxx_hidden_ArtifactEndpoint + } + return nil +} + +func (x *ProvisionInfo) GetControlEndpoint() *pipeline_v1.ApiServiceDescriptor { + if x != nil { + return x.xxx_hidden_ControlEndpoint + } + return nil +} + +func (x *ProvisionInfo) GetDependencies() []*pipeline_v1.ArtifactInformation { + if x != nil { + if x.xxx_hidden_Dependencies != nil { + return *x.xxx_hidden_Dependencies + } + } + return nil +} + +func (x *ProvisionInfo) GetRunnerCapabilities() []string { + if x != nil { + return x.xxx_hidden_RunnerCapabilities + } + return nil +} + +func (x *ProvisionInfo) GetMetadata() map[string]string { + if x != nil { + return x.xxx_hidden_Metadata + } + return nil +} + +func (x *ProvisionInfo) GetSiblingWorkerIds() []string { + if x != nil { + return x.xxx_hidden_SiblingWorkerIds + } + return nil +} + +func (x *ProvisionInfo) SetPipelineOptions(v *structpb.Struct) { + x.xxx_hidden_PipelineOptions = v +} + +func (x *ProvisionInfo) SetRetrievalToken(v string) { + x.xxx_hidden_RetrievalToken = v +} + +func (x *ProvisionInfo) SetStatusEndpoint(v *pipeline_v1.ApiServiceDescriptor) { + x.xxx_hidden_StatusEndpoint = v +} + +func (x *ProvisionInfo) SetLoggingEndpoint(v *pipeline_v1.ApiServiceDescriptor) { + x.xxx_hidden_LoggingEndpoint = v +} + +func (x *ProvisionInfo) SetArtifactEndpoint(v *pipeline_v1.ApiServiceDescriptor) { + x.xxx_hidden_ArtifactEndpoint = v +} + +func (x *ProvisionInfo) SetControlEndpoint(v *pipeline_v1.ApiServiceDescriptor) { + x.xxx_hidden_ControlEndpoint = v +} + +func (x *ProvisionInfo) SetDependencies(v []*pipeline_v1.ArtifactInformation) { + x.xxx_hidden_Dependencies = &v +} + +func (x *ProvisionInfo) SetRunnerCapabilities(v []string) { + x.xxx_hidden_RunnerCapabilities = v +} + +func (x *ProvisionInfo) SetMetadata(v map[string]string) { + x.xxx_hidden_Metadata = v +} + +func (x *ProvisionInfo) SetSiblingWorkerIds(v []string) { + x.xxx_hidden_SiblingWorkerIds = v +} + +func (x *ProvisionInfo) HasPipelineOptions() bool { + if x == nil { + return false + } + return x.xxx_hidden_PipelineOptions != nil +} + +func (x *ProvisionInfo) HasStatusEndpoint() bool { + if x == nil { + return false + } + return x.xxx_hidden_StatusEndpoint != nil +} + +func (x *ProvisionInfo) HasLoggingEndpoint() bool { + if x == nil { + return false + } + return x.xxx_hidden_LoggingEndpoint != nil +} + +func (x *ProvisionInfo) HasArtifactEndpoint() bool { + if x == nil { + return false + } + return x.xxx_hidden_ArtifactEndpoint != nil +} + +func (x *ProvisionInfo) HasControlEndpoint() bool { + if x == nil { + return false + } + return x.xxx_hidden_ControlEndpoint != nil +} + +func (x *ProvisionInfo) ClearPipelineOptions() { + x.xxx_hidden_PipelineOptions = nil +} + +func (x *ProvisionInfo) ClearStatusEndpoint() { + x.xxx_hidden_StatusEndpoint = nil +} + +func (x *ProvisionInfo) ClearLoggingEndpoint() { + x.xxx_hidden_LoggingEndpoint = nil +} + +func (x *ProvisionInfo) ClearArtifactEndpoint() { + x.xxx_hidden_ArtifactEndpoint = nil +} + +func (x *ProvisionInfo) ClearControlEndpoint() { + x.xxx_hidden_ControlEndpoint = nil +} + +type ProvisionInfo_builder struct { + _ [0]func() // Prevents comparability and use of unkeyed literals for the builder. + + // (required) Pipeline options. For non-template jobs, the options are + // identical to what is passed to job submission. + PipelineOptions *structpb.Struct + // (required) The artifact retrieval token produced by + // LegacyArtifactStagingService.CommitManifestResponse. + RetrievalToken string + // (optional) The endpoint that the runner is hosting for the SDK to submit + // status reports to during pipeline execution. This field will only be + // populated if the runner supports SDK status reports. For more details see + // https://s.apache.org/beam-fn-api-harness-status + StatusEndpoint *pipeline_v1.ApiServiceDescriptor + // (optional) The logging endpoint this SDK should use. + LoggingEndpoint *pipeline_v1.ApiServiceDescriptor + // (optional) The artifact retrieval endpoint this SDK should use. + ArtifactEndpoint *pipeline_v1.ApiServiceDescriptor + // (optional) The control endpoint this SDK should use. + ControlEndpoint *pipeline_v1.ApiServiceDescriptor + // The set of dependencies that should be staged into this environment. + Dependencies []*pipeline_v1.ArtifactInformation + // (optional) A set of capabilities that this SDK is allowed to use in its + // interactions with this runner. + RunnerCapabilities []string + // (optional) Runtime environment metadata that are static throughout the + // pipeline execution. + Metadata map[string]string + // (optional) If this environment supports SIBLING_WORKERS, used to indicate + // the ids of sibling workers, if any, that should be started in addition + // to this worker (which already has its own worker id). + SiblingWorkerIds []string +} + +func (b0 ProvisionInfo_builder) Build() *ProvisionInfo { + m0 := &ProvisionInfo{} + b, x := &b0, m0 + _, _ = b, x + x.xxx_hidden_PipelineOptions = b.PipelineOptions + x.xxx_hidden_RetrievalToken = b.RetrievalToken + x.xxx_hidden_StatusEndpoint = b.StatusEndpoint + x.xxx_hidden_LoggingEndpoint = b.LoggingEndpoint + x.xxx_hidden_ArtifactEndpoint = b.ArtifactEndpoint + x.xxx_hidden_ControlEndpoint = b.ControlEndpoint + x.xxx_hidden_Dependencies = &b.Dependencies + x.xxx_hidden_RunnerCapabilities = b.RunnerCapabilities + x.xxx_hidden_Metadata = b.Metadata + x.xxx_hidden_SiblingWorkerIds = b.SiblingWorkerIds + return m0 +} + +var File_org_apache_beam_model_fn_execution_v1_beam_provision_api_proto protoreflect.FileDescriptor + +var file_org_apache_beam_model_fn_execution_v1_beam_provision_api_proto_rawDesc = []byte{ + 0x0a, 0x3e, 0x6f, 0x72, 0x67, 0x2f, 0x61, 0x70, 0x61, 0x63, 0x68, 0x65, 0x2f, 0x62, 0x65, 0x61, + 0x6d, 0x2f, 0x6d, 0x6f, 0x64, 0x65, 0x6c, 0x2f, 0x66, 0x6e, 0x5f, 0x65, 0x78, 0x65, 0x63, 0x75, + 0x74, 0x69, 0x6f, 0x6e, 0x2f, 0x76, 0x31, 0x2f, 0x62, 0x65, 0x61, 0x6d, 0x5f, 0x70, 0x72, 0x6f, + 0x76, 0x69, 0x73, 0x69, 0x6f, 0x6e, 0x5f, 0x61, 0x70, 0x69, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, + 0x12, 0x25, 0x6f, 0x72, 0x67, 0x2e, 0x61, 0x70, 0x61, 0x63, 0x68, 0x65, 0x2e, 0x62, 0x65, 0x61, + 0x6d, 0x2e, 0x6d, 0x6f, 0x64, 0x65, 0x6c, 0x2e, 0x66, 0x6e, 0x5f, 0x65, 0x78, 0x65, 0x63, 0x75, + 0x74, 0x69, 0x6f, 0x6e, 0x2e, 0x76, 0x31, 0x1a, 0x37, 0x6f, 0x72, 0x67, 0x2f, 0x61, 0x70, 0x61, + 0x63, 0x68, 0x65, 0x2f, 0x62, 0x65, 0x61, 0x6d, 0x2f, 0x6d, 0x6f, 0x64, 0x65, 0x6c, 0x2f, 0x70, + 0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e, 0x65, 0x2f, 0x76, 0x31, 0x2f, 0x62, 0x65, 0x61, 0x6d, 0x5f, + 0x72, 0x75, 0x6e, 0x6e, 0x65, 0x72, 0x5f, 0x61, 0x70, 0x69, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, + 0x1a, 0x31, 0x6f, 0x72, 0x67, 0x2f, 0x61, 0x70, 0x61, 0x63, 0x68, 0x65, 0x2f, 0x62, 0x65, 0x61, + 0x6d, 0x2f, 0x6d, 0x6f, 0x64, 0x65, 0x6c, 0x2f, 0x70, 0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e, 0x65, + 0x2f, 0x76, 0x31, 0x2f, 0x65, 0x6e, 0x64, 0x70, 0x6f, 0x69, 0x6e, 0x74, 0x73, 0x2e, 0x70, 0x72, + 0x6f, 0x74, 0x6f, 0x1a, 0x1c, 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2f, 0x70, 0x72, 0x6f, 0x74, + 0x6f, 0x62, 0x75, 0x66, 0x2f, 0x73, 0x74, 0x72, 0x75, 0x63, 0x74, 0x2e, 0x70, 0x72, 0x6f, 0x74, + 0x6f, 0x22, 0x19, 0x0a, 0x17, 0x47, 0x65, 0x74, 0x50, 0x72, 0x6f, 0x76, 0x69, 0x73, 0x69, 0x6f, + 0x6e, 0x49, 0x6e, 0x66, 0x6f, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x22, 0x64, 0x0a, 0x18, + 0x47, 0x65, 0x74, 0x50, 0x72, 0x6f, 0x76, 0x69, 0x73, 0x69, 0x6f, 0x6e, 0x49, 0x6e, 0x66, 0x6f, + 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x48, 0x0a, 0x04, 0x69, 0x6e, 0x66, 0x6f, + 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x34, 0x2e, 0x6f, 0x72, 0x67, 0x2e, 0x61, 0x70, 0x61, + 0x63, 0x68, 0x65, 0x2e, 0x62, 0x65, 0x61, 0x6d, 0x2e, 0x6d, 0x6f, 0x64, 0x65, 0x6c, 0x2e, 0x66, + 0x6e, 0x5f, 0x65, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x2e, 0x76, 0x31, 0x2e, 0x50, + 0x72, 0x6f, 0x76, 0x69, 0x73, 0x69, 0x6f, 0x6e, 0x49, 0x6e, 0x66, 0x6f, 0x52, 0x04, 0x69, 0x6e, + 0x66, 0x6f, 0x22, 0xe4, 0x06, 0x0a, 0x0d, 0x50, 0x72, 0x6f, 0x76, 0x69, 0x73, 0x69, 0x6f, 0x6e, + 0x49, 0x6e, 0x66, 0x6f, 0x12, 0x42, 0x0a, 0x10, 0x70, 0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e, 0x65, + 0x5f, 0x6f, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x17, + 0x2e, 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66, + 0x2e, 0x53, 0x74, 0x72, 0x75, 0x63, 0x74, 0x52, 0x0f, 0x70, 0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e, + 0x65, 0x4f, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x12, 0x27, 0x0a, 0x0f, 0x72, 0x65, 0x74, 0x72, + 0x69, 0x65, 0x76, 0x61, 0x6c, 0x5f, 0x74, 0x6f, 0x6b, 0x65, 0x6e, 0x18, 0x06, 0x20, 0x01, 0x28, + 0x09, 0x52, 0x0e, 0x72, 0x65, 0x74, 0x72, 0x69, 0x65, 0x76, 0x61, 0x6c, 0x54, 0x6f, 0x6b, 0x65, + 0x6e, 0x12, 0x60, 0x0a, 0x0f, 0x73, 0x74, 0x61, 0x74, 0x75, 0x73, 0x5f, 0x65, 0x6e, 0x64, 0x70, + 0x6f, 0x69, 0x6e, 0x74, 0x18, 0x07, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x37, 0x2e, 0x6f, 0x72, 0x67, + 0x2e, 0x61, 0x70, 0x61, 0x63, 0x68, 0x65, 0x2e, 0x62, 0x65, 0x61, 0x6d, 0x2e, 0x6d, 0x6f, 0x64, + 0x65, 0x6c, 0x2e, 0x70, 0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x41, + 0x70, 0x69, 0x53, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, 0x44, 0x65, 0x73, 0x63, 0x72, 0x69, 0x70, + 0x74, 0x6f, 0x72, 0x52, 0x0e, 0x73, 0x74, 0x61, 0x74, 0x75, 0x73, 0x45, 0x6e, 0x64, 0x70, 0x6f, + 0x69, 0x6e, 0x74, 0x12, 0x62, 0x0a, 0x10, 0x6c, 0x6f, 0x67, 0x67, 0x69, 0x6e, 0x67, 0x5f, 0x65, + 0x6e, 0x64, 0x70, 0x6f, 0x69, 0x6e, 0x74, 0x18, 0x08, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x37, 0x2e, + 0x6f, 0x72, 0x67, 0x2e, 0x61, 0x70, 0x61, 0x63, 0x68, 0x65, 0x2e, 0x62, 0x65, 0x61, 0x6d, 0x2e, + 0x6d, 0x6f, 0x64, 0x65, 0x6c, 0x2e, 0x70, 0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e, 0x65, 0x2e, 0x76, + 0x31, 0x2e, 0x41, 0x70, 0x69, 0x53, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, 0x44, 0x65, 0x73, 0x63, + 0x72, 0x69, 0x70, 0x74, 0x6f, 0x72, 0x52, 0x0f, 0x6c, 0x6f, 0x67, 0x67, 0x69, 0x6e, 0x67, 0x45, + 0x6e, 0x64, 0x70, 0x6f, 0x69, 0x6e, 0x74, 0x12, 0x64, 0x0a, 0x11, 0x61, 0x72, 0x74, 0x69, 0x66, + 0x61, 0x63, 0x74, 0x5f, 0x65, 0x6e, 0x64, 0x70, 0x6f, 0x69, 0x6e, 0x74, 0x18, 0x09, 0x20, 0x01, + 0x28, 0x0b, 0x32, 0x37, 0x2e, 0x6f, 0x72, 0x67, 0x2e, 0x61, 0x70, 0x61, 0x63, 0x68, 0x65, 0x2e, + 0x62, 0x65, 0x61, 0x6d, 0x2e, 0x6d, 0x6f, 0x64, 0x65, 0x6c, 0x2e, 0x70, 0x69, 0x70, 0x65, 0x6c, + 0x69, 0x6e, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x41, 0x70, 0x69, 0x53, 0x65, 0x72, 0x76, 0x69, 0x63, + 0x65, 0x44, 0x65, 0x73, 0x63, 0x72, 0x69, 0x70, 0x74, 0x6f, 0x72, 0x52, 0x10, 0x61, 0x72, 0x74, + 0x69, 0x66, 0x61, 0x63, 0x74, 0x45, 0x6e, 0x64, 0x70, 0x6f, 0x69, 0x6e, 0x74, 0x12, 0x62, 0x0a, + 0x10, 0x63, 0x6f, 0x6e, 0x74, 0x72, 0x6f, 0x6c, 0x5f, 0x65, 0x6e, 0x64, 0x70, 0x6f, 0x69, 0x6e, + 0x74, 0x18, 0x0a, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x37, 0x2e, 0x6f, 0x72, 0x67, 0x2e, 0x61, 0x70, + 0x61, 0x63, 0x68, 0x65, 0x2e, 0x62, 0x65, 0x61, 0x6d, 0x2e, 0x6d, 0x6f, 0x64, 0x65, 0x6c, 0x2e, + 0x70, 0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x41, 0x70, 0x69, 0x53, + 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, 0x44, 0x65, 0x73, 0x63, 0x72, 0x69, 0x70, 0x74, 0x6f, 0x72, + 0x52, 0x0f, 0x63, 0x6f, 0x6e, 0x74, 0x72, 0x6f, 0x6c, 0x45, 0x6e, 0x64, 0x70, 0x6f, 0x69, 0x6e, + 0x74, 0x12, 0x5a, 0x0a, 0x0c, 0x64, 0x65, 0x70, 0x65, 0x6e, 0x64, 0x65, 0x6e, 0x63, 0x69, 0x65, + 0x73, 0x18, 0x0b, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x36, 0x2e, 0x6f, 0x72, 0x67, 0x2e, 0x61, 0x70, + 0x61, 0x63, 0x68, 0x65, 0x2e, 0x62, 0x65, 0x61, 0x6d, 0x2e, 0x6d, 0x6f, 0x64, 0x65, 0x6c, 0x2e, + 0x70, 0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x41, 0x72, 0x74, 0x69, + 0x66, 0x61, 0x63, 0x74, 0x49, 0x6e, 0x66, 0x6f, 0x72, 0x6d, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x52, + 0x0c, 0x64, 0x65, 0x70, 0x65, 0x6e, 0x64, 0x65, 0x6e, 0x63, 0x69, 0x65, 0x73, 0x12, 0x2f, 0x0a, + 0x13, 0x72, 0x75, 0x6e, 0x6e, 0x65, 0x72, 0x5f, 0x63, 0x61, 0x70, 0x61, 0x62, 0x69, 0x6c, 0x69, + 0x74, 0x69, 0x65, 0x73, 0x18, 0x0c, 0x20, 0x03, 0x28, 0x09, 0x52, 0x12, 0x72, 0x75, 0x6e, 0x6e, + 0x65, 0x72, 0x43, 0x61, 0x70, 0x61, 0x62, 0x69, 0x6c, 0x69, 0x74, 0x69, 0x65, 0x73, 0x12, 0x5e, + 0x0a, 0x08, 0x6d, 0x65, 0x74, 0x61, 0x64, 0x61, 0x74, 0x61, 0x18, 0x0d, 0x20, 0x03, 0x28, 0x0b, + 0x32, 0x42, 0x2e, 0x6f, 0x72, 0x67, 0x2e, 0x61, 0x70, 0x61, 0x63, 0x68, 0x65, 0x2e, 0x62, 0x65, + 0x61, 0x6d, 0x2e, 0x6d, 0x6f, 0x64, 0x65, 0x6c, 0x2e, 0x66, 0x6e, 0x5f, 0x65, 0x78, 0x65, 0x63, + 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x2e, 0x76, 0x31, 0x2e, 0x50, 0x72, 0x6f, 0x76, 0x69, 0x73, 0x69, + 0x6f, 0x6e, 0x49, 0x6e, 0x66, 0x6f, 0x2e, 0x4d, 0x65, 0x74, 0x61, 0x64, 0x61, 0x74, 0x61, 0x45, + 0x6e, 0x74, 0x72, 0x79, 0x52, 0x08, 0x6d, 0x65, 0x74, 0x61, 0x64, 0x61, 0x74, 0x61, 0x12, 0x2c, + 0x0a, 0x12, 0x73, 0x69, 0x62, 0x6c, 0x69, 0x6e, 0x67, 0x5f, 0x77, 0x6f, 0x72, 0x6b, 0x65, 0x72, + 0x5f, 0x69, 0x64, 0x73, 0x18, 0x0e, 0x20, 0x03, 0x28, 0x09, 0x52, 0x10, 0x73, 0x69, 0x62, 0x6c, + 0x69, 0x6e, 0x67, 0x57, 0x6f, 0x72, 0x6b, 0x65, 0x72, 0x49, 0x64, 0x73, 0x1a, 0x3b, 0x0a, 0x0d, + 0x4d, 0x65, 0x74, 0x61, 0x64, 0x61, 0x74, 0x61, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x12, 0x10, 0x0a, + 0x03, 0x6b, 0x65, 0x79, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x03, 0x6b, 0x65, 0x79, 0x12, + 0x14, 0x0a, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x05, + 0x76, 0x61, 0x6c, 0x75, 0x65, 0x3a, 0x02, 0x38, 0x01, 0x32, 0xa8, 0x01, 0x0a, 0x10, 0x50, 0x72, + 0x6f, 0x76, 0x69, 0x73, 0x69, 0x6f, 0x6e, 0x53, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, 0x12, 0x93, + 0x01, 0x0a, 0x10, 0x47, 0x65, 0x74, 0x50, 0x72, 0x6f, 0x76, 0x69, 0x73, 0x69, 0x6f, 0x6e, 0x49, + 0x6e, 0x66, 0x6f, 0x12, 0x3e, 0x2e, 0x6f, 0x72, 0x67, 0x2e, 0x61, 0x70, 0x61, 0x63, 0x68, 0x65, + 0x2e, 0x62, 0x65, 0x61, 0x6d, 0x2e, 0x6d, 0x6f, 0x64, 0x65, 0x6c, 0x2e, 0x66, 0x6e, 0x5f, 0x65, + 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x2e, 0x76, 0x31, 0x2e, 0x47, 0x65, 0x74, 0x50, + 0x72, 0x6f, 0x76, 0x69, 0x73, 0x69, 0x6f, 0x6e, 0x49, 0x6e, 0x66, 0x6f, 0x52, 0x65, 0x71, 0x75, + 0x65, 0x73, 0x74, 0x1a, 0x3f, 0x2e, 0x6f, 0x72, 0x67, 0x2e, 0x61, 0x70, 0x61, 0x63, 0x68, 0x65, + 0x2e, 0x62, 0x65, 0x61, 0x6d, 0x2e, 0x6d, 0x6f, 0x64, 0x65, 0x6c, 0x2e, 0x66, 0x6e, 0x5f, 0x65, + 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x2e, 0x76, 0x31, 0x2e, 0x47, 0x65, 0x74, 0x50, + 0x72, 0x6f, 0x76, 0x69, 0x73, 0x69, 0x6f, 0x6e, 0x49, 0x6e, 0x66, 0x6f, 0x52, 0x65, 0x73, 0x70, + 0x6f, 0x6e, 0x73, 0x65, 0x42, 0x84, 0x01, 0x0a, 0x24, 0x6f, 0x72, 0x67, 0x2e, 0x61, 0x70, 0x61, + 0x63, 0x68, 0x65, 0x2e, 0x62, 0x65, 0x61, 0x6d, 0x2e, 0x6d, 0x6f, 0x64, 0x65, 0x6c, 0x2e, 0x66, + 0x6e, 0x65, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x2e, 0x76, 0x31, 0x42, 0x0c, 0x50, + 0x72, 0x6f, 0x76, 0x69, 0x73, 0x69, 0x6f, 0x6e, 0x41, 0x70, 0x69, 0x5a, 0x4e, 0x67, 0x69, 0x74, + 0x68, 0x75, 0x62, 0x2e, 0x63, 0x6f, 0x6d, 0x2f, 0x61, 0x70, 0x61, 0x63, 0x68, 0x65, 0x2f, 0x62, + 0x65, 0x61, 0x6d, 0x2f, 0x73, 0x64, 0x6b, 0x73, 0x2f, 0x76, 0x32, 0x2f, 0x67, 0x6f, 0x2f, 0x70, + 0x6b, 0x67, 0x2f, 0x62, 0x65, 0x61, 0x6d, 0x2f, 0x6d, 0x6f, 0x64, 0x65, 0x6c, 0x2f, 0x66, 0x6e, + 0x65, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x76, 0x31, 0x3b, 0x66, 0x6e, 0x65, + 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x76, 0x31, 0x62, 0x06, 0x70, 0x72, 0x6f, + 0x74, 0x6f, 0x33, +} + +var file_org_apache_beam_model_fn_execution_v1_beam_provision_api_proto_msgTypes = make([]protoimpl.MessageInfo, 4) +var file_org_apache_beam_model_fn_execution_v1_beam_provision_api_proto_goTypes = []any{ + (*GetProvisionInfoRequest)(nil), // 0: org.apache.beam.model.fn_execution.v1.GetProvisionInfoRequest + (*GetProvisionInfoResponse)(nil), // 1: org.apache.beam.model.fn_execution.v1.GetProvisionInfoResponse + (*ProvisionInfo)(nil), // 2: org.apache.beam.model.fn_execution.v1.ProvisionInfo + nil, // 3: org.apache.beam.model.fn_execution.v1.ProvisionInfo.MetadataEntry + (*structpb.Struct)(nil), // 4: google.protobuf.Struct + (*pipeline_v1.ApiServiceDescriptor)(nil), // 5: org.apache.beam.model.pipeline.v1.ApiServiceDescriptor + (*pipeline_v1.ArtifactInformation)(nil), // 6: org.apache.beam.model.pipeline.v1.ArtifactInformation +} +var file_org_apache_beam_model_fn_execution_v1_beam_provision_api_proto_depIdxs = []int32{ + 2, // 0: org.apache.beam.model.fn_execution.v1.GetProvisionInfoResponse.info:type_name -> org.apache.beam.model.fn_execution.v1.ProvisionInfo + 4, // 1: org.apache.beam.model.fn_execution.v1.ProvisionInfo.pipeline_options:type_name -> google.protobuf.Struct + 5, // 2: org.apache.beam.model.fn_execution.v1.ProvisionInfo.status_endpoint:type_name -> org.apache.beam.model.pipeline.v1.ApiServiceDescriptor + 5, // 3: org.apache.beam.model.fn_execution.v1.ProvisionInfo.logging_endpoint:type_name -> org.apache.beam.model.pipeline.v1.ApiServiceDescriptor + 5, // 4: org.apache.beam.model.fn_execution.v1.ProvisionInfo.artifact_endpoint:type_name -> org.apache.beam.model.pipeline.v1.ApiServiceDescriptor + 5, // 5: org.apache.beam.model.fn_execution.v1.ProvisionInfo.control_endpoint:type_name -> org.apache.beam.model.pipeline.v1.ApiServiceDescriptor + 6, // 6: org.apache.beam.model.fn_execution.v1.ProvisionInfo.dependencies:type_name -> org.apache.beam.model.pipeline.v1.ArtifactInformation + 3, // 7: org.apache.beam.model.fn_execution.v1.ProvisionInfo.metadata:type_name -> org.apache.beam.model.fn_execution.v1.ProvisionInfo.MetadataEntry + 0, // 8: org.apache.beam.model.fn_execution.v1.ProvisionService.GetProvisionInfo:input_type -> org.apache.beam.model.fn_execution.v1.GetProvisionInfoRequest + 1, // 9: org.apache.beam.model.fn_execution.v1.ProvisionService.GetProvisionInfo:output_type -> org.apache.beam.model.fn_execution.v1.GetProvisionInfoResponse + 9, // [9:10] is the sub-list for method output_type + 8, // [8:9] is the sub-list for method input_type + 8, // [8:8] is the sub-list for extension type_name + 8, // [8:8] is the sub-list for extension extendee + 0, // [0:8] is the sub-list for field type_name +} + +func init() { file_org_apache_beam_model_fn_execution_v1_beam_provision_api_proto_init() } +func file_org_apache_beam_model_fn_execution_v1_beam_provision_api_proto_init() { + if File_org_apache_beam_model_fn_execution_v1_beam_provision_api_proto != nil { + return + } + type x struct{} + out := protoimpl.TypeBuilder{ + File: protoimpl.DescBuilder{ + GoPackagePath: reflect.TypeOf(x{}).PkgPath(), + RawDescriptor: file_org_apache_beam_model_fn_execution_v1_beam_provision_api_proto_rawDesc, + NumEnums: 0, + NumMessages: 4, + NumExtensions: 0, + NumServices: 1, + }, + GoTypes: file_org_apache_beam_model_fn_execution_v1_beam_provision_api_proto_goTypes, + DependencyIndexes: file_org_apache_beam_model_fn_execution_v1_beam_provision_api_proto_depIdxs, + MessageInfos: file_org_apache_beam_model_fn_execution_v1_beam_provision_api_proto_msgTypes, + }.Build() + File_org_apache_beam_model_fn_execution_v1_beam_provision_api_proto = out.File + file_org_apache_beam_model_fn_execution_v1_beam_provision_api_proto_rawDesc = nil + file_org_apache_beam_model_fn_execution_v1_beam_provision_api_proto_goTypes = nil + file_org_apache_beam_model_fn_execution_v1_beam_provision_api_proto_depIdxs = nil +} diff --git a/sdks/go/pkg/beam/model/jobmanagement_v1/beam_artifact_api.pb.go b/sdks/go/pkg/beam/model/jobmanagement_v1/beam_artifact_api.pb.go index 460d62b9fd11..39e833105b41 100644 --- a/sdks/go/pkg/beam/model/jobmanagement_v1/beam_artifact_api.pb.go +++ b/sdks/go/pkg/beam/model/jobmanagement_v1/beam_artifact_api.pb.go @@ -21,10 +21,12 @@ // Code generated by protoc-gen-go. DO NOT EDIT. // versions: -// protoc-gen-go v1.27.1 +// protoc-gen-go v1.36.0 // protoc v5.27.3 // source: org/apache/beam/model/job_management/v1/beam_artifact_api.proto +//go:build !protoopaque + package jobmanagement_v1 import ( @@ -32,7 +34,6 @@ import ( protoreflect "google.golang.org/protobuf/reflect/protoreflect" protoimpl "google.golang.org/protobuf/runtime/protoimpl" reflect "reflect" - sync "sync" ) const ( @@ -81,17 +82,9 @@ func (x CommitManifestResponse_Constants) Number() protoreflect.EnumNumber { return protoreflect.EnumNumber(x) } -// Deprecated: Use CommitManifestResponse_Constants.Descriptor instead. -func (CommitManifestResponse_Constants) EnumDescriptor() ([]byte, []int) { - return file_org_apache_beam_model_job_management_v1_beam_artifact_api_proto_rawDescGZIP(), []int{17, 0} -} - // A request for artifact resolution. type ResolveArtifactsRequest struct { - state protoimpl.MessageState - sizeCache protoimpl.SizeCache - unknownFields protoimpl.UnknownFields - + state protoimpl.MessageState `protogen:"hybrid.v1"` // An (ordered) set of artifacts to (jointly) resolve. Artifacts []*pipeline_v1.ArtifactInformation `protobuf:"bytes,1,rep,name=artifacts,proto3" json:"artifacts,omitempty"` // A set of artifact type urns that are understood by the requester. @@ -99,15 +92,15 @@ type ResolveArtifactsRequest struct { // but other URNs may be used as well with the understanding that they must // be fetch-able as bytes via GetArtifact. PreferredUrns []string `protobuf:"bytes,2,rep,name=preferred_urns,json=preferredUrns,proto3" json:"preferred_urns,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache } func (x *ResolveArtifactsRequest) Reset() { *x = ResolveArtifactsRequest{} - if protoimpl.UnsafeEnabled { - mi := &file_org_apache_beam_model_job_management_v1_beam_artifact_api_proto_msgTypes[0] - ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) - ms.StoreMessageInfo(mi) - } + mi := &file_org_apache_beam_model_job_management_v1_beam_artifact_api_proto_msgTypes[0] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) } func (x *ResolveArtifactsRequest) String() string { @@ -118,7 +111,7 @@ func (*ResolveArtifactsRequest) ProtoMessage() {} func (x *ResolveArtifactsRequest) ProtoReflect() protoreflect.Message { mi := &file_org_apache_beam_model_job_management_v1_beam_artifact_api_proto_msgTypes[0] - if protoimpl.UnsafeEnabled && x != nil { + if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { ms.StoreMessageInfo(mi) @@ -128,11 +121,6 @@ func (x *ResolveArtifactsRequest) ProtoReflect() protoreflect.Message { return mi.MessageOf(x) } -// Deprecated: Use ResolveArtifactsRequest.ProtoReflect.Descriptor instead. -func (*ResolveArtifactsRequest) Descriptor() ([]byte, []int) { - return file_org_apache_beam_model_job_management_v1_beam_artifact_api_proto_rawDescGZIP(), []int{0} -} - func (x *ResolveArtifactsRequest) GetArtifacts() []*pipeline_v1.ArtifactInformation { if x != nil { return x.Artifacts @@ -147,25 +135,51 @@ func (x *ResolveArtifactsRequest) GetPreferredUrns() []string { return nil } +func (x *ResolveArtifactsRequest) SetArtifacts(v []*pipeline_v1.ArtifactInformation) { + x.Artifacts = v +} + +func (x *ResolveArtifactsRequest) SetPreferredUrns(v []string) { + x.PreferredUrns = v +} + +type ResolveArtifactsRequest_builder struct { + _ [0]func() // Prevents comparability and use of unkeyed literals for the builder. + + // An (ordered) set of artifacts to (jointly) resolve. + Artifacts []*pipeline_v1.ArtifactInformation + // A set of artifact type urns that are understood by the requester. + // An attempt should be made to resolve the artifacts in terms of these URNs, + // but other URNs may be used as well with the understanding that they must + // be fetch-able as bytes via GetArtifact. + PreferredUrns []string +} + +func (b0 ResolveArtifactsRequest_builder) Build() *ResolveArtifactsRequest { + m0 := &ResolveArtifactsRequest{} + b, x := &b0, m0 + _, _ = b, x + x.Artifacts = b.Artifacts + x.PreferredUrns = b.PreferredUrns + return m0 +} + // A response for artifact resolution. type ResolveArtifactsResponse struct { - state protoimpl.MessageState - sizeCache protoimpl.SizeCache - unknownFields protoimpl.UnknownFields - + state protoimpl.MessageState `protogen:"hybrid.v1"` // A full (ordered) set of replacements for the set of requested artifacts, // preferably in terms of the requested type URNs. If there is no better // resolution, the original list is returned. - Replacements []*pipeline_v1.ArtifactInformation `protobuf:"bytes,1,rep,name=replacements,proto3" json:"replacements,omitempty"` + Replacements []*pipeline_v1.ArtifactInformation `protobuf:"bytes,1,rep,name=replacements,proto3" json:"replacements,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache } func (x *ResolveArtifactsResponse) Reset() { *x = ResolveArtifactsResponse{} - if protoimpl.UnsafeEnabled { - mi := &file_org_apache_beam_model_job_management_v1_beam_artifact_api_proto_msgTypes[1] - ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) - ms.StoreMessageInfo(mi) - } + mi := &file_org_apache_beam_model_job_management_v1_beam_artifact_api_proto_msgTypes[1] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) } func (x *ResolveArtifactsResponse) String() string { @@ -176,7 +190,7 @@ func (*ResolveArtifactsResponse) ProtoMessage() {} func (x *ResolveArtifactsResponse) ProtoReflect() protoreflect.Message { mi := &file_org_apache_beam_model_job_management_v1_beam_artifact_api_proto_msgTypes[1] - if protoimpl.UnsafeEnabled && x != nil { + if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { ms.StoreMessageInfo(mi) @@ -186,11 +200,6 @@ func (x *ResolveArtifactsResponse) ProtoReflect() protoreflect.Message { return mi.MessageOf(x) } -// Deprecated: Use ResolveArtifactsResponse.ProtoReflect.Descriptor instead. -func (*ResolveArtifactsResponse) Descriptor() ([]byte, []int) { - return file_org_apache_beam_model_job_management_v1_beam_artifact_api_proto_rawDescGZIP(), []int{1} -} - func (x *ResolveArtifactsResponse) GetReplacements() []*pipeline_v1.ArtifactInformation { if x != nil { return x.Replacements @@ -198,22 +207,40 @@ func (x *ResolveArtifactsResponse) GetReplacements() []*pipeline_v1.ArtifactInfo return nil } +func (x *ResolveArtifactsResponse) SetReplacements(v []*pipeline_v1.ArtifactInformation) { + x.Replacements = v +} + +type ResolveArtifactsResponse_builder struct { + _ [0]func() // Prevents comparability and use of unkeyed literals for the builder. + + // A full (ordered) set of replacements for the set of requested artifacts, + // preferably in terms of the requested type URNs. If there is no better + // resolution, the original list is returned. + Replacements []*pipeline_v1.ArtifactInformation +} + +func (b0 ResolveArtifactsResponse_builder) Build() *ResolveArtifactsResponse { + m0 := &ResolveArtifactsResponse{} + b, x := &b0, m0 + _, _ = b, x + x.Replacements = b.Replacements + return m0 +} + // A request to get an artifact. type GetArtifactRequest struct { - state protoimpl.MessageState - sizeCache protoimpl.SizeCache + state protoimpl.MessageState `protogen:"hybrid.v1"` + Artifact *pipeline_v1.ArtifactInformation `protobuf:"bytes,1,opt,name=artifact,proto3" json:"artifact,omitempty"` unknownFields protoimpl.UnknownFields - - Artifact *pipeline_v1.ArtifactInformation `protobuf:"bytes,1,opt,name=artifact,proto3" json:"artifact,omitempty"` + sizeCache protoimpl.SizeCache } func (x *GetArtifactRequest) Reset() { *x = GetArtifactRequest{} - if protoimpl.UnsafeEnabled { - mi := &file_org_apache_beam_model_job_management_v1_beam_artifact_api_proto_msgTypes[2] - ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) - ms.StoreMessageInfo(mi) - } + mi := &file_org_apache_beam_model_job_management_v1_beam_artifact_api_proto_msgTypes[2] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) } func (x *GetArtifactRequest) String() string { @@ -224,7 +251,7 @@ func (*GetArtifactRequest) ProtoMessage() {} func (x *GetArtifactRequest) ProtoReflect() protoreflect.Message { mi := &file_org_apache_beam_model_job_management_v1_beam_artifact_api_proto_msgTypes[2] - if protoimpl.UnsafeEnabled && x != nil { + if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { ms.StoreMessageInfo(mi) @@ -234,11 +261,6 @@ func (x *GetArtifactRequest) ProtoReflect() protoreflect.Message { return mi.MessageOf(x) } -// Deprecated: Use GetArtifactRequest.ProtoReflect.Descriptor instead. -func (*GetArtifactRequest) Descriptor() ([]byte, []int) { - return file_org_apache_beam_model_job_management_v1_beam_artifact_api_proto_rawDescGZIP(), []int{2} -} - func (x *GetArtifactRequest) GetArtifact() *pipeline_v1.ArtifactInformation { if x != nil { return x.Artifact @@ -246,22 +268,48 @@ func (x *GetArtifactRequest) GetArtifact() *pipeline_v1.ArtifactInformation { return nil } +func (x *GetArtifactRequest) SetArtifact(v *pipeline_v1.ArtifactInformation) { + x.Artifact = v +} + +func (x *GetArtifactRequest) HasArtifact() bool { + if x == nil { + return false + } + return x.Artifact != nil +} + +func (x *GetArtifactRequest) ClearArtifact() { + x.Artifact = nil +} + +type GetArtifactRequest_builder struct { + _ [0]func() // Prevents comparability and use of unkeyed literals for the builder. + + Artifact *pipeline_v1.ArtifactInformation +} + +func (b0 GetArtifactRequest_builder) Build() *GetArtifactRequest { + m0 := &GetArtifactRequest{} + b, x := &b0, m0 + _, _ = b, x + x.Artifact = b.Artifact + return m0 +} + // Part of a response to getting an artifact. type GetArtifactResponse struct { - state protoimpl.MessageState - sizeCache protoimpl.SizeCache + state protoimpl.MessageState `protogen:"hybrid.v1"` + Data []byte `protobuf:"bytes,1,opt,name=data,proto3" json:"data,omitempty"` unknownFields protoimpl.UnknownFields - - Data []byte `protobuf:"bytes,1,opt,name=data,proto3" json:"data,omitempty"` + sizeCache protoimpl.SizeCache } func (x *GetArtifactResponse) Reset() { *x = GetArtifactResponse{} - if protoimpl.UnsafeEnabled { - mi := &file_org_apache_beam_model_job_management_v1_beam_artifact_api_proto_msgTypes[3] - ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) - ms.StoreMessageInfo(mi) - } + mi := &file_org_apache_beam_model_job_management_v1_beam_artifact_api_proto_msgTypes[3] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) } func (x *GetArtifactResponse) String() string { @@ -272,7 +320,7 @@ func (*GetArtifactResponse) ProtoMessage() {} func (x *GetArtifactResponse) ProtoReflect() protoreflect.Message { mi := &file_org_apache_beam_model_job_management_v1_beam_artifact_api_proto_msgTypes[3] - if protoimpl.UnsafeEnabled && x != nil { + if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { ms.StoreMessageInfo(mi) @@ -282,11 +330,6 @@ func (x *GetArtifactResponse) ProtoReflect() protoreflect.Message { return mi.MessageOf(x) } -// Deprecated: Use GetArtifactResponse.ProtoReflect.Descriptor instead. -func (*GetArtifactResponse) Descriptor() ([]byte, []int) { - return file_org_apache_beam_model_job_management_v1_beam_artifact_api_proto_rawDescGZIP(), []int{3} -} - func (x *GetArtifactResponse) GetData() []byte { if x != nil { return x.Data @@ -294,26 +337,44 @@ func (x *GetArtifactResponse) GetData() []byte { return nil } +func (x *GetArtifactResponse) SetData(v []byte) { + if v == nil { + v = []byte{} + } + x.Data = v +} + +type GetArtifactResponse_builder struct { + _ [0]func() // Prevents comparability and use of unkeyed literals for the builder. + + Data []byte +} + +func (b0 GetArtifactResponse_builder) Build() *GetArtifactResponse { + m0 := &GetArtifactResponse{} + b, x := &b0, m0 + _, _ = b, x + x.Data = b.Data + return m0 +} + // Wraps an ArtifactRetrievalService request for use in ReverseArtifactRetrievalService. type ArtifactRequestWrapper struct { - state protoimpl.MessageState - sizeCache protoimpl.SizeCache - unknownFields protoimpl.UnknownFields - - // Types that are assignable to Request: + state protoimpl.MessageState `protogen:"hybrid.v1"` + // Types that are valid to be assigned to Request: // // *ArtifactRequestWrapper_ResolveArtifact // *ArtifactRequestWrapper_GetArtifact - Request isArtifactRequestWrapper_Request `protobuf_oneof:"request"` + Request isArtifactRequestWrapper_Request `protobuf_oneof:"request"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache } func (x *ArtifactRequestWrapper) Reset() { *x = ArtifactRequestWrapper{} - if protoimpl.UnsafeEnabled { - mi := &file_org_apache_beam_model_job_management_v1_beam_artifact_api_proto_msgTypes[4] - ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) - ms.StoreMessageInfo(mi) - } + mi := &file_org_apache_beam_model_job_management_v1_beam_artifact_api_proto_msgTypes[4] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) } func (x *ArtifactRequestWrapper) String() string { @@ -324,7 +385,7 @@ func (*ArtifactRequestWrapper) ProtoMessage() {} func (x *ArtifactRequestWrapper) ProtoReflect() protoreflect.Message { mi := &file_org_apache_beam_model_job_management_v1_beam_artifact_api_proto_msgTypes[4] - if protoimpl.UnsafeEnabled && x != nil { + if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { ms.StoreMessageInfo(mi) @@ -334,32 +395,136 @@ func (x *ArtifactRequestWrapper) ProtoReflect() protoreflect.Message { return mi.MessageOf(x) } -// Deprecated: Use ArtifactRequestWrapper.ProtoReflect.Descriptor instead. -func (*ArtifactRequestWrapper) Descriptor() ([]byte, []int) { - return file_org_apache_beam_model_job_management_v1_beam_artifact_api_proto_rawDescGZIP(), []int{4} -} - -func (m *ArtifactRequestWrapper) GetRequest() isArtifactRequestWrapper_Request { - if m != nil { - return m.Request +func (x *ArtifactRequestWrapper) GetRequest() isArtifactRequestWrapper_Request { + if x != nil { + return x.Request } return nil } func (x *ArtifactRequestWrapper) GetResolveArtifact() *ResolveArtifactsRequest { - if x, ok := x.GetRequest().(*ArtifactRequestWrapper_ResolveArtifact); ok { - return x.ResolveArtifact + if x != nil { + if x, ok := x.Request.(*ArtifactRequestWrapper_ResolveArtifact); ok { + return x.ResolveArtifact + } } return nil } func (x *ArtifactRequestWrapper) GetGetArtifact() *GetArtifactRequest { - if x, ok := x.GetRequest().(*ArtifactRequestWrapper_GetArtifact); ok { - return x.GetArtifact + if x != nil { + if x, ok := x.Request.(*ArtifactRequestWrapper_GetArtifact); ok { + return x.GetArtifact + } } return nil } +func (x *ArtifactRequestWrapper) SetResolveArtifact(v *ResolveArtifactsRequest) { + if v == nil { + x.Request = nil + return + } + x.Request = &ArtifactRequestWrapper_ResolveArtifact{v} +} + +func (x *ArtifactRequestWrapper) SetGetArtifact(v *GetArtifactRequest) { + if v == nil { + x.Request = nil + return + } + x.Request = &ArtifactRequestWrapper_GetArtifact{v} +} + +func (x *ArtifactRequestWrapper) HasRequest() bool { + if x == nil { + return false + } + return x.Request != nil +} + +func (x *ArtifactRequestWrapper) HasResolveArtifact() bool { + if x == nil { + return false + } + _, ok := x.Request.(*ArtifactRequestWrapper_ResolveArtifact) + return ok +} + +func (x *ArtifactRequestWrapper) HasGetArtifact() bool { + if x == nil { + return false + } + _, ok := x.Request.(*ArtifactRequestWrapper_GetArtifact) + return ok +} + +func (x *ArtifactRequestWrapper) ClearRequest() { + x.Request = nil +} + +func (x *ArtifactRequestWrapper) ClearResolveArtifact() { + if _, ok := x.Request.(*ArtifactRequestWrapper_ResolveArtifact); ok { + x.Request = nil + } +} + +func (x *ArtifactRequestWrapper) ClearGetArtifact() { + if _, ok := x.Request.(*ArtifactRequestWrapper_GetArtifact); ok { + x.Request = nil + } +} + +const ArtifactRequestWrapper_Request_not_set_case case_ArtifactRequestWrapper_Request = 0 +const ArtifactRequestWrapper_ResolveArtifact_case case_ArtifactRequestWrapper_Request = 1000 +const ArtifactRequestWrapper_GetArtifact_case case_ArtifactRequestWrapper_Request = 1001 + +func (x *ArtifactRequestWrapper) WhichRequest() case_ArtifactRequestWrapper_Request { + if x == nil { + return ArtifactRequestWrapper_Request_not_set_case + } + switch x.Request.(type) { + case *ArtifactRequestWrapper_ResolveArtifact: + return ArtifactRequestWrapper_ResolveArtifact_case + case *ArtifactRequestWrapper_GetArtifact: + return ArtifactRequestWrapper_GetArtifact_case + default: + return ArtifactRequestWrapper_Request_not_set_case + } +} + +type ArtifactRequestWrapper_builder struct { + _ [0]func() // Prevents comparability and use of unkeyed literals for the builder. + + // Fields of oneof Request: + ResolveArtifact *ResolveArtifactsRequest + GetArtifact *GetArtifactRequest + // -- end of Request +} + +func (b0 ArtifactRequestWrapper_builder) Build() *ArtifactRequestWrapper { + m0 := &ArtifactRequestWrapper{} + b, x := &b0, m0 + _, _ = b, x + if b.ResolveArtifact != nil { + x.Request = &ArtifactRequestWrapper_ResolveArtifact{b.ResolveArtifact} + } + if b.GetArtifact != nil { + x.Request = &ArtifactRequestWrapper_GetArtifact{b.GetArtifact} + } + return m0 +} + +type case_ArtifactRequestWrapper_Request protoreflect.FieldNumber + +func (x case_ArtifactRequestWrapper_Request) String() string { + md := file_org_apache_beam_model_job_management_v1_beam_artifact_api_proto_msgTypes[4].Descriptor() + if x == 0 { + return "not set" + } + return protoimpl.X.MessageFieldStringOf(md, protoreflect.FieldNumber(x)) +} + type isArtifactRequestWrapper_Request interface { isArtifactRequestWrapper_Request() } @@ -378,10 +543,7 @@ func (*ArtifactRequestWrapper_GetArtifact) isArtifactRequestWrapper_Request() {} // Wraps an ArtifactRetrievalService response for use in ReverseArtifactRetrievalService. type ArtifactResponseWrapper struct { - state protoimpl.MessageState - sizeCache protoimpl.SizeCache - unknownFields protoimpl.UnknownFields - + state protoimpl.MessageState `protogen:"hybrid.v1"` // A token indicating which job these artifacts are being staged for. StagingToken string `protobuf:"bytes,1,opt,name=staging_token,json=stagingToken,proto3" json:"staging_token,omitempty"` // Whether this is the last response for this request (for those responses that @@ -389,20 +551,20 @@ type ArtifactResponseWrapper struct { IsLast bool `protobuf:"varint,2,opt,name=is_last,json=isLast,proto3" json:"is_last,omitempty"` // The response itself. // - // Types that are assignable to Response: + // Types that are valid to be assigned to Response: // // *ArtifactResponseWrapper_ResolveArtifactResponse // *ArtifactResponseWrapper_GetArtifactResponse - Response isArtifactResponseWrapper_Response `protobuf_oneof:"response"` + Response isArtifactResponseWrapper_Response `protobuf_oneof:"response"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache } func (x *ArtifactResponseWrapper) Reset() { *x = ArtifactResponseWrapper{} - if protoimpl.UnsafeEnabled { - mi := &file_org_apache_beam_model_job_management_v1_beam_artifact_api_proto_msgTypes[5] - ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) - ms.StoreMessageInfo(mi) - } + mi := &file_org_apache_beam_model_job_management_v1_beam_artifact_api_proto_msgTypes[5] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) } func (x *ArtifactResponseWrapper) String() string { @@ -413,7 +575,7 @@ func (*ArtifactResponseWrapper) ProtoMessage() {} func (x *ArtifactResponseWrapper) ProtoReflect() protoreflect.Message { mi := &file_org_apache_beam_model_job_management_v1_beam_artifact_api_proto_msgTypes[5] - if protoimpl.UnsafeEnabled && x != nil { + if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { ms.StoreMessageInfo(mi) @@ -423,11 +585,6 @@ func (x *ArtifactResponseWrapper) ProtoReflect() protoreflect.Message { return mi.MessageOf(x) } -// Deprecated: Use ArtifactResponseWrapper.ProtoReflect.Descriptor instead. -func (*ArtifactResponseWrapper) Descriptor() ([]byte, []int) { - return file_org_apache_beam_model_job_management_v1_beam_artifact_api_proto_rawDescGZIP(), []int{5} -} - func (x *ArtifactResponseWrapper) GetStagingToken() string { if x != nil { return x.StagingToken @@ -442,27 +599,153 @@ func (x *ArtifactResponseWrapper) GetIsLast() bool { return false } -func (m *ArtifactResponseWrapper) GetResponse() isArtifactResponseWrapper_Response { - if m != nil { - return m.Response +func (x *ArtifactResponseWrapper) GetResponse() isArtifactResponseWrapper_Response { + if x != nil { + return x.Response } return nil } func (x *ArtifactResponseWrapper) GetResolveArtifactResponse() *ResolveArtifactsResponse { - if x, ok := x.GetResponse().(*ArtifactResponseWrapper_ResolveArtifactResponse); ok { - return x.ResolveArtifactResponse + if x != nil { + if x, ok := x.Response.(*ArtifactResponseWrapper_ResolveArtifactResponse); ok { + return x.ResolveArtifactResponse + } } return nil } func (x *ArtifactResponseWrapper) GetGetArtifactResponse() *GetArtifactResponse { - if x, ok := x.GetResponse().(*ArtifactResponseWrapper_GetArtifactResponse); ok { - return x.GetArtifactResponse + if x != nil { + if x, ok := x.Response.(*ArtifactResponseWrapper_GetArtifactResponse); ok { + return x.GetArtifactResponse + } } return nil } +func (x *ArtifactResponseWrapper) SetStagingToken(v string) { + x.StagingToken = v +} + +func (x *ArtifactResponseWrapper) SetIsLast(v bool) { + x.IsLast = v +} + +func (x *ArtifactResponseWrapper) SetResolveArtifactResponse(v *ResolveArtifactsResponse) { + if v == nil { + x.Response = nil + return + } + x.Response = &ArtifactResponseWrapper_ResolveArtifactResponse{v} +} + +func (x *ArtifactResponseWrapper) SetGetArtifactResponse(v *GetArtifactResponse) { + if v == nil { + x.Response = nil + return + } + x.Response = &ArtifactResponseWrapper_GetArtifactResponse{v} +} + +func (x *ArtifactResponseWrapper) HasResponse() bool { + if x == nil { + return false + } + return x.Response != nil +} + +func (x *ArtifactResponseWrapper) HasResolveArtifactResponse() bool { + if x == nil { + return false + } + _, ok := x.Response.(*ArtifactResponseWrapper_ResolveArtifactResponse) + return ok +} + +func (x *ArtifactResponseWrapper) HasGetArtifactResponse() bool { + if x == nil { + return false + } + _, ok := x.Response.(*ArtifactResponseWrapper_GetArtifactResponse) + return ok +} + +func (x *ArtifactResponseWrapper) ClearResponse() { + x.Response = nil +} + +func (x *ArtifactResponseWrapper) ClearResolveArtifactResponse() { + if _, ok := x.Response.(*ArtifactResponseWrapper_ResolveArtifactResponse); ok { + x.Response = nil + } +} + +func (x *ArtifactResponseWrapper) ClearGetArtifactResponse() { + if _, ok := x.Response.(*ArtifactResponseWrapper_GetArtifactResponse); ok { + x.Response = nil + } +} + +const ArtifactResponseWrapper_Response_not_set_case case_ArtifactResponseWrapper_Response = 0 +const ArtifactResponseWrapper_ResolveArtifactResponse_case case_ArtifactResponseWrapper_Response = 1000 +const ArtifactResponseWrapper_GetArtifactResponse_case case_ArtifactResponseWrapper_Response = 1001 + +func (x *ArtifactResponseWrapper) WhichResponse() case_ArtifactResponseWrapper_Response { + if x == nil { + return ArtifactResponseWrapper_Response_not_set_case + } + switch x.Response.(type) { + case *ArtifactResponseWrapper_ResolveArtifactResponse: + return ArtifactResponseWrapper_ResolveArtifactResponse_case + case *ArtifactResponseWrapper_GetArtifactResponse: + return ArtifactResponseWrapper_GetArtifactResponse_case + default: + return ArtifactResponseWrapper_Response_not_set_case + } +} + +type ArtifactResponseWrapper_builder struct { + _ [0]func() // Prevents comparability and use of unkeyed literals for the builder. + + // A token indicating which job these artifacts are being staged for. + StagingToken string + // Whether this is the last response for this request (for those responses that + // would typically be terminated by the end of the response stream.) + IsLast bool + // The response itself. + + // Fields of oneof Response: + ResolveArtifactResponse *ResolveArtifactsResponse + GetArtifactResponse *GetArtifactResponse + // -- end of Response +} + +func (b0 ArtifactResponseWrapper_builder) Build() *ArtifactResponseWrapper { + m0 := &ArtifactResponseWrapper{} + b, x := &b0, m0 + _, _ = b, x + x.StagingToken = b.StagingToken + x.IsLast = b.IsLast + if b.ResolveArtifactResponse != nil { + x.Response = &ArtifactResponseWrapper_ResolveArtifactResponse{b.ResolveArtifactResponse} + } + if b.GetArtifactResponse != nil { + x.Response = &ArtifactResponseWrapper_GetArtifactResponse{b.GetArtifactResponse} + } + return m0 +} + +type case_ArtifactResponseWrapper_Response protoreflect.FieldNumber + +func (x case_ArtifactResponseWrapper_Response) String() string { + md := file_org_apache_beam_model_job_management_v1_beam_artifact_api_proto_msgTypes[5].Descriptor() + if x == 0 { + return "not set" + } + return protoimpl.X.MessageFieldStringOf(md, protoreflect.FieldNumber(x)) +} + type isArtifactResponseWrapper_Response interface { isArtifactResponseWrapper_Response() } @@ -481,26 +764,23 @@ func (*ArtifactResponseWrapper_GetArtifactResponse) isArtifactResponseWrapper_Re // An artifact identifier and associated metadata. type ArtifactMetadata struct { - state protoimpl.MessageState - sizeCache protoimpl.SizeCache - unknownFields protoimpl.UnknownFields - + state protoimpl.MessageState `protogen:"hybrid.v1"` // (Required) The name of the artifact. Name string `protobuf:"bytes,1,opt,name=name,proto3" json:"name,omitempty"` // (Optional) The Unix-like permissions of the artifact Permissions uint32 `protobuf:"varint,2,opt,name=permissions,proto3" json:"permissions,omitempty"` // (Optional) The hex-encoded sha256 checksum of the artifact. Used, among other things, by // harness boot code to validate the integrity of the artifact. - Sha256 string `protobuf:"bytes,4,opt,name=sha256,proto3" json:"sha256,omitempty"` + Sha256 string `protobuf:"bytes,4,opt,name=sha256,proto3" json:"sha256,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache } func (x *ArtifactMetadata) Reset() { *x = ArtifactMetadata{} - if protoimpl.UnsafeEnabled { - mi := &file_org_apache_beam_model_job_management_v1_beam_artifact_api_proto_msgTypes[6] - ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) - ms.StoreMessageInfo(mi) - } + mi := &file_org_apache_beam_model_job_management_v1_beam_artifact_api_proto_msgTypes[6] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) } func (x *ArtifactMetadata) String() string { @@ -511,7 +791,7 @@ func (*ArtifactMetadata) ProtoMessage() {} func (x *ArtifactMetadata) ProtoReflect() protoreflect.Message { mi := &file_org_apache_beam_model_job_management_v1_beam_artifact_api_proto_msgTypes[6] - if protoimpl.UnsafeEnabled && x != nil { + if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { ms.StoreMessageInfo(mi) @@ -521,11 +801,6 @@ func (x *ArtifactMetadata) ProtoReflect() protoreflect.Message { return mi.MessageOf(x) } -// Deprecated: Use ArtifactMetadata.ProtoReflect.Descriptor instead. -func (*ArtifactMetadata) Descriptor() ([]byte, []int) { - return file_org_apache_beam_model_job_management_v1_beam_artifact_api_proto_rawDescGZIP(), []int{6} -} - func (x *ArtifactMetadata) GetName() string { if x != nil { return x.Name @@ -547,22 +822,53 @@ func (x *ArtifactMetadata) GetSha256() string { return "" } +func (x *ArtifactMetadata) SetName(v string) { + x.Name = v +} + +func (x *ArtifactMetadata) SetPermissions(v uint32) { + x.Permissions = v +} + +func (x *ArtifactMetadata) SetSha256(v string) { + x.Sha256 = v +} + +type ArtifactMetadata_builder struct { + _ [0]func() // Prevents comparability and use of unkeyed literals for the builder. + + // (Required) The name of the artifact. + Name string + // (Optional) The Unix-like permissions of the artifact + Permissions uint32 + // (Optional) The hex-encoded sha256 checksum of the artifact. Used, among other things, by + // harness boot code to validate the integrity of the artifact. + Sha256 string +} + +func (b0 ArtifactMetadata_builder) Build() *ArtifactMetadata { + m0 := &ArtifactMetadata{} + b, x := &b0, m0 + _, _ = b, x + x.Name = b.Name + x.Permissions = b.Permissions + x.Sha256 = b.Sha256 + return m0 +} + // A collection of artifacts. type Manifest struct { - state protoimpl.MessageState - sizeCache protoimpl.SizeCache + state protoimpl.MessageState `protogen:"hybrid.v1"` + Artifact []*ArtifactMetadata `protobuf:"bytes,1,rep,name=artifact,proto3" json:"artifact,omitempty"` unknownFields protoimpl.UnknownFields - - Artifact []*ArtifactMetadata `protobuf:"bytes,1,rep,name=artifact,proto3" json:"artifact,omitempty"` + sizeCache protoimpl.SizeCache } func (x *Manifest) Reset() { *x = Manifest{} - if protoimpl.UnsafeEnabled { - mi := &file_org_apache_beam_model_job_management_v1_beam_artifact_api_proto_msgTypes[7] - ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) - ms.StoreMessageInfo(mi) - } + mi := &file_org_apache_beam_model_job_management_v1_beam_artifact_api_proto_msgTypes[7] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) } func (x *Manifest) String() string { @@ -573,7 +879,7 @@ func (*Manifest) ProtoMessage() {} func (x *Manifest) ProtoReflect() protoreflect.Message { mi := &file_org_apache_beam_model_job_management_v1_beam_artifact_api_proto_msgTypes[7] - if protoimpl.UnsafeEnabled && x != nil { + if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { ms.StoreMessageInfo(mi) @@ -583,11 +889,6 @@ func (x *Manifest) ProtoReflect() protoreflect.Message { return mi.MessageOf(x) } -// Deprecated: Use Manifest.ProtoReflect.Descriptor instead. -func (*Manifest) Descriptor() ([]byte, []int) { - return file_org_apache_beam_model_job_management_v1_beam_artifact_api_proto_rawDescGZIP(), []int{7} -} - func (x *Manifest) GetArtifact() []*ArtifactMetadata { if x != nil { return x.Artifact @@ -595,23 +896,38 @@ func (x *Manifest) GetArtifact() []*ArtifactMetadata { return nil } +func (x *Manifest) SetArtifact(v []*ArtifactMetadata) { + x.Artifact = v +} + +type Manifest_builder struct { + _ [0]func() // Prevents comparability and use of unkeyed literals for the builder. + + Artifact []*ArtifactMetadata +} + +func (b0 Manifest_builder) Build() *Manifest { + m0 := &Manifest{} + b, x := &b0, m0 + _, _ = b, x + x.Artifact = b.Artifact + return m0 +} + // A manifest with location information. type ProxyManifest struct { - state protoimpl.MessageState - sizeCache protoimpl.SizeCache + state protoimpl.MessageState `protogen:"hybrid.v1"` + Manifest *Manifest `protobuf:"bytes,1,opt,name=manifest,proto3" json:"manifest,omitempty"` + Location []*ProxyManifest_Location `protobuf:"bytes,2,rep,name=location,proto3" json:"location,omitempty"` unknownFields protoimpl.UnknownFields - - Manifest *Manifest `protobuf:"bytes,1,opt,name=manifest,proto3" json:"manifest,omitempty"` - Location []*ProxyManifest_Location `protobuf:"bytes,2,rep,name=location,proto3" json:"location,omitempty"` + sizeCache protoimpl.SizeCache } func (x *ProxyManifest) Reset() { *x = ProxyManifest{} - if protoimpl.UnsafeEnabled { - mi := &file_org_apache_beam_model_job_management_v1_beam_artifact_api_proto_msgTypes[8] - ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) - ms.StoreMessageInfo(mi) - } + mi := &file_org_apache_beam_model_job_management_v1_beam_artifact_api_proto_msgTypes[8] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) } func (x *ProxyManifest) String() string { @@ -622,7 +938,7 @@ func (*ProxyManifest) ProtoMessage() {} func (x *ProxyManifest) ProtoReflect() protoreflect.Message { mi := &file_org_apache_beam_model_job_management_v1_beam_artifact_api_proto_msgTypes[8] - if protoimpl.UnsafeEnabled && x != nil { + if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { ms.StoreMessageInfo(mi) @@ -632,11 +948,6 @@ func (x *ProxyManifest) ProtoReflect() protoreflect.Message { return mi.MessageOf(x) } -// Deprecated: Use ProxyManifest.ProtoReflect.Descriptor instead. -func (*ProxyManifest) Descriptor() ([]byte, []int) { - return file_org_apache_beam_model_job_management_v1_beam_artifact_api_proto_rawDescGZIP(), []int{8} -} - func (x *ProxyManifest) GetManifest() *Manifest { if x != nil { return x.Manifest @@ -651,24 +962,56 @@ func (x *ProxyManifest) GetLocation() []*ProxyManifest_Location { return nil } +func (x *ProxyManifest) SetManifest(v *Manifest) { + x.Manifest = v +} + +func (x *ProxyManifest) SetLocation(v []*ProxyManifest_Location) { + x.Location = v +} + +func (x *ProxyManifest) HasManifest() bool { + if x == nil { + return false + } + return x.Manifest != nil +} + +func (x *ProxyManifest) ClearManifest() { + x.Manifest = nil +} + +type ProxyManifest_builder struct { + _ [0]func() // Prevents comparability and use of unkeyed literals for the builder. + + Manifest *Manifest + Location []*ProxyManifest_Location +} + +func (b0 ProxyManifest_builder) Build() *ProxyManifest { + m0 := &ProxyManifest{} + b, x := &b0, m0 + _, _ = b, x + x.Manifest = b.Manifest + x.Location = b.Location + return m0 +} + // A request to get the manifest of a Job. type GetManifestRequest struct { - state protoimpl.MessageState - sizeCache protoimpl.SizeCache - unknownFields protoimpl.UnknownFields - + state protoimpl.MessageState `protogen:"hybrid.v1"` // (Required) An opaque token representing the entirety of the staged artifacts. // Returned in CommitManifestResponse. RetrievalToken string `protobuf:"bytes,1,opt,name=retrieval_token,json=retrievalToken,proto3" json:"retrieval_token,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache } func (x *GetManifestRequest) Reset() { *x = GetManifestRequest{} - if protoimpl.UnsafeEnabled { - mi := &file_org_apache_beam_model_job_management_v1_beam_artifact_api_proto_msgTypes[9] - ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) - ms.StoreMessageInfo(mi) - } + mi := &file_org_apache_beam_model_job_management_v1_beam_artifact_api_proto_msgTypes[9] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) } func (x *GetManifestRequest) String() string { @@ -679,7 +1022,7 @@ func (*GetManifestRequest) ProtoMessage() {} func (x *GetManifestRequest) ProtoReflect() protoreflect.Message { mi := &file_org_apache_beam_model_job_management_v1_beam_artifact_api_proto_msgTypes[9] - if protoimpl.UnsafeEnabled && x != nil { + if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { ms.StoreMessageInfo(mi) @@ -689,34 +1032,46 @@ func (x *GetManifestRequest) ProtoReflect() protoreflect.Message { return mi.MessageOf(x) } -// Deprecated: Use GetManifestRequest.ProtoReflect.Descriptor instead. -func (*GetManifestRequest) Descriptor() ([]byte, []int) { - return file_org_apache_beam_model_job_management_v1_beam_artifact_api_proto_rawDescGZIP(), []int{9} +func (x *GetManifestRequest) GetRetrievalToken() string { + if x != nil { + return x.RetrievalToken + } + return "" +} + +func (x *GetManifestRequest) SetRetrievalToken(v string) { + x.RetrievalToken = v +} + +type GetManifestRequest_builder struct { + _ [0]func() // Prevents comparability and use of unkeyed literals for the builder. + + // (Required) An opaque token representing the entirety of the staged artifacts. + // Returned in CommitManifestResponse. + RetrievalToken string } -func (x *GetManifestRequest) GetRetrievalToken() string { - if x != nil { - return x.RetrievalToken - } - return "" +func (b0 GetManifestRequest_builder) Build() *GetManifestRequest { + m0 := &GetManifestRequest{} + b, x := &b0, m0 + _, _ = b, x + x.RetrievalToken = b.RetrievalToken + return m0 } // A response containing a job manifest. type GetManifestResponse struct { - state protoimpl.MessageState - sizeCache protoimpl.SizeCache + state protoimpl.MessageState `protogen:"hybrid.v1"` + Manifest *Manifest `protobuf:"bytes,1,opt,name=manifest,proto3" json:"manifest,omitempty"` unknownFields protoimpl.UnknownFields - - Manifest *Manifest `protobuf:"bytes,1,opt,name=manifest,proto3" json:"manifest,omitempty"` + sizeCache protoimpl.SizeCache } func (x *GetManifestResponse) Reset() { *x = GetManifestResponse{} - if protoimpl.UnsafeEnabled { - mi := &file_org_apache_beam_model_job_management_v1_beam_artifact_api_proto_msgTypes[10] - ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) - ms.StoreMessageInfo(mi) - } + mi := &file_org_apache_beam_model_job_management_v1_beam_artifact_api_proto_msgTypes[10] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) } func (x *GetManifestResponse) String() string { @@ -727,7 +1082,7 @@ func (*GetManifestResponse) ProtoMessage() {} func (x *GetManifestResponse) ProtoReflect() protoreflect.Message { mi := &file_org_apache_beam_model_job_management_v1_beam_artifact_api_proto_msgTypes[10] - if protoimpl.UnsafeEnabled && x != nil { + if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { ms.StoreMessageInfo(mi) @@ -737,11 +1092,6 @@ func (x *GetManifestResponse) ProtoReflect() protoreflect.Message { return mi.MessageOf(x) } -// Deprecated: Use GetManifestResponse.ProtoReflect.Descriptor instead. -func (*GetManifestResponse) Descriptor() ([]byte, []int) { - return file_org_apache_beam_model_job_management_v1_beam_artifact_api_proto_rawDescGZIP(), []int{10} -} - func (x *GetManifestResponse) GetManifest() *Manifest { if x != nil { return x.Manifest @@ -749,26 +1099,52 @@ func (x *GetManifestResponse) GetManifest() *Manifest { return nil } +func (x *GetManifestResponse) SetManifest(v *Manifest) { + x.Manifest = v +} + +func (x *GetManifestResponse) HasManifest() bool { + if x == nil { + return false + } + return x.Manifest != nil +} + +func (x *GetManifestResponse) ClearManifest() { + x.Manifest = nil +} + +type GetManifestResponse_builder struct { + _ [0]func() // Prevents comparability and use of unkeyed literals for the builder. + + Manifest *Manifest +} + +func (b0 GetManifestResponse_builder) Build() *GetManifestResponse { + m0 := &GetManifestResponse{} + b, x := &b0, m0 + _, _ = b, x + x.Manifest = b.Manifest + return m0 +} + // A request to get an artifact. The artifact must be present in the manifest for the job. type LegacyGetArtifactRequest struct { - state protoimpl.MessageState - sizeCache protoimpl.SizeCache - unknownFields protoimpl.UnknownFields - + state protoimpl.MessageState `protogen:"hybrid.v1"` // (Required) The name of the artifact to retrieve. Name string `protobuf:"bytes,1,opt,name=name,proto3" json:"name,omitempty"` // (Required) An opaque token representing the entirety of the staged artifacts. // Returned in CommitManifestResponse. RetrievalToken string `protobuf:"bytes,2,opt,name=retrieval_token,json=retrievalToken,proto3" json:"retrieval_token,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache } func (x *LegacyGetArtifactRequest) Reset() { *x = LegacyGetArtifactRequest{} - if protoimpl.UnsafeEnabled { - mi := &file_org_apache_beam_model_job_management_v1_beam_artifact_api_proto_msgTypes[11] - ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) - ms.StoreMessageInfo(mi) - } + mi := &file_org_apache_beam_model_job_management_v1_beam_artifact_api_proto_msgTypes[11] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) } func (x *LegacyGetArtifactRequest) String() string { @@ -779,7 +1155,7 @@ func (*LegacyGetArtifactRequest) ProtoMessage() {} func (x *LegacyGetArtifactRequest) ProtoReflect() protoreflect.Message { mi := &file_org_apache_beam_model_job_management_v1_beam_artifact_api_proto_msgTypes[11] - if protoimpl.UnsafeEnabled && x != nil { + if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { ms.StoreMessageInfo(mi) @@ -789,11 +1165,6 @@ func (x *LegacyGetArtifactRequest) ProtoReflect() protoreflect.Message { return mi.MessageOf(x) } -// Deprecated: Use LegacyGetArtifactRequest.ProtoReflect.Descriptor instead. -func (*LegacyGetArtifactRequest) Descriptor() ([]byte, []int) { - return file_org_apache_beam_model_job_management_v1_beam_artifact_api_proto_rawDescGZIP(), []int{11} -} - func (x *LegacyGetArtifactRequest) GetName() string { if x != nil { return x.Name @@ -808,22 +1179,46 @@ func (x *LegacyGetArtifactRequest) GetRetrievalToken() string { return "" } +func (x *LegacyGetArtifactRequest) SetName(v string) { + x.Name = v +} + +func (x *LegacyGetArtifactRequest) SetRetrievalToken(v string) { + x.RetrievalToken = v +} + +type LegacyGetArtifactRequest_builder struct { + _ [0]func() // Prevents comparability and use of unkeyed literals for the builder. + + // (Required) The name of the artifact to retrieve. + Name string + // (Required) An opaque token representing the entirety of the staged artifacts. + // Returned in CommitManifestResponse. + RetrievalToken string +} + +func (b0 LegacyGetArtifactRequest_builder) Build() *LegacyGetArtifactRequest { + m0 := &LegacyGetArtifactRequest{} + b, x := &b0, m0 + _, _ = b, x + x.Name = b.Name + x.RetrievalToken = b.RetrievalToken + return m0 +} + // Part of an artifact. type ArtifactChunk struct { - state protoimpl.MessageState - sizeCache protoimpl.SizeCache + state protoimpl.MessageState `protogen:"hybrid.v1"` + Data []byte `protobuf:"bytes,1,opt,name=data,proto3" json:"data,omitempty"` unknownFields protoimpl.UnknownFields - - Data []byte `protobuf:"bytes,1,opt,name=data,proto3" json:"data,omitempty"` + sizeCache protoimpl.SizeCache } func (x *ArtifactChunk) Reset() { *x = ArtifactChunk{} - if protoimpl.UnsafeEnabled { - mi := &file_org_apache_beam_model_job_management_v1_beam_artifact_api_proto_msgTypes[12] - ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) - ms.StoreMessageInfo(mi) - } + mi := &file_org_apache_beam_model_job_management_v1_beam_artifact_api_proto_msgTypes[12] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) } func (x *ArtifactChunk) String() string { @@ -834,7 +1229,7 @@ func (*ArtifactChunk) ProtoMessage() {} func (x *ArtifactChunk) ProtoReflect() protoreflect.Message { mi := &file_org_apache_beam_model_job_management_v1_beam_artifact_api_proto_msgTypes[12] - if protoimpl.UnsafeEnabled && x != nil { + if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { ms.StoreMessageInfo(mi) @@ -844,11 +1239,6 @@ func (x *ArtifactChunk) ProtoReflect() protoreflect.Message { return mi.MessageOf(x) } -// Deprecated: Use ArtifactChunk.ProtoReflect.Descriptor instead. -func (*ArtifactChunk) Descriptor() ([]byte, []int) { - return file_org_apache_beam_model_job_management_v1_beam_artifact_api_proto_rawDescGZIP(), []int{12} -} - func (x *ArtifactChunk) GetData() []byte { if x != nil { return x.Data @@ -856,25 +1246,43 @@ func (x *ArtifactChunk) GetData() []byte { return nil } -type PutArtifactMetadata struct { - state protoimpl.MessageState - sizeCache protoimpl.SizeCache - unknownFields protoimpl.UnknownFields +func (x *ArtifactChunk) SetData(v []byte) { + if v == nil { + v = []byte{} + } + x.Data = v +} + +type ArtifactChunk_builder struct { + _ [0]func() // Prevents comparability and use of unkeyed literals for the builder. + Data []byte +} + +func (b0 ArtifactChunk_builder) Build() *ArtifactChunk { + m0 := &ArtifactChunk{} + b, x := &b0, m0 + _, _ = b, x + x.Data = b.Data + return m0 +} + +type PutArtifactMetadata struct { + state protoimpl.MessageState `protogen:"hybrid.v1"` // (Required) A token for artifact staging session. This token can be obtained // from PrepareJob request in JobService StagingSessionToken string `protobuf:"bytes,1,opt,name=staging_session_token,json=stagingSessionToken,proto3" json:"staging_session_token,omitempty"` // (Required) The Artifact metadata. - Metadata *ArtifactMetadata `protobuf:"bytes,2,opt,name=metadata,proto3" json:"metadata,omitempty"` + Metadata *ArtifactMetadata `protobuf:"bytes,2,opt,name=metadata,proto3" json:"metadata,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache } func (x *PutArtifactMetadata) Reset() { *x = PutArtifactMetadata{} - if protoimpl.UnsafeEnabled { - mi := &file_org_apache_beam_model_job_management_v1_beam_artifact_api_proto_msgTypes[13] - ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) - ms.StoreMessageInfo(mi) - } + mi := &file_org_apache_beam_model_job_management_v1_beam_artifact_api_proto_msgTypes[13] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) } func (x *PutArtifactMetadata) String() string { @@ -885,7 +1293,7 @@ func (*PutArtifactMetadata) ProtoMessage() {} func (x *PutArtifactMetadata) ProtoReflect() protoreflect.Message { mi := &file_org_apache_beam_model_job_management_v1_beam_artifact_api_proto_msgTypes[13] - if protoimpl.UnsafeEnabled && x != nil { + if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { ms.StoreMessageInfo(mi) @@ -895,11 +1303,6 @@ func (x *PutArtifactMetadata) ProtoReflect() protoreflect.Message { return mi.MessageOf(x) } -// Deprecated: Use PutArtifactMetadata.ProtoReflect.Descriptor instead. -func (*PutArtifactMetadata) Descriptor() ([]byte, []int) { - return file_org_apache_beam_model_job_management_v1_beam_artifact_api_proto_rawDescGZIP(), []int{13} -} - func (x *PutArtifactMetadata) GetStagingSessionToken() string { if x != nil { return x.StagingSessionToken @@ -914,28 +1317,63 @@ func (x *PutArtifactMetadata) GetMetadata() *ArtifactMetadata { return nil } +func (x *PutArtifactMetadata) SetStagingSessionToken(v string) { + x.StagingSessionToken = v +} + +func (x *PutArtifactMetadata) SetMetadata(v *ArtifactMetadata) { + x.Metadata = v +} + +func (x *PutArtifactMetadata) HasMetadata() bool { + if x == nil { + return false + } + return x.Metadata != nil +} + +func (x *PutArtifactMetadata) ClearMetadata() { + x.Metadata = nil +} + +type PutArtifactMetadata_builder struct { + _ [0]func() // Prevents comparability and use of unkeyed literals for the builder. + + // (Required) A token for artifact staging session. This token can be obtained + // from PrepareJob request in JobService + StagingSessionToken string + // (Required) The Artifact metadata. + Metadata *ArtifactMetadata +} + +func (b0 PutArtifactMetadata_builder) Build() *PutArtifactMetadata { + m0 := &PutArtifactMetadata{} + b, x := &b0, m0 + _, _ = b, x + x.StagingSessionToken = b.StagingSessionToken + x.Metadata = b.Metadata + return m0 +} + // A request to stage an artifact. type PutArtifactRequest struct { - state protoimpl.MessageState - sizeCache protoimpl.SizeCache - unknownFields protoimpl.UnknownFields - + state protoimpl.MessageState `protogen:"hybrid.v1"` // (Required) // - // Types that are assignable to Content: + // Types that are valid to be assigned to Content: // // *PutArtifactRequest_Metadata // *PutArtifactRequest_Data - Content isPutArtifactRequest_Content `protobuf_oneof:"content"` + Content isPutArtifactRequest_Content `protobuf_oneof:"content"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache } func (x *PutArtifactRequest) Reset() { *x = PutArtifactRequest{} - if protoimpl.UnsafeEnabled { - mi := &file_org_apache_beam_model_job_management_v1_beam_artifact_api_proto_msgTypes[14] - ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) - ms.StoreMessageInfo(mi) - } + mi := &file_org_apache_beam_model_job_management_v1_beam_artifact_api_proto_msgTypes[14] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) } func (x *PutArtifactRequest) String() string { @@ -946,7 +1384,7 @@ func (*PutArtifactRequest) ProtoMessage() {} func (x *PutArtifactRequest) ProtoReflect() protoreflect.Message { mi := &file_org_apache_beam_model_job_management_v1_beam_artifact_api_proto_msgTypes[14] - if protoimpl.UnsafeEnabled && x != nil { + if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { ms.StoreMessageInfo(mi) @@ -956,32 +1394,141 @@ func (x *PutArtifactRequest) ProtoReflect() protoreflect.Message { return mi.MessageOf(x) } -// Deprecated: Use PutArtifactRequest.ProtoReflect.Descriptor instead. -func (*PutArtifactRequest) Descriptor() ([]byte, []int) { - return file_org_apache_beam_model_job_management_v1_beam_artifact_api_proto_rawDescGZIP(), []int{14} -} - -func (m *PutArtifactRequest) GetContent() isPutArtifactRequest_Content { - if m != nil { - return m.Content +func (x *PutArtifactRequest) GetContent() isPutArtifactRequest_Content { + if x != nil { + return x.Content } return nil } func (x *PutArtifactRequest) GetMetadata() *PutArtifactMetadata { - if x, ok := x.GetContent().(*PutArtifactRequest_Metadata); ok { - return x.Metadata + if x != nil { + if x, ok := x.Content.(*PutArtifactRequest_Metadata); ok { + return x.Metadata + } } return nil } func (x *PutArtifactRequest) GetData() *ArtifactChunk { - if x, ok := x.GetContent().(*PutArtifactRequest_Data); ok { - return x.Data + if x != nil { + if x, ok := x.Content.(*PutArtifactRequest_Data); ok { + return x.Data + } } return nil } +func (x *PutArtifactRequest) SetMetadata(v *PutArtifactMetadata) { + if v == nil { + x.Content = nil + return + } + x.Content = &PutArtifactRequest_Metadata{v} +} + +func (x *PutArtifactRequest) SetData(v *ArtifactChunk) { + if v == nil { + x.Content = nil + return + } + x.Content = &PutArtifactRequest_Data{v} +} + +func (x *PutArtifactRequest) HasContent() bool { + if x == nil { + return false + } + return x.Content != nil +} + +func (x *PutArtifactRequest) HasMetadata() bool { + if x == nil { + return false + } + _, ok := x.Content.(*PutArtifactRequest_Metadata) + return ok +} + +func (x *PutArtifactRequest) HasData() bool { + if x == nil { + return false + } + _, ok := x.Content.(*PutArtifactRequest_Data) + return ok +} + +func (x *PutArtifactRequest) ClearContent() { + x.Content = nil +} + +func (x *PutArtifactRequest) ClearMetadata() { + if _, ok := x.Content.(*PutArtifactRequest_Metadata); ok { + x.Content = nil + } +} + +func (x *PutArtifactRequest) ClearData() { + if _, ok := x.Content.(*PutArtifactRequest_Data); ok { + x.Content = nil + } +} + +const PutArtifactRequest_Content_not_set_case case_PutArtifactRequest_Content = 0 +const PutArtifactRequest_Metadata_case case_PutArtifactRequest_Content = 1 +const PutArtifactRequest_Data_case case_PutArtifactRequest_Content = 2 + +func (x *PutArtifactRequest) WhichContent() case_PutArtifactRequest_Content { + if x == nil { + return PutArtifactRequest_Content_not_set_case + } + switch x.Content.(type) { + case *PutArtifactRequest_Metadata: + return PutArtifactRequest_Metadata_case + case *PutArtifactRequest_Data: + return PutArtifactRequest_Data_case + default: + return PutArtifactRequest_Content_not_set_case + } +} + +type PutArtifactRequest_builder struct { + _ [0]func() // Prevents comparability and use of unkeyed literals for the builder. + + // (Required) + + // Fields of oneof Content: + // The first message in a PutArtifact call must contain this field. + Metadata *PutArtifactMetadata + // A chunk of the artifact. All messages after the first in a PutArtifact call must contain a + // chunk. + Data *ArtifactChunk + // -- end of Content +} + +func (b0 PutArtifactRequest_builder) Build() *PutArtifactRequest { + m0 := &PutArtifactRequest{} + b, x := &b0, m0 + _, _ = b, x + if b.Metadata != nil { + x.Content = &PutArtifactRequest_Metadata{b.Metadata} + } + if b.Data != nil { + x.Content = &PutArtifactRequest_Data{b.Data} + } + return m0 +} + +type case_PutArtifactRequest_Content protoreflect.FieldNumber + +func (x case_PutArtifactRequest_Content) String() string { + md := file_org_apache_beam_model_job_management_v1_beam_artifact_api_proto_msgTypes[14].Descriptor() + if x == 0 { + return "not set" + } + return protoimpl.X.MessageFieldStringOf(md, protoreflect.FieldNumber(x)) +} + type isPutArtifactRequest_Content interface { isPutArtifactRequest_Content() } @@ -1002,18 +1549,16 @@ func (*PutArtifactRequest_Metadata) isPutArtifactRequest_Content() {} func (*PutArtifactRequest_Data) isPutArtifactRequest_Content() {} type PutArtifactResponse struct { - state protoimpl.MessageState - sizeCache protoimpl.SizeCache + state protoimpl.MessageState `protogen:"hybrid.v1"` unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache } func (x *PutArtifactResponse) Reset() { *x = PutArtifactResponse{} - if protoimpl.UnsafeEnabled { - mi := &file_org_apache_beam_model_job_management_v1_beam_artifact_api_proto_msgTypes[15] - ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) - ms.StoreMessageInfo(mi) - } + mi := &file_org_apache_beam_model_job_management_v1_beam_artifact_api_proto_msgTypes[15] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) } func (x *PutArtifactResponse) String() string { @@ -1024,7 +1569,7 @@ func (*PutArtifactResponse) ProtoMessage() {} func (x *PutArtifactResponse) ProtoReflect() protoreflect.Message { mi := &file_org_apache_beam_model_job_management_v1_beam_artifact_api_proto_msgTypes[15] - if protoimpl.UnsafeEnabled && x != nil { + if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { ms.StoreMessageInfo(mi) @@ -1034,32 +1579,36 @@ func (x *PutArtifactResponse) ProtoReflect() protoreflect.Message { return mi.MessageOf(x) } -// Deprecated: Use PutArtifactResponse.ProtoReflect.Descriptor instead. -func (*PutArtifactResponse) Descriptor() ([]byte, []int) { - return file_org_apache_beam_model_job_management_v1_beam_artifact_api_proto_rawDescGZIP(), []int{15} +type PutArtifactResponse_builder struct { + _ [0]func() // Prevents comparability and use of unkeyed literals for the builder. + +} + +func (b0 PutArtifactResponse_builder) Build() *PutArtifactResponse { + m0 := &PutArtifactResponse{} + b, x := &b0, m0 + _, _ = b, x + return m0 } // A request to commit the manifest for a Job. All artifacts must have been successfully uploaded // before this call is made. type CommitManifestRequest struct { - state protoimpl.MessageState - sizeCache protoimpl.SizeCache - unknownFields protoimpl.UnknownFields - + state protoimpl.MessageState `protogen:"hybrid.v1"` // (Required) The manifest to commit. Manifest *Manifest `protobuf:"bytes,1,opt,name=manifest,proto3" json:"manifest,omitempty"` // (Required) A token for artifact staging session. This token can be obtained // from PrepareJob request in JobService StagingSessionToken string `protobuf:"bytes,2,opt,name=staging_session_token,json=stagingSessionToken,proto3" json:"staging_session_token,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache } func (x *CommitManifestRequest) Reset() { *x = CommitManifestRequest{} - if protoimpl.UnsafeEnabled { - mi := &file_org_apache_beam_model_job_management_v1_beam_artifact_api_proto_msgTypes[16] - ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) - ms.StoreMessageInfo(mi) - } + mi := &file_org_apache_beam_model_job_management_v1_beam_artifact_api_proto_msgTypes[16] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) } func (x *CommitManifestRequest) String() string { @@ -1070,7 +1619,7 @@ func (*CommitManifestRequest) ProtoMessage() {} func (x *CommitManifestRequest) ProtoReflect() protoreflect.Message { mi := &file_org_apache_beam_model_job_management_v1_beam_artifact_api_proto_msgTypes[16] - if protoimpl.UnsafeEnabled && x != nil { + if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { ms.StoreMessageInfo(mi) @@ -1080,11 +1629,6 @@ func (x *CommitManifestRequest) ProtoReflect() protoreflect.Message { return mi.MessageOf(x) } -// Deprecated: Use CommitManifestRequest.ProtoReflect.Descriptor instead. -func (*CommitManifestRequest) Descriptor() ([]byte, []int) { - return file_org_apache_beam_model_job_management_v1_beam_artifact_api_proto_rawDescGZIP(), []int{16} -} - func (x *CommitManifestRequest) GetManifest() *Manifest { if x != nil { return x.Manifest @@ -1099,25 +1643,60 @@ func (x *CommitManifestRequest) GetStagingSessionToken() string { return "" } +func (x *CommitManifestRequest) SetManifest(v *Manifest) { + x.Manifest = v +} + +func (x *CommitManifestRequest) SetStagingSessionToken(v string) { + x.StagingSessionToken = v +} + +func (x *CommitManifestRequest) HasManifest() bool { + if x == nil { + return false + } + return x.Manifest != nil +} + +func (x *CommitManifestRequest) ClearManifest() { + x.Manifest = nil +} + +type CommitManifestRequest_builder struct { + _ [0]func() // Prevents comparability and use of unkeyed literals for the builder. + + // (Required) The manifest to commit. + Manifest *Manifest + // (Required) A token for artifact staging session. This token can be obtained + // from PrepareJob request in JobService + StagingSessionToken string +} + +func (b0 CommitManifestRequest_builder) Build() *CommitManifestRequest { + m0 := &CommitManifestRequest{} + b, x := &b0, m0 + _, _ = b, x + x.Manifest = b.Manifest + x.StagingSessionToken = b.StagingSessionToken + return m0 +} + // The result of committing a manifest. type CommitManifestResponse struct { - state protoimpl.MessageState - sizeCache protoimpl.SizeCache - unknownFields protoimpl.UnknownFields - + state protoimpl.MessageState `protogen:"hybrid.v1"` // (Required) An opaque token representing the entirety of the staged artifacts. // This can be used to retrieve the manifest and artifacts from an associated // LegacyArtifactRetrievalService. RetrievalToken string `protobuf:"bytes,1,opt,name=retrieval_token,json=retrievalToken,proto3" json:"retrieval_token,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache } func (x *CommitManifestResponse) Reset() { *x = CommitManifestResponse{} - if protoimpl.UnsafeEnabled { - mi := &file_org_apache_beam_model_job_management_v1_beam_artifact_api_proto_msgTypes[17] - ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) - ms.StoreMessageInfo(mi) - } + mi := &file_org_apache_beam_model_job_management_v1_beam_artifact_api_proto_msgTypes[17] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) } func (x *CommitManifestResponse) String() string { @@ -1128,7 +1707,7 @@ func (*CommitManifestResponse) ProtoMessage() {} func (x *CommitManifestResponse) ProtoReflect() protoreflect.Message { mi := &file_org_apache_beam_model_job_management_v1_beam_artifact_api_proto_msgTypes[17] - if protoimpl.UnsafeEnabled && x != nil { + if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { ms.StoreMessageInfo(mi) @@ -1138,11 +1717,6 @@ func (x *CommitManifestResponse) ProtoReflect() protoreflect.Message { return mi.MessageOf(x) } -// Deprecated: Use CommitManifestResponse.ProtoReflect.Descriptor instead. -func (*CommitManifestResponse) Descriptor() ([]byte, []int) { - return file_org_apache_beam_model_job_management_v1_beam_artifact_api_proto_rawDescGZIP(), []int{17} -} - func (x *CommitManifestResponse) GetRetrievalToken() string { if x != nil { return x.RetrievalToken @@ -1150,22 +1724,40 @@ func (x *CommitManifestResponse) GetRetrievalToken() string { return "" } +func (x *CommitManifestResponse) SetRetrievalToken(v string) { + x.RetrievalToken = v +} + +type CommitManifestResponse_builder struct { + _ [0]func() // Prevents comparability and use of unkeyed literals for the builder. + + // (Required) An opaque token representing the entirety of the staged artifacts. + // This can be used to retrieve the manifest and artifacts from an associated + // LegacyArtifactRetrievalService. + RetrievalToken string +} + +func (b0 CommitManifestResponse_builder) Build() *CommitManifestResponse { + m0 := &CommitManifestResponse{} + b, x := &b0, m0 + _, _ = b, x + x.RetrievalToken = b.RetrievalToken + return m0 +} + type ProxyManifest_Location struct { - state protoimpl.MessageState - sizeCache protoimpl.SizeCache + state protoimpl.MessageState `protogen:"hybrid.v1"` + Name string `protobuf:"bytes,1,opt,name=name,proto3" json:"name,omitempty"` + Uri string `protobuf:"bytes,2,opt,name=uri,proto3" json:"uri,omitempty"` unknownFields protoimpl.UnknownFields - - Name string `protobuf:"bytes,1,opt,name=name,proto3" json:"name,omitempty"` - Uri string `protobuf:"bytes,2,opt,name=uri,proto3" json:"uri,omitempty"` + sizeCache protoimpl.SizeCache } func (x *ProxyManifest_Location) Reset() { *x = ProxyManifest_Location{} - if protoimpl.UnsafeEnabled { - mi := &file_org_apache_beam_model_job_management_v1_beam_artifact_api_proto_msgTypes[18] - ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) - ms.StoreMessageInfo(mi) - } + mi := &file_org_apache_beam_model_job_management_v1_beam_artifact_api_proto_msgTypes[18] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) } func (x *ProxyManifest_Location) String() string { @@ -1176,7 +1768,7 @@ func (*ProxyManifest_Location) ProtoMessage() {} func (x *ProxyManifest_Location) ProtoReflect() protoreflect.Message { mi := &file_org_apache_beam_model_job_management_v1_beam_artifact_api_proto_msgTypes[18] - if protoimpl.UnsafeEnabled && x != nil { + if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { ms.StoreMessageInfo(mi) @@ -1186,11 +1778,6 @@ func (x *ProxyManifest_Location) ProtoReflect() protoreflect.Message { return mi.MessageOf(x) } -// Deprecated: Use ProxyManifest_Location.ProtoReflect.Descriptor instead. -func (*ProxyManifest_Location) Descriptor() ([]byte, []int) { - return file_org_apache_beam_model_job_management_v1_beam_artifact_api_proto_rawDescGZIP(), []int{8, 0} -} - func (x *ProxyManifest_Location) GetName() string { if x != nil { return x.Name @@ -1205,6 +1792,30 @@ func (x *ProxyManifest_Location) GetUri() string { return "" } +func (x *ProxyManifest_Location) SetName(v string) { + x.Name = v +} + +func (x *ProxyManifest_Location) SetUri(v string) { + x.Uri = v +} + +type ProxyManifest_Location_builder struct { + _ [0]func() // Prevents comparability and use of unkeyed literals for the builder. + + Name string + Uri string +} + +func (b0 ProxyManifest_Location_builder) Build() *ProxyManifest_Location { + m0 := &ProxyManifest_Location{} + b, x := &b0, m0 + _, _ = b, x + x.Name = b.Name + x.Uri = b.Uri + return m0 +} + var File_org_apache_beam_model_job_management_v1_beam_artifact_api_proto protoreflect.FileDescriptor var file_org_apache_beam_model_job_management_v1_beam_artifact_api_proto_rawDesc = []byte{ @@ -1455,21 +2066,9 @@ var file_org_apache_beam_model_job_management_v1_beam_artifact_api_proto_rawDesc 0x72, 0x6f, 0x74, 0x6f, 0x33, } -var ( - file_org_apache_beam_model_job_management_v1_beam_artifact_api_proto_rawDescOnce sync.Once - file_org_apache_beam_model_job_management_v1_beam_artifact_api_proto_rawDescData = file_org_apache_beam_model_job_management_v1_beam_artifact_api_proto_rawDesc -) - -func file_org_apache_beam_model_job_management_v1_beam_artifact_api_proto_rawDescGZIP() []byte { - file_org_apache_beam_model_job_management_v1_beam_artifact_api_proto_rawDescOnce.Do(func() { - file_org_apache_beam_model_job_management_v1_beam_artifact_api_proto_rawDescData = protoimpl.X.CompressGZIP(file_org_apache_beam_model_job_management_v1_beam_artifact_api_proto_rawDescData) - }) - return file_org_apache_beam_model_job_management_v1_beam_artifact_api_proto_rawDescData -} - var file_org_apache_beam_model_job_management_v1_beam_artifact_api_proto_enumTypes = make([]protoimpl.EnumInfo, 1) var file_org_apache_beam_model_job_management_v1_beam_artifact_api_proto_msgTypes = make([]protoimpl.MessageInfo, 19) -var file_org_apache_beam_model_job_management_v1_beam_artifact_api_proto_goTypes = []interface{}{ +var file_org_apache_beam_model_job_management_v1_beam_artifact_api_proto_goTypes = []any{ (CommitManifestResponse_Constants)(0), // 0: org.apache.beam.model.job_management.v1.CommitManifestResponse.Constants (*ResolveArtifactsRequest)(nil), // 1: org.apache.beam.model.job_management.v1.ResolveArtifactsRequest (*ResolveArtifactsResponse)(nil), // 2: org.apache.beam.model.job_management.v1.ResolveArtifactsResponse @@ -1534,245 +2133,15 @@ func file_org_apache_beam_model_job_management_v1_beam_artifact_api_proto_init() if File_org_apache_beam_model_job_management_v1_beam_artifact_api_proto != nil { return } - if !protoimpl.UnsafeEnabled { - file_org_apache_beam_model_job_management_v1_beam_artifact_api_proto_msgTypes[0].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*ResolveArtifactsRequest); i { - case 0: - return &v.state - case 1: - return &v.sizeCache - case 2: - return &v.unknownFields - default: - return nil - } - } - file_org_apache_beam_model_job_management_v1_beam_artifact_api_proto_msgTypes[1].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*ResolveArtifactsResponse); i { - case 0: - return &v.state - case 1: - return &v.sizeCache - case 2: - return &v.unknownFields - default: - return nil - } - } - file_org_apache_beam_model_job_management_v1_beam_artifact_api_proto_msgTypes[2].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*GetArtifactRequest); i { - case 0: - return &v.state - case 1: - return &v.sizeCache - case 2: - return &v.unknownFields - default: - return nil - } - } - file_org_apache_beam_model_job_management_v1_beam_artifact_api_proto_msgTypes[3].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*GetArtifactResponse); i { - case 0: - return &v.state - case 1: - return &v.sizeCache - case 2: - return &v.unknownFields - default: - return nil - } - } - file_org_apache_beam_model_job_management_v1_beam_artifact_api_proto_msgTypes[4].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*ArtifactRequestWrapper); i { - case 0: - return &v.state - case 1: - return &v.sizeCache - case 2: - return &v.unknownFields - default: - return nil - } - } - file_org_apache_beam_model_job_management_v1_beam_artifact_api_proto_msgTypes[5].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*ArtifactResponseWrapper); i { - case 0: - return &v.state - case 1: - return &v.sizeCache - case 2: - return &v.unknownFields - default: - return nil - } - } - file_org_apache_beam_model_job_management_v1_beam_artifact_api_proto_msgTypes[6].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*ArtifactMetadata); i { - case 0: - return &v.state - case 1: - return &v.sizeCache - case 2: - return &v.unknownFields - default: - return nil - } - } - file_org_apache_beam_model_job_management_v1_beam_artifact_api_proto_msgTypes[7].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*Manifest); i { - case 0: - return &v.state - case 1: - return &v.sizeCache - case 2: - return &v.unknownFields - default: - return nil - } - } - file_org_apache_beam_model_job_management_v1_beam_artifact_api_proto_msgTypes[8].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*ProxyManifest); i { - case 0: - return &v.state - case 1: - return &v.sizeCache - case 2: - return &v.unknownFields - default: - return nil - } - } - file_org_apache_beam_model_job_management_v1_beam_artifact_api_proto_msgTypes[9].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*GetManifestRequest); i { - case 0: - return &v.state - case 1: - return &v.sizeCache - case 2: - return &v.unknownFields - default: - return nil - } - } - file_org_apache_beam_model_job_management_v1_beam_artifact_api_proto_msgTypes[10].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*GetManifestResponse); i { - case 0: - return &v.state - case 1: - return &v.sizeCache - case 2: - return &v.unknownFields - default: - return nil - } - } - file_org_apache_beam_model_job_management_v1_beam_artifact_api_proto_msgTypes[11].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*LegacyGetArtifactRequest); i { - case 0: - return &v.state - case 1: - return &v.sizeCache - case 2: - return &v.unknownFields - default: - return nil - } - } - file_org_apache_beam_model_job_management_v1_beam_artifact_api_proto_msgTypes[12].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*ArtifactChunk); i { - case 0: - return &v.state - case 1: - return &v.sizeCache - case 2: - return &v.unknownFields - default: - return nil - } - } - file_org_apache_beam_model_job_management_v1_beam_artifact_api_proto_msgTypes[13].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*PutArtifactMetadata); i { - case 0: - return &v.state - case 1: - return &v.sizeCache - case 2: - return &v.unknownFields - default: - return nil - } - } - file_org_apache_beam_model_job_management_v1_beam_artifact_api_proto_msgTypes[14].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*PutArtifactRequest); i { - case 0: - return &v.state - case 1: - return &v.sizeCache - case 2: - return &v.unknownFields - default: - return nil - } - } - file_org_apache_beam_model_job_management_v1_beam_artifact_api_proto_msgTypes[15].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*PutArtifactResponse); i { - case 0: - return &v.state - case 1: - return &v.sizeCache - case 2: - return &v.unknownFields - default: - return nil - } - } - file_org_apache_beam_model_job_management_v1_beam_artifact_api_proto_msgTypes[16].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*CommitManifestRequest); i { - case 0: - return &v.state - case 1: - return &v.sizeCache - case 2: - return &v.unknownFields - default: - return nil - } - } - file_org_apache_beam_model_job_management_v1_beam_artifact_api_proto_msgTypes[17].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*CommitManifestResponse); i { - case 0: - return &v.state - case 1: - return &v.sizeCache - case 2: - return &v.unknownFields - default: - return nil - } - } - file_org_apache_beam_model_job_management_v1_beam_artifact_api_proto_msgTypes[18].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*ProxyManifest_Location); i { - case 0: - return &v.state - case 1: - return &v.sizeCache - case 2: - return &v.unknownFields - default: - return nil - } - } - } - file_org_apache_beam_model_job_management_v1_beam_artifact_api_proto_msgTypes[4].OneofWrappers = []interface{}{ + file_org_apache_beam_model_job_management_v1_beam_artifact_api_proto_msgTypes[4].OneofWrappers = []any{ (*ArtifactRequestWrapper_ResolveArtifact)(nil), (*ArtifactRequestWrapper_GetArtifact)(nil), } - file_org_apache_beam_model_job_management_v1_beam_artifact_api_proto_msgTypes[5].OneofWrappers = []interface{}{ + file_org_apache_beam_model_job_management_v1_beam_artifact_api_proto_msgTypes[5].OneofWrappers = []any{ (*ArtifactResponseWrapper_ResolveArtifactResponse)(nil), (*ArtifactResponseWrapper_GetArtifactResponse)(nil), } - file_org_apache_beam_model_job_management_v1_beam_artifact_api_proto_msgTypes[14].OneofWrappers = []interface{}{ + file_org_apache_beam_model_job_management_v1_beam_artifact_api_proto_msgTypes[14].OneofWrappers = []any{ (*PutArtifactRequest_Metadata)(nil), (*PutArtifactRequest_Data)(nil), } diff --git a/sdks/go/pkg/beam/model/jobmanagement_v1/beam_artifact_api_grpc.pb.go b/sdks/go/pkg/beam/model/jobmanagement_v1/beam_artifact_api_grpc.pb.go index fcf2e3b2dac9..093d0b5f8d5d 100644 --- a/sdks/go/pkg/beam/model/jobmanagement_v1/beam_artifact_api_grpc.pb.go +++ b/sdks/go/pkg/beam/model/jobmanagement_v1/beam_artifact_api_grpc.pb.go @@ -14,9 +14,14 @@ // WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. // See the License for the specific language governing permissions and // limitations under the License. + +// +// Protocol Buffers describing the Artifact API, for communicating with a runner +// for artifact staging and retrieval over GRPC. + // Code generated by protoc-gen-go-grpc. DO NOT EDIT. // versions: -// - protoc-gen-go-grpc v1.1.0 +// - protoc-gen-go-grpc v1.5.1 // - protoc v5.27.3 // source: org/apache/beam/model/job_management/v1/beam_artifact_api.proto @@ -31,19 +36,26 @@ import ( // This is a compile-time assertion to ensure that this generated file // is compatible with the grpc package it is being compiled against. -// Requires gRPC-Go v1.32.0 or later. -const _ = grpc.SupportPackageIsVersion7 +// Requires gRPC-Go v1.64.0 or later. +const _ = grpc.SupportPackageIsVersion9 + +const ( + ArtifactRetrievalService_ResolveArtifacts_FullMethodName = "/org.apache.beam.model.job_management.v1.ArtifactRetrievalService/ResolveArtifacts" + ArtifactRetrievalService_GetArtifact_FullMethodName = "/org.apache.beam.model.job_management.v1.ArtifactRetrievalService/GetArtifact" +) // ArtifactRetrievalServiceClient is the client API for ArtifactRetrievalService service. // // For semantics around ctx use and closing/ending streaming RPCs, please refer to https://pkg.go.dev/google.golang.org/grpc/?tab=doc#ClientConn.NewStream. +// +// A service to retrieve artifacts for use in a Job. type ArtifactRetrievalServiceClient interface { // Resolves the given artifact references into one or more replacement // artifact references (e.g. a Maven dependency into a (transitive) set // of jars. ResolveArtifacts(ctx context.Context, in *ResolveArtifactsRequest, opts ...grpc.CallOption) (*ResolveArtifactsResponse, error) // Retrieves the given artifact as a stream of bytes. - GetArtifact(ctx context.Context, in *GetArtifactRequest, opts ...grpc.CallOption) (ArtifactRetrievalService_GetArtifactClient, error) + GetArtifact(ctx context.Context, in *GetArtifactRequest, opts ...grpc.CallOption) (grpc.ServerStreamingClient[GetArtifactResponse], error) } type artifactRetrievalServiceClient struct { @@ -55,20 +67,22 @@ func NewArtifactRetrievalServiceClient(cc grpc.ClientConnInterface) ArtifactRetr } func (c *artifactRetrievalServiceClient) ResolveArtifacts(ctx context.Context, in *ResolveArtifactsRequest, opts ...grpc.CallOption) (*ResolveArtifactsResponse, error) { + cOpts := append([]grpc.CallOption{grpc.StaticMethod()}, opts...) out := new(ResolveArtifactsResponse) - err := c.cc.Invoke(ctx, "/org.apache.beam.model.job_management.v1.ArtifactRetrievalService/ResolveArtifacts", in, out, opts...) + err := c.cc.Invoke(ctx, ArtifactRetrievalService_ResolveArtifacts_FullMethodName, in, out, cOpts...) if err != nil { return nil, err } return out, nil } -func (c *artifactRetrievalServiceClient) GetArtifact(ctx context.Context, in *GetArtifactRequest, opts ...grpc.CallOption) (ArtifactRetrievalService_GetArtifactClient, error) { - stream, err := c.cc.NewStream(ctx, &ArtifactRetrievalService_ServiceDesc.Streams[0], "/org.apache.beam.model.job_management.v1.ArtifactRetrievalService/GetArtifact", opts...) +func (c *artifactRetrievalServiceClient) GetArtifact(ctx context.Context, in *GetArtifactRequest, opts ...grpc.CallOption) (grpc.ServerStreamingClient[GetArtifactResponse], error) { + cOpts := append([]grpc.CallOption{grpc.StaticMethod()}, opts...) + stream, err := c.cc.NewStream(ctx, &ArtifactRetrievalService_ServiceDesc.Streams[0], ArtifactRetrievalService_GetArtifact_FullMethodName, cOpts...) if err != nil { return nil, err } - x := &artifactRetrievalServiceGetArtifactClient{stream} + x := &grpc.GenericClientStream[GetArtifactRequest, GetArtifactResponse]{ClientStream: stream} if err := x.ClientStream.SendMsg(in); err != nil { return nil, err } @@ -78,48 +92,40 @@ func (c *artifactRetrievalServiceClient) GetArtifact(ctx context.Context, in *Ge return x, nil } -type ArtifactRetrievalService_GetArtifactClient interface { - Recv() (*GetArtifactResponse, error) - grpc.ClientStream -} - -type artifactRetrievalServiceGetArtifactClient struct { - grpc.ClientStream -} - -func (x *artifactRetrievalServiceGetArtifactClient) Recv() (*GetArtifactResponse, error) { - m := new(GetArtifactResponse) - if err := x.ClientStream.RecvMsg(m); err != nil { - return nil, err - } - return m, nil -} +// This type alias is provided for backwards compatibility with existing code that references the prior non-generic stream type by name. +type ArtifactRetrievalService_GetArtifactClient = grpc.ServerStreamingClient[GetArtifactResponse] // ArtifactRetrievalServiceServer is the server API for ArtifactRetrievalService service. // All implementations must embed UnimplementedArtifactRetrievalServiceServer -// for forward compatibility +// for forward compatibility. +// +// A service to retrieve artifacts for use in a Job. type ArtifactRetrievalServiceServer interface { // Resolves the given artifact references into one or more replacement // artifact references (e.g. a Maven dependency into a (transitive) set // of jars. ResolveArtifacts(context.Context, *ResolveArtifactsRequest) (*ResolveArtifactsResponse, error) // Retrieves the given artifact as a stream of bytes. - GetArtifact(*GetArtifactRequest, ArtifactRetrievalService_GetArtifactServer) error + GetArtifact(*GetArtifactRequest, grpc.ServerStreamingServer[GetArtifactResponse]) error mustEmbedUnimplementedArtifactRetrievalServiceServer() } -// UnimplementedArtifactRetrievalServiceServer must be embedded to have forward compatible implementations. -type UnimplementedArtifactRetrievalServiceServer struct { -} +// UnimplementedArtifactRetrievalServiceServer must be embedded to have +// forward compatible implementations. +// +// NOTE: this should be embedded by value instead of pointer to avoid a nil +// pointer dereference when methods are called. +type UnimplementedArtifactRetrievalServiceServer struct{} func (UnimplementedArtifactRetrievalServiceServer) ResolveArtifacts(context.Context, *ResolveArtifactsRequest) (*ResolveArtifactsResponse, error) { return nil, status.Errorf(codes.Unimplemented, "method ResolveArtifacts not implemented") } -func (UnimplementedArtifactRetrievalServiceServer) GetArtifact(*GetArtifactRequest, ArtifactRetrievalService_GetArtifactServer) error { +func (UnimplementedArtifactRetrievalServiceServer) GetArtifact(*GetArtifactRequest, grpc.ServerStreamingServer[GetArtifactResponse]) error { return status.Errorf(codes.Unimplemented, "method GetArtifact not implemented") } func (UnimplementedArtifactRetrievalServiceServer) mustEmbedUnimplementedArtifactRetrievalServiceServer() { } +func (UnimplementedArtifactRetrievalServiceServer) testEmbeddedByValue() {} // UnsafeArtifactRetrievalServiceServer may be embedded to opt out of forward compatibility for this service. // Use of this interface is not recommended, as added methods to ArtifactRetrievalServiceServer will @@ -129,6 +135,13 @@ type UnsafeArtifactRetrievalServiceServer interface { } func RegisterArtifactRetrievalServiceServer(s grpc.ServiceRegistrar, srv ArtifactRetrievalServiceServer) { + // If the following call pancis, it indicates UnimplementedArtifactRetrievalServiceServer was + // embedded by pointer and is nil. This will cause panics if an + // unimplemented method is ever invoked, so we test this at initialization + // time to prevent it from happening at runtime later due to I/O. + if t, ok := srv.(interface{ testEmbeddedByValue() }); ok { + t.testEmbeddedByValue() + } s.RegisterService(&ArtifactRetrievalService_ServiceDesc, srv) } @@ -142,7 +155,7 @@ func _ArtifactRetrievalService_ResolveArtifacts_Handler(srv interface{}, ctx con } info := &grpc.UnaryServerInfo{ Server: srv, - FullMethod: "/org.apache.beam.model.job_management.v1.ArtifactRetrievalService/ResolveArtifacts", + FullMethod: ArtifactRetrievalService_ResolveArtifacts_FullMethodName, } handler := func(ctx context.Context, req interface{}) (interface{}, error) { return srv.(ArtifactRetrievalServiceServer).ResolveArtifacts(ctx, req.(*ResolveArtifactsRequest)) @@ -155,21 +168,11 @@ func _ArtifactRetrievalService_GetArtifact_Handler(srv interface{}, stream grpc. if err := stream.RecvMsg(m); err != nil { return err } - return srv.(ArtifactRetrievalServiceServer).GetArtifact(m, &artifactRetrievalServiceGetArtifactServer{stream}) -} - -type ArtifactRetrievalService_GetArtifactServer interface { - Send(*GetArtifactResponse) error - grpc.ServerStream -} - -type artifactRetrievalServiceGetArtifactServer struct { - grpc.ServerStream + return srv.(ArtifactRetrievalServiceServer).GetArtifact(m, &grpc.GenericServerStream[GetArtifactRequest, GetArtifactResponse]{ServerStream: stream}) } -func (x *artifactRetrievalServiceGetArtifactServer) Send(m *GetArtifactResponse) error { - return x.ServerStream.SendMsg(m) -} +// This type alias is provided for backwards compatibility with existing code that references the prior non-generic stream type by name. +type ArtifactRetrievalService_GetArtifactServer = grpc.ServerStreamingServer[GetArtifactResponse] // ArtifactRetrievalService_ServiceDesc is the grpc.ServiceDesc for ArtifactRetrievalService service. // It's only intended for direct use with grpc.RegisterService, @@ -193,11 +196,22 @@ var ArtifactRetrievalService_ServiceDesc = grpc.ServiceDesc{ Metadata: "org/apache/beam/model/job_management/v1/beam_artifact_api.proto", } +const ( + ArtifactStagingService_ReverseArtifactRetrievalService_FullMethodName = "/org.apache.beam.model.job_management.v1.ArtifactStagingService/ReverseArtifactRetrievalService" +) + // ArtifactStagingServiceClient is the client API for ArtifactStagingService service. // // For semantics around ctx use and closing/ending streaming RPCs, please refer to https://pkg.go.dev/google.golang.org/grpc/?tab=doc#ClientConn.NewStream. +// +// A service that allows the client to act as an ArtifactRetrievalService, +// for a particular job with the server initiating requests and receiving +// responses. +// +// A client calls the service with an ArtifactResponseWrapper that has the +// staging token set, and thereafter responds to the server's requests. type ArtifactStagingServiceClient interface { - ReverseArtifactRetrievalService(ctx context.Context, opts ...grpc.CallOption) (ArtifactStagingService_ReverseArtifactRetrievalServiceClient, error) + ReverseArtifactRetrievalService(ctx context.Context, opts ...grpc.CallOption) (grpc.BidiStreamingClient[ArtifactResponseWrapper, ArtifactRequestWrapper], error) } type artifactStagingServiceClient struct { @@ -208,54 +222,47 @@ func NewArtifactStagingServiceClient(cc grpc.ClientConnInterface) ArtifactStagin return &artifactStagingServiceClient{cc} } -func (c *artifactStagingServiceClient) ReverseArtifactRetrievalService(ctx context.Context, opts ...grpc.CallOption) (ArtifactStagingService_ReverseArtifactRetrievalServiceClient, error) { - stream, err := c.cc.NewStream(ctx, &ArtifactStagingService_ServiceDesc.Streams[0], "/org.apache.beam.model.job_management.v1.ArtifactStagingService/ReverseArtifactRetrievalService", opts...) +func (c *artifactStagingServiceClient) ReverseArtifactRetrievalService(ctx context.Context, opts ...grpc.CallOption) (grpc.BidiStreamingClient[ArtifactResponseWrapper, ArtifactRequestWrapper], error) { + cOpts := append([]grpc.CallOption{grpc.StaticMethod()}, opts...) + stream, err := c.cc.NewStream(ctx, &ArtifactStagingService_ServiceDesc.Streams[0], ArtifactStagingService_ReverseArtifactRetrievalService_FullMethodName, cOpts...) if err != nil { return nil, err } - x := &artifactStagingServiceReverseArtifactRetrievalServiceClient{stream} + x := &grpc.GenericClientStream[ArtifactResponseWrapper, ArtifactRequestWrapper]{ClientStream: stream} return x, nil } -type ArtifactStagingService_ReverseArtifactRetrievalServiceClient interface { - Send(*ArtifactResponseWrapper) error - Recv() (*ArtifactRequestWrapper, error) - grpc.ClientStream -} - -type artifactStagingServiceReverseArtifactRetrievalServiceClient struct { - grpc.ClientStream -} - -func (x *artifactStagingServiceReverseArtifactRetrievalServiceClient) Send(m *ArtifactResponseWrapper) error { - return x.ClientStream.SendMsg(m) -} - -func (x *artifactStagingServiceReverseArtifactRetrievalServiceClient) Recv() (*ArtifactRequestWrapper, error) { - m := new(ArtifactRequestWrapper) - if err := x.ClientStream.RecvMsg(m); err != nil { - return nil, err - } - return m, nil -} +// This type alias is provided for backwards compatibility with existing code that references the prior non-generic stream type by name. +type ArtifactStagingService_ReverseArtifactRetrievalServiceClient = grpc.BidiStreamingClient[ArtifactResponseWrapper, ArtifactRequestWrapper] // ArtifactStagingServiceServer is the server API for ArtifactStagingService service. // All implementations must embed UnimplementedArtifactStagingServiceServer -// for forward compatibility +// for forward compatibility. +// +// A service that allows the client to act as an ArtifactRetrievalService, +// for a particular job with the server initiating requests and receiving +// responses. +// +// A client calls the service with an ArtifactResponseWrapper that has the +// staging token set, and thereafter responds to the server's requests. type ArtifactStagingServiceServer interface { - ReverseArtifactRetrievalService(ArtifactStagingService_ReverseArtifactRetrievalServiceServer) error + ReverseArtifactRetrievalService(grpc.BidiStreamingServer[ArtifactResponseWrapper, ArtifactRequestWrapper]) error mustEmbedUnimplementedArtifactStagingServiceServer() } -// UnimplementedArtifactStagingServiceServer must be embedded to have forward compatible implementations. -type UnimplementedArtifactStagingServiceServer struct { -} +// UnimplementedArtifactStagingServiceServer must be embedded to have +// forward compatible implementations. +// +// NOTE: this should be embedded by value instead of pointer to avoid a nil +// pointer dereference when methods are called. +type UnimplementedArtifactStagingServiceServer struct{} -func (UnimplementedArtifactStagingServiceServer) ReverseArtifactRetrievalService(ArtifactStagingService_ReverseArtifactRetrievalServiceServer) error { +func (UnimplementedArtifactStagingServiceServer) ReverseArtifactRetrievalService(grpc.BidiStreamingServer[ArtifactResponseWrapper, ArtifactRequestWrapper]) error { return status.Errorf(codes.Unimplemented, "method ReverseArtifactRetrievalService not implemented") } func (UnimplementedArtifactStagingServiceServer) mustEmbedUnimplementedArtifactStagingServiceServer() { } +func (UnimplementedArtifactStagingServiceServer) testEmbeddedByValue() {} // UnsafeArtifactStagingServiceServer may be embedded to opt out of forward compatibility for this service. // Use of this interface is not recommended, as added methods to ArtifactStagingServiceServer will @@ -265,34 +272,22 @@ type UnsafeArtifactStagingServiceServer interface { } func RegisterArtifactStagingServiceServer(s grpc.ServiceRegistrar, srv ArtifactStagingServiceServer) { + // If the following call pancis, it indicates UnimplementedArtifactStagingServiceServer was + // embedded by pointer and is nil. This will cause panics if an + // unimplemented method is ever invoked, so we test this at initialization + // time to prevent it from happening at runtime later due to I/O. + if t, ok := srv.(interface{ testEmbeddedByValue() }); ok { + t.testEmbeddedByValue() + } s.RegisterService(&ArtifactStagingService_ServiceDesc, srv) } func _ArtifactStagingService_ReverseArtifactRetrievalService_Handler(srv interface{}, stream grpc.ServerStream) error { - return srv.(ArtifactStagingServiceServer).ReverseArtifactRetrievalService(&artifactStagingServiceReverseArtifactRetrievalServiceServer{stream}) -} - -type ArtifactStagingService_ReverseArtifactRetrievalServiceServer interface { - Send(*ArtifactRequestWrapper) error - Recv() (*ArtifactResponseWrapper, error) - grpc.ServerStream + return srv.(ArtifactStagingServiceServer).ReverseArtifactRetrievalService(&grpc.GenericServerStream[ArtifactResponseWrapper, ArtifactRequestWrapper]{ServerStream: stream}) } -type artifactStagingServiceReverseArtifactRetrievalServiceServer struct { - grpc.ServerStream -} - -func (x *artifactStagingServiceReverseArtifactRetrievalServiceServer) Send(m *ArtifactRequestWrapper) error { - return x.ServerStream.SendMsg(m) -} - -func (x *artifactStagingServiceReverseArtifactRetrievalServiceServer) Recv() (*ArtifactResponseWrapper, error) { - m := new(ArtifactResponseWrapper) - if err := x.ServerStream.RecvMsg(m); err != nil { - return nil, err - } - return m, nil -} +// This type alias is provided for backwards compatibility with existing code that references the prior non-generic stream type by name. +type ArtifactStagingService_ReverseArtifactRetrievalServiceServer = grpc.BidiStreamingServer[ArtifactResponseWrapper, ArtifactRequestWrapper] // ArtifactStagingService_ServiceDesc is the grpc.ServiceDesc for ArtifactStagingService service. // It's only intended for direct use with grpc.RegisterService, @@ -312,14 +307,21 @@ var ArtifactStagingService_ServiceDesc = grpc.ServiceDesc{ Metadata: "org/apache/beam/model/job_management/v1/beam_artifact_api.proto", } +const ( + LegacyArtifactStagingService_PutArtifact_FullMethodName = "/org.apache.beam.model.job_management.v1.LegacyArtifactStagingService/PutArtifact" + LegacyArtifactStagingService_CommitManifest_FullMethodName = "/org.apache.beam.model.job_management.v1.LegacyArtifactStagingService/CommitManifest" +) + // LegacyArtifactStagingServiceClient is the client API for LegacyArtifactStagingService service. // // For semantics around ctx use and closing/ending streaming RPCs, please refer to https://pkg.go.dev/google.golang.org/grpc/?tab=doc#ClientConn.NewStream. +// +// A service to stage artifacts for use in a Job. type LegacyArtifactStagingServiceClient interface { // Stage an artifact to be available during job execution. The first request must contain the // name of the artifact. All future requests must contain sequential chunks of the content of // the artifact. - PutArtifact(ctx context.Context, opts ...grpc.CallOption) (LegacyArtifactStagingService_PutArtifactClient, error) + PutArtifact(ctx context.Context, opts ...grpc.CallOption) (grpc.ClientStreamingClient[PutArtifactRequest, PutArtifactResponse], error) // Commit the manifest for a Job. All artifacts must have been successfully uploaded // before this call is made. // @@ -335,43 +337,23 @@ func NewLegacyArtifactStagingServiceClient(cc grpc.ClientConnInterface) LegacyAr return &legacyArtifactStagingServiceClient{cc} } -func (c *legacyArtifactStagingServiceClient) PutArtifact(ctx context.Context, opts ...grpc.CallOption) (LegacyArtifactStagingService_PutArtifactClient, error) { - stream, err := c.cc.NewStream(ctx, &LegacyArtifactStagingService_ServiceDesc.Streams[0], "/org.apache.beam.model.job_management.v1.LegacyArtifactStagingService/PutArtifact", opts...) +func (c *legacyArtifactStagingServiceClient) PutArtifact(ctx context.Context, opts ...grpc.CallOption) (grpc.ClientStreamingClient[PutArtifactRequest, PutArtifactResponse], error) { + cOpts := append([]grpc.CallOption{grpc.StaticMethod()}, opts...) + stream, err := c.cc.NewStream(ctx, &LegacyArtifactStagingService_ServiceDesc.Streams[0], LegacyArtifactStagingService_PutArtifact_FullMethodName, cOpts...) if err != nil { return nil, err } - x := &legacyArtifactStagingServicePutArtifactClient{stream} + x := &grpc.GenericClientStream[PutArtifactRequest, PutArtifactResponse]{ClientStream: stream} return x, nil } -type LegacyArtifactStagingService_PutArtifactClient interface { - Send(*PutArtifactRequest) error - CloseAndRecv() (*PutArtifactResponse, error) - grpc.ClientStream -} - -type legacyArtifactStagingServicePutArtifactClient struct { - grpc.ClientStream -} - -func (x *legacyArtifactStagingServicePutArtifactClient) Send(m *PutArtifactRequest) error { - return x.ClientStream.SendMsg(m) -} - -func (x *legacyArtifactStagingServicePutArtifactClient) CloseAndRecv() (*PutArtifactResponse, error) { - if err := x.ClientStream.CloseSend(); err != nil { - return nil, err - } - m := new(PutArtifactResponse) - if err := x.ClientStream.RecvMsg(m); err != nil { - return nil, err - } - return m, nil -} +// This type alias is provided for backwards compatibility with existing code that references the prior non-generic stream type by name. +type LegacyArtifactStagingService_PutArtifactClient = grpc.ClientStreamingClient[PutArtifactRequest, PutArtifactResponse] func (c *legacyArtifactStagingServiceClient) CommitManifest(ctx context.Context, in *CommitManifestRequest, opts ...grpc.CallOption) (*CommitManifestResponse, error) { + cOpts := append([]grpc.CallOption{grpc.StaticMethod()}, opts...) out := new(CommitManifestResponse) - err := c.cc.Invoke(ctx, "/org.apache.beam.model.job_management.v1.LegacyArtifactStagingService/CommitManifest", in, out, opts...) + err := c.cc.Invoke(ctx, LegacyArtifactStagingService_CommitManifest_FullMethodName, in, out, cOpts...) if err != nil { return nil, err } @@ -380,12 +362,14 @@ func (c *legacyArtifactStagingServiceClient) CommitManifest(ctx context.Context, // LegacyArtifactStagingServiceServer is the server API for LegacyArtifactStagingService service. // All implementations must embed UnimplementedLegacyArtifactStagingServiceServer -// for forward compatibility +// for forward compatibility. +// +// A service to stage artifacts for use in a Job. type LegacyArtifactStagingServiceServer interface { // Stage an artifact to be available during job execution. The first request must contain the // name of the artifact. All future requests must contain sequential chunks of the content of // the artifact. - PutArtifact(LegacyArtifactStagingService_PutArtifactServer) error + PutArtifact(grpc.ClientStreamingServer[PutArtifactRequest, PutArtifactResponse]) error // Commit the manifest for a Job. All artifacts must have been successfully uploaded // before this call is made. // @@ -394,11 +378,14 @@ type LegacyArtifactStagingServiceServer interface { mustEmbedUnimplementedLegacyArtifactStagingServiceServer() } -// UnimplementedLegacyArtifactStagingServiceServer must be embedded to have forward compatible implementations. -type UnimplementedLegacyArtifactStagingServiceServer struct { -} +// UnimplementedLegacyArtifactStagingServiceServer must be embedded to have +// forward compatible implementations. +// +// NOTE: this should be embedded by value instead of pointer to avoid a nil +// pointer dereference when methods are called. +type UnimplementedLegacyArtifactStagingServiceServer struct{} -func (UnimplementedLegacyArtifactStagingServiceServer) PutArtifact(LegacyArtifactStagingService_PutArtifactServer) error { +func (UnimplementedLegacyArtifactStagingServiceServer) PutArtifact(grpc.ClientStreamingServer[PutArtifactRequest, PutArtifactResponse]) error { return status.Errorf(codes.Unimplemented, "method PutArtifact not implemented") } func (UnimplementedLegacyArtifactStagingServiceServer) CommitManifest(context.Context, *CommitManifestRequest) (*CommitManifestResponse, error) { @@ -406,6 +393,7 @@ func (UnimplementedLegacyArtifactStagingServiceServer) CommitManifest(context.Co } func (UnimplementedLegacyArtifactStagingServiceServer) mustEmbedUnimplementedLegacyArtifactStagingServiceServer() { } +func (UnimplementedLegacyArtifactStagingServiceServer) testEmbeddedByValue() {} // UnsafeLegacyArtifactStagingServiceServer may be embedded to opt out of forward compatibility for this service. // Use of this interface is not recommended, as added methods to LegacyArtifactStagingServiceServer will @@ -415,34 +403,22 @@ type UnsafeLegacyArtifactStagingServiceServer interface { } func RegisterLegacyArtifactStagingServiceServer(s grpc.ServiceRegistrar, srv LegacyArtifactStagingServiceServer) { + // If the following call pancis, it indicates UnimplementedLegacyArtifactStagingServiceServer was + // embedded by pointer and is nil. This will cause panics if an + // unimplemented method is ever invoked, so we test this at initialization + // time to prevent it from happening at runtime later due to I/O. + if t, ok := srv.(interface{ testEmbeddedByValue() }); ok { + t.testEmbeddedByValue() + } s.RegisterService(&LegacyArtifactStagingService_ServiceDesc, srv) } func _LegacyArtifactStagingService_PutArtifact_Handler(srv interface{}, stream grpc.ServerStream) error { - return srv.(LegacyArtifactStagingServiceServer).PutArtifact(&legacyArtifactStagingServicePutArtifactServer{stream}) -} - -type LegacyArtifactStagingService_PutArtifactServer interface { - SendAndClose(*PutArtifactResponse) error - Recv() (*PutArtifactRequest, error) - grpc.ServerStream -} - -type legacyArtifactStagingServicePutArtifactServer struct { - grpc.ServerStream + return srv.(LegacyArtifactStagingServiceServer).PutArtifact(&grpc.GenericServerStream[PutArtifactRequest, PutArtifactResponse]{ServerStream: stream}) } -func (x *legacyArtifactStagingServicePutArtifactServer) SendAndClose(m *PutArtifactResponse) error { - return x.ServerStream.SendMsg(m) -} - -func (x *legacyArtifactStagingServicePutArtifactServer) Recv() (*PutArtifactRequest, error) { - m := new(PutArtifactRequest) - if err := x.ServerStream.RecvMsg(m); err != nil { - return nil, err - } - return m, nil -} +// This type alias is provided for backwards compatibility with existing code that references the prior non-generic stream type by name. +type LegacyArtifactStagingService_PutArtifactServer = grpc.ClientStreamingServer[PutArtifactRequest, PutArtifactResponse] func _LegacyArtifactStagingService_CommitManifest_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { in := new(CommitManifestRequest) @@ -454,7 +430,7 @@ func _LegacyArtifactStagingService_CommitManifest_Handler(srv interface{}, ctx c } info := &grpc.UnaryServerInfo{ Server: srv, - FullMethod: "/org.apache.beam.model.job_management.v1.LegacyArtifactStagingService/CommitManifest", + FullMethod: LegacyArtifactStagingService_CommitManifest_FullMethodName, } handler := func(ctx context.Context, req interface{}) (interface{}, error) { return srv.(LegacyArtifactStagingServiceServer).CommitManifest(ctx, req.(*CommitManifestRequest)) @@ -484,14 +460,21 @@ var LegacyArtifactStagingService_ServiceDesc = grpc.ServiceDesc{ Metadata: "org/apache/beam/model/job_management/v1/beam_artifact_api.proto", } +const ( + LegacyArtifactRetrievalService_GetManifest_FullMethodName = "/org.apache.beam.model.job_management.v1.LegacyArtifactRetrievalService/GetManifest" + LegacyArtifactRetrievalService_GetArtifact_FullMethodName = "/org.apache.beam.model.job_management.v1.LegacyArtifactRetrievalService/GetArtifact" +) + // LegacyArtifactRetrievalServiceClient is the client API for LegacyArtifactRetrievalService service. // // For semantics around ctx use and closing/ending streaming RPCs, please refer to https://pkg.go.dev/google.golang.org/grpc/?tab=doc#ClientConn.NewStream. +// +// A service to retrieve artifacts for use in a Job. type LegacyArtifactRetrievalServiceClient interface { // Get the manifest for the job GetManifest(ctx context.Context, in *GetManifestRequest, opts ...grpc.CallOption) (*GetManifestResponse, error) // Get an artifact staged for the job. The requested artifact must be within the manifest - GetArtifact(ctx context.Context, in *LegacyGetArtifactRequest, opts ...grpc.CallOption) (LegacyArtifactRetrievalService_GetArtifactClient, error) + GetArtifact(ctx context.Context, in *LegacyGetArtifactRequest, opts ...grpc.CallOption) (grpc.ServerStreamingClient[ArtifactChunk], error) } type legacyArtifactRetrievalServiceClient struct { @@ -503,20 +486,22 @@ func NewLegacyArtifactRetrievalServiceClient(cc grpc.ClientConnInterface) Legacy } func (c *legacyArtifactRetrievalServiceClient) GetManifest(ctx context.Context, in *GetManifestRequest, opts ...grpc.CallOption) (*GetManifestResponse, error) { + cOpts := append([]grpc.CallOption{grpc.StaticMethod()}, opts...) out := new(GetManifestResponse) - err := c.cc.Invoke(ctx, "/org.apache.beam.model.job_management.v1.LegacyArtifactRetrievalService/GetManifest", in, out, opts...) + err := c.cc.Invoke(ctx, LegacyArtifactRetrievalService_GetManifest_FullMethodName, in, out, cOpts...) if err != nil { return nil, err } return out, nil } -func (c *legacyArtifactRetrievalServiceClient) GetArtifact(ctx context.Context, in *LegacyGetArtifactRequest, opts ...grpc.CallOption) (LegacyArtifactRetrievalService_GetArtifactClient, error) { - stream, err := c.cc.NewStream(ctx, &LegacyArtifactRetrievalService_ServiceDesc.Streams[0], "/org.apache.beam.model.job_management.v1.LegacyArtifactRetrievalService/GetArtifact", opts...) +func (c *legacyArtifactRetrievalServiceClient) GetArtifact(ctx context.Context, in *LegacyGetArtifactRequest, opts ...grpc.CallOption) (grpc.ServerStreamingClient[ArtifactChunk], error) { + cOpts := append([]grpc.CallOption{grpc.StaticMethod()}, opts...) + stream, err := c.cc.NewStream(ctx, &LegacyArtifactRetrievalService_ServiceDesc.Streams[0], LegacyArtifactRetrievalService_GetArtifact_FullMethodName, cOpts...) if err != nil { return nil, err } - x := &legacyArtifactRetrievalServiceGetArtifactClient{stream} + x := &grpc.GenericClientStream[LegacyGetArtifactRequest, ArtifactChunk]{ClientStream: stream} if err := x.ClientStream.SendMsg(in); err != nil { return nil, err } @@ -526,46 +511,38 @@ func (c *legacyArtifactRetrievalServiceClient) GetArtifact(ctx context.Context, return x, nil } -type LegacyArtifactRetrievalService_GetArtifactClient interface { - Recv() (*ArtifactChunk, error) - grpc.ClientStream -} - -type legacyArtifactRetrievalServiceGetArtifactClient struct { - grpc.ClientStream -} - -func (x *legacyArtifactRetrievalServiceGetArtifactClient) Recv() (*ArtifactChunk, error) { - m := new(ArtifactChunk) - if err := x.ClientStream.RecvMsg(m); err != nil { - return nil, err - } - return m, nil -} +// This type alias is provided for backwards compatibility with existing code that references the prior non-generic stream type by name. +type LegacyArtifactRetrievalService_GetArtifactClient = grpc.ServerStreamingClient[ArtifactChunk] // LegacyArtifactRetrievalServiceServer is the server API for LegacyArtifactRetrievalService service. // All implementations must embed UnimplementedLegacyArtifactRetrievalServiceServer -// for forward compatibility +// for forward compatibility. +// +// A service to retrieve artifacts for use in a Job. type LegacyArtifactRetrievalServiceServer interface { // Get the manifest for the job GetManifest(context.Context, *GetManifestRequest) (*GetManifestResponse, error) // Get an artifact staged for the job. The requested artifact must be within the manifest - GetArtifact(*LegacyGetArtifactRequest, LegacyArtifactRetrievalService_GetArtifactServer) error + GetArtifact(*LegacyGetArtifactRequest, grpc.ServerStreamingServer[ArtifactChunk]) error mustEmbedUnimplementedLegacyArtifactRetrievalServiceServer() } -// UnimplementedLegacyArtifactRetrievalServiceServer must be embedded to have forward compatible implementations. -type UnimplementedLegacyArtifactRetrievalServiceServer struct { -} +// UnimplementedLegacyArtifactRetrievalServiceServer must be embedded to have +// forward compatible implementations. +// +// NOTE: this should be embedded by value instead of pointer to avoid a nil +// pointer dereference when methods are called. +type UnimplementedLegacyArtifactRetrievalServiceServer struct{} func (UnimplementedLegacyArtifactRetrievalServiceServer) GetManifest(context.Context, *GetManifestRequest) (*GetManifestResponse, error) { return nil, status.Errorf(codes.Unimplemented, "method GetManifest not implemented") } -func (UnimplementedLegacyArtifactRetrievalServiceServer) GetArtifact(*LegacyGetArtifactRequest, LegacyArtifactRetrievalService_GetArtifactServer) error { +func (UnimplementedLegacyArtifactRetrievalServiceServer) GetArtifact(*LegacyGetArtifactRequest, grpc.ServerStreamingServer[ArtifactChunk]) error { return status.Errorf(codes.Unimplemented, "method GetArtifact not implemented") } func (UnimplementedLegacyArtifactRetrievalServiceServer) mustEmbedUnimplementedLegacyArtifactRetrievalServiceServer() { } +func (UnimplementedLegacyArtifactRetrievalServiceServer) testEmbeddedByValue() {} // UnsafeLegacyArtifactRetrievalServiceServer may be embedded to opt out of forward compatibility for this service. // Use of this interface is not recommended, as added methods to LegacyArtifactRetrievalServiceServer will @@ -575,6 +552,13 @@ type UnsafeLegacyArtifactRetrievalServiceServer interface { } func RegisterLegacyArtifactRetrievalServiceServer(s grpc.ServiceRegistrar, srv LegacyArtifactRetrievalServiceServer) { + // If the following call pancis, it indicates UnimplementedLegacyArtifactRetrievalServiceServer was + // embedded by pointer and is nil. This will cause panics if an + // unimplemented method is ever invoked, so we test this at initialization + // time to prevent it from happening at runtime later due to I/O. + if t, ok := srv.(interface{ testEmbeddedByValue() }); ok { + t.testEmbeddedByValue() + } s.RegisterService(&LegacyArtifactRetrievalService_ServiceDesc, srv) } @@ -588,7 +572,7 @@ func _LegacyArtifactRetrievalService_GetManifest_Handler(srv interface{}, ctx co } info := &grpc.UnaryServerInfo{ Server: srv, - FullMethod: "/org.apache.beam.model.job_management.v1.LegacyArtifactRetrievalService/GetManifest", + FullMethod: LegacyArtifactRetrievalService_GetManifest_FullMethodName, } handler := func(ctx context.Context, req interface{}) (interface{}, error) { return srv.(LegacyArtifactRetrievalServiceServer).GetManifest(ctx, req.(*GetManifestRequest)) @@ -601,21 +585,11 @@ func _LegacyArtifactRetrievalService_GetArtifact_Handler(srv interface{}, stream if err := stream.RecvMsg(m); err != nil { return err } - return srv.(LegacyArtifactRetrievalServiceServer).GetArtifact(m, &legacyArtifactRetrievalServiceGetArtifactServer{stream}) -} - -type LegacyArtifactRetrievalService_GetArtifactServer interface { - Send(*ArtifactChunk) error - grpc.ServerStream + return srv.(LegacyArtifactRetrievalServiceServer).GetArtifact(m, &grpc.GenericServerStream[LegacyGetArtifactRequest, ArtifactChunk]{ServerStream: stream}) } -type legacyArtifactRetrievalServiceGetArtifactServer struct { - grpc.ServerStream -} - -func (x *legacyArtifactRetrievalServiceGetArtifactServer) Send(m *ArtifactChunk) error { - return x.ServerStream.SendMsg(m) -} +// This type alias is provided for backwards compatibility with existing code that references the prior non-generic stream type by name. +type LegacyArtifactRetrievalService_GetArtifactServer = grpc.ServerStreamingServer[ArtifactChunk] // LegacyArtifactRetrievalService_ServiceDesc is the grpc.ServiceDesc for LegacyArtifactRetrievalService service. // It's only intended for direct use with grpc.RegisterService, diff --git a/sdks/go/pkg/beam/model/jobmanagement_v1/beam_artifact_api_protoopaque.pb.go b/sdks/go/pkg/beam/model/jobmanagement_v1/beam_artifact_api_protoopaque.pb.go new file mode 100644 index 000000000000..5f8dda75459e --- /dev/null +++ b/sdks/go/pkg/beam/model/jobmanagement_v1/beam_artifact_api_protoopaque.pb.go @@ -0,0 +1,2109 @@ +// +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +// +// Protocol Buffers describing the Artifact API, for communicating with a runner +// for artifact staging and retrieval over GRPC. + +// Code generated by protoc-gen-go. DO NOT EDIT. +// versions: +// protoc-gen-go v1.36.0 +// protoc v5.27.3 +// source: org/apache/beam/model/job_management/v1/beam_artifact_api.proto + +//go:build protoopaque + +package jobmanagement_v1 + +import ( + pipeline_v1 "github.com/apache/beam/sdks/v2/go/pkg/beam/model/pipeline_v1" + protoreflect "google.golang.org/protobuf/reflect/protoreflect" + protoimpl "google.golang.org/protobuf/runtime/protoimpl" + reflect "reflect" +) + +const ( + // Verify that this generated code is sufficiently up-to-date. + _ = protoimpl.EnforceVersion(20 - protoimpl.MinVersion) + // Verify that runtime/protoimpl is sufficiently up-to-date. + _ = protoimpl.EnforceVersion(protoimpl.MaxVersion - 20) +) + +type CommitManifestResponse_Constants int32 + +const ( + // Token indicating that no artifacts were staged and therefore no retrieval attempt is necessary. + CommitManifestResponse_NO_ARTIFACTS_STAGED_TOKEN CommitManifestResponse_Constants = 0 +) + +// Enum value maps for CommitManifestResponse_Constants. +var ( + CommitManifestResponse_Constants_name = map[int32]string{ + 0: "NO_ARTIFACTS_STAGED_TOKEN", + } + CommitManifestResponse_Constants_value = map[string]int32{ + "NO_ARTIFACTS_STAGED_TOKEN": 0, + } +) + +func (x CommitManifestResponse_Constants) Enum() *CommitManifestResponse_Constants { + p := new(CommitManifestResponse_Constants) + *p = x + return p +} + +func (x CommitManifestResponse_Constants) String() string { + return protoimpl.X.EnumStringOf(x.Descriptor(), protoreflect.EnumNumber(x)) +} + +func (CommitManifestResponse_Constants) Descriptor() protoreflect.EnumDescriptor { + return file_org_apache_beam_model_job_management_v1_beam_artifact_api_proto_enumTypes[0].Descriptor() +} + +func (CommitManifestResponse_Constants) Type() protoreflect.EnumType { + return &file_org_apache_beam_model_job_management_v1_beam_artifact_api_proto_enumTypes[0] +} + +func (x CommitManifestResponse_Constants) Number() protoreflect.EnumNumber { + return protoreflect.EnumNumber(x) +} + +// A request for artifact resolution. +type ResolveArtifactsRequest struct { + state protoimpl.MessageState `protogen:"opaque.v1"` + xxx_hidden_Artifacts *[]*pipeline_v1.ArtifactInformation `protobuf:"bytes,1,rep,name=artifacts,proto3" json:"artifacts,omitempty"` + xxx_hidden_PreferredUrns []string `protobuf:"bytes,2,rep,name=preferred_urns,json=preferredUrns,proto3" json:"preferred_urns,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *ResolveArtifactsRequest) Reset() { + *x = ResolveArtifactsRequest{} + mi := &file_org_apache_beam_model_job_management_v1_beam_artifact_api_proto_msgTypes[0] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *ResolveArtifactsRequest) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*ResolveArtifactsRequest) ProtoMessage() {} + +func (x *ResolveArtifactsRequest) ProtoReflect() protoreflect.Message { + mi := &file_org_apache_beam_model_job_management_v1_beam_artifact_api_proto_msgTypes[0] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +func (x *ResolveArtifactsRequest) GetArtifacts() []*pipeline_v1.ArtifactInformation { + if x != nil { + if x.xxx_hidden_Artifacts != nil { + return *x.xxx_hidden_Artifacts + } + } + return nil +} + +func (x *ResolveArtifactsRequest) GetPreferredUrns() []string { + if x != nil { + return x.xxx_hidden_PreferredUrns + } + return nil +} + +func (x *ResolveArtifactsRequest) SetArtifacts(v []*pipeline_v1.ArtifactInformation) { + x.xxx_hidden_Artifacts = &v +} + +func (x *ResolveArtifactsRequest) SetPreferredUrns(v []string) { + x.xxx_hidden_PreferredUrns = v +} + +type ResolveArtifactsRequest_builder struct { + _ [0]func() // Prevents comparability and use of unkeyed literals for the builder. + + // An (ordered) set of artifacts to (jointly) resolve. + Artifacts []*pipeline_v1.ArtifactInformation + // A set of artifact type urns that are understood by the requester. + // An attempt should be made to resolve the artifacts in terms of these URNs, + // but other URNs may be used as well with the understanding that they must + // be fetch-able as bytes via GetArtifact. + PreferredUrns []string +} + +func (b0 ResolveArtifactsRequest_builder) Build() *ResolveArtifactsRequest { + m0 := &ResolveArtifactsRequest{} + b, x := &b0, m0 + _, _ = b, x + x.xxx_hidden_Artifacts = &b.Artifacts + x.xxx_hidden_PreferredUrns = b.PreferredUrns + return m0 +} + +// A response for artifact resolution. +type ResolveArtifactsResponse struct { + state protoimpl.MessageState `protogen:"opaque.v1"` + xxx_hidden_Replacements *[]*pipeline_v1.ArtifactInformation `protobuf:"bytes,1,rep,name=replacements,proto3" json:"replacements,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *ResolveArtifactsResponse) Reset() { + *x = ResolveArtifactsResponse{} + mi := &file_org_apache_beam_model_job_management_v1_beam_artifact_api_proto_msgTypes[1] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *ResolveArtifactsResponse) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*ResolveArtifactsResponse) ProtoMessage() {} + +func (x *ResolveArtifactsResponse) ProtoReflect() protoreflect.Message { + mi := &file_org_apache_beam_model_job_management_v1_beam_artifact_api_proto_msgTypes[1] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +func (x *ResolveArtifactsResponse) GetReplacements() []*pipeline_v1.ArtifactInformation { + if x != nil { + if x.xxx_hidden_Replacements != nil { + return *x.xxx_hidden_Replacements + } + } + return nil +} + +func (x *ResolveArtifactsResponse) SetReplacements(v []*pipeline_v1.ArtifactInformation) { + x.xxx_hidden_Replacements = &v +} + +type ResolveArtifactsResponse_builder struct { + _ [0]func() // Prevents comparability and use of unkeyed literals for the builder. + + // A full (ordered) set of replacements for the set of requested artifacts, + // preferably in terms of the requested type URNs. If there is no better + // resolution, the original list is returned. + Replacements []*pipeline_v1.ArtifactInformation +} + +func (b0 ResolveArtifactsResponse_builder) Build() *ResolveArtifactsResponse { + m0 := &ResolveArtifactsResponse{} + b, x := &b0, m0 + _, _ = b, x + x.xxx_hidden_Replacements = &b.Replacements + return m0 +} + +// A request to get an artifact. +type GetArtifactRequest struct { + state protoimpl.MessageState `protogen:"opaque.v1"` + xxx_hidden_Artifact *pipeline_v1.ArtifactInformation `protobuf:"bytes,1,opt,name=artifact,proto3" json:"artifact,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *GetArtifactRequest) Reset() { + *x = GetArtifactRequest{} + mi := &file_org_apache_beam_model_job_management_v1_beam_artifact_api_proto_msgTypes[2] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *GetArtifactRequest) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*GetArtifactRequest) ProtoMessage() {} + +func (x *GetArtifactRequest) ProtoReflect() protoreflect.Message { + mi := &file_org_apache_beam_model_job_management_v1_beam_artifact_api_proto_msgTypes[2] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +func (x *GetArtifactRequest) GetArtifact() *pipeline_v1.ArtifactInformation { + if x != nil { + return x.xxx_hidden_Artifact + } + return nil +} + +func (x *GetArtifactRequest) SetArtifact(v *pipeline_v1.ArtifactInformation) { + x.xxx_hidden_Artifact = v +} + +func (x *GetArtifactRequest) HasArtifact() bool { + if x == nil { + return false + } + return x.xxx_hidden_Artifact != nil +} + +func (x *GetArtifactRequest) ClearArtifact() { + x.xxx_hidden_Artifact = nil +} + +type GetArtifactRequest_builder struct { + _ [0]func() // Prevents comparability and use of unkeyed literals for the builder. + + Artifact *pipeline_v1.ArtifactInformation +} + +func (b0 GetArtifactRequest_builder) Build() *GetArtifactRequest { + m0 := &GetArtifactRequest{} + b, x := &b0, m0 + _, _ = b, x + x.xxx_hidden_Artifact = b.Artifact + return m0 +} + +// Part of a response to getting an artifact. +type GetArtifactResponse struct { + state protoimpl.MessageState `protogen:"opaque.v1"` + xxx_hidden_Data []byte `protobuf:"bytes,1,opt,name=data,proto3" json:"data,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *GetArtifactResponse) Reset() { + *x = GetArtifactResponse{} + mi := &file_org_apache_beam_model_job_management_v1_beam_artifact_api_proto_msgTypes[3] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *GetArtifactResponse) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*GetArtifactResponse) ProtoMessage() {} + +func (x *GetArtifactResponse) ProtoReflect() protoreflect.Message { + mi := &file_org_apache_beam_model_job_management_v1_beam_artifact_api_proto_msgTypes[3] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +func (x *GetArtifactResponse) GetData() []byte { + if x != nil { + return x.xxx_hidden_Data + } + return nil +} + +func (x *GetArtifactResponse) SetData(v []byte) { + if v == nil { + v = []byte{} + } + x.xxx_hidden_Data = v +} + +type GetArtifactResponse_builder struct { + _ [0]func() // Prevents comparability and use of unkeyed literals for the builder. + + Data []byte +} + +func (b0 GetArtifactResponse_builder) Build() *GetArtifactResponse { + m0 := &GetArtifactResponse{} + b, x := &b0, m0 + _, _ = b, x + x.xxx_hidden_Data = b.Data + return m0 +} + +// Wraps an ArtifactRetrievalService request for use in ReverseArtifactRetrievalService. +type ArtifactRequestWrapper struct { + state protoimpl.MessageState `protogen:"opaque.v1"` + xxx_hidden_Request isArtifactRequestWrapper_Request `protobuf_oneof:"request"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *ArtifactRequestWrapper) Reset() { + *x = ArtifactRequestWrapper{} + mi := &file_org_apache_beam_model_job_management_v1_beam_artifact_api_proto_msgTypes[4] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *ArtifactRequestWrapper) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*ArtifactRequestWrapper) ProtoMessage() {} + +func (x *ArtifactRequestWrapper) ProtoReflect() protoreflect.Message { + mi := &file_org_apache_beam_model_job_management_v1_beam_artifact_api_proto_msgTypes[4] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +func (x *ArtifactRequestWrapper) GetResolveArtifact() *ResolveArtifactsRequest { + if x != nil { + if x, ok := x.xxx_hidden_Request.(*artifactRequestWrapper_ResolveArtifact); ok { + return x.ResolveArtifact + } + } + return nil +} + +func (x *ArtifactRequestWrapper) GetGetArtifact() *GetArtifactRequest { + if x != nil { + if x, ok := x.xxx_hidden_Request.(*artifactRequestWrapper_GetArtifact); ok { + return x.GetArtifact + } + } + return nil +} + +func (x *ArtifactRequestWrapper) SetResolveArtifact(v *ResolveArtifactsRequest) { + if v == nil { + x.xxx_hidden_Request = nil + return + } + x.xxx_hidden_Request = &artifactRequestWrapper_ResolveArtifact{v} +} + +func (x *ArtifactRequestWrapper) SetGetArtifact(v *GetArtifactRequest) { + if v == nil { + x.xxx_hidden_Request = nil + return + } + x.xxx_hidden_Request = &artifactRequestWrapper_GetArtifact{v} +} + +func (x *ArtifactRequestWrapper) HasRequest() bool { + if x == nil { + return false + } + return x.xxx_hidden_Request != nil +} + +func (x *ArtifactRequestWrapper) HasResolveArtifact() bool { + if x == nil { + return false + } + _, ok := x.xxx_hidden_Request.(*artifactRequestWrapper_ResolveArtifact) + return ok +} + +func (x *ArtifactRequestWrapper) HasGetArtifact() bool { + if x == nil { + return false + } + _, ok := x.xxx_hidden_Request.(*artifactRequestWrapper_GetArtifact) + return ok +} + +func (x *ArtifactRequestWrapper) ClearRequest() { + x.xxx_hidden_Request = nil +} + +func (x *ArtifactRequestWrapper) ClearResolveArtifact() { + if _, ok := x.xxx_hidden_Request.(*artifactRequestWrapper_ResolveArtifact); ok { + x.xxx_hidden_Request = nil + } +} + +func (x *ArtifactRequestWrapper) ClearGetArtifact() { + if _, ok := x.xxx_hidden_Request.(*artifactRequestWrapper_GetArtifact); ok { + x.xxx_hidden_Request = nil + } +} + +const ArtifactRequestWrapper_Request_not_set_case case_ArtifactRequestWrapper_Request = 0 +const ArtifactRequestWrapper_ResolveArtifact_case case_ArtifactRequestWrapper_Request = 1000 +const ArtifactRequestWrapper_GetArtifact_case case_ArtifactRequestWrapper_Request = 1001 + +func (x *ArtifactRequestWrapper) WhichRequest() case_ArtifactRequestWrapper_Request { + if x == nil { + return ArtifactRequestWrapper_Request_not_set_case + } + switch x.xxx_hidden_Request.(type) { + case *artifactRequestWrapper_ResolveArtifact: + return ArtifactRequestWrapper_ResolveArtifact_case + case *artifactRequestWrapper_GetArtifact: + return ArtifactRequestWrapper_GetArtifact_case + default: + return ArtifactRequestWrapper_Request_not_set_case + } +} + +type ArtifactRequestWrapper_builder struct { + _ [0]func() // Prevents comparability and use of unkeyed literals for the builder. + + // Fields of oneof xxx_hidden_Request: + ResolveArtifact *ResolveArtifactsRequest + GetArtifact *GetArtifactRequest + // -- end of xxx_hidden_Request +} + +func (b0 ArtifactRequestWrapper_builder) Build() *ArtifactRequestWrapper { + m0 := &ArtifactRequestWrapper{} + b, x := &b0, m0 + _, _ = b, x + if b.ResolveArtifact != nil { + x.xxx_hidden_Request = &artifactRequestWrapper_ResolveArtifact{b.ResolveArtifact} + } + if b.GetArtifact != nil { + x.xxx_hidden_Request = &artifactRequestWrapper_GetArtifact{b.GetArtifact} + } + return m0 +} + +type case_ArtifactRequestWrapper_Request protoreflect.FieldNumber + +func (x case_ArtifactRequestWrapper_Request) String() string { + md := file_org_apache_beam_model_job_management_v1_beam_artifact_api_proto_msgTypes[4].Descriptor() + if x == 0 { + return "not set" + } + return protoimpl.X.MessageFieldStringOf(md, protoreflect.FieldNumber(x)) +} + +type isArtifactRequestWrapper_Request interface { + isArtifactRequestWrapper_Request() +} + +type artifactRequestWrapper_ResolveArtifact struct { + ResolveArtifact *ResolveArtifactsRequest `protobuf:"bytes,1000,opt,name=resolve_artifact,json=resolveArtifact,proto3,oneof"` +} + +type artifactRequestWrapper_GetArtifact struct { + GetArtifact *GetArtifactRequest `protobuf:"bytes,1001,opt,name=get_artifact,json=getArtifact,proto3,oneof"` +} + +func (*artifactRequestWrapper_ResolveArtifact) isArtifactRequestWrapper_Request() {} + +func (*artifactRequestWrapper_GetArtifact) isArtifactRequestWrapper_Request() {} + +// Wraps an ArtifactRetrievalService response for use in ReverseArtifactRetrievalService. +type ArtifactResponseWrapper struct { + state protoimpl.MessageState `protogen:"opaque.v1"` + xxx_hidden_StagingToken string `protobuf:"bytes,1,opt,name=staging_token,json=stagingToken,proto3" json:"staging_token,omitempty"` + xxx_hidden_IsLast bool `protobuf:"varint,2,opt,name=is_last,json=isLast,proto3" json:"is_last,omitempty"` + xxx_hidden_Response isArtifactResponseWrapper_Response `protobuf_oneof:"response"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *ArtifactResponseWrapper) Reset() { + *x = ArtifactResponseWrapper{} + mi := &file_org_apache_beam_model_job_management_v1_beam_artifact_api_proto_msgTypes[5] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *ArtifactResponseWrapper) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*ArtifactResponseWrapper) ProtoMessage() {} + +func (x *ArtifactResponseWrapper) ProtoReflect() protoreflect.Message { + mi := &file_org_apache_beam_model_job_management_v1_beam_artifact_api_proto_msgTypes[5] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +func (x *ArtifactResponseWrapper) GetStagingToken() string { + if x != nil { + return x.xxx_hidden_StagingToken + } + return "" +} + +func (x *ArtifactResponseWrapper) GetIsLast() bool { + if x != nil { + return x.xxx_hidden_IsLast + } + return false +} + +func (x *ArtifactResponseWrapper) GetResolveArtifactResponse() *ResolveArtifactsResponse { + if x != nil { + if x, ok := x.xxx_hidden_Response.(*artifactResponseWrapper_ResolveArtifactResponse); ok { + return x.ResolveArtifactResponse + } + } + return nil +} + +func (x *ArtifactResponseWrapper) GetGetArtifactResponse() *GetArtifactResponse { + if x != nil { + if x, ok := x.xxx_hidden_Response.(*artifactResponseWrapper_GetArtifactResponse); ok { + return x.GetArtifactResponse + } + } + return nil +} + +func (x *ArtifactResponseWrapper) SetStagingToken(v string) { + x.xxx_hidden_StagingToken = v +} + +func (x *ArtifactResponseWrapper) SetIsLast(v bool) { + x.xxx_hidden_IsLast = v +} + +func (x *ArtifactResponseWrapper) SetResolveArtifactResponse(v *ResolveArtifactsResponse) { + if v == nil { + x.xxx_hidden_Response = nil + return + } + x.xxx_hidden_Response = &artifactResponseWrapper_ResolveArtifactResponse{v} +} + +func (x *ArtifactResponseWrapper) SetGetArtifactResponse(v *GetArtifactResponse) { + if v == nil { + x.xxx_hidden_Response = nil + return + } + x.xxx_hidden_Response = &artifactResponseWrapper_GetArtifactResponse{v} +} + +func (x *ArtifactResponseWrapper) HasResponse() bool { + if x == nil { + return false + } + return x.xxx_hidden_Response != nil +} + +func (x *ArtifactResponseWrapper) HasResolveArtifactResponse() bool { + if x == nil { + return false + } + _, ok := x.xxx_hidden_Response.(*artifactResponseWrapper_ResolveArtifactResponse) + return ok +} + +func (x *ArtifactResponseWrapper) HasGetArtifactResponse() bool { + if x == nil { + return false + } + _, ok := x.xxx_hidden_Response.(*artifactResponseWrapper_GetArtifactResponse) + return ok +} + +func (x *ArtifactResponseWrapper) ClearResponse() { + x.xxx_hidden_Response = nil +} + +func (x *ArtifactResponseWrapper) ClearResolveArtifactResponse() { + if _, ok := x.xxx_hidden_Response.(*artifactResponseWrapper_ResolveArtifactResponse); ok { + x.xxx_hidden_Response = nil + } +} + +func (x *ArtifactResponseWrapper) ClearGetArtifactResponse() { + if _, ok := x.xxx_hidden_Response.(*artifactResponseWrapper_GetArtifactResponse); ok { + x.xxx_hidden_Response = nil + } +} + +const ArtifactResponseWrapper_Response_not_set_case case_ArtifactResponseWrapper_Response = 0 +const ArtifactResponseWrapper_ResolveArtifactResponse_case case_ArtifactResponseWrapper_Response = 1000 +const ArtifactResponseWrapper_GetArtifactResponse_case case_ArtifactResponseWrapper_Response = 1001 + +func (x *ArtifactResponseWrapper) WhichResponse() case_ArtifactResponseWrapper_Response { + if x == nil { + return ArtifactResponseWrapper_Response_not_set_case + } + switch x.xxx_hidden_Response.(type) { + case *artifactResponseWrapper_ResolveArtifactResponse: + return ArtifactResponseWrapper_ResolveArtifactResponse_case + case *artifactResponseWrapper_GetArtifactResponse: + return ArtifactResponseWrapper_GetArtifactResponse_case + default: + return ArtifactResponseWrapper_Response_not_set_case + } +} + +type ArtifactResponseWrapper_builder struct { + _ [0]func() // Prevents comparability and use of unkeyed literals for the builder. + + // A token indicating which job these artifacts are being staged for. + StagingToken string + // Whether this is the last response for this request (for those responses that + // would typically be terminated by the end of the response stream.) + IsLast bool + // The response itself. + + // Fields of oneof xxx_hidden_Response: + ResolveArtifactResponse *ResolveArtifactsResponse + GetArtifactResponse *GetArtifactResponse + // -- end of xxx_hidden_Response +} + +func (b0 ArtifactResponseWrapper_builder) Build() *ArtifactResponseWrapper { + m0 := &ArtifactResponseWrapper{} + b, x := &b0, m0 + _, _ = b, x + x.xxx_hidden_StagingToken = b.StagingToken + x.xxx_hidden_IsLast = b.IsLast + if b.ResolveArtifactResponse != nil { + x.xxx_hidden_Response = &artifactResponseWrapper_ResolveArtifactResponse{b.ResolveArtifactResponse} + } + if b.GetArtifactResponse != nil { + x.xxx_hidden_Response = &artifactResponseWrapper_GetArtifactResponse{b.GetArtifactResponse} + } + return m0 +} + +type case_ArtifactResponseWrapper_Response protoreflect.FieldNumber + +func (x case_ArtifactResponseWrapper_Response) String() string { + md := file_org_apache_beam_model_job_management_v1_beam_artifact_api_proto_msgTypes[5].Descriptor() + if x == 0 { + return "not set" + } + return protoimpl.X.MessageFieldStringOf(md, protoreflect.FieldNumber(x)) +} + +type isArtifactResponseWrapper_Response interface { + isArtifactResponseWrapper_Response() +} + +type artifactResponseWrapper_ResolveArtifactResponse struct { + ResolveArtifactResponse *ResolveArtifactsResponse `protobuf:"bytes,1000,opt,name=resolve_artifact_response,json=resolveArtifactResponse,proto3,oneof"` +} + +type artifactResponseWrapper_GetArtifactResponse struct { + GetArtifactResponse *GetArtifactResponse `protobuf:"bytes,1001,opt,name=get_artifact_response,json=getArtifactResponse,proto3,oneof"` +} + +func (*artifactResponseWrapper_ResolveArtifactResponse) isArtifactResponseWrapper_Response() {} + +func (*artifactResponseWrapper_GetArtifactResponse) isArtifactResponseWrapper_Response() {} + +// An artifact identifier and associated metadata. +type ArtifactMetadata struct { + state protoimpl.MessageState `protogen:"opaque.v1"` + xxx_hidden_Name string `protobuf:"bytes,1,opt,name=name,proto3" json:"name,omitempty"` + xxx_hidden_Permissions uint32 `protobuf:"varint,2,opt,name=permissions,proto3" json:"permissions,omitempty"` + xxx_hidden_Sha256 string `protobuf:"bytes,4,opt,name=sha256,proto3" json:"sha256,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *ArtifactMetadata) Reset() { + *x = ArtifactMetadata{} + mi := &file_org_apache_beam_model_job_management_v1_beam_artifact_api_proto_msgTypes[6] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *ArtifactMetadata) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*ArtifactMetadata) ProtoMessage() {} + +func (x *ArtifactMetadata) ProtoReflect() protoreflect.Message { + mi := &file_org_apache_beam_model_job_management_v1_beam_artifact_api_proto_msgTypes[6] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +func (x *ArtifactMetadata) GetName() string { + if x != nil { + return x.xxx_hidden_Name + } + return "" +} + +func (x *ArtifactMetadata) GetPermissions() uint32 { + if x != nil { + return x.xxx_hidden_Permissions + } + return 0 +} + +func (x *ArtifactMetadata) GetSha256() string { + if x != nil { + return x.xxx_hidden_Sha256 + } + return "" +} + +func (x *ArtifactMetadata) SetName(v string) { + x.xxx_hidden_Name = v +} + +func (x *ArtifactMetadata) SetPermissions(v uint32) { + x.xxx_hidden_Permissions = v +} + +func (x *ArtifactMetadata) SetSha256(v string) { + x.xxx_hidden_Sha256 = v +} + +type ArtifactMetadata_builder struct { + _ [0]func() // Prevents comparability and use of unkeyed literals for the builder. + + // (Required) The name of the artifact. + Name string + // (Optional) The Unix-like permissions of the artifact + Permissions uint32 + // (Optional) The hex-encoded sha256 checksum of the artifact. Used, among other things, by + // harness boot code to validate the integrity of the artifact. + Sha256 string +} + +func (b0 ArtifactMetadata_builder) Build() *ArtifactMetadata { + m0 := &ArtifactMetadata{} + b, x := &b0, m0 + _, _ = b, x + x.xxx_hidden_Name = b.Name + x.xxx_hidden_Permissions = b.Permissions + x.xxx_hidden_Sha256 = b.Sha256 + return m0 +} + +// A collection of artifacts. +type Manifest struct { + state protoimpl.MessageState `protogen:"opaque.v1"` + xxx_hidden_Artifact *[]*ArtifactMetadata `protobuf:"bytes,1,rep,name=artifact,proto3" json:"artifact,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *Manifest) Reset() { + *x = Manifest{} + mi := &file_org_apache_beam_model_job_management_v1_beam_artifact_api_proto_msgTypes[7] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *Manifest) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*Manifest) ProtoMessage() {} + +func (x *Manifest) ProtoReflect() protoreflect.Message { + mi := &file_org_apache_beam_model_job_management_v1_beam_artifact_api_proto_msgTypes[7] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +func (x *Manifest) GetArtifact() []*ArtifactMetadata { + if x != nil { + if x.xxx_hidden_Artifact != nil { + return *x.xxx_hidden_Artifact + } + } + return nil +} + +func (x *Manifest) SetArtifact(v []*ArtifactMetadata) { + x.xxx_hidden_Artifact = &v +} + +type Manifest_builder struct { + _ [0]func() // Prevents comparability and use of unkeyed literals for the builder. + + Artifact []*ArtifactMetadata +} + +func (b0 Manifest_builder) Build() *Manifest { + m0 := &Manifest{} + b, x := &b0, m0 + _, _ = b, x + x.xxx_hidden_Artifact = &b.Artifact + return m0 +} + +// A manifest with location information. +type ProxyManifest struct { + state protoimpl.MessageState `protogen:"opaque.v1"` + xxx_hidden_Manifest *Manifest `protobuf:"bytes,1,opt,name=manifest,proto3" json:"manifest,omitempty"` + xxx_hidden_Location *[]*ProxyManifest_Location `protobuf:"bytes,2,rep,name=location,proto3" json:"location,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *ProxyManifest) Reset() { + *x = ProxyManifest{} + mi := &file_org_apache_beam_model_job_management_v1_beam_artifact_api_proto_msgTypes[8] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *ProxyManifest) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*ProxyManifest) ProtoMessage() {} + +func (x *ProxyManifest) ProtoReflect() protoreflect.Message { + mi := &file_org_apache_beam_model_job_management_v1_beam_artifact_api_proto_msgTypes[8] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +func (x *ProxyManifest) GetManifest() *Manifest { + if x != nil { + return x.xxx_hidden_Manifest + } + return nil +} + +func (x *ProxyManifest) GetLocation() []*ProxyManifest_Location { + if x != nil { + if x.xxx_hidden_Location != nil { + return *x.xxx_hidden_Location + } + } + return nil +} + +func (x *ProxyManifest) SetManifest(v *Manifest) { + x.xxx_hidden_Manifest = v +} + +func (x *ProxyManifest) SetLocation(v []*ProxyManifest_Location) { + x.xxx_hidden_Location = &v +} + +func (x *ProxyManifest) HasManifest() bool { + if x == nil { + return false + } + return x.xxx_hidden_Manifest != nil +} + +func (x *ProxyManifest) ClearManifest() { + x.xxx_hidden_Manifest = nil +} + +type ProxyManifest_builder struct { + _ [0]func() // Prevents comparability and use of unkeyed literals for the builder. + + Manifest *Manifest + Location []*ProxyManifest_Location +} + +func (b0 ProxyManifest_builder) Build() *ProxyManifest { + m0 := &ProxyManifest{} + b, x := &b0, m0 + _, _ = b, x + x.xxx_hidden_Manifest = b.Manifest + x.xxx_hidden_Location = &b.Location + return m0 +} + +// A request to get the manifest of a Job. +type GetManifestRequest struct { + state protoimpl.MessageState `protogen:"opaque.v1"` + xxx_hidden_RetrievalToken string `protobuf:"bytes,1,opt,name=retrieval_token,json=retrievalToken,proto3" json:"retrieval_token,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *GetManifestRequest) Reset() { + *x = GetManifestRequest{} + mi := &file_org_apache_beam_model_job_management_v1_beam_artifact_api_proto_msgTypes[9] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *GetManifestRequest) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*GetManifestRequest) ProtoMessage() {} + +func (x *GetManifestRequest) ProtoReflect() protoreflect.Message { + mi := &file_org_apache_beam_model_job_management_v1_beam_artifact_api_proto_msgTypes[9] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +func (x *GetManifestRequest) GetRetrievalToken() string { + if x != nil { + return x.xxx_hidden_RetrievalToken + } + return "" +} + +func (x *GetManifestRequest) SetRetrievalToken(v string) { + x.xxx_hidden_RetrievalToken = v +} + +type GetManifestRequest_builder struct { + _ [0]func() // Prevents comparability and use of unkeyed literals for the builder. + + // (Required) An opaque token representing the entirety of the staged artifacts. + // Returned in CommitManifestResponse. + RetrievalToken string +} + +func (b0 GetManifestRequest_builder) Build() *GetManifestRequest { + m0 := &GetManifestRequest{} + b, x := &b0, m0 + _, _ = b, x + x.xxx_hidden_RetrievalToken = b.RetrievalToken + return m0 +} + +// A response containing a job manifest. +type GetManifestResponse struct { + state protoimpl.MessageState `protogen:"opaque.v1"` + xxx_hidden_Manifest *Manifest `protobuf:"bytes,1,opt,name=manifest,proto3" json:"manifest,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *GetManifestResponse) Reset() { + *x = GetManifestResponse{} + mi := &file_org_apache_beam_model_job_management_v1_beam_artifact_api_proto_msgTypes[10] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *GetManifestResponse) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*GetManifestResponse) ProtoMessage() {} + +func (x *GetManifestResponse) ProtoReflect() protoreflect.Message { + mi := &file_org_apache_beam_model_job_management_v1_beam_artifact_api_proto_msgTypes[10] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +func (x *GetManifestResponse) GetManifest() *Manifest { + if x != nil { + return x.xxx_hidden_Manifest + } + return nil +} + +func (x *GetManifestResponse) SetManifest(v *Manifest) { + x.xxx_hidden_Manifest = v +} + +func (x *GetManifestResponse) HasManifest() bool { + if x == nil { + return false + } + return x.xxx_hidden_Manifest != nil +} + +func (x *GetManifestResponse) ClearManifest() { + x.xxx_hidden_Manifest = nil +} + +type GetManifestResponse_builder struct { + _ [0]func() // Prevents comparability and use of unkeyed literals for the builder. + + Manifest *Manifest +} + +func (b0 GetManifestResponse_builder) Build() *GetManifestResponse { + m0 := &GetManifestResponse{} + b, x := &b0, m0 + _, _ = b, x + x.xxx_hidden_Manifest = b.Manifest + return m0 +} + +// A request to get an artifact. The artifact must be present in the manifest for the job. +type LegacyGetArtifactRequest struct { + state protoimpl.MessageState `protogen:"opaque.v1"` + xxx_hidden_Name string `protobuf:"bytes,1,opt,name=name,proto3" json:"name,omitempty"` + xxx_hidden_RetrievalToken string `protobuf:"bytes,2,opt,name=retrieval_token,json=retrievalToken,proto3" json:"retrieval_token,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *LegacyGetArtifactRequest) Reset() { + *x = LegacyGetArtifactRequest{} + mi := &file_org_apache_beam_model_job_management_v1_beam_artifact_api_proto_msgTypes[11] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *LegacyGetArtifactRequest) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*LegacyGetArtifactRequest) ProtoMessage() {} + +func (x *LegacyGetArtifactRequest) ProtoReflect() protoreflect.Message { + mi := &file_org_apache_beam_model_job_management_v1_beam_artifact_api_proto_msgTypes[11] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +func (x *LegacyGetArtifactRequest) GetName() string { + if x != nil { + return x.xxx_hidden_Name + } + return "" +} + +func (x *LegacyGetArtifactRequest) GetRetrievalToken() string { + if x != nil { + return x.xxx_hidden_RetrievalToken + } + return "" +} + +func (x *LegacyGetArtifactRequest) SetName(v string) { + x.xxx_hidden_Name = v +} + +func (x *LegacyGetArtifactRequest) SetRetrievalToken(v string) { + x.xxx_hidden_RetrievalToken = v +} + +type LegacyGetArtifactRequest_builder struct { + _ [0]func() // Prevents comparability and use of unkeyed literals for the builder. + + // (Required) The name of the artifact to retrieve. + Name string + // (Required) An opaque token representing the entirety of the staged artifacts. + // Returned in CommitManifestResponse. + RetrievalToken string +} + +func (b0 LegacyGetArtifactRequest_builder) Build() *LegacyGetArtifactRequest { + m0 := &LegacyGetArtifactRequest{} + b, x := &b0, m0 + _, _ = b, x + x.xxx_hidden_Name = b.Name + x.xxx_hidden_RetrievalToken = b.RetrievalToken + return m0 +} + +// Part of an artifact. +type ArtifactChunk struct { + state protoimpl.MessageState `protogen:"opaque.v1"` + xxx_hidden_Data []byte `protobuf:"bytes,1,opt,name=data,proto3" json:"data,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *ArtifactChunk) Reset() { + *x = ArtifactChunk{} + mi := &file_org_apache_beam_model_job_management_v1_beam_artifact_api_proto_msgTypes[12] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *ArtifactChunk) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*ArtifactChunk) ProtoMessage() {} + +func (x *ArtifactChunk) ProtoReflect() protoreflect.Message { + mi := &file_org_apache_beam_model_job_management_v1_beam_artifact_api_proto_msgTypes[12] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +func (x *ArtifactChunk) GetData() []byte { + if x != nil { + return x.xxx_hidden_Data + } + return nil +} + +func (x *ArtifactChunk) SetData(v []byte) { + if v == nil { + v = []byte{} + } + x.xxx_hidden_Data = v +} + +type ArtifactChunk_builder struct { + _ [0]func() // Prevents comparability and use of unkeyed literals for the builder. + + Data []byte +} + +func (b0 ArtifactChunk_builder) Build() *ArtifactChunk { + m0 := &ArtifactChunk{} + b, x := &b0, m0 + _, _ = b, x + x.xxx_hidden_Data = b.Data + return m0 +} + +type PutArtifactMetadata struct { + state protoimpl.MessageState `protogen:"opaque.v1"` + xxx_hidden_StagingSessionToken string `protobuf:"bytes,1,opt,name=staging_session_token,json=stagingSessionToken,proto3" json:"staging_session_token,omitempty"` + xxx_hidden_Metadata *ArtifactMetadata `protobuf:"bytes,2,opt,name=metadata,proto3" json:"metadata,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *PutArtifactMetadata) Reset() { + *x = PutArtifactMetadata{} + mi := &file_org_apache_beam_model_job_management_v1_beam_artifact_api_proto_msgTypes[13] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *PutArtifactMetadata) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*PutArtifactMetadata) ProtoMessage() {} + +func (x *PutArtifactMetadata) ProtoReflect() protoreflect.Message { + mi := &file_org_apache_beam_model_job_management_v1_beam_artifact_api_proto_msgTypes[13] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +func (x *PutArtifactMetadata) GetStagingSessionToken() string { + if x != nil { + return x.xxx_hidden_StagingSessionToken + } + return "" +} + +func (x *PutArtifactMetadata) GetMetadata() *ArtifactMetadata { + if x != nil { + return x.xxx_hidden_Metadata + } + return nil +} + +func (x *PutArtifactMetadata) SetStagingSessionToken(v string) { + x.xxx_hidden_StagingSessionToken = v +} + +func (x *PutArtifactMetadata) SetMetadata(v *ArtifactMetadata) { + x.xxx_hidden_Metadata = v +} + +func (x *PutArtifactMetadata) HasMetadata() bool { + if x == nil { + return false + } + return x.xxx_hidden_Metadata != nil +} + +func (x *PutArtifactMetadata) ClearMetadata() { + x.xxx_hidden_Metadata = nil +} + +type PutArtifactMetadata_builder struct { + _ [0]func() // Prevents comparability and use of unkeyed literals for the builder. + + // (Required) A token for artifact staging session. This token can be obtained + // from PrepareJob request in JobService + StagingSessionToken string + // (Required) The Artifact metadata. + Metadata *ArtifactMetadata +} + +func (b0 PutArtifactMetadata_builder) Build() *PutArtifactMetadata { + m0 := &PutArtifactMetadata{} + b, x := &b0, m0 + _, _ = b, x + x.xxx_hidden_StagingSessionToken = b.StagingSessionToken + x.xxx_hidden_Metadata = b.Metadata + return m0 +} + +// A request to stage an artifact. +type PutArtifactRequest struct { + state protoimpl.MessageState `protogen:"opaque.v1"` + xxx_hidden_Content isPutArtifactRequest_Content `protobuf_oneof:"content"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *PutArtifactRequest) Reset() { + *x = PutArtifactRequest{} + mi := &file_org_apache_beam_model_job_management_v1_beam_artifact_api_proto_msgTypes[14] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *PutArtifactRequest) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*PutArtifactRequest) ProtoMessage() {} + +func (x *PutArtifactRequest) ProtoReflect() protoreflect.Message { + mi := &file_org_apache_beam_model_job_management_v1_beam_artifact_api_proto_msgTypes[14] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +func (x *PutArtifactRequest) GetMetadata() *PutArtifactMetadata { + if x != nil { + if x, ok := x.xxx_hidden_Content.(*putArtifactRequest_Metadata); ok { + return x.Metadata + } + } + return nil +} + +func (x *PutArtifactRequest) GetData() *ArtifactChunk { + if x != nil { + if x, ok := x.xxx_hidden_Content.(*putArtifactRequest_Data); ok { + return x.Data + } + } + return nil +} + +func (x *PutArtifactRequest) SetMetadata(v *PutArtifactMetadata) { + if v == nil { + x.xxx_hidden_Content = nil + return + } + x.xxx_hidden_Content = &putArtifactRequest_Metadata{v} +} + +func (x *PutArtifactRequest) SetData(v *ArtifactChunk) { + if v == nil { + x.xxx_hidden_Content = nil + return + } + x.xxx_hidden_Content = &putArtifactRequest_Data{v} +} + +func (x *PutArtifactRequest) HasContent() bool { + if x == nil { + return false + } + return x.xxx_hidden_Content != nil +} + +func (x *PutArtifactRequest) HasMetadata() bool { + if x == nil { + return false + } + _, ok := x.xxx_hidden_Content.(*putArtifactRequest_Metadata) + return ok +} + +func (x *PutArtifactRequest) HasData() bool { + if x == nil { + return false + } + _, ok := x.xxx_hidden_Content.(*putArtifactRequest_Data) + return ok +} + +func (x *PutArtifactRequest) ClearContent() { + x.xxx_hidden_Content = nil +} + +func (x *PutArtifactRequest) ClearMetadata() { + if _, ok := x.xxx_hidden_Content.(*putArtifactRequest_Metadata); ok { + x.xxx_hidden_Content = nil + } +} + +func (x *PutArtifactRequest) ClearData() { + if _, ok := x.xxx_hidden_Content.(*putArtifactRequest_Data); ok { + x.xxx_hidden_Content = nil + } +} + +const PutArtifactRequest_Content_not_set_case case_PutArtifactRequest_Content = 0 +const PutArtifactRequest_Metadata_case case_PutArtifactRequest_Content = 1 +const PutArtifactRequest_Data_case case_PutArtifactRequest_Content = 2 + +func (x *PutArtifactRequest) WhichContent() case_PutArtifactRequest_Content { + if x == nil { + return PutArtifactRequest_Content_not_set_case + } + switch x.xxx_hidden_Content.(type) { + case *putArtifactRequest_Metadata: + return PutArtifactRequest_Metadata_case + case *putArtifactRequest_Data: + return PutArtifactRequest_Data_case + default: + return PutArtifactRequest_Content_not_set_case + } +} + +type PutArtifactRequest_builder struct { + _ [0]func() // Prevents comparability and use of unkeyed literals for the builder. + + // (Required) + + // Fields of oneof xxx_hidden_Content: + // The first message in a PutArtifact call must contain this field. + Metadata *PutArtifactMetadata + // A chunk of the artifact. All messages after the first in a PutArtifact call must contain a + // chunk. + Data *ArtifactChunk + // -- end of xxx_hidden_Content +} + +func (b0 PutArtifactRequest_builder) Build() *PutArtifactRequest { + m0 := &PutArtifactRequest{} + b, x := &b0, m0 + _, _ = b, x + if b.Metadata != nil { + x.xxx_hidden_Content = &putArtifactRequest_Metadata{b.Metadata} + } + if b.Data != nil { + x.xxx_hidden_Content = &putArtifactRequest_Data{b.Data} + } + return m0 +} + +type case_PutArtifactRequest_Content protoreflect.FieldNumber + +func (x case_PutArtifactRequest_Content) String() string { + md := file_org_apache_beam_model_job_management_v1_beam_artifact_api_proto_msgTypes[14].Descriptor() + if x == 0 { + return "not set" + } + return protoimpl.X.MessageFieldStringOf(md, protoreflect.FieldNumber(x)) +} + +type isPutArtifactRequest_Content interface { + isPutArtifactRequest_Content() +} + +type putArtifactRequest_Metadata struct { + // The first message in a PutArtifact call must contain this field. + Metadata *PutArtifactMetadata `protobuf:"bytes,1,opt,name=metadata,proto3,oneof"` +} + +type putArtifactRequest_Data struct { + // A chunk of the artifact. All messages after the first in a PutArtifact call must contain a + // chunk. + Data *ArtifactChunk `protobuf:"bytes,2,opt,name=data,proto3,oneof"` +} + +func (*putArtifactRequest_Metadata) isPutArtifactRequest_Content() {} + +func (*putArtifactRequest_Data) isPutArtifactRequest_Content() {} + +type PutArtifactResponse struct { + state protoimpl.MessageState `protogen:"opaque.v1"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *PutArtifactResponse) Reset() { + *x = PutArtifactResponse{} + mi := &file_org_apache_beam_model_job_management_v1_beam_artifact_api_proto_msgTypes[15] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *PutArtifactResponse) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*PutArtifactResponse) ProtoMessage() {} + +func (x *PutArtifactResponse) ProtoReflect() protoreflect.Message { + mi := &file_org_apache_beam_model_job_management_v1_beam_artifact_api_proto_msgTypes[15] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +type PutArtifactResponse_builder struct { + _ [0]func() // Prevents comparability and use of unkeyed literals for the builder. + +} + +func (b0 PutArtifactResponse_builder) Build() *PutArtifactResponse { + m0 := &PutArtifactResponse{} + b, x := &b0, m0 + _, _ = b, x + return m0 +} + +// A request to commit the manifest for a Job. All artifacts must have been successfully uploaded +// before this call is made. +type CommitManifestRequest struct { + state protoimpl.MessageState `protogen:"opaque.v1"` + xxx_hidden_Manifest *Manifest `protobuf:"bytes,1,opt,name=manifest,proto3" json:"manifest,omitempty"` + xxx_hidden_StagingSessionToken string `protobuf:"bytes,2,opt,name=staging_session_token,json=stagingSessionToken,proto3" json:"staging_session_token,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *CommitManifestRequest) Reset() { + *x = CommitManifestRequest{} + mi := &file_org_apache_beam_model_job_management_v1_beam_artifact_api_proto_msgTypes[16] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *CommitManifestRequest) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*CommitManifestRequest) ProtoMessage() {} + +func (x *CommitManifestRequest) ProtoReflect() protoreflect.Message { + mi := &file_org_apache_beam_model_job_management_v1_beam_artifact_api_proto_msgTypes[16] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +func (x *CommitManifestRequest) GetManifest() *Manifest { + if x != nil { + return x.xxx_hidden_Manifest + } + return nil +} + +func (x *CommitManifestRequest) GetStagingSessionToken() string { + if x != nil { + return x.xxx_hidden_StagingSessionToken + } + return "" +} + +func (x *CommitManifestRequest) SetManifest(v *Manifest) { + x.xxx_hidden_Manifest = v +} + +func (x *CommitManifestRequest) SetStagingSessionToken(v string) { + x.xxx_hidden_StagingSessionToken = v +} + +func (x *CommitManifestRequest) HasManifest() bool { + if x == nil { + return false + } + return x.xxx_hidden_Manifest != nil +} + +func (x *CommitManifestRequest) ClearManifest() { + x.xxx_hidden_Manifest = nil +} + +type CommitManifestRequest_builder struct { + _ [0]func() // Prevents comparability and use of unkeyed literals for the builder. + + // (Required) The manifest to commit. + Manifest *Manifest + // (Required) A token for artifact staging session. This token can be obtained + // from PrepareJob request in JobService + StagingSessionToken string +} + +func (b0 CommitManifestRequest_builder) Build() *CommitManifestRequest { + m0 := &CommitManifestRequest{} + b, x := &b0, m0 + _, _ = b, x + x.xxx_hidden_Manifest = b.Manifest + x.xxx_hidden_StagingSessionToken = b.StagingSessionToken + return m0 +} + +// The result of committing a manifest. +type CommitManifestResponse struct { + state protoimpl.MessageState `protogen:"opaque.v1"` + xxx_hidden_RetrievalToken string `protobuf:"bytes,1,opt,name=retrieval_token,json=retrievalToken,proto3" json:"retrieval_token,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *CommitManifestResponse) Reset() { + *x = CommitManifestResponse{} + mi := &file_org_apache_beam_model_job_management_v1_beam_artifact_api_proto_msgTypes[17] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *CommitManifestResponse) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*CommitManifestResponse) ProtoMessage() {} + +func (x *CommitManifestResponse) ProtoReflect() protoreflect.Message { + mi := &file_org_apache_beam_model_job_management_v1_beam_artifact_api_proto_msgTypes[17] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +func (x *CommitManifestResponse) GetRetrievalToken() string { + if x != nil { + return x.xxx_hidden_RetrievalToken + } + return "" +} + +func (x *CommitManifestResponse) SetRetrievalToken(v string) { + x.xxx_hidden_RetrievalToken = v +} + +type CommitManifestResponse_builder struct { + _ [0]func() // Prevents comparability and use of unkeyed literals for the builder. + + // (Required) An opaque token representing the entirety of the staged artifacts. + // This can be used to retrieve the manifest and artifacts from an associated + // LegacyArtifactRetrievalService. + RetrievalToken string +} + +func (b0 CommitManifestResponse_builder) Build() *CommitManifestResponse { + m0 := &CommitManifestResponse{} + b, x := &b0, m0 + _, _ = b, x + x.xxx_hidden_RetrievalToken = b.RetrievalToken + return m0 +} + +type ProxyManifest_Location struct { + state protoimpl.MessageState `protogen:"opaque.v1"` + xxx_hidden_Name string `protobuf:"bytes,1,opt,name=name,proto3" json:"name,omitempty"` + xxx_hidden_Uri string `protobuf:"bytes,2,opt,name=uri,proto3" json:"uri,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *ProxyManifest_Location) Reset() { + *x = ProxyManifest_Location{} + mi := &file_org_apache_beam_model_job_management_v1_beam_artifact_api_proto_msgTypes[18] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *ProxyManifest_Location) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*ProxyManifest_Location) ProtoMessage() {} + +func (x *ProxyManifest_Location) ProtoReflect() protoreflect.Message { + mi := &file_org_apache_beam_model_job_management_v1_beam_artifact_api_proto_msgTypes[18] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +func (x *ProxyManifest_Location) GetName() string { + if x != nil { + return x.xxx_hidden_Name + } + return "" +} + +func (x *ProxyManifest_Location) GetUri() string { + if x != nil { + return x.xxx_hidden_Uri + } + return "" +} + +func (x *ProxyManifest_Location) SetName(v string) { + x.xxx_hidden_Name = v +} + +func (x *ProxyManifest_Location) SetUri(v string) { + x.xxx_hidden_Uri = v +} + +type ProxyManifest_Location_builder struct { + _ [0]func() // Prevents comparability and use of unkeyed literals for the builder. + + Name string + Uri string +} + +func (b0 ProxyManifest_Location_builder) Build() *ProxyManifest_Location { + m0 := &ProxyManifest_Location{} + b, x := &b0, m0 + _, _ = b, x + x.xxx_hidden_Name = b.Name + x.xxx_hidden_Uri = b.Uri + return m0 +} + +var File_org_apache_beam_model_job_management_v1_beam_artifact_api_proto protoreflect.FileDescriptor + +var file_org_apache_beam_model_job_management_v1_beam_artifact_api_proto_rawDesc = []byte{ + 0x0a, 0x3f, 0x6f, 0x72, 0x67, 0x2f, 0x61, 0x70, 0x61, 0x63, 0x68, 0x65, 0x2f, 0x62, 0x65, 0x61, + 0x6d, 0x2f, 0x6d, 0x6f, 0x64, 0x65, 0x6c, 0x2f, 0x6a, 0x6f, 0x62, 0x5f, 0x6d, 0x61, 0x6e, 0x61, + 0x67, 0x65, 0x6d, 0x65, 0x6e, 0x74, 0x2f, 0x76, 0x31, 0x2f, 0x62, 0x65, 0x61, 0x6d, 0x5f, 0x61, + 0x72, 0x74, 0x69, 0x66, 0x61, 0x63, 0x74, 0x5f, 0x61, 0x70, 0x69, 0x2e, 0x70, 0x72, 0x6f, 0x74, + 0x6f, 0x12, 0x27, 0x6f, 0x72, 0x67, 0x2e, 0x61, 0x70, 0x61, 0x63, 0x68, 0x65, 0x2e, 0x62, 0x65, + 0x61, 0x6d, 0x2e, 0x6d, 0x6f, 0x64, 0x65, 0x6c, 0x2e, 0x6a, 0x6f, 0x62, 0x5f, 0x6d, 0x61, 0x6e, + 0x61, 0x67, 0x65, 0x6d, 0x65, 0x6e, 0x74, 0x2e, 0x76, 0x31, 0x1a, 0x37, 0x6f, 0x72, 0x67, 0x2f, + 0x61, 0x70, 0x61, 0x63, 0x68, 0x65, 0x2f, 0x62, 0x65, 0x61, 0x6d, 0x2f, 0x6d, 0x6f, 0x64, 0x65, + 0x6c, 0x2f, 0x70, 0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e, 0x65, 0x2f, 0x76, 0x31, 0x2f, 0x62, 0x65, + 0x61, 0x6d, 0x5f, 0x72, 0x75, 0x6e, 0x6e, 0x65, 0x72, 0x5f, 0x61, 0x70, 0x69, 0x2e, 0x70, 0x72, + 0x6f, 0x74, 0x6f, 0x22, 0x96, 0x01, 0x0a, 0x17, 0x52, 0x65, 0x73, 0x6f, 0x6c, 0x76, 0x65, 0x41, + 0x72, 0x74, 0x69, 0x66, 0x61, 0x63, 0x74, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, + 0x54, 0x0a, 0x09, 0x61, 0x72, 0x74, 0x69, 0x66, 0x61, 0x63, 0x74, 0x73, 0x18, 0x01, 0x20, 0x03, + 0x28, 0x0b, 0x32, 0x36, 0x2e, 0x6f, 0x72, 0x67, 0x2e, 0x61, 0x70, 0x61, 0x63, 0x68, 0x65, 0x2e, + 0x62, 0x65, 0x61, 0x6d, 0x2e, 0x6d, 0x6f, 0x64, 0x65, 0x6c, 0x2e, 0x70, 0x69, 0x70, 0x65, 0x6c, + 0x69, 0x6e, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x41, 0x72, 0x74, 0x69, 0x66, 0x61, 0x63, 0x74, 0x49, + 0x6e, 0x66, 0x6f, 0x72, 0x6d, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x09, 0x61, 0x72, 0x74, 0x69, + 0x66, 0x61, 0x63, 0x74, 0x73, 0x12, 0x25, 0x0a, 0x0e, 0x70, 0x72, 0x65, 0x66, 0x65, 0x72, 0x72, + 0x65, 0x64, 0x5f, 0x75, 0x72, 0x6e, 0x73, 0x18, 0x02, 0x20, 0x03, 0x28, 0x09, 0x52, 0x0d, 0x70, + 0x72, 0x65, 0x66, 0x65, 0x72, 0x72, 0x65, 0x64, 0x55, 0x72, 0x6e, 0x73, 0x22, 0x76, 0x0a, 0x18, + 0x52, 0x65, 0x73, 0x6f, 0x6c, 0x76, 0x65, 0x41, 0x72, 0x74, 0x69, 0x66, 0x61, 0x63, 0x74, 0x73, + 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x5a, 0x0a, 0x0c, 0x72, 0x65, 0x70, 0x6c, + 0x61, 0x63, 0x65, 0x6d, 0x65, 0x6e, 0x74, 0x73, 0x18, 0x01, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x36, + 0x2e, 0x6f, 0x72, 0x67, 0x2e, 0x61, 0x70, 0x61, 0x63, 0x68, 0x65, 0x2e, 0x62, 0x65, 0x61, 0x6d, + 0x2e, 0x6d, 0x6f, 0x64, 0x65, 0x6c, 0x2e, 0x70, 0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e, 0x65, 0x2e, + 0x76, 0x31, 0x2e, 0x41, 0x72, 0x74, 0x69, 0x66, 0x61, 0x63, 0x74, 0x49, 0x6e, 0x66, 0x6f, 0x72, + 0x6d, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x0c, 0x72, 0x65, 0x70, 0x6c, 0x61, 0x63, 0x65, 0x6d, + 0x65, 0x6e, 0x74, 0x73, 0x22, 0x68, 0x0a, 0x12, 0x47, 0x65, 0x74, 0x41, 0x72, 0x74, 0x69, 0x66, + 0x61, 0x63, 0x74, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x52, 0x0a, 0x08, 0x61, 0x72, + 0x74, 0x69, 0x66, 0x61, 0x63, 0x74, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x36, 0x2e, 0x6f, + 0x72, 0x67, 0x2e, 0x61, 0x70, 0x61, 0x63, 0x68, 0x65, 0x2e, 0x62, 0x65, 0x61, 0x6d, 0x2e, 0x6d, + 0x6f, 0x64, 0x65, 0x6c, 0x2e, 0x70, 0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e, 0x65, 0x2e, 0x76, 0x31, + 0x2e, 0x41, 0x72, 0x74, 0x69, 0x66, 0x61, 0x63, 0x74, 0x49, 0x6e, 0x66, 0x6f, 0x72, 0x6d, 0x61, + 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x08, 0x61, 0x72, 0x74, 0x69, 0x66, 0x61, 0x63, 0x74, 0x22, 0x29, + 0x0a, 0x13, 0x47, 0x65, 0x74, 0x41, 0x72, 0x74, 0x69, 0x66, 0x61, 0x63, 0x74, 0x52, 0x65, 0x73, + 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x12, 0x0a, 0x04, 0x64, 0x61, 0x74, 0x61, 0x18, 0x01, 0x20, + 0x01, 0x28, 0x0c, 0x52, 0x04, 0x64, 0x61, 0x74, 0x61, 0x22, 0xf6, 0x01, 0x0a, 0x16, 0x41, 0x72, + 0x74, 0x69, 0x66, 0x61, 0x63, 0x74, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x57, 0x72, 0x61, + 0x70, 0x70, 0x65, 0x72, 0x12, 0x6e, 0x0a, 0x10, 0x72, 0x65, 0x73, 0x6f, 0x6c, 0x76, 0x65, 0x5f, + 0x61, 0x72, 0x74, 0x69, 0x66, 0x61, 0x63, 0x74, 0x18, 0xe8, 0x07, 0x20, 0x01, 0x28, 0x0b, 0x32, + 0x40, 0x2e, 0x6f, 0x72, 0x67, 0x2e, 0x61, 0x70, 0x61, 0x63, 0x68, 0x65, 0x2e, 0x62, 0x65, 0x61, + 0x6d, 0x2e, 0x6d, 0x6f, 0x64, 0x65, 0x6c, 0x2e, 0x6a, 0x6f, 0x62, 0x5f, 0x6d, 0x61, 0x6e, 0x61, + 0x67, 0x65, 0x6d, 0x65, 0x6e, 0x74, 0x2e, 0x76, 0x31, 0x2e, 0x52, 0x65, 0x73, 0x6f, 0x6c, 0x76, + 0x65, 0x41, 0x72, 0x74, 0x69, 0x66, 0x61, 0x63, 0x74, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, + 0x74, 0x48, 0x00, 0x52, 0x0f, 0x72, 0x65, 0x73, 0x6f, 0x6c, 0x76, 0x65, 0x41, 0x72, 0x74, 0x69, + 0x66, 0x61, 0x63, 0x74, 0x12, 0x61, 0x0a, 0x0c, 0x67, 0x65, 0x74, 0x5f, 0x61, 0x72, 0x74, 0x69, + 0x66, 0x61, 0x63, 0x74, 0x18, 0xe9, 0x07, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x3b, 0x2e, 0x6f, 0x72, + 0x67, 0x2e, 0x61, 0x70, 0x61, 0x63, 0x68, 0x65, 0x2e, 0x62, 0x65, 0x61, 0x6d, 0x2e, 0x6d, 0x6f, + 0x64, 0x65, 0x6c, 0x2e, 0x6a, 0x6f, 0x62, 0x5f, 0x6d, 0x61, 0x6e, 0x61, 0x67, 0x65, 0x6d, 0x65, + 0x6e, 0x74, 0x2e, 0x76, 0x31, 0x2e, 0x47, 0x65, 0x74, 0x41, 0x72, 0x74, 0x69, 0x66, 0x61, 0x63, + 0x74, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x48, 0x00, 0x52, 0x0b, 0x67, 0x65, 0x74, 0x41, + 0x72, 0x74, 0x69, 0x66, 0x61, 0x63, 0x74, 0x42, 0x09, 0x0a, 0x07, 0x72, 0x65, 0x71, 0x75, 0x65, + 0x73, 0x74, 0x22, 0xdb, 0x02, 0x0a, 0x17, 0x41, 0x72, 0x74, 0x69, 0x66, 0x61, 0x63, 0x74, 0x52, + 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x57, 0x72, 0x61, 0x70, 0x70, 0x65, 0x72, 0x12, 0x23, + 0x0a, 0x0d, 0x73, 0x74, 0x61, 0x67, 0x69, 0x6e, 0x67, 0x5f, 0x74, 0x6f, 0x6b, 0x65, 0x6e, 0x18, + 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0c, 0x73, 0x74, 0x61, 0x67, 0x69, 0x6e, 0x67, 0x54, 0x6f, + 0x6b, 0x65, 0x6e, 0x12, 0x17, 0x0a, 0x07, 0x69, 0x73, 0x5f, 0x6c, 0x61, 0x73, 0x74, 0x18, 0x02, + 0x20, 0x01, 0x28, 0x08, 0x52, 0x06, 0x69, 0x73, 0x4c, 0x61, 0x73, 0x74, 0x12, 0x80, 0x01, 0x0a, + 0x19, 0x72, 0x65, 0x73, 0x6f, 0x6c, 0x76, 0x65, 0x5f, 0x61, 0x72, 0x74, 0x69, 0x66, 0x61, 0x63, + 0x74, 0x5f, 0x72, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x18, 0xe8, 0x07, 0x20, 0x01, 0x28, + 0x0b, 0x32, 0x41, 0x2e, 0x6f, 0x72, 0x67, 0x2e, 0x61, 0x70, 0x61, 0x63, 0x68, 0x65, 0x2e, 0x62, + 0x65, 0x61, 0x6d, 0x2e, 0x6d, 0x6f, 0x64, 0x65, 0x6c, 0x2e, 0x6a, 0x6f, 0x62, 0x5f, 0x6d, 0x61, + 0x6e, 0x61, 0x67, 0x65, 0x6d, 0x65, 0x6e, 0x74, 0x2e, 0x76, 0x31, 0x2e, 0x52, 0x65, 0x73, 0x6f, + 0x6c, 0x76, 0x65, 0x41, 0x72, 0x74, 0x69, 0x66, 0x61, 0x63, 0x74, 0x73, 0x52, 0x65, 0x73, 0x70, + 0x6f, 0x6e, 0x73, 0x65, 0x48, 0x00, 0x52, 0x17, 0x72, 0x65, 0x73, 0x6f, 0x6c, 0x76, 0x65, 0x41, + 0x72, 0x74, 0x69, 0x66, 0x61, 0x63, 0x74, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, + 0x73, 0x0a, 0x15, 0x67, 0x65, 0x74, 0x5f, 0x61, 0x72, 0x74, 0x69, 0x66, 0x61, 0x63, 0x74, 0x5f, + 0x72, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x18, 0xe9, 0x07, 0x20, 0x01, 0x28, 0x0b, 0x32, + 0x3c, 0x2e, 0x6f, 0x72, 0x67, 0x2e, 0x61, 0x70, 0x61, 0x63, 0x68, 0x65, 0x2e, 0x62, 0x65, 0x61, + 0x6d, 0x2e, 0x6d, 0x6f, 0x64, 0x65, 0x6c, 0x2e, 0x6a, 0x6f, 0x62, 0x5f, 0x6d, 0x61, 0x6e, 0x61, + 0x67, 0x65, 0x6d, 0x65, 0x6e, 0x74, 0x2e, 0x76, 0x31, 0x2e, 0x47, 0x65, 0x74, 0x41, 0x72, 0x74, + 0x69, 0x66, 0x61, 0x63, 0x74, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x48, 0x00, 0x52, + 0x13, 0x67, 0x65, 0x74, 0x41, 0x72, 0x74, 0x69, 0x66, 0x61, 0x63, 0x74, 0x52, 0x65, 0x73, 0x70, + 0x6f, 0x6e, 0x73, 0x65, 0x42, 0x0a, 0x0a, 0x08, 0x72, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, + 0x22, 0x60, 0x0a, 0x10, 0x41, 0x72, 0x74, 0x69, 0x66, 0x61, 0x63, 0x74, 0x4d, 0x65, 0x74, 0x61, + 0x64, 0x61, 0x74, 0x61, 0x12, 0x12, 0x0a, 0x04, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x01, 0x20, 0x01, + 0x28, 0x09, 0x52, 0x04, 0x6e, 0x61, 0x6d, 0x65, 0x12, 0x20, 0x0a, 0x0b, 0x70, 0x65, 0x72, 0x6d, + 0x69, 0x73, 0x73, 0x69, 0x6f, 0x6e, 0x73, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0d, 0x52, 0x0b, 0x70, + 0x65, 0x72, 0x6d, 0x69, 0x73, 0x73, 0x69, 0x6f, 0x6e, 0x73, 0x12, 0x16, 0x0a, 0x06, 0x73, 0x68, + 0x61, 0x32, 0x35, 0x36, 0x18, 0x04, 0x20, 0x01, 0x28, 0x09, 0x52, 0x06, 0x73, 0x68, 0x61, 0x32, + 0x35, 0x36, 0x22, 0x61, 0x0a, 0x08, 0x4d, 0x61, 0x6e, 0x69, 0x66, 0x65, 0x73, 0x74, 0x12, 0x55, + 0x0a, 0x08, 0x61, 0x72, 0x74, 0x69, 0x66, 0x61, 0x63, 0x74, 0x18, 0x01, 0x20, 0x03, 0x28, 0x0b, + 0x32, 0x39, 0x2e, 0x6f, 0x72, 0x67, 0x2e, 0x61, 0x70, 0x61, 0x63, 0x68, 0x65, 0x2e, 0x62, 0x65, + 0x61, 0x6d, 0x2e, 0x6d, 0x6f, 0x64, 0x65, 0x6c, 0x2e, 0x6a, 0x6f, 0x62, 0x5f, 0x6d, 0x61, 0x6e, + 0x61, 0x67, 0x65, 0x6d, 0x65, 0x6e, 0x74, 0x2e, 0x76, 0x31, 0x2e, 0x41, 0x72, 0x74, 0x69, 0x66, + 0x61, 0x63, 0x74, 0x4d, 0x65, 0x74, 0x61, 0x64, 0x61, 0x74, 0x61, 0x52, 0x08, 0x61, 0x72, 0x74, + 0x69, 0x66, 0x61, 0x63, 0x74, 0x22, 0xed, 0x01, 0x0a, 0x0d, 0x50, 0x72, 0x6f, 0x78, 0x79, 0x4d, + 0x61, 0x6e, 0x69, 0x66, 0x65, 0x73, 0x74, 0x12, 0x4d, 0x0a, 0x08, 0x6d, 0x61, 0x6e, 0x69, 0x66, + 0x65, 0x73, 0x74, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x31, 0x2e, 0x6f, 0x72, 0x67, 0x2e, + 0x61, 0x70, 0x61, 0x63, 0x68, 0x65, 0x2e, 0x62, 0x65, 0x61, 0x6d, 0x2e, 0x6d, 0x6f, 0x64, 0x65, + 0x6c, 0x2e, 0x6a, 0x6f, 0x62, 0x5f, 0x6d, 0x61, 0x6e, 0x61, 0x67, 0x65, 0x6d, 0x65, 0x6e, 0x74, + 0x2e, 0x76, 0x31, 0x2e, 0x4d, 0x61, 0x6e, 0x69, 0x66, 0x65, 0x73, 0x74, 0x52, 0x08, 0x6d, 0x61, + 0x6e, 0x69, 0x66, 0x65, 0x73, 0x74, 0x12, 0x5b, 0x0a, 0x08, 0x6c, 0x6f, 0x63, 0x61, 0x74, 0x69, + 0x6f, 0x6e, 0x18, 0x02, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x3f, 0x2e, 0x6f, 0x72, 0x67, 0x2e, 0x61, + 0x70, 0x61, 0x63, 0x68, 0x65, 0x2e, 0x62, 0x65, 0x61, 0x6d, 0x2e, 0x6d, 0x6f, 0x64, 0x65, 0x6c, + 0x2e, 0x6a, 0x6f, 0x62, 0x5f, 0x6d, 0x61, 0x6e, 0x61, 0x67, 0x65, 0x6d, 0x65, 0x6e, 0x74, 0x2e, + 0x76, 0x31, 0x2e, 0x50, 0x72, 0x6f, 0x78, 0x79, 0x4d, 0x61, 0x6e, 0x69, 0x66, 0x65, 0x73, 0x74, + 0x2e, 0x4c, 0x6f, 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x08, 0x6c, 0x6f, 0x63, 0x61, 0x74, + 0x69, 0x6f, 0x6e, 0x1a, 0x30, 0x0a, 0x08, 0x4c, 0x6f, 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x12, + 0x12, 0x0a, 0x04, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x04, 0x6e, + 0x61, 0x6d, 0x65, 0x12, 0x10, 0x0a, 0x03, 0x75, 0x72, 0x69, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, + 0x52, 0x03, 0x75, 0x72, 0x69, 0x22, 0x3d, 0x0a, 0x12, 0x47, 0x65, 0x74, 0x4d, 0x61, 0x6e, 0x69, + 0x66, 0x65, 0x73, 0x74, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x27, 0x0a, 0x0f, 0x72, + 0x65, 0x74, 0x72, 0x69, 0x65, 0x76, 0x61, 0x6c, 0x5f, 0x74, 0x6f, 0x6b, 0x65, 0x6e, 0x18, 0x01, + 0x20, 0x01, 0x28, 0x09, 0x52, 0x0e, 0x72, 0x65, 0x74, 0x72, 0x69, 0x65, 0x76, 0x61, 0x6c, 0x54, + 0x6f, 0x6b, 0x65, 0x6e, 0x22, 0x64, 0x0a, 0x13, 0x47, 0x65, 0x74, 0x4d, 0x61, 0x6e, 0x69, 0x66, + 0x65, 0x73, 0x74, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x4d, 0x0a, 0x08, 0x6d, + 0x61, 0x6e, 0x69, 0x66, 0x65, 0x73, 0x74, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x31, 0x2e, + 0x6f, 0x72, 0x67, 0x2e, 0x61, 0x70, 0x61, 0x63, 0x68, 0x65, 0x2e, 0x62, 0x65, 0x61, 0x6d, 0x2e, + 0x6d, 0x6f, 0x64, 0x65, 0x6c, 0x2e, 0x6a, 0x6f, 0x62, 0x5f, 0x6d, 0x61, 0x6e, 0x61, 0x67, 0x65, + 0x6d, 0x65, 0x6e, 0x74, 0x2e, 0x76, 0x31, 0x2e, 0x4d, 0x61, 0x6e, 0x69, 0x66, 0x65, 0x73, 0x74, + 0x52, 0x08, 0x6d, 0x61, 0x6e, 0x69, 0x66, 0x65, 0x73, 0x74, 0x22, 0x57, 0x0a, 0x18, 0x4c, 0x65, + 0x67, 0x61, 0x63, 0x79, 0x47, 0x65, 0x74, 0x41, 0x72, 0x74, 0x69, 0x66, 0x61, 0x63, 0x74, 0x52, + 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x12, 0x0a, 0x04, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x01, + 0x20, 0x01, 0x28, 0x09, 0x52, 0x04, 0x6e, 0x61, 0x6d, 0x65, 0x12, 0x27, 0x0a, 0x0f, 0x72, 0x65, + 0x74, 0x72, 0x69, 0x65, 0x76, 0x61, 0x6c, 0x5f, 0x74, 0x6f, 0x6b, 0x65, 0x6e, 0x18, 0x02, 0x20, + 0x01, 0x28, 0x09, 0x52, 0x0e, 0x72, 0x65, 0x74, 0x72, 0x69, 0x65, 0x76, 0x61, 0x6c, 0x54, 0x6f, + 0x6b, 0x65, 0x6e, 0x22, 0x23, 0x0a, 0x0d, 0x41, 0x72, 0x74, 0x69, 0x66, 0x61, 0x63, 0x74, 0x43, + 0x68, 0x75, 0x6e, 0x6b, 0x12, 0x12, 0x0a, 0x04, 0x64, 0x61, 0x74, 0x61, 0x18, 0x01, 0x20, 0x01, + 0x28, 0x0c, 0x52, 0x04, 0x64, 0x61, 0x74, 0x61, 0x22, 0xa0, 0x01, 0x0a, 0x13, 0x50, 0x75, 0x74, + 0x41, 0x72, 0x74, 0x69, 0x66, 0x61, 0x63, 0x74, 0x4d, 0x65, 0x74, 0x61, 0x64, 0x61, 0x74, 0x61, + 0x12, 0x32, 0x0a, 0x15, 0x73, 0x74, 0x61, 0x67, 0x69, 0x6e, 0x67, 0x5f, 0x73, 0x65, 0x73, 0x73, + 0x69, 0x6f, 0x6e, 0x5f, 0x74, 0x6f, 0x6b, 0x65, 0x6e, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, + 0x13, 0x73, 0x74, 0x61, 0x67, 0x69, 0x6e, 0x67, 0x53, 0x65, 0x73, 0x73, 0x69, 0x6f, 0x6e, 0x54, + 0x6f, 0x6b, 0x65, 0x6e, 0x12, 0x55, 0x0a, 0x08, 0x6d, 0x65, 0x74, 0x61, 0x64, 0x61, 0x74, 0x61, + 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x39, 0x2e, 0x6f, 0x72, 0x67, 0x2e, 0x61, 0x70, 0x61, + 0x63, 0x68, 0x65, 0x2e, 0x62, 0x65, 0x61, 0x6d, 0x2e, 0x6d, 0x6f, 0x64, 0x65, 0x6c, 0x2e, 0x6a, + 0x6f, 0x62, 0x5f, 0x6d, 0x61, 0x6e, 0x61, 0x67, 0x65, 0x6d, 0x65, 0x6e, 0x74, 0x2e, 0x76, 0x31, + 0x2e, 0x41, 0x72, 0x74, 0x69, 0x66, 0x61, 0x63, 0x74, 0x4d, 0x65, 0x74, 0x61, 0x64, 0x61, 0x74, + 0x61, 0x52, 0x08, 0x6d, 0x65, 0x74, 0x61, 0x64, 0x61, 0x74, 0x61, 0x22, 0xc9, 0x01, 0x0a, 0x12, + 0x50, 0x75, 0x74, 0x41, 0x72, 0x74, 0x69, 0x66, 0x61, 0x63, 0x74, 0x52, 0x65, 0x71, 0x75, 0x65, + 0x73, 0x74, 0x12, 0x5a, 0x0a, 0x08, 0x6d, 0x65, 0x74, 0x61, 0x64, 0x61, 0x74, 0x61, 0x18, 0x01, + 0x20, 0x01, 0x28, 0x0b, 0x32, 0x3c, 0x2e, 0x6f, 0x72, 0x67, 0x2e, 0x61, 0x70, 0x61, 0x63, 0x68, + 0x65, 0x2e, 0x62, 0x65, 0x61, 0x6d, 0x2e, 0x6d, 0x6f, 0x64, 0x65, 0x6c, 0x2e, 0x6a, 0x6f, 0x62, + 0x5f, 0x6d, 0x61, 0x6e, 0x61, 0x67, 0x65, 0x6d, 0x65, 0x6e, 0x74, 0x2e, 0x76, 0x31, 0x2e, 0x50, + 0x75, 0x74, 0x41, 0x72, 0x74, 0x69, 0x66, 0x61, 0x63, 0x74, 0x4d, 0x65, 0x74, 0x61, 0x64, 0x61, + 0x74, 0x61, 0x48, 0x00, 0x52, 0x08, 0x6d, 0x65, 0x74, 0x61, 0x64, 0x61, 0x74, 0x61, 0x12, 0x4c, + 0x0a, 0x04, 0x64, 0x61, 0x74, 0x61, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x36, 0x2e, 0x6f, + 0x72, 0x67, 0x2e, 0x61, 0x70, 0x61, 0x63, 0x68, 0x65, 0x2e, 0x62, 0x65, 0x61, 0x6d, 0x2e, 0x6d, + 0x6f, 0x64, 0x65, 0x6c, 0x2e, 0x6a, 0x6f, 0x62, 0x5f, 0x6d, 0x61, 0x6e, 0x61, 0x67, 0x65, 0x6d, + 0x65, 0x6e, 0x74, 0x2e, 0x76, 0x31, 0x2e, 0x41, 0x72, 0x74, 0x69, 0x66, 0x61, 0x63, 0x74, 0x43, + 0x68, 0x75, 0x6e, 0x6b, 0x48, 0x00, 0x52, 0x04, 0x64, 0x61, 0x74, 0x61, 0x42, 0x09, 0x0a, 0x07, + 0x63, 0x6f, 0x6e, 0x74, 0x65, 0x6e, 0x74, 0x22, 0x15, 0x0a, 0x13, 0x50, 0x75, 0x74, 0x41, 0x72, + 0x74, 0x69, 0x66, 0x61, 0x63, 0x74, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x9a, + 0x01, 0x0a, 0x15, 0x43, 0x6f, 0x6d, 0x6d, 0x69, 0x74, 0x4d, 0x61, 0x6e, 0x69, 0x66, 0x65, 0x73, + 0x74, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x4d, 0x0a, 0x08, 0x6d, 0x61, 0x6e, 0x69, + 0x66, 0x65, 0x73, 0x74, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x31, 0x2e, 0x6f, 0x72, 0x67, + 0x2e, 0x61, 0x70, 0x61, 0x63, 0x68, 0x65, 0x2e, 0x62, 0x65, 0x61, 0x6d, 0x2e, 0x6d, 0x6f, 0x64, + 0x65, 0x6c, 0x2e, 0x6a, 0x6f, 0x62, 0x5f, 0x6d, 0x61, 0x6e, 0x61, 0x67, 0x65, 0x6d, 0x65, 0x6e, + 0x74, 0x2e, 0x76, 0x31, 0x2e, 0x4d, 0x61, 0x6e, 0x69, 0x66, 0x65, 0x73, 0x74, 0x52, 0x08, 0x6d, + 0x61, 0x6e, 0x69, 0x66, 0x65, 0x73, 0x74, 0x12, 0x32, 0x0a, 0x15, 0x73, 0x74, 0x61, 0x67, 0x69, + 0x6e, 0x67, 0x5f, 0x73, 0x65, 0x73, 0x73, 0x69, 0x6f, 0x6e, 0x5f, 0x74, 0x6f, 0x6b, 0x65, 0x6e, + 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x13, 0x73, 0x74, 0x61, 0x67, 0x69, 0x6e, 0x67, 0x53, + 0x65, 0x73, 0x73, 0x69, 0x6f, 0x6e, 0x54, 0x6f, 0x6b, 0x65, 0x6e, 0x22, 0x8c, 0x01, 0x0a, 0x16, + 0x43, 0x6f, 0x6d, 0x6d, 0x69, 0x74, 0x4d, 0x61, 0x6e, 0x69, 0x66, 0x65, 0x73, 0x74, 0x52, 0x65, + 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x27, 0x0a, 0x0f, 0x72, 0x65, 0x74, 0x72, 0x69, 0x65, + 0x76, 0x61, 0x6c, 0x5f, 0x74, 0x6f, 0x6b, 0x65, 0x6e, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, + 0x0e, 0x72, 0x65, 0x74, 0x72, 0x69, 0x65, 0x76, 0x61, 0x6c, 0x54, 0x6f, 0x6b, 0x65, 0x6e, 0x22, + 0x49, 0x0a, 0x09, 0x43, 0x6f, 0x6e, 0x73, 0x74, 0x61, 0x6e, 0x74, 0x73, 0x12, 0x3c, 0x0a, 0x19, + 0x4e, 0x4f, 0x5f, 0x41, 0x52, 0x54, 0x49, 0x46, 0x41, 0x43, 0x54, 0x53, 0x5f, 0x53, 0x54, 0x41, + 0x47, 0x45, 0x44, 0x5f, 0x54, 0x4f, 0x4b, 0x45, 0x4e, 0x10, 0x00, 0x1a, 0x1d, 0xaa, 0xb4, 0xfa, + 0xc2, 0x05, 0x17, 0x5f, 0x5f, 0x6e, 0x6f, 0x5f, 0x61, 0x72, 0x74, 0x69, 0x66, 0x61, 0x63, 0x74, + 0x73, 0x5f, 0x73, 0x74, 0x61, 0x67, 0x65, 0x64, 0x5f, 0x5f, 0x32, 0xc1, 0x02, 0x0a, 0x18, 0x41, + 0x72, 0x74, 0x69, 0x66, 0x61, 0x63, 0x74, 0x52, 0x65, 0x74, 0x72, 0x69, 0x65, 0x76, 0x61, 0x6c, + 0x53, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, 0x12, 0x97, 0x01, 0x0a, 0x10, 0x52, 0x65, 0x73, 0x6f, + 0x6c, 0x76, 0x65, 0x41, 0x72, 0x74, 0x69, 0x66, 0x61, 0x63, 0x74, 0x73, 0x12, 0x40, 0x2e, 0x6f, + 0x72, 0x67, 0x2e, 0x61, 0x70, 0x61, 0x63, 0x68, 0x65, 0x2e, 0x62, 0x65, 0x61, 0x6d, 0x2e, 0x6d, + 0x6f, 0x64, 0x65, 0x6c, 0x2e, 0x6a, 0x6f, 0x62, 0x5f, 0x6d, 0x61, 0x6e, 0x61, 0x67, 0x65, 0x6d, + 0x65, 0x6e, 0x74, 0x2e, 0x76, 0x31, 0x2e, 0x52, 0x65, 0x73, 0x6f, 0x6c, 0x76, 0x65, 0x41, 0x72, + 0x74, 0x69, 0x66, 0x61, 0x63, 0x74, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x41, + 0x2e, 0x6f, 0x72, 0x67, 0x2e, 0x61, 0x70, 0x61, 0x63, 0x68, 0x65, 0x2e, 0x62, 0x65, 0x61, 0x6d, + 0x2e, 0x6d, 0x6f, 0x64, 0x65, 0x6c, 0x2e, 0x6a, 0x6f, 0x62, 0x5f, 0x6d, 0x61, 0x6e, 0x61, 0x67, + 0x65, 0x6d, 0x65, 0x6e, 0x74, 0x2e, 0x76, 0x31, 0x2e, 0x52, 0x65, 0x73, 0x6f, 0x6c, 0x76, 0x65, + 0x41, 0x72, 0x74, 0x69, 0x66, 0x61, 0x63, 0x74, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, + 0x65, 0x12, 0x8a, 0x01, 0x0a, 0x0b, 0x47, 0x65, 0x74, 0x41, 0x72, 0x74, 0x69, 0x66, 0x61, 0x63, + 0x74, 0x12, 0x3b, 0x2e, 0x6f, 0x72, 0x67, 0x2e, 0x61, 0x70, 0x61, 0x63, 0x68, 0x65, 0x2e, 0x62, + 0x65, 0x61, 0x6d, 0x2e, 0x6d, 0x6f, 0x64, 0x65, 0x6c, 0x2e, 0x6a, 0x6f, 0x62, 0x5f, 0x6d, 0x61, + 0x6e, 0x61, 0x67, 0x65, 0x6d, 0x65, 0x6e, 0x74, 0x2e, 0x76, 0x31, 0x2e, 0x47, 0x65, 0x74, 0x41, + 0x72, 0x74, 0x69, 0x66, 0x61, 0x63, 0x74, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x3c, + 0x2e, 0x6f, 0x72, 0x67, 0x2e, 0x61, 0x70, 0x61, 0x63, 0x68, 0x65, 0x2e, 0x62, 0x65, 0x61, 0x6d, + 0x2e, 0x6d, 0x6f, 0x64, 0x65, 0x6c, 0x2e, 0x6a, 0x6f, 0x62, 0x5f, 0x6d, 0x61, 0x6e, 0x61, 0x67, + 0x65, 0x6d, 0x65, 0x6e, 0x74, 0x2e, 0x76, 0x31, 0x2e, 0x47, 0x65, 0x74, 0x41, 0x72, 0x74, 0x69, + 0x66, 0x61, 0x63, 0x74, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x30, 0x01, 0x32, 0xc3, + 0x01, 0x0a, 0x16, 0x41, 0x72, 0x74, 0x69, 0x66, 0x61, 0x63, 0x74, 0x53, 0x74, 0x61, 0x67, 0x69, + 0x6e, 0x67, 0x53, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, 0x12, 0xa8, 0x01, 0x0a, 0x1f, 0x52, 0x65, + 0x76, 0x65, 0x72, 0x73, 0x65, 0x41, 0x72, 0x74, 0x69, 0x66, 0x61, 0x63, 0x74, 0x52, 0x65, 0x74, + 0x72, 0x69, 0x65, 0x76, 0x61, 0x6c, 0x53, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, 0x12, 0x40, 0x2e, + 0x6f, 0x72, 0x67, 0x2e, 0x61, 0x70, 0x61, 0x63, 0x68, 0x65, 0x2e, 0x62, 0x65, 0x61, 0x6d, 0x2e, + 0x6d, 0x6f, 0x64, 0x65, 0x6c, 0x2e, 0x6a, 0x6f, 0x62, 0x5f, 0x6d, 0x61, 0x6e, 0x61, 0x67, 0x65, + 0x6d, 0x65, 0x6e, 0x74, 0x2e, 0x76, 0x31, 0x2e, 0x41, 0x72, 0x74, 0x69, 0x66, 0x61, 0x63, 0x74, + 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x57, 0x72, 0x61, 0x70, 0x70, 0x65, 0x72, 0x1a, + 0x3f, 0x2e, 0x6f, 0x72, 0x67, 0x2e, 0x61, 0x70, 0x61, 0x63, 0x68, 0x65, 0x2e, 0x62, 0x65, 0x61, + 0x6d, 0x2e, 0x6d, 0x6f, 0x64, 0x65, 0x6c, 0x2e, 0x6a, 0x6f, 0x62, 0x5f, 0x6d, 0x61, 0x6e, 0x61, + 0x67, 0x65, 0x6d, 0x65, 0x6e, 0x74, 0x2e, 0x76, 0x31, 0x2e, 0x41, 0x72, 0x74, 0x69, 0x66, 0x61, + 0x63, 0x74, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x57, 0x72, 0x61, 0x70, 0x70, 0x65, 0x72, + 0x28, 0x01, 0x30, 0x01, 0x32, 0xbf, 0x02, 0x0a, 0x1c, 0x4c, 0x65, 0x67, 0x61, 0x63, 0x79, 0x41, + 0x72, 0x74, 0x69, 0x66, 0x61, 0x63, 0x74, 0x53, 0x74, 0x61, 0x67, 0x69, 0x6e, 0x67, 0x53, 0x65, + 0x72, 0x76, 0x69, 0x63, 0x65, 0x12, 0x8a, 0x01, 0x0a, 0x0b, 0x50, 0x75, 0x74, 0x41, 0x72, 0x74, + 0x69, 0x66, 0x61, 0x63, 0x74, 0x12, 0x3b, 0x2e, 0x6f, 0x72, 0x67, 0x2e, 0x61, 0x70, 0x61, 0x63, + 0x68, 0x65, 0x2e, 0x62, 0x65, 0x61, 0x6d, 0x2e, 0x6d, 0x6f, 0x64, 0x65, 0x6c, 0x2e, 0x6a, 0x6f, + 0x62, 0x5f, 0x6d, 0x61, 0x6e, 0x61, 0x67, 0x65, 0x6d, 0x65, 0x6e, 0x74, 0x2e, 0x76, 0x31, 0x2e, + 0x50, 0x75, 0x74, 0x41, 0x72, 0x74, 0x69, 0x66, 0x61, 0x63, 0x74, 0x52, 0x65, 0x71, 0x75, 0x65, + 0x73, 0x74, 0x1a, 0x3c, 0x2e, 0x6f, 0x72, 0x67, 0x2e, 0x61, 0x70, 0x61, 0x63, 0x68, 0x65, 0x2e, + 0x62, 0x65, 0x61, 0x6d, 0x2e, 0x6d, 0x6f, 0x64, 0x65, 0x6c, 0x2e, 0x6a, 0x6f, 0x62, 0x5f, 0x6d, + 0x61, 0x6e, 0x61, 0x67, 0x65, 0x6d, 0x65, 0x6e, 0x74, 0x2e, 0x76, 0x31, 0x2e, 0x50, 0x75, 0x74, + 0x41, 0x72, 0x74, 0x69, 0x66, 0x61, 0x63, 0x74, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, + 0x28, 0x01, 0x12, 0x91, 0x01, 0x0a, 0x0e, 0x43, 0x6f, 0x6d, 0x6d, 0x69, 0x74, 0x4d, 0x61, 0x6e, + 0x69, 0x66, 0x65, 0x73, 0x74, 0x12, 0x3e, 0x2e, 0x6f, 0x72, 0x67, 0x2e, 0x61, 0x70, 0x61, 0x63, + 0x68, 0x65, 0x2e, 0x62, 0x65, 0x61, 0x6d, 0x2e, 0x6d, 0x6f, 0x64, 0x65, 0x6c, 0x2e, 0x6a, 0x6f, + 0x62, 0x5f, 0x6d, 0x61, 0x6e, 0x61, 0x67, 0x65, 0x6d, 0x65, 0x6e, 0x74, 0x2e, 0x76, 0x31, 0x2e, + 0x43, 0x6f, 0x6d, 0x6d, 0x69, 0x74, 0x4d, 0x61, 0x6e, 0x69, 0x66, 0x65, 0x73, 0x74, 0x52, 0x65, + 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x3f, 0x2e, 0x6f, 0x72, 0x67, 0x2e, 0x61, 0x70, 0x61, 0x63, + 0x68, 0x65, 0x2e, 0x62, 0x65, 0x61, 0x6d, 0x2e, 0x6d, 0x6f, 0x64, 0x65, 0x6c, 0x2e, 0x6a, 0x6f, + 0x62, 0x5f, 0x6d, 0x61, 0x6e, 0x61, 0x67, 0x65, 0x6d, 0x65, 0x6e, 0x74, 0x2e, 0x76, 0x31, 0x2e, + 0x43, 0x6f, 0x6d, 0x6d, 0x69, 0x74, 0x4d, 0x61, 0x6e, 0x69, 0x66, 0x65, 0x73, 0x74, 0x52, 0x65, + 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x32, 0xb8, 0x02, 0x0a, 0x1e, 0x4c, 0x65, 0x67, 0x61, 0x63, + 0x79, 0x41, 0x72, 0x74, 0x69, 0x66, 0x61, 0x63, 0x74, 0x52, 0x65, 0x74, 0x72, 0x69, 0x65, 0x76, + 0x61, 0x6c, 0x53, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, 0x12, 0x88, 0x01, 0x0a, 0x0b, 0x47, 0x65, + 0x74, 0x4d, 0x61, 0x6e, 0x69, 0x66, 0x65, 0x73, 0x74, 0x12, 0x3b, 0x2e, 0x6f, 0x72, 0x67, 0x2e, + 0x61, 0x70, 0x61, 0x63, 0x68, 0x65, 0x2e, 0x62, 0x65, 0x61, 0x6d, 0x2e, 0x6d, 0x6f, 0x64, 0x65, + 0x6c, 0x2e, 0x6a, 0x6f, 0x62, 0x5f, 0x6d, 0x61, 0x6e, 0x61, 0x67, 0x65, 0x6d, 0x65, 0x6e, 0x74, + 0x2e, 0x76, 0x31, 0x2e, 0x47, 0x65, 0x74, 0x4d, 0x61, 0x6e, 0x69, 0x66, 0x65, 0x73, 0x74, 0x52, + 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x3c, 0x2e, 0x6f, 0x72, 0x67, 0x2e, 0x61, 0x70, 0x61, + 0x63, 0x68, 0x65, 0x2e, 0x62, 0x65, 0x61, 0x6d, 0x2e, 0x6d, 0x6f, 0x64, 0x65, 0x6c, 0x2e, 0x6a, + 0x6f, 0x62, 0x5f, 0x6d, 0x61, 0x6e, 0x61, 0x67, 0x65, 0x6d, 0x65, 0x6e, 0x74, 0x2e, 0x76, 0x31, + 0x2e, 0x47, 0x65, 0x74, 0x4d, 0x61, 0x6e, 0x69, 0x66, 0x65, 0x73, 0x74, 0x52, 0x65, 0x73, 0x70, + 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x8a, 0x01, 0x0a, 0x0b, 0x47, 0x65, 0x74, 0x41, 0x72, 0x74, 0x69, + 0x66, 0x61, 0x63, 0x74, 0x12, 0x41, 0x2e, 0x6f, 0x72, 0x67, 0x2e, 0x61, 0x70, 0x61, 0x63, 0x68, + 0x65, 0x2e, 0x62, 0x65, 0x61, 0x6d, 0x2e, 0x6d, 0x6f, 0x64, 0x65, 0x6c, 0x2e, 0x6a, 0x6f, 0x62, + 0x5f, 0x6d, 0x61, 0x6e, 0x61, 0x67, 0x65, 0x6d, 0x65, 0x6e, 0x74, 0x2e, 0x76, 0x31, 0x2e, 0x4c, + 0x65, 0x67, 0x61, 0x63, 0x79, 0x47, 0x65, 0x74, 0x41, 0x72, 0x74, 0x69, 0x66, 0x61, 0x63, 0x74, + 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x36, 0x2e, 0x6f, 0x72, 0x67, 0x2e, 0x61, 0x70, + 0x61, 0x63, 0x68, 0x65, 0x2e, 0x62, 0x65, 0x61, 0x6d, 0x2e, 0x6d, 0x6f, 0x64, 0x65, 0x6c, 0x2e, + 0x6a, 0x6f, 0x62, 0x5f, 0x6d, 0x61, 0x6e, 0x61, 0x67, 0x65, 0x6d, 0x65, 0x6e, 0x74, 0x2e, 0x76, + 0x31, 0x2e, 0x41, 0x72, 0x74, 0x69, 0x66, 0x61, 0x63, 0x74, 0x43, 0x68, 0x75, 0x6e, 0x6b, 0x30, + 0x01, 0x42, 0x89, 0x01, 0x0a, 0x26, 0x6f, 0x72, 0x67, 0x2e, 0x61, 0x70, 0x61, 0x63, 0x68, 0x65, + 0x2e, 0x62, 0x65, 0x61, 0x6d, 0x2e, 0x6d, 0x6f, 0x64, 0x65, 0x6c, 0x2e, 0x6a, 0x6f, 0x62, 0x6d, + 0x61, 0x6e, 0x61, 0x67, 0x65, 0x6d, 0x65, 0x6e, 0x74, 0x2e, 0x76, 0x31, 0x42, 0x0b, 0x41, 0x72, + 0x74, 0x69, 0x66, 0x61, 0x63, 0x74, 0x41, 0x70, 0x69, 0x5a, 0x52, 0x67, 0x69, 0x74, 0x68, 0x75, + 0x62, 0x2e, 0x63, 0x6f, 0x6d, 0x2f, 0x61, 0x70, 0x61, 0x63, 0x68, 0x65, 0x2f, 0x62, 0x65, 0x61, + 0x6d, 0x2f, 0x73, 0x64, 0x6b, 0x73, 0x2f, 0x76, 0x32, 0x2f, 0x67, 0x6f, 0x2f, 0x70, 0x6b, 0x67, + 0x2f, 0x62, 0x65, 0x61, 0x6d, 0x2f, 0x6d, 0x6f, 0x64, 0x65, 0x6c, 0x2f, 0x6a, 0x6f, 0x62, 0x6d, + 0x61, 0x6e, 0x61, 0x67, 0x65, 0x6d, 0x65, 0x6e, 0x74, 0x5f, 0x76, 0x31, 0x3b, 0x6a, 0x6f, 0x62, + 0x6d, 0x61, 0x6e, 0x61, 0x67, 0x65, 0x6d, 0x65, 0x6e, 0x74, 0x5f, 0x76, 0x31, 0x62, 0x06, 0x70, + 0x72, 0x6f, 0x74, 0x6f, 0x33, +} + +var file_org_apache_beam_model_job_management_v1_beam_artifact_api_proto_enumTypes = make([]protoimpl.EnumInfo, 1) +var file_org_apache_beam_model_job_management_v1_beam_artifact_api_proto_msgTypes = make([]protoimpl.MessageInfo, 19) +var file_org_apache_beam_model_job_management_v1_beam_artifact_api_proto_goTypes = []any{ + (CommitManifestResponse_Constants)(0), // 0: org.apache.beam.model.job_management.v1.CommitManifestResponse.Constants + (*ResolveArtifactsRequest)(nil), // 1: org.apache.beam.model.job_management.v1.ResolveArtifactsRequest + (*ResolveArtifactsResponse)(nil), // 2: org.apache.beam.model.job_management.v1.ResolveArtifactsResponse + (*GetArtifactRequest)(nil), // 3: org.apache.beam.model.job_management.v1.GetArtifactRequest + (*GetArtifactResponse)(nil), // 4: org.apache.beam.model.job_management.v1.GetArtifactResponse + (*ArtifactRequestWrapper)(nil), // 5: org.apache.beam.model.job_management.v1.ArtifactRequestWrapper + (*ArtifactResponseWrapper)(nil), // 6: org.apache.beam.model.job_management.v1.ArtifactResponseWrapper + (*ArtifactMetadata)(nil), // 7: org.apache.beam.model.job_management.v1.ArtifactMetadata + (*Manifest)(nil), // 8: org.apache.beam.model.job_management.v1.Manifest + (*ProxyManifest)(nil), // 9: org.apache.beam.model.job_management.v1.ProxyManifest + (*GetManifestRequest)(nil), // 10: org.apache.beam.model.job_management.v1.GetManifestRequest + (*GetManifestResponse)(nil), // 11: org.apache.beam.model.job_management.v1.GetManifestResponse + (*LegacyGetArtifactRequest)(nil), // 12: org.apache.beam.model.job_management.v1.LegacyGetArtifactRequest + (*ArtifactChunk)(nil), // 13: org.apache.beam.model.job_management.v1.ArtifactChunk + (*PutArtifactMetadata)(nil), // 14: org.apache.beam.model.job_management.v1.PutArtifactMetadata + (*PutArtifactRequest)(nil), // 15: org.apache.beam.model.job_management.v1.PutArtifactRequest + (*PutArtifactResponse)(nil), // 16: org.apache.beam.model.job_management.v1.PutArtifactResponse + (*CommitManifestRequest)(nil), // 17: org.apache.beam.model.job_management.v1.CommitManifestRequest + (*CommitManifestResponse)(nil), // 18: org.apache.beam.model.job_management.v1.CommitManifestResponse + (*ProxyManifest_Location)(nil), // 19: org.apache.beam.model.job_management.v1.ProxyManifest.Location + (*pipeline_v1.ArtifactInformation)(nil), // 20: org.apache.beam.model.pipeline.v1.ArtifactInformation +} +var file_org_apache_beam_model_job_management_v1_beam_artifact_api_proto_depIdxs = []int32{ + 20, // 0: org.apache.beam.model.job_management.v1.ResolveArtifactsRequest.artifacts:type_name -> org.apache.beam.model.pipeline.v1.ArtifactInformation + 20, // 1: org.apache.beam.model.job_management.v1.ResolveArtifactsResponse.replacements:type_name -> org.apache.beam.model.pipeline.v1.ArtifactInformation + 20, // 2: org.apache.beam.model.job_management.v1.GetArtifactRequest.artifact:type_name -> org.apache.beam.model.pipeline.v1.ArtifactInformation + 1, // 3: org.apache.beam.model.job_management.v1.ArtifactRequestWrapper.resolve_artifact:type_name -> org.apache.beam.model.job_management.v1.ResolveArtifactsRequest + 3, // 4: org.apache.beam.model.job_management.v1.ArtifactRequestWrapper.get_artifact:type_name -> org.apache.beam.model.job_management.v1.GetArtifactRequest + 2, // 5: org.apache.beam.model.job_management.v1.ArtifactResponseWrapper.resolve_artifact_response:type_name -> org.apache.beam.model.job_management.v1.ResolveArtifactsResponse + 4, // 6: org.apache.beam.model.job_management.v1.ArtifactResponseWrapper.get_artifact_response:type_name -> org.apache.beam.model.job_management.v1.GetArtifactResponse + 7, // 7: org.apache.beam.model.job_management.v1.Manifest.artifact:type_name -> org.apache.beam.model.job_management.v1.ArtifactMetadata + 8, // 8: org.apache.beam.model.job_management.v1.ProxyManifest.manifest:type_name -> org.apache.beam.model.job_management.v1.Manifest + 19, // 9: org.apache.beam.model.job_management.v1.ProxyManifest.location:type_name -> org.apache.beam.model.job_management.v1.ProxyManifest.Location + 8, // 10: org.apache.beam.model.job_management.v1.GetManifestResponse.manifest:type_name -> org.apache.beam.model.job_management.v1.Manifest + 7, // 11: org.apache.beam.model.job_management.v1.PutArtifactMetadata.metadata:type_name -> org.apache.beam.model.job_management.v1.ArtifactMetadata + 14, // 12: org.apache.beam.model.job_management.v1.PutArtifactRequest.metadata:type_name -> org.apache.beam.model.job_management.v1.PutArtifactMetadata + 13, // 13: org.apache.beam.model.job_management.v1.PutArtifactRequest.data:type_name -> org.apache.beam.model.job_management.v1.ArtifactChunk + 8, // 14: org.apache.beam.model.job_management.v1.CommitManifestRequest.manifest:type_name -> org.apache.beam.model.job_management.v1.Manifest + 1, // 15: org.apache.beam.model.job_management.v1.ArtifactRetrievalService.ResolveArtifacts:input_type -> org.apache.beam.model.job_management.v1.ResolveArtifactsRequest + 3, // 16: org.apache.beam.model.job_management.v1.ArtifactRetrievalService.GetArtifact:input_type -> org.apache.beam.model.job_management.v1.GetArtifactRequest + 6, // 17: org.apache.beam.model.job_management.v1.ArtifactStagingService.ReverseArtifactRetrievalService:input_type -> org.apache.beam.model.job_management.v1.ArtifactResponseWrapper + 15, // 18: org.apache.beam.model.job_management.v1.LegacyArtifactStagingService.PutArtifact:input_type -> org.apache.beam.model.job_management.v1.PutArtifactRequest + 17, // 19: org.apache.beam.model.job_management.v1.LegacyArtifactStagingService.CommitManifest:input_type -> org.apache.beam.model.job_management.v1.CommitManifestRequest + 10, // 20: org.apache.beam.model.job_management.v1.LegacyArtifactRetrievalService.GetManifest:input_type -> org.apache.beam.model.job_management.v1.GetManifestRequest + 12, // 21: org.apache.beam.model.job_management.v1.LegacyArtifactRetrievalService.GetArtifact:input_type -> org.apache.beam.model.job_management.v1.LegacyGetArtifactRequest + 2, // 22: org.apache.beam.model.job_management.v1.ArtifactRetrievalService.ResolveArtifacts:output_type -> org.apache.beam.model.job_management.v1.ResolveArtifactsResponse + 4, // 23: org.apache.beam.model.job_management.v1.ArtifactRetrievalService.GetArtifact:output_type -> org.apache.beam.model.job_management.v1.GetArtifactResponse + 5, // 24: org.apache.beam.model.job_management.v1.ArtifactStagingService.ReverseArtifactRetrievalService:output_type -> org.apache.beam.model.job_management.v1.ArtifactRequestWrapper + 16, // 25: org.apache.beam.model.job_management.v1.LegacyArtifactStagingService.PutArtifact:output_type -> org.apache.beam.model.job_management.v1.PutArtifactResponse + 18, // 26: org.apache.beam.model.job_management.v1.LegacyArtifactStagingService.CommitManifest:output_type -> org.apache.beam.model.job_management.v1.CommitManifestResponse + 11, // 27: org.apache.beam.model.job_management.v1.LegacyArtifactRetrievalService.GetManifest:output_type -> org.apache.beam.model.job_management.v1.GetManifestResponse + 13, // 28: org.apache.beam.model.job_management.v1.LegacyArtifactRetrievalService.GetArtifact:output_type -> org.apache.beam.model.job_management.v1.ArtifactChunk + 22, // [22:29] is the sub-list for method output_type + 15, // [15:22] is the sub-list for method input_type + 15, // [15:15] is the sub-list for extension type_name + 15, // [15:15] is the sub-list for extension extendee + 0, // [0:15] is the sub-list for field type_name +} + +func init() { file_org_apache_beam_model_job_management_v1_beam_artifact_api_proto_init() } +func file_org_apache_beam_model_job_management_v1_beam_artifact_api_proto_init() { + if File_org_apache_beam_model_job_management_v1_beam_artifact_api_proto != nil { + return + } + file_org_apache_beam_model_job_management_v1_beam_artifact_api_proto_msgTypes[4].OneofWrappers = []any{ + (*artifactRequestWrapper_ResolveArtifact)(nil), + (*artifactRequestWrapper_GetArtifact)(nil), + } + file_org_apache_beam_model_job_management_v1_beam_artifact_api_proto_msgTypes[5].OneofWrappers = []any{ + (*artifactResponseWrapper_ResolveArtifactResponse)(nil), + (*artifactResponseWrapper_GetArtifactResponse)(nil), + } + file_org_apache_beam_model_job_management_v1_beam_artifact_api_proto_msgTypes[14].OneofWrappers = []any{ + (*putArtifactRequest_Metadata)(nil), + (*putArtifactRequest_Data)(nil), + } + type x struct{} + out := protoimpl.TypeBuilder{ + File: protoimpl.DescBuilder{ + GoPackagePath: reflect.TypeOf(x{}).PkgPath(), + RawDescriptor: file_org_apache_beam_model_job_management_v1_beam_artifact_api_proto_rawDesc, + NumEnums: 1, + NumMessages: 19, + NumExtensions: 0, + NumServices: 4, + }, + GoTypes: file_org_apache_beam_model_job_management_v1_beam_artifact_api_proto_goTypes, + DependencyIndexes: file_org_apache_beam_model_job_management_v1_beam_artifact_api_proto_depIdxs, + EnumInfos: file_org_apache_beam_model_job_management_v1_beam_artifact_api_proto_enumTypes, + MessageInfos: file_org_apache_beam_model_job_management_v1_beam_artifact_api_proto_msgTypes, + }.Build() + File_org_apache_beam_model_job_management_v1_beam_artifact_api_proto = out.File + file_org_apache_beam_model_job_management_v1_beam_artifact_api_proto_rawDesc = nil + file_org_apache_beam_model_job_management_v1_beam_artifact_api_proto_goTypes = nil + file_org_apache_beam_model_job_management_v1_beam_artifact_api_proto_depIdxs = nil +} diff --git a/sdks/go/pkg/beam/model/jobmanagement_v1/beam_expansion_api.pb.go b/sdks/go/pkg/beam/model/jobmanagement_v1/beam_expansion_api.pb.go index 0917b6c96f0b..1ac0145d38c7 100644 --- a/sdks/go/pkg/beam/model/jobmanagement_v1/beam_expansion_api.pb.go +++ b/sdks/go/pkg/beam/model/jobmanagement_v1/beam_expansion_api.pb.go @@ -21,10 +21,12 @@ // Code generated by protoc-gen-go. DO NOT EDIT. // versions: -// protoc-gen-go v1.27.1 +// protoc-gen-go v1.36.0 // protoc v5.27.3 // source: org/apache/beam/model/job_management/v1/beam_expansion_api.proto +//go:build !protoopaque + package jobmanagement_v1 import ( @@ -33,7 +35,6 @@ import ( protoimpl "google.golang.org/protobuf/runtime/protoimpl" structpb "google.golang.org/protobuf/types/known/structpb" reflect "reflect" - sync "sync" ) const ( @@ -44,10 +45,7 @@ const ( ) type ExpansionRequest struct { - state protoimpl.MessageState - sizeCache protoimpl.SizeCache - unknownFields protoimpl.UnknownFields - + state protoimpl.MessageState `protogen:"hybrid.v1"` // Set of components needed to interpret the transform, or which // may be useful for its expansion. This includes the input // PCollections (if any) to the to-be-expanded transform, along @@ -64,22 +62,22 @@ type ExpansionRequest struct { // If it is set, asks the expansion service to use the given // coders for the output PCollections. Note that the request // may not be fulfilled. - OutputCoderRequests map[string]string `protobuf:"bytes,4,rep,name=output_coder_requests,json=outputCoderRequests,proto3" json:"output_coder_requests,omitempty" protobuf_key:"bytes,1,opt,name=key,proto3" protobuf_val:"bytes,2,opt,name=value,proto3"` + OutputCoderRequests map[string]string `protobuf:"bytes,4,rep,name=output_coder_requests,json=outputCoderRequests,proto3" json:"output_coder_requests,omitempty" protobuf_key:"bytes,1,opt,name=key" protobuf_val:"bytes,2,opt,name=value"` // A set of requirements that must be used by the expansion service to // interpret the components provided with this request. Requirements []string `protobuf:"bytes,5,rep,name=requirements,proto3" json:"requirements,omitempty"` // (Optional) A set of Pipeline Options that should be used // when expanding this transform. PipelineOptions *structpb.Struct `protobuf:"bytes,6,opt,name=pipeline_options,json=pipelineOptions,proto3" json:"pipeline_options,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache } func (x *ExpansionRequest) Reset() { *x = ExpansionRequest{} - if protoimpl.UnsafeEnabled { - mi := &file_org_apache_beam_model_job_management_v1_beam_expansion_api_proto_msgTypes[0] - ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) - ms.StoreMessageInfo(mi) - } + mi := &file_org_apache_beam_model_job_management_v1_beam_expansion_api_proto_msgTypes[0] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) } func (x *ExpansionRequest) String() string { @@ -90,7 +88,7 @@ func (*ExpansionRequest) ProtoMessage() {} func (x *ExpansionRequest) ProtoReflect() protoreflect.Message { mi := &file_org_apache_beam_model_job_management_v1_beam_expansion_api_proto_msgTypes[0] - if protoimpl.UnsafeEnabled && x != nil { + if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { ms.StoreMessageInfo(mi) @@ -100,11 +98,6 @@ func (x *ExpansionRequest) ProtoReflect() protoreflect.Message { return mi.MessageOf(x) } -// Deprecated: Use ExpansionRequest.ProtoReflect.Descriptor instead. -func (*ExpansionRequest) Descriptor() ([]byte, []int) { - return file_org_apache_beam_model_job_management_v1_beam_expansion_api_proto_rawDescGZIP(), []int{0} -} - func (x *ExpansionRequest) GetComponents() *pipeline_v1.Components { if x != nil { return x.Components @@ -147,11 +140,106 @@ func (x *ExpansionRequest) GetPipelineOptions() *structpb.Struct { return nil } -type ExpansionResponse struct { - state protoimpl.MessageState - sizeCache protoimpl.SizeCache - unknownFields protoimpl.UnknownFields +func (x *ExpansionRequest) SetComponents(v *pipeline_v1.Components) { + x.Components = v +} + +func (x *ExpansionRequest) SetTransform(v *pipeline_v1.PTransform) { + x.Transform = v +} + +func (x *ExpansionRequest) SetNamespace(v string) { + x.Namespace = v +} + +func (x *ExpansionRequest) SetOutputCoderRequests(v map[string]string) { + x.OutputCoderRequests = v +} + +func (x *ExpansionRequest) SetRequirements(v []string) { + x.Requirements = v +} + +func (x *ExpansionRequest) SetPipelineOptions(v *structpb.Struct) { + x.PipelineOptions = v +} + +func (x *ExpansionRequest) HasComponents() bool { + if x == nil { + return false + } + return x.Components != nil +} + +func (x *ExpansionRequest) HasTransform() bool { + if x == nil { + return false + } + return x.Transform != nil +} + +func (x *ExpansionRequest) HasPipelineOptions() bool { + if x == nil { + return false + } + return x.PipelineOptions != nil +} + +func (x *ExpansionRequest) ClearComponents() { + x.Components = nil +} + +func (x *ExpansionRequest) ClearTransform() { + x.Transform = nil +} + +func (x *ExpansionRequest) ClearPipelineOptions() { + x.PipelineOptions = nil +} + +type ExpansionRequest_builder struct { + _ [0]func() // Prevents comparability and use of unkeyed literals for the builder. + + // Set of components needed to interpret the transform, or which + // may be useful for its expansion. This includes the input + // PCollections (if any) to the to-be-expanded transform, along + // with their coders and windowing strategies. + Components *pipeline_v1.Components + // The actual PTransform to be expaneded according to its spec. + // Its input should be set, but its subtransforms and outputs + // should not be. + Transform *pipeline_v1.PTransform + // A namespace (prefix) to use for the id of any newly created + // components. + Namespace string + // (Optional) Map from a local output tag to a coder id. + // If it is set, asks the expansion service to use the given + // coders for the output PCollections. Note that the request + // may not be fulfilled. + OutputCoderRequests map[string]string + // A set of requirements that must be used by the expansion service to + // interpret the components provided with this request. + Requirements []string + // (Optional) A set of Pipeline Options that should be used + // when expanding this transform. + PipelineOptions *structpb.Struct +} + +func (b0 ExpansionRequest_builder) Build() *ExpansionRequest { + m0 := &ExpansionRequest{} + b, x := &b0, m0 + _, _ = b, x + x.Components = b.Components + x.Transform = b.Transform + x.Namespace = b.Namespace + x.OutputCoderRequests = b.OutputCoderRequests + x.Requirements = b.Requirements + x.PipelineOptions = b.PipelineOptions + return m0 +} +type ExpansionResponse struct { + state protoimpl.MessageState `protogen:"hybrid.v1"` // Set of components needed to execute the expanded transform, // including the (original) inputs, outputs, and subtransforms. Components *pipeline_v1.Components `protobuf:"bytes,1,opt,name=components,proto3" json:"components,omitempty"` @@ -163,16 +251,16 @@ type ExpansionResponse struct { Requirements []string `protobuf:"bytes,3,rep,name=requirements,proto3" json:"requirements,omitempty"` // (Optional) An string representation of any error encountered while // attempting to expand this transform. - Error string `protobuf:"bytes,10,opt,name=error,proto3" json:"error,omitempty"` + Error string `protobuf:"bytes,10,opt,name=error,proto3" json:"error,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache } func (x *ExpansionResponse) Reset() { *x = ExpansionResponse{} - if protoimpl.UnsafeEnabled { - mi := &file_org_apache_beam_model_job_management_v1_beam_expansion_api_proto_msgTypes[1] - ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) - ms.StoreMessageInfo(mi) - } + mi := &file_org_apache_beam_model_job_management_v1_beam_expansion_api_proto_msgTypes[1] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) } func (x *ExpansionResponse) String() string { @@ -183,7 +271,7 @@ func (*ExpansionResponse) ProtoMessage() {} func (x *ExpansionResponse) ProtoReflect() protoreflect.Message { mi := &file_org_apache_beam_model_job_management_v1_beam_expansion_api_proto_msgTypes[1] - if protoimpl.UnsafeEnabled && x != nil { + if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { ms.StoreMessageInfo(mi) @@ -193,11 +281,6 @@ func (x *ExpansionResponse) ProtoReflect() protoreflect.Message { return mi.MessageOf(x) } -// Deprecated: Use ExpansionResponse.ProtoReflect.Descriptor instead. -func (*ExpansionResponse) Descriptor() ([]byte, []int) { - return file_org_apache_beam_model_job_management_v1_beam_expansion_api_proto_rawDescGZIP(), []int{1} -} - func (x *ExpansionResponse) GetComponents() *pipeline_v1.Components { if x != nil { return x.Components @@ -226,19 +309,83 @@ func (x *ExpansionResponse) GetError() string { return "" } +func (x *ExpansionResponse) SetComponents(v *pipeline_v1.Components) { + x.Components = v +} + +func (x *ExpansionResponse) SetTransform(v *pipeline_v1.PTransform) { + x.Transform = v +} + +func (x *ExpansionResponse) SetRequirements(v []string) { + x.Requirements = v +} + +func (x *ExpansionResponse) SetError(v string) { + x.Error = v +} + +func (x *ExpansionResponse) HasComponents() bool { + if x == nil { + return false + } + return x.Components != nil +} + +func (x *ExpansionResponse) HasTransform() bool { + if x == nil { + return false + } + return x.Transform != nil +} + +func (x *ExpansionResponse) ClearComponents() { + x.Components = nil +} + +func (x *ExpansionResponse) ClearTransform() { + x.Transform = nil +} + +type ExpansionResponse_builder struct { + _ [0]func() // Prevents comparability and use of unkeyed literals for the builder. + + // Set of components needed to execute the expanded transform, + // including the (original) inputs, outputs, and subtransforms. + Components *pipeline_v1.Components + // The expanded transform itself, with references to its outputs + // and subtransforms. + Transform *pipeline_v1.PTransform + // A set of requirements that must be appended to this pipeline's + // requirements. + Requirements []string + // (Optional) An string representation of any error encountered while + // attempting to expand this transform. + Error string +} + +func (b0 ExpansionResponse_builder) Build() *ExpansionResponse { + m0 := &ExpansionResponse{} + b, x := &b0, m0 + _, _ = b, x + x.Components = b.Components + x.Transform = b.Transform + x.Requirements = b.Requirements + x.Error = b.Error + return m0 +} + type DiscoverSchemaTransformRequest struct { - state protoimpl.MessageState - sizeCache protoimpl.SizeCache + state protoimpl.MessageState `protogen:"hybrid.v1"` unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache } func (x *DiscoverSchemaTransformRequest) Reset() { *x = DiscoverSchemaTransformRequest{} - if protoimpl.UnsafeEnabled { - mi := &file_org_apache_beam_model_job_management_v1_beam_expansion_api_proto_msgTypes[2] - ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) - ms.StoreMessageInfo(mi) - } + mi := &file_org_apache_beam_model_job_management_v1_beam_expansion_api_proto_msgTypes[2] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) } func (x *DiscoverSchemaTransformRequest) String() string { @@ -249,7 +396,7 @@ func (*DiscoverSchemaTransformRequest) ProtoMessage() {} func (x *DiscoverSchemaTransformRequest) ProtoReflect() protoreflect.Message { mi := &file_org_apache_beam_model_job_management_v1_beam_expansion_api_proto_msgTypes[2] - if protoimpl.UnsafeEnabled && x != nil { + if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { ms.StoreMessageInfo(mi) @@ -259,16 +406,20 @@ func (x *DiscoverSchemaTransformRequest) ProtoReflect() protoreflect.Message { return mi.MessageOf(x) } -// Deprecated: Use DiscoverSchemaTransformRequest.ProtoReflect.Descriptor instead. -func (*DiscoverSchemaTransformRequest) Descriptor() ([]byte, []int) { - return file_org_apache_beam_model_job_management_v1_beam_expansion_api_proto_rawDescGZIP(), []int{2} +type DiscoverSchemaTransformRequest_builder struct { + _ [0]func() // Prevents comparability and use of unkeyed literals for the builder. + } -type SchemaTransformConfig struct { - state protoimpl.MessageState - sizeCache protoimpl.SizeCache - unknownFields protoimpl.UnknownFields +func (b0 DiscoverSchemaTransformRequest_builder) Build() *DiscoverSchemaTransformRequest { + m0 := &DiscoverSchemaTransformRequest{} + b, x := &b0, m0 + _, _ = b, x + return m0 +} +type SchemaTransformConfig struct { + state protoimpl.MessageState `protogen:"hybrid.v1"` // Config schema of the SchemaTransform ConfigSchema *pipeline_v1.Schema `protobuf:"bytes,1,opt,name=config_schema,json=configSchema,proto3" json:"config_schema,omitempty"` // Names of input PCollections @@ -279,16 +430,16 @@ type SchemaTransformConfig struct { // May be markdown formatted. // Note that configuration parameters may also have documentation attached // as part of the config_schema. - Description string `protobuf:"bytes,4,opt,name=description,proto3" json:"description,omitempty"` + Description string `protobuf:"bytes,4,opt,name=description,proto3" json:"description,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache } func (x *SchemaTransformConfig) Reset() { *x = SchemaTransformConfig{} - if protoimpl.UnsafeEnabled { - mi := &file_org_apache_beam_model_job_management_v1_beam_expansion_api_proto_msgTypes[3] - ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) - ms.StoreMessageInfo(mi) - } + mi := &file_org_apache_beam_model_job_management_v1_beam_expansion_api_proto_msgTypes[3] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) } func (x *SchemaTransformConfig) String() string { @@ -299,7 +450,7 @@ func (*SchemaTransformConfig) ProtoMessage() {} func (x *SchemaTransformConfig) ProtoReflect() protoreflect.Message { mi := &file_org_apache_beam_model_job_management_v1_beam_expansion_api_proto_msgTypes[3] - if protoimpl.UnsafeEnabled && x != nil { + if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { ms.StoreMessageInfo(mi) @@ -309,11 +460,6 @@ func (x *SchemaTransformConfig) ProtoReflect() protoreflect.Message { return mi.MessageOf(x) } -// Deprecated: Use SchemaTransformConfig.ProtoReflect.Descriptor instead. -func (*SchemaTransformConfig) Descriptor() ([]byte, []int) { - return file_org_apache_beam_model_job_management_v1_beam_expansion_api_proto_rawDescGZIP(), []int{3} -} - func (x *SchemaTransformConfig) GetConfigSchema() *pipeline_v1.Schema { if x != nil { return x.ConfigSchema @@ -342,25 +488,76 @@ func (x *SchemaTransformConfig) GetDescription() string { return "" } -type DiscoverSchemaTransformResponse struct { - state protoimpl.MessageState - sizeCache protoimpl.SizeCache - unknownFields protoimpl.UnknownFields +func (x *SchemaTransformConfig) SetConfigSchema(v *pipeline_v1.Schema) { + x.ConfigSchema = v +} + +func (x *SchemaTransformConfig) SetInputPcollectionNames(v []string) { + x.InputPcollectionNames = v +} +func (x *SchemaTransformConfig) SetOutputPcollectionNames(v []string) { + x.OutputPcollectionNames = v +} + +func (x *SchemaTransformConfig) SetDescription(v string) { + x.Description = v +} + +func (x *SchemaTransformConfig) HasConfigSchema() bool { + if x == nil { + return false + } + return x.ConfigSchema != nil +} + +func (x *SchemaTransformConfig) ClearConfigSchema() { + x.ConfigSchema = nil +} + +type SchemaTransformConfig_builder struct { + _ [0]func() // Prevents comparability and use of unkeyed literals for the builder. + + // Config schema of the SchemaTransform + ConfigSchema *pipeline_v1.Schema + // Names of input PCollections + InputPcollectionNames []string + // Names of output PCollections + OutputPcollectionNames []string + // Description of this transform and usage used for documentation. + // May be markdown formatted. + // Note that configuration parameters may also have documentation attached + // as part of the config_schema. + Description string +} + +func (b0 SchemaTransformConfig_builder) Build() *SchemaTransformConfig { + m0 := &SchemaTransformConfig{} + b, x := &b0, m0 + _, _ = b, x + x.ConfigSchema = b.ConfigSchema + x.InputPcollectionNames = b.InputPcollectionNames + x.OutputPcollectionNames = b.OutputPcollectionNames + x.Description = b.Description + return m0 +} + +type DiscoverSchemaTransformResponse struct { + state protoimpl.MessageState `protogen:"hybrid.v1"` // A mapping from SchemaTransform ID to schema transform config of discovered // SchemaTransforms - SchemaTransformConfigs map[string]*SchemaTransformConfig `protobuf:"bytes,1,rep,name=schema_transform_configs,json=schemaTransformConfigs,proto3" json:"schema_transform_configs,omitempty" protobuf_key:"bytes,1,opt,name=key,proto3" protobuf_val:"bytes,2,opt,name=value,proto3"` + SchemaTransformConfigs map[string]*SchemaTransformConfig `protobuf:"bytes,1,rep,name=schema_transform_configs,json=schemaTransformConfigs,proto3" json:"schema_transform_configs,omitempty" protobuf_key:"bytes,1,opt,name=key" protobuf_val:"bytes,2,opt,name=value"` // If list of identifies are empty, this may contain an error. - Error string `protobuf:"bytes,2,opt,name=error,proto3" json:"error,omitempty"` + Error string `protobuf:"bytes,2,opt,name=error,proto3" json:"error,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache } func (x *DiscoverSchemaTransformResponse) Reset() { *x = DiscoverSchemaTransformResponse{} - if protoimpl.UnsafeEnabled { - mi := &file_org_apache_beam_model_job_management_v1_beam_expansion_api_proto_msgTypes[4] - ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) - ms.StoreMessageInfo(mi) - } + mi := &file_org_apache_beam_model_job_management_v1_beam_expansion_api_proto_msgTypes[4] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) } func (x *DiscoverSchemaTransformResponse) String() string { @@ -371,7 +568,7 @@ func (*DiscoverSchemaTransformResponse) ProtoMessage() {} func (x *DiscoverSchemaTransformResponse) ProtoReflect() protoreflect.Message { mi := &file_org_apache_beam_model_job_management_v1_beam_expansion_api_proto_msgTypes[4] - if protoimpl.UnsafeEnabled && x != nil { + if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { ms.StoreMessageInfo(mi) @@ -381,11 +578,6 @@ func (x *DiscoverSchemaTransformResponse) ProtoReflect() protoreflect.Message { return mi.MessageOf(x) } -// Deprecated: Use DiscoverSchemaTransformResponse.ProtoReflect.Descriptor instead. -func (*DiscoverSchemaTransformResponse) Descriptor() ([]byte, []int) { - return file_org_apache_beam_model_job_management_v1_beam_expansion_api_proto_rawDescGZIP(), []int{4} -} - func (x *DiscoverSchemaTransformResponse) GetSchemaTransformConfigs() map[string]*SchemaTransformConfig { if x != nil { return x.SchemaTransformConfigs @@ -400,6 +592,33 @@ func (x *DiscoverSchemaTransformResponse) GetError() string { return "" } +func (x *DiscoverSchemaTransformResponse) SetSchemaTransformConfigs(v map[string]*SchemaTransformConfig) { + x.SchemaTransformConfigs = v +} + +func (x *DiscoverSchemaTransformResponse) SetError(v string) { + x.Error = v +} + +type DiscoverSchemaTransformResponse_builder struct { + _ [0]func() // Prevents comparability and use of unkeyed literals for the builder. + + // A mapping from SchemaTransform ID to schema transform config of discovered + // SchemaTransforms + SchemaTransformConfigs map[string]*SchemaTransformConfig + // If list of identifies are empty, this may contain an error. + Error string +} + +func (b0 DiscoverSchemaTransformResponse_builder) Build() *DiscoverSchemaTransformResponse { + m0 := &DiscoverSchemaTransformResponse{} + b, x := &b0, m0 + _, _ = b, x + x.SchemaTransformConfigs = b.SchemaTransformConfigs + x.Error = b.Error + return m0 +} + var File_org_apache_beam_model_job_management_v1_beam_expansion_api_proto protoreflect.FileDescriptor var file_org_apache_beam_model_job_management_v1_beam_expansion_api_proto_rawDesc = []byte{ @@ -535,20 +754,8 @@ var file_org_apache_beam_model_job_management_v1_beam_expansion_api_proto_rawDes 0x5f, 0x76, 0x31, 0x62, 0x06, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x33, } -var ( - file_org_apache_beam_model_job_management_v1_beam_expansion_api_proto_rawDescOnce sync.Once - file_org_apache_beam_model_job_management_v1_beam_expansion_api_proto_rawDescData = file_org_apache_beam_model_job_management_v1_beam_expansion_api_proto_rawDesc -) - -func file_org_apache_beam_model_job_management_v1_beam_expansion_api_proto_rawDescGZIP() []byte { - file_org_apache_beam_model_job_management_v1_beam_expansion_api_proto_rawDescOnce.Do(func() { - file_org_apache_beam_model_job_management_v1_beam_expansion_api_proto_rawDescData = protoimpl.X.CompressGZIP(file_org_apache_beam_model_job_management_v1_beam_expansion_api_proto_rawDescData) - }) - return file_org_apache_beam_model_job_management_v1_beam_expansion_api_proto_rawDescData -} - var file_org_apache_beam_model_job_management_v1_beam_expansion_api_proto_msgTypes = make([]protoimpl.MessageInfo, 7) -var file_org_apache_beam_model_job_management_v1_beam_expansion_api_proto_goTypes = []interface{}{ +var file_org_apache_beam_model_job_management_v1_beam_expansion_api_proto_goTypes = []any{ (*ExpansionRequest)(nil), // 0: org.apache.beam.model.expansion.v1.ExpansionRequest (*ExpansionResponse)(nil), // 1: org.apache.beam.model.expansion.v1.ExpansionResponse (*DiscoverSchemaTransformRequest)(nil), // 2: org.apache.beam.model.expansion.v1.DiscoverSchemaTransformRequest @@ -587,68 +794,6 @@ func file_org_apache_beam_model_job_management_v1_beam_expansion_api_proto_init( if File_org_apache_beam_model_job_management_v1_beam_expansion_api_proto != nil { return } - if !protoimpl.UnsafeEnabled { - file_org_apache_beam_model_job_management_v1_beam_expansion_api_proto_msgTypes[0].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*ExpansionRequest); i { - case 0: - return &v.state - case 1: - return &v.sizeCache - case 2: - return &v.unknownFields - default: - return nil - } - } - file_org_apache_beam_model_job_management_v1_beam_expansion_api_proto_msgTypes[1].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*ExpansionResponse); i { - case 0: - return &v.state - case 1: - return &v.sizeCache - case 2: - return &v.unknownFields - default: - return nil - } - } - file_org_apache_beam_model_job_management_v1_beam_expansion_api_proto_msgTypes[2].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*DiscoverSchemaTransformRequest); i { - case 0: - return &v.state - case 1: - return &v.sizeCache - case 2: - return &v.unknownFields - default: - return nil - } - } - file_org_apache_beam_model_job_management_v1_beam_expansion_api_proto_msgTypes[3].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*SchemaTransformConfig); i { - case 0: - return &v.state - case 1: - return &v.sizeCache - case 2: - return &v.unknownFields - default: - return nil - } - } - file_org_apache_beam_model_job_management_v1_beam_expansion_api_proto_msgTypes[4].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*DiscoverSchemaTransformResponse); i { - case 0: - return &v.state - case 1: - return &v.sizeCache - case 2: - return &v.unknownFields - default: - return nil - } - } - } type x struct{} out := protoimpl.TypeBuilder{ File: protoimpl.DescBuilder{ diff --git a/sdks/go/pkg/beam/model/jobmanagement_v1/beam_expansion_api_grpc.pb.go b/sdks/go/pkg/beam/model/jobmanagement_v1/beam_expansion_api_grpc.pb.go index 9d5e00606bb9..9c7d8e8b8262 100644 --- a/sdks/go/pkg/beam/model/jobmanagement_v1/beam_expansion_api_grpc.pb.go +++ b/sdks/go/pkg/beam/model/jobmanagement_v1/beam_expansion_api_grpc.pb.go @@ -14,9 +14,14 @@ // WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. // See the License for the specific language governing permissions and // limitations under the License. + +// +// Protocol Buffers describing the Expansion API, an api for expanding +// transforms in a remote SDK. + // Code generated by protoc-gen-go-grpc. DO NOT EDIT. // versions: -// - protoc-gen-go-grpc v1.1.0 +// - protoc-gen-go-grpc v1.5.1 // - protoc v5.27.3 // source: org/apache/beam/model/job_management/v1/beam_expansion_api.proto @@ -31,12 +36,19 @@ import ( // This is a compile-time assertion to ensure that this generated file // is compatible with the grpc package it is being compiled against. -// Requires gRPC-Go v1.32.0 or later. -const _ = grpc.SupportPackageIsVersion7 +// Requires gRPC-Go v1.64.0 or later. +const _ = grpc.SupportPackageIsVersion9 + +const ( + ExpansionService_Expand_FullMethodName = "/org.apache.beam.model.expansion.v1.ExpansionService/Expand" + ExpansionService_DiscoverSchemaTransform_FullMethodName = "/org.apache.beam.model.expansion.v1.ExpansionService/DiscoverSchemaTransform" +) // ExpansionServiceClient is the client API for ExpansionService service. // // For semantics around ctx use and closing/ending streaming RPCs, please refer to https://pkg.go.dev/google.golang.org/grpc/?tab=doc#ClientConn.NewStream. +// +// Job Service for constructing pipelines type ExpansionServiceClient interface { Expand(ctx context.Context, in *ExpansionRequest, opts ...grpc.CallOption) (*ExpansionResponse, error) // A RPC to discover already registered SchemaTransformProviders. @@ -53,8 +65,9 @@ func NewExpansionServiceClient(cc grpc.ClientConnInterface) ExpansionServiceClie } func (c *expansionServiceClient) Expand(ctx context.Context, in *ExpansionRequest, opts ...grpc.CallOption) (*ExpansionResponse, error) { + cOpts := append([]grpc.CallOption{grpc.StaticMethod()}, opts...) out := new(ExpansionResponse) - err := c.cc.Invoke(ctx, "/org.apache.beam.model.expansion.v1.ExpansionService/Expand", in, out, opts...) + err := c.cc.Invoke(ctx, ExpansionService_Expand_FullMethodName, in, out, cOpts...) if err != nil { return nil, err } @@ -62,8 +75,9 @@ func (c *expansionServiceClient) Expand(ctx context.Context, in *ExpansionReques } func (c *expansionServiceClient) DiscoverSchemaTransform(ctx context.Context, in *DiscoverSchemaTransformRequest, opts ...grpc.CallOption) (*DiscoverSchemaTransformResponse, error) { + cOpts := append([]grpc.CallOption{grpc.StaticMethod()}, opts...) out := new(DiscoverSchemaTransformResponse) - err := c.cc.Invoke(ctx, "/org.apache.beam.model.expansion.v1.ExpansionService/DiscoverSchemaTransform", in, out, opts...) + err := c.cc.Invoke(ctx, ExpansionService_DiscoverSchemaTransform_FullMethodName, in, out, cOpts...) if err != nil { return nil, err } @@ -72,7 +86,9 @@ func (c *expansionServiceClient) DiscoverSchemaTransform(ctx context.Context, in // ExpansionServiceServer is the server API for ExpansionService service. // All implementations must embed UnimplementedExpansionServiceServer -// for forward compatibility +// for forward compatibility. +// +// Job Service for constructing pipelines type ExpansionServiceServer interface { Expand(context.Context, *ExpansionRequest) (*ExpansionResponse, error) // A RPC to discover already registered SchemaTransformProviders. @@ -81,9 +97,12 @@ type ExpansionServiceServer interface { mustEmbedUnimplementedExpansionServiceServer() } -// UnimplementedExpansionServiceServer must be embedded to have forward compatible implementations. -type UnimplementedExpansionServiceServer struct { -} +// UnimplementedExpansionServiceServer must be embedded to have +// forward compatible implementations. +// +// NOTE: this should be embedded by value instead of pointer to avoid a nil +// pointer dereference when methods are called. +type UnimplementedExpansionServiceServer struct{} func (UnimplementedExpansionServiceServer) Expand(context.Context, *ExpansionRequest) (*ExpansionResponse, error) { return nil, status.Errorf(codes.Unimplemented, "method Expand not implemented") @@ -92,6 +111,7 @@ func (UnimplementedExpansionServiceServer) DiscoverSchemaTransform(context.Conte return nil, status.Errorf(codes.Unimplemented, "method DiscoverSchemaTransform not implemented") } func (UnimplementedExpansionServiceServer) mustEmbedUnimplementedExpansionServiceServer() {} +func (UnimplementedExpansionServiceServer) testEmbeddedByValue() {} // UnsafeExpansionServiceServer may be embedded to opt out of forward compatibility for this service. // Use of this interface is not recommended, as added methods to ExpansionServiceServer will @@ -101,6 +121,13 @@ type UnsafeExpansionServiceServer interface { } func RegisterExpansionServiceServer(s grpc.ServiceRegistrar, srv ExpansionServiceServer) { + // If the following call pancis, it indicates UnimplementedExpansionServiceServer was + // embedded by pointer and is nil. This will cause panics if an + // unimplemented method is ever invoked, so we test this at initialization + // time to prevent it from happening at runtime later due to I/O. + if t, ok := srv.(interface{ testEmbeddedByValue() }); ok { + t.testEmbeddedByValue() + } s.RegisterService(&ExpansionService_ServiceDesc, srv) } @@ -114,7 +141,7 @@ func _ExpansionService_Expand_Handler(srv interface{}, ctx context.Context, dec } info := &grpc.UnaryServerInfo{ Server: srv, - FullMethod: "/org.apache.beam.model.expansion.v1.ExpansionService/Expand", + FullMethod: ExpansionService_Expand_FullMethodName, } handler := func(ctx context.Context, req interface{}) (interface{}, error) { return srv.(ExpansionServiceServer).Expand(ctx, req.(*ExpansionRequest)) @@ -132,7 +159,7 @@ func _ExpansionService_DiscoverSchemaTransform_Handler(srv interface{}, ctx cont } info := &grpc.UnaryServerInfo{ Server: srv, - FullMethod: "/org.apache.beam.model.expansion.v1.ExpansionService/DiscoverSchemaTransform", + FullMethod: ExpansionService_DiscoverSchemaTransform_FullMethodName, } handler := func(ctx context.Context, req interface{}) (interface{}, error) { return srv.(ExpansionServiceServer).DiscoverSchemaTransform(ctx, req.(*DiscoverSchemaTransformRequest)) diff --git a/sdks/go/pkg/beam/model/jobmanagement_v1/beam_expansion_api_protoopaque.pb.go b/sdks/go/pkg/beam/model/jobmanagement_v1/beam_expansion_api_protoopaque.pb.go new file mode 100644 index 000000000000..4bcfbe714ab5 --- /dev/null +++ b/sdks/go/pkg/beam/model/jobmanagement_v1/beam_expansion_api_protoopaque.pb.go @@ -0,0 +1,780 @@ +// +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +// +// Protocol Buffers describing the Expansion API, an api for expanding +// transforms in a remote SDK. + +// Code generated by protoc-gen-go. DO NOT EDIT. +// versions: +// protoc-gen-go v1.36.0 +// protoc v5.27.3 +// source: org/apache/beam/model/job_management/v1/beam_expansion_api.proto + +//go:build protoopaque + +package jobmanagement_v1 + +import ( + pipeline_v1 "github.com/apache/beam/sdks/v2/go/pkg/beam/model/pipeline_v1" + protoreflect "google.golang.org/protobuf/reflect/protoreflect" + protoimpl "google.golang.org/protobuf/runtime/protoimpl" + structpb "google.golang.org/protobuf/types/known/structpb" + reflect "reflect" +) + +const ( + // Verify that this generated code is sufficiently up-to-date. + _ = protoimpl.EnforceVersion(20 - protoimpl.MinVersion) + // Verify that runtime/protoimpl is sufficiently up-to-date. + _ = protoimpl.EnforceVersion(protoimpl.MaxVersion - 20) +) + +type ExpansionRequest struct { + state protoimpl.MessageState `protogen:"opaque.v1"` + xxx_hidden_Components *pipeline_v1.Components `protobuf:"bytes,1,opt,name=components,proto3" json:"components,omitempty"` + xxx_hidden_Transform *pipeline_v1.PTransform `protobuf:"bytes,2,opt,name=transform,proto3" json:"transform,omitempty"` + xxx_hidden_Namespace string `protobuf:"bytes,3,opt,name=namespace,proto3" json:"namespace,omitempty"` + xxx_hidden_OutputCoderRequests map[string]string `protobuf:"bytes,4,rep,name=output_coder_requests,json=outputCoderRequests,proto3" json:"output_coder_requests,omitempty" protobuf_key:"bytes,1,opt,name=key" protobuf_val:"bytes,2,opt,name=value"` + xxx_hidden_Requirements []string `protobuf:"bytes,5,rep,name=requirements,proto3" json:"requirements,omitempty"` + xxx_hidden_PipelineOptions *structpb.Struct `protobuf:"bytes,6,opt,name=pipeline_options,json=pipelineOptions,proto3" json:"pipeline_options,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *ExpansionRequest) Reset() { + *x = ExpansionRequest{} + mi := &file_org_apache_beam_model_job_management_v1_beam_expansion_api_proto_msgTypes[0] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *ExpansionRequest) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*ExpansionRequest) ProtoMessage() {} + +func (x *ExpansionRequest) ProtoReflect() protoreflect.Message { + mi := &file_org_apache_beam_model_job_management_v1_beam_expansion_api_proto_msgTypes[0] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +func (x *ExpansionRequest) GetComponents() *pipeline_v1.Components { + if x != nil { + return x.xxx_hidden_Components + } + return nil +} + +func (x *ExpansionRequest) GetTransform() *pipeline_v1.PTransform { + if x != nil { + return x.xxx_hidden_Transform + } + return nil +} + +func (x *ExpansionRequest) GetNamespace() string { + if x != nil { + return x.xxx_hidden_Namespace + } + return "" +} + +func (x *ExpansionRequest) GetOutputCoderRequests() map[string]string { + if x != nil { + return x.xxx_hidden_OutputCoderRequests + } + return nil +} + +func (x *ExpansionRequest) GetRequirements() []string { + if x != nil { + return x.xxx_hidden_Requirements + } + return nil +} + +func (x *ExpansionRequest) GetPipelineOptions() *structpb.Struct { + if x != nil { + return x.xxx_hidden_PipelineOptions + } + return nil +} + +func (x *ExpansionRequest) SetComponents(v *pipeline_v1.Components) { + x.xxx_hidden_Components = v +} + +func (x *ExpansionRequest) SetTransform(v *pipeline_v1.PTransform) { + x.xxx_hidden_Transform = v +} + +func (x *ExpansionRequest) SetNamespace(v string) { + x.xxx_hidden_Namespace = v +} + +func (x *ExpansionRequest) SetOutputCoderRequests(v map[string]string) { + x.xxx_hidden_OutputCoderRequests = v +} + +func (x *ExpansionRequest) SetRequirements(v []string) { + x.xxx_hidden_Requirements = v +} + +func (x *ExpansionRequest) SetPipelineOptions(v *structpb.Struct) { + x.xxx_hidden_PipelineOptions = v +} + +func (x *ExpansionRequest) HasComponents() bool { + if x == nil { + return false + } + return x.xxx_hidden_Components != nil +} + +func (x *ExpansionRequest) HasTransform() bool { + if x == nil { + return false + } + return x.xxx_hidden_Transform != nil +} + +func (x *ExpansionRequest) HasPipelineOptions() bool { + if x == nil { + return false + } + return x.xxx_hidden_PipelineOptions != nil +} + +func (x *ExpansionRequest) ClearComponents() { + x.xxx_hidden_Components = nil +} + +func (x *ExpansionRequest) ClearTransform() { + x.xxx_hidden_Transform = nil +} + +func (x *ExpansionRequest) ClearPipelineOptions() { + x.xxx_hidden_PipelineOptions = nil +} + +type ExpansionRequest_builder struct { + _ [0]func() // Prevents comparability and use of unkeyed literals for the builder. + + // Set of components needed to interpret the transform, or which + // may be useful for its expansion. This includes the input + // PCollections (if any) to the to-be-expanded transform, along + // with their coders and windowing strategies. + Components *pipeline_v1.Components + // The actual PTransform to be expaneded according to its spec. + // Its input should be set, but its subtransforms and outputs + // should not be. + Transform *pipeline_v1.PTransform + // A namespace (prefix) to use for the id of any newly created + // components. + Namespace string + // (Optional) Map from a local output tag to a coder id. + // If it is set, asks the expansion service to use the given + // coders for the output PCollections. Note that the request + // may not be fulfilled. + OutputCoderRequests map[string]string + // A set of requirements that must be used by the expansion service to + // interpret the components provided with this request. + Requirements []string + // (Optional) A set of Pipeline Options that should be used + // when expanding this transform. + PipelineOptions *structpb.Struct +} + +func (b0 ExpansionRequest_builder) Build() *ExpansionRequest { + m0 := &ExpansionRequest{} + b, x := &b0, m0 + _, _ = b, x + x.xxx_hidden_Components = b.Components + x.xxx_hidden_Transform = b.Transform + x.xxx_hidden_Namespace = b.Namespace + x.xxx_hidden_OutputCoderRequests = b.OutputCoderRequests + x.xxx_hidden_Requirements = b.Requirements + x.xxx_hidden_PipelineOptions = b.PipelineOptions + return m0 +} + +type ExpansionResponse struct { + state protoimpl.MessageState `protogen:"opaque.v1"` + xxx_hidden_Components *pipeline_v1.Components `protobuf:"bytes,1,opt,name=components,proto3" json:"components,omitempty"` + xxx_hidden_Transform *pipeline_v1.PTransform `protobuf:"bytes,2,opt,name=transform,proto3" json:"transform,omitempty"` + xxx_hidden_Requirements []string `protobuf:"bytes,3,rep,name=requirements,proto3" json:"requirements,omitempty"` + xxx_hidden_Error string `protobuf:"bytes,10,opt,name=error,proto3" json:"error,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *ExpansionResponse) Reset() { + *x = ExpansionResponse{} + mi := &file_org_apache_beam_model_job_management_v1_beam_expansion_api_proto_msgTypes[1] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *ExpansionResponse) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*ExpansionResponse) ProtoMessage() {} + +func (x *ExpansionResponse) ProtoReflect() protoreflect.Message { + mi := &file_org_apache_beam_model_job_management_v1_beam_expansion_api_proto_msgTypes[1] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +func (x *ExpansionResponse) GetComponents() *pipeline_v1.Components { + if x != nil { + return x.xxx_hidden_Components + } + return nil +} + +func (x *ExpansionResponse) GetTransform() *pipeline_v1.PTransform { + if x != nil { + return x.xxx_hidden_Transform + } + return nil +} + +func (x *ExpansionResponse) GetRequirements() []string { + if x != nil { + return x.xxx_hidden_Requirements + } + return nil +} + +func (x *ExpansionResponse) GetError() string { + if x != nil { + return x.xxx_hidden_Error + } + return "" +} + +func (x *ExpansionResponse) SetComponents(v *pipeline_v1.Components) { + x.xxx_hidden_Components = v +} + +func (x *ExpansionResponse) SetTransform(v *pipeline_v1.PTransform) { + x.xxx_hidden_Transform = v +} + +func (x *ExpansionResponse) SetRequirements(v []string) { + x.xxx_hidden_Requirements = v +} + +func (x *ExpansionResponse) SetError(v string) { + x.xxx_hidden_Error = v +} + +func (x *ExpansionResponse) HasComponents() bool { + if x == nil { + return false + } + return x.xxx_hidden_Components != nil +} + +func (x *ExpansionResponse) HasTransform() bool { + if x == nil { + return false + } + return x.xxx_hidden_Transform != nil +} + +func (x *ExpansionResponse) ClearComponents() { + x.xxx_hidden_Components = nil +} + +func (x *ExpansionResponse) ClearTransform() { + x.xxx_hidden_Transform = nil +} + +type ExpansionResponse_builder struct { + _ [0]func() // Prevents comparability and use of unkeyed literals for the builder. + + // Set of components needed to execute the expanded transform, + // including the (original) inputs, outputs, and subtransforms. + Components *pipeline_v1.Components + // The expanded transform itself, with references to its outputs + // and subtransforms. + Transform *pipeline_v1.PTransform + // A set of requirements that must be appended to this pipeline's + // requirements. + Requirements []string + // (Optional) An string representation of any error encountered while + // attempting to expand this transform. + Error string +} + +func (b0 ExpansionResponse_builder) Build() *ExpansionResponse { + m0 := &ExpansionResponse{} + b, x := &b0, m0 + _, _ = b, x + x.xxx_hidden_Components = b.Components + x.xxx_hidden_Transform = b.Transform + x.xxx_hidden_Requirements = b.Requirements + x.xxx_hidden_Error = b.Error + return m0 +} + +type DiscoverSchemaTransformRequest struct { + state protoimpl.MessageState `protogen:"opaque.v1"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *DiscoverSchemaTransformRequest) Reset() { + *x = DiscoverSchemaTransformRequest{} + mi := &file_org_apache_beam_model_job_management_v1_beam_expansion_api_proto_msgTypes[2] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *DiscoverSchemaTransformRequest) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*DiscoverSchemaTransformRequest) ProtoMessage() {} + +func (x *DiscoverSchemaTransformRequest) ProtoReflect() protoreflect.Message { + mi := &file_org_apache_beam_model_job_management_v1_beam_expansion_api_proto_msgTypes[2] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +type DiscoverSchemaTransformRequest_builder struct { + _ [0]func() // Prevents comparability and use of unkeyed literals for the builder. + +} + +func (b0 DiscoverSchemaTransformRequest_builder) Build() *DiscoverSchemaTransformRequest { + m0 := &DiscoverSchemaTransformRequest{} + b, x := &b0, m0 + _, _ = b, x + return m0 +} + +type SchemaTransformConfig struct { + state protoimpl.MessageState `protogen:"opaque.v1"` + xxx_hidden_ConfigSchema *pipeline_v1.Schema `protobuf:"bytes,1,opt,name=config_schema,json=configSchema,proto3" json:"config_schema,omitempty"` + xxx_hidden_InputPcollectionNames []string `protobuf:"bytes,2,rep,name=input_pcollection_names,json=inputPcollectionNames,proto3" json:"input_pcollection_names,omitempty"` + xxx_hidden_OutputPcollectionNames []string `protobuf:"bytes,3,rep,name=output_pcollection_names,json=outputPcollectionNames,proto3" json:"output_pcollection_names,omitempty"` + xxx_hidden_Description string `protobuf:"bytes,4,opt,name=description,proto3" json:"description,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *SchemaTransformConfig) Reset() { + *x = SchemaTransformConfig{} + mi := &file_org_apache_beam_model_job_management_v1_beam_expansion_api_proto_msgTypes[3] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *SchemaTransformConfig) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*SchemaTransformConfig) ProtoMessage() {} + +func (x *SchemaTransformConfig) ProtoReflect() protoreflect.Message { + mi := &file_org_apache_beam_model_job_management_v1_beam_expansion_api_proto_msgTypes[3] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +func (x *SchemaTransformConfig) GetConfigSchema() *pipeline_v1.Schema { + if x != nil { + return x.xxx_hidden_ConfigSchema + } + return nil +} + +func (x *SchemaTransformConfig) GetInputPcollectionNames() []string { + if x != nil { + return x.xxx_hidden_InputPcollectionNames + } + return nil +} + +func (x *SchemaTransformConfig) GetOutputPcollectionNames() []string { + if x != nil { + return x.xxx_hidden_OutputPcollectionNames + } + return nil +} + +func (x *SchemaTransformConfig) GetDescription() string { + if x != nil { + return x.xxx_hidden_Description + } + return "" +} + +func (x *SchemaTransformConfig) SetConfigSchema(v *pipeline_v1.Schema) { + x.xxx_hidden_ConfigSchema = v +} + +func (x *SchemaTransformConfig) SetInputPcollectionNames(v []string) { + x.xxx_hidden_InputPcollectionNames = v +} + +func (x *SchemaTransformConfig) SetOutputPcollectionNames(v []string) { + x.xxx_hidden_OutputPcollectionNames = v +} + +func (x *SchemaTransformConfig) SetDescription(v string) { + x.xxx_hidden_Description = v +} + +func (x *SchemaTransformConfig) HasConfigSchema() bool { + if x == nil { + return false + } + return x.xxx_hidden_ConfigSchema != nil +} + +func (x *SchemaTransformConfig) ClearConfigSchema() { + x.xxx_hidden_ConfigSchema = nil +} + +type SchemaTransformConfig_builder struct { + _ [0]func() // Prevents comparability and use of unkeyed literals for the builder. + + // Config schema of the SchemaTransform + ConfigSchema *pipeline_v1.Schema + // Names of input PCollections + InputPcollectionNames []string + // Names of output PCollections + OutputPcollectionNames []string + // Description of this transform and usage used for documentation. + // May be markdown formatted. + // Note that configuration parameters may also have documentation attached + // as part of the config_schema. + Description string +} + +func (b0 SchemaTransformConfig_builder) Build() *SchemaTransformConfig { + m0 := &SchemaTransformConfig{} + b, x := &b0, m0 + _, _ = b, x + x.xxx_hidden_ConfigSchema = b.ConfigSchema + x.xxx_hidden_InputPcollectionNames = b.InputPcollectionNames + x.xxx_hidden_OutputPcollectionNames = b.OutputPcollectionNames + x.xxx_hidden_Description = b.Description + return m0 +} + +type DiscoverSchemaTransformResponse struct { + state protoimpl.MessageState `protogen:"opaque.v1"` + xxx_hidden_SchemaTransformConfigs map[string]*SchemaTransformConfig `protobuf:"bytes,1,rep,name=schema_transform_configs,json=schemaTransformConfigs,proto3" json:"schema_transform_configs,omitempty" protobuf_key:"bytes,1,opt,name=key" protobuf_val:"bytes,2,opt,name=value"` + xxx_hidden_Error string `protobuf:"bytes,2,opt,name=error,proto3" json:"error,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *DiscoverSchemaTransformResponse) Reset() { + *x = DiscoverSchemaTransformResponse{} + mi := &file_org_apache_beam_model_job_management_v1_beam_expansion_api_proto_msgTypes[4] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *DiscoverSchemaTransformResponse) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*DiscoverSchemaTransformResponse) ProtoMessage() {} + +func (x *DiscoverSchemaTransformResponse) ProtoReflect() protoreflect.Message { + mi := &file_org_apache_beam_model_job_management_v1_beam_expansion_api_proto_msgTypes[4] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +func (x *DiscoverSchemaTransformResponse) GetSchemaTransformConfigs() map[string]*SchemaTransformConfig { + if x != nil { + return x.xxx_hidden_SchemaTransformConfigs + } + return nil +} + +func (x *DiscoverSchemaTransformResponse) GetError() string { + if x != nil { + return x.xxx_hidden_Error + } + return "" +} + +func (x *DiscoverSchemaTransformResponse) SetSchemaTransformConfigs(v map[string]*SchemaTransformConfig) { + x.xxx_hidden_SchemaTransformConfigs = v +} + +func (x *DiscoverSchemaTransformResponse) SetError(v string) { + x.xxx_hidden_Error = v +} + +type DiscoverSchemaTransformResponse_builder struct { + _ [0]func() // Prevents comparability and use of unkeyed literals for the builder. + + // A mapping from SchemaTransform ID to schema transform config of discovered + // SchemaTransforms + SchemaTransformConfigs map[string]*SchemaTransformConfig + // If list of identifies are empty, this may contain an error. + Error string +} + +func (b0 DiscoverSchemaTransformResponse_builder) Build() *DiscoverSchemaTransformResponse { + m0 := &DiscoverSchemaTransformResponse{} + b, x := &b0, m0 + _, _ = b, x + x.xxx_hidden_SchemaTransformConfigs = b.SchemaTransformConfigs + x.xxx_hidden_Error = b.Error + return m0 +} + +var File_org_apache_beam_model_job_management_v1_beam_expansion_api_proto protoreflect.FileDescriptor + +var file_org_apache_beam_model_job_management_v1_beam_expansion_api_proto_rawDesc = []byte{ + 0x0a, 0x40, 0x6f, 0x72, 0x67, 0x2f, 0x61, 0x70, 0x61, 0x63, 0x68, 0x65, 0x2f, 0x62, 0x65, 0x61, + 0x6d, 0x2f, 0x6d, 0x6f, 0x64, 0x65, 0x6c, 0x2f, 0x6a, 0x6f, 0x62, 0x5f, 0x6d, 0x61, 0x6e, 0x61, + 0x67, 0x65, 0x6d, 0x65, 0x6e, 0x74, 0x2f, 0x76, 0x31, 0x2f, 0x62, 0x65, 0x61, 0x6d, 0x5f, 0x65, + 0x78, 0x70, 0x61, 0x6e, 0x73, 0x69, 0x6f, 0x6e, 0x5f, 0x61, 0x70, 0x69, 0x2e, 0x70, 0x72, 0x6f, + 0x74, 0x6f, 0x12, 0x22, 0x6f, 0x72, 0x67, 0x2e, 0x61, 0x70, 0x61, 0x63, 0x68, 0x65, 0x2e, 0x62, + 0x65, 0x61, 0x6d, 0x2e, 0x6d, 0x6f, 0x64, 0x65, 0x6c, 0x2e, 0x65, 0x78, 0x70, 0x61, 0x6e, 0x73, + 0x69, 0x6f, 0x6e, 0x2e, 0x76, 0x31, 0x1a, 0x1c, 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2f, 0x70, + 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66, 0x2f, 0x73, 0x74, 0x72, 0x75, 0x63, 0x74, 0x2e, 0x70, + 0x72, 0x6f, 0x74, 0x6f, 0x1a, 0x37, 0x6f, 0x72, 0x67, 0x2f, 0x61, 0x70, 0x61, 0x63, 0x68, 0x65, + 0x2f, 0x62, 0x65, 0x61, 0x6d, 0x2f, 0x6d, 0x6f, 0x64, 0x65, 0x6c, 0x2f, 0x70, 0x69, 0x70, 0x65, + 0x6c, 0x69, 0x6e, 0x65, 0x2f, 0x76, 0x31, 0x2f, 0x62, 0x65, 0x61, 0x6d, 0x5f, 0x72, 0x75, 0x6e, + 0x6e, 0x65, 0x72, 0x5f, 0x61, 0x70, 0x69, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x1a, 0x2e, 0x6f, + 0x72, 0x67, 0x2f, 0x61, 0x70, 0x61, 0x63, 0x68, 0x65, 0x2f, 0x62, 0x65, 0x61, 0x6d, 0x2f, 0x6d, + 0x6f, 0x64, 0x65, 0x6c, 0x2f, 0x70, 0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e, 0x65, 0x2f, 0x76, 0x31, + 0x2f, 0x73, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x22, 0x80, 0x04, + 0x0a, 0x10, 0x45, 0x78, 0x70, 0x61, 0x6e, 0x73, 0x69, 0x6f, 0x6e, 0x52, 0x65, 0x71, 0x75, 0x65, + 0x73, 0x74, 0x12, 0x4d, 0x0a, 0x0a, 0x63, 0x6f, 0x6d, 0x70, 0x6f, 0x6e, 0x65, 0x6e, 0x74, 0x73, + 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x2d, 0x2e, 0x6f, 0x72, 0x67, 0x2e, 0x61, 0x70, 0x61, + 0x63, 0x68, 0x65, 0x2e, 0x62, 0x65, 0x61, 0x6d, 0x2e, 0x6d, 0x6f, 0x64, 0x65, 0x6c, 0x2e, 0x70, + 0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x43, 0x6f, 0x6d, 0x70, 0x6f, + 0x6e, 0x65, 0x6e, 0x74, 0x73, 0x52, 0x0a, 0x63, 0x6f, 0x6d, 0x70, 0x6f, 0x6e, 0x65, 0x6e, 0x74, + 0x73, 0x12, 0x4b, 0x0a, 0x09, 0x74, 0x72, 0x61, 0x6e, 0x73, 0x66, 0x6f, 0x72, 0x6d, 0x18, 0x02, + 0x20, 0x01, 0x28, 0x0b, 0x32, 0x2d, 0x2e, 0x6f, 0x72, 0x67, 0x2e, 0x61, 0x70, 0x61, 0x63, 0x68, + 0x65, 0x2e, 0x62, 0x65, 0x61, 0x6d, 0x2e, 0x6d, 0x6f, 0x64, 0x65, 0x6c, 0x2e, 0x70, 0x69, 0x70, + 0x65, 0x6c, 0x69, 0x6e, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x50, 0x54, 0x72, 0x61, 0x6e, 0x73, 0x66, + 0x6f, 0x72, 0x6d, 0x52, 0x09, 0x74, 0x72, 0x61, 0x6e, 0x73, 0x66, 0x6f, 0x72, 0x6d, 0x12, 0x1c, + 0x0a, 0x09, 0x6e, 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, 0x18, 0x03, 0x20, 0x01, 0x28, + 0x09, 0x52, 0x09, 0x6e, 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, 0x12, 0x81, 0x01, 0x0a, + 0x15, 0x6f, 0x75, 0x74, 0x70, 0x75, 0x74, 0x5f, 0x63, 0x6f, 0x64, 0x65, 0x72, 0x5f, 0x72, 0x65, + 0x71, 0x75, 0x65, 0x73, 0x74, 0x73, 0x18, 0x04, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x4d, 0x2e, 0x6f, + 0x72, 0x67, 0x2e, 0x61, 0x70, 0x61, 0x63, 0x68, 0x65, 0x2e, 0x62, 0x65, 0x61, 0x6d, 0x2e, 0x6d, + 0x6f, 0x64, 0x65, 0x6c, 0x2e, 0x65, 0x78, 0x70, 0x61, 0x6e, 0x73, 0x69, 0x6f, 0x6e, 0x2e, 0x76, + 0x31, 0x2e, 0x45, 0x78, 0x70, 0x61, 0x6e, 0x73, 0x69, 0x6f, 0x6e, 0x52, 0x65, 0x71, 0x75, 0x65, + 0x73, 0x74, 0x2e, 0x4f, 0x75, 0x74, 0x70, 0x75, 0x74, 0x43, 0x6f, 0x64, 0x65, 0x72, 0x52, 0x65, + 0x71, 0x75, 0x65, 0x73, 0x74, 0x73, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x52, 0x13, 0x6f, 0x75, 0x74, + 0x70, 0x75, 0x74, 0x43, 0x6f, 0x64, 0x65, 0x72, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x73, + 0x12, 0x22, 0x0a, 0x0c, 0x72, 0x65, 0x71, 0x75, 0x69, 0x72, 0x65, 0x6d, 0x65, 0x6e, 0x74, 0x73, + 0x18, 0x05, 0x20, 0x03, 0x28, 0x09, 0x52, 0x0c, 0x72, 0x65, 0x71, 0x75, 0x69, 0x72, 0x65, 0x6d, + 0x65, 0x6e, 0x74, 0x73, 0x12, 0x42, 0x0a, 0x10, 0x70, 0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e, 0x65, + 0x5f, 0x6f, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x18, 0x06, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x17, + 0x2e, 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66, + 0x2e, 0x53, 0x74, 0x72, 0x75, 0x63, 0x74, 0x52, 0x0f, 0x70, 0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e, + 0x65, 0x4f, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x1a, 0x46, 0x0a, 0x18, 0x4f, 0x75, 0x74, 0x70, + 0x75, 0x74, 0x43, 0x6f, 0x64, 0x65, 0x72, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x73, 0x45, + 0x6e, 0x74, 0x72, 0x79, 0x12, 0x10, 0x0a, 0x03, 0x6b, 0x65, 0x79, 0x18, 0x01, 0x20, 0x01, 0x28, + 0x09, 0x52, 0x03, 0x6b, 0x65, 0x79, 0x12, 0x14, 0x0a, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x18, + 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x3a, 0x02, 0x38, 0x01, + 0x22, 0xe9, 0x01, 0x0a, 0x11, 0x45, 0x78, 0x70, 0x61, 0x6e, 0x73, 0x69, 0x6f, 0x6e, 0x52, 0x65, + 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x4d, 0x0a, 0x0a, 0x63, 0x6f, 0x6d, 0x70, 0x6f, 0x6e, + 0x65, 0x6e, 0x74, 0x73, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x2d, 0x2e, 0x6f, 0x72, 0x67, + 0x2e, 0x61, 0x70, 0x61, 0x63, 0x68, 0x65, 0x2e, 0x62, 0x65, 0x61, 0x6d, 0x2e, 0x6d, 0x6f, 0x64, + 0x65, 0x6c, 0x2e, 0x70, 0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x43, + 0x6f, 0x6d, 0x70, 0x6f, 0x6e, 0x65, 0x6e, 0x74, 0x73, 0x52, 0x0a, 0x63, 0x6f, 0x6d, 0x70, 0x6f, + 0x6e, 0x65, 0x6e, 0x74, 0x73, 0x12, 0x4b, 0x0a, 0x09, 0x74, 0x72, 0x61, 0x6e, 0x73, 0x66, 0x6f, + 0x72, 0x6d, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x2d, 0x2e, 0x6f, 0x72, 0x67, 0x2e, 0x61, + 0x70, 0x61, 0x63, 0x68, 0x65, 0x2e, 0x62, 0x65, 0x61, 0x6d, 0x2e, 0x6d, 0x6f, 0x64, 0x65, 0x6c, + 0x2e, 0x70, 0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x50, 0x54, 0x72, + 0x61, 0x6e, 0x73, 0x66, 0x6f, 0x72, 0x6d, 0x52, 0x09, 0x74, 0x72, 0x61, 0x6e, 0x73, 0x66, 0x6f, + 0x72, 0x6d, 0x12, 0x22, 0x0a, 0x0c, 0x72, 0x65, 0x71, 0x75, 0x69, 0x72, 0x65, 0x6d, 0x65, 0x6e, + 0x74, 0x73, 0x18, 0x03, 0x20, 0x03, 0x28, 0x09, 0x52, 0x0c, 0x72, 0x65, 0x71, 0x75, 0x69, 0x72, + 0x65, 0x6d, 0x65, 0x6e, 0x74, 0x73, 0x12, 0x14, 0x0a, 0x05, 0x65, 0x72, 0x72, 0x6f, 0x72, 0x18, + 0x0a, 0x20, 0x01, 0x28, 0x09, 0x52, 0x05, 0x65, 0x72, 0x72, 0x6f, 0x72, 0x22, 0x20, 0x0a, 0x1e, + 0x44, 0x69, 0x73, 0x63, 0x6f, 0x76, 0x65, 0x72, 0x53, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x54, 0x72, + 0x61, 0x6e, 0x73, 0x66, 0x6f, 0x72, 0x6d, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x22, 0xfb, + 0x01, 0x0a, 0x15, 0x53, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x54, 0x72, 0x61, 0x6e, 0x73, 0x66, 0x6f, + 0x72, 0x6d, 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x12, 0x4e, 0x0a, 0x0d, 0x63, 0x6f, 0x6e, 0x66, + 0x69, 0x67, 0x5f, 0x73, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, + 0x29, 0x2e, 0x6f, 0x72, 0x67, 0x2e, 0x61, 0x70, 0x61, 0x63, 0x68, 0x65, 0x2e, 0x62, 0x65, 0x61, + 0x6d, 0x2e, 0x6d, 0x6f, 0x64, 0x65, 0x6c, 0x2e, 0x70, 0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e, 0x65, + 0x2e, 0x76, 0x31, 0x2e, 0x53, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x52, 0x0c, 0x63, 0x6f, 0x6e, 0x66, + 0x69, 0x67, 0x53, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x12, 0x36, 0x0a, 0x17, 0x69, 0x6e, 0x70, 0x75, + 0x74, 0x5f, 0x70, 0x63, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x6e, 0x61, + 0x6d, 0x65, 0x73, 0x18, 0x02, 0x20, 0x03, 0x28, 0x09, 0x52, 0x15, 0x69, 0x6e, 0x70, 0x75, 0x74, + 0x50, 0x63, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x4e, 0x61, 0x6d, 0x65, 0x73, + 0x12, 0x38, 0x0a, 0x18, 0x6f, 0x75, 0x74, 0x70, 0x75, 0x74, 0x5f, 0x70, 0x63, 0x6f, 0x6c, 0x6c, + 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x6e, 0x61, 0x6d, 0x65, 0x73, 0x18, 0x03, 0x20, 0x03, + 0x28, 0x09, 0x52, 0x16, 0x6f, 0x75, 0x74, 0x70, 0x75, 0x74, 0x50, 0x63, 0x6f, 0x6c, 0x6c, 0x65, + 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x4e, 0x61, 0x6d, 0x65, 0x73, 0x12, 0x20, 0x0a, 0x0b, 0x64, 0x65, + 0x73, 0x63, 0x72, 0x69, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x18, 0x04, 0x20, 0x01, 0x28, 0x09, 0x52, + 0x0b, 0x64, 0x65, 0x73, 0x63, 0x72, 0x69, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x22, 0xda, 0x02, 0x0a, + 0x1f, 0x44, 0x69, 0x73, 0x63, 0x6f, 0x76, 0x65, 0x72, 0x53, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x54, + 0x72, 0x61, 0x6e, 0x73, 0x66, 0x6f, 0x72, 0x6d, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, + 0x12, 0x99, 0x01, 0x0a, 0x18, 0x73, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x5f, 0x74, 0x72, 0x61, 0x6e, + 0x73, 0x66, 0x6f, 0x72, 0x6d, 0x5f, 0x63, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x73, 0x18, 0x01, 0x20, + 0x03, 0x28, 0x0b, 0x32, 0x5f, 0x2e, 0x6f, 0x72, 0x67, 0x2e, 0x61, 0x70, 0x61, 0x63, 0x68, 0x65, + 0x2e, 0x62, 0x65, 0x61, 0x6d, 0x2e, 0x6d, 0x6f, 0x64, 0x65, 0x6c, 0x2e, 0x65, 0x78, 0x70, 0x61, + 0x6e, 0x73, 0x69, 0x6f, 0x6e, 0x2e, 0x76, 0x31, 0x2e, 0x44, 0x69, 0x73, 0x63, 0x6f, 0x76, 0x65, + 0x72, 0x53, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x54, 0x72, 0x61, 0x6e, 0x73, 0x66, 0x6f, 0x72, 0x6d, + 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x2e, 0x53, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x54, + 0x72, 0x61, 0x6e, 0x73, 0x66, 0x6f, 0x72, 0x6d, 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x73, 0x45, + 0x6e, 0x74, 0x72, 0x79, 0x52, 0x16, 0x73, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x54, 0x72, 0x61, 0x6e, + 0x73, 0x66, 0x6f, 0x72, 0x6d, 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x73, 0x12, 0x14, 0x0a, 0x05, + 0x65, 0x72, 0x72, 0x6f, 0x72, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x05, 0x65, 0x72, 0x72, + 0x6f, 0x72, 0x1a, 0x84, 0x01, 0x0a, 0x1b, 0x53, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x54, 0x72, 0x61, + 0x6e, 0x73, 0x66, 0x6f, 0x72, 0x6d, 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x73, 0x45, 0x6e, 0x74, + 0x72, 0x79, 0x12, 0x10, 0x0a, 0x03, 0x6b, 0x65, 0x79, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, + 0x03, 0x6b, 0x65, 0x79, 0x12, 0x4f, 0x0a, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x18, 0x02, 0x20, + 0x01, 0x28, 0x0b, 0x32, 0x39, 0x2e, 0x6f, 0x72, 0x67, 0x2e, 0x61, 0x70, 0x61, 0x63, 0x68, 0x65, + 0x2e, 0x62, 0x65, 0x61, 0x6d, 0x2e, 0x6d, 0x6f, 0x64, 0x65, 0x6c, 0x2e, 0x65, 0x78, 0x70, 0x61, + 0x6e, 0x73, 0x69, 0x6f, 0x6e, 0x2e, 0x76, 0x31, 0x2e, 0x53, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x54, + 0x72, 0x61, 0x6e, 0x73, 0x66, 0x6f, 0x72, 0x6d, 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x52, 0x05, + 0x76, 0x61, 0x6c, 0x75, 0x65, 0x3a, 0x02, 0x38, 0x01, 0x32, 0xae, 0x02, 0x0a, 0x10, 0x45, 0x78, + 0x70, 0x61, 0x6e, 0x73, 0x69, 0x6f, 0x6e, 0x53, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, 0x12, 0x75, + 0x0a, 0x06, 0x45, 0x78, 0x70, 0x61, 0x6e, 0x64, 0x12, 0x34, 0x2e, 0x6f, 0x72, 0x67, 0x2e, 0x61, + 0x70, 0x61, 0x63, 0x68, 0x65, 0x2e, 0x62, 0x65, 0x61, 0x6d, 0x2e, 0x6d, 0x6f, 0x64, 0x65, 0x6c, + 0x2e, 0x65, 0x78, 0x70, 0x61, 0x6e, 0x73, 0x69, 0x6f, 0x6e, 0x2e, 0x76, 0x31, 0x2e, 0x45, 0x78, + 0x70, 0x61, 0x6e, 0x73, 0x69, 0x6f, 0x6e, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x35, + 0x2e, 0x6f, 0x72, 0x67, 0x2e, 0x61, 0x70, 0x61, 0x63, 0x68, 0x65, 0x2e, 0x62, 0x65, 0x61, 0x6d, + 0x2e, 0x6d, 0x6f, 0x64, 0x65, 0x6c, 0x2e, 0x65, 0x78, 0x70, 0x61, 0x6e, 0x73, 0x69, 0x6f, 0x6e, + 0x2e, 0x76, 0x31, 0x2e, 0x45, 0x78, 0x70, 0x61, 0x6e, 0x73, 0x69, 0x6f, 0x6e, 0x52, 0x65, 0x73, + 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0xa2, 0x01, 0x0a, 0x17, 0x44, 0x69, 0x73, 0x63, 0x6f, 0x76, + 0x65, 0x72, 0x53, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x54, 0x72, 0x61, 0x6e, 0x73, 0x66, 0x6f, 0x72, + 0x6d, 0x12, 0x42, 0x2e, 0x6f, 0x72, 0x67, 0x2e, 0x61, 0x70, 0x61, 0x63, 0x68, 0x65, 0x2e, 0x62, + 0x65, 0x61, 0x6d, 0x2e, 0x6d, 0x6f, 0x64, 0x65, 0x6c, 0x2e, 0x65, 0x78, 0x70, 0x61, 0x6e, 0x73, + 0x69, 0x6f, 0x6e, 0x2e, 0x76, 0x31, 0x2e, 0x44, 0x69, 0x73, 0x63, 0x6f, 0x76, 0x65, 0x72, 0x53, + 0x63, 0x68, 0x65, 0x6d, 0x61, 0x54, 0x72, 0x61, 0x6e, 0x73, 0x66, 0x6f, 0x72, 0x6d, 0x52, 0x65, + 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x43, 0x2e, 0x6f, 0x72, 0x67, 0x2e, 0x61, 0x70, 0x61, 0x63, + 0x68, 0x65, 0x2e, 0x62, 0x65, 0x61, 0x6d, 0x2e, 0x6d, 0x6f, 0x64, 0x65, 0x6c, 0x2e, 0x65, 0x78, + 0x70, 0x61, 0x6e, 0x73, 0x69, 0x6f, 0x6e, 0x2e, 0x76, 0x31, 0x2e, 0x44, 0x69, 0x73, 0x63, 0x6f, + 0x76, 0x65, 0x72, 0x53, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x54, 0x72, 0x61, 0x6e, 0x73, 0x66, 0x6f, + 0x72, 0x6d, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x42, 0x86, 0x01, 0x0a, 0x22, 0x6f, + 0x72, 0x67, 0x2e, 0x61, 0x70, 0x61, 0x63, 0x68, 0x65, 0x2e, 0x62, 0x65, 0x61, 0x6d, 0x2e, 0x6d, + 0x6f, 0x64, 0x65, 0x6c, 0x2e, 0x65, 0x78, 0x70, 0x61, 0x6e, 0x73, 0x69, 0x6f, 0x6e, 0x2e, 0x76, + 0x31, 0x42, 0x0c, 0x45, 0x78, 0x70, 0x61, 0x6e, 0x73, 0x69, 0x6f, 0x6e, 0x41, 0x70, 0x69, 0x5a, + 0x52, 0x67, 0x69, 0x74, 0x68, 0x75, 0x62, 0x2e, 0x63, 0x6f, 0x6d, 0x2f, 0x61, 0x70, 0x61, 0x63, + 0x68, 0x65, 0x2f, 0x62, 0x65, 0x61, 0x6d, 0x2f, 0x73, 0x64, 0x6b, 0x73, 0x2f, 0x76, 0x32, 0x2f, + 0x67, 0x6f, 0x2f, 0x70, 0x6b, 0x67, 0x2f, 0x62, 0x65, 0x61, 0x6d, 0x2f, 0x6d, 0x6f, 0x64, 0x65, + 0x6c, 0x2f, 0x6a, 0x6f, 0x62, 0x6d, 0x61, 0x6e, 0x61, 0x67, 0x65, 0x6d, 0x65, 0x6e, 0x74, 0x5f, + 0x76, 0x31, 0x3b, 0x6a, 0x6f, 0x62, 0x6d, 0x61, 0x6e, 0x61, 0x67, 0x65, 0x6d, 0x65, 0x6e, 0x74, + 0x5f, 0x76, 0x31, 0x62, 0x06, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x33, +} + +var file_org_apache_beam_model_job_management_v1_beam_expansion_api_proto_msgTypes = make([]protoimpl.MessageInfo, 7) +var file_org_apache_beam_model_job_management_v1_beam_expansion_api_proto_goTypes = []any{ + (*ExpansionRequest)(nil), // 0: org.apache.beam.model.expansion.v1.ExpansionRequest + (*ExpansionResponse)(nil), // 1: org.apache.beam.model.expansion.v1.ExpansionResponse + (*DiscoverSchemaTransformRequest)(nil), // 2: org.apache.beam.model.expansion.v1.DiscoverSchemaTransformRequest + (*SchemaTransformConfig)(nil), // 3: org.apache.beam.model.expansion.v1.SchemaTransformConfig + (*DiscoverSchemaTransformResponse)(nil), // 4: org.apache.beam.model.expansion.v1.DiscoverSchemaTransformResponse + nil, // 5: org.apache.beam.model.expansion.v1.ExpansionRequest.OutputCoderRequestsEntry + nil, // 6: org.apache.beam.model.expansion.v1.DiscoverSchemaTransformResponse.SchemaTransformConfigsEntry + (*pipeline_v1.Components)(nil), // 7: org.apache.beam.model.pipeline.v1.Components + (*pipeline_v1.PTransform)(nil), // 8: org.apache.beam.model.pipeline.v1.PTransform + (*structpb.Struct)(nil), // 9: google.protobuf.Struct + (*pipeline_v1.Schema)(nil), // 10: org.apache.beam.model.pipeline.v1.Schema +} +var file_org_apache_beam_model_job_management_v1_beam_expansion_api_proto_depIdxs = []int32{ + 7, // 0: org.apache.beam.model.expansion.v1.ExpansionRequest.components:type_name -> org.apache.beam.model.pipeline.v1.Components + 8, // 1: org.apache.beam.model.expansion.v1.ExpansionRequest.transform:type_name -> org.apache.beam.model.pipeline.v1.PTransform + 5, // 2: org.apache.beam.model.expansion.v1.ExpansionRequest.output_coder_requests:type_name -> org.apache.beam.model.expansion.v1.ExpansionRequest.OutputCoderRequestsEntry + 9, // 3: org.apache.beam.model.expansion.v1.ExpansionRequest.pipeline_options:type_name -> google.protobuf.Struct + 7, // 4: org.apache.beam.model.expansion.v1.ExpansionResponse.components:type_name -> org.apache.beam.model.pipeline.v1.Components + 8, // 5: org.apache.beam.model.expansion.v1.ExpansionResponse.transform:type_name -> org.apache.beam.model.pipeline.v1.PTransform + 10, // 6: org.apache.beam.model.expansion.v1.SchemaTransformConfig.config_schema:type_name -> org.apache.beam.model.pipeline.v1.Schema + 6, // 7: org.apache.beam.model.expansion.v1.DiscoverSchemaTransformResponse.schema_transform_configs:type_name -> org.apache.beam.model.expansion.v1.DiscoverSchemaTransformResponse.SchemaTransformConfigsEntry + 3, // 8: org.apache.beam.model.expansion.v1.DiscoverSchemaTransformResponse.SchemaTransformConfigsEntry.value:type_name -> org.apache.beam.model.expansion.v1.SchemaTransformConfig + 0, // 9: org.apache.beam.model.expansion.v1.ExpansionService.Expand:input_type -> org.apache.beam.model.expansion.v1.ExpansionRequest + 2, // 10: org.apache.beam.model.expansion.v1.ExpansionService.DiscoverSchemaTransform:input_type -> org.apache.beam.model.expansion.v1.DiscoverSchemaTransformRequest + 1, // 11: org.apache.beam.model.expansion.v1.ExpansionService.Expand:output_type -> org.apache.beam.model.expansion.v1.ExpansionResponse + 4, // 12: org.apache.beam.model.expansion.v1.ExpansionService.DiscoverSchemaTransform:output_type -> org.apache.beam.model.expansion.v1.DiscoverSchemaTransformResponse + 11, // [11:13] is the sub-list for method output_type + 9, // [9:11] is the sub-list for method input_type + 9, // [9:9] is the sub-list for extension type_name + 9, // [9:9] is the sub-list for extension extendee + 0, // [0:9] is the sub-list for field type_name +} + +func init() { file_org_apache_beam_model_job_management_v1_beam_expansion_api_proto_init() } +func file_org_apache_beam_model_job_management_v1_beam_expansion_api_proto_init() { + if File_org_apache_beam_model_job_management_v1_beam_expansion_api_proto != nil { + return + } + type x struct{} + out := protoimpl.TypeBuilder{ + File: protoimpl.DescBuilder{ + GoPackagePath: reflect.TypeOf(x{}).PkgPath(), + RawDescriptor: file_org_apache_beam_model_job_management_v1_beam_expansion_api_proto_rawDesc, + NumEnums: 0, + NumMessages: 7, + NumExtensions: 0, + NumServices: 1, + }, + GoTypes: file_org_apache_beam_model_job_management_v1_beam_expansion_api_proto_goTypes, + DependencyIndexes: file_org_apache_beam_model_job_management_v1_beam_expansion_api_proto_depIdxs, + MessageInfos: file_org_apache_beam_model_job_management_v1_beam_expansion_api_proto_msgTypes, + }.Build() + File_org_apache_beam_model_job_management_v1_beam_expansion_api_proto = out.File + file_org_apache_beam_model_job_management_v1_beam_expansion_api_proto_rawDesc = nil + file_org_apache_beam_model_job_management_v1_beam_expansion_api_proto_goTypes = nil + file_org_apache_beam_model_job_management_v1_beam_expansion_api_proto_depIdxs = nil +} diff --git a/sdks/go/pkg/beam/model/jobmanagement_v1/beam_job_api.pb.go b/sdks/go/pkg/beam/model/jobmanagement_v1/beam_job_api.pb.go index 395c520e10b0..86b816699ad0 100644 --- a/sdks/go/pkg/beam/model/jobmanagement_v1/beam_job_api.pb.go +++ b/sdks/go/pkg/beam/model/jobmanagement_v1/beam_job_api.pb.go @@ -21,10 +21,12 @@ // Code generated by protoc-gen-go. DO NOT EDIT. // versions: -// protoc-gen-go v1.27.1 +// protoc-gen-go v1.36.0 // protoc v5.27.3 // source: org/apache/beam/model/job_management/v1/beam_job_api.proto +//go:build !protoopaque + package jobmanagement_v1 import ( @@ -34,7 +36,6 @@ import ( structpb "google.golang.org/protobuf/types/known/structpb" timestamppb "google.golang.org/protobuf/types/known/timestamppb" reflect "reflect" - sync "sync" ) const ( @@ -97,11 +98,6 @@ func (x JobMessage_MessageImportance) Number() protoreflect.EnumNumber { return protoreflect.EnumNumber(x) } -// Deprecated: Use JobMessage_MessageImportance.Descriptor instead. -func (JobMessage_MessageImportance) EnumDescriptor() ([]byte, []int) { - return file_org_apache_beam_model_job_management_v1_beam_job_api_proto_rawDescGZIP(), []int{16, 0} -} - type JobState_Enum int32 const ( @@ -185,11 +181,6 @@ func (x JobState_Enum) Number() protoreflect.EnumNumber { return protoreflect.EnumNumber(x) } -// Deprecated: Use JobState_Enum.Descriptor instead. -func (JobState_Enum) EnumDescriptor() ([]byte, []int) { - return file_org_apache_beam_model_job_management_v1_beam_job_api_proto_rawDescGZIP(), []int{18, 0} -} - type PipelineOptionType_Enum int32 const ( @@ -244,32 +235,24 @@ func (x PipelineOptionType_Enum) Number() protoreflect.EnumNumber { return protoreflect.EnumNumber(x) } -// Deprecated: Use PipelineOptionType_Enum.Descriptor instead. -func (PipelineOptionType_Enum) EnumDescriptor() ([]byte, []int) { - return file_org_apache_beam_model_job_management_v1_beam_job_api_proto_rawDescGZIP(), []int{23, 0} -} - // Prepare is a synchronous request that returns a preparationId back // Throws error GRPC_STATUS_UNAVAILABLE if server is down // Throws error ALREADY_EXISTS if the jobName is reused. Runners are permitted to deduplicate based on the name of the job. // Throws error UNKNOWN for all other issues type PrepareJobRequest struct { - state protoimpl.MessageState - sizeCache protoimpl.SizeCache - unknownFields protoimpl.UnknownFields - - Pipeline *pipeline_v1.Pipeline `protobuf:"bytes,1,opt,name=pipeline,proto3" json:"pipeline,omitempty"` // (required) - PipelineOptions *structpb.Struct `protobuf:"bytes,2,opt,name=pipeline_options,json=pipelineOptions,proto3" json:"pipeline_options,omitempty"` // (required) - JobName string `protobuf:"bytes,3,opt,name=job_name,json=jobName,proto3" json:"job_name,omitempty"` // (required) + state protoimpl.MessageState `protogen:"hybrid.v1"` + Pipeline *pipeline_v1.Pipeline `protobuf:"bytes,1,opt,name=pipeline,proto3" json:"pipeline,omitempty"` // (required) + PipelineOptions *structpb.Struct `protobuf:"bytes,2,opt,name=pipeline_options,json=pipelineOptions,proto3" json:"pipeline_options,omitempty"` // (required) + JobName string `protobuf:"bytes,3,opt,name=job_name,json=jobName,proto3" json:"job_name,omitempty"` // (required) + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache } func (x *PrepareJobRequest) Reset() { *x = PrepareJobRequest{} - if protoimpl.UnsafeEnabled { - mi := &file_org_apache_beam_model_job_management_v1_beam_job_api_proto_msgTypes[0] - ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) - ms.StoreMessageInfo(mi) - } + mi := &file_org_apache_beam_model_job_management_v1_beam_job_api_proto_msgTypes[0] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) } func (x *PrepareJobRequest) String() string { @@ -280,7 +263,7 @@ func (*PrepareJobRequest) ProtoMessage() {} func (x *PrepareJobRequest) ProtoReflect() protoreflect.Message { mi := &file_org_apache_beam_model_job_management_v1_beam_job_api_proto_msgTypes[0] - if protoimpl.UnsafeEnabled && x != nil { + if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { ms.StoreMessageInfo(mi) @@ -290,11 +273,6 @@ func (x *PrepareJobRequest) ProtoReflect() protoreflect.Message { return mi.MessageOf(x) } -// Deprecated: Use PrepareJobRequest.ProtoReflect.Descriptor instead. -func (*PrepareJobRequest) Descriptor() ([]byte, []int) { - return file_org_apache_beam_model_job_management_v1_beam_job_api_proto_rawDescGZIP(), []int{0} -} - func (x *PrepareJobRequest) GetPipeline() *pipeline_v1.Pipeline { if x != nil { return x.Pipeline @@ -316,11 +294,60 @@ func (x *PrepareJobRequest) GetJobName() string { return "" } -type PrepareJobResponse struct { - state protoimpl.MessageState - sizeCache protoimpl.SizeCache - unknownFields protoimpl.UnknownFields +func (x *PrepareJobRequest) SetPipeline(v *pipeline_v1.Pipeline) { + x.Pipeline = v +} + +func (x *PrepareJobRequest) SetPipelineOptions(v *structpb.Struct) { + x.PipelineOptions = v +} + +func (x *PrepareJobRequest) SetJobName(v string) { + x.JobName = v +} + +func (x *PrepareJobRequest) HasPipeline() bool { + if x == nil { + return false + } + return x.Pipeline != nil +} + +func (x *PrepareJobRequest) HasPipelineOptions() bool { + if x == nil { + return false + } + return x.PipelineOptions != nil +} + +func (x *PrepareJobRequest) ClearPipeline() { + x.Pipeline = nil +} + +func (x *PrepareJobRequest) ClearPipelineOptions() { + x.PipelineOptions = nil +} + +type PrepareJobRequest_builder struct { + _ [0]func() // Prevents comparability and use of unkeyed literals for the builder. + + Pipeline *pipeline_v1.Pipeline + PipelineOptions *structpb.Struct + JobName string +} + +func (b0 PrepareJobRequest_builder) Build() *PrepareJobRequest { + m0 := &PrepareJobRequest{} + b, x := &b0, m0 + _, _ = b, x + x.Pipeline = b.Pipeline + x.PipelineOptions = b.PipelineOptions + x.JobName = b.JobName + return m0 +} +type PrepareJobResponse struct { + state protoimpl.MessageState `protogen:"hybrid.v1"` // (required) The ID used to associate calls made while preparing the job. preparationId is used // to run the job. PreparationId string `protobuf:"bytes,1,opt,name=preparation_id,json=preparationId,proto3" json:"preparation_id,omitempty"` @@ -330,15 +357,15 @@ type PrepareJobResponse struct { // (required) Token for the artifact staging. This token also represent an artifact // staging session with the artifact staging service. StagingSessionToken string `protobuf:"bytes,3,opt,name=staging_session_token,json=stagingSessionToken,proto3" json:"staging_session_token,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache } func (x *PrepareJobResponse) Reset() { *x = PrepareJobResponse{} - if protoimpl.UnsafeEnabled { - mi := &file_org_apache_beam_model_job_management_v1_beam_job_api_proto_msgTypes[1] - ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) - ms.StoreMessageInfo(mi) - } + mi := &file_org_apache_beam_model_job_management_v1_beam_job_api_proto_msgTypes[1] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) } func (x *PrepareJobResponse) String() string { @@ -349,7 +376,7 @@ func (*PrepareJobResponse) ProtoMessage() {} func (x *PrepareJobResponse) ProtoReflect() protoreflect.Message { mi := &file_org_apache_beam_model_job_management_v1_beam_job_api_proto_msgTypes[1] - if protoimpl.UnsafeEnabled && x != nil { + if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { ms.StoreMessageInfo(mi) @@ -359,11 +386,6 @@ func (x *PrepareJobResponse) ProtoReflect() protoreflect.Message { return mi.MessageOf(x) } -// Deprecated: Use PrepareJobResponse.ProtoReflect.Descriptor instead. -func (*PrepareJobResponse) Descriptor() ([]byte, []int) { - return file_org_apache_beam_model_job_management_v1_beam_job_api_proto_rawDescGZIP(), []int{1} -} - func (x *PrepareJobResponse) GetPreparationId() string { if x != nil { return x.PreparationId @@ -385,30 +407,74 @@ func (x *PrepareJobResponse) GetStagingSessionToken() string { return "" } +func (x *PrepareJobResponse) SetPreparationId(v string) { + x.PreparationId = v +} + +func (x *PrepareJobResponse) SetArtifactStagingEndpoint(v *pipeline_v1.ApiServiceDescriptor) { + x.ArtifactStagingEndpoint = v +} + +func (x *PrepareJobResponse) SetStagingSessionToken(v string) { + x.StagingSessionToken = v +} + +func (x *PrepareJobResponse) HasArtifactStagingEndpoint() bool { + if x == nil { + return false + } + return x.ArtifactStagingEndpoint != nil +} + +func (x *PrepareJobResponse) ClearArtifactStagingEndpoint() { + x.ArtifactStagingEndpoint = nil +} + +type PrepareJobResponse_builder struct { + _ [0]func() // Prevents comparability and use of unkeyed literals for the builder. + + // (required) The ID used to associate calls made while preparing the job. preparationId is used + // to run the job. + PreparationId string + // An endpoint which exposes the Beam Artifact Staging API. Artifacts used by the job should be + // staged to this endpoint, and will be available during job execution. + ArtifactStagingEndpoint *pipeline_v1.ApiServiceDescriptor + // (required) Token for the artifact staging. This token also represent an artifact + // staging session with the artifact staging service. + StagingSessionToken string +} + +func (b0 PrepareJobResponse_builder) Build() *PrepareJobResponse { + m0 := &PrepareJobResponse{} + b, x := &b0, m0 + _, _ = b, x + x.PreparationId = b.PreparationId + x.ArtifactStagingEndpoint = b.ArtifactStagingEndpoint + x.StagingSessionToken = b.StagingSessionToken + return m0 +} + // Run is a synchronous request that returns a jobId back. // Throws error GRPC_STATUS_UNAVAILABLE if server is down // Throws error NOT_FOUND if the preparation ID does not exist // Throws error UNKNOWN for all other issues type RunJobRequest struct { - state protoimpl.MessageState - sizeCache protoimpl.SizeCache - unknownFields protoimpl.UnknownFields - + state protoimpl.MessageState `protogen:"hybrid.v1"` // (required) The ID provided by an earlier call to prepare. Runs the job. All prerequisite tasks // must have been completed. PreparationId string `protobuf:"bytes,1,opt,name=preparation_id,json=preparationId,proto3" json:"preparation_id,omitempty"` // (optional) If any artifacts have been staged for this job, contains the retrieval_token returned // from the CommitManifestResponse. RetrievalToken string `protobuf:"bytes,2,opt,name=retrieval_token,json=retrievalToken,proto3" json:"retrieval_token,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache } func (x *RunJobRequest) Reset() { *x = RunJobRequest{} - if protoimpl.UnsafeEnabled { - mi := &file_org_apache_beam_model_job_management_v1_beam_job_api_proto_msgTypes[2] - ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) - ms.StoreMessageInfo(mi) - } + mi := &file_org_apache_beam_model_job_management_v1_beam_job_api_proto_msgTypes[2] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) } func (x *RunJobRequest) String() string { @@ -419,7 +485,7 @@ func (*RunJobRequest) ProtoMessage() {} func (x *RunJobRequest) ProtoReflect() protoreflect.Message { mi := &file_org_apache_beam_model_job_management_v1_beam_job_api_proto_msgTypes[2] - if protoimpl.UnsafeEnabled && x != nil { + if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { ms.StoreMessageInfo(mi) @@ -429,11 +495,6 @@ func (x *RunJobRequest) ProtoReflect() protoreflect.Message { return mi.MessageOf(x) } -// Deprecated: Use RunJobRequest.ProtoReflect.Descriptor instead. -func (*RunJobRequest) Descriptor() ([]byte, []int) { - return file_org_apache_beam_model_job_management_v1_beam_job_api_proto_rawDescGZIP(), []int{2} -} - func (x *RunJobRequest) GetPreparationId() string { if x != nil { return x.PreparationId @@ -448,21 +509,46 @@ func (x *RunJobRequest) GetRetrievalToken() string { return "" } +func (x *RunJobRequest) SetPreparationId(v string) { + x.PreparationId = v +} + +func (x *RunJobRequest) SetRetrievalToken(v string) { + x.RetrievalToken = v +} + +type RunJobRequest_builder struct { + _ [0]func() // Prevents comparability and use of unkeyed literals for the builder. + + // (required) The ID provided by an earlier call to prepare. Runs the job. All prerequisite tasks + // must have been completed. + PreparationId string + // (optional) If any artifacts have been staged for this job, contains the retrieval_token returned + // from the CommitManifestResponse. + RetrievalToken string +} + +func (b0 RunJobRequest_builder) Build() *RunJobRequest { + m0 := &RunJobRequest{} + b, x := &b0, m0 + _, _ = b, x + x.PreparationId = b.PreparationId + x.RetrievalToken = b.RetrievalToken + return m0 +} + type RunJobResponse struct { - state protoimpl.MessageState - sizeCache protoimpl.SizeCache + state protoimpl.MessageState `protogen:"hybrid.v1"` + JobId string `protobuf:"bytes,1,opt,name=job_id,json=jobId,proto3" json:"job_id,omitempty"` // (required) The ID for the executing job unknownFields protoimpl.UnknownFields - - JobId string `protobuf:"bytes,1,opt,name=job_id,json=jobId,proto3" json:"job_id,omitempty"` // (required) The ID for the executing job + sizeCache protoimpl.SizeCache } func (x *RunJobResponse) Reset() { *x = RunJobResponse{} - if protoimpl.UnsafeEnabled { - mi := &file_org_apache_beam_model_job_management_v1_beam_job_api_proto_msgTypes[3] - ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) - ms.StoreMessageInfo(mi) - } + mi := &file_org_apache_beam_model_job_management_v1_beam_job_api_proto_msgTypes[3] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) } func (x *RunJobResponse) String() string { @@ -473,7 +559,7 @@ func (*RunJobResponse) ProtoMessage() {} func (x *RunJobResponse) ProtoReflect() protoreflect.Message { mi := &file_org_apache_beam_model_job_management_v1_beam_job_api_proto_msgTypes[3] - if protoimpl.UnsafeEnabled && x != nil { + if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { ms.StoreMessageInfo(mi) @@ -483,11 +569,6 @@ func (x *RunJobResponse) ProtoReflect() protoreflect.Message { return mi.MessageOf(x) } -// Deprecated: Use RunJobResponse.ProtoReflect.Descriptor instead. -func (*RunJobResponse) Descriptor() ([]byte, []int) { - return file_org_apache_beam_model_job_management_v1_beam_job_api_proto_rawDescGZIP(), []int{3} -} - func (x *RunJobResponse) GetJobId() string { if x != nil { return x.JobId @@ -495,24 +576,39 @@ func (x *RunJobResponse) GetJobId() string { return "" } +func (x *RunJobResponse) SetJobId(v string) { + x.JobId = v +} + +type RunJobResponse_builder struct { + _ [0]func() // Prevents comparability and use of unkeyed literals for the builder. + + JobId string +} + +func (b0 RunJobResponse_builder) Build() *RunJobResponse { + m0 := &RunJobResponse{} + b, x := &b0, m0 + _, _ = b, x + x.JobId = b.JobId + return m0 +} + // Cancel is a synchronus request that returns a job state back // Throws error GRPC_STATUS_UNAVAILABLE if server is down // Throws error NOT_FOUND if the jobId is not found type CancelJobRequest struct { - state protoimpl.MessageState - sizeCache protoimpl.SizeCache + state protoimpl.MessageState `protogen:"hybrid.v1"` + JobId string `protobuf:"bytes,1,opt,name=job_id,json=jobId,proto3" json:"job_id,omitempty"` // (required) unknownFields protoimpl.UnknownFields - - JobId string `protobuf:"bytes,1,opt,name=job_id,json=jobId,proto3" json:"job_id,omitempty"` // (required) + sizeCache protoimpl.SizeCache } func (x *CancelJobRequest) Reset() { *x = CancelJobRequest{} - if protoimpl.UnsafeEnabled { - mi := &file_org_apache_beam_model_job_management_v1_beam_job_api_proto_msgTypes[4] - ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) - ms.StoreMessageInfo(mi) - } + mi := &file_org_apache_beam_model_job_management_v1_beam_job_api_proto_msgTypes[4] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) } func (x *CancelJobRequest) String() string { @@ -523,7 +619,7 @@ func (*CancelJobRequest) ProtoMessage() {} func (x *CancelJobRequest) ProtoReflect() protoreflect.Message { mi := &file_org_apache_beam_model_job_management_v1_beam_job_api_proto_msgTypes[4] - if protoimpl.UnsafeEnabled && x != nil { + if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { ms.StoreMessageInfo(mi) @@ -533,11 +629,6 @@ func (x *CancelJobRequest) ProtoReflect() protoreflect.Message { return mi.MessageOf(x) } -// Deprecated: Use CancelJobRequest.ProtoReflect.Descriptor instead. -func (*CancelJobRequest) Descriptor() ([]byte, []int) { - return file_org_apache_beam_model_job_management_v1_beam_job_api_proto_rawDescGZIP(), []int{4} -} - func (x *CancelJobRequest) GetJobId() string { if x != nil { return x.JobId @@ -545,22 +636,37 @@ func (x *CancelJobRequest) GetJobId() string { return "" } +func (x *CancelJobRequest) SetJobId(v string) { + x.JobId = v +} + +type CancelJobRequest_builder struct { + _ [0]func() // Prevents comparability and use of unkeyed literals for the builder. + + JobId string +} + +func (b0 CancelJobRequest_builder) Build() *CancelJobRequest { + m0 := &CancelJobRequest{} + b, x := &b0, m0 + _, _ = b, x + x.JobId = b.JobId + return m0 +} + // Valid responses include any terminal state or CANCELLING type CancelJobResponse struct { - state protoimpl.MessageState - sizeCache protoimpl.SizeCache + state protoimpl.MessageState `protogen:"hybrid.v1"` + State JobState_Enum `protobuf:"varint,1,opt,name=state,proto3,enum=org.apache.beam.model.job_management.v1.JobState_Enum" json:"state,omitempty"` // (required) unknownFields protoimpl.UnknownFields - - State JobState_Enum `protobuf:"varint,1,opt,name=state,proto3,enum=org.apache.beam.model.job_management.v1.JobState_Enum" json:"state,omitempty"` // (required) + sizeCache protoimpl.SizeCache } func (x *CancelJobResponse) Reset() { *x = CancelJobResponse{} - if protoimpl.UnsafeEnabled { - mi := &file_org_apache_beam_model_job_management_v1_beam_job_api_proto_msgTypes[5] - ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) - ms.StoreMessageInfo(mi) - } + mi := &file_org_apache_beam_model_job_management_v1_beam_job_api_proto_msgTypes[5] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) } func (x *CancelJobResponse) String() string { @@ -571,7 +677,7 @@ func (*CancelJobResponse) ProtoMessage() {} func (x *CancelJobResponse) ProtoReflect() protoreflect.Message { mi := &file_org_apache_beam_model_job_management_v1_beam_job_api_proto_msgTypes[5] - if protoimpl.UnsafeEnabled && x != nil { + if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { ms.StoreMessageInfo(mi) @@ -581,11 +687,6 @@ func (x *CancelJobResponse) ProtoReflect() protoreflect.Message { return mi.MessageOf(x) } -// Deprecated: Use CancelJobResponse.ProtoReflect.Descriptor instead. -func (*CancelJobResponse) Descriptor() ([]byte, []int) { - return file_org_apache_beam_model_job_management_v1_beam_job_api_proto_rawDescGZIP(), []int{5} -} - func (x *CancelJobResponse) GetState() JobState_Enum { if x != nil { return x.State @@ -593,25 +694,40 @@ func (x *CancelJobResponse) GetState() JobState_Enum { return JobState_UNSPECIFIED } +func (x *CancelJobResponse) SetState(v JobState_Enum) { + x.State = v +} + +type CancelJobResponse_builder struct { + _ [0]func() // Prevents comparability and use of unkeyed literals for the builder. + + State JobState_Enum +} + +func (b0 CancelJobResponse_builder) Build() *CancelJobResponse { + m0 := &CancelJobResponse{} + b, x := &b0, m0 + _, _ = b, x + x.State = b.State + return m0 +} + // Drain is a request to: // - stop a pipeline Job from ingesting new data from input sources // - process remaining data in the pipeline // - truncate Splittable DoFns type DrainJobRequest struct { - state protoimpl.MessageState - sizeCache protoimpl.SizeCache + state protoimpl.MessageState `protogen:"hybrid.v1"` + JobId string `protobuf:"bytes,1,opt,name=job_id,json=jobId,proto3" json:"job_id,omitempty"` // (required) unknownFields protoimpl.UnknownFields - - JobId string `protobuf:"bytes,1,opt,name=job_id,json=jobId,proto3" json:"job_id,omitempty"` // (required) + sizeCache protoimpl.SizeCache } func (x *DrainJobRequest) Reset() { *x = DrainJobRequest{} - if protoimpl.UnsafeEnabled { - mi := &file_org_apache_beam_model_job_management_v1_beam_job_api_proto_msgTypes[6] - ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) - ms.StoreMessageInfo(mi) - } + mi := &file_org_apache_beam_model_job_management_v1_beam_job_api_proto_msgTypes[6] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) } func (x *DrainJobRequest) String() string { @@ -622,7 +738,7 @@ func (*DrainJobRequest) ProtoMessage() {} func (x *DrainJobRequest) ProtoReflect() protoreflect.Message { mi := &file_org_apache_beam_model_job_management_v1_beam_job_api_proto_msgTypes[6] - if protoimpl.UnsafeEnabled && x != nil { + if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { ms.StoreMessageInfo(mi) @@ -632,11 +748,6 @@ func (x *DrainJobRequest) ProtoReflect() protoreflect.Message { return mi.MessageOf(x) } -// Deprecated: Use DrainJobRequest.ProtoReflect.Descriptor instead. -func (*DrainJobRequest) Descriptor() ([]byte, []int) { - return file_org_apache_beam_model_job_management_v1_beam_job_api_proto_rawDescGZIP(), []int{6} -} - func (x *DrainJobRequest) GetJobId() string { if x != nil { return x.JobId @@ -644,22 +755,37 @@ func (x *DrainJobRequest) GetJobId() string { return "" } +func (x *DrainJobRequest) SetJobId(v string) { + x.JobId = v +} + +type DrainJobRequest_builder struct { + _ [0]func() // Prevents comparability and use of unkeyed literals for the builder. + + JobId string +} + +func (b0 DrainJobRequest_builder) Build() *DrainJobRequest { + m0 := &DrainJobRequest{} + b, x := &b0, m0 + _, _ = b, x + x.JobId = b.JobId + return m0 +} + // Valid responses include any terminal state or DRAINING type DrainJobResponse struct { - state protoimpl.MessageState - sizeCache protoimpl.SizeCache + state protoimpl.MessageState `protogen:"hybrid.v1"` + State JobState_Enum `protobuf:"varint,1,opt,name=state,proto3,enum=org.apache.beam.model.job_management.v1.JobState_Enum" json:"state,omitempty"` // (required) unknownFields protoimpl.UnknownFields - - State JobState_Enum `protobuf:"varint,1,opt,name=state,proto3,enum=org.apache.beam.model.job_management.v1.JobState_Enum" json:"state,omitempty"` // (required) + sizeCache protoimpl.SizeCache } func (x *DrainJobResponse) Reset() { *x = DrainJobResponse{} - if protoimpl.UnsafeEnabled { - mi := &file_org_apache_beam_model_job_management_v1_beam_job_api_proto_msgTypes[7] - ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) - ms.StoreMessageInfo(mi) - } + mi := &file_org_apache_beam_model_job_management_v1_beam_job_api_proto_msgTypes[7] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) } func (x *DrainJobResponse) String() string { @@ -670,7 +796,7 @@ func (*DrainJobResponse) ProtoMessage() {} func (x *DrainJobResponse) ProtoReflect() protoreflect.Message { mi := &file_org_apache_beam_model_job_management_v1_beam_job_api_proto_msgTypes[7] - if protoimpl.UnsafeEnabled && x != nil { + if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { ms.StoreMessageInfo(mi) @@ -680,11 +806,6 @@ func (x *DrainJobResponse) ProtoReflect() protoreflect.Message { return mi.MessageOf(x) } -// Deprecated: Use DrainJobResponse.ProtoReflect.Descriptor instead. -func (*DrainJobResponse) Descriptor() ([]byte, []int) { - return file_org_apache_beam_model_job_management_v1_beam_job_api_proto_rawDescGZIP(), []int{7} -} - func (x *DrainJobResponse) GetState() JobState_Enum { if x != nil { return x.State @@ -692,25 +813,40 @@ func (x *DrainJobResponse) GetState() JobState_Enum { return JobState_UNSPECIFIED } +func (x *DrainJobResponse) SetState(v JobState_Enum) { + x.State = v +} + +type DrainJobResponse_builder struct { + _ [0]func() // Prevents comparability and use of unkeyed literals for the builder. + + State JobState_Enum +} + +func (b0 DrainJobResponse_builder) Build() *DrainJobResponse { + m0 := &DrainJobResponse{} + b, x := &b0, m0 + _, _ = b, x + x.State = b.State + return m0 +} + // A subset of info provided by ProvisionApi.ProvisionInfo type JobInfo struct { - state protoimpl.MessageState - sizeCache protoimpl.SizeCache - unknownFields protoimpl.UnknownFields - - JobId string `protobuf:"bytes,1,opt,name=job_id,json=jobId,proto3" json:"job_id,omitempty"` // (required) - JobName string `protobuf:"bytes,2,opt,name=job_name,json=jobName,proto3" json:"job_name,omitempty"` // (required) - PipelineOptions *structpb.Struct `protobuf:"bytes,3,opt,name=pipeline_options,json=pipelineOptions,proto3" json:"pipeline_options,omitempty"` // (required) - State JobState_Enum `protobuf:"varint,4,opt,name=state,proto3,enum=org.apache.beam.model.job_management.v1.JobState_Enum" json:"state,omitempty"` // (required) + state protoimpl.MessageState `protogen:"hybrid.v1"` + JobId string `protobuf:"bytes,1,opt,name=job_id,json=jobId,proto3" json:"job_id,omitempty"` // (required) + JobName string `protobuf:"bytes,2,opt,name=job_name,json=jobName,proto3" json:"job_name,omitempty"` // (required) + PipelineOptions *structpb.Struct `protobuf:"bytes,3,opt,name=pipeline_options,json=pipelineOptions,proto3" json:"pipeline_options,omitempty"` // (required) + State JobState_Enum `protobuf:"varint,4,opt,name=state,proto3,enum=org.apache.beam.model.job_management.v1.JobState_Enum" json:"state,omitempty"` // (required) + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache } func (x *JobInfo) Reset() { *x = JobInfo{} - if protoimpl.UnsafeEnabled { - mi := &file_org_apache_beam_model_job_management_v1_beam_job_api_proto_msgTypes[8] - ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) - ms.StoreMessageInfo(mi) - } + mi := &file_org_apache_beam_model_job_management_v1_beam_job_api_proto_msgTypes[8] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) } func (x *JobInfo) String() string { @@ -721,7 +857,7 @@ func (*JobInfo) ProtoMessage() {} func (x *JobInfo) ProtoReflect() protoreflect.Message { mi := &file_org_apache_beam_model_job_management_v1_beam_job_api_proto_msgTypes[8] - if protoimpl.UnsafeEnabled && x != nil { + if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { ms.StoreMessageInfo(mi) @@ -731,11 +867,6 @@ func (x *JobInfo) ProtoReflect() protoreflect.Message { return mi.MessageOf(x) } -// Deprecated: Use JobInfo.ProtoReflect.Descriptor instead. -func (*JobInfo) Descriptor() ([]byte, []int) { - return file_org_apache_beam_model_job_management_v1_beam_job_api_proto_rawDescGZIP(), []int{8} -} - func (x *JobInfo) GetJobId() string { if x != nil { return x.JobId @@ -764,21 +895,66 @@ func (x *JobInfo) GetState() JobState_Enum { return JobState_UNSPECIFIED } +func (x *JobInfo) SetJobId(v string) { + x.JobId = v +} + +func (x *JobInfo) SetJobName(v string) { + x.JobName = v +} + +func (x *JobInfo) SetPipelineOptions(v *structpb.Struct) { + x.PipelineOptions = v +} + +func (x *JobInfo) SetState(v JobState_Enum) { + x.State = v +} + +func (x *JobInfo) HasPipelineOptions() bool { + if x == nil { + return false + } + return x.PipelineOptions != nil +} + +func (x *JobInfo) ClearPipelineOptions() { + x.PipelineOptions = nil +} + +type JobInfo_builder struct { + _ [0]func() // Prevents comparability and use of unkeyed literals for the builder. + + JobId string + JobName string + PipelineOptions *structpb.Struct + State JobState_Enum +} + +func (b0 JobInfo_builder) Build() *JobInfo { + m0 := &JobInfo{} + b, x := &b0, m0 + _, _ = b, x + x.JobId = b.JobId + x.JobName = b.JobName + x.PipelineOptions = b.PipelineOptions + x.State = b.State + return m0 +} + // GetJobs is a synchronus request that returns a list of invoked jobs back // Throws error GRPC_STATUS_UNAVAILABLE if server is down type GetJobsRequest struct { - state protoimpl.MessageState - sizeCache protoimpl.SizeCache + state protoimpl.MessageState `protogen:"hybrid.v1"` unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache } func (x *GetJobsRequest) Reset() { *x = GetJobsRequest{} - if protoimpl.UnsafeEnabled { - mi := &file_org_apache_beam_model_job_management_v1_beam_job_api_proto_msgTypes[9] - ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) - ms.StoreMessageInfo(mi) - } + mi := &file_org_apache_beam_model_job_management_v1_beam_job_api_proto_msgTypes[9] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) } func (x *GetJobsRequest) String() string { @@ -789,7 +965,7 @@ func (*GetJobsRequest) ProtoMessage() {} func (x *GetJobsRequest) ProtoReflect() protoreflect.Message { mi := &file_org_apache_beam_model_job_management_v1_beam_job_api_proto_msgTypes[9] - if protoimpl.UnsafeEnabled && x != nil { + if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { ms.StoreMessageInfo(mi) @@ -799,26 +975,30 @@ func (x *GetJobsRequest) ProtoReflect() protoreflect.Message { return mi.MessageOf(x) } -// Deprecated: Use GetJobsRequest.ProtoReflect.Descriptor instead. -func (*GetJobsRequest) Descriptor() ([]byte, []int) { - return file_org_apache_beam_model_job_management_v1_beam_job_api_proto_rawDescGZIP(), []int{9} +type GetJobsRequest_builder struct { + _ [0]func() // Prevents comparability and use of unkeyed literals for the builder. + +} + +func (b0 GetJobsRequest_builder) Build() *GetJobsRequest { + m0 := &GetJobsRequest{} + b, x := &b0, m0 + _, _ = b, x + return m0 } type GetJobsResponse struct { - state protoimpl.MessageState - sizeCache protoimpl.SizeCache + state protoimpl.MessageState `protogen:"hybrid.v1"` + JobInfo []*JobInfo `protobuf:"bytes,1,rep,name=job_info,json=jobInfo,proto3" json:"job_info,omitempty"` // (required) unknownFields protoimpl.UnknownFields - - JobInfo []*JobInfo `protobuf:"bytes,1,rep,name=job_info,json=jobInfo,proto3" json:"job_info,omitempty"` // (required) + sizeCache protoimpl.SizeCache } func (x *GetJobsResponse) Reset() { *x = GetJobsResponse{} - if protoimpl.UnsafeEnabled { - mi := &file_org_apache_beam_model_job_management_v1_beam_job_api_proto_msgTypes[10] - ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) - ms.StoreMessageInfo(mi) - } + mi := &file_org_apache_beam_model_job_management_v1_beam_job_api_proto_msgTypes[10] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) } func (x *GetJobsResponse) String() string { @@ -829,7 +1009,7 @@ func (*GetJobsResponse) ProtoMessage() {} func (x *GetJobsResponse) ProtoReflect() protoreflect.Message { mi := &file_org_apache_beam_model_job_management_v1_beam_job_api_proto_msgTypes[10] - if protoimpl.UnsafeEnabled && x != nil { + if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { ms.StoreMessageInfo(mi) @@ -839,11 +1019,6 @@ func (x *GetJobsResponse) ProtoReflect() protoreflect.Message { return mi.MessageOf(x) } -// Deprecated: Use GetJobsResponse.ProtoReflect.Descriptor instead. -func (*GetJobsResponse) Descriptor() ([]byte, []int) { - return file_org_apache_beam_model_job_management_v1_beam_job_api_proto_rawDescGZIP(), []int{10} -} - func (x *GetJobsResponse) GetJobInfo() []*JobInfo { if x != nil { return x.JobInfo @@ -851,24 +1026,39 @@ func (x *GetJobsResponse) GetJobInfo() []*JobInfo { return nil } +func (x *GetJobsResponse) SetJobInfo(v []*JobInfo) { + x.JobInfo = v +} + +type GetJobsResponse_builder struct { + _ [0]func() // Prevents comparability and use of unkeyed literals for the builder. + + JobInfo []*JobInfo +} + +func (b0 GetJobsResponse_builder) Build() *GetJobsResponse { + m0 := &GetJobsResponse{} + b, x := &b0, m0 + _, _ = b, x + x.JobInfo = b.JobInfo + return m0 +} + // GetState is a synchronus request that returns a job state back // Throws error GRPC_STATUS_UNAVAILABLE if server is down // Throws error NOT_FOUND if the jobId is not found type GetJobStateRequest struct { - state protoimpl.MessageState - sizeCache protoimpl.SizeCache + state protoimpl.MessageState `protogen:"hybrid.v1"` + JobId string `protobuf:"bytes,1,opt,name=job_id,json=jobId,proto3" json:"job_id,omitempty"` // (required) unknownFields protoimpl.UnknownFields - - JobId string `protobuf:"bytes,1,opt,name=job_id,json=jobId,proto3" json:"job_id,omitempty"` // (required) + sizeCache protoimpl.SizeCache } func (x *GetJobStateRequest) Reset() { *x = GetJobStateRequest{} - if protoimpl.UnsafeEnabled { - mi := &file_org_apache_beam_model_job_management_v1_beam_job_api_proto_msgTypes[11] - ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) - ms.StoreMessageInfo(mi) - } + mi := &file_org_apache_beam_model_job_management_v1_beam_job_api_proto_msgTypes[11] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) } func (x *GetJobStateRequest) String() string { @@ -879,7 +1069,7 @@ func (*GetJobStateRequest) ProtoMessage() {} func (x *GetJobStateRequest) ProtoReflect() protoreflect.Message { mi := &file_org_apache_beam_model_job_management_v1_beam_job_api_proto_msgTypes[11] - if protoimpl.UnsafeEnabled && x != nil { + if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { ms.StoreMessageInfo(mi) @@ -889,11 +1079,6 @@ func (x *GetJobStateRequest) ProtoReflect() protoreflect.Message { return mi.MessageOf(x) } -// Deprecated: Use GetJobStateRequest.ProtoReflect.Descriptor instead. -func (*GetJobStateRequest) Descriptor() ([]byte, []int) { - return file_org_apache_beam_model_job_management_v1_beam_job_api_proto_rawDescGZIP(), []int{11} -} - func (x *GetJobStateRequest) GetJobId() string { if x != nil { return x.JobId @@ -901,22 +1086,37 @@ func (x *GetJobStateRequest) GetJobId() string { return "" } +func (x *GetJobStateRequest) SetJobId(v string) { + x.JobId = v +} + +type GetJobStateRequest_builder struct { + _ [0]func() // Prevents comparability and use of unkeyed literals for the builder. + + JobId string +} + +func (b0 GetJobStateRequest_builder) Build() *GetJobStateRequest { + m0 := &GetJobStateRequest{} + b, x := &b0, m0 + _, _ = b, x + x.JobId = b.JobId + return m0 +} + type JobStateEvent struct { - state protoimpl.MessageState - sizeCache protoimpl.SizeCache + state protoimpl.MessageState `protogen:"hybrid.v1"` + State JobState_Enum `protobuf:"varint,1,opt,name=state,proto3,enum=org.apache.beam.model.job_management.v1.JobState_Enum" json:"state,omitempty"` // (required) + Timestamp *timestamppb.Timestamp `protobuf:"bytes,2,opt,name=timestamp,proto3" json:"timestamp,omitempty"` // (required) unknownFields protoimpl.UnknownFields - - State JobState_Enum `protobuf:"varint,1,opt,name=state,proto3,enum=org.apache.beam.model.job_management.v1.JobState_Enum" json:"state,omitempty"` // (required) - Timestamp *timestamppb.Timestamp `protobuf:"bytes,2,opt,name=timestamp,proto3" json:"timestamp,omitempty"` // (required) + sizeCache protoimpl.SizeCache } func (x *JobStateEvent) Reset() { *x = JobStateEvent{} - if protoimpl.UnsafeEnabled { - mi := &file_org_apache_beam_model_job_management_v1_beam_job_api_proto_msgTypes[12] - ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) - ms.StoreMessageInfo(mi) - } + mi := &file_org_apache_beam_model_job_management_v1_beam_job_api_proto_msgTypes[12] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) } func (x *JobStateEvent) String() string { @@ -927,7 +1127,7 @@ func (*JobStateEvent) ProtoMessage() {} func (x *JobStateEvent) ProtoReflect() protoreflect.Message { mi := &file_org_apache_beam_model_job_management_v1_beam_job_api_proto_msgTypes[12] - if protoimpl.UnsafeEnabled && x != nil { + if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { ms.StoreMessageInfo(mi) @@ -937,11 +1137,6 @@ func (x *JobStateEvent) ProtoReflect() protoreflect.Message { return mi.MessageOf(x) } -// Deprecated: Use JobStateEvent.ProtoReflect.Descriptor instead. -func (*JobStateEvent) Descriptor() ([]byte, []int) { - return file_org_apache_beam_model_job_management_v1_beam_job_api_proto_rawDescGZIP(), []int{12} -} - func (x *JobStateEvent) GetState() JobState_Enum { if x != nil { return x.State @@ -956,24 +1151,56 @@ func (x *JobStateEvent) GetTimestamp() *timestamppb.Timestamp { return nil } +func (x *JobStateEvent) SetState(v JobState_Enum) { + x.State = v +} + +func (x *JobStateEvent) SetTimestamp(v *timestamppb.Timestamp) { + x.Timestamp = v +} + +func (x *JobStateEvent) HasTimestamp() bool { + if x == nil { + return false + } + return x.Timestamp != nil +} + +func (x *JobStateEvent) ClearTimestamp() { + x.Timestamp = nil +} + +type JobStateEvent_builder struct { + _ [0]func() // Prevents comparability and use of unkeyed literals for the builder. + + State JobState_Enum + Timestamp *timestamppb.Timestamp +} + +func (b0 JobStateEvent_builder) Build() *JobStateEvent { + m0 := &JobStateEvent{} + b, x := &b0, m0 + _, _ = b, x + x.State = b.State + x.Timestamp = b.Timestamp + return m0 +} + // GetPipeline is a synchronus request that returns a pipeline back // Throws error GRPC_STATUS_UNAVAILABLE if server is down // Throws error NOT_FOUND if the jobId is not found type GetJobPipelineRequest struct { - state protoimpl.MessageState - sizeCache protoimpl.SizeCache + state protoimpl.MessageState `protogen:"hybrid.v1"` + JobId string `protobuf:"bytes,1,opt,name=job_id,json=jobId,proto3" json:"job_id,omitempty"` // (required) unknownFields protoimpl.UnknownFields - - JobId string `protobuf:"bytes,1,opt,name=job_id,json=jobId,proto3" json:"job_id,omitempty"` // (required) + sizeCache protoimpl.SizeCache } func (x *GetJobPipelineRequest) Reset() { *x = GetJobPipelineRequest{} - if protoimpl.UnsafeEnabled { - mi := &file_org_apache_beam_model_job_management_v1_beam_job_api_proto_msgTypes[13] - ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) - ms.StoreMessageInfo(mi) - } + mi := &file_org_apache_beam_model_job_management_v1_beam_job_api_proto_msgTypes[13] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) } func (x *GetJobPipelineRequest) String() string { @@ -984,7 +1211,7 @@ func (*GetJobPipelineRequest) ProtoMessage() {} func (x *GetJobPipelineRequest) ProtoReflect() protoreflect.Message { mi := &file_org_apache_beam_model_job_management_v1_beam_job_api_proto_msgTypes[13] - if protoimpl.UnsafeEnabled && x != nil { + if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { ms.StoreMessageInfo(mi) @@ -994,11 +1221,6 @@ func (x *GetJobPipelineRequest) ProtoReflect() protoreflect.Message { return mi.MessageOf(x) } -// Deprecated: Use GetJobPipelineRequest.ProtoReflect.Descriptor instead. -func (*GetJobPipelineRequest) Descriptor() ([]byte, []int) { - return file_org_apache_beam_model_job_management_v1_beam_job_api_proto_rawDescGZIP(), []int{13} -} - func (x *GetJobPipelineRequest) GetJobId() string { if x != nil { return x.JobId @@ -1006,21 +1228,36 @@ func (x *GetJobPipelineRequest) GetJobId() string { return "" } +func (x *GetJobPipelineRequest) SetJobId(v string) { + x.JobId = v +} + +type GetJobPipelineRequest_builder struct { + _ [0]func() // Prevents comparability and use of unkeyed literals for the builder. + + JobId string +} + +func (b0 GetJobPipelineRequest_builder) Build() *GetJobPipelineRequest { + m0 := &GetJobPipelineRequest{} + b, x := &b0, m0 + _, _ = b, x + x.JobId = b.JobId + return m0 +} + type GetJobPipelineResponse struct { - state protoimpl.MessageState - sizeCache protoimpl.SizeCache + state protoimpl.MessageState `protogen:"hybrid.v1"` + Pipeline *pipeline_v1.Pipeline `protobuf:"bytes,1,opt,name=pipeline,proto3" json:"pipeline,omitempty"` // (required) unknownFields protoimpl.UnknownFields - - Pipeline *pipeline_v1.Pipeline `protobuf:"bytes,1,opt,name=pipeline,proto3" json:"pipeline,omitempty"` // (required) + sizeCache protoimpl.SizeCache } func (x *GetJobPipelineResponse) Reset() { *x = GetJobPipelineResponse{} - if protoimpl.UnsafeEnabled { - mi := &file_org_apache_beam_model_job_management_v1_beam_job_api_proto_msgTypes[14] - ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) - ms.StoreMessageInfo(mi) - } + mi := &file_org_apache_beam_model_job_management_v1_beam_job_api_proto_msgTypes[14] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) } func (x *GetJobPipelineResponse) String() string { @@ -1031,7 +1268,7 @@ func (*GetJobPipelineResponse) ProtoMessage() {} func (x *GetJobPipelineResponse) ProtoReflect() protoreflect.Message { mi := &file_org_apache_beam_model_job_management_v1_beam_job_api_proto_msgTypes[14] - if protoimpl.UnsafeEnabled && x != nil { + if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { ms.StoreMessageInfo(mi) @@ -1041,11 +1278,6 @@ func (x *GetJobPipelineResponse) ProtoReflect() protoreflect.Message { return mi.MessageOf(x) } -// Deprecated: Use GetJobPipelineResponse.ProtoReflect.Descriptor instead. -func (*GetJobPipelineResponse) Descriptor() ([]byte, []int) { - return file_org_apache_beam_model_job_management_v1_beam_job_api_proto_rawDescGZIP(), []int{14} -} - func (x *GetJobPipelineResponse) GetPipeline() *pipeline_v1.Pipeline { if x != nil { return x.Pipeline @@ -1053,25 +1285,51 @@ func (x *GetJobPipelineResponse) GetPipeline() *pipeline_v1.Pipeline { return nil } +func (x *GetJobPipelineResponse) SetPipeline(v *pipeline_v1.Pipeline) { + x.Pipeline = v +} + +func (x *GetJobPipelineResponse) HasPipeline() bool { + if x == nil { + return false + } + return x.Pipeline != nil +} + +func (x *GetJobPipelineResponse) ClearPipeline() { + x.Pipeline = nil +} + +type GetJobPipelineResponse_builder struct { + _ [0]func() // Prevents comparability and use of unkeyed literals for the builder. + + Pipeline *pipeline_v1.Pipeline +} + +func (b0 GetJobPipelineResponse_builder) Build() *GetJobPipelineResponse { + m0 := &GetJobPipelineResponse{} + b, x := &b0, m0 + _, _ = b, x + x.Pipeline = b.Pipeline + return m0 +} + // GetJobMessages is a streaming api for streaming job messages from the service // One request will connect you to the job and you'll get a stream of job state // and job messages back; one is used for logging and the other for detecting // the job ended. type JobMessagesRequest struct { - state protoimpl.MessageState - sizeCache protoimpl.SizeCache + state protoimpl.MessageState `protogen:"hybrid.v1"` + JobId string `protobuf:"bytes,1,opt,name=job_id,json=jobId,proto3" json:"job_id,omitempty"` // (required) unknownFields protoimpl.UnknownFields - - JobId string `protobuf:"bytes,1,opt,name=job_id,json=jobId,proto3" json:"job_id,omitempty"` // (required) + sizeCache protoimpl.SizeCache } func (x *JobMessagesRequest) Reset() { *x = JobMessagesRequest{} - if protoimpl.UnsafeEnabled { - mi := &file_org_apache_beam_model_job_management_v1_beam_job_api_proto_msgTypes[15] - ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) - ms.StoreMessageInfo(mi) - } + mi := &file_org_apache_beam_model_job_management_v1_beam_job_api_proto_msgTypes[15] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) } func (x *JobMessagesRequest) String() string { @@ -1082,7 +1340,7 @@ func (*JobMessagesRequest) ProtoMessage() {} func (x *JobMessagesRequest) ProtoReflect() protoreflect.Message { mi := &file_org_apache_beam_model_job_management_v1_beam_job_api_proto_msgTypes[15] - if protoimpl.UnsafeEnabled && x != nil { + if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { ms.StoreMessageInfo(mi) @@ -1092,11 +1350,6 @@ func (x *JobMessagesRequest) ProtoReflect() protoreflect.Message { return mi.MessageOf(x) } -// Deprecated: Use JobMessagesRequest.ProtoReflect.Descriptor instead. -func (*JobMessagesRequest) Descriptor() ([]byte, []int) { - return file_org_apache_beam_model_job_management_v1_beam_job_api_proto_rawDescGZIP(), []int{15} -} - func (x *JobMessagesRequest) GetJobId() string { if x != nil { return x.JobId @@ -1104,24 +1357,39 @@ func (x *JobMessagesRequest) GetJobId() string { return "" } +func (x *JobMessagesRequest) SetJobId(v string) { + x.JobId = v +} + +type JobMessagesRequest_builder struct { + _ [0]func() // Prevents comparability and use of unkeyed literals for the builder. + + JobId string +} + +func (b0 JobMessagesRequest_builder) Build() *JobMessagesRequest { + m0 := &JobMessagesRequest{} + b, x := &b0, m0 + _, _ = b, x + x.JobId = b.JobId + return m0 +} + type JobMessage struct { - state protoimpl.MessageState - sizeCache protoimpl.SizeCache + state protoimpl.MessageState `protogen:"hybrid.v1"` + MessageId string `protobuf:"bytes,1,opt,name=message_id,json=messageId,proto3" json:"message_id,omitempty"` + Time string `protobuf:"bytes,2,opt,name=time,proto3" json:"time,omitempty"` + Importance JobMessage_MessageImportance `protobuf:"varint,3,opt,name=importance,proto3,enum=org.apache.beam.model.job_management.v1.JobMessage_MessageImportance" json:"importance,omitempty"` + MessageText string `protobuf:"bytes,4,opt,name=message_text,json=messageText,proto3" json:"message_text,omitempty"` unknownFields protoimpl.UnknownFields - - MessageId string `protobuf:"bytes,1,opt,name=message_id,json=messageId,proto3" json:"message_id,omitempty"` - Time string `protobuf:"bytes,2,opt,name=time,proto3" json:"time,omitempty"` - Importance JobMessage_MessageImportance `protobuf:"varint,3,opt,name=importance,proto3,enum=org.apache.beam.model.job_management.v1.JobMessage_MessageImportance" json:"importance,omitempty"` - MessageText string `protobuf:"bytes,4,opt,name=message_text,json=messageText,proto3" json:"message_text,omitempty"` + sizeCache protoimpl.SizeCache } func (x *JobMessage) Reset() { *x = JobMessage{} - if protoimpl.UnsafeEnabled { - mi := &file_org_apache_beam_model_job_management_v1_beam_job_api_proto_msgTypes[16] - ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) - ms.StoreMessageInfo(mi) - } + mi := &file_org_apache_beam_model_job_management_v1_beam_job_api_proto_msgTypes[16] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) } func (x *JobMessage) String() string { @@ -1132,7 +1400,7 @@ func (*JobMessage) ProtoMessage() {} func (x *JobMessage) ProtoReflect() protoreflect.Message { mi := &file_org_apache_beam_model_job_management_v1_beam_job_api_proto_msgTypes[16] - if protoimpl.UnsafeEnabled && x != nil { + if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { ms.StoreMessageInfo(mi) @@ -1142,11 +1410,6 @@ func (x *JobMessage) ProtoReflect() protoreflect.Message { return mi.MessageOf(x) } -// Deprecated: Use JobMessage.ProtoReflect.Descriptor instead. -func (*JobMessage) Descriptor() ([]byte, []int) { - return file_org_apache_beam_model_job_management_v1_beam_job_api_proto_rawDescGZIP(), []int{16} -} - func (x *JobMessage) GetMessageId() string { if x != nil { return x.MessageId @@ -1175,25 +1438,58 @@ func (x *JobMessage) GetMessageText() string { return "" } -type JobMessagesResponse struct { - state protoimpl.MessageState - sizeCache protoimpl.SizeCache - unknownFields protoimpl.UnknownFields +func (x *JobMessage) SetMessageId(v string) { + x.MessageId = v +} + +func (x *JobMessage) SetTime(v string) { + x.Time = v +} + +func (x *JobMessage) SetImportance(v JobMessage_MessageImportance) { + x.Importance = v +} + +func (x *JobMessage) SetMessageText(v string) { + x.MessageText = v +} - // Types that are assignable to Response: +type JobMessage_builder struct { + _ [0]func() // Prevents comparability and use of unkeyed literals for the builder. + + MessageId string + Time string + Importance JobMessage_MessageImportance + MessageText string +} + +func (b0 JobMessage_builder) Build() *JobMessage { + m0 := &JobMessage{} + b, x := &b0, m0 + _, _ = b, x + x.MessageId = b.MessageId + x.Time = b.Time + x.Importance = b.Importance + x.MessageText = b.MessageText + return m0 +} + +type JobMessagesResponse struct { + state protoimpl.MessageState `protogen:"hybrid.v1"` + // Types that are valid to be assigned to Response: // // *JobMessagesResponse_MessageResponse // *JobMessagesResponse_StateResponse - Response isJobMessagesResponse_Response `protobuf_oneof:"response"` + Response isJobMessagesResponse_Response `protobuf_oneof:"response"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache } func (x *JobMessagesResponse) Reset() { *x = JobMessagesResponse{} - if protoimpl.UnsafeEnabled { - mi := &file_org_apache_beam_model_job_management_v1_beam_job_api_proto_msgTypes[17] - ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) - ms.StoreMessageInfo(mi) - } + mi := &file_org_apache_beam_model_job_management_v1_beam_job_api_proto_msgTypes[17] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) } func (x *JobMessagesResponse) String() string { @@ -1204,7 +1500,7 @@ func (*JobMessagesResponse) ProtoMessage() {} func (x *JobMessagesResponse) ProtoReflect() protoreflect.Message { mi := &file_org_apache_beam_model_job_management_v1_beam_job_api_proto_msgTypes[17] - if protoimpl.UnsafeEnabled && x != nil { + if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { ms.StoreMessageInfo(mi) @@ -1214,32 +1510,136 @@ func (x *JobMessagesResponse) ProtoReflect() protoreflect.Message { return mi.MessageOf(x) } -// Deprecated: Use JobMessagesResponse.ProtoReflect.Descriptor instead. -func (*JobMessagesResponse) Descriptor() ([]byte, []int) { - return file_org_apache_beam_model_job_management_v1_beam_job_api_proto_rawDescGZIP(), []int{17} -} - -func (m *JobMessagesResponse) GetResponse() isJobMessagesResponse_Response { - if m != nil { - return m.Response +func (x *JobMessagesResponse) GetResponse() isJobMessagesResponse_Response { + if x != nil { + return x.Response } return nil } func (x *JobMessagesResponse) GetMessageResponse() *JobMessage { - if x, ok := x.GetResponse().(*JobMessagesResponse_MessageResponse); ok { - return x.MessageResponse + if x != nil { + if x, ok := x.Response.(*JobMessagesResponse_MessageResponse); ok { + return x.MessageResponse + } } return nil } func (x *JobMessagesResponse) GetStateResponse() *JobStateEvent { - if x, ok := x.GetResponse().(*JobMessagesResponse_StateResponse); ok { - return x.StateResponse + if x != nil { + if x, ok := x.Response.(*JobMessagesResponse_StateResponse); ok { + return x.StateResponse + } } return nil } +func (x *JobMessagesResponse) SetMessageResponse(v *JobMessage) { + if v == nil { + x.Response = nil + return + } + x.Response = &JobMessagesResponse_MessageResponse{v} +} + +func (x *JobMessagesResponse) SetStateResponse(v *JobStateEvent) { + if v == nil { + x.Response = nil + return + } + x.Response = &JobMessagesResponse_StateResponse{v} +} + +func (x *JobMessagesResponse) HasResponse() bool { + if x == nil { + return false + } + return x.Response != nil +} + +func (x *JobMessagesResponse) HasMessageResponse() bool { + if x == nil { + return false + } + _, ok := x.Response.(*JobMessagesResponse_MessageResponse) + return ok +} + +func (x *JobMessagesResponse) HasStateResponse() bool { + if x == nil { + return false + } + _, ok := x.Response.(*JobMessagesResponse_StateResponse) + return ok +} + +func (x *JobMessagesResponse) ClearResponse() { + x.Response = nil +} + +func (x *JobMessagesResponse) ClearMessageResponse() { + if _, ok := x.Response.(*JobMessagesResponse_MessageResponse); ok { + x.Response = nil + } +} + +func (x *JobMessagesResponse) ClearStateResponse() { + if _, ok := x.Response.(*JobMessagesResponse_StateResponse); ok { + x.Response = nil + } +} + +const JobMessagesResponse_Response_not_set_case case_JobMessagesResponse_Response = 0 +const JobMessagesResponse_MessageResponse_case case_JobMessagesResponse_Response = 1 +const JobMessagesResponse_StateResponse_case case_JobMessagesResponse_Response = 2 + +func (x *JobMessagesResponse) WhichResponse() case_JobMessagesResponse_Response { + if x == nil { + return JobMessagesResponse_Response_not_set_case + } + switch x.Response.(type) { + case *JobMessagesResponse_MessageResponse: + return JobMessagesResponse_MessageResponse_case + case *JobMessagesResponse_StateResponse: + return JobMessagesResponse_StateResponse_case + default: + return JobMessagesResponse_Response_not_set_case + } +} + +type JobMessagesResponse_builder struct { + _ [0]func() // Prevents comparability and use of unkeyed literals for the builder. + + // Fields of oneof Response: + MessageResponse *JobMessage + StateResponse *JobStateEvent + // -- end of Response +} + +func (b0 JobMessagesResponse_builder) Build() *JobMessagesResponse { + m0 := &JobMessagesResponse{} + b, x := &b0, m0 + _, _ = b, x + if b.MessageResponse != nil { + x.Response = &JobMessagesResponse_MessageResponse{b.MessageResponse} + } + if b.StateResponse != nil { + x.Response = &JobMessagesResponse_StateResponse{b.StateResponse} + } + return m0 +} + +type case_JobMessagesResponse_Response protoreflect.FieldNumber + +func (x case_JobMessagesResponse_Response) String() string { + md := file_org_apache_beam_model_job_management_v1_beam_job_api_proto_msgTypes[17].Descriptor() + if x == 0 { + return "not set" + } + return protoimpl.X.MessageFieldStringOf(md, protoreflect.FieldNumber(x)) +} + type isJobMessagesResponse_Response interface { isJobMessagesResponse_Response() } @@ -1269,18 +1669,16 @@ func (*JobMessagesResponse_StateResponse) isJobMessagesResponse_Response() {} // Transitions are optional such that a job may go from STOPPED to RUNNING // without needing to pass through STARTING. type JobState struct { - state protoimpl.MessageState - sizeCache protoimpl.SizeCache + state protoimpl.MessageState `protogen:"hybrid.v1"` unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache } func (x *JobState) Reset() { *x = JobState{} - if protoimpl.UnsafeEnabled { - mi := &file_org_apache_beam_model_job_management_v1_beam_job_api_proto_msgTypes[18] - ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) - ms.StoreMessageInfo(mi) - } + mi := &file_org_apache_beam_model_job_management_v1_beam_job_api_proto_msgTypes[18] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) } func (x *JobState) String() string { @@ -1291,7 +1689,7 @@ func (*JobState) ProtoMessage() {} func (x *JobState) ProtoReflect() protoreflect.Message { mi := &file_org_apache_beam_model_job_management_v1_beam_job_api_proto_msgTypes[18] - if protoimpl.UnsafeEnabled && x != nil { + if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { ms.StoreMessageInfo(mi) @@ -1301,26 +1699,30 @@ func (x *JobState) ProtoReflect() protoreflect.Message { return mi.MessageOf(x) } -// Deprecated: Use JobState.ProtoReflect.Descriptor instead. -func (*JobState) Descriptor() ([]byte, []int) { - return file_org_apache_beam_model_job_management_v1_beam_job_api_proto_rawDescGZIP(), []int{18} +type JobState_builder struct { + _ [0]func() // Prevents comparability and use of unkeyed literals for the builder. + +} + +func (b0 JobState_builder) Build() *JobState { + m0 := &JobState{} + b, x := &b0, m0 + _, _ = b, x + return m0 } type GetJobMetricsRequest struct { - state protoimpl.MessageState - sizeCache protoimpl.SizeCache + state protoimpl.MessageState `protogen:"hybrid.v1"` + JobId string `protobuf:"bytes,1,opt,name=job_id,json=jobId,proto3" json:"job_id,omitempty"` // (required) unknownFields protoimpl.UnknownFields - - JobId string `protobuf:"bytes,1,opt,name=job_id,json=jobId,proto3" json:"job_id,omitempty"` // (required) + sizeCache protoimpl.SizeCache } func (x *GetJobMetricsRequest) Reset() { *x = GetJobMetricsRequest{} - if protoimpl.UnsafeEnabled { - mi := &file_org_apache_beam_model_job_management_v1_beam_job_api_proto_msgTypes[19] - ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) - ms.StoreMessageInfo(mi) - } + mi := &file_org_apache_beam_model_job_management_v1_beam_job_api_proto_msgTypes[19] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) } func (x *GetJobMetricsRequest) String() string { @@ -1331,7 +1733,7 @@ func (*GetJobMetricsRequest) ProtoMessage() {} func (x *GetJobMetricsRequest) ProtoReflect() protoreflect.Message { mi := &file_org_apache_beam_model_job_management_v1_beam_job_api_proto_msgTypes[19] - if protoimpl.UnsafeEnabled && x != nil { + if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { ms.StoreMessageInfo(mi) @@ -1341,11 +1743,6 @@ func (x *GetJobMetricsRequest) ProtoReflect() protoreflect.Message { return mi.MessageOf(x) } -// Deprecated: Use GetJobMetricsRequest.ProtoReflect.Descriptor instead. -func (*GetJobMetricsRequest) Descriptor() ([]byte, []int) { - return file_org_apache_beam_model_job_management_v1_beam_job_api_proto_rawDescGZIP(), []int{19} -} - func (x *GetJobMetricsRequest) GetJobId() string { if x != nil { return x.JobId @@ -1353,21 +1750,36 @@ func (x *GetJobMetricsRequest) GetJobId() string { return "" } +func (x *GetJobMetricsRequest) SetJobId(v string) { + x.JobId = v +} + +type GetJobMetricsRequest_builder struct { + _ [0]func() // Prevents comparability and use of unkeyed literals for the builder. + + JobId string +} + +func (b0 GetJobMetricsRequest_builder) Build() *GetJobMetricsRequest { + m0 := &GetJobMetricsRequest{} + b, x := &b0, m0 + _, _ = b, x + x.JobId = b.JobId + return m0 +} + type GetJobMetricsResponse struct { - state protoimpl.MessageState - sizeCache protoimpl.SizeCache + state protoimpl.MessageState `protogen:"hybrid.v1"` + Metrics *MetricResults `protobuf:"bytes,1,opt,name=metrics,proto3" json:"metrics,omitempty"` unknownFields protoimpl.UnknownFields - - Metrics *MetricResults `protobuf:"bytes,1,opt,name=metrics,proto3" json:"metrics,omitempty"` + sizeCache protoimpl.SizeCache } func (x *GetJobMetricsResponse) Reset() { *x = GetJobMetricsResponse{} - if protoimpl.UnsafeEnabled { - mi := &file_org_apache_beam_model_job_management_v1_beam_job_api_proto_msgTypes[20] - ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) - ms.StoreMessageInfo(mi) - } + mi := &file_org_apache_beam_model_job_management_v1_beam_job_api_proto_msgTypes[20] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) } func (x *GetJobMetricsResponse) String() string { @@ -1378,7 +1790,7 @@ func (*GetJobMetricsResponse) ProtoMessage() {} func (x *GetJobMetricsResponse) ProtoReflect() protoreflect.Message { mi := &file_org_apache_beam_model_job_management_v1_beam_job_api_proto_msgTypes[20] - if protoimpl.UnsafeEnabled && x != nil { + if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { ms.StoreMessageInfo(mi) @@ -1388,11 +1800,6 @@ func (x *GetJobMetricsResponse) ProtoReflect() protoreflect.Message { return mi.MessageOf(x) } -// Deprecated: Use GetJobMetricsResponse.ProtoReflect.Descriptor instead. -func (*GetJobMetricsResponse) Descriptor() ([]byte, []int) { - return file_org_apache_beam_model_job_management_v1_beam_job_api_proto_rawDescGZIP(), []int{20} -} - func (x *GetJobMetricsResponse) GetMetrics() *MetricResults { if x != nil { return x.Metrics @@ -1400,23 +1807,49 @@ func (x *GetJobMetricsResponse) GetMetrics() *MetricResults { return nil } +func (x *GetJobMetricsResponse) SetMetrics(v *MetricResults) { + x.Metrics = v +} + +func (x *GetJobMetricsResponse) HasMetrics() bool { + if x == nil { + return false + } + return x.Metrics != nil +} + +func (x *GetJobMetricsResponse) ClearMetrics() { + x.Metrics = nil +} + +type GetJobMetricsResponse_builder struct { + _ [0]func() // Prevents comparability and use of unkeyed literals for the builder. + + Metrics *MetricResults +} + +func (b0 GetJobMetricsResponse_builder) Build() *GetJobMetricsResponse { + m0 := &GetJobMetricsResponse{} + b, x := &b0, m0 + _, _ = b, x + x.Metrics = b.Metrics + return m0 +} + // All metrics for a given job. Runners may support one or the other or both. type MetricResults struct { - state protoimpl.MessageState - sizeCache protoimpl.SizeCache + state protoimpl.MessageState `protogen:"hybrid.v1"` + Attempted []*pipeline_v1.MonitoringInfo `protobuf:"bytes,1,rep,name=attempted,proto3" json:"attempted,omitempty"` + Committed []*pipeline_v1.MonitoringInfo `protobuf:"bytes,2,rep,name=committed,proto3" json:"committed,omitempty"` unknownFields protoimpl.UnknownFields - - Attempted []*pipeline_v1.MonitoringInfo `protobuf:"bytes,1,rep,name=attempted,proto3" json:"attempted,omitempty"` - Committed []*pipeline_v1.MonitoringInfo `protobuf:"bytes,2,rep,name=committed,proto3" json:"committed,omitempty"` + sizeCache protoimpl.SizeCache } func (x *MetricResults) Reset() { *x = MetricResults{} - if protoimpl.UnsafeEnabled { - mi := &file_org_apache_beam_model_job_management_v1_beam_job_api_proto_msgTypes[21] - ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) - ms.StoreMessageInfo(mi) - } + mi := &file_org_apache_beam_model_job_management_v1_beam_job_api_proto_msgTypes[21] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) } func (x *MetricResults) String() string { @@ -1427,7 +1860,7 @@ func (*MetricResults) ProtoMessage() {} func (x *MetricResults) ProtoReflect() protoreflect.Message { mi := &file_org_apache_beam_model_job_management_v1_beam_job_api_proto_msgTypes[21] - if protoimpl.UnsafeEnabled && x != nil { + if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { ms.StoreMessageInfo(mi) @@ -1437,11 +1870,6 @@ func (x *MetricResults) ProtoReflect() protoreflect.Message { return mi.MessageOf(x) } -// Deprecated: Use MetricResults.ProtoReflect.Descriptor instead. -func (*MetricResults) Descriptor() ([]byte, []int) { - return file_org_apache_beam_model_job_management_v1_beam_job_api_proto_rawDescGZIP(), []int{21} -} - func (x *MetricResults) GetAttempted() []*pipeline_v1.MonitoringInfo { if x != nil { return x.Attempted @@ -1456,23 +1884,45 @@ func (x *MetricResults) GetCommitted() []*pipeline_v1.MonitoringInfo { return nil } +func (x *MetricResults) SetAttempted(v []*pipeline_v1.MonitoringInfo) { + x.Attempted = v +} + +func (x *MetricResults) SetCommitted(v []*pipeline_v1.MonitoringInfo) { + x.Committed = v +} + +type MetricResults_builder struct { + _ [0]func() // Prevents comparability and use of unkeyed literals for the builder. + + Attempted []*pipeline_v1.MonitoringInfo + Committed []*pipeline_v1.MonitoringInfo +} + +func (b0 MetricResults_builder) Build() *MetricResults { + m0 := &MetricResults{} + b, x := &b0, m0 + _, _ = b, x + x.Attempted = b.Attempted + x.Committed = b.Committed + return m0 +} + // DescribePipelineOptions provides metadata about the options supported by a runner. // It will be used by the SDK client to validate the options specified by or // list available options to the user. // Throws error GRPC_STATUS_UNAVAILABLE if server is down type DescribePipelineOptionsRequest struct { - state protoimpl.MessageState - sizeCache protoimpl.SizeCache + state protoimpl.MessageState `protogen:"hybrid.v1"` unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache } func (x *DescribePipelineOptionsRequest) Reset() { *x = DescribePipelineOptionsRequest{} - if protoimpl.UnsafeEnabled { - mi := &file_org_apache_beam_model_job_management_v1_beam_job_api_proto_msgTypes[22] - ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) - ms.StoreMessageInfo(mi) - } + mi := &file_org_apache_beam_model_job_management_v1_beam_job_api_proto_msgTypes[22] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) } func (x *DescribePipelineOptionsRequest) String() string { @@ -1483,7 +1933,7 @@ func (*DescribePipelineOptionsRequest) ProtoMessage() {} func (x *DescribePipelineOptionsRequest) ProtoReflect() protoreflect.Message { mi := &file_org_apache_beam_model_job_management_v1_beam_job_api_proto_msgTypes[22] - if protoimpl.UnsafeEnabled && x != nil { + if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { ms.StoreMessageInfo(mi) @@ -1493,26 +1943,31 @@ func (x *DescribePipelineOptionsRequest) ProtoReflect() protoreflect.Message { return mi.MessageOf(x) } -// Deprecated: Use DescribePipelineOptionsRequest.ProtoReflect.Descriptor instead. -func (*DescribePipelineOptionsRequest) Descriptor() ([]byte, []int) { - return file_org_apache_beam_model_job_management_v1_beam_job_api_proto_rawDescGZIP(), []int{22} +type DescribePipelineOptionsRequest_builder struct { + _ [0]func() // Prevents comparability and use of unkeyed literals for the builder. + +} + +func (b0 DescribePipelineOptionsRequest_builder) Build() *DescribePipelineOptionsRequest { + m0 := &DescribePipelineOptionsRequest{} + b, x := &b0, m0 + _, _ = b, x + return m0 } // Type for pipeline options. // Types mirror those of JSON, since that's how pipeline options are serialized. type PipelineOptionType struct { - state protoimpl.MessageState - sizeCache protoimpl.SizeCache + state protoimpl.MessageState `protogen:"hybrid.v1"` unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache } func (x *PipelineOptionType) Reset() { *x = PipelineOptionType{} - if protoimpl.UnsafeEnabled { - mi := &file_org_apache_beam_model_job_management_v1_beam_job_api_proto_msgTypes[23] - ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) - ms.StoreMessageInfo(mi) - } + mi := &file_org_apache_beam_model_job_management_v1_beam_job_api_proto_msgTypes[23] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) } func (x *PipelineOptionType) String() string { @@ -1523,7 +1978,7 @@ func (*PipelineOptionType) ProtoMessage() {} func (x *PipelineOptionType) ProtoReflect() protoreflect.Message { mi := &file_org_apache_beam_model_job_management_v1_beam_job_api_proto_msgTypes[23] - if protoimpl.UnsafeEnabled && x != nil { + if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { ms.StoreMessageInfo(mi) @@ -1533,17 +1988,21 @@ func (x *PipelineOptionType) ProtoReflect() protoreflect.Message { return mi.MessageOf(x) } -// Deprecated: Use PipelineOptionType.ProtoReflect.Descriptor instead. -func (*PipelineOptionType) Descriptor() ([]byte, []int) { - return file_org_apache_beam_model_job_management_v1_beam_job_api_proto_rawDescGZIP(), []int{23} +type PipelineOptionType_builder struct { + _ [0]func() // Prevents comparability and use of unkeyed literals for the builder. + +} + +func (b0 PipelineOptionType_builder) Build() *PipelineOptionType { + m0 := &PipelineOptionType{} + b, x := &b0, m0 + _, _ = b, x + return m0 } // Metadata for a pipeline option. type PipelineOptionDescriptor struct { - state protoimpl.MessageState - sizeCache protoimpl.SizeCache - unknownFields protoimpl.UnknownFields - + state protoimpl.MessageState `protogen:"hybrid.v1"` // (Required) The option name. Name string `protobuf:"bytes,1,opt,name=name,proto3" json:"name,omitempty"` // (Required) Type of option. @@ -1553,16 +2012,16 @@ type PipelineOptionDescriptor struct { // (Optional) Default value. DefaultValue string `protobuf:"bytes,4,opt,name=default_value,json=defaultValue,proto3" json:"default_value,omitempty"` // (Required) The group this option belongs to. - Group string `protobuf:"bytes,5,opt,name=group,proto3" json:"group,omitempty"` + Group string `protobuf:"bytes,5,opt,name=group,proto3" json:"group,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache } func (x *PipelineOptionDescriptor) Reset() { *x = PipelineOptionDescriptor{} - if protoimpl.UnsafeEnabled { - mi := &file_org_apache_beam_model_job_management_v1_beam_job_api_proto_msgTypes[24] - ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) - ms.StoreMessageInfo(mi) - } + mi := &file_org_apache_beam_model_job_management_v1_beam_job_api_proto_msgTypes[24] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) } func (x *PipelineOptionDescriptor) String() string { @@ -1573,7 +2032,7 @@ func (*PipelineOptionDescriptor) ProtoMessage() {} func (x *PipelineOptionDescriptor) ProtoReflect() protoreflect.Message { mi := &file_org_apache_beam_model_job_management_v1_beam_job_api_proto_msgTypes[24] - if protoimpl.UnsafeEnabled && x != nil { + if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { ms.StoreMessageInfo(mi) @@ -1583,11 +2042,6 @@ func (x *PipelineOptionDescriptor) ProtoReflect() protoreflect.Message { return mi.MessageOf(x) } -// Deprecated: Use PipelineOptionDescriptor.ProtoReflect.Descriptor instead. -func (*PipelineOptionDescriptor) Descriptor() ([]byte, []int) { - return file_org_apache_beam_model_job_management_v1_beam_job_api_proto_rawDescGZIP(), []int{24} -} - func (x *PipelineOptionDescriptor) GetName() string { if x != nil { return x.Name @@ -1623,22 +2077,66 @@ func (x *PipelineOptionDescriptor) GetGroup() string { return "" } -type DescribePipelineOptionsResponse struct { - state protoimpl.MessageState - sizeCache protoimpl.SizeCache - unknownFields protoimpl.UnknownFields +func (x *PipelineOptionDescriptor) SetName(v string) { + x.Name = v +} + +func (x *PipelineOptionDescriptor) SetType(v PipelineOptionType_Enum) { + x.Type = v +} + +func (x *PipelineOptionDescriptor) SetDescription(v string) { + x.Description = v +} + +func (x *PipelineOptionDescriptor) SetDefaultValue(v string) { + x.DefaultValue = v +} + +func (x *PipelineOptionDescriptor) SetGroup(v string) { + x.Group = v +} + +type PipelineOptionDescriptor_builder struct { + _ [0]func() // Prevents comparability and use of unkeyed literals for the builder. + + // (Required) The option name. + Name string + // (Required) Type of option. + Type PipelineOptionType_Enum + // (Optional) Description suitable for display / help text. + Description string + // (Optional) Default value. + DefaultValue string + // (Required) The group this option belongs to. + Group string +} + +func (b0 PipelineOptionDescriptor_builder) Build() *PipelineOptionDescriptor { + m0 := &PipelineOptionDescriptor{} + b, x := &b0, m0 + _, _ = b, x + x.Name = b.Name + x.Type = b.Type + x.Description = b.Description + x.DefaultValue = b.DefaultValue + x.Group = b.Group + return m0 +} +type DescribePipelineOptionsResponse struct { + state protoimpl.MessageState `protogen:"hybrid.v1"` // List of pipeline option descriptors. - Options []*PipelineOptionDescriptor `protobuf:"bytes,1,rep,name=options,proto3" json:"options,omitempty"` + Options []*PipelineOptionDescriptor `protobuf:"bytes,1,rep,name=options,proto3" json:"options,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache } func (x *DescribePipelineOptionsResponse) Reset() { *x = DescribePipelineOptionsResponse{} - if protoimpl.UnsafeEnabled { - mi := &file_org_apache_beam_model_job_management_v1_beam_job_api_proto_msgTypes[25] - ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) - ms.StoreMessageInfo(mi) - } + mi := &file_org_apache_beam_model_job_management_v1_beam_job_api_proto_msgTypes[25] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) } func (x *DescribePipelineOptionsResponse) String() string { @@ -1649,7 +2147,7 @@ func (*DescribePipelineOptionsResponse) ProtoMessage() {} func (x *DescribePipelineOptionsResponse) ProtoReflect() protoreflect.Message { mi := &file_org_apache_beam_model_job_management_v1_beam_job_api_proto_msgTypes[25] - if protoimpl.UnsafeEnabled && x != nil { + if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { ms.StoreMessageInfo(mi) @@ -1659,11 +2157,6 @@ func (x *DescribePipelineOptionsResponse) ProtoReflect() protoreflect.Message { return mi.MessageOf(x) } -// Deprecated: Use DescribePipelineOptionsResponse.ProtoReflect.Descriptor instead. -func (*DescribePipelineOptionsResponse) Descriptor() ([]byte, []int) { - return file_org_apache_beam_model_job_management_v1_beam_job_api_proto_rawDescGZIP(), []int{25} -} - func (x *DescribePipelineOptionsResponse) GetOptions() []*PipelineOptionDescriptor { if x != nil { return x.Options @@ -1671,6 +2164,25 @@ func (x *DescribePipelineOptionsResponse) GetOptions() []*PipelineOptionDescript return nil } +func (x *DescribePipelineOptionsResponse) SetOptions(v []*PipelineOptionDescriptor) { + x.Options = v +} + +type DescribePipelineOptionsResponse_builder struct { + _ [0]func() // Prevents comparability and use of unkeyed literals for the builder. + + // List of pipeline option descriptors. + Options []*PipelineOptionDescriptor +} + +func (b0 DescribePipelineOptionsResponse_builder) Build() *DescribePipelineOptionsResponse { + m0 := &DescribePipelineOptionsResponse{} + b, x := &b0, m0 + _, _ = b, x + x.Options = b.Options + return m0 +} + var File_org_apache_beam_model_job_management_v1_beam_job_api_proto protoreflect.FileDescriptor var file_org_apache_beam_model_job_management_v1_beam_job_api_proto_rawDesc = []byte{ @@ -2000,21 +2512,9 @@ var file_org_apache_beam_model_job_management_v1_beam_job_api_proto_rawDesc = [] 0x74, 0x6f, 0x33, } -var ( - file_org_apache_beam_model_job_management_v1_beam_job_api_proto_rawDescOnce sync.Once - file_org_apache_beam_model_job_management_v1_beam_job_api_proto_rawDescData = file_org_apache_beam_model_job_management_v1_beam_job_api_proto_rawDesc -) - -func file_org_apache_beam_model_job_management_v1_beam_job_api_proto_rawDescGZIP() []byte { - file_org_apache_beam_model_job_management_v1_beam_job_api_proto_rawDescOnce.Do(func() { - file_org_apache_beam_model_job_management_v1_beam_job_api_proto_rawDescData = protoimpl.X.CompressGZIP(file_org_apache_beam_model_job_management_v1_beam_job_api_proto_rawDescData) - }) - return file_org_apache_beam_model_job_management_v1_beam_job_api_proto_rawDescData -} - var file_org_apache_beam_model_job_management_v1_beam_job_api_proto_enumTypes = make([]protoimpl.EnumInfo, 3) var file_org_apache_beam_model_job_management_v1_beam_job_api_proto_msgTypes = make([]protoimpl.MessageInfo, 26) -var file_org_apache_beam_model_job_management_v1_beam_job_api_proto_goTypes = []interface{}{ +var file_org_apache_beam_model_job_management_v1_beam_job_api_proto_goTypes = []any{ (JobMessage_MessageImportance)(0), // 0: org.apache.beam.model.job_management.v1.JobMessage.MessageImportance (JobState_Enum)(0), // 1: org.apache.beam.model.job_management.v1.JobState.Enum (PipelineOptionType_Enum)(0), // 2: org.apache.beam.model.job_management.v1.PipelineOptionType.Enum @@ -2104,321 +2604,7 @@ func file_org_apache_beam_model_job_management_v1_beam_job_api_proto_init() { if File_org_apache_beam_model_job_management_v1_beam_job_api_proto != nil { return } - if !protoimpl.UnsafeEnabled { - file_org_apache_beam_model_job_management_v1_beam_job_api_proto_msgTypes[0].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*PrepareJobRequest); i { - case 0: - return &v.state - case 1: - return &v.sizeCache - case 2: - return &v.unknownFields - default: - return nil - } - } - file_org_apache_beam_model_job_management_v1_beam_job_api_proto_msgTypes[1].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*PrepareJobResponse); i { - case 0: - return &v.state - case 1: - return &v.sizeCache - case 2: - return &v.unknownFields - default: - return nil - } - } - file_org_apache_beam_model_job_management_v1_beam_job_api_proto_msgTypes[2].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*RunJobRequest); i { - case 0: - return &v.state - case 1: - return &v.sizeCache - case 2: - return &v.unknownFields - default: - return nil - } - } - file_org_apache_beam_model_job_management_v1_beam_job_api_proto_msgTypes[3].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*RunJobResponse); i { - case 0: - return &v.state - case 1: - return &v.sizeCache - case 2: - return &v.unknownFields - default: - return nil - } - } - file_org_apache_beam_model_job_management_v1_beam_job_api_proto_msgTypes[4].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*CancelJobRequest); i { - case 0: - return &v.state - case 1: - return &v.sizeCache - case 2: - return &v.unknownFields - default: - return nil - } - } - file_org_apache_beam_model_job_management_v1_beam_job_api_proto_msgTypes[5].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*CancelJobResponse); i { - case 0: - return &v.state - case 1: - return &v.sizeCache - case 2: - return &v.unknownFields - default: - return nil - } - } - file_org_apache_beam_model_job_management_v1_beam_job_api_proto_msgTypes[6].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*DrainJobRequest); i { - case 0: - return &v.state - case 1: - return &v.sizeCache - case 2: - return &v.unknownFields - default: - return nil - } - } - file_org_apache_beam_model_job_management_v1_beam_job_api_proto_msgTypes[7].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*DrainJobResponse); i { - case 0: - return &v.state - case 1: - return &v.sizeCache - case 2: - return &v.unknownFields - default: - return nil - } - } - file_org_apache_beam_model_job_management_v1_beam_job_api_proto_msgTypes[8].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*JobInfo); i { - case 0: - return &v.state - case 1: - return &v.sizeCache - case 2: - return &v.unknownFields - default: - return nil - } - } - file_org_apache_beam_model_job_management_v1_beam_job_api_proto_msgTypes[9].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*GetJobsRequest); i { - case 0: - return &v.state - case 1: - return &v.sizeCache - case 2: - return &v.unknownFields - default: - return nil - } - } - file_org_apache_beam_model_job_management_v1_beam_job_api_proto_msgTypes[10].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*GetJobsResponse); i { - case 0: - return &v.state - case 1: - return &v.sizeCache - case 2: - return &v.unknownFields - default: - return nil - } - } - file_org_apache_beam_model_job_management_v1_beam_job_api_proto_msgTypes[11].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*GetJobStateRequest); i { - case 0: - return &v.state - case 1: - return &v.sizeCache - case 2: - return &v.unknownFields - default: - return nil - } - } - file_org_apache_beam_model_job_management_v1_beam_job_api_proto_msgTypes[12].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*JobStateEvent); i { - case 0: - return &v.state - case 1: - return &v.sizeCache - case 2: - return &v.unknownFields - default: - return nil - } - } - file_org_apache_beam_model_job_management_v1_beam_job_api_proto_msgTypes[13].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*GetJobPipelineRequest); i { - case 0: - return &v.state - case 1: - return &v.sizeCache - case 2: - return &v.unknownFields - default: - return nil - } - } - file_org_apache_beam_model_job_management_v1_beam_job_api_proto_msgTypes[14].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*GetJobPipelineResponse); i { - case 0: - return &v.state - case 1: - return &v.sizeCache - case 2: - return &v.unknownFields - default: - return nil - } - } - file_org_apache_beam_model_job_management_v1_beam_job_api_proto_msgTypes[15].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*JobMessagesRequest); i { - case 0: - return &v.state - case 1: - return &v.sizeCache - case 2: - return &v.unknownFields - default: - return nil - } - } - file_org_apache_beam_model_job_management_v1_beam_job_api_proto_msgTypes[16].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*JobMessage); i { - case 0: - return &v.state - case 1: - return &v.sizeCache - case 2: - return &v.unknownFields - default: - return nil - } - } - file_org_apache_beam_model_job_management_v1_beam_job_api_proto_msgTypes[17].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*JobMessagesResponse); i { - case 0: - return &v.state - case 1: - return &v.sizeCache - case 2: - return &v.unknownFields - default: - return nil - } - } - file_org_apache_beam_model_job_management_v1_beam_job_api_proto_msgTypes[18].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*JobState); i { - case 0: - return &v.state - case 1: - return &v.sizeCache - case 2: - return &v.unknownFields - default: - return nil - } - } - file_org_apache_beam_model_job_management_v1_beam_job_api_proto_msgTypes[19].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*GetJobMetricsRequest); i { - case 0: - return &v.state - case 1: - return &v.sizeCache - case 2: - return &v.unknownFields - default: - return nil - } - } - file_org_apache_beam_model_job_management_v1_beam_job_api_proto_msgTypes[20].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*GetJobMetricsResponse); i { - case 0: - return &v.state - case 1: - return &v.sizeCache - case 2: - return &v.unknownFields - default: - return nil - } - } - file_org_apache_beam_model_job_management_v1_beam_job_api_proto_msgTypes[21].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*MetricResults); i { - case 0: - return &v.state - case 1: - return &v.sizeCache - case 2: - return &v.unknownFields - default: - return nil - } - } - file_org_apache_beam_model_job_management_v1_beam_job_api_proto_msgTypes[22].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*DescribePipelineOptionsRequest); i { - case 0: - return &v.state - case 1: - return &v.sizeCache - case 2: - return &v.unknownFields - default: - return nil - } - } - file_org_apache_beam_model_job_management_v1_beam_job_api_proto_msgTypes[23].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*PipelineOptionType); i { - case 0: - return &v.state - case 1: - return &v.sizeCache - case 2: - return &v.unknownFields - default: - return nil - } - } - file_org_apache_beam_model_job_management_v1_beam_job_api_proto_msgTypes[24].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*PipelineOptionDescriptor); i { - case 0: - return &v.state - case 1: - return &v.sizeCache - case 2: - return &v.unknownFields - default: - return nil - } - } - file_org_apache_beam_model_job_management_v1_beam_job_api_proto_msgTypes[25].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*DescribePipelineOptionsResponse); i { - case 0: - return &v.state - case 1: - return &v.sizeCache - case 2: - return &v.unknownFields - default: - return nil - } - } - } - file_org_apache_beam_model_job_management_v1_beam_job_api_proto_msgTypes[17].OneofWrappers = []interface{}{ + file_org_apache_beam_model_job_management_v1_beam_job_api_proto_msgTypes[17].OneofWrappers = []any{ (*JobMessagesResponse_MessageResponse)(nil), (*JobMessagesResponse_StateResponse)(nil), } diff --git a/sdks/go/pkg/beam/model/jobmanagement_v1/beam_job_api_grpc.pb.go b/sdks/go/pkg/beam/model/jobmanagement_v1/beam_job_api_grpc.pb.go index da6fe3d027f6..d1ba683382f9 100644 --- a/sdks/go/pkg/beam/model/jobmanagement_v1/beam_job_api_grpc.pb.go +++ b/sdks/go/pkg/beam/model/jobmanagement_v1/beam_job_api_grpc.pb.go @@ -14,9 +14,14 @@ // WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. // See the License for the specific language governing permissions and // limitations under the License. + +// +// Protocol Buffers describing the Job API, api for communicating with a runner +// for job submission over GRPC. + // Code generated by protoc-gen-go-grpc. DO NOT EDIT. // versions: -// - protoc-gen-go-grpc v1.1.0 +// - protoc-gen-go-grpc v1.5.1 // - protoc v5.27.3 // source: org/apache/beam/model/job_management/v1/beam_job_api.proto @@ -31,12 +36,28 @@ import ( // This is a compile-time assertion to ensure that this generated file // is compatible with the grpc package it is being compiled against. -// Requires gRPC-Go v1.32.0 or later. -const _ = grpc.SupportPackageIsVersion7 +// Requires gRPC-Go v1.64.0 or later. +const _ = grpc.SupportPackageIsVersion9 + +const ( + JobService_Prepare_FullMethodName = "/org.apache.beam.model.job_management.v1.JobService/Prepare" + JobService_Run_FullMethodName = "/org.apache.beam.model.job_management.v1.JobService/Run" + JobService_GetJobs_FullMethodName = "/org.apache.beam.model.job_management.v1.JobService/GetJobs" + JobService_GetState_FullMethodName = "/org.apache.beam.model.job_management.v1.JobService/GetState" + JobService_GetPipeline_FullMethodName = "/org.apache.beam.model.job_management.v1.JobService/GetPipeline" + JobService_Cancel_FullMethodName = "/org.apache.beam.model.job_management.v1.JobService/Cancel" + JobService_Drain_FullMethodName = "/org.apache.beam.model.job_management.v1.JobService/Drain" + JobService_GetStateStream_FullMethodName = "/org.apache.beam.model.job_management.v1.JobService/GetStateStream" + JobService_GetMessageStream_FullMethodName = "/org.apache.beam.model.job_management.v1.JobService/GetMessageStream" + JobService_GetJobMetrics_FullMethodName = "/org.apache.beam.model.job_management.v1.JobService/GetJobMetrics" + JobService_DescribePipelineOptions_FullMethodName = "/org.apache.beam.model.job_management.v1.JobService/DescribePipelineOptions" +) // JobServiceClient is the client API for JobService service. // // For semantics around ctx use and closing/ending streaming RPCs, please refer to https://pkg.go.dev/google.golang.org/grpc/?tab=doc#ClientConn.NewStream. +// +// Job Service for running RunnerAPI pipelines type JobServiceClient interface { // Prepare a job for execution. The job will not be executed until a call is made to run with the // returned preparationId. @@ -54,9 +75,9 @@ type JobServiceClient interface { // Drain the job Drain(ctx context.Context, in *DrainJobRequest, opts ...grpc.CallOption) (*DrainJobResponse, error) // Subscribe to a stream of state changes of the job, will immediately return the current state of the job as the first response. - GetStateStream(ctx context.Context, in *GetJobStateRequest, opts ...grpc.CallOption) (JobService_GetStateStreamClient, error) + GetStateStream(ctx context.Context, in *GetJobStateRequest, opts ...grpc.CallOption) (grpc.ServerStreamingClient[JobStateEvent], error) // Subscribe to a stream of state changes and messages from the job - GetMessageStream(ctx context.Context, in *JobMessagesRequest, opts ...grpc.CallOption) (JobService_GetMessageStreamClient, error) + GetMessageStream(ctx context.Context, in *JobMessagesRequest, opts ...grpc.CallOption) (grpc.ServerStreamingClient[JobMessagesResponse], error) // Fetch metrics for a given job GetJobMetrics(ctx context.Context, in *GetJobMetricsRequest, opts ...grpc.CallOption) (*GetJobMetricsResponse, error) // Get the supported pipeline options of the runner @@ -72,8 +93,9 @@ func NewJobServiceClient(cc grpc.ClientConnInterface) JobServiceClient { } func (c *jobServiceClient) Prepare(ctx context.Context, in *PrepareJobRequest, opts ...grpc.CallOption) (*PrepareJobResponse, error) { + cOpts := append([]grpc.CallOption{grpc.StaticMethod()}, opts...) out := new(PrepareJobResponse) - err := c.cc.Invoke(ctx, "/org.apache.beam.model.job_management.v1.JobService/Prepare", in, out, opts...) + err := c.cc.Invoke(ctx, JobService_Prepare_FullMethodName, in, out, cOpts...) if err != nil { return nil, err } @@ -81,8 +103,9 @@ func (c *jobServiceClient) Prepare(ctx context.Context, in *PrepareJobRequest, o } func (c *jobServiceClient) Run(ctx context.Context, in *RunJobRequest, opts ...grpc.CallOption) (*RunJobResponse, error) { + cOpts := append([]grpc.CallOption{grpc.StaticMethod()}, opts...) out := new(RunJobResponse) - err := c.cc.Invoke(ctx, "/org.apache.beam.model.job_management.v1.JobService/Run", in, out, opts...) + err := c.cc.Invoke(ctx, JobService_Run_FullMethodName, in, out, cOpts...) if err != nil { return nil, err } @@ -90,8 +113,9 @@ func (c *jobServiceClient) Run(ctx context.Context, in *RunJobRequest, opts ...g } func (c *jobServiceClient) GetJobs(ctx context.Context, in *GetJobsRequest, opts ...grpc.CallOption) (*GetJobsResponse, error) { + cOpts := append([]grpc.CallOption{grpc.StaticMethod()}, opts...) out := new(GetJobsResponse) - err := c.cc.Invoke(ctx, "/org.apache.beam.model.job_management.v1.JobService/GetJobs", in, out, opts...) + err := c.cc.Invoke(ctx, JobService_GetJobs_FullMethodName, in, out, cOpts...) if err != nil { return nil, err } @@ -99,8 +123,9 @@ func (c *jobServiceClient) GetJobs(ctx context.Context, in *GetJobsRequest, opts } func (c *jobServiceClient) GetState(ctx context.Context, in *GetJobStateRequest, opts ...grpc.CallOption) (*JobStateEvent, error) { + cOpts := append([]grpc.CallOption{grpc.StaticMethod()}, opts...) out := new(JobStateEvent) - err := c.cc.Invoke(ctx, "/org.apache.beam.model.job_management.v1.JobService/GetState", in, out, opts...) + err := c.cc.Invoke(ctx, JobService_GetState_FullMethodName, in, out, cOpts...) if err != nil { return nil, err } @@ -108,8 +133,9 @@ func (c *jobServiceClient) GetState(ctx context.Context, in *GetJobStateRequest, } func (c *jobServiceClient) GetPipeline(ctx context.Context, in *GetJobPipelineRequest, opts ...grpc.CallOption) (*GetJobPipelineResponse, error) { + cOpts := append([]grpc.CallOption{grpc.StaticMethod()}, opts...) out := new(GetJobPipelineResponse) - err := c.cc.Invoke(ctx, "/org.apache.beam.model.job_management.v1.JobService/GetPipeline", in, out, opts...) + err := c.cc.Invoke(ctx, JobService_GetPipeline_FullMethodName, in, out, cOpts...) if err != nil { return nil, err } @@ -117,8 +143,9 @@ func (c *jobServiceClient) GetPipeline(ctx context.Context, in *GetJobPipelineRe } func (c *jobServiceClient) Cancel(ctx context.Context, in *CancelJobRequest, opts ...grpc.CallOption) (*CancelJobResponse, error) { + cOpts := append([]grpc.CallOption{grpc.StaticMethod()}, opts...) out := new(CancelJobResponse) - err := c.cc.Invoke(ctx, "/org.apache.beam.model.job_management.v1.JobService/Cancel", in, out, opts...) + err := c.cc.Invoke(ctx, JobService_Cancel_FullMethodName, in, out, cOpts...) if err != nil { return nil, err } @@ -126,20 +153,22 @@ func (c *jobServiceClient) Cancel(ctx context.Context, in *CancelJobRequest, opt } func (c *jobServiceClient) Drain(ctx context.Context, in *DrainJobRequest, opts ...grpc.CallOption) (*DrainJobResponse, error) { + cOpts := append([]grpc.CallOption{grpc.StaticMethod()}, opts...) out := new(DrainJobResponse) - err := c.cc.Invoke(ctx, "/org.apache.beam.model.job_management.v1.JobService/Drain", in, out, opts...) + err := c.cc.Invoke(ctx, JobService_Drain_FullMethodName, in, out, cOpts...) if err != nil { return nil, err } return out, nil } -func (c *jobServiceClient) GetStateStream(ctx context.Context, in *GetJobStateRequest, opts ...grpc.CallOption) (JobService_GetStateStreamClient, error) { - stream, err := c.cc.NewStream(ctx, &JobService_ServiceDesc.Streams[0], "/org.apache.beam.model.job_management.v1.JobService/GetStateStream", opts...) +func (c *jobServiceClient) GetStateStream(ctx context.Context, in *GetJobStateRequest, opts ...grpc.CallOption) (grpc.ServerStreamingClient[JobStateEvent], error) { + cOpts := append([]grpc.CallOption{grpc.StaticMethod()}, opts...) + stream, err := c.cc.NewStream(ctx, &JobService_ServiceDesc.Streams[0], JobService_GetStateStream_FullMethodName, cOpts...) if err != nil { return nil, err } - x := &jobServiceGetStateStreamClient{stream} + x := &grpc.GenericClientStream[GetJobStateRequest, JobStateEvent]{ClientStream: stream} if err := x.ClientStream.SendMsg(in); err != nil { return nil, err } @@ -149,29 +178,16 @@ func (c *jobServiceClient) GetStateStream(ctx context.Context, in *GetJobStateRe return x, nil } -type JobService_GetStateStreamClient interface { - Recv() (*JobStateEvent, error) - grpc.ClientStream -} - -type jobServiceGetStateStreamClient struct { - grpc.ClientStream -} +// This type alias is provided for backwards compatibility with existing code that references the prior non-generic stream type by name. +type JobService_GetStateStreamClient = grpc.ServerStreamingClient[JobStateEvent] -func (x *jobServiceGetStateStreamClient) Recv() (*JobStateEvent, error) { - m := new(JobStateEvent) - if err := x.ClientStream.RecvMsg(m); err != nil { - return nil, err - } - return m, nil -} - -func (c *jobServiceClient) GetMessageStream(ctx context.Context, in *JobMessagesRequest, opts ...grpc.CallOption) (JobService_GetMessageStreamClient, error) { - stream, err := c.cc.NewStream(ctx, &JobService_ServiceDesc.Streams[1], "/org.apache.beam.model.job_management.v1.JobService/GetMessageStream", opts...) +func (c *jobServiceClient) GetMessageStream(ctx context.Context, in *JobMessagesRequest, opts ...grpc.CallOption) (grpc.ServerStreamingClient[JobMessagesResponse], error) { + cOpts := append([]grpc.CallOption{grpc.StaticMethod()}, opts...) + stream, err := c.cc.NewStream(ctx, &JobService_ServiceDesc.Streams[1], JobService_GetMessageStream_FullMethodName, cOpts...) if err != nil { return nil, err } - x := &jobServiceGetMessageStreamClient{stream} + x := &grpc.GenericClientStream[JobMessagesRequest, JobMessagesResponse]{ClientStream: stream} if err := x.ClientStream.SendMsg(in); err != nil { return nil, err } @@ -181,26 +197,13 @@ func (c *jobServiceClient) GetMessageStream(ctx context.Context, in *JobMessages return x, nil } -type JobService_GetMessageStreamClient interface { - Recv() (*JobMessagesResponse, error) - grpc.ClientStream -} - -type jobServiceGetMessageStreamClient struct { - grpc.ClientStream -} - -func (x *jobServiceGetMessageStreamClient) Recv() (*JobMessagesResponse, error) { - m := new(JobMessagesResponse) - if err := x.ClientStream.RecvMsg(m); err != nil { - return nil, err - } - return m, nil -} +// This type alias is provided for backwards compatibility with existing code that references the prior non-generic stream type by name. +type JobService_GetMessageStreamClient = grpc.ServerStreamingClient[JobMessagesResponse] func (c *jobServiceClient) GetJobMetrics(ctx context.Context, in *GetJobMetricsRequest, opts ...grpc.CallOption) (*GetJobMetricsResponse, error) { + cOpts := append([]grpc.CallOption{grpc.StaticMethod()}, opts...) out := new(GetJobMetricsResponse) - err := c.cc.Invoke(ctx, "/org.apache.beam.model.job_management.v1.JobService/GetJobMetrics", in, out, opts...) + err := c.cc.Invoke(ctx, JobService_GetJobMetrics_FullMethodName, in, out, cOpts...) if err != nil { return nil, err } @@ -208,8 +211,9 @@ func (c *jobServiceClient) GetJobMetrics(ctx context.Context, in *GetJobMetricsR } func (c *jobServiceClient) DescribePipelineOptions(ctx context.Context, in *DescribePipelineOptionsRequest, opts ...grpc.CallOption) (*DescribePipelineOptionsResponse, error) { + cOpts := append([]grpc.CallOption{grpc.StaticMethod()}, opts...) out := new(DescribePipelineOptionsResponse) - err := c.cc.Invoke(ctx, "/org.apache.beam.model.job_management.v1.JobService/DescribePipelineOptions", in, out, opts...) + err := c.cc.Invoke(ctx, JobService_DescribePipelineOptions_FullMethodName, in, out, cOpts...) if err != nil { return nil, err } @@ -218,7 +222,9 @@ func (c *jobServiceClient) DescribePipelineOptions(ctx context.Context, in *Desc // JobServiceServer is the server API for JobService service. // All implementations must embed UnimplementedJobServiceServer -// for forward compatibility +// for forward compatibility. +// +// Job Service for running RunnerAPI pipelines type JobServiceServer interface { // Prepare a job for execution. The job will not be executed until a call is made to run with the // returned preparationId. @@ -236,9 +242,9 @@ type JobServiceServer interface { // Drain the job Drain(context.Context, *DrainJobRequest) (*DrainJobResponse, error) // Subscribe to a stream of state changes of the job, will immediately return the current state of the job as the first response. - GetStateStream(*GetJobStateRequest, JobService_GetStateStreamServer) error + GetStateStream(*GetJobStateRequest, grpc.ServerStreamingServer[JobStateEvent]) error // Subscribe to a stream of state changes and messages from the job - GetMessageStream(*JobMessagesRequest, JobService_GetMessageStreamServer) error + GetMessageStream(*JobMessagesRequest, grpc.ServerStreamingServer[JobMessagesResponse]) error // Fetch metrics for a given job GetJobMetrics(context.Context, *GetJobMetricsRequest) (*GetJobMetricsResponse, error) // Get the supported pipeline options of the runner @@ -246,9 +252,12 @@ type JobServiceServer interface { mustEmbedUnimplementedJobServiceServer() } -// UnimplementedJobServiceServer must be embedded to have forward compatible implementations. -type UnimplementedJobServiceServer struct { -} +// UnimplementedJobServiceServer must be embedded to have +// forward compatible implementations. +// +// NOTE: this should be embedded by value instead of pointer to avoid a nil +// pointer dereference when methods are called. +type UnimplementedJobServiceServer struct{} func (UnimplementedJobServiceServer) Prepare(context.Context, *PrepareJobRequest) (*PrepareJobResponse, error) { return nil, status.Errorf(codes.Unimplemented, "method Prepare not implemented") @@ -271,10 +280,10 @@ func (UnimplementedJobServiceServer) Cancel(context.Context, *CancelJobRequest) func (UnimplementedJobServiceServer) Drain(context.Context, *DrainJobRequest) (*DrainJobResponse, error) { return nil, status.Errorf(codes.Unimplemented, "method Drain not implemented") } -func (UnimplementedJobServiceServer) GetStateStream(*GetJobStateRequest, JobService_GetStateStreamServer) error { +func (UnimplementedJobServiceServer) GetStateStream(*GetJobStateRequest, grpc.ServerStreamingServer[JobStateEvent]) error { return status.Errorf(codes.Unimplemented, "method GetStateStream not implemented") } -func (UnimplementedJobServiceServer) GetMessageStream(*JobMessagesRequest, JobService_GetMessageStreamServer) error { +func (UnimplementedJobServiceServer) GetMessageStream(*JobMessagesRequest, grpc.ServerStreamingServer[JobMessagesResponse]) error { return status.Errorf(codes.Unimplemented, "method GetMessageStream not implemented") } func (UnimplementedJobServiceServer) GetJobMetrics(context.Context, *GetJobMetricsRequest) (*GetJobMetricsResponse, error) { @@ -284,6 +293,7 @@ func (UnimplementedJobServiceServer) DescribePipelineOptions(context.Context, *D return nil, status.Errorf(codes.Unimplemented, "method DescribePipelineOptions not implemented") } func (UnimplementedJobServiceServer) mustEmbedUnimplementedJobServiceServer() {} +func (UnimplementedJobServiceServer) testEmbeddedByValue() {} // UnsafeJobServiceServer may be embedded to opt out of forward compatibility for this service. // Use of this interface is not recommended, as added methods to JobServiceServer will @@ -293,6 +303,13 @@ type UnsafeJobServiceServer interface { } func RegisterJobServiceServer(s grpc.ServiceRegistrar, srv JobServiceServer) { + // If the following call pancis, it indicates UnimplementedJobServiceServer was + // embedded by pointer and is nil. This will cause panics if an + // unimplemented method is ever invoked, so we test this at initialization + // time to prevent it from happening at runtime later due to I/O. + if t, ok := srv.(interface{ testEmbeddedByValue() }); ok { + t.testEmbeddedByValue() + } s.RegisterService(&JobService_ServiceDesc, srv) } @@ -306,7 +323,7 @@ func _JobService_Prepare_Handler(srv interface{}, ctx context.Context, dec func( } info := &grpc.UnaryServerInfo{ Server: srv, - FullMethod: "/org.apache.beam.model.job_management.v1.JobService/Prepare", + FullMethod: JobService_Prepare_FullMethodName, } handler := func(ctx context.Context, req interface{}) (interface{}, error) { return srv.(JobServiceServer).Prepare(ctx, req.(*PrepareJobRequest)) @@ -324,7 +341,7 @@ func _JobService_Run_Handler(srv interface{}, ctx context.Context, dec func(inte } info := &grpc.UnaryServerInfo{ Server: srv, - FullMethod: "/org.apache.beam.model.job_management.v1.JobService/Run", + FullMethod: JobService_Run_FullMethodName, } handler := func(ctx context.Context, req interface{}) (interface{}, error) { return srv.(JobServiceServer).Run(ctx, req.(*RunJobRequest)) @@ -342,7 +359,7 @@ func _JobService_GetJobs_Handler(srv interface{}, ctx context.Context, dec func( } info := &grpc.UnaryServerInfo{ Server: srv, - FullMethod: "/org.apache.beam.model.job_management.v1.JobService/GetJobs", + FullMethod: JobService_GetJobs_FullMethodName, } handler := func(ctx context.Context, req interface{}) (interface{}, error) { return srv.(JobServiceServer).GetJobs(ctx, req.(*GetJobsRequest)) @@ -360,7 +377,7 @@ func _JobService_GetState_Handler(srv interface{}, ctx context.Context, dec func } info := &grpc.UnaryServerInfo{ Server: srv, - FullMethod: "/org.apache.beam.model.job_management.v1.JobService/GetState", + FullMethod: JobService_GetState_FullMethodName, } handler := func(ctx context.Context, req interface{}) (interface{}, error) { return srv.(JobServiceServer).GetState(ctx, req.(*GetJobStateRequest)) @@ -378,7 +395,7 @@ func _JobService_GetPipeline_Handler(srv interface{}, ctx context.Context, dec f } info := &grpc.UnaryServerInfo{ Server: srv, - FullMethod: "/org.apache.beam.model.job_management.v1.JobService/GetPipeline", + FullMethod: JobService_GetPipeline_FullMethodName, } handler := func(ctx context.Context, req interface{}) (interface{}, error) { return srv.(JobServiceServer).GetPipeline(ctx, req.(*GetJobPipelineRequest)) @@ -396,7 +413,7 @@ func _JobService_Cancel_Handler(srv interface{}, ctx context.Context, dec func(i } info := &grpc.UnaryServerInfo{ Server: srv, - FullMethod: "/org.apache.beam.model.job_management.v1.JobService/Cancel", + FullMethod: JobService_Cancel_FullMethodName, } handler := func(ctx context.Context, req interface{}) (interface{}, error) { return srv.(JobServiceServer).Cancel(ctx, req.(*CancelJobRequest)) @@ -414,7 +431,7 @@ func _JobService_Drain_Handler(srv interface{}, ctx context.Context, dec func(in } info := &grpc.UnaryServerInfo{ Server: srv, - FullMethod: "/org.apache.beam.model.job_management.v1.JobService/Drain", + FullMethod: JobService_Drain_FullMethodName, } handler := func(ctx context.Context, req interface{}) (interface{}, error) { return srv.(JobServiceServer).Drain(ctx, req.(*DrainJobRequest)) @@ -427,42 +444,22 @@ func _JobService_GetStateStream_Handler(srv interface{}, stream grpc.ServerStrea if err := stream.RecvMsg(m); err != nil { return err } - return srv.(JobServiceServer).GetStateStream(m, &jobServiceGetStateStreamServer{stream}) -} - -type JobService_GetStateStreamServer interface { - Send(*JobStateEvent) error - grpc.ServerStream + return srv.(JobServiceServer).GetStateStream(m, &grpc.GenericServerStream[GetJobStateRequest, JobStateEvent]{ServerStream: stream}) } -type jobServiceGetStateStreamServer struct { - grpc.ServerStream -} - -func (x *jobServiceGetStateStreamServer) Send(m *JobStateEvent) error { - return x.ServerStream.SendMsg(m) -} +// This type alias is provided for backwards compatibility with existing code that references the prior non-generic stream type by name. +type JobService_GetStateStreamServer = grpc.ServerStreamingServer[JobStateEvent] func _JobService_GetMessageStream_Handler(srv interface{}, stream grpc.ServerStream) error { m := new(JobMessagesRequest) if err := stream.RecvMsg(m); err != nil { return err } - return srv.(JobServiceServer).GetMessageStream(m, &jobServiceGetMessageStreamServer{stream}) -} - -type JobService_GetMessageStreamServer interface { - Send(*JobMessagesResponse) error - grpc.ServerStream -} - -type jobServiceGetMessageStreamServer struct { - grpc.ServerStream + return srv.(JobServiceServer).GetMessageStream(m, &grpc.GenericServerStream[JobMessagesRequest, JobMessagesResponse]{ServerStream: stream}) } -func (x *jobServiceGetMessageStreamServer) Send(m *JobMessagesResponse) error { - return x.ServerStream.SendMsg(m) -} +// This type alias is provided for backwards compatibility with existing code that references the prior non-generic stream type by name. +type JobService_GetMessageStreamServer = grpc.ServerStreamingServer[JobMessagesResponse] func _JobService_GetJobMetrics_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { in := new(GetJobMetricsRequest) @@ -474,7 +471,7 @@ func _JobService_GetJobMetrics_Handler(srv interface{}, ctx context.Context, dec } info := &grpc.UnaryServerInfo{ Server: srv, - FullMethod: "/org.apache.beam.model.job_management.v1.JobService/GetJobMetrics", + FullMethod: JobService_GetJobMetrics_FullMethodName, } handler := func(ctx context.Context, req interface{}) (interface{}, error) { return srv.(JobServiceServer).GetJobMetrics(ctx, req.(*GetJobMetricsRequest)) @@ -492,7 +489,7 @@ func _JobService_DescribePipelineOptions_Handler(srv interface{}, ctx context.Co } info := &grpc.UnaryServerInfo{ Server: srv, - FullMethod: "/org.apache.beam.model.job_management.v1.JobService/DescribePipelineOptions", + FullMethod: JobService_DescribePipelineOptions_FullMethodName, } handler := func(ctx context.Context, req interface{}) (interface{}, error) { return srv.(JobServiceServer).DescribePipelineOptions(ctx, req.(*DescribePipelineOptionsRequest)) diff --git a/sdks/go/pkg/beam/model/jobmanagement_v1/beam_job_api_protoopaque.pb.go b/sdks/go/pkg/beam/model/jobmanagement_v1/beam_job_api_protoopaque.pb.go new file mode 100644 index 000000000000..39b31839886c --- /dev/null +++ b/sdks/go/pkg/beam/model/jobmanagement_v1/beam_job_api_protoopaque.pb.go @@ -0,0 +1,2611 @@ +// +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +// +// Protocol Buffers describing the Job API, api for communicating with a runner +// for job submission over GRPC. + +// Code generated by protoc-gen-go. DO NOT EDIT. +// versions: +// protoc-gen-go v1.36.0 +// protoc v5.27.3 +// source: org/apache/beam/model/job_management/v1/beam_job_api.proto + +//go:build protoopaque + +package jobmanagement_v1 + +import ( + pipeline_v1 "github.com/apache/beam/sdks/v2/go/pkg/beam/model/pipeline_v1" + protoreflect "google.golang.org/protobuf/reflect/protoreflect" + protoimpl "google.golang.org/protobuf/runtime/protoimpl" + structpb "google.golang.org/protobuf/types/known/structpb" + timestamppb "google.golang.org/protobuf/types/known/timestamppb" + reflect "reflect" +) + +const ( + // Verify that this generated code is sufficiently up-to-date. + _ = protoimpl.EnforceVersion(20 - protoimpl.MinVersion) + // Verify that runtime/protoimpl is sufficiently up-to-date. + _ = protoimpl.EnforceVersion(protoimpl.MaxVersion - 20) +) + +type JobMessage_MessageImportance int32 + +const ( + JobMessage_MESSAGE_IMPORTANCE_UNSPECIFIED JobMessage_MessageImportance = 0 + JobMessage_JOB_MESSAGE_DEBUG JobMessage_MessageImportance = 1 + JobMessage_JOB_MESSAGE_DETAILED JobMessage_MessageImportance = 2 + JobMessage_JOB_MESSAGE_BASIC JobMessage_MessageImportance = 3 + JobMessage_JOB_MESSAGE_WARNING JobMessage_MessageImportance = 4 + JobMessage_JOB_MESSAGE_ERROR JobMessage_MessageImportance = 5 +) + +// Enum value maps for JobMessage_MessageImportance. +var ( + JobMessage_MessageImportance_name = map[int32]string{ + 0: "MESSAGE_IMPORTANCE_UNSPECIFIED", + 1: "JOB_MESSAGE_DEBUG", + 2: "JOB_MESSAGE_DETAILED", + 3: "JOB_MESSAGE_BASIC", + 4: "JOB_MESSAGE_WARNING", + 5: "JOB_MESSAGE_ERROR", + } + JobMessage_MessageImportance_value = map[string]int32{ + "MESSAGE_IMPORTANCE_UNSPECIFIED": 0, + "JOB_MESSAGE_DEBUG": 1, + "JOB_MESSAGE_DETAILED": 2, + "JOB_MESSAGE_BASIC": 3, + "JOB_MESSAGE_WARNING": 4, + "JOB_MESSAGE_ERROR": 5, + } +) + +func (x JobMessage_MessageImportance) Enum() *JobMessage_MessageImportance { + p := new(JobMessage_MessageImportance) + *p = x + return p +} + +func (x JobMessage_MessageImportance) String() string { + return protoimpl.X.EnumStringOf(x.Descriptor(), protoreflect.EnumNumber(x)) +} + +func (JobMessage_MessageImportance) Descriptor() protoreflect.EnumDescriptor { + return file_org_apache_beam_model_job_management_v1_beam_job_api_proto_enumTypes[0].Descriptor() +} + +func (JobMessage_MessageImportance) Type() protoreflect.EnumType { + return &file_org_apache_beam_model_job_management_v1_beam_job_api_proto_enumTypes[0] +} + +func (x JobMessage_MessageImportance) Number() protoreflect.EnumNumber { + return protoreflect.EnumNumber(x) +} + +type JobState_Enum int32 + +const ( + // The job state reported by a runner cannot be interpreted by the SDK. + JobState_UNSPECIFIED JobState_Enum = 0 + // The job has not yet started. + JobState_STOPPED JobState_Enum = 1 + // The job is currently running. + JobState_RUNNING JobState_Enum = 2 + // The job has successfully completed. (terminal) + JobState_DONE JobState_Enum = 3 + // The job has failed. (terminal) + JobState_FAILED JobState_Enum = 4 + // The job has been explicitly cancelled. (terminal) + JobState_CANCELLED JobState_Enum = 5 + // The job has been updated. (terminal) + JobState_UPDATED JobState_Enum = 6 + // The job is draining its data. (optional) + JobState_DRAINING JobState_Enum = 7 + // The job has completed draining its data. (terminal) + JobState_DRAINED JobState_Enum = 8 + // The job is starting up. + JobState_STARTING JobState_Enum = 9 + // The job is cancelling. (optional) + JobState_CANCELLING JobState_Enum = 10 + // The job is in the process of being updated. (optional) + JobState_UPDATING JobState_Enum = 11 +) + +// Enum value maps for JobState_Enum. +var ( + JobState_Enum_name = map[int32]string{ + 0: "UNSPECIFIED", + 1: "STOPPED", + 2: "RUNNING", + 3: "DONE", + 4: "FAILED", + 5: "CANCELLED", + 6: "UPDATED", + 7: "DRAINING", + 8: "DRAINED", + 9: "STARTING", + 10: "CANCELLING", + 11: "UPDATING", + } + JobState_Enum_value = map[string]int32{ + "UNSPECIFIED": 0, + "STOPPED": 1, + "RUNNING": 2, + "DONE": 3, + "FAILED": 4, + "CANCELLED": 5, + "UPDATED": 6, + "DRAINING": 7, + "DRAINED": 8, + "STARTING": 9, + "CANCELLING": 10, + "UPDATING": 11, + } +) + +func (x JobState_Enum) Enum() *JobState_Enum { + p := new(JobState_Enum) + *p = x + return p +} + +func (x JobState_Enum) String() string { + return protoimpl.X.EnumStringOf(x.Descriptor(), protoreflect.EnumNumber(x)) +} + +func (JobState_Enum) Descriptor() protoreflect.EnumDescriptor { + return file_org_apache_beam_model_job_management_v1_beam_job_api_proto_enumTypes[1].Descriptor() +} + +func (JobState_Enum) Type() protoreflect.EnumType { + return &file_org_apache_beam_model_job_management_v1_beam_job_api_proto_enumTypes[1] +} + +func (x JobState_Enum) Number() protoreflect.EnumNumber { + return protoreflect.EnumNumber(x) +} + +type PipelineOptionType_Enum int32 + +const ( + PipelineOptionType_STRING PipelineOptionType_Enum = 0 + PipelineOptionType_BOOLEAN PipelineOptionType_Enum = 1 + // whole numbers, see https://json-schema.org/understanding-json-schema/reference/numeric.html + PipelineOptionType_INTEGER PipelineOptionType_Enum = 2 + PipelineOptionType_NUMBER PipelineOptionType_Enum = 3 + PipelineOptionType_ARRAY PipelineOptionType_Enum = 4 + PipelineOptionType_OBJECT PipelineOptionType_Enum = 5 +) + +// Enum value maps for PipelineOptionType_Enum. +var ( + PipelineOptionType_Enum_name = map[int32]string{ + 0: "STRING", + 1: "BOOLEAN", + 2: "INTEGER", + 3: "NUMBER", + 4: "ARRAY", + 5: "OBJECT", + } + PipelineOptionType_Enum_value = map[string]int32{ + "STRING": 0, + "BOOLEAN": 1, + "INTEGER": 2, + "NUMBER": 3, + "ARRAY": 4, + "OBJECT": 5, + } +) + +func (x PipelineOptionType_Enum) Enum() *PipelineOptionType_Enum { + p := new(PipelineOptionType_Enum) + *p = x + return p +} + +func (x PipelineOptionType_Enum) String() string { + return protoimpl.X.EnumStringOf(x.Descriptor(), protoreflect.EnumNumber(x)) +} + +func (PipelineOptionType_Enum) Descriptor() protoreflect.EnumDescriptor { + return file_org_apache_beam_model_job_management_v1_beam_job_api_proto_enumTypes[2].Descriptor() +} + +func (PipelineOptionType_Enum) Type() protoreflect.EnumType { + return &file_org_apache_beam_model_job_management_v1_beam_job_api_proto_enumTypes[2] +} + +func (x PipelineOptionType_Enum) Number() protoreflect.EnumNumber { + return protoreflect.EnumNumber(x) +} + +// Prepare is a synchronous request that returns a preparationId back +// Throws error GRPC_STATUS_UNAVAILABLE if server is down +// Throws error ALREADY_EXISTS if the jobName is reused. Runners are permitted to deduplicate based on the name of the job. +// Throws error UNKNOWN for all other issues +type PrepareJobRequest struct { + state protoimpl.MessageState `protogen:"opaque.v1"` + xxx_hidden_Pipeline *pipeline_v1.Pipeline `protobuf:"bytes,1,opt,name=pipeline,proto3" json:"pipeline,omitempty"` + xxx_hidden_PipelineOptions *structpb.Struct `protobuf:"bytes,2,opt,name=pipeline_options,json=pipelineOptions,proto3" json:"pipeline_options,omitempty"` + xxx_hidden_JobName string `protobuf:"bytes,3,opt,name=job_name,json=jobName,proto3" json:"job_name,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *PrepareJobRequest) Reset() { + *x = PrepareJobRequest{} + mi := &file_org_apache_beam_model_job_management_v1_beam_job_api_proto_msgTypes[0] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *PrepareJobRequest) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*PrepareJobRequest) ProtoMessage() {} + +func (x *PrepareJobRequest) ProtoReflect() protoreflect.Message { + mi := &file_org_apache_beam_model_job_management_v1_beam_job_api_proto_msgTypes[0] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +func (x *PrepareJobRequest) GetPipeline() *pipeline_v1.Pipeline { + if x != nil { + return x.xxx_hidden_Pipeline + } + return nil +} + +func (x *PrepareJobRequest) GetPipelineOptions() *structpb.Struct { + if x != nil { + return x.xxx_hidden_PipelineOptions + } + return nil +} + +func (x *PrepareJobRequest) GetJobName() string { + if x != nil { + return x.xxx_hidden_JobName + } + return "" +} + +func (x *PrepareJobRequest) SetPipeline(v *pipeline_v1.Pipeline) { + x.xxx_hidden_Pipeline = v +} + +func (x *PrepareJobRequest) SetPipelineOptions(v *structpb.Struct) { + x.xxx_hidden_PipelineOptions = v +} + +func (x *PrepareJobRequest) SetJobName(v string) { + x.xxx_hidden_JobName = v +} + +func (x *PrepareJobRequest) HasPipeline() bool { + if x == nil { + return false + } + return x.xxx_hidden_Pipeline != nil +} + +func (x *PrepareJobRequest) HasPipelineOptions() bool { + if x == nil { + return false + } + return x.xxx_hidden_PipelineOptions != nil +} + +func (x *PrepareJobRequest) ClearPipeline() { + x.xxx_hidden_Pipeline = nil +} + +func (x *PrepareJobRequest) ClearPipelineOptions() { + x.xxx_hidden_PipelineOptions = nil +} + +type PrepareJobRequest_builder struct { + _ [0]func() // Prevents comparability and use of unkeyed literals for the builder. + + Pipeline *pipeline_v1.Pipeline + PipelineOptions *structpb.Struct + JobName string +} + +func (b0 PrepareJobRequest_builder) Build() *PrepareJobRequest { + m0 := &PrepareJobRequest{} + b, x := &b0, m0 + _, _ = b, x + x.xxx_hidden_Pipeline = b.Pipeline + x.xxx_hidden_PipelineOptions = b.PipelineOptions + x.xxx_hidden_JobName = b.JobName + return m0 +} + +type PrepareJobResponse struct { + state protoimpl.MessageState `protogen:"opaque.v1"` + xxx_hidden_PreparationId string `protobuf:"bytes,1,opt,name=preparation_id,json=preparationId,proto3" json:"preparation_id,omitempty"` + xxx_hidden_ArtifactStagingEndpoint *pipeline_v1.ApiServiceDescriptor `protobuf:"bytes,2,opt,name=artifact_staging_endpoint,json=artifactStagingEndpoint,proto3" json:"artifact_staging_endpoint,omitempty"` + xxx_hidden_StagingSessionToken string `protobuf:"bytes,3,opt,name=staging_session_token,json=stagingSessionToken,proto3" json:"staging_session_token,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *PrepareJobResponse) Reset() { + *x = PrepareJobResponse{} + mi := &file_org_apache_beam_model_job_management_v1_beam_job_api_proto_msgTypes[1] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *PrepareJobResponse) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*PrepareJobResponse) ProtoMessage() {} + +func (x *PrepareJobResponse) ProtoReflect() protoreflect.Message { + mi := &file_org_apache_beam_model_job_management_v1_beam_job_api_proto_msgTypes[1] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +func (x *PrepareJobResponse) GetPreparationId() string { + if x != nil { + return x.xxx_hidden_PreparationId + } + return "" +} + +func (x *PrepareJobResponse) GetArtifactStagingEndpoint() *pipeline_v1.ApiServiceDescriptor { + if x != nil { + return x.xxx_hidden_ArtifactStagingEndpoint + } + return nil +} + +func (x *PrepareJobResponse) GetStagingSessionToken() string { + if x != nil { + return x.xxx_hidden_StagingSessionToken + } + return "" +} + +func (x *PrepareJobResponse) SetPreparationId(v string) { + x.xxx_hidden_PreparationId = v +} + +func (x *PrepareJobResponse) SetArtifactStagingEndpoint(v *pipeline_v1.ApiServiceDescriptor) { + x.xxx_hidden_ArtifactStagingEndpoint = v +} + +func (x *PrepareJobResponse) SetStagingSessionToken(v string) { + x.xxx_hidden_StagingSessionToken = v +} + +func (x *PrepareJobResponse) HasArtifactStagingEndpoint() bool { + if x == nil { + return false + } + return x.xxx_hidden_ArtifactStagingEndpoint != nil +} + +func (x *PrepareJobResponse) ClearArtifactStagingEndpoint() { + x.xxx_hidden_ArtifactStagingEndpoint = nil +} + +type PrepareJobResponse_builder struct { + _ [0]func() // Prevents comparability and use of unkeyed literals for the builder. + + // (required) The ID used to associate calls made while preparing the job. preparationId is used + // to run the job. + PreparationId string + // An endpoint which exposes the Beam Artifact Staging API. Artifacts used by the job should be + // staged to this endpoint, and will be available during job execution. + ArtifactStagingEndpoint *pipeline_v1.ApiServiceDescriptor + // (required) Token for the artifact staging. This token also represent an artifact + // staging session with the artifact staging service. + StagingSessionToken string +} + +func (b0 PrepareJobResponse_builder) Build() *PrepareJobResponse { + m0 := &PrepareJobResponse{} + b, x := &b0, m0 + _, _ = b, x + x.xxx_hidden_PreparationId = b.PreparationId + x.xxx_hidden_ArtifactStagingEndpoint = b.ArtifactStagingEndpoint + x.xxx_hidden_StagingSessionToken = b.StagingSessionToken + return m0 +} + +// Run is a synchronous request that returns a jobId back. +// Throws error GRPC_STATUS_UNAVAILABLE if server is down +// Throws error NOT_FOUND if the preparation ID does not exist +// Throws error UNKNOWN for all other issues +type RunJobRequest struct { + state protoimpl.MessageState `protogen:"opaque.v1"` + xxx_hidden_PreparationId string `protobuf:"bytes,1,opt,name=preparation_id,json=preparationId,proto3" json:"preparation_id,omitempty"` + xxx_hidden_RetrievalToken string `protobuf:"bytes,2,opt,name=retrieval_token,json=retrievalToken,proto3" json:"retrieval_token,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *RunJobRequest) Reset() { + *x = RunJobRequest{} + mi := &file_org_apache_beam_model_job_management_v1_beam_job_api_proto_msgTypes[2] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *RunJobRequest) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*RunJobRequest) ProtoMessage() {} + +func (x *RunJobRequest) ProtoReflect() protoreflect.Message { + mi := &file_org_apache_beam_model_job_management_v1_beam_job_api_proto_msgTypes[2] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +func (x *RunJobRequest) GetPreparationId() string { + if x != nil { + return x.xxx_hidden_PreparationId + } + return "" +} + +func (x *RunJobRequest) GetRetrievalToken() string { + if x != nil { + return x.xxx_hidden_RetrievalToken + } + return "" +} + +func (x *RunJobRequest) SetPreparationId(v string) { + x.xxx_hidden_PreparationId = v +} + +func (x *RunJobRequest) SetRetrievalToken(v string) { + x.xxx_hidden_RetrievalToken = v +} + +type RunJobRequest_builder struct { + _ [0]func() // Prevents comparability and use of unkeyed literals for the builder. + + // (required) The ID provided by an earlier call to prepare. Runs the job. All prerequisite tasks + // must have been completed. + PreparationId string + // (optional) If any artifacts have been staged for this job, contains the retrieval_token returned + // from the CommitManifestResponse. + RetrievalToken string +} + +func (b0 RunJobRequest_builder) Build() *RunJobRequest { + m0 := &RunJobRequest{} + b, x := &b0, m0 + _, _ = b, x + x.xxx_hidden_PreparationId = b.PreparationId + x.xxx_hidden_RetrievalToken = b.RetrievalToken + return m0 +} + +type RunJobResponse struct { + state protoimpl.MessageState `protogen:"opaque.v1"` + xxx_hidden_JobId string `protobuf:"bytes,1,opt,name=job_id,json=jobId,proto3" json:"job_id,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *RunJobResponse) Reset() { + *x = RunJobResponse{} + mi := &file_org_apache_beam_model_job_management_v1_beam_job_api_proto_msgTypes[3] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *RunJobResponse) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*RunJobResponse) ProtoMessage() {} + +func (x *RunJobResponse) ProtoReflect() protoreflect.Message { + mi := &file_org_apache_beam_model_job_management_v1_beam_job_api_proto_msgTypes[3] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +func (x *RunJobResponse) GetJobId() string { + if x != nil { + return x.xxx_hidden_JobId + } + return "" +} + +func (x *RunJobResponse) SetJobId(v string) { + x.xxx_hidden_JobId = v +} + +type RunJobResponse_builder struct { + _ [0]func() // Prevents comparability and use of unkeyed literals for the builder. + + JobId string +} + +func (b0 RunJobResponse_builder) Build() *RunJobResponse { + m0 := &RunJobResponse{} + b, x := &b0, m0 + _, _ = b, x + x.xxx_hidden_JobId = b.JobId + return m0 +} + +// Cancel is a synchronus request that returns a job state back +// Throws error GRPC_STATUS_UNAVAILABLE if server is down +// Throws error NOT_FOUND if the jobId is not found +type CancelJobRequest struct { + state protoimpl.MessageState `protogen:"opaque.v1"` + xxx_hidden_JobId string `protobuf:"bytes,1,opt,name=job_id,json=jobId,proto3" json:"job_id,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *CancelJobRequest) Reset() { + *x = CancelJobRequest{} + mi := &file_org_apache_beam_model_job_management_v1_beam_job_api_proto_msgTypes[4] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *CancelJobRequest) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*CancelJobRequest) ProtoMessage() {} + +func (x *CancelJobRequest) ProtoReflect() protoreflect.Message { + mi := &file_org_apache_beam_model_job_management_v1_beam_job_api_proto_msgTypes[4] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +func (x *CancelJobRequest) GetJobId() string { + if x != nil { + return x.xxx_hidden_JobId + } + return "" +} + +func (x *CancelJobRequest) SetJobId(v string) { + x.xxx_hidden_JobId = v +} + +type CancelJobRequest_builder struct { + _ [0]func() // Prevents comparability and use of unkeyed literals for the builder. + + JobId string +} + +func (b0 CancelJobRequest_builder) Build() *CancelJobRequest { + m0 := &CancelJobRequest{} + b, x := &b0, m0 + _, _ = b, x + x.xxx_hidden_JobId = b.JobId + return m0 +} + +// Valid responses include any terminal state or CANCELLING +type CancelJobResponse struct { + state protoimpl.MessageState `protogen:"opaque.v1"` + xxx_hidden_State JobState_Enum `protobuf:"varint,1,opt,name=state,proto3,enum=org.apache.beam.model.job_management.v1.JobState_Enum" json:"state,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *CancelJobResponse) Reset() { + *x = CancelJobResponse{} + mi := &file_org_apache_beam_model_job_management_v1_beam_job_api_proto_msgTypes[5] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *CancelJobResponse) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*CancelJobResponse) ProtoMessage() {} + +func (x *CancelJobResponse) ProtoReflect() protoreflect.Message { + mi := &file_org_apache_beam_model_job_management_v1_beam_job_api_proto_msgTypes[5] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +func (x *CancelJobResponse) GetState() JobState_Enum { + if x != nil { + return x.xxx_hidden_State + } + return JobState_UNSPECIFIED +} + +func (x *CancelJobResponse) SetState(v JobState_Enum) { + x.xxx_hidden_State = v +} + +type CancelJobResponse_builder struct { + _ [0]func() // Prevents comparability and use of unkeyed literals for the builder. + + State JobState_Enum +} + +func (b0 CancelJobResponse_builder) Build() *CancelJobResponse { + m0 := &CancelJobResponse{} + b, x := &b0, m0 + _, _ = b, x + x.xxx_hidden_State = b.State + return m0 +} + +// Drain is a request to: +// - stop a pipeline Job from ingesting new data from input sources +// - process remaining data in the pipeline +// - truncate Splittable DoFns +type DrainJobRequest struct { + state protoimpl.MessageState `protogen:"opaque.v1"` + xxx_hidden_JobId string `protobuf:"bytes,1,opt,name=job_id,json=jobId,proto3" json:"job_id,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *DrainJobRequest) Reset() { + *x = DrainJobRequest{} + mi := &file_org_apache_beam_model_job_management_v1_beam_job_api_proto_msgTypes[6] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *DrainJobRequest) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*DrainJobRequest) ProtoMessage() {} + +func (x *DrainJobRequest) ProtoReflect() protoreflect.Message { + mi := &file_org_apache_beam_model_job_management_v1_beam_job_api_proto_msgTypes[6] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +func (x *DrainJobRequest) GetJobId() string { + if x != nil { + return x.xxx_hidden_JobId + } + return "" +} + +func (x *DrainJobRequest) SetJobId(v string) { + x.xxx_hidden_JobId = v +} + +type DrainJobRequest_builder struct { + _ [0]func() // Prevents comparability and use of unkeyed literals for the builder. + + JobId string +} + +func (b0 DrainJobRequest_builder) Build() *DrainJobRequest { + m0 := &DrainJobRequest{} + b, x := &b0, m0 + _, _ = b, x + x.xxx_hidden_JobId = b.JobId + return m0 +} + +// Valid responses include any terminal state or DRAINING +type DrainJobResponse struct { + state protoimpl.MessageState `protogen:"opaque.v1"` + xxx_hidden_State JobState_Enum `protobuf:"varint,1,opt,name=state,proto3,enum=org.apache.beam.model.job_management.v1.JobState_Enum" json:"state,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *DrainJobResponse) Reset() { + *x = DrainJobResponse{} + mi := &file_org_apache_beam_model_job_management_v1_beam_job_api_proto_msgTypes[7] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *DrainJobResponse) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*DrainJobResponse) ProtoMessage() {} + +func (x *DrainJobResponse) ProtoReflect() protoreflect.Message { + mi := &file_org_apache_beam_model_job_management_v1_beam_job_api_proto_msgTypes[7] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +func (x *DrainJobResponse) GetState() JobState_Enum { + if x != nil { + return x.xxx_hidden_State + } + return JobState_UNSPECIFIED +} + +func (x *DrainJobResponse) SetState(v JobState_Enum) { + x.xxx_hidden_State = v +} + +type DrainJobResponse_builder struct { + _ [0]func() // Prevents comparability and use of unkeyed literals for the builder. + + State JobState_Enum +} + +func (b0 DrainJobResponse_builder) Build() *DrainJobResponse { + m0 := &DrainJobResponse{} + b, x := &b0, m0 + _, _ = b, x + x.xxx_hidden_State = b.State + return m0 +} + +// A subset of info provided by ProvisionApi.ProvisionInfo +type JobInfo struct { + state protoimpl.MessageState `protogen:"opaque.v1"` + xxx_hidden_JobId string `protobuf:"bytes,1,opt,name=job_id,json=jobId,proto3" json:"job_id,omitempty"` + xxx_hidden_JobName string `protobuf:"bytes,2,opt,name=job_name,json=jobName,proto3" json:"job_name,omitempty"` + xxx_hidden_PipelineOptions *structpb.Struct `protobuf:"bytes,3,opt,name=pipeline_options,json=pipelineOptions,proto3" json:"pipeline_options,omitempty"` + xxx_hidden_State JobState_Enum `protobuf:"varint,4,opt,name=state,proto3,enum=org.apache.beam.model.job_management.v1.JobState_Enum" json:"state,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *JobInfo) Reset() { + *x = JobInfo{} + mi := &file_org_apache_beam_model_job_management_v1_beam_job_api_proto_msgTypes[8] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *JobInfo) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*JobInfo) ProtoMessage() {} + +func (x *JobInfo) ProtoReflect() protoreflect.Message { + mi := &file_org_apache_beam_model_job_management_v1_beam_job_api_proto_msgTypes[8] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +func (x *JobInfo) GetJobId() string { + if x != nil { + return x.xxx_hidden_JobId + } + return "" +} + +func (x *JobInfo) GetJobName() string { + if x != nil { + return x.xxx_hidden_JobName + } + return "" +} + +func (x *JobInfo) GetPipelineOptions() *structpb.Struct { + if x != nil { + return x.xxx_hidden_PipelineOptions + } + return nil +} + +func (x *JobInfo) GetState() JobState_Enum { + if x != nil { + return x.xxx_hidden_State + } + return JobState_UNSPECIFIED +} + +func (x *JobInfo) SetJobId(v string) { + x.xxx_hidden_JobId = v +} + +func (x *JobInfo) SetJobName(v string) { + x.xxx_hidden_JobName = v +} + +func (x *JobInfo) SetPipelineOptions(v *structpb.Struct) { + x.xxx_hidden_PipelineOptions = v +} + +func (x *JobInfo) SetState(v JobState_Enum) { + x.xxx_hidden_State = v +} + +func (x *JobInfo) HasPipelineOptions() bool { + if x == nil { + return false + } + return x.xxx_hidden_PipelineOptions != nil +} + +func (x *JobInfo) ClearPipelineOptions() { + x.xxx_hidden_PipelineOptions = nil +} + +type JobInfo_builder struct { + _ [0]func() // Prevents comparability and use of unkeyed literals for the builder. + + JobId string + JobName string + PipelineOptions *structpb.Struct + State JobState_Enum +} + +func (b0 JobInfo_builder) Build() *JobInfo { + m0 := &JobInfo{} + b, x := &b0, m0 + _, _ = b, x + x.xxx_hidden_JobId = b.JobId + x.xxx_hidden_JobName = b.JobName + x.xxx_hidden_PipelineOptions = b.PipelineOptions + x.xxx_hidden_State = b.State + return m0 +} + +// GetJobs is a synchronus request that returns a list of invoked jobs back +// Throws error GRPC_STATUS_UNAVAILABLE if server is down +type GetJobsRequest struct { + state protoimpl.MessageState `protogen:"opaque.v1"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *GetJobsRequest) Reset() { + *x = GetJobsRequest{} + mi := &file_org_apache_beam_model_job_management_v1_beam_job_api_proto_msgTypes[9] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *GetJobsRequest) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*GetJobsRequest) ProtoMessage() {} + +func (x *GetJobsRequest) ProtoReflect() protoreflect.Message { + mi := &file_org_apache_beam_model_job_management_v1_beam_job_api_proto_msgTypes[9] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +type GetJobsRequest_builder struct { + _ [0]func() // Prevents comparability and use of unkeyed literals for the builder. + +} + +func (b0 GetJobsRequest_builder) Build() *GetJobsRequest { + m0 := &GetJobsRequest{} + b, x := &b0, m0 + _, _ = b, x + return m0 +} + +type GetJobsResponse struct { + state protoimpl.MessageState `protogen:"opaque.v1"` + xxx_hidden_JobInfo *[]*JobInfo `protobuf:"bytes,1,rep,name=job_info,json=jobInfo,proto3" json:"job_info,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *GetJobsResponse) Reset() { + *x = GetJobsResponse{} + mi := &file_org_apache_beam_model_job_management_v1_beam_job_api_proto_msgTypes[10] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *GetJobsResponse) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*GetJobsResponse) ProtoMessage() {} + +func (x *GetJobsResponse) ProtoReflect() protoreflect.Message { + mi := &file_org_apache_beam_model_job_management_v1_beam_job_api_proto_msgTypes[10] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +func (x *GetJobsResponse) GetJobInfo() []*JobInfo { + if x != nil { + if x.xxx_hidden_JobInfo != nil { + return *x.xxx_hidden_JobInfo + } + } + return nil +} + +func (x *GetJobsResponse) SetJobInfo(v []*JobInfo) { + x.xxx_hidden_JobInfo = &v +} + +type GetJobsResponse_builder struct { + _ [0]func() // Prevents comparability and use of unkeyed literals for the builder. + + JobInfo []*JobInfo +} + +func (b0 GetJobsResponse_builder) Build() *GetJobsResponse { + m0 := &GetJobsResponse{} + b, x := &b0, m0 + _, _ = b, x + x.xxx_hidden_JobInfo = &b.JobInfo + return m0 +} + +// GetState is a synchronus request that returns a job state back +// Throws error GRPC_STATUS_UNAVAILABLE if server is down +// Throws error NOT_FOUND if the jobId is not found +type GetJobStateRequest struct { + state protoimpl.MessageState `protogen:"opaque.v1"` + xxx_hidden_JobId string `protobuf:"bytes,1,opt,name=job_id,json=jobId,proto3" json:"job_id,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *GetJobStateRequest) Reset() { + *x = GetJobStateRequest{} + mi := &file_org_apache_beam_model_job_management_v1_beam_job_api_proto_msgTypes[11] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *GetJobStateRequest) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*GetJobStateRequest) ProtoMessage() {} + +func (x *GetJobStateRequest) ProtoReflect() protoreflect.Message { + mi := &file_org_apache_beam_model_job_management_v1_beam_job_api_proto_msgTypes[11] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +func (x *GetJobStateRequest) GetJobId() string { + if x != nil { + return x.xxx_hidden_JobId + } + return "" +} + +func (x *GetJobStateRequest) SetJobId(v string) { + x.xxx_hidden_JobId = v +} + +type GetJobStateRequest_builder struct { + _ [0]func() // Prevents comparability and use of unkeyed literals for the builder. + + JobId string +} + +func (b0 GetJobStateRequest_builder) Build() *GetJobStateRequest { + m0 := &GetJobStateRequest{} + b, x := &b0, m0 + _, _ = b, x + x.xxx_hidden_JobId = b.JobId + return m0 +} + +type JobStateEvent struct { + state protoimpl.MessageState `protogen:"opaque.v1"` + xxx_hidden_State JobState_Enum `protobuf:"varint,1,opt,name=state,proto3,enum=org.apache.beam.model.job_management.v1.JobState_Enum" json:"state,omitempty"` + xxx_hidden_Timestamp *timestamppb.Timestamp `protobuf:"bytes,2,opt,name=timestamp,proto3" json:"timestamp,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *JobStateEvent) Reset() { + *x = JobStateEvent{} + mi := &file_org_apache_beam_model_job_management_v1_beam_job_api_proto_msgTypes[12] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *JobStateEvent) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*JobStateEvent) ProtoMessage() {} + +func (x *JobStateEvent) ProtoReflect() protoreflect.Message { + mi := &file_org_apache_beam_model_job_management_v1_beam_job_api_proto_msgTypes[12] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +func (x *JobStateEvent) GetState() JobState_Enum { + if x != nil { + return x.xxx_hidden_State + } + return JobState_UNSPECIFIED +} + +func (x *JobStateEvent) GetTimestamp() *timestamppb.Timestamp { + if x != nil { + return x.xxx_hidden_Timestamp + } + return nil +} + +func (x *JobStateEvent) SetState(v JobState_Enum) { + x.xxx_hidden_State = v +} + +func (x *JobStateEvent) SetTimestamp(v *timestamppb.Timestamp) { + x.xxx_hidden_Timestamp = v +} + +func (x *JobStateEvent) HasTimestamp() bool { + if x == nil { + return false + } + return x.xxx_hidden_Timestamp != nil +} + +func (x *JobStateEvent) ClearTimestamp() { + x.xxx_hidden_Timestamp = nil +} + +type JobStateEvent_builder struct { + _ [0]func() // Prevents comparability and use of unkeyed literals for the builder. + + State JobState_Enum + Timestamp *timestamppb.Timestamp +} + +func (b0 JobStateEvent_builder) Build() *JobStateEvent { + m0 := &JobStateEvent{} + b, x := &b0, m0 + _, _ = b, x + x.xxx_hidden_State = b.State + x.xxx_hidden_Timestamp = b.Timestamp + return m0 +} + +// GetPipeline is a synchronus request that returns a pipeline back +// Throws error GRPC_STATUS_UNAVAILABLE if server is down +// Throws error NOT_FOUND if the jobId is not found +type GetJobPipelineRequest struct { + state protoimpl.MessageState `protogen:"opaque.v1"` + xxx_hidden_JobId string `protobuf:"bytes,1,opt,name=job_id,json=jobId,proto3" json:"job_id,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *GetJobPipelineRequest) Reset() { + *x = GetJobPipelineRequest{} + mi := &file_org_apache_beam_model_job_management_v1_beam_job_api_proto_msgTypes[13] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *GetJobPipelineRequest) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*GetJobPipelineRequest) ProtoMessage() {} + +func (x *GetJobPipelineRequest) ProtoReflect() protoreflect.Message { + mi := &file_org_apache_beam_model_job_management_v1_beam_job_api_proto_msgTypes[13] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +func (x *GetJobPipelineRequest) GetJobId() string { + if x != nil { + return x.xxx_hidden_JobId + } + return "" +} + +func (x *GetJobPipelineRequest) SetJobId(v string) { + x.xxx_hidden_JobId = v +} + +type GetJobPipelineRequest_builder struct { + _ [0]func() // Prevents comparability and use of unkeyed literals for the builder. + + JobId string +} + +func (b0 GetJobPipelineRequest_builder) Build() *GetJobPipelineRequest { + m0 := &GetJobPipelineRequest{} + b, x := &b0, m0 + _, _ = b, x + x.xxx_hidden_JobId = b.JobId + return m0 +} + +type GetJobPipelineResponse struct { + state protoimpl.MessageState `protogen:"opaque.v1"` + xxx_hidden_Pipeline *pipeline_v1.Pipeline `protobuf:"bytes,1,opt,name=pipeline,proto3" json:"pipeline,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *GetJobPipelineResponse) Reset() { + *x = GetJobPipelineResponse{} + mi := &file_org_apache_beam_model_job_management_v1_beam_job_api_proto_msgTypes[14] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *GetJobPipelineResponse) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*GetJobPipelineResponse) ProtoMessage() {} + +func (x *GetJobPipelineResponse) ProtoReflect() protoreflect.Message { + mi := &file_org_apache_beam_model_job_management_v1_beam_job_api_proto_msgTypes[14] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +func (x *GetJobPipelineResponse) GetPipeline() *pipeline_v1.Pipeline { + if x != nil { + return x.xxx_hidden_Pipeline + } + return nil +} + +func (x *GetJobPipelineResponse) SetPipeline(v *pipeline_v1.Pipeline) { + x.xxx_hidden_Pipeline = v +} + +func (x *GetJobPipelineResponse) HasPipeline() bool { + if x == nil { + return false + } + return x.xxx_hidden_Pipeline != nil +} + +func (x *GetJobPipelineResponse) ClearPipeline() { + x.xxx_hidden_Pipeline = nil +} + +type GetJobPipelineResponse_builder struct { + _ [0]func() // Prevents comparability and use of unkeyed literals for the builder. + + Pipeline *pipeline_v1.Pipeline +} + +func (b0 GetJobPipelineResponse_builder) Build() *GetJobPipelineResponse { + m0 := &GetJobPipelineResponse{} + b, x := &b0, m0 + _, _ = b, x + x.xxx_hidden_Pipeline = b.Pipeline + return m0 +} + +// GetJobMessages is a streaming api for streaming job messages from the service +// One request will connect you to the job and you'll get a stream of job state +// and job messages back; one is used for logging and the other for detecting +// the job ended. +type JobMessagesRequest struct { + state protoimpl.MessageState `protogen:"opaque.v1"` + xxx_hidden_JobId string `protobuf:"bytes,1,opt,name=job_id,json=jobId,proto3" json:"job_id,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *JobMessagesRequest) Reset() { + *x = JobMessagesRequest{} + mi := &file_org_apache_beam_model_job_management_v1_beam_job_api_proto_msgTypes[15] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *JobMessagesRequest) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*JobMessagesRequest) ProtoMessage() {} + +func (x *JobMessagesRequest) ProtoReflect() protoreflect.Message { + mi := &file_org_apache_beam_model_job_management_v1_beam_job_api_proto_msgTypes[15] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +func (x *JobMessagesRequest) GetJobId() string { + if x != nil { + return x.xxx_hidden_JobId + } + return "" +} + +func (x *JobMessagesRequest) SetJobId(v string) { + x.xxx_hidden_JobId = v +} + +type JobMessagesRequest_builder struct { + _ [0]func() // Prevents comparability and use of unkeyed literals for the builder. + + JobId string +} + +func (b0 JobMessagesRequest_builder) Build() *JobMessagesRequest { + m0 := &JobMessagesRequest{} + b, x := &b0, m0 + _, _ = b, x + x.xxx_hidden_JobId = b.JobId + return m0 +} + +type JobMessage struct { + state protoimpl.MessageState `protogen:"opaque.v1"` + xxx_hidden_MessageId string `protobuf:"bytes,1,opt,name=message_id,json=messageId,proto3" json:"message_id,omitempty"` + xxx_hidden_Time string `protobuf:"bytes,2,opt,name=time,proto3" json:"time,omitempty"` + xxx_hidden_Importance JobMessage_MessageImportance `protobuf:"varint,3,opt,name=importance,proto3,enum=org.apache.beam.model.job_management.v1.JobMessage_MessageImportance" json:"importance,omitempty"` + xxx_hidden_MessageText string `protobuf:"bytes,4,opt,name=message_text,json=messageText,proto3" json:"message_text,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *JobMessage) Reset() { + *x = JobMessage{} + mi := &file_org_apache_beam_model_job_management_v1_beam_job_api_proto_msgTypes[16] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *JobMessage) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*JobMessage) ProtoMessage() {} + +func (x *JobMessage) ProtoReflect() protoreflect.Message { + mi := &file_org_apache_beam_model_job_management_v1_beam_job_api_proto_msgTypes[16] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +func (x *JobMessage) GetMessageId() string { + if x != nil { + return x.xxx_hidden_MessageId + } + return "" +} + +func (x *JobMessage) GetTime() string { + if x != nil { + return x.xxx_hidden_Time + } + return "" +} + +func (x *JobMessage) GetImportance() JobMessage_MessageImportance { + if x != nil { + return x.xxx_hidden_Importance + } + return JobMessage_MESSAGE_IMPORTANCE_UNSPECIFIED +} + +func (x *JobMessage) GetMessageText() string { + if x != nil { + return x.xxx_hidden_MessageText + } + return "" +} + +func (x *JobMessage) SetMessageId(v string) { + x.xxx_hidden_MessageId = v +} + +func (x *JobMessage) SetTime(v string) { + x.xxx_hidden_Time = v +} + +func (x *JobMessage) SetImportance(v JobMessage_MessageImportance) { + x.xxx_hidden_Importance = v +} + +func (x *JobMessage) SetMessageText(v string) { + x.xxx_hidden_MessageText = v +} + +type JobMessage_builder struct { + _ [0]func() // Prevents comparability and use of unkeyed literals for the builder. + + MessageId string + Time string + Importance JobMessage_MessageImportance + MessageText string +} + +func (b0 JobMessage_builder) Build() *JobMessage { + m0 := &JobMessage{} + b, x := &b0, m0 + _, _ = b, x + x.xxx_hidden_MessageId = b.MessageId + x.xxx_hidden_Time = b.Time + x.xxx_hidden_Importance = b.Importance + x.xxx_hidden_MessageText = b.MessageText + return m0 +} + +type JobMessagesResponse struct { + state protoimpl.MessageState `protogen:"opaque.v1"` + xxx_hidden_Response isJobMessagesResponse_Response `protobuf_oneof:"response"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *JobMessagesResponse) Reset() { + *x = JobMessagesResponse{} + mi := &file_org_apache_beam_model_job_management_v1_beam_job_api_proto_msgTypes[17] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *JobMessagesResponse) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*JobMessagesResponse) ProtoMessage() {} + +func (x *JobMessagesResponse) ProtoReflect() protoreflect.Message { + mi := &file_org_apache_beam_model_job_management_v1_beam_job_api_proto_msgTypes[17] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +func (x *JobMessagesResponse) GetMessageResponse() *JobMessage { + if x != nil { + if x, ok := x.xxx_hidden_Response.(*jobMessagesResponse_MessageResponse); ok { + return x.MessageResponse + } + } + return nil +} + +func (x *JobMessagesResponse) GetStateResponse() *JobStateEvent { + if x != nil { + if x, ok := x.xxx_hidden_Response.(*jobMessagesResponse_StateResponse); ok { + return x.StateResponse + } + } + return nil +} + +func (x *JobMessagesResponse) SetMessageResponse(v *JobMessage) { + if v == nil { + x.xxx_hidden_Response = nil + return + } + x.xxx_hidden_Response = &jobMessagesResponse_MessageResponse{v} +} + +func (x *JobMessagesResponse) SetStateResponse(v *JobStateEvent) { + if v == nil { + x.xxx_hidden_Response = nil + return + } + x.xxx_hidden_Response = &jobMessagesResponse_StateResponse{v} +} + +func (x *JobMessagesResponse) HasResponse() bool { + if x == nil { + return false + } + return x.xxx_hidden_Response != nil +} + +func (x *JobMessagesResponse) HasMessageResponse() bool { + if x == nil { + return false + } + _, ok := x.xxx_hidden_Response.(*jobMessagesResponse_MessageResponse) + return ok +} + +func (x *JobMessagesResponse) HasStateResponse() bool { + if x == nil { + return false + } + _, ok := x.xxx_hidden_Response.(*jobMessagesResponse_StateResponse) + return ok +} + +func (x *JobMessagesResponse) ClearResponse() { + x.xxx_hidden_Response = nil +} + +func (x *JobMessagesResponse) ClearMessageResponse() { + if _, ok := x.xxx_hidden_Response.(*jobMessagesResponse_MessageResponse); ok { + x.xxx_hidden_Response = nil + } +} + +func (x *JobMessagesResponse) ClearStateResponse() { + if _, ok := x.xxx_hidden_Response.(*jobMessagesResponse_StateResponse); ok { + x.xxx_hidden_Response = nil + } +} + +const JobMessagesResponse_Response_not_set_case case_JobMessagesResponse_Response = 0 +const JobMessagesResponse_MessageResponse_case case_JobMessagesResponse_Response = 1 +const JobMessagesResponse_StateResponse_case case_JobMessagesResponse_Response = 2 + +func (x *JobMessagesResponse) WhichResponse() case_JobMessagesResponse_Response { + if x == nil { + return JobMessagesResponse_Response_not_set_case + } + switch x.xxx_hidden_Response.(type) { + case *jobMessagesResponse_MessageResponse: + return JobMessagesResponse_MessageResponse_case + case *jobMessagesResponse_StateResponse: + return JobMessagesResponse_StateResponse_case + default: + return JobMessagesResponse_Response_not_set_case + } +} + +type JobMessagesResponse_builder struct { + _ [0]func() // Prevents comparability and use of unkeyed literals for the builder. + + // Fields of oneof xxx_hidden_Response: + MessageResponse *JobMessage + StateResponse *JobStateEvent + // -- end of xxx_hidden_Response +} + +func (b0 JobMessagesResponse_builder) Build() *JobMessagesResponse { + m0 := &JobMessagesResponse{} + b, x := &b0, m0 + _, _ = b, x + if b.MessageResponse != nil { + x.xxx_hidden_Response = &jobMessagesResponse_MessageResponse{b.MessageResponse} + } + if b.StateResponse != nil { + x.xxx_hidden_Response = &jobMessagesResponse_StateResponse{b.StateResponse} + } + return m0 +} + +type case_JobMessagesResponse_Response protoreflect.FieldNumber + +func (x case_JobMessagesResponse_Response) String() string { + md := file_org_apache_beam_model_job_management_v1_beam_job_api_proto_msgTypes[17].Descriptor() + if x == 0 { + return "not set" + } + return protoimpl.X.MessageFieldStringOf(md, protoreflect.FieldNumber(x)) +} + +type isJobMessagesResponse_Response interface { + isJobMessagesResponse_Response() +} + +type jobMessagesResponse_MessageResponse struct { + MessageResponse *JobMessage `protobuf:"bytes,1,opt,name=message_response,json=messageResponse,proto3,oneof"` +} + +type jobMessagesResponse_StateResponse struct { + StateResponse *JobStateEvent `protobuf:"bytes,2,opt,name=state_response,json=stateResponse,proto3,oneof"` +} + +func (*jobMessagesResponse_MessageResponse) isJobMessagesResponse_Response() {} + +func (*jobMessagesResponse_StateResponse) isJobMessagesResponse_Response() {} + +// Enumeration of all JobStates +// +// The state transition diagram is: +// +// STOPPED -> STARTING -> RUNNING -> DONE +// \> FAILED +// \> CANCELLING -> CANCELLED +// \> UPDATING -> UPDATED +// \> DRAINING -> DRAINED +// +// Transitions are optional such that a job may go from STOPPED to RUNNING +// without needing to pass through STARTING. +type JobState struct { + state protoimpl.MessageState `protogen:"opaque.v1"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *JobState) Reset() { + *x = JobState{} + mi := &file_org_apache_beam_model_job_management_v1_beam_job_api_proto_msgTypes[18] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *JobState) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*JobState) ProtoMessage() {} + +func (x *JobState) ProtoReflect() protoreflect.Message { + mi := &file_org_apache_beam_model_job_management_v1_beam_job_api_proto_msgTypes[18] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +type JobState_builder struct { + _ [0]func() // Prevents comparability and use of unkeyed literals for the builder. + +} + +func (b0 JobState_builder) Build() *JobState { + m0 := &JobState{} + b, x := &b0, m0 + _, _ = b, x + return m0 +} + +type GetJobMetricsRequest struct { + state protoimpl.MessageState `protogen:"opaque.v1"` + xxx_hidden_JobId string `protobuf:"bytes,1,opt,name=job_id,json=jobId,proto3" json:"job_id,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *GetJobMetricsRequest) Reset() { + *x = GetJobMetricsRequest{} + mi := &file_org_apache_beam_model_job_management_v1_beam_job_api_proto_msgTypes[19] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *GetJobMetricsRequest) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*GetJobMetricsRequest) ProtoMessage() {} + +func (x *GetJobMetricsRequest) ProtoReflect() protoreflect.Message { + mi := &file_org_apache_beam_model_job_management_v1_beam_job_api_proto_msgTypes[19] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +func (x *GetJobMetricsRequest) GetJobId() string { + if x != nil { + return x.xxx_hidden_JobId + } + return "" +} + +func (x *GetJobMetricsRequest) SetJobId(v string) { + x.xxx_hidden_JobId = v +} + +type GetJobMetricsRequest_builder struct { + _ [0]func() // Prevents comparability and use of unkeyed literals for the builder. + + JobId string +} + +func (b0 GetJobMetricsRequest_builder) Build() *GetJobMetricsRequest { + m0 := &GetJobMetricsRequest{} + b, x := &b0, m0 + _, _ = b, x + x.xxx_hidden_JobId = b.JobId + return m0 +} + +type GetJobMetricsResponse struct { + state protoimpl.MessageState `protogen:"opaque.v1"` + xxx_hidden_Metrics *MetricResults `protobuf:"bytes,1,opt,name=metrics,proto3" json:"metrics,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *GetJobMetricsResponse) Reset() { + *x = GetJobMetricsResponse{} + mi := &file_org_apache_beam_model_job_management_v1_beam_job_api_proto_msgTypes[20] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *GetJobMetricsResponse) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*GetJobMetricsResponse) ProtoMessage() {} + +func (x *GetJobMetricsResponse) ProtoReflect() protoreflect.Message { + mi := &file_org_apache_beam_model_job_management_v1_beam_job_api_proto_msgTypes[20] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +func (x *GetJobMetricsResponse) GetMetrics() *MetricResults { + if x != nil { + return x.xxx_hidden_Metrics + } + return nil +} + +func (x *GetJobMetricsResponse) SetMetrics(v *MetricResults) { + x.xxx_hidden_Metrics = v +} + +func (x *GetJobMetricsResponse) HasMetrics() bool { + if x == nil { + return false + } + return x.xxx_hidden_Metrics != nil +} + +func (x *GetJobMetricsResponse) ClearMetrics() { + x.xxx_hidden_Metrics = nil +} + +type GetJobMetricsResponse_builder struct { + _ [0]func() // Prevents comparability and use of unkeyed literals for the builder. + + Metrics *MetricResults +} + +func (b0 GetJobMetricsResponse_builder) Build() *GetJobMetricsResponse { + m0 := &GetJobMetricsResponse{} + b, x := &b0, m0 + _, _ = b, x + x.xxx_hidden_Metrics = b.Metrics + return m0 +} + +// All metrics for a given job. Runners may support one or the other or both. +type MetricResults struct { + state protoimpl.MessageState `protogen:"opaque.v1"` + xxx_hidden_Attempted *[]*pipeline_v1.MonitoringInfo `protobuf:"bytes,1,rep,name=attempted,proto3" json:"attempted,omitempty"` + xxx_hidden_Committed *[]*pipeline_v1.MonitoringInfo `protobuf:"bytes,2,rep,name=committed,proto3" json:"committed,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *MetricResults) Reset() { + *x = MetricResults{} + mi := &file_org_apache_beam_model_job_management_v1_beam_job_api_proto_msgTypes[21] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *MetricResults) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*MetricResults) ProtoMessage() {} + +func (x *MetricResults) ProtoReflect() protoreflect.Message { + mi := &file_org_apache_beam_model_job_management_v1_beam_job_api_proto_msgTypes[21] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +func (x *MetricResults) GetAttempted() []*pipeline_v1.MonitoringInfo { + if x != nil { + if x.xxx_hidden_Attempted != nil { + return *x.xxx_hidden_Attempted + } + } + return nil +} + +func (x *MetricResults) GetCommitted() []*pipeline_v1.MonitoringInfo { + if x != nil { + if x.xxx_hidden_Committed != nil { + return *x.xxx_hidden_Committed + } + } + return nil +} + +func (x *MetricResults) SetAttempted(v []*pipeline_v1.MonitoringInfo) { + x.xxx_hidden_Attempted = &v +} + +func (x *MetricResults) SetCommitted(v []*pipeline_v1.MonitoringInfo) { + x.xxx_hidden_Committed = &v +} + +type MetricResults_builder struct { + _ [0]func() // Prevents comparability and use of unkeyed literals for the builder. + + Attempted []*pipeline_v1.MonitoringInfo + Committed []*pipeline_v1.MonitoringInfo +} + +func (b0 MetricResults_builder) Build() *MetricResults { + m0 := &MetricResults{} + b, x := &b0, m0 + _, _ = b, x + x.xxx_hidden_Attempted = &b.Attempted + x.xxx_hidden_Committed = &b.Committed + return m0 +} + +// DescribePipelineOptions provides metadata about the options supported by a runner. +// It will be used by the SDK client to validate the options specified by or +// list available options to the user. +// Throws error GRPC_STATUS_UNAVAILABLE if server is down +type DescribePipelineOptionsRequest struct { + state protoimpl.MessageState `protogen:"opaque.v1"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *DescribePipelineOptionsRequest) Reset() { + *x = DescribePipelineOptionsRequest{} + mi := &file_org_apache_beam_model_job_management_v1_beam_job_api_proto_msgTypes[22] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *DescribePipelineOptionsRequest) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*DescribePipelineOptionsRequest) ProtoMessage() {} + +func (x *DescribePipelineOptionsRequest) ProtoReflect() protoreflect.Message { + mi := &file_org_apache_beam_model_job_management_v1_beam_job_api_proto_msgTypes[22] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +type DescribePipelineOptionsRequest_builder struct { + _ [0]func() // Prevents comparability and use of unkeyed literals for the builder. + +} + +func (b0 DescribePipelineOptionsRequest_builder) Build() *DescribePipelineOptionsRequest { + m0 := &DescribePipelineOptionsRequest{} + b, x := &b0, m0 + _, _ = b, x + return m0 +} + +// Type for pipeline options. +// Types mirror those of JSON, since that's how pipeline options are serialized. +type PipelineOptionType struct { + state protoimpl.MessageState `protogen:"opaque.v1"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *PipelineOptionType) Reset() { + *x = PipelineOptionType{} + mi := &file_org_apache_beam_model_job_management_v1_beam_job_api_proto_msgTypes[23] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *PipelineOptionType) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*PipelineOptionType) ProtoMessage() {} + +func (x *PipelineOptionType) ProtoReflect() protoreflect.Message { + mi := &file_org_apache_beam_model_job_management_v1_beam_job_api_proto_msgTypes[23] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +type PipelineOptionType_builder struct { + _ [0]func() // Prevents comparability and use of unkeyed literals for the builder. + +} + +func (b0 PipelineOptionType_builder) Build() *PipelineOptionType { + m0 := &PipelineOptionType{} + b, x := &b0, m0 + _, _ = b, x + return m0 +} + +// Metadata for a pipeline option. +type PipelineOptionDescriptor struct { + state protoimpl.MessageState `protogen:"opaque.v1"` + xxx_hidden_Name string `protobuf:"bytes,1,opt,name=name,proto3" json:"name,omitempty"` + xxx_hidden_Type PipelineOptionType_Enum `protobuf:"varint,2,opt,name=type,proto3,enum=org.apache.beam.model.job_management.v1.PipelineOptionType_Enum" json:"type,omitempty"` + xxx_hidden_Description string `protobuf:"bytes,3,opt,name=description,proto3" json:"description,omitempty"` + xxx_hidden_DefaultValue string `protobuf:"bytes,4,opt,name=default_value,json=defaultValue,proto3" json:"default_value,omitempty"` + xxx_hidden_Group string `protobuf:"bytes,5,opt,name=group,proto3" json:"group,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *PipelineOptionDescriptor) Reset() { + *x = PipelineOptionDescriptor{} + mi := &file_org_apache_beam_model_job_management_v1_beam_job_api_proto_msgTypes[24] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *PipelineOptionDescriptor) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*PipelineOptionDescriptor) ProtoMessage() {} + +func (x *PipelineOptionDescriptor) ProtoReflect() protoreflect.Message { + mi := &file_org_apache_beam_model_job_management_v1_beam_job_api_proto_msgTypes[24] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +func (x *PipelineOptionDescriptor) GetName() string { + if x != nil { + return x.xxx_hidden_Name + } + return "" +} + +func (x *PipelineOptionDescriptor) GetType() PipelineOptionType_Enum { + if x != nil { + return x.xxx_hidden_Type + } + return PipelineOptionType_STRING +} + +func (x *PipelineOptionDescriptor) GetDescription() string { + if x != nil { + return x.xxx_hidden_Description + } + return "" +} + +func (x *PipelineOptionDescriptor) GetDefaultValue() string { + if x != nil { + return x.xxx_hidden_DefaultValue + } + return "" +} + +func (x *PipelineOptionDescriptor) GetGroup() string { + if x != nil { + return x.xxx_hidden_Group + } + return "" +} + +func (x *PipelineOptionDescriptor) SetName(v string) { + x.xxx_hidden_Name = v +} + +func (x *PipelineOptionDescriptor) SetType(v PipelineOptionType_Enum) { + x.xxx_hidden_Type = v +} + +func (x *PipelineOptionDescriptor) SetDescription(v string) { + x.xxx_hidden_Description = v +} + +func (x *PipelineOptionDescriptor) SetDefaultValue(v string) { + x.xxx_hidden_DefaultValue = v +} + +func (x *PipelineOptionDescriptor) SetGroup(v string) { + x.xxx_hidden_Group = v +} + +type PipelineOptionDescriptor_builder struct { + _ [0]func() // Prevents comparability and use of unkeyed literals for the builder. + + // (Required) The option name. + Name string + // (Required) Type of option. + Type PipelineOptionType_Enum + // (Optional) Description suitable for display / help text. + Description string + // (Optional) Default value. + DefaultValue string + // (Required) The group this option belongs to. + Group string +} + +func (b0 PipelineOptionDescriptor_builder) Build() *PipelineOptionDescriptor { + m0 := &PipelineOptionDescriptor{} + b, x := &b0, m0 + _, _ = b, x + x.xxx_hidden_Name = b.Name + x.xxx_hidden_Type = b.Type + x.xxx_hidden_Description = b.Description + x.xxx_hidden_DefaultValue = b.DefaultValue + x.xxx_hidden_Group = b.Group + return m0 +} + +type DescribePipelineOptionsResponse struct { + state protoimpl.MessageState `protogen:"opaque.v1"` + xxx_hidden_Options *[]*PipelineOptionDescriptor `protobuf:"bytes,1,rep,name=options,proto3" json:"options,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *DescribePipelineOptionsResponse) Reset() { + *x = DescribePipelineOptionsResponse{} + mi := &file_org_apache_beam_model_job_management_v1_beam_job_api_proto_msgTypes[25] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *DescribePipelineOptionsResponse) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*DescribePipelineOptionsResponse) ProtoMessage() {} + +func (x *DescribePipelineOptionsResponse) ProtoReflect() protoreflect.Message { + mi := &file_org_apache_beam_model_job_management_v1_beam_job_api_proto_msgTypes[25] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +func (x *DescribePipelineOptionsResponse) GetOptions() []*PipelineOptionDescriptor { + if x != nil { + if x.xxx_hidden_Options != nil { + return *x.xxx_hidden_Options + } + } + return nil +} + +func (x *DescribePipelineOptionsResponse) SetOptions(v []*PipelineOptionDescriptor) { + x.xxx_hidden_Options = &v +} + +type DescribePipelineOptionsResponse_builder struct { + _ [0]func() // Prevents comparability and use of unkeyed literals for the builder. + + // List of pipeline option descriptors. + Options []*PipelineOptionDescriptor +} + +func (b0 DescribePipelineOptionsResponse_builder) Build() *DescribePipelineOptionsResponse { + m0 := &DescribePipelineOptionsResponse{} + b, x := &b0, m0 + _, _ = b, x + x.xxx_hidden_Options = &b.Options + return m0 +} + +var File_org_apache_beam_model_job_management_v1_beam_job_api_proto protoreflect.FileDescriptor + +var file_org_apache_beam_model_job_management_v1_beam_job_api_proto_rawDesc = []byte{ + 0x0a, 0x3a, 0x6f, 0x72, 0x67, 0x2f, 0x61, 0x70, 0x61, 0x63, 0x68, 0x65, 0x2f, 0x62, 0x65, 0x61, + 0x6d, 0x2f, 0x6d, 0x6f, 0x64, 0x65, 0x6c, 0x2f, 0x6a, 0x6f, 0x62, 0x5f, 0x6d, 0x61, 0x6e, 0x61, + 0x67, 0x65, 0x6d, 0x65, 0x6e, 0x74, 0x2f, 0x76, 0x31, 0x2f, 0x62, 0x65, 0x61, 0x6d, 0x5f, 0x6a, + 0x6f, 0x62, 0x5f, 0x61, 0x70, 0x69, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x12, 0x27, 0x6f, 0x72, + 0x67, 0x2e, 0x61, 0x70, 0x61, 0x63, 0x68, 0x65, 0x2e, 0x62, 0x65, 0x61, 0x6d, 0x2e, 0x6d, 0x6f, + 0x64, 0x65, 0x6c, 0x2e, 0x6a, 0x6f, 0x62, 0x5f, 0x6d, 0x61, 0x6e, 0x61, 0x67, 0x65, 0x6d, 0x65, + 0x6e, 0x74, 0x2e, 0x76, 0x31, 0x1a, 0x37, 0x6f, 0x72, 0x67, 0x2f, 0x61, 0x70, 0x61, 0x63, 0x68, + 0x65, 0x2f, 0x62, 0x65, 0x61, 0x6d, 0x2f, 0x6d, 0x6f, 0x64, 0x65, 0x6c, 0x2f, 0x70, 0x69, 0x70, + 0x65, 0x6c, 0x69, 0x6e, 0x65, 0x2f, 0x76, 0x31, 0x2f, 0x62, 0x65, 0x61, 0x6d, 0x5f, 0x72, 0x75, + 0x6e, 0x6e, 0x65, 0x72, 0x5f, 0x61, 0x70, 0x69, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x1a, 0x31, + 0x6f, 0x72, 0x67, 0x2f, 0x61, 0x70, 0x61, 0x63, 0x68, 0x65, 0x2f, 0x62, 0x65, 0x61, 0x6d, 0x2f, + 0x6d, 0x6f, 0x64, 0x65, 0x6c, 0x2f, 0x70, 0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e, 0x65, 0x2f, 0x76, + 0x31, 0x2f, 0x65, 0x6e, 0x64, 0x70, 0x6f, 0x69, 0x6e, 0x74, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, + 0x6f, 0x1a, 0x2f, 0x6f, 0x72, 0x67, 0x2f, 0x61, 0x70, 0x61, 0x63, 0x68, 0x65, 0x2f, 0x62, 0x65, + 0x61, 0x6d, 0x2f, 0x6d, 0x6f, 0x64, 0x65, 0x6c, 0x2f, 0x70, 0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e, + 0x65, 0x2f, 0x76, 0x31, 0x2f, 0x6d, 0x65, 0x74, 0x72, 0x69, 0x63, 0x73, 0x2e, 0x70, 0x72, 0x6f, + 0x74, 0x6f, 0x1a, 0x1c, 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2f, 0x70, 0x72, 0x6f, 0x74, 0x6f, + 0x62, 0x75, 0x66, 0x2f, 0x73, 0x74, 0x72, 0x75, 0x63, 0x74, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, + 0x1a, 0x1f, 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2f, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75, + 0x66, 0x2f, 0x74, 0x69, 0x6d, 0x65, 0x73, 0x74, 0x61, 0x6d, 0x70, 0x2e, 0x70, 0x72, 0x6f, 0x74, + 0x6f, 0x22, 0xbb, 0x01, 0x0a, 0x11, 0x50, 0x72, 0x65, 0x70, 0x61, 0x72, 0x65, 0x4a, 0x6f, 0x62, + 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x47, 0x0a, 0x08, 0x70, 0x69, 0x70, 0x65, 0x6c, + 0x69, 0x6e, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x2b, 0x2e, 0x6f, 0x72, 0x67, 0x2e, + 0x61, 0x70, 0x61, 0x63, 0x68, 0x65, 0x2e, 0x62, 0x65, 0x61, 0x6d, 0x2e, 0x6d, 0x6f, 0x64, 0x65, + 0x6c, 0x2e, 0x70, 0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x50, 0x69, + 0x70, 0x65, 0x6c, 0x69, 0x6e, 0x65, 0x52, 0x08, 0x70, 0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e, 0x65, + 0x12, 0x42, 0x0a, 0x10, 0x70, 0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e, 0x65, 0x5f, 0x6f, 0x70, 0x74, + 0x69, 0x6f, 0x6e, 0x73, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x17, 0x2e, 0x67, 0x6f, 0x6f, + 0x67, 0x6c, 0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66, 0x2e, 0x53, 0x74, 0x72, + 0x75, 0x63, 0x74, 0x52, 0x0f, 0x70, 0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e, 0x65, 0x4f, 0x70, 0x74, + 0x69, 0x6f, 0x6e, 0x73, 0x12, 0x19, 0x0a, 0x08, 0x6a, 0x6f, 0x62, 0x5f, 0x6e, 0x61, 0x6d, 0x65, + 0x18, 0x03, 0x20, 0x01, 0x28, 0x09, 0x52, 0x07, 0x6a, 0x6f, 0x62, 0x4e, 0x61, 0x6d, 0x65, 0x22, + 0xe4, 0x01, 0x0a, 0x12, 0x50, 0x72, 0x65, 0x70, 0x61, 0x72, 0x65, 0x4a, 0x6f, 0x62, 0x52, 0x65, + 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x25, 0x0a, 0x0e, 0x70, 0x72, 0x65, 0x70, 0x61, 0x72, + 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x69, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0d, + 0x70, 0x72, 0x65, 0x70, 0x61, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x64, 0x12, 0x73, 0x0a, + 0x19, 0x61, 0x72, 0x74, 0x69, 0x66, 0x61, 0x63, 0x74, 0x5f, 0x73, 0x74, 0x61, 0x67, 0x69, 0x6e, + 0x67, 0x5f, 0x65, 0x6e, 0x64, 0x70, 0x6f, 0x69, 0x6e, 0x74, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, + 0x32, 0x37, 0x2e, 0x6f, 0x72, 0x67, 0x2e, 0x61, 0x70, 0x61, 0x63, 0x68, 0x65, 0x2e, 0x62, 0x65, + 0x61, 0x6d, 0x2e, 0x6d, 0x6f, 0x64, 0x65, 0x6c, 0x2e, 0x70, 0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e, + 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x41, 0x70, 0x69, 0x53, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, 0x44, + 0x65, 0x73, 0x63, 0x72, 0x69, 0x70, 0x74, 0x6f, 0x72, 0x52, 0x17, 0x61, 0x72, 0x74, 0x69, 0x66, + 0x61, 0x63, 0x74, 0x53, 0x74, 0x61, 0x67, 0x69, 0x6e, 0x67, 0x45, 0x6e, 0x64, 0x70, 0x6f, 0x69, + 0x6e, 0x74, 0x12, 0x32, 0x0a, 0x15, 0x73, 0x74, 0x61, 0x67, 0x69, 0x6e, 0x67, 0x5f, 0x73, 0x65, + 0x73, 0x73, 0x69, 0x6f, 0x6e, 0x5f, 0x74, 0x6f, 0x6b, 0x65, 0x6e, 0x18, 0x03, 0x20, 0x01, 0x28, + 0x09, 0x52, 0x13, 0x73, 0x74, 0x61, 0x67, 0x69, 0x6e, 0x67, 0x53, 0x65, 0x73, 0x73, 0x69, 0x6f, + 0x6e, 0x54, 0x6f, 0x6b, 0x65, 0x6e, 0x22, 0x5f, 0x0a, 0x0d, 0x52, 0x75, 0x6e, 0x4a, 0x6f, 0x62, + 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x25, 0x0a, 0x0e, 0x70, 0x72, 0x65, 0x70, 0x61, + 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x69, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, + 0x0d, 0x70, 0x72, 0x65, 0x70, 0x61, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x64, 0x12, 0x27, + 0x0a, 0x0f, 0x72, 0x65, 0x74, 0x72, 0x69, 0x65, 0x76, 0x61, 0x6c, 0x5f, 0x74, 0x6f, 0x6b, 0x65, + 0x6e, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0e, 0x72, 0x65, 0x74, 0x72, 0x69, 0x65, 0x76, + 0x61, 0x6c, 0x54, 0x6f, 0x6b, 0x65, 0x6e, 0x22, 0x27, 0x0a, 0x0e, 0x52, 0x75, 0x6e, 0x4a, 0x6f, + 0x62, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x15, 0x0a, 0x06, 0x6a, 0x6f, 0x62, + 0x5f, 0x69, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x05, 0x6a, 0x6f, 0x62, 0x49, 0x64, + 0x22, 0x29, 0x0a, 0x10, 0x43, 0x61, 0x6e, 0x63, 0x65, 0x6c, 0x4a, 0x6f, 0x62, 0x52, 0x65, 0x71, + 0x75, 0x65, 0x73, 0x74, 0x12, 0x15, 0x0a, 0x06, 0x6a, 0x6f, 0x62, 0x5f, 0x69, 0x64, 0x18, 0x01, + 0x20, 0x01, 0x28, 0x09, 0x52, 0x05, 0x6a, 0x6f, 0x62, 0x49, 0x64, 0x22, 0x61, 0x0a, 0x11, 0x43, + 0x61, 0x6e, 0x63, 0x65, 0x6c, 0x4a, 0x6f, 0x62, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, + 0x12, 0x4c, 0x0a, 0x05, 0x73, 0x74, 0x61, 0x74, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0e, 0x32, + 0x36, 0x2e, 0x6f, 0x72, 0x67, 0x2e, 0x61, 0x70, 0x61, 0x63, 0x68, 0x65, 0x2e, 0x62, 0x65, 0x61, + 0x6d, 0x2e, 0x6d, 0x6f, 0x64, 0x65, 0x6c, 0x2e, 0x6a, 0x6f, 0x62, 0x5f, 0x6d, 0x61, 0x6e, 0x61, + 0x67, 0x65, 0x6d, 0x65, 0x6e, 0x74, 0x2e, 0x76, 0x31, 0x2e, 0x4a, 0x6f, 0x62, 0x53, 0x74, 0x61, + 0x74, 0x65, 0x2e, 0x45, 0x6e, 0x75, 0x6d, 0x52, 0x05, 0x73, 0x74, 0x61, 0x74, 0x65, 0x22, 0x28, + 0x0a, 0x0f, 0x44, 0x72, 0x61, 0x69, 0x6e, 0x4a, 0x6f, 0x62, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, + 0x74, 0x12, 0x15, 0x0a, 0x06, 0x6a, 0x6f, 0x62, 0x5f, 0x69, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, + 0x09, 0x52, 0x05, 0x6a, 0x6f, 0x62, 0x49, 0x64, 0x22, 0x60, 0x0a, 0x10, 0x44, 0x72, 0x61, 0x69, + 0x6e, 0x4a, 0x6f, 0x62, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x4c, 0x0a, 0x05, + 0x73, 0x74, 0x61, 0x74, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0e, 0x32, 0x36, 0x2e, 0x6f, 0x72, + 0x67, 0x2e, 0x61, 0x70, 0x61, 0x63, 0x68, 0x65, 0x2e, 0x62, 0x65, 0x61, 0x6d, 0x2e, 0x6d, 0x6f, + 0x64, 0x65, 0x6c, 0x2e, 0x6a, 0x6f, 0x62, 0x5f, 0x6d, 0x61, 0x6e, 0x61, 0x67, 0x65, 0x6d, 0x65, + 0x6e, 0x74, 0x2e, 0x76, 0x31, 0x2e, 0x4a, 0x6f, 0x62, 0x53, 0x74, 0x61, 0x74, 0x65, 0x2e, 0x45, + 0x6e, 0x75, 0x6d, 0x52, 0x05, 0x73, 0x74, 0x61, 0x74, 0x65, 0x22, 0xcd, 0x01, 0x0a, 0x07, 0x4a, + 0x6f, 0x62, 0x49, 0x6e, 0x66, 0x6f, 0x12, 0x15, 0x0a, 0x06, 0x6a, 0x6f, 0x62, 0x5f, 0x69, 0x64, + 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x05, 0x6a, 0x6f, 0x62, 0x49, 0x64, 0x12, 0x19, 0x0a, + 0x08, 0x6a, 0x6f, 0x62, 0x5f, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, + 0x07, 0x6a, 0x6f, 0x62, 0x4e, 0x61, 0x6d, 0x65, 0x12, 0x42, 0x0a, 0x10, 0x70, 0x69, 0x70, 0x65, + 0x6c, 0x69, 0x6e, 0x65, 0x5f, 0x6f, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x18, 0x03, 0x20, 0x01, + 0x28, 0x0b, 0x32, 0x17, 0x2e, 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74, + 0x6f, 0x62, 0x75, 0x66, 0x2e, 0x53, 0x74, 0x72, 0x75, 0x63, 0x74, 0x52, 0x0f, 0x70, 0x69, 0x70, + 0x65, 0x6c, 0x69, 0x6e, 0x65, 0x4f, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x12, 0x4c, 0x0a, 0x05, + 0x73, 0x74, 0x61, 0x74, 0x65, 0x18, 0x04, 0x20, 0x01, 0x28, 0x0e, 0x32, 0x36, 0x2e, 0x6f, 0x72, + 0x67, 0x2e, 0x61, 0x70, 0x61, 0x63, 0x68, 0x65, 0x2e, 0x62, 0x65, 0x61, 0x6d, 0x2e, 0x6d, 0x6f, + 0x64, 0x65, 0x6c, 0x2e, 0x6a, 0x6f, 0x62, 0x5f, 0x6d, 0x61, 0x6e, 0x61, 0x67, 0x65, 0x6d, 0x65, + 0x6e, 0x74, 0x2e, 0x76, 0x31, 0x2e, 0x4a, 0x6f, 0x62, 0x53, 0x74, 0x61, 0x74, 0x65, 0x2e, 0x45, + 0x6e, 0x75, 0x6d, 0x52, 0x05, 0x73, 0x74, 0x61, 0x74, 0x65, 0x22, 0x10, 0x0a, 0x0e, 0x47, 0x65, + 0x74, 0x4a, 0x6f, 0x62, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x22, 0x5e, 0x0a, 0x0f, + 0x47, 0x65, 0x74, 0x4a, 0x6f, 0x62, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, + 0x4b, 0x0a, 0x08, 0x6a, 0x6f, 0x62, 0x5f, 0x69, 0x6e, 0x66, 0x6f, 0x18, 0x01, 0x20, 0x03, 0x28, + 0x0b, 0x32, 0x30, 0x2e, 0x6f, 0x72, 0x67, 0x2e, 0x61, 0x70, 0x61, 0x63, 0x68, 0x65, 0x2e, 0x62, + 0x65, 0x61, 0x6d, 0x2e, 0x6d, 0x6f, 0x64, 0x65, 0x6c, 0x2e, 0x6a, 0x6f, 0x62, 0x5f, 0x6d, 0x61, + 0x6e, 0x61, 0x67, 0x65, 0x6d, 0x65, 0x6e, 0x74, 0x2e, 0x76, 0x31, 0x2e, 0x4a, 0x6f, 0x62, 0x49, + 0x6e, 0x66, 0x6f, 0x52, 0x07, 0x6a, 0x6f, 0x62, 0x49, 0x6e, 0x66, 0x6f, 0x22, 0x2b, 0x0a, 0x12, + 0x47, 0x65, 0x74, 0x4a, 0x6f, 0x62, 0x53, 0x74, 0x61, 0x74, 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, + 0x73, 0x74, 0x12, 0x15, 0x0a, 0x06, 0x6a, 0x6f, 0x62, 0x5f, 0x69, 0x64, 0x18, 0x01, 0x20, 0x01, + 0x28, 0x09, 0x52, 0x05, 0x6a, 0x6f, 0x62, 0x49, 0x64, 0x22, 0x97, 0x01, 0x0a, 0x0d, 0x4a, 0x6f, + 0x62, 0x53, 0x74, 0x61, 0x74, 0x65, 0x45, 0x76, 0x65, 0x6e, 0x74, 0x12, 0x4c, 0x0a, 0x05, 0x73, + 0x74, 0x61, 0x74, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0e, 0x32, 0x36, 0x2e, 0x6f, 0x72, 0x67, + 0x2e, 0x61, 0x70, 0x61, 0x63, 0x68, 0x65, 0x2e, 0x62, 0x65, 0x61, 0x6d, 0x2e, 0x6d, 0x6f, 0x64, + 0x65, 0x6c, 0x2e, 0x6a, 0x6f, 0x62, 0x5f, 0x6d, 0x61, 0x6e, 0x61, 0x67, 0x65, 0x6d, 0x65, 0x6e, + 0x74, 0x2e, 0x76, 0x31, 0x2e, 0x4a, 0x6f, 0x62, 0x53, 0x74, 0x61, 0x74, 0x65, 0x2e, 0x45, 0x6e, + 0x75, 0x6d, 0x52, 0x05, 0x73, 0x74, 0x61, 0x74, 0x65, 0x12, 0x38, 0x0a, 0x09, 0x74, 0x69, 0x6d, + 0x65, 0x73, 0x74, 0x61, 0x6d, 0x70, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1a, 0x2e, 0x67, + 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66, 0x2e, 0x54, + 0x69, 0x6d, 0x65, 0x73, 0x74, 0x61, 0x6d, 0x70, 0x52, 0x09, 0x74, 0x69, 0x6d, 0x65, 0x73, 0x74, + 0x61, 0x6d, 0x70, 0x22, 0x2e, 0x0a, 0x15, 0x47, 0x65, 0x74, 0x4a, 0x6f, 0x62, 0x50, 0x69, 0x70, + 0x65, 0x6c, 0x69, 0x6e, 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x15, 0x0a, 0x06, + 0x6a, 0x6f, 0x62, 0x5f, 0x69, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x05, 0x6a, 0x6f, + 0x62, 0x49, 0x64, 0x22, 0x61, 0x0a, 0x16, 0x47, 0x65, 0x74, 0x4a, 0x6f, 0x62, 0x50, 0x69, 0x70, + 0x65, 0x6c, 0x69, 0x6e, 0x65, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x47, 0x0a, + 0x08, 0x70, 0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, + 0x2b, 0x2e, 0x6f, 0x72, 0x67, 0x2e, 0x61, 0x70, 0x61, 0x63, 0x68, 0x65, 0x2e, 0x62, 0x65, 0x61, + 0x6d, 0x2e, 0x6d, 0x6f, 0x64, 0x65, 0x6c, 0x2e, 0x70, 0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e, 0x65, + 0x2e, 0x76, 0x31, 0x2e, 0x50, 0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e, 0x65, 0x52, 0x08, 0x70, 0x69, + 0x70, 0x65, 0x6c, 0x69, 0x6e, 0x65, 0x22, 0x2b, 0x0a, 0x12, 0x4a, 0x6f, 0x62, 0x4d, 0x65, 0x73, + 0x73, 0x61, 0x67, 0x65, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x15, 0x0a, 0x06, + 0x6a, 0x6f, 0x62, 0x5f, 0x69, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x05, 0x6a, 0x6f, + 0x62, 0x49, 0x64, 0x22, 0xfb, 0x02, 0x0a, 0x0a, 0x4a, 0x6f, 0x62, 0x4d, 0x65, 0x73, 0x73, 0x61, + 0x67, 0x65, 0x12, 0x1d, 0x0a, 0x0a, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x5f, 0x69, 0x64, + 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x09, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x49, + 0x64, 0x12, 0x12, 0x0a, 0x04, 0x74, 0x69, 0x6d, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, + 0x04, 0x74, 0x69, 0x6d, 0x65, 0x12, 0x65, 0x0a, 0x0a, 0x69, 0x6d, 0x70, 0x6f, 0x72, 0x74, 0x61, + 0x6e, 0x63, 0x65, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0e, 0x32, 0x45, 0x2e, 0x6f, 0x72, 0x67, 0x2e, + 0x61, 0x70, 0x61, 0x63, 0x68, 0x65, 0x2e, 0x62, 0x65, 0x61, 0x6d, 0x2e, 0x6d, 0x6f, 0x64, 0x65, + 0x6c, 0x2e, 0x6a, 0x6f, 0x62, 0x5f, 0x6d, 0x61, 0x6e, 0x61, 0x67, 0x65, 0x6d, 0x65, 0x6e, 0x74, + 0x2e, 0x76, 0x31, 0x2e, 0x4a, 0x6f, 0x62, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x2e, 0x4d, + 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x49, 0x6d, 0x70, 0x6f, 0x72, 0x74, 0x61, 0x6e, 0x63, 0x65, + 0x52, 0x0a, 0x69, 0x6d, 0x70, 0x6f, 0x72, 0x74, 0x61, 0x6e, 0x63, 0x65, 0x12, 0x21, 0x0a, 0x0c, + 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x5f, 0x74, 0x65, 0x78, 0x74, 0x18, 0x04, 0x20, 0x01, + 0x28, 0x09, 0x52, 0x0b, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x54, 0x65, 0x78, 0x74, 0x22, + 0xaf, 0x01, 0x0a, 0x11, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x49, 0x6d, 0x70, 0x6f, 0x72, + 0x74, 0x61, 0x6e, 0x63, 0x65, 0x12, 0x22, 0x0a, 0x1e, 0x4d, 0x45, 0x53, 0x53, 0x41, 0x47, 0x45, + 0x5f, 0x49, 0x4d, 0x50, 0x4f, 0x52, 0x54, 0x41, 0x4e, 0x43, 0x45, 0x5f, 0x55, 0x4e, 0x53, 0x50, + 0x45, 0x43, 0x49, 0x46, 0x49, 0x45, 0x44, 0x10, 0x00, 0x12, 0x15, 0x0a, 0x11, 0x4a, 0x4f, 0x42, + 0x5f, 0x4d, 0x45, 0x53, 0x53, 0x41, 0x47, 0x45, 0x5f, 0x44, 0x45, 0x42, 0x55, 0x47, 0x10, 0x01, + 0x12, 0x18, 0x0a, 0x14, 0x4a, 0x4f, 0x42, 0x5f, 0x4d, 0x45, 0x53, 0x53, 0x41, 0x47, 0x45, 0x5f, + 0x44, 0x45, 0x54, 0x41, 0x49, 0x4c, 0x45, 0x44, 0x10, 0x02, 0x12, 0x15, 0x0a, 0x11, 0x4a, 0x4f, + 0x42, 0x5f, 0x4d, 0x45, 0x53, 0x53, 0x41, 0x47, 0x45, 0x5f, 0x42, 0x41, 0x53, 0x49, 0x43, 0x10, + 0x03, 0x12, 0x17, 0x0a, 0x13, 0x4a, 0x4f, 0x42, 0x5f, 0x4d, 0x45, 0x53, 0x53, 0x41, 0x47, 0x45, + 0x5f, 0x57, 0x41, 0x52, 0x4e, 0x49, 0x4e, 0x47, 0x10, 0x04, 0x12, 0x15, 0x0a, 0x11, 0x4a, 0x4f, + 0x42, 0x5f, 0x4d, 0x45, 0x53, 0x53, 0x41, 0x47, 0x45, 0x5f, 0x45, 0x52, 0x52, 0x4f, 0x52, 0x10, + 0x05, 0x22, 0xe4, 0x01, 0x0a, 0x13, 0x4a, 0x6f, 0x62, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, + 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x60, 0x0a, 0x10, 0x6d, 0x65, 0x73, + 0x73, 0x61, 0x67, 0x65, 0x5f, 0x72, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x18, 0x01, 0x20, + 0x01, 0x28, 0x0b, 0x32, 0x33, 0x2e, 0x6f, 0x72, 0x67, 0x2e, 0x61, 0x70, 0x61, 0x63, 0x68, 0x65, + 0x2e, 0x62, 0x65, 0x61, 0x6d, 0x2e, 0x6d, 0x6f, 0x64, 0x65, 0x6c, 0x2e, 0x6a, 0x6f, 0x62, 0x5f, + 0x6d, 0x61, 0x6e, 0x61, 0x67, 0x65, 0x6d, 0x65, 0x6e, 0x74, 0x2e, 0x76, 0x31, 0x2e, 0x4a, 0x6f, + 0x62, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x48, 0x00, 0x52, 0x0f, 0x6d, 0x65, 0x73, 0x73, + 0x61, 0x67, 0x65, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x5f, 0x0a, 0x0e, 0x73, + 0x74, 0x61, 0x74, 0x65, 0x5f, 0x72, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x18, 0x02, 0x20, + 0x01, 0x28, 0x0b, 0x32, 0x36, 0x2e, 0x6f, 0x72, 0x67, 0x2e, 0x61, 0x70, 0x61, 0x63, 0x68, 0x65, + 0x2e, 0x62, 0x65, 0x61, 0x6d, 0x2e, 0x6d, 0x6f, 0x64, 0x65, 0x6c, 0x2e, 0x6a, 0x6f, 0x62, 0x5f, + 0x6d, 0x61, 0x6e, 0x61, 0x67, 0x65, 0x6d, 0x65, 0x6e, 0x74, 0x2e, 0x76, 0x31, 0x2e, 0x4a, 0x6f, + 0x62, 0x53, 0x74, 0x61, 0x74, 0x65, 0x45, 0x76, 0x65, 0x6e, 0x74, 0x48, 0x00, 0x52, 0x0d, 0x73, + 0x74, 0x61, 0x74, 0x65, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x42, 0x0a, 0x0a, 0x08, + 0x72, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0xb7, 0x01, 0x0a, 0x08, 0x4a, 0x6f, 0x62, + 0x53, 0x74, 0x61, 0x74, 0x65, 0x22, 0xaa, 0x01, 0x0a, 0x04, 0x45, 0x6e, 0x75, 0x6d, 0x12, 0x0f, + 0x0a, 0x0b, 0x55, 0x4e, 0x53, 0x50, 0x45, 0x43, 0x49, 0x46, 0x49, 0x45, 0x44, 0x10, 0x00, 0x12, + 0x0b, 0x0a, 0x07, 0x53, 0x54, 0x4f, 0x50, 0x50, 0x45, 0x44, 0x10, 0x01, 0x12, 0x0b, 0x0a, 0x07, + 0x52, 0x55, 0x4e, 0x4e, 0x49, 0x4e, 0x47, 0x10, 0x02, 0x12, 0x08, 0x0a, 0x04, 0x44, 0x4f, 0x4e, + 0x45, 0x10, 0x03, 0x12, 0x0a, 0x0a, 0x06, 0x46, 0x41, 0x49, 0x4c, 0x45, 0x44, 0x10, 0x04, 0x12, + 0x0d, 0x0a, 0x09, 0x43, 0x41, 0x4e, 0x43, 0x45, 0x4c, 0x4c, 0x45, 0x44, 0x10, 0x05, 0x12, 0x0b, + 0x0a, 0x07, 0x55, 0x50, 0x44, 0x41, 0x54, 0x45, 0x44, 0x10, 0x06, 0x12, 0x0c, 0x0a, 0x08, 0x44, + 0x52, 0x41, 0x49, 0x4e, 0x49, 0x4e, 0x47, 0x10, 0x07, 0x12, 0x0b, 0x0a, 0x07, 0x44, 0x52, 0x41, + 0x49, 0x4e, 0x45, 0x44, 0x10, 0x08, 0x12, 0x0c, 0x0a, 0x08, 0x53, 0x54, 0x41, 0x52, 0x54, 0x49, + 0x4e, 0x47, 0x10, 0x09, 0x12, 0x0e, 0x0a, 0x0a, 0x43, 0x41, 0x4e, 0x43, 0x45, 0x4c, 0x4c, 0x49, + 0x4e, 0x47, 0x10, 0x0a, 0x12, 0x0c, 0x0a, 0x08, 0x55, 0x50, 0x44, 0x41, 0x54, 0x49, 0x4e, 0x47, + 0x10, 0x0b, 0x22, 0x2d, 0x0a, 0x14, 0x47, 0x65, 0x74, 0x4a, 0x6f, 0x62, 0x4d, 0x65, 0x74, 0x72, + 0x69, 0x63, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x15, 0x0a, 0x06, 0x6a, 0x6f, + 0x62, 0x5f, 0x69, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x05, 0x6a, 0x6f, 0x62, 0x49, + 0x64, 0x22, 0x69, 0x0a, 0x15, 0x47, 0x65, 0x74, 0x4a, 0x6f, 0x62, 0x4d, 0x65, 0x74, 0x72, 0x69, + 0x63, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x50, 0x0a, 0x07, 0x6d, 0x65, + 0x74, 0x72, 0x69, 0x63, 0x73, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x36, 0x2e, 0x6f, 0x72, + 0x67, 0x2e, 0x61, 0x70, 0x61, 0x63, 0x68, 0x65, 0x2e, 0x62, 0x65, 0x61, 0x6d, 0x2e, 0x6d, 0x6f, + 0x64, 0x65, 0x6c, 0x2e, 0x6a, 0x6f, 0x62, 0x5f, 0x6d, 0x61, 0x6e, 0x61, 0x67, 0x65, 0x6d, 0x65, + 0x6e, 0x74, 0x2e, 0x76, 0x31, 0x2e, 0x4d, 0x65, 0x74, 0x72, 0x69, 0x63, 0x52, 0x65, 0x73, 0x75, + 0x6c, 0x74, 0x73, 0x52, 0x07, 0x6d, 0x65, 0x74, 0x72, 0x69, 0x63, 0x73, 0x22, 0xb1, 0x01, 0x0a, + 0x0d, 0x4d, 0x65, 0x74, 0x72, 0x69, 0x63, 0x52, 0x65, 0x73, 0x75, 0x6c, 0x74, 0x73, 0x12, 0x4f, + 0x0a, 0x09, 0x61, 0x74, 0x74, 0x65, 0x6d, 0x70, 0x74, 0x65, 0x64, 0x18, 0x01, 0x20, 0x03, 0x28, + 0x0b, 0x32, 0x31, 0x2e, 0x6f, 0x72, 0x67, 0x2e, 0x61, 0x70, 0x61, 0x63, 0x68, 0x65, 0x2e, 0x62, + 0x65, 0x61, 0x6d, 0x2e, 0x6d, 0x6f, 0x64, 0x65, 0x6c, 0x2e, 0x70, 0x69, 0x70, 0x65, 0x6c, 0x69, + 0x6e, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x4d, 0x6f, 0x6e, 0x69, 0x74, 0x6f, 0x72, 0x69, 0x6e, 0x67, + 0x49, 0x6e, 0x66, 0x6f, 0x52, 0x09, 0x61, 0x74, 0x74, 0x65, 0x6d, 0x70, 0x74, 0x65, 0x64, 0x12, + 0x4f, 0x0a, 0x09, 0x63, 0x6f, 0x6d, 0x6d, 0x69, 0x74, 0x74, 0x65, 0x64, 0x18, 0x02, 0x20, 0x03, + 0x28, 0x0b, 0x32, 0x31, 0x2e, 0x6f, 0x72, 0x67, 0x2e, 0x61, 0x70, 0x61, 0x63, 0x68, 0x65, 0x2e, + 0x62, 0x65, 0x61, 0x6d, 0x2e, 0x6d, 0x6f, 0x64, 0x65, 0x6c, 0x2e, 0x70, 0x69, 0x70, 0x65, 0x6c, + 0x69, 0x6e, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x4d, 0x6f, 0x6e, 0x69, 0x74, 0x6f, 0x72, 0x69, 0x6e, + 0x67, 0x49, 0x6e, 0x66, 0x6f, 0x52, 0x09, 0x63, 0x6f, 0x6d, 0x6d, 0x69, 0x74, 0x74, 0x65, 0x64, + 0x22, 0x20, 0x0a, 0x1e, 0x44, 0x65, 0x73, 0x63, 0x72, 0x69, 0x62, 0x65, 0x50, 0x69, 0x70, 0x65, + 0x6c, 0x69, 0x6e, 0x65, 0x4f, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, + 0x73, 0x74, 0x22, 0x65, 0x0a, 0x12, 0x50, 0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e, 0x65, 0x4f, 0x70, + 0x74, 0x69, 0x6f, 0x6e, 0x54, 0x79, 0x70, 0x65, 0x22, 0x4f, 0x0a, 0x04, 0x45, 0x6e, 0x75, 0x6d, + 0x12, 0x0a, 0x0a, 0x06, 0x53, 0x54, 0x52, 0x49, 0x4e, 0x47, 0x10, 0x00, 0x12, 0x0b, 0x0a, 0x07, + 0x42, 0x4f, 0x4f, 0x4c, 0x45, 0x41, 0x4e, 0x10, 0x01, 0x12, 0x0b, 0x0a, 0x07, 0x49, 0x4e, 0x54, + 0x45, 0x47, 0x45, 0x52, 0x10, 0x02, 0x12, 0x0a, 0x0a, 0x06, 0x4e, 0x55, 0x4d, 0x42, 0x45, 0x52, + 0x10, 0x03, 0x12, 0x09, 0x0a, 0x05, 0x41, 0x52, 0x52, 0x41, 0x59, 0x10, 0x04, 0x12, 0x0a, 0x0a, + 0x06, 0x4f, 0x42, 0x4a, 0x45, 0x43, 0x54, 0x10, 0x05, 0x22, 0xe1, 0x01, 0x0a, 0x18, 0x50, 0x69, + 0x70, 0x65, 0x6c, 0x69, 0x6e, 0x65, 0x4f, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x44, 0x65, 0x73, 0x63, + 0x72, 0x69, 0x70, 0x74, 0x6f, 0x72, 0x12, 0x12, 0x0a, 0x04, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x01, + 0x20, 0x01, 0x28, 0x09, 0x52, 0x04, 0x6e, 0x61, 0x6d, 0x65, 0x12, 0x54, 0x0a, 0x04, 0x74, 0x79, + 0x70, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0e, 0x32, 0x40, 0x2e, 0x6f, 0x72, 0x67, 0x2e, 0x61, + 0x70, 0x61, 0x63, 0x68, 0x65, 0x2e, 0x62, 0x65, 0x61, 0x6d, 0x2e, 0x6d, 0x6f, 0x64, 0x65, 0x6c, + 0x2e, 0x6a, 0x6f, 0x62, 0x5f, 0x6d, 0x61, 0x6e, 0x61, 0x67, 0x65, 0x6d, 0x65, 0x6e, 0x74, 0x2e, + 0x76, 0x31, 0x2e, 0x50, 0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e, 0x65, 0x4f, 0x70, 0x74, 0x69, 0x6f, + 0x6e, 0x54, 0x79, 0x70, 0x65, 0x2e, 0x45, 0x6e, 0x75, 0x6d, 0x52, 0x04, 0x74, 0x79, 0x70, 0x65, + 0x12, 0x20, 0x0a, 0x0b, 0x64, 0x65, 0x73, 0x63, 0x72, 0x69, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x18, + 0x03, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0b, 0x64, 0x65, 0x73, 0x63, 0x72, 0x69, 0x70, 0x74, 0x69, + 0x6f, 0x6e, 0x12, 0x23, 0x0a, 0x0d, 0x64, 0x65, 0x66, 0x61, 0x75, 0x6c, 0x74, 0x5f, 0x76, 0x61, + 0x6c, 0x75, 0x65, 0x18, 0x04, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0c, 0x64, 0x65, 0x66, 0x61, 0x75, + 0x6c, 0x74, 0x56, 0x61, 0x6c, 0x75, 0x65, 0x12, 0x14, 0x0a, 0x05, 0x67, 0x72, 0x6f, 0x75, 0x70, + 0x18, 0x05, 0x20, 0x01, 0x28, 0x09, 0x52, 0x05, 0x67, 0x72, 0x6f, 0x75, 0x70, 0x22, 0x7e, 0x0a, + 0x1f, 0x44, 0x65, 0x73, 0x63, 0x72, 0x69, 0x62, 0x65, 0x50, 0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e, + 0x65, 0x4f, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, + 0x12, 0x5b, 0x0a, 0x07, 0x6f, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x18, 0x01, 0x20, 0x03, 0x28, + 0x0b, 0x32, 0x41, 0x2e, 0x6f, 0x72, 0x67, 0x2e, 0x61, 0x70, 0x61, 0x63, 0x68, 0x65, 0x2e, 0x62, + 0x65, 0x61, 0x6d, 0x2e, 0x6d, 0x6f, 0x64, 0x65, 0x6c, 0x2e, 0x6a, 0x6f, 0x62, 0x5f, 0x6d, 0x61, + 0x6e, 0x61, 0x67, 0x65, 0x6d, 0x65, 0x6e, 0x74, 0x2e, 0x76, 0x31, 0x2e, 0x50, 0x69, 0x70, 0x65, + 0x6c, 0x69, 0x6e, 0x65, 0x4f, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x44, 0x65, 0x73, 0x63, 0x72, 0x69, + 0x70, 0x74, 0x6f, 0x72, 0x52, 0x07, 0x6f, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x32, 0xf4, 0x0b, + 0x0a, 0x0a, 0x4a, 0x6f, 0x62, 0x53, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, 0x12, 0x82, 0x01, 0x0a, + 0x07, 0x50, 0x72, 0x65, 0x70, 0x61, 0x72, 0x65, 0x12, 0x3a, 0x2e, 0x6f, 0x72, 0x67, 0x2e, 0x61, + 0x70, 0x61, 0x63, 0x68, 0x65, 0x2e, 0x62, 0x65, 0x61, 0x6d, 0x2e, 0x6d, 0x6f, 0x64, 0x65, 0x6c, + 0x2e, 0x6a, 0x6f, 0x62, 0x5f, 0x6d, 0x61, 0x6e, 0x61, 0x67, 0x65, 0x6d, 0x65, 0x6e, 0x74, 0x2e, + 0x76, 0x31, 0x2e, 0x50, 0x72, 0x65, 0x70, 0x61, 0x72, 0x65, 0x4a, 0x6f, 0x62, 0x52, 0x65, 0x71, + 0x75, 0x65, 0x73, 0x74, 0x1a, 0x3b, 0x2e, 0x6f, 0x72, 0x67, 0x2e, 0x61, 0x70, 0x61, 0x63, 0x68, + 0x65, 0x2e, 0x62, 0x65, 0x61, 0x6d, 0x2e, 0x6d, 0x6f, 0x64, 0x65, 0x6c, 0x2e, 0x6a, 0x6f, 0x62, + 0x5f, 0x6d, 0x61, 0x6e, 0x61, 0x67, 0x65, 0x6d, 0x65, 0x6e, 0x74, 0x2e, 0x76, 0x31, 0x2e, 0x50, + 0x72, 0x65, 0x70, 0x61, 0x72, 0x65, 0x4a, 0x6f, 0x62, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, + 0x65, 0x12, 0x76, 0x0a, 0x03, 0x52, 0x75, 0x6e, 0x12, 0x36, 0x2e, 0x6f, 0x72, 0x67, 0x2e, 0x61, + 0x70, 0x61, 0x63, 0x68, 0x65, 0x2e, 0x62, 0x65, 0x61, 0x6d, 0x2e, 0x6d, 0x6f, 0x64, 0x65, 0x6c, + 0x2e, 0x6a, 0x6f, 0x62, 0x5f, 0x6d, 0x61, 0x6e, 0x61, 0x67, 0x65, 0x6d, 0x65, 0x6e, 0x74, 0x2e, + 0x76, 0x31, 0x2e, 0x52, 0x75, 0x6e, 0x4a, 0x6f, 0x62, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, + 0x1a, 0x37, 0x2e, 0x6f, 0x72, 0x67, 0x2e, 0x61, 0x70, 0x61, 0x63, 0x68, 0x65, 0x2e, 0x62, 0x65, + 0x61, 0x6d, 0x2e, 0x6d, 0x6f, 0x64, 0x65, 0x6c, 0x2e, 0x6a, 0x6f, 0x62, 0x5f, 0x6d, 0x61, 0x6e, + 0x61, 0x67, 0x65, 0x6d, 0x65, 0x6e, 0x74, 0x2e, 0x76, 0x31, 0x2e, 0x52, 0x75, 0x6e, 0x4a, 0x6f, + 0x62, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x7c, 0x0a, 0x07, 0x47, 0x65, 0x74, + 0x4a, 0x6f, 0x62, 0x73, 0x12, 0x37, 0x2e, 0x6f, 0x72, 0x67, 0x2e, 0x61, 0x70, 0x61, 0x63, 0x68, + 0x65, 0x2e, 0x62, 0x65, 0x61, 0x6d, 0x2e, 0x6d, 0x6f, 0x64, 0x65, 0x6c, 0x2e, 0x6a, 0x6f, 0x62, + 0x5f, 0x6d, 0x61, 0x6e, 0x61, 0x67, 0x65, 0x6d, 0x65, 0x6e, 0x74, 0x2e, 0x76, 0x31, 0x2e, 0x47, + 0x65, 0x74, 0x4a, 0x6f, 0x62, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x38, 0x2e, + 0x6f, 0x72, 0x67, 0x2e, 0x61, 0x70, 0x61, 0x63, 0x68, 0x65, 0x2e, 0x62, 0x65, 0x61, 0x6d, 0x2e, + 0x6d, 0x6f, 0x64, 0x65, 0x6c, 0x2e, 0x6a, 0x6f, 0x62, 0x5f, 0x6d, 0x61, 0x6e, 0x61, 0x67, 0x65, + 0x6d, 0x65, 0x6e, 0x74, 0x2e, 0x76, 0x31, 0x2e, 0x47, 0x65, 0x74, 0x4a, 0x6f, 0x62, 0x73, 0x52, + 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x7f, 0x0a, 0x08, 0x47, 0x65, 0x74, 0x53, 0x74, + 0x61, 0x74, 0x65, 0x12, 0x3b, 0x2e, 0x6f, 0x72, 0x67, 0x2e, 0x61, 0x70, 0x61, 0x63, 0x68, 0x65, + 0x2e, 0x62, 0x65, 0x61, 0x6d, 0x2e, 0x6d, 0x6f, 0x64, 0x65, 0x6c, 0x2e, 0x6a, 0x6f, 0x62, 0x5f, + 0x6d, 0x61, 0x6e, 0x61, 0x67, 0x65, 0x6d, 0x65, 0x6e, 0x74, 0x2e, 0x76, 0x31, 0x2e, 0x47, 0x65, + 0x74, 0x4a, 0x6f, 0x62, 0x53, 0x74, 0x61, 0x74, 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, + 0x1a, 0x36, 0x2e, 0x6f, 0x72, 0x67, 0x2e, 0x61, 0x70, 0x61, 0x63, 0x68, 0x65, 0x2e, 0x62, 0x65, + 0x61, 0x6d, 0x2e, 0x6d, 0x6f, 0x64, 0x65, 0x6c, 0x2e, 0x6a, 0x6f, 0x62, 0x5f, 0x6d, 0x61, 0x6e, + 0x61, 0x67, 0x65, 0x6d, 0x65, 0x6e, 0x74, 0x2e, 0x76, 0x31, 0x2e, 0x4a, 0x6f, 0x62, 0x53, 0x74, + 0x61, 0x74, 0x65, 0x45, 0x76, 0x65, 0x6e, 0x74, 0x12, 0x8e, 0x01, 0x0a, 0x0b, 0x47, 0x65, 0x74, + 0x50, 0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e, 0x65, 0x12, 0x3e, 0x2e, 0x6f, 0x72, 0x67, 0x2e, 0x61, + 0x70, 0x61, 0x63, 0x68, 0x65, 0x2e, 0x62, 0x65, 0x61, 0x6d, 0x2e, 0x6d, 0x6f, 0x64, 0x65, 0x6c, + 0x2e, 0x6a, 0x6f, 0x62, 0x5f, 0x6d, 0x61, 0x6e, 0x61, 0x67, 0x65, 0x6d, 0x65, 0x6e, 0x74, 0x2e, + 0x76, 0x31, 0x2e, 0x47, 0x65, 0x74, 0x4a, 0x6f, 0x62, 0x50, 0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e, + 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x3f, 0x2e, 0x6f, 0x72, 0x67, 0x2e, 0x61, + 0x70, 0x61, 0x63, 0x68, 0x65, 0x2e, 0x62, 0x65, 0x61, 0x6d, 0x2e, 0x6d, 0x6f, 0x64, 0x65, 0x6c, + 0x2e, 0x6a, 0x6f, 0x62, 0x5f, 0x6d, 0x61, 0x6e, 0x61, 0x67, 0x65, 0x6d, 0x65, 0x6e, 0x74, 0x2e, + 0x76, 0x31, 0x2e, 0x47, 0x65, 0x74, 0x4a, 0x6f, 0x62, 0x50, 0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e, + 0x65, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x7f, 0x0a, 0x06, 0x43, 0x61, 0x6e, + 0x63, 0x65, 0x6c, 0x12, 0x39, 0x2e, 0x6f, 0x72, 0x67, 0x2e, 0x61, 0x70, 0x61, 0x63, 0x68, 0x65, + 0x2e, 0x62, 0x65, 0x61, 0x6d, 0x2e, 0x6d, 0x6f, 0x64, 0x65, 0x6c, 0x2e, 0x6a, 0x6f, 0x62, 0x5f, + 0x6d, 0x61, 0x6e, 0x61, 0x67, 0x65, 0x6d, 0x65, 0x6e, 0x74, 0x2e, 0x76, 0x31, 0x2e, 0x43, 0x61, + 0x6e, 0x63, 0x65, 0x6c, 0x4a, 0x6f, 0x62, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x3a, + 0x2e, 0x6f, 0x72, 0x67, 0x2e, 0x61, 0x70, 0x61, 0x63, 0x68, 0x65, 0x2e, 0x62, 0x65, 0x61, 0x6d, + 0x2e, 0x6d, 0x6f, 0x64, 0x65, 0x6c, 0x2e, 0x6a, 0x6f, 0x62, 0x5f, 0x6d, 0x61, 0x6e, 0x61, 0x67, + 0x65, 0x6d, 0x65, 0x6e, 0x74, 0x2e, 0x76, 0x31, 0x2e, 0x43, 0x61, 0x6e, 0x63, 0x65, 0x6c, 0x4a, + 0x6f, 0x62, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x7c, 0x0a, 0x05, 0x44, 0x72, + 0x61, 0x69, 0x6e, 0x12, 0x38, 0x2e, 0x6f, 0x72, 0x67, 0x2e, 0x61, 0x70, 0x61, 0x63, 0x68, 0x65, + 0x2e, 0x62, 0x65, 0x61, 0x6d, 0x2e, 0x6d, 0x6f, 0x64, 0x65, 0x6c, 0x2e, 0x6a, 0x6f, 0x62, 0x5f, + 0x6d, 0x61, 0x6e, 0x61, 0x67, 0x65, 0x6d, 0x65, 0x6e, 0x74, 0x2e, 0x76, 0x31, 0x2e, 0x44, 0x72, + 0x61, 0x69, 0x6e, 0x4a, 0x6f, 0x62, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x39, 0x2e, + 0x6f, 0x72, 0x67, 0x2e, 0x61, 0x70, 0x61, 0x63, 0x68, 0x65, 0x2e, 0x62, 0x65, 0x61, 0x6d, 0x2e, + 0x6d, 0x6f, 0x64, 0x65, 0x6c, 0x2e, 0x6a, 0x6f, 0x62, 0x5f, 0x6d, 0x61, 0x6e, 0x61, 0x67, 0x65, + 0x6d, 0x65, 0x6e, 0x74, 0x2e, 0x76, 0x31, 0x2e, 0x44, 0x72, 0x61, 0x69, 0x6e, 0x4a, 0x6f, 0x62, + 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x87, 0x01, 0x0a, 0x0e, 0x47, 0x65, 0x74, + 0x53, 0x74, 0x61, 0x74, 0x65, 0x53, 0x74, 0x72, 0x65, 0x61, 0x6d, 0x12, 0x3b, 0x2e, 0x6f, 0x72, + 0x67, 0x2e, 0x61, 0x70, 0x61, 0x63, 0x68, 0x65, 0x2e, 0x62, 0x65, 0x61, 0x6d, 0x2e, 0x6d, 0x6f, + 0x64, 0x65, 0x6c, 0x2e, 0x6a, 0x6f, 0x62, 0x5f, 0x6d, 0x61, 0x6e, 0x61, 0x67, 0x65, 0x6d, 0x65, + 0x6e, 0x74, 0x2e, 0x76, 0x31, 0x2e, 0x47, 0x65, 0x74, 0x4a, 0x6f, 0x62, 0x53, 0x74, 0x61, 0x74, + 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x36, 0x2e, 0x6f, 0x72, 0x67, 0x2e, 0x61, + 0x70, 0x61, 0x63, 0x68, 0x65, 0x2e, 0x62, 0x65, 0x61, 0x6d, 0x2e, 0x6d, 0x6f, 0x64, 0x65, 0x6c, + 0x2e, 0x6a, 0x6f, 0x62, 0x5f, 0x6d, 0x61, 0x6e, 0x61, 0x67, 0x65, 0x6d, 0x65, 0x6e, 0x74, 0x2e, + 0x76, 0x31, 0x2e, 0x4a, 0x6f, 0x62, 0x53, 0x74, 0x61, 0x74, 0x65, 0x45, 0x76, 0x65, 0x6e, 0x74, + 0x30, 0x01, 0x12, 0x8f, 0x01, 0x0a, 0x10, 0x47, 0x65, 0x74, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, + 0x65, 0x53, 0x74, 0x72, 0x65, 0x61, 0x6d, 0x12, 0x3b, 0x2e, 0x6f, 0x72, 0x67, 0x2e, 0x61, 0x70, + 0x61, 0x63, 0x68, 0x65, 0x2e, 0x62, 0x65, 0x61, 0x6d, 0x2e, 0x6d, 0x6f, 0x64, 0x65, 0x6c, 0x2e, + 0x6a, 0x6f, 0x62, 0x5f, 0x6d, 0x61, 0x6e, 0x61, 0x67, 0x65, 0x6d, 0x65, 0x6e, 0x74, 0x2e, 0x76, + 0x31, 0x2e, 0x4a, 0x6f, 0x62, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x73, 0x52, 0x65, 0x71, + 0x75, 0x65, 0x73, 0x74, 0x1a, 0x3c, 0x2e, 0x6f, 0x72, 0x67, 0x2e, 0x61, 0x70, 0x61, 0x63, 0x68, + 0x65, 0x2e, 0x62, 0x65, 0x61, 0x6d, 0x2e, 0x6d, 0x6f, 0x64, 0x65, 0x6c, 0x2e, 0x6a, 0x6f, 0x62, + 0x5f, 0x6d, 0x61, 0x6e, 0x61, 0x67, 0x65, 0x6d, 0x65, 0x6e, 0x74, 0x2e, 0x76, 0x31, 0x2e, 0x4a, + 0x6f, 0x62, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, + 0x73, 0x65, 0x30, 0x01, 0x12, 0x8e, 0x01, 0x0a, 0x0d, 0x47, 0x65, 0x74, 0x4a, 0x6f, 0x62, 0x4d, + 0x65, 0x74, 0x72, 0x69, 0x63, 0x73, 0x12, 0x3d, 0x2e, 0x6f, 0x72, 0x67, 0x2e, 0x61, 0x70, 0x61, + 0x63, 0x68, 0x65, 0x2e, 0x62, 0x65, 0x61, 0x6d, 0x2e, 0x6d, 0x6f, 0x64, 0x65, 0x6c, 0x2e, 0x6a, + 0x6f, 0x62, 0x5f, 0x6d, 0x61, 0x6e, 0x61, 0x67, 0x65, 0x6d, 0x65, 0x6e, 0x74, 0x2e, 0x76, 0x31, + 0x2e, 0x47, 0x65, 0x74, 0x4a, 0x6f, 0x62, 0x4d, 0x65, 0x74, 0x72, 0x69, 0x63, 0x73, 0x52, 0x65, + 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x3e, 0x2e, 0x6f, 0x72, 0x67, 0x2e, 0x61, 0x70, 0x61, 0x63, + 0x68, 0x65, 0x2e, 0x62, 0x65, 0x61, 0x6d, 0x2e, 0x6d, 0x6f, 0x64, 0x65, 0x6c, 0x2e, 0x6a, 0x6f, + 0x62, 0x5f, 0x6d, 0x61, 0x6e, 0x61, 0x67, 0x65, 0x6d, 0x65, 0x6e, 0x74, 0x2e, 0x76, 0x31, 0x2e, + 0x47, 0x65, 0x74, 0x4a, 0x6f, 0x62, 0x4d, 0x65, 0x74, 0x72, 0x69, 0x63, 0x73, 0x52, 0x65, 0x73, + 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0xac, 0x01, 0x0a, 0x17, 0x44, 0x65, 0x73, 0x63, 0x72, 0x69, + 0x62, 0x65, 0x50, 0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e, 0x65, 0x4f, 0x70, 0x74, 0x69, 0x6f, 0x6e, + 0x73, 0x12, 0x47, 0x2e, 0x6f, 0x72, 0x67, 0x2e, 0x61, 0x70, 0x61, 0x63, 0x68, 0x65, 0x2e, 0x62, + 0x65, 0x61, 0x6d, 0x2e, 0x6d, 0x6f, 0x64, 0x65, 0x6c, 0x2e, 0x6a, 0x6f, 0x62, 0x5f, 0x6d, 0x61, + 0x6e, 0x61, 0x67, 0x65, 0x6d, 0x65, 0x6e, 0x74, 0x2e, 0x76, 0x31, 0x2e, 0x44, 0x65, 0x73, 0x63, + 0x72, 0x69, 0x62, 0x65, 0x50, 0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e, 0x65, 0x4f, 0x70, 0x74, 0x69, + 0x6f, 0x6e, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x48, 0x2e, 0x6f, 0x72, 0x67, + 0x2e, 0x61, 0x70, 0x61, 0x63, 0x68, 0x65, 0x2e, 0x62, 0x65, 0x61, 0x6d, 0x2e, 0x6d, 0x6f, 0x64, + 0x65, 0x6c, 0x2e, 0x6a, 0x6f, 0x62, 0x5f, 0x6d, 0x61, 0x6e, 0x61, 0x67, 0x65, 0x6d, 0x65, 0x6e, + 0x74, 0x2e, 0x76, 0x31, 0x2e, 0x44, 0x65, 0x73, 0x63, 0x72, 0x69, 0x62, 0x65, 0x50, 0x69, 0x70, + 0x65, 0x6c, 0x69, 0x6e, 0x65, 0x4f, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x52, 0x65, 0x73, 0x70, + 0x6f, 0x6e, 0x73, 0x65, 0x42, 0x84, 0x01, 0x0a, 0x26, 0x6f, 0x72, 0x67, 0x2e, 0x61, 0x70, 0x61, + 0x63, 0x68, 0x65, 0x2e, 0x62, 0x65, 0x61, 0x6d, 0x2e, 0x6d, 0x6f, 0x64, 0x65, 0x6c, 0x2e, 0x6a, + 0x6f, 0x62, 0x6d, 0x61, 0x6e, 0x61, 0x67, 0x65, 0x6d, 0x65, 0x6e, 0x74, 0x2e, 0x76, 0x31, 0x42, + 0x06, 0x4a, 0x6f, 0x62, 0x41, 0x70, 0x69, 0x5a, 0x52, 0x67, 0x69, 0x74, 0x68, 0x75, 0x62, 0x2e, + 0x63, 0x6f, 0x6d, 0x2f, 0x61, 0x70, 0x61, 0x63, 0x68, 0x65, 0x2f, 0x62, 0x65, 0x61, 0x6d, 0x2f, + 0x73, 0x64, 0x6b, 0x73, 0x2f, 0x76, 0x32, 0x2f, 0x67, 0x6f, 0x2f, 0x70, 0x6b, 0x67, 0x2f, 0x62, + 0x65, 0x61, 0x6d, 0x2f, 0x6d, 0x6f, 0x64, 0x65, 0x6c, 0x2f, 0x6a, 0x6f, 0x62, 0x6d, 0x61, 0x6e, + 0x61, 0x67, 0x65, 0x6d, 0x65, 0x6e, 0x74, 0x5f, 0x76, 0x31, 0x3b, 0x6a, 0x6f, 0x62, 0x6d, 0x61, + 0x6e, 0x61, 0x67, 0x65, 0x6d, 0x65, 0x6e, 0x74, 0x5f, 0x76, 0x31, 0x62, 0x06, 0x70, 0x72, 0x6f, + 0x74, 0x6f, 0x33, +} + +var file_org_apache_beam_model_job_management_v1_beam_job_api_proto_enumTypes = make([]protoimpl.EnumInfo, 3) +var file_org_apache_beam_model_job_management_v1_beam_job_api_proto_msgTypes = make([]protoimpl.MessageInfo, 26) +var file_org_apache_beam_model_job_management_v1_beam_job_api_proto_goTypes = []any{ + (JobMessage_MessageImportance)(0), // 0: org.apache.beam.model.job_management.v1.JobMessage.MessageImportance + (JobState_Enum)(0), // 1: org.apache.beam.model.job_management.v1.JobState.Enum + (PipelineOptionType_Enum)(0), // 2: org.apache.beam.model.job_management.v1.PipelineOptionType.Enum + (*PrepareJobRequest)(nil), // 3: org.apache.beam.model.job_management.v1.PrepareJobRequest + (*PrepareJobResponse)(nil), // 4: org.apache.beam.model.job_management.v1.PrepareJobResponse + (*RunJobRequest)(nil), // 5: org.apache.beam.model.job_management.v1.RunJobRequest + (*RunJobResponse)(nil), // 6: org.apache.beam.model.job_management.v1.RunJobResponse + (*CancelJobRequest)(nil), // 7: org.apache.beam.model.job_management.v1.CancelJobRequest + (*CancelJobResponse)(nil), // 8: org.apache.beam.model.job_management.v1.CancelJobResponse + (*DrainJobRequest)(nil), // 9: org.apache.beam.model.job_management.v1.DrainJobRequest + (*DrainJobResponse)(nil), // 10: org.apache.beam.model.job_management.v1.DrainJobResponse + (*JobInfo)(nil), // 11: org.apache.beam.model.job_management.v1.JobInfo + (*GetJobsRequest)(nil), // 12: org.apache.beam.model.job_management.v1.GetJobsRequest + (*GetJobsResponse)(nil), // 13: org.apache.beam.model.job_management.v1.GetJobsResponse + (*GetJobStateRequest)(nil), // 14: org.apache.beam.model.job_management.v1.GetJobStateRequest + (*JobStateEvent)(nil), // 15: org.apache.beam.model.job_management.v1.JobStateEvent + (*GetJobPipelineRequest)(nil), // 16: org.apache.beam.model.job_management.v1.GetJobPipelineRequest + (*GetJobPipelineResponse)(nil), // 17: org.apache.beam.model.job_management.v1.GetJobPipelineResponse + (*JobMessagesRequest)(nil), // 18: org.apache.beam.model.job_management.v1.JobMessagesRequest + (*JobMessage)(nil), // 19: org.apache.beam.model.job_management.v1.JobMessage + (*JobMessagesResponse)(nil), // 20: org.apache.beam.model.job_management.v1.JobMessagesResponse + (*JobState)(nil), // 21: org.apache.beam.model.job_management.v1.JobState + (*GetJobMetricsRequest)(nil), // 22: org.apache.beam.model.job_management.v1.GetJobMetricsRequest + (*GetJobMetricsResponse)(nil), // 23: org.apache.beam.model.job_management.v1.GetJobMetricsResponse + (*MetricResults)(nil), // 24: org.apache.beam.model.job_management.v1.MetricResults + (*DescribePipelineOptionsRequest)(nil), // 25: org.apache.beam.model.job_management.v1.DescribePipelineOptionsRequest + (*PipelineOptionType)(nil), // 26: org.apache.beam.model.job_management.v1.PipelineOptionType + (*PipelineOptionDescriptor)(nil), // 27: org.apache.beam.model.job_management.v1.PipelineOptionDescriptor + (*DescribePipelineOptionsResponse)(nil), // 28: org.apache.beam.model.job_management.v1.DescribePipelineOptionsResponse + (*pipeline_v1.Pipeline)(nil), // 29: org.apache.beam.model.pipeline.v1.Pipeline + (*structpb.Struct)(nil), // 30: google.protobuf.Struct + (*pipeline_v1.ApiServiceDescriptor)(nil), // 31: org.apache.beam.model.pipeline.v1.ApiServiceDescriptor + (*timestamppb.Timestamp)(nil), // 32: google.protobuf.Timestamp + (*pipeline_v1.MonitoringInfo)(nil), // 33: org.apache.beam.model.pipeline.v1.MonitoringInfo +} +var file_org_apache_beam_model_job_management_v1_beam_job_api_proto_depIdxs = []int32{ + 29, // 0: org.apache.beam.model.job_management.v1.PrepareJobRequest.pipeline:type_name -> org.apache.beam.model.pipeline.v1.Pipeline + 30, // 1: org.apache.beam.model.job_management.v1.PrepareJobRequest.pipeline_options:type_name -> google.protobuf.Struct + 31, // 2: org.apache.beam.model.job_management.v1.PrepareJobResponse.artifact_staging_endpoint:type_name -> org.apache.beam.model.pipeline.v1.ApiServiceDescriptor + 1, // 3: org.apache.beam.model.job_management.v1.CancelJobResponse.state:type_name -> org.apache.beam.model.job_management.v1.JobState.Enum + 1, // 4: org.apache.beam.model.job_management.v1.DrainJobResponse.state:type_name -> org.apache.beam.model.job_management.v1.JobState.Enum + 30, // 5: org.apache.beam.model.job_management.v1.JobInfo.pipeline_options:type_name -> google.protobuf.Struct + 1, // 6: org.apache.beam.model.job_management.v1.JobInfo.state:type_name -> org.apache.beam.model.job_management.v1.JobState.Enum + 11, // 7: org.apache.beam.model.job_management.v1.GetJobsResponse.job_info:type_name -> org.apache.beam.model.job_management.v1.JobInfo + 1, // 8: org.apache.beam.model.job_management.v1.JobStateEvent.state:type_name -> org.apache.beam.model.job_management.v1.JobState.Enum + 32, // 9: org.apache.beam.model.job_management.v1.JobStateEvent.timestamp:type_name -> google.protobuf.Timestamp + 29, // 10: org.apache.beam.model.job_management.v1.GetJobPipelineResponse.pipeline:type_name -> org.apache.beam.model.pipeline.v1.Pipeline + 0, // 11: org.apache.beam.model.job_management.v1.JobMessage.importance:type_name -> org.apache.beam.model.job_management.v1.JobMessage.MessageImportance + 19, // 12: org.apache.beam.model.job_management.v1.JobMessagesResponse.message_response:type_name -> org.apache.beam.model.job_management.v1.JobMessage + 15, // 13: org.apache.beam.model.job_management.v1.JobMessagesResponse.state_response:type_name -> org.apache.beam.model.job_management.v1.JobStateEvent + 24, // 14: org.apache.beam.model.job_management.v1.GetJobMetricsResponse.metrics:type_name -> org.apache.beam.model.job_management.v1.MetricResults + 33, // 15: org.apache.beam.model.job_management.v1.MetricResults.attempted:type_name -> org.apache.beam.model.pipeline.v1.MonitoringInfo + 33, // 16: org.apache.beam.model.job_management.v1.MetricResults.committed:type_name -> org.apache.beam.model.pipeline.v1.MonitoringInfo + 2, // 17: org.apache.beam.model.job_management.v1.PipelineOptionDescriptor.type:type_name -> org.apache.beam.model.job_management.v1.PipelineOptionType.Enum + 27, // 18: org.apache.beam.model.job_management.v1.DescribePipelineOptionsResponse.options:type_name -> org.apache.beam.model.job_management.v1.PipelineOptionDescriptor + 3, // 19: org.apache.beam.model.job_management.v1.JobService.Prepare:input_type -> org.apache.beam.model.job_management.v1.PrepareJobRequest + 5, // 20: org.apache.beam.model.job_management.v1.JobService.Run:input_type -> org.apache.beam.model.job_management.v1.RunJobRequest + 12, // 21: org.apache.beam.model.job_management.v1.JobService.GetJobs:input_type -> org.apache.beam.model.job_management.v1.GetJobsRequest + 14, // 22: org.apache.beam.model.job_management.v1.JobService.GetState:input_type -> org.apache.beam.model.job_management.v1.GetJobStateRequest + 16, // 23: org.apache.beam.model.job_management.v1.JobService.GetPipeline:input_type -> org.apache.beam.model.job_management.v1.GetJobPipelineRequest + 7, // 24: org.apache.beam.model.job_management.v1.JobService.Cancel:input_type -> org.apache.beam.model.job_management.v1.CancelJobRequest + 9, // 25: org.apache.beam.model.job_management.v1.JobService.Drain:input_type -> org.apache.beam.model.job_management.v1.DrainJobRequest + 14, // 26: org.apache.beam.model.job_management.v1.JobService.GetStateStream:input_type -> org.apache.beam.model.job_management.v1.GetJobStateRequest + 18, // 27: org.apache.beam.model.job_management.v1.JobService.GetMessageStream:input_type -> org.apache.beam.model.job_management.v1.JobMessagesRequest + 22, // 28: org.apache.beam.model.job_management.v1.JobService.GetJobMetrics:input_type -> org.apache.beam.model.job_management.v1.GetJobMetricsRequest + 25, // 29: org.apache.beam.model.job_management.v1.JobService.DescribePipelineOptions:input_type -> org.apache.beam.model.job_management.v1.DescribePipelineOptionsRequest + 4, // 30: org.apache.beam.model.job_management.v1.JobService.Prepare:output_type -> org.apache.beam.model.job_management.v1.PrepareJobResponse + 6, // 31: org.apache.beam.model.job_management.v1.JobService.Run:output_type -> org.apache.beam.model.job_management.v1.RunJobResponse + 13, // 32: org.apache.beam.model.job_management.v1.JobService.GetJobs:output_type -> org.apache.beam.model.job_management.v1.GetJobsResponse + 15, // 33: org.apache.beam.model.job_management.v1.JobService.GetState:output_type -> org.apache.beam.model.job_management.v1.JobStateEvent + 17, // 34: org.apache.beam.model.job_management.v1.JobService.GetPipeline:output_type -> org.apache.beam.model.job_management.v1.GetJobPipelineResponse + 8, // 35: org.apache.beam.model.job_management.v1.JobService.Cancel:output_type -> org.apache.beam.model.job_management.v1.CancelJobResponse + 10, // 36: org.apache.beam.model.job_management.v1.JobService.Drain:output_type -> org.apache.beam.model.job_management.v1.DrainJobResponse + 15, // 37: org.apache.beam.model.job_management.v1.JobService.GetStateStream:output_type -> org.apache.beam.model.job_management.v1.JobStateEvent + 20, // 38: org.apache.beam.model.job_management.v1.JobService.GetMessageStream:output_type -> org.apache.beam.model.job_management.v1.JobMessagesResponse + 23, // 39: org.apache.beam.model.job_management.v1.JobService.GetJobMetrics:output_type -> org.apache.beam.model.job_management.v1.GetJobMetricsResponse + 28, // 40: org.apache.beam.model.job_management.v1.JobService.DescribePipelineOptions:output_type -> org.apache.beam.model.job_management.v1.DescribePipelineOptionsResponse + 30, // [30:41] is the sub-list for method output_type + 19, // [19:30] is the sub-list for method input_type + 19, // [19:19] is the sub-list for extension type_name + 19, // [19:19] is the sub-list for extension extendee + 0, // [0:19] is the sub-list for field type_name +} + +func init() { file_org_apache_beam_model_job_management_v1_beam_job_api_proto_init() } +func file_org_apache_beam_model_job_management_v1_beam_job_api_proto_init() { + if File_org_apache_beam_model_job_management_v1_beam_job_api_proto != nil { + return + } + file_org_apache_beam_model_job_management_v1_beam_job_api_proto_msgTypes[17].OneofWrappers = []any{ + (*jobMessagesResponse_MessageResponse)(nil), + (*jobMessagesResponse_StateResponse)(nil), + } + type x struct{} + out := protoimpl.TypeBuilder{ + File: protoimpl.DescBuilder{ + GoPackagePath: reflect.TypeOf(x{}).PkgPath(), + RawDescriptor: file_org_apache_beam_model_job_management_v1_beam_job_api_proto_rawDesc, + NumEnums: 3, + NumMessages: 26, + NumExtensions: 0, + NumServices: 1, + }, + GoTypes: file_org_apache_beam_model_job_management_v1_beam_job_api_proto_goTypes, + DependencyIndexes: file_org_apache_beam_model_job_management_v1_beam_job_api_proto_depIdxs, + EnumInfos: file_org_apache_beam_model_job_management_v1_beam_job_api_proto_enumTypes, + MessageInfos: file_org_apache_beam_model_job_management_v1_beam_job_api_proto_msgTypes, + }.Build() + File_org_apache_beam_model_job_management_v1_beam_job_api_proto = out.File + file_org_apache_beam_model_job_management_v1_beam_job_api_proto_rawDesc = nil + file_org_apache_beam_model_job_management_v1_beam_job_api_proto_goTypes = nil + file_org_apache_beam_model_job_management_v1_beam_job_api_proto_depIdxs = nil +} diff --git a/sdks/go/pkg/beam/model/pipeline_v1/beam_runner_api.pb.go b/sdks/go/pkg/beam/model/pipeline_v1/beam_runner_api.pb.go index 723256738cb0..a6fde91773ed 100644 --- a/sdks/go/pkg/beam/model/pipeline_v1/beam_runner_api.pb.go +++ b/sdks/go/pkg/beam/model/pipeline_v1/beam_runner_api.pb.go @@ -21,10 +21,12 @@ // Code generated by protoc-gen-go. DO NOT EDIT. // versions: -// protoc-gen-go v1.27.1 +// protoc-gen-go v1.36.0 // protoc v5.27.3 // source: org/apache/beam/model/pipeline/v1/beam_runner_api.proto +//go:build !protoopaque + package pipeline_v1 import ( @@ -32,7 +34,6 @@ import ( protoimpl "google.golang.org/protobuf/runtime/protoimpl" descriptorpb "google.golang.org/protobuf/types/descriptorpb" reflect "reflect" - sync "sync" ) const ( @@ -97,11 +98,6 @@ func (x BeamConstants_Constants) Number() protoreflect.EnumNumber { return protoreflect.EnumNumber(x) } -// Deprecated: Use BeamConstants_Constants.Descriptor instead. -func (BeamConstants_Constants) EnumDescriptor() ([]byte, []int) { - return file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_rawDescGZIP(), []int{0, 0} -} - // Primitive transforms may not specify composite sub-transforms. type StandardPTransforms_Primitives int32 @@ -265,11 +261,6 @@ func (x StandardPTransforms_Primitives) Number() protoreflect.EnumNumber { return protoreflect.EnumNumber(x) } -// Deprecated: Use StandardPTransforms_Primitives.Descriptor instead. -func (StandardPTransforms_Primitives) EnumDescriptor() ([]byte, []int) { - return file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_rawDescGZIP(), []int{4, 0} -} - type StandardPTransforms_DeprecatedPrimitives int32 const ( @@ -315,11 +306,6 @@ func (x StandardPTransforms_DeprecatedPrimitives) Number() protoreflect.EnumNumb return protoreflect.EnumNumber(x) } -// Deprecated: Use StandardPTransforms_DeprecatedPrimitives.Descriptor instead. -func (StandardPTransforms_DeprecatedPrimitives) EnumDescriptor() ([]byte, []int) { - return file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_rawDescGZIP(), []int{4, 1} -} - type StandardPTransforms_Composites int32 const ( @@ -401,11 +387,6 @@ func (x StandardPTransforms_Composites) Number() protoreflect.EnumNumber { return protoreflect.EnumNumber(x) } -// Deprecated: Use StandardPTransforms_Composites.Descriptor instead. -func (StandardPTransforms_Composites) EnumDescriptor() ([]byte, []int) { - return file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_rawDescGZIP(), []int{4, 2} -} - // Payload for all of these: CombinePayload type StandardPTransforms_CombineComponents int32 @@ -477,11 +458,6 @@ func (x StandardPTransforms_CombineComponents) Number() protoreflect.EnumNumber return protoreflect.EnumNumber(x) } -// Deprecated: Use StandardPTransforms_CombineComponents.Descriptor instead. -func (StandardPTransforms_CombineComponents) EnumDescriptor() ([]byte, []int) { - return file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_rawDescGZIP(), []int{4, 3} -} - // Payload for all of these: ParDoPayload containing the user's SDF type StandardPTransforms_SplittableParDoComponents int32 @@ -555,11 +531,6 @@ func (x StandardPTransforms_SplittableParDoComponents) Number() protoreflect.Enu return protoreflect.EnumNumber(x) } -// Deprecated: Use StandardPTransforms_SplittableParDoComponents.Descriptor instead. -func (StandardPTransforms_SplittableParDoComponents) EnumDescriptor() ([]byte, []int) { - return file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_rawDescGZIP(), []int{4, 4} -} - // Payload for all of these: GroupIntoBatchesPayload type StandardPTransforms_GroupIntoBatchesComponents int32 @@ -599,11 +570,6 @@ func (x StandardPTransforms_GroupIntoBatchesComponents) Number() protoreflect.En return protoreflect.EnumNumber(x) } -// Deprecated: Use StandardPTransforms_GroupIntoBatchesComponents.Descriptor instead. -func (StandardPTransforms_GroupIntoBatchesComponents) EnumDescriptor() ([]byte, []int) { - return file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_rawDescGZIP(), []int{4, 5} -} - type StandardSideInputTypes_Enum int32 const ( @@ -653,11 +619,6 @@ func (x StandardSideInputTypes_Enum) Number() protoreflect.EnumNumber { return protoreflect.EnumNumber(x) } -// Deprecated: Use StandardSideInputTypes_Enum.Descriptor instead. -func (StandardSideInputTypes_Enum) EnumDescriptor() ([]byte, []int) { - return file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_rawDescGZIP(), []int{5, 0} -} - type StandardUserStateTypes_Enum int32 const ( @@ -714,11 +675,6 @@ func (x StandardUserStateTypes_Enum) Number() protoreflect.EnumNumber { return protoreflect.EnumNumber(x) } -// Deprecated: Use StandardUserStateTypes_Enum.Descriptor instead. -func (StandardUserStateTypes_Enum) EnumDescriptor() ([]byte, []int) { - return file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_rawDescGZIP(), []int{6, 0} -} - type IsBounded_Enum int32 const ( @@ -763,11 +719,6 @@ func (x IsBounded_Enum) Number() protoreflect.EnumNumber { return protoreflect.EnumNumber(x) } -// Deprecated: Use IsBounded_Enum.Descriptor instead. -func (IsBounded_Enum) EnumDescriptor() ([]byte, []int) { - return file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_rawDescGZIP(), []int{18, 0} -} - type StandardCoders_Enum int32 const ( @@ -1086,11 +1037,6 @@ func (x StandardCoders_Enum) Number() protoreflect.EnumNumber { return protoreflect.EnumNumber(x) } -// Deprecated: Use StandardCoders_Enum.Descriptor instead. -func (StandardCoders_Enum) EnumDescriptor() ([]byte, []int) { - return file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_rawDescGZIP(), []int{30, 0} -} - type MergeStatus_Enum int32 const ( @@ -1146,11 +1092,6 @@ func (x MergeStatus_Enum) Number() protoreflect.EnumNumber { return protoreflect.EnumNumber(x) } -// Deprecated: Use MergeStatus_Enum.Descriptor instead. -func (MergeStatus_Enum) EnumDescriptor() ([]byte, []int) { - return file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_rawDescGZIP(), []int{32, 0} -} - type AccumulationMode_Enum int32 const ( @@ -1201,11 +1142,6 @@ func (x AccumulationMode_Enum) Number() protoreflect.EnumNumber { return protoreflect.EnumNumber(x) } -// Deprecated: Use AccumulationMode_Enum.Descriptor instead. -func (AccumulationMode_Enum) EnumDescriptor() ([]byte, []int) { - return file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_rawDescGZIP(), []int{33, 0} -} - type ClosingBehavior_Enum int32 const ( @@ -1253,11 +1189,6 @@ func (x ClosingBehavior_Enum) Number() protoreflect.EnumNumber { return protoreflect.EnumNumber(x) } -// Deprecated: Use ClosingBehavior_Enum.Descriptor instead. -func (ClosingBehavior_Enum) EnumDescriptor() ([]byte, []int) { - return file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_rawDescGZIP(), []int{34, 0} -} - type OnTimeBehavior_Enum int32 const ( @@ -1305,11 +1236,6 @@ func (x OnTimeBehavior_Enum) Number() protoreflect.EnumNumber { return protoreflect.EnumNumber(x) } -// Deprecated: Use OnTimeBehavior_Enum.Descriptor instead. -func (OnTimeBehavior_Enum) EnumDescriptor() ([]byte, []int) { - return file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_rawDescGZIP(), []int{35, 0} -} - type OutputTime_Enum int32 const ( @@ -1362,11 +1288,6 @@ func (x OutputTime_Enum) Number() protoreflect.EnumNumber { return protoreflect.EnumNumber(x) } -// Deprecated: Use OutputTime_Enum.Descriptor instead. -func (OutputTime_Enum) EnumDescriptor() ([]byte, []int) { - return file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_rawDescGZIP(), []int{36, 0} -} - type TimeDomain_Enum int32 const ( @@ -1414,11 +1335,6 @@ func (x TimeDomain_Enum) Number() protoreflect.EnumNumber { return protoreflect.EnumNumber(x) } -// Deprecated: Use TimeDomain_Enum.Descriptor instead. -func (TimeDomain_Enum) EnumDescriptor() ([]byte, []int) { - return file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_rawDescGZIP(), []int{37, 0} -} - type StandardArtifacts_Types int32 const ( @@ -1484,11 +1400,6 @@ func (x StandardArtifacts_Types) Number() protoreflect.EnumNumber { return protoreflect.EnumNumber(x) } -// Deprecated: Use StandardArtifacts_Types.Descriptor instead. -func (StandardArtifacts_Types) EnumDescriptor() ([]byte, []int) { - return file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_rawDescGZIP(), []int{41, 0} -} - type StandardArtifacts_Roles int32 const ( @@ -1541,11 +1452,6 @@ func (x StandardArtifacts_Roles) Number() protoreflect.EnumNumber { return protoreflect.EnumNumber(x) } -// Deprecated: Use StandardArtifacts_Roles.Descriptor instead. -func (StandardArtifacts_Roles) EnumDescriptor() ([]byte, []int) { - return file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_rawDescGZIP(), []int{41, 1} -} - type StandardEnvironments_Environments int32 const ( @@ -1607,11 +1513,6 @@ func (x StandardEnvironments_Environments) Number() protoreflect.EnumNumber { return protoreflect.EnumNumber(x) } -// Deprecated: Use StandardEnvironments_Environments.Descriptor instead. -func (StandardEnvironments_Environments) EnumDescriptor() ([]byte, []int) { - return file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_rawDescGZIP(), []int{51, 0} -} - type StandardProtocols_Enum int32 const ( @@ -1655,21 +1556,24 @@ const ( // Indicates whether the SDK sets the consuming_received_data field on // the ProcessBundleProgressResponse messages. StandardProtocols_SDK_CONSUMING_RECEIVED_DATA StandardProtocols_Enum = 9 + // Indicates whether the SDK supports ordered list state. + StandardProtocols_ORDERED_LIST_STATE StandardProtocols_Enum = 10 ) // Enum value maps for StandardProtocols_Enum. var ( StandardProtocols_Enum_name = map[int32]string{ - 0: "LEGACY_PROGRESS_REPORTING", - 1: "PROGRESS_REPORTING", - 2: "WORKER_STATUS", - 3: "MULTI_CORE_BUNDLE_PROCESSING", - 5: "SIBLING_WORKERS", - 4: "HARNESS_MONITORING_INFOS", - 6: "CONTROL_REQUEST_ELEMENTS_EMBEDDING", - 7: "STATE_CACHING", - 8: "DATA_SAMPLING", - 9: "SDK_CONSUMING_RECEIVED_DATA", + 0: "LEGACY_PROGRESS_REPORTING", + 1: "PROGRESS_REPORTING", + 2: "WORKER_STATUS", + 3: "MULTI_CORE_BUNDLE_PROCESSING", + 5: "SIBLING_WORKERS", + 4: "HARNESS_MONITORING_INFOS", + 6: "CONTROL_REQUEST_ELEMENTS_EMBEDDING", + 7: "STATE_CACHING", + 8: "DATA_SAMPLING", + 9: "SDK_CONSUMING_RECEIVED_DATA", + 10: "ORDERED_LIST_STATE", } StandardProtocols_Enum_value = map[string]int32{ "LEGACY_PROGRESS_REPORTING": 0, @@ -1682,6 +1586,7 @@ var ( "STATE_CACHING": 7, "DATA_SAMPLING": 8, "SDK_CONSUMING_RECEIVED_DATA": 9, + "ORDERED_LIST_STATE": 10, } ) @@ -1707,11 +1612,6 @@ func (x StandardProtocols_Enum) Number() protoreflect.EnumNumber { return protoreflect.EnumNumber(x) } -// Deprecated: Use StandardProtocols_Enum.Descriptor instead. -func (StandardProtocols_Enum) EnumDescriptor() ([]byte, []int) { - return file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_rawDescGZIP(), []int{56, 0} -} - type StandardRunnerProtocols_Enum int32 const ( @@ -1762,11 +1662,6 @@ func (x StandardRunnerProtocols_Enum) Number() protoreflect.EnumNumber { return protoreflect.EnumNumber(x) } -// Deprecated: Use StandardRunnerProtocols_Enum.Descriptor instead. -func (StandardRunnerProtocols_Enum) EnumDescriptor() ([]byte, []int) { - return file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_rawDescGZIP(), []int{57, 0} -} - type StandardRequirements_Enum int32 const ( @@ -1832,11 +1727,6 @@ func (x StandardRequirements_Enum) Number() protoreflect.EnumNumber { return protoreflect.EnumNumber(x) } -// Deprecated: Use StandardRequirements_Enum.Descriptor instead. -func (StandardRequirements_Enum) EnumDescriptor() ([]byte, []int) { - return file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_rawDescGZIP(), []int{58, 0} -} - type StandardDisplayData_DisplayData int32 const ( @@ -1877,11 +1767,6 @@ func (x StandardDisplayData_DisplayData) Number() protoreflect.EnumNumber { return protoreflect.EnumNumber(x) } -// Deprecated: Use StandardDisplayData_DisplayData.Descriptor instead. -func (StandardDisplayData_DisplayData) EnumDescriptor() ([]byte, []int) { - return file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_rawDescGZIP(), []int{60, 0} -} - type StandardResourceHints_Enum int32 const ( @@ -1936,24 +1821,17 @@ func (x StandardResourceHints_Enum) Number() protoreflect.EnumNumber { return protoreflect.EnumNumber(x) } -// Deprecated: Use StandardResourceHints_Enum.Descriptor instead. -func (StandardResourceHints_Enum) EnumDescriptor() ([]byte, []int) { - return file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_rawDescGZIP(), []int{65, 0} -} - type BeamConstants struct { - state protoimpl.MessageState - sizeCache protoimpl.SizeCache + state protoimpl.MessageState `protogen:"hybrid.v1"` unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache } func (x *BeamConstants) Reset() { *x = BeamConstants{} - if protoimpl.UnsafeEnabled { - mi := &file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_msgTypes[0] - ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) - ms.StoreMessageInfo(mi) - } + mi := &file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_msgTypes[0] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) } func (x *BeamConstants) String() string { @@ -1964,7 +1842,7 @@ func (*BeamConstants) ProtoMessage() {} func (x *BeamConstants) ProtoReflect() protoreflect.Message { mi := &file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_msgTypes[0] - if protoimpl.UnsafeEnabled && x != nil { + if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { ms.StoreMessageInfo(mi) @@ -1974,41 +1852,45 @@ func (x *BeamConstants) ProtoReflect() protoreflect.Message { return mi.MessageOf(x) } -// Deprecated: Use BeamConstants.ProtoReflect.Descriptor instead. -func (*BeamConstants) Descriptor() ([]byte, []int) { - return file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_rawDescGZIP(), []int{0} +type BeamConstants_builder struct { + _ [0]func() // Prevents comparability and use of unkeyed literals for the builder. + +} + +func (b0 BeamConstants_builder) Build() *BeamConstants { + m0 := &BeamConstants{} + b, x := &b0, m0 + _, _ = b, x + return m0 } // A set of mappings from id to message. This is included as an optional field // on any proto message that may contain references needing resolution. type Components struct { - state protoimpl.MessageState - sizeCache protoimpl.SizeCache - unknownFields protoimpl.UnknownFields - + state protoimpl.MessageState `protogen:"hybrid.v1"` // (Required) A map from pipeline-scoped id to PTransform. // // Keys of the transforms map may be used by runners to identify pipeline // steps. Hence it's recommended to use strings that are not too long that // match regex '[A-Za-z0-9-_]+'. - Transforms map[string]*PTransform `protobuf:"bytes,1,rep,name=transforms,proto3" json:"transforms,omitempty" protobuf_key:"bytes,1,opt,name=key,proto3" protobuf_val:"bytes,2,opt,name=value,proto3"` + Transforms map[string]*PTransform `protobuf:"bytes,1,rep,name=transforms,proto3" json:"transforms,omitempty" protobuf_key:"bytes,1,opt,name=key" protobuf_val:"bytes,2,opt,name=value"` // (Required) A map from pipeline-scoped id to PCollection. - Pcollections map[string]*PCollection `protobuf:"bytes,2,rep,name=pcollections,proto3" json:"pcollections,omitempty" protobuf_key:"bytes,1,opt,name=key,proto3" protobuf_val:"bytes,2,opt,name=value,proto3"` + Pcollections map[string]*PCollection `protobuf:"bytes,2,rep,name=pcollections,proto3" json:"pcollections,omitempty" protobuf_key:"bytes,1,opt,name=key" protobuf_val:"bytes,2,opt,name=value"` // (Required) A map from pipeline-scoped id to WindowingStrategy. - WindowingStrategies map[string]*WindowingStrategy `protobuf:"bytes,3,rep,name=windowing_strategies,json=windowingStrategies,proto3" json:"windowing_strategies,omitempty" protobuf_key:"bytes,1,opt,name=key,proto3" protobuf_val:"bytes,2,opt,name=value,proto3"` + WindowingStrategies map[string]*WindowingStrategy `protobuf:"bytes,3,rep,name=windowing_strategies,json=windowingStrategies,proto3" json:"windowing_strategies,omitempty" protobuf_key:"bytes,1,opt,name=key" protobuf_val:"bytes,2,opt,name=value"` // (Required) A map from pipeline-scoped id to Coder. - Coders map[string]*Coder `protobuf:"bytes,4,rep,name=coders,proto3" json:"coders,omitempty" protobuf_key:"bytes,1,opt,name=key,proto3" protobuf_val:"bytes,2,opt,name=value,proto3"` + Coders map[string]*Coder `protobuf:"bytes,4,rep,name=coders,proto3" json:"coders,omitempty" protobuf_key:"bytes,1,opt,name=key" protobuf_val:"bytes,2,opt,name=value"` // (Required) A map from pipeline-scoped id to Environment. - Environments map[string]*Environment `protobuf:"bytes,5,rep,name=environments,proto3" json:"environments,omitempty" protobuf_key:"bytes,1,opt,name=key,proto3" protobuf_val:"bytes,2,opt,name=value,proto3"` + Environments map[string]*Environment `protobuf:"bytes,5,rep,name=environments,proto3" json:"environments,omitempty" protobuf_key:"bytes,1,opt,name=key" protobuf_val:"bytes,2,opt,name=value"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache } func (x *Components) Reset() { *x = Components{} - if protoimpl.UnsafeEnabled { - mi := &file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_msgTypes[1] - ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) - ms.StoreMessageInfo(mi) - } + mi := &file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_msgTypes[1] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) } func (x *Components) String() string { @@ -2019,7 +1901,7 @@ func (*Components) ProtoMessage() {} func (x *Components) ProtoReflect() protoreflect.Message { mi := &file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_msgTypes[1] - if protoimpl.UnsafeEnabled && x != nil { + if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { ms.StoreMessageInfo(mi) @@ -2029,11 +1911,6 @@ func (x *Components) ProtoReflect() protoreflect.Message { return mi.MessageOf(x) } -// Deprecated: Use Components.ProtoReflect.Descriptor instead. -func (*Components) Descriptor() ([]byte, []int) { - return file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_rawDescGZIP(), []int{1} -} - func (x *Components) GetTransforms() map[string]*PTransform { if x != nil { return x.Transforms @@ -2069,6 +1946,57 @@ func (x *Components) GetEnvironments() map[string]*Environment { return nil } +func (x *Components) SetTransforms(v map[string]*PTransform) { + x.Transforms = v +} + +func (x *Components) SetPcollections(v map[string]*PCollection) { + x.Pcollections = v +} + +func (x *Components) SetWindowingStrategies(v map[string]*WindowingStrategy) { + x.WindowingStrategies = v +} + +func (x *Components) SetCoders(v map[string]*Coder) { + x.Coders = v +} + +func (x *Components) SetEnvironments(v map[string]*Environment) { + x.Environments = v +} + +type Components_builder struct { + _ [0]func() // Prevents comparability and use of unkeyed literals for the builder. + + // (Required) A map from pipeline-scoped id to PTransform. + // + // Keys of the transforms map may be used by runners to identify pipeline + // steps. Hence it's recommended to use strings that are not too long that + // match regex '[A-Za-z0-9-_]+'. + Transforms map[string]*PTransform + // (Required) A map from pipeline-scoped id to PCollection. + Pcollections map[string]*PCollection + // (Required) A map from pipeline-scoped id to WindowingStrategy. + WindowingStrategies map[string]*WindowingStrategy + // (Required) A map from pipeline-scoped id to Coder. + Coders map[string]*Coder + // (Required) A map from pipeline-scoped id to Environment. + Environments map[string]*Environment +} + +func (b0 Components_builder) Build() *Components { + m0 := &Components{} + b, x := &b0, m0 + _, _ = b, x + x.Transforms = b.Transforms + x.Pcollections = b.Pcollections + x.WindowingStrategies = b.WindowingStrategies + x.Coders = b.Coders + x.Environments = b.Environments + return m0 +} + // A Pipeline is a hierarchical graph of PTransforms, linked // by PCollections. A typical graph may look like: // @@ -2081,10 +2009,7 @@ func (x *Components) GetEnvironments() map[string]*Environment { // PCollections, SDK environments, coders, etc., for // supporting compact reuse and arbitrary graph structure. type Pipeline struct { - state protoimpl.MessageState - sizeCache protoimpl.SizeCache - unknownFields protoimpl.UnknownFields - + state protoimpl.MessageState `protogen:"hybrid.v1"` // (Required) The coders, UDFs, graph nodes, etc, that make up // this pipeline. Components *Components `protobuf:"bytes,1,opt,name=components,proto3" json:"components,omitempty"` @@ -2100,16 +2025,16 @@ type Pipeline struct { // may indicate that a runner must inspect new fields on a component or // provide additional guarantees when processing specific transforms. // A runner should reject any pipelines with unknown requirements. - Requirements []string `protobuf:"bytes,4,rep,name=requirements,proto3" json:"requirements,omitempty"` + Requirements []string `protobuf:"bytes,4,rep,name=requirements,proto3" json:"requirements,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache } func (x *Pipeline) Reset() { *x = Pipeline{} - if protoimpl.UnsafeEnabled { - mi := &file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_msgTypes[2] - ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) - ms.StoreMessageInfo(mi) - } + mi := &file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_msgTypes[2] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) } func (x *Pipeline) String() string { @@ -2120,7 +2045,7 @@ func (*Pipeline) ProtoMessage() {} func (x *Pipeline) ProtoReflect() protoreflect.Message { mi := &file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_msgTypes[2] - if protoimpl.UnsafeEnabled && x != nil { + if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { ms.StoreMessageInfo(mi) @@ -2130,11 +2055,6 @@ func (x *Pipeline) ProtoReflect() protoreflect.Message { return mi.MessageOf(x) } -// Deprecated: Use Pipeline.ProtoReflect.Descriptor instead. -func (*Pipeline) Descriptor() ([]byte, []int) { - return file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_rawDescGZIP(), []int{2} -} - func (x *Pipeline) GetComponents() *Components { if x != nil { return x.Components @@ -2163,6 +2083,65 @@ func (x *Pipeline) GetRequirements() []string { return nil } +func (x *Pipeline) SetComponents(v *Components) { + x.Components = v +} + +func (x *Pipeline) SetRootTransformIds(v []string) { + x.RootTransformIds = v +} + +func (x *Pipeline) SetDisplayData(v []*DisplayData) { + x.DisplayData = v +} + +func (x *Pipeline) SetRequirements(v []string) { + x.Requirements = v +} + +func (x *Pipeline) HasComponents() bool { + if x == nil { + return false + } + return x.Components != nil +} + +func (x *Pipeline) ClearComponents() { + x.Components = nil +} + +type Pipeline_builder struct { + _ [0]func() // Prevents comparability and use of unkeyed literals for the builder. + + // (Required) The coders, UDFs, graph nodes, etc, that make up + // this pipeline. + Components *Components + // (Required) The ids of all PTransforms that are not contained within another + // PTransform. These must be in shallow topological order, so that traversing + // them recursively in this order yields a recursively topological traversal. + RootTransformIds []string + // (Optional) Static display data for the pipeline. If there is none, + // it may be omitted. + DisplayData []*DisplayData + // (Optional) A set of requirements that the runner MUST understand and be + // able to faithfully provide in order to execute this pipeline. These + // may indicate that a runner must inspect new fields on a component or + // provide additional guarantees when processing specific transforms. + // A runner should reject any pipelines with unknown requirements. + Requirements []string +} + +func (b0 Pipeline_builder) Build() *Pipeline { + m0 := &Pipeline{} + b, x := &b0, m0 + _, _ = b, x + x.Components = b.Components + x.RootTransformIds = b.RootTransformIds + x.DisplayData = b.DisplayData + x.Requirements = b.Requirements + return m0 +} + // Transforms are the operations in your pipeline, and provide a generic // processing framework. You provide processing logic in the form of a function // object (colloquially referred to as “user code”), and your user code is @@ -2181,10 +2160,7 @@ func (x *Pipeline) GetRequirements() []string { // collection. You can also define your own more complex composite transforms to // fit your pipeline’s exact use case. type PTransform struct { - state protoimpl.MessageState - sizeCache protoimpl.SizeCache - unknownFields protoimpl.UnknownFields - + state protoimpl.MessageState `protogen:"hybrid.v1"` // (Required) A unique name for the application node. // // Ideally, this should be stable over multiple evolutions of a pipeline @@ -2226,7 +2202,7 @@ type PTransform struct { // // All inputs are recorded here so that the topological ordering of // the graph is consistent whether or not the payload is understood. - Inputs map[string]string `protobuf:"bytes,3,rep,name=inputs,proto3" json:"inputs,omitempty" protobuf_key:"bytes,1,opt,name=key,proto3" protobuf_val:"bytes,2,opt,name=value,proto3"` + Inputs map[string]string `protobuf:"bytes,3,rep,name=inputs,proto3" json:"inputs,omitempty" protobuf_key:"bytes,1,opt,name=key" protobuf_val:"bytes,2,opt,name=value"` // (Required) A map from local names of outputs (unique only within this map, // and likely embedded in the transform payload and serialized user code) // to PCollection ids. @@ -2236,7 +2212,7 @@ type PTransform struct { // // - for a ParDo transform, these are tags on PCollections, which will be // embedded in the DoFn. - Outputs map[string]string `protobuf:"bytes,4,rep,name=outputs,proto3" json:"outputs,omitempty" protobuf_key:"bytes,1,opt,name=key,proto3" protobuf_val:"bytes,2,opt,name=value,proto3"` + Outputs map[string]string `protobuf:"bytes,4,rep,name=outputs,proto3" json:"outputs,omitempty" protobuf_key:"bytes,1,opt,name=key" protobuf_val:"bytes,2,opt,name=value"` // (Optional) Static display data for this PTransform application. If // there is none, it may be omitted. DisplayData []*DisplayData `protobuf:"bytes,6,rep,name=display_data,json=displayData,proto3" json:"display_data,omitempty"` @@ -2252,16 +2228,16 @@ type PTransform struct { // A runner MAY ignore types of annotations it doesn't understand. Therefore // annotations MUST NOT be used for metadata that can affect correct // execution of the transform. - Annotations map[string][]byte `protobuf:"bytes,8,rep,name=annotations,proto3" json:"annotations,omitempty" protobuf_key:"bytes,1,opt,name=key,proto3" protobuf_val:"bytes,2,opt,name=value,proto3"` + Annotations map[string][]byte `protobuf:"bytes,8,rep,name=annotations,proto3" json:"annotations,omitempty" protobuf_key:"bytes,1,opt,name=key" protobuf_val:"bytes,2,opt,name=value"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache } func (x *PTransform) Reset() { *x = PTransform{} - if protoimpl.UnsafeEnabled { - mi := &file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_msgTypes[3] - ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) - ms.StoreMessageInfo(mi) - } + mi := &file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_msgTypes[3] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) } func (x *PTransform) String() string { @@ -2272,7 +2248,7 @@ func (*PTransform) ProtoMessage() {} func (x *PTransform) ProtoReflect() protoreflect.Message { mi := &file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_msgTypes[3] - if protoimpl.UnsafeEnabled && x != nil { + if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { ms.StoreMessageInfo(mi) @@ -2282,11 +2258,6 @@ func (x *PTransform) ProtoReflect() protoreflect.Message { return mi.MessageOf(x) } -// Deprecated: Use PTransform.ProtoReflect.Descriptor instead. -func (*PTransform) Descriptor() ([]byte, []int) { - return file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_rawDescGZIP(), []int{3} -} - func (x *PTransform) GetUniqueName() string { if x != nil { return x.UniqueName @@ -2343,19 +2314,148 @@ func (x *PTransform) GetAnnotations() map[string][]byte { return nil } +func (x *PTransform) SetUniqueName(v string) { + x.UniqueName = v +} + +func (x *PTransform) SetSpec(v *FunctionSpec) { + x.Spec = v +} + +func (x *PTransform) SetSubtransforms(v []string) { + x.Subtransforms = v +} + +func (x *PTransform) SetInputs(v map[string]string) { + x.Inputs = v +} + +func (x *PTransform) SetOutputs(v map[string]string) { + x.Outputs = v +} + +func (x *PTransform) SetDisplayData(v []*DisplayData) { + x.DisplayData = v +} + +func (x *PTransform) SetEnvironmentId(v string) { + x.EnvironmentId = v +} + +func (x *PTransform) SetAnnotations(v map[string][]byte) { + x.Annotations = v +} + +func (x *PTransform) HasSpec() bool { + if x == nil { + return false + } + return x.Spec != nil +} + +func (x *PTransform) ClearSpec() { + x.Spec = nil +} + +type PTransform_builder struct { + _ [0]func() // Prevents comparability and use of unkeyed literals for the builder. + + // (Required) A unique name for the application node. + // + // Ideally, this should be stable over multiple evolutions of a pipeline + // for the purposes of logging and associating pipeline state with a node, + // etc. + // + // If it is not stable, then the runner decides what will happen. But, most + // importantly, it must always be here and be unique, even if it is + // autogenerated. + UniqueName string + // (Optional) A URN and payload that, together, fully defined the semantics + // of this transform. + // + // If absent, this must be an "anonymous" composite transform. + // + // For primitive transform in the Runner API, this is required, and the + // payloads are well-defined messages. When the URN indicates ParDo it + // is a ParDoPayload, and so on. For some special composite transforms, + // the payload is also officially defined. See StandardPTransforms for + // details. + Spec *FunctionSpec + // (Optional) A list of the ids of transforms that it contains. + // + // Primitive transforms (see StandardPTransforms.Primitives) are not allowed + // to specify subtransforms. + // + // Note that a composite transform may have zero subtransforms as long as it + // only outputs PCollections that are in its inputs. + Subtransforms []string + // (Required) A map from local names of inputs (unique only with this map, and + // likely embedded in the transform payload and serialized user code) to + // PCollection ids. + // + // The payload for this transform may clarify the relationship of these + // inputs. For example: + // + // - for a Flatten transform they are merged + // - for a ParDo transform, some may be side inputs + // + // All inputs are recorded here so that the topological ordering of + // the graph is consistent whether or not the payload is understood. + Inputs map[string]string + // (Required) A map from local names of outputs (unique only within this map, + // and likely embedded in the transform payload and serialized user code) + // to PCollection ids. + // + // The URN or payload for this transform node may clarify the type and + // relationship of these outputs. For example: + // + // - for a ParDo transform, these are tags on PCollections, which will be + // embedded in the DoFn. + Outputs map[string]string + // (Optional) Static display data for this PTransform application. If + // there is none, it may be omitted. + DisplayData []*DisplayData + // Environment where the current PTransform should be executed in. + // + // Transforms that are required to be implemented by a runner must omit this. + // All other transforms are required to specify this. + EnvironmentId string + // (Optional) A map from URNs designating a type of annotation, to the + // annotation in binary format. For example, an annotation could indicate + // that this PTransform has specific privacy properties. + // + // A runner MAY ignore types of annotations it doesn't understand. Therefore + // annotations MUST NOT be used for metadata that can affect correct + // execution of the transform. + Annotations map[string][]byte +} + +func (b0 PTransform_builder) Build() *PTransform { + m0 := &PTransform{} + b, x := &b0, m0 + _, _ = b, x + x.UniqueName = b.UniqueName + x.Spec = b.Spec + x.Subtransforms = b.Subtransforms + x.Inputs = b.Inputs + x.Outputs = b.Outputs + x.DisplayData = b.DisplayData + x.EnvironmentId = b.EnvironmentId + x.Annotations = b.Annotations + return m0 +} + type StandardPTransforms struct { - state protoimpl.MessageState - sizeCache protoimpl.SizeCache + state protoimpl.MessageState `protogen:"hybrid.v1"` unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache } func (x *StandardPTransforms) Reset() { *x = StandardPTransforms{} - if protoimpl.UnsafeEnabled { - mi := &file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_msgTypes[4] - ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) - ms.StoreMessageInfo(mi) - } + mi := &file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_msgTypes[4] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) } func (x *StandardPTransforms) String() string { @@ -2366,7 +2466,7 @@ func (*StandardPTransforms) ProtoMessage() {} func (x *StandardPTransforms) ProtoReflect() protoreflect.Message { mi := &file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_msgTypes[4] - if protoimpl.UnsafeEnabled && x != nil { + if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { ms.StoreMessageInfo(mi) @@ -2376,24 +2476,29 @@ func (x *StandardPTransforms) ProtoReflect() protoreflect.Message { return mi.MessageOf(x) } -// Deprecated: Use StandardPTransforms.ProtoReflect.Descriptor instead. -func (*StandardPTransforms) Descriptor() ([]byte, []int) { - return file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_rawDescGZIP(), []int{4} +type StandardPTransforms_builder struct { + _ [0]func() // Prevents comparability and use of unkeyed literals for the builder. + +} + +func (b0 StandardPTransforms_builder) Build() *StandardPTransforms { + m0 := &StandardPTransforms{} + b, x := &b0, m0 + _, _ = b, x + return m0 } type StandardSideInputTypes struct { - state protoimpl.MessageState - sizeCache protoimpl.SizeCache + state protoimpl.MessageState `protogen:"hybrid.v1"` unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache } func (x *StandardSideInputTypes) Reset() { *x = StandardSideInputTypes{} - if protoimpl.UnsafeEnabled { - mi := &file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_msgTypes[5] - ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) - ms.StoreMessageInfo(mi) - } + mi := &file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_msgTypes[5] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) } func (x *StandardSideInputTypes) String() string { @@ -2404,7 +2509,7 @@ func (*StandardSideInputTypes) ProtoMessage() {} func (x *StandardSideInputTypes) ProtoReflect() protoreflect.Message { mi := &file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_msgTypes[5] - if protoimpl.UnsafeEnabled && x != nil { + if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { ms.StoreMessageInfo(mi) @@ -2414,24 +2519,29 @@ func (x *StandardSideInputTypes) ProtoReflect() protoreflect.Message { return mi.MessageOf(x) } -// Deprecated: Use StandardSideInputTypes.ProtoReflect.Descriptor instead. -func (*StandardSideInputTypes) Descriptor() ([]byte, []int) { - return file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_rawDescGZIP(), []int{5} +type StandardSideInputTypes_builder struct { + _ [0]func() // Prevents comparability and use of unkeyed literals for the builder. + +} + +func (b0 StandardSideInputTypes_builder) Build() *StandardSideInputTypes { + m0 := &StandardSideInputTypes{} + b, x := &b0, m0 + _, _ = b, x + return m0 } type StandardUserStateTypes struct { - state protoimpl.MessageState - sizeCache protoimpl.SizeCache + state protoimpl.MessageState `protogen:"hybrid.v1"` unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache } func (x *StandardUserStateTypes) Reset() { *x = StandardUserStateTypes{} - if protoimpl.UnsafeEnabled { - mi := &file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_msgTypes[6] - ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) - ms.StoreMessageInfo(mi) - } + mi := &file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_msgTypes[6] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) } func (x *StandardUserStateTypes) String() string { @@ -2442,7 +2552,7 @@ func (*StandardUserStateTypes) ProtoMessage() {} func (x *StandardUserStateTypes) ProtoReflect() protoreflect.Message { mi := &file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_msgTypes[6] - if protoimpl.UnsafeEnabled && x != nil { + if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { ms.StoreMessageInfo(mi) @@ -2452,17 +2562,21 @@ func (x *StandardUserStateTypes) ProtoReflect() protoreflect.Message { return mi.MessageOf(x) } -// Deprecated: Use StandardUserStateTypes.ProtoReflect.Descriptor instead. -func (*StandardUserStateTypes) Descriptor() ([]byte, []int) { - return file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_rawDescGZIP(), []int{6} +type StandardUserStateTypes_builder struct { + _ [0]func() // Prevents comparability and use of unkeyed literals for the builder. + +} + +func (b0 StandardUserStateTypes_builder) Build() *StandardUserStateTypes { + m0 := &StandardUserStateTypes{} + b, x := &b0, m0 + _, _ = b, x + return m0 } // A PCollection! type PCollection struct { - state protoimpl.MessageState - sizeCache protoimpl.SizeCache - unknownFields protoimpl.UnknownFields - + state protoimpl.MessageState `protogen:"hybrid.v1"` // (Required) A unique name for the PCollection. // // Ideally, this should be stable over multiple evolutions of a pipeline @@ -2480,16 +2594,16 @@ type PCollection struct { WindowingStrategyId string `protobuf:"bytes,4,opt,name=windowing_strategy_id,json=windowingStrategyId,proto3" json:"windowing_strategy_id,omitempty"` // (Optional) Static display data for the PCollection. If there is none, // it may be omitted. - DisplayData []*DisplayData `protobuf:"bytes,5,rep,name=display_data,json=displayData,proto3" json:"display_data,omitempty"` + DisplayData []*DisplayData `protobuf:"bytes,5,rep,name=display_data,json=displayData,proto3" json:"display_data,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache } func (x *PCollection) Reset() { *x = PCollection{} - if protoimpl.UnsafeEnabled { - mi := &file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_msgTypes[7] - ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) - ms.StoreMessageInfo(mi) - } + mi := &file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_msgTypes[7] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) } func (x *PCollection) String() string { @@ -2500,7 +2614,7 @@ func (*PCollection) ProtoMessage() {} func (x *PCollection) ProtoReflect() protoreflect.Message { mi := &file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_msgTypes[7] - if protoimpl.UnsafeEnabled && x != nil { + if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { ms.StoreMessageInfo(mi) @@ -2510,11 +2624,6 @@ func (x *PCollection) ProtoReflect() protoreflect.Message { return mi.MessageOf(x) } -// Deprecated: Use PCollection.ProtoReflect.Descriptor instead. -func (*PCollection) Descriptor() ([]byte, []int) { - return file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_rawDescGZIP(), []int{7} -} - func (x *PCollection) GetUniqueName() string { if x != nil { return x.UniqueName @@ -2550,25 +2659,77 @@ func (x *PCollection) GetDisplayData() []*DisplayData { return nil } +func (x *PCollection) SetUniqueName(v string) { + x.UniqueName = v +} + +func (x *PCollection) SetCoderId(v string) { + x.CoderId = v +} + +func (x *PCollection) SetIsBounded(v IsBounded_Enum) { + x.IsBounded = v +} + +func (x *PCollection) SetWindowingStrategyId(v string) { + x.WindowingStrategyId = v +} + +func (x *PCollection) SetDisplayData(v []*DisplayData) { + x.DisplayData = v +} + +type PCollection_builder struct { + _ [0]func() // Prevents comparability and use of unkeyed literals for the builder. + + // (Required) A unique name for the PCollection. + // + // Ideally, this should be stable over multiple evolutions of a pipeline + // for the purposes of logging and associating pipeline state with a node, + // etc. + // + // If it is not stable, then the runner decides what will happen. But, most + // importantly, it must always be here, even if it is autogenerated. + UniqueName string + // (Required) The id of the Coder for this PCollection. + CoderId string + // (Required) Whether this PCollection is bounded or unbounded + IsBounded IsBounded_Enum + // (Required) The id of the windowing strategy for this PCollection. + WindowingStrategyId string + // (Optional) Static display data for the PCollection. If there is none, + // it may be omitted. + DisplayData []*DisplayData +} + +func (b0 PCollection_builder) Build() *PCollection { + m0 := &PCollection{} + b, x := &b0, m0 + _, _ = b, x + x.UniqueName = b.UniqueName + x.CoderId = b.CoderId + x.IsBounded = b.IsBounded + x.WindowingStrategyId = b.WindowingStrategyId + x.DisplayData = b.DisplayData + return m0 +} + // The payload for the primitive ParDo transform. type ParDoPayload struct { - state protoimpl.MessageState - sizeCache protoimpl.SizeCache - unknownFields protoimpl.UnknownFields - + state protoimpl.MessageState `protogen:"hybrid.v1"` // (Required) The FunctionSpec of the DoFn. DoFn *FunctionSpec `protobuf:"bytes,1,opt,name=do_fn,json=doFn,proto3" json:"do_fn,omitempty"` // (Optional) A mapping of local input names to side inputs, describing // the expected access pattern. - SideInputs map[string]*SideInput `protobuf:"bytes,3,rep,name=side_inputs,json=sideInputs,proto3" json:"side_inputs,omitempty" protobuf_key:"bytes,1,opt,name=key,proto3" protobuf_val:"bytes,2,opt,name=value,proto3"` + SideInputs map[string]*SideInput `protobuf:"bytes,3,rep,name=side_inputs,json=sideInputs,proto3" json:"side_inputs,omitempty" protobuf_key:"bytes,1,opt,name=key" protobuf_val:"bytes,2,opt,name=value"` // (Optional) A mapping of local state names to state specifications. // If this is set, the stateful processing requirement should also // be placed in the pipeline requirements. - StateSpecs map[string]*StateSpec `protobuf:"bytes,4,rep,name=state_specs,json=stateSpecs,proto3" json:"state_specs,omitempty" protobuf_key:"bytes,1,opt,name=key,proto3" protobuf_val:"bytes,2,opt,name=value,proto3"` + StateSpecs map[string]*StateSpec `protobuf:"bytes,4,rep,name=state_specs,json=stateSpecs,proto3" json:"state_specs,omitempty" protobuf_key:"bytes,1,opt,name=key" protobuf_val:"bytes,2,opt,name=value"` // (Optional) A mapping of local timer family names to timer family // specifications. If this is set, the stateful processing requirement should // also be placed in the pipeline requirements. - TimerFamilySpecs map[string]*TimerFamilySpec `protobuf:"bytes,9,rep,name=timer_family_specs,json=timerFamilySpecs,proto3" json:"timer_family_specs,omitempty" protobuf_key:"bytes,1,opt,name=key,proto3" protobuf_val:"bytes,2,opt,name=value,proto3"` + TimerFamilySpecs map[string]*TimerFamilySpec `protobuf:"bytes,9,rep,name=timer_family_specs,json=timerFamilySpecs,proto3" json:"timer_family_specs,omitempty" protobuf_key:"bytes,1,opt,name=key" protobuf_val:"bytes,2,opt,name=value"` // (Optional) Only set when this ParDo contains a splittable DoFn. // If this is set, the corresponding standard requirement should also // be placed in the pipeline requirements. @@ -2590,15 +2751,15 @@ type ParDoPayload struct { // If this is set, the corresponding standard requirement should also // be placed in the pipeline requirements. OnWindowExpirationTimerFamilySpec string `protobuf:"bytes,12,opt,name=on_window_expiration_timer_family_spec,json=onWindowExpirationTimerFamilySpec,proto3" json:"on_window_expiration_timer_family_spec,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache } func (x *ParDoPayload) Reset() { *x = ParDoPayload{} - if protoimpl.UnsafeEnabled { - mi := &file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_msgTypes[8] - ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) - ms.StoreMessageInfo(mi) - } + mi := &file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_msgTypes[8] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) } func (x *ParDoPayload) String() string { @@ -2609,7 +2770,7 @@ func (*ParDoPayload) ProtoMessage() {} func (x *ParDoPayload) ProtoReflect() protoreflect.Message { mi := &file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_msgTypes[8] - if protoimpl.UnsafeEnabled && x != nil { + if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { ms.StoreMessageInfo(mi) @@ -2619,11 +2780,6 @@ func (x *ParDoPayload) ProtoReflect() protoreflect.Message { return mi.MessageOf(x) } -// Deprecated: Use ParDoPayload.ProtoReflect.Descriptor instead. -func (*ParDoPayload) Descriptor() ([]byte, []int) { - return file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_rawDescGZIP(), []int{8} -} - func (x *ParDoPayload) GetDoFn() *FunctionSpec { if x != nil { return x.DoFn @@ -2687,14 +2843,113 @@ func (x *ParDoPayload) GetOnWindowExpirationTimerFamilySpec() string { return "" } -type StateSpec struct { - state protoimpl.MessageState - sizeCache protoimpl.SizeCache - unknownFields protoimpl.UnknownFields +func (x *ParDoPayload) SetDoFn(v *FunctionSpec) { + x.DoFn = v +} + +func (x *ParDoPayload) SetSideInputs(v map[string]*SideInput) { + x.SideInputs = v +} + +func (x *ParDoPayload) SetStateSpecs(v map[string]*StateSpec) { + x.StateSpecs = v +} + +func (x *ParDoPayload) SetTimerFamilySpecs(v map[string]*TimerFamilySpec) { + x.TimerFamilySpecs = v +} + +func (x *ParDoPayload) SetRestrictionCoderId(v string) { + x.RestrictionCoderId = v +} +func (x *ParDoPayload) SetRequestsFinalization(v bool) { + x.RequestsFinalization = v +} + +func (x *ParDoPayload) SetRequiresTimeSortedInput(v bool) { + x.RequiresTimeSortedInput = v +} + +func (x *ParDoPayload) SetRequiresStableInput(v bool) { + x.RequiresStableInput = v +} + +func (x *ParDoPayload) SetOnWindowExpirationTimerFamilySpec(v string) { + x.OnWindowExpirationTimerFamilySpec = v +} + +func (x *ParDoPayload) HasDoFn() bool { + if x == nil { + return false + } + return x.DoFn != nil +} + +func (x *ParDoPayload) ClearDoFn() { + x.DoFn = nil +} + +type ParDoPayload_builder struct { + _ [0]func() // Prevents comparability and use of unkeyed literals for the builder. + + // (Required) The FunctionSpec of the DoFn. + DoFn *FunctionSpec + // (Optional) A mapping of local input names to side inputs, describing + // the expected access pattern. + SideInputs map[string]*SideInput + // (Optional) A mapping of local state names to state specifications. + // If this is set, the stateful processing requirement should also + // be placed in the pipeline requirements. + StateSpecs map[string]*StateSpec + // (Optional) A mapping of local timer family names to timer family + // specifications. If this is set, the stateful processing requirement should + // also be placed in the pipeline requirements. + TimerFamilySpecs map[string]*TimerFamilySpec + // (Optional) Only set when this ParDo contains a splittable DoFn. + // If this is set, the corresponding standard requirement should also + // be placed in the pipeline requirements. + RestrictionCoderId string + // (Optional) Only set when this ParDo can request bundle finalization. + // If this is set, the corresponding standard requirement should also + // be placed in the pipeline requirements. + RequestsFinalization bool + // Whether this stage requires time sorted input. + // If this is set, the corresponding standard requirement should also + // be placed in the pipeline requirements. + RequiresTimeSortedInput bool + // Whether this stage requires stable input. + // If this is set, the corresponding standard requirement should also + // be placed in the pipeline requirements. + RequiresStableInput bool + // If populated, the name of the timer family spec which should be notified + // on each window expiry. + // If this is set, the corresponding standard requirement should also + // be placed in the pipeline requirements. + OnWindowExpirationTimerFamilySpec string +} + +func (b0 ParDoPayload_builder) Build() *ParDoPayload { + m0 := &ParDoPayload{} + b, x := &b0, m0 + _, _ = b, x + x.DoFn = b.DoFn + x.SideInputs = b.SideInputs + x.StateSpecs = b.StateSpecs + x.TimerFamilySpecs = b.TimerFamilySpecs + x.RestrictionCoderId = b.RestrictionCoderId + x.RequestsFinalization = b.RequestsFinalization + x.RequiresTimeSortedInput = b.RequiresTimeSortedInput + x.RequiresStableInput = b.RequiresStableInput + x.OnWindowExpirationTimerFamilySpec = b.OnWindowExpirationTimerFamilySpec + return m0 +} + +type StateSpec struct { + state protoimpl.MessageState `protogen:"hybrid.v1"` // TODO(BEAM-13930): Deprecate and remove these state specs // - // Types that are assignable to Spec: + // Types that are valid to be assigned to Spec: // // *StateSpec_ReadModifyWriteSpec // *StateSpec_BagSpec @@ -2712,16 +2967,16 @@ type StateSpec struct { // // See StandardUserStateTypes for an enumeration of all user state types // defined. - Protocol *FunctionSpec `protobuf:"bytes,7,opt,name=protocol,proto3" json:"protocol,omitempty"` + Protocol *FunctionSpec `protobuf:"bytes,7,opt,name=protocol,proto3" json:"protocol,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache } func (x *StateSpec) Reset() { *x = StateSpec{} - if protoimpl.UnsafeEnabled { - mi := &file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_msgTypes[9] - ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) - ms.StoreMessageInfo(mi) - } + mi := &file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_msgTypes[9] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) } func (x *StateSpec) String() string { @@ -2732,7 +2987,7 @@ func (*StateSpec) ProtoMessage() {} func (x *StateSpec) ProtoReflect() protoreflect.Message { mi := &file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_msgTypes[9] - if protoimpl.UnsafeEnabled && x != nil { + if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { ms.StoreMessageInfo(mi) @@ -2742,63 +2997,72 @@ func (x *StateSpec) ProtoReflect() protoreflect.Message { return mi.MessageOf(x) } -// Deprecated: Use StateSpec.ProtoReflect.Descriptor instead. -func (*StateSpec) Descriptor() ([]byte, []int) { - return file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_rawDescGZIP(), []int{9} -} - -func (m *StateSpec) GetSpec() isStateSpec_Spec { - if m != nil { - return m.Spec +func (x *StateSpec) GetSpec() isStateSpec_Spec { + if x != nil { + return x.Spec } return nil } func (x *StateSpec) GetReadModifyWriteSpec() *ReadModifyWriteStateSpec { - if x, ok := x.GetSpec().(*StateSpec_ReadModifyWriteSpec); ok { - return x.ReadModifyWriteSpec + if x != nil { + if x, ok := x.Spec.(*StateSpec_ReadModifyWriteSpec); ok { + return x.ReadModifyWriteSpec + } } return nil } func (x *StateSpec) GetBagSpec() *BagStateSpec { - if x, ok := x.GetSpec().(*StateSpec_BagSpec); ok { - return x.BagSpec + if x != nil { + if x, ok := x.Spec.(*StateSpec_BagSpec); ok { + return x.BagSpec + } } return nil } func (x *StateSpec) GetCombiningSpec() *CombiningStateSpec { - if x, ok := x.GetSpec().(*StateSpec_CombiningSpec); ok { - return x.CombiningSpec + if x != nil { + if x, ok := x.Spec.(*StateSpec_CombiningSpec); ok { + return x.CombiningSpec + } } return nil } func (x *StateSpec) GetMapSpec() *MapStateSpec { - if x, ok := x.GetSpec().(*StateSpec_MapSpec); ok { - return x.MapSpec + if x != nil { + if x, ok := x.Spec.(*StateSpec_MapSpec); ok { + return x.MapSpec + } } return nil } func (x *StateSpec) GetSetSpec() *SetStateSpec { - if x, ok := x.GetSpec().(*StateSpec_SetSpec); ok { - return x.SetSpec + if x != nil { + if x, ok := x.Spec.(*StateSpec_SetSpec); ok { + return x.SetSpec + } } return nil } func (x *StateSpec) GetOrderedListSpec() *OrderedListStateSpec { - if x, ok := x.GetSpec().(*StateSpec_OrderedListSpec); ok { - return x.OrderedListSpec + if x != nil { + if x, ok := x.Spec.(*StateSpec_OrderedListSpec); ok { + return x.OrderedListSpec + } } return nil } func (x *StateSpec) GetMultimapSpec() *MultimapStateSpec { - if x, ok := x.GetSpec().(*StateSpec_MultimapSpec); ok { - return x.MultimapSpec + if x != nil { + if x, ok := x.Spec.(*StateSpec_MultimapSpec); ok { + return x.MultimapSpec + } } return nil } @@ -2810,6 +3074,283 @@ func (x *StateSpec) GetProtocol() *FunctionSpec { return nil } +func (x *StateSpec) SetReadModifyWriteSpec(v *ReadModifyWriteStateSpec) { + if v == nil { + x.Spec = nil + return + } + x.Spec = &StateSpec_ReadModifyWriteSpec{v} +} + +func (x *StateSpec) SetBagSpec(v *BagStateSpec) { + if v == nil { + x.Spec = nil + return + } + x.Spec = &StateSpec_BagSpec{v} +} + +func (x *StateSpec) SetCombiningSpec(v *CombiningStateSpec) { + if v == nil { + x.Spec = nil + return + } + x.Spec = &StateSpec_CombiningSpec{v} +} + +func (x *StateSpec) SetMapSpec(v *MapStateSpec) { + if v == nil { + x.Spec = nil + return + } + x.Spec = &StateSpec_MapSpec{v} +} + +func (x *StateSpec) SetSetSpec(v *SetStateSpec) { + if v == nil { + x.Spec = nil + return + } + x.Spec = &StateSpec_SetSpec{v} +} + +func (x *StateSpec) SetOrderedListSpec(v *OrderedListStateSpec) { + if v == nil { + x.Spec = nil + return + } + x.Spec = &StateSpec_OrderedListSpec{v} +} + +func (x *StateSpec) SetMultimapSpec(v *MultimapStateSpec) { + if v == nil { + x.Spec = nil + return + } + x.Spec = &StateSpec_MultimapSpec{v} +} + +func (x *StateSpec) SetProtocol(v *FunctionSpec) { + x.Protocol = v +} + +func (x *StateSpec) HasSpec() bool { + if x == nil { + return false + } + return x.Spec != nil +} + +func (x *StateSpec) HasReadModifyWriteSpec() bool { + if x == nil { + return false + } + _, ok := x.Spec.(*StateSpec_ReadModifyWriteSpec) + return ok +} + +func (x *StateSpec) HasBagSpec() bool { + if x == nil { + return false + } + _, ok := x.Spec.(*StateSpec_BagSpec) + return ok +} + +func (x *StateSpec) HasCombiningSpec() bool { + if x == nil { + return false + } + _, ok := x.Spec.(*StateSpec_CombiningSpec) + return ok +} + +func (x *StateSpec) HasMapSpec() bool { + if x == nil { + return false + } + _, ok := x.Spec.(*StateSpec_MapSpec) + return ok +} + +func (x *StateSpec) HasSetSpec() bool { + if x == nil { + return false + } + _, ok := x.Spec.(*StateSpec_SetSpec) + return ok +} + +func (x *StateSpec) HasOrderedListSpec() bool { + if x == nil { + return false + } + _, ok := x.Spec.(*StateSpec_OrderedListSpec) + return ok +} + +func (x *StateSpec) HasMultimapSpec() bool { + if x == nil { + return false + } + _, ok := x.Spec.(*StateSpec_MultimapSpec) + return ok +} + +func (x *StateSpec) HasProtocol() bool { + if x == nil { + return false + } + return x.Protocol != nil +} + +func (x *StateSpec) ClearSpec() { + x.Spec = nil +} + +func (x *StateSpec) ClearReadModifyWriteSpec() { + if _, ok := x.Spec.(*StateSpec_ReadModifyWriteSpec); ok { + x.Spec = nil + } +} + +func (x *StateSpec) ClearBagSpec() { + if _, ok := x.Spec.(*StateSpec_BagSpec); ok { + x.Spec = nil + } +} + +func (x *StateSpec) ClearCombiningSpec() { + if _, ok := x.Spec.(*StateSpec_CombiningSpec); ok { + x.Spec = nil + } +} + +func (x *StateSpec) ClearMapSpec() { + if _, ok := x.Spec.(*StateSpec_MapSpec); ok { + x.Spec = nil + } +} + +func (x *StateSpec) ClearSetSpec() { + if _, ok := x.Spec.(*StateSpec_SetSpec); ok { + x.Spec = nil + } +} + +func (x *StateSpec) ClearOrderedListSpec() { + if _, ok := x.Spec.(*StateSpec_OrderedListSpec); ok { + x.Spec = nil + } +} + +func (x *StateSpec) ClearMultimapSpec() { + if _, ok := x.Spec.(*StateSpec_MultimapSpec); ok { + x.Spec = nil + } +} + +func (x *StateSpec) ClearProtocol() { + x.Protocol = nil +} + +const StateSpec_Spec_not_set_case case_StateSpec_Spec = 0 +const StateSpec_ReadModifyWriteSpec_case case_StateSpec_Spec = 1 +const StateSpec_BagSpec_case case_StateSpec_Spec = 2 +const StateSpec_CombiningSpec_case case_StateSpec_Spec = 3 +const StateSpec_MapSpec_case case_StateSpec_Spec = 4 +const StateSpec_SetSpec_case case_StateSpec_Spec = 5 +const StateSpec_OrderedListSpec_case case_StateSpec_Spec = 6 +const StateSpec_MultimapSpec_case case_StateSpec_Spec = 8 + +func (x *StateSpec) WhichSpec() case_StateSpec_Spec { + if x == nil { + return StateSpec_Spec_not_set_case + } + switch x.Spec.(type) { + case *StateSpec_ReadModifyWriteSpec: + return StateSpec_ReadModifyWriteSpec_case + case *StateSpec_BagSpec: + return StateSpec_BagSpec_case + case *StateSpec_CombiningSpec: + return StateSpec_CombiningSpec_case + case *StateSpec_MapSpec: + return StateSpec_MapSpec_case + case *StateSpec_SetSpec: + return StateSpec_SetSpec_case + case *StateSpec_OrderedListSpec: + return StateSpec_OrderedListSpec_case + case *StateSpec_MultimapSpec: + return StateSpec_MultimapSpec_case + default: + return StateSpec_Spec_not_set_case + } +} + +type StateSpec_builder struct { + _ [0]func() // Prevents comparability and use of unkeyed literals for the builder. + + // TODO(BEAM-13930): Deprecate and remove these state specs + + // Fields of oneof Spec: + ReadModifyWriteSpec *ReadModifyWriteStateSpec + BagSpec *BagStateSpec + CombiningSpec *CombiningStateSpec + MapSpec *MapStateSpec + SetSpec *SetStateSpec + OrderedListSpec *OrderedListStateSpec + MultimapSpec *MultimapStateSpec + // -- end of Spec + // (Required) URN of the protocol required by this state specification to present + // the desired SDK-specific interface to a UDF. + // + // This protocol defines the SDK harness <-> Runner Harness RPC + // interface for accessing and mutating user state. + // + // See StandardUserStateTypes for an enumeration of all user state types + // defined. + Protocol *FunctionSpec +} + +func (b0 StateSpec_builder) Build() *StateSpec { + m0 := &StateSpec{} + b, x := &b0, m0 + _, _ = b, x + if b.ReadModifyWriteSpec != nil { + x.Spec = &StateSpec_ReadModifyWriteSpec{b.ReadModifyWriteSpec} + } + if b.BagSpec != nil { + x.Spec = &StateSpec_BagSpec{b.BagSpec} + } + if b.CombiningSpec != nil { + x.Spec = &StateSpec_CombiningSpec{b.CombiningSpec} + } + if b.MapSpec != nil { + x.Spec = &StateSpec_MapSpec{b.MapSpec} + } + if b.SetSpec != nil { + x.Spec = &StateSpec_SetSpec{b.SetSpec} + } + if b.OrderedListSpec != nil { + x.Spec = &StateSpec_OrderedListSpec{b.OrderedListSpec} + } + if b.MultimapSpec != nil { + x.Spec = &StateSpec_MultimapSpec{b.MultimapSpec} + } + x.Protocol = b.Protocol + return m0 +} + +type case_StateSpec_Spec protoreflect.FieldNumber + +func (x case_StateSpec_Spec) String() string { + md := file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_msgTypes[9].Descriptor() + if x == 0 { + return "not set" + } + return protoimpl.X.MessageFieldStringOf(md, protoreflect.FieldNumber(x)) +} + type isStateSpec_Spec interface { isStateSpec_Spec() } @@ -2857,20 +3398,17 @@ func (*StateSpec_OrderedListSpec) isStateSpec_Spec() {} func (*StateSpec_MultimapSpec) isStateSpec_Spec() {} type ReadModifyWriteStateSpec struct { - state protoimpl.MessageState - sizeCache protoimpl.SizeCache + state protoimpl.MessageState `protogen:"hybrid.v1"` + CoderId string `protobuf:"bytes,1,opt,name=coder_id,json=coderId,proto3" json:"coder_id,omitempty"` unknownFields protoimpl.UnknownFields - - CoderId string `protobuf:"bytes,1,opt,name=coder_id,json=coderId,proto3" json:"coder_id,omitempty"` + sizeCache protoimpl.SizeCache } func (x *ReadModifyWriteStateSpec) Reset() { *x = ReadModifyWriteStateSpec{} - if protoimpl.UnsafeEnabled { - mi := &file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_msgTypes[10] - ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) - ms.StoreMessageInfo(mi) - } + mi := &file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_msgTypes[10] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) } func (x *ReadModifyWriteStateSpec) String() string { @@ -2881,7 +3419,7 @@ func (*ReadModifyWriteStateSpec) ProtoMessage() {} func (x *ReadModifyWriteStateSpec) ProtoReflect() protoreflect.Message { mi := &file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_msgTypes[10] - if protoimpl.UnsafeEnabled && x != nil { + if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { ms.StoreMessageInfo(mi) @@ -2891,11 +3429,6 @@ func (x *ReadModifyWriteStateSpec) ProtoReflect() protoreflect.Message { return mi.MessageOf(x) } -// Deprecated: Use ReadModifyWriteStateSpec.ProtoReflect.Descriptor instead. -func (*ReadModifyWriteStateSpec) Descriptor() ([]byte, []int) { - return file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_rawDescGZIP(), []int{10} -} - func (x *ReadModifyWriteStateSpec) GetCoderId() string { if x != nil { return x.CoderId @@ -2903,21 +3436,36 @@ func (x *ReadModifyWriteStateSpec) GetCoderId() string { return "" } -type BagStateSpec struct { - state protoimpl.MessageState - sizeCache protoimpl.SizeCache - unknownFields protoimpl.UnknownFields +func (x *ReadModifyWriteStateSpec) SetCoderId(v string) { + x.CoderId = v +} + +type ReadModifyWriteStateSpec_builder struct { + _ [0]func() // Prevents comparability and use of unkeyed literals for the builder. + + CoderId string +} + +func (b0 ReadModifyWriteStateSpec_builder) Build() *ReadModifyWriteStateSpec { + m0 := &ReadModifyWriteStateSpec{} + b, x := &b0, m0 + _, _ = b, x + x.CoderId = b.CoderId + return m0 +} - ElementCoderId string `protobuf:"bytes,1,opt,name=element_coder_id,json=elementCoderId,proto3" json:"element_coder_id,omitempty"` +type BagStateSpec struct { + state protoimpl.MessageState `protogen:"hybrid.v1"` + ElementCoderId string `protobuf:"bytes,1,opt,name=element_coder_id,json=elementCoderId,proto3" json:"element_coder_id,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache } func (x *BagStateSpec) Reset() { *x = BagStateSpec{} - if protoimpl.UnsafeEnabled { - mi := &file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_msgTypes[11] - ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) - ms.StoreMessageInfo(mi) - } + mi := &file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_msgTypes[11] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) } func (x *BagStateSpec) String() string { @@ -2928,7 +3476,7 @@ func (*BagStateSpec) ProtoMessage() {} func (x *BagStateSpec) ProtoReflect() protoreflect.Message { mi := &file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_msgTypes[11] - if protoimpl.UnsafeEnabled && x != nil { + if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { ms.StoreMessageInfo(mi) @@ -2938,11 +3486,6 @@ func (x *BagStateSpec) ProtoReflect() protoreflect.Message { return mi.MessageOf(x) } -// Deprecated: Use BagStateSpec.ProtoReflect.Descriptor instead. -func (*BagStateSpec) Descriptor() ([]byte, []int) { - return file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_rawDescGZIP(), []int{11} -} - func (x *BagStateSpec) GetElementCoderId() string { if x != nil { return x.ElementCoderId @@ -2950,21 +3493,36 @@ func (x *BagStateSpec) GetElementCoderId() string { return "" } -type OrderedListStateSpec struct { - state protoimpl.MessageState - sizeCache protoimpl.SizeCache - unknownFields protoimpl.UnknownFields +func (x *BagStateSpec) SetElementCoderId(v string) { + x.ElementCoderId = v +} + +type BagStateSpec_builder struct { + _ [0]func() // Prevents comparability and use of unkeyed literals for the builder. + + ElementCoderId string +} + +func (b0 BagStateSpec_builder) Build() *BagStateSpec { + m0 := &BagStateSpec{} + b, x := &b0, m0 + _, _ = b, x + x.ElementCoderId = b.ElementCoderId + return m0 +} - ElementCoderId string `protobuf:"bytes,1,opt,name=element_coder_id,json=elementCoderId,proto3" json:"element_coder_id,omitempty"` +type OrderedListStateSpec struct { + state protoimpl.MessageState `protogen:"hybrid.v1"` + ElementCoderId string `protobuf:"bytes,1,opt,name=element_coder_id,json=elementCoderId,proto3" json:"element_coder_id,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache } func (x *OrderedListStateSpec) Reset() { *x = OrderedListStateSpec{} - if protoimpl.UnsafeEnabled { - mi := &file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_msgTypes[12] - ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) - ms.StoreMessageInfo(mi) - } + mi := &file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_msgTypes[12] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) } func (x *OrderedListStateSpec) String() string { @@ -2975,7 +3533,7 @@ func (*OrderedListStateSpec) ProtoMessage() {} func (x *OrderedListStateSpec) ProtoReflect() protoreflect.Message { mi := &file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_msgTypes[12] - if protoimpl.UnsafeEnabled && x != nil { + if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { ms.StoreMessageInfo(mi) @@ -2985,11 +3543,6 @@ func (x *OrderedListStateSpec) ProtoReflect() protoreflect.Message { return mi.MessageOf(x) } -// Deprecated: Use OrderedListStateSpec.ProtoReflect.Descriptor instead. -func (*OrderedListStateSpec) Descriptor() ([]byte, []int) { - return file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_rawDescGZIP(), []int{12} -} - func (x *OrderedListStateSpec) GetElementCoderId() string { if x != nil { return x.ElementCoderId @@ -2997,22 +3550,37 @@ func (x *OrderedListStateSpec) GetElementCoderId() string { return "" } -type CombiningStateSpec struct { - state protoimpl.MessageState - sizeCache protoimpl.SizeCache - unknownFields protoimpl.UnknownFields +func (x *OrderedListStateSpec) SetElementCoderId(v string) { + x.ElementCoderId = v +} + +type OrderedListStateSpec_builder struct { + _ [0]func() // Prevents comparability and use of unkeyed literals for the builder. + + ElementCoderId string +} + +func (b0 OrderedListStateSpec_builder) Build() *OrderedListStateSpec { + m0 := &OrderedListStateSpec{} + b, x := &b0, m0 + _, _ = b, x + x.ElementCoderId = b.ElementCoderId + return m0 +} - AccumulatorCoderId string `protobuf:"bytes,1,opt,name=accumulator_coder_id,json=accumulatorCoderId,proto3" json:"accumulator_coder_id,omitempty"` - CombineFn *FunctionSpec `protobuf:"bytes,2,opt,name=combine_fn,json=combineFn,proto3" json:"combine_fn,omitempty"` +type CombiningStateSpec struct { + state protoimpl.MessageState `protogen:"hybrid.v1"` + AccumulatorCoderId string `protobuf:"bytes,1,opt,name=accumulator_coder_id,json=accumulatorCoderId,proto3" json:"accumulator_coder_id,omitempty"` + CombineFn *FunctionSpec `protobuf:"bytes,2,opt,name=combine_fn,json=combineFn,proto3" json:"combine_fn,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache } func (x *CombiningStateSpec) Reset() { *x = CombiningStateSpec{} - if protoimpl.UnsafeEnabled { - mi := &file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_msgTypes[13] - ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) - ms.StoreMessageInfo(mi) - } + mi := &file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_msgTypes[13] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) } func (x *CombiningStateSpec) String() string { @@ -3023,7 +3591,7 @@ func (*CombiningStateSpec) ProtoMessage() {} func (x *CombiningStateSpec) ProtoReflect() protoreflect.Message { mi := &file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_msgTypes[13] - if protoimpl.UnsafeEnabled && x != nil { + if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { ms.StoreMessageInfo(mi) @@ -3033,11 +3601,6 @@ func (x *CombiningStateSpec) ProtoReflect() protoreflect.Message { return mi.MessageOf(x) } -// Deprecated: Use CombiningStateSpec.ProtoReflect.Descriptor instead. -func (*CombiningStateSpec) Descriptor() ([]byte, []int) { - return file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_rawDescGZIP(), []int{13} -} - func (x *CombiningStateSpec) GetAccumulatorCoderId() string { if x != nil { return x.AccumulatorCoderId @@ -3052,22 +3615,54 @@ func (x *CombiningStateSpec) GetCombineFn() *FunctionSpec { return nil } +func (x *CombiningStateSpec) SetAccumulatorCoderId(v string) { + x.AccumulatorCoderId = v +} + +func (x *CombiningStateSpec) SetCombineFn(v *FunctionSpec) { + x.CombineFn = v +} + +func (x *CombiningStateSpec) HasCombineFn() bool { + if x == nil { + return false + } + return x.CombineFn != nil +} + +func (x *CombiningStateSpec) ClearCombineFn() { + x.CombineFn = nil +} + +type CombiningStateSpec_builder struct { + _ [0]func() // Prevents comparability and use of unkeyed literals for the builder. + + AccumulatorCoderId string + CombineFn *FunctionSpec +} + +func (b0 CombiningStateSpec_builder) Build() *CombiningStateSpec { + m0 := &CombiningStateSpec{} + b, x := &b0, m0 + _, _ = b, x + x.AccumulatorCoderId = b.AccumulatorCoderId + x.CombineFn = b.CombineFn + return m0 +} + type MapStateSpec struct { - state protoimpl.MessageState - sizeCache protoimpl.SizeCache + state protoimpl.MessageState `protogen:"hybrid.v1"` + KeyCoderId string `protobuf:"bytes,1,opt,name=key_coder_id,json=keyCoderId,proto3" json:"key_coder_id,omitempty"` + ValueCoderId string `protobuf:"bytes,2,opt,name=value_coder_id,json=valueCoderId,proto3" json:"value_coder_id,omitempty"` unknownFields protoimpl.UnknownFields - - KeyCoderId string `protobuf:"bytes,1,opt,name=key_coder_id,json=keyCoderId,proto3" json:"key_coder_id,omitempty"` - ValueCoderId string `protobuf:"bytes,2,opt,name=value_coder_id,json=valueCoderId,proto3" json:"value_coder_id,omitempty"` + sizeCache protoimpl.SizeCache } func (x *MapStateSpec) Reset() { *x = MapStateSpec{} - if protoimpl.UnsafeEnabled { - mi := &file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_msgTypes[14] - ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) - ms.StoreMessageInfo(mi) - } + mi := &file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_msgTypes[14] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) } func (x *MapStateSpec) String() string { @@ -3078,7 +3673,7 @@ func (*MapStateSpec) ProtoMessage() {} func (x *MapStateSpec) ProtoReflect() protoreflect.Message { mi := &file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_msgTypes[14] - if protoimpl.UnsafeEnabled && x != nil { + if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { ms.StoreMessageInfo(mi) @@ -3088,11 +3683,6 @@ func (x *MapStateSpec) ProtoReflect() protoreflect.Message { return mi.MessageOf(x) } -// Deprecated: Use MapStateSpec.ProtoReflect.Descriptor instead. -func (*MapStateSpec) Descriptor() ([]byte, []int) { - return file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_rawDescGZIP(), []int{14} -} - func (x *MapStateSpec) GetKeyCoderId() string { if x != nil { return x.KeyCoderId @@ -3107,22 +3697,43 @@ func (x *MapStateSpec) GetValueCoderId() string { return "" } +func (x *MapStateSpec) SetKeyCoderId(v string) { + x.KeyCoderId = v +} + +func (x *MapStateSpec) SetValueCoderId(v string) { + x.ValueCoderId = v +} + +type MapStateSpec_builder struct { + _ [0]func() // Prevents comparability and use of unkeyed literals for the builder. + + KeyCoderId string + ValueCoderId string +} + +func (b0 MapStateSpec_builder) Build() *MapStateSpec { + m0 := &MapStateSpec{} + b, x := &b0, m0 + _, _ = b, x + x.KeyCoderId = b.KeyCoderId + x.ValueCoderId = b.ValueCoderId + return m0 +} + type MultimapStateSpec struct { - state protoimpl.MessageState - sizeCache protoimpl.SizeCache + state protoimpl.MessageState `protogen:"hybrid.v1"` + KeyCoderId string `protobuf:"bytes,1,opt,name=key_coder_id,json=keyCoderId,proto3" json:"key_coder_id,omitempty"` + ValueCoderId string `protobuf:"bytes,2,opt,name=value_coder_id,json=valueCoderId,proto3" json:"value_coder_id,omitempty"` unknownFields protoimpl.UnknownFields - - KeyCoderId string `protobuf:"bytes,1,opt,name=key_coder_id,json=keyCoderId,proto3" json:"key_coder_id,omitempty"` - ValueCoderId string `protobuf:"bytes,2,opt,name=value_coder_id,json=valueCoderId,proto3" json:"value_coder_id,omitempty"` + sizeCache protoimpl.SizeCache } func (x *MultimapStateSpec) Reset() { *x = MultimapStateSpec{} - if protoimpl.UnsafeEnabled { - mi := &file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_msgTypes[15] - ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) - ms.StoreMessageInfo(mi) - } + mi := &file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_msgTypes[15] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) } func (x *MultimapStateSpec) String() string { @@ -3133,7 +3744,7 @@ func (*MultimapStateSpec) ProtoMessage() {} func (x *MultimapStateSpec) ProtoReflect() protoreflect.Message { mi := &file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_msgTypes[15] - if protoimpl.UnsafeEnabled && x != nil { + if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { ms.StoreMessageInfo(mi) @@ -3143,11 +3754,6 @@ func (x *MultimapStateSpec) ProtoReflect() protoreflect.Message { return mi.MessageOf(x) } -// Deprecated: Use MultimapStateSpec.ProtoReflect.Descriptor instead. -func (*MultimapStateSpec) Descriptor() ([]byte, []int) { - return file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_rawDescGZIP(), []int{15} -} - func (x *MultimapStateSpec) GetKeyCoderId() string { if x != nil { return x.KeyCoderId @@ -3162,21 +3768,42 @@ func (x *MultimapStateSpec) GetValueCoderId() string { return "" } -type SetStateSpec struct { - state protoimpl.MessageState - sizeCache protoimpl.SizeCache - unknownFields protoimpl.UnknownFields +func (x *MultimapStateSpec) SetKeyCoderId(v string) { + x.KeyCoderId = v +} + +func (x *MultimapStateSpec) SetValueCoderId(v string) { + x.ValueCoderId = v +} + +type MultimapStateSpec_builder struct { + _ [0]func() // Prevents comparability and use of unkeyed literals for the builder. + + KeyCoderId string + ValueCoderId string +} + +func (b0 MultimapStateSpec_builder) Build() *MultimapStateSpec { + m0 := &MultimapStateSpec{} + b, x := &b0, m0 + _, _ = b, x + x.KeyCoderId = b.KeyCoderId + x.ValueCoderId = b.ValueCoderId + return m0 +} - ElementCoderId string `protobuf:"bytes,1,opt,name=element_coder_id,json=elementCoderId,proto3" json:"element_coder_id,omitempty"` +type SetStateSpec struct { + state protoimpl.MessageState `protogen:"hybrid.v1"` + ElementCoderId string `protobuf:"bytes,1,opt,name=element_coder_id,json=elementCoderId,proto3" json:"element_coder_id,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache } func (x *SetStateSpec) Reset() { *x = SetStateSpec{} - if protoimpl.UnsafeEnabled { - mi := &file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_msgTypes[16] - ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) - ms.StoreMessageInfo(mi) - } + mi := &file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_msgTypes[16] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) } func (x *SetStateSpec) String() string { @@ -3187,7 +3814,7 @@ func (*SetStateSpec) ProtoMessage() {} func (x *SetStateSpec) ProtoReflect() protoreflect.Message { mi := &file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_msgTypes[16] - if protoimpl.UnsafeEnabled && x != nil { + if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { ms.StoreMessageInfo(mi) @@ -3197,11 +3824,6 @@ func (x *SetStateSpec) ProtoReflect() protoreflect.Message { return mi.MessageOf(x) } -// Deprecated: Use SetStateSpec.ProtoReflect.Descriptor instead. -func (*SetStateSpec) Descriptor() ([]byte, []int) { - return file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_rawDescGZIP(), []int{16} -} - func (x *SetStateSpec) GetElementCoderId() string { if x != nil { return x.ElementCoderId @@ -3209,22 +3831,37 @@ func (x *SetStateSpec) GetElementCoderId() string { return "" } -type TimerFamilySpec struct { - state protoimpl.MessageState - sizeCache protoimpl.SizeCache - unknownFields protoimpl.UnknownFields +func (x *SetStateSpec) SetElementCoderId(v string) { + x.ElementCoderId = v +} + +type SetStateSpec_builder struct { + _ [0]func() // Prevents comparability and use of unkeyed literals for the builder. + + ElementCoderId string +} + +func (b0 SetStateSpec_builder) Build() *SetStateSpec { + m0 := &SetStateSpec{} + b, x := &b0, m0 + _, _ = b, x + x.ElementCoderId = b.ElementCoderId + return m0 +} - TimeDomain TimeDomain_Enum `protobuf:"varint,1,opt,name=time_domain,json=timeDomain,proto3,enum=org.apache.beam.model.pipeline.v1.TimeDomain_Enum" json:"time_domain,omitempty"` - TimerFamilyCoderId string `protobuf:"bytes,2,opt,name=timer_family_coder_id,json=timerFamilyCoderId,proto3" json:"timer_family_coder_id,omitempty"` +type TimerFamilySpec struct { + state protoimpl.MessageState `protogen:"hybrid.v1"` + TimeDomain TimeDomain_Enum `protobuf:"varint,1,opt,name=time_domain,json=timeDomain,proto3,enum=org.apache.beam.model.pipeline.v1.TimeDomain_Enum" json:"time_domain,omitempty"` + TimerFamilyCoderId string `protobuf:"bytes,2,opt,name=timer_family_coder_id,json=timerFamilyCoderId,proto3" json:"timer_family_coder_id,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache } func (x *TimerFamilySpec) Reset() { *x = TimerFamilySpec{} - if protoimpl.UnsafeEnabled { - mi := &file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_msgTypes[17] - ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) - ms.StoreMessageInfo(mi) - } + mi := &file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_msgTypes[17] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) } func (x *TimerFamilySpec) String() string { @@ -3235,7 +3872,7 @@ func (*TimerFamilySpec) ProtoMessage() {} func (x *TimerFamilySpec) ProtoReflect() protoreflect.Message { mi := &file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_msgTypes[17] - if protoimpl.UnsafeEnabled && x != nil { + if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { ms.StoreMessageInfo(mi) @@ -3245,11 +3882,6 @@ func (x *TimerFamilySpec) ProtoReflect() protoreflect.Message { return mi.MessageOf(x) } -// Deprecated: Use TimerFamilySpec.ProtoReflect.Descriptor instead. -func (*TimerFamilySpec) Descriptor() ([]byte, []int) { - return file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_rawDescGZIP(), []int{17} -} - func (x *TimerFamilySpec) GetTimeDomain() TimeDomain_Enum { if x != nil { return x.TimeDomain @@ -3264,19 +3896,41 @@ func (x *TimerFamilySpec) GetTimerFamilyCoderId() string { return "" } +func (x *TimerFamilySpec) SetTimeDomain(v TimeDomain_Enum) { + x.TimeDomain = v +} + +func (x *TimerFamilySpec) SetTimerFamilyCoderId(v string) { + x.TimerFamilyCoderId = v +} + +type TimerFamilySpec_builder struct { + _ [0]func() // Prevents comparability and use of unkeyed literals for the builder. + + TimeDomain TimeDomain_Enum + TimerFamilyCoderId string +} + +func (b0 TimerFamilySpec_builder) Build() *TimerFamilySpec { + m0 := &TimerFamilySpec{} + b, x := &b0, m0 + _, _ = b, x + x.TimeDomain = b.TimeDomain + x.TimerFamilyCoderId = b.TimerFamilyCoderId + return m0 +} + type IsBounded struct { - state protoimpl.MessageState - sizeCache protoimpl.SizeCache + state protoimpl.MessageState `protogen:"hybrid.v1"` unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache } func (x *IsBounded) Reset() { *x = IsBounded{} - if protoimpl.UnsafeEnabled { - mi := &file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_msgTypes[18] - ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) - ms.StoreMessageInfo(mi) - } + mi := &file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_msgTypes[18] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) } func (x *IsBounded) String() string { @@ -3287,7 +3941,7 @@ func (*IsBounded) ProtoMessage() {} func (x *IsBounded) ProtoReflect() protoreflect.Message { mi := &file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_msgTypes[18] - if protoimpl.UnsafeEnabled && x != nil { + if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { ms.StoreMessageInfo(mi) @@ -3297,30 +3951,34 @@ func (x *IsBounded) ProtoReflect() protoreflect.Message { return mi.MessageOf(x) } -// Deprecated: Use IsBounded.ProtoReflect.Descriptor instead. -func (*IsBounded) Descriptor() ([]byte, []int) { - return file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_rawDescGZIP(), []int{18} +type IsBounded_builder struct { + _ [0]func() // Prevents comparability and use of unkeyed literals for the builder. + +} + +func (b0 IsBounded_builder) Build() *IsBounded { + m0 := &IsBounded{} + b, x := &b0, m0 + _, _ = b, x + return m0 } // The payload for the primitive Read transform. type ReadPayload struct { - state protoimpl.MessageState - sizeCache protoimpl.SizeCache - unknownFields protoimpl.UnknownFields - + state protoimpl.MessageState `protogen:"hybrid.v1"` // (Required) The FunctionSpec of the source for this Read. Source *FunctionSpec `protobuf:"bytes,1,opt,name=source,proto3" json:"source,omitempty"` // (Required) Whether the source is bounded or unbounded - IsBounded IsBounded_Enum `protobuf:"varint,2,opt,name=is_bounded,json=isBounded,proto3,enum=org.apache.beam.model.pipeline.v1.IsBounded_Enum" json:"is_bounded,omitempty"` + IsBounded IsBounded_Enum `protobuf:"varint,2,opt,name=is_bounded,json=isBounded,proto3,enum=org.apache.beam.model.pipeline.v1.IsBounded_Enum" json:"is_bounded,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache } func (x *ReadPayload) Reset() { *x = ReadPayload{} - if protoimpl.UnsafeEnabled { - mi := &file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_msgTypes[19] - ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) - ms.StoreMessageInfo(mi) - } + mi := &file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_msgTypes[19] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) } func (x *ReadPayload) String() string { @@ -3331,7 +3989,7 @@ func (*ReadPayload) ProtoMessage() {} func (x *ReadPayload) ProtoReflect() protoreflect.Message { mi := &file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_msgTypes[19] - if protoimpl.UnsafeEnabled && x != nil { + if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { ms.StoreMessageInfo(mi) @@ -3341,11 +3999,6 @@ func (x *ReadPayload) ProtoReflect() protoreflect.Message { return mi.MessageOf(x) } -// Deprecated: Use ReadPayload.ProtoReflect.Descriptor instead. -func (*ReadPayload) Descriptor() ([]byte, []int) { - return file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_rawDescGZIP(), []int{19} -} - func (x *ReadPayload) GetSource() *FunctionSpec { if x != nil { return x.Source @@ -3360,23 +4013,57 @@ func (x *ReadPayload) GetIsBounded() IsBounded_Enum { return IsBounded_UNSPECIFIED } +func (x *ReadPayload) SetSource(v *FunctionSpec) { + x.Source = v +} + +func (x *ReadPayload) SetIsBounded(v IsBounded_Enum) { + x.IsBounded = v +} + +func (x *ReadPayload) HasSource() bool { + if x == nil { + return false + } + return x.Source != nil +} + +func (x *ReadPayload) ClearSource() { + x.Source = nil +} + +type ReadPayload_builder struct { + _ [0]func() // Prevents comparability and use of unkeyed literals for the builder. + + // (Required) The FunctionSpec of the source for this Read. + Source *FunctionSpec + // (Required) Whether the source is bounded or unbounded + IsBounded IsBounded_Enum +} + +func (b0 ReadPayload_builder) Build() *ReadPayload { + m0 := &ReadPayload{} + b, x := &b0, m0 + _, _ = b, x + x.Source = b.Source + x.IsBounded = b.IsBounded + return m0 +} + // The payload for the WindowInto transform. type WindowIntoPayload struct { - state protoimpl.MessageState - sizeCache protoimpl.SizeCache - unknownFields protoimpl.UnknownFields - + state protoimpl.MessageState `protogen:"hybrid.v1"` // (Required) The FunctionSpec of the WindowFn. - WindowFn *FunctionSpec `protobuf:"bytes,1,opt,name=window_fn,json=windowFn,proto3" json:"window_fn,omitempty"` + WindowFn *FunctionSpec `protobuf:"bytes,1,opt,name=window_fn,json=windowFn,proto3" json:"window_fn,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache } func (x *WindowIntoPayload) Reset() { *x = WindowIntoPayload{} - if protoimpl.UnsafeEnabled { - mi := &file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_msgTypes[20] - ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) - ms.StoreMessageInfo(mi) - } + mi := &file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_msgTypes[20] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) } func (x *WindowIntoPayload) String() string { @@ -3387,7 +4074,7 @@ func (*WindowIntoPayload) ProtoMessage() {} func (x *WindowIntoPayload) ProtoReflect() protoreflect.Message { mi := &file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_msgTypes[20] - if protoimpl.UnsafeEnabled && x != nil { + if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { ms.StoreMessageInfo(mi) @@ -3397,11 +4084,6 @@ func (x *WindowIntoPayload) ProtoReflect() protoreflect.Message { return mi.MessageOf(x) } -// Deprecated: Use WindowIntoPayload.ProtoReflect.Descriptor instead. -func (*WindowIntoPayload) Descriptor() ([]byte, []int) { - return file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_rawDescGZIP(), []int{20} -} - func (x *WindowIntoPayload) GetWindowFn() *FunctionSpec { if x != nil { return x.WindowFn @@ -3409,25 +4091,52 @@ func (x *WindowIntoPayload) GetWindowFn() *FunctionSpec { return nil } +func (x *WindowIntoPayload) SetWindowFn(v *FunctionSpec) { + x.WindowFn = v +} + +func (x *WindowIntoPayload) HasWindowFn() bool { + if x == nil { + return false + } + return x.WindowFn != nil +} + +func (x *WindowIntoPayload) ClearWindowFn() { + x.WindowFn = nil +} + +type WindowIntoPayload_builder struct { + _ [0]func() // Prevents comparability and use of unkeyed literals for the builder. + + // (Required) The FunctionSpec of the WindowFn. + WindowFn *FunctionSpec +} + +func (b0 WindowIntoPayload_builder) Build() *WindowIntoPayload { + m0 := &WindowIntoPayload{} + b, x := &b0, m0 + _, _ = b, x + x.WindowFn = b.WindowFn + return m0 +} + // The payload for the special-but-not-primitive Combine transform. type CombinePayload struct { - state protoimpl.MessageState - sizeCache protoimpl.SizeCache - unknownFields protoimpl.UnknownFields - + state protoimpl.MessageState `protogen:"hybrid.v1"` // (Required) The FunctionSpec of the CombineFn. CombineFn *FunctionSpec `protobuf:"bytes,1,opt,name=combine_fn,json=combineFn,proto3" json:"combine_fn,omitempty"` // (Required) A reference to the Coder to use for accumulators of the CombineFn AccumulatorCoderId string `protobuf:"bytes,2,opt,name=accumulator_coder_id,json=accumulatorCoderId,proto3" json:"accumulator_coder_id,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache } func (x *CombinePayload) Reset() { *x = CombinePayload{} - if protoimpl.UnsafeEnabled { - mi := &file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_msgTypes[21] - ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) - ms.StoreMessageInfo(mi) - } + mi := &file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_msgTypes[21] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) } func (x *CombinePayload) String() string { @@ -3438,7 +4147,7 @@ func (*CombinePayload) ProtoMessage() {} func (x *CombinePayload) ProtoReflect() protoreflect.Message { mi := &file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_msgTypes[21] - if protoimpl.UnsafeEnabled && x != nil { + if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { ms.StoreMessageInfo(mi) @@ -3448,11 +4157,6 @@ func (x *CombinePayload) ProtoReflect() protoreflect.Message { return mi.MessageOf(x) } -// Deprecated: Use CombinePayload.ProtoReflect.Descriptor instead. -func (*CombinePayload) Descriptor() ([]byte, []int) { - return file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_rawDescGZIP(), []int{21} -} - func (x *CombinePayload) GetCombineFn() *FunctionSpec { if x != nil { return x.CombineFn @@ -3467,28 +4171,62 @@ func (x *CombinePayload) GetAccumulatorCoderId() string { return "" } +func (x *CombinePayload) SetCombineFn(v *FunctionSpec) { + x.CombineFn = v +} + +func (x *CombinePayload) SetAccumulatorCoderId(v string) { + x.AccumulatorCoderId = v +} + +func (x *CombinePayload) HasCombineFn() bool { + if x == nil { + return false + } + return x.CombineFn != nil +} + +func (x *CombinePayload) ClearCombineFn() { + x.CombineFn = nil +} + +type CombinePayload_builder struct { + _ [0]func() // Prevents comparability and use of unkeyed literals for the builder. + + // (Required) The FunctionSpec of the CombineFn. + CombineFn *FunctionSpec + // (Required) A reference to the Coder to use for accumulators of the CombineFn + AccumulatorCoderId string +} + +func (b0 CombinePayload_builder) Build() *CombinePayload { + m0 := &CombinePayload{} + b, x := &b0, m0 + _, _ = b, x + x.CombineFn = b.CombineFn + x.AccumulatorCoderId = b.AccumulatorCoderId + return m0 +} + // The payload for the test-only primitive TestStream type TestStreamPayload struct { - state protoimpl.MessageState - sizeCache protoimpl.SizeCache - unknownFields protoimpl.UnknownFields - + state protoimpl.MessageState `protogen:"hybrid.v1"` // (Required) the coder for elements in the TestStream events CoderId string `protobuf:"bytes,1,opt,name=coder_id,json=coderId,proto3" json:"coder_id,omitempty"` // (Optional) If specified, the TestStream will replay these events. Events []*TestStreamPayload_Event `protobuf:"bytes,2,rep,name=events,proto3" json:"events,omitempty"` // (Optional) If specified, points to a TestStreamService to be // used to retrieve events. - Endpoint *ApiServiceDescriptor `protobuf:"bytes,3,opt,name=endpoint,proto3" json:"endpoint,omitempty"` + Endpoint *ApiServiceDescriptor `protobuf:"bytes,3,opt,name=endpoint,proto3" json:"endpoint,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache } func (x *TestStreamPayload) Reset() { *x = TestStreamPayload{} - if protoimpl.UnsafeEnabled { - mi := &file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_msgTypes[22] - ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) - ms.StoreMessageInfo(mi) - } + mi := &file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_msgTypes[22] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) } func (x *TestStreamPayload) String() string { @@ -3499,7 +4237,7 @@ func (*TestStreamPayload) ProtoMessage() {} func (x *TestStreamPayload) ProtoReflect() protoreflect.Message { mi := &file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_msgTypes[22] - if protoimpl.UnsafeEnabled && x != nil { + if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { ms.StoreMessageInfo(mi) @@ -3509,11 +4247,6 @@ func (x *TestStreamPayload) ProtoReflect() protoreflect.Message { return mi.MessageOf(x) } -// Deprecated: Use TestStreamPayload.ProtoReflect.Descriptor instead. -func (*TestStreamPayload) Descriptor() ([]byte, []int) { - return file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_rawDescGZIP(), []int{22} -} - func (x *TestStreamPayload) GetCoderId() string { if x != nil { return x.CoderId @@ -3535,25 +4268,67 @@ func (x *TestStreamPayload) GetEndpoint() *ApiServiceDescriptor { return nil } -type EventsRequest struct { - state protoimpl.MessageState - sizeCache protoimpl.SizeCache - unknownFields protoimpl.UnknownFields +func (x *TestStreamPayload) SetCoderId(v string) { + x.CoderId = v +} + +func (x *TestStreamPayload) SetEvents(v []*TestStreamPayload_Event) { + x.Events = v +} + +func (x *TestStreamPayload) SetEndpoint(v *ApiServiceDescriptor) { + x.Endpoint = v +} + +func (x *TestStreamPayload) HasEndpoint() bool { + if x == nil { + return false + } + return x.Endpoint != nil +} + +func (x *TestStreamPayload) ClearEndpoint() { + x.Endpoint = nil +} + +type TestStreamPayload_builder struct { + _ [0]func() // Prevents comparability and use of unkeyed literals for the builder. + + // (Required) the coder for elements in the TestStream events + CoderId string + // (Optional) If specified, the TestStream will replay these events. + Events []*TestStreamPayload_Event + // (Optional) If specified, points to a TestStreamService to be + // used to retrieve events. + Endpoint *ApiServiceDescriptor +} +func (b0 TestStreamPayload_builder) Build() *TestStreamPayload { + m0 := &TestStreamPayload{} + b, x := &b0, m0 + _, _ = b, x + x.CoderId = b.CoderId + x.Events = b.Events + x.Endpoint = b.Endpoint + return m0 +} + +type EventsRequest struct { + state protoimpl.MessageState `protogen:"hybrid.v1"` // The set of PCollections to read from. These are the PTransform outputs // local names. These are a subset of the TestStream's outputs. This allows // Interactive Beam to cache many PCollections from a pipeline then replay a // subset of them. - OutputIds []string `protobuf:"bytes,1,rep,name=output_ids,json=outputIds,proto3" json:"output_ids,omitempty"` + OutputIds []string `protobuf:"bytes,1,rep,name=output_ids,json=outputIds,proto3" json:"output_ids,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache } func (x *EventsRequest) Reset() { *x = EventsRequest{} - if protoimpl.UnsafeEnabled { - mi := &file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_msgTypes[23] - ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) - ms.StoreMessageInfo(mi) - } + mi := &file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_msgTypes[23] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) } func (x *EventsRequest) String() string { @@ -3564,7 +4339,7 @@ func (*EventsRequest) ProtoMessage() {} func (x *EventsRequest) ProtoReflect() protoreflect.Message { mi := &file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_msgTypes[23] - if protoimpl.UnsafeEnabled && x != nil { + if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { ms.StoreMessageInfo(mi) @@ -3574,11 +4349,6 @@ func (x *EventsRequest) ProtoReflect() protoreflect.Message { return mi.MessageOf(x) } -// Deprecated: Use EventsRequest.ProtoReflect.Descriptor instead. -func (*EventsRequest) Descriptor() ([]byte, []int) { - return file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_rawDescGZIP(), []int{23} -} - func (x *EventsRequest) GetOutputIds() []string { if x != nil { return x.OutputIds @@ -3586,31 +4356,50 @@ func (x *EventsRequest) GetOutputIds() []string { return nil } +func (x *EventsRequest) SetOutputIds(v []string) { + x.OutputIds = v +} + +type EventsRequest_builder struct { + _ [0]func() // Prevents comparability and use of unkeyed literals for the builder. + + // The set of PCollections to read from. These are the PTransform outputs + // local names. These are a subset of the TestStream's outputs. This allows + // Interactive Beam to cache many PCollections from a pipeline then replay a + // subset of them. + OutputIds []string +} + +func (b0 EventsRequest_builder) Build() *EventsRequest { + m0 := &EventsRequest{} + b, x := &b0, m0 + _, _ = b, x + x.OutputIds = b.OutputIds + return m0 +} + // The payload for the special-but-not-primitive WriteFiles transform. type WriteFilesPayload struct { - state protoimpl.MessageState - sizeCache protoimpl.SizeCache - unknownFields protoimpl.UnknownFields - + state protoimpl.MessageState `protogen:"hybrid.v1"` // (Required) The FunctionSpec of the FileBasedSink. Sink *FunctionSpec `protobuf:"bytes,1,opt,name=sink,proto3" json:"sink,omitempty"` // (Required) The format function. FormatFunction *FunctionSpec `protobuf:"bytes,2,opt,name=format_function,json=formatFunction,proto3" json:"format_function,omitempty"` WindowedWrites bool `protobuf:"varint,3,opt,name=windowed_writes,json=windowedWrites,proto3" json:"windowed_writes,omitempty"` RunnerDeterminedSharding bool `protobuf:"varint,4,opt,name=runner_determined_sharding,json=runnerDeterminedSharding,proto3" json:"runner_determined_sharding,omitempty"` - SideInputs map[string]*SideInput `protobuf:"bytes,5,rep,name=side_inputs,json=sideInputs,proto3" json:"side_inputs,omitempty" protobuf_key:"bytes,1,opt,name=key,proto3" protobuf_val:"bytes,2,opt,name=value,proto3"` + SideInputs map[string]*SideInput `protobuf:"bytes,5,rep,name=side_inputs,json=sideInputs,proto3" json:"side_inputs,omitempty" protobuf_key:"bytes,1,opt,name=key" protobuf_val:"bytes,2,opt,name=value"` // This is different from runner based sharding. This is done by the runner backend, where as runner_determined_sharding // is by the runner translator - AutoSharded bool `protobuf:"varint,6,opt,name=auto_sharded,json=autoSharded,proto3" json:"auto_sharded,omitempty"` + AutoSharded bool `protobuf:"varint,6,opt,name=auto_sharded,json=autoSharded,proto3" json:"auto_sharded,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache } func (x *WriteFilesPayload) Reset() { *x = WriteFilesPayload{} - if protoimpl.UnsafeEnabled { - mi := &file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_msgTypes[24] - ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) - ms.StoreMessageInfo(mi) - } + mi := &file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_msgTypes[24] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) } func (x *WriteFilesPayload) String() string { @@ -3621,7 +4410,7 @@ func (*WriteFilesPayload) ProtoMessage() {} func (x *WriteFilesPayload) ProtoReflect() protoreflect.Message { mi := &file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_msgTypes[24] - if protoimpl.UnsafeEnabled && x != nil { + if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { ms.StoreMessageInfo(mi) @@ -3631,11 +4420,6 @@ func (x *WriteFilesPayload) ProtoReflect() protoreflect.Message { return mi.MessageOf(x) } -// Deprecated: Use WriteFilesPayload.ProtoReflect.Descriptor instead. -func (*WriteFilesPayload) Descriptor() ([]byte, []int) { - return file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_rawDescGZIP(), []int{24} -} - func (x *WriteFilesPayload) GetSink() *FunctionSpec { if x != nil { return x.Sink @@ -3678,6 +4462,80 @@ func (x *WriteFilesPayload) GetAutoSharded() bool { return false } +func (x *WriteFilesPayload) SetSink(v *FunctionSpec) { + x.Sink = v +} + +func (x *WriteFilesPayload) SetFormatFunction(v *FunctionSpec) { + x.FormatFunction = v +} + +func (x *WriteFilesPayload) SetWindowedWrites(v bool) { + x.WindowedWrites = v +} + +func (x *WriteFilesPayload) SetRunnerDeterminedSharding(v bool) { + x.RunnerDeterminedSharding = v +} + +func (x *WriteFilesPayload) SetSideInputs(v map[string]*SideInput) { + x.SideInputs = v +} + +func (x *WriteFilesPayload) SetAutoSharded(v bool) { + x.AutoSharded = v +} + +func (x *WriteFilesPayload) HasSink() bool { + if x == nil { + return false + } + return x.Sink != nil +} + +func (x *WriteFilesPayload) HasFormatFunction() bool { + if x == nil { + return false + } + return x.FormatFunction != nil +} + +func (x *WriteFilesPayload) ClearSink() { + x.Sink = nil +} + +func (x *WriteFilesPayload) ClearFormatFunction() { + x.FormatFunction = nil +} + +type WriteFilesPayload_builder struct { + _ [0]func() // Prevents comparability and use of unkeyed literals for the builder. + + // (Required) The FunctionSpec of the FileBasedSink. + Sink *FunctionSpec + // (Required) The format function. + FormatFunction *FunctionSpec + WindowedWrites bool + RunnerDeterminedSharding bool + SideInputs map[string]*SideInput + // This is different from runner based sharding. This is done by the runner backend, where as runner_determined_sharding + // is by the runner translator + AutoSharded bool +} + +func (b0 WriteFilesPayload_builder) Build() *WriteFilesPayload { + m0 := &WriteFilesPayload{} + b, x := &b0, m0 + _, _ = b, x + x.Sink = b.Sink + x.FormatFunction = b.FormatFunction + x.WindowedWrites = b.WindowedWrites + x.RunnerDeterminedSharding = b.RunnerDeterminedSharding + x.SideInputs = b.SideInputs + x.AutoSharded = b.AutoSharded + return m0 +} + // Payload used by Google Cloud Pub/Sub read transform. // This can be used by runners that wish to override Beam Pub/Sub read transform // with a native implementation. @@ -3686,10 +4544,7 @@ func (x *WriteFilesPayload) GetAutoSharded() bool { // The output of PubSubReadPayload should be bytes of serialized PubsubMessage // proto if with_attributes == true. Otherwise, the bytes is the raw payload. type PubSubReadPayload struct { - state protoimpl.MessageState - sizeCache protoimpl.SizeCache - unknownFields protoimpl.UnknownFields - + state protoimpl.MessageState `protogen:"hybrid.v1"` // Topic to read from. Exactly one of topic or subscription should be set. // Topic format is: /topics/project_id/subscription_name Topic string `protobuf:"bytes,1,opt,name=topic,proto3" json:"topic,omitempty"` @@ -3706,15 +4561,15 @@ type PubSubReadPayload struct { TopicRuntimeOverridden string `protobuf:"bytes,6,opt,name=topic_runtime_overridden,json=topicRuntimeOverridden,proto3" json:"topic_runtime_overridden,omitempty"` // If set, the subscription that is expected to be provided during runtime. SubscriptionRuntimeOverridden string `protobuf:"bytes,7,opt,name=subscription_runtime_overridden,json=subscriptionRuntimeOverridden,proto3" json:"subscription_runtime_overridden,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache } func (x *PubSubReadPayload) Reset() { *x = PubSubReadPayload{} - if protoimpl.UnsafeEnabled { - mi := &file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_msgTypes[25] - ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) - ms.StoreMessageInfo(mi) - } + mi := &file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_msgTypes[25] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) } func (x *PubSubReadPayload) String() string { @@ -3725,7 +4580,7 @@ func (*PubSubReadPayload) ProtoMessage() {} func (x *PubSubReadPayload) ProtoReflect() protoreflect.Message { mi := &file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_msgTypes[25] - if protoimpl.UnsafeEnabled && x != nil { + if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { ms.StoreMessageInfo(mi) @@ -3735,11 +4590,6 @@ func (x *PubSubReadPayload) ProtoReflect() protoreflect.Message { return mi.MessageOf(x) } -// Deprecated: Use PubSubReadPayload.ProtoReflect.Descriptor instead. -func (*PubSubReadPayload) Descriptor() ([]byte, []int) { - return file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_rawDescGZIP(), []int{25} -} - func (x *PubSubReadPayload) GetTopic() string { if x != nil { return x.Topic @@ -3789,6 +4639,69 @@ func (x *PubSubReadPayload) GetSubscriptionRuntimeOverridden() string { return "" } +func (x *PubSubReadPayload) SetTopic(v string) { + x.Topic = v +} + +func (x *PubSubReadPayload) SetSubscription(v string) { + x.Subscription = v +} + +func (x *PubSubReadPayload) SetTimestampAttribute(v string) { + x.TimestampAttribute = v +} + +func (x *PubSubReadPayload) SetIdAttribute(v string) { + x.IdAttribute = v +} + +func (x *PubSubReadPayload) SetWithAttributes(v bool) { + x.WithAttributes = v +} + +func (x *PubSubReadPayload) SetTopicRuntimeOverridden(v string) { + x.TopicRuntimeOverridden = v +} + +func (x *PubSubReadPayload) SetSubscriptionRuntimeOverridden(v string) { + x.SubscriptionRuntimeOverridden = v +} + +type PubSubReadPayload_builder struct { + _ [0]func() // Prevents comparability and use of unkeyed literals for the builder. + + // Topic to read from. Exactly one of topic or subscription should be set. + // Topic format is: /topics/project_id/subscription_name + Topic string + // Subscription to read from. Exactly one of topic or subscription should be set. + // Subscription format is: /subscriptions/project_id/subscription_name + Subscription string + // Attribute that provides element timestamps. + TimestampAttribute string + // Attribute to be used for uniquely identifying messages. + IdAttribute string + // If true, reads Pub/Sub payload as well as attributes. If false, reads only the payload. + WithAttributes bool + // If set, the topic is expected to be provided during runtime. + TopicRuntimeOverridden string + // If set, the subscription that is expected to be provided during runtime. + SubscriptionRuntimeOverridden string +} + +func (b0 PubSubReadPayload_builder) Build() *PubSubReadPayload { + m0 := &PubSubReadPayload{} + b, x := &b0, m0 + _, _ = b, x + x.Topic = b.Topic + x.Subscription = b.Subscription + x.TimestampAttribute = b.TimestampAttribute + x.IdAttribute = b.IdAttribute + x.WithAttributes = b.WithAttributes + x.TopicRuntimeOverridden = b.TopicRuntimeOverridden + x.SubscriptionRuntimeOverridden = b.SubscriptionRuntimeOverridden + return m0 +} + // Payload used by Google Cloud Pub/Sub write transform. // This can be used by runners that wish to override Beam Pub/Sub write transform // with a native implementation. @@ -3797,10 +4710,7 @@ func (x *PubSubReadPayload) GetSubscriptionRuntimeOverridden() string { // The output of PubSubWritePayload should be bytes if serialized PubsubMessage // proto. type PubSubWritePayload struct { - state protoimpl.MessageState - sizeCache protoimpl.SizeCache - unknownFields protoimpl.UnknownFields - + state protoimpl.MessageState `protogen:"hybrid.v1"` // Topic to write to. // Topic format is: /topics/project_id/subscription_name Topic string `protobuf:"bytes,1,opt,name=topic,proto3" json:"topic,omitempty"` @@ -3810,15 +4720,15 @@ type PubSubWritePayload struct { IdAttribute string `protobuf:"bytes,3,opt,name=id_attribute,json=idAttribute,proto3" json:"id_attribute,omitempty"` // If set, the topic is expected to be provided during runtime. TopicRuntimeOverridden string `protobuf:"bytes,4,opt,name=topic_runtime_overridden,json=topicRuntimeOverridden,proto3" json:"topic_runtime_overridden,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache } func (x *PubSubWritePayload) Reset() { *x = PubSubWritePayload{} - if protoimpl.UnsafeEnabled { - mi := &file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_msgTypes[26] - ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) - ms.StoreMessageInfo(mi) - } + mi := &file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_msgTypes[26] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) } func (x *PubSubWritePayload) String() string { @@ -3829,7 +4739,7 @@ func (*PubSubWritePayload) ProtoMessage() {} func (x *PubSubWritePayload) ProtoReflect() protoreflect.Message { mi := &file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_msgTypes[26] - if protoimpl.UnsafeEnabled && x != nil { + if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { ms.StoreMessageInfo(mi) @@ -3839,11 +4749,6 @@ func (x *PubSubWritePayload) ProtoReflect() protoreflect.Message { return mi.MessageOf(x) } -// Deprecated: Use PubSubWritePayload.ProtoReflect.Descriptor instead. -func (*PubSubWritePayload) Descriptor() ([]byte, []int) { - return file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_rawDescGZIP(), []int{26} -} - func (x *PubSubWritePayload) GetTopic() string { if x != nil { return x.Topic @@ -3872,27 +4777,65 @@ func (x *PubSubWritePayload) GetTopicRuntimeOverridden() string { return "" } +func (x *PubSubWritePayload) SetTopic(v string) { + x.Topic = v +} + +func (x *PubSubWritePayload) SetTimestampAttribute(v string) { + x.TimestampAttribute = v +} + +func (x *PubSubWritePayload) SetIdAttribute(v string) { + x.IdAttribute = v +} + +func (x *PubSubWritePayload) SetTopicRuntimeOverridden(v string) { + x.TopicRuntimeOverridden = v +} + +type PubSubWritePayload_builder struct { + _ [0]func() // Prevents comparability and use of unkeyed literals for the builder. + + // Topic to write to. + // Topic format is: /topics/project_id/subscription_name + Topic string + // Attribute that provides element timestamps. + TimestampAttribute string + // Attribute that uniquely identify messages. + IdAttribute string + // If set, the topic is expected to be provided during runtime. + TopicRuntimeOverridden string +} + +func (b0 PubSubWritePayload_builder) Build() *PubSubWritePayload { + m0 := &PubSubWritePayload{} + b, x := &b0, m0 + _, _ = b, x + x.Topic = b.Topic + x.TimestampAttribute = b.TimestampAttribute + x.IdAttribute = b.IdAttribute + x.TopicRuntimeOverridden = b.TopicRuntimeOverridden + return m0 +} + // Payload for GroupIntoBatches composite transform. type GroupIntoBatchesPayload struct { - state protoimpl.MessageState - sizeCache protoimpl.SizeCache - unknownFields protoimpl.UnknownFields - + state protoimpl.MessageState `protogen:"hybrid.v1"` // Max size of a batch. BatchSize int64 `protobuf:"varint,1,opt,name=batch_size,json=batchSize,proto3" json:"batch_size,omitempty"` // Max byte size of a batch in element. BatchSizeBytes int64 `protobuf:"varint,3,opt,name=batch_size_bytes,json=batchSizeBytes,proto3" json:"batch_size_bytes,omitempty"` // (Optional) Max duration a batch is allowed to be cached in states. MaxBufferingDurationMillis int64 `protobuf:"varint,2,opt,name=max_buffering_duration_millis,json=maxBufferingDurationMillis,proto3" json:"max_buffering_duration_millis,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache } func (x *GroupIntoBatchesPayload) Reset() { *x = GroupIntoBatchesPayload{} - if protoimpl.UnsafeEnabled { - mi := &file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_msgTypes[27] - ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) - ms.StoreMessageInfo(mi) - } + mi := &file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_msgTypes[27] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) } func (x *GroupIntoBatchesPayload) String() string { @@ -3903,7 +4846,7 @@ func (*GroupIntoBatchesPayload) ProtoMessage() {} func (x *GroupIntoBatchesPayload) ProtoReflect() protoreflect.Message { mi := &file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_msgTypes[27] - if protoimpl.UnsafeEnabled && x != nil { + if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { ms.StoreMessageInfo(mi) @@ -3913,11 +4856,6 @@ func (x *GroupIntoBatchesPayload) ProtoReflect() protoreflect.Message { return mi.MessageOf(x) } -// Deprecated: Use GroupIntoBatchesPayload.ProtoReflect.Descriptor instead. -func (*GroupIntoBatchesPayload) Descriptor() ([]byte, []int) { - return file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_rawDescGZIP(), []int{27} -} - func (x *GroupIntoBatchesPayload) GetBatchSize() int64 { if x != nil { return x.BatchSize @@ -3939,21 +4877,51 @@ func (x *GroupIntoBatchesPayload) GetMaxBufferingDurationMillis() int64 { return 0 } -type RedistributePayload struct { - state protoimpl.MessageState - sizeCache protoimpl.SizeCache - unknownFields protoimpl.UnknownFields +func (x *GroupIntoBatchesPayload) SetBatchSize(v int64) { + x.BatchSize = v +} + +func (x *GroupIntoBatchesPayload) SetBatchSizeBytes(v int64) { + x.BatchSizeBytes = v +} + +func (x *GroupIntoBatchesPayload) SetMaxBufferingDurationMillis(v int64) { + x.MaxBufferingDurationMillis = v +} + +type GroupIntoBatchesPayload_builder struct { + _ [0]func() // Prevents comparability and use of unkeyed literals for the builder. + + // Max size of a batch. + BatchSize int64 + // Max byte size of a batch in element. + BatchSizeBytes int64 + // (Optional) Max duration a batch is allowed to be cached in states. + MaxBufferingDurationMillis int64 +} + +func (b0 GroupIntoBatchesPayload_builder) Build() *GroupIntoBatchesPayload { + m0 := &GroupIntoBatchesPayload{} + b, x := &b0, m0 + _, _ = b, x + x.BatchSize = b.BatchSize + x.BatchSizeBytes = b.BatchSizeBytes + x.MaxBufferingDurationMillis = b.MaxBufferingDurationMillis + return m0 +} - AllowDuplicates bool `protobuf:"varint,1,opt,name=allow_duplicates,json=allowDuplicates,proto3" json:"allow_duplicates,omitempty"` +type RedistributePayload struct { + state protoimpl.MessageState `protogen:"hybrid.v1"` + AllowDuplicates bool `protobuf:"varint,1,opt,name=allow_duplicates,json=allowDuplicates,proto3" json:"allow_duplicates,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache } func (x *RedistributePayload) Reset() { *x = RedistributePayload{} - if protoimpl.UnsafeEnabled { - mi := &file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_msgTypes[28] - ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) - ms.StoreMessageInfo(mi) - } + mi := &file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_msgTypes[28] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) } func (x *RedistributePayload) String() string { @@ -3964,7 +4932,7 @@ func (*RedistributePayload) ProtoMessage() {} func (x *RedistributePayload) ProtoReflect() protoreflect.Message { mi := &file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_msgTypes[28] - if protoimpl.UnsafeEnabled && x != nil { + if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { ms.StoreMessageInfo(mi) @@ -3974,11 +4942,6 @@ func (x *RedistributePayload) ProtoReflect() protoreflect.Message { return mi.MessageOf(x) } -// Deprecated: Use RedistributePayload.ProtoReflect.Descriptor instead. -func (*RedistributePayload) Descriptor() ([]byte, []int) { - return file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_rawDescGZIP(), []int{28} -} - func (x *RedistributePayload) GetAllowDuplicates() bool { if x != nil { return x.AllowDuplicates @@ -3986,13 +4949,28 @@ func (x *RedistributePayload) GetAllowDuplicates() bool { return false } +func (x *RedistributePayload) SetAllowDuplicates(v bool) { + x.AllowDuplicates = v +} + +type RedistributePayload_builder struct { + _ [0]func() // Prevents comparability and use of unkeyed literals for the builder. + + AllowDuplicates bool +} + +func (b0 RedistributePayload_builder) Build() *RedistributePayload { + m0 := &RedistributePayload{} + b, x := &b0, m0 + _, _ = b, x + x.AllowDuplicates = b.AllowDuplicates + return m0 +} + // A coder, the binary format for serialization and deserialization of data in // a pipeline. type Coder struct { - state protoimpl.MessageState - sizeCache protoimpl.SizeCache - unknownFields protoimpl.UnknownFields - + state protoimpl.MessageState `protogen:"hybrid.v1"` // (Required) A specification for the coder, as a URN plus parameters. This // may be a cross-language agreed-upon format, or it may be a "custom coder" // that can only be used by a particular SDK. It does not include component @@ -4003,15 +4981,15 @@ type Coder struct { // this is a list of the components. In order for encodings to be identical, // the FunctionSpec and all components must be identical, recursively. ComponentCoderIds []string `protobuf:"bytes,2,rep,name=component_coder_ids,json=componentCoderIds,proto3" json:"component_coder_ids,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache } func (x *Coder) Reset() { *x = Coder{} - if protoimpl.UnsafeEnabled { - mi := &file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_msgTypes[29] - ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) - ms.StoreMessageInfo(mi) - } + mi := &file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_msgTypes[29] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) } func (x *Coder) String() string { @@ -4022,7 +5000,7 @@ func (*Coder) ProtoMessage() {} func (x *Coder) ProtoReflect() protoreflect.Message { mi := &file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_msgTypes[29] - if protoimpl.UnsafeEnabled && x != nil { + if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { ms.StoreMessageInfo(mi) @@ -4032,11 +5010,6 @@ func (x *Coder) ProtoReflect() protoreflect.Message { return mi.MessageOf(x) } -// Deprecated: Use Coder.ProtoReflect.Descriptor instead. -func (*Coder) Descriptor() ([]byte, []int) { - return file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_rawDescGZIP(), []int{29} -} - func (x *Coder) GetSpec() *FunctionSpec { if x != nil { return x.Spec @@ -4051,19 +5024,60 @@ func (x *Coder) GetComponentCoderIds() []string { return nil } +func (x *Coder) SetSpec(v *FunctionSpec) { + x.Spec = v +} + +func (x *Coder) SetComponentCoderIds(v []string) { + x.ComponentCoderIds = v +} + +func (x *Coder) HasSpec() bool { + if x == nil { + return false + } + return x.Spec != nil +} + +func (x *Coder) ClearSpec() { + x.Spec = nil +} + +type Coder_builder struct { + _ [0]func() // Prevents comparability and use of unkeyed literals for the builder. + + // (Required) A specification for the coder, as a URN plus parameters. This + // may be a cross-language agreed-upon format, or it may be a "custom coder" + // that can only be used by a particular SDK. It does not include component + // coders, as it is beneficial for these to be comprehensible to a runner + // regardless of whether the binary format is agreed-upon. + Spec *FunctionSpec + // (Optional) If this coder is parametric, such as ListCoder(VarIntCoder), + // this is a list of the components. In order for encodings to be identical, + // the FunctionSpec and all components must be identical, recursively. + ComponentCoderIds []string +} + +func (b0 Coder_builder) Build() *Coder { + m0 := &Coder{} + b, x := &b0, m0 + _, _ = b, x + x.Spec = b.Spec + x.ComponentCoderIds = b.ComponentCoderIds + return m0 +} + type StandardCoders struct { - state protoimpl.MessageState - sizeCache protoimpl.SizeCache + state protoimpl.MessageState `protogen:"hybrid.v1"` unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache } func (x *StandardCoders) Reset() { *x = StandardCoders{} - if protoimpl.UnsafeEnabled { - mi := &file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_msgTypes[30] - ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) - ms.StoreMessageInfo(mi) - } + mi := &file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_msgTypes[30] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) } func (x *StandardCoders) String() string { @@ -4074,7 +5088,7 @@ func (*StandardCoders) ProtoMessage() {} func (x *StandardCoders) ProtoReflect() protoreflect.Message { mi := &file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_msgTypes[30] - if protoimpl.UnsafeEnabled && x != nil { + if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { ms.StoreMessageInfo(mi) @@ -4084,9 +5098,16 @@ func (x *StandardCoders) ProtoReflect() protoreflect.Message { return mi.MessageOf(x) } -// Deprecated: Use StandardCoders.ProtoReflect.Descriptor instead. -func (*StandardCoders) Descriptor() ([]byte, []int) { - return file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_rawDescGZIP(), []int{30} +type StandardCoders_builder struct { + _ [0]func() // Prevents comparability and use of unkeyed literals for the builder. + +} + +func (b0 StandardCoders_builder) Build() *StandardCoders { + m0 := &StandardCoders{} + b, x := &b0, m0 + _, _ = b, x + return m0 } // A windowing strategy describes the window function, triggering, allowed @@ -4094,10 +5115,7 @@ func (*StandardCoders) Descriptor() ([]byte, []int) { // // TODO: consider inlining field on PCollection type WindowingStrategy struct { - state protoimpl.MessageState - sizeCache protoimpl.SizeCache - unknownFields protoimpl.UnknownFields - + state protoimpl.MessageState `protogen:"hybrid.v1"` // (Required) The FunctionSpec of the UDF that assigns windows, // merges windows, and shifts timestamps before they are // combined according to the OutputTime. @@ -4139,15 +5157,15 @@ type WindowingStrategy struct { // Runner that executes the pipeline may choose to override this if needed. // If not specified, environment will be decided by the runner. EnvironmentId string `protobuf:"bytes,11,opt,name=environment_id,json=environmentId,proto3" json:"environment_id,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache } func (x *WindowingStrategy) Reset() { *x = WindowingStrategy{} - if protoimpl.UnsafeEnabled { - mi := &file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_msgTypes[31] - ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) - ms.StoreMessageInfo(mi) - } + mi := &file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_msgTypes[31] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) } func (x *WindowingStrategy) String() string { @@ -4158,7 +5176,7 @@ func (*WindowingStrategy) ProtoMessage() {} func (x *WindowingStrategy) ProtoReflect() protoreflect.Message { mi := &file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_msgTypes[31] - if protoimpl.UnsafeEnabled && x != nil { + if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { ms.StoreMessageInfo(mi) @@ -4168,11 +5186,6 @@ func (x *WindowingStrategy) ProtoReflect() protoreflect.Message { return mi.MessageOf(x) } -// Deprecated: Use WindowingStrategy.ProtoReflect.Descriptor instead. -func (*WindowingStrategy) Descriptor() ([]byte, []int) { - return file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_rawDescGZIP(), []int{31} -} - func (x *WindowingStrategy) GetWindowFn() *FunctionSpec { if x != nil { return x.WindowFn @@ -4250,103 +5263,241 @@ func (x *WindowingStrategy) GetEnvironmentId() string { return "" } -// Whether or not a PCollection's WindowFn is non-merging, merging, or -// merging-but-already-merged, in which case a subsequent GroupByKey is almost -// always going to do something the user does not want -type MergeStatus struct { - state protoimpl.MessageState - sizeCache protoimpl.SizeCache - unknownFields protoimpl.UnknownFields +func (x *WindowingStrategy) SetWindowFn(v *FunctionSpec) { + x.WindowFn = v } -func (x *MergeStatus) Reset() { - *x = MergeStatus{} - if protoimpl.UnsafeEnabled { - mi := &file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_msgTypes[32] - ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) - ms.StoreMessageInfo(mi) - } +func (x *WindowingStrategy) SetMergeStatus(v MergeStatus_Enum) { + x.MergeStatus = v } -func (x *MergeStatus) String() string { - return protoimpl.X.MessageStringOf(x) +func (x *WindowingStrategy) SetWindowCoderId(v string) { + x.WindowCoderId = v } -func (*MergeStatus) ProtoMessage() {} - -func (x *MergeStatus) ProtoReflect() protoreflect.Message { - mi := &file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_msgTypes[32] - if protoimpl.UnsafeEnabled && x != nil { - ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) - if ms.LoadMessageInfo() == nil { - ms.StoreMessageInfo(mi) - } - return ms - } - return mi.MessageOf(x) +func (x *WindowingStrategy) SetTrigger(v *Trigger) { + x.Trigger = v } -// Deprecated: Use MergeStatus.ProtoReflect.Descriptor instead. -func (*MergeStatus) Descriptor() ([]byte, []int) { - return file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_rawDescGZIP(), []int{32} +func (x *WindowingStrategy) SetAccumulationMode(v AccumulationMode_Enum) { + x.AccumulationMode = v } -// Whether or not subsequent outputs of aggregations should be entire -// replacement values or just the aggregation of inputs received since -// the prior output. -type AccumulationMode struct { - state protoimpl.MessageState - sizeCache protoimpl.SizeCache - unknownFields protoimpl.UnknownFields +func (x *WindowingStrategy) SetOutputTime(v OutputTime_Enum) { + x.OutputTime = v } -func (x *AccumulationMode) Reset() { - *x = AccumulationMode{} - if protoimpl.UnsafeEnabled { - mi := &file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_msgTypes[33] - ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) - ms.StoreMessageInfo(mi) - } +func (x *WindowingStrategy) SetClosingBehavior(v ClosingBehavior_Enum) { + x.ClosingBehavior = v } -func (x *AccumulationMode) String() string { - return protoimpl.X.MessageStringOf(x) +func (x *WindowingStrategy) SetAllowedLateness(v int64) { + x.AllowedLateness = v } -func (*AccumulationMode) ProtoMessage() {} +func (x *WindowingStrategy) SetOnTimeBehavior(v OnTimeBehavior_Enum) { + x.OnTimeBehavior = v +} -func (x *AccumulationMode) ProtoReflect() protoreflect.Message { - mi := &file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_msgTypes[33] - if protoimpl.UnsafeEnabled && x != nil { - ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) - if ms.LoadMessageInfo() == nil { - ms.StoreMessageInfo(mi) - } - return ms - } - return mi.MessageOf(x) +func (x *WindowingStrategy) SetAssignsToOneWindow(v bool) { + x.AssignsToOneWindow = v } -// Deprecated: Use AccumulationMode.ProtoReflect.Descriptor instead. -func (*AccumulationMode) Descriptor() ([]byte, []int) { - return file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_rawDescGZIP(), []int{33} +func (x *WindowingStrategy) SetEnvironmentId(v string) { + x.EnvironmentId = v +} + +func (x *WindowingStrategy) HasWindowFn() bool { + if x == nil { + return false + } + return x.WindowFn != nil +} + +func (x *WindowingStrategy) HasTrigger() bool { + if x == nil { + return false + } + return x.Trigger != nil +} + +func (x *WindowingStrategy) ClearWindowFn() { + x.WindowFn = nil +} + +func (x *WindowingStrategy) ClearTrigger() { + x.Trigger = nil +} + +type WindowingStrategy_builder struct { + _ [0]func() // Prevents comparability and use of unkeyed literals for the builder. + + // (Required) The FunctionSpec of the UDF that assigns windows, + // merges windows, and shifts timestamps before they are + // combined according to the OutputTime. + WindowFn *FunctionSpec + // (Required) Whether or not the window fn is merging. + // + // This knowledge is required for many optimizations. + MergeStatus MergeStatus_Enum + // (Required) The coder for the windows of this PCollection. + WindowCoderId string + // (Required) The trigger to use when grouping this PCollection. + Trigger *Trigger + // (Required) The accumulation mode indicates whether new panes are a full + // replacement for prior panes or whether they are deltas to be combined + // with other panes (the combine should correspond to whatever the upstream + // grouping transform is). + AccumulationMode AccumulationMode_Enum + // (Required) The OutputTime specifies, for a grouping transform, how to + // compute the aggregate timestamp. The window_fn will first possibly shift + // it later, then the OutputTime takes the max, min, or ignores it and takes + // the end of window. + // + // This is actually only for input to grouping transforms, but since they + // may be introduced in runner-specific ways, it is carried along with the + // windowing strategy. + OutputTime OutputTime_Enum + // (Required) Indicate when output should be omitted upon window expiration. + ClosingBehavior ClosingBehavior_Enum + // (Required) The duration, in milliseconds, beyond the end of a window at + // which the window becomes droppable. + AllowedLateness int64 + // (Required) Indicate whether empty on-time panes should be omitted. + OnTimeBehavior OnTimeBehavior_Enum + // (Required) Whether or not the window fn assigns inputs to exactly one window + // + // This knowledge is required for some optimizations + AssignsToOneWindow bool + // (Optional) Environment where the current window_fn should be applied in. + // Runner that executes the pipeline may choose to override this if needed. + // If not specified, environment will be decided by the runner. + EnvironmentId string +} + +func (b0 WindowingStrategy_builder) Build() *WindowingStrategy { + m0 := &WindowingStrategy{} + b, x := &b0, m0 + _, _ = b, x + x.WindowFn = b.WindowFn + x.MergeStatus = b.MergeStatus + x.WindowCoderId = b.WindowCoderId + x.Trigger = b.Trigger + x.AccumulationMode = b.AccumulationMode + x.OutputTime = b.OutputTime + x.ClosingBehavior = b.ClosingBehavior + x.AllowedLateness = b.AllowedLateness + x.OnTimeBehavior = b.OnTimeBehavior + x.AssignsToOneWindow = b.AssignsToOneWindow + x.EnvironmentId = b.EnvironmentId + return m0 +} + +// Whether or not a PCollection's WindowFn is non-merging, merging, or +// merging-but-already-merged, in which case a subsequent GroupByKey is almost +// always going to do something the user does not want +type MergeStatus struct { + state protoimpl.MessageState `protogen:"hybrid.v1"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *MergeStatus) Reset() { + *x = MergeStatus{} + mi := &file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_msgTypes[32] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *MergeStatus) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*MergeStatus) ProtoMessage() {} + +func (x *MergeStatus) ProtoReflect() protoreflect.Message { + mi := &file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_msgTypes[32] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +type MergeStatus_builder struct { + _ [0]func() // Prevents comparability and use of unkeyed literals for the builder. + +} + +func (b0 MergeStatus_builder) Build() *MergeStatus { + m0 := &MergeStatus{} + b, x := &b0, m0 + _, _ = b, x + return m0 +} + +// Whether or not subsequent outputs of aggregations should be entire +// replacement values or just the aggregation of inputs received since +// the prior output. +type AccumulationMode struct { + state protoimpl.MessageState `protogen:"hybrid.v1"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *AccumulationMode) Reset() { + *x = AccumulationMode{} + mi := &file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_msgTypes[33] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *AccumulationMode) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*AccumulationMode) ProtoMessage() {} + +func (x *AccumulationMode) ProtoReflect() protoreflect.Message { + mi := &file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_msgTypes[33] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +type AccumulationMode_builder struct { + _ [0]func() // Prevents comparability and use of unkeyed literals for the builder. + +} + +func (b0 AccumulationMode_builder) Build() *AccumulationMode { + m0 := &AccumulationMode{} + b, x := &b0, m0 + _, _ = b, x + return m0 } // Controls whether or not an aggregating transform should output data // when a window expires. type ClosingBehavior struct { - state protoimpl.MessageState - sizeCache protoimpl.SizeCache + state protoimpl.MessageState `protogen:"hybrid.v1"` unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache } func (x *ClosingBehavior) Reset() { *x = ClosingBehavior{} - if protoimpl.UnsafeEnabled { - mi := &file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_msgTypes[34] - ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) - ms.StoreMessageInfo(mi) - } + mi := &file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_msgTypes[34] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) } func (x *ClosingBehavior) String() string { @@ -4357,7 +5508,7 @@ func (*ClosingBehavior) ProtoMessage() {} func (x *ClosingBehavior) ProtoReflect() protoreflect.Message { mi := &file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_msgTypes[34] - if protoimpl.UnsafeEnabled && x != nil { + if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { ms.StoreMessageInfo(mi) @@ -4367,26 +5518,31 @@ func (x *ClosingBehavior) ProtoReflect() protoreflect.Message { return mi.MessageOf(x) } -// Deprecated: Use ClosingBehavior.ProtoReflect.Descriptor instead. -func (*ClosingBehavior) Descriptor() ([]byte, []int) { - return file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_rawDescGZIP(), []int{34} +type ClosingBehavior_builder struct { + _ [0]func() // Prevents comparability and use of unkeyed literals for the builder. + +} + +func (b0 ClosingBehavior_builder) Build() *ClosingBehavior { + m0 := &ClosingBehavior{} + b, x := &b0, m0 + _, _ = b, x + return m0 } // Controls whether or not an aggregating transform should output data // when an on-time pane is empty. type OnTimeBehavior struct { - state protoimpl.MessageState - sizeCache protoimpl.SizeCache + state protoimpl.MessageState `protogen:"hybrid.v1"` unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache } func (x *OnTimeBehavior) Reset() { *x = OnTimeBehavior{} - if protoimpl.UnsafeEnabled { - mi := &file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_msgTypes[35] - ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) - ms.StoreMessageInfo(mi) - } + mi := &file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_msgTypes[35] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) } func (x *OnTimeBehavior) String() string { @@ -4397,7 +5553,7 @@ func (*OnTimeBehavior) ProtoMessage() {} func (x *OnTimeBehavior) ProtoReflect() protoreflect.Message { mi := &file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_msgTypes[35] - if protoimpl.UnsafeEnabled && x != nil { + if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { ms.StoreMessageInfo(mi) @@ -4407,26 +5563,31 @@ func (x *OnTimeBehavior) ProtoReflect() protoreflect.Message { return mi.MessageOf(x) } -// Deprecated: Use OnTimeBehavior.ProtoReflect.Descriptor instead. -func (*OnTimeBehavior) Descriptor() ([]byte, []int) { - return file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_rawDescGZIP(), []int{35} +type OnTimeBehavior_builder struct { + _ [0]func() // Prevents comparability and use of unkeyed literals for the builder. + +} + +func (b0 OnTimeBehavior_builder) Build() *OnTimeBehavior { + m0 := &OnTimeBehavior{} + b, x := &b0, m0 + _, _ = b, x + return m0 } // When a number of windowed, timestamped inputs are aggregated, the timestamp // for the resulting output. type OutputTime struct { - state protoimpl.MessageState - sizeCache protoimpl.SizeCache + state protoimpl.MessageState `protogen:"hybrid.v1"` unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache } func (x *OutputTime) Reset() { *x = OutputTime{} - if protoimpl.UnsafeEnabled { - mi := &file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_msgTypes[36] - ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) - ms.StoreMessageInfo(mi) - } + mi := &file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_msgTypes[36] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) } func (x *OutputTime) String() string { @@ -4437,7 +5598,7 @@ func (*OutputTime) ProtoMessage() {} func (x *OutputTime) ProtoReflect() protoreflect.Message { mi := &file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_msgTypes[36] - if protoimpl.UnsafeEnabled && x != nil { + if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { ms.StoreMessageInfo(mi) @@ -4447,25 +5608,30 @@ func (x *OutputTime) ProtoReflect() protoreflect.Message { return mi.MessageOf(x) } -// Deprecated: Use OutputTime.ProtoReflect.Descriptor instead. -func (*OutputTime) Descriptor() ([]byte, []int) { - return file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_rawDescGZIP(), []int{36} +type OutputTime_builder struct { + _ [0]func() // Prevents comparability and use of unkeyed literals for the builder. + +} + +func (b0 OutputTime_builder) Build() *OutputTime { + m0 := &OutputTime{} + b, x := &b0, m0 + _, _ = b, x + return m0 } // The different time domains in the Beam model. type TimeDomain struct { - state protoimpl.MessageState - sizeCache protoimpl.SizeCache + state protoimpl.MessageState `protogen:"hybrid.v1"` unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache } func (x *TimeDomain) Reset() { *x = TimeDomain{} - if protoimpl.UnsafeEnabled { - mi := &file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_msgTypes[37] - ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) - ms.StoreMessageInfo(mi) - } + mi := &file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_msgTypes[37] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) } func (x *TimeDomain) String() string { @@ -4476,7 +5642,7 @@ func (*TimeDomain) ProtoMessage() {} func (x *TimeDomain) ProtoReflect() protoreflect.Message { mi := &file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_msgTypes[37] - if protoimpl.UnsafeEnabled && x != nil { + if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { ms.StoreMessageInfo(mi) @@ -4486,9 +5652,16 @@ func (x *TimeDomain) ProtoReflect() protoreflect.Message { return mi.MessageOf(x) } -// Deprecated: Use TimeDomain.ProtoReflect.Descriptor instead. -func (*TimeDomain) Descriptor() ([]byte, []int) { - return file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_rawDescGZIP(), []int{37} +type TimeDomain_builder struct { + _ [0]func() // Prevents comparability and use of unkeyed literals for the builder. + +} + +func (b0 TimeDomain_builder) Build() *TimeDomain { + m0 := &TimeDomain{} + b, x := &b0, m0 + _, _ = b, x + return m0 } // A small DSL for expressing when to emit new aggregations @@ -4496,13 +5669,10 @@ func (*TimeDomain) Descriptor() ([]byte, []int) { // // A trigger is described in terms of when it is _ready_ to permit output. type Trigger struct { - state protoimpl.MessageState - sizeCache protoimpl.SizeCache - unknownFields protoimpl.UnknownFields - + state protoimpl.MessageState `protogen:"hybrid.v1"` // The full disjoint union of possible triggers. // - // Types that are assignable to Trigger: + // Types that are valid to be assigned to Trigger: // // *Trigger_AfterAll_ // *Trigger_AfterAny_ @@ -4516,16 +5686,16 @@ type Trigger struct { // *Trigger_Never_ // *Trigger_OrFinally_ // *Trigger_Repeat_ - Trigger isTrigger_Trigger `protobuf_oneof:"trigger"` + Trigger isTrigger_Trigger `protobuf_oneof:"trigger"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache } func (x *Trigger) Reset() { *x = Trigger{} - if protoimpl.UnsafeEnabled { - mi := &file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_msgTypes[38] - ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) - ms.StoreMessageInfo(mi) - } + mi := &file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_msgTypes[38] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) } func (x *Trigger) String() string { @@ -4536,7 +5706,7 @@ func (*Trigger) ProtoMessage() {} func (x *Trigger) ProtoReflect() protoreflect.Message { mi := &file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_msgTypes[38] - if protoimpl.UnsafeEnabled && x != nil { + if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { ms.StoreMessageInfo(mi) @@ -4546,165 +5716,581 @@ func (x *Trigger) ProtoReflect() protoreflect.Message { return mi.MessageOf(x) } -// Deprecated: Use Trigger.ProtoReflect.Descriptor instead. -func (*Trigger) Descriptor() ([]byte, []int) { - return file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_rawDescGZIP(), []int{38} -} - -func (m *Trigger) GetTrigger() isTrigger_Trigger { - if m != nil { - return m.Trigger +func (x *Trigger) GetTrigger() isTrigger_Trigger { + if x != nil { + return x.Trigger } return nil } func (x *Trigger) GetAfterAll() *Trigger_AfterAll { - if x, ok := x.GetTrigger().(*Trigger_AfterAll_); ok { - return x.AfterAll + if x != nil { + if x, ok := x.Trigger.(*Trigger_AfterAll_); ok { + return x.AfterAll + } } return nil } func (x *Trigger) GetAfterAny() *Trigger_AfterAny { - if x, ok := x.GetTrigger().(*Trigger_AfterAny_); ok { - return x.AfterAny + if x != nil { + if x, ok := x.Trigger.(*Trigger_AfterAny_); ok { + return x.AfterAny + } } return nil } func (x *Trigger) GetAfterEach() *Trigger_AfterEach { - if x, ok := x.GetTrigger().(*Trigger_AfterEach_); ok { - return x.AfterEach + if x != nil { + if x, ok := x.Trigger.(*Trigger_AfterEach_); ok { + return x.AfterEach + } } return nil } func (x *Trigger) GetAfterEndOfWindow() *Trigger_AfterEndOfWindow { - if x, ok := x.GetTrigger().(*Trigger_AfterEndOfWindow_); ok { - return x.AfterEndOfWindow + if x != nil { + if x, ok := x.Trigger.(*Trigger_AfterEndOfWindow_); ok { + return x.AfterEndOfWindow + } } return nil } func (x *Trigger) GetAfterProcessingTime() *Trigger_AfterProcessingTime { - if x, ok := x.GetTrigger().(*Trigger_AfterProcessingTime_); ok { - return x.AfterProcessingTime + if x != nil { + if x, ok := x.Trigger.(*Trigger_AfterProcessingTime_); ok { + return x.AfterProcessingTime + } } return nil } func (x *Trigger) GetAfterSynchronizedProcessingTime() *Trigger_AfterSynchronizedProcessingTime { - if x, ok := x.GetTrigger().(*Trigger_AfterSynchronizedProcessingTime_); ok { - return x.AfterSynchronizedProcessingTime + if x != nil { + if x, ok := x.Trigger.(*Trigger_AfterSynchronizedProcessingTime_); ok { + return x.AfterSynchronizedProcessingTime + } } return nil } func (x *Trigger) GetAlways() *Trigger_Always { - if x, ok := x.GetTrigger().(*Trigger_Always_); ok { - return x.Always + if x != nil { + if x, ok := x.Trigger.(*Trigger_Always_); ok { + return x.Always + } } return nil } func (x *Trigger) GetDefault() *Trigger_Default { - if x, ok := x.GetTrigger().(*Trigger_Default_); ok { - return x.Default + if x != nil { + if x, ok := x.Trigger.(*Trigger_Default_); ok { + return x.Default + } } return nil } func (x *Trigger) GetElementCount() *Trigger_ElementCount { - if x, ok := x.GetTrigger().(*Trigger_ElementCount_); ok { - return x.ElementCount + if x != nil { + if x, ok := x.Trigger.(*Trigger_ElementCount_); ok { + return x.ElementCount + } } return nil } func (x *Trigger) GetNever() *Trigger_Never { - if x, ok := x.GetTrigger().(*Trigger_Never_); ok { - return x.Never + if x != nil { + if x, ok := x.Trigger.(*Trigger_Never_); ok { + return x.Never + } } return nil } func (x *Trigger) GetOrFinally() *Trigger_OrFinally { - if x, ok := x.GetTrigger().(*Trigger_OrFinally_); ok { - return x.OrFinally + if x != nil { + if x, ok := x.Trigger.(*Trigger_OrFinally_); ok { + return x.OrFinally + } } return nil } func (x *Trigger) GetRepeat() *Trigger_Repeat { - if x, ok := x.GetTrigger().(*Trigger_Repeat_); ok { - return x.Repeat + if x != nil { + if x, ok := x.Trigger.(*Trigger_Repeat_); ok { + return x.Repeat + } } return nil } -type isTrigger_Trigger interface { - isTrigger_Trigger() +func (x *Trigger) SetAfterAll(v *Trigger_AfterAll) { + if v == nil { + x.Trigger = nil + return + } + x.Trigger = &Trigger_AfterAll_{v} } -type Trigger_AfterAll_ struct { - AfterAll *Trigger_AfterAll `protobuf:"bytes,1,opt,name=after_all,json=afterAll,proto3,oneof"` +func (x *Trigger) SetAfterAny(v *Trigger_AfterAny) { + if v == nil { + x.Trigger = nil + return + } + x.Trigger = &Trigger_AfterAny_{v} } -type Trigger_AfterAny_ struct { - AfterAny *Trigger_AfterAny `protobuf:"bytes,2,opt,name=after_any,json=afterAny,proto3,oneof"` +func (x *Trigger) SetAfterEach(v *Trigger_AfterEach) { + if v == nil { + x.Trigger = nil + return + } + x.Trigger = &Trigger_AfterEach_{v} } -type Trigger_AfterEach_ struct { - AfterEach *Trigger_AfterEach `protobuf:"bytes,3,opt,name=after_each,json=afterEach,proto3,oneof"` +func (x *Trigger) SetAfterEndOfWindow(v *Trigger_AfterEndOfWindow) { + if v == nil { + x.Trigger = nil + return + } + x.Trigger = &Trigger_AfterEndOfWindow_{v} } -type Trigger_AfterEndOfWindow_ struct { - AfterEndOfWindow *Trigger_AfterEndOfWindow `protobuf:"bytes,4,opt,name=after_end_of_window,json=afterEndOfWindow,proto3,oneof"` +func (x *Trigger) SetAfterProcessingTime(v *Trigger_AfterProcessingTime) { + if v == nil { + x.Trigger = nil + return + } + x.Trigger = &Trigger_AfterProcessingTime_{v} } -type Trigger_AfterProcessingTime_ struct { - AfterProcessingTime *Trigger_AfterProcessingTime `protobuf:"bytes,5,opt,name=after_processing_time,json=afterProcessingTime,proto3,oneof"` +func (x *Trigger) SetAfterSynchronizedProcessingTime(v *Trigger_AfterSynchronizedProcessingTime) { + if v == nil { + x.Trigger = nil + return + } + x.Trigger = &Trigger_AfterSynchronizedProcessingTime_{v} } -type Trigger_AfterSynchronizedProcessingTime_ struct { - AfterSynchronizedProcessingTime *Trigger_AfterSynchronizedProcessingTime `protobuf:"bytes,6,opt,name=after_synchronized_processing_time,json=afterSynchronizedProcessingTime,proto3,oneof"` +func (x *Trigger) SetAlways(v *Trigger_Always) { + if v == nil { + x.Trigger = nil + return + } + x.Trigger = &Trigger_Always_{v} } -type Trigger_Always_ struct { - Always *Trigger_Always `protobuf:"bytes,12,opt,name=always,proto3,oneof"` +func (x *Trigger) SetDefault(v *Trigger_Default) { + if v == nil { + x.Trigger = nil + return + } + x.Trigger = &Trigger_Default_{v} } -type Trigger_Default_ struct { - Default *Trigger_Default `protobuf:"bytes,7,opt,name=default,proto3,oneof"` +func (x *Trigger) SetElementCount(v *Trigger_ElementCount) { + if v == nil { + x.Trigger = nil + return + } + x.Trigger = &Trigger_ElementCount_{v} } -type Trigger_ElementCount_ struct { - ElementCount *Trigger_ElementCount `protobuf:"bytes,8,opt,name=element_count,json=elementCount,proto3,oneof"` +func (x *Trigger) SetNever(v *Trigger_Never) { + if v == nil { + x.Trigger = nil + return + } + x.Trigger = &Trigger_Never_{v} } -type Trigger_Never_ struct { - Never *Trigger_Never `protobuf:"bytes,9,opt,name=never,proto3,oneof"` +func (x *Trigger) SetOrFinally(v *Trigger_OrFinally) { + if v == nil { + x.Trigger = nil + return + } + x.Trigger = &Trigger_OrFinally_{v} } -type Trigger_OrFinally_ struct { - OrFinally *Trigger_OrFinally `protobuf:"bytes,10,opt,name=or_finally,json=orFinally,proto3,oneof"` +func (x *Trigger) SetRepeat(v *Trigger_Repeat) { + if v == nil { + x.Trigger = nil + return + } + x.Trigger = &Trigger_Repeat_{v} } -type Trigger_Repeat_ struct { - Repeat *Trigger_Repeat `protobuf:"bytes,11,opt,name=repeat,proto3,oneof"` +func (x *Trigger) HasTrigger() bool { + if x == nil { + return false + } + return x.Trigger != nil } -func (*Trigger_AfterAll_) isTrigger_Trigger() {} +func (x *Trigger) HasAfterAll() bool { + if x == nil { + return false + } + _, ok := x.Trigger.(*Trigger_AfterAll_) + return ok +} -func (*Trigger_AfterAny_) isTrigger_Trigger() {} +func (x *Trigger) HasAfterAny() bool { + if x == nil { + return false + } + _, ok := x.Trigger.(*Trigger_AfterAny_) + return ok +} -func (*Trigger_AfterEach_) isTrigger_Trigger() {} +func (x *Trigger) HasAfterEach() bool { + if x == nil { + return false + } + _, ok := x.Trigger.(*Trigger_AfterEach_) + return ok +} -func (*Trigger_AfterEndOfWindow_) isTrigger_Trigger() {} +func (x *Trigger) HasAfterEndOfWindow() bool { + if x == nil { + return false + } + _, ok := x.Trigger.(*Trigger_AfterEndOfWindow_) + return ok +} -func (*Trigger_AfterProcessingTime_) isTrigger_Trigger() {} +func (x *Trigger) HasAfterProcessingTime() bool { + if x == nil { + return false + } + _, ok := x.Trigger.(*Trigger_AfterProcessingTime_) + return ok +} -func (*Trigger_AfterSynchronizedProcessingTime_) isTrigger_Trigger() {} +func (x *Trigger) HasAfterSynchronizedProcessingTime() bool { + if x == nil { + return false + } + _, ok := x.Trigger.(*Trigger_AfterSynchronizedProcessingTime_) + return ok +} + +func (x *Trigger) HasAlways() bool { + if x == nil { + return false + } + _, ok := x.Trigger.(*Trigger_Always_) + return ok +} + +func (x *Trigger) HasDefault() bool { + if x == nil { + return false + } + _, ok := x.Trigger.(*Trigger_Default_) + return ok +} + +func (x *Trigger) HasElementCount() bool { + if x == nil { + return false + } + _, ok := x.Trigger.(*Trigger_ElementCount_) + return ok +} + +func (x *Trigger) HasNever() bool { + if x == nil { + return false + } + _, ok := x.Trigger.(*Trigger_Never_) + return ok +} + +func (x *Trigger) HasOrFinally() bool { + if x == nil { + return false + } + _, ok := x.Trigger.(*Trigger_OrFinally_) + return ok +} + +func (x *Trigger) HasRepeat() bool { + if x == nil { + return false + } + _, ok := x.Trigger.(*Trigger_Repeat_) + return ok +} + +func (x *Trigger) ClearTrigger() { + x.Trigger = nil +} + +func (x *Trigger) ClearAfterAll() { + if _, ok := x.Trigger.(*Trigger_AfterAll_); ok { + x.Trigger = nil + } +} + +func (x *Trigger) ClearAfterAny() { + if _, ok := x.Trigger.(*Trigger_AfterAny_); ok { + x.Trigger = nil + } +} + +func (x *Trigger) ClearAfterEach() { + if _, ok := x.Trigger.(*Trigger_AfterEach_); ok { + x.Trigger = nil + } +} + +func (x *Trigger) ClearAfterEndOfWindow() { + if _, ok := x.Trigger.(*Trigger_AfterEndOfWindow_); ok { + x.Trigger = nil + } +} + +func (x *Trigger) ClearAfterProcessingTime() { + if _, ok := x.Trigger.(*Trigger_AfterProcessingTime_); ok { + x.Trigger = nil + } +} + +func (x *Trigger) ClearAfterSynchronizedProcessingTime() { + if _, ok := x.Trigger.(*Trigger_AfterSynchronizedProcessingTime_); ok { + x.Trigger = nil + } +} + +func (x *Trigger) ClearAlways() { + if _, ok := x.Trigger.(*Trigger_Always_); ok { + x.Trigger = nil + } +} + +func (x *Trigger) ClearDefault() { + if _, ok := x.Trigger.(*Trigger_Default_); ok { + x.Trigger = nil + } +} + +func (x *Trigger) ClearElementCount() { + if _, ok := x.Trigger.(*Trigger_ElementCount_); ok { + x.Trigger = nil + } +} + +func (x *Trigger) ClearNever() { + if _, ok := x.Trigger.(*Trigger_Never_); ok { + x.Trigger = nil + } +} + +func (x *Trigger) ClearOrFinally() { + if _, ok := x.Trigger.(*Trigger_OrFinally_); ok { + x.Trigger = nil + } +} + +func (x *Trigger) ClearRepeat() { + if _, ok := x.Trigger.(*Trigger_Repeat_); ok { + x.Trigger = nil + } +} + +const Trigger_Trigger_not_set_case case_Trigger_Trigger = 0 +const Trigger_AfterAll_case case_Trigger_Trigger = 1 +const Trigger_AfterAny_case case_Trigger_Trigger = 2 +const Trigger_AfterEach_case case_Trigger_Trigger = 3 +const Trigger_AfterEndOfWindow_case case_Trigger_Trigger = 4 +const Trigger_AfterProcessingTime_case case_Trigger_Trigger = 5 +const Trigger_AfterSynchronizedProcessingTime_case case_Trigger_Trigger = 6 +const Trigger_Always_case case_Trigger_Trigger = 12 +const Trigger_Default_case case_Trigger_Trigger = 7 +const Trigger_ElementCount_case case_Trigger_Trigger = 8 +const Trigger_Never_case case_Trigger_Trigger = 9 +const Trigger_OrFinally_case case_Trigger_Trigger = 10 +const Trigger_Repeat_case case_Trigger_Trigger = 11 + +func (x *Trigger) WhichTrigger() case_Trigger_Trigger { + if x == nil { + return Trigger_Trigger_not_set_case + } + switch x.Trigger.(type) { + case *Trigger_AfterAll_: + return Trigger_AfterAll_case + case *Trigger_AfterAny_: + return Trigger_AfterAny_case + case *Trigger_AfterEach_: + return Trigger_AfterEach_case + case *Trigger_AfterEndOfWindow_: + return Trigger_AfterEndOfWindow_case + case *Trigger_AfterProcessingTime_: + return Trigger_AfterProcessingTime_case + case *Trigger_AfterSynchronizedProcessingTime_: + return Trigger_AfterSynchronizedProcessingTime_case + case *Trigger_Always_: + return Trigger_Always_case + case *Trigger_Default_: + return Trigger_Default_case + case *Trigger_ElementCount_: + return Trigger_ElementCount_case + case *Trigger_Never_: + return Trigger_Never_case + case *Trigger_OrFinally_: + return Trigger_OrFinally_case + case *Trigger_Repeat_: + return Trigger_Repeat_case + default: + return Trigger_Trigger_not_set_case + } +} + +type Trigger_builder struct { + _ [0]func() // Prevents comparability and use of unkeyed literals for the builder. + + // The full disjoint union of possible triggers. + + // Fields of oneof Trigger: + AfterAll *Trigger_AfterAll + AfterAny *Trigger_AfterAny + AfterEach *Trigger_AfterEach + AfterEndOfWindow *Trigger_AfterEndOfWindow + AfterProcessingTime *Trigger_AfterProcessingTime + AfterSynchronizedProcessingTime *Trigger_AfterSynchronizedProcessingTime + Always *Trigger_Always + Default *Trigger_Default + ElementCount *Trigger_ElementCount + Never *Trigger_Never + OrFinally *Trigger_OrFinally + Repeat *Trigger_Repeat + // -- end of Trigger +} + +func (b0 Trigger_builder) Build() *Trigger { + m0 := &Trigger{} + b, x := &b0, m0 + _, _ = b, x + if b.AfterAll != nil { + x.Trigger = &Trigger_AfterAll_{b.AfterAll} + } + if b.AfterAny != nil { + x.Trigger = &Trigger_AfterAny_{b.AfterAny} + } + if b.AfterEach != nil { + x.Trigger = &Trigger_AfterEach_{b.AfterEach} + } + if b.AfterEndOfWindow != nil { + x.Trigger = &Trigger_AfterEndOfWindow_{b.AfterEndOfWindow} + } + if b.AfterProcessingTime != nil { + x.Trigger = &Trigger_AfterProcessingTime_{b.AfterProcessingTime} + } + if b.AfterSynchronizedProcessingTime != nil { + x.Trigger = &Trigger_AfterSynchronizedProcessingTime_{b.AfterSynchronizedProcessingTime} + } + if b.Always != nil { + x.Trigger = &Trigger_Always_{b.Always} + } + if b.Default != nil { + x.Trigger = &Trigger_Default_{b.Default} + } + if b.ElementCount != nil { + x.Trigger = &Trigger_ElementCount_{b.ElementCount} + } + if b.Never != nil { + x.Trigger = &Trigger_Never_{b.Never} + } + if b.OrFinally != nil { + x.Trigger = &Trigger_OrFinally_{b.OrFinally} + } + if b.Repeat != nil { + x.Trigger = &Trigger_Repeat_{b.Repeat} + } + return m0 +} + +type case_Trigger_Trigger protoreflect.FieldNumber + +func (x case_Trigger_Trigger) String() string { + md := file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_msgTypes[38].Descriptor() + if x == 0 { + return "not set" + } + return protoimpl.X.MessageFieldStringOf(md, protoreflect.FieldNumber(x)) +} + +type isTrigger_Trigger interface { + isTrigger_Trigger() +} + +type Trigger_AfterAll_ struct { + AfterAll *Trigger_AfterAll `protobuf:"bytes,1,opt,name=after_all,json=afterAll,proto3,oneof"` +} + +type Trigger_AfterAny_ struct { + AfterAny *Trigger_AfterAny `protobuf:"bytes,2,opt,name=after_any,json=afterAny,proto3,oneof"` +} + +type Trigger_AfterEach_ struct { + AfterEach *Trigger_AfterEach `protobuf:"bytes,3,opt,name=after_each,json=afterEach,proto3,oneof"` +} + +type Trigger_AfterEndOfWindow_ struct { + AfterEndOfWindow *Trigger_AfterEndOfWindow `protobuf:"bytes,4,opt,name=after_end_of_window,json=afterEndOfWindow,proto3,oneof"` +} + +type Trigger_AfterProcessingTime_ struct { + AfterProcessingTime *Trigger_AfterProcessingTime `protobuf:"bytes,5,opt,name=after_processing_time,json=afterProcessingTime,proto3,oneof"` +} + +type Trigger_AfterSynchronizedProcessingTime_ struct { + AfterSynchronizedProcessingTime *Trigger_AfterSynchronizedProcessingTime `protobuf:"bytes,6,opt,name=after_synchronized_processing_time,json=afterSynchronizedProcessingTime,proto3,oneof"` +} + +type Trigger_Always_ struct { + Always *Trigger_Always `protobuf:"bytes,12,opt,name=always,proto3,oneof"` +} + +type Trigger_Default_ struct { + Default *Trigger_Default `protobuf:"bytes,7,opt,name=default,proto3,oneof"` +} + +type Trigger_ElementCount_ struct { + ElementCount *Trigger_ElementCount `protobuf:"bytes,8,opt,name=element_count,json=elementCount,proto3,oneof"` +} + +type Trigger_Never_ struct { + Never *Trigger_Never `protobuf:"bytes,9,opt,name=never,proto3,oneof"` +} + +type Trigger_OrFinally_ struct { + OrFinally *Trigger_OrFinally `protobuf:"bytes,10,opt,name=or_finally,json=orFinally,proto3,oneof"` +} + +type Trigger_Repeat_ struct { + Repeat *Trigger_Repeat `protobuf:"bytes,11,opt,name=repeat,proto3,oneof"` +} + +func (*Trigger_AfterAll_) isTrigger_Trigger() {} + +func (*Trigger_AfterAny_) isTrigger_Trigger() {} + +func (*Trigger_AfterEach_) isTrigger_Trigger() {} + +func (*Trigger_AfterEndOfWindow_) isTrigger_Trigger() {} + +func (*Trigger_AfterProcessingTime_) isTrigger_Trigger() {} + +func (*Trigger_AfterSynchronizedProcessingTime_) isTrigger_Trigger() {} func (*Trigger_Always_) isTrigger_Trigger() {} @@ -4723,24 +6309,21 @@ func (*Trigger_Repeat_) isTrigger_Trigger() {} // Primarily used by AfterProcessingTime triggers to transform // the arrival time of input to a target time for firing. type TimestampTransform struct { - state protoimpl.MessageState - sizeCache protoimpl.SizeCache - unknownFields protoimpl.UnknownFields - - // Types that are assignable to TimestampTransform: + state protoimpl.MessageState `protogen:"hybrid.v1"` + // Types that are valid to be assigned to TimestampTransform: // // *TimestampTransform_Delay_ // *TimestampTransform_AlignTo_ TimestampTransform isTimestampTransform_TimestampTransform `protobuf_oneof:"timestamp_transform"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache } func (x *TimestampTransform) Reset() { *x = TimestampTransform{} - if protoimpl.UnsafeEnabled { - mi := &file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_msgTypes[39] - ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) - ms.StoreMessageInfo(mi) - } + mi := &file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_msgTypes[39] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) } func (x *TimestampTransform) String() string { @@ -4751,7 +6334,7 @@ func (*TimestampTransform) ProtoMessage() {} func (x *TimestampTransform) ProtoReflect() protoreflect.Message { mi := &file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_msgTypes[39] - if protoimpl.UnsafeEnabled && x != nil { + if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { ms.StoreMessageInfo(mi) @@ -4761,32 +6344,136 @@ func (x *TimestampTransform) ProtoReflect() protoreflect.Message { return mi.MessageOf(x) } -// Deprecated: Use TimestampTransform.ProtoReflect.Descriptor instead. -func (*TimestampTransform) Descriptor() ([]byte, []int) { - return file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_rawDescGZIP(), []int{39} -} - -func (m *TimestampTransform) GetTimestampTransform() isTimestampTransform_TimestampTransform { - if m != nil { - return m.TimestampTransform +func (x *TimestampTransform) GetTimestampTransform() isTimestampTransform_TimestampTransform { + if x != nil { + return x.TimestampTransform } return nil } func (x *TimestampTransform) GetDelay() *TimestampTransform_Delay { - if x, ok := x.GetTimestampTransform().(*TimestampTransform_Delay_); ok { - return x.Delay + if x != nil { + if x, ok := x.TimestampTransform.(*TimestampTransform_Delay_); ok { + return x.Delay + } } return nil } func (x *TimestampTransform) GetAlignTo() *TimestampTransform_AlignTo { - if x, ok := x.GetTimestampTransform().(*TimestampTransform_AlignTo_); ok { - return x.AlignTo + if x != nil { + if x, ok := x.TimestampTransform.(*TimestampTransform_AlignTo_); ok { + return x.AlignTo + } } return nil } +func (x *TimestampTransform) SetDelay(v *TimestampTransform_Delay) { + if v == nil { + x.TimestampTransform = nil + return + } + x.TimestampTransform = &TimestampTransform_Delay_{v} +} + +func (x *TimestampTransform) SetAlignTo(v *TimestampTransform_AlignTo) { + if v == nil { + x.TimestampTransform = nil + return + } + x.TimestampTransform = &TimestampTransform_AlignTo_{v} +} + +func (x *TimestampTransform) HasTimestampTransform() bool { + if x == nil { + return false + } + return x.TimestampTransform != nil +} + +func (x *TimestampTransform) HasDelay() bool { + if x == nil { + return false + } + _, ok := x.TimestampTransform.(*TimestampTransform_Delay_) + return ok +} + +func (x *TimestampTransform) HasAlignTo() bool { + if x == nil { + return false + } + _, ok := x.TimestampTransform.(*TimestampTransform_AlignTo_) + return ok +} + +func (x *TimestampTransform) ClearTimestampTransform() { + x.TimestampTransform = nil +} + +func (x *TimestampTransform) ClearDelay() { + if _, ok := x.TimestampTransform.(*TimestampTransform_Delay_); ok { + x.TimestampTransform = nil + } +} + +func (x *TimestampTransform) ClearAlignTo() { + if _, ok := x.TimestampTransform.(*TimestampTransform_AlignTo_); ok { + x.TimestampTransform = nil + } +} + +const TimestampTransform_TimestampTransform_not_set_case case_TimestampTransform_TimestampTransform = 0 +const TimestampTransform_Delay_case case_TimestampTransform_TimestampTransform = 1 +const TimestampTransform_AlignTo_case case_TimestampTransform_TimestampTransform = 2 + +func (x *TimestampTransform) WhichTimestampTransform() case_TimestampTransform_TimestampTransform { + if x == nil { + return TimestampTransform_TimestampTransform_not_set_case + } + switch x.TimestampTransform.(type) { + case *TimestampTransform_Delay_: + return TimestampTransform_Delay_case + case *TimestampTransform_AlignTo_: + return TimestampTransform_AlignTo_case + default: + return TimestampTransform_TimestampTransform_not_set_case + } +} + +type TimestampTransform_builder struct { + _ [0]func() // Prevents comparability and use of unkeyed literals for the builder. + + // Fields of oneof TimestampTransform: + Delay *TimestampTransform_Delay + AlignTo *TimestampTransform_AlignTo + // -- end of TimestampTransform +} + +func (b0 TimestampTransform_builder) Build() *TimestampTransform { + m0 := &TimestampTransform{} + b, x := &b0, m0 + _, _ = b, x + if b.Delay != nil { + x.TimestampTransform = &TimestampTransform_Delay_{b.Delay} + } + if b.AlignTo != nil { + x.TimestampTransform = &TimestampTransform_AlignTo_{b.AlignTo} + } + return m0 +} + +type case_TimestampTransform_TimestampTransform protoreflect.FieldNumber + +func (x case_TimestampTransform_TimestampTransform) String() string { + md := file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_msgTypes[39].Descriptor() + if x == 0 { + return "not set" + } + return protoimpl.X.MessageFieldStringOf(md, protoreflect.FieldNumber(x)) +} + type isTimestampTransform_TimestampTransform interface { isTimestampTransform_TimestampTransform() } @@ -4805,10 +6492,7 @@ func (*TimestampTransform_AlignTo_) isTimestampTransform_TimestampTransform() {} // A specification for how to "side input" a PCollection. type SideInput struct { - state protoimpl.MessageState - sizeCache protoimpl.SizeCache - unknownFields protoimpl.UnknownFields - + state protoimpl.MessageState `protogen:"hybrid.v1"` // (Required) URN of the access pattern required by the `view_fn` to present // the desired SDK-specific interface to a UDF. // @@ -4831,15 +6515,15 @@ type SideInput struct { // can specify that the side input should be accessed according to the day // in which that hour falls. WindowMappingFn *FunctionSpec `protobuf:"bytes,3,opt,name=window_mapping_fn,json=windowMappingFn,proto3" json:"window_mapping_fn,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache } func (x *SideInput) Reset() { *x = SideInput{} - if protoimpl.UnsafeEnabled { - mi := &file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_msgTypes[40] - ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) - ms.StoreMessageInfo(mi) - } + mi := &file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_msgTypes[40] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) } func (x *SideInput) String() string { @@ -4850,7 +6534,7 @@ func (*SideInput) ProtoMessage() {} func (x *SideInput) ProtoReflect() protoreflect.Message { mi := &file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_msgTypes[40] - if protoimpl.UnsafeEnabled && x != nil { + if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { ms.StoreMessageInfo(mi) @@ -4860,11 +6544,6 @@ func (x *SideInput) ProtoReflect() protoreflect.Message { return mi.MessageOf(x) } -// Deprecated: Use SideInput.ProtoReflect.Descriptor instead. -func (*SideInput) Descriptor() ([]byte, []int) { - return file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_rawDescGZIP(), []int{40} -} - func (x *SideInput) GetAccessPattern() *FunctionSpec { if x != nil { return x.AccessPattern @@ -4886,85 +6565,164 @@ func (x *SideInput) GetWindowMappingFn() *FunctionSpec { return nil } -type StandardArtifacts struct { - state protoimpl.MessageState - sizeCache protoimpl.SizeCache - unknownFields protoimpl.UnknownFields +func (x *SideInput) SetAccessPattern(v *FunctionSpec) { + x.AccessPattern = v } -func (x *StandardArtifacts) Reset() { - *x = StandardArtifacts{} - if protoimpl.UnsafeEnabled { - mi := &file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_msgTypes[41] - ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) - ms.StoreMessageInfo(mi) - } +func (x *SideInput) SetViewFn(v *FunctionSpec) { + x.ViewFn = v } -func (x *StandardArtifacts) String() string { - return protoimpl.X.MessageStringOf(x) +func (x *SideInput) SetWindowMappingFn(v *FunctionSpec) { + x.WindowMappingFn = v } -func (*StandardArtifacts) ProtoMessage() {} - -func (x *StandardArtifacts) ProtoReflect() protoreflect.Message { - mi := &file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_msgTypes[41] - if protoimpl.UnsafeEnabled && x != nil { - ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) - if ms.LoadMessageInfo() == nil { - ms.StoreMessageInfo(mi) - } - return ms +func (x *SideInput) HasAccessPattern() bool { + if x == nil { + return false } - return mi.MessageOf(x) + return x.AccessPattern != nil } -// Deprecated: Use StandardArtifacts.ProtoReflect.Descriptor instead. -func (*StandardArtifacts) Descriptor() ([]byte, []int) { - return file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_rawDescGZIP(), []int{41} +func (x *SideInput) HasViewFn() bool { + if x == nil { + return false + } + return x.ViewFn != nil } -type ArtifactFilePayload struct { - state protoimpl.MessageState - sizeCache protoimpl.SizeCache - unknownFields protoimpl.UnknownFields +func (x *SideInput) HasWindowMappingFn() bool { + if x == nil { + return false + } + return x.WindowMappingFn != nil +} - // a string for an artifact file path e.g. "/tmp/foo.jar" - Path string `protobuf:"bytes,1,opt,name=path,proto3" json:"path,omitempty"` - // The hex-encoded sha256 checksum of the artifact. - Sha256 string `protobuf:"bytes,2,opt,name=sha256,proto3" json:"sha256,omitempty"` +func (x *SideInput) ClearAccessPattern() { + x.AccessPattern = nil } -func (x *ArtifactFilePayload) Reset() { - *x = ArtifactFilePayload{} - if protoimpl.UnsafeEnabled { - mi := &file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_msgTypes[42] - ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) - ms.StoreMessageInfo(mi) - } +func (x *SideInput) ClearViewFn() { + x.ViewFn = nil } -func (x *ArtifactFilePayload) String() string { - return protoimpl.X.MessageStringOf(x) +func (x *SideInput) ClearWindowMappingFn() { + x.WindowMappingFn = nil } -func (*ArtifactFilePayload) ProtoMessage() {} +type SideInput_builder struct { + _ [0]func() // Prevents comparability and use of unkeyed literals for the builder. -func (x *ArtifactFilePayload) ProtoReflect() protoreflect.Message { - mi := &file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_msgTypes[42] - if protoimpl.UnsafeEnabled && x != nil { - ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) - if ms.LoadMessageInfo() == nil { - ms.StoreMessageInfo(mi) - } - return ms + // (Required) URN of the access pattern required by the `view_fn` to present + // the desired SDK-specific interface to a UDF. + // + // This access pattern defines the SDK harness <-> Runner Harness RPC + // interface for accessing a side input. + // + // See StandardSideInputTypes for an enumeration of all side input types + // defined. + AccessPattern *FunctionSpec + // (Required) The FunctionSpec of the UDF that adapts a particular + // access_pattern to a user-facing view type. + // + // For example, View.asSingleton() may include a `view_fn` that adapts a + // specially-designed multimap to a single value per window. + ViewFn *FunctionSpec + // (Required) The FunctionSpec of the UDF that maps a main input window + // to a side input window. + // + // For example, when the main input is in fixed windows of one hour, this + // can specify that the side input should be accessed according to the day + // in which that hour falls. + WindowMappingFn *FunctionSpec +} + +func (b0 SideInput_builder) Build() *SideInput { + m0 := &SideInput{} + b, x := &b0, m0 + _, _ = b, x + x.AccessPattern = b.AccessPattern + x.ViewFn = b.ViewFn + x.WindowMappingFn = b.WindowMappingFn + return m0 +} + +type StandardArtifacts struct { + state protoimpl.MessageState `protogen:"hybrid.v1"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *StandardArtifacts) Reset() { + *x = StandardArtifacts{} + mi := &file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_msgTypes[41] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *StandardArtifacts) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*StandardArtifacts) ProtoMessage() {} + +func (x *StandardArtifacts) ProtoReflect() protoreflect.Message { + mi := &file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_msgTypes[41] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms } return mi.MessageOf(x) } -// Deprecated: Use ArtifactFilePayload.ProtoReflect.Descriptor instead. -func (*ArtifactFilePayload) Descriptor() ([]byte, []int) { - return file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_rawDescGZIP(), []int{42} +type StandardArtifacts_builder struct { + _ [0]func() // Prevents comparability and use of unkeyed literals for the builder. + +} + +func (b0 StandardArtifacts_builder) Build() *StandardArtifacts { + m0 := &StandardArtifacts{} + b, x := &b0, m0 + _, _ = b, x + return m0 +} + +type ArtifactFilePayload struct { + state protoimpl.MessageState `protogen:"hybrid.v1"` + // a string for an artifact file path e.g. "/tmp/foo.jar" + Path string `protobuf:"bytes,1,opt,name=path,proto3" json:"path,omitempty"` + // The hex-encoded sha256 checksum of the artifact. + Sha256 string `protobuf:"bytes,2,opt,name=sha256,proto3" json:"sha256,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *ArtifactFilePayload) Reset() { + *x = ArtifactFilePayload{} + mi := &file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_msgTypes[42] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *ArtifactFilePayload) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*ArtifactFilePayload) ProtoMessage() {} + +func (x *ArtifactFilePayload) ProtoReflect() protoreflect.Message { + mi := &file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_msgTypes[42] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) } func (x *ArtifactFilePayload) GetPath() string { @@ -4981,24 +6739,47 @@ func (x *ArtifactFilePayload) GetSha256() string { return "" } -type ArtifactUrlPayload struct { - state protoimpl.MessageState - sizeCache protoimpl.SizeCache - unknownFields protoimpl.UnknownFields +func (x *ArtifactFilePayload) SetPath(v string) { + x.Path = v +} + +func (x *ArtifactFilePayload) SetSha256(v string) { + x.Sha256 = v +} + +type ArtifactFilePayload_builder struct { + _ [0]func() // Prevents comparability and use of unkeyed literals for the builder. + // a string for an artifact file path e.g. "/tmp/foo.jar" + Path string + // The hex-encoded sha256 checksum of the artifact. + Sha256 string +} + +func (b0 ArtifactFilePayload_builder) Build() *ArtifactFilePayload { + m0 := &ArtifactFilePayload{} + b, x := &b0, m0 + _, _ = b, x + x.Path = b.Path + x.Sha256 = b.Sha256 + return m0 +} + +type ArtifactUrlPayload struct { + state protoimpl.MessageState `protogen:"hybrid.v1"` // a string for an artifact URL e.g. "https://.../foo.jar" or "gs://tmp/foo.jar" Url string `protobuf:"bytes,1,opt,name=url,proto3" json:"url,omitempty"` // (Optional) The hex-encoded sha256 checksum of the artifact if available. - Sha256 string `protobuf:"bytes,2,opt,name=sha256,proto3" json:"sha256,omitempty"` + Sha256 string `protobuf:"bytes,2,opt,name=sha256,proto3" json:"sha256,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache } func (x *ArtifactUrlPayload) Reset() { *x = ArtifactUrlPayload{} - if protoimpl.UnsafeEnabled { - mi := &file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_msgTypes[43] - ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) - ms.StoreMessageInfo(mi) - } + mi := &file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_msgTypes[43] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) } func (x *ArtifactUrlPayload) String() string { @@ -5009,7 +6790,7 @@ func (*ArtifactUrlPayload) ProtoMessage() {} func (x *ArtifactUrlPayload) ProtoReflect() protoreflect.Message { mi := &file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_msgTypes[43] - if protoimpl.UnsafeEnabled && x != nil { + if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { ms.StoreMessageInfo(mi) @@ -5019,11 +6800,6 @@ func (x *ArtifactUrlPayload) ProtoReflect() protoreflect.Message { return mi.MessageOf(x) } -// Deprecated: Use ArtifactUrlPayload.ProtoReflect.Descriptor instead. -func (*ArtifactUrlPayload) Descriptor() ([]byte, []int) { - return file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_rawDescGZIP(), []int{43} -} - func (x *ArtifactUrlPayload) GetUrl() string { if x != nil { return x.Url @@ -5038,22 +6814,45 @@ func (x *ArtifactUrlPayload) GetSha256() string { return "" } -type EmbeddedFilePayload struct { - state protoimpl.MessageState - sizeCache protoimpl.SizeCache - unknownFields protoimpl.UnknownFields +func (x *ArtifactUrlPayload) SetUrl(v string) { + x.Url = v +} + +func (x *ArtifactUrlPayload) SetSha256(v string) { + x.Sha256 = v +} + +type ArtifactUrlPayload_builder struct { + _ [0]func() // Prevents comparability and use of unkeyed literals for the builder. + + // a string for an artifact URL e.g. "https://.../foo.jar" or "gs://tmp/foo.jar" + Url string + // (Optional) The hex-encoded sha256 checksum of the artifact if available. + Sha256 string +} + +func (b0 ArtifactUrlPayload_builder) Build() *ArtifactUrlPayload { + m0 := &ArtifactUrlPayload{} + b, x := &b0, m0 + _, _ = b, x + x.Url = b.Url + x.Sha256 = b.Sha256 + return m0 +} +type EmbeddedFilePayload struct { + state protoimpl.MessageState `protogen:"hybrid.v1"` // raw data bytes for an embedded artifact - Data []byte `protobuf:"bytes,1,opt,name=data,proto3" json:"data,omitempty"` + Data []byte `protobuf:"bytes,1,opt,name=data,proto3" json:"data,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache } func (x *EmbeddedFilePayload) Reset() { *x = EmbeddedFilePayload{} - if protoimpl.UnsafeEnabled { - mi := &file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_msgTypes[44] - ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) - ms.StoreMessageInfo(mi) - } + mi := &file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_msgTypes[44] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) } func (x *EmbeddedFilePayload) String() string { @@ -5064,7 +6863,7 @@ func (*EmbeddedFilePayload) ProtoMessage() {} func (x *EmbeddedFilePayload) ProtoReflect() protoreflect.Message { mi := &file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_msgTypes[44] - if protoimpl.UnsafeEnabled && x != nil { + if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { ms.StoreMessageInfo(mi) @@ -5074,11 +6873,6 @@ func (x *EmbeddedFilePayload) ProtoReflect() protoreflect.Message { return mi.MessageOf(x) } -// Deprecated: Use EmbeddedFilePayload.ProtoReflect.Descriptor instead. -func (*EmbeddedFilePayload) Descriptor() ([]byte, []int) { - return file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_rawDescGZIP(), []int{44} -} - func (x *EmbeddedFilePayload) GetData() []byte { if x != nil { return x.Data @@ -5086,24 +6880,43 @@ func (x *EmbeddedFilePayload) GetData() []byte { return nil } -type PyPIPayload struct { - state protoimpl.MessageState - sizeCache protoimpl.SizeCache - unknownFields protoimpl.UnknownFields +func (x *EmbeddedFilePayload) SetData(v []byte) { + if v == nil { + v = []byte{} + } + x.Data = v +} + +type EmbeddedFilePayload_builder struct { + _ [0]func() // Prevents comparability and use of unkeyed literals for the builder. + + // raw data bytes for an embedded artifact + Data []byte +} + +func (b0 EmbeddedFilePayload_builder) Build() *EmbeddedFilePayload { + m0 := &EmbeddedFilePayload{} + b, x := &b0, m0 + _, _ = b, x + x.Data = b.Data + return m0 +} +type PyPIPayload struct { + state protoimpl.MessageState `protogen:"hybrid.v1"` // Pypi compatible artifact id e.g. "apache-beam" ArtifactId string `protobuf:"bytes,1,opt,name=artifact_id,json=artifactId,proto3" json:"artifact_id,omitempty"` // Pypi compatible version string. - Version string `protobuf:"bytes,2,opt,name=version,proto3" json:"version,omitempty"` + Version string `protobuf:"bytes,2,opt,name=version,proto3" json:"version,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache } func (x *PyPIPayload) Reset() { *x = PyPIPayload{} - if protoimpl.UnsafeEnabled { - mi := &file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_msgTypes[45] - ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) - ms.StoreMessageInfo(mi) - } + mi := &file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_msgTypes[45] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) } func (x *PyPIPayload) String() string { @@ -5114,7 +6927,7 @@ func (*PyPIPayload) ProtoMessage() {} func (x *PyPIPayload) ProtoReflect() protoreflect.Message { mi := &file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_msgTypes[45] - if protoimpl.UnsafeEnabled && x != nil { + if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { ms.StoreMessageInfo(mi) @@ -5124,11 +6937,6 @@ func (x *PyPIPayload) ProtoReflect() protoreflect.Message { return mi.MessageOf(x) } -// Deprecated: Use PyPIPayload.ProtoReflect.Descriptor instead. -func (*PyPIPayload) Descriptor() ([]byte, []int) { - return file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_rawDescGZIP(), []int{45} -} - func (x *PyPIPayload) GetArtifactId() string { if x != nil { return x.ArtifactId @@ -5143,25 +6951,48 @@ func (x *PyPIPayload) GetVersion() string { return "" } -type MavenPayload struct { - state protoimpl.MessageState - sizeCache protoimpl.SizeCache - unknownFields protoimpl.UnknownFields +func (x *PyPIPayload) SetArtifactId(v string) { + x.ArtifactId = v +} + +func (x *PyPIPayload) SetVersion(v string) { + x.Version = v +} + +type PyPIPayload_builder struct { + _ [0]func() // Prevents comparability and use of unkeyed literals for the builder. + // Pypi compatible artifact id e.g. "apache-beam" + ArtifactId string + // Pypi compatible version string. + Version string +} + +func (b0 PyPIPayload_builder) Build() *PyPIPayload { + m0 := &PyPIPayload{} + b, x := &b0, m0 + _, _ = b, x + x.ArtifactId = b.ArtifactId + x.Version = b.Version + return m0 +} + +type MavenPayload struct { + state protoimpl.MessageState `protogen:"hybrid.v1"` // A string specifying Maven artifact. // The standard format is "groupId:artifactId:version[:packaging[:classifier]]" Artifact string `protobuf:"bytes,1,opt,name=artifact,proto3" json:"artifact,omitempty"` // (Optional) Repository URL. If not specified, Maven central is used by default. RepositoryUrl string `protobuf:"bytes,2,opt,name=repository_url,json=repositoryUrl,proto3" json:"repository_url,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache } func (x *MavenPayload) Reset() { *x = MavenPayload{} - if protoimpl.UnsafeEnabled { - mi := &file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_msgTypes[46] - ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) - ms.StoreMessageInfo(mi) - } + mi := &file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_msgTypes[46] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) } func (x *MavenPayload) String() string { @@ -5172,7 +7003,7 @@ func (*MavenPayload) ProtoMessage() {} func (x *MavenPayload) ProtoReflect() protoreflect.Message { mi := &file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_msgTypes[46] - if protoimpl.UnsafeEnabled && x != nil { + if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { ms.StoreMessageInfo(mi) @@ -5182,11 +7013,6 @@ func (x *MavenPayload) ProtoReflect() protoreflect.Message { return mi.MessageOf(x) } -// Deprecated: Use MavenPayload.ProtoReflect.Descriptor instead. -func (*MavenPayload) Descriptor() ([]byte, []int) { - return file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_rawDescGZIP(), []int{46} -} - func (x *MavenPayload) GetArtifact() string { if x != nil { return x.Artifact @@ -5201,25 +7027,49 @@ func (x *MavenPayload) GetRepositoryUrl() string { return "" } -type DeferredArtifactPayload struct { - state protoimpl.MessageState - sizeCache protoimpl.SizeCache - unknownFields protoimpl.UnknownFields +func (x *MavenPayload) SetArtifact(v string) { + x.Artifact = v +} + +func (x *MavenPayload) SetRepositoryUrl(v string) { + x.RepositoryUrl = v +} + +type MavenPayload_builder struct { + _ [0]func() // Prevents comparability and use of unkeyed literals for the builder. + + // A string specifying Maven artifact. + // The standard format is "groupId:artifactId:version[:packaging[:classifier]]" + Artifact string + // (Optional) Repository URL. If not specified, Maven central is used by default. + RepositoryUrl string +} + +func (b0 MavenPayload_builder) Build() *MavenPayload { + m0 := &MavenPayload{} + b, x := &b0, m0 + _, _ = b, x + x.Artifact = b.Artifact + x.RepositoryUrl = b.RepositoryUrl + return m0 +} +type DeferredArtifactPayload struct { + state protoimpl.MessageState `protogen:"hybrid.v1"` // A unique string identifier assigned by the creator of this payload. The creator may use this key to confirm // whether they can parse the data. Key string `protobuf:"bytes,1,opt,name=key,proto3" json:"key,omitempty"` // Data for deferred artifacts. Interpretation of bytes is delegated to the creator of this payload. - Data []byte `protobuf:"bytes,2,opt,name=data,proto3" json:"data,omitempty"` + Data []byte `protobuf:"bytes,2,opt,name=data,proto3" json:"data,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache } func (x *DeferredArtifactPayload) Reset() { *x = DeferredArtifactPayload{} - if protoimpl.UnsafeEnabled { - mi := &file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_msgTypes[47] - ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) - ms.StoreMessageInfo(mi) - } + mi := &file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_msgTypes[47] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) } func (x *DeferredArtifactPayload) String() string { @@ -5230,7 +7080,7 @@ func (*DeferredArtifactPayload) ProtoMessage() {} func (x *DeferredArtifactPayload) ProtoReflect() protoreflect.Message { mi := &file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_msgTypes[47] - if protoimpl.UnsafeEnabled && x != nil { + if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { ms.StoreMessageInfo(mi) @@ -5240,11 +7090,6 @@ func (x *DeferredArtifactPayload) ProtoReflect() protoreflect.Message { return mi.MessageOf(x) } -// Deprecated: Use DeferredArtifactPayload.ProtoReflect.Descriptor instead. -func (*DeferredArtifactPayload) Descriptor() ([]byte, []int) { - return file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_rawDescGZIP(), []int{47} -} - func (x *DeferredArtifactPayload) GetKey() string { if x != nil { return x.Key @@ -5259,22 +7104,49 @@ func (x *DeferredArtifactPayload) GetData() []byte { return nil } -type ArtifactStagingToRolePayload struct { - state protoimpl.MessageState - sizeCache protoimpl.SizeCache - unknownFields protoimpl.UnknownFields +func (x *DeferredArtifactPayload) SetKey(v string) { + x.Key = v +} + +func (x *DeferredArtifactPayload) SetData(v []byte) { + if v == nil { + v = []byte{} + } + x.Data = v +} + +type DeferredArtifactPayload_builder struct { + _ [0]func() // Prevents comparability and use of unkeyed literals for the builder. + + // A unique string identifier assigned by the creator of this payload. The creator may use this key to confirm + // whether they can parse the data. + Key string + // Data for deferred artifacts. Interpretation of bytes is delegated to the creator of this payload. + Data []byte +} + +func (b0 DeferredArtifactPayload_builder) Build() *DeferredArtifactPayload { + m0 := &DeferredArtifactPayload{} + b, x := &b0, m0 + _, _ = b, x + x.Key = b.Key + x.Data = b.Data + return m0 +} +type ArtifactStagingToRolePayload struct { + state protoimpl.MessageState `protogen:"hybrid.v1"` // A generated staged name (relative path under staging directory). - StagedName string `protobuf:"bytes,1,opt,name=staged_name,json=stagedName,proto3" json:"staged_name,omitempty"` + StagedName string `protobuf:"bytes,1,opt,name=staged_name,json=stagedName,proto3" json:"staged_name,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache } func (x *ArtifactStagingToRolePayload) Reset() { *x = ArtifactStagingToRolePayload{} - if protoimpl.UnsafeEnabled { - mi := &file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_msgTypes[48] - ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) - ms.StoreMessageInfo(mi) - } + mi := &file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_msgTypes[48] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) } func (x *ArtifactStagingToRolePayload) String() string { @@ -5285,7 +7157,7 @@ func (*ArtifactStagingToRolePayload) ProtoMessage() {} func (x *ArtifactStagingToRolePayload) ProtoReflect() protoreflect.Message { mi := &file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_msgTypes[48] - if protoimpl.UnsafeEnabled && x != nil { + if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { ms.StoreMessageInfo(mi) @@ -5295,11 +7167,6 @@ func (x *ArtifactStagingToRolePayload) ProtoReflect() protoreflect.Message { return mi.MessageOf(x) } -// Deprecated: Use ArtifactStagingToRolePayload.ProtoReflect.Descriptor instead. -func (*ArtifactStagingToRolePayload) Descriptor() ([]byte, []int) { - return file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_rawDescGZIP(), []int{48} -} - func (x *ArtifactStagingToRolePayload) GetStagedName() string { if x != nil { return x.StagedName @@ -5307,26 +7174,42 @@ func (x *ArtifactStagingToRolePayload) GetStagedName() string { return "" } -type ArtifactInformation struct { - state protoimpl.MessageState - sizeCache protoimpl.SizeCache - unknownFields protoimpl.UnknownFields +func (x *ArtifactStagingToRolePayload) SetStagedName(v string) { + x.StagedName = v +} + +type ArtifactStagingToRolePayload_builder struct { + _ [0]func() // Prevents comparability and use of unkeyed literals for the builder. + + // A generated staged name (relative path under staging directory). + StagedName string +} + +func (b0 ArtifactStagingToRolePayload_builder) Build() *ArtifactStagingToRolePayload { + m0 := &ArtifactStagingToRolePayload{} + b, x := &b0, m0 + _, _ = b, x + x.StagedName = b.StagedName + return m0 +} +type ArtifactInformation struct { + state protoimpl.MessageState `protogen:"hybrid.v1"` // A URN that describes the type of artifact TypeUrn string `protobuf:"bytes,1,opt,name=type_urn,json=typeUrn,proto3" json:"type_urn,omitempty"` TypePayload []byte `protobuf:"bytes,2,opt,name=type_payload,json=typePayload,proto3" json:"type_payload,omitempty"` // A URN that describes the role of artifact - RoleUrn string `protobuf:"bytes,3,opt,name=role_urn,json=roleUrn,proto3" json:"role_urn,omitempty"` - RolePayload []byte `protobuf:"bytes,4,opt,name=role_payload,json=rolePayload,proto3" json:"role_payload,omitempty"` + RoleUrn string `protobuf:"bytes,3,opt,name=role_urn,json=roleUrn,proto3" json:"role_urn,omitempty"` + RolePayload []byte `protobuf:"bytes,4,opt,name=role_payload,json=rolePayload,proto3" json:"role_payload,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache } func (x *ArtifactInformation) Reset() { *x = ArtifactInformation{} - if protoimpl.UnsafeEnabled { - mi := &file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_msgTypes[49] - ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) - ms.StoreMessageInfo(mi) - } + mi := &file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_msgTypes[49] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) } func (x *ArtifactInformation) String() string { @@ -5337,7 +7220,7 @@ func (*ArtifactInformation) ProtoMessage() {} func (x *ArtifactInformation) ProtoReflect() protoreflect.Message { mi := &file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_msgTypes[49] - if protoimpl.UnsafeEnabled && x != nil { + if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { ms.StoreMessageInfo(mi) @@ -5347,11 +7230,6 @@ func (x *ArtifactInformation) ProtoReflect() protoreflect.Message { return mi.MessageOf(x) } -// Deprecated: Use ArtifactInformation.ProtoReflect.Descriptor instead. -func (*ArtifactInformation) Descriptor() ([]byte, []int) { - return file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_rawDescGZIP(), []int{49} -} - func (x *ArtifactInformation) GetTypeUrn() string { if x != nil { return x.TypeUrn @@ -5380,13 +7258,54 @@ func (x *ArtifactInformation) GetRolePayload() []byte { return nil } +func (x *ArtifactInformation) SetTypeUrn(v string) { + x.TypeUrn = v +} + +func (x *ArtifactInformation) SetTypePayload(v []byte) { + if v == nil { + v = []byte{} + } + x.TypePayload = v +} + +func (x *ArtifactInformation) SetRoleUrn(v string) { + x.RoleUrn = v +} + +func (x *ArtifactInformation) SetRolePayload(v []byte) { + if v == nil { + v = []byte{} + } + x.RolePayload = v +} + +type ArtifactInformation_builder struct { + _ [0]func() // Prevents comparability and use of unkeyed literals for the builder. + + // A URN that describes the type of artifact + TypeUrn string + TypePayload []byte + // A URN that describes the role of artifact + RoleUrn string + RolePayload []byte +} + +func (b0 ArtifactInformation_builder) Build() *ArtifactInformation { + m0 := &ArtifactInformation{} + b, x := &b0, m0 + _, _ = b, x + x.TypeUrn = b.TypeUrn + x.TypePayload = b.TypePayload + x.RoleUrn = b.RoleUrn + x.RolePayload = b.RolePayload + return m0 +} + // An environment for executing UDFs. By default, an SDK container URL, but // can also be a process forked by a command, or an externally managed process. type Environment struct { - state protoimpl.MessageState - sizeCache protoimpl.SizeCache - unknownFields protoimpl.UnknownFields - + state protoimpl.MessageState `protogen:"hybrid.v1"` // (Required) The URN of the payload Urn string `protobuf:"bytes,2,opt,name=urn,proto3" json:"urn,omitempty"` // (Optional) The data specifying any parameters to the URN. If @@ -5406,16 +7325,16 @@ type Environment struct { // (Optional) A mapping of resource URNs to requested values. The encoding // of the values is specified by the URN. Resource hints are advisory; // a runner is free to ignore resource hints that it does not understand. - ResourceHints map[string][]byte `protobuf:"bytes,7,rep,name=resource_hints,json=resourceHints,proto3" json:"resource_hints,omitempty" protobuf_key:"bytes,1,opt,name=key,proto3" protobuf_val:"bytes,2,opt,name=value,proto3"` + ResourceHints map[string][]byte `protobuf:"bytes,7,rep,name=resource_hints,json=resourceHints,proto3" json:"resource_hints,omitempty" protobuf_key:"bytes,1,opt,name=key" protobuf_val:"bytes,2,opt,name=value"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache } func (x *Environment) Reset() { *x = Environment{} - if protoimpl.UnsafeEnabled { - mi := &file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_msgTypes[50] - ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) - ms.StoreMessageInfo(mi) - } + mi := &file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_msgTypes[50] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) } func (x *Environment) String() string { @@ -5426,7 +7345,7 @@ func (*Environment) ProtoMessage() {} func (x *Environment) ProtoReflect() protoreflect.Message { mi := &file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_msgTypes[50] - if protoimpl.UnsafeEnabled && x != nil { + if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { ms.StoreMessageInfo(mi) @@ -5436,11 +7355,6 @@ func (x *Environment) ProtoReflect() protoreflect.Message { return mi.MessageOf(x) } -// Deprecated: Use Environment.ProtoReflect.Descriptor instead. -func (*Environment) Descriptor() ([]byte, []int) { - return file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_rawDescGZIP(), []int{50} -} - func (x *Environment) GetUrn() string { if x != nil { return x.Urn @@ -5483,19 +7397,82 @@ func (x *Environment) GetResourceHints() map[string][]byte { return nil } +func (x *Environment) SetUrn(v string) { + x.Urn = v +} + +func (x *Environment) SetPayload(v []byte) { + if v == nil { + v = []byte{} + } + x.Payload = v +} + +func (x *Environment) SetDisplayData(v []*DisplayData) { + x.DisplayData = v +} + +func (x *Environment) SetCapabilities(v []string) { + x.Capabilities = v +} + +func (x *Environment) SetDependencies(v []*ArtifactInformation) { + x.Dependencies = v +} + +func (x *Environment) SetResourceHints(v map[string][]byte) { + x.ResourceHints = v +} + +type Environment_builder struct { + _ [0]func() // Prevents comparability and use of unkeyed literals for the builder. + + // (Required) The URN of the payload + Urn string + // (Optional) The data specifying any parameters to the URN. If + // the URN does not require any arguments, this may be omitted. + Payload []byte + // (Optional) Static display data for the environment. If there is none, + // it may be omitted. + DisplayData []*DisplayData + // (Optional) A set of capabilities this environment supports. This is + // typically a list of common URNs designating coders, transforms, etc. that + // this environment understands (and a runner MAY use) despite not + // appearing in the pipeline proto. This may also be used to indicate + // support of optional protocols not tied to a concrete component. + Capabilities []string + // (Optional) artifact dependency information used for executing UDFs in this environment. + Dependencies []*ArtifactInformation + // (Optional) A mapping of resource URNs to requested values. The encoding + // of the values is specified by the URN. Resource hints are advisory; + // a runner is free to ignore resource hints that it does not understand. + ResourceHints map[string][]byte +} + +func (b0 Environment_builder) Build() *Environment { + m0 := &Environment{} + b, x := &b0, m0 + _, _ = b, x + x.Urn = b.Urn + x.Payload = b.Payload + x.DisplayData = b.DisplayData + x.Capabilities = b.Capabilities + x.Dependencies = b.Dependencies + x.ResourceHints = b.ResourceHints + return m0 +} + type StandardEnvironments struct { - state protoimpl.MessageState - sizeCache protoimpl.SizeCache + state protoimpl.MessageState `protogen:"hybrid.v1"` unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache } func (x *StandardEnvironments) Reset() { *x = StandardEnvironments{} - if protoimpl.UnsafeEnabled { - mi := &file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_msgTypes[51] - ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) - ms.StoreMessageInfo(mi) - } + mi := &file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_msgTypes[51] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) } func (x *StandardEnvironments) String() string { @@ -5506,7 +7483,7 @@ func (*StandardEnvironments) ProtoMessage() {} func (x *StandardEnvironments) ProtoReflect() protoreflect.Message { mi := &file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_msgTypes[51] - if protoimpl.UnsafeEnabled && x != nil { + if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { ms.StoreMessageInfo(mi) @@ -5516,27 +7493,31 @@ func (x *StandardEnvironments) ProtoReflect() protoreflect.Message { return mi.MessageOf(x) } -// Deprecated: Use StandardEnvironments.ProtoReflect.Descriptor instead. -func (*StandardEnvironments) Descriptor() ([]byte, []int) { - return file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_rawDescGZIP(), []int{51} +type StandardEnvironments_builder struct { + _ [0]func() // Prevents comparability and use of unkeyed literals for the builder. + +} + +func (b0 StandardEnvironments_builder) Build() *StandardEnvironments { + m0 := &StandardEnvironments{} + b, x := &b0, m0 + _, _ = b, x + return m0 } // The payload of a Docker image type DockerPayload struct { - state protoimpl.MessageState - sizeCache protoimpl.SizeCache - unknownFields protoimpl.UnknownFields - - ContainerImage string `protobuf:"bytes,1,opt,name=container_image,json=containerImage,proto3" json:"container_image,omitempty"` // implicitly linux_amd64. + state protoimpl.MessageState `protogen:"hybrid.v1"` + ContainerImage string `protobuf:"bytes,1,opt,name=container_image,json=containerImage,proto3" json:"container_image,omitempty"` // implicitly linux_amd64. + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache } func (x *DockerPayload) Reset() { *x = DockerPayload{} - if protoimpl.UnsafeEnabled { - mi := &file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_msgTypes[52] - ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) - ms.StoreMessageInfo(mi) - } + mi := &file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_msgTypes[52] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) } func (x *DockerPayload) String() string { @@ -5547,7 +7528,7 @@ func (*DockerPayload) ProtoMessage() {} func (x *DockerPayload) ProtoReflect() protoreflect.Message { mi := &file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_msgTypes[52] - if protoimpl.UnsafeEnabled && x != nil { + if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { ms.StoreMessageInfo(mi) @@ -5557,11 +7538,6 @@ func (x *DockerPayload) ProtoReflect() protoreflect.Message { return mi.MessageOf(x) } -// Deprecated: Use DockerPayload.ProtoReflect.Descriptor instead. -func (*DockerPayload) Descriptor() ([]byte, []int) { - return file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_rawDescGZIP(), []int{52} -} - func (x *DockerPayload) GetContainerImage() string { if x != nil { return x.ContainerImage @@ -5569,24 +7545,39 @@ func (x *DockerPayload) GetContainerImage() string { return "" } +func (x *DockerPayload) SetContainerImage(v string) { + x.ContainerImage = v +} + +type DockerPayload_builder struct { + _ [0]func() // Prevents comparability and use of unkeyed literals for the builder. + + ContainerImage string +} + +func (b0 DockerPayload_builder) Build() *DockerPayload { + m0 := &DockerPayload{} + b, x := &b0, m0 + _, _ = b, x + x.ContainerImage = b.ContainerImage + return m0 +} + type ProcessPayload struct { - state protoimpl.MessageState - sizeCache protoimpl.SizeCache + state protoimpl.MessageState `protogen:"hybrid.v1"` + Os string `protobuf:"bytes,1,opt,name=os,proto3" json:"os,omitempty"` // "linux", "darwin", .. + Arch string `protobuf:"bytes,2,opt,name=arch,proto3" json:"arch,omitempty"` // "amd64", .. + Command string `protobuf:"bytes,3,opt,name=command,proto3" json:"command,omitempty"` // process to execute + Env map[string]string `protobuf:"bytes,4,rep,name=env,proto3" json:"env,omitempty" protobuf_key:"bytes,1,opt,name=key" protobuf_val:"bytes,2,opt,name=value"` // Environment variables unknownFields protoimpl.UnknownFields - - Os string `protobuf:"bytes,1,opt,name=os,proto3" json:"os,omitempty"` // "linux", "darwin", .. - Arch string `protobuf:"bytes,2,opt,name=arch,proto3" json:"arch,omitempty"` // "amd64", .. - Command string `protobuf:"bytes,3,opt,name=command,proto3" json:"command,omitempty"` // process to execute - Env map[string]string `protobuf:"bytes,4,rep,name=env,proto3" json:"env,omitempty" protobuf_key:"bytes,1,opt,name=key,proto3" protobuf_val:"bytes,2,opt,name=value,proto3"` // Environment variables + sizeCache protoimpl.SizeCache } func (x *ProcessPayload) Reset() { *x = ProcessPayload{} - if protoimpl.UnsafeEnabled { - mi := &file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_msgTypes[53] - ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) - ms.StoreMessageInfo(mi) - } + mi := &file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_msgTypes[53] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) } func (x *ProcessPayload) String() string { @@ -5597,7 +7588,7 @@ func (*ProcessPayload) ProtoMessage() {} func (x *ProcessPayload) ProtoReflect() protoreflect.Message { mi := &file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_msgTypes[53] - if protoimpl.UnsafeEnabled && x != nil { + if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { ms.StoreMessageInfo(mi) @@ -5607,11 +7598,6 @@ func (x *ProcessPayload) ProtoReflect() protoreflect.Message { return mi.MessageOf(x) } -// Deprecated: Use ProcessPayload.ProtoReflect.Descriptor instead. -func (*ProcessPayload) Descriptor() ([]byte, []int) { - return file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_rawDescGZIP(), []int{53} -} - func (x *ProcessPayload) GetOs() string { if x != nil { return x.Os @@ -5640,22 +7626,55 @@ func (x *ProcessPayload) GetEnv() map[string]string { return nil } +func (x *ProcessPayload) SetOs(v string) { + x.Os = v +} + +func (x *ProcessPayload) SetArch(v string) { + x.Arch = v +} + +func (x *ProcessPayload) SetCommand(v string) { + x.Command = v +} + +func (x *ProcessPayload) SetEnv(v map[string]string) { + x.Env = v +} + +type ProcessPayload_builder struct { + _ [0]func() // Prevents comparability and use of unkeyed literals for the builder. + + Os string + Arch string + Command string + Env map[string]string +} + +func (b0 ProcessPayload_builder) Build() *ProcessPayload { + m0 := &ProcessPayload{} + b, x := &b0, m0 + _, _ = b, x + x.Os = b.Os + x.Arch = b.Arch + x.Command = b.Command + x.Env = b.Env + return m0 +} + type ExternalPayload struct { - state protoimpl.MessageState - sizeCache protoimpl.SizeCache + state protoimpl.MessageState `protogen:"hybrid.v1"` + Endpoint *ApiServiceDescriptor `protobuf:"bytes,1,opt,name=endpoint,proto3" json:"endpoint,omitempty"` // Serving BeamFnExternalWorkerPool API. + Params map[string]string `protobuf:"bytes,2,rep,name=params,proto3" json:"params,omitempty" protobuf_key:"bytes,1,opt,name=key" protobuf_val:"bytes,2,opt,name=value"` // Arbitrary extra parameters to pass unknownFields protoimpl.UnknownFields - - Endpoint *ApiServiceDescriptor `protobuf:"bytes,1,opt,name=endpoint,proto3" json:"endpoint,omitempty"` // Serving BeamFnExternalWorkerPool API. - Params map[string]string `protobuf:"bytes,2,rep,name=params,proto3" json:"params,omitempty" protobuf_key:"bytes,1,opt,name=key,proto3" protobuf_val:"bytes,2,opt,name=value,proto3"` // Arbitrary extra parameters to pass + sizeCache protoimpl.SizeCache } func (x *ExternalPayload) Reset() { *x = ExternalPayload{} - if protoimpl.UnsafeEnabled { - mi := &file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_msgTypes[54] - ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) - ms.StoreMessageInfo(mi) - } + mi := &file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_msgTypes[54] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) } func (x *ExternalPayload) String() string { @@ -5666,7 +7685,7 @@ func (*ExternalPayload) ProtoMessage() {} func (x *ExternalPayload) ProtoReflect() protoreflect.Message { mi := &file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_msgTypes[54] - if protoimpl.UnsafeEnabled && x != nil { + if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { ms.StoreMessageInfo(mi) @@ -5676,11 +7695,6 @@ func (x *ExternalPayload) ProtoReflect() protoreflect.Message { return mi.MessageOf(x) } -// Deprecated: Use ExternalPayload.ProtoReflect.Descriptor instead. -func (*ExternalPayload) Descriptor() ([]byte, []int) { - return file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_rawDescGZIP(), []int{54} -} - func (x *ExternalPayload) GetEndpoint() *ApiServiceDescriptor { if x != nil { return x.Endpoint @@ -5695,22 +7709,54 @@ func (x *ExternalPayload) GetParams() map[string]string { return nil } -type AnyOfEnvironmentPayload struct { - state protoimpl.MessageState - sizeCache protoimpl.SizeCache - unknownFields protoimpl.UnknownFields +func (x *ExternalPayload) SetEndpoint(v *ApiServiceDescriptor) { + x.Endpoint = v +} + +func (x *ExternalPayload) SetParams(v map[string]string) { + x.Params = v +} + +func (x *ExternalPayload) HasEndpoint() bool { + if x == nil { + return false + } + return x.Endpoint != nil +} + +func (x *ExternalPayload) ClearEndpoint() { + x.Endpoint = nil +} + +type ExternalPayload_builder struct { + _ [0]func() // Prevents comparability and use of unkeyed literals for the builder. + + Endpoint *ApiServiceDescriptor + Params map[string]string +} +func (b0 ExternalPayload_builder) Build() *ExternalPayload { + m0 := &ExternalPayload{} + b, x := &b0, m0 + _, _ = b, x + x.Endpoint = b.Endpoint + x.Params = b.Params + return m0 +} + +type AnyOfEnvironmentPayload struct { + state protoimpl.MessageState `protogen:"hybrid.v1"` // Each is fully contained (with their own dependencies, capabilities, etc.) - Environments []*Environment `protobuf:"bytes,1,rep,name=environments,proto3" json:"environments,omitempty"` + Environments []*Environment `protobuf:"bytes,1,rep,name=environments,proto3" json:"environments,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache } func (x *AnyOfEnvironmentPayload) Reset() { *x = AnyOfEnvironmentPayload{} - if protoimpl.UnsafeEnabled { - mi := &file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_msgTypes[55] - ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) - ms.StoreMessageInfo(mi) - } + mi := &file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_msgTypes[55] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) } func (x *AnyOfEnvironmentPayload) String() string { @@ -5721,7 +7767,7 @@ func (*AnyOfEnvironmentPayload) ProtoMessage() {} func (x *AnyOfEnvironmentPayload) ProtoReflect() protoreflect.Message { mi := &file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_msgTypes[55] - if protoimpl.UnsafeEnabled && x != nil { + if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { ms.StoreMessageInfo(mi) @@ -5731,11 +7777,6 @@ func (x *AnyOfEnvironmentPayload) ProtoReflect() protoreflect.Message { return mi.MessageOf(x) } -// Deprecated: Use AnyOfEnvironmentPayload.ProtoReflect.Descriptor instead. -func (*AnyOfEnvironmentPayload) Descriptor() ([]byte, []int) { - return file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_rawDescGZIP(), []int{55} -} - func (x *AnyOfEnvironmentPayload) GetEnvironments() []*Environment { if x != nil { return x.Environments @@ -5743,22 +7784,39 @@ func (x *AnyOfEnvironmentPayload) GetEnvironments() []*Environment { return nil } +func (x *AnyOfEnvironmentPayload) SetEnvironments(v []*Environment) { + x.Environments = v +} + +type AnyOfEnvironmentPayload_builder struct { + _ [0]func() // Prevents comparability and use of unkeyed literals for the builder. + + // Each is fully contained (with their own dependencies, capabilities, etc.) + Environments []*Environment +} + +func (b0 AnyOfEnvironmentPayload_builder) Build() *AnyOfEnvironmentPayload { + m0 := &AnyOfEnvironmentPayload{} + b, x := &b0, m0 + _, _ = b, x + x.Environments = b.Environments + return m0 +} + // These URNs are used to indicate capabilities of environments that cannot // simply be expressed as a component (such as a Coder or PTransform) that this // environment understands. type StandardProtocols struct { - state protoimpl.MessageState - sizeCache protoimpl.SizeCache + state protoimpl.MessageState `protogen:"hybrid.v1"` unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache } func (x *StandardProtocols) Reset() { *x = StandardProtocols{} - if protoimpl.UnsafeEnabled { - mi := &file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_msgTypes[56] - ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) - ms.StoreMessageInfo(mi) - } + mi := &file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_msgTypes[56] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) } func (x *StandardProtocols) String() string { @@ -5769,7 +7827,7 @@ func (*StandardProtocols) ProtoMessage() {} func (x *StandardProtocols) ProtoReflect() protoreflect.Message { mi := &file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_msgTypes[56] - if protoimpl.UnsafeEnabled && x != nil { + if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { ms.StoreMessageInfo(mi) @@ -5779,26 +7837,31 @@ func (x *StandardProtocols) ProtoReflect() protoreflect.Message { return mi.MessageOf(x) } -// Deprecated: Use StandardProtocols.ProtoReflect.Descriptor instead. -func (*StandardProtocols) Descriptor() ([]byte, []int) { - return file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_rawDescGZIP(), []int{56} +type StandardProtocols_builder struct { + _ [0]func() // Prevents comparability and use of unkeyed literals for the builder. + +} + +func (b0 StandardProtocols_builder) Build() *StandardProtocols { + m0 := &StandardProtocols{} + b, x := &b0, m0 + _, _ = b, x + return m0 } // These URNs are used to indicate capabilities of runner that an environment // may take advantage of when interacting with this runner. type StandardRunnerProtocols struct { - state protoimpl.MessageState - sizeCache protoimpl.SizeCache + state protoimpl.MessageState `protogen:"hybrid.v1"` unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache } func (x *StandardRunnerProtocols) Reset() { *x = StandardRunnerProtocols{} - if protoimpl.UnsafeEnabled { - mi := &file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_msgTypes[57] - ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) - ms.StoreMessageInfo(mi) - } + mi := &file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_msgTypes[57] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) } func (x *StandardRunnerProtocols) String() string { @@ -5809,7 +7872,7 @@ func (*StandardRunnerProtocols) ProtoMessage() {} func (x *StandardRunnerProtocols) ProtoReflect() protoreflect.Message { mi := &file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_msgTypes[57] - if protoimpl.UnsafeEnabled && x != nil { + if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { ms.StoreMessageInfo(mi) @@ -5819,9 +7882,16 @@ func (x *StandardRunnerProtocols) ProtoReflect() protoreflect.Message { return mi.MessageOf(x) } -// Deprecated: Use StandardRunnerProtocols.ProtoReflect.Descriptor instead. -func (*StandardRunnerProtocols) Descriptor() ([]byte, []int) { - return file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_rawDescGZIP(), []int{57} +type StandardRunnerProtocols_builder struct { + _ [0]func() // Prevents comparability and use of unkeyed literals for the builder. + +} + +func (b0 StandardRunnerProtocols_builder) Build() *StandardRunnerProtocols { + m0 := &StandardRunnerProtocols{} + b, x := &b0, m0 + _, _ = b, x + return m0 } // These URNs are used to indicate requirements of a pipeline that cannot @@ -5830,18 +7900,16 @@ func (*StandardRunnerProtocols) Descriptor() ([]byte, []int) { // of a transform must be inspected and respected (which allows new fields // to be added in a forwards-compatible way). type StandardRequirements struct { - state protoimpl.MessageState - sizeCache protoimpl.SizeCache + state protoimpl.MessageState `protogen:"hybrid.v1"` unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache } func (x *StandardRequirements) Reset() { *x = StandardRequirements{} - if protoimpl.UnsafeEnabled { - mi := &file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_msgTypes[58] - ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) - ms.StoreMessageInfo(mi) - } + mi := &file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_msgTypes[58] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) } func (x *StandardRequirements) String() string { @@ -5852,7 +7920,7 @@ func (*StandardRequirements) ProtoMessage() {} func (x *StandardRequirements) ProtoReflect() protoreflect.Message { mi := &file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_msgTypes[58] - if protoimpl.UnsafeEnabled && x != nil { + if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { ms.StoreMessageInfo(mi) @@ -5862,9 +7930,16 @@ func (x *StandardRequirements) ProtoReflect() protoreflect.Message { return mi.MessageOf(x) } -// Deprecated: Use StandardRequirements.ProtoReflect.Descriptor instead. -func (*StandardRequirements) Descriptor() ([]byte, []int) { - return file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_rawDescGZIP(), []int{58} +type StandardRequirements_builder struct { + _ [0]func() // Prevents comparability and use of unkeyed literals for the builder. + +} + +func (b0 StandardRequirements_builder) Build() *StandardRequirements { + m0 := &StandardRequirements{} + b, x := &b0, m0 + _, _ = b, x + return m0 } // A URN along with a parameter object whose schema is determined by the @@ -5895,10 +7970,7 @@ func (*StandardRequirements) Descriptor() ([]byte, []int) { // then be an opaque message such as bytes in a // language-specific serialization format. type FunctionSpec struct { - state protoimpl.MessageState - sizeCache protoimpl.SizeCache - unknownFields protoimpl.UnknownFields - + state protoimpl.MessageState `protogen:"hybrid.v1"` // (Required) A URN that describes the accompanying payload. // For any URN that is not recognized (by whomever is inspecting // it) the parameter payload should be treated as opaque and @@ -5906,16 +7978,16 @@ type FunctionSpec struct { Urn string `protobuf:"bytes,1,opt,name=urn,proto3" json:"urn,omitempty"` // (Optional) The data specifying any parameters to the URN. If // the URN does not require any arguments, this may be omitted. - Payload []byte `protobuf:"bytes,3,opt,name=payload,proto3" json:"payload,omitempty"` + Payload []byte `protobuf:"bytes,3,opt,name=payload,proto3" json:"payload,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache } func (x *FunctionSpec) Reset() { *x = FunctionSpec{} - if protoimpl.UnsafeEnabled { - mi := &file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_msgTypes[59] - ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) - ms.StoreMessageInfo(mi) - } + mi := &file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_msgTypes[59] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) } func (x *FunctionSpec) String() string { @@ -5926,7 +7998,7 @@ func (*FunctionSpec) ProtoMessage() {} func (x *FunctionSpec) ProtoReflect() protoreflect.Message { mi := &file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_msgTypes[59] - if protoimpl.UnsafeEnabled && x != nil { + if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { ms.StoreMessageInfo(mi) @@ -5936,11 +8008,6 @@ func (x *FunctionSpec) ProtoReflect() protoreflect.Message { return mi.MessageOf(x) } -// Deprecated: Use FunctionSpec.ProtoReflect.Descriptor instead. -func (*FunctionSpec) Descriptor() ([]byte, []int) { - return file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_rawDescGZIP(), []int{59} -} - func (x *FunctionSpec) GetUrn() string { if x != nil { return x.Urn @@ -5955,6 +8022,39 @@ func (x *FunctionSpec) GetPayload() []byte { return nil } +func (x *FunctionSpec) SetUrn(v string) { + x.Urn = v +} + +func (x *FunctionSpec) SetPayload(v []byte) { + if v == nil { + v = []byte{} + } + x.Payload = v +} + +type FunctionSpec_builder struct { + _ [0]func() // Prevents comparability and use of unkeyed literals for the builder. + + // (Required) A URN that describes the accompanying payload. + // For any URN that is not recognized (by whomever is inspecting + // it) the parameter payload should be treated as opaque and + // passed as-is. + Urn string + // (Optional) The data specifying any parameters to the URN. If + // the URN does not require any arguments, this may be omitted. + Payload []byte +} + +func (b0 FunctionSpec_builder) Build() *FunctionSpec { + m0 := &FunctionSpec{} + b, x := &b0, m0 + _, _ = b, x + x.Urn = b.Urn + x.Payload = b.Payload + return m0 +} + // A set of well known URNs describing display data. // // All descriptions must contain how the value should be classified and how it @@ -5962,18 +8062,16 @@ func (x *FunctionSpec) GetPayload() []byte { // information about the pipeline in addition to an encoding while others only // specify the encoding itself. type StandardDisplayData struct { - state protoimpl.MessageState - sizeCache protoimpl.SizeCache + state protoimpl.MessageState `protogen:"hybrid.v1"` unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache } func (x *StandardDisplayData) Reset() { *x = StandardDisplayData{} - if protoimpl.UnsafeEnabled { - mi := &file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_msgTypes[60] - ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) - ms.StoreMessageInfo(mi) - } + mi := &file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_msgTypes[60] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) } func (x *StandardDisplayData) String() string { @@ -5984,7 +8082,7 @@ func (*StandardDisplayData) ProtoMessage() {} func (x *StandardDisplayData) ProtoReflect() protoreflect.Message { mi := &file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_msgTypes[60] - if protoimpl.UnsafeEnabled && x != nil { + if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { ms.StoreMessageInfo(mi) @@ -5994,21 +8092,25 @@ func (x *StandardDisplayData) ProtoReflect() protoreflect.Message { return mi.MessageOf(x) } -// Deprecated: Use StandardDisplayData.ProtoReflect.Descriptor instead. -func (*StandardDisplayData) Descriptor() ([]byte, []int) { - return file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_rawDescGZIP(), []int{60} +type StandardDisplayData_builder struct { + _ [0]func() // Prevents comparability and use of unkeyed literals for the builder. + } -type LabelledPayload struct { - state protoimpl.MessageState - sizeCache protoimpl.SizeCache - unknownFields protoimpl.UnknownFields +func (b0 StandardDisplayData_builder) Build() *StandardDisplayData { + m0 := &StandardDisplayData{} + b, x := &b0, m0 + _, _ = b, x + return m0 +} +type LabelledPayload struct { + state protoimpl.MessageState `protogen:"hybrid.v1"` // (Required) A human readable label for the value. Label string `protobuf:"bytes,1,opt,name=label,proto3" json:"label,omitempty"` // (Required) A value which will be displayed to the user. // - // Types that are assignable to Value: + // Types that are valid to be assigned to Value: // // *LabelledPayload_StringValue // *LabelledPayload_BoolValue @@ -6018,16 +8120,16 @@ type LabelledPayload struct { // (Required) The key identifies the actual content of the metadata. Key string `protobuf:"bytes,6,opt,name=key,proto3" json:"key,omitempty"` // (Required) The namespace describes the context that specified the key. - Namespace string `protobuf:"bytes,7,opt,name=namespace,proto3" json:"namespace,omitempty"` + Namespace string `protobuf:"bytes,7,opt,name=namespace,proto3" json:"namespace,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache } func (x *LabelledPayload) Reset() { *x = LabelledPayload{} - if protoimpl.UnsafeEnabled { - mi := &file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_msgTypes[61] - ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) - ms.StoreMessageInfo(mi) - } + mi := &file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_msgTypes[61] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) } func (x *LabelledPayload) String() string { @@ -6038,7 +8140,7 @@ func (*LabelledPayload) ProtoMessage() {} func (x *LabelledPayload) ProtoReflect() protoreflect.Message { mi := &file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_msgTypes[61] - if protoimpl.UnsafeEnabled && x != nil { + if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { ms.StoreMessageInfo(mi) @@ -6048,11 +8150,6 @@ func (x *LabelledPayload) ProtoReflect() protoreflect.Message { return mi.MessageOf(x) } -// Deprecated: Use LabelledPayload.ProtoReflect.Descriptor instead. -func (*LabelledPayload) Descriptor() ([]byte, []int) { - return file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_rawDescGZIP(), []int{61} -} - func (x *LabelledPayload) GetLabel() string { if x != nil { return x.Label @@ -6060,37 +8157,45 @@ func (x *LabelledPayload) GetLabel() string { return "" } -func (m *LabelledPayload) GetValue() isLabelledPayload_Value { - if m != nil { - return m.Value +func (x *LabelledPayload) GetValue() isLabelledPayload_Value { + if x != nil { + return x.Value } return nil } func (x *LabelledPayload) GetStringValue() string { - if x, ok := x.GetValue().(*LabelledPayload_StringValue); ok { - return x.StringValue + if x != nil { + if x, ok := x.Value.(*LabelledPayload_StringValue); ok { + return x.StringValue + } } return "" } func (x *LabelledPayload) GetBoolValue() bool { - if x, ok := x.GetValue().(*LabelledPayload_BoolValue); ok { - return x.BoolValue + if x != nil { + if x, ok := x.Value.(*LabelledPayload_BoolValue); ok { + return x.BoolValue + } } return false } func (x *LabelledPayload) GetDoubleValue() float64 { - if x, ok := x.GetValue().(*LabelledPayload_DoubleValue); ok { - return x.DoubleValue + if x != nil { + if x, ok := x.Value.(*LabelledPayload_DoubleValue); ok { + return x.DoubleValue + } } return 0 } func (x *LabelledPayload) GetIntValue() int64 { - if x, ok := x.GetValue().(*LabelledPayload_IntValue); ok { - return x.IntValue + if x != nil { + if x, ok := x.Value.(*LabelledPayload_IntValue); ok { + return x.IntValue + } } return 0 } @@ -6109,6 +8214,176 @@ func (x *LabelledPayload) GetNamespace() string { return "" } +func (x *LabelledPayload) SetLabel(v string) { + x.Label = v +} + +func (x *LabelledPayload) SetStringValue(v string) { + x.Value = &LabelledPayload_StringValue{v} +} + +func (x *LabelledPayload) SetBoolValue(v bool) { + x.Value = &LabelledPayload_BoolValue{v} +} + +func (x *LabelledPayload) SetDoubleValue(v float64) { + x.Value = &LabelledPayload_DoubleValue{v} +} + +func (x *LabelledPayload) SetIntValue(v int64) { + x.Value = &LabelledPayload_IntValue{v} +} + +func (x *LabelledPayload) SetKey(v string) { + x.Key = v +} + +func (x *LabelledPayload) SetNamespace(v string) { + x.Namespace = v +} + +func (x *LabelledPayload) HasValue() bool { + if x == nil { + return false + } + return x.Value != nil +} + +func (x *LabelledPayload) HasStringValue() bool { + if x == nil { + return false + } + _, ok := x.Value.(*LabelledPayload_StringValue) + return ok +} + +func (x *LabelledPayload) HasBoolValue() bool { + if x == nil { + return false + } + _, ok := x.Value.(*LabelledPayload_BoolValue) + return ok +} + +func (x *LabelledPayload) HasDoubleValue() bool { + if x == nil { + return false + } + _, ok := x.Value.(*LabelledPayload_DoubleValue) + return ok +} + +func (x *LabelledPayload) HasIntValue() bool { + if x == nil { + return false + } + _, ok := x.Value.(*LabelledPayload_IntValue) + return ok +} + +func (x *LabelledPayload) ClearValue() { + x.Value = nil +} + +func (x *LabelledPayload) ClearStringValue() { + if _, ok := x.Value.(*LabelledPayload_StringValue); ok { + x.Value = nil + } +} + +func (x *LabelledPayload) ClearBoolValue() { + if _, ok := x.Value.(*LabelledPayload_BoolValue); ok { + x.Value = nil + } +} + +func (x *LabelledPayload) ClearDoubleValue() { + if _, ok := x.Value.(*LabelledPayload_DoubleValue); ok { + x.Value = nil + } +} + +func (x *LabelledPayload) ClearIntValue() { + if _, ok := x.Value.(*LabelledPayload_IntValue); ok { + x.Value = nil + } +} + +const LabelledPayload_Value_not_set_case case_LabelledPayload_Value = 0 +const LabelledPayload_StringValue_case case_LabelledPayload_Value = 2 +const LabelledPayload_BoolValue_case case_LabelledPayload_Value = 3 +const LabelledPayload_DoubleValue_case case_LabelledPayload_Value = 4 +const LabelledPayload_IntValue_case case_LabelledPayload_Value = 5 + +func (x *LabelledPayload) WhichValue() case_LabelledPayload_Value { + if x == nil { + return LabelledPayload_Value_not_set_case + } + switch x.Value.(type) { + case *LabelledPayload_StringValue: + return LabelledPayload_StringValue_case + case *LabelledPayload_BoolValue: + return LabelledPayload_BoolValue_case + case *LabelledPayload_DoubleValue: + return LabelledPayload_DoubleValue_case + case *LabelledPayload_IntValue: + return LabelledPayload_IntValue_case + default: + return LabelledPayload_Value_not_set_case + } +} + +type LabelledPayload_builder struct { + _ [0]func() // Prevents comparability and use of unkeyed literals for the builder. + + // (Required) A human readable label for the value. + Label string + // (Required) A value which will be displayed to the user. + + // Fields of oneof Value: + StringValue *string + BoolValue *bool + DoubleValue *float64 + IntValue *int64 + // -- end of Value + // (Required) The key identifies the actual content of the metadata. + Key string + // (Required) The namespace describes the context that specified the key. + Namespace string +} + +func (b0 LabelledPayload_builder) Build() *LabelledPayload { + m0 := &LabelledPayload{} + b, x := &b0, m0 + _, _ = b, x + x.Label = b.Label + if b.StringValue != nil { + x.Value = &LabelledPayload_StringValue{*b.StringValue} + } + if b.BoolValue != nil { + x.Value = &LabelledPayload_BoolValue{*b.BoolValue} + } + if b.DoubleValue != nil { + x.Value = &LabelledPayload_DoubleValue{*b.DoubleValue} + } + if b.IntValue != nil { + x.Value = &LabelledPayload_IntValue{*b.IntValue} + } + x.Key = b.Key + x.Namespace = b.Namespace + return m0 +} + +type case_LabelledPayload_Value protoreflect.FieldNumber + +func (x case_LabelledPayload_Value) String() string { + md := file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_msgTypes[61].Descriptor() + if x == 0 { + return "not set" + } + return protoimpl.X.MessageFieldStringOf(md, protoreflect.FieldNumber(x)) +} + type isLabelledPayload_Value interface { isLabelledPayload_Value() } @@ -6141,26 +8416,23 @@ func (*LabelledPayload_IntValue) isLabelledPayload_Value() {} // useful for pipeline runners IOs and diagnostic dashboards to display details // about annotated components. type DisplayData struct { - state protoimpl.MessageState - sizeCache protoimpl.SizeCache - unknownFields protoimpl.UnknownFields - + state protoimpl.MessageState `protogen:"hybrid.v1"` // A key used to describe the type of display data. See StandardDisplayData // for the set of well known urns describing how the payload is meant to be // interpreted. Urn string `protobuf:"bytes,1,opt,name=urn,proto3" json:"urn,omitempty"` // (Optional) The data specifying any parameters to the URN. If // the URN does not require any arguments, this may be omitted. - Payload []byte `protobuf:"bytes,2,opt,name=payload,proto3" json:"payload,omitempty"` + Payload []byte `protobuf:"bytes,2,opt,name=payload,proto3" json:"payload,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache } func (x *DisplayData) Reset() { *x = DisplayData{} - if protoimpl.UnsafeEnabled { - mi := &file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_msgTypes[62] - ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) - ms.StoreMessageInfo(mi) - } + mi := &file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_msgTypes[62] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) } func (x *DisplayData) String() string { @@ -6171,7 +8443,7 @@ func (*DisplayData) ProtoMessage() {} func (x *DisplayData) ProtoReflect() protoreflect.Message { mi := &file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_msgTypes[62] - if protoimpl.UnsafeEnabled && x != nil { + if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { ms.StoreMessageInfo(mi) @@ -6181,11 +8453,6 @@ func (x *DisplayData) ProtoReflect() protoreflect.Message { return mi.MessageOf(x) } -// Deprecated: Use DisplayData.ProtoReflect.Descriptor instead. -func (*DisplayData) Descriptor() ([]byte, []int) { - return file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_rawDescGZIP(), []int{62} -} - func (x *DisplayData) GetUrn() string { if x != nil { return x.Urn @@ -6200,13 +8467,42 @@ func (x *DisplayData) GetPayload() []byte { return nil } +func (x *DisplayData) SetUrn(v string) { + x.Urn = v +} + +func (x *DisplayData) SetPayload(v []byte) { + if v == nil { + v = []byte{} + } + x.Payload = v +} + +type DisplayData_builder struct { + _ [0]func() // Prevents comparability and use of unkeyed literals for the builder. + + // A key used to describe the type of display data. See StandardDisplayData + // for the set of well known urns describing how the payload is meant to be + // interpreted. + Urn string + // (Optional) The data specifying any parameters to the URN. If + // the URN does not require any arguments, this may be omitted. + Payload []byte +} + +func (b0 DisplayData_builder) Build() *DisplayData { + m0 := &DisplayData{} + b, x := &b0, m0 + _, _ = b, x + x.Urn = b.Urn + x.Payload = b.Payload + return m0 +} + // A disjoint union of all the things that may contain references // that require Components to resolve. type MessageWithComponents struct { - state protoimpl.MessageState - sizeCache protoimpl.SizeCache - unknownFields protoimpl.UnknownFields - + state protoimpl.MessageState `protogen:"hybrid.v1"` // (Optional) The by-reference components of the root message, // enabling a standalone message. // @@ -6216,7 +8512,7 @@ type MessageWithComponents struct { // (Required) The root message that may contain pointers // that should be resolved by looking inside components. // - // Types that are assignable to Root: + // Types that are valid to be assigned to Root: // // *MessageWithComponents_Coder // *MessageWithComponents_CombinePayload @@ -6228,123 +8524,500 @@ type MessageWithComponents struct { // *MessageWithComponents_SideInput // *MessageWithComponents_WindowIntoPayload // *MessageWithComponents_WindowingStrategy - Root isMessageWithComponents_Root `protobuf_oneof:"root"` + Root isMessageWithComponents_Root `protobuf_oneof:"root"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *MessageWithComponents) Reset() { + *x = MessageWithComponents{} + mi := &file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_msgTypes[63] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *MessageWithComponents) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*MessageWithComponents) ProtoMessage() {} + +func (x *MessageWithComponents) ProtoReflect() protoreflect.Message { + mi := &file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_msgTypes[63] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +func (x *MessageWithComponents) GetComponents() *Components { + if x != nil { + return x.Components + } + return nil +} + +func (x *MessageWithComponents) GetRoot() isMessageWithComponents_Root { + if x != nil { + return x.Root + } + return nil +} + +func (x *MessageWithComponents) GetCoder() *Coder { + if x != nil { + if x, ok := x.Root.(*MessageWithComponents_Coder); ok { + return x.Coder + } + } + return nil +} + +func (x *MessageWithComponents) GetCombinePayload() *CombinePayload { + if x != nil { + if x, ok := x.Root.(*MessageWithComponents_CombinePayload); ok { + return x.CombinePayload + } + } + return nil +} + +func (x *MessageWithComponents) GetFunctionSpec() *FunctionSpec { + if x != nil { + if x, ok := x.Root.(*MessageWithComponents_FunctionSpec); ok { + return x.FunctionSpec + } + } + return nil +} + +func (x *MessageWithComponents) GetParDoPayload() *ParDoPayload { + if x != nil { + if x, ok := x.Root.(*MessageWithComponents_ParDoPayload); ok { + return x.ParDoPayload + } + } + return nil +} + +func (x *MessageWithComponents) GetPtransform() *PTransform { + if x != nil { + if x, ok := x.Root.(*MessageWithComponents_Ptransform); ok { + return x.Ptransform + } + } + return nil +} + +func (x *MessageWithComponents) GetPcollection() *PCollection { + if x != nil { + if x, ok := x.Root.(*MessageWithComponents_Pcollection); ok { + return x.Pcollection + } + } + return nil +} + +func (x *MessageWithComponents) GetReadPayload() *ReadPayload { + if x != nil { + if x, ok := x.Root.(*MessageWithComponents_ReadPayload); ok { + return x.ReadPayload + } + } + return nil +} + +func (x *MessageWithComponents) GetSideInput() *SideInput { + if x != nil { + if x, ok := x.Root.(*MessageWithComponents_SideInput); ok { + return x.SideInput + } + } + return nil +} + +func (x *MessageWithComponents) GetWindowIntoPayload() *WindowIntoPayload { + if x != nil { + if x, ok := x.Root.(*MessageWithComponents_WindowIntoPayload); ok { + return x.WindowIntoPayload + } + } + return nil +} + +func (x *MessageWithComponents) GetWindowingStrategy() *WindowingStrategy { + if x != nil { + if x, ok := x.Root.(*MessageWithComponents_WindowingStrategy); ok { + return x.WindowingStrategy + } + } + return nil +} + +func (x *MessageWithComponents) SetComponents(v *Components) { + x.Components = v +} + +func (x *MessageWithComponents) SetCoder(v *Coder) { + if v == nil { + x.Root = nil + return + } + x.Root = &MessageWithComponents_Coder{v} +} + +func (x *MessageWithComponents) SetCombinePayload(v *CombinePayload) { + if v == nil { + x.Root = nil + return + } + x.Root = &MessageWithComponents_CombinePayload{v} +} + +func (x *MessageWithComponents) SetFunctionSpec(v *FunctionSpec) { + if v == nil { + x.Root = nil + return + } + x.Root = &MessageWithComponents_FunctionSpec{v} +} + +func (x *MessageWithComponents) SetParDoPayload(v *ParDoPayload) { + if v == nil { + x.Root = nil + return + } + x.Root = &MessageWithComponents_ParDoPayload{v} +} + +func (x *MessageWithComponents) SetPtransform(v *PTransform) { + if v == nil { + x.Root = nil + return + } + x.Root = &MessageWithComponents_Ptransform{v} +} + +func (x *MessageWithComponents) SetPcollection(v *PCollection) { + if v == nil { + x.Root = nil + return + } + x.Root = &MessageWithComponents_Pcollection{v} +} + +func (x *MessageWithComponents) SetReadPayload(v *ReadPayload) { + if v == nil { + x.Root = nil + return + } + x.Root = &MessageWithComponents_ReadPayload{v} +} + +func (x *MessageWithComponents) SetSideInput(v *SideInput) { + if v == nil { + x.Root = nil + return + } + x.Root = &MessageWithComponents_SideInput{v} +} + +func (x *MessageWithComponents) SetWindowIntoPayload(v *WindowIntoPayload) { + if v == nil { + x.Root = nil + return + } + x.Root = &MessageWithComponents_WindowIntoPayload{v} +} + +func (x *MessageWithComponents) SetWindowingStrategy(v *WindowingStrategy) { + if v == nil { + x.Root = nil + return + } + x.Root = &MessageWithComponents_WindowingStrategy{v} +} + +func (x *MessageWithComponents) HasComponents() bool { + if x == nil { + return false + } + return x.Components != nil +} + +func (x *MessageWithComponents) HasRoot() bool { + if x == nil { + return false + } + return x.Root != nil +} + +func (x *MessageWithComponents) HasCoder() bool { + if x == nil { + return false + } + _, ok := x.Root.(*MessageWithComponents_Coder) + return ok +} + +func (x *MessageWithComponents) HasCombinePayload() bool { + if x == nil { + return false + } + _, ok := x.Root.(*MessageWithComponents_CombinePayload) + return ok +} + +func (x *MessageWithComponents) HasFunctionSpec() bool { + if x == nil { + return false + } + _, ok := x.Root.(*MessageWithComponents_FunctionSpec) + return ok +} + +func (x *MessageWithComponents) HasParDoPayload() bool { + if x == nil { + return false + } + _, ok := x.Root.(*MessageWithComponents_ParDoPayload) + return ok +} + +func (x *MessageWithComponents) HasPtransform() bool { + if x == nil { + return false + } + _, ok := x.Root.(*MessageWithComponents_Ptransform) + return ok +} + +func (x *MessageWithComponents) HasPcollection() bool { + if x == nil { + return false + } + _, ok := x.Root.(*MessageWithComponents_Pcollection) + return ok +} + +func (x *MessageWithComponents) HasReadPayload() bool { + if x == nil { + return false + } + _, ok := x.Root.(*MessageWithComponents_ReadPayload) + return ok +} + +func (x *MessageWithComponents) HasSideInput() bool { + if x == nil { + return false + } + _, ok := x.Root.(*MessageWithComponents_SideInput) + return ok } -func (x *MessageWithComponents) Reset() { - *x = MessageWithComponents{} - if protoimpl.UnsafeEnabled { - mi := &file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_msgTypes[63] - ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) - ms.StoreMessageInfo(mi) +func (x *MessageWithComponents) HasWindowIntoPayload() bool { + if x == nil { + return false } + _, ok := x.Root.(*MessageWithComponents_WindowIntoPayload) + return ok } -func (x *MessageWithComponents) String() string { - return protoimpl.X.MessageStringOf(x) +func (x *MessageWithComponents) HasWindowingStrategy() bool { + if x == nil { + return false + } + _, ok := x.Root.(*MessageWithComponents_WindowingStrategy) + return ok } -func (*MessageWithComponents) ProtoMessage() {} +func (x *MessageWithComponents) ClearComponents() { + x.Components = nil +} -func (x *MessageWithComponents) ProtoReflect() protoreflect.Message { - mi := &file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_msgTypes[63] - if protoimpl.UnsafeEnabled && x != nil { - ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) - if ms.LoadMessageInfo() == nil { - ms.StoreMessageInfo(mi) - } - return ms - } - return mi.MessageOf(x) +func (x *MessageWithComponents) ClearRoot() { + x.Root = nil } -// Deprecated: Use MessageWithComponents.ProtoReflect.Descriptor instead. -func (*MessageWithComponents) Descriptor() ([]byte, []int) { - return file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_rawDescGZIP(), []int{63} +func (x *MessageWithComponents) ClearCoder() { + if _, ok := x.Root.(*MessageWithComponents_Coder); ok { + x.Root = nil + } } -func (x *MessageWithComponents) GetComponents() *Components { - if x != nil { - return x.Components +func (x *MessageWithComponents) ClearCombinePayload() { + if _, ok := x.Root.(*MessageWithComponents_CombinePayload); ok { + x.Root = nil } - return nil } -func (m *MessageWithComponents) GetRoot() isMessageWithComponents_Root { - if m != nil { - return m.Root +func (x *MessageWithComponents) ClearFunctionSpec() { + if _, ok := x.Root.(*MessageWithComponents_FunctionSpec); ok { + x.Root = nil } - return nil } -func (x *MessageWithComponents) GetCoder() *Coder { - if x, ok := x.GetRoot().(*MessageWithComponents_Coder); ok { - return x.Coder +func (x *MessageWithComponents) ClearParDoPayload() { + if _, ok := x.Root.(*MessageWithComponents_ParDoPayload); ok { + x.Root = nil } - return nil } -func (x *MessageWithComponents) GetCombinePayload() *CombinePayload { - if x, ok := x.GetRoot().(*MessageWithComponents_CombinePayload); ok { - return x.CombinePayload +func (x *MessageWithComponents) ClearPtransform() { + if _, ok := x.Root.(*MessageWithComponents_Ptransform); ok { + x.Root = nil } - return nil } -func (x *MessageWithComponents) GetFunctionSpec() *FunctionSpec { - if x, ok := x.GetRoot().(*MessageWithComponents_FunctionSpec); ok { - return x.FunctionSpec +func (x *MessageWithComponents) ClearPcollection() { + if _, ok := x.Root.(*MessageWithComponents_Pcollection); ok { + x.Root = nil } - return nil } -func (x *MessageWithComponents) GetParDoPayload() *ParDoPayload { - if x, ok := x.GetRoot().(*MessageWithComponents_ParDoPayload); ok { - return x.ParDoPayload +func (x *MessageWithComponents) ClearReadPayload() { + if _, ok := x.Root.(*MessageWithComponents_ReadPayload); ok { + x.Root = nil } - return nil } -func (x *MessageWithComponents) GetPtransform() *PTransform { - if x, ok := x.GetRoot().(*MessageWithComponents_Ptransform); ok { - return x.Ptransform +func (x *MessageWithComponents) ClearSideInput() { + if _, ok := x.Root.(*MessageWithComponents_SideInput); ok { + x.Root = nil } - return nil } -func (x *MessageWithComponents) GetPcollection() *PCollection { - if x, ok := x.GetRoot().(*MessageWithComponents_Pcollection); ok { - return x.Pcollection +func (x *MessageWithComponents) ClearWindowIntoPayload() { + if _, ok := x.Root.(*MessageWithComponents_WindowIntoPayload); ok { + x.Root = nil } - return nil } -func (x *MessageWithComponents) GetReadPayload() *ReadPayload { - if x, ok := x.GetRoot().(*MessageWithComponents_ReadPayload); ok { - return x.ReadPayload +func (x *MessageWithComponents) ClearWindowingStrategy() { + if _, ok := x.Root.(*MessageWithComponents_WindowingStrategy); ok { + x.Root = nil } - return nil } -func (x *MessageWithComponents) GetSideInput() *SideInput { - if x, ok := x.GetRoot().(*MessageWithComponents_SideInput); ok { - return x.SideInput +const MessageWithComponents_Root_not_set_case case_MessageWithComponents_Root = 0 +const MessageWithComponents_Coder_case case_MessageWithComponents_Root = 2 +const MessageWithComponents_CombinePayload_case case_MessageWithComponents_Root = 3 +const MessageWithComponents_FunctionSpec_case case_MessageWithComponents_Root = 4 +const MessageWithComponents_ParDoPayload_case case_MessageWithComponents_Root = 6 +const MessageWithComponents_Ptransform_case case_MessageWithComponents_Root = 7 +const MessageWithComponents_Pcollection_case case_MessageWithComponents_Root = 8 +const MessageWithComponents_ReadPayload_case case_MessageWithComponents_Root = 9 +const MessageWithComponents_SideInput_case case_MessageWithComponents_Root = 11 +const MessageWithComponents_WindowIntoPayload_case case_MessageWithComponents_Root = 12 +const MessageWithComponents_WindowingStrategy_case case_MessageWithComponents_Root = 13 + +func (x *MessageWithComponents) WhichRoot() case_MessageWithComponents_Root { + if x == nil { + return MessageWithComponents_Root_not_set_case + } + switch x.Root.(type) { + case *MessageWithComponents_Coder: + return MessageWithComponents_Coder_case + case *MessageWithComponents_CombinePayload: + return MessageWithComponents_CombinePayload_case + case *MessageWithComponents_FunctionSpec: + return MessageWithComponents_FunctionSpec_case + case *MessageWithComponents_ParDoPayload: + return MessageWithComponents_ParDoPayload_case + case *MessageWithComponents_Ptransform: + return MessageWithComponents_Ptransform_case + case *MessageWithComponents_Pcollection: + return MessageWithComponents_Pcollection_case + case *MessageWithComponents_ReadPayload: + return MessageWithComponents_ReadPayload_case + case *MessageWithComponents_SideInput: + return MessageWithComponents_SideInput_case + case *MessageWithComponents_WindowIntoPayload: + return MessageWithComponents_WindowIntoPayload_case + case *MessageWithComponents_WindowingStrategy: + return MessageWithComponents_WindowingStrategy_case + default: + return MessageWithComponents_Root_not_set_case } - return nil } -func (x *MessageWithComponents) GetWindowIntoPayload() *WindowIntoPayload { - if x, ok := x.GetRoot().(*MessageWithComponents_WindowIntoPayload); ok { - return x.WindowIntoPayload +type MessageWithComponents_builder struct { + _ [0]func() // Prevents comparability and use of unkeyed literals for the builder. + + // (Optional) The by-reference components of the root message, + // enabling a standalone message. + // + // If this is absent, it is expected that there are no + // references. + Components *Components + // (Required) The root message that may contain pointers + // that should be resolved by looking inside components. + + // Fields of oneof Root: + Coder *Coder + CombinePayload *CombinePayload + FunctionSpec *FunctionSpec + ParDoPayload *ParDoPayload + Ptransform *PTransform + Pcollection *PCollection + ReadPayload *ReadPayload + SideInput *SideInput + WindowIntoPayload *WindowIntoPayload + WindowingStrategy *WindowingStrategy + // -- end of Root +} + +func (b0 MessageWithComponents_builder) Build() *MessageWithComponents { + m0 := &MessageWithComponents{} + b, x := &b0, m0 + _, _ = b, x + x.Components = b.Components + if b.Coder != nil { + x.Root = &MessageWithComponents_Coder{b.Coder} } - return nil + if b.CombinePayload != nil { + x.Root = &MessageWithComponents_CombinePayload{b.CombinePayload} + } + if b.FunctionSpec != nil { + x.Root = &MessageWithComponents_FunctionSpec{b.FunctionSpec} + } + if b.ParDoPayload != nil { + x.Root = &MessageWithComponents_ParDoPayload{b.ParDoPayload} + } + if b.Ptransform != nil { + x.Root = &MessageWithComponents_Ptransform{b.Ptransform} + } + if b.Pcollection != nil { + x.Root = &MessageWithComponents_Pcollection{b.Pcollection} + } + if b.ReadPayload != nil { + x.Root = &MessageWithComponents_ReadPayload{b.ReadPayload} + } + if b.SideInput != nil { + x.Root = &MessageWithComponents_SideInput{b.SideInput} + } + if b.WindowIntoPayload != nil { + x.Root = &MessageWithComponents_WindowIntoPayload{b.WindowIntoPayload} + } + if b.WindowingStrategy != nil { + x.Root = &MessageWithComponents_WindowingStrategy{b.WindowingStrategy} + } + return m0 } -func (x *MessageWithComponents) GetWindowingStrategy() *WindowingStrategy { - if x, ok := x.GetRoot().(*MessageWithComponents_WindowingStrategy); ok { - return x.WindowingStrategy +type case_MessageWithComponents_Root protoreflect.FieldNumber + +func (x case_MessageWithComponents_Root) String() string { + md := file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_msgTypes[63].Descriptor() + if x == 0 { + return "not set" } - return nil + return protoimpl.X.MessageFieldStringOf(md, protoreflect.FieldNumber(x)) } type isMessageWithComponents_Root interface { @@ -6414,10 +9087,7 @@ func (*MessageWithComponents_WindowingStrategy) isMessageWithComponents_Root() { // The payload for an executable stage. This will eventually be passed to an SDK in the form of a // ProcessBundleDescriptor. type ExecutableStagePayload struct { - state protoimpl.MessageState - sizeCache protoimpl.SizeCache - unknownFields protoimpl.UnknownFields - + state protoimpl.MessageState `protogen:"hybrid.v1"` // (Required) Environment in which this stage executes. // // We use an environment rather than environment id @@ -6449,15 +9119,15 @@ type ExecutableStagePayload struct { // The timerfamilies required for this executable stage. Each timer familyof each PTransform within // this ExecutableStagePayload must be represented within this field. TimerFamilies []*ExecutableStagePayload_TimerFamilyId `protobuf:"bytes,10,rep,name=timerFamilies,proto3" json:"timerFamilies,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache } func (x *ExecutableStagePayload) Reset() { *x = ExecutableStagePayload{} - if protoimpl.UnsafeEnabled { - mi := &file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_msgTypes[64] - ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) - ms.StoreMessageInfo(mi) - } + mi := &file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_msgTypes[64] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) } func (x *ExecutableStagePayload) String() string { @@ -6468,7 +9138,7 @@ func (*ExecutableStagePayload) ProtoMessage() {} func (x *ExecutableStagePayload) ProtoReflect() protoreflect.Message { mi := &file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_msgTypes[64] - if protoimpl.UnsafeEnabled && x != nil { + if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { ms.StoreMessageInfo(mi) @@ -6478,11 +9148,6 @@ func (x *ExecutableStagePayload) ProtoReflect() protoreflect.Message { return mi.MessageOf(x) } -// Deprecated: Use ExecutableStagePayload.ProtoReflect.Descriptor instead. -func (*ExecutableStagePayload) Descriptor() ([]byte, []int) { - return file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_rawDescGZIP(), []int{64} -} - func (x *ExecutableStagePayload) GetEnvironment() *Environment { if x != nil { return x.Environment @@ -6553,21 +9218,134 @@ func (x *ExecutableStagePayload) GetTimerFamilies() []*ExecutableStagePayload_Ti return nil } +func (x *ExecutableStagePayload) SetEnvironment(v *Environment) { + x.Environment = v +} + +func (x *ExecutableStagePayload) SetWireCoderSettings(v []*ExecutableStagePayload_WireCoderSetting) { + x.WireCoderSettings = v +} + +func (x *ExecutableStagePayload) SetInput(v string) { + x.Input = v +} + +func (x *ExecutableStagePayload) SetSideInputs(v []*ExecutableStagePayload_SideInputId) { + x.SideInputs = v +} + +func (x *ExecutableStagePayload) SetTransforms(v []string) { + x.Transforms = v +} + +func (x *ExecutableStagePayload) SetOutputs(v []string) { + x.Outputs = v +} + +func (x *ExecutableStagePayload) SetComponents(v *Components) { + x.Components = v +} + +func (x *ExecutableStagePayload) SetUserStates(v []*ExecutableStagePayload_UserStateId) { + x.UserStates = v +} + +func (x *ExecutableStagePayload) SetTimers(v []*ExecutableStagePayload_TimerId) { + x.Timers = v +} + +func (x *ExecutableStagePayload) SetTimerFamilies(v []*ExecutableStagePayload_TimerFamilyId) { + x.TimerFamilies = v +} + +func (x *ExecutableStagePayload) HasEnvironment() bool { + if x == nil { + return false + } + return x.Environment != nil +} + +func (x *ExecutableStagePayload) HasComponents() bool { + if x == nil { + return false + } + return x.Components != nil +} + +func (x *ExecutableStagePayload) ClearEnvironment() { + x.Environment = nil +} + +func (x *ExecutableStagePayload) ClearComponents() { + x.Components = nil +} + +type ExecutableStagePayload_builder struct { + _ [0]func() // Prevents comparability and use of unkeyed literals for the builder. + + // (Required) Environment in which this stage executes. + // + // We use an environment rather than environment id + // because ExecutableStages use environments directly. This may change in the future. + Environment *Environment + // The wire coder settings of this executable stage + WireCoderSettings []*ExecutableStagePayload_WireCoderSetting + // (Required) Input PCollection id. This must be present as a value in the inputs of any + // PTransform the ExecutableStagePayload is the payload of. + Input string + // The side inputs required for this executable stage. Each side input of each PTransform within + // this ExecutableStagePayload must be represented within this field. + SideInputs []*ExecutableStagePayload_SideInputId + // PTransform ids contained within this executable stage. This must contain at least one + // PTransform id. + Transforms []string + // Output PCollection ids. This must be equal to the values of the outputs of any + // PTransform the ExecutableStagePayload is the payload of. + Outputs []string + // (Required) The components for the Executable Stage. This must contain all of the Transforms + // in transforms, and the closure of all of the components they recognize. + Components *Components + // The user states required for this executable stage. Each user state of each PTransform within + // this ExecutableStagePayload must be represented within this field. + UserStates []*ExecutableStagePayload_UserStateId + // The timers required for this executable stage. Each timer of each PTransform within + // this ExecutableStagePayload must be represented within this field. + Timers []*ExecutableStagePayload_TimerId + // The timerfamilies required for this executable stage. Each timer familyof each PTransform within + // this ExecutableStagePayload must be represented within this field. + TimerFamilies []*ExecutableStagePayload_TimerFamilyId +} + +func (b0 ExecutableStagePayload_builder) Build() *ExecutableStagePayload { + m0 := &ExecutableStagePayload{} + b, x := &b0, m0 + _, _ = b, x + x.Environment = b.Environment + x.WireCoderSettings = b.WireCoderSettings + x.Input = b.Input + x.SideInputs = b.SideInputs + x.Transforms = b.Transforms + x.Outputs = b.Outputs + x.Components = b.Components + x.UserStates = b.UserStates + x.Timers = b.Timers + x.TimerFamilies = b.TimerFamilies + return m0 +} + // See https://beam.apache.org/documentation/runtime/resource-hints/ for additional documentation // on the behavior of StandardResourceHint. type StandardResourceHints struct { - state protoimpl.MessageState - sizeCache protoimpl.SizeCache + state protoimpl.MessageState `protogen:"hybrid.v1"` unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache } func (x *StandardResourceHints) Reset() { *x = StandardResourceHints{} - if protoimpl.UnsafeEnabled { - mi := &file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_msgTypes[65] - ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) - ms.StoreMessageInfo(mi) - } + mi := &file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_msgTypes[65] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) } func (x *StandardResourceHints) String() string { @@ -6578,7 +9356,7 @@ func (*StandardResourceHints) ProtoMessage() {} func (x *StandardResourceHints) ProtoReflect() protoreflect.Message { mi := &file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_msgTypes[65] - if protoimpl.UnsafeEnabled && x != nil { + if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { ms.StoreMessageInfo(mi) @@ -6588,31 +9366,35 @@ func (x *StandardResourceHints) ProtoReflect() protoreflect.Message { return mi.MessageOf(x) } -// Deprecated: Use StandardResourceHints.ProtoReflect.Descriptor instead. -func (*StandardResourceHints) Descriptor() ([]byte, []int) { - return file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_rawDescGZIP(), []int{65} +type StandardResourceHints_builder struct { + _ [0]func() // Prevents comparability and use of unkeyed literals for the builder. + } -type TestStreamPayload_Event struct { - state protoimpl.MessageState - sizeCache protoimpl.SizeCache - unknownFields protoimpl.UnknownFields +func (b0 StandardResourceHints_builder) Build() *StandardResourceHints { + m0 := &StandardResourceHints{} + b, x := &b0, m0 + _, _ = b, x + return m0 +} - // Types that are assignable to Event: +type TestStreamPayload_Event struct { + state protoimpl.MessageState `protogen:"hybrid.v1"` + // Types that are valid to be assigned to Event: // // *TestStreamPayload_Event_WatermarkEvent // *TestStreamPayload_Event_ProcessingTimeEvent // *TestStreamPayload_Event_ElementEvent - Event isTestStreamPayload_Event_Event `protobuf_oneof:"event"` + Event isTestStreamPayload_Event_Event `protobuf_oneof:"event"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache } func (x *TestStreamPayload_Event) Reset() { *x = TestStreamPayload_Event{} - if protoimpl.UnsafeEnabled { - mi := &file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_msgTypes[77] - ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) - ms.StoreMessageInfo(mi) - } + mi := &file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_msgTypes[77] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) } func (x *TestStreamPayload_Event) String() string { @@ -6623,7 +9405,7 @@ func (*TestStreamPayload_Event) ProtoMessage() {} func (x *TestStreamPayload_Event) ProtoReflect() protoreflect.Message { mi := &file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_msgTypes[77] - if protoimpl.UnsafeEnabled && x != nil { + if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { ms.StoreMessageInfo(mi) @@ -6633,39 +9415,174 @@ func (x *TestStreamPayload_Event) ProtoReflect() protoreflect.Message { return mi.MessageOf(x) } -// Deprecated: Use TestStreamPayload_Event.ProtoReflect.Descriptor instead. -func (*TestStreamPayload_Event) Descriptor() ([]byte, []int) { - return file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_rawDescGZIP(), []int{22, 0} -} - -func (m *TestStreamPayload_Event) GetEvent() isTestStreamPayload_Event_Event { - if m != nil { - return m.Event +func (x *TestStreamPayload_Event) GetEvent() isTestStreamPayload_Event_Event { + if x != nil { + return x.Event } return nil } func (x *TestStreamPayload_Event) GetWatermarkEvent() *TestStreamPayload_Event_AdvanceWatermark { - if x, ok := x.GetEvent().(*TestStreamPayload_Event_WatermarkEvent); ok { - return x.WatermarkEvent + if x != nil { + if x, ok := x.Event.(*TestStreamPayload_Event_WatermarkEvent); ok { + return x.WatermarkEvent + } } return nil } func (x *TestStreamPayload_Event) GetProcessingTimeEvent() *TestStreamPayload_Event_AdvanceProcessingTime { - if x, ok := x.GetEvent().(*TestStreamPayload_Event_ProcessingTimeEvent); ok { - return x.ProcessingTimeEvent + if x != nil { + if x, ok := x.Event.(*TestStreamPayload_Event_ProcessingTimeEvent); ok { + return x.ProcessingTimeEvent + } } return nil } func (x *TestStreamPayload_Event) GetElementEvent() *TestStreamPayload_Event_AddElements { - if x, ok := x.GetEvent().(*TestStreamPayload_Event_ElementEvent); ok { - return x.ElementEvent + if x != nil { + if x, ok := x.Event.(*TestStreamPayload_Event_ElementEvent); ok { + return x.ElementEvent + } } return nil } +func (x *TestStreamPayload_Event) SetWatermarkEvent(v *TestStreamPayload_Event_AdvanceWatermark) { + if v == nil { + x.Event = nil + return + } + x.Event = &TestStreamPayload_Event_WatermarkEvent{v} +} + +func (x *TestStreamPayload_Event) SetProcessingTimeEvent(v *TestStreamPayload_Event_AdvanceProcessingTime) { + if v == nil { + x.Event = nil + return + } + x.Event = &TestStreamPayload_Event_ProcessingTimeEvent{v} +} + +func (x *TestStreamPayload_Event) SetElementEvent(v *TestStreamPayload_Event_AddElements) { + if v == nil { + x.Event = nil + return + } + x.Event = &TestStreamPayload_Event_ElementEvent{v} +} + +func (x *TestStreamPayload_Event) HasEvent() bool { + if x == nil { + return false + } + return x.Event != nil +} + +func (x *TestStreamPayload_Event) HasWatermarkEvent() bool { + if x == nil { + return false + } + _, ok := x.Event.(*TestStreamPayload_Event_WatermarkEvent) + return ok +} + +func (x *TestStreamPayload_Event) HasProcessingTimeEvent() bool { + if x == nil { + return false + } + _, ok := x.Event.(*TestStreamPayload_Event_ProcessingTimeEvent) + return ok +} + +func (x *TestStreamPayload_Event) HasElementEvent() bool { + if x == nil { + return false + } + _, ok := x.Event.(*TestStreamPayload_Event_ElementEvent) + return ok +} + +func (x *TestStreamPayload_Event) ClearEvent() { + x.Event = nil +} + +func (x *TestStreamPayload_Event) ClearWatermarkEvent() { + if _, ok := x.Event.(*TestStreamPayload_Event_WatermarkEvent); ok { + x.Event = nil + } +} + +func (x *TestStreamPayload_Event) ClearProcessingTimeEvent() { + if _, ok := x.Event.(*TestStreamPayload_Event_ProcessingTimeEvent); ok { + x.Event = nil + } +} + +func (x *TestStreamPayload_Event) ClearElementEvent() { + if _, ok := x.Event.(*TestStreamPayload_Event_ElementEvent); ok { + x.Event = nil + } +} + +const TestStreamPayload_Event_Event_not_set_case case_TestStreamPayload_Event_Event = 0 +const TestStreamPayload_Event_WatermarkEvent_case case_TestStreamPayload_Event_Event = 1 +const TestStreamPayload_Event_ProcessingTimeEvent_case case_TestStreamPayload_Event_Event = 2 +const TestStreamPayload_Event_ElementEvent_case case_TestStreamPayload_Event_Event = 3 + +func (x *TestStreamPayload_Event) WhichEvent() case_TestStreamPayload_Event_Event { + if x == nil { + return TestStreamPayload_Event_Event_not_set_case + } + switch x.Event.(type) { + case *TestStreamPayload_Event_WatermarkEvent: + return TestStreamPayload_Event_WatermarkEvent_case + case *TestStreamPayload_Event_ProcessingTimeEvent: + return TestStreamPayload_Event_ProcessingTimeEvent_case + case *TestStreamPayload_Event_ElementEvent: + return TestStreamPayload_Event_ElementEvent_case + default: + return TestStreamPayload_Event_Event_not_set_case + } +} + +type TestStreamPayload_Event_builder struct { + _ [0]func() // Prevents comparability and use of unkeyed literals for the builder. + + // Fields of oneof Event: + WatermarkEvent *TestStreamPayload_Event_AdvanceWatermark + ProcessingTimeEvent *TestStreamPayload_Event_AdvanceProcessingTime + ElementEvent *TestStreamPayload_Event_AddElements + // -- end of Event +} + +func (b0 TestStreamPayload_Event_builder) Build() *TestStreamPayload_Event { + m0 := &TestStreamPayload_Event{} + b, x := &b0, m0 + _, _ = b, x + if b.WatermarkEvent != nil { + x.Event = &TestStreamPayload_Event_WatermarkEvent{b.WatermarkEvent} + } + if b.ProcessingTimeEvent != nil { + x.Event = &TestStreamPayload_Event_ProcessingTimeEvent{b.ProcessingTimeEvent} + } + if b.ElementEvent != nil { + x.Event = &TestStreamPayload_Event_ElementEvent{b.ElementEvent} + } + return m0 +} + +type case_TestStreamPayload_Event_Event protoreflect.FieldNumber + +func (x case_TestStreamPayload_Event_Event) String() string { + md := file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_msgTypes[77].Descriptor() + if x == 0 { + return "not set" + } + return protoimpl.X.MessageFieldStringOf(md, protoreflect.FieldNumber(x)) +} + type isTestStreamPayload_Event_Event interface { isTestStreamPayload_Event_Event() } @@ -6690,24 +9607,21 @@ func (*TestStreamPayload_Event_ElementEvent) isTestStreamPayload_Event_Event() { // A single element inside of the TestStream. type TestStreamPayload_TimestampedElement struct { - state protoimpl.MessageState - sizeCache protoimpl.SizeCache - unknownFields protoimpl.UnknownFields - + state protoimpl.MessageState `protogen:"hybrid.v1"` // (Required) The element encoded. Currently the TestStream only supports // encoding primitives. EncodedElement []byte `protobuf:"bytes,1,opt,name=encoded_element,json=encodedElement,proto3" json:"encoded_element,omitempty"` // (Required) The event timestamp in millisecond of this element. - Timestamp int64 `protobuf:"varint,2,opt,name=timestamp,proto3" json:"timestamp,omitempty"` + Timestamp int64 `protobuf:"varint,2,opt,name=timestamp,proto3" json:"timestamp,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache } func (x *TestStreamPayload_TimestampedElement) Reset() { *x = TestStreamPayload_TimestampedElement{} - if protoimpl.UnsafeEnabled { - mi := &file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_msgTypes[78] - ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) - ms.StoreMessageInfo(mi) - } + mi := &file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_msgTypes[78] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) } func (x *TestStreamPayload_TimestampedElement) String() string { @@ -6718,7 +9632,7 @@ func (*TestStreamPayload_TimestampedElement) ProtoMessage() {} func (x *TestStreamPayload_TimestampedElement) ProtoReflect() protoreflect.Message { mi := &file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_msgTypes[78] - if protoimpl.UnsafeEnabled && x != nil { + if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { ms.StoreMessageInfo(mi) @@ -6728,11 +9642,6 @@ func (x *TestStreamPayload_TimestampedElement) ProtoReflect() protoreflect.Messa return mi.MessageOf(x) } -// Deprecated: Use TestStreamPayload_TimestampedElement.ProtoReflect.Descriptor instead. -func (*TestStreamPayload_TimestampedElement) Descriptor() ([]byte, []int) { - return file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_rawDescGZIP(), []int{22, 1} -} - func (x *TestStreamPayload_TimestampedElement) GetEncodedElement() []byte { if x != nil { return x.EncodedElement @@ -6747,27 +9656,54 @@ func (x *TestStreamPayload_TimestampedElement) GetTimestamp() int64 { return 0 } +func (x *TestStreamPayload_TimestampedElement) SetEncodedElement(v []byte) { + if v == nil { + v = []byte{} + } + x.EncodedElement = v +} + +func (x *TestStreamPayload_TimestampedElement) SetTimestamp(v int64) { + x.Timestamp = v +} + +type TestStreamPayload_TimestampedElement_builder struct { + _ [0]func() // Prevents comparability and use of unkeyed literals for the builder. + + // (Required) The element encoded. Currently the TestStream only supports + // encoding primitives. + EncodedElement []byte + // (Required) The event timestamp in millisecond of this element. + Timestamp int64 +} + +func (b0 TestStreamPayload_TimestampedElement_builder) Build() *TestStreamPayload_TimestampedElement { + m0 := &TestStreamPayload_TimestampedElement{} + b, x := &b0, m0 + _, _ = b, x + x.EncodedElement = b.EncodedElement + x.Timestamp = b.Timestamp + return m0 +} + // Advances the watermark to the specified timestamp. type TestStreamPayload_Event_AdvanceWatermark struct { - state protoimpl.MessageState - sizeCache protoimpl.SizeCache - unknownFields protoimpl.UnknownFields - + state protoimpl.MessageState `protogen:"hybrid.v1"` // (Required) The watermark in millisecond to advance to. NewWatermark int64 `protobuf:"varint,1,opt,name=new_watermark,json=newWatermark,proto3" json:"new_watermark,omitempty"` // (Optional) The output watermark tag for a PCollection. If unspecified // or with an empty string, this will default to the Main PCollection // Output - Tag string `protobuf:"bytes,2,opt,name=tag,proto3" json:"tag,omitempty"` + Tag string `protobuf:"bytes,2,opt,name=tag,proto3" json:"tag,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache } func (x *TestStreamPayload_Event_AdvanceWatermark) Reset() { *x = TestStreamPayload_Event_AdvanceWatermark{} - if protoimpl.UnsafeEnabled { - mi := &file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_msgTypes[79] - ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) - ms.StoreMessageInfo(mi) - } + mi := &file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_msgTypes[79] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) } func (x *TestStreamPayload_Event_AdvanceWatermark) String() string { @@ -6778,7 +9714,7 @@ func (*TestStreamPayload_Event_AdvanceWatermark) ProtoMessage() {} func (x *TestStreamPayload_Event_AdvanceWatermark) ProtoReflect() protoreflect.Message { mi := &file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_msgTypes[79] - if protoimpl.UnsafeEnabled && x != nil { + if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { ms.StoreMessageInfo(mi) @@ -6788,11 +9724,6 @@ func (x *TestStreamPayload_Event_AdvanceWatermark) ProtoReflect() protoreflect.M return mi.MessageOf(x) } -// Deprecated: Use TestStreamPayload_Event_AdvanceWatermark.ProtoReflect.Descriptor instead. -func (*TestStreamPayload_Event_AdvanceWatermark) Descriptor() ([]byte, []int) { - return file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_rawDescGZIP(), []int{22, 0, 0} -} - func (x *TestStreamPayload_Event_AdvanceWatermark) GetNewWatermark() int64 { if x != nil { return x.NewWatermark @@ -6807,23 +9738,48 @@ func (x *TestStreamPayload_Event_AdvanceWatermark) GetTag() string { return "" } +func (x *TestStreamPayload_Event_AdvanceWatermark) SetNewWatermark(v int64) { + x.NewWatermark = v +} + +func (x *TestStreamPayload_Event_AdvanceWatermark) SetTag(v string) { + x.Tag = v +} + +type TestStreamPayload_Event_AdvanceWatermark_builder struct { + _ [0]func() // Prevents comparability and use of unkeyed literals for the builder. + + // (Required) The watermark in millisecond to advance to. + NewWatermark int64 + // (Optional) The output watermark tag for a PCollection. If unspecified + // or with an empty string, this will default to the Main PCollection + // Output + Tag string +} + +func (b0 TestStreamPayload_Event_AdvanceWatermark_builder) Build() *TestStreamPayload_Event_AdvanceWatermark { + m0 := &TestStreamPayload_Event_AdvanceWatermark{} + b, x := &b0, m0 + _, _ = b, x + x.NewWatermark = b.NewWatermark + x.Tag = b.Tag + return m0 +} + // Advances the processing time clock by the specified amount. type TestStreamPayload_Event_AdvanceProcessingTime struct { - state protoimpl.MessageState - sizeCache protoimpl.SizeCache - unknownFields protoimpl.UnknownFields - + state protoimpl.MessageState `protogen:"hybrid.v1"` // (Required) The duration in millisecond to advance by. AdvanceDuration int64 `protobuf:"varint,1,opt,name=advance_duration,json=advanceDuration,proto3" json:"advance_duration,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache } func (x *TestStreamPayload_Event_AdvanceProcessingTime) Reset() { *x = TestStreamPayload_Event_AdvanceProcessingTime{} - if protoimpl.UnsafeEnabled { - mi := &file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_msgTypes[80] - ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) - ms.StoreMessageInfo(mi) - } + mi := &file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_msgTypes[80] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) } func (x *TestStreamPayload_Event_AdvanceProcessingTime) String() string { @@ -6834,7 +9790,7 @@ func (*TestStreamPayload_Event_AdvanceProcessingTime) ProtoMessage() {} func (x *TestStreamPayload_Event_AdvanceProcessingTime) ProtoReflect() protoreflect.Message { mi := &file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_msgTypes[80] - if protoimpl.UnsafeEnabled && x != nil { + if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { ms.StoreMessageInfo(mi) @@ -6844,11 +9800,6 @@ func (x *TestStreamPayload_Event_AdvanceProcessingTime) ProtoReflect() protorefl return mi.MessageOf(x) } -// Deprecated: Use TestStreamPayload_Event_AdvanceProcessingTime.ProtoReflect.Descriptor instead. -func (*TestStreamPayload_Event_AdvanceProcessingTime) Descriptor() ([]byte, []int) { - return file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_rawDescGZIP(), []int{22, 0, 1} -} - func (x *TestStreamPayload_Event_AdvanceProcessingTime) GetAdvanceDuration() int64 { if x != nil { return x.AdvanceDuration @@ -6856,27 +9807,43 @@ func (x *TestStreamPayload_Event_AdvanceProcessingTime) GetAdvanceDuration() int return 0 } +func (x *TestStreamPayload_Event_AdvanceProcessingTime) SetAdvanceDuration(v int64) { + x.AdvanceDuration = v +} + +type TestStreamPayload_Event_AdvanceProcessingTime_builder struct { + _ [0]func() // Prevents comparability and use of unkeyed literals for the builder. + + // (Required) The duration in millisecond to advance by. + AdvanceDuration int64 +} + +func (b0 TestStreamPayload_Event_AdvanceProcessingTime_builder) Build() *TestStreamPayload_Event_AdvanceProcessingTime { + m0 := &TestStreamPayload_Event_AdvanceProcessingTime{} + b, x := &b0, m0 + _, _ = b, x + x.AdvanceDuration = b.AdvanceDuration + return m0 +} + // Adds elements to the stream to be emitted. type TestStreamPayload_Event_AddElements struct { - state protoimpl.MessageState - sizeCache protoimpl.SizeCache - unknownFields protoimpl.UnknownFields - + state protoimpl.MessageState `protogen:"hybrid.v1"` // (Required) The elements to add to the TestStream. Elements []*TestStreamPayload_TimestampedElement `protobuf:"bytes,1,rep,name=elements,proto3" json:"elements,omitempty"` // (Optional) The output PCollection tag to add these elements to. If // unspecified or with an empty string, this will default to the Main // PCollection Output. - Tag string `protobuf:"bytes,3,opt,name=tag,proto3" json:"tag,omitempty"` + Tag string `protobuf:"bytes,3,opt,name=tag,proto3" json:"tag,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache } func (x *TestStreamPayload_Event_AddElements) Reset() { *x = TestStreamPayload_Event_AddElements{} - if protoimpl.UnsafeEnabled { - mi := &file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_msgTypes[81] - ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) - ms.StoreMessageInfo(mi) - } + mi := &file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_msgTypes[81] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) } func (x *TestStreamPayload_Event_AddElements) String() string { @@ -6887,7 +9854,7 @@ func (*TestStreamPayload_Event_AddElements) ProtoMessage() {} func (x *TestStreamPayload_Event_AddElements) ProtoReflect() protoreflect.Message { mi := &file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_msgTypes[81] - if protoimpl.UnsafeEnabled && x != nil { + if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { ms.StoreMessageInfo(mi) @@ -6897,11 +9864,6 @@ func (x *TestStreamPayload_Event_AddElements) ProtoReflect() protoreflect.Messag return mi.MessageOf(x) } -// Deprecated: Use TestStreamPayload_Event_AddElements.ProtoReflect.Descriptor instead. -func (*TestStreamPayload_Event_AddElements) Descriptor() ([]byte, []int) { - return file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_rawDescGZIP(), []int{22, 0, 2} -} - func (x *TestStreamPayload_Event_AddElements) GetElements() []*TestStreamPayload_TimestampedElement { if x != nil { return x.Elements @@ -6916,22 +9878,47 @@ func (x *TestStreamPayload_Event_AddElements) GetTag() string { return "" } +func (x *TestStreamPayload_Event_AddElements) SetElements(v []*TestStreamPayload_TimestampedElement) { + x.Elements = v +} + +func (x *TestStreamPayload_Event_AddElements) SetTag(v string) { + x.Tag = v +} + +type TestStreamPayload_Event_AddElements_builder struct { + _ [0]func() // Prevents comparability and use of unkeyed literals for the builder. + + // (Required) The elements to add to the TestStream. + Elements []*TestStreamPayload_TimestampedElement + // (Optional) The output PCollection tag to add these elements to. If + // unspecified or with an empty string, this will default to the Main + // PCollection Output. + Tag string +} + +func (b0 TestStreamPayload_Event_AddElements_builder) Build() *TestStreamPayload_Event_AddElements { + m0 := &TestStreamPayload_Event_AddElements{} + b, x := &b0, m0 + _, _ = b, x + x.Elements = b.Elements + x.Tag = b.Tag + return m0 +} + // Ready when all subtriggers are ready. type Trigger_AfterAll struct { - state protoimpl.MessageState - sizeCache protoimpl.SizeCache + state protoimpl.MessageState `protogen:"hybrid.v1"` + Subtriggers []*Trigger `protobuf:"bytes,1,rep,name=subtriggers,proto3" json:"subtriggers,omitempty"` unknownFields protoimpl.UnknownFields - - Subtriggers []*Trigger `protobuf:"bytes,1,rep,name=subtriggers,proto3" json:"subtriggers,omitempty"` + sizeCache protoimpl.SizeCache } func (x *Trigger_AfterAll) Reset() { *x = Trigger_AfterAll{} - if protoimpl.UnsafeEnabled { - mi := &file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_msgTypes[83] - ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) - ms.StoreMessageInfo(mi) - } + mi := &file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_msgTypes[83] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) } func (x *Trigger_AfterAll) String() string { @@ -6942,7 +9929,7 @@ func (*Trigger_AfterAll) ProtoMessage() {} func (x *Trigger_AfterAll) ProtoReflect() protoreflect.Message { mi := &file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_msgTypes[83] - if protoimpl.UnsafeEnabled && x != nil { + if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { ms.StoreMessageInfo(mi) @@ -6952,11 +9939,6 @@ func (x *Trigger_AfterAll) ProtoReflect() protoreflect.Message { return mi.MessageOf(x) } -// Deprecated: Use Trigger_AfterAll.ProtoReflect.Descriptor instead. -func (*Trigger_AfterAll) Descriptor() ([]byte, []int) { - return file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_rawDescGZIP(), []int{38, 0} -} - func (x *Trigger_AfterAll) GetSubtriggers() []*Trigger { if x != nil { return x.Subtriggers @@ -6964,22 +9946,37 @@ func (x *Trigger_AfterAll) GetSubtriggers() []*Trigger { return nil } +func (x *Trigger_AfterAll) SetSubtriggers(v []*Trigger) { + x.Subtriggers = v +} + +type Trigger_AfterAll_builder struct { + _ [0]func() // Prevents comparability and use of unkeyed literals for the builder. + + Subtriggers []*Trigger +} + +func (b0 Trigger_AfterAll_builder) Build() *Trigger_AfterAll { + m0 := &Trigger_AfterAll{} + b, x := &b0, m0 + _, _ = b, x + x.Subtriggers = b.Subtriggers + return m0 +} + // Ready when any subtrigger is ready. type Trigger_AfterAny struct { - state protoimpl.MessageState - sizeCache protoimpl.SizeCache + state protoimpl.MessageState `protogen:"hybrid.v1"` + Subtriggers []*Trigger `protobuf:"bytes,1,rep,name=subtriggers,proto3" json:"subtriggers,omitempty"` unknownFields protoimpl.UnknownFields - - Subtriggers []*Trigger `protobuf:"bytes,1,rep,name=subtriggers,proto3" json:"subtriggers,omitempty"` + sizeCache protoimpl.SizeCache } func (x *Trigger_AfterAny) Reset() { *x = Trigger_AfterAny{} - if protoimpl.UnsafeEnabled { - mi := &file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_msgTypes[84] - ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) - ms.StoreMessageInfo(mi) - } + mi := &file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_msgTypes[84] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) } func (x *Trigger_AfterAny) String() string { @@ -6990,7 +9987,7 @@ func (*Trigger_AfterAny) ProtoMessage() {} func (x *Trigger_AfterAny) ProtoReflect() protoreflect.Message { mi := &file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_msgTypes[84] - if protoimpl.UnsafeEnabled && x != nil { + if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { ms.StoreMessageInfo(mi) @@ -7000,11 +9997,6 @@ func (x *Trigger_AfterAny) ProtoReflect() protoreflect.Message { return mi.MessageOf(x) } -// Deprecated: Use Trigger_AfterAny.ProtoReflect.Descriptor instead. -func (*Trigger_AfterAny) Descriptor() ([]byte, []int) { - return file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_rawDescGZIP(), []int{38, 1} -} - func (x *Trigger_AfterAny) GetSubtriggers() []*Trigger { if x != nil { return x.Subtriggers @@ -7012,23 +10004,38 @@ func (x *Trigger_AfterAny) GetSubtriggers() []*Trigger { return nil } +func (x *Trigger_AfterAny) SetSubtriggers(v []*Trigger) { + x.Subtriggers = v +} + +type Trigger_AfterAny_builder struct { + _ [0]func() // Prevents comparability and use of unkeyed literals for the builder. + + Subtriggers []*Trigger +} + +func (b0 Trigger_AfterAny_builder) Build() *Trigger_AfterAny { + m0 := &Trigger_AfterAny{} + b, x := &b0, m0 + _, _ = b, x + x.Subtriggers = b.Subtriggers + return m0 +} + // Starting with the first subtrigger, ready when the _current_ subtrigger // is ready. After output, advances the current trigger by one. type Trigger_AfterEach struct { - state protoimpl.MessageState - sizeCache protoimpl.SizeCache + state protoimpl.MessageState `protogen:"hybrid.v1"` + Subtriggers []*Trigger `protobuf:"bytes,1,rep,name=subtriggers,proto3" json:"subtriggers,omitempty"` unknownFields protoimpl.UnknownFields - - Subtriggers []*Trigger `protobuf:"bytes,1,rep,name=subtriggers,proto3" json:"subtriggers,omitempty"` + sizeCache protoimpl.SizeCache } func (x *Trigger_AfterEach) Reset() { *x = Trigger_AfterEach{} - if protoimpl.UnsafeEnabled { - mi := &file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_msgTypes[85] - ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) - ms.StoreMessageInfo(mi) - } + mi := &file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_msgTypes[85] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) } func (x *Trigger_AfterEach) String() string { @@ -7039,7 +10046,7 @@ func (*Trigger_AfterEach) ProtoMessage() {} func (x *Trigger_AfterEach) ProtoReflect() protoreflect.Message { mi := &file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_msgTypes[85] - if protoimpl.UnsafeEnabled && x != nil { + if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { ms.StoreMessageInfo(mi) @@ -7049,11 +10056,6 @@ func (x *Trigger_AfterEach) ProtoReflect() protoreflect.Message { return mi.MessageOf(x) } -// Deprecated: Use Trigger_AfterEach.ProtoReflect.Descriptor instead. -func (*Trigger_AfterEach) Descriptor() ([]byte, []int) { - return file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_rawDescGZIP(), []int{38, 2} -} - func (x *Trigger_AfterEach) GetSubtriggers() []*Trigger { if x != nil { return x.Subtriggers @@ -7061,29 +10063,44 @@ func (x *Trigger_AfterEach) GetSubtriggers() []*Trigger { return nil } +func (x *Trigger_AfterEach) SetSubtriggers(v []*Trigger) { + x.Subtriggers = v +} + +type Trigger_AfterEach_builder struct { + _ [0]func() // Prevents comparability and use of unkeyed literals for the builder. + + Subtriggers []*Trigger +} + +func (b0 Trigger_AfterEach_builder) Build() *Trigger_AfterEach { + m0 := &Trigger_AfterEach{} + b, x := &b0, m0 + _, _ = b, x + x.Subtriggers = b.Subtriggers + return m0 +} + // Ready after the input watermark is past the end of the window. // // May have implicitly-repeated subtriggers for early and late firings. // When the end of the window is reached, the trigger transitions between // the subtriggers. type Trigger_AfterEndOfWindow struct { - state protoimpl.MessageState - sizeCache protoimpl.SizeCache - unknownFields protoimpl.UnknownFields - + state protoimpl.MessageState `protogen:"hybrid.v1"` // (Optional) A trigger governing output prior to the end of the window. EarlyFirings *Trigger `protobuf:"bytes,1,opt,name=early_firings,json=earlyFirings,proto3" json:"early_firings,omitempty"` // (Optional) A trigger governing output after the end of the window. - LateFirings *Trigger `protobuf:"bytes,2,opt,name=late_firings,json=lateFirings,proto3" json:"late_firings,omitempty"` + LateFirings *Trigger `protobuf:"bytes,2,opt,name=late_firings,json=lateFirings,proto3" json:"late_firings,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache } func (x *Trigger_AfterEndOfWindow) Reset() { *x = Trigger_AfterEndOfWindow{} - if protoimpl.UnsafeEnabled { - mi := &file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_msgTypes[86] - ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) - ms.StoreMessageInfo(mi) - } + mi := &file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_msgTypes[86] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) } func (x *Trigger_AfterEndOfWindow) String() string { @@ -7094,7 +10111,7 @@ func (*Trigger_AfterEndOfWindow) ProtoMessage() {} func (x *Trigger_AfterEndOfWindow) ProtoReflect() protoreflect.Message { mi := &file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_msgTypes[86] - if protoimpl.UnsafeEnabled && x != nil { + if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { ms.StoreMessageInfo(mi) @@ -7104,11 +10121,6 @@ func (x *Trigger_AfterEndOfWindow) ProtoReflect() protoreflect.Message { return mi.MessageOf(x) } -// Deprecated: Use Trigger_AfterEndOfWindow.ProtoReflect.Descriptor instead. -func (*Trigger_AfterEndOfWindow) Descriptor() ([]byte, []int) { - return file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_rawDescGZIP(), []int{38, 3} -} - func (x *Trigger_AfterEndOfWindow) GetEarlyFirings() *Trigger { if x != nil { return x.EarlyFirings @@ -7123,24 +10135,69 @@ func (x *Trigger_AfterEndOfWindow) GetLateFirings() *Trigger { return nil } +func (x *Trigger_AfterEndOfWindow) SetEarlyFirings(v *Trigger) { + x.EarlyFirings = v +} + +func (x *Trigger_AfterEndOfWindow) SetLateFirings(v *Trigger) { + x.LateFirings = v +} + +func (x *Trigger_AfterEndOfWindow) HasEarlyFirings() bool { + if x == nil { + return false + } + return x.EarlyFirings != nil +} + +func (x *Trigger_AfterEndOfWindow) HasLateFirings() bool { + if x == nil { + return false + } + return x.LateFirings != nil +} + +func (x *Trigger_AfterEndOfWindow) ClearEarlyFirings() { + x.EarlyFirings = nil +} + +func (x *Trigger_AfterEndOfWindow) ClearLateFirings() { + x.LateFirings = nil +} + +type Trigger_AfterEndOfWindow_builder struct { + _ [0]func() // Prevents comparability and use of unkeyed literals for the builder. + + // (Optional) A trigger governing output prior to the end of the window. + EarlyFirings *Trigger + // (Optional) A trigger governing output after the end of the window. + LateFirings *Trigger +} + +func (b0 Trigger_AfterEndOfWindow_builder) Build() *Trigger_AfterEndOfWindow { + m0 := &Trigger_AfterEndOfWindow{} + b, x := &b0, m0 + _, _ = b, x + x.EarlyFirings = b.EarlyFirings + x.LateFirings = b.LateFirings + return m0 +} + // After input arrives, ready when the specified delay has passed. type Trigger_AfterProcessingTime struct { - state protoimpl.MessageState - sizeCache protoimpl.SizeCache - unknownFields protoimpl.UnknownFields - + state protoimpl.MessageState `protogen:"hybrid.v1"` // (Required) The transforms to apply to an arriving element's timestamp, // in order TimestampTransforms []*TimestampTransform `protobuf:"bytes,1,rep,name=timestamp_transforms,json=timestampTransforms,proto3" json:"timestamp_transforms,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache } func (x *Trigger_AfterProcessingTime) Reset() { *x = Trigger_AfterProcessingTime{} - if protoimpl.UnsafeEnabled { - mi := &file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_msgTypes[87] - ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) - ms.StoreMessageInfo(mi) - } + mi := &file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_msgTypes[87] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) } func (x *Trigger_AfterProcessingTime) String() string { @@ -7151,7 +10208,7 @@ func (*Trigger_AfterProcessingTime) ProtoMessage() {} func (x *Trigger_AfterProcessingTime) ProtoReflect() protoreflect.Message { mi := &file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_msgTypes[87] - if protoimpl.UnsafeEnabled && x != nil { + if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { ms.StoreMessageInfo(mi) @@ -7161,11 +10218,6 @@ func (x *Trigger_AfterProcessingTime) ProtoReflect() protoreflect.Message { return mi.MessageOf(x) } -// Deprecated: Use Trigger_AfterProcessingTime.ProtoReflect.Descriptor instead. -func (*Trigger_AfterProcessingTime) Descriptor() ([]byte, []int) { - return file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_rawDescGZIP(), []int{38, 4} -} - func (x *Trigger_AfterProcessingTime) GetTimestampTransforms() []*TimestampTransform { if x != nil { return x.TimestampTransforms @@ -7173,21 +10225,39 @@ func (x *Trigger_AfterProcessingTime) GetTimestampTransforms() []*TimestampTrans return nil } +func (x *Trigger_AfterProcessingTime) SetTimestampTransforms(v []*TimestampTransform) { + x.TimestampTransforms = v +} + +type Trigger_AfterProcessingTime_builder struct { + _ [0]func() // Prevents comparability and use of unkeyed literals for the builder. + + // (Required) The transforms to apply to an arriving element's timestamp, + // in order + TimestampTransforms []*TimestampTransform +} + +func (b0 Trigger_AfterProcessingTime_builder) Build() *Trigger_AfterProcessingTime { + m0 := &Trigger_AfterProcessingTime{} + b, x := &b0, m0 + _, _ = b, x + x.TimestampTransforms = b.TimestampTransforms + return m0 +} + // Ready whenever upstream processing time has all caught up with // the arrival time of an input element type Trigger_AfterSynchronizedProcessingTime struct { - state protoimpl.MessageState - sizeCache protoimpl.SizeCache + state protoimpl.MessageState `protogen:"hybrid.v1"` unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache } func (x *Trigger_AfterSynchronizedProcessingTime) Reset() { *x = Trigger_AfterSynchronizedProcessingTime{} - if protoimpl.UnsafeEnabled { - mi := &file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_msgTypes[88] - ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) - ms.StoreMessageInfo(mi) - } + mi := &file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_msgTypes[88] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) } func (x *Trigger_AfterSynchronizedProcessingTime) String() string { @@ -7198,7 +10268,7 @@ func (*Trigger_AfterSynchronizedProcessingTime) ProtoMessage() {} func (x *Trigger_AfterSynchronizedProcessingTime) ProtoReflect() protoreflect.Message { mi := &file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_msgTypes[88] - if protoimpl.UnsafeEnabled && x != nil { + if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { ms.StoreMessageInfo(mi) @@ -7208,26 +10278,31 @@ func (x *Trigger_AfterSynchronizedProcessingTime) ProtoReflect() protoreflect.Me return mi.MessageOf(x) } -// Deprecated: Use Trigger_AfterSynchronizedProcessingTime.ProtoReflect.Descriptor instead. -func (*Trigger_AfterSynchronizedProcessingTime) Descriptor() ([]byte, []int) { - return file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_rawDescGZIP(), []int{38, 5} +type Trigger_AfterSynchronizedProcessingTime_builder struct { + _ [0]func() // Prevents comparability and use of unkeyed literals for the builder. + +} + +func (b0 Trigger_AfterSynchronizedProcessingTime_builder) Build() *Trigger_AfterSynchronizedProcessingTime { + m0 := &Trigger_AfterSynchronizedProcessingTime{} + b, x := &b0, m0 + _, _ = b, x + return m0 } // The default trigger. Equivalent to Repeat { AfterEndOfWindow } but // specially denoted to indicate the user did not alter the triggering. type Trigger_Default struct { - state protoimpl.MessageState - sizeCache protoimpl.SizeCache + state protoimpl.MessageState `protogen:"hybrid.v1"` unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache } func (x *Trigger_Default) Reset() { *x = Trigger_Default{} - if protoimpl.UnsafeEnabled { - mi := &file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_msgTypes[89] - ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) - ms.StoreMessageInfo(mi) - } + mi := &file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_msgTypes[89] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) } func (x *Trigger_Default) String() string { @@ -7238,7 +10313,7 @@ func (*Trigger_Default) ProtoMessage() {} func (x *Trigger_Default) ProtoReflect() protoreflect.Message { mi := &file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_msgTypes[89] - if protoimpl.UnsafeEnabled && x != nil { + if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { ms.StoreMessageInfo(mi) @@ -7248,27 +10323,31 @@ func (x *Trigger_Default) ProtoReflect() protoreflect.Message { return mi.MessageOf(x) } -// Deprecated: Use Trigger_Default.ProtoReflect.Descriptor instead. -func (*Trigger_Default) Descriptor() ([]byte, []int) { - return file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_rawDescGZIP(), []int{38, 6} +type Trigger_Default_builder struct { + _ [0]func() // Prevents comparability and use of unkeyed literals for the builder. + +} + +func (b0 Trigger_Default_builder) Build() *Trigger_Default { + m0 := &Trigger_Default{} + b, x := &b0, m0 + _, _ = b, x + return m0 } // Ready whenever the requisite number of input elements have arrived type Trigger_ElementCount struct { - state protoimpl.MessageState - sizeCache protoimpl.SizeCache + state protoimpl.MessageState `protogen:"hybrid.v1"` + ElementCount int32 `protobuf:"varint,1,opt,name=element_count,json=elementCount,proto3" json:"element_count,omitempty"` unknownFields protoimpl.UnknownFields - - ElementCount int32 `protobuf:"varint,1,opt,name=element_count,json=elementCount,proto3" json:"element_count,omitempty"` + sizeCache protoimpl.SizeCache } func (x *Trigger_ElementCount) Reset() { *x = Trigger_ElementCount{} - if protoimpl.UnsafeEnabled { - mi := &file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_msgTypes[90] - ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) - ms.StoreMessageInfo(mi) - } + mi := &file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_msgTypes[90] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) } func (x *Trigger_ElementCount) String() string { @@ -7279,7 +10358,7 @@ func (*Trigger_ElementCount) ProtoMessage() {} func (x *Trigger_ElementCount) ProtoReflect() protoreflect.Message { mi := &file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_msgTypes[90] - if protoimpl.UnsafeEnabled && x != nil { + if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { ms.StoreMessageInfo(mi) @@ -7289,11 +10368,6 @@ func (x *Trigger_ElementCount) ProtoReflect() protoreflect.Message { return mi.MessageOf(x) } -// Deprecated: Use Trigger_ElementCount.ProtoReflect.Descriptor instead. -func (*Trigger_ElementCount) Descriptor() ([]byte, []int) { - return file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_rawDescGZIP(), []int{38, 7} -} - func (x *Trigger_ElementCount) GetElementCount() int32 { if x != nil { return x.ElementCount @@ -7301,21 +10375,37 @@ func (x *Trigger_ElementCount) GetElementCount() int32 { return 0 } +func (x *Trigger_ElementCount) SetElementCount(v int32) { + x.ElementCount = v +} + +type Trigger_ElementCount_builder struct { + _ [0]func() // Prevents comparability and use of unkeyed literals for the builder. + + ElementCount int32 +} + +func (b0 Trigger_ElementCount_builder) Build() *Trigger_ElementCount { + m0 := &Trigger_ElementCount{} + b, x := &b0, m0 + _, _ = b, x + x.ElementCount = b.ElementCount + return m0 +} + // Never ready. There will only be an ON_TIME output and a final // output at window expiration. type Trigger_Never struct { - state protoimpl.MessageState - sizeCache protoimpl.SizeCache + state protoimpl.MessageState `protogen:"hybrid.v1"` unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache } func (x *Trigger_Never) Reset() { *x = Trigger_Never{} - if protoimpl.UnsafeEnabled { - mi := &file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_msgTypes[91] - ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) - ms.StoreMessageInfo(mi) - } + mi := &file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_msgTypes[91] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) } func (x *Trigger_Never) String() string { @@ -7326,7 +10416,7 @@ func (*Trigger_Never) ProtoMessage() {} func (x *Trigger_Never) ProtoReflect() protoreflect.Message { mi := &file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_msgTypes[91] - if protoimpl.UnsafeEnabled && x != nil { + if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { ms.StoreMessageInfo(mi) @@ -7336,26 +10426,31 @@ func (x *Trigger_Never) ProtoReflect() protoreflect.Message { return mi.MessageOf(x) } -// Deprecated: Use Trigger_Never.ProtoReflect.Descriptor instead. -func (*Trigger_Never) Descriptor() ([]byte, []int) { - return file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_rawDescGZIP(), []int{38, 8} +type Trigger_Never_builder struct { + _ [0]func() // Prevents comparability and use of unkeyed literals for the builder. + +} + +func (b0 Trigger_Never_builder) Build() *Trigger_Never { + m0 := &Trigger_Never{} + b, x := &b0, m0 + _, _ = b, x + return m0 } // Always ready. This can also be expressed as ElementCount(1) but // is more explicit. type Trigger_Always struct { - state protoimpl.MessageState - sizeCache protoimpl.SizeCache + state protoimpl.MessageState `protogen:"hybrid.v1"` unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache } func (x *Trigger_Always) Reset() { *x = Trigger_Always{} - if protoimpl.UnsafeEnabled { - mi := &file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_msgTypes[92] - ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) - ms.StoreMessageInfo(mi) - } + mi := &file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_msgTypes[92] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) } func (x *Trigger_Always) String() string { @@ -7366,7 +10461,7 @@ func (*Trigger_Always) ProtoMessage() {} func (x *Trigger_Always) ProtoReflect() protoreflect.Message { mi := &file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_msgTypes[92] - if protoimpl.UnsafeEnabled && x != nil { + if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { ms.StoreMessageInfo(mi) @@ -7376,31 +10471,35 @@ func (x *Trigger_Always) ProtoReflect() protoreflect.Message { return mi.MessageOf(x) } -// Deprecated: Use Trigger_Always.ProtoReflect.Descriptor instead. -func (*Trigger_Always) Descriptor() ([]byte, []int) { - return file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_rawDescGZIP(), []int{38, 9} +type Trigger_Always_builder struct { + _ [0]func() // Prevents comparability and use of unkeyed literals for the builder. + +} + +func (b0 Trigger_Always_builder) Build() *Trigger_Always { + m0 := &Trigger_Always{} + b, x := &b0, m0 + _, _ = b, x + return m0 } // Ready whenever either of its subtriggers are ready, but finishes output // when the finally subtrigger fires. type Trigger_OrFinally struct { - state protoimpl.MessageState - sizeCache protoimpl.SizeCache - unknownFields protoimpl.UnknownFields - + state protoimpl.MessageState `protogen:"hybrid.v1"` // (Required) Trigger governing main output; may fire repeatedly. Main *Trigger `protobuf:"bytes,1,opt,name=main,proto3" json:"main,omitempty"` // (Required) Trigger governing termination of output. - Finally *Trigger `protobuf:"bytes,2,opt,name=finally,proto3" json:"finally,omitempty"` + Finally *Trigger `protobuf:"bytes,2,opt,name=finally,proto3" json:"finally,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache } func (x *Trigger_OrFinally) Reset() { *x = Trigger_OrFinally{} - if protoimpl.UnsafeEnabled { - mi := &file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_msgTypes[93] - ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) - ms.StoreMessageInfo(mi) - } + mi := &file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_msgTypes[93] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) } func (x *Trigger_OrFinally) String() string { @@ -7411,7 +10510,7 @@ func (*Trigger_OrFinally) ProtoMessage() {} func (x *Trigger_OrFinally) ProtoReflect() protoreflect.Message { mi := &file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_msgTypes[93] - if protoimpl.UnsafeEnabled && x != nil { + if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { ms.StoreMessageInfo(mi) @@ -7421,11 +10520,6 @@ func (x *Trigger_OrFinally) ProtoReflect() protoreflect.Message { return mi.MessageOf(x) } -// Deprecated: Use Trigger_OrFinally.ProtoReflect.Descriptor instead. -func (*Trigger_OrFinally) Descriptor() ([]byte, []int) { - return file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_rawDescGZIP(), []int{38, 10} -} - func (x *Trigger_OrFinally) GetMain() *Trigger { if x != nil { return x.Main @@ -7440,24 +10534,69 @@ func (x *Trigger_OrFinally) GetFinally() *Trigger { return nil } +func (x *Trigger_OrFinally) SetMain(v *Trigger) { + x.Main = v +} + +func (x *Trigger_OrFinally) SetFinally(v *Trigger) { + x.Finally = v +} + +func (x *Trigger_OrFinally) HasMain() bool { + if x == nil { + return false + } + return x.Main != nil +} + +func (x *Trigger_OrFinally) HasFinally() bool { + if x == nil { + return false + } + return x.Finally != nil +} + +func (x *Trigger_OrFinally) ClearMain() { + x.Main = nil +} + +func (x *Trigger_OrFinally) ClearFinally() { + x.Finally = nil +} + +type Trigger_OrFinally_builder struct { + _ [0]func() // Prevents comparability and use of unkeyed literals for the builder. + + // (Required) Trigger governing main output; may fire repeatedly. + Main *Trigger + // (Required) Trigger governing termination of output. + Finally *Trigger +} + +func (b0 Trigger_OrFinally_builder) Build() *Trigger_OrFinally { + m0 := &Trigger_OrFinally{} + b, x := &b0, m0 + _, _ = b, x + x.Main = b.Main + x.Finally = b.Finally + return m0 +} + // Ready whenever the subtrigger is ready; resets state when the subtrigger // completes. type Trigger_Repeat struct { - state protoimpl.MessageState - sizeCache protoimpl.SizeCache - unknownFields protoimpl.UnknownFields - + state protoimpl.MessageState `protogen:"hybrid.v1"` // (Require) Trigger that is run repeatedly. - Subtrigger *Trigger `protobuf:"bytes,1,opt,name=subtrigger,proto3" json:"subtrigger,omitempty"` + Subtrigger *Trigger `protobuf:"bytes,1,opt,name=subtrigger,proto3" json:"subtrigger,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache } func (x *Trigger_Repeat) Reset() { *x = Trigger_Repeat{} - if protoimpl.UnsafeEnabled { - mi := &file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_msgTypes[94] - ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) - ms.StoreMessageInfo(mi) - } + mi := &file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_msgTypes[94] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) } func (x *Trigger_Repeat) String() string { @@ -7468,7 +10607,7 @@ func (*Trigger_Repeat) ProtoMessage() {} func (x *Trigger_Repeat) ProtoReflect() protoreflect.Message { mi := &file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_msgTypes[94] - if protoimpl.UnsafeEnabled && x != nil { + if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { ms.StoreMessageInfo(mi) @@ -7478,11 +10617,6 @@ func (x *Trigger_Repeat) ProtoReflect() protoreflect.Message { return mi.MessageOf(x) } -// Deprecated: Use Trigger_Repeat.ProtoReflect.Descriptor instead. -func (*Trigger_Repeat) Descriptor() ([]byte, []int) { - return file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_rawDescGZIP(), []int{38, 11} -} - func (x *Trigger_Repeat) GetSubtrigger() *Trigger { if x != nil { return x.Subtrigger @@ -7490,22 +10624,49 @@ func (x *Trigger_Repeat) GetSubtrigger() *Trigger { return nil } -type TimestampTransform_Delay struct { - state protoimpl.MessageState - sizeCache protoimpl.SizeCache - unknownFields protoimpl.UnknownFields +func (x *Trigger_Repeat) SetSubtrigger(v *Trigger) { + x.Subtrigger = v +} +func (x *Trigger_Repeat) HasSubtrigger() bool { + if x == nil { + return false + } + return x.Subtrigger != nil +} + +func (x *Trigger_Repeat) ClearSubtrigger() { + x.Subtrigger = nil +} + +type Trigger_Repeat_builder struct { + _ [0]func() // Prevents comparability and use of unkeyed literals for the builder. + + // (Require) Trigger that is run repeatedly. + Subtrigger *Trigger +} + +func (b0 Trigger_Repeat_builder) Build() *Trigger_Repeat { + m0 := &Trigger_Repeat{} + b, x := &b0, m0 + _, _ = b, x + x.Subtrigger = b.Subtrigger + return m0 +} + +type TimestampTransform_Delay struct { + state protoimpl.MessageState `protogen:"hybrid.v1"` // (Required) The delay, in milliseconds. - DelayMillis int64 `protobuf:"varint,1,opt,name=delay_millis,json=delayMillis,proto3" json:"delay_millis,omitempty"` + DelayMillis int64 `protobuf:"varint,1,opt,name=delay_millis,json=delayMillis,proto3" json:"delay_millis,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache } func (x *TimestampTransform_Delay) Reset() { *x = TimestampTransform_Delay{} - if protoimpl.UnsafeEnabled { - mi := &file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_msgTypes[95] - ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) - ms.StoreMessageInfo(mi) - } + mi := &file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_msgTypes[95] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) } func (x *TimestampTransform_Delay) String() string { @@ -7516,7 +10677,7 @@ func (*TimestampTransform_Delay) ProtoMessage() {} func (x *TimestampTransform_Delay) ProtoReflect() protoreflect.Message { mi := &file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_msgTypes[95] - if protoimpl.UnsafeEnabled && x != nil { + if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { ms.StoreMessageInfo(mi) @@ -7526,11 +10687,6 @@ func (x *TimestampTransform_Delay) ProtoReflect() protoreflect.Message { return mi.MessageOf(x) } -// Deprecated: Use TimestampTransform_Delay.ProtoReflect.Descriptor instead. -func (*TimestampTransform_Delay) Descriptor() ([]byte, []int) { - return file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_rawDescGZIP(), []int{39, 0} -} - func (x *TimestampTransform_Delay) GetDelayMillis() int64 { if x != nil { return x.DelayMillis @@ -7538,26 +10694,42 @@ func (x *TimestampTransform_Delay) GetDelayMillis() int64 { return 0 } -type TimestampTransform_AlignTo struct { - state protoimpl.MessageState - sizeCache protoimpl.SizeCache - unknownFields protoimpl.UnknownFields +func (x *TimestampTransform_Delay) SetDelayMillis(v int64) { + x.DelayMillis = v +} +type TimestampTransform_Delay_builder struct { + _ [0]func() // Prevents comparability and use of unkeyed literals for the builder. + + // (Required) The delay, in milliseconds. + DelayMillis int64 +} + +func (b0 TimestampTransform_Delay_builder) Build() *TimestampTransform_Delay { + m0 := &TimestampTransform_Delay{} + b, x := &b0, m0 + _, _ = b, x + x.DelayMillis = b.DelayMillis + return m0 +} + +type TimestampTransform_AlignTo struct { + state protoimpl.MessageState `protogen:"hybrid.v1"` // (Required) A duration to which delays should be quantized // in milliseconds. Period int64 `protobuf:"varint,3,opt,name=period,proto3" json:"period,omitempty"` // (Required) An offset from 0 for the quantization specified by // alignment_size, in milliseconds - Offset int64 `protobuf:"varint,4,opt,name=offset,proto3" json:"offset,omitempty"` + Offset int64 `protobuf:"varint,4,opt,name=offset,proto3" json:"offset,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache } func (x *TimestampTransform_AlignTo) Reset() { *x = TimestampTransform_AlignTo{} - if protoimpl.UnsafeEnabled { - mi := &file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_msgTypes[96] - ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) - ms.StoreMessageInfo(mi) - } + mi := &file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_msgTypes[96] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) } func (x *TimestampTransform_AlignTo) String() string { @@ -7568,7 +10740,7 @@ func (*TimestampTransform_AlignTo) ProtoMessage() {} func (x *TimestampTransform_AlignTo) ProtoReflect() protoreflect.Message { mi := &file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_msgTypes[96] - if protoimpl.UnsafeEnabled && x != nil { + if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { ms.StoreMessageInfo(mi) @@ -7578,11 +10750,6 @@ func (x *TimestampTransform_AlignTo) ProtoReflect() protoreflect.Message { return mi.MessageOf(x) } -// Deprecated: Use TimestampTransform_AlignTo.ProtoReflect.Descriptor instead. -func (*TimestampTransform_AlignTo) Descriptor() ([]byte, []int) { - return file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_rawDescGZIP(), []int{39, 1} -} - func (x *TimestampTransform_AlignTo) GetPeriod() int64 { if x != nil { return x.Period @@ -7597,26 +10764,51 @@ func (x *TimestampTransform_AlignTo) GetOffset() int64 { return 0 } +func (x *TimestampTransform_AlignTo) SetPeriod(v int64) { + x.Period = v +} + +func (x *TimestampTransform_AlignTo) SetOffset(v int64) { + x.Offset = v +} + +type TimestampTransform_AlignTo_builder struct { + _ [0]func() // Prevents comparability and use of unkeyed literals for the builder. + + // (Required) A duration to which delays should be quantized + // in milliseconds. + Period int64 + // (Required) An offset from 0 for the quantization specified by + // alignment_size, in milliseconds + Offset int64 +} + +func (b0 TimestampTransform_AlignTo_builder) Build() *TimestampTransform_AlignTo { + m0 := &TimestampTransform_AlignTo{} + b, x := &b0, m0 + _, _ = b, x + x.Period = b.Period + x.Offset = b.Offset + return m0 +} + // A reference to a side input. Side inputs are uniquely identified by PTransform id and // local name. type ExecutableStagePayload_SideInputId struct { - state protoimpl.MessageState - sizeCache protoimpl.SizeCache - unknownFields protoimpl.UnknownFields - + state protoimpl.MessageState `protogen:"hybrid.v1"` // (Required) The id of the PTransform that references this side input. TransformId string `protobuf:"bytes,1,opt,name=transform_id,json=transformId,proto3" json:"transform_id,omitempty"` // (Required) The local name of this side input from the PTransform that references it. - LocalName string `protobuf:"bytes,2,opt,name=local_name,json=localName,proto3" json:"local_name,omitempty"` + LocalName string `protobuf:"bytes,2,opt,name=local_name,json=localName,proto3" json:"local_name,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache } func (x *ExecutableStagePayload_SideInputId) Reset() { *x = ExecutableStagePayload_SideInputId{} - if protoimpl.UnsafeEnabled { - mi := &file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_msgTypes[100] - ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) - ms.StoreMessageInfo(mi) - } + mi := &file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_msgTypes[100] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) } func (x *ExecutableStagePayload_SideInputId) String() string { @@ -7627,7 +10819,7 @@ func (*ExecutableStagePayload_SideInputId) ProtoMessage() {} func (x *ExecutableStagePayload_SideInputId) ProtoReflect() protoreflect.Message { mi := &file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_msgTypes[100] - if protoimpl.UnsafeEnabled && x != nil { + if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { ms.StoreMessageInfo(mi) @@ -7637,11 +10829,6 @@ func (x *ExecutableStagePayload_SideInputId) ProtoReflect() protoreflect.Message return mi.MessageOf(x) } -// Deprecated: Use ExecutableStagePayload_SideInputId.ProtoReflect.Descriptor instead. -func (*ExecutableStagePayload_SideInputId) Descriptor() ([]byte, []int) { - return file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_rawDescGZIP(), []int{64, 0} -} - func (x *ExecutableStagePayload_SideInputId) GetTransformId() string { if x != nil { return x.TransformId @@ -7656,26 +10843,49 @@ func (x *ExecutableStagePayload_SideInputId) GetLocalName() string { return "" } +func (x *ExecutableStagePayload_SideInputId) SetTransformId(v string) { + x.TransformId = v +} + +func (x *ExecutableStagePayload_SideInputId) SetLocalName(v string) { + x.LocalName = v +} + +type ExecutableStagePayload_SideInputId_builder struct { + _ [0]func() // Prevents comparability and use of unkeyed literals for the builder. + + // (Required) The id of the PTransform that references this side input. + TransformId string + // (Required) The local name of this side input from the PTransform that references it. + LocalName string +} + +func (b0 ExecutableStagePayload_SideInputId_builder) Build() *ExecutableStagePayload_SideInputId { + m0 := &ExecutableStagePayload_SideInputId{} + b, x := &b0, m0 + _, _ = b, x + x.TransformId = b.TransformId + x.LocalName = b.LocalName + return m0 +} + // A reference to user state. User states are uniquely identified by PTransform id and // local name. type ExecutableStagePayload_UserStateId struct { - state protoimpl.MessageState - sizeCache protoimpl.SizeCache - unknownFields protoimpl.UnknownFields - + state protoimpl.MessageState `protogen:"hybrid.v1"` // (Required) The id of the PTransform that references this user state. TransformId string `protobuf:"bytes,1,opt,name=transform_id,json=transformId,proto3" json:"transform_id,omitempty"` // (Required) The local name of this user state for the PTransform that references it. - LocalName string `protobuf:"bytes,2,opt,name=local_name,json=localName,proto3" json:"local_name,omitempty"` + LocalName string `protobuf:"bytes,2,opt,name=local_name,json=localName,proto3" json:"local_name,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache } func (x *ExecutableStagePayload_UserStateId) Reset() { *x = ExecutableStagePayload_UserStateId{} - if protoimpl.UnsafeEnabled { - mi := &file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_msgTypes[101] - ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) - ms.StoreMessageInfo(mi) - } + mi := &file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_msgTypes[101] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) } func (x *ExecutableStagePayload_UserStateId) String() string { @@ -7686,7 +10896,7 @@ func (*ExecutableStagePayload_UserStateId) ProtoMessage() {} func (x *ExecutableStagePayload_UserStateId) ProtoReflect() protoreflect.Message { mi := &file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_msgTypes[101] - if protoimpl.UnsafeEnabled && x != nil { + if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { ms.StoreMessageInfo(mi) @@ -7696,11 +10906,6 @@ func (x *ExecutableStagePayload_UserStateId) ProtoReflect() protoreflect.Message return mi.MessageOf(x) } -// Deprecated: Use ExecutableStagePayload_UserStateId.ProtoReflect.Descriptor instead. -func (*ExecutableStagePayload_UserStateId) Descriptor() ([]byte, []int) { - return file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_rawDescGZIP(), []int{64, 1} -} - func (x *ExecutableStagePayload_UserStateId) GetTransformId() string { if x != nil { return x.TransformId @@ -7715,26 +10920,49 @@ func (x *ExecutableStagePayload_UserStateId) GetLocalName() string { return "" } +func (x *ExecutableStagePayload_UserStateId) SetTransformId(v string) { + x.TransformId = v +} + +func (x *ExecutableStagePayload_UserStateId) SetLocalName(v string) { + x.LocalName = v +} + +type ExecutableStagePayload_UserStateId_builder struct { + _ [0]func() // Prevents comparability and use of unkeyed literals for the builder. + + // (Required) The id of the PTransform that references this user state. + TransformId string + // (Required) The local name of this user state for the PTransform that references it. + LocalName string +} + +func (b0 ExecutableStagePayload_UserStateId_builder) Build() *ExecutableStagePayload_UserStateId { + m0 := &ExecutableStagePayload_UserStateId{} + b, x := &b0, m0 + _, _ = b, x + x.TransformId = b.TransformId + x.LocalName = b.LocalName + return m0 +} + // A reference to a timer. Timers are uniquely identified by PTransform id and // local name. type ExecutableStagePayload_TimerId struct { - state protoimpl.MessageState - sizeCache protoimpl.SizeCache - unknownFields protoimpl.UnknownFields - + state protoimpl.MessageState `protogen:"hybrid.v1"` // (Required) The id of the PTransform that references this timer. TransformId string `protobuf:"bytes,1,opt,name=transform_id,json=transformId,proto3" json:"transform_id,omitempty"` // (Required) The local name of this timer for the PTransform that references it. - LocalName string `protobuf:"bytes,2,opt,name=local_name,json=localName,proto3" json:"local_name,omitempty"` + LocalName string `protobuf:"bytes,2,opt,name=local_name,json=localName,proto3" json:"local_name,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache } func (x *ExecutableStagePayload_TimerId) Reset() { *x = ExecutableStagePayload_TimerId{} - if protoimpl.UnsafeEnabled { - mi := &file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_msgTypes[102] - ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) - ms.StoreMessageInfo(mi) - } + mi := &file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_msgTypes[102] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) } func (x *ExecutableStagePayload_TimerId) String() string { @@ -7745,7 +10973,7 @@ func (*ExecutableStagePayload_TimerId) ProtoMessage() {} func (x *ExecutableStagePayload_TimerId) ProtoReflect() protoreflect.Message { mi := &file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_msgTypes[102] - if protoimpl.UnsafeEnabled && x != nil { + if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { ms.StoreMessageInfo(mi) @@ -7755,11 +10983,6 @@ func (x *ExecutableStagePayload_TimerId) ProtoReflect() protoreflect.Message { return mi.MessageOf(x) } -// Deprecated: Use ExecutableStagePayload_TimerId.ProtoReflect.Descriptor instead. -func (*ExecutableStagePayload_TimerId) Descriptor() ([]byte, []int) { - return file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_rawDescGZIP(), []int{64, 2} -} - func (x *ExecutableStagePayload_TimerId) GetTransformId() string { if x != nil { return x.TransformId @@ -7774,26 +10997,49 @@ func (x *ExecutableStagePayload_TimerId) GetLocalName() string { return "" } +func (x *ExecutableStagePayload_TimerId) SetTransformId(v string) { + x.TransformId = v +} + +func (x *ExecutableStagePayload_TimerId) SetLocalName(v string) { + x.LocalName = v +} + +type ExecutableStagePayload_TimerId_builder struct { + _ [0]func() // Prevents comparability and use of unkeyed literals for the builder. + + // (Required) The id of the PTransform that references this timer. + TransformId string + // (Required) The local name of this timer for the PTransform that references it. + LocalName string +} + +func (b0 ExecutableStagePayload_TimerId_builder) Build() *ExecutableStagePayload_TimerId { + m0 := &ExecutableStagePayload_TimerId{} + b, x := &b0, m0 + _, _ = b, x + x.TransformId = b.TransformId + x.LocalName = b.LocalName + return m0 +} + // A reference to a timer. Timers are uniquely identified by PTransform id and // local name. type ExecutableStagePayload_TimerFamilyId struct { - state protoimpl.MessageState - sizeCache protoimpl.SizeCache - unknownFields protoimpl.UnknownFields - + state protoimpl.MessageState `protogen:"hybrid.v1"` // (Required) The id of the PTransform that references this timer family. TransformId string `protobuf:"bytes,1,opt,name=transform_id,json=transformId,proto3" json:"transform_id,omitempty"` // (Required) The local name of this timer family for the PTransform that references it. - LocalName string `protobuf:"bytes,2,opt,name=local_name,json=localName,proto3" json:"local_name,omitempty"` + LocalName string `protobuf:"bytes,2,opt,name=local_name,json=localName,proto3" json:"local_name,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache } func (x *ExecutableStagePayload_TimerFamilyId) Reset() { *x = ExecutableStagePayload_TimerFamilyId{} - if protoimpl.UnsafeEnabled { - mi := &file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_msgTypes[103] - ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) - ms.StoreMessageInfo(mi) - } + mi := &file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_msgTypes[103] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) } func (x *ExecutableStagePayload_TimerFamilyId) String() string { @@ -7804,7 +11050,7 @@ func (*ExecutableStagePayload_TimerFamilyId) ProtoMessage() {} func (x *ExecutableStagePayload_TimerFamilyId) ProtoReflect() protoreflect.Message { mi := &file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_msgTypes[103] - if protoimpl.UnsafeEnabled && x != nil { + if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { ms.StoreMessageInfo(mi) @@ -7814,11 +11060,6 @@ func (x *ExecutableStagePayload_TimerFamilyId) ProtoReflect() protoreflect.Messa return mi.MessageOf(x) } -// Deprecated: Use ExecutableStagePayload_TimerFamilyId.ProtoReflect.Descriptor instead. -func (*ExecutableStagePayload_TimerFamilyId) Descriptor() ([]byte, []int) { - return file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_rawDescGZIP(), []int{64, 3} -} - func (x *ExecutableStagePayload_TimerFamilyId) GetTransformId() string { if x != nil { return x.TransformId @@ -7833,12 +11074,35 @@ func (x *ExecutableStagePayload_TimerFamilyId) GetLocalName() string { return "" } +func (x *ExecutableStagePayload_TimerFamilyId) SetTransformId(v string) { + x.TransformId = v +} + +func (x *ExecutableStagePayload_TimerFamilyId) SetLocalName(v string) { + x.LocalName = v +} + +type ExecutableStagePayload_TimerFamilyId_builder struct { + _ [0]func() // Prevents comparability and use of unkeyed literals for the builder. + + // (Required) The id of the PTransform that references this timer family. + TransformId string + // (Required) The local name of this timer family for the PTransform that references it. + LocalName string +} + +func (b0 ExecutableStagePayload_TimerFamilyId_builder) Build() *ExecutableStagePayload_TimerFamilyId { + m0 := &ExecutableStagePayload_TimerFamilyId{} + b, x := &b0, m0 + _, _ = b, x + x.TransformId = b.TransformId + x.LocalName = b.LocalName + return m0 +} + // Settings that decide the coder type of wire coder. type ExecutableStagePayload_WireCoderSetting struct { - state protoimpl.MessageState - sizeCache protoimpl.SizeCache - unknownFields protoimpl.UnknownFields - + state protoimpl.MessageState `protogen:"hybrid.v1"` // (Required) The URN of the wire coder. // Note that only windowed value coder or parameterized windowed value coder are supported. Urn string `protobuf:"bytes,1,opt,name=urn,proto3" json:"urn,omitempty"` @@ -7851,20 +11115,20 @@ type ExecutableStagePayload_WireCoderSetting struct { Payload []byte `protobuf:"bytes,2,opt,name=payload,proto3" json:"payload,omitempty"` // (Required) The target(PCollection or Timer) this setting applies to. // - // Types that are assignable to Target: + // Types that are valid to be assigned to Target: // // *ExecutableStagePayload_WireCoderSetting_InputOrOutputId // *ExecutableStagePayload_WireCoderSetting_Timer - Target isExecutableStagePayload_WireCoderSetting_Target `protobuf_oneof:"target"` + Target isExecutableStagePayload_WireCoderSetting_Target `protobuf_oneof:"target"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache } func (x *ExecutableStagePayload_WireCoderSetting) Reset() { *x = ExecutableStagePayload_WireCoderSetting{} - if protoimpl.UnsafeEnabled { - mi := &file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_msgTypes[104] - ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) - ms.StoreMessageInfo(mi) - } + mi := &file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_msgTypes[104] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) } func (x *ExecutableStagePayload_WireCoderSetting) String() string { @@ -7875,7 +11139,7 @@ func (*ExecutableStagePayload_WireCoderSetting) ProtoMessage() {} func (x *ExecutableStagePayload_WireCoderSetting) ProtoReflect() protoreflect.Message { mi := &file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_msgTypes[104] - if protoimpl.UnsafeEnabled && x != nil { + if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { ms.StoreMessageInfo(mi) @@ -7885,11 +11149,6 @@ func (x *ExecutableStagePayload_WireCoderSetting) ProtoReflect() protoreflect.Me return mi.MessageOf(x) } -// Deprecated: Use ExecutableStagePayload_WireCoderSetting.ProtoReflect.Descriptor instead. -func (*ExecutableStagePayload_WireCoderSetting) Descriptor() ([]byte, []int) { - return file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_rawDescGZIP(), []int{64, 4} -} - func (x *ExecutableStagePayload_WireCoderSetting) GetUrn() string { if x != nil { return x.Urn @@ -7904,27 +11163,159 @@ func (x *ExecutableStagePayload_WireCoderSetting) GetPayload() []byte { return nil } -func (m *ExecutableStagePayload_WireCoderSetting) GetTarget() isExecutableStagePayload_WireCoderSetting_Target { - if m != nil { - return m.Target +func (x *ExecutableStagePayload_WireCoderSetting) GetTarget() isExecutableStagePayload_WireCoderSetting_Target { + if x != nil { + return x.Target } return nil } func (x *ExecutableStagePayload_WireCoderSetting) GetInputOrOutputId() string { - if x, ok := x.GetTarget().(*ExecutableStagePayload_WireCoderSetting_InputOrOutputId); ok { - return x.InputOrOutputId + if x != nil { + if x, ok := x.Target.(*ExecutableStagePayload_WireCoderSetting_InputOrOutputId); ok { + return x.InputOrOutputId + } } return "" } func (x *ExecutableStagePayload_WireCoderSetting) GetTimer() *ExecutableStagePayload_TimerId { - if x, ok := x.GetTarget().(*ExecutableStagePayload_WireCoderSetting_Timer); ok { - return x.Timer + if x != nil { + if x, ok := x.Target.(*ExecutableStagePayload_WireCoderSetting_Timer); ok { + return x.Timer + } } return nil } +func (x *ExecutableStagePayload_WireCoderSetting) SetUrn(v string) { + x.Urn = v +} + +func (x *ExecutableStagePayload_WireCoderSetting) SetPayload(v []byte) { + if v == nil { + v = []byte{} + } + x.Payload = v +} + +func (x *ExecutableStagePayload_WireCoderSetting) SetInputOrOutputId(v string) { + x.Target = &ExecutableStagePayload_WireCoderSetting_InputOrOutputId{v} +} + +func (x *ExecutableStagePayload_WireCoderSetting) SetTimer(v *ExecutableStagePayload_TimerId) { + if v == nil { + x.Target = nil + return + } + x.Target = &ExecutableStagePayload_WireCoderSetting_Timer{v} +} + +func (x *ExecutableStagePayload_WireCoderSetting) HasTarget() bool { + if x == nil { + return false + } + return x.Target != nil +} + +func (x *ExecutableStagePayload_WireCoderSetting) HasInputOrOutputId() bool { + if x == nil { + return false + } + _, ok := x.Target.(*ExecutableStagePayload_WireCoderSetting_InputOrOutputId) + return ok +} + +func (x *ExecutableStagePayload_WireCoderSetting) HasTimer() bool { + if x == nil { + return false + } + _, ok := x.Target.(*ExecutableStagePayload_WireCoderSetting_Timer) + return ok +} + +func (x *ExecutableStagePayload_WireCoderSetting) ClearTarget() { + x.Target = nil +} + +func (x *ExecutableStagePayload_WireCoderSetting) ClearInputOrOutputId() { + if _, ok := x.Target.(*ExecutableStagePayload_WireCoderSetting_InputOrOutputId); ok { + x.Target = nil + } +} + +func (x *ExecutableStagePayload_WireCoderSetting) ClearTimer() { + if _, ok := x.Target.(*ExecutableStagePayload_WireCoderSetting_Timer); ok { + x.Target = nil + } +} + +const ExecutableStagePayload_WireCoderSetting_Target_not_set_case case_ExecutableStagePayload_WireCoderSetting_Target = 0 +const ExecutableStagePayload_WireCoderSetting_InputOrOutputId_case case_ExecutableStagePayload_WireCoderSetting_Target = 3 +const ExecutableStagePayload_WireCoderSetting_Timer_case case_ExecutableStagePayload_WireCoderSetting_Target = 4 + +func (x *ExecutableStagePayload_WireCoderSetting) WhichTarget() case_ExecutableStagePayload_WireCoderSetting_Target { + if x == nil { + return ExecutableStagePayload_WireCoderSetting_Target_not_set_case + } + switch x.Target.(type) { + case *ExecutableStagePayload_WireCoderSetting_InputOrOutputId: + return ExecutableStagePayload_WireCoderSetting_InputOrOutputId_case + case *ExecutableStagePayload_WireCoderSetting_Timer: + return ExecutableStagePayload_WireCoderSetting_Timer_case + default: + return ExecutableStagePayload_WireCoderSetting_Target_not_set_case + } +} + +type ExecutableStagePayload_WireCoderSetting_builder struct { + _ [0]func() // Prevents comparability and use of unkeyed literals for the builder. + + // (Required) The URN of the wire coder. + // Note that only windowed value coder or parameterized windowed value coder are supported. + Urn string + // (Optional) The data specifying any parameters to the URN. If + // the URN is beam:coder:windowed_value:v1, this may be omitted. If the URN is + // beam:coder:param_windowed_value:v1, the payload is an encoded windowed + // value using the beam:coder:windowed_value:v1 coder parameterized by + // a beam:coder:bytes:v1 element coder and the window coder that this + // param_windowed_value coder uses. + Payload []byte + // (Required) The target(PCollection or Timer) this setting applies to. + + // Fields of oneof Target: + // The input or output PCollection id this setting applies to. + InputOrOutputId *string + // The timer id this setting applies to. + Timer *ExecutableStagePayload_TimerId + // -- end of Target +} + +func (b0 ExecutableStagePayload_WireCoderSetting_builder) Build() *ExecutableStagePayload_WireCoderSetting { + m0 := &ExecutableStagePayload_WireCoderSetting{} + b, x := &b0, m0 + _, _ = b, x + x.Urn = b.Urn + x.Payload = b.Payload + if b.InputOrOutputId != nil { + x.Target = &ExecutableStagePayload_WireCoderSetting_InputOrOutputId{*b.InputOrOutputId} + } + if b.Timer != nil { + x.Target = &ExecutableStagePayload_WireCoderSetting_Timer{b.Timer} + } + return m0 +} + +type case_ExecutableStagePayload_WireCoderSetting_Target protoreflect.FieldNumber + +func (x case_ExecutableStagePayload_WireCoderSetting_Target) String() string { + md := file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_msgTypes[104].Descriptor() + if x == 0 { + return "not set" + } + return protoimpl.X.MessageFieldStringOf(md, protoreflect.FieldNumber(x)) +} + type isExecutableStagePayload_WireCoderSetting_Target interface { isExecutableStagePayload_WireCoderSetting_Target() } @@ -9138,9 +12529,9 @@ var file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_rawDesc = []byt 0x32, 0x2e, 0x2e, 0x6f, 0x72, 0x67, 0x2e, 0x61, 0x70, 0x61, 0x63, 0x68, 0x65, 0x2e, 0x62, 0x65, 0x61, 0x6d, 0x2e, 0x6d, 0x6f, 0x64, 0x65, 0x6c, 0x2e, 0x70, 0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x45, 0x6e, 0x76, 0x69, 0x72, 0x6f, 0x6e, 0x6d, 0x65, 0x6e, 0x74, - 0x52, 0x0c, 0x65, 0x6e, 0x76, 0x69, 0x72, 0x6f, 0x6e, 0x6d, 0x65, 0x6e, 0x74, 0x73, 0x22, 0xef, - 0x05, 0x0a, 0x11, 0x53, 0x74, 0x61, 0x6e, 0x64, 0x61, 0x72, 0x64, 0x50, 0x72, 0x6f, 0x74, 0x6f, - 0x63, 0x6f, 0x6c, 0x73, 0x22, 0xd9, 0x05, 0x0a, 0x04, 0x45, 0x6e, 0x75, 0x6d, 0x12, 0x48, 0x0a, + 0x52, 0x0c, 0x65, 0x6e, 0x76, 0x69, 0x72, 0x6f, 0x6e, 0x6d, 0x65, 0x6e, 0x74, 0x73, 0x22, 0xb2, + 0x06, 0x0a, 0x11, 0x53, 0x74, 0x61, 0x6e, 0x64, 0x61, 0x72, 0x64, 0x50, 0x72, 0x6f, 0x74, 0x6f, + 0x63, 0x6f, 0x6c, 0x73, 0x22, 0x9c, 0x06, 0x0a, 0x04, 0x45, 0x6e, 0x75, 0x6d, 0x12, 0x48, 0x0a, 0x19, 0x4c, 0x45, 0x47, 0x41, 0x43, 0x59, 0x5f, 0x50, 0x52, 0x4f, 0x47, 0x52, 0x45, 0x53, 0x53, 0x5f, 0x52, 0x45, 0x50, 0x4f, 0x52, 0x54, 0x49, 0x4e, 0x47, 0x10, 0x00, 0x1a, 0x29, 0xa2, 0xb4, 0xfa, 0xc2, 0x05, 0x23, 0x62, 0x65, 0x61, 0x6d, 0x3a, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x63, 0x6f, @@ -9186,286 +12577,279 @@ var file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_rawDesc = []byt 0xfa, 0xc2, 0x05, 0x2c, 0x62, 0x65, 0x61, 0x6d, 0x3a, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x63, 0x6f, 0x6c, 0x3a, 0x73, 0x64, 0x6b, 0x5f, 0x63, 0x6f, 0x6e, 0x73, 0x75, 0x6d, 0x69, 0x6e, 0x67, 0x5f, 0x72, 0x65, 0x63, 0x65, 0x69, 0x76, 0x65, 0x64, 0x5f, 0x64, 0x61, 0x74, 0x61, 0x3a, 0x76, 0x31, - 0x22, 0xb5, 0x02, 0x0a, 0x17, 0x53, 0x74, 0x61, 0x6e, 0x64, 0x61, 0x72, 0x64, 0x52, 0x75, 0x6e, - 0x6e, 0x65, 0x72, 0x50, 0x72, 0x6f, 0x74, 0x6f, 0x63, 0x6f, 0x6c, 0x73, 0x22, 0x99, 0x02, 0x0a, - 0x04, 0x45, 0x6e, 0x75, 0x6d, 0x12, 0x4f, 0x0a, 0x19, 0x4d, 0x4f, 0x4e, 0x49, 0x54, 0x4f, 0x52, - 0x49, 0x4e, 0x47, 0x5f, 0x49, 0x4e, 0x46, 0x4f, 0x5f, 0x53, 0x48, 0x4f, 0x52, 0x54, 0x5f, 0x49, - 0x44, 0x53, 0x10, 0x00, 0x1a, 0x30, 0xa2, 0xb4, 0xfa, 0xc2, 0x05, 0x2a, 0x62, 0x65, 0x61, 0x6d, - 0x3a, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x63, 0x6f, 0x6c, 0x3a, 0x6d, 0x6f, 0x6e, 0x69, 0x74, 0x6f, - 0x72, 0x69, 0x6e, 0x67, 0x5f, 0x69, 0x6e, 0x66, 0x6f, 0x5f, 0x73, 0x68, 0x6f, 0x72, 0x74, 0x5f, - 0x69, 0x64, 0x73, 0x3a, 0x76, 0x31, 0x12, 0x63, 0x0a, 0x23, 0x43, 0x4f, 0x4e, 0x54, 0x52, 0x4f, - 0x4c, 0x5f, 0x52, 0x45, 0x53, 0x50, 0x4f, 0x4e, 0x53, 0x45, 0x5f, 0x45, 0x4c, 0x45, 0x4d, 0x45, - 0x4e, 0x54, 0x53, 0x5f, 0x45, 0x4d, 0x42, 0x45, 0x44, 0x44, 0x49, 0x4e, 0x47, 0x10, 0x06, 0x1a, - 0x3a, 0xa2, 0xb4, 0xfa, 0xc2, 0x05, 0x34, 0x62, 0x65, 0x61, 0x6d, 0x3a, 0x70, 0x72, 0x6f, 0x74, - 0x6f, 0x63, 0x6f, 0x6c, 0x3a, 0x63, 0x6f, 0x6e, 0x74, 0x72, 0x6f, 0x6c, 0x5f, 0x72, 0x65, 0x73, - 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x5f, 0x65, 0x6c, 0x65, 0x6d, 0x65, 0x6e, 0x74, 0x73, 0x5f, 0x65, - 0x6d, 0x62, 0x65, 0x64, 0x64, 0x69, 0x6e, 0x67, 0x3a, 0x76, 0x31, 0x12, 0x5b, 0x0a, 0x1f, 0x4d, - 0x55, 0x4c, 0x54, 0x49, 0x4d, 0x41, 0x50, 0x5f, 0x4b, 0x45, 0x59, 0x53, 0x5f, 0x56, 0x41, 0x4c, - 0x55, 0x45, 0x53, 0x5f, 0x53, 0x49, 0x44, 0x45, 0x5f, 0x49, 0x4e, 0x50, 0x55, 0x54, 0x10, 0x07, - 0x1a, 0x36, 0xa2, 0xb4, 0xfa, 0xc2, 0x05, 0x30, 0x62, 0x65, 0x61, 0x6d, 0x3a, 0x70, 0x72, 0x6f, - 0x74, 0x6f, 0x63, 0x6f, 0x6c, 0x3a, 0x6d, 0x75, 0x6c, 0x74, 0x69, 0x6d, 0x61, 0x70, 0x5f, 0x6b, - 0x65, 0x79, 0x73, 0x5f, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x73, 0x5f, 0x73, 0x69, 0x64, 0x65, 0x5f, - 0x69, 0x6e, 0x70, 0x75, 0x74, 0x3a, 0x76, 0x31, 0x22, 0xff, 0x03, 0x0a, 0x14, 0x53, 0x74, 0x61, - 0x6e, 0x64, 0x61, 0x72, 0x64, 0x52, 0x65, 0x71, 0x75, 0x69, 0x72, 0x65, 0x6d, 0x65, 0x6e, 0x74, - 0x73, 0x22, 0xe6, 0x03, 0x0a, 0x04, 0x45, 0x6e, 0x75, 0x6d, 0x12, 0x4a, 0x0a, 0x1c, 0x52, 0x45, - 0x51, 0x55, 0x49, 0x52, 0x45, 0x53, 0x5f, 0x53, 0x54, 0x41, 0x54, 0x45, 0x46, 0x55, 0x4c, 0x5f, - 0x50, 0x52, 0x4f, 0x43, 0x45, 0x53, 0x53, 0x49, 0x4e, 0x47, 0x10, 0x00, 0x1a, 0x28, 0xa2, 0xb4, - 0xfa, 0xc2, 0x05, 0x22, 0x62, 0x65, 0x61, 0x6d, 0x3a, 0x72, 0x65, 0x71, 0x75, 0x69, 0x72, 0x65, - 0x6d, 0x65, 0x6e, 0x74, 0x3a, 0x70, 0x61, 0x72, 0x64, 0x6f, 0x3a, 0x73, 0x74, 0x61, 0x74, 0x65, - 0x66, 0x75, 0x6c, 0x3a, 0x76, 0x31, 0x12, 0x4e, 0x0a, 0x1c, 0x52, 0x45, 0x51, 0x55, 0x49, 0x52, - 0x45, 0x53, 0x5f, 0x42, 0x55, 0x4e, 0x44, 0x4c, 0x45, 0x5f, 0x46, 0x49, 0x4e, 0x41, 0x4c, 0x49, - 0x5a, 0x41, 0x54, 0x49, 0x4f, 0x4e, 0x10, 0x01, 0x1a, 0x2c, 0xa2, 0xb4, 0xfa, 0xc2, 0x05, 0x26, - 0x62, 0x65, 0x61, 0x6d, 0x3a, 0x72, 0x65, 0x71, 0x75, 0x69, 0x72, 0x65, 0x6d, 0x65, 0x6e, 0x74, - 0x3a, 0x70, 0x61, 0x72, 0x64, 0x6f, 0x3a, 0x66, 0x69, 0x6e, 0x61, 0x6c, 0x69, 0x7a, 0x61, 0x74, - 0x69, 0x6f, 0x6e, 0x3a, 0x76, 0x31, 0x12, 0x47, 0x0a, 0x15, 0x52, 0x45, 0x51, 0x55, 0x49, 0x52, - 0x45, 0x53, 0x5f, 0x53, 0x54, 0x41, 0x42, 0x4c, 0x45, 0x5f, 0x49, 0x4e, 0x50, 0x55, 0x54, 0x10, - 0x02, 0x1a, 0x2c, 0xa2, 0xb4, 0xfa, 0xc2, 0x05, 0x26, 0x62, 0x65, 0x61, 0x6d, 0x3a, 0x72, 0x65, - 0x71, 0x75, 0x69, 0x72, 0x65, 0x6d, 0x65, 0x6e, 0x74, 0x3a, 0x70, 0x61, 0x72, 0x64, 0x6f, 0x3a, - 0x73, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x5f, 0x69, 0x6e, 0x70, 0x75, 0x74, 0x3a, 0x76, 0x31, 0x12, - 0x51, 0x0a, 0x1a, 0x52, 0x45, 0x51, 0x55, 0x49, 0x52, 0x45, 0x53, 0x5f, 0x54, 0x49, 0x4d, 0x45, - 0x5f, 0x53, 0x4f, 0x52, 0x54, 0x45, 0x44, 0x5f, 0x49, 0x4e, 0x50, 0x55, 0x54, 0x10, 0x03, 0x1a, - 0x31, 0xa2, 0xb4, 0xfa, 0xc2, 0x05, 0x2b, 0x62, 0x65, 0x61, 0x6d, 0x3a, 0x72, 0x65, 0x71, 0x75, - 0x69, 0x72, 0x65, 0x6d, 0x65, 0x6e, 0x74, 0x3a, 0x70, 0x61, 0x72, 0x64, 0x6f, 0x3a, 0x74, 0x69, - 0x6d, 0x65, 0x5f, 0x73, 0x6f, 0x72, 0x74, 0x65, 0x64, 0x5f, 0x69, 0x6e, 0x70, 0x75, 0x74, 0x3a, - 0x76, 0x31, 0x12, 0x4d, 0x0a, 0x18, 0x52, 0x45, 0x51, 0x55, 0x49, 0x52, 0x45, 0x53, 0x5f, 0x53, - 0x50, 0x4c, 0x49, 0x54, 0x54, 0x41, 0x42, 0x4c, 0x45, 0x5f, 0x44, 0x4f, 0x46, 0x4e, 0x10, 0x04, - 0x1a, 0x2f, 0xa2, 0xb4, 0xfa, 0xc2, 0x05, 0x29, 0x62, 0x65, 0x61, 0x6d, 0x3a, 0x72, 0x65, 0x71, - 0x75, 0x69, 0x72, 0x65, 0x6d, 0x65, 0x6e, 0x74, 0x3a, 0x70, 0x61, 0x72, 0x64, 0x6f, 0x3a, 0x73, - 0x70, 0x6c, 0x69, 0x74, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x5f, 0x64, 0x6f, 0x66, 0x6e, 0x3a, 0x76, - 0x31, 0x12, 0x57, 0x0a, 0x1d, 0x52, 0x45, 0x51, 0x55, 0x49, 0x52, 0x45, 0x53, 0x5f, 0x4f, 0x4e, - 0x5f, 0x57, 0x49, 0x4e, 0x44, 0x4f, 0x57, 0x5f, 0x45, 0x58, 0x50, 0x49, 0x52, 0x41, 0x54, 0x49, - 0x4f, 0x4e, 0x10, 0x05, 0x1a, 0x34, 0xa2, 0xb4, 0xfa, 0xc2, 0x05, 0x2e, 0x62, 0x65, 0x61, 0x6d, + 0x12, 0x41, 0x0a, 0x12, 0x4f, 0x52, 0x44, 0x45, 0x52, 0x45, 0x44, 0x5f, 0x4c, 0x49, 0x53, 0x54, + 0x5f, 0x53, 0x54, 0x41, 0x54, 0x45, 0x10, 0x0a, 0x1a, 0x29, 0xa2, 0xb4, 0xfa, 0xc2, 0x05, 0x23, + 0x62, 0x65, 0x61, 0x6d, 0x3a, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x63, 0x6f, 0x6c, 0x3a, 0x6f, 0x72, + 0x64, 0x65, 0x72, 0x65, 0x64, 0x5f, 0x6c, 0x69, 0x73, 0x74, 0x5f, 0x73, 0x74, 0x61, 0x74, 0x65, + 0x3a, 0x76, 0x31, 0x22, 0xb5, 0x02, 0x0a, 0x17, 0x53, 0x74, 0x61, 0x6e, 0x64, 0x61, 0x72, 0x64, + 0x52, 0x75, 0x6e, 0x6e, 0x65, 0x72, 0x50, 0x72, 0x6f, 0x74, 0x6f, 0x63, 0x6f, 0x6c, 0x73, 0x22, + 0x99, 0x02, 0x0a, 0x04, 0x45, 0x6e, 0x75, 0x6d, 0x12, 0x4f, 0x0a, 0x19, 0x4d, 0x4f, 0x4e, 0x49, + 0x54, 0x4f, 0x52, 0x49, 0x4e, 0x47, 0x5f, 0x49, 0x4e, 0x46, 0x4f, 0x5f, 0x53, 0x48, 0x4f, 0x52, + 0x54, 0x5f, 0x49, 0x44, 0x53, 0x10, 0x00, 0x1a, 0x30, 0xa2, 0xb4, 0xfa, 0xc2, 0x05, 0x2a, 0x62, + 0x65, 0x61, 0x6d, 0x3a, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x63, 0x6f, 0x6c, 0x3a, 0x6d, 0x6f, 0x6e, + 0x69, 0x74, 0x6f, 0x72, 0x69, 0x6e, 0x67, 0x5f, 0x69, 0x6e, 0x66, 0x6f, 0x5f, 0x73, 0x68, 0x6f, + 0x72, 0x74, 0x5f, 0x69, 0x64, 0x73, 0x3a, 0x76, 0x31, 0x12, 0x63, 0x0a, 0x23, 0x43, 0x4f, 0x4e, + 0x54, 0x52, 0x4f, 0x4c, 0x5f, 0x52, 0x45, 0x53, 0x50, 0x4f, 0x4e, 0x53, 0x45, 0x5f, 0x45, 0x4c, + 0x45, 0x4d, 0x45, 0x4e, 0x54, 0x53, 0x5f, 0x45, 0x4d, 0x42, 0x45, 0x44, 0x44, 0x49, 0x4e, 0x47, + 0x10, 0x06, 0x1a, 0x3a, 0xa2, 0xb4, 0xfa, 0xc2, 0x05, 0x34, 0x62, 0x65, 0x61, 0x6d, 0x3a, 0x70, + 0x72, 0x6f, 0x74, 0x6f, 0x63, 0x6f, 0x6c, 0x3a, 0x63, 0x6f, 0x6e, 0x74, 0x72, 0x6f, 0x6c, 0x5f, + 0x72, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x5f, 0x65, 0x6c, 0x65, 0x6d, 0x65, 0x6e, 0x74, + 0x73, 0x5f, 0x65, 0x6d, 0x62, 0x65, 0x64, 0x64, 0x69, 0x6e, 0x67, 0x3a, 0x76, 0x31, 0x12, 0x5b, + 0x0a, 0x1f, 0x4d, 0x55, 0x4c, 0x54, 0x49, 0x4d, 0x41, 0x50, 0x5f, 0x4b, 0x45, 0x59, 0x53, 0x5f, + 0x56, 0x41, 0x4c, 0x55, 0x45, 0x53, 0x5f, 0x53, 0x49, 0x44, 0x45, 0x5f, 0x49, 0x4e, 0x50, 0x55, + 0x54, 0x10, 0x07, 0x1a, 0x36, 0xa2, 0xb4, 0xfa, 0xc2, 0x05, 0x30, 0x62, 0x65, 0x61, 0x6d, 0x3a, + 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x63, 0x6f, 0x6c, 0x3a, 0x6d, 0x75, 0x6c, 0x74, 0x69, 0x6d, 0x61, + 0x70, 0x5f, 0x6b, 0x65, 0x79, 0x73, 0x5f, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x73, 0x5f, 0x73, 0x69, + 0x64, 0x65, 0x5f, 0x69, 0x6e, 0x70, 0x75, 0x74, 0x3a, 0x76, 0x31, 0x22, 0xff, 0x03, 0x0a, 0x14, + 0x53, 0x74, 0x61, 0x6e, 0x64, 0x61, 0x72, 0x64, 0x52, 0x65, 0x71, 0x75, 0x69, 0x72, 0x65, 0x6d, + 0x65, 0x6e, 0x74, 0x73, 0x22, 0xe6, 0x03, 0x0a, 0x04, 0x45, 0x6e, 0x75, 0x6d, 0x12, 0x4a, 0x0a, + 0x1c, 0x52, 0x45, 0x51, 0x55, 0x49, 0x52, 0x45, 0x53, 0x5f, 0x53, 0x54, 0x41, 0x54, 0x45, 0x46, + 0x55, 0x4c, 0x5f, 0x50, 0x52, 0x4f, 0x43, 0x45, 0x53, 0x53, 0x49, 0x4e, 0x47, 0x10, 0x00, 0x1a, + 0x28, 0xa2, 0xb4, 0xfa, 0xc2, 0x05, 0x22, 0x62, 0x65, 0x61, 0x6d, 0x3a, 0x72, 0x65, 0x71, 0x75, + 0x69, 0x72, 0x65, 0x6d, 0x65, 0x6e, 0x74, 0x3a, 0x70, 0x61, 0x72, 0x64, 0x6f, 0x3a, 0x73, 0x74, + 0x61, 0x74, 0x65, 0x66, 0x75, 0x6c, 0x3a, 0x76, 0x31, 0x12, 0x4e, 0x0a, 0x1c, 0x52, 0x45, 0x51, + 0x55, 0x49, 0x52, 0x45, 0x53, 0x5f, 0x42, 0x55, 0x4e, 0x44, 0x4c, 0x45, 0x5f, 0x46, 0x49, 0x4e, + 0x41, 0x4c, 0x49, 0x5a, 0x41, 0x54, 0x49, 0x4f, 0x4e, 0x10, 0x01, 0x1a, 0x2c, 0xa2, 0xb4, 0xfa, + 0xc2, 0x05, 0x26, 0x62, 0x65, 0x61, 0x6d, 0x3a, 0x72, 0x65, 0x71, 0x75, 0x69, 0x72, 0x65, 0x6d, + 0x65, 0x6e, 0x74, 0x3a, 0x70, 0x61, 0x72, 0x64, 0x6f, 0x3a, 0x66, 0x69, 0x6e, 0x61, 0x6c, 0x69, + 0x7a, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x3a, 0x76, 0x31, 0x12, 0x47, 0x0a, 0x15, 0x52, 0x45, 0x51, + 0x55, 0x49, 0x52, 0x45, 0x53, 0x5f, 0x53, 0x54, 0x41, 0x42, 0x4c, 0x45, 0x5f, 0x49, 0x4e, 0x50, + 0x55, 0x54, 0x10, 0x02, 0x1a, 0x2c, 0xa2, 0xb4, 0xfa, 0xc2, 0x05, 0x26, 0x62, 0x65, 0x61, 0x6d, 0x3a, 0x72, 0x65, 0x71, 0x75, 0x69, 0x72, 0x65, 0x6d, 0x65, 0x6e, 0x74, 0x3a, 0x70, 0x61, 0x72, - 0x64, 0x6f, 0x3a, 0x6f, 0x6e, 0x5f, 0x77, 0x69, 0x6e, 0x64, 0x6f, 0x77, 0x5f, 0x65, 0x78, 0x70, - 0x69, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x3a, 0x76, 0x31, 0x22, 0x3a, 0x0a, 0x0c, 0x46, 0x75, - 0x6e, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x53, 0x70, 0x65, 0x63, 0x12, 0x10, 0x0a, 0x03, 0x75, 0x72, - 0x6e, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x03, 0x75, 0x72, 0x6e, 0x12, 0x18, 0x0a, 0x07, - 0x70, 0x61, 0x79, 0x6c, 0x6f, 0x61, 0x64, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0c, 0x52, 0x07, 0x70, - 0x61, 0x79, 0x6c, 0x6f, 0x61, 0x64, 0x22, 0x57, 0x0a, 0x13, 0x53, 0x74, 0x61, 0x6e, 0x64, 0x61, - 0x72, 0x64, 0x44, 0x69, 0x73, 0x70, 0x6c, 0x61, 0x79, 0x44, 0x61, 0x74, 0x61, 0x22, 0x40, 0x0a, - 0x0b, 0x44, 0x69, 0x73, 0x70, 0x6c, 0x61, 0x79, 0x44, 0x61, 0x74, 0x61, 0x12, 0x31, 0x0a, 0x08, - 0x4c, 0x41, 0x42, 0x45, 0x4c, 0x4c, 0x45, 0x44, 0x10, 0x00, 0x1a, 0x23, 0xa2, 0xb4, 0xfa, 0xc2, - 0x05, 0x1d, 0x62, 0x65, 0x61, 0x6d, 0x3a, 0x64, 0x69, 0x73, 0x70, 0x6c, 0x61, 0x79, 0x5f, 0x64, - 0x61, 0x74, 0x61, 0x3a, 0x6c, 0x61, 0x62, 0x65, 0x6c, 0x6c, 0x65, 0x64, 0x3a, 0x76, 0x31, 0x22, - 0xea, 0x01, 0x0a, 0x0f, 0x4c, 0x61, 0x62, 0x65, 0x6c, 0x6c, 0x65, 0x64, 0x50, 0x61, 0x79, 0x6c, - 0x6f, 0x61, 0x64, 0x12, 0x14, 0x0a, 0x05, 0x6c, 0x61, 0x62, 0x65, 0x6c, 0x18, 0x01, 0x20, 0x01, - 0x28, 0x09, 0x52, 0x05, 0x6c, 0x61, 0x62, 0x65, 0x6c, 0x12, 0x23, 0x0a, 0x0c, 0x73, 0x74, 0x72, - 0x69, 0x6e, 0x67, 0x5f, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x48, - 0x00, 0x52, 0x0b, 0x73, 0x74, 0x72, 0x69, 0x6e, 0x67, 0x56, 0x61, 0x6c, 0x75, 0x65, 0x12, 0x1f, - 0x0a, 0x0a, 0x62, 0x6f, 0x6f, 0x6c, 0x5f, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x18, 0x03, 0x20, 0x01, - 0x28, 0x08, 0x48, 0x00, 0x52, 0x09, 0x62, 0x6f, 0x6f, 0x6c, 0x56, 0x61, 0x6c, 0x75, 0x65, 0x12, - 0x23, 0x0a, 0x0c, 0x64, 0x6f, 0x75, 0x62, 0x6c, 0x65, 0x5f, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x18, - 0x04, 0x20, 0x01, 0x28, 0x01, 0x48, 0x00, 0x52, 0x0b, 0x64, 0x6f, 0x75, 0x62, 0x6c, 0x65, 0x56, - 0x61, 0x6c, 0x75, 0x65, 0x12, 0x1d, 0x0a, 0x09, 0x69, 0x6e, 0x74, 0x5f, 0x76, 0x61, 0x6c, 0x75, - 0x65, 0x18, 0x05, 0x20, 0x01, 0x28, 0x03, 0x48, 0x00, 0x52, 0x08, 0x69, 0x6e, 0x74, 0x56, 0x61, - 0x6c, 0x75, 0x65, 0x12, 0x10, 0x0a, 0x03, 0x6b, 0x65, 0x79, 0x18, 0x06, 0x20, 0x01, 0x28, 0x09, - 0x52, 0x03, 0x6b, 0x65, 0x79, 0x12, 0x1c, 0x0a, 0x09, 0x6e, 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, - 0x63, 0x65, 0x18, 0x07, 0x20, 0x01, 0x28, 0x09, 0x52, 0x09, 0x6e, 0x61, 0x6d, 0x65, 0x73, 0x70, - 0x61, 0x63, 0x65, 0x42, 0x07, 0x0a, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x22, 0x39, 0x0a, 0x0b, - 0x44, 0x69, 0x73, 0x70, 0x6c, 0x61, 0x79, 0x44, 0x61, 0x74, 0x61, 0x12, 0x10, 0x0a, 0x03, 0x75, - 0x72, 0x6e, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x03, 0x75, 0x72, 0x6e, 0x12, 0x18, 0x0a, - 0x07, 0x70, 0x61, 0x79, 0x6c, 0x6f, 0x61, 0x64, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0c, 0x52, 0x07, - 0x70, 0x61, 0x79, 0x6c, 0x6f, 0x61, 0x64, 0x22, 0xd7, 0x07, 0x0a, 0x15, 0x4d, 0x65, 0x73, 0x73, - 0x61, 0x67, 0x65, 0x57, 0x69, 0x74, 0x68, 0x43, 0x6f, 0x6d, 0x70, 0x6f, 0x6e, 0x65, 0x6e, 0x74, - 0x73, 0x12, 0x4d, 0x0a, 0x0a, 0x63, 0x6f, 0x6d, 0x70, 0x6f, 0x6e, 0x65, 0x6e, 0x74, 0x73, 0x18, - 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x2d, 0x2e, 0x6f, 0x72, 0x67, 0x2e, 0x61, 0x70, 0x61, 0x63, - 0x68, 0x65, 0x2e, 0x62, 0x65, 0x61, 0x6d, 0x2e, 0x6d, 0x6f, 0x64, 0x65, 0x6c, 0x2e, 0x70, 0x69, - 0x70, 0x65, 0x6c, 0x69, 0x6e, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x43, 0x6f, 0x6d, 0x70, 0x6f, 0x6e, - 0x65, 0x6e, 0x74, 0x73, 0x52, 0x0a, 0x63, 0x6f, 0x6d, 0x70, 0x6f, 0x6e, 0x65, 0x6e, 0x74, 0x73, - 0x12, 0x40, 0x0a, 0x05, 0x63, 0x6f, 0x64, 0x65, 0x72, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, - 0x28, 0x2e, 0x6f, 0x72, 0x67, 0x2e, 0x61, 0x70, 0x61, 0x63, 0x68, 0x65, 0x2e, 0x62, 0x65, 0x61, - 0x6d, 0x2e, 0x6d, 0x6f, 0x64, 0x65, 0x6c, 0x2e, 0x70, 0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e, 0x65, - 0x2e, 0x76, 0x31, 0x2e, 0x43, 0x6f, 0x64, 0x65, 0x72, 0x48, 0x00, 0x52, 0x05, 0x63, 0x6f, 0x64, - 0x65, 0x72, 0x12, 0x5c, 0x0a, 0x0f, 0x63, 0x6f, 0x6d, 0x62, 0x69, 0x6e, 0x65, 0x5f, 0x70, 0x61, - 0x79, 0x6c, 0x6f, 0x61, 0x64, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x31, 0x2e, 0x6f, 0x72, - 0x67, 0x2e, 0x61, 0x70, 0x61, 0x63, 0x68, 0x65, 0x2e, 0x62, 0x65, 0x61, 0x6d, 0x2e, 0x6d, 0x6f, - 0x64, 0x65, 0x6c, 0x2e, 0x70, 0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e, 0x65, 0x2e, 0x76, 0x31, 0x2e, - 0x43, 0x6f, 0x6d, 0x62, 0x69, 0x6e, 0x65, 0x50, 0x61, 0x79, 0x6c, 0x6f, 0x61, 0x64, 0x48, 0x00, - 0x52, 0x0e, 0x63, 0x6f, 0x6d, 0x62, 0x69, 0x6e, 0x65, 0x50, 0x61, 0x79, 0x6c, 0x6f, 0x61, 0x64, - 0x12, 0x56, 0x0a, 0x0d, 0x66, 0x75, 0x6e, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x73, 0x70, 0x65, - 0x63, 0x18, 0x04, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x2f, 0x2e, 0x6f, 0x72, 0x67, 0x2e, 0x61, 0x70, - 0x61, 0x63, 0x68, 0x65, 0x2e, 0x62, 0x65, 0x61, 0x6d, 0x2e, 0x6d, 0x6f, 0x64, 0x65, 0x6c, 0x2e, - 0x70, 0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x46, 0x75, 0x6e, 0x63, - 0x74, 0x69, 0x6f, 0x6e, 0x53, 0x70, 0x65, 0x63, 0x48, 0x00, 0x52, 0x0c, 0x66, 0x75, 0x6e, 0x63, - 0x74, 0x69, 0x6f, 0x6e, 0x53, 0x70, 0x65, 0x63, 0x12, 0x57, 0x0a, 0x0e, 0x70, 0x61, 0x72, 0x5f, - 0x64, 0x6f, 0x5f, 0x70, 0x61, 0x79, 0x6c, 0x6f, 0x61, 0x64, 0x18, 0x06, 0x20, 0x01, 0x28, 0x0b, - 0x32, 0x2f, 0x2e, 0x6f, 0x72, 0x67, 0x2e, 0x61, 0x70, 0x61, 0x63, 0x68, 0x65, 0x2e, 0x62, 0x65, - 0x61, 0x6d, 0x2e, 0x6d, 0x6f, 0x64, 0x65, 0x6c, 0x2e, 0x70, 0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e, - 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x50, 0x61, 0x72, 0x44, 0x6f, 0x50, 0x61, 0x79, 0x6c, 0x6f, 0x61, - 0x64, 0x48, 0x00, 0x52, 0x0c, 0x70, 0x61, 0x72, 0x44, 0x6f, 0x50, 0x61, 0x79, 0x6c, 0x6f, 0x61, - 0x64, 0x12, 0x4f, 0x0a, 0x0a, 0x70, 0x74, 0x72, 0x61, 0x6e, 0x73, 0x66, 0x6f, 0x72, 0x6d, 0x18, - 0x07, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x2d, 0x2e, 0x6f, 0x72, 0x67, 0x2e, 0x61, 0x70, 0x61, 0x63, - 0x68, 0x65, 0x2e, 0x62, 0x65, 0x61, 0x6d, 0x2e, 0x6d, 0x6f, 0x64, 0x65, 0x6c, 0x2e, 0x70, 0x69, - 0x70, 0x65, 0x6c, 0x69, 0x6e, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x50, 0x54, 0x72, 0x61, 0x6e, 0x73, - 0x66, 0x6f, 0x72, 0x6d, 0x48, 0x00, 0x52, 0x0a, 0x70, 0x74, 0x72, 0x61, 0x6e, 0x73, 0x66, 0x6f, - 0x72, 0x6d, 0x12, 0x52, 0x0a, 0x0b, 0x70, 0x63, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, - 0x6e, 0x18, 0x08, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x2e, 0x2e, 0x6f, 0x72, 0x67, 0x2e, 0x61, 0x70, - 0x61, 0x63, 0x68, 0x65, 0x2e, 0x62, 0x65, 0x61, 0x6d, 0x2e, 0x6d, 0x6f, 0x64, 0x65, 0x6c, 0x2e, - 0x70, 0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x50, 0x43, 0x6f, 0x6c, - 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x48, 0x00, 0x52, 0x0b, 0x70, 0x63, 0x6f, 0x6c, 0x6c, - 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x53, 0x0a, 0x0c, 0x72, 0x65, 0x61, 0x64, 0x5f, 0x70, - 0x61, 0x79, 0x6c, 0x6f, 0x61, 0x64, 0x18, 0x09, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x2e, 0x2e, 0x6f, - 0x72, 0x67, 0x2e, 0x61, 0x70, 0x61, 0x63, 0x68, 0x65, 0x2e, 0x62, 0x65, 0x61, 0x6d, 0x2e, 0x6d, - 0x6f, 0x64, 0x65, 0x6c, 0x2e, 0x70, 0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e, 0x65, 0x2e, 0x76, 0x31, - 0x2e, 0x52, 0x65, 0x61, 0x64, 0x50, 0x61, 0x79, 0x6c, 0x6f, 0x61, 0x64, 0x48, 0x00, 0x52, 0x0b, - 0x72, 0x65, 0x61, 0x64, 0x50, 0x61, 0x79, 0x6c, 0x6f, 0x61, 0x64, 0x12, 0x4d, 0x0a, 0x0a, 0x73, - 0x69, 0x64, 0x65, 0x5f, 0x69, 0x6e, 0x70, 0x75, 0x74, 0x18, 0x0b, 0x20, 0x01, 0x28, 0x0b, 0x32, - 0x2c, 0x2e, 0x6f, 0x72, 0x67, 0x2e, 0x61, 0x70, 0x61, 0x63, 0x68, 0x65, 0x2e, 0x62, 0x65, 0x61, - 0x6d, 0x2e, 0x6d, 0x6f, 0x64, 0x65, 0x6c, 0x2e, 0x70, 0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e, 0x65, - 0x2e, 0x76, 0x31, 0x2e, 0x53, 0x69, 0x64, 0x65, 0x49, 0x6e, 0x70, 0x75, 0x74, 0x48, 0x00, 0x52, - 0x09, 0x73, 0x69, 0x64, 0x65, 0x49, 0x6e, 0x70, 0x75, 0x74, 0x12, 0x66, 0x0a, 0x13, 0x77, 0x69, - 0x6e, 0x64, 0x6f, 0x77, 0x5f, 0x69, 0x6e, 0x74, 0x6f, 0x5f, 0x70, 0x61, 0x79, 0x6c, 0x6f, 0x61, - 0x64, 0x18, 0x0c, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x34, 0x2e, 0x6f, 0x72, 0x67, 0x2e, 0x61, 0x70, - 0x61, 0x63, 0x68, 0x65, 0x2e, 0x62, 0x65, 0x61, 0x6d, 0x2e, 0x6d, 0x6f, 0x64, 0x65, 0x6c, 0x2e, - 0x70, 0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x57, 0x69, 0x6e, 0x64, - 0x6f, 0x77, 0x49, 0x6e, 0x74, 0x6f, 0x50, 0x61, 0x79, 0x6c, 0x6f, 0x61, 0x64, 0x48, 0x00, 0x52, - 0x11, 0x77, 0x69, 0x6e, 0x64, 0x6f, 0x77, 0x49, 0x6e, 0x74, 0x6f, 0x50, 0x61, 0x79, 0x6c, 0x6f, - 0x61, 0x64, 0x12, 0x65, 0x0a, 0x12, 0x77, 0x69, 0x6e, 0x64, 0x6f, 0x77, 0x69, 0x6e, 0x67, 0x5f, - 0x73, 0x74, 0x72, 0x61, 0x74, 0x65, 0x67, 0x79, 0x18, 0x0d, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x34, + 0x64, 0x6f, 0x3a, 0x73, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x5f, 0x69, 0x6e, 0x70, 0x75, 0x74, 0x3a, + 0x76, 0x31, 0x12, 0x51, 0x0a, 0x1a, 0x52, 0x45, 0x51, 0x55, 0x49, 0x52, 0x45, 0x53, 0x5f, 0x54, + 0x49, 0x4d, 0x45, 0x5f, 0x53, 0x4f, 0x52, 0x54, 0x45, 0x44, 0x5f, 0x49, 0x4e, 0x50, 0x55, 0x54, + 0x10, 0x03, 0x1a, 0x31, 0xa2, 0xb4, 0xfa, 0xc2, 0x05, 0x2b, 0x62, 0x65, 0x61, 0x6d, 0x3a, 0x72, + 0x65, 0x71, 0x75, 0x69, 0x72, 0x65, 0x6d, 0x65, 0x6e, 0x74, 0x3a, 0x70, 0x61, 0x72, 0x64, 0x6f, + 0x3a, 0x74, 0x69, 0x6d, 0x65, 0x5f, 0x73, 0x6f, 0x72, 0x74, 0x65, 0x64, 0x5f, 0x69, 0x6e, 0x70, + 0x75, 0x74, 0x3a, 0x76, 0x31, 0x12, 0x4d, 0x0a, 0x18, 0x52, 0x45, 0x51, 0x55, 0x49, 0x52, 0x45, + 0x53, 0x5f, 0x53, 0x50, 0x4c, 0x49, 0x54, 0x54, 0x41, 0x42, 0x4c, 0x45, 0x5f, 0x44, 0x4f, 0x46, + 0x4e, 0x10, 0x04, 0x1a, 0x2f, 0xa2, 0xb4, 0xfa, 0xc2, 0x05, 0x29, 0x62, 0x65, 0x61, 0x6d, 0x3a, + 0x72, 0x65, 0x71, 0x75, 0x69, 0x72, 0x65, 0x6d, 0x65, 0x6e, 0x74, 0x3a, 0x70, 0x61, 0x72, 0x64, + 0x6f, 0x3a, 0x73, 0x70, 0x6c, 0x69, 0x74, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x5f, 0x64, 0x6f, 0x66, + 0x6e, 0x3a, 0x76, 0x31, 0x12, 0x57, 0x0a, 0x1d, 0x52, 0x45, 0x51, 0x55, 0x49, 0x52, 0x45, 0x53, + 0x5f, 0x4f, 0x4e, 0x5f, 0x57, 0x49, 0x4e, 0x44, 0x4f, 0x57, 0x5f, 0x45, 0x58, 0x50, 0x49, 0x52, + 0x41, 0x54, 0x49, 0x4f, 0x4e, 0x10, 0x05, 0x1a, 0x34, 0xa2, 0xb4, 0xfa, 0xc2, 0x05, 0x2e, 0x62, + 0x65, 0x61, 0x6d, 0x3a, 0x72, 0x65, 0x71, 0x75, 0x69, 0x72, 0x65, 0x6d, 0x65, 0x6e, 0x74, 0x3a, + 0x70, 0x61, 0x72, 0x64, 0x6f, 0x3a, 0x6f, 0x6e, 0x5f, 0x77, 0x69, 0x6e, 0x64, 0x6f, 0x77, 0x5f, + 0x65, 0x78, 0x70, 0x69, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x3a, 0x76, 0x31, 0x22, 0x3a, 0x0a, + 0x0c, 0x46, 0x75, 0x6e, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x53, 0x70, 0x65, 0x63, 0x12, 0x10, 0x0a, + 0x03, 0x75, 0x72, 0x6e, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x03, 0x75, 0x72, 0x6e, 0x12, + 0x18, 0x0a, 0x07, 0x70, 0x61, 0x79, 0x6c, 0x6f, 0x61, 0x64, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0c, + 0x52, 0x07, 0x70, 0x61, 0x79, 0x6c, 0x6f, 0x61, 0x64, 0x22, 0x57, 0x0a, 0x13, 0x53, 0x74, 0x61, + 0x6e, 0x64, 0x61, 0x72, 0x64, 0x44, 0x69, 0x73, 0x70, 0x6c, 0x61, 0x79, 0x44, 0x61, 0x74, 0x61, + 0x22, 0x40, 0x0a, 0x0b, 0x44, 0x69, 0x73, 0x70, 0x6c, 0x61, 0x79, 0x44, 0x61, 0x74, 0x61, 0x12, + 0x31, 0x0a, 0x08, 0x4c, 0x41, 0x42, 0x45, 0x4c, 0x4c, 0x45, 0x44, 0x10, 0x00, 0x1a, 0x23, 0xa2, + 0xb4, 0xfa, 0xc2, 0x05, 0x1d, 0x62, 0x65, 0x61, 0x6d, 0x3a, 0x64, 0x69, 0x73, 0x70, 0x6c, 0x61, + 0x79, 0x5f, 0x64, 0x61, 0x74, 0x61, 0x3a, 0x6c, 0x61, 0x62, 0x65, 0x6c, 0x6c, 0x65, 0x64, 0x3a, + 0x76, 0x31, 0x22, 0xea, 0x01, 0x0a, 0x0f, 0x4c, 0x61, 0x62, 0x65, 0x6c, 0x6c, 0x65, 0x64, 0x50, + 0x61, 0x79, 0x6c, 0x6f, 0x61, 0x64, 0x12, 0x14, 0x0a, 0x05, 0x6c, 0x61, 0x62, 0x65, 0x6c, 0x18, + 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x05, 0x6c, 0x61, 0x62, 0x65, 0x6c, 0x12, 0x23, 0x0a, 0x0c, + 0x73, 0x74, 0x72, 0x69, 0x6e, 0x67, 0x5f, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x18, 0x02, 0x20, 0x01, + 0x28, 0x09, 0x48, 0x00, 0x52, 0x0b, 0x73, 0x74, 0x72, 0x69, 0x6e, 0x67, 0x56, 0x61, 0x6c, 0x75, + 0x65, 0x12, 0x1f, 0x0a, 0x0a, 0x62, 0x6f, 0x6f, 0x6c, 0x5f, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x18, + 0x03, 0x20, 0x01, 0x28, 0x08, 0x48, 0x00, 0x52, 0x09, 0x62, 0x6f, 0x6f, 0x6c, 0x56, 0x61, 0x6c, + 0x75, 0x65, 0x12, 0x23, 0x0a, 0x0c, 0x64, 0x6f, 0x75, 0x62, 0x6c, 0x65, 0x5f, 0x76, 0x61, 0x6c, + 0x75, 0x65, 0x18, 0x04, 0x20, 0x01, 0x28, 0x01, 0x48, 0x00, 0x52, 0x0b, 0x64, 0x6f, 0x75, 0x62, + 0x6c, 0x65, 0x56, 0x61, 0x6c, 0x75, 0x65, 0x12, 0x1d, 0x0a, 0x09, 0x69, 0x6e, 0x74, 0x5f, 0x76, + 0x61, 0x6c, 0x75, 0x65, 0x18, 0x05, 0x20, 0x01, 0x28, 0x03, 0x48, 0x00, 0x52, 0x08, 0x69, 0x6e, + 0x74, 0x56, 0x61, 0x6c, 0x75, 0x65, 0x12, 0x10, 0x0a, 0x03, 0x6b, 0x65, 0x79, 0x18, 0x06, 0x20, + 0x01, 0x28, 0x09, 0x52, 0x03, 0x6b, 0x65, 0x79, 0x12, 0x1c, 0x0a, 0x09, 0x6e, 0x61, 0x6d, 0x65, + 0x73, 0x70, 0x61, 0x63, 0x65, 0x18, 0x07, 0x20, 0x01, 0x28, 0x09, 0x52, 0x09, 0x6e, 0x61, 0x6d, + 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, 0x42, 0x07, 0x0a, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x22, + 0x39, 0x0a, 0x0b, 0x44, 0x69, 0x73, 0x70, 0x6c, 0x61, 0x79, 0x44, 0x61, 0x74, 0x61, 0x12, 0x10, + 0x0a, 0x03, 0x75, 0x72, 0x6e, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x03, 0x75, 0x72, 0x6e, + 0x12, 0x18, 0x0a, 0x07, 0x70, 0x61, 0x79, 0x6c, 0x6f, 0x61, 0x64, 0x18, 0x02, 0x20, 0x01, 0x28, + 0x0c, 0x52, 0x07, 0x70, 0x61, 0x79, 0x6c, 0x6f, 0x61, 0x64, 0x22, 0xd7, 0x07, 0x0a, 0x15, 0x4d, + 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x57, 0x69, 0x74, 0x68, 0x43, 0x6f, 0x6d, 0x70, 0x6f, 0x6e, + 0x65, 0x6e, 0x74, 0x73, 0x12, 0x4d, 0x0a, 0x0a, 0x63, 0x6f, 0x6d, 0x70, 0x6f, 0x6e, 0x65, 0x6e, + 0x74, 0x73, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x2d, 0x2e, 0x6f, 0x72, 0x67, 0x2e, 0x61, + 0x70, 0x61, 0x63, 0x68, 0x65, 0x2e, 0x62, 0x65, 0x61, 0x6d, 0x2e, 0x6d, 0x6f, 0x64, 0x65, 0x6c, + 0x2e, 0x70, 0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x43, 0x6f, 0x6d, + 0x70, 0x6f, 0x6e, 0x65, 0x6e, 0x74, 0x73, 0x52, 0x0a, 0x63, 0x6f, 0x6d, 0x70, 0x6f, 0x6e, 0x65, + 0x6e, 0x74, 0x73, 0x12, 0x40, 0x0a, 0x05, 0x63, 0x6f, 0x64, 0x65, 0x72, 0x18, 0x02, 0x20, 0x01, + 0x28, 0x0b, 0x32, 0x28, 0x2e, 0x6f, 0x72, 0x67, 0x2e, 0x61, 0x70, 0x61, 0x63, 0x68, 0x65, 0x2e, + 0x62, 0x65, 0x61, 0x6d, 0x2e, 0x6d, 0x6f, 0x64, 0x65, 0x6c, 0x2e, 0x70, 0x69, 0x70, 0x65, 0x6c, + 0x69, 0x6e, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x43, 0x6f, 0x64, 0x65, 0x72, 0x48, 0x00, 0x52, 0x05, + 0x63, 0x6f, 0x64, 0x65, 0x72, 0x12, 0x5c, 0x0a, 0x0f, 0x63, 0x6f, 0x6d, 0x62, 0x69, 0x6e, 0x65, + 0x5f, 0x70, 0x61, 0x79, 0x6c, 0x6f, 0x61, 0x64, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x31, 0x2e, 0x6f, 0x72, 0x67, 0x2e, 0x61, 0x70, 0x61, 0x63, 0x68, 0x65, 0x2e, 0x62, 0x65, 0x61, 0x6d, 0x2e, 0x6d, 0x6f, 0x64, 0x65, 0x6c, 0x2e, 0x70, 0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e, 0x65, 0x2e, - 0x76, 0x31, 0x2e, 0x57, 0x69, 0x6e, 0x64, 0x6f, 0x77, 0x69, 0x6e, 0x67, 0x53, 0x74, 0x72, 0x61, - 0x74, 0x65, 0x67, 0x79, 0x48, 0x00, 0x52, 0x11, 0x77, 0x69, 0x6e, 0x64, 0x6f, 0x77, 0x69, 0x6e, - 0x67, 0x53, 0x74, 0x72, 0x61, 0x74, 0x65, 0x67, 0x79, 0x42, 0x06, 0x0a, 0x04, 0x72, 0x6f, 0x6f, - 0x74, 0x22, 0xb6, 0x0a, 0x0a, 0x16, 0x45, 0x78, 0x65, 0x63, 0x75, 0x74, 0x61, 0x62, 0x6c, 0x65, - 0x53, 0x74, 0x61, 0x67, 0x65, 0x50, 0x61, 0x79, 0x6c, 0x6f, 0x61, 0x64, 0x12, 0x50, 0x0a, 0x0b, - 0x65, 0x6e, 0x76, 0x69, 0x72, 0x6f, 0x6e, 0x6d, 0x65, 0x6e, 0x74, 0x18, 0x01, 0x20, 0x01, 0x28, - 0x0b, 0x32, 0x2e, 0x2e, 0x6f, 0x72, 0x67, 0x2e, 0x61, 0x70, 0x61, 0x63, 0x68, 0x65, 0x2e, 0x62, + 0x76, 0x31, 0x2e, 0x43, 0x6f, 0x6d, 0x62, 0x69, 0x6e, 0x65, 0x50, 0x61, 0x79, 0x6c, 0x6f, 0x61, + 0x64, 0x48, 0x00, 0x52, 0x0e, 0x63, 0x6f, 0x6d, 0x62, 0x69, 0x6e, 0x65, 0x50, 0x61, 0x79, 0x6c, + 0x6f, 0x61, 0x64, 0x12, 0x56, 0x0a, 0x0d, 0x66, 0x75, 0x6e, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x5f, + 0x73, 0x70, 0x65, 0x63, 0x18, 0x04, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x2f, 0x2e, 0x6f, 0x72, 0x67, + 0x2e, 0x61, 0x70, 0x61, 0x63, 0x68, 0x65, 0x2e, 0x62, 0x65, 0x61, 0x6d, 0x2e, 0x6d, 0x6f, 0x64, + 0x65, 0x6c, 0x2e, 0x70, 0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x46, + 0x75, 0x6e, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x53, 0x70, 0x65, 0x63, 0x48, 0x00, 0x52, 0x0c, 0x66, + 0x75, 0x6e, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x53, 0x70, 0x65, 0x63, 0x12, 0x57, 0x0a, 0x0e, 0x70, + 0x61, 0x72, 0x5f, 0x64, 0x6f, 0x5f, 0x70, 0x61, 0x79, 0x6c, 0x6f, 0x61, 0x64, 0x18, 0x06, 0x20, + 0x01, 0x28, 0x0b, 0x32, 0x2f, 0x2e, 0x6f, 0x72, 0x67, 0x2e, 0x61, 0x70, 0x61, 0x63, 0x68, 0x65, + 0x2e, 0x62, 0x65, 0x61, 0x6d, 0x2e, 0x6d, 0x6f, 0x64, 0x65, 0x6c, 0x2e, 0x70, 0x69, 0x70, 0x65, + 0x6c, 0x69, 0x6e, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x50, 0x61, 0x72, 0x44, 0x6f, 0x50, 0x61, 0x79, + 0x6c, 0x6f, 0x61, 0x64, 0x48, 0x00, 0x52, 0x0c, 0x70, 0x61, 0x72, 0x44, 0x6f, 0x50, 0x61, 0x79, + 0x6c, 0x6f, 0x61, 0x64, 0x12, 0x4f, 0x0a, 0x0a, 0x70, 0x74, 0x72, 0x61, 0x6e, 0x73, 0x66, 0x6f, + 0x72, 0x6d, 0x18, 0x07, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x2d, 0x2e, 0x6f, 0x72, 0x67, 0x2e, 0x61, + 0x70, 0x61, 0x63, 0x68, 0x65, 0x2e, 0x62, 0x65, 0x61, 0x6d, 0x2e, 0x6d, 0x6f, 0x64, 0x65, 0x6c, + 0x2e, 0x70, 0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x50, 0x54, 0x72, + 0x61, 0x6e, 0x73, 0x66, 0x6f, 0x72, 0x6d, 0x48, 0x00, 0x52, 0x0a, 0x70, 0x74, 0x72, 0x61, 0x6e, + 0x73, 0x66, 0x6f, 0x72, 0x6d, 0x12, 0x52, 0x0a, 0x0b, 0x70, 0x63, 0x6f, 0x6c, 0x6c, 0x65, 0x63, + 0x74, 0x69, 0x6f, 0x6e, 0x18, 0x08, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x2e, 0x2e, 0x6f, 0x72, 0x67, + 0x2e, 0x61, 0x70, 0x61, 0x63, 0x68, 0x65, 0x2e, 0x62, 0x65, 0x61, 0x6d, 0x2e, 0x6d, 0x6f, 0x64, + 0x65, 0x6c, 0x2e, 0x70, 0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x50, + 0x43, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x48, 0x00, 0x52, 0x0b, 0x70, 0x63, + 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x53, 0x0a, 0x0c, 0x72, 0x65, 0x61, + 0x64, 0x5f, 0x70, 0x61, 0x79, 0x6c, 0x6f, 0x61, 0x64, 0x18, 0x09, 0x20, 0x01, 0x28, 0x0b, 0x32, + 0x2e, 0x2e, 0x6f, 0x72, 0x67, 0x2e, 0x61, 0x70, 0x61, 0x63, 0x68, 0x65, 0x2e, 0x62, 0x65, 0x61, + 0x6d, 0x2e, 0x6d, 0x6f, 0x64, 0x65, 0x6c, 0x2e, 0x70, 0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e, 0x65, + 0x2e, 0x76, 0x31, 0x2e, 0x52, 0x65, 0x61, 0x64, 0x50, 0x61, 0x79, 0x6c, 0x6f, 0x61, 0x64, 0x48, + 0x00, 0x52, 0x0b, 0x72, 0x65, 0x61, 0x64, 0x50, 0x61, 0x79, 0x6c, 0x6f, 0x61, 0x64, 0x12, 0x4d, + 0x0a, 0x0a, 0x73, 0x69, 0x64, 0x65, 0x5f, 0x69, 0x6e, 0x70, 0x75, 0x74, 0x18, 0x0b, 0x20, 0x01, + 0x28, 0x0b, 0x32, 0x2c, 0x2e, 0x6f, 0x72, 0x67, 0x2e, 0x61, 0x70, 0x61, 0x63, 0x68, 0x65, 0x2e, + 0x62, 0x65, 0x61, 0x6d, 0x2e, 0x6d, 0x6f, 0x64, 0x65, 0x6c, 0x2e, 0x70, 0x69, 0x70, 0x65, 0x6c, + 0x69, 0x6e, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x53, 0x69, 0x64, 0x65, 0x49, 0x6e, 0x70, 0x75, 0x74, + 0x48, 0x00, 0x52, 0x09, 0x73, 0x69, 0x64, 0x65, 0x49, 0x6e, 0x70, 0x75, 0x74, 0x12, 0x66, 0x0a, + 0x13, 0x77, 0x69, 0x6e, 0x64, 0x6f, 0x77, 0x5f, 0x69, 0x6e, 0x74, 0x6f, 0x5f, 0x70, 0x61, 0x79, + 0x6c, 0x6f, 0x61, 0x64, 0x18, 0x0c, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x34, 0x2e, 0x6f, 0x72, 0x67, + 0x2e, 0x61, 0x70, 0x61, 0x63, 0x68, 0x65, 0x2e, 0x62, 0x65, 0x61, 0x6d, 0x2e, 0x6d, 0x6f, 0x64, + 0x65, 0x6c, 0x2e, 0x70, 0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x57, + 0x69, 0x6e, 0x64, 0x6f, 0x77, 0x49, 0x6e, 0x74, 0x6f, 0x50, 0x61, 0x79, 0x6c, 0x6f, 0x61, 0x64, + 0x48, 0x00, 0x52, 0x11, 0x77, 0x69, 0x6e, 0x64, 0x6f, 0x77, 0x49, 0x6e, 0x74, 0x6f, 0x50, 0x61, + 0x79, 0x6c, 0x6f, 0x61, 0x64, 0x12, 0x65, 0x0a, 0x12, 0x77, 0x69, 0x6e, 0x64, 0x6f, 0x77, 0x69, + 0x6e, 0x67, 0x5f, 0x73, 0x74, 0x72, 0x61, 0x74, 0x65, 0x67, 0x79, 0x18, 0x0d, 0x20, 0x01, 0x28, + 0x0b, 0x32, 0x34, 0x2e, 0x6f, 0x72, 0x67, 0x2e, 0x61, 0x70, 0x61, 0x63, 0x68, 0x65, 0x2e, 0x62, 0x65, 0x61, 0x6d, 0x2e, 0x6d, 0x6f, 0x64, 0x65, 0x6c, 0x2e, 0x70, 0x69, 0x70, 0x65, 0x6c, 0x69, - 0x6e, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x45, 0x6e, 0x76, 0x69, 0x72, 0x6f, 0x6e, 0x6d, 0x65, 0x6e, - 0x74, 0x52, 0x0b, 0x65, 0x6e, 0x76, 0x69, 0x72, 0x6f, 0x6e, 0x6d, 0x65, 0x6e, 0x74, 0x12, 0x7a, - 0x0a, 0x13, 0x77, 0x69, 0x72, 0x65, 0x5f, 0x63, 0x6f, 0x64, 0x65, 0x72, 0x5f, 0x73, 0x65, 0x74, - 0x74, 0x69, 0x6e, 0x67, 0x73, 0x18, 0x09, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x4a, 0x2e, 0x6f, 0x72, - 0x67, 0x2e, 0x61, 0x70, 0x61, 0x63, 0x68, 0x65, 0x2e, 0x62, 0x65, 0x61, 0x6d, 0x2e, 0x6d, 0x6f, - 0x64, 0x65, 0x6c, 0x2e, 0x70, 0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e, 0x65, 0x2e, 0x76, 0x31, 0x2e, - 0x45, 0x78, 0x65, 0x63, 0x75, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x53, 0x74, 0x61, 0x67, 0x65, 0x50, - 0x61, 0x79, 0x6c, 0x6f, 0x61, 0x64, 0x2e, 0x57, 0x69, 0x72, 0x65, 0x43, 0x6f, 0x64, 0x65, 0x72, - 0x53, 0x65, 0x74, 0x74, 0x69, 0x6e, 0x67, 0x52, 0x11, 0x77, 0x69, 0x72, 0x65, 0x43, 0x6f, 0x64, - 0x65, 0x72, 0x53, 0x65, 0x74, 0x74, 0x69, 0x6e, 0x67, 0x73, 0x12, 0x14, 0x0a, 0x05, 0x69, 0x6e, - 0x70, 0x75, 0x74, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x05, 0x69, 0x6e, 0x70, 0x75, 0x74, - 0x12, 0x66, 0x0a, 0x0b, 0x73, 0x69, 0x64, 0x65, 0x5f, 0x69, 0x6e, 0x70, 0x75, 0x74, 0x73, 0x18, - 0x03, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x45, 0x2e, 0x6f, 0x72, 0x67, 0x2e, 0x61, 0x70, 0x61, 0x63, - 0x68, 0x65, 0x2e, 0x62, 0x65, 0x61, 0x6d, 0x2e, 0x6d, 0x6f, 0x64, 0x65, 0x6c, 0x2e, 0x70, 0x69, - 0x70, 0x65, 0x6c, 0x69, 0x6e, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x45, 0x78, 0x65, 0x63, 0x75, 0x74, - 0x61, 0x62, 0x6c, 0x65, 0x53, 0x74, 0x61, 0x67, 0x65, 0x50, 0x61, 0x79, 0x6c, 0x6f, 0x61, 0x64, - 0x2e, 0x53, 0x69, 0x64, 0x65, 0x49, 0x6e, 0x70, 0x75, 0x74, 0x49, 0x64, 0x52, 0x0a, 0x73, 0x69, - 0x64, 0x65, 0x49, 0x6e, 0x70, 0x75, 0x74, 0x73, 0x12, 0x1e, 0x0a, 0x0a, 0x74, 0x72, 0x61, 0x6e, - 0x73, 0x66, 0x6f, 0x72, 0x6d, 0x73, 0x18, 0x04, 0x20, 0x03, 0x28, 0x09, 0x52, 0x0a, 0x74, 0x72, - 0x61, 0x6e, 0x73, 0x66, 0x6f, 0x72, 0x6d, 0x73, 0x12, 0x18, 0x0a, 0x07, 0x6f, 0x75, 0x74, 0x70, - 0x75, 0x74, 0x73, 0x18, 0x05, 0x20, 0x03, 0x28, 0x09, 0x52, 0x07, 0x6f, 0x75, 0x74, 0x70, 0x75, - 0x74, 0x73, 0x12, 0x4d, 0x0a, 0x0a, 0x63, 0x6f, 0x6d, 0x70, 0x6f, 0x6e, 0x65, 0x6e, 0x74, 0x73, - 0x18, 0x06, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x2d, 0x2e, 0x6f, 0x72, 0x67, 0x2e, 0x61, 0x70, 0x61, - 0x63, 0x68, 0x65, 0x2e, 0x62, 0x65, 0x61, 0x6d, 0x2e, 0x6d, 0x6f, 0x64, 0x65, 0x6c, 0x2e, 0x70, - 0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x43, 0x6f, 0x6d, 0x70, 0x6f, - 0x6e, 0x65, 0x6e, 0x74, 0x73, 0x52, 0x0a, 0x63, 0x6f, 0x6d, 0x70, 0x6f, 0x6e, 0x65, 0x6e, 0x74, - 0x73, 0x12, 0x66, 0x0a, 0x0b, 0x75, 0x73, 0x65, 0x72, 0x5f, 0x73, 0x74, 0x61, 0x74, 0x65, 0x73, - 0x18, 0x07, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x45, 0x2e, 0x6f, 0x72, 0x67, 0x2e, 0x61, 0x70, 0x61, - 0x63, 0x68, 0x65, 0x2e, 0x62, 0x65, 0x61, 0x6d, 0x2e, 0x6d, 0x6f, 0x64, 0x65, 0x6c, 0x2e, 0x70, - 0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x45, 0x78, 0x65, 0x63, 0x75, - 0x74, 0x61, 0x62, 0x6c, 0x65, 0x53, 0x74, 0x61, 0x67, 0x65, 0x50, 0x61, 0x79, 0x6c, 0x6f, 0x61, - 0x64, 0x2e, 0x55, 0x73, 0x65, 0x72, 0x53, 0x74, 0x61, 0x74, 0x65, 0x49, 0x64, 0x52, 0x0a, 0x75, - 0x73, 0x65, 0x72, 0x53, 0x74, 0x61, 0x74, 0x65, 0x73, 0x12, 0x59, 0x0a, 0x06, 0x74, 0x69, 0x6d, - 0x65, 0x72, 0x73, 0x18, 0x08, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x41, 0x2e, 0x6f, 0x72, 0x67, 0x2e, + 0x6e, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x57, 0x69, 0x6e, 0x64, 0x6f, 0x77, 0x69, 0x6e, 0x67, 0x53, + 0x74, 0x72, 0x61, 0x74, 0x65, 0x67, 0x79, 0x48, 0x00, 0x52, 0x11, 0x77, 0x69, 0x6e, 0x64, 0x6f, + 0x77, 0x69, 0x6e, 0x67, 0x53, 0x74, 0x72, 0x61, 0x74, 0x65, 0x67, 0x79, 0x42, 0x06, 0x0a, 0x04, + 0x72, 0x6f, 0x6f, 0x74, 0x22, 0xb6, 0x0a, 0x0a, 0x16, 0x45, 0x78, 0x65, 0x63, 0x75, 0x74, 0x61, + 0x62, 0x6c, 0x65, 0x53, 0x74, 0x61, 0x67, 0x65, 0x50, 0x61, 0x79, 0x6c, 0x6f, 0x61, 0x64, 0x12, + 0x50, 0x0a, 0x0b, 0x65, 0x6e, 0x76, 0x69, 0x72, 0x6f, 0x6e, 0x6d, 0x65, 0x6e, 0x74, 0x18, 0x01, + 0x20, 0x01, 0x28, 0x0b, 0x32, 0x2e, 0x2e, 0x6f, 0x72, 0x67, 0x2e, 0x61, 0x70, 0x61, 0x63, 0x68, + 0x65, 0x2e, 0x62, 0x65, 0x61, 0x6d, 0x2e, 0x6d, 0x6f, 0x64, 0x65, 0x6c, 0x2e, 0x70, 0x69, 0x70, + 0x65, 0x6c, 0x69, 0x6e, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x45, 0x6e, 0x76, 0x69, 0x72, 0x6f, 0x6e, + 0x6d, 0x65, 0x6e, 0x74, 0x52, 0x0b, 0x65, 0x6e, 0x76, 0x69, 0x72, 0x6f, 0x6e, 0x6d, 0x65, 0x6e, + 0x74, 0x12, 0x7a, 0x0a, 0x13, 0x77, 0x69, 0x72, 0x65, 0x5f, 0x63, 0x6f, 0x64, 0x65, 0x72, 0x5f, + 0x73, 0x65, 0x74, 0x74, 0x69, 0x6e, 0x67, 0x73, 0x18, 0x09, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x4a, + 0x2e, 0x6f, 0x72, 0x67, 0x2e, 0x61, 0x70, 0x61, 0x63, 0x68, 0x65, 0x2e, 0x62, 0x65, 0x61, 0x6d, + 0x2e, 0x6d, 0x6f, 0x64, 0x65, 0x6c, 0x2e, 0x70, 0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e, 0x65, 0x2e, + 0x76, 0x31, 0x2e, 0x45, 0x78, 0x65, 0x63, 0x75, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x53, 0x74, 0x61, + 0x67, 0x65, 0x50, 0x61, 0x79, 0x6c, 0x6f, 0x61, 0x64, 0x2e, 0x57, 0x69, 0x72, 0x65, 0x43, 0x6f, + 0x64, 0x65, 0x72, 0x53, 0x65, 0x74, 0x74, 0x69, 0x6e, 0x67, 0x52, 0x11, 0x77, 0x69, 0x72, 0x65, + 0x43, 0x6f, 0x64, 0x65, 0x72, 0x53, 0x65, 0x74, 0x74, 0x69, 0x6e, 0x67, 0x73, 0x12, 0x14, 0x0a, + 0x05, 0x69, 0x6e, 0x70, 0x75, 0x74, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x05, 0x69, 0x6e, + 0x70, 0x75, 0x74, 0x12, 0x66, 0x0a, 0x0b, 0x73, 0x69, 0x64, 0x65, 0x5f, 0x69, 0x6e, 0x70, 0x75, + 0x74, 0x73, 0x18, 0x03, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x45, 0x2e, 0x6f, 0x72, 0x67, 0x2e, 0x61, + 0x70, 0x61, 0x63, 0x68, 0x65, 0x2e, 0x62, 0x65, 0x61, 0x6d, 0x2e, 0x6d, 0x6f, 0x64, 0x65, 0x6c, + 0x2e, 0x70, 0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x45, 0x78, 0x65, + 0x63, 0x75, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x53, 0x74, 0x61, 0x67, 0x65, 0x50, 0x61, 0x79, 0x6c, + 0x6f, 0x61, 0x64, 0x2e, 0x53, 0x69, 0x64, 0x65, 0x49, 0x6e, 0x70, 0x75, 0x74, 0x49, 0x64, 0x52, + 0x0a, 0x73, 0x69, 0x64, 0x65, 0x49, 0x6e, 0x70, 0x75, 0x74, 0x73, 0x12, 0x1e, 0x0a, 0x0a, 0x74, + 0x72, 0x61, 0x6e, 0x73, 0x66, 0x6f, 0x72, 0x6d, 0x73, 0x18, 0x04, 0x20, 0x03, 0x28, 0x09, 0x52, + 0x0a, 0x74, 0x72, 0x61, 0x6e, 0x73, 0x66, 0x6f, 0x72, 0x6d, 0x73, 0x12, 0x18, 0x0a, 0x07, 0x6f, + 0x75, 0x74, 0x70, 0x75, 0x74, 0x73, 0x18, 0x05, 0x20, 0x03, 0x28, 0x09, 0x52, 0x07, 0x6f, 0x75, + 0x74, 0x70, 0x75, 0x74, 0x73, 0x12, 0x4d, 0x0a, 0x0a, 0x63, 0x6f, 0x6d, 0x70, 0x6f, 0x6e, 0x65, + 0x6e, 0x74, 0x73, 0x18, 0x06, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x2d, 0x2e, 0x6f, 0x72, 0x67, 0x2e, + 0x61, 0x70, 0x61, 0x63, 0x68, 0x65, 0x2e, 0x62, 0x65, 0x61, 0x6d, 0x2e, 0x6d, 0x6f, 0x64, 0x65, + 0x6c, 0x2e, 0x70, 0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x43, 0x6f, + 0x6d, 0x70, 0x6f, 0x6e, 0x65, 0x6e, 0x74, 0x73, 0x52, 0x0a, 0x63, 0x6f, 0x6d, 0x70, 0x6f, 0x6e, + 0x65, 0x6e, 0x74, 0x73, 0x12, 0x66, 0x0a, 0x0b, 0x75, 0x73, 0x65, 0x72, 0x5f, 0x73, 0x74, 0x61, + 0x74, 0x65, 0x73, 0x18, 0x07, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x45, 0x2e, 0x6f, 0x72, 0x67, 0x2e, 0x61, 0x70, 0x61, 0x63, 0x68, 0x65, 0x2e, 0x62, 0x65, 0x61, 0x6d, 0x2e, 0x6d, 0x6f, 0x64, 0x65, 0x6c, 0x2e, 0x70, 0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x45, 0x78, 0x65, 0x63, 0x75, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x53, 0x74, 0x61, 0x67, 0x65, 0x50, 0x61, 0x79, - 0x6c, 0x6f, 0x61, 0x64, 0x2e, 0x54, 0x69, 0x6d, 0x65, 0x72, 0x49, 0x64, 0x52, 0x06, 0x74, 0x69, - 0x6d, 0x65, 0x72, 0x73, 0x12, 0x6d, 0x0a, 0x0d, 0x74, 0x69, 0x6d, 0x65, 0x72, 0x46, 0x61, 0x6d, - 0x69, 0x6c, 0x69, 0x65, 0x73, 0x18, 0x0a, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x47, 0x2e, 0x6f, 0x72, - 0x67, 0x2e, 0x61, 0x70, 0x61, 0x63, 0x68, 0x65, 0x2e, 0x62, 0x65, 0x61, 0x6d, 0x2e, 0x6d, 0x6f, - 0x64, 0x65, 0x6c, 0x2e, 0x70, 0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e, 0x65, 0x2e, 0x76, 0x31, 0x2e, - 0x45, 0x78, 0x65, 0x63, 0x75, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x53, 0x74, 0x61, 0x67, 0x65, 0x50, - 0x61, 0x79, 0x6c, 0x6f, 0x61, 0x64, 0x2e, 0x54, 0x69, 0x6d, 0x65, 0x72, 0x46, 0x61, 0x6d, 0x69, - 0x6c, 0x79, 0x49, 0x64, 0x52, 0x0d, 0x74, 0x69, 0x6d, 0x65, 0x72, 0x46, 0x61, 0x6d, 0x69, 0x6c, - 0x69, 0x65, 0x73, 0x1a, 0x4f, 0x0a, 0x0b, 0x53, 0x69, 0x64, 0x65, 0x49, 0x6e, 0x70, 0x75, 0x74, - 0x49, 0x64, 0x12, 0x21, 0x0a, 0x0c, 0x74, 0x72, 0x61, 0x6e, 0x73, 0x66, 0x6f, 0x72, 0x6d, 0x5f, - 0x69, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0b, 0x74, 0x72, 0x61, 0x6e, 0x73, 0x66, - 0x6f, 0x72, 0x6d, 0x49, 0x64, 0x12, 0x1d, 0x0a, 0x0a, 0x6c, 0x6f, 0x63, 0x61, 0x6c, 0x5f, 0x6e, - 0x61, 0x6d, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x09, 0x6c, 0x6f, 0x63, 0x61, 0x6c, - 0x4e, 0x61, 0x6d, 0x65, 0x1a, 0x4f, 0x0a, 0x0b, 0x55, 0x73, 0x65, 0x72, 0x53, 0x74, 0x61, 0x74, - 0x65, 0x49, 0x64, 0x12, 0x21, 0x0a, 0x0c, 0x74, 0x72, 0x61, 0x6e, 0x73, 0x66, 0x6f, 0x72, 0x6d, - 0x5f, 0x69, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0b, 0x74, 0x72, 0x61, 0x6e, 0x73, - 0x66, 0x6f, 0x72, 0x6d, 0x49, 0x64, 0x12, 0x1d, 0x0a, 0x0a, 0x6c, 0x6f, 0x63, 0x61, 0x6c, 0x5f, - 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x09, 0x6c, 0x6f, 0x63, 0x61, - 0x6c, 0x4e, 0x61, 0x6d, 0x65, 0x1a, 0x4b, 0x0a, 0x07, 0x54, 0x69, 0x6d, 0x65, 0x72, 0x49, 0x64, - 0x12, 0x21, 0x0a, 0x0c, 0x74, 0x72, 0x61, 0x6e, 0x73, 0x66, 0x6f, 0x72, 0x6d, 0x5f, 0x69, 0x64, - 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0b, 0x74, 0x72, 0x61, 0x6e, 0x73, 0x66, 0x6f, 0x72, - 0x6d, 0x49, 0x64, 0x12, 0x1d, 0x0a, 0x0a, 0x6c, 0x6f, 0x63, 0x61, 0x6c, 0x5f, 0x6e, 0x61, 0x6d, - 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x09, 0x6c, 0x6f, 0x63, 0x61, 0x6c, 0x4e, 0x61, - 0x6d, 0x65, 0x1a, 0x51, 0x0a, 0x0d, 0x54, 0x69, 0x6d, 0x65, 0x72, 0x46, 0x61, 0x6d, 0x69, 0x6c, - 0x79, 0x49, 0x64, 0x12, 0x21, 0x0a, 0x0c, 0x74, 0x72, 0x61, 0x6e, 0x73, 0x66, 0x6f, 0x72, 0x6d, + 0x6c, 0x6f, 0x61, 0x64, 0x2e, 0x55, 0x73, 0x65, 0x72, 0x53, 0x74, 0x61, 0x74, 0x65, 0x49, 0x64, + 0x52, 0x0a, 0x75, 0x73, 0x65, 0x72, 0x53, 0x74, 0x61, 0x74, 0x65, 0x73, 0x12, 0x59, 0x0a, 0x06, + 0x74, 0x69, 0x6d, 0x65, 0x72, 0x73, 0x18, 0x08, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x41, 0x2e, 0x6f, + 0x72, 0x67, 0x2e, 0x61, 0x70, 0x61, 0x63, 0x68, 0x65, 0x2e, 0x62, 0x65, 0x61, 0x6d, 0x2e, 0x6d, + 0x6f, 0x64, 0x65, 0x6c, 0x2e, 0x70, 0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e, 0x65, 0x2e, 0x76, 0x31, + 0x2e, 0x45, 0x78, 0x65, 0x63, 0x75, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x53, 0x74, 0x61, 0x67, 0x65, + 0x50, 0x61, 0x79, 0x6c, 0x6f, 0x61, 0x64, 0x2e, 0x54, 0x69, 0x6d, 0x65, 0x72, 0x49, 0x64, 0x52, + 0x06, 0x74, 0x69, 0x6d, 0x65, 0x72, 0x73, 0x12, 0x6d, 0x0a, 0x0d, 0x74, 0x69, 0x6d, 0x65, 0x72, + 0x46, 0x61, 0x6d, 0x69, 0x6c, 0x69, 0x65, 0x73, 0x18, 0x0a, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x47, + 0x2e, 0x6f, 0x72, 0x67, 0x2e, 0x61, 0x70, 0x61, 0x63, 0x68, 0x65, 0x2e, 0x62, 0x65, 0x61, 0x6d, + 0x2e, 0x6d, 0x6f, 0x64, 0x65, 0x6c, 0x2e, 0x70, 0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e, 0x65, 0x2e, + 0x76, 0x31, 0x2e, 0x45, 0x78, 0x65, 0x63, 0x75, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x53, 0x74, 0x61, + 0x67, 0x65, 0x50, 0x61, 0x79, 0x6c, 0x6f, 0x61, 0x64, 0x2e, 0x54, 0x69, 0x6d, 0x65, 0x72, 0x46, + 0x61, 0x6d, 0x69, 0x6c, 0x79, 0x49, 0x64, 0x52, 0x0d, 0x74, 0x69, 0x6d, 0x65, 0x72, 0x46, 0x61, + 0x6d, 0x69, 0x6c, 0x69, 0x65, 0x73, 0x1a, 0x4f, 0x0a, 0x0b, 0x53, 0x69, 0x64, 0x65, 0x49, 0x6e, + 0x70, 0x75, 0x74, 0x49, 0x64, 0x12, 0x21, 0x0a, 0x0c, 0x74, 0x72, 0x61, 0x6e, 0x73, 0x66, 0x6f, + 0x72, 0x6d, 0x5f, 0x69, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0b, 0x74, 0x72, 0x61, + 0x6e, 0x73, 0x66, 0x6f, 0x72, 0x6d, 0x49, 0x64, 0x12, 0x1d, 0x0a, 0x0a, 0x6c, 0x6f, 0x63, 0x61, + 0x6c, 0x5f, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x09, 0x6c, 0x6f, + 0x63, 0x61, 0x6c, 0x4e, 0x61, 0x6d, 0x65, 0x1a, 0x4f, 0x0a, 0x0b, 0x55, 0x73, 0x65, 0x72, 0x53, + 0x74, 0x61, 0x74, 0x65, 0x49, 0x64, 0x12, 0x21, 0x0a, 0x0c, 0x74, 0x72, 0x61, 0x6e, 0x73, 0x66, + 0x6f, 0x72, 0x6d, 0x5f, 0x69, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0b, 0x74, 0x72, + 0x61, 0x6e, 0x73, 0x66, 0x6f, 0x72, 0x6d, 0x49, 0x64, 0x12, 0x1d, 0x0a, 0x0a, 0x6c, 0x6f, 0x63, + 0x61, 0x6c, 0x5f, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x09, 0x6c, + 0x6f, 0x63, 0x61, 0x6c, 0x4e, 0x61, 0x6d, 0x65, 0x1a, 0x4b, 0x0a, 0x07, 0x54, 0x69, 0x6d, 0x65, + 0x72, 0x49, 0x64, 0x12, 0x21, 0x0a, 0x0c, 0x74, 0x72, 0x61, 0x6e, 0x73, 0x66, 0x6f, 0x72, 0x6d, 0x5f, 0x69, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0b, 0x74, 0x72, 0x61, 0x6e, 0x73, 0x66, 0x6f, 0x72, 0x6d, 0x49, 0x64, 0x12, 0x1d, 0x0a, 0x0a, 0x6c, 0x6f, 0x63, 0x61, 0x6c, 0x5f, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x09, 0x6c, 0x6f, 0x63, 0x61, - 0x6c, 0x4e, 0x61, 0x6d, 0x65, 0x1a, 0xd2, 0x01, 0x0a, 0x10, 0x57, 0x69, 0x72, 0x65, 0x43, 0x6f, - 0x64, 0x65, 0x72, 0x53, 0x65, 0x74, 0x74, 0x69, 0x6e, 0x67, 0x12, 0x10, 0x0a, 0x03, 0x75, 0x72, - 0x6e, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x03, 0x75, 0x72, 0x6e, 0x12, 0x18, 0x0a, 0x07, - 0x70, 0x61, 0x79, 0x6c, 0x6f, 0x61, 0x64, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0c, 0x52, 0x07, 0x70, - 0x61, 0x79, 0x6c, 0x6f, 0x61, 0x64, 0x12, 0x2d, 0x0a, 0x12, 0x69, 0x6e, 0x70, 0x75, 0x74, 0x5f, - 0x6f, 0x72, 0x5f, 0x6f, 0x75, 0x74, 0x70, 0x75, 0x74, 0x5f, 0x69, 0x64, 0x18, 0x03, 0x20, 0x01, - 0x28, 0x09, 0x48, 0x00, 0x52, 0x0f, 0x69, 0x6e, 0x70, 0x75, 0x74, 0x4f, 0x72, 0x4f, 0x75, 0x74, - 0x70, 0x75, 0x74, 0x49, 0x64, 0x12, 0x59, 0x0a, 0x05, 0x74, 0x69, 0x6d, 0x65, 0x72, 0x18, 0x04, - 0x20, 0x01, 0x28, 0x0b, 0x32, 0x41, 0x2e, 0x6f, 0x72, 0x67, 0x2e, 0x61, 0x70, 0x61, 0x63, 0x68, + 0x6c, 0x4e, 0x61, 0x6d, 0x65, 0x1a, 0x51, 0x0a, 0x0d, 0x54, 0x69, 0x6d, 0x65, 0x72, 0x46, 0x61, + 0x6d, 0x69, 0x6c, 0x79, 0x49, 0x64, 0x12, 0x21, 0x0a, 0x0c, 0x74, 0x72, 0x61, 0x6e, 0x73, 0x66, + 0x6f, 0x72, 0x6d, 0x5f, 0x69, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0b, 0x74, 0x72, + 0x61, 0x6e, 0x73, 0x66, 0x6f, 0x72, 0x6d, 0x49, 0x64, 0x12, 0x1d, 0x0a, 0x0a, 0x6c, 0x6f, 0x63, + 0x61, 0x6c, 0x5f, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x09, 0x6c, + 0x6f, 0x63, 0x61, 0x6c, 0x4e, 0x61, 0x6d, 0x65, 0x1a, 0xd2, 0x01, 0x0a, 0x10, 0x57, 0x69, 0x72, + 0x65, 0x43, 0x6f, 0x64, 0x65, 0x72, 0x53, 0x65, 0x74, 0x74, 0x69, 0x6e, 0x67, 0x12, 0x10, 0x0a, + 0x03, 0x75, 0x72, 0x6e, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x03, 0x75, 0x72, 0x6e, 0x12, + 0x18, 0x0a, 0x07, 0x70, 0x61, 0x79, 0x6c, 0x6f, 0x61, 0x64, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0c, + 0x52, 0x07, 0x70, 0x61, 0x79, 0x6c, 0x6f, 0x61, 0x64, 0x12, 0x2d, 0x0a, 0x12, 0x69, 0x6e, 0x70, + 0x75, 0x74, 0x5f, 0x6f, 0x72, 0x5f, 0x6f, 0x75, 0x74, 0x70, 0x75, 0x74, 0x5f, 0x69, 0x64, 0x18, + 0x03, 0x20, 0x01, 0x28, 0x09, 0x48, 0x00, 0x52, 0x0f, 0x69, 0x6e, 0x70, 0x75, 0x74, 0x4f, 0x72, + 0x4f, 0x75, 0x74, 0x70, 0x75, 0x74, 0x49, 0x64, 0x12, 0x59, 0x0a, 0x05, 0x74, 0x69, 0x6d, 0x65, + 0x72, 0x18, 0x04, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x41, 0x2e, 0x6f, 0x72, 0x67, 0x2e, 0x61, 0x70, + 0x61, 0x63, 0x68, 0x65, 0x2e, 0x62, 0x65, 0x61, 0x6d, 0x2e, 0x6d, 0x6f, 0x64, 0x65, 0x6c, 0x2e, + 0x70, 0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x45, 0x78, 0x65, 0x63, + 0x75, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x53, 0x74, 0x61, 0x67, 0x65, 0x50, 0x61, 0x79, 0x6c, 0x6f, + 0x61, 0x64, 0x2e, 0x54, 0x69, 0x6d, 0x65, 0x72, 0x49, 0x64, 0x48, 0x00, 0x52, 0x05, 0x74, 0x69, + 0x6d, 0x65, 0x72, 0x42, 0x08, 0x0a, 0x06, 0x74, 0x61, 0x72, 0x67, 0x65, 0x74, 0x22, 0xc2, 0x01, + 0x0a, 0x15, 0x53, 0x74, 0x61, 0x6e, 0x64, 0x61, 0x72, 0x64, 0x52, 0x65, 0x73, 0x6f, 0x75, 0x72, + 0x63, 0x65, 0x48, 0x69, 0x6e, 0x74, 0x73, 0x22, 0xa8, 0x01, 0x0a, 0x04, 0x45, 0x6e, 0x75, 0x6d, + 0x12, 0x34, 0x0a, 0x0b, 0x41, 0x43, 0x43, 0x45, 0x4c, 0x45, 0x52, 0x41, 0x54, 0x4f, 0x52, 0x10, + 0x00, 0x1a, 0x23, 0xa2, 0xb4, 0xfa, 0xc2, 0x05, 0x1d, 0x62, 0x65, 0x61, 0x6d, 0x3a, 0x72, 0x65, + 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x73, 0x3a, 0x61, 0x63, 0x63, 0x65, 0x6c, 0x65, 0x72, 0x61, + 0x74, 0x6f, 0x72, 0x3a, 0x76, 0x31, 0x12, 0x38, 0x0a, 0x0d, 0x4d, 0x49, 0x4e, 0x5f, 0x52, 0x41, + 0x4d, 0x5f, 0x42, 0x59, 0x54, 0x45, 0x53, 0x10, 0x01, 0x1a, 0x25, 0xa2, 0xb4, 0xfa, 0xc2, 0x05, + 0x1f, 0x62, 0x65, 0x61, 0x6d, 0x3a, 0x72, 0x65, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x73, 0x3a, + 0x6d, 0x69, 0x6e, 0x5f, 0x72, 0x61, 0x6d, 0x5f, 0x62, 0x79, 0x74, 0x65, 0x73, 0x3a, 0x76, 0x31, + 0x12, 0x30, 0x0a, 0x09, 0x43, 0x50, 0x55, 0x5f, 0x43, 0x4f, 0x55, 0x4e, 0x54, 0x10, 0x02, 0x1a, + 0x21, 0xa2, 0xb4, 0xfa, 0xc2, 0x05, 0x1b, 0x62, 0x65, 0x61, 0x6d, 0x3a, 0x72, 0x65, 0x73, 0x6f, + 0x75, 0x72, 0x63, 0x65, 0x73, 0x3a, 0x63, 0x70, 0x75, 0x5f, 0x63, 0x6f, 0x75, 0x6e, 0x74, 0x3a, + 0x76, 0x31, 0x32, 0x8f, 0x01, 0x0a, 0x11, 0x54, 0x65, 0x73, 0x74, 0x53, 0x74, 0x72, 0x65, 0x61, + 0x6d, 0x53, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, 0x12, 0x7a, 0x0a, 0x06, 0x45, 0x76, 0x65, 0x6e, + 0x74, 0x73, 0x12, 0x30, 0x2e, 0x6f, 0x72, 0x67, 0x2e, 0x61, 0x70, 0x61, 0x63, 0x68, 0x65, 0x2e, + 0x62, 0x65, 0x61, 0x6d, 0x2e, 0x6d, 0x6f, 0x64, 0x65, 0x6c, 0x2e, 0x70, 0x69, 0x70, 0x65, 0x6c, + 0x69, 0x6e, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x45, 0x76, 0x65, 0x6e, 0x74, 0x73, 0x52, 0x65, 0x71, + 0x75, 0x65, 0x73, 0x74, 0x1a, 0x3a, 0x2e, 0x6f, 0x72, 0x67, 0x2e, 0x61, 0x70, 0x61, 0x63, 0x68, 0x65, 0x2e, 0x62, 0x65, 0x61, 0x6d, 0x2e, 0x6d, 0x6f, 0x64, 0x65, 0x6c, 0x2e, 0x70, 0x69, 0x70, - 0x65, 0x6c, 0x69, 0x6e, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x45, 0x78, 0x65, 0x63, 0x75, 0x74, 0x61, - 0x62, 0x6c, 0x65, 0x53, 0x74, 0x61, 0x67, 0x65, 0x50, 0x61, 0x79, 0x6c, 0x6f, 0x61, 0x64, 0x2e, - 0x54, 0x69, 0x6d, 0x65, 0x72, 0x49, 0x64, 0x48, 0x00, 0x52, 0x05, 0x74, 0x69, 0x6d, 0x65, 0x72, - 0x42, 0x08, 0x0a, 0x06, 0x74, 0x61, 0x72, 0x67, 0x65, 0x74, 0x22, 0xc2, 0x01, 0x0a, 0x15, 0x53, - 0x74, 0x61, 0x6e, 0x64, 0x61, 0x72, 0x64, 0x52, 0x65, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x48, - 0x69, 0x6e, 0x74, 0x73, 0x22, 0xa8, 0x01, 0x0a, 0x04, 0x45, 0x6e, 0x75, 0x6d, 0x12, 0x34, 0x0a, - 0x0b, 0x41, 0x43, 0x43, 0x45, 0x4c, 0x45, 0x52, 0x41, 0x54, 0x4f, 0x52, 0x10, 0x00, 0x1a, 0x23, - 0xa2, 0xb4, 0xfa, 0xc2, 0x05, 0x1d, 0x62, 0x65, 0x61, 0x6d, 0x3a, 0x72, 0x65, 0x73, 0x6f, 0x75, - 0x72, 0x63, 0x65, 0x73, 0x3a, 0x61, 0x63, 0x63, 0x65, 0x6c, 0x65, 0x72, 0x61, 0x74, 0x6f, 0x72, - 0x3a, 0x76, 0x31, 0x12, 0x38, 0x0a, 0x0d, 0x4d, 0x49, 0x4e, 0x5f, 0x52, 0x41, 0x4d, 0x5f, 0x42, - 0x59, 0x54, 0x45, 0x53, 0x10, 0x01, 0x1a, 0x25, 0xa2, 0xb4, 0xfa, 0xc2, 0x05, 0x1f, 0x62, 0x65, - 0x61, 0x6d, 0x3a, 0x72, 0x65, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x73, 0x3a, 0x6d, 0x69, 0x6e, - 0x5f, 0x72, 0x61, 0x6d, 0x5f, 0x62, 0x79, 0x74, 0x65, 0x73, 0x3a, 0x76, 0x31, 0x12, 0x30, 0x0a, - 0x09, 0x43, 0x50, 0x55, 0x5f, 0x43, 0x4f, 0x55, 0x4e, 0x54, 0x10, 0x02, 0x1a, 0x21, 0xa2, 0xb4, - 0xfa, 0xc2, 0x05, 0x1b, 0x62, 0x65, 0x61, 0x6d, 0x3a, 0x72, 0x65, 0x73, 0x6f, 0x75, 0x72, 0x63, - 0x65, 0x73, 0x3a, 0x63, 0x70, 0x75, 0x5f, 0x63, 0x6f, 0x75, 0x6e, 0x74, 0x3a, 0x76, 0x31, 0x32, - 0x8f, 0x01, 0x0a, 0x11, 0x54, 0x65, 0x73, 0x74, 0x53, 0x74, 0x72, 0x65, 0x61, 0x6d, 0x53, 0x65, - 0x72, 0x76, 0x69, 0x63, 0x65, 0x12, 0x7a, 0x0a, 0x06, 0x45, 0x76, 0x65, 0x6e, 0x74, 0x73, 0x12, - 0x30, 0x2e, 0x6f, 0x72, 0x67, 0x2e, 0x61, 0x70, 0x61, 0x63, 0x68, 0x65, 0x2e, 0x62, 0x65, 0x61, - 0x6d, 0x2e, 0x6d, 0x6f, 0x64, 0x65, 0x6c, 0x2e, 0x70, 0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e, 0x65, - 0x2e, 0x76, 0x31, 0x2e, 0x45, 0x76, 0x65, 0x6e, 0x74, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, - 0x74, 0x1a, 0x3a, 0x2e, 0x6f, 0x72, 0x67, 0x2e, 0x61, 0x70, 0x61, 0x63, 0x68, 0x65, 0x2e, 0x62, + 0x65, 0x6c, 0x69, 0x6e, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x54, 0x65, 0x73, 0x74, 0x53, 0x74, 0x72, + 0x65, 0x61, 0x6d, 0x50, 0x61, 0x79, 0x6c, 0x6f, 0x61, 0x64, 0x2e, 0x45, 0x76, 0x65, 0x6e, 0x74, + 0x22, 0x00, 0x30, 0x01, 0x3a, 0x3f, 0x0a, 0x08, 0x62, 0x65, 0x61, 0x6d, 0x5f, 0x75, 0x72, 0x6e, + 0x12, 0x21, 0x2e, 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, + 0x75, 0x66, 0x2e, 0x45, 0x6e, 0x75, 0x6d, 0x56, 0x61, 0x6c, 0x75, 0x65, 0x4f, 0x70, 0x74, 0x69, + 0x6f, 0x6e, 0x73, 0x18, 0xc4, 0xa6, 0xaf, 0x58, 0x20, 0x01, 0x28, 0x09, 0x52, 0x07, 0x62, 0x65, + 0x61, 0x6d, 0x55, 0x72, 0x6e, 0x3a, 0x49, 0x0a, 0x0d, 0x62, 0x65, 0x61, 0x6d, 0x5f, 0x63, 0x6f, + 0x6e, 0x73, 0x74, 0x61, 0x6e, 0x74, 0x12, 0x21, 0x2e, 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2e, + 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66, 0x2e, 0x45, 0x6e, 0x75, 0x6d, 0x56, 0x61, 0x6c, + 0x75, 0x65, 0x4f, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x18, 0xc5, 0xa6, 0xaf, 0x58, 0x20, 0x01, + 0x28, 0x09, 0x52, 0x0c, 0x62, 0x65, 0x61, 0x6d, 0x43, 0x6f, 0x6e, 0x73, 0x74, 0x61, 0x6e, 0x74, + 0x42, 0x78, 0x0a, 0x21, 0x6f, 0x72, 0x67, 0x2e, 0x61, 0x70, 0x61, 0x63, 0x68, 0x65, 0x2e, 0x62, 0x65, 0x61, 0x6d, 0x2e, 0x6d, 0x6f, 0x64, 0x65, 0x6c, 0x2e, 0x70, 0x69, 0x70, 0x65, 0x6c, 0x69, - 0x6e, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x54, 0x65, 0x73, 0x74, 0x53, 0x74, 0x72, 0x65, 0x61, 0x6d, - 0x50, 0x61, 0x79, 0x6c, 0x6f, 0x61, 0x64, 0x2e, 0x45, 0x76, 0x65, 0x6e, 0x74, 0x22, 0x00, 0x30, - 0x01, 0x3a, 0x3f, 0x0a, 0x08, 0x62, 0x65, 0x61, 0x6d, 0x5f, 0x75, 0x72, 0x6e, 0x12, 0x21, 0x2e, - 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66, 0x2e, - 0x45, 0x6e, 0x75, 0x6d, 0x56, 0x61, 0x6c, 0x75, 0x65, 0x4f, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x73, - 0x18, 0xc4, 0xa6, 0xaf, 0x58, 0x20, 0x01, 0x28, 0x09, 0x52, 0x07, 0x62, 0x65, 0x61, 0x6d, 0x55, - 0x72, 0x6e, 0x3a, 0x49, 0x0a, 0x0d, 0x62, 0x65, 0x61, 0x6d, 0x5f, 0x63, 0x6f, 0x6e, 0x73, 0x74, - 0x61, 0x6e, 0x74, 0x12, 0x21, 0x2e, 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2e, 0x70, 0x72, 0x6f, - 0x74, 0x6f, 0x62, 0x75, 0x66, 0x2e, 0x45, 0x6e, 0x75, 0x6d, 0x56, 0x61, 0x6c, 0x75, 0x65, 0x4f, - 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x18, 0xc5, 0xa6, 0xaf, 0x58, 0x20, 0x01, 0x28, 0x09, 0x52, - 0x0c, 0x62, 0x65, 0x61, 0x6d, 0x43, 0x6f, 0x6e, 0x73, 0x74, 0x61, 0x6e, 0x74, 0x42, 0x78, 0x0a, - 0x21, 0x6f, 0x72, 0x67, 0x2e, 0x61, 0x70, 0x61, 0x63, 0x68, 0x65, 0x2e, 0x62, 0x65, 0x61, 0x6d, - 0x2e, 0x6d, 0x6f, 0x64, 0x65, 0x6c, 0x2e, 0x70, 0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e, 0x65, 0x2e, - 0x76, 0x31, 0x42, 0x09, 0x52, 0x75, 0x6e, 0x6e, 0x65, 0x72, 0x41, 0x70, 0x69, 0x5a, 0x48, 0x67, - 0x69, 0x74, 0x68, 0x75, 0x62, 0x2e, 0x63, 0x6f, 0x6d, 0x2f, 0x61, 0x70, 0x61, 0x63, 0x68, 0x65, - 0x2f, 0x62, 0x65, 0x61, 0x6d, 0x2f, 0x73, 0x64, 0x6b, 0x73, 0x2f, 0x76, 0x32, 0x2f, 0x67, 0x6f, - 0x2f, 0x70, 0x6b, 0x67, 0x2f, 0x62, 0x65, 0x61, 0x6d, 0x2f, 0x6d, 0x6f, 0x64, 0x65, 0x6c, 0x2f, - 0x70, 0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e, 0x65, 0x5f, 0x76, 0x31, 0x3b, 0x70, 0x69, 0x70, 0x65, - 0x6c, 0x69, 0x6e, 0x65, 0x5f, 0x76, 0x31, 0x62, 0x06, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x33, -} - -var ( - file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_rawDescOnce sync.Once - file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_rawDescData = file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_rawDesc -) - -func file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_rawDescGZIP() []byte { - file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_rawDescOnce.Do(func() { - file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_rawDescData = protoimpl.X.CompressGZIP(file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_rawDescData) - }) - return file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_rawDescData + 0x6e, 0x65, 0x2e, 0x76, 0x31, 0x42, 0x09, 0x52, 0x75, 0x6e, 0x6e, 0x65, 0x72, 0x41, 0x70, 0x69, + 0x5a, 0x48, 0x67, 0x69, 0x74, 0x68, 0x75, 0x62, 0x2e, 0x63, 0x6f, 0x6d, 0x2f, 0x61, 0x70, 0x61, + 0x63, 0x68, 0x65, 0x2f, 0x62, 0x65, 0x61, 0x6d, 0x2f, 0x73, 0x64, 0x6b, 0x73, 0x2f, 0x76, 0x32, + 0x2f, 0x67, 0x6f, 0x2f, 0x70, 0x6b, 0x67, 0x2f, 0x62, 0x65, 0x61, 0x6d, 0x2f, 0x6d, 0x6f, 0x64, + 0x65, 0x6c, 0x2f, 0x70, 0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e, 0x65, 0x5f, 0x76, 0x31, 0x3b, 0x70, + 0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e, 0x65, 0x5f, 0x76, 0x31, 0x62, 0x06, 0x70, 0x72, 0x6f, 0x74, + 0x6f, 0x33, } var file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_enumTypes = make([]protoimpl.EnumInfo, 25) var file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_msgTypes = make([]protoimpl.MessageInfo, 105) -var file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_goTypes = []interface{}{ +var file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_goTypes = []any{ (BeamConstants_Constants)(0), // 0: org.apache.beam.model.pipeline.v1.BeamConstants.Constants (StandardPTransforms_Primitives)(0), // 1: org.apache.beam.model.pipeline.v1.StandardPTransforms.Primitives (StandardPTransforms_DeprecatedPrimitives)(0), // 2: org.apache.beam.model.pipeline.v1.StandardPTransforms.DeprecatedPrimitives @@ -9727,1089 +13111,7 @@ func file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_init() { return } file_org_apache_beam_model_pipeline_v1_endpoints_proto_init() - if !protoimpl.UnsafeEnabled { - file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_msgTypes[0].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*BeamConstants); i { - case 0: - return &v.state - case 1: - return &v.sizeCache - case 2: - return &v.unknownFields - default: - return nil - } - } - file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_msgTypes[1].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*Components); i { - case 0: - return &v.state - case 1: - return &v.sizeCache - case 2: - return &v.unknownFields - default: - return nil - } - } - file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_msgTypes[2].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*Pipeline); i { - case 0: - return &v.state - case 1: - return &v.sizeCache - case 2: - return &v.unknownFields - default: - return nil - } - } - file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_msgTypes[3].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*PTransform); i { - case 0: - return &v.state - case 1: - return &v.sizeCache - case 2: - return &v.unknownFields - default: - return nil - } - } - file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_msgTypes[4].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*StandardPTransforms); i { - case 0: - return &v.state - case 1: - return &v.sizeCache - case 2: - return &v.unknownFields - default: - return nil - } - } - file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_msgTypes[5].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*StandardSideInputTypes); i { - case 0: - return &v.state - case 1: - return &v.sizeCache - case 2: - return &v.unknownFields - default: - return nil - } - } - file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_msgTypes[6].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*StandardUserStateTypes); i { - case 0: - return &v.state - case 1: - return &v.sizeCache - case 2: - return &v.unknownFields - default: - return nil - } - } - file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_msgTypes[7].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*PCollection); i { - case 0: - return &v.state - case 1: - return &v.sizeCache - case 2: - return &v.unknownFields - default: - return nil - } - } - file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_msgTypes[8].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*ParDoPayload); i { - case 0: - return &v.state - case 1: - return &v.sizeCache - case 2: - return &v.unknownFields - default: - return nil - } - } - file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_msgTypes[9].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*StateSpec); i { - case 0: - return &v.state - case 1: - return &v.sizeCache - case 2: - return &v.unknownFields - default: - return nil - } - } - file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_msgTypes[10].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*ReadModifyWriteStateSpec); i { - case 0: - return &v.state - case 1: - return &v.sizeCache - case 2: - return &v.unknownFields - default: - return nil - } - } - file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_msgTypes[11].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*BagStateSpec); i { - case 0: - return &v.state - case 1: - return &v.sizeCache - case 2: - return &v.unknownFields - default: - return nil - } - } - file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_msgTypes[12].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*OrderedListStateSpec); i { - case 0: - return &v.state - case 1: - return &v.sizeCache - case 2: - return &v.unknownFields - default: - return nil - } - } - file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_msgTypes[13].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*CombiningStateSpec); i { - case 0: - return &v.state - case 1: - return &v.sizeCache - case 2: - return &v.unknownFields - default: - return nil - } - } - file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_msgTypes[14].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*MapStateSpec); i { - case 0: - return &v.state - case 1: - return &v.sizeCache - case 2: - return &v.unknownFields - default: - return nil - } - } - file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_msgTypes[15].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*MultimapStateSpec); i { - case 0: - return &v.state - case 1: - return &v.sizeCache - case 2: - return &v.unknownFields - default: - return nil - } - } - file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_msgTypes[16].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*SetStateSpec); i { - case 0: - return &v.state - case 1: - return &v.sizeCache - case 2: - return &v.unknownFields - default: - return nil - } - } - file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_msgTypes[17].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*TimerFamilySpec); i { - case 0: - return &v.state - case 1: - return &v.sizeCache - case 2: - return &v.unknownFields - default: - return nil - } - } - file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_msgTypes[18].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*IsBounded); i { - case 0: - return &v.state - case 1: - return &v.sizeCache - case 2: - return &v.unknownFields - default: - return nil - } - } - file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_msgTypes[19].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*ReadPayload); i { - case 0: - return &v.state - case 1: - return &v.sizeCache - case 2: - return &v.unknownFields - default: - return nil - } - } - file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_msgTypes[20].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*WindowIntoPayload); i { - case 0: - return &v.state - case 1: - return &v.sizeCache - case 2: - return &v.unknownFields - default: - return nil - } - } - file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_msgTypes[21].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*CombinePayload); i { - case 0: - return &v.state - case 1: - return &v.sizeCache - case 2: - return &v.unknownFields - default: - return nil - } - } - file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_msgTypes[22].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*TestStreamPayload); i { - case 0: - return &v.state - case 1: - return &v.sizeCache - case 2: - return &v.unknownFields - default: - return nil - } - } - file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_msgTypes[23].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*EventsRequest); i { - case 0: - return &v.state - case 1: - return &v.sizeCache - case 2: - return &v.unknownFields - default: - return nil - } - } - file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_msgTypes[24].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*WriteFilesPayload); i { - case 0: - return &v.state - case 1: - return &v.sizeCache - case 2: - return &v.unknownFields - default: - return nil - } - } - file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_msgTypes[25].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*PubSubReadPayload); i { - case 0: - return &v.state - case 1: - return &v.sizeCache - case 2: - return &v.unknownFields - default: - return nil - } - } - file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_msgTypes[26].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*PubSubWritePayload); i { - case 0: - return &v.state - case 1: - return &v.sizeCache - case 2: - return &v.unknownFields - default: - return nil - } - } - file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_msgTypes[27].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*GroupIntoBatchesPayload); i { - case 0: - return &v.state - case 1: - return &v.sizeCache - case 2: - return &v.unknownFields - default: - return nil - } - } - file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_msgTypes[28].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*RedistributePayload); i { - case 0: - return &v.state - case 1: - return &v.sizeCache - case 2: - return &v.unknownFields - default: - return nil - } - } - file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_msgTypes[29].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*Coder); i { - case 0: - return &v.state - case 1: - return &v.sizeCache - case 2: - return &v.unknownFields - default: - return nil - } - } - file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_msgTypes[30].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*StandardCoders); i { - case 0: - return &v.state - case 1: - return &v.sizeCache - case 2: - return &v.unknownFields - default: - return nil - } - } - file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_msgTypes[31].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*WindowingStrategy); i { - case 0: - return &v.state - case 1: - return &v.sizeCache - case 2: - return &v.unknownFields - default: - return nil - } - } - file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_msgTypes[32].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*MergeStatus); i { - case 0: - return &v.state - case 1: - return &v.sizeCache - case 2: - return &v.unknownFields - default: - return nil - } - } - file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_msgTypes[33].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*AccumulationMode); i { - case 0: - return &v.state - case 1: - return &v.sizeCache - case 2: - return &v.unknownFields - default: - return nil - } - } - file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_msgTypes[34].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*ClosingBehavior); i { - case 0: - return &v.state - case 1: - return &v.sizeCache - case 2: - return &v.unknownFields - default: - return nil - } - } - file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_msgTypes[35].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*OnTimeBehavior); i { - case 0: - return &v.state - case 1: - return &v.sizeCache - case 2: - return &v.unknownFields - default: - return nil - } - } - file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_msgTypes[36].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*OutputTime); i { - case 0: - return &v.state - case 1: - return &v.sizeCache - case 2: - return &v.unknownFields - default: - return nil - } - } - file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_msgTypes[37].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*TimeDomain); i { - case 0: - return &v.state - case 1: - return &v.sizeCache - case 2: - return &v.unknownFields - default: - return nil - } - } - file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_msgTypes[38].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*Trigger); i { - case 0: - return &v.state - case 1: - return &v.sizeCache - case 2: - return &v.unknownFields - default: - return nil - } - } - file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_msgTypes[39].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*TimestampTransform); i { - case 0: - return &v.state - case 1: - return &v.sizeCache - case 2: - return &v.unknownFields - default: - return nil - } - } - file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_msgTypes[40].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*SideInput); i { - case 0: - return &v.state - case 1: - return &v.sizeCache - case 2: - return &v.unknownFields - default: - return nil - } - } - file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_msgTypes[41].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*StandardArtifacts); i { - case 0: - return &v.state - case 1: - return &v.sizeCache - case 2: - return &v.unknownFields - default: - return nil - } - } - file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_msgTypes[42].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*ArtifactFilePayload); i { - case 0: - return &v.state - case 1: - return &v.sizeCache - case 2: - return &v.unknownFields - default: - return nil - } - } - file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_msgTypes[43].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*ArtifactUrlPayload); i { - case 0: - return &v.state - case 1: - return &v.sizeCache - case 2: - return &v.unknownFields - default: - return nil - } - } - file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_msgTypes[44].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*EmbeddedFilePayload); i { - case 0: - return &v.state - case 1: - return &v.sizeCache - case 2: - return &v.unknownFields - default: - return nil - } - } - file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_msgTypes[45].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*PyPIPayload); i { - case 0: - return &v.state - case 1: - return &v.sizeCache - case 2: - return &v.unknownFields - default: - return nil - } - } - file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_msgTypes[46].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*MavenPayload); i { - case 0: - return &v.state - case 1: - return &v.sizeCache - case 2: - return &v.unknownFields - default: - return nil - } - } - file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_msgTypes[47].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*DeferredArtifactPayload); i { - case 0: - return &v.state - case 1: - return &v.sizeCache - case 2: - return &v.unknownFields - default: - return nil - } - } - file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_msgTypes[48].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*ArtifactStagingToRolePayload); i { - case 0: - return &v.state - case 1: - return &v.sizeCache - case 2: - return &v.unknownFields - default: - return nil - } - } - file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_msgTypes[49].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*ArtifactInformation); i { - case 0: - return &v.state - case 1: - return &v.sizeCache - case 2: - return &v.unknownFields - default: - return nil - } - } - file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_msgTypes[50].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*Environment); i { - case 0: - return &v.state - case 1: - return &v.sizeCache - case 2: - return &v.unknownFields - default: - return nil - } - } - file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_msgTypes[51].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*StandardEnvironments); i { - case 0: - return &v.state - case 1: - return &v.sizeCache - case 2: - return &v.unknownFields - default: - return nil - } - } - file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_msgTypes[52].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*DockerPayload); i { - case 0: - return &v.state - case 1: - return &v.sizeCache - case 2: - return &v.unknownFields - default: - return nil - } - } - file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_msgTypes[53].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*ProcessPayload); i { - case 0: - return &v.state - case 1: - return &v.sizeCache - case 2: - return &v.unknownFields - default: - return nil - } - } - file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_msgTypes[54].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*ExternalPayload); i { - case 0: - return &v.state - case 1: - return &v.sizeCache - case 2: - return &v.unknownFields - default: - return nil - } - } - file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_msgTypes[55].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*AnyOfEnvironmentPayload); i { - case 0: - return &v.state - case 1: - return &v.sizeCache - case 2: - return &v.unknownFields - default: - return nil - } - } - file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_msgTypes[56].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*StandardProtocols); i { - case 0: - return &v.state - case 1: - return &v.sizeCache - case 2: - return &v.unknownFields - default: - return nil - } - } - file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_msgTypes[57].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*StandardRunnerProtocols); i { - case 0: - return &v.state - case 1: - return &v.sizeCache - case 2: - return &v.unknownFields - default: - return nil - } - } - file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_msgTypes[58].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*StandardRequirements); i { - case 0: - return &v.state - case 1: - return &v.sizeCache - case 2: - return &v.unknownFields - default: - return nil - } - } - file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_msgTypes[59].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*FunctionSpec); i { - case 0: - return &v.state - case 1: - return &v.sizeCache - case 2: - return &v.unknownFields - default: - return nil - } - } - file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_msgTypes[60].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*StandardDisplayData); i { - case 0: - return &v.state - case 1: - return &v.sizeCache - case 2: - return &v.unknownFields - default: - return nil - } - } - file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_msgTypes[61].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*LabelledPayload); i { - case 0: - return &v.state - case 1: - return &v.sizeCache - case 2: - return &v.unknownFields - default: - return nil - } - } - file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_msgTypes[62].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*DisplayData); i { - case 0: - return &v.state - case 1: - return &v.sizeCache - case 2: - return &v.unknownFields - default: - return nil - } - } - file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_msgTypes[63].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*MessageWithComponents); i { - case 0: - return &v.state - case 1: - return &v.sizeCache - case 2: - return &v.unknownFields - default: - return nil - } - } - file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_msgTypes[64].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*ExecutableStagePayload); i { - case 0: - return &v.state - case 1: - return &v.sizeCache - case 2: - return &v.unknownFields - default: - return nil - } - } - file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_msgTypes[65].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*StandardResourceHints); i { - case 0: - return &v.state - case 1: - return &v.sizeCache - case 2: - return &v.unknownFields - default: - return nil - } - } - file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_msgTypes[77].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*TestStreamPayload_Event); i { - case 0: - return &v.state - case 1: - return &v.sizeCache - case 2: - return &v.unknownFields - default: - return nil - } - } - file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_msgTypes[78].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*TestStreamPayload_TimestampedElement); i { - case 0: - return &v.state - case 1: - return &v.sizeCache - case 2: - return &v.unknownFields - default: - return nil - } - } - file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_msgTypes[79].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*TestStreamPayload_Event_AdvanceWatermark); i { - case 0: - return &v.state - case 1: - return &v.sizeCache - case 2: - return &v.unknownFields - default: - return nil - } - } - file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_msgTypes[80].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*TestStreamPayload_Event_AdvanceProcessingTime); i { - case 0: - return &v.state - case 1: - return &v.sizeCache - case 2: - return &v.unknownFields - default: - return nil - } - } - file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_msgTypes[81].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*TestStreamPayload_Event_AddElements); i { - case 0: - return &v.state - case 1: - return &v.sizeCache - case 2: - return &v.unknownFields - default: - return nil - } - } - file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_msgTypes[83].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*Trigger_AfterAll); i { - case 0: - return &v.state - case 1: - return &v.sizeCache - case 2: - return &v.unknownFields - default: - return nil - } - } - file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_msgTypes[84].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*Trigger_AfterAny); i { - case 0: - return &v.state - case 1: - return &v.sizeCache - case 2: - return &v.unknownFields - default: - return nil - } - } - file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_msgTypes[85].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*Trigger_AfterEach); i { - case 0: - return &v.state - case 1: - return &v.sizeCache - case 2: - return &v.unknownFields - default: - return nil - } - } - file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_msgTypes[86].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*Trigger_AfterEndOfWindow); i { - case 0: - return &v.state - case 1: - return &v.sizeCache - case 2: - return &v.unknownFields - default: - return nil - } - } - file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_msgTypes[87].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*Trigger_AfterProcessingTime); i { - case 0: - return &v.state - case 1: - return &v.sizeCache - case 2: - return &v.unknownFields - default: - return nil - } - } - file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_msgTypes[88].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*Trigger_AfterSynchronizedProcessingTime); i { - case 0: - return &v.state - case 1: - return &v.sizeCache - case 2: - return &v.unknownFields - default: - return nil - } - } - file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_msgTypes[89].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*Trigger_Default); i { - case 0: - return &v.state - case 1: - return &v.sizeCache - case 2: - return &v.unknownFields - default: - return nil - } - } - file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_msgTypes[90].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*Trigger_ElementCount); i { - case 0: - return &v.state - case 1: - return &v.sizeCache - case 2: - return &v.unknownFields - default: - return nil - } - } - file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_msgTypes[91].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*Trigger_Never); i { - case 0: - return &v.state - case 1: - return &v.sizeCache - case 2: - return &v.unknownFields - default: - return nil - } - } - file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_msgTypes[92].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*Trigger_Always); i { - case 0: - return &v.state - case 1: - return &v.sizeCache - case 2: - return &v.unknownFields - default: - return nil - } - } - file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_msgTypes[93].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*Trigger_OrFinally); i { - case 0: - return &v.state - case 1: - return &v.sizeCache - case 2: - return &v.unknownFields - default: - return nil - } - } - file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_msgTypes[94].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*Trigger_Repeat); i { - case 0: - return &v.state - case 1: - return &v.sizeCache - case 2: - return &v.unknownFields - default: - return nil - } - } - file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_msgTypes[95].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*TimestampTransform_Delay); i { - case 0: - return &v.state - case 1: - return &v.sizeCache - case 2: - return &v.unknownFields - default: - return nil - } - } - file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_msgTypes[96].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*TimestampTransform_AlignTo); i { - case 0: - return &v.state - case 1: - return &v.sizeCache - case 2: - return &v.unknownFields - default: - return nil - } - } - file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_msgTypes[100].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*ExecutableStagePayload_SideInputId); i { - case 0: - return &v.state - case 1: - return &v.sizeCache - case 2: - return &v.unknownFields - default: - return nil - } - } - file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_msgTypes[101].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*ExecutableStagePayload_UserStateId); i { - case 0: - return &v.state - case 1: - return &v.sizeCache - case 2: - return &v.unknownFields - default: - return nil - } - } - file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_msgTypes[102].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*ExecutableStagePayload_TimerId); i { - case 0: - return &v.state - case 1: - return &v.sizeCache - case 2: - return &v.unknownFields - default: - return nil - } - } - file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_msgTypes[103].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*ExecutableStagePayload_TimerFamilyId); i { - case 0: - return &v.state - case 1: - return &v.sizeCache - case 2: - return &v.unknownFields - default: - return nil - } - } - file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_msgTypes[104].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*ExecutableStagePayload_WireCoderSetting); i { - case 0: - return &v.state - case 1: - return &v.sizeCache - case 2: - return &v.unknownFields - default: - return nil - } - } - } - file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_msgTypes[9].OneofWrappers = []interface{}{ + file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_msgTypes[9].OneofWrappers = []any{ (*StateSpec_ReadModifyWriteSpec)(nil), (*StateSpec_BagSpec)(nil), (*StateSpec_CombiningSpec)(nil), @@ -10818,7 +13120,7 @@ func file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_init() { (*StateSpec_OrderedListSpec)(nil), (*StateSpec_MultimapSpec)(nil), } - file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_msgTypes[38].OneofWrappers = []interface{}{ + file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_msgTypes[38].OneofWrappers = []any{ (*Trigger_AfterAll_)(nil), (*Trigger_AfterAny_)(nil), (*Trigger_AfterEach_)(nil), @@ -10832,17 +13134,17 @@ func file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_init() { (*Trigger_OrFinally_)(nil), (*Trigger_Repeat_)(nil), } - file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_msgTypes[39].OneofWrappers = []interface{}{ + file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_msgTypes[39].OneofWrappers = []any{ (*TimestampTransform_Delay_)(nil), (*TimestampTransform_AlignTo_)(nil), } - file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_msgTypes[61].OneofWrappers = []interface{}{ + file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_msgTypes[61].OneofWrappers = []any{ (*LabelledPayload_StringValue)(nil), (*LabelledPayload_BoolValue)(nil), (*LabelledPayload_DoubleValue)(nil), (*LabelledPayload_IntValue)(nil), } - file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_msgTypes[63].OneofWrappers = []interface{}{ + file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_msgTypes[63].OneofWrappers = []any{ (*MessageWithComponents_Coder)(nil), (*MessageWithComponents_CombinePayload)(nil), (*MessageWithComponents_FunctionSpec)(nil), @@ -10854,12 +13156,12 @@ func file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_init() { (*MessageWithComponents_WindowIntoPayload)(nil), (*MessageWithComponents_WindowingStrategy)(nil), } - file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_msgTypes[77].OneofWrappers = []interface{}{ + file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_msgTypes[77].OneofWrappers = []any{ (*TestStreamPayload_Event_WatermarkEvent)(nil), (*TestStreamPayload_Event_ProcessingTimeEvent)(nil), (*TestStreamPayload_Event_ElementEvent)(nil), } - file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_msgTypes[104].OneofWrappers = []interface{}{ + file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_msgTypes[104].OneofWrappers = []any{ (*ExecutableStagePayload_WireCoderSetting_InputOrOutputId)(nil), (*ExecutableStagePayload_WireCoderSetting_Timer)(nil), } diff --git a/sdks/go/pkg/beam/model/pipeline_v1/beam_runner_api_grpc.pb.go b/sdks/go/pkg/beam/model/pipeline_v1/beam_runner_api_grpc.pb.go index ee426152d761..847c439583b0 100644 --- a/sdks/go/pkg/beam/model/pipeline_v1/beam_runner_api_grpc.pb.go +++ b/sdks/go/pkg/beam/model/pipeline_v1/beam_runner_api_grpc.pb.go @@ -14,9 +14,14 @@ // WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. // See the License for the specific language governing permissions and // limitations under the License. + +// +// Protocol Buffers describing the Runner API, which is the runner-independent, +// SDK-independent definition of the Beam model. + // Code generated by protoc-gen-go-grpc. DO NOT EDIT. // versions: -// - protoc-gen-go-grpc v1.1.0 +// - protoc-gen-go-grpc v1.5.1 // - protoc v5.27.3 // source: org/apache/beam/model/pipeline/v1/beam_runner_api.proto @@ -31,15 +36,19 @@ import ( // This is a compile-time assertion to ensure that this generated file // is compatible with the grpc package it is being compiled against. -// Requires gRPC-Go v1.32.0 or later. -const _ = grpc.SupportPackageIsVersion7 +// Requires gRPC-Go v1.64.0 or later. +const _ = grpc.SupportPackageIsVersion9 + +const ( + TestStreamService_Events_FullMethodName = "/org.apache.beam.model.pipeline.v1.TestStreamService/Events" +) // TestStreamServiceClient is the client API for TestStreamService service. // // For semantics around ctx use and closing/ending streaming RPCs, please refer to https://pkg.go.dev/google.golang.org/grpc/?tab=doc#ClientConn.NewStream. type TestStreamServiceClient interface { // A TestStream will request for events using this RPC. - Events(ctx context.Context, in *EventsRequest, opts ...grpc.CallOption) (TestStreamService_EventsClient, error) + Events(ctx context.Context, in *EventsRequest, opts ...grpc.CallOption) (grpc.ServerStreamingClient[TestStreamPayload_Event], error) } type testStreamServiceClient struct { @@ -50,12 +59,13 @@ func NewTestStreamServiceClient(cc grpc.ClientConnInterface) TestStreamServiceCl return &testStreamServiceClient{cc} } -func (c *testStreamServiceClient) Events(ctx context.Context, in *EventsRequest, opts ...grpc.CallOption) (TestStreamService_EventsClient, error) { - stream, err := c.cc.NewStream(ctx, &TestStreamService_ServiceDesc.Streams[0], "/org.apache.beam.model.pipeline.v1.TestStreamService/Events", opts...) +func (c *testStreamServiceClient) Events(ctx context.Context, in *EventsRequest, opts ...grpc.CallOption) (grpc.ServerStreamingClient[TestStreamPayload_Event], error) { + cOpts := append([]grpc.CallOption{grpc.StaticMethod()}, opts...) + stream, err := c.cc.NewStream(ctx, &TestStreamService_ServiceDesc.Streams[0], TestStreamService_Events_FullMethodName, cOpts...) if err != nil { return nil, err } - x := &testStreamServiceEventsClient{stream} + x := &grpc.GenericClientStream[EventsRequest, TestStreamPayload_Event]{ClientStream: stream} if err := x.ClientStream.SendMsg(in); err != nil { return nil, err } @@ -65,40 +75,30 @@ func (c *testStreamServiceClient) Events(ctx context.Context, in *EventsRequest, return x, nil } -type TestStreamService_EventsClient interface { - Recv() (*TestStreamPayload_Event, error) - grpc.ClientStream -} - -type testStreamServiceEventsClient struct { - grpc.ClientStream -} - -func (x *testStreamServiceEventsClient) Recv() (*TestStreamPayload_Event, error) { - m := new(TestStreamPayload_Event) - if err := x.ClientStream.RecvMsg(m); err != nil { - return nil, err - } - return m, nil -} +// This type alias is provided for backwards compatibility with existing code that references the prior non-generic stream type by name. +type TestStreamService_EventsClient = grpc.ServerStreamingClient[TestStreamPayload_Event] // TestStreamServiceServer is the server API for TestStreamService service. // All implementations must embed UnimplementedTestStreamServiceServer -// for forward compatibility +// for forward compatibility. type TestStreamServiceServer interface { // A TestStream will request for events using this RPC. - Events(*EventsRequest, TestStreamService_EventsServer) error + Events(*EventsRequest, grpc.ServerStreamingServer[TestStreamPayload_Event]) error mustEmbedUnimplementedTestStreamServiceServer() } -// UnimplementedTestStreamServiceServer must be embedded to have forward compatible implementations. -type UnimplementedTestStreamServiceServer struct { -} +// UnimplementedTestStreamServiceServer must be embedded to have +// forward compatible implementations. +// +// NOTE: this should be embedded by value instead of pointer to avoid a nil +// pointer dereference when methods are called. +type UnimplementedTestStreamServiceServer struct{} -func (UnimplementedTestStreamServiceServer) Events(*EventsRequest, TestStreamService_EventsServer) error { +func (UnimplementedTestStreamServiceServer) Events(*EventsRequest, grpc.ServerStreamingServer[TestStreamPayload_Event]) error { return status.Errorf(codes.Unimplemented, "method Events not implemented") } func (UnimplementedTestStreamServiceServer) mustEmbedUnimplementedTestStreamServiceServer() {} +func (UnimplementedTestStreamServiceServer) testEmbeddedByValue() {} // UnsafeTestStreamServiceServer may be embedded to opt out of forward compatibility for this service. // Use of this interface is not recommended, as added methods to TestStreamServiceServer will @@ -108,6 +108,13 @@ type UnsafeTestStreamServiceServer interface { } func RegisterTestStreamServiceServer(s grpc.ServiceRegistrar, srv TestStreamServiceServer) { + // If the following call pancis, it indicates UnimplementedTestStreamServiceServer was + // embedded by pointer and is nil. This will cause panics if an + // unimplemented method is ever invoked, so we test this at initialization + // time to prevent it from happening at runtime later due to I/O. + if t, ok := srv.(interface{ testEmbeddedByValue() }); ok { + t.testEmbeddedByValue() + } s.RegisterService(&TestStreamService_ServiceDesc, srv) } @@ -116,21 +123,11 @@ func _TestStreamService_Events_Handler(srv interface{}, stream grpc.ServerStream if err := stream.RecvMsg(m); err != nil { return err } - return srv.(TestStreamServiceServer).Events(m, &testStreamServiceEventsServer{stream}) + return srv.(TestStreamServiceServer).Events(m, &grpc.GenericServerStream[EventsRequest, TestStreamPayload_Event]{ServerStream: stream}) } -type TestStreamService_EventsServer interface { - Send(*TestStreamPayload_Event) error - grpc.ServerStream -} - -type testStreamServiceEventsServer struct { - grpc.ServerStream -} - -func (x *testStreamServiceEventsServer) Send(m *TestStreamPayload_Event) error { - return x.ServerStream.SendMsg(m) -} +// This type alias is provided for backwards compatibility with existing code that references the prior non-generic stream type by name. +type TestStreamService_EventsServer = grpc.ServerStreamingServer[TestStreamPayload_Event] // TestStreamService_ServiceDesc is the grpc.ServiceDesc for TestStreamService service. // It's only intended for direct use with grpc.RegisterService, diff --git a/sdks/go/pkg/beam/model/pipeline_v1/beam_runner_api_protoopaque.pb.go b/sdks/go/pkg/beam/model/pipeline_v1/beam_runner_api_protoopaque.pb.go new file mode 100644 index 000000000000..67311c583b4a --- /dev/null +++ b/sdks/go/pkg/beam/model/pipeline_v1/beam_runner_api_protoopaque.pb.go @@ -0,0 +1,12779 @@ +// +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +// +// Protocol Buffers describing the Runner API, which is the runner-independent, +// SDK-independent definition of the Beam model. + +// Code generated by protoc-gen-go. DO NOT EDIT. +// versions: +// protoc-gen-go v1.36.0 +// protoc v5.27.3 +// source: org/apache/beam/model/pipeline/v1/beam_runner_api.proto + +//go:build protoopaque + +package pipeline_v1 + +import ( + protoreflect "google.golang.org/protobuf/reflect/protoreflect" + protoimpl "google.golang.org/protobuf/runtime/protoimpl" + descriptorpb "google.golang.org/protobuf/types/descriptorpb" + reflect "reflect" +) + +const ( + // Verify that this generated code is sufficiently up-to-date. + _ = protoimpl.EnforceVersion(20 - protoimpl.MinVersion) + // Verify that runtime/protoimpl is sufficiently up-to-date. + _ = protoimpl.EnforceVersion(protoimpl.MaxVersion - 20) +) + +type BeamConstants_Constants int32 + +const ( + // All timestamps of elements or window boundaries must be within + // the interval [MIN_TIMESTAMP_MILLIS, MAX_TIMESTAMP_MILLIS]. + // The smallest representable timestamp of an element or a window boundary. + BeamConstants_MIN_TIMESTAMP_MILLIS BeamConstants_Constants = 0 + // The largest representable timestamp of an element or a window boundary. + BeamConstants_MAX_TIMESTAMP_MILLIS BeamConstants_Constants = 1 + // The maximum timestamp for the global window. + // Triggers use max timestamp to set timers' timestamp. Timers fire when + // the watermark passes their timestamps. So, the timestamp needs to be + // smaller than the MAX_TIMESTAMP_MILLIS. + // One standard day is subtracted from MAX_TIMESTAMP_MILLIS to make sure + // the max timestamp is smaller than MAX_TIMESTAMP_MILLIS even after rounding up + // to seconds or minutes. + BeamConstants_GLOBAL_WINDOW_MAX_TIMESTAMP_MILLIS BeamConstants_Constants = 2 +) + +// Enum value maps for BeamConstants_Constants. +var ( + BeamConstants_Constants_name = map[int32]string{ + 0: "MIN_TIMESTAMP_MILLIS", + 1: "MAX_TIMESTAMP_MILLIS", + 2: "GLOBAL_WINDOW_MAX_TIMESTAMP_MILLIS", + } + BeamConstants_Constants_value = map[string]int32{ + "MIN_TIMESTAMP_MILLIS": 0, + "MAX_TIMESTAMP_MILLIS": 1, + "GLOBAL_WINDOW_MAX_TIMESTAMP_MILLIS": 2, + } +) + +func (x BeamConstants_Constants) Enum() *BeamConstants_Constants { + p := new(BeamConstants_Constants) + *p = x + return p +} + +func (x BeamConstants_Constants) String() string { + return protoimpl.X.EnumStringOf(x.Descriptor(), protoreflect.EnumNumber(x)) +} + +func (BeamConstants_Constants) Descriptor() protoreflect.EnumDescriptor { + return file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_enumTypes[0].Descriptor() +} + +func (BeamConstants_Constants) Type() protoreflect.EnumType { + return &file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_enumTypes[0] +} + +func (x BeamConstants_Constants) Number() protoreflect.EnumNumber { + return protoreflect.EnumNumber(x) +} + +// Primitive transforms may not specify composite sub-transforms. +type StandardPTransforms_Primitives int32 + +const ( + // ParDo is a Beam transform for generic parallel processing. The ParDo + // processing paradigm is similar to the “Map” phase of a + // Map/Shuffle/Reduce-style algorithm: a ParDo transform considers each + // element in the input PCollection, performs some processing function + // (your user code) on that element, and emits zero, one, or multiple + // elements to an output PCollection. + // + // See https://beam.apache.org/documentation/programming-guide/#pardo + // for additional details. + // + // Payload: ParDoPayload + StandardPTransforms_PAR_DO StandardPTransforms_Primitives = 0 + // Flatten is a Beam transform for PCollection objects that store the same + // data type. Flatten merges multiple PCollection objects into a single + // logical PCollection. + // + // See https://beam.apache.org/documentation/programming-guide/#flatten + // for additional details. + // + // Payload: None + StandardPTransforms_FLATTEN StandardPTransforms_Primitives = 1 + // GroupByKey is a Beam transform for processing collections of key/value + // pairs. It’s a parallel reduction operation, analogous to the Shuffle + // phase of a Map/Shuffle/Reduce-style algorithm. The input to GroupByKey is + // a collection of key/value pairs that represents a multimap, where the + // collection contains multiple pairs that have the same key, but different + // values. Given such a collection, you use GroupByKey to collect all of the + // values associated with each unique key. + // + // See https://beam.apache.org/documentation/programming-guide/#groupbykey + // for additional details. + // + // Never defines an environment as the runner is required to implement this + // transform. + // + // Payload: None + StandardPTransforms_GROUP_BY_KEY StandardPTransforms_Primitives = 2 + // A transform which produces a single empty byte array at the minimum + // timestamp in the GlobalWindow. + // + // Never defines an environment as the runner is required to implement this + // transform. + // + // Payload: None + StandardPTransforms_IMPULSE StandardPTransforms_Primitives = 3 + // Windowing subdivides a PCollection according to the timestamps of its + // individual elements. Transforms that aggregate multiple elements, such as + // GroupByKey and Combine, work implicitly on a per-window basis — they + // process each PCollection as a succession of multiple, finite windows, + // though the entire collection itself may be of unbounded size. + // + // See https://beam.apache.org/documentation/programming-guide/#windowing + // for additional details. + // + // Payload: WindowIntoPayload + StandardPTransforms_ASSIGN_WINDOWS StandardPTransforms_Primitives = 4 + // A testing input that generates an unbounded {@link PCollection} of + // elements, advancing the watermark and processing time as elements are + // emitted. After all of the specified elements are emitted, ceases to + // produce output. + // + // See https://beam.apache.org/blog/2016/10/20/test-stream.html + // for additional details. + // + // Payload: TestStreamPayload + StandardPTransforms_TEST_STREAM StandardPTransforms_Primitives = 5 + // Represents mapping of main input window onto side input window. + // + // Side input window mapping function: + // Input: KV + // Output: KV + // + // For each main input window, the side input window is returned. The + // nonce is used by a runner to associate each input with its output. + // The nonce is represented as an opaque set of bytes. + // + // Payload: SideInput#window_mapping_fn FunctionSpec + StandardPTransforms_MAP_WINDOWS StandardPTransforms_Primitives = 6 + // Used to merge windows during a GroupByKey. + // + // Window merging function: + // Input: KV> + // Output: KV, iterable>>> + // + // For each set of original windows, a list of all unmerged windows is + // output alongside a map of merged window to set of consumed windows. + // All original windows must be contained in either the unmerged original + // window set or one of the consumed original window sets. Each original + // window can only be part of one output set. The nonce is used by a runner + // to associate each input with its output. The nonce is represented as an + // opaque set of bytes. + // + // Payload: WindowingStrategy#window_fn FunctionSpec + StandardPTransforms_MERGE_WINDOWS StandardPTransforms_Primitives = 7 + // A transform that translates a given element to its human-readable + // representation. + // + // Input: KV + // Output: KV + // + // For each given element, the implementation returns the best-effort + // human-readable representation. When possible, the implementation could + // call a user-overridable method on the type. For example, Java could + // call `toString()`, Python could call `str()`, Golang could call + // `String()`. The nonce is used by a runner to associate each input with + // its output. The nonce is represented as an opaque set of bytes. + // + // Payload: none + StandardPTransforms_TO_STRING StandardPTransforms_Primitives = 8 +) + +// Enum value maps for StandardPTransforms_Primitives. +var ( + StandardPTransforms_Primitives_name = map[int32]string{ + 0: "PAR_DO", + 1: "FLATTEN", + 2: "GROUP_BY_KEY", + 3: "IMPULSE", + 4: "ASSIGN_WINDOWS", + 5: "TEST_STREAM", + 6: "MAP_WINDOWS", + 7: "MERGE_WINDOWS", + 8: "TO_STRING", + } + StandardPTransforms_Primitives_value = map[string]int32{ + "PAR_DO": 0, + "FLATTEN": 1, + "GROUP_BY_KEY": 2, + "IMPULSE": 3, + "ASSIGN_WINDOWS": 4, + "TEST_STREAM": 5, + "MAP_WINDOWS": 6, + "MERGE_WINDOWS": 7, + "TO_STRING": 8, + } +) + +func (x StandardPTransforms_Primitives) Enum() *StandardPTransforms_Primitives { + p := new(StandardPTransforms_Primitives) + *p = x + return p +} + +func (x StandardPTransforms_Primitives) String() string { + return protoimpl.X.EnumStringOf(x.Descriptor(), protoreflect.EnumNumber(x)) +} + +func (StandardPTransforms_Primitives) Descriptor() protoreflect.EnumDescriptor { + return file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_enumTypes[1].Descriptor() +} + +func (StandardPTransforms_Primitives) Type() protoreflect.EnumType { + return &file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_enumTypes[1] +} + +func (x StandardPTransforms_Primitives) Number() protoreflect.EnumNumber { + return protoreflect.EnumNumber(x) +} + +type StandardPTransforms_DeprecatedPrimitives int32 + +const ( + // Represents the operation to read a Bounded or Unbounded source. + // Payload: ReadPayload. + StandardPTransforms_READ StandardPTransforms_DeprecatedPrimitives = 0 + // Runners should move away from translating `CreatePCollectionView` and treat this as + // part of the translation for a `ParDo` side input. + StandardPTransforms_CREATE_VIEW StandardPTransforms_DeprecatedPrimitives = 1 +) + +// Enum value maps for StandardPTransforms_DeprecatedPrimitives. +var ( + StandardPTransforms_DeprecatedPrimitives_name = map[int32]string{ + 0: "READ", + 1: "CREATE_VIEW", + } + StandardPTransforms_DeprecatedPrimitives_value = map[string]int32{ + "READ": 0, + "CREATE_VIEW": 1, + } +) + +func (x StandardPTransforms_DeprecatedPrimitives) Enum() *StandardPTransforms_DeprecatedPrimitives { + p := new(StandardPTransforms_DeprecatedPrimitives) + *p = x + return p +} + +func (x StandardPTransforms_DeprecatedPrimitives) String() string { + return protoimpl.X.EnumStringOf(x.Descriptor(), protoreflect.EnumNumber(x)) +} + +func (StandardPTransforms_DeprecatedPrimitives) Descriptor() protoreflect.EnumDescriptor { + return file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_enumTypes[2].Descriptor() +} + +func (StandardPTransforms_DeprecatedPrimitives) Type() protoreflect.EnumType { + return &file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_enumTypes[2] +} + +func (x StandardPTransforms_DeprecatedPrimitives) Number() protoreflect.EnumNumber { + return protoreflect.EnumNumber(x) +} + +type StandardPTransforms_Composites int32 + +const ( + // Represents the Combine.perKey() operation. + // If this is produced by an SDK, it is assumed that the SDK understands + // each of CombineComponents. + // Payload: CombinePayload + StandardPTransforms_COMBINE_PER_KEY StandardPTransforms_Composites = 0 + // Represents the Combine.globally() operation. + // If this is produced by an SDK, it is assumed that the SDK understands + // each of CombineComponents. + // Payload: CombinePayload + StandardPTransforms_COMBINE_GLOBALLY StandardPTransforms_Composites = 1 + // Represents the Reshuffle operation. + StandardPTransforms_RESHUFFLE StandardPTransforms_Composites = 2 + // Less well-known. Payload: WriteFilesPayload. + StandardPTransforms_WRITE_FILES StandardPTransforms_Composites = 3 + // Payload: PubSubReadPayload. + StandardPTransforms_PUBSUB_READ StandardPTransforms_Composites = 4 + // Payload: PubSubWritePayload. + StandardPTransforms_PUBSUB_WRITE StandardPTransforms_Composites = 5 + // Used for pubsub dynamic destinations. + // Payload: PubSubWritePayload. + StandardPTransforms_PUBSUB_WRITE_V2 StandardPTransforms_Composites = 7 + // Represents the GroupIntoBatches.WithShardedKey operation. + // Payload: GroupIntoBatchesPayload + StandardPTransforms_GROUP_INTO_BATCHES_WITH_SHARDED_KEY StandardPTransforms_Composites = 6 + StandardPTransforms_REDISTRIBUTE_BY_KEY StandardPTransforms_Composites = 8 + StandardPTransforms_REDISTRIBUTE_ARBITRARILY StandardPTransforms_Composites = 9 +) + +// Enum value maps for StandardPTransforms_Composites. +var ( + StandardPTransforms_Composites_name = map[int32]string{ + 0: "COMBINE_PER_KEY", + 1: "COMBINE_GLOBALLY", + 2: "RESHUFFLE", + 3: "WRITE_FILES", + 4: "PUBSUB_READ", + 5: "PUBSUB_WRITE", + 7: "PUBSUB_WRITE_V2", + 6: "GROUP_INTO_BATCHES_WITH_SHARDED_KEY", + 8: "REDISTRIBUTE_BY_KEY", + 9: "REDISTRIBUTE_ARBITRARILY", + } + StandardPTransforms_Composites_value = map[string]int32{ + "COMBINE_PER_KEY": 0, + "COMBINE_GLOBALLY": 1, + "RESHUFFLE": 2, + "WRITE_FILES": 3, + "PUBSUB_READ": 4, + "PUBSUB_WRITE": 5, + "PUBSUB_WRITE_V2": 7, + "GROUP_INTO_BATCHES_WITH_SHARDED_KEY": 6, + "REDISTRIBUTE_BY_KEY": 8, + "REDISTRIBUTE_ARBITRARILY": 9, + } +) + +func (x StandardPTransforms_Composites) Enum() *StandardPTransforms_Composites { + p := new(StandardPTransforms_Composites) + *p = x + return p +} + +func (x StandardPTransforms_Composites) String() string { + return protoimpl.X.EnumStringOf(x.Descriptor(), protoreflect.EnumNumber(x)) +} + +func (StandardPTransforms_Composites) Descriptor() protoreflect.EnumDescriptor { + return file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_enumTypes[3].Descriptor() +} + +func (StandardPTransforms_Composites) Type() protoreflect.EnumType { + return &file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_enumTypes[3] +} + +func (x StandardPTransforms_Composites) Number() protoreflect.EnumNumber { + return protoreflect.EnumNumber(x) +} + +// Payload for all of these: CombinePayload +type StandardPTransforms_CombineComponents int32 + +const ( + // Represents the Pre-Combine part of a lifted Combine Per Key, as described + // in the following document: + // https://s.apache.org/beam-runner-api-combine-model#heading=h.ta0g6ase8z07 + // Payload: CombinePayload + StandardPTransforms_COMBINE_PER_KEY_PRECOMBINE StandardPTransforms_CombineComponents = 0 + // Represents the Merge Accumulators part of a lifted Combine Per Key, as + // described in the following document: + // https://s.apache.org/beam-runner-api-combine-model#heading=h.jco9rvatld5m + // Payload: CombinePayload + StandardPTransforms_COMBINE_PER_KEY_MERGE_ACCUMULATORS StandardPTransforms_CombineComponents = 1 + // Represents the Extract Outputs part of a lifted Combine Per Key, as + // described in the following document: + // https://s.apache.org/beam-runner-api-combine-model#heading=h.i9i6p8gtl6ku + // Payload: CombinePayload + StandardPTransforms_COMBINE_PER_KEY_EXTRACT_OUTPUTS StandardPTransforms_CombineComponents = 2 + // Represents the Combine Grouped Values transform, as described in the + // following document: + // https://s.apache.org/beam-runner-api-combine-model#heading=h.aj86ew4v1wk + // Payload: CombinePayload + StandardPTransforms_COMBINE_GROUPED_VALUES StandardPTransforms_CombineComponents = 3 + // Represents the Convert To Accumulators transform, as described in the + // following document: + // https://s.apache.org/beam-runner-api-combine-model#heading=h.h5697l1scd9x + // Payload: CombinePayload + StandardPTransforms_COMBINE_PER_KEY_CONVERT_TO_ACCUMULATORS StandardPTransforms_CombineComponents = 4 +) + +// Enum value maps for StandardPTransforms_CombineComponents. +var ( + StandardPTransforms_CombineComponents_name = map[int32]string{ + 0: "COMBINE_PER_KEY_PRECOMBINE", + 1: "COMBINE_PER_KEY_MERGE_ACCUMULATORS", + 2: "COMBINE_PER_KEY_EXTRACT_OUTPUTS", + 3: "COMBINE_GROUPED_VALUES", + 4: "COMBINE_PER_KEY_CONVERT_TO_ACCUMULATORS", + } + StandardPTransforms_CombineComponents_value = map[string]int32{ + "COMBINE_PER_KEY_PRECOMBINE": 0, + "COMBINE_PER_KEY_MERGE_ACCUMULATORS": 1, + "COMBINE_PER_KEY_EXTRACT_OUTPUTS": 2, + "COMBINE_GROUPED_VALUES": 3, + "COMBINE_PER_KEY_CONVERT_TO_ACCUMULATORS": 4, + } +) + +func (x StandardPTransforms_CombineComponents) Enum() *StandardPTransforms_CombineComponents { + p := new(StandardPTransforms_CombineComponents) + *p = x + return p +} + +func (x StandardPTransforms_CombineComponents) String() string { + return protoimpl.X.EnumStringOf(x.Descriptor(), protoreflect.EnumNumber(x)) +} + +func (StandardPTransforms_CombineComponents) Descriptor() protoreflect.EnumDescriptor { + return file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_enumTypes[4].Descriptor() +} + +func (StandardPTransforms_CombineComponents) Type() protoreflect.EnumType { + return &file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_enumTypes[4] +} + +func (x StandardPTransforms_CombineComponents) Number() protoreflect.EnumNumber { + return protoreflect.EnumNumber(x) +} + +// Payload for all of these: ParDoPayload containing the user's SDF +type StandardPTransforms_SplittableParDoComponents int32 + +const ( + // Pairs the input element with its initial restriction. + // Input: element; output: KV(element, restriction). + StandardPTransforms_PAIR_WITH_RESTRICTION StandardPTransforms_SplittableParDoComponents = 0 + // Splits the restriction of each element/restriction pair and returns the + // resulting splits, with a corresponding floating point size estimation + // for each. + // + // A reasonable value for size is the number of bytes expected to be + // produced by this (element, restriction) pair. + // + // Input: KV(element, restriction) + // Output: KV(KV(element, restriction), size)) + StandardPTransforms_SPLIT_AND_SIZE_RESTRICTIONS StandardPTransforms_SplittableParDoComponents = 1 + // Applies the DoFn to every element and restriction. + // + // All primary and residuals returned from checkpointing or splitting must + // have the same type as the input to this transform. + // + // Input: KV(KV(element, restriction), size); output: DoFn's output. + StandardPTransforms_PROCESS_SIZED_ELEMENTS_AND_RESTRICTIONS StandardPTransforms_SplittableParDoComponents = 2 + // Truncates the restriction of each element/restriction pair and returns + // the finite restriction which will be processed when a pipeline is + // drained. See + // https://docs.google.com/document/d/1NExwHlj-2q2WUGhSO4jTu8XGhDPmm3cllSN8IMmWci8/edit#. + // for additional details about drain. + // + // Input: KV(KV(element, restriction), size); + // Output: KV(KV(element, restriction), size). + StandardPTransforms_TRUNCATE_SIZED_RESTRICTION StandardPTransforms_SplittableParDoComponents = 3 +) + +// Enum value maps for StandardPTransforms_SplittableParDoComponents. +var ( + StandardPTransforms_SplittableParDoComponents_name = map[int32]string{ + 0: "PAIR_WITH_RESTRICTION", + 1: "SPLIT_AND_SIZE_RESTRICTIONS", + 2: "PROCESS_SIZED_ELEMENTS_AND_RESTRICTIONS", + 3: "TRUNCATE_SIZED_RESTRICTION", + } + StandardPTransforms_SplittableParDoComponents_value = map[string]int32{ + "PAIR_WITH_RESTRICTION": 0, + "SPLIT_AND_SIZE_RESTRICTIONS": 1, + "PROCESS_SIZED_ELEMENTS_AND_RESTRICTIONS": 2, + "TRUNCATE_SIZED_RESTRICTION": 3, + } +) + +func (x StandardPTransforms_SplittableParDoComponents) Enum() *StandardPTransforms_SplittableParDoComponents { + p := new(StandardPTransforms_SplittableParDoComponents) + *p = x + return p +} + +func (x StandardPTransforms_SplittableParDoComponents) String() string { + return protoimpl.X.EnumStringOf(x.Descriptor(), protoreflect.EnumNumber(x)) +} + +func (StandardPTransforms_SplittableParDoComponents) Descriptor() protoreflect.EnumDescriptor { + return file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_enumTypes[5].Descriptor() +} + +func (StandardPTransforms_SplittableParDoComponents) Type() protoreflect.EnumType { + return &file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_enumTypes[5] +} + +func (x StandardPTransforms_SplittableParDoComponents) Number() protoreflect.EnumNumber { + return protoreflect.EnumNumber(x) +} + +// Payload for all of these: GroupIntoBatchesPayload +type StandardPTransforms_GroupIntoBatchesComponents int32 + +const ( + StandardPTransforms_GROUP_INTO_BATCHES StandardPTransforms_GroupIntoBatchesComponents = 0 +) + +// Enum value maps for StandardPTransforms_GroupIntoBatchesComponents. +var ( + StandardPTransforms_GroupIntoBatchesComponents_name = map[int32]string{ + 0: "GROUP_INTO_BATCHES", + } + StandardPTransforms_GroupIntoBatchesComponents_value = map[string]int32{ + "GROUP_INTO_BATCHES": 0, + } +) + +func (x StandardPTransforms_GroupIntoBatchesComponents) Enum() *StandardPTransforms_GroupIntoBatchesComponents { + p := new(StandardPTransforms_GroupIntoBatchesComponents) + *p = x + return p +} + +func (x StandardPTransforms_GroupIntoBatchesComponents) String() string { + return protoimpl.X.EnumStringOf(x.Descriptor(), protoreflect.EnumNumber(x)) +} + +func (StandardPTransforms_GroupIntoBatchesComponents) Descriptor() protoreflect.EnumDescriptor { + return file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_enumTypes[6].Descriptor() +} + +func (StandardPTransforms_GroupIntoBatchesComponents) Type() protoreflect.EnumType { + return &file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_enumTypes[6] +} + +func (x StandardPTransforms_GroupIntoBatchesComponents) Number() protoreflect.EnumNumber { + return protoreflect.EnumNumber(x) +} + +type StandardSideInputTypes_Enum int32 + +const ( + // Represents a view over a PCollection. + // + // StateGetRequests performed on this side input must use + // StateKey.IterableSideInput. + StandardSideInputTypes_ITERABLE StandardSideInputTypes_Enum = 0 + // Represents a view over a PCollection>. + // + // StateGetRequests performed on this side input must use + // StateKey.MultimapKeysSideInput or StateKey.MultimapSideInput. + StandardSideInputTypes_MULTIMAP StandardSideInputTypes_Enum = 1 +) + +// Enum value maps for StandardSideInputTypes_Enum. +var ( + StandardSideInputTypes_Enum_name = map[int32]string{ + 0: "ITERABLE", + 1: "MULTIMAP", + } + StandardSideInputTypes_Enum_value = map[string]int32{ + "ITERABLE": 0, + "MULTIMAP": 1, + } +) + +func (x StandardSideInputTypes_Enum) Enum() *StandardSideInputTypes_Enum { + p := new(StandardSideInputTypes_Enum) + *p = x + return p +} + +func (x StandardSideInputTypes_Enum) String() string { + return protoimpl.X.EnumStringOf(x.Descriptor(), protoreflect.EnumNumber(x)) +} + +func (StandardSideInputTypes_Enum) Descriptor() protoreflect.EnumDescriptor { + return file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_enumTypes[7].Descriptor() +} + +func (StandardSideInputTypes_Enum) Type() protoreflect.EnumType { + return &file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_enumTypes[7] +} + +func (x StandardSideInputTypes_Enum) Number() protoreflect.EnumNumber { + return protoreflect.EnumNumber(x) +} + +type StandardUserStateTypes_Enum int32 + +const ( + // Represents a user state specification that supports a bag. + // + // StateRequests performed on this user state must use + // StateKey.BagUserState. + StandardUserStateTypes_BAG StandardUserStateTypes_Enum = 0 + // Represents a user state specification that supports a multimap. + // + // StateRequests performed on this user state must use + // StateKey.MultimapKeysUserState or StateKey.MultimapUserState. + StandardUserStateTypes_MULTIMAP StandardUserStateTypes_Enum = 1 + // Represents a user state specification that supports an ordered list. + // + // StateRequests performed on this user state must use + // StateKey.OrderedListUserState. + StandardUserStateTypes_ORDERED_LIST StandardUserStateTypes_Enum = 2 +) + +// Enum value maps for StandardUserStateTypes_Enum. +var ( + StandardUserStateTypes_Enum_name = map[int32]string{ + 0: "BAG", + 1: "MULTIMAP", + 2: "ORDERED_LIST", + } + StandardUserStateTypes_Enum_value = map[string]int32{ + "BAG": 0, + "MULTIMAP": 1, + "ORDERED_LIST": 2, + } +) + +func (x StandardUserStateTypes_Enum) Enum() *StandardUserStateTypes_Enum { + p := new(StandardUserStateTypes_Enum) + *p = x + return p +} + +func (x StandardUserStateTypes_Enum) String() string { + return protoimpl.X.EnumStringOf(x.Descriptor(), protoreflect.EnumNumber(x)) +} + +func (StandardUserStateTypes_Enum) Descriptor() protoreflect.EnumDescriptor { + return file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_enumTypes[8].Descriptor() +} + +func (StandardUserStateTypes_Enum) Type() protoreflect.EnumType { + return &file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_enumTypes[8] +} + +func (x StandardUserStateTypes_Enum) Number() protoreflect.EnumNumber { + return protoreflect.EnumNumber(x) +} + +type IsBounded_Enum int32 + +const ( + IsBounded_UNSPECIFIED IsBounded_Enum = 0 + IsBounded_UNBOUNDED IsBounded_Enum = 1 + IsBounded_BOUNDED IsBounded_Enum = 2 +) + +// Enum value maps for IsBounded_Enum. +var ( + IsBounded_Enum_name = map[int32]string{ + 0: "UNSPECIFIED", + 1: "UNBOUNDED", + 2: "BOUNDED", + } + IsBounded_Enum_value = map[string]int32{ + "UNSPECIFIED": 0, + "UNBOUNDED": 1, + "BOUNDED": 2, + } +) + +func (x IsBounded_Enum) Enum() *IsBounded_Enum { + p := new(IsBounded_Enum) + *p = x + return p +} + +func (x IsBounded_Enum) String() string { + return protoimpl.X.EnumStringOf(x.Descriptor(), protoreflect.EnumNumber(x)) +} + +func (IsBounded_Enum) Descriptor() protoreflect.EnumDescriptor { + return file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_enumTypes[9].Descriptor() +} + +func (IsBounded_Enum) Type() protoreflect.EnumType { + return &file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_enumTypes[9] +} + +func (x IsBounded_Enum) Number() protoreflect.EnumNumber { + return protoreflect.EnumNumber(x) +} + +type StandardCoders_Enum int32 + +const ( + // Components: None + StandardCoders_BYTES StandardCoders_Enum = 0 + // Components: None + StandardCoders_STRING_UTF8 StandardCoders_Enum = 10 + // Components: The key and value coder, in that order. + StandardCoders_KV StandardCoders_Enum = 1 + // Components: None + StandardCoders_BOOL StandardCoders_Enum = 12 + // Variable length Encodes a 64-bit integer. + // Components: None + StandardCoders_VARINT StandardCoders_Enum = 2 + // Encodes the floating point value as a big-endian 64-bit integer + // according to the IEEE 754 double format bit layout. + // Components: None + StandardCoders_DOUBLE StandardCoders_Enum = 11 + // Encodes an iterable of elements. + // + // The encoding for an iterable [e1...eN] of known length N is + // + // fixed32(N) + // encode(e1) encode(e2) encode(e3) ... encode(eN) + // + // If the length is unknown, it is batched up into groups of size b1..bM + // and encoded as + // + // fixed32(-1) + // varInt64(b1) encode(e1) encode(e2) ... encode(e_b1) + // varInt64(b2) encode(e_(b1+1)) encode(e_(b1+2)) ... encode(e_(b1+b2)) + // ... + // varInt64(bM) encode(e_(N-bM+1)) encode(e_(N-bM+2)) ... encode(eN) + // varInt64(0) + // + // Components: Coder for a single element. + StandardCoders_ITERABLE StandardCoders_Enum = 3 + // Encodes a timer containing a user key, a dynamic timer tag, a clear bit, + // a fire timestamp, a hold timestamp, the windows and the paneinfo. + // The encoding is represented as: + // + // user key - user defined key, uses the component coder. + // dynamic timer tag - a string which identifies a timer. + // windows - uses component coders. + // clear bit - a boolean set for clearing the timer. + // fire timestamp - a big endian 8 byte integer representing millis-since-epoch. + // The encoded representation is shifted so that the byte representation of + // negative values are lexicographically ordered before the byte representation + // of positive values. This is typically done by subtracting -9223372036854775808 + // from the value and encoding it as a signed big endian integer. Example values: + // + // -9223372036854775808: 00 00 00 00 00 00 00 00 + // -255: 7F FF FF FF FF FF FF 01 + // -1: 7F FF FF FF FF FF FF FF + // 0: 80 00 00 00 00 00 00 00 + // 1: 80 00 00 00 00 00 00 01 + // 256: 80 00 00 00 00 00 01 00 + // 9223372036854775807: FF FF FF FF FF FF FF FF + // hold timestamp - similar to the fire timestamp. + // paneinfo - similar to the paneinfo of the windowed_value. + // + // Components: Coder for the key and windows. + StandardCoders_TIMER StandardCoders_Enum = 4 + // Components: None + StandardCoders_INTERVAL_WINDOW StandardCoders_Enum = 5 + // Components: The coder to attach a length prefix to + StandardCoders_LENGTH_PREFIX StandardCoders_Enum = 6 + // Components: None + StandardCoders_GLOBAL_WINDOW StandardCoders_Enum = 7 + // Encodes an element, the windows it is in, the timestamp of the element, + // and the pane of the element. The encoding is represented as: + // timestamp windows pane element + // + // timestamp - A big endian 8 byte integer representing millis-since-epoch. + // The encoded representation is shifted so that the byte representation + // of negative values are lexicographically ordered before the byte + // representation of positive values. This is typically done by + // subtracting -9223372036854775808 from the value and encoding it as a + // signed big endian integer. Example values: + // + // -9223372036854775808: 00 00 00 00 00 00 00 00 + // -255: 7F FF FF FF FF FF FF 01 + // -1: 7F FF FF FF FF FF FF FF + // 0: 80 00 00 00 00 00 00 00 + // 1: 80 00 00 00 00 00 00 01 + // 256: 80 00 00 00 00 00 01 00 + // 9223372036854775807: FF FF FF FF FF FF FF FF + // + // windows - The windows are encoded using the beam:coder:iterable:v1 + // format, where the windows are encoded using the supplied window + // coder. + // + // pane - The first byte of the pane info determines which type of + // encoding is used, as well as the is_first, is_last, and timing + // fields. If this byte is bits [0 1 2 3 4 5 6 7], then: + // * bits [0 1 2 3] determine the encoding as follows: + // 0000 - The entire pane info is encoded as a single byte. + // The is_first, is_last, and timing fields are encoded + // as below, and the index and non-speculative index are + // both zero (and hence are not encoded here). + // 0001 - The pane info is encoded as this byte plus a single + // VarInt encoed integer representing the pane index. The + // non-speculative index can be derived as follows: + // -1 if the pane is early, otherwise equal to index. + // 0010 - The pane info is encoded as this byte plus two VarInt + // encoded integers representing the pane index and + // non-speculative index respectively. + // * bits [4 5] encode the timing as follows: + // 00 - early + // 01 - on time + // 10 - late + // 11 - unknown + // * bit 6 is 1 if this is the first pane, 0 otherwise. + // * bit 7 is 1 if this is the last pane, 0 otherwise. + // + // element - The element incoded using the supplied element coder. + // + // Components: The element coder and the window coder, in that order. + StandardCoders_WINDOWED_VALUE StandardCoders_Enum = 8 + // A windowed value coder with parameterized timestamp, windows and pane info. + // Encodes an element with only the value of the windowed value. + // Decodes the value and assigns the parameterized timestamp, windows and pane info to the + // windowed value. + // Components: The element coder and the window coder, in that order + // The payload of this coder is an encoded windowed value using the + // beam:coder:windowed_value:v1 coder parameterized by a beam:coder:bytes:v1 + // element coder and the window coder that this param_windowed_value coder uses. + StandardCoders_PARAM_WINDOWED_VALUE StandardCoders_Enum = 14 + // Encodes an iterable of elements, some of which may be stored elsewhere. + // + // The encoding for a state-backed iterable is the same as that for + // an iterable, but the final varInt64(0) terminating the set of batches + // may instead be replaced by + // + // varInt64(-1) + // varInt64(len(token)) + // token + // + // where token is an opaque byte string that can be used to fetch the + // remainder of the iterable (e.g. over the state API). + // + // Components: Coder for a single element. + StandardCoders_STATE_BACKED_ITERABLE StandardCoders_Enum = 9 + // Encodes an arbitrary user defined window and its max timestamp (inclusive). + // The encoding format is: + // + // maxTimestamp window + // + // maxTimestamp - A big endian 8 byte integer representing millis-since-epoch. + // The encoded representation is shifted so that the byte representation + // of negative values are lexicographically ordered before the byte + // representation of positive values. This is typically done by + // subtracting -9223372036854775808 from the value and encoding it as a + // signed big endian integer. Example values: + // + // -9223372036854775808: 00 00 00 00 00 00 00 00 + // -255: 7F FF FF FF FF FF FF 01 + // -1: 7F FF FF FF FF FF FF FF + // 0: 80 00 00 00 00 00 00 00 + // 1: 80 00 00 00 00 00 00 01 + // 256: 80 00 00 00 00 00 01 00 + // 9223372036854775807: FF FF FF FF FF FF FF FF + // + // window - the window is encoded using the supplied window coder. + // + // Components: Coder for the custom window type. + StandardCoders_CUSTOM_WINDOW StandardCoders_Enum = 16 + // Encodes a "row", an element with a known schema, defined by an + // instance of Schema from schema.proto. + // + // A row is encoded as the concatenation of: + // - The number of attributes in the schema, encoded with + // beam:coder:varint:v1. This makes it possible to detect certain + // allowed schema changes (appending or removing columns) in + // long-running streaming pipelines. + // - A byte array representing a packed bitset indicating null fields (a + // 1 indicating a null) encoded with beam:coder:bytes:v1. The unused + // bits in the last byte must be set to 0. If there are no nulls an + // empty byte array is encoded. + // The two-byte bitset (not including the lenghth-prefix) for the row + // [NULL, 0, 0, 0, NULL, 0, 0, NULL, 0, NULL] would be + // [0b10010001, 0b00000010] + // - An encoding for each non-null field, concatenated together. + // + // Schema types are mapped to coders as follows: + // + // AtomicType: + // BYTE: not yet a standard coder (https://github.com/apache/beam/issues/19815) + // INT16: not yet a standard coder (https://github.com/apache/beam/issues/19815) + // INT32: beam:coder:varint:v1 + // INT64: beam:coder:varint:v1 + // FLOAT: not yet a standard coder (https://github.com/apache/beam/issues/19815) + // DOUBLE: beam:coder:double:v1 + // STRING: beam:coder:string_utf8:v1 + // BOOLEAN: beam:coder:bool:v1 + // BYTES: beam:coder:bytes:v1 + // ArrayType: beam:coder:iterable:v1 (always has a known length) + // MapType: not a standard coder, specification defined below. + // RowType: beam:coder:row:v1 + // LogicalType: Uses the coder for its representation. + // + // The MapType is encoded by: + // - An INT32 representing the size of the map (N) + // - Followed by N interleaved keys and values, encoded with their + // corresponding coder. + // + // Nullable types in container types (ArrayType, MapType) per the + // encoding described for general Nullable types below. + // + // Logical types understood by all SDKs should be defined in schema.proto. + // Example of well known logical types: + // + // beam:logical_type:schema:v1 + // - Representation type: BYTES + // - A Beam Schema stored as a serialized proto. + // + // The payload for RowCoder is an instance of Schema. + // Components: None + StandardCoders_ROW StandardCoders_Enum = 13 + // Encodes a user key and a shard id which is an opaque byte string. + // + // The encoding for a sharded key consists of a shard id byte string and the + // encoded user key in the following order: + // + // - shard id using beam:coder:bytes:v1 + // - encoded user key + // + // Examples: + // user key with an empty shard id + // + // 0x00 + // encode(user_key) + // + // user key with a shard id taking up two bytes. + // + // 0x02 + // 0x11 0x22 + // encode(user_key) + // + // Components: the user key coder. + StandardCoders_SHARDED_KEY StandardCoders_Enum = 15 + // Wraps a coder of a potentially null value + // A Nullable Type is encoded by: + // - A one byte null indicator, 0x00 for null values, or 0x01 for present + // values. + // - For present values the null indicator is followed by the value + // encoded with it's corresponding coder. + // + // Components: single coder for the value + StandardCoders_NULLABLE StandardCoders_Enum = 17 +) + +// Enum value maps for StandardCoders_Enum. +var ( + StandardCoders_Enum_name = map[int32]string{ + 0: "BYTES", + 10: "STRING_UTF8", + 1: "KV", + 12: "BOOL", + 2: "VARINT", + 11: "DOUBLE", + 3: "ITERABLE", + 4: "TIMER", + 5: "INTERVAL_WINDOW", + 6: "LENGTH_PREFIX", + 7: "GLOBAL_WINDOW", + 8: "WINDOWED_VALUE", + 14: "PARAM_WINDOWED_VALUE", + 9: "STATE_BACKED_ITERABLE", + 16: "CUSTOM_WINDOW", + 13: "ROW", + 15: "SHARDED_KEY", + 17: "NULLABLE", + } + StandardCoders_Enum_value = map[string]int32{ + "BYTES": 0, + "STRING_UTF8": 10, + "KV": 1, + "BOOL": 12, + "VARINT": 2, + "DOUBLE": 11, + "ITERABLE": 3, + "TIMER": 4, + "INTERVAL_WINDOW": 5, + "LENGTH_PREFIX": 6, + "GLOBAL_WINDOW": 7, + "WINDOWED_VALUE": 8, + "PARAM_WINDOWED_VALUE": 14, + "STATE_BACKED_ITERABLE": 9, + "CUSTOM_WINDOW": 16, + "ROW": 13, + "SHARDED_KEY": 15, + "NULLABLE": 17, + } +) + +func (x StandardCoders_Enum) Enum() *StandardCoders_Enum { + p := new(StandardCoders_Enum) + *p = x + return p +} + +func (x StandardCoders_Enum) String() string { + return protoimpl.X.EnumStringOf(x.Descriptor(), protoreflect.EnumNumber(x)) +} + +func (StandardCoders_Enum) Descriptor() protoreflect.EnumDescriptor { + return file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_enumTypes[10].Descriptor() +} + +func (StandardCoders_Enum) Type() protoreflect.EnumType { + return &file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_enumTypes[10] +} + +func (x StandardCoders_Enum) Number() protoreflect.EnumNumber { + return protoreflect.EnumNumber(x) +} + +type MergeStatus_Enum int32 + +const ( + MergeStatus_UNSPECIFIED MergeStatus_Enum = 0 + // The WindowFn does not require merging. + // Examples: global window, FixedWindows, SlidingWindows + MergeStatus_NON_MERGING MergeStatus_Enum = 1 + // The WindowFn is merging and the PCollection has not had merging + // performed. + // Example: Sessions prior to a GroupByKey + MergeStatus_NEEDS_MERGE MergeStatus_Enum = 2 + // The WindowFn is merging and the PCollection has had merging occur + // already. + // Example: Sessions after a GroupByKey + MergeStatus_ALREADY_MERGED MergeStatus_Enum = 3 +) + +// Enum value maps for MergeStatus_Enum. +var ( + MergeStatus_Enum_name = map[int32]string{ + 0: "UNSPECIFIED", + 1: "NON_MERGING", + 2: "NEEDS_MERGE", + 3: "ALREADY_MERGED", + } + MergeStatus_Enum_value = map[string]int32{ + "UNSPECIFIED": 0, + "NON_MERGING": 1, + "NEEDS_MERGE": 2, + "ALREADY_MERGED": 3, + } +) + +func (x MergeStatus_Enum) Enum() *MergeStatus_Enum { + p := new(MergeStatus_Enum) + *p = x + return p +} + +func (x MergeStatus_Enum) String() string { + return protoimpl.X.EnumStringOf(x.Descriptor(), protoreflect.EnumNumber(x)) +} + +func (MergeStatus_Enum) Descriptor() protoreflect.EnumDescriptor { + return file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_enumTypes[11].Descriptor() +} + +func (MergeStatus_Enum) Type() protoreflect.EnumType { + return &file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_enumTypes[11] +} + +func (x MergeStatus_Enum) Number() protoreflect.EnumNumber { + return protoreflect.EnumNumber(x) +} + +type AccumulationMode_Enum int32 + +const ( + AccumulationMode_UNSPECIFIED AccumulationMode_Enum = 0 + // The aggregation is discarded when it is output + AccumulationMode_DISCARDING AccumulationMode_Enum = 1 + // The aggregation is accumulated across outputs + AccumulationMode_ACCUMULATING AccumulationMode_Enum = 2 + // The aggregation emits retractions when it is output + AccumulationMode_RETRACTING AccumulationMode_Enum = 3 +) + +// Enum value maps for AccumulationMode_Enum. +var ( + AccumulationMode_Enum_name = map[int32]string{ + 0: "UNSPECIFIED", + 1: "DISCARDING", + 2: "ACCUMULATING", + 3: "RETRACTING", + } + AccumulationMode_Enum_value = map[string]int32{ + "UNSPECIFIED": 0, + "DISCARDING": 1, + "ACCUMULATING": 2, + "RETRACTING": 3, + } +) + +func (x AccumulationMode_Enum) Enum() *AccumulationMode_Enum { + p := new(AccumulationMode_Enum) + *p = x + return p +} + +func (x AccumulationMode_Enum) String() string { + return protoimpl.X.EnumStringOf(x.Descriptor(), protoreflect.EnumNumber(x)) +} + +func (AccumulationMode_Enum) Descriptor() protoreflect.EnumDescriptor { + return file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_enumTypes[12].Descriptor() +} + +func (AccumulationMode_Enum) Type() protoreflect.EnumType { + return &file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_enumTypes[12] +} + +func (x AccumulationMode_Enum) Number() protoreflect.EnumNumber { + return protoreflect.EnumNumber(x) +} + +type ClosingBehavior_Enum int32 + +const ( + ClosingBehavior_UNSPECIFIED ClosingBehavior_Enum = 0 + // Emit output when a window expires, whether or not there has been + // any new data since the last output. + ClosingBehavior_EMIT_ALWAYS ClosingBehavior_Enum = 1 + // Only emit output when new data has arrives since the last output + ClosingBehavior_EMIT_IF_NONEMPTY ClosingBehavior_Enum = 2 +) + +// Enum value maps for ClosingBehavior_Enum. +var ( + ClosingBehavior_Enum_name = map[int32]string{ + 0: "UNSPECIFIED", + 1: "EMIT_ALWAYS", + 2: "EMIT_IF_NONEMPTY", + } + ClosingBehavior_Enum_value = map[string]int32{ + "UNSPECIFIED": 0, + "EMIT_ALWAYS": 1, + "EMIT_IF_NONEMPTY": 2, + } +) + +func (x ClosingBehavior_Enum) Enum() *ClosingBehavior_Enum { + p := new(ClosingBehavior_Enum) + *p = x + return p +} + +func (x ClosingBehavior_Enum) String() string { + return protoimpl.X.EnumStringOf(x.Descriptor(), protoreflect.EnumNumber(x)) +} + +func (ClosingBehavior_Enum) Descriptor() protoreflect.EnumDescriptor { + return file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_enumTypes[13].Descriptor() +} + +func (ClosingBehavior_Enum) Type() protoreflect.EnumType { + return &file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_enumTypes[13] +} + +func (x ClosingBehavior_Enum) Number() protoreflect.EnumNumber { + return protoreflect.EnumNumber(x) +} + +type OnTimeBehavior_Enum int32 + +const ( + OnTimeBehavior_UNSPECIFIED OnTimeBehavior_Enum = 0 + // Always fire the on-time pane. Even if there is no new data since + // the previous firing, an element will be produced. + OnTimeBehavior_FIRE_ALWAYS OnTimeBehavior_Enum = 1 + // Only fire the on-time pane if there is new data since the previous firing. + OnTimeBehavior_FIRE_IF_NONEMPTY OnTimeBehavior_Enum = 2 +) + +// Enum value maps for OnTimeBehavior_Enum. +var ( + OnTimeBehavior_Enum_name = map[int32]string{ + 0: "UNSPECIFIED", + 1: "FIRE_ALWAYS", + 2: "FIRE_IF_NONEMPTY", + } + OnTimeBehavior_Enum_value = map[string]int32{ + "UNSPECIFIED": 0, + "FIRE_ALWAYS": 1, + "FIRE_IF_NONEMPTY": 2, + } +) + +func (x OnTimeBehavior_Enum) Enum() *OnTimeBehavior_Enum { + p := new(OnTimeBehavior_Enum) + *p = x + return p +} + +func (x OnTimeBehavior_Enum) String() string { + return protoimpl.X.EnumStringOf(x.Descriptor(), protoreflect.EnumNumber(x)) +} + +func (OnTimeBehavior_Enum) Descriptor() protoreflect.EnumDescriptor { + return file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_enumTypes[14].Descriptor() +} + +func (OnTimeBehavior_Enum) Type() protoreflect.EnumType { + return &file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_enumTypes[14] +} + +func (x OnTimeBehavior_Enum) Number() protoreflect.EnumNumber { + return protoreflect.EnumNumber(x) +} + +type OutputTime_Enum int32 + +const ( + OutputTime_UNSPECIFIED OutputTime_Enum = 0 + // The output has the timestamp of the end of the window. + OutputTime_END_OF_WINDOW OutputTime_Enum = 1 + // The output has the latest timestamp of the input elements since + // the last output. + OutputTime_LATEST_IN_PANE OutputTime_Enum = 2 + // The output has the earliest timestamp of the input elements since + // the last output. + OutputTime_EARLIEST_IN_PANE OutputTime_Enum = 3 +) + +// Enum value maps for OutputTime_Enum. +var ( + OutputTime_Enum_name = map[int32]string{ + 0: "UNSPECIFIED", + 1: "END_OF_WINDOW", + 2: "LATEST_IN_PANE", + 3: "EARLIEST_IN_PANE", + } + OutputTime_Enum_value = map[string]int32{ + "UNSPECIFIED": 0, + "END_OF_WINDOW": 1, + "LATEST_IN_PANE": 2, + "EARLIEST_IN_PANE": 3, + } +) + +func (x OutputTime_Enum) Enum() *OutputTime_Enum { + p := new(OutputTime_Enum) + *p = x + return p +} + +func (x OutputTime_Enum) String() string { + return protoimpl.X.EnumStringOf(x.Descriptor(), protoreflect.EnumNumber(x)) +} + +func (OutputTime_Enum) Descriptor() protoreflect.EnumDescriptor { + return file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_enumTypes[15].Descriptor() +} + +func (OutputTime_Enum) Type() protoreflect.EnumType { + return &file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_enumTypes[15] +} + +func (x OutputTime_Enum) Number() protoreflect.EnumNumber { + return protoreflect.EnumNumber(x) +} + +type TimeDomain_Enum int32 + +const ( + TimeDomain_UNSPECIFIED TimeDomain_Enum = 0 + // Event time is time from the perspective of the data + TimeDomain_EVENT_TIME TimeDomain_Enum = 1 + // Processing time is time from the perspective of the + // execution of your pipeline + TimeDomain_PROCESSING_TIME TimeDomain_Enum = 2 +) + +// Enum value maps for TimeDomain_Enum. +var ( + TimeDomain_Enum_name = map[int32]string{ + 0: "UNSPECIFIED", + 1: "EVENT_TIME", + 2: "PROCESSING_TIME", + } + TimeDomain_Enum_value = map[string]int32{ + "UNSPECIFIED": 0, + "EVENT_TIME": 1, + "PROCESSING_TIME": 2, + } +) + +func (x TimeDomain_Enum) Enum() *TimeDomain_Enum { + p := new(TimeDomain_Enum) + *p = x + return p +} + +func (x TimeDomain_Enum) String() string { + return protoimpl.X.EnumStringOf(x.Descriptor(), protoreflect.EnumNumber(x)) +} + +func (TimeDomain_Enum) Descriptor() protoreflect.EnumDescriptor { + return file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_enumTypes[16].Descriptor() +} + +func (TimeDomain_Enum) Type() protoreflect.EnumType { + return &file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_enumTypes[16] +} + +func (x TimeDomain_Enum) Number() protoreflect.EnumNumber { + return protoreflect.EnumNumber(x) +} + +type StandardArtifacts_Types int32 + +const ( + // A URN for locally-accessible artifact files. + // payload: ArtifactFilePayload + StandardArtifacts_FILE StandardArtifacts_Types = 0 + // A URN for artifacts described by URLs. + // payload: ArtifactUrlPayload + StandardArtifacts_URL StandardArtifacts_Types = 1 + // A URN for artifacts embedded in ArtifactInformation proto. + // payload: EmbeddedFilePayload. + StandardArtifacts_EMBEDDED StandardArtifacts_Types = 2 + // A URN for Python artifacts hosted on PYPI. + // payload: PypiPayload + StandardArtifacts_PYPI StandardArtifacts_Types = 3 + // A URN for Java artifacts hosted on a Maven repository. + // payload: MavenPayload + StandardArtifacts_MAVEN StandardArtifacts_Types = 4 + // A URN for deferred artifacts. + // payload: DeferredArtifactPayload + StandardArtifacts_DEFERRED StandardArtifacts_Types = 5 +) + +// Enum value maps for StandardArtifacts_Types. +var ( + StandardArtifacts_Types_name = map[int32]string{ + 0: "FILE", + 1: "URL", + 2: "EMBEDDED", + 3: "PYPI", + 4: "MAVEN", + 5: "DEFERRED", + } + StandardArtifacts_Types_value = map[string]int32{ + "FILE": 0, + "URL": 1, + "EMBEDDED": 2, + "PYPI": 3, + "MAVEN": 4, + "DEFERRED": 5, + } +) + +func (x StandardArtifacts_Types) Enum() *StandardArtifacts_Types { + p := new(StandardArtifacts_Types) + *p = x + return p +} + +func (x StandardArtifacts_Types) String() string { + return protoimpl.X.EnumStringOf(x.Descriptor(), protoreflect.EnumNumber(x)) +} + +func (StandardArtifacts_Types) Descriptor() protoreflect.EnumDescriptor { + return file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_enumTypes[17].Descriptor() +} + +func (StandardArtifacts_Types) Type() protoreflect.EnumType { + return &file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_enumTypes[17] +} + +func (x StandardArtifacts_Types) Number() protoreflect.EnumNumber { + return protoreflect.EnumNumber(x) +} + +type StandardArtifacts_Roles int32 + +const ( + // A URN for staging-to role. + // payload: ArtifactStagingToRolePayload + StandardArtifacts_STAGING_TO StandardArtifacts_Roles = 0 + // A URN for pip-requirements-file role. + // payload: None + StandardArtifacts_PIP_REQUIREMENTS_FILE StandardArtifacts_Roles = 1 + // A URN for the Go worker binary role. + // This represents the executable for a Go SDK environment. + // A Go environment may have one such artifact with this role. + // payload: None + StandardArtifacts_GO_WORKER_BINARY StandardArtifacts_Roles = 2 +) + +// Enum value maps for StandardArtifacts_Roles. +var ( + StandardArtifacts_Roles_name = map[int32]string{ + 0: "STAGING_TO", + 1: "PIP_REQUIREMENTS_FILE", + 2: "GO_WORKER_BINARY", + } + StandardArtifacts_Roles_value = map[string]int32{ + "STAGING_TO": 0, + "PIP_REQUIREMENTS_FILE": 1, + "GO_WORKER_BINARY": 2, + } +) + +func (x StandardArtifacts_Roles) Enum() *StandardArtifacts_Roles { + p := new(StandardArtifacts_Roles) + *p = x + return p +} + +func (x StandardArtifacts_Roles) String() string { + return protoimpl.X.EnumStringOf(x.Descriptor(), protoreflect.EnumNumber(x)) +} + +func (StandardArtifacts_Roles) Descriptor() protoreflect.EnumDescriptor { + return file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_enumTypes[18].Descriptor() +} + +func (StandardArtifacts_Roles) Type() protoreflect.EnumType { + return &file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_enumTypes[18] +} + +func (x StandardArtifacts_Roles) Number() protoreflect.EnumNumber { + return protoreflect.EnumNumber(x) +} + +type StandardEnvironments_Environments int32 + +const ( + // A managed docker container to run user code. + // Payload should be DockerPayload. + StandardEnvironments_DOCKER StandardEnvironments_Environments = 0 + // A managed native process to run user code. + // Payload should be ProcessPayload. + StandardEnvironments_PROCESS StandardEnvironments_Environments = 1 + // An external non managed process to run user code. + // Payload should be ExternalPayload. + StandardEnvironments_EXTERNAL StandardEnvironments_Environments = 2 + // Used as a stub when context is missing a runner-provided default environment. + StandardEnvironments_DEFAULT StandardEnvironments_Environments = 3 + // A selection of equivalent fully-specified environments a runner may use. + // Note that this environment itself does not declare any dependencies or capabilities, + // as those may differ among the several alternatives. + // Payload should be AnyOfEnvironmentPayload. + StandardEnvironments_ANYOF StandardEnvironments_Environments = 4 +) + +// Enum value maps for StandardEnvironments_Environments. +var ( + StandardEnvironments_Environments_name = map[int32]string{ + 0: "DOCKER", + 1: "PROCESS", + 2: "EXTERNAL", + 3: "DEFAULT", + 4: "ANYOF", + } + StandardEnvironments_Environments_value = map[string]int32{ + "DOCKER": 0, + "PROCESS": 1, + "EXTERNAL": 2, + "DEFAULT": 3, + "ANYOF": 4, + } +) + +func (x StandardEnvironments_Environments) Enum() *StandardEnvironments_Environments { + p := new(StandardEnvironments_Environments) + *p = x + return p +} + +func (x StandardEnvironments_Environments) String() string { + return protoimpl.X.EnumStringOf(x.Descriptor(), protoreflect.EnumNumber(x)) +} + +func (StandardEnvironments_Environments) Descriptor() protoreflect.EnumDescriptor { + return file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_enumTypes[19].Descriptor() +} + +func (StandardEnvironments_Environments) Type() protoreflect.EnumType { + return &file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_enumTypes[19] +} + +func (x StandardEnvironments_Environments) Number() protoreflect.EnumNumber { + return protoreflect.EnumNumber(x) +} + +type StandardProtocols_Enum int32 + +const ( + // Indicates suport for progress reporting via the legacy Metrics proto. + StandardProtocols_LEGACY_PROGRESS_REPORTING StandardProtocols_Enum = 0 + // Indicates suport for progress reporting via the new MonitoringInfo proto. + StandardProtocols_PROGRESS_REPORTING StandardProtocols_Enum = 1 + // Indicates suport for worker status protocol defined at + // https://s.apache.org/beam-fn-api-harness-status. + StandardProtocols_WORKER_STATUS StandardProtocols_Enum = 2 + // Indicates this SDK can take advantage of multiple cores when processing + // concurrent process bundle requests. (Note that all SDKs must process + // an unbounded number of concurrent process bundle requests; this capability + // simply indicates this SDK can actually parallelize the work across multiple + // cores. + StandardProtocols_MULTI_CORE_BUNDLE_PROCESSING StandardProtocols_Enum = 3 + // Indicates this SDK can cheaply spawn sibling workers (e.g. within the + // same container) to work around the fact that it cannot take advantage + // of multiple cores (i.e. MULTI_CORE_BUNDLE_PROCESSING is not set). + StandardProtocols_SIBLING_WORKERS StandardProtocols_Enum = 5 + // Indicates that this SDK handles the InstructionRequest of type + // HarnessMonitoringInfosRequest. + // A request to provide full MonitoringInfo data associated with + // the entire SDK harness process, not specific to a bundle. + StandardProtocols_HARNESS_MONITORING_INFOS StandardProtocols_Enum = 4 + // Indicates that this SDK can process elements embedded in the + // ProcessBundleRequest. See more about the protocol at + // https://s.apache.org/beam-fn-api-control-data-embedding + StandardProtocols_CONTROL_REQUEST_ELEMENTS_EMBEDDING StandardProtocols_Enum = 6 + // Indicates that this SDK can cache user state and side inputs across + // bundle boundaries. This is a hint to runners that runners can rely on the + // SDKs ability to store the data in memory reducing the amount of memory + // used overall. + StandardProtocols_STATE_CACHING StandardProtocols_Enum = 7 + // Indicates that this SDK can sample in-flight elements. These samples can + // then be queried using the SampleDataRequest. Samples are uniquely associated + // with a PCollection. Meaning, samples are taken for each PCollection + // during bundle processing. This is disabled by default and enabled with the + // `enable_data_sampling` experiment. + StandardProtocols_DATA_SAMPLING StandardProtocols_Enum = 8 + // Indicates whether the SDK sets the consuming_received_data field on + // the ProcessBundleProgressResponse messages. + StandardProtocols_SDK_CONSUMING_RECEIVED_DATA StandardProtocols_Enum = 9 + // Indicates whether the SDK supports ordered list state. + StandardProtocols_ORDERED_LIST_STATE StandardProtocols_Enum = 10 +) + +// Enum value maps for StandardProtocols_Enum. +var ( + StandardProtocols_Enum_name = map[int32]string{ + 0: "LEGACY_PROGRESS_REPORTING", + 1: "PROGRESS_REPORTING", + 2: "WORKER_STATUS", + 3: "MULTI_CORE_BUNDLE_PROCESSING", + 5: "SIBLING_WORKERS", + 4: "HARNESS_MONITORING_INFOS", + 6: "CONTROL_REQUEST_ELEMENTS_EMBEDDING", + 7: "STATE_CACHING", + 8: "DATA_SAMPLING", + 9: "SDK_CONSUMING_RECEIVED_DATA", + 10: "ORDERED_LIST_STATE", + } + StandardProtocols_Enum_value = map[string]int32{ + "LEGACY_PROGRESS_REPORTING": 0, + "PROGRESS_REPORTING": 1, + "WORKER_STATUS": 2, + "MULTI_CORE_BUNDLE_PROCESSING": 3, + "SIBLING_WORKERS": 5, + "HARNESS_MONITORING_INFOS": 4, + "CONTROL_REQUEST_ELEMENTS_EMBEDDING": 6, + "STATE_CACHING": 7, + "DATA_SAMPLING": 8, + "SDK_CONSUMING_RECEIVED_DATA": 9, + "ORDERED_LIST_STATE": 10, + } +) + +func (x StandardProtocols_Enum) Enum() *StandardProtocols_Enum { + p := new(StandardProtocols_Enum) + *p = x + return p +} + +func (x StandardProtocols_Enum) String() string { + return protoimpl.X.EnumStringOf(x.Descriptor(), protoreflect.EnumNumber(x)) +} + +func (StandardProtocols_Enum) Descriptor() protoreflect.EnumDescriptor { + return file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_enumTypes[20].Descriptor() +} + +func (StandardProtocols_Enum) Type() protoreflect.EnumType { + return &file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_enumTypes[20] +} + +func (x StandardProtocols_Enum) Number() protoreflect.EnumNumber { + return protoreflect.EnumNumber(x) +} + +type StandardRunnerProtocols_Enum int32 + +const ( + // Indicates suport the MonitoringInfo short id protocol. + StandardRunnerProtocols_MONITORING_INFO_SHORT_IDS StandardRunnerProtocols_Enum = 0 + // Indicates that this runner can process elements embedded in the + // ProcessBundleResponse. See more about the protocol at + // https://s.apache.org/beam-fn-api-control-data-embedding + StandardRunnerProtocols_CONTROL_RESPONSE_ELEMENTS_EMBEDDING StandardRunnerProtocols_Enum = 6 + // Indicates that this runner can handle the multimap_keys_values_side_input + // style read of a multimap side input. + StandardRunnerProtocols_MULTIMAP_KEYS_VALUES_SIDE_INPUT StandardRunnerProtocols_Enum = 7 +) + +// Enum value maps for StandardRunnerProtocols_Enum. +var ( + StandardRunnerProtocols_Enum_name = map[int32]string{ + 0: "MONITORING_INFO_SHORT_IDS", + 6: "CONTROL_RESPONSE_ELEMENTS_EMBEDDING", + 7: "MULTIMAP_KEYS_VALUES_SIDE_INPUT", + } + StandardRunnerProtocols_Enum_value = map[string]int32{ + "MONITORING_INFO_SHORT_IDS": 0, + "CONTROL_RESPONSE_ELEMENTS_EMBEDDING": 6, + "MULTIMAP_KEYS_VALUES_SIDE_INPUT": 7, + } +) + +func (x StandardRunnerProtocols_Enum) Enum() *StandardRunnerProtocols_Enum { + p := new(StandardRunnerProtocols_Enum) + *p = x + return p +} + +func (x StandardRunnerProtocols_Enum) String() string { + return protoimpl.X.EnumStringOf(x.Descriptor(), protoreflect.EnumNumber(x)) +} + +func (StandardRunnerProtocols_Enum) Descriptor() protoreflect.EnumDescriptor { + return file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_enumTypes[21].Descriptor() +} + +func (StandardRunnerProtocols_Enum) Type() protoreflect.EnumType { + return &file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_enumTypes[21] +} + +func (x StandardRunnerProtocols_Enum) Number() protoreflect.EnumNumber { + return protoreflect.EnumNumber(x) +} + +type StandardRequirements_Enum int32 + +const ( + // This requirement indicates the state_specs and timer_family_specs fields of ParDo + // transform payloads must be inspected. + StandardRequirements_REQUIRES_STATEFUL_PROCESSING StandardRequirements_Enum = 0 + // This requirement indicates the requests_finalization field of ParDo + // transform payloads must be inspected. + StandardRequirements_REQUIRES_BUNDLE_FINALIZATION StandardRequirements_Enum = 1 + // This requirement indicates the requires_stable_input field of ParDo + // transform payloads must be inspected. + StandardRequirements_REQUIRES_STABLE_INPUT StandardRequirements_Enum = 2 + // This requirement indicates the requires_time_sorted_input field of ParDo + // transform payloads must be inspected. + StandardRequirements_REQUIRES_TIME_SORTED_INPUT StandardRequirements_Enum = 3 + // This requirement indicates the restriction_coder_id field of ParDo + // transform payloads must be inspected. + StandardRequirements_REQUIRES_SPLITTABLE_DOFN StandardRequirements_Enum = 4 + // This requirement indicates that the on_window_expiration_timer_family_spec field + // of ParDo transform payloads must be inspected. + StandardRequirements_REQUIRES_ON_WINDOW_EXPIRATION StandardRequirements_Enum = 5 +) + +// Enum value maps for StandardRequirements_Enum. +var ( + StandardRequirements_Enum_name = map[int32]string{ + 0: "REQUIRES_STATEFUL_PROCESSING", + 1: "REQUIRES_BUNDLE_FINALIZATION", + 2: "REQUIRES_STABLE_INPUT", + 3: "REQUIRES_TIME_SORTED_INPUT", + 4: "REQUIRES_SPLITTABLE_DOFN", + 5: "REQUIRES_ON_WINDOW_EXPIRATION", + } + StandardRequirements_Enum_value = map[string]int32{ + "REQUIRES_STATEFUL_PROCESSING": 0, + "REQUIRES_BUNDLE_FINALIZATION": 1, + "REQUIRES_STABLE_INPUT": 2, + "REQUIRES_TIME_SORTED_INPUT": 3, + "REQUIRES_SPLITTABLE_DOFN": 4, + "REQUIRES_ON_WINDOW_EXPIRATION": 5, + } +) + +func (x StandardRequirements_Enum) Enum() *StandardRequirements_Enum { + p := new(StandardRequirements_Enum) + *p = x + return p +} + +func (x StandardRequirements_Enum) String() string { + return protoimpl.X.EnumStringOf(x.Descriptor(), protoreflect.EnumNumber(x)) +} + +func (StandardRequirements_Enum) Descriptor() protoreflect.EnumDescriptor { + return file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_enumTypes[22].Descriptor() +} + +func (StandardRequirements_Enum) Type() protoreflect.EnumType { + return &file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_enumTypes[22] +} + +func (x StandardRequirements_Enum) Number() protoreflect.EnumNumber { + return protoreflect.EnumNumber(x) +} + +type StandardDisplayData_DisplayData int32 + +const ( + // A string label and value. Has a payload containing an encoded + // LabelledPayload. + StandardDisplayData_LABELLED StandardDisplayData_DisplayData = 0 +) + +// Enum value maps for StandardDisplayData_DisplayData. +var ( + StandardDisplayData_DisplayData_name = map[int32]string{ + 0: "LABELLED", + } + StandardDisplayData_DisplayData_value = map[string]int32{ + "LABELLED": 0, + } +) + +func (x StandardDisplayData_DisplayData) Enum() *StandardDisplayData_DisplayData { + p := new(StandardDisplayData_DisplayData) + *p = x + return p +} + +func (x StandardDisplayData_DisplayData) String() string { + return protoimpl.X.EnumStringOf(x.Descriptor(), protoreflect.EnumNumber(x)) +} + +func (StandardDisplayData_DisplayData) Descriptor() protoreflect.EnumDescriptor { + return file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_enumTypes[23].Descriptor() +} + +func (StandardDisplayData_DisplayData) Type() protoreflect.EnumType { + return &file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_enumTypes[23] +} + +func (x StandardDisplayData_DisplayData) Number() protoreflect.EnumNumber { + return protoreflect.EnumNumber(x) +} + +type StandardResourceHints_Enum int32 + +const ( + // Describes hardware accelerators that are desired to have in the execution environment. + // Payload: ASCII encoded string with the following format: "type:;count:;" where type + // is an accelerator sku, count is the number of accelerators per worker, and options are + // related options flags. + StandardResourceHints_ACCELERATOR StandardResourceHints_Enum = 0 + // Describes desired minimal available RAM size in transform's execution environment. + // SDKs should convert the size to bytes, but can allow users to specify human-friendly units (e.g. GiB). + // Payload: ASCII encoded string of the base 10 representation of an integer number of bytes. + StandardResourceHints_MIN_RAM_BYTES StandardResourceHints_Enum = 1 + // Describes desired number of CPUs available in transform's execution environment. + // SDKs should accept and validate a positive integer count. + // Payload: ASCII encoded string of the base 10 representation of an integer number of CPUs. + StandardResourceHints_CPU_COUNT StandardResourceHints_Enum = 2 +) + +// Enum value maps for StandardResourceHints_Enum. +var ( + StandardResourceHints_Enum_name = map[int32]string{ + 0: "ACCELERATOR", + 1: "MIN_RAM_BYTES", + 2: "CPU_COUNT", + } + StandardResourceHints_Enum_value = map[string]int32{ + "ACCELERATOR": 0, + "MIN_RAM_BYTES": 1, + "CPU_COUNT": 2, + } +) + +func (x StandardResourceHints_Enum) Enum() *StandardResourceHints_Enum { + p := new(StandardResourceHints_Enum) + *p = x + return p +} + +func (x StandardResourceHints_Enum) String() string { + return protoimpl.X.EnumStringOf(x.Descriptor(), protoreflect.EnumNumber(x)) +} + +func (StandardResourceHints_Enum) Descriptor() protoreflect.EnumDescriptor { + return file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_enumTypes[24].Descriptor() +} + +func (StandardResourceHints_Enum) Type() protoreflect.EnumType { + return &file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_enumTypes[24] +} + +func (x StandardResourceHints_Enum) Number() protoreflect.EnumNumber { + return protoreflect.EnumNumber(x) +} + +type BeamConstants struct { + state protoimpl.MessageState `protogen:"opaque.v1"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *BeamConstants) Reset() { + *x = BeamConstants{} + mi := &file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_msgTypes[0] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *BeamConstants) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*BeamConstants) ProtoMessage() {} + +func (x *BeamConstants) ProtoReflect() protoreflect.Message { + mi := &file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_msgTypes[0] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +type BeamConstants_builder struct { + _ [0]func() // Prevents comparability and use of unkeyed literals for the builder. + +} + +func (b0 BeamConstants_builder) Build() *BeamConstants { + m0 := &BeamConstants{} + b, x := &b0, m0 + _, _ = b, x + return m0 +} + +// A set of mappings from id to message. This is included as an optional field +// on any proto message that may contain references needing resolution. +type Components struct { + state protoimpl.MessageState `protogen:"opaque.v1"` + xxx_hidden_Transforms map[string]*PTransform `protobuf:"bytes,1,rep,name=transforms,proto3" json:"transforms,omitempty" protobuf_key:"bytes,1,opt,name=key" protobuf_val:"bytes,2,opt,name=value"` + xxx_hidden_Pcollections map[string]*PCollection `protobuf:"bytes,2,rep,name=pcollections,proto3" json:"pcollections,omitempty" protobuf_key:"bytes,1,opt,name=key" protobuf_val:"bytes,2,opt,name=value"` + xxx_hidden_WindowingStrategies map[string]*WindowingStrategy `protobuf:"bytes,3,rep,name=windowing_strategies,json=windowingStrategies,proto3" json:"windowing_strategies,omitempty" protobuf_key:"bytes,1,opt,name=key" protobuf_val:"bytes,2,opt,name=value"` + xxx_hidden_Coders map[string]*Coder `protobuf:"bytes,4,rep,name=coders,proto3" json:"coders,omitempty" protobuf_key:"bytes,1,opt,name=key" protobuf_val:"bytes,2,opt,name=value"` + xxx_hidden_Environments map[string]*Environment `protobuf:"bytes,5,rep,name=environments,proto3" json:"environments,omitempty" protobuf_key:"bytes,1,opt,name=key" protobuf_val:"bytes,2,opt,name=value"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *Components) Reset() { + *x = Components{} + mi := &file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_msgTypes[1] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *Components) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*Components) ProtoMessage() {} + +func (x *Components) ProtoReflect() protoreflect.Message { + mi := &file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_msgTypes[1] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +func (x *Components) GetTransforms() map[string]*PTransform { + if x != nil { + return x.xxx_hidden_Transforms + } + return nil +} + +func (x *Components) GetPcollections() map[string]*PCollection { + if x != nil { + return x.xxx_hidden_Pcollections + } + return nil +} + +func (x *Components) GetWindowingStrategies() map[string]*WindowingStrategy { + if x != nil { + return x.xxx_hidden_WindowingStrategies + } + return nil +} + +func (x *Components) GetCoders() map[string]*Coder { + if x != nil { + return x.xxx_hidden_Coders + } + return nil +} + +func (x *Components) GetEnvironments() map[string]*Environment { + if x != nil { + return x.xxx_hidden_Environments + } + return nil +} + +func (x *Components) SetTransforms(v map[string]*PTransform) { + x.xxx_hidden_Transforms = v +} + +func (x *Components) SetPcollections(v map[string]*PCollection) { + x.xxx_hidden_Pcollections = v +} + +func (x *Components) SetWindowingStrategies(v map[string]*WindowingStrategy) { + x.xxx_hidden_WindowingStrategies = v +} + +func (x *Components) SetCoders(v map[string]*Coder) { + x.xxx_hidden_Coders = v +} + +func (x *Components) SetEnvironments(v map[string]*Environment) { + x.xxx_hidden_Environments = v +} + +type Components_builder struct { + _ [0]func() // Prevents comparability and use of unkeyed literals for the builder. + + // (Required) A map from pipeline-scoped id to PTransform. + // + // Keys of the transforms map may be used by runners to identify pipeline + // steps. Hence it's recommended to use strings that are not too long that + // match regex '[A-Za-z0-9-_]+'. + Transforms map[string]*PTransform + // (Required) A map from pipeline-scoped id to PCollection. + Pcollections map[string]*PCollection + // (Required) A map from pipeline-scoped id to WindowingStrategy. + WindowingStrategies map[string]*WindowingStrategy + // (Required) A map from pipeline-scoped id to Coder. + Coders map[string]*Coder + // (Required) A map from pipeline-scoped id to Environment. + Environments map[string]*Environment +} + +func (b0 Components_builder) Build() *Components { + m0 := &Components{} + b, x := &b0, m0 + _, _ = b, x + x.xxx_hidden_Transforms = b.Transforms + x.xxx_hidden_Pcollections = b.Pcollections + x.xxx_hidden_WindowingStrategies = b.WindowingStrategies + x.xxx_hidden_Coders = b.Coders + x.xxx_hidden_Environments = b.Environments + return m0 +} + +// A Pipeline is a hierarchical graph of PTransforms, linked +// by PCollections. A typical graph may look like: +// +// Impulse -> PCollection -> ParDo -> PCollection -> GroupByKey -> ... +// \> PCollection -> ParDo -> ... +// \> ParDo -> ... +// Impulse -> PCollection -> ParDo -> PCollection -> ... +// +// This is represented by a number of by-reference maps to transforms, +// PCollections, SDK environments, coders, etc., for +// supporting compact reuse and arbitrary graph structure. +type Pipeline struct { + state protoimpl.MessageState `protogen:"opaque.v1"` + xxx_hidden_Components *Components `protobuf:"bytes,1,opt,name=components,proto3" json:"components,omitempty"` + xxx_hidden_RootTransformIds []string `protobuf:"bytes,2,rep,name=root_transform_ids,json=rootTransformIds,proto3" json:"root_transform_ids,omitempty"` + xxx_hidden_DisplayData *[]*DisplayData `protobuf:"bytes,3,rep,name=display_data,json=displayData,proto3" json:"display_data,omitempty"` + xxx_hidden_Requirements []string `protobuf:"bytes,4,rep,name=requirements,proto3" json:"requirements,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *Pipeline) Reset() { + *x = Pipeline{} + mi := &file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_msgTypes[2] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *Pipeline) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*Pipeline) ProtoMessage() {} + +func (x *Pipeline) ProtoReflect() protoreflect.Message { + mi := &file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_msgTypes[2] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +func (x *Pipeline) GetComponents() *Components { + if x != nil { + return x.xxx_hidden_Components + } + return nil +} + +func (x *Pipeline) GetRootTransformIds() []string { + if x != nil { + return x.xxx_hidden_RootTransformIds + } + return nil +} + +func (x *Pipeline) GetDisplayData() []*DisplayData { + if x != nil { + if x.xxx_hidden_DisplayData != nil { + return *x.xxx_hidden_DisplayData + } + } + return nil +} + +func (x *Pipeline) GetRequirements() []string { + if x != nil { + return x.xxx_hidden_Requirements + } + return nil +} + +func (x *Pipeline) SetComponents(v *Components) { + x.xxx_hidden_Components = v +} + +func (x *Pipeline) SetRootTransformIds(v []string) { + x.xxx_hidden_RootTransformIds = v +} + +func (x *Pipeline) SetDisplayData(v []*DisplayData) { + x.xxx_hidden_DisplayData = &v +} + +func (x *Pipeline) SetRequirements(v []string) { + x.xxx_hidden_Requirements = v +} + +func (x *Pipeline) HasComponents() bool { + if x == nil { + return false + } + return x.xxx_hidden_Components != nil +} + +func (x *Pipeline) ClearComponents() { + x.xxx_hidden_Components = nil +} + +type Pipeline_builder struct { + _ [0]func() // Prevents comparability and use of unkeyed literals for the builder. + + // (Required) The coders, UDFs, graph nodes, etc, that make up + // this pipeline. + Components *Components + // (Required) The ids of all PTransforms that are not contained within another + // PTransform. These must be in shallow topological order, so that traversing + // them recursively in this order yields a recursively topological traversal. + RootTransformIds []string + // (Optional) Static display data for the pipeline. If there is none, + // it may be omitted. + DisplayData []*DisplayData + // (Optional) A set of requirements that the runner MUST understand and be + // able to faithfully provide in order to execute this pipeline. These + // may indicate that a runner must inspect new fields on a component or + // provide additional guarantees when processing specific transforms. + // A runner should reject any pipelines with unknown requirements. + Requirements []string +} + +func (b0 Pipeline_builder) Build() *Pipeline { + m0 := &Pipeline{} + b, x := &b0, m0 + _, _ = b, x + x.xxx_hidden_Components = b.Components + x.xxx_hidden_RootTransformIds = b.RootTransformIds + x.xxx_hidden_DisplayData = &b.DisplayData + x.xxx_hidden_Requirements = b.Requirements + return m0 +} + +// Transforms are the operations in your pipeline, and provide a generic +// processing framework. You provide processing logic in the form of a function +// object (colloquially referred to as “user code”), and your user code is +// applied to each element of an input PCollection (or more than one +// PCollection). Depending on the pipeline runner and back-end that you choose, +// many different workers across a cluster may execute instances of your user +// code in parallel. The user code running on each worker generates the output +// elements that are ultimately added to the final output PCollection that the +// transform produces. +// +// The Beam SDKs contain a number of different transforms that you can apply to +// your pipeline’s PCollections. These include general-purpose core transforms, +// such as ParDo or Combine. There are also pre-written composite transforms +// included in the SDKs, which combine one or more of the core transforms in a +// useful processing pattern, such as counting or combining elements in a +// collection. You can also define your own more complex composite transforms to +// fit your pipeline’s exact use case. +type PTransform struct { + state protoimpl.MessageState `protogen:"opaque.v1"` + xxx_hidden_UniqueName string `protobuf:"bytes,5,opt,name=unique_name,json=uniqueName,proto3" json:"unique_name,omitempty"` + xxx_hidden_Spec *FunctionSpec `protobuf:"bytes,1,opt,name=spec,proto3" json:"spec,omitempty"` + xxx_hidden_Subtransforms []string `protobuf:"bytes,2,rep,name=subtransforms,proto3" json:"subtransforms,omitempty"` + xxx_hidden_Inputs map[string]string `protobuf:"bytes,3,rep,name=inputs,proto3" json:"inputs,omitempty" protobuf_key:"bytes,1,opt,name=key" protobuf_val:"bytes,2,opt,name=value"` + xxx_hidden_Outputs map[string]string `protobuf:"bytes,4,rep,name=outputs,proto3" json:"outputs,omitempty" protobuf_key:"bytes,1,opt,name=key" protobuf_val:"bytes,2,opt,name=value"` + xxx_hidden_DisplayData *[]*DisplayData `protobuf:"bytes,6,rep,name=display_data,json=displayData,proto3" json:"display_data,omitempty"` + xxx_hidden_EnvironmentId string `protobuf:"bytes,7,opt,name=environment_id,json=environmentId,proto3" json:"environment_id,omitempty"` + xxx_hidden_Annotations map[string][]byte `protobuf:"bytes,8,rep,name=annotations,proto3" json:"annotations,omitempty" protobuf_key:"bytes,1,opt,name=key" protobuf_val:"bytes,2,opt,name=value"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *PTransform) Reset() { + *x = PTransform{} + mi := &file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_msgTypes[3] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *PTransform) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*PTransform) ProtoMessage() {} + +func (x *PTransform) ProtoReflect() protoreflect.Message { + mi := &file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_msgTypes[3] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +func (x *PTransform) GetUniqueName() string { + if x != nil { + return x.xxx_hidden_UniqueName + } + return "" +} + +func (x *PTransform) GetSpec() *FunctionSpec { + if x != nil { + return x.xxx_hidden_Spec + } + return nil +} + +func (x *PTransform) GetSubtransforms() []string { + if x != nil { + return x.xxx_hidden_Subtransforms + } + return nil +} + +func (x *PTransform) GetInputs() map[string]string { + if x != nil { + return x.xxx_hidden_Inputs + } + return nil +} + +func (x *PTransform) GetOutputs() map[string]string { + if x != nil { + return x.xxx_hidden_Outputs + } + return nil +} + +func (x *PTransform) GetDisplayData() []*DisplayData { + if x != nil { + if x.xxx_hidden_DisplayData != nil { + return *x.xxx_hidden_DisplayData + } + } + return nil +} + +func (x *PTransform) GetEnvironmentId() string { + if x != nil { + return x.xxx_hidden_EnvironmentId + } + return "" +} + +func (x *PTransform) GetAnnotations() map[string][]byte { + if x != nil { + return x.xxx_hidden_Annotations + } + return nil +} + +func (x *PTransform) SetUniqueName(v string) { + x.xxx_hidden_UniqueName = v +} + +func (x *PTransform) SetSpec(v *FunctionSpec) { + x.xxx_hidden_Spec = v +} + +func (x *PTransform) SetSubtransforms(v []string) { + x.xxx_hidden_Subtransforms = v +} + +func (x *PTransform) SetInputs(v map[string]string) { + x.xxx_hidden_Inputs = v +} + +func (x *PTransform) SetOutputs(v map[string]string) { + x.xxx_hidden_Outputs = v +} + +func (x *PTransform) SetDisplayData(v []*DisplayData) { + x.xxx_hidden_DisplayData = &v +} + +func (x *PTransform) SetEnvironmentId(v string) { + x.xxx_hidden_EnvironmentId = v +} + +func (x *PTransform) SetAnnotations(v map[string][]byte) { + x.xxx_hidden_Annotations = v +} + +func (x *PTransform) HasSpec() bool { + if x == nil { + return false + } + return x.xxx_hidden_Spec != nil +} + +func (x *PTransform) ClearSpec() { + x.xxx_hidden_Spec = nil +} + +type PTransform_builder struct { + _ [0]func() // Prevents comparability and use of unkeyed literals for the builder. + + // (Required) A unique name for the application node. + // + // Ideally, this should be stable over multiple evolutions of a pipeline + // for the purposes of logging and associating pipeline state with a node, + // etc. + // + // If it is not stable, then the runner decides what will happen. But, most + // importantly, it must always be here and be unique, even if it is + // autogenerated. + UniqueName string + // (Optional) A URN and payload that, together, fully defined the semantics + // of this transform. + // + // If absent, this must be an "anonymous" composite transform. + // + // For primitive transform in the Runner API, this is required, and the + // payloads are well-defined messages. When the URN indicates ParDo it + // is a ParDoPayload, and so on. For some special composite transforms, + // the payload is also officially defined. See StandardPTransforms for + // details. + Spec *FunctionSpec + // (Optional) A list of the ids of transforms that it contains. + // + // Primitive transforms (see StandardPTransforms.Primitives) are not allowed + // to specify subtransforms. + // + // Note that a composite transform may have zero subtransforms as long as it + // only outputs PCollections that are in its inputs. + Subtransforms []string + // (Required) A map from local names of inputs (unique only with this map, and + // likely embedded in the transform payload and serialized user code) to + // PCollection ids. + // + // The payload for this transform may clarify the relationship of these + // inputs. For example: + // + // - for a Flatten transform they are merged + // - for a ParDo transform, some may be side inputs + // + // All inputs are recorded here so that the topological ordering of + // the graph is consistent whether or not the payload is understood. + Inputs map[string]string + // (Required) A map from local names of outputs (unique only within this map, + // and likely embedded in the transform payload and serialized user code) + // to PCollection ids. + // + // The URN or payload for this transform node may clarify the type and + // relationship of these outputs. For example: + // + // - for a ParDo transform, these are tags on PCollections, which will be + // embedded in the DoFn. + Outputs map[string]string + // (Optional) Static display data for this PTransform application. If + // there is none, it may be omitted. + DisplayData []*DisplayData + // Environment where the current PTransform should be executed in. + // + // Transforms that are required to be implemented by a runner must omit this. + // All other transforms are required to specify this. + EnvironmentId string + // (Optional) A map from URNs designating a type of annotation, to the + // annotation in binary format. For example, an annotation could indicate + // that this PTransform has specific privacy properties. + // + // A runner MAY ignore types of annotations it doesn't understand. Therefore + // annotations MUST NOT be used for metadata that can affect correct + // execution of the transform. + Annotations map[string][]byte +} + +func (b0 PTransform_builder) Build() *PTransform { + m0 := &PTransform{} + b, x := &b0, m0 + _, _ = b, x + x.xxx_hidden_UniqueName = b.UniqueName + x.xxx_hidden_Spec = b.Spec + x.xxx_hidden_Subtransforms = b.Subtransforms + x.xxx_hidden_Inputs = b.Inputs + x.xxx_hidden_Outputs = b.Outputs + x.xxx_hidden_DisplayData = &b.DisplayData + x.xxx_hidden_EnvironmentId = b.EnvironmentId + x.xxx_hidden_Annotations = b.Annotations + return m0 +} + +type StandardPTransforms struct { + state protoimpl.MessageState `protogen:"opaque.v1"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *StandardPTransforms) Reset() { + *x = StandardPTransforms{} + mi := &file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_msgTypes[4] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *StandardPTransforms) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*StandardPTransforms) ProtoMessage() {} + +func (x *StandardPTransforms) ProtoReflect() protoreflect.Message { + mi := &file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_msgTypes[4] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +type StandardPTransforms_builder struct { + _ [0]func() // Prevents comparability and use of unkeyed literals for the builder. + +} + +func (b0 StandardPTransforms_builder) Build() *StandardPTransforms { + m0 := &StandardPTransforms{} + b, x := &b0, m0 + _, _ = b, x + return m0 +} + +type StandardSideInputTypes struct { + state protoimpl.MessageState `protogen:"opaque.v1"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *StandardSideInputTypes) Reset() { + *x = StandardSideInputTypes{} + mi := &file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_msgTypes[5] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *StandardSideInputTypes) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*StandardSideInputTypes) ProtoMessage() {} + +func (x *StandardSideInputTypes) ProtoReflect() protoreflect.Message { + mi := &file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_msgTypes[5] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +type StandardSideInputTypes_builder struct { + _ [0]func() // Prevents comparability and use of unkeyed literals for the builder. + +} + +func (b0 StandardSideInputTypes_builder) Build() *StandardSideInputTypes { + m0 := &StandardSideInputTypes{} + b, x := &b0, m0 + _, _ = b, x + return m0 +} + +type StandardUserStateTypes struct { + state protoimpl.MessageState `protogen:"opaque.v1"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *StandardUserStateTypes) Reset() { + *x = StandardUserStateTypes{} + mi := &file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_msgTypes[6] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *StandardUserStateTypes) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*StandardUserStateTypes) ProtoMessage() {} + +func (x *StandardUserStateTypes) ProtoReflect() protoreflect.Message { + mi := &file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_msgTypes[6] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +type StandardUserStateTypes_builder struct { + _ [0]func() // Prevents comparability and use of unkeyed literals for the builder. + +} + +func (b0 StandardUserStateTypes_builder) Build() *StandardUserStateTypes { + m0 := &StandardUserStateTypes{} + b, x := &b0, m0 + _, _ = b, x + return m0 +} + +// A PCollection! +type PCollection struct { + state protoimpl.MessageState `protogen:"opaque.v1"` + xxx_hidden_UniqueName string `protobuf:"bytes,1,opt,name=unique_name,json=uniqueName,proto3" json:"unique_name,omitempty"` + xxx_hidden_CoderId string `protobuf:"bytes,2,opt,name=coder_id,json=coderId,proto3" json:"coder_id,omitempty"` + xxx_hidden_IsBounded IsBounded_Enum `protobuf:"varint,3,opt,name=is_bounded,json=isBounded,proto3,enum=org.apache.beam.model.pipeline.v1.IsBounded_Enum" json:"is_bounded,omitempty"` + xxx_hidden_WindowingStrategyId string `protobuf:"bytes,4,opt,name=windowing_strategy_id,json=windowingStrategyId,proto3" json:"windowing_strategy_id,omitempty"` + xxx_hidden_DisplayData *[]*DisplayData `protobuf:"bytes,5,rep,name=display_data,json=displayData,proto3" json:"display_data,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *PCollection) Reset() { + *x = PCollection{} + mi := &file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_msgTypes[7] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *PCollection) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*PCollection) ProtoMessage() {} + +func (x *PCollection) ProtoReflect() protoreflect.Message { + mi := &file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_msgTypes[7] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +func (x *PCollection) GetUniqueName() string { + if x != nil { + return x.xxx_hidden_UniqueName + } + return "" +} + +func (x *PCollection) GetCoderId() string { + if x != nil { + return x.xxx_hidden_CoderId + } + return "" +} + +func (x *PCollection) GetIsBounded() IsBounded_Enum { + if x != nil { + return x.xxx_hidden_IsBounded + } + return IsBounded_UNSPECIFIED +} + +func (x *PCollection) GetWindowingStrategyId() string { + if x != nil { + return x.xxx_hidden_WindowingStrategyId + } + return "" +} + +func (x *PCollection) GetDisplayData() []*DisplayData { + if x != nil { + if x.xxx_hidden_DisplayData != nil { + return *x.xxx_hidden_DisplayData + } + } + return nil +} + +func (x *PCollection) SetUniqueName(v string) { + x.xxx_hidden_UniqueName = v +} + +func (x *PCollection) SetCoderId(v string) { + x.xxx_hidden_CoderId = v +} + +func (x *PCollection) SetIsBounded(v IsBounded_Enum) { + x.xxx_hidden_IsBounded = v +} + +func (x *PCollection) SetWindowingStrategyId(v string) { + x.xxx_hidden_WindowingStrategyId = v +} + +func (x *PCollection) SetDisplayData(v []*DisplayData) { + x.xxx_hidden_DisplayData = &v +} + +type PCollection_builder struct { + _ [0]func() // Prevents comparability and use of unkeyed literals for the builder. + + // (Required) A unique name for the PCollection. + // + // Ideally, this should be stable over multiple evolutions of a pipeline + // for the purposes of logging and associating pipeline state with a node, + // etc. + // + // If it is not stable, then the runner decides what will happen. But, most + // importantly, it must always be here, even if it is autogenerated. + UniqueName string + // (Required) The id of the Coder for this PCollection. + CoderId string + // (Required) Whether this PCollection is bounded or unbounded + IsBounded IsBounded_Enum + // (Required) The id of the windowing strategy for this PCollection. + WindowingStrategyId string + // (Optional) Static display data for the PCollection. If there is none, + // it may be omitted. + DisplayData []*DisplayData +} + +func (b0 PCollection_builder) Build() *PCollection { + m0 := &PCollection{} + b, x := &b0, m0 + _, _ = b, x + x.xxx_hidden_UniqueName = b.UniqueName + x.xxx_hidden_CoderId = b.CoderId + x.xxx_hidden_IsBounded = b.IsBounded + x.xxx_hidden_WindowingStrategyId = b.WindowingStrategyId + x.xxx_hidden_DisplayData = &b.DisplayData + return m0 +} + +// The payload for the primitive ParDo transform. +type ParDoPayload struct { + state protoimpl.MessageState `protogen:"opaque.v1"` + xxx_hidden_DoFn *FunctionSpec `protobuf:"bytes,1,opt,name=do_fn,json=doFn,proto3" json:"do_fn,omitempty"` + xxx_hidden_SideInputs map[string]*SideInput `protobuf:"bytes,3,rep,name=side_inputs,json=sideInputs,proto3" json:"side_inputs,omitempty" protobuf_key:"bytes,1,opt,name=key" protobuf_val:"bytes,2,opt,name=value"` + xxx_hidden_StateSpecs map[string]*StateSpec `protobuf:"bytes,4,rep,name=state_specs,json=stateSpecs,proto3" json:"state_specs,omitempty" protobuf_key:"bytes,1,opt,name=key" protobuf_val:"bytes,2,opt,name=value"` + xxx_hidden_TimerFamilySpecs map[string]*TimerFamilySpec `protobuf:"bytes,9,rep,name=timer_family_specs,json=timerFamilySpecs,proto3" json:"timer_family_specs,omitempty" protobuf_key:"bytes,1,opt,name=key" protobuf_val:"bytes,2,opt,name=value"` + xxx_hidden_RestrictionCoderId string `protobuf:"bytes,7,opt,name=restriction_coder_id,json=restrictionCoderId,proto3" json:"restriction_coder_id,omitempty"` + xxx_hidden_RequestsFinalization bool `protobuf:"varint,8,opt,name=requests_finalization,json=requestsFinalization,proto3" json:"requests_finalization,omitempty"` + xxx_hidden_RequiresTimeSortedInput bool `protobuf:"varint,10,opt,name=requires_time_sorted_input,json=requiresTimeSortedInput,proto3" json:"requires_time_sorted_input,omitempty"` + xxx_hidden_RequiresStableInput bool `protobuf:"varint,11,opt,name=requires_stable_input,json=requiresStableInput,proto3" json:"requires_stable_input,omitempty"` + xxx_hidden_OnWindowExpirationTimerFamilySpec string `protobuf:"bytes,12,opt,name=on_window_expiration_timer_family_spec,json=onWindowExpirationTimerFamilySpec,proto3" json:"on_window_expiration_timer_family_spec,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *ParDoPayload) Reset() { + *x = ParDoPayload{} + mi := &file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_msgTypes[8] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *ParDoPayload) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*ParDoPayload) ProtoMessage() {} + +func (x *ParDoPayload) ProtoReflect() protoreflect.Message { + mi := &file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_msgTypes[8] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +func (x *ParDoPayload) GetDoFn() *FunctionSpec { + if x != nil { + return x.xxx_hidden_DoFn + } + return nil +} + +func (x *ParDoPayload) GetSideInputs() map[string]*SideInput { + if x != nil { + return x.xxx_hidden_SideInputs + } + return nil +} + +func (x *ParDoPayload) GetStateSpecs() map[string]*StateSpec { + if x != nil { + return x.xxx_hidden_StateSpecs + } + return nil +} + +func (x *ParDoPayload) GetTimerFamilySpecs() map[string]*TimerFamilySpec { + if x != nil { + return x.xxx_hidden_TimerFamilySpecs + } + return nil +} + +func (x *ParDoPayload) GetRestrictionCoderId() string { + if x != nil { + return x.xxx_hidden_RestrictionCoderId + } + return "" +} + +func (x *ParDoPayload) GetRequestsFinalization() bool { + if x != nil { + return x.xxx_hidden_RequestsFinalization + } + return false +} + +func (x *ParDoPayload) GetRequiresTimeSortedInput() bool { + if x != nil { + return x.xxx_hidden_RequiresTimeSortedInput + } + return false +} + +func (x *ParDoPayload) GetRequiresStableInput() bool { + if x != nil { + return x.xxx_hidden_RequiresStableInput + } + return false +} + +func (x *ParDoPayload) GetOnWindowExpirationTimerFamilySpec() string { + if x != nil { + return x.xxx_hidden_OnWindowExpirationTimerFamilySpec + } + return "" +} + +func (x *ParDoPayload) SetDoFn(v *FunctionSpec) { + x.xxx_hidden_DoFn = v +} + +func (x *ParDoPayload) SetSideInputs(v map[string]*SideInput) { + x.xxx_hidden_SideInputs = v +} + +func (x *ParDoPayload) SetStateSpecs(v map[string]*StateSpec) { + x.xxx_hidden_StateSpecs = v +} + +func (x *ParDoPayload) SetTimerFamilySpecs(v map[string]*TimerFamilySpec) { + x.xxx_hidden_TimerFamilySpecs = v +} + +func (x *ParDoPayload) SetRestrictionCoderId(v string) { + x.xxx_hidden_RestrictionCoderId = v +} + +func (x *ParDoPayload) SetRequestsFinalization(v bool) { + x.xxx_hidden_RequestsFinalization = v +} + +func (x *ParDoPayload) SetRequiresTimeSortedInput(v bool) { + x.xxx_hidden_RequiresTimeSortedInput = v +} + +func (x *ParDoPayload) SetRequiresStableInput(v bool) { + x.xxx_hidden_RequiresStableInput = v +} + +func (x *ParDoPayload) SetOnWindowExpirationTimerFamilySpec(v string) { + x.xxx_hidden_OnWindowExpirationTimerFamilySpec = v +} + +func (x *ParDoPayload) HasDoFn() bool { + if x == nil { + return false + } + return x.xxx_hidden_DoFn != nil +} + +func (x *ParDoPayload) ClearDoFn() { + x.xxx_hidden_DoFn = nil +} + +type ParDoPayload_builder struct { + _ [0]func() // Prevents comparability and use of unkeyed literals for the builder. + + // (Required) The FunctionSpec of the DoFn. + DoFn *FunctionSpec + // (Optional) A mapping of local input names to side inputs, describing + // the expected access pattern. + SideInputs map[string]*SideInput + // (Optional) A mapping of local state names to state specifications. + // If this is set, the stateful processing requirement should also + // be placed in the pipeline requirements. + StateSpecs map[string]*StateSpec + // (Optional) A mapping of local timer family names to timer family + // specifications. If this is set, the stateful processing requirement should + // also be placed in the pipeline requirements. + TimerFamilySpecs map[string]*TimerFamilySpec + // (Optional) Only set when this ParDo contains a splittable DoFn. + // If this is set, the corresponding standard requirement should also + // be placed in the pipeline requirements. + RestrictionCoderId string + // (Optional) Only set when this ParDo can request bundle finalization. + // If this is set, the corresponding standard requirement should also + // be placed in the pipeline requirements. + RequestsFinalization bool + // Whether this stage requires time sorted input. + // If this is set, the corresponding standard requirement should also + // be placed in the pipeline requirements. + RequiresTimeSortedInput bool + // Whether this stage requires stable input. + // If this is set, the corresponding standard requirement should also + // be placed in the pipeline requirements. + RequiresStableInput bool + // If populated, the name of the timer family spec which should be notified + // on each window expiry. + // If this is set, the corresponding standard requirement should also + // be placed in the pipeline requirements. + OnWindowExpirationTimerFamilySpec string +} + +func (b0 ParDoPayload_builder) Build() *ParDoPayload { + m0 := &ParDoPayload{} + b, x := &b0, m0 + _, _ = b, x + x.xxx_hidden_DoFn = b.DoFn + x.xxx_hidden_SideInputs = b.SideInputs + x.xxx_hidden_StateSpecs = b.StateSpecs + x.xxx_hidden_TimerFamilySpecs = b.TimerFamilySpecs + x.xxx_hidden_RestrictionCoderId = b.RestrictionCoderId + x.xxx_hidden_RequestsFinalization = b.RequestsFinalization + x.xxx_hidden_RequiresTimeSortedInput = b.RequiresTimeSortedInput + x.xxx_hidden_RequiresStableInput = b.RequiresStableInput + x.xxx_hidden_OnWindowExpirationTimerFamilySpec = b.OnWindowExpirationTimerFamilySpec + return m0 +} + +type StateSpec struct { + state protoimpl.MessageState `protogen:"opaque.v1"` + xxx_hidden_Spec isStateSpec_Spec `protobuf_oneof:"spec"` + xxx_hidden_Protocol *FunctionSpec `protobuf:"bytes,7,opt,name=protocol,proto3" json:"protocol,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *StateSpec) Reset() { + *x = StateSpec{} + mi := &file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_msgTypes[9] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *StateSpec) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*StateSpec) ProtoMessage() {} + +func (x *StateSpec) ProtoReflect() protoreflect.Message { + mi := &file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_msgTypes[9] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +func (x *StateSpec) GetReadModifyWriteSpec() *ReadModifyWriteStateSpec { + if x != nil { + if x, ok := x.xxx_hidden_Spec.(*stateSpec_ReadModifyWriteSpec); ok { + return x.ReadModifyWriteSpec + } + } + return nil +} + +func (x *StateSpec) GetBagSpec() *BagStateSpec { + if x != nil { + if x, ok := x.xxx_hidden_Spec.(*stateSpec_BagSpec); ok { + return x.BagSpec + } + } + return nil +} + +func (x *StateSpec) GetCombiningSpec() *CombiningStateSpec { + if x != nil { + if x, ok := x.xxx_hidden_Spec.(*stateSpec_CombiningSpec); ok { + return x.CombiningSpec + } + } + return nil +} + +func (x *StateSpec) GetMapSpec() *MapStateSpec { + if x != nil { + if x, ok := x.xxx_hidden_Spec.(*stateSpec_MapSpec); ok { + return x.MapSpec + } + } + return nil +} + +func (x *StateSpec) GetSetSpec() *SetStateSpec { + if x != nil { + if x, ok := x.xxx_hidden_Spec.(*stateSpec_SetSpec); ok { + return x.SetSpec + } + } + return nil +} + +func (x *StateSpec) GetOrderedListSpec() *OrderedListStateSpec { + if x != nil { + if x, ok := x.xxx_hidden_Spec.(*stateSpec_OrderedListSpec); ok { + return x.OrderedListSpec + } + } + return nil +} + +func (x *StateSpec) GetMultimapSpec() *MultimapStateSpec { + if x != nil { + if x, ok := x.xxx_hidden_Spec.(*stateSpec_MultimapSpec); ok { + return x.MultimapSpec + } + } + return nil +} + +func (x *StateSpec) GetProtocol() *FunctionSpec { + if x != nil { + return x.xxx_hidden_Protocol + } + return nil +} + +func (x *StateSpec) SetReadModifyWriteSpec(v *ReadModifyWriteStateSpec) { + if v == nil { + x.xxx_hidden_Spec = nil + return + } + x.xxx_hidden_Spec = &stateSpec_ReadModifyWriteSpec{v} +} + +func (x *StateSpec) SetBagSpec(v *BagStateSpec) { + if v == nil { + x.xxx_hidden_Spec = nil + return + } + x.xxx_hidden_Spec = &stateSpec_BagSpec{v} +} + +func (x *StateSpec) SetCombiningSpec(v *CombiningStateSpec) { + if v == nil { + x.xxx_hidden_Spec = nil + return + } + x.xxx_hidden_Spec = &stateSpec_CombiningSpec{v} +} + +func (x *StateSpec) SetMapSpec(v *MapStateSpec) { + if v == nil { + x.xxx_hidden_Spec = nil + return + } + x.xxx_hidden_Spec = &stateSpec_MapSpec{v} +} + +func (x *StateSpec) SetSetSpec(v *SetStateSpec) { + if v == nil { + x.xxx_hidden_Spec = nil + return + } + x.xxx_hidden_Spec = &stateSpec_SetSpec{v} +} + +func (x *StateSpec) SetOrderedListSpec(v *OrderedListStateSpec) { + if v == nil { + x.xxx_hidden_Spec = nil + return + } + x.xxx_hidden_Spec = &stateSpec_OrderedListSpec{v} +} + +func (x *StateSpec) SetMultimapSpec(v *MultimapStateSpec) { + if v == nil { + x.xxx_hidden_Spec = nil + return + } + x.xxx_hidden_Spec = &stateSpec_MultimapSpec{v} +} + +func (x *StateSpec) SetProtocol(v *FunctionSpec) { + x.xxx_hidden_Protocol = v +} + +func (x *StateSpec) HasSpec() bool { + if x == nil { + return false + } + return x.xxx_hidden_Spec != nil +} + +func (x *StateSpec) HasReadModifyWriteSpec() bool { + if x == nil { + return false + } + _, ok := x.xxx_hidden_Spec.(*stateSpec_ReadModifyWriteSpec) + return ok +} + +func (x *StateSpec) HasBagSpec() bool { + if x == nil { + return false + } + _, ok := x.xxx_hidden_Spec.(*stateSpec_BagSpec) + return ok +} + +func (x *StateSpec) HasCombiningSpec() bool { + if x == nil { + return false + } + _, ok := x.xxx_hidden_Spec.(*stateSpec_CombiningSpec) + return ok +} + +func (x *StateSpec) HasMapSpec() bool { + if x == nil { + return false + } + _, ok := x.xxx_hidden_Spec.(*stateSpec_MapSpec) + return ok +} + +func (x *StateSpec) HasSetSpec() bool { + if x == nil { + return false + } + _, ok := x.xxx_hidden_Spec.(*stateSpec_SetSpec) + return ok +} + +func (x *StateSpec) HasOrderedListSpec() bool { + if x == nil { + return false + } + _, ok := x.xxx_hidden_Spec.(*stateSpec_OrderedListSpec) + return ok +} + +func (x *StateSpec) HasMultimapSpec() bool { + if x == nil { + return false + } + _, ok := x.xxx_hidden_Spec.(*stateSpec_MultimapSpec) + return ok +} + +func (x *StateSpec) HasProtocol() bool { + if x == nil { + return false + } + return x.xxx_hidden_Protocol != nil +} + +func (x *StateSpec) ClearSpec() { + x.xxx_hidden_Spec = nil +} + +func (x *StateSpec) ClearReadModifyWriteSpec() { + if _, ok := x.xxx_hidden_Spec.(*stateSpec_ReadModifyWriteSpec); ok { + x.xxx_hidden_Spec = nil + } +} + +func (x *StateSpec) ClearBagSpec() { + if _, ok := x.xxx_hidden_Spec.(*stateSpec_BagSpec); ok { + x.xxx_hidden_Spec = nil + } +} + +func (x *StateSpec) ClearCombiningSpec() { + if _, ok := x.xxx_hidden_Spec.(*stateSpec_CombiningSpec); ok { + x.xxx_hidden_Spec = nil + } +} + +func (x *StateSpec) ClearMapSpec() { + if _, ok := x.xxx_hidden_Spec.(*stateSpec_MapSpec); ok { + x.xxx_hidden_Spec = nil + } +} + +func (x *StateSpec) ClearSetSpec() { + if _, ok := x.xxx_hidden_Spec.(*stateSpec_SetSpec); ok { + x.xxx_hidden_Spec = nil + } +} + +func (x *StateSpec) ClearOrderedListSpec() { + if _, ok := x.xxx_hidden_Spec.(*stateSpec_OrderedListSpec); ok { + x.xxx_hidden_Spec = nil + } +} + +func (x *StateSpec) ClearMultimapSpec() { + if _, ok := x.xxx_hidden_Spec.(*stateSpec_MultimapSpec); ok { + x.xxx_hidden_Spec = nil + } +} + +func (x *StateSpec) ClearProtocol() { + x.xxx_hidden_Protocol = nil +} + +const StateSpec_Spec_not_set_case case_StateSpec_Spec = 0 +const StateSpec_ReadModifyWriteSpec_case case_StateSpec_Spec = 1 +const StateSpec_BagSpec_case case_StateSpec_Spec = 2 +const StateSpec_CombiningSpec_case case_StateSpec_Spec = 3 +const StateSpec_MapSpec_case case_StateSpec_Spec = 4 +const StateSpec_SetSpec_case case_StateSpec_Spec = 5 +const StateSpec_OrderedListSpec_case case_StateSpec_Spec = 6 +const StateSpec_MultimapSpec_case case_StateSpec_Spec = 8 + +func (x *StateSpec) WhichSpec() case_StateSpec_Spec { + if x == nil { + return StateSpec_Spec_not_set_case + } + switch x.xxx_hidden_Spec.(type) { + case *stateSpec_ReadModifyWriteSpec: + return StateSpec_ReadModifyWriteSpec_case + case *stateSpec_BagSpec: + return StateSpec_BagSpec_case + case *stateSpec_CombiningSpec: + return StateSpec_CombiningSpec_case + case *stateSpec_MapSpec: + return StateSpec_MapSpec_case + case *stateSpec_SetSpec: + return StateSpec_SetSpec_case + case *stateSpec_OrderedListSpec: + return StateSpec_OrderedListSpec_case + case *stateSpec_MultimapSpec: + return StateSpec_MultimapSpec_case + default: + return StateSpec_Spec_not_set_case + } +} + +type StateSpec_builder struct { + _ [0]func() // Prevents comparability and use of unkeyed literals for the builder. + + // TODO(BEAM-13930): Deprecate and remove these state specs + + // Fields of oneof xxx_hidden_Spec: + ReadModifyWriteSpec *ReadModifyWriteStateSpec + BagSpec *BagStateSpec + CombiningSpec *CombiningStateSpec + MapSpec *MapStateSpec + SetSpec *SetStateSpec + OrderedListSpec *OrderedListStateSpec + MultimapSpec *MultimapStateSpec + // -- end of xxx_hidden_Spec + // (Required) URN of the protocol required by this state specification to present + // the desired SDK-specific interface to a UDF. + // + // This protocol defines the SDK harness <-> Runner Harness RPC + // interface for accessing and mutating user state. + // + // See StandardUserStateTypes for an enumeration of all user state types + // defined. + Protocol *FunctionSpec +} + +func (b0 StateSpec_builder) Build() *StateSpec { + m0 := &StateSpec{} + b, x := &b0, m0 + _, _ = b, x + if b.ReadModifyWriteSpec != nil { + x.xxx_hidden_Spec = &stateSpec_ReadModifyWriteSpec{b.ReadModifyWriteSpec} + } + if b.BagSpec != nil { + x.xxx_hidden_Spec = &stateSpec_BagSpec{b.BagSpec} + } + if b.CombiningSpec != nil { + x.xxx_hidden_Spec = &stateSpec_CombiningSpec{b.CombiningSpec} + } + if b.MapSpec != nil { + x.xxx_hidden_Spec = &stateSpec_MapSpec{b.MapSpec} + } + if b.SetSpec != nil { + x.xxx_hidden_Spec = &stateSpec_SetSpec{b.SetSpec} + } + if b.OrderedListSpec != nil { + x.xxx_hidden_Spec = &stateSpec_OrderedListSpec{b.OrderedListSpec} + } + if b.MultimapSpec != nil { + x.xxx_hidden_Spec = &stateSpec_MultimapSpec{b.MultimapSpec} + } + x.xxx_hidden_Protocol = b.Protocol + return m0 +} + +type case_StateSpec_Spec protoreflect.FieldNumber + +func (x case_StateSpec_Spec) String() string { + md := file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_msgTypes[9].Descriptor() + if x == 0 { + return "not set" + } + return protoimpl.X.MessageFieldStringOf(md, protoreflect.FieldNumber(x)) +} + +type isStateSpec_Spec interface { + isStateSpec_Spec() +} + +type stateSpec_ReadModifyWriteSpec struct { + ReadModifyWriteSpec *ReadModifyWriteStateSpec `protobuf:"bytes,1,opt,name=read_modify_write_spec,json=readModifyWriteSpec,proto3,oneof"` +} + +type stateSpec_BagSpec struct { + BagSpec *BagStateSpec `protobuf:"bytes,2,opt,name=bag_spec,json=bagSpec,proto3,oneof"` +} + +type stateSpec_CombiningSpec struct { + CombiningSpec *CombiningStateSpec `protobuf:"bytes,3,opt,name=combining_spec,json=combiningSpec,proto3,oneof"` +} + +type stateSpec_MapSpec struct { + MapSpec *MapStateSpec `protobuf:"bytes,4,opt,name=map_spec,json=mapSpec,proto3,oneof"` +} + +type stateSpec_SetSpec struct { + SetSpec *SetStateSpec `protobuf:"bytes,5,opt,name=set_spec,json=setSpec,proto3,oneof"` +} + +type stateSpec_OrderedListSpec struct { + OrderedListSpec *OrderedListStateSpec `protobuf:"bytes,6,opt,name=ordered_list_spec,json=orderedListSpec,proto3,oneof"` +} + +type stateSpec_MultimapSpec struct { + MultimapSpec *MultimapStateSpec `protobuf:"bytes,8,opt,name=multimap_spec,json=multimapSpec,proto3,oneof"` +} + +func (*stateSpec_ReadModifyWriteSpec) isStateSpec_Spec() {} + +func (*stateSpec_BagSpec) isStateSpec_Spec() {} + +func (*stateSpec_CombiningSpec) isStateSpec_Spec() {} + +func (*stateSpec_MapSpec) isStateSpec_Spec() {} + +func (*stateSpec_SetSpec) isStateSpec_Spec() {} + +func (*stateSpec_OrderedListSpec) isStateSpec_Spec() {} + +func (*stateSpec_MultimapSpec) isStateSpec_Spec() {} + +type ReadModifyWriteStateSpec struct { + state protoimpl.MessageState `protogen:"opaque.v1"` + xxx_hidden_CoderId string `protobuf:"bytes,1,opt,name=coder_id,json=coderId,proto3" json:"coder_id,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *ReadModifyWriteStateSpec) Reset() { + *x = ReadModifyWriteStateSpec{} + mi := &file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_msgTypes[10] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *ReadModifyWriteStateSpec) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*ReadModifyWriteStateSpec) ProtoMessage() {} + +func (x *ReadModifyWriteStateSpec) ProtoReflect() protoreflect.Message { + mi := &file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_msgTypes[10] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +func (x *ReadModifyWriteStateSpec) GetCoderId() string { + if x != nil { + return x.xxx_hidden_CoderId + } + return "" +} + +func (x *ReadModifyWriteStateSpec) SetCoderId(v string) { + x.xxx_hidden_CoderId = v +} + +type ReadModifyWriteStateSpec_builder struct { + _ [0]func() // Prevents comparability and use of unkeyed literals for the builder. + + CoderId string +} + +func (b0 ReadModifyWriteStateSpec_builder) Build() *ReadModifyWriteStateSpec { + m0 := &ReadModifyWriteStateSpec{} + b, x := &b0, m0 + _, _ = b, x + x.xxx_hidden_CoderId = b.CoderId + return m0 +} + +type BagStateSpec struct { + state protoimpl.MessageState `protogen:"opaque.v1"` + xxx_hidden_ElementCoderId string `protobuf:"bytes,1,opt,name=element_coder_id,json=elementCoderId,proto3" json:"element_coder_id,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *BagStateSpec) Reset() { + *x = BagStateSpec{} + mi := &file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_msgTypes[11] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *BagStateSpec) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*BagStateSpec) ProtoMessage() {} + +func (x *BagStateSpec) ProtoReflect() protoreflect.Message { + mi := &file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_msgTypes[11] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +func (x *BagStateSpec) GetElementCoderId() string { + if x != nil { + return x.xxx_hidden_ElementCoderId + } + return "" +} + +func (x *BagStateSpec) SetElementCoderId(v string) { + x.xxx_hidden_ElementCoderId = v +} + +type BagStateSpec_builder struct { + _ [0]func() // Prevents comparability and use of unkeyed literals for the builder. + + ElementCoderId string +} + +func (b0 BagStateSpec_builder) Build() *BagStateSpec { + m0 := &BagStateSpec{} + b, x := &b0, m0 + _, _ = b, x + x.xxx_hidden_ElementCoderId = b.ElementCoderId + return m0 +} + +type OrderedListStateSpec struct { + state protoimpl.MessageState `protogen:"opaque.v1"` + xxx_hidden_ElementCoderId string `protobuf:"bytes,1,opt,name=element_coder_id,json=elementCoderId,proto3" json:"element_coder_id,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *OrderedListStateSpec) Reset() { + *x = OrderedListStateSpec{} + mi := &file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_msgTypes[12] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *OrderedListStateSpec) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*OrderedListStateSpec) ProtoMessage() {} + +func (x *OrderedListStateSpec) ProtoReflect() protoreflect.Message { + mi := &file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_msgTypes[12] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +func (x *OrderedListStateSpec) GetElementCoderId() string { + if x != nil { + return x.xxx_hidden_ElementCoderId + } + return "" +} + +func (x *OrderedListStateSpec) SetElementCoderId(v string) { + x.xxx_hidden_ElementCoderId = v +} + +type OrderedListStateSpec_builder struct { + _ [0]func() // Prevents comparability and use of unkeyed literals for the builder. + + ElementCoderId string +} + +func (b0 OrderedListStateSpec_builder) Build() *OrderedListStateSpec { + m0 := &OrderedListStateSpec{} + b, x := &b0, m0 + _, _ = b, x + x.xxx_hidden_ElementCoderId = b.ElementCoderId + return m0 +} + +type CombiningStateSpec struct { + state protoimpl.MessageState `protogen:"opaque.v1"` + xxx_hidden_AccumulatorCoderId string `protobuf:"bytes,1,opt,name=accumulator_coder_id,json=accumulatorCoderId,proto3" json:"accumulator_coder_id,omitempty"` + xxx_hidden_CombineFn *FunctionSpec `protobuf:"bytes,2,opt,name=combine_fn,json=combineFn,proto3" json:"combine_fn,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *CombiningStateSpec) Reset() { + *x = CombiningStateSpec{} + mi := &file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_msgTypes[13] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *CombiningStateSpec) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*CombiningStateSpec) ProtoMessage() {} + +func (x *CombiningStateSpec) ProtoReflect() protoreflect.Message { + mi := &file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_msgTypes[13] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +func (x *CombiningStateSpec) GetAccumulatorCoderId() string { + if x != nil { + return x.xxx_hidden_AccumulatorCoderId + } + return "" +} + +func (x *CombiningStateSpec) GetCombineFn() *FunctionSpec { + if x != nil { + return x.xxx_hidden_CombineFn + } + return nil +} + +func (x *CombiningStateSpec) SetAccumulatorCoderId(v string) { + x.xxx_hidden_AccumulatorCoderId = v +} + +func (x *CombiningStateSpec) SetCombineFn(v *FunctionSpec) { + x.xxx_hidden_CombineFn = v +} + +func (x *CombiningStateSpec) HasCombineFn() bool { + if x == nil { + return false + } + return x.xxx_hidden_CombineFn != nil +} + +func (x *CombiningStateSpec) ClearCombineFn() { + x.xxx_hidden_CombineFn = nil +} + +type CombiningStateSpec_builder struct { + _ [0]func() // Prevents comparability and use of unkeyed literals for the builder. + + AccumulatorCoderId string + CombineFn *FunctionSpec +} + +func (b0 CombiningStateSpec_builder) Build() *CombiningStateSpec { + m0 := &CombiningStateSpec{} + b, x := &b0, m0 + _, _ = b, x + x.xxx_hidden_AccumulatorCoderId = b.AccumulatorCoderId + x.xxx_hidden_CombineFn = b.CombineFn + return m0 +} + +type MapStateSpec struct { + state protoimpl.MessageState `protogen:"opaque.v1"` + xxx_hidden_KeyCoderId string `protobuf:"bytes,1,opt,name=key_coder_id,json=keyCoderId,proto3" json:"key_coder_id,omitempty"` + xxx_hidden_ValueCoderId string `protobuf:"bytes,2,opt,name=value_coder_id,json=valueCoderId,proto3" json:"value_coder_id,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *MapStateSpec) Reset() { + *x = MapStateSpec{} + mi := &file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_msgTypes[14] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *MapStateSpec) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*MapStateSpec) ProtoMessage() {} + +func (x *MapStateSpec) ProtoReflect() protoreflect.Message { + mi := &file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_msgTypes[14] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +func (x *MapStateSpec) GetKeyCoderId() string { + if x != nil { + return x.xxx_hidden_KeyCoderId + } + return "" +} + +func (x *MapStateSpec) GetValueCoderId() string { + if x != nil { + return x.xxx_hidden_ValueCoderId + } + return "" +} + +func (x *MapStateSpec) SetKeyCoderId(v string) { + x.xxx_hidden_KeyCoderId = v +} + +func (x *MapStateSpec) SetValueCoderId(v string) { + x.xxx_hidden_ValueCoderId = v +} + +type MapStateSpec_builder struct { + _ [0]func() // Prevents comparability and use of unkeyed literals for the builder. + + KeyCoderId string + ValueCoderId string +} + +func (b0 MapStateSpec_builder) Build() *MapStateSpec { + m0 := &MapStateSpec{} + b, x := &b0, m0 + _, _ = b, x + x.xxx_hidden_KeyCoderId = b.KeyCoderId + x.xxx_hidden_ValueCoderId = b.ValueCoderId + return m0 +} + +type MultimapStateSpec struct { + state protoimpl.MessageState `protogen:"opaque.v1"` + xxx_hidden_KeyCoderId string `protobuf:"bytes,1,opt,name=key_coder_id,json=keyCoderId,proto3" json:"key_coder_id,omitempty"` + xxx_hidden_ValueCoderId string `protobuf:"bytes,2,opt,name=value_coder_id,json=valueCoderId,proto3" json:"value_coder_id,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *MultimapStateSpec) Reset() { + *x = MultimapStateSpec{} + mi := &file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_msgTypes[15] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *MultimapStateSpec) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*MultimapStateSpec) ProtoMessage() {} + +func (x *MultimapStateSpec) ProtoReflect() protoreflect.Message { + mi := &file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_msgTypes[15] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +func (x *MultimapStateSpec) GetKeyCoderId() string { + if x != nil { + return x.xxx_hidden_KeyCoderId + } + return "" +} + +func (x *MultimapStateSpec) GetValueCoderId() string { + if x != nil { + return x.xxx_hidden_ValueCoderId + } + return "" +} + +func (x *MultimapStateSpec) SetKeyCoderId(v string) { + x.xxx_hidden_KeyCoderId = v +} + +func (x *MultimapStateSpec) SetValueCoderId(v string) { + x.xxx_hidden_ValueCoderId = v +} + +type MultimapStateSpec_builder struct { + _ [0]func() // Prevents comparability and use of unkeyed literals for the builder. + + KeyCoderId string + ValueCoderId string +} + +func (b0 MultimapStateSpec_builder) Build() *MultimapStateSpec { + m0 := &MultimapStateSpec{} + b, x := &b0, m0 + _, _ = b, x + x.xxx_hidden_KeyCoderId = b.KeyCoderId + x.xxx_hidden_ValueCoderId = b.ValueCoderId + return m0 +} + +type SetStateSpec struct { + state protoimpl.MessageState `protogen:"opaque.v1"` + xxx_hidden_ElementCoderId string `protobuf:"bytes,1,opt,name=element_coder_id,json=elementCoderId,proto3" json:"element_coder_id,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *SetStateSpec) Reset() { + *x = SetStateSpec{} + mi := &file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_msgTypes[16] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *SetStateSpec) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*SetStateSpec) ProtoMessage() {} + +func (x *SetStateSpec) ProtoReflect() protoreflect.Message { + mi := &file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_msgTypes[16] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +func (x *SetStateSpec) GetElementCoderId() string { + if x != nil { + return x.xxx_hidden_ElementCoderId + } + return "" +} + +func (x *SetStateSpec) SetElementCoderId(v string) { + x.xxx_hidden_ElementCoderId = v +} + +type SetStateSpec_builder struct { + _ [0]func() // Prevents comparability and use of unkeyed literals for the builder. + + ElementCoderId string +} + +func (b0 SetStateSpec_builder) Build() *SetStateSpec { + m0 := &SetStateSpec{} + b, x := &b0, m0 + _, _ = b, x + x.xxx_hidden_ElementCoderId = b.ElementCoderId + return m0 +} + +type TimerFamilySpec struct { + state protoimpl.MessageState `protogen:"opaque.v1"` + xxx_hidden_TimeDomain TimeDomain_Enum `protobuf:"varint,1,opt,name=time_domain,json=timeDomain,proto3,enum=org.apache.beam.model.pipeline.v1.TimeDomain_Enum" json:"time_domain,omitempty"` + xxx_hidden_TimerFamilyCoderId string `protobuf:"bytes,2,opt,name=timer_family_coder_id,json=timerFamilyCoderId,proto3" json:"timer_family_coder_id,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *TimerFamilySpec) Reset() { + *x = TimerFamilySpec{} + mi := &file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_msgTypes[17] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *TimerFamilySpec) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*TimerFamilySpec) ProtoMessage() {} + +func (x *TimerFamilySpec) ProtoReflect() protoreflect.Message { + mi := &file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_msgTypes[17] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +func (x *TimerFamilySpec) GetTimeDomain() TimeDomain_Enum { + if x != nil { + return x.xxx_hidden_TimeDomain + } + return TimeDomain_UNSPECIFIED +} + +func (x *TimerFamilySpec) GetTimerFamilyCoderId() string { + if x != nil { + return x.xxx_hidden_TimerFamilyCoderId + } + return "" +} + +func (x *TimerFamilySpec) SetTimeDomain(v TimeDomain_Enum) { + x.xxx_hidden_TimeDomain = v +} + +func (x *TimerFamilySpec) SetTimerFamilyCoderId(v string) { + x.xxx_hidden_TimerFamilyCoderId = v +} + +type TimerFamilySpec_builder struct { + _ [0]func() // Prevents comparability and use of unkeyed literals for the builder. + + TimeDomain TimeDomain_Enum + TimerFamilyCoderId string +} + +func (b0 TimerFamilySpec_builder) Build() *TimerFamilySpec { + m0 := &TimerFamilySpec{} + b, x := &b0, m0 + _, _ = b, x + x.xxx_hidden_TimeDomain = b.TimeDomain + x.xxx_hidden_TimerFamilyCoderId = b.TimerFamilyCoderId + return m0 +} + +type IsBounded struct { + state protoimpl.MessageState `protogen:"opaque.v1"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *IsBounded) Reset() { + *x = IsBounded{} + mi := &file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_msgTypes[18] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *IsBounded) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*IsBounded) ProtoMessage() {} + +func (x *IsBounded) ProtoReflect() protoreflect.Message { + mi := &file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_msgTypes[18] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +type IsBounded_builder struct { + _ [0]func() // Prevents comparability and use of unkeyed literals for the builder. + +} + +func (b0 IsBounded_builder) Build() *IsBounded { + m0 := &IsBounded{} + b, x := &b0, m0 + _, _ = b, x + return m0 +} + +// The payload for the primitive Read transform. +type ReadPayload struct { + state protoimpl.MessageState `protogen:"opaque.v1"` + xxx_hidden_Source *FunctionSpec `protobuf:"bytes,1,opt,name=source,proto3" json:"source,omitempty"` + xxx_hidden_IsBounded IsBounded_Enum `protobuf:"varint,2,opt,name=is_bounded,json=isBounded,proto3,enum=org.apache.beam.model.pipeline.v1.IsBounded_Enum" json:"is_bounded,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *ReadPayload) Reset() { + *x = ReadPayload{} + mi := &file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_msgTypes[19] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *ReadPayload) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*ReadPayload) ProtoMessage() {} + +func (x *ReadPayload) ProtoReflect() protoreflect.Message { + mi := &file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_msgTypes[19] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +func (x *ReadPayload) GetSource() *FunctionSpec { + if x != nil { + return x.xxx_hidden_Source + } + return nil +} + +func (x *ReadPayload) GetIsBounded() IsBounded_Enum { + if x != nil { + return x.xxx_hidden_IsBounded + } + return IsBounded_UNSPECIFIED +} + +func (x *ReadPayload) SetSource(v *FunctionSpec) { + x.xxx_hidden_Source = v +} + +func (x *ReadPayload) SetIsBounded(v IsBounded_Enum) { + x.xxx_hidden_IsBounded = v +} + +func (x *ReadPayload) HasSource() bool { + if x == nil { + return false + } + return x.xxx_hidden_Source != nil +} + +func (x *ReadPayload) ClearSource() { + x.xxx_hidden_Source = nil +} + +type ReadPayload_builder struct { + _ [0]func() // Prevents comparability and use of unkeyed literals for the builder. + + // (Required) The FunctionSpec of the source for this Read. + Source *FunctionSpec + // (Required) Whether the source is bounded or unbounded + IsBounded IsBounded_Enum +} + +func (b0 ReadPayload_builder) Build() *ReadPayload { + m0 := &ReadPayload{} + b, x := &b0, m0 + _, _ = b, x + x.xxx_hidden_Source = b.Source + x.xxx_hidden_IsBounded = b.IsBounded + return m0 +} + +// The payload for the WindowInto transform. +type WindowIntoPayload struct { + state protoimpl.MessageState `protogen:"opaque.v1"` + xxx_hidden_WindowFn *FunctionSpec `protobuf:"bytes,1,opt,name=window_fn,json=windowFn,proto3" json:"window_fn,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *WindowIntoPayload) Reset() { + *x = WindowIntoPayload{} + mi := &file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_msgTypes[20] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *WindowIntoPayload) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*WindowIntoPayload) ProtoMessage() {} + +func (x *WindowIntoPayload) ProtoReflect() protoreflect.Message { + mi := &file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_msgTypes[20] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +func (x *WindowIntoPayload) GetWindowFn() *FunctionSpec { + if x != nil { + return x.xxx_hidden_WindowFn + } + return nil +} + +func (x *WindowIntoPayload) SetWindowFn(v *FunctionSpec) { + x.xxx_hidden_WindowFn = v +} + +func (x *WindowIntoPayload) HasWindowFn() bool { + if x == nil { + return false + } + return x.xxx_hidden_WindowFn != nil +} + +func (x *WindowIntoPayload) ClearWindowFn() { + x.xxx_hidden_WindowFn = nil +} + +type WindowIntoPayload_builder struct { + _ [0]func() // Prevents comparability and use of unkeyed literals for the builder. + + // (Required) The FunctionSpec of the WindowFn. + WindowFn *FunctionSpec +} + +func (b0 WindowIntoPayload_builder) Build() *WindowIntoPayload { + m0 := &WindowIntoPayload{} + b, x := &b0, m0 + _, _ = b, x + x.xxx_hidden_WindowFn = b.WindowFn + return m0 +} + +// The payload for the special-but-not-primitive Combine transform. +type CombinePayload struct { + state protoimpl.MessageState `protogen:"opaque.v1"` + xxx_hidden_CombineFn *FunctionSpec `protobuf:"bytes,1,opt,name=combine_fn,json=combineFn,proto3" json:"combine_fn,omitempty"` + xxx_hidden_AccumulatorCoderId string `protobuf:"bytes,2,opt,name=accumulator_coder_id,json=accumulatorCoderId,proto3" json:"accumulator_coder_id,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *CombinePayload) Reset() { + *x = CombinePayload{} + mi := &file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_msgTypes[21] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *CombinePayload) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*CombinePayload) ProtoMessage() {} + +func (x *CombinePayload) ProtoReflect() protoreflect.Message { + mi := &file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_msgTypes[21] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +func (x *CombinePayload) GetCombineFn() *FunctionSpec { + if x != nil { + return x.xxx_hidden_CombineFn + } + return nil +} + +func (x *CombinePayload) GetAccumulatorCoderId() string { + if x != nil { + return x.xxx_hidden_AccumulatorCoderId + } + return "" +} + +func (x *CombinePayload) SetCombineFn(v *FunctionSpec) { + x.xxx_hidden_CombineFn = v +} + +func (x *CombinePayload) SetAccumulatorCoderId(v string) { + x.xxx_hidden_AccumulatorCoderId = v +} + +func (x *CombinePayload) HasCombineFn() bool { + if x == nil { + return false + } + return x.xxx_hidden_CombineFn != nil +} + +func (x *CombinePayload) ClearCombineFn() { + x.xxx_hidden_CombineFn = nil +} + +type CombinePayload_builder struct { + _ [0]func() // Prevents comparability and use of unkeyed literals for the builder. + + // (Required) The FunctionSpec of the CombineFn. + CombineFn *FunctionSpec + // (Required) A reference to the Coder to use for accumulators of the CombineFn + AccumulatorCoderId string +} + +func (b0 CombinePayload_builder) Build() *CombinePayload { + m0 := &CombinePayload{} + b, x := &b0, m0 + _, _ = b, x + x.xxx_hidden_CombineFn = b.CombineFn + x.xxx_hidden_AccumulatorCoderId = b.AccumulatorCoderId + return m0 +} + +// The payload for the test-only primitive TestStream +type TestStreamPayload struct { + state protoimpl.MessageState `protogen:"opaque.v1"` + xxx_hidden_CoderId string `protobuf:"bytes,1,opt,name=coder_id,json=coderId,proto3" json:"coder_id,omitempty"` + xxx_hidden_Events *[]*TestStreamPayload_Event `protobuf:"bytes,2,rep,name=events,proto3" json:"events,omitempty"` + xxx_hidden_Endpoint *ApiServiceDescriptor `protobuf:"bytes,3,opt,name=endpoint,proto3" json:"endpoint,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *TestStreamPayload) Reset() { + *x = TestStreamPayload{} + mi := &file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_msgTypes[22] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *TestStreamPayload) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*TestStreamPayload) ProtoMessage() {} + +func (x *TestStreamPayload) ProtoReflect() protoreflect.Message { + mi := &file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_msgTypes[22] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +func (x *TestStreamPayload) GetCoderId() string { + if x != nil { + return x.xxx_hidden_CoderId + } + return "" +} + +func (x *TestStreamPayload) GetEvents() []*TestStreamPayload_Event { + if x != nil { + if x.xxx_hidden_Events != nil { + return *x.xxx_hidden_Events + } + } + return nil +} + +func (x *TestStreamPayload) GetEndpoint() *ApiServiceDescriptor { + if x != nil { + return x.xxx_hidden_Endpoint + } + return nil +} + +func (x *TestStreamPayload) SetCoderId(v string) { + x.xxx_hidden_CoderId = v +} + +func (x *TestStreamPayload) SetEvents(v []*TestStreamPayload_Event) { + x.xxx_hidden_Events = &v +} + +func (x *TestStreamPayload) SetEndpoint(v *ApiServiceDescriptor) { + x.xxx_hidden_Endpoint = v +} + +func (x *TestStreamPayload) HasEndpoint() bool { + if x == nil { + return false + } + return x.xxx_hidden_Endpoint != nil +} + +func (x *TestStreamPayload) ClearEndpoint() { + x.xxx_hidden_Endpoint = nil +} + +type TestStreamPayload_builder struct { + _ [0]func() // Prevents comparability and use of unkeyed literals for the builder. + + // (Required) the coder for elements in the TestStream events + CoderId string + // (Optional) If specified, the TestStream will replay these events. + Events []*TestStreamPayload_Event + // (Optional) If specified, points to a TestStreamService to be + // used to retrieve events. + Endpoint *ApiServiceDescriptor +} + +func (b0 TestStreamPayload_builder) Build() *TestStreamPayload { + m0 := &TestStreamPayload{} + b, x := &b0, m0 + _, _ = b, x + x.xxx_hidden_CoderId = b.CoderId + x.xxx_hidden_Events = &b.Events + x.xxx_hidden_Endpoint = b.Endpoint + return m0 +} + +type EventsRequest struct { + state protoimpl.MessageState `protogen:"opaque.v1"` + xxx_hidden_OutputIds []string `protobuf:"bytes,1,rep,name=output_ids,json=outputIds,proto3" json:"output_ids,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *EventsRequest) Reset() { + *x = EventsRequest{} + mi := &file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_msgTypes[23] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *EventsRequest) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*EventsRequest) ProtoMessage() {} + +func (x *EventsRequest) ProtoReflect() protoreflect.Message { + mi := &file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_msgTypes[23] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +func (x *EventsRequest) GetOutputIds() []string { + if x != nil { + return x.xxx_hidden_OutputIds + } + return nil +} + +func (x *EventsRequest) SetOutputIds(v []string) { + x.xxx_hidden_OutputIds = v +} + +type EventsRequest_builder struct { + _ [0]func() // Prevents comparability and use of unkeyed literals for the builder. + + // The set of PCollections to read from. These are the PTransform outputs + // local names. These are a subset of the TestStream's outputs. This allows + // Interactive Beam to cache many PCollections from a pipeline then replay a + // subset of them. + OutputIds []string +} + +func (b0 EventsRequest_builder) Build() *EventsRequest { + m0 := &EventsRequest{} + b, x := &b0, m0 + _, _ = b, x + x.xxx_hidden_OutputIds = b.OutputIds + return m0 +} + +// The payload for the special-but-not-primitive WriteFiles transform. +type WriteFilesPayload struct { + state protoimpl.MessageState `protogen:"opaque.v1"` + xxx_hidden_Sink *FunctionSpec `protobuf:"bytes,1,opt,name=sink,proto3" json:"sink,omitempty"` + xxx_hidden_FormatFunction *FunctionSpec `protobuf:"bytes,2,opt,name=format_function,json=formatFunction,proto3" json:"format_function,omitempty"` + xxx_hidden_WindowedWrites bool `protobuf:"varint,3,opt,name=windowed_writes,json=windowedWrites,proto3" json:"windowed_writes,omitempty"` + xxx_hidden_RunnerDeterminedSharding bool `protobuf:"varint,4,opt,name=runner_determined_sharding,json=runnerDeterminedSharding,proto3" json:"runner_determined_sharding,omitempty"` + xxx_hidden_SideInputs map[string]*SideInput `protobuf:"bytes,5,rep,name=side_inputs,json=sideInputs,proto3" json:"side_inputs,omitempty" protobuf_key:"bytes,1,opt,name=key" protobuf_val:"bytes,2,opt,name=value"` + xxx_hidden_AutoSharded bool `protobuf:"varint,6,opt,name=auto_sharded,json=autoSharded,proto3" json:"auto_sharded,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *WriteFilesPayload) Reset() { + *x = WriteFilesPayload{} + mi := &file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_msgTypes[24] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *WriteFilesPayload) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*WriteFilesPayload) ProtoMessage() {} + +func (x *WriteFilesPayload) ProtoReflect() protoreflect.Message { + mi := &file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_msgTypes[24] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +func (x *WriteFilesPayload) GetSink() *FunctionSpec { + if x != nil { + return x.xxx_hidden_Sink + } + return nil +} + +func (x *WriteFilesPayload) GetFormatFunction() *FunctionSpec { + if x != nil { + return x.xxx_hidden_FormatFunction + } + return nil +} + +func (x *WriteFilesPayload) GetWindowedWrites() bool { + if x != nil { + return x.xxx_hidden_WindowedWrites + } + return false +} + +func (x *WriteFilesPayload) GetRunnerDeterminedSharding() bool { + if x != nil { + return x.xxx_hidden_RunnerDeterminedSharding + } + return false +} + +func (x *WriteFilesPayload) GetSideInputs() map[string]*SideInput { + if x != nil { + return x.xxx_hidden_SideInputs + } + return nil +} + +func (x *WriteFilesPayload) GetAutoSharded() bool { + if x != nil { + return x.xxx_hidden_AutoSharded + } + return false +} + +func (x *WriteFilesPayload) SetSink(v *FunctionSpec) { + x.xxx_hidden_Sink = v +} + +func (x *WriteFilesPayload) SetFormatFunction(v *FunctionSpec) { + x.xxx_hidden_FormatFunction = v +} + +func (x *WriteFilesPayload) SetWindowedWrites(v bool) { + x.xxx_hidden_WindowedWrites = v +} + +func (x *WriteFilesPayload) SetRunnerDeterminedSharding(v bool) { + x.xxx_hidden_RunnerDeterminedSharding = v +} + +func (x *WriteFilesPayload) SetSideInputs(v map[string]*SideInput) { + x.xxx_hidden_SideInputs = v +} + +func (x *WriteFilesPayload) SetAutoSharded(v bool) { + x.xxx_hidden_AutoSharded = v +} + +func (x *WriteFilesPayload) HasSink() bool { + if x == nil { + return false + } + return x.xxx_hidden_Sink != nil +} + +func (x *WriteFilesPayload) HasFormatFunction() bool { + if x == nil { + return false + } + return x.xxx_hidden_FormatFunction != nil +} + +func (x *WriteFilesPayload) ClearSink() { + x.xxx_hidden_Sink = nil +} + +func (x *WriteFilesPayload) ClearFormatFunction() { + x.xxx_hidden_FormatFunction = nil +} + +type WriteFilesPayload_builder struct { + _ [0]func() // Prevents comparability and use of unkeyed literals for the builder. + + // (Required) The FunctionSpec of the FileBasedSink. + Sink *FunctionSpec + // (Required) The format function. + FormatFunction *FunctionSpec + WindowedWrites bool + RunnerDeterminedSharding bool + SideInputs map[string]*SideInput + // This is different from runner based sharding. This is done by the runner backend, where as runner_determined_sharding + // is by the runner translator + AutoSharded bool +} + +func (b0 WriteFilesPayload_builder) Build() *WriteFilesPayload { + m0 := &WriteFilesPayload{} + b, x := &b0, m0 + _, _ = b, x + x.xxx_hidden_Sink = b.Sink + x.xxx_hidden_FormatFunction = b.FormatFunction + x.xxx_hidden_WindowedWrites = b.WindowedWrites + x.xxx_hidden_RunnerDeterminedSharding = b.RunnerDeterminedSharding + x.xxx_hidden_SideInputs = b.SideInputs + x.xxx_hidden_AutoSharded = b.AutoSharded + return m0 +} + +// Payload used by Google Cloud Pub/Sub read transform. +// This can be used by runners that wish to override Beam Pub/Sub read transform +// with a native implementation. +// The SDK should guarantee that only one of topic, subscription, +// topic_runtime_overridden and subscription_runtime_overridden is set. +// The output of PubSubReadPayload should be bytes of serialized PubsubMessage +// proto if with_attributes == true. Otherwise, the bytes is the raw payload. +type PubSubReadPayload struct { + state protoimpl.MessageState `protogen:"opaque.v1"` + xxx_hidden_Topic string `protobuf:"bytes,1,opt,name=topic,proto3" json:"topic,omitempty"` + xxx_hidden_Subscription string `protobuf:"bytes,2,opt,name=subscription,proto3" json:"subscription,omitempty"` + xxx_hidden_TimestampAttribute string `protobuf:"bytes,3,opt,name=timestamp_attribute,json=timestampAttribute,proto3" json:"timestamp_attribute,omitempty"` + xxx_hidden_IdAttribute string `protobuf:"bytes,4,opt,name=id_attribute,json=idAttribute,proto3" json:"id_attribute,omitempty"` + xxx_hidden_WithAttributes bool `protobuf:"varint,5,opt,name=with_attributes,json=withAttributes,proto3" json:"with_attributes,omitempty"` + xxx_hidden_TopicRuntimeOverridden string `protobuf:"bytes,6,opt,name=topic_runtime_overridden,json=topicRuntimeOverridden,proto3" json:"topic_runtime_overridden,omitempty"` + xxx_hidden_SubscriptionRuntimeOverridden string `protobuf:"bytes,7,opt,name=subscription_runtime_overridden,json=subscriptionRuntimeOverridden,proto3" json:"subscription_runtime_overridden,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *PubSubReadPayload) Reset() { + *x = PubSubReadPayload{} + mi := &file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_msgTypes[25] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *PubSubReadPayload) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*PubSubReadPayload) ProtoMessage() {} + +func (x *PubSubReadPayload) ProtoReflect() protoreflect.Message { + mi := &file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_msgTypes[25] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +func (x *PubSubReadPayload) GetTopic() string { + if x != nil { + return x.xxx_hidden_Topic + } + return "" +} + +func (x *PubSubReadPayload) GetSubscription() string { + if x != nil { + return x.xxx_hidden_Subscription + } + return "" +} + +func (x *PubSubReadPayload) GetTimestampAttribute() string { + if x != nil { + return x.xxx_hidden_TimestampAttribute + } + return "" +} + +func (x *PubSubReadPayload) GetIdAttribute() string { + if x != nil { + return x.xxx_hidden_IdAttribute + } + return "" +} + +func (x *PubSubReadPayload) GetWithAttributes() bool { + if x != nil { + return x.xxx_hidden_WithAttributes + } + return false +} + +func (x *PubSubReadPayload) GetTopicRuntimeOverridden() string { + if x != nil { + return x.xxx_hidden_TopicRuntimeOverridden + } + return "" +} + +func (x *PubSubReadPayload) GetSubscriptionRuntimeOverridden() string { + if x != nil { + return x.xxx_hidden_SubscriptionRuntimeOverridden + } + return "" +} + +func (x *PubSubReadPayload) SetTopic(v string) { + x.xxx_hidden_Topic = v +} + +func (x *PubSubReadPayload) SetSubscription(v string) { + x.xxx_hidden_Subscription = v +} + +func (x *PubSubReadPayload) SetTimestampAttribute(v string) { + x.xxx_hidden_TimestampAttribute = v +} + +func (x *PubSubReadPayload) SetIdAttribute(v string) { + x.xxx_hidden_IdAttribute = v +} + +func (x *PubSubReadPayload) SetWithAttributes(v bool) { + x.xxx_hidden_WithAttributes = v +} + +func (x *PubSubReadPayload) SetTopicRuntimeOverridden(v string) { + x.xxx_hidden_TopicRuntimeOverridden = v +} + +func (x *PubSubReadPayload) SetSubscriptionRuntimeOverridden(v string) { + x.xxx_hidden_SubscriptionRuntimeOverridden = v +} + +type PubSubReadPayload_builder struct { + _ [0]func() // Prevents comparability and use of unkeyed literals for the builder. + + // Topic to read from. Exactly one of topic or subscription should be set. + // Topic format is: /topics/project_id/subscription_name + Topic string + // Subscription to read from. Exactly one of topic or subscription should be set. + // Subscription format is: /subscriptions/project_id/subscription_name + Subscription string + // Attribute that provides element timestamps. + TimestampAttribute string + // Attribute to be used for uniquely identifying messages. + IdAttribute string + // If true, reads Pub/Sub payload as well as attributes. If false, reads only the payload. + WithAttributes bool + // If set, the topic is expected to be provided during runtime. + TopicRuntimeOverridden string + // If set, the subscription that is expected to be provided during runtime. + SubscriptionRuntimeOverridden string +} + +func (b0 PubSubReadPayload_builder) Build() *PubSubReadPayload { + m0 := &PubSubReadPayload{} + b, x := &b0, m0 + _, _ = b, x + x.xxx_hidden_Topic = b.Topic + x.xxx_hidden_Subscription = b.Subscription + x.xxx_hidden_TimestampAttribute = b.TimestampAttribute + x.xxx_hidden_IdAttribute = b.IdAttribute + x.xxx_hidden_WithAttributes = b.WithAttributes + x.xxx_hidden_TopicRuntimeOverridden = b.TopicRuntimeOverridden + x.xxx_hidden_SubscriptionRuntimeOverridden = b.SubscriptionRuntimeOverridden + return m0 +} + +// Payload used by Google Cloud Pub/Sub write transform. +// This can be used by runners that wish to override Beam Pub/Sub write transform +// with a native implementation. +// The SDK should guarantee that only one of topic and topic_runtime_overridden +// is set. +// The output of PubSubWritePayload should be bytes if serialized PubsubMessage +// proto. +type PubSubWritePayload struct { + state protoimpl.MessageState `protogen:"opaque.v1"` + xxx_hidden_Topic string `protobuf:"bytes,1,opt,name=topic,proto3" json:"topic,omitempty"` + xxx_hidden_TimestampAttribute string `protobuf:"bytes,2,opt,name=timestamp_attribute,json=timestampAttribute,proto3" json:"timestamp_attribute,omitempty"` + xxx_hidden_IdAttribute string `protobuf:"bytes,3,opt,name=id_attribute,json=idAttribute,proto3" json:"id_attribute,omitempty"` + xxx_hidden_TopicRuntimeOverridden string `protobuf:"bytes,4,opt,name=topic_runtime_overridden,json=topicRuntimeOverridden,proto3" json:"topic_runtime_overridden,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *PubSubWritePayload) Reset() { + *x = PubSubWritePayload{} + mi := &file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_msgTypes[26] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *PubSubWritePayload) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*PubSubWritePayload) ProtoMessage() {} + +func (x *PubSubWritePayload) ProtoReflect() protoreflect.Message { + mi := &file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_msgTypes[26] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +func (x *PubSubWritePayload) GetTopic() string { + if x != nil { + return x.xxx_hidden_Topic + } + return "" +} + +func (x *PubSubWritePayload) GetTimestampAttribute() string { + if x != nil { + return x.xxx_hidden_TimestampAttribute + } + return "" +} + +func (x *PubSubWritePayload) GetIdAttribute() string { + if x != nil { + return x.xxx_hidden_IdAttribute + } + return "" +} + +func (x *PubSubWritePayload) GetTopicRuntimeOverridden() string { + if x != nil { + return x.xxx_hidden_TopicRuntimeOverridden + } + return "" +} + +func (x *PubSubWritePayload) SetTopic(v string) { + x.xxx_hidden_Topic = v +} + +func (x *PubSubWritePayload) SetTimestampAttribute(v string) { + x.xxx_hidden_TimestampAttribute = v +} + +func (x *PubSubWritePayload) SetIdAttribute(v string) { + x.xxx_hidden_IdAttribute = v +} + +func (x *PubSubWritePayload) SetTopicRuntimeOverridden(v string) { + x.xxx_hidden_TopicRuntimeOverridden = v +} + +type PubSubWritePayload_builder struct { + _ [0]func() // Prevents comparability and use of unkeyed literals for the builder. + + // Topic to write to. + // Topic format is: /topics/project_id/subscription_name + Topic string + // Attribute that provides element timestamps. + TimestampAttribute string + // Attribute that uniquely identify messages. + IdAttribute string + // If set, the topic is expected to be provided during runtime. + TopicRuntimeOverridden string +} + +func (b0 PubSubWritePayload_builder) Build() *PubSubWritePayload { + m0 := &PubSubWritePayload{} + b, x := &b0, m0 + _, _ = b, x + x.xxx_hidden_Topic = b.Topic + x.xxx_hidden_TimestampAttribute = b.TimestampAttribute + x.xxx_hidden_IdAttribute = b.IdAttribute + x.xxx_hidden_TopicRuntimeOverridden = b.TopicRuntimeOverridden + return m0 +} + +// Payload for GroupIntoBatches composite transform. +type GroupIntoBatchesPayload struct { + state protoimpl.MessageState `protogen:"opaque.v1"` + xxx_hidden_BatchSize int64 `protobuf:"varint,1,opt,name=batch_size,json=batchSize,proto3" json:"batch_size,omitempty"` + xxx_hidden_BatchSizeBytes int64 `protobuf:"varint,3,opt,name=batch_size_bytes,json=batchSizeBytes,proto3" json:"batch_size_bytes,omitempty"` + xxx_hidden_MaxBufferingDurationMillis int64 `protobuf:"varint,2,opt,name=max_buffering_duration_millis,json=maxBufferingDurationMillis,proto3" json:"max_buffering_duration_millis,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *GroupIntoBatchesPayload) Reset() { + *x = GroupIntoBatchesPayload{} + mi := &file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_msgTypes[27] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *GroupIntoBatchesPayload) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*GroupIntoBatchesPayload) ProtoMessage() {} + +func (x *GroupIntoBatchesPayload) ProtoReflect() protoreflect.Message { + mi := &file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_msgTypes[27] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +func (x *GroupIntoBatchesPayload) GetBatchSize() int64 { + if x != nil { + return x.xxx_hidden_BatchSize + } + return 0 +} + +func (x *GroupIntoBatchesPayload) GetBatchSizeBytes() int64 { + if x != nil { + return x.xxx_hidden_BatchSizeBytes + } + return 0 +} + +func (x *GroupIntoBatchesPayload) GetMaxBufferingDurationMillis() int64 { + if x != nil { + return x.xxx_hidden_MaxBufferingDurationMillis + } + return 0 +} + +func (x *GroupIntoBatchesPayload) SetBatchSize(v int64) { + x.xxx_hidden_BatchSize = v +} + +func (x *GroupIntoBatchesPayload) SetBatchSizeBytes(v int64) { + x.xxx_hidden_BatchSizeBytes = v +} + +func (x *GroupIntoBatchesPayload) SetMaxBufferingDurationMillis(v int64) { + x.xxx_hidden_MaxBufferingDurationMillis = v +} + +type GroupIntoBatchesPayload_builder struct { + _ [0]func() // Prevents comparability and use of unkeyed literals for the builder. + + // Max size of a batch. + BatchSize int64 + // Max byte size of a batch in element. + BatchSizeBytes int64 + // (Optional) Max duration a batch is allowed to be cached in states. + MaxBufferingDurationMillis int64 +} + +func (b0 GroupIntoBatchesPayload_builder) Build() *GroupIntoBatchesPayload { + m0 := &GroupIntoBatchesPayload{} + b, x := &b0, m0 + _, _ = b, x + x.xxx_hidden_BatchSize = b.BatchSize + x.xxx_hidden_BatchSizeBytes = b.BatchSizeBytes + x.xxx_hidden_MaxBufferingDurationMillis = b.MaxBufferingDurationMillis + return m0 +} + +type RedistributePayload struct { + state protoimpl.MessageState `protogen:"opaque.v1"` + xxx_hidden_AllowDuplicates bool `protobuf:"varint,1,opt,name=allow_duplicates,json=allowDuplicates,proto3" json:"allow_duplicates,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *RedistributePayload) Reset() { + *x = RedistributePayload{} + mi := &file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_msgTypes[28] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *RedistributePayload) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*RedistributePayload) ProtoMessage() {} + +func (x *RedistributePayload) ProtoReflect() protoreflect.Message { + mi := &file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_msgTypes[28] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +func (x *RedistributePayload) GetAllowDuplicates() bool { + if x != nil { + return x.xxx_hidden_AllowDuplicates + } + return false +} + +func (x *RedistributePayload) SetAllowDuplicates(v bool) { + x.xxx_hidden_AllowDuplicates = v +} + +type RedistributePayload_builder struct { + _ [0]func() // Prevents comparability and use of unkeyed literals for the builder. + + AllowDuplicates bool +} + +func (b0 RedistributePayload_builder) Build() *RedistributePayload { + m0 := &RedistributePayload{} + b, x := &b0, m0 + _, _ = b, x + x.xxx_hidden_AllowDuplicates = b.AllowDuplicates + return m0 +} + +// A coder, the binary format for serialization and deserialization of data in +// a pipeline. +type Coder struct { + state protoimpl.MessageState `protogen:"opaque.v1"` + xxx_hidden_Spec *FunctionSpec `protobuf:"bytes,1,opt,name=spec,proto3" json:"spec,omitempty"` + xxx_hidden_ComponentCoderIds []string `protobuf:"bytes,2,rep,name=component_coder_ids,json=componentCoderIds,proto3" json:"component_coder_ids,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *Coder) Reset() { + *x = Coder{} + mi := &file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_msgTypes[29] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *Coder) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*Coder) ProtoMessage() {} + +func (x *Coder) ProtoReflect() protoreflect.Message { + mi := &file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_msgTypes[29] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +func (x *Coder) GetSpec() *FunctionSpec { + if x != nil { + return x.xxx_hidden_Spec + } + return nil +} + +func (x *Coder) GetComponentCoderIds() []string { + if x != nil { + return x.xxx_hidden_ComponentCoderIds + } + return nil +} + +func (x *Coder) SetSpec(v *FunctionSpec) { + x.xxx_hidden_Spec = v +} + +func (x *Coder) SetComponentCoderIds(v []string) { + x.xxx_hidden_ComponentCoderIds = v +} + +func (x *Coder) HasSpec() bool { + if x == nil { + return false + } + return x.xxx_hidden_Spec != nil +} + +func (x *Coder) ClearSpec() { + x.xxx_hidden_Spec = nil +} + +type Coder_builder struct { + _ [0]func() // Prevents comparability and use of unkeyed literals for the builder. + + // (Required) A specification for the coder, as a URN plus parameters. This + // may be a cross-language agreed-upon format, or it may be a "custom coder" + // that can only be used by a particular SDK. It does not include component + // coders, as it is beneficial for these to be comprehensible to a runner + // regardless of whether the binary format is agreed-upon. + Spec *FunctionSpec + // (Optional) If this coder is parametric, such as ListCoder(VarIntCoder), + // this is a list of the components. In order for encodings to be identical, + // the FunctionSpec and all components must be identical, recursively. + ComponentCoderIds []string +} + +func (b0 Coder_builder) Build() *Coder { + m0 := &Coder{} + b, x := &b0, m0 + _, _ = b, x + x.xxx_hidden_Spec = b.Spec + x.xxx_hidden_ComponentCoderIds = b.ComponentCoderIds + return m0 +} + +type StandardCoders struct { + state protoimpl.MessageState `protogen:"opaque.v1"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *StandardCoders) Reset() { + *x = StandardCoders{} + mi := &file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_msgTypes[30] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *StandardCoders) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*StandardCoders) ProtoMessage() {} + +func (x *StandardCoders) ProtoReflect() protoreflect.Message { + mi := &file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_msgTypes[30] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +type StandardCoders_builder struct { + _ [0]func() // Prevents comparability and use of unkeyed literals for the builder. + +} + +func (b0 StandardCoders_builder) Build() *StandardCoders { + m0 := &StandardCoders{} + b, x := &b0, m0 + _, _ = b, x + return m0 +} + +// A windowing strategy describes the window function, triggering, allowed +// lateness, and accumulation mode for a PCollection. +// +// TODO: consider inlining field on PCollection +type WindowingStrategy struct { + state protoimpl.MessageState `protogen:"opaque.v1"` + xxx_hidden_WindowFn *FunctionSpec `protobuf:"bytes,1,opt,name=window_fn,json=windowFn,proto3" json:"window_fn,omitempty"` + xxx_hidden_MergeStatus MergeStatus_Enum `protobuf:"varint,2,opt,name=merge_status,json=mergeStatus,proto3,enum=org.apache.beam.model.pipeline.v1.MergeStatus_Enum" json:"merge_status,omitempty"` + xxx_hidden_WindowCoderId string `protobuf:"bytes,3,opt,name=window_coder_id,json=windowCoderId,proto3" json:"window_coder_id,omitempty"` + xxx_hidden_Trigger *Trigger `protobuf:"bytes,4,opt,name=trigger,proto3" json:"trigger,omitempty"` + xxx_hidden_AccumulationMode AccumulationMode_Enum `protobuf:"varint,5,opt,name=accumulation_mode,json=accumulationMode,proto3,enum=org.apache.beam.model.pipeline.v1.AccumulationMode_Enum" json:"accumulation_mode,omitempty"` + xxx_hidden_OutputTime OutputTime_Enum `protobuf:"varint,6,opt,name=output_time,json=outputTime,proto3,enum=org.apache.beam.model.pipeline.v1.OutputTime_Enum" json:"output_time,omitempty"` + xxx_hidden_ClosingBehavior ClosingBehavior_Enum `protobuf:"varint,7,opt,name=closing_behavior,json=closingBehavior,proto3,enum=org.apache.beam.model.pipeline.v1.ClosingBehavior_Enum" json:"closing_behavior,omitempty"` + xxx_hidden_AllowedLateness int64 `protobuf:"varint,8,opt,name=allowed_lateness,json=allowedLateness,proto3" json:"allowed_lateness,omitempty"` + xxx_hidden_OnTimeBehavior OnTimeBehavior_Enum `protobuf:"varint,9,opt,name=on_time_behavior,json=onTimeBehavior,proto3,enum=org.apache.beam.model.pipeline.v1.OnTimeBehavior_Enum" json:"on_time_behavior,omitempty"` + xxx_hidden_AssignsToOneWindow bool `protobuf:"varint,10,opt,name=assigns_to_one_window,json=assignsToOneWindow,proto3" json:"assigns_to_one_window,omitempty"` + xxx_hidden_EnvironmentId string `protobuf:"bytes,11,opt,name=environment_id,json=environmentId,proto3" json:"environment_id,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *WindowingStrategy) Reset() { + *x = WindowingStrategy{} + mi := &file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_msgTypes[31] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *WindowingStrategy) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*WindowingStrategy) ProtoMessage() {} + +func (x *WindowingStrategy) ProtoReflect() protoreflect.Message { + mi := &file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_msgTypes[31] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +func (x *WindowingStrategy) GetWindowFn() *FunctionSpec { + if x != nil { + return x.xxx_hidden_WindowFn + } + return nil +} + +func (x *WindowingStrategy) GetMergeStatus() MergeStatus_Enum { + if x != nil { + return x.xxx_hidden_MergeStatus + } + return MergeStatus_UNSPECIFIED +} + +func (x *WindowingStrategy) GetWindowCoderId() string { + if x != nil { + return x.xxx_hidden_WindowCoderId + } + return "" +} + +func (x *WindowingStrategy) GetTrigger() *Trigger { + if x != nil { + return x.xxx_hidden_Trigger + } + return nil +} + +func (x *WindowingStrategy) GetAccumulationMode() AccumulationMode_Enum { + if x != nil { + return x.xxx_hidden_AccumulationMode + } + return AccumulationMode_UNSPECIFIED +} + +func (x *WindowingStrategy) GetOutputTime() OutputTime_Enum { + if x != nil { + return x.xxx_hidden_OutputTime + } + return OutputTime_UNSPECIFIED +} + +func (x *WindowingStrategy) GetClosingBehavior() ClosingBehavior_Enum { + if x != nil { + return x.xxx_hidden_ClosingBehavior + } + return ClosingBehavior_UNSPECIFIED +} + +func (x *WindowingStrategy) GetAllowedLateness() int64 { + if x != nil { + return x.xxx_hidden_AllowedLateness + } + return 0 +} + +func (x *WindowingStrategy) GetOnTimeBehavior() OnTimeBehavior_Enum { + if x != nil { + return x.xxx_hidden_OnTimeBehavior + } + return OnTimeBehavior_UNSPECIFIED +} + +func (x *WindowingStrategy) GetAssignsToOneWindow() bool { + if x != nil { + return x.xxx_hidden_AssignsToOneWindow + } + return false +} + +func (x *WindowingStrategy) GetEnvironmentId() string { + if x != nil { + return x.xxx_hidden_EnvironmentId + } + return "" +} + +func (x *WindowingStrategy) SetWindowFn(v *FunctionSpec) { + x.xxx_hidden_WindowFn = v +} + +func (x *WindowingStrategy) SetMergeStatus(v MergeStatus_Enum) { + x.xxx_hidden_MergeStatus = v +} + +func (x *WindowingStrategy) SetWindowCoderId(v string) { + x.xxx_hidden_WindowCoderId = v +} + +func (x *WindowingStrategy) SetTrigger(v *Trigger) { + x.xxx_hidden_Trigger = v +} + +func (x *WindowingStrategy) SetAccumulationMode(v AccumulationMode_Enum) { + x.xxx_hidden_AccumulationMode = v +} + +func (x *WindowingStrategy) SetOutputTime(v OutputTime_Enum) { + x.xxx_hidden_OutputTime = v +} + +func (x *WindowingStrategy) SetClosingBehavior(v ClosingBehavior_Enum) { + x.xxx_hidden_ClosingBehavior = v +} + +func (x *WindowingStrategy) SetAllowedLateness(v int64) { + x.xxx_hidden_AllowedLateness = v +} + +func (x *WindowingStrategy) SetOnTimeBehavior(v OnTimeBehavior_Enum) { + x.xxx_hidden_OnTimeBehavior = v +} + +func (x *WindowingStrategy) SetAssignsToOneWindow(v bool) { + x.xxx_hidden_AssignsToOneWindow = v +} + +func (x *WindowingStrategy) SetEnvironmentId(v string) { + x.xxx_hidden_EnvironmentId = v +} + +func (x *WindowingStrategy) HasWindowFn() bool { + if x == nil { + return false + } + return x.xxx_hidden_WindowFn != nil +} + +func (x *WindowingStrategy) HasTrigger() bool { + if x == nil { + return false + } + return x.xxx_hidden_Trigger != nil +} + +func (x *WindowingStrategy) ClearWindowFn() { + x.xxx_hidden_WindowFn = nil +} + +func (x *WindowingStrategy) ClearTrigger() { + x.xxx_hidden_Trigger = nil +} + +type WindowingStrategy_builder struct { + _ [0]func() // Prevents comparability and use of unkeyed literals for the builder. + + // (Required) The FunctionSpec of the UDF that assigns windows, + // merges windows, and shifts timestamps before they are + // combined according to the OutputTime. + WindowFn *FunctionSpec + // (Required) Whether or not the window fn is merging. + // + // This knowledge is required for many optimizations. + MergeStatus MergeStatus_Enum + // (Required) The coder for the windows of this PCollection. + WindowCoderId string + // (Required) The trigger to use when grouping this PCollection. + Trigger *Trigger + // (Required) The accumulation mode indicates whether new panes are a full + // replacement for prior panes or whether they are deltas to be combined + // with other panes (the combine should correspond to whatever the upstream + // grouping transform is). + AccumulationMode AccumulationMode_Enum + // (Required) The OutputTime specifies, for a grouping transform, how to + // compute the aggregate timestamp. The window_fn will first possibly shift + // it later, then the OutputTime takes the max, min, or ignores it and takes + // the end of window. + // + // This is actually only for input to grouping transforms, but since they + // may be introduced in runner-specific ways, it is carried along with the + // windowing strategy. + OutputTime OutputTime_Enum + // (Required) Indicate when output should be omitted upon window expiration. + ClosingBehavior ClosingBehavior_Enum + // (Required) The duration, in milliseconds, beyond the end of a window at + // which the window becomes droppable. + AllowedLateness int64 + // (Required) Indicate whether empty on-time panes should be omitted. + OnTimeBehavior OnTimeBehavior_Enum + // (Required) Whether or not the window fn assigns inputs to exactly one window + // + // This knowledge is required for some optimizations + AssignsToOneWindow bool + // (Optional) Environment where the current window_fn should be applied in. + // Runner that executes the pipeline may choose to override this if needed. + // If not specified, environment will be decided by the runner. + EnvironmentId string +} + +func (b0 WindowingStrategy_builder) Build() *WindowingStrategy { + m0 := &WindowingStrategy{} + b, x := &b0, m0 + _, _ = b, x + x.xxx_hidden_WindowFn = b.WindowFn + x.xxx_hidden_MergeStatus = b.MergeStatus + x.xxx_hidden_WindowCoderId = b.WindowCoderId + x.xxx_hidden_Trigger = b.Trigger + x.xxx_hidden_AccumulationMode = b.AccumulationMode + x.xxx_hidden_OutputTime = b.OutputTime + x.xxx_hidden_ClosingBehavior = b.ClosingBehavior + x.xxx_hidden_AllowedLateness = b.AllowedLateness + x.xxx_hidden_OnTimeBehavior = b.OnTimeBehavior + x.xxx_hidden_AssignsToOneWindow = b.AssignsToOneWindow + x.xxx_hidden_EnvironmentId = b.EnvironmentId + return m0 +} + +// Whether or not a PCollection's WindowFn is non-merging, merging, or +// merging-but-already-merged, in which case a subsequent GroupByKey is almost +// always going to do something the user does not want +type MergeStatus struct { + state protoimpl.MessageState `protogen:"opaque.v1"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *MergeStatus) Reset() { + *x = MergeStatus{} + mi := &file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_msgTypes[32] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *MergeStatus) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*MergeStatus) ProtoMessage() {} + +func (x *MergeStatus) ProtoReflect() protoreflect.Message { + mi := &file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_msgTypes[32] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +type MergeStatus_builder struct { + _ [0]func() // Prevents comparability and use of unkeyed literals for the builder. + +} + +func (b0 MergeStatus_builder) Build() *MergeStatus { + m0 := &MergeStatus{} + b, x := &b0, m0 + _, _ = b, x + return m0 +} + +// Whether or not subsequent outputs of aggregations should be entire +// replacement values or just the aggregation of inputs received since +// the prior output. +type AccumulationMode struct { + state protoimpl.MessageState `protogen:"opaque.v1"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *AccumulationMode) Reset() { + *x = AccumulationMode{} + mi := &file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_msgTypes[33] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *AccumulationMode) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*AccumulationMode) ProtoMessage() {} + +func (x *AccumulationMode) ProtoReflect() protoreflect.Message { + mi := &file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_msgTypes[33] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +type AccumulationMode_builder struct { + _ [0]func() // Prevents comparability and use of unkeyed literals for the builder. + +} + +func (b0 AccumulationMode_builder) Build() *AccumulationMode { + m0 := &AccumulationMode{} + b, x := &b0, m0 + _, _ = b, x + return m0 +} + +// Controls whether or not an aggregating transform should output data +// when a window expires. +type ClosingBehavior struct { + state protoimpl.MessageState `protogen:"opaque.v1"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *ClosingBehavior) Reset() { + *x = ClosingBehavior{} + mi := &file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_msgTypes[34] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *ClosingBehavior) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*ClosingBehavior) ProtoMessage() {} + +func (x *ClosingBehavior) ProtoReflect() protoreflect.Message { + mi := &file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_msgTypes[34] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +type ClosingBehavior_builder struct { + _ [0]func() // Prevents comparability and use of unkeyed literals for the builder. + +} + +func (b0 ClosingBehavior_builder) Build() *ClosingBehavior { + m0 := &ClosingBehavior{} + b, x := &b0, m0 + _, _ = b, x + return m0 +} + +// Controls whether or not an aggregating transform should output data +// when an on-time pane is empty. +type OnTimeBehavior struct { + state protoimpl.MessageState `protogen:"opaque.v1"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *OnTimeBehavior) Reset() { + *x = OnTimeBehavior{} + mi := &file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_msgTypes[35] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *OnTimeBehavior) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*OnTimeBehavior) ProtoMessage() {} + +func (x *OnTimeBehavior) ProtoReflect() protoreflect.Message { + mi := &file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_msgTypes[35] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +type OnTimeBehavior_builder struct { + _ [0]func() // Prevents comparability and use of unkeyed literals for the builder. + +} + +func (b0 OnTimeBehavior_builder) Build() *OnTimeBehavior { + m0 := &OnTimeBehavior{} + b, x := &b0, m0 + _, _ = b, x + return m0 +} + +// When a number of windowed, timestamped inputs are aggregated, the timestamp +// for the resulting output. +type OutputTime struct { + state protoimpl.MessageState `protogen:"opaque.v1"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *OutputTime) Reset() { + *x = OutputTime{} + mi := &file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_msgTypes[36] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *OutputTime) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*OutputTime) ProtoMessage() {} + +func (x *OutputTime) ProtoReflect() protoreflect.Message { + mi := &file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_msgTypes[36] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +type OutputTime_builder struct { + _ [0]func() // Prevents comparability and use of unkeyed literals for the builder. + +} + +func (b0 OutputTime_builder) Build() *OutputTime { + m0 := &OutputTime{} + b, x := &b0, m0 + _, _ = b, x + return m0 +} + +// The different time domains in the Beam model. +type TimeDomain struct { + state protoimpl.MessageState `protogen:"opaque.v1"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *TimeDomain) Reset() { + *x = TimeDomain{} + mi := &file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_msgTypes[37] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *TimeDomain) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*TimeDomain) ProtoMessage() {} + +func (x *TimeDomain) ProtoReflect() protoreflect.Message { + mi := &file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_msgTypes[37] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +type TimeDomain_builder struct { + _ [0]func() // Prevents comparability and use of unkeyed literals for the builder. + +} + +func (b0 TimeDomain_builder) Build() *TimeDomain { + m0 := &TimeDomain{} + b, x := &b0, m0 + _, _ = b, x + return m0 +} + +// A small DSL for expressing when to emit new aggregations +// from a GroupByKey or CombinePerKey +// +// A trigger is described in terms of when it is _ready_ to permit output. +type Trigger struct { + state protoimpl.MessageState `protogen:"opaque.v1"` + xxx_hidden_Trigger isTrigger_Trigger `protobuf_oneof:"trigger"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *Trigger) Reset() { + *x = Trigger{} + mi := &file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_msgTypes[38] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *Trigger) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*Trigger) ProtoMessage() {} + +func (x *Trigger) ProtoReflect() protoreflect.Message { + mi := &file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_msgTypes[38] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +func (x *Trigger) GetAfterAll() *Trigger_AfterAll { + if x != nil { + if x, ok := x.xxx_hidden_Trigger.(*trigger_AfterAll_); ok { + return x.AfterAll + } + } + return nil +} + +func (x *Trigger) GetAfterAny() *Trigger_AfterAny { + if x != nil { + if x, ok := x.xxx_hidden_Trigger.(*trigger_AfterAny_); ok { + return x.AfterAny + } + } + return nil +} + +func (x *Trigger) GetAfterEach() *Trigger_AfterEach { + if x != nil { + if x, ok := x.xxx_hidden_Trigger.(*trigger_AfterEach_); ok { + return x.AfterEach + } + } + return nil +} + +func (x *Trigger) GetAfterEndOfWindow() *Trigger_AfterEndOfWindow { + if x != nil { + if x, ok := x.xxx_hidden_Trigger.(*trigger_AfterEndOfWindow_); ok { + return x.AfterEndOfWindow + } + } + return nil +} + +func (x *Trigger) GetAfterProcessingTime() *Trigger_AfterProcessingTime { + if x != nil { + if x, ok := x.xxx_hidden_Trigger.(*trigger_AfterProcessingTime_); ok { + return x.AfterProcessingTime + } + } + return nil +} + +func (x *Trigger) GetAfterSynchronizedProcessingTime() *Trigger_AfterSynchronizedProcessingTime { + if x != nil { + if x, ok := x.xxx_hidden_Trigger.(*trigger_AfterSynchronizedProcessingTime_); ok { + return x.AfterSynchronizedProcessingTime + } + } + return nil +} + +func (x *Trigger) GetAlways() *Trigger_Always { + if x != nil { + if x, ok := x.xxx_hidden_Trigger.(*trigger_Always_); ok { + return x.Always + } + } + return nil +} + +func (x *Trigger) GetDefault() *Trigger_Default { + if x != nil { + if x, ok := x.xxx_hidden_Trigger.(*trigger_Default_); ok { + return x.Default + } + } + return nil +} + +func (x *Trigger) GetElementCount() *Trigger_ElementCount { + if x != nil { + if x, ok := x.xxx_hidden_Trigger.(*trigger_ElementCount_); ok { + return x.ElementCount + } + } + return nil +} + +func (x *Trigger) GetNever() *Trigger_Never { + if x != nil { + if x, ok := x.xxx_hidden_Trigger.(*trigger_Never_); ok { + return x.Never + } + } + return nil +} + +func (x *Trigger) GetOrFinally() *Trigger_OrFinally { + if x != nil { + if x, ok := x.xxx_hidden_Trigger.(*trigger_OrFinally_); ok { + return x.OrFinally + } + } + return nil +} + +func (x *Trigger) GetRepeat() *Trigger_Repeat { + if x != nil { + if x, ok := x.xxx_hidden_Trigger.(*trigger_Repeat_); ok { + return x.Repeat + } + } + return nil +} + +func (x *Trigger) SetAfterAll(v *Trigger_AfterAll) { + if v == nil { + x.xxx_hidden_Trigger = nil + return + } + x.xxx_hidden_Trigger = &trigger_AfterAll_{v} +} + +func (x *Trigger) SetAfterAny(v *Trigger_AfterAny) { + if v == nil { + x.xxx_hidden_Trigger = nil + return + } + x.xxx_hidden_Trigger = &trigger_AfterAny_{v} +} + +func (x *Trigger) SetAfterEach(v *Trigger_AfterEach) { + if v == nil { + x.xxx_hidden_Trigger = nil + return + } + x.xxx_hidden_Trigger = &trigger_AfterEach_{v} +} + +func (x *Trigger) SetAfterEndOfWindow(v *Trigger_AfterEndOfWindow) { + if v == nil { + x.xxx_hidden_Trigger = nil + return + } + x.xxx_hidden_Trigger = &trigger_AfterEndOfWindow_{v} +} + +func (x *Trigger) SetAfterProcessingTime(v *Trigger_AfterProcessingTime) { + if v == nil { + x.xxx_hidden_Trigger = nil + return + } + x.xxx_hidden_Trigger = &trigger_AfterProcessingTime_{v} +} + +func (x *Trigger) SetAfterSynchronizedProcessingTime(v *Trigger_AfterSynchronizedProcessingTime) { + if v == nil { + x.xxx_hidden_Trigger = nil + return + } + x.xxx_hidden_Trigger = &trigger_AfterSynchronizedProcessingTime_{v} +} + +func (x *Trigger) SetAlways(v *Trigger_Always) { + if v == nil { + x.xxx_hidden_Trigger = nil + return + } + x.xxx_hidden_Trigger = &trigger_Always_{v} +} + +func (x *Trigger) SetDefault(v *Trigger_Default) { + if v == nil { + x.xxx_hidden_Trigger = nil + return + } + x.xxx_hidden_Trigger = &trigger_Default_{v} +} + +func (x *Trigger) SetElementCount(v *Trigger_ElementCount) { + if v == nil { + x.xxx_hidden_Trigger = nil + return + } + x.xxx_hidden_Trigger = &trigger_ElementCount_{v} +} + +func (x *Trigger) SetNever(v *Trigger_Never) { + if v == nil { + x.xxx_hidden_Trigger = nil + return + } + x.xxx_hidden_Trigger = &trigger_Never_{v} +} + +func (x *Trigger) SetOrFinally(v *Trigger_OrFinally) { + if v == nil { + x.xxx_hidden_Trigger = nil + return + } + x.xxx_hidden_Trigger = &trigger_OrFinally_{v} +} + +func (x *Trigger) SetRepeat(v *Trigger_Repeat) { + if v == nil { + x.xxx_hidden_Trigger = nil + return + } + x.xxx_hidden_Trigger = &trigger_Repeat_{v} +} + +func (x *Trigger) HasTrigger() bool { + if x == nil { + return false + } + return x.xxx_hidden_Trigger != nil +} + +func (x *Trigger) HasAfterAll() bool { + if x == nil { + return false + } + _, ok := x.xxx_hidden_Trigger.(*trigger_AfterAll_) + return ok +} + +func (x *Trigger) HasAfterAny() bool { + if x == nil { + return false + } + _, ok := x.xxx_hidden_Trigger.(*trigger_AfterAny_) + return ok +} + +func (x *Trigger) HasAfterEach() bool { + if x == nil { + return false + } + _, ok := x.xxx_hidden_Trigger.(*trigger_AfterEach_) + return ok +} + +func (x *Trigger) HasAfterEndOfWindow() bool { + if x == nil { + return false + } + _, ok := x.xxx_hidden_Trigger.(*trigger_AfterEndOfWindow_) + return ok +} + +func (x *Trigger) HasAfterProcessingTime() bool { + if x == nil { + return false + } + _, ok := x.xxx_hidden_Trigger.(*trigger_AfterProcessingTime_) + return ok +} + +func (x *Trigger) HasAfterSynchronizedProcessingTime() bool { + if x == nil { + return false + } + _, ok := x.xxx_hidden_Trigger.(*trigger_AfterSynchronizedProcessingTime_) + return ok +} + +func (x *Trigger) HasAlways() bool { + if x == nil { + return false + } + _, ok := x.xxx_hidden_Trigger.(*trigger_Always_) + return ok +} + +func (x *Trigger) HasDefault() bool { + if x == nil { + return false + } + _, ok := x.xxx_hidden_Trigger.(*trigger_Default_) + return ok +} + +func (x *Trigger) HasElementCount() bool { + if x == nil { + return false + } + _, ok := x.xxx_hidden_Trigger.(*trigger_ElementCount_) + return ok +} + +func (x *Trigger) HasNever() bool { + if x == nil { + return false + } + _, ok := x.xxx_hidden_Trigger.(*trigger_Never_) + return ok +} + +func (x *Trigger) HasOrFinally() bool { + if x == nil { + return false + } + _, ok := x.xxx_hidden_Trigger.(*trigger_OrFinally_) + return ok +} + +func (x *Trigger) HasRepeat() bool { + if x == nil { + return false + } + _, ok := x.xxx_hidden_Trigger.(*trigger_Repeat_) + return ok +} + +func (x *Trigger) ClearTrigger() { + x.xxx_hidden_Trigger = nil +} + +func (x *Trigger) ClearAfterAll() { + if _, ok := x.xxx_hidden_Trigger.(*trigger_AfterAll_); ok { + x.xxx_hidden_Trigger = nil + } +} + +func (x *Trigger) ClearAfterAny() { + if _, ok := x.xxx_hidden_Trigger.(*trigger_AfterAny_); ok { + x.xxx_hidden_Trigger = nil + } +} + +func (x *Trigger) ClearAfterEach() { + if _, ok := x.xxx_hidden_Trigger.(*trigger_AfterEach_); ok { + x.xxx_hidden_Trigger = nil + } +} + +func (x *Trigger) ClearAfterEndOfWindow() { + if _, ok := x.xxx_hidden_Trigger.(*trigger_AfterEndOfWindow_); ok { + x.xxx_hidden_Trigger = nil + } +} + +func (x *Trigger) ClearAfterProcessingTime() { + if _, ok := x.xxx_hidden_Trigger.(*trigger_AfterProcessingTime_); ok { + x.xxx_hidden_Trigger = nil + } +} + +func (x *Trigger) ClearAfterSynchronizedProcessingTime() { + if _, ok := x.xxx_hidden_Trigger.(*trigger_AfterSynchronizedProcessingTime_); ok { + x.xxx_hidden_Trigger = nil + } +} + +func (x *Trigger) ClearAlways() { + if _, ok := x.xxx_hidden_Trigger.(*trigger_Always_); ok { + x.xxx_hidden_Trigger = nil + } +} + +func (x *Trigger) ClearDefault() { + if _, ok := x.xxx_hidden_Trigger.(*trigger_Default_); ok { + x.xxx_hidden_Trigger = nil + } +} + +func (x *Trigger) ClearElementCount() { + if _, ok := x.xxx_hidden_Trigger.(*trigger_ElementCount_); ok { + x.xxx_hidden_Trigger = nil + } +} + +func (x *Trigger) ClearNever() { + if _, ok := x.xxx_hidden_Trigger.(*trigger_Never_); ok { + x.xxx_hidden_Trigger = nil + } +} + +func (x *Trigger) ClearOrFinally() { + if _, ok := x.xxx_hidden_Trigger.(*trigger_OrFinally_); ok { + x.xxx_hidden_Trigger = nil + } +} + +func (x *Trigger) ClearRepeat() { + if _, ok := x.xxx_hidden_Trigger.(*trigger_Repeat_); ok { + x.xxx_hidden_Trigger = nil + } +} + +const Trigger_Trigger_not_set_case case_Trigger_Trigger = 0 +const Trigger_AfterAll_case case_Trigger_Trigger = 1 +const Trigger_AfterAny_case case_Trigger_Trigger = 2 +const Trigger_AfterEach_case case_Trigger_Trigger = 3 +const Trigger_AfterEndOfWindow_case case_Trigger_Trigger = 4 +const Trigger_AfterProcessingTime_case case_Trigger_Trigger = 5 +const Trigger_AfterSynchronizedProcessingTime_case case_Trigger_Trigger = 6 +const Trigger_Always_case case_Trigger_Trigger = 12 +const Trigger_Default_case case_Trigger_Trigger = 7 +const Trigger_ElementCount_case case_Trigger_Trigger = 8 +const Trigger_Never_case case_Trigger_Trigger = 9 +const Trigger_OrFinally_case case_Trigger_Trigger = 10 +const Trigger_Repeat_case case_Trigger_Trigger = 11 + +func (x *Trigger) WhichTrigger() case_Trigger_Trigger { + if x == nil { + return Trigger_Trigger_not_set_case + } + switch x.xxx_hidden_Trigger.(type) { + case *trigger_AfterAll_: + return Trigger_AfterAll_case + case *trigger_AfterAny_: + return Trigger_AfterAny_case + case *trigger_AfterEach_: + return Trigger_AfterEach_case + case *trigger_AfterEndOfWindow_: + return Trigger_AfterEndOfWindow_case + case *trigger_AfterProcessingTime_: + return Trigger_AfterProcessingTime_case + case *trigger_AfterSynchronizedProcessingTime_: + return Trigger_AfterSynchronizedProcessingTime_case + case *trigger_Always_: + return Trigger_Always_case + case *trigger_Default_: + return Trigger_Default_case + case *trigger_ElementCount_: + return Trigger_ElementCount_case + case *trigger_Never_: + return Trigger_Never_case + case *trigger_OrFinally_: + return Trigger_OrFinally_case + case *trigger_Repeat_: + return Trigger_Repeat_case + default: + return Trigger_Trigger_not_set_case + } +} + +type Trigger_builder struct { + _ [0]func() // Prevents comparability and use of unkeyed literals for the builder. + + // The full disjoint union of possible triggers. + + // Fields of oneof xxx_hidden_Trigger: + AfterAll *Trigger_AfterAll + AfterAny *Trigger_AfterAny + AfterEach *Trigger_AfterEach + AfterEndOfWindow *Trigger_AfterEndOfWindow + AfterProcessingTime *Trigger_AfterProcessingTime + AfterSynchronizedProcessingTime *Trigger_AfterSynchronizedProcessingTime + Always *Trigger_Always + Default *Trigger_Default + ElementCount *Trigger_ElementCount + Never *Trigger_Never + OrFinally *Trigger_OrFinally + Repeat *Trigger_Repeat + // -- end of xxx_hidden_Trigger +} + +func (b0 Trigger_builder) Build() *Trigger { + m0 := &Trigger{} + b, x := &b0, m0 + _, _ = b, x + if b.AfterAll != nil { + x.xxx_hidden_Trigger = &trigger_AfterAll_{b.AfterAll} + } + if b.AfterAny != nil { + x.xxx_hidden_Trigger = &trigger_AfterAny_{b.AfterAny} + } + if b.AfterEach != nil { + x.xxx_hidden_Trigger = &trigger_AfterEach_{b.AfterEach} + } + if b.AfterEndOfWindow != nil { + x.xxx_hidden_Trigger = &trigger_AfterEndOfWindow_{b.AfterEndOfWindow} + } + if b.AfterProcessingTime != nil { + x.xxx_hidden_Trigger = &trigger_AfterProcessingTime_{b.AfterProcessingTime} + } + if b.AfterSynchronizedProcessingTime != nil { + x.xxx_hidden_Trigger = &trigger_AfterSynchronizedProcessingTime_{b.AfterSynchronizedProcessingTime} + } + if b.Always != nil { + x.xxx_hidden_Trigger = &trigger_Always_{b.Always} + } + if b.Default != nil { + x.xxx_hidden_Trigger = &trigger_Default_{b.Default} + } + if b.ElementCount != nil { + x.xxx_hidden_Trigger = &trigger_ElementCount_{b.ElementCount} + } + if b.Never != nil { + x.xxx_hidden_Trigger = &trigger_Never_{b.Never} + } + if b.OrFinally != nil { + x.xxx_hidden_Trigger = &trigger_OrFinally_{b.OrFinally} + } + if b.Repeat != nil { + x.xxx_hidden_Trigger = &trigger_Repeat_{b.Repeat} + } + return m0 +} + +type case_Trigger_Trigger protoreflect.FieldNumber + +func (x case_Trigger_Trigger) String() string { + md := file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_msgTypes[38].Descriptor() + if x == 0 { + return "not set" + } + return protoimpl.X.MessageFieldStringOf(md, protoreflect.FieldNumber(x)) +} + +type isTrigger_Trigger interface { + isTrigger_Trigger() +} + +type trigger_AfterAll_ struct { + AfterAll *Trigger_AfterAll `protobuf:"bytes,1,opt,name=after_all,json=afterAll,proto3,oneof"` +} + +type trigger_AfterAny_ struct { + AfterAny *Trigger_AfterAny `protobuf:"bytes,2,opt,name=after_any,json=afterAny,proto3,oneof"` +} + +type trigger_AfterEach_ struct { + AfterEach *Trigger_AfterEach `protobuf:"bytes,3,opt,name=after_each,json=afterEach,proto3,oneof"` +} + +type trigger_AfterEndOfWindow_ struct { + AfterEndOfWindow *Trigger_AfterEndOfWindow `protobuf:"bytes,4,opt,name=after_end_of_window,json=afterEndOfWindow,proto3,oneof"` +} + +type trigger_AfterProcessingTime_ struct { + AfterProcessingTime *Trigger_AfterProcessingTime `protobuf:"bytes,5,opt,name=after_processing_time,json=afterProcessingTime,proto3,oneof"` +} + +type trigger_AfterSynchronizedProcessingTime_ struct { + AfterSynchronizedProcessingTime *Trigger_AfterSynchronizedProcessingTime `protobuf:"bytes,6,opt,name=after_synchronized_processing_time,json=afterSynchronizedProcessingTime,proto3,oneof"` +} + +type trigger_Always_ struct { + Always *Trigger_Always `protobuf:"bytes,12,opt,name=always,proto3,oneof"` +} + +type trigger_Default_ struct { + Default *Trigger_Default `protobuf:"bytes,7,opt,name=default,proto3,oneof"` +} + +type trigger_ElementCount_ struct { + ElementCount *Trigger_ElementCount `protobuf:"bytes,8,opt,name=element_count,json=elementCount,proto3,oneof"` +} + +type trigger_Never_ struct { + Never *Trigger_Never `protobuf:"bytes,9,opt,name=never,proto3,oneof"` +} + +type trigger_OrFinally_ struct { + OrFinally *Trigger_OrFinally `protobuf:"bytes,10,opt,name=or_finally,json=orFinally,proto3,oneof"` +} + +type trigger_Repeat_ struct { + Repeat *Trigger_Repeat `protobuf:"bytes,11,opt,name=repeat,proto3,oneof"` +} + +func (*trigger_AfterAll_) isTrigger_Trigger() {} + +func (*trigger_AfterAny_) isTrigger_Trigger() {} + +func (*trigger_AfterEach_) isTrigger_Trigger() {} + +func (*trigger_AfterEndOfWindow_) isTrigger_Trigger() {} + +func (*trigger_AfterProcessingTime_) isTrigger_Trigger() {} + +func (*trigger_AfterSynchronizedProcessingTime_) isTrigger_Trigger() {} + +func (*trigger_Always_) isTrigger_Trigger() {} + +func (*trigger_Default_) isTrigger_Trigger() {} + +func (*trigger_ElementCount_) isTrigger_Trigger() {} + +func (*trigger_Never_) isTrigger_Trigger() {} + +func (*trigger_OrFinally_) isTrigger_Trigger() {} + +func (*trigger_Repeat_) isTrigger_Trigger() {} + +// A specification for a transformation on a timestamp. +// +// Primarily used by AfterProcessingTime triggers to transform +// the arrival time of input to a target time for firing. +type TimestampTransform struct { + state protoimpl.MessageState `protogen:"opaque.v1"` + xxx_hidden_TimestampTransform isTimestampTransform_TimestampTransform `protobuf_oneof:"timestamp_transform"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *TimestampTransform) Reset() { + *x = TimestampTransform{} + mi := &file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_msgTypes[39] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *TimestampTransform) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*TimestampTransform) ProtoMessage() {} + +func (x *TimestampTransform) ProtoReflect() protoreflect.Message { + mi := &file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_msgTypes[39] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +func (x *TimestampTransform) GetDelay() *TimestampTransform_Delay { + if x != nil { + if x, ok := x.xxx_hidden_TimestampTransform.(*timestampTransform_Delay_); ok { + return x.Delay + } + } + return nil +} + +func (x *TimestampTransform) GetAlignTo() *TimestampTransform_AlignTo { + if x != nil { + if x, ok := x.xxx_hidden_TimestampTransform.(*timestampTransform_AlignTo_); ok { + return x.AlignTo + } + } + return nil +} + +func (x *TimestampTransform) SetDelay(v *TimestampTransform_Delay) { + if v == nil { + x.xxx_hidden_TimestampTransform = nil + return + } + x.xxx_hidden_TimestampTransform = ×tampTransform_Delay_{v} +} + +func (x *TimestampTransform) SetAlignTo(v *TimestampTransform_AlignTo) { + if v == nil { + x.xxx_hidden_TimestampTransform = nil + return + } + x.xxx_hidden_TimestampTransform = ×tampTransform_AlignTo_{v} +} + +func (x *TimestampTransform) HasTimestampTransform() bool { + if x == nil { + return false + } + return x.xxx_hidden_TimestampTransform != nil +} + +func (x *TimestampTransform) HasDelay() bool { + if x == nil { + return false + } + _, ok := x.xxx_hidden_TimestampTransform.(*timestampTransform_Delay_) + return ok +} + +func (x *TimestampTransform) HasAlignTo() bool { + if x == nil { + return false + } + _, ok := x.xxx_hidden_TimestampTransform.(*timestampTransform_AlignTo_) + return ok +} + +func (x *TimestampTransform) ClearTimestampTransform() { + x.xxx_hidden_TimestampTransform = nil +} + +func (x *TimestampTransform) ClearDelay() { + if _, ok := x.xxx_hidden_TimestampTransform.(*timestampTransform_Delay_); ok { + x.xxx_hidden_TimestampTransform = nil + } +} + +func (x *TimestampTransform) ClearAlignTo() { + if _, ok := x.xxx_hidden_TimestampTransform.(*timestampTransform_AlignTo_); ok { + x.xxx_hidden_TimestampTransform = nil + } +} + +const TimestampTransform_TimestampTransform_not_set_case case_TimestampTransform_TimestampTransform = 0 +const TimestampTransform_Delay_case case_TimestampTransform_TimestampTransform = 1 +const TimestampTransform_AlignTo_case case_TimestampTransform_TimestampTransform = 2 + +func (x *TimestampTransform) WhichTimestampTransform() case_TimestampTransform_TimestampTransform { + if x == nil { + return TimestampTransform_TimestampTransform_not_set_case + } + switch x.xxx_hidden_TimestampTransform.(type) { + case *timestampTransform_Delay_: + return TimestampTransform_Delay_case + case *timestampTransform_AlignTo_: + return TimestampTransform_AlignTo_case + default: + return TimestampTransform_TimestampTransform_not_set_case + } +} + +type TimestampTransform_builder struct { + _ [0]func() // Prevents comparability and use of unkeyed literals for the builder. + + // Fields of oneof xxx_hidden_TimestampTransform: + Delay *TimestampTransform_Delay + AlignTo *TimestampTransform_AlignTo + // -- end of xxx_hidden_TimestampTransform +} + +func (b0 TimestampTransform_builder) Build() *TimestampTransform { + m0 := &TimestampTransform{} + b, x := &b0, m0 + _, _ = b, x + if b.Delay != nil { + x.xxx_hidden_TimestampTransform = ×tampTransform_Delay_{b.Delay} + } + if b.AlignTo != nil { + x.xxx_hidden_TimestampTransform = ×tampTransform_AlignTo_{b.AlignTo} + } + return m0 +} + +type case_TimestampTransform_TimestampTransform protoreflect.FieldNumber + +func (x case_TimestampTransform_TimestampTransform) String() string { + md := file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_msgTypes[39].Descriptor() + if x == 0 { + return "not set" + } + return protoimpl.X.MessageFieldStringOf(md, protoreflect.FieldNumber(x)) +} + +type isTimestampTransform_TimestampTransform interface { + isTimestampTransform_TimestampTransform() +} + +type timestampTransform_Delay_ struct { + Delay *TimestampTransform_Delay `protobuf:"bytes,1,opt,name=delay,proto3,oneof"` +} + +type timestampTransform_AlignTo_ struct { + AlignTo *TimestampTransform_AlignTo `protobuf:"bytes,2,opt,name=align_to,json=alignTo,proto3,oneof"` +} + +func (*timestampTransform_Delay_) isTimestampTransform_TimestampTransform() {} + +func (*timestampTransform_AlignTo_) isTimestampTransform_TimestampTransform() {} + +// A specification for how to "side input" a PCollection. +type SideInput struct { + state protoimpl.MessageState `protogen:"opaque.v1"` + xxx_hidden_AccessPattern *FunctionSpec `protobuf:"bytes,1,opt,name=access_pattern,json=accessPattern,proto3" json:"access_pattern,omitempty"` + xxx_hidden_ViewFn *FunctionSpec `protobuf:"bytes,2,opt,name=view_fn,json=viewFn,proto3" json:"view_fn,omitempty"` + xxx_hidden_WindowMappingFn *FunctionSpec `protobuf:"bytes,3,opt,name=window_mapping_fn,json=windowMappingFn,proto3" json:"window_mapping_fn,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *SideInput) Reset() { + *x = SideInput{} + mi := &file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_msgTypes[40] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *SideInput) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*SideInput) ProtoMessage() {} + +func (x *SideInput) ProtoReflect() protoreflect.Message { + mi := &file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_msgTypes[40] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +func (x *SideInput) GetAccessPattern() *FunctionSpec { + if x != nil { + return x.xxx_hidden_AccessPattern + } + return nil +} + +func (x *SideInput) GetViewFn() *FunctionSpec { + if x != nil { + return x.xxx_hidden_ViewFn + } + return nil +} + +func (x *SideInput) GetWindowMappingFn() *FunctionSpec { + if x != nil { + return x.xxx_hidden_WindowMappingFn + } + return nil +} + +func (x *SideInput) SetAccessPattern(v *FunctionSpec) { + x.xxx_hidden_AccessPattern = v +} + +func (x *SideInput) SetViewFn(v *FunctionSpec) { + x.xxx_hidden_ViewFn = v +} + +func (x *SideInput) SetWindowMappingFn(v *FunctionSpec) { + x.xxx_hidden_WindowMappingFn = v +} + +func (x *SideInput) HasAccessPattern() bool { + if x == nil { + return false + } + return x.xxx_hidden_AccessPattern != nil +} + +func (x *SideInput) HasViewFn() bool { + if x == nil { + return false + } + return x.xxx_hidden_ViewFn != nil +} + +func (x *SideInput) HasWindowMappingFn() bool { + if x == nil { + return false + } + return x.xxx_hidden_WindowMappingFn != nil +} + +func (x *SideInput) ClearAccessPattern() { + x.xxx_hidden_AccessPattern = nil +} + +func (x *SideInput) ClearViewFn() { + x.xxx_hidden_ViewFn = nil +} + +func (x *SideInput) ClearWindowMappingFn() { + x.xxx_hidden_WindowMappingFn = nil +} + +type SideInput_builder struct { + _ [0]func() // Prevents comparability and use of unkeyed literals for the builder. + + // (Required) URN of the access pattern required by the `view_fn` to present + // the desired SDK-specific interface to a UDF. + // + // This access pattern defines the SDK harness <-> Runner Harness RPC + // interface for accessing a side input. + // + // See StandardSideInputTypes for an enumeration of all side input types + // defined. + AccessPattern *FunctionSpec + // (Required) The FunctionSpec of the UDF that adapts a particular + // access_pattern to a user-facing view type. + // + // For example, View.asSingleton() may include a `view_fn` that adapts a + // specially-designed multimap to a single value per window. + ViewFn *FunctionSpec + // (Required) The FunctionSpec of the UDF that maps a main input window + // to a side input window. + // + // For example, when the main input is in fixed windows of one hour, this + // can specify that the side input should be accessed according to the day + // in which that hour falls. + WindowMappingFn *FunctionSpec +} + +func (b0 SideInput_builder) Build() *SideInput { + m0 := &SideInput{} + b, x := &b0, m0 + _, _ = b, x + x.xxx_hidden_AccessPattern = b.AccessPattern + x.xxx_hidden_ViewFn = b.ViewFn + x.xxx_hidden_WindowMappingFn = b.WindowMappingFn + return m0 +} + +type StandardArtifacts struct { + state protoimpl.MessageState `protogen:"opaque.v1"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *StandardArtifacts) Reset() { + *x = StandardArtifacts{} + mi := &file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_msgTypes[41] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *StandardArtifacts) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*StandardArtifacts) ProtoMessage() {} + +func (x *StandardArtifacts) ProtoReflect() protoreflect.Message { + mi := &file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_msgTypes[41] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +type StandardArtifacts_builder struct { + _ [0]func() // Prevents comparability and use of unkeyed literals for the builder. + +} + +func (b0 StandardArtifacts_builder) Build() *StandardArtifacts { + m0 := &StandardArtifacts{} + b, x := &b0, m0 + _, _ = b, x + return m0 +} + +type ArtifactFilePayload struct { + state protoimpl.MessageState `protogen:"opaque.v1"` + xxx_hidden_Path string `protobuf:"bytes,1,opt,name=path,proto3" json:"path,omitempty"` + xxx_hidden_Sha256 string `protobuf:"bytes,2,opt,name=sha256,proto3" json:"sha256,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *ArtifactFilePayload) Reset() { + *x = ArtifactFilePayload{} + mi := &file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_msgTypes[42] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *ArtifactFilePayload) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*ArtifactFilePayload) ProtoMessage() {} + +func (x *ArtifactFilePayload) ProtoReflect() protoreflect.Message { + mi := &file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_msgTypes[42] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +func (x *ArtifactFilePayload) GetPath() string { + if x != nil { + return x.xxx_hidden_Path + } + return "" +} + +func (x *ArtifactFilePayload) GetSha256() string { + if x != nil { + return x.xxx_hidden_Sha256 + } + return "" +} + +func (x *ArtifactFilePayload) SetPath(v string) { + x.xxx_hidden_Path = v +} + +func (x *ArtifactFilePayload) SetSha256(v string) { + x.xxx_hidden_Sha256 = v +} + +type ArtifactFilePayload_builder struct { + _ [0]func() // Prevents comparability and use of unkeyed literals for the builder. + + // a string for an artifact file path e.g. "/tmp/foo.jar" + Path string + // The hex-encoded sha256 checksum of the artifact. + Sha256 string +} + +func (b0 ArtifactFilePayload_builder) Build() *ArtifactFilePayload { + m0 := &ArtifactFilePayload{} + b, x := &b0, m0 + _, _ = b, x + x.xxx_hidden_Path = b.Path + x.xxx_hidden_Sha256 = b.Sha256 + return m0 +} + +type ArtifactUrlPayload struct { + state protoimpl.MessageState `protogen:"opaque.v1"` + xxx_hidden_Url string `protobuf:"bytes,1,opt,name=url,proto3" json:"url,omitempty"` + xxx_hidden_Sha256 string `protobuf:"bytes,2,opt,name=sha256,proto3" json:"sha256,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *ArtifactUrlPayload) Reset() { + *x = ArtifactUrlPayload{} + mi := &file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_msgTypes[43] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *ArtifactUrlPayload) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*ArtifactUrlPayload) ProtoMessage() {} + +func (x *ArtifactUrlPayload) ProtoReflect() protoreflect.Message { + mi := &file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_msgTypes[43] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +func (x *ArtifactUrlPayload) GetUrl() string { + if x != nil { + return x.xxx_hidden_Url + } + return "" +} + +func (x *ArtifactUrlPayload) GetSha256() string { + if x != nil { + return x.xxx_hidden_Sha256 + } + return "" +} + +func (x *ArtifactUrlPayload) SetUrl(v string) { + x.xxx_hidden_Url = v +} + +func (x *ArtifactUrlPayload) SetSha256(v string) { + x.xxx_hidden_Sha256 = v +} + +type ArtifactUrlPayload_builder struct { + _ [0]func() // Prevents comparability and use of unkeyed literals for the builder. + + // a string for an artifact URL e.g. "https://.../foo.jar" or "gs://tmp/foo.jar" + Url string + // (Optional) The hex-encoded sha256 checksum of the artifact if available. + Sha256 string +} + +func (b0 ArtifactUrlPayload_builder) Build() *ArtifactUrlPayload { + m0 := &ArtifactUrlPayload{} + b, x := &b0, m0 + _, _ = b, x + x.xxx_hidden_Url = b.Url + x.xxx_hidden_Sha256 = b.Sha256 + return m0 +} + +type EmbeddedFilePayload struct { + state protoimpl.MessageState `protogen:"opaque.v1"` + xxx_hidden_Data []byte `protobuf:"bytes,1,opt,name=data,proto3" json:"data,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *EmbeddedFilePayload) Reset() { + *x = EmbeddedFilePayload{} + mi := &file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_msgTypes[44] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *EmbeddedFilePayload) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*EmbeddedFilePayload) ProtoMessage() {} + +func (x *EmbeddedFilePayload) ProtoReflect() protoreflect.Message { + mi := &file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_msgTypes[44] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +func (x *EmbeddedFilePayload) GetData() []byte { + if x != nil { + return x.xxx_hidden_Data + } + return nil +} + +func (x *EmbeddedFilePayload) SetData(v []byte) { + if v == nil { + v = []byte{} + } + x.xxx_hidden_Data = v +} + +type EmbeddedFilePayload_builder struct { + _ [0]func() // Prevents comparability and use of unkeyed literals for the builder. + + // raw data bytes for an embedded artifact + Data []byte +} + +func (b0 EmbeddedFilePayload_builder) Build() *EmbeddedFilePayload { + m0 := &EmbeddedFilePayload{} + b, x := &b0, m0 + _, _ = b, x + x.xxx_hidden_Data = b.Data + return m0 +} + +type PyPIPayload struct { + state protoimpl.MessageState `protogen:"opaque.v1"` + xxx_hidden_ArtifactId string `protobuf:"bytes,1,opt,name=artifact_id,json=artifactId,proto3" json:"artifact_id,omitempty"` + xxx_hidden_Version string `protobuf:"bytes,2,opt,name=version,proto3" json:"version,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *PyPIPayload) Reset() { + *x = PyPIPayload{} + mi := &file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_msgTypes[45] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *PyPIPayload) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*PyPIPayload) ProtoMessage() {} + +func (x *PyPIPayload) ProtoReflect() protoreflect.Message { + mi := &file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_msgTypes[45] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +func (x *PyPIPayload) GetArtifactId() string { + if x != nil { + return x.xxx_hidden_ArtifactId + } + return "" +} + +func (x *PyPIPayload) GetVersion() string { + if x != nil { + return x.xxx_hidden_Version + } + return "" +} + +func (x *PyPIPayload) SetArtifactId(v string) { + x.xxx_hidden_ArtifactId = v +} + +func (x *PyPIPayload) SetVersion(v string) { + x.xxx_hidden_Version = v +} + +type PyPIPayload_builder struct { + _ [0]func() // Prevents comparability and use of unkeyed literals for the builder. + + // Pypi compatible artifact id e.g. "apache-beam" + ArtifactId string + // Pypi compatible version string. + Version string +} + +func (b0 PyPIPayload_builder) Build() *PyPIPayload { + m0 := &PyPIPayload{} + b, x := &b0, m0 + _, _ = b, x + x.xxx_hidden_ArtifactId = b.ArtifactId + x.xxx_hidden_Version = b.Version + return m0 +} + +type MavenPayload struct { + state protoimpl.MessageState `protogen:"opaque.v1"` + xxx_hidden_Artifact string `protobuf:"bytes,1,opt,name=artifact,proto3" json:"artifact,omitempty"` + xxx_hidden_RepositoryUrl string `protobuf:"bytes,2,opt,name=repository_url,json=repositoryUrl,proto3" json:"repository_url,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *MavenPayload) Reset() { + *x = MavenPayload{} + mi := &file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_msgTypes[46] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *MavenPayload) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*MavenPayload) ProtoMessage() {} + +func (x *MavenPayload) ProtoReflect() protoreflect.Message { + mi := &file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_msgTypes[46] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +func (x *MavenPayload) GetArtifact() string { + if x != nil { + return x.xxx_hidden_Artifact + } + return "" +} + +func (x *MavenPayload) GetRepositoryUrl() string { + if x != nil { + return x.xxx_hidden_RepositoryUrl + } + return "" +} + +func (x *MavenPayload) SetArtifact(v string) { + x.xxx_hidden_Artifact = v +} + +func (x *MavenPayload) SetRepositoryUrl(v string) { + x.xxx_hidden_RepositoryUrl = v +} + +type MavenPayload_builder struct { + _ [0]func() // Prevents comparability and use of unkeyed literals for the builder. + + // A string specifying Maven artifact. + // The standard format is "groupId:artifactId:version[:packaging[:classifier]]" + Artifact string + // (Optional) Repository URL. If not specified, Maven central is used by default. + RepositoryUrl string +} + +func (b0 MavenPayload_builder) Build() *MavenPayload { + m0 := &MavenPayload{} + b, x := &b0, m0 + _, _ = b, x + x.xxx_hidden_Artifact = b.Artifact + x.xxx_hidden_RepositoryUrl = b.RepositoryUrl + return m0 +} + +type DeferredArtifactPayload struct { + state protoimpl.MessageState `protogen:"opaque.v1"` + xxx_hidden_Key string `protobuf:"bytes,1,opt,name=key,proto3" json:"key,omitempty"` + xxx_hidden_Data []byte `protobuf:"bytes,2,opt,name=data,proto3" json:"data,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *DeferredArtifactPayload) Reset() { + *x = DeferredArtifactPayload{} + mi := &file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_msgTypes[47] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *DeferredArtifactPayload) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*DeferredArtifactPayload) ProtoMessage() {} + +func (x *DeferredArtifactPayload) ProtoReflect() protoreflect.Message { + mi := &file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_msgTypes[47] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +func (x *DeferredArtifactPayload) GetKey() string { + if x != nil { + return x.xxx_hidden_Key + } + return "" +} + +func (x *DeferredArtifactPayload) GetData() []byte { + if x != nil { + return x.xxx_hidden_Data + } + return nil +} + +func (x *DeferredArtifactPayload) SetKey(v string) { + x.xxx_hidden_Key = v +} + +func (x *DeferredArtifactPayload) SetData(v []byte) { + if v == nil { + v = []byte{} + } + x.xxx_hidden_Data = v +} + +type DeferredArtifactPayload_builder struct { + _ [0]func() // Prevents comparability and use of unkeyed literals for the builder. + + // A unique string identifier assigned by the creator of this payload. The creator may use this key to confirm + // whether they can parse the data. + Key string + // Data for deferred artifacts. Interpretation of bytes is delegated to the creator of this payload. + Data []byte +} + +func (b0 DeferredArtifactPayload_builder) Build() *DeferredArtifactPayload { + m0 := &DeferredArtifactPayload{} + b, x := &b0, m0 + _, _ = b, x + x.xxx_hidden_Key = b.Key + x.xxx_hidden_Data = b.Data + return m0 +} + +type ArtifactStagingToRolePayload struct { + state protoimpl.MessageState `protogen:"opaque.v1"` + xxx_hidden_StagedName string `protobuf:"bytes,1,opt,name=staged_name,json=stagedName,proto3" json:"staged_name,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *ArtifactStagingToRolePayload) Reset() { + *x = ArtifactStagingToRolePayload{} + mi := &file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_msgTypes[48] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *ArtifactStagingToRolePayload) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*ArtifactStagingToRolePayload) ProtoMessage() {} + +func (x *ArtifactStagingToRolePayload) ProtoReflect() protoreflect.Message { + mi := &file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_msgTypes[48] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +func (x *ArtifactStagingToRolePayload) GetStagedName() string { + if x != nil { + return x.xxx_hidden_StagedName + } + return "" +} + +func (x *ArtifactStagingToRolePayload) SetStagedName(v string) { + x.xxx_hidden_StagedName = v +} + +type ArtifactStagingToRolePayload_builder struct { + _ [0]func() // Prevents comparability and use of unkeyed literals for the builder. + + // A generated staged name (relative path under staging directory). + StagedName string +} + +func (b0 ArtifactStagingToRolePayload_builder) Build() *ArtifactStagingToRolePayload { + m0 := &ArtifactStagingToRolePayload{} + b, x := &b0, m0 + _, _ = b, x + x.xxx_hidden_StagedName = b.StagedName + return m0 +} + +type ArtifactInformation struct { + state protoimpl.MessageState `protogen:"opaque.v1"` + xxx_hidden_TypeUrn string `protobuf:"bytes,1,opt,name=type_urn,json=typeUrn,proto3" json:"type_urn,omitempty"` + xxx_hidden_TypePayload []byte `protobuf:"bytes,2,opt,name=type_payload,json=typePayload,proto3" json:"type_payload,omitempty"` + xxx_hidden_RoleUrn string `protobuf:"bytes,3,opt,name=role_urn,json=roleUrn,proto3" json:"role_urn,omitempty"` + xxx_hidden_RolePayload []byte `protobuf:"bytes,4,opt,name=role_payload,json=rolePayload,proto3" json:"role_payload,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *ArtifactInformation) Reset() { + *x = ArtifactInformation{} + mi := &file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_msgTypes[49] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *ArtifactInformation) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*ArtifactInformation) ProtoMessage() {} + +func (x *ArtifactInformation) ProtoReflect() protoreflect.Message { + mi := &file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_msgTypes[49] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +func (x *ArtifactInformation) GetTypeUrn() string { + if x != nil { + return x.xxx_hidden_TypeUrn + } + return "" +} + +func (x *ArtifactInformation) GetTypePayload() []byte { + if x != nil { + return x.xxx_hidden_TypePayload + } + return nil +} + +func (x *ArtifactInformation) GetRoleUrn() string { + if x != nil { + return x.xxx_hidden_RoleUrn + } + return "" +} + +func (x *ArtifactInformation) GetRolePayload() []byte { + if x != nil { + return x.xxx_hidden_RolePayload + } + return nil +} + +func (x *ArtifactInformation) SetTypeUrn(v string) { + x.xxx_hidden_TypeUrn = v +} + +func (x *ArtifactInformation) SetTypePayload(v []byte) { + if v == nil { + v = []byte{} + } + x.xxx_hidden_TypePayload = v +} + +func (x *ArtifactInformation) SetRoleUrn(v string) { + x.xxx_hidden_RoleUrn = v +} + +func (x *ArtifactInformation) SetRolePayload(v []byte) { + if v == nil { + v = []byte{} + } + x.xxx_hidden_RolePayload = v +} + +type ArtifactInformation_builder struct { + _ [0]func() // Prevents comparability and use of unkeyed literals for the builder. + + // A URN that describes the type of artifact + TypeUrn string + TypePayload []byte + // A URN that describes the role of artifact + RoleUrn string + RolePayload []byte +} + +func (b0 ArtifactInformation_builder) Build() *ArtifactInformation { + m0 := &ArtifactInformation{} + b, x := &b0, m0 + _, _ = b, x + x.xxx_hidden_TypeUrn = b.TypeUrn + x.xxx_hidden_TypePayload = b.TypePayload + x.xxx_hidden_RoleUrn = b.RoleUrn + x.xxx_hidden_RolePayload = b.RolePayload + return m0 +} + +// An environment for executing UDFs. By default, an SDK container URL, but +// can also be a process forked by a command, or an externally managed process. +type Environment struct { + state protoimpl.MessageState `protogen:"opaque.v1"` + xxx_hidden_Urn string `protobuf:"bytes,2,opt,name=urn,proto3" json:"urn,omitempty"` + xxx_hidden_Payload []byte `protobuf:"bytes,3,opt,name=payload,proto3" json:"payload,omitempty"` + xxx_hidden_DisplayData *[]*DisplayData `protobuf:"bytes,4,rep,name=display_data,json=displayData,proto3" json:"display_data,omitempty"` + xxx_hidden_Capabilities []string `protobuf:"bytes,5,rep,name=capabilities,proto3" json:"capabilities,omitempty"` + xxx_hidden_Dependencies *[]*ArtifactInformation `protobuf:"bytes,6,rep,name=dependencies,proto3" json:"dependencies,omitempty"` + xxx_hidden_ResourceHints map[string][]byte `protobuf:"bytes,7,rep,name=resource_hints,json=resourceHints,proto3" json:"resource_hints,omitempty" protobuf_key:"bytes,1,opt,name=key" protobuf_val:"bytes,2,opt,name=value"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *Environment) Reset() { + *x = Environment{} + mi := &file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_msgTypes[50] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *Environment) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*Environment) ProtoMessage() {} + +func (x *Environment) ProtoReflect() protoreflect.Message { + mi := &file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_msgTypes[50] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +func (x *Environment) GetUrn() string { + if x != nil { + return x.xxx_hidden_Urn + } + return "" +} + +func (x *Environment) GetPayload() []byte { + if x != nil { + return x.xxx_hidden_Payload + } + return nil +} + +func (x *Environment) GetDisplayData() []*DisplayData { + if x != nil { + if x.xxx_hidden_DisplayData != nil { + return *x.xxx_hidden_DisplayData + } + } + return nil +} + +func (x *Environment) GetCapabilities() []string { + if x != nil { + return x.xxx_hidden_Capabilities + } + return nil +} + +func (x *Environment) GetDependencies() []*ArtifactInformation { + if x != nil { + if x.xxx_hidden_Dependencies != nil { + return *x.xxx_hidden_Dependencies + } + } + return nil +} + +func (x *Environment) GetResourceHints() map[string][]byte { + if x != nil { + return x.xxx_hidden_ResourceHints + } + return nil +} + +func (x *Environment) SetUrn(v string) { + x.xxx_hidden_Urn = v +} + +func (x *Environment) SetPayload(v []byte) { + if v == nil { + v = []byte{} + } + x.xxx_hidden_Payload = v +} + +func (x *Environment) SetDisplayData(v []*DisplayData) { + x.xxx_hidden_DisplayData = &v +} + +func (x *Environment) SetCapabilities(v []string) { + x.xxx_hidden_Capabilities = v +} + +func (x *Environment) SetDependencies(v []*ArtifactInformation) { + x.xxx_hidden_Dependencies = &v +} + +func (x *Environment) SetResourceHints(v map[string][]byte) { + x.xxx_hidden_ResourceHints = v +} + +type Environment_builder struct { + _ [0]func() // Prevents comparability and use of unkeyed literals for the builder. + + // (Required) The URN of the payload + Urn string + // (Optional) The data specifying any parameters to the URN. If + // the URN does not require any arguments, this may be omitted. + Payload []byte + // (Optional) Static display data for the environment. If there is none, + // it may be omitted. + DisplayData []*DisplayData + // (Optional) A set of capabilities this environment supports. This is + // typically a list of common URNs designating coders, transforms, etc. that + // this environment understands (and a runner MAY use) despite not + // appearing in the pipeline proto. This may also be used to indicate + // support of optional protocols not tied to a concrete component. + Capabilities []string + // (Optional) artifact dependency information used for executing UDFs in this environment. + Dependencies []*ArtifactInformation + // (Optional) A mapping of resource URNs to requested values. The encoding + // of the values is specified by the URN. Resource hints are advisory; + // a runner is free to ignore resource hints that it does not understand. + ResourceHints map[string][]byte +} + +func (b0 Environment_builder) Build() *Environment { + m0 := &Environment{} + b, x := &b0, m0 + _, _ = b, x + x.xxx_hidden_Urn = b.Urn + x.xxx_hidden_Payload = b.Payload + x.xxx_hidden_DisplayData = &b.DisplayData + x.xxx_hidden_Capabilities = b.Capabilities + x.xxx_hidden_Dependencies = &b.Dependencies + x.xxx_hidden_ResourceHints = b.ResourceHints + return m0 +} + +type StandardEnvironments struct { + state protoimpl.MessageState `protogen:"opaque.v1"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *StandardEnvironments) Reset() { + *x = StandardEnvironments{} + mi := &file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_msgTypes[51] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *StandardEnvironments) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*StandardEnvironments) ProtoMessage() {} + +func (x *StandardEnvironments) ProtoReflect() protoreflect.Message { + mi := &file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_msgTypes[51] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +type StandardEnvironments_builder struct { + _ [0]func() // Prevents comparability and use of unkeyed literals for the builder. + +} + +func (b0 StandardEnvironments_builder) Build() *StandardEnvironments { + m0 := &StandardEnvironments{} + b, x := &b0, m0 + _, _ = b, x + return m0 +} + +// The payload of a Docker image +type DockerPayload struct { + state protoimpl.MessageState `protogen:"opaque.v1"` + xxx_hidden_ContainerImage string `protobuf:"bytes,1,opt,name=container_image,json=containerImage,proto3" json:"container_image,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *DockerPayload) Reset() { + *x = DockerPayload{} + mi := &file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_msgTypes[52] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *DockerPayload) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*DockerPayload) ProtoMessage() {} + +func (x *DockerPayload) ProtoReflect() protoreflect.Message { + mi := &file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_msgTypes[52] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +func (x *DockerPayload) GetContainerImage() string { + if x != nil { + return x.xxx_hidden_ContainerImage + } + return "" +} + +func (x *DockerPayload) SetContainerImage(v string) { + x.xxx_hidden_ContainerImage = v +} + +type DockerPayload_builder struct { + _ [0]func() // Prevents comparability and use of unkeyed literals for the builder. + + ContainerImage string +} + +func (b0 DockerPayload_builder) Build() *DockerPayload { + m0 := &DockerPayload{} + b, x := &b0, m0 + _, _ = b, x + x.xxx_hidden_ContainerImage = b.ContainerImage + return m0 +} + +type ProcessPayload struct { + state protoimpl.MessageState `protogen:"opaque.v1"` + xxx_hidden_Os string `protobuf:"bytes,1,opt,name=os,proto3" json:"os,omitempty"` + xxx_hidden_Arch string `protobuf:"bytes,2,opt,name=arch,proto3" json:"arch,omitempty"` + xxx_hidden_Command string `protobuf:"bytes,3,opt,name=command,proto3" json:"command,omitempty"` + xxx_hidden_Env map[string]string `protobuf:"bytes,4,rep,name=env,proto3" json:"env,omitempty" protobuf_key:"bytes,1,opt,name=key" protobuf_val:"bytes,2,opt,name=value"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *ProcessPayload) Reset() { + *x = ProcessPayload{} + mi := &file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_msgTypes[53] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *ProcessPayload) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*ProcessPayload) ProtoMessage() {} + +func (x *ProcessPayload) ProtoReflect() protoreflect.Message { + mi := &file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_msgTypes[53] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +func (x *ProcessPayload) GetOs() string { + if x != nil { + return x.xxx_hidden_Os + } + return "" +} + +func (x *ProcessPayload) GetArch() string { + if x != nil { + return x.xxx_hidden_Arch + } + return "" +} + +func (x *ProcessPayload) GetCommand() string { + if x != nil { + return x.xxx_hidden_Command + } + return "" +} + +func (x *ProcessPayload) GetEnv() map[string]string { + if x != nil { + return x.xxx_hidden_Env + } + return nil +} + +func (x *ProcessPayload) SetOs(v string) { + x.xxx_hidden_Os = v +} + +func (x *ProcessPayload) SetArch(v string) { + x.xxx_hidden_Arch = v +} + +func (x *ProcessPayload) SetCommand(v string) { + x.xxx_hidden_Command = v +} + +func (x *ProcessPayload) SetEnv(v map[string]string) { + x.xxx_hidden_Env = v +} + +type ProcessPayload_builder struct { + _ [0]func() // Prevents comparability and use of unkeyed literals for the builder. + + Os string + Arch string + Command string + Env map[string]string +} + +func (b0 ProcessPayload_builder) Build() *ProcessPayload { + m0 := &ProcessPayload{} + b, x := &b0, m0 + _, _ = b, x + x.xxx_hidden_Os = b.Os + x.xxx_hidden_Arch = b.Arch + x.xxx_hidden_Command = b.Command + x.xxx_hidden_Env = b.Env + return m0 +} + +type ExternalPayload struct { + state protoimpl.MessageState `protogen:"opaque.v1"` + xxx_hidden_Endpoint *ApiServiceDescriptor `protobuf:"bytes,1,opt,name=endpoint,proto3" json:"endpoint,omitempty"` + xxx_hidden_Params map[string]string `protobuf:"bytes,2,rep,name=params,proto3" json:"params,omitempty" protobuf_key:"bytes,1,opt,name=key" protobuf_val:"bytes,2,opt,name=value"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *ExternalPayload) Reset() { + *x = ExternalPayload{} + mi := &file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_msgTypes[54] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *ExternalPayload) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*ExternalPayload) ProtoMessage() {} + +func (x *ExternalPayload) ProtoReflect() protoreflect.Message { + mi := &file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_msgTypes[54] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +func (x *ExternalPayload) GetEndpoint() *ApiServiceDescriptor { + if x != nil { + return x.xxx_hidden_Endpoint + } + return nil +} + +func (x *ExternalPayload) GetParams() map[string]string { + if x != nil { + return x.xxx_hidden_Params + } + return nil +} + +func (x *ExternalPayload) SetEndpoint(v *ApiServiceDescriptor) { + x.xxx_hidden_Endpoint = v +} + +func (x *ExternalPayload) SetParams(v map[string]string) { + x.xxx_hidden_Params = v +} + +func (x *ExternalPayload) HasEndpoint() bool { + if x == nil { + return false + } + return x.xxx_hidden_Endpoint != nil +} + +func (x *ExternalPayload) ClearEndpoint() { + x.xxx_hidden_Endpoint = nil +} + +type ExternalPayload_builder struct { + _ [0]func() // Prevents comparability and use of unkeyed literals for the builder. + + Endpoint *ApiServiceDescriptor + Params map[string]string +} + +func (b0 ExternalPayload_builder) Build() *ExternalPayload { + m0 := &ExternalPayload{} + b, x := &b0, m0 + _, _ = b, x + x.xxx_hidden_Endpoint = b.Endpoint + x.xxx_hidden_Params = b.Params + return m0 +} + +type AnyOfEnvironmentPayload struct { + state protoimpl.MessageState `protogen:"opaque.v1"` + xxx_hidden_Environments *[]*Environment `protobuf:"bytes,1,rep,name=environments,proto3" json:"environments,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *AnyOfEnvironmentPayload) Reset() { + *x = AnyOfEnvironmentPayload{} + mi := &file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_msgTypes[55] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *AnyOfEnvironmentPayload) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*AnyOfEnvironmentPayload) ProtoMessage() {} + +func (x *AnyOfEnvironmentPayload) ProtoReflect() protoreflect.Message { + mi := &file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_msgTypes[55] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +func (x *AnyOfEnvironmentPayload) GetEnvironments() []*Environment { + if x != nil { + if x.xxx_hidden_Environments != nil { + return *x.xxx_hidden_Environments + } + } + return nil +} + +func (x *AnyOfEnvironmentPayload) SetEnvironments(v []*Environment) { + x.xxx_hidden_Environments = &v +} + +type AnyOfEnvironmentPayload_builder struct { + _ [0]func() // Prevents comparability and use of unkeyed literals for the builder. + + // Each is fully contained (with their own dependencies, capabilities, etc.) + Environments []*Environment +} + +func (b0 AnyOfEnvironmentPayload_builder) Build() *AnyOfEnvironmentPayload { + m0 := &AnyOfEnvironmentPayload{} + b, x := &b0, m0 + _, _ = b, x + x.xxx_hidden_Environments = &b.Environments + return m0 +} + +// These URNs are used to indicate capabilities of environments that cannot +// simply be expressed as a component (such as a Coder or PTransform) that this +// environment understands. +type StandardProtocols struct { + state protoimpl.MessageState `protogen:"opaque.v1"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *StandardProtocols) Reset() { + *x = StandardProtocols{} + mi := &file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_msgTypes[56] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *StandardProtocols) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*StandardProtocols) ProtoMessage() {} + +func (x *StandardProtocols) ProtoReflect() protoreflect.Message { + mi := &file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_msgTypes[56] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +type StandardProtocols_builder struct { + _ [0]func() // Prevents comparability and use of unkeyed literals for the builder. + +} + +func (b0 StandardProtocols_builder) Build() *StandardProtocols { + m0 := &StandardProtocols{} + b, x := &b0, m0 + _, _ = b, x + return m0 +} + +// These URNs are used to indicate capabilities of runner that an environment +// may take advantage of when interacting with this runner. +type StandardRunnerProtocols struct { + state protoimpl.MessageState `protogen:"opaque.v1"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *StandardRunnerProtocols) Reset() { + *x = StandardRunnerProtocols{} + mi := &file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_msgTypes[57] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *StandardRunnerProtocols) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*StandardRunnerProtocols) ProtoMessage() {} + +func (x *StandardRunnerProtocols) ProtoReflect() protoreflect.Message { + mi := &file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_msgTypes[57] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +type StandardRunnerProtocols_builder struct { + _ [0]func() // Prevents comparability and use of unkeyed literals for the builder. + +} + +func (b0 StandardRunnerProtocols_builder) Build() *StandardRunnerProtocols { + m0 := &StandardRunnerProtocols{} + b, x := &b0, m0 + _, _ = b, x + return m0 +} + +// These URNs are used to indicate requirements of a pipeline that cannot +// simply be expressed as a component (such as a Coder or PTransform) that the +// runner must understand. In many cases, this indicates a particular field +// of a transform must be inspected and respected (which allows new fields +// to be added in a forwards-compatible way). +type StandardRequirements struct { + state protoimpl.MessageState `protogen:"opaque.v1"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *StandardRequirements) Reset() { + *x = StandardRequirements{} + mi := &file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_msgTypes[58] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *StandardRequirements) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*StandardRequirements) ProtoMessage() {} + +func (x *StandardRequirements) ProtoReflect() protoreflect.Message { + mi := &file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_msgTypes[58] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +type StandardRequirements_builder struct { + _ [0]func() // Prevents comparability and use of unkeyed literals for the builder. + +} + +func (b0 StandardRequirements_builder) Build() *StandardRequirements { + m0 := &StandardRequirements{} + b, x := &b0, m0 + _, _ = b, x + return m0 +} + +// A URN along with a parameter object whose schema is determined by the +// URN. +// +// This structure is reused in two distinct, but compatible, ways: +// +// 1. This can be a specification of the function over PCollections +// that a PTransform computes. +// 2. This can be a specification of a user-defined function, possibly +// SDK-specific. (external to this message must be adequate context +// to indicate the environment in which the UDF can be understood). +// +// Though not explicit in this proto, there are two possibilities +// for the relationship of a runner to this specification that +// one should bear in mind: +// +// 1. The runner understands the URN. For example, it might be +// a well-known URN like "beam:transform:Top" or +// "beam:window_fn:FixedWindows" with +// an agreed-upon payload (e.g. a number or duration, +// respectively). +// 2. The runner does not understand the URN. It might be an +// SDK specific URN such as "beam:dofn:javasdk:1.0" +// that indicates to the SDK what the payload is, +// such as a serialized Java DoFn from a particular +// version of the Beam Java SDK. The payload will often +// then be an opaque message such as bytes in a +// language-specific serialization format. +type FunctionSpec struct { + state protoimpl.MessageState `protogen:"opaque.v1"` + xxx_hidden_Urn string `protobuf:"bytes,1,opt,name=urn,proto3" json:"urn,omitempty"` + xxx_hidden_Payload []byte `protobuf:"bytes,3,opt,name=payload,proto3" json:"payload,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *FunctionSpec) Reset() { + *x = FunctionSpec{} + mi := &file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_msgTypes[59] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *FunctionSpec) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*FunctionSpec) ProtoMessage() {} + +func (x *FunctionSpec) ProtoReflect() protoreflect.Message { + mi := &file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_msgTypes[59] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +func (x *FunctionSpec) GetUrn() string { + if x != nil { + return x.xxx_hidden_Urn + } + return "" +} + +func (x *FunctionSpec) GetPayload() []byte { + if x != nil { + return x.xxx_hidden_Payload + } + return nil +} + +func (x *FunctionSpec) SetUrn(v string) { + x.xxx_hidden_Urn = v +} + +func (x *FunctionSpec) SetPayload(v []byte) { + if v == nil { + v = []byte{} + } + x.xxx_hidden_Payload = v +} + +type FunctionSpec_builder struct { + _ [0]func() // Prevents comparability and use of unkeyed literals for the builder. + + // (Required) A URN that describes the accompanying payload. + // For any URN that is not recognized (by whomever is inspecting + // it) the parameter payload should be treated as opaque and + // passed as-is. + Urn string + // (Optional) The data specifying any parameters to the URN. If + // the URN does not require any arguments, this may be omitted. + Payload []byte +} + +func (b0 FunctionSpec_builder) Build() *FunctionSpec { + m0 := &FunctionSpec{} + b, x := &b0, m0 + _, _ = b, x + x.xxx_hidden_Urn = b.Urn + x.xxx_hidden_Payload = b.Payload + return m0 +} + +// A set of well known URNs describing display data. +// +// All descriptions must contain how the value should be classified and how it +// is encoded. Note that some types are logical types which convey contextual +// information about the pipeline in addition to an encoding while others only +// specify the encoding itself. +type StandardDisplayData struct { + state protoimpl.MessageState `protogen:"opaque.v1"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *StandardDisplayData) Reset() { + *x = StandardDisplayData{} + mi := &file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_msgTypes[60] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *StandardDisplayData) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*StandardDisplayData) ProtoMessage() {} + +func (x *StandardDisplayData) ProtoReflect() protoreflect.Message { + mi := &file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_msgTypes[60] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +type StandardDisplayData_builder struct { + _ [0]func() // Prevents comparability and use of unkeyed literals for the builder. + +} + +func (b0 StandardDisplayData_builder) Build() *StandardDisplayData { + m0 := &StandardDisplayData{} + b, x := &b0, m0 + _, _ = b, x + return m0 +} + +type LabelledPayload struct { + state protoimpl.MessageState `protogen:"opaque.v1"` + xxx_hidden_Label string `protobuf:"bytes,1,opt,name=label,proto3" json:"label,omitempty"` + xxx_hidden_Value isLabelledPayload_Value `protobuf_oneof:"value"` + xxx_hidden_Key string `protobuf:"bytes,6,opt,name=key,proto3" json:"key,omitempty"` + xxx_hidden_Namespace string `protobuf:"bytes,7,opt,name=namespace,proto3" json:"namespace,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *LabelledPayload) Reset() { + *x = LabelledPayload{} + mi := &file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_msgTypes[61] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *LabelledPayload) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*LabelledPayload) ProtoMessage() {} + +func (x *LabelledPayload) ProtoReflect() protoreflect.Message { + mi := &file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_msgTypes[61] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +func (x *LabelledPayload) GetLabel() string { + if x != nil { + return x.xxx_hidden_Label + } + return "" +} + +func (x *LabelledPayload) GetStringValue() string { + if x != nil { + if x, ok := x.xxx_hidden_Value.(*labelledPayload_StringValue); ok { + return x.StringValue + } + } + return "" +} + +func (x *LabelledPayload) GetBoolValue() bool { + if x != nil { + if x, ok := x.xxx_hidden_Value.(*labelledPayload_BoolValue); ok { + return x.BoolValue + } + } + return false +} + +func (x *LabelledPayload) GetDoubleValue() float64 { + if x != nil { + if x, ok := x.xxx_hidden_Value.(*labelledPayload_DoubleValue); ok { + return x.DoubleValue + } + } + return 0 +} + +func (x *LabelledPayload) GetIntValue() int64 { + if x != nil { + if x, ok := x.xxx_hidden_Value.(*labelledPayload_IntValue); ok { + return x.IntValue + } + } + return 0 +} + +func (x *LabelledPayload) GetKey() string { + if x != nil { + return x.xxx_hidden_Key + } + return "" +} + +func (x *LabelledPayload) GetNamespace() string { + if x != nil { + return x.xxx_hidden_Namespace + } + return "" +} + +func (x *LabelledPayload) SetLabel(v string) { + x.xxx_hidden_Label = v +} + +func (x *LabelledPayload) SetStringValue(v string) { + x.xxx_hidden_Value = &labelledPayload_StringValue{v} +} + +func (x *LabelledPayload) SetBoolValue(v bool) { + x.xxx_hidden_Value = &labelledPayload_BoolValue{v} +} + +func (x *LabelledPayload) SetDoubleValue(v float64) { + x.xxx_hidden_Value = &labelledPayload_DoubleValue{v} +} + +func (x *LabelledPayload) SetIntValue(v int64) { + x.xxx_hidden_Value = &labelledPayload_IntValue{v} +} + +func (x *LabelledPayload) SetKey(v string) { + x.xxx_hidden_Key = v +} + +func (x *LabelledPayload) SetNamespace(v string) { + x.xxx_hidden_Namespace = v +} + +func (x *LabelledPayload) HasValue() bool { + if x == nil { + return false + } + return x.xxx_hidden_Value != nil +} + +func (x *LabelledPayload) HasStringValue() bool { + if x == nil { + return false + } + _, ok := x.xxx_hidden_Value.(*labelledPayload_StringValue) + return ok +} + +func (x *LabelledPayload) HasBoolValue() bool { + if x == nil { + return false + } + _, ok := x.xxx_hidden_Value.(*labelledPayload_BoolValue) + return ok +} + +func (x *LabelledPayload) HasDoubleValue() bool { + if x == nil { + return false + } + _, ok := x.xxx_hidden_Value.(*labelledPayload_DoubleValue) + return ok +} + +func (x *LabelledPayload) HasIntValue() bool { + if x == nil { + return false + } + _, ok := x.xxx_hidden_Value.(*labelledPayload_IntValue) + return ok +} + +func (x *LabelledPayload) ClearValue() { + x.xxx_hidden_Value = nil +} + +func (x *LabelledPayload) ClearStringValue() { + if _, ok := x.xxx_hidden_Value.(*labelledPayload_StringValue); ok { + x.xxx_hidden_Value = nil + } +} + +func (x *LabelledPayload) ClearBoolValue() { + if _, ok := x.xxx_hidden_Value.(*labelledPayload_BoolValue); ok { + x.xxx_hidden_Value = nil + } +} + +func (x *LabelledPayload) ClearDoubleValue() { + if _, ok := x.xxx_hidden_Value.(*labelledPayload_DoubleValue); ok { + x.xxx_hidden_Value = nil + } +} + +func (x *LabelledPayload) ClearIntValue() { + if _, ok := x.xxx_hidden_Value.(*labelledPayload_IntValue); ok { + x.xxx_hidden_Value = nil + } +} + +const LabelledPayload_Value_not_set_case case_LabelledPayload_Value = 0 +const LabelledPayload_StringValue_case case_LabelledPayload_Value = 2 +const LabelledPayload_BoolValue_case case_LabelledPayload_Value = 3 +const LabelledPayload_DoubleValue_case case_LabelledPayload_Value = 4 +const LabelledPayload_IntValue_case case_LabelledPayload_Value = 5 + +func (x *LabelledPayload) WhichValue() case_LabelledPayload_Value { + if x == nil { + return LabelledPayload_Value_not_set_case + } + switch x.xxx_hidden_Value.(type) { + case *labelledPayload_StringValue: + return LabelledPayload_StringValue_case + case *labelledPayload_BoolValue: + return LabelledPayload_BoolValue_case + case *labelledPayload_DoubleValue: + return LabelledPayload_DoubleValue_case + case *labelledPayload_IntValue: + return LabelledPayload_IntValue_case + default: + return LabelledPayload_Value_not_set_case + } +} + +type LabelledPayload_builder struct { + _ [0]func() // Prevents comparability and use of unkeyed literals for the builder. + + // (Required) A human readable label for the value. + Label string + // (Required) A value which will be displayed to the user. + + // Fields of oneof xxx_hidden_Value: + StringValue *string + BoolValue *bool + DoubleValue *float64 + IntValue *int64 + // -- end of xxx_hidden_Value + // (Required) The key identifies the actual content of the metadata. + Key string + // (Required) The namespace describes the context that specified the key. + Namespace string +} + +func (b0 LabelledPayload_builder) Build() *LabelledPayload { + m0 := &LabelledPayload{} + b, x := &b0, m0 + _, _ = b, x + x.xxx_hidden_Label = b.Label + if b.StringValue != nil { + x.xxx_hidden_Value = &labelledPayload_StringValue{*b.StringValue} + } + if b.BoolValue != nil { + x.xxx_hidden_Value = &labelledPayload_BoolValue{*b.BoolValue} + } + if b.DoubleValue != nil { + x.xxx_hidden_Value = &labelledPayload_DoubleValue{*b.DoubleValue} + } + if b.IntValue != nil { + x.xxx_hidden_Value = &labelledPayload_IntValue{*b.IntValue} + } + x.xxx_hidden_Key = b.Key + x.xxx_hidden_Namespace = b.Namespace + return m0 +} + +type case_LabelledPayload_Value protoreflect.FieldNumber + +func (x case_LabelledPayload_Value) String() string { + md := file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_msgTypes[61].Descriptor() + if x == 0 { + return "not set" + } + return protoimpl.X.MessageFieldStringOf(md, protoreflect.FieldNumber(x)) +} + +type isLabelledPayload_Value interface { + isLabelledPayload_Value() +} + +type labelledPayload_StringValue struct { + StringValue string `protobuf:"bytes,2,opt,name=string_value,json=stringValue,proto3,oneof"` +} + +type labelledPayload_BoolValue struct { + BoolValue bool `protobuf:"varint,3,opt,name=bool_value,json=boolValue,proto3,oneof"` +} + +type labelledPayload_DoubleValue struct { + DoubleValue float64 `protobuf:"fixed64,4,opt,name=double_value,json=doubleValue,proto3,oneof"` +} + +type labelledPayload_IntValue struct { + IntValue int64 `protobuf:"varint,5,opt,name=int_value,json=intValue,proto3,oneof"` +} + +func (*labelledPayload_StringValue) isLabelledPayload_Value() {} + +func (*labelledPayload_BoolValue) isLabelledPayload_Value() {} + +func (*labelledPayload_DoubleValue) isLabelledPayload_Value() {} + +func (*labelledPayload_IntValue) isLabelledPayload_Value() {} + +// Static display data associated with a pipeline component. Display data is +// useful for pipeline runners IOs and diagnostic dashboards to display details +// about annotated components. +type DisplayData struct { + state protoimpl.MessageState `protogen:"opaque.v1"` + xxx_hidden_Urn string `protobuf:"bytes,1,opt,name=urn,proto3" json:"urn,omitempty"` + xxx_hidden_Payload []byte `protobuf:"bytes,2,opt,name=payload,proto3" json:"payload,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *DisplayData) Reset() { + *x = DisplayData{} + mi := &file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_msgTypes[62] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *DisplayData) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*DisplayData) ProtoMessage() {} + +func (x *DisplayData) ProtoReflect() protoreflect.Message { + mi := &file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_msgTypes[62] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +func (x *DisplayData) GetUrn() string { + if x != nil { + return x.xxx_hidden_Urn + } + return "" +} + +func (x *DisplayData) GetPayload() []byte { + if x != nil { + return x.xxx_hidden_Payload + } + return nil +} + +func (x *DisplayData) SetUrn(v string) { + x.xxx_hidden_Urn = v +} + +func (x *DisplayData) SetPayload(v []byte) { + if v == nil { + v = []byte{} + } + x.xxx_hidden_Payload = v +} + +type DisplayData_builder struct { + _ [0]func() // Prevents comparability and use of unkeyed literals for the builder. + + // A key used to describe the type of display data. See StandardDisplayData + // for the set of well known urns describing how the payload is meant to be + // interpreted. + Urn string + // (Optional) The data specifying any parameters to the URN. If + // the URN does not require any arguments, this may be omitted. + Payload []byte +} + +func (b0 DisplayData_builder) Build() *DisplayData { + m0 := &DisplayData{} + b, x := &b0, m0 + _, _ = b, x + x.xxx_hidden_Urn = b.Urn + x.xxx_hidden_Payload = b.Payload + return m0 +} + +// A disjoint union of all the things that may contain references +// that require Components to resolve. +type MessageWithComponents struct { + state protoimpl.MessageState `protogen:"opaque.v1"` + xxx_hidden_Components *Components `protobuf:"bytes,1,opt,name=components,proto3" json:"components,omitempty"` + xxx_hidden_Root isMessageWithComponents_Root `protobuf_oneof:"root"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *MessageWithComponents) Reset() { + *x = MessageWithComponents{} + mi := &file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_msgTypes[63] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *MessageWithComponents) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*MessageWithComponents) ProtoMessage() {} + +func (x *MessageWithComponents) ProtoReflect() protoreflect.Message { + mi := &file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_msgTypes[63] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +func (x *MessageWithComponents) GetComponents() *Components { + if x != nil { + return x.xxx_hidden_Components + } + return nil +} + +func (x *MessageWithComponents) GetCoder() *Coder { + if x != nil { + if x, ok := x.xxx_hidden_Root.(*messageWithComponents_Coder); ok { + return x.Coder + } + } + return nil +} + +func (x *MessageWithComponents) GetCombinePayload() *CombinePayload { + if x != nil { + if x, ok := x.xxx_hidden_Root.(*messageWithComponents_CombinePayload); ok { + return x.CombinePayload + } + } + return nil +} + +func (x *MessageWithComponents) GetFunctionSpec() *FunctionSpec { + if x != nil { + if x, ok := x.xxx_hidden_Root.(*messageWithComponents_FunctionSpec); ok { + return x.FunctionSpec + } + } + return nil +} + +func (x *MessageWithComponents) GetParDoPayload() *ParDoPayload { + if x != nil { + if x, ok := x.xxx_hidden_Root.(*messageWithComponents_ParDoPayload); ok { + return x.ParDoPayload + } + } + return nil +} + +func (x *MessageWithComponents) GetPtransform() *PTransform { + if x != nil { + if x, ok := x.xxx_hidden_Root.(*messageWithComponents_Ptransform); ok { + return x.Ptransform + } + } + return nil +} + +func (x *MessageWithComponents) GetPcollection() *PCollection { + if x != nil { + if x, ok := x.xxx_hidden_Root.(*messageWithComponents_Pcollection); ok { + return x.Pcollection + } + } + return nil +} + +func (x *MessageWithComponents) GetReadPayload() *ReadPayload { + if x != nil { + if x, ok := x.xxx_hidden_Root.(*messageWithComponents_ReadPayload); ok { + return x.ReadPayload + } + } + return nil +} + +func (x *MessageWithComponents) GetSideInput() *SideInput { + if x != nil { + if x, ok := x.xxx_hidden_Root.(*messageWithComponents_SideInput); ok { + return x.SideInput + } + } + return nil +} + +func (x *MessageWithComponents) GetWindowIntoPayload() *WindowIntoPayload { + if x != nil { + if x, ok := x.xxx_hidden_Root.(*messageWithComponents_WindowIntoPayload); ok { + return x.WindowIntoPayload + } + } + return nil +} + +func (x *MessageWithComponents) GetWindowingStrategy() *WindowingStrategy { + if x != nil { + if x, ok := x.xxx_hidden_Root.(*messageWithComponents_WindowingStrategy); ok { + return x.WindowingStrategy + } + } + return nil +} + +func (x *MessageWithComponents) SetComponents(v *Components) { + x.xxx_hidden_Components = v +} + +func (x *MessageWithComponents) SetCoder(v *Coder) { + if v == nil { + x.xxx_hidden_Root = nil + return + } + x.xxx_hidden_Root = &messageWithComponents_Coder{v} +} + +func (x *MessageWithComponents) SetCombinePayload(v *CombinePayload) { + if v == nil { + x.xxx_hidden_Root = nil + return + } + x.xxx_hidden_Root = &messageWithComponents_CombinePayload{v} +} + +func (x *MessageWithComponents) SetFunctionSpec(v *FunctionSpec) { + if v == nil { + x.xxx_hidden_Root = nil + return + } + x.xxx_hidden_Root = &messageWithComponents_FunctionSpec{v} +} + +func (x *MessageWithComponents) SetParDoPayload(v *ParDoPayload) { + if v == nil { + x.xxx_hidden_Root = nil + return + } + x.xxx_hidden_Root = &messageWithComponents_ParDoPayload{v} +} + +func (x *MessageWithComponents) SetPtransform(v *PTransform) { + if v == nil { + x.xxx_hidden_Root = nil + return + } + x.xxx_hidden_Root = &messageWithComponents_Ptransform{v} +} + +func (x *MessageWithComponents) SetPcollection(v *PCollection) { + if v == nil { + x.xxx_hidden_Root = nil + return + } + x.xxx_hidden_Root = &messageWithComponents_Pcollection{v} +} + +func (x *MessageWithComponents) SetReadPayload(v *ReadPayload) { + if v == nil { + x.xxx_hidden_Root = nil + return + } + x.xxx_hidden_Root = &messageWithComponents_ReadPayload{v} +} + +func (x *MessageWithComponents) SetSideInput(v *SideInput) { + if v == nil { + x.xxx_hidden_Root = nil + return + } + x.xxx_hidden_Root = &messageWithComponents_SideInput{v} +} + +func (x *MessageWithComponents) SetWindowIntoPayload(v *WindowIntoPayload) { + if v == nil { + x.xxx_hidden_Root = nil + return + } + x.xxx_hidden_Root = &messageWithComponents_WindowIntoPayload{v} +} + +func (x *MessageWithComponents) SetWindowingStrategy(v *WindowingStrategy) { + if v == nil { + x.xxx_hidden_Root = nil + return + } + x.xxx_hidden_Root = &messageWithComponents_WindowingStrategy{v} +} + +func (x *MessageWithComponents) HasComponents() bool { + if x == nil { + return false + } + return x.xxx_hidden_Components != nil +} + +func (x *MessageWithComponents) HasRoot() bool { + if x == nil { + return false + } + return x.xxx_hidden_Root != nil +} + +func (x *MessageWithComponents) HasCoder() bool { + if x == nil { + return false + } + _, ok := x.xxx_hidden_Root.(*messageWithComponents_Coder) + return ok +} + +func (x *MessageWithComponents) HasCombinePayload() bool { + if x == nil { + return false + } + _, ok := x.xxx_hidden_Root.(*messageWithComponents_CombinePayload) + return ok +} + +func (x *MessageWithComponents) HasFunctionSpec() bool { + if x == nil { + return false + } + _, ok := x.xxx_hidden_Root.(*messageWithComponents_FunctionSpec) + return ok +} + +func (x *MessageWithComponents) HasParDoPayload() bool { + if x == nil { + return false + } + _, ok := x.xxx_hidden_Root.(*messageWithComponents_ParDoPayload) + return ok +} + +func (x *MessageWithComponents) HasPtransform() bool { + if x == nil { + return false + } + _, ok := x.xxx_hidden_Root.(*messageWithComponents_Ptransform) + return ok +} + +func (x *MessageWithComponents) HasPcollection() bool { + if x == nil { + return false + } + _, ok := x.xxx_hidden_Root.(*messageWithComponents_Pcollection) + return ok +} + +func (x *MessageWithComponents) HasReadPayload() bool { + if x == nil { + return false + } + _, ok := x.xxx_hidden_Root.(*messageWithComponents_ReadPayload) + return ok +} + +func (x *MessageWithComponents) HasSideInput() bool { + if x == nil { + return false + } + _, ok := x.xxx_hidden_Root.(*messageWithComponents_SideInput) + return ok +} + +func (x *MessageWithComponents) HasWindowIntoPayload() bool { + if x == nil { + return false + } + _, ok := x.xxx_hidden_Root.(*messageWithComponents_WindowIntoPayload) + return ok +} + +func (x *MessageWithComponents) HasWindowingStrategy() bool { + if x == nil { + return false + } + _, ok := x.xxx_hidden_Root.(*messageWithComponents_WindowingStrategy) + return ok +} + +func (x *MessageWithComponents) ClearComponents() { + x.xxx_hidden_Components = nil +} + +func (x *MessageWithComponents) ClearRoot() { + x.xxx_hidden_Root = nil +} + +func (x *MessageWithComponents) ClearCoder() { + if _, ok := x.xxx_hidden_Root.(*messageWithComponents_Coder); ok { + x.xxx_hidden_Root = nil + } +} + +func (x *MessageWithComponents) ClearCombinePayload() { + if _, ok := x.xxx_hidden_Root.(*messageWithComponents_CombinePayload); ok { + x.xxx_hidden_Root = nil + } +} + +func (x *MessageWithComponents) ClearFunctionSpec() { + if _, ok := x.xxx_hidden_Root.(*messageWithComponents_FunctionSpec); ok { + x.xxx_hidden_Root = nil + } +} + +func (x *MessageWithComponents) ClearParDoPayload() { + if _, ok := x.xxx_hidden_Root.(*messageWithComponents_ParDoPayload); ok { + x.xxx_hidden_Root = nil + } +} + +func (x *MessageWithComponents) ClearPtransform() { + if _, ok := x.xxx_hidden_Root.(*messageWithComponents_Ptransform); ok { + x.xxx_hidden_Root = nil + } +} + +func (x *MessageWithComponents) ClearPcollection() { + if _, ok := x.xxx_hidden_Root.(*messageWithComponents_Pcollection); ok { + x.xxx_hidden_Root = nil + } +} + +func (x *MessageWithComponents) ClearReadPayload() { + if _, ok := x.xxx_hidden_Root.(*messageWithComponents_ReadPayload); ok { + x.xxx_hidden_Root = nil + } +} + +func (x *MessageWithComponents) ClearSideInput() { + if _, ok := x.xxx_hidden_Root.(*messageWithComponents_SideInput); ok { + x.xxx_hidden_Root = nil + } +} + +func (x *MessageWithComponents) ClearWindowIntoPayload() { + if _, ok := x.xxx_hidden_Root.(*messageWithComponents_WindowIntoPayload); ok { + x.xxx_hidden_Root = nil + } +} + +func (x *MessageWithComponents) ClearWindowingStrategy() { + if _, ok := x.xxx_hidden_Root.(*messageWithComponents_WindowingStrategy); ok { + x.xxx_hidden_Root = nil + } +} + +const MessageWithComponents_Root_not_set_case case_MessageWithComponents_Root = 0 +const MessageWithComponents_Coder_case case_MessageWithComponents_Root = 2 +const MessageWithComponents_CombinePayload_case case_MessageWithComponents_Root = 3 +const MessageWithComponents_FunctionSpec_case case_MessageWithComponents_Root = 4 +const MessageWithComponents_ParDoPayload_case case_MessageWithComponents_Root = 6 +const MessageWithComponents_Ptransform_case case_MessageWithComponents_Root = 7 +const MessageWithComponents_Pcollection_case case_MessageWithComponents_Root = 8 +const MessageWithComponents_ReadPayload_case case_MessageWithComponents_Root = 9 +const MessageWithComponents_SideInput_case case_MessageWithComponents_Root = 11 +const MessageWithComponents_WindowIntoPayload_case case_MessageWithComponents_Root = 12 +const MessageWithComponents_WindowingStrategy_case case_MessageWithComponents_Root = 13 + +func (x *MessageWithComponents) WhichRoot() case_MessageWithComponents_Root { + if x == nil { + return MessageWithComponents_Root_not_set_case + } + switch x.xxx_hidden_Root.(type) { + case *messageWithComponents_Coder: + return MessageWithComponents_Coder_case + case *messageWithComponents_CombinePayload: + return MessageWithComponents_CombinePayload_case + case *messageWithComponents_FunctionSpec: + return MessageWithComponents_FunctionSpec_case + case *messageWithComponents_ParDoPayload: + return MessageWithComponents_ParDoPayload_case + case *messageWithComponents_Ptransform: + return MessageWithComponents_Ptransform_case + case *messageWithComponents_Pcollection: + return MessageWithComponents_Pcollection_case + case *messageWithComponents_ReadPayload: + return MessageWithComponents_ReadPayload_case + case *messageWithComponents_SideInput: + return MessageWithComponents_SideInput_case + case *messageWithComponents_WindowIntoPayload: + return MessageWithComponents_WindowIntoPayload_case + case *messageWithComponents_WindowingStrategy: + return MessageWithComponents_WindowingStrategy_case + default: + return MessageWithComponents_Root_not_set_case + } +} + +type MessageWithComponents_builder struct { + _ [0]func() // Prevents comparability and use of unkeyed literals for the builder. + + // (Optional) The by-reference components of the root message, + // enabling a standalone message. + // + // If this is absent, it is expected that there are no + // references. + Components *Components + // (Required) The root message that may contain pointers + // that should be resolved by looking inside components. + + // Fields of oneof xxx_hidden_Root: + Coder *Coder + CombinePayload *CombinePayload + FunctionSpec *FunctionSpec + ParDoPayload *ParDoPayload + Ptransform *PTransform + Pcollection *PCollection + ReadPayload *ReadPayload + SideInput *SideInput + WindowIntoPayload *WindowIntoPayload + WindowingStrategy *WindowingStrategy + // -- end of xxx_hidden_Root +} + +func (b0 MessageWithComponents_builder) Build() *MessageWithComponents { + m0 := &MessageWithComponents{} + b, x := &b0, m0 + _, _ = b, x + x.xxx_hidden_Components = b.Components + if b.Coder != nil { + x.xxx_hidden_Root = &messageWithComponents_Coder{b.Coder} + } + if b.CombinePayload != nil { + x.xxx_hidden_Root = &messageWithComponents_CombinePayload{b.CombinePayload} + } + if b.FunctionSpec != nil { + x.xxx_hidden_Root = &messageWithComponents_FunctionSpec{b.FunctionSpec} + } + if b.ParDoPayload != nil { + x.xxx_hidden_Root = &messageWithComponents_ParDoPayload{b.ParDoPayload} + } + if b.Ptransform != nil { + x.xxx_hidden_Root = &messageWithComponents_Ptransform{b.Ptransform} + } + if b.Pcollection != nil { + x.xxx_hidden_Root = &messageWithComponents_Pcollection{b.Pcollection} + } + if b.ReadPayload != nil { + x.xxx_hidden_Root = &messageWithComponents_ReadPayload{b.ReadPayload} + } + if b.SideInput != nil { + x.xxx_hidden_Root = &messageWithComponents_SideInput{b.SideInput} + } + if b.WindowIntoPayload != nil { + x.xxx_hidden_Root = &messageWithComponents_WindowIntoPayload{b.WindowIntoPayload} + } + if b.WindowingStrategy != nil { + x.xxx_hidden_Root = &messageWithComponents_WindowingStrategy{b.WindowingStrategy} + } + return m0 +} + +type case_MessageWithComponents_Root protoreflect.FieldNumber + +func (x case_MessageWithComponents_Root) String() string { + md := file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_msgTypes[63].Descriptor() + if x == 0 { + return "not set" + } + return protoimpl.X.MessageFieldStringOf(md, protoreflect.FieldNumber(x)) +} + +type isMessageWithComponents_Root interface { + isMessageWithComponents_Root() +} + +type messageWithComponents_Coder struct { + Coder *Coder `protobuf:"bytes,2,opt,name=coder,proto3,oneof"` +} + +type messageWithComponents_CombinePayload struct { + CombinePayload *CombinePayload `protobuf:"bytes,3,opt,name=combine_payload,json=combinePayload,proto3,oneof"` +} + +type messageWithComponents_FunctionSpec struct { + FunctionSpec *FunctionSpec `protobuf:"bytes,4,opt,name=function_spec,json=functionSpec,proto3,oneof"` +} + +type messageWithComponents_ParDoPayload struct { + ParDoPayload *ParDoPayload `protobuf:"bytes,6,opt,name=par_do_payload,json=parDoPayload,proto3,oneof"` +} + +type messageWithComponents_Ptransform struct { + Ptransform *PTransform `protobuf:"bytes,7,opt,name=ptransform,proto3,oneof"` +} + +type messageWithComponents_Pcollection struct { + Pcollection *PCollection `protobuf:"bytes,8,opt,name=pcollection,proto3,oneof"` +} + +type messageWithComponents_ReadPayload struct { + ReadPayload *ReadPayload `protobuf:"bytes,9,opt,name=read_payload,json=readPayload,proto3,oneof"` +} + +type messageWithComponents_SideInput struct { + SideInput *SideInput `protobuf:"bytes,11,opt,name=side_input,json=sideInput,proto3,oneof"` +} + +type messageWithComponents_WindowIntoPayload struct { + WindowIntoPayload *WindowIntoPayload `protobuf:"bytes,12,opt,name=window_into_payload,json=windowIntoPayload,proto3,oneof"` +} + +type messageWithComponents_WindowingStrategy struct { + WindowingStrategy *WindowingStrategy `protobuf:"bytes,13,opt,name=windowing_strategy,json=windowingStrategy,proto3,oneof"` +} + +func (*messageWithComponents_Coder) isMessageWithComponents_Root() {} + +func (*messageWithComponents_CombinePayload) isMessageWithComponents_Root() {} + +func (*messageWithComponents_FunctionSpec) isMessageWithComponents_Root() {} + +func (*messageWithComponents_ParDoPayload) isMessageWithComponents_Root() {} + +func (*messageWithComponents_Ptransform) isMessageWithComponents_Root() {} + +func (*messageWithComponents_Pcollection) isMessageWithComponents_Root() {} + +func (*messageWithComponents_ReadPayload) isMessageWithComponents_Root() {} + +func (*messageWithComponents_SideInput) isMessageWithComponents_Root() {} + +func (*messageWithComponents_WindowIntoPayload) isMessageWithComponents_Root() {} + +func (*messageWithComponents_WindowingStrategy) isMessageWithComponents_Root() {} + +// The payload for an executable stage. This will eventually be passed to an SDK in the form of a +// ProcessBundleDescriptor. +type ExecutableStagePayload struct { + state protoimpl.MessageState `protogen:"opaque.v1"` + xxx_hidden_Environment *Environment `protobuf:"bytes,1,opt,name=environment,proto3" json:"environment,omitempty"` + xxx_hidden_WireCoderSettings *[]*ExecutableStagePayload_WireCoderSetting `protobuf:"bytes,9,rep,name=wire_coder_settings,json=wireCoderSettings,proto3" json:"wire_coder_settings,omitempty"` + xxx_hidden_Input string `protobuf:"bytes,2,opt,name=input,proto3" json:"input,omitempty"` + xxx_hidden_SideInputs *[]*ExecutableStagePayload_SideInputId `protobuf:"bytes,3,rep,name=side_inputs,json=sideInputs,proto3" json:"side_inputs,omitempty"` + xxx_hidden_Transforms []string `protobuf:"bytes,4,rep,name=transforms,proto3" json:"transforms,omitempty"` + xxx_hidden_Outputs []string `protobuf:"bytes,5,rep,name=outputs,proto3" json:"outputs,omitempty"` + xxx_hidden_Components *Components `protobuf:"bytes,6,opt,name=components,proto3" json:"components,omitempty"` + xxx_hidden_UserStates *[]*ExecutableStagePayload_UserStateId `protobuf:"bytes,7,rep,name=user_states,json=userStates,proto3" json:"user_states,omitempty"` + xxx_hidden_Timers *[]*ExecutableStagePayload_TimerId `protobuf:"bytes,8,rep,name=timers,proto3" json:"timers,omitempty"` + xxx_hidden_TimerFamilies *[]*ExecutableStagePayload_TimerFamilyId `protobuf:"bytes,10,rep,name=timerFamilies,proto3" json:"timerFamilies,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *ExecutableStagePayload) Reset() { + *x = ExecutableStagePayload{} + mi := &file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_msgTypes[64] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *ExecutableStagePayload) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*ExecutableStagePayload) ProtoMessage() {} + +func (x *ExecutableStagePayload) ProtoReflect() protoreflect.Message { + mi := &file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_msgTypes[64] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +func (x *ExecutableStagePayload) GetEnvironment() *Environment { + if x != nil { + return x.xxx_hidden_Environment + } + return nil +} + +func (x *ExecutableStagePayload) GetWireCoderSettings() []*ExecutableStagePayload_WireCoderSetting { + if x != nil { + if x.xxx_hidden_WireCoderSettings != nil { + return *x.xxx_hidden_WireCoderSettings + } + } + return nil +} + +func (x *ExecutableStagePayload) GetInput() string { + if x != nil { + return x.xxx_hidden_Input + } + return "" +} + +func (x *ExecutableStagePayload) GetSideInputs() []*ExecutableStagePayload_SideInputId { + if x != nil { + if x.xxx_hidden_SideInputs != nil { + return *x.xxx_hidden_SideInputs + } + } + return nil +} + +func (x *ExecutableStagePayload) GetTransforms() []string { + if x != nil { + return x.xxx_hidden_Transforms + } + return nil +} + +func (x *ExecutableStagePayload) GetOutputs() []string { + if x != nil { + return x.xxx_hidden_Outputs + } + return nil +} + +func (x *ExecutableStagePayload) GetComponents() *Components { + if x != nil { + return x.xxx_hidden_Components + } + return nil +} + +func (x *ExecutableStagePayload) GetUserStates() []*ExecutableStagePayload_UserStateId { + if x != nil { + if x.xxx_hidden_UserStates != nil { + return *x.xxx_hidden_UserStates + } + } + return nil +} + +func (x *ExecutableStagePayload) GetTimers() []*ExecutableStagePayload_TimerId { + if x != nil { + if x.xxx_hidden_Timers != nil { + return *x.xxx_hidden_Timers + } + } + return nil +} + +func (x *ExecutableStagePayload) GetTimerFamilies() []*ExecutableStagePayload_TimerFamilyId { + if x != nil { + if x.xxx_hidden_TimerFamilies != nil { + return *x.xxx_hidden_TimerFamilies + } + } + return nil +} + +func (x *ExecutableStagePayload) SetEnvironment(v *Environment) { + x.xxx_hidden_Environment = v +} + +func (x *ExecutableStagePayload) SetWireCoderSettings(v []*ExecutableStagePayload_WireCoderSetting) { + x.xxx_hidden_WireCoderSettings = &v +} + +func (x *ExecutableStagePayload) SetInput(v string) { + x.xxx_hidden_Input = v +} + +func (x *ExecutableStagePayload) SetSideInputs(v []*ExecutableStagePayload_SideInputId) { + x.xxx_hidden_SideInputs = &v +} + +func (x *ExecutableStagePayload) SetTransforms(v []string) { + x.xxx_hidden_Transforms = v +} + +func (x *ExecutableStagePayload) SetOutputs(v []string) { + x.xxx_hidden_Outputs = v +} + +func (x *ExecutableStagePayload) SetComponents(v *Components) { + x.xxx_hidden_Components = v +} + +func (x *ExecutableStagePayload) SetUserStates(v []*ExecutableStagePayload_UserStateId) { + x.xxx_hidden_UserStates = &v +} + +func (x *ExecutableStagePayload) SetTimers(v []*ExecutableStagePayload_TimerId) { + x.xxx_hidden_Timers = &v +} + +func (x *ExecutableStagePayload) SetTimerFamilies(v []*ExecutableStagePayload_TimerFamilyId) { + x.xxx_hidden_TimerFamilies = &v +} + +func (x *ExecutableStagePayload) HasEnvironment() bool { + if x == nil { + return false + } + return x.xxx_hidden_Environment != nil +} + +func (x *ExecutableStagePayload) HasComponents() bool { + if x == nil { + return false + } + return x.xxx_hidden_Components != nil +} + +func (x *ExecutableStagePayload) ClearEnvironment() { + x.xxx_hidden_Environment = nil +} + +func (x *ExecutableStagePayload) ClearComponents() { + x.xxx_hidden_Components = nil +} + +type ExecutableStagePayload_builder struct { + _ [0]func() // Prevents comparability and use of unkeyed literals for the builder. + + // (Required) Environment in which this stage executes. + // + // We use an environment rather than environment id + // because ExecutableStages use environments directly. This may change in the future. + Environment *Environment + // The wire coder settings of this executable stage + WireCoderSettings []*ExecutableStagePayload_WireCoderSetting + // (Required) Input PCollection id. This must be present as a value in the inputs of any + // PTransform the ExecutableStagePayload is the payload of. + Input string + // The side inputs required for this executable stage. Each side input of each PTransform within + // this ExecutableStagePayload must be represented within this field. + SideInputs []*ExecutableStagePayload_SideInputId + // PTransform ids contained within this executable stage. This must contain at least one + // PTransform id. + Transforms []string + // Output PCollection ids. This must be equal to the values of the outputs of any + // PTransform the ExecutableStagePayload is the payload of. + Outputs []string + // (Required) The components for the Executable Stage. This must contain all of the Transforms + // in transforms, and the closure of all of the components they recognize. + Components *Components + // The user states required for this executable stage. Each user state of each PTransform within + // this ExecutableStagePayload must be represented within this field. + UserStates []*ExecutableStagePayload_UserStateId + // The timers required for this executable stage. Each timer of each PTransform within + // this ExecutableStagePayload must be represented within this field. + Timers []*ExecutableStagePayload_TimerId + // The timerfamilies required for this executable stage. Each timer familyof each PTransform within + // this ExecutableStagePayload must be represented within this field. + TimerFamilies []*ExecutableStagePayload_TimerFamilyId +} + +func (b0 ExecutableStagePayload_builder) Build() *ExecutableStagePayload { + m0 := &ExecutableStagePayload{} + b, x := &b0, m0 + _, _ = b, x + x.xxx_hidden_Environment = b.Environment + x.xxx_hidden_WireCoderSettings = &b.WireCoderSettings + x.xxx_hidden_Input = b.Input + x.xxx_hidden_SideInputs = &b.SideInputs + x.xxx_hidden_Transforms = b.Transforms + x.xxx_hidden_Outputs = b.Outputs + x.xxx_hidden_Components = b.Components + x.xxx_hidden_UserStates = &b.UserStates + x.xxx_hidden_Timers = &b.Timers + x.xxx_hidden_TimerFamilies = &b.TimerFamilies + return m0 +} + +// See https://beam.apache.org/documentation/runtime/resource-hints/ for additional documentation +// on the behavior of StandardResourceHint. +type StandardResourceHints struct { + state protoimpl.MessageState `protogen:"opaque.v1"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *StandardResourceHints) Reset() { + *x = StandardResourceHints{} + mi := &file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_msgTypes[65] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *StandardResourceHints) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*StandardResourceHints) ProtoMessage() {} + +func (x *StandardResourceHints) ProtoReflect() protoreflect.Message { + mi := &file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_msgTypes[65] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +type StandardResourceHints_builder struct { + _ [0]func() // Prevents comparability and use of unkeyed literals for the builder. + +} + +func (b0 StandardResourceHints_builder) Build() *StandardResourceHints { + m0 := &StandardResourceHints{} + b, x := &b0, m0 + _, _ = b, x + return m0 +} + +type TestStreamPayload_Event struct { + state protoimpl.MessageState `protogen:"opaque.v1"` + xxx_hidden_Event isTestStreamPayload_Event_Event `protobuf_oneof:"event"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *TestStreamPayload_Event) Reset() { + *x = TestStreamPayload_Event{} + mi := &file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_msgTypes[77] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *TestStreamPayload_Event) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*TestStreamPayload_Event) ProtoMessage() {} + +func (x *TestStreamPayload_Event) ProtoReflect() protoreflect.Message { + mi := &file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_msgTypes[77] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +func (x *TestStreamPayload_Event) GetWatermarkEvent() *TestStreamPayload_Event_AdvanceWatermark { + if x != nil { + if x, ok := x.xxx_hidden_Event.(*testStreamPayload_Event_WatermarkEvent); ok { + return x.WatermarkEvent + } + } + return nil +} + +func (x *TestStreamPayload_Event) GetProcessingTimeEvent() *TestStreamPayload_Event_AdvanceProcessingTime { + if x != nil { + if x, ok := x.xxx_hidden_Event.(*testStreamPayload_Event_ProcessingTimeEvent); ok { + return x.ProcessingTimeEvent + } + } + return nil +} + +func (x *TestStreamPayload_Event) GetElementEvent() *TestStreamPayload_Event_AddElements { + if x != nil { + if x, ok := x.xxx_hidden_Event.(*testStreamPayload_Event_ElementEvent); ok { + return x.ElementEvent + } + } + return nil +} + +func (x *TestStreamPayload_Event) SetWatermarkEvent(v *TestStreamPayload_Event_AdvanceWatermark) { + if v == nil { + x.xxx_hidden_Event = nil + return + } + x.xxx_hidden_Event = &testStreamPayload_Event_WatermarkEvent{v} +} + +func (x *TestStreamPayload_Event) SetProcessingTimeEvent(v *TestStreamPayload_Event_AdvanceProcessingTime) { + if v == nil { + x.xxx_hidden_Event = nil + return + } + x.xxx_hidden_Event = &testStreamPayload_Event_ProcessingTimeEvent{v} +} + +func (x *TestStreamPayload_Event) SetElementEvent(v *TestStreamPayload_Event_AddElements) { + if v == nil { + x.xxx_hidden_Event = nil + return + } + x.xxx_hidden_Event = &testStreamPayload_Event_ElementEvent{v} +} + +func (x *TestStreamPayload_Event) HasEvent() bool { + if x == nil { + return false + } + return x.xxx_hidden_Event != nil +} + +func (x *TestStreamPayload_Event) HasWatermarkEvent() bool { + if x == nil { + return false + } + _, ok := x.xxx_hidden_Event.(*testStreamPayload_Event_WatermarkEvent) + return ok +} + +func (x *TestStreamPayload_Event) HasProcessingTimeEvent() bool { + if x == nil { + return false + } + _, ok := x.xxx_hidden_Event.(*testStreamPayload_Event_ProcessingTimeEvent) + return ok +} + +func (x *TestStreamPayload_Event) HasElementEvent() bool { + if x == nil { + return false + } + _, ok := x.xxx_hidden_Event.(*testStreamPayload_Event_ElementEvent) + return ok +} + +func (x *TestStreamPayload_Event) ClearEvent() { + x.xxx_hidden_Event = nil +} + +func (x *TestStreamPayload_Event) ClearWatermarkEvent() { + if _, ok := x.xxx_hidden_Event.(*testStreamPayload_Event_WatermarkEvent); ok { + x.xxx_hidden_Event = nil + } +} + +func (x *TestStreamPayload_Event) ClearProcessingTimeEvent() { + if _, ok := x.xxx_hidden_Event.(*testStreamPayload_Event_ProcessingTimeEvent); ok { + x.xxx_hidden_Event = nil + } +} + +func (x *TestStreamPayload_Event) ClearElementEvent() { + if _, ok := x.xxx_hidden_Event.(*testStreamPayload_Event_ElementEvent); ok { + x.xxx_hidden_Event = nil + } +} + +const TestStreamPayload_Event_Event_not_set_case case_TestStreamPayload_Event_Event = 0 +const TestStreamPayload_Event_WatermarkEvent_case case_TestStreamPayload_Event_Event = 1 +const TestStreamPayload_Event_ProcessingTimeEvent_case case_TestStreamPayload_Event_Event = 2 +const TestStreamPayload_Event_ElementEvent_case case_TestStreamPayload_Event_Event = 3 + +func (x *TestStreamPayload_Event) WhichEvent() case_TestStreamPayload_Event_Event { + if x == nil { + return TestStreamPayload_Event_Event_not_set_case + } + switch x.xxx_hidden_Event.(type) { + case *testStreamPayload_Event_WatermarkEvent: + return TestStreamPayload_Event_WatermarkEvent_case + case *testStreamPayload_Event_ProcessingTimeEvent: + return TestStreamPayload_Event_ProcessingTimeEvent_case + case *testStreamPayload_Event_ElementEvent: + return TestStreamPayload_Event_ElementEvent_case + default: + return TestStreamPayload_Event_Event_not_set_case + } +} + +type TestStreamPayload_Event_builder struct { + _ [0]func() // Prevents comparability and use of unkeyed literals for the builder. + + // Fields of oneof xxx_hidden_Event: + WatermarkEvent *TestStreamPayload_Event_AdvanceWatermark + ProcessingTimeEvent *TestStreamPayload_Event_AdvanceProcessingTime + ElementEvent *TestStreamPayload_Event_AddElements + // -- end of xxx_hidden_Event +} + +func (b0 TestStreamPayload_Event_builder) Build() *TestStreamPayload_Event { + m0 := &TestStreamPayload_Event{} + b, x := &b0, m0 + _, _ = b, x + if b.WatermarkEvent != nil { + x.xxx_hidden_Event = &testStreamPayload_Event_WatermarkEvent{b.WatermarkEvent} + } + if b.ProcessingTimeEvent != nil { + x.xxx_hidden_Event = &testStreamPayload_Event_ProcessingTimeEvent{b.ProcessingTimeEvent} + } + if b.ElementEvent != nil { + x.xxx_hidden_Event = &testStreamPayload_Event_ElementEvent{b.ElementEvent} + } + return m0 +} + +type case_TestStreamPayload_Event_Event protoreflect.FieldNumber + +func (x case_TestStreamPayload_Event_Event) String() string { + md := file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_msgTypes[77].Descriptor() + if x == 0 { + return "not set" + } + return protoimpl.X.MessageFieldStringOf(md, protoreflect.FieldNumber(x)) +} + +type isTestStreamPayload_Event_Event interface { + isTestStreamPayload_Event_Event() +} + +type testStreamPayload_Event_WatermarkEvent struct { + WatermarkEvent *TestStreamPayload_Event_AdvanceWatermark `protobuf:"bytes,1,opt,name=watermark_event,json=watermarkEvent,proto3,oneof"` +} + +type testStreamPayload_Event_ProcessingTimeEvent struct { + ProcessingTimeEvent *TestStreamPayload_Event_AdvanceProcessingTime `protobuf:"bytes,2,opt,name=processing_time_event,json=processingTimeEvent,proto3,oneof"` +} + +type testStreamPayload_Event_ElementEvent struct { + ElementEvent *TestStreamPayload_Event_AddElements `protobuf:"bytes,3,opt,name=element_event,json=elementEvent,proto3,oneof"` +} + +func (*testStreamPayload_Event_WatermarkEvent) isTestStreamPayload_Event_Event() {} + +func (*testStreamPayload_Event_ProcessingTimeEvent) isTestStreamPayload_Event_Event() {} + +func (*testStreamPayload_Event_ElementEvent) isTestStreamPayload_Event_Event() {} + +// A single element inside of the TestStream. +type TestStreamPayload_TimestampedElement struct { + state protoimpl.MessageState `protogen:"opaque.v1"` + xxx_hidden_EncodedElement []byte `protobuf:"bytes,1,opt,name=encoded_element,json=encodedElement,proto3" json:"encoded_element,omitempty"` + xxx_hidden_Timestamp int64 `protobuf:"varint,2,opt,name=timestamp,proto3" json:"timestamp,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *TestStreamPayload_TimestampedElement) Reset() { + *x = TestStreamPayload_TimestampedElement{} + mi := &file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_msgTypes[78] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *TestStreamPayload_TimestampedElement) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*TestStreamPayload_TimestampedElement) ProtoMessage() {} + +func (x *TestStreamPayload_TimestampedElement) ProtoReflect() protoreflect.Message { + mi := &file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_msgTypes[78] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +func (x *TestStreamPayload_TimestampedElement) GetEncodedElement() []byte { + if x != nil { + return x.xxx_hidden_EncodedElement + } + return nil +} + +func (x *TestStreamPayload_TimestampedElement) GetTimestamp() int64 { + if x != nil { + return x.xxx_hidden_Timestamp + } + return 0 +} + +func (x *TestStreamPayload_TimestampedElement) SetEncodedElement(v []byte) { + if v == nil { + v = []byte{} + } + x.xxx_hidden_EncodedElement = v +} + +func (x *TestStreamPayload_TimestampedElement) SetTimestamp(v int64) { + x.xxx_hidden_Timestamp = v +} + +type TestStreamPayload_TimestampedElement_builder struct { + _ [0]func() // Prevents comparability and use of unkeyed literals for the builder. + + // (Required) The element encoded. Currently the TestStream only supports + // encoding primitives. + EncodedElement []byte + // (Required) The event timestamp in millisecond of this element. + Timestamp int64 +} + +func (b0 TestStreamPayload_TimestampedElement_builder) Build() *TestStreamPayload_TimestampedElement { + m0 := &TestStreamPayload_TimestampedElement{} + b, x := &b0, m0 + _, _ = b, x + x.xxx_hidden_EncodedElement = b.EncodedElement + x.xxx_hidden_Timestamp = b.Timestamp + return m0 +} + +// Advances the watermark to the specified timestamp. +type TestStreamPayload_Event_AdvanceWatermark struct { + state protoimpl.MessageState `protogen:"opaque.v1"` + xxx_hidden_NewWatermark int64 `protobuf:"varint,1,opt,name=new_watermark,json=newWatermark,proto3" json:"new_watermark,omitempty"` + xxx_hidden_Tag string `protobuf:"bytes,2,opt,name=tag,proto3" json:"tag,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *TestStreamPayload_Event_AdvanceWatermark) Reset() { + *x = TestStreamPayload_Event_AdvanceWatermark{} + mi := &file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_msgTypes[79] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *TestStreamPayload_Event_AdvanceWatermark) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*TestStreamPayload_Event_AdvanceWatermark) ProtoMessage() {} + +func (x *TestStreamPayload_Event_AdvanceWatermark) ProtoReflect() protoreflect.Message { + mi := &file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_msgTypes[79] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +func (x *TestStreamPayload_Event_AdvanceWatermark) GetNewWatermark() int64 { + if x != nil { + return x.xxx_hidden_NewWatermark + } + return 0 +} + +func (x *TestStreamPayload_Event_AdvanceWatermark) GetTag() string { + if x != nil { + return x.xxx_hidden_Tag + } + return "" +} + +func (x *TestStreamPayload_Event_AdvanceWatermark) SetNewWatermark(v int64) { + x.xxx_hidden_NewWatermark = v +} + +func (x *TestStreamPayload_Event_AdvanceWatermark) SetTag(v string) { + x.xxx_hidden_Tag = v +} + +type TestStreamPayload_Event_AdvanceWatermark_builder struct { + _ [0]func() // Prevents comparability and use of unkeyed literals for the builder. + + // (Required) The watermark in millisecond to advance to. + NewWatermark int64 + // (Optional) The output watermark tag for a PCollection. If unspecified + // or with an empty string, this will default to the Main PCollection + // Output + Tag string +} + +func (b0 TestStreamPayload_Event_AdvanceWatermark_builder) Build() *TestStreamPayload_Event_AdvanceWatermark { + m0 := &TestStreamPayload_Event_AdvanceWatermark{} + b, x := &b0, m0 + _, _ = b, x + x.xxx_hidden_NewWatermark = b.NewWatermark + x.xxx_hidden_Tag = b.Tag + return m0 +} + +// Advances the processing time clock by the specified amount. +type TestStreamPayload_Event_AdvanceProcessingTime struct { + state protoimpl.MessageState `protogen:"opaque.v1"` + xxx_hidden_AdvanceDuration int64 `protobuf:"varint,1,opt,name=advance_duration,json=advanceDuration,proto3" json:"advance_duration,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *TestStreamPayload_Event_AdvanceProcessingTime) Reset() { + *x = TestStreamPayload_Event_AdvanceProcessingTime{} + mi := &file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_msgTypes[80] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *TestStreamPayload_Event_AdvanceProcessingTime) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*TestStreamPayload_Event_AdvanceProcessingTime) ProtoMessage() {} + +func (x *TestStreamPayload_Event_AdvanceProcessingTime) ProtoReflect() protoreflect.Message { + mi := &file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_msgTypes[80] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +func (x *TestStreamPayload_Event_AdvanceProcessingTime) GetAdvanceDuration() int64 { + if x != nil { + return x.xxx_hidden_AdvanceDuration + } + return 0 +} + +func (x *TestStreamPayload_Event_AdvanceProcessingTime) SetAdvanceDuration(v int64) { + x.xxx_hidden_AdvanceDuration = v +} + +type TestStreamPayload_Event_AdvanceProcessingTime_builder struct { + _ [0]func() // Prevents comparability and use of unkeyed literals for the builder. + + // (Required) The duration in millisecond to advance by. + AdvanceDuration int64 +} + +func (b0 TestStreamPayload_Event_AdvanceProcessingTime_builder) Build() *TestStreamPayload_Event_AdvanceProcessingTime { + m0 := &TestStreamPayload_Event_AdvanceProcessingTime{} + b, x := &b0, m0 + _, _ = b, x + x.xxx_hidden_AdvanceDuration = b.AdvanceDuration + return m0 +} + +// Adds elements to the stream to be emitted. +type TestStreamPayload_Event_AddElements struct { + state protoimpl.MessageState `protogen:"opaque.v1"` + xxx_hidden_Elements *[]*TestStreamPayload_TimestampedElement `protobuf:"bytes,1,rep,name=elements,proto3" json:"elements,omitempty"` + xxx_hidden_Tag string `protobuf:"bytes,3,opt,name=tag,proto3" json:"tag,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *TestStreamPayload_Event_AddElements) Reset() { + *x = TestStreamPayload_Event_AddElements{} + mi := &file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_msgTypes[81] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *TestStreamPayload_Event_AddElements) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*TestStreamPayload_Event_AddElements) ProtoMessage() {} + +func (x *TestStreamPayload_Event_AddElements) ProtoReflect() protoreflect.Message { + mi := &file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_msgTypes[81] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +func (x *TestStreamPayload_Event_AddElements) GetElements() []*TestStreamPayload_TimestampedElement { + if x != nil { + if x.xxx_hidden_Elements != nil { + return *x.xxx_hidden_Elements + } + } + return nil +} + +func (x *TestStreamPayload_Event_AddElements) GetTag() string { + if x != nil { + return x.xxx_hidden_Tag + } + return "" +} + +func (x *TestStreamPayload_Event_AddElements) SetElements(v []*TestStreamPayload_TimestampedElement) { + x.xxx_hidden_Elements = &v +} + +func (x *TestStreamPayload_Event_AddElements) SetTag(v string) { + x.xxx_hidden_Tag = v +} + +type TestStreamPayload_Event_AddElements_builder struct { + _ [0]func() // Prevents comparability and use of unkeyed literals for the builder. + + // (Required) The elements to add to the TestStream. + Elements []*TestStreamPayload_TimestampedElement + // (Optional) The output PCollection tag to add these elements to. If + // unspecified or with an empty string, this will default to the Main + // PCollection Output. + Tag string +} + +func (b0 TestStreamPayload_Event_AddElements_builder) Build() *TestStreamPayload_Event_AddElements { + m0 := &TestStreamPayload_Event_AddElements{} + b, x := &b0, m0 + _, _ = b, x + x.xxx_hidden_Elements = &b.Elements + x.xxx_hidden_Tag = b.Tag + return m0 +} + +// Ready when all subtriggers are ready. +type Trigger_AfterAll struct { + state protoimpl.MessageState `protogen:"opaque.v1"` + xxx_hidden_Subtriggers *[]*Trigger `protobuf:"bytes,1,rep,name=subtriggers,proto3" json:"subtriggers,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *Trigger_AfterAll) Reset() { + *x = Trigger_AfterAll{} + mi := &file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_msgTypes[83] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *Trigger_AfterAll) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*Trigger_AfterAll) ProtoMessage() {} + +func (x *Trigger_AfterAll) ProtoReflect() protoreflect.Message { + mi := &file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_msgTypes[83] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +func (x *Trigger_AfterAll) GetSubtriggers() []*Trigger { + if x != nil { + if x.xxx_hidden_Subtriggers != nil { + return *x.xxx_hidden_Subtriggers + } + } + return nil +} + +func (x *Trigger_AfterAll) SetSubtriggers(v []*Trigger) { + x.xxx_hidden_Subtriggers = &v +} + +type Trigger_AfterAll_builder struct { + _ [0]func() // Prevents comparability and use of unkeyed literals for the builder. + + Subtriggers []*Trigger +} + +func (b0 Trigger_AfterAll_builder) Build() *Trigger_AfterAll { + m0 := &Trigger_AfterAll{} + b, x := &b0, m0 + _, _ = b, x + x.xxx_hidden_Subtriggers = &b.Subtriggers + return m0 +} + +// Ready when any subtrigger is ready. +type Trigger_AfterAny struct { + state protoimpl.MessageState `protogen:"opaque.v1"` + xxx_hidden_Subtriggers *[]*Trigger `protobuf:"bytes,1,rep,name=subtriggers,proto3" json:"subtriggers,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *Trigger_AfterAny) Reset() { + *x = Trigger_AfterAny{} + mi := &file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_msgTypes[84] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *Trigger_AfterAny) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*Trigger_AfterAny) ProtoMessage() {} + +func (x *Trigger_AfterAny) ProtoReflect() protoreflect.Message { + mi := &file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_msgTypes[84] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +func (x *Trigger_AfterAny) GetSubtriggers() []*Trigger { + if x != nil { + if x.xxx_hidden_Subtriggers != nil { + return *x.xxx_hidden_Subtriggers + } + } + return nil +} + +func (x *Trigger_AfterAny) SetSubtriggers(v []*Trigger) { + x.xxx_hidden_Subtriggers = &v +} + +type Trigger_AfterAny_builder struct { + _ [0]func() // Prevents comparability and use of unkeyed literals for the builder. + + Subtriggers []*Trigger +} + +func (b0 Trigger_AfterAny_builder) Build() *Trigger_AfterAny { + m0 := &Trigger_AfterAny{} + b, x := &b0, m0 + _, _ = b, x + x.xxx_hidden_Subtriggers = &b.Subtriggers + return m0 +} + +// Starting with the first subtrigger, ready when the _current_ subtrigger +// is ready. After output, advances the current trigger by one. +type Trigger_AfterEach struct { + state protoimpl.MessageState `protogen:"opaque.v1"` + xxx_hidden_Subtriggers *[]*Trigger `protobuf:"bytes,1,rep,name=subtriggers,proto3" json:"subtriggers,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *Trigger_AfterEach) Reset() { + *x = Trigger_AfterEach{} + mi := &file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_msgTypes[85] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *Trigger_AfterEach) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*Trigger_AfterEach) ProtoMessage() {} + +func (x *Trigger_AfterEach) ProtoReflect() protoreflect.Message { + mi := &file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_msgTypes[85] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +func (x *Trigger_AfterEach) GetSubtriggers() []*Trigger { + if x != nil { + if x.xxx_hidden_Subtriggers != nil { + return *x.xxx_hidden_Subtriggers + } + } + return nil +} + +func (x *Trigger_AfterEach) SetSubtriggers(v []*Trigger) { + x.xxx_hidden_Subtriggers = &v +} + +type Trigger_AfterEach_builder struct { + _ [0]func() // Prevents comparability and use of unkeyed literals for the builder. + + Subtriggers []*Trigger +} + +func (b0 Trigger_AfterEach_builder) Build() *Trigger_AfterEach { + m0 := &Trigger_AfterEach{} + b, x := &b0, m0 + _, _ = b, x + x.xxx_hidden_Subtriggers = &b.Subtriggers + return m0 +} + +// Ready after the input watermark is past the end of the window. +// +// May have implicitly-repeated subtriggers for early and late firings. +// When the end of the window is reached, the trigger transitions between +// the subtriggers. +type Trigger_AfterEndOfWindow struct { + state protoimpl.MessageState `protogen:"opaque.v1"` + xxx_hidden_EarlyFirings *Trigger `protobuf:"bytes,1,opt,name=early_firings,json=earlyFirings,proto3" json:"early_firings,omitempty"` + xxx_hidden_LateFirings *Trigger `protobuf:"bytes,2,opt,name=late_firings,json=lateFirings,proto3" json:"late_firings,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *Trigger_AfterEndOfWindow) Reset() { + *x = Trigger_AfterEndOfWindow{} + mi := &file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_msgTypes[86] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *Trigger_AfterEndOfWindow) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*Trigger_AfterEndOfWindow) ProtoMessage() {} + +func (x *Trigger_AfterEndOfWindow) ProtoReflect() protoreflect.Message { + mi := &file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_msgTypes[86] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +func (x *Trigger_AfterEndOfWindow) GetEarlyFirings() *Trigger { + if x != nil { + return x.xxx_hidden_EarlyFirings + } + return nil +} + +func (x *Trigger_AfterEndOfWindow) GetLateFirings() *Trigger { + if x != nil { + return x.xxx_hidden_LateFirings + } + return nil +} + +func (x *Trigger_AfterEndOfWindow) SetEarlyFirings(v *Trigger) { + x.xxx_hidden_EarlyFirings = v +} + +func (x *Trigger_AfterEndOfWindow) SetLateFirings(v *Trigger) { + x.xxx_hidden_LateFirings = v +} + +func (x *Trigger_AfterEndOfWindow) HasEarlyFirings() bool { + if x == nil { + return false + } + return x.xxx_hidden_EarlyFirings != nil +} + +func (x *Trigger_AfterEndOfWindow) HasLateFirings() bool { + if x == nil { + return false + } + return x.xxx_hidden_LateFirings != nil +} + +func (x *Trigger_AfterEndOfWindow) ClearEarlyFirings() { + x.xxx_hidden_EarlyFirings = nil +} + +func (x *Trigger_AfterEndOfWindow) ClearLateFirings() { + x.xxx_hidden_LateFirings = nil +} + +type Trigger_AfterEndOfWindow_builder struct { + _ [0]func() // Prevents comparability and use of unkeyed literals for the builder. + + // (Optional) A trigger governing output prior to the end of the window. + EarlyFirings *Trigger + // (Optional) A trigger governing output after the end of the window. + LateFirings *Trigger +} + +func (b0 Trigger_AfterEndOfWindow_builder) Build() *Trigger_AfterEndOfWindow { + m0 := &Trigger_AfterEndOfWindow{} + b, x := &b0, m0 + _, _ = b, x + x.xxx_hidden_EarlyFirings = b.EarlyFirings + x.xxx_hidden_LateFirings = b.LateFirings + return m0 +} + +// After input arrives, ready when the specified delay has passed. +type Trigger_AfterProcessingTime struct { + state protoimpl.MessageState `protogen:"opaque.v1"` + xxx_hidden_TimestampTransforms *[]*TimestampTransform `protobuf:"bytes,1,rep,name=timestamp_transforms,json=timestampTransforms,proto3" json:"timestamp_transforms,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *Trigger_AfterProcessingTime) Reset() { + *x = Trigger_AfterProcessingTime{} + mi := &file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_msgTypes[87] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *Trigger_AfterProcessingTime) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*Trigger_AfterProcessingTime) ProtoMessage() {} + +func (x *Trigger_AfterProcessingTime) ProtoReflect() protoreflect.Message { + mi := &file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_msgTypes[87] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +func (x *Trigger_AfterProcessingTime) GetTimestampTransforms() []*TimestampTransform { + if x != nil { + if x.xxx_hidden_TimestampTransforms != nil { + return *x.xxx_hidden_TimestampTransforms + } + } + return nil +} + +func (x *Trigger_AfterProcessingTime) SetTimestampTransforms(v []*TimestampTransform) { + x.xxx_hidden_TimestampTransforms = &v +} + +type Trigger_AfterProcessingTime_builder struct { + _ [0]func() // Prevents comparability and use of unkeyed literals for the builder. + + // (Required) The transforms to apply to an arriving element's timestamp, + // in order + TimestampTransforms []*TimestampTransform +} + +func (b0 Trigger_AfterProcessingTime_builder) Build() *Trigger_AfterProcessingTime { + m0 := &Trigger_AfterProcessingTime{} + b, x := &b0, m0 + _, _ = b, x + x.xxx_hidden_TimestampTransforms = &b.TimestampTransforms + return m0 +} + +// Ready whenever upstream processing time has all caught up with +// the arrival time of an input element +type Trigger_AfterSynchronizedProcessingTime struct { + state protoimpl.MessageState `protogen:"opaque.v1"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *Trigger_AfterSynchronizedProcessingTime) Reset() { + *x = Trigger_AfterSynchronizedProcessingTime{} + mi := &file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_msgTypes[88] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *Trigger_AfterSynchronizedProcessingTime) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*Trigger_AfterSynchronizedProcessingTime) ProtoMessage() {} + +func (x *Trigger_AfterSynchronizedProcessingTime) ProtoReflect() protoreflect.Message { + mi := &file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_msgTypes[88] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +type Trigger_AfterSynchronizedProcessingTime_builder struct { + _ [0]func() // Prevents comparability and use of unkeyed literals for the builder. + +} + +func (b0 Trigger_AfterSynchronizedProcessingTime_builder) Build() *Trigger_AfterSynchronizedProcessingTime { + m0 := &Trigger_AfterSynchronizedProcessingTime{} + b, x := &b0, m0 + _, _ = b, x + return m0 +} + +// The default trigger. Equivalent to Repeat { AfterEndOfWindow } but +// specially denoted to indicate the user did not alter the triggering. +type Trigger_Default struct { + state protoimpl.MessageState `protogen:"opaque.v1"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *Trigger_Default) Reset() { + *x = Trigger_Default{} + mi := &file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_msgTypes[89] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *Trigger_Default) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*Trigger_Default) ProtoMessage() {} + +func (x *Trigger_Default) ProtoReflect() protoreflect.Message { + mi := &file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_msgTypes[89] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +type Trigger_Default_builder struct { + _ [0]func() // Prevents comparability and use of unkeyed literals for the builder. + +} + +func (b0 Trigger_Default_builder) Build() *Trigger_Default { + m0 := &Trigger_Default{} + b, x := &b0, m0 + _, _ = b, x + return m0 +} + +// Ready whenever the requisite number of input elements have arrived +type Trigger_ElementCount struct { + state protoimpl.MessageState `protogen:"opaque.v1"` + xxx_hidden_ElementCount int32 `protobuf:"varint,1,opt,name=element_count,json=elementCount,proto3" json:"element_count,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *Trigger_ElementCount) Reset() { + *x = Trigger_ElementCount{} + mi := &file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_msgTypes[90] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *Trigger_ElementCount) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*Trigger_ElementCount) ProtoMessage() {} + +func (x *Trigger_ElementCount) ProtoReflect() protoreflect.Message { + mi := &file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_msgTypes[90] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +func (x *Trigger_ElementCount) GetElementCount() int32 { + if x != nil { + return x.xxx_hidden_ElementCount + } + return 0 +} + +func (x *Trigger_ElementCount) SetElementCount(v int32) { + x.xxx_hidden_ElementCount = v +} + +type Trigger_ElementCount_builder struct { + _ [0]func() // Prevents comparability and use of unkeyed literals for the builder. + + ElementCount int32 +} + +func (b0 Trigger_ElementCount_builder) Build() *Trigger_ElementCount { + m0 := &Trigger_ElementCount{} + b, x := &b0, m0 + _, _ = b, x + x.xxx_hidden_ElementCount = b.ElementCount + return m0 +} + +// Never ready. There will only be an ON_TIME output and a final +// output at window expiration. +type Trigger_Never struct { + state protoimpl.MessageState `protogen:"opaque.v1"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *Trigger_Never) Reset() { + *x = Trigger_Never{} + mi := &file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_msgTypes[91] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *Trigger_Never) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*Trigger_Never) ProtoMessage() {} + +func (x *Trigger_Never) ProtoReflect() protoreflect.Message { + mi := &file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_msgTypes[91] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +type Trigger_Never_builder struct { + _ [0]func() // Prevents comparability and use of unkeyed literals for the builder. + +} + +func (b0 Trigger_Never_builder) Build() *Trigger_Never { + m0 := &Trigger_Never{} + b, x := &b0, m0 + _, _ = b, x + return m0 +} + +// Always ready. This can also be expressed as ElementCount(1) but +// is more explicit. +type Trigger_Always struct { + state protoimpl.MessageState `protogen:"opaque.v1"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *Trigger_Always) Reset() { + *x = Trigger_Always{} + mi := &file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_msgTypes[92] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *Trigger_Always) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*Trigger_Always) ProtoMessage() {} + +func (x *Trigger_Always) ProtoReflect() protoreflect.Message { + mi := &file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_msgTypes[92] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +type Trigger_Always_builder struct { + _ [0]func() // Prevents comparability and use of unkeyed literals for the builder. + +} + +func (b0 Trigger_Always_builder) Build() *Trigger_Always { + m0 := &Trigger_Always{} + b, x := &b0, m0 + _, _ = b, x + return m0 +} + +// Ready whenever either of its subtriggers are ready, but finishes output +// when the finally subtrigger fires. +type Trigger_OrFinally struct { + state protoimpl.MessageState `protogen:"opaque.v1"` + xxx_hidden_Main *Trigger `protobuf:"bytes,1,opt,name=main,proto3" json:"main,omitempty"` + xxx_hidden_Finally *Trigger `protobuf:"bytes,2,opt,name=finally,proto3" json:"finally,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *Trigger_OrFinally) Reset() { + *x = Trigger_OrFinally{} + mi := &file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_msgTypes[93] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *Trigger_OrFinally) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*Trigger_OrFinally) ProtoMessage() {} + +func (x *Trigger_OrFinally) ProtoReflect() protoreflect.Message { + mi := &file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_msgTypes[93] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +func (x *Trigger_OrFinally) GetMain() *Trigger { + if x != nil { + return x.xxx_hidden_Main + } + return nil +} + +func (x *Trigger_OrFinally) GetFinally() *Trigger { + if x != nil { + return x.xxx_hidden_Finally + } + return nil +} + +func (x *Trigger_OrFinally) SetMain(v *Trigger) { + x.xxx_hidden_Main = v +} + +func (x *Trigger_OrFinally) SetFinally(v *Trigger) { + x.xxx_hidden_Finally = v +} + +func (x *Trigger_OrFinally) HasMain() bool { + if x == nil { + return false + } + return x.xxx_hidden_Main != nil +} + +func (x *Trigger_OrFinally) HasFinally() bool { + if x == nil { + return false + } + return x.xxx_hidden_Finally != nil +} + +func (x *Trigger_OrFinally) ClearMain() { + x.xxx_hidden_Main = nil +} + +func (x *Trigger_OrFinally) ClearFinally() { + x.xxx_hidden_Finally = nil +} + +type Trigger_OrFinally_builder struct { + _ [0]func() // Prevents comparability and use of unkeyed literals for the builder. + + // (Required) Trigger governing main output; may fire repeatedly. + Main *Trigger + // (Required) Trigger governing termination of output. + Finally *Trigger +} + +func (b0 Trigger_OrFinally_builder) Build() *Trigger_OrFinally { + m0 := &Trigger_OrFinally{} + b, x := &b0, m0 + _, _ = b, x + x.xxx_hidden_Main = b.Main + x.xxx_hidden_Finally = b.Finally + return m0 +} + +// Ready whenever the subtrigger is ready; resets state when the subtrigger +// completes. +type Trigger_Repeat struct { + state protoimpl.MessageState `protogen:"opaque.v1"` + xxx_hidden_Subtrigger *Trigger `protobuf:"bytes,1,opt,name=subtrigger,proto3" json:"subtrigger,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *Trigger_Repeat) Reset() { + *x = Trigger_Repeat{} + mi := &file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_msgTypes[94] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *Trigger_Repeat) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*Trigger_Repeat) ProtoMessage() {} + +func (x *Trigger_Repeat) ProtoReflect() protoreflect.Message { + mi := &file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_msgTypes[94] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +func (x *Trigger_Repeat) GetSubtrigger() *Trigger { + if x != nil { + return x.xxx_hidden_Subtrigger + } + return nil +} + +func (x *Trigger_Repeat) SetSubtrigger(v *Trigger) { + x.xxx_hidden_Subtrigger = v +} + +func (x *Trigger_Repeat) HasSubtrigger() bool { + if x == nil { + return false + } + return x.xxx_hidden_Subtrigger != nil +} + +func (x *Trigger_Repeat) ClearSubtrigger() { + x.xxx_hidden_Subtrigger = nil +} + +type Trigger_Repeat_builder struct { + _ [0]func() // Prevents comparability and use of unkeyed literals for the builder. + + // (Require) Trigger that is run repeatedly. + Subtrigger *Trigger +} + +func (b0 Trigger_Repeat_builder) Build() *Trigger_Repeat { + m0 := &Trigger_Repeat{} + b, x := &b0, m0 + _, _ = b, x + x.xxx_hidden_Subtrigger = b.Subtrigger + return m0 +} + +type TimestampTransform_Delay struct { + state protoimpl.MessageState `protogen:"opaque.v1"` + xxx_hidden_DelayMillis int64 `protobuf:"varint,1,opt,name=delay_millis,json=delayMillis,proto3" json:"delay_millis,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *TimestampTransform_Delay) Reset() { + *x = TimestampTransform_Delay{} + mi := &file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_msgTypes[95] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *TimestampTransform_Delay) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*TimestampTransform_Delay) ProtoMessage() {} + +func (x *TimestampTransform_Delay) ProtoReflect() protoreflect.Message { + mi := &file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_msgTypes[95] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +func (x *TimestampTransform_Delay) GetDelayMillis() int64 { + if x != nil { + return x.xxx_hidden_DelayMillis + } + return 0 +} + +func (x *TimestampTransform_Delay) SetDelayMillis(v int64) { + x.xxx_hidden_DelayMillis = v +} + +type TimestampTransform_Delay_builder struct { + _ [0]func() // Prevents comparability and use of unkeyed literals for the builder. + + // (Required) The delay, in milliseconds. + DelayMillis int64 +} + +func (b0 TimestampTransform_Delay_builder) Build() *TimestampTransform_Delay { + m0 := &TimestampTransform_Delay{} + b, x := &b0, m0 + _, _ = b, x + x.xxx_hidden_DelayMillis = b.DelayMillis + return m0 +} + +type TimestampTransform_AlignTo struct { + state protoimpl.MessageState `protogen:"opaque.v1"` + xxx_hidden_Period int64 `protobuf:"varint,3,opt,name=period,proto3" json:"period,omitempty"` + xxx_hidden_Offset int64 `protobuf:"varint,4,opt,name=offset,proto3" json:"offset,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *TimestampTransform_AlignTo) Reset() { + *x = TimestampTransform_AlignTo{} + mi := &file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_msgTypes[96] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *TimestampTransform_AlignTo) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*TimestampTransform_AlignTo) ProtoMessage() {} + +func (x *TimestampTransform_AlignTo) ProtoReflect() protoreflect.Message { + mi := &file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_msgTypes[96] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +func (x *TimestampTransform_AlignTo) GetPeriod() int64 { + if x != nil { + return x.xxx_hidden_Period + } + return 0 +} + +func (x *TimestampTransform_AlignTo) GetOffset() int64 { + if x != nil { + return x.xxx_hidden_Offset + } + return 0 +} + +func (x *TimestampTransform_AlignTo) SetPeriod(v int64) { + x.xxx_hidden_Period = v +} + +func (x *TimestampTransform_AlignTo) SetOffset(v int64) { + x.xxx_hidden_Offset = v +} + +type TimestampTransform_AlignTo_builder struct { + _ [0]func() // Prevents comparability and use of unkeyed literals for the builder. + + // (Required) A duration to which delays should be quantized + // in milliseconds. + Period int64 + // (Required) An offset from 0 for the quantization specified by + // alignment_size, in milliseconds + Offset int64 +} + +func (b0 TimestampTransform_AlignTo_builder) Build() *TimestampTransform_AlignTo { + m0 := &TimestampTransform_AlignTo{} + b, x := &b0, m0 + _, _ = b, x + x.xxx_hidden_Period = b.Period + x.xxx_hidden_Offset = b.Offset + return m0 +} + +// A reference to a side input. Side inputs are uniquely identified by PTransform id and +// local name. +type ExecutableStagePayload_SideInputId struct { + state protoimpl.MessageState `protogen:"opaque.v1"` + xxx_hidden_TransformId string `protobuf:"bytes,1,opt,name=transform_id,json=transformId,proto3" json:"transform_id,omitempty"` + xxx_hidden_LocalName string `protobuf:"bytes,2,opt,name=local_name,json=localName,proto3" json:"local_name,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *ExecutableStagePayload_SideInputId) Reset() { + *x = ExecutableStagePayload_SideInputId{} + mi := &file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_msgTypes[100] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *ExecutableStagePayload_SideInputId) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*ExecutableStagePayload_SideInputId) ProtoMessage() {} + +func (x *ExecutableStagePayload_SideInputId) ProtoReflect() protoreflect.Message { + mi := &file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_msgTypes[100] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +func (x *ExecutableStagePayload_SideInputId) GetTransformId() string { + if x != nil { + return x.xxx_hidden_TransformId + } + return "" +} + +func (x *ExecutableStagePayload_SideInputId) GetLocalName() string { + if x != nil { + return x.xxx_hidden_LocalName + } + return "" +} + +func (x *ExecutableStagePayload_SideInputId) SetTransformId(v string) { + x.xxx_hidden_TransformId = v +} + +func (x *ExecutableStagePayload_SideInputId) SetLocalName(v string) { + x.xxx_hidden_LocalName = v +} + +type ExecutableStagePayload_SideInputId_builder struct { + _ [0]func() // Prevents comparability and use of unkeyed literals for the builder. + + // (Required) The id of the PTransform that references this side input. + TransformId string + // (Required) The local name of this side input from the PTransform that references it. + LocalName string +} + +func (b0 ExecutableStagePayload_SideInputId_builder) Build() *ExecutableStagePayload_SideInputId { + m0 := &ExecutableStagePayload_SideInputId{} + b, x := &b0, m0 + _, _ = b, x + x.xxx_hidden_TransformId = b.TransformId + x.xxx_hidden_LocalName = b.LocalName + return m0 +} + +// A reference to user state. User states are uniquely identified by PTransform id and +// local name. +type ExecutableStagePayload_UserStateId struct { + state protoimpl.MessageState `protogen:"opaque.v1"` + xxx_hidden_TransformId string `protobuf:"bytes,1,opt,name=transform_id,json=transformId,proto3" json:"transform_id,omitempty"` + xxx_hidden_LocalName string `protobuf:"bytes,2,opt,name=local_name,json=localName,proto3" json:"local_name,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *ExecutableStagePayload_UserStateId) Reset() { + *x = ExecutableStagePayload_UserStateId{} + mi := &file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_msgTypes[101] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *ExecutableStagePayload_UserStateId) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*ExecutableStagePayload_UserStateId) ProtoMessage() {} + +func (x *ExecutableStagePayload_UserStateId) ProtoReflect() protoreflect.Message { + mi := &file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_msgTypes[101] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +func (x *ExecutableStagePayload_UserStateId) GetTransformId() string { + if x != nil { + return x.xxx_hidden_TransformId + } + return "" +} + +func (x *ExecutableStagePayload_UserStateId) GetLocalName() string { + if x != nil { + return x.xxx_hidden_LocalName + } + return "" +} + +func (x *ExecutableStagePayload_UserStateId) SetTransformId(v string) { + x.xxx_hidden_TransformId = v +} + +func (x *ExecutableStagePayload_UserStateId) SetLocalName(v string) { + x.xxx_hidden_LocalName = v +} + +type ExecutableStagePayload_UserStateId_builder struct { + _ [0]func() // Prevents comparability and use of unkeyed literals for the builder. + + // (Required) The id of the PTransform that references this user state. + TransformId string + // (Required) The local name of this user state for the PTransform that references it. + LocalName string +} + +func (b0 ExecutableStagePayload_UserStateId_builder) Build() *ExecutableStagePayload_UserStateId { + m0 := &ExecutableStagePayload_UserStateId{} + b, x := &b0, m0 + _, _ = b, x + x.xxx_hidden_TransformId = b.TransformId + x.xxx_hidden_LocalName = b.LocalName + return m0 +} + +// A reference to a timer. Timers are uniquely identified by PTransform id and +// local name. +type ExecutableStagePayload_TimerId struct { + state protoimpl.MessageState `protogen:"opaque.v1"` + xxx_hidden_TransformId string `protobuf:"bytes,1,opt,name=transform_id,json=transformId,proto3" json:"transform_id,omitempty"` + xxx_hidden_LocalName string `protobuf:"bytes,2,opt,name=local_name,json=localName,proto3" json:"local_name,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *ExecutableStagePayload_TimerId) Reset() { + *x = ExecutableStagePayload_TimerId{} + mi := &file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_msgTypes[102] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *ExecutableStagePayload_TimerId) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*ExecutableStagePayload_TimerId) ProtoMessage() {} + +func (x *ExecutableStagePayload_TimerId) ProtoReflect() protoreflect.Message { + mi := &file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_msgTypes[102] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +func (x *ExecutableStagePayload_TimerId) GetTransformId() string { + if x != nil { + return x.xxx_hidden_TransformId + } + return "" +} + +func (x *ExecutableStagePayload_TimerId) GetLocalName() string { + if x != nil { + return x.xxx_hidden_LocalName + } + return "" +} + +func (x *ExecutableStagePayload_TimerId) SetTransformId(v string) { + x.xxx_hidden_TransformId = v +} + +func (x *ExecutableStagePayload_TimerId) SetLocalName(v string) { + x.xxx_hidden_LocalName = v +} + +type ExecutableStagePayload_TimerId_builder struct { + _ [0]func() // Prevents comparability and use of unkeyed literals for the builder. + + // (Required) The id of the PTransform that references this timer. + TransformId string + // (Required) The local name of this timer for the PTransform that references it. + LocalName string +} + +func (b0 ExecutableStagePayload_TimerId_builder) Build() *ExecutableStagePayload_TimerId { + m0 := &ExecutableStagePayload_TimerId{} + b, x := &b0, m0 + _, _ = b, x + x.xxx_hidden_TransformId = b.TransformId + x.xxx_hidden_LocalName = b.LocalName + return m0 +} + +// A reference to a timer. Timers are uniquely identified by PTransform id and +// local name. +type ExecutableStagePayload_TimerFamilyId struct { + state protoimpl.MessageState `protogen:"opaque.v1"` + xxx_hidden_TransformId string `protobuf:"bytes,1,opt,name=transform_id,json=transformId,proto3" json:"transform_id,omitempty"` + xxx_hidden_LocalName string `protobuf:"bytes,2,opt,name=local_name,json=localName,proto3" json:"local_name,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *ExecutableStagePayload_TimerFamilyId) Reset() { + *x = ExecutableStagePayload_TimerFamilyId{} + mi := &file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_msgTypes[103] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *ExecutableStagePayload_TimerFamilyId) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*ExecutableStagePayload_TimerFamilyId) ProtoMessage() {} + +func (x *ExecutableStagePayload_TimerFamilyId) ProtoReflect() protoreflect.Message { + mi := &file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_msgTypes[103] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +func (x *ExecutableStagePayload_TimerFamilyId) GetTransformId() string { + if x != nil { + return x.xxx_hidden_TransformId + } + return "" +} + +func (x *ExecutableStagePayload_TimerFamilyId) GetLocalName() string { + if x != nil { + return x.xxx_hidden_LocalName + } + return "" +} + +func (x *ExecutableStagePayload_TimerFamilyId) SetTransformId(v string) { + x.xxx_hidden_TransformId = v +} + +func (x *ExecutableStagePayload_TimerFamilyId) SetLocalName(v string) { + x.xxx_hidden_LocalName = v +} + +type ExecutableStagePayload_TimerFamilyId_builder struct { + _ [0]func() // Prevents comparability and use of unkeyed literals for the builder. + + // (Required) The id of the PTransform that references this timer family. + TransformId string + // (Required) The local name of this timer family for the PTransform that references it. + LocalName string +} + +func (b0 ExecutableStagePayload_TimerFamilyId_builder) Build() *ExecutableStagePayload_TimerFamilyId { + m0 := &ExecutableStagePayload_TimerFamilyId{} + b, x := &b0, m0 + _, _ = b, x + x.xxx_hidden_TransformId = b.TransformId + x.xxx_hidden_LocalName = b.LocalName + return m0 +} + +// Settings that decide the coder type of wire coder. +type ExecutableStagePayload_WireCoderSetting struct { + state protoimpl.MessageState `protogen:"opaque.v1"` + xxx_hidden_Urn string `protobuf:"bytes,1,opt,name=urn,proto3" json:"urn,omitempty"` + xxx_hidden_Payload []byte `protobuf:"bytes,2,opt,name=payload,proto3" json:"payload,omitempty"` + xxx_hidden_Target isExecutableStagePayload_WireCoderSetting_Target `protobuf_oneof:"target"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *ExecutableStagePayload_WireCoderSetting) Reset() { + *x = ExecutableStagePayload_WireCoderSetting{} + mi := &file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_msgTypes[104] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *ExecutableStagePayload_WireCoderSetting) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*ExecutableStagePayload_WireCoderSetting) ProtoMessage() {} + +func (x *ExecutableStagePayload_WireCoderSetting) ProtoReflect() protoreflect.Message { + mi := &file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_msgTypes[104] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +func (x *ExecutableStagePayload_WireCoderSetting) GetUrn() string { + if x != nil { + return x.xxx_hidden_Urn + } + return "" +} + +func (x *ExecutableStagePayload_WireCoderSetting) GetPayload() []byte { + if x != nil { + return x.xxx_hidden_Payload + } + return nil +} + +func (x *ExecutableStagePayload_WireCoderSetting) GetInputOrOutputId() string { + if x != nil { + if x, ok := x.xxx_hidden_Target.(*executableStagePayload_WireCoderSetting_InputOrOutputId); ok { + return x.InputOrOutputId + } + } + return "" +} + +func (x *ExecutableStagePayload_WireCoderSetting) GetTimer() *ExecutableStagePayload_TimerId { + if x != nil { + if x, ok := x.xxx_hidden_Target.(*executableStagePayload_WireCoderSetting_Timer); ok { + return x.Timer + } + } + return nil +} + +func (x *ExecutableStagePayload_WireCoderSetting) SetUrn(v string) { + x.xxx_hidden_Urn = v +} + +func (x *ExecutableStagePayload_WireCoderSetting) SetPayload(v []byte) { + if v == nil { + v = []byte{} + } + x.xxx_hidden_Payload = v +} + +func (x *ExecutableStagePayload_WireCoderSetting) SetInputOrOutputId(v string) { + x.xxx_hidden_Target = &executableStagePayload_WireCoderSetting_InputOrOutputId{v} +} + +func (x *ExecutableStagePayload_WireCoderSetting) SetTimer(v *ExecutableStagePayload_TimerId) { + if v == nil { + x.xxx_hidden_Target = nil + return + } + x.xxx_hidden_Target = &executableStagePayload_WireCoderSetting_Timer{v} +} + +func (x *ExecutableStagePayload_WireCoderSetting) HasTarget() bool { + if x == nil { + return false + } + return x.xxx_hidden_Target != nil +} + +func (x *ExecutableStagePayload_WireCoderSetting) HasInputOrOutputId() bool { + if x == nil { + return false + } + _, ok := x.xxx_hidden_Target.(*executableStagePayload_WireCoderSetting_InputOrOutputId) + return ok +} + +func (x *ExecutableStagePayload_WireCoderSetting) HasTimer() bool { + if x == nil { + return false + } + _, ok := x.xxx_hidden_Target.(*executableStagePayload_WireCoderSetting_Timer) + return ok +} + +func (x *ExecutableStagePayload_WireCoderSetting) ClearTarget() { + x.xxx_hidden_Target = nil +} + +func (x *ExecutableStagePayload_WireCoderSetting) ClearInputOrOutputId() { + if _, ok := x.xxx_hidden_Target.(*executableStagePayload_WireCoderSetting_InputOrOutputId); ok { + x.xxx_hidden_Target = nil + } +} + +func (x *ExecutableStagePayload_WireCoderSetting) ClearTimer() { + if _, ok := x.xxx_hidden_Target.(*executableStagePayload_WireCoderSetting_Timer); ok { + x.xxx_hidden_Target = nil + } +} + +const ExecutableStagePayload_WireCoderSetting_Target_not_set_case case_ExecutableStagePayload_WireCoderSetting_Target = 0 +const ExecutableStagePayload_WireCoderSetting_InputOrOutputId_case case_ExecutableStagePayload_WireCoderSetting_Target = 3 +const ExecutableStagePayload_WireCoderSetting_Timer_case case_ExecutableStagePayload_WireCoderSetting_Target = 4 + +func (x *ExecutableStagePayload_WireCoderSetting) WhichTarget() case_ExecutableStagePayload_WireCoderSetting_Target { + if x == nil { + return ExecutableStagePayload_WireCoderSetting_Target_not_set_case + } + switch x.xxx_hidden_Target.(type) { + case *executableStagePayload_WireCoderSetting_InputOrOutputId: + return ExecutableStagePayload_WireCoderSetting_InputOrOutputId_case + case *executableStagePayload_WireCoderSetting_Timer: + return ExecutableStagePayload_WireCoderSetting_Timer_case + default: + return ExecutableStagePayload_WireCoderSetting_Target_not_set_case + } +} + +type ExecutableStagePayload_WireCoderSetting_builder struct { + _ [0]func() // Prevents comparability and use of unkeyed literals for the builder. + + // (Required) The URN of the wire coder. + // Note that only windowed value coder or parameterized windowed value coder are supported. + Urn string + // (Optional) The data specifying any parameters to the URN. If + // the URN is beam:coder:windowed_value:v1, this may be omitted. If the URN is + // beam:coder:param_windowed_value:v1, the payload is an encoded windowed + // value using the beam:coder:windowed_value:v1 coder parameterized by + // a beam:coder:bytes:v1 element coder and the window coder that this + // param_windowed_value coder uses. + Payload []byte + // (Required) The target(PCollection or Timer) this setting applies to. + + // Fields of oneof xxx_hidden_Target: + // The input or output PCollection id this setting applies to. + InputOrOutputId *string + // The timer id this setting applies to. + Timer *ExecutableStagePayload_TimerId + // -- end of xxx_hidden_Target +} + +func (b0 ExecutableStagePayload_WireCoderSetting_builder) Build() *ExecutableStagePayload_WireCoderSetting { + m0 := &ExecutableStagePayload_WireCoderSetting{} + b, x := &b0, m0 + _, _ = b, x + x.xxx_hidden_Urn = b.Urn + x.xxx_hidden_Payload = b.Payload + if b.InputOrOutputId != nil { + x.xxx_hidden_Target = &executableStagePayload_WireCoderSetting_InputOrOutputId{*b.InputOrOutputId} + } + if b.Timer != nil { + x.xxx_hidden_Target = &executableStagePayload_WireCoderSetting_Timer{b.Timer} + } + return m0 +} + +type case_ExecutableStagePayload_WireCoderSetting_Target protoreflect.FieldNumber + +func (x case_ExecutableStagePayload_WireCoderSetting_Target) String() string { + md := file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_msgTypes[104].Descriptor() + if x == 0 { + return "not set" + } + return protoimpl.X.MessageFieldStringOf(md, protoreflect.FieldNumber(x)) +} + +type isExecutableStagePayload_WireCoderSetting_Target interface { + isExecutableStagePayload_WireCoderSetting_Target() +} + +type executableStagePayload_WireCoderSetting_InputOrOutputId struct { + // The input or output PCollection id this setting applies to. + InputOrOutputId string `protobuf:"bytes,3,opt,name=input_or_output_id,json=inputOrOutputId,proto3,oneof"` +} + +type executableStagePayload_WireCoderSetting_Timer struct { + // The timer id this setting applies to. + Timer *ExecutableStagePayload_TimerId `protobuf:"bytes,4,opt,name=timer,proto3,oneof"` +} + +func (*executableStagePayload_WireCoderSetting_InputOrOutputId) isExecutableStagePayload_WireCoderSetting_Target() { +} + +func (*executableStagePayload_WireCoderSetting_Timer) isExecutableStagePayload_WireCoderSetting_Target() { +} + +var file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_extTypes = []protoimpl.ExtensionInfo{ + { + ExtendedType: (*descriptorpb.EnumValueOptions)(nil), + ExtensionType: (*string)(nil), + Field: 185324356, + Name: "org.apache.beam.model.pipeline.v1.beam_urn", + Tag: "bytes,185324356,opt,name=beam_urn", + Filename: "org/apache/beam/model/pipeline/v1/beam_runner_api.proto", + }, + { + ExtendedType: (*descriptorpb.EnumValueOptions)(nil), + ExtensionType: (*string)(nil), + Field: 185324357, + Name: "org.apache.beam.model.pipeline.v1.beam_constant", + Tag: "bytes,185324357,opt,name=beam_constant", + Filename: "org/apache/beam/model/pipeline/v1/beam_runner_api.proto", + }, +} + +// Extension fields to descriptorpb.EnumValueOptions. +var ( + // An extension to be used for specifying the standard URN of various + // pipeline entities, e.g. transforms, functions, coders etc. + // Code should refer to the URNs of those entities by extracting + // it from the (beam_urn) extension, rather than by hard-coding + // the URN. + // + // The recommended pattern for declaring it is (exemplified by coders): + // + // message StandardCoders { + // enum Enum { + // BYTES = 0 [(beam_urn) = "beam:coder:bytes:v1"]; + // ... + // } + // } + // + // If there are multiple categories of entities of this type, use the + // following pattern (exemplified by PTransforms): + // + // message StandardPTransforms { + // enum Primitives { + // ... + // } + // enum Composites { + // ... + // } + // } + // + // optional string beam_urn = 185324356; + E_BeamUrn = &file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_extTypes[0] + // A value to store other constants + // + // optional string beam_constant = 185324357; + E_BeamConstant = &file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_extTypes[1] +) + +var File_org_apache_beam_model_pipeline_v1_beam_runner_api_proto protoreflect.FileDescriptor + +var file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_rawDesc = []byte{ + 0x0a, 0x37, 0x6f, 0x72, 0x67, 0x2f, 0x61, 0x70, 0x61, 0x63, 0x68, 0x65, 0x2f, 0x62, 0x65, 0x61, + 0x6d, 0x2f, 0x6d, 0x6f, 0x64, 0x65, 0x6c, 0x2f, 0x70, 0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e, 0x65, + 0x2f, 0x76, 0x31, 0x2f, 0x62, 0x65, 0x61, 0x6d, 0x5f, 0x72, 0x75, 0x6e, 0x6e, 0x65, 0x72, 0x5f, + 0x61, 0x70, 0x69, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x12, 0x21, 0x6f, 0x72, 0x67, 0x2e, 0x61, + 0x70, 0x61, 0x63, 0x68, 0x65, 0x2e, 0x62, 0x65, 0x61, 0x6d, 0x2e, 0x6d, 0x6f, 0x64, 0x65, 0x6c, + 0x2e, 0x70, 0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e, 0x65, 0x2e, 0x76, 0x31, 0x1a, 0x31, 0x6f, 0x72, + 0x67, 0x2f, 0x61, 0x70, 0x61, 0x63, 0x68, 0x65, 0x2f, 0x62, 0x65, 0x61, 0x6d, 0x2f, 0x6d, 0x6f, + 0x64, 0x65, 0x6c, 0x2f, 0x70, 0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e, 0x65, 0x2f, 0x76, 0x31, 0x2f, + 0x65, 0x6e, 0x64, 0x70, 0x6f, 0x69, 0x6e, 0x74, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x1a, + 0x20, 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2f, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66, + 0x2f, 0x64, 0x65, 0x73, 0x63, 0x72, 0x69, 0x70, 0x74, 0x6f, 0x72, 0x2e, 0x70, 0x72, 0x6f, 0x74, + 0x6f, 0x22, 0xc2, 0x01, 0x0a, 0x0d, 0x42, 0x65, 0x61, 0x6d, 0x43, 0x6f, 0x6e, 0x73, 0x74, 0x61, + 0x6e, 0x74, 0x73, 0x22, 0xb0, 0x01, 0x0a, 0x09, 0x43, 0x6f, 0x6e, 0x73, 0x74, 0x61, 0x6e, 0x74, + 0x73, 0x12, 0x31, 0x0a, 0x14, 0x4d, 0x49, 0x4e, 0x5f, 0x54, 0x49, 0x4d, 0x45, 0x53, 0x54, 0x41, + 0x4d, 0x50, 0x5f, 0x4d, 0x49, 0x4c, 0x4c, 0x49, 0x53, 0x10, 0x00, 0x1a, 0x17, 0xaa, 0xb4, 0xfa, + 0xc2, 0x05, 0x11, 0x2d, 0x39, 0x32, 0x32, 0x33, 0x33, 0x37, 0x32, 0x30, 0x33, 0x36, 0x38, 0x35, + 0x34, 0x37, 0x37, 0x35, 0x12, 0x30, 0x0a, 0x14, 0x4d, 0x41, 0x58, 0x5f, 0x54, 0x49, 0x4d, 0x45, + 0x53, 0x54, 0x41, 0x4d, 0x50, 0x5f, 0x4d, 0x49, 0x4c, 0x4c, 0x49, 0x53, 0x10, 0x01, 0x1a, 0x16, + 0xaa, 0xb4, 0xfa, 0xc2, 0x05, 0x10, 0x39, 0x32, 0x32, 0x33, 0x33, 0x37, 0x32, 0x30, 0x33, 0x36, + 0x38, 0x35, 0x34, 0x37, 0x37, 0x35, 0x12, 0x3e, 0x0a, 0x22, 0x47, 0x4c, 0x4f, 0x42, 0x41, 0x4c, + 0x5f, 0x57, 0x49, 0x4e, 0x44, 0x4f, 0x57, 0x5f, 0x4d, 0x41, 0x58, 0x5f, 0x54, 0x49, 0x4d, 0x45, + 0x53, 0x54, 0x41, 0x4d, 0x50, 0x5f, 0x4d, 0x49, 0x4c, 0x4c, 0x49, 0x53, 0x10, 0x02, 0x1a, 0x16, + 0xaa, 0xb4, 0xfa, 0xc2, 0x05, 0x10, 0x39, 0x32, 0x32, 0x33, 0x33, 0x37, 0x31, 0x39, 0x35, 0x30, + 0x34, 0x35, 0x34, 0x37, 0x37, 0x35, 0x22, 0xb6, 0x08, 0x0a, 0x0a, 0x43, 0x6f, 0x6d, 0x70, 0x6f, + 0x6e, 0x65, 0x6e, 0x74, 0x73, 0x12, 0x5d, 0x0a, 0x0a, 0x74, 0x72, 0x61, 0x6e, 0x73, 0x66, 0x6f, + 0x72, 0x6d, 0x73, 0x18, 0x01, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x3d, 0x2e, 0x6f, 0x72, 0x67, 0x2e, + 0x61, 0x70, 0x61, 0x63, 0x68, 0x65, 0x2e, 0x62, 0x65, 0x61, 0x6d, 0x2e, 0x6d, 0x6f, 0x64, 0x65, + 0x6c, 0x2e, 0x70, 0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x43, 0x6f, + 0x6d, 0x70, 0x6f, 0x6e, 0x65, 0x6e, 0x74, 0x73, 0x2e, 0x54, 0x72, 0x61, 0x6e, 0x73, 0x66, 0x6f, + 0x72, 0x6d, 0x73, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x52, 0x0a, 0x74, 0x72, 0x61, 0x6e, 0x73, 0x66, + 0x6f, 0x72, 0x6d, 0x73, 0x12, 0x63, 0x0a, 0x0c, 0x70, 0x63, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, + 0x69, 0x6f, 0x6e, 0x73, 0x18, 0x02, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x3f, 0x2e, 0x6f, 0x72, 0x67, + 0x2e, 0x61, 0x70, 0x61, 0x63, 0x68, 0x65, 0x2e, 0x62, 0x65, 0x61, 0x6d, 0x2e, 0x6d, 0x6f, 0x64, + 0x65, 0x6c, 0x2e, 0x70, 0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x43, + 0x6f, 0x6d, 0x70, 0x6f, 0x6e, 0x65, 0x6e, 0x74, 0x73, 0x2e, 0x50, 0x63, 0x6f, 0x6c, 0x6c, 0x65, + 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x52, 0x0c, 0x70, 0x63, 0x6f, + 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x12, 0x79, 0x0a, 0x14, 0x77, 0x69, 0x6e, + 0x64, 0x6f, 0x77, 0x69, 0x6e, 0x67, 0x5f, 0x73, 0x74, 0x72, 0x61, 0x74, 0x65, 0x67, 0x69, 0x65, + 0x73, 0x18, 0x03, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x46, 0x2e, 0x6f, 0x72, 0x67, 0x2e, 0x61, 0x70, + 0x61, 0x63, 0x68, 0x65, 0x2e, 0x62, 0x65, 0x61, 0x6d, 0x2e, 0x6d, 0x6f, 0x64, 0x65, 0x6c, 0x2e, + 0x70, 0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x43, 0x6f, 0x6d, 0x70, + 0x6f, 0x6e, 0x65, 0x6e, 0x74, 0x73, 0x2e, 0x57, 0x69, 0x6e, 0x64, 0x6f, 0x77, 0x69, 0x6e, 0x67, + 0x53, 0x74, 0x72, 0x61, 0x74, 0x65, 0x67, 0x69, 0x65, 0x73, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x52, + 0x13, 0x77, 0x69, 0x6e, 0x64, 0x6f, 0x77, 0x69, 0x6e, 0x67, 0x53, 0x74, 0x72, 0x61, 0x74, 0x65, + 0x67, 0x69, 0x65, 0x73, 0x12, 0x51, 0x0a, 0x06, 0x63, 0x6f, 0x64, 0x65, 0x72, 0x73, 0x18, 0x04, + 0x20, 0x03, 0x28, 0x0b, 0x32, 0x39, 0x2e, 0x6f, 0x72, 0x67, 0x2e, 0x61, 0x70, 0x61, 0x63, 0x68, + 0x65, 0x2e, 0x62, 0x65, 0x61, 0x6d, 0x2e, 0x6d, 0x6f, 0x64, 0x65, 0x6c, 0x2e, 0x70, 0x69, 0x70, + 0x65, 0x6c, 0x69, 0x6e, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x43, 0x6f, 0x6d, 0x70, 0x6f, 0x6e, 0x65, + 0x6e, 0x74, 0x73, 0x2e, 0x43, 0x6f, 0x64, 0x65, 0x72, 0x73, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x52, + 0x06, 0x63, 0x6f, 0x64, 0x65, 0x72, 0x73, 0x12, 0x63, 0x0a, 0x0c, 0x65, 0x6e, 0x76, 0x69, 0x72, + 0x6f, 0x6e, 0x6d, 0x65, 0x6e, 0x74, 0x73, 0x18, 0x05, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x3f, 0x2e, + 0x6f, 0x72, 0x67, 0x2e, 0x61, 0x70, 0x61, 0x63, 0x68, 0x65, 0x2e, 0x62, 0x65, 0x61, 0x6d, 0x2e, + 0x6d, 0x6f, 0x64, 0x65, 0x6c, 0x2e, 0x70, 0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e, 0x65, 0x2e, 0x76, + 0x31, 0x2e, 0x43, 0x6f, 0x6d, 0x70, 0x6f, 0x6e, 0x65, 0x6e, 0x74, 0x73, 0x2e, 0x45, 0x6e, 0x76, + 0x69, 0x72, 0x6f, 0x6e, 0x6d, 0x65, 0x6e, 0x74, 0x73, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x52, 0x0c, + 0x65, 0x6e, 0x76, 0x69, 0x72, 0x6f, 0x6e, 0x6d, 0x65, 0x6e, 0x74, 0x73, 0x1a, 0x6c, 0x0a, 0x0f, + 0x54, 0x72, 0x61, 0x6e, 0x73, 0x66, 0x6f, 0x72, 0x6d, 0x73, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x12, + 0x10, 0x0a, 0x03, 0x6b, 0x65, 0x79, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x03, 0x6b, 0x65, + 0x79, 0x12, 0x43, 0x0a, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, + 0x32, 0x2d, 0x2e, 0x6f, 0x72, 0x67, 0x2e, 0x61, 0x70, 0x61, 0x63, 0x68, 0x65, 0x2e, 0x62, 0x65, + 0x61, 0x6d, 0x2e, 0x6d, 0x6f, 0x64, 0x65, 0x6c, 0x2e, 0x70, 0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e, + 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x50, 0x54, 0x72, 0x61, 0x6e, 0x73, 0x66, 0x6f, 0x72, 0x6d, 0x52, + 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x3a, 0x02, 0x38, 0x01, 0x1a, 0x6f, 0x0a, 0x11, 0x50, 0x63, + 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x12, + 0x10, 0x0a, 0x03, 0x6b, 0x65, 0x79, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x03, 0x6b, 0x65, + 0x79, 0x12, 0x44, 0x0a, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, + 0x32, 0x2e, 0x2e, 0x6f, 0x72, 0x67, 0x2e, 0x61, 0x70, 0x61, 0x63, 0x68, 0x65, 0x2e, 0x62, 0x65, + 0x61, 0x6d, 0x2e, 0x6d, 0x6f, 0x64, 0x65, 0x6c, 0x2e, 0x70, 0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e, + 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x50, 0x43, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, + 0x52, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x3a, 0x02, 0x38, 0x01, 0x1a, 0x7c, 0x0a, 0x18, 0x57, + 0x69, 0x6e, 0x64, 0x6f, 0x77, 0x69, 0x6e, 0x67, 0x53, 0x74, 0x72, 0x61, 0x74, 0x65, 0x67, 0x69, + 0x65, 0x73, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x12, 0x10, 0x0a, 0x03, 0x6b, 0x65, 0x79, 0x18, 0x01, + 0x20, 0x01, 0x28, 0x09, 0x52, 0x03, 0x6b, 0x65, 0x79, 0x12, 0x4a, 0x0a, 0x05, 0x76, 0x61, 0x6c, + 0x75, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x34, 0x2e, 0x6f, 0x72, 0x67, 0x2e, 0x61, + 0x70, 0x61, 0x63, 0x68, 0x65, 0x2e, 0x62, 0x65, 0x61, 0x6d, 0x2e, 0x6d, 0x6f, 0x64, 0x65, 0x6c, + 0x2e, 0x70, 0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x57, 0x69, 0x6e, + 0x64, 0x6f, 0x77, 0x69, 0x6e, 0x67, 0x53, 0x74, 0x72, 0x61, 0x74, 0x65, 0x67, 0x79, 0x52, 0x05, + 0x76, 0x61, 0x6c, 0x75, 0x65, 0x3a, 0x02, 0x38, 0x01, 0x1a, 0x63, 0x0a, 0x0b, 0x43, 0x6f, 0x64, + 0x65, 0x72, 0x73, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x12, 0x10, 0x0a, 0x03, 0x6b, 0x65, 0x79, 0x18, + 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x03, 0x6b, 0x65, 0x79, 0x12, 0x3e, 0x0a, 0x05, 0x76, 0x61, + 0x6c, 0x75, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x28, 0x2e, 0x6f, 0x72, 0x67, 0x2e, + 0x61, 0x70, 0x61, 0x63, 0x68, 0x65, 0x2e, 0x62, 0x65, 0x61, 0x6d, 0x2e, 0x6d, 0x6f, 0x64, 0x65, + 0x6c, 0x2e, 0x70, 0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x43, 0x6f, + 0x64, 0x65, 0x72, 0x52, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x3a, 0x02, 0x38, 0x01, 0x1a, 0x6f, + 0x0a, 0x11, 0x45, 0x6e, 0x76, 0x69, 0x72, 0x6f, 0x6e, 0x6d, 0x65, 0x6e, 0x74, 0x73, 0x45, 0x6e, + 0x74, 0x72, 0x79, 0x12, 0x10, 0x0a, 0x03, 0x6b, 0x65, 0x79, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, + 0x52, 0x03, 0x6b, 0x65, 0x79, 0x12, 0x44, 0x0a, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x18, 0x02, + 0x20, 0x01, 0x28, 0x0b, 0x32, 0x2e, 0x2e, 0x6f, 0x72, 0x67, 0x2e, 0x61, 0x70, 0x61, 0x63, 0x68, + 0x65, 0x2e, 0x62, 0x65, 0x61, 0x6d, 0x2e, 0x6d, 0x6f, 0x64, 0x65, 0x6c, 0x2e, 0x70, 0x69, 0x70, + 0x65, 0x6c, 0x69, 0x6e, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x45, 0x6e, 0x76, 0x69, 0x72, 0x6f, 0x6e, + 0x6d, 0x65, 0x6e, 0x74, 0x52, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x3a, 0x02, 0x38, 0x01, 0x22, + 0xfe, 0x01, 0x0a, 0x08, 0x50, 0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e, 0x65, 0x12, 0x4d, 0x0a, 0x0a, + 0x63, 0x6f, 0x6d, 0x70, 0x6f, 0x6e, 0x65, 0x6e, 0x74, 0x73, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, + 0x32, 0x2d, 0x2e, 0x6f, 0x72, 0x67, 0x2e, 0x61, 0x70, 0x61, 0x63, 0x68, 0x65, 0x2e, 0x62, 0x65, + 0x61, 0x6d, 0x2e, 0x6d, 0x6f, 0x64, 0x65, 0x6c, 0x2e, 0x70, 0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e, + 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x43, 0x6f, 0x6d, 0x70, 0x6f, 0x6e, 0x65, 0x6e, 0x74, 0x73, 0x52, + 0x0a, 0x63, 0x6f, 0x6d, 0x70, 0x6f, 0x6e, 0x65, 0x6e, 0x74, 0x73, 0x12, 0x2c, 0x0a, 0x12, 0x72, + 0x6f, 0x6f, 0x74, 0x5f, 0x74, 0x72, 0x61, 0x6e, 0x73, 0x66, 0x6f, 0x72, 0x6d, 0x5f, 0x69, 0x64, + 0x73, 0x18, 0x02, 0x20, 0x03, 0x28, 0x09, 0x52, 0x10, 0x72, 0x6f, 0x6f, 0x74, 0x54, 0x72, 0x61, + 0x6e, 0x73, 0x66, 0x6f, 0x72, 0x6d, 0x49, 0x64, 0x73, 0x12, 0x51, 0x0a, 0x0c, 0x64, 0x69, 0x73, + 0x70, 0x6c, 0x61, 0x79, 0x5f, 0x64, 0x61, 0x74, 0x61, 0x18, 0x03, 0x20, 0x03, 0x28, 0x0b, 0x32, + 0x2e, 0x2e, 0x6f, 0x72, 0x67, 0x2e, 0x61, 0x70, 0x61, 0x63, 0x68, 0x65, 0x2e, 0x62, 0x65, 0x61, + 0x6d, 0x2e, 0x6d, 0x6f, 0x64, 0x65, 0x6c, 0x2e, 0x70, 0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e, 0x65, + 0x2e, 0x76, 0x31, 0x2e, 0x44, 0x69, 0x73, 0x70, 0x6c, 0x61, 0x79, 0x44, 0x61, 0x74, 0x61, 0x52, + 0x0b, 0x64, 0x69, 0x73, 0x70, 0x6c, 0x61, 0x79, 0x44, 0x61, 0x74, 0x61, 0x12, 0x22, 0x0a, 0x0c, + 0x72, 0x65, 0x71, 0x75, 0x69, 0x72, 0x65, 0x6d, 0x65, 0x6e, 0x74, 0x73, 0x18, 0x04, 0x20, 0x03, + 0x28, 0x09, 0x52, 0x0c, 0x72, 0x65, 0x71, 0x75, 0x69, 0x72, 0x65, 0x6d, 0x65, 0x6e, 0x74, 0x73, + 0x22, 0xd4, 0x05, 0x0a, 0x0a, 0x50, 0x54, 0x72, 0x61, 0x6e, 0x73, 0x66, 0x6f, 0x72, 0x6d, 0x12, + 0x1f, 0x0a, 0x0b, 0x75, 0x6e, 0x69, 0x71, 0x75, 0x65, 0x5f, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x05, + 0x20, 0x01, 0x28, 0x09, 0x52, 0x0a, 0x75, 0x6e, 0x69, 0x71, 0x75, 0x65, 0x4e, 0x61, 0x6d, 0x65, + 0x12, 0x43, 0x0a, 0x04, 0x73, 0x70, 0x65, 0x63, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x2f, + 0x2e, 0x6f, 0x72, 0x67, 0x2e, 0x61, 0x70, 0x61, 0x63, 0x68, 0x65, 0x2e, 0x62, 0x65, 0x61, 0x6d, + 0x2e, 0x6d, 0x6f, 0x64, 0x65, 0x6c, 0x2e, 0x70, 0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e, 0x65, 0x2e, + 0x76, 0x31, 0x2e, 0x46, 0x75, 0x6e, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x53, 0x70, 0x65, 0x63, 0x52, + 0x04, 0x73, 0x70, 0x65, 0x63, 0x12, 0x24, 0x0a, 0x0d, 0x73, 0x75, 0x62, 0x74, 0x72, 0x61, 0x6e, + 0x73, 0x66, 0x6f, 0x72, 0x6d, 0x73, 0x18, 0x02, 0x20, 0x03, 0x28, 0x09, 0x52, 0x0d, 0x73, 0x75, + 0x62, 0x74, 0x72, 0x61, 0x6e, 0x73, 0x66, 0x6f, 0x72, 0x6d, 0x73, 0x12, 0x51, 0x0a, 0x06, 0x69, + 0x6e, 0x70, 0x75, 0x74, 0x73, 0x18, 0x03, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x39, 0x2e, 0x6f, 0x72, + 0x67, 0x2e, 0x61, 0x70, 0x61, 0x63, 0x68, 0x65, 0x2e, 0x62, 0x65, 0x61, 0x6d, 0x2e, 0x6d, 0x6f, + 0x64, 0x65, 0x6c, 0x2e, 0x70, 0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e, 0x65, 0x2e, 0x76, 0x31, 0x2e, + 0x50, 0x54, 0x72, 0x61, 0x6e, 0x73, 0x66, 0x6f, 0x72, 0x6d, 0x2e, 0x49, 0x6e, 0x70, 0x75, 0x74, + 0x73, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x52, 0x06, 0x69, 0x6e, 0x70, 0x75, 0x74, 0x73, 0x12, 0x54, + 0x0a, 0x07, 0x6f, 0x75, 0x74, 0x70, 0x75, 0x74, 0x73, 0x18, 0x04, 0x20, 0x03, 0x28, 0x0b, 0x32, + 0x3a, 0x2e, 0x6f, 0x72, 0x67, 0x2e, 0x61, 0x70, 0x61, 0x63, 0x68, 0x65, 0x2e, 0x62, 0x65, 0x61, + 0x6d, 0x2e, 0x6d, 0x6f, 0x64, 0x65, 0x6c, 0x2e, 0x70, 0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e, 0x65, + 0x2e, 0x76, 0x31, 0x2e, 0x50, 0x54, 0x72, 0x61, 0x6e, 0x73, 0x66, 0x6f, 0x72, 0x6d, 0x2e, 0x4f, + 0x75, 0x74, 0x70, 0x75, 0x74, 0x73, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x52, 0x07, 0x6f, 0x75, 0x74, + 0x70, 0x75, 0x74, 0x73, 0x12, 0x51, 0x0a, 0x0c, 0x64, 0x69, 0x73, 0x70, 0x6c, 0x61, 0x79, 0x5f, + 0x64, 0x61, 0x74, 0x61, 0x18, 0x06, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x2e, 0x2e, 0x6f, 0x72, 0x67, + 0x2e, 0x61, 0x70, 0x61, 0x63, 0x68, 0x65, 0x2e, 0x62, 0x65, 0x61, 0x6d, 0x2e, 0x6d, 0x6f, 0x64, + 0x65, 0x6c, 0x2e, 0x70, 0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x44, + 0x69, 0x73, 0x70, 0x6c, 0x61, 0x79, 0x44, 0x61, 0x74, 0x61, 0x52, 0x0b, 0x64, 0x69, 0x73, 0x70, + 0x6c, 0x61, 0x79, 0x44, 0x61, 0x74, 0x61, 0x12, 0x25, 0x0a, 0x0e, 0x65, 0x6e, 0x76, 0x69, 0x72, + 0x6f, 0x6e, 0x6d, 0x65, 0x6e, 0x74, 0x5f, 0x69, 0x64, 0x18, 0x07, 0x20, 0x01, 0x28, 0x09, 0x52, + 0x0d, 0x65, 0x6e, 0x76, 0x69, 0x72, 0x6f, 0x6e, 0x6d, 0x65, 0x6e, 0x74, 0x49, 0x64, 0x12, 0x60, + 0x0a, 0x0b, 0x61, 0x6e, 0x6e, 0x6f, 0x74, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x18, 0x08, 0x20, + 0x03, 0x28, 0x0b, 0x32, 0x3e, 0x2e, 0x6f, 0x72, 0x67, 0x2e, 0x61, 0x70, 0x61, 0x63, 0x68, 0x65, + 0x2e, 0x62, 0x65, 0x61, 0x6d, 0x2e, 0x6d, 0x6f, 0x64, 0x65, 0x6c, 0x2e, 0x70, 0x69, 0x70, 0x65, + 0x6c, 0x69, 0x6e, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x50, 0x54, 0x72, 0x61, 0x6e, 0x73, 0x66, 0x6f, + 0x72, 0x6d, 0x2e, 0x41, 0x6e, 0x6e, 0x6f, 0x74, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x45, 0x6e, + 0x74, 0x72, 0x79, 0x52, 0x0b, 0x61, 0x6e, 0x6e, 0x6f, 0x74, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x73, + 0x1a, 0x39, 0x0a, 0x0b, 0x49, 0x6e, 0x70, 0x75, 0x74, 0x73, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x12, + 0x10, 0x0a, 0x03, 0x6b, 0x65, 0x79, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x03, 0x6b, 0x65, + 0x79, 0x12, 0x14, 0x0a, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, + 0x52, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x3a, 0x02, 0x38, 0x01, 0x1a, 0x3a, 0x0a, 0x0c, 0x4f, + 0x75, 0x74, 0x70, 0x75, 0x74, 0x73, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x12, 0x10, 0x0a, 0x03, 0x6b, + 0x65, 0x79, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x03, 0x6b, 0x65, 0x79, 0x12, 0x14, 0x0a, + 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x05, 0x76, 0x61, + 0x6c, 0x75, 0x65, 0x3a, 0x02, 0x38, 0x01, 0x1a, 0x3e, 0x0a, 0x10, 0x41, 0x6e, 0x6e, 0x6f, 0x74, + 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x12, 0x10, 0x0a, 0x03, 0x6b, + 0x65, 0x79, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x03, 0x6b, 0x65, 0x79, 0x12, 0x14, 0x0a, + 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0c, 0x52, 0x05, 0x76, 0x61, + 0x6c, 0x75, 0x65, 0x3a, 0x02, 0x38, 0x01, 0x22, 0xdb, 0x11, 0x0a, 0x13, 0x53, 0x74, 0x61, 0x6e, + 0x64, 0x61, 0x72, 0x64, 0x50, 0x54, 0x72, 0x61, 0x6e, 0x73, 0x66, 0x6f, 0x72, 0x6d, 0x73, 0x22, + 0xdb, 0x03, 0x0a, 0x0a, 0x50, 0x72, 0x69, 0x6d, 0x69, 0x74, 0x69, 0x76, 0x65, 0x73, 0x12, 0x29, + 0x0a, 0x06, 0x50, 0x41, 0x52, 0x5f, 0x44, 0x4f, 0x10, 0x00, 0x1a, 0x1d, 0xa2, 0xb4, 0xfa, 0xc2, + 0x05, 0x17, 0x62, 0x65, 0x61, 0x6d, 0x3a, 0x74, 0x72, 0x61, 0x6e, 0x73, 0x66, 0x6f, 0x72, 0x6d, + 0x3a, 0x70, 0x61, 0x72, 0x64, 0x6f, 0x3a, 0x76, 0x31, 0x12, 0x2c, 0x0a, 0x07, 0x46, 0x4c, 0x41, + 0x54, 0x54, 0x45, 0x4e, 0x10, 0x01, 0x1a, 0x1f, 0xa2, 0xb4, 0xfa, 0xc2, 0x05, 0x19, 0x62, 0x65, + 0x61, 0x6d, 0x3a, 0x74, 0x72, 0x61, 0x6e, 0x73, 0x66, 0x6f, 0x72, 0x6d, 0x3a, 0x66, 0x6c, 0x61, + 0x74, 0x74, 0x65, 0x6e, 0x3a, 0x76, 0x31, 0x12, 0x36, 0x0a, 0x0c, 0x47, 0x52, 0x4f, 0x55, 0x50, + 0x5f, 0x42, 0x59, 0x5f, 0x4b, 0x45, 0x59, 0x10, 0x02, 0x1a, 0x24, 0xa2, 0xb4, 0xfa, 0xc2, 0x05, + 0x1e, 0x62, 0x65, 0x61, 0x6d, 0x3a, 0x74, 0x72, 0x61, 0x6e, 0x73, 0x66, 0x6f, 0x72, 0x6d, 0x3a, + 0x67, 0x72, 0x6f, 0x75, 0x70, 0x5f, 0x62, 0x79, 0x5f, 0x6b, 0x65, 0x79, 0x3a, 0x76, 0x31, 0x12, + 0x2c, 0x0a, 0x07, 0x49, 0x4d, 0x50, 0x55, 0x4c, 0x53, 0x45, 0x10, 0x03, 0x1a, 0x1f, 0xa2, 0xb4, + 0xfa, 0xc2, 0x05, 0x19, 0x62, 0x65, 0x61, 0x6d, 0x3a, 0x74, 0x72, 0x61, 0x6e, 0x73, 0x66, 0x6f, + 0x72, 0x6d, 0x3a, 0x69, 0x6d, 0x70, 0x75, 0x6c, 0x73, 0x65, 0x3a, 0x76, 0x31, 0x12, 0x37, 0x0a, + 0x0e, 0x41, 0x53, 0x53, 0x49, 0x47, 0x4e, 0x5f, 0x57, 0x49, 0x4e, 0x44, 0x4f, 0x57, 0x53, 0x10, + 0x04, 0x1a, 0x23, 0xa2, 0xb4, 0xfa, 0xc2, 0x05, 0x1d, 0x62, 0x65, 0x61, 0x6d, 0x3a, 0x74, 0x72, + 0x61, 0x6e, 0x73, 0x66, 0x6f, 0x72, 0x6d, 0x3a, 0x77, 0x69, 0x6e, 0x64, 0x6f, 0x77, 0x5f, 0x69, + 0x6e, 0x74, 0x6f, 0x3a, 0x76, 0x31, 0x12, 0x33, 0x0a, 0x0b, 0x54, 0x45, 0x53, 0x54, 0x5f, 0x53, + 0x54, 0x52, 0x45, 0x41, 0x4d, 0x10, 0x05, 0x1a, 0x22, 0xa2, 0xb4, 0xfa, 0xc2, 0x05, 0x1c, 0x62, + 0x65, 0x61, 0x6d, 0x3a, 0x74, 0x72, 0x61, 0x6e, 0x73, 0x66, 0x6f, 0x72, 0x6d, 0x3a, 0x74, 0x65, + 0x73, 0x74, 0x73, 0x74, 0x72, 0x65, 0x61, 0x6d, 0x3a, 0x76, 0x31, 0x12, 0x34, 0x0a, 0x0b, 0x4d, + 0x41, 0x50, 0x5f, 0x57, 0x49, 0x4e, 0x44, 0x4f, 0x57, 0x53, 0x10, 0x06, 0x1a, 0x23, 0xa2, 0xb4, + 0xfa, 0xc2, 0x05, 0x1d, 0x62, 0x65, 0x61, 0x6d, 0x3a, 0x74, 0x72, 0x61, 0x6e, 0x73, 0x66, 0x6f, + 0x72, 0x6d, 0x3a, 0x6d, 0x61, 0x70, 0x5f, 0x77, 0x69, 0x6e, 0x64, 0x6f, 0x77, 0x73, 0x3a, 0x76, + 0x31, 0x12, 0x38, 0x0a, 0x0d, 0x4d, 0x45, 0x52, 0x47, 0x45, 0x5f, 0x57, 0x49, 0x4e, 0x44, 0x4f, + 0x57, 0x53, 0x10, 0x07, 0x1a, 0x25, 0xa2, 0xb4, 0xfa, 0xc2, 0x05, 0x1f, 0x62, 0x65, 0x61, 0x6d, + 0x3a, 0x74, 0x72, 0x61, 0x6e, 0x73, 0x66, 0x6f, 0x72, 0x6d, 0x3a, 0x6d, 0x65, 0x72, 0x67, 0x65, + 0x5f, 0x77, 0x69, 0x6e, 0x64, 0x6f, 0x77, 0x73, 0x3a, 0x76, 0x31, 0x12, 0x30, 0x0a, 0x09, 0x54, + 0x4f, 0x5f, 0x53, 0x54, 0x52, 0x49, 0x4e, 0x47, 0x10, 0x08, 0x1a, 0x21, 0xa2, 0xb4, 0xfa, 0xc2, + 0x05, 0x1b, 0x62, 0x65, 0x61, 0x6d, 0x3a, 0x74, 0x72, 0x61, 0x6e, 0x73, 0x66, 0x6f, 0x72, 0x6d, + 0x3a, 0x74, 0x6f, 0x5f, 0x73, 0x74, 0x72, 0x69, 0x6e, 0x67, 0x3a, 0x76, 0x31, 0x22, 0x74, 0x0a, + 0x14, 0x44, 0x65, 0x70, 0x72, 0x65, 0x63, 0x61, 0x74, 0x65, 0x64, 0x50, 0x72, 0x69, 0x6d, 0x69, + 0x74, 0x69, 0x76, 0x65, 0x73, 0x12, 0x26, 0x0a, 0x04, 0x52, 0x45, 0x41, 0x44, 0x10, 0x00, 0x1a, + 0x1c, 0xa2, 0xb4, 0xfa, 0xc2, 0x05, 0x16, 0x62, 0x65, 0x61, 0x6d, 0x3a, 0x74, 0x72, 0x61, 0x6e, + 0x73, 0x66, 0x6f, 0x72, 0x6d, 0x3a, 0x72, 0x65, 0x61, 0x64, 0x3a, 0x76, 0x31, 0x12, 0x34, 0x0a, + 0x0b, 0x43, 0x52, 0x45, 0x41, 0x54, 0x45, 0x5f, 0x56, 0x49, 0x45, 0x57, 0x10, 0x01, 0x1a, 0x23, + 0xa2, 0xb4, 0xfa, 0xc2, 0x05, 0x1d, 0x62, 0x65, 0x61, 0x6d, 0x3a, 0x74, 0x72, 0x61, 0x6e, 0x73, + 0x66, 0x6f, 0x72, 0x6d, 0x3a, 0x63, 0x72, 0x65, 0x61, 0x74, 0x65, 0x5f, 0x76, 0x69, 0x65, 0x77, + 0x3a, 0x76, 0x31, 0x22, 0x97, 0x05, 0x0a, 0x0a, 0x43, 0x6f, 0x6d, 0x70, 0x6f, 0x73, 0x69, 0x74, + 0x65, 0x73, 0x12, 0x3c, 0x0a, 0x0f, 0x43, 0x4f, 0x4d, 0x42, 0x49, 0x4e, 0x45, 0x5f, 0x50, 0x45, + 0x52, 0x5f, 0x4b, 0x45, 0x59, 0x10, 0x00, 0x1a, 0x27, 0xa2, 0xb4, 0xfa, 0xc2, 0x05, 0x21, 0x62, + 0x65, 0x61, 0x6d, 0x3a, 0x74, 0x72, 0x61, 0x6e, 0x73, 0x66, 0x6f, 0x72, 0x6d, 0x3a, 0x63, 0x6f, + 0x6d, 0x62, 0x69, 0x6e, 0x65, 0x5f, 0x70, 0x65, 0x72, 0x5f, 0x6b, 0x65, 0x79, 0x3a, 0x76, 0x31, + 0x12, 0x3e, 0x0a, 0x10, 0x43, 0x4f, 0x4d, 0x42, 0x49, 0x4e, 0x45, 0x5f, 0x47, 0x4c, 0x4f, 0x42, + 0x41, 0x4c, 0x4c, 0x59, 0x10, 0x01, 0x1a, 0x28, 0xa2, 0xb4, 0xfa, 0xc2, 0x05, 0x22, 0x62, 0x65, + 0x61, 0x6d, 0x3a, 0x74, 0x72, 0x61, 0x6e, 0x73, 0x66, 0x6f, 0x72, 0x6d, 0x3a, 0x63, 0x6f, 0x6d, + 0x62, 0x69, 0x6e, 0x65, 0x5f, 0x67, 0x6c, 0x6f, 0x62, 0x61, 0x6c, 0x6c, 0x79, 0x3a, 0x76, 0x31, + 0x12, 0x30, 0x0a, 0x09, 0x52, 0x45, 0x53, 0x48, 0x55, 0x46, 0x46, 0x4c, 0x45, 0x10, 0x02, 0x1a, + 0x21, 0xa2, 0xb4, 0xfa, 0xc2, 0x05, 0x1b, 0x62, 0x65, 0x61, 0x6d, 0x3a, 0x74, 0x72, 0x61, 0x6e, + 0x73, 0x66, 0x6f, 0x72, 0x6d, 0x3a, 0x72, 0x65, 0x73, 0x68, 0x75, 0x66, 0x66, 0x6c, 0x65, 0x3a, + 0x76, 0x31, 0x12, 0x34, 0x0a, 0x0b, 0x57, 0x52, 0x49, 0x54, 0x45, 0x5f, 0x46, 0x49, 0x4c, 0x45, + 0x53, 0x10, 0x03, 0x1a, 0x23, 0xa2, 0xb4, 0xfa, 0xc2, 0x05, 0x1d, 0x62, 0x65, 0x61, 0x6d, 0x3a, + 0x74, 0x72, 0x61, 0x6e, 0x73, 0x66, 0x6f, 0x72, 0x6d, 0x3a, 0x77, 0x72, 0x69, 0x74, 0x65, 0x5f, + 0x66, 0x69, 0x6c, 0x65, 0x73, 0x3a, 0x76, 0x31, 0x12, 0x34, 0x0a, 0x0b, 0x50, 0x55, 0x42, 0x53, + 0x55, 0x42, 0x5f, 0x52, 0x45, 0x41, 0x44, 0x10, 0x04, 0x1a, 0x23, 0xa2, 0xb4, 0xfa, 0xc2, 0x05, + 0x1d, 0x62, 0x65, 0x61, 0x6d, 0x3a, 0x74, 0x72, 0x61, 0x6e, 0x73, 0x66, 0x6f, 0x72, 0x6d, 0x3a, + 0x70, 0x75, 0x62, 0x73, 0x75, 0x62, 0x5f, 0x72, 0x65, 0x61, 0x64, 0x3a, 0x76, 0x31, 0x12, 0x36, + 0x0a, 0x0c, 0x50, 0x55, 0x42, 0x53, 0x55, 0x42, 0x5f, 0x57, 0x52, 0x49, 0x54, 0x45, 0x10, 0x05, + 0x1a, 0x24, 0xa2, 0xb4, 0xfa, 0xc2, 0x05, 0x1e, 0x62, 0x65, 0x61, 0x6d, 0x3a, 0x74, 0x72, 0x61, + 0x6e, 0x73, 0x66, 0x6f, 0x72, 0x6d, 0x3a, 0x70, 0x75, 0x62, 0x73, 0x75, 0x62, 0x5f, 0x77, 0x72, + 0x69, 0x74, 0x65, 0x3a, 0x76, 0x31, 0x12, 0x39, 0x0a, 0x0f, 0x50, 0x55, 0x42, 0x53, 0x55, 0x42, + 0x5f, 0x57, 0x52, 0x49, 0x54, 0x45, 0x5f, 0x56, 0x32, 0x10, 0x07, 0x1a, 0x24, 0xa2, 0xb4, 0xfa, + 0xc2, 0x05, 0x1e, 0x62, 0x65, 0x61, 0x6d, 0x3a, 0x74, 0x72, 0x61, 0x6e, 0x73, 0x66, 0x6f, 0x72, + 0x6d, 0x3a, 0x70, 0x75, 0x62, 0x73, 0x75, 0x62, 0x5f, 0x77, 0x72, 0x69, 0x74, 0x65, 0x3a, 0x76, + 0x32, 0x12, 0x64, 0x0a, 0x23, 0x47, 0x52, 0x4f, 0x55, 0x50, 0x5f, 0x49, 0x4e, 0x54, 0x4f, 0x5f, + 0x42, 0x41, 0x54, 0x43, 0x48, 0x45, 0x53, 0x5f, 0x57, 0x49, 0x54, 0x48, 0x5f, 0x53, 0x48, 0x41, + 0x52, 0x44, 0x45, 0x44, 0x5f, 0x4b, 0x45, 0x59, 0x10, 0x06, 0x1a, 0x3b, 0xa2, 0xb4, 0xfa, 0xc2, + 0x05, 0x35, 0x62, 0x65, 0x61, 0x6d, 0x3a, 0x74, 0x72, 0x61, 0x6e, 0x73, 0x66, 0x6f, 0x72, 0x6d, + 0x3a, 0x67, 0x72, 0x6f, 0x75, 0x70, 0x5f, 0x69, 0x6e, 0x74, 0x6f, 0x5f, 0x62, 0x61, 0x74, 0x63, + 0x68, 0x65, 0x73, 0x5f, 0x77, 0x69, 0x74, 0x68, 0x5f, 0x73, 0x68, 0x61, 0x72, 0x64, 0x65, 0x64, + 0x5f, 0x6b, 0x65, 0x79, 0x3a, 0x76, 0x31, 0x12, 0x44, 0x0a, 0x13, 0x52, 0x45, 0x44, 0x49, 0x53, + 0x54, 0x52, 0x49, 0x42, 0x55, 0x54, 0x45, 0x5f, 0x42, 0x59, 0x5f, 0x4b, 0x45, 0x59, 0x10, 0x08, + 0x1a, 0x2b, 0xa2, 0xb4, 0xfa, 0xc2, 0x05, 0x25, 0x62, 0x65, 0x61, 0x6d, 0x3a, 0x74, 0x72, 0x61, + 0x6e, 0x73, 0x66, 0x6f, 0x72, 0x6d, 0x3a, 0x72, 0x65, 0x64, 0x69, 0x73, 0x74, 0x72, 0x69, 0x62, + 0x75, 0x74, 0x65, 0x5f, 0x62, 0x79, 0x5f, 0x6b, 0x65, 0x79, 0x3a, 0x76, 0x31, 0x12, 0x4e, 0x0a, + 0x18, 0x52, 0x45, 0x44, 0x49, 0x53, 0x54, 0x52, 0x49, 0x42, 0x55, 0x54, 0x45, 0x5f, 0x41, 0x52, + 0x42, 0x49, 0x54, 0x52, 0x41, 0x52, 0x49, 0x4c, 0x59, 0x10, 0x09, 0x1a, 0x30, 0xa2, 0xb4, 0xfa, + 0xc2, 0x05, 0x2a, 0x62, 0x65, 0x61, 0x6d, 0x3a, 0x74, 0x72, 0x61, 0x6e, 0x73, 0x66, 0x6f, 0x72, + 0x6d, 0x3a, 0x72, 0x65, 0x64, 0x69, 0x73, 0x74, 0x72, 0x69, 0x62, 0x75, 0x74, 0x65, 0x5f, 0x61, + 0x72, 0x62, 0x69, 0x74, 0x72, 0x61, 0x72, 0x69, 0x6c, 0x79, 0x3a, 0x76, 0x31, 0x22, 0xe3, 0x03, + 0x0a, 0x11, 0x43, 0x6f, 0x6d, 0x62, 0x69, 0x6e, 0x65, 0x43, 0x6f, 0x6d, 0x70, 0x6f, 0x6e, 0x65, + 0x6e, 0x74, 0x73, 0x12, 0x52, 0x0a, 0x1a, 0x43, 0x4f, 0x4d, 0x42, 0x49, 0x4e, 0x45, 0x5f, 0x50, + 0x45, 0x52, 0x5f, 0x4b, 0x45, 0x59, 0x5f, 0x50, 0x52, 0x45, 0x43, 0x4f, 0x4d, 0x42, 0x49, 0x4e, + 0x45, 0x10, 0x00, 0x1a, 0x32, 0xa2, 0xb4, 0xfa, 0xc2, 0x05, 0x2c, 0x62, 0x65, 0x61, 0x6d, 0x3a, + 0x74, 0x72, 0x61, 0x6e, 0x73, 0x66, 0x6f, 0x72, 0x6d, 0x3a, 0x63, 0x6f, 0x6d, 0x62, 0x69, 0x6e, + 0x65, 0x5f, 0x70, 0x65, 0x72, 0x5f, 0x6b, 0x65, 0x79, 0x5f, 0x70, 0x72, 0x65, 0x63, 0x6f, 0x6d, + 0x62, 0x69, 0x6e, 0x65, 0x3a, 0x76, 0x31, 0x12, 0x62, 0x0a, 0x22, 0x43, 0x4f, 0x4d, 0x42, 0x49, + 0x4e, 0x45, 0x5f, 0x50, 0x45, 0x52, 0x5f, 0x4b, 0x45, 0x59, 0x5f, 0x4d, 0x45, 0x52, 0x47, 0x45, + 0x5f, 0x41, 0x43, 0x43, 0x55, 0x4d, 0x55, 0x4c, 0x41, 0x54, 0x4f, 0x52, 0x53, 0x10, 0x01, 0x1a, + 0x3a, 0xa2, 0xb4, 0xfa, 0xc2, 0x05, 0x34, 0x62, 0x65, 0x61, 0x6d, 0x3a, 0x74, 0x72, 0x61, 0x6e, + 0x73, 0x66, 0x6f, 0x72, 0x6d, 0x3a, 0x63, 0x6f, 0x6d, 0x62, 0x69, 0x6e, 0x65, 0x5f, 0x70, 0x65, + 0x72, 0x5f, 0x6b, 0x65, 0x79, 0x5f, 0x6d, 0x65, 0x72, 0x67, 0x65, 0x5f, 0x61, 0x63, 0x63, 0x75, + 0x6d, 0x75, 0x6c, 0x61, 0x74, 0x6f, 0x72, 0x73, 0x3a, 0x76, 0x31, 0x12, 0x5c, 0x0a, 0x1f, 0x43, + 0x4f, 0x4d, 0x42, 0x49, 0x4e, 0x45, 0x5f, 0x50, 0x45, 0x52, 0x5f, 0x4b, 0x45, 0x59, 0x5f, 0x45, + 0x58, 0x54, 0x52, 0x41, 0x43, 0x54, 0x5f, 0x4f, 0x55, 0x54, 0x50, 0x55, 0x54, 0x53, 0x10, 0x02, + 0x1a, 0x37, 0xa2, 0xb4, 0xfa, 0xc2, 0x05, 0x31, 0x62, 0x65, 0x61, 0x6d, 0x3a, 0x74, 0x72, 0x61, + 0x6e, 0x73, 0x66, 0x6f, 0x72, 0x6d, 0x3a, 0x63, 0x6f, 0x6d, 0x62, 0x69, 0x6e, 0x65, 0x5f, 0x70, + 0x65, 0x72, 0x5f, 0x6b, 0x65, 0x79, 0x5f, 0x65, 0x78, 0x74, 0x72, 0x61, 0x63, 0x74, 0x5f, 0x6f, + 0x75, 0x74, 0x70, 0x75, 0x74, 0x73, 0x3a, 0x76, 0x31, 0x12, 0x4a, 0x0a, 0x16, 0x43, 0x4f, 0x4d, + 0x42, 0x49, 0x4e, 0x45, 0x5f, 0x47, 0x52, 0x4f, 0x55, 0x50, 0x45, 0x44, 0x5f, 0x56, 0x41, 0x4c, + 0x55, 0x45, 0x53, 0x10, 0x03, 0x1a, 0x2e, 0xa2, 0xb4, 0xfa, 0xc2, 0x05, 0x28, 0x62, 0x65, 0x61, + 0x6d, 0x3a, 0x74, 0x72, 0x61, 0x6e, 0x73, 0x66, 0x6f, 0x72, 0x6d, 0x3a, 0x63, 0x6f, 0x6d, 0x62, + 0x69, 0x6e, 0x65, 0x5f, 0x67, 0x72, 0x6f, 0x75, 0x70, 0x65, 0x64, 0x5f, 0x76, 0x61, 0x6c, 0x75, + 0x65, 0x73, 0x3a, 0x76, 0x31, 0x12, 0x6c, 0x0a, 0x27, 0x43, 0x4f, 0x4d, 0x42, 0x49, 0x4e, 0x45, + 0x5f, 0x50, 0x45, 0x52, 0x5f, 0x4b, 0x45, 0x59, 0x5f, 0x43, 0x4f, 0x4e, 0x56, 0x45, 0x52, 0x54, + 0x5f, 0x54, 0x4f, 0x5f, 0x41, 0x43, 0x43, 0x55, 0x4d, 0x55, 0x4c, 0x41, 0x54, 0x4f, 0x52, 0x53, + 0x10, 0x04, 0x1a, 0x3f, 0xa2, 0xb4, 0xfa, 0xc2, 0x05, 0x39, 0x62, 0x65, 0x61, 0x6d, 0x3a, 0x74, + 0x72, 0x61, 0x6e, 0x73, 0x66, 0x6f, 0x72, 0x6d, 0x3a, 0x63, 0x6f, 0x6d, 0x62, 0x69, 0x6e, 0x65, + 0x5f, 0x70, 0x65, 0x72, 0x5f, 0x6b, 0x65, 0x79, 0x5f, 0x63, 0x6f, 0x6e, 0x76, 0x65, 0x72, 0x74, + 0x5f, 0x74, 0x6f, 0x5f, 0x61, 0x63, 0x63, 0x75, 0x6d, 0x75, 0x6c, 0x61, 0x74, 0x6f, 0x72, 0x73, + 0x3a, 0x76, 0x31, 0x22, 0x8d, 0x03, 0x0a, 0x19, 0x53, 0x70, 0x6c, 0x69, 0x74, 0x74, 0x61, 0x62, + 0x6c, 0x65, 0x50, 0x61, 0x72, 0x44, 0x6f, 0x43, 0x6f, 0x6d, 0x70, 0x6f, 0x6e, 0x65, 0x6e, 0x74, + 0x73, 0x12, 0x4c, 0x0a, 0x15, 0x50, 0x41, 0x49, 0x52, 0x5f, 0x57, 0x49, 0x54, 0x48, 0x5f, 0x52, + 0x45, 0x53, 0x54, 0x52, 0x49, 0x43, 0x54, 0x49, 0x4f, 0x4e, 0x10, 0x00, 0x1a, 0x31, 0xa2, 0xb4, + 0xfa, 0xc2, 0x05, 0x2b, 0x62, 0x65, 0x61, 0x6d, 0x3a, 0x74, 0x72, 0x61, 0x6e, 0x73, 0x66, 0x6f, + 0x72, 0x6d, 0x3a, 0x73, 0x64, 0x66, 0x5f, 0x70, 0x61, 0x69, 0x72, 0x5f, 0x77, 0x69, 0x74, 0x68, + 0x5f, 0x72, 0x65, 0x73, 0x74, 0x72, 0x69, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x3a, 0x76, 0x31, 0x12, + 0x58, 0x0a, 0x1b, 0x53, 0x50, 0x4c, 0x49, 0x54, 0x5f, 0x41, 0x4e, 0x44, 0x5f, 0x53, 0x49, 0x5a, + 0x45, 0x5f, 0x52, 0x45, 0x53, 0x54, 0x52, 0x49, 0x43, 0x54, 0x49, 0x4f, 0x4e, 0x53, 0x10, 0x01, + 0x1a, 0x37, 0xa2, 0xb4, 0xfa, 0xc2, 0x05, 0x31, 0x62, 0x65, 0x61, 0x6d, 0x3a, 0x74, 0x72, 0x61, + 0x6e, 0x73, 0x66, 0x6f, 0x72, 0x6d, 0x3a, 0x73, 0x64, 0x66, 0x5f, 0x73, 0x70, 0x6c, 0x69, 0x74, + 0x5f, 0x61, 0x6e, 0x64, 0x5f, 0x73, 0x69, 0x7a, 0x65, 0x5f, 0x72, 0x65, 0x73, 0x74, 0x72, 0x69, + 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x3a, 0x76, 0x31, 0x12, 0x6f, 0x0a, 0x27, 0x50, 0x52, 0x4f, + 0x43, 0x45, 0x53, 0x53, 0x5f, 0x53, 0x49, 0x5a, 0x45, 0x44, 0x5f, 0x45, 0x4c, 0x45, 0x4d, 0x45, + 0x4e, 0x54, 0x53, 0x5f, 0x41, 0x4e, 0x44, 0x5f, 0x52, 0x45, 0x53, 0x54, 0x52, 0x49, 0x43, 0x54, + 0x49, 0x4f, 0x4e, 0x53, 0x10, 0x02, 0x1a, 0x42, 0xa2, 0xb4, 0xfa, 0xc2, 0x05, 0x3c, 0x62, 0x65, + 0x61, 0x6d, 0x3a, 0x74, 0x72, 0x61, 0x6e, 0x73, 0x66, 0x6f, 0x72, 0x6d, 0x3a, 0x73, 0x64, 0x66, + 0x5f, 0x70, 0x72, 0x6f, 0x63, 0x65, 0x73, 0x73, 0x5f, 0x73, 0x69, 0x7a, 0x65, 0x64, 0x5f, 0x65, + 0x6c, 0x65, 0x6d, 0x65, 0x6e, 0x74, 0x5f, 0x61, 0x6e, 0x64, 0x5f, 0x72, 0x65, 0x73, 0x74, 0x72, + 0x69, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x3a, 0x76, 0x31, 0x12, 0x57, 0x0a, 0x1a, 0x54, 0x52, + 0x55, 0x4e, 0x43, 0x41, 0x54, 0x45, 0x5f, 0x53, 0x49, 0x5a, 0x45, 0x44, 0x5f, 0x52, 0x45, 0x53, + 0x54, 0x52, 0x49, 0x43, 0x54, 0x49, 0x4f, 0x4e, 0x10, 0x03, 0x1a, 0x37, 0xa2, 0xb4, 0xfa, 0xc2, + 0x05, 0x31, 0x62, 0x65, 0x61, 0x6d, 0x3a, 0x74, 0x72, 0x61, 0x6e, 0x73, 0x66, 0x6f, 0x72, 0x6d, + 0x3a, 0x73, 0x64, 0x66, 0x5f, 0x74, 0x72, 0x75, 0x6e, 0x63, 0x61, 0x74, 0x65, 0x5f, 0x73, 0x69, + 0x7a, 0x65, 0x64, 0x5f, 0x72, 0x65, 0x73, 0x74, 0x72, 0x69, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x73, + 0x3a, 0x76, 0x31, 0x22, 0x60, 0x0a, 0x1a, 0x47, 0x72, 0x6f, 0x75, 0x70, 0x49, 0x6e, 0x74, 0x6f, + 0x42, 0x61, 0x74, 0x63, 0x68, 0x65, 0x73, 0x43, 0x6f, 0x6d, 0x70, 0x6f, 0x6e, 0x65, 0x6e, 0x74, + 0x73, 0x12, 0x42, 0x0a, 0x12, 0x47, 0x52, 0x4f, 0x55, 0x50, 0x5f, 0x49, 0x4e, 0x54, 0x4f, 0x5f, + 0x42, 0x41, 0x54, 0x43, 0x48, 0x45, 0x53, 0x10, 0x00, 0x1a, 0x2a, 0xa2, 0xb4, 0xfa, 0xc2, 0x05, + 0x24, 0x62, 0x65, 0x61, 0x6d, 0x3a, 0x74, 0x72, 0x61, 0x6e, 0x73, 0x66, 0x6f, 0x72, 0x6d, 0x3a, + 0x67, 0x72, 0x6f, 0x75, 0x70, 0x5f, 0x69, 0x6e, 0x74, 0x6f, 0x5f, 0x62, 0x61, 0x74, 0x63, 0x68, + 0x65, 0x73, 0x3a, 0x76, 0x31, 0x22, 0x82, 0x01, 0x0a, 0x16, 0x53, 0x74, 0x61, 0x6e, 0x64, 0x61, + 0x72, 0x64, 0x53, 0x69, 0x64, 0x65, 0x49, 0x6e, 0x70, 0x75, 0x74, 0x54, 0x79, 0x70, 0x65, 0x73, + 0x22, 0x68, 0x0a, 0x04, 0x45, 0x6e, 0x75, 0x6d, 0x12, 0x2f, 0x0a, 0x08, 0x49, 0x54, 0x45, 0x52, + 0x41, 0x42, 0x4c, 0x45, 0x10, 0x00, 0x1a, 0x21, 0xa2, 0xb4, 0xfa, 0xc2, 0x05, 0x1b, 0x62, 0x65, + 0x61, 0x6d, 0x3a, 0x73, 0x69, 0x64, 0x65, 0x5f, 0x69, 0x6e, 0x70, 0x75, 0x74, 0x3a, 0x69, 0x74, + 0x65, 0x72, 0x61, 0x62, 0x6c, 0x65, 0x3a, 0x76, 0x31, 0x12, 0x2f, 0x0a, 0x08, 0x4d, 0x55, 0x4c, + 0x54, 0x49, 0x4d, 0x41, 0x50, 0x10, 0x01, 0x1a, 0x21, 0xa2, 0xb4, 0xfa, 0xc2, 0x05, 0x1b, 0x62, + 0x65, 0x61, 0x6d, 0x3a, 0x73, 0x69, 0x64, 0x65, 0x5f, 0x69, 0x6e, 0x70, 0x75, 0x74, 0x3a, 0x6d, + 0x75, 0x6c, 0x74, 0x69, 0x6d, 0x61, 0x70, 0x3a, 0x76, 0x31, 0x22, 0xb2, 0x01, 0x0a, 0x16, 0x53, + 0x74, 0x61, 0x6e, 0x64, 0x61, 0x72, 0x64, 0x55, 0x73, 0x65, 0x72, 0x53, 0x74, 0x61, 0x74, 0x65, + 0x54, 0x79, 0x70, 0x65, 0x73, 0x22, 0x97, 0x01, 0x0a, 0x04, 0x45, 0x6e, 0x75, 0x6d, 0x12, 0x25, + 0x0a, 0x03, 0x42, 0x41, 0x47, 0x10, 0x00, 0x1a, 0x1c, 0xa2, 0xb4, 0xfa, 0xc2, 0x05, 0x16, 0x62, + 0x65, 0x61, 0x6d, 0x3a, 0x75, 0x73, 0x65, 0x72, 0x5f, 0x73, 0x74, 0x61, 0x74, 0x65, 0x3a, 0x62, + 0x61, 0x67, 0x3a, 0x76, 0x31, 0x12, 0x2f, 0x0a, 0x08, 0x4d, 0x55, 0x4c, 0x54, 0x49, 0x4d, 0x41, + 0x50, 0x10, 0x01, 0x1a, 0x21, 0xa2, 0xb4, 0xfa, 0xc2, 0x05, 0x1b, 0x62, 0x65, 0x61, 0x6d, 0x3a, + 0x75, 0x73, 0x65, 0x72, 0x5f, 0x73, 0x74, 0x61, 0x74, 0x65, 0x3a, 0x6d, 0x75, 0x6c, 0x74, 0x69, + 0x6d, 0x61, 0x70, 0x3a, 0x76, 0x31, 0x12, 0x37, 0x0a, 0x0c, 0x4f, 0x52, 0x44, 0x45, 0x52, 0x45, + 0x44, 0x5f, 0x4c, 0x49, 0x53, 0x54, 0x10, 0x02, 0x1a, 0x25, 0xa2, 0xb4, 0xfa, 0xc2, 0x05, 0x1f, + 0x62, 0x65, 0x61, 0x6d, 0x3a, 0x75, 0x73, 0x65, 0x72, 0x5f, 0x73, 0x74, 0x61, 0x74, 0x65, 0x3a, + 0x6f, 0x72, 0x64, 0x65, 0x72, 0x65, 0x64, 0x5f, 0x6c, 0x69, 0x73, 0x74, 0x3a, 0x76, 0x31, 0x22, + 0xa2, 0x02, 0x0a, 0x0b, 0x50, 0x43, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x12, + 0x1f, 0x0a, 0x0b, 0x75, 0x6e, 0x69, 0x71, 0x75, 0x65, 0x5f, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x01, + 0x20, 0x01, 0x28, 0x09, 0x52, 0x0a, 0x75, 0x6e, 0x69, 0x71, 0x75, 0x65, 0x4e, 0x61, 0x6d, 0x65, + 0x12, 0x19, 0x0a, 0x08, 0x63, 0x6f, 0x64, 0x65, 0x72, 0x5f, 0x69, 0x64, 0x18, 0x02, 0x20, 0x01, + 0x28, 0x09, 0x52, 0x07, 0x63, 0x6f, 0x64, 0x65, 0x72, 0x49, 0x64, 0x12, 0x50, 0x0a, 0x0a, 0x69, + 0x73, 0x5f, 0x62, 0x6f, 0x75, 0x6e, 0x64, 0x65, 0x64, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0e, 0x32, + 0x31, 0x2e, 0x6f, 0x72, 0x67, 0x2e, 0x61, 0x70, 0x61, 0x63, 0x68, 0x65, 0x2e, 0x62, 0x65, 0x61, + 0x6d, 0x2e, 0x6d, 0x6f, 0x64, 0x65, 0x6c, 0x2e, 0x70, 0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e, 0x65, + 0x2e, 0x76, 0x31, 0x2e, 0x49, 0x73, 0x42, 0x6f, 0x75, 0x6e, 0x64, 0x65, 0x64, 0x2e, 0x45, 0x6e, + 0x75, 0x6d, 0x52, 0x09, 0x69, 0x73, 0x42, 0x6f, 0x75, 0x6e, 0x64, 0x65, 0x64, 0x12, 0x32, 0x0a, + 0x15, 0x77, 0x69, 0x6e, 0x64, 0x6f, 0x77, 0x69, 0x6e, 0x67, 0x5f, 0x73, 0x74, 0x72, 0x61, 0x74, + 0x65, 0x67, 0x79, 0x5f, 0x69, 0x64, 0x18, 0x04, 0x20, 0x01, 0x28, 0x09, 0x52, 0x13, 0x77, 0x69, + 0x6e, 0x64, 0x6f, 0x77, 0x69, 0x6e, 0x67, 0x53, 0x74, 0x72, 0x61, 0x74, 0x65, 0x67, 0x79, 0x49, + 0x64, 0x12, 0x51, 0x0a, 0x0c, 0x64, 0x69, 0x73, 0x70, 0x6c, 0x61, 0x79, 0x5f, 0x64, 0x61, 0x74, + 0x61, 0x18, 0x05, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x2e, 0x2e, 0x6f, 0x72, 0x67, 0x2e, 0x61, 0x70, + 0x61, 0x63, 0x68, 0x65, 0x2e, 0x62, 0x65, 0x61, 0x6d, 0x2e, 0x6d, 0x6f, 0x64, 0x65, 0x6c, 0x2e, + 0x70, 0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x44, 0x69, 0x73, 0x70, + 0x6c, 0x61, 0x79, 0x44, 0x61, 0x74, 0x61, 0x52, 0x0b, 0x64, 0x69, 0x73, 0x70, 0x6c, 0x61, 0x79, + 0x44, 0x61, 0x74, 0x61, 0x22, 0x91, 0x08, 0x0a, 0x0c, 0x50, 0x61, 0x72, 0x44, 0x6f, 0x50, 0x61, + 0x79, 0x6c, 0x6f, 0x61, 0x64, 0x12, 0x44, 0x0a, 0x05, 0x64, 0x6f, 0x5f, 0x66, 0x6e, 0x18, 0x01, + 0x20, 0x01, 0x28, 0x0b, 0x32, 0x2f, 0x2e, 0x6f, 0x72, 0x67, 0x2e, 0x61, 0x70, 0x61, 0x63, 0x68, + 0x65, 0x2e, 0x62, 0x65, 0x61, 0x6d, 0x2e, 0x6d, 0x6f, 0x64, 0x65, 0x6c, 0x2e, 0x70, 0x69, 0x70, + 0x65, 0x6c, 0x69, 0x6e, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x46, 0x75, 0x6e, 0x63, 0x74, 0x69, 0x6f, + 0x6e, 0x53, 0x70, 0x65, 0x63, 0x52, 0x04, 0x64, 0x6f, 0x46, 0x6e, 0x12, 0x60, 0x0a, 0x0b, 0x73, + 0x69, 0x64, 0x65, 0x5f, 0x69, 0x6e, 0x70, 0x75, 0x74, 0x73, 0x18, 0x03, 0x20, 0x03, 0x28, 0x0b, + 0x32, 0x3f, 0x2e, 0x6f, 0x72, 0x67, 0x2e, 0x61, 0x70, 0x61, 0x63, 0x68, 0x65, 0x2e, 0x62, 0x65, + 0x61, 0x6d, 0x2e, 0x6d, 0x6f, 0x64, 0x65, 0x6c, 0x2e, 0x70, 0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e, + 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x50, 0x61, 0x72, 0x44, 0x6f, 0x50, 0x61, 0x79, 0x6c, 0x6f, 0x61, + 0x64, 0x2e, 0x53, 0x69, 0x64, 0x65, 0x49, 0x6e, 0x70, 0x75, 0x74, 0x73, 0x45, 0x6e, 0x74, 0x72, + 0x79, 0x52, 0x0a, 0x73, 0x69, 0x64, 0x65, 0x49, 0x6e, 0x70, 0x75, 0x74, 0x73, 0x12, 0x60, 0x0a, + 0x0b, 0x73, 0x74, 0x61, 0x74, 0x65, 0x5f, 0x73, 0x70, 0x65, 0x63, 0x73, 0x18, 0x04, 0x20, 0x03, + 0x28, 0x0b, 0x32, 0x3f, 0x2e, 0x6f, 0x72, 0x67, 0x2e, 0x61, 0x70, 0x61, 0x63, 0x68, 0x65, 0x2e, + 0x62, 0x65, 0x61, 0x6d, 0x2e, 0x6d, 0x6f, 0x64, 0x65, 0x6c, 0x2e, 0x70, 0x69, 0x70, 0x65, 0x6c, + 0x69, 0x6e, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x50, 0x61, 0x72, 0x44, 0x6f, 0x50, 0x61, 0x79, 0x6c, + 0x6f, 0x61, 0x64, 0x2e, 0x53, 0x74, 0x61, 0x74, 0x65, 0x53, 0x70, 0x65, 0x63, 0x73, 0x45, 0x6e, + 0x74, 0x72, 0x79, 0x52, 0x0a, 0x73, 0x74, 0x61, 0x74, 0x65, 0x53, 0x70, 0x65, 0x63, 0x73, 0x12, + 0x73, 0x0a, 0x12, 0x74, 0x69, 0x6d, 0x65, 0x72, 0x5f, 0x66, 0x61, 0x6d, 0x69, 0x6c, 0x79, 0x5f, + 0x73, 0x70, 0x65, 0x63, 0x73, 0x18, 0x09, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x45, 0x2e, 0x6f, 0x72, + 0x67, 0x2e, 0x61, 0x70, 0x61, 0x63, 0x68, 0x65, 0x2e, 0x62, 0x65, 0x61, 0x6d, 0x2e, 0x6d, 0x6f, + 0x64, 0x65, 0x6c, 0x2e, 0x70, 0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e, 0x65, 0x2e, 0x76, 0x31, 0x2e, + 0x50, 0x61, 0x72, 0x44, 0x6f, 0x50, 0x61, 0x79, 0x6c, 0x6f, 0x61, 0x64, 0x2e, 0x54, 0x69, 0x6d, + 0x65, 0x72, 0x46, 0x61, 0x6d, 0x69, 0x6c, 0x79, 0x53, 0x70, 0x65, 0x63, 0x73, 0x45, 0x6e, 0x74, + 0x72, 0x79, 0x52, 0x10, 0x74, 0x69, 0x6d, 0x65, 0x72, 0x46, 0x61, 0x6d, 0x69, 0x6c, 0x79, 0x53, + 0x70, 0x65, 0x63, 0x73, 0x12, 0x30, 0x0a, 0x14, 0x72, 0x65, 0x73, 0x74, 0x72, 0x69, 0x63, 0x74, + 0x69, 0x6f, 0x6e, 0x5f, 0x63, 0x6f, 0x64, 0x65, 0x72, 0x5f, 0x69, 0x64, 0x18, 0x07, 0x20, 0x01, + 0x28, 0x09, 0x52, 0x12, 0x72, 0x65, 0x73, 0x74, 0x72, 0x69, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x43, + 0x6f, 0x64, 0x65, 0x72, 0x49, 0x64, 0x12, 0x33, 0x0a, 0x15, 0x72, 0x65, 0x71, 0x75, 0x65, 0x73, + 0x74, 0x73, 0x5f, 0x66, 0x69, 0x6e, 0x61, 0x6c, 0x69, 0x7a, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x18, + 0x08, 0x20, 0x01, 0x28, 0x08, 0x52, 0x14, 0x72, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x73, 0x46, + 0x69, 0x6e, 0x61, 0x6c, 0x69, 0x7a, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x3b, 0x0a, 0x1a, 0x72, + 0x65, 0x71, 0x75, 0x69, 0x72, 0x65, 0x73, 0x5f, 0x74, 0x69, 0x6d, 0x65, 0x5f, 0x73, 0x6f, 0x72, + 0x74, 0x65, 0x64, 0x5f, 0x69, 0x6e, 0x70, 0x75, 0x74, 0x18, 0x0a, 0x20, 0x01, 0x28, 0x08, 0x52, + 0x17, 0x72, 0x65, 0x71, 0x75, 0x69, 0x72, 0x65, 0x73, 0x54, 0x69, 0x6d, 0x65, 0x53, 0x6f, 0x72, + 0x74, 0x65, 0x64, 0x49, 0x6e, 0x70, 0x75, 0x74, 0x12, 0x32, 0x0a, 0x15, 0x72, 0x65, 0x71, 0x75, + 0x69, 0x72, 0x65, 0x73, 0x5f, 0x73, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x5f, 0x69, 0x6e, 0x70, 0x75, + 0x74, 0x18, 0x0b, 0x20, 0x01, 0x28, 0x08, 0x52, 0x13, 0x72, 0x65, 0x71, 0x75, 0x69, 0x72, 0x65, + 0x73, 0x53, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x49, 0x6e, 0x70, 0x75, 0x74, 0x12, 0x51, 0x0a, 0x26, + 0x6f, 0x6e, 0x5f, 0x77, 0x69, 0x6e, 0x64, 0x6f, 0x77, 0x5f, 0x65, 0x78, 0x70, 0x69, 0x72, 0x61, + 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x74, 0x69, 0x6d, 0x65, 0x72, 0x5f, 0x66, 0x61, 0x6d, 0x69, 0x6c, + 0x79, 0x5f, 0x73, 0x70, 0x65, 0x63, 0x18, 0x0c, 0x20, 0x01, 0x28, 0x09, 0x52, 0x21, 0x6f, 0x6e, + 0x57, 0x69, 0x6e, 0x64, 0x6f, 0x77, 0x45, 0x78, 0x70, 0x69, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, + 0x54, 0x69, 0x6d, 0x65, 0x72, 0x46, 0x61, 0x6d, 0x69, 0x6c, 0x79, 0x53, 0x70, 0x65, 0x63, 0x1a, + 0x6b, 0x0a, 0x0f, 0x53, 0x69, 0x64, 0x65, 0x49, 0x6e, 0x70, 0x75, 0x74, 0x73, 0x45, 0x6e, 0x74, + 0x72, 0x79, 0x12, 0x10, 0x0a, 0x03, 0x6b, 0x65, 0x79, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, + 0x03, 0x6b, 0x65, 0x79, 0x12, 0x42, 0x0a, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x18, 0x02, 0x20, + 0x01, 0x28, 0x0b, 0x32, 0x2c, 0x2e, 0x6f, 0x72, 0x67, 0x2e, 0x61, 0x70, 0x61, 0x63, 0x68, 0x65, + 0x2e, 0x62, 0x65, 0x61, 0x6d, 0x2e, 0x6d, 0x6f, 0x64, 0x65, 0x6c, 0x2e, 0x70, 0x69, 0x70, 0x65, + 0x6c, 0x69, 0x6e, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x53, 0x69, 0x64, 0x65, 0x49, 0x6e, 0x70, 0x75, + 0x74, 0x52, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x3a, 0x02, 0x38, 0x01, 0x1a, 0x6b, 0x0a, 0x0f, + 0x53, 0x74, 0x61, 0x74, 0x65, 0x53, 0x70, 0x65, 0x63, 0x73, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x12, + 0x10, 0x0a, 0x03, 0x6b, 0x65, 0x79, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x03, 0x6b, 0x65, + 0x79, 0x12, 0x42, 0x0a, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, + 0x32, 0x2c, 0x2e, 0x6f, 0x72, 0x67, 0x2e, 0x61, 0x70, 0x61, 0x63, 0x68, 0x65, 0x2e, 0x62, 0x65, + 0x61, 0x6d, 0x2e, 0x6d, 0x6f, 0x64, 0x65, 0x6c, 0x2e, 0x70, 0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e, + 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x53, 0x74, 0x61, 0x74, 0x65, 0x53, 0x70, 0x65, 0x63, 0x52, 0x05, + 0x76, 0x61, 0x6c, 0x75, 0x65, 0x3a, 0x02, 0x38, 0x01, 0x1a, 0x77, 0x0a, 0x15, 0x54, 0x69, 0x6d, + 0x65, 0x72, 0x46, 0x61, 0x6d, 0x69, 0x6c, 0x79, 0x53, 0x70, 0x65, 0x63, 0x73, 0x45, 0x6e, 0x74, + 0x72, 0x79, 0x12, 0x10, 0x0a, 0x03, 0x6b, 0x65, 0x79, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, + 0x03, 0x6b, 0x65, 0x79, 0x12, 0x48, 0x0a, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x18, 0x02, 0x20, + 0x01, 0x28, 0x0b, 0x32, 0x32, 0x2e, 0x6f, 0x72, 0x67, 0x2e, 0x61, 0x70, 0x61, 0x63, 0x68, 0x65, + 0x2e, 0x62, 0x65, 0x61, 0x6d, 0x2e, 0x6d, 0x6f, 0x64, 0x65, 0x6c, 0x2e, 0x70, 0x69, 0x70, 0x65, + 0x6c, 0x69, 0x6e, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x54, 0x69, 0x6d, 0x65, 0x72, 0x46, 0x61, 0x6d, + 0x69, 0x6c, 0x79, 0x53, 0x70, 0x65, 0x63, 0x52, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x3a, 0x02, + 0x38, 0x01, 0x4a, 0x04, 0x08, 0x06, 0x10, 0x07, 0x22, 0xe2, 0x05, 0x0a, 0x09, 0x53, 0x74, 0x61, + 0x74, 0x65, 0x53, 0x70, 0x65, 0x63, 0x12, 0x72, 0x0a, 0x16, 0x72, 0x65, 0x61, 0x64, 0x5f, 0x6d, + 0x6f, 0x64, 0x69, 0x66, 0x79, 0x5f, 0x77, 0x72, 0x69, 0x74, 0x65, 0x5f, 0x73, 0x70, 0x65, 0x63, + 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x3b, 0x2e, 0x6f, 0x72, 0x67, 0x2e, 0x61, 0x70, 0x61, + 0x63, 0x68, 0x65, 0x2e, 0x62, 0x65, 0x61, 0x6d, 0x2e, 0x6d, 0x6f, 0x64, 0x65, 0x6c, 0x2e, 0x70, + 0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x52, 0x65, 0x61, 0x64, 0x4d, + 0x6f, 0x64, 0x69, 0x66, 0x79, 0x57, 0x72, 0x69, 0x74, 0x65, 0x53, 0x74, 0x61, 0x74, 0x65, 0x53, + 0x70, 0x65, 0x63, 0x48, 0x00, 0x52, 0x13, 0x72, 0x65, 0x61, 0x64, 0x4d, 0x6f, 0x64, 0x69, 0x66, + 0x79, 0x57, 0x72, 0x69, 0x74, 0x65, 0x53, 0x70, 0x65, 0x63, 0x12, 0x4c, 0x0a, 0x08, 0x62, 0x61, + 0x67, 0x5f, 0x73, 0x70, 0x65, 0x63, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x2f, 0x2e, 0x6f, + 0x72, 0x67, 0x2e, 0x61, 0x70, 0x61, 0x63, 0x68, 0x65, 0x2e, 0x62, 0x65, 0x61, 0x6d, 0x2e, 0x6d, + 0x6f, 0x64, 0x65, 0x6c, 0x2e, 0x70, 0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e, 0x65, 0x2e, 0x76, 0x31, + 0x2e, 0x42, 0x61, 0x67, 0x53, 0x74, 0x61, 0x74, 0x65, 0x53, 0x70, 0x65, 0x63, 0x48, 0x00, 0x52, + 0x07, 0x62, 0x61, 0x67, 0x53, 0x70, 0x65, 0x63, 0x12, 0x5e, 0x0a, 0x0e, 0x63, 0x6f, 0x6d, 0x62, + 0x69, 0x6e, 0x69, 0x6e, 0x67, 0x5f, 0x73, 0x70, 0x65, 0x63, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0b, + 0x32, 0x35, 0x2e, 0x6f, 0x72, 0x67, 0x2e, 0x61, 0x70, 0x61, 0x63, 0x68, 0x65, 0x2e, 0x62, 0x65, + 0x61, 0x6d, 0x2e, 0x6d, 0x6f, 0x64, 0x65, 0x6c, 0x2e, 0x70, 0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e, + 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x43, 0x6f, 0x6d, 0x62, 0x69, 0x6e, 0x69, 0x6e, 0x67, 0x53, 0x74, + 0x61, 0x74, 0x65, 0x53, 0x70, 0x65, 0x63, 0x48, 0x00, 0x52, 0x0d, 0x63, 0x6f, 0x6d, 0x62, 0x69, + 0x6e, 0x69, 0x6e, 0x67, 0x53, 0x70, 0x65, 0x63, 0x12, 0x4c, 0x0a, 0x08, 0x6d, 0x61, 0x70, 0x5f, + 0x73, 0x70, 0x65, 0x63, 0x18, 0x04, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x2f, 0x2e, 0x6f, 0x72, 0x67, + 0x2e, 0x61, 0x70, 0x61, 0x63, 0x68, 0x65, 0x2e, 0x62, 0x65, 0x61, 0x6d, 0x2e, 0x6d, 0x6f, 0x64, + 0x65, 0x6c, 0x2e, 0x70, 0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x4d, + 0x61, 0x70, 0x53, 0x74, 0x61, 0x74, 0x65, 0x53, 0x70, 0x65, 0x63, 0x48, 0x00, 0x52, 0x07, 0x6d, + 0x61, 0x70, 0x53, 0x70, 0x65, 0x63, 0x12, 0x4c, 0x0a, 0x08, 0x73, 0x65, 0x74, 0x5f, 0x73, 0x70, + 0x65, 0x63, 0x18, 0x05, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x2f, 0x2e, 0x6f, 0x72, 0x67, 0x2e, 0x61, + 0x70, 0x61, 0x63, 0x68, 0x65, 0x2e, 0x62, 0x65, 0x61, 0x6d, 0x2e, 0x6d, 0x6f, 0x64, 0x65, 0x6c, + 0x2e, 0x70, 0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x53, 0x65, 0x74, + 0x53, 0x74, 0x61, 0x74, 0x65, 0x53, 0x70, 0x65, 0x63, 0x48, 0x00, 0x52, 0x07, 0x73, 0x65, 0x74, + 0x53, 0x70, 0x65, 0x63, 0x12, 0x65, 0x0a, 0x11, 0x6f, 0x72, 0x64, 0x65, 0x72, 0x65, 0x64, 0x5f, + 0x6c, 0x69, 0x73, 0x74, 0x5f, 0x73, 0x70, 0x65, 0x63, 0x18, 0x06, 0x20, 0x01, 0x28, 0x0b, 0x32, + 0x37, 0x2e, 0x6f, 0x72, 0x67, 0x2e, 0x61, 0x70, 0x61, 0x63, 0x68, 0x65, 0x2e, 0x62, 0x65, 0x61, + 0x6d, 0x2e, 0x6d, 0x6f, 0x64, 0x65, 0x6c, 0x2e, 0x70, 0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e, 0x65, + 0x2e, 0x76, 0x31, 0x2e, 0x4f, 0x72, 0x64, 0x65, 0x72, 0x65, 0x64, 0x4c, 0x69, 0x73, 0x74, 0x53, + 0x74, 0x61, 0x74, 0x65, 0x53, 0x70, 0x65, 0x63, 0x48, 0x00, 0x52, 0x0f, 0x6f, 0x72, 0x64, 0x65, + 0x72, 0x65, 0x64, 0x4c, 0x69, 0x73, 0x74, 0x53, 0x70, 0x65, 0x63, 0x12, 0x5b, 0x0a, 0x0d, 0x6d, + 0x75, 0x6c, 0x74, 0x69, 0x6d, 0x61, 0x70, 0x5f, 0x73, 0x70, 0x65, 0x63, 0x18, 0x08, 0x20, 0x01, + 0x28, 0x0b, 0x32, 0x34, 0x2e, 0x6f, 0x72, 0x67, 0x2e, 0x61, 0x70, 0x61, 0x63, 0x68, 0x65, 0x2e, + 0x62, 0x65, 0x61, 0x6d, 0x2e, 0x6d, 0x6f, 0x64, 0x65, 0x6c, 0x2e, 0x70, 0x69, 0x70, 0x65, 0x6c, + 0x69, 0x6e, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x4d, 0x75, 0x6c, 0x74, 0x69, 0x6d, 0x61, 0x70, 0x53, + 0x74, 0x61, 0x74, 0x65, 0x53, 0x70, 0x65, 0x63, 0x48, 0x00, 0x52, 0x0c, 0x6d, 0x75, 0x6c, 0x74, + 0x69, 0x6d, 0x61, 0x70, 0x53, 0x70, 0x65, 0x63, 0x12, 0x4b, 0x0a, 0x08, 0x70, 0x72, 0x6f, 0x74, + 0x6f, 0x63, 0x6f, 0x6c, 0x18, 0x07, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x2f, 0x2e, 0x6f, 0x72, 0x67, + 0x2e, 0x61, 0x70, 0x61, 0x63, 0x68, 0x65, 0x2e, 0x62, 0x65, 0x61, 0x6d, 0x2e, 0x6d, 0x6f, 0x64, + 0x65, 0x6c, 0x2e, 0x70, 0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x46, + 0x75, 0x6e, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x53, 0x70, 0x65, 0x63, 0x52, 0x08, 0x70, 0x72, 0x6f, + 0x74, 0x6f, 0x63, 0x6f, 0x6c, 0x42, 0x06, 0x0a, 0x04, 0x73, 0x70, 0x65, 0x63, 0x22, 0x35, 0x0a, + 0x18, 0x52, 0x65, 0x61, 0x64, 0x4d, 0x6f, 0x64, 0x69, 0x66, 0x79, 0x57, 0x72, 0x69, 0x74, 0x65, + 0x53, 0x74, 0x61, 0x74, 0x65, 0x53, 0x70, 0x65, 0x63, 0x12, 0x19, 0x0a, 0x08, 0x63, 0x6f, 0x64, + 0x65, 0x72, 0x5f, 0x69, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x07, 0x63, 0x6f, 0x64, + 0x65, 0x72, 0x49, 0x64, 0x22, 0x38, 0x0a, 0x0c, 0x42, 0x61, 0x67, 0x53, 0x74, 0x61, 0x74, 0x65, + 0x53, 0x70, 0x65, 0x63, 0x12, 0x28, 0x0a, 0x10, 0x65, 0x6c, 0x65, 0x6d, 0x65, 0x6e, 0x74, 0x5f, + 0x63, 0x6f, 0x64, 0x65, 0x72, 0x5f, 0x69, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0e, + 0x65, 0x6c, 0x65, 0x6d, 0x65, 0x6e, 0x74, 0x43, 0x6f, 0x64, 0x65, 0x72, 0x49, 0x64, 0x22, 0x40, + 0x0a, 0x14, 0x4f, 0x72, 0x64, 0x65, 0x72, 0x65, 0x64, 0x4c, 0x69, 0x73, 0x74, 0x53, 0x74, 0x61, + 0x74, 0x65, 0x53, 0x70, 0x65, 0x63, 0x12, 0x28, 0x0a, 0x10, 0x65, 0x6c, 0x65, 0x6d, 0x65, 0x6e, + 0x74, 0x5f, 0x63, 0x6f, 0x64, 0x65, 0x72, 0x5f, 0x69, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, + 0x52, 0x0e, 0x65, 0x6c, 0x65, 0x6d, 0x65, 0x6e, 0x74, 0x43, 0x6f, 0x64, 0x65, 0x72, 0x49, 0x64, + 0x22, 0x96, 0x01, 0x0a, 0x12, 0x43, 0x6f, 0x6d, 0x62, 0x69, 0x6e, 0x69, 0x6e, 0x67, 0x53, 0x74, + 0x61, 0x74, 0x65, 0x53, 0x70, 0x65, 0x63, 0x12, 0x30, 0x0a, 0x14, 0x61, 0x63, 0x63, 0x75, 0x6d, + 0x75, 0x6c, 0x61, 0x74, 0x6f, 0x72, 0x5f, 0x63, 0x6f, 0x64, 0x65, 0x72, 0x5f, 0x69, 0x64, 0x18, + 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x12, 0x61, 0x63, 0x63, 0x75, 0x6d, 0x75, 0x6c, 0x61, 0x74, + 0x6f, 0x72, 0x43, 0x6f, 0x64, 0x65, 0x72, 0x49, 0x64, 0x12, 0x4e, 0x0a, 0x0a, 0x63, 0x6f, 0x6d, + 0x62, 0x69, 0x6e, 0x65, 0x5f, 0x66, 0x6e, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x2f, 0x2e, + 0x6f, 0x72, 0x67, 0x2e, 0x61, 0x70, 0x61, 0x63, 0x68, 0x65, 0x2e, 0x62, 0x65, 0x61, 0x6d, 0x2e, + 0x6d, 0x6f, 0x64, 0x65, 0x6c, 0x2e, 0x70, 0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e, 0x65, 0x2e, 0x76, + 0x31, 0x2e, 0x46, 0x75, 0x6e, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x53, 0x70, 0x65, 0x63, 0x52, 0x09, + 0x63, 0x6f, 0x6d, 0x62, 0x69, 0x6e, 0x65, 0x46, 0x6e, 0x22, 0x56, 0x0a, 0x0c, 0x4d, 0x61, 0x70, + 0x53, 0x74, 0x61, 0x74, 0x65, 0x53, 0x70, 0x65, 0x63, 0x12, 0x20, 0x0a, 0x0c, 0x6b, 0x65, 0x79, + 0x5f, 0x63, 0x6f, 0x64, 0x65, 0x72, 0x5f, 0x69, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, + 0x0a, 0x6b, 0x65, 0x79, 0x43, 0x6f, 0x64, 0x65, 0x72, 0x49, 0x64, 0x12, 0x24, 0x0a, 0x0e, 0x76, + 0x61, 0x6c, 0x75, 0x65, 0x5f, 0x63, 0x6f, 0x64, 0x65, 0x72, 0x5f, 0x69, 0x64, 0x18, 0x02, 0x20, + 0x01, 0x28, 0x09, 0x52, 0x0c, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x43, 0x6f, 0x64, 0x65, 0x72, 0x49, + 0x64, 0x22, 0x5b, 0x0a, 0x11, 0x4d, 0x75, 0x6c, 0x74, 0x69, 0x6d, 0x61, 0x70, 0x53, 0x74, 0x61, + 0x74, 0x65, 0x53, 0x70, 0x65, 0x63, 0x12, 0x20, 0x0a, 0x0c, 0x6b, 0x65, 0x79, 0x5f, 0x63, 0x6f, + 0x64, 0x65, 0x72, 0x5f, 0x69, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0a, 0x6b, 0x65, + 0x79, 0x43, 0x6f, 0x64, 0x65, 0x72, 0x49, 0x64, 0x12, 0x24, 0x0a, 0x0e, 0x76, 0x61, 0x6c, 0x75, + 0x65, 0x5f, 0x63, 0x6f, 0x64, 0x65, 0x72, 0x5f, 0x69, 0x64, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, + 0x52, 0x0c, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x43, 0x6f, 0x64, 0x65, 0x72, 0x49, 0x64, 0x22, 0x38, + 0x0a, 0x0c, 0x53, 0x65, 0x74, 0x53, 0x74, 0x61, 0x74, 0x65, 0x53, 0x70, 0x65, 0x63, 0x12, 0x28, + 0x0a, 0x10, 0x65, 0x6c, 0x65, 0x6d, 0x65, 0x6e, 0x74, 0x5f, 0x63, 0x6f, 0x64, 0x65, 0x72, 0x5f, + 0x69, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0e, 0x65, 0x6c, 0x65, 0x6d, 0x65, 0x6e, + 0x74, 0x43, 0x6f, 0x64, 0x65, 0x72, 0x49, 0x64, 0x22, 0x99, 0x01, 0x0a, 0x0f, 0x54, 0x69, 0x6d, + 0x65, 0x72, 0x46, 0x61, 0x6d, 0x69, 0x6c, 0x79, 0x53, 0x70, 0x65, 0x63, 0x12, 0x53, 0x0a, 0x0b, + 0x74, 0x69, 0x6d, 0x65, 0x5f, 0x64, 0x6f, 0x6d, 0x61, 0x69, 0x6e, 0x18, 0x01, 0x20, 0x01, 0x28, + 0x0e, 0x32, 0x32, 0x2e, 0x6f, 0x72, 0x67, 0x2e, 0x61, 0x70, 0x61, 0x63, 0x68, 0x65, 0x2e, 0x62, + 0x65, 0x61, 0x6d, 0x2e, 0x6d, 0x6f, 0x64, 0x65, 0x6c, 0x2e, 0x70, 0x69, 0x70, 0x65, 0x6c, 0x69, + 0x6e, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x54, 0x69, 0x6d, 0x65, 0x44, 0x6f, 0x6d, 0x61, 0x69, 0x6e, + 0x2e, 0x45, 0x6e, 0x75, 0x6d, 0x52, 0x0a, 0x74, 0x69, 0x6d, 0x65, 0x44, 0x6f, 0x6d, 0x61, 0x69, + 0x6e, 0x12, 0x31, 0x0a, 0x15, 0x74, 0x69, 0x6d, 0x65, 0x72, 0x5f, 0x66, 0x61, 0x6d, 0x69, 0x6c, + 0x79, 0x5f, 0x63, 0x6f, 0x64, 0x65, 0x72, 0x5f, 0x69, 0x64, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, + 0x52, 0x12, 0x74, 0x69, 0x6d, 0x65, 0x72, 0x46, 0x61, 0x6d, 0x69, 0x6c, 0x79, 0x43, 0x6f, 0x64, + 0x65, 0x72, 0x49, 0x64, 0x22, 0x40, 0x0a, 0x09, 0x49, 0x73, 0x42, 0x6f, 0x75, 0x6e, 0x64, 0x65, + 0x64, 0x22, 0x33, 0x0a, 0x04, 0x45, 0x6e, 0x75, 0x6d, 0x12, 0x0f, 0x0a, 0x0b, 0x55, 0x4e, 0x53, + 0x50, 0x45, 0x43, 0x49, 0x46, 0x49, 0x45, 0x44, 0x10, 0x00, 0x12, 0x0d, 0x0a, 0x09, 0x55, 0x4e, + 0x42, 0x4f, 0x55, 0x4e, 0x44, 0x45, 0x44, 0x10, 0x01, 0x12, 0x0b, 0x0a, 0x07, 0x42, 0x4f, 0x55, + 0x4e, 0x44, 0x45, 0x44, 0x10, 0x02, 0x22, 0xa8, 0x01, 0x0a, 0x0b, 0x52, 0x65, 0x61, 0x64, 0x50, + 0x61, 0x79, 0x6c, 0x6f, 0x61, 0x64, 0x12, 0x47, 0x0a, 0x06, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, + 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x2f, 0x2e, 0x6f, 0x72, 0x67, 0x2e, 0x61, 0x70, 0x61, + 0x63, 0x68, 0x65, 0x2e, 0x62, 0x65, 0x61, 0x6d, 0x2e, 0x6d, 0x6f, 0x64, 0x65, 0x6c, 0x2e, 0x70, + 0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x46, 0x75, 0x6e, 0x63, 0x74, + 0x69, 0x6f, 0x6e, 0x53, 0x70, 0x65, 0x63, 0x52, 0x06, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x12, + 0x50, 0x0a, 0x0a, 0x69, 0x73, 0x5f, 0x62, 0x6f, 0x75, 0x6e, 0x64, 0x65, 0x64, 0x18, 0x02, 0x20, + 0x01, 0x28, 0x0e, 0x32, 0x31, 0x2e, 0x6f, 0x72, 0x67, 0x2e, 0x61, 0x70, 0x61, 0x63, 0x68, 0x65, + 0x2e, 0x62, 0x65, 0x61, 0x6d, 0x2e, 0x6d, 0x6f, 0x64, 0x65, 0x6c, 0x2e, 0x70, 0x69, 0x70, 0x65, + 0x6c, 0x69, 0x6e, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x49, 0x73, 0x42, 0x6f, 0x75, 0x6e, 0x64, 0x65, + 0x64, 0x2e, 0x45, 0x6e, 0x75, 0x6d, 0x52, 0x09, 0x69, 0x73, 0x42, 0x6f, 0x75, 0x6e, 0x64, 0x65, + 0x64, 0x22, 0x61, 0x0a, 0x11, 0x57, 0x69, 0x6e, 0x64, 0x6f, 0x77, 0x49, 0x6e, 0x74, 0x6f, 0x50, + 0x61, 0x79, 0x6c, 0x6f, 0x61, 0x64, 0x12, 0x4c, 0x0a, 0x09, 0x77, 0x69, 0x6e, 0x64, 0x6f, 0x77, + 0x5f, 0x66, 0x6e, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x2f, 0x2e, 0x6f, 0x72, 0x67, 0x2e, + 0x61, 0x70, 0x61, 0x63, 0x68, 0x65, 0x2e, 0x62, 0x65, 0x61, 0x6d, 0x2e, 0x6d, 0x6f, 0x64, 0x65, + 0x6c, 0x2e, 0x70, 0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x46, 0x75, + 0x6e, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x53, 0x70, 0x65, 0x63, 0x52, 0x08, 0x77, 0x69, 0x6e, 0x64, + 0x6f, 0x77, 0x46, 0x6e, 0x22, 0x92, 0x01, 0x0a, 0x0e, 0x43, 0x6f, 0x6d, 0x62, 0x69, 0x6e, 0x65, + 0x50, 0x61, 0x79, 0x6c, 0x6f, 0x61, 0x64, 0x12, 0x4e, 0x0a, 0x0a, 0x63, 0x6f, 0x6d, 0x62, 0x69, + 0x6e, 0x65, 0x5f, 0x66, 0x6e, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x2f, 0x2e, 0x6f, 0x72, + 0x67, 0x2e, 0x61, 0x70, 0x61, 0x63, 0x68, 0x65, 0x2e, 0x62, 0x65, 0x61, 0x6d, 0x2e, 0x6d, 0x6f, + 0x64, 0x65, 0x6c, 0x2e, 0x70, 0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e, 0x65, 0x2e, 0x76, 0x31, 0x2e, + 0x46, 0x75, 0x6e, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x53, 0x70, 0x65, 0x63, 0x52, 0x09, 0x63, 0x6f, + 0x6d, 0x62, 0x69, 0x6e, 0x65, 0x46, 0x6e, 0x12, 0x30, 0x0a, 0x14, 0x61, 0x63, 0x63, 0x75, 0x6d, + 0x75, 0x6c, 0x61, 0x74, 0x6f, 0x72, 0x5f, 0x63, 0x6f, 0x64, 0x65, 0x72, 0x5f, 0x69, 0x64, 0x18, + 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x12, 0x61, 0x63, 0x63, 0x75, 0x6d, 0x75, 0x6c, 0x61, 0x74, + 0x6f, 0x72, 0x43, 0x6f, 0x64, 0x65, 0x72, 0x49, 0x64, 0x22, 0xcd, 0x07, 0x0a, 0x11, 0x54, 0x65, + 0x73, 0x74, 0x53, 0x74, 0x72, 0x65, 0x61, 0x6d, 0x50, 0x61, 0x79, 0x6c, 0x6f, 0x61, 0x64, 0x12, + 0x19, 0x0a, 0x08, 0x63, 0x6f, 0x64, 0x65, 0x72, 0x5f, 0x69, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, + 0x09, 0x52, 0x07, 0x63, 0x6f, 0x64, 0x65, 0x72, 0x49, 0x64, 0x12, 0x52, 0x0a, 0x06, 0x65, 0x76, + 0x65, 0x6e, 0x74, 0x73, 0x18, 0x02, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x3a, 0x2e, 0x6f, 0x72, 0x67, + 0x2e, 0x61, 0x70, 0x61, 0x63, 0x68, 0x65, 0x2e, 0x62, 0x65, 0x61, 0x6d, 0x2e, 0x6d, 0x6f, 0x64, + 0x65, 0x6c, 0x2e, 0x70, 0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x54, + 0x65, 0x73, 0x74, 0x53, 0x74, 0x72, 0x65, 0x61, 0x6d, 0x50, 0x61, 0x79, 0x6c, 0x6f, 0x61, 0x64, + 0x2e, 0x45, 0x76, 0x65, 0x6e, 0x74, 0x52, 0x06, 0x65, 0x76, 0x65, 0x6e, 0x74, 0x73, 0x12, 0x53, + 0x0a, 0x08, 0x65, 0x6e, 0x64, 0x70, 0x6f, 0x69, 0x6e, 0x74, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0b, + 0x32, 0x37, 0x2e, 0x6f, 0x72, 0x67, 0x2e, 0x61, 0x70, 0x61, 0x63, 0x68, 0x65, 0x2e, 0x62, 0x65, + 0x61, 0x6d, 0x2e, 0x6d, 0x6f, 0x64, 0x65, 0x6c, 0x2e, 0x70, 0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e, + 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x41, 0x70, 0x69, 0x53, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, 0x44, + 0x65, 0x73, 0x63, 0x72, 0x69, 0x70, 0x74, 0x6f, 0x72, 0x52, 0x08, 0x65, 0x6e, 0x64, 0x70, 0x6f, + 0x69, 0x6e, 0x74, 0x1a, 0x96, 0x05, 0x0a, 0x05, 0x45, 0x76, 0x65, 0x6e, 0x74, 0x12, 0x76, 0x0a, + 0x0f, 0x77, 0x61, 0x74, 0x65, 0x72, 0x6d, 0x61, 0x72, 0x6b, 0x5f, 0x65, 0x76, 0x65, 0x6e, 0x74, + 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x4b, 0x2e, 0x6f, 0x72, 0x67, 0x2e, 0x61, 0x70, 0x61, + 0x63, 0x68, 0x65, 0x2e, 0x62, 0x65, 0x61, 0x6d, 0x2e, 0x6d, 0x6f, 0x64, 0x65, 0x6c, 0x2e, 0x70, + 0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x54, 0x65, 0x73, 0x74, 0x53, + 0x74, 0x72, 0x65, 0x61, 0x6d, 0x50, 0x61, 0x79, 0x6c, 0x6f, 0x61, 0x64, 0x2e, 0x45, 0x76, 0x65, + 0x6e, 0x74, 0x2e, 0x41, 0x64, 0x76, 0x61, 0x6e, 0x63, 0x65, 0x57, 0x61, 0x74, 0x65, 0x72, 0x6d, + 0x61, 0x72, 0x6b, 0x48, 0x00, 0x52, 0x0e, 0x77, 0x61, 0x74, 0x65, 0x72, 0x6d, 0x61, 0x72, 0x6b, + 0x45, 0x76, 0x65, 0x6e, 0x74, 0x12, 0x86, 0x01, 0x0a, 0x15, 0x70, 0x72, 0x6f, 0x63, 0x65, 0x73, + 0x73, 0x69, 0x6e, 0x67, 0x5f, 0x74, 0x69, 0x6d, 0x65, 0x5f, 0x65, 0x76, 0x65, 0x6e, 0x74, 0x18, + 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x50, 0x2e, 0x6f, 0x72, 0x67, 0x2e, 0x61, 0x70, 0x61, 0x63, + 0x68, 0x65, 0x2e, 0x62, 0x65, 0x61, 0x6d, 0x2e, 0x6d, 0x6f, 0x64, 0x65, 0x6c, 0x2e, 0x70, 0x69, + 0x70, 0x65, 0x6c, 0x69, 0x6e, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x54, 0x65, 0x73, 0x74, 0x53, 0x74, + 0x72, 0x65, 0x61, 0x6d, 0x50, 0x61, 0x79, 0x6c, 0x6f, 0x61, 0x64, 0x2e, 0x45, 0x76, 0x65, 0x6e, + 0x74, 0x2e, 0x41, 0x64, 0x76, 0x61, 0x6e, 0x63, 0x65, 0x50, 0x72, 0x6f, 0x63, 0x65, 0x73, 0x73, + 0x69, 0x6e, 0x67, 0x54, 0x69, 0x6d, 0x65, 0x48, 0x00, 0x52, 0x13, 0x70, 0x72, 0x6f, 0x63, 0x65, + 0x73, 0x73, 0x69, 0x6e, 0x67, 0x54, 0x69, 0x6d, 0x65, 0x45, 0x76, 0x65, 0x6e, 0x74, 0x12, 0x6d, + 0x0a, 0x0d, 0x65, 0x6c, 0x65, 0x6d, 0x65, 0x6e, 0x74, 0x5f, 0x65, 0x76, 0x65, 0x6e, 0x74, 0x18, + 0x03, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x46, 0x2e, 0x6f, 0x72, 0x67, 0x2e, 0x61, 0x70, 0x61, 0x63, + 0x68, 0x65, 0x2e, 0x62, 0x65, 0x61, 0x6d, 0x2e, 0x6d, 0x6f, 0x64, 0x65, 0x6c, 0x2e, 0x70, 0x69, + 0x70, 0x65, 0x6c, 0x69, 0x6e, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x54, 0x65, 0x73, 0x74, 0x53, 0x74, + 0x72, 0x65, 0x61, 0x6d, 0x50, 0x61, 0x79, 0x6c, 0x6f, 0x61, 0x64, 0x2e, 0x45, 0x76, 0x65, 0x6e, + 0x74, 0x2e, 0x41, 0x64, 0x64, 0x45, 0x6c, 0x65, 0x6d, 0x65, 0x6e, 0x74, 0x73, 0x48, 0x00, 0x52, + 0x0c, 0x65, 0x6c, 0x65, 0x6d, 0x65, 0x6e, 0x74, 0x45, 0x76, 0x65, 0x6e, 0x74, 0x1a, 0x49, 0x0a, + 0x10, 0x41, 0x64, 0x76, 0x61, 0x6e, 0x63, 0x65, 0x57, 0x61, 0x74, 0x65, 0x72, 0x6d, 0x61, 0x72, + 0x6b, 0x12, 0x23, 0x0a, 0x0d, 0x6e, 0x65, 0x77, 0x5f, 0x77, 0x61, 0x74, 0x65, 0x72, 0x6d, 0x61, + 0x72, 0x6b, 0x18, 0x01, 0x20, 0x01, 0x28, 0x03, 0x52, 0x0c, 0x6e, 0x65, 0x77, 0x57, 0x61, 0x74, + 0x65, 0x72, 0x6d, 0x61, 0x72, 0x6b, 0x12, 0x10, 0x0a, 0x03, 0x74, 0x61, 0x67, 0x18, 0x02, 0x20, + 0x01, 0x28, 0x09, 0x52, 0x03, 0x74, 0x61, 0x67, 0x1a, 0x42, 0x0a, 0x15, 0x41, 0x64, 0x76, 0x61, + 0x6e, 0x63, 0x65, 0x50, 0x72, 0x6f, 0x63, 0x65, 0x73, 0x73, 0x69, 0x6e, 0x67, 0x54, 0x69, 0x6d, + 0x65, 0x12, 0x29, 0x0a, 0x10, 0x61, 0x64, 0x76, 0x61, 0x6e, 0x63, 0x65, 0x5f, 0x64, 0x75, 0x72, + 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x18, 0x01, 0x20, 0x01, 0x28, 0x03, 0x52, 0x0f, 0x61, 0x64, 0x76, + 0x61, 0x6e, 0x63, 0x65, 0x44, 0x75, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x1a, 0x84, 0x01, 0x0a, + 0x0b, 0x41, 0x64, 0x64, 0x45, 0x6c, 0x65, 0x6d, 0x65, 0x6e, 0x74, 0x73, 0x12, 0x63, 0x0a, 0x08, + 0x65, 0x6c, 0x65, 0x6d, 0x65, 0x6e, 0x74, 0x73, 0x18, 0x01, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x47, + 0x2e, 0x6f, 0x72, 0x67, 0x2e, 0x61, 0x70, 0x61, 0x63, 0x68, 0x65, 0x2e, 0x62, 0x65, 0x61, 0x6d, + 0x2e, 0x6d, 0x6f, 0x64, 0x65, 0x6c, 0x2e, 0x70, 0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e, 0x65, 0x2e, + 0x76, 0x31, 0x2e, 0x54, 0x65, 0x73, 0x74, 0x53, 0x74, 0x72, 0x65, 0x61, 0x6d, 0x50, 0x61, 0x79, + 0x6c, 0x6f, 0x61, 0x64, 0x2e, 0x54, 0x69, 0x6d, 0x65, 0x73, 0x74, 0x61, 0x6d, 0x70, 0x65, 0x64, + 0x45, 0x6c, 0x65, 0x6d, 0x65, 0x6e, 0x74, 0x52, 0x08, 0x65, 0x6c, 0x65, 0x6d, 0x65, 0x6e, 0x74, + 0x73, 0x12, 0x10, 0x0a, 0x03, 0x74, 0x61, 0x67, 0x18, 0x03, 0x20, 0x01, 0x28, 0x09, 0x52, 0x03, + 0x74, 0x61, 0x67, 0x42, 0x07, 0x0a, 0x05, 0x65, 0x76, 0x65, 0x6e, 0x74, 0x1a, 0x5b, 0x0a, 0x12, + 0x54, 0x69, 0x6d, 0x65, 0x73, 0x74, 0x61, 0x6d, 0x70, 0x65, 0x64, 0x45, 0x6c, 0x65, 0x6d, 0x65, + 0x6e, 0x74, 0x12, 0x27, 0x0a, 0x0f, 0x65, 0x6e, 0x63, 0x6f, 0x64, 0x65, 0x64, 0x5f, 0x65, 0x6c, + 0x65, 0x6d, 0x65, 0x6e, 0x74, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0c, 0x52, 0x0e, 0x65, 0x6e, 0x63, + 0x6f, 0x64, 0x65, 0x64, 0x45, 0x6c, 0x65, 0x6d, 0x65, 0x6e, 0x74, 0x12, 0x1c, 0x0a, 0x09, 0x74, + 0x69, 0x6d, 0x65, 0x73, 0x74, 0x61, 0x6d, 0x70, 0x18, 0x02, 0x20, 0x01, 0x28, 0x03, 0x52, 0x09, + 0x74, 0x69, 0x6d, 0x65, 0x73, 0x74, 0x61, 0x6d, 0x70, 0x22, 0x2e, 0x0a, 0x0d, 0x45, 0x76, 0x65, + 0x6e, 0x74, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x1d, 0x0a, 0x0a, 0x6f, 0x75, + 0x74, 0x70, 0x75, 0x74, 0x5f, 0x69, 0x64, 0x73, 0x18, 0x01, 0x20, 0x03, 0x28, 0x09, 0x52, 0x09, + 0x6f, 0x75, 0x74, 0x70, 0x75, 0x74, 0x49, 0x64, 0x73, 0x22, 0x90, 0x04, 0x0a, 0x11, 0x57, 0x72, + 0x69, 0x74, 0x65, 0x46, 0x69, 0x6c, 0x65, 0x73, 0x50, 0x61, 0x79, 0x6c, 0x6f, 0x61, 0x64, 0x12, + 0x43, 0x0a, 0x04, 0x73, 0x69, 0x6e, 0x6b, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x2f, 0x2e, + 0x6f, 0x72, 0x67, 0x2e, 0x61, 0x70, 0x61, 0x63, 0x68, 0x65, 0x2e, 0x62, 0x65, 0x61, 0x6d, 0x2e, + 0x6d, 0x6f, 0x64, 0x65, 0x6c, 0x2e, 0x70, 0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e, 0x65, 0x2e, 0x76, + 0x31, 0x2e, 0x46, 0x75, 0x6e, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x53, 0x70, 0x65, 0x63, 0x52, 0x04, + 0x73, 0x69, 0x6e, 0x6b, 0x12, 0x58, 0x0a, 0x0f, 0x66, 0x6f, 0x72, 0x6d, 0x61, 0x74, 0x5f, 0x66, + 0x75, 0x6e, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x2f, 0x2e, + 0x6f, 0x72, 0x67, 0x2e, 0x61, 0x70, 0x61, 0x63, 0x68, 0x65, 0x2e, 0x62, 0x65, 0x61, 0x6d, 0x2e, + 0x6d, 0x6f, 0x64, 0x65, 0x6c, 0x2e, 0x70, 0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e, 0x65, 0x2e, 0x76, + 0x31, 0x2e, 0x46, 0x75, 0x6e, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x53, 0x70, 0x65, 0x63, 0x52, 0x0e, + 0x66, 0x6f, 0x72, 0x6d, 0x61, 0x74, 0x46, 0x75, 0x6e, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x27, + 0x0a, 0x0f, 0x77, 0x69, 0x6e, 0x64, 0x6f, 0x77, 0x65, 0x64, 0x5f, 0x77, 0x72, 0x69, 0x74, 0x65, + 0x73, 0x18, 0x03, 0x20, 0x01, 0x28, 0x08, 0x52, 0x0e, 0x77, 0x69, 0x6e, 0x64, 0x6f, 0x77, 0x65, + 0x64, 0x57, 0x72, 0x69, 0x74, 0x65, 0x73, 0x12, 0x3c, 0x0a, 0x1a, 0x72, 0x75, 0x6e, 0x6e, 0x65, + 0x72, 0x5f, 0x64, 0x65, 0x74, 0x65, 0x72, 0x6d, 0x69, 0x6e, 0x65, 0x64, 0x5f, 0x73, 0x68, 0x61, + 0x72, 0x64, 0x69, 0x6e, 0x67, 0x18, 0x04, 0x20, 0x01, 0x28, 0x08, 0x52, 0x18, 0x72, 0x75, 0x6e, + 0x6e, 0x65, 0x72, 0x44, 0x65, 0x74, 0x65, 0x72, 0x6d, 0x69, 0x6e, 0x65, 0x64, 0x53, 0x68, 0x61, + 0x72, 0x64, 0x69, 0x6e, 0x67, 0x12, 0x65, 0x0a, 0x0b, 0x73, 0x69, 0x64, 0x65, 0x5f, 0x69, 0x6e, + 0x70, 0x75, 0x74, 0x73, 0x18, 0x05, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x44, 0x2e, 0x6f, 0x72, 0x67, + 0x2e, 0x61, 0x70, 0x61, 0x63, 0x68, 0x65, 0x2e, 0x62, 0x65, 0x61, 0x6d, 0x2e, 0x6d, 0x6f, 0x64, + 0x65, 0x6c, 0x2e, 0x70, 0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x57, + 0x72, 0x69, 0x74, 0x65, 0x46, 0x69, 0x6c, 0x65, 0x73, 0x50, 0x61, 0x79, 0x6c, 0x6f, 0x61, 0x64, + 0x2e, 0x53, 0x69, 0x64, 0x65, 0x49, 0x6e, 0x70, 0x75, 0x74, 0x73, 0x45, 0x6e, 0x74, 0x72, 0x79, + 0x52, 0x0a, 0x73, 0x69, 0x64, 0x65, 0x49, 0x6e, 0x70, 0x75, 0x74, 0x73, 0x12, 0x21, 0x0a, 0x0c, + 0x61, 0x75, 0x74, 0x6f, 0x5f, 0x73, 0x68, 0x61, 0x72, 0x64, 0x65, 0x64, 0x18, 0x06, 0x20, 0x01, + 0x28, 0x08, 0x52, 0x0b, 0x61, 0x75, 0x74, 0x6f, 0x53, 0x68, 0x61, 0x72, 0x64, 0x65, 0x64, 0x1a, + 0x6b, 0x0a, 0x0f, 0x53, 0x69, 0x64, 0x65, 0x49, 0x6e, 0x70, 0x75, 0x74, 0x73, 0x45, 0x6e, 0x74, + 0x72, 0x79, 0x12, 0x10, 0x0a, 0x03, 0x6b, 0x65, 0x79, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, + 0x03, 0x6b, 0x65, 0x79, 0x12, 0x42, 0x0a, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x18, 0x02, 0x20, + 0x01, 0x28, 0x0b, 0x32, 0x2c, 0x2e, 0x6f, 0x72, 0x67, 0x2e, 0x61, 0x70, 0x61, 0x63, 0x68, 0x65, + 0x2e, 0x62, 0x65, 0x61, 0x6d, 0x2e, 0x6d, 0x6f, 0x64, 0x65, 0x6c, 0x2e, 0x70, 0x69, 0x70, 0x65, + 0x6c, 0x69, 0x6e, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x53, 0x69, 0x64, 0x65, 0x49, 0x6e, 0x70, 0x75, + 0x74, 0x52, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x3a, 0x02, 0x38, 0x01, 0x22, 0xcc, 0x02, 0x0a, + 0x11, 0x50, 0x75, 0x62, 0x53, 0x75, 0x62, 0x52, 0x65, 0x61, 0x64, 0x50, 0x61, 0x79, 0x6c, 0x6f, + 0x61, 0x64, 0x12, 0x14, 0x0a, 0x05, 0x74, 0x6f, 0x70, 0x69, 0x63, 0x18, 0x01, 0x20, 0x01, 0x28, + 0x09, 0x52, 0x05, 0x74, 0x6f, 0x70, 0x69, 0x63, 0x12, 0x22, 0x0a, 0x0c, 0x73, 0x75, 0x62, 0x73, + 0x63, 0x72, 0x69, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0c, + 0x73, 0x75, 0x62, 0x73, 0x63, 0x72, 0x69, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x2f, 0x0a, 0x13, + 0x74, 0x69, 0x6d, 0x65, 0x73, 0x74, 0x61, 0x6d, 0x70, 0x5f, 0x61, 0x74, 0x74, 0x72, 0x69, 0x62, + 0x75, 0x74, 0x65, 0x18, 0x03, 0x20, 0x01, 0x28, 0x09, 0x52, 0x12, 0x74, 0x69, 0x6d, 0x65, 0x73, + 0x74, 0x61, 0x6d, 0x70, 0x41, 0x74, 0x74, 0x72, 0x69, 0x62, 0x75, 0x74, 0x65, 0x12, 0x21, 0x0a, + 0x0c, 0x69, 0x64, 0x5f, 0x61, 0x74, 0x74, 0x72, 0x69, 0x62, 0x75, 0x74, 0x65, 0x18, 0x04, 0x20, + 0x01, 0x28, 0x09, 0x52, 0x0b, 0x69, 0x64, 0x41, 0x74, 0x74, 0x72, 0x69, 0x62, 0x75, 0x74, 0x65, + 0x12, 0x27, 0x0a, 0x0f, 0x77, 0x69, 0x74, 0x68, 0x5f, 0x61, 0x74, 0x74, 0x72, 0x69, 0x62, 0x75, + 0x74, 0x65, 0x73, 0x18, 0x05, 0x20, 0x01, 0x28, 0x08, 0x52, 0x0e, 0x77, 0x69, 0x74, 0x68, 0x41, + 0x74, 0x74, 0x72, 0x69, 0x62, 0x75, 0x74, 0x65, 0x73, 0x12, 0x38, 0x0a, 0x18, 0x74, 0x6f, 0x70, + 0x69, 0x63, 0x5f, 0x72, 0x75, 0x6e, 0x74, 0x69, 0x6d, 0x65, 0x5f, 0x6f, 0x76, 0x65, 0x72, 0x72, + 0x69, 0x64, 0x64, 0x65, 0x6e, 0x18, 0x06, 0x20, 0x01, 0x28, 0x09, 0x52, 0x16, 0x74, 0x6f, 0x70, + 0x69, 0x63, 0x52, 0x75, 0x6e, 0x74, 0x69, 0x6d, 0x65, 0x4f, 0x76, 0x65, 0x72, 0x72, 0x69, 0x64, + 0x64, 0x65, 0x6e, 0x12, 0x46, 0x0a, 0x1f, 0x73, 0x75, 0x62, 0x73, 0x63, 0x72, 0x69, 0x70, 0x74, + 0x69, 0x6f, 0x6e, 0x5f, 0x72, 0x75, 0x6e, 0x74, 0x69, 0x6d, 0x65, 0x5f, 0x6f, 0x76, 0x65, 0x72, + 0x72, 0x69, 0x64, 0x64, 0x65, 0x6e, 0x18, 0x07, 0x20, 0x01, 0x28, 0x09, 0x52, 0x1d, 0x73, 0x75, + 0x62, 0x73, 0x63, 0x72, 0x69, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x75, 0x6e, 0x74, 0x69, 0x6d, + 0x65, 0x4f, 0x76, 0x65, 0x72, 0x72, 0x69, 0x64, 0x64, 0x65, 0x6e, 0x22, 0xb8, 0x01, 0x0a, 0x12, + 0x50, 0x75, 0x62, 0x53, 0x75, 0x62, 0x57, 0x72, 0x69, 0x74, 0x65, 0x50, 0x61, 0x79, 0x6c, 0x6f, + 0x61, 0x64, 0x12, 0x14, 0x0a, 0x05, 0x74, 0x6f, 0x70, 0x69, 0x63, 0x18, 0x01, 0x20, 0x01, 0x28, + 0x09, 0x52, 0x05, 0x74, 0x6f, 0x70, 0x69, 0x63, 0x12, 0x2f, 0x0a, 0x13, 0x74, 0x69, 0x6d, 0x65, + 0x73, 0x74, 0x61, 0x6d, 0x70, 0x5f, 0x61, 0x74, 0x74, 0x72, 0x69, 0x62, 0x75, 0x74, 0x65, 0x18, + 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x12, 0x74, 0x69, 0x6d, 0x65, 0x73, 0x74, 0x61, 0x6d, 0x70, + 0x41, 0x74, 0x74, 0x72, 0x69, 0x62, 0x75, 0x74, 0x65, 0x12, 0x21, 0x0a, 0x0c, 0x69, 0x64, 0x5f, + 0x61, 0x74, 0x74, 0x72, 0x69, 0x62, 0x75, 0x74, 0x65, 0x18, 0x03, 0x20, 0x01, 0x28, 0x09, 0x52, + 0x0b, 0x69, 0x64, 0x41, 0x74, 0x74, 0x72, 0x69, 0x62, 0x75, 0x74, 0x65, 0x12, 0x38, 0x0a, 0x18, + 0x74, 0x6f, 0x70, 0x69, 0x63, 0x5f, 0x72, 0x75, 0x6e, 0x74, 0x69, 0x6d, 0x65, 0x5f, 0x6f, 0x76, + 0x65, 0x72, 0x72, 0x69, 0x64, 0x64, 0x65, 0x6e, 0x18, 0x04, 0x20, 0x01, 0x28, 0x09, 0x52, 0x16, + 0x74, 0x6f, 0x70, 0x69, 0x63, 0x52, 0x75, 0x6e, 0x74, 0x69, 0x6d, 0x65, 0x4f, 0x76, 0x65, 0x72, + 0x72, 0x69, 0x64, 0x64, 0x65, 0x6e, 0x22, 0xa5, 0x01, 0x0a, 0x17, 0x47, 0x72, 0x6f, 0x75, 0x70, + 0x49, 0x6e, 0x74, 0x6f, 0x42, 0x61, 0x74, 0x63, 0x68, 0x65, 0x73, 0x50, 0x61, 0x79, 0x6c, 0x6f, + 0x61, 0x64, 0x12, 0x1d, 0x0a, 0x0a, 0x62, 0x61, 0x74, 0x63, 0x68, 0x5f, 0x73, 0x69, 0x7a, 0x65, + 0x18, 0x01, 0x20, 0x01, 0x28, 0x03, 0x52, 0x09, 0x62, 0x61, 0x74, 0x63, 0x68, 0x53, 0x69, 0x7a, + 0x65, 0x12, 0x28, 0x0a, 0x10, 0x62, 0x61, 0x74, 0x63, 0x68, 0x5f, 0x73, 0x69, 0x7a, 0x65, 0x5f, + 0x62, 0x79, 0x74, 0x65, 0x73, 0x18, 0x03, 0x20, 0x01, 0x28, 0x03, 0x52, 0x0e, 0x62, 0x61, 0x74, + 0x63, 0x68, 0x53, 0x69, 0x7a, 0x65, 0x42, 0x79, 0x74, 0x65, 0x73, 0x12, 0x41, 0x0a, 0x1d, 0x6d, + 0x61, 0x78, 0x5f, 0x62, 0x75, 0x66, 0x66, 0x65, 0x72, 0x69, 0x6e, 0x67, 0x5f, 0x64, 0x75, 0x72, + 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x6d, 0x69, 0x6c, 0x6c, 0x69, 0x73, 0x18, 0x02, 0x20, 0x01, + 0x28, 0x03, 0x52, 0x1a, 0x6d, 0x61, 0x78, 0x42, 0x75, 0x66, 0x66, 0x65, 0x72, 0x69, 0x6e, 0x67, + 0x44, 0x75, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x4d, 0x69, 0x6c, 0x6c, 0x69, 0x73, 0x22, 0x40, + 0x0a, 0x13, 0x52, 0x65, 0x64, 0x69, 0x73, 0x74, 0x72, 0x69, 0x62, 0x75, 0x74, 0x65, 0x50, 0x61, + 0x79, 0x6c, 0x6f, 0x61, 0x64, 0x12, 0x29, 0x0a, 0x10, 0x61, 0x6c, 0x6c, 0x6f, 0x77, 0x5f, 0x64, + 0x75, 0x70, 0x6c, 0x69, 0x63, 0x61, 0x74, 0x65, 0x73, 0x18, 0x01, 0x20, 0x01, 0x28, 0x08, 0x52, + 0x0f, 0x61, 0x6c, 0x6c, 0x6f, 0x77, 0x44, 0x75, 0x70, 0x6c, 0x69, 0x63, 0x61, 0x74, 0x65, 0x73, + 0x22, 0x7c, 0x0a, 0x05, 0x43, 0x6f, 0x64, 0x65, 0x72, 0x12, 0x43, 0x0a, 0x04, 0x73, 0x70, 0x65, + 0x63, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x2f, 0x2e, 0x6f, 0x72, 0x67, 0x2e, 0x61, 0x70, + 0x61, 0x63, 0x68, 0x65, 0x2e, 0x62, 0x65, 0x61, 0x6d, 0x2e, 0x6d, 0x6f, 0x64, 0x65, 0x6c, 0x2e, + 0x70, 0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x46, 0x75, 0x6e, 0x63, + 0x74, 0x69, 0x6f, 0x6e, 0x53, 0x70, 0x65, 0x63, 0x52, 0x04, 0x73, 0x70, 0x65, 0x63, 0x12, 0x2e, + 0x0a, 0x13, 0x63, 0x6f, 0x6d, 0x70, 0x6f, 0x6e, 0x65, 0x6e, 0x74, 0x5f, 0x63, 0x6f, 0x64, 0x65, + 0x72, 0x5f, 0x69, 0x64, 0x73, 0x18, 0x02, 0x20, 0x03, 0x28, 0x09, 0x52, 0x11, 0x63, 0x6f, 0x6d, + 0x70, 0x6f, 0x6e, 0x65, 0x6e, 0x74, 0x43, 0x6f, 0x64, 0x65, 0x72, 0x49, 0x64, 0x73, 0x22, 0xf5, + 0x06, 0x0a, 0x0e, 0x53, 0x74, 0x61, 0x6e, 0x64, 0x61, 0x72, 0x64, 0x43, 0x6f, 0x64, 0x65, 0x72, + 0x73, 0x22, 0xe2, 0x06, 0x0a, 0x04, 0x45, 0x6e, 0x75, 0x6d, 0x12, 0x24, 0x0a, 0x05, 0x42, 0x59, + 0x54, 0x45, 0x53, 0x10, 0x00, 0x1a, 0x19, 0xa2, 0xb4, 0xfa, 0xc2, 0x05, 0x13, 0x62, 0x65, 0x61, + 0x6d, 0x3a, 0x63, 0x6f, 0x64, 0x65, 0x72, 0x3a, 0x62, 0x79, 0x74, 0x65, 0x73, 0x3a, 0x76, 0x31, + 0x12, 0x30, 0x0a, 0x0b, 0x53, 0x54, 0x52, 0x49, 0x4e, 0x47, 0x5f, 0x55, 0x54, 0x46, 0x38, 0x10, + 0x0a, 0x1a, 0x1f, 0xa2, 0xb4, 0xfa, 0xc2, 0x05, 0x19, 0x62, 0x65, 0x61, 0x6d, 0x3a, 0x63, 0x6f, + 0x64, 0x65, 0x72, 0x3a, 0x73, 0x74, 0x72, 0x69, 0x6e, 0x67, 0x5f, 0x75, 0x74, 0x66, 0x38, 0x3a, + 0x76, 0x31, 0x12, 0x1e, 0x0a, 0x02, 0x4b, 0x56, 0x10, 0x01, 0x1a, 0x16, 0xa2, 0xb4, 0xfa, 0xc2, + 0x05, 0x10, 0x62, 0x65, 0x61, 0x6d, 0x3a, 0x63, 0x6f, 0x64, 0x65, 0x72, 0x3a, 0x6b, 0x76, 0x3a, + 0x76, 0x31, 0x12, 0x22, 0x0a, 0x04, 0x42, 0x4f, 0x4f, 0x4c, 0x10, 0x0c, 0x1a, 0x18, 0xa2, 0xb4, + 0xfa, 0xc2, 0x05, 0x12, 0x62, 0x65, 0x61, 0x6d, 0x3a, 0x63, 0x6f, 0x64, 0x65, 0x72, 0x3a, 0x62, + 0x6f, 0x6f, 0x6c, 0x3a, 0x76, 0x31, 0x12, 0x26, 0x0a, 0x06, 0x56, 0x41, 0x52, 0x49, 0x4e, 0x54, + 0x10, 0x02, 0x1a, 0x1a, 0xa2, 0xb4, 0xfa, 0xc2, 0x05, 0x14, 0x62, 0x65, 0x61, 0x6d, 0x3a, 0x63, + 0x6f, 0x64, 0x65, 0x72, 0x3a, 0x76, 0x61, 0x72, 0x69, 0x6e, 0x74, 0x3a, 0x76, 0x31, 0x12, 0x26, + 0x0a, 0x06, 0x44, 0x4f, 0x55, 0x42, 0x4c, 0x45, 0x10, 0x0b, 0x1a, 0x1a, 0xa2, 0xb4, 0xfa, 0xc2, + 0x05, 0x14, 0x62, 0x65, 0x61, 0x6d, 0x3a, 0x63, 0x6f, 0x64, 0x65, 0x72, 0x3a, 0x64, 0x6f, 0x75, + 0x62, 0x6c, 0x65, 0x3a, 0x76, 0x31, 0x12, 0x2a, 0x0a, 0x08, 0x49, 0x54, 0x45, 0x52, 0x41, 0x42, + 0x4c, 0x45, 0x10, 0x03, 0x1a, 0x1c, 0xa2, 0xb4, 0xfa, 0xc2, 0x05, 0x16, 0x62, 0x65, 0x61, 0x6d, + 0x3a, 0x63, 0x6f, 0x64, 0x65, 0x72, 0x3a, 0x69, 0x74, 0x65, 0x72, 0x61, 0x62, 0x6c, 0x65, 0x3a, + 0x76, 0x31, 0x12, 0x24, 0x0a, 0x05, 0x54, 0x49, 0x4d, 0x45, 0x52, 0x10, 0x04, 0x1a, 0x19, 0xa2, + 0xb4, 0xfa, 0xc2, 0x05, 0x13, 0x62, 0x65, 0x61, 0x6d, 0x3a, 0x63, 0x6f, 0x64, 0x65, 0x72, 0x3a, + 0x74, 0x69, 0x6d, 0x65, 0x72, 0x3a, 0x76, 0x31, 0x12, 0x38, 0x0a, 0x0f, 0x49, 0x4e, 0x54, 0x45, + 0x52, 0x56, 0x41, 0x4c, 0x5f, 0x57, 0x49, 0x4e, 0x44, 0x4f, 0x57, 0x10, 0x05, 0x1a, 0x23, 0xa2, + 0xb4, 0xfa, 0xc2, 0x05, 0x1d, 0x62, 0x65, 0x61, 0x6d, 0x3a, 0x63, 0x6f, 0x64, 0x65, 0x72, 0x3a, + 0x69, 0x6e, 0x74, 0x65, 0x72, 0x76, 0x61, 0x6c, 0x5f, 0x77, 0x69, 0x6e, 0x64, 0x6f, 0x77, 0x3a, + 0x76, 0x31, 0x12, 0x34, 0x0a, 0x0d, 0x4c, 0x45, 0x4e, 0x47, 0x54, 0x48, 0x5f, 0x50, 0x52, 0x45, + 0x46, 0x49, 0x58, 0x10, 0x06, 0x1a, 0x21, 0xa2, 0xb4, 0xfa, 0xc2, 0x05, 0x1b, 0x62, 0x65, 0x61, + 0x6d, 0x3a, 0x63, 0x6f, 0x64, 0x65, 0x72, 0x3a, 0x6c, 0x65, 0x6e, 0x67, 0x74, 0x68, 0x5f, 0x70, + 0x72, 0x65, 0x66, 0x69, 0x78, 0x3a, 0x76, 0x31, 0x12, 0x34, 0x0a, 0x0d, 0x47, 0x4c, 0x4f, 0x42, + 0x41, 0x4c, 0x5f, 0x57, 0x49, 0x4e, 0x44, 0x4f, 0x57, 0x10, 0x07, 0x1a, 0x21, 0xa2, 0xb4, 0xfa, + 0xc2, 0x05, 0x1b, 0x62, 0x65, 0x61, 0x6d, 0x3a, 0x63, 0x6f, 0x64, 0x65, 0x72, 0x3a, 0x67, 0x6c, + 0x6f, 0x62, 0x61, 0x6c, 0x5f, 0x77, 0x69, 0x6e, 0x64, 0x6f, 0x77, 0x3a, 0x76, 0x31, 0x12, 0x36, + 0x0a, 0x0e, 0x57, 0x49, 0x4e, 0x44, 0x4f, 0x57, 0x45, 0x44, 0x5f, 0x56, 0x41, 0x4c, 0x55, 0x45, + 0x10, 0x08, 0x1a, 0x22, 0xa2, 0xb4, 0xfa, 0xc2, 0x05, 0x1c, 0x62, 0x65, 0x61, 0x6d, 0x3a, 0x63, + 0x6f, 0x64, 0x65, 0x72, 0x3a, 0x77, 0x69, 0x6e, 0x64, 0x6f, 0x77, 0x65, 0x64, 0x5f, 0x76, 0x61, + 0x6c, 0x75, 0x65, 0x3a, 0x76, 0x31, 0x12, 0x42, 0x0a, 0x14, 0x50, 0x41, 0x52, 0x41, 0x4d, 0x5f, + 0x57, 0x49, 0x4e, 0x44, 0x4f, 0x57, 0x45, 0x44, 0x5f, 0x56, 0x41, 0x4c, 0x55, 0x45, 0x10, 0x0e, + 0x1a, 0x28, 0xa2, 0xb4, 0xfa, 0xc2, 0x05, 0x22, 0x62, 0x65, 0x61, 0x6d, 0x3a, 0x63, 0x6f, 0x64, + 0x65, 0x72, 0x3a, 0x70, 0x61, 0x72, 0x61, 0x6d, 0x5f, 0x77, 0x69, 0x6e, 0x64, 0x6f, 0x77, 0x65, + 0x64, 0x5f, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x3a, 0x76, 0x31, 0x12, 0x44, 0x0a, 0x15, 0x53, 0x54, + 0x41, 0x54, 0x45, 0x5f, 0x42, 0x41, 0x43, 0x4b, 0x45, 0x44, 0x5f, 0x49, 0x54, 0x45, 0x52, 0x41, + 0x42, 0x4c, 0x45, 0x10, 0x09, 0x1a, 0x29, 0xa2, 0xb4, 0xfa, 0xc2, 0x05, 0x23, 0x62, 0x65, 0x61, + 0x6d, 0x3a, 0x63, 0x6f, 0x64, 0x65, 0x72, 0x3a, 0x73, 0x74, 0x61, 0x74, 0x65, 0x5f, 0x62, 0x61, + 0x63, 0x6b, 0x65, 0x64, 0x5f, 0x69, 0x74, 0x65, 0x72, 0x61, 0x62, 0x6c, 0x65, 0x3a, 0x76, 0x31, + 0x12, 0x34, 0x0a, 0x0d, 0x43, 0x55, 0x53, 0x54, 0x4f, 0x4d, 0x5f, 0x57, 0x49, 0x4e, 0x44, 0x4f, + 0x57, 0x10, 0x10, 0x1a, 0x21, 0xa2, 0xb4, 0xfa, 0xc2, 0x05, 0x1b, 0x62, 0x65, 0x61, 0x6d, 0x3a, + 0x63, 0x6f, 0x64, 0x65, 0x72, 0x3a, 0x63, 0x75, 0x73, 0x74, 0x6f, 0x6d, 0x5f, 0x77, 0x69, 0x6e, + 0x64, 0x6f, 0x77, 0x3a, 0x76, 0x31, 0x12, 0x20, 0x0a, 0x03, 0x52, 0x4f, 0x57, 0x10, 0x0d, 0x1a, + 0x17, 0xa2, 0xb4, 0xfa, 0xc2, 0x05, 0x11, 0x62, 0x65, 0x61, 0x6d, 0x3a, 0x63, 0x6f, 0x64, 0x65, + 0x72, 0x3a, 0x72, 0x6f, 0x77, 0x3a, 0x76, 0x31, 0x12, 0x30, 0x0a, 0x0b, 0x53, 0x48, 0x41, 0x52, + 0x44, 0x45, 0x44, 0x5f, 0x4b, 0x45, 0x59, 0x10, 0x0f, 0x1a, 0x1f, 0xa2, 0xb4, 0xfa, 0xc2, 0x05, + 0x19, 0x62, 0x65, 0x61, 0x6d, 0x3a, 0x63, 0x6f, 0x64, 0x65, 0x72, 0x3a, 0x73, 0x68, 0x61, 0x72, + 0x64, 0x65, 0x64, 0x5f, 0x6b, 0x65, 0x79, 0x3a, 0x76, 0x31, 0x12, 0x2a, 0x0a, 0x08, 0x4e, 0x55, + 0x4c, 0x4c, 0x41, 0x42, 0x4c, 0x45, 0x10, 0x11, 0x1a, 0x1c, 0xa2, 0xb4, 0xfa, 0xc2, 0x05, 0x16, + 0x62, 0x65, 0x61, 0x6d, 0x3a, 0x63, 0x6f, 0x64, 0x65, 0x72, 0x3a, 0x6e, 0x75, 0x6c, 0x6c, 0x61, + 0x62, 0x6c, 0x65, 0x3a, 0x76, 0x31, 0x22, 0xae, 0x06, 0x0a, 0x11, 0x57, 0x69, 0x6e, 0x64, 0x6f, + 0x77, 0x69, 0x6e, 0x67, 0x53, 0x74, 0x72, 0x61, 0x74, 0x65, 0x67, 0x79, 0x12, 0x4c, 0x0a, 0x09, + 0x77, 0x69, 0x6e, 0x64, 0x6f, 0x77, 0x5f, 0x66, 0x6e, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, + 0x2f, 0x2e, 0x6f, 0x72, 0x67, 0x2e, 0x61, 0x70, 0x61, 0x63, 0x68, 0x65, 0x2e, 0x62, 0x65, 0x61, + 0x6d, 0x2e, 0x6d, 0x6f, 0x64, 0x65, 0x6c, 0x2e, 0x70, 0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e, 0x65, + 0x2e, 0x76, 0x31, 0x2e, 0x46, 0x75, 0x6e, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x53, 0x70, 0x65, 0x63, + 0x52, 0x08, 0x77, 0x69, 0x6e, 0x64, 0x6f, 0x77, 0x46, 0x6e, 0x12, 0x56, 0x0a, 0x0c, 0x6d, 0x65, + 0x72, 0x67, 0x65, 0x5f, 0x73, 0x74, 0x61, 0x74, 0x75, 0x73, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0e, + 0x32, 0x33, 0x2e, 0x6f, 0x72, 0x67, 0x2e, 0x61, 0x70, 0x61, 0x63, 0x68, 0x65, 0x2e, 0x62, 0x65, + 0x61, 0x6d, 0x2e, 0x6d, 0x6f, 0x64, 0x65, 0x6c, 0x2e, 0x70, 0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e, + 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x4d, 0x65, 0x72, 0x67, 0x65, 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, + 0x2e, 0x45, 0x6e, 0x75, 0x6d, 0x52, 0x0b, 0x6d, 0x65, 0x72, 0x67, 0x65, 0x53, 0x74, 0x61, 0x74, + 0x75, 0x73, 0x12, 0x26, 0x0a, 0x0f, 0x77, 0x69, 0x6e, 0x64, 0x6f, 0x77, 0x5f, 0x63, 0x6f, 0x64, + 0x65, 0x72, 0x5f, 0x69, 0x64, 0x18, 0x03, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0d, 0x77, 0x69, 0x6e, + 0x64, 0x6f, 0x77, 0x43, 0x6f, 0x64, 0x65, 0x72, 0x49, 0x64, 0x12, 0x44, 0x0a, 0x07, 0x74, 0x72, + 0x69, 0x67, 0x67, 0x65, 0x72, 0x18, 0x04, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x2a, 0x2e, 0x6f, 0x72, + 0x67, 0x2e, 0x61, 0x70, 0x61, 0x63, 0x68, 0x65, 0x2e, 0x62, 0x65, 0x61, 0x6d, 0x2e, 0x6d, 0x6f, + 0x64, 0x65, 0x6c, 0x2e, 0x70, 0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e, 0x65, 0x2e, 0x76, 0x31, 0x2e, + 0x54, 0x72, 0x69, 0x67, 0x67, 0x65, 0x72, 0x52, 0x07, 0x74, 0x72, 0x69, 0x67, 0x67, 0x65, 0x72, + 0x12, 0x65, 0x0a, 0x11, 0x61, 0x63, 0x63, 0x75, 0x6d, 0x75, 0x6c, 0x61, 0x74, 0x69, 0x6f, 0x6e, + 0x5f, 0x6d, 0x6f, 0x64, 0x65, 0x18, 0x05, 0x20, 0x01, 0x28, 0x0e, 0x32, 0x38, 0x2e, 0x6f, 0x72, + 0x67, 0x2e, 0x61, 0x70, 0x61, 0x63, 0x68, 0x65, 0x2e, 0x62, 0x65, 0x61, 0x6d, 0x2e, 0x6d, 0x6f, + 0x64, 0x65, 0x6c, 0x2e, 0x70, 0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e, 0x65, 0x2e, 0x76, 0x31, 0x2e, + 0x41, 0x63, 0x63, 0x75, 0x6d, 0x75, 0x6c, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x4d, 0x6f, 0x64, 0x65, + 0x2e, 0x45, 0x6e, 0x75, 0x6d, 0x52, 0x10, 0x61, 0x63, 0x63, 0x75, 0x6d, 0x75, 0x6c, 0x61, 0x74, + 0x69, 0x6f, 0x6e, 0x4d, 0x6f, 0x64, 0x65, 0x12, 0x53, 0x0a, 0x0b, 0x6f, 0x75, 0x74, 0x70, 0x75, + 0x74, 0x5f, 0x74, 0x69, 0x6d, 0x65, 0x18, 0x06, 0x20, 0x01, 0x28, 0x0e, 0x32, 0x32, 0x2e, 0x6f, + 0x72, 0x67, 0x2e, 0x61, 0x70, 0x61, 0x63, 0x68, 0x65, 0x2e, 0x62, 0x65, 0x61, 0x6d, 0x2e, 0x6d, + 0x6f, 0x64, 0x65, 0x6c, 0x2e, 0x70, 0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e, 0x65, 0x2e, 0x76, 0x31, + 0x2e, 0x4f, 0x75, 0x74, 0x70, 0x75, 0x74, 0x54, 0x69, 0x6d, 0x65, 0x2e, 0x45, 0x6e, 0x75, 0x6d, + 0x52, 0x0a, 0x6f, 0x75, 0x74, 0x70, 0x75, 0x74, 0x54, 0x69, 0x6d, 0x65, 0x12, 0x62, 0x0a, 0x10, + 0x63, 0x6c, 0x6f, 0x73, 0x69, 0x6e, 0x67, 0x5f, 0x62, 0x65, 0x68, 0x61, 0x76, 0x69, 0x6f, 0x72, + 0x18, 0x07, 0x20, 0x01, 0x28, 0x0e, 0x32, 0x37, 0x2e, 0x6f, 0x72, 0x67, 0x2e, 0x61, 0x70, 0x61, + 0x63, 0x68, 0x65, 0x2e, 0x62, 0x65, 0x61, 0x6d, 0x2e, 0x6d, 0x6f, 0x64, 0x65, 0x6c, 0x2e, 0x70, + 0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x43, 0x6c, 0x6f, 0x73, 0x69, + 0x6e, 0x67, 0x42, 0x65, 0x68, 0x61, 0x76, 0x69, 0x6f, 0x72, 0x2e, 0x45, 0x6e, 0x75, 0x6d, 0x52, + 0x0f, 0x63, 0x6c, 0x6f, 0x73, 0x69, 0x6e, 0x67, 0x42, 0x65, 0x68, 0x61, 0x76, 0x69, 0x6f, 0x72, + 0x12, 0x29, 0x0a, 0x10, 0x61, 0x6c, 0x6c, 0x6f, 0x77, 0x65, 0x64, 0x5f, 0x6c, 0x61, 0x74, 0x65, + 0x6e, 0x65, 0x73, 0x73, 0x18, 0x08, 0x20, 0x01, 0x28, 0x03, 0x52, 0x0f, 0x61, 0x6c, 0x6c, 0x6f, + 0x77, 0x65, 0x64, 0x4c, 0x61, 0x74, 0x65, 0x6e, 0x65, 0x73, 0x73, 0x12, 0x60, 0x0a, 0x10, 0x6f, + 0x6e, 0x5f, 0x74, 0x69, 0x6d, 0x65, 0x5f, 0x62, 0x65, 0x68, 0x61, 0x76, 0x69, 0x6f, 0x72, 0x18, + 0x09, 0x20, 0x01, 0x28, 0x0e, 0x32, 0x36, 0x2e, 0x6f, 0x72, 0x67, 0x2e, 0x61, 0x70, 0x61, 0x63, + 0x68, 0x65, 0x2e, 0x62, 0x65, 0x61, 0x6d, 0x2e, 0x6d, 0x6f, 0x64, 0x65, 0x6c, 0x2e, 0x70, 0x69, + 0x70, 0x65, 0x6c, 0x69, 0x6e, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x4f, 0x6e, 0x54, 0x69, 0x6d, 0x65, + 0x42, 0x65, 0x68, 0x61, 0x76, 0x69, 0x6f, 0x72, 0x2e, 0x45, 0x6e, 0x75, 0x6d, 0x52, 0x0e, 0x6f, + 0x6e, 0x54, 0x69, 0x6d, 0x65, 0x42, 0x65, 0x68, 0x61, 0x76, 0x69, 0x6f, 0x72, 0x12, 0x31, 0x0a, + 0x15, 0x61, 0x73, 0x73, 0x69, 0x67, 0x6e, 0x73, 0x5f, 0x74, 0x6f, 0x5f, 0x6f, 0x6e, 0x65, 0x5f, + 0x77, 0x69, 0x6e, 0x64, 0x6f, 0x77, 0x18, 0x0a, 0x20, 0x01, 0x28, 0x08, 0x52, 0x12, 0x61, 0x73, + 0x73, 0x69, 0x67, 0x6e, 0x73, 0x54, 0x6f, 0x4f, 0x6e, 0x65, 0x57, 0x69, 0x6e, 0x64, 0x6f, 0x77, + 0x12, 0x25, 0x0a, 0x0e, 0x65, 0x6e, 0x76, 0x69, 0x72, 0x6f, 0x6e, 0x6d, 0x65, 0x6e, 0x74, 0x5f, + 0x69, 0x64, 0x18, 0x0b, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0d, 0x65, 0x6e, 0x76, 0x69, 0x72, 0x6f, + 0x6e, 0x6d, 0x65, 0x6e, 0x74, 0x49, 0x64, 0x22, 0x5c, 0x0a, 0x0b, 0x4d, 0x65, 0x72, 0x67, 0x65, + 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, 0x22, 0x4d, 0x0a, 0x04, 0x45, 0x6e, 0x75, 0x6d, 0x12, 0x0f, + 0x0a, 0x0b, 0x55, 0x4e, 0x53, 0x50, 0x45, 0x43, 0x49, 0x46, 0x49, 0x45, 0x44, 0x10, 0x00, 0x12, + 0x0f, 0x0a, 0x0b, 0x4e, 0x4f, 0x4e, 0x5f, 0x4d, 0x45, 0x52, 0x47, 0x49, 0x4e, 0x47, 0x10, 0x01, + 0x12, 0x0f, 0x0a, 0x0b, 0x4e, 0x45, 0x45, 0x44, 0x53, 0x5f, 0x4d, 0x45, 0x52, 0x47, 0x45, 0x10, + 0x02, 0x12, 0x12, 0x0a, 0x0e, 0x41, 0x4c, 0x52, 0x45, 0x41, 0x44, 0x59, 0x5f, 0x4d, 0x45, 0x52, + 0x47, 0x45, 0x44, 0x10, 0x03, 0x22, 0x5d, 0x0a, 0x10, 0x41, 0x63, 0x63, 0x75, 0x6d, 0x75, 0x6c, + 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x4d, 0x6f, 0x64, 0x65, 0x22, 0x49, 0x0a, 0x04, 0x45, 0x6e, 0x75, + 0x6d, 0x12, 0x0f, 0x0a, 0x0b, 0x55, 0x4e, 0x53, 0x50, 0x45, 0x43, 0x49, 0x46, 0x49, 0x45, 0x44, + 0x10, 0x00, 0x12, 0x0e, 0x0a, 0x0a, 0x44, 0x49, 0x53, 0x43, 0x41, 0x52, 0x44, 0x49, 0x4e, 0x47, + 0x10, 0x01, 0x12, 0x10, 0x0a, 0x0c, 0x41, 0x43, 0x43, 0x55, 0x4d, 0x55, 0x4c, 0x41, 0x54, 0x49, + 0x4e, 0x47, 0x10, 0x02, 0x12, 0x0e, 0x0a, 0x0a, 0x52, 0x45, 0x54, 0x52, 0x41, 0x43, 0x54, 0x49, + 0x4e, 0x47, 0x10, 0x03, 0x22, 0x51, 0x0a, 0x0f, 0x43, 0x6c, 0x6f, 0x73, 0x69, 0x6e, 0x67, 0x42, + 0x65, 0x68, 0x61, 0x76, 0x69, 0x6f, 0x72, 0x22, 0x3e, 0x0a, 0x04, 0x45, 0x6e, 0x75, 0x6d, 0x12, + 0x0f, 0x0a, 0x0b, 0x55, 0x4e, 0x53, 0x50, 0x45, 0x43, 0x49, 0x46, 0x49, 0x45, 0x44, 0x10, 0x00, + 0x12, 0x0f, 0x0a, 0x0b, 0x45, 0x4d, 0x49, 0x54, 0x5f, 0x41, 0x4c, 0x57, 0x41, 0x59, 0x53, 0x10, + 0x01, 0x12, 0x14, 0x0a, 0x10, 0x45, 0x4d, 0x49, 0x54, 0x5f, 0x49, 0x46, 0x5f, 0x4e, 0x4f, 0x4e, + 0x45, 0x4d, 0x50, 0x54, 0x59, 0x10, 0x02, 0x22, 0x50, 0x0a, 0x0e, 0x4f, 0x6e, 0x54, 0x69, 0x6d, + 0x65, 0x42, 0x65, 0x68, 0x61, 0x76, 0x69, 0x6f, 0x72, 0x22, 0x3e, 0x0a, 0x04, 0x45, 0x6e, 0x75, + 0x6d, 0x12, 0x0f, 0x0a, 0x0b, 0x55, 0x4e, 0x53, 0x50, 0x45, 0x43, 0x49, 0x46, 0x49, 0x45, 0x44, + 0x10, 0x00, 0x12, 0x0f, 0x0a, 0x0b, 0x46, 0x49, 0x52, 0x45, 0x5f, 0x41, 0x4c, 0x57, 0x41, 0x59, + 0x53, 0x10, 0x01, 0x12, 0x14, 0x0a, 0x10, 0x46, 0x49, 0x52, 0x45, 0x5f, 0x49, 0x46, 0x5f, 0x4e, + 0x4f, 0x4e, 0x45, 0x4d, 0x50, 0x54, 0x59, 0x10, 0x02, 0x22, 0x62, 0x0a, 0x0a, 0x4f, 0x75, 0x74, + 0x70, 0x75, 0x74, 0x54, 0x69, 0x6d, 0x65, 0x22, 0x54, 0x0a, 0x04, 0x45, 0x6e, 0x75, 0x6d, 0x12, + 0x0f, 0x0a, 0x0b, 0x55, 0x4e, 0x53, 0x50, 0x45, 0x43, 0x49, 0x46, 0x49, 0x45, 0x44, 0x10, 0x00, + 0x12, 0x11, 0x0a, 0x0d, 0x45, 0x4e, 0x44, 0x5f, 0x4f, 0x46, 0x5f, 0x57, 0x49, 0x4e, 0x44, 0x4f, + 0x57, 0x10, 0x01, 0x12, 0x12, 0x0a, 0x0e, 0x4c, 0x41, 0x54, 0x45, 0x53, 0x54, 0x5f, 0x49, 0x4e, + 0x5f, 0x50, 0x41, 0x4e, 0x45, 0x10, 0x02, 0x12, 0x14, 0x0a, 0x10, 0x45, 0x41, 0x52, 0x4c, 0x49, + 0x45, 0x53, 0x54, 0x5f, 0x49, 0x4e, 0x5f, 0x50, 0x41, 0x4e, 0x45, 0x10, 0x03, 0x22, 0x6e, 0x0a, + 0x0a, 0x54, 0x69, 0x6d, 0x65, 0x44, 0x6f, 0x6d, 0x61, 0x69, 0x6e, 0x22, 0x60, 0x0a, 0x04, 0x45, + 0x6e, 0x75, 0x6d, 0x12, 0x0f, 0x0a, 0x0b, 0x55, 0x4e, 0x53, 0x50, 0x45, 0x43, 0x49, 0x46, 0x49, + 0x45, 0x44, 0x10, 0x00, 0x12, 0x0e, 0x0a, 0x0a, 0x45, 0x56, 0x45, 0x4e, 0x54, 0x5f, 0x54, 0x49, + 0x4d, 0x45, 0x10, 0x01, 0x12, 0x13, 0x0a, 0x0f, 0x50, 0x52, 0x4f, 0x43, 0x45, 0x53, 0x53, 0x49, + 0x4e, 0x47, 0x5f, 0x54, 0x49, 0x4d, 0x45, 0x10, 0x02, 0x22, 0x04, 0x08, 0x03, 0x10, 0x03, 0x2a, + 0x1c, 0x53, 0x59, 0x4e, 0x43, 0x48, 0x52, 0x4f, 0x4e, 0x49, 0x5a, 0x45, 0x44, 0x5f, 0x50, 0x52, + 0x4f, 0x43, 0x45, 0x53, 0x53, 0x49, 0x4e, 0x47, 0x5f, 0x54, 0x49, 0x4d, 0x45, 0x22, 0xa3, 0x10, + 0x0a, 0x07, 0x54, 0x72, 0x69, 0x67, 0x67, 0x65, 0x72, 0x12, 0x52, 0x0a, 0x09, 0x61, 0x66, 0x74, + 0x65, 0x72, 0x5f, 0x61, 0x6c, 0x6c, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x33, 0x2e, 0x6f, + 0x72, 0x67, 0x2e, 0x61, 0x70, 0x61, 0x63, 0x68, 0x65, 0x2e, 0x62, 0x65, 0x61, 0x6d, 0x2e, 0x6d, + 0x6f, 0x64, 0x65, 0x6c, 0x2e, 0x70, 0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e, 0x65, 0x2e, 0x76, 0x31, + 0x2e, 0x54, 0x72, 0x69, 0x67, 0x67, 0x65, 0x72, 0x2e, 0x41, 0x66, 0x74, 0x65, 0x72, 0x41, 0x6c, + 0x6c, 0x48, 0x00, 0x52, 0x08, 0x61, 0x66, 0x74, 0x65, 0x72, 0x41, 0x6c, 0x6c, 0x12, 0x52, 0x0a, + 0x09, 0x61, 0x66, 0x74, 0x65, 0x72, 0x5f, 0x61, 0x6e, 0x79, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, + 0x32, 0x33, 0x2e, 0x6f, 0x72, 0x67, 0x2e, 0x61, 0x70, 0x61, 0x63, 0x68, 0x65, 0x2e, 0x62, 0x65, + 0x61, 0x6d, 0x2e, 0x6d, 0x6f, 0x64, 0x65, 0x6c, 0x2e, 0x70, 0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e, + 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x54, 0x72, 0x69, 0x67, 0x67, 0x65, 0x72, 0x2e, 0x41, 0x66, 0x74, + 0x65, 0x72, 0x41, 0x6e, 0x79, 0x48, 0x00, 0x52, 0x08, 0x61, 0x66, 0x74, 0x65, 0x72, 0x41, 0x6e, + 0x79, 0x12, 0x55, 0x0a, 0x0a, 0x61, 0x66, 0x74, 0x65, 0x72, 0x5f, 0x65, 0x61, 0x63, 0x68, 0x18, + 0x03, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x34, 0x2e, 0x6f, 0x72, 0x67, 0x2e, 0x61, 0x70, 0x61, 0x63, + 0x68, 0x65, 0x2e, 0x62, 0x65, 0x61, 0x6d, 0x2e, 0x6d, 0x6f, 0x64, 0x65, 0x6c, 0x2e, 0x70, 0x69, + 0x70, 0x65, 0x6c, 0x69, 0x6e, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x54, 0x72, 0x69, 0x67, 0x67, 0x65, + 0x72, 0x2e, 0x41, 0x66, 0x74, 0x65, 0x72, 0x45, 0x61, 0x63, 0x68, 0x48, 0x00, 0x52, 0x09, 0x61, + 0x66, 0x74, 0x65, 0x72, 0x45, 0x61, 0x63, 0x68, 0x12, 0x6c, 0x0a, 0x13, 0x61, 0x66, 0x74, 0x65, + 0x72, 0x5f, 0x65, 0x6e, 0x64, 0x5f, 0x6f, 0x66, 0x5f, 0x77, 0x69, 0x6e, 0x64, 0x6f, 0x77, 0x18, + 0x04, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x3b, 0x2e, 0x6f, 0x72, 0x67, 0x2e, 0x61, 0x70, 0x61, 0x63, + 0x68, 0x65, 0x2e, 0x62, 0x65, 0x61, 0x6d, 0x2e, 0x6d, 0x6f, 0x64, 0x65, 0x6c, 0x2e, 0x70, 0x69, + 0x70, 0x65, 0x6c, 0x69, 0x6e, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x54, 0x72, 0x69, 0x67, 0x67, 0x65, + 0x72, 0x2e, 0x41, 0x66, 0x74, 0x65, 0x72, 0x45, 0x6e, 0x64, 0x4f, 0x66, 0x57, 0x69, 0x6e, 0x64, + 0x6f, 0x77, 0x48, 0x00, 0x52, 0x10, 0x61, 0x66, 0x74, 0x65, 0x72, 0x45, 0x6e, 0x64, 0x4f, 0x66, + 0x57, 0x69, 0x6e, 0x64, 0x6f, 0x77, 0x12, 0x74, 0x0a, 0x15, 0x61, 0x66, 0x74, 0x65, 0x72, 0x5f, + 0x70, 0x72, 0x6f, 0x63, 0x65, 0x73, 0x73, 0x69, 0x6e, 0x67, 0x5f, 0x74, 0x69, 0x6d, 0x65, 0x18, + 0x05, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x3e, 0x2e, 0x6f, 0x72, 0x67, 0x2e, 0x61, 0x70, 0x61, 0x63, + 0x68, 0x65, 0x2e, 0x62, 0x65, 0x61, 0x6d, 0x2e, 0x6d, 0x6f, 0x64, 0x65, 0x6c, 0x2e, 0x70, 0x69, + 0x70, 0x65, 0x6c, 0x69, 0x6e, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x54, 0x72, 0x69, 0x67, 0x67, 0x65, + 0x72, 0x2e, 0x41, 0x66, 0x74, 0x65, 0x72, 0x50, 0x72, 0x6f, 0x63, 0x65, 0x73, 0x73, 0x69, 0x6e, + 0x67, 0x54, 0x69, 0x6d, 0x65, 0x48, 0x00, 0x52, 0x13, 0x61, 0x66, 0x74, 0x65, 0x72, 0x50, 0x72, + 0x6f, 0x63, 0x65, 0x73, 0x73, 0x69, 0x6e, 0x67, 0x54, 0x69, 0x6d, 0x65, 0x12, 0x99, 0x01, 0x0a, + 0x22, 0x61, 0x66, 0x74, 0x65, 0x72, 0x5f, 0x73, 0x79, 0x6e, 0x63, 0x68, 0x72, 0x6f, 0x6e, 0x69, + 0x7a, 0x65, 0x64, 0x5f, 0x70, 0x72, 0x6f, 0x63, 0x65, 0x73, 0x73, 0x69, 0x6e, 0x67, 0x5f, 0x74, + 0x69, 0x6d, 0x65, 0x18, 0x06, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x4a, 0x2e, 0x6f, 0x72, 0x67, 0x2e, + 0x61, 0x70, 0x61, 0x63, 0x68, 0x65, 0x2e, 0x62, 0x65, 0x61, 0x6d, 0x2e, 0x6d, 0x6f, 0x64, 0x65, + 0x6c, 0x2e, 0x70, 0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x54, 0x72, + 0x69, 0x67, 0x67, 0x65, 0x72, 0x2e, 0x41, 0x66, 0x74, 0x65, 0x72, 0x53, 0x79, 0x6e, 0x63, 0x68, + 0x72, 0x6f, 0x6e, 0x69, 0x7a, 0x65, 0x64, 0x50, 0x72, 0x6f, 0x63, 0x65, 0x73, 0x73, 0x69, 0x6e, + 0x67, 0x54, 0x69, 0x6d, 0x65, 0x48, 0x00, 0x52, 0x1f, 0x61, 0x66, 0x74, 0x65, 0x72, 0x53, 0x79, + 0x6e, 0x63, 0x68, 0x72, 0x6f, 0x6e, 0x69, 0x7a, 0x65, 0x64, 0x50, 0x72, 0x6f, 0x63, 0x65, 0x73, + 0x73, 0x69, 0x6e, 0x67, 0x54, 0x69, 0x6d, 0x65, 0x12, 0x4b, 0x0a, 0x06, 0x61, 0x6c, 0x77, 0x61, + 0x79, 0x73, 0x18, 0x0c, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x31, 0x2e, 0x6f, 0x72, 0x67, 0x2e, 0x61, + 0x70, 0x61, 0x63, 0x68, 0x65, 0x2e, 0x62, 0x65, 0x61, 0x6d, 0x2e, 0x6d, 0x6f, 0x64, 0x65, 0x6c, + 0x2e, 0x70, 0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x54, 0x72, 0x69, + 0x67, 0x67, 0x65, 0x72, 0x2e, 0x41, 0x6c, 0x77, 0x61, 0x79, 0x73, 0x48, 0x00, 0x52, 0x06, 0x61, + 0x6c, 0x77, 0x61, 0x79, 0x73, 0x12, 0x4e, 0x0a, 0x07, 0x64, 0x65, 0x66, 0x61, 0x75, 0x6c, 0x74, + 0x18, 0x07, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x32, 0x2e, 0x6f, 0x72, 0x67, 0x2e, 0x61, 0x70, 0x61, + 0x63, 0x68, 0x65, 0x2e, 0x62, 0x65, 0x61, 0x6d, 0x2e, 0x6d, 0x6f, 0x64, 0x65, 0x6c, 0x2e, 0x70, + 0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x54, 0x72, 0x69, 0x67, 0x67, + 0x65, 0x72, 0x2e, 0x44, 0x65, 0x66, 0x61, 0x75, 0x6c, 0x74, 0x48, 0x00, 0x52, 0x07, 0x64, 0x65, + 0x66, 0x61, 0x75, 0x6c, 0x74, 0x12, 0x5e, 0x0a, 0x0d, 0x65, 0x6c, 0x65, 0x6d, 0x65, 0x6e, 0x74, + 0x5f, 0x63, 0x6f, 0x75, 0x6e, 0x74, 0x18, 0x08, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x37, 0x2e, 0x6f, + 0x72, 0x67, 0x2e, 0x61, 0x70, 0x61, 0x63, 0x68, 0x65, 0x2e, 0x62, 0x65, 0x61, 0x6d, 0x2e, 0x6d, + 0x6f, 0x64, 0x65, 0x6c, 0x2e, 0x70, 0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e, 0x65, 0x2e, 0x76, 0x31, + 0x2e, 0x54, 0x72, 0x69, 0x67, 0x67, 0x65, 0x72, 0x2e, 0x45, 0x6c, 0x65, 0x6d, 0x65, 0x6e, 0x74, + 0x43, 0x6f, 0x75, 0x6e, 0x74, 0x48, 0x00, 0x52, 0x0c, 0x65, 0x6c, 0x65, 0x6d, 0x65, 0x6e, 0x74, + 0x43, 0x6f, 0x75, 0x6e, 0x74, 0x12, 0x48, 0x0a, 0x05, 0x6e, 0x65, 0x76, 0x65, 0x72, 0x18, 0x09, + 0x20, 0x01, 0x28, 0x0b, 0x32, 0x30, 0x2e, 0x6f, 0x72, 0x67, 0x2e, 0x61, 0x70, 0x61, 0x63, 0x68, + 0x65, 0x2e, 0x62, 0x65, 0x61, 0x6d, 0x2e, 0x6d, 0x6f, 0x64, 0x65, 0x6c, 0x2e, 0x70, 0x69, 0x70, + 0x65, 0x6c, 0x69, 0x6e, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x54, 0x72, 0x69, 0x67, 0x67, 0x65, 0x72, + 0x2e, 0x4e, 0x65, 0x76, 0x65, 0x72, 0x48, 0x00, 0x52, 0x05, 0x6e, 0x65, 0x76, 0x65, 0x72, 0x12, + 0x55, 0x0a, 0x0a, 0x6f, 0x72, 0x5f, 0x66, 0x69, 0x6e, 0x61, 0x6c, 0x6c, 0x79, 0x18, 0x0a, 0x20, + 0x01, 0x28, 0x0b, 0x32, 0x34, 0x2e, 0x6f, 0x72, 0x67, 0x2e, 0x61, 0x70, 0x61, 0x63, 0x68, 0x65, + 0x2e, 0x62, 0x65, 0x61, 0x6d, 0x2e, 0x6d, 0x6f, 0x64, 0x65, 0x6c, 0x2e, 0x70, 0x69, 0x70, 0x65, + 0x6c, 0x69, 0x6e, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x54, 0x72, 0x69, 0x67, 0x67, 0x65, 0x72, 0x2e, + 0x4f, 0x72, 0x46, 0x69, 0x6e, 0x61, 0x6c, 0x6c, 0x79, 0x48, 0x00, 0x52, 0x09, 0x6f, 0x72, 0x46, + 0x69, 0x6e, 0x61, 0x6c, 0x6c, 0x79, 0x12, 0x4b, 0x0a, 0x06, 0x72, 0x65, 0x70, 0x65, 0x61, 0x74, + 0x18, 0x0b, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x31, 0x2e, 0x6f, 0x72, 0x67, 0x2e, 0x61, 0x70, 0x61, + 0x63, 0x68, 0x65, 0x2e, 0x62, 0x65, 0x61, 0x6d, 0x2e, 0x6d, 0x6f, 0x64, 0x65, 0x6c, 0x2e, 0x70, + 0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x54, 0x72, 0x69, 0x67, 0x67, + 0x65, 0x72, 0x2e, 0x52, 0x65, 0x70, 0x65, 0x61, 0x74, 0x48, 0x00, 0x52, 0x06, 0x72, 0x65, 0x70, + 0x65, 0x61, 0x74, 0x1a, 0x58, 0x0a, 0x08, 0x41, 0x66, 0x74, 0x65, 0x72, 0x41, 0x6c, 0x6c, 0x12, + 0x4c, 0x0a, 0x0b, 0x73, 0x75, 0x62, 0x74, 0x72, 0x69, 0x67, 0x67, 0x65, 0x72, 0x73, 0x18, 0x01, + 0x20, 0x03, 0x28, 0x0b, 0x32, 0x2a, 0x2e, 0x6f, 0x72, 0x67, 0x2e, 0x61, 0x70, 0x61, 0x63, 0x68, + 0x65, 0x2e, 0x62, 0x65, 0x61, 0x6d, 0x2e, 0x6d, 0x6f, 0x64, 0x65, 0x6c, 0x2e, 0x70, 0x69, 0x70, + 0x65, 0x6c, 0x69, 0x6e, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x54, 0x72, 0x69, 0x67, 0x67, 0x65, 0x72, + 0x52, 0x0b, 0x73, 0x75, 0x62, 0x74, 0x72, 0x69, 0x67, 0x67, 0x65, 0x72, 0x73, 0x1a, 0x58, 0x0a, + 0x08, 0x41, 0x66, 0x74, 0x65, 0x72, 0x41, 0x6e, 0x79, 0x12, 0x4c, 0x0a, 0x0b, 0x73, 0x75, 0x62, + 0x74, 0x72, 0x69, 0x67, 0x67, 0x65, 0x72, 0x73, 0x18, 0x01, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x2a, + 0x2e, 0x6f, 0x72, 0x67, 0x2e, 0x61, 0x70, 0x61, 0x63, 0x68, 0x65, 0x2e, 0x62, 0x65, 0x61, 0x6d, + 0x2e, 0x6d, 0x6f, 0x64, 0x65, 0x6c, 0x2e, 0x70, 0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e, 0x65, 0x2e, + 0x76, 0x31, 0x2e, 0x54, 0x72, 0x69, 0x67, 0x67, 0x65, 0x72, 0x52, 0x0b, 0x73, 0x75, 0x62, 0x74, + 0x72, 0x69, 0x67, 0x67, 0x65, 0x72, 0x73, 0x1a, 0x59, 0x0a, 0x09, 0x41, 0x66, 0x74, 0x65, 0x72, + 0x45, 0x61, 0x63, 0x68, 0x12, 0x4c, 0x0a, 0x0b, 0x73, 0x75, 0x62, 0x74, 0x72, 0x69, 0x67, 0x67, + 0x65, 0x72, 0x73, 0x18, 0x01, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x2a, 0x2e, 0x6f, 0x72, 0x67, 0x2e, + 0x61, 0x70, 0x61, 0x63, 0x68, 0x65, 0x2e, 0x62, 0x65, 0x61, 0x6d, 0x2e, 0x6d, 0x6f, 0x64, 0x65, + 0x6c, 0x2e, 0x70, 0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x54, 0x72, + 0x69, 0x67, 0x67, 0x65, 0x72, 0x52, 0x0b, 0x73, 0x75, 0x62, 0x74, 0x72, 0x69, 0x67, 0x67, 0x65, + 0x72, 0x73, 0x1a, 0xb2, 0x01, 0x0a, 0x10, 0x41, 0x66, 0x74, 0x65, 0x72, 0x45, 0x6e, 0x64, 0x4f, + 0x66, 0x57, 0x69, 0x6e, 0x64, 0x6f, 0x77, 0x12, 0x4f, 0x0a, 0x0d, 0x65, 0x61, 0x72, 0x6c, 0x79, + 0x5f, 0x66, 0x69, 0x72, 0x69, 0x6e, 0x67, 0x73, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x2a, + 0x2e, 0x6f, 0x72, 0x67, 0x2e, 0x61, 0x70, 0x61, 0x63, 0x68, 0x65, 0x2e, 0x62, 0x65, 0x61, 0x6d, + 0x2e, 0x6d, 0x6f, 0x64, 0x65, 0x6c, 0x2e, 0x70, 0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e, 0x65, 0x2e, + 0x76, 0x31, 0x2e, 0x54, 0x72, 0x69, 0x67, 0x67, 0x65, 0x72, 0x52, 0x0c, 0x65, 0x61, 0x72, 0x6c, + 0x79, 0x46, 0x69, 0x72, 0x69, 0x6e, 0x67, 0x73, 0x12, 0x4d, 0x0a, 0x0c, 0x6c, 0x61, 0x74, 0x65, + 0x5f, 0x66, 0x69, 0x72, 0x69, 0x6e, 0x67, 0x73, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x2a, + 0x2e, 0x6f, 0x72, 0x67, 0x2e, 0x61, 0x70, 0x61, 0x63, 0x68, 0x65, 0x2e, 0x62, 0x65, 0x61, 0x6d, + 0x2e, 0x6d, 0x6f, 0x64, 0x65, 0x6c, 0x2e, 0x70, 0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e, 0x65, 0x2e, + 0x76, 0x31, 0x2e, 0x54, 0x72, 0x69, 0x67, 0x67, 0x65, 0x72, 0x52, 0x0b, 0x6c, 0x61, 0x74, 0x65, + 0x46, 0x69, 0x72, 0x69, 0x6e, 0x67, 0x73, 0x1a, 0x7f, 0x0a, 0x13, 0x41, 0x66, 0x74, 0x65, 0x72, + 0x50, 0x72, 0x6f, 0x63, 0x65, 0x73, 0x73, 0x69, 0x6e, 0x67, 0x54, 0x69, 0x6d, 0x65, 0x12, 0x68, + 0x0a, 0x14, 0x74, 0x69, 0x6d, 0x65, 0x73, 0x74, 0x61, 0x6d, 0x70, 0x5f, 0x74, 0x72, 0x61, 0x6e, + 0x73, 0x66, 0x6f, 0x72, 0x6d, 0x73, 0x18, 0x01, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x35, 0x2e, 0x6f, + 0x72, 0x67, 0x2e, 0x61, 0x70, 0x61, 0x63, 0x68, 0x65, 0x2e, 0x62, 0x65, 0x61, 0x6d, 0x2e, 0x6d, + 0x6f, 0x64, 0x65, 0x6c, 0x2e, 0x70, 0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e, 0x65, 0x2e, 0x76, 0x31, + 0x2e, 0x54, 0x69, 0x6d, 0x65, 0x73, 0x74, 0x61, 0x6d, 0x70, 0x54, 0x72, 0x61, 0x6e, 0x73, 0x66, + 0x6f, 0x72, 0x6d, 0x52, 0x13, 0x74, 0x69, 0x6d, 0x65, 0x73, 0x74, 0x61, 0x6d, 0x70, 0x54, 0x72, + 0x61, 0x6e, 0x73, 0x66, 0x6f, 0x72, 0x6d, 0x73, 0x1a, 0x21, 0x0a, 0x1f, 0x41, 0x66, 0x74, 0x65, + 0x72, 0x53, 0x79, 0x6e, 0x63, 0x68, 0x72, 0x6f, 0x6e, 0x69, 0x7a, 0x65, 0x64, 0x50, 0x72, 0x6f, + 0x63, 0x65, 0x73, 0x73, 0x69, 0x6e, 0x67, 0x54, 0x69, 0x6d, 0x65, 0x1a, 0x09, 0x0a, 0x07, 0x44, + 0x65, 0x66, 0x61, 0x75, 0x6c, 0x74, 0x1a, 0x33, 0x0a, 0x0c, 0x45, 0x6c, 0x65, 0x6d, 0x65, 0x6e, + 0x74, 0x43, 0x6f, 0x75, 0x6e, 0x74, 0x12, 0x23, 0x0a, 0x0d, 0x65, 0x6c, 0x65, 0x6d, 0x65, 0x6e, + 0x74, 0x5f, 0x63, 0x6f, 0x75, 0x6e, 0x74, 0x18, 0x01, 0x20, 0x01, 0x28, 0x05, 0x52, 0x0c, 0x65, + 0x6c, 0x65, 0x6d, 0x65, 0x6e, 0x74, 0x43, 0x6f, 0x75, 0x6e, 0x74, 0x1a, 0x07, 0x0a, 0x05, 0x4e, + 0x65, 0x76, 0x65, 0x72, 0x1a, 0x08, 0x0a, 0x06, 0x41, 0x6c, 0x77, 0x61, 0x79, 0x73, 0x1a, 0x91, + 0x01, 0x0a, 0x09, 0x4f, 0x72, 0x46, 0x69, 0x6e, 0x61, 0x6c, 0x6c, 0x79, 0x12, 0x3e, 0x0a, 0x04, + 0x6d, 0x61, 0x69, 0x6e, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x2a, 0x2e, 0x6f, 0x72, 0x67, + 0x2e, 0x61, 0x70, 0x61, 0x63, 0x68, 0x65, 0x2e, 0x62, 0x65, 0x61, 0x6d, 0x2e, 0x6d, 0x6f, 0x64, + 0x65, 0x6c, 0x2e, 0x70, 0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x54, + 0x72, 0x69, 0x67, 0x67, 0x65, 0x72, 0x52, 0x04, 0x6d, 0x61, 0x69, 0x6e, 0x12, 0x44, 0x0a, 0x07, + 0x66, 0x69, 0x6e, 0x61, 0x6c, 0x6c, 0x79, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x2a, 0x2e, + 0x6f, 0x72, 0x67, 0x2e, 0x61, 0x70, 0x61, 0x63, 0x68, 0x65, 0x2e, 0x62, 0x65, 0x61, 0x6d, 0x2e, + 0x6d, 0x6f, 0x64, 0x65, 0x6c, 0x2e, 0x70, 0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e, 0x65, 0x2e, 0x76, + 0x31, 0x2e, 0x54, 0x72, 0x69, 0x67, 0x67, 0x65, 0x72, 0x52, 0x07, 0x66, 0x69, 0x6e, 0x61, 0x6c, + 0x6c, 0x79, 0x1a, 0x54, 0x0a, 0x06, 0x52, 0x65, 0x70, 0x65, 0x61, 0x74, 0x12, 0x4a, 0x0a, 0x0a, + 0x73, 0x75, 0x62, 0x74, 0x72, 0x69, 0x67, 0x67, 0x65, 0x72, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, + 0x32, 0x2a, 0x2e, 0x6f, 0x72, 0x67, 0x2e, 0x61, 0x70, 0x61, 0x63, 0x68, 0x65, 0x2e, 0x62, 0x65, + 0x61, 0x6d, 0x2e, 0x6d, 0x6f, 0x64, 0x65, 0x6c, 0x2e, 0x70, 0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e, + 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x54, 0x72, 0x69, 0x67, 0x67, 0x65, 0x72, 0x52, 0x0a, 0x73, 0x75, + 0x62, 0x74, 0x72, 0x69, 0x67, 0x67, 0x65, 0x72, 0x42, 0x09, 0x0a, 0x07, 0x74, 0x72, 0x69, 0x67, + 0x67, 0x65, 0x72, 0x22, 0xc3, 0x02, 0x0a, 0x12, 0x54, 0x69, 0x6d, 0x65, 0x73, 0x74, 0x61, 0x6d, + 0x70, 0x54, 0x72, 0x61, 0x6e, 0x73, 0x66, 0x6f, 0x72, 0x6d, 0x12, 0x53, 0x0a, 0x05, 0x64, 0x65, + 0x6c, 0x61, 0x79, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x3b, 0x2e, 0x6f, 0x72, 0x67, 0x2e, + 0x61, 0x70, 0x61, 0x63, 0x68, 0x65, 0x2e, 0x62, 0x65, 0x61, 0x6d, 0x2e, 0x6d, 0x6f, 0x64, 0x65, + 0x6c, 0x2e, 0x70, 0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x54, 0x69, + 0x6d, 0x65, 0x73, 0x74, 0x61, 0x6d, 0x70, 0x54, 0x72, 0x61, 0x6e, 0x73, 0x66, 0x6f, 0x72, 0x6d, + 0x2e, 0x44, 0x65, 0x6c, 0x61, 0x79, 0x48, 0x00, 0x52, 0x05, 0x64, 0x65, 0x6c, 0x61, 0x79, 0x12, + 0x5a, 0x0a, 0x08, 0x61, 0x6c, 0x69, 0x67, 0x6e, 0x5f, 0x74, 0x6f, 0x18, 0x02, 0x20, 0x01, 0x28, + 0x0b, 0x32, 0x3d, 0x2e, 0x6f, 0x72, 0x67, 0x2e, 0x61, 0x70, 0x61, 0x63, 0x68, 0x65, 0x2e, 0x62, + 0x65, 0x61, 0x6d, 0x2e, 0x6d, 0x6f, 0x64, 0x65, 0x6c, 0x2e, 0x70, 0x69, 0x70, 0x65, 0x6c, 0x69, + 0x6e, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x54, 0x69, 0x6d, 0x65, 0x73, 0x74, 0x61, 0x6d, 0x70, 0x54, + 0x72, 0x61, 0x6e, 0x73, 0x66, 0x6f, 0x72, 0x6d, 0x2e, 0x41, 0x6c, 0x69, 0x67, 0x6e, 0x54, 0x6f, + 0x48, 0x00, 0x52, 0x07, 0x61, 0x6c, 0x69, 0x67, 0x6e, 0x54, 0x6f, 0x1a, 0x2a, 0x0a, 0x05, 0x44, + 0x65, 0x6c, 0x61, 0x79, 0x12, 0x21, 0x0a, 0x0c, 0x64, 0x65, 0x6c, 0x61, 0x79, 0x5f, 0x6d, 0x69, + 0x6c, 0x6c, 0x69, 0x73, 0x18, 0x01, 0x20, 0x01, 0x28, 0x03, 0x52, 0x0b, 0x64, 0x65, 0x6c, 0x61, + 0x79, 0x4d, 0x69, 0x6c, 0x6c, 0x69, 0x73, 0x1a, 0x39, 0x0a, 0x07, 0x41, 0x6c, 0x69, 0x67, 0x6e, + 0x54, 0x6f, 0x12, 0x16, 0x0a, 0x06, 0x70, 0x65, 0x72, 0x69, 0x6f, 0x64, 0x18, 0x03, 0x20, 0x01, + 0x28, 0x03, 0x52, 0x06, 0x70, 0x65, 0x72, 0x69, 0x6f, 0x64, 0x12, 0x16, 0x0a, 0x06, 0x6f, 0x66, + 0x66, 0x73, 0x65, 0x74, 0x18, 0x04, 0x20, 0x01, 0x28, 0x03, 0x52, 0x06, 0x6f, 0x66, 0x66, 0x73, + 0x65, 0x74, 0x42, 0x15, 0x0a, 0x13, 0x74, 0x69, 0x6d, 0x65, 0x73, 0x74, 0x61, 0x6d, 0x70, 0x5f, + 0x74, 0x72, 0x61, 0x6e, 0x73, 0x66, 0x6f, 0x72, 0x6d, 0x22, 0x8a, 0x02, 0x0a, 0x09, 0x53, 0x69, + 0x64, 0x65, 0x49, 0x6e, 0x70, 0x75, 0x74, 0x12, 0x56, 0x0a, 0x0e, 0x61, 0x63, 0x63, 0x65, 0x73, + 0x73, 0x5f, 0x70, 0x61, 0x74, 0x74, 0x65, 0x72, 0x6e, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, + 0x2f, 0x2e, 0x6f, 0x72, 0x67, 0x2e, 0x61, 0x70, 0x61, 0x63, 0x68, 0x65, 0x2e, 0x62, 0x65, 0x61, + 0x6d, 0x2e, 0x6d, 0x6f, 0x64, 0x65, 0x6c, 0x2e, 0x70, 0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e, 0x65, + 0x2e, 0x76, 0x31, 0x2e, 0x46, 0x75, 0x6e, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x53, 0x70, 0x65, 0x63, + 0x52, 0x0d, 0x61, 0x63, 0x63, 0x65, 0x73, 0x73, 0x50, 0x61, 0x74, 0x74, 0x65, 0x72, 0x6e, 0x12, + 0x48, 0x0a, 0x07, 0x76, 0x69, 0x65, 0x77, 0x5f, 0x66, 0x6e, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, + 0x32, 0x2f, 0x2e, 0x6f, 0x72, 0x67, 0x2e, 0x61, 0x70, 0x61, 0x63, 0x68, 0x65, 0x2e, 0x62, 0x65, + 0x61, 0x6d, 0x2e, 0x6d, 0x6f, 0x64, 0x65, 0x6c, 0x2e, 0x70, 0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e, + 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x46, 0x75, 0x6e, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x53, 0x70, 0x65, + 0x63, 0x52, 0x06, 0x76, 0x69, 0x65, 0x77, 0x46, 0x6e, 0x12, 0x5b, 0x0a, 0x11, 0x77, 0x69, 0x6e, + 0x64, 0x6f, 0x77, 0x5f, 0x6d, 0x61, 0x70, 0x70, 0x69, 0x6e, 0x67, 0x5f, 0x66, 0x6e, 0x18, 0x03, + 0x20, 0x01, 0x28, 0x0b, 0x32, 0x2f, 0x2e, 0x6f, 0x72, 0x67, 0x2e, 0x61, 0x70, 0x61, 0x63, 0x68, + 0x65, 0x2e, 0x62, 0x65, 0x61, 0x6d, 0x2e, 0x6d, 0x6f, 0x64, 0x65, 0x6c, 0x2e, 0x70, 0x69, 0x70, + 0x65, 0x6c, 0x69, 0x6e, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x46, 0x75, 0x6e, 0x63, 0x74, 0x69, 0x6f, + 0x6e, 0x53, 0x70, 0x65, 0x63, 0x52, 0x0f, 0x77, 0x69, 0x6e, 0x64, 0x6f, 0x77, 0x4d, 0x61, 0x70, + 0x70, 0x69, 0x6e, 0x67, 0x46, 0x6e, 0x22, 0x89, 0x04, 0x0a, 0x11, 0x53, 0x74, 0x61, 0x6e, 0x64, + 0x61, 0x72, 0x64, 0x41, 0x72, 0x74, 0x69, 0x66, 0x61, 0x63, 0x74, 0x73, 0x22, 0x9f, 0x02, 0x0a, + 0x05, 0x54, 0x79, 0x70, 0x65, 0x73, 0x12, 0x2a, 0x0a, 0x04, 0x46, 0x49, 0x4c, 0x45, 0x10, 0x00, + 0x1a, 0x20, 0xa2, 0xb4, 0xfa, 0xc2, 0x05, 0x1a, 0x62, 0x65, 0x61, 0x6d, 0x3a, 0x61, 0x72, 0x74, + 0x69, 0x66, 0x61, 0x63, 0x74, 0x3a, 0x74, 0x79, 0x70, 0x65, 0x3a, 0x66, 0x69, 0x6c, 0x65, 0x3a, + 0x76, 0x31, 0x12, 0x28, 0x0a, 0x03, 0x55, 0x52, 0x4c, 0x10, 0x01, 0x1a, 0x1f, 0xa2, 0xb4, 0xfa, + 0xc2, 0x05, 0x19, 0x62, 0x65, 0x61, 0x6d, 0x3a, 0x61, 0x72, 0x74, 0x69, 0x66, 0x61, 0x63, 0x74, + 0x3a, 0x74, 0x79, 0x70, 0x65, 0x3a, 0x75, 0x72, 0x6c, 0x3a, 0x76, 0x31, 0x12, 0x32, 0x0a, 0x08, + 0x45, 0x4d, 0x42, 0x45, 0x44, 0x44, 0x45, 0x44, 0x10, 0x02, 0x1a, 0x24, 0xa2, 0xb4, 0xfa, 0xc2, + 0x05, 0x1e, 0x62, 0x65, 0x61, 0x6d, 0x3a, 0x61, 0x72, 0x74, 0x69, 0x66, 0x61, 0x63, 0x74, 0x3a, + 0x74, 0x79, 0x70, 0x65, 0x3a, 0x65, 0x6d, 0x62, 0x65, 0x64, 0x64, 0x65, 0x64, 0x3a, 0x76, 0x31, + 0x12, 0x2a, 0x0a, 0x04, 0x50, 0x59, 0x50, 0x49, 0x10, 0x03, 0x1a, 0x20, 0xa2, 0xb4, 0xfa, 0xc2, + 0x05, 0x1a, 0x62, 0x65, 0x61, 0x6d, 0x3a, 0x61, 0x72, 0x74, 0x69, 0x66, 0x61, 0x63, 0x74, 0x3a, + 0x74, 0x79, 0x70, 0x65, 0x3a, 0x70, 0x79, 0x70, 0x69, 0x3a, 0x76, 0x31, 0x12, 0x2c, 0x0a, 0x05, + 0x4d, 0x41, 0x56, 0x45, 0x4e, 0x10, 0x04, 0x1a, 0x21, 0xa2, 0xb4, 0xfa, 0xc2, 0x05, 0x1b, 0x62, + 0x65, 0x61, 0x6d, 0x3a, 0x61, 0x72, 0x74, 0x69, 0x66, 0x61, 0x63, 0x74, 0x3a, 0x74, 0x79, 0x70, + 0x65, 0x3a, 0x6d, 0x61, 0x76, 0x65, 0x6e, 0x3a, 0x76, 0x31, 0x12, 0x32, 0x0a, 0x08, 0x44, 0x45, + 0x46, 0x45, 0x52, 0x52, 0x45, 0x44, 0x10, 0x05, 0x1a, 0x24, 0xa2, 0xb4, 0xfa, 0xc2, 0x05, 0x1e, + 0x62, 0x65, 0x61, 0x6d, 0x3a, 0x61, 0x72, 0x74, 0x69, 0x66, 0x61, 0x63, 0x74, 0x3a, 0x74, 0x79, + 0x70, 0x65, 0x3a, 0x64, 0x65, 0x66, 0x65, 0x72, 0x72, 0x65, 0x64, 0x3a, 0x76, 0x31, 0x22, 0xd1, + 0x01, 0x0a, 0x05, 0x52, 0x6f, 0x6c, 0x65, 0x73, 0x12, 0x36, 0x0a, 0x0a, 0x53, 0x54, 0x41, 0x47, + 0x49, 0x4e, 0x47, 0x5f, 0x54, 0x4f, 0x10, 0x00, 0x1a, 0x26, 0xa2, 0xb4, 0xfa, 0xc2, 0x05, 0x20, + 0x62, 0x65, 0x61, 0x6d, 0x3a, 0x61, 0x72, 0x74, 0x69, 0x66, 0x61, 0x63, 0x74, 0x3a, 0x72, 0x6f, + 0x6c, 0x65, 0x3a, 0x73, 0x74, 0x61, 0x67, 0x69, 0x6e, 0x67, 0x5f, 0x74, 0x6f, 0x3a, 0x76, 0x31, + 0x12, 0x4c, 0x0a, 0x15, 0x50, 0x49, 0x50, 0x5f, 0x52, 0x45, 0x51, 0x55, 0x49, 0x52, 0x45, 0x4d, + 0x45, 0x4e, 0x54, 0x53, 0x5f, 0x46, 0x49, 0x4c, 0x45, 0x10, 0x01, 0x1a, 0x31, 0xa2, 0xb4, 0xfa, + 0xc2, 0x05, 0x2b, 0x62, 0x65, 0x61, 0x6d, 0x3a, 0x61, 0x72, 0x74, 0x69, 0x66, 0x61, 0x63, 0x74, + 0x3a, 0x72, 0x6f, 0x6c, 0x65, 0x3a, 0x70, 0x69, 0x70, 0x5f, 0x72, 0x65, 0x71, 0x75, 0x69, 0x72, + 0x65, 0x6d, 0x65, 0x6e, 0x74, 0x73, 0x5f, 0x66, 0x69, 0x6c, 0x65, 0x3a, 0x76, 0x31, 0x12, 0x42, + 0x0a, 0x10, 0x47, 0x4f, 0x5f, 0x57, 0x4f, 0x52, 0x4b, 0x45, 0x52, 0x5f, 0x42, 0x49, 0x4e, 0x41, + 0x52, 0x59, 0x10, 0x02, 0x1a, 0x2c, 0xa2, 0xb4, 0xfa, 0xc2, 0x05, 0x26, 0x62, 0x65, 0x61, 0x6d, + 0x3a, 0x61, 0x72, 0x74, 0x69, 0x66, 0x61, 0x63, 0x74, 0x3a, 0x72, 0x6f, 0x6c, 0x65, 0x3a, 0x67, + 0x6f, 0x5f, 0x77, 0x6f, 0x72, 0x6b, 0x65, 0x72, 0x5f, 0x62, 0x69, 0x6e, 0x61, 0x72, 0x79, 0x3a, + 0x76, 0x31, 0x22, 0x41, 0x0a, 0x13, 0x41, 0x72, 0x74, 0x69, 0x66, 0x61, 0x63, 0x74, 0x46, 0x69, + 0x6c, 0x65, 0x50, 0x61, 0x79, 0x6c, 0x6f, 0x61, 0x64, 0x12, 0x12, 0x0a, 0x04, 0x70, 0x61, 0x74, + 0x68, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x04, 0x70, 0x61, 0x74, 0x68, 0x12, 0x16, 0x0a, + 0x06, 0x73, 0x68, 0x61, 0x32, 0x35, 0x36, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x06, 0x73, + 0x68, 0x61, 0x32, 0x35, 0x36, 0x22, 0x3e, 0x0a, 0x12, 0x41, 0x72, 0x74, 0x69, 0x66, 0x61, 0x63, + 0x74, 0x55, 0x72, 0x6c, 0x50, 0x61, 0x79, 0x6c, 0x6f, 0x61, 0x64, 0x12, 0x10, 0x0a, 0x03, 0x75, + 0x72, 0x6c, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x03, 0x75, 0x72, 0x6c, 0x12, 0x16, 0x0a, + 0x06, 0x73, 0x68, 0x61, 0x32, 0x35, 0x36, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x06, 0x73, + 0x68, 0x61, 0x32, 0x35, 0x36, 0x22, 0x29, 0x0a, 0x13, 0x45, 0x6d, 0x62, 0x65, 0x64, 0x64, 0x65, + 0x64, 0x46, 0x69, 0x6c, 0x65, 0x50, 0x61, 0x79, 0x6c, 0x6f, 0x61, 0x64, 0x12, 0x12, 0x0a, 0x04, + 0x64, 0x61, 0x74, 0x61, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0c, 0x52, 0x04, 0x64, 0x61, 0x74, 0x61, + 0x22, 0x48, 0x0a, 0x0b, 0x50, 0x79, 0x50, 0x49, 0x50, 0x61, 0x79, 0x6c, 0x6f, 0x61, 0x64, 0x12, + 0x1f, 0x0a, 0x0b, 0x61, 0x72, 0x74, 0x69, 0x66, 0x61, 0x63, 0x74, 0x5f, 0x69, 0x64, 0x18, 0x01, + 0x20, 0x01, 0x28, 0x09, 0x52, 0x0a, 0x61, 0x72, 0x74, 0x69, 0x66, 0x61, 0x63, 0x74, 0x49, 0x64, + 0x12, 0x18, 0x0a, 0x07, 0x76, 0x65, 0x72, 0x73, 0x69, 0x6f, 0x6e, 0x18, 0x02, 0x20, 0x01, 0x28, + 0x09, 0x52, 0x07, 0x76, 0x65, 0x72, 0x73, 0x69, 0x6f, 0x6e, 0x22, 0x51, 0x0a, 0x0c, 0x4d, 0x61, + 0x76, 0x65, 0x6e, 0x50, 0x61, 0x79, 0x6c, 0x6f, 0x61, 0x64, 0x12, 0x1a, 0x0a, 0x08, 0x61, 0x72, + 0x74, 0x69, 0x66, 0x61, 0x63, 0x74, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x08, 0x61, 0x72, + 0x74, 0x69, 0x66, 0x61, 0x63, 0x74, 0x12, 0x25, 0x0a, 0x0e, 0x72, 0x65, 0x70, 0x6f, 0x73, 0x69, + 0x74, 0x6f, 0x72, 0x79, 0x5f, 0x75, 0x72, 0x6c, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0d, + 0x72, 0x65, 0x70, 0x6f, 0x73, 0x69, 0x74, 0x6f, 0x72, 0x79, 0x55, 0x72, 0x6c, 0x22, 0x3f, 0x0a, + 0x17, 0x44, 0x65, 0x66, 0x65, 0x72, 0x72, 0x65, 0x64, 0x41, 0x72, 0x74, 0x69, 0x66, 0x61, 0x63, + 0x74, 0x50, 0x61, 0x79, 0x6c, 0x6f, 0x61, 0x64, 0x12, 0x10, 0x0a, 0x03, 0x6b, 0x65, 0x79, 0x18, + 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x03, 0x6b, 0x65, 0x79, 0x12, 0x12, 0x0a, 0x04, 0x64, 0x61, + 0x74, 0x61, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0c, 0x52, 0x04, 0x64, 0x61, 0x74, 0x61, 0x22, 0x3f, + 0x0a, 0x1c, 0x41, 0x72, 0x74, 0x69, 0x66, 0x61, 0x63, 0x74, 0x53, 0x74, 0x61, 0x67, 0x69, 0x6e, + 0x67, 0x54, 0x6f, 0x52, 0x6f, 0x6c, 0x65, 0x50, 0x61, 0x79, 0x6c, 0x6f, 0x61, 0x64, 0x12, 0x1f, + 0x0a, 0x0b, 0x73, 0x74, 0x61, 0x67, 0x65, 0x64, 0x5f, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x01, 0x20, + 0x01, 0x28, 0x09, 0x52, 0x0a, 0x73, 0x74, 0x61, 0x67, 0x65, 0x64, 0x4e, 0x61, 0x6d, 0x65, 0x22, + 0x91, 0x01, 0x0a, 0x13, 0x41, 0x72, 0x74, 0x69, 0x66, 0x61, 0x63, 0x74, 0x49, 0x6e, 0x66, 0x6f, + 0x72, 0x6d, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x19, 0x0a, 0x08, 0x74, 0x79, 0x70, 0x65, 0x5f, + 0x75, 0x72, 0x6e, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x07, 0x74, 0x79, 0x70, 0x65, 0x55, + 0x72, 0x6e, 0x12, 0x21, 0x0a, 0x0c, 0x74, 0x79, 0x70, 0x65, 0x5f, 0x70, 0x61, 0x79, 0x6c, 0x6f, + 0x61, 0x64, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0c, 0x52, 0x0b, 0x74, 0x79, 0x70, 0x65, 0x50, 0x61, + 0x79, 0x6c, 0x6f, 0x61, 0x64, 0x12, 0x19, 0x0a, 0x08, 0x72, 0x6f, 0x6c, 0x65, 0x5f, 0x75, 0x72, + 0x6e, 0x18, 0x03, 0x20, 0x01, 0x28, 0x09, 0x52, 0x07, 0x72, 0x6f, 0x6c, 0x65, 0x55, 0x72, 0x6e, + 0x12, 0x21, 0x0a, 0x0c, 0x72, 0x6f, 0x6c, 0x65, 0x5f, 0x70, 0x61, 0x79, 0x6c, 0x6f, 0x61, 0x64, + 0x18, 0x04, 0x20, 0x01, 0x28, 0x0c, 0x52, 0x0b, 0x72, 0x6f, 0x6c, 0x65, 0x50, 0x61, 0x79, 0x6c, + 0x6f, 0x61, 0x64, 0x22, 0xbe, 0x03, 0x0a, 0x0b, 0x45, 0x6e, 0x76, 0x69, 0x72, 0x6f, 0x6e, 0x6d, + 0x65, 0x6e, 0x74, 0x12, 0x10, 0x0a, 0x03, 0x75, 0x72, 0x6e, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, + 0x52, 0x03, 0x75, 0x72, 0x6e, 0x12, 0x18, 0x0a, 0x07, 0x70, 0x61, 0x79, 0x6c, 0x6f, 0x61, 0x64, + 0x18, 0x03, 0x20, 0x01, 0x28, 0x0c, 0x52, 0x07, 0x70, 0x61, 0x79, 0x6c, 0x6f, 0x61, 0x64, 0x12, + 0x51, 0x0a, 0x0c, 0x64, 0x69, 0x73, 0x70, 0x6c, 0x61, 0x79, 0x5f, 0x64, 0x61, 0x74, 0x61, 0x18, + 0x04, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x2e, 0x2e, 0x6f, 0x72, 0x67, 0x2e, 0x61, 0x70, 0x61, 0x63, + 0x68, 0x65, 0x2e, 0x62, 0x65, 0x61, 0x6d, 0x2e, 0x6d, 0x6f, 0x64, 0x65, 0x6c, 0x2e, 0x70, 0x69, + 0x70, 0x65, 0x6c, 0x69, 0x6e, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x44, 0x69, 0x73, 0x70, 0x6c, 0x61, + 0x79, 0x44, 0x61, 0x74, 0x61, 0x52, 0x0b, 0x64, 0x69, 0x73, 0x70, 0x6c, 0x61, 0x79, 0x44, 0x61, + 0x74, 0x61, 0x12, 0x22, 0x0a, 0x0c, 0x63, 0x61, 0x70, 0x61, 0x62, 0x69, 0x6c, 0x69, 0x74, 0x69, + 0x65, 0x73, 0x18, 0x05, 0x20, 0x03, 0x28, 0x09, 0x52, 0x0c, 0x63, 0x61, 0x70, 0x61, 0x62, 0x69, + 0x6c, 0x69, 0x74, 0x69, 0x65, 0x73, 0x12, 0x5a, 0x0a, 0x0c, 0x64, 0x65, 0x70, 0x65, 0x6e, 0x64, + 0x65, 0x6e, 0x63, 0x69, 0x65, 0x73, 0x18, 0x06, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x36, 0x2e, 0x6f, + 0x72, 0x67, 0x2e, 0x61, 0x70, 0x61, 0x63, 0x68, 0x65, 0x2e, 0x62, 0x65, 0x61, 0x6d, 0x2e, 0x6d, + 0x6f, 0x64, 0x65, 0x6c, 0x2e, 0x70, 0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e, 0x65, 0x2e, 0x76, 0x31, + 0x2e, 0x41, 0x72, 0x74, 0x69, 0x66, 0x61, 0x63, 0x74, 0x49, 0x6e, 0x66, 0x6f, 0x72, 0x6d, 0x61, + 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x0c, 0x64, 0x65, 0x70, 0x65, 0x6e, 0x64, 0x65, 0x6e, 0x63, 0x69, + 0x65, 0x73, 0x12, 0x68, 0x0a, 0x0e, 0x72, 0x65, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x5f, 0x68, + 0x69, 0x6e, 0x74, 0x73, 0x18, 0x07, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x41, 0x2e, 0x6f, 0x72, 0x67, + 0x2e, 0x61, 0x70, 0x61, 0x63, 0x68, 0x65, 0x2e, 0x62, 0x65, 0x61, 0x6d, 0x2e, 0x6d, 0x6f, 0x64, + 0x65, 0x6c, 0x2e, 0x70, 0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x45, + 0x6e, 0x76, 0x69, 0x72, 0x6f, 0x6e, 0x6d, 0x65, 0x6e, 0x74, 0x2e, 0x52, 0x65, 0x73, 0x6f, 0x75, + 0x72, 0x63, 0x65, 0x48, 0x69, 0x6e, 0x74, 0x73, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x52, 0x0d, 0x72, + 0x65, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x48, 0x69, 0x6e, 0x74, 0x73, 0x1a, 0x40, 0x0a, 0x12, + 0x52, 0x65, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x48, 0x69, 0x6e, 0x74, 0x73, 0x45, 0x6e, 0x74, + 0x72, 0x79, 0x12, 0x10, 0x0a, 0x03, 0x6b, 0x65, 0x79, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, + 0x03, 0x6b, 0x65, 0x79, 0x12, 0x14, 0x0a, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x18, 0x02, 0x20, + 0x01, 0x28, 0x0c, 0x52, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x3a, 0x02, 0x38, 0x01, 0x4a, 0x04, + 0x08, 0x01, 0x10, 0x02, 0x22, 0xeb, 0x01, 0x0a, 0x14, 0x53, 0x74, 0x61, 0x6e, 0x64, 0x61, 0x72, + 0x64, 0x45, 0x6e, 0x76, 0x69, 0x72, 0x6f, 0x6e, 0x6d, 0x65, 0x6e, 0x74, 0x73, 0x22, 0xd2, 0x01, + 0x0a, 0x0c, 0x45, 0x6e, 0x76, 0x69, 0x72, 0x6f, 0x6e, 0x6d, 0x65, 0x6e, 0x74, 0x73, 0x12, 0x24, + 0x0a, 0x06, 0x44, 0x4f, 0x43, 0x4b, 0x45, 0x52, 0x10, 0x00, 0x1a, 0x18, 0xa2, 0xb4, 0xfa, 0xc2, + 0x05, 0x12, 0x62, 0x65, 0x61, 0x6d, 0x3a, 0x65, 0x6e, 0x76, 0x3a, 0x64, 0x6f, 0x63, 0x6b, 0x65, + 0x72, 0x3a, 0x76, 0x31, 0x12, 0x26, 0x0a, 0x07, 0x50, 0x52, 0x4f, 0x43, 0x45, 0x53, 0x53, 0x10, + 0x01, 0x1a, 0x19, 0xa2, 0xb4, 0xfa, 0xc2, 0x05, 0x13, 0x62, 0x65, 0x61, 0x6d, 0x3a, 0x65, 0x6e, + 0x76, 0x3a, 0x70, 0x72, 0x6f, 0x63, 0x65, 0x73, 0x73, 0x3a, 0x76, 0x31, 0x12, 0x28, 0x0a, 0x08, + 0x45, 0x58, 0x54, 0x45, 0x52, 0x4e, 0x41, 0x4c, 0x10, 0x02, 0x1a, 0x1a, 0xa2, 0xb4, 0xfa, 0xc2, + 0x05, 0x14, 0x62, 0x65, 0x61, 0x6d, 0x3a, 0x65, 0x6e, 0x76, 0x3a, 0x65, 0x78, 0x74, 0x65, 0x72, + 0x6e, 0x61, 0x6c, 0x3a, 0x76, 0x31, 0x12, 0x26, 0x0a, 0x07, 0x44, 0x45, 0x46, 0x41, 0x55, 0x4c, + 0x54, 0x10, 0x03, 0x1a, 0x19, 0xa2, 0xb4, 0xfa, 0xc2, 0x05, 0x13, 0x62, 0x65, 0x61, 0x6d, 0x3a, + 0x65, 0x6e, 0x76, 0x3a, 0x64, 0x65, 0x66, 0x61, 0x75, 0x6c, 0x74, 0x3a, 0x76, 0x31, 0x12, 0x22, + 0x0a, 0x05, 0x41, 0x4e, 0x59, 0x4f, 0x46, 0x10, 0x04, 0x1a, 0x17, 0xa2, 0xb4, 0xfa, 0xc2, 0x05, + 0x11, 0x62, 0x65, 0x61, 0x6d, 0x3a, 0x65, 0x6e, 0x76, 0x3a, 0x61, 0x6e, 0x79, 0x6f, 0x66, 0x3a, + 0x76, 0x31, 0x22, 0x38, 0x0a, 0x0d, 0x44, 0x6f, 0x63, 0x6b, 0x65, 0x72, 0x50, 0x61, 0x79, 0x6c, + 0x6f, 0x61, 0x64, 0x12, 0x27, 0x0a, 0x0f, 0x63, 0x6f, 0x6e, 0x74, 0x61, 0x69, 0x6e, 0x65, 0x72, + 0x5f, 0x69, 0x6d, 0x61, 0x67, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0e, 0x63, 0x6f, + 0x6e, 0x74, 0x61, 0x69, 0x6e, 0x65, 0x72, 0x49, 0x6d, 0x61, 0x67, 0x65, 0x22, 0xd4, 0x01, 0x0a, + 0x0e, 0x50, 0x72, 0x6f, 0x63, 0x65, 0x73, 0x73, 0x50, 0x61, 0x79, 0x6c, 0x6f, 0x61, 0x64, 0x12, + 0x0e, 0x0a, 0x02, 0x6f, 0x73, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x02, 0x6f, 0x73, 0x12, + 0x12, 0x0a, 0x04, 0x61, 0x72, 0x63, 0x68, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x04, 0x61, + 0x72, 0x63, 0x68, 0x12, 0x18, 0x0a, 0x07, 0x63, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x18, 0x03, + 0x20, 0x01, 0x28, 0x09, 0x52, 0x07, 0x63, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x12, 0x4c, 0x0a, + 0x03, 0x65, 0x6e, 0x76, 0x18, 0x04, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x3a, 0x2e, 0x6f, 0x72, 0x67, + 0x2e, 0x61, 0x70, 0x61, 0x63, 0x68, 0x65, 0x2e, 0x62, 0x65, 0x61, 0x6d, 0x2e, 0x6d, 0x6f, 0x64, + 0x65, 0x6c, 0x2e, 0x70, 0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x50, + 0x72, 0x6f, 0x63, 0x65, 0x73, 0x73, 0x50, 0x61, 0x79, 0x6c, 0x6f, 0x61, 0x64, 0x2e, 0x45, 0x6e, + 0x76, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x52, 0x03, 0x65, 0x6e, 0x76, 0x1a, 0x36, 0x0a, 0x08, 0x45, + 0x6e, 0x76, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x12, 0x10, 0x0a, 0x03, 0x6b, 0x65, 0x79, 0x18, 0x01, + 0x20, 0x01, 0x28, 0x09, 0x52, 0x03, 0x6b, 0x65, 0x79, 0x12, 0x14, 0x0a, 0x05, 0x76, 0x61, 0x6c, + 0x75, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x3a, + 0x02, 0x38, 0x01, 0x22, 0xf9, 0x01, 0x0a, 0x0f, 0x45, 0x78, 0x74, 0x65, 0x72, 0x6e, 0x61, 0x6c, + 0x50, 0x61, 0x79, 0x6c, 0x6f, 0x61, 0x64, 0x12, 0x53, 0x0a, 0x08, 0x65, 0x6e, 0x64, 0x70, 0x6f, + 0x69, 0x6e, 0x74, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x37, 0x2e, 0x6f, 0x72, 0x67, 0x2e, + 0x61, 0x70, 0x61, 0x63, 0x68, 0x65, 0x2e, 0x62, 0x65, 0x61, 0x6d, 0x2e, 0x6d, 0x6f, 0x64, 0x65, + 0x6c, 0x2e, 0x70, 0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x41, 0x70, + 0x69, 0x53, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, 0x44, 0x65, 0x73, 0x63, 0x72, 0x69, 0x70, 0x74, + 0x6f, 0x72, 0x52, 0x08, 0x65, 0x6e, 0x64, 0x70, 0x6f, 0x69, 0x6e, 0x74, 0x12, 0x56, 0x0a, 0x06, + 0x70, 0x61, 0x72, 0x61, 0x6d, 0x73, 0x18, 0x02, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x3e, 0x2e, 0x6f, + 0x72, 0x67, 0x2e, 0x61, 0x70, 0x61, 0x63, 0x68, 0x65, 0x2e, 0x62, 0x65, 0x61, 0x6d, 0x2e, 0x6d, + 0x6f, 0x64, 0x65, 0x6c, 0x2e, 0x70, 0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e, 0x65, 0x2e, 0x76, 0x31, + 0x2e, 0x45, 0x78, 0x74, 0x65, 0x72, 0x6e, 0x61, 0x6c, 0x50, 0x61, 0x79, 0x6c, 0x6f, 0x61, 0x64, + 0x2e, 0x50, 0x61, 0x72, 0x61, 0x6d, 0x73, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x52, 0x06, 0x70, 0x61, + 0x72, 0x61, 0x6d, 0x73, 0x1a, 0x39, 0x0a, 0x0b, 0x50, 0x61, 0x72, 0x61, 0x6d, 0x73, 0x45, 0x6e, + 0x74, 0x72, 0x79, 0x12, 0x10, 0x0a, 0x03, 0x6b, 0x65, 0x79, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, + 0x52, 0x03, 0x6b, 0x65, 0x79, 0x12, 0x14, 0x0a, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x18, 0x02, + 0x20, 0x01, 0x28, 0x09, 0x52, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x3a, 0x02, 0x38, 0x01, 0x22, + 0x6d, 0x0a, 0x17, 0x41, 0x6e, 0x79, 0x4f, 0x66, 0x45, 0x6e, 0x76, 0x69, 0x72, 0x6f, 0x6e, 0x6d, + 0x65, 0x6e, 0x74, 0x50, 0x61, 0x79, 0x6c, 0x6f, 0x61, 0x64, 0x12, 0x52, 0x0a, 0x0c, 0x65, 0x6e, + 0x76, 0x69, 0x72, 0x6f, 0x6e, 0x6d, 0x65, 0x6e, 0x74, 0x73, 0x18, 0x01, 0x20, 0x03, 0x28, 0x0b, + 0x32, 0x2e, 0x2e, 0x6f, 0x72, 0x67, 0x2e, 0x61, 0x70, 0x61, 0x63, 0x68, 0x65, 0x2e, 0x62, 0x65, + 0x61, 0x6d, 0x2e, 0x6d, 0x6f, 0x64, 0x65, 0x6c, 0x2e, 0x70, 0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e, + 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x45, 0x6e, 0x76, 0x69, 0x72, 0x6f, 0x6e, 0x6d, 0x65, 0x6e, 0x74, + 0x52, 0x0c, 0x65, 0x6e, 0x76, 0x69, 0x72, 0x6f, 0x6e, 0x6d, 0x65, 0x6e, 0x74, 0x73, 0x22, 0xb2, + 0x06, 0x0a, 0x11, 0x53, 0x74, 0x61, 0x6e, 0x64, 0x61, 0x72, 0x64, 0x50, 0x72, 0x6f, 0x74, 0x6f, + 0x63, 0x6f, 0x6c, 0x73, 0x22, 0x9c, 0x06, 0x0a, 0x04, 0x45, 0x6e, 0x75, 0x6d, 0x12, 0x48, 0x0a, + 0x19, 0x4c, 0x45, 0x47, 0x41, 0x43, 0x59, 0x5f, 0x50, 0x52, 0x4f, 0x47, 0x52, 0x45, 0x53, 0x53, + 0x5f, 0x52, 0x45, 0x50, 0x4f, 0x52, 0x54, 0x49, 0x4e, 0x47, 0x10, 0x00, 0x1a, 0x29, 0xa2, 0xb4, + 0xfa, 0xc2, 0x05, 0x23, 0x62, 0x65, 0x61, 0x6d, 0x3a, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x63, 0x6f, + 0x6c, 0x3a, 0x70, 0x72, 0x6f, 0x67, 0x72, 0x65, 0x73, 0x73, 0x5f, 0x72, 0x65, 0x70, 0x6f, 0x72, + 0x74, 0x69, 0x6e, 0x67, 0x3a, 0x76, 0x30, 0x12, 0x41, 0x0a, 0x12, 0x50, 0x52, 0x4f, 0x47, 0x52, + 0x45, 0x53, 0x53, 0x5f, 0x52, 0x45, 0x50, 0x4f, 0x52, 0x54, 0x49, 0x4e, 0x47, 0x10, 0x01, 0x1a, + 0x29, 0xa2, 0xb4, 0xfa, 0xc2, 0x05, 0x23, 0x62, 0x65, 0x61, 0x6d, 0x3a, 0x70, 0x72, 0x6f, 0x74, + 0x6f, 0x63, 0x6f, 0x6c, 0x3a, 0x70, 0x72, 0x6f, 0x67, 0x72, 0x65, 0x73, 0x73, 0x5f, 0x72, 0x65, + 0x70, 0x6f, 0x72, 0x74, 0x69, 0x6e, 0x67, 0x3a, 0x76, 0x31, 0x12, 0x37, 0x0a, 0x0d, 0x57, 0x4f, + 0x52, 0x4b, 0x45, 0x52, 0x5f, 0x53, 0x54, 0x41, 0x54, 0x55, 0x53, 0x10, 0x02, 0x1a, 0x24, 0xa2, + 0xb4, 0xfa, 0xc2, 0x05, 0x1e, 0x62, 0x65, 0x61, 0x6d, 0x3a, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x63, + 0x6f, 0x6c, 0x3a, 0x77, 0x6f, 0x72, 0x6b, 0x65, 0x72, 0x5f, 0x73, 0x74, 0x61, 0x74, 0x75, 0x73, + 0x3a, 0x76, 0x31, 0x12, 0x55, 0x0a, 0x1c, 0x4d, 0x55, 0x4c, 0x54, 0x49, 0x5f, 0x43, 0x4f, 0x52, + 0x45, 0x5f, 0x42, 0x55, 0x4e, 0x44, 0x4c, 0x45, 0x5f, 0x50, 0x52, 0x4f, 0x43, 0x45, 0x53, 0x53, + 0x49, 0x4e, 0x47, 0x10, 0x03, 0x1a, 0x33, 0xa2, 0xb4, 0xfa, 0xc2, 0x05, 0x2d, 0x62, 0x65, 0x61, + 0x6d, 0x3a, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x63, 0x6f, 0x6c, 0x3a, 0x6d, 0x75, 0x6c, 0x74, 0x69, + 0x5f, 0x63, 0x6f, 0x72, 0x65, 0x5f, 0x62, 0x75, 0x6e, 0x64, 0x6c, 0x65, 0x5f, 0x70, 0x72, 0x6f, + 0x63, 0x65, 0x73, 0x73, 0x69, 0x6e, 0x67, 0x3a, 0x76, 0x31, 0x12, 0x3b, 0x0a, 0x0f, 0x53, 0x49, + 0x42, 0x4c, 0x49, 0x4e, 0x47, 0x5f, 0x57, 0x4f, 0x52, 0x4b, 0x45, 0x52, 0x53, 0x10, 0x05, 0x1a, + 0x26, 0xa2, 0xb4, 0xfa, 0xc2, 0x05, 0x20, 0x62, 0x65, 0x61, 0x6d, 0x3a, 0x70, 0x72, 0x6f, 0x74, + 0x6f, 0x63, 0x6f, 0x6c, 0x3a, 0x73, 0x69, 0x62, 0x6c, 0x69, 0x6e, 0x67, 0x5f, 0x77, 0x6f, 0x72, + 0x6b, 0x65, 0x72, 0x73, 0x3a, 0x76, 0x31, 0x12, 0x4d, 0x0a, 0x18, 0x48, 0x41, 0x52, 0x4e, 0x45, + 0x53, 0x53, 0x5f, 0x4d, 0x4f, 0x4e, 0x49, 0x54, 0x4f, 0x52, 0x49, 0x4e, 0x47, 0x5f, 0x49, 0x4e, + 0x46, 0x4f, 0x53, 0x10, 0x04, 0x1a, 0x2f, 0xa2, 0xb4, 0xfa, 0xc2, 0x05, 0x29, 0x62, 0x65, 0x61, + 0x6d, 0x3a, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x63, 0x6f, 0x6c, 0x3a, 0x68, 0x61, 0x72, 0x6e, 0x65, + 0x73, 0x73, 0x5f, 0x6d, 0x6f, 0x6e, 0x69, 0x74, 0x6f, 0x72, 0x69, 0x6e, 0x67, 0x5f, 0x69, 0x6e, + 0x66, 0x6f, 0x73, 0x3a, 0x76, 0x31, 0x12, 0x61, 0x0a, 0x22, 0x43, 0x4f, 0x4e, 0x54, 0x52, 0x4f, + 0x4c, 0x5f, 0x52, 0x45, 0x51, 0x55, 0x45, 0x53, 0x54, 0x5f, 0x45, 0x4c, 0x45, 0x4d, 0x45, 0x4e, + 0x54, 0x53, 0x5f, 0x45, 0x4d, 0x42, 0x45, 0x44, 0x44, 0x49, 0x4e, 0x47, 0x10, 0x06, 0x1a, 0x39, + 0xa2, 0xb4, 0xfa, 0xc2, 0x05, 0x33, 0x62, 0x65, 0x61, 0x6d, 0x3a, 0x70, 0x72, 0x6f, 0x74, 0x6f, + 0x63, 0x6f, 0x6c, 0x3a, 0x63, 0x6f, 0x6e, 0x74, 0x72, 0x6f, 0x6c, 0x5f, 0x72, 0x65, 0x71, 0x75, + 0x65, 0x73, 0x74, 0x5f, 0x65, 0x6c, 0x65, 0x6d, 0x65, 0x6e, 0x74, 0x73, 0x5f, 0x65, 0x6d, 0x62, + 0x65, 0x64, 0x64, 0x69, 0x6e, 0x67, 0x3a, 0x76, 0x31, 0x12, 0x37, 0x0a, 0x0d, 0x53, 0x54, 0x41, + 0x54, 0x45, 0x5f, 0x43, 0x41, 0x43, 0x48, 0x49, 0x4e, 0x47, 0x10, 0x07, 0x1a, 0x24, 0xa2, 0xb4, + 0xfa, 0xc2, 0x05, 0x1e, 0x62, 0x65, 0x61, 0x6d, 0x3a, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x63, 0x6f, + 0x6c, 0x3a, 0x73, 0x74, 0x61, 0x74, 0x65, 0x5f, 0x63, 0x61, 0x63, 0x68, 0x69, 0x6e, 0x67, 0x3a, + 0x76, 0x31, 0x12, 0x37, 0x0a, 0x0d, 0x44, 0x41, 0x54, 0x41, 0x5f, 0x53, 0x41, 0x4d, 0x50, 0x4c, + 0x49, 0x4e, 0x47, 0x10, 0x08, 0x1a, 0x24, 0xa2, 0xb4, 0xfa, 0xc2, 0x05, 0x1e, 0x62, 0x65, 0x61, + 0x6d, 0x3a, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x63, 0x6f, 0x6c, 0x3a, 0x64, 0x61, 0x74, 0x61, 0x5f, + 0x73, 0x61, 0x6d, 0x70, 0x6c, 0x69, 0x6e, 0x67, 0x3a, 0x76, 0x31, 0x12, 0x53, 0x0a, 0x1b, 0x53, + 0x44, 0x4b, 0x5f, 0x43, 0x4f, 0x4e, 0x53, 0x55, 0x4d, 0x49, 0x4e, 0x47, 0x5f, 0x52, 0x45, 0x43, + 0x45, 0x49, 0x56, 0x45, 0x44, 0x5f, 0x44, 0x41, 0x54, 0x41, 0x10, 0x09, 0x1a, 0x32, 0xa2, 0xb4, + 0xfa, 0xc2, 0x05, 0x2c, 0x62, 0x65, 0x61, 0x6d, 0x3a, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x63, 0x6f, + 0x6c, 0x3a, 0x73, 0x64, 0x6b, 0x5f, 0x63, 0x6f, 0x6e, 0x73, 0x75, 0x6d, 0x69, 0x6e, 0x67, 0x5f, + 0x72, 0x65, 0x63, 0x65, 0x69, 0x76, 0x65, 0x64, 0x5f, 0x64, 0x61, 0x74, 0x61, 0x3a, 0x76, 0x31, + 0x12, 0x41, 0x0a, 0x12, 0x4f, 0x52, 0x44, 0x45, 0x52, 0x45, 0x44, 0x5f, 0x4c, 0x49, 0x53, 0x54, + 0x5f, 0x53, 0x54, 0x41, 0x54, 0x45, 0x10, 0x0a, 0x1a, 0x29, 0xa2, 0xb4, 0xfa, 0xc2, 0x05, 0x23, + 0x62, 0x65, 0x61, 0x6d, 0x3a, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x63, 0x6f, 0x6c, 0x3a, 0x6f, 0x72, + 0x64, 0x65, 0x72, 0x65, 0x64, 0x5f, 0x6c, 0x69, 0x73, 0x74, 0x5f, 0x73, 0x74, 0x61, 0x74, 0x65, + 0x3a, 0x76, 0x31, 0x22, 0xb5, 0x02, 0x0a, 0x17, 0x53, 0x74, 0x61, 0x6e, 0x64, 0x61, 0x72, 0x64, + 0x52, 0x75, 0x6e, 0x6e, 0x65, 0x72, 0x50, 0x72, 0x6f, 0x74, 0x6f, 0x63, 0x6f, 0x6c, 0x73, 0x22, + 0x99, 0x02, 0x0a, 0x04, 0x45, 0x6e, 0x75, 0x6d, 0x12, 0x4f, 0x0a, 0x19, 0x4d, 0x4f, 0x4e, 0x49, + 0x54, 0x4f, 0x52, 0x49, 0x4e, 0x47, 0x5f, 0x49, 0x4e, 0x46, 0x4f, 0x5f, 0x53, 0x48, 0x4f, 0x52, + 0x54, 0x5f, 0x49, 0x44, 0x53, 0x10, 0x00, 0x1a, 0x30, 0xa2, 0xb4, 0xfa, 0xc2, 0x05, 0x2a, 0x62, + 0x65, 0x61, 0x6d, 0x3a, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x63, 0x6f, 0x6c, 0x3a, 0x6d, 0x6f, 0x6e, + 0x69, 0x74, 0x6f, 0x72, 0x69, 0x6e, 0x67, 0x5f, 0x69, 0x6e, 0x66, 0x6f, 0x5f, 0x73, 0x68, 0x6f, + 0x72, 0x74, 0x5f, 0x69, 0x64, 0x73, 0x3a, 0x76, 0x31, 0x12, 0x63, 0x0a, 0x23, 0x43, 0x4f, 0x4e, + 0x54, 0x52, 0x4f, 0x4c, 0x5f, 0x52, 0x45, 0x53, 0x50, 0x4f, 0x4e, 0x53, 0x45, 0x5f, 0x45, 0x4c, + 0x45, 0x4d, 0x45, 0x4e, 0x54, 0x53, 0x5f, 0x45, 0x4d, 0x42, 0x45, 0x44, 0x44, 0x49, 0x4e, 0x47, + 0x10, 0x06, 0x1a, 0x3a, 0xa2, 0xb4, 0xfa, 0xc2, 0x05, 0x34, 0x62, 0x65, 0x61, 0x6d, 0x3a, 0x70, + 0x72, 0x6f, 0x74, 0x6f, 0x63, 0x6f, 0x6c, 0x3a, 0x63, 0x6f, 0x6e, 0x74, 0x72, 0x6f, 0x6c, 0x5f, + 0x72, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x5f, 0x65, 0x6c, 0x65, 0x6d, 0x65, 0x6e, 0x74, + 0x73, 0x5f, 0x65, 0x6d, 0x62, 0x65, 0x64, 0x64, 0x69, 0x6e, 0x67, 0x3a, 0x76, 0x31, 0x12, 0x5b, + 0x0a, 0x1f, 0x4d, 0x55, 0x4c, 0x54, 0x49, 0x4d, 0x41, 0x50, 0x5f, 0x4b, 0x45, 0x59, 0x53, 0x5f, + 0x56, 0x41, 0x4c, 0x55, 0x45, 0x53, 0x5f, 0x53, 0x49, 0x44, 0x45, 0x5f, 0x49, 0x4e, 0x50, 0x55, + 0x54, 0x10, 0x07, 0x1a, 0x36, 0xa2, 0xb4, 0xfa, 0xc2, 0x05, 0x30, 0x62, 0x65, 0x61, 0x6d, 0x3a, + 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x63, 0x6f, 0x6c, 0x3a, 0x6d, 0x75, 0x6c, 0x74, 0x69, 0x6d, 0x61, + 0x70, 0x5f, 0x6b, 0x65, 0x79, 0x73, 0x5f, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x73, 0x5f, 0x73, 0x69, + 0x64, 0x65, 0x5f, 0x69, 0x6e, 0x70, 0x75, 0x74, 0x3a, 0x76, 0x31, 0x22, 0xff, 0x03, 0x0a, 0x14, + 0x53, 0x74, 0x61, 0x6e, 0x64, 0x61, 0x72, 0x64, 0x52, 0x65, 0x71, 0x75, 0x69, 0x72, 0x65, 0x6d, + 0x65, 0x6e, 0x74, 0x73, 0x22, 0xe6, 0x03, 0x0a, 0x04, 0x45, 0x6e, 0x75, 0x6d, 0x12, 0x4a, 0x0a, + 0x1c, 0x52, 0x45, 0x51, 0x55, 0x49, 0x52, 0x45, 0x53, 0x5f, 0x53, 0x54, 0x41, 0x54, 0x45, 0x46, + 0x55, 0x4c, 0x5f, 0x50, 0x52, 0x4f, 0x43, 0x45, 0x53, 0x53, 0x49, 0x4e, 0x47, 0x10, 0x00, 0x1a, + 0x28, 0xa2, 0xb4, 0xfa, 0xc2, 0x05, 0x22, 0x62, 0x65, 0x61, 0x6d, 0x3a, 0x72, 0x65, 0x71, 0x75, + 0x69, 0x72, 0x65, 0x6d, 0x65, 0x6e, 0x74, 0x3a, 0x70, 0x61, 0x72, 0x64, 0x6f, 0x3a, 0x73, 0x74, + 0x61, 0x74, 0x65, 0x66, 0x75, 0x6c, 0x3a, 0x76, 0x31, 0x12, 0x4e, 0x0a, 0x1c, 0x52, 0x45, 0x51, + 0x55, 0x49, 0x52, 0x45, 0x53, 0x5f, 0x42, 0x55, 0x4e, 0x44, 0x4c, 0x45, 0x5f, 0x46, 0x49, 0x4e, + 0x41, 0x4c, 0x49, 0x5a, 0x41, 0x54, 0x49, 0x4f, 0x4e, 0x10, 0x01, 0x1a, 0x2c, 0xa2, 0xb4, 0xfa, + 0xc2, 0x05, 0x26, 0x62, 0x65, 0x61, 0x6d, 0x3a, 0x72, 0x65, 0x71, 0x75, 0x69, 0x72, 0x65, 0x6d, + 0x65, 0x6e, 0x74, 0x3a, 0x70, 0x61, 0x72, 0x64, 0x6f, 0x3a, 0x66, 0x69, 0x6e, 0x61, 0x6c, 0x69, + 0x7a, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x3a, 0x76, 0x31, 0x12, 0x47, 0x0a, 0x15, 0x52, 0x45, 0x51, + 0x55, 0x49, 0x52, 0x45, 0x53, 0x5f, 0x53, 0x54, 0x41, 0x42, 0x4c, 0x45, 0x5f, 0x49, 0x4e, 0x50, + 0x55, 0x54, 0x10, 0x02, 0x1a, 0x2c, 0xa2, 0xb4, 0xfa, 0xc2, 0x05, 0x26, 0x62, 0x65, 0x61, 0x6d, + 0x3a, 0x72, 0x65, 0x71, 0x75, 0x69, 0x72, 0x65, 0x6d, 0x65, 0x6e, 0x74, 0x3a, 0x70, 0x61, 0x72, + 0x64, 0x6f, 0x3a, 0x73, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x5f, 0x69, 0x6e, 0x70, 0x75, 0x74, 0x3a, + 0x76, 0x31, 0x12, 0x51, 0x0a, 0x1a, 0x52, 0x45, 0x51, 0x55, 0x49, 0x52, 0x45, 0x53, 0x5f, 0x54, + 0x49, 0x4d, 0x45, 0x5f, 0x53, 0x4f, 0x52, 0x54, 0x45, 0x44, 0x5f, 0x49, 0x4e, 0x50, 0x55, 0x54, + 0x10, 0x03, 0x1a, 0x31, 0xa2, 0xb4, 0xfa, 0xc2, 0x05, 0x2b, 0x62, 0x65, 0x61, 0x6d, 0x3a, 0x72, + 0x65, 0x71, 0x75, 0x69, 0x72, 0x65, 0x6d, 0x65, 0x6e, 0x74, 0x3a, 0x70, 0x61, 0x72, 0x64, 0x6f, + 0x3a, 0x74, 0x69, 0x6d, 0x65, 0x5f, 0x73, 0x6f, 0x72, 0x74, 0x65, 0x64, 0x5f, 0x69, 0x6e, 0x70, + 0x75, 0x74, 0x3a, 0x76, 0x31, 0x12, 0x4d, 0x0a, 0x18, 0x52, 0x45, 0x51, 0x55, 0x49, 0x52, 0x45, + 0x53, 0x5f, 0x53, 0x50, 0x4c, 0x49, 0x54, 0x54, 0x41, 0x42, 0x4c, 0x45, 0x5f, 0x44, 0x4f, 0x46, + 0x4e, 0x10, 0x04, 0x1a, 0x2f, 0xa2, 0xb4, 0xfa, 0xc2, 0x05, 0x29, 0x62, 0x65, 0x61, 0x6d, 0x3a, + 0x72, 0x65, 0x71, 0x75, 0x69, 0x72, 0x65, 0x6d, 0x65, 0x6e, 0x74, 0x3a, 0x70, 0x61, 0x72, 0x64, + 0x6f, 0x3a, 0x73, 0x70, 0x6c, 0x69, 0x74, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x5f, 0x64, 0x6f, 0x66, + 0x6e, 0x3a, 0x76, 0x31, 0x12, 0x57, 0x0a, 0x1d, 0x52, 0x45, 0x51, 0x55, 0x49, 0x52, 0x45, 0x53, + 0x5f, 0x4f, 0x4e, 0x5f, 0x57, 0x49, 0x4e, 0x44, 0x4f, 0x57, 0x5f, 0x45, 0x58, 0x50, 0x49, 0x52, + 0x41, 0x54, 0x49, 0x4f, 0x4e, 0x10, 0x05, 0x1a, 0x34, 0xa2, 0xb4, 0xfa, 0xc2, 0x05, 0x2e, 0x62, + 0x65, 0x61, 0x6d, 0x3a, 0x72, 0x65, 0x71, 0x75, 0x69, 0x72, 0x65, 0x6d, 0x65, 0x6e, 0x74, 0x3a, + 0x70, 0x61, 0x72, 0x64, 0x6f, 0x3a, 0x6f, 0x6e, 0x5f, 0x77, 0x69, 0x6e, 0x64, 0x6f, 0x77, 0x5f, + 0x65, 0x78, 0x70, 0x69, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x3a, 0x76, 0x31, 0x22, 0x3a, 0x0a, + 0x0c, 0x46, 0x75, 0x6e, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x53, 0x70, 0x65, 0x63, 0x12, 0x10, 0x0a, + 0x03, 0x75, 0x72, 0x6e, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x03, 0x75, 0x72, 0x6e, 0x12, + 0x18, 0x0a, 0x07, 0x70, 0x61, 0x79, 0x6c, 0x6f, 0x61, 0x64, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0c, + 0x52, 0x07, 0x70, 0x61, 0x79, 0x6c, 0x6f, 0x61, 0x64, 0x22, 0x57, 0x0a, 0x13, 0x53, 0x74, 0x61, + 0x6e, 0x64, 0x61, 0x72, 0x64, 0x44, 0x69, 0x73, 0x70, 0x6c, 0x61, 0x79, 0x44, 0x61, 0x74, 0x61, + 0x22, 0x40, 0x0a, 0x0b, 0x44, 0x69, 0x73, 0x70, 0x6c, 0x61, 0x79, 0x44, 0x61, 0x74, 0x61, 0x12, + 0x31, 0x0a, 0x08, 0x4c, 0x41, 0x42, 0x45, 0x4c, 0x4c, 0x45, 0x44, 0x10, 0x00, 0x1a, 0x23, 0xa2, + 0xb4, 0xfa, 0xc2, 0x05, 0x1d, 0x62, 0x65, 0x61, 0x6d, 0x3a, 0x64, 0x69, 0x73, 0x70, 0x6c, 0x61, + 0x79, 0x5f, 0x64, 0x61, 0x74, 0x61, 0x3a, 0x6c, 0x61, 0x62, 0x65, 0x6c, 0x6c, 0x65, 0x64, 0x3a, + 0x76, 0x31, 0x22, 0xea, 0x01, 0x0a, 0x0f, 0x4c, 0x61, 0x62, 0x65, 0x6c, 0x6c, 0x65, 0x64, 0x50, + 0x61, 0x79, 0x6c, 0x6f, 0x61, 0x64, 0x12, 0x14, 0x0a, 0x05, 0x6c, 0x61, 0x62, 0x65, 0x6c, 0x18, + 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x05, 0x6c, 0x61, 0x62, 0x65, 0x6c, 0x12, 0x23, 0x0a, 0x0c, + 0x73, 0x74, 0x72, 0x69, 0x6e, 0x67, 0x5f, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x18, 0x02, 0x20, 0x01, + 0x28, 0x09, 0x48, 0x00, 0x52, 0x0b, 0x73, 0x74, 0x72, 0x69, 0x6e, 0x67, 0x56, 0x61, 0x6c, 0x75, + 0x65, 0x12, 0x1f, 0x0a, 0x0a, 0x62, 0x6f, 0x6f, 0x6c, 0x5f, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x18, + 0x03, 0x20, 0x01, 0x28, 0x08, 0x48, 0x00, 0x52, 0x09, 0x62, 0x6f, 0x6f, 0x6c, 0x56, 0x61, 0x6c, + 0x75, 0x65, 0x12, 0x23, 0x0a, 0x0c, 0x64, 0x6f, 0x75, 0x62, 0x6c, 0x65, 0x5f, 0x76, 0x61, 0x6c, + 0x75, 0x65, 0x18, 0x04, 0x20, 0x01, 0x28, 0x01, 0x48, 0x00, 0x52, 0x0b, 0x64, 0x6f, 0x75, 0x62, + 0x6c, 0x65, 0x56, 0x61, 0x6c, 0x75, 0x65, 0x12, 0x1d, 0x0a, 0x09, 0x69, 0x6e, 0x74, 0x5f, 0x76, + 0x61, 0x6c, 0x75, 0x65, 0x18, 0x05, 0x20, 0x01, 0x28, 0x03, 0x48, 0x00, 0x52, 0x08, 0x69, 0x6e, + 0x74, 0x56, 0x61, 0x6c, 0x75, 0x65, 0x12, 0x10, 0x0a, 0x03, 0x6b, 0x65, 0x79, 0x18, 0x06, 0x20, + 0x01, 0x28, 0x09, 0x52, 0x03, 0x6b, 0x65, 0x79, 0x12, 0x1c, 0x0a, 0x09, 0x6e, 0x61, 0x6d, 0x65, + 0x73, 0x70, 0x61, 0x63, 0x65, 0x18, 0x07, 0x20, 0x01, 0x28, 0x09, 0x52, 0x09, 0x6e, 0x61, 0x6d, + 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, 0x42, 0x07, 0x0a, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x22, + 0x39, 0x0a, 0x0b, 0x44, 0x69, 0x73, 0x70, 0x6c, 0x61, 0x79, 0x44, 0x61, 0x74, 0x61, 0x12, 0x10, + 0x0a, 0x03, 0x75, 0x72, 0x6e, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x03, 0x75, 0x72, 0x6e, + 0x12, 0x18, 0x0a, 0x07, 0x70, 0x61, 0x79, 0x6c, 0x6f, 0x61, 0x64, 0x18, 0x02, 0x20, 0x01, 0x28, + 0x0c, 0x52, 0x07, 0x70, 0x61, 0x79, 0x6c, 0x6f, 0x61, 0x64, 0x22, 0xd7, 0x07, 0x0a, 0x15, 0x4d, + 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x57, 0x69, 0x74, 0x68, 0x43, 0x6f, 0x6d, 0x70, 0x6f, 0x6e, + 0x65, 0x6e, 0x74, 0x73, 0x12, 0x4d, 0x0a, 0x0a, 0x63, 0x6f, 0x6d, 0x70, 0x6f, 0x6e, 0x65, 0x6e, + 0x74, 0x73, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x2d, 0x2e, 0x6f, 0x72, 0x67, 0x2e, 0x61, + 0x70, 0x61, 0x63, 0x68, 0x65, 0x2e, 0x62, 0x65, 0x61, 0x6d, 0x2e, 0x6d, 0x6f, 0x64, 0x65, 0x6c, + 0x2e, 0x70, 0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x43, 0x6f, 0x6d, + 0x70, 0x6f, 0x6e, 0x65, 0x6e, 0x74, 0x73, 0x52, 0x0a, 0x63, 0x6f, 0x6d, 0x70, 0x6f, 0x6e, 0x65, + 0x6e, 0x74, 0x73, 0x12, 0x40, 0x0a, 0x05, 0x63, 0x6f, 0x64, 0x65, 0x72, 0x18, 0x02, 0x20, 0x01, + 0x28, 0x0b, 0x32, 0x28, 0x2e, 0x6f, 0x72, 0x67, 0x2e, 0x61, 0x70, 0x61, 0x63, 0x68, 0x65, 0x2e, + 0x62, 0x65, 0x61, 0x6d, 0x2e, 0x6d, 0x6f, 0x64, 0x65, 0x6c, 0x2e, 0x70, 0x69, 0x70, 0x65, 0x6c, + 0x69, 0x6e, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x43, 0x6f, 0x64, 0x65, 0x72, 0x48, 0x00, 0x52, 0x05, + 0x63, 0x6f, 0x64, 0x65, 0x72, 0x12, 0x5c, 0x0a, 0x0f, 0x63, 0x6f, 0x6d, 0x62, 0x69, 0x6e, 0x65, + 0x5f, 0x70, 0x61, 0x79, 0x6c, 0x6f, 0x61, 0x64, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x31, + 0x2e, 0x6f, 0x72, 0x67, 0x2e, 0x61, 0x70, 0x61, 0x63, 0x68, 0x65, 0x2e, 0x62, 0x65, 0x61, 0x6d, + 0x2e, 0x6d, 0x6f, 0x64, 0x65, 0x6c, 0x2e, 0x70, 0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e, 0x65, 0x2e, + 0x76, 0x31, 0x2e, 0x43, 0x6f, 0x6d, 0x62, 0x69, 0x6e, 0x65, 0x50, 0x61, 0x79, 0x6c, 0x6f, 0x61, + 0x64, 0x48, 0x00, 0x52, 0x0e, 0x63, 0x6f, 0x6d, 0x62, 0x69, 0x6e, 0x65, 0x50, 0x61, 0x79, 0x6c, + 0x6f, 0x61, 0x64, 0x12, 0x56, 0x0a, 0x0d, 0x66, 0x75, 0x6e, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x5f, + 0x73, 0x70, 0x65, 0x63, 0x18, 0x04, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x2f, 0x2e, 0x6f, 0x72, 0x67, + 0x2e, 0x61, 0x70, 0x61, 0x63, 0x68, 0x65, 0x2e, 0x62, 0x65, 0x61, 0x6d, 0x2e, 0x6d, 0x6f, 0x64, + 0x65, 0x6c, 0x2e, 0x70, 0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x46, + 0x75, 0x6e, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x53, 0x70, 0x65, 0x63, 0x48, 0x00, 0x52, 0x0c, 0x66, + 0x75, 0x6e, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x53, 0x70, 0x65, 0x63, 0x12, 0x57, 0x0a, 0x0e, 0x70, + 0x61, 0x72, 0x5f, 0x64, 0x6f, 0x5f, 0x70, 0x61, 0x79, 0x6c, 0x6f, 0x61, 0x64, 0x18, 0x06, 0x20, + 0x01, 0x28, 0x0b, 0x32, 0x2f, 0x2e, 0x6f, 0x72, 0x67, 0x2e, 0x61, 0x70, 0x61, 0x63, 0x68, 0x65, + 0x2e, 0x62, 0x65, 0x61, 0x6d, 0x2e, 0x6d, 0x6f, 0x64, 0x65, 0x6c, 0x2e, 0x70, 0x69, 0x70, 0x65, + 0x6c, 0x69, 0x6e, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x50, 0x61, 0x72, 0x44, 0x6f, 0x50, 0x61, 0x79, + 0x6c, 0x6f, 0x61, 0x64, 0x48, 0x00, 0x52, 0x0c, 0x70, 0x61, 0x72, 0x44, 0x6f, 0x50, 0x61, 0x79, + 0x6c, 0x6f, 0x61, 0x64, 0x12, 0x4f, 0x0a, 0x0a, 0x70, 0x74, 0x72, 0x61, 0x6e, 0x73, 0x66, 0x6f, + 0x72, 0x6d, 0x18, 0x07, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x2d, 0x2e, 0x6f, 0x72, 0x67, 0x2e, 0x61, + 0x70, 0x61, 0x63, 0x68, 0x65, 0x2e, 0x62, 0x65, 0x61, 0x6d, 0x2e, 0x6d, 0x6f, 0x64, 0x65, 0x6c, + 0x2e, 0x70, 0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x50, 0x54, 0x72, + 0x61, 0x6e, 0x73, 0x66, 0x6f, 0x72, 0x6d, 0x48, 0x00, 0x52, 0x0a, 0x70, 0x74, 0x72, 0x61, 0x6e, + 0x73, 0x66, 0x6f, 0x72, 0x6d, 0x12, 0x52, 0x0a, 0x0b, 0x70, 0x63, 0x6f, 0x6c, 0x6c, 0x65, 0x63, + 0x74, 0x69, 0x6f, 0x6e, 0x18, 0x08, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x2e, 0x2e, 0x6f, 0x72, 0x67, + 0x2e, 0x61, 0x70, 0x61, 0x63, 0x68, 0x65, 0x2e, 0x62, 0x65, 0x61, 0x6d, 0x2e, 0x6d, 0x6f, 0x64, + 0x65, 0x6c, 0x2e, 0x70, 0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x50, + 0x43, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x48, 0x00, 0x52, 0x0b, 0x70, 0x63, + 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x53, 0x0a, 0x0c, 0x72, 0x65, 0x61, + 0x64, 0x5f, 0x70, 0x61, 0x79, 0x6c, 0x6f, 0x61, 0x64, 0x18, 0x09, 0x20, 0x01, 0x28, 0x0b, 0x32, + 0x2e, 0x2e, 0x6f, 0x72, 0x67, 0x2e, 0x61, 0x70, 0x61, 0x63, 0x68, 0x65, 0x2e, 0x62, 0x65, 0x61, + 0x6d, 0x2e, 0x6d, 0x6f, 0x64, 0x65, 0x6c, 0x2e, 0x70, 0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e, 0x65, + 0x2e, 0x76, 0x31, 0x2e, 0x52, 0x65, 0x61, 0x64, 0x50, 0x61, 0x79, 0x6c, 0x6f, 0x61, 0x64, 0x48, + 0x00, 0x52, 0x0b, 0x72, 0x65, 0x61, 0x64, 0x50, 0x61, 0x79, 0x6c, 0x6f, 0x61, 0x64, 0x12, 0x4d, + 0x0a, 0x0a, 0x73, 0x69, 0x64, 0x65, 0x5f, 0x69, 0x6e, 0x70, 0x75, 0x74, 0x18, 0x0b, 0x20, 0x01, + 0x28, 0x0b, 0x32, 0x2c, 0x2e, 0x6f, 0x72, 0x67, 0x2e, 0x61, 0x70, 0x61, 0x63, 0x68, 0x65, 0x2e, + 0x62, 0x65, 0x61, 0x6d, 0x2e, 0x6d, 0x6f, 0x64, 0x65, 0x6c, 0x2e, 0x70, 0x69, 0x70, 0x65, 0x6c, + 0x69, 0x6e, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x53, 0x69, 0x64, 0x65, 0x49, 0x6e, 0x70, 0x75, 0x74, + 0x48, 0x00, 0x52, 0x09, 0x73, 0x69, 0x64, 0x65, 0x49, 0x6e, 0x70, 0x75, 0x74, 0x12, 0x66, 0x0a, + 0x13, 0x77, 0x69, 0x6e, 0x64, 0x6f, 0x77, 0x5f, 0x69, 0x6e, 0x74, 0x6f, 0x5f, 0x70, 0x61, 0x79, + 0x6c, 0x6f, 0x61, 0x64, 0x18, 0x0c, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x34, 0x2e, 0x6f, 0x72, 0x67, + 0x2e, 0x61, 0x70, 0x61, 0x63, 0x68, 0x65, 0x2e, 0x62, 0x65, 0x61, 0x6d, 0x2e, 0x6d, 0x6f, 0x64, + 0x65, 0x6c, 0x2e, 0x70, 0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x57, + 0x69, 0x6e, 0x64, 0x6f, 0x77, 0x49, 0x6e, 0x74, 0x6f, 0x50, 0x61, 0x79, 0x6c, 0x6f, 0x61, 0x64, + 0x48, 0x00, 0x52, 0x11, 0x77, 0x69, 0x6e, 0x64, 0x6f, 0x77, 0x49, 0x6e, 0x74, 0x6f, 0x50, 0x61, + 0x79, 0x6c, 0x6f, 0x61, 0x64, 0x12, 0x65, 0x0a, 0x12, 0x77, 0x69, 0x6e, 0x64, 0x6f, 0x77, 0x69, + 0x6e, 0x67, 0x5f, 0x73, 0x74, 0x72, 0x61, 0x74, 0x65, 0x67, 0x79, 0x18, 0x0d, 0x20, 0x01, 0x28, + 0x0b, 0x32, 0x34, 0x2e, 0x6f, 0x72, 0x67, 0x2e, 0x61, 0x70, 0x61, 0x63, 0x68, 0x65, 0x2e, 0x62, + 0x65, 0x61, 0x6d, 0x2e, 0x6d, 0x6f, 0x64, 0x65, 0x6c, 0x2e, 0x70, 0x69, 0x70, 0x65, 0x6c, 0x69, + 0x6e, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x57, 0x69, 0x6e, 0x64, 0x6f, 0x77, 0x69, 0x6e, 0x67, 0x53, + 0x74, 0x72, 0x61, 0x74, 0x65, 0x67, 0x79, 0x48, 0x00, 0x52, 0x11, 0x77, 0x69, 0x6e, 0x64, 0x6f, + 0x77, 0x69, 0x6e, 0x67, 0x53, 0x74, 0x72, 0x61, 0x74, 0x65, 0x67, 0x79, 0x42, 0x06, 0x0a, 0x04, + 0x72, 0x6f, 0x6f, 0x74, 0x22, 0xb6, 0x0a, 0x0a, 0x16, 0x45, 0x78, 0x65, 0x63, 0x75, 0x74, 0x61, + 0x62, 0x6c, 0x65, 0x53, 0x74, 0x61, 0x67, 0x65, 0x50, 0x61, 0x79, 0x6c, 0x6f, 0x61, 0x64, 0x12, + 0x50, 0x0a, 0x0b, 0x65, 0x6e, 0x76, 0x69, 0x72, 0x6f, 0x6e, 0x6d, 0x65, 0x6e, 0x74, 0x18, 0x01, + 0x20, 0x01, 0x28, 0x0b, 0x32, 0x2e, 0x2e, 0x6f, 0x72, 0x67, 0x2e, 0x61, 0x70, 0x61, 0x63, 0x68, + 0x65, 0x2e, 0x62, 0x65, 0x61, 0x6d, 0x2e, 0x6d, 0x6f, 0x64, 0x65, 0x6c, 0x2e, 0x70, 0x69, 0x70, + 0x65, 0x6c, 0x69, 0x6e, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x45, 0x6e, 0x76, 0x69, 0x72, 0x6f, 0x6e, + 0x6d, 0x65, 0x6e, 0x74, 0x52, 0x0b, 0x65, 0x6e, 0x76, 0x69, 0x72, 0x6f, 0x6e, 0x6d, 0x65, 0x6e, + 0x74, 0x12, 0x7a, 0x0a, 0x13, 0x77, 0x69, 0x72, 0x65, 0x5f, 0x63, 0x6f, 0x64, 0x65, 0x72, 0x5f, + 0x73, 0x65, 0x74, 0x74, 0x69, 0x6e, 0x67, 0x73, 0x18, 0x09, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x4a, + 0x2e, 0x6f, 0x72, 0x67, 0x2e, 0x61, 0x70, 0x61, 0x63, 0x68, 0x65, 0x2e, 0x62, 0x65, 0x61, 0x6d, + 0x2e, 0x6d, 0x6f, 0x64, 0x65, 0x6c, 0x2e, 0x70, 0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e, 0x65, 0x2e, + 0x76, 0x31, 0x2e, 0x45, 0x78, 0x65, 0x63, 0x75, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x53, 0x74, 0x61, + 0x67, 0x65, 0x50, 0x61, 0x79, 0x6c, 0x6f, 0x61, 0x64, 0x2e, 0x57, 0x69, 0x72, 0x65, 0x43, 0x6f, + 0x64, 0x65, 0x72, 0x53, 0x65, 0x74, 0x74, 0x69, 0x6e, 0x67, 0x52, 0x11, 0x77, 0x69, 0x72, 0x65, + 0x43, 0x6f, 0x64, 0x65, 0x72, 0x53, 0x65, 0x74, 0x74, 0x69, 0x6e, 0x67, 0x73, 0x12, 0x14, 0x0a, + 0x05, 0x69, 0x6e, 0x70, 0x75, 0x74, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x05, 0x69, 0x6e, + 0x70, 0x75, 0x74, 0x12, 0x66, 0x0a, 0x0b, 0x73, 0x69, 0x64, 0x65, 0x5f, 0x69, 0x6e, 0x70, 0x75, + 0x74, 0x73, 0x18, 0x03, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x45, 0x2e, 0x6f, 0x72, 0x67, 0x2e, 0x61, + 0x70, 0x61, 0x63, 0x68, 0x65, 0x2e, 0x62, 0x65, 0x61, 0x6d, 0x2e, 0x6d, 0x6f, 0x64, 0x65, 0x6c, + 0x2e, 0x70, 0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x45, 0x78, 0x65, + 0x63, 0x75, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x53, 0x74, 0x61, 0x67, 0x65, 0x50, 0x61, 0x79, 0x6c, + 0x6f, 0x61, 0x64, 0x2e, 0x53, 0x69, 0x64, 0x65, 0x49, 0x6e, 0x70, 0x75, 0x74, 0x49, 0x64, 0x52, + 0x0a, 0x73, 0x69, 0x64, 0x65, 0x49, 0x6e, 0x70, 0x75, 0x74, 0x73, 0x12, 0x1e, 0x0a, 0x0a, 0x74, + 0x72, 0x61, 0x6e, 0x73, 0x66, 0x6f, 0x72, 0x6d, 0x73, 0x18, 0x04, 0x20, 0x03, 0x28, 0x09, 0x52, + 0x0a, 0x74, 0x72, 0x61, 0x6e, 0x73, 0x66, 0x6f, 0x72, 0x6d, 0x73, 0x12, 0x18, 0x0a, 0x07, 0x6f, + 0x75, 0x74, 0x70, 0x75, 0x74, 0x73, 0x18, 0x05, 0x20, 0x03, 0x28, 0x09, 0x52, 0x07, 0x6f, 0x75, + 0x74, 0x70, 0x75, 0x74, 0x73, 0x12, 0x4d, 0x0a, 0x0a, 0x63, 0x6f, 0x6d, 0x70, 0x6f, 0x6e, 0x65, + 0x6e, 0x74, 0x73, 0x18, 0x06, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x2d, 0x2e, 0x6f, 0x72, 0x67, 0x2e, + 0x61, 0x70, 0x61, 0x63, 0x68, 0x65, 0x2e, 0x62, 0x65, 0x61, 0x6d, 0x2e, 0x6d, 0x6f, 0x64, 0x65, + 0x6c, 0x2e, 0x70, 0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x43, 0x6f, + 0x6d, 0x70, 0x6f, 0x6e, 0x65, 0x6e, 0x74, 0x73, 0x52, 0x0a, 0x63, 0x6f, 0x6d, 0x70, 0x6f, 0x6e, + 0x65, 0x6e, 0x74, 0x73, 0x12, 0x66, 0x0a, 0x0b, 0x75, 0x73, 0x65, 0x72, 0x5f, 0x73, 0x74, 0x61, + 0x74, 0x65, 0x73, 0x18, 0x07, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x45, 0x2e, 0x6f, 0x72, 0x67, 0x2e, + 0x61, 0x70, 0x61, 0x63, 0x68, 0x65, 0x2e, 0x62, 0x65, 0x61, 0x6d, 0x2e, 0x6d, 0x6f, 0x64, 0x65, + 0x6c, 0x2e, 0x70, 0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x45, 0x78, + 0x65, 0x63, 0x75, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x53, 0x74, 0x61, 0x67, 0x65, 0x50, 0x61, 0x79, + 0x6c, 0x6f, 0x61, 0x64, 0x2e, 0x55, 0x73, 0x65, 0x72, 0x53, 0x74, 0x61, 0x74, 0x65, 0x49, 0x64, + 0x52, 0x0a, 0x75, 0x73, 0x65, 0x72, 0x53, 0x74, 0x61, 0x74, 0x65, 0x73, 0x12, 0x59, 0x0a, 0x06, + 0x74, 0x69, 0x6d, 0x65, 0x72, 0x73, 0x18, 0x08, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x41, 0x2e, 0x6f, + 0x72, 0x67, 0x2e, 0x61, 0x70, 0x61, 0x63, 0x68, 0x65, 0x2e, 0x62, 0x65, 0x61, 0x6d, 0x2e, 0x6d, + 0x6f, 0x64, 0x65, 0x6c, 0x2e, 0x70, 0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e, 0x65, 0x2e, 0x76, 0x31, + 0x2e, 0x45, 0x78, 0x65, 0x63, 0x75, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x53, 0x74, 0x61, 0x67, 0x65, + 0x50, 0x61, 0x79, 0x6c, 0x6f, 0x61, 0x64, 0x2e, 0x54, 0x69, 0x6d, 0x65, 0x72, 0x49, 0x64, 0x52, + 0x06, 0x74, 0x69, 0x6d, 0x65, 0x72, 0x73, 0x12, 0x6d, 0x0a, 0x0d, 0x74, 0x69, 0x6d, 0x65, 0x72, + 0x46, 0x61, 0x6d, 0x69, 0x6c, 0x69, 0x65, 0x73, 0x18, 0x0a, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x47, + 0x2e, 0x6f, 0x72, 0x67, 0x2e, 0x61, 0x70, 0x61, 0x63, 0x68, 0x65, 0x2e, 0x62, 0x65, 0x61, 0x6d, + 0x2e, 0x6d, 0x6f, 0x64, 0x65, 0x6c, 0x2e, 0x70, 0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e, 0x65, 0x2e, + 0x76, 0x31, 0x2e, 0x45, 0x78, 0x65, 0x63, 0x75, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x53, 0x74, 0x61, + 0x67, 0x65, 0x50, 0x61, 0x79, 0x6c, 0x6f, 0x61, 0x64, 0x2e, 0x54, 0x69, 0x6d, 0x65, 0x72, 0x46, + 0x61, 0x6d, 0x69, 0x6c, 0x79, 0x49, 0x64, 0x52, 0x0d, 0x74, 0x69, 0x6d, 0x65, 0x72, 0x46, 0x61, + 0x6d, 0x69, 0x6c, 0x69, 0x65, 0x73, 0x1a, 0x4f, 0x0a, 0x0b, 0x53, 0x69, 0x64, 0x65, 0x49, 0x6e, + 0x70, 0x75, 0x74, 0x49, 0x64, 0x12, 0x21, 0x0a, 0x0c, 0x74, 0x72, 0x61, 0x6e, 0x73, 0x66, 0x6f, + 0x72, 0x6d, 0x5f, 0x69, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0b, 0x74, 0x72, 0x61, + 0x6e, 0x73, 0x66, 0x6f, 0x72, 0x6d, 0x49, 0x64, 0x12, 0x1d, 0x0a, 0x0a, 0x6c, 0x6f, 0x63, 0x61, + 0x6c, 0x5f, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x09, 0x6c, 0x6f, + 0x63, 0x61, 0x6c, 0x4e, 0x61, 0x6d, 0x65, 0x1a, 0x4f, 0x0a, 0x0b, 0x55, 0x73, 0x65, 0x72, 0x53, + 0x74, 0x61, 0x74, 0x65, 0x49, 0x64, 0x12, 0x21, 0x0a, 0x0c, 0x74, 0x72, 0x61, 0x6e, 0x73, 0x66, + 0x6f, 0x72, 0x6d, 0x5f, 0x69, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0b, 0x74, 0x72, + 0x61, 0x6e, 0x73, 0x66, 0x6f, 0x72, 0x6d, 0x49, 0x64, 0x12, 0x1d, 0x0a, 0x0a, 0x6c, 0x6f, 0x63, + 0x61, 0x6c, 0x5f, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x09, 0x6c, + 0x6f, 0x63, 0x61, 0x6c, 0x4e, 0x61, 0x6d, 0x65, 0x1a, 0x4b, 0x0a, 0x07, 0x54, 0x69, 0x6d, 0x65, + 0x72, 0x49, 0x64, 0x12, 0x21, 0x0a, 0x0c, 0x74, 0x72, 0x61, 0x6e, 0x73, 0x66, 0x6f, 0x72, 0x6d, + 0x5f, 0x69, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0b, 0x74, 0x72, 0x61, 0x6e, 0x73, + 0x66, 0x6f, 0x72, 0x6d, 0x49, 0x64, 0x12, 0x1d, 0x0a, 0x0a, 0x6c, 0x6f, 0x63, 0x61, 0x6c, 0x5f, + 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x09, 0x6c, 0x6f, 0x63, 0x61, + 0x6c, 0x4e, 0x61, 0x6d, 0x65, 0x1a, 0x51, 0x0a, 0x0d, 0x54, 0x69, 0x6d, 0x65, 0x72, 0x46, 0x61, + 0x6d, 0x69, 0x6c, 0x79, 0x49, 0x64, 0x12, 0x21, 0x0a, 0x0c, 0x74, 0x72, 0x61, 0x6e, 0x73, 0x66, + 0x6f, 0x72, 0x6d, 0x5f, 0x69, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0b, 0x74, 0x72, + 0x61, 0x6e, 0x73, 0x66, 0x6f, 0x72, 0x6d, 0x49, 0x64, 0x12, 0x1d, 0x0a, 0x0a, 0x6c, 0x6f, 0x63, + 0x61, 0x6c, 0x5f, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x09, 0x6c, + 0x6f, 0x63, 0x61, 0x6c, 0x4e, 0x61, 0x6d, 0x65, 0x1a, 0xd2, 0x01, 0x0a, 0x10, 0x57, 0x69, 0x72, + 0x65, 0x43, 0x6f, 0x64, 0x65, 0x72, 0x53, 0x65, 0x74, 0x74, 0x69, 0x6e, 0x67, 0x12, 0x10, 0x0a, + 0x03, 0x75, 0x72, 0x6e, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x03, 0x75, 0x72, 0x6e, 0x12, + 0x18, 0x0a, 0x07, 0x70, 0x61, 0x79, 0x6c, 0x6f, 0x61, 0x64, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0c, + 0x52, 0x07, 0x70, 0x61, 0x79, 0x6c, 0x6f, 0x61, 0x64, 0x12, 0x2d, 0x0a, 0x12, 0x69, 0x6e, 0x70, + 0x75, 0x74, 0x5f, 0x6f, 0x72, 0x5f, 0x6f, 0x75, 0x74, 0x70, 0x75, 0x74, 0x5f, 0x69, 0x64, 0x18, + 0x03, 0x20, 0x01, 0x28, 0x09, 0x48, 0x00, 0x52, 0x0f, 0x69, 0x6e, 0x70, 0x75, 0x74, 0x4f, 0x72, + 0x4f, 0x75, 0x74, 0x70, 0x75, 0x74, 0x49, 0x64, 0x12, 0x59, 0x0a, 0x05, 0x74, 0x69, 0x6d, 0x65, + 0x72, 0x18, 0x04, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x41, 0x2e, 0x6f, 0x72, 0x67, 0x2e, 0x61, 0x70, + 0x61, 0x63, 0x68, 0x65, 0x2e, 0x62, 0x65, 0x61, 0x6d, 0x2e, 0x6d, 0x6f, 0x64, 0x65, 0x6c, 0x2e, + 0x70, 0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x45, 0x78, 0x65, 0x63, + 0x75, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x53, 0x74, 0x61, 0x67, 0x65, 0x50, 0x61, 0x79, 0x6c, 0x6f, + 0x61, 0x64, 0x2e, 0x54, 0x69, 0x6d, 0x65, 0x72, 0x49, 0x64, 0x48, 0x00, 0x52, 0x05, 0x74, 0x69, + 0x6d, 0x65, 0x72, 0x42, 0x08, 0x0a, 0x06, 0x74, 0x61, 0x72, 0x67, 0x65, 0x74, 0x22, 0xc2, 0x01, + 0x0a, 0x15, 0x53, 0x74, 0x61, 0x6e, 0x64, 0x61, 0x72, 0x64, 0x52, 0x65, 0x73, 0x6f, 0x75, 0x72, + 0x63, 0x65, 0x48, 0x69, 0x6e, 0x74, 0x73, 0x22, 0xa8, 0x01, 0x0a, 0x04, 0x45, 0x6e, 0x75, 0x6d, + 0x12, 0x34, 0x0a, 0x0b, 0x41, 0x43, 0x43, 0x45, 0x4c, 0x45, 0x52, 0x41, 0x54, 0x4f, 0x52, 0x10, + 0x00, 0x1a, 0x23, 0xa2, 0xb4, 0xfa, 0xc2, 0x05, 0x1d, 0x62, 0x65, 0x61, 0x6d, 0x3a, 0x72, 0x65, + 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x73, 0x3a, 0x61, 0x63, 0x63, 0x65, 0x6c, 0x65, 0x72, 0x61, + 0x74, 0x6f, 0x72, 0x3a, 0x76, 0x31, 0x12, 0x38, 0x0a, 0x0d, 0x4d, 0x49, 0x4e, 0x5f, 0x52, 0x41, + 0x4d, 0x5f, 0x42, 0x59, 0x54, 0x45, 0x53, 0x10, 0x01, 0x1a, 0x25, 0xa2, 0xb4, 0xfa, 0xc2, 0x05, + 0x1f, 0x62, 0x65, 0x61, 0x6d, 0x3a, 0x72, 0x65, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x73, 0x3a, + 0x6d, 0x69, 0x6e, 0x5f, 0x72, 0x61, 0x6d, 0x5f, 0x62, 0x79, 0x74, 0x65, 0x73, 0x3a, 0x76, 0x31, + 0x12, 0x30, 0x0a, 0x09, 0x43, 0x50, 0x55, 0x5f, 0x43, 0x4f, 0x55, 0x4e, 0x54, 0x10, 0x02, 0x1a, + 0x21, 0xa2, 0xb4, 0xfa, 0xc2, 0x05, 0x1b, 0x62, 0x65, 0x61, 0x6d, 0x3a, 0x72, 0x65, 0x73, 0x6f, + 0x75, 0x72, 0x63, 0x65, 0x73, 0x3a, 0x63, 0x70, 0x75, 0x5f, 0x63, 0x6f, 0x75, 0x6e, 0x74, 0x3a, + 0x76, 0x31, 0x32, 0x8f, 0x01, 0x0a, 0x11, 0x54, 0x65, 0x73, 0x74, 0x53, 0x74, 0x72, 0x65, 0x61, + 0x6d, 0x53, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, 0x12, 0x7a, 0x0a, 0x06, 0x45, 0x76, 0x65, 0x6e, + 0x74, 0x73, 0x12, 0x30, 0x2e, 0x6f, 0x72, 0x67, 0x2e, 0x61, 0x70, 0x61, 0x63, 0x68, 0x65, 0x2e, + 0x62, 0x65, 0x61, 0x6d, 0x2e, 0x6d, 0x6f, 0x64, 0x65, 0x6c, 0x2e, 0x70, 0x69, 0x70, 0x65, 0x6c, + 0x69, 0x6e, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x45, 0x76, 0x65, 0x6e, 0x74, 0x73, 0x52, 0x65, 0x71, + 0x75, 0x65, 0x73, 0x74, 0x1a, 0x3a, 0x2e, 0x6f, 0x72, 0x67, 0x2e, 0x61, 0x70, 0x61, 0x63, 0x68, + 0x65, 0x2e, 0x62, 0x65, 0x61, 0x6d, 0x2e, 0x6d, 0x6f, 0x64, 0x65, 0x6c, 0x2e, 0x70, 0x69, 0x70, + 0x65, 0x6c, 0x69, 0x6e, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x54, 0x65, 0x73, 0x74, 0x53, 0x74, 0x72, + 0x65, 0x61, 0x6d, 0x50, 0x61, 0x79, 0x6c, 0x6f, 0x61, 0x64, 0x2e, 0x45, 0x76, 0x65, 0x6e, 0x74, + 0x22, 0x00, 0x30, 0x01, 0x3a, 0x3f, 0x0a, 0x08, 0x62, 0x65, 0x61, 0x6d, 0x5f, 0x75, 0x72, 0x6e, + 0x12, 0x21, 0x2e, 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, + 0x75, 0x66, 0x2e, 0x45, 0x6e, 0x75, 0x6d, 0x56, 0x61, 0x6c, 0x75, 0x65, 0x4f, 0x70, 0x74, 0x69, + 0x6f, 0x6e, 0x73, 0x18, 0xc4, 0xa6, 0xaf, 0x58, 0x20, 0x01, 0x28, 0x09, 0x52, 0x07, 0x62, 0x65, + 0x61, 0x6d, 0x55, 0x72, 0x6e, 0x3a, 0x49, 0x0a, 0x0d, 0x62, 0x65, 0x61, 0x6d, 0x5f, 0x63, 0x6f, + 0x6e, 0x73, 0x74, 0x61, 0x6e, 0x74, 0x12, 0x21, 0x2e, 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2e, + 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66, 0x2e, 0x45, 0x6e, 0x75, 0x6d, 0x56, 0x61, 0x6c, + 0x75, 0x65, 0x4f, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x18, 0xc5, 0xa6, 0xaf, 0x58, 0x20, 0x01, + 0x28, 0x09, 0x52, 0x0c, 0x62, 0x65, 0x61, 0x6d, 0x43, 0x6f, 0x6e, 0x73, 0x74, 0x61, 0x6e, 0x74, + 0x42, 0x78, 0x0a, 0x21, 0x6f, 0x72, 0x67, 0x2e, 0x61, 0x70, 0x61, 0x63, 0x68, 0x65, 0x2e, 0x62, + 0x65, 0x61, 0x6d, 0x2e, 0x6d, 0x6f, 0x64, 0x65, 0x6c, 0x2e, 0x70, 0x69, 0x70, 0x65, 0x6c, 0x69, + 0x6e, 0x65, 0x2e, 0x76, 0x31, 0x42, 0x09, 0x52, 0x75, 0x6e, 0x6e, 0x65, 0x72, 0x41, 0x70, 0x69, + 0x5a, 0x48, 0x67, 0x69, 0x74, 0x68, 0x75, 0x62, 0x2e, 0x63, 0x6f, 0x6d, 0x2f, 0x61, 0x70, 0x61, + 0x63, 0x68, 0x65, 0x2f, 0x62, 0x65, 0x61, 0x6d, 0x2f, 0x73, 0x64, 0x6b, 0x73, 0x2f, 0x76, 0x32, + 0x2f, 0x67, 0x6f, 0x2f, 0x70, 0x6b, 0x67, 0x2f, 0x62, 0x65, 0x61, 0x6d, 0x2f, 0x6d, 0x6f, 0x64, + 0x65, 0x6c, 0x2f, 0x70, 0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e, 0x65, 0x5f, 0x76, 0x31, 0x3b, 0x70, + 0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e, 0x65, 0x5f, 0x76, 0x31, 0x62, 0x06, 0x70, 0x72, 0x6f, 0x74, + 0x6f, 0x33, +} + +var file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_enumTypes = make([]protoimpl.EnumInfo, 25) +var file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_msgTypes = make([]protoimpl.MessageInfo, 105) +var file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_goTypes = []any{ + (BeamConstants_Constants)(0), // 0: org.apache.beam.model.pipeline.v1.BeamConstants.Constants + (StandardPTransforms_Primitives)(0), // 1: org.apache.beam.model.pipeline.v1.StandardPTransforms.Primitives + (StandardPTransforms_DeprecatedPrimitives)(0), // 2: org.apache.beam.model.pipeline.v1.StandardPTransforms.DeprecatedPrimitives + (StandardPTransforms_Composites)(0), // 3: org.apache.beam.model.pipeline.v1.StandardPTransforms.Composites + (StandardPTransforms_CombineComponents)(0), // 4: org.apache.beam.model.pipeline.v1.StandardPTransforms.CombineComponents + (StandardPTransforms_SplittableParDoComponents)(0), // 5: org.apache.beam.model.pipeline.v1.StandardPTransforms.SplittableParDoComponents + (StandardPTransforms_GroupIntoBatchesComponents)(0), // 6: org.apache.beam.model.pipeline.v1.StandardPTransforms.GroupIntoBatchesComponents + (StandardSideInputTypes_Enum)(0), // 7: org.apache.beam.model.pipeline.v1.StandardSideInputTypes.Enum + (StandardUserStateTypes_Enum)(0), // 8: org.apache.beam.model.pipeline.v1.StandardUserStateTypes.Enum + (IsBounded_Enum)(0), // 9: org.apache.beam.model.pipeline.v1.IsBounded.Enum + (StandardCoders_Enum)(0), // 10: org.apache.beam.model.pipeline.v1.StandardCoders.Enum + (MergeStatus_Enum)(0), // 11: org.apache.beam.model.pipeline.v1.MergeStatus.Enum + (AccumulationMode_Enum)(0), // 12: org.apache.beam.model.pipeline.v1.AccumulationMode.Enum + (ClosingBehavior_Enum)(0), // 13: org.apache.beam.model.pipeline.v1.ClosingBehavior.Enum + (OnTimeBehavior_Enum)(0), // 14: org.apache.beam.model.pipeline.v1.OnTimeBehavior.Enum + (OutputTime_Enum)(0), // 15: org.apache.beam.model.pipeline.v1.OutputTime.Enum + (TimeDomain_Enum)(0), // 16: org.apache.beam.model.pipeline.v1.TimeDomain.Enum + (StandardArtifacts_Types)(0), // 17: org.apache.beam.model.pipeline.v1.StandardArtifacts.Types + (StandardArtifacts_Roles)(0), // 18: org.apache.beam.model.pipeline.v1.StandardArtifacts.Roles + (StandardEnvironments_Environments)(0), // 19: org.apache.beam.model.pipeline.v1.StandardEnvironments.Environments + (StandardProtocols_Enum)(0), // 20: org.apache.beam.model.pipeline.v1.StandardProtocols.Enum + (StandardRunnerProtocols_Enum)(0), // 21: org.apache.beam.model.pipeline.v1.StandardRunnerProtocols.Enum + (StandardRequirements_Enum)(0), // 22: org.apache.beam.model.pipeline.v1.StandardRequirements.Enum + (StandardDisplayData_DisplayData)(0), // 23: org.apache.beam.model.pipeline.v1.StandardDisplayData.DisplayData + (StandardResourceHints_Enum)(0), // 24: org.apache.beam.model.pipeline.v1.StandardResourceHints.Enum + (*BeamConstants)(nil), // 25: org.apache.beam.model.pipeline.v1.BeamConstants + (*Components)(nil), // 26: org.apache.beam.model.pipeline.v1.Components + (*Pipeline)(nil), // 27: org.apache.beam.model.pipeline.v1.Pipeline + (*PTransform)(nil), // 28: org.apache.beam.model.pipeline.v1.PTransform + (*StandardPTransforms)(nil), // 29: org.apache.beam.model.pipeline.v1.StandardPTransforms + (*StandardSideInputTypes)(nil), // 30: org.apache.beam.model.pipeline.v1.StandardSideInputTypes + (*StandardUserStateTypes)(nil), // 31: org.apache.beam.model.pipeline.v1.StandardUserStateTypes + (*PCollection)(nil), // 32: org.apache.beam.model.pipeline.v1.PCollection + (*ParDoPayload)(nil), // 33: org.apache.beam.model.pipeline.v1.ParDoPayload + (*StateSpec)(nil), // 34: org.apache.beam.model.pipeline.v1.StateSpec + (*ReadModifyWriteStateSpec)(nil), // 35: org.apache.beam.model.pipeline.v1.ReadModifyWriteStateSpec + (*BagStateSpec)(nil), // 36: org.apache.beam.model.pipeline.v1.BagStateSpec + (*OrderedListStateSpec)(nil), // 37: org.apache.beam.model.pipeline.v1.OrderedListStateSpec + (*CombiningStateSpec)(nil), // 38: org.apache.beam.model.pipeline.v1.CombiningStateSpec + (*MapStateSpec)(nil), // 39: org.apache.beam.model.pipeline.v1.MapStateSpec + (*MultimapStateSpec)(nil), // 40: org.apache.beam.model.pipeline.v1.MultimapStateSpec + (*SetStateSpec)(nil), // 41: org.apache.beam.model.pipeline.v1.SetStateSpec + (*TimerFamilySpec)(nil), // 42: org.apache.beam.model.pipeline.v1.TimerFamilySpec + (*IsBounded)(nil), // 43: org.apache.beam.model.pipeline.v1.IsBounded + (*ReadPayload)(nil), // 44: org.apache.beam.model.pipeline.v1.ReadPayload + (*WindowIntoPayload)(nil), // 45: org.apache.beam.model.pipeline.v1.WindowIntoPayload + (*CombinePayload)(nil), // 46: org.apache.beam.model.pipeline.v1.CombinePayload + (*TestStreamPayload)(nil), // 47: org.apache.beam.model.pipeline.v1.TestStreamPayload + (*EventsRequest)(nil), // 48: org.apache.beam.model.pipeline.v1.EventsRequest + (*WriteFilesPayload)(nil), // 49: org.apache.beam.model.pipeline.v1.WriteFilesPayload + (*PubSubReadPayload)(nil), // 50: org.apache.beam.model.pipeline.v1.PubSubReadPayload + (*PubSubWritePayload)(nil), // 51: org.apache.beam.model.pipeline.v1.PubSubWritePayload + (*GroupIntoBatchesPayload)(nil), // 52: org.apache.beam.model.pipeline.v1.GroupIntoBatchesPayload + (*RedistributePayload)(nil), // 53: org.apache.beam.model.pipeline.v1.RedistributePayload + (*Coder)(nil), // 54: org.apache.beam.model.pipeline.v1.Coder + (*StandardCoders)(nil), // 55: org.apache.beam.model.pipeline.v1.StandardCoders + (*WindowingStrategy)(nil), // 56: org.apache.beam.model.pipeline.v1.WindowingStrategy + (*MergeStatus)(nil), // 57: org.apache.beam.model.pipeline.v1.MergeStatus + (*AccumulationMode)(nil), // 58: org.apache.beam.model.pipeline.v1.AccumulationMode + (*ClosingBehavior)(nil), // 59: org.apache.beam.model.pipeline.v1.ClosingBehavior + (*OnTimeBehavior)(nil), // 60: org.apache.beam.model.pipeline.v1.OnTimeBehavior + (*OutputTime)(nil), // 61: org.apache.beam.model.pipeline.v1.OutputTime + (*TimeDomain)(nil), // 62: org.apache.beam.model.pipeline.v1.TimeDomain + (*Trigger)(nil), // 63: org.apache.beam.model.pipeline.v1.Trigger + (*TimestampTransform)(nil), // 64: org.apache.beam.model.pipeline.v1.TimestampTransform + (*SideInput)(nil), // 65: org.apache.beam.model.pipeline.v1.SideInput + (*StandardArtifacts)(nil), // 66: org.apache.beam.model.pipeline.v1.StandardArtifacts + (*ArtifactFilePayload)(nil), // 67: org.apache.beam.model.pipeline.v1.ArtifactFilePayload + (*ArtifactUrlPayload)(nil), // 68: org.apache.beam.model.pipeline.v1.ArtifactUrlPayload + (*EmbeddedFilePayload)(nil), // 69: org.apache.beam.model.pipeline.v1.EmbeddedFilePayload + (*PyPIPayload)(nil), // 70: org.apache.beam.model.pipeline.v1.PyPIPayload + (*MavenPayload)(nil), // 71: org.apache.beam.model.pipeline.v1.MavenPayload + (*DeferredArtifactPayload)(nil), // 72: org.apache.beam.model.pipeline.v1.DeferredArtifactPayload + (*ArtifactStagingToRolePayload)(nil), // 73: org.apache.beam.model.pipeline.v1.ArtifactStagingToRolePayload + (*ArtifactInformation)(nil), // 74: org.apache.beam.model.pipeline.v1.ArtifactInformation + (*Environment)(nil), // 75: org.apache.beam.model.pipeline.v1.Environment + (*StandardEnvironments)(nil), // 76: org.apache.beam.model.pipeline.v1.StandardEnvironments + (*DockerPayload)(nil), // 77: org.apache.beam.model.pipeline.v1.DockerPayload + (*ProcessPayload)(nil), // 78: org.apache.beam.model.pipeline.v1.ProcessPayload + (*ExternalPayload)(nil), // 79: org.apache.beam.model.pipeline.v1.ExternalPayload + (*AnyOfEnvironmentPayload)(nil), // 80: org.apache.beam.model.pipeline.v1.AnyOfEnvironmentPayload + (*StandardProtocols)(nil), // 81: org.apache.beam.model.pipeline.v1.StandardProtocols + (*StandardRunnerProtocols)(nil), // 82: org.apache.beam.model.pipeline.v1.StandardRunnerProtocols + (*StandardRequirements)(nil), // 83: org.apache.beam.model.pipeline.v1.StandardRequirements + (*FunctionSpec)(nil), // 84: org.apache.beam.model.pipeline.v1.FunctionSpec + (*StandardDisplayData)(nil), // 85: org.apache.beam.model.pipeline.v1.StandardDisplayData + (*LabelledPayload)(nil), // 86: org.apache.beam.model.pipeline.v1.LabelledPayload + (*DisplayData)(nil), // 87: org.apache.beam.model.pipeline.v1.DisplayData + (*MessageWithComponents)(nil), // 88: org.apache.beam.model.pipeline.v1.MessageWithComponents + (*ExecutableStagePayload)(nil), // 89: org.apache.beam.model.pipeline.v1.ExecutableStagePayload + (*StandardResourceHints)(nil), // 90: org.apache.beam.model.pipeline.v1.StandardResourceHints + nil, // 91: org.apache.beam.model.pipeline.v1.Components.TransformsEntry + nil, // 92: org.apache.beam.model.pipeline.v1.Components.PcollectionsEntry + nil, // 93: org.apache.beam.model.pipeline.v1.Components.WindowingStrategiesEntry + nil, // 94: org.apache.beam.model.pipeline.v1.Components.CodersEntry + nil, // 95: org.apache.beam.model.pipeline.v1.Components.EnvironmentsEntry + nil, // 96: org.apache.beam.model.pipeline.v1.PTransform.InputsEntry + nil, // 97: org.apache.beam.model.pipeline.v1.PTransform.OutputsEntry + nil, // 98: org.apache.beam.model.pipeline.v1.PTransform.AnnotationsEntry + nil, // 99: org.apache.beam.model.pipeline.v1.ParDoPayload.SideInputsEntry + nil, // 100: org.apache.beam.model.pipeline.v1.ParDoPayload.StateSpecsEntry + nil, // 101: org.apache.beam.model.pipeline.v1.ParDoPayload.TimerFamilySpecsEntry + (*TestStreamPayload_Event)(nil), // 102: org.apache.beam.model.pipeline.v1.TestStreamPayload.Event + (*TestStreamPayload_TimestampedElement)(nil), // 103: org.apache.beam.model.pipeline.v1.TestStreamPayload.TimestampedElement + (*TestStreamPayload_Event_AdvanceWatermark)(nil), // 104: org.apache.beam.model.pipeline.v1.TestStreamPayload.Event.AdvanceWatermark + (*TestStreamPayload_Event_AdvanceProcessingTime)(nil), // 105: org.apache.beam.model.pipeline.v1.TestStreamPayload.Event.AdvanceProcessingTime + (*TestStreamPayload_Event_AddElements)(nil), // 106: org.apache.beam.model.pipeline.v1.TestStreamPayload.Event.AddElements + nil, // 107: org.apache.beam.model.pipeline.v1.WriteFilesPayload.SideInputsEntry + (*Trigger_AfterAll)(nil), // 108: org.apache.beam.model.pipeline.v1.Trigger.AfterAll + (*Trigger_AfterAny)(nil), // 109: org.apache.beam.model.pipeline.v1.Trigger.AfterAny + (*Trigger_AfterEach)(nil), // 110: org.apache.beam.model.pipeline.v1.Trigger.AfterEach + (*Trigger_AfterEndOfWindow)(nil), // 111: org.apache.beam.model.pipeline.v1.Trigger.AfterEndOfWindow + (*Trigger_AfterProcessingTime)(nil), // 112: org.apache.beam.model.pipeline.v1.Trigger.AfterProcessingTime + (*Trigger_AfterSynchronizedProcessingTime)(nil), // 113: org.apache.beam.model.pipeline.v1.Trigger.AfterSynchronizedProcessingTime + (*Trigger_Default)(nil), // 114: org.apache.beam.model.pipeline.v1.Trigger.Default + (*Trigger_ElementCount)(nil), // 115: org.apache.beam.model.pipeline.v1.Trigger.ElementCount + (*Trigger_Never)(nil), // 116: org.apache.beam.model.pipeline.v1.Trigger.Never + (*Trigger_Always)(nil), // 117: org.apache.beam.model.pipeline.v1.Trigger.Always + (*Trigger_OrFinally)(nil), // 118: org.apache.beam.model.pipeline.v1.Trigger.OrFinally + (*Trigger_Repeat)(nil), // 119: org.apache.beam.model.pipeline.v1.Trigger.Repeat + (*TimestampTransform_Delay)(nil), // 120: org.apache.beam.model.pipeline.v1.TimestampTransform.Delay + (*TimestampTransform_AlignTo)(nil), // 121: org.apache.beam.model.pipeline.v1.TimestampTransform.AlignTo + nil, // 122: org.apache.beam.model.pipeline.v1.Environment.ResourceHintsEntry + nil, // 123: org.apache.beam.model.pipeline.v1.ProcessPayload.EnvEntry + nil, // 124: org.apache.beam.model.pipeline.v1.ExternalPayload.ParamsEntry + (*ExecutableStagePayload_SideInputId)(nil), // 125: org.apache.beam.model.pipeline.v1.ExecutableStagePayload.SideInputId + (*ExecutableStagePayload_UserStateId)(nil), // 126: org.apache.beam.model.pipeline.v1.ExecutableStagePayload.UserStateId + (*ExecutableStagePayload_TimerId)(nil), // 127: org.apache.beam.model.pipeline.v1.ExecutableStagePayload.TimerId + (*ExecutableStagePayload_TimerFamilyId)(nil), // 128: org.apache.beam.model.pipeline.v1.ExecutableStagePayload.TimerFamilyId + (*ExecutableStagePayload_WireCoderSetting)(nil), // 129: org.apache.beam.model.pipeline.v1.ExecutableStagePayload.WireCoderSetting + (*ApiServiceDescriptor)(nil), // 130: org.apache.beam.model.pipeline.v1.ApiServiceDescriptor + (*descriptorpb.EnumValueOptions)(nil), // 131: google.protobuf.EnumValueOptions +} +var file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_depIdxs = []int32{ + 91, // 0: org.apache.beam.model.pipeline.v1.Components.transforms:type_name -> org.apache.beam.model.pipeline.v1.Components.TransformsEntry + 92, // 1: org.apache.beam.model.pipeline.v1.Components.pcollections:type_name -> org.apache.beam.model.pipeline.v1.Components.PcollectionsEntry + 93, // 2: org.apache.beam.model.pipeline.v1.Components.windowing_strategies:type_name -> org.apache.beam.model.pipeline.v1.Components.WindowingStrategiesEntry + 94, // 3: org.apache.beam.model.pipeline.v1.Components.coders:type_name -> org.apache.beam.model.pipeline.v1.Components.CodersEntry + 95, // 4: org.apache.beam.model.pipeline.v1.Components.environments:type_name -> org.apache.beam.model.pipeline.v1.Components.EnvironmentsEntry + 26, // 5: org.apache.beam.model.pipeline.v1.Pipeline.components:type_name -> org.apache.beam.model.pipeline.v1.Components + 87, // 6: org.apache.beam.model.pipeline.v1.Pipeline.display_data:type_name -> org.apache.beam.model.pipeline.v1.DisplayData + 84, // 7: org.apache.beam.model.pipeline.v1.PTransform.spec:type_name -> org.apache.beam.model.pipeline.v1.FunctionSpec + 96, // 8: org.apache.beam.model.pipeline.v1.PTransform.inputs:type_name -> org.apache.beam.model.pipeline.v1.PTransform.InputsEntry + 97, // 9: org.apache.beam.model.pipeline.v1.PTransform.outputs:type_name -> org.apache.beam.model.pipeline.v1.PTransform.OutputsEntry + 87, // 10: org.apache.beam.model.pipeline.v1.PTransform.display_data:type_name -> org.apache.beam.model.pipeline.v1.DisplayData + 98, // 11: org.apache.beam.model.pipeline.v1.PTransform.annotations:type_name -> org.apache.beam.model.pipeline.v1.PTransform.AnnotationsEntry + 9, // 12: org.apache.beam.model.pipeline.v1.PCollection.is_bounded:type_name -> org.apache.beam.model.pipeline.v1.IsBounded.Enum + 87, // 13: org.apache.beam.model.pipeline.v1.PCollection.display_data:type_name -> org.apache.beam.model.pipeline.v1.DisplayData + 84, // 14: org.apache.beam.model.pipeline.v1.ParDoPayload.do_fn:type_name -> org.apache.beam.model.pipeline.v1.FunctionSpec + 99, // 15: org.apache.beam.model.pipeline.v1.ParDoPayload.side_inputs:type_name -> org.apache.beam.model.pipeline.v1.ParDoPayload.SideInputsEntry + 100, // 16: org.apache.beam.model.pipeline.v1.ParDoPayload.state_specs:type_name -> org.apache.beam.model.pipeline.v1.ParDoPayload.StateSpecsEntry + 101, // 17: org.apache.beam.model.pipeline.v1.ParDoPayload.timer_family_specs:type_name -> org.apache.beam.model.pipeline.v1.ParDoPayload.TimerFamilySpecsEntry + 35, // 18: org.apache.beam.model.pipeline.v1.StateSpec.read_modify_write_spec:type_name -> org.apache.beam.model.pipeline.v1.ReadModifyWriteStateSpec + 36, // 19: org.apache.beam.model.pipeline.v1.StateSpec.bag_spec:type_name -> org.apache.beam.model.pipeline.v1.BagStateSpec + 38, // 20: org.apache.beam.model.pipeline.v1.StateSpec.combining_spec:type_name -> org.apache.beam.model.pipeline.v1.CombiningStateSpec + 39, // 21: org.apache.beam.model.pipeline.v1.StateSpec.map_spec:type_name -> org.apache.beam.model.pipeline.v1.MapStateSpec + 41, // 22: org.apache.beam.model.pipeline.v1.StateSpec.set_spec:type_name -> org.apache.beam.model.pipeline.v1.SetStateSpec + 37, // 23: org.apache.beam.model.pipeline.v1.StateSpec.ordered_list_spec:type_name -> org.apache.beam.model.pipeline.v1.OrderedListStateSpec + 40, // 24: org.apache.beam.model.pipeline.v1.StateSpec.multimap_spec:type_name -> org.apache.beam.model.pipeline.v1.MultimapStateSpec + 84, // 25: org.apache.beam.model.pipeline.v1.StateSpec.protocol:type_name -> org.apache.beam.model.pipeline.v1.FunctionSpec + 84, // 26: org.apache.beam.model.pipeline.v1.CombiningStateSpec.combine_fn:type_name -> org.apache.beam.model.pipeline.v1.FunctionSpec + 16, // 27: org.apache.beam.model.pipeline.v1.TimerFamilySpec.time_domain:type_name -> org.apache.beam.model.pipeline.v1.TimeDomain.Enum + 84, // 28: org.apache.beam.model.pipeline.v1.ReadPayload.source:type_name -> org.apache.beam.model.pipeline.v1.FunctionSpec + 9, // 29: org.apache.beam.model.pipeline.v1.ReadPayload.is_bounded:type_name -> org.apache.beam.model.pipeline.v1.IsBounded.Enum + 84, // 30: org.apache.beam.model.pipeline.v1.WindowIntoPayload.window_fn:type_name -> org.apache.beam.model.pipeline.v1.FunctionSpec + 84, // 31: org.apache.beam.model.pipeline.v1.CombinePayload.combine_fn:type_name -> org.apache.beam.model.pipeline.v1.FunctionSpec + 102, // 32: org.apache.beam.model.pipeline.v1.TestStreamPayload.events:type_name -> org.apache.beam.model.pipeline.v1.TestStreamPayload.Event + 130, // 33: org.apache.beam.model.pipeline.v1.TestStreamPayload.endpoint:type_name -> org.apache.beam.model.pipeline.v1.ApiServiceDescriptor + 84, // 34: org.apache.beam.model.pipeline.v1.WriteFilesPayload.sink:type_name -> org.apache.beam.model.pipeline.v1.FunctionSpec + 84, // 35: org.apache.beam.model.pipeline.v1.WriteFilesPayload.format_function:type_name -> org.apache.beam.model.pipeline.v1.FunctionSpec + 107, // 36: org.apache.beam.model.pipeline.v1.WriteFilesPayload.side_inputs:type_name -> org.apache.beam.model.pipeline.v1.WriteFilesPayload.SideInputsEntry + 84, // 37: org.apache.beam.model.pipeline.v1.Coder.spec:type_name -> org.apache.beam.model.pipeline.v1.FunctionSpec + 84, // 38: org.apache.beam.model.pipeline.v1.WindowingStrategy.window_fn:type_name -> org.apache.beam.model.pipeline.v1.FunctionSpec + 11, // 39: org.apache.beam.model.pipeline.v1.WindowingStrategy.merge_status:type_name -> org.apache.beam.model.pipeline.v1.MergeStatus.Enum + 63, // 40: org.apache.beam.model.pipeline.v1.WindowingStrategy.trigger:type_name -> org.apache.beam.model.pipeline.v1.Trigger + 12, // 41: org.apache.beam.model.pipeline.v1.WindowingStrategy.accumulation_mode:type_name -> org.apache.beam.model.pipeline.v1.AccumulationMode.Enum + 15, // 42: org.apache.beam.model.pipeline.v1.WindowingStrategy.output_time:type_name -> org.apache.beam.model.pipeline.v1.OutputTime.Enum + 13, // 43: org.apache.beam.model.pipeline.v1.WindowingStrategy.closing_behavior:type_name -> org.apache.beam.model.pipeline.v1.ClosingBehavior.Enum + 14, // 44: org.apache.beam.model.pipeline.v1.WindowingStrategy.on_time_behavior:type_name -> org.apache.beam.model.pipeline.v1.OnTimeBehavior.Enum + 108, // 45: org.apache.beam.model.pipeline.v1.Trigger.after_all:type_name -> org.apache.beam.model.pipeline.v1.Trigger.AfterAll + 109, // 46: org.apache.beam.model.pipeline.v1.Trigger.after_any:type_name -> org.apache.beam.model.pipeline.v1.Trigger.AfterAny + 110, // 47: org.apache.beam.model.pipeline.v1.Trigger.after_each:type_name -> org.apache.beam.model.pipeline.v1.Trigger.AfterEach + 111, // 48: org.apache.beam.model.pipeline.v1.Trigger.after_end_of_window:type_name -> org.apache.beam.model.pipeline.v1.Trigger.AfterEndOfWindow + 112, // 49: org.apache.beam.model.pipeline.v1.Trigger.after_processing_time:type_name -> org.apache.beam.model.pipeline.v1.Trigger.AfterProcessingTime + 113, // 50: org.apache.beam.model.pipeline.v1.Trigger.after_synchronized_processing_time:type_name -> org.apache.beam.model.pipeline.v1.Trigger.AfterSynchronizedProcessingTime + 117, // 51: org.apache.beam.model.pipeline.v1.Trigger.always:type_name -> org.apache.beam.model.pipeline.v1.Trigger.Always + 114, // 52: org.apache.beam.model.pipeline.v1.Trigger.default:type_name -> org.apache.beam.model.pipeline.v1.Trigger.Default + 115, // 53: org.apache.beam.model.pipeline.v1.Trigger.element_count:type_name -> org.apache.beam.model.pipeline.v1.Trigger.ElementCount + 116, // 54: org.apache.beam.model.pipeline.v1.Trigger.never:type_name -> org.apache.beam.model.pipeline.v1.Trigger.Never + 118, // 55: org.apache.beam.model.pipeline.v1.Trigger.or_finally:type_name -> org.apache.beam.model.pipeline.v1.Trigger.OrFinally + 119, // 56: org.apache.beam.model.pipeline.v1.Trigger.repeat:type_name -> org.apache.beam.model.pipeline.v1.Trigger.Repeat + 120, // 57: org.apache.beam.model.pipeline.v1.TimestampTransform.delay:type_name -> org.apache.beam.model.pipeline.v1.TimestampTransform.Delay + 121, // 58: org.apache.beam.model.pipeline.v1.TimestampTransform.align_to:type_name -> org.apache.beam.model.pipeline.v1.TimestampTransform.AlignTo + 84, // 59: org.apache.beam.model.pipeline.v1.SideInput.access_pattern:type_name -> org.apache.beam.model.pipeline.v1.FunctionSpec + 84, // 60: org.apache.beam.model.pipeline.v1.SideInput.view_fn:type_name -> org.apache.beam.model.pipeline.v1.FunctionSpec + 84, // 61: org.apache.beam.model.pipeline.v1.SideInput.window_mapping_fn:type_name -> org.apache.beam.model.pipeline.v1.FunctionSpec + 87, // 62: org.apache.beam.model.pipeline.v1.Environment.display_data:type_name -> org.apache.beam.model.pipeline.v1.DisplayData + 74, // 63: org.apache.beam.model.pipeline.v1.Environment.dependencies:type_name -> org.apache.beam.model.pipeline.v1.ArtifactInformation + 122, // 64: org.apache.beam.model.pipeline.v1.Environment.resource_hints:type_name -> org.apache.beam.model.pipeline.v1.Environment.ResourceHintsEntry + 123, // 65: org.apache.beam.model.pipeline.v1.ProcessPayload.env:type_name -> org.apache.beam.model.pipeline.v1.ProcessPayload.EnvEntry + 130, // 66: org.apache.beam.model.pipeline.v1.ExternalPayload.endpoint:type_name -> org.apache.beam.model.pipeline.v1.ApiServiceDescriptor + 124, // 67: org.apache.beam.model.pipeline.v1.ExternalPayload.params:type_name -> org.apache.beam.model.pipeline.v1.ExternalPayload.ParamsEntry + 75, // 68: org.apache.beam.model.pipeline.v1.AnyOfEnvironmentPayload.environments:type_name -> org.apache.beam.model.pipeline.v1.Environment + 26, // 69: org.apache.beam.model.pipeline.v1.MessageWithComponents.components:type_name -> org.apache.beam.model.pipeline.v1.Components + 54, // 70: org.apache.beam.model.pipeline.v1.MessageWithComponents.coder:type_name -> org.apache.beam.model.pipeline.v1.Coder + 46, // 71: org.apache.beam.model.pipeline.v1.MessageWithComponents.combine_payload:type_name -> org.apache.beam.model.pipeline.v1.CombinePayload + 84, // 72: org.apache.beam.model.pipeline.v1.MessageWithComponents.function_spec:type_name -> org.apache.beam.model.pipeline.v1.FunctionSpec + 33, // 73: org.apache.beam.model.pipeline.v1.MessageWithComponents.par_do_payload:type_name -> org.apache.beam.model.pipeline.v1.ParDoPayload + 28, // 74: org.apache.beam.model.pipeline.v1.MessageWithComponents.ptransform:type_name -> org.apache.beam.model.pipeline.v1.PTransform + 32, // 75: org.apache.beam.model.pipeline.v1.MessageWithComponents.pcollection:type_name -> org.apache.beam.model.pipeline.v1.PCollection + 44, // 76: org.apache.beam.model.pipeline.v1.MessageWithComponents.read_payload:type_name -> org.apache.beam.model.pipeline.v1.ReadPayload + 65, // 77: org.apache.beam.model.pipeline.v1.MessageWithComponents.side_input:type_name -> org.apache.beam.model.pipeline.v1.SideInput + 45, // 78: org.apache.beam.model.pipeline.v1.MessageWithComponents.window_into_payload:type_name -> org.apache.beam.model.pipeline.v1.WindowIntoPayload + 56, // 79: org.apache.beam.model.pipeline.v1.MessageWithComponents.windowing_strategy:type_name -> org.apache.beam.model.pipeline.v1.WindowingStrategy + 75, // 80: org.apache.beam.model.pipeline.v1.ExecutableStagePayload.environment:type_name -> org.apache.beam.model.pipeline.v1.Environment + 129, // 81: org.apache.beam.model.pipeline.v1.ExecutableStagePayload.wire_coder_settings:type_name -> org.apache.beam.model.pipeline.v1.ExecutableStagePayload.WireCoderSetting + 125, // 82: org.apache.beam.model.pipeline.v1.ExecutableStagePayload.side_inputs:type_name -> org.apache.beam.model.pipeline.v1.ExecutableStagePayload.SideInputId + 26, // 83: org.apache.beam.model.pipeline.v1.ExecutableStagePayload.components:type_name -> org.apache.beam.model.pipeline.v1.Components + 126, // 84: org.apache.beam.model.pipeline.v1.ExecutableStagePayload.user_states:type_name -> org.apache.beam.model.pipeline.v1.ExecutableStagePayload.UserStateId + 127, // 85: org.apache.beam.model.pipeline.v1.ExecutableStagePayload.timers:type_name -> org.apache.beam.model.pipeline.v1.ExecutableStagePayload.TimerId + 128, // 86: org.apache.beam.model.pipeline.v1.ExecutableStagePayload.timerFamilies:type_name -> org.apache.beam.model.pipeline.v1.ExecutableStagePayload.TimerFamilyId + 28, // 87: org.apache.beam.model.pipeline.v1.Components.TransformsEntry.value:type_name -> org.apache.beam.model.pipeline.v1.PTransform + 32, // 88: org.apache.beam.model.pipeline.v1.Components.PcollectionsEntry.value:type_name -> org.apache.beam.model.pipeline.v1.PCollection + 56, // 89: org.apache.beam.model.pipeline.v1.Components.WindowingStrategiesEntry.value:type_name -> org.apache.beam.model.pipeline.v1.WindowingStrategy + 54, // 90: org.apache.beam.model.pipeline.v1.Components.CodersEntry.value:type_name -> org.apache.beam.model.pipeline.v1.Coder + 75, // 91: org.apache.beam.model.pipeline.v1.Components.EnvironmentsEntry.value:type_name -> org.apache.beam.model.pipeline.v1.Environment + 65, // 92: org.apache.beam.model.pipeline.v1.ParDoPayload.SideInputsEntry.value:type_name -> org.apache.beam.model.pipeline.v1.SideInput + 34, // 93: org.apache.beam.model.pipeline.v1.ParDoPayload.StateSpecsEntry.value:type_name -> org.apache.beam.model.pipeline.v1.StateSpec + 42, // 94: org.apache.beam.model.pipeline.v1.ParDoPayload.TimerFamilySpecsEntry.value:type_name -> org.apache.beam.model.pipeline.v1.TimerFamilySpec + 104, // 95: org.apache.beam.model.pipeline.v1.TestStreamPayload.Event.watermark_event:type_name -> org.apache.beam.model.pipeline.v1.TestStreamPayload.Event.AdvanceWatermark + 105, // 96: org.apache.beam.model.pipeline.v1.TestStreamPayload.Event.processing_time_event:type_name -> org.apache.beam.model.pipeline.v1.TestStreamPayload.Event.AdvanceProcessingTime + 106, // 97: org.apache.beam.model.pipeline.v1.TestStreamPayload.Event.element_event:type_name -> org.apache.beam.model.pipeline.v1.TestStreamPayload.Event.AddElements + 103, // 98: org.apache.beam.model.pipeline.v1.TestStreamPayload.Event.AddElements.elements:type_name -> org.apache.beam.model.pipeline.v1.TestStreamPayload.TimestampedElement + 65, // 99: org.apache.beam.model.pipeline.v1.WriteFilesPayload.SideInputsEntry.value:type_name -> org.apache.beam.model.pipeline.v1.SideInput + 63, // 100: org.apache.beam.model.pipeline.v1.Trigger.AfterAll.subtriggers:type_name -> org.apache.beam.model.pipeline.v1.Trigger + 63, // 101: org.apache.beam.model.pipeline.v1.Trigger.AfterAny.subtriggers:type_name -> org.apache.beam.model.pipeline.v1.Trigger + 63, // 102: org.apache.beam.model.pipeline.v1.Trigger.AfterEach.subtriggers:type_name -> org.apache.beam.model.pipeline.v1.Trigger + 63, // 103: org.apache.beam.model.pipeline.v1.Trigger.AfterEndOfWindow.early_firings:type_name -> org.apache.beam.model.pipeline.v1.Trigger + 63, // 104: org.apache.beam.model.pipeline.v1.Trigger.AfterEndOfWindow.late_firings:type_name -> org.apache.beam.model.pipeline.v1.Trigger + 64, // 105: org.apache.beam.model.pipeline.v1.Trigger.AfterProcessingTime.timestamp_transforms:type_name -> org.apache.beam.model.pipeline.v1.TimestampTransform + 63, // 106: org.apache.beam.model.pipeline.v1.Trigger.OrFinally.main:type_name -> org.apache.beam.model.pipeline.v1.Trigger + 63, // 107: org.apache.beam.model.pipeline.v1.Trigger.OrFinally.finally:type_name -> org.apache.beam.model.pipeline.v1.Trigger + 63, // 108: org.apache.beam.model.pipeline.v1.Trigger.Repeat.subtrigger:type_name -> org.apache.beam.model.pipeline.v1.Trigger + 127, // 109: org.apache.beam.model.pipeline.v1.ExecutableStagePayload.WireCoderSetting.timer:type_name -> org.apache.beam.model.pipeline.v1.ExecutableStagePayload.TimerId + 131, // 110: org.apache.beam.model.pipeline.v1.beam_urn:extendee -> google.protobuf.EnumValueOptions + 131, // 111: org.apache.beam.model.pipeline.v1.beam_constant:extendee -> google.protobuf.EnumValueOptions + 48, // 112: org.apache.beam.model.pipeline.v1.TestStreamService.Events:input_type -> org.apache.beam.model.pipeline.v1.EventsRequest + 102, // 113: org.apache.beam.model.pipeline.v1.TestStreamService.Events:output_type -> org.apache.beam.model.pipeline.v1.TestStreamPayload.Event + 113, // [113:114] is the sub-list for method output_type + 112, // [112:113] is the sub-list for method input_type + 112, // [112:112] is the sub-list for extension type_name + 110, // [110:112] is the sub-list for extension extendee + 0, // [0:110] is the sub-list for field type_name +} + +func init() { file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_init() } +func file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_init() { + if File_org_apache_beam_model_pipeline_v1_beam_runner_api_proto != nil { + return + } + file_org_apache_beam_model_pipeline_v1_endpoints_proto_init() + file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_msgTypes[9].OneofWrappers = []any{ + (*stateSpec_ReadModifyWriteSpec)(nil), + (*stateSpec_BagSpec)(nil), + (*stateSpec_CombiningSpec)(nil), + (*stateSpec_MapSpec)(nil), + (*stateSpec_SetSpec)(nil), + (*stateSpec_OrderedListSpec)(nil), + (*stateSpec_MultimapSpec)(nil), + } + file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_msgTypes[38].OneofWrappers = []any{ + (*trigger_AfterAll_)(nil), + (*trigger_AfterAny_)(nil), + (*trigger_AfterEach_)(nil), + (*trigger_AfterEndOfWindow_)(nil), + (*trigger_AfterProcessingTime_)(nil), + (*trigger_AfterSynchronizedProcessingTime_)(nil), + (*trigger_Always_)(nil), + (*trigger_Default_)(nil), + (*trigger_ElementCount_)(nil), + (*trigger_Never_)(nil), + (*trigger_OrFinally_)(nil), + (*trigger_Repeat_)(nil), + } + file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_msgTypes[39].OneofWrappers = []any{ + (*timestampTransform_Delay_)(nil), + (*timestampTransform_AlignTo_)(nil), + } + file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_msgTypes[61].OneofWrappers = []any{ + (*labelledPayload_StringValue)(nil), + (*labelledPayload_BoolValue)(nil), + (*labelledPayload_DoubleValue)(nil), + (*labelledPayload_IntValue)(nil), + } + file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_msgTypes[63].OneofWrappers = []any{ + (*messageWithComponents_Coder)(nil), + (*messageWithComponents_CombinePayload)(nil), + (*messageWithComponents_FunctionSpec)(nil), + (*messageWithComponents_ParDoPayload)(nil), + (*messageWithComponents_Ptransform)(nil), + (*messageWithComponents_Pcollection)(nil), + (*messageWithComponents_ReadPayload)(nil), + (*messageWithComponents_SideInput)(nil), + (*messageWithComponents_WindowIntoPayload)(nil), + (*messageWithComponents_WindowingStrategy)(nil), + } + file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_msgTypes[77].OneofWrappers = []any{ + (*testStreamPayload_Event_WatermarkEvent)(nil), + (*testStreamPayload_Event_ProcessingTimeEvent)(nil), + (*testStreamPayload_Event_ElementEvent)(nil), + } + file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_msgTypes[104].OneofWrappers = []any{ + (*executableStagePayload_WireCoderSetting_InputOrOutputId)(nil), + (*executableStagePayload_WireCoderSetting_Timer)(nil), + } + type x struct{} + out := protoimpl.TypeBuilder{ + File: protoimpl.DescBuilder{ + GoPackagePath: reflect.TypeOf(x{}).PkgPath(), + RawDescriptor: file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_rawDesc, + NumEnums: 25, + NumMessages: 105, + NumExtensions: 2, + NumServices: 1, + }, + GoTypes: file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_goTypes, + DependencyIndexes: file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_depIdxs, + EnumInfos: file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_enumTypes, + MessageInfos: file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_msgTypes, + ExtensionInfos: file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_extTypes, + }.Build() + File_org_apache_beam_model_pipeline_v1_beam_runner_api_proto = out.File + file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_rawDesc = nil + file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_goTypes = nil + file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_depIdxs = nil +} diff --git a/sdks/go/pkg/beam/model/pipeline_v1/endpoints.pb.go b/sdks/go/pkg/beam/model/pipeline_v1/endpoints.pb.go index a7a9b193bfe6..640dcac7af5d 100644 --- a/sdks/go/pkg/beam/model/pipeline_v1/endpoints.pb.go +++ b/sdks/go/pkg/beam/model/pipeline_v1/endpoints.pb.go @@ -20,17 +20,18 @@ // Code generated by protoc-gen-go. DO NOT EDIT. // versions: -// protoc-gen-go v1.27.1 +// protoc-gen-go v1.36.0 // protoc v5.27.3 // source: org/apache/beam/model/pipeline/v1/endpoints.proto +//go:build !protoopaque + package pipeline_v1 import ( protoreflect "google.golang.org/protobuf/reflect/protoreflect" protoimpl "google.golang.org/protobuf/runtime/protoimpl" reflect "reflect" - sync "sync" ) const ( @@ -42,25 +43,22 @@ const ( // A description of how to connect to a Beam API endpoint. type ApiServiceDescriptor struct { - state protoimpl.MessageState - sizeCache protoimpl.SizeCache - unknownFields protoimpl.UnknownFields - + state protoimpl.MessageState `protogen:"hybrid.v1"` // (Required) The URL to connect to. Url string `protobuf:"bytes,1,opt,name=url,proto3" json:"url,omitempty"` // (Optional) The method for authentication. If unspecified, access to the // url is already being performed in a trusted context (e.g. localhost, // private network). Authentication *AuthenticationSpec `protobuf:"bytes,2,opt,name=authentication,proto3" json:"authentication,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache } func (x *ApiServiceDescriptor) Reset() { *x = ApiServiceDescriptor{} - if protoimpl.UnsafeEnabled { - mi := &file_org_apache_beam_model_pipeline_v1_endpoints_proto_msgTypes[0] - ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) - ms.StoreMessageInfo(mi) - } + mi := &file_org_apache_beam_model_pipeline_v1_endpoints_proto_msgTypes[0] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) } func (x *ApiServiceDescriptor) String() string { @@ -71,7 +69,7 @@ func (*ApiServiceDescriptor) ProtoMessage() {} func (x *ApiServiceDescriptor) ProtoReflect() protoreflect.Message { mi := &file_org_apache_beam_model_pipeline_v1_endpoints_proto_msgTypes[0] - if protoimpl.UnsafeEnabled && x != nil { + if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { ms.StoreMessageInfo(mi) @@ -81,11 +79,6 @@ func (x *ApiServiceDescriptor) ProtoReflect() protoreflect.Message { return mi.MessageOf(x) } -// Deprecated: Use ApiServiceDescriptor.ProtoReflect.Descriptor instead. -func (*ApiServiceDescriptor) Descriptor() ([]byte, []int) { - return file_org_apache_beam_model_pipeline_v1_endpoints_proto_rawDescGZIP(), []int{0} -} - func (x *ApiServiceDescriptor) GetUrl() string { if x != nil { return x.Url @@ -100,11 +93,47 @@ func (x *ApiServiceDescriptor) GetAuthentication() *AuthenticationSpec { return nil } -type AuthenticationSpec struct { - state protoimpl.MessageState - sizeCache protoimpl.SizeCache - unknownFields protoimpl.UnknownFields +func (x *ApiServiceDescriptor) SetUrl(v string) { + x.Url = v +} + +func (x *ApiServiceDescriptor) SetAuthentication(v *AuthenticationSpec) { + x.Authentication = v +} +func (x *ApiServiceDescriptor) HasAuthentication() bool { + if x == nil { + return false + } + return x.Authentication != nil +} + +func (x *ApiServiceDescriptor) ClearAuthentication() { + x.Authentication = nil +} + +type ApiServiceDescriptor_builder struct { + _ [0]func() // Prevents comparability and use of unkeyed literals for the builder. + + // (Required) The URL to connect to. + Url string + // (Optional) The method for authentication. If unspecified, access to the + // url is already being performed in a trusted context (e.g. localhost, + // private network). + Authentication *AuthenticationSpec +} + +func (b0 ApiServiceDescriptor_builder) Build() *ApiServiceDescriptor { + m0 := &ApiServiceDescriptor{} + b, x := &b0, m0 + _, _ = b, x + x.Url = b.Url + x.Authentication = b.Authentication + return m0 +} + +type AuthenticationSpec struct { + state protoimpl.MessageState `protogen:"hybrid.v1"` // (Required) A URN that describes the accompanying payload. // For any URN that is not recognized (by whomever is inspecting // it) the parameter payload should be treated as opaque and @@ -112,16 +141,16 @@ type AuthenticationSpec struct { Urn string `protobuf:"bytes,1,opt,name=urn,proto3" json:"urn,omitempty"` // (Optional) The data specifying any parameters to the URN. If // the URN does not require any arguments, this may be omitted. - Payload []byte `protobuf:"bytes,2,opt,name=payload,proto3" json:"payload,omitempty"` + Payload []byte `protobuf:"bytes,2,opt,name=payload,proto3" json:"payload,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache } func (x *AuthenticationSpec) Reset() { *x = AuthenticationSpec{} - if protoimpl.UnsafeEnabled { - mi := &file_org_apache_beam_model_pipeline_v1_endpoints_proto_msgTypes[1] - ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) - ms.StoreMessageInfo(mi) - } + mi := &file_org_apache_beam_model_pipeline_v1_endpoints_proto_msgTypes[1] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) } func (x *AuthenticationSpec) String() string { @@ -132,7 +161,7 @@ func (*AuthenticationSpec) ProtoMessage() {} func (x *AuthenticationSpec) ProtoReflect() protoreflect.Message { mi := &file_org_apache_beam_model_pipeline_v1_endpoints_proto_msgTypes[1] - if protoimpl.UnsafeEnabled && x != nil { + if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { ms.StoreMessageInfo(mi) @@ -142,11 +171,6 @@ func (x *AuthenticationSpec) ProtoReflect() protoreflect.Message { return mi.MessageOf(x) } -// Deprecated: Use AuthenticationSpec.ProtoReflect.Descriptor instead. -func (*AuthenticationSpec) Descriptor() ([]byte, []int) { - return file_org_apache_beam_model_pipeline_v1_endpoints_proto_rawDescGZIP(), []int{1} -} - func (x *AuthenticationSpec) GetUrn() string { if x != nil { return x.Urn @@ -161,6 +185,39 @@ func (x *AuthenticationSpec) GetPayload() []byte { return nil } +func (x *AuthenticationSpec) SetUrn(v string) { + x.Urn = v +} + +func (x *AuthenticationSpec) SetPayload(v []byte) { + if v == nil { + v = []byte{} + } + x.Payload = v +} + +type AuthenticationSpec_builder struct { + _ [0]func() // Prevents comparability and use of unkeyed literals for the builder. + + // (Required) A URN that describes the accompanying payload. + // For any URN that is not recognized (by whomever is inspecting + // it) the parameter payload should be treated as opaque and + // passed as-is. + Urn string + // (Optional) The data specifying any parameters to the URN. If + // the URN does not require any arguments, this may be omitted. + Payload []byte +} + +func (b0 AuthenticationSpec_builder) Build() *AuthenticationSpec { + m0 := &AuthenticationSpec{} + b, x := &b0, m0 + _, _ = b, x + x.Urn = b.Urn + x.Payload = b.Payload + return m0 +} + var File_org_apache_beam_model_pipeline_v1_endpoints_proto protoreflect.FileDescriptor var file_org_apache_beam_model_pipeline_v1_endpoints_proto_rawDesc = []byte{ @@ -193,20 +250,8 @@ var file_org_apache_beam_model_pipeline_v1_endpoints_proto_rawDesc = []byte{ 0x6f, 0x74, 0x6f, 0x33, } -var ( - file_org_apache_beam_model_pipeline_v1_endpoints_proto_rawDescOnce sync.Once - file_org_apache_beam_model_pipeline_v1_endpoints_proto_rawDescData = file_org_apache_beam_model_pipeline_v1_endpoints_proto_rawDesc -) - -func file_org_apache_beam_model_pipeline_v1_endpoints_proto_rawDescGZIP() []byte { - file_org_apache_beam_model_pipeline_v1_endpoints_proto_rawDescOnce.Do(func() { - file_org_apache_beam_model_pipeline_v1_endpoints_proto_rawDescData = protoimpl.X.CompressGZIP(file_org_apache_beam_model_pipeline_v1_endpoints_proto_rawDescData) - }) - return file_org_apache_beam_model_pipeline_v1_endpoints_proto_rawDescData -} - var file_org_apache_beam_model_pipeline_v1_endpoints_proto_msgTypes = make([]protoimpl.MessageInfo, 2) -var file_org_apache_beam_model_pipeline_v1_endpoints_proto_goTypes = []interface{}{ +var file_org_apache_beam_model_pipeline_v1_endpoints_proto_goTypes = []any{ (*ApiServiceDescriptor)(nil), // 0: org.apache.beam.model.pipeline.v1.ApiServiceDescriptor (*AuthenticationSpec)(nil), // 1: org.apache.beam.model.pipeline.v1.AuthenticationSpec } @@ -224,32 +269,6 @@ func file_org_apache_beam_model_pipeline_v1_endpoints_proto_init() { if File_org_apache_beam_model_pipeline_v1_endpoints_proto != nil { return } - if !protoimpl.UnsafeEnabled { - file_org_apache_beam_model_pipeline_v1_endpoints_proto_msgTypes[0].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*ApiServiceDescriptor); i { - case 0: - return &v.state - case 1: - return &v.sizeCache - case 2: - return &v.unknownFields - default: - return nil - } - } - file_org_apache_beam_model_pipeline_v1_endpoints_proto_msgTypes[1].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*AuthenticationSpec); i { - case 0: - return &v.state - case 1: - return &v.sizeCache - case 2: - return &v.unknownFields - default: - return nil - } - } - } type x struct{} out := protoimpl.TypeBuilder{ File: protoimpl.DescBuilder{ diff --git a/sdks/go/pkg/beam/model/pipeline_v1/endpoints_protoopaque.pb.go b/sdks/go/pkg/beam/model/pipeline_v1/endpoints_protoopaque.pb.go new file mode 100644 index 000000000000..2bce6fe2009f --- /dev/null +++ b/sdks/go/pkg/beam/model/pipeline_v1/endpoints_protoopaque.pb.go @@ -0,0 +1,280 @@ +// +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +// +// Protocol Buffers describing endpoints containing a service. + +// Code generated by protoc-gen-go. DO NOT EDIT. +// versions: +// protoc-gen-go v1.36.0 +// protoc v5.27.3 +// source: org/apache/beam/model/pipeline/v1/endpoints.proto + +//go:build protoopaque + +package pipeline_v1 + +import ( + protoreflect "google.golang.org/protobuf/reflect/protoreflect" + protoimpl "google.golang.org/protobuf/runtime/protoimpl" + reflect "reflect" +) + +const ( + // Verify that this generated code is sufficiently up-to-date. + _ = protoimpl.EnforceVersion(20 - protoimpl.MinVersion) + // Verify that runtime/protoimpl is sufficiently up-to-date. + _ = protoimpl.EnforceVersion(protoimpl.MaxVersion - 20) +) + +// A description of how to connect to a Beam API endpoint. +type ApiServiceDescriptor struct { + state protoimpl.MessageState `protogen:"opaque.v1"` + xxx_hidden_Url string `protobuf:"bytes,1,opt,name=url,proto3" json:"url,omitempty"` + xxx_hidden_Authentication *AuthenticationSpec `protobuf:"bytes,2,opt,name=authentication,proto3" json:"authentication,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *ApiServiceDescriptor) Reset() { + *x = ApiServiceDescriptor{} + mi := &file_org_apache_beam_model_pipeline_v1_endpoints_proto_msgTypes[0] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *ApiServiceDescriptor) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*ApiServiceDescriptor) ProtoMessage() {} + +func (x *ApiServiceDescriptor) ProtoReflect() protoreflect.Message { + mi := &file_org_apache_beam_model_pipeline_v1_endpoints_proto_msgTypes[0] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +func (x *ApiServiceDescriptor) GetUrl() string { + if x != nil { + return x.xxx_hidden_Url + } + return "" +} + +func (x *ApiServiceDescriptor) GetAuthentication() *AuthenticationSpec { + if x != nil { + return x.xxx_hidden_Authentication + } + return nil +} + +func (x *ApiServiceDescriptor) SetUrl(v string) { + x.xxx_hidden_Url = v +} + +func (x *ApiServiceDescriptor) SetAuthentication(v *AuthenticationSpec) { + x.xxx_hidden_Authentication = v +} + +func (x *ApiServiceDescriptor) HasAuthentication() bool { + if x == nil { + return false + } + return x.xxx_hidden_Authentication != nil +} + +func (x *ApiServiceDescriptor) ClearAuthentication() { + x.xxx_hidden_Authentication = nil +} + +type ApiServiceDescriptor_builder struct { + _ [0]func() // Prevents comparability and use of unkeyed literals for the builder. + + // (Required) The URL to connect to. + Url string + // (Optional) The method for authentication. If unspecified, access to the + // url is already being performed in a trusted context (e.g. localhost, + // private network). + Authentication *AuthenticationSpec +} + +func (b0 ApiServiceDescriptor_builder) Build() *ApiServiceDescriptor { + m0 := &ApiServiceDescriptor{} + b, x := &b0, m0 + _, _ = b, x + x.xxx_hidden_Url = b.Url + x.xxx_hidden_Authentication = b.Authentication + return m0 +} + +type AuthenticationSpec struct { + state protoimpl.MessageState `protogen:"opaque.v1"` + xxx_hidden_Urn string `protobuf:"bytes,1,opt,name=urn,proto3" json:"urn,omitempty"` + xxx_hidden_Payload []byte `protobuf:"bytes,2,opt,name=payload,proto3" json:"payload,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *AuthenticationSpec) Reset() { + *x = AuthenticationSpec{} + mi := &file_org_apache_beam_model_pipeline_v1_endpoints_proto_msgTypes[1] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *AuthenticationSpec) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*AuthenticationSpec) ProtoMessage() {} + +func (x *AuthenticationSpec) ProtoReflect() protoreflect.Message { + mi := &file_org_apache_beam_model_pipeline_v1_endpoints_proto_msgTypes[1] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +func (x *AuthenticationSpec) GetUrn() string { + if x != nil { + return x.xxx_hidden_Urn + } + return "" +} + +func (x *AuthenticationSpec) GetPayload() []byte { + if x != nil { + return x.xxx_hidden_Payload + } + return nil +} + +func (x *AuthenticationSpec) SetUrn(v string) { + x.xxx_hidden_Urn = v +} + +func (x *AuthenticationSpec) SetPayload(v []byte) { + if v == nil { + v = []byte{} + } + x.xxx_hidden_Payload = v +} + +type AuthenticationSpec_builder struct { + _ [0]func() // Prevents comparability and use of unkeyed literals for the builder. + + // (Required) A URN that describes the accompanying payload. + // For any URN that is not recognized (by whomever is inspecting + // it) the parameter payload should be treated as opaque and + // passed as-is. + Urn string + // (Optional) The data specifying any parameters to the URN. If + // the URN does not require any arguments, this may be omitted. + Payload []byte +} + +func (b0 AuthenticationSpec_builder) Build() *AuthenticationSpec { + m0 := &AuthenticationSpec{} + b, x := &b0, m0 + _, _ = b, x + x.xxx_hidden_Urn = b.Urn + x.xxx_hidden_Payload = b.Payload + return m0 +} + +var File_org_apache_beam_model_pipeline_v1_endpoints_proto protoreflect.FileDescriptor + +var file_org_apache_beam_model_pipeline_v1_endpoints_proto_rawDesc = []byte{ + 0x0a, 0x31, 0x6f, 0x72, 0x67, 0x2f, 0x61, 0x70, 0x61, 0x63, 0x68, 0x65, 0x2f, 0x62, 0x65, 0x61, + 0x6d, 0x2f, 0x6d, 0x6f, 0x64, 0x65, 0x6c, 0x2f, 0x70, 0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e, 0x65, + 0x2f, 0x76, 0x31, 0x2f, 0x65, 0x6e, 0x64, 0x70, 0x6f, 0x69, 0x6e, 0x74, 0x73, 0x2e, 0x70, 0x72, + 0x6f, 0x74, 0x6f, 0x12, 0x21, 0x6f, 0x72, 0x67, 0x2e, 0x61, 0x70, 0x61, 0x63, 0x68, 0x65, 0x2e, + 0x62, 0x65, 0x61, 0x6d, 0x2e, 0x6d, 0x6f, 0x64, 0x65, 0x6c, 0x2e, 0x70, 0x69, 0x70, 0x65, 0x6c, + 0x69, 0x6e, 0x65, 0x2e, 0x76, 0x31, 0x22, 0x87, 0x01, 0x0a, 0x14, 0x41, 0x70, 0x69, 0x53, 0x65, + 0x72, 0x76, 0x69, 0x63, 0x65, 0x44, 0x65, 0x73, 0x63, 0x72, 0x69, 0x70, 0x74, 0x6f, 0x72, 0x12, + 0x10, 0x0a, 0x03, 0x75, 0x72, 0x6c, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x03, 0x75, 0x72, + 0x6c, 0x12, 0x5d, 0x0a, 0x0e, 0x61, 0x75, 0x74, 0x68, 0x65, 0x6e, 0x74, 0x69, 0x63, 0x61, 0x74, + 0x69, 0x6f, 0x6e, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x35, 0x2e, 0x6f, 0x72, 0x67, 0x2e, + 0x61, 0x70, 0x61, 0x63, 0x68, 0x65, 0x2e, 0x62, 0x65, 0x61, 0x6d, 0x2e, 0x6d, 0x6f, 0x64, 0x65, + 0x6c, 0x2e, 0x70, 0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x41, 0x75, + 0x74, 0x68, 0x65, 0x6e, 0x74, 0x69, 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x53, 0x70, 0x65, 0x63, + 0x52, 0x0e, 0x61, 0x75, 0x74, 0x68, 0x65, 0x6e, 0x74, 0x69, 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, + 0x22, 0x40, 0x0a, 0x12, 0x41, 0x75, 0x74, 0x68, 0x65, 0x6e, 0x74, 0x69, 0x63, 0x61, 0x74, 0x69, + 0x6f, 0x6e, 0x53, 0x70, 0x65, 0x63, 0x12, 0x10, 0x0a, 0x03, 0x75, 0x72, 0x6e, 0x18, 0x01, 0x20, + 0x01, 0x28, 0x09, 0x52, 0x03, 0x75, 0x72, 0x6e, 0x12, 0x18, 0x0a, 0x07, 0x70, 0x61, 0x79, 0x6c, + 0x6f, 0x61, 0x64, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0c, 0x52, 0x07, 0x70, 0x61, 0x79, 0x6c, 0x6f, + 0x61, 0x64, 0x42, 0x78, 0x0a, 0x21, 0x6f, 0x72, 0x67, 0x2e, 0x61, 0x70, 0x61, 0x63, 0x68, 0x65, + 0x2e, 0x62, 0x65, 0x61, 0x6d, 0x2e, 0x6d, 0x6f, 0x64, 0x65, 0x6c, 0x2e, 0x70, 0x69, 0x70, 0x65, + 0x6c, 0x69, 0x6e, 0x65, 0x2e, 0x76, 0x31, 0x42, 0x09, 0x45, 0x6e, 0x64, 0x70, 0x6f, 0x69, 0x6e, + 0x74, 0x73, 0x5a, 0x48, 0x67, 0x69, 0x74, 0x68, 0x75, 0x62, 0x2e, 0x63, 0x6f, 0x6d, 0x2f, 0x61, + 0x70, 0x61, 0x63, 0x68, 0x65, 0x2f, 0x62, 0x65, 0x61, 0x6d, 0x2f, 0x73, 0x64, 0x6b, 0x73, 0x2f, + 0x76, 0x32, 0x2f, 0x67, 0x6f, 0x2f, 0x70, 0x6b, 0x67, 0x2f, 0x62, 0x65, 0x61, 0x6d, 0x2f, 0x6d, + 0x6f, 0x64, 0x65, 0x6c, 0x2f, 0x70, 0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e, 0x65, 0x5f, 0x76, 0x31, + 0x3b, 0x70, 0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e, 0x65, 0x5f, 0x76, 0x31, 0x62, 0x06, 0x70, 0x72, + 0x6f, 0x74, 0x6f, 0x33, +} + +var file_org_apache_beam_model_pipeline_v1_endpoints_proto_msgTypes = make([]protoimpl.MessageInfo, 2) +var file_org_apache_beam_model_pipeline_v1_endpoints_proto_goTypes = []any{ + (*ApiServiceDescriptor)(nil), // 0: org.apache.beam.model.pipeline.v1.ApiServiceDescriptor + (*AuthenticationSpec)(nil), // 1: org.apache.beam.model.pipeline.v1.AuthenticationSpec +} +var file_org_apache_beam_model_pipeline_v1_endpoints_proto_depIdxs = []int32{ + 1, // 0: org.apache.beam.model.pipeline.v1.ApiServiceDescriptor.authentication:type_name -> org.apache.beam.model.pipeline.v1.AuthenticationSpec + 1, // [1:1] is the sub-list for method output_type + 1, // [1:1] is the sub-list for method input_type + 1, // [1:1] is the sub-list for extension type_name + 1, // [1:1] is the sub-list for extension extendee + 0, // [0:1] is the sub-list for field type_name +} + +func init() { file_org_apache_beam_model_pipeline_v1_endpoints_proto_init() } +func file_org_apache_beam_model_pipeline_v1_endpoints_proto_init() { + if File_org_apache_beam_model_pipeline_v1_endpoints_proto != nil { + return + } + type x struct{} + out := protoimpl.TypeBuilder{ + File: protoimpl.DescBuilder{ + GoPackagePath: reflect.TypeOf(x{}).PkgPath(), + RawDescriptor: file_org_apache_beam_model_pipeline_v1_endpoints_proto_rawDesc, + NumEnums: 0, + NumMessages: 2, + NumExtensions: 0, + NumServices: 0, + }, + GoTypes: file_org_apache_beam_model_pipeline_v1_endpoints_proto_goTypes, + DependencyIndexes: file_org_apache_beam_model_pipeline_v1_endpoints_proto_depIdxs, + MessageInfos: file_org_apache_beam_model_pipeline_v1_endpoints_proto_msgTypes, + }.Build() + File_org_apache_beam_model_pipeline_v1_endpoints_proto = out.File + file_org_apache_beam_model_pipeline_v1_endpoints_proto_rawDesc = nil + file_org_apache_beam_model_pipeline_v1_endpoints_proto_goTypes = nil + file_org_apache_beam_model_pipeline_v1_endpoints_proto_depIdxs = nil +} diff --git a/sdks/go/pkg/beam/model/pipeline_v1/external_transforms.pb.go b/sdks/go/pkg/beam/model/pipeline_v1/external_transforms.pb.go index 20945940d117..b03b60f2f099 100644 --- a/sdks/go/pkg/beam/model/pipeline_v1/external_transforms.pb.go +++ b/sdks/go/pkg/beam/model/pipeline_v1/external_transforms.pb.go @@ -20,17 +20,18 @@ // Code generated by protoc-gen-go. DO NOT EDIT. // versions: -// protoc-gen-go v1.27.1 +// protoc-gen-go v1.36.0 // protoc v5.27.3 // source: org/apache/beam/model/pipeline/v1/external_transforms.proto +//go:build !protoopaque + package pipeline_v1 import ( protoreflect "google.golang.org/protobuf/reflect/protoreflect" protoimpl "google.golang.org/protobuf/runtime/protoimpl" reflect "reflect" - sync "sync" ) const ( @@ -85,9 +86,57 @@ func (x ExpansionMethods_Enum) Number() protoreflect.EnumNumber { return protoreflect.EnumNumber(x) } -// Deprecated: Use ExpansionMethods_Enum.Descriptor instead. -func (ExpansionMethods_Enum) EnumDescriptor() ([]byte, []int) { - return file_org_apache_beam_model_pipeline_v1_external_transforms_proto_rawDescGZIP(), []int{1, 0} +type ManagedTransforms_Urns int32 + +const ( + ManagedTransforms_ICEBERG_READ ManagedTransforms_Urns = 0 + ManagedTransforms_ICEBERG_WRITE ManagedTransforms_Urns = 1 + ManagedTransforms_KAFKA_READ ManagedTransforms_Urns = 2 + ManagedTransforms_KAFKA_WRITE ManagedTransforms_Urns = 3 + ManagedTransforms_BIGQUERY_READ ManagedTransforms_Urns = 4 + ManagedTransforms_BIGQUERY_WRITE ManagedTransforms_Urns = 5 +) + +// Enum value maps for ManagedTransforms_Urns. +var ( + ManagedTransforms_Urns_name = map[int32]string{ + 0: "ICEBERG_READ", + 1: "ICEBERG_WRITE", + 2: "KAFKA_READ", + 3: "KAFKA_WRITE", + 4: "BIGQUERY_READ", + 5: "BIGQUERY_WRITE", + } + ManagedTransforms_Urns_value = map[string]int32{ + "ICEBERG_READ": 0, + "ICEBERG_WRITE": 1, + "KAFKA_READ": 2, + "KAFKA_WRITE": 3, + "BIGQUERY_READ": 4, + "BIGQUERY_WRITE": 5, + } +) + +func (x ManagedTransforms_Urns) Enum() *ManagedTransforms_Urns { + p := new(ManagedTransforms_Urns) + *p = x + return p +} + +func (x ManagedTransforms_Urns) String() string { + return protoimpl.X.EnumStringOf(x.Descriptor(), protoreflect.EnumNumber(x)) +} + +func (ManagedTransforms_Urns) Descriptor() protoreflect.EnumDescriptor { + return file_org_apache_beam_model_pipeline_v1_external_transforms_proto_enumTypes[1].Descriptor() +} + +func (ManagedTransforms_Urns) Type() protoreflect.EnumType { + return &file_org_apache_beam_model_pipeline_v1_external_transforms_proto_enumTypes[1] +} + +func (x ManagedTransforms_Urns) Number() protoreflect.EnumNumber { + return protoreflect.EnumNumber(x) } type Annotations_Enum int32 @@ -130,43 +179,35 @@ func (x Annotations_Enum) String() string { } func (Annotations_Enum) Descriptor() protoreflect.EnumDescriptor { - return file_org_apache_beam_model_pipeline_v1_external_transforms_proto_enumTypes[1].Descriptor() + return file_org_apache_beam_model_pipeline_v1_external_transforms_proto_enumTypes[2].Descriptor() } func (Annotations_Enum) Type() protoreflect.EnumType { - return &file_org_apache_beam_model_pipeline_v1_external_transforms_proto_enumTypes[1] + return &file_org_apache_beam_model_pipeline_v1_external_transforms_proto_enumTypes[2] } func (x Annotations_Enum) Number() protoreflect.EnumNumber { return protoreflect.EnumNumber(x) } -// Deprecated: Use Annotations_Enum.Descriptor instead. -func (Annotations_Enum) EnumDescriptor() ([]byte, []int) { - return file_org_apache_beam_model_pipeline_v1_external_transforms_proto_rawDescGZIP(), []int{4, 0} -} - // A configuration payload for an external transform. // Used as the payload of ExternalTransform as part of an ExpansionRequest. type ExternalConfigurationPayload struct { - state protoimpl.MessageState - sizeCache protoimpl.SizeCache - unknownFields protoimpl.UnknownFields - + state protoimpl.MessageState `protogen:"hybrid.v1"` // A schema for use in beam:coder:row:v1 Schema *Schema `protobuf:"bytes,1,opt,name=schema,proto3" json:"schema,omitempty"` // A payload which can be decoded using beam:coder:row:v1 and the given // schema. - Payload []byte `protobuf:"bytes,2,opt,name=payload,proto3" json:"payload,omitempty"` + Payload []byte `protobuf:"bytes,2,opt,name=payload,proto3" json:"payload,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache } func (x *ExternalConfigurationPayload) Reset() { *x = ExternalConfigurationPayload{} - if protoimpl.UnsafeEnabled { - mi := &file_org_apache_beam_model_pipeline_v1_external_transforms_proto_msgTypes[0] - ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) - ms.StoreMessageInfo(mi) - } + mi := &file_org_apache_beam_model_pipeline_v1_external_transforms_proto_msgTypes[0] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) } func (x *ExternalConfigurationPayload) String() string { @@ -177,7 +218,7 @@ func (*ExternalConfigurationPayload) ProtoMessage() {} func (x *ExternalConfigurationPayload) ProtoReflect() protoreflect.Message { mi := &file_org_apache_beam_model_pipeline_v1_external_transforms_proto_msgTypes[0] - if protoimpl.UnsafeEnabled && x != nil { + if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { ms.StoreMessageInfo(mi) @@ -187,11 +228,6 @@ func (x *ExternalConfigurationPayload) ProtoReflect() protoreflect.Message { return mi.MessageOf(x) } -// Deprecated: Use ExternalConfigurationPayload.ProtoReflect.Descriptor instead. -func (*ExternalConfigurationPayload) Descriptor() ([]byte, []int) { - return file_org_apache_beam_model_pipeline_v1_external_transforms_proto_rawDescGZIP(), []int{0} -} - func (x *ExternalConfigurationPayload) GetSchema() *Schema { if x != nil { return x.Schema @@ -206,22 +242,61 @@ func (x *ExternalConfigurationPayload) GetPayload() []byte { return nil } +func (x *ExternalConfigurationPayload) SetSchema(v *Schema) { + x.Schema = v +} + +func (x *ExternalConfigurationPayload) SetPayload(v []byte) { + if v == nil { + v = []byte{} + } + x.Payload = v +} + +func (x *ExternalConfigurationPayload) HasSchema() bool { + if x == nil { + return false + } + return x.Schema != nil +} + +func (x *ExternalConfigurationPayload) ClearSchema() { + x.Schema = nil +} + +type ExternalConfigurationPayload_builder struct { + _ [0]func() // Prevents comparability and use of unkeyed literals for the builder. + + // A schema for use in beam:coder:row:v1 + Schema *Schema + // A payload which can be decoded using beam:coder:row:v1 and the given + // schema. + Payload []byte +} + +func (b0 ExternalConfigurationPayload_builder) Build() *ExternalConfigurationPayload { + m0 := &ExternalConfigurationPayload{} + b, x := &b0, m0 + _, _ = b, x + x.Schema = b.Schema + x.Payload = b.Payload + return m0 +} + // Defines specific expansion methods that may be used to expand cross-language // transforms. // Has to be set as the URN of the transform of the expansion request. type ExpansionMethods struct { - state protoimpl.MessageState - sizeCache protoimpl.SizeCache + state protoimpl.MessageState `protogen:"hybrid.v1"` unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache } func (x *ExpansionMethods) Reset() { *x = ExpansionMethods{} - if protoimpl.UnsafeEnabled { - mi := &file_org_apache_beam_model_pipeline_v1_external_transforms_proto_msgTypes[1] - ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) - ms.StoreMessageInfo(mi) - } + mi := &file_org_apache_beam_model_pipeline_v1_external_transforms_proto_msgTypes[1] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) } func (x *ExpansionMethods) String() string { @@ -232,7 +307,51 @@ func (*ExpansionMethods) ProtoMessage() {} func (x *ExpansionMethods) ProtoReflect() protoreflect.Message { mi := &file_org_apache_beam_model_pipeline_v1_external_transforms_proto_msgTypes[1] - if protoimpl.UnsafeEnabled && x != nil { + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +type ExpansionMethods_builder struct { + _ [0]func() // Prevents comparability and use of unkeyed literals for the builder. + +} + +func (b0 ExpansionMethods_builder) Build() *ExpansionMethods { + m0 := &ExpansionMethods{} + b, x := &b0, m0 + _, _ = b, x + return m0 +} + +// Defines the URNs for managed transforms. +type ManagedTransforms struct { + state protoimpl.MessageState `protogen:"hybrid.v1"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *ManagedTransforms) Reset() { + *x = ManagedTransforms{} + mi := &file_org_apache_beam_model_pipeline_v1_external_transforms_proto_msgTypes[2] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *ManagedTransforms) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*ManagedTransforms) ProtoMessage() {} + +func (x *ManagedTransforms) ProtoReflect() protoreflect.Message { + mi := &file_org_apache_beam_model_pipeline_v1_external_transforms_proto_msgTypes[2] + if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { ms.StoreMessageInfo(mi) @@ -242,19 +361,23 @@ func (x *ExpansionMethods) ProtoReflect() protoreflect.Message { return mi.MessageOf(x) } -// Deprecated: Use ExpansionMethods.ProtoReflect.Descriptor instead. -func (*ExpansionMethods) Descriptor() ([]byte, []int) { - return file_org_apache_beam_model_pipeline_v1_external_transforms_proto_rawDescGZIP(), []int{1} +type ManagedTransforms_builder struct { + _ [0]func() // Prevents comparability and use of unkeyed literals for the builder. + +} + +func (b0 ManagedTransforms_builder) Build() *ManagedTransforms { + m0 := &ManagedTransforms{} + b, x := &b0, m0 + _, _ = b, x + return m0 } // A configuration payload for an external transform. // Used to define a Java transform that can be directly instantiated by a Java // expansion service. type JavaClassLookupPayload struct { - state protoimpl.MessageState - sizeCache protoimpl.SizeCache - unknownFields protoimpl.UnknownFields - + state protoimpl.MessageState `protogen:"hybrid.v1"` // Name of the Java transform class. ClassName string `protobuf:"bytes,1,opt,name=class_name,json=className,proto3" json:"class_name,omitempty"` // A static method to construct the initial instance of the transform. @@ -276,15 +399,15 @@ type JavaClassLookupPayload struct { // When constructing the transform object, given builder methods will be // applied in order. BuilderMethods []*BuilderMethod `protobuf:"bytes,5,rep,name=builder_methods,json=builderMethods,proto3" json:"builder_methods,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache } func (x *JavaClassLookupPayload) Reset() { *x = JavaClassLookupPayload{} - if protoimpl.UnsafeEnabled { - mi := &file_org_apache_beam_model_pipeline_v1_external_transforms_proto_msgTypes[2] - ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) - ms.StoreMessageInfo(mi) - } + mi := &file_org_apache_beam_model_pipeline_v1_external_transforms_proto_msgTypes[3] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) } func (x *JavaClassLookupPayload) String() string { @@ -294,8 +417,8 @@ func (x *JavaClassLookupPayload) String() string { func (*JavaClassLookupPayload) ProtoMessage() {} func (x *JavaClassLookupPayload) ProtoReflect() protoreflect.Message { - mi := &file_org_apache_beam_model_pipeline_v1_external_transforms_proto_msgTypes[2] - if protoimpl.UnsafeEnabled && x != nil { + mi := &file_org_apache_beam_model_pipeline_v1_external_transforms_proto_msgTypes[3] + if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { ms.StoreMessageInfo(mi) @@ -305,11 +428,6 @@ func (x *JavaClassLookupPayload) ProtoReflect() protoreflect.Message { return mi.MessageOf(x) } -// Deprecated: Use JavaClassLookupPayload.ProtoReflect.Descriptor instead. -func (*JavaClassLookupPayload) Descriptor() ([]byte, []int) { - return file_org_apache_beam_model_pipeline_v1_external_transforms_proto_rawDescGZIP(), []int{2} -} - func (x *JavaClassLookupPayload) GetClassName() string { if x != nil { return x.ClassName @@ -345,15 +463,84 @@ func (x *JavaClassLookupPayload) GetBuilderMethods() []*BuilderMethod { return nil } +func (x *JavaClassLookupPayload) SetClassName(v string) { + x.ClassName = v +} + +func (x *JavaClassLookupPayload) SetConstructorMethod(v string) { + x.ConstructorMethod = v +} + +func (x *JavaClassLookupPayload) SetConstructorSchema(v *Schema) { + x.ConstructorSchema = v +} + +func (x *JavaClassLookupPayload) SetConstructorPayload(v []byte) { + if v == nil { + v = []byte{} + } + x.ConstructorPayload = v +} + +func (x *JavaClassLookupPayload) SetBuilderMethods(v []*BuilderMethod) { + x.BuilderMethods = v +} + +func (x *JavaClassLookupPayload) HasConstructorSchema() bool { + if x == nil { + return false + } + return x.ConstructorSchema != nil +} + +func (x *JavaClassLookupPayload) ClearConstructorSchema() { + x.ConstructorSchema = nil +} + +type JavaClassLookupPayload_builder struct { + _ [0]func() // Prevents comparability and use of unkeyed literals for the builder. + + // Name of the Java transform class. + ClassName string + // A static method to construct the initial instance of the transform. + // If not provided, the transform should be instantiated using a class + // constructor. + ConstructorMethod string + // The top level fields of the schema represent the method parameters in + // order. + // If able, top level field names are also verified against the method + // parameters for a match. + // Any field names in the form 'ignore[0-9]+' will not be used for validation + // hence that format can be used to represent arbitrary field names. + ConstructorSchema *Schema + // A payload which can be decoded using beam:coder:row:v1 and the provided + // constructor schema. + ConstructorPayload []byte + // Set of builder methods and corresponding parameters to apply after the + // transform object is constructed. + // When constructing the transform object, given builder methods will be + // applied in order. + BuilderMethods []*BuilderMethod +} + +func (b0 JavaClassLookupPayload_builder) Build() *JavaClassLookupPayload { + m0 := &JavaClassLookupPayload{} + b, x := &b0, m0 + _, _ = b, x + x.ClassName = b.ClassName + x.ConstructorMethod = b.ConstructorMethod + x.ConstructorSchema = b.ConstructorSchema + x.ConstructorPayload = b.ConstructorPayload + x.BuilderMethods = b.BuilderMethods + return m0 +} + // This represents a builder method of the transform class that should be // applied in-order after instantiating the initial transform object. // Each builder method may take one or more parameters and has to return an // instance of the transform object. type BuilderMethod struct { - state protoimpl.MessageState - sizeCache protoimpl.SizeCache - unknownFields protoimpl.UnknownFields - + state protoimpl.MessageState `protogen:"hybrid.v1"` // Name of the builder method Name string `protobuf:"bytes,1,opt,name=name,proto3" json:"name,omitempty"` // The top level fields of the schema represent the method parameters in @@ -365,16 +552,16 @@ type BuilderMethod struct { Schema *Schema `protobuf:"bytes,2,opt,name=schema,proto3" json:"schema,omitempty"` // A payload which can be decoded using beam:coder:row:v1 and the builder // method schema. - Payload []byte `protobuf:"bytes,3,opt,name=payload,proto3" json:"payload,omitempty"` + Payload []byte `protobuf:"bytes,3,opt,name=payload,proto3" json:"payload,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache } func (x *BuilderMethod) Reset() { *x = BuilderMethod{} - if protoimpl.UnsafeEnabled { - mi := &file_org_apache_beam_model_pipeline_v1_external_transforms_proto_msgTypes[3] - ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) - ms.StoreMessageInfo(mi) - } + mi := &file_org_apache_beam_model_pipeline_v1_external_transforms_proto_msgTypes[4] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) } func (x *BuilderMethod) String() string { @@ -384,8 +571,8 @@ func (x *BuilderMethod) String() string { func (*BuilderMethod) ProtoMessage() {} func (x *BuilderMethod) ProtoReflect() protoreflect.Message { - mi := &file_org_apache_beam_model_pipeline_v1_external_transforms_proto_msgTypes[3] - if protoimpl.UnsafeEnabled && x != nil { + mi := &file_org_apache_beam_model_pipeline_v1_external_transforms_proto_msgTypes[4] + if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { ms.StoreMessageInfo(mi) @@ -395,11 +582,6 @@ func (x *BuilderMethod) ProtoReflect() protoreflect.Message { return mi.MessageOf(x) } -// Deprecated: Use BuilderMethod.ProtoReflect.Descriptor instead. -func (*BuilderMethod) Descriptor() ([]byte, []int) { - return file_org_apache_beam_model_pipeline_v1_external_transforms_proto_rawDescGZIP(), []int{3} -} - func (x *BuilderMethod) GetName() string { if x != nil { return x.Name @@ -421,19 +603,70 @@ func (x *BuilderMethod) GetPayload() []byte { return nil } +func (x *BuilderMethod) SetName(v string) { + x.Name = v +} + +func (x *BuilderMethod) SetSchema(v *Schema) { + x.Schema = v +} + +func (x *BuilderMethod) SetPayload(v []byte) { + if v == nil { + v = []byte{} + } + x.Payload = v +} + +func (x *BuilderMethod) HasSchema() bool { + if x == nil { + return false + } + return x.Schema != nil +} + +func (x *BuilderMethod) ClearSchema() { + x.Schema = nil +} + +type BuilderMethod_builder struct { + _ [0]func() // Prevents comparability and use of unkeyed literals for the builder. + + // Name of the builder method + Name string + // The top level fields of the schema represent the method parameters in + // order. + // If able, top level field names are also verified against the method + // parameters for a match. + // Any field names in the form 'ignore[0-9]+' will not be used for validation + // hence that format can be used to represent arbitrary field names. + Schema *Schema + // A payload which can be decoded using beam:coder:row:v1 and the builder + // method schema. + Payload []byte +} + +func (b0 BuilderMethod_builder) Build() *BuilderMethod { + m0 := &BuilderMethod{} + b, x := &b0, m0 + _, _ = b, x + x.Name = b.Name + x.Schema = b.Schema + x.Payload = b.Payload + return m0 +} + type Annotations struct { - state protoimpl.MessageState - sizeCache protoimpl.SizeCache + state protoimpl.MessageState `protogen:"hybrid.v1"` unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache } func (x *Annotations) Reset() { *x = Annotations{} - if protoimpl.UnsafeEnabled { - mi := &file_org_apache_beam_model_pipeline_v1_external_transforms_proto_msgTypes[4] - ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) - ms.StoreMessageInfo(mi) - } + mi := &file_org_apache_beam_model_pipeline_v1_external_transforms_proto_msgTypes[5] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) } func (x *Annotations) String() string { @@ -443,8 +676,8 @@ func (x *Annotations) String() string { func (*Annotations) ProtoMessage() {} func (x *Annotations) ProtoReflect() protoreflect.Message { - mi := &file_org_apache_beam_model_pipeline_v1_external_transforms_proto_msgTypes[4] - if protoimpl.UnsafeEnabled && x != nil { + mi := &file_org_apache_beam_model_pipeline_v1_external_transforms_proto_msgTypes[5] + if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { ms.StoreMessageInfo(mi) @@ -454,9 +687,16 @@ func (x *Annotations) ProtoReflect() protoreflect.Message { return mi.MessageOf(x) } -// Deprecated: Use Annotations.ProtoReflect.Descriptor instead. -func (*Annotations) Descriptor() ([]byte, []int) { - return file_org_apache_beam_model_pipeline_v1_external_transforms_proto_rawDescGZIP(), []int{4} +type Annotations_builder struct { + _ [0]func() // Prevents comparability and use of unkeyed literals for the builder. + +} + +func (b0 Annotations_builder) Build() *Annotations { + m0 := &Annotations{} + b, x := &b0, m0 + _, _ = b, x + return m0 } // Payload for a Schema-aware PTransform. @@ -464,10 +704,7 @@ func (*Annotations) Descriptor() ([]byte, []int) { // and is configured using Beam Schema-compatible parameters. // The information available in the payload can be used to instantiate the schema-aware transform. type SchemaTransformPayload struct { - state protoimpl.MessageState - sizeCache protoimpl.SizeCache - unknownFields protoimpl.UnknownFields - + state protoimpl.MessageState `protogen:"hybrid.v1"` // The identifier of the SchemaTransform (typically a URN). Identifier string `protobuf:"bytes,1,opt,name=identifier,proto3" json:"identifier,omitempty"` // The configuration schema of the SchemaTransform. @@ -477,15 +714,15 @@ type SchemaTransformPayload struct { // The schema of the Row should be compatible with the schema of the // SchemaTransform. ConfigurationRow []byte `protobuf:"bytes,3,opt,name=configuration_row,json=configurationRow,proto3" json:"configuration_row,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache } func (x *SchemaTransformPayload) Reset() { *x = SchemaTransformPayload{} - if protoimpl.UnsafeEnabled { - mi := &file_org_apache_beam_model_pipeline_v1_external_transforms_proto_msgTypes[5] - ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) - ms.StoreMessageInfo(mi) - } + mi := &file_org_apache_beam_model_pipeline_v1_external_transforms_proto_msgTypes[6] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) } func (x *SchemaTransformPayload) String() string { @@ -495,8 +732,8 @@ func (x *SchemaTransformPayload) String() string { func (*SchemaTransformPayload) ProtoMessage() {} func (x *SchemaTransformPayload) ProtoReflect() protoreflect.Message { - mi := &file_org_apache_beam_model_pipeline_v1_external_transforms_proto_msgTypes[5] - if protoimpl.UnsafeEnabled && x != nil { + mi := &file_org_apache_beam_model_pipeline_v1_external_transforms_proto_msgTypes[6] + if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { ms.StoreMessageInfo(mi) @@ -506,11 +743,6 @@ func (x *SchemaTransformPayload) ProtoReflect() protoreflect.Message { return mi.MessageOf(x) } -// Deprecated: Use SchemaTransformPayload.ProtoReflect.Descriptor instead. -func (*SchemaTransformPayload) Descriptor() ([]byte, []int) { - return file_org_apache_beam_model_pipeline_v1_external_transforms_proto_rawDescGZIP(), []int{5} -} - func (x *SchemaTransformPayload) GetIdentifier() string { if x != nil { return x.Identifier @@ -532,6 +764,56 @@ func (x *SchemaTransformPayload) GetConfigurationRow() []byte { return nil } +func (x *SchemaTransformPayload) SetIdentifier(v string) { + x.Identifier = v +} + +func (x *SchemaTransformPayload) SetConfigurationSchema(v *Schema) { + x.ConfigurationSchema = v +} + +func (x *SchemaTransformPayload) SetConfigurationRow(v []byte) { + if v == nil { + v = []byte{} + } + x.ConfigurationRow = v +} + +func (x *SchemaTransformPayload) HasConfigurationSchema() bool { + if x == nil { + return false + } + return x.ConfigurationSchema != nil +} + +func (x *SchemaTransformPayload) ClearConfigurationSchema() { + x.ConfigurationSchema = nil +} + +type SchemaTransformPayload_builder struct { + _ [0]func() // Prevents comparability and use of unkeyed literals for the builder. + + // The identifier of the SchemaTransform (typically a URN). + Identifier string + // The configuration schema of the SchemaTransform. + ConfigurationSchema *Schema + // The configuration of the SchemaTransform. + // Should be decodable via beam:coder:row:v1. + // The schema of the Row should be compatible with the schema of the + // SchemaTransform. + ConfigurationRow []byte +} + +func (b0 SchemaTransformPayload_builder) Build() *SchemaTransformPayload { + m0 := &SchemaTransformPayload{} + b, x := &b0, m0 + _, _ = b, x + x.Identifier = b.Identifier + x.ConfigurationSchema = b.ConfigurationSchema + x.ConfigurationRow = b.ConfigurationRow + return m0 +} + var File_org_apache_beam_model_pipeline_v1_external_transforms_proto protoreflect.FileDescriptor var file_org_apache_beam_model_pipeline_v1_external_transforms_proto_rawDesc = []byte{ @@ -566,111 +848,133 @@ var file_org_apache_beam_model_pipeline_v1_external_transforms_proto_rawDesc = [ 0x52, 0x4d, 0x10, 0x01, 0x1a, 0x2f, 0xa2, 0xb4, 0xfa, 0xc2, 0x05, 0x29, 0x62, 0x65, 0x61, 0x6d, 0x3a, 0x65, 0x78, 0x70, 0x61, 0x6e, 0x73, 0x69, 0x6f, 0x6e, 0x3a, 0x70, 0x61, 0x79, 0x6c, 0x6f, 0x61, 0x64, 0x3a, 0x73, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x74, 0x72, 0x61, 0x6e, 0x73, 0x66, 0x6f, - 0x72, 0x6d, 0x3a, 0x76, 0x31, 0x22, 0xcc, 0x02, 0x0a, 0x16, 0x4a, 0x61, 0x76, 0x61, 0x43, 0x6c, - 0x61, 0x73, 0x73, 0x4c, 0x6f, 0x6f, 0x6b, 0x75, 0x70, 0x50, 0x61, 0x79, 0x6c, 0x6f, 0x61, 0x64, - 0x12, 0x1d, 0x0a, 0x0a, 0x63, 0x6c, 0x61, 0x73, 0x73, 0x5f, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x01, - 0x20, 0x01, 0x28, 0x09, 0x52, 0x09, 0x63, 0x6c, 0x61, 0x73, 0x73, 0x4e, 0x61, 0x6d, 0x65, 0x12, - 0x2d, 0x0a, 0x12, 0x63, 0x6f, 0x6e, 0x73, 0x74, 0x72, 0x75, 0x63, 0x74, 0x6f, 0x72, 0x5f, 0x6d, - 0x65, 0x74, 0x68, 0x6f, 0x64, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x11, 0x63, 0x6f, 0x6e, - 0x73, 0x74, 0x72, 0x75, 0x63, 0x74, 0x6f, 0x72, 0x4d, 0x65, 0x74, 0x68, 0x6f, 0x64, 0x12, 0x58, - 0x0a, 0x12, 0x63, 0x6f, 0x6e, 0x73, 0x74, 0x72, 0x75, 0x63, 0x74, 0x6f, 0x72, 0x5f, 0x73, 0x63, - 0x68, 0x65, 0x6d, 0x61, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x29, 0x2e, 0x6f, 0x72, 0x67, - 0x2e, 0x61, 0x70, 0x61, 0x63, 0x68, 0x65, 0x2e, 0x62, 0x65, 0x61, 0x6d, 0x2e, 0x6d, 0x6f, 0x64, - 0x65, 0x6c, 0x2e, 0x70, 0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x53, - 0x63, 0x68, 0x65, 0x6d, 0x61, 0x52, 0x11, 0x63, 0x6f, 0x6e, 0x73, 0x74, 0x72, 0x75, 0x63, 0x74, - 0x6f, 0x72, 0x53, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x12, 0x2f, 0x0a, 0x13, 0x63, 0x6f, 0x6e, 0x73, - 0x74, 0x72, 0x75, 0x63, 0x74, 0x6f, 0x72, 0x5f, 0x70, 0x61, 0x79, 0x6c, 0x6f, 0x61, 0x64, 0x18, - 0x04, 0x20, 0x01, 0x28, 0x0c, 0x52, 0x12, 0x63, 0x6f, 0x6e, 0x73, 0x74, 0x72, 0x75, 0x63, 0x74, - 0x6f, 0x72, 0x50, 0x61, 0x79, 0x6c, 0x6f, 0x61, 0x64, 0x12, 0x59, 0x0a, 0x0f, 0x62, 0x75, 0x69, - 0x6c, 0x64, 0x65, 0x72, 0x5f, 0x6d, 0x65, 0x74, 0x68, 0x6f, 0x64, 0x73, 0x18, 0x05, 0x20, 0x03, - 0x28, 0x0b, 0x32, 0x30, 0x2e, 0x6f, 0x72, 0x67, 0x2e, 0x61, 0x70, 0x61, 0x63, 0x68, 0x65, 0x2e, - 0x62, 0x65, 0x61, 0x6d, 0x2e, 0x6d, 0x6f, 0x64, 0x65, 0x6c, 0x2e, 0x70, 0x69, 0x70, 0x65, 0x6c, - 0x69, 0x6e, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x42, 0x75, 0x69, 0x6c, 0x64, 0x65, 0x72, 0x4d, 0x65, - 0x74, 0x68, 0x6f, 0x64, 0x52, 0x0e, 0x62, 0x75, 0x69, 0x6c, 0x64, 0x65, 0x72, 0x4d, 0x65, 0x74, - 0x68, 0x6f, 0x64, 0x73, 0x22, 0x80, 0x01, 0x0a, 0x0d, 0x42, 0x75, 0x69, 0x6c, 0x64, 0x65, 0x72, - 0x4d, 0x65, 0x74, 0x68, 0x6f, 0x64, 0x12, 0x12, 0x0a, 0x04, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x01, - 0x20, 0x01, 0x28, 0x09, 0x52, 0x04, 0x6e, 0x61, 0x6d, 0x65, 0x12, 0x41, 0x0a, 0x06, 0x73, 0x63, + 0x72, 0x6d, 0x3a, 0x76, 0x31, 0x22, 0xfa, 0x03, 0x0a, 0x11, 0x4d, 0x61, 0x6e, 0x61, 0x67, 0x65, + 0x64, 0x54, 0x72, 0x61, 0x6e, 0x73, 0x66, 0x6f, 0x72, 0x6d, 0x73, 0x22, 0xe4, 0x03, 0x0a, 0x04, + 0x55, 0x72, 0x6e, 0x73, 0x12, 0x4c, 0x0a, 0x0c, 0x49, 0x43, 0x45, 0x42, 0x45, 0x52, 0x47, 0x5f, + 0x52, 0x45, 0x41, 0x44, 0x10, 0x00, 0x1a, 0x3a, 0xa2, 0xb4, 0xfa, 0xc2, 0x05, 0x34, 0x62, 0x65, + 0x61, 0x6d, 0x3a, 0x73, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x74, 0x72, 0x61, 0x6e, 0x73, 0x66, 0x6f, + 0x72, 0x6d, 0x3a, 0x6f, 0x72, 0x67, 0x2e, 0x61, 0x70, 0x61, 0x63, 0x68, 0x65, 0x2e, 0x62, 0x65, + 0x61, 0x6d, 0x3a, 0x69, 0x63, 0x65, 0x62, 0x65, 0x72, 0x67, 0x5f, 0x72, 0x65, 0x61, 0x64, 0x3a, + 0x76, 0x31, 0x12, 0x4e, 0x0a, 0x0d, 0x49, 0x43, 0x45, 0x42, 0x45, 0x52, 0x47, 0x5f, 0x57, 0x52, + 0x49, 0x54, 0x45, 0x10, 0x01, 0x1a, 0x3b, 0xa2, 0xb4, 0xfa, 0xc2, 0x05, 0x35, 0x62, 0x65, 0x61, + 0x6d, 0x3a, 0x73, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x74, 0x72, 0x61, 0x6e, 0x73, 0x66, 0x6f, 0x72, + 0x6d, 0x3a, 0x6f, 0x72, 0x67, 0x2e, 0x61, 0x70, 0x61, 0x63, 0x68, 0x65, 0x2e, 0x62, 0x65, 0x61, + 0x6d, 0x3a, 0x69, 0x63, 0x65, 0x62, 0x65, 0x72, 0x67, 0x5f, 0x77, 0x72, 0x69, 0x74, 0x65, 0x3a, + 0x76, 0x31, 0x12, 0x48, 0x0a, 0x0a, 0x4b, 0x41, 0x46, 0x4b, 0x41, 0x5f, 0x52, 0x45, 0x41, 0x44, + 0x10, 0x02, 0x1a, 0x38, 0xa2, 0xb4, 0xfa, 0xc2, 0x05, 0x32, 0x62, 0x65, 0x61, 0x6d, 0x3a, 0x73, + 0x63, 0x68, 0x65, 0x6d, 0x61, 0x74, 0x72, 0x61, 0x6e, 0x73, 0x66, 0x6f, 0x72, 0x6d, 0x3a, 0x6f, + 0x72, 0x67, 0x2e, 0x61, 0x70, 0x61, 0x63, 0x68, 0x65, 0x2e, 0x62, 0x65, 0x61, 0x6d, 0x3a, 0x6b, + 0x61, 0x66, 0x6b, 0x61, 0x5f, 0x72, 0x65, 0x61, 0x64, 0x3a, 0x76, 0x31, 0x12, 0x4a, 0x0a, 0x0b, + 0x4b, 0x41, 0x46, 0x4b, 0x41, 0x5f, 0x57, 0x52, 0x49, 0x54, 0x45, 0x10, 0x03, 0x1a, 0x39, 0xa2, + 0xb4, 0xfa, 0xc2, 0x05, 0x33, 0x62, 0x65, 0x61, 0x6d, 0x3a, 0x73, 0x63, 0x68, 0x65, 0x6d, 0x61, + 0x74, 0x72, 0x61, 0x6e, 0x73, 0x66, 0x6f, 0x72, 0x6d, 0x3a, 0x6f, 0x72, 0x67, 0x2e, 0x61, 0x70, + 0x61, 0x63, 0x68, 0x65, 0x2e, 0x62, 0x65, 0x61, 0x6d, 0x3a, 0x6b, 0x61, 0x66, 0x6b, 0x61, 0x5f, + 0x77, 0x72, 0x69, 0x74, 0x65, 0x3a, 0x76, 0x31, 0x12, 0x56, 0x0a, 0x0d, 0x42, 0x49, 0x47, 0x51, + 0x55, 0x45, 0x52, 0x59, 0x5f, 0x52, 0x45, 0x41, 0x44, 0x10, 0x04, 0x1a, 0x43, 0xa2, 0xb4, 0xfa, + 0xc2, 0x05, 0x3d, 0x62, 0x65, 0x61, 0x6d, 0x3a, 0x73, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x74, 0x72, + 0x61, 0x6e, 0x73, 0x66, 0x6f, 0x72, 0x6d, 0x3a, 0x6f, 0x72, 0x67, 0x2e, 0x61, 0x70, 0x61, 0x63, + 0x68, 0x65, 0x2e, 0x62, 0x65, 0x61, 0x6d, 0x3a, 0x62, 0x69, 0x67, 0x71, 0x75, 0x65, 0x72, 0x79, + 0x5f, 0x73, 0x74, 0x6f, 0x72, 0x61, 0x67, 0x65, 0x5f, 0x72, 0x65, 0x61, 0x64, 0x3a, 0x76, 0x31, + 0x12, 0x50, 0x0a, 0x0e, 0x42, 0x49, 0x47, 0x51, 0x55, 0x45, 0x52, 0x59, 0x5f, 0x57, 0x52, 0x49, + 0x54, 0x45, 0x10, 0x05, 0x1a, 0x3c, 0xa2, 0xb4, 0xfa, 0xc2, 0x05, 0x36, 0x62, 0x65, 0x61, 0x6d, + 0x3a, 0x73, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x74, 0x72, 0x61, 0x6e, 0x73, 0x66, 0x6f, 0x72, 0x6d, + 0x3a, 0x6f, 0x72, 0x67, 0x2e, 0x61, 0x70, 0x61, 0x63, 0x68, 0x65, 0x2e, 0x62, 0x65, 0x61, 0x6d, + 0x3a, 0x62, 0x69, 0x67, 0x71, 0x75, 0x65, 0x72, 0x79, 0x5f, 0x77, 0x72, 0x69, 0x74, 0x65, 0x3a, + 0x76, 0x31, 0x22, 0xcc, 0x02, 0x0a, 0x16, 0x4a, 0x61, 0x76, 0x61, 0x43, 0x6c, 0x61, 0x73, 0x73, + 0x4c, 0x6f, 0x6f, 0x6b, 0x75, 0x70, 0x50, 0x61, 0x79, 0x6c, 0x6f, 0x61, 0x64, 0x12, 0x1d, 0x0a, + 0x0a, 0x63, 0x6c, 0x61, 0x73, 0x73, 0x5f, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, + 0x09, 0x52, 0x09, 0x63, 0x6c, 0x61, 0x73, 0x73, 0x4e, 0x61, 0x6d, 0x65, 0x12, 0x2d, 0x0a, 0x12, + 0x63, 0x6f, 0x6e, 0x73, 0x74, 0x72, 0x75, 0x63, 0x74, 0x6f, 0x72, 0x5f, 0x6d, 0x65, 0x74, 0x68, + 0x6f, 0x64, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x11, 0x63, 0x6f, 0x6e, 0x73, 0x74, 0x72, + 0x75, 0x63, 0x74, 0x6f, 0x72, 0x4d, 0x65, 0x74, 0x68, 0x6f, 0x64, 0x12, 0x58, 0x0a, 0x12, 0x63, + 0x6f, 0x6e, 0x73, 0x74, 0x72, 0x75, 0x63, 0x74, 0x6f, 0x72, 0x5f, 0x73, 0x63, 0x68, 0x65, 0x6d, + 0x61, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x29, 0x2e, 0x6f, 0x72, 0x67, 0x2e, 0x61, 0x70, + 0x61, 0x63, 0x68, 0x65, 0x2e, 0x62, 0x65, 0x61, 0x6d, 0x2e, 0x6d, 0x6f, 0x64, 0x65, 0x6c, 0x2e, + 0x70, 0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x53, 0x63, 0x68, 0x65, + 0x6d, 0x61, 0x52, 0x11, 0x63, 0x6f, 0x6e, 0x73, 0x74, 0x72, 0x75, 0x63, 0x74, 0x6f, 0x72, 0x53, + 0x63, 0x68, 0x65, 0x6d, 0x61, 0x12, 0x2f, 0x0a, 0x13, 0x63, 0x6f, 0x6e, 0x73, 0x74, 0x72, 0x75, + 0x63, 0x74, 0x6f, 0x72, 0x5f, 0x70, 0x61, 0x79, 0x6c, 0x6f, 0x61, 0x64, 0x18, 0x04, 0x20, 0x01, + 0x28, 0x0c, 0x52, 0x12, 0x63, 0x6f, 0x6e, 0x73, 0x74, 0x72, 0x75, 0x63, 0x74, 0x6f, 0x72, 0x50, + 0x61, 0x79, 0x6c, 0x6f, 0x61, 0x64, 0x12, 0x59, 0x0a, 0x0f, 0x62, 0x75, 0x69, 0x6c, 0x64, 0x65, + 0x72, 0x5f, 0x6d, 0x65, 0x74, 0x68, 0x6f, 0x64, 0x73, 0x18, 0x05, 0x20, 0x03, 0x28, 0x0b, 0x32, + 0x30, 0x2e, 0x6f, 0x72, 0x67, 0x2e, 0x61, 0x70, 0x61, 0x63, 0x68, 0x65, 0x2e, 0x62, 0x65, 0x61, + 0x6d, 0x2e, 0x6d, 0x6f, 0x64, 0x65, 0x6c, 0x2e, 0x70, 0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e, 0x65, + 0x2e, 0x76, 0x31, 0x2e, 0x42, 0x75, 0x69, 0x6c, 0x64, 0x65, 0x72, 0x4d, 0x65, 0x74, 0x68, 0x6f, + 0x64, 0x52, 0x0e, 0x62, 0x75, 0x69, 0x6c, 0x64, 0x65, 0x72, 0x4d, 0x65, 0x74, 0x68, 0x6f, 0x64, + 0x73, 0x22, 0x80, 0x01, 0x0a, 0x0d, 0x42, 0x75, 0x69, 0x6c, 0x64, 0x65, 0x72, 0x4d, 0x65, 0x74, + 0x68, 0x6f, 0x64, 0x12, 0x12, 0x0a, 0x04, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, + 0x09, 0x52, 0x04, 0x6e, 0x61, 0x6d, 0x65, 0x12, 0x41, 0x0a, 0x06, 0x73, 0x63, 0x68, 0x65, 0x6d, + 0x61, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x29, 0x2e, 0x6f, 0x72, 0x67, 0x2e, 0x61, 0x70, + 0x61, 0x63, 0x68, 0x65, 0x2e, 0x62, 0x65, 0x61, 0x6d, 0x2e, 0x6d, 0x6f, 0x64, 0x65, 0x6c, 0x2e, + 0x70, 0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x53, 0x63, 0x68, 0x65, + 0x6d, 0x61, 0x52, 0x06, 0x73, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x12, 0x18, 0x0a, 0x07, 0x70, 0x61, + 0x79, 0x6c, 0x6f, 0x61, 0x64, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0c, 0x52, 0x07, 0x70, 0x61, 0x79, + 0x6c, 0x6f, 0x61, 0x64, 0x22, 0xfa, 0x01, 0x0a, 0x0b, 0x41, 0x6e, 0x6e, 0x6f, 0x74, 0x61, 0x74, + 0x69, 0x6f, 0x6e, 0x73, 0x22, 0xea, 0x01, 0x0a, 0x04, 0x45, 0x6e, 0x75, 0x6d, 0x12, 0x24, 0x0a, + 0x0e, 0x43, 0x4f, 0x4e, 0x46, 0x49, 0x47, 0x5f, 0x52, 0x4f, 0x57, 0x5f, 0x4b, 0x45, 0x59, 0x10, + 0x00, 0x1a, 0x10, 0xaa, 0xb4, 0xfa, 0xc2, 0x05, 0x0a, 0x63, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x5f, + 0x72, 0x6f, 0x77, 0x12, 0x32, 0x0a, 0x15, 0x43, 0x4f, 0x4e, 0x46, 0x49, 0x47, 0x5f, 0x52, 0x4f, + 0x57, 0x5f, 0x53, 0x43, 0x48, 0x45, 0x4d, 0x41, 0x5f, 0x4b, 0x45, 0x59, 0x10, 0x01, 0x1a, 0x17, + 0xaa, 0xb4, 0xfa, 0xc2, 0x05, 0x11, 0x63, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x5f, 0x72, 0x6f, 0x77, + 0x5f, 0x73, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x12, 0x36, 0x0a, 0x17, 0x53, 0x43, 0x48, 0x45, 0x4d, + 0x41, 0x54, 0x52, 0x41, 0x4e, 0x53, 0x46, 0x4f, 0x52, 0x4d, 0x5f, 0x55, 0x52, 0x4e, 0x5f, 0x4b, + 0x45, 0x59, 0x10, 0x02, 0x1a, 0x19, 0xaa, 0xb4, 0xfa, 0xc2, 0x05, 0x13, 0x73, 0x63, 0x68, 0x65, + 0x6d, 0x61, 0x74, 0x72, 0x61, 0x6e, 0x73, 0x66, 0x6f, 0x72, 0x6d, 0x5f, 0x75, 0x72, 0x6e, 0x12, + 0x50, 0x0a, 0x24, 0x4d, 0x41, 0x4e, 0x41, 0x47, 0x45, 0x44, 0x5f, 0x55, 0x4e, 0x44, 0x45, 0x52, + 0x4c, 0x59, 0x49, 0x4e, 0x47, 0x5f, 0x54, 0x52, 0x41, 0x4e, 0x53, 0x46, 0x4f, 0x52, 0x4d, 0x5f, + 0x55, 0x52, 0x4e, 0x5f, 0x4b, 0x45, 0x59, 0x10, 0x03, 0x1a, 0x26, 0xaa, 0xb4, 0xfa, 0xc2, 0x05, + 0x20, 0x6d, 0x61, 0x6e, 0x61, 0x67, 0x65, 0x64, 0x5f, 0x75, 0x6e, 0x64, 0x65, 0x72, 0x6c, 0x79, + 0x69, 0x6e, 0x67, 0x5f, 0x74, 0x72, 0x61, 0x6e, 0x73, 0x66, 0x6f, 0x72, 0x6d, 0x5f, 0x75, 0x72, + 0x6e, 0x22, 0xc3, 0x01, 0x0a, 0x16, 0x53, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x54, 0x72, 0x61, 0x6e, + 0x73, 0x66, 0x6f, 0x72, 0x6d, 0x50, 0x61, 0x79, 0x6c, 0x6f, 0x61, 0x64, 0x12, 0x1e, 0x0a, 0x0a, + 0x69, 0x64, 0x65, 0x6e, 0x74, 0x69, 0x66, 0x69, 0x65, 0x72, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, + 0x52, 0x0a, 0x69, 0x64, 0x65, 0x6e, 0x74, 0x69, 0x66, 0x69, 0x65, 0x72, 0x12, 0x5c, 0x0a, 0x14, + 0x63, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x75, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x73, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x29, 0x2e, 0x6f, 0x72, 0x67, 0x2e, 0x61, 0x70, 0x61, 0x63, 0x68, 0x65, 0x2e, 0x62, 0x65, 0x61, 0x6d, 0x2e, 0x6d, 0x6f, 0x64, 0x65, 0x6c, 0x2e, 0x70, 0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x53, - 0x63, 0x68, 0x65, 0x6d, 0x61, 0x52, 0x06, 0x73, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x12, 0x18, 0x0a, - 0x07, 0x70, 0x61, 0x79, 0x6c, 0x6f, 0x61, 0x64, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0c, 0x52, 0x07, - 0x70, 0x61, 0x79, 0x6c, 0x6f, 0x61, 0x64, 0x22, 0xfa, 0x01, 0x0a, 0x0b, 0x41, 0x6e, 0x6e, 0x6f, - 0x74, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x22, 0xea, 0x01, 0x0a, 0x04, 0x45, 0x6e, 0x75, 0x6d, - 0x12, 0x24, 0x0a, 0x0e, 0x43, 0x4f, 0x4e, 0x46, 0x49, 0x47, 0x5f, 0x52, 0x4f, 0x57, 0x5f, 0x4b, - 0x45, 0x59, 0x10, 0x00, 0x1a, 0x10, 0xaa, 0xb4, 0xfa, 0xc2, 0x05, 0x0a, 0x63, 0x6f, 0x6e, 0x66, - 0x69, 0x67, 0x5f, 0x72, 0x6f, 0x77, 0x12, 0x32, 0x0a, 0x15, 0x43, 0x4f, 0x4e, 0x46, 0x49, 0x47, - 0x5f, 0x52, 0x4f, 0x57, 0x5f, 0x53, 0x43, 0x48, 0x45, 0x4d, 0x41, 0x5f, 0x4b, 0x45, 0x59, 0x10, - 0x01, 0x1a, 0x17, 0xaa, 0xb4, 0xfa, 0xc2, 0x05, 0x11, 0x63, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x5f, - 0x72, 0x6f, 0x77, 0x5f, 0x73, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x12, 0x36, 0x0a, 0x17, 0x53, 0x43, - 0x48, 0x45, 0x4d, 0x41, 0x54, 0x52, 0x41, 0x4e, 0x53, 0x46, 0x4f, 0x52, 0x4d, 0x5f, 0x55, 0x52, - 0x4e, 0x5f, 0x4b, 0x45, 0x59, 0x10, 0x02, 0x1a, 0x19, 0xaa, 0xb4, 0xfa, 0xc2, 0x05, 0x13, 0x73, - 0x63, 0x68, 0x65, 0x6d, 0x61, 0x74, 0x72, 0x61, 0x6e, 0x73, 0x66, 0x6f, 0x72, 0x6d, 0x5f, 0x75, - 0x72, 0x6e, 0x12, 0x50, 0x0a, 0x24, 0x4d, 0x41, 0x4e, 0x41, 0x47, 0x45, 0x44, 0x5f, 0x55, 0x4e, - 0x44, 0x45, 0x52, 0x4c, 0x59, 0x49, 0x4e, 0x47, 0x5f, 0x54, 0x52, 0x41, 0x4e, 0x53, 0x46, 0x4f, - 0x52, 0x4d, 0x5f, 0x55, 0x52, 0x4e, 0x5f, 0x4b, 0x45, 0x59, 0x10, 0x03, 0x1a, 0x26, 0xaa, 0xb4, - 0xfa, 0xc2, 0x05, 0x20, 0x6d, 0x61, 0x6e, 0x61, 0x67, 0x65, 0x64, 0x5f, 0x75, 0x6e, 0x64, 0x65, - 0x72, 0x6c, 0x79, 0x69, 0x6e, 0x67, 0x5f, 0x74, 0x72, 0x61, 0x6e, 0x73, 0x66, 0x6f, 0x72, 0x6d, - 0x5f, 0x75, 0x72, 0x6e, 0x22, 0xc3, 0x01, 0x0a, 0x16, 0x53, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x54, - 0x72, 0x61, 0x6e, 0x73, 0x66, 0x6f, 0x72, 0x6d, 0x50, 0x61, 0x79, 0x6c, 0x6f, 0x61, 0x64, 0x12, - 0x1e, 0x0a, 0x0a, 0x69, 0x64, 0x65, 0x6e, 0x74, 0x69, 0x66, 0x69, 0x65, 0x72, 0x18, 0x01, 0x20, - 0x01, 0x28, 0x09, 0x52, 0x0a, 0x69, 0x64, 0x65, 0x6e, 0x74, 0x69, 0x66, 0x69, 0x65, 0x72, 0x12, - 0x5c, 0x0a, 0x14, 0x63, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x75, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, - 0x5f, 0x73, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x29, 0x2e, - 0x6f, 0x72, 0x67, 0x2e, 0x61, 0x70, 0x61, 0x63, 0x68, 0x65, 0x2e, 0x62, 0x65, 0x61, 0x6d, 0x2e, - 0x6d, 0x6f, 0x64, 0x65, 0x6c, 0x2e, 0x70, 0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e, 0x65, 0x2e, 0x76, - 0x31, 0x2e, 0x53, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x52, 0x13, 0x63, 0x6f, 0x6e, 0x66, 0x69, 0x67, - 0x75, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x53, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x12, 0x2b, 0x0a, - 0x11, 0x63, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x75, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x72, - 0x6f, 0x77, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0c, 0x52, 0x10, 0x63, 0x6f, 0x6e, 0x66, 0x69, 0x67, - 0x75, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x6f, 0x77, 0x42, 0x81, 0x01, 0x0a, 0x21, 0x6f, - 0x72, 0x67, 0x2e, 0x61, 0x70, 0x61, 0x63, 0x68, 0x65, 0x2e, 0x62, 0x65, 0x61, 0x6d, 0x2e, 0x6d, - 0x6f, 0x64, 0x65, 0x6c, 0x2e, 0x70, 0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e, 0x65, 0x2e, 0x76, 0x31, - 0x42, 0x12, 0x45, 0x78, 0x74, 0x65, 0x72, 0x6e, 0x61, 0x6c, 0x54, 0x72, 0x61, 0x6e, 0x73, 0x66, - 0x6f, 0x72, 0x6d, 0x73, 0x5a, 0x48, 0x67, 0x69, 0x74, 0x68, 0x75, 0x62, 0x2e, 0x63, 0x6f, 0x6d, - 0x2f, 0x61, 0x70, 0x61, 0x63, 0x68, 0x65, 0x2f, 0x62, 0x65, 0x61, 0x6d, 0x2f, 0x73, 0x64, 0x6b, - 0x73, 0x2f, 0x76, 0x32, 0x2f, 0x67, 0x6f, 0x2f, 0x70, 0x6b, 0x67, 0x2f, 0x62, 0x65, 0x61, 0x6d, - 0x2f, 0x6d, 0x6f, 0x64, 0x65, 0x6c, 0x2f, 0x70, 0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e, 0x65, 0x5f, - 0x76, 0x31, 0x3b, 0x70, 0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e, 0x65, 0x5f, 0x76, 0x31, 0x62, 0x06, - 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x33, -} - -var ( - file_org_apache_beam_model_pipeline_v1_external_transforms_proto_rawDescOnce sync.Once - file_org_apache_beam_model_pipeline_v1_external_transforms_proto_rawDescData = file_org_apache_beam_model_pipeline_v1_external_transforms_proto_rawDesc -) - -func file_org_apache_beam_model_pipeline_v1_external_transforms_proto_rawDescGZIP() []byte { - file_org_apache_beam_model_pipeline_v1_external_transforms_proto_rawDescOnce.Do(func() { - file_org_apache_beam_model_pipeline_v1_external_transforms_proto_rawDescData = protoimpl.X.CompressGZIP(file_org_apache_beam_model_pipeline_v1_external_transforms_proto_rawDescData) - }) - return file_org_apache_beam_model_pipeline_v1_external_transforms_proto_rawDescData -} - -var file_org_apache_beam_model_pipeline_v1_external_transforms_proto_enumTypes = make([]protoimpl.EnumInfo, 2) -var file_org_apache_beam_model_pipeline_v1_external_transforms_proto_msgTypes = make([]protoimpl.MessageInfo, 6) -var file_org_apache_beam_model_pipeline_v1_external_transforms_proto_goTypes = []interface{}{ + 0x63, 0x68, 0x65, 0x6d, 0x61, 0x52, 0x13, 0x63, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x75, 0x72, 0x61, + 0x74, 0x69, 0x6f, 0x6e, 0x53, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x12, 0x2b, 0x0a, 0x11, 0x63, 0x6f, + 0x6e, 0x66, 0x69, 0x67, 0x75, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x72, 0x6f, 0x77, 0x18, + 0x03, 0x20, 0x01, 0x28, 0x0c, 0x52, 0x10, 0x63, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x75, 0x72, 0x61, + 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x6f, 0x77, 0x42, 0x81, 0x01, 0x0a, 0x21, 0x6f, 0x72, 0x67, 0x2e, + 0x61, 0x70, 0x61, 0x63, 0x68, 0x65, 0x2e, 0x62, 0x65, 0x61, 0x6d, 0x2e, 0x6d, 0x6f, 0x64, 0x65, + 0x6c, 0x2e, 0x70, 0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e, 0x65, 0x2e, 0x76, 0x31, 0x42, 0x12, 0x45, + 0x78, 0x74, 0x65, 0x72, 0x6e, 0x61, 0x6c, 0x54, 0x72, 0x61, 0x6e, 0x73, 0x66, 0x6f, 0x72, 0x6d, + 0x73, 0x5a, 0x48, 0x67, 0x69, 0x74, 0x68, 0x75, 0x62, 0x2e, 0x63, 0x6f, 0x6d, 0x2f, 0x61, 0x70, + 0x61, 0x63, 0x68, 0x65, 0x2f, 0x62, 0x65, 0x61, 0x6d, 0x2f, 0x73, 0x64, 0x6b, 0x73, 0x2f, 0x76, + 0x32, 0x2f, 0x67, 0x6f, 0x2f, 0x70, 0x6b, 0x67, 0x2f, 0x62, 0x65, 0x61, 0x6d, 0x2f, 0x6d, 0x6f, + 0x64, 0x65, 0x6c, 0x2f, 0x70, 0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e, 0x65, 0x5f, 0x76, 0x31, 0x3b, + 0x70, 0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e, 0x65, 0x5f, 0x76, 0x31, 0x62, 0x06, 0x70, 0x72, 0x6f, + 0x74, 0x6f, 0x33, +} + +var file_org_apache_beam_model_pipeline_v1_external_transforms_proto_enumTypes = make([]protoimpl.EnumInfo, 3) +var file_org_apache_beam_model_pipeline_v1_external_transforms_proto_msgTypes = make([]protoimpl.MessageInfo, 7) +var file_org_apache_beam_model_pipeline_v1_external_transforms_proto_goTypes = []any{ (ExpansionMethods_Enum)(0), // 0: org.apache.beam.model.pipeline.v1.ExpansionMethods.Enum - (Annotations_Enum)(0), // 1: org.apache.beam.model.pipeline.v1.Annotations.Enum - (*ExternalConfigurationPayload)(nil), // 2: org.apache.beam.model.pipeline.v1.ExternalConfigurationPayload - (*ExpansionMethods)(nil), // 3: org.apache.beam.model.pipeline.v1.ExpansionMethods - (*JavaClassLookupPayload)(nil), // 4: org.apache.beam.model.pipeline.v1.JavaClassLookupPayload - (*BuilderMethod)(nil), // 5: org.apache.beam.model.pipeline.v1.BuilderMethod - (*Annotations)(nil), // 6: org.apache.beam.model.pipeline.v1.Annotations - (*SchemaTransformPayload)(nil), // 7: org.apache.beam.model.pipeline.v1.SchemaTransformPayload - (*Schema)(nil), // 8: org.apache.beam.model.pipeline.v1.Schema + (ManagedTransforms_Urns)(0), // 1: org.apache.beam.model.pipeline.v1.ManagedTransforms.Urns + (Annotations_Enum)(0), // 2: org.apache.beam.model.pipeline.v1.Annotations.Enum + (*ExternalConfigurationPayload)(nil), // 3: org.apache.beam.model.pipeline.v1.ExternalConfigurationPayload + (*ExpansionMethods)(nil), // 4: org.apache.beam.model.pipeline.v1.ExpansionMethods + (*ManagedTransforms)(nil), // 5: org.apache.beam.model.pipeline.v1.ManagedTransforms + (*JavaClassLookupPayload)(nil), // 6: org.apache.beam.model.pipeline.v1.JavaClassLookupPayload + (*BuilderMethod)(nil), // 7: org.apache.beam.model.pipeline.v1.BuilderMethod + (*Annotations)(nil), // 8: org.apache.beam.model.pipeline.v1.Annotations + (*SchemaTransformPayload)(nil), // 9: org.apache.beam.model.pipeline.v1.SchemaTransformPayload + (*Schema)(nil), // 10: org.apache.beam.model.pipeline.v1.Schema } var file_org_apache_beam_model_pipeline_v1_external_transforms_proto_depIdxs = []int32{ - 8, // 0: org.apache.beam.model.pipeline.v1.ExternalConfigurationPayload.schema:type_name -> org.apache.beam.model.pipeline.v1.Schema - 8, // 1: org.apache.beam.model.pipeline.v1.JavaClassLookupPayload.constructor_schema:type_name -> org.apache.beam.model.pipeline.v1.Schema - 5, // 2: org.apache.beam.model.pipeline.v1.JavaClassLookupPayload.builder_methods:type_name -> org.apache.beam.model.pipeline.v1.BuilderMethod - 8, // 3: org.apache.beam.model.pipeline.v1.BuilderMethod.schema:type_name -> org.apache.beam.model.pipeline.v1.Schema - 8, // 4: org.apache.beam.model.pipeline.v1.SchemaTransformPayload.configuration_schema:type_name -> org.apache.beam.model.pipeline.v1.Schema - 5, // [5:5] is the sub-list for method output_type - 5, // [5:5] is the sub-list for method input_type - 5, // [5:5] is the sub-list for extension type_name - 5, // [5:5] is the sub-list for extension extendee - 0, // [0:5] is the sub-list for field type_name + 10, // 0: org.apache.beam.model.pipeline.v1.ExternalConfigurationPayload.schema:type_name -> org.apache.beam.model.pipeline.v1.Schema + 10, // 1: org.apache.beam.model.pipeline.v1.JavaClassLookupPayload.constructor_schema:type_name -> org.apache.beam.model.pipeline.v1.Schema + 7, // 2: org.apache.beam.model.pipeline.v1.JavaClassLookupPayload.builder_methods:type_name -> org.apache.beam.model.pipeline.v1.BuilderMethod + 10, // 3: org.apache.beam.model.pipeline.v1.BuilderMethod.schema:type_name -> org.apache.beam.model.pipeline.v1.Schema + 10, // 4: org.apache.beam.model.pipeline.v1.SchemaTransformPayload.configuration_schema:type_name -> org.apache.beam.model.pipeline.v1.Schema + 5, // [5:5] is the sub-list for method output_type + 5, // [5:5] is the sub-list for method input_type + 5, // [5:5] is the sub-list for extension type_name + 5, // [5:5] is the sub-list for extension extendee + 0, // [0:5] is the sub-list for field type_name } func init() { file_org_apache_beam_model_pipeline_v1_external_transforms_proto_init() } @@ -680,87 +984,13 @@ func file_org_apache_beam_model_pipeline_v1_external_transforms_proto_init() { } file_org_apache_beam_model_pipeline_v1_schema_proto_init() file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_init() - if !protoimpl.UnsafeEnabled { - file_org_apache_beam_model_pipeline_v1_external_transforms_proto_msgTypes[0].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*ExternalConfigurationPayload); i { - case 0: - return &v.state - case 1: - return &v.sizeCache - case 2: - return &v.unknownFields - default: - return nil - } - } - file_org_apache_beam_model_pipeline_v1_external_transforms_proto_msgTypes[1].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*ExpansionMethods); i { - case 0: - return &v.state - case 1: - return &v.sizeCache - case 2: - return &v.unknownFields - default: - return nil - } - } - file_org_apache_beam_model_pipeline_v1_external_transforms_proto_msgTypes[2].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*JavaClassLookupPayload); i { - case 0: - return &v.state - case 1: - return &v.sizeCache - case 2: - return &v.unknownFields - default: - return nil - } - } - file_org_apache_beam_model_pipeline_v1_external_transforms_proto_msgTypes[3].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*BuilderMethod); i { - case 0: - return &v.state - case 1: - return &v.sizeCache - case 2: - return &v.unknownFields - default: - return nil - } - } - file_org_apache_beam_model_pipeline_v1_external_transforms_proto_msgTypes[4].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*Annotations); i { - case 0: - return &v.state - case 1: - return &v.sizeCache - case 2: - return &v.unknownFields - default: - return nil - } - } - file_org_apache_beam_model_pipeline_v1_external_transforms_proto_msgTypes[5].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*SchemaTransformPayload); i { - case 0: - return &v.state - case 1: - return &v.sizeCache - case 2: - return &v.unknownFields - default: - return nil - } - } - } type x struct{} out := protoimpl.TypeBuilder{ File: protoimpl.DescBuilder{ GoPackagePath: reflect.TypeOf(x{}).PkgPath(), RawDescriptor: file_org_apache_beam_model_pipeline_v1_external_transforms_proto_rawDesc, - NumEnums: 2, - NumMessages: 6, + NumEnums: 3, + NumMessages: 7, NumExtensions: 0, NumServices: 0, }, diff --git a/sdks/go/pkg/beam/model/pipeline_v1/external_transforms_protoopaque.pb.go b/sdks/go/pkg/beam/model/pipeline_v1/external_transforms_protoopaque.pb.go new file mode 100644 index 000000000000..d3dc7c0c4a2c --- /dev/null +++ b/sdks/go/pkg/beam/model/pipeline_v1/external_transforms_protoopaque.pb.go @@ -0,0 +1,974 @@ +// +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +// +// Protocol Buffers describing the external transforms available. + +// Code generated by protoc-gen-go. DO NOT EDIT. +// versions: +// protoc-gen-go v1.36.0 +// protoc v5.27.3 +// source: org/apache/beam/model/pipeline/v1/external_transforms.proto + +//go:build protoopaque + +package pipeline_v1 + +import ( + protoreflect "google.golang.org/protobuf/reflect/protoreflect" + protoimpl "google.golang.org/protobuf/runtime/protoimpl" + reflect "reflect" +) + +const ( + // Verify that this generated code is sufficiently up-to-date. + _ = protoimpl.EnforceVersion(20 - protoimpl.MinVersion) + // Verify that runtime/protoimpl is sufficiently up-to-date. + _ = protoimpl.EnforceVersion(protoimpl.MaxVersion - 20) +) + +type ExpansionMethods_Enum int32 + +const ( + // Expand a Java transform using specified constructor and builder methods. + // Transform payload will be of type JavaClassLookupPayload. + ExpansionMethods_JAVA_CLASS_LOOKUP ExpansionMethods_Enum = 0 + // Expanding a SchemaTransform identified by the expansion service. + // Transform payload will be of type SchemaTransformPayload. + ExpansionMethods_SCHEMA_TRANSFORM ExpansionMethods_Enum = 1 +) + +// Enum value maps for ExpansionMethods_Enum. +var ( + ExpansionMethods_Enum_name = map[int32]string{ + 0: "JAVA_CLASS_LOOKUP", + 1: "SCHEMA_TRANSFORM", + } + ExpansionMethods_Enum_value = map[string]int32{ + "JAVA_CLASS_LOOKUP": 0, + "SCHEMA_TRANSFORM": 1, + } +) + +func (x ExpansionMethods_Enum) Enum() *ExpansionMethods_Enum { + p := new(ExpansionMethods_Enum) + *p = x + return p +} + +func (x ExpansionMethods_Enum) String() string { + return protoimpl.X.EnumStringOf(x.Descriptor(), protoreflect.EnumNumber(x)) +} + +func (ExpansionMethods_Enum) Descriptor() protoreflect.EnumDescriptor { + return file_org_apache_beam_model_pipeline_v1_external_transforms_proto_enumTypes[0].Descriptor() +} + +func (ExpansionMethods_Enum) Type() protoreflect.EnumType { + return &file_org_apache_beam_model_pipeline_v1_external_transforms_proto_enumTypes[0] +} + +func (x ExpansionMethods_Enum) Number() protoreflect.EnumNumber { + return protoreflect.EnumNumber(x) +} + +type ManagedTransforms_Urns int32 + +const ( + ManagedTransforms_ICEBERG_READ ManagedTransforms_Urns = 0 + ManagedTransforms_ICEBERG_WRITE ManagedTransforms_Urns = 1 + ManagedTransforms_KAFKA_READ ManagedTransforms_Urns = 2 + ManagedTransforms_KAFKA_WRITE ManagedTransforms_Urns = 3 + ManagedTransforms_BIGQUERY_READ ManagedTransforms_Urns = 4 + ManagedTransforms_BIGQUERY_WRITE ManagedTransforms_Urns = 5 +) + +// Enum value maps for ManagedTransforms_Urns. +var ( + ManagedTransforms_Urns_name = map[int32]string{ + 0: "ICEBERG_READ", + 1: "ICEBERG_WRITE", + 2: "KAFKA_READ", + 3: "KAFKA_WRITE", + 4: "BIGQUERY_READ", + 5: "BIGQUERY_WRITE", + } + ManagedTransforms_Urns_value = map[string]int32{ + "ICEBERG_READ": 0, + "ICEBERG_WRITE": 1, + "KAFKA_READ": 2, + "KAFKA_WRITE": 3, + "BIGQUERY_READ": 4, + "BIGQUERY_WRITE": 5, + } +) + +func (x ManagedTransforms_Urns) Enum() *ManagedTransforms_Urns { + p := new(ManagedTransforms_Urns) + *p = x + return p +} + +func (x ManagedTransforms_Urns) String() string { + return protoimpl.X.EnumStringOf(x.Descriptor(), protoreflect.EnumNumber(x)) +} + +func (ManagedTransforms_Urns) Descriptor() protoreflect.EnumDescriptor { + return file_org_apache_beam_model_pipeline_v1_external_transforms_proto_enumTypes[1].Descriptor() +} + +func (ManagedTransforms_Urns) Type() protoreflect.EnumType { + return &file_org_apache_beam_model_pipeline_v1_external_transforms_proto_enumTypes[1] +} + +func (x ManagedTransforms_Urns) Number() protoreflect.EnumNumber { + return protoreflect.EnumNumber(x) +} + +type Annotations_Enum int32 + +const ( + // The annotation key for the encoded configuration Row used to build a transform + Annotations_CONFIG_ROW_KEY Annotations_Enum = 0 + // The annotation key for the configuration Schema used to decode the configuration Row + Annotations_CONFIG_ROW_SCHEMA_KEY Annotations_Enum = 1 + // If ths transform is a SchemaTransform, this is the annotation key for the SchemaTransform's URN + Annotations_SCHEMATRANSFORM_URN_KEY Annotations_Enum = 2 + // If the transform is a ManagedSchemaTransform, this is the annotation key for the underlying SchemaTransform's URN + Annotations_MANAGED_UNDERLYING_TRANSFORM_URN_KEY Annotations_Enum = 3 +) + +// Enum value maps for Annotations_Enum. +var ( + Annotations_Enum_name = map[int32]string{ + 0: "CONFIG_ROW_KEY", + 1: "CONFIG_ROW_SCHEMA_KEY", + 2: "SCHEMATRANSFORM_URN_KEY", + 3: "MANAGED_UNDERLYING_TRANSFORM_URN_KEY", + } + Annotations_Enum_value = map[string]int32{ + "CONFIG_ROW_KEY": 0, + "CONFIG_ROW_SCHEMA_KEY": 1, + "SCHEMATRANSFORM_URN_KEY": 2, + "MANAGED_UNDERLYING_TRANSFORM_URN_KEY": 3, + } +) + +func (x Annotations_Enum) Enum() *Annotations_Enum { + p := new(Annotations_Enum) + *p = x + return p +} + +func (x Annotations_Enum) String() string { + return protoimpl.X.EnumStringOf(x.Descriptor(), protoreflect.EnumNumber(x)) +} + +func (Annotations_Enum) Descriptor() protoreflect.EnumDescriptor { + return file_org_apache_beam_model_pipeline_v1_external_transforms_proto_enumTypes[2].Descriptor() +} + +func (Annotations_Enum) Type() protoreflect.EnumType { + return &file_org_apache_beam_model_pipeline_v1_external_transforms_proto_enumTypes[2] +} + +func (x Annotations_Enum) Number() protoreflect.EnumNumber { + return protoreflect.EnumNumber(x) +} + +// A configuration payload for an external transform. +// Used as the payload of ExternalTransform as part of an ExpansionRequest. +type ExternalConfigurationPayload struct { + state protoimpl.MessageState `protogen:"opaque.v1"` + xxx_hidden_Schema *Schema `protobuf:"bytes,1,opt,name=schema,proto3" json:"schema,omitempty"` + xxx_hidden_Payload []byte `protobuf:"bytes,2,opt,name=payload,proto3" json:"payload,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *ExternalConfigurationPayload) Reset() { + *x = ExternalConfigurationPayload{} + mi := &file_org_apache_beam_model_pipeline_v1_external_transforms_proto_msgTypes[0] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *ExternalConfigurationPayload) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*ExternalConfigurationPayload) ProtoMessage() {} + +func (x *ExternalConfigurationPayload) ProtoReflect() protoreflect.Message { + mi := &file_org_apache_beam_model_pipeline_v1_external_transforms_proto_msgTypes[0] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +func (x *ExternalConfigurationPayload) GetSchema() *Schema { + if x != nil { + return x.xxx_hidden_Schema + } + return nil +} + +func (x *ExternalConfigurationPayload) GetPayload() []byte { + if x != nil { + return x.xxx_hidden_Payload + } + return nil +} + +func (x *ExternalConfigurationPayload) SetSchema(v *Schema) { + x.xxx_hidden_Schema = v +} + +func (x *ExternalConfigurationPayload) SetPayload(v []byte) { + if v == nil { + v = []byte{} + } + x.xxx_hidden_Payload = v +} + +func (x *ExternalConfigurationPayload) HasSchema() bool { + if x == nil { + return false + } + return x.xxx_hidden_Schema != nil +} + +func (x *ExternalConfigurationPayload) ClearSchema() { + x.xxx_hidden_Schema = nil +} + +type ExternalConfigurationPayload_builder struct { + _ [0]func() // Prevents comparability and use of unkeyed literals for the builder. + + // A schema for use in beam:coder:row:v1 + Schema *Schema + // A payload which can be decoded using beam:coder:row:v1 and the given + // schema. + Payload []byte +} + +func (b0 ExternalConfigurationPayload_builder) Build() *ExternalConfigurationPayload { + m0 := &ExternalConfigurationPayload{} + b, x := &b0, m0 + _, _ = b, x + x.xxx_hidden_Schema = b.Schema + x.xxx_hidden_Payload = b.Payload + return m0 +} + +// Defines specific expansion methods that may be used to expand cross-language +// transforms. +// Has to be set as the URN of the transform of the expansion request. +type ExpansionMethods struct { + state protoimpl.MessageState `protogen:"opaque.v1"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *ExpansionMethods) Reset() { + *x = ExpansionMethods{} + mi := &file_org_apache_beam_model_pipeline_v1_external_transforms_proto_msgTypes[1] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *ExpansionMethods) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*ExpansionMethods) ProtoMessage() {} + +func (x *ExpansionMethods) ProtoReflect() protoreflect.Message { + mi := &file_org_apache_beam_model_pipeline_v1_external_transforms_proto_msgTypes[1] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +type ExpansionMethods_builder struct { + _ [0]func() // Prevents comparability and use of unkeyed literals for the builder. + +} + +func (b0 ExpansionMethods_builder) Build() *ExpansionMethods { + m0 := &ExpansionMethods{} + b, x := &b0, m0 + _, _ = b, x + return m0 +} + +// Defines the URNs for managed transforms. +type ManagedTransforms struct { + state protoimpl.MessageState `protogen:"opaque.v1"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *ManagedTransforms) Reset() { + *x = ManagedTransforms{} + mi := &file_org_apache_beam_model_pipeline_v1_external_transforms_proto_msgTypes[2] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *ManagedTransforms) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*ManagedTransforms) ProtoMessage() {} + +func (x *ManagedTransforms) ProtoReflect() protoreflect.Message { + mi := &file_org_apache_beam_model_pipeline_v1_external_transforms_proto_msgTypes[2] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +type ManagedTransforms_builder struct { + _ [0]func() // Prevents comparability and use of unkeyed literals for the builder. + +} + +func (b0 ManagedTransforms_builder) Build() *ManagedTransforms { + m0 := &ManagedTransforms{} + b, x := &b0, m0 + _, _ = b, x + return m0 +} + +// A configuration payload for an external transform. +// Used to define a Java transform that can be directly instantiated by a Java +// expansion service. +type JavaClassLookupPayload struct { + state protoimpl.MessageState `protogen:"opaque.v1"` + xxx_hidden_ClassName string `protobuf:"bytes,1,opt,name=class_name,json=className,proto3" json:"class_name,omitempty"` + xxx_hidden_ConstructorMethod string `protobuf:"bytes,2,opt,name=constructor_method,json=constructorMethod,proto3" json:"constructor_method,omitempty"` + xxx_hidden_ConstructorSchema *Schema `protobuf:"bytes,3,opt,name=constructor_schema,json=constructorSchema,proto3" json:"constructor_schema,omitempty"` + xxx_hidden_ConstructorPayload []byte `protobuf:"bytes,4,opt,name=constructor_payload,json=constructorPayload,proto3" json:"constructor_payload,omitempty"` + xxx_hidden_BuilderMethods *[]*BuilderMethod `protobuf:"bytes,5,rep,name=builder_methods,json=builderMethods,proto3" json:"builder_methods,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *JavaClassLookupPayload) Reset() { + *x = JavaClassLookupPayload{} + mi := &file_org_apache_beam_model_pipeline_v1_external_transforms_proto_msgTypes[3] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *JavaClassLookupPayload) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*JavaClassLookupPayload) ProtoMessage() {} + +func (x *JavaClassLookupPayload) ProtoReflect() protoreflect.Message { + mi := &file_org_apache_beam_model_pipeline_v1_external_transforms_proto_msgTypes[3] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +func (x *JavaClassLookupPayload) GetClassName() string { + if x != nil { + return x.xxx_hidden_ClassName + } + return "" +} + +func (x *JavaClassLookupPayload) GetConstructorMethod() string { + if x != nil { + return x.xxx_hidden_ConstructorMethod + } + return "" +} + +func (x *JavaClassLookupPayload) GetConstructorSchema() *Schema { + if x != nil { + return x.xxx_hidden_ConstructorSchema + } + return nil +} + +func (x *JavaClassLookupPayload) GetConstructorPayload() []byte { + if x != nil { + return x.xxx_hidden_ConstructorPayload + } + return nil +} + +func (x *JavaClassLookupPayload) GetBuilderMethods() []*BuilderMethod { + if x != nil { + if x.xxx_hidden_BuilderMethods != nil { + return *x.xxx_hidden_BuilderMethods + } + } + return nil +} + +func (x *JavaClassLookupPayload) SetClassName(v string) { + x.xxx_hidden_ClassName = v +} + +func (x *JavaClassLookupPayload) SetConstructorMethod(v string) { + x.xxx_hidden_ConstructorMethod = v +} + +func (x *JavaClassLookupPayload) SetConstructorSchema(v *Schema) { + x.xxx_hidden_ConstructorSchema = v +} + +func (x *JavaClassLookupPayload) SetConstructorPayload(v []byte) { + if v == nil { + v = []byte{} + } + x.xxx_hidden_ConstructorPayload = v +} + +func (x *JavaClassLookupPayload) SetBuilderMethods(v []*BuilderMethod) { + x.xxx_hidden_BuilderMethods = &v +} + +func (x *JavaClassLookupPayload) HasConstructorSchema() bool { + if x == nil { + return false + } + return x.xxx_hidden_ConstructorSchema != nil +} + +func (x *JavaClassLookupPayload) ClearConstructorSchema() { + x.xxx_hidden_ConstructorSchema = nil +} + +type JavaClassLookupPayload_builder struct { + _ [0]func() // Prevents comparability and use of unkeyed literals for the builder. + + // Name of the Java transform class. + ClassName string + // A static method to construct the initial instance of the transform. + // If not provided, the transform should be instantiated using a class + // constructor. + ConstructorMethod string + // The top level fields of the schema represent the method parameters in + // order. + // If able, top level field names are also verified against the method + // parameters for a match. + // Any field names in the form 'ignore[0-9]+' will not be used for validation + // hence that format can be used to represent arbitrary field names. + ConstructorSchema *Schema + // A payload which can be decoded using beam:coder:row:v1 and the provided + // constructor schema. + ConstructorPayload []byte + // Set of builder methods and corresponding parameters to apply after the + // transform object is constructed. + // When constructing the transform object, given builder methods will be + // applied in order. + BuilderMethods []*BuilderMethod +} + +func (b0 JavaClassLookupPayload_builder) Build() *JavaClassLookupPayload { + m0 := &JavaClassLookupPayload{} + b, x := &b0, m0 + _, _ = b, x + x.xxx_hidden_ClassName = b.ClassName + x.xxx_hidden_ConstructorMethod = b.ConstructorMethod + x.xxx_hidden_ConstructorSchema = b.ConstructorSchema + x.xxx_hidden_ConstructorPayload = b.ConstructorPayload + x.xxx_hidden_BuilderMethods = &b.BuilderMethods + return m0 +} + +// This represents a builder method of the transform class that should be +// applied in-order after instantiating the initial transform object. +// Each builder method may take one or more parameters and has to return an +// instance of the transform object. +type BuilderMethod struct { + state protoimpl.MessageState `protogen:"opaque.v1"` + xxx_hidden_Name string `protobuf:"bytes,1,opt,name=name,proto3" json:"name,omitempty"` + xxx_hidden_Schema *Schema `protobuf:"bytes,2,opt,name=schema,proto3" json:"schema,omitempty"` + xxx_hidden_Payload []byte `protobuf:"bytes,3,opt,name=payload,proto3" json:"payload,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *BuilderMethod) Reset() { + *x = BuilderMethod{} + mi := &file_org_apache_beam_model_pipeline_v1_external_transforms_proto_msgTypes[4] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *BuilderMethod) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*BuilderMethod) ProtoMessage() {} + +func (x *BuilderMethod) ProtoReflect() protoreflect.Message { + mi := &file_org_apache_beam_model_pipeline_v1_external_transforms_proto_msgTypes[4] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +func (x *BuilderMethod) GetName() string { + if x != nil { + return x.xxx_hidden_Name + } + return "" +} + +func (x *BuilderMethod) GetSchema() *Schema { + if x != nil { + return x.xxx_hidden_Schema + } + return nil +} + +func (x *BuilderMethod) GetPayload() []byte { + if x != nil { + return x.xxx_hidden_Payload + } + return nil +} + +func (x *BuilderMethod) SetName(v string) { + x.xxx_hidden_Name = v +} + +func (x *BuilderMethod) SetSchema(v *Schema) { + x.xxx_hidden_Schema = v +} + +func (x *BuilderMethod) SetPayload(v []byte) { + if v == nil { + v = []byte{} + } + x.xxx_hidden_Payload = v +} + +func (x *BuilderMethod) HasSchema() bool { + if x == nil { + return false + } + return x.xxx_hidden_Schema != nil +} + +func (x *BuilderMethod) ClearSchema() { + x.xxx_hidden_Schema = nil +} + +type BuilderMethod_builder struct { + _ [0]func() // Prevents comparability and use of unkeyed literals for the builder. + + // Name of the builder method + Name string + // The top level fields of the schema represent the method parameters in + // order. + // If able, top level field names are also verified against the method + // parameters for a match. + // Any field names in the form 'ignore[0-9]+' will not be used for validation + // hence that format can be used to represent arbitrary field names. + Schema *Schema + // A payload which can be decoded using beam:coder:row:v1 and the builder + // method schema. + Payload []byte +} + +func (b0 BuilderMethod_builder) Build() *BuilderMethod { + m0 := &BuilderMethod{} + b, x := &b0, m0 + _, _ = b, x + x.xxx_hidden_Name = b.Name + x.xxx_hidden_Schema = b.Schema + x.xxx_hidden_Payload = b.Payload + return m0 +} + +type Annotations struct { + state protoimpl.MessageState `protogen:"opaque.v1"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *Annotations) Reset() { + *x = Annotations{} + mi := &file_org_apache_beam_model_pipeline_v1_external_transforms_proto_msgTypes[5] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *Annotations) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*Annotations) ProtoMessage() {} + +func (x *Annotations) ProtoReflect() protoreflect.Message { + mi := &file_org_apache_beam_model_pipeline_v1_external_transforms_proto_msgTypes[5] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +type Annotations_builder struct { + _ [0]func() // Prevents comparability and use of unkeyed literals for the builder. + +} + +func (b0 Annotations_builder) Build() *Annotations { + m0 := &Annotations{} + b, x := &b0, m0 + _, _ = b, x + return m0 +} + +// Payload for a Schema-aware PTransform. +// This is a transform that is aware of its input and output PCollection schemas +// and is configured using Beam Schema-compatible parameters. +// The information available in the payload can be used to instantiate the schema-aware transform. +type SchemaTransformPayload struct { + state protoimpl.MessageState `protogen:"opaque.v1"` + xxx_hidden_Identifier string `protobuf:"bytes,1,opt,name=identifier,proto3" json:"identifier,omitempty"` + xxx_hidden_ConfigurationSchema *Schema `protobuf:"bytes,2,opt,name=configuration_schema,json=configurationSchema,proto3" json:"configuration_schema,omitempty"` + xxx_hidden_ConfigurationRow []byte `protobuf:"bytes,3,opt,name=configuration_row,json=configurationRow,proto3" json:"configuration_row,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *SchemaTransformPayload) Reset() { + *x = SchemaTransformPayload{} + mi := &file_org_apache_beam_model_pipeline_v1_external_transforms_proto_msgTypes[6] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *SchemaTransformPayload) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*SchemaTransformPayload) ProtoMessage() {} + +func (x *SchemaTransformPayload) ProtoReflect() protoreflect.Message { + mi := &file_org_apache_beam_model_pipeline_v1_external_transforms_proto_msgTypes[6] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +func (x *SchemaTransformPayload) GetIdentifier() string { + if x != nil { + return x.xxx_hidden_Identifier + } + return "" +} + +func (x *SchemaTransformPayload) GetConfigurationSchema() *Schema { + if x != nil { + return x.xxx_hidden_ConfigurationSchema + } + return nil +} + +func (x *SchemaTransformPayload) GetConfigurationRow() []byte { + if x != nil { + return x.xxx_hidden_ConfigurationRow + } + return nil +} + +func (x *SchemaTransformPayload) SetIdentifier(v string) { + x.xxx_hidden_Identifier = v +} + +func (x *SchemaTransformPayload) SetConfigurationSchema(v *Schema) { + x.xxx_hidden_ConfigurationSchema = v +} + +func (x *SchemaTransformPayload) SetConfigurationRow(v []byte) { + if v == nil { + v = []byte{} + } + x.xxx_hidden_ConfigurationRow = v +} + +func (x *SchemaTransformPayload) HasConfigurationSchema() bool { + if x == nil { + return false + } + return x.xxx_hidden_ConfigurationSchema != nil +} + +func (x *SchemaTransformPayload) ClearConfigurationSchema() { + x.xxx_hidden_ConfigurationSchema = nil +} + +type SchemaTransformPayload_builder struct { + _ [0]func() // Prevents comparability and use of unkeyed literals for the builder. + + // The identifier of the SchemaTransform (typically a URN). + Identifier string + // The configuration schema of the SchemaTransform. + ConfigurationSchema *Schema + // The configuration of the SchemaTransform. + // Should be decodable via beam:coder:row:v1. + // The schema of the Row should be compatible with the schema of the + // SchemaTransform. + ConfigurationRow []byte +} + +func (b0 SchemaTransformPayload_builder) Build() *SchemaTransformPayload { + m0 := &SchemaTransformPayload{} + b, x := &b0, m0 + _, _ = b, x + x.xxx_hidden_Identifier = b.Identifier + x.xxx_hidden_ConfigurationSchema = b.ConfigurationSchema + x.xxx_hidden_ConfigurationRow = b.ConfigurationRow + return m0 +} + +var File_org_apache_beam_model_pipeline_v1_external_transforms_proto protoreflect.FileDescriptor + +var file_org_apache_beam_model_pipeline_v1_external_transforms_proto_rawDesc = []byte{ + 0x0a, 0x3b, 0x6f, 0x72, 0x67, 0x2f, 0x61, 0x70, 0x61, 0x63, 0x68, 0x65, 0x2f, 0x62, 0x65, 0x61, + 0x6d, 0x2f, 0x6d, 0x6f, 0x64, 0x65, 0x6c, 0x2f, 0x70, 0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e, 0x65, + 0x2f, 0x76, 0x31, 0x2f, 0x65, 0x78, 0x74, 0x65, 0x72, 0x6e, 0x61, 0x6c, 0x5f, 0x74, 0x72, 0x61, + 0x6e, 0x73, 0x66, 0x6f, 0x72, 0x6d, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x12, 0x21, 0x6f, + 0x72, 0x67, 0x2e, 0x61, 0x70, 0x61, 0x63, 0x68, 0x65, 0x2e, 0x62, 0x65, 0x61, 0x6d, 0x2e, 0x6d, + 0x6f, 0x64, 0x65, 0x6c, 0x2e, 0x70, 0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e, 0x65, 0x2e, 0x76, 0x31, + 0x1a, 0x2e, 0x6f, 0x72, 0x67, 0x2f, 0x61, 0x70, 0x61, 0x63, 0x68, 0x65, 0x2f, 0x62, 0x65, 0x61, + 0x6d, 0x2f, 0x6d, 0x6f, 0x64, 0x65, 0x6c, 0x2f, 0x70, 0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e, 0x65, + 0x2f, 0x76, 0x31, 0x2f, 0x73, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, + 0x1a, 0x37, 0x6f, 0x72, 0x67, 0x2f, 0x61, 0x70, 0x61, 0x63, 0x68, 0x65, 0x2f, 0x62, 0x65, 0x61, + 0x6d, 0x2f, 0x6d, 0x6f, 0x64, 0x65, 0x6c, 0x2f, 0x70, 0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e, 0x65, + 0x2f, 0x76, 0x31, 0x2f, 0x62, 0x65, 0x61, 0x6d, 0x5f, 0x72, 0x75, 0x6e, 0x6e, 0x65, 0x72, 0x5f, + 0x61, 0x70, 0x69, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x22, 0x7b, 0x0a, 0x1c, 0x45, 0x78, 0x74, + 0x65, 0x72, 0x6e, 0x61, 0x6c, 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x75, 0x72, 0x61, 0x74, 0x69, + 0x6f, 0x6e, 0x50, 0x61, 0x79, 0x6c, 0x6f, 0x61, 0x64, 0x12, 0x41, 0x0a, 0x06, 0x73, 0x63, 0x68, + 0x65, 0x6d, 0x61, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x29, 0x2e, 0x6f, 0x72, 0x67, 0x2e, + 0x61, 0x70, 0x61, 0x63, 0x68, 0x65, 0x2e, 0x62, 0x65, 0x61, 0x6d, 0x2e, 0x6d, 0x6f, 0x64, 0x65, + 0x6c, 0x2e, 0x70, 0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x53, 0x63, + 0x68, 0x65, 0x6d, 0x61, 0x52, 0x06, 0x73, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x12, 0x18, 0x0a, 0x07, + 0x70, 0x61, 0x79, 0x6c, 0x6f, 0x61, 0x64, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0c, 0x52, 0x07, 0x70, + 0x61, 0x79, 0x6c, 0x6f, 0x61, 0x64, 0x22, 0xac, 0x01, 0x0a, 0x10, 0x45, 0x78, 0x70, 0x61, 0x6e, + 0x73, 0x69, 0x6f, 0x6e, 0x4d, 0x65, 0x74, 0x68, 0x6f, 0x64, 0x73, 0x22, 0x97, 0x01, 0x0a, 0x04, + 0x45, 0x6e, 0x75, 0x6d, 0x12, 0x48, 0x0a, 0x11, 0x4a, 0x41, 0x56, 0x41, 0x5f, 0x43, 0x4c, 0x41, + 0x53, 0x53, 0x5f, 0x4c, 0x4f, 0x4f, 0x4b, 0x55, 0x50, 0x10, 0x00, 0x1a, 0x31, 0xa2, 0xb4, 0xfa, + 0xc2, 0x05, 0x2b, 0x62, 0x65, 0x61, 0x6d, 0x3a, 0x65, 0x78, 0x70, 0x61, 0x6e, 0x73, 0x69, 0x6f, + 0x6e, 0x3a, 0x70, 0x61, 0x79, 0x6c, 0x6f, 0x61, 0x64, 0x3a, 0x6a, 0x61, 0x76, 0x61, 0x5f, 0x63, + 0x6c, 0x61, 0x73, 0x73, 0x5f, 0x6c, 0x6f, 0x6f, 0x6b, 0x75, 0x70, 0x3a, 0x76, 0x31, 0x12, 0x45, + 0x0a, 0x10, 0x53, 0x43, 0x48, 0x45, 0x4d, 0x41, 0x5f, 0x54, 0x52, 0x41, 0x4e, 0x53, 0x46, 0x4f, + 0x52, 0x4d, 0x10, 0x01, 0x1a, 0x2f, 0xa2, 0xb4, 0xfa, 0xc2, 0x05, 0x29, 0x62, 0x65, 0x61, 0x6d, + 0x3a, 0x65, 0x78, 0x70, 0x61, 0x6e, 0x73, 0x69, 0x6f, 0x6e, 0x3a, 0x70, 0x61, 0x79, 0x6c, 0x6f, + 0x61, 0x64, 0x3a, 0x73, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x74, 0x72, 0x61, 0x6e, 0x73, 0x66, 0x6f, + 0x72, 0x6d, 0x3a, 0x76, 0x31, 0x22, 0xfa, 0x03, 0x0a, 0x11, 0x4d, 0x61, 0x6e, 0x61, 0x67, 0x65, + 0x64, 0x54, 0x72, 0x61, 0x6e, 0x73, 0x66, 0x6f, 0x72, 0x6d, 0x73, 0x22, 0xe4, 0x03, 0x0a, 0x04, + 0x55, 0x72, 0x6e, 0x73, 0x12, 0x4c, 0x0a, 0x0c, 0x49, 0x43, 0x45, 0x42, 0x45, 0x52, 0x47, 0x5f, + 0x52, 0x45, 0x41, 0x44, 0x10, 0x00, 0x1a, 0x3a, 0xa2, 0xb4, 0xfa, 0xc2, 0x05, 0x34, 0x62, 0x65, + 0x61, 0x6d, 0x3a, 0x73, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x74, 0x72, 0x61, 0x6e, 0x73, 0x66, 0x6f, + 0x72, 0x6d, 0x3a, 0x6f, 0x72, 0x67, 0x2e, 0x61, 0x70, 0x61, 0x63, 0x68, 0x65, 0x2e, 0x62, 0x65, + 0x61, 0x6d, 0x3a, 0x69, 0x63, 0x65, 0x62, 0x65, 0x72, 0x67, 0x5f, 0x72, 0x65, 0x61, 0x64, 0x3a, + 0x76, 0x31, 0x12, 0x4e, 0x0a, 0x0d, 0x49, 0x43, 0x45, 0x42, 0x45, 0x52, 0x47, 0x5f, 0x57, 0x52, + 0x49, 0x54, 0x45, 0x10, 0x01, 0x1a, 0x3b, 0xa2, 0xb4, 0xfa, 0xc2, 0x05, 0x35, 0x62, 0x65, 0x61, + 0x6d, 0x3a, 0x73, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x74, 0x72, 0x61, 0x6e, 0x73, 0x66, 0x6f, 0x72, + 0x6d, 0x3a, 0x6f, 0x72, 0x67, 0x2e, 0x61, 0x70, 0x61, 0x63, 0x68, 0x65, 0x2e, 0x62, 0x65, 0x61, + 0x6d, 0x3a, 0x69, 0x63, 0x65, 0x62, 0x65, 0x72, 0x67, 0x5f, 0x77, 0x72, 0x69, 0x74, 0x65, 0x3a, + 0x76, 0x31, 0x12, 0x48, 0x0a, 0x0a, 0x4b, 0x41, 0x46, 0x4b, 0x41, 0x5f, 0x52, 0x45, 0x41, 0x44, + 0x10, 0x02, 0x1a, 0x38, 0xa2, 0xb4, 0xfa, 0xc2, 0x05, 0x32, 0x62, 0x65, 0x61, 0x6d, 0x3a, 0x73, + 0x63, 0x68, 0x65, 0x6d, 0x61, 0x74, 0x72, 0x61, 0x6e, 0x73, 0x66, 0x6f, 0x72, 0x6d, 0x3a, 0x6f, + 0x72, 0x67, 0x2e, 0x61, 0x70, 0x61, 0x63, 0x68, 0x65, 0x2e, 0x62, 0x65, 0x61, 0x6d, 0x3a, 0x6b, + 0x61, 0x66, 0x6b, 0x61, 0x5f, 0x72, 0x65, 0x61, 0x64, 0x3a, 0x76, 0x31, 0x12, 0x4a, 0x0a, 0x0b, + 0x4b, 0x41, 0x46, 0x4b, 0x41, 0x5f, 0x57, 0x52, 0x49, 0x54, 0x45, 0x10, 0x03, 0x1a, 0x39, 0xa2, + 0xb4, 0xfa, 0xc2, 0x05, 0x33, 0x62, 0x65, 0x61, 0x6d, 0x3a, 0x73, 0x63, 0x68, 0x65, 0x6d, 0x61, + 0x74, 0x72, 0x61, 0x6e, 0x73, 0x66, 0x6f, 0x72, 0x6d, 0x3a, 0x6f, 0x72, 0x67, 0x2e, 0x61, 0x70, + 0x61, 0x63, 0x68, 0x65, 0x2e, 0x62, 0x65, 0x61, 0x6d, 0x3a, 0x6b, 0x61, 0x66, 0x6b, 0x61, 0x5f, + 0x77, 0x72, 0x69, 0x74, 0x65, 0x3a, 0x76, 0x31, 0x12, 0x56, 0x0a, 0x0d, 0x42, 0x49, 0x47, 0x51, + 0x55, 0x45, 0x52, 0x59, 0x5f, 0x52, 0x45, 0x41, 0x44, 0x10, 0x04, 0x1a, 0x43, 0xa2, 0xb4, 0xfa, + 0xc2, 0x05, 0x3d, 0x62, 0x65, 0x61, 0x6d, 0x3a, 0x73, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x74, 0x72, + 0x61, 0x6e, 0x73, 0x66, 0x6f, 0x72, 0x6d, 0x3a, 0x6f, 0x72, 0x67, 0x2e, 0x61, 0x70, 0x61, 0x63, + 0x68, 0x65, 0x2e, 0x62, 0x65, 0x61, 0x6d, 0x3a, 0x62, 0x69, 0x67, 0x71, 0x75, 0x65, 0x72, 0x79, + 0x5f, 0x73, 0x74, 0x6f, 0x72, 0x61, 0x67, 0x65, 0x5f, 0x72, 0x65, 0x61, 0x64, 0x3a, 0x76, 0x31, + 0x12, 0x50, 0x0a, 0x0e, 0x42, 0x49, 0x47, 0x51, 0x55, 0x45, 0x52, 0x59, 0x5f, 0x57, 0x52, 0x49, + 0x54, 0x45, 0x10, 0x05, 0x1a, 0x3c, 0xa2, 0xb4, 0xfa, 0xc2, 0x05, 0x36, 0x62, 0x65, 0x61, 0x6d, + 0x3a, 0x73, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x74, 0x72, 0x61, 0x6e, 0x73, 0x66, 0x6f, 0x72, 0x6d, + 0x3a, 0x6f, 0x72, 0x67, 0x2e, 0x61, 0x70, 0x61, 0x63, 0x68, 0x65, 0x2e, 0x62, 0x65, 0x61, 0x6d, + 0x3a, 0x62, 0x69, 0x67, 0x71, 0x75, 0x65, 0x72, 0x79, 0x5f, 0x77, 0x72, 0x69, 0x74, 0x65, 0x3a, + 0x76, 0x31, 0x22, 0xcc, 0x02, 0x0a, 0x16, 0x4a, 0x61, 0x76, 0x61, 0x43, 0x6c, 0x61, 0x73, 0x73, + 0x4c, 0x6f, 0x6f, 0x6b, 0x75, 0x70, 0x50, 0x61, 0x79, 0x6c, 0x6f, 0x61, 0x64, 0x12, 0x1d, 0x0a, + 0x0a, 0x63, 0x6c, 0x61, 0x73, 0x73, 0x5f, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, + 0x09, 0x52, 0x09, 0x63, 0x6c, 0x61, 0x73, 0x73, 0x4e, 0x61, 0x6d, 0x65, 0x12, 0x2d, 0x0a, 0x12, + 0x63, 0x6f, 0x6e, 0x73, 0x74, 0x72, 0x75, 0x63, 0x74, 0x6f, 0x72, 0x5f, 0x6d, 0x65, 0x74, 0x68, + 0x6f, 0x64, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x11, 0x63, 0x6f, 0x6e, 0x73, 0x74, 0x72, + 0x75, 0x63, 0x74, 0x6f, 0x72, 0x4d, 0x65, 0x74, 0x68, 0x6f, 0x64, 0x12, 0x58, 0x0a, 0x12, 0x63, + 0x6f, 0x6e, 0x73, 0x74, 0x72, 0x75, 0x63, 0x74, 0x6f, 0x72, 0x5f, 0x73, 0x63, 0x68, 0x65, 0x6d, + 0x61, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x29, 0x2e, 0x6f, 0x72, 0x67, 0x2e, 0x61, 0x70, + 0x61, 0x63, 0x68, 0x65, 0x2e, 0x62, 0x65, 0x61, 0x6d, 0x2e, 0x6d, 0x6f, 0x64, 0x65, 0x6c, 0x2e, + 0x70, 0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x53, 0x63, 0x68, 0x65, + 0x6d, 0x61, 0x52, 0x11, 0x63, 0x6f, 0x6e, 0x73, 0x74, 0x72, 0x75, 0x63, 0x74, 0x6f, 0x72, 0x53, + 0x63, 0x68, 0x65, 0x6d, 0x61, 0x12, 0x2f, 0x0a, 0x13, 0x63, 0x6f, 0x6e, 0x73, 0x74, 0x72, 0x75, + 0x63, 0x74, 0x6f, 0x72, 0x5f, 0x70, 0x61, 0x79, 0x6c, 0x6f, 0x61, 0x64, 0x18, 0x04, 0x20, 0x01, + 0x28, 0x0c, 0x52, 0x12, 0x63, 0x6f, 0x6e, 0x73, 0x74, 0x72, 0x75, 0x63, 0x74, 0x6f, 0x72, 0x50, + 0x61, 0x79, 0x6c, 0x6f, 0x61, 0x64, 0x12, 0x59, 0x0a, 0x0f, 0x62, 0x75, 0x69, 0x6c, 0x64, 0x65, + 0x72, 0x5f, 0x6d, 0x65, 0x74, 0x68, 0x6f, 0x64, 0x73, 0x18, 0x05, 0x20, 0x03, 0x28, 0x0b, 0x32, + 0x30, 0x2e, 0x6f, 0x72, 0x67, 0x2e, 0x61, 0x70, 0x61, 0x63, 0x68, 0x65, 0x2e, 0x62, 0x65, 0x61, + 0x6d, 0x2e, 0x6d, 0x6f, 0x64, 0x65, 0x6c, 0x2e, 0x70, 0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e, 0x65, + 0x2e, 0x76, 0x31, 0x2e, 0x42, 0x75, 0x69, 0x6c, 0x64, 0x65, 0x72, 0x4d, 0x65, 0x74, 0x68, 0x6f, + 0x64, 0x52, 0x0e, 0x62, 0x75, 0x69, 0x6c, 0x64, 0x65, 0x72, 0x4d, 0x65, 0x74, 0x68, 0x6f, 0x64, + 0x73, 0x22, 0x80, 0x01, 0x0a, 0x0d, 0x42, 0x75, 0x69, 0x6c, 0x64, 0x65, 0x72, 0x4d, 0x65, 0x74, + 0x68, 0x6f, 0x64, 0x12, 0x12, 0x0a, 0x04, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, + 0x09, 0x52, 0x04, 0x6e, 0x61, 0x6d, 0x65, 0x12, 0x41, 0x0a, 0x06, 0x73, 0x63, 0x68, 0x65, 0x6d, + 0x61, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x29, 0x2e, 0x6f, 0x72, 0x67, 0x2e, 0x61, 0x70, + 0x61, 0x63, 0x68, 0x65, 0x2e, 0x62, 0x65, 0x61, 0x6d, 0x2e, 0x6d, 0x6f, 0x64, 0x65, 0x6c, 0x2e, + 0x70, 0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x53, 0x63, 0x68, 0x65, + 0x6d, 0x61, 0x52, 0x06, 0x73, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x12, 0x18, 0x0a, 0x07, 0x70, 0x61, + 0x79, 0x6c, 0x6f, 0x61, 0x64, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0c, 0x52, 0x07, 0x70, 0x61, 0x79, + 0x6c, 0x6f, 0x61, 0x64, 0x22, 0xfa, 0x01, 0x0a, 0x0b, 0x41, 0x6e, 0x6e, 0x6f, 0x74, 0x61, 0x74, + 0x69, 0x6f, 0x6e, 0x73, 0x22, 0xea, 0x01, 0x0a, 0x04, 0x45, 0x6e, 0x75, 0x6d, 0x12, 0x24, 0x0a, + 0x0e, 0x43, 0x4f, 0x4e, 0x46, 0x49, 0x47, 0x5f, 0x52, 0x4f, 0x57, 0x5f, 0x4b, 0x45, 0x59, 0x10, + 0x00, 0x1a, 0x10, 0xaa, 0xb4, 0xfa, 0xc2, 0x05, 0x0a, 0x63, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x5f, + 0x72, 0x6f, 0x77, 0x12, 0x32, 0x0a, 0x15, 0x43, 0x4f, 0x4e, 0x46, 0x49, 0x47, 0x5f, 0x52, 0x4f, + 0x57, 0x5f, 0x53, 0x43, 0x48, 0x45, 0x4d, 0x41, 0x5f, 0x4b, 0x45, 0x59, 0x10, 0x01, 0x1a, 0x17, + 0xaa, 0xb4, 0xfa, 0xc2, 0x05, 0x11, 0x63, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x5f, 0x72, 0x6f, 0x77, + 0x5f, 0x73, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x12, 0x36, 0x0a, 0x17, 0x53, 0x43, 0x48, 0x45, 0x4d, + 0x41, 0x54, 0x52, 0x41, 0x4e, 0x53, 0x46, 0x4f, 0x52, 0x4d, 0x5f, 0x55, 0x52, 0x4e, 0x5f, 0x4b, + 0x45, 0x59, 0x10, 0x02, 0x1a, 0x19, 0xaa, 0xb4, 0xfa, 0xc2, 0x05, 0x13, 0x73, 0x63, 0x68, 0x65, + 0x6d, 0x61, 0x74, 0x72, 0x61, 0x6e, 0x73, 0x66, 0x6f, 0x72, 0x6d, 0x5f, 0x75, 0x72, 0x6e, 0x12, + 0x50, 0x0a, 0x24, 0x4d, 0x41, 0x4e, 0x41, 0x47, 0x45, 0x44, 0x5f, 0x55, 0x4e, 0x44, 0x45, 0x52, + 0x4c, 0x59, 0x49, 0x4e, 0x47, 0x5f, 0x54, 0x52, 0x41, 0x4e, 0x53, 0x46, 0x4f, 0x52, 0x4d, 0x5f, + 0x55, 0x52, 0x4e, 0x5f, 0x4b, 0x45, 0x59, 0x10, 0x03, 0x1a, 0x26, 0xaa, 0xb4, 0xfa, 0xc2, 0x05, + 0x20, 0x6d, 0x61, 0x6e, 0x61, 0x67, 0x65, 0x64, 0x5f, 0x75, 0x6e, 0x64, 0x65, 0x72, 0x6c, 0x79, + 0x69, 0x6e, 0x67, 0x5f, 0x74, 0x72, 0x61, 0x6e, 0x73, 0x66, 0x6f, 0x72, 0x6d, 0x5f, 0x75, 0x72, + 0x6e, 0x22, 0xc3, 0x01, 0x0a, 0x16, 0x53, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x54, 0x72, 0x61, 0x6e, + 0x73, 0x66, 0x6f, 0x72, 0x6d, 0x50, 0x61, 0x79, 0x6c, 0x6f, 0x61, 0x64, 0x12, 0x1e, 0x0a, 0x0a, + 0x69, 0x64, 0x65, 0x6e, 0x74, 0x69, 0x66, 0x69, 0x65, 0x72, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, + 0x52, 0x0a, 0x69, 0x64, 0x65, 0x6e, 0x74, 0x69, 0x66, 0x69, 0x65, 0x72, 0x12, 0x5c, 0x0a, 0x14, + 0x63, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x75, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x73, 0x63, + 0x68, 0x65, 0x6d, 0x61, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x29, 0x2e, 0x6f, 0x72, 0x67, + 0x2e, 0x61, 0x70, 0x61, 0x63, 0x68, 0x65, 0x2e, 0x62, 0x65, 0x61, 0x6d, 0x2e, 0x6d, 0x6f, 0x64, + 0x65, 0x6c, 0x2e, 0x70, 0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x53, + 0x63, 0x68, 0x65, 0x6d, 0x61, 0x52, 0x13, 0x63, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x75, 0x72, 0x61, + 0x74, 0x69, 0x6f, 0x6e, 0x53, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x12, 0x2b, 0x0a, 0x11, 0x63, 0x6f, + 0x6e, 0x66, 0x69, 0x67, 0x75, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x72, 0x6f, 0x77, 0x18, + 0x03, 0x20, 0x01, 0x28, 0x0c, 0x52, 0x10, 0x63, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x75, 0x72, 0x61, + 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x6f, 0x77, 0x42, 0x81, 0x01, 0x0a, 0x21, 0x6f, 0x72, 0x67, 0x2e, + 0x61, 0x70, 0x61, 0x63, 0x68, 0x65, 0x2e, 0x62, 0x65, 0x61, 0x6d, 0x2e, 0x6d, 0x6f, 0x64, 0x65, + 0x6c, 0x2e, 0x70, 0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e, 0x65, 0x2e, 0x76, 0x31, 0x42, 0x12, 0x45, + 0x78, 0x74, 0x65, 0x72, 0x6e, 0x61, 0x6c, 0x54, 0x72, 0x61, 0x6e, 0x73, 0x66, 0x6f, 0x72, 0x6d, + 0x73, 0x5a, 0x48, 0x67, 0x69, 0x74, 0x68, 0x75, 0x62, 0x2e, 0x63, 0x6f, 0x6d, 0x2f, 0x61, 0x70, + 0x61, 0x63, 0x68, 0x65, 0x2f, 0x62, 0x65, 0x61, 0x6d, 0x2f, 0x73, 0x64, 0x6b, 0x73, 0x2f, 0x76, + 0x32, 0x2f, 0x67, 0x6f, 0x2f, 0x70, 0x6b, 0x67, 0x2f, 0x62, 0x65, 0x61, 0x6d, 0x2f, 0x6d, 0x6f, + 0x64, 0x65, 0x6c, 0x2f, 0x70, 0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e, 0x65, 0x5f, 0x76, 0x31, 0x3b, + 0x70, 0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e, 0x65, 0x5f, 0x76, 0x31, 0x62, 0x06, 0x70, 0x72, 0x6f, + 0x74, 0x6f, 0x33, +} + +var file_org_apache_beam_model_pipeline_v1_external_transforms_proto_enumTypes = make([]protoimpl.EnumInfo, 3) +var file_org_apache_beam_model_pipeline_v1_external_transforms_proto_msgTypes = make([]protoimpl.MessageInfo, 7) +var file_org_apache_beam_model_pipeline_v1_external_transforms_proto_goTypes = []any{ + (ExpansionMethods_Enum)(0), // 0: org.apache.beam.model.pipeline.v1.ExpansionMethods.Enum + (ManagedTransforms_Urns)(0), // 1: org.apache.beam.model.pipeline.v1.ManagedTransforms.Urns + (Annotations_Enum)(0), // 2: org.apache.beam.model.pipeline.v1.Annotations.Enum + (*ExternalConfigurationPayload)(nil), // 3: org.apache.beam.model.pipeline.v1.ExternalConfigurationPayload + (*ExpansionMethods)(nil), // 4: org.apache.beam.model.pipeline.v1.ExpansionMethods + (*ManagedTransforms)(nil), // 5: org.apache.beam.model.pipeline.v1.ManagedTransforms + (*JavaClassLookupPayload)(nil), // 6: org.apache.beam.model.pipeline.v1.JavaClassLookupPayload + (*BuilderMethod)(nil), // 7: org.apache.beam.model.pipeline.v1.BuilderMethod + (*Annotations)(nil), // 8: org.apache.beam.model.pipeline.v1.Annotations + (*SchemaTransformPayload)(nil), // 9: org.apache.beam.model.pipeline.v1.SchemaTransformPayload + (*Schema)(nil), // 10: org.apache.beam.model.pipeline.v1.Schema +} +var file_org_apache_beam_model_pipeline_v1_external_transforms_proto_depIdxs = []int32{ + 10, // 0: org.apache.beam.model.pipeline.v1.ExternalConfigurationPayload.schema:type_name -> org.apache.beam.model.pipeline.v1.Schema + 10, // 1: org.apache.beam.model.pipeline.v1.JavaClassLookupPayload.constructor_schema:type_name -> org.apache.beam.model.pipeline.v1.Schema + 7, // 2: org.apache.beam.model.pipeline.v1.JavaClassLookupPayload.builder_methods:type_name -> org.apache.beam.model.pipeline.v1.BuilderMethod + 10, // 3: org.apache.beam.model.pipeline.v1.BuilderMethod.schema:type_name -> org.apache.beam.model.pipeline.v1.Schema + 10, // 4: org.apache.beam.model.pipeline.v1.SchemaTransformPayload.configuration_schema:type_name -> org.apache.beam.model.pipeline.v1.Schema + 5, // [5:5] is the sub-list for method output_type + 5, // [5:5] is the sub-list for method input_type + 5, // [5:5] is the sub-list for extension type_name + 5, // [5:5] is the sub-list for extension extendee + 0, // [0:5] is the sub-list for field type_name +} + +func init() { file_org_apache_beam_model_pipeline_v1_external_transforms_proto_init() } +func file_org_apache_beam_model_pipeline_v1_external_transforms_proto_init() { + if File_org_apache_beam_model_pipeline_v1_external_transforms_proto != nil { + return + } + file_org_apache_beam_model_pipeline_v1_schema_proto_init() + file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_init() + type x struct{} + out := protoimpl.TypeBuilder{ + File: protoimpl.DescBuilder{ + GoPackagePath: reflect.TypeOf(x{}).PkgPath(), + RawDescriptor: file_org_apache_beam_model_pipeline_v1_external_transforms_proto_rawDesc, + NumEnums: 3, + NumMessages: 7, + NumExtensions: 0, + NumServices: 0, + }, + GoTypes: file_org_apache_beam_model_pipeline_v1_external_transforms_proto_goTypes, + DependencyIndexes: file_org_apache_beam_model_pipeline_v1_external_transforms_proto_depIdxs, + EnumInfos: file_org_apache_beam_model_pipeline_v1_external_transforms_proto_enumTypes, + MessageInfos: file_org_apache_beam_model_pipeline_v1_external_transforms_proto_msgTypes, + }.Build() + File_org_apache_beam_model_pipeline_v1_external_transforms_proto = out.File + file_org_apache_beam_model_pipeline_v1_external_transforms_proto_rawDesc = nil + file_org_apache_beam_model_pipeline_v1_external_transforms_proto_goTypes = nil + file_org_apache_beam_model_pipeline_v1_external_transforms_proto_depIdxs = nil +} diff --git a/sdks/go/pkg/beam/model/pipeline_v1/metrics.pb.go b/sdks/go/pkg/beam/model/pipeline_v1/metrics.pb.go index 4b5bbf21e6e2..8312015272e6 100644 --- a/sdks/go/pkg/beam/model/pipeline_v1/metrics.pb.go +++ b/sdks/go/pkg/beam/model/pipeline_v1/metrics.pb.go @@ -20,10 +20,12 @@ // Code generated by protoc-gen-go. DO NOT EDIT. // versions: -// protoc-gen-go v1.27.1 +// protoc-gen-go v1.36.0 // protoc v5.27.3 // source: org/apache/beam/model/pipeline/v1/metrics.proto +//go:build !protoopaque + package pipeline_v1 import ( @@ -32,7 +34,6 @@ import ( descriptorpb "google.golang.org/protobuf/types/descriptorpb" timestamppb "google.golang.org/protobuf/types/known/timestamppb" reflect "reflect" - sync "sync" ) const ( @@ -78,7 +79,9 @@ const ( // Represents the smallest set of double values seen across bundles. MonitoringInfoSpecs_USER_BOTTOM_N_DOUBLE MonitoringInfoSpecs_Enum = 9 // Represents a set of strings seen across bundles. - MonitoringInfoSpecs_USER_SET_STRING MonitoringInfoSpecs_Enum = 21 + MonitoringInfoSpecs_USER_SET_STRING MonitoringInfoSpecs_Enum = 21 + // Represents a set of strings seen across bundles. + MonitoringInfoSpecs_USER_BOUNDED_TRIE MonitoringInfoSpecs_Enum = 22 MonitoringInfoSpecs_ELEMENT_COUNT MonitoringInfoSpecs_Enum = 10 MonitoringInfoSpecs_SAMPLED_BYTE_SIZE MonitoringInfoSpecs_Enum = 11 MonitoringInfoSpecs_START_BUNDLE_MSECS MonitoringInfoSpecs_Enum = 12 @@ -115,6 +118,7 @@ var ( 8: "USER_BOTTOM_N_INT64", 9: "USER_BOTTOM_N_DOUBLE", 21: "USER_SET_STRING", + 22: "USER_BOUNDED_TRIE", 10: "ELEMENT_COUNT", 11: "SAMPLED_BYTE_SIZE", 12: "START_BUNDLE_MSECS", @@ -139,6 +143,7 @@ var ( "USER_BOTTOM_N_INT64": 8, "USER_BOTTOM_N_DOUBLE": 9, "USER_SET_STRING": 21, + "USER_BOUNDED_TRIE": 22, "ELEMENT_COUNT": 10, "SAMPLED_BYTE_SIZE": 11, "START_BUNDLE_MSECS": 12, @@ -175,11 +180,6 @@ func (x MonitoringInfoSpecs_Enum) Number() protoreflect.EnumNumber { return protoreflect.EnumNumber(x) } -// Deprecated: Use MonitoringInfoSpecs_Enum.Descriptor instead. -func (MonitoringInfoSpecs_Enum) EnumDescriptor() ([]byte, []int) { - return file_org_apache_beam_model_pipeline_v1_metrics_proto_rawDescGZIP(), []int{2, 0} -} - type MonitoringInfo_MonitoringInfoLabels int32 const ( @@ -304,11 +304,6 @@ func (x MonitoringInfo_MonitoringInfoLabels) Number() protoreflect.EnumNumber { return protoreflect.EnumNumber(x) } -// Deprecated: Use MonitoringInfo_MonitoringInfoLabels.Descriptor instead. -func (MonitoringInfo_MonitoringInfoLabels) EnumDescriptor() ([]byte, []int) { - return file_org_apache_beam_model_pipeline_v1_metrics_proto_rawDescGZIP(), []int{4, 0} -} - type MonitoringInfoTypeUrns_Enum int32 const ( @@ -397,6 +392,10 @@ const ( // - iter: beam:coder:iterable:v1 // - valueX: beam:coder:stringutf8:v1 MonitoringInfoTypeUrns_SET_STRING_TYPE MonitoringInfoTypeUrns_Enum = 11 + // Represents a bounded trie of strings. + // + // Encoding: BoundedTrie proto + MonitoringInfoTypeUrns_BOUNDED_TRIE_TYPE MonitoringInfoTypeUrns_Enum = 12 ) // Enum value maps for MonitoringInfoTypeUrns_Enum. @@ -414,6 +413,7 @@ var ( 9: "BOTTOM_N_DOUBLE_TYPE", 10: "PROGRESS_TYPE", 11: "SET_STRING_TYPE", + 12: "BOUNDED_TRIE_TYPE", } MonitoringInfoTypeUrns_Enum_value = map[string]int32{ "SUM_INT64_TYPE": 0, @@ -428,6 +428,7 @@ var ( "BOTTOM_N_DOUBLE_TYPE": 9, "PROGRESS_TYPE": 10, "SET_STRING_TYPE": 11, + "BOUNDED_TRIE_TYPE": 12, } ) @@ -453,19 +454,11 @@ func (x MonitoringInfoTypeUrns_Enum) Number() protoreflect.EnumNumber { return protoreflect.EnumNumber(x) } -// Deprecated: Use MonitoringInfoTypeUrns_Enum.Descriptor instead. -func (MonitoringInfoTypeUrns_Enum) EnumDescriptor() ([]byte, []int) { - return file_org_apache_beam_model_pipeline_v1_metrics_proto_rawDescGZIP(), []int{5, 0} -} - // A specification for describing a well known MonitoringInfo. // // All specifications are uniquely identified by the urn. type MonitoringInfoSpec struct { - state protoimpl.MessageState - sizeCache protoimpl.SizeCache - unknownFields protoimpl.UnknownFields - + state protoimpl.MessageState `protogen:"hybrid.v1"` // Defines the semantic meaning of the metric or monitored state. // // See MonitoringInfoSpecs.Enum for the set of well known metrics/monitored @@ -479,16 +472,16 @@ type MonitoringInfoSpec struct { RequiredLabels []string `protobuf:"bytes,3,rep,name=required_labels,json=requiredLabels,proto3" json:"required_labels,omitempty"` // Extra non functional parts of the spec for descriptive purposes. // i.e. description, units, etc. - Annotations []*Annotation `protobuf:"bytes,4,rep,name=annotations,proto3" json:"annotations,omitempty"` + Annotations []*Annotation `protobuf:"bytes,4,rep,name=annotations,proto3" json:"annotations,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache } func (x *MonitoringInfoSpec) Reset() { *x = MonitoringInfoSpec{} - if protoimpl.UnsafeEnabled { - mi := &file_org_apache_beam_model_pipeline_v1_metrics_proto_msgTypes[0] - ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) - ms.StoreMessageInfo(mi) - } + mi := &file_org_apache_beam_model_pipeline_v1_metrics_proto_msgTypes[0] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) } func (x *MonitoringInfoSpec) String() string { @@ -499,7 +492,7 @@ func (*MonitoringInfoSpec) ProtoMessage() {} func (x *MonitoringInfoSpec) ProtoReflect() protoreflect.Message { mi := &file_org_apache_beam_model_pipeline_v1_metrics_proto_msgTypes[0] - if protoimpl.UnsafeEnabled && x != nil { + if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { ms.StoreMessageInfo(mi) @@ -509,11 +502,6 @@ func (x *MonitoringInfoSpec) ProtoReflect() protoreflect.Message { return mi.MessageOf(x) } -// Deprecated: Use MonitoringInfoSpec.ProtoReflect.Descriptor instead. -func (*MonitoringInfoSpec) Descriptor() ([]byte, []int) { - return file_org_apache_beam_model_pipeline_v1_metrics_proto_rawDescGZIP(), []int{0} -} - func (x *MonitoringInfoSpec) GetUrn() string { if x != nil { return x.Urn @@ -542,23 +530,66 @@ func (x *MonitoringInfoSpec) GetAnnotations() []*Annotation { return nil } +func (x *MonitoringInfoSpec) SetUrn(v string) { + x.Urn = v +} + +func (x *MonitoringInfoSpec) SetType(v string) { + x.Type = v +} + +func (x *MonitoringInfoSpec) SetRequiredLabels(v []string) { + x.RequiredLabels = v +} + +func (x *MonitoringInfoSpec) SetAnnotations(v []*Annotation) { + x.Annotations = v +} + +type MonitoringInfoSpec_builder struct { + _ [0]func() // Prevents comparability and use of unkeyed literals for the builder. + + // Defines the semantic meaning of the metric or monitored state. + // + // See MonitoringInfoSpecs.Enum for the set of well known metrics/monitored + // state. + Urn string + // Defines the required encoding and aggregation method for the payload. + // + // See MonitoringInfoTypeUrns.Enum for the set of well known types. + Type string + // The list of required labels for the specified urn and type. + RequiredLabels []string + // Extra non functional parts of the spec for descriptive purposes. + // i.e. description, units, etc. + Annotations []*Annotation +} + +func (b0 MonitoringInfoSpec_builder) Build() *MonitoringInfoSpec { + m0 := &MonitoringInfoSpec{} + b, x := &b0, m0 + _, _ = b, x + x.Urn = b.Urn + x.Type = b.Type + x.RequiredLabels = b.RequiredLabels + x.Annotations = b.Annotations + return m0 +} + // The key name and value string of MonitoringInfo annotations. type Annotation struct { - state protoimpl.MessageState - sizeCache protoimpl.SizeCache + state protoimpl.MessageState `protogen:"hybrid.v1"` + Key string `protobuf:"bytes,1,opt,name=key,proto3" json:"key,omitempty"` + Value string `protobuf:"bytes,2,opt,name=value,proto3" json:"value,omitempty"` unknownFields protoimpl.UnknownFields - - Key string `protobuf:"bytes,1,opt,name=key,proto3" json:"key,omitempty"` - Value string `protobuf:"bytes,2,opt,name=value,proto3" json:"value,omitempty"` + sizeCache protoimpl.SizeCache } func (x *Annotation) Reset() { *x = Annotation{} - if protoimpl.UnsafeEnabled { - mi := &file_org_apache_beam_model_pipeline_v1_metrics_proto_msgTypes[1] - ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) - ms.StoreMessageInfo(mi) - } + mi := &file_org_apache_beam_model_pipeline_v1_metrics_proto_msgTypes[1] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) } func (x *Annotation) String() string { @@ -569,7 +600,7 @@ func (*Annotation) ProtoMessage() {} func (x *Annotation) ProtoReflect() protoreflect.Message { mi := &file_org_apache_beam_model_pipeline_v1_metrics_proto_msgTypes[1] - if protoimpl.UnsafeEnabled && x != nil { + if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { ms.StoreMessageInfo(mi) @@ -579,11 +610,6 @@ func (x *Annotation) ProtoReflect() protoreflect.Message { return mi.MessageOf(x) } -// Deprecated: Use Annotation.ProtoReflect.Descriptor instead. -func (*Annotation) Descriptor() ([]byte, []int) { - return file_org_apache_beam_model_pipeline_v1_metrics_proto_rawDescGZIP(), []int{1} -} - func (x *Annotation) GetKey() string { if x != nil { return x.Key @@ -598,20 +624,42 @@ func (x *Annotation) GetValue() string { return "" } +func (x *Annotation) SetKey(v string) { + x.Key = v +} + +func (x *Annotation) SetValue(v string) { + x.Value = v +} + +type Annotation_builder struct { + _ [0]func() // Prevents comparability and use of unkeyed literals for the builder. + + Key string + Value string +} + +func (b0 Annotation_builder) Build() *Annotation { + m0 := &Annotation{} + b, x := &b0, m0 + _, _ = b, x + x.Key = b.Key + x.Value = b.Value + return m0 +} + // A set of well known MonitoringInfo specifications. type MonitoringInfoSpecs struct { - state protoimpl.MessageState - sizeCache protoimpl.SizeCache + state protoimpl.MessageState `protogen:"hybrid.v1"` unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache } func (x *MonitoringInfoSpecs) Reset() { *x = MonitoringInfoSpecs{} - if protoimpl.UnsafeEnabled { - mi := &file_org_apache_beam_model_pipeline_v1_metrics_proto_msgTypes[2] - ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) - ms.StoreMessageInfo(mi) - } + mi := &file_org_apache_beam_model_pipeline_v1_metrics_proto_msgTypes[2] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) } func (x *MonitoringInfoSpecs) String() string { @@ -622,7 +670,7 @@ func (*MonitoringInfoSpecs) ProtoMessage() {} func (x *MonitoringInfoSpecs) ProtoReflect() protoreflect.Message { mi := &file_org_apache_beam_model_pipeline_v1_metrics_proto_msgTypes[2] - if protoimpl.UnsafeEnabled && x != nil { + if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { ms.StoreMessageInfo(mi) @@ -632,29 +680,33 @@ func (x *MonitoringInfoSpecs) ProtoReflect() protoreflect.Message { return mi.MessageOf(x) } -// Deprecated: Use MonitoringInfoSpecs.ProtoReflect.Descriptor instead. -func (*MonitoringInfoSpecs) Descriptor() ([]byte, []int) { - return file_org_apache_beam_model_pipeline_v1_metrics_proto_rawDescGZIP(), []int{2} +type MonitoringInfoSpecs_builder struct { + _ [0]func() // Prevents comparability and use of unkeyed literals for the builder. + +} + +func (b0 MonitoringInfoSpecs_builder) Build() *MonitoringInfoSpecs { + m0 := &MonitoringInfoSpecs{} + b, x := &b0, m0 + _, _ = b, x + return m0 } // A set of properties for the MonitoringInfoLabel, this is useful to obtain // the proper label string for the MonitoringInfoLabel. type MonitoringInfoLabelProps struct { - state protoimpl.MessageState - sizeCache protoimpl.SizeCache - unknownFields protoimpl.UnknownFields - + state protoimpl.MessageState `protogen:"hybrid.v1"` // The label key to use in the MonitoringInfo labels map. - Name string `protobuf:"bytes,1,opt,name=name,proto3" json:"name,omitempty"` + Name string `protobuf:"bytes,1,opt,name=name,proto3" json:"name,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache } func (x *MonitoringInfoLabelProps) Reset() { *x = MonitoringInfoLabelProps{} - if protoimpl.UnsafeEnabled { - mi := &file_org_apache_beam_model_pipeline_v1_metrics_proto_msgTypes[3] - ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) - ms.StoreMessageInfo(mi) - } + mi := &file_org_apache_beam_model_pipeline_v1_metrics_proto_msgTypes[3] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) } func (x *MonitoringInfoLabelProps) String() string { @@ -665,7 +717,7 @@ func (*MonitoringInfoLabelProps) ProtoMessage() {} func (x *MonitoringInfoLabelProps) ProtoReflect() protoreflect.Message { mi := &file_org_apache_beam_model_pipeline_v1_metrics_proto_msgTypes[3] - if protoimpl.UnsafeEnabled && x != nil { + if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { ms.StoreMessageInfo(mi) @@ -675,11 +727,6 @@ func (x *MonitoringInfoLabelProps) ProtoReflect() protoreflect.Message { return mi.MessageOf(x) } -// Deprecated: Use MonitoringInfoLabelProps.ProtoReflect.Descriptor instead. -func (*MonitoringInfoLabelProps) Descriptor() ([]byte, []int) { - return file_org_apache_beam_model_pipeline_v1_metrics_proto_rawDescGZIP(), []int{3} -} - func (x *MonitoringInfoLabelProps) GetName() string { if x != nil { return x.Name @@ -687,11 +734,27 @@ func (x *MonitoringInfoLabelProps) GetName() string { return "" } -type MonitoringInfo struct { - state protoimpl.MessageState - sizeCache protoimpl.SizeCache - unknownFields protoimpl.UnknownFields +func (x *MonitoringInfoLabelProps) SetName(v string) { + x.Name = v +} + +type MonitoringInfoLabelProps_builder struct { + _ [0]func() // Prevents comparability and use of unkeyed literals for the builder. + + // The label key to use in the MonitoringInfo labels map. + Name string +} + +func (b0 MonitoringInfoLabelProps_builder) Build() *MonitoringInfoLabelProps { + m0 := &MonitoringInfoLabelProps{} + b, x := &b0, m0 + _, _ = b, x + x.Name = b.Name + return m0 +} +type MonitoringInfo struct { + state protoimpl.MessageState `protogen:"hybrid.v1"` // (Required) Defines the semantic meaning of the metric or monitored state. // // See MonitoringInfoSpecs.Enum for the set of well known metrics/monitored @@ -716,7 +779,7 @@ type MonitoringInfo struct { // together for all updates having the same URN and labels. Some systems such // as Stackdriver will be able to aggregate the metrics using a subset of the // provided labels - Labels map[string]string `protobuf:"bytes,4,rep,name=labels,proto3" json:"labels,omitempty" protobuf_key:"bytes,1,opt,name=key,proto3" protobuf_val:"bytes,2,opt,name=value,proto3"` + Labels map[string]string `protobuf:"bytes,4,rep,name=labels,proto3" json:"labels,omitempty" protobuf_key:"bytes,1,opt,name=key" protobuf_val:"bytes,2,opt,name=value"` // This indicates the start of the time range over which this value was // measured. // This is needed by some external metric aggregation services @@ -727,16 +790,16 @@ type MonitoringInfo struct { // HarnessMonitoringInfos should set this start_time once, when the // MonitoringInfo is first reported. // ProcessBundle MonitoringInfos should set a start_time for each bundle. - StartTime *timestamppb.Timestamp `protobuf:"bytes,5,opt,name=start_time,json=startTime,proto3" json:"start_time,omitempty"` + StartTime *timestamppb.Timestamp `protobuf:"bytes,5,opt,name=start_time,json=startTime,proto3" json:"start_time,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache } func (x *MonitoringInfo) Reset() { *x = MonitoringInfo{} - if protoimpl.UnsafeEnabled { - mi := &file_org_apache_beam_model_pipeline_v1_metrics_proto_msgTypes[4] - ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) - ms.StoreMessageInfo(mi) - } + mi := &file_org_apache_beam_model_pipeline_v1_metrics_proto_msgTypes[4] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) } func (x *MonitoringInfo) String() string { @@ -747,7 +810,7 @@ func (*MonitoringInfo) ProtoMessage() {} func (x *MonitoringInfo) ProtoReflect() protoreflect.Message { mi := &file_org_apache_beam_model_pipeline_v1_metrics_proto_msgTypes[4] - if protoimpl.UnsafeEnabled && x != nil { + if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { ms.StoreMessageInfo(mi) @@ -757,11 +820,6 @@ func (x *MonitoringInfo) ProtoReflect() protoreflect.Message { return mi.MessageOf(x) } -// Deprecated: Use MonitoringInfo.ProtoReflect.Descriptor instead. -func (*MonitoringInfo) Descriptor() ([]byte, []int) { - return file_org_apache_beam_model_pipeline_v1_metrics_proto_rawDescGZIP(), []int{4} -} - func (x *MonitoringInfo) GetUrn() string { if x != nil { return x.Urn @@ -797,20 +855,105 @@ func (x *MonitoringInfo) GetStartTime() *timestamppb.Timestamp { return nil } +func (x *MonitoringInfo) SetUrn(v string) { + x.Urn = v +} + +func (x *MonitoringInfo) SetType(v string) { + x.Type = v +} + +func (x *MonitoringInfo) SetPayload(v []byte) { + if v == nil { + v = []byte{} + } + x.Payload = v +} + +func (x *MonitoringInfo) SetLabels(v map[string]string) { + x.Labels = v +} + +func (x *MonitoringInfo) SetStartTime(v *timestamppb.Timestamp) { + x.StartTime = v +} + +func (x *MonitoringInfo) HasStartTime() bool { + if x == nil { + return false + } + return x.StartTime != nil +} + +func (x *MonitoringInfo) ClearStartTime() { + x.StartTime = nil +} + +type MonitoringInfo_builder struct { + _ [0]func() // Prevents comparability and use of unkeyed literals for the builder. + + // (Required) Defines the semantic meaning of the metric or monitored state. + // + // See MonitoringInfoSpecs.Enum for the set of well known metrics/monitored + // state. + Urn string + // (Required) Defines the encoding and aggregation method for the payload. + // + // See MonitoringInfoTypeUrns.Enum for the set of well known types. + Type string + // (Required) The metric or monitored state encoded as per the specification + // defined by the type. + Payload []byte + // A set of key and value labels which define the scope of the metric. For + // well known URNs, the set of required labels is provided by the associated + // MonitoringInfoSpec. + // + // Either a well defined entity id for matching the enum names in + // the MonitoringInfoLabels enum or any arbitrary label + // set by a custom metric or user metric. + // + // A monitoring system is expected to be able to aggregate the metrics + // together for all updates having the same URN and labels. Some systems such + // as Stackdriver will be able to aggregate the metrics using a subset of the + // provided labels + Labels map[string]string + // This indicates the start of the time range over which this value was + // measured. + // This is needed by some external metric aggregation services + // to indicate when the reporter of the metric first began collecting the + // cumulative value for the timeseries. + // If the SDK Harness restarts, it should reset the start_time, and reset + // the collection of cumulative metrics (i.e. start to count again from 0). + // HarnessMonitoringInfos should set this start_time once, when the + // MonitoringInfo is first reported. + // ProcessBundle MonitoringInfos should set a start_time for each bundle. + StartTime *timestamppb.Timestamp +} + +func (b0 MonitoringInfo_builder) Build() *MonitoringInfo { + m0 := &MonitoringInfo{} + b, x := &b0, m0 + _, _ = b, x + x.Urn = b.Urn + x.Type = b.Type + x.Payload = b.Payload + x.Labels = b.Labels + x.StartTime = b.StartTime + return m0 +} + // A set of well known URNs that specify the encoding and aggregation method. type MonitoringInfoTypeUrns struct { - state protoimpl.MessageState - sizeCache protoimpl.SizeCache + state protoimpl.MessageState `protogen:"hybrid.v1"` unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache } func (x *MonitoringInfoTypeUrns) Reset() { *x = MonitoringInfoTypeUrns{} - if protoimpl.UnsafeEnabled { - mi := &file_org_apache_beam_model_pipeline_v1_metrics_proto_msgTypes[5] - ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) - ms.StoreMessageInfo(mi) - } + mi := &file_org_apache_beam_model_pipeline_v1_metrics_proto_msgTypes[5] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) } func (x *MonitoringInfoTypeUrns) String() string { @@ -821,7 +964,56 @@ func (*MonitoringInfoTypeUrns) ProtoMessage() {} func (x *MonitoringInfoTypeUrns) ProtoReflect() protoreflect.Message { mi := &file_org_apache_beam_model_pipeline_v1_metrics_proto_msgTypes[5] - if protoimpl.UnsafeEnabled && x != nil { + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +type MonitoringInfoTypeUrns_builder struct { + _ [0]func() // Prevents comparability and use of unkeyed literals for the builder. + +} + +func (b0 MonitoringInfoTypeUrns_builder) Build() *MonitoringInfoTypeUrns { + m0 := &MonitoringInfoTypeUrns{} + b, x := &b0, m0 + _, _ = b, x + return m0 +} + +// A single node in a BoundedTrie. +type BoundedTrieNode struct { + state protoimpl.MessageState `protogen:"hybrid.v1"` + // Whether this node has been truncated. + // A truncated leaf represents possibly many children with the same prefix. + Truncated bool `protobuf:"varint,1,opt,name=truncated,proto3" json:"truncated,omitempty"` + // Children of this node. Must be empty if truncated is true. + Children map[string]*BoundedTrieNode `protobuf:"bytes,2,rep,name=children,proto3" json:"children,omitempty" protobuf_key:"bytes,1,opt,name=key" protobuf_val:"bytes,2,opt,name=value"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *BoundedTrieNode) Reset() { + *x = BoundedTrieNode{} + mi := &file_org_apache_beam_model_pipeline_v1_metrics_proto_msgTypes[6] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *BoundedTrieNode) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*BoundedTrieNode) ProtoMessage() {} + +func (x *BoundedTrieNode) ProtoReflect() protoreflect.Message { + mi := &file_org_apache_beam_model_pipeline_v1_metrics_proto_msgTypes[6] + if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { ms.StoreMessageInfo(mi) @@ -831,9 +1023,148 @@ func (x *MonitoringInfoTypeUrns) ProtoReflect() protoreflect.Message { return mi.MessageOf(x) } -// Deprecated: Use MonitoringInfoTypeUrns.ProtoReflect.Descriptor instead. -func (*MonitoringInfoTypeUrns) Descriptor() ([]byte, []int) { - return file_org_apache_beam_model_pipeline_v1_metrics_proto_rawDescGZIP(), []int{5} +func (x *BoundedTrieNode) GetTruncated() bool { + if x != nil { + return x.Truncated + } + return false +} + +func (x *BoundedTrieNode) GetChildren() map[string]*BoundedTrieNode { + if x != nil { + return x.Children + } + return nil +} + +func (x *BoundedTrieNode) SetTruncated(v bool) { + x.Truncated = v +} + +func (x *BoundedTrieNode) SetChildren(v map[string]*BoundedTrieNode) { + x.Children = v +} + +type BoundedTrieNode_builder struct { + _ [0]func() // Prevents comparability and use of unkeyed literals for the builder. + + // Whether this node has been truncated. + // A truncated leaf represents possibly many children with the same prefix. + Truncated bool + // Children of this node. Must be empty if truncated is true. + Children map[string]*BoundedTrieNode +} + +func (b0 BoundedTrieNode_builder) Build() *BoundedTrieNode { + m0 := &BoundedTrieNode{} + b, x := &b0, m0 + _, _ = b, x + x.Truncated = b.Truncated + x.Children = b.Children + return m0 +} + +// The message type used for encoding metrics of type bounded trie. +type BoundedTrie struct { + state protoimpl.MessageState `protogen:"hybrid.v1"` + // The maximum number of elements to store before truncation. + Bound int32 `protobuf:"varint,1,opt,name=bound,proto3" json:"bound,omitempty"` + // A compact representation of all the elements in this trie. + Root *BoundedTrieNode `protobuf:"bytes,2,opt,name=root,proto3" json:"root,omitempty"` + // A more efficient representation for metrics consisting of a single value. + Singleton []string `protobuf:"bytes,3,rep,name=singleton,proto3" json:"singleton,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *BoundedTrie) Reset() { + *x = BoundedTrie{} + mi := &file_org_apache_beam_model_pipeline_v1_metrics_proto_msgTypes[7] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *BoundedTrie) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*BoundedTrie) ProtoMessage() {} + +func (x *BoundedTrie) ProtoReflect() protoreflect.Message { + mi := &file_org_apache_beam_model_pipeline_v1_metrics_proto_msgTypes[7] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +func (x *BoundedTrie) GetBound() int32 { + if x != nil { + return x.Bound + } + return 0 +} + +func (x *BoundedTrie) GetRoot() *BoundedTrieNode { + if x != nil { + return x.Root + } + return nil +} + +func (x *BoundedTrie) GetSingleton() []string { + if x != nil { + return x.Singleton + } + return nil +} + +func (x *BoundedTrie) SetBound(v int32) { + x.Bound = v +} + +func (x *BoundedTrie) SetRoot(v *BoundedTrieNode) { + x.Root = v +} + +func (x *BoundedTrie) SetSingleton(v []string) { + x.Singleton = v +} + +func (x *BoundedTrie) HasRoot() bool { + if x == nil { + return false + } + return x.Root != nil +} + +func (x *BoundedTrie) ClearRoot() { + x.Root = nil +} + +type BoundedTrie_builder struct { + _ [0]func() // Prevents comparability and use of unkeyed literals for the builder. + + // The maximum number of elements to store before truncation. + Bound int32 + // A compact representation of all the elements in this trie. + Root *BoundedTrieNode + // A more efficient representation for metrics consisting of a single value. + Singleton []string +} + +func (b0 BoundedTrie_builder) Build() *BoundedTrie { + m0 := &BoundedTrie{} + b, x := &b0, m0 + _, _ = b, x + x.Bound = b.Bound + x.Root = b.Root + x.Singleton = b.Singleton + return m0 } var file_org_apache_beam_model_pipeline_v1_metrics_proto_extTypes = []protoimpl.ExtensionInfo{ @@ -895,9 +1226,9 @@ var file_org_apache_beam_model_pipeline_v1_metrics_proto_rawDesc = []byte{ 0x74, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x22, 0x34, 0x0a, 0x0a, 0x41, 0x6e, 0x6e, 0x6f, 0x74, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x10, 0x0a, 0x03, 0x6b, 0x65, 0x79, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x03, 0x6b, 0x65, 0x79, 0x12, 0x14, 0x0a, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, - 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x22, 0xfe, 0x24, + 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x22, 0xb1, 0x26, 0x0a, 0x13, 0x4d, 0x6f, 0x6e, 0x69, 0x74, 0x6f, 0x72, 0x69, 0x6e, 0x67, 0x49, 0x6e, 0x66, 0x6f, - 0x53, 0x70, 0x65, 0x63, 0x73, 0x22, 0xe6, 0x24, 0x0a, 0x04, 0x45, 0x6e, 0x75, 0x6d, 0x12, 0xa7, + 0x53, 0x70, 0x65, 0x63, 0x73, 0x22, 0x99, 0x26, 0x0a, 0x04, 0x45, 0x6e, 0x75, 0x6d, 0x12, 0xa7, 0x01, 0x0a, 0x0e, 0x55, 0x53, 0x45, 0x52, 0x5f, 0x53, 0x55, 0x4d, 0x5f, 0x49, 0x4e, 0x54, 0x36, 0x34, 0x10, 0x00, 0x1a, 0x92, 0x01, 0xd2, 0xa7, 0xa7, 0x96, 0x06, 0x8b, 0x01, 0x0a, 0x1d, 0x62, 0x65, 0x61, 0x6d, 0x3a, 0x6d, 0x65, 0x74, 0x72, 0x69, 0x63, 0x3a, 0x75, 0x73, 0x65, 0x72, 0x3a, @@ -1023,361 +1354,389 @@ var file_org_apache_beam_model_pipeline_v1_metrics_proto_rawDesc = []byte{ 0x64, 0x65, 0x73, 0x63, 0x72, 0x69, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x23, 0x55, 0x52, 0x4e, 0x20, 0x75, 0x74, 0x69, 0x6c, 0x69, 0x7a, 0x65, 0x64, 0x20, 0x74, 0x6f, 0x20, 0x72, 0x65, 0x70, 0x6f, 0x72, 0x74, 0x20, 0x75, 0x73, 0x65, 0x72, 0x20, 0x6d, 0x65, 0x74, 0x72, 0x69, 0x63, 0x2e, - 0x12, 0xad, 0x01, 0x0a, 0x0d, 0x45, 0x4c, 0x45, 0x4d, 0x45, 0x4e, 0x54, 0x5f, 0x43, 0x4f, 0x55, - 0x4e, 0x54, 0x10, 0x0a, 0x1a, 0x99, 0x01, 0xd2, 0xa7, 0xa7, 0x96, 0x06, 0x92, 0x01, 0x0a, 0x1c, - 0x62, 0x65, 0x61, 0x6d, 0x3a, 0x6d, 0x65, 0x74, 0x72, 0x69, 0x63, 0x3a, 0x65, 0x6c, 0x65, 0x6d, - 0x65, 0x6e, 0x74, 0x5f, 0x63, 0x6f, 0x75, 0x6e, 0x74, 0x3a, 0x76, 0x31, 0x12, 0x19, 0x62, 0x65, - 0x61, 0x6d, 0x3a, 0x6d, 0x65, 0x74, 0x72, 0x69, 0x63, 0x73, 0x3a, 0x73, 0x75, 0x6d, 0x5f, 0x69, - 0x6e, 0x74, 0x36, 0x34, 0x3a, 0x76, 0x31, 0x1a, 0x0b, 0x50, 0x43, 0x4f, 0x4c, 0x4c, 0x45, 0x43, - 0x54, 0x49, 0x4f, 0x4e, 0x22, 0x4a, 0x0a, 0x0b, 0x64, 0x65, 0x73, 0x63, 0x72, 0x69, 0x70, 0x74, - 0x69, 0x6f, 0x6e, 0x12, 0x3b, 0x54, 0x68, 0x65, 0x20, 0x74, 0x6f, 0x74, 0x61, 0x6c, 0x20, 0x65, - 0x6c, 0x65, 0x6d, 0x65, 0x6e, 0x74, 0x73, 0x20, 0x6f, 0x75, 0x74, 0x70, 0x75, 0x74, 0x20, 0x74, - 0x6f, 0x20, 0x61, 0x20, 0x50, 0x63, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x20, - 0x62, 0x79, 0x20, 0x61, 0x20, 0x50, 0x54, 0x72, 0x61, 0x6e, 0x73, 0x66, 0x6f, 0x72, 0x6d, 0x2e, - 0x12, 0xcd, 0x02, 0x0a, 0x11, 0x53, 0x41, 0x4d, 0x50, 0x4c, 0x45, 0x44, 0x5f, 0x42, 0x59, 0x54, - 0x45, 0x5f, 0x53, 0x49, 0x5a, 0x45, 0x10, 0x0b, 0x1a, 0xb5, 0x02, 0xd2, 0xa7, 0xa7, 0x96, 0x06, - 0xae, 0x02, 0x0a, 0x20, 0x62, 0x65, 0x61, 0x6d, 0x3a, 0x6d, 0x65, 0x74, 0x72, 0x69, 0x63, 0x3a, - 0x73, 0x61, 0x6d, 0x70, 0x6c, 0x65, 0x64, 0x5f, 0x62, 0x79, 0x74, 0x65, 0x5f, 0x73, 0x69, 0x7a, - 0x65, 0x3a, 0x76, 0x31, 0x12, 0x22, 0x62, 0x65, 0x61, 0x6d, 0x3a, 0x6d, 0x65, 0x74, 0x72, 0x69, - 0x63, 0x73, 0x3a, 0x64, 0x69, 0x73, 0x74, 0x72, 0x69, 0x62, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x5f, - 0x69, 0x6e, 0x74, 0x36, 0x34, 0x3a, 0x76, 0x31, 0x1a, 0x0b, 0x50, 0x43, 0x4f, 0x4c, 0x4c, 0x45, - 0x43, 0x54, 0x49, 0x4f, 0x4e, 0x22, 0xd8, 0x01, 0x0a, 0x0b, 0x64, 0x65, 0x73, 0x63, 0x72, 0x69, - 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0xc8, 0x01, 0x54, 0x68, 0x65, 0x20, 0x74, 0x6f, 0x74, 0x61, - 0x6c, 0x20, 0x62, 0x79, 0x74, 0x65, 0x20, 0x73, 0x69, 0x7a, 0x65, 0x20, 0x61, 0x6e, 0x64, 0x20, - 0x63, 0x6f, 0x75, 0x6e, 0x74, 0x20, 0x6f, 0x66, 0x20, 0x61, 0x20, 0x73, 0x61, 0x6d, 0x70, 0x6c, - 0x65, 0x64, 0x20, 0x20, 0x73, 0x65, 0x74, 0x20, 0x28, 0x6f, 0x72, 0x20, 0x61, 0x6c, 0x6c, 0x29, - 0x20, 0x6f, 0x66, 0x20, 0x65, 0x6c, 0x65, 0x6d, 0x65, 0x6e, 0x74, 0x73, 0x20, 0x69, 0x6e, 0x20, - 0x74, 0x68, 0x65, 0x20, 0x70, 0x63, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x2e, - 0x20, 0x53, 0x61, 0x6d, 0x70, 0x6c, 0x69, 0x6e, 0x67, 0x20, 0x69, 0x73, 0x20, 0x75, 0x73, 0x65, - 0x64, 0x20, 0x20, 0x62, 0x65, 0x63, 0x61, 0x75, 0x73, 0x65, 0x20, 0x63, 0x61, 0x6c, 0x63, 0x75, - 0x6c, 0x61, 0x74, 0x69, 0x6e, 0x67, 0x20, 0x74, 0x68, 0x65, 0x20, 0x62, 0x79, 0x74, 0x65, 0x20, - 0x63, 0x6f, 0x75, 0x6e, 0x74, 0x20, 0x69, 0x6e, 0x76, 0x6f, 0x6c, 0x76, 0x65, 0x73, 0x20, 0x73, - 0x65, 0x72, 0x69, 0x61, 0x6c, 0x69, 0x7a, 0x69, 0x6e, 0x67, 0x20, 0x74, 0x68, 0x65, 0x20, 0x20, - 0x65, 0x6c, 0x65, 0x6d, 0x65, 0x6e, 0x74, 0x73, 0x20, 0x77, 0x68, 0x69, 0x63, 0x68, 0x20, 0x69, - 0x73, 0x20, 0x43, 0x50, 0x55, 0x20, 0x69, 0x6e, 0x74, 0x65, 0x6e, 0x73, 0x69, 0x76, 0x65, 0x2e, - 0x12, 0xd9, 0x01, 0x0a, 0x12, 0x53, 0x54, 0x41, 0x52, 0x54, 0x5f, 0x42, 0x55, 0x4e, 0x44, 0x4c, - 0x45, 0x5f, 0x4d, 0x53, 0x45, 0x43, 0x53, 0x10, 0x0c, 0x1a, 0xc0, 0x01, 0xd2, 0xa7, 0xa7, 0x96, - 0x06, 0xb9, 0x01, 0x0a, 0x36, 0x62, 0x65, 0x61, 0x6d, 0x3a, 0x6d, 0x65, 0x74, 0x72, 0x69, 0x63, - 0x3a, 0x70, 0x61, 0x72, 0x64, 0x6f, 0x5f, 0x65, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, - 0x5f, 0x74, 0x69, 0x6d, 0x65, 0x3a, 0x73, 0x74, 0x61, 0x72, 0x74, 0x5f, 0x62, 0x75, 0x6e, 0x64, - 0x6c, 0x65, 0x5f, 0x6d, 0x73, 0x65, 0x63, 0x73, 0x3a, 0x76, 0x31, 0x12, 0x19, 0x62, 0x65, 0x61, - 0x6d, 0x3a, 0x6d, 0x65, 0x74, 0x72, 0x69, 0x63, 0x73, 0x3a, 0x73, 0x75, 0x6d, 0x5f, 0x69, 0x6e, - 0x74, 0x36, 0x34, 0x3a, 0x76, 0x31, 0x1a, 0x0a, 0x50, 0x54, 0x52, 0x41, 0x4e, 0x53, 0x46, 0x4f, - 0x52, 0x4d, 0x22, 0x58, 0x0a, 0x0b, 0x64, 0x65, 0x73, 0x63, 0x72, 0x69, 0x70, 0x74, 0x69, 0x6f, - 0x6e, 0x12, 0x49, 0x54, 0x68, 0x65, 0x20, 0x74, 0x6f, 0x74, 0x61, 0x6c, 0x20, 0x65, 0x73, 0x74, - 0x69, 0x6d, 0x61, 0x74, 0x65, 0x64, 0x20, 0x65, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, - 0x20, 0x74, 0x69, 0x6d, 0x65, 0x20, 0x6f, 0x66, 0x20, 0x74, 0x68, 0x65, 0x20, 0x73, 0x74, 0x61, - 0x72, 0x74, 0x20, 0x62, 0x75, 0x6e, 0x64, 0x6c, 0x65, 0x66, 0x75, 0x6e, 0x63, 0x74, 0x69, 0x6f, - 0x6e, 0x20, 0x69, 0x6e, 0x20, 0x61, 0x20, 0x70, 0x61, 0x72, 0x64, 0x6f, 0x12, 0xdf, 0x01, 0x0a, - 0x14, 0x50, 0x52, 0x4f, 0x43, 0x45, 0x53, 0x53, 0x5f, 0x42, 0x55, 0x4e, 0x44, 0x4c, 0x45, 0x5f, - 0x4d, 0x53, 0x45, 0x43, 0x53, 0x10, 0x0d, 0x1a, 0xc4, 0x01, 0xd2, 0xa7, 0xa7, 0x96, 0x06, 0xbd, - 0x01, 0x0a, 0x38, 0x62, 0x65, 0x61, 0x6d, 0x3a, 0x6d, 0x65, 0x74, 0x72, 0x69, 0x63, 0x3a, 0x70, - 0x61, 0x72, 0x64, 0x6f, 0x5f, 0x65, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x74, - 0x69, 0x6d, 0x65, 0x3a, 0x70, 0x72, 0x6f, 0x63, 0x65, 0x73, 0x73, 0x5f, 0x62, 0x75, 0x6e, 0x64, - 0x6c, 0x65, 0x5f, 0x6d, 0x73, 0x65, 0x63, 0x73, 0x3a, 0x76, 0x31, 0x12, 0x19, 0x62, 0x65, 0x61, - 0x6d, 0x3a, 0x6d, 0x65, 0x74, 0x72, 0x69, 0x63, 0x73, 0x3a, 0x73, 0x75, 0x6d, 0x5f, 0x69, 0x6e, - 0x74, 0x36, 0x34, 0x3a, 0x76, 0x31, 0x1a, 0x0a, 0x50, 0x54, 0x52, 0x41, 0x4e, 0x53, 0x46, 0x4f, - 0x52, 0x4d, 0x22, 0x5a, 0x0a, 0x0b, 0x64, 0x65, 0x73, 0x63, 0x72, 0x69, 0x70, 0x74, 0x69, 0x6f, - 0x6e, 0x12, 0x4b, 0x54, 0x68, 0x65, 0x20, 0x74, 0x6f, 0x74, 0x61, 0x6c, 0x20, 0x65, 0x73, 0x74, - 0x69, 0x6d, 0x61, 0x74, 0x65, 0x64, 0x20, 0x65, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, - 0x20, 0x74, 0x69, 0x6d, 0x65, 0x20, 0x6f, 0x66, 0x20, 0x74, 0x68, 0x65, 0x20, 0x70, 0x72, 0x6f, - 0x63, 0x65, 0x73, 0x73, 0x20, 0x62, 0x75, 0x6e, 0x64, 0x6c, 0x65, 0x66, 0x75, 0x6e, 0x63, 0x74, - 0x69, 0x6f, 0x6e, 0x20, 0x69, 0x6e, 0x20, 0x61, 0x20, 0x70, 0x61, 0x72, 0x64, 0x6f, 0x12, 0xdd, - 0x01, 0x0a, 0x13, 0x46, 0x49, 0x4e, 0x49, 0x53, 0x48, 0x5f, 0x42, 0x55, 0x4e, 0x44, 0x4c, 0x45, - 0x5f, 0x4d, 0x53, 0x45, 0x43, 0x53, 0x10, 0x0e, 0x1a, 0xc3, 0x01, 0xd2, 0xa7, 0xa7, 0x96, 0x06, - 0xbc, 0x01, 0x0a, 0x37, 0x62, 0x65, 0x61, 0x6d, 0x3a, 0x6d, 0x65, 0x74, 0x72, 0x69, 0x63, 0x3a, - 0x70, 0x61, 0x72, 0x64, 0x6f, 0x5f, 0x65, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x5f, - 0x74, 0x69, 0x6d, 0x65, 0x3a, 0x66, 0x69, 0x6e, 0x69, 0x73, 0x68, 0x5f, 0x62, 0x75, 0x6e, 0x64, - 0x6c, 0x65, 0x5f, 0x6d, 0x73, 0x65, 0x63, 0x73, 0x3a, 0x76, 0x31, 0x12, 0x19, 0x62, 0x65, 0x61, - 0x6d, 0x3a, 0x6d, 0x65, 0x74, 0x72, 0x69, 0x63, 0x73, 0x3a, 0x73, 0x75, 0x6d, 0x5f, 0x69, 0x6e, - 0x74, 0x36, 0x34, 0x3a, 0x76, 0x31, 0x1a, 0x0a, 0x50, 0x54, 0x52, 0x41, 0x4e, 0x53, 0x46, 0x4f, - 0x52, 0x4d, 0x22, 0x5a, 0x0a, 0x0b, 0x64, 0x65, 0x73, 0x63, 0x72, 0x69, 0x70, 0x74, 0x69, 0x6f, - 0x6e, 0x12, 0x4b, 0x54, 0x68, 0x65, 0x20, 0x74, 0x6f, 0x74, 0x61, 0x6c, 0x20, 0x65, 0x73, 0x74, - 0x69, 0x6d, 0x61, 0x74, 0x65, 0x64, 0x20, 0x65, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, - 0x20, 0x74, 0x69, 0x6d, 0x65, 0x20, 0x6f, 0x66, 0x20, 0x74, 0x68, 0x65, 0x20, 0x66, 0x69, 0x6e, - 0x69, 0x73, 0x68, 0x20, 0x62, 0x75, 0x6e, 0x64, 0x6c, 0x65, 0x20, 0x66, 0x75, 0x6e, 0x63, 0x74, - 0x69, 0x6f, 0x6e, 0x20, 0x69, 0x6e, 0x20, 0x61, 0x20, 0x70, 0x61, 0x72, 0x64, 0x6f, 0x12, 0xbb, - 0x01, 0x0a, 0x0b, 0x54, 0x4f, 0x54, 0x41, 0x4c, 0x5f, 0x4d, 0x53, 0x45, 0x43, 0x53, 0x10, 0x0f, - 0x1a, 0xa9, 0x01, 0xd2, 0xa7, 0xa7, 0x96, 0x06, 0xa2, 0x01, 0x0a, 0x34, 0x62, 0x65, 0x61, 0x6d, - 0x3a, 0x6d, 0x65, 0x74, 0x72, 0x69, 0x63, 0x3a, 0x70, 0x74, 0x72, 0x61, 0x6e, 0x73, 0x66, 0x6f, - 0x72, 0x6d, 0x5f, 0x65, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x74, 0x69, 0x6d, - 0x65, 0x3a, 0x74, 0x6f, 0x74, 0x61, 0x6c, 0x5f, 0x6d, 0x73, 0x65, 0x63, 0x73, 0x3a, 0x76, 0x31, - 0x12, 0x19, 0x62, 0x65, 0x61, 0x6d, 0x3a, 0x6d, 0x65, 0x74, 0x72, 0x69, 0x63, 0x73, 0x3a, 0x73, - 0x75, 0x6d, 0x5f, 0x69, 0x6e, 0x74, 0x36, 0x34, 0x3a, 0x76, 0x31, 0x1a, 0x0a, 0x50, 0x54, 0x52, - 0x41, 0x4e, 0x53, 0x46, 0x4f, 0x52, 0x4d, 0x22, 0x43, 0x0a, 0x0b, 0x64, 0x65, 0x73, 0x63, 0x72, - 0x69, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x34, 0x54, 0x68, 0x65, 0x20, 0x74, 0x6f, 0x74, 0x61, - 0x6c, 0x20, 0x65, 0x73, 0x74, 0x69, 0x6d, 0x61, 0x74, 0x65, 0x64, 0x20, 0x65, 0x78, 0x65, 0x63, - 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x20, 0x74, 0x69, 0x6d, 0x65, 0x20, 0x6f, 0x66, 0x20, 0x74, 0x68, - 0x65, 0x20, 0x70, 0x74, 0x72, 0x61, 0x6e, 0x73, 0x66, 0x6f, 0x72, 0x6d, 0x12, 0x9f, 0x02, 0x0a, - 0x0e, 0x57, 0x4f, 0x52, 0x4b, 0x5f, 0x52, 0x45, 0x4d, 0x41, 0x49, 0x4e, 0x49, 0x4e, 0x47, 0x10, - 0x10, 0x1a, 0x8a, 0x02, 0xd2, 0xa7, 0xa7, 0x96, 0x06, 0x83, 0x02, 0x0a, 0x2c, 0x62, 0x65, 0x61, - 0x6d, 0x3a, 0x6d, 0x65, 0x74, 0x72, 0x69, 0x63, 0x3a, 0x70, 0x74, 0x72, 0x61, 0x6e, 0x73, 0x66, - 0x6f, 0x72, 0x6d, 0x5f, 0x70, 0x72, 0x6f, 0x67, 0x72, 0x65, 0x73, 0x73, 0x3a, 0x72, 0x65, 0x6d, - 0x61, 0x69, 0x6e, 0x69, 0x6e, 0x67, 0x3a, 0x76, 0x31, 0x12, 0x18, 0x62, 0x65, 0x61, 0x6d, 0x3a, - 0x6d, 0x65, 0x74, 0x72, 0x69, 0x63, 0x73, 0x3a, 0x70, 0x72, 0x6f, 0x67, 0x72, 0x65, 0x73, 0x73, - 0x3a, 0x76, 0x31, 0x1a, 0x0a, 0x50, 0x54, 0x52, 0x41, 0x4e, 0x53, 0x46, 0x4f, 0x52, 0x4d, 0x22, - 0xac, 0x01, 0x0a, 0x0b, 0x64, 0x65, 0x73, 0x63, 0x72, 0x69, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x12, - 0x9c, 0x01, 0x54, 0x68, 0x65, 0x20, 0x72, 0x65, 0x6d, 0x61, 0x69, 0x6e, 0x69, 0x6e, 0x67, 0x20, - 0x61, 0x6d, 0x6f, 0x75, 0x6e, 0x74, 0x20, 0x6f, 0x66, 0x20, 0x77, 0x6f, 0x72, 0x6b, 0x20, 0x66, - 0x6f, 0x72, 0x20, 0x65, 0x61, 0x63, 0x68, 0x20, 0x61, 0x63, 0x74, 0x69, 0x76, 0x65, 0x20, 0x65, - 0x6c, 0x65, 0x6d, 0x65, 0x6e, 0x74, 0x2e, 0x20, 0x45, 0x61, 0x63, 0x68, 0x20, 0x61, 0x63, 0x74, - 0x69, 0x76, 0x65, 0x20, 0x65, 0x6c, 0x65, 0x6d, 0x65, 0x6e, 0x74, 0x20, 0x72, 0x65, 0x70, 0x72, - 0x65, 0x73, 0x65, 0x6e, 0x74, 0x73, 0x20, 0x61, 0x6e, 0x20, 0x69, 0x6e, 0x64, 0x65, 0x70, 0x65, - 0x6e, 0x64, 0x65, 0x6e, 0x74, 0x20, 0x61, 0x6d, 0x6f, 0x75, 0x6e, 0x74, 0x20, 0x6f, 0x66, 0x20, - 0x77, 0x6f, 0x72, 0x6b, 0x20, 0x6e, 0x6f, 0x74, 0x20, 0x73, 0x68, 0x61, 0x72, 0x65, 0x64, 0x20, - 0x77, 0x69, 0x74, 0x68, 0x20, 0x61, 0x6e, 0x79, 0x20, 0x6f, 0x74, 0x68, 0x65, 0x72, 0x20, 0x61, - 0x63, 0x74, 0x69, 0x76, 0x65, 0x20, 0x65, 0x6c, 0x65, 0x6d, 0x65, 0x6e, 0x74, 0x2e, 0x12, 0x9f, - 0x02, 0x0a, 0x0e, 0x57, 0x4f, 0x52, 0x4b, 0x5f, 0x43, 0x4f, 0x4d, 0x50, 0x4c, 0x45, 0x54, 0x45, - 0x44, 0x10, 0x11, 0x1a, 0x8a, 0x02, 0xd2, 0xa7, 0xa7, 0x96, 0x06, 0x83, 0x02, 0x0a, 0x2c, 0x62, + 0x12, 0xb0, 0x01, 0x0a, 0x11, 0x55, 0x53, 0x45, 0x52, 0x5f, 0x42, 0x4f, 0x55, 0x4e, 0x44, 0x45, + 0x44, 0x5f, 0x54, 0x52, 0x49, 0x45, 0x10, 0x16, 0x1a, 0x98, 0x01, 0xd2, 0xa7, 0xa7, 0x96, 0x06, + 0x91, 0x01, 0x0a, 0x20, 0x62, 0x65, 0x61, 0x6d, 0x3a, 0x6d, 0x65, 0x74, 0x72, 0x69, 0x63, 0x3a, + 0x75, 0x73, 0x65, 0x72, 0x3a, 0x62, 0x6f, 0x75, 0x6e, 0x64, 0x65, 0x64, 0x5f, 0x74, 0x72, 0x69, + 0x65, 0x3a, 0x76, 0x31, 0x12, 0x1c, 0x62, 0x65, 0x61, 0x6d, 0x3a, 0x6d, 0x65, 0x74, 0x72, 0x69, + 0x63, 0x73, 0x3a, 0x62, 0x6f, 0x75, 0x6e, 0x64, 0x65, 0x64, 0x5f, 0x74, 0x72, 0x69, 0x65, 0x3a, + 0x76, 0x31, 0x1a, 0x0a, 0x50, 0x54, 0x52, 0x41, 0x4e, 0x53, 0x46, 0x4f, 0x52, 0x4d, 0x1a, 0x09, + 0x4e, 0x41, 0x4d, 0x45, 0x53, 0x50, 0x41, 0x43, 0x45, 0x1a, 0x04, 0x4e, 0x41, 0x4d, 0x45, 0x22, + 0x32, 0x0a, 0x0b, 0x64, 0x65, 0x73, 0x63, 0x72, 0x69, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x23, + 0x55, 0x52, 0x4e, 0x20, 0x75, 0x74, 0x69, 0x6c, 0x69, 0x7a, 0x65, 0x64, 0x20, 0x74, 0x6f, 0x20, + 0x72, 0x65, 0x70, 0x6f, 0x72, 0x74, 0x20, 0x75, 0x73, 0x65, 0x72, 0x20, 0x6d, 0x65, 0x74, 0x72, + 0x69, 0x63, 0x2e, 0x12, 0xad, 0x01, 0x0a, 0x0d, 0x45, 0x4c, 0x45, 0x4d, 0x45, 0x4e, 0x54, 0x5f, + 0x43, 0x4f, 0x55, 0x4e, 0x54, 0x10, 0x0a, 0x1a, 0x99, 0x01, 0xd2, 0xa7, 0xa7, 0x96, 0x06, 0x92, + 0x01, 0x0a, 0x1c, 0x62, 0x65, 0x61, 0x6d, 0x3a, 0x6d, 0x65, 0x74, 0x72, 0x69, 0x63, 0x3a, 0x65, + 0x6c, 0x65, 0x6d, 0x65, 0x6e, 0x74, 0x5f, 0x63, 0x6f, 0x75, 0x6e, 0x74, 0x3a, 0x76, 0x31, 0x12, + 0x19, 0x62, 0x65, 0x61, 0x6d, 0x3a, 0x6d, 0x65, 0x74, 0x72, 0x69, 0x63, 0x73, 0x3a, 0x73, 0x75, + 0x6d, 0x5f, 0x69, 0x6e, 0x74, 0x36, 0x34, 0x3a, 0x76, 0x31, 0x1a, 0x0b, 0x50, 0x43, 0x4f, 0x4c, + 0x4c, 0x45, 0x43, 0x54, 0x49, 0x4f, 0x4e, 0x22, 0x4a, 0x0a, 0x0b, 0x64, 0x65, 0x73, 0x63, 0x72, + 0x69, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x3b, 0x54, 0x68, 0x65, 0x20, 0x74, 0x6f, 0x74, 0x61, + 0x6c, 0x20, 0x65, 0x6c, 0x65, 0x6d, 0x65, 0x6e, 0x74, 0x73, 0x20, 0x6f, 0x75, 0x74, 0x70, 0x75, + 0x74, 0x20, 0x74, 0x6f, 0x20, 0x61, 0x20, 0x50, 0x63, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, + 0x6f, 0x6e, 0x20, 0x62, 0x79, 0x20, 0x61, 0x20, 0x50, 0x54, 0x72, 0x61, 0x6e, 0x73, 0x66, 0x6f, + 0x72, 0x6d, 0x2e, 0x12, 0xcd, 0x02, 0x0a, 0x11, 0x53, 0x41, 0x4d, 0x50, 0x4c, 0x45, 0x44, 0x5f, + 0x42, 0x59, 0x54, 0x45, 0x5f, 0x53, 0x49, 0x5a, 0x45, 0x10, 0x0b, 0x1a, 0xb5, 0x02, 0xd2, 0xa7, + 0xa7, 0x96, 0x06, 0xae, 0x02, 0x0a, 0x20, 0x62, 0x65, 0x61, 0x6d, 0x3a, 0x6d, 0x65, 0x74, 0x72, + 0x69, 0x63, 0x3a, 0x73, 0x61, 0x6d, 0x70, 0x6c, 0x65, 0x64, 0x5f, 0x62, 0x79, 0x74, 0x65, 0x5f, + 0x73, 0x69, 0x7a, 0x65, 0x3a, 0x76, 0x31, 0x12, 0x22, 0x62, 0x65, 0x61, 0x6d, 0x3a, 0x6d, 0x65, + 0x74, 0x72, 0x69, 0x63, 0x73, 0x3a, 0x64, 0x69, 0x73, 0x74, 0x72, 0x69, 0x62, 0x75, 0x74, 0x69, + 0x6f, 0x6e, 0x5f, 0x69, 0x6e, 0x74, 0x36, 0x34, 0x3a, 0x76, 0x31, 0x1a, 0x0b, 0x50, 0x43, 0x4f, + 0x4c, 0x4c, 0x45, 0x43, 0x54, 0x49, 0x4f, 0x4e, 0x22, 0xd8, 0x01, 0x0a, 0x0b, 0x64, 0x65, 0x73, + 0x63, 0x72, 0x69, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0xc8, 0x01, 0x54, 0x68, 0x65, 0x20, 0x74, + 0x6f, 0x74, 0x61, 0x6c, 0x20, 0x62, 0x79, 0x74, 0x65, 0x20, 0x73, 0x69, 0x7a, 0x65, 0x20, 0x61, + 0x6e, 0x64, 0x20, 0x63, 0x6f, 0x75, 0x6e, 0x74, 0x20, 0x6f, 0x66, 0x20, 0x61, 0x20, 0x73, 0x61, + 0x6d, 0x70, 0x6c, 0x65, 0x64, 0x20, 0x20, 0x73, 0x65, 0x74, 0x20, 0x28, 0x6f, 0x72, 0x20, 0x61, + 0x6c, 0x6c, 0x29, 0x20, 0x6f, 0x66, 0x20, 0x65, 0x6c, 0x65, 0x6d, 0x65, 0x6e, 0x74, 0x73, 0x20, + 0x69, 0x6e, 0x20, 0x74, 0x68, 0x65, 0x20, 0x70, 0x63, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, + 0x6f, 0x6e, 0x2e, 0x20, 0x53, 0x61, 0x6d, 0x70, 0x6c, 0x69, 0x6e, 0x67, 0x20, 0x69, 0x73, 0x20, + 0x75, 0x73, 0x65, 0x64, 0x20, 0x20, 0x62, 0x65, 0x63, 0x61, 0x75, 0x73, 0x65, 0x20, 0x63, 0x61, + 0x6c, 0x63, 0x75, 0x6c, 0x61, 0x74, 0x69, 0x6e, 0x67, 0x20, 0x74, 0x68, 0x65, 0x20, 0x62, 0x79, + 0x74, 0x65, 0x20, 0x63, 0x6f, 0x75, 0x6e, 0x74, 0x20, 0x69, 0x6e, 0x76, 0x6f, 0x6c, 0x76, 0x65, + 0x73, 0x20, 0x73, 0x65, 0x72, 0x69, 0x61, 0x6c, 0x69, 0x7a, 0x69, 0x6e, 0x67, 0x20, 0x74, 0x68, + 0x65, 0x20, 0x20, 0x65, 0x6c, 0x65, 0x6d, 0x65, 0x6e, 0x74, 0x73, 0x20, 0x77, 0x68, 0x69, 0x63, + 0x68, 0x20, 0x69, 0x73, 0x20, 0x43, 0x50, 0x55, 0x20, 0x69, 0x6e, 0x74, 0x65, 0x6e, 0x73, 0x69, + 0x76, 0x65, 0x2e, 0x12, 0xd9, 0x01, 0x0a, 0x12, 0x53, 0x54, 0x41, 0x52, 0x54, 0x5f, 0x42, 0x55, + 0x4e, 0x44, 0x4c, 0x45, 0x5f, 0x4d, 0x53, 0x45, 0x43, 0x53, 0x10, 0x0c, 0x1a, 0xc0, 0x01, 0xd2, + 0xa7, 0xa7, 0x96, 0x06, 0xb9, 0x01, 0x0a, 0x36, 0x62, 0x65, 0x61, 0x6d, 0x3a, 0x6d, 0x65, 0x74, + 0x72, 0x69, 0x63, 0x3a, 0x70, 0x61, 0x72, 0x64, 0x6f, 0x5f, 0x65, 0x78, 0x65, 0x63, 0x75, 0x74, + 0x69, 0x6f, 0x6e, 0x5f, 0x74, 0x69, 0x6d, 0x65, 0x3a, 0x73, 0x74, 0x61, 0x72, 0x74, 0x5f, 0x62, + 0x75, 0x6e, 0x64, 0x6c, 0x65, 0x5f, 0x6d, 0x73, 0x65, 0x63, 0x73, 0x3a, 0x76, 0x31, 0x12, 0x19, + 0x62, 0x65, 0x61, 0x6d, 0x3a, 0x6d, 0x65, 0x74, 0x72, 0x69, 0x63, 0x73, 0x3a, 0x73, 0x75, 0x6d, + 0x5f, 0x69, 0x6e, 0x74, 0x36, 0x34, 0x3a, 0x76, 0x31, 0x1a, 0x0a, 0x50, 0x54, 0x52, 0x41, 0x4e, + 0x53, 0x46, 0x4f, 0x52, 0x4d, 0x22, 0x58, 0x0a, 0x0b, 0x64, 0x65, 0x73, 0x63, 0x72, 0x69, 0x70, + 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x49, 0x54, 0x68, 0x65, 0x20, 0x74, 0x6f, 0x74, 0x61, 0x6c, 0x20, + 0x65, 0x73, 0x74, 0x69, 0x6d, 0x61, 0x74, 0x65, 0x64, 0x20, 0x65, 0x78, 0x65, 0x63, 0x75, 0x74, + 0x69, 0x6f, 0x6e, 0x20, 0x74, 0x69, 0x6d, 0x65, 0x20, 0x6f, 0x66, 0x20, 0x74, 0x68, 0x65, 0x20, + 0x73, 0x74, 0x61, 0x72, 0x74, 0x20, 0x62, 0x75, 0x6e, 0x64, 0x6c, 0x65, 0x66, 0x75, 0x6e, 0x63, + 0x74, 0x69, 0x6f, 0x6e, 0x20, 0x69, 0x6e, 0x20, 0x61, 0x20, 0x70, 0x61, 0x72, 0x64, 0x6f, 0x12, + 0xdf, 0x01, 0x0a, 0x14, 0x50, 0x52, 0x4f, 0x43, 0x45, 0x53, 0x53, 0x5f, 0x42, 0x55, 0x4e, 0x44, + 0x4c, 0x45, 0x5f, 0x4d, 0x53, 0x45, 0x43, 0x53, 0x10, 0x0d, 0x1a, 0xc4, 0x01, 0xd2, 0xa7, 0xa7, + 0x96, 0x06, 0xbd, 0x01, 0x0a, 0x38, 0x62, 0x65, 0x61, 0x6d, 0x3a, 0x6d, 0x65, 0x74, 0x72, 0x69, + 0x63, 0x3a, 0x70, 0x61, 0x72, 0x64, 0x6f, 0x5f, 0x65, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, + 0x6e, 0x5f, 0x74, 0x69, 0x6d, 0x65, 0x3a, 0x70, 0x72, 0x6f, 0x63, 0x65, 0x73, 0x73, 0x5f, 0x62, + 0x75, 0x6e, 0x64, 0x6c, 0x65, 0x5f, 0x6d, 0x73, 0x65, 0x63, 0x73, 0x3a, 0x76, 0x31, 0x12, 0x19, + 0x62, 0x65, 0x61, 0x6d, 0x3a, 0x6d, 0x65, 0x74, 0x72, 0x69, 0x63, 0x73, 0x3a, 0x73, 0x75, 0x6d, + 0x5f, 0x69, 0x6e, 0x74, 0x36, 0x34, 0x3a, 0x76, 0x31, 0x1a, 0x0a, 0x50, 0x54, 0x52, 0x41, 0x4e, + 0x53, 0x46, 0x4f, 0x52, 0x4d, 0x22, 0x5a, 0x0a, 0x0b, 0x64, 0x65, 0x73, 0x63, 0x72, 0x69, 0x70, + 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x4b, 0x54, 0x68, 0x65, 0x20, 0x74, 0x6f, 0x74, 0x61, 0x6c, 0x20, + 0x65, 0x73, 0x74, 0x69, 0x6d, 0x61, 0x74, 0x65, 0x64, 0x20, 0x65, 0x78, 0x65, 0x63, 0x75, 0x74, + 0x69, 0x6f, 0x6e, 0x20, 0x74, 0x69, 0x6d, 0x65, 0x20, 0x6f, 0x66, 0x20, 0x74, 0x68, 0x65, 0x20, + 0x70, 0x72, 0x6f, 0x63, 0x65, 0x73, 0x73, 0x20, 0x62, 0x75, 0x6e, 0x64, 0x6c, 0x65, 0x66, 0x75, + 0x6e, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x20, 0x69, 0x6e, 0x20, 0x61, 0x20, 0x70, 0x61, 0x72, 0x64, + 0x6f, 0x12, 0xdd, 0x01, 0x0a, 0x13, 0x46, 0x49, 0x4e, 0x49, 0x53, 0x48, 0x5f, 0x42, 0x55, 0x4e, + 0x44, 0x4c, 0x45, 0x5f, 0x4d, 0x53, 0x45, 0x43, 0x53, 0x10, 0x0e, 0x1a, 0xc3, 0x01, 0xd2, 0xa7, + 0xa7, 0x96, 0x06, 0xbc, 0x01, 0x0a, 0x37, 0x62, 0x65, 0x61, 0x6d, 0x3a, 0x6d, 0x65, 0x74, 0x72, + 0x69, 0x63, 0x3a, 0x70, 0x61, 0x72, 0x64, 0x6f, 0x5f, 0x65, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, + 0x6f, 0x6e, 0x5f, 0x74, 0x69, 0x6d, 0x65, 0x3a, 0x66, 0x69, 0x6e, 0x69, 0x73, 0x68, 0x5f, 0x62, + 0x75, 0x6e, 0x64, 0x6c, 0x65, 0x5f, 0x6d, 0x73, 0x65, 0x63, 0x73, 0x3a, 0x76, 0x31, 0x12, 0x19, + 0x62, 0x65, 0x61, 0x6d, 0x3a, 0x6d, 0x65, 0x74, 0x72, 0x69, 0x63, 0x73, 0x3a, 0x73, 0x75, 0x6d, + 0x5f, 0x69, 0x6e, 0x74, 0x36, 0x34, 0x3a, 0x76, 0x31, 0x1a, 0x0a, 0x50, 0x54, 0x52, 0x41, 0x4e, + 0x53, 0x46, 0x4f, 0x52, 0x4d, 0x22, 0x5a, 0x0a, 0x0b, 0x64, 0x65, 0x73, 0x63, 0x72, 0x69, 0x70, + 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x4b, 0x54, 0x68, 0x65, 0x20, 0x74, 0x6f, 0x74, 0x61, 0x6c, 0x20, + 0x65, 0x73, 0x74, 0x69, 0x6d, 0x61, 0x74, 0x65, 0x64, 0x20, 0x65, 0x78, 0x65, 0x63, 0x75, 0x74, + 0x69, 0x6f, 0x6e, 0x20, 0x74, 0x69, 0x6d, 0x65, 0x20, 0x6f, 0x66, 0x20, 0x74, 0x68, 0x65, 0x20, + 0x66, 0x69, 0x6e, 0x69, 0x73, 0x68, 0x20, 0x62, 0x75, 0x6e, 0x64, 0x6c, 0x65, 0x20, 0x66, 0x75, + 0x6e, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x20, 0x69, 0x6e, 0x20, 0x61, 0x20, 0x70, 0x61, 0x72, 0x64, + 0x6f, 0x12, 0xbb, 0x01, 0x0a, 0x0b, 0x54, 0x4f, 0x54, 0x41, 0x4c, 0x5f, 0x4d, 0x53, 0x45, 0x43, + 0x53, 0x10, 0x0f, 0x1a, 0xa9, 0x01, 0xd2, 0xa7, 0xa7, 0x96, 0x06, 0xa2, 0x01, 0x0a, 0x34, 0x62, 0x65, 0x61, 0x6d, 0x3a, 0x6d, 0x65, 0x74, 0x72, 0x69, 0x63, 0x3a, 0x70, 0x74, 0x72, 0x61, 0x6e, - 0x73, 0x66, 0x6f, 0x72, 0x6d, 0x5f, 0x70, 0x72, 0x6f, 0x67, 0x72, 0x65, 0x73, 0x73, 0x3a, 0x63, - 0x6f, 0x6d, 0x70, 0x6c, 0x65, 0x74, 0x65, 0x64, 0x3a, 0x76, 0x31, 0x12, 0x18, 0x62, 0x65, 0x61, - 0x6d, 0x3a, 0x6d, 0x65, 0x74, 0x72, 0x69, 0x63, 0x73, 0x3a, 0x70, 0x72, 0x6f, 0x67, 0x72, 0x65, - 0x73, 0x73, 0x3a, 0x76, 0x31, 0x1a, 0x0a, 0x50, 0x54, 0x52, 0x41, 0x4e, 0x53, 0x46, 0x4f, 0x52, - 0x4d, 0x22, 0xac, 0x01, 0x0a, 0x0b, 0x64, 0x65, 0x73, 0x63, 0x72, 0x69, 0x70, 0x74, 0x69, 0x6f, - 0x6e, 0x12, 0x9c, 0x01, 0x54, 0x68, 0x65, 0x20, 0x72, 0x65, 0x6d, 0x61, 0x69, 0x6e, 0x69, 0x6e, - 0x67, 0x20, 0x61, 0x6d, 0x6f, 0x75, 0x6e, 0x74, 0x20, 0x6f, 0x66, 0x20, 0x77, 0x6f, 0x72, 0x6b, - 0x20, 0x66, 0x6f, 0x72, 0x20, 0x65, 0x61, 0x63, 0x68, 0x20, 0x61, 0x63, 0x74, 0x69, 0x76, 0x65, - 0x20, 0x65, 0x6c, 0x65, 0x6d, 0x65, 0x6e, 0x74, 0x2e, 0x20, 0x45, 0x61, 0x63, 0x68, 0x20, 0x61, - 0x63, 0x74, 0x69, 0x76, 0x65, 0x20, 0x65, 0x6c, 0x65, 0x6d, 0x65, 0x6e, 0x74, 0x20, 0x72, 0x65, - 0x70, 0x72, 0x65, 0x73, 0x65, 0x6e, 0x74, 0x73, 0x20, 0x61, 0x6e, 0x20, 0x69, 0x6e, 0x64, 0x65, - 0x70, 0x65, 0x6e, 0x64, 0x65, 0x6e, 0x74, 0x20, 0x61, 0x6d, 0x6f, 0x75, 0x6e, 0x74, 0x20, 0x6f, - 0x66, 0x20, 0x77, 0x6f, 0x72, 0x6b, 0x20, 0x6e, 0x6f, 0x74, 0x20, 0x73, 0x68, 0x61, 0x72, 0x65, - 0x64, 0x20, 0x77, 0x69, 0x74, 0x68, 0x20, 0x61, 0x6e, 0x79, 0x20, 0x6f, 0x74, 0x68, 0x65, 0x72, - 0x20, 0x61, 0x63, 0x74, 0x69, 0x76, 0x65, 0x20, 0x65, 0x6c, 0x65, 0x6d, 0x65, 0x6e, 0x74, 0x2e, - 0x12, 0xa8, 0x01, 0x0a, 0x17, 0x44, 0x41, 0x54, 0x41, 0x5f, 0x43, 0x48, 0x41, 0x4e, 0x4e, 0x45, - 0x4c, 0x5f, 0x52, 0x45, 0x41, 0x44, 0x5f, 0x49, 0x4e, 0x44, 0x45, 0x58, 0x10, 0x12, 0x1a, 0x8a, - 0x01, 0xd2, 0xa7, 0xa7, 0x96, 0x06, 0x83, 0x01, 0x0a, 0x26, 0x62, 0x65, 0x61, 0x6d, 0x3a, 0x6d, - 0x65, 0x74, 0x72, 0x69, 0x63, 0x3a, 0x64, 0x61, 0x74, 0x61, 0x5f, 0x63, 0x68, 0x61, 0x6e, 0x6e, - 0x65, 0x6c, 0x3a, 0x72, 0x65, 0x61, 0x64, 0x5f, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x3a, 0x76, 0x31, - 0x12, 0x19, 0x62, 0x65, 0x61, 0x6d, 0x3a, 0x6d, 0x65, 0x74, 0x72, 0x69, 0x63, 0x73, 0x3a, 0x73, - 0x75, 0x6d, 0x5f, 0x69, 0x6e, 0x74, 0x36, 0x34, 0x3a, 0x76, 0x31, 0x1a, 0x0a, 0x50, 0x54, 0x52, - 0x41, 0x4e, 0x53, 0x46, 0x4f, 0x52, 0x4d, 0x22, 0x32, 0x0a, 0x0b, 0x64, 0x65, 0x73, 0x63, 0x72, - 0x69, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x23, 0x54, 0x68, 0x65, 0x20, 0x72, 0x65, 0x61, 0x64, - 0x20, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x20, 0x6f, 0x66, 0x20, 0x74, 0x68, 0x65, 0x20, 0x64, 0x61, - 0x74, 0x61, 0x20, 0x63, 0x68, 0x61, 0x6e, 0x6e, 0x65, 0x6c, 0x2e, 0x12, 0x8a, 0x02, 0x0a, 0x11, - 0x41, 0x50, 0x49, 0x5f, 0x52, 0x45, 0x51, 0x55, 0x45, 0x53, 0x54, 0x5f, 0x43, 0x4f, 0x55, 0x4e, - 0x54, 0x10, 0x13, 0x1a, 0xf2, 0x01, 0xd2, 0xa7, 0xa7, 0x96, 0x06, 0xeb, 0x01, 0x0a, 0x23, 0x62, - 0x65, 0x61, 0x6d, 0x3a, 0x6d, 0x65, 0x74, 0x72, 0x69, 0x63, 0x3a, 0x69, 0x6f, 0x3a, 0x61, 0x70, - 0x69, 0x5f, 0x72, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x5f, 0x63, 0x6f, 0x75, 0x6e, 0x74, 0x3a, - 0x76, 0x31, 0x12, 0x19, 0x62, 0x65, 0x61, 0x6d, 0x3a, 0x6d, 0x65, 0x74, 0x72, 0x69, 0x63, 0x73, - 0x3a, 0x73, 0x75, 0x6d, 0x5f, 0x69, 0x6e, 0x74, 0x36, 0x34, 0x3a, 0x76, 0x31, 0x1a, 0x07, 0x53, - 0x45, 0x52, 0x56, 0x49, 0x43, 0x45, 0x1a, 0x06, 0x4d, 0x45, 0x54, 0x48, 0x4f, 0x44, 0x1a, 0x08, - 0x52, 0x45, 0x53, 0x4f, 0x55, 0x52, 0x43, 0x45, 0x1a, 0x0a, 0x50, 0x54, 0x52, 0x41, 0x4e, 0x53, - 0x46, 0x4f, 0x52, 0x4d, 0x1a, 0x06, 0x53, 0x54, 0x41, 0x54, 0x55, 0x53, 0x22, 0x62, 0x0a, 0x0b, - 0x64, 0x65, 0x73, 0x63, 0x72, 0x69, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x53, 0x52, 0x65, 0x71, - 0x75, 0x65, 0x73, 0x74, 0x20, 0x63, 0x6f, 0x75, 0x6e, 0x74, 0x73, 0x20, 0x77, 0x69, 0x74, 0x68, - 0x20, 0x73, 0x74, 0x61, 0x74, 0x75, 0x73, 0x20, 0x6d, 0x61, 0x64, 0x65, 0x20, 0x74, 0x6f, 0x20, - 0x49, 0x4f, 0x20, 0x73, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, 0x20, 0x41, 0x50, 0x49, 0x73, 0x20, - 0x74, 0x6f, 0x20, 0x62, 0x61, 0x74, 0x63, 0x68, 0x20, 0x72, 0x65, 0x61, 0x64, 0x20, 0x6f, 0x72, - 0x20, 0x77, 0x72, 0x69, 0x74, 0x65, 0x20, 0x65, 0x6c, 0x65, 0x6d, 0x65, 0x6e, 0x74, 0x73, 0x2e, - 0x22, 0x16, 0x0a, 0x0e, 0x70, 0x72, 0x6f, 0x63, 0x65, 0x73, 0x73, 0x5f, 0x6d, 0x65, 0x74, 0x72, - 0x69, 0x63, 0x12, 0x04, 0x74, 0x72, 0x75, 0x65, 0x12, 0xb3, 0x02, 0x0a, 0x15, 0x41, 0x50, 0x49, - 0x5f, 0x52, 0x45, 0x51, 0x55, 0x45, 0x53, 0x54, 0x5f, 0x4c, 0x41, 0x54, 0x45, 0x4e, 0x43, 0x49, - 0x45, 0x53, 0x10, 0x14, 0x1a, 0x97, 0x02, 0xd2, 0xa7, 0xa7, 0x96, 0x06, 0x90, 0x02, 0x0a, 0x27, - 0x62, 0x65, 0x61, 0x6d, 0x3a, 0x6d, 0x65, 0x74, 0x72, 0x69, 0x63, 0x3a, 0x69, 0x6f, 0x3a, 0x61, - 0x70, 0x69, 0x5f, 0x72, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x5f, 0x6c, 0x61, 0x74, 0x65, 0x6e, - 0x63, 0x69, 0x65, 0x73, 0x3a, 0x76, 0x31, 0x12, 0x1f, 0x62, 0x65, 0x61, 0x6d, 0x3a, 0x6d, 0x65, - 0x74, 0x72, 0x69, 0x63, 0x73, 0x3a, 0x68, 0x69, 0x73, 0x74, 0x6f, 0x67, 0x72, 0x61, 0x6d, 0x5f, - 0x69, 0x6e, 0x74, 0x36, 0x34, 0x3a, 0x76, 0x31, 0x1a, 0x07, 0x53, 0x45, 0x52, 0x56, 0x49, 0x43, - 0x45, 0x1a, 0x06, 0x4d, 0x45, 0x54, 0x48, 0x4f, 0x44, 0x1a, 0x08, 0x52, 0x45, 0x53, 0x4f, 0x55, - 0x52, 0x43, 0x45, 0x1a, 0x0a, 0x50, 0x54, 0x52, 0x41, 0x4e, 0x53, 0x46, 0x4f, 0x52, 0x4d, 0x22, - 0x6e, 0x0a, 0x0b, 0x64, 0x65, 0x73, 0x63, 0x72, 0x69, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x5f, - 0x48, 0x69, 0x73, 0x74, 0x6f, 0x67, 0x72, 0x61, 0x6d, 0x20, 0x63, 0x6f, 0x75, 0x6e, 0x74, 0x73, - 0x20, 0x66, 0x6f, 0x72, 0x20, 0x72, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x20, 0x6c, 0x61, 0x74, - 0x65, 0x6e, 0x63, 0x69, 0x65, 0x73, 0x20, 0x6d, 0x61, 0x64, 0x65, 0x20, 0x74, 0x6f, 0x20, 0x49, - 0x4f, 0x20, 0x73, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, 0x20, 0x41, 0x50, 0x49, 0x73, 0x20, 0x74, - 0x6f, 0x20, 0x62, 0x61, 0x74, 0x63, 0x68, 0x20, 0x72, 0x65, 0x61, 0x64, 0x20, 0x6f, 0x72, 0x20, - 0x77, 0x72, 0x69, 0x74, 0x65, 0x20, 0x65, 0x6c, 0x65, 0x6d, 0x65, 0x6e, 0x74, 0x73, 0x2e, 0x22, - 0x15, 0x0a, 0x05, 0x75, 0x6e, 0x69, 0x74, 0x73, 0x12, 0x0c, 0x4d, 0x69, 0x6c, 0x6c, 0x69, 0x73, - 0x65, 0x63, 0x6f, 0x6e, 0x64, 0x73, 0x22, 0x16, 0x0a, 0x0e, 0x70, 0x72, 0x6f, 0x63, 0x65, 0x73, - 0x73, 0x5f, 0x6d, 0x65, 0x74, 0x72, 0x69, 0x63, 0x12, 0x04, 0x74, 0x72, 0x75, 0x65, 0x22, 0x2e, - 0x0a, 0x18, 0x4d, 0x6f, 0x6e, 0x69, 0x74, 0x6f, 0x72, 0x69, 0x6e, 0x67, 0x49, 0x6e, 0x66, 0x6f, - 0x4c, 0x61, 0x62, 0x65, 0x6c, 0x50, 0x72, 0x6f, 0x70, 0x73, 0x12, 0x12, 0x0a, 0x04, 0x6e, 0x61, - 0x6d, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x04, 0x6e, 0x61, 0x6d, 0x65, 0x22, 0xcd, - 0x0b, 0x0a, 0x0e, 0x4d, 0x6f, 0x6e, 0x69, 0x74, 0x6f, 0x72, 0x69, 0x6e, 0x67, 0x49, 0x6e, 0x66, - 0x6f, 0x12, 0x10, 0x0a, 0x03, 0x75, 0x72, 0x6e, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x03, - 0x75, 0x72, 0x6e, 0x12, 0x12, 0x0a, 0x04, 0x74, 0x79, 0x70, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, - 0x09, 0x52, 0x04, 0x74, 0x79, 0x70, 0x65, 0x12, 0x18, 0x0a, 0x07, 0x70, 0x61, 0x79, 0x6c, 0x6f, - 0x61, 0x64, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0c, 0x52, 0x07, 0x70, 0x61, 0x79, 0x6c, 0x6f, 0x61, - 0x64, 0x12, 0x55, 0x0a, 0x06, 0x6c, 0x61, 0x62, 0x65, 0x6c, 0x73, 0x18, 0x04, 0x20, 0x03, 0x28, - 0x0b, 0x32, 0x3d, 0x2e, 0x6f, 0x72, 0x67, 0x2e, 0x61, 0x70, 0x61, 0x63, 0x68, 0x65, 0x2e, 0x62, - 0x65, 0x61, 0x6d, 0x2e, 0x6d, 0x6f, 0x64, 0x65, 0x6c, 0x2e, 0x70, 0x69, 0x70, 0x65, 0x6c, 0x69, - 0x6e, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x4d, 0x6f, 0x6e, 0x69, 0x74, 0x6f, 0x72, 0x69, 0x6e, 0x67, - 0x49, 0x6e, 0x66, 0x6f, 0x2e, 0x4c, 0x61, 0x62, 0x65, 0x6c, 0x73, 0x45, 0x6e, 0x74, 0x72, 0x79, - 0x52, 0x06, 0x6c, 0x61, 0x62, 0x65, 0x6c, 0x73, 0x12, 0x39, 0x0a, 0x0a, 0x73, 0x74, 0x61, 0x72, - 0x74, 0x5f, 0x74, 0x69, 0x6d, 0x65, 0x18, 0x05, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1a, 0x2e, 0x67, - 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66, 0x2e, 0x54, - 0x69, 0x6d, 0x65, 0x73, 0x74, 0x61, 0x6d, 0x70, 0x52, 0x09, 0x73, 0x74, 0x61, 0x72, 0x74, 0x54, - 0x69, 0x6d, 0x65, 0x1a, 0x39, 0x0a, 0x0b, 0x4c, 0x61, 0x62, 0x65, 0x6c, 0x73, 0x45, 0x6e, 0x74, - 0x72, 0x79, 0x12, 0x10, 0x0a, 0x03, 0x6b, 0x65, 0x79, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, - 0x03, 0x6b, 0x65, 0x79, 0x12, 0x14, 0x0a, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x18, 0x02, 0x20, - 0x01, 0x28, 0x09, 0x52, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x3a, 0x02, 0x38, 0x01, 0x22, 0xad, - 0x09, 0x0a, 0x14, 0x4d, 0x6f, 0x6e, 0x69, 0x74, 0x6f, 0x72, 0x69, 0x6e, 0x67, 0x49, 0x6e, 0x66, - 0x6f, 0x4c, 0x61, 0x62, 0x65, 0x6c, 0x73, 0x12, 0x21, 0x0a, 0x09, 0x54, 0x52, 0x41, 0x4e, 0x53, - 0x46, 0x4f, 0x52, 0x4d, 0x10, 0x00, 0x1a, 0x12, 0xa2, 0xd4, 0xe0, 0xe5, 0x03, 0x0c, 0x0a, 0x0a, - 0x50, 0x54, 0x52, 0x41, 0x4e, 0x53, 0x46, 0x4f, 0x52, 0x4d, 0x12, 0x24, 0x0a, 0x0b, 0x50, 0x43, - 0x4f, 0x4c, 0x4c, 0x45, 0x43, 0x54, 0x49, 0x4f, 0x4e, 0x10, 0x01, 0x1a, 0x13, 0xa2, 0xd4, 0xe0, - 0xe5, 0x03, 0x0d, 0x0a, 0x0b, 0x50, 0x43, 0x4f, 0x4c, 0x4c, 0x45, 0x43, 0x54, 0x49, 0x4f, 0x4e, - 0x12, 0x32, 0x0a, 0x12, 0x57, 0x49, 0x4e, 0x44, 0x4f, 0x57, 0x49, 0x4e, 0x47, 0x5f, 0x53, 0x54, - 0x52, 0x41, 0x54, 0x45, 0x47, 0x59, 0x10, 0x02, 0x1a, 0x1a, 0xa2, 0xd4, 0xe0, 0xe5, 0x03, 0x14, - 0x0a, 0x12, 0x57, 0x49, 0x4e, 0x44, 0x4f, 0x57, 0x49, 0x4e, 0x47, 0x5f, 0x53, 0x54, 0x52, 0x41, - 0x54, 0x45, 0x47, 0x59, 0x12, 0x18, 0x0a, 0x05, 0x43, 0x4f, 0x44, 0x45, 0x52, 0x10, 0x03, 0x1a, - 0x0d, 0xa2, 0xd4, 0xe0, 0xe5, 0x03, 0x07, 0x0a, 0x05, 0x43, 0x4f, 0x44, 0x45, 0x52, 0x12, 0x24, - 0x0a, 0x0b, 0x45, 0x4e, 0x56, 0x49, 0x52, 0x4f, 0x4e, 0x4d, 0x45, 0x4e, 0x54, 0x10, 0x04, 0x1a, - 0x13, 0xa2, 0xd4, 0xe0, 0xe5, 0x03, 0x0d, 0x0a, 0x0b, 0x45, 0x4e, 0x56, 0x49, 0x52, 0x4f, 0x4e, - 0x4d, 0x45, 0x4e, 0x54, 0x12, 0x20, 0x0a, 0x09, 0x4e, 0x41, 0x4d, 0x45, 0x53, 0x50, 0x41, 0x43, - 0x45, 0x10, 0x05, 0x1a, 0x11, 0xa2, 0xd4, 0xe0, 0xe5, 0x03, 0x0b, 0x0a, 0x09, 0x4e, 0x41, 0x4d, - 0x45, 0x53, 0x50, 0x41, 0x43, 0x45, 0x12, 0x16, 0x0a, 0x04, 0x4e, 0x41, 0x4d, 0x45, 0x10, 0x06, - 0x1a, 0x0c, 0xa2, 0xd4, 0xe0, 0xe5, 0x03, 0x06, 0x0a, 0x04, 0x4e, 0x41, 0x4d, 0x45, 0x12, 0x1c, - 0x0a, 0x07, 0x53, 0x45, 0x52, 0x56, 0x49, 0x43, 0x45, 0x10, 0x07, 0x1a, 0x0f, 0xa2, 0xd4, 0xe0, - 0xe5, 0x03, 0x09, 0x0a, 0x07, 0x53, 0x45, 0x52, 0x56, 0x49, 0x43, 0x45, 0x12, 0x1a, 0x0a, 0x06, - 0x4d, 0x45, 0x54, 0x48, 0x4f, 0x44, 0x10, 0x08, 0x1a, 0x0e, 0xa2, 0xd4, 0xe0, 0xe5, 0x03, 0x08, - 0x0a, 0x06, 0x4d, 0x45, 0x54, 0x48, 0x4f, 0x44, 0x12, 0x1e, 0x0a, 0x08, 0x52, 0x45, 0x53, 0x4f, - 0x55, 0x52, 0x43, 0x45, 0x10, 0x09, 0x1a, 0x10, 0xa2, 0xd4, 0xe0, 0xe5, 0x03, 0x0a, 0x0a, 0x08, - 0x52, 0x45, 0x53, 0x4f, 0x55, 0x52, 0x43, 0x45, 0x12, 0x1a, 0x0a, 0x06, 0x53, 0x54, 0x41, 0x54, - 0x55, 0x53, 0x10, 0x0a, 0x1a, 0x0e, 0xa2, 0xd4, 0xe0, 0xe5, 0x03, 0x08, 0x0a, 0x06, 0x53, 0x54, - 0x41, 0x54, 0x55, 0x53, 0x12, 0x34, 0x0a, 0x13, 0x42, 0x49, 0x47, 0x51, 0x55, 0x45, 0x52, 0x59, - 0x5f, 0x50, 0x52, 0x4f, 0x4a, 0x45, 0x43, 0x54, 0x5f, 0x49, 0x44, 0x10, 0x0b, 0x1a, 0x1b, 0xa2, - 0xd4, 0xe0, 0xe5, 0x03, 0x15, 0x0a, 0x13, 0x42, 0x49, 0x47, 0x51, 0x55, 0x45, 0x52, 0x59, 0x5f, - 0x50, 0x52, 0x4f, 0x4a, 0x45, 0x43, 0x54, 0x5f, 0x49, 0x44, 0x12, 0x2e, 0x0a, 0x10, 0x42, 0x49, - 0x47, 0x51, 0x55, 0x45, 0x52, 0x59, 0x5f, 0x44, 0x41, 0x54, 0x41, 0x53, 0x45, 0x54, 0x10, 0x0c, - 0x1a, 0x18, 0xa2, 0xd4, 0xe0, 0xe5, 0x03, 0x12, 0x0a, 0x10, 0x42, 0x49, 0x47, 0x51, 0x55, 0x45, - 0x52, 0x59, 0x5f, 0x44, 0x41, 0x54, 0x41, 0x53, 0x45, 0x54, 0x12, 0x2a, 0x0a, 0x0e, 0x42, 0x49, - 0x47, 0x51, 0x55, 0x45, 0x52, 0x59, 0x5f, 0x54, 0x41, 0x42, 0x4c, 0x45, 0x10, 0x0d, 0x1a, 0x16, - 0xa2, 0xd4, 0xe0, 0xe5, 0x03, 0x10, 0x0a, 0x0e, 0x42, 0x49, 0x47, 0x51, 0x55, 0x45, 0x52, 0x59, - 0x5f, 0x54, 0x41, 0x42, 0x4c, 0x45, 0x12, 0x28, 0x0a, 0x0d, 0x42, 0x49, 0x47, 0x51, 0x55, 0x45, - 0x52, 0x59, 0x5f, 0x56, 0x49, 0x45, 0x57, 0x10, 0x0e, 0x1a, 0x15, 0xa2, 0xd4, 0xe0, 0xe5, 0x03, - 0x0f, 0x0a, 0x0d, 0x42, 0x49, 0x47, 0x51, 0x55, 0x45, 0x52, 0x59, 0x5f, 0x56, 0x49, 0x45, 0x57, - 0x12, 0x34, 0x0a, 0x13, 0x42, 0x49, 0x47, 0x51, 0x55, 0x45, 0x52, 0x59, 0x5f, 0x51, 0x55, 0x45, - 0x52, 0x59, 0x5f, 0x4e, 0x41, 0x4d, 0x45, 0x10, 0x0f, 0x1a, 0x1b, 0xa2, 0xd4, 0xe0, 0xe5, 0x03, - 0x15, 0x0a, 0x13, 0x42, 0x49, 0x47, 0x51, 0x55, 0x45, 0x52, 0x59, 0x5f, 0x51, 0x55, 0x45, 0x52, - 0x59, 0x5f, 0x4e, 0x41, 0x4d, 0x45, 0x12, 0x22, 0x0a, 0x0a, 0x47, 0x43, 0x53, 0x5f, 0x42, 0x55, - 0x43, 0x4b, 0x45, 0x54, 0x10, 0x10, 0x1a, 0x12, 0xa2, 0xd4, 0xe0, 0xe5, 0x03, 0x0c, 0x0a, 0x0a, - 0x47, 0x43, 0x53, 0x5f, 0x42, 0x55, 0x43, 0x4b, 0x45, 0x54, 0x12, 0x2a, 0x0a, 0x0e, 0x47, 0x43, - 0x53, 0x5f, 0x50, 0x52, 0x4f, 0x4a, 0x45, 0x43, 0x54, 0x5f, 0x49, 0x44, 0x10, 0x11, 0x1a, 0x16, - 0xa2, 0xd4, 0xe0, 0xe5, 0x03, 0x10, 0x0a, 0x0e, 0x47, 0x43, 0x53, 0x5f, 0x50, 0x52, 0x4f, 0x4a, - 0x45, 0x43, 0x54, 0x5f, 0x49, 0x44, 0x12, 0x30, 0x0a, 0x11, 0x44, 0x41, 0x54, 0x41, 0x53, 0x54, - 0x4f, 0x52, 0x45, 0x5f, 0x50, 0x52, 0x4f, 0x4a, 0x45, 0x43, 0x54, 0x10, 0x12, 0x1a, 0x19, 0xa2, - 0xd4, 0xe0, 0xe5, 0x03, 0x13, 0x0a, 0x11, 0x44, 0x41, 0x54, 0x41, 0x53, 0x54, 0x4f, 0x52, 0x45, - 0x5f, 0x50, 0x52, 0x4f, 0x4a, 0x45, 0x43, 0x54, 0x12, 0x34, 0x0a, 0x13, 0x44, 0x41, 0x54, 0x41, - 0x53, 0x54, 0x4f, 0x52, 0x45, 0x5f, 0x4e, 0x41, 0x4d, 0x45, 0x53, 0x50, 0x41, 0x43, 0x45, 0x10, - 0x13, 0x1a, 0x1b, 0xa2, 0xd4, 0xe0, 0xe5, 0x03, 0x15, 0x0a, 0x13, 0x44, 0x41, 0x54, 0x41, 0x53, - 0x54, 0x4f, 0x52, 0x45, 0x5f, 0x4e, 0x41, 0x4d, 0x45, 0x53, 0x50, 0x41, 0x43, 0x45, 0x12, 0x34, - 0x0a, 0x13, 0x42, 0x49, 0x47, 0x54, 0x41, 0x42, 0x4c, 0x45, 0x5f, 0x50, 0x52, 0x4f, 0x4a, 0x45, - 0x43, 0x54, 0x5f, 0x49, 0x44, 0x10, 0x14, 0x1a, 0x1b, 0xa2, 0xd4, 0xe0, 0xe5, 0x03, 0x15, 0x0a, - 0x13, 0x42, 0x49, 0x47, 0x54, 0x41, 0x42, 0x4c, 0x45, 0x5f, 0x50, 0x52, 0x4f, 0x4a, 0x45, 0x43, - 0x54, 0x5f, 0x49, 0x44, 0x12, 0x24, 0x0a, 0x0b, 0x49, 0x4e, 0x53, 0x54, 0x41, 0x4e, 0x43, 0x45, - 0x5f, 0x49, 0x44, 0x10, 0x15, 0x1a, 0x13, 0xa2, 0xd4, 0xe0, 0xe5, 0x03, 0x0d, 0x0a, 0x0b, 0x49, - 0x4e, 0x53, 0x54, 0x41, 0x4e, 0x43, 0x45, 0x5f, 0x49, 0x44, 0x12, 0x1e, 0x0a, 0x08, 0x54, 0x41, - 0x42, 0x4c, 0x45, 0x5f, 0x49, 0x44, 0x10, 0x16, 0x1a, 0x10, 0xa2, 0xd4, 0xe0, 0xe5, 0x03, 0x0a, - 0x0a, 0x08, 0x54, 0x41, 0x42, 0x4c, 0x45, 0x5f, 0x49, 0x44, 0x12, 0x32, 0x0a, 0x12, 0x53, 0x50, - 0x41, 0x4e, 0x4e, 0x45, 0x52, 0x5f, 0x50, 0x52, 0x4f, 0x4a, 0x45, 0x43, 0x54, 0x5f, 0x49, 0x44, - 0x10, 0x17, 0x1a, 0x1a, 0xa2, 0xd4, 0xe0, 0xe5, 0x03, 0x14, 0x0a, 0x12, 0x53, 0x50, 0x41, 0x4e, - 0x4e, 0x45, 0x52, 0x5f, 0x50, 0x52, 0x4f, 0x4a, 0x45, 0x43, 0x54, 0x5f, 0x49, 0x44, 0x12, 0x34, - 0x0a, 0x13, 0x53, 0x50, 0x41, 0x4e, 0x4e, 0x45, 0x52, 0x5f, 0x44, 0x41, 0x54, 0x41, 0x42, 0x41, - 0x53, 0x45, 0x5f, 0x49, 0x44, 0x10, 0x18, 0x1a, 0x1b, 0xa2, 0xd4, 0xe0, 0xe5, 0x03, 0x15, 0x0a, - 0x13, 0x53, 0x50, 0x41, 0x4e, 0x4e, 0x45, 0x52, 0x5f, 0x44, 0x41, 0x54, 0x41, 0x42, 0x41, 0x53, - 0x45, 0x5f, 0x49, 0x44, 0x12, 0x2e, 0x0a, 0x10, 0x53, 0x50, 0x41, 0x4e, 0x4e, 0x45, 0x52, 0x5f, - 0x54, 0x41, 0x42, 0x4c, 0x45, 0x5f, 0x49, 0x44, 0x10, 0x19, 0x1a, 0x18, 0xa2, 0xd4, 0xe0, 0xe5, - 0x03, 0x12, 0x0a, 0x10, 0x53, 0x50, 0x41, 0x4e, 0x4e, 0x45, 0x52, 0x5f, 0x54, 0x41, 0x42, 0x4c, - 0x45, 0x5f, 0x49, 0x44, 0x12, 0x34, 0x0a, 0x13, 0x53, 0x50, 0x41, 0x4e, 0x4e, 0x45, 0x52, 0x5f, - 0x49, 0x4e, 0x53, 0x54, 0x41, 0x4e, 0x43, 0x45, 0x5f, 0x49, 0x44, 0x10, 0x1a, 0x1a, 0x1b, 0xa2, - 0xd4, 0xe0, 0xe5, 0x03, 0x15, 0x0a, 0x13, 0x53, 0x50, 0x41, 0x4e, 0x4e, 0x45, 0x52, 0x5f, 0x49, - 0x4e, 0x53, 0x54, 0x41, 0x4e, 0x43, 0x45, 0x5f, 0x49, 0x44, 0x12, 0x32, 0x0a, 0x12, 0x53, 0x50, - 0x41, 0x4e, 0x4e, 0x45, 0x52, 0x5f, 0x51, 0x55, 0x45, 0x52, 0x59, 0x5f, 0x4e, 0x41, 0x4d, 0x45, - 0x10, 0x1b, 0x1a, 0x1a, 0xa2, 0xd4, 0xe0, 0xe5, 0x03, 0x14, 0x0a, 0x12, 0x53, 0x50, 0x41, 0x4e, - 0x4e, 0x45, 0x52, 0x5f, 0x51, 0x55, 0x45, 0x52, 0x59, 0x5f, 0x4e, 0x41, 0x4d, 0x45, 0x22, 0xf3, - 0x05, 0x0a, 0x16, 0x4d, 0x6f, 0x6e, 0x69, 0x74, 0x6f, 0x72, 0x69, 0x6e, 0x67, 0x49, 0x6e, 0x66, - 0x6f, 0x54, 0x79, 0x70, 0x65, 0x55, 0x72, 0x6e, 0x73, 0x22, 0xd8, 0x05, 0x0a, 0x04, 0x45, 0x6e, - 0x75, 0x6d, 0x12, 0x33, 0x0a, 0x0e, 0x53, 0x55, 0x4d, 0x5f, 0x49, 0x4e, 0x54, 0x36, 0x34, 0x5f, - 0x54, 0x59, 0x50, 0x45, 0x10, 0x00, 0x1a, 0x1f, 0xa2, 0xb4, 0xfa, 0xc2, 0x05, 0x19, 0x62, 0x65, - 0x61, 0x6d, 0x3a, 0x6d, 0x65, 0x74, 0x72, 0x69, 0x63, 0x73, 0x3a, 0x73, 0x75, 0x6d, 0x5f, 0x69, - 0x6e, 0x74, 0x36, 0x34, 0x3a, 0x76, 0x31, 0x12, 0x35, 0x0a, 0x0f, 0x53, 0x55, 0x4d, 0x5f, 0x44, - 0x4f, 0x55, 0x42, 0x4c, 0x45, 0x5f, 0x54, 0x59, 0x50, 0x45, 0x10, 0x01, 0x1a, 0x20, 0xa2, 0xb4, - 0xfa, 0xc2, 0x05, 0x1a, 0x62, 0x65, 0x61, 0x6d, 0x3a, 0x6d, 0x65, 0x74, 0x72, 0x69, 0x63, 0x73, - 0x3a, 0x73, 0x75, 0x6d, 0x5f, 0x64, 0x6f, 0x75, 0x62, 0x6c, 0x65, 0x3a, 0x76, 0x31, 0x12, 0x45, - 0x0a, 0x17, 0x44, 0x49, 0x53, 0x54, 0x52, 0x49, 0x42, 0x55, 0x54, 0x49, 0x4f, 0x4e, 0x5f, 0x49, - 0x4e, 0x54, 0x36, 0x34, 0x5f, 0x54, 0x59, 0x50, 0x45, 0x10, 0x02, 0x1a, 0x28, 0xa2, 0xb4, 0xfa, - 0xc2, 0x05, 0x22, 0x62, 0x65, 0x61, 0x6d, 0x3a, 0x6d, 0x65, 0x74, 0x72, 0x69, 0x63, 0x73, 0x3a, - 0x64, 0x69, 0x73, 0x74, 0x72, 0x69, 0x62, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x69, 0x6e, 0x74, - 0x36, 0x34, 0x3a, 0x76, 0x31, 0x12, 0x47, 0x0a, 0x18, 0x44, 0x49, 0x53, 0x54, 0x52, 0x49, 0x42, - 0x55, 0x54, 0x49, 0x4f, 0x4e, 0x5f, 0x44, 0x4f, 0x55, 0x42, 0x4c, 0x45, 0x5f, 0x54, 0x59, 0x50, - 0x45, 0x10, 0x03, 0x1a, 0x29, 0xa2, 0xb4, 0xfa, 0xc2, 0x05, 0x23, 0x62, 0x65, 0x61, 0x6d, 0x3a, - 0x6d, 0x65, 0x74, 0x72, 0x69, 0x63, 0x73, 0x3a, 0x64, 0x69, 0x73, 0x74, 0x72, 0x69, 0x62, 0x75, - 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x64, 0x6f, 0x75, 0x62, 0x6c, 0x65, 0x3a, 0x76, 0x31, 0x12, 0x39, - 0x0a, 0x11, 0x4c, 0x41, 0x54, 0x45, 0x53, 0x54, 0x5f, 0x49, 0x4e, 0x54, 0x36, 0x34, 0x5f, 0x54, - 0x59, 0x50, 0x45, 0x10, 0x04, 0x1a, 0x22, 0xa2, 0xb4, 0xfa, 0xc2, 0x05, 0x1c, 0x62, 0x65, 0x61, - 0x6d, 0x3a, 0x6d, 0x65, 0x74, 0x72, 0x69, 0x63, 0x73, 0x3a, 0x6c, 0x61, 0x74, 0x65, 0x73, 0x74, - 0x5f, 0x69, 0x6e, 0x74, 0x36, 0x34, 0x3a, 0x76, 0x31, 0x12, 0x3b, 0x0a, 0x12, 0x4c, 0x41, 0x54, - 0x45, 0x53, 0x54, 0x5f, 0x44, 0x4f, 0x55, 0x42, 0x4c, 0x45, 0x5f, 0x54, 0x59, 0x50, 0x45, 0x10, - 0x05, 0x1a, 0x23, 0xa2, 0xb4, 0xfa, 0xc2, 0x05, 0x1d, 0x62, 0x65, 0x61, 0x6d, 0x3a, 0x6d, 0x65, - 0x74, 0x72, 0x69, 0x63, 0x73, 0x3a, 0x6c, 0x61, 0x74, 0x65, 0x73, 0x74, 0x5f, 0x64, 0x6f, 0x75, - 0x62, 0x6c, 0x65, 0x3a, 0x76, 0x31, 0x12, 0x37, 0x0a, 0x10, 0x54, 0x4f, 0x50, 0x5f, 0x4e, 0x5f, - 0x49, 0x4e, 0x54, 0x36, 0x34, 0x5f, 0x54, 0x59, 0x50, 0x45, 0x10, 0x06, 0x1a, 0x21, 0xa2, 0xb4, - 0xfa, 0xc2, 0x05, 0x1b, 0x62, 0x65, 0x61, 0x6d, 0x3a, 0x6d, 0x65, 0x74, 0x72, 0x69, 0x63, 0x73, - 0x3a, 0x74, 0x6f, 0x70, 0x5f, 0x6e, 0x5f, 0x69, 0x6e, 0x74, 0x36, 0x34, 0x3a, 0x76, 0x31, 0x12, - 0x39, 0x0a, 0x11, 0x54, 0x4f, 0x50, 0x5f, 0x4e, 0x5f, 0x44, 0x4f, 0x55, 0x42, 0x4c, 0x45, 0x5f, - 0x54, 0x59, 0x50, 0x45, 0x10, 0x07, 0x1a, 0x22, 0xa2, 0xb4, 0xfa, 0xc2, 0x05, 0x1c, 0x62, 0x65, - 0x61, 0x6d, 0x3a, 0x6d, 0x65, 0x74, 0x72, 0x69, 0x63, 0x73, 0x3a, 0x74, 0x6f, 0x70, 0x5f, 0x6e, - 0x5f, 0x64, 0x6f, 0x75, 0x62, 0x6c, 0x65, 0x3a, 0x76, 0x31, 0x12, 0x3d, 0x0a, 0x13, 0x42, 0x4f, - 0x54, 0x54, 0x4f, 0x4d, 0x5f, 0x4e, 0x5f, 0x49, 0x4e, 0x54, 0x36, 0x34, 0x5f, 0x54, 0x59, 0x50, - 0x45, 0x10, 0x08, 0x1a, 0x24, 0xa2, 0xb4, 0xfa, 0xc2, 0x05, 0x1e, 0x62, 0x65, 0x61, 0x6d, 0x3a, - 0x6d, 0x65, 0x74, 0x72, 0x69, 0x63, 0x73, 0x3a, 0x62, 0x6f, 0x74, 0x74, 0x6f, 0x6d, 0x5f, 0x6e, - 0x5f, 0x69, 0x6e, 0x74, 0x36, 0x34, 0x3a, 0x76, 0x31, 0x12, 0x3f, 0x0a, 0x14, 0x42, 0x4f, 0x54, - 0x54, 0x4f, 0x4d, 0x5f, 0x4e, 0x5f, 0x44, 0x4f, 0x55, 0x42, 0x4c, 0x45, 0x5f, 0x54, 0x59, 0x50, - 0x45, 0x10, 0x09, 0x1a, 0x25, 0xa2, 0xb4, 0xfa, 0xc2, 0x05, 0x1f, 0x62, 0x65, 0x61, 0x6d, 0x3a, - 0x6d, 0x65, 0x74, 0x72, 0x69, 0x63, 0x73, 0x3a, 0x62, 0x6f, 0x74, 0x74, 0x6f, 0x6d, 0x5f, 0x6e, - 0x5f, 0x64, 0x6f, 0x75, 0x62, 0x6c, 0x65, 0x3a, 0x76, 0x31, 0x12, 0x31, 0x0a, 0x0d, 0x50, 0x52, - 0x4f, 0x47, 0x52, 0x45, 0x53, 0x53, 0x5f, 0x54, 0x59, 0x50, 0x45, 0x10, 0x0a, 0x1a, 0x1e, 0xa2, - 0xb4, 0xfa, 0xc2, 0x05, 0x18, 0x62, 0x65, 0x61, 0x6d, 0x3a, 0x6d, 0x65, 0x74, 0x72, 0x69, 0x63, - 0x73, 0x3a, 0x70, 0x72, 0x6f, 0x67, 0x72, 0x65, 0x73, 0x73, 0x3a, 0x76, 0x31, 0x12, 0x35, 0x0a, - 0x0f, 0x53, 0x45, 0x54, 0x5f, 0x53, 0x54, 0x52, 0x49, 0x4e, 0x47, 0x5f, 0x54, 0x59, 0x50, 0x45, - 0x10, 0x0b, 0x1a, 0x20, 0xa2, 0xb4, 0xfa, 0xc2, 0x05, 0x1a, 0x62, 0x65, 0x61, 0x6d, 0x3a, 0x6d, - 0x65, 0x74, 0x72, 0x69, 0x63, 0x73, 0x3a, 0x73, 0x65, 0x74, 0x5f, 0x73, 0x74, 0x72, 0x69, 0x6e, - 0x67, 0x3a, 0x76, 0x31, 0x3a, 0x82, 0x01, 0x0a, 0x0b, 0x6c, 0x61, 0x62, 0x65, 0x6c, 0x5f, 0x70, - 0x72, 0x6f, 0x70, 0x73, 0x12, 0x21, 0x2e, 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2e, 0x70, 0x72, - 0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66, 0x2e, 0x45, 0x6e, 0x75, 0x6d, 0x56, 0x61, 0x6c, 0x75, 0x65, - 0x4f, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x18, 0xc4, 0x8a, 0xdc, 0x3c, 0x20, 0x01, 0x28, 0x0b, - 0x32, 0x3b, 0x2e, 0x6f, 0x72, 0x67, 0x2e, 0x61, 0x70, 0x61, 0x63, 0x68, 0x65, 0x2e, 0x62, 0x65, - 0x61, 0x6d, 0x2e, 0x6d, 0x6f, 0x64, 0x65, 0x6c, 0x2e, 0x70, 0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e, - 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x4d, 0x6f, 0x6e, 0x69, 0x74, 0x6f, 0x72, 0x69, 0x6e, 0x67, 0x49, - 0x6e, 0x66, 0x6f, 0x4c, 0x61, 0x62, 0x65, 0x6c, 0x50, 0x72, 0x6f, 0x70, 0x73, 0x52, 0x0a, 0x6c, - 0x61, 0x62, 0x65, 0x6c, 0x50, 0x72, 0x6f, 0x70, 0x73, 0x3a, 0x8d, 0x01, 0x0a, 0x14, 0x6d, 0x6f, - 0x6e, 0x69, 0x74, 0x6f, 0x72, 0x69, 0x6e, 0x67, 0x5f, 0x69, 0x6e, 0x66, 0x6f, 0x5f, 0x73, 0x70, - 0x65, 0x63, 0x12, 0x21, 0x2e, 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74, - 0x6f, 0x62, 0x75, 0x66, 0x2e, 0x45, 0x6e, 0x75, 0x6d, 0x56, 0x61, 0x6c, 0x75, 0x65, 0x4f, 0x70, - 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x18, 0xfa, 0xf4, 0xe4, 0x62, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x35, - 0x2e, 0x6f, 0x72, 0x67, 0x2e, 0x61, 0x70, 0x61, 0x63, 0x68, 0x65, 0x2e, 0x62, 0x65, 0x61, 0x6d, - 0x2e, 0x6d, 0x6f, 0x64, 0x65, 0x6c, 0x2e, 0x70, 0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e, 0x65, 0x2e, - 0x76, 0x31, 0x2e, 0x4d, 0x6f, 0x6e, 0x69, 0x74, 0x6f, 0x72, 0x69, 0x6e, 0x67, 0x49, 0x6e, 0x66, - 0x6f, 0x53, 0x70, 0x65, 0x63, 0x52, 0x12, 0x6d, 0x6f, 0x6e, 0x69, 0x74, 0x6f, 0x72, 0x69, 0x6e, - 0x67, 0x49, 0x6e, 0x66, 0x6f, 0x53, 0x70, 0x65, 0x63, 0x42, 0x79, 0x0a, 0x21, 0x6f, 0x72, 0x67, - 0x2e, 0x61, 0x70, 0x61, 0x63, 0x68, 0x65, 0x2e, 0x62, 0x65, 0x61, 0x6d, 0x2e, 0x6d, 0x6f, 0x64, - 0x65, 0x6c, 0x2e, 0x70, 0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e, 0x65, 0x2e, 0x76, 0x31, 0x42, 0x0a, - 0x4d, 0x65, 0x74, 0x72, 0x69, 0x63, 0x73, 0x41, 0x70, 0x69, 0x5a, 0x48, 0x67, 0x69, 0x74, 0x68, - 0x75, 0x62, 0x2e, 0x63, 0x6f, 0x6d, 0x2f, 0x61, 0x70, 0x61, 0x63, 0x68, 0x65, 0x2f, 0x62, 0x65, - 0x61, 0x6d, 0x2f, 0x73, 0x64, 0x6b, 0x73, 0x2f, 0x76, 0x32, 0x2f, 0x67, 0x6f, 0x2f, 0x70, 0x6b, - 0x67, 0x2f, 0x62, 0x65, 0x61, 0x6d, 0x2f, 0x6d, 0x6f, 0x64, 0x65, 0x6c, 0x2f, 0x70, 0x69, 0x70, - 0x65, 0x6c, 0x69, 0x6e, 0x65, 0x5f, 0x76, 0x31, 0x3b, 0x70, 0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e, - 0x65, 0x5f, 0x76, 0x31, 0x62, 0x06, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x33, -} - -var ( - file_org_apache_beam_model_pipeline_v1_metrics_proto_rawDescOnce sync.Once - file_org_apache_beam_model_pipeline_v1_metrics_proto_rawDescData = file_org_apache_beam_model_pipeline_v1_metrics_proto_rawDesc -) - -func file_org_apache_beam_model_pipeline_v1_metrics_proto_rawDescGZIP() []byte { - file_org_apache_beam_model_pipeline_v1_metrics_proto_rawDescOnce.Do(func() { - file_org_apache_beam_model_pipeline_v1_metrics_proto_rawDescData = protoimpl.X.CompressGZIP(file_org_apache_beam_model_pipeline_v1_metrics_proto_rawDescData) - }) - return file_org_apache_beam_model_pipeline_v1_metrics_proto_rawDescData + 0x73, 0x66, 0x6f, 0x72, 0x6d, 0x5f, 0x65, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x5f, + 0x74, 0x69, 0x6d, 0x65, 0x3a, 0x74, 0x6f, 0x74, 0x61, 0x6c, 0x5f, 0x6d, 0x73, 0x65, 0x63, 0x73, + 0x3a, 0x76, 0x31, 0x12, 0x19, 0x62, 0x65, 0x61, 0x6d, 0x3a, 0x6d, 0x65, 0x74, 0x72, 0x69, 0x63, + 0x73, 0x3a, 0x73, 0x75, 0x6d, 0x5f, 0x69, 0x6e, 0x74, 0x36, 0x34, 0x3a, 0x76, 0x31, 0x1a, 0x0a, + 0x50, 0x54, 0x52, 0x41, 0x4e, 0x53, 0x46, 0x4f, 0x52, 0x4d, 0x22, 0x43, 0x0a, 0x0b, 0x64, 0x65, + 0x73, 0x63, 0x72, 0x69, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x34, 0x54, 0x68, 0x65, 0x20, 0x74, + 0x6f, 0x74, 0x61, 0x6c, 0x20, 0x65, 0x73, 0x74, 0x69, 0x6d, 0x61, 0x74, 0x65, 0x64, 0x20, 0x65, + 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x20, 0x74, 0x69, 0x6d, 0x65, 0x20, 0x6f, 0x66, + 0x20, 0x74, 0x68, 0x65, 0x20, 0x70, 0x74, 0x72, 0x61, 0x6e, 0x73, 0x66, 0x6f, 0x72, 0x6d, 0x12, + 0x9f, 0x02, 0x0a, 0x0e, 0x57, 0x4f, 0x52, 0x4b, 0x5f, 0x52, 0x45, 0x4d, 0x41, 0x49, 0x4e, 0x49, + 0x4e, 0x47, 0x10, 0x10, 0x1a, 0x8a, 0x02, 0xd2, 0xa7, 0xa7, 0x96, 0x06, 0x83, 0x02, 0x0a, 0x2c, + 0x62, 0x65, 0x61, 0x6d, 0x3a, 0x6d, 0x65, 0x74, 0x72, 0x69, 0x63, 0x3a, 0x70, 0x74, 0x72, 0x61, + 0x6e, 0x73, 0x66, 0x6f, 0x72, 0x6d, 0x5f, 0x70, 0x72, 0x6f, 0x67, 0x72, 0x65, 0x73, 0x73, 0x3a, + 0x72, 0x65, 0x6d, 0x61, 0x69, 0x6e, 0x69, 0x6e, 0x67, 0x3a, 0x76, 0x31, 0x12, 0x18, 0x62, 0x65, + 0x61, 0x6d, 0x3a, 0x6d, 0x65, 0x74, 0x72, 0x69, 0x63, 0x73, 0x3a, 0x70, 0x72, 0x6f, 0x67, 0x72, + 0x65, 0x73, 0x73, 0x3a, 0x76, 0x31, 0x1a, 0x0a, 0x50, 0x54, 0x52, 0x41, 0x4e, 0x53, 0x46, 0x4f, + 0x52, 0x4d, 0x22, 0xac, 0x01, 0x0a, 0x0b, 0x64, 0x65, 0x73, 0x63, 0x72, 0x69, 0x70, 0x74, 0x69, + 0x6f, 0x6e, 0x12, 0x9c, 0x01, 0x54, 0x68, 0x65, 0x20, 0x72, 0x65, 0x6d, 0x61, 0x69, 0x6e, 0x69, + 0x6e, 0x67, 0x20, 0x61, 0x6d, 0x6f, 0x75, 0x6e, 0x74, 0x20, 0x6f, 0x66, 0x20, 0x77, 0x6f, 0x72, + 0x6b, 0x20, 0x66, 0x6f, 0x72, 0x20, 0x65, 0x61, 0x63, 0x68, 0x20, 0x61, 0x63, 0x74, 0x69, 0x76, + 0x65, 0x20, 0x65, 0x6c, 0x65, 0x6d, 0x65, 0x6e, 0x74, 0x2e, 0x20, 0x45, 0x61, 0x63, 0x68, 0x20, + 0x61, 0x63, 0x74, 0x69, 0x76, 0x65, 0x20, 0x65, 0x6c, 0x65, 0x6d, 0x65, 0x6e, 0x74, 0x20, 0x72, + 0x65, 0x70, 0x72, 0x65, 0x73, 0x65, 0x6e, 0x74, 0x73, 0x20, 0x61, 0x6e, 0x20, 0x69, 0x6e, 0x64, + 0x65, 0x70, 0x65, 0x6e, 0x64, 0x65, 0x6e, 0x74, 0x20, 0x61, 0x6d, 0x6f, 0x75, 0x6e, 0x74, 0x20, + 0x6f, 0x66, 0x20, 0x77, 0x6f, 0x72, 0x6b, 0x20, 0x6e, 0x6f, 0x74, 0x20, 0x73, 0x68, 0x61, 0x72, + 0x65, 0x64, 0x20, 0x77, 0x69, 0x74, 0x68, 0x20, 0x61, 0x6e, 0x79, 0x20, 0x6f, 0x74, 0x68, 0x65, + 0x72, 0x20, 0x61, 0x63, 0x74, 0x69, 0x76, 0x65, 0x20, 0x65, 0x6c, 0x65, 0x6d, 0x65, 0x6e, 0x74, + 0x2e, 0x12, 0x9f, 0x02, 0x0a, 0x0e, 0x57, 0x4f, 0x52, 0x4b, 0x5f, 0x43, 0x4f, 0x4d, 0x50, 0x4c, + 0x45, 0x54, 0x45, 0x44, 0x10, 0x11, 0x1a, 0x8a, 0x02, 0xd2, 0xa7, 0xa7, 0x96, 0x06, 0x83, 0x02, + 0x0a, 0x2c, 0x62, 0x65, 0x61, 0x6d, 0x3a, 0x6d, 0x65, 0x74, 0x72, 0x69, 0x63, 0x3a, 0x70, 0x74, + 0x72, 0x61, 0x6e, 0x73, 0x66, 0x6f, 0x72, 0x6d, 0x5f, 0x70, 0x72, 0x6f, 0x67, 0x72, 0x65, 0x73, + 0x73, 0x3a, 0x63, 0x6f, 0x6d, 0x70, 0x6c, 0x65, 0x74, 0x65, 0x64, 0x3a, 0x76, 0x31, 0x12, 0x18, + 0x62, 0x65, 0x61, 0x6d, 0x3a, 0x6d, 0x65, 0x74, 0x72, 0x69, 0x63, 0x73, 0x3a, 0x70, 0x72, 0x6f, + 0x67, 0x72, 0x65, 0x73, 0x73, 0x3a, 0x76, 0x31, 0x1a, 0x0a, 0x50, 0x54, 0x52, 0x41, 0x4e, 0x53, + 0x46, 0x4f, 0x52, 0x4d, 0x22, 0xac, 0x01, 0x0a, 0x0b, 0x64, 0x65, 0x73, 0x63, 0x72, 0x69, 0x70, + 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x9c, 0x01, 0x54, 0x68, 0x65, 0x20, 0x72, 0x65, 0x6d, 0x61, 0x69, + 0x6e, 0x69, 0x6e, 0x67, 0x20, 0x61, 0x6d, 0x6f, 0x75, 0x6e, 0x74, 0x20, 0x6f, 0x66, 0x20, 0x77, + 0x6f, 0x72, 0x6b, 0x20, 0x66, 0x6f, 0x72, 0x20, 0x65, 0x61, 0x63, 0x68, 0x20, 0x61, 0x63, 0x74, + 0x69, 0x76, 0x65, 0x20, 0x65, 0x6c, 0x65, 0x6d, 0x65, 0x6e, 0x74, 0x2e, 0x20, 0x45, 0x61, 0x63, + 0x68, 0x20, 0x61, 0x63, 0x74, 0x69, 0x76, 0x65, 0x20, 0x65, 0x6c, 0x65, 0x6d, 0x65, 0x6e, 0x74, + 0x20, 0x72, 0x65, 0x70, 0x72, 0x65, 0x73, 0x65, 0x6e, 0x74, 0x73, 0x20, 0x61, 0x6e, 0x20, 0x69, + 0x6e, 0x64, 0x65, 0x70, 0x65, 0x6e, 0x64, 0x65, 0x6e, 0x74, 0x20, 0x61, 0x6d, 0x6f, 0x75, 0x6e, + 0x74, 0x20, 0x6f, 0x66, 0x20, 0x77, 0x6f, 0x72, 0x6b, 0x20, 0x6e, 0x6f, 0x74, 0x20, 0x73, 0x68, + 0x61, 0x72, 0x65, 0x64, 0x20, 0x77, 0x69, 0x74, 0x68, 0x20, 0x61, 0x6e, 0x79, 0x20, 0x6f, 0x74, + 0x68, 0x65, 0x72, 0x20, 0x61, 0x63, 0x74, 0x69, 0x76, 0x65, 0x20, 0x65, 0x6c, 0x65, 0x6d, 0x65, + 0x6e, 0x74, 0x2e, 0x12, 0xa8, 0x01, 0x0a, 0x17, 0x44, 0x41, 0x54, 0x41, 0x5f, 0x43, 0x48, 0x41, + 0x4e, 0x4e, 0x45, 0x4c, 0x5f, 0x52, 0x45, 0x41, 0x44, 0x5f, 0x49, 0x4e, 0x44, 0x45, 0x58, 0x10, + 0x12, 0x1a, 0x8a, 0x01, 0xd2, 0xa7, 0xa7, 0x96, 0x06, 0x83, 0x01, 0x0a, 0x26, 0x62, 0x65, 0x61, + 0x6d, 0x3a, 0x6d, 0x65, 0x74, 0x72, 0x69, 0x63, 0x3a, 0x64, 0x61, 0x74, 0x61, 0x5f, 0x63, 0x68, + 0x61, 0x6e, 0x6e, 0x65, 0x6c, 0x3a, 0x72, 0x65, 0x61, 0x64, 0x5f, 0x69, 0x6e, 0x64, 0x65, 0x78, + 0x3a, 0x76, 0x31, 0x12, 0x19, 0x62, 0x65, 0x61, 0x6d, 0x3a, 0x6d, 0x65, 0x74, 0x72, 0x69, 0x63, + 0x73, 0x3a, 0x73, 0x75, 0x6d, 0x5f, 0x69, 0x6e, 0x74, 0x36, 0x34, 0x3a, 0x76, 0x31, 0x1a, 0x0a, + 0x50, 0x54, 0x52, 0x41, 0x4e, 0x53, 0x46, 0x4f, 0x52, 0x4d, 0x22, 0x32, 0x0a, 0x0b, 0x64, 0x65, + 0x73, 0x63, 0x72, 0x69, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x23, 0x54, 0x68, 0x65, 0x20, 0x72, + 0x65, 0x61, 0x64, 0x20, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x20, 0x6f, 0x66, 0x20, 0x74, 0x68, 0x65, + 0x20, 0x64, 0x61, 0x74, 0x61, 0x20, 0x63, 0x68, 0x61, 0x6e, 0x6e, 0x65, 0x6c, 0x2e, 0x12, 0x8a, + 0x02, 0x0a, 0x11, 0x41, 0x50, 0x49, 0x5f, 0x52, 0x45, 0x51, 0x55, 0x45, 0x53, 0x54, 0x5f, 0x43, + 0x4f, 0x55, 0x4e, 0x54, 0x10, 0x13, 0x1a, 0xf2, 0x01, 0xd2, 0xa7, 0xa7, 0x96, 0x06, 0xeb, 0x01, + 0x0a, 0x23, 0x62, 0x65, 0x61, 0x6d, 0x3a, 0x6d, 0x65, 0x74, 0x72, 0x69, 0x63, 0x3a, 0x69, 0x6f, + 0x3a, 0x61, 0x70, 0x69, 0x5f, 0x72, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x5f, 0x63, 0x6f, 0x75, + 0x6e, 0x74, 0x3a, 0x76, 0x31, 0x12, 0x19, 0x62, 0x65, 0x61, 0x6d, 0x3a, 0x6d, 0x65, 0x74, 0x72, + 0x69, 0x63, 0x73, 0x3a, 0x73, 0x75, 0x6d, 0x5f, 0x69, 0x6e, 0x74, 0x36, 0x34, 0x3a, 0x76, 0x31, + 0x1a, 0x07, 0x53, 0x45, 0x52, 0x56, 0x49, 0x43, 0x45, 0x1a, 0x06, 0x4d, 0x45, 0x54, 0x48, 0x4f, + 0x44, 0x1a, 0x08, 0x52, 0x45, 0x53, 0x4f, 0x55, 0x52, 0x43, 0x45, 0x1a, 0x0a, 0x50, 0x54, 0x52, + 0x41, 0x4e, 0x53, 0x46, 0x4f, 0x52, 0x4d, 0x1a, 0x06, 0x53, 0x54, 0x41, 0x54, 0x55, 0x53, 0x22, + 0x62, 0x0a, 0x0b, 0x64, 0x65, 0x73, 0x63, 0x72, 0x69, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x53, + 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x20, 0x63, 0x6f, 0x75, 0x6e, 0x74, 0x73, 0x20, 0x77, + 0x69, 0x74, 0x68, 0x20, 0x73, 0x74, 0x61, 0x74, 0x75, 0x73, 0x20, 0x6d, 0x61, 0x64, 0x65, 0x20, + 0x74, 0x6f, 0x20, 0x49, 0x4f, 0x20, 0x73, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, 0x20, 0x41, 0x50, + 0x49, 0x73, 0x20, 0x74, 0x6f, 0x20, 0x62, 0x61, 0x74, 0x63, 0x68, 0x20, 0x72, 0x65, 0x61, 0x64, + 0x20, 0x6f, 0x72, 0x20, 0x77, 0x72, 0x69, 0x74, 0x65, 0x20, 0x65, 0x6c, 0x65, 0x6d, 0x65, 0x6e, + 0x74, 0x73, 0x2e, 0x22, 0x16, 0x0a, 0x0e, 0x70, 0x72, 0x6f, 0x63, 0x65, 0x73, 0x73, 0x5f, 0x6d, + 0x65, 0x74, 0x72, 0x69, 0x63, 0x12, 0x04, 0x74, 0x72, 0x75, 0x65, 0x12, 0xb3, 0x02, 0x0a, 0x15, + 0x41, 0x50, 0x49, 0x5f, 0x52, 0x45, 0x51, 0x55, 0x45, 0x53, 0x54, 0x5f, 0x4c, 0x41, 0x54, 0x45, + 0x4e, 0x43, 0x49, 0x45, 0x53, 0x10, 0x14, 0x1a, 0x97, 0x02, 0xd2, 0xa7, 0xa7, 0x96, 0x06, 0x90, + 0x02, 0x0a, 0x27, 0x62, 0x65, 0x61, 0x6d, 0x3a, 0x6d, 0x65, 0x74, 0x72, 0x69, 0x63, 0x3a, 0x69, + 0x6f, 0x3a, 0x61, 0x70, 0x69, 0x5f, 0x72, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x5f, 0x6c, 0x61, + 0x74, 0x65, 0x6e, 0x63, 0x69, 0x65, 0x73, 0x3a, 0x76, 0x31, 0x12, 0x1f, 0x62, 0x65, 0x61, 0x6d, + 0x3a, 0x6d, 0x65, 0x74, 0x72, 0x69, 0x63, 0x73, 0x3a, 0x68, 0x69, 0x73, 0x74, 0x6f, 0x67, 0x72, + 0x61, 0x6d, 0x5f, 0x69, 0x6e, 0x74, 0x36, 0x34, 0x3a, 0x76, 0x31, 0x1a, 0x07, 0x53, 0x45, 0x52, + 0x56, 0x49, 0x43, 0x45, 0x1a, 0x06, 0x4d, 0x45, 0x54, 0x48, 0x4f, 0x44, 0x1a, 0x08, 0x52, 0x45, + 0x53, 0x4f, 0x55, 0x52, 0x43, 0x45, 0x1a, 0x0a, 0x50, 0x54, 0x52, 0x41, 0x4e, 0x53, 0x46, 0x4f, + 0x52, 0x4d, 0x22, 0x6e, 0x0a, 0x0b, 0x64, 0x65, 0x73, 0x63, 0x72, 0x69, 0x70, 0x74, 0x69, 0x6f, + 0x6e, 0x12, 0x5f, 0x48, 0x69, 0x73, 0x74, 0x6f, 0x67, 0x72, 0x61, 0x6d, 0x20, 0x63, 0x6f, 0x75, + 0x6e, 0x74, 0x73, 0x20, 0x66, 0x6f, 0x72, 0x20, 0x72, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x20, + 0x6c, 0x61, 0x74, 0x65, 0x6e, 0x63, 0x69, 0x65, 0x73, 0x20, 0x6d, 0x61, 0x64, 0x65, 0x20, 0x74, + 0x6f, 0x20, 0x49, 0x4f, 0x20, 0x73, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, 0x20, 0x41, 0x50, 0x49, + 0x73, 0x20, 0x74, 0x6f, 0x20, 0x62, 0x61, 0x74, 0x63, 0x68, 0x20, 0x72, 0x65, 0x61, 0x64, 0x20, + 0x6f, 0x72, 0x20, 0x77, 0x72, 0x69, 0x74, 0x65, 0x20, 0x65, 0x6c, 0x65, 0x6d, 0x65, 0x6e, 0x74, + 0x73, 0x2e, 0x22, 0x15, 0x0a, 0x05, 0x75, 0x6e, 0x69, 0x74, 0x73, 0x12, 0x0c, 0x4d, 0x69, 0x6c, + 0x6c, 0x69, 0x73, 0x65, 0x63, 0x6f, 0x6e, 0x64, 0x73, 0x22, 0x16, 0x0a, 0x0e, 0x70, 0x72, 0x6f, + 0x63, 0x65, 0x73, 0x73, 0x5f, 0x6d, 0x65, 0x74, 0x72, 0x69, 0x63, 0x12, 0x04, 0x74, 0x72, 0x75, + 0x65, 0x22, 0x2e, 0x0a, 0x18, 0x4d, 0x6f, 0x6e, 0x69, 0x74, 0x6f, 0x72, 0x69, 0x6e, 0x67, 0x49, + 0x6e, 0x66, 0x6f, 0x4c, 0x61, 0x62, 0x65, 0x6c, 0x50, 0x72, 0x6f, 0x70, 0x73, 0x12, 0x12, 0x0a, + 0x04, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x04, 0x6e, 0x61, 0x6d, + 0x65, 0x22, 0xcd, 0x0b, 0x0a, 0x0e, 0x4d, 0x6f, 0x6e, 0x69, 0x74, 0x6f, 0x72, 0x69, 0x6e, 0x67, + 0x49, 0x6e, 0x66, 0x6f, 0x12, 0x10, 0x0a, 0x03, 0x75, 0x72, 0x6e, 0x18, 0x01, 0x20, 0x01, 0x28, + 0x09, 0x52, 0x03, 0x75, 0x72, 0x6e, 0x12, 0x12, 0x0a, 0x04, 0x74, 0x79, 0x70, 0x65, 0x18, 0x02, + 0x20, 0x01, 0x28, 0x09, 0x52, 0x04, 0x74, 0x79, 0x70, 0x65, 0x12, 0x18, 0x0a, 0x07, 0x70, 0x61, + 0x79, 0x6c, 0x6f, 0x61, 0x64, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0c, 0x52, 0x07, 0x70, 0x61, 0x79, + 0x6c, 0x6f, 0x61, 0x64, 0x12, 0x55, 0x0a, 0x06, 0x6c, 0x61, 0x62, 0x65, 0x6c, 0x73, 0x18, 0x04, + 0x20, 0x03, 0x28, 0x0b, 0x32, 0x3d, 0x2e, 0x6f, 0x72, 0x67, 0x2e, 0x61, 0x70, 0x61, 0x63, 0x68, + 0x65, 0x2e, 0x62, 0x65, 0x61, 0x6d, 0x2e, 0x6d, 0x6f, 0x64, 0x65, 0x6c, 0x2e, 0x70, 0x69, 0x70, + 0x65, 0x6c, 0x69, 0x6e, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x4d, 0x6f, 0x6e, 0x69, 0x74, 0x6f, 0x72, + 0x69, 0x6e, 0x67, 0x49, 0x6e, 0x66, 0x6f, 0x2e, 0x4c, 0x61, 0x62, 0x65, 0x6c, 0x73, 0x45, 0x6e, + 0x74, 0x72, 0x79, 0x52, 0x06, 0x6c, 0x61, 0x62, 0x65, 0x6c, 0x73, 0x12, 0x39, 0x0a, 0x0a, 0x73, + 0x74, 0x61, 0x72, 0x74, 0x5f, 0x74, 0x69, 0x6d, 0x65, 0x18, 0x05, 0x20, 0x01, 0x28, 0x0b, 0x32, + 0x1a, 0x2e, 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75, + 0x66, 0x2e, 0x54, 0x69, 0x6d, 0x65, 0x73, 0x74, 0x61, 0x6d, 0x70, 0x52, 0x09, 0x73, 0x74, 0x61, + 0x72, 0x74, 0x54, 0x69, 0x6d, 0x65, 0x1a, 0x39, 0x0a, 0x0b, 0x4c, 0x61, 0x62, 0x65, 0x6c, 0x73, + 0x45, 0x6e, 0x74, 0x72, 0x79, 0x12, 0x10, 0x0a, 0x03, 0x6b, 0x65, 0x79, 0x18, 0x01, 0x20, 0x01, + 0x28, 0x09, 0x52, 0x03, 0x6b, 0x65, 0x79, 0x12, 0x14, 0x0a, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, + 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x3a, 0x02, 0x38, + 0x01, 0x22, 0xad, 0x09, 0x0a, 0x14, 0x4d, 0x6f, 0x6e, 0x69, 0x74, 0x6f, 0x72, 0x69, 0x6e, 0x67, + 0x49, 0x6e, 0x66, 0x6f, 0x4c, 0x61, 0x62, 0x65, 0x6c, 0x73, 0x12, 0x21, 0x0a, 0x09, 0x54, 0x52, + 0x41, 0x4e, 0x53, 0x46, 0x4f, 0x52, 0x4d, 0x10, 0x00, 0x1a, 0x12, 0xa2, 0xd4, 0xe0, 0xe5, 0x03, + 0x0c, 0x0a, 0x0a, 0x50, 0x54, 0x52, 0x41, 0x4e, 0x53, 0x46, 0x4f, 0x52, 0x4d, 0x12, 0x24, 0x0a, + 0x0b, 0x50, 0x43, 0x4f, 0x4c, 0x4c, 0x45, 0x43, 0x54, 0x49, 0x4f, 0x4e, 0x10, 0x01, 0x1a, 0x13, + 0xa2, 0xd4, 0xe0, 0xe5, 0x03, 0x0d, 0x0a, 0x0b, 0x50, 0x43, 0x4f, 0x4c, 0x4c, 0x45, 0x43, 0x54, + 0x49, 0x4f, 0x4e, 0x12, 0x32, 0x0a, 0x12, 0x57, 0x49, 0x4e, 0x44, 0x4f, 0x57, 0x49, 0x4e, 0x47, + 0x5f, 0x53, 0x54, 0x52, 0x41, 0x54, 0x45, 0x47, 0x59, 0x10, 0x02, 0x1a, 0x1a, 0xa2, 0xd4, 0xe0, + 0xe5, 0x03, 0x14, 0x0a, 0x12, 0x57, 0x49, 0x4e, 0x44, 0x4f, 0x57, 0x49, 0x4e, 0x47, 0x5f, 0x53, + 0x54, 0x52, 0x41, 0x54, 0x45, 0x47, 0x59, 0x12, 0x18, 0x0a, 0x05, 0x43, 0x4f, 0x44, 0x45, 0x52, + 0x10, 0x03, 0x1a, 0x0d, 0xa2, 0xd4, 0xe0, 0xe5, 0x03, 0x07, 0x0a, 0x05, 0x43, 0x4f, 0x44, 0x45, + 0x52, 0x12, 0x24, 0x0a, 0x0b, 0x45, 0x4e, 0x56, 0x49, 0x52, 0x4f, 0x4e, 0x4d, 0x45, 0x4e, 0x54, + 0x10, 0x04, 0x1a, 0x13, 0xa2, 0xd4, 0xe0, 0xe5, 0x03, 0x0d, 0x0a, 0x0b, 0x45, 0x4e, 0x56, 0x49, + 0x52, 0x4f, 0x4e, 0x4d, 0x45, 0x4e, 0x54, 0x12, 0x20, 0x0a, 0x09, 0x4e, 0x41, 0x4d, 0x45, 0x53, + 0x50, 0x41, 0x43, 0x45, 0x10, 0x05, 0x1a, 0x11, 0xa2, 0xd4, 0xe0, 0xe5, 0x03, 0x0b, 0x0a, 0x09, + 0x4e, 0x41, 0x4d, 0x45, 0x53, 0x50, 0x41, 0x43, 0x45, 0x12, 0x16, 0x0a, 0x04, 0x4e, 0x41, 0x4d, + 0x45, 0x10, 0x06, 0x1a, 0x0c, 0xa2, 0xd4, 0xe0, 0xe5, 0x03, 0x06, 0x0a, 0x04, 0x4e, 0x41, 0x4d, + 0x45, 0x12, 0x1c, 0x0a, 0x07, 0x53, 0x45, 0x52, 0x56, 0x49, 0x43, 0x45, 0x10, 0x07, 0x1a, 0x0f, + 0xa2, 0xd4, 0xe0, 0xe5, 0x03, 0x09, 0x0a, 0x07, 0x53, 0x45, 0x52, 0x56, 0x49, 0x43, 0x45, 0x12, + 0x1a, 0x0a, 0x06, 0x4d, 0x45, 0x54, 0x48, 0x4f, 0x44, 0x10, 0x08, 0x1a, 0x0e, 0xa2, 0xd4, 0xe0, + 0xe5, 0x03, 0x08, 0x0a, 0x06, 0x4d, 0x45, 0x54, 0x48, 0x4f, 0x44, 0x12, 0x1e, 0x0a, 0x08, 0x52, + 0x45, 0x53, 0x4f, 0x55, 0x52, 0x43, 0x45, 0x10, 0x09, 0x1a, 0x10, 0xa2, 0xd4, 0xe0, 0xe5, 0x03, + 0x0a, 0x0a, 0x08, 0x52, 0x45, 0x53, 0x4f, 0x55, 0x52, 0x43, 0x45, 0x12, 0x1a, 0x0a, 0x06, 0x53, + 0x54, 0x41, 0x54, 0x55, 0x53, 0x10, 0x0a, 0x1a, 0x0e, 0xa2, 0xd4, 0xe0, 0xe5, 0x03, 0x08, 0x0a, + 0x06, 0x53, 0x54, 0x41, 0x54, 0x55, 0x53, 0x12, 0x34, 0x0a, 0x13, 0x42, 0x49, 0x47, 0x51, 0x55, + 0x45, 0x52, 0x59, 0x5f, 0x50, 0x52, 0x4f, 0x4a, 0x45, 0x43, 0x54, 0x5f, 0x49, 0x44, 0x10, 0x0b, + 0x1a, 0x1b, 0xa2, 0xd4, 0xe0, 0xe5, 0x03, 0x15, 0x0a, 0x13, 0x42, 0x49, 0x47, 0x51, 0x55, 0x45, + 0x52, 0x59, 0x5f, 0x50, 0x52, 0x4f, 0x4a, 0x45, 0x43, 0x54, 0x5f, 0x49, 0x44, 0x12, 0x2e, 0x0a, + 0x10, 0x42, 0x49, 0x47, 0x51, 0x55, 0x45, 0x52, 0x59, 0x5f, 0x44, 0x41, 0x54, 0x41, 0x53, 0x45, + 0x54, 0x10, 0x0c, 0x1a, 0x18, 0xa2, 0xd4, 0xe0, 0xe5, 0x03, 0x12, 0x0a, 0x10, 0x42, 0x49, 0x47, + 0x51, 0x55, 0x45, 0x52, 0x59, 0x5f, 0x44, 0x41, 0x54, 0x41, 0x53, 0x45, 0x54, 0x12, 0x2a, 0x0a, + 0x0e, 0x42, 0x49, 0x47, 0x51, 0x55, 0x45, 0x52, 0x59, 0x5f, 0x54, 0x41, 0x42, 0x4c, 0x45, 0x10, + 0x0d, 0x1a, 0x16, 0xa2, 0xd4, 0xe0, 0xe5, 0x03, 0x10, 0x0a, 0x0e, 0x42, 0x49, 0x47, 0x51, 0x55, + 0x45, 0x52, 0x59, 0x5f, 0x54, 0x41, 0x42, 0x4c, 0x45, 0x12, 0x28, 0x0a, 0x0d, 0x42, 0x49, 0x47, + 0x51, 0x55, 0x45, 0x52, 0x59, 0x5f, 0x56, 0x49, 0x45, 0x57, 0x10, 0x0e, 0x1a, 0x15, 0xa2, 0xd4, + 0xe0, 0xe5, 0x03, 0x0f, 0x0a, 0x0d, 0x42, 0x49, 0x47, 0x51, 0x55, 0x45, 0x52, 0x59, 0x5f, 0x56, + 0x49, 0x45, 0x57, 0x12, 0x34, 0x0a, 0x13, 0x42, 0x49, 0x47, 0x51, 0x55, 0x45, 0x52, 0x59, 0x5f, + 0x51, 0x55, 0x45, 0x52, 0x59, 0x5f, 0x4e, 0x41, 0x4d, 0x45, 0x10, 0x0f, 0x1a, 0x1b, 0xa2, 0xd4, + 0xe0, 0xe5, 0x03, 0x15, 0x0a, 0x13, 0x42, 0x49, 0x47, 0x51, 0x55, 0x45, 0x52, 0x59, 0x5f, 0x51, + 0x55, 0x45, 0x52, 0x59, 0x5f, 0x4e, 0x41, 0x4d, 0x45, 0x12, 0x22, 0x0a, 0x0a, 0x47, 0x43, 0x53, + 0x5f, 0x42, 0x55, 0x43, 0x4b, 0x45, 0x54, 0x10, 0x10, 0x1a, 0x12, 0xa2, 0xd4, 0xe0, 0xe5, 0x03, + 0x0c, 0x0a, 0x0a, 0x47, 0x43, 0x53, 0x5f, 0x42, 0x55, 0x43, 0x4b, 0x45, 0x54, 0x12, 0x2a, 0x0a, + 0x0e, 0x47, 0x43, 0x53, 0x5f, 0x50, 0x52, 0x4f, 0x4a, 0x45, 0x43, 0x54, 0x5f, 0x49, 0x44, 0x10, + 0x11, 0x1a, 0x16, 0xa2, 0xd4, 0xe0, 0xe5, 0x03, 0x10, 0x0a, 0x0e, 0x47, 0x43, 0x53, 0x5f, 0x50, + 0x52, 0x4f, 0x4a, 0x45, 0x43, 0x54, 0x5f, 0x49, 0x44, 0x12, 0x30, 0x0a, 0x11, 0x44, 0x41, 0x54, + 0x41, 0x53, 0x54, 0x4f, 0x52, 0x45, 0x5f, 0x50, 0x52, 0x4f, 0x4a, 0x45, 0x43, 0x54, 0x10, 0x12, + 0x1a, 0x19, 0xa2, 0xd4, 0xe0, 0xe5, 0x03, 0x13, 0x0a, 0x11, 0x44, 0x41, 0x54, 0x41, 0x53, 0x54, + 0x4f, 0x52, 0x45, 0x5f, 0x50, 0x52, 0x4f, 0x4a, 0x45, 0x43, 0x54, 0x12, 0x34, 0x0a, 0x13, 0x44, + 0x41, 0x54, 0x41, 0x53, 0x54, 0x4f, 0x52, 0x45, 0x5f, 0x4e, 0x41, 0x4d, 0x45, 0x53, 0x50, 0x41, + 0x43, 0x45, 0x10, 0x13, 0x1a, 0x1b, 0xa2, 0xd4, 0xe0, 0xe5, 0x03, 0x15, 0x0a, 0x13, 0x44, 0x41, + 0x54, 0x41, 0x53, 0x54, 0x4f, 0x52, 0x45, 0x5f, 0x4e, 0x41, 0x4d, 0x45, 0x53, 0x50, 0x41, 0x43, + 0x45, 0x12, 0x34, 0x0a, 0x13, 0x42, 0x49, 0x47, 0x54, 0x41, 0x42, 0x4c, 0x45, 0x5f, 0x50, 0x52, + 0x4f, 0x4a, 0x45, 0x43, 0x54, 0x5f, 0x49, 0x44, 0x10, 0x14, 0x1a, 0x1b, 0xa2, 0xd4, 0xe0, 0xe5, + 0x03, 0x15, 0x0a, 0x13, 0x42, 0x49, 0x47, 0x54, 0x41, 0x42, 0x4c, 0x45, 0x5f, 0x50, 0x52, 0x4f, + 0x4a, 0x45, 0x43, 0x54, 0x5f, 0x49, 0x44, 0x12, 0x24, 0x0a, 0x0b, 0x49, 0x4e, 0x53, 0x54, 0x41, + 0x4e, 0x43, 0x45, 0x5f, 0x49, 0x44, 0x10, 0x15, 0x1a, 0x13, 0xa2, 0xd4, 0xe0, 0xe5, 0x03, 0x0d, + 0x0a, 0x0b, 0x49, 0x4e, 0x53, 0x54, 0x41, 0x4e, 0x43, 0x45, 0x5f, 0x49, 0x44, 0x12, 0x1e, 0x0a, + 0x08, 0x54, 0x41, 0x42, 0x4c, 0x45, 0x5f, 0x49, 0x44, 0x10, 0x16, 0x1a, 0x10, 0xa2, 0xd4, 0xe0, + 0xe5, 0x03, 0x0a, 0x0a, 0x08, 0x54, 0x41, 0x42, 0x4c, 0x45, 0x5f, 0x49, 0x44, 0x12, 0x32, 0x0a, + 0x12, 0x53, 0x50, 0x41, 0x4e, 0x4e, 0x45, 0x52, 0x5f, 0x50, 0x52, 0x4f, 0x4a, 0x45, 0x43, 0x54, + 0x5f, 0x49, 0x44, 0x10, 0x17, 0x1a, 0x1a, 0xa2, 0xd4, 0xe0, 0xe5, 0x03, 0x14, 0x0a, 0x12, 0x53, + 0x50, 0x41, 0x4e, 0x4e, 0x45, 0x52, 0x5f, 0x50, 0x52, 0x4f, 0x4a, 0x45, 0x43, 0x54, 0x5f, 0x49, + 0x44, 0x12, 0x34, 0x0a, 0x13, 0x53, 0x50, 0x41, 0x4e, 0x4e, 0x45, 0x52, 0x5f, 0x44, 0x41, 0x54, + 0x41, 0x42, 0x41, 0x53, 0x45, 0x5f, 0x49, 0x44, 0x10, 0x18, 0x1a, 0x1b, 0xa2, 0xd4, 0xe0, 0xe5, + 0x03, 0x15, 0x0a, 0x13, 0x53, 0x50, 0x41, 0x4e, 0x4e, 0x45, 0x52, 0x5f, 0x44, 0x41, 0x54, 0x41, + 0x42, 0x41, 0x53, 0x45, 0x5f, 0x49, 0x44, 0x12, 0x2e, 0x0a, 0x10, 0x53, 0x50, 0x41, 0x4e, 0x4e, + 0x45, 0x52, 0x5f, 0x54, 0x41, 0x42, 0x4c, 0x45, 0x5f, 0x49, 0x44, 0x10, 0x19, 0x1a, 0x18, 0xa2, + 0xd4, 0xe0, 0xe5, 0x03, 0x12, 0x0a, 0x10, 0x53, 0x50, 0x41, 0x4e, 0x4e, 0x45, 0x52, 0x5f, 0x54, + 0x41, 0x42, 0x4c, 0x45, 0x5f, 0x49, 0x44, 0x12, 0x34, 0x0a, 0x13, 0x53, 0x50, 0x41, 0x4e, 0x4e, + 0x45, 0x52, 0x5f, 0x49, 0x4e, 0x53, 0x54, 0x41, 0x4e, 0x43, 0x45, 0x5f, 0x49, 0x44, 0x10, 0x1a, + 0x1a, 0x1b, 0xa2, 0xd4, 0xe0, 0xe5, 0x03, 0x15, 0x0a, 0x13, 0x53, 0x50, 0x41, 0x4e, 0x4e, 0x45, + 0x52, 0x5f, 0x49, 0x4e, 0x53, 0x54, 0x41, 0x4e, 0x43, 0x45, 0x5f, 0x49, 0x44, 0x12, 0x32, 0x0a, + 0x12, 0x53, 0x50, 0x41, 0x4e, 0x4e, 0x45, 0x52, 0x5f, 0x51, 0x55, 0x45, 0x52, 0x59, 0x5f, 0x4e, + 0x41, 0x4d, 0x45, 0x10, 0x1b, 0x1a, 0x1a, 0xa2, 0xd4, 0xe0, 0xe5, 0x03, 0x14, 0x0a, 0x12, 0x53, + 0x50, 0x41, 0x4e, 0x4e, 0x45, 0x52, 0x5f, 0x51, 0x55, 0x45, 0x52, 0x59, 0x5f, 0x4e, 0x41, 0x4d, + 0x45, 0x22, 0xae, 0x06, 0x0a, 0x16, 0x4d, 0x6f, 0x6e, 0x69, 0x74, 0x6f, 0x72, 0x69, 0x6e, 0x67, + 0x49, 0x6e, 0x66, 0x6f, 0x54, 0x79, 0x70, 0x65, 0x55, 0x72, 0x6e, 0x73, 0x22, 0x93, 0x06, 0x0a, + 0x04, 0x45, 0x6e, 0x75, 0x6d, 0x12, 0x33, 0x0a, 0x0e, 0x53, 0x55, 0x4d, 0x5f, 0x49, 0x4e, 0x54, + 0x36, 0x34, 0x5f, 0x54, 0x59, 0x50, 0x45, 0x10, 0x00, 0x1a, 0x1f, 0xa2, 0xb4, 0xfa, 0xc2, 0x05, + 0x19, 0x62, 0x65, 0x61, 0x6d, 0x3a, 0x6d, 0x65, 0x74, 0x72, 0x69, 0x63, 0x73, 0x3a, 0x73, 0x75, + 0x6d, 0x5f, 0x69, 0x6e, 0x74, 0x36, 0x34, 0x3a, 0x76, 0x31, 0x12, 0x35, 0x0a, 0x0f, 0x53, 0x55, + 0x4d, 0x5f, 0x44, 0x4f, 0x55, 0x42, 0x4c, 0x45, 0x5f, 0x54, 0x59, 0x50, 0x45, 0x10, 0x01, 0x1a, + 0x20, 0xa2, 0xb4, 0xfa, 0xc2, 0x05, 0x1a, 0x62, 0x65, 0x61, 0x6d, 0x3a, 0x6d, 0x65, 0x74, 0x72, + 0x69, 0x63, 0x73, 0x3a, 0x73, 0x75, 0x6d, 0x5f, 0x64, 0x6f, 0x75, 0x62, 0x6c, 0x65, 0x3a, 0x76, + 0x31, 0x12, 0x45, 0x0a, 0x17, 0x44, 0x49, 0x53, 0x54, 0x52, 0x49, 0x42, 0x55, 0x54, 0x49, 0x4f, + 0x4e, 0x5f, 0x49, 0x4e, 0x54, 0x36, 0x34, 0x5f, 0x54, 0x59, 0x50, 0x45, 0x10, 0x02, 0x1a, 0x28, + 0xa2, 0xb4, 0xfa, 0xc2, 0x05, 0x22, 0x62, 0x65, 0x61, 0x6d, 0x3a, 0x6d, 0x65, 0x74, 0x72, 0x69, + 0x63, 0x73, 0x3a, 0x64, 0x69, 0x73, 0x74, 0x72, 0x69, 0x62, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x5f, + 0x69, 0x6e, 0x74, 0x36, 0x34, 0x3a, 0x76, 0x31, 0x12, 0x47, 0x0a, 0x18, 0x44, 0x49, 0x53, 0x54, + 0x52, 0x49, 0x42, 0x55, 0x54, 0x49, 0x4f, 0x4e, 0x5f, 0x44, 0x4f, 0x55, 0x42, 0x4c, 0x45, 0x5f, + 0x54, 0x59, 0x50, 0x45, 0x10, 0x03, 0x1a, 0x29, 0xa2, 0xb4, 0xfa, 0xc2, 0x05, 0x23, 0x62, 0x65, + 0x61, 0x6d, 0x3a, 0x6d, 0x65, 0x74, 0x72, 0x69, 0x63, 0x73, 0x3a, 0x64, 0x69, 0x73, 0x74, 0x72, + 0x69, 0x62, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x64, 0x6f, 0x75, 0x62, 0x6c, 0x65, 0x3a, 0x76, + 0x31, 0x12, 0x39, 0x0a, 0x11, 0x4c, 0x41, 0x54, 0x45, 0x53, 0x54, 0x5f, 0x49, 0x4e, 0x54, 0x36, + 0x34, 0x5f, 0x54, 0x59, 0x50, 0x45, 0x10, 0x04, 0x1a, 0x22, 0xa2, 0xb4, 0xfa, 0xc2, 0x05, 0x1c, + 0x62, 0x65, 0x61, 0x6d, 0x3a, 0x6d, 0x65, 0x74, 0x72, 0x69, 0x63, 0x73, 0x3a, 0x6c, 0x61, 0x74, + 0x65, 0x73, 0x74, 0x5f, 0x69, 0x6e, 0x74, 0x36, 0x34, 0x3a, 0x76, 0x31, 0x12, 0x3b, 0x0a, 0x12, + 0x4c, 0x41, 0x54, 0x45, 0x53, 0x54, 0x5f, 0x44, 0x4f, 0x55, 0x42, 0x4c, 0x45, 0x5f, 0x54, 0x59, + 0x50, 0x45, 0x10, 0x05, 0x1a, 0x23, 0xa2, 0xb4, 0xfa, 0xc2, 0x05, 0x1d, 0x62, 0x65, 0x61, 0x6d, + 0x3a, 0x6d, 0x65, 0x74, 0x72, 0x69, 0x63, 0x73, 0x3a, 0x6c, 0x61, 0x74, 0x65, 0x73, 0x74, 0x5f, + 0x64, 0x6f, 0x75, 0x62, 0x6c, 0x65, 0x3a, 0x76, 0x31, 0x12, 0x37, 0x0a, 0x10, 0x54, 0x4f, 0x50, + 0x5f, 0x4e, 0x5f, 0x49, 0x4e, 0x54, 0x36, 0x34, 0x5f, 0x54, 0x59, 0x50, 0x45, 0x10, 0x06, 0x1a, + 0x21, 0xa2, 0xb4, 0xfa, 0xc2, 0x05, 0x1b, 0x62, 0x65, 0x61, 0x6d, 0x3a, 0x6d, 0x65, 0x74, 0x72, + 0x69, 0x63, 0x73, 0x3a, 0x74, 0x6f, 0x70, 0x5f, 0x6e, 0x5f, 0x69, 0x6e, 0x74, 0x36, 0x34, 0x3a, + 0x76, 0x31, 0x12, 0x39, 0x0a, 0x11, 0x54, 0x4f, 0x50, 0x5f, 0x4e, 0x5f, 0x44, 0x4f, 0x55, 0x42, + 0x4c, 0x45, 0x5f, 0x54, 0x59, 0x50, 0x45, 0x10, 0x07, 0x1a, 0x22, 0xa2, 0xb4, 0xfa, 0xc2, 0x05, + 0x1c, 0x62, 0x65, 0x61, 0x6d, 0x3a, 0x6d, 0x65, 0x74, 0x72, 0x69, 0x63, 0x73, 0x3a, 0x74, 0x6f, + 0x70, 0x5f, 0x6e, 0x5f, 0x64, 0x6f, 0x75, 0x62, 0x6c, 0x65, 0x3a, 0x76, 0x31, 0x12, 0x3d, 0x0a, + 0x13, 0x42, 0x4f, 0x54, 0x54, 0x4f, 0x4d, 0x5f, 0x4e, 0x5f, 0x49, 0x4e, 0x54, 0x36, 0x34, 0x5f, + 0x54, 0x59, 0x50, 0x45, 0x10, 0x08, 0x1a, 0x24, 0xa2, 0xb4, 0xfa, 0xc2, 0x05, 0x1e, 0x62, 0x65, + 0x61, 0x6d, 0x3a, 0x6d, 0x65, 0x74, 0x72, 0x69, 0x63, 0x73, 0x3a, 0x62, 0x6f, 0x74, 0x74, 0x6f, + 0x6d, 0x5f, 0x6e, 0x5f, 0x69, 0x6e, 0x74, 0x36, 0x34, 0x3a, 0x76, 0x31, 0x12, 0x3f, 0x0a, 0x14, + 0x42, 0x4f, 0x54, 0x54, 0x4f, 0x4d, 0x5f, 0x4e, 0x5f, 0x44, 0x4f, 0x55, 0x42, 0x4c, 0x45, 0x5f, + 0x54, 0x59, 0x50, 0x45, 0x10, 0x09, 0x1a, 0x25, 0xa2, 0xb4, 0xfa, 0xc2, 0x05, 0x1f, 0x62, 0x65, + 0x61, 0x6d, 0x3a, 0x6d, 0x65, 0x74, 0x72, 0x69, 0x63, 0x73, 0x3a, 0x62, 0x6f, 0x74, 0x74, 0x6f, + 0x6d, 0x5f, 0x6e, 0x5f, 0x64, 0x6f, 0x75, 0x62, 0x6c, 0x65, 0x3a, 0x76, 0x31, 0x12, 0x31, 0x0a, + 0x0d, 0x50, 0x52, 0x4f, 0x47, 0x52, 0x45, 0x53, 0x53, 0x5f, 0x54, 0x59, 0x50, 0x45, 0x10, 0x0a, + 0x1a, 0x1e, 0xa2, 0xb4, 0xfa, 0xc2, 0x05, 0x18, 0x62, 0x65, 0x61, 0x6d, 0x3a, 0x6d, 0x65, 0x74, + 0x72, 0x69, 0x63, 0x73, 0x3a, 0x70, 0x72, 0x6f, 0x67, 0x72, 0x65, 0x73, 0x73, 0x3a, 0x76, 0x31, + 0x12, 0x35, 0x0a, 0x0f, 0x53, 0x45, 0x54, 0x5f, 0x53, 0x54, 0x52, 0x49, 0x4e, 0x47, 0x5f, 0x54, + 0x59, 0x50, 0x45, 0x10, 0x0b, 0x1a, 0x20, 0xa2, 0xb4, 0xfa, 0xc2, 0x05, 0x1a, 0x62, 0x65, 0x61, + 0x6d, 0x3a, 0x6d, 0x65, 0x74, 0x72, 0x69, 0x63, 0x73, 0x3a, 0x73, 0x65, 0x74, 0x5f, 0x73, 0x74, + 0x72, 0x69, 0x6e, 0x67, 0x3a, 0x76, 0x31, 0x12, 0x39, 0x0a, 0x11, 0x42, 0x4f, 0x55, 0x4e, 0x44, + 0x45, 0x44, 0x5f, 0x54, 0x52, 0x49, 0x45, 0x5f, 0x54, 0x59, 0x50, 0x45, 0x10, 0x0c, 0x1a, 0x22, + 0xa2, 0xb4, 0xfa, 0xc2, 0x05, 0x1c, 0x62, 0x65, 0x61, 0x6d, 0x3a, 0x6d, 0x65, 0x74, 0x72, 0x69, + 0x63, 0x73, 0x3a, 0x62, 0x6f, 0x75, 0x6e, 0x64, 0x65, 0x64, 0x5f, 0x74, 0x72, 0x69, 0x65, 0x3a, + 0x76, 0x31, 0x22, 0xfe, 0x01, 0x0a, 0x0f, 0x42, 0x6f, 0x75, 0x6e, 0x64, 0x65, 0x64, 0x54, 0x72, + 0x69, 0x65, 0x4e, 0x6f, 0x64, 0x65, 0x12, 0x1c, 0x0a, 0x09, 0x74, 0x72, 0x75, 0x6e, 0x63, 0x61, + 0x74, 0x65, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x08, 0x52, 0x09, 0x74, 0x72, 0x75, 0x6e, 0x63, + 0x61, 0x74, 0x65, 0x64, 0x12, 0x5c, 0x0a, 0x08, 0x63, 0x68, 0x69, 0x6c, 0x64, 0x72, 0x65, 0x6e, + 0x18, 0x02, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x40, 0x2e, 0x6f, 0x72, 0x67, 0x2e, 0x61, 0x70, 0x61, + 0x63, 0x68, 0x65, 0x2e, 0x62, 0x65, 0x61, 0x6d, 0x2e, 0x6d, 0x6f, 0x64, 0x65, 0x6c, 0x2e, 0x70, + 0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x42, 0x6f, 0x75, 0x6e, 0x64, + 0x65, 0x64, 0x54, 0x72, 0x69, 0x65, 0x4e, 0x6f, 0x64, 0x65, 0x2e, 0x43, 0x68, 0x69, 0x6c, 0x64, + 0x72, 0x65, 0x6e, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x52, 0x08, 0x63, 0x68, 0x69, 0x6c, 0x64, 0x72, + 0x65, 0x6e, 0x1a, 0x6f, 0x0a, 0x0d, 0x43, 0x68, 0x69, 0x6c, 0x64, 0x72, 0x65, 0x6e, 0x45, 0x6e, + 0x74, 0x72, 0x79, 0x12, 0x10, 0x0a, 0x03, 0x6b, 0x65, 0x79, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, + 0x52, 0x03, 0x6b, 0x65, 0x79, 0x12, 0x48, 0x0a, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x18, 0x02, + 0x20, 0x01, 0x28, 0x0b, 0x32, 0x32, 0x2e, 0x6f, 0x72, 0x67, 0x2e, 0x61, 0x70, 0x61, 0x63, 0x68, + 0x65, 0x2e, 0x62, 0x65, 0x61, 0x6d, 0x2e, 0x6d, 0x6f, 0x64, 0x65, 0x6c, 0x2e, 0x70, 0x69, 0x70, + 0x65, 0x6c, 0x69, 0x6e, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x42, 0x6f, 0x75, 0x6e, 0x64, 0x65, 0x64, + 0x54, 0x72, 0x69, 0x65, 0x4e, 0x6f, 0x64, 0x65, 0x52, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x3a, + 0x02, 0x38, 0x01, 0x22, 0x89, 0x01, 0x0a, 0x0b, 0x42, 0x6f, 0x75, 0x6e, 0x64, 0x65, 0x64, 0x54, + 0x72, 0x69, 0x65, 0x12, 0x14, 0x0a, 0x05, 0x62, 0x6f, 0x75, 0x6e, 0x64, 0x18, 0x01, 0x20, 0x01, + 0x28, 0x05, 0x52, 0x05, 0x62, 0x6f, 0x75, 0x6e, 0x64, 0x12, 0x46, 0x0a, 0x04, 0x72, 0x6f, 0x6f, + 0x74, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x32, 0x2e, 0x6f, 0x72, 0x67, 0x2e, 0x61, 0x70, + 0x61, 0x63, 0x68, 0x65, 0x2e, 0x62, 0x65, 0x61, 0x6d, 0x2e, 0x6d, 0x6f, 0x64, 0x65, 0x6c, 0x2e, + 0x70, 0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x42, 0x6f, 0x75, 0x6e, + 0x64, 0x65, 0x64, 0x54, 0x72, 0x69, 0x65, 0x4e, 0x6f, 0x64, 0x65, 0x52, 0x04, 0x72, 0x6f, 0x6f, + 0x74, 0x12, 0x1c, 0x0a, 0x09, 0x73, 0x69, 0x6e, 0x67, 0x6c, 0x65, 0x74, 0x6f, 0x6e, 0x18, 0x03, + 0x20, 0x03, 0x28, 0x09, 0x52, 0x09, 0x73, 0x69, 0x6e, 0x67, 0x6c, 0x65, 0x74, 0x6f, 0x6e, 0x3a, + 0x82, 0x01, 0x0a, 0x0b, 0x6c, 0x61, 0x62, 0x65, 0x6c, 0x5f, 0x70, 0x72, 0x6f, 0x70, 0x73, 0x12, + 0x21, 0x2e, 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75, + 0x66, 0x2e, 0x45, 0x6e, 0x75, 0x6d, 0x56, 0x61, 0x6c, 0x75, 0x65, 0x4f, 0x70, 0x74, 0x69, 0x6f, + 0x6e, 0x73, 0x18, 0xc4, 0x8a, 0xdc, 0x3c, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x3b, 0x2e, 0x6f, 0x72, + 0x67, 0x2e, 0x61, 0x70, 0x61, 0x63, 0x68, 0x65, 0x2e, 0x62, 0x65, 0x61, 0x6d, 0x2e, 0x6d, 0x6f, + 0x64, 0x65, 0x6c, 0x2e, 0x70, 0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e, 0x65, 0x2e, 0x76, 0x31, 0x2e, + 0x4d, 0x6f, 0x6e, 0x69, 0x74, 0x6f, 0x72, 0x69, 0x6e, 0x67, 0x49, 0x6e, 0x66, 0x6f, 0x4c, 0x61, + 0x62, 0x65, 0x6c, 0x50, 0x72, 0x6f, 0x70, 0x73, 0x52, 0x0a, 0x6c, 0x61, 0x62, 0x65, 0x6c, 0x50, + 0x72, 0x6f, 0x70, 0x73, 0x3a, 0x8d, 0x01, 0x0a, 0x14, 0x6d, 0x6f, 0x6e, 0x69, 0x74, 0x6f, 0x72, + 0x69, 0x6e, 0x67, 0x5f, 0x69, 0x6e, 0x66, 0x6f, 0x5f, 0x73, 0x70, 0x65, 0x63, 0x12, 0x21, 0x2e, + 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66, 0x2e, + 0x45, 0x6e, 0x75, 0x6d, 0x56, 0x61, 0x6c, 0x75, 0x65, 0x4f, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x73, + 0x18, 0xfa, 0xf4, 0xe4, 0x62, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x35, 0x2e, 0x6f, 0x72, 0x67, 0x2e, + 0x61, 0x70, 0x61, 0x63, 0x68, 0x65, 0x2e, 0x62, 0x65, 0x61, 0x6d, 0x2e, 0x6d, 0x6f, 0x64, 0x65, + 0x6c, 0x2e, 0x70, 0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x4d, 0x6f, + 0x6e, 0x69, 0x74, 0x6f, 0x72, 0x69, 0x6e, 0x67, 0x49, 0x6e, 0x66, 0x6f, 0x53, 0x70, 0x65, 0x63, + 0x52, 0x12, 0x6d, 0x6f, 0x6e, 0x69, 0x74, 0x6f, 0x72, 0x69, 0x6e, 0x67, 0x49, 0x6e, 0x66, 0x6f, + 0x53, 0x70, 0x65, 0x63, 0x42, 0x79, 0x0a, 0x21, 0x6f, 0x72, 0x67, 0x2e, 0x61, 0x70, 0x61, 0x63, + 0x68, 0x65, 0x2e, 0x62, 0x65, 0x61, 0x6d, 0x2e, 0x6d, 0x6f, 0x64, 0x65, 0x6c, 0x2e, 0x70, 0x69, + 0x70, 0x65, 0x6c, 0x69, 0x6e, 0x65, 0x2e, 0x76, 0x31, 0x42, 0x0a, 0x4d, 0x65, 0x74, 0x72, 0x69, + 0x63, 0x73, 0x41, 0x70, 0x69, 0x5a, 0x48, 0x67, 0x69, 0x74, 0x68, 0x75, 0x62, 0x2e, 0x63, 0x6f, + 0x6d, 0x2f, 0x61, 0x70, 0x61, 0x63, 0x68, 0x65, 0x2f, 0x62, 0x65, 0x61, 0x6d, 0x2f, 0x73, 0x64, + 0x6b, 0x73, 0x2f, 0x76, 0x32, 0x2f, 0x67, 0x6f, 0x2f, 0x70, 0x6b, 0x67, 0x2f, 0x62, 0x65, 0x61, + 0x6d, 0x2f, 0x6d, 0x6f, 0x64, 0x65, 0x6c, 0x2f, 0x70, 0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e, 0x65, + 0x5f, 0x76, 0x31, 0x3b, 0x70, 0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e, 0x65, 0x5f, 0x76, 0x31, 0x62, + 0x06, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x33, } var file_org_apache_beam_model_pipeline_v1_metrics_proto_enumTypes = make([]protoimpl.EnumInfo, 3) -var file_org_apache_beam_model_pipeline_v1_metrics_proto_msgTypes = make([]protoimpl.MessageInfo, 7) -var file_org_apache_beam_model_pipeline_v1_metrics_proto_goTypes = []interface{}{ +var file_org_apache_beam_model_pipeline_v1_metrics_proto_msgTypes = make([]protoimpl.MessageInfo, 10) +var file_org_apache_beam_model_pipeline_v1_metrics_proto_goTypes = []any{ (MonitoringInfoSpecs_Enum)(0), // 0: org.apache.beam.model.pipeline.v1.MonitoringInfoSpecs.Enum (MonitoringInfo_MonitoringInfoLabels)(0), // 1: org.apache.beam.model.pipeline.v1.MonitoringInfo.MonitoringInfoLabels (MonitoringInfoTypeUrns_Enum)(0), // 2: org.apache.beam.model.pipeline.v1.MonitoringInfoTypeUrns.Enum @@ -1387,23 +1746,29 @@ var file_org_apache_beam_model_pipeline_v1_metrics_proto_goTypes = []interface{} (*MonitoringInfoLabelProps)(nil), // 6: org.apache.beam.model.pipeline.v1.MonitoringInfoLabelProps (*MonitoringInfo)(nil), // 7: org.apache.beam.model.pipeline.v1.MonitoringInfo (*MonitoringInfoTypeUrns)(nil), // 8: org.apache.beam.model.pipeline.v1.MonitoringInfoTypeUrns - nil, // 9: org.apache.beam.model.pipeline.v1.MonitoringInfo.LabelsEntry - (*timestamppb.Timestamp)(nil), // 10: google.protobuf.Timestamp - (*descriptorpb.EnumValueOptions)(nil), // 11: google.protobuf.EnumValueOptions + (*BoundedTrieNode)(nil), // 9: org.apache.beam.model.pipeline.v1.BoundedTrieNode + (*BoundedTrie)(nil), // 10: org.apache.beam.model.pipeline.v1.BoundedTrie + nil, // 11: org.apache.beam.model.pipeline.v1.MonitoringInfo.LabelsEntry + nil, // 12: org.apache.beam.model.pipeline.v1.BoundedTrieNode.ChildrenEntry + (*timestamppb.Timestamp)(nil), // 13: google.protobuf.Timestamp + (*descriptorpb.EnumValueOptions)(nil), // 14: google.protobuf.EnumValueOptions } var file_org_apache_beam_model_pipeline_v1_metrics_proto_depIdxs = []int32{ 4, // 0: org.apache.beam.model.pipeline.v1.MonitoringInfoSpec.annotations:type_name -> org.apache.beam.model.pipeline.v1.Annotation - 9, // 1: org.apache.beam.model.pipeline.v1.MonitoringInfo.labels:type_name -> org.apache.beam.model.pipeline.v1.MonitoringInfo.LabelsEntry - 10, // 2: org.apache.beam.model.pipeline.v1.MonitoringInfo.start_time:type_name -> google.protobuf.Timestamp - 11, // 3: org.apache.beam.model.pipeline.v1.label_props:extendee -> google.protobuf.EnumValueOptions - 11, // 4: org.apache.beam.model.pipeline.v1.monitoring_info_spec:extendee -> google.protobuf.EnumValueOptions - 6, // 5: org.apache.beam.model.pipeline.v1.label_props:type_name -> org.apache.beam.model.pipeline.v1.MonitoringInfoLabelProps - 3, // 6: org.apache.beam.model.pipeline.v1.monitoring_info_spec:type_name -> org.apache.beam.model.pipeline.v1.MonitoringInfoSpec - 7, // [7:7] is the sub-list for method output_type - 7, // [7:7] is the sub-list for method input_type - 5, // [5:7] is the sub-list for extension type_name - 3, // [3:5] is the sub-list for extension extendee - 0, // [0:3] is the sub-list for field type_name + 11, // 1: org.apache.beam.model.pipeline.v1.MonitoringInfo.labels:type_name -> org.apache.beam.model.pipeline.v1.MonitoringInfo.LabelsEntry + 13, // 2: org.apache.beam.model.pipeline.v1.MonitoringInfo.start_time:type_name -> google.protobuf.Timestamp + 12, // 3: org.apache.beam.model.pipeline.v1.BoundedTrieNode.children:type_name -> org.apache.beam.model.pipeline.v1.BoundedTrieNode.ChildrenEntry + 9, // 4: org.apache.beam.model.pipeline.v1.BoundedTrie.root:type_name -> org.apache.beam.model.pipeline.v1.BoundedTrieNode + 9, // 5: org.apache.beam.model.pipeline.v1.BoundedTrieNode.ChildrenEntry.value:type_name -> org.apache.beam.model.pipeline.v1.BoundedTrieNode + 14, // 6: org.apache.beam.model.pipeline.v1.label_props:extendee -> google.protobuf.EnumValueOptions + 14, // 7: org.apache.beam.model.pipeline.v1.monitoring_info_spec:extendee -> google.protobuf.EnumValueOptions + 6, // 8: org.apache.beam.model.pipeline.v1.label_props:type_name -> org.apache.beam.model.pipeline.v1.MonitoringInfoLabelProps + 3, // 9: org.apache.beam.model.pipeline.v1.monitoring_info_spec:type_name -> org.apache.beam.model.pipeline.v1.MonitoringInfoSpec + 10, // [10:10] is the sub-list for method output_type + 10, // [10:10] is the sub-list for method input_type + 8, // [8:10] is the sub-list for extension type_name + 6, // [6:8] is the sub-list for extension extendee + 0, // [0:6] is the sub-list for field type_name } func init() { file_org_apache_beam_model_pipeline_v1_metrics_proto_init() } @@ -1412,87 +1777,13 @@ func file_org_apache_beam_model_pipeline_v1_metrics_proto_init() { return } file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_init() - if !protoimpl.UnsafeEnabled { - file_org_apache_beam_model_pipeline_v1_metrics_proto_msgTypes[0].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*MonitoringInfoSpec); i { - case 0: - return &v.state - case 1: - return &v.sizeCache - case 2: - return &v.unknownFields - default: - return nil - } - } - file_org_apache_beam_model_pipeline_v1_metrics_proto_msgTypes[1].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*Annotation); i { - case 0: - return &v.state - case 1: - return &v.sizeCache - case 2: - return &v.unknownFields - default: - return nil - } - } - file_org_apache_beam_model_pipeline_v1_metrics_proto_msgTypes[2].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*MonitoringInfoSpecs); i { - case 0: - return &v.state - case 1: - return &v.sizeCache - case 2: - return &v.unknownFields - default: - return nil - } - } - file_org_apache_beam_model_pipeline_v1_metrics_proto_msgTypes[3].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*MonitoringInfoLabelProps); i { - case 0: - return &v.state - case 1: - return &v.sizeCache - case 2: - return &v.unknownFields - default: - return nil - } - } - file_org_apache_beam_model_pipeline_v1_metrics_proto_msgTypes[4].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*MonitoringInfo); i { - case 0: - return &v.state - case 1: - return &v.sizeCache - case 2: - return &v.unknownFields - default: - return nil - } - } - file_org_apache_beam_model_pipeline_v1_metrics_proto_msgTypes[5].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*MonitoringInfoTypeUrns); i { - case 0: - return &v.state - case 1: - return &v.sizeCache - case 2: - return &v.unknownFields - default: - return nil - } - } - } type x struct{} out := protoimpl.TypeBuilder{ File: protoimpl.DescBuilder{ GoPackagePath: reflect.TypeOf(x{}).PkgPath(), RawDescriptor: file_org_apache_beam_model_pipeline_v1_metrics_proto_rawDesc, NumEnums: 3, - NumMessages: 7, + NumMessages: 10, NumExtensions: 2, NumServices: 0, }, diff --git a/sdks/go/pkg/beam/model/pipeline_v1/metrics_protoopaque.pb.go b/sdks/go/pkg/beam/model/pipeline_v1/metrics_protoopaque.pb.go new file mode 100644 index 000000000000..fdee49599b50 --- /dev/null +++ b/sdks/go/pkg/beam/model/pipeline_v1/metrics_protoopaque.pb.go @@ -0,0 +1,1754 @@ +// +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +// +// Protocol Buffers for metrics classes, used in the Fn API, Job API, and by SDKs. + +// Code generated by protoc-gen-go. DO NOT EDIT. +// versions: +// protoc-gen-go v1.36.0 +// protoc v5.27.3 +// source: org/apache/beam/model/pipeline/v1/metrics.proto + +//go:build protoopaque + +package pipeline_v1 + +import ( + protoreflect "google.golang.org/protobuf/reflect/protoreflect" + protoimpl "google.golang.org/protobuf/runtime/protoimpl" + descriptorpb "google.golang.org/protobuf/types/descriptorpb" + timestamppb "google.golang.org/protobuf/types/known/timestamppb" + reflect "reflect" +) + +const ( + // Verify that this generated code is sufficiently up-to-date. + _ = protoimpl.EnforceVersion(20 - protoimpl.MinVersion) + // Verify that runtime/protoimpl is sufficiently up-to-date. + _ = protoimpl.EnforceVersion(protoimpl.MaxVersion - 20) +) + +type MonitoringInfoSpecs_Enum int32 + +const ( + // Represents an integer counter where values are summed across bundles. + MonitoringInfoSpecs_USER_SUM_INT64 MonitoringInfoSpecs_Enum = 0 + // Represents a double counter where values are summed across bundles. + MonitoringInfoSpecs_USER_SUM_DOUBLE MonitoringInfoSpecs_Enum = 1 + // Represents a distribution of an integer value where: + // - count: represents the number of values seen across all bundles + // - sum: represents the total of the value across all bundles + // - min: represents the smallest value seen across all bundles + // - max: represents the largest value seen across all bundles + MonitoringInfoSpecs_USER_DISTRIBUTION_INT64 MonitoringInfoSpecs_Enum = 2 + // Represents a distribution of a double value where: + // - count: represents the number of values seen across all bundles + // - sum: represents the total of the value across all bundles + // - min: represents the smallest value seen across all bundles + // - max: represents the largest value seen across all bundles + MonitoringInfoSpecs_USER_DISTRIBUTION_DOUBLE MonitoringInfoSpecs_Enum = 3 + // Represents the latest seen integer value. The timestamp is used to + // provide an "ordering" over multiple values to determine which is the + // latest. + MonitoringInfoSpecs_USER_LATEST_INT64 MonitoringInfoSpecs_Enum = 4 + // Represents the latest seen double value. The timestamp is used to + // provide an "ordering" over multiple values to determine which is the + // latest. + MonitoringInfoSpecs_USER_LATEST_DOUBLE MonitoringInfoSpecs_Enum = 5 + // Represents the largest set of integer values seen across bundles. + MonitoringInfoSpecs_USER_TOP_N_INT64 MonitoringInfoSpecs_Enum = 6 + // Represents the largest set of double values seen across bundles. + MonitoringInfoSpecs_USER_TOP_N_DOUBLE MonitoringInfoSpecs_Enum = 7 + // Represents the smallest set of integer values seen across bundles. + MonitoringInfoSpecs_USER_BOTTOM_N_INT64 MonitoringInfoSpecs_Enum = 8 + // Represents the smallest set of double values seen across bundles. + MonitoringInfoSpecs_USER_BOTTOM_N_DOUBLE MonitoringInfoSpecs_Enum = 9 + // Represents a set of strings seen across bundles. + MonitoringInfoSpecs_USER_SET_STRING MonitoringInfoSpecs_Enum = 21 + // Represents a set of strings seen across bundles. + MonitoringInfoSpecs_USER_BOUNDED_TRIE MonitoringInfoSpecs_Enum = 22 + MonitoringInfoSpecs_ELEMENT_COUNT MonitoringInfoSpecs_Enum = 10 + MonitoringInfoSpecs_SAMPLED_BYTE_SIZE MonitoringInfoSpecs_Enum = 11 + MonitoringInfoSpecs_START_BUNDLE_MSECS MonitoringInfoSpecs_Enum = 12 + MonitoringInfoSpecs_PROCESS_BUNDLE_MSECS MonitoringInfoSpecs_Enum = 13 + MonitoringInfoSpecs_FINISH_BUNDLE_MSECS MonitoringInfoSpecs_Enum = 14 + MonitoringInfoSpecs_TOTAL_MSECS MonitoringInfoSpecs_Enum = 15 + // All values reported across all beam:metric:ptransform_progress:.*:v1 + // metrics are of the same magnitude. + MonitoringInfoSpecs_WORK_REMAINING MonitoringInfoSpecs_Enum = 16 + // All values reported across all beam:metric:ptransform_progress:.*:v1 + // metrics are of the same magnitude. + MonitoringInfoSpecs_WORK_COMPLETED MonitoringInfoSpecs_Enum = 17 + // The (0-based) index of the latest item processed from the data channel. + // This gives an indication of the SDKs progress through the data channel, + // and is a lower bound on where it is able to split. + // For an SDK that processes items sequentially, this is equivalently the + // number of items fully processed (or -1 if processing has not yet started). + MonitoringInfoSpecs_DATA_CHANNEL_READ_INDEX MonitoringInfoSpecs_Enum = 18 + MonitoringInfoSpecs_API_REQUEST_COUNT MonitoringInfoSpecs_Enum = 19 + MonitoringInfoSpecs_API_REQUEST_LATENCIES MonitoringInfoSpecs_Enum = 20 +) + +// Enum value maps for MonitoringInfoSpecs_Enum. +var ( + MonitoringInfoSpecs_Enum_name = map[int32]string{ + 0: "USER_SUM_INT64", + 1: "USER_SUM_DOUBLE", + 2: "USER_DISTRIBUTION_INT64", + 3: "USER_DISTRIBUTION_DOUBLE", + 4: "USER_LATEST_INT64", + 5: "USER_LATEST_DOUBLE", + 6: "USER_TOP_N_INT64", + 7: "USER_TOP_N_DOUBLE", + 8: "USER_BOTTOM_N_INT64", + 9: "USER_BOTTOM_N_DOUBLE", + 21: "USER_SET_STRING", + 22: "USER_BOUNDED_TRIE", + 10: "ELEMENT_COUNT", + 11: "SAMPLED_BYTE_SIZE", + 12: "START_BUNDLE_MSECS", + 13: "PROCESS_BUNDLE_MSECS", + 14: "FINISH_BUNDLE_MSECS", + 15: "TOTAL_MSECS", + 16: "WORK_REMAINING", + 17: "WORK_COMPLETED", + 18: "DATA_CHANNEL_READ_INDEX", + 19: "API_REQUEST_COUNT", + 20: "API_REQUEST_LATENCIES", + } + MonitoringInfoSpecs_Enum_value = map[string]int32{ + "USER_SUM_INT64": 0, + "USER_SUM_DOUBLE": 1, + "USER_DISTRIBUTION_INT64": 2, + "USER_DISTRIBUTION_DOUBLE": 3, + "USER_LATEST_INT64": 4, + "USER_LATEST_DOUBLE": 5, + "USER_TOP_N_INT64": 6, + "USER_TOP_N_DOUBLE": 7, + "USER_BOTTOM_N_INT64": 8, + "USER_BOTTOM_N_DOUBLE": 9, + "USER_SET_STRING": 21, + "USER_BOUNDED_TRIE": 22, + "ELEMENT_COUNT": 10, + "SAMPLED_BYTE_SIZE": 11, + "START_BUNDLE_MSECS": 12, + "PROCESS_BUNDLE_MSECS": 13, + "FINISH_BUNDLE_MSECS": 14, + "TOTAL_MSECS": 15, + "WORK_REMAINING": 16, + "WORK_COMPLETED": 17, + "DATA_CHANNEL_READ_INDEX": 18, + "API_REQUEST_COUNT": 19, + "API_REQUEST_LATENCIES": 20, + } +) + +func (x MonitoringInfoSpecs_Enum) Enum() *MonitoringInfoSpecs_Enum { + p := new(MonitoringInfoSpecs_Enum) + *p = x + return p +} + +func (x MonitoringInfoSpecs_Enum) String() string { + return protoimpl.X.EnumStringOf(x.Descriptor(), protoreflect.EnumNumber(x)) +} + +func (MonitoringInfoSpecs_Enum) Descriptor() protoreflect.EnumDescriptor { + return file_org_apache_beam_model_pipeline_v1_metrics_proto_enumTypes[0].Descriptor() +} + +func (MonitoringInfoSpecs_Enum) Type() protoreflect.EnumType { + return &file_org_apache_beam_model_pipeline_v1_metrics_proto_enumTypes[0] +} + +func (x MonitoringInfoSpecs_Enum) Number() protoreflect.EnumNumber { + return protoreflect.EnumNumber(x) +} + +type MonitoringInfo_MonitoringInfoLabels int32 + +const ( + // The values used for TRANSFORM, PCOLLECTION, WINDOWING_STRATEGY + // CODER, ENVIRONMENT, etc. must always match the keys used to + // refer to them. For actively processed bundles, these should match the + // values within the ProcessBundleDescriptor. For job management APIs, + // these should match values within the original pipeline representation. + MonitoringInfo_TRANSFORM MonitoringInfo_MonitoringInfoLabels = 0 + MonitoringInfo_PCOLLECTION MonitoringInfo_MonitoringInfoLabels = 1 + MonitoringInfo_WINDOWING_STRATEGY MonitoringInfo_MonitoringInfoLabels = 2 + MonitoringInfo_CODER MonitoringInfo_MonitoringInfoLabels = 3 + MonitoringInfo_ENVIRONMENT MonitoringInfo_MonitoringInfoLabels = 4 + MonitoringInfo_NAMESPACE MonitoringInfo_MonitoringInfoLabels = 5 + MonitoringInfo_NAME MonitoringInfo_MonitoringInfoLabels = 6 + MonitoringInfo_SERVICE MonitoringInfo_MonitoringInfoLabels = 7 + MonitoringInfo_METHOD MonitoringInfo_MonitoringInfoLabels = 8 + MonitoringInfo_RESOURCE MonitoringInfo_MonitoringInfoLabels = 9 + MonitoringInfo_STATUS MonitoringInfo_MonitoringInfoLabels = 10 + MonitoringInfo_BIGQUERY_PROJECT_ID MonitoringInfo_MonitoringInfoLabels = 11 + MonitoringInfo_BIGQUERY_DATASET MonitoringInfo_MonitoringInfoLabels = 12 + MonitoringInfo_BIGQUERY_TABLE MonitoringInfo_MonitoringInfoLabels = 13 + MonitoringInfo_BIGQUERY_VIEW MonitoringInfo_MonitoringInfoLabels = 14 + MonitoringInfo_BIGQUERY_QUERY_NAME MonitoringInfo_MonitoringInfoLabels = 15 + MonitoringInfo_GCS_BUCKET MonitoringInfo_MonitoringInfoLabels = 16 + MonitoringInfo_GCS_PROJECT_ID MonitoringInfo_MonitoringInfoLabels = 17 + MonitoringInfo_DATASTORE_PROJECT MonitoringInfo_MonitoringInfoLabels = 18 + MonitoringInfo_DATASTORE_NAMESPACE MonitoringInfo_MonitoringInfoLabels = 19 + MonitoringInfo_BIGTABLE_PROJECT_ID MonitoringInfo_MonitoringInfoLabels = 20 + MonitoringInfo_INSTANCE_ID MonitoringInfo_MonitoringInfoLabels = 21 + MonitoringInfo_TABLE_ID MonitoringInfo_MonitoringInfoLabels = 22 + MonitoringInfo_SPANNER_PROJECT_ID MonitoringInfo_MonitoringInfoLabels = 23 + MonitoringInfo_SPANNER_DATABASE_ID MonitoringInfo_MonitoringInfoLabels = 24 + MonitoringInfo_SPANNER_TABLE_ID MonitoringInfo_MonitoringInfoLabels = 25 + MonitoringInfo_SPANNER_INSTANCE_ID MonitoringInfo_MonitoringInfoLabels = 26 + MonitoringInfo_SPANNER_QUERY_NAME MonitoringInfo_MonitoringInfoLabels = 27 +) + +// Enum value maps for MonitoringInfo_MonitoringInfoLabels. +var ( + MonitoringInfo_MonitoringInfoLabels_name = map[int32]string{ + 0: "TRANSFORM", + 1: "PCOLLECTION", + 2: "WINDOWING_STRATEGY", + 3: "CODER", + 4: "ENVIRONMENT", + 5: "NAMESPACE", + 6: "NAME", + 7: "SERVICE", + 8: "METHOD", + 9: "RESOURCE", + 10: "STATUS", + 11: "BIGQUERY_PROJECT_ID", + 12: "BIGQUERY_DATASET", + 13: "BIGQUERY_TABLE", + 14: "BIGQUERY_VIEW", + 15: "BIGQUERY_QUERY_NAME", + 16: "GCS_BUCKET", + 17: "GCS_PROJECT_ID", + 18: "DATASTORE_PROJECT", + 19: "DATASTORE_NAMESPACE", + 20: "BIGTABLE_PROJECT_ID", + 21: "INSTANCE_ID", + 22: "TABLE_ID", + 23: "SPANNER_PROJECT_ID", + 24: "SPANNER_DATABASE_ID", + 25: "SPANNER_TABLE_ID", + 26: "SPANNER_INSTANCE_ID", + 27: "SPANNER_QUERY_NAME", + } + MonitoringInfo_MonitoringInfoLabels_value = map[string]int32{ + "TRANSFORM": 0, + "PCOLLECTION": 1, + "WINDOWING_STRATEGY": 2, + "CODER": 3, + "ENVIRONMENT": 4, + "NAMESPACE": 5, + "NAME": 6, + "SERVICE": 7, + "METHOD": 8, + "RESOURCE": 9, + "STATUS": 10, + "BIGQUERY_PROJECT_ID": 11, + "BIGQUERY_DATASET": 12, + "BIGQUERY_TABLE": 13, + "BIGQUERY_VIEW": 14, + "BIGQUERY_QUERY_NAME": 15, + "GCS_BUCKET": 16, + "GCS_PROJECT_ID": 17, + "DATASTORE_PROJECT": 18, + "DATASTORE_NAMESPACE": 19, + "BIGTABLE_PROJECT_ID": 20, + "INSTANCE_ID": 21, + "TABLE_ID": 22, + "SPANNER_PROJECT_ID": 23, + "SPANNER_DATABASE_ID": 24, + "SPANNER_TABLE_ID": 25, + "SPANNER_INSTANCE_ID": 26, + "SPANNER_QUERY_NAME": 27, + } +) + +func (x MonitoringInfo_MonitoringInfoLabels) Enum() *MonitoringInfo_MonitoringInfoLabels { + p := new(MonitoringInfo_MonitoringInfoLabels) + *p = x + return p +} + +func (x MonitoringInfo_MonitoringInfoLabels) String() string { + return protoimpl.X.EnumStringOf(x.Descriptor(), protoreflect.EnumNumber(x)) +} + +func (MonitoringInfo_MonitoringInfoLabels) Descriptor() protoreflect.EnumDescriptor { + return file_org_apache_beam_model_pipeline_v1_metrics_proto_enumTypes[1].Descriptor() +} + +func (MonitoringInfo_MonitoringInfoLabels) Type() protoreflect.EnumType { + return &file_org_apache_beam_model_pipeline_v1_metrics_proto_enumTypes[1] +} + +func (x MonitoringInfo_MonitoringInfoLabels) Number() protoreflect.EnumNumber { + return protoreflect.EnumNumber(x) +} + +type MonitoringInfoTypeUrns_Enum int32 + +const ( + // Represents an integer counter where values are summed across bundles. + // + // Encoding: + // - value: beam:coder:varint:v1 + MonitoringInfoTypeUrns_SUM_INT64_TYPE MonitoringInfoTypeUrns_Enum = 0 + // Represents a double counter where values are summed across bundles. + // + // Encoding: + // + // value: beam:coder:double:v1 + MonitoringInfoTypeUrns_SUM_DOUBLE_TYPE MonitoringInfoTypeUrns_Enum = 1 + // Represents a distribution of an integer value where: + // - count: represents the number of values seen across all bundles + // - sum: represents the total of the value across all bundles + // - min: represents the smallest value seen across all bundles + // - max: represents the largest value seen across all bundles + // + // Encoding: + // - count: beam:coder:varint:v1 + // - sum: beam:coder:varint:v1 + // - min: beam:coder:varint:v1 + // - max: beam:coder:varint:v1 + MonitoringInfoTypeUrns_DISTRIBUTION_INT64_TYPE MonitoringInfoTypeUrns_Enum = 2 + // Represents a distribution of a double value where: + // - count: represents the number of values seen across all bundles + // - sum: represents the total of the value across all bundles + // - min: represents the smallest value seen across all bundles + // - max: represents the largest value seen across all bundles + // + // Encoding: + // - count: beam:coder:varint:v1 + // - sum: beam:coder:double:v1 + // - min: beam:coder:double:v1 + // - max: beam:coder:double:v1 + MonitoringInfoTypeUrns_DISTRIBUTION_DOUBLE_TYPE MonitoringInfoTypeUrns_Enum = 3 + // Represents the latest seen integer value. The timestamp is used to + // provide an "ordering" over multiple values to determine which is the + // latest. + // + // Encoding: + // - timestamp: beam:coder:varint:v1 (milliseconds since epoch) + // - value: beam:coder:varint:v1 + MonitoringInfoTypeUrns_LATEST_INT64_TYPE MonitoringInfoTypeUrns_Enum = 4 + // Represents the latest seen double value. The timestamp is used to + // provide an "ordering" over multiple values to determine which is the + // latest. + // + // Encoding: + // - timestamp: beam:coder:varint:v1 (milliseconds since epoch) + // - value: beam:coder:double:v1 + MonitoringInfoTypeUrns_LATEST_DOUBLE_TYPE MonitoringInfoTypeUrns_Enum = 5 + // Represents the largest set of integer values seen across bundles. + // + // Encoding: ... + // - iter: beam:coder:iterable:v1 + // - valueX: beam:coder:varint:v1 + MonitoringInfoTypeUrns_TOP_N_INT64_TYPE MonitoringInfoTypeUrns_Enum = 6 + // Represents the largest set of double values seen across bundles. + // + // Encoding: ... + // - iter: beam:coder:iterable:v1 + // - valueX: beam:coder... + // - iter: beam:coder:iterable:v1 + // - valueX: beam:coder:varint:v1 + MonitoringInfoTypeUrns_BOTTOM_N_INT64_TYPE MonitoringInfoTypeUrns_Enum = 8 + // Represents the smallest set of double values seen across bundles. + // + // Encoding: ... + // - iter: beam:coder:iterable:v1 + // - valueX: beam:coder:double:v1 + MonitoringInfoTypeUrns_BOTTOM_N_DOUBLE_TYPE MonitoringInfoTypeUrns_Enum = 9 + // Encoding: ... + // - iter: beam:coder:iterable:v1 + // - valueX: beam:coder:double:v1 + MonitoringInfoTypeUrns_PROGRESS_TYPE MonitoringInfoTypeUrns_Enum = 10 + // Represents a set of strings. + // + // Encoding: ... + // - iter: beam:coder:iterable:v1 + // - valueX: beam:coder:stringutf8:v1 + MonitoringInfoTypeUrns_SET_STRING_TYPE MonitoringInfoTypeUrns_Enum = 11 + // Represents a bounded trie of strings. + // + // Encoding: BoundedTrie proto + MonitoringInfoTypeUrns_BOUNDED_TRIE_TYPE MonitoringInfoTypeUrns_Enum = 12 +) + +// Enum value maps for MonitoringInfoTypeUrns_Enum. +var ( + MonitoringInfoTypeUrns_Enum_name = map[int32]string{ + 0: "SUM_INT64_TYPE", + 1: "SUM_DOUBLE_TYPE", + 2: "DISTRIBUTION_INT64_TYPE", + 3: "DISTRIBUTION_DOUBLE_TYPE", + 4: "LATEST_INT64_TYPE", + 5: "LATEST_DOUBLE_TYPE", + 6: "TOP_N_INT64_TYPE", + 7: "TOP_N_DOUBLE_TYPE", + 8: "BOTTOM_N_INT64_TYPE", + 9: "BOTTOM_N_DOUBLE_TYPE", + 10: "PROGRESS_TYPE", + 11: "SET_STRING_TYPE", + 12: "BOUNDED_TRIE_TYPE", + } + MonitoringInfoTypeUrns_Enum_value = map[string]int32{ + "SUM_INT64_TYPE": 0, + "SUM_DOUBLE_TYPE": 1, + "DISTRIBUTION_INT64_TYPE": 2, + "DISTRIBUTION_DOUBLE_TYPE": 3, + "LATEST_INT64_TYPE": 4, + "LATEST_DOUBLE_TYPE": 5, + "TOP_N_INT64_TYPE": 6, + "TOP_N_DOUBLE_TYPE": 7, + "BOTTOM_N_INT64_TYPE": 8, + "BOTTOM_N_DOUBLE_TYPE": 9, + "PROGRESS_TYPE": 10, + "SET_STRING_TYPE": 11, + "BOUNDED_TRIE_TYPE": 12, + } +) + +func (x MonitoringInfoTypeUrns_Enum) Enum() *MonitoringInfoTypeUrns_Enum { + p := new(MonitoringInfoTypeUrns_Enum) + *p = x + return p +} + +func (x MonitoringInfoTypeUrns_Enum) String() string { + return protoimpl.X.EnumStringOf(x.Descriptor(), protoreflect.EnumNumber(x)) +} + +func (MonitoringInfoTypeUrns_Enum) Descriptor() protoreflect.EnumDescriptor { + return file_org_apache_beam_model_pipeline_v1_metrics_proto_enumTypes[2].Descriptor() +} + +func (MonitoringInfoTypeUrns_Enum) Type() protoreflect.EnumType { + return &file_org_apache_beam_model_pipeline_v1_metrics_proto_enumTypes[2] +} + +func (x MonitoringInfoTypeUrns_Enum) Number() protoreflect.EnumNumber { + return protoreflect.EnumNumber(x) +} + +// A specification for describing a well known MonitoringInfo. +// +// All specifications are uniquely identified by the urn. +type MonitoringInfoSpec struct { + state protoimpl.MessageState `protogen:"opaque.v1"` + xxx_hidden_Urn string `protobuf:"bytes,1,opt,name=urn,proto3" json:"urn,omitempty"` + xxx_hidden_Type string `protobuf:"bytes,2,opt,name=type,proto3" json:"type,omitempty"` + xxx_hidden_RequiredLabels []string `protobuf:"bytes,3,rep,name=required_labels,json=requiredLabels,proto3" json:"required_labels,omitempty"` + xxx_hidden_Annotations *[]*Annotation `protobuf:"bytes,4,rep,name=annotations,proto3" json:"annotations,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *MonitoringInfoSpec) Reset() { + *x = MonitoringInfoSpec{} + mi := &file_org_apache_beam_model_pipeline_v1_metrics_proto_msgTypes[0] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *MonitoringInfoSpec) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*MonitoringInfoSpec) ProtoMessage() {} + +func (x *MonitoringInfoSpec) ProtoReflect() protoreflect.Message { + mi := &file_org_apache_beam_model_pipeline_v1_metrics_proto_msgTypes[0] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +func (x *MonitoringInfoSpec) GetUrn() string { + if x != nil { + return x.xxx_hidden_Urn + } + return "" +} + +func (x *MonitoringInfoSpec) GetType() string { + if x != nil { + return x.xxx_hidden_Type + } + return "" +} + +func (x *MonitoringInfoSpec) GetRequiredLabels() []string { + if x != nil { + return x.xxx_hidden_RequiredLabels + } + return nil +} + +func (x *MonitoringInfoSpec) GetAnnotations() []*Annotation { + if x != nil { + if x.xxx_hidden_Annotations != nil { + return *x.xxx_hidden_Annotations + } + } + return nil +} + +func (x *MonitoringInfoSpec) SetUrn(v string) { + x.xxx_hidden_Urn = v +} + +func (x *MonitoringInfoSpec) SetType(v string) { + x.xxx_hidden_Type = v +} + +func (x *MonitoringInfoSpec) SetRequiredLabels(v []string) { + x.xxx_hidden_RequiredLabels = v +} + +func (x *MonitoringInfoSpec) SetAnnotations(v []*Annotation) { + x.xxx_hidden_Annotations = &v +} + +type MonitoringInfoSpec_builder struct { + _ [0]func() // Prevents comparability and use of unkeyed literals for the builder. + + // Defines the semantic meaning of the metric or monitored state. + // + // See MonitoringInfoSpecs.Enum for the set of well known metrics/monitored + // state. + Urn string + // Defines the required encoding and aggregation method for the payload. + // + // See MonitoringInfoTypeUrns.Enum for the set of well known types. + Type string + // The list of required labels for the specified urn and type. + RequiredLabels []string + // Extra non functional parts of the spec for descriptive purposes. + // i.e. description, units, etc. + Annotations []*Annotation +} + +func (b0 MonitoringInfoSpec_builder) Build() *MonitoringInfoSpec { + m0 := &MonitoringInfoSpec{} + b, x := &b0, m0 + _, _ = b, x + x.xxx_hidden_Urn = b.Urn + x.xxx_hidden_Type = b.Type + x.xxx_hidden_RequiredLabels = b.RequiredLabels + x.xxx_hidden_Annotations = &b.Annotations + return m0 +} + +// The key name and value string of MonitoringInfo annotations. +type Annotation struct { + state protoimpl.MessageState `protogen:"opaque.v1"` + xxx_hidden_Key string `protobuf:"bytes,1,opt,name=key,proto3" json:"key,omitempty"` + xxx_hidden_Value string `protobuf:"bytes,2,opt,name=value,proto3" json:"value,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *Annotation) Reset() { + *x = Annotation{} + mi := &file_org_apache_beam_model_pipeline_v1_metrics_proto_msgTypes[1] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *Annotation) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*Annotation) ProtoMessage() {} + +func (x *Annotation) ProtoReflect() protoreflect.Message { + mi := &file_org_apache_beam_model_pipeline_v1_metrics_proto_msgTypes[1] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +func (x *Annotation) GetKey() string { + if x != nil { + return x.xxx_hidden_Key + } + return "" +} + +func (x *Annotation) GetValue() string { + if x != nil { + return x.xxx_hidden_Value + } + return "" +} + +func (x *Annotation) SetKey(v string) { + x.xxx_hidden_Key = v +} + +func (x *Annotation) SetValue(v string) { + x.xxx_hidden_Value = v +} + +type Annotation_builder struct { + _ [0]func() // Prevents comparability and use of unkeyed literals for the builder. + + Key string + Value string +} + +func (b0 Annotation_builder) Build() *Annotation { + m0 := &Annotation{} + b, x := &b0, m0 + _, _ = b, x + x.xxx_hidden_Key = b.Key + x.xxx_hidden_Value = b.Value + return m0 +} + +// A set of well known MonitoringInfo specifications. +type MonitoringInfoSpecs struct { + state protoimpl.MessageState `protogen:"opaque.v1"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *MonitoringInfoSpecs) Reset() { + *x = MonitoringInfoSpecs{} + mi := &file_org_apache_beam_model_pipeline_v1_metrics_proto_msgTypes[2] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *MonitoringInfoSpecs) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*MonitoringInfoSpecs) ProtoMessage() {} + +func (x *MonitoringInfoSpecs) ProtoReflect() protoreflect.Message { + mi := &file_org_apache_beam_model_pipeline_v1_metrics_proto_msgTypes[2] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +type MonitoringInfoSpecs_builder struct { + _ [0]func() // Prevents comparability and use of unkeyed literals for the builder. + +} + +func (b0 MonitoringInfoSpecs_builder) Build() *MonitoringInfoSpecs { + m0 := &MonitoringInfoSpecs{} + b, x := &b0, m0 + _, _ = b, x + return m0 +} + +// A set of properties for the MonitoringInfoLabel, this is useful to obtain +// the proper label string for the MonitoringInfoLabel. +type MonitoringInfoLabelProps struct { + state protoimpl.MessageState `protogen:"opaque.v1"` + xxx_hidden_Name string `protobuf:"bytes,1,opt,name=name,proto3" json:"name,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *MonitoringInfoLabelProps) Reset() { + *x = MonitoringInfoLabelProps{} + mi := &file_org_apache_beam_model_pipeline_v1_metrics_proto_msgTypes[3] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *MonitoringInfoLabelProps) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*MonitoringInfoLabelProps) ProtoMessage() {} + +func (x *MonitoringInfoLabelProps) ProtoReflect() protoreflect.Message { + mi := &file_org_apache_beam_model_pipeline_v1_metrics_proto_msgTypes[3] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +func (x *MonitoringInfoLabelProps) GetName() string { + if x != nil { + return x.xxx_hidden_Name + } + return "" +} + +func (x *MonitoringInfoLabelProps) SetName(v string) { + x.xxx_hidden_Name = v +} + +type MonitoringInfoLabelProps_builder struct { + _ [0]func() // Prevents comparability and use of unkeyed literals for the builder. + + // The label key to use in the MonitoringInfo labels map. + Name string +} + +func (b0 MonitoringInfoLabelProps_builder) Build() *MonitoringInfoLabelProps { + m0 := &MonitoringInfoLabelProps{} + b, x := &b0, m0 + _, _ = b, x + x.xxx_hidden_Name = b.Name + return m0 +} + +type MonitoringInfo struct { + state protoimpl.MessageState `protogen:"opaque.v1"` + xxx_hidden_Urn string `protobuf:"bytes,1,opt,name=urn,proto3" json:"urn,omitempty"` + xxx_hidden_Type string `protobuf:"bytes,2,opt,name=type,proto3" json:"type,omitempty"` + xxx_hidden_Payload []byte `protobuf:"bytes,3,opt,name=payload,proto3" json:"payload,omitempty"` + xxx_hidden_Labels map[string]string `protobuf:"bytes,4,rep,name=labels,proto3" json:"labels,omitempty" protobuf_key:"bytes,1,opt,name=key" protobuf_val:"bytes,2,opt,name=value"` + xxx_hidden_StartTime *timestamppb.Timestamp `protobuf:"bytes,5,opt,name=start_time,json=startTime,proto3" json:"start_time,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *MonitoringInfo) Reset() { + *x = MonitoringInfo{} + mi := &file_org_apache_beam_model_pipeline_v1_metrics_proto_msgTypes[4] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *MonitoringInfo) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*MonitoringInfo) ProtoMessage() {} + +func (x *MonitoringInfo) ProtoReflect() protoreflect.Message { + mi := &file_org_apache_beam_model_pipeline_v1_metrics_proto_msgTypes[4] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +func (x *MonitoringInfo) GetUrn() string { + if x != nil { + return x.xxx_hidden_Urn + } + return "" +} + +func (x *MonitoringInfo) GetType() string { + if x != nil { + return x.xxx_hidden_Type + } + return "" +} + +func (x *MonitoringInfo) GetPayload() []byte { + if x != nil { + return x.xxx_hidden_Payload + } + return nil +} + +func (x *MonitoringInfo) GetLabels() map[string]string { + if x != nil { + return x.xxx_hidden_Labels + } + return nil +} + +func (x *MonitoringInfo) GetStartTime() *timestamppb.Timestamp { + if x != nil { + return x.xxx_hidden_StartTime + } + return nil +} + +func (x *MonitoringInfo) SetUrn(v string) { + x.xxx_hidden_Urn = v +} + +func (x *MonitoringInfo) SetType(v string) { + x.xxx_hidden_Type = v +} + +func (x *MonitoringInfo) SetPayload(v []byte) { + if v == nil { + v = []byte{} + } + x.xxx_hidden_Payload = v +} + +func (x *MonitoringInfo) SetLabels(v map[string]string) { + x.xxx_hidden_Labels = v +} + +func (x *MonitoringInfo) SetStartTime(v *timestamppb.Timestamp) { + x.xxx_hidden_StartTime = v +} + +func (x *MonitoringInfo) HasStartTime() bool { + if x == nil { + return false + } + return x.xxx_hidden_StartTime != nil +} + +func (x *MonitoringInfo) ClearStartTime() { + x.xxx_hidden_StartTime = nil +} + +type MonitoringInfo_builder struct { + _ [0]func() // Prevents comparability and use of unkeyed literals for the builder. + + // (Required) Defines the semantic meaning of the metric or monitored state. + // + // See MonitoringInfoSpecs.Enum for the set of well known metrics/monitored + // state. + Urn string + // (Required) Defines the encoding and aggregation method for the payload. + // + // See MonitoringInfoTypeUrns.Enum for the set of well known types. + Type string + // (Required) The metric or monitored state encoded as per the specification + // defined by the type. + Payload []byte + // A set of key and value labels which define the scope of the metric. For + // well known URNs, the set of required labels is provided by the associated + // MonitoringInfoSpec. + // + // Either a well defined entity id for matching the enum names in + // the MonitoringInfoLabels enum or any arbitrary label + // set by a custom metric or user metric. + // + // A monitoring system is expected to be able to aggregate the metrics + // together for all updates having the same URN and labels. Some systems such + // as Stackdriver will be able to aggregate the metrics using a subset of the + // provided labels + Labels map[string]string + // This indicates the start of the time range over which this value was + // measured. + // This is needed by some external metric aggregation services + // to indicate when the reporter of the metric first began collecting the + // cumulative value for the timeseries. + // If the SDK Harness restarts, it should reset the start_time, and reset + // the collection of cumulative metrics (i.e. start to count again from 0). + // HarnessMonitoringInfos should set this start_time once, when the + // MonitoringInfo is first reported. + // ProcessBundle MonitoringInfos should set a start_time for each bundle. + StartTime *timestamppb.Timestamp +} + +func (b0 MonitoringInfo_builder) Build() *MonitoringInfo { + m0 := &MonitoringInfo{} + b, x := &b0, m0 + _, _ = b, x + x.xxx_hidden_Urn = b.Urn + x.xxx_hidden_Type = b.Type + x.xxx_hidden_Payload = b.Payload + x.xxx_hidden_Labels = b.Labels + x.xxx_hidden_StartTime = b.StartTime + return m0 +} + +// A set of well known URNs that specify the encoding and aggregation method. +type MonitoringInfoTypeUrns struct { + state protoimpl.MessageState `protogen:"opaque.v1"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *MonitoringInfoTypeUrns) Reset() { + *x = MonitoringInfoTypeUrns{} + mi := &file_org_apache_beam_model_pipeline_v1_metrics_proto_msgTypes[5] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *MonitoringInfoTypeUrns) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*MonitoringInfoTypeUrns) ProtoMessage() {} + +func (x *MonitoringInfoTypeUrns) ProtoReflect() protoreflect.Message { + mi := &file_org_apache_beam_model_pipeline_v1_metrics_proto_msgTypes[5] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +type MonitoringInfoTypeUrns_builder struct { + _ [0]func() // Prevents comparability and use of unkeyed literals for the builder. + +} + +func (b0 MonitoringInfoTypeUrns_builder) Build() *MonitoringInfoTypeUrns { + m0 := &MonitoringInfoTypeUrns{} + b, x := &b0, m0 + _, _ = b, x + return m0 +} + +// A single node in a BoundedTrie. +type BoundedTrieNode struct { + state protoimpl.MessageState `protogen:"opaque.v1"` + xxx_hidden_Truncated bool `protobuf:"varint,1,opt,name=truncated,proto3" json:"truncated,omitempty"` + xxx_hidden_Children map[string]*BoundedTrieNode `protobuf:"bytes,2,rep,name=children,proto3" json:"children,omitempty" protobuf_key:"bytes,1,opt,name=key" protobuf_val:"bytes,2,opt,name=value"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *BoundedTrieNode) Reset() { + *x = BoundedTrieNode{} + mi := &file_org_apache_beam_model_pipeline_v1_metrics_proto_msgTypes[6] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *BoundedTrieNode) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*BoundedTrieNode) ProtoMessage() {} + +func (x *BoundedTrieNode) ProtoReflect() protoreflect.Message { + mi := &file_org_apache_beam_model_pipeline_v1_metrics_proto_msgTypes[6] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +func (x *BoundedTrieNode) GetTruncated() bool { + if x != nil { + return x.xxx_hidden_Truncated + } + return false +} + +func (x *BoundedTrieNode) GetChildren() map[string]*BoundedTrieNode { + if x != nil { + return x.xxx_hidden_Children + } + return nil +} + +func (x *BoundedTrieNode) SetTruncated(v bool) { + x.xxx_hidden_Truncated = v +} + +func (x *BoundedTrieNode) SetChildren(v map[string]*BoundedTrieNode) { + x.xxx_hidden_Children = v +} + +type BoundedTrieNode_builder struct { + _ [0]func() // Prevents comparability and use of unkeyed literals for the builder. + + // Whether this node has been truncated. + // A truncated leaf represents possibly many children with the same prefix. + Truncated bool + // Children of this node. Must be empty if truncated is true. + Children map[string]*BoundedTrieNode +} + +func (b0 BoundedTrieNode_builder) Build() *BoundedTrieNode { + m0 := &BoundedTrieNode{} + b, x := &b0, m0 + _, _ = b, x + x.xxx_hidden_Truncated = b.Truncated + x.xxx_hidden_Children = b.Children + return m0 +} + +// The message type used for encoding metrics of type bounded trie. +type BoundedTrie struct { + state protoimpl.MessageState `protogen:"opaque.v1"` + xxx_hidden_Bound int32 `protobuf:"varint,1,opt,name=bound,proto3" json:"bound,omitempty"` + xxx_hidden_Root *BoundedTrieNode `protobuf:"bytes,2,opt,name=root,proto3" json:"root,omitempty"` + xxx_hidden_Singleton []string `protobuf:"bytes,3,rep,name=singleton,proto3" json:"singleton,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *BoundedTrie) Reset() { + *x = BoundedTrie{} + mi := &file_org_apache_beam_model_pipeline_v1_metrics_proto_msgTypes[7] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *BoundedTrie) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*BoundedTrie) ProtoMessage() {} + +func (x *BoundedTrie) ProtoReflect() protoreflect.Message { + mi := &file_org_apache_beam_model_pipeline_v1_metrics_proto_msgTypes[7] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +func (x *BoundedTrie) GetBound() int32 { + if x != nil { + return x.xxx_hidden_Bound + } + return 0 +} + +func (x *BoundedTrie) GetRoot() *BoundedTrieNode { + if x != nil { + return x.xxx_hidden_Root + } + return nil +} + +func (x *BoundedTrie) GetSingleton() []string { + if x != nil { + return x.xxx_hidden_Singleton + } + return nil +} + +func (x *BoundedTrie) SetBound(v int32) { + x.xxx_hidden_Bound = v +} + +func (x *BoundedTrie) SetRoot(v *BoundedTrieNode) { + x.xxx_hidden_Root = v +} + +func (x *BoundedTrie) SetSingleton(v []string) { + x.xxx_hidden_Singleton = v +} + +func (x *BoundedTrie) HasRoot() bool { + if x == nil { + return false + } + return x.xxx_hidden_Root != nil +} + +func (x *BoundedTrie) ClearRoot() { + x.xxx_hidden_Root = nil +} + +type BoundedTrie_builder struct { + _ [0]func() // Prevents comparability and use of unkeyed literals for the builder. + + // The maximum number of elements to store before truncation. + Bound int32 + // A compact representation of all the elements in this trie. + Root *BoundedTrieNode + // A more efficient representation for metrics consisting of a single value. + Singleton []string +} + +func (b0 BoundedTrie_builder) Build() *BoundedTrie { + m0 := &BoundedTrie{} + b, x := &b0, m0 + _, _ = b, x + x.xxx_hidden_Bound = b.Bound + x.xxx_hidden_Root = b.Root + x.xxx_hidden_Singleton = b.Singleton + return m0 +} + +var file_org_apache_beam_model_pipeline_v1_metrics_proto_extTypes = []protoimpl.ExtensionInfo{ + { + ExtendedType: (*descriptorpb.EnumValueOptions)(nil), + ExtensionType: (*MonitoringInfoLabelProps)(nil), + Field: 127337796, + Name: "org.apache.beam.model.pipeline.v1.label_props", + Tag: "bytes,127337796,opt,name=label_props", + Filename: "org/apache/beam/model/pipeline/v1/metrics.proto", + }, + { + ExtendedType: (*descriptorpb.EnumValueOptions)(nil), + ExtensionType: (*MonitoringInfoSpec)(nil), + Field: 207174266, + Name: "org.apache.beam.model.pipeline.v1.monitoring_info_spec", + Tag: "bytes,207174266,opt,name=monitoring_info_spec", + Filename: "org/apache/beam/model/pipeline/v1/metrics.proto", + }, +} + +// Extension fields to descriptorpb.EnumValueOptions. +var ( + // optional org.apache.beam.model.pipeline.v1.MonitoringInfoLabelProps label_props = 127337796; + E_LabelProps = &file_org_apache_beam_model_pipeline_v1_metrics_proto_extTypes[0] // From: commit 0x7970544. + // Enum extension to store the MonitoringInfoSpecs. + // + // optional org.apache.beam.model.pipeline.v1.MonitoringInfoSpec monitoring_info_spec = 207174266; + E_MonitoringInfoSpec = &file_org_apache_beam_model_pipeline_v1_metrics_proto_extTypes[1] +) + +var File_org_apache_beam_model_pipeline_v1_metrics_proto protoreflect.FileDescriptor + +var file_org_apache_beam_model_pipeline_v1_metrics_proto_rawDesc = []byte{ + 0x0a, 0x2f, 0x6f, 0x72, 0x67, 0x2f, 0x61, 0x70, 0x61, 0x63, 0x68, 0x65, 0x2f, 0x62, 0x65, 0x61, + 0x6d, 0x2f, 0x6d, 0x6f, 0x64, 0x65, 0x6c, 0x2f, 0x70, 0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e, 0x65, + 0x2f, 0x76, 0x31, 0x2f, 0x6d, 0x65, 0x74, 0x72, 0x69, 0x63, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, + 0x6f, 0x12, 0x21, 0x6f, 0x72, 0x67, 0x2e, 0x61, 0x70, 0x61, 0x63, 0x68, 0x65, 0x2e, 0x62, 0x65, + 0x61, 0x6d, 0x2e, 0x6d, 0x6f, 0x64, 0x65, 0x6c, 0x2e, 0x70, 0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e, + 0x65, 0x2e, 0x76, 0x31, 0x1a, 0x37, 0x6f, 0x72, 0x67, 0x2f, 0x61, 0x70, 0x61, 0x63, 0x68, 0x65, + 0x2f, 0x62, 0x65, 0x61, 0x6d, 0x2f, 0x6d, 0x6f, 0x64, 0x65, 0x6c, 0x2f, 0x70, 0x69, 0x70, 0x65, + 0x6c, 0x69, 0x6e, 0x65, 0x2f, 0x76, 0x31, 0x2f, 0x62, 0x65, 0x61, 0x6d, 0x5f, 0x72, 0x75, 0x6e, + 0x6e, 0x65, 0x72, 0x5f, 0x61, 0x70, 0x69, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x1a, 0x20, 0x67, + 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2f, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66, 0x2f, 0x64, + 0x65, 0x73, 0x63, 0x72, 0x69, 0x70, 0x74, 0x6f, 0x72, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x1a, + 0x1f, 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2f, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66, + 0x2f, 0x74, 0x69, 0x6d, 0x65, 0x73, 0x74, 0x61, 0x6d, 0x70, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, + 0x22, 0xb4, 0x01, 0x0a, 0x12, 0x4d, 0x6f, 0x6e, 0x69, 0x74, 0x6f, 0x72, 0x69, 0x6e, 0x67, 0x49, + 0x6e, 0x66, 0x6f, 0x53, 0x70, 0x65, 0x63, 0x12, 0x10, 0x0a, 0x03, 0x75, 0x72, 0x6e, 0x18, 0x01, + 0x20, 0x01, 0x28, 0x09, 0x52, 0x03, 0x75, 0x72, 0x6e, 0x12, 0x12, 0x0a, 0x04, 0x74, 0x79, 0x70, + 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x04, 0x74, 0x79, 0x70, 0x65, 0x12, 0x27, 0x0a, + 0x0f, 0x72, 0x65, 0x71, 0x75, 0x69, 0x72, 0x65, 0x64, 0x5f, 0x6c, 0x61, 0x62, 0x65, 0x6c, 0x73, + 0x18, 0x03, 0x20, 0x03, 0x28, 0x09, 0x52, 0x0e, 0x72, 0x65, 0x71, 0x75, 0x69, 0x72, 0x65, 0x64, + 0x4c, 0x61, 0x62, 0x65, 0x6c, 0x73, 0x12, 0x4f, 0x0a, 0x0b, 0x61, 0x6e, 0x6e, 0x6f, 0x74, 0x61, + 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x18, 0x04, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x2d, 0x2e, 0x6f, 0x72, + 0x67, 0x2e, 0x61, 0x70, 0x61, 0x63, 0x68, 0x65, 0x2e, 0x62, 0x65, 0x61, 0x6d, 0x2e, 0x6d, 0x6f, + 0x64, 0x65, 0x6c, 0x2e, 0x70, 0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e, 0x65, 0x2e, 0x76, 0x31, 0x2e, + 0x41, 0x6e, 0x6e, 0x6f, 0x74, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x0b, 0x61, 0x6e, 0x6e, 0x6f, + 0x74, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x22, 0x34, 0x0a, 0x0a, 0x41, 0x6e, 0x6e, 0x6f, 0x74, + 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x10, 0x0a, 0x03, 0x6b, 0x65, 0x79, 0x18, 0x01, 0x20, 0x01, + 0x28, 0x09, 0x52, 0x03, 0x6b, 0x65, 0x79, 0x12, 0x14, 0x0a, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, + 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x22, 0xb1, 0x26, + 0x0a, 0x13, 0x4d, 0x6f, 0x6e, 0x69, 0x74, 0x6f, 0x72, 0x69, 0x6e, 0x67, 0x49, 0x6e, 0x66, 0x6f, + 0x53, 0x70, 0x65, 0x63, 0x73, 0x22, 0x99, 0x26, 0x0a, 0x04, 0x45, 0x6e, 0x75, 0x6d, 0x12, 0xa7, + 0x01, 0x0a, 0x0e, 0x55, 0x53, 0x45, 0x52, 0x5f, 0x53, 0x55, 0x4d, 0x5f, 0x49, 0x4e, 0x54, 0x36, + 0x34, 0x10, 0x00, 0x1a, 0x92, 0x01, 0xd2, 0xa7, 0xa7, 0x96, 0x06, 0x8b, 0x01, 0x0a, 0x1d, 0x62, + 0x65, 0x61, 0x6d, 0x3a, 0x6d, 0x65, 0x74, 0x72, 0x69, 0x63, 0x3a, 0x75, 0x73, 0x65, 0x72, 0x3a, + 0x73, 0x75, 0x6d, 0x5f, 0x69, 0x6e, 0x74, 0x36, 0x34, 0x3a, 0x76, 0x31, 0x12, 0x19, 0x62, 0x65, + 0x61, 0x6d, 0x3a, 0x6d, 0x65, 0x74, 0x72, 0x69, 0x63, 0x73, 0x3a, 0x73, 0x75, 0x6d, 0x5f, 0x69, + 0x6e, 0x74, 0x36, 0x34, 0x3a, 0x76, 0x31, 0x1a, 0x0a, 0x50, 0x54, 0x52, 0x41, 0x4e, 0x53, 0x46, + 0x4f, 0x52, 0x4d, 0x1a, 0x09, 0x4e, 0x41, 0x4d, 0x45, 0x53, 0x50, 0x41, 0x43, 0x45, 0x1a, 0x04, + 0x4e, 0x41, 0x4d, 0x45, 0x22, 0x32, 0x0a, 0x0b, 0x64, 0x65, 0x73, 0x63, 0x72, 0x69, 0x70, 0x74, + 0x69, 0x6f, 0x6e, 0x12, 0x23, 0x55, 0x52, 0x4e, 0x20, 0x75, 0x74, 0x69, 0x6c, 0x69, 0x7a, 0x65, + 0x64, 0x20, 0x74, 0x6f, 0x20, 0x72, 0x65, 0x70, 0x6f, 0x72, 0x74, 0x20, 0x75, 0x73, 0x65, 0x72, + 0x20, 0x6d, 0x65, 0x74, 0x72, 0x69, 0x63, 0x2e, 0x12, 0xaa, 0x01, 0x0a, 0x0f, 0x55, 0x53, 0x45, + 0x52, 0x5f, 0x53, 0x55, 0x4d, 0x5f, 0x44, 0x4f, 0x55, 0x42, 0x4c, 0x45, 0x10, 0x01, 0x1a, 0x94, + 0x01, 0xd2, 0xa7, 0xa7, 0x96, 0x06, 0x8d, 0x01, 0x0a, 0x1e, 0x62, 0x65, 0x61, 0x6d, 0x3a, 0x6d, + 0x65, 0x74, 0x72, 0x69, 0x63, 0x3a, 0x75, 0x73, 0x65, 0x72, 0x3a, 0x73, 0x75, 0x6d, 0x5f, 0x64, + 0x6f, 0x75, 0x62, 0x6c, 0x65, 0x3a, 0x76, 0x31, 0x12, 0x1a, 0x62, 0x65, 0x61, 0x6d, 0x3a, 0x6d, + 0x65, 0x74, 0x72, 0x69, 0x63, 0x73, 0x3a, 0x73, 0x75, 0x6d, 0x5f, 0x64, 0x6f, 0x75, 0x62, 0x6c, + 0x65, 0x3a, 0x76, 0x31, 0x1a, 0x0a, 0x50, 0x54, 0x52, 0x41, 0x4e, 0x53, 0x46, 0x4f, 0x52, 0x4d, + 0x1a, 0x09, 0x4e, 0x41, 0x4d, 0x45, 0x53, 0x50, 0x41, 0x43, 0x45, 0x1a, 0x04, 0x4e, 0x41, 0x4d, + 0x45, 0x22, 0x32, 0x0a, 0x0b, 0x64, 0x65, 0x73, 0x63, 0x72, 0x69, 0x70, 0x74, 0x69, 0x6f, 0x6e, + 0x12, 0x23, 0x55, 0x52, 0x4e, 0x20, 0x75, 0x74, 0x69, 0x6c, 0x69, 0x7a, 0x65, 0x64, 0x20, 0x74, + 0x6f, 0x20, 0x72, 0x65, 0x70, 0x6f, 0x72, 0x74, 0x20, 0x75, 0x73, 0x65, 0x72, 0x20, 0x6d, 0x65, + 0x74, 0x72, 0x69, 0x63, 0x2e, 0x12, 0xc2, 0x01, 0x0a, 0x17, 0x55, 0x53, 0x45, 0x52, 0x5f, 0x44, + 0x49, 0x53, 0x54, 0x52, 0x49, 0x42, 0x55, 0x54, 0x49, 0x4f, 0x4e, 0x5f, 0x49, 0x4e, 0x54, 0x36, + 0x34, 0x10, 0x02, 0x1a, 0xa4, 0x01, 0xd2, 0xa7, 0xa7, 0x96, 0x06, 0x9d, 0x01, 0x0a, 0x26, 0x62, + 0x65, 0x61, 0x6d, 0x3a, 0x6d, 0x65, 0x74, 0x72, 0x69, 0x63, 0x3a, 0x75, 0x73, 0x65, 0x72, 0x3a, + 0x64, 0x69, 0x73, 0x74, 0x72, 0x69, 0x62, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x69, 0x6e, 0x74, + 0x36, 0x34, 0x3a, 0x76, 0x31, 0x12, 0x22, 0x62, 0x65, 0x61, 0x6d, 0x3a, 0x6d, 0x65, 0x74, 0x72, + 0x69, 0x63, 0x73, 0x3a, 0x64, 0x69, 0x73, 0x74, 0x72, 0x69, 0x62, 0x75, 0x74, 0x69, 0x6f, 0x6e, + 0x5f, 0x69, 0x6e, 0x74, 0x36, 0x34, 0x3a, 0x76, 0x31, 0x1a, 0x0a, 0x50, 0x54, 0x52, 0x41, 0x4e, + 0x53, 0x46, 0x4f, 0x52, 0x4d, 0x1a, 0x09, 0x4e, 0x41, 0x4d, 0x45, 0x53, 0x50, 0x41, 0x43, 0x45, + 0x1a, 0x04, 0x4e, 0x41, 0x4d, 0x45, 0x22, 0x32, 0x0a, 0x0b, 0x64, 0x65, 0x73, 0x63, 0x72, 0x69, + 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x23, 0x55, 0x52, 0x4e, 0x20, 0x75, 0x74, 0x69, 0x6c, 0x69, + 0x7a, 0x65, 0x64, 0x20, 0x74, 0x6f, 0x20, 0x72, 0x65, 0x70, 0x6f, 0x72, 0x74, 0x20, 0x75, 0x73, + 0x65, 0x72, 0x20, 0x6d, 0x65, 0x74, 0x72, 0x69, 0x63, 0x2e, 0x12, 0xc5, 0x01, 0x0a, 0x18, 0x55, + 0x53, 0x45, 0x52, 0x5f, 0x44, 0x49, 0x53, 0x54, 0x52, 0x49, 0x42, 0x55, 0x54, 0x49, 0x4f, 0x4e, + 0x5f, 0x44, 0x4f, 0x55, 0x42, 0x4c, 0x45, 0x10, 0x03, 0x1a, 0xa6, 0x01, 0xd2, 0xa7, 0xa7, 0x96, + 0x06, 0x9f, 0x01, 0x0a, 0x27, 0x62, 0x65, 0x61, 0x6d, 0x3a, 0x6d, 0x65, 0x74, 0x72, 0x69, 0x63, + 0x3a, 0x75, 0x73, 0x65, 0x72, 0x3a, 0x64, 0x69, 0x73, 0x74, 0x72, 0x69, 0x62, 0x75, 0x74, 0x69, + 0x6f, 0x6e, 0x5f, 0x64, 0x6f, 0x75, 0x62, 0x6c, 0x65, 0x3a, 0x76, 0x31, 0x12, 0x23, 0x62, 0x65, + 0x61, 0x6d, 0x3a, 0x6d, 0x65, 0x74, 0x72, 0x69, 0x63, 0x73, 0x3a, 0x64, 0x69, 0x73, 0x74, 0x72, + 0x69, 0x62, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x64, 0x6f, 0x75, 0x62, 0x6c, 0x65, 0x3a, 0x76, + 0x31, 0x1a, 0x0a, 0x50, 0x54, 0x52, 0x41, 0x4e, 0x53, 0x46, 0x4f, 0x52, 0x4d, 0x1a, 0x09, 0x4e, + 0x41, 0x4d, 0x45, 0x53, 0x50, 0x41, 0x43, 0x45, 0x1a, 0x04, 0x4e, 0x41, 0x4d, 0x45, 0x22, 0x32, + 0x0a, 0x0b, 0x64, 0x65, 0x73, 0x63, 0x72, 0x69, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x23, 0x55, + 0x52, 0x4e, 0x20, 0x75, 0x74, 0x69, 0x6c, 0x69, 0x7a, 0x65, 0x64, 0x20, 0x74, 0x6f, 0x20, 0x72, + 0x65, 0x70, 0x6f, 0x72, 0x74, 0x20, 0x75, 0x73, 0x65, 0x72, 0x20, 0x6d, 0x65, 0x74, 0x72, 0x69, + 0x63, 0x2e, 0x12, 0xb0, 0x01, 0x0a, 0x11, 0x55, 0x53, 0x45, 0x52, 0x5f, 0x4c, 0x41, 0x54, 0x45, + 0x53, 0x54, 0x5f, 0x49, 0x4e, 0x54, 0x36, 0x34, 0x10, 0x04, 0x1a, 0x98, 0x01, 0xd2, 0xa7, 0xa7, + 0x96, 0x06, 0x91, 0x01, 0x0a, 0x20, 0x62, 0x65, 0x61, 0x6d, 0x3a, 0x6d, 0x65, 0x74, 0x72, 0x69, + 0x63, 0x3a, 0x75, 0x73, 0x65, 0x72, 0x3a, 0x6c, 0x61, 0x74, 0x65, 0x73, 0x74, 0x5f, 0x69, 0x6e, + 0x74, 0x36, 0x34, 0x3a, 0x76, 0x31, 0x12, 0x1c, 0x62, 0x65, 0x61, 0x6d, 0x3a, 0x6d, 0x65, 0x74, + 0x72, 0x69, 0x63, 0x73, 0x3a, 0x6c, 0x61, 0x74, 0x65, 0x73, 0x74, 0x5f, 0x69, 0x6e, 0x74, 0x36, + 0x34, 0x3a, 0x76, 0x31, 0x1a, 0x0a, 0x50, 0x54, 0x52, 0x41, 0x4e, 0x53, 0x46, 0x4f, 0x52, 0x4d, + 0x1a, 0x09, 0x4e, 0x41, 0x4d, 0x45, 0x53, 0x50, 0x41, 0x43, 0x45, 0x1a, 0x04, 0x4e, 0x41, 0x4d, + 0x45, 0x22, 0x32, 0x0a, 0x0b, 0x64, 0x65, 0x73, 0x63, 0x72, 0x69, 0x70, 0x74, 0x69, 0x6f, 0x6e, + 0x12, 0x23, 0x55, 0x52, 0x4e, 0x20, 0x75, 0x74, 0x69, 0x6c, 0x69, 0x7a, 0x65, 0x64, 0x20, 0x74, + 0x6f, 0x20, 0x72, 0x65, 0x70, 0x6f, 0x72, 0x74, 0x20, 0x75, 0x73, 0x65, 0x72, 0x20, 0x6d, 0x65, + 0x74, 0x72, 0x69, 0x63, 0x2e, 0x12, 0xb3, 0x01, 0x0a, 0x12, 0x55, 0x53, 0x45, 0x52, 0x5f, 0x4c, + 0x41, 0x54, 0x45, 0x53, 0x54, 0x5f, 0x44, 0x4f, 0x55, 0x42, 0x4c, 0x45, 0x10, 0x05, 0x1a, 0x9a, + 0x01, 0xd2, 0xa7, 0xa7, 0x96, 0x06, 0x93, 0x01, 0x0a, 0x21, 0x62, 0x65, 0x61, 0x6d, 0x3a, 0x6d, + 0x65, 0x74, 0x72, 0x69, 0x63, 0x3a, 0x75, 0x73, 0x65, 0x72, 0x3a, 0x6c, 0x61, 0x74, 0x65, 0x73, + 0x74, 0x5f, 0x64, 0x6f, 0x75, 0x62, 0x6c, 0x65, 0x3a, 0x76, 0x31, 0x12, 0x1d, 0x62, 0x65, 0x61, + 0x6d, 0x3a, 0x6d, 0x65, 0x74, 0x72, 0x69, 0x63, 0x73, 0x3a, 0x6c, 0x61, 0x74, 0x65, 0x73, 0x74, + 0x5f, 0x64, 0x6f, 0x75, 0x62, 0x6c, 0x65, 0x3a, 0x76, 0x31, 0x1a, 0x0a, 0x50, 0x54, 0x52, 0x41, + 0x4e, 0x53, 0x46, 0x4f, 0x52, 0x4d, 0x1a, 0x09, 0x4e, 0x41, 0x4d, 0x45, 0x53, 0x50, 0x41, 0x43, + 0x45, 0x1a, 0x04, 0x4e, 0x41, 0x4d, 0x45, 0x22, 0x32, 0x0a, 0x0b, 0x64, 0x65, 0x73, 0x63, 0x72, + 0x69, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x23, 0x55, 0x52, 0x4e, 0x20, 0x75, 0x74, 0x69, 0x6c, + 0x69, 0x7a, 0x65, 0x64, 0x20, 0x74, 0x6f, 0x20, 0x72, 0x65, 0x70, 0x6f, 0x72, 0x74, 0x20, 0x75, + 0x73, 0x65, 0x72, 0x20, 0x6d, 0x65, 0x74, 0x72, 0x69, 0x63, 0x2e, 0x12, 0xad, 0x01, 0x0a, 0x10, + 0x55, 0x53, 0x45, 0x52, 0x5f, 0x54, 0x4f, 0x50, 0x5f, 0x4e, 0x5f, 0x49, 0x4e, 0x54, 0x36, 0x34, + 0x10, 0x06, 0x1a, 0x96, 0x01, 0xd2, 0xa7, 0xa7, 0x96, 0x06, 0x8f, 0x01, 0x0a, 0x1f, 0x62, 0x65, + 0x61, 0x6d, 0x3a, 0x6d, 0x65, 0x74, 0x72, 0x69, 0x63, 0x3a, 0x75, 0x73, 0x65, 0x72, 0x3a, 0x74, + 0x6f, 0x70, 0x5f, 0x6e, 0x5f, 0x69, 0x6e, 0x74, 0x36, 0x34, 0x3a, 0x76, 0x31, 0x12, 0x1b, 0x62, + 0x65, 0x61, 0x6d, 0x3a, 0x6d, 0x65, 0x74, 0x72, 0x69, 0x63, 0x73, 0x3a, 0x74, 0x6f, 0x70, 0x5f, + 0x6e, 0x5f, 0x69, 0x6e, 0x74, 0x36, 0x34, 0x3a, 0x76, 0x31, 0x1a, 0x0a, 0x50, 0x54, 0x52, 0x41, + 0x4e, 0x53, 0x46, 0x4f, 0x52, 0x4d, 0x1a, 0x09, 0x4e, 0x41, 0x4d, 0x45, 0x53, 0x50, 0x41, 0x43, + 0x45, 0x1a, 0x04, 0x4e, 0x41, 0x4d, 0x45, 0x22, 0x32, 0x0a, 0x0b, 0x64, 0x65, 0x73, 0x63, 0x72, + 0x69, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x23, 0x55, 0x52, 0x4e, 0x20, 0x75, 0x74, 0x69, 0x6c, + 0x69, 0x7a, 0x65, 0x64, 0x20, 0x74, 0x6f, 0x20, 0x72, 0x65, 0x70, 0x6f, 0x72, 0x74, 0x20, 0x75, + 0x73, 0x65, 0x72, 0x20, 0x6d, 0x65, 0x74, 0x72, 0x69, 0x63, 0x2e, 0x12, 0xb0, 0x01, 0x0a, 0x11, + 0x55, 0x53, 0x45, 0x52, 0x5f, 0x54, 0x4f, 0x50, 0x5f, 0x4e, 0x5f, 0x44, 0x4f, 0x55, 0x42, 0x4c, + 0x45, 0x10, 0x07, 0x1a, 0x98, 0x01, 0xd2, 0xa7, 0xa7, 0x96, 0x06, 0x91, 0x01, 0x0a, 0x20, 0x62, + 0x65, 0x61, 0x6d, 0x3a, 0x6d, 0x65, 0x74, 0x72, 0x69, 0x63, 0x3a, 0x75, 0x73, 0x65, 0x72, 0x3a, + 0x74, 0x6f, 0x70, 0x5f, 0x6e, 0x5f, 0x64, 0x6f, 0x75, 0x62, 0x6c, 0x65, 0x3a, 0x76, 0x31, 0x12, + 0x1c, 0x62, 0x65, 0x61, 0x6d, 0x3a, 0x6d, 0x65, 0x74, 0x72, 0x69, 0x63, 0x73, 0x3a, 0x74, 0x6f, + 0x70, 0x5f, 0x6e, 0x5f, 0x64, 0x6f, 0x75, 0x62, 0x6c, 0x65, 0x3a, 0x76, 0x31, 0x1a, 0x0a, 0x50, + 0x54, 0x52, 0x41, 0x4e, 0x53, 0x46, 0x4f, 0x52, 0x4d, 0x1a, 0x09, 0x4e, 0x41, 0x4d, 0x45, 0x53, + 0x50, 0x41, 0x43, 0x45, 0x1a, 0x04, 0x4e, 0x41, 0x4d, 0x45, 0x22, 0x32, 0x0a, 0x0b, 0x64, 0x65, + 0x73, 0x63, 0x72, 0x69, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x23, 0x55, 0x52, 0x4e, 0x20, 0x75, + 0x74, 0x69, 0x6c, 0x69, 0x7a, 0x65, 0x64, 0x20, 0x74, 0x6f, 0x20, 0x72, 0x65, 0x70, 0x6f, 0x72, + 0x74, 0x20, 0x75, 0x73, 0x65, 0x72, 0x20, 0x6d, 0x65, 0x74, 0x72, 0x69, 0x63, 0x2e, 0x12, 0xb6, + 0x01, 0x0a, 0x13, 0x55, 0x53, 0x45, 0x52, 0x5f, 0x42, 0x4f, 0x54, 0x54, 0x4f, 0x4d, 0x5f, 0x4e, + 0x5f, 0x49, 0x4e, 0x54, 0x36, 0x34, 0x10, 0x08, 0x1a, 0x9c, 0x01, 0xd2, 0xa7, 0xa7, 0x96, 0x06, + 0x95, 0x01, 0x0a, 0x22, 0x62, 0x65, 0x61, 0x6d, 0x3a, 0x6d, 0x65, 0x74, 0x72, 0x69, 0x63, 0x3a, + 0x75, 0x73, 0x65, 0x72, 0x3a, 0x62, 0x6f, 0x74, 0x74, 0x6f, 0x6d, 0x5f, 0x6e, 0x5f, 0x69, 0x6e, + 0x74, 0x36, 0x34, 0x3a, 0x76, 0x31, 0x12, 0x1e, 0x62, 0x65, 0x61, 0x6d, 0x3a, 0x6d, 0x65, 0x74, + 0x72, 0x69, 0x63, 0x73, 0x3a, 0x62, 0x6f, 0x74, 0x74, 0x6f, 0x6d, 0x5f, 0x6e, 0x5f, 0x69, 0x6e, + 0x74, 0x36, 0x34, 0x3a, 0x76, 0x31, 0x1a, 0x0a, 0x50, 0x54, 0x52, 0x41, 0x4e, 0x53, 0x46, 0x4f, + 0x52, 0x4d, 0x1a, 0x09, 0x4e, 0x41, 0x4d, 0x45, 0x53, 0x50, 0x41, 0x43, 0x45, 0x1a, 0x04, 0x4e, + 0x41, 0x4d, 0x45, 0x22, 0x32, 0x0a, 0x0b, 0x64, 0x65, 0x73, 0x63, 0x72, 0x69, 0x70, 0x74, 0x69, + 0x6f, 0x6e, 0x12, 0x23, 0x55, 0x52, 0x4e, 0x20, 0x75, 0x74, 0x69, 0x6c, 0x69, 0x7a, 0x65, 0x64, + 0x20, 0x74, 0x6f, 0x20, 0x72, 0x65, 0x70, 0x6f, 0x72, 0x74, 0x20, 0x75, 0x73, 0x65, 0x72, 0x20, + 0x6d, 0x65, 0x74, 0x72, 0x69, 0x63, 0x2e, 0x12, 0xb9, 0x01, 0x0a, 0x14, 0x55, 0x53, 0x45, 0x52, + 0x5f, 0x42, 0x4f, 0x54, 0x54, 0x4f, 0x4d, 0x5f, 0x4e, 0x5f, 0x44, 0x4f, 0x55, 0x42, 0x4c, 0x45, + 0x10, 0x09, 0x1a, 0x9e, 0x01, 0xd2, 0xa7, 0xa7, 0x96, 0x06, 0x97, 0x01, 0x0a, 0x23, 0x62, 0x65, + 0x61, 0x6d, 0x3a, 0x6d, 0x65, 0x74, 0x72, 0x69, 0x63, 0x3a, 0x75, 0x73, 0x65, 0x72, 0x3a, 0x62, + 0x6f, 0x74, 0x74, 0x6f, 0x6d, 0x5f, 0x6e, 0x5f, 0x64, 0x6f, 0x75, 0x62, 0x6c, 0x65, 0x3a, 0x76, + 0x31, 0x12, 0x1f, 0x62, 0x65, 0x61, 0x6d, 0x3a, 0x6d, 0x65, 0x74, 0x72, 0x69, 0x63, 0x73, 0x3a, + 0x62, 0x6f, 0x74, 0x74, 0x6f, 0x6d, 0x5f, 0x6e, 0x5f, 0x64, 0x6f, 0x75, 0x62, 0x6c, 0x65, 0x3a, + 0x76, 0x31, 0x1a, 0x0a, 0x50, 0x54, 0x52, 0x41, 0x4e, 0x53, 0x46, 0x4f, 0x52, 0x4d, 0x1a, 0x09, + 0x4e, 0x41, 0x4d, 0x45, 0x53, 0x50, 0x41, 0x43, 0x45, 0x1a, 0x04, 0x4e, 0x41, 0x4d, 0x45, 0x22, + 0x32, 0x0a, 0x0b, 0x64, 0x65, 0x73, 0x63, 0x72, 0x69, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x23, + 0x55, 0x52, 0x4e, 0x20, 0x75, 0x74, 0x69, 0x6c, 0x69, 0x7a, 0x65, 0x64, 0x20, 0x74, 0x6f, 0x20, + 0x72, 0x65, 0x70, 0x6f, 0x72, 0x74, 0x20, 0x75, 0x73, 0x65, 0x72, 0x20, 0x6d, 0x65, 0x74, 0x72, + 0x69, 0x63, 0x2e, 0x12, 0xaa, 0x01, 0x0a, 0x0f, 0x55, 0x53, 0x45, 0x52, 0x5f, 0x53, 0x45, 0x54, + 0x5f, 0x53, 0x54, 0x52, 0x49, 0x4e, 0x47, 0x10, 0x15, 0x1a, 0x94, 0x01, 0xd2, 0xa7, 0xa7, 0x96, + 0x06, 0x8d, 0x01, 0x0a, 0x1e, 0x62, 0x65, 0x61, 0x6d, 0x3a, 0x6d, 0x65, 0x74, 0x72, 0x69, 0x63, + 0x3a, 0x75, 0x73, 0x65, 0x72, 0x3a, 0x73, 0x65, 0x74, 0x5f, 0x73, 0x74, 0x72, 0x69, 0x6e, 0x67, + 0x3a, 0x76, 0x31, 0x12, 0x1a, 0x62, 0x65, 0x61, 0x6d, 0x3a, 0x6d, 0x65, 0x74, 0x72, 0x69, 0x63, + 0x73, 0x3a, 0x73, 0x65, 0x74, 0x5f, 0x73, 0x74, 0x72, 0x69, 0x6e, 0x67, 0x3a, 0x76, 0x31, 0x1a, + 0x0a, 0x50, 0x54, 0x52, 0x41, 0x4e, 0x53, 0x46, 0x4f, 0x52, 0x4d, 0x1a, 0x09, 0x4e, 0x41, 0x4d, + 0x45, 0x53, 0x50, 0x41, 0x43, 0x45, 0x1a, 0x04, 0x4e, 0x41, 0x4d, 0x45, 0x22, 0x32, 0x0a, 0x0b, + 0x64, 0x65, 0x73, 0x63, 0x72, 0x69, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x23, 0x55, 0x52, 0x4e, + 0x20, 0x75, 0x74, 0x69, 0x6c, 0x69, 0x7a, 0x65, 0x64, 0x20, 0x74, 0x6f, 0x20, 0x72, 0x65, 0x70, + 0x6f, 0x72, 0x74, 0x20, 0x75, 0x73, 0x65, 0x72, 0x20, 0x6d, 0x65, 0x74, 0x72, 0x69, 0x63, 0x2e, + 0x12, 0xb0, 0x01, 0x0a, 0x11, 0x55, 0x53, 0x45, 0x52, 0x5f, 0x42, 0x4f, 0x55, 0x4e, 0x44, 0x45, + 0x44, 0x5f, 0x54, 0x52, 0x49, 0x45, 0x10, 0x16, 0x1a, 0x98, 0x01, 0xd2, 0xa7, 0xa7, 0x96, 0x06, + 0x91, 0x01, 0x0a, 0x20, 0x62, 0x65, 0x61, 0x6d, 0x3a, 0x6d, 0x65, 0x74, 0x72, 0x69, 0x63, 0x3a, + 0x75, 0x73, 0x65, 0x72, 0x3a, 0x62, 0x6f, 0x75, 0x6e, 0x64, 0x65, 0x64, 0x5f, 0x74, 0x72, 0x69, + 0x65, 0x3a, 0x76, 0x31, 0x12, 0x1c, 0x62, 0x65, 0x61, 0x6d, 0x3a, 0x6d, 0x65, 0x74, 0x72, 0x69, + 0x63, 0x73, 0x3a, 0x62, 0x6f, 0x75, 0x6e, 0x64, 0x65, 0x64, 0x5f, 0x74, 0x72, 0x69, 0x65, 0x3a, + 0x76, 0x31, 0x1a, 0x0a, 0x50, 0x54, 0x52, 0x41, 0x4e, 0x53, 0x46, 0x4f, 0x52, 0x4d, 0x1a, 0x09, + 0x4e, 0x41, 0x4d, 0x45, 0x53, 0x50, 0x41, 0x43, 0x45, 0x1a, 0x04, 0x4e, 0x41, 0x4d, 0x45, 0x22, + 0x32, 0x0a, 0x0b, 0x64, 0x65, 0x73, 0x63, 0x72, 0x69, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x23, + 0x55, 0x52, 0x4e, 0x20, 0x75, 0x74, 0x69, 0x6c, 0x69, 0x7a, 0x65, 0x64, 0x20, 0x74, 0x6f, 0x20, + 0x72, 0x65, 0x70, 0x6f, 0x72, 0x74, 0x20, 0x75, 0x73, 0x65, 0x72, 0x20, 0x6d, 0x65, 0x74, 0x72, + 0x69, 0x63, 0x2e, 0x12, 0xad, 0x01, 0x0a, 0x0d, 0x45, 0x4c, 0x45, 0x4d, 0x45, 0x4e, 0x54, 0x5f, + 0x43, 0x4f, 0x55, 0x4e, 0x54, 0x10, 0x0a, 0x1a, 0x99, 0x01, 0xd2, 0xa7, 0xa7, 0x96, 0x06, 0x92, + 0x01, 0x0a, 0x1c, 0x62, 0x65, 0x61, 0x6d, 0x3a, 0x6d, 0x65, 0x74, 0x72, 0x69, 0x63, 0x3a, 0x65, + 0x6c, 0x65, 0x6d, 0x65, 0x6e, 0x74, 0x5f, 0x63, 0x6f, 0x75, 0x6e, 0x74, 0x3a, 0x76, 0x31, 0x12, + 0x19, 0x62, 0x65, 0x61, 0x6d, 0x3a, 0x6d, 0x65, 0x74, 0x72, 0x69, 0x63, 0x73, 0x3a, 0x73, 0x75, + 0x6d, 0x5f, 0x69, 0x6e, 0x74, 0x36, 0x34, 0x3a, 0x76, 0x31, 0x1a, 0x0b, 0x50, 0x43, 0x4f, 0x4c, + 0x4c, 0x45, 0x43, 0x54, 0x49, 0x4f, 0x4e, 0x22, 0x4a, 0x0a, 0x0b, 0x64, 0x65, 0x73, 0x63, 0x72, + 0x69, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x3b, 0x54, 0x68, 0x65, 0x20, 0x74, 0x6f, 0x74, 0x61, + 0x6c, 0x20, 0x65, 0x6c, 0x65, 0x6d, 0x65, 0x6e, 0x74, 0x73, 0x20, 0x6f, 0x75, 0x74, 0x70, 0x75, + 0x74, 0x20, 0x74, 0x6f, 0x20, 0x61, 0x20, 0x50, 0x63, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, + 0x6f, 0x6e, 0x20, 0x62, 0x79, 0x20, 0x61, 0x20, 0x50, 0x54, 0x72, 0x61, 0x6e, 0x73, 0x66, 0x6f, + 0x72, 0x6d, 0x2e, 0x12, 0xcd, 0x02, 0x0a, 0x11, 0x53, 0x41, 0x4d, 0x50, 0x4c, 0x45, 0x44, 0x5f, + 0x42, 0x59, 0x54, 0x45, 0x5f, 0x53, 0x49, 0x5a, 0x45, 0x10, 0x0b, 0x1a, 0xb5, 0x02, 0xd2, 0xa7, + 0xa7, 0x96, 0x06, 0xae, 0x02, 0x0a, 0x20, 0x62, 0x65, 0x61, 0x6d, 0x3a, 0x6d, 0x65, 0x74, 0x72, + 0x69, 0x63, 0x3a, 0x73, 0x61, 0x6d, 0x70, 0x6c, 0x65, 0x64, 0x5f, 0x62, 0x79, 0x74, 0x65, 0x5f, + 0x73, 0x69, 0x7a, 0x65, 0x3a, 0x76, 0x31, 0x12, 0x22, 0x62, 0x65, 0x61, 0x6d, 0x3a, 0x6d, 0x65, + 0x74, 0x72, 0x69, 0x63, 0x73, 0x3a, 0x64, 0x69, 0x73, 0x74, 0x72, 0x69, 0x62, 0x75, 0x74, 0x69, + 0x6f, 0x6e, 0x5f, 0x69, 0x6e, 0x74, 0x36, 0x34, 0x3a, 0x76, 0x31, 0x1a, 0x0b, 0x50, 0x43, 0x4f, + 0x4c, 0x4c, 0x45, 0x43, 0x54, 0x49, 0x4f, 0x4e, 0x22, 0xd8, 0x01, 0x0a, 0x0b, 0x64, 0x65, 0x73, + 0x63, 0x72, 0x69, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0xc8, 0x01, 0x54, 0x68, 0x65, 0x20, 0x74, + 0x6f, 0x74, 0x61, 0x6c, 0x20, 0x62, 0x79, 0x74, 0x65, 0x20, 0x73, 0x69, 0x7a, 0x65, 0x20, 0x61, + 0x6e, 0x64, 0x20, 0x63, 0x6f, 0x75, 0x6e, 0x74, 0x20, 0x6f, 0x66, 0x20, 0x61, 0x20, 0x73, 0x61, + 0x6d, 0x70, 0x6c, 0x65, 0x64, 0x20, 0x20, 0x73, 0x65, 0x74, 0x20, 0x28, 0x6f, 0x72, 0x20, 0x61, + 0x6c, 0x6c, 0x29, 0x20, 0x6f, 0x66, 0x20, 0x65, 0x6c, 0x65, 0x6d, 0x65, 0x6e, 0x74, 0x73, 0x20, + 0x69, 0x6e, 0x20, 0x74, 0x68, 0x65, 0x20, 0x70, 0x63, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, + 0x6f, 0x6e, 0x2e, 0x20, 0x53, 0x61, 0x6d, 0x70, 0x6c, 0x69, 0x6e, 0x67, 0x20, 0x69, 0x73, 0x20, + 0x75, 0x73, 0x65, 0x64, 0x20, 0x20, 0x62, 0x65, 0x63, 0x61, 0x75, 0x73, 0x65, 0x20, 0x63, 0x61, + 0x6c, 0x63, 0x75, 0x6c, 0x61, 0x74, 0x69, 0x6e, 0x67, 0x20, 0x74, 0x68, 0x65, 0x20, 0x62, 0x79, + 0x74, 0x65, 0x20, 0x63, 0x6f, 0x75, 0x6e, 0x74, 0x20, 0x69, 0x6e, 0x76, 0x6f, 0x6c, 0x76, 0x65, + 0x73, 0x20, 0x73, 0x65, 0x72, 0x69, 0x61, 0x6c, 0x69, 0x7a, 0x69, 0x6e, 0x67, 0x20, 0x74, 0x68, + 0x65, 0x20, 0x20, 0x65, 0x6c, 0x65, 0x6d, 0x65, 0x6e, 0x74, 0x73, 0x20, 0x77, 0x68, 0x69, 0x63, + 0x68, 0x20, 0x69, 0x73, 0x20, 0x43, 0x50, 0x55, 0x20, 0x69, 0x6e, 0x74, 0x65, 0x6e, 0x73, 0x69, + 0x76, 0x65, 0x2e, 0x12, 0xd9, 0x01, 0x0a, 0x12, 0x53, 0x54, 0x41, 0x52, 0x54, 0x5f, 0x42, 0x55, + 0x4e, 0x44, 0x4c, 0x45, 0x5f, 0x4d, 0x53, 0x45, 0x43, 0x53, 0x10, 0x0c, 0x1a, 0xc0, 0x01, 0xd2, + 0xa7, 0xa7, 0x96, 0x06, 0xb9, 0x01, 0x0a, 0x36, 0x62, 0x65, 0x61, 0x6d, 0x3a, 0x6d, 0x65, 0x74, + 0x72, 0x69, 0x63, 0x3a, 0x70, 0x61, 0x72, 0x64, 0x6f, 0x5f, 0x65, 0x78, 0x65, 0x63, 0x75, 0x74, + 0x69, 0x6f, 0x6e, 0x5f, 0x74, 0x69, 0x6d, 0x65, 0x3a, 0x73, 0x74, 0x61, 0x72, 0x74, 0x5f, 0x62, + 0x75, 0x6e, 0x64, 0x6c, 0x65, 0x5f, 0x6d, 0x73, 0x65, 0x63, 0x73, 0x3a, 0x76, 0x31, 0x12, 0x19, + 0x62, 0x65, 0x61, 0x6d, 0x3a, 0x6d, 0x65, 0x74, 0x72, 0x69, 0x63, 0x73, 0x3a, 0x73, 0x75, 0x6d, + 0x5f, 0x69, 0x6e, 0x74, 0x36, 0x34, 0x3a, 0x76, 0x31, 0x1a, 0x0a, 0x50, 0x54, 0x52, 0x41, 0x4e, + 0x53, 0x46, 0x4f, 0x52, 0x4d, 0x22, 0x58, 0x0a, 0x0b, 0x64, 0x65, 0x73, 0x63, 0x72, 0x69, 0x70, + 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x49, 0x54, 0x68, 0x65, 0x20, 0x74, 0x6f, 0x74, 0x61, 0x6c, 0x20, + 0x65, 0x73, 0x74, 0x69, 0x6d, 0x61, 0x74, 0x65, 0x64, 0x20, 0x65, 0x78, 0x65, 0x63, 0x75, 0x74, + 0x69, 0x6f, 0x6e, 0x20, 0x74, 0x69, 0x6d, 0x65, 0x20, 0x6f, 0x66, 0x20, 0x74, 0x68, 0x65, 0x20, + 0x73, 0x74, 0x61, 0x72, 0x74, 0x20, 0x62, 0x75, 0x6e, 0x64, 0x6c, 0x65, 0x66, 0x75, 0x6e, 0x63, + 0x74, 0x69, 0x6f, 0x6e, 0x20, 0x69, 0x6e, 0x20, 0x61, 0x20, 0x70, 0x61, 0x72, 0x64, 0x6f, 0x12, + 0xdf, 0x01, 0x0a, 0x14, 0x50, 0x52, 0x4f, 0x43, 0x45, 0x53, 0x53, 0x5f, 0x42, 0x55, 0x4e, 0x44, + 0x4c, 0x45, 0x5f, 0x4d, 0x53, 0x45, 0x43, 0x53, 0x10, 0x0d, 0x1a, 0xc4, 0x01, 0xd2, 0xa7, 0xa7, + 0x96, 0x06, 0xbd, 0x01, 0x0a, 0x38, 0x62, 0x65, 0x61, 0x6d, 0x3a, 0x6d, 0x65, 0x74, 0x72, 0x69, + 0x63, 0x3a, 0x70, 0x61, 0x72, 0x64, 0x6f, 0x5f, 0x65, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, + 0x6e, 0x5f, 0x74, 0x69, 0x6d, 0x65, 0x3a, 0x70, 0x72, 0x6f, 0x63, 0x65, 0x73, 0x73, 0x5f, 0x62, + 0x75, 0x6e, 0x64, 0x6c, 0x65, 0x5f, 0x6d, 0x73, 0x65, 0x63, 0x73, 0x3a, 0x76, 0x31, 0x12, 0x19, + 0x62, 0x65, 0x61, 0x6d, 0x3a, 0x6d, 0x65, 0x74, 0x72, 0x69, 0x63, 0x73, 0x3a, 0x73, 0x75, 0x6d, + 0x5f, 0x69, 0x6e, 0x74, 0x36, 0x34, 0x3a, 0x76, 0x31, 0x1a, 0x0a, 0x50, 0x54, 0x52, 0x41, 0x4e, + 0x53, 0x46, 0x4f, 0x52, 0x4d, 0x22, 0x5a, 0x0a, 0x0b, 0x64, 0x65, 0x73, 0x63, 0x72, 0x69, 0x70, + 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x4b, 0x54, 0x68, 0x65, 0x20, 0x74, 0x6f, 0x74, 0x61, 0x6c, 0x20, + 0x65, 0x73, 0x74, 0x69, 0x6d, 0x61, 0x74, 0x65, 0x64, 0x20, 0x65, 0x78, 0x65, 0x63, 0x75, 0x74, + 0x69, 0x6f, 0x6e, 0x20, 0x74, 0x69, 0x6d, 0x65, 0x20, 0x6f, 0x66, 0x20, 0x74, 0x68, 0x65, 0x20, + 0x70, 0x72, 0x6f, 0x63, 0x65, 0x73, 0x73, 0x20, 0x62, 0x75, 0x6e, 0x64, 0x6c, 0x65, 0x66, 0x75, + 0x6e, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x20, 0x69, 0x6e, 0x20, 0x61, 0x20, 0x70, 0x61, 0x72, 0x64, + 0x6f, 0x12, 0xdd, 0x01, 0x0a, 0x13, 0x46, 0x49, 0x4e, 0x49, 0x53, 0x48, 0x5f, 0x42, 0x55, 0x4e, + 0x44, 0x4c, 0x45, 0x5f, 0x4d, 0x53, 0x45, 0x43, 0x53, 0x10, 0x0e, 0x1a, 0xc3, 0x01, 0xd2, 0xa7, + 0xa7, 0x96, 0x06, 0xbc, 0x01, 0x0a, 0x37, 0x62, 0x65, 0x61, 0x6d, 0x3a, 0x6d, 0x65, 0x74, 0x72, + 0x69, 0x63, 0x3a, 0x70, 0x61, 0x72, 0x64, 0x6f, 0x5f, 0x65, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, + 0x6f, 0x6e, 0x5f, 0x74, 0x69, 0x6d, 0x65, 0x3a, 0x66, 0x69, 0x6e, 0x69, 0x73, 0x68, 0x5f, 0x62, + 0x75, 0x6e, 0x64, 0x6c, 0x65, 0x5f, 0x6d, 0x73, 0x65, 0x63, 0x73, 0x3a, 0x76, 0x31, 0x12, 0x19, + 0x62, 0x65, 0x61, 0x6d, 0x3a, 0x6d, 0x65, 0x74, 0x72, 0x69, 0x63, 0x73, 0x3a, 0x73, 0x75, 0x6d, + 0x5f, 0x69, 0x6e, 0x74, 0x36, 0x34, 0x3a, 0x76, 0x31, 0x1a, 0x0a, 0x50, 0x54, 0x52, 0x41, 0x4e, + 0x53, 0x46, 0x4f, 0x52, 0x4d, 0x22, 0x5a, 0x0a, 0x0b, 0x64, 0x65, 0x73, 0x63, 0x72, 0x69, 0x70, + 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x4b, 0x54, 0x68, 0x65, 0x20, 0x74, 0x6f, 0x74, 0x61, 0x6c, 0x20, + 0x65, 0x73, 0x74, 0x69, 0x6d, 0x61, 0x74, 0x65, 0x64, 0x20, 0x65, 0x78, 0x65, 0x63, 0x75, 0x74, + 0x69, 0x6f, 0x6e, 0x20, 0x74, 0x69, 0x6d, 0x65, 0x20, 0x6f, 0x66, 0x20, 0x74, 0x68, 0x65, 0x20, + 0x66, 0x69, 0x6e, 0x69, 0x73, 0x68, 0x20, 0x62, 0x75, 0x6e, 0x64, 0x6c, 0x65, 0x20, 0x66, 0x75, + 0x6e, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x20, 0x69, 0x6e, 0x20, 0x61, 0x20, 0x70, 0x61, 0x72, 0x64, + 0x6f, 0x12, 0xbb, 0x01, 0x0a, 0x0b, 0x54, 0x4f, 0x54, 0x41, 0x4c, 0x5f, 0x4d, 0x53, 0x45, 0x43, + 0x53, 0x10, 0x0f, 0x1a, 0xa9, 0x01, 0xd2, 0xa7, 0xa7, 0x96, 0x06, 0xa2, 0x01, 0x0a, 0x34, 0x62, + 0x65, 0x61, 0x6d, 0x3a, 0x6d, 0x65, 0x74, 0x72, 0x69, 0x63, 0x3a, 0x70, 0x74, 0x72, 0x61, 0x6e, + 0x73, 0x66, 0x6f, 0x72, 0x6d, 0x5f, 0x65, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x5f, + 0x74, 0x69, 0x6d, 0x65, 0x3a, 0x74, 0x6f, 0x74, 0x61, 0x6c, 0x5f, 0x6d, 0x73, 0x65, 0x63, 0x73, + 0x3a, 0x76, 0x31, 0x12, 0x19, 0x62, 0x65, 0x61, 0x6d, 0x3a, 0x6d, 0x65, 0x74, 0x72, 0x69, 0x63, + 0x73, 0x3a, 0x73, 0x75, 0x6d, 0x5f, 0x69, 0x6e, 0x74, 0x36, 0x34, 0x3a, 0x76, 0x31, 0x1a, 0x0a, + 0x50, 0x54, 0x52, 0x41, 0x4e, 0x53, 0x46, 0x4f, 0x52, 0x4d, 0x22, 0x43, 0x0a, 0x0b, 0x64, 0x65, + 0x73, 0x63, 0x72, 0x69, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x34, 0x54, 0x68, 0x65, 0x20, 0x74, + 0x6f, 0x74, 0x61, 0x6c, 0x20, 0x65, 0x73, 0x74, 0x69, 0x6d, 0x61, 0x74, 0x65, 0x64, 0x20, 0x65, + 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x20, 0x74, 0x69, 0x6d, 0x65, 0x20, 0x6f, 0x66, + 0x20, 0x74, 0x68, 0x65, 0x20, 0x70, 0x74, 0x72, 0x61, 0x6e, 0x73, 0x66, 0x6f, 0x72, 0x6d, 0x12, + 0x9f, 0x02, 0x0a, 0x0e, 0x57, 0x4f, 0x52, 0x4b, 0x5f, 0x52, 0x45, 0x4d, 0x41, 0x49, 0x4e, 0x49, + 0x4e, 0x47, 0x10, 0x10, 0x1a, 0x8a, 0x02, 0xd2, 0xa7, 0xa7, 0x96, 0x06, 0x83, 0x02, 0x0a, 0x2c, + 0x62, 0x65, 0x61, 0x6d, 0x3a, 0x6d, 0x65, 0x74, 0x72, 0x69, 0x63, 0x3a, 0x70, 0x74, 0x72, 0x61, + 0x6e, 0x73, 0x66, 0x6f, 0x72, 0x6d, 0x5f, 0x70, 0x72, 0x6f, 0x67, 0x72, 0x65, 0x73, 0x73, 0x3a, + 0x72, 0x65, 0x6d, 0x61, 0x69, 0x6e, 0x69, 0x6e, 0x67, 0x3a, 0x76, 0x31, 0x12, 0x18, 0x62, 0x65, + 0x61, 0x6d, 0x3a, 0x6d, 0x65, 0x74, 0x72, 0x69, 0x63, 0x73, 0x3a, 0x70, 0x72, 0x6f, 0x67, 0x72, + 0x65, 0x73, 0x73, 0x3a, 0x76, 0x31, 0x1a, 0x0a, 0x50, 0x54, 0x52, 0x41, 0x4e, 0x53, 0x46, 0x4f, + 0x52, 0x4d, 0x22, 0xac, 0x01, 0x0a, 0x0b, 0x64, 0x65, 0x73, 0x63, 0x72, 0x69, 0x70, 0x74, 0x69, + 0x6f, 0x6e, 0x12, 0x9c, 0x01, 0x54, 0x68, 0x65, 0x20, 0x72, 0x65, 0x6d, 0x61, 0x69, 0x6e, 0x69, + 0x6e, 0x67, 0x20, 0x61, 0x6d, 0x6f, 0x75, 0x6e, 0x74, 0x20, 0x6f, 0x66, 0x20, 0x77, 0x6f, 0x72, + 0x6b, 0x20, 0x66, 0x6f, 0x72, 0x20, 0x65, 0x61, 0x63, 0x68, 0x20, 0x61, 0x63, 0x74, 0x69, 0x76, + 0x65, 0x20, 0x65, 0x6c, 0x65, 0x6d, 0x65, 0x6e, 0x74, 0x2e, 0x20, 0x45, 0x61, 0x63, 0x68, 0x20, + 0x61, 0x63, 0x74, 0x69, 0x76, 0x65, 0x20, 0x65, 0x6c, 0x65, 0x6d, 0x65, 0x6e, 0x74, 0x20, 0x72, + 0x65, 0x70, 0x72, 0x65, 0x73, 0x65, 0x6e, 0x74, 0x73, 0x20, 0x61, 0x6e, 0x20, 0x69, 0x6e, 0x64, + 0x65, 0x70, 0x65, 0x6e, 0x64, 0x65, 0x6e, 0x74, 0x20, 0x61, 0x6d, 0x6f, 0x75, 0x6e, 0x74, 0x20, + 0x6f, 0x66, 0x20, 0x77, 0x6f, 0x72, 0x6b, 0x20, 0x6e, 0x6f, 0x74, 0x20, 0x73, 0x68, 0x61, 0x72, + 0x65, 0x64, 0x20, 0x77, 0x69, 0x74, 0x68, 0x20, 0x61, 0x6e, 0x79, 0x20, 0x6f, 0x74, 0x68, 0x65, + 0x72, 0x20, 0x61, 0x63, 0x74, 0x69, 0x76, 0x65, 0x20, 0x65, 0x6c, 0x65, 0x6d, 0x65, 0x6e, 0x74, + 0x2e, 0x12, 0x9f, 0x02, 0x0a, 0x0e, 0x57, 0x4f, 0x52, 0x4b, 0x5f, 0x43, 0x4f, 0x4d, 0x50, 0x4c, + 0x45, 0x54, 0x45, 0x44, 0x10, 0x11, 0x1a, 0x8a, 0x02, 0xd2, 0xa7, 0xa7, 0x96, 0x06, 0x83, 0x02, + 0x0a, 0x2c, 0x62, 0x65, 0x61, 0x6d, 0x3a, 0x6d, 0x65, 0x74, 0x72, 0x69, 0x63, 0x3a, 0x70, 0x74, + 0x72, 0x61, 0x6e, 0x73, 0x66, 0x6f, 0x72, 0x6d, 0x5f, 0x70, 0x72, 0x6f, 0x67, 0x72, 0x65, 0x73, + 0x73, 0x3a, 0x63, 0x6f, 0x6d, 0x70, 0x6c, 0x65, 0x74, 0x65, 0x64, 0x3a, 0x76, 0x31, 0x12, 0x18, + 0x62, 0x65, 0x61, 0x6d, 0x3a, 0x6d, 0x65, 0x74, 0x72, 0x69, 0x63, 0x73, 0x3a, 0x70, 0x72, 0x6f, + 0x67, 0x72, 0x65, 0x73, 0x73, 0x3a, 0x76, 0x31, 0x1a, 0x0a, 0x50, 0x54, 0x52, 0x41, 0x4e, 0x53, + 0x46, 0x4f, 0x52, 0x4d, 0x22, 0xac, 0x01, 0x0a, 0x0b, 0x64, 0x65, 0x73, 0x63, 0x72, 0x69, 0x70, + 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x9c, 0x01, 0x54, 0x68, 0x65, 0x20, 0x72, 0x65, 0x6d, 0x61, 0x69, + 0x6e, 0x69, 0x6e, 0x67, 0x20, 0x61, 0x6d, 0x6f, 0x75, 0x6e, 0x74, 0x20, 0x6f, 0x66, 0x20, 0x77, + 0x6f, 0x72, 0x6b, 0x20, 0x66, 0x6f, 0x72, 0x20, 0x65, 0x61, 0x63, 0x68, 0x20, 0x61, 0x63, 0x74, + 0x69, 0x76, 0x65, 0x20, 0x65, 0x6c, 0x65, 0x6d, 0x65, 0x6e, 0x74, 0x2e, 0x20, 0x45, 0x61, 0x63, + 0x68, 0x20, 0x61, 0x63, 0x74, 0x69, 0x76, 0x65, 0x20, 0x65, 0x6c, 0x65, 0x6d, 0x65, 0x6e, 0x74, + 0x20, 0x72, 0x65, 0x70, 0x72, 0x65, 0x73, 0x65, 0x6e, 0x74, 0x73, 0x20, 0x61, 0x6e, 0x20, 0x69, + 0x6e, 0x64, 0x65, 0x70, 0x65, 0x6e, 0x64, 0x65, 0x6e, 0x74, 0x20, 0x61, 0x6d, 0x6f, 0x75, 0x6e, + 0x74, 0x20, 0x6f, 0x66, 0x20, 0x77, 0x6f, 0x72, 0x6b, 0x20, 0x6e, 0x6f, 0x74, 0x20, 0x73, 0x68, + 0x61, 0x72, 0x65, 0x64, 0x20, 0x77, 0x69, 0x74, 0x68, 0x20, 0x61, 0x6e, 0x79, 0x20, 0x6f, 0x74, + 0x68, 0x65, 0x72, 0x20, 0x61, 0x63, 0x74, 0x69, 0x76, 0x65, 0x20, 0x65, 0x6c, 0x65, 0x6d, 0x65, + 0x6e, 0x74, 0x2e, 0x12, 0xa8, 0x01, 0x0a, 0x17, 0x44, 0x41, 0x54, 0x41, 0x5f, 0x43, 0x48, 0x41, + 0x4e, 0x4e, 0x45, 0x4c, 0x5f, 0x52, 0x45, 0x41, 0x44, 0x5f, 0x49, 0x4e, 0x44, 0x45, 0x58, 0x10, + 0x12, 0x1a, 0x8a, 0x01, 0xd2, 0xa7, 0xa7, 0x96, 0x06, 0x83, 0x01, 0x0a, 0x26, 0x62, 0x65, 0x61, + 0x6d, 0x3a, 0x6d, 0x65, 0x74, 0x72, 0x69, 0x63, 0x3a, 0x64, 0x61, 0x74, 0x61, 0x5f, 0x63, 0x68, + 0x61, 0x6e, 0x6e, 0x65, 0x6c, 0x3a, 0x72, 0x65, 0x61, 0x64, 0x5f, 0x69, 0x6e, 0x64, 0x65, 0x78, + 0x3a, 0x76, 0x31, 0x12, 0x19, 0x62, 0x65, 0x61, 0x6d, 0x3a, 0x6d, 0x65, 0x74, 0x72, 0x69, 0x63, + 0x73, 0x3a, 0x73, 0x75, 0x6d, 0x5f, 0x69, 0x6e, 0x74, 0x36, 0x34, 0x3a, 0x76, 0x31, 0x1a, 0x0a, + 0x50, 0x54, 0x52, 0x41, 0x4e, 0x53, 0x46, 0x4f, 0x52, 0x4d, 0x22, 0x32, 0x0a, 0x0b, 0x64, 0x65, + 0x73, 0x63, 0x72, 0x69, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x23, 0x54, 0x68, 0x65, 0x20, 0x72, + 0x65, 0x61, 0x64, 0x20, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x20, 0x6f, 0x66, 0x20, 0x74, 0x68, 0x65, + 0x20, 0x64, 0x61, 0x74, 0x61, 0x20, 0x63, 0x68, 0x61, 0x6e, 0x6e, 0x65, 0x6c, 0x2e, 0x12, 0x8a, + 0x02, 0x0a, 0x11, 0x41, 0x50, 0x49, 0x5f, 0x52, 0x45, 0x51, 0x55, 0x45, 0x53, 0x54, 0x5f, 0x43, + 0x4f, 0x55, 0x4e, 0x54, 0x10, 0x13, 0x1a, 0xf2, 0x01, 0xd2, 0xa7, 0xa7, 0x96, 0x06, 0xeb, 0x01, + 0x0a, 0x23, 0x62, 0x65, 0x61, 0x6d, 0x3a, 0x6d, 0x65, 0x74, 0x72, 0x69, 0x63, 0x3a, 0x69, 0x6f, + 0x3a, 0x61, 0x70, 0x69, 0x5f, 0x72, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x5f, 0x63, 0x6f, 0x75, + 0x6e, 0x74, 0x3a, 0x76, 0x31, 0x12, 0x19, 0x62, 0x65, 0x61, 0x6d, 0x3a, 0x6d, 0x65, 0x74, 0x72, + 0x69, 0x63, 0x73, 0x3a, 0x73, 0x75, 0x6d, 0x5f, 0x69, 0x6e, 0x74, 0x36, 0x34, 0x3a, 0x76, 0x31, + 0x1a, 0x07, 0x53, 0x45, 0x52, 0x56, 0x49, 0x43, 0x45, 0x1a, 0x06, 0x4d, 0x45, 0x54, 0x48, 0x4f, + 0x44, 0x1a, 0x08, 0x52, 0x45, 0x53, 0x4f, 0x55, 0x52, 0x43, 0x45, 0x1a, 0x0a, 0x50, 0x54, 0x52, + 0x41, 0x4e, 0x53, 0x46, 0x4f, 0x52, 0x4d, 0x1a, 0x06, 0x53, 0x54, 0x41, 0x54, 0x55, 0x53, 0x22, + 0x62, 0x0a, 0x0b, 0x64, 0x65, 0x73, 0x63, 0x72, 0x69, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x53, + 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x20, 0x63, 0x6f, 0x75, 0x6e, 0x74, 0x73, 0x20, 0x77, + 0x69, 0x74, 0x68, 0x20, 0x73, 0x74, 0x61, 0x74, 0x75, 0x73, 0x20, 0x6d, 0x61, 0x64, 0x65, 0x20, + 0x74, 0x6f, 0x20, 0x49, 0x4f, 0x20, 0x73, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, 0x20, 0x41, 0x50, + 0x49, 0x73, 0x20, 0x74, 0x6f, 0x20, 0x62, 0x61, 0x74, 0x63, 0x68, 0x20, 0x72, 0x65, 0x61, 0x64, + 0x20, 0x6f, 0x72, 0x20, 0x77, 0x72, 0x69, 0x74, 0x65, 0x20, 0x65, 0x6c, 0x65, 0x6d, 0x65, 0x6e, + 0x74, 0x73, 0x2e, 0x22, 0x16, 0x0a, 0x0e, 0x70, 0x72, 0x6f, 0x63, 0x65, 0x73, 0x73, 0x5f, 0x6d, + 0x65, 0x74, 0x72, 0x69, 0x63, 0x12, 0x04, 0x74, 0x72, 0x75, 0x65, 0x12, 0xb3, 0x02, 0x0a, 0x15, + 0x41, 0x50, 0x49, 0x5f, 0x52, 0x45, 0x51, 0x55, 0x45, 0x53, 0x54, 0x5f, 0x4c, 0x41, 0x54, 0x45, + 0x4e, 0x43, 0x49, 0x45, 0x53, 0x10, 0x14, 0x1a, 0x97, 0x02, 0xd2, 0xa7, 0xa7, 0x96, 0x06, 0x90, + 0x02, 0x0a, 0x27, 0x62, 0x65, 0x61, 0x6d, 0x3a, 0x6d, 0x65, 0x74, 0x72, 0x69, 0x63, 0x3a, 0x69, + 0x6f, 0x3a, 0x61, 0x70, 0x69, 0x5f, 0x72, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x5f, 0x6c, 0x61, + 0x74, 0x65, 0x6e, 0x63, 0x69, 0x65, 0x73, 0x3a, 0x76, 0x31, 0x12, 0x1f, 0x62, 0x65, 0x61, 0x6d, + 0x3a, 0x6d, 0x65, 0x74, 0x72, 0x69, 0x63, 0x73, 0x3a, 0x68, 0x69, 0x73, 0x74, 0x6f, 0x67, 0x72, + 0x61, 0x6d, 0x5f, 0x69, 0x6e, 0x74, 0x36, 0x34, 0x3a, 0x76, 0x31, 0x1a, 0x07, 0x53, 0x45, 0x52, + 0x56, 0x49, 0x43, 0x45, 0x1a, 0x06, 0x4d, 0x45, 0x54, 0x48, 0x4f, 0x44, 0x1a, 0x08, 0x52, 0x45, + 0x53, 0x4f, 0x55, 0x52, 0x43, 0x45, 0x1a, 0x0a, 0x50, 0x54, 0x52, 0x41, 0x4e, 0x53, 0x46, 0x4f, + 0x52, 0x4d, 0x22, 0x6e, 0x0a, 0x0b, 0x64, 0x65, 0x73, 0x63, 0x72, 0x69, 0x70, 0x74, 0x69, 0x6f, + 0x6e, 0x12, 0x5f, 0x48, 0x69, 0x73, 0x74, 0x6f, 0x67, 0x72, 0x61, 0x6d, 0x20, 0x63, 0x6f, 0x75, + 0x6e, 0x74, 0x73, 0x20, 0x66, 0x6f, 0x72, 0x20, 0x72, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x20, + 0x6c, 0x61, 0x74, 0x65, 0x6e, 0x63, 0x69, 0x65, 0x73, 0x20, 0x6d, 0x61, 0x64, 0x65, 0x20, 0x74, + 0x6f, 0x20, 0x49, 0x4f, 0x20, 0x73, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, 0x20, 0x41, 0x50, 0x49, + 0x73, 0x20, 0x74, 0x6f, 0x20, 0x62, 0x61, 0x74, 0x63, 0x68, 0x20, 0x72, 0x65, 0x61, 0x64, 0x20, + 0x6f, 0x72, 0x20, 0x77, 0x72, 0x69, 0x74, 0x65, 0x20, 0x65, 0x6c, 0x65, 0x6d, 0x65, 0x6e, 0x74, + 0x73, 0x2e, 0x22, 0x15, 0x0a, 0x05, 0x75, 0x6e, 0x69, 0x74, 0x73, 0x12, 0x0c, 0x4d, 0x69, 0x6c, + 0x6c, 0x69, 0x73, 0x65, 0x63, 0x6f, 0x6e, 0x64, 0x73, 0x22, 0x16, 0x0a, 0x0e, 0x70, 0x72, 0x6f, + 0x63, 0x65, 0x73, 0x73, 0x5f, 0x6d, 0x65, 0x74, 0x72, 0x69, 0x63, 0x12, 0x04, 0x74, 0x72, 0x75, + 0x65, 0x22, 0x2e, 0x0a, 0x18, 0x4d, 0x6f, 0x6e, 0x69, 0x74, 0x6f, 0x72, 0x69, 0x6e, 0x67, 0x49, + 0x6e, 0x66, 0x6f, 0x4c, 0x61, 0x62, 0x65, 0x6c, 0x50, 0x72, 0x6f, 0x70, 0x73, 0x12, 0x12, 0x0a, + 0x04, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x04, 0x6e, 0x61, 0x6d, + 0x65, 0x22, 0xcd, 0x0b, 0x0a, 0x0e, 0x4d, 0x6f, 0x6e, 0x69, 0x74, 0x6f, 0x72, 0x69, 0x6e, 0x67, + 0x49, 0x6e, 0x66, 0x6f, 0x12, 0x10, 0x0a, 0x03, 0x75, 0x72, 0x6e, 0x18, 0x01, 0x20, 0x01, 0x28, + 0x09, 0x52, 0x03, 0x75, 0x72, 0x6e, 0x12, 0x12, 0x0a, 0x04, 0x74, 0x79, 0x70, 0x65, 0x18, 0x02, + 0x20, 0x01, 0x28, 0x09, 0x52, 0x04, 0x74, 0x79, 0x70, 0x65, 0x12, 0x18, 0x0a, 0x07, 0x70, 0x61, + 0x79, 0x6c, 0x6f, 0x61, 0x64, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0c, 0x52, 0x07, 0x70, 0x61, 0x79, + 0x6c, 0x6f, 0x61, 0x64, 0x12, 0x55, 0x0a, 0x06, 0x6c, 0x61, 0x62, 0x65, 0x6c, 0x73, 0x18, 0x04, + 0x20, 0x03, 0x28, 0x0b, 0x32, 0x3d, 0x2e, 0x6f, 0x72, 0x67, 0x2e, 0x61, 0x70, 0x61, 0x63, 0x68, + 0x65, 0x2e, 0x62, 0x65, 0x61, 0x6d, 0x2e, 0x6d, 0x6f, 0x64, 0x65, 0x6c, 0x2e, 0x70, 0x69, 0x70, + 0x65, 0x6c, 0x69, 0x6e, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x4d, 0x6f, 0x6e, 0x69, 0x74, 0x6f, 0x72, + 0x69, 0x6e, 0x67, 0x49, 0x6e, 0x66, 0x6f, 0x2e, 0x4c, 0x61, 0x62, 0x65, 0x6c, 0x73, 0x45, 0x6e, + 0x74, 0x72, 0x79, 0x52, 0x06, 0x6c, 0x61, 0x62, 0x65, 0x6c, 0x73, 0x12, 0x39, 0x0a, 0x0a, 0x73, + 0x74, 0x61, 0x72, 0x74, 0x5f, 0x74, 0x69, 0x6d, 0x65, 0x18, 0x05, 0x20, 0x01, 0x28, 0x0b, 0x32, + 0x1a, 0x2e, 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75, + 0x66, 0x2e, 0x54, 0x69, 0x6d, 0x65, 0x73, 0x74, 0x61, 0x6d, 0x70, 0x52, 0x09, 0x73, 0x74, 0x61, + 0x72, 0x74, 0x54, 0x69, 0x6d, 0x65, 0x1a, 0x39, 0x0a, 0x0b, 0x4c, 0x61, 0x62, 0x65, 0x6c, 0x73, + 0x45, 0x6e, 0x74, 0x72, 0x79, 0x12, 0x10, 0x0a, 0x03, 0x6b, 0x65, 0x79, 0x18, 0x01, 0x20, 0x01, + 0x28, 0x09, 0x52, 0x03, 0x6b, 0x65, 0x79, 0x12, 0x14, 0x0a, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, + 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x3a, 0x02, 0x38, + 0x01, 0x22, 0xad, 0x09, 0x0a, 0x14, 0x4d, 0x6f, 0x6e, 0x69, 0x74, 0x6f, 0x72, 0x69, 0x6e, 0x67, + 0x49, 0x6e, 0x66, 0x6f, 0x4c, 0x61, 0x62, 0x65, 0x6c, 0x73, 0x12, 0x21, 0x0a, 0x09, 0x54, 0x52, + 0x41, 0x4e, 0x53, 0x46, 0x4f, 0x52, 0x4d, 0x10, 0x00, 0x1a, 0x12, 0xa2, 0xd4, 0xe0, 0xe5, 0x03, + 0x0c, 0x0a, 0x0a, 0x50, 0x54, 0x52, 0x41, 0x4e, 0x53, 0x46, 0x4f, 0x52, 0x4d, 0x12, 0x24, 0x0a, + 0x0b, 0x50, 0x43, 0x4f, 0x4c, 0x4c, 0x45, 0x43, 0x54, 0x49, 0x4f, 0x4e, 0x10, 0x01, 0x1a, 0x13, + 0xa2, 0xd4, 0xe0, 0xe5, 0x03, 0x0d, 0x0a, 0x0b, 0x50, 0x43, 0x4f, 0x4c, 0x4c, 0x45, 0x43, 0x54, + 0x49, 0x4f, 0x4e, 0x12, 0x32, 0x0a, 0x12, 0x57, 0x49, 0x4e, 0x44, 0x4f, 0x57, 0x49, 0x4e, 0x47, + 0x5f, 0x53, 0x54, 0x52, 0x41, 0x54, 0x45, 0x47, 0x59, 0x10, 0x02, 0x1a, 0x1a, 0xa2, 0xd4, 0xe0, + 0xe5, 0x03, 0x14, 0x0a, 0x12, 0x57, 0x49, 0x4e, 0x44, 0x4f, 0x57, 0x49, 0x4e, 0x47, 0x5f, 0x53, + 0x54, 0x52, 0x41, 0x54, 0x45, 0x47, 0x59, 0x12, 0x18, 0x0a, 0x05, 0x43, 0x4f, 0x44, 0x45, 0x52, + 0x10, 0x03, 0x1a, 0x0d, 0xa2, 0xd4, 0xe0, 0xe5, 0x03, 0x07, 0x0a, 0x05, 0x43, 0x4f, 0x44, 0x45, + 0x52, 0x12, 0x24, 0x0a, 0x0b, 0x45, 0x4e, 0x56, 0x49, 0x52, 0x4f, 0x4e, 0x4d, 0x45, 0x4e, 0x54, + 0x10, 0x04, 0x1a, 0x13, 0xa2, 0xd4, 0xe0, 0xe5, 0x03, 0x0d, 0x0a, 0x0b, 0x45, 0x4e, 0x56, 0x49, + 0x52, 0x4f, 0x4e, 0x4d, 0x45, 0x4e, 0x54, 0x12, 0x20, 0x0a, 0x09, 0x4e, 0x41, 0x4d, 0x45, 0x53, + 0x50, 0x41, 0x43, 0x45, 0x10, 0x05, 0x1a, 0x11, 0xa2, 0xd4, 0xe0, 0xe5, 0x03, 0x0b, 0x0a, 0x09, + 0x4e, 0x41, 0x4d, 0x45, 0x53, 0x50, 0x41, 0x43, 0x45, 0x12, 0x16, 0x0a, 0x04, 0x4e, 0x41, 0x4d, + 0x45, 0x10, 0x06, 0x1a, 0x0c, 0xa2, 0xd4, 0xe0, 0xe5, 0x03, 0x06, 0x0a, 0x04, 0x4e, 0x41, 0x4d, + 0x45, 0x12, 0x1c, 0x0a, 0x07, 0x53, 0x45, 0x52, 0x56, 0x49, 0x43, 0x45, 0x10, 0x07, 0x1a, 0x0f, + 0xa2, 0xd4, 0xe0, 0xe5, 0x03, 0x09, 0x0a, 0x07, 0x53, 0x45, 0x52, 0x56, 0x49, 0x43, 0x45, 0x12, + 0x1a, 0x0a, 0x06, 0x4d, 0x45, 0x54, 0x48, 0x4f, 0x44, 0x10, 0x08, 0x1a, 0x0e, 0xa2, 0xd4, 0xe0, + 0xe5, 0x03, 0x08, 0x0a, 0x06, 0x4d, 0x45, 0x54, 0x48, 0x4f, 0x44, 0x12, 0x1e, 0x0a, 0x08, 0x52, + 0x45, 0x53, 0x4f, 0x55, 0x52, 0x43, 0x45, 0x10, 0x09, 0x1a, 0x10, 0xa2, 0xd4, 0xe0, 0xe5, 0x03, + 0x0a, 0x0a, 0x08, 0x52, 0x45, 0x53, 0x4f, 0x55, 0x52, 0x43, 0x45, 0x12, 0x1a, 0x0a, 0x06, 0x53, + 0x54, 0x41, 0x54, 0x55, 0x53, 0x10, 0x0a, 0x1a, 0x0e, 0xa2, 0xd4, 0xe0, 0xe5, 0x03, 0x08, 0x0a, + 0x06, 0x53, 0x54, 0x41, 0x54, 0x55, 0x53, 0x12, 0x34, 0x0a, 0x13, 0x42, 0x49, 0x47, 0x51, 0x55, + 0x45, 0x52, 0x59, 0x5f, 0x50, 0x52, 0x4f, 0x4a, 0x45, 0x43, 0x54, 0x5f, 0x49, 0x44, 0x10, 0x0b, + 0x1a, 0x1b, 0xa2, 0xd4, 0xe0, 0xe5, 0x03, 0x15, 0x0a, 0x13, 0x42, 0x49, 0x47, 0x51, 0x55, 0x45, + 0x52, 0x59, 0x5f, 0x50, 0x52, 0x4f, 0x4a, 0x45, 0x43, 0x54, 0x5f, 0x49, 0x44, 0x12, 0x2e, 0x0a, + 0x10, 0x42, 0x49, 0x47, 0x51, 0x55, 0x45, 0x52, 0x59, 0x5f, 0x44, 0x41, 0x54, 0x41, 0x53, 0x45, + 0x54, 0x10, 0x0c, 0x1a, 0x18, 0xa2, 0xd4, 0xe0, 0xe5, 0x03, 0x12, 0x0a, 0x10, 0x42, 0x49, 0x47, + 0x51, 0x55, 0x45, 0x52, 0x59, 0x5f, 0x44, 0x41, 0x54, 0x41, 0x53, 0x45, 0x54, 0x12, 0x2a, 0x0a, + 0x0e, 0x42, 0x49, 0x47, 0x51, 0x55, 0x45, 0x52, 0x59, 0x5f, 0x54, 0x41, 0x42, 0x4c, 0x45, 0x10, + 0x0d, 0x1a, 0x16, 0xa2, 0xd4, 0xe0, 0xe5, 0x03, 0x10, 0x0a, 0x0e, 0x42, 0x49, 0x47, 0x51, 0x55, + 0x45, 0x52, 0x59, 0x5f, 0x54, 0x41, 0x42, 0x4c, 0x45, 0x12, 0x28, 0x0a, 0x0d, 0x42, 0x49, 0x47, + 0x51, 0x55, 0x45, 0x52, 0x59, 0x5f, 0x56, 0x49, 0x45, 0x57, 0x10, 0x0e, 0x1a, 0x15, 0xa2, 0xd4, + 0xe0, 0xe5, 0x03, 0x0f, 0x0a, 0x0d, 0x42, 0x49, 0x47, 0x51, 0x55, 0x45, 0x52, 0x59, 0x5f, 0x56, + 0x49, 0x45, 0x57, 0x12, 0x34, 0x0a, 0x13, 0x42, 0x49, 0x47, 0x51, 0x55, 0x45, 0x52, 0x59, 0x5f, + 0x51, 0x55, 0x45, 0x52, 0x59, 0x5f, 0x4e, 0x41, 0x4d, 0x45, 0x10, 0x0f, 0x1a, 0x1b, 0xa2, 0xd4, + 0xe0, 0xe5, 0x03, 0x15, 0x0a, 0x13, 0x42, 0x49, 0x47, 0x51, 0x55, 0x45, 0x52, 0x59, 0x5f, 0x51, + 0x55, 0x45, 0x52, 0x59, 0x5f, 0x4e, 0x41, 0x4d, 0x45, 0x12, 0x22, 0x0a, 0x0a, 0x47, 0x43, 0x53, + 0x5f, 0x42, 0x55, 0x43, 0x4b, 0x45, 0x54, 0x10, 0x10, 0x1a, 0x12, 0xa2, 0xd4, 0xe0, 0xe5, 0x03, + 0x0c, 0x0a, 0x0a, 0x47, 0x43, 0x53, 0x5f, 0x42, 0x55, 0x43, 0x4b, 0x45, 0x54, 0x12, 0x2a, 0x0a, + 0x0e, 0x47, 0x43, 0x53, 0x5f, 0x50, 0x52, 0x4f, 0x4a, 0x45, 0x43, 0x54, 0x5f, 0x49, 0x44, 0x10, + 0x11, 0x1a, 0x16, 0xa2, 0xd4, 0xe0, 0xe5, 0x03, 0x10, 0x0a, 0x0e, 0x47, 0x43, 0x53, 0x5f, 0x50, + 0x52, 0x4f, 0x4a, 0x45, 0x43, 0x54, 0x5f, 0x49, 0x44, 0x12, 0x30, 0x0a, 0x11, 0x44, 0x41, 0x54, + 0x41, 0x53, 0x54, 0x4f, 0x52, 0x45, 0x5f, 0x50, 0x52, 0x4f, 0x4a, 0x45, 0x43, 0x54, 0x10, 0x12, + 0x1a, 0x19, 0xa2, 0xd4, 0xe0, 0xe5, 0x03, 0x13, 0x0a, 0x11, 0x44, 0x41, 0x54, 0x41, 0x53, 0x54, + 0x4f, 0x52, 0x45, 0x5f, 0x50, 0x52, 0x4f, 0x4a, 0x45, 0x43, 0x54, 0x12, 0x34, 0x0a, 0x13, 0x44, + 0x41, 0x54, 0x41, 0x53, 0x54, 0x4f, 0x52, 0x45, 0x5f, 0x4e, 0x41, 0x4d, 0x45, 0x53, 0x50, 0x41, + 0x43, 0x45, 0x10, 0x13, 0x1a, 0x1b, 0xa2, 0xd4, 0xe0, 0xe5, 0x03, 0x15, 0x0a, 0x13, 0x44, 0x41, + 0x54, 0x41, 0x53, 0x54, 0x4f, 0x52, 0x45, 0x5f, 0x4e, 0x41, 0x4d, 0x45, 0x53, 0x50, 0x41, 0x43, + 0x45, 0x12, 0x34, 0x0a, 0x13, 0x42, 0x49, 0x47, 0x54, 0x41, 0x42, 0x4c, 0x45, 0x5f, 0x50, 0x52, + 0x4f, 0x4a, 0x45, 0x43, 0x54, 0x5f, 0x49, 0x44, 0x10, 0x14, 0x1a, 0x1b, 0xa2, 0xd4, 0xe0, 0xe5, + 0x03, 0x15, 0x0a, 0x13, 0x42, 0x49, 0x47, 0x54, 0x41, 0x42, 0x4c, 0x45, 0x5f, 0x50, 0x52, 0x4f, + 0x4a, 0x45, 0x43, 0x54, 0x5f, 0x49, 0x44, 0x12, 0x24, 0x0a, 0x0b, 0x49, 0x4e, 0x53, 0x54, 0x41, + 0x4e, 0x43, 0x45, 0x5f, 0x49, 0x44, 0x10, 0x15, 0x1a, 0x13, 0xa2, 0xd4, 0xe0, 0xe5, 0x03, 0x0d, + 0x0a, 0x0b, 0x49, 0x4e, 0x53, 0x54, 0x41, 0x4e, 0x43, 0x45, 0x5f, 0x49, 0x44, 0x12, 0x1e, 0x0a, + 0x08, 0x54, 0x41, 0x42, 0x4c, 0x45, 0x5f, 0x49, 0x44, 0x10, 0x16, 0x1a, 0x10, 0xa2, 0xd4, 0xe0, + 0xe5, 0x03, 0x0a, 0x0a, 0x08, 0x54, 0x41, 0x42, 0x4c, 0x45, 0x5f, 0x49, 0x44, 0x12, 0x32, 0x0a, + 0x12, 0x53, 0x50, 0x41, 0x4e, 0x4e, 0x45, 0x52, 0x5f, 0x50, 0x52, 0x4f, 0x4a, 0x45, 0x43, 0x54, + 0x5f, 0x49, 0x44, 0x10, 0x17, 0x1a, 0x1a, 0xa2, 0xd4, 0xe0, 0xe5, 0x03, 0x14, 0x0a, 0x12, 0x53, + 0x50, 0x41, 0x4e, 0x4e, 0x45, 0x52, 0x5f, 0x50, 0x52, 0x4f, 0x4a, 0x45, 0x43, 0x54, 0x5f, 0x49, + 0x44, 0x12, 0x34, 0x0a, 0x13, 0x53, 0x50, 0x41, 0x4e, 0x4e, 0x45, 0x52, 0x5f, 0x44, 0x41, 0x54, + 0x41, 0x42, 0x41, 0x53, 0x45, 0x5f, 0x49, 0x44, 0x10, 0x18, 0x1a, 0x1b, 0xa2, 0xd4, 0xe0, 0xe5, + 0x03, 0x15, 0x0a, 0x13, 0x53, 0x50, 0x41, 0x4e, 0x4e, 0x45, 0x52, 0x5f, 0x44, 0x41, 0x54, 0x41, + 0x42, 0x41, 0x53, 0x45, 0x5f, 0x49, 0x44, 0x12, 0x2e, 0x0a, 0x10, 0x53, 0x50, 0x41, 0x4e, 0x4e, + 0x45, 0x52, 0x5f, 0x54, 0x41, 0x42, 0x4c, 0x45, 0x5f, 0x49, 0x44, 0x10, 0x19, 0x1a, 0x18, 0xa2, + 0xd4, 0xe0, 0xe5, 0x03, 0x12, 0x0a, 0x10, 0x53, 0x50, 0x41, 0x4e, 0x4e, 0x45, 0x52, 0x5f, 0x54, + 0x41, 0x42, 0x4c, 0x45, 0x5f, 0x49, 0x44, 0x12, 0x34, 0x0a, 0x13, 0x53, 0x50, 0x41, 0x4e, 0x4e, + 0x45, 0x52, 0x5f, 0x49, 0x4e, 0x53, 0x54, 0x41, 0x4e, 0x43, 0x45, 0x5f, 0x49, 0x44, 0x10, 0x1a, + 0x1a, 0x1b, 0xa2, 0xd4, 0xe0, 0xe5, 0x03, 0x15, 0x0a, 0x13, 0x53, 0x50, 0x41, 0x4e, 0x4e, 0x45, + 0x52, 0x5f, 0x49, 0x4e, 0x53, 0x54, 0x41, 0x4e, 0x43, 0x45, 0x5f, 0x49, 0x44, 0x12, 0x32, 0x0a, + 0x12, 0x53, 0x50, 0x41, 0x4e, 0x4e, 0x45, 0x52, 0x5f, 0x51, 0x55, 0x45, 0x52, 0x59, 0x5f, 0x4e, + 0x41, 0x4d, 0x45, 0x10, 0x1b, 0x1a, 0x1a, 0xa2, 0xd4, 0xe0, 0xe5, 0x03, 0x14, 0x0a, 0x12, 0x53, + 0x50, 0x41, 0x4e, 0x4e, 0x45, 0x52, 0x5f, 0x51, 0x55, 0x45, 0x52, 0x59, 0x5f, 0x4e, 0x41, 0x4d, + 0x45, 0x22, 0xae, 0x06, 0x0a, 0x16, 0x4d, 0x6f, 0x6e, 0x69, 0x74, 0x6f, 0x72, 0x69, 0x6e, 0x67, + 0x49, 0x6e, 0x66, 0x6f, 0x54, 0x79, 0x70, 0x65, 0x55, 0x72, 0x6e, 0x73, 0x22, 0x93, 0x06, 0x0a, + 0x04, 0x45, 0x6e, 0x75, 0x6d, 0x12, 0x33, 0x0a, 0x0e, 0x53, 0x55, 0x4d, 0x5f, 0x49, 0x4e, 0x54, + 0x36, 0x34, 0x5f, 0x54, 0x59, 0x50, 0x45, 0x10, 0x00, 0x1a, 0x1f, 0xa2, 0xb4, 0xfa, 0xc2, 0x05, + 0x19, 0x62, 0x65, 0x61, 0x6d, 0x3a, 0x6d, 0x65, 0x74, 0x72, 0x69, 0x63, 0x73, 0x3a, 0x73, 0x75, + 0x6d, 0x5f, 0x69, 0x6e, 0x74, 0x36, 0x34, 0x3a, 0x76, 0x31, 0x12, 0x35, 0x0a, 0x0f, 0x53, 0x55, + 0x4d, 0x5f, 0x44, 0x4f, 0x55, 0x42, 0x4c, 0x45, 0x5f, 0x54, 0x59, 0x50, 0x45, 0x10, 0x01, 0x1a, + 0x20, 0xa2, 0xb4, 0xfa, 0xc2, 0x05, 0x1a, 0x62, 0x65, 0x61, 0x6d, 0x3a, 0x6d, 0x65, 0x74, 0x72, + 0x69, 0x63, 0x73, 0x3a, 0x73, 0x75, 0x6d, 0x5f, 0x64, 0x6f, 0x75, 0x62, 0x6c, 0x65, 0x3a, 0x76, + 0x31, 0x12, 0x45, 0x0a, 0x17, 0x44, 0x49, 0x53, 0x54, 0x52, 0x49, 0x42, 0x55, 0x54, 0x49, 0x4f, + 0x4e, 0x5f, 0x49, 0x4e, 0x54, 0x36, 0x34, 0x5f, 0x54, 0x59, 0x50, 0x45, 0x10, 0x02, 0x1a, 0x28, + 0xa2, 0xb4, 0xfa, 0xc2, 0x05, 0x22, 0x62, 0x65, 0x61, 0x6d, 0x3a, 0x6d, 0x65, 0x74, 0x72, 0x69, + 0x63, 0x73, 0x3a, 0x64, 0x69, 0x73, 0x74, 0x72, 0x69, 0x62, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x5f, + 0x69, 0x6e, 0x74, 0x36, 0x34, 0x3a, 0x76, 0x31, 0x12, 0x47, 0x0a, 0x18, 0x44, 0x49, 0x53, 0x54, + 0x52, 0x49, 0x42, 0x55, 0x54, 0x49, 0x4f, 0x4e, 0x5f, 0x44, 0x4f, 0x55, 0x42, 0x4c, 0x45, 0x5f, + 0x54, 0x59, 0x50, 0x45, 0x10, 0x03, 0x1a, 0x29, 0xa2, 0xb4, 0xfa, 0xc2, 0x05, 0x23, 0x62, 0x65, + 0x61, 0x6d, 0x3a, 0x6d, 0x65, 0x74, 0x72, 0x69, 0x63, 0x73, 0x3a, 0x64, 0x69, 0x73, 0x74, 0x72, + 0x69, 0x62, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x64, 0x6f, 0x75, 0x62, 0x6c, 0x65, 0x3a, 0x76, + 0x31, 0x12, 0x39, 0x0a, 0x11, 0x4c, 0x41, 0x54, 0x45, 0x53, 0x54, 0x5f, 0x49, 0x4e, 0x54, 0x36, + 0x34, 0x5f, 0x54, 0x59, 0x50, 0x45, 0x10, 0x04, 0x1a, 0x22, 0xa2, 0xb4, 0xfa, 0xc2, 0x05, 0x1c, + 0x62, 0x65, 0x61, 0x6d, 0x3a, 0x6d, 0x65, 0x74, 0x72, 0x69, 0x63, 0x73, 0x3a, 0x6c, 0x61, 0x74, + 0x65, 0x73, 0x74, 0x5f, 0x69, 0x6e, 0x74, 0x36, 0x34, 0x3a, 0x76, 0x31, 0x12, 0x3b, 0x0a, 0x12, + 0x4c, 0x41, 0x54, 0x45, 0x53, 0x54, 0x5f, 0x44, 0x4f, 0x55, 0x42, 0x4c, 0x45, 0x5f, 0x54, 0x59, + 0x50, 0x45, 0x10, 0x05, 0x1a, 0x23, 0xa2, 0xb4, 0xfa, 0xc2, 0x05, 0x1d, 0x62, 0x65, 0x61, 0x6d, + 0x3a, 0x6d, 0x65, 0x74, 0x72, 0x69, 0x63, 0x73, 0x3a, 0x6c, 0x61, 0x74, 0x65, 0x73, 0x74, 0x5f, + 0x64, 0x6f, 0x75, 0x62, 0x6c, 0x65, 0x3a, 0x76, 0x31, 0x12, 0x37, 0x0a, 0x10, 0x54, 0x4f, 0x50, + 0x5f, 0x4e, 0x5f, 0x49, 0x4e, 0x54, 0x36, 0x34, 0x5f, 0x54, 0x59, 0x50, 0x45, 0x10, 0x06, 0x1a, + 0x21, 0xa2, 0xb4, 0xfa, 0xc2, 0x05, 0x1b, 0x62, 0x65, 0x61, 0x6d, 0x3a, 0x6d, 0x65, 0x74, 0x72, + 0x69, 0x63, 0x73, 0x3a, 0x74, 0x6f, 0x70, 0x5f, 0x6e, 0x5f, 0x69, 0x6e, 0x74, 0x36, 0x34, 0x3a, + 0x76, 0x31, 0x12, 0x39, 0x0a, 0x11, 0x54, 0x4f, 0x50, 0x5f, 0x4e, 0x5f, 0x44, 0x4f, 0x55, 0x42, + 0x4c, 0x45, 0x5f, 0x54, 0x59, 0x50, 0x45, 0x10, 0x07, 0x1a, 0x22, 0xa2, 0xb4, 0xfa, 0xc2, 0x05, + 0x1c, 0x62, 0x65, 0x61, 0x6d, 0x3a, 0x6d, 0x65, 0x74, 0x72, 0x69, 0x63, 0x73, 0x3a, 0x74, 0x6f, + 0x70, 0x5f, 0x6e, 0x5f, 0x64, 0x6f, 0x75, 0x62, 0x6c, 0x65, 0x3a, 0x76, 0x31, 0x12, 0x3d, 0x0a, + 0x13, 0x42, 0x4f, 0x54, 0x54, 0x4f, 0x4d, 0x5f, 0x4e, 0x5f, 0x49, 0x4e, 0x54, 0x36, 0x34, 0x5f, + 0x54, 0x59, 0x50, 0x45, 0x10, 0x08, 0x1a, 0x24, 0xa2, 0xb4, 0xfa, 0xc2, 0x05, 0x1e, 0x62, 0x65, + 0x61, 0x6d, 0x3a, 0x6d, 0x65, 0x74, 0x72, 0x69, 0x63, 0x73, 0x3a, 0x62, 0x6f, 0x74, 0x74, 0x6f, + 0x6d, 0x5f, 0x6e, 0x5f, 0x69, 0x6e, 0x74, 0x36, 0x34, 0x3a, 0x76, 0x31, 0x12, 0x3f, 0x0a, 0x14, + 0x42, 0x4f, 0x54, 0x54, 0x4f, 0x4d, 0x5f, 0x4e, 0x5f, 0x44, 0x4f, 0x55, 0x42, 0x4c, 0x45, 0x5f, + 0x54, 0x59, 0x50, 0x45, 0x10, 0x09, 0x1a, 0x25, 0xa2, 0xb4, 0xfa, 0xc2, 0x05, 0x1f, 0x62, 0x65, + 0x61, 0x6d, 0x3a, 0x6d, 0x65, 0x74, 0x72, 0x69, 0x63, 0x73, 0x3a, 0x62, 0x6f, 0x74, 0x74, 0x6f, + 0x6d, 0x5f, 0x6e, 0x5f, 0x64, 0x6f, 0x75, 0x62, 0x6c, 0x65, 0x3a, 0x76, 0x31, 0x12, 0x31, 0x0a, + 0x0d, 0x50, 0x52, 0x4f, 0x47, 0x52, 0x45, 0x53, 0x53, 0x5f, 0x54, 0x59, 0x50, 0x45, 0x10, 0x0a, + 0x1a, 0x1e, 0xa2, 0xb4, 0xfa, 0xc2, 0x05, 0x18, 0x62, 0x65, 0x61, 0x6d, 0x3a, 0x6d, 0x65, 0x74, + 0x72, 0x69, 0x63, 0x73, 0x3a, 0x70, 0x72, 0x6f, 0x67, 0x72, 0x65, 0x73, 0x73, 0x3a, 0x76, 0x31, + 0x12, 0x35, 0x0a, 0x0f, 0x53, 0x45, 0x54, 0x5f, 0x53, 0x54, 0x52, 0x49, 0x4e, 0x47, 0x5f, 0x54, + 0x59, 0x50, 0x45, 0x10, 0x0b, 0x1a, 0x20, 0xa2, 0xb4, 0xfa, 0xc2, 0x05, 0x1a, 0x62, 0x65, 0x61, + 0x6d, 0x3a, 0x6d, 0x65, 0x74, 0x72, 0x69, 0x63, 0x73, 0x3a, 0x73, 0x65, 0x74, 0x5f, 0x73, 0x74, + 0x72, 0x69, 0x6e, 0x67, 0x3a, 0x76, 0x31, 0x12, 0x39, 0x0a, 0x11, 0x42, 0x4f, 0x55, 0x4e, 0x44, + 0x45, 0x44, 0x5f, 0x54, 0x52, 0x49, 0x45, 0x5f, 0x54, 0x59, 0x50, 0x45, 0x10, 0x0c, 0x1a, 0x22, + 0xa2, 0xb4, 0xfa, 0xc2, 0x05, 0x1c, 0x62, 0x65, 0x61, 0x6d, 0x3a, 0x6d, 0x65, 0x74, 0x72, 0x69, + 0x63, 0x73, 0x3a, 0x62, 0x6f, 0x75, 0x6e, 0x64, 0x65, 0x64, 0x5f, 0x74, 0x72, 0x69, 0x65, 0x3a, + 0x76, 0x31, 0x22, 0xfe, 0x01, 0x0a, 0x0f, 0x42, 0x6f, 0x75, 0x6e, 0x64, 0x65, 0x64, 0x54, 0x72, + 0x69, 0x65, 0x4e, 0x6f, 0x64, 0x65, 0x12, 0x1c, 0x0a, 0x09, 0x74, 0x72, 0x75, 0x6e, 0x63, 0x61, + 0x74, 0x65, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x08, 0x52, 0x09, 0x74, 0x72, 0x75, 0x6e, 0x63, + 0x61, 0x74, 0x65, 0x64, 0x12, 0x5c, 0x0a, 0x08, 0x63, 0x68, 0x69, 0x6c, 0x64, 0x72, 0x65, 0x6e, + 0x18, 0x02, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x40, 0x2e, 0x6f, 0x72, 0x67, 0x2e, 0x61, 0x70, 0x61, + 0x63, 0x68, 0x65, 0x2e, 0x62, 0x65, 0x61, 0x6d, 0x2e, 0x6d, 0x6f, 0x64, 0x65, 0x6c, 0x2e, 0x70, + 0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x42, 0x6f, 0x75, 0x6e, 0x64, + 0x65, 0x64, 0x54, 0x72, 0x69, 0x65, 0x4e, 0x6f, 0x64, 0x65, 0x2e, 0x43, 0x68, 0x69, 0x6c, 0x64, + 0x72, 0x65, 0x6e, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x52, 0x08, 0x63, 0x68, 0x69, 0x6c, 0x64, 0x72, + 0x65, 0x6e, 0x1a, 0x6f, 0x0a, 0x0d, 0x43, 0x68, 0x69, 0x6c, 0x64, 0x72, 0x65, 0x6e, 0x45, 0x6e, + 0x74, 0x72, 0x79, 0x12, 0x10, 0x0a, 0x03, 0x6b, 0x65, 0x79, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, + 0x52, 0x03, 0x6b, 0x65, 0x79, 0x12, 0x48, 0x0a, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x18, 0x02, + 0x20, 0x01, 0x28, 0x0b, 0x32, 0x32, 0x2e, 0x6f, 0x72, 0x67, 0x2e, 0x61, 0x70, 0x61, 0x63, 0x68, + 0x65, 0x2e, 0x62, 0x65, 0x61, 0x6d, 0x2e, 0x6d, 0x6f, 0x64, 0x65, 0x6c, 0x2e, 0x70, 0x69, 0x70, + 0x65, 0x6c, 0x69, 0x6e, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x42, 0x6f, 0x75, 0x6e, 0x64, 0x65, 0x64, + 0x54, 0x72, 0x69, 0x65, 0x4e, 0x6f, 0x64, 0x65, 0x52, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x3a, + 0x02, 0x38, 0x01, 0x22, 0x89, 0x01, 0x0a, 0x0b, 0x42, 0x6f, 0x75, 0x6e, 0x64, 0x65, 0x64, 0x54, + 0x72, 0x69, 0x65, 0x12, 0x14, 0x0a, 0x05, 0x62, 0x6f, 0x75, 0x6e, 0x64, 0x18, 0x01, 0x20, 0x01, + 0x28, 0x05, 0x52, 0x05, 0x62, 0x6f, 0x75, 0x6e, 0x64, 0x12, 0x46, 0x0a, 0x04, 0x72, 0x6f, 0x6f, + 0x74, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x32, 0x2e, 0x6f, 0x72, 0x67, 0x2e, 0x61, 0x70, + 0x61, 0x63, 0x68, 0x65, 0x2e, 0x62, 0x65, 0x61, 0x6d, 0x2e, 0x6d, 0x6f, 0x64, 0x65, 0x6c, 0x2e, + 0x70, 0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x42, 0x6f, 0x75, 0x6e, + 0x64, 0x65, 0x64, 0x54, 0x72, 0x69, 0x65, 0x4e, 0x6f, 0x64, 0x65, 0x52, 0x04, 0x72, 0x6f, 0x6f, + 0x74, 0x12, 0x1c, 0x0a, 0x09, 0x73, 0x69, 0x6e, 0x67, 0x6c, 0x65, 0x74, 0x6f, 0x6e, 0x18, 0x03, + 0x20, 0x03, 0x28, 0x09, 0x52, 0x09, 0x73, 0x69, 0x6e, 0x67, 0x6c, 0x65, 0x74, 0x6f, 0x6e, 0x3a, + 0x82, 0x01, 0x0a, 0x0b, 0x6c, 0x61, 0x62, 0x65, 0x6c, 0x5f, 0x70, 0x72, 0x6f, 0x70, 0x73, 0x12, + 0x21, 0x2e, 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75, + 0x66, 0x2e, 0x45, 0x6e, 0x75, 0x6d, 0x56, 0x61, 0x6c, 0x75, 0x65, 0x4f, 0x70, 0x74, 0x69, 0x6f, + 0x6e, 0x73, 0x18, 0xc4, 0x8a, 0xdc, 0x3c, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x3b, 0x2e, 0x6f, 0x72, + 0x67, 0x2e, 0x61, 0x70, 0x61, 0x63, 0x68, 0x65, 0x2e, 0x62, 0x65, 0x61, 0x6d, 0x2e, 0x6d, 0x6f, + 0x64, 0x65, 0x6c, 0x2e, 0x70, 0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e, 0x65, 0x2e, 0x76, 0x31, 0x2e, + 0x4d, 0x6f, 0x6e, 0x69, 0x74, 0x6f, 0x72, 0x69, 0x6e, 0x67, 0x49, 0x6e, 0x66, 0x6f, 0x4c, 0x61, + 0x62, 0x65, 0x6c, 0x50, 0x72, 0x6f, 0x70, 0x73, 0x52, 0x0a, 0x6c, 0x61, 0x62, 0x65, 0x6c, 0x50, + 0x72, 0x6f, 0x70, 0x73, 0x3a, 0x8d, 0x01, 0x0a, 0x14, 0x6d, 0x6f, 0x6e, 0x69, 0x74, 0x6f, 0x72, + 0x69, 0x6e, 0x67, 0x5f, 0x69, 0x6e, 0x66, 0x6f, 0x5f, 0x73, 0x70, 0x65, 0x63, 0x12, 0x21, 0x2e, + 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66, 0x2e, + 0x45, 0x6e, 0x75, 0x6d, 0x56, 0x61, 0x6c, 0x75, 0x65, 0x4f, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x73, + 0x18, 0xfa, 0xf4, 0xe4, 0x62, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x35, 0x2e, 0x6f, 0x72, 0x67, 0x2e, + 0x61, 0x70, 0x61, 0x63, 0x68, 0x65, 0x2e, 0x62, 0x65, 0x61, 0x6d, 0x2e, 0x6d, 0x6f, 0x64, 0x65, + 0x6c, 0x2e, 0x70, 0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x4d, 0x6f, + 0x6e, 0x69, 0x74, 0x6f, 0x72, 0x69, 0x6e, 0x67, 0x49, 0x6e, 0x66, 0x6f, 0x53, 0x70, 0x65, 0x63, + 0x52, 0x12, 0x6d, 0x6f, 0x6e, 0x69, 0x74, 0x6f, 0x72, 0x69, 0x6e, 0x67, 0x49, 0x6e, 0x66, 0x6f, + 0x53, 0x70, 0x65, 0x63, 0x42, 0x79, 0x0a, 0x21, 0x6f, 0x72, 0x67, 0x2e, 0x61, 0x70, 0x61, 0x63, + 0x68, 0x65, 0x2e, 0x62, 0x65, 0x61, 0x6d, 0x2e, 0x6d, 0x6f, 0x64, 0x65, 0x6c, 0x2e, 0x70, 0x69, + 0x70, 0x65, 0x6c, 0x69, 0x6e, 0x65, 0x2e, 0x76, 0x31, 0x42, 0x0a, 0x4d, 0x65, 0x74, 0x72, 0x69, + 0x63, 0x73, 0x41, 0x70, 0x69, 0x5a, 0x48, 0x67, 0x69, 0x74, 0x68, 0x75, 0x62, 0x2e, 0x63, 0x6f, + 0x6d, 0x2f, 0x61, 0x70, 0x61, 0x63, 0x68, 0x65, 0x2f, 0x62, 0x65, 0x61, 0x6d, 0x2f, 0x73, 0x64, + 0x6b, 0x73, 0x2f, 0x76, 0x32, 0x2f, 0x67, 0x6f, 0x2f, 0x70, 0x6b, 0x67, 0x2f, 0x62, 0x65, 0x61, + 0x6d, 0x2f, 0x6d, 0x6f, 0x64, 0x65, 0x6c, 0x2f, 0x70, 0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e, 0x65, + 0x5f, 0x76, 0x31, 0x3b, 0x70, 0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e, 0x65, 0x5f, 0x76, 0x31, 0x62, + 0x06, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x33, +} + +var file_org_apache_beam_model_pipeline_v1_metrics_proto_enumTypes = make([]protoimpl.EnumInfo, 3) +var file_org_apache_beam_model_pipeline_v1_metrics_proto_msgTypes = make([]protoimpl.MessageInfo, 10) +var file_org_apache_beam_model_pipeline_v1_metrics_proto_goTypes = []any{ + (MonitoringInfoSpecs_Enum)(0), // 0: org.apache.beam.model.pipeline.v1.MonitoringInfoSpecs.Enum + (MonitoringInfo_MonitoringInfoLabels)(0), // 1: org.apache.beam.model.pipeline.v1.MonitoringInfo.MonitoringInfoLabels + (MonitoringInfoTypeUrns_Enum)(0), // 2: org.apache.beam.model.pipeline.v1.MonitoringInfoTypeUrns.Enum + (*MonitoringInfoSpec)(nil), // 3: org.apache.beam.model.pipeline.v1.MonitoringInfoSpec + (*Annotation)(nil), // 4: org.apache.beam.model.pipeline.v1.Annotation + (*MonitoringInfoSpecs)(nil), // 5: org.apache.beam.model.pipeline.v1.MonitoringInfoSpecs + (*MonitoringInfoLabelProps)(nil), // 6: org.apache.beam.model.pipeline.v1.MonitoringInfoLabelProps + (*MonitoringInfo)(nil), // 7: org.apache.beam.model.pipeline.v1.MonitoringInfo + (*MonitoringInfoTypeUrns)(nil), // 8: org.apache.beam.model.pipeline.v1.MonitoringInfoTypeUrns + (*BoundedTrieNode)(nil), // 9: org.apache.beam.model.pipeline.v1.BoundedTrieNode + (*BoundedTrie)(nil), // 10: org.apache.beam.model.pipeline.v1.BoundedTrie + nil, // 11: org.apache.beam.model.pipeline.v1.MonitoringInfo.LabelsEntry + nil, // 12: org.apache.beam.model.pipeline.v1.BoundedTrieNode.ChildrenEntry + (*timestamppb.Timestamp)(nil), // 13: google.protobuf.Timestamp + (*descriptorpb.EnumValueOptions)(nil), // 14: google.protobuf.EnumValueOptions +} +var file_org_apache_beam_model_pipeline_v1_metrics_proto_depIdxs = []int32{ + 4, // 0: org.apache.beam.model.pipeline.v1.MonitoringInfoSpec.annotations:type_name -> org.apache.beam.model.pipeline.v1.Annotation + 11, // 1: org.apache.beam.model.pipeline.v1.MonitoringInfo.labels:type_name -> org.apache.beam.model.pipeline.v1.MonitoringInfo.LabelsEntry + 13, // 2: org.apache.beam.model.pipeline.v1.MonitoringInfo.start_time:type_name -> google.protobuf.Timestamp + 12, // 3: org.apache.beam.model.pipeline.v1.BoundedTrieNode.children:type_name -> org.apache.beam.model.pipeline.v1.BoundedTrieNode.ChildrenEntry + 9, // 4: org.apache.beam.model.pipeline.v1.BoundedTrie.root:type_name -> org.apache.beam.model.pipeline.v1.BoundedTrieNode + 9, // 5: org.apache.beam.model.pipeline.v1.BoundedTrieNode.ChildrenEntry.value:type_name -> org.apache.beam.model.pipeline.v1.BoundedTrieNode + 14, // 6: org.apache.beam.model.pipeline.v1.label_props:extendee -> google.protobuf.EnumValueOptions + 14, // 7: org.apache.beam.model.pipeline.v1.monitoring_info_spec:extendee -> google.protobuf.EnumValueOptions + 6, // 8: org.apache.beam.model.pipeline.v1.label_props:type_name -> org.apache.beam.model.pipeline.v1.MonitoringInfoLabelProps + 3, // 9: org.apache.beam.model.pipeline.v1.monitoring_info_spec:type_name -> org.apache.beam.model.pipeline.v1.MonitoringInfoSpec + 10, // [10:10] is the sub-list for method output_type + 10, // [10:10] is the sub-list for method input_type + 8, // [8:10] is the sub-list for extension type_name + 6, // [6:8] is the sub-list for extension extendee + 0, // [0:6] is the sub-list for field type_name +} + +func init() { file_org_apache_beam_model_pipeline_v1_metrics_proto_init() } +func file_org_apache_beam_model_pipeline_v1_metrics_proto_init() { + if File_org_apache_beam_model_pipeline_v1_metrics_proto != nil { + return + } + file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_init() + type x struct{} + out := protoimpl.TypeBuilder{ + File: protoimpl.DescBuilder{ + GoPackagePath: reflect.TypeOf(x{}).PkgPath(), + RawDescriptor: file_org_apache_beam_model_pipeline_v1_metrics_proto_rawDesc, + NumEnums: 3, + NumMessages: 10, + NumExtensions: 2, + NumServices: 0, + }, + GoTypes: file_org_apache_beam_model_pipeline_v1_metrics_proto_goTypes, + DependencyIndexes: file_org_apache_beam_model_pipeline_v1_metrics_proto_depIdxs, + EnumInfos: file_org_apache_beam_model_pipeline_v1_metrics_proto_enumTypes, + MessageInfos: file_org_apache_beam_model_pipeline_v1_metrics_proto_msgTypes, + ExtensionInfos: file_org_apache_beam_model_pipeline_v1_metrics_proto_extTypes, + }.Build() + File_org_apache_beam_model_pipeline_v1_metrics_proto = out.File + file_org_apache_beam_model_pipeline_v1_metrics_proto_rawDesc = nil + file_org_apache_beam_model_pipeline_v1_metrics_proto_goTypes = nil + file_org_apache_beam_model_pipeline_v1_metrics_proto_depIdxs = nil +} diff --git a/sdks/go/pkg/beam/model/pipeline_v1/schema.pb.go b/sdks/go/pkg/beam/model/pipeline_v1/schema.pb.go index 757842cc3806..fa9f54058e91 100644 --- a/sdks/go/pkg/beam/model/pipeline_v1/schema.pb.go +++ b/sdks/go/pkg/beam/model/pipeline_v1/schema.pb.go @@ -23,17 +23,18 @@ // Code generated by protoc-gen-go. DO NOT EDIT. // versions: -// protoc-gen-go v1.27.1 +// protoc-gen-go v1.36.0 // protoc v5.27.3 // source: org/apache/beam/model/pipeline/v1/schema.proto +//go:build !protoopaque + package pipeline_v1 import ( protoreflect "google.golang.org/protobuf/reflect/protoreflect" protoimpl "google.golang.org/protobuf/runtime/protoimpl" reflect "reflect" - sync "sync" ) const ( @@ -108,11 +109,6 @@ func (x AtomicType) Number() protoreflect.EnumNumber { return protoreflect.EnumNumber(x) } -// Deprecated: Use AtomicType.Descriptor instead. -func (AtomicType) EnumDescriptor() ([]byte, []int) { - return file_org_apache_beam_model_pipeline_v1_schema_proto_rawDescGZIP(), []int{0} -} - type LogicalTypes_Enum int32 const ( @@ -208,16 +204,8 @@ func (x LogicalTypes_Enum) Number() protoreflect.EnumNumber { return protoreflect.EnumNumber(x) } -// Deprecated: Use LogicalTypes_Enum.Descriptor instead. -func (LogicalTypes_Enum) EnumDescriptor() ([]byte, []int) { - return file_org_apache_beam_model_pipeline_v1_schema_proto_rawDescGZIP(), []int{8, 0} -} - type Schema struct { - state protoimpl.MessageState - sizeCache protoimpl.SizeCache - unknownFields protoimpl.UnknownFields - + state protoimpl.MessageState `protogen:"hybrid.v1"` // List of fields for this schema. Two fields may not share a name. Fields []*Field `protobuf:"bytes,1,rep,name=fields,proto3" json:"fields,omitempty"` // REQUIRED. An RFC 4122 UUID. @@ -225,15 +213,15 @@ type Schema struct { Options []*Option `protobuf:"bytes,3,rep,name=options,proto3" json:"options,omitempty"` // Indicates that encoding positions have been overridden. EncodingPositionsSet bool `protobuf:"varint,4,opt,name=encoding_positions_set,json=encodingPositionsSet,proto3" json:"encoding_positions_set,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache } func (x *Schema) Reset() { *x = Schema{} - if protoimpl.UnsafeEnabled { - mi := &file_org_apache_beam_model_pipeline_v1_schema_proto_msgTypes[0] - ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) - ms.StoreMessageInfo(mi) - } + mi := &file_org_apache_beam_model_pipeline_v1_schema_proto_msgTypes[0] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) } func (x *Schema) String() string { @@ -244,7 +232,7 @@ func (*Schema) ProtoMessage() {} func (x *Schema) ProtoReflect() protoreflect.Message { mi := &file_org_apache_beam_model_pipeline_v1_schema_proto_msgTypes[0] - if protoimpl.UnsafeEnabled && x != nil { + if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { ms.StoreMessageInfo(mi) @@ -254,11 +242,6 @@ func (x *Schema) ProtoReflect() protoreflect.Message { return mi.MessageOf(x) } -// Deprecated: Use Schema.ProtoReflect.Descriptor instead. -func (*Schema) Descriptor() ([]byte, []int) { - return file_org_apache_beam_model_pipeline_v1_schema_proto_rawDescGZIP(), []int{0} -} - func (x *Schema) GetFields() []*Field { if x != nil { return x.Fields @@ -287,11 +270,47 @@ func (x *Schema) GetEncodingPositionsSet() bool { return false } -type Field struct { - state protoimpl.MessageState - sizeCache protoimpl.SizeCache - unknownFields protoimpl.UnknownFields +func (x *Schema) SetFields(v []*Field) { + x.Fields = v +} + +func (x *Schema) SetId(v string) { + x.Id = v +} + +func (x *Schema) SetOptions(v []*Option) { + x.Options = v +} + +func (x *Schema) SetEncodingPositionsSet(v bool) { + x.EncodingPositionsSet = v +} + +type Schema_builder struct { + _ [0]func() // Prevents comparability and use of unkeyed literals for the builder. + + // List of fields for this schema. Two fields may not share a name. + Fields []*Field + // REQUIRED. An RFC 4122 UUID. + Id string + Options []*Option + // Indicates that encoding positions have been overridden. + EncodingPositionsSet bool +} + +func (b0 Schema_builder) Build() *Schema { + m0 := &Schema{} + b, x := &b0, m0 + _, _ = b, x + x.Fields = b.Fields + x.Id = b.Id + x.Options = b.Options + x.EncodingPositionsSet = b.EncodingPositionsSet + return m0 +} +type Field struct { + state protoimpl.MessageState `protogen:"hybrid.v1"` // REQUIRED. Name of this field within the schema. Name string `protobuf:"bytes,1,opt,name=name,proto3" json:"name,omitempty"` // OPTIONAL. Human readable description of this field, such as the query that generated it. @@ -307,15 +326,15 @@ type Field struct { // defined, otherwise this field is ignored. EncodingPosition int32 `protobuf:"varint,5,opt,name=encoding_position,json=encodingPosition,proto3" json:"encoding_position,omitempty"` Options []*Option `protobuf:"bytes,6,rep,name=options,proto3" json:"options,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache } func (x *Field) Reset() { *x = Field{} - if protoimpl.UnsafeEnabled { - mi := &file_org_apache_beam_model_pipeline_v1_schema_proto_msgTypes[1] - ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) - ms.StoreMessageInfo(mi) - } + mi := &file_org_apache_beam_model_pipeline_v1_schema_proto_msgTypes[1] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) } func (x *Field) String() string { @@ -326,7 +345,7 @@ func (*Field) ProtoMessage() {} func (x *Field) ProtoReflect() protoreflect.Message { mi := &file_org_apache_beam_model_pipeline_v1_schema_proto_msgTypes[1] - if protoimpl.UnsafeEnabled && x != nil { + if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { ms.StoreMessageInfo(mi) @@ -336,11 +355,6 @@ func (x *Field) ProtoReflect() protoreflect.Message { return mi.MessageOf(x) } -// Deprecated: Use Field.ProtoReflect.Descriptor instead. -func (*Field) Descriptor() ([]byte, []int) { - return file_org_apache_beam_model_pipeline_v1_schema_proto_rawDescGZIP(), []int{1} -} - func (x *Field) GetName() string { if x != nil { return x.Name @@ -383,13 +397,78 @@ func (x *Field) GetOptions() []*Option { return nil } -type FieldType struct { - state protoimpl.MessageState - sizeCache protoimpl.SizeCache - unknownFields protoimpl.UnknownFields +func (x *Field) SetName(v string) { + x.Name = v +} + +func (x *Field) SetDescription(v string) { + x.Description = v +} + +func (x *Field) SetType(v *FieldType) { + x.Type = v +} + +func (x *Field) SetId(v int32) { + x.Id = v +} + +func (x *Field) SetEncodingPosition(v int32) { + x.EncodingPosition = v +} + +func (x *Field) SetOptions(v []*Option) { + x.Options = v +} + +func (x *Field) HasType() bool { + if x == nil { + return false + } + return x.Type != nil +} + +func (x *Field) ClearType() { + x.Type = nil +} + +type Field_builder struct { + _ [0]func() // Prevents comparability and use of unkeyed literals for the builder. + + // REQUIRED. Name of this field within the schema. + Name string + // OPTIONAL. Human readable description of this field, such as the query that generated it. + Description string + Type *FieldType + Id int32 + // OPTIONAL. The position of this field's data when encoded, e.g. with beam:coder:row:v1. + // Either no fields in a given row are have encoding position populated, + // or all of them are. Used to support backwards compatibility with schema + // changes. + // If no fields have encoding position populated the order of encoding is the same as the order in the Schema. + // If this Field is part of a Schema where encoding_positions_set is True then encoding_position must be + // defined, otherwise this field is ignored. + EncodingPosition int32 + Options []*Option +} + +func (b0 Field_builder) Build() *Field { + m0 := &Field{} + b, x := &b0, m0 + _, _ = b, x + x.Name = b.Name + x.Description = b.Description + x.Type = b.Type + x.Id = b.Id + x.EncodingPosition = b.EncodingPosition + x.Options = b.Options + return m0 +} - Nullable bool `protobuf:"varint,1,opt,name=nullable,proto3" json:"nullable,omitempty"` - // Types that are assignable to TypeInfo: +type FieldType struct { + state protoimpl.MessageState `protogen:"hybrid.v1"` + Nullable bool `protobuf:"varint,1,opt,name=nullable,proto3" json:"nullable,omitempty"` + // Types that are valid to be assigned to TypeInfo: // // *FieldType_AtomicType // *FieldType_ArrayType @@ -397,16 +476,16 @@ type FieldType struct { // *FieldType_MapType // *FieldType_RowType // *FieldType_LogicalType - TypeInfo isFieldType_TypeInfo `protobuf_oneof:"type_info"` + TypeInfo isFieldType_TypeInfo `protobuf_oneof:"type_info"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache } func (x *FieldType) Reset() { *x = FieldType{} - if protoimpl.UnsafeEnabled { - mi := &file_org_apache_beam_model_pipeline_v1_schema_proto_msgTypes[2] - ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) - ms.StoreMessageInfo(mi) - } + mi := &file_org_apache_beam_model_pipeline_v1_schema_proto_msgTypes[2] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) } func (x *FieldType) String() string { @@ -417,7 +496,7 @@ func (*FieldType) ProtoMessage() {} func (x *FieldType) ProtoReflect() protoreflect.Message { mi := &file_org_apache_beam_model_pipeline_v1_schema_proto_msgTypes[2] - if protoimpl.UnsafeEnabled && x != nil { + if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { ms.StoreMessageInfo(mi) @@ -427,11 +506,6 @@ func (x *FieldType) ProtoReflect() protoreflect.Message { return mi.MessageOf(x) } -// Deprecated: Use FieldType.ProtoReflect.Descriptor instead. -func (*FieldType) Descriptor() ([]byte, []int) { - return file_org_apache_beam_model_pipeline_v1_schema_proto_rawDescGZIP(), []int{2} -} - func (x *FieldType) GetNullable() bool { if x != nil { return x.Nullable @@ -439,55 +513,290 @@ func (x *FieldType) GetNullable() bool { return false } -func (m *FieldType) GetTypeInfo() isFieldType_TypeInfo { - if m != nil { - return m.TypeInfo +func (x *FieldType) GetTypeInfo() isFieldType_TypeInfo { + if x != nil { + return x.TypeInfo } return nil } func (x *FieldType) GetAtomicType() AtomicType { - if x, ok := x.GetTypeInfo().(*FieldType_AtomicType); ok { - return x.AtomicType + if x != nil { + if x, ok := x.TypeInfo.(*FieldType_AtomicType); ok { + return x.AtomicType + } } return AtomicType_UNSPECIFIED } func (x *FieldType) GetArrayType() *ArrayType { - if x, ok := x.GetTypeInfo().(*FieldType_ArrayType); ok { - return x.ArrayType + if x != nil { + if x, ok := x.TypeInfo.(*FieldType_ArrayType); ok { + return x.ArrayType + } } return nil } func (x *FieldType) GetIterableType() *IterableType { - if x, ok := x.GetTypeInfo().(*FieldType_IterableType); ok { - return x.IterableType + if x != nil { + if x, ok := x.TypeInfo.(*FieldType_IterableType); ok { + return x.IterableType + } } return nil } func (x *FieldType) GetMapType() *MapType { - if x, ok := x.GetTypeInfo().(*FieldType_MapType); ok { - return x.MapType + if x != nil { + if x, ok := x.TypeInfo.(*FieldType_MapType); ok { + return x.MapType + } } return nil } func (x *FieldType) GetRowType() *RowType { - if x, ok := x.GetTypeInfo().(*FieldType_RowType); ok { - return x.RowType + if x != nil { + if x, ok := x.TypeInfo.(*FieldType_RowType); ok { + return x.RowType + } } return nil } func (x *FieldType) GetLogicalType() *LogicalType { - if x, ok := x.GetTypeInfo().(*FieldType_LogicalType); ok { - return x.LogicalType + if x != nil { + if x, ok := x.TypeInfo.(*FieldType_LogicalType); ok { + return x.LogicalType + } } return nil } +func (x *FieldType) SetNullable(v bool) { + x.Nullable = v +} + +func (x *FieldType) SetAtomicType(v AtomicType) { + x.TypeInfo = &FieldType_AtomicType{v} +} + +func (x *FieldType) SetArrayType(v *ArrayType) { + if v == nil { + x.TypeInfo = nil + return + } + x.TypeInfo = &FieldType_ArrayType{v} +} + +func (x *FieldType) SetIterableType(v *IterableType) { + if v == nil { + x.TypeInfo = nil + return + } + x.TypeInfo = &FieldType_IterableType{v} +} + +func (x *FieldType) SetMapType(v *MapType) { + if v == nil { + x.TypeInfo = nil + return + } + x.TypeInfo = &FieldType_MapType{v} +} + +func (x *FieldType) SetRowType(v *RowType) { + if v == nil { + x.TypeInfo = nil + return + } + x.TypeInfo = &FieldType_RowType{v} +} + +func (x *FieldType) SetLogicalType(v *LogicalType) { + if v == nil { + x.TypeInfo = nil + return + } + x.TypeInfo = &FieldType_LogicalType{v} +} + +func (x *FieldType) HasTypeInfo() bool { + if x == nil { + return false + } + return x.TypeInfo != nil +} + +func (x *FieldType) HasAtomicType() bool { + if x == nil { + return false + } + _, ok := x.TypeInfo.(*FieldType_AtomicType) + return ok +} + +func (x *FieldType) HasArrayType() bool { + if x == nil { + return false + } + _, ok := x.TypeInfo.(*FieldType_ArrayType) + return ok +} + +func (x *FieldType) HasIterableType() bool { + if x == nil { + return false + } + _, ok := x.TypeInfo.(*FieldType_IterableType) + return ok +} + +func (x *FieldType) HasMapType() bool { + if x == nil { + return false + } + _, ok := x.TypeInfo.(*FieldType_MapType) + return ok +} + +func (x *FieldType) HasRowType() bool { + if x == nil { + return false + } + _, ok := x.TypeInfo.(*FieldType_RowType) + return ok +} + +func (x *FieldType) HasLogicalType() bool { + if x == nil { + return false + } + _, ok := x.TypeInfo.(*FieldType_LogicalType) + return ok +} + +func (x *FieldType) ClearTypeInfo() { + x.TypeInfo = nil +} + +func (x *FieldType) ClearAtomicType() { + if _, ok := x.TypeInfo.(*FieldType_AtomicType); ok { + x.TypeInfo = nil + } +} + +func (x *FieldType) ClearArrayType() { + if _, ok := x.TypeInfo.(*FieldType_ArrayType); ok { + x.TypeInfo = nil + } +} + +func (x *FieldType) ClearIterableType() { + if _, ok := x.TypeInfo.(*FieldType_IterableType); ok { + x.TypeInfo = nil + } +} + +func (x *FieldType) ClearMapType() { + if _, ok := x.TypeInfo.(*FieldType_MapType); ok { + x.TypeInfo = nil + } +} + +func (x *FieldType) ClearRowType() { + if _, ok := x.TypeInfo.(*FieldType_RowType); ok { + x.TypeInfo = nil + } +} + +func (x *FieldType) ClearLogicalType() { + if _, ok := x.TypeInfo.(*FieldType_LogicalType); ok { + x.TypeInfo = nil + } +} + +const FieldType_TypeInfo_not_set_case case_FieldType_TypeInfo = 0 +const FieldType_AtomicType_case case_FieldType_TypeInfo = 2 +const FieldType_ArrayType_case case_FieldType_TypeInfo = 3 +const FieldType_IterableType_case case_FieldType_TypeInfo = 4 +const FieldType_MapType_case case_FieldType_TypeInfo = 5 +const FieldType_RowType_case case_FieldType_TypeInfo = 6 +const FieldType_LogicalType_case case_FieldType_TypeInfo = 7 + +func (x *FieldType) WhichTypeInfo() case_FieldType_TypeInfo { + if x == nil { + return FieldType_TypeInfo_not_set_case + } + switch x.TypeInfo.(type) { + case *FieldType_AtomicType: + return FieldType_AtomicType_case + case *FieldType_ArrayType: + return FieldType_ArrayType_case + case *FieldType_IterableType: + return FieldType_IterableType_case + case *FieldType_MapType: + return FieldType_MapType_case + case *FieldType_RowType: + return FieldType_RowType_case + case *FieldType_LogicalType: + return FieldType_LogicalType_case + default: + return FieldType_TypeInfo_not_set_case + } +} + +type FieldType_builder struct { + _ [0]func() // Prevents comparability and use of unkeyed literals for the builder. + + Nullable bool + // Fields of oneof TypeInfo: + AtomicType *AtomicType + ArrayType *ArrayType + IterableType *IterableType + MapType *MapType + RowType *RowType + LogicalType *LogicalType + // -- end of TypeInfo +} + +func (b0 FieldType_builder) Build() *FieldType { + m0 := &FieldType{} + b, x := &b0, m0 + _, _ = b, x + x.Nullable = b.Nullable + if b.AtomicType != nil { + x.TypeInfo = &FieldType_AtomicType{*b.AtomicType} + } + if b.ArrayType != nil { + x.TypeInfo = &FieldType_ArrayType{b.ArrayType} + } + if b.IterableType != nil { + x.TypeInfo = &FieldType_IterableType{b.IterableType} + } + if b.MapType != nil { + x.TypeInfo = &FieldType_MapType{b.MapType} + } + if b.RowType != nil { + x.TypeInfo = &FieldType_RowType{b.RowType} + } + if b.LogicalType != nil { + x.TypeInfo = &FieldType_LogicalType{b.LogicalType} + } + return m0 +} + +type case_FieldType_TypeInfo protoreflect.FieldNumber + +func (x case_FieldType_TypeInfo) String() string { + md := file_org_apache_beam_model_pipeline_v1_schema_proto_msgTypes[2].Descriptor() + if x == 0 { + return "not set" + } + return protoimpl.X.MessageFieldStringOf(md, protoreflect.FieldNumber(x)) +} + type isFieldType_TypeInfo interface { isFieldType_TypeInfo() } @@ -529,20 +838,17 @@ func (*FieldType_RowType) isFieldType_TypeInfo() {} func (*FieldType_LogicalType) isFieldType_TypeInfo() {} type ArrayType struct { - state protoimpl.MessageState - sizeCache protoimpl.SizeCache + state protoimpl.MessageState `protogen:"hybrid.v1"` + ElementType *FieldType `protobuf:"bytes,1,opt,name=element_type,json=elementType,proto3" json:"element_type,omitempty"` unknownFields protoimpl.UnknownFields - - ElementType *FieldType `protobuf:"bytes,1,opt,name=element_type,json=elementType,proto3" json:"element_type,omitempty"` + sizeCache protoimpl.SizeCache } func (x *ArrayType) Reset() { *x = ArrayType{} - if protoimpl.UnsafeEnabled { - mi := &file_org_apache_beam_model_pipeline_v1_schema_proto_msgTypes[3] - ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) - ms.StoreMessageInfo(mi) - } + mi := &file_org_apache_beam_model_pipeline_v1_schema_proto_msgTypes[3] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) } func (x *ArrayType) String() string { @@ -553,7 +859,7 @@ func (*ArrayType) ProtoMessage() {} func (x *ArrayType) ProtoReflect() protoreflect.Message { mi := &file_org_apache_beam_model_pipeline_v1_schema_proto_msgTypes[3] - if protoimpl.UnsafeEnabled && x != nil { + if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { ms.StoreMessageInfo(mi) @@ -563,11 +869,6 @@ func (x *ArrayType) ProtoReflect() protoreflect.Message { return mi.MessageOf(x) } -// Deprecated: Use ArrayType.ProtoReflect.Descriptor instead. -func (*ArrayType) Descriptor() ([]byte, []int) { - return file_org_apache_beam_model_pipeline_v1_schema_proto_rawDescGZIP(), []int{3} -} - func (x *ArrayType) GetElementType() *FieldType { if x != nil { return x.ElementType @@ -575,21 +876,47 @@ func (x *ArrayType) GetElementType() *FieldType { return nil } +func (x *ArrayType) SetElementType(v *FieldType) { + x.ElementType = v +} + +func (x *ArrayType) HasElementType() bool { + if x == nil { + return false + } + return x.ElementType != nil +} + +func (x *ArrayType) ClearElementType() { + x.ElementType = nil +} + +type ArrayType_builder struct { + _ [0]func() // Prevents comparability and use of unkeyed literals for the builder. + + ElementType *FieldType +} + +func (b0 ArrayType_builder) Build() *ArrayType { + m0 := &ArrayType{} + b, x := &b0, m0 + _, _ = b, x + x.ElementType = b.ElementType + return m0 +} + type IterableType struct { - state protoimpl.MessageState - sizeCache protoimpl.SizeCache + state protoimpl.MessageState `protogen:"hybrid.v1"` + ElementType *FieldType `protobuf:"bytes,1,opt,name=element_type,json=elementType,proto3" json:"element_type,omitempty"` unknownFields protoimpl.UnknownFields - - ElementType *FieldType `protobuf:"bytes,1,opt,name=element_type,json=elementType,proto3" json:"element_type,omitempty"` + sizeCache protoimpl.SizeCache } func (x *IterableType) Reset() { *x = IterableType{} - if protoimpl.UnsafeEnabled { - mi := &file_org_apache_beam_model_pipeline_v1_schema_proto_msgTypes[4] - ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) - ms.StoreMessageInfo(mi) - } + mi := &file_org_apache_beam_model_pipeline_v1_schema_proto_msgTypes[4] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) } func (x *IterableType) String() string { @@ -600,7 +927,7 @@ func (*IterableType) ProtoMessage() {} func (x *IterableType) ProtoReflect() protoreflect.Message { mi := &file_org_apache_beam_model_pipeline_v1_schema_proto_msgTypes[4] - if protoimpl.UnsafeEnabled && x != nil { + if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { ms.StoreMessageInfo(mi) @@ -610,11 +937,6 @@ func (x *IterableType) ProtoReflect() protoreflect.Message { return mi.MessageOf(x) } -// Deprecated: Use IterableType.ProtoReflect.Descriptor instead. -func (*IterableType) Descriptor() ([]byte, []int) { - return file_org_apache_beam_model_pipeline_v1_schema_proto_rawDescGZIP(), []int{4} -} - func (x *IterableType) GetElementType() *FieldType { if x != nil { return x.ElementType @@ -622,22 +944,48 @@ func (x *IterableType) GetElementType() *FieldType { return nil } +func (x *IterableType) SetElementType(v *FieldType) { + x.ElementType = v +} + +func (x *IterableType) HasElementType() bool { + if x == nil { + return false + } + return x.ElementType != nil +} + +func (x *IterableType) ClearElementType() { + x.ElementType = nil +} + +type IterableType_builder struct { + _ [0]func() // Prevents comparability and use of unkeyed literals for the builder. + + ElementType *FieldType +} + +func (b0 IterableType_builder) Build() *IterableType { + m0 := &IterableType{} + b, x := &b0, m0 + _, _ = b, x + x.ElementType = b.ElementType + return m0 +} + type MapType struct { - state protoimpl.MessageState - sizeCache protoimpl.SizeCache + state protoimpl.MessageState `protogen:"hybrid.v1"` + KeyType *FieldType `protobuf:"bytes,1,opt,name=key_type,json=keyType,proto3" json:"key_type,omitempty"` + ValueType *FieldType `protobuf:"bytes,2,opt,name=value_type,json=valueType,proto3" json:"value_type,omitempty"` unknownFields protoimpl.UnknownFields - - KeyType *FieldType `protobuf:"bytes,1,opt,name=key_type,json=keyType,proto3" json:"key_type,omitempty"` - ValueType *FieldType `protobuf:"bytes,2,opt,name=value_type,json=valueType,proto3" json:"value_type,omitempty"` + sizeCache protoimpl.SizeCache } func (x *MapType) Reset() { *x = MapType{} - if protoimpl.UnsafeEnabled { - mi := &file_org_apache_beam_model_pipeline_v1_schema_proto_msgTypes[5] - ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) - ms.StoreMessageInfo(mi) - } + mi := &file_org_apache_beam_model_pipeline_v1_schema_proto_msgTypes[5] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) } func (x *MapType) String() string { @@ -648,7 +996,7 @@ func (*MapType) ProtoMessage() {} func (x *MapType) ProtoReflect() protoreflect.Message { mi := &file_org_apache_beam_model_pipeline_v1_schema_proto_msgTypes[5] - if protoimpl.UnsafeEnabled && x != nil { + if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { ms.StoreMessageInfo(mi) @@ -658,11 +1006,6 @@ func (x *MapType) ProtoReflect() protoreflect.Message { return mi.MessageOf(x) } -// Deprecated: Use MapType.ProtoReflect.Descriptor instead. -func (*MapType) Descriptor() ([]byte, []int) { - return file_org_apache_beam_model_pipeline_v1_schema_proto_rawDescGZIP(), []int{5} -} - func (x *MapType) GetKeyType() *FieldType { if x != nil { return x.KeyType @@ -677,21 +1020,64 @@ func (x *MapType) GetValueType() *FieldType { return nil } +func (x *MapType) SetKeyType(v *FieldType) { + x.KeyType = v +} + +func (x *MapType) SetValueType(v *FieldType) { + x.ValueType = v +} + +func (x *MapType) HasKeyType() bool { + if x == nil { + return false + } + return x.KeyType != nil +} + +func (x *MapType) HasValueType() bool { + if x == nil { + return false + } + return x.ValueType != nil +} + +func (x *MapType) ClearKeyType() { + x.KeyType = nil +} + +func (x *MapType) ClearValueType() { + x.ValueType = nil +} + +type MapType_builder struct { + _ [0]func() // Prevents comparability and use of unkeyed literals for the builder. + + KeyType *FieldType + ValueType *FieldType +} + +func (b0 MapType_builder) Build() *MapType { + m0 := &MapType{} + b, x := &b0, m0 + _, _ = b, x + x.KeyType = b.KeyType + x.ValueType = b.ValueType + return m0 +} + type RowType struct { - state protoimpl.MessageState - sizeCache protoimpl.SizeCache + state protoimpl.MessageState `protogen:"hybrid.v1"` + Schema *Schema `protobuf:"bytes,1,opt,name=schema,proto3" json:"schema,omitempty"` unknownFields protoimpl.UnknownFields - - Schema *Schema `protobuf:"bytes,1,opt,name=schema,proto3" json:"schema,omitempty"` + sizeCache protoimpl.SizeCache } func (x *RowType) Reset() { *x = RowType{} - if protoimpl.UnsafeEnabled { - mi := &file_org_apache_beam_model_pipeline_v1_schema_proto_msgTypes[6] - ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) - ms.StoreMessageInfo(mi) - } + mi := &file_org_apache_beam_model_pipeline_v1_schema_proto_msgTypes[6] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) } func (x *RowType) String() string { @@ -702,7 +1088,7 @@ func (*RowType) ProtoMessage() {} func (x *RowType) ProtoReflect() protoreflect.Message { mi := &file_org_apache_beam_model_pipeline_v1_schema_proto_msgTypes[6] - if protoimpl.UnsafeEnabled && x != nil { + if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { ms.StoreMessageInfo(mi) @@ -712,11 +1098,6 @@ func (x *RowType) ProtoReflect() protoreflect.Message { return mi.MessageOf(x) } -// Deprecated: Use RowType.ProtoReflect.Descriptor instead. -func (*RowType) Descriptor() ([]byte, []int) { - return file_org_apache_beam_model_pipeline_v1_schema_proto_rawDescGZIP(), []int{6} -} - func (x *RowType) GetSchema() *Schema { if x != nil { return x.Schema @@ -724,25 +1105,51 @@ func (x *RowType) GetSchema() *Schema { return nil } -type LogicalType struct { - state protoimpl.MessageState - sizeCache protoimpl.SizeCache - unknownFields protoimpl.UnknownFields +func (x *RowType) SetSchema(v *Schema) { + x.Schema = v +} + +func (x *RowType) HasSchema() bool { + if x == nil { + return false + } + return x.Schema != nil +} + +func (x *RowType) ClearSchema() { + x.Schema = nil +} + +type RowType_builder struct { + _ [0]func() // Prevents comparability and use of unkeyed literals for the builder. - Urn string `protobuf:"bytes,1,opt,name=urn,proto3" json:"urn,omitempty"` - Payload []byte `protobuf:"bytes,2,opt,name=payload,proto3" json:"payload,omitempty"` - Representation *FieldType `protobuf:"bytes,3,opt,name=representation,proto3" json:"representation,omitempty"` - ArgumentType *FieldType `protobuf:"bytes,4,opt,name=argument_type,json=argumentType,proto3" json:"argument_type,omitempty"` - Argument *FieldValue `protobuf:"bytes,5,opt,name=argument,proto3" json:"argument,omitempty"` + Schema *Schema +} + +func (b0 RowType_builder) Build() *RowType { + m0 := &RowType{} + b, x := &b0, m0 + _, _ = b, x + x.Schema = b.Schema + return m0 +} + +type LogicalType struct { + state protoimpl.MessageState `protogen:"hybrid.v1"` + Urn string `protobuf:"bytes,1,opt,name=urn,proto3" json:"urn,omitempty"` + Payload []byte `protobuf:"bytes,2,opt,name=payload,proto3" json:"payload,omitempty"` + Representation *FieldType `protobuf:"bytes,3,opt,name=representation,proto3" json:"representation,omitempty"` + ArgumentType *FieldType `protobuf:"bytes,4,opt,name=argument_type,json=argumentType,proto3" json:"argument_type,omitempty"` + Argument *FieldValue `protobuf:"bytes,5,opt,name=argument,proto3" json:"argument,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache } func (x *LogicalType) Reset() { *x = LogicalType{} - if protoimpl.UnsafeEnabled { - mi := &file_org_apache_beam_model_pipeline_v1_schema_proto_msgTypes[7] - ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) - ms.StoreMessageInfo(mi) - } + mi := &file_org_apache_beam_model_pipeline_v1_schema_proto_msgTypes[7] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) } func (x *LogicalType) String() string { @@ -753,7 +1160,7 @@ func (*LogicalType) ProtoMessage() {} func (x *LogicalType) ProtoReflect() protoreflect.Message { mi := &file_org_apache_beam_model_pipeline_v1_schema_proto_msgTypes[7] - if protoimpl.UnsafeEnabled && x != nil { + if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { ms.StoreMessageInfo(mi) @@ -763,11 +1170,6 @@ func (x *LogicalType) ProtoReflect() protoreflect.Message { return mi.MessageOf(x) } -// Deprecated: Use LogicalType.ProtoReflect.Descriptor instead. -func (*LogicalType) Descriptor() ([]byte, []int) { - return file_org_apache_beam_model_pipeline_v1_schema_proto_rawDescGZIP(), []int{7} -} - func (x *LogicalType) GetUrn() string { if x != nil { return x.Urn @@ -803,21 +1205,97 @@ func (x *LogicalType) GetArgument() *FieldValue { return nil } +func (x *LogicalType) SetUrn(v string) { + x.Urn = v +} + +func (x *LogicalType) SetPayload(v []byte) { + if v == nil { + v = []byte{} + } + x.Payload = v +} + +func (x *LogicalType) SetRepresentation(v *FieldType) { + x.Representation = v +} + +func (x *LogicalType) SetArgumentType(v *FieldType) { + x.ArgumentType = v +} + +func (x *LogicalType) SetArgument(v *FieldValue) { + x.Argument = v +} + +func (x *LogicalType) HasRepresentation() bool { + if x == nil { + return false + } + return x.Representation != nil +} + +func (x *LogicalType) HasArgumentType() bool { + if x == nil { + return false + } + return x.ArgumentType != nil +} + +func (x *LogicalType) HasArgument() bool { + if x == nil { + return false + } + return x.Argument != nil +} + +func (x *LogicalType) ClearRepresentation() { + x.Representation = nil +} + +func (x *LogicalType) ClearArgumentType() { + x.ArgumentType = nil +} + +func (x *LogicalType) ClearArgument() { + x.Argument = nil +} + +type LogicalType_builder struct { + _ [0]func() // Prevents comparability and use of unkeyed literals for the builder. + + Urn string + Payload []byte + Representation *FieldType + ArgumentType *FieldType + Argument *FieldValue +} + +func (b0 LogicalType_builder) Build() *LogicalType { + m0 := &LogicalType{} + b, x := &b0, m0 + _, _ = b, x + x.Urn = b.Urn + x.Payload = b.Payload + x.Representation = b.Representation + x.ArgumentType = b.ArgumentType + x.Argument = b.Argument + return m0 +} + // Universally defined Logical types for Row schemas. // These logical types are supposed to be understood by all SDKs. type LogicalTypes struct { - state protoimpl.MessageState - sizeCache protoimpl.SizeCache + state protoimpl.MessageState `protogen:"hybrid.v1"` unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache } func (x *LogicalTypes) Reset() { *x = LogicalTypes{} - if protoimpl.UnsafeEnabled { - mi := &file_org_apache_beam_model_pipeline_v1_schema_proto_msgTypes[8] - ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) - ms.StoreMessageInfo(mi) - } + mi := &file_org_apache_beam_model_pipeline_v1_schema_proto_msgTypes[8] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) } func (x *LogicalTypes) String() string { @@ -828,7 +1306,7 @@ func (*LogicalTypes) ProtoMessage() {} func (x *LogicalTypes) ProtoReflect() protoreflect.Message { mi := &file_org_apache_beam_model_pipeline_v1_schema_proto_msgTypes[8] - if protoimpl.UnsafeEnabled && x != nil { + if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { ms.StoreMessageInfo(mi) @@ -838,32 +1316,36 @@ func (x *LogicalTypes) ProtoReflect() protoreflect.Message { return mi.MessageOf(x) } -// Deprecated: Use LogicalTypes.ProtoReflect.Descriptor instead. -func (*LogicalTypes) Descriptor() ([]byte, []int) { - return file_org_apache_beam_model_pipeline_v1_schema_proto_rawDescGZIP(), []int{8} +type LogicalTypes_builder struct { + _ [0]func() // Prevents comparability and use of unkeyed literals for the builder. + } -type Option struct { - state protoimpl.MessageState - sizeCache protoimpl.SizeCache - unknownFields protoimpl.UnknownFields +func (b0 LogicalTypes_builder) Build() *LogicalTypes { + m0 := &LogicalTypes{} + b, x := &b0, m0 + _, _ = b, x + return m0 +} +type Option struct { + state protoimpl.MessageState `protogen:"hybrid.v1"` // REQUIRED. Identifier for the option. Name string `protobuf:"bytes,1,opt,name=name,proto3" json:"name,omitempty"` // REQUIRED. Type specifier for the structure of value. // Conventionally, options that don't require additional configuration should // use a boolean type, with the value set to true. - Type *FieldType `protobuf:"bytes,2,opt,name=type,proto3" json:"type,omitempty"` - Value *FieldValue `protobuf:"bytes,3,opt,name=value,proto3" json:"value,omitempty"` + Type *FieldType `protobuf:"bytes,2,opt,name=type,proto3" json:"type,omitempty"` + Value *FieldValue `protobuf:"bytes,3,opt,name=value,proto3" json:"value,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache } func (x *Option) Reset() { *x = Option{} - if protoimpl.UnsafeEnabled { - mi := &file_org_apache_beam_model_pipeline_v1_schema_proto_msgTypes[9] - ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) - ms.StoreMessageInfo(mi) - } + mi := &file_org_apache_beam_model_pipeline_v1_schema_proto_msgTypes[9] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) } func (x *Option) String() string { @@ -874,7 +1356,7 @@ func (*Option) ProtoMessage() {} func (x *Option) ProtoReflect() protoreflect.Message { mi := &file_org_apache_beam_model_pipeline_v1_schema_proto_msgTypes[9] - if protoimpl.UnsafeEnabled && x != nil { + if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { ms.StoreMessageInfo(mi) @@ -884,11 +1366,6 @@ func (x *Option) ProtoReflect() protoreflect.Message { return mi.MessageOf(x) } -// Deprecated: Use Option.ProtoReflect.Descriptor instead. -func (*Option) Descriptor() ([]byte, []int) { - return file_org_apache_beam_model_pipeline_v1_schema_proto_rawDescGZIP(), []int{9} -} - func (x *Option) GetName() string { if x != nil { return x.Name @@ -910,21 +1387,74 @@ func (x *Option) GetValue() *FieldValue { return nil } +func (x *Option) SetName(v string) { + x.Name = v +} + +func (x *Option) SetType(v *FieldType) { + x.Type = v +} + +func (x *Option) SetValue(v *FieldValue) { + x.Value = v +} + +func (x *Option) HasType() bool { + if x == nil { + return false + } + return x.Type != nil +} + +func (x *Option) HasValue() bool { + if x == nil { + return false + } + return x.Value != nil +} + +func (x *Option) ClearType() { + x.Type = nil +} + +func (x *Option) ClearValue() { + x.Value = nil +} + +type Option_builder struct { + _ [0]func() // Prevents comparability and use of unkeyed literals for the builder. + + // REQUIRED. Identifier for the option. + Name string + // REQUIRED. Type specifier for the structure of value. + // Conventionally, options that don't require additional configuration should + // use a boolean type, with the value set to true. + Type *FieldType + Value *FieldValue +} + +func (b0 Option_builder) Build() *Option { + m0 := &Option{} + b, x := &b0, m0 + _, _ = b, x + x.Name = b.Name + x.Type = b.Type + x.Value = b.Value + return m0 +} + type Row struct { - state protoimpl.MessageState - sizeCache protoimpl.SizeCache + state protoimpl.MessageState `protogen:"hybrid.v1"` + Values []*FieldValue `protobuf:"bytes,1,rep,name=values,proto3" json:"values,omitempty"` unknownFields protoimpl.UnknownFields - - Values []*FieldValue `protobuf:"bytes,1,rep,name=values,proto3" json:"values,omitempty"` + sizeCache protoimpl.SizeCache } func (x *Row) Reset() { *x = Row{} - if protoimpl.UnsafeEnabled { - mi := &file_org_apache_beam_model_pipeline_v1_schema_proto_msgTypes[10] - ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) - ms.StoreMessageInfo(mi) - } + mi := &file_org_apache_beam_model_pipeline_v1_schema_proto_msgTypes[10] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) } func (x *Row) String() string { @@ -935,7 +1465,7 @@ func (*Row) ProtoMessage() {} func (x *Row) ProtoReflect() protoreflect.Message { mi := &file_org_apache_beam_model_pipeline_v1_schema_proto_msgTypes[10] - if protoimpl.UnsafeEnabled && x != nil { + if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { ms.StoreMessageInfo(mi) @@ -945,11 +1475,6 @@ func (x *Row) ProtoReflect() protoreflect.Message { return mi.MessageOf(x) } -// Deprecated: Use Row.ProtoReflect.Descriptor instead. -func (*Row) Descriptor() ([]byte, []int) { - return file_org_apache_beam_model_pipeline_v1_schema_proto_rawDescGZIP(), []int{10} -} - func (x *Row) GetValues() []*FieldValue { if x != nil { return x.Values @@ -957,14 +1482,29 @@ func (x *Row) GetValues() []*FieldValue { return nil } -type FieldValue struct { - state protoimpl.MessageState - sizeCache protoimpl.SizeCache - unknownFields protoimpl.UnknownFields +func (x *Row) SetValues(v []*FieldValue) { + x.Values = v +} + +type Row_builder struct { + _ [0]func() // Prevents comparability and use of unkeyed literals for the builder. + + Values []*FieldValue +} + +func (b0 Row_builder) Build() *Row { + m0 := &Row{} + b, x := &b0, m0 + _, _ = b, x + x.Values = b.Values + return m0 +} +type FieldValue struct { + state protoimpl.MessageState `protogen:"hybrid.v1"` // If none of these are set, value is considered null. // - // Types that are assignable to FieldValue: + // Types that are valid to be assigned to FieldValue: // // *FieldValue_AtomicValue // *FieldValue_ArrayValue @@ -972,16 +1512,16 @@ type FieldValue struct { // *FieldValue_MapValue // *FieldValue_RowValue // *FieldValue_LogicalTypeValue - FieldValue isFieldValue_FieldValue `protobuf_oneof:"field_value"` + FieldValue isFieldValue_FieldValue `protobuf_oneof:"field_value"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache } func (x *FieldValue) Reset() { *x = FieldValue{} - if protoimpl.UnsafeEnabled { - mi := &file_org_apache_beam_model_pipeline_v1_schema_proto_msgTypes[11] - ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) - ms.StoreMessageInfo(mi) - } + mi := &file_org_apache_beam_model_pipeline_v1_schema_proto_msgTypes[11] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) } func (x *FieldValue) String() string { @@ -992,7 +1532,7 @@ func (*FieldValue) ProtoMessage() {} func (x *FieldValue) ProtoReflect() protoreflect.Message { mi := &file_org_apache_beam_model_pipeline_v1_schema_proto_msgTypes[11] - if protoimpl.UnsafeEnabled && x != nil { + if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { ms.StoreMessageInfo(mi) @@ -1002,219 +1542,739 @@ func (x *FieldValue) ProtoReflect() protoreflect.Message { return mi.MessageOf(x) } -// Deprecated: Use FieldValue.ProtoReflect.Descriptor instead. -func (*FieldValue) Descriptor() ([]byte, []int) { - return file_org_apache_beam_model_pipeline_v1_schema_proto_rawDescGZIP(), []int{11} +func (x *FieldValue) GetFieldValue() isFieldValue_FieldValue { + if x != nil { + return x.FieldValue + } + return nil +} + +func (x *FieldValue) GetAtomicValue() *AtomicTypeValue { + if x != nil { + if x, ok := x.FieldValue.(*FieldValue_AtomicValue); ok { + return x.AtomicValue + } + } + return nil +} + +func (x *FieldValue) GetArrayValue() *ArrayTypeValue { + if x != nil { + if x, ok := x.FieldValue.(*FieldValue_ArrayValue); ok { + return x.ArrayValue + } + } + return nil +} + +func (x *FieldValue) GetIterableValue() *IterableTypeValue { + if x != nil { + if x, ok := x.FieldValue.(*FieldValue_IterableValue); ok { + return x.IterableValue + } + } + return nil +} + +func (x *FieldValue) GetMapValue() *MapTypeValue { + if x != nil { + if x, ok := x.FieldValue.(*FieldValue_MapValue); ok { + return x.MapValue + } + } + return nil +} + +func (x *FieldValue) GetRowValue() *Row { + if x != nil { + if x, ok := x.FieldValue.(*FieldValue_RowValue); ok { + return x.RowValue + } + } + return nil +} + +func (x *FieldValue) GetLogicalTypeValue() *LogicalTypeValue { + if x != nil { + if x, ok := x.FieldValue.(*FieldValue_LogicalTypeValue); ok { + return x.LogicalTypeValue + } + } + return nil +} + +func (x *FieldValue) SetAtomicValue(v *AtomicTypeValue) { + if v == nil { + x.FieldValue = nil + return + } + x.FieldValue = &FieldValue_AtomicValue{v} +} + +func (x *FieldValue) SetArrayValue(v *ArrayTypeValue) { + if v == nil { + x.FieldValue = nil + return + } + x.FieldValue = &FieldValue_ArrayValue{v} +} + +func (x *FieldValue) SetIterableValue(v *IterableTypeValue) { + if v == nil { + x.FieldValue = nil + return + } + x.FieldValue = &FieldValue_IterableValue{v} +} + +func (x *FieldValue) SetMapValue(v *MapTypeValue) { + if v == nil { + x.FieldValue = nil + return + } + x.FieldValue = &FieldValue_MapValue{v} +} + +func (x *FieldValue) SetRowValue(v *Row) { + if v == nil { + x.FieldValue = nil + return + } + x.FieldValue = &FieldValue_RowValue{v} +} + +func (x *FieldValue) SetLogicalTypeValue(v *LogicalTypeValue) { + if v == nil { + x.FieldValue = nil + return + } + x.FieldValue = &FieldValue_LogicalTypeValue{v} +} + +func (x *FieldValue) HasFieldValue() bool { + if x == nil { + return false + } + return x.FieldValue != nil +} + +func (x *FieldValue) HasAtomicValue() bool { + if x == nil { + return false + } + _, ok := x.FieldValue.(*FieldValue_AtomicValue) + return ok +} + +func (x *FieldValue) HasArrayValue() bool { + if x == nil { + return false + } + _, ok := x.FieldValue.(*FieldValue_ArrayValue) + return ok +} + +func (x *FieldValue) HasIterableValue() bool { + if x == nil { + return false + } + _, ok := x.FieldValue.(*FieldValue_IterableValue) + return ok +} + +func (x *FieldValue) HasMapValue() bool { + if x == nil { + return false + } + _, ok := x.FieldValue.(*FieldValue_MapValue) + return ok +} + +func (x *FieldValue) HasRowValue() bool { + if x == nil { + return false + } + _, ok := x.FieldValue.(*FieldValue_RowValue) + return ok +} + +func (x *FieldValue) HasLogicalTypeValue() bool { + if x == nil { + return false + } + _, ok := x.FieldValue.(*FieldValue_LogicalTypeValue) + return ok +} + +func (x *FieldValue) ClearFieldValue() { + x.FieldValue = nil +} + +func (x *FieldValue) ClearAtomicValue() { + if _, ok := x.FieldValue.(*FieldValue_AtomicValue); ok { + x.FieldValue = nil + } +} + +func (x *FieldValue) ClearArrayValue() { + if _, ok := x.FieldValue.(*FieldValue_ArrayValue); ok { + x.FieldValue = nil + } +} + +func (x *FieldValue) ClearIterableValue() { + if _, ok := x.FieldValue.(*FieldValue_IterableValue); ok { + x.FieldValue = nil + } +} + +func (x *FieldValue) ClearMapValue() { + if _, ok := x.FieldValue.(*FieldValue_MapValue); ok { + x.FieldValue = nil + } +} + +func (x *FieldValue) ClearRowValue() { + if _, ok := x.FieldValue.(*FieldValue_RowValue); ok { + x.FieldValue = nil + } +} + +func (x *FieldValue) ClearLogicalTypeValue() { + if _, ok := x.FieldValue.(*FieldValue_LogicalTypeValue); ok { + x.FieldValue = nil + } +} + +const FieldValue_FieldValue_not_set_case case_FieldValue_FieldValue = 0 +const FieldValue_AtomicValue_case case_FieldValue_FieldValue = 1 +const FieldValue_ArrayValue_case case_FieldValue_FieldValue = 2 +const FieldValue_IterableValue_case case_FieldValue_FieldValue = 3 +const FieldValue_MapValue_case case_FieldValue_FieldValue = 4 +const FieldValue_RowValue_case case_FieldValue_FieldValue = 5 +const FieldValue_LogicalTypeValue_case case_FieldValue_FieldValue = 6 + +func (x *FieldValue) WhichFieldValue() case_FieldValue_FieldValue { + if x == nil { + return FieldValue_FieldValue_not_set_case + } + switch x.FieldValue.(type) { + case *FieldValue_AtomicValue: + return FieldValue_AtomicValue_case + case *FieldValue_ArrayValue: + return FieldValue_ArrayValue_case + case *FieldValue_IterableValue: + return FieldValue_IterableValue_case + case *FieldValue_MapValue: + return FieldValue_MapValue_case + case *FieldValue_RowValue: + return FieldValue_RowValue_case + case *FieldValue_LogicalTypeValue: + return FieldValue_LogicalTypeValue_case + default: + return FieldValue_FieldValue_not_set_case + } +} + +type FieldValue_builder struct { + _ [0]func() // Prevents comparability and use of unkeyed literals for the builder. + + // If none of these are set, value is considered null. + + // Fields of oneof FieldValue: + AtomicValue *AtomicTypeValue + ArrayValue *ArrayTypeValue + IterableValue *IterableTypeValue + MapValue *MapTypeValue + RowValue *Row + LogicalTypeValue *LogicalTypeValue + // -- end of FieldValue +} + +func (b0 FieldValue_builder) Build() *FieldValue { + m0 := &FieldValue{} + b, x := &b0, m0 + _, _ = b, x + if b.AtomicValue != nil { + x.FieldValue = &FieldValue_AtomicValue{b.AtomicValue} + } + if b.ArrayValue != nil { + x.FieldValue = &FieldValue_ArrayValue{b.ArrayValue} + } + if b.IterableValue != nil { + x.FieldValue = &FieldValue_IterableValue{b.IterableValue} + } + if b.MapValue != nil { + x.FieldValue = &FieldValue_MapValue{b.MapValue} + } + if b.RowValue != nil { + x.FieldValue = &FieldValue_RowValue{b.RowValue} + } + if b.LogicalTypeValue != nil { + x.FieldValue = &FieldValue_LogicalTypeValue{b.LogicalTypeValue} + } + return m0 +} + +type case_FieldValue_FieldValue protoreflect.FieldNumber + +func (x case_FieldValue_FieldValue) String() string { + md := file_org_apache_beam_model_pipeline_v1_schema_proto_msgTypes[11].Descriptor() + if x == 0 { + return "not set" + } + return protoimpl.X.MessageFieldStringOf(md, protoreflect.FieldNumber(x)) +} + +type isFieldValue_FieldValue interface { + isFieldValue_FieldValue() +} + +type FieldValue_AtomicValue struct { + AtomicValue *AtomicTypeValue `protobuf:"bytes,1,opt,name=atomic_value,json=atomicValue,proto3,oneof"` +} + +type FieldValue_ArrayValue struct { + ArrayValue *ArrayTypeValue `protobuf:"bytes,2,opt,name=array_value,json=arrayValue,proto3,oneof"` +} + +type FieldValue_IterableValue struct { + IterableValue *IterableTypeValue `protobuf:"bytes,3,opt,name=iterable_value,json=iterableValue,proto3,oneof"` +} + +type FieldValue_MapValue struct { + MapValue *MapTypeValue `protobuf:"bytes,4,opt,name=map_value,json=mapValue,proto3,oneof"` +} + +type FieldValue_RowValue struct { + RowValue *Row `protobuf:"bytes,5,opt,name=row_value,json=rowValue,proto3,oneof"` +} + +type FieldValue_LogicalTypeValue struct { + LogicalTypeValue *LogicalTypeValue `protobuf:"bytes,6,opt,name=logical_type_value,json=logicalTypeValue,proto3,oneof"` +} + +func (*FieldValue_AtomicValue) isFieldValue_FieldValue() {} + +func (*FieldValue_ArrayValue) isFieldValue_FieldValue() {} + +func (*FieldValue_IterableValue) isFieldValue_FieldValue() {} + +func (*FieldValue_MapValue) isFieldValue_FieldValue() {} + +func (*FieldValue_RowValue) isFieldValue_FieldValue() {} + +func (*FieldValue_LogicalTypeValue) isFieldValue_FieldValue() {} + +type AtomicTypeValue struct { + state protoimpl.MessageState `protogen:"hybrid.v1"` + // Types that are valid to be assigned to Value: + // + // *AtomicTypeValue_Byte + // *AtomicTypeValue_Int16 + // *AtomicTypeValue_Int32 + // *AtomicTypeValue_Int64 + // *AtomicTypeValue_Float + // *AtomicTypeValue_Double + // *AtomicTypeValue_String_ + // *AtomicTypeValue_Boolean + // *AtomicTypeValue_Bytes + Value isAtomicTypeValue_Value `protobuf_oneof:"value"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *AtomicTypeValue) Reset() { + *x = AtomicTypeValue{} + mi := &file_org_apache_beam_model_pipeline_v1_schema_proto_msgTypes[12] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *AtomicTypeValue) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*AtomicTypeValue) ProtoMessage() {} + +func (x *AtomicTypeValue) ProtoReflect() protoreflect.Message { + mi := &file_org_apache_beam_model_pipeline_v1_schema_proto_msgTypes[12] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +func (x *AtomicTypeValue) GetValue() isAtomicTypeValue_Value { + if x != nil { + return x.Value + } + return nil +} + +func (x *AtomicTypeValue) GetByte() int32 { + if x != nil { + if x, ok := x.Value.(*AtomicTypeValue_Byte); ok { + return x.Byte + } + } + return 0 +} + +func (x *AtomicTypeValue) GetInt16() int32 { + if x != nil { + if x, ok := x.Value.(*AtomicTypeValue_Int16); ok { + return x.Int16 + } + } + return 0 +} + +func (x *AtomicTypeValue) GetInt32() int32 { + if x != nil { + if x, ok := x.Value.(*AtomicTypeValue_Int32); ok { + return x.Int32 + } + } + return 0 +} + +func (x *AtomicTypeValue) GetInt64() int64 { + if x != nil { + if x, ok := x.Value.(*AtomicTypeValue_Int64); ok { + return x.Int64 + } + } + return 0 +} + +func (x *AtomicTypeValue) GetFloat() float32 { + if x != nil { + if x, ok := x.Value.(*AtomicTypeValue_Float); ok { + return x.Float + } + } + return 0 } -func (m *FieldValue) GetFieldValue() isFieldValue_FieldValue { - if m != nil { - return m.FieldValue +func (x *AtomicTypeValue) GetDouble() float64 { + if x != nil { + if x, ok := x.Value.(*AtomicTypeValue_Double); ok { + return x.Double + } } - return nil + return 0 } -func (x *FieldValue) GetAtomicValue() *AtomicTypeValue { - if x, ok := x.GetFieldValue().(*FieldValue_AtomicValue); ok { - return x.AtomicValue +func (x *AtomicTypeValue) GetString() string { + if x != nil { + if x, ok := x.Value.(*AtomicTypeValue_String_); ok { + return x.String_ + } } - return nil + return "" } -func (x *FieldValue) GetArrayValue() *ArrayTypeValue { - if x, ok := x.GetFieldValue().(*FieldValue_ArrayValue); ok { - return x.ArrayValue - } - return nil +// Deprecated: Use GetString instead. +func (x *AtomicTypeValue) GetString_() string { + return x.GetString() } -func (x *FieldValue) GetIterableValue() *IterableTypeValue { - if x, ok := x.GetFieldValue().(*FieldValue_IterableValue); ok { - return x.IterableValue +func (x *AtomicTypeValue) GetBoolean() bool { + if x != nil { + if x, ok := x.Value.(*AtomicTypeValue_Boolean); ok { + return x.Boolean + } } - return nil + return false } -func (x *FieldValue) GetMapValue() *MapTypeValue { - if x, ok := x.GetFieldValue().(*FieldValue_MapValue); ok { - return x.MapValue +func (x *AtomicTypeValue) GetBytes() []byte { + if x != nil { + if x, ok := x.Value.(*AtomicTypeValue_Bytes); ok { + return x.Bytes + } } return nil } -func (x *FieldValue) GetRowValue() *Row { - if x, ok := x.GetFieldValue().(*FieldValue_RowValue); ok { - return x.RowValue - } - return nil +func (x *AtomicTypeValue) SetByte(v int32) { + x.Value = &AtomicTypeValue_Byte{v} } -func (x *FieldValue) GetLogicalTypeValue() *LogicalTypeValue { - if x, ok := x.GetFieldValue().(*FieldValue_LogicalTypeValue); ok { - return x.LogicalTypeValue - } - return nil +func (x *AtomicTypeValue) SetInt16(v int32) { + x.Value = &AtomicTypeValue_Int16{v} } -type isFieldValue_FieldValue interface { - isFieldValue_FieldValue() +func (x *AtomicTypeValue) SetInt32(v int32) { + x.Value = &AtomicTypeValue_Int32{v} } -type FieldValue_AtomicValue struct { - AtomicValue *AtomicTypeValue `protobuf:"bytes,1,opt,name=atomic_value,json=atomicValue,proto3,oneof"` +func (x *AtomicTypeValue) SetInt64(v int64) { + x.Value = &AtomicTypeValue_Int64{v} } -type FieldValue_ArrayValue struct { - ArrayValue *ArrayTypeValue `protobuf:"bytes,2,opt,name=array_value,json=arrayValue,proto3,oneof"` +func (x *AtomicTypeValue) SetFloat(v float32) { + x.Value = &AtomicTypeValue_Float{v} } -type FieldValue_IterableValue struct { - IterableValue *IterableTypeValue `protobuf:"bytes,3,opt,name=iterable_value,json=iterableValue,proto3,oneof"` +func (x *AtomicTypeValue) SetDouble(v float64) { + x.Value = &AtomicTypeValue_Double{v} } -type FieldValue_MapValue struct { - MapValue *MapTypeValue `protobuf:"bytes,4,opt,name=map_value,json=mapValue,proto3,oneof"` +func (x *AtomicTypeValue) SetString(v string) { + x.Value = &AtomicTypeValue_String_{v} } -type FieldValue_RowValue struct { - RowValue *Row `protobuf:"bytes,5,opt,name=row_value,json=rowValue,proto3,oneof"` +func (x *AtomicTypeValue) SetBoolean(v bool) { + x.Value = &AtomicTypeValue_Boolean{v} } -type FieldValue_LogicalTypeValue struct { - LogicalTypeValue *LogicalTypeValue `protobuf:"bytes,6,opt,name=logical_type_value,json=logicalTypeValue,proto3,oneof"` +func (x *AtomicTypeValue) SetBytes(v []byte) { + if v == nil { + v = []byte{} + } + x.Value = &AtomicTypeValue_Bytes{v} } -func (*FieldValue_AtomicValue) isFieldValue_FieldValue() {} +func (x *AtomicTypeValue) HasValue() bool { + if x == nil { + return false + } + return x.Value != nil +} -func (*FieldValue_ArrayValue) isFieldValue_FieldValue() {} +func (x *AtomicTypeValue) HasByte() bool { + if x == nil { + return false + } + _, ok := x.Value.(*AtomicTypeValue_Byte) + return ok +} -func (*FieldValue_IterableValue) isFieldValue_FieldValue() {} +func (x *AtomicTypeValue) HasInt16() bool { + if x == nil { + return false + } + _, ok := x.Value.(*AtomicTypeValue_Int16) + return ok +} -func (*FieldValue_MapValue) isFieldValue_FieldValue() {} +func (x *AtomicTypeValue) HasInt32() bool { + if x == nil { + return false + } + _, ok := x.Value.(*AtomicTypeValue_Int32) + return ok +} -func (*FieldValue_RowValue) isFieldValue_FieldValue() {} +func (x *AtomicTypeValue) HasInt64() bool { + if x == nil { + return false + } + _, ok := x.Value.(*AtomicTypeValue_Int64) + return ok +} -func (*FieldValue_LogicalTypeValue) isFieldValue_FieldValue() {} +func (x *AtomicTypeValue) HasFloat() bool { + if x == nil { + return false + } + _, ok := x.Value.(*AtomicTypeValue_Float) + return ok +} -type AtomicTypeValue struct { - state protoimpl.MessageState - sizeCache protoimpl.SizeCache - unknownFields protoimpl.UnknownFields +func (x *AtomicTypeValue) HasDouble() bool { + if x == nil { + return false + } + _, ok := x.Value.(*AtomicTypeValue_Double) + return ok +} - // Types that are assignable to Value: - // - // *AtomicTypeValue_Byte - // *AtomicTypeValue_Int16 - // *AtomicTypeValue_Int32 - // *AtomicTypeValue_Int64 - // *AtomicTypeValue_Float - // *AtomicTypeValue_Double - // *AtomicTypeValue_String_ - // *AtomicTypeValue_Boolean - // *AtomicTypeValue_Bytes - Value isAtomicTypeValue_Value `protobuf_oneof:"value"` +func (x *AtomicTypeValue) HasString() bool { + if x == nil { + return false + } + _, ok := x.Value.(*AtomicTypeValue_String_) + return ok } -func (x *AtomicTypeValue) Reset() { - *x = AtomicTypeValue{} - if protoimpl.UnsafeEnabled { - mi := &file_org_apache_beam_model_pipeline_v1_schema_proto_msgTypes[12] - ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) - ms.StoreMessageInfo(mi) +func (x *AtomicTypeValue) HasBoolean() bool { + if x == nil { + return false } + _, ok := x.Value.(*AtomicTypeValue_Boolean) + return ok } -func (x *AtomicTypeValue) String() string { - return protoimpl.X.MessageStringOf(x) +func (x *AtomicTypeValue) HasBytes() bool { + if x == nil { + return false + } + _, ok := x.Value.(*AtomicTypeValue_Bytes) + return ok } -func (*AtomicTypeValue) ProtoMessage() {} +func (x *AtomicTypeValue) ClearValue() { + x.Value = nil +} -func (x *AtomicTypeValue) ProtoReflect() protoreflect.Message { - mi := &file_org_apache_beam_model_pipeline_v1_schema_proto_msgTypes[12] - if protoimpl.UnsafeEnabled && x != nil { - ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) - if ms.LoadMessageInfo() == nil { - ms.StoreMessageInfo(mi) - } - return ms +func (x *AtomicTypeValue) ClearByte() { + if _, ok := x.Value.(*AtomicTypeValue_Byte); ok { + x.Value = nil } - return mi.MessageOf(x) } -// Deprecated: Use AtomicTypeValue.ProtoReflect.Descriptor instead. -func (*AtomicTypeValue) Descriptor() ([]byte, []int) { - return file_org_apache_beam_model_pipeline_v1_schema_proto_rawDescGZIP(), []int{12} +func (x *AtomicTypeValue) ClearInt16() { + if _, ok := x.Value.(*AtomicTypeValue_Int16); ok { + x.Value = nil + } } -func (m *AtomicTypeValue) GetValue() isAtomicTypeValue_Value { - if m != nil { - return m.Value +func (x *AtomicTypeValue) ClearInt32() { + if _, ok := x.Value.(*AtomicTypeValue_Int32); ok { + x.Value = nil } - return nil } -func (x *AtomicTypeValue) GetByte() int32 { - if x, ok := x.GetValue().(*AtomicTypeValue_Byte); ok { - return x.Byte +func (x *AtomicTypeValue) ClearInt64() { + if _, ok := x.Value.(*AtomicTypeValue_Int64); ok { + x.Value = nil } - return 0 } -func (x *AtomicTypeValue) GetInt16() int32 { - if x, ok := x.GetValue().(*AtomicTypeValue_Int16); ok { - return x.Int16 +func (x *AtomicTypeValue) ClearFloat() { + if _, ok := x.Value.(*AtomicTypeValue_Float); ok { + x.Value = nil } - return 0 } -func (x *AtomicTypeValue) GetInt32() int32 { - if x, ok := x.GetValue().(*AtomicTypeValue_Int32); ok { - return x.Int32 +func (x *AtomicTypeValue) ClearDouble() { + if _, ok := x.Value.(*AtomicTypeValue_Double); ok { + x.Value = nil } - return 0 } -func (x *AtomicTypeValue) GetInt64() int64 { - if x, ok := x.GetValue().(*AtomicTypeValue_Int64); ok { - return x.Int64 +func (x *AtomicTypeValue) ClearString() { + if _, ok := x.Value.(*AtomicTypeValue_String_); ok { + x.Value = nil } - return 0 } -func (x *AtomicTypeValue) GetFloat() float32 { - if x, ok := x.GetValue().(*AtomicTypeValue_Float); ok { - return x.Float +func (x *AtomicTypeValue) ClearBoolean() { + if _, ok := x.Value.(*AtomicTypeValue_Boolean); ok { + x.Value = nil } - return 0 } -func (x *AtomicTypeValue) GetDouble() float64 { - if x, ok := x.GetValue().(*AtomicTypeValue_Double); ok { - return x.Double +func (x *AtomicTypeValue) ClearBytes() { + if _, ok := x.Value.(*AtomicTypeValue_Bytes); ok { + x.Value = nil } - return 0 } -func (x *AtomicTypeValue) GetString_() string { - if x, ok := x.GetValue().(*AtomicTypeValue_String_); ok { - return x.String_ +const AtomicTypeValue_Value_not_set_case case_AtomicTypeValue_Value = 0 +const AtomicTypeValue_Byte_case case_AtomicTypeValue_Value = 1 +const AtomicTypeValue_Int16_case case_AtomicTypeValue_Value = 2 +const AtomicTypeValue_Int32_case case_AtomicTypeValue_Value = 3 +const AtomicTypeValue_Int64_case case_AtomicTypeValue_Value = 4 +const AtomicTypeValue_Float_case case_AtomicTypeValue_Value = 5 +const AtomicTypeValue_Double_case case_AtomicTypeValue_Value = 6 +const AtomicTypeValue_String__case case_AtomicTypeValue_Value = 7 +const AtomicTypeValue_Boolean_case case_AtomicTypeValue_Value = 8 +const AtomicTypeValue_Bytes_case case_AtomicTypeValue_Value = 9 + +func (x *AtomicTypeValue) WhichValue() case_AtomicTypeValue_Value { + if x == nil { + return AtomicTypeValue_Value_not_set_case + } + switch x.Value.(type) { + case *AtomicTypeValue_Byte: + return AtomicTypeValue_Byte_case + case *AtomicTypeValue_Int16: + return AtomicTypeValue_Int16_case + case *AtomicTypeValue_Int32: + return AtomicTypeValue_Int32_case + case *AtomicTypeValue_Int64: + return AtomicTypeValue_Int64_case + case *AtomicTypeValue_Float: + return AtomicTypeValue_Float_case + case *AtomicTypeValue_Double: + return AtomicTypeValue_Double_case + case *AtomicTypeValue_String_: + return AtomicTypeValue_String__case + case *AtomicTypeValue_Boolean: + return AtomicTypeValue_Boolean_case + case *AtomicTypeValue_Bytes: + return AtomicTypeValue_Bytes_case + default: + return AtomicTypeValue_Value_not_set_case } - return "" } -func (x *AtomicTypeValue) GetBoolean() bool { - if x, ok := x.GetValue().(*AtomicTypeValue_Boolean); ok { - return x.Boolean +type AtomicTypeValue_builder struct { + _ [0]func() // Prevents comparability and use of unkeyed literals for the builder. + + // Fields of oneof Value: + Byte *int32 + Int16 *int32 + Int32 *int32 + Int64 *int64 + Float *float32 + Double *float64 + String *string + Boolean *bool + Bytes []byte + // -- end of Value +} + +func (b0 AtomicTypeValue_builder) Build() *AtomicTypeValue { + m0 := &AtomicTypeValue{} + b, x := &b0, m0 + _, _ = b, x + if b.Byte != nil { + x.Value = &AtomicTypeValue_Byte{*b.Byte} } - return false + if b.Int16 != nil { + x.Value = &AtomicTypeValue_Int16{*b.Int16} + } + if b.Int32 != nil { + x.Value = &AtomicTypeValue_Int32{*b.Int32} + } + if b.Int64 != nil { + x.Value = &AtomicTypeValue_Int64{*b.Int64} + } + if b.Float != nil { + x.Value = &AtomicTypeValue_Float{*b.Float} + } + if b.Double != nil { + x.Value = &AtomicTypeValue_Double{*b.Double} + } + if b.String != nil { + x.Value = &AtomicTypeValue_String_{*b.String} + } + if b.Boolean != nil { + x.Value = &AtomicTypeValue_Boolean{*b.Boolean} + } + if b.Bytes != nil { + x.Value = &AtomicTypeValue_Bytes{b.Bytes} + } + return m0 } -func (x *AtomicTypeValue) GetBytes() []byte { - if x, ok := x.GetValue().(*AtomicTypeValue_Bytes); ok { - return x.Bytes +type case_AtomicTypeValue_Value protoreflect.FieldNumber + +func (x case_AtomicTypeValue_Value) String() string { + md := file_org_apache_beam_model_pipeline_v1_schema_proto_msgTypes[12].Descriptor() + if x == 0 { + return "not set" } - return nil + return protoimpl.X.MessageFieldStringOf(md, protoreflect.FieldNumber(x)) } type isAtomicTypeValue_Value interface { @@ -1276,20 +2336,17 @@ func (*AtomicTypeValue_Boolean) isAtomicTypeValue_Value() {} func (*AtomicTypeValue_Bytes) isAtomicTypeValue_Value() {} type ArrayTypeValue struct { - state protoimpl.MessageState - sizeCache protoimpl.SizeCache + state protoimpl.MessageState `protogen:"hybrid.v1"` + Element []*FieldValue `protobuf:"bytes,1,rep,name=element,proto3" json:"element,omitempty"` unknownFields protoimpl.UnknownFields - - Element []*FieldValue `protobuf:"bytes,1,rep,name=element,proto3" json:"element,omitempty"` + sizeCache protoimpl.SizeCache } func (x *ArrayTypeValue) Reset() { *x = ArrayTypeValue{} - if protoimpl.UnsafeEnabled { - mi := &file_org_apache_beam_model_pipeline_v1_schema_proto_msgTypes[13] - ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) - ms.StoreMessageInfo(mi) - } + mi := &file_org_apache_beam_model_pipeline_v1_schema_proto_msgTypes[13] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) } func (x *ArrayTypeValue) String() string { @@ -1300,7 +2357,7 @@ func (*ArrayTypeValue) ProtoMessage() {} func (x *ArrayTypeValue) ProtoReflect() protoreflect.Message { mi := &file_org_apache_beam_model_pipeline_v1_schema_proto_msgTypes[13] - if protoimpl.UnsafeEnabled && x != nil { + if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { ms.StoreMessageInfo(mi) @@ -1310,11 +2367,6 @@ func (x *ArrayTypeValue) ProtoReflect() protoreflect.Message { return mi.MessageOf(x) } -// Deprecated: Use ArrayTypeValue.ProtoReflect.Descriptor instead. -func (*ArrayTypeValue) Descriptor() ([]byte, []int) { - return file_org_apache_beam_model_pipeline_v1_schema_proto_rawDescGZIP(), []int{13} -} - func (x *ArrayTypeValue) GetElement() []*FieldValue { if x != nil { return x.Element @@ -1322,21 +2374,36 @@ func (x *ArrayTypeValue) GetElement() []*FieldValue { return nil } +func (x *ArrayTypeValue) SetElement(v []*FieldValue) { + x.Element = v +} + +type ArrayTypeValue_builder struct { + _ [0]func() // Prevents comparability and use of unkeyed literals for the builder. + + Element []*FieldValue +} + +func (b0 ArrayTypeValue_builder) Build() *ArrayTypeValue { + m0 := &ArrayTypeValue{} + b, x := &b0, m0 + _, _ = b, x + x.Element = b.Element + return m0 +} + type IterableTypeValue struct { - state protoimpl.MessageState - sizeCache protoimpl.SizeCache + state protoimpl.MessageState `protogen:"hybrid.v1"` + Element []*FieldValue `protobuf:"bytes,1,rep,name=element,proto3" json:"element,omitempty"` unknownFields protoimpl.UnknownFields - - Element []*FieldValue `protobuf:"bytes,1,rep,name=element,proto3" json:"element,omitempty"` + sizeCache protoimpl.SizeCache } func (x *IterableTypeValue) Reset() { *x = IterableTypeValue{} - if protoimpl.UnsafeEnabled { - mi := &file_org_apache_beam_model_pipeline_v1_schema_proto_msgTypes[14] - ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) - ms.StoreMessageInfo(mi) - } + mi := &file_org_apache_beam_model_pipeline_v1_schema_proto_msgTypes[14] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) } func (x *IterableTypeValue) String() string { @@ -1347,7 +2414,7 @@ func (*IterableTypeValue) ProtoMessage() {} func (x *IterableTypeValue) ProtoReflect() protoreflect.Message { mi := &file_org_apache_beam_model_pipeline_v1_schema_proto_msgTypes[14] - if protoimpl.UnsafeEnabled && x != nil { + if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { ms.StoreMessageInfo(mi) @@ -1357,11 +2424,6 @@ func (x *IterableTypeValue) ProtoReflect() protoreflect.Message { return mi.MessageOf(x) } -// Deprecated: Use IterableTypeValue.ProtoReflect.Descriptor instead. -func (*IterableTypeValue) Descriptor() ([]byte, []int) { - return file_org_apache_beam_model_pipeline_v1_schema_proto_rawDescGZIP(), []int{14} -} - func (x *IterableTypeValue) GetElement() []*FieldValue { if x != nil { return x.Element @@ -1369,21 +2431,36 @@ func (x *IterableTypeValue) GetElement() []*FieldValue { return nil } +func (x *IterableTypeValue) SetElement(v []*FieldValue) { + x.Element = v +} + +type IterableTypeValue_builder struct { + _ [0]func() // Prevents comparability and use of unkeyed literals for the builder. + + Element []*FieldValue +} + +func (b0 IterableTypeValue_builder) Build() *IterableTypeValue { + m0 := &IterableTypeValue{} + b, x := &b0, m0 + _, _ = b, x + x.Element = b.Element + return m0 +} + type MapTypeValue struct { - state protoimpl.MessageState - sizeCache protoimpl.SizeCache + state protoimpl.MessageState `protogen:"hybrid.v1"` + Entries []*MapTypeEntry `protobuf:"bytes,1,rep,name=entries,proto3" json:"entries,omitempty"` unknownFields protoimpl.UnknownFields - - Entries []*MapTypeEntry `protobuf:"bytes,1,rep,name=entries,proto3" json:"entries,omitempty"` + sizeCache protoimpl.SizeCache } func (x *MapTypeValue) Reset() { *x = MapTypeValue{} - if protoimpl.UnsafeEnabled { - mi := &file_org_apache_beam_model_pipeline_v1_schema_proto_msgTypes[15] - ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) - ms.StoreMessageInfo(mi) - } + mi := &file_org_apache_beam_model_pipeline_v1_schema_proto_msgTypes[15] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) } func (x *MapTypeValue) String() string { @@ -1394,7 +2471,7 @@ func (*MapTypeValue) ProtoMessage() {} func (x *MapTypeValue) ProtoReflect() protoreflect.Message { mi := &file_org_apache_beam_model_pipeline_v1_schema_proto_msgTypes[15] - if protoimpl.UnsafeEnabled && x != nil { + if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { ms.StoreMessageInfo(mi) @@ -1404,11 +2481,6 @@ func (x *MapTypeValue) ProtoReflect() protoreflect.Message { return mi.MessageOf(x) } -// Deprecated: Use MapTypeValue.ProtoReflect.Descriptor instead. -func (*MapTypeValue) Descriptor() ([]byte, []int) { - return file_org_apache_beam_model_pipeline_v1_schema_proto_rawDescGZIP(), []int{15} -} - func (x *MapTypeValue) GetEntries() []*MapTypeEntry { if x != nil { return x.Entries @@ -1416,22 +2488,37 @@ func (x *MapTypeValue) GetEntries() []*MapTypeEntry { return nil } +func (x *MapTypeValue) SetEntries(v []*MapTypeEntry) { + x.Entries = v +} + +type MapTypeValue_builder struct { + _ [0]func() // Prevents comparability and use of unkeyed literals for the builder. + + Entries []*MapTypeEntry +} + +func (b0 MapTypeValue_builder) Build() *MapTypeValue { + m0 := &MapTypeValue{} + b, x := &b0, m0 + _, _ = b, x + x.Entries = b.Entries + return m0 +} + type MapTypeEntry struct { - state protoimpl.MessageState - sizeCache protoimpl.SizeCache + state protoimpl.MessageState `protogen:"hybrid.v1"` + Key *FieldValue `protobuf:"bytes,1,opt,name=key,proto3" json:"key,omitempty"` + Value *FieldValue `protobuf:"bytes,2,opt,name=value,proto3" json:"value,omitempty"` unknownFields protoimpl.UnknownFields - - Key *FieldValue `protobuf:"bytes,1,opt,name=key,proto3" json:"key,omitempty"` - Value *FieldValue `protobuf:"bytes,2,opt,name=value,proto3" json:"value,omitempty"` + sizeCache protoimpl.SizeCache } func (x *MapTypeEntry) Reset() { *x = MapTypeEntry{} - if protoimpl.UnsafeEnabled { - mi := &file_org_apache_beam_model_pipeline_v1_schema_proto_msgTypes[16] - ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) - ms.StoreMessageInfo(mi) - } + mi := &file_org_apache_beam_model_pipeline_v1_schema_proto_msgTypes[16] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) } func (x *MapTypeEntry) String() string { @@ -1442,7 +2529,7 @@ func (*MapTypeEntry) ProtoMessage() {} func (x *MapTypeEntry) ProtoReflect() protoreflect.Message { mi := &file_org_apache_beam_model_pipeline_v1_schema_proto_msgTypes[16] - if protoimpl.UnsafeEnabled && x != nil { + if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { ms.StoreMessageInfo(mi) @@ -1452,11 +2539,6 @@ func (x *MapTypeEntry) ProtoReflect() protoreflect.Message { return mi.MessageOf(x) } -// Deprecated: Use MapTypeEntry.ProtoReflect.Descriptor instead. -func (*MapTypeEntry) Descriptor() ([]byte, []int) { - return file_org_apache_beam_model_pipeline_v1_schema_proto_rawDescGZIP(), []int{16} -} - func (x *MapTypeEntry) GetKey() *FieldValue { if x != nil { return x.Key @@ -1471,21 +2553,64 @@ func (x *MapTypeEntry) GetValue() *FieldValue { return nil } +func (x *MapTypeEntry) SetKey(v *FieldValue) { + x.Key = v +} + +func (x *MapTypeEntry) SetValue(v *FieldValue) { + x.Value = v +} + +func (x *MapTypeEntry) HasKey() bool { + if x == nil { + return false + } + return x.Key != nil +} + +func (x *MapTypeEntry) HasValue() bool { + if x == nil { + return false + } + return x.Value != nil +} + +func (x *MapTypeEntry) ClearKey() { + x.Key = nil +} + +func (x *MapTypeEntry) ClearValue() { + x.Value = nil +} + +type MapTypeEntry_builder struct { + _ [0]func() // Prevents comparability and use of unkeyed literals for the builder. + + Key *FieldValue + Value *FieldValue +} + +func (b0 MapTypeEntry_builder) Build() *MapTypeEntry { + m0 := &MapTypeEntry{} + b, x := &b0, m0 + _, _ = b, x + x.Key = b.Key + x.Value = b.Value + return m0 +} + type LogicalTypeValue struct { - state protoimpl.MessageState - sizeCache protoimpl.SizeCache + state protoimpl.MessageState `protogen:"hybrid.v1"` + Value *FieldValue `protobuf:"bytes,1,opt,name=value,proto3" json:"value,omitempty"` unknownFields protoimpl.UnknownFields - - Value *FieldValue `protobuf:"bytes,1,opt,name=value,proto3" json:"value,omitempty"` + sizeCache protoimpl.SizeCache } func (x *LogicalTypeValue) Reset() { *x = LogicalTypeValue{} - if protoimpl.UnsafeEnabled { - mi := &file_org_apache_beam_model_pipeline_v1_schema_proto_msgTypes[17] - ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) - ms.StoreMessageInfo(mi) - } + mi := &file_org_apache_beam_model_pipeline_v1_schema_proto_msgTypes[17] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) } func (x *LogicalTypeValue) String() string { @@ -1496,7 +2621,7 @@ func (*LogicalTypeValue) ProtoMessage() {} func (x *LogicalTypeValue) ProtoReflect() protoreflect.Message { mi := &file_org_apache_beam_model_pipeline_v1_schema_proto_msgTypes[17] - if protoimpl.UnsafeEnabled && x != nil { + if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { ms.StoreMessageInfo(mi) @@ -1506,11 +2631,6 @@ func (x *LogicalTypeValue) ProtoReflect() protoreflect.Message { return mi.MessageOf(x) } -// Deprecated: Use LogicalTypeValue.ProtoReflect.Descriptor instead. -func (*LogicalTypeValue) Descriptor() ([]byte, []int) { - return file_org_apache_beam_model_pipeline_v1_schema_proto_rawDescGZIP(), []int{17} -} - func (x *LogicalTypeValue) GetValue() *FieldValue { if x != nil { return x.Value @@ -1518,6 +2638,35 @@ func (x *LogicalTypeValue) GetValue() *FieldValue { return nil } +func (x *LogicalTypeValue) SetValue(v *FieldValue) { + x.Value = v +} + +func (x *LogicalTypeValue) HasValue() bool { + if x == nil { + return false + } + return x.Value != nil +} + +func (x *LogicalTypeValue) ClearValue() { + x.Value = nil +} + +type LogicalTypeValue_builder struct { + _ [0]func() // Prevents comparability and use of unkeyed literals for the builder. + + Value *FieldValue +} + +func (b0 LogicalTypeValue_builder) Build() *LogicalTypeValue { + m0 := &LogicalTypeValue{} + b, x := &b0, m0 + _, _ = b, x + x.Value = b.Value + return m0 +} + var File_org_apache_beam_model_pipeline_v1_schema_proto protoreflect.FileDescriptor var file_org_apache_beam_model_pipeline_v1_schema_proto_rawDesc = []byte{ @@ -1786,21 +2935,9 @@ var file_org_apache_beam_model_pipeline_v1_schema_proto_rawDesc = []byte{ 0x74, 0x6f, 0x33, } -var ( - file_org_apache_beam_model_pipeline_v1_schema_proto_rawDescOnce sync.Once - file_org_apache_beam_model_pipeline_v1_schema_proto_rawDescData = file_org_apache_beam_model_pipeline_v1_schema_proto_rawDesc -) - -func file_org_apache_beam_model_pipeline_v1_schema_proto_rawDescGZIP() []byte { - file_org_apache_beam_model_pipeline_v1_schema_proto_rawDescOnce.Do(func() { - file_org_apache_beam_model_pipeline_v1_schema_proto_rawDescData = protoimpl.X.CompressGZIP(file_org_apache_beam_model_pipeline_v1_schema_proto_rawDescData) - }) - return file_org_apache_beam_model_pipeline_v1_schema_proto_rawDescData -} - var file_org_apache_beam_model_pipeline_v1_schema_proto_enumTypes = make([]protoimpl.EnumInfo, 2) var file_org_apache_beam_model_pipeline_v1_schema_proto_msgTypes = make([]protoimpl.MessageInfo, 18) -var file_org_apache_beam_model_pipeline_v1_schema_proto_goTypes = []interface{}{ +var file_org_apache_beam_model_pipeline_v1_schema_proto_goTypes = []any{ (AtomicType)(0), // 0: org.apache.beam.model.pipeline.v1.AtomicType (LogicalTypes_Enum)(0), // 1: org.apache.beam.model.pipeline.v1.LogicalTypes.Enum (*Schema)(nil), // 2: org.apache.beam.model.pipeline.v1.Schema @@ -1869,225 +3006,7 @@ func file_org_apache_beam_model_pipeline_v1_schema_proto_init() { return } file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_init() - if !protoimpl.UnsafeEnabled { - file_org_apache_beam_model_pipeline_v1_schema_proto_msgTypes[0].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*Schema); i { - case 0: - return &v.state - case 1: - return &v.sizeCache - case 2: - return &v.unknownFields - default: - return nil - } - } - file_org_apache_beam_model_pipeline_v1_schema_proto_msgTypes[1].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*Field); i { - case 0: - return &v.state - case 1: - return &v.sizeCache - case 2: - return &v.unknownFields - default: - return nil - } - } - file_org_apache_beam_model_pipeline_v1_schema_proto_msgTypes[2].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*FieldType); i { - case 0: - return &v.state - case 1: - return &v.sizeCache - case 2: - return &v.unknownFields - default: - return nil - } - } - file_org_apache_beam_model_pipeline_v1_schema_proto_msgTypes[3].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*ArrayType); i { - case 0: - return &v.state - case 1: - return &v.sizeCache - case 2: - return &v.unknownFields - default: - return nil - } - } - file_org_apache_beam_model_pipeline_v1_schema_proto_msgTypes[4].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*IterableType); i { - case 0: - return &v.state - case 1: - return &v.sizeCache - case 2: - return &v.unknownFields - default: - return nil - } - } - file_org_apache_beam_model_pipeline_v1_schema_proto_msgTypes[5].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*MapType); i { - case 0: - return &v.state - case 1: - return &v.sizeCache - case 2: - return &v.unknownFields - default: - return nil - } - } - file_org_apache_beam_model_pipeline_v1_schema_proto_msgTypes[6].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*RowType); i { - case 0: - return &v.state - case 1: - return &v.sizeCache - case 2: - return &v.unknownFields - default: - return nil - } - } - file_org_apache_beam_model_pipeline_v1_schema_proto_msgTypes[7].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*LogicalType); i { - case 0: - return &v.state - case 1: - return &v.sizeCache - case 2: - return &v.unknownFields - default: - return nil - } - } - file_org_apache_beam_model_pipeline_v1_schema_proto_msgTypes[8].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*LogicalTypes); i { - case 0: - return &v.state - case 1: - return &v.sizeCache - case 2: - return &v.unknownFields - default: - return nil - } - } - file_org_apache_beam_model_pipeline_v1_schema_proto_msgTypes[9].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*Option); i { - case 0: - return &v.state - case 1: - return &v.sizeCache - case 2: - return &v.unknownFields - default: - return nil - } - } - file_org_apache_beam_model_pipeline_v1_schema_proto_msgTypes[10].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*Row); i { - case 0: - return &v.state - case 1: - return &v.sizeCache - case 2: - return &v.unknownFields - default: - return nil - } - } - file_org_apache_beam_model_pipeline_v1_schema_proto_msgTypes[11].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*FieldValue); i { - case 0: - return &v.state - case 1: - return &v.sizeCache - case 2: - return &v.unknownFields - default: - return nil - } - } - file_org_apache_beam_model_pipeline_v1_schema_proto_msgTypes[12].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*AtomicTypeValue); i { - case 0: - return &v.state - case 1: - return &v.sizeCache - case 2: - return &v.unknownFields - default: - return nil - } - } - file_org_apache_beam_model_pipeline_v1_schema_proto_msgTypes[13].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*ArrayTypeValue); i { - case 0: - return &v.state - case 1: - return &v.sizeCache - case 2: - return &v.unknownFields - default: - return nil - } - } - file_org_apache_beam_model_pipeline_v1_schema_proto_msgTypes[14].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*IterableTypeValue); i { - case 0: - return &v.state - case 1: - return &v.sizeCache - case 2: - return &v.unknownFields - default: - return nil - } - } - file_org_apache_beam_model_pipeline_v1_schema_proto_msgTypes[15].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*MapTypeValue); i { - case 0: - return &v.state - case 1: - return &v.sizeCache - case 2: - return &v.unknownFields - default: - return nil - } - } - file_org_apache_beam_model_pipeline_v1_schema_proto_msgTypes[16].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*MapTypeEntry); i { - case 0: - return &v.state - case 1: - return &v.sizeCache - case 2: - return &v.unknownFields - default: - return nil - } - } - file_org_apache_beam_model_pipeline_v1_schema_proto_msgTypes[17].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*LogicalTypeValue); i { - case 0: - return &v.state - case 1: - return &v.sizeCache - case 2: - return &v.unknownFields - default: - return nil - } - } - } - file_org_apache_beam_model_pipeline_v1_schema_proto_msgTypes[2].OneofWrappers = []interface{}{ + file_org_apache_beam_model_pipeline_v1_schema_proto_msgTypes[2].OneofWrappers = []any{ (*FieldType_AtomicType)(nil), (*FieldType_ArrayType)(nil), (*FieldType_IterableType)(nil), @@ -2095,7 +3014,7 @@ func file_org_apache_beam_model_pipeline_v1_schema_proto_init() { (*FieldType_RowType)(nil), (*FieldType_LogicalType)(nil), } - file_org_apache_beam_model_pipeline_v1_schema_proto_msgTypes[11].OneofWrappers = []interface{}{ + file_org_apache_beam_model_pipeline_v1_schema_proto_msgTypes[11].OneofWrappers = []any{ (*FieldValue_AtomicValue)(nil), (*FieldValue_ArrayValue)(nil), (*FieldValue_IterableValue)(nil), @@ -2103,7 +3022,7 @@ func file_org_apache_beam_model_pipeline_v1_schema_proto_init() { (*FieldValue_RowValue)(nil), (*FieldValue_LogicalTypeValue)(nil), } - file_org_apache_beam_model_pipeline_v1_schema_proto_msgTypes[12].OneofWrappers = []interface{}{ + file_org_apache_beam_model_pipeline_v1_schema_proto_msgTypes[12].OneofWrappers = []any{ (*AtomicTypeValue_Byte)(nil), (*AtomicTypeValue_Int16)(nil), (*AtomicTypeValue_Int32)(nil), diff --git a/sdks/go/pkg/beam/model/pipeline_v1/schema_protoopaque.pb.go b/sdks/go/pkg/beam/model/pipeline_v1/schema_protoopaque.pb.go new file mode 100644 index 000000000000..2465a7b3e284 --- /dev/null +++ b/sdks/go/pkg/beam/model/pipeline_v1/schema_protoopaque.pb.go @@ -0,0 +1,2998 @@ +// +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +// Protocol Buffers describing Beam Schemas, a portable representation for +// complex types. +// +// The primary application of Schema is as the payload for the standard coder +// "beam:coder:row:v1", defined in beam_runner_api.proto + +// Code generated by protoc-gen-go. DO NOT EDIT. +// versions: +// protoc-gen-go v1.36.0 +// protoc v5.27.3 +// source: org/apache/beam/model/pipeline/v1/schema.proto + +//go:build protoopaque + +package pipeline_v1 + +import ( + protoreflect "google.golang.org/protobuf/reflect/protoreflect" + protoimpl "google.golang.org/protobuf/runtime/protoimpl" + reflect "reflect" +) + +const ( + // Verify that this generated code is sufficiently up-to-date. + _ = protoimpl.EnforceVersion(20 - protoimpl.MinVersion) + // Verify that runtime/protoimpl is sufficiently up-to-date. + _ = protoimpl.EnforceVersion(protoimpl.MaxVersion - 20) +) + +type AtomicType int32 + +const ( + AtomicType_UNSPECIFIED AtomicType = 0 + AtomicType_BYTE AtomicType = 1 + AtomicType_INT16 AtomicType = 2 + AtomicType_INT32 AtomicType = 3 + AtomicType_INT64 AtomicType = 4 + AtomicType_FLOAT AtomicType = 5 + AtomicType_DOUBLE AtomicType = 6 + AtomicType_STRING AtomicType = 7 + AtomicType_BOOLEAN AtomicType = 8 + AtomicType_BYTES AtomicType = 9 +) + +// Enum value maps for AtomicType. +var ( + AtomicType_name = map[int32]string{ + 0: "UNSPECIFIED", + 1: "BYTE", + 2: "INT16", + 3: "INT32", + 4: "INT64", + 5: "FLOAT", + 6: "DOUBLE", + 7: "STRING", + 8: "BOOLEAN", + 9: "BYTES", + } + AtomicType_value = map[string]int32{ + "UNSPECIFIED": 0, + "BYTE": 1, + "INT16": 2, + "INT32": 3, + "INT64": 4, + "FLOAT": 5, + "DOUBLE": 6, + "STRING": 7, + "BOOLEAN": 8, + "BYTES": 9, + } +) + +func (x AtomicType) Enum() *AtomicType { + p := new(AtomicType) + *p = x + return p +} + +func (x AtomicType) String() string { + return protoimpl.X.EnumStringOf(x.Descriptor(), protoreflect.EnumNumber(x)) +} + +func (AtomicType) Descriptor() protoreflect.EnumDescriptor { + return file_org_apache_beam_model_pipeline_v1_schema_proto_enumTypes[0].Descriptor() +} + +func (AtomicType) Type() protoreflect.EnumType { + return &file_org_apache_beam_model_pipeline_v1_schema_proto_enumTypes[0] +} + +func (x AtomicType) Number() protoreflect.EnumNumber { + return protoreflect.EnumNumber(x) +} + +type LogicalTypes_Enum int32 + +const ( + // A URN for Python Callable logical type + // - Representation type: STRING + // - Language type: In Python SDK, PythonCallableWithSource. + // In any other SDKs, a wrapper object for a string which + // can be evaluated to a Python Callable object. + LogicalTypes_PYTHON_CALLABLE LogicalTypes_Enum = 0 + // A URN for MicrosInstant type + // - Representation type: ROW + // - A timestamp without a timezone where seconds + micros represents the + // amount of time since the epoch. + LogicalTypes_MICROS_INSTANT LogicalTypes_Enum = 1 + // A URN for MillisInstant type + // - Representation type: INT64 + // - A timestamp without a timezone represented by the number of + // milliseconds since the epoch. The INT64 value is encoded with + // big-endian shifted such that lexicographic ordering of the bytes + // corresponds to chronological order. + LogicalTypes_MILLIS_INSTANT LogicalTypes_Enum = 2 + // A URN for Decimal type + // - Representation type: BYTES + // - A decimal number with variable scale. Its BYTES + // representation consists of an integer (INT32) scale followed by a + // two's complement encoded big integer. + LogicalTypes_DECIMAL LogicalTypes_Enum = 3 + // A URN for FixedLengthBytes type + // - Representation type: BYTES + // - Argument type: INT32. + // A fixed-length bytes with its length as the argument. + LogicalTypes_FIXED_BYTES LogicalTypes_Enum = 4 + // A URN for VariableLengthBytes type + // - Representation type: BYTES + // - Argument type: INT32. + // A variable-length bytes with its maximum length as the argument. + LogicalTypes_VAR_BYTES LogicalTypes_Enum = 5 + // A URN for FixedLengthString type + // - Representation type: STRING + // - Argument type: INT32. + // A fixed-length string with its length as the argument. + LogicalTypes_FIXED_CHAR LogicalTypes_Enum = 6 + // A URN for VariableLengthString type + // - Representation type: STRING + // - Argument type: INT32. + // A variable-length string with its maximum length as the argument. + LogicalTypes_VAR_CHAR LogicalTypes_Enum = 7 +) + +// Enum value maps for LogicalTypes_Enum. +var ( + LogicalTypes_Enum_name = map[int32]string{ + 0: "PYTHON_CALLABLE", + 1: "MICROS_INSTANT", + 2: "MILLIS_INSTANT", + 3: "DECIMAL", + 4: "FIXED_BYTES", + 5: "VAR_BYTES", + 6: "FIXED_CHAR", + 7: "VAR_CHAR", + } + LogicalTypes_Enum_value = map[string]int32{ + "PYTHON_CALLABLE": 0, + "MICROS_INSTANT": 1, + "MILLIS_INSTANT": 2, + "DECIMAL": 3, + "FIXED_BYTES": 4, + "VAR_BYTES": 5, + "FIXED_CHAR": 6, + "VAR_CHAR": 7, + } +) + +func (x LogicalTypes_Enum) Enum() *LogicalTypes_Enum { + p := new(LogicalTypes_Enum) + *p = x + return p +} + +func (x LogicalTypes_Enum) String() string { + return protoimpl.X.EnumStringOf(x.Descriptor(), protoreflect.EnumNumber(x)) +} + +func (LogicalTypes_Enum) Descriptor() protoreflect.EnumDescriptor { + return file_org_apache_beam_model_pipeline_v1_schema_proto_enumTypes[1].Descriptor() +} + +func (LogicalTypes_Enum) Type() protoreflect.EnumType { + return &file_org_apache_beam_model_pipeline_v1_schema_proto_enumTypes[1] +} + +func (x LogicalTypes_Enum) Number() protoreflect.EnumNumber { + return protoreflect.EnumNumber(x) +} + +type Schema struct { + state protoimpl.MessageState `protogen:"opaque.v1"` + xxx_hidden_Fields *[]*Field `protobuf:"bytes,1,rep,name=fields,proto3" json:"fields,omitempty"` + xxx_hidden_Id string `protobuf:"bytes,2,opt,name=id,proto3" json:"id,omitempty"` + xxx_hidden_Options *[]*Option `protobuf:"bytes,3,rep,name=options,proto3" json:"options,omitempty"` + xxx_hidden_EncodingPositionsSet bool `protobuf:"varint,4,opt,name=encoding_positions_set,json=encodingPositionsSet,proto3" json:"encoding_positions_set,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *Schema) Reset() { + *x = Schema{} + mi := &file_org_apache_beam_model_pipeline_v1_schema_proto_msgTypes[0] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *Schema) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*Schema) ProtoMessage() {} + +func (x *Schema) ProtoReflect() protoreflect.Message { + mi := &file_org_apache_beam_model_pipeline_v1_schema_proto_msgTypes[0] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +func (x *Schema) GetFields() []*Field { + if x != nil { + if x.xxx_hidden_Fields != nil { + return *x.xxx_hidden_Fields + } + } + return nil +} + +func (x *Schema) GetId() string { + if x != nil { + return x.xxx_hidden_Id + } + return "" +} + +func (x *Schema) GetOptions() []*Option { + if x != nil { + if x.xxx_hidden_Options != nil { + return *x.xxx_hidden_Options + } + } + return nil +} + +func (x *Schema) GetEncodingPositionsSet() bool { + if x != nil { + return x.xxx_hidden_EncodingPositionsSet + } + return false +} + +func (x *Schema) SetFields(v []*Field) { + x.xxx_hidden_Fields = &v +} + +func (x *Schema) SetId(v string) { + x.xxx_hidden_Id = v +} + +func (x *Schema) SetOptions(v []*Option) { + x.xxx_hidden_Options = &v +} + +func (x *Schema) SetEncodingPositionsSet(v bool) { + x.xxx_hidden_EncodingPositionsSet = v +} + +type Schema_builder struct { + _ [0]func() // Prevents comparability and use of unkeyed literals for the builder. + + // List of fields for this schema. Two fields may not share a name. + Fields []*Field + // REQUIRED. An RFC 4122 UUID. + Id string + Options []*Option + // Indicates that encoding positions have been overridden. + EncodingPositionsSet bool +} + +func (b0 Schema_builder) Build() *Schema { + m0 := &Schema{} + b, x := &b0, m0 + _, _ = b, x + x.xxx_hidden_Fields = &b.Fields + x.xxx_hidden_Id = b.Id + x.xxx_hidden_Options = &b.Options + x.xxx_hidden_EncodingPositionsSet = b.EncodingPositionsSet + return m0 +} + +type Field struct { + state protoimpl.MessageState `protogen:"opaque.v1"` + xxx_hidden_Name string `protobuf:"bytes,1,opt,name=name,proto3" json:"name,omitempty"` + xxx_hidden_Description string `protobuf:"bytes,2,opt,name=description,proto3" json:"description,omitempty"` + xxx_hidden_Type *FieldType `protobuf:"bytes,3,opt,name=type,proto3" json:"type,omitempty"` + xxx_hidden_Id int32 `protobuf:"varint,4,opt,name=id,proto3" json:"id,omitempty"` + xxx_hidden_EncodingPosition int32 `protobuf:"varint,5,opt,name=encoding_position,json=encodingPosition,proto3" json:"encoding_position,omitempty"` + xxx_hidden_Options *[]*Option `protobuf:"bytes,6,rep,name=options,proto3" json:"options,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *Field) Reset() { + *x = Field{} + mi := &file_org_apache_beam_model_pipeline_v1_schema_proto_msgTypes[1] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *Field) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*Field) ProtoMessage() {} + +func (x *Field) ProtoReflect() protoreflect.Message { + mi := &file_org_apache_beam_model_pipeline_v1_schema_proto_msgTypes[1] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +func (x *Field) GetName() string { + if x != nil { + return x.xxx_hidden_Name + } + return "" +} + +func (x *Field) GetDescription() string { + if x != nil { + return x.xxx_hidden_Description + } + return "" +} + +func (x *Field) GetType() *FieldType { + if x != nil { + return x.xxx_hidden_Type + } + return nil +} + +func (x *Field) GetId() int32 { + if x != nil { + return x.xxx_hidden_Id + } + return 0 +} + +func (x *Field) GetEncodingPosition() int32 { + if x != nil { + return x.xxx_hidden_EncodingPosition + } + return 0 +} + +func (x *Field) GetOptions() []*Option { + if x != nil { + if x.xxx_hidden_Options != nil { + return *x.xxx_hidden_Options + } + } + return nil +} + +func (x *Field) SetName(v string) { + x.xxx_hidden_Name = v +} + +func (x *Field) SetDescription(v string) { + x.xxx_hidden_Description = v +} + +func (x *Field) SetType(v *FieldType) { + x.xxx_hidden_Type = v +} + +func (x *Field) SetId(v int32) { + x.xxx_hidden_Id = v +} + +func (x *Field) SetEncodingPosition(v int32) { + x.xxx_hidden_EncodingPosition = v +} + +func (x *Field) SetOptions(v []*Option) { + x.xxx_hidden_Options = &v +} + +func (x *Field) HasType() bool { + if x == nil { + return false + } + return x.xxx_hidden_Type != nil +} + +func (x *Field) ClearType() { + x.xxx_hidden_Type = nil +} + +type Field_builder struct { + _ [0]func() // Prevents comparability and use of unkeyed literals for the builder. + + // REQUIRED. Name of this field within the schema. + Name string + // OPTIONAL. Human readable description of this field, such as the query that generated it. + Description string + Type *FieldType + Id int32 + // OPTIONAL. The position of this field's data when encoded, e.g. with beam:coder:row:v1. + // Either no fields in a given row are have encoding position populated, + // or all of them are. Used to support backwards compatibility with schema + // changes. + // If no fields have encoding position populated the order of encoding is the same as the order in the Schema. + // If this Field is part of a Schema where encoding_positions_set is True then encoding_position must be + // defined, otherwise this field is ignored. + EncodingPosition int32 + Options []*Option +} + +func (b0 Field_builder) Build() *Field { + m0 := &Field{} + b, x := &b0, m0 + _, _ = b, x + x.xxx_hidden_Name = b.Name + x.xxx_hidden_Description = b.Description + x.xxx_hidden_Type = b.Type + x.xxx_hidden_Id = b.Id + x.xxx_hidden_EncodingPosition = b.EncodingPosition + x.xxx_hidden_Options = &b.Options + return m0 +} + +type FieldType struct { + state protoimpl.MessageState `protogen:"opaque.v1"` + xxx_hidden_Nullable bool `protobuf:"varint,1,opt,name=nullable,proto3" json:"nullable,omitempty"` + xxx_hidden_TypeInfo isFieldType_TypeInfo `protobuf_oneof:"type_info"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *FieldType) Reset() { + *x = FieldType{} + mi := &file_org_apache_beam_model_pipeline_v1_schema_proto_msgTypes[2] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *FieldType) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*FieldType) ProtoMessage() {} + +func (x *FieldType) ProtoReflect() protoreflect.Message { + mi := &file_org_apache_beam_model_pipeline_v1_schema_proto_msgTypes[2] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +func (x *FieldType) GetNullable() bool { + if x != nil { + return x.xxx_hidden_Nullable + } + return false +} + +func (x *FieldType) GetAtomicType() AtomicType { + if x != nil { + if x, ok := x.xxx_hidden_TypeInfo.(*fieldType_AtomicType); ok { + return x.AtomicType + } + } + return AtomicType_UNSPECIFIED +} + +func (x *FieldType) GetArrayType() *ArrayType { + if x != nil { + if x, ok := x.xxx_hidden_TypeInfo.(*fieldType_ArrayType); ok { + return x.ArrayType + } + } + return nil +} + +func (x *FieldType) GetIterableType() *IterableType { + if x != nil { + if x, ok := x.xxx_hidden_TypeInfo.(*fieldType_IterableType); ok { + return x.IterableType + } + } + return nil +} + +func (x *FieldType) GetMapType() *MapType { + if x != nil { + if x, ok := x.xxx_hidden_TypeInfo.(*fieldType_MapType); ok { + return x.MapType + } + } + return nil +} + +func (x *FieldType) GetRowType() *RowType { + if x != nil { + if x, ok := x.xxx_hidden_TypeInfo.(*fieldType_RowType); ok { + return x.RowType + } + } + return nil +} + +func (x *FieldType) GetLogicalType() *LogicalType { + if x != nil { + if x, ok := x.xxx_hidden_TypeInfo.(*fieldType_LogicalType); ok { + return x.LogicalType + } + } + return nil +} + +func (x *FieldType) SetNullable(v bool) { + x.xxx_hidden_Nullable = v +} + +func (x *FieldType) SetAtomicType(v AtomicType) { + x.xxx_hidden_TypeInfo = &fieldType_AtomicType{v} +} + +func (x *FieldType) SetArrayType(v *ArrayType) { + if v == nil { + x.xxx_hidden_TypeInfo = nil + return + } + x.xxx_hidden_TypeInfo = &fieldType_ArrayType{v} +} + +func (x *FieldType) SetIterableType(v *IterableType) { + if v == nil { + x.xxx_hidden_TypeInfo = nil + return + } + x.xxx_hidden_TypeInfo = &fieldType_IterableType{v} +} + +func (x *FieldType) SetMapType(v *MapType) { + if v == nil { + x.xxx_hidden_TypeInfo = nil + return + } + x.xxx_hidden_TypeInfo = &fieldType_MapType{v} +} + +func (x *FieldType) SetRowType(v *RowType) { + if v == nil { + x.xxx_hidden_TypeInfo = nil + return + } + x.xxx_hidden_TypeInfo = &fieldType_RowType{v} +} + +func (x *FieldType) SetLogicalType(v *LogicalType) { + if v == nil { + x.xxx_hidden_TypeInfo = nil + return + } + x.xxx_hidden_TypeInfo = &fieldType_LogicalType{v} +} + +func (x *FieldType) HasTypeInfo() bool { + if x == nil { + return false + } + return x.xxx_hidden_TypeInfo != nil +} + +func (x *FieldType) HasAtomicType() bool { + if x == nil { + return false + } + _, ok := x.xxx_hidden_TypeInfo.(*fieldType_AtomicType) + return ok +} + +func (x *FieldType) HasArrayType() bool { + if x == nil { + return false + } + _, ok := x.xxx_hidden_TypeInfo.(*fieldType_ArrayType) + return ok +} + +func (x *FieldType) HasIterableType() bool { + if x == nil { + return false + } + _, ok := x.xxx_hidden_TypeInfo.(*fieldType_IterableType) + return ok +} + +func (x *FieldType) HasMapType() bool { + if x == nil { + return false + } + _, ok := x.xxx_hidden_TypeInfo.(*fieldType_MapType) + return ok +} + +func (x *FieldType) HasRowType() bool { + if x == nil { + return false + } + _, ok := x.xxx_hidden_TypeInfo.(*fieldType_RowType) + return ok +} + +func (x *FieldType) HasLogicalType() bool { + if x == nil { + return false + } + _, ok := x.xxx_hidden_TypeInfo.(*fieldType_LogicalType) + return ok +} + +func (x *FieldType) ClearTypeInfo() { + x.xxx_hidden_TypeInfo = nil +} + +func (x *FieldType) ClearAtomicType() { + if _, ok := x.xxx_hidden_TypeInfo.(*fieldType_AtomicType); ok { + x.xxx_hidden_TypeInfo = nil + } +} + +func (x *FieldType) ClearArrayType() { + if _, ok := x.xxx_hidden_TypeInfo.(*fieldType_ArrayType); ok { + x.xxx_hidden_TypeInfo = nil + } +} + +func (x *FieldType) ClearIterableType() { + if _, ok := x.xxx_hidden_TypeInfo.(*fieldType_IterableType); ok { + x.xxx_hidden_TypeInfo = nil + } +} + +func (x *FieldType) ClearMapType() { + if _, ok := x.xxx_hidden_TypeInfo.(*fieldType_MapType); ok { + x.xxx_hidden_TypeInfo = nil + } +} + +func (x *FieldType) ClearRowType() { + if _, ok := x.xxx_hidden_TypeInfo.(*fieldType_RowType); ok { + x.xxx_hidden_TypeInfo = nil + } +} + +func (x *FieldType) ClearLogicalType() { + if _, ok := x.xxx_hidden_TypeInfo.(*fieldType_LogicalType); ok { + x.xxx_hidden_TypeInfo = nil + } +} + +const FieldType_TypeInfo_not_set_case case_FieldType_TypeInfo = 0 +const FieldType_AtomicType_case case_FieldType_TypeInfo = 2 +const FieldType_ArrayType_case case_FieldType_TypeInfo = 3 +const FieldType_IterableType_case case_FieldType_TypeInfo = 4 +const FieldType_MapType_case case_FieldType_TypeInfo = 5 +const FieldType_RowType_case case_FieldType_TypeInfo = 6 +const FieldType_LogicalType_case case_FieldType_TypeInfo = 7 + +func (x *FieldType) WhichTypeInfo() case_FieldType_TypeInfo { + if x == nil { + return FieldType_TypeInfo_not_set_case + } + switch x.xxx_hidden_TypeInfo.(type) { + case *fieldType_AtomicType: + return FieldType_AtomicType_case + case *fieldType_ArrayType: + return FieldType_ArrayType_case + case *fieldType_IterableType: + return FieldType_IterableType_case + case *fieldType_MapType: + return FieldType_MapType_case + case *fieldType_RowType: + return FieldType_RowType_case + case *fieldType_LogicalType: + return FieldType_LogicalType_case + default: + return FieldType_TypeInfo_not_set_case + } +} + +type FieldType_builder struct { + _ [0]func() // Prevents comparability and use of unkeyed literals for the builder. + + Nullable bool + // Fields of oneof xxx_hidden_TypeInfo: + AtomicType *AtomicType + ArrayType *ArrayType + IterableType *IterableType + MapType *MapType + RowType *RowType + LogicalType *LogicalType + // -- end of xxx_hidden_TypeInfo +} + +func (b0 FieldType_builder) Build() *FieldType { + m0 := &FieldType{} + b, x := &b0, m0 + _, _ = b, x + x.xxx_hidden_Nullable = b.Nullable + if b.AtomicType != nil { + x.xxx_hidden_TypeInfo = &fieldType_AtomicType{*b.AtomicType} + } + if b.ArrayType != nil { + x.xxx_hidden_TypeInfo = &fieldType_ArrayType{b.ArrayType} + } + if b.IterableType != nil { + x.xxx_hidden_TypeInfo = &fieldType_IterableType{b.IterableType} + } + if b.MapType != nil { + x.xxx_hidden_TypeInfo = &fieldType_MapType{b.MapType} + } + if b.RowType != nil { + x.xxx_hidden_TypeInfo = &fieldType_RowType{b.RowType} + } + if b.LogicalType != nil { + x.xxx_hidden_TypeInfo = &fieldType_LogicalType{b.LogicalType} + } + return m0 +} + +type case_FieldType_TypeInfo protoreflect.FieldNumber + +func (x case_FieldType_TypeInfo) String() string { + md := file_org_apache_beam_model_pipeline_v1_schema_proto_msgTypes[2].Descriptor() + if x == 0 { + return "not set" + } + return protoimpl.X.MessageFieldStringOf(md, protoreflect.FieldNumber(x)) +} + +type isFieldType_TypeInfo interface { + isFieldType_TypeInfo() +} + +type fieldType_AtomicType struct { + AtomicType AtomicType `protobuf:"varint,2,opt,name=atomic_type,json=atomicType,proto3,enum=org.apache.beam.model.pipeline.v1.AtomicType,oneof"` +} + +type fieldType_ArrayType struct { + ArrayType *ArrayType `protobuf:"bytes,3,opt,name=array_type,json=arrayType,proto3,oneof"` +} + +type fieldType_IterableType struct { + IterableType *IterableType `protobuf:"bytes,4,opt,name=iterable_type,json=iterableType,proto3,oneof"` +} + +type fieldType_MapType struct { + MapType *MapType `protobuf:"bytes,5,opt,name=map_type,json=mapType,proto3,oneof"` +} + +type fieldType_RowType struct { + RowType *RowType `protobuf:"bytes,6,opt,name=row_type,json=rowType,proto3,oneof"` +} + +type fieldType_LogicalType struct { + LogicalType *LogicalType `protobuf:"bytes,7,opt,name=logical_type,json=logicalType,proto3,oneof"` +} + +func (*fieldType_AtomicType) isFieldType_TypeInfo() {} + +func (*fieldType_ArrayType) isFieldType_TypeInfo() {} + +func (*fieldType_IterableType) isFieldType_TypeInfo() {} + +func (*fieldType_MapType) isFieldType_TypeInfo() {} + +func (*fieldType_RowType) isFieldType_TypeInfo() {} + +func (*fieldType_LogicalType) isFieldType_TypeInfo() {} + +type ArrayType struct { + state protoimpl.MessageState `protogen:"opaque.v1"` + xxx_hidden_ElementType *FieldType `protobuf:"bytes,1,opt,name=element_type,json=elementType,proto3" json:"element_type,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *ArrayType) Reset() { + *x = ArrayType{} + mi := &file_org_apache_beam_model_pipeline_v1_schema_proto_msgTypes[3] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *ArrayType) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*ArrayType) ProtoMessage() {} + +func (x *ArrayType) ProtoReflect() protoreflect.Message { + mi := &file_org_apache_beam_model_pipeline_v1_schema_proto_msgTypes[3] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +func (x *ArrayType) GetElementType() *FieldType { + if x != nil { + return x.xxx_hidden_ElementType + } + return nil +} + +func (x *ArrayType) SetElementType(v *FieldType) { + x.xxx_hidden_ElementType = v +} + +func (x *ArrayType) HasElementType() bool { + if x == nil { + return false + } + return x.xxx_hidden_ElementType != nil +} + +func (x *ArrayType) ClearElementType() { + x.xxx_hidden_ElementType = nil +} + +type ArrayType_builder struct { + _ [0]func() // Prevents comparability and use of unkeyed literals for the builder. + + ElementType *FieldType +} + +func (b0 ArrayType_builder) Build() *ArrayType { + m0 := &ArrayType{} + b, x := &b0, m0 + _, _ = b, x + x.xxx_hidden_ElementType = b.ElementType + return m0 +} + +type IterableType struct { + state protoimpl.MessageState `protogen:"opaque.v1"` + xxx_hidden_ElementType *FieldType `protobuf:"bytes,1,opt,name=element_type,json=elementType,proto3" json:"element_type,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *IterableType) Reset() { + *x = IterableType{} + mi := &file_org_apache_beam_model_pipeline_v1_schema_proto_msgTypes[4] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *IterableType) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*IterableType) ProtoMessage() {} + +func (x *IterableType) ProtoReflect() protoreflect.Message { + mi := &file_org_apache_beam_model_pipeline_v1_schema_proto_msgTypes[4] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +func (x *IterableType) GetElementType() *FieldType { + if x != nil { + return x.xxx_hidden_ElementType + } + return nil +} + +func (x *IterableType) SetElementType(v *FieldType) { + x.xxx_hidden_ElementType = v +} + +func (x *IterableType) HasElementType() bool { + if x == nil { + return false + } + return x.xxx_hidden_ElementType != nil +} + +func (x *IterableType) ClearElementType() { + x.xxx_hidden_ElementType = nil +} + +type IterableType_builder struct { + _ [0]func() // Prevents comparability and use of unkeyed literals for the builder. + + ElementType *FieldType +} + +func (b0 IterableType_builder) Build() *IterableType { + m0 := &IterableType{} + b, x := &b0, m0 + _, _ = b, x + x.xxx_hidden_ElementType = b.ElementType + return m0 +} + +type MapType struct { + state protoimpl.MessageState `protogen:"opaque.v1"` + xxx_hidden_KeyType *FieldType `protobuf:"bytes,1,opt,name=key_type,json=keyType,proto3" json:"key_type,omitempty"` + xxx_hidden_ValueType *FieldType `protobuf:"bytes,2,opt,name=value_type,json=valueType,proto3" json:"value_type,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *MapType) Reset() { + *x = MapType{} + mi := &file_org_apache_beam_model_pipeline_v1_schema_proto_msgTypes[5] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *MapType) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*MapType) ProtoMessage() {} + +func (x *MapType) ProtoReflect() protoreflect.Message { + mi := &file_org_apache_beam_model_pipeline_v1_schema_proto_msgTypes[5] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +func (x *MapType) GetKeyType() *FieldType { + if x != nil { + return x.xxx_hidden_KeyType + } + return nil +} + +func (x *MapType) GetValueType() *FieldType { + if x != nil { + return x.xxx_hidden_ValueType + } + return nil +} + +func (x *MapType) SetKeyType(v *FieldType) { + x.xxx_hidden_KeyType = v +} + +func (x *MapType) SetValueType(v *FieldType) { + x.xxx_hidden_ValueType = v +} + +func (x *MapType) HasKeyType() bool { + if x == nil { + return false + } + return x.xxx_hidden_KeyType != nil +} + +func (x *MapType) HasValueType() bool { + if x == nil { + return false + } + return x.xxx_hidden_ValueType != nil +} + +func (x *MapType) ClearKeyType() { + x.xxx_hidden_KeyType = nil +} + +func (x *MapType) ClearValueType() { + x.xxx_hidden_ValueType = nil +} + +type MapType_builder struct { + _ [0]func() // Prevents comparability and use of unkeyed literals for the builder. + + KeyType *FieldType + ValueType *FieldType +} + +func (b0 MapType_builder) Build() *MapType { + m0 := &MapType{} + b, x := &b0, m0 + _, _ = b, x + x.xxx_hidden_KeyType = b.KeyType + x.xxx_hidden_ValueType = b.ValueType + return m0 +} + +type RowType struct { + state protoimpl.MessageState `protogen:"opaque.v1"` + xxx_hidden_Schema *Schema `protobuf:"bytes,1,opt,name=schema,proto3" json:"schema,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *RowType) Reset() { + *x = RowType{} + mi := &file_org_apache_beam_model_pipeline_v1_schema_proto_msgTypes[6] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *RowType) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*RowType) ProtoMessage() {} + +func (x *RowType) ProtoReflect() protoreflect.Message { + mi := &file_org_apache_beam_model_pipeline_v1_schema_proto_msgTypes[6] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +func (x *RowType) GetSchema() *Schema { + if x != nil { + return x.xxx_hidden_Schema + } + return nil +} + +func (x *RowType) SetSchema(v *Schema) { + x.xxx_hidden_Schema = v +} + +func (x *RowType) HasSchema() bool { + if x == nil { + return false + } + return x.xxx_hidden_Schema != nil +} + +func (x *RowType) ClearSchema() { + x.xxx_hidden_Schema = nil +} + +type RowType_builder struct { + _ [0]func() // Prevents comparability and use of unkeyed literals for the builder. + + Schema *Schema +} + +func (b0 RowType_builder) Build() *RowType { + m0 := &RowType{} + b, x := &b0, m0 + _, _ = b, x + x.xxx_hidden_Schema = b.Schema + return m0 +} + +type LogicalType struct { + state protoimpl.MessageState `protogen:"opaque.v1"` + xxx_hidden_Urn string `protobuf:"bytes,1,opt,name=urn,proto3" json:"urn,omitempty"` + xxx_hidden_Payload []byte `protobuf:"bytes,2,opt,name=payload,proto3" json:"payload,omitempty"` + xxx_hidden_Representation *FieldType `protobuf:"bytes,3,opt,name=representation,proto3" json:"representation,omitempty"` + xxx_hidden_ArgumentType *FieldType `protobuf:"bytes,4,opt,name=argument_type,json=argumentType,proto3" json:"argument_type,omitempty"` + xxx_hidden_Argument *FieldValue `protobuf:"bytes,5,opt,name=argument,proto3" json:"argument,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *LogicalType) Reset() { + *x = LogicalType{} + mi := &file_org_apache_beam_model_pipeline_v1_schema_proto_msgTypes[7] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *LogicalType) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*LogicalType) ProtoMessage() {} + +func (x *LogicalType) ProtoReflect() protoreflect.Message { + mi := &file_org_apache_beam_model_pipeline_v1_schema_proto_msgTypes[7] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +func (x *LogicalType) GetUrn() string { + if x != nil { + return x.xxx_hidden_Urn + } + return "" +} + +func (x *LogicalType) GetPayload() []byte { + if x != nil { + return x.xxx_hidden_Payload + } + return nil +} + +func (x *LogicalType) GetRepresentation() *FieldType { + if x != nil { + return x.xxx_hidden_Representation + } + return nil +} + +func (x *LogicalType) GetArgumentType() *FieldType { + if x != nil { + return x.xxx_hidden_ArgumentType + } + return nil +} + +func (x *LogicalType) GetArgument() *FieldValue { + if x != nil { + return x.xxx_hidden_Argument + } + return nil +} + +func (x *LogicalType) SetUrn(v string) { + x.xxx_hidden_Urn = v +} + +func (x *LogicalType) SetPayload(v []byte) { + if v == nil { + v = []byte{} + } + x.xxx_hidden_Payload = v +} + +func (x *LogicalType) SetRepresentation(v *FieldType) { + x.xxx_hidden_Representation = v +} + +func (x *LogicalType) SetArgumentType(v *FieldType) { + x.xxx_hidden_ArgumentType = v +} + +func (x *LogicalType) SetArgument(v *FieldValue) { + x.xxx_hidden_Argument = v +} + +func (x *LogicalType) HasRepresentation() bool { + if x == nil { + return false + } + return x.xxx_hidden_Representation != nil +} + +func (x *LogicalType) HasArgumentType() bool { + if x == nil { + return false + } + return x.xxx_hidden_ArgumentType != nil +} + +func (x *LogicalType) HasArgument() bool { + if x == nil { + return false + } + return x.xxx_hidden_Argument != nil +} + +func (x *LogicalType) ClearRepresentation() { + x.xxx_hidden_Representation = nil +} + +func (x *LogicalType) ClearArgumentType() { + x.xxx_hidden_ArgumentType = nil +} + +func (x *LogicalType) ClearArgument() { + x.xxx_hidden_Argument = nil +} + +type LogicalType_builder struct { + _ [0]func() // Prevents comparability and use of unkeyed literals for the builder. + + Urn string + Payload []byte + Representation *FieldType + ArgumentType *FieldType + Argument *FieldValue +} + +func (b0 LogicalType_builder) Build() *LogicalType { + m0 := &LogicalType{} + b, x := &b0, m0 + _, _ = b, x + x.xxx_hidden_Urn = b.Urn + x.xxx_hidden_Payload = b.Payload + x.xxx_hidden_Representation = b.Representation + x.xxx_hidden_ArgumentType = b.ArgumentType + x.xxx_hidden_Argument = b.Argument + return m0 +} + +// Universally defined Logical types for Row schemas. +// These logical types are supposed to be understood by all SDKs. +type LogicalTypes struct { + state protoimpl.MessageState `protogen:"opaque.v1"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *LogicalTypes) Reset() { + *x = LogicalTypes{} + mi := &file_org_apache_beam_model_pipeline_v1_schema_proto_msgTypes[8] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *LogicalTypes) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*LogicalTypes) ProtoMessage() {} + +func (x *LogicalTypes) ProtoReflect() protoreflect.Message { + mi := &file_org_apache_beam_model_pipeline_v1_schema_proto_msgTypes[8] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +type LogicalTypes_builder struct { + _ [0]func() // Prevents comparability and use of unkeyed literals for the builder. + +} + +func (b0 LogicalTypes_builder) Build() *LogicalTypes { + m0 := &LogicalTypes{} + b, x := &b0, m0 + _, _ = b, x + return m0 +} + +type Option struct { + state protoimpl.MessageState `protogen:"opaque.v1"` + xxx_hidden_Name string `protobuf:"bytes,1,opt,name=name,proto3" json:"name,omitempty"` + xxx_hidden_Type *FieldType `protobuf:"bytes,2,opt,name=type,proto3" json:"type,omitempty"` + xxx_hidden_Value *FieldValue `protobuf:"bytes,3,opt,name=value,proto3" json:"value,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *Option) Reset() { + *x = Option{} + mi := &file_org_apache_beam_model_pipeline_v1_schema_proto_msgTypes[9] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *Option) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*Option) ProtoMessage() {} + +func (x *Option) ProtoReflect() protoreflect.Message { + mi := &file_org_apache_beam_model_pipeline_v1_schema_proto_msgTypes[9] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +func (x *Option) GetName() string { + if x != nil { + return x.xxx_hidden_Name + } + return "" +} + +func (x *Option) GetType() *FieldType { + if x != nil { + return x.xxx_hidden_Type + } + return nil +} + +func (x *Option) GetValue() *FieldValue { + if x != nil { + return x.xxx_hidden_Value + } + return nil +} + +func (x *Option) SetName(v string) { + x.xxx_hidden_Name = v +} + +func (x *Option) SetType(v *FieldType) { + x.xxx_hidden_Type = v +} + +func (x *Option) SetValue(v *FieldValue) { + x.xxx_hidden_Value = v +} + +func (x *Option) HasType() bool { + if x == nil { + return false + } + return x.xxx_hidden_Type != nil +} + +func (x *Option) HasValue() bool { + if x == nil { + return false + } + return x.xxx_hidden_Value != nil +} + +func (x *Option) ClearType() { + x.xxx_hidden_Type = nil +} + +func (x *Option) ClearValue() { + x.xxx_hidden_Value = nil +} + +type Option_builder struct { + _ [0]func() // Prevents comparability and use of unkeyed literals for the builder. + + // REQUIRED. Identifier for the option. + Name string + // REQUIRED. Type specifier for the structure of value. + // Conventionally, options that don't require additional configuration should + // use a boolean type, with the value set to true. + Type *FieldType + Value *FieldValue +} + +func (b0 Option_builder) Build() *Option { + m0 := &Option{} + b, x := &b0, m0 + _, _ = b, x + x.xxx_hidden_Name = b.Name + x.xxx_hidden_Type = b.Type + x.xxx_hidden_Value = b.Value + return m0 +} + +type Row struct { + state protoimpl.MessageState `protogen:"opaque.v1"` + xxx_hidden_Values *[]*FieldValue `protobuf:"bytes,1,rep,name=values,proto3" json:"values,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *Row) Reset() { + *x = Row{} + mi := &file_org_apache_beam_model_pipeline_v1_schema_proto_msgTypes[10] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *Row) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*Row) ProtoMessage() {} + +func (x *Row) ProtoReflect() protoreflect.Message { + mi := &file_org_apache_beam_model_pipeline_v1_schema_proto_msgTypes[10] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +func (x *Row) GetValues() []*FieldValue { + if x != nil { + if x.xxx_hidden_Values != nil { + return *x.xxx_hidden_Values + } + } + return nil +} + +func (x *Row) SetValues(v []*FieldValue) { + x.xxx_hidden_Values = &v +} + +type Row_builder struct { + _ [0]func() // Prevents comparability and use of unkeyed literals for the builder. + + Values []*FieldValue +} + +func (b0 Row_builder) Build() *Row { + m0 := &Row{} + b, x := &b0, m0 + _, _ = b, x + x.xxx_hidden_Values = &b.Values + return m0 +} + +type FieldValue struct { + state protoimpl.MessageState `protogen:"opaque.v1"` + xxx_hidden_FieldValue isFieldValue_FieldValue `protobuf_oneof:"field_value"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *FieldValue) Reset() { + *x = FieldValue{} + mi := &file_org_apache_beam_model_pipeline_v1_schema_proto_msgTypes[11] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *FieldValue) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*FieldValue) ProtoMessage() {} + +func (x *FieldValue) ProtoReflect() protoreflect.Message { + mi := &file_org_apache_beam_model_pipeline_v1_schema_proto_msgTypes[11] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +func (x *FieldValue) GetAtomicValue() *AtomicTypeValue { + if x != nil { + if x, ok := x.xxx_hidden_FieldValue.(*fieldValue_AtomicValue); ok { + return x.AtomicValue + } + } + return nil +} + +func (x *FieldValue) GetArrayValue() *ArrayTypeValue { + if x != nil { + if x, ok := x.xxx_hidden_FieldValue.(*fieldValue_ArrayValue); ok { + return x.ArrayValue + } + } + return nil +} + +func (x *FieldValue) GetIterableValue() *IterableTypeValue { + if x != nil { + if x, ok := x.xxx_hidden_FieldValue.(*fieldValue_IterableValue); ok { + return x.IterableValue + } + } + return nil +} + +func (x *FieldValue) GetMapValue() *MapTypeValue { + if x != nil { + if x, ok := x.xxx_hidden_FieldValue.(*fieldValue_MapValue); ok { + return x.MapValue + } + } + return nil +} + +func (x *FieldValue) GetRowValue() *Row { + if x != nil { + if x, ok := x.xxx_hidden_FieldValue.(*fieldValue_RowValue); ok { + return x.RowValue + } + } + return nil +} + +func (x *FieldValue) GetLogicalTypeValue() *LogicalTypeValue { + if x != nil { + if x, ok := x.xxx_hidden_FieldValue.(*fieldValue_LogicalTypeValue); ok { + return x.LogicalTypeValue + } + } + return nil +} + +func (x *FieldValue) SetAtomicValue(v *AtomicTypeValue) { + if v == nil { + x.xxx_hidden_FieldValue = nil + return + } + x.xxx_hidden_FieldValue = &fieldValue_AtomicValue{v} +} + +func (x *FieldValue) SetArrayValue(v *ArrayTypeValue) { + if v == nil { + x.xxx_hidden_FieldValue = nil + return + } + x.xxx_hidden_FieldValue = &fieldValue_ArrayValue{v} +} + +func (x *FieldValue) SetIterableValue(v *IterableTypeValue) { + if v == nil { + x.xxx_hidden_FieldValue = nil + return + } + x.xxx_hidden_FieldValue = &fieldValue_IterableValue{v} +} + +func (x *FieldValue) SetMapValue(v *MapTypeValue) { + if v == nil { + x.xxx_hidden_FieldValue = nil + return + } + x.xxx_hidden_FieldValue = &fieldValue_MapValue{v} +} + +func (x *FieldValue) SetRowValue(v *Row) { + if v == nil { + x.xxx_hidden_FieldValue = nil + return + } + x.xxx_hidden_FieldValue = &fieldValue_RowValue{v} +} + +func (x *FieldValue) SetLogicalTypeValue(v *LogicalTypeValue) { + if v == nil { + x.xxx_hidden_FieldValue = nil + return + } + x.xxx_hidden_FieldValue = &fieldValue_LogicalTypeValue{v} +} + +func (x *FieldValue) HasFieldValue() bool { + if x == nil { + return false + } + return x.xxx_hidden_FieldValue != nil +} + +func (x *FieldValue) HasAtomicValue() bool { + if x == nil { + return false + } + _, ok := x.xxx_hidden_FieldValue.(*fieldValue_AtomicValue) + return ok +} + +func (x *FieldValue) HasArrayValue() bool { + if x == nil { + return false + } + _, ok := x.xxx_hidden_FieldValue.(*fieldValue_ArrayValue) + return ok +} + +func (x *FieldValue) HasIterableValue() bool { + if x == nil { + return false + } + _, ok := x.xxx_hidden_FieldValue.(*fieldValue_IterableValue) + return ok +} + +func (x *FieldValue) HasMapValue() bool { + if x == nil { + return false + } + _, ok := x.xxx_hidden_FieldValue.(*fieldValue_MapValue) + return ok +} + +func (x *FieldValue) HasRowValue() bool { + if x == nil { + return false + } + _, ok := x.xxx_hidden_FieldValue.(*fieldValue_RowValue) + return ok +} + +func (x *FieldValue) HasLogicalTypeValue() bool { + if x == nil { + return false + } + _, ok := x.xxx_hidden_FieldValue.(*fieldValue_LogicalTypeValue) + return ok +} + +func (x *FieldValue) ClearFieldValue() { + x.xxx_hidden_FieldValue = nil +} + +func (x *FieldValue) ClearAtomicValue() { + if _, ok := x.xxx_hidden_FieldValue.(*fieldValue_AtomicValue); ok { + x.xxx_hidden_FieldValue = nil + } +} + +func (x *FieldValue) ClearArrayValue() { + if _, ok := x.xxx_hidden_FieldValue.(*fieldValue_ArrayValue); ok { + x.xxx_hidden_FieldValue = nil + } +} + +func (x *FieldValue) ClearIterableValue() { + if _, ok := x.xxx_hidden_FieldValue.(*fieldValue_IterableValue); ok { + x.xxx_hidden_FieldValue = nil + } +} + +func (x *FieldValue) ClearMapValue() { + if _, ok := x.xxx_hidden_FieldValue.(*fieldValue_MapValue); ok { + x.xxx_hidden_FieldValue = nil + } +} + +func (x *FieldValue) ClearRowValue() { + if _, ok := x.xxx_hidden_FieldValue.(*fieldValue_RowValue); ok { + x.xxx_hidden_FieldValue = nil + } +} + +func (x *FieldValue) ClearLogicalTypeValue() { + if _, ok := x.xxx_hidden_FieldValue.(*fieldValue_LogicalTypeValue); ok { + x.xxx_hidden_FieldValue = nil + } +} + +const FieldValue_FieldValue_not_set_case case_FieldValue_FieldValue = 0 +const FieldValue_AtomicValue_case case_FieldValue_FieldValue = 1 +const FieldValue_ArrayValue_case case_FieldValue_FieldValue = 2 +const FieldValue_IterableValue_case case_FieldValue_FieldValue = 3 +const FieldValue_MapValue_case case_FieldValue_FieldValue = 4 +const FieldValue_RowValue_case case_FieldValue_FieldValue = 5 +const FieldValue_LogicalTypeValue_case case_FieldValue_FieldValue = 6 + +func (x *FieldValue) WhichFieldValue() case_FieldValue_FieldValue { + if x == nil { + return FieldValue_FieldValue_not_set_case + } + switch x.xxx_hidden_FieldValue.(type) { + case *fieldValue_AtomicValue: + return FieldValue_AtomicValue_case + case *fieldValue_ArrayValue: + return FieldValue_ArrayValue_case + case *fieldValue_IterableValue: + return FieldValue_IterableValue_case + case *fieldValue_MapValue: + return FieldValue_MapValue_case + case *fieldValue_RowValue: + return FieldValue_RowValue_case + case *fieldValue_LogicalTypeValue: + return FieldValue_LogicalTypeValue_case + default: + return FieldValue_FieldValue_not_set_case + } +} + +type FieldValue_builder struct { + _ [0]func() // Prevents comparability and use of unkeyed literals for the builder. + + // If none of these are set, value is considered null. + + // Fields of oneof xxx_hidden_FieldValue: + AtomicValue *AtomicTypeValue + ArrayValue *ArrayTypeValue + IterableValue *IterableTypeValue + MapValue *MapTypeValue + RowValue *Row + LogicalTypeValue *LogicalTypeValue + // -- end of xxx_hidden_FieldValue +} + +func (b0 FieldValue_builder) Build() *FieldValue { + m0 := &FieldValue{} + b, x := &b0, m0 + _, _ = b, x + if b.AtomicValue != nil { + x.xxx_hidden_FieldValue = &fieldValue_AtomicValue{b.AtomicValue} + } + if b.ArrayValue != nil { + x.xxx_hidden_FieldValue = &fieldValue_ArrayValue{b.ArrayValue} + } + if b.IterableValue != nil { + x.xxx_hidden_FieldValue = &fieldValue_IterableValue{b.IterableValue} + } + if b.MapValue != nil { + x.xxx_hidden_FieldValue = &fieldValue_MapValue{b.MapValue} + } + if b.RowValue != nil { + x.xxx_hidden_FieldValue = &fieldValue_RowValue{b.RowValue} + } + if b.LogicalTypeValue != nil { + x.xxx_hidden_FieldValue = &fieldValue_LogicalTypeValue{b.LogicalTypeValue} + } + return m0 +} + +type case_FieldValue_FieldValue protoreflect.FieldNumber + +func (x case_FieldValue_FieldValue) String() string { + md := file_org_apache_beam_model_pipeline_v1_schema_proto_msgTypes[11].Descriptor() + if x == 0 { + return "not set" + } + return protoimpl.X.MessageFieldStringOf(md, protoreflect.FieldNumber(x)) +} + +type isFieldValue_FieldValue interface { + isFieldValue_FieldValue() +} + +type fieldValue_AtomicValue struct { + AtomicValue *AtomicTypeValue `protobuf:"bytes,1,opt,name=atomic_value,json=atomicValue,proto3,oneof"` +} + +type fieldValue_ArrayValue struct { + ArrayValue *ArrayTypeValue `protobuf:"bytes,2,opt,name=array_value,json=arrayValue,proto3,oneof"` +} + +type fieldValue_IterableValue struct { + IterableValue *IterableTypeValue `protobuf:"bytes,3,opt,name=iterable_value,json=iterableValue,proto3,oneof"` +} + +type fieldValue_MapValue struct { + MapValue *MapTypeValue `protobuf:"bytes,4,opt,name=map_value,json=mapValue,proto3,oneof"` +} + +type fieldValue_RowValue struct { + RowValue *Row `protobuf:"bytes,5,opt,name=row_value,json=rowValue,proto3,oneof"` +} + +type fieldValue_LogicalTypeValue struct { + LogicalTypeValue *LogicalTypeValue `protobuf:"bytes,6,opt,name=logical_type_value,json=logicalTypeValue,proto3,oneof"` +} + +func (*fieldValue_AtomicValue) isFieldValue_FieldValue() {} + +func (*fieldValue_ArrayValue) isFieldValue_FieldValue() {} + +func (*fieldValue_IterableValue) isFieldValue_FieldValue() {} + +func (*fieldValue_MapValue) isFieldValue_FieldValue() {} + +func (*fieldValue_RowValue) isFieldValue_FieldValue() {} + +func (*fieldValue_LogicalTypeValue) isFieldValue_FieldValue() {} + +type AtomicTypeValue struct { + state protoimpl.MessageState `protogen:"opaque.v1"` + xxx_hidden_Value isAtomicTypeValue_Value `protobuf_oneof:"value"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *AtomicTypeValue) Reset() { + *x = AtomicTypeValue{} + mi := &file_org_apache_beam_model_pipeline_v1_schema_proto_msgTypes[12] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *AtomicTypeValue) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*AtomicTypeValue) ProtoMessage() {} + +func (x *AtomicTypeValue) ProtoReflect() protoreflect.Message { + mi := &file_org_apache_beam_model_pipeline_v1_schema_proto_msgTypes[12] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +func (x *AtomicTypeValue) GetByte() int32 { + if x != nil { + if x, ok := x.xxx_hidden_Value.(*atomicTypeValue_Byte); ok { + return x.Byte + } + } + return 0 +} + +func (x *AtomicTypeValue) GetInt16() int32 { + if x != nil { + if x, ok := x.xxx_hidden_Value.(*atomicTypeValue_Int16); ok { + return x.Int16 + } + } + return 0 +} + +func (x *AtomicTypeValue) GetInt32() int32 { + if x != nil { + if x, ok := x.xxx_hidden_Value.(*atomicTypeValue_Int32); ok { + return x.Int32 + } + } + return 0 +} + +func (x *AtomicTypeValue) GetInt64() int64 { + if x != nil { + if x, ok := x.xxx_hidden_Value.(*atomicTypeValue_Int64); ok { + return x.Int64 + } + } + return 0 +} + +func (x *AtomicTypeValue) GetFloat() float32 { + if x != nil { + if x, ok := x.xxx_hidden_Value.(*atomicTypeValue_Float); ok { + return x.Float + } + } + return 0 +} + +func (x *AtomicTypeValue) GetDouble() float64 { + if x != nil { + if x, ok := x.xxx_hidden_Value.(*atomicTypeValue_Double); ok { + return x.Double + } + } + return 0 +} + +func (x *AtomicTypeValue) GetString() string { + if x != nil { + if x, ok := x.xxx_hidden_Value.(*atomicTypeValue_String_); ok { + return x.String_ + } + } + return "" +} + +func (x *AtomicTypeValue) GetBoolean() bool { + if x != nil { + if x, ok := x.xxx_hidden_Value.(*atomicTypeValue_Boolean); ok { + return x.Boolean + } + } + return false +} + +func (x *AtomicTypeValue) GetBytes() []byte { + if x != nil { + if x, ok := x.xxx_hidden_Value.(*atomicTypeValue_Bytes); ok { + return x.Bytes + } + } + return nil +} + +func (x *AtomicTypeValue) SetByte(v int32) { + x.xxx_hidden_Value = &atomicTypeValue_Byte{v} +} + +func (x *AtomicTypeValue) SetInt16(v int32) { + x.xxx_hidden_Value = &atomicTypeValue_Int16{v} +} + +func (x *AtomicTypeValue) SetInt32(v int32) { + x.xxx_hidden_Value = &atomicTypeValue_Int32{v} +} + +func (x *AtomicTypeValue) SetInt64(v int64) { + x.xxx_hidden_Value = &atomicTypeValue_Int64{v} +} + +func (x *AtomicTypeValue) SetFloat(v float32) { + x.xxx_hidden_Value = &atomicTypeValue_Float{v} +} + +func (x *AtomicTypeValue) SetDouble(v float64) { + x.xxx_hidden_Value = &atomicTypeValue_Double{v} +} + +func (x *AtomicTypeValue) SetString(v string) { + x.xxx_hidden_Value = &atomicTypeValue_String_{v} +} + +func (x *AtomicTypeValue) SetBoolean(v bool) { + x.xxx_hidden_Value = &atomicTypeValue_Boolean{v} +} + +func (x *AtomicTypeValue) SetBytes(v []byte) { + if v == nil { + v = []byte{} + } + x.xxx_hidden_Value = &atomicTypeValue_Bytes{v} +} + +func (x *AtomicTypeValue) HasValue() bool { + if x == nil { + return false + } + return x.xxx_hidden_Value != nil +} + +func (x *AtomicTypeValue) HasByte() bool { + if x == nil { + return false + } + _, ok := x.xxx_hidden_Value.(*atomicTypeValue_Byte) + return ok +} + +func (x *AtomicTypeValue) HasInt16() bool { + if x == nil { + return false + } + _, ok := x.xxx_hidden_Value.(*atomicTypeValue_Int16) + return ok +} + +func (x *AtomicTypeValue) HasInt32() bool { + if x == nil { + return false + } + _, ok := x.xxx_hidden_Value.(*atomicTypeValue_Int32) + return ok +} + +func (x *AtomicTypeValue) HasInt64() bool { + if x == nil { + return false + } + _, ok := x.xxx_hidden_Value.(*atomicTypeValue_Int64) + return ok +} + +func (x *AtomicTypeValue) HasFloat() bool { + if x == nil { + return false + } + _, ok := x.xxx_hidden_Value.(*atomicTypeValue_Float) + return ok +} + +func (x *AtomicTypeValue) HasDouble() bool { + if x == nil { + return false + } + _, ok := x.xxx_hidden_Value.(*atomicTypeValue_Double) + return ok +} + +func (x *AtomicTypeValue) HasString() bool { + if x == nil { + return false + } + _, ok := x.xxx_hidden_Value.(*atomicTypeValue_String_) + return ok +} + +func (x *AtomicTypeValue) HasBoolean() bool { + if x == nil { + return false + } + _, ok := x.xxx_hidden_Value.(*atomicTypeValue_Boolean) + return ok +} + +func (x *AtomicTypeValue) HasBytes() bool { + if x == nil { + return false + } + _, ok := x.xxx_hidden_Value.(*atomicTypeValue_Bytes) + return ok +} + +func (x *AtomicTypeValue) ClearValue() { + x.xxx_hidden_Value = nil +} + +func (x *AtomicTypeValue) ClearByte() { + if _, ok := x.xxx_hidden_Value.(*atomicTypeValue_Byte); ok { + x.xxx_hidden_Value = nil + } +} + +func (x *AtomicTypeValue) ClearInt16() { + if _, ok := x.xxx_hidden_Value.(*atomicTypeValue_Int16); ok { + x.xxx_hidden_Value = nil + } +} + +func (x *AtomicTypeValue) ClearInt32() { + if _, ok := x.xxx_hidden_Value.(*atomicTypeValue_Int32); ok { + x.xxx_hidden_Value = nil + } +} + +func (x *AtomicTypeValue) ClearInt64() { + if _, ok := x.xxx_hidden_Value.(*atomicTypeValue_Int64); ok { + x.xxx_hidden_Value = nil + } +} + +func (x *AtomicTypeValue) ClearFloat() { + if _, ok := x.xxx_hidden_Value.(*atomicTypeValue_Float); ok { + x.xxx_hidden_Value = nil + } +} + +func (x *AtomicTypeValue) ClearDouble() { + if _, ok := x.xxx_hidden_Value.(*atomicTypeValue_Double); ok { + x.xxx_hidden_Value = nil + } +} + +func (x *AtomicTypeValue) ClearString() { + if _, ok := x.xxx_hidden_Value.(*atomicTypeValue_String_); ok { + x.xxx_hidden_Value = nil + } +} + +func (x *AtomicTypeValue) ClearBoolean() { + if _, ok := x.xxx_hidden_Value.(*atomicTypeValue_Boolean); ok { + x.xxx_hidden_Value = nil + } +} + +func (x *AtomicTypeValue) ClearBytes() { + if _, ok := x.xxx_hidden_Value.(*atomicTypeValue_Bytes); ok { + x.xxx_hidden_Value = nil + } +} + +const AtomicTypeValue_Value_not_set_case case_AtomicTypeValue_Value = 0 +const AtomicTypeValue_Byte_case case_AtomicTypeValue_Value = 1 +const AtomicTypeValue_Int16_case case_AtomicTypeValue_Value = 2 +const AtomicTypeValue_Int32_case case_AtomicTypeValue_Value = 3 +const AtomicTypeValue_Int64_case case_AtomicTypeValue_Value = 4 +const AtomicTypeValue_Float_case case_AtomicTypeValue_Value = 5 +const AtomicTypeValue_Double_case case_AtomicTypeValue_Value = 6 +const AtomicTypeValue_String__case case_AtomicTypeValue_Value = 7 +const AtomicTypeValue_Boolean_case case_AtomicTypeValue_Value = 8 +const AtomicTypeValue_Bytes_case case_AtomicTypeValue_Value = 9 + +func (x *AtomicTypeValue) WhichValue() case_AtomicTypeValue_Value { + if x == nil { + return AtomicTypeValue_Value_not_set_case + } + switch x.xxx_hidden_Value.(type) { + case *atomicTypeValue_Byte: + return AtomicTypeValue_Byte_case + case *atomicTypeValue_Int16: + return AtomicTypeValue_Int16_case + case *atomicTypeValue_Int32: + return AtomicTypeValue_Int32_case + case *atomicTypeValue_Int64: + return AtomicTypeValue_Int64_case + case *atomicTypeValue_Float: + return AtomicTypeValue_Float_case + case *atomicTypeValue_Double: + return AtomicTypeValue_Double_case + case *atomicTypeValue_String_: + return AtomicTypeValue_String__case + case *atomicTypeValue_Boolean: + return AtomicTypeValue_Boolean_case + case *atomicTypeValue_Bytes: + return AtomicTypeValue_Bytes_case + default: + return AtomicTypeValue_Value_not_set_case + } +} + +type AtomicTypeValue_builder struct { + _ [0]func() // Prevents comparability and use of unkeyed literals for the builder. + + // Fields of oneof xxx_hidden_Value: + Byte *int32 + Int16 *int32 + Int32 *int32 + Int64 *int64 + Float *float32 + Double *float64 + String *string + Boolean *bool + Bytes []byte + // -- end of xxx_hidden_Value +} + +func (b0 AtomicTypeValue_builder) Build() *AtomicTypeValue { + m0 := &AtomicTypeValue{} + b, x := &b0, m0 + _, _ = b, x + if b.Byte != nil { + x.xxx_hidden_Value = &atomicTypeValue_Byte{*b.Byte} + } + if b.Int16 != nil { + x.xxx_hidden_Value = &atomicTypeValue_Int16{*b.Int16} + } + if b.Int32 != nil { + x.xxx_hidden_Value = &atomicTypeValue_Int32{*b.Int32} + } + if b.Int64 != nil { + x.xxx_hidden_Value = &atomicTypeValue_Int64{*b.Int64} + } + if b.Float != nil { + x.xxx_hidden_Value = &atomicTypeValue_Float{*b.Float} + } + if b.Double != nil { + x.xxx_hidden_Value = &atomicTypeValue_Double{*b.Double} + } + if b.String != nil { + x.xxx_hidden_Value = &atomicTypeValue_String_{*b.String} + } + if b.Boolean != nil { + x.xxx_hidden_Value = &atomicTypeValue_Boolean{*b.Boolean} + } + if b.Bytes != nil { + x.xxx_hidden_Value = &atomicTypeValue_Bytes{b.Bytes} + } + return m0 +} + +type case_AtomicTypeValue_Value protoreflect.FieldNumber + +func (x case_AtomicTypeValue_Value) String() string { + md := file_org_apache_beam_model_pipeline_v1_schema_proto_msgTypes[12].Descriptor() + if x == 0 { + return "not set" + } + return protoimpl.X.MessageFieldStringOf(md, protoreflect.FieldNumber(x)) +} + +type isAtomicTypeValue_Value interface { + isAtomicTypeValue_Value() +} + +type atomicTypeValue_Byte struct { + Byte int32 `protobuf:"varint,1,opt,name=byte,proto3,oneof"` +} + +type atomicTypeValue_Int16 struct { + Int16 int32 `protobuf:"varint,2,opt,name=int16,proto3,oneof"` +} + +type atomicTypeValue_Int32 struct { + Int32 int32 `protobuf:"varint,3,opt,name=int32,proto3,oneof"` +} + +type atomicTypeValue_Int64 struct { + Int64 int64 `protobuf:"varint,4,opt,name=int64,proto3,oneof"` +} + +type atomicTypeValue_Float struct { + Float float32 `protobuf:"fixed32,5,opt,name=float,proto3,oneof"` +} + +type atomicTypeValue_Double struct { + Double float64 `protobuf:"fixed64,6,opt,name=double,proto3,oneof"` +} + +type atomicTypeValue_String_ struct { + String_ string `protobuf:"bytes,7,opt,name=string,proto3,oneof"` +} + +type atomicTypeValue_Boolean struct { + Boolean bool `protobuf:"varint,8,opt,name=boolean,proto3,oneof"` +} + +type atomicTypeValue_Bytes struct { + Bytes []byte `protobuf:"bytes,9,opt,name=bytes,proto3,oneof"` +} + +func (*atomicTypeValue_Byte) isAtomicTypeValue_Value() {} + +func (*atomicTypeValue_Int16) isAtomicTypeValue_Value() {} + +func (*atomicTypeValue_Int32) isAtomicTypeValue_Value() {} + +func (*atomicTypeValue_Int64) isAtomicTypeValue_Value() {} + +func (*atomicTypeValue_Float) isAtomicTypeValue_Value() {} + +func (*atomicTypeValue_Double) isAtomicTypeValue_Value() {} + +func (*atomicTypeValue_String_) isAtomicTypeValue_Value() {} + +func (*atomicTypeValue_Boolean) isAtomicTypeValue_Value() {} + +func (*atomicTypeValue_Bytes) isAtomicTypeValue_Value() {} + +type ArrayTypeValue struct { + state protoimpl.MessageState `protogen:"opaque.v1"` + xxx_hidden_Element *[]*FieldValue `protobuf:"bytes,1,rep,name=element,proto3" json:"element,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *ArrayTypeValue) Reset() { + *x = ArrayTypeValue{} + mi := &file_org_apache_beam_model_pipeline_v1_schema_proto_msgTypes[13] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *ArrayTypeValue) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*ArrayTypeValue) ProtoMessage() {} + +func (x *ArrayTypeValue) ProtoReflect() protoreflect.Message { + mi := &file_org_apache_beam_model_pipeline_v1_schema_proto_msgTypes[13] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +func (x *ArrayTypeValue) GetElement() []*FieldValue { + if x != nil { + if x.xxx_hidden_Element != nil { + return *x.xxx_hidden_Element + } + } + return nil +} + +func (x *ArrayTypeValue) SetElement(v []*FieldValue) { + x.xxx_hidden_Element = &v +} + +type ArrayTypeValue_builder struct { + _ [0]func() // Prevents comparability and use of unkeyed literals for the builder. + + Element []*FieldValue +} + +func (b0 ArrayTypeValue_builder) Build() *ArrayTypeValue { + m0 := &ArrayTypeValue{} + b, x := &b0, m0 + _, _ = b, x + x.xxx_hidden_Element = &b.Element + return m0 +} + +type IterableTypeValue struct { + state protoimpl.MessageState `protogen:"opaque.v1"` + xxx_hidden_Element *[]*FieldValue `protobuf:"bytes,1,rep,name=element,proto3" json:"element,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *IterableTypeValue) Reset() { + *x = IterableTypeValue{} + mi := &file_org_apache_beam_model_pipeline_v1_schema_proto_msgTypes[14] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *IterableTypeValue) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*IterableTypeValue) ProtoMessage() {} + +func (x *IterableTypeValue) ProtoReflect() protoreflect.Message { + mi := &file_org_apache_beam_model_pipeline_v1_schema_proto_msgTypes[14] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +func (x *IterableTypeValue) GetElement() []*FieldValue { + if x != nil { + if x.xxx_hidden_Element != nil { + return *x.xxx_hidden_Element + } + } + return nil +} + +func (x *IterableTypeValue) SetElement(v []*FieldValue) { + x.xxx_hidden_Element = &v +} + +type IterableTypeValue_builder struct { + _ [0]func() // Prevents comparability and use of unkeyed literals for the builder. + + Element []*FieldValue +} + +func (b0 IterableTypeValue_builder) Build() *IterableTypeValue { + m0 := &IterableTypeValue{} + b, x := &b0, m0 + _, _ = b, x + x.xxx_hidden_Element = &b.Element + return m0 +} + +type MapTypeValue struct { + state protoimpl.MessageState `protogen:"opaque.v1"` + xxx_hidden_Entries *[]*MapTypeEntry `protobuf:"bytes,1,rep,name=entries,proto3" json:"entries,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *MapTypeValue) Reset() { + *x = MapTypeValue{} + mi := &file_org_apache_beam_model_pipeline_v1_schema_proto_msgTypes[15] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *MapTypeValue) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*MapTypeValue) ProtoMessage() {} + +func (x *MapTypeValue) ProtoReflect() protoreflect.Message { + mi := &file_org_apache_beam_model_pipeline_v1_schema_proto_msgTypes[15] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +func (x *MapTypeValue) GetEntries() []*MapTypeEntry { + if x != nil { + if x.xxx_hidden_Entries != nil { + return *x.xxx_hidden_Entries + } + } + return nil +} + +func (x *MapTypeValue) SetEntries(v []*MapTypeEntry) { + x.xxx_hidden_Entries = &v +} + +type MapTypeValue_builder struct { + _ [0]func() // Prevents comparability and use of unkeyed literals for the builder. + + Entries []*MapTypeEntry +} + +func (b0 MapTypeValue_builder) Build() *MapTypeValue { + m0 := &MapTypeValue{} + b, x := &b0, m0 + _, _ = b, x + x.xxx_hidden_Entries = &b.Entries + return m0 +} + +type MapTypeEntry struct { + state protoimpl.MessageState `protogen:"opaque.v1"` + xxx_hidden_Key *FieldValue `protobuf:"bytes,1,opt,name=key,proto3" json:"key,omitempty"` + xxx_hidden_Value *FieldValue `protobuf:"bytes,2,opt,name=value,proto3" json:"value,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *MapTypeEntry) Reset() { + *x = MapTypeEntry{} + mi := &file_org_apache_beam_model_pipeline_v1_schema_proto_msgTypes[16] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *MapTypeEntry) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*MapTypeEntry) ProtoMessage() {} + +func (x *MapTypeEntry) ProtoReflect() protoreflect.Message { + mi := &file_org_apache_beam_model_pipeline_v1_schema_proto_msgTypes[16] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +func (x *MapTypeEntry) GetKey() *FieldValue { + if x != nil { + return x.xxx_hidden_Key + } + return nil +} + +func (x *MapTypeEntry) GetValue() *FieldValue { + if x != nil { + return x.xxx_hidden_Value + } + return nil +} + +func (x *MapTypeEntry) SetKey(v *FieldValue) { + x.xxx_hidden_Key = v +} + +func (x *MapTypeEntry) SetValue(v *FieldValue) { + x.xxx_hidden_Value = v +} + +func (x *MapTypeEntry) HasKey() bool { + if x == nil { + return false + } + return x.xxx_hidden_Key != nil +} + +func (x *MapTypeEntry) HasValue() bool { + if x == nil { + return false + } + return x.xxx_hidden_Value != nil +} + +func (x *MapTypeEntry) ClearKey() { + x.xxx_hidden_Key = nil +} + +func (x *MapTypeEntry) ClearValue() { + x.xxx_hidden_Value = nil +} + +type MapTypeEntry_builder struct { + _ [0]func() // Prevents comparability and use of unkeyed literals for the builder. + + Key *FieldValue + Value *FieldValue +} + +func (b0 MapTypeEntry_builder) Build() *MapTypeEntry { + m0 := &MapTypeEntry{} + b, x := &b0, m0 + _, _ = b, x + x.xxx_hidden_Key = b.Key + x.xxx_hidden_Value = b.Value + return m0 +} + +type LogicalTypeValue struct { + state protoimpl.MessageState `protogen:"opaque.v1"` + xxx_hidden_Value *FieldValue `protobuf:"bytes,1,opt,name=value,proto3" json:"value,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *LogicalTypeValue) Reset() { + *x = LogicalTypeValue{} + mi := &file_org_apache_beam_model_pipeline_v1_schema_proto_msgTypes[17] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *LogicalTypeValue) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*LogicalTypeValue) ProtoMessage() {} + +func (x *LogicalTypeValue) ProtoReflect() protoreflect.Message { + mi := &file_org_apache_beam_model_pipeline_v1_schema_proto_msgTypes[17] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +func (x *LogicalTypeValue) GetValue() *FieldValue { + if x != nil { + return x.xxx_hidden_Value + } + return nil +} + +func (x *LogicalTypeValue) SetValue(v *FieldValue) { + x.xxx_hidden_Value = v +} + +func (x *LogicalTypeValue) HasValue() bool { + if x == nil { + return false + } + return x.xxx_hidden_Value != nil +} + +func (x *LogicalTypeValue) ClearValue() { + x.xxx_hidden_Value = nil +} + +type LogicalTypeValue_builder struct { + _ [0]func() // Prevents comparability and use of unkeyed literals for the builder. + + Value *FieldValue +} + +func (b0 LogicalTypeValue_builder) Build() *LogicalTypeValue { + m0 := &LogicalTypeValue{} + b, x := &b0, m0 + _, _ = b, x + x.xxx_hidden_Value = b.Value + return m0 +} + +var File_org_apache_beam_model_pipeline_v1_schema_proto protoreflect.FileDescriptor + +var file_org_apache_beam_model_pipeline_v1_schema_proto_rawDesc = []byte{ + 0x0a, 0x2e, 0x6f, 0x72, 0x67, 0x2f, 0x61, 0x70, 0x61, 0x63, 0x68, 0x65, 0x2f, 0x62, 0x65, 0x61, + 0x6d, 0x2f, 0x6d, 0x6f, 0x64, 0x65, 0x6c, 0x2f, 0x70, 0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e, 0x65, + 0x2f, 0x76, 0x31, 0x2f, 0x73, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, + 0x12, 0x21, 0x6f, 0x72, 0x67, 0x2e, 0x61, 0x70, 0x61, 0x63, 0x68, 0x65, 0x2e, 0x62, 0x65, 0x61, + 0x6d, 0x2e, 0x6d, 0x6f, 0x64, 0x65, 0x6c, 0x2e, 0x70, 0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e, 0x65, + 0x2e, 0x76, 0x31, 0x1a, 0x37, 0x6f, 0x72, 0x67, 0x2f, 0x61, 0x70, 0x61, 0x63, 0x68, 0x65, 0x2f, + 0x62, 0x65, 0x61, 0x6d, 0x2f, 0x6d, 0x6f, 0x64, 0x65, 0x6c, 0x2f, 0x70, 0x69, 0x70, 0x65, 0x6c, + 0x69, 0x6e, 0x65, 0x2f, 0x76, 0x31, 0x2f, 0x62, 0x65, 0x61, 0x6d, 0x5f, 0x72, 0x75, 0x6e, 0x6e, + 0x65, 0x72, 0x5f, 0x61, 0x70, 0x69, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x22, 0xd5, 0x01, 0x0a, + 0x06, 0x53, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x12, 0x40, 0x0a, 0x06, 0x66, 0x69, 0x65, 0x6c, 0x64, + 0x73, 0x18, 0x01, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x28, 0x2e, 0x6f, 0x72, 0x67, 0x2e, 0x61, 0x70, + 0x61, 0x63, 0x68, 0x65, 0x2e, 0x62, 0x65, 0x61, 0x6d, 0x2e, 0x6d, 0x6f, 0x64, 0x65, 0x6c, 0x2e, + 0x70, 0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x46, 0x69, 0x65, 0x6c, + 0x64, 0x52, 0x06, 0x66, 0x69, 0x65, 0x6c, 0x64, 0x73, 0x12, 0x0e, 0x0a, 0x02, 0x69, 0x64, 0x18, + 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x02, 0x69, 0x64, 0x12, 0x43, 0x0a, 0x07, 0x6f, 0x70, 0x74, + 0x69, 0x6f, 0x6e, 0x73, 0x18, 0x03, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x29, 0x2e, 0x6f, 0x72, 0x67, + 0x2e, 0x61, 0x70, 0x61, 0x63, 0x68, 0x65, 0x2e, 0x62, 0x65, 0x61, 0x6d, 0x2e, 0x6d, 0x6f, 0x64, + 0x65, 0x6c, 0x2e, 0x70, 0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x4f, + 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x07, 0x6f, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x12, 0x34, + 0x0a, 0x16, 0x65, 0x6e, 0x63, 0x6f, 0x64, 0x69, 0x6e, 0x67, 0x5f, 0x70, 0x6f, 0x73, 0x69, 0x74, + 0x69, 0x6f, 0x6e, 0x73, 0x5f, 0x73, 0x65, 0x74, 0x18, 0x04, 0x20, 0x01, 0x28, 0x08, 0x52, 0x14, + 0x65, 0x6e, 0x63, 0x6f, 0x64, 0x69, 0x6e, 0x67, 0x50, 0x6f, 0x73, 0x69, 0x74, 0x69, 0x6f, 0x6e, + 0x73, 0x53, 0x65, 0x74, 0x22, 0x81, 0x02, 0x0a, 0x05, 0x46, 0x69, 0x65, 0x6c, 0x64, 0x12, 0x12, + 0x0a, 0x04, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x04, 0x6e, 0x61, + 0x6d, 0x65, 0x12, 0x20, 0x0a, 0x0b, 0x64, 0x65, 0x73, 0x63, 0x72, 0x69, 0x70, 0x74, 0x69, 0x6f, + 0x6e, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0b, 0x64, 0x65, 0x73, 0x63, 0x72, 0x69, 0x70, + 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x40, 0x0a, 0x04, 0x74, 0x79, 0x70, 0x65, 0x18, 0x03, 0x20, 0x01, + 0x28, 0x0b, 0x32, 0x2c, 0x2e, 0x6f, 0x72, 0x67, 0x2e, 0x61, 0x70, 0x61, 0x63, 0x68, 0x65, 0x2e, + 0x62, 0x65, 0x61, 0x6d, 0x2e, 0x6d, 0x6f, 0x64, 0x65, 0x6c, 0x2e, 0x70, 0x69, 0x70, 0x65, 0x6c, + 0x69, 0x6e, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x46, 0x69, 0x65, 0x6c, 0x64, 0x54, 0x79, 0x70, 0x65, + 0x52, 0x04, 0x74, 0x79, 0x70, 0x65, 0x12, 0x0e, 0x0a, 0x02, 0x69, 0x64, 0x18, 0x04, 0x20, 0x01, + 0x28, 0x05, 0x52, 0x02, 0x69, 0x64, 0x12, 0x2b, 0x0a, 0x11, 0x65, 0x6e, 0x63, 0x6f, 0x64, 0x69, + 0x6e, 0x67, 0x5f, 0x70, 0x6f, 0x73, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x18, 0x05, 0x20, 0x01, 0x28, + 0x05, 0x52, 0x10, 0x65, 0x6e, 0x63, 0x6f, 0x64, 0x69, 0x6e, 0x67, 0x50, 0x6f, 0x73, 0x69, 0x74, + 0x69, 0x6f, 0x6e, 0x12, 0x43, 0x0a, 0x07, 0x6f, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x18, 0x06, + 0x20, 0x03, 0x28, 0x0b, 0x32, 0x29, 0x2e, 0x6f, 0x72, 0x67, 0x2e, 0x61, 0x70, 0x61, 0x63, 0x68, + 0x65, 0x2e, 0x62, 0x65, 0x61, 0x6d, 0x2e, 0x6d, 0x6f, 0x64, 0x65, 0x6c, 0x2e, 0x70, 0x69, 0x70, + 0x65, 0x6c, 0x69, 0x6e, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x4f, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x52, + 0x07, 0x6f, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x22, 0x94, 0x04, 0x0a, 0x09, 0x46, 0x69, 0x65, + 0x6c, 0x64, 0x54, 0x79, 0x70, 0x65, 0x12, 0x1a, 0x0a, 0x08, 0x6e, 0x75, 0x6c, 0x6c, 0x61, 0x62, + 0x6c, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x08, 0x52, 0x08, 0x6e, 0x75, 0x6c, 0x6c, 0x61, 0x62, + 0x6c, 0x65, 0x12, 0x50, 0x0a, 0x0b, 0x61, 0x74, 0x6f, 0x6d, 0x69, 0x63, 0x5f, 0x74, 0x79, 0x70, + 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0e, 0x32, 0x2d, 0x2e, 0x6f, 0x72, 0x67, 0x2e, 0x61, 0x70, + 0x61, 0x63, 0x68, 0x65, 0x2e, 0x62, 0x65, 0x61, 0x6d, 0x2e, 0x6d, 0x6f, 0x64, 0x65, 0x6c, 0x2e, + 0x70, 0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x41, 0x74, 0x6f, 0x6d, + 0x69, 0x63, 0x54, 0x79, 0x70, 0x65, 0x48, 0x00, 0x52, 0x0a, 0x61, 0x74, 0x6f, 0x6d, 0x69, 0x63, + 0x54, 0x79, 0x70, 0x65, 0x12, 0x4d, 0x0a, 0x0a, 0x61, 0x72, 0x72, 0x61, 0x79, 0x5f, 0x74, 0x79, + 0x70, 0x65, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x2c, 0x2e, 0x6f, 0x72, 0x67, 0x2e, 0x61, + 0x70, 0x61, 0x63, 0x68, 0x65, 0x2e, 0x62, 0x65, 0x61, 0x6d, 0x2e, 0x6d, 0x6f, 0x64, 0x65, 0x6c, + 0x2e, 0x70, 0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x41, 0x72, 0x72, + 0x61, 0x79, 0x54, 0x79, 0x70, 0x65, 0x48, 0x00, 0x52, 0x09, 0x61, 0x72, 0x72, 0x61, 0x79, 0x54, + 0x79, 0x70, 0x65, 0x12, 0x56, 0x0a, 0x0d, 0x69, 0x74, 0x65, 0x72, 0x61, 0x62, 0x6c, 0x65, 0x5f, + 0x74, 0x79, 0x70, 0x65, 0x18, 0x04, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x2f, 0x2e, 0x6f, 0x72, 0x67, + 0x2e, 0x61, 0x70, 0x61, 0x63, 0x68, 0x65, 0x2e, 0x62, 0x65, 0x61, 0x6d, 0x2e, 0x6d, 0x6f, 0x64, + 0x65, 0x6c, 0x2e, 0x70, 0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x49, + 0x74, 0x65, 0x72, 0x61, 0x62, 0x6c, 0x65, 0x54, 0x79, 0x70, 0x65, 0x48, 0x00, 0x52, 0x0c, 0x69, + 0x74, 0x65, 0x72, 0x61, 0x62, 0x6c, 0x65, 0x54, 0x79, 0x70, 0x65, 0x12, 0x47, 0x0a, 0x08, 0x6d, + 0x61, 0x70, 0x5f, 0x74, 0x79, 0x70, 0x65, 0x18, 0x05, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x2a, 0x2e, + 0x6f, 0x72, 0x67, 0x2e, 0x61, 0x70, 0x61, 0x63, 0x68, 0x65, 0x2e, 0x62, 0x65, 0x61, 0x6d, 0x2e, + 0x6d, 0x6f, 0x64, 0x65, 0x6c, 0x2e, 0x70, 0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e, 0x65, 0x2e, 0x76, + 0x31, 0x2e, 0x4d, 0x61, 0x70, 0x54, 0x79, 0x70, 0x65, 0x48, 0x00, 0x52, 0x07, 0x6d, 0x61, 0x70, + 0x54, 0x79, 0x70, 0x65, 0x12, 0x47, 0x0a, 0x08, 0x72, 0x6f, 0x77, 0x5f, 0x74, 0x79, 0x70, 0x65, + 0x18, 0x06, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x2a, 0x2e, 0x6f, 0x72, 0x67, 0x2e, 0x61, 0x70, 0x61, + 0x63, 0x68, 0x65, 0x2e, 0x62, 0x65, 0x61, 0x6d, 0x2e, 0x6d, 0x6f, 0x64, 0x65, 0x6c, 0x2e, 0x70, + 0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x52, 0x6f, 0x77, 0x54, 0x79, + 0x70, 0x65, 0x48, 0x00, 0x52, 0x07, 0x72, 0x6f, 0x77, 0x54, 0x79, 0x70, 0x65, 0x12, 0x53, 0x0a, + 0x0c, 0x6c, 0x6f, 0x67, 0x69, 0x63, 0x61, 0x6c, 0x5f, 0x74, 0x79, 0x70, 0x65, 0x18, 0x07, 0x20, + 0x01, 0x28, 0x0b, 0x32, 0x2e, 0x2e, 0x6f, 0x72, 0x67, 0x2e, 0x61, 0x70, 0x61, 0x63, 0x68, 0x65, + 0x2e, 0x62, 0x65, 0x61, 0x6d, 0x2e, 0x6d, 0x6f, 0x64, 0x65, 0x6c, 0x2e, 0x70, 0x69, 0x70, 0x65, + 0x6c, 0x69, 0x6e, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x4c, 0x6f, 0x67, 0x69, 0x63, 0x61, 0x6c, 0x54, + 0x79, 0x70, 0x65, 0x48, 0x00, 0x52, 0x0b, 0x6c, 0x6f, 0x67, 0x69, 0x63, 0x61, 0x6c, 0x54, 0x79, + 0x70, 0x65, 0x42, 0x0b, 0x0a, 0x09, 0x74, 0x79, 0x70, 0x65, 0x5f, 0x69, 0x6e, 0x66, 0x6f, 0x22, + 0x5c, 0x0a, 0x09, 0x41, 0x72, 0x72, 0x61, 0x79, 0x54, 0x79, 0x70, 0x65, 0x12, 0x4f, 0x0a, 0x0c, + 0x65, 0x6c, 0x65, 0x6d, 0x65, 0x6e, 0x74, 0x5f, 0x74, 0x79, 0x70, 0x65, 0x18, 0x01, 0x20, 0x01, + 0x28, 0x0b, 0x32, 0x2c, 0x2e, 0x6f, 0x72, 0x67, 0x2e, 0x61, 0x70, 0x61, 0x63, 0x68, 0x65, 0x2e, + 0x62, 0x65, 0x61, 0x6d, 0x2e, 0x6d, 0x6f, 0x64, 0x65, 0x6c, 0x2e, 0x70, 0x69, 0x70, 0x65, 0x6c, + 0x69, 0x6e, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x46, 0x69, 0x65, 0x6c, 0x64, 0x54, 0x79, 0x70, 0x65, + 0x52, 0x0b, 0x65, 0x6c, 0x65, 0x6d, 0x65, 0x6e, 0x74, 0x54, 0x79, 0x70, 0x65, 0x22, 0x5f, 0x0a, + 0x0c, 0x49, 0x74, 0x65, 0x72, 0x61, 0x62, 0x6c, 0x65, 0x54, 0x79, 0x70, 0x65, 0x12, 0x4f, 0x0a, + 0x0c, 0x65, 0x6c, 0x65, 0x6d, 0x65, 0x6e, 0x74, 0x5f, 0x74, 0x79, 0x70, 0x65, 0x18, 0x01, 0x20, + 0x01, 0x28, 0x0b, 0x32, 0x2c, 0x2e, 0x6f, 0x72, 0x67, 0x2e, 0x61, 0x70, 0x61, 0x63, 0x68, 0x65, + 0x2e, 0x62, 0x65, 0x61, 0x6d, 0x2e, 0x6d, 0x6f, 0x64, 0x65, 0x6c, 0x2e, 0x70, 0x69, 0x70, 0x65, + 0x6c, 0x69, 0x6e, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x46, 0x69, 0x65, 0x6c, 0x64, 0x54, 0x79, 0x70, + 0x65, 0x52, 0x0b, 0x65, 0x6c, 0x65, 0x6d, 0x65, 0x6e, 0x74, 0x54, 0x79, 0x70, 0x65, 0x22, 0x9f, + 0x01, 0x0a, 0x07, 0x4d, 0x61, 0x70, 0x54, 0x79, 0x70, 0x65, 0x12, 0x47, 0x0a, 0x08, 0x6b, 0x65, + 0x79, 0x5f, 0x74, 0x79, 0x70, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x2c, 0x2e, 0x6f, + 0x72, 0x67, 0x2e, 0x61, 0x70, 0x61, 0x63, 0x68, 0x65, 0x2e, 0x62, 0x65, 0x61, 0x6d, 0x2e, 0x6d, + 0x6f, 0x64, 0x65, 0x6c, 0x2e, 0x70, 0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e, 0x65, 0x2e, 0x76, 0x31, + 0x2e, 0x46, 0x69, 0x65, 0x6c, 0x64, 0x54, 0x79, 0x70, 0x65, 0x52, 0x07, 0x6b, 0x65, 0x79, 0x54, + 0x79, 0x70, 0x65, 0x12, 0x4b, 0x0a, 0x0a, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x5f, 0x74, 0x79, 0x70, + 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x2c, 0x2e, 0x6f, 0x72, 0x67, 0x2e, 0x61, 0x70, + 0x61, 0x63, 0x68, 0x65, 0x2e, 0x62, 0x65, 0x61, 0x6d, 0x2e, 0x6d, 0x6f, 0x64, 0x65, 0x6c, 0x2e, + 0x70, 0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x46, 0x69, 0x65, 0x6c, + 0x64, 0x54, 0x79, 0x70, 0x65, 0x52, 0x09, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x54, 0x79, 0x70, 0x65, + 0x22, 0x4c, 0x0a, 0x07, 0x52, 0x6f, 0x77, 0x54, 0x79, 0x70, 0x65, 0x12, 0x41, 0x0a, 0x06, 0x73, + 0x63, 0x68, 0x65, 0x6d, 0x61, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x29, 0x2e, 0x6f, 0x72, + 0x67, 0x2e, 0x61, 0x70, 0x61, 0x63, 0x68, 0x65, 0x2e, 0x62, 0x65, 0x61, 0x6d, 0x2e, 0x6d, 0x6f, + 0x64, 0x65, 0x6c, 0x2e, 0x70, 0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e, 0x65, 0x2e, 0x76, 0x31, 0x2e, + 0x53, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x52, 0x06, 0x73, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x22, 0xad, + 0x02, 0x0a, 0x0b, 0x4c, 0x6f, 0x67, 0x69, 0x63, 0x61, 0x6c, 0x54, 0x79, 0x70, 0x65, 0x12, 0x10, + 0x0a, 0x03, 0x75, 0x72, 0x6e, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x03, 0x75, 0x72, 0x6e, + 0x12, 0x18, 0x0a, 0x07, 0x70, 0x61, 0x79, 0x6c, 0x6f, 0x61, 0x64, 0x18, 0x02, 0x20, 0x01, 0x28, + 0x0c, 0x52, 0x07, 0x70, 0x61, 0x79, 0x6c, 0x6f, 0x61, 0x64, 0x12, 0x54, 0x0a, 0x0e, 0x72, 0x65, + 0x70, 0x72, 0x65, 0x73, 0x65, 0x6e, 0x74, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x18, 0x03, 0x20, 0x01, + 0x28, 0x0b, 0x32, 0x2c, 0x2e, 0x6f, 0x72, 0x67, 0x2e, 0x61, 0x70, 0x61, 0x63, 0x68, 0x65, 0x2e, + 0x62, 0x65, 0x61, 0x6d, 0x2e, 0x6d, 0x6f, 0x64, 0x65, 0x6c, 0x2e, 0x70, 0x69, 0x70, 0x65, 0x6c, + 0x69, 0x6e, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x46, 0x69, 0x65, 0x6c, 0x64, 0x54, 0x79, 0x70, 0x65, + 0x52, 0x0e, 0x72, 0x65, 0x70, 0x72, 0x65, 0x73, 0x65, 0x6e, 0x74, 0x61, 0x74, 0x69, 0x6f, 0x6e, + 0x12, 0x51, 0x0a, 0x0d, 0x61, 0x72, 0x67, 0x75, 0x6d, 0x65, 0x6e, 0x74, 0x5f, 0x74, 0x79, 0x70, + 0x65, 0x18, 0x04, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x2c, 0x2e, 0x6f, 0x72, 0x67, 0x2e, 0x61, 0x70, + 0x61, 0x63, 0x68, 0x65, 0x2e, 0x62, 0x65, 0x61, 0x6d, 0x2e, 0x6d, 0x6f, 0x64, 0x65, 0x6c, 0x2e, + 0x70, 0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x46, 0x69, 0x65, 0x6c, + 0x64, 0x54, 0x79, 0x70, 0x65, 0x52, 0x0c, 0x61, 0x72, 0x67, 0x75, 0x6d, 0x65, 0x6e, 0x74, 0x54, + 0x79, 0x70, 0x65, 0x12, 0x49, 0x0a, 0x08, 0x61, 0x72, 0x67, 0x75, 0x6d, 0x65, 0x6e, 0x74, 0x18, + 0x05, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x2d, 0x2e, 0x6f, 0x72, 0x67, 0x2e, 0x61, 0x70, 0x61, 0x63, + 0x68, 0x65, 0x2e, 0x62, 0x65, 0x61, 0x6d, 0x2e, 0x6d, 0x6f, 0x64, 0x65, 0x6c, 0x2e, 0x70, 0x69, + 0x70, 0x65, 0x6c, 0x69, 0x6e, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x46, 0x69, 0x65, 0x6c, 0x64, 0x56, + 0x61, 0x6c, 0x75, 0x65, 0x52, 0x08, 0x61, 0x72, 0x67, 0x75, 0x6d, 0x65, 0x6e, 0x74, 0x22, 0xdf, + 0x03, 0x0a, 0x0c, 0x4c, 0x6f, 0x67, 0x69, 0x63, 0x61, 0x6c, 0x54, 0x79, 0x70, 0x65, 0x73, 0x22, + 0xce, 0x03, 0x0a, 0x04, 0x45, 0x6e, 0x75, 0x6d, 0x12, 0x3f, 0x0a, 0x0f, 0x50, 0x59, 0x54, 0x48, + 0x4f, 0x4e, 0x5f, 0x43, 0x41, 0x4c, 0x4c, 0x41, 0x42, 0x4c, 0x45, 0x10, 0x00, 0x1a, 0x2a, 0xa2, + 0xb4, 0xfa, 0xc2, 0x05, 0x24, 0x62, 0x65, 0x61, 0x6d, 0x3a, 0x6c, 0x6f, 0x67, 0x69, 0x63, 0x61, + 0x6c, 0x5f, 0x74, 0x79, 0x70, 0x65, 0x3a, 0x70, 0x79, 0x74, 0x68, 0x6f, 0x6e, 0x5f, 0x63, 0x61, + 0x6c, 0x6c, 0x61, 0x62, 0x6c, 0x65, 0x3a, 0x76, 0x31, 0x12, 0x3d, 0x0a, 0x0e, 0x4d, 0x49, 0x43, + 0x52, 0x4f, 0x53, 0x5f, 0x49, 0x4e, 0x53, 0x54, 0x41, 0x4e, 0x54, 0x10, 0x01, 0x1a, 0x29, 0xa2, + 0xb4, 0xfa, 0xc2, 0x05, 0x23, 0x62, 0x65, 0x61, 0x6d, 0x3a, 0x6c, 0x6f, 0x67, 0x69, 0x63, 0x61, + 0x6c, 0x5f, 0x74, 0x79, 0x70, 0x65, 0x3a, 0x6d, 0x69, 0x63, 0x72, 0x6f, 0x73, 0x5f, 0x69, 0x6e, + 0x73, 0x74, 0x61, 0x6e, 0x74, 0x3a, 0x76, 0x31, 0x12, 0x3d, 0x0a, 0x0e, 0x4d, 0x49, 0x4c, 0x4c, + 0x49, 0x53, 0x5f, 0x49, 0x4e, 0x53, 0x54, 0x41, 0x4e, 0x54, 0x10, 0x02, 0x1a, 0x29, 0xa2, 0xb4, + 0xfa, 0xc2, 0x05, 0x23, 0x62, 0x65, 0x61, 0x6d, 0x3a, 0x6c, 0x6f, 0x67, 0x69, 0x63, 0x61, 0x6c, + 0x5f, 0x74, 0x79, 0x70, 0x65, 0x3a, 0x6d, 0x69, 0x6c, 0x6c, 0x69, 0x73, 0x5f, 0x69, 0x6e, 0x73, + 0x74, 0x61, 0x6e, 0x74, 0x3a, 0x76, 0x31, 0x12, 0x2f, 0x0a, 0x07, 0x44, 0x45, 0x43, 0x49, 0x4d, + 0x41, 0x4c, 0x10, 0x03, 0x1a, 0x22, 0xa2, 0xb4, 0xfa, 0xc2, 0x05, 0x1c, 0x62, 0x65, 0x61, 0x6d, + 0x3a, 0x6c, 0x6f, 0x67, 0x69, 0x63, 0x61, 0x6c, 0x5f, 0x74, 0x79, 0x70, 0x65, 0x3a, 0x64, 0x65, + 0x63, 0x69, 0x6d, 0x61, 0x6c, 0x3a, 0x76, 0x31, 0x12, 0x37, 0x0a, 0x0b, 0x46, 0x49, 0x58, 0x45, + 0x44, 0x5f, 0x42, 0x59, 0x54, 0x45, 0x53, 0x10, 0x04, 0x1a, 0x26, 0xa2, 0xb4, 0xfa, 0xc2, 0x05, + 0x20, 0x62, 0x65, 0x61, 0x6d, 0x3a, 0x6c, 0x6f, 0x67, 0x69, 0x63, 0x61, 0x6c, 0x5f, 0x74, 0x79, + 0x70, 0x65, 0x3a, 0x66, 0x69, 0x78, 0x65, 0x64, 0x5f, 0x62, 0x79, 0x74, 0x65, 0x73, 0x3a, 0x76, + 0x31, 0x12, 0x33, 0x0a, 0x09, 0x56, 0x41, 0x52, 0x5f, 0x42, 0x59, 0x54, 0x45, 0x53, 0x10, 0x05, + 0x1a, 0x24, 0xa2, 0xb4, 0xfa, 0xc2, 0x05, 0x1e, 0x62, 0x65, 0x61, 0x6d, 0x3a, 0x6c, 0x6f, 0x67, + 0x69, 0x63, 0x61, 0x6c, 0x5f, 0x74, 0x79, 0x70, 0x65, 0x3a, 0x76, 0x61, 0x72, 0x5f, 0x62, 0x79, + 0x74, 0x65, 0x73, 0x3a, 0x76, 0x31, 0x12, 0x35, 0x0a, 0x0a, 0x46, 0x49, 0x58, 0x45, 0x44, 0x5f, + 0x43, 0x48, 0x41, 0x52, 0x10, 0x06, 0x1a, 0x25, 0xa2, 0xb4, 0xfa, 0xc2, 0x05, 0x1f, 0x62, 0x65, + 0x61, 0x6d, 0x3a, 0x6c, 0x6f, 0x67, 0x69, 0x63, 0x61, 0x6c, 0x5f, 0x74, 0x79, 0x70, 0x65, 0x3a, + 0x66, 0x69, 0x78, 0x65, 0x64, 0x5f, 0x63, 0x68, 0x61, 0x72, 0x3a, 0x76, 0x31, 0x12, 0x31, 0x0a, + 0x08, 0x56, 0x41, 0x52, 0x5f, 0x43, 0x48, 0x41, 0x52, 0x10, 0x07, 0x1a, 0x23, 0xa2, 0xb4, 0xfa, + 0xc2, 0x05, 0x1d, 0x62, 0x65, 0x61, 0x6d, 0x3a, 0x6c, 0x6f, 0x67, 0x69, 0x63, 0x61, 0x6c, 0x5f, + 0x74, 0x79, 0x70, 0x65, 0x3a, 0x76, 0x61, 0x72, 0x5f, 0x63, 0x68, 0x61, 0x72, 0x3a, 0x76, 0x31, + 0x22, 0xa3, 0x01, 0x0a, 0x06, 0x4f, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x12, 0x0a, 0x04, 0x6e, + 0x61, 0x6d, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x04, 0x6e, 0x61, 0x6d, 0x65, 0x12, + 0x40, 0x0a, 0x04, 0x74, 0x79, 0x70, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x2c, 0x2e, + 0x6f, 0x72, 0x67, 0x2e, 0x61, 0x70, 0x61, 0x63, 0x68, 0x65, 0x2e, 0x62, 0x65, 0x61, 0x6d, 0x2e, + 0x6d, 0x6f, 0x64, 0x65, 0x6c, 0x2e, 0x70, 0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e, 0x65, 0x2e, 0x76, + 0x31, 0x2e, 0x46, 0x69, 0x65, 0x6c, 0x64, 0x54, 0x79, 0x70, 0x65, 0x52, 0x04, 0x74, 0x79, 0x70, + 0x65, 0x12, 0x43, 0x0a, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0b, + 0x32, 0x2d, 0x2e, 0x6f, 0x72, 0x67, 0x2e, 0x61, 0x70, 0x61, 0x63, 0x68, 0x65, 0x2e, 0x62, 0x65, + 0x61, 0x6d, 0x2e, 0x6d, 0x6f, 0x64, 0x65, 0x6c, 0x2e, 0x70, 0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e, + 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x46, 0x69, 0x65, 0x6c, 0x64, 0x56, 0x61, 0x6c, 0x75, 0x65, 0x52, + 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x22, 0x4c, 0x0a, 0x03, 0x52, 0x6f, 0x77, 0x12, 0x45, 0x0a, + 0x06, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x73, 0x18, 0x01, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x2d, 0x2e, + 0x6f, 0x72, 0x67, 0x2e, 0x61, 0x70, 0x61, 0x63, 0x68, 0x65, 0x2e, 0x62, 0x65, 0x61, 0x6d, 0x2e, + 0x6d, 0x6f, 0x64, 0x65, 0x6c, 0x2e, 0x70, 0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e, 0x65, 0x2e, 0x76, + 0x31, 0x2e, 0x46, 0x69, 0x65, 0x6c, 0x64, 0x56, 0x61, 0x6c, 0x75, 0x65, 0x52, 0x06, 0x76, 0x61, + 0x6c, 0x75, 0x65, 0x73, 0x22, 0xa5, 0x04, 0x0a, 0x0a, 0x46, 0x69, 0x65, 0x6c, 0x64, 0x56, 0x61, + 0x6c, 0x75, 0x65, 0x12, 0x57, 0x0a, 0x0c, 0x61, 0x74, 0x6f, 0x6d, 0x69, 0x63, 0x5f, 0x76, 0x61, + 0x6c, 0x75, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x32, 0x2e, 0x6f, 0x72, 0x67, 0x2e, + 0x61, 0x70, 0x61, 0x63, 0x68, 0x65, 0x2e, 0x62, 0x65, 0x61, 0x6d, 0x2e, 0x6d, 0x6f, 0x64, 0x65, + 0x6c, 0x2e, 0x70, 0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x41, 0x74, + 0x6f, 0x6d, 0x69, 0x63, 0x54, 0x79, 0x70, 0x65, 0x56, 0x61, 0x6c, 0x75, 0x65, 0x48, 0x00, 0x52, + 0x0b, 0x61, 0x74, 0x6f, 0x6d, 0x69, 0x63, 0x56, 0x61, 0x6c, 0x75, 0x65, 0x12, 0x54, 0x0a, 0x0b, + 0x61, 0x72, 0x72, 0x61, 0x79, 0x5f, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, + 0x0b, 0x32, 0x31, 0x2e, 0x6f, 0x72, 0x67, 0x2e, 0x61, 0x70, 0x61, 0x63, 0x68, 0x65, 0x2e, 0x62, + 0x65, 0x61, 0x6d, 0x2e, 0x6d, 0x6f, 0x64, 0x65, 0x6c, 0x2e, 0x70, 0x69, 0x70, 0x65, 0x6c, 0x69, + 0x6e, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x41, 0x72, 0x72, 0x61, 0x79, 0x54, 0x79, 0x70, 0x65, 0x56, + 0x61, 0x6c, 0x75, 0x65, 0x48, 0x00, 0x52, 0x0a, 0x61, 0x72, 0x72, 0x61, 0x79, 0x56, 0x61, 0x6c, + 0x75, 0x65, 0x12, 0x5d, 0x0a, 0x0e, 0x69, 0x74, 0x65, 0x72, 0x61, 0x62, 0x6c, 0x65, 0x5f, 0x76, + 0x61, 0x6c, 0x75, 0x65, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x34, 0x2e, 0x6f, 0x72, 0x67, + 0x2e, 0x61, 0x70, 0x61, 0x63, 0x68, 0x65, 0x2e, 0x62, 0x65, 0x61, 0x6d, 0x2e, 0x6d, 0x6f, 0x64, + 0x65, 0x6c, 0x2e, 0x70, 0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x49, + 0x74, 0x65, 0x72, 0x61, 0x62, 0x6c, 0x65, 0x54, 0x79, 0x70, 0x65, 0x56, 0x61, 0x6c, 0x75, 0x65, + 0x48, 0x00, 0x52, 0x0d, 0x69, 0x74, 0x65, 0x72, 0x61, 0x62, 0x6c, 0x65, 0x56, 0x61, 0x6c, 0x75, + 0x65, 0x12, 0x4e, 0x0a, 0x09, 0x6d, 0x61, 0x70, 0x5f, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x18, 0x04, + 0x20, 0x01, 0x28, 0x0b, 0x32, 0x2f, 0x2e, 0x6f, 0x72, 0x67, 0x2e, 0x61, 0x70, 0x61, 0x63, 0x68, + 0x65, 0x2e, 0x62, 0x65, 0x61, 0x6d, 0x2e, 0x6d, 0x6f, 0x64, 0x65, 0x6c, 0x2e, 0x70, 0x69, 0x70, + 0x65, 0x6c, 0x69, 0x6e, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x4d, 0x61, 0x70, 0x54, 0x79, 0x70, 0x65, + 0x56, 0x61, 0x6c, 0x75, 0x65, 0x48, 0x00, 0x52, 0x08, 0x6d, 0x61, 0x70, 0x56, 0x61, 0x6c, 0x75, + 0x65, 0x12, 0x45, 0x0a, 0x09, 0x72, 0x6f, 0x77, 0x5f, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x18, 0x05, + 0x20, 0x01, 0x28, 0x0b, 0x32, 0x26, 0x2e, 0x6f, 0x72, 0x67, 0x2e, 0x61, 0x70, 0x61, 0x63, 0x68, + 0x65, 0x2e, 0x62, 0x65, 0x61, 0x6d, 0x2e, 0x6d, 0x6f, 0x64, 0x65, 0x6c, 0x2e, 0x70, 0x69, 0x70, + 0x65, 0x6c, 0x69, 0x6e, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x52, 0x6f, 0x77, 0x48, 0x00, 0x52, 0x08, + 0x72, 0x6f, 0x77, 0x56, 0x61, 0x6c, 0x75, 0x65, 0x12, 0x63, 0x0a, 0x12, 0x6c, 0x6f, 0x67, 0x69, + 0x63, 0x61, 0x6c, 0x5f, 0x74, 0x79, 0x70, 0x65, 0x5f, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x18, 0x06, + 0x20, 0x01, 0x28, 0x0b, 0x32, 0x33, 0x2e, 0x6f, 0x72, 0x67, 0x2e, 0x61, 0x70, 0x61, 0x63, 0x68, + 0x65, 0x2e, 0x62, 0x65, 0x61, 0x6d, 0x2e, 0x6d, 0x6f, 0x64, 0x65, 0x6c, 0x2e, 0x70, 0x69, 0x70, + 0x65, 0x6c, 0x69, 0x6e, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x4c, 0x6f, 0x67, 0x69, 0x63, 0x61, 0x6c, + 0x54, 0x79, 0x70, 0x65, 0x56, 0x61, 0x6c, 0x75, 0x65, 0x48, 0x00, 0x52, 0x10, 0x6c, 0x6f, 0x67, + 0x69, 0x63, 0x61, 0x6c, 0x54, 0x79, 0x70, 0x65, 0x56, 0x61, 0x6c, 0x75, 0x65, 0x42, 0x0d, 0x0a, + 0x0b, 0x66, 0x69, 0x65, 0x6c, 0x64, 0x5f, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x22, 0xf8, 0x01, 0x0a, + 0x0f, 0x41, 0x74, 0x6f, 0x6d, 0x69, 0x63, 0x54, 0x79, 0x70, 0x65, 0x56, 0x61, 0x6c, 0x75, 0x65, + 0x12, 0x14, 0x0a, 0x04, 0x62, 0x79, 0x74, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x05, 0x48, 0x00, + 0x52, 0x04, 0x62, 0x79, 0x74, 0x65, 0x12, 0x16, 0x0a, 0x05, 0x69, 0x6e, 0x74, 0x31, 0x36, 0x18, + 0x02, 0x20, 0x01, 0x28, 0x05, 0x48, 0x00, 0x52, 0x05, 0x69, 0x6e, 0x74, 0x31, 0x36, 0x12, 0x16, + 0x0a, 0x05, 0x69, 0x6e, 0x74, 0x33, 0x32, 0x18, 0x03, 0x20, 0x01, 0x28, 0x05, 0x48, 0x00, 0x52, + 0x05, 0x69, 0x6e, 0x74, 0x33, 0x32, 0x12, 0x16, 0x0a, 0x05, 0x69, 0x6e, 0x74, 0x36, 0x34, 0x18, + 0x04, 0x20, 0x01, 0x28, 0x03, 0x48, 0x00, 0x52, 0x05, 0x69, 0x6e, 0x74, 0x36, 0x34, 0x12, 0x16, + 0x0a, 0x05, 0x66, 0x6c, 0x6f, 0x61, 0x74, 0x18, 0x05, 0x20, 0x01, 0x28, 0x02, 0x48, 0x00, 0x52, + 0x05, 0x66, 0x6c, 0x6f, 0x61, 0x74, 0x12, 0x18, 0x0a, 0x06, 0x64, 0x6f, 0x75, 0x62, 0x6c, 0x65, + 0x18, 0x06, 0x20, 0x01, 0x28, 0x01, 0x48, 0x00, 0x52, 0x06, 0x64, 0x6f, 0x75, 0x62, 0x6c, 0x65, + 0x12, 0x18, 0x0a, 0x06, 0x73, 0x74, 0x72, 0x69, 0x6e, 0x67, 0x18, 0x07, 0x20, 0x01, 0x28, 0x09, + 0x48, 0x00, 0x52, 0x06, 0x73, 0x74, 0x72, 0x69, 0x6e, 0x67, 0x12, 0x1a, 0x0a, 0x07, 0x62, 0x6f, + 0x6f, 0x6c, 0x65, 0x61, 0x6e, 0x18, 0x08, 0x20, 0x01, 0x28, 0x08, 0x48, 0x00, 0x52, 0x07, 0x62, + 0x6f, 0x6f, 0x6c, 0x65, 0x61, 0x6e, 0x12, 0x16, 0x0a, 0x05, 0x62, 0x79, 0x74, 0x65, 0x73, 0x18, + 0x09, 0x20, 0x01, 0x28, 0x0c, 0x48, 0x00, 0x52, 0x05, 0x62, 0x79, 0x74, 0x65, 0x73, 0x42, 0x07, + 0x0a, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x22, 0x59, 0x0a, 0x0e, 0x41, 0x72, 0x72, 0x61, 0x79, + 0x54, 0x79, 0x70, 0x65, 0x56, 0x61, 0x6c, 0x75, 0x65, 0x12, 0x47, 0x0a, 0x07, 0x65, 0x6c, 0x65, + 0x6d, 0x65, 0x6e, 0x74, 0x18, 0x01, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x2d, 0x2e, 0x6f, 0x72, 0x67, + 0x2e, 0x61, 0x70, 0x61, 0x63, 0x68, 0x65, 0x2e, 0x62, 0x65, 0x61, 0x6d, 0x2e, 0x6d, 0x6f, 0x64, + 0x65, 0x6c, 0x2e, 0x70, 0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x46, + 0x69, 0x65, 0x6c, 0x64, 0x56, 0x61, 0x6c, 0x75, 0x65, 0x52, 0x07, 0x65, 0x6c, 0x65, 0x6d, 0x65, + 0x6e, 0x74, 0x22, 0x5c, 0x0a, 0x11, 0x49, 0x74, 0x65, 0x72, 0x61, 0x62, 0x6c, 0x65, 0x54, 0x79, + 0x70, 0x65, 0x56, 0x61, 0x6c, 0x75, 0x65, 0x12, 0x47, 0x0a, 0x07, 0x65, 0x6c, 0x65, 0x6d, 0x65, + 0x6e, 0x74, 0x18, 0x01, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x2d, 0x2e, 0x6f, 0x72, 0x67, 0x2e, 0x61, + 0x70, 0x61, 0x63, 0x68, 0x65, 0x2e, 0x62, 0x65, 0x61, 0x6d, 0x2e, 0x6d, 0x6f, 0x64, 0x65, 0x6c, + 0x2e, 0x70, 0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x46, 0x69, 0x65, + 0x6c, 0x64, 0x56, 0x61, 0x6c, 0x75, 0x65, 0x52, 0x07, 0x65, 0x6c, 0x65, 0x6d, 0x65, 0x6e, 0x74, + 0x22, 0x59, 0x0a, 0x0c, 0x4d, 0x61, 0x70, 0x54, 0x79, 0x70, 0x65, 0x56, 0x61, 0x6c, 0x75, 0x65, + 0x12, 0x49, 0x0a, 0x07, 0x65, 0x6e, 0x74, 0x72, 0x69, 0x65, 0x73, 0x18, 0x01, 0x20, 0x03, 0x28, + 0x0b, 0x32, 0x2f, 0x2e, 0x6f, 0x72, 0x67, 0x2e, 0x61, 0x70, 0x61, 0x63, 0x68, 0x65, 0x2e, 0x62, + 0x65, 0x61, 0x6d, 0x2e, 0x6d, 0x6f, 0x64, 0x65, 0x6c, 0x2e, 0x70, 0x69, 0x70, 0x65, 0x6c, 0x69, + 0x6e, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x4d, 0x61, 0x70, 0x54, 0x79, 0x70, 0x65, 0x45, 0x6e, 0x74, + 0x72, 0x79, 0x52, 0x07, 0x65, 0x6e, 0x74, 0x72, 0x69, 0x65, 0x73, 0x22, 0x94, 0x01, 0x0a, 0x0c, + 0x4d, 0x61, 0x70, 0x54, 0x79, 0x70, 0x65, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x12, 0x3f, 0x0a, 0x03, + 0x6b, 0x65, 0x79, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x2d, 0x2e, 0x6f, 0x72, 0x67, 0x2e, + 0x61, 0x70, 0x61, 0x63, 0x68, 0x65, 0x2e, 0x62, 0x65, 0x61, 0x6d, 0x2e, 0x6d, 0x6f, 0x64, 0x65, + 0x6c, 0x2e, 0x70, 0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x46, 0x69, + 0x65, 0x6c, 0x64, 0x56, 0x61, 0x6c, 0x75, 0x65, 0x52, 0x03, 0x6b, 0x65, 0x79, 0x12, 0x43, 0x0a, + 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x2d, 0x2e, 0x6f, + 0x72, 0x67, 0x2e, 0x61, 0x70, 0x61, 0x63, 0x68, 0x65, 0x2e, 0x62, 0x65, 0x61, 0x6d, 0x2e, 0x6d, + 0x6f, 0x64, 0x65, 0x6c, 0x2e, 0x70, 0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e, 0x65, 0x2e, 0x76, 0x31, + 0x2e, 0x46, 0x69, 0x65, 0x6c, 0x64, 0x56, 0x61, 0x6c, 0x75, 0x65, 0x52, 0x05, 0x76, 0x61, 0x6c, + 0x75, 0x65, 0x22, 0x57, 0x0a, 0x10, 0x4c, 0x6f, 0x67, 0x69, 0x63, 0x61, 0x6c, 0x54, 0x79, 0x70, + 0x65, 0x56, 0x61, 0x6c, 0x75, 0x65, 0x12, 0x43, 0x0a, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x18, + 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x2d, 0x2e, 0x6f, 0x72, 0x67, 0x2e, 0x61, 0x70, 0x61, 0x63, + 0x68, 0x65, 0x2e, 0x62, 0x65, 0x61, 0x6d, 0x2e, 0x6d, 0x6f, 0x64, 0x65, 0x6c, 0x2e, 0x70, 0x69, + 0x70, 0x65, 0x6c, 0x69, 0x6e, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x46, 0x69, 0x65, 0x6c, 0x64, 0x56, + 0x61, 0x6c, 0x75, 0x65, 0x52, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x2a, 0x83, 0x01, 0x0a, 0x0a, + 0x41, 0x74, 0x6f, 0x6d, 0x69, 0x63, 0x54, 0x79, 0x70, 0x65, 0x12, 0x0f, 0x0a, 0x0b, 0x55, 0x4e, + 0x53, 0x50, 0x45, 0x43, 0x49, 0x46, 0x49, 0x45, 0x44, 0x10, 0x00, 0x12, 0x08, 0x0a, 0x04, 0x42, + 0x59, 0x54, 0x45, 0x10, 0x01, 0x12, 0x09, 0x0a, 0x05, 0x49, 0x4e, 0x54, 0x31, 0x36, 0x10, 0x02, + 0x12, 0x09, 0x0a, 0x05, 0x49, 0x4e, 0x54, 0x33, 0x32, 0x10, 0x03, 0x12, 0x09, 0x0a, 0x05, 0x49, + 0x4e, 0x54, 0x36, 0x34, 0x10, 0x04, 0x12, 0x09, 0x0a, 0x05, 0x46, 0x4c, 0x4f, 0x41, 0x54, 0x10, + 0x05, 0x12, 0x0a, 0x0a, 0x06, 0x44, 0x4f, 0x55, 0x42, 0x4c, 0x45, 0x10, 0x06, 0x12, 0x0a, 0x0a, + 0x06, 0x53, 0x54, 0x52, 0x49, 0x4e, 0x47, 0x10, 0x07, 0x12, 0x0b, 0x0a, 0x07, 0x42, 0x4f, 0x4f, + 0x4c, 0x45, 0x41, 0x4e, 0x10, 0x08, 0x12, 0x09, 0x0a, 0x05, 0x42, 0x59, 0x54, 0x45, 0x53, 0x10, + 0x09, 0x42, 0x78, 0x0a, 0x21, 0x6f, 0x72, 0x67, 0x2e, 0x61, 0x70, 0x61, 0x63, 0x68, 0x65, 0x2e, + 0x62, 0x65, 0x61, 0x6d, 0x2e, 0x6d, 0x6f, 0x64, 0x65, 0x6c, 0x2e, 0x70, 0x69, 0x70, 0x65, 0x6c, + 0x69, 0x6e, 0x65, 0x2e, 0x76, 0x31, 0x42, 0x09, 0x53, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x41, 0x70, + 0x69, 0x5a, 0x48, 0x67, 0x69, 0x74, 0x68, 0x75, 0x62, 0x2e, 0x63, 0x6f, 0x6d, 0x2f, 0x61, 0x70, + 0x61, 0x63, 0x68, 0x65, 0x2f, 0x62, 0x65, 0x61, 0x6d, 0x2f, 0x73, 0x64, 0x6b, 0x73, 0x2f, 0x76, + 0x32, 0x2f, 0x67, 0x6f, 0x2f, 0x70, 0x6b, 0x67, 0x2f, 0x62, 0x65, 0x61, 0x6d, 0x2f, 0x6d, 0x6f, + 0x64, 0x65, 0x6c, 0x2f, 0x70, 0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e, 0x65, 0x5f, 0x76, 0x31, 0x3b, + 0x70, 0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e, 0x65, 0x5f, 0x76, 0x31, 0x62, 0x06, 0x70, 0x72, 0x6f, + 0x74, 0x6f, 0x33, +} + +var file_org_apache_beam_model_pipeline_v1_schema_proto_enumTypes = make([]protoimpl.EnumInfo, 2) +var file_org_apache_beam_model_pipeline_v1_schema_proto_msgTypes = make([]protoimpl.MessageInfo, 18) +var file_org_apache_beam_model_pipeline_v1_schema_proto_goTypes = []any{ + (AtomicType)(0), // 0: org.apache.beam.model.pipeline.v1.AtomicType + (LogicalTypes_Enum)(0), // 1: org.apache.beam.model.pipeline.v1.LogicalTypes.Enum + (*Schema)(nil), // 2: org.apache.beam.model.pipeline.v1.Schema + (*Field)(nil), // 3: org.apache.beam.model.pipeline.v1.Field + (*FieldType)(nil), // 4: org.apache.beam.model.pipeline.v1.FieldType + (*ArrayType)(nil), // 5: org.apache.beam.model.pipeline.v1.ArrayType + (*IterableType)(nil), // 6: org.apache.beam.model.pipeline.v1.IterableType + (*MapType)(nil), // 7: org.apache.beam.model.pipeline.v1.MapType + (*RowType)(nil), // 8: org.apache.beam.model.pipeline.v1.RowType + (*LogicalType)(nil), // 9: org.apache.beam.model.pipeline.v1.LogicalType + (*LogicalTypes)(nil), // 10: org.apache.beam.model.pipeline.v1.LogicalTypes + (*Option)(nil), // 11: org.apache.beam.model.pipeline.v1.Option + (*Row)(nil), // 12: org.apache.beam.model.pipeline.v1.Row + (*FieldValue)(nil), // 13: org.apache.beam.model.pipeline.v1.FieldValue + (*AtomicTypeValue)(nil), // 14: org.apache.beam.model.pipeline.v1.AtomicTypeValue + (*ArrayTypeValue)(nil), // 15: org.apache.beam.model.pipeline.v1.ArrayTypeValue + (*IterableTypeValue)(nil), // 16: org.apache.beam.model.pipeline.v1.IterableTypeValue + (*MapTypeValue)(nil), // 17: org.apache.beam.model.pipeline.v1.MapTypeValue + (*MapTypeEntry)(nil), // 18: org.apache.beam.model.pipeline.v1.MapTypeEntry + (*LogicalTypeValue)(nil), // 19: org.apache.beam.model.pipeline.v1.LogicalTypeValue +} +var file_org_apache_beam_model_pipeline_v1_schema_proto_depIdxs = []int32{ + 3, // 0: org.apache.beam.model.pipeline.v1.Schema.fields:type_name -> org.apache.beam.model.pipeline.v1.Field + 11, // 1: org.apache.beam.model.pipeline.v1.Schema.options:type_name -> org.apache.beam.model.pipeline.v1.Option + 4, // 2: org.apache.beam.model.pipeline.v1.Field.type:type_name -> org.apache.beam.model.pipeline.v1.FieldType + 11, // 3: org.apache.beam.model.pipeline.v1.Field.options:type_name -> org.apache.beam.model.pipeline.v1.Option + 0, // 4: org.apache.beam.model.pipeline.v1.FieldType.atomic_type:type_name -> org.apache.beam.model.pipeline.v1.AtomicType + 5, // 5: org.apache.beam.model.pipeline.v1.FieldType.array_type:type_name -> org.apache.beam.model.pipeline.v1.ArrayType + 6, // 6: org.apache.beam.model.pipeline.v1.FieldType.iterable_type:type_name -> org.apache.beam.model.pipeline.v1.IterableType + 7, // 7: org.apache.beam.model.pipeline.v1.FieldType.map_type:type_name -> org.apache.beam.model.pipeline.v1.MapType + 8, // 8: org.apache.beam.model.pipeline.v1.FieldType.row_type:type_name -> org.apache.beam.model.pipeline.v1.RowType + 9, // 9: org.apache.beam.model.pipeline.v1.FieldType.logical_type:type_name -> org.apache.beam.model.pipeline.v1.LogicalType + 4, // 10: org.apache.beam.model.pipeline.v1.ArrayType.element_type:type_name -> org.apache.beam.model.pipeline.v1.FieldType + 4, // 11: org.apache.beam.model.pipeline.v1.IterableType.element_type:type_name -> org.apache.beam.model.pipeline.v1.FieldType + 4, // 12: org.apache.beam.model.pipeline.v1.MapType.key_type:type_name -> org.apache.beam.model.pipeline.v1.FieldType + 4, // 13: org.apache.beam.model.pipeline.v1.MapType.value_type:type_name -> org.apache.beam.model.pipeline.v1.FieldType + 2, // 14: org.apache.beam.model.pipeline.v1.RowType.schema:type_name -> org.apache.beam.model.pipeline.v1.Schema + 4, // 15: org.apache.beam.model.pipeline.v1.LogicalType.representation:type_name -> org.apache.beam.model.pipeline.v1.FieldType + 4, // 16: org.apache.beam.model.pipeline.v1.LogicalType.argument_type:type_name -> org.apache.beam.model.pipeline.v1.FieldType + 13, // 17: org.apache.beam.model.pipeline.v1.LogicalType.argument:type_name -> org.apache.beam.model.pipeline.v1.FieldValue + 4, // 18: org.apache.beam.model.pipeline.v1.Option.type:type_name -> org.apache.beam.model.pipeline.v1.FieldType + 13, // 19: org.apache.beam.model.pipeline.v1.Option.value:type_name -> org.apache.beam.model.pipeline.v1.FieldValue + 13, // 20: org.apache.beam.model.pipeline.v1.Row.values:type_name -> org.apache.beam.model.pipeline.v1.FieldValue + 14, // 21: org.apache.beam.model.pipeline.v1.FieldValue.atomic_value:type_name -> org.apache.beam.model.pipeline.v1.AtomicTypeValue + 15, // 22: org.apache.beam.model.pipeline.v1.FieldValue.array_value:type_name -> org.apache.beam.model.pipeline.v1.ArrayTypeValue + 16, // 23: org.apache.beam.model.pipeline.v1.FieldValue.iterable_value:type_name -> org.apache.beam.model.pipeline.v1.IterableTypeValue + 17, // 24: org.apache.beam.model.pipeline.v1.FieldValue.map_value:type_name -> org.apache.beam.model.pipeline.v1.MapTypeValue + 12, // 25: org.apache.beam.model.pipeline.v1.FieldValue.row_value:type_name -> org.apache.beam.model.pipeline.v1.Row + 19, // 26: org.apache.beam.model.pipeline.v1.FieldValue.logical_type_value:type_name -> org.apache.beam.model.pipeline.v1.LogicalTypeValue + 13, // 27: org.apache.beam.model.pipeline.v1.ArrayTypeValue.element:type_name -> org.apache.beam.model.pipeline.v1.FieldValue + 13, // 28: org.apache.beam.model.pipeline.v1.IterableTypeValue.element:type_name -> org.apache.beam.model.pipeline.v1.FieldValue + 18, // 29: org.apache.beam.model.pipeline.v1.MapTypeValue.entries:type_name -> org.apache.beam.model.pipeline.v1.MapTypeEntry + 13, // 30: org.apache.beam.model.pipeline.v1.MapTypeEntry.key:type_name -> org.apache.beam.model.pipeline.v1.FieldValue + 13, // 31: org.apache.beam.model.pipeline.v1.MapTypeEntry.value:type_name -> org.apache.beam.model.pipeline.v1.FieldValue + 13, // 32: org.apache.beam.model.pipeline.v1.LogicalTypeValue.value:type_name -> org.apache.beam.model.pipeline.v1.FieldValue + 33, // [33:33] is the sub-list for method output_type + 33, // [33:33] is the sub-list for method input_type + 33, // [33:33] is the sub-list for extension type_name + 33, // [33:33] is the sub-list for extension extendee + 0, // [0:33] is the sub-list for field type_name +} + +func init() { file_org_apache_beam_model_pipeline_v1_schema_proto_init() } +func file_org_apache_beam_model_pipeline_v1_schema_proto_init() { + if File_org_apache_beam_model_pipeline_v1_schema_proto != nil { + return + } + file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_init() + file_org_apache_beam_model_pipeline_v1_schema_proto_msgTypes[2].OneofWrappers = []any{ + (*fieldType_AtomicType)(nil), + (*fieldType_ArrayType)(nil), + (*fieldType_IterableType)(nil), + (*fieldType_MapType)(nil), + (*fieldType_RowType)(nil), + (*fieldType_LogicalType)(nil), + } + file_org_apache_beam_model_pipeline_v1_schema_proto_msgTypes[11].OneofWrappers = []any{ + (*fieldValue_AtomicValue)(nil), + (*fieldValue_ArrayValue)(nil), + (*fieldValue_IterableValue)(nil), + (*fieldValue_MapValue)(nil), + (*fieldValue_RowValue)(nil), + (*fieldValue_LogicalTypeValue)(nil), + } + file_org_apache_beam_model_pipeline_v1_schema_proto_msgTypes[12].OneofWrappers = []any{ + (*atomicTypeValue_Byte)(nil), + (*atomicTypeValue_Int16)(nil), + (*atomicTypeValue_Int32)(nil), + (*atomicTypeValue_Int64)(nil), + (*atomicTypeValue_Float)(nil), + (*atomicTypeValue_Double)(nil), + (*atomicTypeValue_String_)(nil), + (*atomicTypeValue_Boolean)(nil), + (*atomicTypeValue_Bytes)(nil), + } + type x struct{} + out := protoimpl.TypeBuilder{ + File: protoimpl.DescBuilder{ + GoPackagePath: reflect.TypeOf(x{}).PkgPath(), + RawDescriptor: file_org_apache_beam_model_pipeline_v1_schema_proto_rawDesc, + NumEnums: 2, + NumMessages: 18, + NumExtensions: 0, + NumServices: 0, + }, + GoTypes: file_org_apache_beam_model_pipeline_v1_schema_proto_goTypes, + DependencyIndexes: file_org_apache_beam_model_pipeline_v1_schema_proto_depIdxs, + EnumInfos: file_org_apache_beam_model_pipeline_v1_schema_proto_enumTypes, + MessageInfos: file_org_apache_beam_model_pipeline_v1_schema_proto_msgTypes, + }.Build() + File_org_apache_beam_model_pipeline_v1_schema_proto = out.File + file_org_apache_beam_model_pipeline_v1_schema_proto_rawDesc = nil + file_org_apache_beam_model_pipeline_v1_schema_proto_goTypes = nil + file_org_apache_beam_model_pipeline_v1_schema_proto_depIdxs = nil +} diff --git a/sdks/go/pkg/beam/model/pipeline_v1/standard_window_fns.pb.go b/sdks/go/pkg/beam/model/pipeline_v1/standard_window_fns.pb.go index 5a7c3ebd81e0..6e49f3ee5da0 100644 --- a/sdks/go/pkg/beam/model/pipeline_v1/standard_window_fns.pb.go +++ b/sdks/go/pkg/beam/model/pipeline_v1/standard_window_fns.pb.go @@ -21,10 +21,12 @@ // Code generated by protoc-gen-go. DO NOT EDIT. // versions: -// protoc-gen-go v1.27.1 +// protoc-gen-go v1.36.0 // protoc v5.27.3 // source: org/apache/beam/model/pipeline/v1/standard_window_fns.proto +//go:build !protoopaque + package pipeline_v1 import ( @@ -33,7 +35,6 @@ import ( durationpb "google.golang.org/protobuf/types/known/durationpb" timestamppb "google.golang.org/protobuf/types/known/timestamppb" reflect "reflect" - sync "sync" ) const ( @@ -81,11 +82,6 @@ func (x GlobalWindowsPayload_Enum) Number() protoreflect.EnumNumber { return protoreflect.EnumNumber(x) } -// Deprecated: Use GlobalWindowsPayload_Enum.Descriptor instead. -func (GlobalWindowsPayload_Enum) EnumDescriptor() ([]byte, []int) { - return file_org_apache_beam_model_pipeline_v1_standard_window_fns_proto_rawDescGZIP(), []int{0, 0} -} - type FixedWindowsPayload_Enum int32 const ( @@ -124,11 +120,6 @@ func (x FixedWindowsPayload_Enum) Number() protoreflect.EnumNumber { return protoreflect.EnumNumber(x) } -// Deprecated: Use FixedWindowsPayload_Enum.Descriptor instead. -func (FixedWindowsPayload_Enum) EnumDescriptor() ([]byte, []int) { - return file_org_apache_beam_model_pipeline_v1_standard_window_fns_proto_rawDescGZIP(), []int{1, 0} -} - type SlidingWindowsPayload_Enum int32 const ( @@ -167,11 +158,6 @@ func (x SlidingWindowsPayload_Enum) Number() protoreflect.EnumNumber { return protoreflect.EnumNumber(x) } -// Deprecated: Use SlidingWindowsPayload_Enum.Descriptor instead. -func (SlidingWindowsPayload_Enum) EnumDescriptor() ([]byte, []int) { - return file_org_apache_beam_model_pipeline_v1_standard_window_fns_proto_rawDescGZIP(), []int{2, 0} -} - type SessionWindowsPayload_Enum int32 const ( @@ -210,29 +196,22 @@ func (x SessionWindowsPayload_Enum) Number() protoreflect.EnumNumber { return protoreflect.EnumNumber(x) } -// Deprecated: Use SessionWindowsPayload_Enum.Descriptor instead. -func (SessionWindowsPayload_Enum) EnumDescriptor() ([]byte, []int) { - return file_org_apache_beam_model_pipeline_v1_standard_window_fns_proto_rawDescGZIP(), []int{3, 0} -} - // By default, all data in a PCollection is assigned to the single global // window. See BeamConstants for the time span this window encompasses. // // See https://beam.apache.org/documentation/programming-guide/#single-global-window // for additional details. type GlobalWindowsPayload struct { - state protoimpl.MessageState - sizeCache protoimpl.SizeCache + state protoimpl.MessageState `protogen:"hybrid.v1"` unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache } func (x *GlobalWindowsPayload) Reset() { *x = GlobalWindowsPayload{} - if protoimpl.UnsafeEnabled { - mi := &file_org_apache_beam_model_pipeline_v1_standard_window_fns_proto_msgTypes[0] - ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) - ms.StoreMessageInfo(mi) - } + mi := &file_org_apache_beam_model_pipeline_v1_standard_window_fns_proto_msgTypes[0] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) } func (x *GlobalWindowsPayload) String() string { @@ -243,7 +222,7 @@ func (*GlobalWindowsPayload) ProtoMessage() {} func (x *GlobalWindowsPayload) ProtoReflect() protoreflect.Message { mi := &file_org_apache_beam_model_pipeline_v1_standard_window_fns_proto_msgTypes[0] - if protoimpl.UnsafeEnabled && x != nil { + if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { ms.StoreMessageInfo(mi) @@ -253,9 +232,16 @@ func (x *GlobalWindowsPayload) ProtoReflect() protoreflect.Message { return mi.MessageOf(x) } -// Deprecated: Use GlobalWindowsPayload.ProtoReflect.Descriptor instead. -func (*GlobalWindowsPayload) Descriptor() ([]byte, []int) { - return file_org_apache_beam_model_pipeline_v1_standard_window_fns_proto_rawDescGZIP(), []int{0} +type GlobalWindowsPayload_builder struct { + _ [0]func() // Prevents comparability and use of unkeyed literals for the builder. + +} + +func (b0 GlobalWindowsPayload_builder) Build() *GlobalWindowsPayload { + m0 := &GlobalWindowsPayload{} + b, x := &b0, m0 + _, _ = b, x + return m0 } // A fixed time window represents a consistent duration size, non overlapping @@ -264,24 +250,21 @@ func (*GlobalWindowsPayload) Descriptor() ([]byte, []int) { // See https://beam.apache.org/documentation/programming-guide/#fixed-time-windows // for additional details. type FixedWindowsPayload struct { - state protoimpl.MessageState - sizeCache protoimpl.SizeCache - unknownFields protoimpl.UnknownFields - + state protoimpl.MessageState `protogen:"hybrid.v1"` // (Required) Represents the size of the window. Size *durationpb.Duration `protobuf:"bytes,1,opt,name=size,proto3" json:"size,omitempty"` // (Required) Represents the timestamp of when the first window begins. // Window N will start at offset + N * size. - Offset *timestamppb.Timestamp `protobuf:"bytes,2,opt,name=offset,proto3" json:"offset,omitempty"` + Offset *timestamppb.Timestamp `protobuf:"bytes,2,opt,name=offset,proto3" json:"offset,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache } func (x *FixedWindowsPayload) Reset() { *x = FixedWindowsPayload{} - if protoimpl.UnsafeEnabled { - mi := &file_org_apache_beam_model_pipeline_v1_standard_window_fns_proto_msgTypes[1] - ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) - ms.StoreMessageInfo(mi) - } + mi := &file_org_apache_beam_model_pipeline_v1_standard_window_fns_proto_msgTypes[1] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) } func (x *FixedWindowsPayload) String() string { @@ -292,7 +275,7 @@ func (*FixedWindowsPayload) ProtoMessage() {} func (x *FixedWindowsPayload) ProtoReflect() protoreflect.Message { mi := &file_org_apache_beam_model_pipeline_v1_standard_window_fns_proto_msgTypes[1] - if protoimpl.UnsafeEnabled && x != nil { + if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { ms.StoreMessageInfo(mi) @@ -302,11 +285,6 @@ func (x *FixedWindowsPayload) ProtoReflect() protoreflect.Message { return mi.MessageOf(x) } -// Deprecated: Use FixedWindowsPayload.ProtoReflect.Descriptor instead. -func (*FixedWindowsPayload) Descriptor() ([]byte, []int) { - return file_org_apache_beam_model_pipeline_v1_standard_window_fns_proto_rawDescGZIP(), []int{1} -} - func (x *FixedWindowsPayload) GetSize() *durationpb.Duration { if x != nil { return x.Size @@ -321,6 +299,55 @@ func (x *FixedWindowsPayload) GetOffset() *timestamppb.Timestamp { return nil } +func (x *FixedWindowsPayload) SetSize(v *durationpb.Duration) { + x.Size = v +} + +func (x *FixedWindowsPayload) SetOffset(v *timestamppb.Timestamp) { + x.Offset = v +} + +func (x *FixedWindowsPayload) HasSize() bool { + if x == nil { + return false + } + return x.Size != nil +} + +func (x *FixedWindowsPayload) HasOffset() bool { + if x == nil { + return false + } + return x.Offset != nil +} + +func (x *FixedWindowsPayload) ClearSize() { + x.Size = nil +} + +func (x *FixedWindowsPayload) ClearOffset() { + x.Offset = nil +} + +type FixedWindowsPayload_builder struct { + _ [0]func() // Prevents comparability and use of unkeyed literals for the builder. + + // (Required) Represents the size of the window. + Size *durationpb.Duration + // (Required) Represents the timestamp of when the first window begins. + // Window N will start at offset + N * size. + Offset *timestamppb.Timestamp +} + +func (b0 FixedWindowsPayload_builder) Build() *FixedWindowsPayload { + m0 := &FixedWindowsPayload{} + b, x := &b0, m0 + _, _ = b, x + x.Size = b.Size + x.Offset = b.Offset + return m0 +} + // A sliding time window represents time intervals in the data stream that can // overlap. For example, each window might capture 60 seconds worth of data, but // a new window starts every 30 seconds. The frequency with which sliding @@ -336,26 +363,23 @@ func (x *FixedWindowsPayload) GetOffset() *timestamppb.Timestamp { // See https://beam.apache.org/documentation/programming-guide/#sliding-time-windows // for additional details. type SlidingWindowsPayload struct { - state protoimpl.MessageState - sizeCache protoimpl.SizeCache - unknownFields protoimpl.UnknownFields - + state protoimpl.MessageState `protogen:"hybrid.v1"` // (Required) Represents the size of the window. Size *durationpb.Duration `protobuf:"bytes,1,opt,name=size,proto3" json:"size,omitempty"` // (Required) Represents the timestamp of when the first window begins. // Window N will start at offset + N * period. Offset *timestamppb.Timestamp `protobuf:"bytes,2,opt,name=offset,proto3" json:"offset,omitempty"` // (Required) Represents the amount of time between each start of a window. - Period *durationpb.Duration `protobuf:"bytes,3,opt,name=period,proto3" json:"period,omitempty"` + Period *durationpb.Duration `protobuf:"bytes,3,opt,name=period,proto3" json:"period,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache } func (x *SlidingWindowsPayload) Reset() { *x = SlidingWindowsPayload{} - if protoimpl.UnsafeEnabled { - mi := &file_org_apache_beam_model_pipeline_v1_standard_window_fns_proto_msgTypes[2] - ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) - ms.StoreMessageInfo(mi) - } + mi := &file_org_apache_beam_model_pipeline_v1_standard_window_fns_proto_msgTypes[2] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) } func (x *SlidingWindowsPayload) String() string { @@ -366,7 +390,7 @@ func (*SlidingWindowsPayload) ProtoMessage() {} func (x *SlidingWindowsPayload) ProtoReflect() protoreflect.Message { mi := &file_org_apache_beam_model_pipeline_v1_standard_window_fns_proto_msgTypes[2] - if protoimpl.UnsafeEnabled && x != nil { + if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { ms.StoreMessageInfo(mi) @@ -376,11 +400,6 @@ func (x *SlidingWindowsPayload) ProtoReflect() protoreflect.Message { return mi.MessageOf(x) } -// Deprecated: Use SlidingWindowsPayload.ProtoReflect.Descriptor instead. -func (*SlidingWindowsPayload) Descriptor() ([]byte, []int) { - return file_org_apache_beam_model_pipeline_v1_standard_window_fns_proto_rawDescGZIP(), []int{2} -} - func (x *SlidingWindowsPayload) GetSize() *durationpb.Duration { if x != nil { return x.Size @@ -402,6 +421,73 @@ func (x *SlidingWindowsPayload) GetPeriod() *durationpb.Duration { return nil } +func (x *SlidingWindowsPayload) SetSize(v *durationpb.Duration) { + x.Size = v +} + +func (x *SlidingWindowsPayload) SetOffset(v *timestamppb.Timestamp) { + x.Offset = v +} + +func (x *SlidingWindowsPayload) SetPeriod(v *durationpb.Duration) { + x.Period = v +} + +func (x *SlidingWindowsPayload) HasSize() bool { + if x == nil { + return false + } + return x.Size != nil +} + +func (x *SlidingWindowsPayload) HasOffset() bool { + if x == nil { + return false + } + return x.Offset != nil +} + +func (x *SlidingWindowsPayload) HasPeriod() bool { + if x == nil { + return false + } + return x.Period != nil +} + +func (x *SlidingWindowsPayload) ClearSize() { + x.Size = nil +} + +func (x *SlidingWindowsPayload) ClearOffset() { + x.Offset = nil +} + +func (x *SlidingWindowsPayload) ClearPeriod() { + x.Period = nil +} + +type SlidingWindowsPayload_builder struct { + _ [0]func() // Prevents comparability and use of unkeyed literals for the builder. + + // (Required) Represents the size of the window. + Size *durationpb.Duration + // (Required) Represents the timestamp of when the first window begins. + // Window N will start at offset + N * period. + Offset *timestamppb.Timestamp + // (Required) Represents the amount of time between each start of a window. + Period *durationpb.Duration +} + +func (b0 SlidingWindowsPayload_builder) Build() *SlidingWindowsPayload { + m0 := &SlidingWindowsPayload{} + b, x := &b0, m0 + _, _ = b, x + x.Size = b.Size + x.Offset = b.Offset + x.Period = b.Period + return m0 +} + // A session window function defines windows that contain elements that are // within a certain gap size of another element. Session windowing applies // on a per-key basis and is useful for data that is irregularly distributed @@ -413,21 +499,18 @@ func (x *SlidingWindowsPayload) GetPeriod() *durationpb.Duration { // See https://beam.apache.org/documentation/programming-guide/#session-windows // for additional details. type SessionWindowsPayload struct { - state protoimpl.MessageState - sizeCache protoimpl.SizeCache - unknownFields protoimpl.UnknownFields - + state protoimpl.MessageState `protogen:"hybrid.v1"` // (Required) Minimum duration of gaps between sessions. - GapSize *durationpb.Duration `protobuf:"bytes,1,opt,name=gap_size,json=gapSize,proto3" json:"gap_size,omitempty"` + GapSize *durationpb.Duration `protobuf:"bytes,1,opt,name=gap_size,json=gapSize,proto3" json:"gap_size,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache } func (x *SessionWindowsPayload) Reset() { *x = SessionWindowsPayload{} - if protoimpl.UnsafeEnabled { - mi := &file_org_apache_beam_model_pipeline_v1_standard_window_fns_proto_msgTypes[3] - ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) - ms.StoreMessageInfo(mi) - } + mi := &file_org_apache_beam_model_pipeline_v1_standard_window_fns_proto_msgTypes[3] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) } func (x *SessionWindowsPayload) String() string { @@ -438,7 +521,7 @@ func (*SessionWindowsPayload) ProtoMessage() {} func (x *SessionWindowsPayload) ProtoReflect() protoreflect.Message { mi := &file_org_apache_beam_model_pipeline_v1_standard_window_fns_proto_msgTypes[3] - if protoimpl.UnsafeEnabled && x != nil { + if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { ms.StoreMessageInfo(mi) @@ -448,11 +531,6 @@ func (x *SessionWindowsPayload) ProtoReflect() protoreflect.Message { return mi.MessageOf(x) } -// Deprecated: Use SessionWindowsPayload.ProtoReflect.Descriptor instead. -func (*SessionWindowsPayload) Descriptor() ([]byte, []int) { - return file_org_apache_beam_model_pipeline_v1_standard_window_fns_proto_rawDescGZIP(), []int{3} -} - func (x *SessionWindowsPayload) GetGapSize() *durationpb.Duration { if x != nil { return x.GapSize @@ -460,6 +538,36 @@ func (x *SessionWindowsPayload) GetGapSize() *durationpb.Duration { return nil } +func (x *SessionWindowsPayload) SetGapSize(v *durationpb.Duration) { + x.GapSize = v +} + +func (x *SessionWindowsPayload) HasGapSize() bool { + if x == nil { + return false + } + return x.GapSize != nil +} + +func (x *SessionWindowsPayload) ClearGapSize() { + x.GapSize = nil +} + +type SessionWindowsPayload_builder struct { + _ [0]func() // Prevents comparability and use of unkeyed literals for the builder. + + // (Required) Minimum duration of gaps between sessions. + GapSize *durationpb.Duration +} + +func (b0 SessionWindowsPayload_builder) Build() *SessionWindowsPayload { + m0 := &SessionWindowsPayload{} + b, x := &b0, m0 + _, _ = b, x + x.GapSize = b.GapSize + return m0 +} + var File_org_apache_beam_model_pipeline_v1_standard_window_fns_proto protoreflect.FileDescriptor var file_org_apache_beam_model_pipeline_v1_standard_window_fns_proto_rawDesc = []byte{ @@ -529,21 +637,9 @@ var file_org_apache_beam_model_pipeline_v1_standard_window_fns_proto_rawDesc = [ 0x31, 0x62, 0x06, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x33, } -var ( - file_org_apache_beam_model_pipeline_v1_standard_window_fns_proto_rawDescOnce sync.Once - file_org_apache_beam_model_pipeline_v1_standard_window_fns_proto_rawDescData = file_org_apache_beam_model_pipeline_v1_standard_window_fns_proto_rawDesc -) - -func file_org_apache_beam_model_pipeline_v1_standard_window_fns_proto_rawDescGZIP() []byte { - file_org_apache_beam_model_pipeline_v1_standard_window_fns_proto_rawDescOnce.Do(func() { - file_org_apache_beam_model_pipeline_v1_standard_window_fns_proto_rawDescData = protoimpl.X.CompressGZIP(file_org_apache_beam_model_pipeline_v1_standard_window_fns_proto_rawDescData) - }) - return file_org_apache_beam_model_pipeline_v1_standard_window_fns_proto_rawDescData -} - var file_org_apache_beam_model_pipeline_v1_standard_window_fns_proto_enumTypes = make([]protoimpl.EnumInfo, 4) var file_org_apache_beam_model_pipeline_v1_standard_window_fns_proto_msgTypes = make([]protoimpl.MessageInfo, 4) -var file_org_apache_beam_model_pipeline_v1_standard_window_fns_proto_goTypes = []interface{}{ +var file_org_apache_beam_model_pipeline_v1_standard_window_fns_proto_goTypes = []any{ (GlobalWindowsPayload_Enum)(0), // 0: org.apache.beam.model.pipeline.v1.GlobalWindowsPayload.Enum (FixedWindowsPayload_Enum)(0), // 1: org.apache.beam.model.pipeline.v1.FixedWindowsPayload.Enum (SlidingWindowsPayload_Enum)(0), // 2: org.apache.beam.model.pipeline.v1.SlidingWindowsPayload.Enum @@ -575,56 +671,6 @@ func file_org_apache_beam_model_pipeline_v1_standard_window_fns_proto_init() { return } file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_init() - if !protoimpl.UnsafeEnabled { - file_org_apache_beam_model_pipeline_v1_standard_window_fns_proto_msgTypes[0].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*GlobalWindowsPayload); i { - case 0: - return &v.state - case 1: - return &v.sizeCache - case 2: - return &v.unknownFields - default: - return nil - } - } - file_org_apache_beam_model_pipeline_v1_standard_window_fns_proto_msgTypes[1].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*FixedWindowsPayload); i { - case 0: - return &v.state - case 1: - return &v.sizeCache - case 2: - return &v.unknownFields - default: - return nil - } - } - file_org_apache_beam_model_pipeline_v1_standard_window_fns_proto_msgTypes[2].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*SlidingWindowsPayload); i { - case 0: - return &v.state - case 1: - return &v.sizeCache - case 2: - return &v.unknownFields - default: - return nil - } - } - file_org_apache_beam_model_pipeline_v1_standard_window_fns_proto_msgTypes[3].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*SessionWindowsPayload); i { - case 0: - return &v.state - case 1: - return &v.sizeCache - case 2: - return &v.unknownFields - default: - return nil - } - } - } type x struct{} out := protoimpl.TypeBuilder{ File: protoimpl.DescBuilder{ diff --git a/sdks/go/pkg/beam/model/pipeline_v1/standard_window_fns_protoopaque.pb.go b/sdks/go/pkg/beam/model/pipeline_v1/standard_window_fns_protoopaque.pb.go new file mode 100644 index 000000000000..3c7c50dde7c0 --- /dev/null +++ b/sdks/go/pkg/beam/model/pipeline_v1/standard_window_fns_protoopaque.pb.go @@ -0,0 +1,685 @@ +// +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +// +// Protocol Buffers describing the Runner API, which is the runner-independent, +// SDK-independent definition of the Beam model. + +// Code generated by protoc-gen-go. DO NOT EDIT. +// versions: +// protoc-gen-go v1.36.0 +// protoc v5.27.3 +// source: org/apache/beam/model/pipeline/v1/standard_window_fns.proto + +//go:build protoopaque + +package pipeline_v1 + +import ( + protoreflect "google.golang.org/protobuf/reflect/protoreflect" + protoimpl "google.golang.org/protobuf/runtime/protoimpl" + durationpb "google.golang.org/protobuf/types/known/durationpb" + timestamppb "google.golang.org/protobuf/types/known/timestamppb" + reflect "reflect" +) + +const ( + // Verify that this generated code is sufficiently up-to-date. + _ = protoimpl.EnforceVersion(20 - protoimpl.MinVersion) + // Verify that runtime/protoimpl is sufficiently up-to-date. + _ = protoimpl.EnforceVersion(protoimpl.MaxVersion - 20) +) + +type GlobalWindowsPayload_Enum int32 + +const ( + GlobalWindowsPayload_PROPERTIES GlobalWindowsPayload_Enum = 0 +) + +// Enum value maps for GlobalWindowsPayload_Enum. +var ( + GlobalWindowsPayload_Enum_name = map[int32]string{ + 0: "PROPERTIES", + } + GlobalWindowsPayload_Enum_value = map[string]int32{ + "PROPERTIES": 0, + } +) + +func (x GlobalWindowsPayload_Enum) Enum() *GlobalWindowsPayload_Enum { + p := new(GlobalWindowsPayload_Enum) + *p = x + return p +} + +func (x GlobalWindowsPayload_Enum) String() string { + return protoimpl.X.EnumStringOf(x.Descriptor(), protoreflect.EnumNumber(x)) +} + +func (GlobalWindowsPayload_Enum) Descriptor() protoreflect.EnumDescriptor { + return file_org_apache_beam_model_pipeline_v1_standard_window_fns_proto_enumTypes[0].Descriptor() +} + +func (GlobalWindowsPayload_Enum) Type() protoreflect.EnumType { + return &file_org_apache_beam_model_pipeline_v1_standard_window_fns_proto_enumTypes[0] +} + +func (x GlobalWindowsPayload_Enum) Number() protoreflect.EnumNumber { + return protoreflect.EnumNumber(x) +} + +type FixedWindowsPayload_Enum int32 + +const ( + FixedWindowsPayload_PROPERTIES FixedWindowsPayload_Enum = 0 +) + +// Enum value maps for FixedWindowsPayload_Enum. +var ( + FixedWindowsPayload_Enum_name = map[int32]string{ + 0: "PROPERTIES", + } + FixedWindowsPayload_Enum_value = map[string]int32{ + "PROPERTIES": 0, + } +) + +func (x FixedWindowsPayload_Enum) Enum() *FixedWindowsPayload_Enum { + p := new(FixedWindowsPayload_Enum) + *p = x + return p +} + +func (x FixedWindowsPayload_Enum) String() string { + return protoimpl.X.EnumStringOf(x.Descriptor(), protoreflect.EnumNumber(x)) +} + +func (FixedWindowsPayload_Enum) Descriptor() protoreflect.EnumDescriptor { + return file_org_apache_beam_model_pipeline_v1_standard_window_fns_proto_enumTypes[1].Descriptor() +} + +func (FixedWindowsPayload_Enum) Type() protoreflect.EnumType { + return &file_org_apache_beam_model_pipeline_v1_standard_window_fns_proto_enumTypes[1] +} + +func (x FixedWindowsPayload_Enum) Number() protoreflect.EnumNumber { + return protoreflect.EnumNumber(x) +} + +type SlidingWindowsPayload_Enum int32 + +const ( + SlidingWindowsPayload_PROPERTIES SlidingWindowsPayload_Enum = 0 +) + +// Enum value maps for SlidingWindowsPayload_Enum. +var ( + SlidingWindowsPayload_Enum_name = map[int32]string{ + 0: "PROPERTIES", + } + SlidingWindowsPayload_Enum_value = map[string]int32{ + "PROPERTIES": 0, + } +) + +func (x SlidingWindowsPayload_Enum) Enum() *SlidingWindowsPayload_Enum { + p := new(SlidingWindowsPayload_Enum) + *p = x + return p +} + +func (x SlidingWindowsPayload_Enum) String() string { + return protoimpl.X.EnumStringOf(x.Descriptor(), protoreflect.EnumNumber(x)) +} + +func (SlidingWindowsPayload_Enum) Descriptor() protoreflect.EnumDescriptor { + return file_org_apache_beam_model_pipeline_v1_standard_window_fns_proto_enumTypes[2].Descriptor() +} + +func (SlidingWindowsPayload_Enum) Type() protoreflect.EnumType { + return &file_org_apache_beam_model_pipeline_v1_standard_window_fns_proto_enumTypes[2] +} + +func (x SlidingWindowsPayload_Enum) Number() protoreflect.EnumNumber { + return protoreflect.EnumNumber(x) +} + +type SessionWindowsPayload_Enum int32 + +const ( + SessionWindowsPayload_PROPERTIES SessionWindowsPayload_Enum = 0 +) + +// Enum value maps for SessionWindowsPayload_Enum. +var ( + SessionWindowsPayload_Enum_name = map[int32]string{ + 0: "PROPERTIES", + } + SessionWindowsPayload_Enum_value = map[string]int32{ + "PROPERTIES": 0, + } +) + +func (x SessionWindowsPayload_Enum) Enum() *SessionWindowsPayload_Enum { + p := new(SessionWindowsPayload_Enum) + *p = x + return p +} + +func (x SessionWindowsPayload_Enum) String() string { + return protoimpl.X.EnumStringOf(x.Descriptor(), protoreflect.EnumNumber(x)) +} + +func (SessionWindowsPayload_Enum) Descriptor() protoreflect.EnumDescriptor { + return file_org_apache_beam_model_pipeline_v1_standard_window_fns_proto_enumTypes[3].Descriptor() +} + +func (SessionWindowsPayload_Enum) Type() protoreflect.EnumType { + return &file_org_apache_beam_model_pipeline_v1_standard_window_fns_proto_enumTypes[3] +} + +func (x SessionWindowsPayload_Enum) Number() protoreflect.EnumNumber { + return protoreflect.EnumNumber(x) +} + +// By default, all data in a PCollection is assigned to the single global +// window. See BeamConstants for the time span this window encompasses. +// +// See https://beam.apache.org/documentation/programming-guide/#single-global-window +// for additional details. +type GlobalWindowsPayload struct { + state protoimpl.MessageState `protogen:"opaque.v1"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *GlobalWindowsPayload) Reset() { + *x = GlobalWindowsPayload{} + mi := &file_org_apache_beam_model_pipeline_v1_standard_window_fns_proto_msgTypes[0] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *GlobalWindowsPayload) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*GlobalWindowsPayload) ProtoMessage() {} + +func (x *GlobalWindowsPayload) ProtoReflect() protoreflect.Message { + mi := &file_org_apache_beam_model_pipeline_v1_standard_window_fns_proto_msgTypes[0] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +type GlobalWindowsPayload_builder struct { + _ [0]func() // Prevents comparability and use of unkeyed literals for the builder. + +} + +func (b0 GlobalWindowsPayload_builder) Build() *GlobalWindowsPayload { + m0 := &GlobalWindowsPayload{} + b, x := &b0, m0 + _, _ = b, x + return m0 +} + +// A fixed time window represents a consistent duration size, non overlapping +// time interval in the data stream. +// +// See https://beam.apache.org/documentation/programming-guide/#fixed-time-windows +// for additional details. +type FixedWindowsPayload struct { + state protoimpl.MessageState `protogen:"opaque.v1"` + xxx_hidden_Size *durationpb.Duration `protobuf:"bytes,1,opt,name=size,proto3" json:"size,omitempty"` + xxx_hidden_Offset *timestamppb.Timestamp `protobuf:"bytes,2,opt,name=offset,proto3" json:"offset,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *FixedWindowsPayload) Reset() { + *x = FixedWindowsPayload{} + mi := &file_org_apache_beam_model_pipeline_v1_standard_window_fns_proto_msgTypes[1] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *FixedWindowsPayload) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*FixedWindowsPayload) ProtoMessage() {} + +func (x *FixedWindowsPayload) ProtoReflect() protoreflect.Message { + mi := &file_org_apache_beam_model_pipeline_v1_standard_window_fns_proto_msgTypes[1] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +func (x *FixedWindowsPayload) GetSize() *durationpb.Duration { + if x != nil { + return x.xxx_hidden_Size + } + return nil +} + +func (x *FixedWindowsPayload) GetOffset() *timestamppb.Timestamp { + if x != nil { + return x.xxx_hidden_Offset + } + return nil +} + +func (x *FixedWindowsPayload) SetSize(v *durationpb.Duration) { + x.xxx_hidden_Size = v +} + +func (x *FixedWindowsPayload) SetOffset(v *timestamppb.Timestamp) { + x.xxx_hidden_Offset = v +} + +func (x *FixedWindowsPayload) HasSize() bool { + if x == nil { + return false + } + return x.xxx_hidden_Size != nil +} + +func (x *FixedWindowsPayload) HasOffset() bool { + if x == nil { + return false + } + return x.xxx_hidden_Offset != nil +} + +func (x *FixedWindowsPayload) ClearSize() { + x.xxx_hidden_Size = nil +} + +func (x *FixedWindowsPayload) ClearOffset() { + x.xxx_hidden_Offset = nil +} + +type FixedWindowsPayload_builder struct { + _ [0]func() // Prevents comparability and use of unkeyed literals for the builder. + + // (Required) Represents the size of the window. + Size *durationpb.Duration + // (Required) Represents the timestamp of when the first window begins. + // Window N will start at offset + N * size. + Offset *timestamppb.Timestamp +} + +func (b0 FixedWindowsPayload_builder) Build() *FixedWindowsPayload { + m0 := &FixedWindowsPayload{} + b, x := &b0, m0 + _, _ = b, x + x.xxx_hidden_Size = b.Size + x.xxx_hidden_Offset = b.Offset + return m0 +} + +// A sliding time window represents time intervals in the data stream that can +// overlap. For example, each window might capture 60 seconds worth of data, but +// a new window starts every 30 seconds. The frequency with which sliding +// windows begin is called the period. Therefore, our example would have a +// window size of 60 seconds and a period of 30 seconds. +// +// Because multiple windows overlap, most elements in a data set will belong to +// more than one window. This kind of windowing is useful for taking running +// averages of data; using sliding time windows, you can compute a running +// average of the past 60 seconds’ worth of data, updated every 30 seconds, in +// our example. +// +// See https://beam.apache.org/documentation/programming-guide/#sliding-time-windows +// for additional details. +type SlidingWindowsPayload struct { + state protoimpl.MessageState `protogen:"opaque.v1"` + xxx_hidden_Size *durationpb.Duration `protobuf:"bytes,1,opt,name=size,proto3" json:"size,omitempty"` + xxx_hidden_Offset *timestamppb.Timestamp `protobuf:"bytes,2,opt,name=offset,proto3" json:"offset,omitempty"` + xxx_hidden_Period *durationpb.Duration `protobuf:"bytes,3,opt,name=period,proto3" json:"period,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *SlidingWindowsPayload) Reset() { + *x = SlidingWindowsPayload{} + mi := &file_org_apache_beam_model_pipeline_v1_standard_window_fns_proto_msgTypes[2] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *SlidingWindowsPayload) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*SlidingWindowsPayload) ProtoMessage() {} + +func (x *SlidingWindowsPayload) ProtoReflect() protoreflect.Message { + mi := &file_org_apache_beam_model_pipeline_v1_standard_window_fns_proto_msgTypes[2] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +func (x *SlidingWindowsPayload) GetSize() *durationpb.Duration { + if x != nil { + return x.xxx_hidden_Size + } + return nil +} + +func (x *SlidingWindowsPayload) GetOffset() *timestamppb.Timestamp { + if x != nil { + return x.xxx_hidden_Offset + } + return nil +} + +func (x *SlidingWindowsPayload) GetPeriod() *durationpb.Duration { + if x != nil { + return x.xxx_hidden_Period + } + return nil +} + +func (x *SlidingWindowsPayload) SetSize(v *durationpb.Duration) { + x.xxx_hidden_Size = v +} + +func (x *SlidingWindowsPayload) SetOffset(v *timestamppb.Timestamp) { + x.xxx_hidden_Offset = v +} + +func (x *SlidingWindowsPayload) SetPeriod(v *durationpb.Duration) { + x.xxx_hidden_Period = v +} + +func (x *SlidingWindowsPayload) HasSize() bool { + if x == nil { + return false + } + return x.xxx_hidden_Size != nil +} + +func (x *SlidingWindowsPayload) HasOffset() bool { + if x == nil { + return false + } + return x.xxx_hidden_Offset != nil +} + +func (x *SlidingWindowsPayload) HasPeriod() bool { + if x == nil { + return false + } + return x.xxx_hidden_Period != nil +} + +func (x *SlidingWindowsPayload) ClearSize() { + x.xxx_hidden_Size = nil +} + +func (x *SlidingWindowsPayload) ClearOffset() { + x.xxx_hidden_Offset = nil +} + +func (x *SlidingWindowsPayload) ClearPeriod() { + x.xxx_hidden_Period = nil +} + +type SlidingWindowsPayload_builder struct { + _ [0]func() // Prevents comparability and use of unkeyed literals for the builder. + + // (Required) Represents the size of the window. + Size *durationpb.Duration + // (Required) Represents the timestamp of when the first window begins. + // Window N will start at offset + N * period. + Offset *timestamppb.Timestamp + // (Required) Represents the amount of time between each start of a window. + Period *durationpb.Duration +} + +func (b0 SlidingWindowsPayload_builder) Build() *SlidingWindowsPayload { + m0 := &SlidingWindowsPayload{} + b, x := &b0, m0 + _, _ = b, x + x.xxx_hidden_Size = b.Size + x.xxx_hidden_Offset = b.Offset + x.xxx_hidden_Period = b.Period + return m0 +} + +// A session window function defines windows that contain elements that are +// within a certain gap size of another element. Session windowing applies +// on a per-key basis and is useful for data that is irregularly distributed +// with respect to time. For example, a data stream representing user mouse +// activity may have long periods of idle time interspersed with high +// concentrations of clicks. If data arrives after the minimum specified gap +// size duration, this initiates the start of a new window. +// +// See https://beam.apache.org/documentation/programming-guide/#session-windows +// for additional details. +type SessionWindowsPayload struct { + state protoimpl.MessageState `protogen:"opaque.v1"` + xxx_hidden_GapSize *durationpb.Duration `protobuf:"bytes,1,opt,name=gap_size,json=gapSize,proto3" json:"gap_size,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *SessionWindowsPayload) Reset() { + *x = SessionWindowsPayload{} + mi := &file_org_apache_beam_model_pipeline_v1_standard_window_fns_proto_msgTypes[3] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *SessionWindowsPayload) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*SessionWindowsPayload) ProtoMessage() {} + +func (x *SessionWindowsPayload) ProtoReflect() protoreflect.Message { + mi := &file_org_apache_beam_model_pipeline_v1_standard_window_fns_proto_msgTypes[3] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +func (x *SessionWindowsPayload) GetGapSize() *durationpb.Duration { + if x != nil { + return x.xxx_hidden_GapSize + } + return nil +} + +func (x *SessionWindowsPayload) SetGapSize(v *durationpb.Duration) { + x.xxx_hidden_GapSize = v +} + +func (x *SessionWindowsPayload) HasGapSize() bool { + if x == nil { + return false + } + return x.xxx_hidden_GapSize != nil +} + +func (x *SessionWindowsPayload) ClearGapSize() { + x.xxx_hidden_GapSize = nil +} + +type SessionWindowsPayload_builder struct { + _ [0]func() // Prevents comparability and use of unkeyed literals for the builder. + + // (Required) Minimum duration of gaps between sessions. + GapSize *durationpb.Duration +} + +func (b0 SessionWindowsPayload_builder) Build() *SessionWindowsPayload { + m0 := &SessionWindowsPayload{} + b, x := &b0, m0 + _, _ = b, x + x.xxx_hidden_GapSize = b.GapSize + return m0 +} + +var File_org_apache_beam_model_pipeline_v1_standard_window_fns_proto protoreflect.FileDescriptor + +var file_org_apache_beam_model_pipeline_v1_standard_window_fns_proto_rawDesc = []byte{ + 0x0a, 0x3b, 0x6f, 0x72, 0x67, 0x2f, 0x61, 0x70, 0x61, 0x63, 0x68, 0x65, 0x2f, 0x62, 0x65, 0x61, + 0x6d, 0x2f, 0x6d, 0x6f, 0x64, 0x65, 0x6c, 0x2f, 0x70, 0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e, 0x65, + 0x2f, 0x76, 0x31, 0x2f, 0x73, 0x74, 0x61, 0x6e, 0x64, 0x61, 0x72, 0x64, 0x5f, 0x77, 0x69, 0x6e, + 0x64, 0x6f, 0x77, 0x5f, 0x66, 0x6e, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x12, 0x21, 0x6f, + 0x72, 0x67, 0x2e, 0x61, 0x70, 0x61, 0x63, 0x68, 0x65, 0x2e, 0x62, 0x65, 0x61, 0x6d, 0x2e, 0x6d, + 0x6f, 0x64, 0x65, 0x6c, 0x2e, 0x70, 0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e, 0x65, 0x2e, 0x76, 0x31, + 0x1a, 0x37, 0x6f, 0x72, 0x67, 0x2f, 0x61, 0x70, 0x61, 0x63, 0x68, 0x65, 0x2f, 0x62, 0x65, 0x61, + 0x6d, 0x2f, 0x6d, 0x6f, 0x64, 0x65, 0x6c, 0x2f, 0x70, 0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e, 0x65, + 0x2f, 0x76, 0x31, 0x2f, 0x62, 0x65, 0x61, 0x6d, 0x5f, 0x72, 0x75, 0x6e, 0x6e, 0x65, 0x72, 0x5f, + 0x61, 0x70, 0x69, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x1a, 0x1e, 0x67, 0x6f, 0x6f, 0x67, 0x6c, + 0x65, 0x2f, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66, 0x2f, 0x64, 0x75, 0x72, 0x61, 0x74, + 0x69, 0x6f, 0x6e, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x1a, 0x1f, 0x67, 0x6f, 0x6f, 0x67, 0x6c, + 0x65, 0x2f, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66, 0x2f, 0x74, 0x69, 0x6d, 0x65, 0x73, + 0x74, 0x61, 0x6d, 0x70, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x22, 0x56, 0x0a, 0x14, 0x47, 0x6c, + 0x6f, 0x62, 0x61, 0x6c, 0x57, 0x69, 0x6e, 0x64, 0x6f, 0x77, 0x73, 0x50, 0x61, 0x79, 0x6c, 0x6f, + 0x61, 0x64, 0x22, 0x3e, 0x0a, 0x04, 0x45, 0x6e, 0x75, 0x6d, 0x12, 0x36, 0x0a, 0x0a, 0x50, 0x52, + 0x4f, 0x50, 0x45, 0x52, 0x54, 0x49, 0x45, 0x53, 0x10, 0x00, 0x1a, 0x26, 0xa2, 0xb4, 0xfa, 0xc2, + 0x05, 0x20, 0x62, 0x65, 0x61, 0x6d, 0x3a, 0x77, 0x69, 0x6e, 0x64, 0x6f, 0x77, 0x5f, 0x66, 0x6e, + 0x3a, 0x67, 0x6c, 0x6f, 0x62, 0x61, 0x6c, 0x5f, 0x77, 0x69, 0x6e, 0x64, 0x6f, 0x77, 0x73, 0x3a, + 0x76, 0x31, 0x22, 0xb7, 0x01, 0x0a, 0x13, 0x46, 0x69, 0x78, 0x65, 0x64, 0x57, 0x69, 0x6e, 0x64, + 0x6f, 0x77, 0x73, 0x50, 0x61, 0x79, 0x6c, 0x6f, 0x61, 0x64, 0x12, 0x2d, 0x0a, 0x04, 0x73, 0x69, + 0x7a, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x19, 0x2e, 0x67, 0x6f, 0x6f, 0x67, 0x6c, + 0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66, 0x2e, 0x44, 0x75, 0x72, 0x61, 0x74, + 0x69, 0x6f, 0x6e, 0x52, 0x04, 0x73, 0x69, 0x7a, 0x65, 0x12, 0x32, 0x0a, 0x06, 0x6f, 0x66, 0x66, + 0x73, 0x65, 0x74, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1a, 0x2e, 0x67, 0x6f, 0x6f, 0x67, + 0x6c, 0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66, 0x2e, 0x54, 0x69, 0x6d, 0x65, + 0x73, 0x74, 0x61, 0x6d, 0x70, 0x52, 0x06, 0x6f, 0x66, 0x66, 0x73, 0x65, 0x74, 0x22, 0x3d, 0x0a, + 0x04, 0x45, 0x6e, 0x75, 0x6d, 0x12, 0x35, 0x0a, 0x0a, 0x50, 0x52, 0x4f, 0x50, 0x45, 0x52, 0x54, + 0x49, 0x45, 0x53, 0x10, 0x00, 0x1a, 0x25, 0xa2, 0xb4, 0xfa, 0xc2, 0x05, 0x1f, 0x62, 0x65, 0x61, + 0x6d, 0x3a, 0x77, 0x69, 0x6e, 0x64, 0x6f, 0x77, 0x5f, 0x66, 0x6e, 0x3a, 0x66, 0x69, 0x78, 0x65, + 0x64, 0x5f, 0x77, 0x69, 0x6e, 0x64, 0x6f, 0x77, 0x73, 0x3a, 0x76, 0x31, 0x22, 0xee, 0x01, 0x0a, + 0x15, 0x53, 0x6c, 0x69, 0x64, 0x69, 0x6e, 0x67, 0x57, 0x69, 0x6e, 0x64, 0x6f, 0x77, 0x73, 0x50, + 0x61, 0x79, 0x6c, 0x6f, 0x61, 0x64, 0x12, 0x2d, 0x0a, 0x04, 0x73, 0x69, 0x7a, 0x65, 0x18, 0x01, + 0x20, 0x01, 0x28, 0x0b, 0x32, 0x19, 0x2e, 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2e, 0x70, 0x72, + 0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66, 0x2e, 0x44, 0x75, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x52, + 0x04, 0x73, 0x69, 0x7a, 0x65, 0x12, 0x32, 0x0a, 0x06, 0x6f, 0x66, 0x66, 0x73, 0x65, 0x74, 0x18, + 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1a, 0x2e, 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2e, 0x70, + 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66, 0x2e, 0x54, 0x69, 0x6d, 0x65, 0x73, 0x74, 0x61, 0x6d, + 0x70, 0x52, 0x06, 0x6f, 0x66, 0x66, 0x73, 0x65, 0x74, 0x12, 0x31, 0x0a, 0x06, 0x70, 0x65, 0x72, + 0x69, 0x6f, 0x64, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x19, 0x2e, 0x67, 0x6f, 0x6f, 0x67, + 0x6c, 0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66, 0x2e, 0x44, 0x75, 0x72, 0x61, + 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x06, 0x70, 0x65, 0x72, 0x69, 0x6f, 0x64, 0x22, 0x3f, 0x0a, 0x04, + 0x45, 0x6e, 0x75, 0x6d, 0x12, 0x37, 0x0a, 0x0a, 0x50, 0x52, 0x4f, 0x50, 0x45, 0x52, 0x54, 0x49, + 0x45, 0x53, 0x10, 0x00, 0x1a, 0x27, 0xa2, 0xb4, 0xfa, 0xc2, 0x05, 0x21, 0x62, 0x65, 0x61, 0x6d, + 0x3a, 0x77, 0x69, 0x6e, 0x64, 0x6f, 0x77, 0x5f, 0x66, 0x6e, 0x3a, 0x73, 0x6c, 0x69, 0x64, 0x69, + 0x6e, 0x67, 0x5f, 0x77, 0x69, 0x6e, 0x64, 0x6f, 0x77, 0x73, 0x3a, 0x76, 0x31, 0x22, 0x8e, 0x01, + 0x0a, 0x15, 0x53, 0x65, 0x73, 0x73, 0x69, 0x6f, 0x6e, 0x57, 0x69, 0x6e, 0x64, 0x6f, 0x77, 0x73, + 0x50, 0x61, 0x79, 0x6c, 0x6f, 0x61, 0x64, 0x12, 0x34, 0x0a, 0x08, 0x67, 0x61, 0x70, 0x5f, 0x73, + 0x69, 0x7a, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x19, 0x2e, 0x67, 0x6f, 0x6f, 0x67, + 0x6c, 0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66, 0x2e, 0x44, 0x75, 0x72, 0x61, + 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x07, 0x67, 0x61, 0x70, 0x53, 0x69, 0x7a, 0x65, 0x22, 0x3f, 0x0a, + 0x04, 0x45, 0x6e, 0x75, 0x6d, 0x12, 0x37, 0x0a, 0x0a, 0x50, 0x52, 0x4f, 0x50, 0x45, 0x52, 0x54, + 0x49, 0x45, 0x53, 0x10, 0x00, 0x1a, 0x27, 0xa2, 0xb4, 0xfa, 0xc2, 0x05, 0x21, 0x62, 0x65, 0x61, + 0x6d, 0x3a, 0x77, 0x69, 0x6e, 0x64, 0x6f, 0x77, 0x5f, 0x66, 0x6e, 0x3a, 0x73, 0x65, 0x73, 0x73, + 0x69, 0x6f, 0x6e, 0x5f, 0x77, 0x69, 0x6e, 0x64, 0x6f, 0x77, 0x73, 0x3a, 0x76, 0x31, 0x42, 0x80, + 0x01, 0x0a, 0x21, 0x6f, 0x72, 0x67, 0x2e, 0x61, 0x70, 0x61, 0x63, 0x68, 0x65, 0x2e, 0x62, 0x65, + 0x61, 0x6d, 0x2e, 0x6d, 0x6f, 0x64, 0x65, 0x6c, 0x2e, 0x70, 0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e, + 0x65, 0x2e, 0x76, 0x31, 0x42, 0x11, 0x53, 0x74, 0x61, 0x6e, 0x64, 0x61, 0x72, 0x64, 0x57, 0x69, + 0x6e, 0x64, 0x6f, 0x77, 0x46, 0x6e, 0x73, 0x5a, 0x48, 0x67, 0x69, 0x74, 0x68, 0x75, 0x62, 0x2e, + 0x63, 0x6f, 0x6d, 0x2f, 0x61, 0x70, 0x61, 0x63, 0x68, 0x65, 0x2f, 0x62, 0x65, 0x61, 0x6d, 0x2f, + 0x73, 0x64, 0x6b, 0x73, 0x2f, 0x76, 0x32, 0x2f, 0x67, 0x6f, 0x2f, 0x70, 0x6b, 0x67, 0x2f, 0x62, + 0x65, 0x61, 0x6d, 0x2f, 0x6d, 0x6f, 0x64, 0x65, 0x6c, 0x2f, 0x70, 0x69, 0x70, 0x65, 0x6c, 0x69, + 0x6e, 0x65, 0x5f, 0x76, 0x31, 0x3b, 0x70, 0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e, 0x65, 0x5f, 0x76, + 0x31, 0x62, 0x06, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x33, +} + +var file_org_apache_beam_model_pipeline_v1_standard_window_fns_proto_enumTypes = make([]protoimpl.EnumInfo, 4) +var file_org_apache_beam_model_pipeline_v1_standard_window_fns_proto_msgTypes = make([]protoimpl.MessageInfo, 4) +var file_org_apache_beam_model_pipeline_v1_standard_window_fns_proto_goTypes = []any{ + (GlobalWindowsPayload_Enum)(0), // 0: org.apache.beam.model.pipeline.v1.GlobalWindowsPayload.Enum + (FixedWindowsPayload_Enum)(0), // 1: org.apache.beam.model.pipeline.v1.FixedWindowsPayload.Enum + (SlidingWindowsPayload_Enum)(0), // 2: org.apache.beam.model.pipeline.v1.SlidingWindowsPayload.Enum + (SessionWindowsPayload_Enum)(0), // 3: org.apache.beam.model.pipeline.v1.SessionWindowsPayload.Enum + (*GlobalWindowsPayload)(nil), // 4: org.apache.beam.model.pipeline.v1.GlobalWindowsPayload + (*FixedWindowsPayload)(nil), // 5: org.apache.beam.model.pipeline.v1.FixedWindowsPayload + (*SlidingWindowsPayload)(nil), // 6: org.apache.beam.model.pipeline.v1.SlidingWindowsPayload + (*SessionWindowsPayload)(nil), // 7: org.apache.beam.model.pipeline.v1.SessionWindowsPayload + (*durationpb.Duration)(nil), // 8: google.protobuf.Duration + (*timestamppb.Timestamp)(nil), // 9: google.protobuf.Timestamp +} +var file_org_apache_beam_model_pipeline_v1_standard_window_fns_proto_depIdxs = []int32{ + 8, // 0: org.apache.beam.model.pipeline.v1.FixedWindowsPayload.size:type_name -> google.protobuf.Duration + 9, // 1: org.apache.beam.model.pipeline.v1.FixedWindowsPayload.offset:type_name -> google.protobuf.Timestamp + 8, // 2: org.apache.beam.model.pipeline.v1.SlidingWindowsPayload.size:type_name -> google.protobuf.Duration + 9, // 3: org.apache.beam.model.pipeline.v1.SlidingWindowsPayload.offset:type_name -> google.protobuf.Timestamp + 8, // 4: org.apache.beam.model.pipeline.v1.SlidingWindowsPayload.period:type_name -> google.protobuf.Duration + 8, // 5: org.apache.beam.model.pipeline.v1.SessionWindowsPayload.gap_size:type_name -> google.protobuf.Duration + 6, // [6:6] is the sub-list for method output_type + 6, // [6:6] is the sub-list for method input_type + 6, // [6:6] is the sub-list for extension type_name + 6, // [6:6] is the sub-list for extension extendee + 0, // [0:6] is the sub-list for field type_name +} + +func init() { file_org_apache_beam_model_pipeline_v1_standard_window_fns_proto_init() } +func file_org_apache_beam_model_pipeline_v1_standard_window_fns_proto_init() { + if File_org_apache_beam_model_pipeline_v1_standard_window_fns_proto != nil { + return + } + file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_init() + type x struct{} + out := protoimpl.TypeBuilder{ + File: protoimpl.DescBuilder{ + GoPackagePath: reflect.TypeOf(x{}).PkgPath(), + RawDescriptor: file_org_apache_beam_model_pipeline_v1_standard_window_fns_proto_rawDesc, + NumEnums: 4, + NumMessages: 4, + NumExtensions: 0, + NumServices: 0, + }, + GoTypes: file_org_apache_beam_model_pipeline_v1_standard_window_fns_proto_goTypes, + DependencyIndexes: file_org_apache_beam_model_pipeline_v1_standard_window_fns_proto_depIdxs, + EnumInfos: file_org_apache_beam_model_pipeline_v1_standard_window_fns_proto_enumTypes, + MessageInfos: file_org_apache_beam_model_pipeline_v1_standard_window_fns_proto_msgTypes, + }.Build() + File_org_apache_beam_model_pipeline_v1_standard_window_fns_proto = out.File + file_org_apache_beam_model_pipeline_v1_standard_window_fns_proto_rawDesc = nil + file_org_apache_beam_model_pipeline_v1_standard_window_fns_proto_goTypes = nil + file_org_apache_beam_model_pipeline_v1_standard_window_fns_proto_depIdxs = nil +} diff --git a/sdks/go/pkg/beam/provision/provision_test.go b/sdks/go/pkg/beam/provision/provision_test.go index 162998a21e97..d0a4bb7e1216 100644 --- a/sdks/go/pkg/beam/provision/provision_test.go +++ b/sdks/go/pkg/beam/provision/provision_test.go @@ -66,7 +66,7 @@ type ProvisionServiceServicer struct { } func (p ProvisionServiceServicer) GetProvisionInfo(ctx context.Context, req *fnpb.GetProvisionInfoRequest) (*fnpb.GetProvisionInfoResponse, error) { - return &fnpb.GetProvisionInfoResponse{Info: &fnpb.ProvisionInfo{RetrievalToken: "token"}}, nil + return fnpb.GetProvisionInfoResponse_builder{Info: fnpb.ProvisionInfo_builder{RetrievalToken: "token"}.Build()}.Build(), nil } func setup(addr *string, wg *sync.WaitGroup) { @@ -103,7 +103,7 @@ func TestProvisionInfo(t *testing.T) { if err != nil { t.Errorf("error in response: %v", err) } - want := &fnpb.ProvisionInfo{RetrievalToken: "token"} + want := fnpb.ProvisionInfo_builder{RetrievalToken: "token"}.Build() if got.GetRetrievalToken() != want.GetRetrievalToken() { t.Errorf("provision.Info() = %v, want %v", got, want) } diff --git a/sdks/go/pkg/beam/runners/dataflow/dataflowlib/execute.go b/sdks/go/pkg/beam/runners/dataflow/dataflowlib/execute.go index 806b8940ae99..ce3388b385ac 100644 --- a/sdks/go/pkg/beam/runners/dataflow/dataflowlib/execute.go +++ b/sdks/go/pkg/beam/runners/dataflow/dataflowlib/execute.go @@ -73,10 +73,10 @@ func Execute(ctx context.Context, raw *pipepb.Pipeline, opts *JobOptions, worker if err := graphx.UpdateDefaultEnvWorkerType( graphx.URNArtifactURLType, - protox.MustEncode(&pipepb.ArtifactUrlPayload{ + protox.MustEncode(pipepb.ArtifactUrlPayload_builder{ Url: workerURL, Sha256: hash, - }), raw); err != nil { + }.Build()), raw); err != nil { return presult, err } diff --git a/sdks/go/pkg/beam/runners/dataflow/dataflowlib/job_test.go b/sdks/go/pkg/beam/runners/dataflow/dataflowlib/job_test.go index fb44ff9c0133..1177b543e00e 100644 --- a/sdks/go/pkg/beam/runners/dataflow/dataflowlib/job_test.go +++ b/sdks/go/pkg/beam/runners/dataflow/dataflowlib/job_test.go @@ -223,12 +223,12 @@ func Test_containerImages(t *testing.T) { display := []string{} for _, i := range imgs { - envs[i.id] = &pipepb.Environment{ + envs[i.id] = pipepb.Environment_builder{ Capabilities: i.caps, - Payload: protox.MustEncode(&pipepb.DockerPayload{ + Payload: protox.MustEncode(pipepb.DockerPayload_builder{ ContainerImage: i.image, - }), - } + }.Build()), + }.Build() images = append(images, &df.SdkHarnessContainerImage{ ContainerImage: i.image, UseSingleCorePerContainer: i.single, @@ -257,11 +257,11 @@ func Test_containerImages(t *testing.T) { for _, test := range tests { t.Run(test.name, func(t *testing.T) { - pipeline := &pipepb.Pipeline{ - Components: &pipepb.Components{ + pipeline := pipepb.Pipeline_builder{ + Components: pipepb.Components_builder{ Environments: test.envs, - }, - } + }.Build(), + }.Build() gotImages, gotDisplay, err := containerImages(pipeline) if err != nil { t.Fatalf("containerImages(...) error = %v, want nil", err) diff --git a/sdks/go/pkg/beam/runners/dataflow/dataflowlib/metrics_test.go b/sdks/go/pkg/beam/runners/dataflow/dataflowlib/metrics_test.go index 803665e010ca..0351938ae021 100644 --- a/sdks/go/pkg/beam/runners/dataflow/dataflowlib/metrics_test.go +++ b/sdks/go/pkg/beam/runners/dataflow/dataflowlib/metrics_test.go @@ -114,14 +114,14 @@ func newMetricStructuredName(name, namespace string, attempted bool) df.MetricSt } func newPipeline(stepName string) (*pipepb.Pipeline, error) { - p := &pipepb.Pipeline{ - Components: &pipepb.Components{ + p := pipepb.Pipeline_builder{ + Components: pipepb.Components_builder{ Transforms: map[string]*pipepb.PTransform{ - "e5": { + "e5": pipepb.PTransform_builder{ UniqueName: stepName, - }, + }.Build(), }, - }, - } + }.Build(), + }.Build() return p, nil } diff --git a/sdks/go/pkg/beam/runners/prism/internal/coders.go b/sdks/go/pkg/beam/runners/prism/internal/coders.go index ffea90e79065..23c1b4ad409d 100644 --- a/sdks/go/pkg/beam/runners/prism/internal/coders.go +++ b/sdks/go/pkg/beam/runners/prism/internal/coders.go @@ -70,12 +70,12 @@ func makeWindowedValueCoder(pID string, comps *pipepb.Components, coders map[str // Produce ID for the Windowed Value Coder wvcID := "cwv_" + pID - wInC := &pipepb.Coder{ - Spec: &pipepb.FunctionSpec{ + wInC := pipepb.Coder_builder{ + Spec: pipepb.FunctionSpec_builder{ Urn: urns.CoderWindowedValue, - }, + }.Build(), ComponentCoderIds: []string{cID, wcID}, - } + }.Build() // Populate the coders to send with the new windowed value coder. coders[wvcID] = wInC return wvcID, nil @@ -127,12 +127,12 @@ func lpUnknownCoders(cID string, bundle, base map[string]*pipepb.Coder) (string, // we must LP it, and we return the LP'd version. leaf := isLeafCoder(c) if len(c.GetComponentCoderIds()) == 0 && !leaf { - lpc := &pipepb.Coder{ - Spec: &pipepb.FunctionSpec{ + lpc := pipepb.Coder_builder{ + Spec: pipepb.FunctionSpec_builder{ Urn: urns.CoderLengthPrefix, - }, + }.Build(), ComponentCoderIds: []string{cID}, - } + }.Build() bundle[lpcID] = lpc return lpcID, nil } @@ -158,10 +158,10 @@ func lpUnknownCoders(cID string, bundle, base map[string]*pipepb.Coder) (string, comps = append(comps, rcc) } if needNewComposite { - lpc := &pipepb.Coder{ + lpc := pipepb.Coder_builder{ Spec: c.GetSpec(), ComponentCoderIds: comps, - } + }.Build() bundle[lpcID] = lpc return lpcID, nil } @@ -283,9 +283,9 @@ func debugCoder(cid string, coders map[string]*pipepb.Coder) string { b.WriteString(cid) b.WriteRune('\n') c := coders[cid] - if len(c.ComponentCoderIds) > 0 { + if len(c.GetComponentCoderIds()) > 0 { b.WriteRune('\t') - b.WriteString(strings.Join(c.ComponentCoderIds, ", ")) + b.WriteString(strings.Join(c.GetComponentCoderIds(), ", ")) b.WriteRune('\n') for _, ccid := range c.GetComponentCoderIds() { b.WriteString(debugCoder(ccid, coders)) diff --git a/sdks/go/pkg/beam/runners/prism/internal/coders_test.go b/sdks/go/pkg/beam/runners/prism/internal/coders_test.go index 4656a94e03ec..d4dbe526d059 100644 --- a/sdks/go/pkg/beam/runners/prism/internal/coders_test.go +++ b/sdks/go/pkg/beam/runners/prism/internal/coders_test.go @@ -49,11 +49,11 @@ func Test_isLeafCoder(t *testing.T) { {urns.CoderKV, false}, } for _, test := range tests { - undertest := &pipepb.Coder{ - Spec: &pipepb.FunctionSpec{ + undertest := pipepb.Coder_builder{ + Spec: pipepb.FunctionSpec_builder{ Urn: test.urn, - }, - } + }.Build(), + }.Build() if got, want := isLeafCoder(undertest), test.isLeaf; got != want { t.Errorf("isLeafCoder(%v) = %v, want %v", test.urn, got, want) } @@ -63,18 +63,18 @@ func Test_isLeafCoder(t *testing.T) { func Test_makeWindowedValueCoder(t *testing.T) { coders := map[string]*pipepb.Coder{} - gotID, err := makeWindowedValueCoder("testPID", &pipepb.Components{ + gotID, err := makeWindowedValueCoder("testPID", pipepb.Components_builder{ Pcollections: map[string]*pipepb.PCollection{ - "testPID": {CoderId: "testCoderID"}, + "testPID": pipepb.PCollection_builder{CoderId: "testCoderID"}.Build(), }, Coders: map[string]*pipepb.Coder{ - "testCoderID": { - Spec: &pipepb.FunctionSpec{ + "testCoderID": pipepb.Coder_builder{ + Spec: pipepb.FunctionSpec_builder{ Urn: urns.CoderBool, - }, - }, + }.Build(), + }.Build(), }, - }, coders) + }.Build(), coders) if err != nil { t.Errorf("makeWindowedValueCoder(...) = error %v, want nil", err) } @@ -103,11 +103,11 @@ func Test_makeWindowCoders(t *testing.T) { }, engine.WinInterval}, } for _, test := range tests { - undertest := &pipepb.Coder{ - Spec: &pipepb.FunctionSpec{ + undertest := pipepb.Coder_builder{ + Spec: pipepb.FunctionSpec_builder{ Urn: test.urn, - }, - } + }.Build(), + }.Build() gotCoderType, dec, enc := makeWindowCoders(undertest) if got, want := gotCoderType, test.coderType; got != want { @@ -141,23 +141,23 @@ func Test_lpUnknownCoders(t *testing.T) { {"alreadyProcessed", urns.CoderBool, nil, map[string]*pipepb.Coder{ - "test": {Spec: &pipepb.FunctionSpec{Urn: urns.CoderBool}}, + "test": pipepb.Coder_builder{Spec: pipepb.FunctionSpec_builder{Urn: urns.CoderBool}.Build()}.Build(), }, map[string]*pipepb.Coder{}, map[string]*pipepb.Coder{ - "test": {Spec: &pipepb.FunctionSpec{Urn: urns.CoderBool}}, + "test": pipepb.Coder_builder{Spec: pipepb.FunctionSpec_builder{Urn: urns.CoderBool}.Build()}.Build(), }, }, {"alreadyProcessedLP", urns.CoderBool, nil, map[string]*pipepb.Coder{ - "test_lp": {Spec: &pipepb.FunctionSpec{Urn: urns.CoderLengthPrefix}, ComponentCoderIds: []string{"test"}}, - "test": {Spec: &pipepb.FunctionSpec{Urn: urns.CoderBool}}, + "test_lp": pipepb.Coder_builder{Spec: pipepb.FunctionSpec_builder{Urn: urns.CoderLengthPrefix}.Build(), ComponentCoderIds: []string{"test"}}.Build(), + "test": pipepb.Coder_builder{Spec: pipepb.FunctionSpec_builder{Urn: urns.CoderBool}.Build()}.Build(), }, map[string]*pipepb.Coder{}, map[string]*pipepb.Coder{ - "test_lp": {Spec: &pipepb.FunctionSpec{Urn: urns.CoderLengthPrefix}, ComponentCoderIds: []string{"test"}}, - "test": {Spec: &pipepb.FunctionSpec{Urn: urns.CoderBool}}, + "test_lp": pipepb.Coder_builder{Spec: pipepb.FunctionSpec_builder{Urn: urns.CoderLengthPrefix}.Build(), ComponentCoderIds: []string{"test"}}.Build(), + "test": pipepb.Coder_builder{Spec: pipepb.FunctionSpec_builder{Urn: urns.CoderBool}.Build()}.Build(), }, }, {"noNeedForLP", @@ -165,7 +165,7 @@ func Test_lpUnknownCoders(t *testing.T) { map[string]*pipepb.Coder{}, map[string]*pipepb.Coder{}, map[string]*pipepb.Coder{ - "test": {Spec: &pipepb.FunctionSpec{Urn: urns.CoderBool}}, + "test": pipepb.Coder_builder{Spec: pipepb.FunctionSpec_builder{Urn: urns.CoderBool}.Build()}.Build(), }, }, {"needLP", @@ -173,43 +173,43 @@ func Test_lpUnknownCoders(t *testing.T) { map[string]*pipepb.Coder{}, map[string]*pipepb.Coder{}, map[string]*pipepb.Coder{ - "test_lp": {Spec: &pipepb.FunctionSpec{Urn: urns.CoderLengthPrefix}, ComponentCoderIds: []string{"test"}}, - "test": {Spec: &pipepb.FunctionSpec{Urn: urns.CoderRow}}, + "test_lp": pipepb.Coder_builder{Spec: pipepb.FunctionSpec_builder{Urn: urns.CoderLengthPrefix}.Build(), ComponentCoderIds: []string{"test"}}.Build(), + "test": pipepb.Coder_builder{Spec: pipepb.FunctionSpec_builder{Urn: urns.CoderRow}.Build()}.Build(), }, }, {"needLP_recurse", urns.CoderKV, []string{"k", "v"}, map[string]*pipepb.Coder{}, map[string]*pipepb.Coder{ - "k": {Spec: &pipepb.FunctionSpec{Urn: urns.CoderRow}}, - "v": {Spec: &pipepb.FunctionSpec{Urn: urns.CoderBool}}, + "k": pipepb.Coder_builder{Spec: pipepb.FunctionSpec_builder{Urn: urns.CoderRow}.Build()}.Build(), + "v": pipepb.Coder_builder{Spec: pipepb.FunctionSpec_builder{Urn: urns.CoderBool}.Build()}.Build(), }, map[string]*pipepb.Coder{ - "test_lp": {Spec: &pipepb.FunctionSpec{Urn: urns.CoderKV}, ComponentCoderIds: []string{"k_lp", "v"}}, - "test": {Spec: &pipepb.FunctionSpec{Urn: urns.CoderKV}, ComponentCoderIds: []string{"k", "v"}}, - "k_lp": {Spec: &pipepb.FunctionSpec{Urn: urns.CoderLengthPrefix}, ComponentCoderIds: []string{"k"}}, - "k": {Spec: &pipepb.FunctionSpec{Urn: urns.CoderRow}}, - "v": {Spec: &pipepb.FunctionSpec{Urn: urns.CoderBool}}, + "test_lp": pipepb.Coder_builder{Spec: pipepb.FunctionSpec_builder{Urn: urns.CoderKV}.Build(), ComponentCoderIds: []string{"k_lp", "v"}}.Build(), + "test": pipepb.Coder_builder{Spec: pipepb.FunctionSpec_builder{Urn: urns.CoderKV}.Build(), ComponentCoderIds: []string{"k", "v"}}.Build(), + "k_lp": pipepb.Coder_builder{Spec: pipepb.FunctionSpec_builder{Urn: urns.CoderLengthPrefix}.Build(), ComponentCoderIds: []string{"k"}}.Build(), + "k": pipepb.Coder_builder{Spec: pipepb.FunctionSpec_builder{Urn: urns.CoderRow}.Build()}.Build(), + "v": pipepb.Coder_builder{Spec: pipepb.FunctionSpec_builder{Urn: urns.CoderBool}.Build()}.Build(), }, }, {"alreadyLP", urns.CoderLengthPrefix, []string{"k"}, map[string]*pipepb.Coder{}, map[string]*pipepb.Coder{ - "k": {Spec: &pipepb.FunctionSpec{Urn: urns.CoderRow}}, + "k": pipepb.Coder_builder{Spec: pipepb.FunctionSpec_builder{Urn: urns.CoderRow}.Build()}.Build(), }, map[string]*pipepb.Coder{ - "test": {Spec: &pipepb.FunctionSpec{Urn: urns.CoderLengthPrefix}, ComponentCoderIds: []string{"k"}}, - "k": {Spec: &pipepb.FunctionSpec{Urn: urns.CoderRow}}, + "test": pipepb.Coder_builder{Spec: pipepb.FunctionSpec_builder{Urn: urns.CoderLengthPrefix}.Build(), ComponentCoderIds: []string{"k"}}.Build(), + "k": pipepb.Coder_builder{Spec: pipepb.FunctionSpec_builder{Urn: urns.CoderRow}.Build()}.Build(), }, }, } for _, test := range tests { t.Run(test.name, func(t *testing.T) { // Add the initial coder to base. - test.base["test"] = &pipepb.Coder{ - Spec: &pipepb.FunctionSpec{Urn: test.urn}, + test.base["test"] = pipepb.Coder_builder{ + Spec: pipepb.FunctionSpec_builder{Urn: test.urn}.Build(), ComponentCoderIds: test.components, - } + }.Build() lpUnknownCoders("test", test.bundle, test.base) @@ -228,50 +228,50 @@ func Test_reconcileCoders(t *testing.T) { }{ {name: "noChanges", bundle: map[string]*pipepb.Coder{ - "a": {Spec: &pipepb.FunctionSpec{Urn: urns.CoderBool}}, + "a": pipepb.Coder_builder{Spec: pipepb.FunctionSpec_builder{Urn: urns.CoderBool}.Build()}.Build(), }, base: map[string]*pipepb.Coder{ - "a": {Spec: &pipepb.FunctionSpec{Urn: urns.CoderBool}}, - "b": {Spec: &pipepb.FunctionSpec{Urn: urns.CoderBytes}}, - "c": {Spec: &pipepb.FunctionSpec{Urn: urns.CoderStringUTF8}}, + "a": pipepb.Coder_builder{Spec: pipepb.FunctionSpec_builder{Urn: urns.CoderBool}.Build()}.Build(), + "b": pipepb.Coder_builder{Spec: pipepb.FunctionSpec_builder{Urn: urns.CoderBytes}.Build()}.Build(), + "c": pipepb.Coder_builder{Spec: pipepb.FunctionSpec_builder{Urn: urns.CoderStringUTF8}.Build()}.Build(), }, want: map[string]*pipepb.Coder{ - "a": {Spec: &pipepb.FunctionSpec{Urn: urns.CoderBool}}, + "a": pipepb.Coder_builder{Spec: pipepb.FunctionSpec_builder{Urn: urns.CoderBool}.Build()}.Build(), }, }, {name: "KV", bundle: map[string]*pipepb.Coder{ - "kv": {Spec: &pipepb.FunctionSpec{Urn: urns.CoderKV}, ComponentCoderIds: []string{"k", "v"}}, + "kv": pipepb.Coder_builder{Spec: pipepb.FunctionSpec_builder{Urn: urns.CoderKV}.Build(), ComponentCoderIds: []string{"k", "v"}}.Build(), }, base: map[string]*pipepb.Coder{ - "kv": {Spec: &pipepb.FunctionSpec{Urn: urns.CoderKV}, ComponentCoderIds: []string{"k", "v"}}, - "k": {Spec: &pipepb.FunctionSpec{Urn: urns.CoderBool}}, - "v": {Spec: &pipepb.FunctionSpec{Urn: urns.CoderBool}}, + "kv": pipepb.Coder_builder{Spec: pipepb.FunctionSpec_builder{Urn: urns.CoderKV}.Build(), ComponentCoderIds: []string{"k", "v"}}.Build(), + "k": pipepb.Coder_builder{Spec: pipepb.FunctionSpec_builder{Urn: urns.CoderBool}.Build()}.Build(), + "v": pipepb.Coder_builder{Spec: pipepb.FunctionSpec_builder{Urn: urns.CoderBool}.Build()}.Build(), }, want: map[string]*pipepb.Coder{ - "kv": {Spec: &pipepb.FunctionSpec{Urn: urns.CoderKV}, ComponentCoderIds: []string{"k", "v"}}, - "k": {Spec: &pipepb.FunctionSpec{Urn: urns.CoderBool}}, - "v": {Spec: &pipepb.FunctionSpec{Urn: urns.CoderBool}}, + "kv": pipepb.Coder_builder{Spec: pipepb.FunctionSpec_builder{Urn: urns.CoderKV}.Build(), ComponentCoderIds: []string{"k", "v"}}.Build(), + "k": pipepb.Coder_builder{Spec: pipepb.FunctionSpec_builder{Urn: urns.CoderBool}.Build()}.Build(), + "v": pipepb.Coder_builder{Spec: pipepb.FunctionSpec_builder{Urn: urns.CoderBool}.Build()}.Build(), }, }, {name: "KV-nested", bundle: map[string]*pipepb.Coder{ - "kv": {Spec: &pipepb.FunctionSpec{Urn: urns.CoderKV}, ComponentCoderIds: []string{"k", "v"}}, + "kv": pipepb.Coder_builder{Spec: pipepb.FunctionSpec_builder{Urn: urns.CoderKV}.Build(), ComponentCoderIds: []string{"k", "v"}}.Build(), }, base: map[string]*pipepb.Coder{ - "kv": {Spec: &pipepb.FunctionSpec{Urn: urns.CoderKV}, ComponentCoderIds: []string{"k", "v"}}, - "k": {Spec: &pipepb.FunctionSpec{Urn: urns.CoderKV}, ComponentCoderIds: []string{"a", "b"}}, - "v": {Spec: &pipepb.FunctionSpec{Urn: urns.CoderBool}}, - "a": {Spec: &pipepb.FunctionSpec{Urn: urns.CoderBytes}}, - "b": {Spec: &pipepb.FunctionSpec{Urn: urns.CoderRow}}, - "c": {Spec: &pipepb.FunctionSpec{Urn: urns.CoderStringUTF8}}, + "kv": pipepb.Coder_builder{Spec: pipepb.FunctionSpec_builder{Urn: urns.CoderKV}.Build(), ComponentCoderIds: []string{"k", "v"}}.Build(), + "k": pipepb.Coder_builder{Spec: pipepb.FunctionSpec_builder{Urn: urns.CoderKV}.Build(), ComponentCoderIds: []string{"a", "b"}}.Build(), + "v": pipepb.Coder_builder{Spec: pipepb.FunctionSpec_builder{Urn: urns.CoderBool}.Build()}.Build(), + "a": pipepb.Coder_builder{Spec: pipepb.FunctionSpec_builder{Urn: urns.CoderBytes}.Build()}.Build(), + "b": pipepb.Coder_builder{Spec: pipepb.FunctionSpec_builder{Urn: urns.CoderRow}.Build()}.Build(), + "c": pipepb.Coder_builder{Spec: pipepb.FunctionSpec_builder{Urn: urns.CoderStringUTF8}.Build()}.Build(), }, want: map[string]*pipepb.Coder{ - "kv": {Spec: &pipepb.FunctionSpec{Urn: urns.CoderKV}, ComponentCoderIds: []string{"k", "v"}}, - "k": {Spec: &pipepb.FunctionSpec{Urn: urns.CoderKV}, ComponentCoderIds: []string{"a", "b"}}, - "v": {Spec: &pipepb.FunctionSpec{Urn: urns.CoderBool}}, - "a": {Spec: &pipepb.FunctionSpec{Urn: urns.CoderBytes}}, - "b": {Spec: &pipepb.FunctionSpec{Urn: urns.CoderRow}}, + "kv": pipepb.Coder_builder{Spec: pipepb.FunctionSpec_builder{Urn: urns.CoderKV}.Build(), ComponentCoderIds: []string{"k", "v"}}.Build(), + "k": pipepb.Coder_builder{Spec: pipepb.FunctionSpec_builder{Urn: urns.CoderKV}.Build(), ComponentCoderIds: []string{"a", "b"}}.Build(), + "v": pipepb.Coder_builder{Spec: pipepb.FunctionSpec_builder{Urn: urns.CoderBool}.Build()}.Build(), + "a": pipepb.Coder_builder{Spec: pipepb.FunctionSpec_builder{Urn: urns.CoderBytes}.Build()}.Build(), + "b": pipepb.Coder_builder{Spec: pipepb.FunctionSpec_builder{Urn: urns.CoderRow}.Build()}.Build(), }, }, } @@ -299,75 +299,75 @@ func Test_pullDecoder(t *testing.T) { }{ { "bytes", - &pipepb.Coder{ - Spec: &pipepb.FunctionSpec{ + pipepb.Coder_builder{ + Spec: pipepb.FunctionSpec_builder{ Urn: urns.CoderBytes, - }, - }, + }.Build(), + }.Build(), map[string]*pipepb.Coder{}, []byte{3, 1, 2, 3}, }, { "varint", - &pipepb.Coder{ - Spec: &pipepb.FunctionSpec{ + pipepb.Coder_builder{ + Spec: pipepb.FunctionSpec_builder{ Urn: urns.CoderVarInt, - }, - }, + }.Build(), + }.Build(), map[string]*pipepb.Coder{}, []byte{255, 3}, }, { "bool", - &pipepb.Coder{ - Spec: &pipepb.FunctionSpec{ + pipepb.Coder_builder{ + Spec: pipepb.FunctionSpec_builder{ Urn: urns.CoderBool, - }, - }, + }.Build(), + }.Build(), map[string]*pipepb.Coder{}, []byte{1}, }, { "double", - &pipepb.Coder{ - Spec: &pipepb.FunctionSpec{ + pipepb.Coder_builder{ + Spec: pipepb.FunctionSpec_builder{ Urn: urns.CoderDouble, - }, - }, + }.Build(), + }.Build(), map[string]*pipepb.Coder{}, doubleBytes, }, { "iterable", - &pipepb.Coder{ - Spec: &pipepb.FunctionSpec{ + pipepb.Coder_builder{ + Spec: pipepb.FunctionSpec_builder{ Urn: urns.CoderIterable, - }, + }.Build(), ComponentCoderIds: []string{"elm"}, - }, + }.Build(), map[string]*pipepb.Coder{ - "elm": { - Spec: &pipepb.FunctionSpec{ + "elm": pipepb.Coder_builder{ + Spec: pipepb.FunctionSpec_builder{ Urn: urns.CoderVarInt, - }, - }, + }.Build(), + }.Build(), }, []byte{4, 0, 1, 2, 3}, }, { "kv", - &pipepb.Coder{ - Spec: &pipepb.FunctionSpec{ + pipepb.Coder_builder{ + Spec: pipepb.FunctionSpec_builder{ Urn: urns.CoderKV, - }, + }.Build(), ComponentCoderIds: []string{"key", "value"}, - }, + }.Build(), map[string]*pipepb.Coder{ - "key": { - Spec: &pipepb.FunctionSpec{ + "key": pipepb.Coder_builder{ + Spec: pipepb.FunctionSpec_builder{ Urn: urns.CoderVarInt, - }, - }, - "value": { - Spec: &pipepb.FunctionSpec{ + }.Build(), + }.Build(), + "value": pipepb.Coder_builder{ + Spec: pipepb.FunctionSpec_builder{ Urn: urns.CoderBool, - }, - }, + }.Build(), + }.Build(), }, []byte{3, 0}, }, diff --git a/sdks/go/pkg/beam/runners/prism/internal/environments.go b/sdks/go/pkg/beam/runners/prism/internal/environments.go index 2f960a04f0cb..e0b4e250a4af 100644 --- a/sdks/go/pkg/beam/runners/prism/internal/environments.go +++ b/sdks/go/pkg/beam/runners/prism/internal/environments.go @@ -78,17 +78,17 @@ func externalEnvironment(ctx context.Context, ep *pipepb.ExternalPayload, wk *wo defer conn.Close() pool := fnpb.NewBeamFnExternalWorkerPoolClient(conn) - endpoint := &pipepb.ApiServiceDescriptor{ + endpoint := pipepb.ApiServiceDescriptor_builder{ Url: wk.Endpoint(), - } - pool.StartWorker(ctx, &fnpb.StartWorkerRequest{ + }.Build() + pool.StartWorker(ctx, fnpb.StartWorkerRequest_builder{ WorkerId: wk.ID, ControlEndpoint: endpoint, LoggingEndpoint: endpoint, ArtifactEndpoint: endpoint, ProvisionEndpoint: endpoint, Params: ep.GetParams(), - }) + }.Build()) // Job processing happens here, but orchestrated by other goroutines // This goroutine blocks until the context is cancelled, signalling // that the pool runner should stop the worker. @@ -96,9 +96,9 @@ func externalEnvironment(ctx context.Context, ep *pipepb.ExternalPayload, wk *wo // Previous context cancelled so we need a new one // for this request. - pool.StopWorker(context.Background(), &fnpb.StopWorkerRequest{ + pool.StopWorker(context.Background(), fnpb.StopWorkerRequest_builder{ WorkerId: wk.ID, - }) + }.Build()) wk.Stop() } diff --git a/sdks/go/pkg/beam/runners/prism/internal/execute.go b/sdks/go/pkg/beam/runners/prism/internal/execute.go index 8b56c30eb61b..594af1759a56 100644 --- a/sdks/go/pkg/beam/runners/prism/internal/execute.go +++ b/sdks/go/pkg/beam/runners/prism/internal/execute.go @@ -276,27 +276,27 @@ func executePipeline(ctx context.Context, wks map[string]*worker.W, j *jobservic } } - tsb := em.AddTestStream(stage.ID, t.Outputs) + tsb := em.AddTestStream(stage.ID, t.GetOutputs()) for _, e := range pyld.GetEvents() { - switch ev := e.GetEvent().(type) { - case *pipepb.TestStreamPayload_Event_ElementEvent: + switch ev := e.WhichEvent(); ev { + case pipepb.TestStreamPayload_Event_ElementEvent_case: var elms []engine.TestStreamElement - for _, e := range ev.ElementEvent.GetElements() { + for _, e := range e.GetElementEvent().GetElements() { elms = append(elms, engine.TestStreamElement{Encoded: mayLP(e.GetEncodedElement()), EventTime: mtime.Time(e.GetTimestamp())}) } - tsb.AddElementEvent(ev.ElementEvent.GetTag(), elms) - ev.ElementEvent.GetTag() - case *pipepb.TestStreamPayload_Event_WatermarkEvent: - tsb.AddWatermarkEvent(ev.WatermarkEvent.GetTag(), mtime.Time(ev.WatermarkEvent.GetNewWatermark())) - case *pipepb.TestStreamPayload_Event_ProcessingTimeEvent: - if ev.ProcessingTimeEvent.GetAdvanceDuration() == int64(mtime.MaxTimestamp) { + tsb.AddElementEvent(e.GetElementEvent().GetTag(), elms) + e.GetElementEvent().GetTag() + case pipepb.TestStreamPayload_Event_WatermarkEvent_case: + tsb.AddWatermarkEvent(e.GetWatermarkEvent().GetTag(), mtime.Time(e.GetWatermarkEvent().GetNewWatermark())) + case pipepb.TestStreamPayload_Event_ProcessingTimeEvent_case: + if e.GetProcessingTimeEvent().GetAdvanceDuration() == int64(mtime.MaxTimestamp) { // TODO: Determine the SDK common formalism for setting processing time to infinity. tsb.AddProcessingTimeEvent(time.Duration(mtime.MaxTimestamp)) } else { - tsb.AddProcessingTimeEvent(time.Duration(ev.ProcessingTimeEvent.GetAdvanceDuration()) * time.Millisecond) + tsb.AddProcessingTimeEvent(time.Duration(e.GetProcessingTimeEvent().GetAdvanceDuration()) * time.Millisecond) } default: - return fmt.Errorf("prism error building stage %v - unknown TestStream event type: %T", stage.ID, ev) + return fmt.Errorf("prism error building stage %v - unknown TestStream event type: %v", stage.ID, ev) } } diff --git a/sdks/go/pkg/beam/runners/prism/internal/handlecombine.go b/sdks/go/pkg/beam/runners/prism/internal/handlecombine.go index 5d6d6dee6bf4..7fedad92e1d7 100644 --- a/sdks/go/pkg/beam/runners/prism/internal/handlecombine.go +++ b/sdks/go/pkg/beam/runners/prism/internal/handlecombine.go @@ -62,11 +62,11 @@ func (h *combine) PrepareTransform(tid string, t *pipepb.PTransform, comps *pipe // If we aren't lifting, the "default impl" for combines should be sufficient. if !h.config.EnableLifting { return prepareResult{ - SubbedComps: &pipepb.Components{ + SubbedComps: pipepb.Components_builder{ Transforms: map[string]*pipepb.PTransform{ tid: t, }, - }, + }.Build(), } } @@ -125,7 +125,7 @@ func (h *combine) PrepareTransform(tid string, t *pipepb.PTransform, comps *pipe // Now we can start synthesis! // Coder IDs - aID := cmb.AccumulatorCoderId + aID := cmb.GetAccumulatorCoderId() ckvprefix := "c" + tid + "_kv_" @@ -154,30 +154,30 @@ func (h *combine) PrepareTransform(tid string, t *pipepb.PTransform, comps *pipe extractEID := eprefix + "extract" coder := func(urn string, componentIDs ...string) *pipepb.Coder { - return &pipepb.Coder{ - Spec: &pipepb.FunctionSpec{ + return pipepb.Coder_builder{ + Spec: pipepb.FunctionSpec_builder{ Urn: urn, - }, + }.Build(), ComponentCoderIds: componentIDs, - } + }.Build() } pcol := func(name, coderID string) *pipepb.PCollection { - return &pipepb.PCollection{ + return pipepb.PCollection_builder{ UniqueName: name, CoderId: coderID, IsBounded: inputPCol.GetIsBounded(), WindowingStrategyId: inputPCol.GetWindowingStrategyId(), - } + }.Build() } tform := func(name, urn, in, out, env string) *pipepb.PTransform { - return &pipepb.PTransform{ + return pipepb.PTransform_builder{ UniqueName: name, - Spec: &pipepb.FunctionSpec{ + Spec: pipepb.FunctionSpec_builder{ Urn: urn, Payload: cmbPayload, - }, + }.Build(), Inputs: map[string]string{ "i0": in, }, @@ -185,10 +185,10 @@ func (h *combine) PrepareTransform(tid string, t *pipepb.PTransform, comps *pipe "i0": out, }, EnvironmentId: env, - } + }.Build() } - newComps := &pipepb.Components{ + newComps := pipepb.Components_builder{ Coders: map[string]*pipepb.Coder{ iterACID: coder(urns.CoderIterable, aID), kvkaCID: coder(urns.CoderKV, kID, aID), @@ -205,7 +205,7 @@ func (h *combine) PrepareTransform(tid string, t *pipepb.PTransform, comps *pipe mergeEID: tform(mergeEID, urns.TransformMerge, groupedNID, mergedNID, t.GetEnvironmentId()), extractEID: tform(extractEID, urns.TransformExtract, mergedNID, pcolOutID, t.GetEnvironmentId()), }, - } + }.Build() // We don't need to remove the composite, since we don't add it in // when we return the new transforms, so it's not in the topology. diff --git a/sdks/go/pkg/beam/runners/prism/internal/handlecombine_test.go b/sdks/go/pkg/beam/runners/prism/internal/handlecombine_test.go index 92fd8d78d46d..cae823910635 100644 --- a/sdks/go/pkg/beam/runners/prism/internal/handlecombine_test.go +++ b/sdks/go/pkg/beam/runners/prism/internal/handlecombine_test.go @@ -28,18 +28,18 @@ import ( func TestHandleCombine(t *testing.T) { undertest := "UnderTest" - combineTransform := &pipepb.PTransform{ + combineTransform := pipepb.PTransform_builder{ UniqueName: "COMBINE", - Spec: &pipepb.FunctionSpec{ + Spec: pipepb.FunctionSpec_builder{ Urn: urns.TransformCombinePerKey, - Payload: protox.MustEncode(&pipepb.CombinePayload{ - CombineFn: &pipepb.FunctionSpec{ + Payload: protox.MustEncode(pipepb.CombinePayload_builder{ + CombineFn: pipepb.FunctionSpec_builder{ Urn: "foo", Payload: []byte("bar"), - }, + }.Build(), AccumulatorCoderId: "AccumID", - }), - }, + }.Build()), + }.Build(), Inputs: map[string]string{ "input": "combineIn", }, @@ -50,39 +50,39 @@ func TestHandleCombine(t *testing.T) { "gbk", "combine_values", }, - } - combineValuesTransform := &pipepb.PTransform{ + }.Build() + combineValuesTransform := pipepb.PTransform_builder{ UniqueName: "combine_values", Subtransforms: []string{ "bonus_leaf", }, - } + }.Build() basePCollectionMap := map[string]*pipepb.PCollection{ - "combineIn": { + "combineIn": pipepb.PCollection_builder{ CoderId: "inputCoder", - }, - "combineOut": { + }.Build(), + "combineOut": pipepb.PCollection_builder{ CoderId: "outputCoder", - }, + }.Build(), } baseCoderMap := map[string]*pipepb.Coder{ - "int": { - Spec: &pipepb.FunctionSpec{Urn: urns.CoderVarInt}, - }, - "string": { - Spec: &pipepb.FunctionSpec{Urn: urns.CoderStringUTF8}, - }, - "AccumID": { - Spec: &pipepb.FunctionSpec{Urn: urns.CoderBytes}, - }, - "inputCoder": { - Spec: &pipepb.FunctionSpec{Urn: urns.CoderKV}, + "int": pipepb.Coder_builder{ + Spec: pipepb.FunctionSpec_builder{Urn: urns.CoderVarInt}.Build(), + }.Build(), + "string": pipepb.Coder_builder{ + Spec: pipepb.FunctionSpec_builder{Urn: urns.CoderStringUTF8}.Build(), + }.Build(), + "AccumID": pipepb.Coder_builder{ + Spec: pipepb.FunctionSpec_builder{Urn: urns.CoderBytes}.Build(), + }.Build(), + "inputCoder": pipepb.Coder_builder{ + Spec: pipepb.FunctionSpec_builder{Urn: urns.CoderKV}.Build(), ComponentCoderIds: []string{"int", "int"}, - }, - "outputCoder": { - Spec: &pipepb.FunctionSpec{Urn: urns.CoderKV}, + }.Build(), + "outputCoder": pipepb.Coder_builder{ + Spec: pipepb.FunctionSpec_builder{Urn: urns.CoderKV}.Build(), ComponentCoderIds: []string{"int", "string"}, - }, + }.Build(), } tests := []struct { @@ -95,104 +95,104 @@ func TestHandleCombine(t *testing.T) { { name: "unlifted", lifted: false, - comps: &pipepb.Components{ + comps: pipepb.Components_builder{ Transforms: map[string]*pipepb.PTransform{ undertest: combineTransform, "combine_values": combineValuesTransform, }, Pcollections: basePCollectionMap, Coders: baseCoderMap, - }, + }.Build(), want: prepareResult{ - SubbedComps: &pipepb.Components{ + SubbedComps: pipepb.Components_builder{ Transforms: map[string]*pipepb.PTransform{ undertest: combineTransform, }, - }, + }.Build(), }, }, { name: "lifted", lifted: true, - comps: &pipepb.Components{ + comps: pipepb.Components_builder{ Transforms: map[string]*pipepb.PTransform{ undertest: combineTransform, "combine_values": combineValuesTransform, }, Pcollections: basePCollectionMap, Coders: baseCoderMap, - }, + }.Build(), want: prepareResult{ RemovedLeaves: []string{"gbk", "combine_values", "bonus_leaf"}, - SubbedComps: &pipepb.Components{ + SubbedComps: pipepb.Components_builder{ Coders: map[string]*pipepb.Coder{ - "cUnderTest_iter_AccumID": { - Spec: &pipepb.FunctionSpec{Urn: urns.CoderIterable}, + "cUnderTest_iter_AccumID": pipepb.Coder_builder{ + Spec: pipepb.FunctionSpec_builder{Urn: urns.CoderIterable}.Build(), ComponentCoderIds: []string{"AccumID"}, - }, - "cUnderTest_kv_int_AccumID": { - Spec: &pipepb.FunctionSpec{Urn: urns.CoderKV}, + }.Build(), + "cUnderTest_kv_int_AccumID": pipepb.Coder_builder{ + Spec: pipepb.FunctionSpec_builder{Urn: urns.CoderKV}.Build(), ComponentCoderIds: []string{"int", "AccumID"}, - }, - "cUnderTest_kv_int_iterAccumID": { - Spec: &pipepb.FunctionSpec{Urn: urns.CoderKV}, + }.Build(), + "cUnderTest_kv_int_iterAccumID": pipepb.Coder_builder{ + Spec: pipepb.FunctionSpec_builder{Urn: urns.CoderKV}.Build(), ComponentCoderIds: []string{"int", "cUnderTest_iter_AccumID"}, - }, + }.Build(), }, Pcollections: map[string]*pipepb.PCollection{ - "nUnderTest_lifted": { + "nUnderTest_lifted": pipepb.PCollection_builder{ UniqueName: "nUnderTest_lifted", CoderId: "cUnderTest_kv_int_AccumID", - }, - "nUnderTest_grouped": { + }.Build(), + "nUnderTest_grouped": pipepb.PCollection_builder{ UniqueName: "nUnderTest_grouped", CoderId: "cUnderTest_kv_int_iterAccumID", - }, - "nUnderTest_merged": { + }.Build(), + "nUnderTest_merged": pipepb.PCollection_builder{ UniqueName: "nUnderTest_merged", CoderId: "cUnderTest_kv_int_AccumID", - }, + }.Build(), }, Transforms: map[string]*pipepb.PTransform{ - "eUnderTest_lift": { + "eUnderTest_lift": pipepb.PTransform_builder{ UniqueName: "eUnderTest_lift", - Spec: &pipepb.FunctionSpec{ + Spec: pipepb.FunctionSpec_builder{ Urn: urns.TransformPreCombine, Payload: combineTransform.GetSpec().GetPayload(), - }, + }.Build(), Inputs: map[string]string{"i0": "combineIn"}, Outputs: map[string]string{"i0": "nUnderTest_lifted"}, - }, - "eUnderTest_gbk": { + }.Build(), + "eUnderTest_gbk": pipepb.PTransform_builder{ UniqueName: "eUnderTest_gbk", - Spec: &pipepb.FunctionSpec{ + Spec: pipepb.FunctionSpec_builder{ Urn: urns.TransformGBK, // Technically shouldn't be here, but since GBK execution doesn't escape the runner // this never gets examined. Payload: combineTransform.GetSpec().GetPayload(), - }, + }.Build(), Inputs: map[string]string{"i0": "nUnderTest_lifted"}, Outputs: map[string]string{"i0": "nUnderTest_grouped"}, - }, - "eUnderTest_merge": { + }.Build(), + "eUnderTest_merge": pipepb.PTransform_builder{ UniqueName: "eUnderTest_merge", - Spec: &pipepb.FunctionSpec{ + Spec: pipepb.FunctionSpec_builder{ Urn: urns.TransformMerge, Payload: combineTransform.GetSpec().GetPayload(), - }, + }.Build(), Inputs: map[string]string{"i0": "nUnderTest_grouped"}, Outputs: map[string]string{"i0": "nUnderTest_merged"}, - }, - "eUnderTest_extract": { + }.Build(), + "eUnderTest_extract": pipepb.PTransform_builder{ UniqueName: "eUnderTest_extract", - Spec: &pipepb.FunctionSpec{ + Spec: pipepb.FunctionSpec_builder{ Urn: urns.TransformExtract, Payload: combineTransform.GetSpec().GetPayload(), - }, + }.Build(), Inputs: map[string]string{"i0": "nUnderTest_merged"}, Outputs: map[string]string{"i0": "combineOut"}, - }, + }.Build(), }, - }, + }.Build(), }, }, } diff --git a/sdks/go/pkg/beam/runners/prism/internal/handlepardo.go b/sdks/go/pkg/beam/runners/prism/internal/handlepardo.go index 13e9b6f1b79d..ca50599750f9 100644 --- a/sdks/go/pkg/beam/runners/prism/internal/handlepardo.go +++ b/sdks/go/pkg/beam/runners/prism/internal/handlepardo.go @@ -78,7 +78,7 @@ func (h *pardo) PrepareTransform(tid string, t *pipepb.PTransform, comps *pipepb } // Lets check for and remove anything that makes things less simple. - if pdo.RestrictionCoderId == "" { + if pdo.GetRestrictionCoderId() == "" { // Which inputs are Side inputs don't change the graph further, // so they're not included here. Any nearly any ParDo can have them. @@ -86,16 +86,16 @@ func (h *pardo) PrepareTransform(tid string, t *pipepb.PTransform, comps *pipepb // StatefulDoFns need to be marked as being roots. var forcedRoots []string - if len(pdo.StateSpecs)+len(pdo.TimerFamilySpecs) > 0 { + if len(pdo.GetStateSpecs())+len(pdo.GetTimerFamilySpecs()) > 0 { forcedRoots = append(forcedRoots, tid) } return prepareResult{ - SubbedComps: &pipepb.Components{ + SubbedComps: pipepb.Components_builder{ Transforms: map[string]*pipepb.PTransform{ tid: t, }, - }, + }.Build(), ForcedRoots: forcedRoots, } } @@ -146,7 +146,7 @@ func (h *pardo) PrepareTransform(tid string, t *pipepb.PTransform, comps *pipepb var pcolInID, inputLocalID string for localID, globalID := range t.GetInputs() { // The parallel input is the one that isn't a side input. - if _, ok := pdo.SideInputs[localID]; !ok { + if _, ok := pdo.GetSideInputs()[localID]; !ok { inputLocalID = localID pcolInID = globalID break @@ -154,18 +154,18 @@ func (h *pardo) PrepareTransform(tid string, t *pipepb.PTransform, comps *pipepb } inputPCol := comps.GetPcollections()[pcolInID] cEID := inputPCol.GetCoderId() - cRID := pdo.RestrictionCoderId + cRID := pdo.GetRestrictionCoderId() cSID := "c" + tid + "size" ckvERID := "c" + tid + "kv_ele_rest" ckvERSID := ckvERID + "_size" coder := func(urn string, componentIDs ...string) *pipepb.Coder { - return &pipepb.Coder{ - Spec: &pipepb.FunctionSpec{ + return pipepb.Coder_builder{ + Spec: pipepb.FunctionSpec_builder{ Urn: urn, - }, + }.Build(), ComponentCoderIds: componentIDs, - } + }.Build() } coders := map[string]*pipepb.Coder{ @@ -184,12 +184,12 @@ func (h *pardo) PrepareTransform(tid string, t *pipepb.PTransform, comps *pipepb nSPLITnSIZEDID := "n" + tid + "_splitnsized" pcol := func(name, coderID string) *pipepb.PCollection { - return &pipepb.PCollection{ + return pipepb.PCollection_builder{ UniqueName: name, CoderId: coderID, IsBounded: inputPCol.GetIsBounded(), WindowingStrategyId: inputPCol.GetWindowingStrategyId(), - } + }.Build() } pcols := map[string]*pipepb.PCollection{ @@ -211,18 +211,18 @@ func (h *pardo) PrepareTransform(tid string, t *pipepb.PTransform, comps *pipepb // could lead to an additional fusion oppportunity. newInputs := maps.Clone(t.GetInputs()) newInputs[inputLocalID] = in - return &pipepb.PTransform{ + return pipepb.PTransform_builder{ UniqueName: name, - Spec: &pipepb.FunctionSpec{ + Spec: pipepb.FunctionSpec_builder{ Urn: urn, Payload: pardoPayload, - }, + }.Build(), Inputs: newInputs, Outputs: map[string]string{ "i0": out, }, EnvironmentId: t.GetEnvironmentId(), - } + }.Build() } newInputs := maps.Clone(t.GetInputs()) @@ -231,23 +231,23 @@ func (h *pardo) PrepareTransform(tid string, t *pipepb.PTransform, comps *pipepb tforms := map[string]*pipepb.PTransform{ ePWRID: tform(ePWRID, urns.TransformPairWithRestriction, pcolInID, nPWRID), eSPLITnSIZEDID: tform(eSPLITnSIZEDID, urns.TransformSplitAndSize, nPWRID, nSPLITnSIZEDID), - eProcessID: { + eProcessID: pipepb.PTransform_builder{ UniqueName: eProcessID, - Spec: &pipepb.FunctionSpec{ + Spec: pipepb.FunctionSpec_builder{ Urn: urns.TransformProcessSizedElements, Payload: pardoPayload, - }, + }.Build(), Inputs: newInputs, Outputs: t.GetOutputs(), EnvironmentId: t.GetEnvironmentId(), - }, + }.Build(), } return prepareResult{ - SubbedComps: &pipepb.Components{ + SubbedComps: pipepb.Components_builder{ Coders: coders, Pcollections: pcols, Transforms: tforms, - }, + }.Build(), RemovedLeaves: removeSubTransforms(comps, t.GetSubtransforms()), // Force ProcessSized to be a root to ensure SDFs are able to split // between elements or within elements. diff --git a/sdks/go/pkg/beam/runners/prism/internal/handlerunner.go b/sdks/go/pkg/beam/runners/prism/internal/handlerunner.go index be9d39ad02b7..50bbafc6523a 100644 --- a/sdks/go/pkg/beam/runners/prism/internal/handlerunner.go +++ b/sdks/go/pkg/beam/runners/prism/internal/handlerunner.go @@ -84,7 +84,7 @@ func (h *runner) PrepareTransform(tid string, t *pipepb.PTransform, comps *pipep func (h *runner) handleFlatten(tid string, t *pipepb.PTransform, comps *pipepb.Components) prepareResult { if !h.config.SDKFlatten { - t.EnvironmentId = "" // force the flatten to be a runner transform due to configuration. + t.SetEnvironmentId("") // force the flatten to be a runner transform due to configuration. forcedRoots := []string{tid} // Have runner side transforms be roots. // Force runner flatten consumers to be roots. @@ -108,20 +108,20 @@ func (h *runner) handleFlatten(tid string, t *pipepb.PTransform, comps *pipepb.C coderSubs := map[string]*pipepb.Coder{} for _, p := range t.GetInputs() { inPCol := comps.GetPcollections()[p] - if inPCol.CoderId != outPCol.CoderId { - coderSubs[inPCol.CoderId] = outCoder + if inPCol.GetCoderId() != outPCol.GetCoderId() { + coderSubs[inPCol.GetCoderId()] = outCoder } } // Return the new components which is the transforms consumer return prepareResult{ // We sub this flatten with itself, to not drop it. - SubbedComps: &pipepb.Components{ + SubbedComps: pipepb.Components_builder{ Transforms: map[string]*pipepb.PTransform{ tid: t, }, Coders: coderSubs, - }, + }.Build(), RemovedLeaves: nil, ForcedRoots: forcedRoots, } diff --git a/sdks/go/pkg/beam/runners/prism/internal/jobservices/artifact.go b/sdks/go/pkg/beam/runners/prism/internal/jobservices/artifact.go index e42e3e7ca666..6ebd4051fe44 100644 --- a/sdks/go/pkg/beam/runners/prism/internal/jobservices/artifact.go +++ b/sdks/go/pkg/beam/runners/prism/internal/jobservices/artifact.go @@ -43,13 +43,11 @@ func (s *Server) ReverseArtifactRetrievalService(stream jobpb.ArtifactStagingSer slog.Group("dep", slog.String("urn", dep.GetTypeUrn()), slog.String("payload", string(dep.GetTypePayload())))) - stream.Send(&jobpb.ArtifactRequestWrapper{ - Request: &jobpb.ArtifactRequestWrapper_GetArtifact{ - GetArtifact: &jobpb.GetArtifactRequest{ - Artifact: dep, - }, - }, - }) + stream.Send(jobpb.ArtifactRequestWrapper_builder{ + GetArtifact: jobpb.GetArtifactRequest_builder{ + Artifact: dep, + }.Build(), + }.Build()) var buf bytes.Buffer for { in, err := stream.Recv() @@ -71,12 +69,12 @@ func (s *Server) ReverseArtifactRetrievalService(stream jobpb.ArtifactStagingSer } // Here's where we go through each environment's artifacts. // We do nothing with them. - switch req := in.GetResponse().(type) { - case *jobpb.ArtifactResponseWrapper_GetArtifactResponse: - buf.Write(req.GetArtifactResponse.GetData()) + switch in.WhichResponse() { + case jobpb.ArtifactResponseWrapper_GetArtifactResponse_case: + buf.Write(in.GetGetArtifactResponse().GetData()) - case *jobpb.ArtifactResponseWrapper_ResolveArtifactResponse: - err := fmt.Errorf("unexpected ResolveArtifactResponse to GetArtifact: %v", in.GetResponse()) + case jobpb.ArtifactResponseWrapper_ResolveArtifactResponse_case: + err := fmt.Errorf("unexpected ResolveArtifactResponse to GetArtifact: %v", in.WhichResponse()) slog.Error("GetArtifact failure", slog.Any("error", err)) return err } @@ -91,9 +89,9 @@ func (s *Server) ReverseArtifactRetrievalService(stream jobpb.ArtifactStagingSer } func (s *Server) ResolveArtifacts(_ context.Context, req *jobpb.ResolveArtifactsRequest) (*jobpb.ResolveArtifactsResponse, error) { - return &jobpb.ResolveArtifactsResponse{ + return jobpb.ResolveArtifactsResponse_builder{ Replacements: req.GetArtifacts(), - }, nil + }.Build(), nil } func (s *Server) GetArtifact(req *jobpb.GetArtifactRequest, stream jobpb.ArtifactRetrievalService_GetArtifactServer) error { @@ -107,13 +105,13 @@ func (s *Server) GetArtifact(req *jobpb.GetArtifactRequest, stream jobpb.Artifac chunk := 128 * 1024 * 1024 // 128 MB var i int for i+chunk < len(buf) { - stream.Send(&jobpb.GetArtifactResponse{ + stream.Send(jobpb.GetArtifactResponse_builder{ Data: buf[i : i+chunk], - }) + }.Build()) i += chunk } - stream.Send(&jobpb.GetArtifactResponse{ + stream.Send(jobpb.GetArtifactResponse_builder{ Data: buf[i:], - }) + }.Build()) return nil } diff --git a/sdks/go/pkg/beam/runners/prism/internal/jobservices/management.go b/sdks/go/pkg/beam/runners/prism/internal/jobservices/management.go index af559a92ab46..ed6905306801 100644 --- a/sdks/go/pkg/beam/runners/prism/internal/jobservices/management.go +++ b/sdks/go/pkg/beam/runners/prism/internal/jobservices/management.go @@ -196,7 +196,7 @@ func (s *Server) Prepare(ctx context.Context, req *jobpb.PrepareJobRequest) (_ * return nil, wrapped } - t.EnvironmentId = "" // Unset the environment, to ensure it's handled prism side. + t.SetEnvironmentId("") // Unset the environment, to ensure it's handled prism side. testStreamIds = append(testStreamIds, tid) default: @@ -243,17 +243,17 @@ func (s *Server) Prepare(ctx context.Context, req *jobpb.PrepareJobRequest) (_ * pipepb.OutputTime_EARLIEST_IN_PANE, pipepb.OutputTime_LATEST_IN_PANE) // Non default triggers should fail. if ws.GetTrigger().GetDefault() == nil { - dt := &pipepb.Trigger{ - Trigger: &pipepb.Trigger_Default_{}, - } + dt := pipepb.Trigger_builder{ + Default: &pipepb.Trigger_Default{}, + }.Build() // Allow Never and Always triggers to unblock iteration on Java and Python SDKs. // Without multiple firings, these will be very similar to the default trigger. - nt := &pipepb.Trigger{ - Trigger: &pipepb.Trigger_Never_{}, - } - at := &pipepb.Trigger{ - Trigger: &pipepb.Trigger_Always_{}, - } + nt := pipepb.Trigger_builder{ + Never: &pipepb.Trigger_Never{}, + }.Build() + at := pipepb.Trigger_builder{ + Always: &pipepb.Trigger_Always{}, + }.Build() check("WindowingStrategy.Trigger", ws.GetTrigger().String(), dt.String(), nt.String(), at.String()) } } @@ -265,13 +265,13 @@ func (s *Server) Prepare(ctx context.Context, req *jobpb.PrepareJobRequest) (_ * job.Failed(err) return nil, err } - return &jobpb.PrepareJobResponse{ + return jobpb.PrepareJobResponse_builder{ PreparationId: job.key, StagingSessionToken: job.key, - ArtifactStagingEndpoint: &pipepb.ApiServiceDescriptor{ + ArtifactStagingEndpoint: pipepb.ApiServiceDescriptor_builder{ Url: s.Endpoint(), - }, - }, nil + }.Build(), + }.Build(), nil } func (s *Server) Run(ctx context.Context, req *jobpb.RunJobRequest) (*jobpb.RunJobResponse, error) { @@ -282,9 +282,9 @@ func (s *Server) Run(ctx context.Context, req *jobpb.RunJobRequest) (*jobpb.RunJ // Bring up a background goroutine to allow the job to continue processing. go s.execute(job) - return &jobpb.RunJobResponse{ + return jobpb.RunJobResponse_builder{ JobId: job.key, - }, nil + }.Build(), nil } // Cancel a Job requested by the CancelJobRequest for jobs not in an already terminal state. @@ -300,16 +300,16 @@ func (s *Server) Cancel(_ context.Context, req *jobpb.CancelJobRequest) (*jobpb. switch state { case jobpb.JobState_CANCELLED, jobpb.JobState_DONE, jobpb.JobState_DRAINED, jobpb.JobState_UPDATED, jobpb.JobState_FAILED: // Already at terminal state. - return &jobpb.CancelJobResponse{ + return jobpb.CancelJobResponse_builder{ State: state, - }, nil + }.Build(), nil } job.SendMsg("canceling " + job.String()) job.Canceling() job.CancelFn(ErrCancel) - return &jobpb.CancelJobResponse{ + return jobpb.CancelJobResponse_builder{ State: jobpb.JobState_CANCELLING, - }, nil + }.Build(), nil } // GetMessageStream subscribes to a stream of state changes and messages from the job. If throughput @@ -336,14 +336,12 @@ func (s *Server) GetMessageStream(req *jobpb.JobMessagesRequest, stream jobpb.Jo return nil case jobpb.JobState_FAILED: // Ensure we send an error message with the cause of the job failure. - stream.Send(&jobpb.JobMessagesResponse{ - Response: &jobpb.JobMessagesResponse_MessageResponse{ - MessageResponse: &jobpb.JobMessage{ - MessageText: job.failureErr.Error(), - Importance: jobpb.JobMessage_JOB_MESSAGE_ERROR, - }, - }, - }) + stream.Send(jobpb.JobMessagesResponse_builder{ + MessageResponse: jobpb.JobMessage_builder{ + MessageText: job.failureErr.Error(), + Importance: jobpb.JobMessage_JOB_MESSAGE_ERROR, + }.Build(), + }.Build()) return nil } job.streamCond.Wait() @@ -360,27 +358,23 @@ func (s *Server) GetMessageStream(req *jobpb.JobMessagesRequest, stream jobpb.Jo } for curMsg < job.maxMsg && len(job.msgs) > 0 { msg := job.msgs[curMsg-job.minMsg] - stream.Send(&jobpb.JobMessagesResponse{ - Response: &jobpb.JobMessagesResponse_MessageResponse{ - MessageResponse: &jobpb.JobMessage{ - MessageText: msg, - Importance: jobpb.JobMessage_JOB_MESSAGE_BASIC, - }, - }, - }) + stream.Send(jobpb.JobMessagesResponse_builder{ + MessageResponse: jobpb.JobMessage_builder{ + MessageText: msg, + Importance: jobpb.JobMessage_JOB_MESSAGE_BASIC, + }.Build(), + }.Build()) curMsg++ } if curState <= job.stateIdx { state = job.state.Load().(jobpb.JobState_Enum) curState = job.stateIdx + 1 job.streamCond.L.Unlock() - stream.Send(&jobpb.JobMessagesResponse{ - Response: &jobpb.JobMessagesResponse_StateResponse{ - StateResponse: &jobpb.JobStateEvent{ - State: state, - }, - }, - }) + stream.Send(jobpb.JobMessagesResponse_builder{ + StateResponse: jobpb.JobStateEvent_builder{ + State: state, + }.Build(), + }.Build()) job.streamCond.L.Lock() } } @@ -392,12 +386,12 @@ func (s *Server) GetJobMetrics(ctx context.Context, req *jobpb.GetJobMetricsRequ if j == nil { return nil, fmt.Errorf("GetJobMetrics: unknown jobID: %v", req.GetJobId()) } - return &jobpb.GetJobMetricsResponse{ - Metrics: &jobpb.MetricResults{ + return jobpb.GetJobMetricsResponse_builder{ + Metrics: jobpb.MetricResults_builder{ Attempted: j.metrics.Results(tentative), Committed: j.metrics.Results(committed), - }, - }, nil + }.Build(), + }.Build(), nil } // GetJobs returns the set of active jobs and associated metadata. @@ -407,12 +401,12 @@ func (s *Server) GetJobs(context.Context, *jobpb.GetJobsRequest) (*jobpb.GetJobs resp := &jobpb.GetJobsResponse{} for key, job := range s.jobs { - resp.JobInfo = append(resp.JobInfo, &jobpb.JobInfo{ + resp.SetJobInfo(append(resp.GetJobInfo(), jobpb.JobInfo_builder{ JobId: key, JobName: job.jobName, State: job.state.Load().(jobpb.JobState_Enum), PipelineOptions: job.options, - }) + }.Build())) } return resp, nil } @@ -426,9 +420,9 @@ func (s *Server) GetPipeline(_ context.Context, req *jobpb.GetJobPipelineRequest if !ok { return nil, fmt.Errorf("job with id %v not found", req.GetJobId()) } - return &jobpb.GetJobPipelineResponse{ + return jobpb.GetJobPipelineResponse_builder{ Pipeline: j.Pipeline, - }, nil + }.Build(), nil } // GetState returns the current state of the job with the requested id. @@ -440,18 +434,18 @@ func (s *Server) GetState(_ context.Context, req *jobpb.GetJobStateRequest) (*jo if !ok { return nil, fmt.Errorf("job with id %v not found", req.GetJobId()) } - return &jobpb.JobStateEvent{ + return jobpb.JobStateEvent_builder{ State: j.state.Load().(jobpb.JobState_Enum), Timestamp: timestamppb.New(j.stateTime), - }, nil + }.Build(), nil } // DescribePipelineOptions is a no-op since it's unclear how it is to function. // Apparently only implemented in the Python SDK. func (s *Server) DescribePipelineOptions(context.Context, *jobpb.DescribePipelineOptionsRequest) (*jobpb.DescribePipelineOptionsResponse, error) { - return &jobpb.DescribePipelineOptionsResponse{ + return jobpb.DescribePipelineOptionsResponse_builder{ Options: []*jobpb.PipelineOptionDescriptor{}, - }, nil + }.Build(), nil } // GetStateStream returns the job state as it changes. @@ -469,10 +463,10 @@ func (s *Server) GetStateStream(req *jobpb.GetJobStateRequest, stream jobpb.JobS state := job.state.Load().(jobpb.JobState_Enum) for { job.streamCond.L.Unlock() - stream.Send(&jobpb.JobStateEvent{ + stream.Send(jobpb.JobStateEvent_builder{ State: state, Timestamp: timestamppb.Now(), - }) + }.Build()) job.streamCond.L.Lock() switch state { case jobpb.JobState_CANCELLED, jobpb.JobState_DONE, jobpb.JobState_DRAINED, jobpb.JobState_UPDATED, jobpb.JobState_FAILED: diff --git a/sdks/go/pkg/beam/runners/prism/internal/jobservices/management_test.go b/sdks/go/pkg/beam/runners/prism/internal/jobservices/management_test.go index 5aad58b4a86f..47de5aabf583 100644 --- a/sdks/go/pkg/beam/runners/prism/internal/jobservices/management_test.go +++ b/sdks/go/pkg/beam/runners/prism/internal/jobservices/management_test.go @@ -40,9 +40,9 @@ import ( func TestServer(t *testing.T) { wantName := "testJob" - wantPipeline := &pipepb.Pipeline{ + wantPipeline := pipepb.Pipeline_builder{ Requirements: []string{urns.RequirementSplittableDoFn}, - } + }.Build() cmpOpts := []cmp.Option{protocmp.Transform(), cmpopts.EquateEmpty()} tests := []struct { @@ -72,15 +72,15 @@ func TestServer(t *testing.T) { if err != nil { t.Fatalf("GetJobs() = %v, want nil", err) } - if diff := cmp.Diff(&jobpb.GetJobsResponse{ + if diff := cmp.Diff(jobpb.GetJobsResponse_builder{ JobInfo: []*jobpb.JobInfo{ - { + jobpb.JobInfo_builder{ JobId: "job-001", // Expected initial JobID. JobName: wantName, State: jobpb.JobState_STOPPED, - }, + }.Build(), }, - }, resp, cmpOpts...); diff != "" { + }.Build(), resp, cmpOpts...); diff != "" { t.Errorf("GetJobs() (-want, +got):\n%v", diff) } }, @@ -89,86 +89,86 @@ func TestServer(t *testing.T) { if err != nil { t.Fatalf("GetJobs() = %v, want nil", err) } - if diff := cmp.Diff(&jobpb.GetJobsResponse{ + if diff := cmp.Diff(jobpb.GetJobsResponse_builder{ JobInfo: []*jobpb.JobInfo{ - { + jobpb.JobInfo_builder{ JobId: jobID, JobName: wantName, State: jobpb.JobState_DONE, - }, + }.Build(), }, - }, resp, cmpOpts...); diff != "" { + }.Build(), resp, cmpOpts...); diff != "" { t.Errorf("GetJobs() (-want, +got):\n%v", diff) } }, }, { name: "GetMetrics", noJobsCheck: func(ctx context.Context, t *testing.T, undertest *Server) { - _, err := undertest.GetJobMetrics(ctx, &jobpb.GetJobMetricsRequest{JobId: "job-001"}) + _, err := undertest.GetJobMetrics(ctx, jobpb.GetJobMetricsRequest_builder{JobId: "job-001"}.Build()) if err == nil { t.Errorf("GetPipeline(\"job-001\") = %v, want not found error", err) } }, postPrepCheck: func(ctx context.Context, t *testing.T, undertest *Server) { - resp, err := undertest.GetJobMetrics(ctx, &jobpb.GetJobMetricsRequest{JobId: "job-001"}) + resp, err := undertest.GetJobMetrics(ctx, jobpb.GetJobMetricsRequest_builder{JobId: "job-001"}.Build()) if err != nil { t.Errorf("GetPipeline(\"job-001\") = %v, want nil", err) } - if diff := cmp.Diff(&jobpb.GetJobMetricsResponse{ + if diff := cmp.Diff(jobpb.GetJobMetricsResponse_builder{ Metrics: &jobpb.MetricResults{}, - }, resp, cmpOpts...); diff != "" { + }.Build(), resp, cmpOpts...); diff != "" { t.Errorf("GetPipeline(\"job-001\") (-want, +got):\n%v", diff) } }, postRunCheck: func(ctx context.Context, t *testing.T, undertest *Server, jobID string) { - resp, err := undertest.GetJobMetrics(ctx, &jobpb.GetJobMetricsRequest{JobId: jobID}) + resp, err := undertest.GetJobMetrics(ctx, jobpb.GetJobMetricsRequest_builder{JobId: jobID}.Build()) if err != nil { t.Errorf("GetPipeline(\"job-001\") = %v, want nil", err) } - if diff := cmp.Diff(&jobpb.GetJobMetricsResponse{ - Metrics: &jobpb.MetricResults{ + if diff := cmp.Diff(jobpb.GetJobMetricsResponse_builder{ + Metrics: jobpb.MetricResults_builder{ Committed: []*pipepb.MonitoringInfo{ - { + pipepb.MonitoringInfo_builder{ Urn: metricsx.UrnToString(metricsx.UrnElementCount), Type: metricsx.UrnToType(metricsx.UrnElementCount), Payload: []byte("\x01"), Labels: map[string]string{ "PCOLLECTION": "id.out", }, - }, + }.Build(), }, - }, - }, resp, cmpOpts...); diff != "" { + }.Build(), + }.Build(), resp, cmpOpts...); diff != "" { t.Errorf("GetPipeline(\"job-001\") (-want, +got):\n%v", diff) } }, }, { name: "GetPipeline", noJobsCheck: func(ctx context.Context, t *testing.T, undertest *Server) { - _, err := undertest.GetPipeline(ctx, &jobpb.GetJobPipelineRequest{JobId: "job-001"}) + _, err := undertest.GetPipeline(ctx, jobpb.GetJobPipelineRequest_builder{JobId: "job-001"}.Build()) if err == nil { t.Errorf("GetPipeline(\"job-001\") = %v, want not found error", err) } }, postPrepCheck: func(ctx context.Context, t *testing.T, undertest *Server) { - resp, err := undertest.GetPipeline(ctx, &jobpb.GetJobPipelineRequest{JobId: "job-001"}) + resp, err := undertest.GetPipeline(ctx, jobpb.GetJobPipelineRequest_builder{JobId: "job-001"}.Build()) if err != nil { t.Errorf("GetPipeline(\"job-001\") = %v, want nil", err) } - if diff := cmp.Diff(&jobpb.GetJobPipelineResponse{ + if diff := cmp.Diff(jobpb.GetJobPipelineResponse_builder{ Pipeline: wantPipeline, - }, resp, cmpOpts...); diff != "" { + }.Build(), resp, cmpOpts...); diff != "" { t.Errorf("GetPipeline(\"job-001\") (-want, +got):\n%v", diff) } }, postRunCheck: func(ctx context.Context, t *testing.T, undertest *Server, jobID string) { - resp, err := undertest.GetPipeline(ctx, &jobpb.GetJobPipelineRequest{JobId: jobID}) + resp, err := undertest.GetPipeline(ctx, jobpb.GetJobPipelineRequest_builder{JobId: jobID}.Build()) if err != nil { t.Errorf("GetPipeline(\"job-001\") = %v, want nil", err) } - if diff := cmp.Diff(&jobpb.GetJobPipelineResponse{ + if diff := cmp.Diff(jobpb.GetJobPipelineResponse_builder{ Pipeline: wantPipeline, - }, resp, cmpOpts...); diff != "" { + }.Build(), resp, cmpOpts...); diff != "" { t.Errorf("GetPipeline(\"job-001\") (-want, +got):\n%v", diff) } }, @@ -178,7 +178,7 @@ func TestServer(t *testing.T) { postRunState: jobpb.JobState_RUNNING, noJobsCheck: func(ctx context.Context, t *testing.T, undertest *Server) { id := "job-001" - _, err := undertest.Cancel(ctx, &jobpb.CancelJobRequest{JobId: id}) + _, err := undertest.Cancel(ctx, jobpb.CancelJobRequest_builder{JobId: id}.Build()) // Cancel currently returns nil, nil when Job not found if err != nil { t.Errorf("Cancel(%q) = %v, want not found error", id, err) @@ -186,25 +186,25 @@ func TestServer(t *testing.T) { }, postPrepCheck: func(ctx context.Context, t *testing.T, undertest *Server) { id := "job-001" - resp, err := undertest.Cancel(ctx, &jobpb.CancelJobRequest{JobId: id}) + resp, err := undertest.Cancel(ctx, jobpb.CancelJobRequest_builder{JobId: id}.Build()) if err != nil { t.Errorf("Cancel(%q) = %v, want not found error", id, err) } - if diff := cmp.Diff(&jobpb.CancelJobResponse{ + if diff := cmp.Diff(jobpb.CancelJobResponse_builder{ State: jobpb.JobState_CANCELLING, - }, resp, cmpOpts...); diff != "" { + }.Build(), resp, cmpOpts...); diff != "" { t.Errorf("Cancel(%q) (-want, +got):\n%s", id, diff) } }, postRunCheck: func(ctx context.Context, t *testing.T, undertest *Server, jobID string) { id := "job-001" - resp, err := undertest.Cancel(ctx, &jobpb.CancelJobRequest{JobId: id}) + resp, err := undertest.Cancel(ctx, jobpb.CancelJobRequest_builder{JobId: id}.Build()) if err != nil { t.Errorf("Cancel(%q) = %v, want not found error", id, err) } - if diff := cmp.Diff(&jobpb.CancelJobResponse{ + if diff := cmp.Diff(jobpb.CancelJobResponse_builder{ State: jobpb.JobState_CANCELLING, - }, resp, cmpOpts...); diff != "" { + }.Build(), resp, cmpOpts...); diff != "" { t.Errorf("Cancel(%q) (-want, +got):\n%s", id, diff) } }, @@ -219,23 +219,23 @@ func TestServer(t *testing.T) { shortIDCount := "elemCount" shortIDSize := "elemSize" - j.AddMetricShortIDs(&fnpb.MonitoringInfosMetadataResponse{ + j.AddMetricShortIDs(fnpb.MonitoringInfosMetadataResponse_builder{ MonitoringInfo: map[string]*pipepb.MonitoringInfo{ - shortIDCount: { + shortIDCount: pipepb.MonitoringInfo_builder{ Urn: metricsx.UrnToString(metricsx.UrnElementCount), Type: metricsx.UrnToType(metricsx.UrnElementCount), Labels: map[string]string{ "PCOLLECTION": "id.out", }, - }, + }.Build(), }, - }) - j.ContributeFinalMetrics(&fnpb.ProcessBundleResponse{ + }.Build()) + j.ContributeFinalMetrics(fnpb.ProcessBundleResponse_builder{ MonitoringData: map[string][]byte{ shortIDCount: countData, shortIDSize: sizeData, }, - }) + }.Build()) state := jobpb.JobState_DONE if test.postRunState != jobpb.JobState_UNSPECIFIED { state = test.postRunState @@ -247,19 +247,19 @@ func TestServer(t *testing.T) { ctx := context.Background() test.noJobsCheck(ctx, t, undertest) - prepResp, err := undertest.Prepare(ctx, &jobpb.PrepareJobRequest{ + prepResp, err := undertest.Prepare(ctx, jobpb.PrepareJobRequest_builder{ Pipeline: wantPipeline, JobName: wantName, - }) + }.Build()) if err != nil { t.Fatalf("Prepare(%v) = %v, want nil", wantName, err) } test.postPrepCheck(ctx, t, undertest) - jrResp, err := undertest.Run(ctx, &jobpb.RunJobRequest{ + jrResp, err := undertest.Run(ctx, jobpb.RunJobRequest_builder{ PreparationId: prepResp.GetPreparationId(), - }) + }.Build()) if err != nil { t.Fatalf("Run(%v) = %v, want nil", wantName, err) } @@ -271,9 +271,9 @@ func TestServer(t *testing.T) { func TestGetMessageStream(t *testing.T) { wantName := "testJob" - wantPipeline := &pipepb.Pipeline{ + wantPipeline := pipepb.Pipeline_builder{ Requirements: []string{urns.RequirementSplittableDoFn}, - } + }.Build() var called sync.WaitGroup called.Add(1) ctx, _, clientConn := serveTestServer(t, func(j *Job) { @@ -289,9 +289,9 @@ func TestGetMessageStream(t *testing.T) { jobCli := jobpb.NewJobServiceClient(clientConn) // PreJob submission - msgStream, err := jobCli.GetMessageStream(ctx, &jobpb.JobMessagesRequest{ + msgStream, err := jobCli.GetMessageStream(ctx, jobpb.JobMessagesRequest_builder{ JobId: "job-001", - }) + }.Build()) if err != nil { t.Errorf("GetMessageStream: wanted successful connection, got %v", err) } @@ -300,18 +300,18 @@ func TestGetMessageStream(t *testing.T) { t.Error("wanted error on non-existent job, but didn't happen.") } - prepResp, err := jobCli.Prepare(ctx, &jobpb.PrepareJobRequest{ + prepResp, err := jobCli.Prepare(ctx, jobpb.PrepareJobRequest_builder{ Pipeline: wantPipeline, JobName: wantName, - }) + }.Build()) if err != nil { t.Fatalf("Prepare(%v) = %v, want nil", wantName, err) } // Post Job submission - msgStream, err = jobCli.GetMessageStream(ctx, &jobpb.JobMessagesRequest{ + msgStream, err = jobCli.GetMessageStream(ctx, jobpb.JobMessagesRequest_builder{ JobId: "job-001", - }) + }.Build()) if err != nil { t.Errorf("GetMessageStream: wanted successful connection, got %v", err) } @@ -323,9 +323,9 @@ func TestGetMessageStream(t *testing.T) { t.Errorf("GetMessageStream().Recv() = %v, want %v", got, want) } - _, err = jobCli.Run(ctx, &jobpb.RunJobRequest{ + _, err = jobCli.Run(ctx, jobpb.RunJobRequest_builder{ PreparationId: prepResp.GetPreparationId(), - }) + }.Build()) if err != nil { t.Fatalf("Run(%v) = %v, want nil", wantName, err) } @@ -363,9 +363,9 @@ func TestGetMessageStream(t *testing.T) { // Create a new message stream, we should still get a tail of messages (in this case, all of them) // And the final state. - msgStream, err = jobCli.GetMessageStream(ctx, &jobpb.JobMessagesRequest{ + msgStream, err = jobCli.GetMessageStream(ctx, jobpb.JobMessagesRequest_builder{ JobId: "job-001", - }) + }.Build()) if err != nil { t.Errorf("GetMessageStream: wanted successful connection, got %v", err) } @@ -401,9 +401,9 @@ func TestGetMessageStream(t *testing.T) { func TestGetMessageStream_BufferCycling(t *testing.T) { wantName := "testJob" - wantPipeline := &pipepb.Pipeline{ + wantPipeline := pipepb.Pipeline_builder{ Requirements: []string{urns.RequirementSplittableDoFn}, - } + }.Build() var called sync.WaitGroup called.Add(1) ctx, _, clientConn := serveTestServer(t, func(j *Job) { @@ -418,16 +418,16 @@ func TestGetMessageStream_BufferCycling(t *testing.T) { }) jobCli := jobpb.NewJobServiceClient(clientConn) - prepResp, err := jobCli.Prepare(ctx, &jobpb.PrepareJobRequest{ + prepResp, err := jobCli.Prepare(ctx, jobpb.PrepareJobRequest_builder{ Pipeline: wantPipeline, JobName: wantName, - }) + }.Build()) if err != nil { t.Fatalf("Prepare(%v) = %v, want nil", wantName, err) } - _, err = jobCli.Run(ctx, &jobpb.RunJobRequest{ + _, err = jobCli.Run(ctx, jobpb.RunJobRequest_builder{ PreparationId: prepResp.GetPreparationId(), - }) + }.Build()) if err != nil { t.Fatalf("Run(%v) = %v, want nil", wantName, err) } @@ -436,9 +436,9 @@ func TestGetMessageStream_BufferCycling(t *testing.T) { // Create a new message stream, we should still get a tail of messages (in this case, all of them) // And the final state. - msgStream, err := jobCli.GetMessageStream(ctx, &jobpb.JobMessagesRequest{ + msgStream, err := jobCli.GetMessageStream(ctx, jobpb.JobMessagesRequest_builder{ JobId: "job-001", - }) + }.Build()) if err != nil { t.Errorf("GetMessageStream: wanted successful connection, got %v", err) } diff --git a/sdks/go/pkg/beam/runners/prism/internal/jobservices/metrics.go b/sdks/go/pkg/beam/runners/prism/internal/jobservices/metrics.go index bbbdfd1eba4f..ca35323b0386 100644 --- a/sdks/go/pkg/beam/runners/prism/internal/jobservices/metrics.go +++ b/sdks/go/pkg/beam/runners/prism/internal/jobservices/metrics.go @@ -232,12 +232,12 @@ func (m *sumInt64) accumulate(pyld []byte) error { func (m *sumInt64) toProto(key metricKey) *pipepb.MonitoringInfo { var buf bytes.Buffer coder.EncodeVarInt(m.sum, &buf) - return &pipepb.MonitoringInfo{ + return pipepb.MonitoringInfo_builder{ Urn: key.Urn(), Type: getMetTyp(pipepb.MonitoringInfoTypeUrns_SUM_INT64_TYPE), Payload: buf.Bytes(), Labels: key.Labels(), - } + }.Build() } type sumFloat64 struct { @@ -256,12 +256,12 @@ func (m *sumFloat64) accumulate(pyld []byte) error { func (m *sumFloat64) toProto(key metricKey) *pipepb.MonitoringInfo { var buf bytes.Buffer coder.EncodeDouble(m.sum, &buf) - return &pipepb.MonitoringInfo{ + return pipepb.MonitoringInfo_builder{ Urn: key.Urn(), Type: getMetTyp(pipepb.MonitoringInfoTypeUrns_SUM_DOUBLE_TYPE), Payload: buf.Bytes(), Labels: key.Labels(), - } + }.Build() } type progress struct { @@ -293,12 +293,12 @@ func (m *progress) toProto(key metricKey) *pipepb.MonitoringInfo { for _, v := range m.snap { coder.EncodeDouble(v, &buf) } - return &pipepb.MonitoringInfo{ + return pipepb.MonitoringInfo_builder{ Urn: key.Urn(), Type: getMetTyp(pipepb.MonitoringInfoTypeUrns_PROGRESS_TYPE), Payload: buf.Bytes(), Labels: key.Labels(), - } + }.Build() } func ordMin[T constraints.Ordered](a T, b T) T { @@ -350,12 +350,12 @@ func (m *distributionInt64) toProto(key metricKey) *pipepb.MonitoringInfo { coder.EncodeVarInt(m.dist.Sum, &buf) coder.EncodeVarInt(m.dist.Min, &buf) coder.EncodeVarInt(m.dist.Max, &buf) - return &pipepb.MonitoringInfo{ + return pipepb.MonitoringInfo_builder{ Urn: key.Urn(), Type: getMetTyp(pipepb.MonitoringInfoTypeUrns_DISTRIBUTION_INT64_TYPE), Payload: buf.Bytes(), Labels: key.Labels(), - } + }.Build() } type gaugeInt64 struct { @@ -387,12 +387,12 @@ func (m *gaugeInt64) toProto(key metricKey) *pipepb.MonitoringInfo { var buf bytes.Buffer coder.EncodeVarInt(m.millisSinceEpoch, &buf) coder.EncodeVarInt(m.val, &buf) - return &pipepb.MonitoringInfo{ + return pipepb.MonitoringInfo_builder{ Urn: key.Urn(), Type: getMetTyp(pipepb.MonitoringInfoTypeUrns_LATEST_INT64_TYPE), Payload: buf.Bytes(), Labels: key.Labels(), - } + }.Build() } type gaugeFloat64 struct { @@ -424,12 +424,12 @@ func (m *gaugeFloat64) toProto(key metricKey) *pipepb.MonitoringInfo { var buf bytes.Buffer coder.EncodeVarInt(m.millisSinceEpoch, &buf) coder.EncodeDouble(m.val, &buf) - return &pipepb.MonitoringInfo{ + return pipepb.MonitoringInfo_builder{ Urn: key.Urn(), Type: getMetTyp(pipepb.MonitoringInfoTypeUrns_LATEST_DOUBLE_TYPE), Payload: buf.Bytes(), Labels: key.Labels(), - } + }.Build() } type stringSet struct { @@ -460,12 +460,12 @@ func (m *stringSet) toProto(key metricKey) *pipepb.MonitoringInfo { for k := range m.set { coder.EncodeStringUTF8(k, &buf) } - return &pipepb.MonitoringInfo{ + return pipepb.MonitoringInfo_builder{ Urn: key.Urn(), Type: getMetTyp(pipepb.MonitoringInfoTypeUrns_SET_STRING_TYPE), Payload: buf.Bytes(), Labels: key.Labels(), - } + }.Build() } type durability int @@ -589,7 +589,7 @@ func (m *metricsStore) AddShortIDs(resp *fnpb.MonitoringInfosMetadataResponse) { urn := mi.GetUrn() ops, ok := mUrn2Ops[urn] if !ok { - slog.Debug("unknown metrics urn", slog.String("shortID", short), slog.String("urn", urn), slog.String("type", mi.Type)) + slog.Debug("unknown metrics urn", slog.String("shortID", short), slog.String("urn", urn), slog.String("type", mi.GetType())) continue } key := ops.keyFn(urn, mi.GetLabels()) diff --git a/sdks/go/pkg/beam/runners/prism/internal/jobservices/metrics_test.go b/sdks/go/pkg/beam/runners/prism/internal/jobservices/metrics_test.go index 339d862292fd..183d4814017a 100644 --- a/sdks/go/pkg/beam/runners/prism/internal/jobservices/metrics_test.go +++ b/sdks/go/pkg/beam/runners/prism/internal/jobservices/metrics_test.go @@ -40,16 +40,16 @@ func makeInfo(enum pipepb.MonitoringInfoSpecs_Enum) *pipepb.MonitoringInfo { for _, l := range spec.GetRequiredLabels() { labels[l] = l } - return &pipepb.MonitoringInfo{ + return pipepb.MonitoringInfo_builder{ Urn: spec.GetUrn(), Type: spec.GetType(), Labels: labels, - } + }.Build() } func makeInfoWBytes(enum pipepb.MonitoringInfoSpecs_Enum, payload []byte) *pipepb.MonitoringInfo { info := makeInfo(enum) - info.Payload = payload + info.SetPayload(payload) return info } @@ -173,14 +173,14 @@ func Test_metricsStore_ContributeMetrics(t *testing.T) { t.Run(test.name, func(t *testing.T) { ms := metricsStore{} - ms.AddShortIDs(&fnpb.MonitoringInfosMetadataResponse{ + ms.AddShortIDs(fnpb.MonitoringInfosMetadataResponse_builder{ MonitoringInfo: test.shortIDs, - }) + }.Build()) for _, payload := range test.input { - ms.ContributeFinalMetrics(&fnpb.ProcessBundleResponse{ + ms.ContributeFinalMetrics(fnpb.ProcessBundleResponse_builder{ MonitoringData: payload, - }) + }.Build()) } got := ms.Results(committed) diff --git a/sdks/go/pkg/beam/runners/prism/internal/jobservices/server_test.go b/sdks/go/pkg/beam/runners/prism/internal/jobservices/server_test.go index fb72048d478c..2a274dc47023 100644 --- a/sdks/go/pkg/beam/runners/prism/internal/jobservices/server_test.go +++ b/sdks/go/pkg/beam/runners/prism/internal/jobservices/server_test.go @@ -47,15 +47,15 @@ func TestServer_JobLifecycle(t *testing.T) { }) ctx := context.Background() - wantPipeline := &pipepb.Pipeline{ + wantPipeline := pipepb.Pipeline_builder{ Requirements: []string{urns.RequirementSplittableDoFn}, - } + }.Build() wantName := "testJob" - resp, err := undertest.Prepare(ctx, &jobpb.PrepareJobRequest{ + resp, err := undertest.Prepare(ctx, jobpb.PrepareJobRequest_builder{ Pipeline: wantPipeline, JobName: wantName, - }) + }.Build()) if err != nil { t.Fatalf("server.Prepare() = %v, want nil", err) } @@ -64,9 +64,9 @@ func TestServer_JobLifecycle(t *testing.T) { t.Fatalf("server.Prepare() = returned empty preparation ID, want non-empty: %v", prototext.Format(resp)) } - runResp, err := undertest.Run(ctx, &jobpb.RunJobRequest{ + runResp, err := undertest.Run(ctx, jobpb.RunJobRequest_builder{ PreparationId: resp.GetPreparationId(), - }) + }.Build()) if err != nil { t.Fatalf("server.Run() = %v, want nil", err) } @@ -94,15 +94,15 @@ func TestServer_RunThenCancel(t *testing.T) { }) ctx := context.Background() - wantPipeline := &pipepb.Pipeline{ + wantPipeline := pipepb.Pipeline_builder{ Requirements: []string{urns.RequirementSplittableDoFn}, - } + }.Build() wantName := "testJob" - resp, err := undertest.Prepare(ctx, &jobpb.PrepareJobRequest{ + resp, err := undertest.Prepare(ctx, jobpb.PrepareJobRequest_builder{ Pipeline: wantPipeline, JobName: wantName, - }) + }.Build()) if err != nil { t.Fatalf("server.Prepare() = %v, want nil", err) } @@ -111,9 +111,9 @@ func TestServer_RunThenCancel(t *testing.T) { t.Fatalf("server.Prepare() = returned empty preparation ID, want non-empty: %v", prototext.Format(resp)) } - runResp, err := undertest.Run(ctx, &jobpb.RunJobRequest{ + runResp, err := undertest.Run(ctx, jobpb.RunJobRequest_builder{ PreparationId: resp.GetPreparationId(), - }) + }.Build()) if err != nil { t.Fatalf("server.Run() = %v, want nil", err) } @@ -121,24 +121,24 @@ func TestServer_RunThenCancel(t *testing.T) { t.Fatalf("server.Run() = returned empty preparation ID, want non-empty") } - cancelResp, err := undertest.Cancel(ctx, &jobpb.CancelJobRequest{ + cancelResp, err := undertest.Cancel(ctx, jobpb.CancelJobRequest_builder{ JobId: runResp.GetJobId(), - }) + }.Build()) if err != nil { t.Fatalf("server.Canceling() = %v, want nil", err) } - if cancelResp.State != jobpb.JobState_CANCELLING { - t.Fatalf("server.Canceling() = %v, want %v", cancelResp.State, jobpb.JobState_CANCELLING) + if cancelResp.GetState() != jobpb.JobState_CANCELLING { + t.Fatalf("server.Canceling() = %v, want %v", cancelResp.GetState(), jobpb.JobState_CANCELLING) } called.Wait() - stateResp, err := undertest.GetState(ctx, &jobpb.GetJobStateRequest{JobId: runResp.GetJobId()}) + stateResp, err := undertest.GetState(ctx, jobpb.GetJobStateRequest_builder{JobId: runResp.GetJobId()}.Build()) if err != nil { t.Fatalf("server.GetState() = %v, want nil", err) } - if stateResp.State != jobpb.JobState_CANCELLED { - t.Fatalf("server.GetState() = %v, want %v", stateResp.State, jobpb.JobState_CANCELLED) + if stateResp.GetState() != jobpb.JobState_CANCELLED { + t.Fatalf("server.GetState() = %v, want %v", stateResp.GetState(), jobpb.JobState_CANCELLED) } } diff --git a/sdks/go/pkg/beam/runners/prism/internal/preprocess.go b/sdks/go/pkg/beam/runners/prism/internal/preprocess.go index 0d3ec7c365c1..33344e2f5164 100644 --- a/sdks/go/pkg/beam/runners/prism/internal/preprocess.go +++ b/sdks/go/pkg/beam/runners/prism/internal/preprocess.go @@ -279,7 +279,7 @@ func defaultFusion(topological []string, comps *pipepb.Components, facts fusionF transforms: []string{tid}, } // TODO validate that fused stages have the same environment. - stg.envID = comps.GetTransforms()[tid].EnvironmentId + stg.envID = comps.GetTransforms()[tid].GetEnvironmentId() stages = append(stages, stg) @@ -445,7 +445,7 @@ func finalizeStage(stg *stage, comps *pipepb.Components, pipelineFacts *fusionFa if err := (proto.UnmarshalOptions{}).Unmarshal(t.GetSpec().GetPayload(), pardo); err != nil { return fmt.Errorf("unable to decode ParDoPayload for %v", link.Transform) } - stg.finalize = pardo.RequestsFinalization + stg.finalize = pardo.GetRequestsFinalization() if len(pardo.GetTimerFamilySpecs())+len(pardo.GetStateSpecs())+len(pardo.GetOnWindowExpirationTimerFamilySpec()) > 0 { stg.stateful = true } diff --git a/sdks/go/pkg/beam/runners/prism/internal/preprocess_test.go b/sdks/go/pkg/beam/runners/prism/internal/preprocess_test.go index 56879a3455f2..88a559dc7958 100644 --- a/sdks/go/pkg/beam/runners/prism/internal/preprocess_test.go +++ b/sdks/go/pkg/beam/runners/prism/internal/preprocess_test.go @@ -37,94 +37,94 @@ func Test_preprocessor_preProcessGraph(t *testing.T) { }{ { name: "noPreparer", - input: &pipepb.Components{ + input: pipepb.Components_builder{ Transforms: map[string]*pipepb.PTransform{ - "e1": { + "e1": pipepb.PTransform_builder{ UniqueName: "e1", - Spec: &pipepb.FunctionSpec{ + Spec: pipepb.FunctionSpec_builder{ Urn: "defaultUrn", - }, - }, + }.Build(), + }.Build(), }, - }, + }.Build(), wantStages: []*stage{{transforms: []string{"e1"}}}, - wantComponents: &pipepb.Components{ + wantComponents: pipepb.Components_builder{ Transforms: map[string]*pipepb.PTransform{ - "e1": { + "e1": pipepb.PTransform_builder{ UniqueName: "e1", - Spec: &pipepb.FunctionSpec{ + Spec: pipepb.FunctionSpec_builder{ Urn: "defaultUrn", - }, - }, + }.Build(), + }.Build(), }, - }, + }.Build(), }, { name: "preparer", forcedRoots: []string{"e1_early", "e1_late"}, - input: &pipepb.Components{ + input: pipepb.Components_builder{ Transforms: map[string]*pipepb.PTransform{ - "e1": { + "e1": pipepb.PTransform_builder{ UniqueName: "e1", - Spec: &pipepb.FunctionSpec{ + Spec: pipepb.FunctionSpec_builder{ Urn: "test_urn", - }, - }, + }.Build(), + }.Build(), }, // Initialize maps because they always are by proto unmarshallers. Pcollections: map[string]*pipepb.PCollection{}, WindowingStrategies: map[string]*pipepb.WindowingStrategy{}, Coders: map[string]*pipepb.Coder{}, Environments: map[string]*pipepb.Environment{}, - }, + }.Build(), wantStages: []*stage{ {transforms: []string{"e1_early"}, envID: "env1", outputs: []link{{Transform: "e1_early", Local: "i0", Global: "pcol1"}}}, {transforms: []string{"e1_late"}, envID: "env1", primaryInput: "pcol1"}}, - wantComponents: &pipepb.Components{ + wantComponents: pipepb.Components_builder{ Transforms: map[string]*pipepb.PTransform{ // Original is always kept - "e1": { + "e1": pipepb.PTransform_builder{ UniqueName: "e1", - Spec: &pipepb.FunctionSpec{ + Spec: pipepb.FunctionSpec_builder{ Urn: "test_urn", - }, - }, - "e1_early": { + }.Build(), + }.Build(), + "e1_early": pipepb.PTransform_builder{ UniqueName: "e1_early", - Spec: &pipepb.FunctionSpec{ + Spec: pipepb.FunctionSpec_builder{ Urn: "defaultUrn", - }, + }.Build(), Outputs: map[string]string{"i0": "pcol1"}, EnvironmentId: "env1", - }, - "e1_late": { + }.Build(), + "e1_late": pipepb.PTransform_builder{ UniqueName: "e1_late", - Spec: &pipepb.FunctionSpec{ + Spec: pipepb.FunctionSpec_builder{ Urn: "defaultUrn", - }, + }.Build(), Inputs: map[string]string{"i0": "pcol1"}, EnvironmentId: "env1", - }, + }.Build(), }, Pcollections: map[string]*pipepb.PCollection{ - "pcol1": { + "pcol1": pipepb.PCollection_builder{ UniqueName: "pcol1", CoderId: "coder1", WindowingStrategyId: "ws1", - }, + }.Build(), }, Coders: map[string]*pipepb.Coder{ - "coder1": {Spec: &pipepb.FunctionSpec{Urn: "coder1"}}, + "coder1": pipepb.Coder_builder{Spec: pipepb.FunctionSpec_builder{Urn: "coder1"}.Build()}.Build(), }, WindowingStrategies: map[string]*pipepb.WindowingStrategy{ - "ws1": {WindowCoderId: "global"}, + "ws1": pipepb.WindowingStrategy_builder{WindowCoderId: "global"}.Build(), }, Environments: map[string]*pipepb.Environment{ - "env1": {Urn: "env1"}, + "env1": pipepb.Environment_builder{Urn: "env1"}.Build(), }, - }, + }.Build(), }, } for _, test := range tests { @@ -160,54 +160,54 @@ func (p *testPreparer) PrepareUrns() []string { func (p *testPreparer) PrepareTransform(tid string, t *pipepb.PTransform, comps *pipepb.Components) prepareResult { return prepareResult{ ForcedRoots: p.ForcedRoots, - SubbedComps: &pipepb.Components{ + SubbedComps: pipepb.Components_builder{ Transforms: map[string]*pipepb.PTransform{ - "e1_early": { + "e1_early": pipepb.PTransform_builder{ UniqueName: "e1_early", - Spec: &pipepb.FunctionSpec{ + Spec: pipepb.FunctionSpec_builder{ Urn: "defaultUrn", - }, + }.Build(), Outputs: map[string]string{"i0": "pcol1"}, EnvironmentId: "env1", - }, - "e1_late": { + }.Build(), + "e1_late": pipepb.PTransform_builder{ UniqueName: "e1_late", - Spec: &pipepb.FunctionSpec{ + Spec: pipepb.FunctionSpec_builder{ Urn: "defaultUrn", - }, + }.Build(), Inputs: map[string]string{"i0": "pcol1"}, EnvironmentId: "env1", - }, + }.Build(), }, Pcollections: map[string]*pipepb.PCollection{ - "pcol1": { + "pcol1": pipepb.PCollection_builder{ UniqueName: "pcol1", CoderId: "coder1", WindowingStrategyId: "ws1", - }, + }.Build(), }, Coders: map[string]*pipepb.Coder{ - "coder1": {Spec: &pipepb.FunctionSpec{Urn: "coder1"}}, + "coder1": pipepb.Coder_builder{Spec: pipepb.FunctionSpec_builder{Urn: "coder1"}.Build()}.Build(), }, WindowingStrategies: map[string]*pipepb.WindowingStrategy{ - "ws1": {WindowCoderId: "global"}, + "ws1": pipepb.WindowingStrategy_builder{WindowCoderId: "global"}.Build(), }, Environments: map[string]*pipepb.Environment{ - "env1": {Urn: "env1"}, + "env1": pipepb.Environment_builder{Urn: "env1"}.Build(), }, - }, + }.Build(), RemovedLeaves: []string{"e1"}, } } func TestComputeFacts(t *testing.T) { sideInputSpec := func(sis map[string]*pipepb.SideInput) *pipepb.FunctionSpec { - return &pipepb.FunctionSpec{ + return pipepb.FunctionSpec_builder{ Urn: urns.TransformParDo, - Payload: protox.MustEncode(&pipepb.ParDoPayload{ + Payload: protox.MustEncode(pipepb.ParDoPayload_builder{ SideInputs: sis, - }), - } + }.Build()), + }.Build() } tests := []struct { @@ -220,14 +220,14 @@ func TestComputeFacts(t *testing.T) { { name: "single_transform", topological: []string{"t1"}, - comps: &pipepb.Components{ + comps: pipepb.Components_builder{ Transforms: map[string]*pipepb.PTransform{ - "t1": { + "t1": pipepb.PTransform_builder{ Inputs: map[string]string{}, Outputs: map[string]string{}, - }, + }.Build(), }, - }, + }.Build(), want: &fusionFacts{ PcolProducers: map[string]link{}, PcolConsumers: map[string][]link{}, @@ -242,18 +242,18 @@ func TestComputeFacts(t *testing.T) { }, { name: "t2_consumes_n1_as_primary", topological: []string{"t1", "t2"}, - comps: &pipepb.Components{ + comps: pipepb.Components_builder{ Transforms: map[string]*pipepb.PTransform{ - "t1": { + "t1": pipepb.PTransform_builder{ Inputs: map[string]string{}, Outputs: map[string]string{"o0": "n1"}, - }, - "t2": { + }.Build(), + "t2": pipepb.PTransform_builder{ Inputs: map[string]string{"i0": "n1"}, Outputs: map[string]string{}, - }, + }.Build(), }, - }, + }.Build(), want: &fusionFacts{ PcolProducers: map[string]link{ "n1": {Transform: "t1", Local: "o0", Global: "n1"}, @@ -274,21 +274,21 @@ func TestComputeFacts(t *testing.T) { }, { name: "t2_consumes_n1_as_side", topological: []string{"t1", "t2"}, - comps: &pipepb.Components{ + comps: pipepb.Components_builder{ Transforms: map[string]*pipepb.PTransform{ - "t1": { + "t1": pipepb.PTransform_builder{ Inputs: map[string]string{}, Outputs: map[string]string{"o0": "n1"}, - }, - "t2": { + }.Build(), + "t2": pipepb.PTransform_builder{ Inputs: map[string]string{"i0": "n1"}, Outputs: map[string]string{}, Spec: sideInputSpec(map[string]*pipepb.SideInput{ "i0": {}, }), - }, + }.Build(), }, - }, + }.Build(), want: &fusionFacts{ PcolProducers: map[string]link{ "n1": {Transform: "t1", Local: "o0", Global: "n1"}, @@ -311,25 +311,25 @@ func TestComputeFacts(t *testing.T) { }, { name: "t2_consumes_n2_as_side_n1_as_primary_produces_n2", topological: []string{"t1", "t2", "t3"}, - comps: &pipepb.Components{ + comps: pipepb.Components_builder{ Transforms: map[string]*pipepb.PTransform{ - "t1": { + "t1": pipepb.PTransform_builder{ Inputs: map[string]string{}, Outputs: map[string]string{"o0": "n1"}, - }, - "t2": { + }.Build(), + "t2": pipepb.PTransform_builder{ Inputs: map[string]string{"i0": "n1"}, Outputs: map[string]string{"o0": "n2"}, Spec: sideInputSpec(map[string]*pipepb.SideInput{ "i0": {}, }), - }, - "t3": { + }.Build(), + "t3": pipepb.PTransform_builder{ Inputs: map[string]string{"i0": "n2"}, Outputs: map[string]string{}, - }, + }.Build(), }, - }, + }.Build(), want: &fusionFacts{ PcolProducers: map[string]link{ "n1": {Transform: "t1", Local: "o0", Global: "n1"}, @@ -356,25 +356,25 @@ func TestComputeFacts(t *testing.T) { }, { name: "t3_consumes_n1_as_side_n2_as_primary", topological: []string{"t1", "t2", "t3"}, - comps: &pipepb.Components{ + comps: pipepb.Components_builder{ Transforms: map[string]*pipepb.PTransform{ - "t1": { + "t1": pipepb.PTransform_builder{ Inputs: map[string]string{}, Outputs: map[string]string{"o0": "n1"}, - }, - "t2": { + }.Build(), + "t2": pipepb.PTransform_builder{ Inputs: map[string]string{}, Outputs: map[string]string{"o0": "n2"}, - }, - "t3": { + }.Build(), + "t3": pipepb.PTransform_builder{ Inputs: map[string]string{"i0": "n2", "i1": "n1"}, Outputs: map[string]string{}, Spec: sideInputSpec(map[string]*pipepb.SideInput{ "i1": {}, }), - }, + }.Build(), }, - }, + }.Build(), want: &fusionFacts{ PcolProducers: map[string]link{ "n1": {Transform: "t1", Local: "o0", Global: "n1"}, @@ -401,25 +401,25 @@ func TestComputeFacts(t *testing.T) { }, { name: "t3_consumes_n2_as_side", topological: []string{"t1", "t2", "t3"}, - comps: &pipepb.Components{ + comps: pipepb.Components_builder{ Transforms: map[string]*pipepb.PTransform{ - "t1": { + "t1": pipepb.PTransform_builder{ Inputs: map[string]string{}, Outputs: map[string]string{"o0": "n1"}, - }, - "t2": { + }.Build(), + "t2": pipepb.PTransform_builder{ Inputs: map[string]string{"i0": "n1"}, Outputs: map[string]string{"o0": "n2"}, - }, - "t3": { + }.Build(), + "t3": pipepb.PTransform_builder{ Inputs: map[string]string{"i1": "n2"}, Outputs: map[string]string{}, Spec: sideInputSpec(map[string]*pipepb.SideInput{ "i1": {}, }), - }, + }.Build(), }, - }, + }.Build(), want: &fusionFacts{ PcolProducers: map[string]link{ "n1": {Transform: "t1", Local: "o0", Global: "n1"}, @@ -446,32 +446,32 @@ func TestComputeFacts(t *testing.T) { }, { name: "criss_cross", topological: []string{"t1", "t2", "t3", "t4"}, - comps: &pipepb.Components{ + comps: pipepb.Components_builder{ Transforms: map[string]*pipepb.PTransform{ - "t1": { + "t1": pipepb.PTransform_builder{ Inputs: map[string]string{}, Outputs: map[string]string{"o0": "n1"}, - }, - "t2": { + }.Build(), + "t2": pipepb.PTransform_builder{ Inputs: map[string]string{}, Outputs: map[string]string{"o0": "n2"}, - }, - "t3": { + }.Build(), + "t3": pipepb.PTransform_builder{ Inputs: map[string]string{"i0": "n1", "i1": "n2"}, Outputs: map[string]string{}, Spec: sideInputSpec(map[string]*pipepb.SideInput{ "i1": {}, }), - }, - "t4": { + }.Build(), + "t4": pipepb.PTransform_builder{ Inputs: map[string]string{"i0": "n2", "i1": "n1"}, Outputs: map[string]string{}, Spec: sideInputSpec(map[string]*pipepb.SideInput{ "i1": {}, }), - }, + }.Build(), }, - }, + }.Build(), want: &fusionFacts{ PcolProducers: map[string]link{ "n1": {Transform: "t1", Local: "o0", Global: "n1"}, @@ -501,48 +501,48 @@ func TestComputeFacts(t *testing.T) { }, { name: "long_criss_cross_tail_cross", topological: []string{"r1", "r2", "r3", "l1", "l2", "l3", "r4", "l4"}, - comps: &pipepb.Components{ + comps: pipepb.Components_builder{ Transforms: map[string]*pipepb.PTransform{ - "r1": { + "r1": pipepb.PTransform_builder{ Inputs: map[string]string{}, Outputs: map[string]string{"o0": "n1"}, - }, - "r2": { + }.Build(), + "r2": pipepb.PTransform_builder{ Inputs: map[string]string{"i0": "n1"}, Outputs: map[string]string{"o0": "n2"}, - }, - "r3": { + }.Build(), + "r3": pipepb.PTransform_builder{ Inputs: map[string]string{"i0": "n2"}, Outputs: map[string]string{"o0": "n3"}, - }, - "l1": { + }.Build(), + "l1": pipepb.PTransform_builder{ Inputs: map[string]string{}, Outputs: map[string]string{"o0": "n4"}, - }, - "l2": { + }.Build(), + "l2": pipepb.PTransform_builder{ Inputs: map[string]string{"i0": "n4"}, Outputs: map[string]string{"o0": "n5"}, - }, - "l3": { + }.Build(), + "l3": pipepb.PTransform_builder{ Inputs: map[string]string{"i0": "n5"}, Outputs: map[string]string{"o0": "n6"}, - }, - "r4": { + }.Build(), + "r4": pipepb.PTransform_builder{ Inputs: map[string]string{"i0": "n3", "i1": "n6"}, Outputs: map[string]string{}, Spec: sideInputSpec(map[string]*pipepb.SideInput{ "i1": {}, }), - }, - "l4": { + }.Build(), + "l4": pipepb.PTransform_builder{ Inputs: map[string]string{"i0": "n6", "i1": "n3"}, Outputs: map[string]string{}, Spec: sideInputSpec(map[string]*pipepb.SideInput{ "i1": {}, }), - }, + }.Build(), }, - }, + }.Build(), want: &fusionFacts{ PcolProducers: map[string]link{ "n1": {Transform: "r1", Local: "o0", Global: "n1"}, @@ -588,48 +588,48 @@ func TestComputeFacts(t *testing.T) { }, { name: "long_criss_cross_head_cross", topological: []string{"r1", "r2", "r3", "l1", "l2", "l3", "r4", "l4"}, - comps: &pipepb.Components{ + comps: pipepb.Components_builder{ Transforms: map[string]*pipepb.PTransform{ - "r1": { + "r1": pipepb.PTransform_builder{ Inputs: map[string]string{}, Outputs: map[string]string{"o0": "n1"}, - }, - "r2": { + }.Build(), + "r2": pipepb.PTransform_builder{ Inputs: map[string]string{"i0": "n1"}, Outputs: map[string]string{"o0": "n2"}, - }, - "r3": { + }.Build(), + "r3": pipepb.PTransform_builder{ Inputs: map[string]string{"i0": "n2"}, Outputs: map[string]string{"o0": "n3"}, - }, - "l1": { + }.Build(), + "l1": pipepb.PTransform_builder{ Inputs: map[string]string{}, Outputs: map[string]string{"o0": "n4"}, - }, - "l2": { + }.Build(), + "l2": pipepb.PTransform_builder{ Inputs: map[string]string{"i0": "n4"}, Outputs: map[string]string{"o0": "n5"}, - }, - "l3": { + }.Build(), + "l3": pipepb.PTransform_builder{ Inputs: map[string]string{"i0": "n5"}, Outputs: map[string]string{"o0": "n6"}, - }, - "r4": { + }.Build(), + "r4": pipepb.PTransform_builder{ Inputs: map[string]string{"i0": "n3", "i1": "n4"}, Outputs: map[string]string{}, Spec: sideInputSpec(map[string]*pipepb.SideInput{ "i1": {}, }), - }, - "l4": { + }.Build(), + "l4": pipepb.PTransform_builder{ Inputs: map[string]string{"i0": "n6", "i1": "n1"}, Outputs: map[string]string{}, Spec: sideInputSpec(map[string]*pipepb.SideInput{ "i1": {}, }), - }, + }.Build(), }, - }, + }.Build(), want: &fusionFacts{ PcolProducers: map[string]link{ "n1": {Transform: "r1", Local: "o0", Global: "n1"}, diff --git a/sdks/go/pkg/beam/runners/prism/internal/stage.go b/sdks/go/pkg/beam/runners/prism/internal/stage.go index e1e942a06f0c..42f5e898b266 100644 --- a/sdks/go/pkg/beam/runners/prism/internal/stage.go +++ b/sdks/go/pkg/beam/runners/prism/internal/stage.go @@ -360,15 +360,15 @@ func getSideInputs(t *pipepb.PTransform) (map[string]*pipepb.SideInput, error) { } func portFor(wInCid string, wk *worker.W) []byte { - sourcePort := &fnpb.RemoteGrpcPort{ + sourcePort := fnpb.RemoteGrpcPort_builder{ CoderId: wInCid, - ApiServiceDescriptor: &pipepb.ApiServiceDescriptor{ + ApiServiceDescriptor: pipepb.ApiServiceDescriptor_builder{ Url: wk.Endpoint(), - }, - } + }.Build(), + }.Build() sourcePortBytes, err := proto.Marshal(sourcePort) if err != nil { - slog.Error("bad port", slog.Any("error", err), slog.String("endpoint", sourcePort.ApiServiceDescriptor.GetUrl())) + slog.Error("bad port", slog.Any("error", err), slog.String("endpoint", sourcePort.GetApiServiceDescriptor().GetUrl())) } return sourcePortBytes } @@ -436,13 +436,13 @@ func buildDescriptor(stg *stage, comps *pipepb.Components, wk *worker.W, em *eng } *cid = newCid } - switch s := s.GetSpec().(type) { - case *pipepb.StateSpec_BagSpec: - rewriteCoder(&s.BagSpec.ElementCoderId) - case *pipepb.StateSpec_SetSpec: - rewriteCoder(&s.SetSpec.ElementCoderId) - case *pipepb.StateSpec_OrderedListSpec: - rewriteCoder(&s.OrderedListSpec.ElementCoderId) + switch s.WhichSpec() { + case pipepb.StateSpec_BagSpec_case: + rewriteCoder(proto.String(s.GetBagSpec().GetElementCoderId())) + case pipepb.StateSpec_SetSpec_case: + rewriteCoder(proto.String(s.GetSetSpec().GetElementCoderId())) + case pipepb.StateSpec_OrderedListSpec_case: + rewriteCoder(proto.String(s.GetOrderedListSpec().GetElementCoderId())) // Add the length determination helper for OrderedList state values. if stg.stateTypeLen == nil { stg.stateTypeLen = map[engine.LinkID]func([]byte) int{} @@ -452,7 +452,7 @@ func buildDescriptor(stg *stage, comps *pipepb.Components, wk *worker.W, em *eng Local: stateID, } var fn func([]byte) int - switch v := coders[s.OrderedListSpec.GetElementCoderId()]; v.GetSpec().GetUrn() { + switch v := coders[s.GetOrderedListSpec().GetElementCoderId()]; v.GetSpec().GetUrn() { case urns.CoderBool: fn = func(_ []byte) int { return 1 @@ -472,19 +472,19 @@ func buildDescriptor(stg *stage, comps *pipepb.Components, wk *worker.W, em *eng return int(l) + n } default: - rewriteErr = fmt.Errorf("unknown coder used for ordered list state after re-write id: %v coder: %v, for state %v for transform %v in stage %v", s.OrderedListSpec.GetElementCoderId(), v, stateID, tid, stg.ID) + rewriteErr = fmt.Errorf("unknown coder used for ordered list state after re-write id: %v coder: %v, for state %v for transform %v in stage %v", s.GetOrderedListSpec().GetElementCoderId(), v, stateID, tid, stg.ID) } stg.stateTypeLen[linkID] = fn - case *pipepb.StateSpec_CombiningSpec: - rewriteCoder(&s.CombiningSpec.AccumulatorCoderId) - case *pipepb.StateSpec_MapSpec: - rewriteCoder(&s.MapSpec.KeyCoderId) - rewriteCoder(&s.MapSpec.ValueCoderId) - case *pipepb.StateSpec_MultimapSpec: - rewriteCoder(&s.MultimapSpec.KeyCoderId) - rewriteCoder(&s.MultimapSpec.ValueCoderId) - case *pipepb.StateSpec_ReadModifyWriteSpec: - rewriteCoder(&s.ReadModifyWriteSpec.CoderId) + case pipepb.StateSpec_CombiningSpec_case: + rewriteCoder(proto.String(s.GetCombiningSpec().GetAccumulatorCoderId())) + case pipepb.StateSpec_MapSpec_case: + rewriteCoder(proto.String(s.GetMapSpec().GetKeyCoderId())) + rewriteCoder(proto.String(s.GetMapSpec().GetValueCoderId())) + case pipepb.StateSpec_MultimapSpec_case: + rewriteCoder(proto.String(s.GetMultimapSpec().GetKeyCoderId())) + rewriteCoder(proto.String(s.GetMultimapSpec().GetValueCoderId())) + case pipepb.StateSpec_ReadModifyWriteSpec_case: + rewriteCoder(proto.String(s.GetReadModifyWriteSpec().GetCoderId())) } if rewriteErr != nil { return rewriteErr @@ -492,7 +492,7 @@ func buildDescriptor(stg *stage, comps *pipepb.Components, wk *worker.W, em *eng } for timerID, v := range pardo.GetTimerFamilySpecs() { stg.hasTimers = append(stg.hasTimers, engine.StaticTimerID{TransformID: tid, TimerFamily: timerID}) - if v.TimeDomain == pipepb.TimeDomain_PROCESSING_TIME { + if v.GetTimeDomain() == pipepb.TimeDomain_PROCESSING_TIME { if stg.processingTimeTimers == nil { stg.processingTimeTimers = map[string]bool{} } @@ -503,14 +503,14 @@ func buildDescriptor(stg *stage, comps *pipepb.Components, wk *worker.W, em *eng if err != nil { return fmt.Errorf("unable to rewrite coder %v for timer %v for transform %v in stage %v: %w", v.GetTimerFamilyCoderId(), timerID, tid, stg.ID, err) } - v.TimerFamilyCoderId = newCid + v.SetTimerFamilyCoderId(newCid) } if rewrite { pyld, err := proto.MarshalOptions{}.Marshal(pardo) if err != nil { return fmt.Errorf("unable to encode ParDoPayload for %v in stage %v after rewrite", tid, stg.ID) } - t.Spec.Payload = pyld + t.GetSpec().SetPayload(pyld) } } if len(transforms) == 0 { @@ -559,13 +559,13 @@ func buildDescriptor(stg *stage, comps *pipepb.Components, wk *worker.W, em *eng if oCID != nCID { // Add a synthetic PCollection set with the new coder. newGlobal := si.Global + "_prismside" - pcollections[newGlobal] = &pipepb.PCollection{ + pcollections[newGlobal] = pipepb.PCollection_builder{ DisplayData: col.GetDisplayData(), UniqueName: col.GetUniqueName(), CoderId: nCID, IsBounded: col.GetIsBounded(), - WindowingStrategyId: col.WindowingStrategyId, - } + WindowingStrategyId: col.GetWindowingStrategyId(), + }.Build() // Update side inputs to point to new PCollection with any replaced coders. transforms[si.Transform].GetInputs()[si.Local] = newGlobal // TODO: replace si.Global with newGlobal? @@ -585,7 +585,7 @@ func buildDescriptor(stg *stage, comps *pipepb.Components, wk *worker.W, em *eng col := clonePColToBundle(stg.primaryInput) if newCID, err := lpUnknownCoders(col.GetCoderId(), coders, comps.GetCoders()); err == nil && col.GetCoderId() != newCID { - col.CoderId = newCID + col.SetCoderId(newCID) } else if err != nil { return fmt.Errorf("buildDescriptor: couldn't rewrite coder %q for primary input pcollection %q: %w", col.GetCoderId(), stg.primaryInput, err) } @@ -618,7 +618,7 @@ func buildDescriptor(stg *stage, comps *pipepb.Components, wk *worker.W, em *eng for _, pid := range stg.internalCols { col := clonePColToBundle(pid) if newCID, err := lpUnknownCoders(col.GetCoderId(), coders, comps.GetCoders()); err == nil && col.GetCoderId() != newCID { - col.CoderId = newCID + col.SetCoderId(newCID) } else if err != nil { return fmt.Errorf("buildDescriptor: coder couldn't rewrite coder %q for internal pcollection %q: %w", col.GetCoderId(), pid, err) } @@ -633,22 +633,22 @@ func buildDescriptor(stg *stage, comps *pipepb.Components, wk *worker.W, em *eng var timerServiceDescriptor *pipepb.ApiServiceDescriptor if len(stg.hasTimers) > 0 { - timerServiceDescriptor = &pipepb.ApiServiceDescriptor{ + timerServiceDescriptor = pipepb.ApiServiceDescriptor_builder{ Url: wk.Endpoint(), - } + }.Build() } - desc := &fnpb.ProcessBundleDescriptor{ + desc := fnpb.ProcessBundleDescriptor_builder{ Id: stg.ID, Transforms: transforms, WindowingStrategies: comps.GetWindowingStrategies(), Pcollections: pcollections, Coders: coders, - StateApiServiceDescriptor: &pipepb.ApiServiceDescriptor{ + StateApiServiceDescriptor: pipepb.ApiServiceDescriptor_builder{ Url: wk.Endpoint(), - }, + }.Build(), TimerApiServiceDescriptor: timerServiceDescriptor, - } + }.Build() stg.desc = desc stg.prepareSides = func(b *worker.B, watermark mtime.Time) { @@ -747,29 +747,29 @@ func handleSideInput(link engine.LinkID, comps *pipepb.Components, transforms ma } func sourceTransform(parentID string, sourcePortBytes []byte, outPID string) *pipepb.PTransform { - source := &pipepb.PTransform{ + source := pipepb.PTransform_builder{ UniqueName: parentID, - Spec: &pipepb.FunctionSpec{ + Spec: pipepb.FunctionSpec_builder{ Urn: urns.TransformSource, Payload: sourcePortBytes, - }, + }.Build(), Outputs: map[string]string{ "i0": outPID, }, - } + }.Build() return source } func sinkTransform(sinkID string, sinkPortBytes []byte, inPID string) *pipepb.PTransform { - source := &pipepb.PTransform{ + source := pipepb.PTransform_builder{ UniqueName: sinkID, - Spec: &pipepb.FunctionSpec{ + Spec: pipepb.FunctionSpec_builder{ Urn: urns.TransformSink, Payload: sinkPortBytes, - }, + }.Build(), Inputs: map[string]string{ "i0": inPID, }, - } + }.Build() return source } diff --git a/sdks/go/pkg/beam/runners/prism/internal/web/web.go b/sdks/go/pkg/beam/runners/prism/internal/web/web.go index b14778e4462c..ec9ce7d8534d 100644 --- a/sdks/go/pkg/beam/runners/prism/internal/web/web.go +++ b/sdks/go/pkg/beam/runners/prism/internal/web/web.go @@ -149,17 +149,17 @@ func (h *jobDetailsHandler) ServeHTTP(w http.ResponseWriter, r *http.Request) { var metsResp *jobpb.GetJobMetricsResponse var stateResp *jobpb.JobStateEvent errg.Go(func() error { - resp, err := h.Jobcli.GetPipeline(ctx, &jobpb.GetJobPipelineRequest{JobId: jobID}) + resp, err := h.Jobcli.GetPipeline(ctx, jobpb.GetJobPipelineRequest_builder{JobId: jobID}.Build()) pipeResp = resp return err }) errg.Go(func() error { - resp, err := h.Jobcli.GetJobMetrics(ctx, &jobpb.GetJobMetricsRequest{JobId: jobID}) + resp, err := h.Jobcli.GetJobMetrics(ctx, jobpb.GetJobMetricsRequest_builder{JobId: jobID}.Build()) metsResp = resp return err }) errg.Go(func() error { - resp, err := h.Jobcli.GetState(ctx, &jobpb.GetJobStateRequest{JobId: jobID}) + resp, err := h.Jobcli.GetState(ctx, jobpb.GetJobStateRequest_builder{JobId: jobID}.Build()) stateResp = resp return err }) @@ -339,7 +339,7 @@ func (h *jobsConsoleHandler) ServeHTTP(w http.ResponseWriter, r *http.Request) { data.Jobs = resp.GetJobInfo() sort.Slice(data.Jobs, func(i, j int) bool { a, b := data.Jobs[i], data.Jobs[j] - return a.JobId < b.JobId + return a.GetJobId() < b.GetJobId() }) renderPage(indexPage, data, w) @@ -401,9 +401,9 @@ func (h *jobCancelHandler) ServeHTTP(w http.ResponseWriter, r *http.Request) { } // Forward JobId from POST body avoids direct json Unmarshall on composite types containing protobuf message types. - resp, err := h.Jobcli.Cancel(r.Context(), &jobpb.CancelJobRequest{ + resp, err := h.Jobcli.Cancel(r.Context(), jobpb.CancelJobRequest_builder{ JobId: cancelRequest.JobID, - }) + }.Build()) if err != nil { statusCode := status.Code(err) httpCode := http.StatusInternalServerError diff --git a/sdks/go/pkg/beam/runners/prism/internal/worker/bundle.go b/sdks/go/pkg/beam/runners/prism/internal/worker/bundle.go index 14cd84aef821..ffdaaf4057a4 100644 --- a/sdks/go/pkg/beam/runners/prism/internal/worker/bundle.go +++ b/sdks/go/pkg/beam/runners/prism/internal/worker/bundle.go @@ -126,14 +126,12 @@ func (b *B) ProcessOn(ctx context.Context, wk *W) <-chan struct{} { slog.Debug("processing", "bundle", b, "worker", wk) // Tell the SDK to start processing the bundle. - req := &fnpb.InstructionRequest{ + req := fnpb.InstructionRequest_builder{ InstructionId: b.InstID, - Request: &fnpb.InstructionRequest_ProcessBundle{ - ProcessBundle: &fnpb.ProcessBundleRequest{ - ProcessBundleDescriptorId: b.PBDID, - }, - }, - } + ProcessBundle: fnpb.ProcessBundleRequest_builder{ + ProcessBundleDescriptorId: b.PBDID, + }.Build(), + }.Build() select { case <-wk.StoppedChan: // The worker was stopped before req was sent. @@ -154,22 +152,22 @@ func (b *B) ProcessOn(ctx context.Context, wk *W) <-chan struct{} { dataBuf := bytes.Join(block.Bytes, []byte{}) switch block.Kind { case engine.BlockData: - elms.Data = []*fnpb.Elements_Data{ - { + elms.SetData([]*fnpb.Elements_Data{ + fnpb.Elements_Data_builder{ InstructionId: b.InstID, TransformId: b.InputTransformID, Data: dataBuf, - }, - } + }.Build(), + }) case engine.BlockTimer: - elms.Timers = []*fnpb.Elements_Timers{ - { + elms.SetTimers([]*fnpb.Elements_Timers{ + fnpb.Elements_Timers_builder{ InstructionId: b.InstID, TransformId: block.Transform, TimerFamilyId: block.Family, Timers: dataBuf, - }, - } + }.Build(), + }) default: panic("unknown engine.Block kind") } @@ -188,12 +186,12 @@ func (b *B) ProcessOn(ctx context.Context, wk *W) <-chan struct{} { // Send last of everything for now. timers := make([]*fnpb.Elements_Timers, 0, len(b.HasTimers)) for _, tid := range b.HasTimers { - timers = append(timers, &fnpb.Elements_Timers{ + timers = append(timers, fnpb.Elements_Timers_builder{ InstructionId: b.InstID, TransformId: tid.TransformID, TimerFamilyId: tid.TimerFamily, IsLast: true, - }) + }.Build()) } select { case <-wk.StoppedChan: @@ -202,16 +200,16 @@ func (b *B) ProcessOn(ctx context.Context, wk *W) <-chan struct{} { case <-ctx.Done(): b.DataOrTimerDone() return b.DataWait - case wk.DataReqs <- &fnpb.Elements{ + case wk.DataReqs <- fnpb.Elements_builder{ Timers: timers, Data: []*fnpb.Elements_Data{ - { + fnpb.Elements_Data_builder{ InstructionId: b.InstID, TransformId: b.InputTransformID, IsLast: true, - }, + }.Build(), }, - }: + }.Build(): } return b.DataWait @@ -225,25 +223,21 @@ func (b *B) Cleanup(wk *W) { } func (b *B) Finalize(ctx context.Context, wk *W) (*fnpb.FinalizeBundleResponse, error) { - resp := wk.sendInstruction(ctx, &fnpb.InstructionRequest{ - Request: &fnpb.InstructionRequest_FinalizeBundle{ - FinalizeBundle: &fnpb.FinalizeBundleRequest{ - InstructionId: b.InstID, - }, - }, - }) + resp := wk.sendInstruction(ctx, fnpb.InstructionRequest_builder{ + FinalizeBundle: fnpb.FinalizeBundleRequest_builder{ + InstructionId: b.InstID, + }.Build(), + }.Build()) return resp.GetFinalizeBundle(), nil } // Progress sends a progress request for the given bundle to the passed in worker, blocking on the response. func (b *B) Progress(ctx context.Context, wk *W) (*fnpb.ProcessBundleProgressResponse, error) { - resp := wk.sendInstruction(ctx, &fnpb.InstructionRequest{ - Request: &fnpb.InstructionRequest_ProcessBundleProgress{ - ProcessBundleProgress: &fnpb.ProcessBundleProgressRequest{ - InstructionId: b.InstID, - }, - }, - }) + resp := wk.sendInstruction(ctx, fnpb.InstructionRequest_builder{ + ProcessBundleProgress: fnpb.ProcessBundleProgressRequest_builder{ + InstructionId: b.InstID, + }.Build(), + }.Build()) if resp.GetError() != "" { return nil, fmt.Errorf("progress[%v] error from SDK: %v", b.InstID, resp.GetError()) } @@ -252,20 +246,18 @@ func (b *B) Progress(ctx context.Context, wk *W) (*fnpb.ProcessBundleProgressRes // Split sends a split request for the given bundle to the passed in worker, blocking on the response. func (b *B) Split(ctx context.Context, wk *W, fraction float64, allowedSplits []int64) (*fnpb.ProcessBundleSplitResponse, error) { - resp := wk.sendInstruction(ctx, &fnpb.InstructionRequest{ - Request: &fnpb.InstructionRequest_ProcessBundleSplit{ - ProcessBundleSplit: &fnpb.ProcessBundleSplitRequest{ - InstructionId: b.InstID, - DesiredSplits: map[string]*fnpb.ProcessBundleSplitRequest_DesiredSplit{ - b.InputTransformID: { - FractionOfRemainder: fraction, - AllowedSplitPoints: allowedSplits, - EstimatedInputElements: int64(b.EstimatedInputElements), - }, - }, + resp := wk.sendInstruction(ctx, fnpb.InstructionRequest_builder{ + ProcessBundleSplit: fnpb.ProcessBundleSplitRequest_builder{ + InstructionId: b.InstID, + DesiredSplits: map[string]*fnpb.ProcessBundleSplitRequest_DesiredSplit{ + b.InputTransformID: fnpb.ProcessBundleSplitRequest_DesiredSplit_builder{ + FractionOfRemainder: fraction, + AllowedSplitPoints: allowedSplits, + EstimatedInputElements: int64(b.EstimatedInputElements), + }.Build(), }, - }, - }) + }.Build(), + }.Build()) if resp.GetError() != "" { return nil, fmt.Errorf("split[%v] error from SDK: %v", b.InstID, resp.GetError()) } diff --git a/sdks/go/pkg/beam/runners/prism/internal/worker/worker.go b/sdks/go/pkg/beam/runners/prism/internal/worker/worker.go index 9d9058975b26..6b62e060001f 100644 --- a/sdks/go/pkg/beam/runners/prism/internal/worker/worker.go +++ b/sdks/go/pkg/beam/runners/prism/internal/worker/worker.go @@ -175,20 +175,20 @@ func (wk *W) NextInst() string { var minsev = fnpb.LogEntry_Severity_DEBUG func (wk *W) GetProvisionInfo(_ context.Context, _ *fnpb.GetProvisionInfoRequest) (*fnpb.GetProvisionInfoResponse, error) { - endpoint := &pipepb.ApiServiceDescriptor{ + endpoint := pipepb.ApiServiceDescriptor_builder{ Url: wk.Endpoint(), - } - resp := &fnpb.GetProvisionInfoResponse{ - Info: &fnpb.ProvisionInfo{ + }.Build() + resp := fnpb.GetProvisionInfoResponse_builder{ + Info: fnpb.ProvisionInfo_builder{ // TODO: Include runner capabilities with the per job configuration. RunnerCapabilities: []string{ urns.CapabilityMonitoringInfoShortIDs, }, LoggingEndpoint: endpoint, ControlEndpoint: endpoint, - ArtifactEndpoint: &pipepb.ApiServiceDescriptor{ + ArtifactEndpoint: pipepb.ApiServiceDescriptor_builder{ Url: wk.ArtifactEndpoint, - }, + }.Build(), RetrievalToken: wk.JobKey, Dependencies: wk.EnvPb.GetDependencies(), @@ -199,8 +199,8 @@ func (wk *W) GetProvisionInfo(_ context.Context, _ *fnpb.GetProvisionInfoRequest "runner_version": core.SdkVersion, "variant": "test", }, - }, - } + }.Build(), + }.Build() return resp, nil } @@ -223,7 +223,7 @@ func (wk *W) Logging(stream fnpb.BeamFnLogging_LoggingServer) error { } for _, l := range in.GetLogEntries() { // TODO base this on a per pipeline logging setting. - if l.Severity < minsev { + if l.GetSeverity() < minsev { continue } @@ -352,10 +352,10 @@ func (wk *W) Control(ctrl fnpb.BeamFnControl_ControlServer) error { msg := fmt.Sprintf("SDK worker disconnected: %v, %v active instructions, error: %v", wk.String(), len(wk.activeInstructions), err) for instID, b := range wk.activeInstructions { - b.Respond(&fnpb.InstructionResponse{ + b.Respond(fnpb.InstructionResponse_builder{ InstructionId: instID, Error: msg, - }) + }.Build()) } // Soft shutdown to prevent GRPC shutdown from being blocked by this // streaming call. @@ -481,15 +481,15 @@ func (wk *W) State(state fnpb.BeamFnState_StateServer) error { slog.Warn("state request after bundle inactive", "instruction", req.GetInstructionId(), "worker", wk) continue } - switch req.GetRequest().(type) { - case *fnpb.StateRequest_Get: + switch req.WhichRequest() { + case fnpb.StateRequest_Get_case: // TODO: move data handling to be pcollection based. key := req.GetStateKey() slog.Debug("StateRequest_Get", "request", prototext.Format(req), "bundle", b) var data [][]byte - switch key.GetType().(type) { - case *fnpb.StateKey_IterableSideInput_: + switch key.WhichType() { + case fnpb.StateKey_IterableSideInput_case: ikey := key.GetIterableSideInput() wKey := ikey.GetWindow() var w typex.Window @@ -511,7 +511,7 @@ func (wk *W) State(state fnpb.BeamFnState_StateServer) error { data = winMap[w] - case *fnpb.StateKey_MultimapKeysSideInput_: + case fnpb.StateKey_MultimapKeysSideInput_case: mmkey := key.GetMultimapKeysSideInput() wKey := mmkey.GetWindow() var w typex.Window = window.GlobalWindow{} @@ -526,7 +526,7 @@ func (wk *W) State(state fnpb.BeamFnState_StateServer) error { data = append(data, []byte(k)) } - case *fnpb.StateKey_MultimapSideInput_: + case fnpb.StateKey_MultimapSideInput_case: mmkey := key.GetMultimapSideInput() wKey := mmkey.GetWindow() var w typex.Window @@ -545,16 +545,16 @@ func (wk *W) State(state fnpb.BeamFnState_StateServer) error { data = winMap[w][string(dKey)] - case *fnpb.StateKey_BagUserState_: + case fnpb.StateKey_BagUserState_case: bagkey := key.GetBagUserState() data = b.OutputData.GetBagState(engine.LinkID{Transform: bagkey.GetTransformId(), Local: bagkey.GetUserStateId()}, bagkey.GetWindow(), bagkey.GetKey()) - case *fnpb.StateKey_MultimapUserState_: + case fnpb.StateKey_MultimapUserState_case: mmkey := key.GetMultimapUserState() data = b.OutputData.GetMultimapState(engine.LinkID{Transform: mmkey.GetTransformId(), Local: mmkey.GetUserStateId()}, mmkey.GetWindow(), mmkey.GetKey(), mmkey.GetMapKey()) - case *fnpb.StateKey_MultimapKeysUserState_: + case fnpb.StateKey_MultimapKeysUserState_case: mmkey := key.GetMultimapKeysUserState() data = b.OutputData.GetMultimapKeysState(engine.LinkID{Transform: mmkey.GetTransformId(), Local: mmkey.GetUserStateId()}, mmkey.GetWindow(), mmkey.GetKey()) - case *fnpb.StateKey_OrderedListUserState_: + case fnpb.StateKey_OrderedListUserState_case: olkey := key.GetOrderedListUserState() data = b.OutputData.GetOrderedListState( engine.LinkID{Transform: olkey.GetTransformId(), Local: olkey.GetUserStateId()}, @@ -565,25 +565,23 @@ func (wk *W) State(state fnpb.BeamFnState_StateServer) error { // Encode the runner iterable (no length, just consecutive elements), and send it out. // This is also where we can handle things like State Backed Iterables. - responses <- &fnpb.StateResponse{ + responses <- fnpb.StateResponse_builder{ Id: req.GetId(), - Response: &fnpb.StateResponse_Get{ - Get: &fnpb.StateGetResponse{ - Data: bytes.Join(data, []byte{}), - }, - }, - } + Get: fnpb.StateGetResponse_builder{ + Data: bytes.Join(data, []byte{}), + }.Build(), + }.Build() - case *fnpb.StateRequest_Append: + case fnpb.StateRequest_Append_case: key := req.GetStateKey() - switch key.GetType().(type) { - case *fnpb.StateKey_BagUserState_: + switch key.WhichType() { + case fnpb.StateKey_BagUserState_case: bagkey := key.GetBagUserState() b.OutputData.AppendBagState(engine.LinkID{Transform: bagkey.GetTransformId(), Local: bagkey.GetUserStateId()}, bagkey.GetWindow(), bagkey.GetKey(), req.GetAppend().GetData()) - case *fnpb.StateKey_MultimapUserState_: + case fnpb.StateKey_MultimapUserState_case: mmkey := key.GetMultimapUserState() b.OutputData.AppendMultimapState(engine.LinkID{Transform: mmkey.GetTransformId(), Local: mmkey.GetUserStateId()}, mmkey.GetWindow(), mmkey.GetKey(), mmkey.GetMapKey(), req.GetAppend().GetData()) - case *fnpb.StateKey_OrderedListUserState_: + case fnpb.StateKey_OrderedListUserState_case: olkey := key.GetOrderedListUserState() b.OutputData.AppendOrderedListState( engine.LinkID{Transform: olkey.GetTransformId(), Local: olkey.GetUserStateId()}, @@ -592,38 +590,34 @@ func (wk *W) State(state fnpb.BeamFnState_StateServer) error { panic(fmt.Sprintf("unsupported StateKey Append type: %T: %v", key.GetType(), prototext.Format(key))) } - responses <- &fnpb.StateResponse{ - Id: req.GetId(), - Response: &fnpb.StateResponse_Append{ - Append: &fnpb.StateAppendResponse{}, - }, - } + responses <- fnpb.StateResponse_builder{ + Id: req.GetId(), + Append: &fnpb.StateAppendResponse{}, + }.Build() - case *fnpb.StateRequest_Clear: + case fnpb.StateRequest_Clear_case: key := req.GetStateKey() - switch key.GetType().(type) { - case *fnpb.StateKey_BagUserState_: + switch key.WhichType() { + case fnpb.StateKey_BagUserState_case: bagkey := key.GetBagUserState() b.OutputData.ClearBagState(engine.LinkID{Transform: bagkey.GetTransformId(), Local: bagkey.GetUserStateId()}, bagkey.GetWindow(), bagkey.GetKey()) - case *fnpb.StateKey_MultimapUserState_: + case fnpb.StateKey_MultimapUserState_case: mmkey := key.GetMultimapUserState() b.OutputData.ClearMultimapState(engine.LinkID{Transform: mmkey.GetTransformId(), Local: mmkey.GetUserStateId()}, mmkey.GetWindow(), mmkey.GetKey(), mmkey.GetMapKey()) - case *fnpb.StateKey_MultimapKeysUserState_: + case fnpb.StateKey_MultimapKeysUserState_case: mmkey := key.GetMultimapUserState() b.OutputData.ClearMultimapKeysState(engine.LinkID{Transform: mmkey.GetTransformId(), Local: mmkey.GetUserStateId()}, mmkey.GetWindow(), mmkey.GetKey()) - case *fnpb.StateKey_OrderedListUserState_: + case fnpb.StateKey_OrderedListUserState_case: olkey := key.GetOrderedListUserState() b.OutputData.ClearOrderedListState(engine.LinkID{Transform: olkey.GetTransformId(), Local: olkey.GetUserStateId()}, olkey.GetWindow(), olkey.GetKey(), olkey.GetRange().GetStart(), olkey.GetRange().GetEnd()) default: panic(fmt.Sprintf("unsupported StateKey Clear type: %T: %v", key.GetType(), prototext.Format(key))) } - responses <- &fnpb.StateResponse{ - Id: req.GetId(), - Response: &fnpb.StateResponse_Clear{ - Clear: &fnpb.StateClearResponse{}, - }, - } + responses <- fnpb.StateResponse_builder{ + Id: req.GetId(), + Clear: &fnpb.StateClearResponse{}, + }.Build() default: panic(fmt.Sprintf("unsupported StateRequest kind %T: %v", req.GetRequest(), prototext.Format(req))) @@ -668,32 +662,32 @@ func (wk *W) sendInstruction(ctx context.Context, req *fnpb.InstructionRequest) chanResponderPool.Put(cr) }() - req.InstructionId = progInst + req.SetInstructionId(progInst) if wk.Stopped() { return nil } select { case <-wk.StoppedChan: - return &fnpb.InstructionResponse{ + return fnpb.InstructionResponse_builder{ InstructionId: progInst, Error: "worker stopped before send", - } + }.Build() case wk.InstReqs <- req: // desired outcome } select { case <-wk.StoppedChan: - return &fnpb.InstructionResponse{ + return fnpb.InstructionResponse_builder{ InstructionId: progInst, Error: "worker stopped before receive", - } + }.Build() case <-ctx.Done(): - return &fnpb.InstructionResponse{ + return fnpb.InstructionResponse_builder{ InstructionId: progInst, Error: "context canceled before receive", - } + }.Build() case resp := <-cr.Resp: // Protos are safe as nil, so just return directly. return resp @@ -702,11 +696,9 @@ func (wk *W) sendInstruction(ctx context.Context, req *fnpb.InstructionRequest) // MonitoringMetadata is a convenience method to request the metadata for monitoring shortIDs. func (wk *W) MonitoringMetadata(ctx context.Context, unknownIDs []string) *fnpb.MonitoringInfosMetadataResponse { - return wk.sendInstruction(ctx, &fnpb.InstructionRequest{ - Request: &fnpb.InstructionRequest_MonitoringInfos{ - MonitoringInfos: &fnpb.MonitoringInfosMetadataRequest{ - MonitoringInfoId: unknownIDs, - }, - }, - }).GetMonitoringInfos() + return wk.sendInstruction(ctx, fnpb.InstructionRequest_builder{ + MonitoringInfos: fnpb.MonitoringInfosMetadataRequest_builder{ + MonitoringInfoId: unknownIDs, + }.Build(), + }.Build()).GetMonitoringInfos() } diff --git a/sdks/go/pkg/beam/runners/prism/internal/worker/worker_test.go b/sdks/go/pkg/beam/runners/prism/internal/worker/worker_test.go index 469e0e2f3d83..66422508a55b 100644 --- a/sdks/go/pkg/beam/runners/prism/internal/worker/worker_test.go +++ b/sdks/go/pkg/beam/runners/prism/internal/worker/worker_test.go @@ -60,13 +60,13 @@ func TestWorker_GetProcessBundleDescriptor(t *testing.T) { w := New("test", "testEnv") id := "available" - w.Descriptors[id] = &fnpb.ProcessBundleDescriptor{ + w.Descriptors[id] = fnpb.ProcessBundleDescriptor_builder{ Id: id, - } + }.Build() - pbd, err := w.GetProcessBundleDescriptor(context.Background(), &fnpb.GetProcessBundleDescriptorRequest{ + pbd, err := w.GetProcessBundleDescriptor(context.Background(), fnpb.GetProcessBundleDescriptorRequest_builder{ ProcessBundleDescriptorId: id, - }) + }.Build()) if err != nil { t.Errorf("got GetProcessBundleDescriptor(%q) error: %v, want nil", id, err) } @@ -74,9 +74,9 @@ func TestWorker_GetProcessBundleDescriptor(t *testing.T) { t.Errorf("got GetProcessBundleDescriptor(%q) = %v, want id %v", id, got, want) } - pbd, err = w.GetProcessBundleDescriptor(context.Background(), &fnpb.GetProcessBundleDescriptorRequest{ + pbd, err = w.GetProcessBundleDescriptor(context.Background(), fnpb.GetProcessBundleDescriptorRequest_builder{ ProcessBundleDescriptorId: "unknown", - }) + }.Build()) if err == nil { t.Errorf("got GetProcessBundleDescriptor(%q) = %v, want error", "unknown", pbd) } @@ -128,21 +128,21 @@ func TestWorker_Logging(t *testing.T) { t.Fatal("couldn't create log client:", err) } - logStream.Send(&fnpb.LogEntry_List{ - LogEntries: []*fnpb.LogEntry{{ + logStream.Send(fnpb.LogEntry_List_builder{ + LogEntries: []*fnpb.LogEntry{fnpb.LogEntry_builder{ Severity: fnpb.LogEntry_Severity_INFO, Message: "squeamish ossiphrage", LogLocation: "intentionally.go:124", - }}, - }) + }.Build()}, + }.Build()) - logStream.Send(&fnpb.LogEntry_List{ - LogEntries: []*fnpb.LogEntry{{ + logStream.Send(fnpb.LogEntry_List_builder{ + LogEntries: []*fnpb.LogEntry{fnpb.LogEntry_builder{ Severity: fnpb.LogEntry_Severity_INFO, Message: "squeamish ossiphrage the second", LogLocation: "intentionally bad log location", - }}, - }) + }.Build()}, + }.Build()) // TODO: Connect to the job management service. // At this point job messages are just logged to wherever the prism runner executes @@ -168,21 +168,19 @@ func TestWorker_Control_HappyPath(t *testing.T) { wk.activeInstructions[instID] = b b.ProcessOn(ctx, wk) - ctrlStream.Send(&fnpb.InstructionResponse{ + ctrlStream.Send(fnpb.InstructionResponse_builder{ InstructionId: instID, - Response: &fnpb.InstructionResponse_ProcessBundle{ - ProcessBundle: &fnpb.ProcessBundleResponse{ - RequiresFinalization: true, // Simple thing to check. - }, - }, - }) + ProcessBundle: fnpb.ProcessBundleResponse_builder{ + RequiresFinalization: true, // Simple thing to check. + }.Build(), + }.Build()) if err := ctrlStream.CloseSend(); err != nil { t.Errorf("ctrlStream.CloseSend() = %v", err) } resp := <-b.Resp - if !resp.RequiresFinalization { + if !resp.GetRequiresFinalization() { t.Errorf("got %v, want response that Requires Finalization", resp) } } @@ -218,9 +216,9 @@ func TestWorker_Data_HappyPath(t *testing.T) { b.ProcessOn(ctx, wk) }() - wk.InstReqs <- &fnpb.InstructionRequest{ + wk.InstReqs <- fnpb.InstructionRequest_builder{ InstructionId: instID, - } + }.Build() elements, err := dataStream.Recv() if err != nil { @@ -281,20 +279,16 @@ func TestWorker_State_Iterable(t *testing.T) { }, } - stateStream.Send(&fnpb.StateRequest{ + stateStream.Send(fnpb.StateRequest_builder{ Id: "first", InstructionId: instID, - Request: &fnpb.StateRequest_Get{ - Get: &fnpb.StateGetRequest{}, - }, - StateKey: &fnpb.StateKey{Type: &fnpb.StateKey_IterableSideInput_{ - IterableSideInput: &fnpb.StateKey_IterableSideInput{ - TransformId: "transformID", - SideInputId: "i1", - Window: []byte{}, // Global Windows - }, - }}, - }) + Get: &fnpb.StateGetRequest{}, + StateKey: fnpb.StateKey_builder{IterableSideInput: fnpb.StateKey_IterableSideInput_builder{ + TransformId: "transformID", + SideInputId: "i1", + Window: []byte{}, // Global Windows + }.Build()}.Build(), + }.Build()) resp, err := stateStream.Recv() if err != nil { @@ -354,20 +348,16 @@ func TestWorker_State_MultimapKeysSideInput(t *testing.T) { }, } - stateStream.Send(&fnpb.StateRequest{ + stateStream.Send(fnpb.StateRequest_builder{ Id: "first", InstructionId: instID, - Request: &fnpb.StateRequest_Get{ - Get: &fnpb.StateGetRequest{}, - }, - StateKey: &fnpb.StateKey{Type: &fnpb.StateKey_MultimapKeysSideInput_{ - MultimapKeysSideInput: &fnpb.StateKey_MultimapKeysSideInput{ - TransformId: "transformID", - SideInputId: "i1", - Window: encW, - }, - }}, - }) + Get: &fnpb.StateGetRequest{}, + StateKey: fnpb.StateKey_builder{MultimapKeysSideInput: fnpb.StateKey_MultimapKeysSideInput_builder{ + TransformId: "transformID", + SideInputId: "i1", + Window: encW, + }.Build()}.Build(), + }.Build()) resp, err := stateStream.Recv() if err != nil { @@ -433,21 +423,17 @@ func TestWorker_State_MultimapSideInput(t *testing.T) { "b": {12}, } for _, key := range testKey { - stateStream.Send(&fnpb.StateRequest{ + stateStream.Send(fnpb.StateRequest_builder{ Id: "first", InstructionId: instID, - Request: &fnpb.StateRequest_Get{ - Get: &fnpb.StateGetRequest{}, - }, - StateKey: &fnpb.StateKey{Type: &fnpb.StateKey_MultimapSideInput_{ - MultimapSideInput: &fnpb.StateKey_MultimapSideInput{ - TransformId: "transformID", - SideInputId: "i1", - Window: encW, - Key: []byte(key), - }, - }}, - }) + Get: &fnpb.StateGetRequest{}, + StateKey: fnpb.StateKey_builder{MultimapSideInput: fnpb.StateKey_MultimapSideInput_builder{ + TransformId: "transformID", + SideInputId: "i1", + Window: encW, + Key: []byte(key), + }.Build()}.Build(), + }.Build()) resp, err := stateStream.Recv() if err != nil { diff --git a/sdks/go/pkg/beam/runners/universal/extworker/extworker.go b/sdks/go/pkg/beam/runners/universal/extworker/extworker.go index a7fc308d2193..4ebd025fc1b0 100644 --- a/sdks/go/pkg/beam/runners/universal/extworker/extworker.go +++ b/sdks/go/pkg/beam/runners/universal/extworker/extworker.go @@ -67,24 +67,24 @@ func (s *Loopback) StartWorker(ctx context.Context, req *fnpb.StartWorkerRequest s.mu.Lock() defer s.mu.Unlock() if s.workers == nil { - return &fnpb.StartWorkerResponse{ + return fnpb.StartWorkerResponse_builder{ Error: "worker pool shutting down", - }, nil + }.Build(), nil } if _, ok := s.workers[req.GetWorkerId()]; ok { - return &fnpb.StartWorkerResponse{ + return fnpb.StartWorkerResponse_builder{ Error: fmt.Sprintf("worker with ID %q already exists", req.GetWorkerId()), - }, nil + }.Build(), nil } if req.GetLoggingEndpoint() == nil { - return &fnpb.StartWorkerResponse{Error: fmt.Sprintf("Missing logging endpoint for worker %v", req.GetWorkerId())}, nil + return fnpb.StartWorkerResponse_builder{Error: fmt.Sprintf("Missing logging endpoint for worker %v", req.GetWorkerId())}.Build(), nil } if req.GetControlEndpoint() == nil { - return &fnpb.StartWorkerResponse{Error: fmt.Sprintf("Missing control endpoint for worker %v", req.GetWorkerId())}, nil + return fnpb.StartWorkerResponse_builder{Error: fmt.Sprintf("Missing control endpoint for worker %v", req.GetWorkerId())}.Build(), nil } - if req.GetLoggingEndpoint().Authentication != nil || req.GetControlEndpoint().Authentication != nil { - return &fnpb.StartWorkerResponse{Error: "[BEAM-10610] Secure endpoints not supported."}, nil + if req.GetLoggingEndpoint().HasAuthentication() || req.GetControlEndpoint().HasAuthentication() { + return fnpb.StartWorkerResponse_builder{Error: "[BEAM-10610] Secure endpoints not supported."}.Build(), nil } ctx = grpcx.WriteWorkerID(s.root, req.GetWorkerId()) @@ -126,9 +126,9 @@ func (s *Loopback) StopWorker(ctx context.Context, req *fnpb.StopWorkerRequest) delete(s.workers, req.GetWorkerId()) return &fnpb.StopWorkerResponse{}, nil } - return &fnpb.StopWorkerResponse{ + return fnpb.StopWorkerResponse_builder{ Error: fmt.Sprintf("no worker with id %q running", req.GetWorkerId()), - }, nil + }.Build(), nil } diff --git a/sdks/go/pkg/beam/runners/universal/extworker/extworker_test.go b/sdks/go/pkg/beam/runners/universal/extworker/extworker_test.go index e7444570b9c6..3add99f91f98 100644 --- a/sdks/go/pkg/beam/runners/universal/extworker/extworker_test.go +++ b/sdks/go/pkg/beam/runners/universal/extworker/extworker_test.go @@ -24,15 +24,15 @@ import ( ) func TestLoopback(t *testing.T) { - endpoint := &pipepb.ApiServiceDescriptor{ + endpoint := pipepb.ApiServiceDescriptor_builder{ Url: "localhost:0", - } - secureEndpoint := &pipepb.ApiServiceDescriptor{ + }.Build() + secureEndpoint := pipepb.ApiServiceDescriptor_builder{ Url: "localhost:0", - Authentication: &pipepb.AuthenticationSpec{ + Authentication: pipepb.AuthenticationSpec_builder{ Urn: "beam:authentication:oauth2_client_credentials_grant:v1", - }, - } + }.Build(), + }.Build() ctx := context.Background() server, err := StartLoopback(ctx, 0) @@ -45,60 +45,60 @@ func TestLoopback(t *testing.T) { errExpected bool }{ { - req: &fnpb.StartWorkerRequest{ + req: fnpb.StartWorkerRequest_builder{ WorkerId: "Worker1", ControlEndpoint: endpoint, LoggingEndpoint: endpoint, - }, + }.Build(), }, { - req: &fnpb.StartWorkerRequest{ + req: fnpb.StartWorkerRequest_builder{ WorkerId: "Worker2", ControlEndpoint: endpoint, LoggingEndpoint: endpoint, - }, + }.Build(), }, { - req: &fnpb.StartWorkerRequest{ + req: fnpb.StartWorkerRequest_builder{ WorkerId: "Worker1", ControlEndpoint: endpoint, LoggingEndpoint: endpoint, - }, + }.Build(), errExpected: true, // Repeated start }, { - req: &fnpb.StartWorkerRequest{ + req: fnpb.StartWorkerRequest_builder{ WorkerId: "missingControl", LoggingEndpoint: endpoint, - }, + }.Build(), errExpected: true, }, { - req: &fnpb.StartWorkerRequest{ + req: fnpb.StartWorkerRequest_builder{ WorkerId: "missingLogging", ControlEndpoint: endpoint, - }, + }.Build(), errExpected: true, }, { - req: &fnpb.StartWorkerRequest{ + req: fnpb.StartWorkerRequest_builder{ WorkerId: "secureLogging", LoggingEndpoint: secureEndpoint, ControlEndpoint: endpoint, - }, + }.Build(), errExpected: true, }, { - req: &fnpb.StartWorkerRequest{ + req: fnpb.StartWorkerRequest_builder{ WorkerId: "secureControl", LoggingEndpoint: endpoint, ControlEndpoint: secureEndpoint, - }, + }.Build(), errExpected: true, }, } for _, test := range startTests { resp, err := server.StartWorker(ctx, test.req) if test.errExpected { - if err != nil || resp.Error == "" { + if err != nil || resp.GetError() == "" { t.Errorf("Expected error starting %v: err: %v, resp: %v", test.req.GetWorkerId(), err, resp) } } else { - if err != nil || resp.Error != "" { + if err != nil || resp.GetError() != "" { t.Errorf("Unexpected error starting %v: err: %v, resp: %v", test.req.GetWorkerId(), err, resp) } } @@ -108,29 +108,29 @@ func TestLoopback(t *testing.T) { errExpected bool }{ { - req: &fnpb.StopWorkerRequest{ + req: fnpb.StopWorkerRequest_builder{ WorkerId: "Worker1", - }, + }.Build(), }, { - req: &fnpb.StopWorkerRequest{ + req: fnpb.StopWorkerRequest_builder{ WorkerId: "Worker1", - }, + }.Build(), errExpected: true, }, { - req: &fnpb.StopWorkerRequest{ + req: fnpb.StopWorkerRequest_builder{ WorkerId: "NonExistent", - }, + }.Build(), errExpected: true, }, } for _, test := range stopTests { resp, err := server.StopWorker(ctx, test.req) if test.errExpected { - if err != nil || resp.Error == "" { + if err != nil || resp.GetError() == "" { t.Errorf("Expected error starting %v: err: %v, resp: %v", test.req.GetWorkerId(), err, resp) } } else { - if err != nil || resp.Error != "" { + if err != nil || resp.GetError() != "" { t.Errorf("Unexpected error starting %v: err: %v, resp: %v", test.req.GetWorkerId(), err, resp) } } diff --git a/sdks/go/pkg/beam/runners/universal/runnerlib/execute.go b/sdks/go/pkg/beam/runners/universal/runnerlib/execute.go index eb854dbfcdba..51962d4f23b3 100644 --- a/sdks/go/pkg/beam/runners/universal/runnerlib/execute.go +++ b/sdks/go/pkg/beam/runners/universal/runnerlib/execute.go @@ -133,10 +133,10 @@ func UpdateGoEnvironmentWorker(worker string, p *pipepb.Pipeline) error { return errors.WithContextf(err, "unable to read worker binary %v, only read %d bytes", worker, n) } hash := hex.EncodeToString(sha256W.Sum(nil)) - pyld := protox.MustEncode(&pipepb.ArtifactFilePayload{ + pyld := protox.MustEncode(pipepb.ArtifactFilePayload_builder{ Path: worker, Sha256: hash, - }) + }.Build()) if err := graphx.UpdateDefaultEnvWorkerType(graphx.URNArtifactFileType, pyld, p); err != nil { return err } @@ -149,14 +149,14 @@ type universalPipelineResult struct { } func newUniversalPipelineResult(ctx context.Context, jobID string, client jobpb.JobServiceClient, p *pipepb.Pipeline) (*universalPipelineResult, error) { - request := &jobpb.GetJobMetricsRequest{JobId: jobID} + request := jobpb.GetJobMetricsRequest_builder{JobId: jobID}.Build() response, err := client.GetJobMetrics(ctx, request) if err != nil { return &universalPipelineResult{jobID, nil}, errors.Wrap(err, "failed to get metrics") } monitoredStates := response.GetMetrics() - metrics := metricsx.FromMonitoringInfos(p, monitoredStates.Attempted, monitoredStates.Committed) + metrics := metricsx.FromMonitoringInfos(p, monitoredStates.GetAttempted(), monitoredStates.GetCommitted()) return &universalPipelineResult{jobID, metrics}, nil } diff --git a/sdks/go/pkg/beam/runners/universal/runnerlib/job.go b/sdks/go/pkg/beam/runners/universal/runnerlib/job.go index 7d6a3027e47e..b7468ca84897 100644 --- a/sdks/go/pkg/beam/runners/universal/runnerlib/job.go +++ b/sdks/go/pkg/beam/runners/universal/runnerlib/job.go @@ -68,11 +68,11 @@ func Prepare(ctx context.Context, client jobpb.JobServiceClient, p *pipepb.Pipel if err != nil { return "", "", "", errors.WithContext(err, "producing pipeline options") } - req := &jobpb.PrepareJobRequest{ + req := jobpb.PrepareJobRequest_builder{ Pipeline: p, PipelineOptions: options, JobName: opt.Name, - } + }.Build() resp, err := client.Prepare(ctx, req) if err != nil { return "", "", "", errors.Wrap(err, "job failed to prepare") @@ -82,10 +82,10 @@ func Prepare(ctx context.Context, client jobpb.JobServiceClient, p *pipepb.Pipel // Submit submits a job to the given job service. It returns a jobID, if successful. func Submit(ctx context.Context, client jobpb.JobServiceClient, id, token string) (string, error) { - req := &jobpb.RunJobRequest{ + req := jobpb.RunJobRequest_builder{ PreparationId: id, RetrievalToken: token, - } + }.Build() resp, err := client.Run(ctx, req) if err != nil { @@ -97,7 +97,7 @@ func Submit(ctx context.Context, client jobpb.JobServiceClient, id, token string // WaitForCompletion monitors the given job until completion. It logs any messages // and state changes received. func WaitForCompletion(ctx context.Context, client jobpb.JobServiceClient, jobID string) error { - stream, err := client.GetMessageStream(ctx, &jobpb.JobMessagesRequest{JobId: jobID}) + stream, err := client.GetMessageStream(ctx, jobpb.JobMessagesRequest_builder{JobId: jobID}.Build()) if err != nil { return errors.Wrap(err, "failed to get job stream") } @@ -124,7 +124,7 @@ func WaitForCompletion(ctx context.Context, client jobpb.JobServiceClient, jobID log.Infof(ctx, "Job[%v] state: %v", jobID, resp.GetState().String()) - switch resp.State { + switch resp.GetState() { case jobpb.JobState_DONE, jobpb.JobState_CANCELLED: return nil case jobpb.JobState_FAILED: diff --git a/sdks/go/pkg/beam/runners/universal/runnerlib/stage.go b/sdks/go/pkg/beam/runners/universal/runnerlib/stage.go index 85d6fdc7e2ca..8684dcff8938 100644 --- a/sdks/go/pkg/beam/runners/universal/runnerlib/stage.go +++ b/sdks/go/pkg/beam/runners/universal/runnerlib/stage.go @@ -82,7 +82,7 @@ func stageFiles(ctx context.Context, cc *grpc.ClientConn, binary, st string) err } }() - if err := stream.Send(&jobpb.ArtifactResponseWrapper{StagingToken: st}); err != nil { + if err := stream.Send(jobpb.ArtifactResponseWrapper_builder{StagingToken: st}.Build()); err != nil { return errors.Wrapf(err, "failed to send staging token") } @@ -95,20 +95,18 @@ func stageFiles(ctx context.Context, cc *grpc.ClientConn, binary, st string) err return err } - switch request := in.Request.(type) { - case *jobpb.ArtifactRequestWrapper_ResolveArtifact: - err = stream.Send(&jobpb.ArtifactResponseWrapper{ - Response: &jobpb.ArtifactResponseWrapper_ResolveArtifactResponse{ - ResolveArtifactResponse: &jobpb.ResolveArtifactsResponse{ - Replacements: request.ResolveArtifact.Artifacts, - }, - }}) + switch request := in.WhichRequest(); request { + case jobpb.ArtifactRequestWrapper_ResolveArtifact_case: + err = stream.Send(jobpb.ArtifactResponseWrapper_builder{ + ResolveArtifactResponse: jobpb.ResolveArtifactsResponse_builder{ + Replacements: in.GetResolveArtifact().GetArtifacts(), + }.Build()}.Build()) if err != nil { return err } - case *jobpb.ArtifactRequestWrapper_GetArtifact: - switch typeUrn := request.GetArtifact.Artifact.TypeUrn; typeUrn { + case jobpb.ArtifactRequestWrapper_GetArtifact_case: + switch typeUrn := in.GetGetArtifact().GetArtifact().GetTypeUrn(); typeUrn { // TODO(https://github.com/apache/beam/issues/21459): Legacy Type URN. If requested, provide the binary. // To be removed later in 2022, once thoroughly obsolete. case graphx.URNArtifactGoWorker: @@ -119,8 +117,8 @@ func stageFiles(ctx context.Context, cc *grpc.ClientConn, binary, st string) err return errors.Wrapf(err, "failed to stage Go worker binary: %v", binary) } case graphx.URNArtifactFileType: - typePl := pipepb.ArtifactFilePayload{} - if err := proto.Unmarshal(request.GetArtifact.Artifact.TypePayload, &typePl); err != nil { + typePl := &pipepb.ArtifactFilePayload{} + if err := proto.Unmarshal(in.GetGetArtifact().GetArtifact().GetTypePayload(), typePl); err != nil { return errors.Wrap(err, "failed to parse artifact file payload") } if err := stageFile(typePl.GetPath(), stream); err != nil { @@ -135,7 +133,7 @@ func stageFiles(ctx context.Context, cc *grpc.ClientConn, binary, st string) err } default: - return errors.Errorf("request has unexpected type %T", request) + return errors.Errorf("request has unexpected type %v", request) } } } @@ -151,12 +149,10 @@ func stageFile(filename string, stream jobpb.ArtifactStagingService_ReverseArtif for { n, err := fd.Read(data) if n > 0 { - sendErr := stream.Send(&jobpb.ArtifactResponseWrapper{ - Response: &jobpb.ArtifactResponseWrapper_GetArtifactResponse{ - GetArtifactResponse: &jobpb.GetArtifactResponse{ - Data: data[:n], - }, - }}) + sendErr := stream.Send(jobpb.ArtifactResponseWrapper_builder{ + GetArtifactResponse: jobpb.GetArtifactResponse_builder{ + Data: data[:n], + }.Build()}.Build()) if sendErr == io.EOF { return sendErr } @@ -167,11 +163,9 @@ func stageFile(filename string, stream jobpb.ArtifactStagingService_ReverseArtif } if err == io.EOF { - sendErr := stream.Send(&jobpb.ArtifactResponseWrapper{ - IsLast: true, - Response: &jobpb.ArtifactResponseWrapper_GetArtifactResponse{ - GetArtifactResponse: &jobpb.GetArtifactResponse{}, - }}) + sendErr := stream.Send(jobpb.ArtifactResponseWrapper_builder{ + IsLast: true, + GetArtifactResponse: &jobpb.GetArtifactResponse{}}.Build()) return sendErr } diff --git a/sdks/go/pkg/beam/runners/universal/runnerlib/stage_test.go b/sdks/go/pkg/beam/runners/universal/runnerlib/stage_test.go index 81e4402fcbd1..10ceea7f1446 100644 --- a/sdks/go/pkg/beam/runners/universal/runnerlib/stage_test.go +++ b/sdks/go/pkg/beam/runners/universal/runnerlib/stage_test.go @@ -104,21 +104,19 @@ func (das *dummyArtifactServer) ReverseArtifactRetrievalService(stream jobpb.Art // Check artifact Resolve Requests. wantArts := []*pipepb.ArtifactInformation{ - { + pipepb.ArtifactInformation_builder{ TypeUrn: "dummy", TypePayload: []byte("dummy"), RoleUrn: "dummy", RolePayload: []byte("dummy"), - }, + }.Build(), } - if err := stream.Send(&jobpb.ArtifactRequestWrapper{ - Request: &jobpb.ArtifactRequestWrapper_ResolveArtifact{ - ResolveArtifact: &jobpb.ResolveArtifactsRequest{ - Artifacts: wantArts, - }, - }, - }); err != nil { + if err := stream.Send(jobpb.ArtifactRequestWrapper_builder{ + ResolveArtifact: jobpb.ResolveArtifactsRequest_builder{ + Artifacts: wantArts, + }.Build(), + }.Build()); err != nil { das.t.Fatalf("unexpected err on artifact resolve request Send: %v", err) } @@ -135,21 +133,19 @@ func (das *dummyArtifactServer) ReverseArtifactRetrievalService(stream jobpb.Art das.t.Errorf("diff on artifact resolve response (-want, +got):\n%v", d) } - typePl := protox.MustEncode(&pipepb.ArtifactFilePayload{ + typePl := protox.MustEncode(pipepb.ArtifactFilePayload_builder{ Path: das.reqFile, - }) + }.Build()) // Check file upload requests - if err := stream.Send(&jobpb.ArtifactRequestWrapper{ - Request: &jobpb.ArtifactRequestWrapper_GetArtifact{ - GetArtifact: &jobpb.GetArtifactRequest{ - Artifact: &pipepb.ArtifactInformation{ - TypeUrn: graphx.URNArtifactFileType, - TypePayload: typePl, - }, - }, - }, - }); err != nil { + if err := stream.Send(jobpb.ArtifactRequestWrapper_builder{ + GetArtifact: jobpb.GetArtifactRequest_builder{ + Artifact: pipepb.ArtifactInformation_builder{ + TypeUrn: graphx.URNArtifactFileType, + TypePayload: typePl, + }.Build(), + }.Build(), + }.Build()); err != nil { das.t.Fatalf("unexpected err on get artifact request Send: %v", err) } diff --git a/sdks/go/pkg/beam/testing/teststream/teststream.go b/sdks/go/pkg/beam/testing/teststream/teststream.go index c13e2cee9e07..de860adfd33c 100644 --- a/sdks/go/pkg/beam/testing/teststream/teststream.go +++ b/sdks/go/pkg/beam/testing/teststream/teststream.go @@ -34,6 +34,7 @@ import ( "github.com/apache/beam/sdks/v2/go/pkg/beam/core/util/protox" pipepb "github.com/apache/beam/sdks/v2/go/pkg/beam/model/pipeline_v1" + "google.golang.org/protobuf/proto" ) const urn = "beam:transform:teststream:v1" @@ -60,14 +61,14 @@ func NewConfig() Config { // defined manually. Currently does not support authentication, so the TestStreamService should // be accessed in a trusted context. func (c *Config) setEndpoint(url string) { - c.endpoint.Url = url + c.endpoint.SetUrl(url) } // createPayload converts the Config object into a TestStreamPayload to be sent to the runner. func (c *Config) createPayload() *pipepb.TestStreamPayload { // c0 is always the first coder in the pipeline, and inserting the TestStream as the first // element in the pipeline guarantees that the c0 coder corresponds to the type it outputs. - return &pipepb.TestStreamPayload{CoderId: "c0", Events: c.events, Endpoint: c.endpoint} + return pipepb.TestStreamPayload_builder{CoderId: "c0", Events: c.events, Endpoint: c.endpoint}.Build() } // AdvanceWatermark adds an event to the Config Events struct advancing the watermark for the PCollection @@ -76,9 +77,9 @@ func (c *Config) AdvanceWatermark(timestamp int64) error { if c.watermark >= timestamp { return fmt.Errorf("watermark must be monotonally increasing, is at %v, got %v", c.watermark, timestamp) } - watermarkAdvance := &pipepb.TestStreamPayload_Event_AdvanceWatermark{NewWatermark: timestamp} + watermarkAdvance := pipepb.TestStreamPayload_Event_AdvanceWatermark_builder{NewWatermark: timestamp}.Build() watermarkEvent := &pipepb.TestStreamPayload_Event_WatermarkEvent{WatermarkEvent: watermarkAdvance} - c.events = append(c.events, &pipepb.TestStreamPayload_Event{Event: watermarkEvent}) + c.events = append(c.events, pipepb.TestStreamPayload_Event_builder{WatermarkEvent: proto.ValueOrDefault(watermarkEvent.WatermarkEvent)}.Build()) c.watermark = timestamp return nil } @@ -91,9 +92,9 @@ func (c *Config) AdvanceWatermarkToInfinity() error { // AdvanceProcessingTime adds an event advancing the processing time by a given duration. // This advancement is applied to all of the PCollections output by the TestStream. func (c *Config) AdvanceProcessingTime(duration int64) { - processingAdvance := &pipepb.TestStreamPayload_Event_AdvanceProcessingTime{AdvanceDuration: duration} + processingAdvance := pipepb.TestStreamPayload_Event_AdvanceProcessingTime_builder{AdvanceDuration: duration}.Build() processingEvent := &pipepb.TestStreamPayload_Event_ProcessingTimeEvent{ProcessingTimeEvent: processingAdvance} - c.events = append(c.events, &pipepb.TestStreamPayload_Event{Event: processingEvent}) + c.events = append(c.events, pipepb.TestStreamPayload_Event_builder{ProcessingTimeEvent: proto.ValueOrDefault(processingEvent.ProcessingTimeEvent)}.Build()) } // AdvanceProcessingTimeToInfinity moves the TestStream processing time to the largest possible @@ -128,11 +129,11 @@ func (c *Config) AddElements(timestamp int64, elements ...any) error { if err := enc.Encode(e, &buf); err != nil { return fmt.Errorf("encoding value %v failed, got %v", e, err) } - newElements = append(newElements, &pipepb.TestStreamPayload_TimestampedElement{EncodedElement: buf.Bytes(), Timestamp: timestamp}) + newElements = append(newElements, pipepb.TestStreamPayload_TimestampedElement_builder{EncodedElement: buf.Bytes(), Timestamp: timestamp}.Build()) } - addElementsEvent := &pipepb.TestStreamPayload_Event_AddElements{Elements: newElements} + addElementsEvent := pipepb.TestStreamPayload_Event_AddElements_builder{Elements: newElements}.Build() elementEvent := &pipepb.TestStreamPayload_Event_ElementEvent{ElementEvent: addElementsEvent} - c.events = append(c.events, &pipepb.TestStreamPayload_Event{Event: elementEvent}) + c.events = append(c.events, pipepb.TestStreamPayload_Event_builder{ElementEvent: proto.ValueOrDefault(elementEvent.ElementEvent)}.Build()) return nil } diff --git a/sdks/go/pkg/beam/testing/teststream/teststream_test.go b/sdks/go/pkg/beam/testing/teststream/teststream_test.go index 22c38b03c6e0..a46d19a21e0b 100644 --- a/sdks/go/pkg/beam/testing/teststream/teststream_test.go +++ b/sdks/go/pkg/beam/testing/teststream/teststream_test.go @@ -32,7 +32,7 @@ func TestNewConfig(t *testing.T) { if len(con.events) != 0 { t.Errorf("config has too many elements, expected 0, got %v", len(con.events)) } - if con.endpoint.Url != "" { + if con.endpoint.GetUrl() != "" { t.Errorf("config has URL endpoint when it should be empty") } } @@ -46,7 +46,7 @@ func TestAdvanceWatermark(t *testing.T) { if len(con.events) != 1 { t.Fatalf("want only 1 event in config, got %v", len(con.events)) } - if eventWatermark := con.events[0].GetWatermarkEvent().NewWatermark; eventWatermark != 500 { + if eventWatermark := con.events[0].GetWatermarkEvent().GetNewWatermark(); eventWatermark != 500 { t.Errorf("want watermark in event to be 500, got %v", eventWatermark) } } diff --git a/sdks/go/pkg/beam/transforms/xlang/schema/external.go b/sdks/go/pkg/beam/transforms/xlang/schema/external.go index 55a858b9cf9e..b443f1f80ed2 100644 --- a/sdks/go/pkg/beam/transforms/xlang/schema/external.go +++ b/sdks/go/pkg/beam/transforms/xlang/schema/external.go @@ -76,11 +76,11 @@ func Transform(scope beam.Scope, config any, transformIdentifier string, opts .. panic(err) } - pl, err := proto.Marshal(&pipepb.SchemaTransformPayload{ + pl, err := proto.Marshal(pipepb.SchemaTransformPayload_builder{ Identifier: transformIdentifier, ConfigurationSchema: ecp.GetSchema(), ConfigurationRow: ecp.GetPayload(), - }) + }.Build()) if err != nil { panic(err) } diff --git a/sdks/go/pkg/beam/x/hooks/perf/perf.go b/sdks/go/pkg/beam/x/hooks/perf/perf.go index e45949df4534..0c0cb79f4e2e 100644 --- a/sdks/go/pkg/beam/x/hooks/perf/perf.go +++ b/sdks/go/pkg/beam/x/hooks/perf/perf.go @@ -63,7 +63,7 @@ func init() { pprof.StopCPUProfile() for _, h := range enabledProfCaptureHooks { name, opts := hooks.Decode(h) - if err := profCaptureHookRegistry[name](opts)(ctx, fmt.Sprintf("prof%s", req.InstructionId), &cpuProfBuf); err != nil { + if err := profCaptureHookRegistry[name](opts)(ctx, fmt.Sprintf("prof%s", req.GetInstructionId()), &cpuProfBuf); err != nil { return err } } @@ -92,7 +92,7 @@ func init() { trace.Stop() for _, h := range enabledTraceCaptureHooks { name, opts := hooks.Decode(h) - if err := traceCaptureHookRegistry[name](opts)(ctx, fmt.Sprintf("trace_prof%s", req.InstructionId), &traceProfBuf); err != nil { + if err := traceCaptureHookRegistry[name](opts)(ctx, fmt.Sprintf("trace_prof%s", req.GetInstructionId()), &traceProfBuf); err != nil { return err } } @@ -121,7 +121,7 @@ func init() { pprof.WriteHeapProfile(&heapProfBuf) for _, h := range enabledHeapCaptureHooks { name, opts := hooks.Decode(h) - if err := heapCaptureHookRegistry[name](opts)(ctx, fmt.Sprintf("heap%s", req.InstructionId), &heapProfBuf); err != nil { + if err := heapCaptureHookRegistry[name](opts)(ctx, fmt.Sprintf("heap%s", req.GetInstructionId()), &heapProfBuf); err != nil { return err } } diff --git a/sdks/go/scripts/genproto.sh b/sdks/go/scripts/genproto.sh index 8bdfd2d6f33a..abd642f2e81f 100755 --- a/sdks/go/scripts/genproto.sh +++ b/sdks/go/scripts/genproto.sh @@ -75,20 +75,8 @@ GEN_DIR=$PWD # NB: Keep these two versions in sync with those defined in # the go.mod. -go install google.golang.org/grpc/cmd/protoc-gen-go-grpc@ebf6a4b -go install google.golang.org/protobuf/cmd/protoc-gen-go@v1.27.1 - -function insert_license_header() { - local depth="$1" - # protoc-gen-go-grpc does not yet output comments from the original - # proto file so we need to manually add the license header. - while IFS= read -d $'\0' -r file ; do - tmp_file=$(mktemp) - echo "$LICENSE" > $tmp_file - cat $file >> $tmp_file - mv $tmp_file $file - done < <(find $GEN_DIR $depth -iname "*grpc.pb.go" -print0) -} +go install google.golang.org/grpc/cmd/protoc-gen-go-grpc@v1.5.1 +go install google.golang.org/protobuf/cmd/protoc-gen-go@v1.36.0 function gen_go_sdk_protos() { LIBRARY_PATH="${PWD##${SDK_PATH}/}" @@ -114,13 +102,12 @@ function gen_go_sdk_protos() { protoc \ "${INCLUDES[@]}" \ + --go_opt=default_api_level=API_HYBRID \ --go_opt=module=github.com/apache/beam/sdks/v2 \ --go-grpc_opt=module=github.com/apache/beam/sdks/v2 \ --go_out=$PKG_MAP:. \ --go-grpc_out=. \ $PROTOS - - insert_license_header '-d 1' } function gen_beam_model_protos() { @@ -142,14 +129,13 @@ function gen_beam_model_protos() { do protoc \ "${INCLUDES[@]}" \ + --go_opt=default_api_level=API_HYBRID \ --go_opt=module=github.com/apache/beam/sdks/v2 \ --go-grpc_opt=module=github.com/apache/beam/sdks/v2 \ --go_out="$PROJECT_ROOT/sdks" \ --go-grpc_out="$PROJECT_ROOT/sdks" \ $package done - - insert_license_header } if [[ $1 == "model" ]]; then diff --git a/sdks/go/test/regression/coders/fromyaml/fromyaml.go b/sdks/go/test/regression/coders/fromyaml/fromyaml.go index 466f361b0138..8c124d964a42 100644 --- a/sdks/go/test/regression/coders/fromyaml/fromyaml.go +++ b/sdks/go/test/regression/coders/fromyaml/fromyaml.go @@ -486,13 +486,13 @@ func (s *Spec) parseCoder(c Coder) string { for _, comp := range c.Components { compIDs = append(compIDs, s.parseCoder(comp)) } - s.coderPBs[id] = &pipepb.Coder{ - Spec: &pipepb.FunctionSpec{ + s.coderPBs[id] = pipepb.Coder_builder{ + Spec: pipepb.FunctionSpec_builder{ Urn: c.Urn, Payload: []byte(c.Payload), - }, + }.Build(), ComponentCoderIds: compIDs, - } + }.Build() return id } diff --git a/sdks/java/container/boot.go b/sdks/java/container/boot.go index c23e50dcf1b0..6143eb4da0e1 100644 --- a/sdks/java/container/boot.go +++ b/sdks/java/container/boot.go @@ -125,8 +125,8 @@ func main() { if err := tools.MakePipelineOptionsFileAndEnvVar(options); err != nil { logger.Fatalf(ctx, "Failed to load pipeline options to worker: %v", err) } - os.Setenv("LOGGING_API_SERVICE_DESCRIPTOR", (&pipepb.ApiServiceDescriptor{Url: *loggingEndpoint}).String()) - os.Setenv("CONTROL_API_SERVICE_DESCRIPTOR", (&pipepb.ApiServiceDescriptor{Url: *controlEndpoint}).String()) + os.Setenv("LOGGING_API_SERVICE_DESCRIPTOR", (pipepb.ApiServiceDescriptor_builder{Url: *loggingEndpoint}.Build()).String()) + os.Setenv("CONTROL_API_SERVICE_DESCRIPTOR", (pipepb.ApiServiceDescriptor_builder{Url: *controlEndpoint}.Build()).String()) os.Setenv("RUNNER_CAPABILITIES", strings.Join(info.GetRunnerCapabilities(), " ")) if info.GetStatusEndpoint() != nil { diff --git a/sdks/java/container/boot_test.go b/sdks/java/container/boot_test.go index 61d67e93ecbb..d1ae4b0d4615 100644 --- a/sdks/java/container/boot_test.go +++ b/sdks/java/container/boot_test.go @@ -80,13 +80,13 @@ func TestBuildOptions(t *testing.T) { } func TestHeapSizeLimit(t *testing.T) { - if lim := HeapSizeLimit(0); lim != 1 << 30 { + if lim := HeapSizeLimit(0); lim != 1<<30 { t.Errorf("HeapSizeLimit(0). Actual (%d). want 1 GB", lim) } - if lim := HeapSizeLimit(2 << 30); lim != (2 << 30) * 7 / 10 { + if lim := HeapSizeLimit(2 << 30); lim != (2<<30)*7/10 { t.Errorf("HeapSizeLimit(2 GB). Actual (%d). want 1.4 GB", lim) } - if lim := HeapSizeLimit(200 << 30); lim != (200 - 32) << 30 { + if lim := HeapSizeLimit(200 << 30); lim != (200-32)<<30 { t.Errorf("HeapSizeLimit(200 GB). Actual (%d). want 168 GB", lim) } } diff --git a/sdks/java/expansion-service/container/boot.go b/sdks/java/expansion-service/container/boot.go index 936e4665183a..fab66f50f1fa 100644 --- a/sdks/java/expansion-service/container/boot.go +++ b/sdks/java/expansion-service/container/boot.go @@ -15,8 +15,9 @@ // Boot code for the Java SDK expansion service. // Contract: -// https://github.com/apache/beam/blob/master/model/job-management/src/main/proto/org/apache/beam/model/job_management/v1/beam_expansion_api.proto -// https://github.com/apache/beam/blob/master/model/job-management/src/main/proto/org/apache/beam/model/job_management/v1/beam_artifact_api.proto +// +// https://github.com/apache/beam/blob/master/model/job-management/src/main/proto/org/apache/beam/model/job_management/v1/beam_expansion_api.proto +// https://github.com/apache/beam/blob/master/model/job-management/src/main/proto/org/apache/beam/model/job_management/v1/beam_artifact_api.proto package main import ( @@ -37,14 +38,14 @@ import ( // - Expansion service port // - Dependencies (for loading SchemaTransforms) // - Config file path. Config file contains: -// - Allow-list -// - Per-transform dependencies config. +// - Allow-list +// - Per-transform dependencies config. var ( id = flag.String("id", "", "Local identifier (required)") port = flag.Int("port", 0, "Port for the expansion service (required)") dependencies_dir = flag.String("dependencies_dir", "", "A directory containing the set of jar files to load transforms from (required)") config_file = flag.String("config_file", "", "Expansion service config YAML file. (required)") - use_alts = flag.Bool("use_alts", false, "Starts an Expansion Service with support for gRPC ALTS authentication") + use_alts = flag.Bool("use_alts", false, "Starts an Expansion Service with support for gRPC ALTS authentication") ) const entrypoint = "org.apache.beam.sdk.expansion.service.ExpansionService" diff --git a/sdks/java/transform-service/controller-container/boot.go b/sdks/java/transform-service/controller-container/boot.go index 7cf792d5e2d8..b50268b81e5f 100644 --- a/sdks/java/transform-service/controller-container/boot.go +++ b/sdks/java/transform-service/controller-container/boot.go @@ -15,8 +15,9 @@ // Boot code for the transform service controller. // Contract: -// https://github.com/apache/beam/blob/master/model/job-management/src/main/proto/org/apache/beam/model/job_management/v1/beam_expansion_api.proto -// https://github.com/apache/beam/blob/master/model/job-management/src/main/proto/org/apache/beam/model/job_management/v1/beam_artifact_api.proto +// +// https://github.com/apache/beam/blob/master/model/job-management/src/main/proto/org/apache/beam/model/job_management/v1/beam_expansion_api.proto +// https://github.com/apache/beam/blob/master/model/job-management/src/main/proto/org/apache/beam/model/job_management/v1/beam_artifact_api.proto package main import ( @@ -36,7 +37,7 @@ import ( // Args: // - Transform service port // - Config file path. Config file contains: -// - A list of expansion services +// - A list of expansion services var ( port = flag.Int("port", 0, "Port for the expansion service (required)") config_file = flag.String("config_file", "", "Transform service config YAML file. (required)") diff --git a/sdks/python/container/boot.go b/sdks/python/container/boot.go index b7cbc07dca68..f3094be5d964 100644 --- a/sdks/python/container/boot.go +++ b/sdks/python/container/boot.go @@ -206,7 +206,7 @@ func launchSDKProcess() error { logger.Printf(ctx, "Found artifact: %s", name) fileNames[i] = name - if v.RoleUrn == artifact.URNPipRequirementsFile { + if v.GetRoleUrn() == artifact.URNPipRequirementsFile { requirementsFiles = append(requirementsFiles, name) } } @@ -222,8 +222,8 @@ func launchSDKProcess() error { os.Setenv("PIPELINE_OPTIONS", options) os.Setenv("SEMI_PERSISTENT_DIRECTORY", *semiPersistDir) - os.Setenv("LOGGING_API_SERVICE_DESCRIPTOR", (&pipepb.ApiServiceDescriptor{Url: *loggingEndpoint}).String()) - os.Setenv("CONTROL_API_SERVICE_DESCRIPTOR", (&pipepb.ApiServiceDescriptor{Url: *controlEndpoint}).String()) + os.Setenv("LOGGING_API_SERVICE_DESCRIPTOR", (pipepb.ApiServiceDescriptor_builder{Url: *loggingEndpoint}.Build()).String()) + os.Setenv("CONTROL_API_SERVICE_DESCRIPTOR", (pipepb.ApiServiceDescriptor_builder{Url: *controlEndpoint}.Build()).String()) os.Setenv("RUNNER_CAPABILITIES", strings.Join(info.GetRunnerCapabilities(), " ")) if info.GetStatusEndpoint() != nil { diff --git a/sdks/typescript/container/boot.go b/sdks/typescript/container/boot.go index 44f94f804330..ccc3d308a144 100644 --- a/sdks/typescript/container/boot.go +++ b/sdks/typescript/container/boot.go @@ -104,13 +104,13 @@ func main() { for _, v := range artifacts { name, _ := artifact.MustExtractFilePayload(v) path := filepath.Join(dir, name) - if v.RoleUrn == "beam:artifact:type:npm_dep:v1" { + if v.GetRoleUrn() == "beam:artifact:type:npm_dep:v1" { // Npm cannot handle arbitrary suffixes. suffixedPath := path + ".tar" if err := os.Rename(path, suffixedPath); err != nil { logger.Fatalf(ctx, "unable to rename %v to %v: %v", path, suffixedPath, err) } - npmOverrides[string(v.RolePayload)] = suffixedPath + npmOverrides[string(v.GetRolePayload())] = suffixedPath } } if len(npmOverrides) > 0 { @@ -140,7 +140,7 @@ func main() { for _, v := range artifacts { name, _ := artifact.MustExtractFilePayload(v) path := filepath.Join(dir, name) - if v.RoleUrn == "beam:artifact:type:npm:v1" { + if v.GetRoleUrn() == "beam:artifact:type:npm:v1" { // Npm cannot handle arbitrary suffixes. suffixedPath := path + ".tar" if err := os.Rename(path, suffixedPath); err != nil {