├── .gitignore ├── examples └── greeter │ └── main.go ├── go.mod ├── go.sum ├── pkg └── statefun │ ├── context.go │ ├── egress.go │ ├── handler.go │ ├── handler_test.go │ ├── internal │ ├── cell.go │ └── protocol │ │ ├── kafka-egress.pb.go │ │ ├── kinesis-egress.pb.go │ │ └── request-reply.pb.go │ ├── message.go │ ├── message_test.go │ ├── stateful_function.go │ ├── storage.go │ ├── typename.go │ ├── typename_test.go │ ├── types.go │ ├── types_test.go │ └── value_spec.go └── statefun-sdk-go.iml /.gitignore: -------------------------------------------------------------------------------- 1 | # Binaries for programs and plugins 2 | *.exe 3 | *.exe~ 4 | *.dll 5 | *.so 6 | *.dylib 7 | *.idea/* 8 | 9 | examples/checkpoint-dir/ 10 | test/test 11 | # Test binary, built with `go test -c` 12 | *.test 13 | 14 | # Output of the go coverage tool, specifically when used with LiteIDE 15 | *.out 16 | 17 | # Dependency directories (remove the comment below to include it) 18 | # vendor/ 19 | -------------------------------------------------------------------------------- /examples/greeter/main.go: -------------------------------------------------------------------------------- 1 | package main 2 | 3 | import ( 4 | "fmt" 5 | "log" 6 | "net/http" 7 | "statefun-sdk-go/pkg/statefun" 8 | ) 9 | 10 | type GreetRequest struct { 11 | Name string `json:"name"` 12 | Visits int32 `json:"visits"` 13 | } 14 | 15 | var GreetRequestType = statefun.MakeJsonType(statefun.TypeNameFrom("example/GreetRequest")) 16 | 17 | var PersonFunc = statefun.TypeNameFrom("example/person") 18 | 19 | var GreeterFunc = statefun.TypeNameFrom("example/greeter") 20 | 21 | var KafkaEgress = statefun.TypeNameFrom("example/greets") 22 | 23 | type Person struct { 24 | Visits statefun.ValueSpec 25 | } 26 | 27 | func (p Person) Invoke(ctx statefun.Context, msg statefun.Message) error { 28 | 29 | var visits int32 30 | _ = ctx.Storage().Get(p.Visits, &visits) 31 | visits += 1 32 | ctx.Storage().Set(p.Visits, visits) 33 | 34 | request := GreetRequest{} 35 | _ = msg.As(GreetRequestType, &request) 36 | request.Visits = visits 37 | 38 | ctx.Send(statefun.MessageBuilder{ 39 | Target: statefun.Address{ 40 | FunctionType: GreeterFunc, 41 | Id: request.Name, 42 | }, 43 | Value: request, 44 | ValueType: GreetRequestType, 45 | }) 46 | 47 | return nil 48 | } 49 | 50 | func greeter(ctx statefun.Context, msg statefun.Message) error { 51 | 52 | var request GreetRequest 53 | _ = msg.As(GreetRequestType, &request) 54 | 55 | greeting := computeGreeting(request.Name, request.Visits) 56 | 57 | ctx.SendEgress(statefun.KafkaEgressBuilder{ 58 | Target: KafkaEgress, 59 | Topic: "greetings", 60 | Key: request.Name, 61 | Value: greeting, 62 | }) 63 | 64 | return nil 65 | } 66 | 67 | func computeGreeting(name string, seen int32) string { 68 | templates := []string{"", "Welcome %s", "Nice to see you again %s", "Third time is the charm %s"} 69 | if int(seen) < len(templates) { 70 | return fmt.Sprintf(templates[seen], name) 71 | } 72 | 73 | return fmt.Sprintf("Nice to see you for the %dth time %s", seen, name) 74 | } 75 | 76 | func main() { 77 | 78 | builder := statefun.StatefulFunctionsBuilder() 79 | 80 | person := Person{ 81 | Visits: statefun.ValueSpec{ 82 | Name: "visits", 83 | ValueType: statefun.Int32Type, 84 | }} 85 | 86 | _ = builder.WithSpec(statefun.StatefulFunctionSpec{ 87 | FunctionType: PersonFunc, 88 | States: []statefun.ValueSpec{person.Visits}, 89 | Function: person, 90 | }) 91 | 92 | _ = builder.WithSpec(statefun.StatefulFunctionSpec{ 93 | FunctionType: GreeterFunc, 94 | Function: statefun.StatefulFunctionPointer(greeter), 95 | }) 96 | 97 | http.Handle("/statefun", builder.AsHandler()) 98 | log.Fatal(http.ListenAndServe(":8000", nil)) 99 | } 100 | -------------------------------------------------------------------------------- /go.mod: -------------------------------------------------------------------------------- 1 | module statefun-sdk-go 2 | 3 | go 1.15 4 | 5 | require ( 6 | github.com/golang/protobuf v1.5.0 7 | github.com/stretchr/testify v1.7.0 8 | google.golang.org/protobuf v1.26.0 9 | ) 10 | -------------------------------------------------------------------------------- /go.sum: -------------------------------------------------------------------------------- 1 | github.com/davecgh/go-spew v1.1.0 h1:ZDRjVQ15GmhC3fiQ8ni8+OwkZQO4DARzQgrnXU1Liz8= 2 | github.com/davecgh/go-spew v1.1.0/go.mod h1:J7Y8YcW2NihsgmVo/mv3lAwl/skON4iLHjSsI+c5H38= 3 | github.com/golang/protobuf v1.5.0 h1:LUVKkCeviFUMKqHa4tXIIij/lbhnMbP7Fn5wKdKkRh4= 4 | github.com/golang/protobuf v1.5.0/go.mod h1:FsONVRAS9T7sI+LIUmWTfcYkHO4aIWwzhcaSAoJOfIk= 5 | github.com/google/go-cmp v0.5.5 h1:Khx7svrCpmxxtHBq5j2mp/xVjsi8hQMfNLvJFAlrGgU= 6 | github.com/google/go-cmp v0.5.5/go.mod h1:v8dTdLbMG2kIc/vJvl+f65V22dbkXbowE6jgT/gNBxE= 7 | github.com/pmezard/go-difflib v1.0.0 h1:4DBwDE0NGyQoBHbLQYPwSUPoCMWR5BEzIk/f1lZbAQM= 8 | github.com/pmezard/go-difflib v1.0.0/go.mod h1:iKH77koFhYxTK1pcRnkKkqfTogsbg7gZNVY4sRDYZ/4= 9 | github.com/stretchr/objx v0.1.0/go.mod h1:HFkY916IF+rwdDfMAkV7OtwuqBVzrE8GR6GFx+wExME= 10 | github.com/stretchr/testify v1.7.0 h1:nwc3DEeHmmLAfoZucVR881uASk0Mfjw8xYJ99tb5CcY= 11 | github.com/stretchr/testify v1.7.0/go.mod h1:6Fq8oRcR53rry900zMqJjRRixrwX3KX962/h/Wwjteg= 12 | golang.org/x/xerrors v0.0.0-20191204190536-9bdfabe68543 h1:E7g+9GITq07hpfrRu66IVDexMakfv52eLZ2CXBWiKr4= 13 | golang.org/x/xerrors v0.0.0-20191204190536-9bdfabe68543/go.mod h1:I/5z698sn9Ka8TeJc9MKroUUfqBBauWjQqLJ2OPfmY0= 14 | google.golang.org/protobuf v1.26.0-rc.1/go.mod h1:jlhhOSvTdKEhbULTjvd4ARK9grFBp09yW+WbY/TyQbw= 15 | google.golang.org/protobuf v1.26.0 h1:bxAC2xTBsZGibn2RTntX0oH50xLsqy1OxA9tTL3p/lk= 16 | google.golang.org/protobuf v1.26.0/go.mod h1:9q0QmTI4eRPtz6boOQmLYwt+qCgq0jsYwAQnmE0givc= 17 | gopkg.in/check.v1 v0.0.0-20161208181325-20d25e280405 h1:yhCVgyC4o1eVCa2tZl7eS0r+SDo693bJlVdllGtEeKM= 18 | gopkg.in/check.v1 v0.0.0-20161208181325-20d25e280405/go.mod h1:Co6ibVJAznAaIkqp8huTwlJQCZ016jof/cbN4VW5Yz0= 19 | gopkg.in/yaml.v3 v3.0.0-20200313102051-9f266ea9e77c h1:dUUwHk2QECo/6vqA44rthZ8ie2QXMNeKRTHCNY2nXvo= 20 | gopkg.in/yaml.v3 v3.0.0-20200313102051-9f266ea9e77c/go.mod h1:K4uyk7z7BCEPqu6E+C64Yfv1cQ7kz7rIZviUmN+EgEM= 21 | -------------------------------------------------------------------------------- /pkg/statefun/context.go: -------------------------------------------------------------------------------- 1 | package statefun 2 | 3 | import ( 4 | "context" 5 | "statefun-sdk-go/pkg/statefun/internal/protocol" 6 | "sync" 7 | "time" 8 | ) 9 | 10 | // A Context contains information about the current function invocation, such as the invoked 11 | // function instance's and caller's Address. It is also used for side-effects as a result of 12 | // the invocation such as send messages to other functions or egresses, and provides access to 13 | // AddressScopedStorage scoped to the current Address. This type is also a context.Context 14 | // and can be used to ensure any spawned go routines do not outlive the current function 15 | // invocation. 16 | type Context interface { 17 | context.Context 18 | 19 | // The current invoked function instance's Address. 20 | Self() Address 21 | 22 | // The caller function instance's Address, if applicable. This is nil 23 | // if the message was sent to this function via an ingress. 24 | Caller() *Address 25 | 26 | // Sends out a MessageBuilder to another function. 27 | Send(message MessageBuilder) 28 | 29 | // Sends out a MessageBuilder to another function, after a specified time.Duration delay. 30 | SendAfter(delay time.Duration, message MessageBuilder) 31 | 32 | // Sends out an EgressBuilder to an egress. 33 | SendEgress(egress EgressBuilder) 34 | 35 | // The AddressScopedStorage, providing access to stored values scoped to the 36 | // current invoked function instance's Address (which is obtainable using Self()). 37 | Storage() AddressScopedStorage 38 | } 39 | 40 | type statefunContext struct { 41 | sync.Mutex 42 | context.Context 43 | self Address 44 | caller *Address 45 | storage *storage 46 | response *protocol.FromFunction_InvocationResponse 47 | } 48 | 49 | func (s *statefunContext) Storage() AddressScopedStorage { 50 | return s.storage 51 | } 52 | 53 | func (s *statefunContext) Self() Address { 54 | return s.self 55 | } 56 | 57 | func (s *statefunContext) Caller() *Address { 58 | return s.caller 59 | } 60 | 61 | func (s *statefunContext) Send(message MessageBuilder) { 62 | msg, err := message.ToMessage() 63 | 64 | if err != nil { 65 | panic(err) 66 | } 67 | 68 | invocation := &protocol.FromFunction_Invocation{ 69 | Target: msg.target, 70 | Argument: msg.typedValue, 71 | } 72 | 73 | s.Lock() 74 | s.response.OutgoingMessages = append(s.response.OutgoingMessages, invocation) 75 | s.Unlock() 76 | } 77 | 78 | func (s *statefunContext) SendAfter(delay time.Duration, message MessageBuilder) { 79 | msg, err := message.ToMessage() 80 | 81 | if err != nil { 82 | panic(err) 83 | } 84 | 85 | invocation := &protocol.FromFunction_DelayedInvocation{ 86 | Target: msg.target, 87 | Argument: msg.typedValue, 88 | DelayInMs: delay.Milliseconds(), 89 | } 90 | 91 | s.Lock() 92 | s.response.DelayedInvocations = append(s.response.DelayedInvocations, invocation) 93 | s.Unlock() 94 | } 95 | 96 | func (s *statefunContext) SendEgress(egress EgressBuilder) { 97 | msg, err := egress.toEgressMessage() 98 | 99 | if err != nil { 100 | panic(err) 101 | } 102 | 103 | s.Lock() 104 | s.response.OutgoingEgresses = append(s.response.OutgoingEgresses, msg) 105 | s.Unlock() 106 | } 107 | -------------------------------------------------------------------------------- /pkg/statefun/egress.go: -------------------------------------------------------------------------------- 1 | package statefun 2 | 3 | import ( 4 | "bytes" 5 | "encoding/binary" 6 | "errors" 7 | "google.golang.org/protobuf/proto" 8 | "statefun-sdk-go/pkg/statefun/internal/protocol" 9 | ) 10 | 11 | type EgressBuilder interface { 12 | toEgressMessage() (*protocol.FromFunction_EgressMessage, error) 13 | } 14 | 15 | // Builds a message that can be emitted to a Kafka generic egress. 16 | // If a ValueType is provided, then Value will be serialized according to the 17 | // provided ValueType's serializer. Otherwise we will try to convert Value to bytes 18 | // if it is one of: 19 | // - utf-8 string 20 | // - []bytes 21 | // - an int (as defined by Kafka's serialization format) 22 | // - float (as defined by Kafka's serialization format) 23 | type KafkaEgressBuilder struct { 24 | // The TypeName as specified in module.yaml 25 | Target TypeName 26 | 27 | // The Kafka destination topic for that record 28 | Topic string 29 | 30 | // The utf8 encoded string key to produce (can be empty) 31 | Key string 32 | 33 | // The value to produce 34 | Value interface{} 35 | 36 | // An optional hint to this values type 37 | ValueType SimpleType 38 | } 39 | 40 | func (k KafkaEgressBuilder) isEnvelope() {} 41 | 42 | func (k KafkaEgressBuilder) toEgressMessage() (*protocol.FromFunction_EgressMessage, error) { 43 | if k.Target == nil { 44 | return nil, errors.New("an egress record requires a Target") 45 | } 46 | if k.Topic == "" { 47 | return nil, errors.New("A Kafka record requires a topic") 48 | } 49 | 50 | if k.Value == nil { 51 | return nil, errors.New("A Kafka record requires a value") 52 | } 53 | 54 | buffer := bytes.Buffer{} 55 | if k.ValueType != nil { 56 | if err := k.ValueType.Serialize(&buffer, k.Value); err != nil { 57 | return nil, err 58 | } 59 | } else { 60 | switch value := k.Value.(type) { 61 | case string: 62 | _ = StringType.Serialize(&buffer, value) 63 | case []byte: 64 | buffer.Write(value) 65 | case int, int32, int64, float32, float64: 66 | if err := binary.Write(&buffer, binary.BigEndian, value); err != nil { 67 | return nil, err 68 | } 69 | default: 70 | return nil, errors.New("unable to convert value to bytes") 71 | } 72 | } 73 | 74 | kafka := protocol.KafkaProducerRecord{ 75 | Key: k.Key, 76 | ValueBytes: buffer.Bytes(), 77 | Topic: k.Topic, 78 | } 79 | 80 | value, err := proto.Marshal(&kafka) 81 | if err != nil { 82 | return nil, err 83 | } 84 | 85 | return &protocol.FromFunction_EgressMessage{ 86 | EgressNamespace: k.Target.GetNamespace(), 87 | EgressType: k.Target.GetType(), 88 | Argument: &protocol.TypedValue{ 89 | Typename: "type.googleapis.com/io.statefun.sdk.egress.KafkaProducerRecord", 90 | HasValue: true, 91 | Value: value, 92 | }, 93 | }, nil 94 | } 95 | 96 | // Builds a message that can be emitted to a Kinesis generic egress. 97 | // If a ValueType is provided, then Value will be serialized according to the 98 | // provided ValueType's serializer. Otherwise we will try to convert Value to bytes 99 | // if it is one of: 100 | // - utf-8 string 101 | // - []byte 102 | type KinesisEgressBuilder struct { 103 | // The TypeName as specified in module.yaml 104 | Target TypeName 105 | 106 | // The Kinesis destination stream for that record 107 | Stream string 108 | 109 | // The value to produce 110 | Value interface{} 111 | 112 | // An optional hint to this value type 113 | ValueType SimpleType 114 | 115 | // The utf8 encoded string partition key to use 116 | PartitionKey string 117 | 118 | // A utf8 encoded string explicit hash key to use (can be empty) 119 | ExplicitHashKey string 120 | } 121 | 122 | func (k KinesisEgressBuilder) toEgressMessage() (*protocol.FromFunction_EgressMessage, error) { 123 | if k.Target == nil { 124 | return nil, errors.New("an egress record requires a Target") 125 | } else if k.Stream == "" { 126 | return nil, errors.New("missing destination Kinesis stream") 127 | } else if k.Value == nil { 128 | return nil, errors.New("missing value") 129 | } else if k.PartitionKey == "" { 130 | return nil, errors.New("missing partition key") 131 | } 132 | 133 | buffer := bytes.Buffer{} 134 | if k.ValueType != nil { 135 | if err := k.ValueType.Serialize(&buffer, k.Value); err != nil { 136 | return nil, err 137 | } 138 | } else { 139 | switch value := k.Value.(type) { 140 | case string: 141 | _ = StringType.Serialize(&buffer, value) 142 | case []byte: 143 | buffer.Write(value) 144 | default: 145 | return nil, errors.New("unable to convert value to bytes") 146 | } 147 | } 148 | 149 | kinesis := protocol.KinesisEgressRecord{ 150 | PartitionKey: k.PartitionKey, 151 | ValueBytes: buffer.Bytes(), 152 | Stream: k.Stream, 153 | ExplicitHashKey: k.ExplicitHashKey, 154 | } 155 | 156 | value, err := proto.Marshal(&kinesis) 157 | if err != nil { 158 | return nil, err 159 | } 160 | 161 | return &protocol.FromFunction_EgressMessage{ 162 | EgressNamespace: k.Target.GetNamespace(), 163 | EgressType: k.Target.GetType(), 164 | Argument: &protocol.TypedValue{ 165 | Typename: "type.googleapis.com/io.statefun.sdk.egress.KinesisEgressRecord", 166 | HasValue: true, 167 | Value: value, 168 | }, 169 | }, nil 170 | } 171 | 172 | // Create a generic egress record. For Kafka 173 | // and Kinesis see KafkaEgressBuilder and 174 | // KinesisEgressBuilder respectively 175 | type GenericEgressBuilder struct { 176 | // The TypeName as specified when registered 177 | Target TypeName 178 | 179 | // The value to produce 180 | Value interface{} 181 | 182 | // The values type 183 | ValueType SimpleType 184 | } 185 | 186 | func (g GenericEgressBuilder) toEgressMessage() (*protocol.FromFunction_EgressMessage, error) { 187 | if g.Target == nil { 188 | return nil, errors.New("an egress record requires a Target") 189 | } else if g.ValueType == nil { 190 | return nil, errors.New("missing value type") 191 | } else if g.Value == nil { 192 | return nil, errors.New("missing value") 193 | } 194 | 195 | buffer := bytes.Buffer{} 196 | if err := g.ValueType.Serialize(&buffer, g.Value); err != nil { 197 | return nil, err 198 | } 199 | 200 | return &protocol.FromFunction_EgressMessage{ 201 | EgressNamespace: g.Target.GetNamespace(), 202 | EgressType: g.Target.GetType(), 203 | Argument: &protocol.TypedValue{ 204 | Typename: g.ValueType.GetTypeName().String(), 205 | HasValue: true, 206 | Value: buffer.Bytes(), 207 | }, 208 | }, nil 209 | } 210 | -------------------------------------------------------------------------------- /pkg/statefun/handler.go: -------------------------------------------------------------------------------- 1 | package statefun 2 | 3 | import ( 4 | "bytes" 5 | "context" 6 | "fmt" 7 | "google.golang.org/protobuf/proto" 8 | "log" 9 | "net/http" 10 | "statefun-sdk-go/pkg/statefun/internal/protocol" 11 | ) 12 | 13 | // A registry for multiple StatefulFunction's. A RequestReplyHandler 14 | // can be created from the registry that understands how to dispatch 15 | // invocation requests to the registered functions as well as encode 16 | // side-effects (e.g., sending messages to other functions or updating 17 | // values in storage) as the response. 18 | type StatefulFunctions interface { 19 | 20 | // Registers a StatefulFunctionSpec, which will be 21 | // used to build the runtime function. 22 | WithSpec(spec StatefulFunctionSpec) error 23 | 24 | // Creates a RequestReplyHandler from the registered 25 | // function specs. 26 | AsHandler() RequestReplyHandler 27 | } 28 | 29 | // The RequestReplyHandler processes messages 30 | // from the runtime, invokes functions, and encodes 31 | // side effects. The handler implements http.Handler 32 | // so it can easily be embedded in standard Go server 33 | // frameworks. 34 | type RequestReplyHandler interface { 35 | http.Handler 36 | 37 | // This method provides compliance with AWS Lambda handler 38 | Invoke(ctx context.Context, payload []byte) ([]byte, error) 39 | } 40 | 41 | // Creates a new StatefulFunctions registry. 42 | func StatefulFunctionsBuilder() StatefulFunctions { 43 | return &handler{ 44 | module: map[TypeName]StatefulFunction{}, 45 | stateSpecs: map[TypeName]map[string]*protocol.FromFunction_PersistedValueSpec{}, 46 | } 47 | } 48 | 49 | type handler struct { 50 | module map[TypeName]StatefulFunction 51 | stateSpecs map[TypeName]map[string]*protocol.FromFunction_PersistedValueSpec 52 | } 53 | 54 | func (h *handler) WithSpec(spec StatefulFunctionSpec) error { 55 | if _, exists := h.module[spec.FunctionType]; exists { 56 | return fmt.Errorf("failed to register Stateful Function %s, there is already a spec registered under that tpe", spec.FunctionType) 57 | } 58 | 59 | if spec.Function == nil { 60 | return fmt.Errorf("failed to register Stateful Function %s, the Function instance cannot be nil", spec.FunctionType) 61 | } 62 | 63 | h.module[spec.FunctionType] = spec.Function 64 | h.stateSpecs[spec.FunctionType] = make(map[string]*protocol.FromFunction_PersistedValueSpec, len(spec.States)) 65 | 66 | for _, state := range spec.States { 67 | if err := validateValueSpec(state); err != nil { 68 | return fmt.Errorf("failed to register Stateful Function %s: %w", spec.FunctionType, err) 69 | } 70 | 71 | expiration := &protocol.FromFunction_ExpirationSpec{} 72 | switch state.Expiration.expirationType { 73 | case none: 74 | expiration.Mode = protocol.FromFunction_ExpirationSpec_NONE 75 | case expireAfterWrite: 76 | expiration.Mode = protocol.FromFunction_ExpirationSpec_AFTER_WRITE 77 | expiration.ExpireAfterMillis = state.Expiration.duration.Milliseconds() 78 | case expireAfterCall: 79 | expiration.Mode = protocol.FromFunction_ExpirationSpec_AFTER_INVOKE 80 | expiration.ExpireAfterMillis = state.Expiration.duration.Milliseconds() 81 | } 82 | 83 | h.stateSpecs[spec.FunctionType][state.Name] = &protocol.FromFunction_PersistedValueSpec{ 84 | StateName: state.Name, 85 | ExpirationSpec: expiration, 86 | TypeTypename: state.ValueType.GetTypeName().String(), 87 | } 88 | } 89 | 90 | return nil 91 | } 92 | 93 | func (h *handler) AsHandler() RequestReplyHandler { 94 | log.Println("Create RequestReplyHandler") 95 | for typeName := range h.module { 96 | log.Printf("> Registering %s\n", typeName) 97 | } 98 | return h 99 | } 100 | 101 | func (h *handler) ServeHTTP(writer http.ResponseWriter, request *http.Request) { 102 | if request.Method != "POST" { 103 | http.Error(writer, "invalid request method", http.StatusMethodNotAllowed) 104 | return 105 | } 106 | 107 | contentType := request.Header.Get("Content-type") 108 | if contentType != "" && contentType != "application/octet-stream" { 109 | http.Error(writer, "invalid content type", http.StatusUnsupportedMediaType) 110 | return 111 | } 112 | 113 | if request.Body == nil || request.ContentLength == 0 { 114 | http.Error(writer, "empty request body", http.StatusBadRequest) 115 | return 116 | } 117 | 118 | buffer := bytes.Buffer{} 119 | if _, err := buffer.ReadFrom(request.Body); err != nil { 120 | http.Error(writer, err.Error(), http.StatusBadRequest) 121 | return 122 | } 123 | 124 | response, err := h.Invoke(request.Context(), buffer.Bytes()) 125 | if err != nil { 126 | log.Printf(err.Error()) 127 | http.Error(writer, err.Error(), http.StatusInternalServerError) 128 | return 129 | } 130 | 131 | _, _ = writer.Write(response) 132 | } 133 | 134 | func (h *handler) Invoke(ctx context.Context, payload []byte) ([]byte, error) { 135 | toFunction := protocol.ToFunction{} 136 | if err := proto.Unmarshal(payload, &toFunction); err != nil { 137 | return nil, fmt.Errorf("failed to unmarshal ToFunction: %w", err) 138 | } 139 | 140 | fromFunction, err := h.invoke(ctx, &toFunction) 141 | if err != nil { 142 | return nil, err 143 | } 144 | 145 | return proto.Marshal(fromFunction) 146 | } 147 | 148 | func (h *handler) invoke(ctx context.Context, toFunction *protocol.ToFunction) (from *protocol.FromFunction, err error) { 149 | batch := toFunction.GetInvocation() 150 | self := addressFromInternal(batch.Target) 151 | function, exists := h.module[self.FunctionType] 152 | 153 | defer func() { 154 | if r := recover(); r != nil { 155 | switch r := r.(type) { 156 | case error: 157 | err = fmt.Errorf("failed to execute invocation for %s: %w", batch.Target, r) 158 | default: 159 | log.Fatal(r) 160 | } 161 | } 162 | }() 163 | 164 | if !exists { 165 | return nil, fmt.Errorf("unknown function type %s", self.FunctionType) 166 | } 167 | 168 | storageFactory := newStorageFactory(batch, h.stateSpecs[self.FunctionType]) 169 | 170 | if missing := storageFactory.getMissingSpecs(); missing != nil { 171 | return &protocol.FromFunction{ 172 | Response: &protocol.FromFunction_IncompleteInvocationContext_{ 173 | IncompleteInvocationContext: &protocol.FromFunction_IncompleteInvocationContext{ 174 | MissingValues: missing, 175 | }, 176 | }, 177 | }, nil 178 | } 179 | 180 | storage := storageFactory.getStorage() 181 | response := &protocol.FromFunction_InvocationResponse{} 182 | 183 | for _, invocation := range batch.Invocations { 184 | select { 185 | case <-ctx.Done(): 186 | return nil, ctx.Err() 187 | default: 188 | sContext := statefunContext{ 189 | self: self, 190 | storage: storage, 191 | response: response, 192 | } 193 | 194 | var cancel context.CancelFunc 195 | sContext.Context, cancel = context.WithCancel(ctx) 196 | 197 | var caller Address 198 | if invocation.Caller != nil { 199 | caller = addressFromInternal(invocation.Caller) 200 | } 201 | sContext.caller = &caller 202 | msg := Message{ 203 | target: batch.Target, 204 | typedValue: invocation.Argument, 205 | } 206 | err = function.Invoke(&sContext, msg) 207 | cancel() 208 | 209 | if err != nil { 210 | return 211 | } 212 | } 213 | } 214 | 215 | response.StateMutations = storage.getStateMutations() 216 | from = &protocol.FromFunction{ 217 | Response: &protocol.FromFunction_InvocationResult{ 218 | InvocationResult: response, 219 | }, 220 | } 221 | 222 | return 223 | } 224 | -------------------------------------------------------------------------------- /pkg/statefun/handler_test.go: -------------------------------------------------------------------------------- 1 | package statefun 2 | 3 | import ( 4 | "bytes" 5 | "github.com/golang/protobuf/proto" 6 | "github.com/stretchr/testify/assert" 7 | "io/ioutil" 8 | "net/http" 9 | "net/http/httptest" 10 | "statefun-sdk-go/pkg/statefun/internal/protocol" 11 | "testing" 12 | "time" 13 | ) 14 | 15 | var Seen = ValueSpec{ 16 | Name: "seen", 17 | ValueType: Int32Type, 18 | } 19 | 20 | func greeter(ctx Context, msg Message) error { 21 | if msg.IsString() { 22 | _ = msg.AsString() 23 | } 24 | 25 | var seen int32 26 | ctx.Storage().Get(Seen, &seen) 27 | seen += 1 28 | ctx.Storage().Set(Seen, seen) 29 | 30 | ctx.Send(MessageBuilder{ 31 | Target: Address{ 32 | FunctionType: TypeNameFrom("org.foo/greeter-java"), 33 | Id: "0", 34 | }, 35 | Value: seen, 36 | }) 37 | 38 | ctx.SendAfter(time.Duration(1)*time.Hour, MessageBuilder{ 39 | Target: Address{ 40 | FunctionType: TypeNameFrom("night/owl"), 41 | Id: "1", 42 | }, 43 | Value: "hoo hoo", 44 | }) 45 | 46 | ctx.SendEgress(KafkaEgressBuilder{ 47 | Target: TypeNameFrom("e/kafka"), 48 | Topic: "out", 49 | Key: "abc", 50 | Value: int32(133742), 51 | }) 52 | 53 | ctx.SendEgress(KinesisEgressBuilder{ 54 | Target: TypeNameFrom("e/kinesis"), 55 | Stream: "out", 56 | Value: "hello there", 57 | PartitionKey: "abc", 58 | }) 59 | 60 | return nil 61 | } 62 | 63 | func TestMissingStateValues(t *testing.T) { 64 | builder := StatefulFunctionsBuilder() 65 | err := builder.WithSpec(StatefulFunctionSpec{ 66 | FunctionType: TypeNameFrom("org.foo/greeter"), 67 | States: []ValueSpec{Seen}, 68 | Function: StatefulFunctionPointer(greeter), 69 | }) 70 | 71 | assert.NoError(t, err, "registering a function should succeed") 72 | 73 | server := httptest.NewServer(builder.AsHandler()) 74 | defer server.Close() 75 | 76 | toFunction := protocol.ToFunction{ 77 | Request: &protocol.ToFunction_Invocation_{ 78 | Invocation: &protocol.ToFunction_InvocationBatchRequest{ 79 | Target: &protocol.Address{ 80 | Namespace: "org.foo", 81 | Type: "greeter", 82 | Id: "0", 83 | }, 84 | State: nil, 85 | Invocations: []*protocol.ToFunction_Invocation{ 86 | { 87 | Caller: nil, 88 | Argument: toTypedValue(StringType, "Hello"), 89 | }, 90 | }, 91 | }, 92 | }, 93 | } 94 | 95 | request, _ := proto.Marshal(&toFunction) 96 | response, err := http.Post(server.URL, "application/octet-stream", bytes.NewReader(request)) 97 | 98 | assert.NoError(t, err) 99 | assert.Equal(t, http.StatusOK, response.StatusCode, "received non-200 response") 100 | 101 | var from protocol.FromFunction 102 | responsebody, _ := ioutil.ReadAll(response.Body) 103 | _ = proto.Unmarshal(responsebody, &from) 104 | 105 | ctx := from.GetIncompleteInvocationContext() 106 | assert.NotNil(t, ctx, "missing state context should not be nil") 107 | assert.Equal(t, "seen", ctx.MissingValues[0].StateName) 108 | } 109 | 110 | func TestHandler(t *testing.T) { 111 | builder := StatefulFunctionsBuilder() 112 | err := builder.WithSpec(StatefulFunctionSpec{ 113 | FunctionType: TypeNameFrom("org.foo/greeter"), 114 | States: []ValueSpec{Seen}, 115 | Function: StatefulFunctionPointer(greeter), 116 | }) 117 | 118 | assert.NoError(t, err, "registering a function should succeed") 119 | server := httptest.NewServer(builder.AsHandler()) 120 | defer server.Close() 121 | 122 | toFunction := protocol.ToFunction{ 123 | Request: &protocol.ToFunction_Invocation_{ 124 | Invocation: &protocol.ToFunction_InvocationBatchRequest{ 125 | Target: &protocol.Address{ 126 | Namespace: "org.foo", 127 | Type: "greeter", 128 | Id: "0", 129 | }, 130 | State: []*protocol.ToFunction_PersistedValue{ 131 | { 132 | StateName: "seen", 133 | StateValue: &protocol.TypedValue{ 134 | Typename: "io.statefun.types/int", 135 | HasValue: false, 136 | Value: nil, 137 | }, 138 | }, 139 | }, 140 | Invocations: []*protocol.ToFunction_Invocation{ 141 | { 142 | Caller: nil, 143 | Argument: toTypedValue(StringType, "Hello"), 144 | }, 145 | }, 146 | }, 147 | }, 148 | } 149 | 150 | request, _ := proto.Marshal(&toFunction) 151 | response, err := http.Post(server.URL, "application/octet-stream", bytes.NewReader(request)) 152 | 153 | assert.NoError(t, err) 154 | assert.Equal(t, http.StatusOK, response.StatusCode, "received non-200 response") 155 | 156 | var from protocol.FromFunction 157 | responsebody, _ := ioutil.ReadAll(response.Body) 158 | _ = proto.Unmarshal(responsebody, &from) 159 | 160 | result := from.GetInvocationResult() 161 | assert.NotNil(t, result, "invocation result should not be nil") 162 | 163 | assert.Equal(t, "seen", result.StateMutations[0].StateName) 164 | assert.Equal(t, protocol.FromFunction_PersistedValueMutation_MODIFY, result.StateMutations[0].MutationType) 165 | 166 | assert.Equal(t, &protocol.Address{ 167 | Namespace: "org.foo", 168 | Type: "greeter-java", 169 | Id: "0", 170 | }, result.OutgoingMessages[0].Target) 171 | assert.Equal(t, "io.statefun.types/int", result.OutgoingMessages[0].Argument.Typename) 172 | 173 | assert.Equal(t, int64(1000*60*60), result.DelayedInvocations[0].DelayInMs) 174 | assert.Equal(t, "io.statefun.types/string", result.DelayedInvocations[0].Argument.Typename) 175 | 176 | assert.Equal(t, "e", result.OutgoingEgresses[0].EgressNamespace) 177 | assert.Equal(t, "kafka", result.OutgoingEgresses[0].EgressType) 178 | assert.Equal(t, "type.googleapis.com/io.statefun.sdk.egress.KafkaProducerRecord", result.OutgoingEgresses[0].Argument.Typename) 179 | } 180 | 181 | func BenchmarkHandler(t *testing.B) { 182 | builder := StatefulFunctionsBuilder() 183 | _ = builder.WithSpec(StatefulFunctionSpec{ 184 | FunctionType: TypeNameFrom("org.foo/greeter"), 185 | States: []ValueSpec{Seen}, 186 | Function: StatefulFunctionPointer(greeter), 187 | }) 188 | 189 | server := httptest.NewServer(builder.AsHandler()) 190 | defer server.Close() 191 | 192 | toFunction := protocol.ToFunction{ 193 | Request: &protocol.ToFunction_Invocation_{ 194 | Invocation: &protocol.ToFunction_InvocationBatchRequest{ 195 | Target: &protocol.Address{ 196 | Namespace: "org.foo", 197 | Type: "greeter", 198 | Id: "0", 199 | }, 200 | State: []*protocol.ToFunction_PersistedValue{ 201 | { 202 | StateName: "seen", 203 | StateValue: &protocol.TypedValue{ 204 | Typename: "io.statefun.types/int", 205 | HasValue: false, 206 | Value: nil, 207 | }, 208 | }, 209 | }, 210 | Invocations: []*protocol.ToFunction_Invocation{ 211 | { 212 | Caller: nil, 213 | Argument: toTypedValue(StringType, "Hello"), 214 | }, 215 | }, 216 | }, 217 | }, 218 | } 219 | 220 | request, _ := proto.Marshal(&toFunction) 221 | 222 | t.ReportAllocs() 223 | 224 | for i := 0; i < t.N; i++ { 225 | response, _ := http.Post(server.URL, "application/octet-stream", bytes.NewReader(request)) 226 | _ = response.Body.Close() 227 | } 228 | } 229 | 230 | func toTypedValue(valueType SimpleType, value interface{}) *protocol.TypedValue { 231 | buffer := bytes.Buffer{} 232 | if err := valueType.Serialize(&buffer, value); err != nil { 233 | panic(err) 234 | } 235 | 236 | return &protocol.TypedValue{ 237 | Typename: valueType.GetTypeName().String(), 238 | HasValue: true, 239 | Value: buffer.Bytes(), 240 | } 241 | } 242 | -------------------------------------------------------------------------------- /pkg/statefun/internal/cell.go: -------------------------------------------------------------------------------- 1 | package internal 2 | 3 | import ( 4 | "bytes" 5 | "statefun-sdk-go/pkg/statefun/internal/protocol" 6 | ) 7 | 8 | // A mutable persisted value. 9 | // This struct is not thread safe. 10 | type Cell struct { 11 | typedValue *protocol.TypedValue 12 | buffer bytes.Buffer 13 | mutated bool 14 | } 15 | 16 | func NewCell(state *protocol.ToFunction_PersistedValue) *Cell { 17 | c := &Cell{ 18 | typedValue: state.StateValue, 19 | } 20 | 21 | _, _ = c.buffer.Read(c.typedValue.Value) 22 | c.typedValue.Value = nil 23 | return c 24 | } 25 | 26 | func (c *Cell) Read(p []byte) (n int, err error) { 27 | return c.buffer.Read(p) 28 | } 29 | 30 | func (c *Cell) Write(p []byte) (n int, err error) { 31 | c.buffer.Reset() 32 | c.mutated = true 33 | c.typedValue.HasValue = true 34 | return c.buffer.Write(p) 35 | } 36 | 37 | func (c *Cell) Reset() { 38 | c.mutated = true 39 | c.typedValue.HasValue = false 40 | c.buffer.Reset() 41 | } 42 | 43 | func (c Cell) HasValue() bool { 44 | return c.typedValue.HasValue 45 | } 46 | 47 | func (c Cell) GetStateMutation(name string) *protocol.FromFunction_PersistedValueMutation { 48 | if !c.mutated { 49 | return nil 50 | } 51 | 52 | mutationType := protocol.FromFunction_PersistedValueMutation_DELETE 53 | if c.typedValue.HasValue { 54 | mutationType = protocol.FromFunction_PersistedValueMutation_MODIFY 55 | c.typedValue.Value = c.buffer.Bytes() 56 | } 57 | 58 | return &protocol.FromFunction_PersistedValueMutation{ 59 | MutationType: mutationType, 60 | StateName: name, 61 | StateValue: c.typedValue, 62 | } 63 | } 64 | -------------------------------------------------------------------------------- /pkg/statefun/internal/protocol/kafka-egress.pb.go: -------------------------------------------------------------------------------- 1 | // 2 | // Licensed to the Apache Software Foundation (ASF) under one 3 | // or more contributor license agreements. See the NOTICE file 4 | // distributed with this work for additional information 5 | // regarding copyright ownership. The ASF licenses this file 6 | // to you under the Apache License, Version 2.0 (the 7 | // "License"); you may not use this file except in compliance 8 | // with the License. You may obtain a copy of the License at 9 | // 10 | // http://www.apache.org/licenses/LICENSE-2.0 11 | // 12 | // Unless required by applicable law or agreed to in writing, software 13 | // distributed under the License is distributed on an "AS IS" BASIS, 14 | // WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | // See the License for the specific language governing permissions and 16 | // limitations under the License. 17 | 18 | // Code generated by protoc-gen-go. DO NOT EDIT. 19 | // versions: 20 | // protoc-gen-go v1.26.0 21 | // protoc v3.14.0 22 | // source: kafka-egress.proto 23 | 24 | package protocol 25 | 26 | import ( 27 | protoreflect "google.golang.org/protobuf/reflect/protoreflect" 28 | protoimpl "google.golang.org/protobuf/runtime/protoimpl" 29 | reflect "reflect" 30 | sync "sync" 31 | ) 32 | 33 | const ( 34 | // Verify that this generated code is sufficiently up-to-date. 35 | _ = protoimpl.EnforceVersion(20 - protoimpl.MinVersion) 36 | // Verify that runtime/protoimpl is sufficiently up-to-date. 37 | _ = protoimpl.EnforceVersion(protoimpl.MaxVersion - 20) 38 | ) 39 | 40 | type KafkaProducerRecord struct { 41 | state protoimpl.MessageState 42 | sizeCache protoimpl.SizeCache 43 | unknownFields protoimpl.UnknownFields 44 | 45 | Key string `protobuf:"bytes,1,opt,name=key,proto3" json:"key,omitempty"` 46 | ValueBytes []byte `protobuf:"bytes,2,opt,name=value_bytes,json=valueBytes,proto3" json:"value_bytes,omitempty"` 47 | Topic string `protobuf:"bytes,3,opt,name=topic,proto3" json:"topic,omitempty"` 48 | } 49 | 50 | func (x *KafkaProducerRecord) Reset() { 51 | *x = KafkaProducerRecord{} 52 | if protoimpl.UnsafeEnabled { 53 | mi := &file_kafka_egress_proto_msgTypes[0] 54 | ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) 55 | ms.StoreMessageInfo(mi) 56 | } 57 | } 58 | 59 | func (x *KafkaProducerRecord) String() string { 60 | return protoimpl.X.MessageStringOf(x) 61 | } 62 | 63 | func (*KafkaProducerRecord) ProtoMessage() {} 64 | 65 | func (x *KafkaProducerRecord) ProtoReflect() protoreflect.Message { 66 | mi := &file_kafka_egress_proto_msgTypes[0] 67 | if protoimpl.UnsafeEnabled && x != nil { 68 | ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) 69 | if ms.LoadMessageInfo() == nil { 70 | ms.StoreMessageInfo(mi) 71 | } 72 | return ms 73 | } 74 | return mi.MessageOf(x) 75 | } 76 | 77 | // Deprecated: Use KafkaProducerRecord.ProtoReflect.Descriptor instead. 78 | func (*KafkaProducerRecord) Descriptor() ([]byte, []int) { 79 | return file_kafka_egress_proto_rawDescGZIP(), []int{0} 80 | } 81 | 82 | func (x *KafkaProducerRecord) GetKey() string { 83 | if x != nil { 84 | return x.Key 85 | } 86 | return "" 87 | } 88 | 89 | func (x *KafkaProducerRecord) GetValueBytes() []byte { 90 | if x != nil { 91 | return x.ValueBytes 92 | } 93 | return nil 94 | } 95 | 96 | func (x *KafkaProducerRecord) GetTopic() string { 97 | if x != nil { 98 | return x.Topic 99 | } 100 | return "" 101 | } 102 | 103 | var File_kafka_egress_proto protoreflect.FileDescriptor 104 | 105 | var file_kafka_egress_proto_rawDesc = []byte{ 106 | 0x0a, 0x12, 0x6b, 0x61, 0x66, 0x6b, 0x61, 0x2d, 0x65, 0x67, 0x72, 0x65, 0x73, 0x73, 0x2e, 0x70, 107 | 0x72, 0x6f, 0x74, 0x6f, 0x12, 0x16, 0x69, 0x6f, 0x2e, 0x73, 0x74, 0x61, 0x74, 0x65, 0x66, 0x75, 108 | 0x6e, 0x2e, 0x73, 0x64, 0x6b, 0x2e, 0x65, 0x67, 0x72, 0x65, 0x73, 0x73, 0x22, 0x5e, 0x0a, 0x13, 109 | 0x4b, 0x61, 0x66, 0x6b, 0x61, 0x50, 0x72, 0x6f, 0x64, 0x75, 0x63, 0x65, 0x72, 0x52, 0x65, 0x63, 110 | 0x6f, 0x72, 0x64, 0x12, 0x10, 0x0a, 0x03, 0x6b, 0x65, 0x79, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 111 | 0x52, 0x03, 0x6b, 0x65, 0x79, 0x12, 0x1f, 0x0a, 0x0b, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x5f, 0x62, 112 | 0x79, 0x74, 0x65, 0x73, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0c, 0x52, 0x0a, 0x76, 0x61, 0x6c, 0x75, 113 | 0x65, 0x42, 0x79, 0x74, 0x65, 0x73, 0x12, 0x14, 0x0a, 0x05, 0x74, 0x6f, 0x70, 0x69, 0x63, 0x18, 114 | 0x03, 0x20, 0x01, 0x28, 0x09, 0x52, 0x05, 0x74, 0x6f, 0x70, 0x69, 0x63, 0x42, 0x3e, 0x0a, 0x2e, 115 | 0x6f, 0x72, 0x67, 0x2e, 0x61, 0x70, 0x61, 0x63, 0x68, 0x65, 0x2e, 0x66, 0x6c, 0x69, 0x6e, 0x6b, 116 | 0x2e, 0x73, 0x74, 0x61, 0x74, 0x65, 0x66, 0x75, 0x6e, 0x2e, 0x73, 0x64, 0x6b, 0x2e, 0x65, 0x67, 117 | 0x72, 0x65, 0x73, 0x73, 0x2e, 0x67, 0x65, 0x6e, 0x65, 0x72, 0x61, 0x74, 0x65, 0x64, 0x50, 0x01, 118 | 0x5a, 0x0a, 0x2e, 0x2f, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x63, 0x6f, 0x6c, 0x62, 0x06, 0x70, 0x72, 119 | 0x6f, 0x74, 0x6f, 0x33, 120 | } 121 | 122 | var ( 123 | file_kafka_egress_proto_rawDescOnce sync.Once 124 | file_kafka_egress_proto_rawDescData = file_kafka_egress_proto_rawDesc 125 | ) 126 | 127 | func file_kafka_egress_proto_rawDescGZIP() []byte { 128 | file_kafka_egress_proto_rawDescOnce.Do(func() { 129 | file_kafka_egress_proto_rawDescData = protoimpl.X.CompressGZIP(file_kafka_egress_proto_rawDescData) 130 | }) 131 | return file_kafka_egress_proto_rawDescData 132 | } 133 | 134 | var file_kafka_egress_proto_msgTypes = make([]protoimpl.MessageInfo, 1) 135 | var file_kafka_egress_proto_goTypes = []interface{}{ 136 | (*KafkaProducerRecord)(nil), // 0: io.statefun.sdk.egress.KafkaProducerRecord 137 | } 138 | var file_kafka_egress_proto_depIdxs = []int32{ 139 | 0, // [0:0] is the sub-list for method output_type 140 | 0, // [0:0] is the sub-list for method input_type 141 | 0, // [0:0] is the sub-list for extension type_name 142 | 0, // [0:0] is the sub-list for extension extendee 143 | 0, // [0:0] is the sub-list for field type_name 144 | } 145 | 146 | func init() { file_kafka_egress_proto_init() } 147 | func file_kafka_egress_proto_init() { 148 | if File_kafka_egress_proto != nil { 149 | return 150 | } 151 | if !protoimpl.UnsafeEnabled { 152 | file_kafka_egress_proto_msgTypes[0].Exporter = func(v interface{}, i int) interface{} { 153 | switch v := v.(*KafkaProducerRecord); i { 154 | case 0: 155 | return &v.state 156 | case 1: 157 | return &v.sizeCache 158 | case 2: 159 | return &v.unknownFields 160 | default: 161 | return nil 162 | } 163 | } 164 | } 165 | type x struct{} 166 | out := protoimpl.TypeBuilder{ 167 | File: protoimpl.DescBuilder{ 168 | GoPackagePath: reflect.TypeOf(x{}).PkgPath(), 169 | RawDescriptor: file_kafka_egress_proto_rawDesc, 170 | NumEnums: 0, 171 | NumMessages: 1, 172 | NumExtensions: 0, 173 | NumServices: 0, 174 | }, 175 | GoTypes: file_kafka_egress_proto_goTypes, 176 | DependencyIndexes: file_kafka_egress_proto_depIdxs, 177 | MessageInfos: file_kafka_egress_proto_msgTypes, 178 | }.Build() 179 | File_kafka_egress_proto = out.File 180 | file_kafka_egress_proto_rawDesc = nil 181 | file_kafka_egress_proto_goTypes = nil 182 | file_kafka_egress_proto_depIdxs = nil 183 | } 184 | -------------------------------------------------------------------------------- /pkg/statefun/internal/protocol/kinesis-egress.pb.go: -------------------------------------------------------------------------------- 1 | // 2 | // Licensed to the Apache Software Foundation (ASF) under one 3 | // or more contributor license agreements. See the NOTICE file 4 | // distributed with this work for additional information 5 | // regarding copyright ownership. The ASF licenses this file 6 | // to you under the Apache License, Version 2.0 (the 7 | // "License"); you may not use this file except in compliance 8 | // with the License. You may obtain a copy of the License at 9 | // 10 | // http://www.apache.org/licenses/LICENSE-2.0 11 | // 12 | // Unless required by applicable law or agreed to in writing, software 13 | // distributed under the License is distributed on an "AS IS" BASIS, 14 | // WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | // See the License for the specific language governing permissions and 16 | // limitations under the License. 17 | 18 | // Code generated by protoc-gen-go. DO NOT EDIT. 19 | // versions: 20 | // protoc-gen-go v1.26.0 21 | // protoc v3.14.0 22 | // source: kinesis-egress.proto 23 | 24 | package protocol 25 | 26 | import ( 27 | protoreflect "google.golang.org/protobuf/reflect/protoreflect" 28 | protoimpl "google.golang.org/protobuf/runtime/protoimpl" 29 | reflect "reflect" 30 | sync "sync" 31 | ) 32 | 33 | const ( 34 | // Verify that this generated code is sufficiently up-to-date. 35 | _ = protoimpl.EnforceVersion(20 - protoimpl.MinVersion) 36 | // Verify that runtime/protoimpl is sufficiently up-to-date. 37 | _ = protoimpl.EnforceVersion(protoimpl.MaxVersion - 20) 38 | ) 39 | 40 | type KinesisEgressRecord struct { 41 | state protoimpl.MessageState 42 | sizeCache protoimpl.SizeCache 43 | unknownFields protoimpl.UnknownFields 44 | 45 | PartitionKey string `protobuf:"bytes,1,opt,name=partition_key,json=partitionKey,proto3" json:"partition_key,omitempty"` 46 | ValueBytes []byte `protobuf:"bytes,2,opt,name=value_bytes,json=valueBytes,proto3" json:"value_bytes,omitempty"` 47 | Stream string `protobuf:"bytes,3,opt,name=stream,proto3" json:"stream,omitempty"` 48 | ExplicitHashKey string `protobuf:"bytes,4,opt,name=explicit_hash_key,json=explicitHashKey,proto3" json:"explicit_hash_key,omitempty"` 49 | } 50 | 51 | func (x *KinesisEgressRecord) Reset() { 52 | *x = KinesisEgressRecord{} 53 | if protoimpl.UnsafeEnabled { 54 | mi := &file_kinesis_egress_proto_msgTypes[0] 55 | ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) 56 | ms.StoreMessageInfo(mi) 57 | } 58 | } 59 | 60 | func (x *KinesisEgressRecord) String() string { 61 | return protoimpl.X.MessageStringOf(x) 62 | } 63 | 64 | func (*KinesisEgressRecord) ProtoMessage() {} 65 | 66 | func (x *KinesisEgressRecord) ProtoReflect() protoreflect.Message { 67 | mi := &file_kinesis_egress_proto_msgTypes[0] 68 | if protoimpl.UnsafeEnabled && x != nil { 69 | ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) 70 | if ms.LoadMessageInfo() == nil { 71 | ms.StoreMessageInfo(mi) 72 | } 73 | return ms 74 | } 75 | return mi.MessageOf(x) 76 | } 77 | 78 | // Deprecated: Use KinesisEgressRecord.ProtoReflect.Descriptor instead. 79 | func (*KinesisEgressRecord) Descriptor() ([]byte, []int) { 80 | return file_kinesis_egress_proto_rawDescGZIP(), []int{0} 81 | } 82 | 83 | func (x *KinesisEgressRecord) GetPartitionKey() string { 84 | if x != nil { 85 | return x.PartitionKey 86 | } 87 | return "" 88 | } 89 | 90 | func (x *KinesisEgressRecord) GetValueBytes() []byte { 91 | if x != nil { 92 | return x.ValueBytes 93 | } 94 | return nil 95 | } 96 | 97 | func (x *KinesisEgressRecord) GetStream() string { 98 | if x != nil { 99 | return x.Stream 100 | } 101 | return "" 102 | } 103 | 104 | func (x *KinesisEgressRecord) GetExplicitHashKey() string { 105 | if x != nil { 106 | return x.ExplicitHashKey 107 | } 108 | return "" 109 | } 110 | 111 | var File_kinesis_egress_proto protoreflect.FileDescriptor 112 | 113 | var file_kinesis_egress_proto_rawDesc = []byte{ 114 | 0x0a, 0x14, 0x6b, 0x69, 0x6e, 0x65, 0x73, 0x69, 0x73, 0x2d, 0x65, 0x67, 0x72, 0x65, 0x73, 0x73, 115 | 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x12, 0x16, 0x69, 0x6f, 0x2e, 0x73, 0x74, 0x61, 0x74, 0x65, 116 | 0x66, 0x75, 0x6e, 0x2e, 0x73, 0x64, 0x6b, 0x2e, 0x65, 0x67, 0x72, 0x65, 0x73, 0x73, 0x22, 0x9f, 117 | 0x01, 0x0a, 0x13, 0x4b, 0x69, 0x6e, 0x65, 0x73, 0x69, 0x73, 0x45, 0x67, 0x72, 0x65, 0x73, 0x73, 118 | 0x52, 0x65, 0x63, 0x6f, 0x72, 0x64, 0x12, 0x23, 0x0a, 0x0d, 0x70, 0x61, 0x72, 0x74, 0x69, 0x74, 119 | 0x69, 0x6f, 0x6e, 0x5f, 0x6b, 0x65, 0x79, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0c, 0x70, 120 | 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x4b, 0x65, 0x79, 0x12, 0x1f, 0x0a, 0x0b, 0x76, 121 | 0x61, 0x6c, 0x75, 0x65, 0x5f, 0x62, 0x79, 0x74, 0x65, 0x73, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0c, 122 | 0x52, 0x0a, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x42, 0x79, 0x74, 0x65, 0x73, 0x12, 0x16, 0x0a, 0x06, 123 | 0x73, 0x74, 0x72, 0x65, 0x61, 0x6d, 0x18, 0x03, 0x20, 0x01, 0x28, 0x09, 0x52, 0x06, 0x73, 0x74, 124 | 0x72, 0x65, 0x61, 0x6d, 0x12, 0x2a, 0x0a, 0x11, 0x65, 0x78, 0x70, 0x6c, 0x69, 0x63, 0x69, 0x74, 125 | 0x5f, 0x68, 0x61, 0x73, 0x68, 0x5f, 0x6b, 0x65, 0x79, 0x18, 0x04, 0x20, 0x01, 0x28, 0x09, 0x52, 126 | 0x0f, 0x65, 0x78, 0x70, 0x6c, 0x69, 0x63, 0x69, 0x74, 0x48, 0x61, 0x73, 0x68, 0x4b, 0x65, 0x79, 127 | 0x42, 0x3e, 0x0a, 0x2e, 0x6f, 0x72, 0x67, 0x2e, 0x61, 0x70, 0x61, 0x63, 0x68, 0x65, 0x2e, 0x66, 128 | 0x6c, 0x69, 0x6e, 0x6b, 0x2e, 0x73, 0x74, 0x61, 0x74, 0x65, 0x66, 0x75, 0x6e, 0x2e, 0x73, 0x64, 129 | 0x6b, 0x2e, 0x65, 0x67, 0x72, 0x65, 0x73, 0x73, 0x2e, 0x67, 0x65, 0x6e, 0x65, 0x72, 0x61, 0x74, 130 | 0x65, 0x64, 0x50, 0x01, 0x5a, 0x0a, 0x2e, 0x2f, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x63, 0x6f, 0x6c, 131 | 0x62, 0x06, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x33, 132 | } 133 | 134 | var ( 135 | file_kinesis_egress_proto_rawDescOnce sync.Once 136 | file_kinesis_egress_proto_rawDescData = file_kinesis_egress_proto_rawDesc 137 | ) 138 | 139 | func file_kinesis_egress_proto_rawDescGZIP() []byte { 140 | file_kinesis_egress_proto_rawDescOnce.Do(func() { 141 | file_kinesis_egress_proto_rawDescData = protoimpl.X.CompressGZIP(file_kinesis_egress_proto_rawDescData) 142 | }) 143 | return file_kinesis_egress_proto_rawDescData 144 | } 145 | 146 | var file_kinesis_egress_proto_msgTypes = make([]protoimpl.MessageInfo, 1) 147 | var file_kinesis_egress_proto_goTypes = []interface{}{ 148 | (*KinesisEgressRecord)(nil), // 0: io.statefun.sdk.egress.KinesisEgressRecord 149 | } 150 | var file_kinesis_egress_proto_depIdxs = []int32{ 151 | 0, // [0:0] is the sub-list for method output_type 152 | 0, // [0:0] is the sub-list for method input_type 153 | 0, // [0:0] is the sub-list for extension type_name 154 | 0, // [0:0] is the sub-list for extension extendee 155 | 0, // [0:0] is the sub-list for field type_name 156 | } 157 | 158 | func init() { file_kinesis_egress_proto_init() } 159 | func file_kinesis_egress_proto_init() { 160 | if File_kinesis_egress_proto != nil { 161 | return 162 | } 163 | if !protoimpl.UnsafeEnabled { 164 | file_kinesis_egress_proto_msgTypes[0].Exporter = func(v interface{}, i int) interface{} { 165 | switch v := v.(*KinesisEgressRecord); i { 166 | case 0: 167 | return &v.state 168 | case 1: 169 | return &v.sizeCache 170 | case 2: 171 | return &v.unknownFields 172 | default: 173 | return nil 174 | } 175 | } 176 | } 177 | type x struct{} 178 | out := protoimpl.TypeBuilder{ 179 | File: protoimpl.DescBuilder{ 180 | GoPackagePath: reflect.TypeOf(x{}).PkgPath(), 181 | RawDescriptor: file_kinesis_egress_proto_rawDesc, 182 | NumEnums: 0, 183 | NumMessages: 1, 184 | NumExtensions: 0, 185 | NumServices: 0, 186 | }, 187 | GoTypes: file_kinesis_egress_proto_goTypes, 188 | DependencyIndexes: file_kinesis_egress_proto_depIdxs, 189 | MessageInfos: file_kinesis_egress_proto_msgTypes, 190 | }.Build() 191 | File_kinesis_egress_proto = out.File 192 | file_kinesis_egress_proto_rawDesc = nil 193 | file_kinesis_egress_proto_goTypes = nil 194 | file_kinesis_egress_proto_depIdxs = nil 195 | } 196 | -------------------------------------------------------------------------------- /pkg/statefun/internal/protocol/request-reply.pb.go: -------------------------------------------------------------------------------- 1 | // 2 | // Licensed to the Apache Software Foundation (ASF) under one 3 | // or more contributor license agreements. See the NOTICE file 4 | // distributed with this work for additional information 5 | // regarding copyright ownership. The ASF licenses this file 6 | // to you under the Apache License, Version 2.0 (the 7 | // "License"); you may not use this file except in compliance 8 | // with the License. You may obtain a copy of the License at 9 | // 10 | // http://www.apache.org/licenses/LICENSE-2.0 11 | // 12 | // Unless required by applicable law or agreed to in writing, software 13 | // distributed under the License is distributed on an "AS IS" BASIS, 14 | // WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | // See the License for the specific language governing permissions and 16 | // limitations under the License. 17 | 18 | // Code generated by protoc-gen-go. DO NOT EDIT. 19 | // versions: 20 | // protoc-gen-go v1.26.0 21 | // protoc v3.14.0 22 | // source: request-reply.proto 23 | 24 | package protocol 25 | 26 | import ( 27 | protoreflect "google.golang.org/protobuf/reflect/protoreflect" 28 | protoimpl "google.golang.org/protobuf/runtime/protoimpl" 29 | reflect "reflect" 30 | sync "sync" 31 | ) 32 | 33 | const ( 34 | // Verify that this generated code is sufficiently up-to-date. 35 | _ = protoimpl.EnforceVersion(20 - protoimpl.MinVersion) 36 | // Verify that runtime/protoimpl is sufficiently up-to-date. 37 | _ = protoimpl.EnforceVersion(protoimpl.MaxVersion - 20) 38 | ) 39 | 40 | type FromFunction_PersistedValueMutation_MutationType int32 41 | 42 | const ( 43 | FromFunction_PersistedValueMutation_DELETE FromFunction_PersistedValueMutation_MutationType = 0 44 | FromFunction_PersistedValueMutation_MODIFY FromFunction_PersistedValueMutation_MutationType = 1 45 | ) 46 | 47 | // Enum value maps for FromFunction_PersistedValueMutation_MutationType. 48 | var ( 49 | FromFunction_PersistedValueMutation_MutationType_name = map[int32]string{ 50 | 0: "DELETE", 51 | 1: "MODIFY", 52 | } 53 | FromFunction_PersistedValueMutation_MutationType_value = map[string]int32{ 54 | "DELETE": 0, 55 | "MODIFY": 1, 56 | } 57 | ) 58 | 59 | func (x FromFunction_PersistedValueMutation_MutationType) Enum() *FromFunction_PersistedValueMutation_MutationType { 60 | p := new(FromFunction_PersistedValueMutation_MutationType) 61 | *p = x 62 | return p 63 | } 64 | 65 | func (x FromFunction_PersistedValueMutation_MutationType) String() string { 66 | return protoimpl.X.EnumStringOf(x.Descriptor(), protoreflect.EnumNumber(x)) 67 | } 68 | 69 | func (FromFunction_PersistedValueMutation_MutationType) Descriptor() protoreflect.EnumDescriptor { 70 | return file_request_reply_proto_enumTypes[0].Descriptor() 71 | } 72 | 73 | func (FromFunction_PersistedValueMutation_MutationType) Type() protoreflect.EnumType { 74 | return &file_request_reply_proto_enumTypes[0] 75 | } 76 | 77 | func (x FromFunction_PersistedValueMutation_MutationType) Number() protoreflect.EnumNumber { 78 | return protoreflect.EnumNumber(x) 79 | } 80 | 81 | // Deprecated: Use FromFunction_PersistedValueMutation_MutationType.Descriptor instead. 82 | func (FromFunction_PersistedValueMutation_MutationType) EnumDescriptor() ([]byte, []int) { 83 | return file_request_reply_proto_rawDescGZIP(), []int{3, 0, 0} 84 | } 85 | 86 | type FromFunction_ExpirationSpec_ExpireMode int32 87 | 88 | const ( 89 | FromFunction_ExpirationSpec_NONE FromFunction_ExpirationSpec_ExpireMode = 0 90 | FromFunction_ExpirationSpec_AFTER_WRITE FromFunction_ExpirationSpec_ExpireMode = 1 91 | FromFunction_ExpirationSpec_AFTER_INVOKE FromFunction_ExpirationSpec_ExpireMode = 2 92 | ) 93 | 94 | // Enum value maps for FromFunction_ExpirationSpec_ExpireMode. 95 | var ( 96 | FromFunction_ExpirationSpec_ExpireMode_name = map[int32]string{ 97 | 0: "NONE", 98 | 1: "AFTER_WRITE", 99 | 2: "AFTER_INVOKE", 100 | } 101 | FromFunction_ExpirationSpec_ExpireMode_value = map[string]int32{ 102 | "NONE": 0, 103 | "AFTER_WRITE": 1, 104 | "AFTER_INVOKE": 2, 105 | } 106 | ) 107 | 108 | func (x FromFunction_ExpirationSpec_ExpireMode) Enum() *FromFunction_ExpirationSpec_ExpireMode { 109 | p := new(FromFunction_ExpirationSpec_ExpireMode) 110 | *p = x 111 | return p 112 | } 113 | 114 | func (x FromFunction_ExpirationSpec_ExpireMode) String() string { 115 | return protoimpl.X.EnumStringOf(x.Descriptor(), protoreflect.EnumNumber(x)) 116 | } 117 | 118 | func (FromFunction_ExpirationSpec_ExpireMode) Descriptor() protoreflect.EnumDescriptor { 119 | return file_request_reply_proto_enumTypes[1].Descriptor() 120 | } 121 | 122 | func (FromFunction_ExpirationSpec_ExpireMode) Type() protoreflect.EnumType { 123 | return &file_request_reply_proto_enumTypes[1] 124 | } 125 | 126 | func (x FromFunction_ExpirationSpec_ExpireMode) Number() protoreflect.EnumNumber { 127 | return protoreflect.EnumNumber(x) 128 | } 129 | 130 | // Deprecated: Use FromFunction_ExpirationSpec_ExpireMode.Descriptor instead. 131 | func (FromFunction_ExpirationSpec_ExpireMode) EnumDescriptor() ([]byte, []int) { 132 | return file_request_reply_proto_rawDescGZIP(), []int{3, 5, 0} 133 | } 134 | 135 | // An Address is the unique identity of an individual StatefulFunction, containing 136 | // a function's type and an unique identifier within the type. The function's 137 | // type denotes the "class" of function to invoke, while the unique identifier addresses the 138 | // invocation to a specific function instance. 139 | type Address struct { 140 | state protoimpl.MessageState 141 | sizeCache protoimpl.SizeCache 142 | unknownFields protoimpl.UnknownFields 143 | 144 | Namespace string `protobuf:"bytes,1,opt,name=namespace,proto3" json:"namespace,omitempty"` 145 | Type string `protobuf:"bytes,2,opt,name=type,proto3" json:"type,omitempty"` 146 | Id string `protobuf:"bytes,3,opt,name=id,proto3" json:"id,omitempty"` 147 | } 148 | 149 | func (x *Address) Reset() { 150 | *x = Address{} 151 | if protoimpl.UnsafeEnabled { 152 | mi := &file_request_reply_proto_msgTypes[0] 153 | ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) 154 | ms.StoreMessageInfo(mi) 155 | } 156 | } 157 | 158 | func (x *Address) String() string { 159 | return protoimpl.X.MessageStringOf(x) 160 | } 161 | 162 | func (*Address) ProtoMessage() {} 163 | 164 | func (x *Address) ProtoReflect() protoreflect.Message { 165 | mi := &file_request_reply_proto_msgTypes[0] 166 | if protoimpl.UnsafeEnabled && x != nil { 167 | ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) 168 | if ms.LoadMessageInfo() == nil { 169 | ms.StoreMessageInfo(mi) 170 | } 171 | return ms 172 | } 173 | return mi.MessageOf(x) 174 | } 175 | 176 | // Deprecated: Use Address.ProtoReflect.Descriptor instead. 177 | func (*Address) Descriptor() ([]byte, []int) { 178 | return file_request_reply_proto_rawDescGZIP(), []int{0} 179 | } 180 | 181 | func (x *Address) GetNamespace() string { 182 | if x != nil { 183 | return x.Namespace 184 | } 185 | return "" 186 | } 187 | 188 | func (x *Address) GetType() string { 189 | if x != nil { 190 | return x.Type 191 | } 192 | return "" 193 | } 194 | 195 | func (x *Address) GetId() string { 196 | if x != nil { 197 | return x.Id 198 | } 199 | return "" 200 | } 201 | 202 | type TypedValue struct { 203 | state protoimpl.MessageState 204 | sizeCache protoimpl.SizeCache 205 | unknownFields protoimpl.UnknownFields 206 | 207 | Typename string `protobuf:"bytes,1,opt,name=typename,proto3" json:"typename,omitempty"` 208 | // has_value is set to differentiate a zero length value bytes explicitly set, 209 | // or a non existing value. 210 | HasValue bool `protobuf:"varint,2,opt,name=has_value,json=hasValue,proto3" json:"has_value,omitempty"` 211 | Value []byte `protobuf:"bytes,3,opt,name=value,proto3" json:"value,omitempty"` 212 | } 213 | 214 | func (x *TypedValue) Reset() { 215 | *x = TypedValue{} 216 | if protoimpl.UnsafeEnabled { 217 | mi := &file_request_reply_proto_msgTypes[1] 218 | ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) 219 | ms.StoreMessageInfo(mi) 220 | } 221 | } 222 | 223 | func (x *TypedValue) String() string { 224 | return protoimpl.X.MessageStringOf(x) 225 | } 226 | 227 | func (*TypedValue) ProtoMessage() {} 228 | 229 | func (x *TypedValue) ProtoReflect() protoreflect.Message { 230 | mi := &file_request_reply_proto_msgTypes[1] 231 | if protoimpl.UnsafeEnabled && x != nil { 232 | ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) 233 | if ms.LoadMessageInfo() == nil { 234 | ms.StoreMessageInfo(mi) 235 | } 236 | return ms 237 | } 238 | return mi.MessageOf(x) 239 | } 240 | 241 | // Deprecated: Use TypedValue.ProtoReflect.Descriptor instead. 242 | func (*TypedValue) Descriptor() ([]byte, []int) { 243 | return file_request_reply_proto_rawDescGZIP(), []int{1} 244 | } 245 | 246 | func (x *TypedValue) GetTypename() string { 247 | if x != nil { 248 | return x.Typename 249 | } 250 | return "" 251 | } 252 | 253 | func (x *TypedValue) GetHasValue() bool { 254 | if x != nil { 255 | return x.HasValue 256 | } 257 | return false 258 | } 259 | 260 | func (x *TypedValue) GetValue() []byte { 261 | if x != nil { 262 | return x.Value 263 | } 264 | return nil 265 | } 266 | 267 | // The following section contains all the message types that are sent 268 | // from Flink to a remote function. 269 | type ToFunction struct { 270 | state protoimpl.MessageState 271 | sizeCache protoimpl.SizeCache 272 | unknownFields protoimpl.UnknownFields 273 | 274 | // Types that are assignable to Request: 275 | // *ToFunction_Invocation_ 276 | Request isToFunction_Request `protobuf_oneof:"request"` 277 | } 278 | 279 | func (x *ToFunction) Reset() { 280 | *x = ToFunction{} 281 | if protoimpl.UnsafeEnabled { 282 | mi := &file_request_reply_proto_msgTypes[2] 283 | ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) 284 | ms.StoreMessageInfo(mi) 285 | } 286 | } 287 | 288 | func (x *ToFunction) String() string { 289 | return protoimpl.X.MessageStringOf(x) 290 | } 291 | 292 | func (*ToFunction) ProtoMessage() {} 293 | 294 | func (x *ToFunction) ProtoReflect() protoreflect.Message { 295 | mi := &file_request_reply_proto_msgTypes[2] 296 | if protoimpl.UnsafeEnabled && x != nil { 297 | ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) 298 | if ms.LoadMessageInfo() == nil { 299 | ms.StoreMessageInfo(mi) 300 | } 301 | return ms 302 | } 303 | return mi.MessageOf(x) 304 | } 305 | 306 | // Deprecated: Use ToFunction.ProtoReflect.Descriptor instead. 307 | func (*ToFunction) Descriptor() ([]byte, []int) { 308 | return file_request_reply_proto_rawDescGZIP(), []int{2} 309 | } 310 | 311 | func (m *ToFunction) GetRequest() isToFunction_Request { 312 | if m != nil { 313 | return m.Request 314 | } 315 | return nil 316 | } 317 | 318 | func (x *ToFunction) GetInvocation() *ToFunction_InvocationBatchRequest { 319 | if x, ok := x.GetRequest().(*ToFunction_Invocation_); ok { 320 | return x.Invocation 321 | } 322 | return nil 323 | } 324 | 325 | type isToFunction_Request interface { 326 | isToFunction_Request() 327 | } 328 | 329 | type ToFunction_Invocation_ struct { 330 | Invocation *ToFunction_InvocationBatchRequest `protobuf:"bytes,100,opt,name=invocation,proto3,oneof"` 331 | } 332 | 333 | func (*ToFunction_Invocation_) isToFunction_Request() {} 334 | 335 | // The following section contains messages sent from a remote function back to Flink. 336 | type FromFunction struct { 337 | state protoimpl.MessageState 338 | sizeCache protoimpl.SizeCache 339 | unknownFields protoimpl.UnknownFields 340 | 341 | // Response sent from the function, as a result of an io.statefun.sdk.reqreply.ToFunction.InvocationBatchRequest. 342 | // It can be one of the following types: 343 | // - io.statefun.sdk.reqreply.FromFunction.InvocationResponse 344 | // - io.statefun.sdk.reqreply.FromFunction.IncompleteInvocationContext 345 | // 346 | // Types that are assignable to Response: 347 | // *FromFunction_InvocationResult 348 | // *FromFunction_IncompleteInvocationContext_ 349 | Response isFromFunction_Response `protobuf_oneof:"response"` 350 | } 351 | 352 | func (x *FromFunction) Reset() { 353 | *x = FromFunction{} 354 | if protoimpl.UnsafeEnabled { 355 | mi := &file_request_reply_proto_msgTypes[3] 356 | ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) 357 | ms.StoreMessageInfo(mi) 358 | } 359 | } 360 | 361 | func (x *FromFunction) String() string { 362 | return protoimpl.X.MessageStringOf(x) 363 | } 364 | 365 | func (*FromFunction) ProtoMessage() {} 366 | 367 | func (x *FromFunction) ProtoReflect() protoreflect.Message { 368 | mi := &file_request_reply_proto_msgTypes[3] 369 | if protoimpl.UnsafeEnabled && x != nil { 370 | ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) 371 | if ms.LoadMessageInfo() == nil { 372 | ms.StoreMessageInfo(mi) 373 | } 374 | return ms 375 | } 376 | return mi.MessageOf(x) 377 | } 378 | 379 | // Deprecated: Use FromFunction.ProtoReflect.Descriptor instead. 380 | func (*FromFunction) Descriptor() ([]byte, []int) { 381 | return file_request_reply_proto_rawDescGZIP(), []int{3} 382 | } 383 | 384 | func (m *FromFunction) GetResponse() isFromFunction_Response { 385 | if m != nil { 386 | return m.Response 387 | } 388 | return nil 389 | } 390 | 391 | func (x *FromFunction) GetInvocationResult() *FromFunction_InvocationResponse { 392 | if x, ok := x.GetResponse().(*FromFunction_InvocationResult); ok { 393 | return x.InvocationResult 394 | } 395 | return nil 396 | } 397 | 398 | func (x *FromFunction) GetIncompleteInvocationContext() *FromFunction_IncompleteInvocationContext { 399 | if x, ok := x.GetResponse().(*FromFunction_IncompleteInvocationContext_); ok { 400 | return x.IncompleteInvocationContext 401 | } 402 | return nil 403 | } 404 | 405 | type isFromFunction_Response interface { 406 | isFromFunction_Response() 407 | } 408 | 409 | type FromFunction_InvocationResult struct { 410 | InvocationResult *FromFunction_InvocationResponse `protobuf:"bytes,100,opt,name=invocation_result,json=invocationResult,proto3,oneof"` 411 | } 412 | 413 | type FromFunction_IncompleteInvocationContext_ struct { 414 | IncompleteInvocationContext *FromFunction_IncompleteInvocationContext `protobuf:"bytes,101,opt,name=incomplete_invocation_context,json=incompleteInvocationContext,proto3,oneof"` 415 | } 416 | 417 | func (*FromFunction_InvocationResult) isFromFunction_Response() {} 418 | 419 | func (*FromFunction_IncompleteInvocationContext_) isFromFunction_Response() {} 420 | 421 | // PersistedValue represents a PersistedValue's value that is managed by Flink on behalf of a remote function. 422 | type ToFunction_PersistedValue struct { 423 | state protoimpl.MessageState 424 | sizeCache protoimpl.SizeCache 425 | unknownFields protoimpl.UnknownFields 426 | 427 | // The unique name of the persisted state. 428 | StateName string `protobuf:"bytes,1,opt,name=state_name,json=stateName,proto3" json:"state_name,omitempty"` 429 | // The serialized state value 430 | StateValue *TypedValue `protobuf:"bytes,2,opt,name=state_value,json=stateValue,proto3" json:"state_value,omitempty"` 431 | } 432 | 433 | func (x *ToFunction_PersistedValue) Reset() { 434 | *x = ToFunction_PersistedValue{} 435 | if protoimpl.UnsafeEnabled { 436 | mi := &file_request_reply_proto_msgTypes[4] 437 | ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) 438 | ms.StoreMessageInfo(mi) 439 | } 440 | } 441 | 442 | func (x *ToFunction_PersistedValue) String() string { 443 | return protoimpl.X.MessageStringOf(x) 444 | } 445 | 446 | func (*ToFunction_PersistedValue) ProtoMessage() {} 447 | 448 | func (x *ToFunction_PersistedValue) ProtoReflect() protoreflect.Message { 449 | mi := &file_request_reply_proto_msgTypes[4] 450 | if protoimpl.UnsafeEnabled && x != nil { 451 | ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) 452 | if ms.LoadMessageInfo() == nil { 453 | ms.StoreMessageInfo(mi) 454 | } 455 | return ms 456 | } 457 | return mi.MessageOf(x) 458 | } 459 | 460 | // Deprecated: Use ToFunction_PersistedValue.ProtoReflect.Descriptor instead. 461 | func (*ToFunction_PersistedValue) Descriptor() ([]byte, []int) { 462 | return file_request_reply_proto_rawDescGZIP(), []int{2, 0} 463 | } 464 | 465 | func (x *ToFunction_PersistedValue) GetStateName() string { 466 | if x != nil { 467 | return x.StateName 468 | } 469 | return "" 470 | } 471 | 472 | func (x *ToFunction_PersistedValue) GetStateValue() *TypedValue { 473 | if x != nil { 474 | return x.StateValue 475 | } 476 | return nil 477 | } 478 | 479 | // Invocation represents a remote function call, it associated with an (optional) return address, 480 | // and an argument. 481 | type ToFunction_Invocation struct { 482 | state protoimpl.MessageState 483 | sizeCache protoimpl.SizeCache 484 | unknownFields protoimpl.UnknownFields 485 | 486 | // The address of the function that requested the invocation (possibly absent) 487 | Caller *Address `protobuf:"bytes,1,opt,name=caller,proto3" json:"caller,omitempty"` 488 | // The invocation argument (aka the message sent to the target function) 489 | Argument *TypedValue `protobuf:"bytes,2,opt,name=argument,proto3" json:"argument,omitempty"` 490 | } 491 | 492 | func (x *ToFunction_Invocation) Reset() { 493 | *x = ToFunction_Invocation{} 494 | if protoimpl.UnsafeEnabled { 495 | mi := &file_request_reply_proto_msgTypes[5] 496 | ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) 497 | ms.StoreMessageInfo(mi) 498 | } 499 | } 500 | 501 | func (x *ToFunction_Invocation) String() string { 502 | return protoimpl.X.MessageStringOf(x) 503 | } 504 | 505 | func (*ToFunction_Invocation) ProtoMessage() {} 506 | 507 | func (x *ToFunction_Invocation) ProtoReflect() protoreflect.Message { 508 | mi := &file_request_reply_proto_msgTypes[5] 509 | if protoimpl.UnsafeEnabled && x != nil { 510 | ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) 511 | if ms.LoadMessageInfo() == nil { 512 | ms.StoreMessageInfo(mi) 513 | } 514 | return ms 515 | } 516 | return mi.MessageOf(x) 517 | } 518 | 519 | // Deprecated: Use ToFunction_Invocation.ProtoReflect.Descriptor instead. 520 | func (*ToFunction_Invocation) Descriptor() ([]byte, []int) { 521 | return file_request_reply_proto_rawDescGZIP(), []int{2, 1} 522 | } 523 | 524 | func (x *ToFunction_Invocation) GetCaller() *Address { 525 | if x != nil { 526 | return x.Caller 527 | } 528 | return nil 529 | } 530 | 531 | func (x *ToFunction_Invocation) GetArgument() *TypedValue { 532 | if x != nil { 533 | return x.Argument 534 | } 535 | return nil 536 | } 537 | 538 | // InvocationBatchRequest represents a request to invoke a remote function. It is always associated with a target 539 | // address (the function to invoke), and a list of values for registered state. 540 | type ToFunction_InvocationBatchRequest struct { 541 | state protoimpl.MessageState 542 | sizeCache protoimpl.SizeCache 543 | unknownFields protoimpl.UnknownFields 544 | 545 | // The address of the function to invoke 546 | Target *Address `protobuf:"bytes,1,opt,name=target,proto3" json:"target,omitempty"` 547 | // A list of PersistedValues that were registered as a persisted state. 548 | State []*ToFunction_PersistedValue `protobuf:"bytes,2,rep,name=state,proto3" json:"state,omitempty"` 549 | // A non empty (at least one) list of invocations 550 | Invocations []*ToFunction_Invocation `protobuf:"bytes,3,rep,name=invocations,proto3" json:"invocations,omitempty"` 551 | } 552 | 553 | func (x *ToFunction_InvocationBatchRequest) Reset() { 554 | *x = ToFunction_InvocationBatchRequest{} 555 | if protoimpl.UnsafeEnabled { 556 | mi := &file_request_reply_proto_msgTypes[6] 557 | ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) 558 | ms.StoreMessageInfo(mi) 559 | } 560 | } 561 | 562 | func (x *ToFunction_InvocationBatchRequest) String() string { 563 | return protoimpl.X.MessageStringOf(x) 564 | } 565 | 566 | func (*ToFunction_InvocationBatchRequest) ProtoMessage() {} 567 | 568 | func (x *ToFunction_InvocationBatchRequest) ProtoReflect() protoreflect.Message { 569 | mi := &file_request_reply_proto_msgTypes[6] 570 | if protoimpl.UnsafeEnabled && x != nil { 571 | ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) 572 | if ms.LoadMessageInfo() == nil { 573 | ms.StoreMessageInfo(mi) 574 | } 575 | return ms 576 | } 577 | return mi.MessageOf(x) 578 | } 579 | 580 | // Deprecated: Use ToFunction_InvocationBatchRequest.ProtoReflect.Descriptor instead. 581 | func (*ToFunction_InvocationBatchRequest) Descriptor() ([]byte, []int) { 582 | return file_request_reply_proto_rawDescGZIP(), []int{2, 2} 583 | } 584 | 585 | func (x *ToFunction_InvocationBatchRequest) GetTarget() *Address { 586 | if x != nil { 587 | return x.Target 588 | } 589 | return nil 590 | } 591 | 592 | func (x *ToFunction_InvocationBatchRequest) GetState() []*ToFunction_PersistedValue { 593 | if x != nil { 594 | return x.State 595 | } 596 | return nil 597 | } 598 | 599 | func (x *ToFunction_InvocationBatchRequest) GetInvocations() []*ToFunction_Invocation { 600 | if x != nil { 601 | return x.Invocations 602 | } 603 | return nil 604 | } 605 | 606 | // MutatePersistedValueCommand represents a command sent from a remote function to Flink, 607 | // requesting a change to a persisted value. 608 | type FromFunction_PersistedValueMutation struct { 609 | state protoimpl.MessageState 610 | sizeCache protoimpl.SizeCache 611 | unknownFields protoimpl.UnknownFields 612 | 613 | MutationType FromFunction_PersistedValueMutation_MutationType `protobuf:"varint,1,opt,name=mutation_type,json=mutationType,proto3,enum=io.statefun.sdk.reqreply.FromFunction_PersistedValueMutation_MutationType" json:"mutation_type,omitempty"` 614 | StateName string `protobuf:"bytes,2,opt,name=state_name,json=stateName,proto3" json:"state_name,omitempty"` 615 | StateValue *TypedValue `protobuf:"bytes,3,opt,name=state_value,json=stateValue,proto3" json:"state_value,omitempty"` 616 | } 617 | 618 | func (x *FromFunction_PersistedValueMutation) Reset() { 619 | *x = FromFunction_PersistedValueMutation{} 620 | if protoimpl.UnsafeEnabled { 621 | mi := &file_request_reply_proto_msgTypes[7] 622 | ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) 623 | ms.StoreMessageInfo(mi) 624 | } 625 | } 626 | 627 | func (x *FromFunction_PersistedValueMutation) String() string { 628 | return protoimpl.X.MessageStringOf(x) 629 | } 630 | 631 | func (*FromFunction_PersistedValueMutation) ProtoMessage() {} 632 | 633 | func (x *FromFunction_PersistedValueMutation) ProtoReflect() protoreflect.Message { 634 | mi := &file_request_reply_proto_msgTypes[7] 635 | if protoimpl.UnsafeEnabled && x != nil { 636 | ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) 637 | if ms.LoadMessageInfo() == nil { 638 | ms.StoreMessageInfo(mi) 639 | } 640 | return ms 641 | } 642 | return mi.MessageOf(x) 643 | } 644 | 645 | // Deprecated: Use FromFunction_PersistedValueMutation.ProtoReflect.Descriptor instead. 646 | func (*FromFunction_PersistedValueMutation) Descriptor() ([]byte, []int) { 647 | return file_request_reply_proto_rawDescGZIP(), []int{3, 0} 648 | } 649 | 650 | func (x *FromFunction_PersistedValueMutation) GetMutationType() FromFunction_PersistedValueMutation_MutationType { 651 | if x != nil { 652 | return x.MutationType 653 | } 654 | return FromFunction_PersistedValueMutation_DELETE 655 | } 656 | 657 | func (x *FromFunction_PersistedValueMutation) GetStateName() string { 658 | if x != nil { 659 | return x.StateName 660 | } 661 | return "" 662 | } 663 | 664 | func (x *FromFunction_PersistedValueMutation) GetStateValue() *TypedValue { 665 | if x != nil { 666 | return x.StateValue 667 | } 668 | return nil 669 | } 670 | 671 | // Invocation represents a remote function call, it associated with a (mandatory) target address, 672 | // and an argument. 673 | type FromFunction_Invocation struct { 674 | state protoimpl.MessageState 675 | sizeCache protoimpl.SizeCache 676 | unknownFields protoimpl.UnknownFields 677 | 678 | // The target function to invoke 679 | Target *Address `protobuf:"bytes,1,opt,name=target,proto3" json:"target,omitempty"` 680 | // The invocation argument (aka the message sent to the target function) 681 | Argument *TypedValue `protobuf:"bytes,2,opt,name=argument,proto3" json:"argument,omitempty"` 682 | } 683 | 684 | func (x *FromFunction_Invocation) Reset() { 685 | *x = FromFunction_Invocation{} 686 | if protoimpl.UnsafeEnabled { 687 | mi := &file_request_reply_proto_msgTypes[8] 688 | ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) 689 | ms.StoreMessageInfo(mi) 690 | } 691 | } 692 | 693 | func (x *FromFunction_Invocation) String() string { 694 | return protoimpl.X.MessageStringOf(x) 695 | } 696 | 697 | func (*FromFunction_Invocation) ProtoMessage() {} 698 | 699 | func (x *FromFunction_Invocation) ProtoReflect() protoreflect.Message { 700 | mi := &file_request_reply_proto_msgTypes[8] 701 | if protoimpl.UnsafeEnabled && x != nil { 702 | ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) 703 | if ms.LoadMessageInfo() == nil { 704 | ms.StoreMessageInfo(mi) 705 | } 706 | return ms 707 | } 708 | return mi.MessageOf(x) 709 | } 710 | 711 | // Deprecated: Use FromFunction_Invocation.ProtoReflect.Descriptor instead. 712 | func (*FromFunction_Invocation) Descriptor() ([]byte, []int) { 713 | return file_request_reply_proto_rawDescGZIP(), []int{3, 1} 714 | } 715 | 716 | func (x *FromFunction_Invocation) GetTarget() *Address { 717 | if x != nil { 718 | return x.Target 719 | } 720 | return nil 721 | } 722 | 723 | func (x *FromFunction_Invocation) GetArgument() *TypedValue { 724 | if x != nil { 725 | return x.Argument 726 | } 727 | return nil 728 | } 729 | 730 | // DelayedInvocation represents a delayed remote function call with a target address, an argument 731 | // and a delay in milliseconds, after which this message to be sent. 732 | type FromFunction_DelayedInvocation struct { 733 | state protoimpl.MessageState 734 | sizeCache protoimpl.SizeCache 735 | unknownFields protoimpl.UnknownFields 736 | 737 | // the amount of milliseconds to wait before sending this message 738 | DelayInMs int64 `protobuf:"varint,1,opt,name=delay_in_ms,json=delayInMs,proto3" json:"delay_in_ms,omitempty"` 739 | // the target address to send this message to 740 | Target *Address `protobuf:"bytes,2,opt,name=target,proto3" json:"target,omitempty"` 741 | // the invocation argument 742 | Argument *TypedValue `protobuf:"bytes,3,opt,name=argument,proto3" json:"argument,omitempty"` 743 | } 744 | 745 | func (x *FromFunction_DelayedInvocation) Reset() { 746 | *x = FromFunction_DelayedInvocation{} 747 | if protoimpl.UnsafeEnabled { 748 | mi := &file_request_reply_proto_msgTypes[9] 749 | ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) 750 | ms.StoreMessageInfo(mi) 751 | } 752 | } 753 | 754 | func (x *FromFunction_DelayedInvocation) String() string { 755 | return protoimpl.X.MessageStringOf(x) 756 | } 757 | 758 | func (*FromFunction_DelayedInvocation) ProtoMessage() {} 759 | 760 | func (x *FromFunction_DelayedInvocation) ProtoReflect() protoreflect.Message { 761 | mi := &file_request_reply_proto_msgTypes[9] 762 | if protoimpl.UnsafeEnabled && x != nil { 763 | ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) 764 | if ms.LoadMessageInfo() == nil { 765 | ms.StoreMessageInfo(mi) 766 | } 767 | return ms 768 | } 769 | return mi.MessageOf(x) 770 | } 771 | 772 | // Deprecated: Use FromFunction_DelayedInvocation.ProtoReflect.Descriptor instead. 773 | func (*FromFunction_DelayedInvocation) Descriptor() ([]byte, []int) { 774 | return file_request_reply_proto_rawDescGZIP(), []int{3, 2} 775 | } 776 | 777 | func (x *FromFunction_DelayedInvocation) GetDelayInMs() int64 { 778 | if x != nil { 779 | return x.DelayInMs 780 | } 781 | return 0 782 | } 783 | 784 | func (x *FromFunction_DelayedInvocation) GetTarget() *Address { 785 | if x != nil { 786 | return x.Target 787 | } 788 | return nil 789 | } 790 | 791 | func (x *FromFunction_DelayedInvocation) GetArgument() *TypedValue { 792 | if x != nil { 793 | return x.Argument 794 | } 795 | return nil 796 | } 797 | 798 | // EgressMessage an argument to forward to an egress. 799 | // An egress is identified by a namespace and type (see EgressIdentifier SDK class). 800 | // The argument is an io.statefun.sdk.reqreply.TypedValue. 801 | type FromFunction_EgressMessage struct { 802 | state protoimpl.MessageState 803 | sizeCache protoimpl.SizeCache 804 | unknownFields protoimpl.UnknownFields 805 | 806 | // The target egress namespace 807 | EgressNamespace string `protobuf:"bytes,1,opt,name=egress_namespace,json=egressNamespace,proto3" json:"egress_namespace,omitempty"` 808 | // The target egress type 809 | EgressType string `protobuf:"bytes,2,opt,name=egress_type,json=egressType,proto3" json:"egress_type,omitempty"` 810 | // egress argument 811 | Argument *TypedValue `protobuf:"bytes,3,opt,name=argument,proto3" json:"argument,omitempty"` 812 | } 813 | 814 | func (x *FromFunction_EgressMessage) Reset() { 815 | *x = FromFunction_EgressMessage{} 816 | if protoimpl.UnsafeEnabled { 817 | mi := &file_request_reply_proto_msgTypes[10] 818 | ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) 819 | ms.StoreMessageInfo(mi) 820 | } 821 | } 822 | 823 | func (x *FromFunction_EgressMessage) String() string { 824 | return protoimpl.X.MessageStringOf(x) 825 | } 826 | 827 | func (*FromFunction_EgressMessage) ProtoMessage() {} 828 | 829 | func (x *FromFunction_EgressMessage) ProtoReflect() protoreflect.Message { 830 | mi := &file_request_reply_proto_msgTypes[10] 831 | if protoimpl.UnsafeEnabled && x != nil { 832 | ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) 833 | if ms.LoadMessageInfo() == nil { 834 | ms.StoreMessageInfo(mi) 835 | } 836 | return ms 837 | } 838 | return mi.MessageOf(x) 839 | } 840 | 841 | // Deprecated: Use FromFunction_EgressMessage.ProtoReflect.Descriptor instead. 842 | func (*FromFunction_EgressMessage) Descriptor() ([]byte, []int) { 843 | return file_request_reply_proto_rawDescGZIP(), []int{3, 3} 844 | } 845 | 846 | func (x *FromFunction_EgressMessage) GetEgressNamespace() string { 847 | if x != nil { 848 | return x.EgressNamespace 849 | } 850 | return "" 851 | } 852 | 853 | func (x *FromFunction_EgressMessage) GetEgressType() string { 854 | if x != nil { 855 | return x.EgressType 856 | } 857 | return "" 858 | } 859 | 860 | func (x *FromFunction_EgressMessage) GetArgument() *TypedValue { 861 | if x != nil { 862 | return x.Argument 863 | } 864 | return nil 865 | } 866 | 867 | // InvocationResponse represents a result of an io.statefun.sdk.reqreply.ToFunction.InvocationBatchRequest 868 | // it contains a list of state mutation to preform as a result of computing this batch, and a list of outgoing messages. 869 | type FromFunction_InvocationResponse struct { 870 | state protoimpl.MessageState 871 | sizeCache protoimpl.SizeCache 872 | unknownFields protoimpl.UnknownFields 873 | 874 | StateMutations []*FromFunction_PersistedValueMutation `protobuf:"bytes,1,rep,name=state_mutations,json=stateMutations,proto3" json:"state_mutations,omitempty"` 875 | OutgoingMessages []*FromFunction_Invocation `protobuf:"bytes,2,rep,name=outgoing_messages,json=outgoingMessages,proto3" json:"outgoing_messages,omitempty"` 876 | DelayedInvocations []*FromFunction_DelayedInvocation `protobuf:"bytes,3,rep,name=delayed_invocations,json=delayedInvocations,proto3" json:"delayed_invocations,omitempty"` 877 | OutgoingEgresses []*FromFunction_EgressMessage `protobuf:"bytes,4,rep,name=outgoing_egresses,json=outgoingEgresses,proto3" json:"outgoing_egresses,omitempty"` 878 | } 879 | 880 | func (x *FromFunction_InvocationResponse) Reset() { 881 | *x = FromFunction_InvocationResponse{} 882 | if protoimpl.UnsafeEnabled { 883 | mi := &file_request_reply_proto_msgTypes[11] 884 | ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) 885 | ms.StoreMessageInfo(mi) 886 | } 887 | } 888 | 889 | func (x *FromFunction_InvocationResponse) String() string { 890 | return protoimpl.X.MessageStringOf(x) 891 | } 892 | 893 | func (*FromFunction_InvocationResponse) ProtoMessage() {} 894 | 895 | func (x *FromFunction_InvocationResponse) ProtoReflect() protoreflect.Message { 896 | mi := &file_request_reply_proto_msgTypes[11] 897 | if protoimpl.UnsafeEnabled && x != nil { 898 | ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) 899 | if ms.LoadMessageInfo() == nil { 900 | ms.StoreMessageInfo(mi) 901 | } 902 | return ms 903 | } 904 | return mi.MessageOf(x) 905 | } 906 | 907 | // Deprecated: Use FromFunction_InvocationResponse.ProtoReflect.Descriptor instead. 908 | func (*FromFunction_InvocationResponse) Descriptor() ([]byte, []int) { 909 | return file_request_reply_proto_rawDescGZIP(), []int{3, 4} 910 | } 911 | 912 | func (x *FromFunction_InvocationResponse) GetStateMutations() []*FromFunction_PersistedValueMutation { 913 | if x != nil { 914 | return x.StateMutations 915 | } 916 | return nil 917 | } 918 | 919 | func (x *FromFunction_InvocationResponse) GetOutgoingMessages() []*FromFunction_Invocation { 920 | if x != nil { 921 | return x.OutgoingMessages 922 | } 923 | return nil 924 | } 925 | 926 | func (x *FromFunction_InvocationResponse) GetDelayedInvocations() []*FromFunction_DelayedInvocation { 927 | if x != nil { 928 | return x.DelayedInvocations 929 | } 930 | return nil 931 | } 932 | 933 | func (x *FromFunction_InvocationResponse) GetOutgoingEgresses() []*FromFunction_EgressMessage { 934 | if x != nil { 935 | return x.OutgoingEgresses 936 | } 937 | return nil 938 | } 939 | 940 | // ExpirationSpec represents TTL (Time-To-Live) configuration for persisted states. 941 | type FromFunction_ExpirationSpec struct { 942 | state protoimpl.MessageState 943 | sizeCache protoimpl.SizeCache 944 | unknownFields protoimpl.UnknownFields 945 | 946 | Mode FromFunction_ExpirationSpec_ExpireMode `protobuf:"varint,1,opt,name=mode,proto3,enum=io.statefun.sdk.reqreply.FromFunction_ExpirationSpec_ExpireMode" json:"mode,omitempty"` 947 | ExpireAfterMillis int64 `protobuf:"varint,2,opt,name=expire_after_millis,json=expireAfterMillis,proto3" json:"expire_after_millis,omitempty"` 948 | } 949 | 950 | func (x *FromFunction_ExpirationSpec) Reset() { 951 | *x = FromFunction_ExpirationSpec{} 952 | if protoimpl.UnsafeEnabled { 953 | mi := &file_request_reply_proto_msgTypes[12] 954 | ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) 955 | ms.StoreMessageInfo(mi) 956 | } 957 | } 958 | 959 | func (x *FromFunction_ExpirationSpec) String() string { 960 | return protoimpl.X.MessageStringOf(x) 961 | } 962 | 963 | func (*FromFunction_ExpirationSpec) ProtoMessage() {} 964 | 965 | func (x *FromFunction_ExpirationSpec) ProtoReflect() protoreflect.Message { 966 | mi := &file_request_reply_proto_msgTypes[12] 967 | if protoimpl.UnsafeEnabled && x != nil { 968 | ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) 969 | if ms.LoadMessageInfo() == nil { 970 | ms.StoreMessageInfo(mi) 971 | } 972 | return ms 973 | } 974 | return mi.MessageOf(x) 975 | } 976 | 977 | // Deprecated: Use FromFunction_ExpirationSpec.ProtoReflect.Descriptor instead. 978 | func (*FromFunction_ExpirationSpec) Descriptor() ([]byte, []int) { 979 | return file_request_reply_proto_rawDescGZIP(), []int{3, 5} 980 | } 981 | 982 | func (x *FromFunction_ExpirationSpec) GetMode() FromFunction_ExpirationSpec_ExpireMode { 983 | if x != nil { 984 | return x.Mode 985 | } 986 | return FromFunction_ExpirationSpec_NONE 987 | } 988 | 989 | func (x *FromFunction_ExpirationSpec) GetExpireAfterMillis() int64 { 990 | if x != nil { 991 | return x.ExpireAfterMillis 992 | } 993 | return 0 994 | } 995 | 996 | // PersistedValueSpec represents specifications of a function's persisted value state. 997 | type FromFunction_PersistedValueSpec struct { 998 | state protoimpl.MessageState 999 | sizeCache protoimpl.SizeCache 1000 | unknownFields protoimpl.UnknownFields 1001 | 1002 | StateName string `protobuf:"bytes,1,opt,name=state_name,json=stateName,proto3" json:"state_name,omitempty"` 1003 | ExpirationSpec *FromFunction_ExpirationSpec `protobuf:"bytes,2,opt,name=expiration_spec,json=expirationSpec,proto3" json:"expiration_spec,omitempty"` 1004 | TypeTypename string `protobuf:"bytes,3,opt,name=type_typename,json=typeTypename,proto3" json:"type_typename,omitempty"` 1005 | } 1006 | 1007 | func (x *FromFunction_PersistedValueSpec) Reset() { 1008 | *x = FromFunction_PersistedValueSpec{} 1009 | if protoimpl.UnsafeEnabled { 1010 | mi := &file_request_reply_proto_msgTypes[13] 1011 | ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) 1012 | ms.StoreMessageInfo(mi) 1013 | } 1014 | } 1015 | 1016 | func (x *FromFunction_PersistedValueSpec) String() string { 1017 | return protoimpl.X.MessageStringOf(x) 1018 | } 1019 | 1020 | func (*FromFunction_PersistedValueSpec) ProtoMessage() {} 1021 | 1022 | func (x *FromFunction_PersistedValueSpec) ProtoReflect() protoreflect.Message { 1023 | mi := &file_request_reply_proto_msgTypes[13] 1024 | if protoimpl.UnsafeEnabled && x != nil { 1025 | ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) 1026 | if ms.LoadMessageInfo() == nil { 1027 | ms.StoreMessageInfo(mi) 1028 | } 1029 | return ms 1030 | } 1031 | return mi.MessageOf(x) 1032 | } 1033 | 1034 | // Deprecated: Use FromFunction_PersistedValueSpec.ProtoReflect.Descriptor instead. 1035 | func (*FromFunction_PersistedValueSpec) Descriptor() ([]byte, []int) { 1036 | return file_request_reply_proto_rawDescGZIP(), []int{3, 6} 1037 | } 1038 | 1039 | func (x *FromFunction_PersistedValueSpec) GetStateName() string { 1040 | if x != nil { 1041 | return x.StateName 1042 | } 1043 | return "" 1044 | } 1045 | 1046 | func (x *FromFunction_PersistedValueSpec) GetExpirationSpec() *FromFunction_ExpirationSpec { 1047 | if x != nil { 1048 | return x.ExpirationSpec 1049 | } 1050 | return nil 1051 | } 1052 | 1053 | func (x *FromFunction_PersistedValueSpec) GetTypeTypename() string { 1054 | if x != nil { 1055 | return x.TypeTypename 1056 | } 1057 | return "" 1058 | } 1059 | 1060 | // IncompleteInvocationContext represents a result of an io.statefun.sdk.reqreply.ToFunction.InvocationBatchRequest, 1061 | // which should be used as the response if the InvocationBatchRequest provided incomplete information about the 1062 | // invocation, e.g. insufficient state values were provided. 1063 | type FromFunction_IncompleteInvocationContext struct { 1064 | state protoimpl.MessageState 1065 | sizeCache protoimpl.SizeCache 1066 | unknownFields protoimpl.UnknownFields 1067 | 1068 | MissingValues []*FromFunction_PersistedValueSpec `protobuf:"bytes,1,rep,name=missing_values,json=missingValues,proto3" json:"missing_values,omitempty"` 1069 | } 1070 | 1071 | func (x *FromFunction_IncompleteInvocationContext) Reset() { 1072 | *x = FromFunction_IncompleteInvocationContext{} 1073 | if protoimpl.UnsafeEnabled { 1074 | mi := &file_request_reply_proto_msgTypes[14] 1075 | ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) 1076 | ms.StoreMessageInfo(mi) 1077 | } 1078 | } 1079 | 1080 | func (x *FromFunction_IncompleteInvocationContext) String() string { 1081 | return protoimpl.X.MessageStringOf(x) 1082 | } 1083 | 1084 | func (*FromFunction_IncompleteInvocationContext) ProtoMessage() {} 1085 | 1086 | func (x *FromFunction_IncompleteInvocationContext) ProtoReflect() protoreflect.Message { 1087 | mi := &file_request_reply_proto_msgTypes[14] 1088 | if protoimpl.UnsafeEnabled && x != nil { 1089 | ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) 1090 | if ms.LoadMessageInfo() == nil { 1091 | ms.StoreMessageInfo(mi) 1092 | } 1093 | return ms 1094 | } 1095 | return mi.MessageOf(x) 1096 | } 1097 | 1098 | // Deprecated: Use FromFunction_IncompleteInvocationContext.ProtoReflect.Descriptor instead. 1099 | func (*FromFunction_IncompleteInvocationContext) Descriptor() ([]byte, []int) { 1100 | return file_request_reply_proto_rawDescGZIP(), []int{3, 7} 1101 | } 1102 | 1103 | func (x *FromFunction_IncompleteInvocationContext) GetMissingValues() []*FromFunction_PersistedValueSpec { 1104 | if x != nil { 1105 | return x.MissingValues 1106 | } 1107 | return nil 1108 | } 1109 | 1110 | var File_request_reply_proto protoreflect.FileDescriptor 1111 | 1112 | var file_request_reply_proto_rawDesc = []byte{ 1113 | 0x0a, 0x13, 0x72, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x2d, 0x72, 0x65, 0x70, 0x6c, 0x79, 0x2e, 1114 | 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x12, 0x18, 0x69, 0x6f, 0x2e, 0x73, 0x74, 0x61, 0x74, 0x65, 0x66, 1115 | 0x75, 0x6e, 0x2e, 0x73, 0x64, 0x6b, 0x2e, 0x72, 0x65, 0x71, 0x72, 0x65, 0x70, 0x6c, 0x79, 0x22, 1116 | 0x4b, 0x0a, 0x07, 0x41, 0x64, 0x64, 0x72, 0x65, 0x73, 0x73, 0x12, 0x1c, 0x0a, 0x09, 0x6e, 0x61, 1117 | 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x09, 0x6e, 1118 | 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, 0x12, 0x12, 0x0a, 0x04, 0x74, 0x79, 0x70, 0x65, 1119 | 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x04, 0x74, 0x79, 0x70, 0x65, 0x12, 0x0e, 0x0a, 0x02, 1120 | 0x69, 0x64, 0x18, 0x03, 0x20, 0x01, 0x28, 0x09, 0x52, 0x02, 0x69, 0x64, 0x22, 0x5b, 0x0a, 0x0a, 1121 | 0x54, 0x79, 0x70, 0x65, 0x64, 0x56, 0x61, 0x6c, 0x75, 0x65, 0x12, 0x1a, 0x0a, 0x08, 0x74, 0x79, 1122 | 0x70, 0x65, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x08, 0x74, 0x79, 1123 | 0x70, 0x65, 0x6e, 0x61, 0x6d, 0x65, 0x12, 0x1b, 0x0a, 0x09, 0x68, 0x61, 0x73, 0x5f, 0x76, 0x61, 1124 | 0x6c, 0x75, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x08, 0x52, 0x08, 0x68, 0x61, 0x73, 0x56, 0x61, 1125 | 0x6c, 0x75, 0x65, 0x12, 0x14, 0x0a, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x18, 0x03, 0x20, 0x01, 1126 | 0x28, 0x0c, 0x52, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x22, 0xee, 0x04, 0x0a, 0x0a, 0x54, 0x6f, 1127 | 0x46, 0x75, 0x6e, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x5d, 0x0a, 0x0a, 0x69, 0x6e, 0x76, 0x6f, 1128 | 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x18, 0x64, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x3b, 0x2e, 0x69, 1129 | 0x6f, 0x2e, 0x73, 0x74, 0x61, 0x74, 0x65, 0x66, 0x75, 0x6e, 0x2e, 0x73, 0x64, 0x6b, 0x2e, 0x72, 1130 | 0x65, 0x71, 0x72, 0x65, 0x70, 0x6c, 0x79, 0x2e, 0x54, 0x6f, 0x46, 0x75, 0x6e, 0x63, 0x74, 0x69, 1131 | 0x6f, 0x6e, 0x2e, 0x49, 0x6e, 0x76, 0x6f, 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x42, 0x61, 0x74, 1132 | 0x63, 0x68, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x48, 0x00, 0x52, 0x0a, 0x69, 0x6e, 0x76, 1133 | 0x6f, 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x1a, 0x76, 0x0a, 0x0e, 0x50, 0x65, 0x72, 0x73, 0x69, 1134 | 0x73, 0x74, 0x65, 0x64, 0x56, 0x61, 0x6c, 0x75, 0x65, 0x12, 0x1d, 0x0a, 0x0a, 0x73, 0x74, 0x61, 1135 | 0x74, 0x65, 0x5f, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x09, 0x73, 1136 | 0x74, 0x61, 0x74, 0x65, 0x4e, 0x61, 0x6d, 0x65, 0x12, 0x45, 0x0a, 0x0b, 0x73, 0x74, 0x61, 0x74, 1137 | 0x65, 0x5f, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x24, 0x2e, 1138 | 0x69, 0x6f, 0x2e, 0x73, 0x74, 0x61, 0x74, 0x65, 0x66, 0x75, 0x6e, 0x2e, 0x73, 0x64, 0x6b, 0x2e, 1139 | 0x72, 0x65, 0x71, 0x72, 0x65, 0x70, 0x6c, 0x79, 0x2e, 0x54, 0x79, 0x70, 0x65, 0x64, 0x56, 0x61, 1140 | 0x6c, 0x75, 0x65, 0x52, 0x0a, 0x73, 0x74, 0x61, 0x74, 0x65, 0x56, 0x61, 0x6c, 0x75, 0x65, 0x1a, 1141 | 0x89, 0x01, 0x0a, 0x0a, 0x49, 0x6e, 0x76, 0x6f, 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x39, 1142 | 0x0a, 0x06, 0x63, 0x61, 0x6c, 0x6c, 0x65, 0x72, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x21, 1143 | 0x2e, 0x69, 0x6f, 0x2e, 0x73, 0x74, 0x61, 0x74, 0x65, 0x66, 0x75, 0x6e, 0x2e, 0x73, 0x64, 0x6b, 1144 | 0x2e, 0x72, 0x65, 0x71, 0x72, 0x65, 0x70, 0x6c, 0x79, 0x2e, 0x41, 0x64, 0x64, 0x72, 0x65, 0x73, 1145 | 0x73, 0x52, 0x06, 0x63, 0x61, 0x6c, 0x6c, 0x65, 0x72, 0x12, 0x40, 0x0a, 0x08, 0x61, 0x72, 0x67, 1146 | 0x75, 0x6d, 0x65, 0x6e, 0x74, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x24, 0x2e, 0x69, 0x6f, 1147 | 0x2e, 0x73, 0x74, 0x61, 0x74, 0x65, 0x66, 0x75, 0x6e, 0x2e, 0x73, 0x64, 0x6b, 0x2e, 0x72, 0x65, 1148 | 0x71, 0x72, 0x65, 0x70, 0x6c, 0x79, 0x2e, 0x54, 0x79, 0x70, 0x65, 0x64, 0x56, 0x61, 0x6c, 0x75, 1149 | 0x65, 0x52, 0x08, 0x61, 0x72, 0x67, 0x75, 0x6d, 0x65, 0x6e, 0x74, 0x1a, 0xf1, 0x01, 0x0a, 0x16, 1150 | 0x49, 0x6e, 0x76, 0x6f, 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x42, 0x61, 0x74, 0x63, 0x68, 0x52, 1151 | 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x39, 0x0a, 0x06, 0x74, 0x61, 0x72, 0x67, 0x65, 0x74, 1152 | 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x21, 0x2e, 0x69, 0x6f, 0x2e, 0x73, 0x74, 0x61, 0x74, 1153 | 0x65, 0x66, 0x75, 0x6e, 0x2e, 0x73, 0x64, 0x6b, 0x2e, 0x72, 0x65, 0x71, 0x72, 0x65, 0x70, 0x6c, 1154 | 0x79, 0x2e, 0x41, 0x64, 0x64, 0x72, 0x65, 0x73, 0x73, 0x52, 0x06, 0x74, 0x61, 0x72, 0x67, 0x65, 1155 | 0x74, 0x12, 0x49, 0x0a, 0x05, 0x73, 0x74, 0x61, 0x74, 0x65, 0x18, 0x02, 0x20, 0x03, 0x28, 0x0b, 1156 | 0x32, 0x33, 0x2e, 0x69, 0x6f, 0x2e, 0x73, 0x74, 0x61, 0x74, 0x65, 0x66, 0x75, 0x6e, 0x2e, 0x73, 1157 | 0x64, 0x6b, 0x2e, 0x72, 0x65, 0x71, 0x72, 0x65, 0x70, 0x6c, 0x79, 0x2e, 0x54, 0x6f, 0x46, 0x75, 1158 | 0x6e, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x2e, 0x50, 0x65, 0x72, 0x73, 0x69, 0x73, 0x74, 0x65, 0x64, 1159 | 0x56, 0x61, 0x6c, 0x75, 0x65, 0x52, 0x05, 0x73, 0x74, 0x61, 0x74, 0x65, 0x12, 0x51, 0x0a, 0x0b, 1160 | 0x69, 0x6e, 0x76, 0x6f, 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x18, 0x03, 0x20, 0x03, 0x28, 1161 | 0x0b, 0x32, 0x2f, 0x2e, 0x69, 0x6f, 0x2e, 0x73, 0x74, 0x61, 0x74, 0x65, 0x66, 0x75, 0x6e, 0x2e, 1162 | 0x73, 0x64, 0x6b, 0x2e, 0x72, 0x65, 0x71, 0x72, 0x65, 0x70, 0x6c, 0x79, 0x2e, 0x54, 0x6f, 0x46, 1163 | 0x75, 0x6e, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x2e, 0x49, 0x6e, 0x76, 0x6f, 0x63, 0x61, 0x74, 0x69, 1164 | 0x6f, 0x6e, 0x52, 0x0b, 0x69, 0x6e, 0x76, 0x6f, 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x42, 1165 | 0x09, 0x0a, 0x07, 0x72, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x22, 0xc5, 0x0f, 0x0a, 0x0c, 0x46, 1166 | 0x72, 0x6f, 0x6d, 0x46, 0x75, 0x6e, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x68, 0x0a, 0x11, 0x69, 1167 | 0x6e, 0x76, 0x6f, 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x72, 0x65, 0x73, 0x75, 0x6c, 0x74, 1168 | 0x18, 0x64, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x39, 0x2e, 0x69, 0x6f, 0x2e, 0x73, 0x74, 0x61, 0x74, 1169 | 0x65, 0x66, 0x75, 0x6e, 0x2e, 0x73, 0x64, 0x6b, 0x2e, 0x72, 0x65, 0x71, 0x72, 0x65, 0x70, 0x6c, 1170 | 0x79, 0x2e, 0x46, 0x72, 0x6f, 0x6d, 0x46, 0x75, 0x6e, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x2e, 0x49, 1171 | 0x6e, 0x76, 0x6f, 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 1172 | 0x65, 0x48, 0x00, 0x52, 0x10, 0x69, 0x6e, 0x76, 0x6f, 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x52, 1173 | 0x65, 0x73, 0x75, 0x6c, 0x74, 0x12, 0x88, 0x01, 0x0a, 0x1d, 0x69, 0x6e, 0x63, 0x6f, 0x6d, 0x70, 1174 | 0x6c, 0x65, 0x74, 0x65, 0x5f, 0x69, 0x6e, 0x76, 0x6f, 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 1175 | 0x63, 0x6f, 0x6e, 0x74, 0x65, 0x78, 0x74, 0x18, 0x65, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x42, 0x2e, 1176 | 0x69, 0x6f, 0x2e, 0x73, 0x74, 0x61, 0x74, 0x65, 0x66, 0x75, 0x6e, 0x2e, 0x73, 0x64, 0x6b, 0x2e, 1177 | 0x72, 0x65, 0x71, 0x72, 0x65, 0x70, 0x6c, 0x79, 0x2e, 0x46, 0x72, 0x6f, 0x6d, 0x46, 0x75, 0x6e, 1178 | 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x2e, 0x49, 0x6e, 0x63, 0x6f, 0x6d, 0x70, 0x6c, 0x65, 0x74, 0x65, 1179 | 0x49, 0x6e, 0x76, 0x6f, 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x43, 0x6f, 0x6e, 0x74, 0x65, 0x78, 1180 | 0x74, 0x48, 0x00, 0x52, 0x1b, 0x69, 0x6e, 0x63, 0x6f, 0x6d, 0x70, 0x6c, 0x65, 0x74, 0x65, 0x49, 1181 | 0x6e, 0x76, 0x6f, 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x43, 0x6f, 0x6e, 0x74, 0x65, 0x78, 0x74, 1182 | 0x1a, 0x97, 0x02, 0x0a, 0x16, 0x50, 0x65, 0x72, 0x73, 0x69, 0x73, 0x74, 0x65, 0x64, 0x56, 0x61, 1183 | 0x6c, 0x75, 0x65, 0x4d, 0x75, 0x74, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x6f, 0x0a, 0x0d, 0x6d, 1184 | 0x75, 0x74, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x74, 0x79, 0x70, 0x65, 0x18, 0x01, 0x20, 0x01, 1185 | 0x28, 0x0e, 0x32, 0x4a, 0x2e, 0x69, 0x6f, 0x2e, 0x73, 0x74, 0x61, 0x74, 0x65, 0x66, 0x75, 0x6e, 1186 | 0x2e, 0x73, 0x64, 0x6b, 0x2e, 0x72, 0x65, 0x71, 0x72, 0x65, 0x70, 0x6c, 0x79, 0x2e, 0x46, 0x72, 1187 | 0x6f, 0x6d, 0x46, 0x75, 0x6e, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x2e, 0x50, 0x65, 0x72, 0x73, 0x69, 1188 | 0x73, 0x74, 0x65, 0x64, 0x56, 0x61, 0x6c, 0x75, 0x65, 0x4d, 0x75, 0x74, 0x61, 0x74, 0x69, 0x6f, 1189 | 0x6e, 0x2e, 0x4d, 0x75, 0x74, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x54, 0x79, 0x70, 0x65, 0x52, 0x0c, 1190 | 0x6d, 0x75, 0x74, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x54, 0x79, 0x70, 0x65, 0x12, 0x1d, 0x0a, 0x0a, 1191 | 0x73, 0x74, 0x61, 0x74, 0x65, 0x5f, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 1192 | 0x52, 0x09, 0x73, 0x74, 0x61, 0x74, 0x65, 0x4e, 0x61, 0x6d, 0x65, 0x12, 0x45, 0x0a, 0x0b, 0x73, 1193 | 0x74, 0x61, 0x74, 0x65, 0x5f, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0b, 1194 | 0x32, 0x24, 0x2e, 0x69, 0x6f, 0x2e, 0x73, 0x74, 0x61, 0x74, 0x65, 0x66, 0x75, 0x6e, 0x2e, 0x73, 1195 | 0x64, 0x6b, 0x2e, 0x72, 0x65, 0x71, 0x72, 0x65, 0x70, 0x6c, 0x79, 0x2e, 0x54, 0x79, 0x70, 0x65, 1196 | 0x64, 0x56, 0x61, 0x6c, 0x75, 0x65, 0x52, 0x0a, 0x73, 0x74, 0x61, 0x74, 0x65, 0x56, 0x61, 0x6c, 1197 | 0x75, 0x65, 0x22, 0x26, 0x0a, 0x0c, 0x4d, 0x75, 0x74, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x54, 0x79, 1198 | 0x70, 0x65, 0x12, 0x0a, 0x0a, 0x06, 0x44, 0x45, 0x4c, 0x45, 0x54, 0x45, 0x10, 0x00, 0x12, 0x0a, 1199 | 0x0a, 0x06, 0x4d, 0x4f, 0x44, 0x49, 0x46, 0x59, 0x10, 0x01, 0x1a, 0x89, 0x01, 0x0a, 0x0a, 0x49, 1200 | 0x6e, 0x76, 0x6f, 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x39, 0x0a, 0x06, 0x74, 0x61, 0x72, 1201 | 0x67, 0x65, 0x74, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x21, 0x2e, 0x69, 0x6f, 0x2e, 0x73, 1202 | 0x74, 0x61, 0x74, 0x65, 0x66, 0x75, 0x6e, 0x2e, 0x73, 0x64, 0x6b, 0x2e, 0x72, 0x65, 0x71, 0x72, 1203 | 0x65, 0x70, 0x6c, 0x79, 0x2e, 0x41, 0x64, 0x64, 0x72, 0x65, 0x73, 0x73, 0x52, 0x06, 0x74, 0x61, 1204 | 0x72, 0x67, 0x65, 0x74, 0x12, 0x40, 0x0a, 0x08, 0x61, 0x72, 0x67, 0x75, 0x6d, 0x65, 0x6e, 0x74, 1205 | 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x24, 0x2e, 0x69, 0x6f, 0x2e, 0x73, 0x74, 0x61, 0x74, 1206 | 0x65, 0x66, 0x75, 0x6e, 0x2e, 0x73, 0x64, 0x6b, 0x2e, 0x72, 0x65, 0x71, 0x72, 0x65, 0x70, 0x6c, 1207 | 0x79, 0x2e, 0x54, 0x79, 0x70, 0x65, 0x64, 0x56, 0x61, 0x6c, 0x75, 0x65, 0x52, 0x08, 0x61, 0x72, 1208 | 0x67, 0x75, 0x6d, 0x65, 0x6e, 0x74, 0x1a, 0xb0, 0x01, 0x0a, 0x11, 0x44, 0x65, 0x6c, 0x61, 0x79, 1209 | 0x65, 0x64, 0x49, 0x6e, 0x76, 0x6f, 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x1e, 0x0a, 0x0b, 1210 | 0x64, 0x65, 0x6c, 0x61, 0x79, 0x5f, 0x69, 0x6e, 0x5f, 0x6d, 0x73, 0x18, 0x01, 0x20, 0x01, 0x28, 1211 | 0x03, 0x52, 0x09, 0x64, 0x65, 0x6c, 0x61, 0x79, 0x49, 0x6e, 0x4d, 0x73, 0x12, 0x39, 0x0a, 0x06, 1212 | 0x74, 0x61, 0x72, 0x67, 0x65, 0x74, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x21, 0x2e, 0x69, 1213 | 0x6f, 0x2e, 0x73, 0x74, 0x61, 0x74, 0x65, 0x66, 0x75, 0x6e, 0x2e, 0x73, 0x64, 0x6b, 0x2e, 0x72, 1214 | 0x65, 0x71, 0x72, 0x65, 0x70, 0x6c, 0x79, 0x2e, 0x41, 0x64, 0x64, 0x72, 0x65, 0x73, 0x73, 0x52, 1215 | 0x06, 0x74, 0x61, 0x72, 0x67, 0x65, 0x74, 0x12, 0x40, 0x0a, 0x08, 0x61, 0x72, 0x67, 0x75, 0x6d, 1216 | 0x65, 0x6e, 0x74, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x24, 0x2e, 0x69, 0x6f, 0x2e, 0x73, 1217 | 0x74, 0x61, 0x74, 0x65, 0x66, 0x75, 0x6e, 0x2e, 0x73, 0x64, 0x6b, 0x2e, 0x72, 0x65, 0x71, 0x72, 1218 | 0x65, 0x70, 0x6c, 0x79, 0x2e, 0x54, 0x79, 0x70, 0x65, 0x64, 0x56, 0x61, 0x6c, 0x75, 0x65, 0x52, 1219 | 0x08, 0x61, 0x72, 0x67, 0x75, 0x6d, 0x65, 0x6e, 0x74, 0x1a, 0x9d, 0x01, 0x0a, 0x0d, 0x45, 0x67, 1220 | 0x72, 0x65, 0x73, 0x73, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x12, 0x29, 0x0a, 0x10, 0x65, 1221 | 0x67, 0x72, 0x65, 0x73, 0x73, 0x5f, 0x6e, 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, 0x18, 1222 | 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0f, 0x65, 0x67, 0x72, 0x65, 0x73, 0x73, 0x4e, 0x61, 0x6d, 1223 | 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, 0x12, 0x1f, 0x0a, 0x0b, 0x65, 0x67, 0x72, 0x65, 0x73, 0x73, 1224 | 0x5f, 0x74, 0x79, 0x70, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0a, 0x65, 0x67, 0x72, 1225 | 0x65, 0x73, 0x73, 0x54, 0x79, 0x70, 0x65, 0x12, 0x40, 0x0a, 0x08, 0x61, 0x72, 0x67, 0x75, 0x6d, 1226 | 0x65, 0x6e, 0x74, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x24, 0x2e, 0x69, 0x6f, 0x2e, 0x73, 1227 | 0x74, 0x61, 0x74, 0x65, 0x66, 0x75, 0x6e, 0x2e, 0x73, 0x64, 0x6b, 0x2e, 0x72, 0x65, 0x71, 0x72, 1228 | 0x65, 0x70, 0x6c, 0x79, 0x2e, 0x54, 0x79, 0x70, 0x65, 0x64, 0x56, 0x61, 0x6c, 0x75, 0x65, 0x52, 1229 | 0x08, 0x61, 0x72, 0x67, 0x75, 0x6d, 0x65, 0x6e, 0x74, 0x1a, 0xaa, 0x03, 0x0a, 0x12, 0x49, 0x6e, 1230 | 0x76, 0x6f, 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 1231 | 0x12, 0x66, 0x0a, 0x0f, 0x73, 0x74, 0x61, 0x74, 0x65, 0x5f, 0x6d, 0x75, 0x74, 0x61, 0x74, 0x69, 1232 | 0x6f, 0x6e, 0x73, 0x18, 0x01, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x3d, 0x2e, 0x69, 0x6f, 0x2e, 0x73, 1233 | 0x74, 0x61, 0x74, 0x65, 0x66, 0x75, 0x6e, 0x2e, 0x73, 0x64, 0x6b, 0x2e, 0x72, 0x65, 0x71, 0x72, 1234 | 0x65, 0x70, 0x6c, 0x79, 0x2e, 0x46, 0x72, 0x6f, 0x6d, 0x46, 0x75, 0x6e, 0x63, 0x74, 0x69, 0x6f, 1235 | 0x6e, 0x2e, 0x50, 0x65, 0x72, 0x73, 0x69, 0x73, 0x74, 0x65, 0x64, 0x56, 0x61, 0x6c, 0x75, 0x65, 1236 | 0x4d, 0x75, 0x74, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x0e, 0x73, 0x74, 0x61, 0x74, 0x65, 0x4d, 1237 | 0x75, 0x74, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x12, 0x5e, 0x0a, 0x11, 0x6f, 0x75, 0x74, 0x67, 1238 | 0x6f, 0x69, 0x6e, 0x67, 0x5f, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x73, 0x18, 0x02, 0x20, 1239 | 0x03, 0x28, 0x0b, 0x32, 0x31, 0x2e, 0x69, 0x6f, 0x2e, 0x73, 0x74, 0x61, 0x74, 0x65, 0x66, 0x75, 1240 | 0x6e, 0x2e, 0x73, 0x64, 0x6b, 0x2e, 0x72, 0x65, 0x71, 0x72, 0x65, 0x70, 0x6c, 0x79, 0x2e, 0x46, 1241 | 0x72, 0x6f, 0x6d, 0x46, 0x75, 0x6e, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x2e, 0x49, 0x6e, 0x76, 0x6f, 1242 | 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x10, 0x6f, 0x75, 0x74, 0x67, 0x6f, 0x69, 0x6e, 0x67, 1243 | 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x73, 0x12, 0x69, 0x0a, 0x13, 0x64, 0x65, 0x6c, 0x61, 1244 | 0x79, 0x65, 0x64, 0x5f, 0x69, 0x6e, 0x76, 0x6f, 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x18, 1245 | 0x03, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x38, 0x2e, 0x69, 0x6f, 0x2e, 0x73, 0x74, 0x61, 0x74, 0x65, 1246 | 0x66, 0x75, 0x6e, 0x2e, 0x73, 0x64, 0x6b, 0x2e, 0x72, 0x65, 0x71, 0x72, 0x65, 0x70, 0x6c, 0x79, 1247 | 0x2e, 0x46, 0x72, 0x6f, 0x6d, 0x46, 0x75, 0x6e, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x2e, 0x44, 0x65, 1248 | 0x6c, 0x61, 0x79, 0x65, 0x64, 0x49, 0x6e, 0x76, 0x6f, 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x52, 1249 | 0x12, 0x64, 0x65, 0x6c, 0x61, 0x79, 0x65, 0x64, 0x49, 0x6e, 0x76, 0x6f, 0x63, 0x61, 0x74, 0x69, 1250 | 0x6f, 0x6e, 0x73, 0x12, 0x61, 0x0a, 0x11, 0x6f, 0x75, 0x74, 0x67, 0x6f, 0x69, 0x6e, 0x67, 0x5f, 1251 | 0x65, 0x67, 0x72, 0x65, 0x73, 0x73, 0x65, 0x73, 0x18, 0x04, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x34, 1252 | 0x2e, 0x69, 0x6f, 0x2e, 0x73, 0x74, 0x61, 0x74, 0x65, 0x66, 0x75, 0x6e, 0x2e, 0x73, 0x64, 0x6b, 1253 | 0x2e, 0x72, 0x65, 0x71, 0x72, 0x65, 0x70, 0x6c, 0x79, 0x2e, 0x46, 0x72, 0x6f, 0x6d, 0x46, 0x75, 1254 | 0x6e, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x2e, 0x45, 0x67, 0x72, 0x65, 0x73, 0x73, 0x4d, 0x65, 0x73, 1255 | 0x73, 0x61, 0x67, 0x65, 0x52, 0x10, 0x6f, 0x75, 0x74, 0x67, 0x6f, 0x69, 0x6e, 0x67, 0x45, 0x67, 1256 | 0x72, 0x65, 0x73, 0x73, 0x65, 0x73, 0x1a, 0xd1, 0x01, 0x0a, 0x0e, 0x45, 0x78, 0x70, 0x69, 0x72, 1257 | 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x53, 0x70, 0x65, 0x63, 0x12, 0x54, 0x0a, 0x04, 0x6d, 0x6f, 0x64, 1258 | 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0e, 0x32, 0x40, 0x2e, 0x69, 0x6f, 0x2e, 0x73, 0x74, 0x61, 1259 | 0x74, 0x65, 0x66, 0x75, 0x6e, 0x2e, 0x73, 0x64, 0x6b, 0x2e, 0x72, 0x65, 0x71, 0x72, 0x65, 0x70, 1260 | 0x6c, 0x79, 0x2e, 0x46, 0x72, 0x6f, 0x6d, 0x46, 0x75, 0x6e, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x2e, 1261 | 0x45, 0x78, 0x70, 0x69, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x53, 0x70, 0x65, 0x63, 0x2e, 0x45, 1262 | 0x78, 0x70, 0x69, 0x72, 0x65, 0x4d, 0x6f, 0x64, 0x65, 0x52, 0x04, 0x6d, 0x6f, 0x64, 0x65, 0x12, 1263 | 0x2e, 0x0a, 0x13, 0x65, 0x78, 0x70, 0x69, 0x72, 0x65, 0x5f, 0x61, 0x66, 0x74, 0x65, 0x72, 0x5f, 1264 | 0x6d, 0x69, 0x6c, 0x6c, 0x69, 0x73, 0x18, 0x02, 0x20, 0x01, 0x28, 0x03, 0x52, 0x11, 0x65, 0x78, 1265 | 0x70, 0x69, 0x72, 0x65, 0x41, 0x66, 0x74, 0x65, 0x72, 0x4d, 0x69, 0x6c, 0x6c, 0x69, 0x73, 0x22, 1266 | 0x39, 0x0a, 0x0a, 0x45, 0x78, 0x70, 0x69, 0x72, 0x65, 0x4d, 0x6f, 0x64, 0x65, 0x12, 0x08, 0x0a, 1267 | 0x04, 0x4e, 0x4f, 0x4e, 0x45, 0x10, 0x00, 0x12, 0x0f, 0x0a, 0x0b, 0x41, 0x46, 0x54, 0x45, 0x52, 1268 | 0x5f, 0x57, 0x52, 0x49, 0x54, 0x45, 0x10, 0x01, 0x12, 0x10, 0x0a, 0x0c, 0x41, 0x46, 0x54, 0x45, 1269 | 0x52, 0x5f, 0x49, 0x4e, 0x56, 0x4f, 0x4b, 0x45, 0x10, 0x02, 0x1a, 0xb8, 0x01, 0x0a, 0x12, 0x50, 1270 | 0x65, 0x72, 0x73, 0x69, 0x73, 0x74, 0x65, 0x64, 0x56, 0x61, 0x6c, 0x75, 0x65, 0x53, 0x70, 0x65, 1271 | 0x63, 0x12, 0x1d, 0x0a, 0x0a, 0x73, 0x74, 0x61, 0x74, 0x65, 0x5f, 0x6e, 0x61, 0x6d, 0x65, 0x18, 1272 | 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x09, 0x73, 0x74, 0x61, 0x74, 0x65, 0x4e, 0x61, 0x6d, 0x65, 1273 | 0x12, 0x5e, 0x0a, 0x0f, 0x65, 0x78, 0x70, 0x69, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x73, 1274 | 0x70, 0x65, 0x63, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x35, 0x2e, 0x69, 0x6f, 0x2e, 0x73, 1275 | 0x74, 0x61, 0x74, 0x65, 0x66, 0x75, 0x6e, 0x2e, 0x73, 0x64, 0x6b, 0x2e, 0x72, 0x65, 0x71, 0x72, 1276 | 0x65, 0x70, 0x6c, 0x79, 0x2e, 0x46, 0x72, 0x6f, 0x6d, 0x46, 0x75, 0x6e, 0x63, 0x74, 0x69, 0x6f, 1277 | 0x6e, 0x2e, 0x45, 0x78, 0x70, 0x69, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x53, 0x70, 0x65, 0x63, 1278 | 0x52, 0x0e, 0x65, 0x78, 0x70, 0x69, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x53, 0x70, 0x65, 0x63, 1279 | 0x12, 0x23, 0x0a, 0x0d, 0x74, 0x79, 0x70, 0x65, 0x5f, 0x74, 0x79, 0x70, 0x65, 0x6e, 0x61, 0x6d, 1280 | 0x65, 0x18, 0x03, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0c, 0x74, 0x79, 0x70, 0x65, 0x54, 0x79, 0x70, 1281 | 0x65, 0x6e, 0x61, 0x6d, 0x65, 0x1a, 0x7f, 0x0a, 0x1b, 0x49, 0x6e, 0x63, 0x6f, 0x6d, 0x70, 0x6c, 1282 | 0x65, 0x74, 0x65, 0x49, 0x6e, 0x76, 0x6f, 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x43, 0x6f, 0x6e, 1283 | 0x74, 0x65, 0x78, 0x74, 0x12, 0x60, 0x0a, 0x0e, 0x6d, 0x69, 0x73, 0x73, 0x69, 0x6e, 0x67, 0x5f, 1284 | 0x76, 0x61, 0x6c, 0x75, 0x65, 0x73, 0x18, 0x01, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x39, 0x2e, 0x69, 1285 | 0x6f, 0x2e, 0x73, 0x74, 0x61, 0x74, 0x65, 0x66, 0x75, 0x6e, 0x2e, 0x73, 0x64, 0x6b, 0x2e, 0x72, 1286 | 0x65, 0x71, 0x72, 0x65, 0x70, 0x6c, 0x79, 0x2e, 0x46, 0x72, 0x6f, 0x6d, 0x46, 0x75, 0x6e, 0x63, 1287 | 0x74, 0x69, 0x6f, 0x6e, 0x2e, 0x50, 0x65, 0x72, 0x73, 0x69, 0x73, 0x74, 0x65, 0x64, 0x56, 0x61, 1288 | 0x6c, 0x75, 0x65, 0x53, 0x70, 0x65, 0x63, 0x52, 0x0d, 0x6d, 0x69, 0x73, 0x73, 0x69, 0x6e, 0x67, 1289 | 0x56, 0x61, 0x6c, 0x75, 0x65, 0x73, 0x42, 0x0a, 0x0a, 0x08, 0x72, 0x65, 0x73, 0x70, 0x6f, 0x6e, 1290 | 0x73, 0x65, 0x42, 0x40, 0x0a, 0x30, 0x6f, 0x72, 0x67, 0x2e, 0x61, 0x70, 0x61, 0x63, 0x68, 0x65, 1291 | 0x2e, 0x66, 0x6c, 0x69, 0x6e, 0x6b, 0x2e, 0x73, 0x74, 0x61, 0x74, 0x65, 0x66, 0x75, 0x6e, 0x2e, 1292 | 0x73, 0x64, 0x6b, 0x2e, 0x72, 0x65, 0x71, 0x72, 0x65, 0x70, 0x6c, 0x79, 0x2e, 0x67, 0x65, 0x6e, 1293 | 0x65, 0x72, 0x61, 0x74, 0x65, 0x64, 0x50, 0x01, 0x5a, 0x0a, 0x2e, 0x2f, 0x70, 0x72, 0x6f, 0x74, 1294 | 0x6f, 0x63, 0x6f, 0x6c, 0x62, 0x06, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x33, 1295 | } 1296 | 1297 | var ( 1298 | file_request_reply_proto_rawDescOnce sync.Once 1299 | file_request_reply_proto_rawDescData = file_request_reply_proto_rawDesc 1300 | ) 1301 | 1302 | func file_request_reply_proto_rawDescGZIP() []byte { 1303 | file_request_reply_proto_rawDescOnce.Do(func() { 1304 | file_request_reply_proto_rawDescData = protoimpl.X.CompressGZIP(file_request_reply_proto_rawDescData) 1305 | }) 1306 | return file_request_reply_proto_rawDescData 1307 | } 1308 | 1309 | var file_request_reply_proto_enumTypes = make([]protoimpl.EnumInfo, 2) 1310 | var file_request_reply_proto_msgTypes = make([]protoimpl.MessageInfo, 15) 1311 | var file_request_reply_proto_goTypes = []interface{}{ 1312 | (FromFunction_PersistedValueMutation_MutationType)(0), // 0: io.statefun.sdk.reqreply.FromFunction.PersistedValueMutation.MutationType 1313 | (FromFunction_ExpirationSpec_ExpireMode)(0), // 1: io.statefun.sdk.reqreply.FromFunction.ExpirationSpec.ExpireMode 1314 | (*Address)(nil), // 2: io.statefun.sdk.reqreply.Address 1315 | (*TypedValue)(nil), // 3: io.statefun.sdk.reqreply.TypedValue 1316 | (*ToFunction)(nil), // 4: io.statefun.sdk.reqreply.ToFunction 1317 | (*FromFunction)(nil), // 5: io.statefun.sdk.reqreply.FromFunction 1318 | (*ToFunction_PersistedValue)(nil), // 6: io.statefun.sdk.reqreply.ToFunction.PersistedValue 1319 | (*ToFunction_Invocation)(nil), // 7: io.statefun.sdk.reqreply.ToFunction.Invocation 1320 | (*ToFunction_InvocationBatchRequest)(nil), // 8: io.statefun.sdk.reqreply.ToFunction.InvocationBatchRequest 1321 | (*FromFunction_PersistedValueMutation)(nil), // 9: io.statefun.sdk.reqreply.FromFunction.PersistedValueMutation 1322 | (*FromFunction_Invocation)(nil), // 10: io.statefun.sdk.reqreply.FromFunction.Invocation 1323 | (*FromFunction_DelayedInvocation)(nil), // 11: io.statefun.sdk.reqreply.FromFunction.DelayedInvocation 1324 | (*FromFunction_EgressMessage)(nil), // 12: io.statefun.sdk.reqreply.FromFunction.EgressMessage 1325 | (*FromFunction_InvocationResponse)(nil), // 13: io.statefun.sdk.reqreply.FromFunction.InvocationResponse 1326 | (*FromFunction_ExpirationSpec)(nil), // 14: io.statefun.sdk.reqreply.FromFunction.ExpirationSpec 1327 | (*FromFunction_PersistedValueSpec)(nil), // 15: io.statefun.sdk.reqreply.FromFunction.PersistedValueSpec 1328 | (*FromFunction_IncompleteInvocationContext)(nil), // 16: io.statefun.sdk.reqreply.FromFunction.IncompleteInvocationContext 1329 | } 1330 | var file_request_reply_proto_depIdxs = []int32{ 1331 | 8, // 0: io.statefun.sdk.reqreply.ToFunction.invocation:type_name -> io.statefun.sdk.reqreply.ToFunction.InvocationBatchRequest 1332 | 13, // 1: io.statefun.sdk.reqreply.FromFunction.invocation_result:type_name -> io.statefun.sdk.reqreply.FromFunction.InvocationResponse 1333 | 16, // 2: io.statefun.sdk.reqreply.FromFunction.incomplete_invocation_context:type_name -> io.statefun.sdk.reqreply.FromFunction.IncompleteInvocationContext 1334 | 3, // 3: io.statefun.sdk.reqreply.ToFunction.PersistedValue.state_value:type_name -> io.statefun.sdk.reqreply.TypedValue 1335 | 2, // 4: io.statefun.sdk.reqreply.ToFunction.Invocation.caller:type_name -> io.statefun.sdk.reqreply.Address 1336 | 3, // 5: io.statefun.sdk.reqreply.ToFunction.Invocation.argument:type_name -> io.statefun.sdk.reqreply.TypedValue 1337 | 2, // 6: io.statefun.sdk.reqreply.ToFunction.InvocationBatchRequest.target:type_name -> io.statefun.sdk.reqreply.Address 1338 | 6, // 7: io.statefun.sdk.reqreply.ToFunction.InvocationBatchRequest.state:type_name -> io.statefun.sdk.reqreply.ToFunction.PersistedValue 1339 | 7, // 8: io.statefun.sdk.reqreply.ToFunction.InvocationBatchRequest.invocations:type_name -> io.statefun.sdk.reqreply.ToFunction.Invocation 1340 | 0, // 9: io.statefun.sdk.reqreply.FromFunction.PersistedValueMutation.mutation_type:type_name -> io.statefun.sdk.reqreply.FromFunction.PersistedValueMutation.MutationType 1341 | 3, // 10: io.statefun.sdk.reqreply.FromFunction.PersistedValueMutation.state_value:type_name -> io.statefun.sdk.reqreply.TypedValue 1342 | 2, // 11: io.statefun.sdk.reqreply.FromFunction.Invocation.target:type_name -> io.statefun.sdk.reqreply.Address 1343 | 3, // 12: io.statefun.sdk.reqreply.FromFunction.Invocation.argument:type_name -> io.statefun.sdk.reqreply.TypedValue 1344 | 2, // 13: io.statefun.sdk.reqreply.FromFunction.DelayedInvocation.target:type_name -> io.statefun.sdk.reqreply.Address 1345 | 3, // 14: io.statefun.sdk.reqreply.FromFunction.DelayedInvocation.argument:type_name -> io.statefun.sdk.reqreply.TypedValue 1346 | 3, // 15: io.statefun.sdk.reqreply.FromFunction.EgressMessage.argument:type_name -> io.statefun.sdk.reqreply.TypedValue 1347 | 9, // 16: io.statefun.sdk.reqreply.FromFunction.InvocationResponse.state_mutations:type_name -> io.statefun.sdk.reqreply.FromFunction.PersistedValueMutation 1348 | 10, // 17: io.statefun.sdk.reqreply.FromFunction.InvocationResponse.outgoing_messages:type_name -> io.statefun.sdk.reqreply.FromFunction.Invocation 1349 | 11, // 18: io.statefun.sdk.reqreply.FromFunction.InvocationResponse.delayed_invocations:type_name -> io.statefun.sdk.reqreply.FromFunction.DelayedInvocation 1350 | 12, // 19: io.statefun.sdk.reqreply.FromFunction.InvocationResponse.outgoing_egresses:type_name -> io.statefun.sdk.reqreply.FromFunction.EgressMessage 1351 | 1, // 20: io.statefun.sdk.reqreply.FromFunction.ExpirationSpec.mode:type_name -> io.statefun.sdk.reqreply.FromFunction.ExpirationSpec.ExpireMode 1352 | 14, // 21: io.statefun.sdk.reqreply.FromFunction.PersistedValueSpec.expiration_spec:type_name -> io.statefun.sdk.reqreply.FromFunction.ExpirationSpec 1353 | 15, // 22: io.statefun.sdk.reqreply.FromFunction.IncompleteInvocationContext.missing_values:type_name -> io.statefun.sdk.reqreply.FromFunction.PersistedValueSpec 1354 | 23, // [23:23] is the sub-list for method output_type 1355 | 23, // [23:23] is the sub-list for method input_type 1356 | 23, // [23:23] is the sub-list for extension type_name 1357 | 23, // [23:23] is the sub-list for extension extendee 1358 | 0, // [0:23] is the sub-list for field type_name 1359 | } 1360 | 1361 | func init() { file_request_reply_proto_init() } 1362 | func file_request_reply_proto_init() { 1363 | if File_request_reply_proto != nil { 1364 | return 1365 | } 1366 | if !protoimpl.UnsafeEnabled { 1367 | file_request_reply_proto_msgTypes[0].Exporter = func(v interface{}, i int) interface{} { 1368 | switch v := v.(*Address); i { 1369 | case 0: 1370 | return &v.state 1371 | case 1: 1372 | return &v.sizeCache 1373 | case 2: 1374 | return &v.unknownFields 1375 | default: 1376 | return nil 1377 | } 1378 | } 1379 | file_request_reply_proto_msgTypes[1].Exporter = func(v interface{}, i int) interface{} { 1380 | switch v := v.(*TypedValue); i { 1381 | case 0: 1382 | return &v.state 1383 | case 1: 1384 | return &v.sizeCache 1385 | case 2: 1386 | return &v.unknownFields 1387 | default: 1388 | return nil 1389 | } 1390 | } 1391 | file_request_reply_proto_msgTypes[2].Exporter = func(v interface{}, i int) interface{} { 1392 | switch v := v.(*ToFunction); i { 1393 | case 0: 1394 | return &v.state 1395 | case 1: 1396 | return &v.sizeCache 1397 | case 2: 1398 | return &v.unknownFields 1399 | default: 1400 | return nil 1401 | } 1402 | } 1403 | file_request_reply_proto_msgTypes[3].Exporter = func(v interface{}, i int) interface{} { 1404 | switch v := v.(*FromFunction); i { 1405 | case 0: 1406 | return &v.state 1407 | case 1: 1408 | return &v.sizeCache 1409 | case 2: 1410 | return &v.unknownFields 1411 | default: 1412 | return nil 1413 | } 1414 | } 1415 | file_request_reply_proto_msgTypes[4].Exporter = func(v interface{}, i int) interface{} { 1416 | switch v := v.(*ToFunction_PersistedValue); i { 1417 | case 0: 1418 | return &v.state 1419 | case 1: 1420 | return &v.sizeCache 1421 | case 2: 1422 | return &v.unknownFields 1423 | default: 1424 | return nil 1425 | } 1426 | } 1427 | file_request_reply_proto_msgTypes[5].Exporter = func(v interface{}, i int) interface{} { 1428 | switch v := v.(*ToFunction_Invocation); i { 1429 | case 0: 1430 | return &v.state 1431 | case 1: 1432 | return &v.sizeCache 1433 | case 2: 1434 | return &v.unknownFields 1435 | default: 1436 | return nil 1437 | } 1438 | } 1439 | file_request_reply_proto_msgTypes[6].Exporter = func(v interface{}, i int) interface{} { 1440 | switch v := v.(*ToFunction_InvocationBatchRequest); i { 1441 | case 0: 1442 | return &v.state 1443 | case 1: 1444 | return &v.sizeCache 1445 | case 2: 1446 | return &v.unknownFields 1447 | default: 1448 | return nil 1449 | } 1450 | } 1451 | file_request_reply_proto_msgTypes[7].Exporter = func(v interface{}, i int) interface{} { 1452 | switch v := v.(*FromFunction_PersistedValueMutation); i { 1453 | case 0: 1454 | return &v.state 1455 | case 1: 1456 | return &v.sizeCache 1457 | case 2: 1458 | return &v.unknownFields 1459 | default: 1460 | return nil 1461 | } 1462 | } 1463 | file_request_reply_proto_msgTypes[8].Exporter = func(v interface{}, i int) interface{} { 1464 | switch v := v.(*FromFunction_Invocation); i { 1465 | case 0: 1466 | return &v.state 1467 | case 1: 1468 | return &v.sizeCache 1469 | case 2: 1470 | return &v.unknownFields 1471 | default: 1472 | return nil 1473 | } 1474 | } 1475 | file_request_reply_proto_msgTypes[9].Exporter = func(v interface{}, i int) interface{} { 1476 | switch v := v.(*FromFunction_DelayedInvocation); i { 1477 | case 0: 1478 | return &v.state 1479 | case 1: 1480 | return &v.sizeCache 1481 | case 2: 1482 | return &v.unknownFields 1483 | default: 1484 | return nil 1485 | } 1486 | } 1487 | file_request_reply_proto_msgTypes[10].Exporter = func(v interface{}, i int) interface{} { 1488 | switch v := v.(*FromFunction_EgressMessage); i { 1489 | case 0: 1490 | return &v.state 1491 | case 1: 1492 | return &v.sizeCache 1493 | case 2: 1494 | return &v.unknownFields 1495 | default: 1496 | return nil 1497 | } 1498 | } 1499 | file_request_reply_proto_msgTypes[11].Exporter = func(v interface{}, i int) interface{} { 1500 | switch v := v.(*FromFunction_InvocationResponse); i { 1501 | case 0: 1502 | return &v.state 1503 | case 1: 1504 | return &v.sizeCache 1505 | case 2: 1506 | return &v.unknownFields 1507 | default: 1508 | return nil 1509 | } 1510 | } 1511 | file_request_reply_proto_msgTypes[12].Exporter = func(v interface{}, i int) interface{} { 1512 | switch v := v.(*FromFunction_ExpirationSpec); i { 1513 | case 0: 1514 | return &v.state 1515 | case 1: 1516 | return &v.sizeCache 1517 | case 2: 1518 | return &v.unknownFields 1519 | default: 1520 | return nil 1521 | } 1522 | } 1523 | file_request_reply_proto_msgTypes[13].Exporter = func(v interface{}, i int) interface{} { 1524 | switch v := v.(*FromFunction_PersistedValueSpec); i { 1525 | case 0: 1526 | return &v.state 1527 | case 1: 1528 | return &v.sizeCache 1529 | case 2: 1530 | return &v.unknownFields 1531 | default: 1532 | return nil 1533 | } 1534 | } 1535 | file_request_reply_proto_msgTypes[14].Exporter = func(v interface{}, i int) interface{} { 1536 | switch v := v.(*FromFunction_IncompleteInvocationContext); i { 1537 | case 0: 1538 | return &v.state 1539 | case 1: 1540 | return &v.sizeCache 1541 | case 2: 1542 | return &v.unknownFields 1543 | default: 1544 | return nil 1545 | } 1546 | } 1547 | } 1548 | file_request_reply_proto_msgTypes[2].OneofWrappers = []interface{}{ 1549 | (*ToFunction_Invocation_)(nil), 1550 | } 1551 | file_request_reply_proto_msgTypes[3].OneofWrappers = []interface{}{ 1552 | (*FromFunction_InvocationResult)(nil), 1553 | (*FromFunction_IncompleteInvocationContext_)(nil), 1554 | } 1555 | type x struct{} 1556 | out := protoimpl.TypeBuilder{ 1557 | File: protoimpl.DescBuilder{ 1558 | GoPackagePath: reflect.TypeOf(x{}).PkgPath(), 1559 | RawDescriptor: file_request_reply_proto_rawDesc, 1560 | NumEnums: 2, 1561 | NumMessages: 15, 1562 | NumExtensions: 0, 1563 | NumServices: 0, 1564 | }, 1565 | GoTypes: file_request_reply_proto_goTypes, 1566 | DependencyIndexes: file_request_reply_proto_depIdxs, 1567 | EnumInfos: file_request_reply_proto_enumTypes, 1568 | MessageInfos: file_request_reply_proto_msgTypes, 1569 | }.Build() 1570 | File_request_reply_proto = out.File 1571 | file_request_reply_proto_rawDesc = nil 1572 | file_request_reply_proto_goTypes = nil 1573 | file_request_reply_proto_depIdxs = nil 1574 | } 1575 | -------------------------------------------------------------------------------- /pkg/statefun/message.go: -------------------------------------------------------------------------------- 1 | package statefun 2 | 3 | import ( 4 | "bytes" 5 | "errors" 6 | "fmt" 7 | "statefun-sdk-go/pkg/statefun/internal/protocol" 8 | ) 9 | 10 | type MessageBuilder struct { 11 | Target Address 12 | Value interface{} 13 | ValueType SimpleType 14 | } 15 | 16 | func (m MessageBuilder) ToMessage() (Message, error) { 17 | if m.Target == (Address{}) { 18 | return Message{}, errors.New("a message must have a non-empty target") 19 | } 20 | 21 | if m.Value == nil { 22 | return Message{}, errors.New("a message cannot have a nil value") 23 | } 24 | 25 | if m.ValueType == nil { 26 | switch m.Value.(type) { 27 | case int: 28 | return Message{}, errors.New("ambiguous integer type; please specify int32 or int64") 29 | case bool, *bool: 30 | m.ValueType = BoolType 31 | case int32, *int32: 32 | m.ValueType = Int32Type 33 | case int64, *int64: 34 | m.ValueType = Int64Type 35 | case float32, *float32: 36 | m.ValueType = Float32Type 37 | case float64, *float64: 38 | m.ValueType = Float64Type 39 | case string, *string: 40 | m.ValueType = StringType 41 | default: 42 | return Message{}, errors.New("message contains non-primitive type, please supply a non-nil SimpleType") 43 | } 44 | } 45 | 46 | buffer := bytes.Buffer{} 47 | err := m.ValueType.Serialize(&buffer, m.Value) 48 | if err != nil { 49 | return Message{}, err 50 | } 51 | 52 | return Message{ 53 | target: &protocol.Address{ 54 | Namespace: m.Target.FunctionType.GetNamespace(), 55 | Type: m.Target.FunctionType.GetType(), 56 | Id: m.Target.Id, 57 | }, 58 | typedValue: &protocol.TypedValue{ 59 | Typename: m.ValueType.GetTypeName().String(), 60 | HasValue: true, 61 | Value: buffer.Bytes(), 62 | }, 63 | }, nil 64 | } 65 | 66 | type Message struct { 67 | target *protocol.Address 68 | typedValue *protocol.TypedValue 69 | } 70 | 71 | func (m *Message) IsBool() bool { 72 | return m.Is(BoolType) 73 | } 74 | 75 | func (m *Message) AsBool() bool { 76 | var receiver bool 77 | if err := BoolType.Deserialize(bytes.NewReader(m.typedValue.Value), &receiver); err != nil { 78 | panic(fmt.Errorf("failed to deserialize message: %w", err)) 79 | } 80 | return receiver 81 | } 82 | 83 | func (m *Message) IsInt32() bool { 84 | return m.Is(Int32Type) 85 | } 86 | 87 | func (m *Message) AsInt32() int32 { 88 | var receiver int32 89 | if err := Int32Type.Deserialize(bytes.NewReader(m.typedValue.Value), &receiver); err != nil { 90 | panic(fmt.Errorf("failed to deserialize message: %w", err)) 91 | } 92 | return receiver 93 | } 94 | 95 | func (m *Message) IsInt64() bool { 96 | return m.Is(Int64Type) 97 | } 98 | 99 | func (m *Message) AsInt64() int64 { 100 | var receiver int64 101 | if err := Int64Type.Deserialize(bytes.NewReader(m.typedValue.Value), &receiver); err != nil { 102 | panic(fmt.Errorf("failed to deserialize message: %w", err)) 103 | } 104 | return receiver 105 | } 106 | 107 | func (m *Message) IsFloat32() bool { 108 | return m.Is(Float32Type) 109 | } 110 | 111 | func (m *Message) AsFloat32() float32 { 112 | var receiver float32 113 | if err := Float32Type.Deserialize(bytes.NewReader(m.typedValue.Value), &receiver); err != nil { 114 | panic(fmt.Errorf("failed to deserialize message: %w", err)) 115 | } 116 | return receiver 117 | } 118 | 119 | func (m *Message) IsFloat64() bool { 120 | return m.Is(Float64Type) 121 | } 122 | 123 | func (m *Message) AsFloat64() float64 { 124 | var receiver float64 125 | if err := Float64Type.Deserialize(bytes.NewReader(m.typedValue.Value), &receiver); err != nil { 126 | panic(fmt.Errorf("failed to deserialize message: %w", err)) 127 | } 128 | return receiver 129 | } 130 | 131 | func (m *Message) IsString() bool { 132 | return m.Is(StringType) 133 | } 134 | 135 | func (m *Message) AsString() string { 136 | var receiver string 137 | if err := StringType.Deserialize(bytes.NewReader(m.typedValue.Value), &receiver); err != nil { 138 | panic(fmt.Errorf("failed to deserialize message: %w", err)) 139 | } 140 | 141 | return receiver 142 | } 143 | 144 | func (m *Message) Is(t SimpleType) bool { 145 | return t.GetTypeName().String() == m.typedValue.Typename 146 | } 147 | 148 | func (m *Message) As(t SimpleType, receiver interface{}) error { 149 | return t.Deserialize(bytes.NewReader(m.typedValue.Value), receiver) 150 | } 151 | 152 | func (m *Message) ValueTypeName() TypeName { 153 | return TypeNameFrom(m.typedValue.Typename) 154 | } 155 | 156 | func (m *Message) RawValue() []byte { 157 | return m.typedValue.Value 158 | } 159 | -------------------------------------------------------------------------------- /pkg/statefun/message_test.go: -------------------------------------------------------------------------------- 1 | package statefun 2 | 3 | import ( 4 | "github.com/stretchr/testify/assert" 5 | "testing" 6 | ) 7 | 8 | func TestBasicIntMessage(t *testing.T) { 9 | typename, err := ParseTypeName("foo/bar") 10 | assert.NoError(t, err) 11 | 12 | message, err := MessageBuilder{ 13 | Target: Address{ 14 | FunctionType: typename, 15 | Id: "a", 16 | }, 17 | Value: int32(1), 18 | }.ToMessage() 19 | 20 | assert.NoError(t, err) 21 | assert.True(t, message.IsInt32()) 22 | 23 | value := message.AsInt32() 24 | assert.Equal(t, value, int32(1)) 25 | } 26 | 27 | func TestMessageWithType(t *testing.T) { 28 | typename, err := ParseTypeName("foo/bar") 29 | assert.NoError(t, err) 30 | 31 | message, err := MessageBuilder{ 32 | Target: Address{ 33 | FunctionType: typename, 34 | Id: "a", 35 | }, 36 | Value: float32(5.0), 37 | ValueType: Float32Type, 38 | }.ToMessage() 39 | 40 | assert.NoError(t, err) 41 | assert.True(t, message.IsFloat32()) 42 | 43 | value := message.AsFloat32() 44 | assert.Equal(t, value, float32(5.0)) 45 | } 46 | -------------------------------------------------------------------------------- /pkg/statefun/stateful_function.go: -------------------------------------------------------------------------------- 1 | package statefun 2 | 3 | // A StatefulFunction is a user-defined function that can be invoked with a given input. 4 | // This is the primitive building block for a Stateful Functions application. 5 | // 6 | // Concept 7 | // 8 | // Each individual StatefulFunction is an uniquely invokable "instance" of a registered 9 | // StatefulFunctionSpec. Each instance is identified by an Address, representing the 10 | // function's unique id (a string) within its type. From a user's perspective, it would seem as if 11 | // for each unique function id, there exists a stateful instance of the function that is always 12 | // available to be invoked within a Stateful Functions application. 13 | // 14 | // Invoking a StatefulFunction 15 | // 16 | // An individual StatefulFunction can be invoked with arbitrary input from any another 17 | // StatefulFunction (including itself), or routed from ingresses. To invoke a 18 | // StatefulFunction, the caller simply needs to know the Address of the target function. 19 | // 20 | // As a result of invoking a StatefulFunction, the function may continue to invoke other 21 | // functions, access persisted values, or send messages to egresses. 22 | // 23 | // Persistent State 24 | // 25 | // Each individual StatefulFunction may have persistent values written to storage that is 26 | // maintained by the system, providing consistent exactly-once and fault-tolerant guarantees. Please 27 | // see docs in ValueSpec and AddressScopedStorage for an overview of how to 28 | // register persistent values and access the storage. 29 | type StatefulFunction interface { 30 | 31 | // The method called for each message. The passed Context 32 | // is canceled as soon as Invoke returns as a signal to 33 | // any spawned go routines. The method may return 34 | // an Error to signal the invocation failed and should 35 | // be reattempted. 36 | Invoke(ctx Context, message Message) error 37 | } 38 | 39 | // Specification for a Stateful Function, identifiable 40 | // by a unique TypeName. 41 | type StatefulFunctionSpec struct { 42 | // The unique TypeName associated 43 | // the the StatefulFunction being defined. 44 | FunctionType TypeName 45 | 46 | // A slice of registered ValueSpec's that will be used 47 | // by this function. A function may only access values 48 | // that have been eagerly registered as part of its spec. 49 | States []ValueSpec 50 | 51 | // The physical StatefulFunction instance. 52 | Function StatefulFunction 53 | } 54 | 55 | // The StatefulFunctionPointer type is an adapter to allow the use of 56 | // ordinary functions as StatefulFunction's. If f is a function 57 | // with the appropriate signature, StatefulFunctionPointer(f) is a 58 | // StatefulFunction that calls f. 59 | type StatefulFunctionPointer func(Context, Message) error 60 | 61 | func (s StatefulFunctionPointer) Invoke(ctx Context, message Message) error { 62 | return s(ctx, message) 63 | } 64 | -------------------------------------------------------------------------------- /pkg/statefun/storage.go: -------------------------------------------------------------------------------- 1 | package statefun 2 | 3 | import ( 4 | "fmt" 5 | "statefun-sdk-go/pkg/statefun/internal" 6 | "statefun-sdk-go/pkg/statefun/internal/protocol" 7 | "sync" 8 | ) 9 | 10 | // An AddressScopedStorage is used for reading and writing persistent 11 | // values that are managed by the Stateful Functions runtime for 12 | // fault-tolerance and consistency. 13 | // 14 | // All access to the storage is scoped to the current function instance, 15 | // identified by the instance's Address. This means that within an 16 | // invocation, function instances may only access its own persisted 17 | // values through this storage. 18 | type AddressScopedStorage interface { 19 | 20 | // Gets the values of the provided ValueSpec, scoped to the 21 | // current invoked Address and stores the result in the value 22 | // pointed to by receiver. The method will return false 23 | // if there is no value for the spec in storage 24 | // so callers can differentiate between missing and 25 | // the types zero value. 26 | Get(spec ValueSpec, receiver interface{}) (exists bool) 27 | 28 | // Sets the value for the provided ValueSpec, scoped 29 | // to the current invoked Address. 30 | Set(spec ValueSpec, value interface{}) 31 | 32 | // Removes the prior value set for the the provided 33 | // ValueSpec, scoped to the current invoked Address. 34 | // 35 | // After removing the value, calling Get for the same 36 | // spec under the same Address will return false. 37 | Remove(spec ValueSpec) 38 | } 39 | 40 | type storage struct { 41 | mutex sync.RWMutex 42 | cells map[string]*internal.Cell 43 | } 44 | 45 | type storageFactory interface { 46 | getStorage() *storage 47 | 48 | getMissingSpecs() []*protocol.FromFunction_PersistedValueSpec 49 | } 50 | 51 | func newStorageFactory( 52 | batch *protocol.ToFunction_InvocationBatchRequest, 53 | specs map[string]*protocol.FromFunction_PersistedValueSpec, 54 | ) storageFactory { 55 | storage := &storage{ 56 | cells: make(map[string]*internal.Cell, len(specs)), 57 | } 58 | 59 | states := make(map[string]*protocol.FromFunction_PersistedValueSpec, len(specs)) 60 | for k, v := range specs { 61 | states[k] = v 62 | } 63 | 64 | for _, state := range batch.State { 65 | if _, exists := states[state.StateName]; !exists { 66 | continue 67 | } 68 | 69 | delete(states, state.StateName) 70 | 71 | storage.cells[state.StateName] = internal.NewCell(state) 72 | } 73 | 74 | if len(states) > 0 { 75 | var missing = make([]*protocol.FromFunction_PersistedValueSpec, 0, len(states)) 76 | for _, spec := range states { 77 | missing = append(missing, spec) 78 | } 79 | 80 | return MissingSpecs(missing) 81 | } else { 82 | return storage 83 | } 84 | } 85 | 86 | func (s *storage) getStorage() *storage { 87 | return s 88 | } 89 | 90 | func (s *storage) getMissingSpecs() []*protocol.FromFunction_PersistedValueSpec { 91 | return nil 92 | } 93 | 94 | func (s *storage) Get(spec ValueSpec, receiver interface{}) bool { 95 | s.mutex.RLock() 96 | defer s.mutex.RUnlock() 97 | 98 | cell, ok := s.cells[spec.Name] 99 | if !ok { 100 | panic(fmt.Errorf("unregistered ValueSpec %s", spec.Name)) 101 | } 102 | 103 | if !cell.HasValue() { 104 | return false 105 | } 106 | 107 | if err := spec.ValueType.Deserialize(cell, receiver); err != nil { 108 | panic(fmt.Errorf("failed to deserialize %s: %w", spec.Name, err)) 109 | } 110 | 111 | return true 112 | } 113 | 114 | func (s *storage) Set(spec ValueSpec, value interface{}) { 115 | s.mutex.Lock() 116 | defer s.mutex.Unlock() 117 | 118 | cell, ok := s.cells[spec.Name] 119 | if !ok { 120 | panic(fmt.Errorf("unregistered ValueSpec %s", spec.Name)) 121 | } 122 | 123 | err := spec.ValueType.Serialize(cell, value) 124 | if err != nil { 125 | panic(fmt.Errorf("failed to serialize %s: %w", spec.Name, err)) 126 | } 127 | } 128 | 129 | func (s *storage) Remove(spec ValueSpec) { 130 | s.mutex.Lock() 131 | defer s.mutex.Unlock() 132 | 133 | cell, ok := s.cells[spec.Name] 134 | if !ok { 135 | panic(fmt.Errorf("unregistered ValueSpec %s", spec.Name)) 136 | } 137 | 138 | cell.Reset() 139 | } 140 | 141 | func (s *storage) getStateMutations() []*protocol.FromFunction_PersistedValueMutation { 142 | mutations := make([]*protocol.FromFunction_PersistedValueMutation, 0) 143 | for name, cell := range s.cells { 144 | if mutation := cell.GetStateMutation(name); mutation != nil { 145 | mutations = append(mutations, mutation) 146 | } 147 | } 148 | 149 | return mutations 150 | } 151 | 152 | type MissingSpecs []*protocol.FromFunction_PersistedValueSpec 153 | 154 | func (m MissingSpecs) getStorage() *storage { 155 | return nil 156 | } 157 | 158 | func (m MissingSpecs) getMissingSpecs() []*protocol.FromFunction_PersistedValueSpec { 159 | return m 160 | } 161 | -------------------------------------------------------------------------------- /pkg/statefun/typename.go: -------------------------------------------------------------------------------- 1 | package statefun 2 | 3 | import ( 4 | "errors" 5 | "fmt" 6 | "statefun-sdk-go/pkg/statefun/internal/protocol" 7 | "strings" 8 | ) 9 | 10 | var ( 11 | boolTypeName = TypeNameFrom("io.statefun.types/bool") 12 | int32TypeName = TypeNameFrom("io.statefun.types/int") 13 | int64TypeName = TypeNameFrom("io.statefun.types/long") 14 | float32TypeName = TypeNameFrom("io.statefun.types/float") 15 | float64ypeName = TypeNameFrom("io.statefun.types/double") 16 | stringTypeName = TypeNameFrom("io.statefun.types/string") 17 | ) 18 | 19 | // A TypeName is used to uniquely identify objects within 20 | // a Stateful Functions application, including functions, 21 | // egresses, and types. TypeName's serve as an integral 22 | // part of identifying these objects for message delivery 23 | // as well as message data serialization and deserialization. 24 | type TypeName interface { 25 | fmt.Stringer 26 | GetNamespace() string 27 | GetType() string 28 | } 29 | 30 | type typeName struct { 31 | namespace string 32 | tpe string 33 | typenameString string 34 | } 35 | 36 | func (t typeName) String() string { 37 | return t.typenameString 38 | } 39 | 40 | func (t typeName) GetNamespace() string { 41 | return t.namespace 42 | } 43 | 44 | func (t typeName) GetType() string { 45 | return t.tpe 46 | } 47 | 48 | // Creates a TypeName from a canonical string 49 | // in the format `/`. This Function 50 | // assumes correctly formatted strings and will panic 51 | // on error. For runtime error handling please 52 | // see ParseTypeName. 53 | func TypeNameFrom(typename string) TypeName { 54 | result, err := ParseTypeName(typename) 55 | if err != nil { 56 | panic(err) 57 | } 58 | 59 | return result 60 | } 61 | 62 | // Creates a TypeName from a canonical string 63 | // in the format `/`. 64 | func ParseTypeName(typename string) (TypeName, error) { 65 | position := strings.LastIndex(typename, "/") 66 | if position <= 0 || position == len(typename)-1 { 67 | return nil, fmt.Errorf("%v does not conform to the / format", typename) 68 | } 69 | 70 | namespace := typename[:position] 71 | name := typename[position+1:] 72 | 73 | if namespace[len(namespace)-1] == '/' { 74 | namespace = namespace[:len(namespace)-1] 75 | } 76 | 77 | return TypeNameFromParts(namespace, name) 78 | } 79 | 80 | func TypeNameFromParts(namespace, tpe string) (TypeName, error) { 81 | if len(namespace) == 0 { 82 | return nil, errors.New("namespace cannot be empty") 83 | } 84 | 85 | if len(tpe) == 0 { 86 | return nil, errors.New("type cannot be empty") 87 | } 88 | 89 | return typeName{ 90 | namespace: namespace, 91 | tpe: tpe, 92 | typenameString: fmt.Sprintf("%s/%s", namespace, tpe), 93 | }, nil 94 | } 95 | 96 | // An Address is the unique identity of an individual StatefulFunction, 97 | // containing all of the function's FunctionType and a unique identifier 98 | // within the type. The function's type denotes the type (or class) of function 99 | // to invoke, while the unique identifier addresses the invocation to a specific 100 | // function instance. 101 | type Address struct { 102 | FunctionType TypeName 103 | Id string 104 | } 105 | 106 | func (a Address) String() string { 107 | return fmt.Sprintf("Address(%s, %s, %s)", a.FunctionType.GetNamespace(), a.FunctionType.GetType(), a.Id) 108 | } 109 | 110 | func addressFromInternal(a *protocol.Address) Address { 111 | name, _ := TypeNameFromParts(a.Namespace, a.Type) 112 | return Address{ 113 | FunctionType: name, 114 | Id: a.Id, 115 | } 116 | } 117 | -------------------------------------------------------------------------------- /pkg/statefun/typename_test.go: -------------------------------------------------------------------------------- 1 | package statefun 2 | 3 | import ( 4 | "github.com/stretchr/testify/assert" 5 | "testing" 6 | ) 7 | 8 | func TestTypeNameParse(t *testing.T) { 9 | typename, err := ParseTypeName("namespace/tpe") 10 | 11 | assert.NoError(t, err) 12 | assert.Equal(t, typename.GetNamespace(), "namespace") 13 | assert.Equal(t, typename.GetType(), "tpe") 14 | } 15 | 16 | func TestNoNamespace(t *testing.T) { 17 | _, err := ParseTypeName("/bar") 18 | assert.Error(t, err) 19 | } 20 | 21 | func TestNoName(t *testing.T) { 22 | _, err := ParseTypeName("n/") 23 | assert.Error(t, err) 24 | } 25 | 26 | func TestNoNamespaceOrName(t *testing.T) { 27 | _, err := ParseTypeName("/") 28 | assert.Error(t, err) 29 | } 30 | 31 | func TestEmptyString(t *testing.T) { 32 | _, err := ParseTypeName("") 33 | assert.Error(t, err) 34 | } 35 | -------------------------------------------------------------------------------- /pkg/statefun/types.go: -------------------------------------------------------------------------------- 1 | package statefun 2 | 3 | import ( 4 | "encoding/binary" 5 | "encoding/json" 6 | "errors" 7 | "google.golang.org/protobuf/proto" 8 | "io" 9 | "io/ioutil" 10 | "log" 11 | "strings" 12 | ) 13 | 14 | // This interface is the core abstraction used byt Stateful 15 | // Function's type system, and consists of a few things 16 | // that StateFun uses to handle Message's and ValueSpec's 17 | // 18 | // 1. TypeName to identify the type. 19 | // 2. (De)serialization methods for marshalling and unmarshalling data 20 | // 21 | // Cross-language primitive types 22 | // 23 | // StateFun's type system has cross-language support for common primitive 24 | // types, such as boolean, integer (int32), long (int64), etc. These 25 | // primitive types have built-in SimpleType's implemented for them already 26 | // with predefined TypeName's. 27 | // 28 | // These primitives have standard encoding across all StateFun language 29 | // SDKs, so functions in various other languages (Java, Python, etc) can 30 | // message Golang functions by directly sending supported primitive 31 | // values as message arguments. Moreover, the type system is used for 32 | // state values as well; so you can expect that a function can safely 33 | // read previous state after reimplementing it in a different language. 34 | // 35 | // Common custom types 36 | // 37 | // The type system is also very easily extensible to support more complex types. 38 | // The Go SDK ships with predefined support for JSON and Protobuf - see MakeJsonType 39 | // MakeProtobufType. For other formats, it is just a matter of implementing 40 | // your own SimpleType with a custom typename and serializer. 41 | type SimpleType interface { 42 | GetTypeName() TypeName 43 | 44 | Deserialize(r io.Reader, receiver interface{}) error 45 | 46 | Serialize(writer io.Writer, data interface{}) error 47 | } 48 | 49 | type PrimitiveType int 50 | 51 | const ( 52 | BoolType PrimitiveType = iota 53 | Int32Type 54 | Int64Type 55 | Float32Type 56 | Float64Type 57 | StringType 58 | ) 59 | 60 | func (p PrimitiveType) GetTypeName() TypeName { 61 | switch p { 62 | case BoolType: 63 | return boolTypeName 64 | case Int32Type: 65 | return int32TypeName 66 | case Int64Type: 67 | return int64TypeName 68 | case Float32Type: 69 | return float32TypeName 70 | case Float64Type: 71 | return float64ypeName 72 | case StringType: 73 | return stringTypeName 74 | default: 75 | log.Fatalf("unknown primitive type %v", p) 76 | // unreachable 77 | return nil 78 | } 79 | } 80 | 81 | func (p PrimitiveType) Deserialize(r io.Reader, receiver interface{}) error { 82 | switch p { 83 | case BoolType: 84 | switch receiver.(type) { 85 | case *bool: 86 | return binary.Read(r, binary.BigEndian, receiver) 87 | default: 88 | return errors.New("receiver must be of type bool or *bool") 89 | } 90 | case Int32Type: 91 | switch receiver.(type) { 92 | case *int32: 93 | return binary.Read(r, binary.BigEndian, receiver) 94 | default: 95 | return errors.New("receiver must be of type *int32") 96 | } 97 | case Int64Type: 98 | switch receiver.(type) { 99 | case *int64: 100 | return binary.Read(r, binary.BigEndian, receiver) 101 | default: 102 | return errors.New("receiver must be of type *int64") 103 | } 104 | case Float32Type: 105 | switch receiver.(type) { 106 | case *float32: 107 | return binary.Read(r, binary.BigEndian, receiver) 108 | default: 109 | return errors.New("receiver must be of type *float32") 110 | } 111 | case Float64Type: 112 | switch receiver.(type) { 113 | case *float64: 114 | return binary.Read(r, binary.BigEndian, receiver) 115 | default: 116 | return errors.New("receiver must be of type *float64") 117 | } 118 | case StringType: 119 | switch receiver := receiver.(type) { 120 | case *string: 121 | buf := strings.Builder{} 122 | _, _ = io.Copy(&buf, r) 123 | *receiver = buf.String() 124 | return nil 125 | default: 126 | return errors.New("receiver must be of type *string") 127 | } 128 | default: 129 | log.Fatalf("unknown primitive type %v", p) 130 | // unreachable 131 | return nil 132 | } 133 | } 134 | 135 | func (p PrimitiveType) Serialize(writer io.Writer, data interface{}) error { 136 | switch p { 137 | case BoolType: 138 | switch data.(type) { 139 | case bool, *bool: 140 | return binary.Write(writer, binary.BigEndian, data) 141 | default: 142 | return errors.New("data must be of type bool or *bool") 143 | } 144 | case Int32Type: 145 | switch data.(type) { 146 | case int32, *int32: 147 | return binary.Write(writer, binary.BigEndian, data) 148 | default: 149 | return errors.New("data must be of type int32 or *int32") 150 | } 151 | case Int64Type: 152 | switch data.(type) { 153 | case int64, *int64: 154 | return binary.Write(writer, binary.BigEndian, data) 155 | default: 156 | return errors.New("data must be of type int64 or *int64") 157 | } 158 | case Float32Type: 159 | switch data.(type) { 160 | case float32, *float32: 161 | return binary.Write(writer, binary.BigEndian, data) 162 | default: 163 | return errors.New("data must be of type float32 or *float32") 164 | } 165 | case Float64Type: 166 | switch data.(type) { 167 | case float64, *float64: 168 | return binary.Write(writer, binary.BigEndian, data) 169 | default: 170 | return errors.New("data must be of type float64 or *float64") 171 | } 172 | case StringType: 173 | switch data := data.(type) { 174 | case string: 175 | switch writer := writer.(type) { 176 | case io.StringWriter: 177 | _, err := writer.WriteString(data) 178 | return err 179 | default: 180 | _, err := writer.Write([]byte(data)) 181 | return err 182 | } 183 | case *string: 184 | switch writer := writer.(type) { 185 | case io.StringWriter: 186 | _, err := writer.WriteString(*data) 187 | return err 188 | default: 189 | _, err := writer.Write([]byte(*data)) 190 | return err 191 | } 192 | default: 193 | return errors.New("data must be of type string or *string") 194 | } 195 | default: 196 | log.Fatalf("unknown primitive type %v", p) 197 | // unreachable 198 | return nil 199 | } 200 | } 201 | 202 | type jsonType struct { 203 | typeName TypeName 204 | } 205 | 206 | // Creates a new SimpleType with a given TypeName 207 | // using the standard Go JSON library. 208 | func MakeJsonType(name TypeName) SimpleType { 209 | return jsonType{typeName: name} 210 | } 211 | 212 | func (j jsonType) GetTypeName() TypeName { 213 | return j.typeName 214 | } 215 | 216 | func (j jsonType) Deserialize(r io.Reader, receiver interface{}) error { 217 | return json.NewDecoder(r).Decode(receiver) 218 | } 219 | 220 | func (j jsonType) Serialize(writer io.Writer, data interface{}) error { 221 | return json.NewEncoder(writer).Encode(data) 222 | } 223 | 224 | type protoType struct { 225 | typeName TypeName 226 | } 227 | 228 | // Creates a new SimpleType for the given protobuf Message. 229 | func MakeProtobufType(m proto.Message) SimpleType { 230 | return MakeProtobufTypeWithNamespace(m, "type.googleapis.com") 231 | } 232 | 233 | // Creates a new SimpleType for the given protobuf Message with a custom namespace. 234 | func MakeProtobufTypeWithNamespace(m proto.Message, namespace string) SimpleType { 235 | name := proto.MessageName(m) 236 | tName, _ := TypeNameFromParts(namespace, string(name)) 237 | return protoType{ 238 | typeName: tName, 239 | } 240 | } 241 | 242 | func (p protoType) GetTypeName() TypeName { 243 | return p.typeName 244 | } 245 | 246 | func (p protoType) Deserialize(r io.Reader, receiver interface{}) error { 247 | switch receiver := receiver.(type) { 248 | case proto.Message: 249 | data, err := ioutil.ReadAll(r) 250 | if err != nil { 251 | return err 252 | } 253 | 254 | return proto.Unmarshal(data, receiver) 255 | default: 256 | return errors.New("receiver must implement proto.Message") 257 | } 258 | } 259 | 260 | func (p protoType) Serialize(writer io.Writer, data interface{}) error { 261 | switch data := data.(type) { 262 | case proto.Message: 263 | if value, err := proto.Marshal(data); err != nil { 264 | return err 265 | } else { 266 | _, err = writer.Write(value) 267 | return err 268 | } 269 | 270 | default: 271 | return errors.New("data must implement proto.Message") 272 | } 273 | } 274 | -------------------------------------------------------------------------------- /pkg/statefun/types_test.go: -------------------------------------------------------------------------------- 1 | package statefun 2 | 3 | import ( 4 | "bytes" 5 | "github.com/stretchr/testify/assert" 6 | "testing" 7 | ) 8 | 9 | func TestBoolType(t *testing.T) { 10 | buffer := bytes.Buffer{} 11 | err := BoolType.Serialize(&buffer, true) 12 | assert.NoError(t, err) 13 | 14 | var result bool 15 | err = BoolType.Deserialize(bytes.NewReader(buffer.Bytes()), &result) 16 | assert.NoError(t, err) 17 | 18 | assert.True(t, result) 19 | } 20 | 21 | func TestIntType(t *testing.T) { 22 | buffer := bytes.Buffer{} 23 | err := Int32Type.Serialize(&buffer, int32(1)) 24 | assert.NoError(t, err) 25 | 26 | var result int32 27 | err = Int32Type.Deserialize(bytes.NewReader(buffer.Bytes()), &result) 28 | assert.NoError(t, err) 29 | 30 | assert.Equal(t, result, int32(1)) 31 | } 32 | 33 | func TestLongType(t *testing.T) { 34 | buffer := bytes.Buffer{} 35 | err := Int64Type.Serialize(&buffer, int64(1<<45)) 36 | assert.NoError(t, err) 37 | 38 | var result int64 39 | err = Int64Type.Deserialize(bytes.NewReader(buffer.Bytes()), &result) 40 | assert.NoError(t, err) 41 | 42 | assert.Equal(t, result, int64(1<<45)) 43 | } 44 | 45 | func TestFloatType(t *testing.T) { 46 | buffer := bytes.Buffer{} 47 | err := Float32Type.Serialize(&buffer, float32(0.5)) 48 | assert.NoError(t, err) 49 | 50 | var result float32 51 | err = Float32Type.Deserialize(bytes.NewReader(buffer.Bytes()), &result) 52 | assert.NoError(t, err) 53 | 54 | assert.Equal(t, result, float32(0.5)) 55 | } 56 | 57 | func TestDoubleType(t *testing.T) { 58 | buffer := bytes.Buffer{} 59 | err := Float64Type.Serialize(&buffer, float64(1e-20)) 60 | assert.NoError(t, err) 61 | 62 | var result float64 63 | err = Float64Type.Deserialize(bytes.NewReader(buffer.Bytes()), &result) 64 | assert.NoError(t, err) 65 | 66 | assert.Equal(t, result, float64(1e-20)) 67 | } 68 | 69 | func TestStringType(t *testing.T) { 70 | buffer := bytes.Buffer{} 71 | err := StringType.Serialize(&buffer, "hello world") 72 | assert.NoError(t, err) 73 | 74 | var result string 75 | err = StringType.Deserialize(bytes.NewReader(buffer.Bytes()), &result) 76 | assert.NoError(t, err) 77 | 78 | assert.Equal(t, result, "hello world") 79 | } 80 | 81 | type User struct { 82 | FirstName string `json:"first_name"` 83 | LastName string `json:"last_name"` 84 | } 85 | 86 | func TestJsonType(t *testing.T) { 87 | buffer := bytes.Buffer{} 88 | userType := MakeJsonType(TypeNameFrom("org.foo.bar/UserJson")) 89 | 90 | err := userType.Serialize(&buffer, User{"bob", "mop"}) 91 | assert.NoError(t, err) 92 | 93 | var result User 94 | err = userType.Deserialize(bytes.NewReader(buffer.Bytes()), &result) 95 | assert.NoError(t, err) 96 | 97 | assert.Equal(t, result, User{"bob", "mop"}) 98 | } 99 | -------------------------------------------------------------------------------- /pkg/statefun/value_spec.go: -------------------------------------------------------------------------------- 1 | package statefun 2 | 3 | import ( 4 | "fmt" 5 | "log" 6 | "regexp" 7 | "time" 8 | ) 9 | 10 | type expirationType int 11 | 12 | const ( 13 | none expirationType = iota 14 | expireAfterCall 15 | expireAfterWrite 16 | ) 17 | 18 | func (e expirationType) String() string { 19 | switch e { 20 | case expireAfterCall: 21 | return "expire_after_call" 22 | case expireAfterWrite: 23 | return "expire_after_write" 24 | case none: 25 | return "none" 26 | default: 27 | panic("unknown Expiration type") 28 | } 29 | } 30 | 31 | // State Expiration Configuration 32 | // 33 | // Defines the way state can be auto expired by the runtime. 34 | // State Expiration (also known as TTL) can be used to keep 35 | // state from growing arbitrarily by assigning an Expiration 36 | // date to a value. 37 | // 38 | // State can be expired after a duration has passed since either 39 | // the last write to the state, or the last call to the Function. 40 | type Expiration struct { 41 | expirationType 42 | duration time.Duration 43 | } 44 | 45 | func (e Expiration) String() string { 46 | return fmt.Sprintf("Expiration{mode=%v, duration=%v}", e.expirationType.String(), e.duration.String()) 47 | } 48 | 49 | // Returns an Expiration configuration that would expire 50 | // a duration after the last invocation of the Function. 51 | func ExpireAfterCall(duration time.Duration) Expiration { 52 | return Expiration{ 53 | expireAfterCall, 54 | duration, 55 | } 56 | } 57 | 58 | // Returns an Expiration configuration that would expire 59 | // a duration after the last write. 60 | func ExpireAfterWrite(duration time.Duration) Expiration { 61 | return Expiration{ 62 | expireAfterWrite, 63 | duration, 64 | } 65 | } 66 | 67 | // A ValueSpec identifies a registered persistent value of a function, which will be 68 | // managed by the Stateful Functions runtime for consistency and fault-tolerance. A 69 | // ValueSpec is registered for a function by configuring it on the function's 70 | // associated StatefulFunctionSpec. 71 | type ValueSpec struct { 72 | // The given tpe of the persistent value. The tpe must be a valid 73 | // identifier conforming to the following rules: 74 | // 75 | // 1. First character must be an alphabet letter [a-z] / [A-Z], or an underscore '_'. 76 | // 2. Remaining characters can be an alphabet letter [a-z] / [A-Z], a digit [0-9], or 77 | // an underscore '-'. 78 | // 3. Must not contain any spaces. 79 | Name string 80 | 81 | // The SimpleType of the persistent value. Either 82 | // a built-in PrimitiveType or custom implementation. 83 | ValueType SimpleType 84 | 85 | // An optional expiration configuration. 86 | Expiration Expiration 87 | } 88 | 89 | const invalidNameMessage = ` 90 | invalid state tpe %s. state names can only start with alphabet letters [a-z][A-Z] or an underscore '_' followed by zero or more characters that are alphanumeric or underscores 91 | ` 92 | 93 | func validateValueSpec(s ValueSpec) error { 94 | matched, err := regexp.MatchString("^[a-zA-Z_][a-zA-Z_\\d]*$", s.Name) 95 | if err != nil { 96 | log.Panicf("invalid regex; this is a bug: %v", err) 97 | } 98 | 99 | if !matched { 100 | return fmt.Errorf(invalidNameMessage, s.Name) 101 | } 102 | 103 | return nil 104 | } 105 | -------------------------------------------------------------------------------- /statefun-sdk-go.iml: -------------------------------------------------------------------------------- 1 | 2 | 3 | 4 | 5 | 6 | 7 | 8 | 9 | --------------------------------------------------------------------------------