diff --git a/chasm/engine.go b/chasm/engine.go index 7711967c53c..a2061a5bb7f 100644 --- a/chasm/engine.go +++ b/chasm/engine.go @@ -4,6 +4,12 @@ package chasm import ( "context" + "reflect" + "time" + + commonpb "go.temporal.io/api/common/v1" + "go.temporal.io/api/serviceerror" + "google.golang.org/protobuf/proto" ) // NoValue is a sentinel type representing no value. @@ -46,6 +52,54 @@ type Engine interface { func(MutableContext, Component, any) error, ...TransitionOption, ) ([]byte, error) + + ListExecutions( + context.Context, + reflect.Type, + *ListExecutionsRequest, + ) (*ListExecutionsResponse[*commonpb.Payload], error) + + CountExecutions( + context.Context, + reflect.Type, + *CountExecutionsRequest, + ) (*CountExecutionsResponse, error) +} + +type ListExecutionsResponse[M proto.Message] struct { + Executions []*ExecutionInfo[M] + NextPageToken []byte +} +type ExecutionInfo[M proto.Message] struct { + BusinessID string + RunID string + StartTime time.Time + CloseTime time.Time + HistoryLength int64 + HistorySizeBytes int64 + StateTransitionCount int64 + ChasmSearchAttributes SearchAttributesMap + CustomSearchAttributes map[string]*commonpb.Payload + Memo *commonpb.Memo + ChasmMemo M +} + +type ListExecutionsRequest struct { + NamespaceID string + NamespaceName string + Query string + PageSize int + NextPageToken []byte +} + +type CountExecutionsRequest struct { + NamespaceID string + NamespaceName string + Query string +} + +type CountExecutionsResponse struct { + Count int64 } type BusinessIDReusePolicy int @@ -304,3 +358,73 @@ func engineFromContext( } return e } + +// ListExecutions lists the executions of a CHASM archetype given an initial query. +// The query string can specify any combination of CHASM, custom, and predefined/system search attributes. +// The generic parameter C is the CHASM component type used for executions and search attribute filtering. +// The generic parameter M is the type of the memo payload to be unmarshaled from the execution. +// PageSize is required, must be greater than 0. +// NextPageToken is optional, set on subsequent requests to continue listing the next page of executions. +// Note: For CHASM executions, TemporalNamespaceDivision is the predefined search attribute +// that is used to identify the archetype of the execution. +// If the query string does not specify TemporalNamespaceDivision, the archetype C of the request will be used to filter the executions. +// If the initial query already specifies TemporalNamespaceDivision, the archetype C of the request will +// only be used to get the registered SearchAttributes. +func ListExecutions[C Component, M proto.Message]( + ctx context.Context, + request *ListExecutionsRequest, +) (*ListExecutionsResponse[M], error) { + archetypeType := reflect.TypeFor[C]() + response, err := engineFromContext(ctx).ListExecutions(ctx, archetypeType, request) + if err != nil { + return nil, err + } + + // Convert response, unmarshaling ChasmMemo to type M + executions := make([]*ExecutionInfo[M], len(response.Executions)) + for i, execution := range response.Executions { + chasmMemoInterface := reflect.New(reflect.TypeFor[M]().Elem()).Interface() + chasmMemo, ok := chasmMemoInterface.(M) + if !ok { + return nil, serviceerror.NewInternalf("failed to cast chasm memo to type %s", reflect.TypeFor[M]().String()) + } + err := proto.Unmarshal(execution.ChasmMemo.Data, chasmMemo) + if err != nil { + return nil, serviceerror.NewInternalf("failed to unmarshal chasm memo: %v", err) + } + executions[i] = &ExecutionInfo[M]{ + BusinessID: execution.BusinessID, + RunID: execution.RunID, + StartTime: execution.StartTime, + CloseTime: execution.CloseTime, + HistoryLength: execution.HistoryLength, + HistorySizeBytes: execution.HistorySizeBytes, + StateTransitionCount: execution.StateTransitionCount, + ChasmSearchAttributes: execution.ChasmSearchAttributes, + CustomSearchAttributes: execution.CustomSearchAttributes, + Memo: execution.Memo, + ChasmMemo: chasmMemo, + } + } + + return &ListExecutionsResponse[M]{ + Executions: executions, + NextPageToken: response.NextPageToken, + }, nil +} + +// CountExecutions counts the executions of a CHASM archetype given an initial query. +// The generic parameter C is the CHASM component type used for executions and search attribute filtering. +// The query string can specify any combination of CHASM, custom, and predefined/system search attributes. +// Note: For CHASM executions, TemporalNamespaceDivision is the predefined search attribute +// that is used to identify the archetype of the execution. +// If the query string does not specify TemporalNamespaceDivision, the archetype C of the request will be used to count the executions. +// If the initial query already specifies TemporalNamespaceDivision, the archetype C of the request will +// only be used to get the registered SearchAttributes. +func CountExecutions[C Component]( + ctx context.Context, + request *CountExecutionsRequest, +) (*CountExecutionsResponse, error) { + archetypeType := reflect.TypeFor[C]() + return engineFromContext(ctx).CountExecutions(ctx, archetypeType, request) +} diff --git a/chasm/engine_mock.go b/chasm/engine_mock.go index cafdd48efd3..35126f297c4 100644 --- a/chasm/engine_mock.go +++ b/chasm/engine_mock.go @@ -13,6 +13,7 @@ import ( context "context" reflect "reflect" + common "go.temporal.io/api/common/v1" gomock "go.uber.org/mock/gomock" ) @@ -40,6 +41,36 @@ func (m *MockEngine) EXPECT() *MockEngineMockRecorder { return m.recorder } +// CountExecutions mocks base method. +func (m *MockEngine) CountExecutions(arg0 context.Context, arg1 reflect.Type, arg2 *CountExecutionsRequest) (*CountExecutionsResponse, error) { + m.ctrl.T.Helper() + ret := m.ctrl.Call(m, "CountExecutions", arg0, arg1, arg2) + ret0, _ := ret[0].(*CountExecutionsResponse) + ret1, _ := ret[1].(error) + return ret0, ret1 +} + +// CountExecutions indicates an expected call of CountExecutions. +func (mr *MockEngineMockRecorder) CountExecutions(arg0, arg1, arg2 any) *gomock.Call { + mr.mock.ctrl.T.Helper() + return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "CountExecutions", reflect.TypeOf((*MockEngine)(nil).CountExecutions), arg0, arg1, arg2) +} + +// ListExecutions mocks base method. +func (m *MockEngine) ListExecutions(arg0 context.Context, arg1 reflect.Type, arg2 *ListExecutionsRequest) (*ListExecutionsResponse[*common.Payload], error) { + m.ctrl.T.Helper() + ret := m.ctrl.Call(m, "ListExecutions", arg0, arg1, arg2) + ret0, _ := ret[0].(*ListExecutionsResponse[*common.Payload]) + ret1, _ := ret[1].(error) + return ret0, ret1 +} + +// ListExecutions indicates an expected call of ListExecutions. +func (mr *MockEngineMockRecorder) ListExecutions(arg0, arg1, arg2 any) *gomock.Call { + mr.mock.ctrl.T.Helper() + return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "ListExecutions", reflect.TypeOf((*MockEngine)(nil).ListExecutions), arg0, arg1, arg2) +} + // NewExecution mocks base method. func (m *MockEngine) NewExecution(arg0 context.Context, arg1 ComponentRef, arg2 func(MutableContext) (Component, error), arg3 ...TransitionOption) (ExecutionKey, []byte, error) { m.ctrl.T.Helper() diff --git a/chasm/export_test.go b/chasm/export_test.go index eed0b206e68..b99be5df7ea 100644 --- a/chasm/export_test.go +++ b/chasm/export_test.go @@ -21,10 +21,6 @@ func (r *Registry) ComponentOf(componentGoType reflect.Type) (*RegistrableCompon return r.componentOf(componentGoType) } -func (r *Registry) ComponentByID(id uint32) (*RegistrableComponent, bool) { - return r.componentByID(id) -} - func (r *Registry) TaskFor(taskInstance any) (*RegistrableTask, bool) { return r.taskFor(taskInstance) } diff --git a/chasm/lib/tests/payload.go b/chasm/lib/tests/payload.go index bd292ac1a3c..761db66f66e 100644 --- a/chasm/lib/tests/payload.go +++ b/chasm/lib/tests/payload.go @@ -6,6 +6,7 @@ import ( "go.temporal.io/server/chasm" "go.temporal.io/server/chasm/lib/tests/gen/testspb/v1" "go.temporal.io/server/common" + "google.golang.org/protobuf/proto" "google.golang.org/protobuf/types/known/timestamppb" ) @@ -154,11 +155,6 @@ func (s *PayloadStore) SearchAttributes( } // Memo implements chasm.VisibilityMemoProvider interface -func (s *PayloadStore) Memo( - _ chasm.Context, -) map[string]chasm.VisibilityValue { - return map[string]chasm.VisibilityValue{ - TotalCountMemoFieldName: chasm.VisibilityValueInt64(s.State.TotalCount), - TotalSizeMemoFieldName: chasm.VisibilityValueInt64(s.State.TotalSize), - } +func (s *PayloadStore) Memo(_ chasm.Context) proto.Message { + return s.State } diff --git a/chasm/ref_test.go b/chasm/ref_test.go index 960a3f9eaa8..98e252c6ff0 100644 --- a/chasm/ref_test.go +++ b/chasm/ref_test.go @@ -51,7 +51,7 @@ func (s *componentRefSuite) TestArchetypeID() { archetypeID, err := ref.ArchetypeID(s.registry) s.NoError(err) - rc, ok := s.registry.ComponentOf(reflect.TypeFor[*TestComponent]()) + rc, ok := s.registry.componentOf(reflect.TypeFor[*TestComponent]()) s.True(ok) s.Equal(rc.componentID, archetypeID) @@ -68,7 +68,7 @@ func (s *componentRefSuite) TestShardingKey() { shardingKey, err := ref.ShardingKey(s.registry) s.NoError(err) - rc, ok := s.registry.ComponentOf(reflect.TypeFor[*TestComponent]()) + rc, ok := s.registry.componentOf(reflect.TypeFor[*TestComponent]()) s.True(ok) s.Equal(rc.shardingFn(executionKey), shardingKey) diff --git a/chasm/registrable_component.go b/chasm/registrable_component.go index b62561833f9..6c71a618f23 100644 --- a/chasm/registrable_component.go +++ b/chasm/registrable_component.go @@ -115,6 +115,11 @@ func (rc *RegistrableComponent) registerToLibrary( return fqn, rc.componentID, nil } +// SearchAttributesMapper returns the search attributes mapper for this component. +func (rc *RegistrableComponent) SearchAttributesMapper() *VisibilitySearchAttributesMapper { + return rc.searchAttributesMapper +} + // fqType returns the fully qualified name of the component, which is a combination of // the library name and the component type. This is used to uniquely identify // the component in the registry. diff --git a/chasm/registry.go b/chasm/registry.go index f819958916c..82592400652 100644 --- a/chasm/registry.go +++ b/chasm/registry.go @@ -81,6 +81,15 @@ func (r *Registry) ComponentFqnByID(id uint32) (string, bool) { return fqn, ok } +// ComponentByID returns the registrable component for a given archetype ID. +func (r *Registry) ComponentByID(id uint32) (*RegistrableComponent, bool) { + fqn, ok := r.componentFqnByID[id] + if !ok { + return nil, false + } + return r.component(fqn) +} + // ComponentIDFor converts registered component instance to component type ID. // This method should only be used by CHASM framework internal code, // NOT CHASM library developers. @@ -136,6 +145,17 @@ func (r *Registry) componentOf(componentGoType reflect.Type) (*RegistrableCompon return rc, ok } +// ArchetypeIDOf returns the ArchetypeID for the given component Go type. +// This method should only be used by CHASM framework internal, +// NOT CHASM library developers. +func (r *Registry) ArchetypeIDOf(componentGoType reflect.Type) (ArchetypeID, bool) { + rc, ok := r.componentByGoType[componentGoType] + if !ok { + return UnspecifiedArchetypeID, false + } + return rc.componentID, true +} + func (r *Registry) taskOf(taskGoType reflect.Type) (*RegistrableTask, bool) { rt, ok := r.taskByGoType[taskGoType] return rt, ok diff --git a/chasm/registry_test.go b/chasm/registry_test.go index 47dbae070e1..c364217dcc3 100644 --- a/chasm/registry_test.go +++ b/chasm/registry_test.go @@ -218,6 +218,47 @@ func (s *RegistryTestSuite) TestRegistry_RegisterComponents_Error() { require.Contains(t, err.Error(), "must be struct or pointer to struct") }) + s.Run("duplicate search attribute alias panics", func() { + s.Require().PanicsWithValue("registrable component validation error: search attribute alias \"MyAlias\" is already defined", + func() { + chasm.NewRegistrableComponent[*chasm.MockComponent]( + "Component1", + chasm.WithSearchAttributes( + chasm.NewSearchAttributeBool("MyAlias", chasm.SearchAttributeFieldBool01), + chasm.NewSearchAttributeInt("MyAlias", chasm.SearchAttributeFieldInt01), + ), + ) + }, + ) + }) + + s.Run("duplicate search attribute field panics", func() { + s.Require().PanicsWithValue("registrable component validation error: search attribute field \"TemporalBool01\" is already defined", + func() { + chasm.NewRegistrableComponent[*chasm.MockComponent]( + "Component1", + chasm.WithSearchAttributes( + chasm.NewSearchAttributeBool("Alias1", chasm.SearchAttributeFieldBool01), + chasm.NewSearchAttributeBool("Alias2", chasm.SearchAttributeFieldBool01), + ), + ) + }, + ) + }) + + s.Run("valid search attributes do not panic", func() { + s.Require().NotPanics(func() { + chasm.NewRegistrableComponent[*chasm.MockComponent]( + "Component1", + chasm.WithSearchAttributes( + chasm.NewSearchAttributeBool("Completed", chasm.SearchAttributeFieldBool01), + chasm.NewSearchAttributeInt("Count", chasm.SearchAttributeFieldInt01), + chasm.NewSearchAttributeKeyword("Status", chasm.SearchAttributeFieldKeyword01), + ), + ) + }) + }) + } func (s *RegistryTestSuite) TestRegistry_RegisterTasks_Error() { diff --git a/chasm/search_attribute.go b/chasm/search_attribute.go index afc35eae522..bfa723d30f8 100644 --- a/chasm/search_attribute.go +++ b/chasm/search_attribute.go @@ -1,10 +1,12 @@ package chasm import ( + "errors" "fmt" "time" enumspb "go.temporal.io/api/enums/v1" + "go.temporal.io/api/serviceerror" "go.temporal.io/server/common/searchattribute/sadefs" ) @@ -65,13 +67,19 @@ var ( ) var ( - _ SearchAttribute = (*searchAttributeDefinition)(nil) _ SearchAttribute = (*SearchAttributeBool)(nil) _ SearchAttribute = (*SearchAttributeDateTime)(nil) _ SearchAttribute = (*SearchAttributeInt)(nil) _ SearchAttribute = (*SearchAttributeDouble)(nil) _ SearchAttribute = (*SearchAttributeKeyword)(nil) _ SearchAttribute = (*SearchAttributeKeywordList)(nil) + + _ typedSearchAttribute[bool] = (*SearchAttributeBool)(nil) + _ typedSearchAttribute[time.Time] = (*SearchAttributeDateTime)(nil) + _ typedSearchAttribute[int64] = (*SearchAttributeInt)(nil) + _ typedSearchAttribute[float64] = (*SearchAttributeDouble)(nil) + _ typedSearchAttribute[string] = (*SearchAttributeKeyword)(nil) + _ typedSearchAttribute[[]string] = (*SearchAttributeKeywordList)(nil) ) type ( @@ -80,6 +88,11 @@ type ( definition() searchAttributeDefinition } + typedSearchAttribute[T any] interface { + SearchAttribute + typeMarker(T) + } + searchAttributeDefinition struct { alias string field string @@ -208,6 +221,8 @@ func (s SearchAttributeBool) Value(value bool) SearchAttributeKeyValue { } } +func (s SearchAttributeBool) typeMarker(_ bool) {} + // SearchAttributeDateTime is a search attribute for a datetime value. type SearchAttributeDateTime struct { searchAttributeDefinition @@ -243,6 +258,8 @@ func (s SearchAttributeDateTime) Value(value time.Time) SearchAttributeKeyValue } } +func (s SearchAttributeDateTime) typeMarker(_ time.Time) {} + // SearchAttributeInt is a search attribute for an integer value. type SearchAttributeInt struct { searchAttributeDefinition @@ -278,6 +295,8 @@ func (s SearchAttributeInt) Value(value int64) SearchAttributeKeyValue { } } +func (s SearchAttributeInt) typeMarker(_ int64) {} + // SearchAttributeDouble is a search attribute for a double value. type SearchAttributeDouble struct { searchAttributeDefinition @@ -313,6 +332,8 @@ func (s SearchAttributeDouble) Value(value float64) SearchAttributeKeyValue { } } +func (s SearchAttributeDouble) typeMarker(_ float64) {} + // SearchAttributeKeyword is a search attribute for a keyword value. type SearchAttributeKeyword struct { searchAttributeDefinition @@ -348,6 +369,8 @@ func (s SearchAttributeKeyword) Value(value string) SearchAttributeKeyValue { } } +func (s SearchAttributeKeyword) typeMarker(_ string) {} + // SearchAttributeKeywordList is a search attribute for a keyword list value. type SearchAttributeKeywordList struct { searchAttributeDefinition @@ -382,3 +405,101 @@ func (s SearchAttributeKeywordList) Value(value []string) SearchAttributeKeyValu Value: VisibilityValueStringSlice(value), } } + +func (s SearchAttributeKeywordList) typeMarker(_ []string) {} + +// SearchAttributesMap wraps search attribute values with type-safe access. +type SearchAttributesMap struct { + values map[string]VisibilityValue +} + +// NewSearchAttributesMap creates a new SearchAttributeMap from raw values. +func NewSearchAttributesMap(values map[string]VisibilityValue) SearchAttributesMap { + return SearchAttributesMap{values: values} +} + +// GetValue returns the value for a given SearchAttribute with compile-time type safety. +// The return type T is inferred from the SearchAttribute's type parameter. +// For example, SearchAttributeBool will return a bool value. +// If the value is not found or the type does not match, the zero value for the type T is returned and the second return value is false. +func GetValue[T any](m SearchAttributesMap, sa typedSearchAttribute[T]) (val T, ok bool) { + var zero T + if len(m.values) == 0 { + return zero, false + } + + alias := sa.definition().alias + visibilityValue, exists := m.values[alias] + if !exists { + return zero, false + } + + finalVal, ok := visibilityValue.Value().(T) + if !ok { + return zero, false + } + return finalVal, true +} + +// convertToVisibilityValue converts a value to VisibilityValue based on its runtime type. +func convertToVisibilityValue(value interface{}) VisibilityValue { + switch val := value.(type) { + case int: + return VisibilityValueInt64(int64(val)) + case int32: + return VisibilityValueInt64(int64(val)) + case int64: + return VisibilityValueInt64(val) + case float32: + return VisibilityValueFloat64(float64(val)) + case float64: + return VisibilityValueFloat64(val) + case bool: + return VisibilityValueBool(val) + case time.Time: + return VisibilityValueTime(val) + case string: + // Try to parse as datetime first + if parsedTime, err := time.Parse(time.RFC3339, val); err == nil { + return VisibilityValueTime(parsedTime) + } + return VisibilityValueString(val) + case []byte: + return VisibilityValueByteSlice(val) + case []string: + return VisibilityValueStringSlice(val) + default: + // Return as string if type is unknown + return VisibilityValueString(fmt.Sprintf("%v", val)) + } +} + +// AliasChasmSearchAttributes converts search attribute values to VisibilityValue and aliases field names. +// It takes a map of field names to interface{} values, converts them to VisibilityValue based on their runtime type, +// and then aliases the field names using the mapper. +func AliasChasmSearchAttributes( + chasmSearchAttributes map[string]interface{}, + mapper *VisibilitySearchAttributesMapper, +) (map[string]VisibilityValue, error) { + if len(chasmSearchAttributes) == 0 { + return nil, nil + } + + chasmSAs := make(map[string]VisibilityValue, len(chasmSearchAttributes)) + for fieldName, value := range chasmSearchAttributes { + visibilityValue := convertToVisibilityValue(value) + aliasName, err := mapper.Alias(fieldName) + if err != nil { + // Silently ignore serviceerror.InvalidArgument because it indicates unmapped field, search attribute is not registered. + // IMPORTANT: Chasm search attributes must be registered with the CHASM Registry using the WithSearchAttributes() option. + var invalidArgumentErr *serviceerror.InvalidArgument + if errors.As(err, &invalidArgumentErr) { + continue + } + return nil, err + } + chasmSAs[aliasName] = visibilityValue + } + + return chasmSAs, nil +} diff --git a/chasm/search_attribute_test.go b/chasm/search_attribute_test.go new file mode 100644 index 00000000000..d0ada33be2e --- /dev/null +++ b/chasm/search_attribute_test.go @@ -0,0 +1,81 @@ +package chasm + +import ( + "testing" + "time" + + "github.com/stretchr/testify/assert" +) + +func TestSearchAttributesMap_Get(t *testing.T) { + // Define test search attributes + boolAttr := NewSearchAttributeBool("completed", SearchAttributeFieldBool01) + intAttr := NewSearchAttributeInt("count", SearchAttributeFieldInt01) + doubleAttr := NewSearchAttributeDouble("score", SearchAttributeFieldDouble01) + keywordAttr := NewSearchAttributeKeyword("status", SearchAttributeFieldKeyword01) + datetimeAttr := NewSearchAttributeDateTime("timestamp", SearchAttributeFieldDateTime01) + keywordListAttr := NewSearchAttributeKeywordList("tags", SearchAttributeFieldKeywordList01) + + now := time.Now() + + // Create map with test values + values := map[string]VisibilityValue{ + "completed": VisibilityValueBool(true), + "count": VisibilityValueInt64(42), + "score": VisibilityValueFloat64(3.14), + "status": VisibilityValueString("active"), + "timestamp": VisibilityValueTime(now), + "tags": VisibilityValueStringSlice([]string{"tag1", "tag2"}), + } + m := NewSearchAttributesMap(values) + + t.Run("GetBool", func(t *testing.T) { + val, ok := GetValue(m, boolAttr) + assert.True(t, ok) + assert.True(t, val) + }) + + t.Run("GetInt64", func(t *testing.T) { + val, ok := GetValue(m, intAttr) + assert.True(t, ok) + assert.Equal(t, int64(42), val) + }) + + t.Run("GetFloat64", func(t *testing.T) { + val, ok := GetValue(m, doubleAttr) + assert.True(t, ok) + assert.InDelta(t, 3.14, val, 0.0001) + }) + + t.Run("GetString", func(t *testing.T) { + val, ok := GetValue(m, keywordAttr) + assert.True(t, ok) + assert.Equal(t, "active", val) + }) + + t.Run("GetTime", func(t *testing.T) { + val, ok := GetValue(m, datetimeAttr) + assert.True(t, ok) + assert.True(t, now.Equal(val)) + }) + + t.Run("GetStringSlice", func(t *testing.T) { + val, ok := GetValue(m, keywordListAttr) + assert.True(t, ok) + assert.Equal(t, []string{"tag1", "tag2"}, val) + }) + + t.Run("NotFound", func(t *testing.T) { + missingAttr := NewSearchAttributeBool("missing", SearchAttributeFieldBool02) + val, ok := GetValue(m, missingAttr) + assert.False(t, ok) + assert.False(t, val) + }) + + t.Run("NilMap", func(t *testing.T) { + emptyMap := NewSearchAttributesMap(nil) + val, ok := GetValue(emptyMap, boolAttr) + assert.False(t, ok) + assert.False(t, val) + }) +} diff --git a/chasm/test_component_test.go b/chasm/test_component_test.go index 99ebd813df8..6ce1a9f7697 100644 --- a/chasm/test_component_test.go +++ b/chasm/test_component_test.go @@ -10,6 +10,7 @@ import ( commonpb "go.temporal.io/api/common/v1" enumspb "go.temporal.io/api/enums/v1" persistencespb "go.temporal.io/server/api/persistence/v1" + "google.golang.org/protobuf/proto" ) type ( @@ -118,10 +119,8 @@ func (tc *TestComponent) SearchAttributes(_ Context) []SearchAttributeKeyValue { } // Memo implements VisibilityMemoProvider interface. -func (tc *TestComponent) Memo(_ Context) map[string]VisibilityValue { - return map[string]VisibilityValue{ - TestComponentStartTimeMemoKey: VisibilityValueTime(tc.ComponentData.GetStartTime().AsTime()), - } +func (tc *TestComponent) Memo(_ Context) proto.Message { + return tc.ComponentData } func (tsc1 *TestSubComponent1) LifecycleState(_ Context) LifecycleState { diff --git a/chasm/test_visibility.go b/chasm/test_visibility.go new file mode 100644 index 00000000000..b525e89264f --- /dev/null +++ b/chasm/test_visibility.go @@ -0,0 +1,20 @@ +package chasm + +import enumspb "go.temporal.io/api/enums/v1" + +// NewTestVisibilitySearchAttributesMapper creates a new VisibilitySearchAttributesMapper. +// For testing only. +func NewTestVisibilitySearchAttributesMapper( + fieldToAlias map[string]string, + saTypeMap map[string]enumspb.IndexedValueType, +) *VisibilitySearchAttributesMapper { + aliasToField := make(map[string]string, len(fieldToAlias)) + for field, alias := range fieldToAlias { + aliasToField[alias] = field + } + return &VisibilitySearchAttributesMapper{ + aliasToField: aliasToField, + fieldToAlias: fieldToAlias, + saTypeMap: saTypeMap, + } +} diff --git a/chasm/tree.go b/chasm/tree.go index 084541767d0..54ac429a21c 100644 --- a/chasm/tree.go +++ b/chasm/tree.go @@ -144,7 +144,7 @@ type ( // Root component's search attributes and memo at the start of a transaction. // They will be updated upon CloseTransaction() if they are changed. currentSA map[string]VisibilityValue - currentMemo map[string]VisibilityValue + currentMemo proto.Message needsPointerResolution bool } @@ -1496,7 +1496,7 @@ func (n *Node) closeTransactionForceUpdateVisibility( memoProvider, ok := rootComponent.(VisibilityMemoProvider) if ok { newMemo := memoProvider.Memo(immutableContext) - if !maps.EqualFunc(n.currentMemo, newMemo, isVisibilityValueEqual) { + if !proto.Equal(n.currentMemo, newMemo) { needUpdate = true } n.currentMemo = newMemo diff --git a/chasm/tree_test.go b/chasm/tree_test.go index 310be49a019..e900fdd46a2 100644 --- a/chasm/tree_test.go +++ b/chasm/tree_test.go @@ -880,7 +880,10 @@ func (s *nodeSuite) TestApplyMutation() { root, err := s.newTestTree(persistenceNodes) s.NoError(err) s.Len(root.currentSA, 3) - s.Len(root.currentMemo, 1) + s.NotNil(root.currentMemo) + initialMemo, ok := root.currentMemo.(*protoMessageType) + s.True(ok) + s.ProtoEqual(&protoMessageType{}, initialMemo) // Manually deserialize some tasks to populate the taskValueCache _, err = root.deserializeComponentTask(root.serializedNode.Metadata.GetComponentAttributes().PureTasks[0]) @@ -953,10 +956,15 @@ func (s *nodeSuite) TestApplyMutation() { s.Equal(updatedRoot, root.serializedNode) s.NotNil(root.value) s.Len(root.currentSA, 3) - s.Len(root.currentMemo, 1) + s.Len(root.currentSA, 3) s.Contains(root.currentSA, "TemporalDatetime01") s.True(root.currentSA["TemporalDatetime01"].(VisibilityValueTime).Equal(VisibilityValueTime(now))) - s.True(root.currentMemo[TestComponentStartTimeMemoKey].(VisibilityValueTime).Equal(VisibilityValueTime(now))) + + // Validate memo content. + s.NotNil(root.currentMemo) + decodedMemo, ok := root.currentMemo.(*protoMessageType) + s.True(ok, "currentMemo should be of type *protoMessageType") + s.True(decodedMemo.StartTime.AsTime().Equal(now)) // Validate the "child" node got updated. nodeSC1, ok := root.children["SubComponent1"] @@ -1088,10 +1096,8 @@ func (s *nodeSuite) TestApplySnapshot() { // Validate visibility search attributes and memo are updated as well. s.Len(root.currentSA, 3) - s.Len(root.currentMemo, 1) s.Contains(root.currentSA, "TemporalDatetime01") s.True(root.currentSA["TemporalDatetime01"].(VisibilityValueTime).Equal(VisibilityValueTime(now.AsTime()))) - s.True(root.currentMemo[TestComponentStartTimeMemoKey].(VisibilityValueTime).Equal(VisibilityValueTime(now.AsTime()))) } func (s *nodeSuite) TestApplyMutation_OutOfOrder() { @@ -1936,8 +1942,8 @@ func (s *nodeSuite) TestCloseTransaction_ForceUpdateVisibility_RootLifecycleChan s.Equal(enumsspb.WORKFLOW_EXECUTION_STATE_RUNNING, s.nodeBackend.UpdateCalls[0].State) s.Equal(enumspb.WORKFLOW_EXECUTION_STATUS_RUNNING, s.nodeBackend.UpdateCalls[0].Status) - // Some change unrelated to visibility - // Visibility component should not be updated. + // Change ComponentData which is used as Memo. Even though lifecycle didn't change, + // visibility should be updated because memo changed. nextTransitionCount = 2 testComponent, err = node.Component(chasmCtx, ComponentRef{componentPath: rootPath}) s.NoError(err) @@ -1949,8 +1955,9 @@ func (s *nodeSuite) TestCloseTransaction_ForceUpdateVisibility_RootLifecycleChan } mutation, err = node.CloseTransaction() s.NoError(err) - _, ok = mutation.UpdatedNodes["Visibility"] - s.False(ok) + pVisibilityNode, ok = mutation.UpdatedNodes["Visibility"] + s.True(ok, "visibility should be updated when memo changes") + s.Len(pVisibilityNode.GetMetadata().GetComponentAttributes().SideEffectTasks, 1) // Close the run, visibility should be force updated // even if not explicitly updated. diff --git a/chasm/visibility.go b/chasm/visibility.go index e6e0de39457..c8229bbb3c8 100644 --- a/chasm/visibility.go +++ b/chasm/visibility.go @@ -9,9 +9,13 @@ import ( "go.temporal.io/api/serviceerror" persistencespb "go.temporal.io/server/api/persistence/v1" "go.temporal.io/server/common/payload" + "google.golang.org/protobuf/proto" ) const ( + UserMemoKey = "__user__" + ChasmMemoKey = "__chasm__" + visibilityComponentType = "core.vis" visibilityTaskType = "core.visTask" ) @@ -34,7 +38,7 @@ type VisibilitySearchAttributesProvider interface { // a transaction, if a visibility task needs to be generated to update the // visibility record with the returned memo. type VisibilityMemoProvider interface { - Memo(Context) map[string]VisibilityValue + Memo(Context) proto.Message } // VisibilitySearchAttributesMapper is a mapper for CHASM search attributes. @@ -76,6 +80,19 @@ func (v *VisibilitySearchAttributesMapper) SATypeMap() map[string]enumspb.Indexe return v.saTypeMap } +// ValueType returns the type of a CHASM search attribute field. +// Returns an error if the field is not found in the type map. +func (v *VisibilitySearchAttributesMapper) ValueType(fieldName string) (enumspb.IndexedValueType, error) { + if v == nil { + return enumspb.INDEXED_VALUE_TYPE_UNSPECIFIED, serviceerror.NewInvalidArgument("visibility search attributes mapper not defined") + } + typ, ok := v.saTypeMap[fieldName] + if !ok { + return enumspb.INDEXED_VALUE_TYPE_UNSPECIFIED, serviceerror.NewInvalidArgumentf("visibility search attributes mapper has no registered field %q", fieldName) + } + return typ, nil +} + type Visibility struct { UnimplementedComponent diff --git a/chasm/visibility_value.go b/chasm/visibility_value.go index 2123c7db05f..7c2a6696d8c 100644 --- a/chasm/visibility_value.go +++ b/chasm/visibility_value.go @@ -13,6 +13,7 @@ import ( type VisibilityValue interface { MustEncode() *commonpb.Payload Equal(VisibilityValue) bool + Value() any } type VisibilityValueInt int @@ -31,6 +32,10 @@ func (v VisibilityValueInt) Equal(other VisibilityValue) bool { return v == ov } +func (v VisibilityValueInt) Value() any { + return int(v) +} + type VisibilityValueInt32 int32 func (v VisibilityValueInt32) MustEncode() *commonpb.Payload { @@ -47,6 +52,10 @@ func (v VisibilityValueInt32) Equal(other VisibilityValue) bool { return v == ov } +func (v VisibilityValueInt32) Value() any { + return int32(v) +} + type VisibilityValueInt64 int64 func (v VisibilityValueInt64) MustEncode() *commonpb.Payload { @@ -63,6 +72,10 @@ func (v VisibilityValueInt64) Equal(other VisibilityValue) bool { return v == ov } +func (v VisibilityValueInt64) Value() any { + return int64(v) +} + type VisibilityValueString string func (v VisibilityValueString) MustEncode() *commonpb.Payload { @@ -79,6 +92,10 @@ func (v VisibilityValueString) Equal(other VisibilityValue) bool { return v == ov } +func (v VisibilityValueString) Value() any { + return string(v) +} + type VisibilityValueBool bool func (v VisibilityValueBool) MustEncode() *commonpb.Payload { @@ -95,6 +112,10 @@ func (v VisibilityValueBool) Equal(other VisibilityValue) bool { return v == ov } +func (v VisibilityValueBool) Value() any { + return bool(v) +} + type VisibilityValueFloat64 float64 func (v VisibilityValueFloat64) MustEncode() *commonpb.Payload { @@ -111,6 +132,10 @@ func (v VisibilityValueFloat64) Equal(other VisibilityValue) bool { return v == ov } +func (v VisibilityValueFloat64) Value() any { + return float64(v) +} + type VisibilityValueTime time.Time func (v VisibilityValueTime) MustEncode() *commonpb.Payload { @@ -127,6 +152,10 @@ func (v VisibilityValueTime) Equal(other VisibilityValue) bool { return time.Time(v).Equal(time.Time(ov)) } +func (v VisibilityValueTime) Value() any { + return time.Time(v) +} + type VisibilityValueByteSlice []byte func (v VisibilityValueByteSlice) MustEncode() *commonpb.Payload { @@ -141,6 +170,10 @@ func (v VisibilityValueByteSlice) Equal(other VisibilityValue) bool { return slices.Equal(v, ov) } +func (v VisibilityValueByteSlice) Value() any { + return []byte(v) +} + type VisibilityValueStringSlice []string func (v VisibilityValueStringSlice) MustEncode() *commonpb.Payload { @@ -157,6 +190,10 @@ func (v VisibilityValueStringSlice) Equal(other VisibilityValue) bool { return slices.Equal(v, ov) } +func (v VisibilityValueStringSlice) Value() any { + return []string(v) +} + func isVisibilityValueEqual(v1, v2 VisibilityValue) bool { if v1 == nil && v2 == nil { return true diff --git a/common/metrics/metric_defs.go b/common/metrics/metric_defs.go index 3a1169dfd33..b57a73abacf 100644 --- a/common/metrics/metric_defs.go +++ b/common/metrics/metric_defs.go @@ -280,10 +280,14 @@ const ( VisibilityPersistenceDeleteWorkflowExecutionScope = "DeleteWorkflowExecution" // VisibilityPersistenceListWorkflowExecutionsScope tracks ListWorkflowExecutions calls made by service to visibility persistence layer VisibilityPersistenceListWorkflowExecutionsScope = "ListWorkflowExecutions" + // VisibilityPersistenceListChasmExecutionsScope tracks ListChasmExecutions calls made by service to visibility persistence layer + VisibilityPersistenceListChasmExecutionsScope = "ListChasmExecutions" // VisibilityPersistenceScanWorkflowExecutionsScope tracks ScanWorkflowExecutions calls made by service to visibility persistence layer VisibilityPersistenceScanWorkflowExecutionsScope = "ScanWorkflowExecutions" // VisibilityPersistenceCountWorkflowExecutionsScope tracks CountWorkflowExecutions calls made by service to visibility persistence layer VisibilityPersistenceCountWorkflowExecutionsScope = "CountWorkflowExecutions" + // VisibilityPersistenceCountChasmExecutionsScope tracks CountChasmExecutions calls made by service to visibility persistence layer + VisibilityPersistenceCountChasmExecutionsScope = "CountChasmExecutions" // VisibilityPersistenceGetWorkflowExecutionScope tracks GetWorkflowExecution calls made by service to visibility persistence layer VisibilityPersistenceGetWorkflowExecutionScope = "GetWorkflowExecution" // VisibilityPersistenceAddSearchAttributesScope tracks AddSearchAttributes calls made by service to visibility persistence layer diff --git a/common/persistence/tests/visibility_persistence_suite.go b/common/persistence/tests/visibility_persistence_suite.go index e5631e76b1f..d31760ca68b 100644 --- a/common/persistence/tests/visibility_persistence_suite.go +++ b/common/persistence/tests/visibility_persistence_suite.go @@ -11,6 +11,7 @@ import ( enumspb "go.temporal.io/api/enums/v1" workflowpb "go.temporal.io/api/workflow/v1" "go.temporal.io/api/workflowservice/v1" + "go.temporal.io/server/chasm" "go.temporal.io/server/common/debug" "go.temporal.io/server/common/dynamicconfig" "go.temporal.io/server/common/log/tag" @@ -70,6 +71,7 @@ func (s *VisibilityPersistenceSuite) SetupSuite() { s.SearchAttributesProvider, s.SearchAttributesMapperProvider, s.NamespaceRegistry, + chasm.NewRegistry(nil), dynamicconfig.GetIntPropertyFn(1000), dynamicconfig.GetIntPropertyFn(1000), dynamicconfig.GetFloatPropertyFn(0.2), diff --git a/common/persistence/visibility/factory.go b/common/persistence/visibility/factory.go index e90f55e77b5..8684f9b0d2d 100644 --- a/common/persistence/visibility/factory.go +++ b/common/persistence/visibility/factory.go @@ -1,6 +1,7 @@ package visibility import ( + "go.temporal.io/server/chasm" "go.temporal.io/server/common/config" "go.temporal.io/server/common/dynamicconfig" "go.temporal.io/server/common/log" @@ -36,6 +37,7 @@ func NewManager( searchAttributesProvider searchattribute.Provider, searchAttributesMapperProvider searchattribute.MapperProvider, namespaceRegistry namespace.Registry, + chasmRegistry *chasm.Registry, maxReadQPS dynamicconfig.IntPropertyFn, maxWriteQPS dynamicconfig.IntPropertyFn, @@ -59,6 +61,7 @@ func NewManager( searchAttributesProvider, searchAttributesMapperProvider, namespaceRegistry, + chasmRegistry, maxReadQPS, maxWriteQPS, operatorRPSRatio, @@ -85,6 +88,7 @@ func NewManager( searchAttributesProvider, searchAttributesMapperProvider, namespaceRegistry, + chasmRegistry, maxReadQPS, maxWriteQPS, operatorRPSRatio, @@ -167,6 +171,7 @@ func newVisibilityManagerFromDataStoreConfig( searchAttributesProvider searchattribute.Provider, searchAttributesMapperProvider searchattribute.MapperProvider, namespaceRegistry namespace.Registry, + chasmRegistry *chasm.Registry, maxReadQPS dynamicconfig.IntPropertyFn, maxWriteQPS dynamicconfig.IntPropertyFn, @@ -187,6 +192,7 @@ func newVisibilityManagerFromDataStoreConfig( searchAttributesProvider, searchAttributesMapperProvider, namespaceRegistry, + chasmRegistry, visibilityDisableOrderByClause, visibilityEnableManualPagination, visibilityEnableUnifiedQueryConverter, @@ -221,6 +227,7 @@ func newVisibilityStoreFromDataStoreConfig( searchAttributesProvider searchattribute.Provider, searchAttributesMapperProvider searchattribute.MapperProvider, namespaceRegistry namespace.Registry, + chasmRegistry *chasm.Registry, visibilityDisableOrderByClause dynamicconfig.BoolPropertyFnWithNamespaceFilter, visibilityEnableManualPagination dynamicconfig.BoolPropertyFnWithNamespaceFilter, visibilityEnableUnifiedQueryConverter dynamicconfig.BoolPropertyFn, @@ -238,6 +245,7 @@ func newVisibilityStoreFromDataStoreConfig( persistenceResolver, searchAttributesProvider, searchAttributesMapperProvider, + chasmRegistry, visibilityEnableUnifiedQueryConverter, logger, metricsHandler, @@ -248,6 +256,7 @@ func newVisibilityStoreFromDataStoreConfig( esProcessorConfig, searchAttributesProvider, searchAttributesMapperProvider, + chasmRegistry, visibilityDisableOrderByClause, visibilityEnableManualPagination, visibilityEnableUnifiedQueryConverter, diff --git a/common/persistence/visibility/manager/visibility_manager.go b/common/persistence/visibility/manager/visibility_manager.go index 40d124f901f..bb8da3121d0 100644 --- a/common/persistence/visibility/manager/visibility_manager.go +++ b/common/persistence/visibility/manager/visibility_manager.go @@ -11,6 +11,7 @@ import ( enumspb "go.temporal.io/api/enums/v1" workflowpb "go.temporal.io/api/workflow/v1" "go.temporal.io/api/workflowservice/v1" + "go.temporal.io/server/chasm" "go.temporal.io/server/common/namespace" "go.temporal.io/server/common/persistence" ) @@ -33,7 +34,9 @@ type ( // Read APIs. ListWorkflowExecutions(ctx context.Context, request *ListWorkflowExecutionsRequestV2) (*ListWorkflowExecutionsResponse, error) + ListChasmExecutions(ctx context.Context, request *ListChasmExecutionsRequest) (*chasm.ListExecutionsResponse[*commonpb.Payload], error) CountWorkflowExecutions(ctx context.Context, request *CountWorkflowExecutionsRequest) (*CountWorkflowExecutionsResponse, error) + CountChasmExecutions(ctx context.Context, request *CountChasmExecutionsRequest) (*chasm.CountExecutionsResponse, error) GetWorkflowExecution(ctx context.Context, request *GetWorkflowExecutionRequest) (*GetWorkflowExecutionResponse, error) // Admin APIs @@ -110,6 +113,28 @@ type ( NextPageToken []byte } + ListChasmExecutionsRequest struct { + ArchetypeID chasm.ArchetypeID + NamespaceID namespace.ID + Namespace namespace.Name + PageSize int // Maximum number of workflow executions per page + Query string + // Token to continue reading next page of workflow executions. + // Pass in empty slice for first page. + NextPageToken []byte + } + + CountChasmExecutionsRequest struct { + ArchetypeID chasm.ArchetypeID + NamespaceID namespace.ID + Namespace namespace.Name + Query string + } + + CountChasmExecutionsResponse struct { + Count int64 + } + // CountWorkflowExecutionsRequest is request from CountWorkflowExecutions CountWorkflowExecutionsRequest struct { NamespaceID namespace.ID diff --git a/common/persistence/visibility/manager/visibility_manager_mock.go b/common/persistence/visibility/manager/visibility_manager_mock.go index cba8f826a6c..e2291de3dcf 100644 --- a/common/persistence/visibility/manager/visibility_manager_mock.go +++ b/common/persistence/visibility/manager/visibility_manager_mock.go @@ -13,6 +13,8 @@ import ( context "context" reflect "reflect" + common "go.temporal.io/api/common/v1" + chasm "go.temporal.io/server/chasm" namespace "go.temporal.io/server/common/namespace" gomock "go.uber.org/mock/gomock" ) @@ -67,6 +69,21 @@ func (mr *MockVisibilityManagerMockRecorder) Close() *gomock.Call { return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "Close", reflect.TypeOf((*MockVisibilityManager)(nil).Close)) } +// CountChasmExecutions mocks base method. +func (m *MockVisibilityManager) CountChasmExecutions(ctx context.Context, request *CountChasmExecutionsRequest) (*chasm.CountExecutionsResponse, error) { + m.ctrl.T.Helper() + ret := m.ctrl.Call(m, "CountChasmExecutions", ctx, request) + ret0, _ := ret[0].(*chasm.CountExecutionsResponse) + ret1, _ := ret[1].(error) + return ret0, ret1 +} + +// CountChasmExecutions indicates an expected call of CountChasmExecutions. +func (mr *MockVisibilityManagerMockRecorder) CountChasmExecutions(ctx, request any) *gomock.Call { + mr.mock.ctrl.T.Helper() + return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "CountChasmExecutions", reflect.TypeOf((*MockVisibilityManager)(nil).CountChasmExecutions), ctx, request) +} + // CountWorkflowExecutions mocks base method. func (m *MockVisibilityManager) CountWorkflowExecutions(ctx context.Context, request *CountWorkflowExecutionsRequest) (*CountWorkflowExecutionsResponse, error) { m.ctrl.T.Helper() @@ -167,6 +184,21 @@ func (mr *MockVisibilityManagerMockRecorder) HasStoreName(stName any) *gomock.Ca return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "HasStoreName", reflect.TypeOf((*MockVisibilityManager)(nil).HasStoreName), stName) } +// ListChasmExecutions mocks base method. +func (m *MockVisibilityManager) ListChasmExecutions(ctx context.Context, request *ListChasmExecutionsRequest) (*chasm.ListExecutionsResponse[*common.Payload], error) { + m.ctrl.T.Helper() + ret := m.ctrl.Call(m, "ListChasmExecutions", ctx, request) + ret0, _ := ret[0].(*chasm.ListExecutionsResponse[*common.Payload]) + ret1, _ := ret[1].(error) + return ret0, ret1 +} + +// ListChasmExecutions indicates an expected call of ListChasmExecutions. +func (mr *MockVisibilityManagerMockRecorder) ListChasmExecutions(ctx, request any) *gomock.Call { + mr.mock.ctrl.T.Helper() + return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "ListChasmExecutions", reflect.TypeOf((*MockVisibilityManager)(nil).ListChasmExecutions), ctx, request) +} + // ListWorkflowExecutions mocks base method. func (m *MockVisibilityManager) ListWorkflowExecutions(ctx context.Context, request *ListWorkflowExecutionsRequestV2) (*ListWorkflowExecutionsResponse, error) { m.ctrl.T.Helper() diff --git a/common/persistence/visibility/store/elasticsearch/converter.go b/common/persistence/visibility/store/elasticsearch/converter.go index f6c9282f4c2..fcf77a5e5db 100644 --- a/common/persistence/visibility/store/elasticsearch/converter.go +++ b/common/persistence/visibility/store/elasticsearch/converter.go @@ -2,6 +2,7 @@ package elasticsearch import ( "github.com/temporalio/sqlparser" + "go.temporal.io/server/chasm" "go.temporal.io/server/common/persistence/visibility/store/query" "go.temporal.io/server/common/searchattribute" ) @@ -23,6 +24,7 @@ func NewQueryConverterLegacy( fnInterceptor query.FieldNameInterceptor, fvInterceptor query.FieldValuesInterceptor, saNameType searchattribute.NameTypeMap, + chasmMapper *chasm.VisibilitySearchAttributesMapper, ) *query.ConverterLegacy { if fnInterceptor == nil { fnInterceptor = &query.NopFieldNameInterceptor{} @@ -33,7 +35,7 @@ func NewQueryConverterLegacy( } rangeCond := query.NewRangeCondConverter(fnInterceptor, fvInterceptor, true) - comparisonExpr := query.NewComparisonExprConverter(fnInterceptor, fvInterceptor, allowedComparisonOperators, saNameType) + comparisonExpr := query.NewComparisonExprConverter(fnInterceptor, fvInterceptor, allowedComparisonOperators, saNameType, chasmMapper) is := query.NewIsConverter(fnInterceptor) whereConverter := &query.WhereConverter{ diff --git a/common/persistence/visibility/store/elasticsearch/converter_test.go b/common/persistence/visibility/store/elasticsearch/converter_test.go index 917821948dd..33c2e7ce54d 100644 --- a/common/persistence/visibility/store/elasticsearch/converter_test.go +++ b/common/persistence/visibility/store/elasticsearch/converter_test.go @@ -126,7 +126,7 @@ var testNameTypeMap = searchattribute.NewNameTypeMapStub( ) func TestSupportedSelectWhere(t *testing.T) { - c := NewQueryConverterLegacy(nil, nil, testNameTypeMap) + c := NewQueryConverterLegacy(nil, nil, testNameTypeMap, nil) for sql, expectedJson := range supportedWhereCases { queryParams, err := c.ConvertWhereOrderBy(sql) @@ -140,7 +140,7 @@ func TestSupportedSelectWhere(t *testing.T) { } func TestEmptySelectWhere(t *testing.T) { - c := NewQueryConverterLegacy(nil, nil, testNameTypeMap) + c := NewQueryConverterLegacy(nil, nil, testNameTypeMap, nil) queryParams, err := c.ConvertWhereOrderBy("") assert.NoError(t, err) @@ -157,7 +157,7 @@ func TestEmptySelectWhere(t *testing.T) { } func TestSupportedSelectWhereOrder(t *testing.T) { - c := NewQueryConverterLegacy(nil, nil, testNameTypeMap) + c := NewQueryConverterLegacy(nil, nil, testNameTypeMap, nil) for sql, expectedJson := range supportedWhereOrderCases { queryParams, err := c.ConvertWhereOrderBy(sql) @@ -178,7 +178,7 @@ func TestSupportedSelectWhereOrder(t *testing.T) { } func TestSupportedSelectWhereGroupBy(t *testing.T) { - c := NewQueryConverterLegacy(nil, nil, testNameTypeMap) + c := NewQueryConverterLegacy(nil, nil, testNameTypeMap, nil) for sql, expectedJson := range supportedWhereGroupByCases { queryParams, err := c.ConvertWhereOrderBy(sql) @@ -196,7 +196,7 @@ func TestSupportedSelectWhereGroupBy(t *testing.T) { } func TestErrors(t *testing.T) { - c := NewQueryConverterLegacy(nil, nil, testNameTypeMap) + c := NewQueryConverterLegacy(nil, nil, testNameTypeMap, nil) for sql, expectedErrMessage := range errorCases { _, err := c.ConvertSql(sql) assert.Contains(t, err.Error(), expectedErrMessage, sql) diff --git a/common/persistence/visibility/store/elasticsearch/query_interceptors.go b/common/persistence/visibility/store/elasticsearch/query_interceptors.go index ff9dc6a1e59..13dfb72b435 100644 --- a/common/persistence/visibility/store/elasticsearch/query_interceptors.go +++ b/common/persistence/visibility/store/elasticsearch/query_interceptors.go @@ -2,10 +2,12 @@ package elasticsearch import ( "fmt" + "maps" "strconv" "time" enumspb "go.temporal.io/api/enums/v1" + "go.temporal.io/server/chasm" "go.temporal.io/server/common/namespace" "go.temporal.io/server/common/persistence/visibility/store/query" "go.temporal.io/server/common/primitives" @@ -19,11 +21,13 @@ type ( searchAttributesTypeMap searchattribute.NameTypeMap searchAttributesMapperProvider searchattribute.MapperProvider seenNamespaceDivision bool + chasmMapper *chasm.VisibilitySearchAttributesMapper } valuesInterceptor struct { namespace namespace.Name searchAttributesTypeMap searchattribute.NameTypeMap + chasmMapper *chasm.VisibilitySearchAttributesMapper } ) @@ -31,21 +35,26 @@ func NewNameInterceptor( namespaceName namespace.Name, saTypeMap searchattribute.NameTypeMap, searchAttributesMapperProvider searchattribute.MapperProvider, + chasmMapper *chasm.VisibilitySearchAttributesMapper, ) *nameInterceptor { return &nameInterceptor{ namespace: namespaceName, searchAttributesTypeMap: saTypeMap, searchAttributesMapperProvider: searchAttributesMapperProvider, + seenNamespaceDivision: false, + chasmMapper: chasmMapper, } } func NewValuesInterceptor( namespaceName namespace.Name, saTypeMap searchattribute.NameTypeMap, + chasmMapper *chasm.VisibilitySearchAttributesMapper, ) *valuesInterceptor { return &valuesInterceptor{ namespace: namespaceName, searchAttributesTypeMap: saTypeMap, + chasmMapper: chasmMapper, } } @@ -55,7 +64,8 @@ func (ni *nameInterceptor) Name(name string, usage query.FieldNameUsage) (string if err != nil { return "", err } - fieldName, fieldType, err := query.ResolveSearchAttributeAlias(name, ni.namespace, mapper, ni.searchAttributesTypeMap) + fieldName, fieldType, err := query.ResolveSearchAttributeAlias(name, ni.namespace, mapper, + ni.searchAttributesTypeMap, ni.chasmMapper) if err != nil { return "", err } @@ -86,7 +96,15 @@ func (ni *nameInterceptor) Name(name string, usage query.FieldNameUsage) (string } func (vi *valuesInterceptor) Values(name string, fieldName string, values ...interface{}) ([]interface{}, error) { - fieldType, err := vi.searchAttributesTypeMap.GetType(fieldName) + var fieldType enumspb.IndexedValueType + var err error + + combinedTypeMap := make(map[string]enumspb.IndexedValueType) + maps.Copy(combinedTypeMap, vi.searchAttributesTypeMap.Custom()) + maps.Copy(combinedTypeMap, vi.chasmMapper.SATypeMap()) + finalTypeMap := searchattribute.NewNameTypeMap(combinedTypeMap) + + fieldType, err = finalTypeMap.GetType(fieldName) if err != nil { return nil, query.NewConverterError("invalid search attribute: %s", name) } diff --git a/common/persistence/visibility/store/elasticsearch/query_interceptors_test.go b/common/persistence/visibility/store/elasticsearch/query_interceptors_test.go index 1b6fa6ba7ee..7fcab557cc3 100644 --- a/common/persistence/visibility/store/elasticsearch/query_interceptors_test.go +++ b/common/persistence/visibility/store/elasticsearch/query_interceptors_test.go @@ -37,6 +37,7 @@ func (s *QueryInterceptorSuite) TestTimeProcessFunc() { vi := NewValuesInterceptor( "test-namespace", searchattribute.TestEsNameTypeMap(), + nil, ) cases := []struct { @@ -74,6 +75,7 @@ func (s *QueryInterceptorSuite) TestStatusProcessFunc() { vi := NewValuesInterceptor( "test-namespace", searchattribute.TestEsNameTypeMap(), + nil, ) cases := []struct { @@ -117,6 +119,7 @@ func (s *QueryInterceptorSuite) TestDurationProcessFunc() { vi := NewValuesInterceptor( "test-namespace", searchattribute.TestEsNameTypeMap(), + nil, ) cases := []struct { @@ -173,6 +176,7 @@ func (s *QueryInterceptorSuite) TestValuesInterceptor_ScheduleIDToWorkflowID() { vi := NewValuesInterceptor( "test-namespace", searchattribute.TestEsNameTypeMap(), + nil, ) values, err := vi.Values(sadefs.ScheduleID, sadefs.WorkflowID, "test-schedule-id") @@ -195,6 +199,7 @@ func (s *QueryInterceptorSuite) TestValuesInterceptor_NoTransformation() { vi := NewValuesInterceptor( "test-namespace", searchattribute.TestEsNameTypeMapWithScheduleID(), + nil, ) values, err := vi.Values(sadefs.ScheduleID, sadefs.ScheduleID, "test-workflow-id") diff --git a/common/persistence/visibility/store/elasticsearch/visibility_store.go b/common/persistence/visibility/store/elasticsearch/visibility_store.go index c8646ddab53..a6c31eaecf6 100644 --- a/common/persistence/visibility/store/elasticsearch/visibility_store.go +++ b/common/persistence/visibility/store/elasticsearch/visibility_store.go @@ -7,6 +7,7 @@ import ( "encoding/json" "errors" "fmt" + "maps" "math" "strconv" "strings" @@ -18,6 +19,7 @@ import ( enumspb "go.temporal.io/api/enums/v1" "go.temporal.io/api/serviceerror" "go.temporal.io/api/workflowservice/v1" + "go.temporal.io/server/chasm" "go.temporal.io/server/common/dynamicconfig" "go.temporal.io/server/common/log" "go.temporal.io/server/common/metrics" @@ -44,6 +46,7 @@ type ( index string searchAttributesProvider searchattribute.Provider searchAttributesMapperProvider searchattribute.MapperProvider + chasmRegistry *chasm.Registry processor Processor processorAckTimeout dynamicconfig.DurationPropertyFn disableOrderByClause dynamicconfig.BoolPropertyFnWithNamespaceFilter @@ -67,6 +70,16 @@ type ( desc bool missing_first bool } + + searchParametersInternal struct { + NamespaceName namespace.Name + NamespaceID namespace.ID + Query string + PageSize int + NextPageToken []byte + ChasmMapper *chasm.VisibilitySearchAttributesMapper + ArchetypeID chasm.ArchetypeID + } ) var _ store.VisibilityStore = (*VisibilityStore)(nil) @@ -112,6 +125,7 @@ func NewVisibilityStore( processorConfig *ProcessorConfig, searchAttributesProvider searchattribute.Provider, searchAttributesMapperProvider searchattribute.MapperProvider, + chasmRegistry *chasm.Registry, disableOrderByClause dynamicconfig.BoolPropertyFnWithNamespaceFilter, enableManualPagination dynamicconfig.BoolPropertyFnWithNamespaceFilter, enableUnifiedQueryConverter dynamicconfig.BoolPropertyFn, @@ -145,6 +159,7 @@ func NewVisibilityStore( index: cfg.GetVisibilityIndex(), searchAttributesProvider: searchAttributesProvider, searchAttributesMapperProvider: searchAttributesMapperProvider, + chasmRegistry: chasmRegistry, processor: processor, processorAckTimeout: processorAckTimeout, disableOrderByClause: disableOrderByClause, @@ -367,7 +382,63 @@ func (s *VisibilityStore) ListWorkflowExecutions( return nil, ConvertElasticsearchClientError("ListWorkflowExecutions failed", err) } - return s.GetListWorkflowExecutionsResponse(searchResult, request.Namespace, request.PageSize) + return s.GetListWorkflowExecutionsResponse(searchResult, request.Namespace, request.PageSize, nil) +} + +func (s *VisibilityStore) ListChasmExecutions( + ctx context.Context, + request *manager.ListChasmExecutionsRequest, +) (*store.InternalListWorkflowExecutionsResponse, error) { + rc, ok := s.chasmRegistry.ComponentByID(request.ArchetypeID) + if !ok { + return nil, serviceerror.NewInvalidArgument(fmt.Sprintf("unknown archetype ID: %d", request.ArchetypeID)) + } + chasmMapper := rc.SearchAttributesMapper() + + p, err := s.BuildChasmSearchParameters(request, s.GetListFieldSorter, chasmMapper) + if err != nil { + return nil, err + } + + searchResult, err := s.esClient.Search(ctx, p) + if err != nil { + return nil, ConvertElasticsearchClientError("ListChasmExecutions failed", err) + } + + return s.GetListWorkflowExecutionsResponse(searchResult, request.Namespace, request.PageSize, chasmMapper) +} + +func (s *VisibilityStore) CountChasmExecutions( + ctx context.Context, + request *manager.CountChasmExecutionsRequest, +) (*manager.CountChasmExecutionsResponse, error) { + rc, ok := s.chasmRegistry.ComponentByID(request.ArchetypeID) + if !ok { + return nil, serviceerror.NewInvalidArgument(fmt.Sprintf("unknown archetype ID: %d", request.ArchetypeID)) + } + mapper := rc.SearchAttributesMapper() + + var queryParams *esQueryParams + var err error + if s.enableUnifiedQueryConverter() { + queryParams, err = s.convertQuery(request.Namespace, request.NamespaceID, request.Query, mapper, request.ArchetypeID) + if err != nil { + return nil, err + } + } else { + queryParamsLegacy, err := s.convertQueryLegacy(request.Namespace, request.NamespaceID, request.Query, mapper, request.ArchetypeID) + if err != nil { + return nil, err + } + queryParams = (*esQueryParams)(queryParamsLegacy) + } + + count, err := s.esClient.Count(ctx, s.index, queryParams.Query) + if err != nil { + return nil, ConvertElasticsearchClientError("CountChasmExecutions failed", err) + } + + return &manager.CountChasmExecutionsResponse{Count: count}, nil } func (s *VisibilityStore) CountWorkflowExecutions( @@ -377,12 +448,12 @@ func (s *VisibilityStore) CountWorkflowExecutions( var queryParams *esQueryParams var err error if s.enableUnifiedQueryConverter() { - queryParams, err = s.convertQuery(request.Namespace, request.NamespaceID, request.Query) + queryParams, err = s.convertQuery(request.Namespace, request.NamespaceID, request.Query, nil, chasm.UnspecifiedArchetypeID) if err != nil { return nil, err } } else { - queryParamsLegacy, err := s.convertQueryLegacy(request.Namespace, request.NamespaceID, request.Query) + queryParamsLegacy, err := s.convertQueryLegacy(request.Namespace, request.NamespaceID, request.Query, nil, chasm.UnspecifiedArchetypeID) if err != nil { return nil, err } @@ -468,7 +539,7 @@ func (s *VisibilityStore) GetWorkflowExecution( ) } - workflowExecutionInfo, err := s.ParseESDoc(result.Id, result.Source, typeMap, request.Namespace) + workflowExecutionInfo, err := s.ParseESDoc(result.Id, result.Source, typeMap, request.Namespace, nil) if err != nil { return nil, err } @@ -481,19 +552,51 @@ func (s *VisibilityStore) GetWorkflowExecution( func (s *VisibilityStore) BuildSearchParametersV2( request *manager.ListWorkflowExecutionsRequestV2, getFieldSorter func([]elastic.Sorter) ([]elastic.Sorter, error), +) (*client.SearchParameters, error) { + return s.buildSearchParametersInternal(&searchParametersInternal{ + NamespaceName: request.Namespace, + NamespaceID: request.NamespaceID, + Query: request.Query, + PageSize: request.PageSize, + NextPageToken: request.NextPageToken, + ChasmMapper: nil, + ArchetypeID: chasm.UnspecifiedArchetypeID, + }) +} + +func (s *VisibilityStore) BuildChasmSearchParameters( + request *manager.ListChasmExecutionsRequest, + getFieldSorter func([]elastic.Sorter) ([]elastic.Sorter, error), + chasmMapper *chasm.VisibilitySearchAttributesMapper, +) (*client.SearchParameters, error) { + return s.buildSearchParametersInternal(&searchParametersInternal{ + NamespaceName: request.Namespace, + NamespaceID: request.NamespaceID, + Query: request.Query, + PageSize: request.PageSize, + NextPageToken: request.NextPageToken, + ChasmMapper: chasmMapper, + ArchetypeID: request.ArchetypeID, + }) +} + +func (s *VisibilityStore) buildSearchParametersInternal( + params *searchParametersInternal, ) (*client.SearchParameters, error) { var queryParams *esQueryParams var err error if s.enableUnifiedQueryConverter() { - queryParams, err = s.convertQuery(request.Namespace, request.NamespaceID, request.Query) + queryParams, err = s.convertQuery(params.NamespaceName, params.NamespaceID, params.Query, params.ChasmMapper, params.ArchetypeID) if err != nil { return nil, err } } else { queryParamsLegacy, err := s.convertQueryLegacy( - request.Namespace, - request.NamespaceID, - request.Query, + params.NamespaceName, + params.NamespaceID, + params.Query, + params.ChasmMapper, + params.ArchetypeID, ) if err != nil { return nil, err @@ -503,7 +606,7 @@ func (s *VisibilityStore) BuildSearchParametersV2( searchParams := &client.SearchParameters{ Index: s.index, - PageSize: request.PageSize, + PageSize: params.PageSize, Query: queryParams.Query, } @@ -516,26 +619,26 @@ func (s *VisibilityStore) BuildSearchParametersV2( // using a field that was not indexed by ES. Since slow queries can block // writes for unreasonably long, this option forbids the usage of ORDER BY // clause to prevent slow down issues. - if s.disableOrderByClause(request.Namespace.String()) && len(queryParams.Sorter) > 0 { + if s.disableOrderByClause(params.NamespaceName.String()) && len(queryParams.Sorter) > 0 { return nil, serviceerror.NewInvalidArgument("ORDER BY clause is not supported") } if len(queryParams.Sorter) > 0 { // If params.Sorter is not empty, then it's using custom order by. - s.metricsHandler.WithTags(metrics.NamespaceTag(request.Namespace.String())). + s.metricsHandler.WithTags(metrics.NamespaceTag(params.NamespaceName.String())). Counter(metrics.ElasticsearchCustomOrderByClauseCount.Name()).Record(1) } - searchParams.Sorter, err = getFieldSorter(queryParams.Sorter) + searchParams.Sorter, err = s.GetListFieldSorter(queryParams.Sorter) if err != nil { return nil, err } - pageToken, err := s.deserializePageToken(request.NextPageToken) + pageToken, err := s.deserializePageToken(params.NextPageToken) if err != nil { return nil, err } - err = s.processPageToken(searchParams, pageToken, request.Namespace) + err = s.processPageToken(searchParams, pageToken, params.NamespaceName) if err != nil { return nil, err } @@ -597,6 +700,8 @@ func (s *VisibilityStore) convertQuery( namespaceName namespace.Name, namespaceID namespace.ID, queryString string, + chasmMapper *chasm.VisibilitySearchAttributesMapper, + archetypeID chasm.ArchetypeID, ) (res *esQueryParams, err error) { defer func() { // Convert ConverterError to InvalidArgument and pass through all other errors (which should be @@ -617,7 +722,10 @@ func (s *VisibilityStore) convertQuery( return nil, err } - c := query.NewQueryConverter(&queryConverter{}, namespaceName, saTypeMap, saMapper) + c := query.NewQueryConverter(&queryConverter{}, namespaceName, saTypeMap, saMapper). + WithChasmMapper(chasmMapper). + WithArchetypeID(archetypeID) + queryParams, err := c.Convert(queryString) if err != nil { return nil, err @@ -662,16 +770,19 @@ func (s *VisibilityStore) convertQueryLegacy( namespace namespace.Name, namespaceID namespace.ID, requestQueryStr string, + chasmMapper *chasm.VisibilitySearchAttributesMapper, + archetypeID chasm.ArchetypeID, ) (*query.QueryParamsLegacy, error) { saTypeMap, err := s.searchAttributesProvider.GetSearchAttributes(s.index, false) if err != nil { return nil, serviceerror.NewUnavailablef("unable to read search attribute types: %v", err) } - nameInterceptor := NewNameInterceptor(namespace, saTypeMap, s.searchAttributesMapperProvider) + nameInterceptor := NewNameInterceptor(namespace, saTypeMap, s.searchAttributesMapperProvider, chasmMapper) queryConverter := NewQueryConverterLegacy( nameInterceptor, - NewValuesInterceptor(namespace, saTypeMap), + NewValuesInterceptor(namespace, saTypeMap, chasmMapper), saTypeMap, + chasmMapper, ) queryParams, err := queryConverter.ConvertWhereOrderBy(requestQueryStr) if err != nil { @@ -689,7 +800,11 @@ func (s *VisibilityStore) convertQueryLegacy( // If the query did not explicitly filter on TemporalNamespaceDivision somehow, then add a // "must not exist" (i.e. "is null") query for it. if !nameInterceptor.seenNamespaceDivision { - namespaceFilterQuery.MustNot(elastic.NewExistsQuery(sadefs.TemporalNamespaceDivision)) + if archetypeID != chasm.UnspecifiedArchetypeID { + namespaceFilterQuery.Filter(elastic.NewTermQuery(sadefs.TemporalNamespaceDivision, strconv.Itoa(int(archetypeID)))) + } else { + namespaceFilterQuery.MustNot(elastic.NewExistsQuery(sadefs.TemporalNamespaceDivision)) + } } if queryParams.Query != nil { @@ -705,9 +820,7 @@ func (s *VisibilityStore) GetListFieldSorter(fieldSorts []elastic.Sorter) ([]ela return defaultSorter, nil } res := make([]elastic.Sorter, len(fieldSorts)+1) - for i, fs := range fieldSorts { - res[i] = fs - } + copy(res, fieldSorts) // RunID is explicit tiebreaker. res[len(res)-1] = elastic.NewFieldSort(sadefs.RunID).Desc() @@ -718,8 +831,8 @@ func (s *VisibilityStore) GetListWorkflowExecutionsResponse( searchResult *elastic.SearchResult, namespace namespace.Name, pageSize int, + chasmMapper *chasm.VisibilitySearchAttributesMapper, ) (*store.InternalListWorkflowExecutionsResponse, error) { - if searchResult.Hits == nil || len(searchResult.Hits.Hits) == 0 { return &store.InternalListWorkflowExecutionsResponse{}, nil } @@ -734,7 +847,7 @@ func (s *VisibilityStore) GetListWorkflowExecutionsResponse( } var lastHitSort []interface{} for _, hit := range searchResult.Hits.Hits { - workflowExecutionInfo, err := s.ParseESDoc(hit.Id, hit.Source, typeMap, namespace) + workflowExecutionInfo, err := s.ParseESDoc(hit.Id, hit.Source, typeMap, namespace, chasmMapper) if err != nil { return nil, err } @@ -867,6 +980,7 @@ func (s *VisibilityStore) ParseESDoc( docSource json.RawMessage, saTypeMap searchattribute.NameTypeMap, namespaceName namespace.Name, + chasmMapper *chasm.VisibilitySearchAttributesMapper, ) (*store.InternalWorkflowExecutionInfo, error) { logParseError := func(fieldName string, fieldValue interface{}, err error, docID string) error { metrics.ElasticsearchDocumentParseFailuresCount.With(s.metricsHandler).Record(1) @@ -882,11 +996,17 @@ func (s *VisibilityStore) ParseESDoc( return nil, serviceerror.NewInternalf("unable to unmarshal JSON from Elasticsearch document(%s): %v", docID, err) } + combinedTypeMap := make(map[string]enumspb.IndexedValueType) + maps.Copy(combinedTypeMap, saTypeMap.Custom()) + maps.Copy(combinedTypeMap, chasmMapper.SATypeMap()) + finalTypeMap := searchattribute.NewNameTypeMap(combinedTypeMap) + var ( isValidType bool memo []byte memoEncoding string customSearchAttributes map[string]interface{} + chasmSearchAttributes map[string]interface{} ) record := &store.InternalWorkflowExecutionInfo{} for fieldName, fieldValue := range sourceMap { @@ -912,7 +1032,7 @@ func (s *VisibilityStore) ParseESDoc( continue } - fieldType, err := saTypeMap.GetType(fieldName) + fieldType, err := finalTypeMap.GetType(fieldName) if err != nil { // Silently ignore ErrInvalidName because it indicates an unknown field in an Elasticsearch document. if errors.Is(err, searchattribute.ErrInvalidName) { @@ -965,37 +1085,43 @@ func (s *VisibilityStore) ParseESDoc( case sadefs.RootRunID: record.RootRunID = fieldValueParsed.(string) default: - // All custom and predefined search attributes are handled here. - if customSearchAttributes == nil { - customSearchAttributes = map[string]interface{}{} + if sadefs.IsChasmSearchAttribute(fieldName) { + if chasmSearchAttributes == nil { + chasmSearchAttributes = map[string]interface{}{} + } + chasmSearchAttributes[fieldName] = fieldValueParsed + } else { + if customSearchAttributes == nil { + customSearchAttributes = map[string]interface{}{} + } + customSearchAttributes[fieldName] = fieldValueParsed } - customSearchAttributes[fieldName] = fieldValueParsed } } - if customSearchAttributes != nil { - var err error - record.SearchAttributes, err = searchattribute.Encode(customSearchAttributes, &saTypeMap) - if err != nil { - metrics.ElasticsearchDocumentParseFailuresCount.With(s.metricsHandler).Record(1) - return nil, serviceerror.NewInternalf( - "Unable to encode custom search attributes of Elasticsearch document(%s): %v", - docID, - err, - ) - } - aliasedSas, err := searchattribute.AliasFields( - s.searchAttributesMapperProvider, - record.SearchAttributes, - namespaceName.String(), + var err error + record.ChasmSearchAttributes, err = chasm.AliasChasmSearchAttributes(chasmSearchAttributes, chasmMapper) + if err != nil { + return nil, err + } + + record.SearchAttributes, err = searchattribute.Encode(customSearchAttributes, &saTypeMap) + if err != nil { + metrics.ElasticsearchDocumentParseFailuresCount.With(s.metricsHandler).Record(1) + return nil, serviceerror.NewInternalf( + "Unable to encode custom search attributes of Elasticsearch document(%s): %v", + docID, + err, ) - if err != nil { - return nil, err - } + } - if aliasedSas != record.SearchAttributes { - record.SearchAttributes = aliasedSas - } + record.SearchAttributes, err = searchattribute.AliasFields( + s.searchAttributesMapperProvider, + record.SearchAttributes, + namespaceName.String(), + ) + if err != nil { + return nil, err } if memoEncoding != "" { @@ -1052,9 +1178,7 @@ func (s *VisibilityStore) parseCountGroupByResponse( return fmt.Errorf("unable to parse 'doc_count' field: %w", err) } groupValues := make([]*commonpb.Payload, len(groupByFields)) - for i := range bucketValues { - groupValues[i] = bucketValues[i] - } + copy(groupValues, bucketValues) response.Groups = append( response.Groups, &workflowservice.CountWorkflowExecutionsResponse_AggregationGroup{ diff --git a/common/persistence/visibility/store/elasticsearch/visibility_store_read_test.go b/common/persistence/visibility/store/elasticsearch/visibility_store_read_test.go index c6c648ae0c9..b3a15bac44f 100644 --- a/common/persistence/visibility/store/elasticsearch/visibility_store_read_test.go +++ b/common/persistence/visibility/store/elasticsearch/visibility_store_read_test.go @@ -18,8 +18,10 @@ import ( "go.temporal.io/api/serviceerror" "go.temporal.io/api/temporalproto" "go.temporal.io/api/workflowservice/v1" + "go.temporal.io/server/chasm" "go.temporal.io/server/common/debug" "go.temporal.io/server/common/dynamicconfig" + "go.temporal.io/server/common/log" "go.temporal.io/server/common/metrics" "go.temporal.io/server/common/namespace" "go.temporal.io/server/common/persistence/visibility/manager" @@ -43,6 +45,12 @@ type ( mockProcessor *MockProcessor mockMetricsHandler *metrics.MockHandler mockSearchAttributesMapperProvider *searchattribute.MockMapperProvider + chasmRegistry *chasm.Registry + } + + // Test component for CHASM visibility tests + testChasmComponent struct { + chasm.UnimplementedComponent } ) @@ -120,11 +128,32 @@ func (s *ESVisibilitySuite) SetupTest() { s.mockProcessor = NewMockProcessor(s.controller) s.mockESClient = client.NewMockClient(s.controller) s.mockSearchAttributesMapperProvider = searchattribute.NewMockMapperProvider(s.controller) + + // Setup CHASM registry for tests + library := chasm.NewMockLibrary(s.controller) + library.EXPECT().Name().Return("TestLibrary").AnyTimes() + rc := chasm.NewRegistrableComponent[*testChasmComponent]( + "TestComponent", + chasm.WithSearchAttributes( + chasm.NewSearchAttributeBool("ChasmCompleted", chasm.SearchAttributeFieldBool01), + chasm.NewSearchAttributeKeyword("ChasmStatus", chasm.SearchAttributeFieldKeyword01), + chasm.NewSearchAttributeInt("ChasmCount", chasm.SearchAttributeFieldInt01), + chasm.NewSearchAttributeDouble("ChasmScore", chasm.SearchAttributeFieldDouble01), + chasm.NewSearchAttributeDateTime("ChasmStartTime", chasm.SearchAttributeFieldDateTime01), + ), + ) + library.EXPECT().Components().Return([]*chasm.RegistrableComponent{rc}).AnyTimes() + library.EXPECT().Tasks().Return(nil).AnyTimes() + s.chasmRegistry = chasm.NewRegistry(log.NewNoopLogger()) + err := s.chasmRegistry.Register(library) + s.NoError(err) + s.visibilityStore = &VisibilityStore{ esClient: s.mockESClient, index: testIndex, searchAttributesProvider: searchattribute.NewTestEsProvider(), searchAttributesMapperProvider: s.mockSearchAttributesMapperProvider, + chasmRegistry: s.chasmRegistry, processor: s.mockProcessor, processorAckTimeout: esProcessorAckTimeout, disableOrderByClause: visibilityDisableOrderByClause, @@ -141,6 +170,10 @@ func (s *ESVisibilitySuite) TearDownTest() { s.controller.Finish() } +func (tc *testChasmComponent) LifecycleState(_ chasm.Context) chasm.LifecycleState { + return chasm.LifecycleStateRunning +} + func (s *ESVisibilitySuite) TestGetListFieldSorter() { // test defaultSorter is returned when fieldSorts is empty @@ -299,97 +332,97 @@ func (s *ESVisibilitySuite) Test_convertQueryLegacy() { s.visibilityStore.searchAttributesMapperProvider = searchattribute.NewTestMapperProvider(nil) query := `WorkflowId = 'wid'` - queryParams, err := s.visibilityStore.convertQueryLegacy(testNamespace, testNamespaceID, query) + queryParams, err := s.visibilityStore.convertQueryLegacy(testNamespace, testNamespaceID, query, nil, chasm.UnspecifiedArchetypeID) s.NoError(err) s.Equal(`{"bool":{"filter":[{"term":{"NamespaceId":"bfd5c907-f899-4baf-a7b2-2ab85e623ebd"}},{"bool":{"filter":{"term":{"WorkflowId":"wid"}}}}],"must_not":{"exists":{"field":"TemporalNamespaceDivision"}}}}`, s.queryToJSON(queryParams.Query)) s.Nil(queryParams.Sorter) query = `WorkflowId = 'wid' or WorkflowId = 'another-wid'` - queryParams, err = s.visibilityStore.convertQueryLegacy(testNamespace, testNamespaceID, query) + queryParams, err = s.visibilityStore.convertQueryLegacy(testNamespace, testNamespaceID, query, nil, chasm.UnspecifiedArchetypeID) s.NoError(err) s.Equal(`{"bool":{"filter":[{"term":{"NamespaceId":"bfd5c907-f899-4baf-a7b2-2ab85e623ebd"}},{"bool":{"should":[{"term":{"WorkflowId":"wid"}},{"term":{"WorkflowId":"another-wid"}}]}}],"must_not":{"exists":{"field":"TemporalNamespaceDivision"}}}}`, s.queryToJSON(queryParams.Query)) s.Nil(queryParams.Sorter) query = `WorkflowId = 'wid' order by StartTime desc` - queryParams, err = s.visibilityStore.convertQueryLegacy(testNamespace, testNamespaceID, query) + queryParams, err = s.visibilityStore.convertQueryLegacy(testNamespace, testNamespaceID, query, nil, chasm.UnspecifiedArchetypeID) s.NoError(err) s.Equal(`{"bool":{"filter":[{"term":{"NamespaceId":"bfd5c907-f899-4baf-a7b2-2ab85e623ebd"}},{"bool":{"filter":{"term":{"WorkflowId":"wid"}}}}],"must_not":{"exists":{"field":"TemporalNamespaceDivision"}}}}`, s.queryToJSON(queryParams.Query)) s.Equal(`[{"StartTime":{"order":"desc"}}]`, s.sorterToJSON(queryParams.Sorter)) query = `WorkflowId = 'wid' and CloseTime is null` - queryParams, err = s.visibilityStore.convertQueryLegacy(testNamespace, testNamespaceID, query) + queryParams, err = s.visibilityStore.convertQueryLegacy(testNamespace, testNamespaceID, query, nil, chasm.UnspecifiedArchetypeID) s.NoError(err) s.Equal(`{"bool":{"filter":[{"term":{"NamespaceId":"bfd5c907-f899-4baf-a7b2-2ab85e623ebd"}},{"bool":{"filter":[{"term":{"WorkflowId":"wid"}},{"bool":{"must_not":{"exists":{"field":"CloseTime"}}}}]}}],"must_not":{"exists":{"field":"TemporalNamespaceDivision"}}}}`, s.queryToJSON(queryParams.Query)) s.Nil(queryParams.Sorter) query = `WorkflowId = 'wid' or CloseTime is null` - queryParams, err = s.visibilityStore.convertQueryLegacy(testNamespace, testNamespaceID, query) + queryParams, err = s.visibilityStore.convertQueryLegacy(testNamespace, testNamespaceID, query, nil, chasm.UnspecifiedArchetypeID) s.NoError(err) s.Equal(`{"bool":{"filter":[{"term":{"NamespaceId":"bfd5c907-f899-4baf-a7b2-2ab85e623ebd"}},{"bool":{"should":[{"term":{"WorkflowId":"wid"}},{"bool":{"must_not":{"exists":{"field":"CloseTime"}}}}]}}],"must_not":{"exists":{"field":"TemporalNamespaceDivision"}}}}`, s.queryToJSON(queryParams.Query)) s.Nil(queryParams.Sorter) query = `CloseTime is null order by CloseTime desc` - queryParams, err = s.visibilityStore.convertQueryLegacy(testNamespace, testNamespaceID, query) + queryParams, err = s.visibilityStore.convertQueryLegacy(testNamespace, testNamespaceID, query, nil, chasm.UnspecifiedArchetypeID) s.NoError(err) s.Equal(`{"bool":{"filter":[{"term":{"NamespaceId":"bfd5c907-f899-4baf-a7b2-2ab85e623ebd"}},{"bool":{"must_not":{"exists":{"field":"CloseTime"}}}}],"must_not":{"exists":{"field":"TemporalNamespaceDivision"}}}}`, s.queryToJSON(queryParams.Query)) s.Equal(`[{"CloseTime":{"order":"desc"}}]`, s.sorterToJSON(queryParams.Sorter)) query = `StartTime = "2018-06-07T15:04:05.123456789-08:00"` - queryParams, err = s.visibilityStore.convertQueryLegacy(testNamespace, testNamespaceID, query) + queryParams, err = s.visibilityStore.convertQueryLegacy(testNamespace, testNamespaceID, query, nil, chasm.UnspecifiedArchetypeID) s.NoError(err) s.JSONEq(`{"bool":{"filter":[{"term":{"NamespaceId":"bfd5c907-f899-4baf-a7b2-2ab85e623ebd"}},{"bool":{"filter":{"match":{"StartTime":{"query":"2018-06-07T15:04:05.123456789-08:00"}}}}}],"must_not":{"exists":{"field":"TemporalNamespaceDivision"}}}}`, s.queryToJSON(queryParams.Query)) s.Nil(queryParams.Sorter) query = `WorkflowId = 'wid' and StartTime > "2018-06-07T15:04:05+00:00"` - queryParams, err = s.visibilityStore.convertQueryLegacy(testNamespace, testNamespaceID, query) + queryParams, err = s.visibilityStore.convertQueryLegacy(testNamespace, testNamespaceID, query, nil, chasm.UnspecifiedArchetypeID) s.NoError(err) s.Equal(`{"bool":{"filter":[{"term":{"NamespaceId":"bfd5c907-f899-4baf-a7b2-2ab85e623ebd"}},{"bool":{"filter":[{"term":{"WorkflowId":"wid"}},{"range":{"StartTime":{"from":"2018-06-07T15:04:05+00:00","include_lower":false,"include_upper":true,"to":null}}}]}}],"must_not":{"exists":{"field":"TemporalNamespaceDivision"}}}}`, s.queryToJSON(queryParams.Query)) s.Nil(queryParams.Sorter) query = `ExecutionTime < 1000000` - queryParams, err = s.visibilityStore.convertQueryLegacy(testNamespace, testNamespaceID, query) + queryParams, err = s.visibilityStore.convertQueryLegacy(testNamespace, testNamespaceID, query, nil, chasm.UnspecifiedArchetypeID) s.NoError(err) s.Equal(`{"bool":{"filter":[{"term":{"NamespaceId":"bfd5c907-f899-4baf-a7b2-2ab85e623ebd"}},{"bool":{"filter":{"range":{"ExecutionTime":{"from":null,"include_lower":true,"include_upper":false,"to":"1970-01-01T00:00:00.001Z"}}}}}],"must_not":{"exists":{"field":"TemporalNamespaceDivision"}}}}`, s.queryToJSON(queryParams.Query)) s.Nil(queryParams.Sorter) query = `ExecutionTime between 1 and 2` - queryParams, err = s.visibilityStore.convertQueryLegacy(testNamespace, testNamespaceID, query) + queryParams, err = s.visibilityStore.convertQueryLegacy(testNamespace, testNamespaceID, query, nil, chasm.UnspecifiedArchetypeID) s.NoError(err) s.Equal(`{"bool":{"filter":[{"term":{"NamespaceId":"bfd5c907-f899-4baf-a7b2-2ab85e623ebd"}},{"bool":{"filter":{"range":{"ExecutionTime":{"from":"1970-01-01T00:00:00.000000001Z","include_lower":true,"include_upper":true,"to":"1970-01-01T00:00:00.000000002Z"}}}}}],"must_not":{"exists":{"field":"TemporalNamespaceDivision"}}}}`, s.queryToJSON(queryParams.Query)) s.Nil(queryParams.Sorter) query = `ExecutionTime < 1000000 or ExecutionTime > 2000000` - queryParams, err = s.visibilityStore.convertQueryLegacy(testNamespace, testNamespaceID, query) + queryParams, err = s.visibilityStore.convertQueryLegacy(testNamespace, testNamespaceID, query, nil, chasm.UnspecifiedArchetypeID) s.NoError(err) s.Equal(`{"bool":{"filter":[{"term":{"NamespaceId":"bfd5c907-f899-4baf-a7b2-2ab85e623ebd"}},{"bool":{"should":[{"range":{"ExecutionTime":{"from":null,"include_lower":true,"include_upper":false,"to":"1970-01-01T00:00:00.001Z"}}},{"range":{"ExecutionTime":{"from":"1970-01-01T00:00:00.002Z","include_lower":false,"include_upper":true,"to":null}}}]}}],"must_not":{"exists":{"field":"TemporalNamespaceDivision"}}}}`, s.queryToJSON(queryParams.Query)) s.Nil(queryParams.Sorter) query = `order by ExecutionTime` - queryParams, err = s.visibilityStore.convertQueryLegacy(testNamespace, testNamespaceID, query) + queryParams, err = s.visibilityStore.convertQueryLegacy(testNamespace, testNamespaceID, query, nil, chasm.UnspecifiedArchetypeID) s.NoError(err) s.Equal(`{"bool":{"filter":{"term":{"NamespaceId":"bfd5c907-f899-4baf-a7b2-2ab85e623ebd"}},"must_not":{"exists":{"field":"TemporalNamespaceDivision"}}}}`, s.queryToJSON(queryParams.Query)) s.Equal(`[{"ExecutionTime":{"order":"asc"}}]`, s.sorterToJSON(queryParams.Sorter)) query = `order by StartTime desc, CloseTime asc` - queryParams, err = s.visibilityStore.convertQueryLegacy(testNamespace, testNamespaceID, query) + queryParams, err = s.visibilityStore.convertQueryLegacy(testNamespace, testNamespaceID, query, nil, chasm.UnspecifiedArchetypeID) s.NoError(err) s.Equal(`{"bool":{"filter":{"term":{"NamespaceId":"bfd5c907-f899-4baf-a7b2-2ab85e623ebd"}},"must_not":{"exists":{"field":"TemporalNamespaceDivision"}}}}`, s.queryToJSON(queryParams.Query)) s.Equal(`[{"StartTime":{"order":"desc"}},{"CloseTime":{"order":"asc"}}]`, s.sorterToJSON(queryParams.Sorter)) query = `order by CustomTextField desc` - _, err = s.visibilityStore.convertQueryLegacy(testNamespace, testNamespaceID, query) + _, err = s.visibilityStore.convertQueryLegacy(testNamespace, testNamespaceID, query, nil, chasm.UnspecifiedArchetypeID) s.Error(err) s.IsType(&serviceerror.InvalidArgument{}, err) s.Equal(err.(*serviceerror.InvalidArgument).Error(), "invalid query: unable to convert 'order by' column name: unable to sort by field of Text type, use field of type Keyword") query = `order by CustomIntField asc` - queryParams, err = s.visibilityStore.convertQueryLegacy(testNamespace, testNamespaceID, query) + queryParams, err = s.visibilityStore.convertQueryLegacy(testNamespace, testNamespaceID, query, nil, chasm.UnspecifiedArchetypeID) s.NoError(err) s.Equal(`{"bool":{"filter":{"term":{"NamespaceId":"bfd5c907-f899-4baf-a7b2-2ab85e623ebd"}},"must_not":{"exists":{"field":"TemporalNamespaceDivision"}}}}`, s.queryToJSON(queryParams.Query)) s.Equal(`[{"CustomIntField":{"order":"asc"}}]`, s.sorterToJSON(queryParams.Sorter)) query = `ExecutionTime < "unable to parse"` - queryParams, err = s.visibilityStore.convertQueryLegacy(testNamespace, testNamespaceID, query) + queryParams, err = s.visibilityStore.convertQueryLegacy(testNamespace, testNamespaceID, query, nil, chasm.UnspecifiedArchetypeID) s.Error(err) s.IsType(&serviceerror.InvalidArgument{}, err) s.Equal(err.Error(), "invalid query: unable to convert filter expression: unable to convert values of comparison expression: invalid value for search attribute ExecutionTime of type Datetime: \"unable to parse\"") @@ -397,63 +430,63 @@ func (s *ESVisibilitySuite) Test_convertQueryLegacy() { // invalid union injection query = `WorkflowId = 'wid' union select * from dummy` - queryParams, err = s.visibilityStore.convertQueryLegacy(testNamespace, testNamespaceID, query) + queryParams, err = s.visibilityStore.convertQueryLegacy(testNamespace, testNamespaceID, query, nil, chasm.UnspecifiedArchetypeID) s.Error(err) s.Nil(queryParams) } func (s *ESVisibilitySuite) Test_convertQueryLegacy_Mapper() { query := `WorkflowId = 'wid'` - queryParams, err := s.visibilityStore.convertQueryLegacy(testNamespace, testNamespaceID, query) + queryParams, err := s.visibilityStore.convertQueryLegacy(testNamespace, testNamespaceID, query, nil, chasm.UnspecifiedArchetypeID) s.NoError(err) s.Equal(`{"bool":{"filter":[{"term":{"NamespaceId":"bfd5c907-f899-4baf-a7b2-2ab85e623ebd"}},{"bool":{"filter":{"term":{"WorkflowId":"wid"}}}}],"must_not":{"exists":{"field":"TemporalNamespaceDivision"}}}}`, s.queryToJSON(queryParams.Query)) s.Nil(queryParams.Sorter) query = "`AliasForCustomKeywordField` = 'pid'" - queryParams, err = s.visibilityStore.convertQueryLegacy(testNamespace, testNamespaceID, query) + queryParams, err = s.visibilityStore.convertQueryLegacy(testNamespace, testNamespaceID, query, nil, chasm.UnspecifiedArchetypeID) s.NoError(err) s.Equal(`{"bool":{"filter":[{"term":{"NamespaceId":"bfd5c907-f899-4baf-a7b2-2ab85e623ebd"}},{"bool":{"filter":{"term":{"CustomKeywordField":"pid"}}}}],"must_not":{"exists":{"field":"TemporalNamespaceDivision"}}}}`, s.queryToJSON(queryParams.Query)) s.Nil(queryParams.Sorter) query = "`AliasWithHyphenFor-CustomKeywordField` = 'pid'" - queryParams, err = s.visibilityStore.convertQueryLegacy(testNamespace, testNamespaceID, query) + queryParams, err = s.visibilityStore.convertQueryLegacy(testNamespace, testNamespaceID, query, nil, chasm.UnspecifiedArchetypeID) s.NoError(err) s.Equal(`{"bool":{"filter":[{"term":{"NamespaceId":"bfd5c907-f899-4baf-a7b2-2ab85e623ebd"}},{"bool":{"filter":{"term":{"CustomKeywordField":"pid"}}}}],"must_not":{"exists":{"field":"TemporalNamespaceDivision"}}}}`, s.queryToJSON(queryParams.Query)) s.Nil(queryParams.Sorter) query = `CustomKeywordField = 'pid'` - queryParams, err = s.visibilityStore.convertQueryLegacy(testNamespace, testNamespaceID, query) + queryParams, err = s.visibilityStore.convertQueryLegacy(testNamespace, testNamespaceID, query, nil, chasm.UnspecifiedArchetypeID) s.NoError(err) s.JSONEq(`{"bool":{"filter":[{"term":{"NamespaceId":"bfd5c907-f899-4baf-a7b2-2ab85e623ebd"}},{"bool":{"filter":{"term":{"CustomKeywordField":"pid"}}}}],"must_not":{"exists":{"field":"TemporalNamespaceDivision"}}}}`, s.queryToJSON(queryParams.Query)) s.Nil(queryParams.Sorter) query = `AliasForUnknownField = 'pid'` - _, err = s.visibilityStore.convertQueryLegacy(testNamespace, testNamespaceID, query) + _, err = s.visibilityStore.convertQueryLegacy(testNamespace, testNamespaceID, query, nil, chasm.UnspecifiedArchetypeID) s.Error(err) var invalidArgumentErr *serviceerror.InvalidArgument s.ErrorAs(err, &invalidArgumentErr) s.EqualError(err, "invalid query: unable to convert filter expression: unable to convert left side of \"AliasForUnknownField = 'pid'\": invalid search attribute: AliasForUnknownField") query = `order by ExecutionTime` - queryParams, err = s.visibilityStore.convertQueryLegacy(testNamespace, testNamespaceID, query) + queryParams, err = s.visibilityStore.convertQueryLegacy(testNamespace, testNamespaceID, query, nil, chasm.UnspecifiedArchetypeID) s.NoError(err) s.Equal(`{"bool":{"filter":{"term":{"NamespaceId":"bfd5c907-f899-4baf-a7b2-2ab85e623ebd"}},"must_not":{"exists":{"field":"TemporalNamespaceDivision"}}}}`, s.queryToJSON(queryParams.Query)) s.Equal(`[{"ExecutionTime":{"order":"asc"}}]`, s.sorterToJSON(queryParams.Sorter)) query = `order by AliasForCustomKeywordField asc` - queryParams, err = s.visibilityStore.convertQueryLegacy(testNamespace, testNamespaceID, query) + queryParams, err = s.visibilityStore.convertQueryLegacy(testNamespace, testNamespaceID, query, nil, chasm.UnspecifiedArchetypeID) s.NoError(err) s.Equal(`{"bool":{"filter":{"term":{"NamespaceId":"bfd5c907-f899-4baf-a7b2-2ab85e623ebd"}},"must_not":{"exists":{"field":"TemporalNamespaceDivision"}}}}`, s.queryToJSON(queryParams.Query)) s.Equal(`[{"CustomKeywordField":{"order":"asc"}}]`, s.sorterToJSON(queryParams.Sorter)) query = `order by CustomKeywordField asc` - queryParams, err = s.visibilityStore.convertQueryLegacy(testNamespace, testNamespaceID, query) + queryParams, err = s.visibilityStore.convertQueryLegacy(testNamespace, testNamespaceID, query, nil, chasm.UnspecifiedArchetypeID) s.NoError(err) s.JSONEq(`{"bool":{"filter":{"term":{"NamespaceId":"bfd5c907-f899-4baf-a7b2-2ab85e623ebd"}},"must_not":{"exists":{"field":"TemporalNamespaceDivision"}}}}`, s.queryToJSON(queryParams.Query)) s.NotNil(queryParams.Sorter) query = `order by AliasForUnknownField asc` - _, err = s.visibilityStore.convertQueryLegacy(testNamespace, testNamespaceID, query) + _, err = s.visibilityStore.convertQueryLegacy(testNamespace, testNamespaceID, query, nil, chasm.UnspecifiedArchetypeID) s.Error(err) s.ErrorAs(err, &invalidArgumentErr) s.EqualError(err, "invalid query: unable to convert 'order by' column name: invalid search attribute: AliasForUnknownField") @@ -462,20 +495,20 @@ func (s *ESVisibilitySuite) Test_convertQueryLegacy_Mapper() { func (s *ESVisibilitySuite) Test_convertQueryLegacy_Mapper_Error() { query := `WorkflowId = 'wid'` - queryParams, err := s.visibilityStore.convertQueryLegacy(testNamespace, testNamespaceID, query) + queryParams, err := s.visibilityStore.convertQueryLegacy(testNamespace, testNamespaceID, query, nil, chasm.UnspecifiedArchetypeID) s.NoError(err) s.Equal(`{"bool":{"filter":[{"term":{"NamespaceId":"bfd5c907-f899-4baf-a7b2-2ab85e623ebd"}},{"bool":{"filter":{"term":{"WorkflowId":"wid"}}}}],"must_not":{"exists":{"field":"TemporalNamespaceDivision"}}}}`, s.queryToJSON(queryParams.Query)) s.Nil(queryParams.Sorter) query = `ProductId = 'pid'` - _, err = s.visibilityStore.convertQueryLegacy(testNamespace, testNamespaceID, query) + _, err = s.visibilityStore.convertQueryLegacy(testNamespace, testNamespaceID, query, nil, chasm.UnspecifiedArchetypeID) s.Error(err) var invalidArgumentErr *serviceerror.InvalidArgument s.ErrorAs(err, &invalidArgumentErr) s.EqualError(err, "invalid query: unable to convert filter expression: unable to convert left side of \"ProductId = 'pid'\": invalid search attribute: ProductId") query = `order by ExecutionTime` - queryParams, err = s.visibilityStore.convertQueryLegacy(testNamespace, testNamespaceID, query) + queryParams, err = s.visibilityStore.convertQueryLegacy(testNamespace, testNamespaceID, query, nil, chasm.UnspecifiedArchetypeID) s.NoError(err) s.Equal(`{"bool":{"filter":{"term":{"NamespaceId":"bfd5c907-f899-4baf-a7b2-2ab85e623ebd"}},"must_not":{"exists":{"field":"TemporalNamespaceDivision"}}}}`, s.queryToJSON(queryParams.Query)) s.Equal(`[{"ExecutionTime":{"order":"asc"}}]`, s.sorterToJSON(queryParams.Sorter)) @@ -586,7 +619,7 @@ func (s *ESVisibilitySuite) Test_convertQuery() { for _, tc := range testCases { s.Run(tc.name, func() { - got, err := s.visibilityStore.convertQuery(testNamespace, testNamespaceID, tc.query) + got, err := s.visibilityStore.convertQuery(testNamespace, testNamespaceID, tc.query, nil, 0) if tc.err != "" { s.Error(err) s.ErrorContains(err, tc.err) @@ -606,7 +639,7 @@ func (s *ESVisibilitySuite) TestGetListWorkflowExecutionsResponse() { Hits: &elastic.SearchHits{ TotalHits: &elastic.TotalHits{}, }} - resp, err := s.visibilityStore.GetListWorkflowExecutionsResponse(searchResult, testNamespace, 1) + resp, err := s.visibilityStore.GetListWorkflowExecutionsResponse(searchResult, testNamespace, 1, nil) s.NoError(err) s.Equal(0, len(resp.NextPageToken)) s.Equal(0, len(resp.Executions)) @@ -629,14 +662,14 @@ func (s *ESVisibilitySuite) TestGetListWorkflowExecutionsResponse() { } searchResult.Hits.Hits = []*elastic.SearchHit{searchHit} searchResult.Hits.TotalHits.Value = 1 - resp, err = s.visibilityStore.GetListWorkflowExecutionsResponse(searchResult, testNamespace, 1) + resp, err = s.visibilityStore.GetListWorkflowExecutionsResponse(searchResult, testNamespace, 1, nil) s.NoError(err) serializedToken, _ := s.visibilityStore.serializePageToken(&visibilityPageToken{SearchAfter: []interface{}{1547596872371234567, "e481009e-14b3-45ae-91af-dce6e2a88365"}}) s.Equal(serializedToken, resp.NextPageToken) s.Equal(1, len(resp.Executions)) // test for last page hits - resp, err = s.visibilityStore.GetListWorkflowExecutionsResponse(searchResult, testNamespace, 2) + resp, err = s.visibilityStore.GetListWorkflowExecutionsResponse(searchResult, testNamespace, 2, nil) s.NoError(err) s.Equal(0, len(resp.NextPageToken)) s.Equal(1, len(resp.Executions)) @@ -647,7 +680,7 @@ func (s *ESVisibilitySuite) TestGetListWorkflowExecutionsResponse() { searchResult.Hits.Hits = append(searchResult.Hits.Hits, searchHit) } numOfHits := len(searchResult.Hits.Hits) - resp, err = s.visibilityStore.GetListWorkflowExecutionsResponse(searchResult, testNamespace, numOfHits) + resp, err = s.visibilityStore.GetListWorkflowExecutionsResponse(searchResult, testNamespace, numOfHits, nil) s.NoError(err) s.Equal(numOfHits, len(resp.Executions)) nextPageToken, err := s.visibilityStore.deserializePageToken(resp.NextPageToken) @@ -657,7 +690,7 @@ func (s *ESVisibilitySuite) TestGetListWorkflowExecutionsResponse() { s.Equal(int64(1547596872371234567), resultSortValue) s.Equal("e481009e-14b3-45ae-91af-dce6e2a88365", nextPageToken.SearchAfter[1]) // for last page - resp, err = s.visibilityStore.GetListWorkflowExecutionsResponse(searchResult, testNamespace, numOfHits+1) + resp, err = s.visibilityStore.GetListWorkflowExecutionsResponse(searchResult, testNamespace, numOfHits+1, nil) s.NoError(err) s.Equal(0, len(resp.NextPageToken)) s.Equal(numOfHits, len(resp.Executions)) @@ -720,7 +753,7 @@ func (s *ESVisibilitySuite) TestParseESDoc() { "WorkflowId": "6bfbc1e5-6ce4-4e22-bbfb-e0faa9a7a604-1-2256", "WorkflowType": "TestWorkflowExecute"}`) // test for open - info, err := s.visibilityStore.ParseESDoc("", docSource, saTypeMap, testNamespace) + info, err := s.visibilityStore.ParseESDoc("", docSource, saTypeMap, testNamespace, nil) s.NoError(err) s.NotNil(info) s.Equal("6bfbc1e5-6ce4-4e22-bbfb-e0faa9a7a604-1-2256", info.WorkflowID) @@ -744,7 +777,7 @@ func (s *ESVisibilitySuite) TestParseESDoc() { "StartTime": "2021-06-11T15:04:07.980-07:00", "WorkflowId": "6bfbc1e5-6ce4-4e22-bbfb-e0faa9a7a604-1-2256", "WorkflowType": "TestWorkflowExecute"}`) - info, err = s.visibilityStore.ParseESDoc("", docSource, saTypeMap, testNamespace) + info, err = s.visibilityStore.ParseESDoc("", docSource, saTypeMap, testNamespace, nil) s.NoError(err) s.NotNil(info) s.Equal("6bfbc1e5-6ce4-4e22-bbfb-e0faa9a7a604-1-2256", info.WorkflowID) @@ -764,7 +797,7 @@ func (s *ESVisibilitySuite) TestParseESDoc() { // test for error case docSource = []byte(`corrupted data`) s.mockMetricsHandler.EXPECT().Counter(metrics.ElasticsearchDocumentParseFailuresCount.Name()).Return(metrics.NoopCounterMetricFunc) - info, err = s.visibilityStore.ParseESDoc("", docSource, saTypeMap, testNamespace) + info, err = s.visibilityStore.ParseESDoc("", docSource, saTypeMap, testNamespace, nil) s.Error(err) s.Nil(info) } @@ -782,7 +815,7 @@ func (s *ESVisibilitySuite) TestParseESDoc_SearchAttributes() { "CustomIntField": [111,222], "CustomBoolField": true, "UnknownField": "random"}`) - info, err := s.visibilityStore.ParseESDoc("", docSource, saTypeMap, testNamespace) + info, err := s.visibilityStore.ParseESDoc("", docSource, saTypeMap, testNamespace, nil) s.NoError(err) s.NotNil(info) customSearchAttributes, err := searchattribute.Decode(info.SearchAttributes, &saTypeMap, true) @@ -826,7 +859,7 @@ func (s *ESVisibilitySuite) TestParseESDoc_SearchAttributes_WithMapper() { "CustomBoolField": true, "UnknownField": "random"}`) - info, err := s.visibilityStore.ParseESDoc("", docSource, saTypeMap, testNamespace) + info, err := s.visibilityStore.ParseESDoc("", docSource, saTypeMap, testNamespace, nil) s.NoError(err) s.NotNil(info) @@ -1849,3 +1882,445 @@ func (s *ESVisibilitySuite) Test_parsePageTokenValue() { }) } } + +func (s *ESVisibilitySuite) Test_convertQueryLegacy_ChasmMapper() { + chasmMapper := chasm.NewTestVisibilitySearchAttributesMapper( + map[string]string{ + "TemporalBool01": "ChasmCompleted", + "TemporalKeyword01": "ChasmStatus", + "TemporalInt01": "ChasmCount", + "TemporalDouble01": "ChasmScore", + "TemporalDatetime01": "ChasmStartTime", + }, + map[string]enumspb.IndexedValueType{ + "TemporalBool01": enumspb.INDEXED_VALUE_TYPE_BOOL, + "TemporalKeyword01": enumspb.INDEXED_VALUE_TYPE_KEYWORD, + "TemporalInt01": enumspb.INDEXED_VALUE_TYPE_INT, + "TemporalDouble01": enumspb.INDEXED_VALUE_TYPE_DOUBLE, + "TemporalDatetime01": enumspb.INDEXED_VALUE_TYPE_DATETIME, + }, + ) + + queryStr := `ChasmCompleted = true` + queryParams, err := s.visibilityStore.convertQueryLegacy(testNamespace, testNamespaceID, queryStr, chasmMapper, chasm.UnspecifiedArchetypeID) + s.NoError(err) + // Legacy converter uses "match" for non-KEYWORD types, "term" for KEYWORD types + s.JSONEq(`{"bool":{"filter":[{"term":{"NamespaceId":"bfd5c907-f899-4baf-a7b2-2ab85e623ebd"}},{"bool":{"filter":{"match":{"TemporalBool01":{"query":true}}}}}],"must_not":{"exists":{"field":"TemporalNamespaceDivision"}}}}`, s.queryToJSON(queryParams.Query)) + s.Nil(queryParams.Sorter) + + queryStr = `ChasmStatus = 'active'` + queryParams, err = s.visibilityStore.convertQueryLegacy(testNamespace, testNamespaceID, queryStr, chasmMapper, chasm.UnspecifiedArchetypeID) + s.NoError(err) + s.JSONEq(`{"bool":{"filter":[{"term":{"NamespaceId":"bfd5c907-f899-4baf-a7b2-2ab85e623ebd"}},{"bool":{"filter":{"term":{"TemporalKeyword01":"active"}}}}],"must_not":{"exists":{"field":"TemporalNamespaceDivision"}}}}`, s.queryToJSON(queryParams.Query)) + s.Nil(queryParams.Sorter) + + queryStr = `ChasmCount = 42` + queryParams, err = s.visibilityStore.convertQueryLegacy(testNamespace, testNamespaceID, queryStr, chasmMapper, chasm.UnspecifiedArchetypeID) + s.NoError(err) + s.JSONEq(`{"bool":{"filter":[{"term":{"NamespaceId":"bfd5c907-f899-4baf-a7b2-2ab85e623ebd"}},{"bool":{"filter":{"match":{"TemporalInt01":{"query":42}}}}}],"must_not":{"exists":{"field":"TemporalNamespaceDivision"}}}}`, s.queryToJSON(queryParams.Query)) + s.Nil(queryParams.Sorter) + + queryStr = `ChasmScore = 3.14` + queryParams, err = s.visibilityStore.convertQueryLegacy(testNamespace, testNamespaceID, queryStr, chasmMapper, chasm.UnspecifiedArchetypeID) + s.NoError(err) + s.JSONEq(`{"bool":{"filter":[{"term":{"NamespaceId":"bfd5c907-f899-4baf-a7b2-2ab85e623ebd"}},{"bool":{"filter":{"match":{"TemporalDouble01":{"query":3.14}}}}}],"must_not":{"exists":{"field":"TemporalNamespaceDivision"}}}}`, s.queryToJSON(queryParams.Query)) + s.Nil(queryParams.Sorter) + + queryStr = `ChasmStartTime = "2018-06-07T15:04:05.123456789-08:00"` + queryParams, err = s.visibilityStore.convertQueryLegacy(testNamespace, testNamespaceID, queryStr, chasmMapper, chasm.UnspecifiedArchetypeID) + s.NoError(err) + s.JSONEq(`{"bool":{"filter":[{"term":{"NamespaceId":"bfd5c907-f899-4baf-a7b2-2ab85e623ebd"}},{"bool":{"filter":{"match":{"TemporalDatetime01":{"query":"2018-06-07T15:04:05.123456789-08:00"}}}}}],"must_not":{"exists":{"field":"TemporalNamespaceDivision"}}}}`, s.queryToJSON(queryParams.Query)) + s.Nil(queryParams.Sorter) + + queryStr = `ChasmCompleted = true order by ChasmStatus asc` + queryParams, err = s.visibilityStore.convertQueryLegacy(testNamespace, testNamespaceID, queryStr, chasmMapper, chasm.UnspecifiedArchetypeID) + s.NoError(err) + s.JSONEq(`{"bool":{"filter":[{"term":{"NamespaceId":"bfd5c907-f899-4baf-a7b2-2ab85e623ebd"}},{"bool":{"filter":{"match":{"TemporalBool01":{"query":true}}}}}],"must_not":{"exists":{"field":"TemporalNamespaceDivision"}}}}`, s.queryToJSON(queryParams.Query)) + s.JSONEq(`[{"TemporalKeyword01":{"order":"asc"}}]`, s.sorterToJSON(queryParams.Sorter)) + + queryStr = `ChasmStatus = 'active' and WorkflowId = 'wid'` + queryParams, err = s.visibilityStore.convertQueryLegacy(testNamespace, testNamespaceID, queryStr, chasmMapper, chasm.UnspecifiedArchetypeID) + s.NoError(err) + s.JSONEq(`{"bool":{"filter":[{"term":{"NamespaceId":"bfd5c907-f899-4baf-a7b2-2ab85e623ebd"}},{"bool":{"filter":[{"term":{"TemporalKeyword01":"active"}},{"term":{"WorkflowId":"wid"}}]}}],"must_not":{"exists":{"field":"TemporalNamespaceDivision"}}}}`, s.queryToJSON(queryParams.Query)) + s.Nil(queryParams.Sorter) + + queryStr = `UnknownChasmField = 'value'` + _, err = s.visibilityStore.convertQueryLegacy(testNamespace, testNamespaceID, queryStr, chasmMapper, chasm.UnspecifiedArchetypeID) + s.Error(err) + var invalidArgumentErr *serviceerror.InvalidArgument + s.ErrorAs(err, &invalidArgumentErr) + s.Contains(err.Error(), "invalid search attribute: UnknownChasmField") +} + +func (s *ESVisibilitySuite) Test_convertQuery_ChasmMapper() { + chasmMapper := chasm.NewTestVisibilitySearchAttributesMapper( + map[string]string{ + "TemporalBool01": "ChasmCompleted", + "TemporalKeyword01": "ChasmStatus", + "TemporalInt01": "ChasmCount", + }, + map[string]enumspb.IndexedValueType{ + "TemporalBool01": enumspb.INDEXED_VALUE_TYPE_BOOL, + "TemporalKeyword01": enumspb.INDEXED_VALUE_TYPE_KEYWORD, + "TemporalInt01": enumspb.INDEXED_VALUE_TYPE_INT, + }, + ) + + namespaceIDQuery := elastic.NewTermQuery(sadefs.NamespaceID, testNamespaceID.String()) + + testCases := []struct { + name string + query string + want *esQueryParams + err string + }{ + { + name: "chasm bool attribute", + query: "ChasmCompleted = true", + want: &esQueryParams{ + Query: elastic.NewBoolQuery().Filter( + namespaceIDQuery, + elastic.NewBoolQuery().Filter( + namespaceDivisionIsNull, + elastic.NewTermQuery("TemporalBool01", true), + ), + ), + Sorter: []elastic.Sorter{}, + GroupBy: []string{}, + }, + }, + { + name: "chasm keyword attribute", + query: "ChasmStatus = 'active'", + want: &esQueryParams{ + Query: elastic.NewBoolQuery().Filter( + namespaceIDQuery, + elastic.NewBoolQuery().Filter( + namespaceDivisionIsNull, + elastic.NewTermQuery("TemporalKeyword01", "active"), + ), + ), + Sorter: []elastic.Sorter{}, + GroupBy: []string{}, + }, + }, + { + name: "chasm int attribute", + query: "ChasmCount = 42", + want: &esQueryParams{ + Query: elastic.NewBoolQuery().Filter( + namespaceIDQuery, + elastic.NewBoolQuery().Filter( + namespaceDivisionIsNull, + elastic.NewTermQuery("TemporalInt01", int64(42)), + ), + ), + Sorter: []elastic.Sorter{}, + GroupBy: []string{}, + }, + }, + { + name: "chasm attribute with order by", + query: "ChasmCompleted = true ORDER BY ChasmStatus", + want: &esQueryParams{ + Query: elastic.NewBoolQuery().Filter( + namespaceIDQuery, + elastic.NewBoolQuery().Filter( + namespaceDivisionIsNull, + elastic.NewTermQuery("TemporalBool01", true), + ), + ), + Sorter: []elastic.Sorter{elastic.NewFieldSort("TemporalKeyword01")}, + GroupBy: []string{}, + }, + }, + { + name: "chasm and regular attribute", + query: "ChasmStatus = 'active' AND WorkflowId = 'wid'", + want: &esQueryParams{ + Query: elastic.NewBoolQuery().Filter( + namespaceIDQuery, + elastic.NewBoolQuery().Filter( + namespaceDivisionIsNull, + elastic.NewBoolQuery().Filter( + elastic.NewTermQuery("TemporalKeyword01", "active"), + elastic.NewTermQuery(sadefs.WorkflowID, "wid"), + ), + ), + ), + Sorter: []elastic.Sorter{}, + GroupBy: []string{}, + }, + }, + { + name: "invalid chasm attribute", + query: "UnknownChasmField = 'value'", + err: query.InvalidExpressionErrMessage, + }, + } + + for _, tc := range testCases { + s.Run(tc.name, func() { + got, err := s.visibilityStore.convertQuery(testNamespace, testNamespaceID, tc.query, chasmMapper, chasm.UnspecifiedArchetypeID) + if tc.err != "" { + s.Error(err) + s.ErrorContains(err, tc.err) + var invalidArgumentErr *serviceerror.InvalidArgument + s.ErrorAs(err, &invalidArgumentErr) + } else { + s.NoError(err) + s.Equal(tc.want, got) + } + }) + } +} + +func (s *ESVisibilitySuite) TestBuildSearchParametersV2_ChasmMapper() { + chasmMapper := chasm.NewTestVisibilitySearchAttributesMapper( + map[string]string{ + "TemporalBool01": "ChasmCompleted", + "TemporalKeyword01": "ChasmStatus", + }, + map[string]enumspb.IndexedValueType{ + "TemporalBool01": enumspb.INDEXED_VALUE_TYPE_BOOL, + "TemporalKeyword01": enumspb.INDEXED_VALUE_TYPE_KEYWORD, + }, + ) + + request := &manager.ListWorkflowExecutionsRequestV2{ + NamespaceID: testNamespaceID, + Namespace: testNamespace, + PageSize: testPageSize, + } + + matchNamespaceQuery := elastic.NewTermQuery(sadefs.NamespaceID, request.NamespaceID.String()) + + request.Query = `ChasmCompleted = true` + filterQuery := elastic.NewTermQuery("TemporalBool01", true) + boolQuery := elastic.NewBoolQuery().Filter( + matchNamespaceQuery, + elastic.NewBoolQuery().Filter(namespaceDivisionIsNull, filterQuery), + ) + p, err := s.visibilityStore.BuildChasmSearchParameters( + &manager.ListChasmExecutionsRequest{ + NamespaceID: testNamespaceID, + Namespace: testNamespace, + PageSize: testPageSize, + Query: request.Query, + }, + s.visibilityStore.GetListFieldSorter, + chasmMapper, + ) + s.NoError(err) + s.Equal(&client.SearchParameters{ + Index: testIndex, + Query: boolQuery, + SearchAfter: nil, + PageSize: testPageSize, + Sorter: defaultSorter, + }, p) + + request.Query = `ChasmStatus = 'active' ORDER BY ChasmStatus` + filterQuery = elastic.NewTermQuery("TemporalKeyword01", "active") + boolQuery = elastic.NewBoolQuery().Filter( + matchNamespaceQuery, + elastic.NewBoolQuery().Filter(namespaceDivisionIsNull, filterQuery), + ) + s.mockMetricsHandler.EXPECT().WithTags(metrics.NamespaceTag(request.Namespace.String())).Return(s.mockMetricsHandler) + s.mockMetricsHandler.EXPECT().Counter(metrics.ElasticsearchCustomOrderByClauseCount.Name()).Return(metrics.NoopCounterMetricFunc) + p, err = s.visibilityStore.BuildChasmSearchParameters( + &manager.ListChasmExecutionsRequest{ + NamespaceID: testNamespaceID, + Namespace: testNamespace, + PageSize: testPageSize, + Query: request.Query, + }, + s.visibilityStore.GetListFieldSorter, + chasmMapper, + ) + s.NoError(err) + s.Equal(&client.SearchParameters{ + Index: testIndex, + Query: boolQuery, + SearchAfter: nil, + PageSize: testPageSize, + Sorter: []elastic.Sorter{ + elastic.NewFieldSort("TemporalKeyword01").Asc(), + elastic.NewFieldSort(sadefs.RunID).Desc(), + }, + }, p) +} + +func (s *ESVisibilitySuite) TestParseESDoc_ChasmSearchAttributes() { + chasmMapper := chasm.NewTestVisibilitySearchAttributesMapper( + map[string]string{ + "TemporalBool01": "ChasmCompleted", + "TemporalKeyword01": "ChasmStatus", + "TemporalInt01": "ChasmCount", + "TemporalDouble01": "ChasmScore", + "TemporalDatetime01": "ChasmStartTime", + }, + map[string]enumspb.IndexedValueType{ + "TemporalBool01": enumspb.INDEXED_VALUE_TYPE_BOOL, + "TemporalKeyword01": enumspb.INDEXED_VALUE_TYPE_KEYWORD, + "TemporalInt01": enumspb.INDEXED_VALUE_TYPE_INT, + "TemporalDouble01": enumspb.INDEXED_VALUE_TYPE_DOUBLE, + "TemporalDatetime01": enumspb.INDEXED_VALUE_TYPE_DATETIME, + }, + ) + + saTypeMap := searchattribute.TestEsNameTypeMap() + docSource := []byte(`{ + "ExecutionStatus": "Completed", + "NamespaceId": "bfd5c907-f899-4baf-a7b2-2ab85e623ebd", + "RunId": "e481009e-14b3-45ae-91af-dce6e2a88365", + "StartTime": "2021-06-11T15:04:07.980-07:00", + "WorkflowId": "6bfbc1e5-6ce4-4e22-bbfb-e0faa9a7a604-1-2256", + "WorkflowType": "TestWorkflowExecute", + "TemporalBool01": true, + "TemporalKeyword01": "active", + "TemporalInt01": 42, + "TemporalDouble01": 3.14, + "TemporalDatetime01": "2018-06-07T15:04:05.123456789-08:00" + }`) + + info, err := s.visibilityStore.ParseESDoc("", docSource, saTypeMap, testNamespace, chasmMapper) + s.NoError(err) + s.NotNil(info) + s.Equal("6bfbc1e5-6ce4-4e22-bbfb-e0faa9a7a604-1-2256", info.WorkflowID) + s.Equal("e481009e-14b3-45ae-91af-dce6e2a88365", info.RunID) + s.Equal("TestWorkflowExecute", info.TypeName) + s.Equal(enumspb.WORKFLOW_EXECUTION_STATUS_COMPLETED, info.Status) + + s.NotNil(info.ChasmSearchAttributes) + s.Len(info.ChasmSearchAttributes, 5) + completedVal, ok := info.ChasmSearchAttributes["ChasmCompleted"].(chasm.VisibilityValueBool) + s.True(ok) + s.True(bool(completedVal)) + statusVal, ok := info.ChasmSearchAttributes["ChasmStatus"].(chasm.VisibilityValueString) + s.True(ok) + s.Equal("active", string(statusVal)) + countVal, ok := info.ChasmSearchAttributes["ChasmCount"].(chasm.VisibilityValueInt64) + s.True(ok) + s.Equal(int64(42), int64(countVal)) + scoreVal, ok := info.ChasmSearchAttributes["ChasmScore"].(chasm.VisibilityValueFloat64) + s.True(ok) + s.InDelta(3.14, float64(scoreVal), 0.001) + expectedTime, err := time.Parse(time.RFC3339Nano, "2018-06-07T15:04:05.123456789-08:00") + s.NoError(err) + timeVal, ok := info.ChasmSearchAttributes["ChasmStartTime"].(chasm.VisibilityValueTime) + s.True(ok) + s.Equal(expectedTime, time.Time(timeVal)) +} + +func (s *ESVisibilitySuite) TestParseESDoc_ChasmSearchAttributes_NoMapper() { + saTypeMap := searchattribute.TestEsNameTypeMap() + docSource := []byte(`{ + "ExecutionStatus": "Completed", + "NamespaceId": "bfd5c907-f899-4baf-a7b2-2ab85e623ebd", + "RunId": "e481009e-14b3-45ae-91af-dce6e2a88365", + "StartTime": "2021-06-11T15:04:07.980-07:00", + "WorkflowId": "6bfbc1e5-6ce4-4e22-bbfb-e0faa9a7a604-1-2256", + "WorkflowType": "TestWorkflowExecute", + "TemporalBool01": true, + "TemporalKeyword01": "active" + }`) + + info, err := s.visibilityStore.ParseESDoc("", docSource, saTypeMap, testNamespace, nil) + s.NoError(err) + s.NotNil(info) + s.Nil(info.ChasmSearchAttributes) +} + +func (s *ESVisibilitySuite) TestNameInterceptor_ChasmMapper() { + chasmMapper := chasm.NewTestVisibilitySearchAttributesMapper( + map[string]string{ + "TemporalBool01": "ChasmCompleted", + "TemporalKeyword01": "ChasmStatus", + }, + map[string]enumspb.IndexedValueType{ + "TemporalBool01": enumspb.INDEXED_VALUE_TYPE_BOOL, + "TemporalKeyword01": enumspb.INDEXED_VALUE_TYPE_KEYWORD, + }, + ) + + ni := NewNameInterceptor( + testNamespace, + searchattribute.TestEsNameTypeMap(), + s.mockSearchAttributesMapperProvider, + chasmMapper, + ) + + fieldName, err := ni.Name("ChasmCompleted", query.FieldNameFilter) + s.NoError(err) + s.Equal("TemporalBool01", fieldName) + + fieldName, err = ni.Name("ChasmStatus", query.FieldNameFilter) + s.NoError(err) + s.Equal("TemporalKeyword01", fieldName) + + fieldName, err = ni.Name("ChasmStatus", query.FieldNameSorter) + s.NoError(err) + s.Equal("TemporalKeyword01", fieldName) + + _, err = ni.Name("UnknownChasmField", query.FieldNameFilter) + s.Error(err) + var converterErr *query.ConverterError + s.ErrorAs(err, &converterErr) +} + +func (s *ESVisibilitySuite) TestValuesInterceptor_ChasmMapper() { + chasmMapper := chasm.NewTestVisibilitySearchAttributesMapper( + map[string]string{ + "TemporalBool01": "ChasmCompleted", + "TemporalKeyword01": "ChasmStatus", + "TemporalInt01": "ChasmCount", + "TemporalDouble01": "ChasmScore", + "TemporalDatetime01": "ChasmStartTime", + }, + map[string]enumspb.IndexedValueType{ + "TemporalBool01": enumspb.INDEXED_VALUE_TYPE_BOOL, + "TemporalKeyword01": enumspb.INDEXED_VALUE_TYPE_KEYWORD, + "TemporalInt01": enumspb.INDEXED_VALUE_TYPE_INT, + "TemporalDouble01": enumspb.INDEXED_VALUE_TYPE_DOUBLE, + "TemporalDatetime01": enumspb.INDEXED_VALUE_TYPE_DATETIME, + }, + ) + + vi := NewValuesInterceptor( + testNamespace, + searchattribute.TestEsNameTypeMap(), + chasmMapper, + ) + + values, err := vi.Values("ChasmCompleted", "TemporalBool01", true) + s.NoError(err) + s.Len(values, 1) + s.Equal(true, values[0]) + + values, err = vi.Values("ChasmStatus", "TemporalKeyword01", "active") + s.NoError(err) + s.Len(values, 1) + s.Equal("active", values[0]) + + values, err = vi.Values("ChasmCount", "TemporalInt01", int64(42)) + s.NoError(err) + s.Len(values, 1) + s.Equal(int64(42), values[0]) + + values, err = vi.Values("ChasmScore", "TemporalDouble01", 3.14) + s.NoError(err) + s.Len(values, 1) + s.InDelta(3.14, values[0], 0.0001) + + testTime := time.Unix(0, 1528358645123456789).UTC() + values, err = vi.Values("ChasmStartTime", "TemporalDatetime01", testTime.UnixNano()) + s.NoError(err) + s.Len(values, 1) + s.Equal("2018-06-07T08:04:05.123456789Z", values[0]) + + _, err = vi.Values("ChasmCompleted", "TemporalBool01", "not-a-bool") + s.Error(err) + var converterErr *query.ConverterError + s.ErrorAs(err, &converterErr) +} diff --git a/common/persistence/visibility/store/query/converter.go b/common/persistence/visibility/store/query/converter.go index a0b58f8ddf0..b5f7fbeeb29 100644 --- a/common/persistence/visibility/store/query/converter.go +++ b/common/persistence/visibility/store/query/converter.go @@ -5,12 +5,14 @@ package query import ( "fmt" "slices" + "strconv" "strings" "time" "github.com/temporalio/sqlparser" enumspb "go.temporal.io/api/enums/v1" "go.temporal.io/api/serviceerror" + "go.temporal.io/server/chasm" "go.temporal.io/server/common/namespace" "go.temporal.io/server/common/primitives" "go.temporal.io/server/common/searchattribute" @@ -58,6 +60,8 @@ type ( namespaceName namespace.Name saTypeMap searchattribute.NameTypeMap saMapper searchattribute.Mapper + chasmMapper *chasm.VisibilitySearchAttributesMapper + archetypeID chasm.ArchetypeID seenNamespaceDivision bool } @@ -150,6 +154,20 @@ func (c *QueryConverter[ExprT]) WithSearchAttributeInterceptor( return c } +func (c *QueryConverter[ExprT]) WithChasmMapper( + chasmMapper *chasm.VisibilitySearchAttributesMapper, +) *QueryConverter[ExprT] { + c.chasmMapper = chasmMapper + return c +} + +func (c *QueryConverter[ExprT]) WithArchetypeID( + archetypeID chasm.ArchetypeID, +) *QueryConverter[ExprT] { + c.archetypeID = archetypeID + return c +} + func (c *QueryConverter[ExprT]) SeenNamespaceDivision() bool { return c.seenNamespaceDivision } @@ -163,17 +181,29 @@ func (c *QueryConverter[ExprT]) Convert( } // If the query did not explicitly filter on TemporalNamespaceDivision, - // then add "is null" query to it. + // try setting the namespace division filter based on the archetype ID, + // else filter by null (no division). var namespaceDivisionExpr ExprT if !c.seenNamespaceDivision { nsDivisionCol := NamespaceDivisionSAColumn() if err := c.saInterceptor.Intercept(nsDivisionCol); err != nil { return nil, err } - namespaceDivisionExpr, err = c.storeQC.ConvertIsExpr( - sqlparser.IsNullStr, - nsDivisionCol, - ) + + if c.archetypeID != chasm.UnspecifiedArchetypeID { + // For CHASM queries, filter by archetype ID + namespaceDivisionExpr, err = c.storeQC.ConvertComparisonExpr( + sqlparser.EqualStr, + nsDivisionCol, + strconv.Itoa(int(c.archetypeID)), + ) + } else { + // For regular workflow queries, filter by null (no division) + namespaceDivisionExpr, err = c.storeQC.ConvertIsExpr( + sqlparser.IsNullStr, + nsDivisionCol, + ) + } if err != nil { return nil, err } @@ -476,6 +506,22 @@ func (c *QueryConverter[ExprT]) resolveSearchAttributeAlias( fieldName, fieldType = fn, ft return true } + // resolveChasmSA only returns true if `alias` is a CHASM search attribute. + resolveChasmSA := func(alias string) bool { + if c.chasmMapper == nil { + return false + } + fn, err := c.chasmMapper.Field(alias) + if err != nil { + return false + } + ft, err := c.chasmMapper.ValueType(fn) + if err != nil { + return false + } + fieldName, fieldType = fn, ft + return true + } var err error fieldName = alias @@ -483,12 +529,16 @@ func (c *QueryConverter[ExprT]) resolveSearchAttributeAlias( if sadefs.IsMappable(alias) && resolveCSA(alias) { return } - // Second, check if it's a system/reserved search attribute. + // Second, check if it's a CHASM search attribute. + if resolveChasmSA(alias) { + return + } + // Third, check if it's a system/reserved search attribute. fieldType, err = c.saTypeMap.GetType(fieldName) if err == nil { return } - // Third, check for special aliases or adding/removing the `Temporal` prefix. + // Fourth, check for special aliases or adding/removing the `Temporal` prefix. if strings.TrimPrefix(alias, sadefs.ReservedPrefix) == sadefs.ScheduleID { fieldName = sadefs.WorkflowID } else if strings.HasPrefix(fieldName, sadefs.ReservedPrefix) { diff --git a/common/persistence/visibility/store/query/converter_legacy.go b/common/persistence/visibility/store/query/converter_legacy.go index 3766c559b98..a5ed7ce78af 100644 --- a/common/persistence/visibility/store/query/converter_legacy.go +++ b/common/persistence/visibility/store/query/converter_legacy.go @@ -4,11 +4,13 @@ package query import ( "errors" "fmt" + "maps" "strings" "github.com/olivere/elastic/v7" "github.com/temporalio/sqlparser" enumspb "go.temporal.io/api/enums/v1" + "go.temporal.io/server/chasm" "go.temporal.io/server/common/searchattribute" "go.temporal.io/server/common/sqlquery" ) @@ -49,7 +51,8 @@ type ( fnInterceptor FieldNameInterceptor fvInterceptor FieldValuesInterceptor allowedOperators map[string]struct{} - saNameType searchattribute.NameTypeMap + finalTypeMap searchattribute.NameTypeMap + chasmMapper *chasm.VisibilitySearchAttributesMapper } isConverter struct { @@ -144,7 +147,8 @@ func NewComparisonExprConverter( fnInterceptor FieldNameInterceptor, fvInterceptor FieldValuesInterceptor, allowedOperators map[string]struct{}, - saNameType searchattribute.NameTypeMap, + customSaNameType searchattribute.NameTypeMap, + chasmMapper *chasm.VisibilitySearchAttributesMapper, ) ExprConverter { if fnInterceptor == nil { fnInterceptor = &NopFieldNameInterceptor{} @@ -152,11 +156,18 @@ func NewComparisonExprConverter( if fvInterceptor == nil { fvInterceptor = &NopFieldValuesInterceptor{} } + + combinedTypeMap := make(map[string]enumspb.IndexedValueType) + maps.Copy(combinedTypeMap, customSaNameType.Custom()) + maps.Copy(combinedTypeMap, chasmMapper.SATypeMap()) + finalTypeMap := searchattribute.NewNameTypeMap(combinedTypeMap) + return &comparisonExprConverter{ fnInterceptor: fnInterceptor, fvInterceptor: fvInterceptor, allowedOperators: allowedOperators, - saNameType: saNameType, + finalTypeMap: finalTypeMap, + chasmMapper: chasmMapper, } } @@ -451,7 +462,7 @@ func (c *comparisonExprConverter) Convert(expr sqlparser.Expr) (elastic.Query, e return nil, NewConverterError("operator '%v' not allowed in comparison expression", comparisonExpr.Operator) } - tp, err := c.saNameType.GetType(colName) + tp, err := c.finalTypeMap.GetType(colName) if err != nil { return nil, err } diff --git a/common/persistence/visibility/store/query/converter_test.go b/common/persistence/visibility/store/query/converter_test.go index 00622ecc1b1..d5004abc604 100644 --- a/common/persistence/visibility/store/query/converter_test.go +++ b/common/persistence/visibility/store/query/converter_test.go @@ -10,6 +10,7 @@ import ( "github.com/temporalio/sqlparser" enumspb "go.temporal.io/api/enums/v1" "go.temporal.io/api/serviceerror" + "go.temporal.io/server/chasm" "go.temporal.io/server/common/namespace" "go.temporal.io/server/common/primitives" "go.temporal.io/server/common/searchattribute" @@ -2527,3 +2528,132 @@ func parseWhereString(where string) sqlparser.Expr { } return stmt.(*sqlparser.Select).Where.Expr } + +func TestQueryConverter_WithChasmMapper(t *testing.T) { + t.Parallel() + r := require.New(t) + ctrl := gomock.NewController(t) + storeQCMock := NewMockStoreQueryConverter[sqlparser.Expr](ctrl) + + chasmMapper := chasm.NewTestVisibilitySearchAttributesMapper( + map[string]string{ + "TemporalKeyword01": "ChasmStatus", + }, + map[string]enumspb.IndexedValueType{ + "TemporalKeyword01": enumspb.INDEXED_VALUE_TYPE_KEYWORD, + }, + ) + + c := NewQueryConverter( + storeQCMock, + testNamespaceName, + searchattribute.TestNameTypeMap(), + &searchattribute.TestMapper{}, + ) + r.Nil(c.chasmMapper) + + c = c.WithChasmMapper(chasmMapper) + r.Equal(chasmMapper, c.chasmMapper) + + c = c.WithChasmMapper(nil) + r.Nil(c.chasmMapper) +} + +func TestQueryConverter_WithArchetypeID(t *testing.T) { + t.Parallel() + r := require.New(t) + ctrl := gomock.NewController(t) + storeQCMock := NewMockStoreQueryConverter[sqlparser.Expr](ctrl) + + c := NewQueryConverter( + storeQCMock, + testNamespaceName, + searchattribute.TestNameTypeMap(), + &searchattribute.TestMapper{}, + ) + r.Equal(chasm.UnspecifiedArchetypeID, c.archetypeID) + + c = c.WithArchetypeID(123) + r.Equal(chasm.ArchetypeID(123), c.archetypeID) + + c = c.WithArchetypeID(chasm.UnspecifiedArchetypeID) + r.Equal(chasm.UnspecifiedArchetypeID, c.archetypeID) +} + +func TestQueryConverter_ResolveSearchAttributeAlias_WithChasmMapper(t *testing.T) { + t.Parallel() + ctrl := gomock.NewController(t) + storeQCMock := NewMockStoreQueryConverter[sqlparser.Expr](ctrl) + + chasmMapper := chasm.NewTestVisibilitySearchAttributesMapper( + map[string]string{ + "TemporalBool01": "ChasmCompleted", + "TemporalKeyword01": "ChasmStatus", + }, + map[string]enumspb.IndexedValueType{ + "TemporalBool01": enumspb.INDEXED_VALUE_TYPE_BOOL, + "TemporalKeyword01": enumspb.INDEXED_VALUE_TYPE_KEYWORD, + }, + ) + + queryConverter := NewQueryConverter( + storeQCMock, + testNamespaceName, + searchattribute.TestNameTypeMap(), + &searchattribute.TestMapper{}, + ).WithChasmMapper(chasmMapper) + + testCases := []struct { + name string + expectedFieldName string + expectedFieldType enumspb.IndexedValueType + expectedErr bool + expectNamespaceDivision bool + }{ + { + name: "ChasmCompleted", + expectedFieldName: "TemporalBool01", + expectedFieldType: enumspb.INDEXED_VALUE_TYPE_BOOL, + expectedErr: false, + expectNamespaceDivision: false, + }, + { + name: "ChasmStatus", + expectedFieldName: "TemporalKeyword01", + expectedFieldType: enumspb.INDEXED_VALUE_TYPE_KEYWORD, + expectedErr: false, + expectNamespaceDivision: false, + }, + { + name: "TemporalNamespaceDivision", + expectedFieldName: "TemporalNamespaceDivision", + expectedFieldType: enumspb.INDEXED_VALUE_TYPE_KEYWORD, + expectedErr: false, + expectNamespaceDivision: true, + }, + { + name: "NonExistentChasmAlias", + expectedFieldName: "", + expectedFieldType: enumspb.INDEXED_VALUE_TYPE_UNSPECIFIED, + expectedErr: true, + expectNamespaceDivision: false, + }, + } + + for _, tc := range testCases { + t.Run(tc.name, func(t *testing.T) { + r := require.New(t) + fieldName, fieldType, err := queryConverter.resolveSearchAttributeAlias(tc.name) + if tc.expectedErr { + r.Error(err) + // Note: fieldName may have been set during resolution attempts + r.Equal(enumspb.INDEXED_VALUE_TYPE_UNSPECIFIED, fieldType) + } else { + r.NoError(err) + r.Equal(tc.expectedFieldName, fieldName) + r.Equal(tc.expectedFieldType, fieldType) + // Note: seenNamespaceDivision is only set in convertColName, not resolveSearchAttributeAlias + } + }) + } +} diff --git a/common/persistence/visibility/store/query/interceptors_legacy_test.go b/common/persistence/visibility/store/query/interceptors_legacy_test.go index bffd66a2f18..4fe5b12c9c2 100644 --- a/common/persistence/visibility/store/query/interceptors_legacy_test.go +++ b/common/persistence/visibility/store/query/interceptors_legacy_test.go @@ -104,7 +104,7 @@ func getTestConverter(fnInterceptor FieldNameInterceptor, fvInterceptor FieldVal nil, nil, NewRangeCondConverter(fnInterceptor, fvInterceptor, false), - NewComparisonExprConverter(fnInterceptor, fvInterceptor, map[string]struct{}{sqlparser.EqualStr: {}, sqlparser.InStr: {}}, testNameTypeMap), + NewComparisonExprConverter(fnInterceptor, fvInterceptor, map[string]struct{}{sqlparser.EqualStr: {}, sqlparser.InStr: {}}, testNameTypeMap, nil), nil) return NewConverterLegacy(fnInterceptor, whereConverter) } diff --git a/common/persistence/visibility/store/query/resolve.go b/common/persistence/visibility/store/query/resolve.go index 5594c741a3a..b829cd8f861 100644 --- a/common/persistence/visibility/store/query/resolve.go +++ b/common/persistence/visibility/store/query/resolve.go @@ -4,6 +4,7 @@ import ( "strings" enumspb "go.temporal.io/api/enums/v1" + "go.temporal.io/server/chasm" "go.temporal.io/server/common/namespace" "go.temporal.io/server/common/searchattribute" "go.temporal.io/server/common/searchattribute/sadefs" @@ -12,14 +13,15 @@ import ( // ResolveSearchAttributeAlias resolves the search attribute alias for the given name. The process is: // 1. If the name has the "Temporal", skip mapping to a custom search attribute. // 2. If the search attribute exists in the visibility mapper, pass it through. -// 3. If it exists as a system / predefined attribute, map it. -// 3.1 Some pre-defined attributes are already defined with the Temporal prefix, so both options need to be checked. -// 4. In the future, also need to lookup in the CHASM archetype search attribute mapping +// 3. If the search attribute exists in the CHASM mapper, pass it through. +// 4. If it exists as a system / predefined attribute, map it. +// 4.1 Some pre-defined attributes are already defined with the Temporal prefix, so both options need to be checked. func ResolveSearchAttributeAlias( name string, ns namespace.Name, mapper searchattribute.Mapper, saTypeMap searchattribute.NameTypeMap, + chasmMapper *chasm.VisibilitySearchAttributesMapper, ) (string, enumspb.IndexedValueType, error) { if sadefs.IsMappable(name) { // First check if the visibility mapper can handle this field (e.g., custom search attributes) @@ -35,6 +37,11 @@ func ResolveSearchAttributeAlias( saType, _ := saTypeMap.GetType(sadefs.WorkflowID) return sadefs.WorkflowID, saType, nil } + + fieldName, fieldType = tryChasmMapper(name, chasmMapper) + if fieldName != "" { + return fieldName, fieldType, nil + } } fieldName, fieldType, found := tryDirectAndPrefixedLookup(name, saTypeMap) @@ -75,6 +82,23 @@ func tryVisibilityMapper( return fieldName, fieldType } +func tryChasmMapper(name string, chasmMapper *chasm.VisibilitySearchAttributesMapper) (string, enumspb.IndexedValueType) { + if chasmMapper == nil { + return "", enumspb.INDEXED_VALUE_TYPE_UNSPECIFIED + } + + fieldName, err := chasmMapper.Field(name) + if err != nil { + return "", enumspb.INDEXED_VALUE_TYPE_UNSPECIFIED + } + + fieldType, err := chasmMapper.ValueType(fieldName) + if err != nil { + return "", enumspb.INDEXED_VALUE_TYPE_UNSPECIFIED + } + return fieldName, fieldType +} + func tryDirectAndPrefixedLookup(name string, saTypeMap searchattribute.NameTypeMap) (string, enumspb.IndexedValueType, bool) { if saType, err := saTypeMap.GetType(name); err == nil { return name, saType, true diff --git a/common/persistence/visibility/store/query/resolve_test.go b/common/persistence/visibility/store/query/resolve_test.go index 454ee01070b..6c7a09e6de6 100644 --- a/common/persistence/visibility/store/query/resolve_test.go +++ b/common/persistence/visibility/store/query/resolve_test.go @@ -7,6 +7,7 @@ import ( "github.com/stretchr/testify/require" enumspb "go.temporal.io/api/enums/v1" "go.temporal.io/api/serviceerror" + "go.temporal.io/server/chasm" "go.temporal.io/server/common/namespace" "go.temporal.io/server/common/searchattribute" "go.temporal.io/server/common/searchattribute/sadefs" @@ -48,7 +49,7 @@ func TestResolveSearchAttributeAlias(t *testing.T) { for _, tc := range cases { t.Run(tc.name, func(t *testing.T) { - fieldName, fieldType, err := ResolveSearchAttributeAlias(tc.name, ns, mapper, saTypeMap) + fieldName, fieldType, err := ResolveSearchAttributeAlias(tc.name, ns, mapper, saTypeMap, nil) require.Equal(t, tc.expectedFieldName, fieldName) require.Equal(t, tc.expectedFieldType, fieldType) require.Equal(t, tc.expectedErr, err != nil) @@ -74,7 +75,7 @@ func TestResolveSearchAttributeAlias_CustomScheduleID(t *testing.T) { } // When ScheduleID is a custom search attribute, it should use the custom attribute, not transform to WorkflowID - fieldName, fieldType, err := ResolveSearchAttributeAlias(sadefs.ScheduleID, ns, mapper, saTypeMapWithCustomScheduleID) + fieldName, fieldType, err := ResolveSearchAttributeAlias(sadefs.ScheduleID, ns, mapper, saTypeMapWithCustomScheduleID, nil) require.NoError(t, err) require.Equal(t, sadefs.ScheduleID, fieldName) require.Equal(t, enumspb.INDEXED_VALUE_TYPE_KEYWORD_LIST, fieldType) @@ -102,3 +103,151 @@ func (m customMapper) GetFieldName(alias string, ns string) (string, error) { fmt.Sprintf("Namespace %s has no mapping defined for search attribute %s", ns, alias), ) } + +func TestResolveSearchAttributeAlias_WithChasmMapper(t *testing.T) { + ns := namespace.Name("test-namespace") + saTypeMap := searchattribute.NewNameTypeMapStub(map[string]enumspb.IndexedValueType{ + "ExecutionStatus": enumspb.INDEXED_VALUE_TYPE_KEYWORD, + "StartTime": enumspb.INDEXED_VALUE_TYPE_DATETIME, + }) + mapper := customMapper{ + fieldToAlias: map[string]string{}, + aliasToField: map[string]string{}, + } + + chasmMapper := chasm.NewTestVisibilitySearchAttributesMapper( + map[string]string{ + "TemporalBool01": "ChasmCompleted", + "TemporalKeyword01": "ChasmStatus", + }, + map[string]enumspb.IndexedValueType{ + "TemporalBool01": enumspb.INDEXED_VALUE_TYPE_BOOL, + "TemporalKeyword01": enumspb.INDEXED_VALUE_TYPE_KEYWORD, + }, + ) + + testCases := []struct { + name string + chasmMapper *chasm.VisibilitySearchAttributesMapper + expectedFieldName string + expectedFieldType enumspb.IndexedValueType + expectedErr bool + }{ + { + name: "ChasmCompleted", + chasmMapper: chasmMapper, + expectedFieldName: "TemporalBool01", + expectedFieldType: enumspb.INDEXED_VALUE_TYPE_BOOL, + expectedErr: false, + }, + { + name: "ChasmStatus", + chasmMapper: chasmMapper, + expectedFieldName: "TemporalKeyword01", + expectedFieldType: enumspb.INDEXED_VALUE_TYPE_KEYWORD, + expectedErr: false, + }, + { + name: "ExecutionStatus", + chasmMapper: chasmMapper, + expectedFieldName: "ExecutionStatus", + expectedFieldType: enumspb.INDEXED_VALUE_TYPE_KEYWORD, + expectedErr: false, + }, + { + name: "NonExistentChasmAlias", + chasmMapper: chasmMapper, + expectedFieldName: "", + expectedFieldType: enumspb.INDEXED_VALUE_TYPE_UNSPECIFIED, + expectedErr: true, + }, + { + name: "ChasmCompleted", + chasmMapper: nil, + expectedFieldName: "", + expectedFieldType: enumspb.INDEXED_VALUE_TYPE_UNSPECIFIED, + expectedErr: true, + }, + } + + for _, tc := range testCases { + t.Run(tc.name, func(t *testing.T) { + fieldName, fieldType, err := ResolveSearchAttributeAlias(tc.name, ns, mapper, saTypeMap, tc.chasmMapper) + require.Equal(t, tc.expectedFieldName, fieldName) + require.Equal(t, tc.expectedFieldType, fieldType) + require.Equal(t, tc.expectedErr, err != nil) + }) + } +} + +func TestResolveSearchAttributeAlias_ChasmPriority(t *testing.T) { + ns := namespace.Name("test-namespace") + saTypeMap := searchattribute.NewNameTypeMapStub(map[string]enumspb.IndexedValueType{ + "MyCustomField": enumspb.INDEXED_VALUE_TYPE_KEYWORD, + "StartTime": enumspb.INDEXED_VALUE_TYPE_DATETIME, + }) + mapper := customMapper{ + fieldToAlias: map[string]string{ + "MyCustomField": "MyCustomField", + }, + aliasToField: map[string]string{ + "MyCustomField": "MyCustomField", + }, + } + + chasmMapper := chasm.NewTestVisibilitySearchAttributesMapper( + map[string]string{ + "TemporalKeyword01": "MyCustomField", + }, + map[string]enumspb.IndexedValueType{ + "TemporalKeyword01": enumspb.INDEXED_VALUE_TYPE_KEYWORD, + }, + ) + + // Custom SA should take precedence over CHASM SA + fieldName, fieldType, err := ResolveSearchAttributeAlias("MyCustomField", ns, mapper, saTypeMap, chasmMapper) + require.NoError(t, err) + require.Equal(t, "MyCustomField", fieldName) + require.Equal(t, enumspb.INDEXED_VALUE_TYPE_KEYWORD, fieldType) + + // CHASM SA should be used when custom SA doesn't exist + chasmMapper2 := chasm.NewTestVisibilitySearchAttributesMapper( + map[string]string{ + "TemporalKeyword01": "ChasmOnlyField", + }, + map[string]enumspb.IndexedValueType{ + "TemporalKeyword01": enumspb.INDEXED_VALUE_TYPE_KEYWORD, + }, + ) + fieldName2, fieldType2, err2 := ResolveSearchAttributeAlias("ChasmOnlyField", ns, mapper, saTypeMap, chasmMapper2) + require.NoError(t, err2) + require.Equal(t, "TemporalKeyword01", fieldName2) + require.Equal(t, enumspb.INDEXED_VALUE_TYPE_KEYWORD, fieldType2) +} + +func TestResolveSearchAttributeAlias_ChasmMissingType(t *testing.T) { + ns := namespace.Name("test-namespace") + saTypeMap := searchattribute.NewNameTypeMapStub(map[string]enumspb.IndexedValueType{ + "StartTime": enumspb.INDEXED_VALUE_TYPE_DATETIME, + }) + mapper := customMapper{ + fieldToAlias: map[string]string{}, + aliasToField: map[string]string{}, + } + + // CHASM mapper with field that doesn't exist in type map + chasmMapper := chasm.NewTestVisibilitySearchAttributesMapper( + map[string]string{ + "TemporalKeyword01": "ChasmField", + }, + map[string]enumspb.IndexedValueType{ + // Missing TemporalKeyword01 in type map + }, + ) + + // Should fall back to system SA resolution + fieldName, fieldType, err := ResolveSearchAttributeAlias("ChasmField", ns, mapper, saTypeMap, chasmMapper) + require.Error(t, err) + require.Empty(t, fieldName) + require.Equal(t, enumspb.INDEXED_VALUE_TYPE_UNSPECIFIED, fieldType) +} diff --git a/common/persistence/visibility/store/sql/query_converter_legacy.go b/common/persistence/visibility/store/sql/query_converter_legacy.go index 7c6c8eb0851..2f023f10b90 100644 --- a/common/persistence/visibility/store/sql/query_converter_legacy.go +++ b/common/persistence/visibility/store/sql/query_converter_legacy.go @@ -9,6 +9,7 @@ import ( "github.com/temporalio/sqlparser" enumspb "go.temporal.io/api/enums/v1" + "go.temporal.io/server/chasm" "go.temporal.io/server/common/namespace" "go.temporal.io/server/common/persistence/sql/sqlplugin" "go.temporal.io/server/common/persistence/visibility/store/query" @@ -47,6 +48,9 @@ type ( queryString string seenNamespaceDivision bool + + chasmMapper *chasm.VisibilitySearchAttributesMapper + archetypeID chasm.ArchetypeID } queryParamsLegacy struct { @@ -97,6 +101,8 @@ func newQueryConverterInternal( saTypeMap searchattribute.NameTypeMap, saMapper searchattribute.Mapper, queryString string, + chasmMapper *chasm.VisibilitySearchAttributesMapper, + archetypeID chasm.ArchetypeID, ) *QueryConverterLegacy { return &QueryConverterLegacy{ pluginQueryConverterLegacy: pqc, @@ -107,6 +113,9 @@ func newQueryConverterInternal( queryString: queryString, seenNamespaceDivision: false, + + chasmMapper: chasmMapper, + archetypeID: archetypeID, } } @@ -222,13 +231,23 @@ func (c *QueryConverterLegacy) convertSelectStmt(sel *sqlparser.Select) error { // This logic comes from elasticsearch/visibility_store.go#convertQuery function. // If the query did not explicitly filter on TemporalNamespaceDivision, - // then add "is null" query to it. + // try setting the namespace division filter based on the archetype ID, + // else filter by null (no division). if !c.seenNamespaceDivision { - namespaceDivisionExpr := &sqlparser.IsExpr{ - Operator: sqlparser.IsNullStr, - Expr: newColName( - sadefs.GetSqlDbColName(sadefs.TemporalNamespaceDivision), - ), + var namespaceDivisionExpr sqlparser.Expr + if c.archetypeID != chasm.UnspecifiedArchetypeID { + namespaceDivisionExpr = &sqlparser.ComparisonExpr{ + Operator: sqlparser.EqualStr, + Left: newColName(sadefs.GetSqlDbColName(sadefs.TemporalNamespaceDivision)), + Right: sqlparser.NewStrVal([]byte(strconv.Itoa(int(c.archetypeID)))), + } + } else { + namespaceDivisionExpr = &sqlparser.IsExpr{ + Operator: sqlparser.IsNullStr, + Expr: newColName( + sadefs.GetSqlDbColName(sadefs.TemporalNamespaceDivision), + ), + } } if sel.Where.Expr == nil { sel.Where.Expr = namespaceDivisionExpr @@ -425,7 +444,8 @@ func (c *QueryConverterLegacy) convertColName(exprRef *sqlparser.Expr) (*saColNa ) } saAlias := strings.ReplaceAll(sqlparser.String(expr), "`", "") - saFieldName, saType, err := query.ResolveSearchAttributeAlias(saAlias, c.namespaceName, c.saMapper, c.saTypeMap) + + saFieldName, saType, err := query.ResolveSearchAttributeAlias(saAlias, c.namespaceName, c.saMapper, c.saTypeMap, c.chasmMapper) if err != nil { return nil, query.NewConverterError( "%s: column name '%s' is not a valid search attribute", diff --git a/common/persistence/visibility/store/sql/query_converter_legacy_factory.go b/common/persistence/visibility/store/sql/query_converter_legacy_factory.go index 3bb6aa28732..0c61dc4e293 100644 --- a/common/persistence/visibility/store/sql/query_converter_legacy_factory.go +++ b/common/persistence/visibility/store/sql/query_converter_legacy_factory.go @@ -1,6 +1,7 @@ package sql import ( + "go.temporal.io/server/chasm" "go.temporal.io/server/common/namespace" "go.temporal.io/server/common/persistence/sql/sqlplugin/mysql" "go.temporal.io/server/common/persistence/sql/sqlplugin/postgresql" @@ -15,14 +16,16 @@ func NewQueryConverterLegacy( saTypeMap searchattribute.NameTypeMap, saMapper searchattribute.Mapper, queryString string, + chasmMapper *chasm.VisibilitySearchAttributesMapper, + archetypeID chasm.ArchetypeID, ) *QueryConverterLegacy { switch pluginName { case mysql.PluginName: - return newMySQLQueryConverter(namespaceName, namespaceID, saTypeMap, saMapper, queryString) + return newMySQLQueryConverter(namespaceName, namespaceID, saTypeMap, saMapper, queryString, chasmMapper, archetypeID) case postgresql.PluginName, postgresql.PluginNamePGX: - return newPostgreSQLQueryConverter(namespaceName, namespaceID, saTypeMap, saMapper, queryString) + return newPostgreSQLQueryConverter(namespaceName, namespaceID, saTypeMap, saMapper, queryString, chasmMapper, archetypeID) case sqlite.PluginName: - return newSqliteQueryConverter(namespaceName, namespaceID, saTypeMap, saMapper, queryString) + return newSqliteQueryConverter(namespaceName, namespaceID, saTypeMap, saMapper, queryString, chasmMapper, archetypeID) default: return nil } diff --git a/common/persistence/visibility/store/sql/query_converter_legacy_mysql.go b/common/persistence/visibility/store/sql/query_converter_legacy_mysql.go index d81b98e6732..833d4c58f46 100644 --- a/common/persistence/visibility/store/sql/query_converter_legacy_mysql.go +++ b/common/persistence/visibility/store/sql/query_converter_legacy_mysql.go @@ -6,6 +6,7 @@ import ( "strings" "github.com/temporalio/sqlparser" + "go.temporal.io/server/chasm" "go.temporal.io/server/common/namespace" "go.temporal.io/server/common/persistence/sql/sqlplugin" "go.temporal.io/server/common/persistence/visibility/store/query" @@ -64,6 +65,8 @@ func newMySQLQueryConverter( saTypeMap searchattribute.NameTypeMap, saMapper searchattribute.Mapper, queryString string, + chasmMapper *chasm.VisibilitySearchAttributesMapper, + archetypeID chasm.ArchetypeID, ) *QueryConverterLegacy { return newQueryConverterInternal( &mysqlQueryConverter{}, @@ -72,6 +75,8 @@ func newMySQLQueryConverter( saTypeMap, saMapper, queryString, + chasmMapper, + archetypeID, ) } diff --git a/common/persistence/visibility/store/sql/query_converter_legacy_postgresql.go b/common/persistence/visibility/store/sql/query_converter_legacy_postgresql.go index 1ce0b124980..7c67a06214f 100644 --- a/common/persistence/visibility/store/sql/query_converter_legacy_postgresql.go +++ b/common/persistence/visibility/store/sql/query_converter_legacy_postgresql.go @@ -5,6 +5,7 @@ import ( "strings" "github.com/temporalio/sqlparser" + "go.temporal.io/server/chasm" "go.temporal.io/server/common/namespace" "go.temporal.io/server/common/persistence/sql/sqlplugin" "go.temporal.io/server/common/persistence/visibility/store/query" @@ -45,6 +46,8 @@ func newPostgreSQLQueryConverter( saTypeMap searchattribute.NameTypeMap, saMapper searchattribute.Mapper, queryString string, + chasmMapper *chasm.VisibilitySearchAttributesMapper, + archetypeID chasm.ArchetypeID, ) *QueryConverterLegacy { return newQueryConverterInternal( &pgQueryConverter{}, @@ -53,6 +56,8 @@ func newPostgreSQLQueryConverter( saTypeMap, saMapper, queryString, + chasmMapper, + archetypeID, ) } diff --git a/common/persistence/visibility/store/sql/query_converter_legacy_sqlite.go b/common/persistence/visibility/store/sql/query_converter_legacy_sqlite.go index ade2c03e5c4..732171ef977 100644 --- a/common/persistence/visibility/store/sql/query_converter_legacy_sqlite.go +++ b/common/persistence/visibility/store/sql/query_converter_legacy_sqlite.go @@ -5,6 +5,7 @@ import ( "strings" "github.com/temporalio/sqlparser" + "go.temporal.io/server/chasm" "go.temporal.io/server/common/namespace" "go.temporal.io/server/common/persistence/sql/sqlplugin" "go.temporal.io/server/common/persistence/visibility/store/query" @@ -29,6 +30,8 @@ func newSqliteQueryConverter( saTypeMap searchattribute.NameTypeMap, saMapper searchattribute.Mapper, queryString string, + chasmMapper *chasm.VisibilitySearchAttributesMapper, + archetypeID chasm.ArchetypeID, ) *QueryConverterLegacy { return newQueryConverterInternal( &sqliteQueryConverter{}, @@ -37,6 +40,8 @@ func newSqliteQueryConverter( saTypeMap, saMapper, queryString, + chasmMapper, + archetypeID, ) } diff --git a/common/persistence/visibility/store/sql/query_converter_legacy_test.go b/common/persistence/visibility/store/sql/query_converter_legacy_test.go index a343cc47ad8..bb370f482f5 100644 --- a/common/persistence/visibility/store/sql/query_converter_legacy_test.go +++ b/common/persistence/visibility/store/sql/query_converter_legacy_test.go @@ -11,6 +11,7 @@ import ( "github.com/temporalio/sqlparser" enumspb "go.temporal.io/api/enums/v1" "go.temporal.io/api/serviceerror" + "go.temporal.io/server/chasm" "go.temporal.io/server/common/namespace" "go.temporal.io/server/common/persistence/visibility/store/query" "go.temporal.io/server/common/primitives" @@ -52,6 +53,8 @@ func (s *queryConverterSuite) SetupTest() { searchattribute.TestNameTypeMap(), &searchattribute.TestMapper{}, "", + nil, + chasm.UnspecifiedArchetypeID, ) } @@ -146,6 +149,8 @@ func (s *queryConverterSuite) TestConvertWhereString() { searchattribute.TestNameTypeMap(), &searchattribute.TestMapper{}, "", + nil, + chasm.UnspecifiedArchetypeID, ) qp, err := qc.convertWhereString(tc.input) if tc.err == nil { @@ -1086,3 +1091,184 @@ func (m *FlexibleMapper) GetAlias(fieldName, ns string) (string, error) { func (m *FlexibleMapper) GetFieldName(alias, ns string) (string, error) { return m.GetFieldNameFunc(alias, ns) } + +func (s *queryConverterSuite) TestConvertColName_WithChasmMapper() { + chasmMapper := chasm.NewTestVisibilitySearchAttributesMapper( + map[string]string{ + "TemporalBool01": "ChasmCompleted", + "TemporalKeyword01": "ChasmStatus", + }, + map[string]enumspb.IndexedValueType{ + "TemporalBool01": enumspb.INDEXED_VALUE_TYPE_BOOL, + "TemporalKeyword01": enumspb.INDEXED_VALUE_TYPE_KEYWORD, + }, + ) + + qc := newQueryConverterInternal( + s.pqc, + testNamespaceName, + testNamespaceID, + searchattribute.TestNameTypeMap(), + &searchattribute.TestMapper{}, + "", + chasmMapper, + chasm.UnspecifiedArchetypeID, + ) + + testCases := []struct { + name string + input string + expectedFieldName string + expectedFieldType enumspb.IndexedValueType + expectedErr bool + }{ + { + name: "ChasmCompleted", + input: "ChasmCompleted", + expectedFieldName: "TemporalBool01", + expectedFieldType: enumspb.INDEXED_VALUE_TYPE_BOOL, + expectedErr: false, + }, + { + name: "ChasmStatus", + input: "ChasmStatus", + expectedFieldName: "TemporalKeyword01", + expectedFieldType: enumspb.INDEXED_VALUE_TYPE_KEYWORD, + expectedErr: false, + }, + { + name: "NonExistentChasmAlias", + input: "NonExistentChasmAlias", + expectedFieldName: "", + expectedFieldType: enumspb.INDEXED_VALUE_TYPE_UNSPECIFIED, + expectedErr: true, + }, + } + + for _, tc := range testCases { + s.Run(tc.name, func() { + sql := fmt.Sprintf("select * from table1 where %s = 'value'", tc.input) + stmt, err := sqlparser.Parse(sql) + s.NoError(err) + expr := stmt.(*sqlparser.Select).Where.Expr.(*sqlparser.ComparisonExpr).Left + colName, err := qc.convertColName(&expr) + if tc.expectedErr { + s.Error(err) + } else { + s.NoError(err) + s.Equal(tc.expectedFieldName, colName.fieldName) + s.Equal(tc.expectedFieldType, colName.valueType) + } + }) + } +} + +func (s *queryConverterSuite) TestConvertWhereString_WithChasmMapper() { + chasmMapper := chasm.NewTestVisibilitySearchAttributesMapper( + map[string]string{ + "TemporalKeyword01": "ChasmStatus", + }, + map[string]enumspb.IndexedValueType{ + "TemporalKeyword01": enumspb.INDEXED_VALUE_TYPE_KEYWORD, + }, + ) + + qc := newQueryConverterInternal( + s.pqc, + testNamespaceName, + testNamespaceID, + searchattribute.TestNameTypeMap(), + &searchattribute.TestMapper{}, + "", + chasmMapper, + chasm.UnspecifiedArchetypeID, + ) + + testCases := []struct { + name string + input string + expected string + err error + }{ + { + name: "CHASM search attribute", + input: "ChasmStatus = 'active'", + expected: "(TemporalKeyword01 = 'active') and TemporalNamespaceDivision is null", + err: nil, + }, + { + name: "CHASM search attribute with namespace division", + input: "ChasmStatus = 'active' AND TemporalNamespaceDivision = '123'", + expected: "(TemporalKeyword01 = 'active' and TemporalNamespaceDivision = '123')", + err: nil, + }, + } + + for _, tc := range testCases { + s.Run(tc.name, func() { + qp, err := qc.convertWhereString(tc.input) + if tc.err == nil { + s.NoError(err) + s.Equal(tc.expected, qp.queryString) + } else { + s.Error(err) + s.Equal(tc.err, err) + } + }) + } +} + +func (s *queryConverterSuite) TestConvertWhereString_WithArchetypeID() { + testCases := []struct { + name string + input string + archetypeID chasm.ArchetypeID + expected string + err error + }{ + { + name: "with archetypeID", + input: "AliasForInt01 = 1", + archetypeID: 123, + expected: "(Int01 = 1) and TemporalNamespaceDivision = '123'", + err: nil, + }, + { + name: "with UnspecifiedArchetypeID", + input: "AliasForInt01 = 1", + archetypeID: chasm.UnspecifiedArchetypeID, + expected: "(Int01 = 1) and TemporalNamespaceDivision is null", + err: nil, + }, + { + name: "with explicit namespace division and archetypeID", + input: "AliasForInt01 = 1 AND TemporalNamespaceDivision = '456'", + archetypeID: 123, + expected: "(Int01 = 1 and TemporalNamespaceDivision = '456')", + err: nil, + }, + } + + for _, tc := range testCases { + s.Run(tc.name, func() { + qc := newQueryConverterInternal( + s.pqc, + testNamespaceName, + testNamespaceID, + searchattribute.TestNameTypeMap(), + &searchattribute.TestMapper{}, + "", + nil, + tc.archetypeID, + ) + qp, err := qc.convertWhereString(tc.input) + if tc.err == nil { + s.NoError(err) + s.Equal(tc.expected, qp.queryString) + } else { + s.Error(err) + s.Equal(tc.err, err) + } + }) + } +} diff --git a/common/persistence/visibility/store/sql/visibility_store.go b/common/persistence/visibility/store/sql/visibility_store.go index 25acdf74e1a..8af02133e41 100644 --- a/common/persistence/visibility/store/sql/visibility_store.go +++ b/common/persistence/visibility/store/sql/visibility_store.go @@ -4,6 +4,7 @@ import ( "context" "errors" "fmt" + "maps" "time" "github.com/temporalio/sqlparser" @@ -11,6 +12,7 @@ import ( enumspb "go.temporal.io/api/enums/v1" "go.temporal.io/api/serviceerror" "go.temporal.io/api/workflowservice/v1" + "go.temporal.io/server/chasm" "go.temporal.io/server/common/config" "go.temporal.io/server/common/dynamicconfig" "go.temporal.io/server/common/log" @@ -32,9 +34,20 @@ type ( sqlStore persistencesql.SqlStore searchAttributesProvider searchattribute.Provider searchAttributesMapperProvider searchattribute.MapperProvider + chasmRegistry *chasm.Registry enableUnifiedQueryConverter dynamicconfig.BoolPropertyFn } + + listExecutionsRequestInternal struct { + NamespaceID namespace.ID + Namespace namespace.Name + Query string + PageSize int + NextPageToken []byte + ArchetypeID chasm.ArchetypeID + ChasmMapper *chasm.VisibilitySearchAttributesMapper + } ) var _ store.VisibilityStore = (*VisibilityStore)(nil) @@ -47,6 +60,7 @@ func NewSQLVisibilityStore( r resolver.ServiceResolver, searchAttributesProvider searchattribute.Provider, searchAttributesMapperProvider searchattribute.MapperProvider, + chasmRegistry *chasm.Registry, enableUnifiedQueryConverter dynamicconfig.BoolPropertyFn, logger log.Logger, metricsHandler metrics.Handler, @@ -60,6 +74,7 @@ func NewSQLVisibilityStore( sqlStore: persistencesql.NewSqlStore(db, logger), searchAttributesProvider: searchAttributesProvider, searchAttributesMapperProvider: searchAttributesMapperProvider, + chasmRegistry: chasmRegistry, enableUnifiedQueryConverter: enableUnifiedQueryConverter, }, nil @@ -175,15 +190,88 @@ func (s *VisibilityStore) ListWorkflowExecutions( return s.listWorkflowExecutionsLegacy(ctx, request) } -func (s *VisibilityStore) listWorkflowExecutions( +func (s *VisibilityStore) ListChasmExecutions( ctx context.Context, - request *manager.ListWorkflowExecutionsRequestV2, + request *manager.ListChasmExecutionsRequest, ) (*store.InternalListWorkflowExecutionsResponse, error) { + rc, ok := s.chasmRegistry.ComponentByID(request.ArchetypeID) + if !ok { + return nil, serviceerror.NewInvalidArgument(fmt.Sprintf("unknown archetype ID: %d", request.ArchetypeID)) + } + mapper := rc.SearchAttributesMapper() + + requestInternal := &listExecutionsRequestInternal{ + NamespaceID: request.NamespaceID, + Namespace: request.Namespace, + Query: request.Query, + PageSize: request.PageSize, + NextPageToken: request.NextPageToken, + ChasmMapper: mapper, + ArchetypeID: request.ArchetypeID, + } + + if s.enableUnifiedQueryConverter() { + return s.listExecutionsInternal(ctx, requestInternal) + } + + return s.listExecutionsInternalLegacy(ctx, requestInternal) +} + +func (s *VisibilityStore) CountChasmExecutions( + ctx context.Context, + request *manager.CountChasmExecutionsRequest, +) (*manager.CountChasmExecutionsResponse, error) { + rc, ok := s.chasmRegistry.ComponentByID(request.ArchetypeID) + if !ok { + return nil, serviceerror.NewInvalidArgument(fmt.Sprintf("unknown archetype ID: %d", request.ArchetypeID)) + } + mapper := rc.SearchAttributesMapper() + + if s.enableUnifiedQueryConverter() { + return s.countChasmExecutions(ctx, request, mapper) + } + return s.countChasmExecutionsLegacy(ctx, request, mapper) +} + +func (s *VisibilityStore) countChasmExecutions( + ctx context.Context, + request *manager.CountChasmExecutionsRequest, + mapper *chasm.VisibilitySearchAttributesMapper, +) (*manager.CountChasmExecutionsResponse, error) { sqlQC, err := NewSQLQueryConverter(s.GetName()) if err != nil { return nil, err } + queryParams, err := s.buildQueryParams(request.NamespaceID, request.Namespace, request.Query, mapper, request.ArchetypeID, sqlQC) + if err != nil { + var converterErr *query.ConverterError + if errors.As(err, &converterErr) { + return nil, converterErr.ToInvalidArgument() + } + return nil, err + } + + queryString, queryArgs := sqlQC.BuildCountStmt(queryParams) + selectFilter := &sqlplugin.VisibilitySelectFilter{ + Query: queryString, + QueryArgs: queryArgs, + } + + count, err := s.sqlStore.DB.CountFromVisibility(ctx, *selectFilter) + if err != nil { + return nil, serviceerror.NewUnavailable( + fmt.Sprintf("CountChasmExecutions operation failed. Query failed: %v", err)) + } + + return &manager.CountChasmExecutionsResponse{Count: count}, nil +} + +func (s *VisibilityStore) countChasmExecutionsLegacy( + ctx context.Context, + request *manager.CountChasmExecutionsRequest, + mapper *chasm.VisibilitySearchAttributesMapper, +) (*manager.CountChasmExecutionsResponse, error) { saTypeMap, err := s.searchAttributesProvider.GetSearchAttributes(s.GetIndexName(), false) if err != nil { return nil, err @@ -194,14 +282,57 @@ func (s *VisibilityStore) listWorkflowExecutions( return nil, err } - queryParams, err := buildQueryParams( + converter := NewQueryConverterLegacy( + s.GetName(), request.Namespace, request.NamespaceID, - request.Query, - sqlQC, saTypeMap, saMapper, + request.Query, + mapper, + request.ArchetypeID, ) + selectFilter, err := converter.BuildCountStmt() + if err != nil { + var converterErr *query.ConverterError + if errors.As(err, &converterErr) { + return nil, converterErr.ToInvalidArgument() + } + return nil, err + } + + count, err := s.sqlStore.DB.CountFromVisibility(ctx, *selectFilter) + if err != nil { + return nil, serviceerror.NewUnavailable( + fmt.Sprintf("CountChasmExecutions operation failed. Query failed: %v", err)) + } + + return &manager.CountChasmExecutionsResponse{Count: count}, nil +} + +func (s *VisibilityStore) listWorkflowExecutions( + ctx context.Context, + request *manager.ListWorkflowExecutionsRequestV2, +) (*store.InternalListWorkflowExecutionsResponse, error) { + return s.listExecutionsInternal(ctx, &listExecutionsRequestInternal{ + NamespaceID: request.NamespaceID, + Namespace: request.Namespace, + Query: request.Query, + PageSize: request.PageSize, + NextPageToken: request.NextPageToken, + }) +} + +func (s *VisibilityStore) listExecutionsInternal( + ctx context.Context, + request *listExecutionsRequestInternal, +) (*store.InternalListWorkflowExecutionsResponse, error) { + sqlQC, err := NewSQLQueryConverter(s.GetName()) + if err != nil { + return nil, err + } + + queryParams, err := s.buildQueryParams(request.NamespaceID, request.Namespace, request.Query, request.ChasmMapper, request.ArchetypeID, sqlQC) if err != nil { // Convert ConverterError to InvalidArgument and pass through all other errors (which should be // only mapper errors). @@ -217,9 +348,13 @@ func (s *VisibilityStore) listWorkflowExecutions( return nil, err } - queryString, queryArgs := sqlQC.BuildSelectStmt(queryParams, request.PageSize, pageToken) + sqlQueryString, queryArgs := sqlQC.BuildSelectStmt( + queryParams, + request.PageSize, + pageToken, + ) selectFilter := &sqlplugin.VisibilitySelectFilter{ - Query: queryString, + Query: sqlQueryString, QueryArgs: queryArgs, } @@ -234,20 +369,20 @@ func (s *VisibilityStore) listWorkflowExecutions( var infos = make([]*store.InternalWorkflowExecutionInfo, len(rows)) for i, row := range rows { - infos[i], err = s.rowToInfo(&row, request.Namespace) + infos[i], err = s.rowToInfo(&row, request.Namespace, request.ChasmMapper) if err != nil { return nil, err } } - var nextPageToken []byte + var nextPageTokenResult []byte if len(rows) > 0 && len(rows) == request.PageSize { lastRow := rows[len(rows)-1] closeTime := maxDatetime if lastRow.CloseTime != nil { closeTime = *lastRow.CloseTime } - nextPageToken, err = sqlplugin.SerializeVisibilityPageToken(&sqlplugin.VisibilityPageToken{ + nextPageTokenResult, err = sqlplugin.SerializeVisibilityPageToken(&sqlplugin.VisibilityPageToken{ CloseTime: closeTime, StartTime: lastRow.StartTime, RunID: lastRow.RunID, @@ -258,13 +393,28 @@ func (s *VisibilityStore) listWorkflowExecutions( } return &store.InternalListWorkflowExecutionsResponse{ Executions: infos, - NextPageToken: nextPageToken, + NextPageToken: nextPageTokenResult, }, nil } func (s *VisibilityStore) listWorkflowExecutionsLegacy( ctx context.Context, request *manager.ListWorkflowExecutionsRequestV2, +) (*store.InternalListWorkflowExecutionsResponse, error) { + return s.listExecutionsInternalLegacy(ctx, &listExecutionsRequestInternal{ + NamespaceID: request.NamespaceID, + Namespace: request.Namespace, + Query: request.Query, + PageSize: request.PageSize, + NextPageToken: request.NextPageToken, + ChasmMapper: nil, + ArchetypeID: chasm.UnspecifiedArchetypeID, + }) +} + +func (s *VisibilityStore) listExecutionsInternalLegacy( + ctx context.Context, + request *listExecutionsRequestInternal, ) (*store.InternalListWorkflowExecutionsResponse, error) { saTypeMap, err := s.searchAttributesProvider.GetSearchAttributes(s.GetIndexName(), false) if err != nil { @@ -283,6 +433,8 @@ func (s *VisibilityStore) listWorkflowExecutionsLegacy( saTypeMap, saMapper, request.Query, + request.ChasmMapper, + request.ArchetypeID, ) selectFilter, err := converter.BuildSelectStmt(request.PageSize, request.NextPageToken) if err != nil { @@ -305,7 +457,7 @@ func (s *VisibilityStore) listWorkflowExecutionsLegacy( var infos = make([]*store.InternalWorkflowExecutionInfo, len(rows)) for i, row := range rows { - infos[i], err = s.rowToInfo(&row, request.Namespace) + infos[i], err = s.rowToInfo(&row, request.Namespace, request.ChasmMapper) if err != nil { return nil, err } @@ -364,6 +516,8 @@ func (s *VisibilityStore) countWorkflowExecutionsLegacy( saTypeMap, saMapper, request.Query, + nil, + chasm.UnspecifiedArchetypeID, ) selectFilter, err := converter.BuildCountStmt() if err != nil { @@ -397,24 +551,7 @@ func (s *VisibilityStore) countWorkflowExecutions( return nil, err } - saTypeMap, err := s.searchAttributesProvider.GetSearchAttributes(s.GetIndexName(), false) - if err != nil { - return nil, err - } - - saMapper, err := s.searchAttributesMapperProvider.GetMapper(request.Namespace) - if err != nil { - return nil, err - } - - queryParams, err := buildQueryParams( - request.Namespace, - request.NamespaceID, - request.Query, - sqlQC, - saTypeMap, - saMapper, - ) + queryParams, err := s.buildQueryParams(request.NamespaceID, request.Namespace, request.Query, nil, chasm.UnspecifiedArchetypeID, sqlQC) if err != nil { // Convert ConverterError to InvalidArgument and pass through all other errors (which should be // only mapper errors). @@ -496,6 +633,55 @@ func (s *VisibilityStore) countGroupByWorkflowExecutions( return resp, nil } +func (s *VisibilityStore) buildQueryParams( + namespaceID namespace.ID, + namespaceName namespace.Name, + queryString string, + chasmMapper *chasm.VisibilitySearchAttributesMapper, + archetypeID chasm.ArchetypeID, + sqlQC *SQLQueryConverter, +) (*query.QueryParams[sqlparser.Expr], error) { + saTypeMap, err := s.searchAttributesProvider.GetSearchAttributes(s.GetIndexName(), false) + if err != nil { + return nil, err + } + + saMapper, err := s.searchAttributesMapperProvider.GetMapper(namespaceName) + if err != nil { + return nil, err + } + + c := query.NewQueryConverter(sqlQC, namespaceName, saTypeMap, saMapper). + WithChasmMapper(chasmMapper). + WithArchetypeID(archetypeID) + + queryParams, err := c.Convert(queryString) + if err != nil { + return nil, err + } + + nsFilterExpr, err := sqlQC.ConvertComparisonExpr( + sqlparser.EqualStr, + query.NamespaceIDSAColumn, + namespaceID.String(), + ) + if err != nil { + return nil, err + } + + queryParams.QueryExpr, err = sqlQC.BuildAndExpr(nsFilterExpr, queryParams.QueryExpr) + if err != nil { + return nil, err + } + + // ORDER BY is not support in SQL visibility store + if len(queryParams.OrderBy) > 0 { + return nil, query.NewConverterError("%s: 'ORDER BY' clause", query.NotSupportedErrMessage) + } + + return queryParams, nil +} + func (s *VisibilityStore) GetWorkflowExecution( ctx context.Context, request *manager.GetWorkflowExecutionRequest, @@ -508,7 +694,7 @@ func (s *VisibilityStore) GetWorkflowExecution( return nil, serviceerror.NewUnavailable( fmt.Sprintf("GetWorkflowExecution operation failed. Select failed: %v", err)) } - info, err := s.rowToInfo(row, request.Namespace) + info, err := s.rowToInfo(row, request.Namespace, nil) if err != nil { return nil, err } @@ -552,10 +738,7 @@ func (s *VisibilityStore) prepareSearchAttributesForDb( return nil, nil } - saTypeMap, err := s.searchAttributesProvider.GetSearchAttributes( - s.GetIndexName(), - false, - ) + saTypeMap, err := s.searchAttributesProvider.GetSearchAttributes(s.GetIndexName(), false) if err != nil { return nil, serviceerror.NewUnavailable( fmt.Sprintf("Unable to read search attributes types: %v", err)) @@ -587,6 +770,7 @@ func (s *VisibilityStore) prepareSearchAttributesForDb( func (s *VisibilityStore) rowToInfo( row *sqlplugin.VisibilityRow, nsName namespace.Name, + chasmMapper *chasm.VisibilitySearchAttributesMapper, ) (*store.InternalWorkflowExecutionInfo, error) { if row.ExecutionTime.UnixNano() == 0 { row.ExecutionTime = row.StartTime @@ -604,11 +788,13 @@ func (s *VisibilityStore) rowToInfo( Memo: persistence.NewDataBlob(row.Memo, row.Encoding), } if row.SearchAttributes != nil && len(*row.SearchAttributes) > 0 { - searchAttributes, err := s.processRowSearchAttributes(*row.SearchAttributes, nsName) + // Always separate CHASM and custom search attributes + chasmSAs, customSAs, err := s.processRowSearchAttributes(*row.SearchAttributes, nsName, chasmMapper) if err != nil { return nil, err } - info.SearchAttributes = searchAttributes + info.ChasmSearchAttributes = chasmSAs + info.SearchAttributes = customSAs } if row.CloseTime != nil { info.CloseTime = *row.CloseTime @@ -637,26 +823,29 @@ func (s *VisibilityStore) rowToInfo( func (s *VisibilityStore) processRowSearchAttributes( rowSearchAttributes sqlplugin.VisibilitySearchAttributes, nsName namespace.Name, -) (*commonpb.SearchAttributes, error) { - saTypeMap, err := s.searchAttributesProvider.GetSearchAttributes( - s.GetIndexName(), - false, - ) + chasmMapper *chasm.VisibilitySearchAttributesMapper, +) (map[string]chasm.VisibilityValue, *commonpb.SearchAttributes, error) { + saTypeMap, err := s.searchAttributesProvider.GetSearchAttributes(s.GetIndexName(), false) if err != nil { - return nil, serviceerror.NewUnavailable( + return nil, nil, serviceerror.NewUnavailable( fmt.Sprintf("Unable to read search attributes types: %v", err)) } - // In SQLite, keyword list can return a string when there's only one element. - // This changes it into a slice. + + // Build combined type map for SQLite keyword list fix + combinedTypeMap := make(map[string]enumspb.IndexedValueType) + maps.Copy(combinedTypeMap, saTypeMap.Custom()) + maps.Copy(combinedTypeMap, chasmMapper.SATypeMap()) + finalTypeMap := searchattribute.NewNameTypeMap(combinedTypeMap) + + // Split row search attributes into CHASM and custom based on field name pattern + chasmRowSAs := sqlplugin.VisibilitySearchAttributes{} + customRowSAs := sqlplugin.VisibilitySearchAttributes{} + for name, value := range rowSearchAttributes { - // TODO: CHASM search attributes are not in the typeMap and SQL only stores raw values (no metadata). - // The Encode() call below will fail to add type metadata, causing decode issues. - if sadefs.IsChasmSearchAttribute(name) { - continue - } - tp, err := saTypeMap.GetType(name) + // Fix SQLite keyword list handling (convert string to []string for keyword lists) + tp, err := finalTypeMap.GetType(name) if err != nil { - return nil, err + return nil, nil, err } if tp == enumspb.INDEXED_VALUE_TYPE_KEYWORD_LIST { switch v := value.(type) { @@ -665,25 +854,38 @@ func (s *VisibilityStore) processRowSearchAttributes( case string: (rowSearchAttributes)[name] = []string{v} default: - return nil, serviceerror.NewInternal( + return nil, nil, serviceerror.NewInternal( fmt.Sprintf("Unexpected data type for keyword list: %T (expected list of strings)", v), ) } } + + if sadefs.IsChasmSearchAttribute(name) { + chasmRowSAs[name] = value + } else { + customRowSAs[name] = value + } } - searchAttributes, err := searchattribute.Encode(rowSearchAttributes, &saTypeMap) + + chasmSAs, err := chasm.AliasChasmSearchAttributes(chasmRowSAs, chasmMapper) if err != nil { - return nil, err + return nil, nil, err } - aliasedSas, err := searchattribute.AliasFields( + + customSAs, err := searchattribute.Encode(customRowSAs, &saTypeMap) + if err != nil { + return nil, nil, err + } + customSAs, err = searchattribute.AliasFields( s.searchAttributesMapperProvider, - searchAttributes, + customSAs, nsName.String(), ) if err != nil { - return nil, err + return nil, nil, err } - return aliasedSas, nil + + return chasmSAs, customSAs, nil } func (s *VisibilityStore) AddSearchAttributes( diff --git a/common/persistence/visibility/store/visibility_store.go b/common/persistence/visibility/store/visibility_store.go index d5ff78403c8..f21f497c623 100644 --- a/common/persistence/visibility/store/visibility_store.go +++ b/common/persistence/visibility/store/visibility_store.go @@ -9,6 +9,7 @@ import ( commonpb "go.temporal.io/api/common/v1" enumspb "go.temporal.io/api/enums/v1" + "go.temporal.io/server/chasm" "go.temporal.io/server/common/persistence" "go.temporal.io/server/common/persistence/visibility/manager" ) @@ -33,7 +34,9 @@ type ( // Read APIs. ListWorkflowExecutions(ctx context.Context, request *manager.ListWorkflowExecutionsRequestV2) (*InternalListWorkflowExecutionsResponse, error) + ListChasmExecutions(ctx context.Context, request *manager.ListChasmExecutionsRequest) (*InternalListWorkflowExecutionsResponse, error) CountWorkflowExecutions(ctx context.Context, request *manager.CountWorkflowExecutionsRequest) (*manager.CountWorkflowExecutionsResponse, error) + CountChasmExecutions(ctx context.Context, request *manager.CountChasmExecutionsRequest) (*manager.CountChasmExecutionsResponse, error) GetWorkflowExecution(ctx context.Context, request *manager.GetWorkflowExecutionRequest) (*InternalGetWorkflowExecutionResponse, error) // Admin APIs @@ -44,29 +47,30 @@ type ( AddSearchAttributes(ctx context.Context, request *manager.AddSearchAttributesRequest) error } - // InternalWorkflowExecutionInfo is visibility info for internal response + // InternalWorkflowExecutionInfo is internal visibility info for workflow execution InternalWorkflowExecutionInfo struct { - WorkflowID string - RunID string - TypeName string - StartTime time.Time - ExecutionTime time.Time - CloseTime time.Time - ExecutionDuration time.Duration - Status enumspb.WorkflowExecutionStatus - HistoryLength int64 - HistorySizeBytes int64 - StateTransitionCount int64 - Memo *commonpb.DataBlob - TaskQueue string - SearchAttributes *commonpb.SearchAttributes - ParentWorkflowID string - ParentRunID string - RootWorkflowID string - RootRunID string + WorkflowID string + RunID string + TypeName string + StartTime time.Time + ExecutionTime time.Time + CloseTime time.Time + ExecutionDuration time.Duration + Status enumspb.WorkflowExecutionStatus + HistoryLength int64 + HistorySizeBytes int64 + StateTransitionCount int64 + Memo *commonpb.DataBlob + TaskQueue string + ChasmSearchAttributes map[string]chasm.VisibilityValue + SearchAttributes *commonpb.SearchAttributes + ParentWorkflowID string + ParentRunID string + RootWorkflowID string + RootRunID string } - // InternalListWorkflowExecutionsResponse is response from ListWorkflowExecutions + // InternalListWorkflowExecutionsResponse is response from ListWorkflowExecutions and ListChasmExecutions InternalListWorkflowExecutionsResponse struct { Executions []*InternalWorkflowExecutionInfo // Token to read next page if there are more workflow executions beyond page size. diff --git a/common/persistence/visibility/store/visibility_store_mock.go b/common/persistence/visibility/store/visibility_store_mock.go index f904006d170..3306917960a 100644 --- a/common/persistence/visibility/store/visibility_store_mock.go +++ b/common/persistence/visibility/store/visibility_store_mock.go @@ -67,6 +67,21 @@ func (mr *MockVisibilityStoreMockRecorder) Close() *gomock.Call { return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "Close", reflect.TypeOf((*MockVisibilityStore)(nil).Close)) } +// CountChasmExecutions mocks base method. +func (m *MockVisibilityStore) CountChasmExecutions(ctx context.Context, request *manager.CountChasmExecutionsRequest) (*manager.CountChasmExecutionsResponse, error) { + m.ctrl.T.Helper() + ret := m.ctrl.Call(m, "CountChasmExecutions", ctx, request) + ret0, _ := ret[0].(*manager.CountChasmExecutionsResponse) + ret1, _ := ret[1].(error) + return ret0, ret1 +} + +// CountChasmExecutions indicates an expected call of CountChasmExecutions. +func (mr *MockVisibilityStoreMockRecorder) CountChasmExecutions(ctx, request any) *gomock.Call { + mr.mock.ctrl.T.Helper() + return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "CountChasmExecutions", reflect.TypeOf((*MockVisibilityStore)(nil).CountChasmExecutions), ctx, request) +} + // CountWorkflowExecutions mocks base method. func (m *MockVisibilityStore) CountWorkflowExecutions(ctx context.Context, request *manager.CountWorkflowExecutionsRequest) (*manager.CountWorkflowExecutionsResponse, error) { m.ctrl.T.Helper() @@ -139,6 +154,21 @@ func (mr *MockVisibilityStoreMockRecorder) GetWorkflowExecution(ctx, request any return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "GetWorkflowExecution", reflect.TypeOf((*MockVisibilityStore)(nil).GetWorkflowExecution), ctx, request) } +// ListChasmExecutions mocks base method. +func (m *MockVisibilityStore) ListChasmExecutions(ctx context.Context, request *manager.ListChasmExecutionsRequest) (*InternalListWorkflowExecutionsResponse, error) { + m.ctrl.T.Helper() + ret := m.ctrl.Call(m, "ListChasmExecutions", ctx, request) + ret0, _ := ret[0].(*InternalListWorkflowExecutionsResponse) + ret1, _ := ret[1].(error) + return ret0, ret1 +} + +// ListChasmExecutions indicates an expected call of ListChasmExecutions. +func (mr *MockVisibilityStoreMockRecorder) ListChasmExecutions(ctx, request any) *gomock.Call { + mr.mock.ctrl.T.Helper() + return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "ListChasmExecutions", reflect.TypeOf((*MockVisibilityStore)(nil).ListChasmExecutions), ctx, request) +} + // ListWorkflowExecutions mocks base method. func (m *MockVisibilityStore) ListWorkflowExecutions(ctx context.Context, request *manager.ListWorkflowExecutionsRequestV2) (*InternalListWorkflowExecutionsResponse, error) { m.ctrl.T.Helper() diff --git a/common/persistence/visibility/visibility_manager_dual.go b/common/persistence/visibility/visibility_manager_dual.go index 2d0dc845eeb..9bd009f4d81 100644 --- a/common/persistence/visibility/visibility_manager_dual.go +++ b/common/persistence/visibility/visibility_manager_dual.go @@ -5,6 +5,8 @@ import ( "errors" "sync" + commonpb "go.temporal.io/api/common/v1" + "go.temporal.io/server/chasm" "go.temporal.io/server/common/dynamicconfig" "go.temporal.io/server/common/namespace" "go.temporal.io/server/common/persistence/visibility/manager" @@ -159,6 +161,19 @@ func (v *VisibilityManagerDual) ListWorkflowExecutions( ) } +func (v *VisibilityManagerDual) ListChasmExecutions( + ctx context.Context, + request *manager.ListChasmExecutionsRequest, +) (*chasm.ListExecutionsResponse[*commonpb.Payload], error) { + return dualReadWrapper( + ctx, + v, + request, + request.Namespace, + manager.VisibilityManager.ListChasmExecutions, + ) +} + func (v *VisibilityManagerDual) CountWorkflowExecutions( ctx context.Context, request *manager.CountWorkflowExecutionsRequest, @@ -172,6 +187,19 @@ func (v *VisibilityManagerDual) CountWorkflowExecutions( ) } +func (v *VisibilityManagerDual) CountChasmExecutions( + ctx context.Context, + request *manager.CountChasmExecutionsRequest, +) (*chasm.CountExecutionsResponse, error) { + return dualReadWrapper( + ctx, + v, + request, + request.Namespace, + manager.VisibilityManager.CountChasmExecutions, + ) +} + func (v *VisibilityManagerDual) GetWorkflowExecution( ctx context.Context, request *manager.GetWorkflowExecutionRequest, diff --git a/common/persistence/visibility/visibility_manager_impl.go b/common/persistence/visibility/visibility_manager_impl.go index a0402eb3872..fdc4df07745 100644 --- a/common/persistence/visibility/visibility_manager_impl.go +++ b/common/persistence/visibility/visibility_manager_impl.go @@ -3,14 +3,18 @@ package visibility import ( "context" "fmt" + "strconv" "time" commonpb "go.temporal.io/api/common/v1" enumspb "go.temporal.io/api/enums/v1" "go.temporal.io/api/serviceerror" workflowpb "go.temporal.io/api/workflow/v1" + "go.temporal.io/server/chasm" "go.temporal.io/server/common/log" + "go.temporal.io/server/common/log/tag" "go.temporal.io/server/common/namespace" + "go.temporal.io/server/common/payload" "go.temporal.io/server/common/persistence/serialization" "go.temporal.io/server/common/persistence/visibility/manager" "go.temporal.io/server/common/persistence/visibility/store" @@ -140,6 +144,71 @@ func (p *visibilityManagerImpl) ListWorkflowExecutions( return p.convertInternalListResponse(response) } +func (p *visibilityManagerImpl) ListChasmExecutions( + ctx context.Context, + request *manager.ListChasmExecutionsRequest, +) (*chasm.ListExecutionsResponse[*commonpb.Payload], error) { + response, err := p.store.ListChasmExecutions(ctx, request) + if err != nil { + return nil, err + } + executions := make([]*chasm.ExecutionInfo[*commonpb.Payload], 0, len(response.Executions)) + for _, exec := range response.Executions { + combinedMemo, err := deserializeMemo(exec.Memo) + if err != nil { + return nil, err + } + + var userMemo *commonpb.Memo + var chasmMemoPayload *commonpb.Payload + + if isChasmExecution(exec.SearchAttributes) && combinedMemo != nil { + // Archetype exists - split memo into user and chasm parts + userPayload := combinedMemo.Fields[chasm.UserMemoKey] + if err := payload.Decode(userPayload, &userMemo); err != nil { + p.logger.Error("failed to decode user memo", tag.Error(err)) + userMemo = nil + } + chasmMemoPayload = combinedMemo.Fields[chasm.ChasmMemoKey] + } else { + // Archetype doesn't match or no combined memo - return entire memo as user memo + userMemo = combinedMemo + chasmMemoPayload = nil + } + + executions = append(executions, &chasm.ExecutionInfo[*commonpb.Payload]{ + BusinessID: exec.WorkflowID, + RunID: exec.RunID, + StartTime: exec.StartTime, + CloseTime: exec.CloseTime, + HistoryLength: exec.HistoryLength, + HistorySizeBytes: exec.HistorySizeBytes, + StateTransitionCount: exec.StateTransitionCount, + ChasmSearchAttributes: chasm.NewSearchAttributesMap(exec.ChasmSearchAttributes), + CustomSearchAttributes: exec.SearchAttributes.GetIndexedFields(), + Memo: userMemo, + ChasmMemo: chasmMemoPayload, + }) + } + + return &chasm.ListExecutionsResponse[*commonpb.Payload]{ + Executions: executions, + NextPageToken: response.NextPageToken, + }, nil +} + +func (p *visibilityManagerImpl) CountChasmExecutions( + ctx context.Context, + request *manager.CountChasmExecutionsRequest, +) (*chasm.CountExecutionsResponse, error) { + response, err := p.store.CountChasmExecutions(ctx, request) + if err != nil { + return nil, err + } + + return &chasm.CountExecutionsResponse{Count: response.Count}, nil +} + func (p *visibilityManagerImpl) CountWorkflowExecutions( ctx context.Context, request *manager.CountWorkflowExecutionsRequest, @@ -344,3 +413,15 @@ func serializeMemo(memo *commonpb.Memo) (*commonpb.DataBlob, error) { EncodingType: MemoEncoding, }, nil } + +func isChasmExecution(searchAttributes *commonpb.SearchAttributes) bool { + if archetypePayload, ok := searchAttributes.GetIndexedFields()[sadefs.TemporalNamespaceDivision]; ok { + var archetypeIDStr string + if err := payload.Decode(archetypePayload, &archetypeIDStr); err == nil { + if _, err := strconv.Atoi(archetypeIDStr); err == nil { + return true + } + } + } + return false +} diff --git a/common/persistence/visibility/visibility_manager_rate_limited.go b/common/persistence/visibility/visibility_manager_rate_limited.go index d5f6537217c..811245f3d40 100644 --- a/common/persistence/visibility/visibility_manager_rate_limited.go +++ b/common/persistence/visibility/visibility_manager_rate_limited.go @@ -4,6 +4,8 @@ import ( "context" "time" + commonpb "go.temporal.io/api/common/v1" + "go.temporal.io/server/chasm" "go.temporal.io/server/common/dynamicconfig" "go.temporal.io/server/common/headers" "go.temporal.io/server/common/namespace" @@ -116,6 +118,16 @@ func (m *visibilityManagerRateLimited) ListWorkflowExecutions( return m.delegate.ListWorkflowExecutions(ctx, request) } +func (m *visibilityManagerRateLimited) ListChasmExecutions( + ctx context.Context, + request *manager.ListChasmExecutionsRequest, +) (*chasm.ListExecutionsResponse[*commonpb.Payload], error) { + if ok := allow(ctx, "ListChasmExecutions", m.readRateLimiter); !ok { + return nil, persistence.ErrPersistenceSystemLimitExceeded + } + return m.delegate.ListChasmExecutions(ctx, request) +} + func (m *visibilityManagerRateLimited) CountWorkflowExecutions( ctx context.Context, request *manager.CountWorkflowExecutionsRequest, @@ -126,6 +138,16 @@ func (m *visibilityManagerRateLimited) CountWorkflowExecutions( return m.delegate.CountWorkflowExecutions(ctx, request) } +func (m *visibilityManagerRateLimited) CountChasmExecutions( + ctx context.Context, + request *manager.CountChasmExecutionsRequest, +) (*chasm.CountExecutionsResponse, error) { + if ok := allow(ctx, "CountChasmExecutions", m.readRateLimiter); !ok { + return nil, persistence.ErrPersistenceSystemLimitExceeded + } + return m.delegate.CountChasmExecutions(ctx, request) +} + func (m *visibilityManagerRateLimited) GetWorkflowExecution( ctx context.Context, request *manager.GetWorkflowExecutionRequest, diff --git a/common/persistence/visibility/visiblity_manager_metrics.go b/common/persistence/visibility/visiblity_manager_metrics.go index dc720eddcb9..5e97068bbf6 100644 --- a/common/persistence/visibility/visiblity_manager_metrics.go +++ b/common/persistence/visibility/visiblity_manager_metrics.go @@ -4,7 +4,9 @@ import ( "context" "time" + commonpb "go.temporal.io/api/common/v1" "go.temporal.io/api/serviceerror" + "go.temporal.io/server/chasm" "go.temporal.io/server/common/dynamicconfig" "go.temporal.io/server/common/log" "go.temporal.io/server/common/log/tag" @@ -129,6 +131,24 @@ func (m *visibilityManagerMetrics) ListWorkflowExecutions( return response, m.updateErrorMetric(handler, err) } +func (m *visibilityManagerMetrics) ListChasmExecutions( + ctx context.Context, + request *manager.ListChasmExecutionsRequest, +) (*chasm.ListExecutionsResponse[*commonpb.Payload], error) { + handler, startTime := m.tagScope(metrics.VisibilityPersistenceListChasmExecutionsScope) + response, err := m.delegate.ListChasmExecutions(ctx, request) + elapsed := time.Since(startTime) + if elapsed > m.slowQueryThreshold() { + m.logger.Warn("List query exceeded threshold", + tag.NewDurationTag("duration", elapsed), + tag.NewStringTag("visibility-query", request.Query), + tag.NewStringerTag("namespace", request.Namespace), + ) + } + metrics.VisibilityPersistenceLatency.With(handler).Record(elapsed) + return response, m.updateErrorMetric(handler, err) +} + func (m *visibilityManagerMetrics) CountWorkflowExecutions( ctx context.Context, request *manager.CountWorkflowExecutionsRequest, @@ -139,6 +159,16 @@ func (m *visibilityManagerMetrics) CountWorkflowExecutions( return response, m.updateErrorMetric(handler, err) } +func (m *visibilityManagerMetrics) CountChasmExecutions( + ctx context.Context, + request *manager.CountChasmExecutionsRequest, +) (*chasm.CountExecutionsResponse, error) { + handler, startTime := m.tagScope(metrics.VisibilityPersistenceCountChasmExecutionsScope) + response, err := m.delegate.CountChasmExecutions(ctx, request) + metrics.VisibilityPersistenceLatency.With(handler).Record(time.Since(startTime)) + return response, m.updateErrorMetric(handler, err) +} + func (m *visibilityManagerMetrics) GetWorkflowExecution( ctx context.Context, request *manager.GetWorkflowExecutionRequest, diff --git a/common/searchattribute/encode.go b/common/searchattribute/encode.go index fc12d85cdaa..2865a6c2efb 100644 --- a/common/searchattribute/encode.go +++ b/common/searchattribute/encode.go @@ -29,13 +29,10 @@ func Encode(searchAttributes map[string]interface{}, typeMap *NameTypeMap) (*com saType := enumspb.INDEXED_VALUE_TYPE_UNSPECIFIED if typeMap != nil { saType, err = typeMap.getType(saName, customCategory|predefinedCategory) - if err != nil && !sadefs.IsChasmSearchAttribute(saName) { + if err != nil { lastErr = err continue } - // TODO: CHASM search attributes read from visibility stores (e.g., during queries) - // will not have type metadata set, which may cause issues on the decode path. - // This is acceptable for now as CHASM query support is not yet implemented. sadefs.SetMetadataType(valPayload, saType) } } diff --git a/common/searchattribute/mapper.go b/common/searchattribute/mapper.go index bd2a720756c..27ce0129b68 100644 --- a/common/searchattribute/mapper.go +++ b/common/searchattribute/mapper.go @@ -115,7 +115,7 @@ func (m *mapperProviderImpl) GetMapper(nsName namespace.Name) (Mapper, error) { }, nil } -// AliasFields returns SearchAttributes struct where each search attribute name is replaced with alias. +// AliasFields returns SearchAttributes struct where each custom search attribute name is replaced with alias. // If no replacement where made, it returns nil which means that original SearchAttributes struct should be used. func AliasFields( mapperProvider MapperProvider, diff --git a/common/searchattribute/name_type_map.go b/common/searchattribute/name_type_map.go index 25c3aaa2ee7..2dee541540b 100644 --- a/common/searchattribute/name_type_map.go +++ b/common/searchattribute/name_type_map.go @@ -24,6 +24,11 @@ const ( customCategory ) +var ( + system = sadefs.System() + predefined = sadefs.Predefined() +) + func buildIndexNameTypeMap(indexSearchAttributes map[string]*persistencespb.IndexSearchAttributes) map[string]NameTypeMap { indexNameTypeMap := make(map[string]NameTypeMap, len(indexSearchAttributes)) for indexName, customSearchAttributes := range indexSearchAttributes { @@ -34,9 +39,14 @@ func buildIndexNameTypeMap(indexSearchAttributes map[string]*persistencespb.Inde return indexNameTypeMap } +// NewNameTypeMap creates a new NameTypeMap with the given custom search attributes. +func NewNameTypeMap(customSearchAttributes map[string]enumspb.IndexedValueType) NameTypeMap { + return NameTypeMap{ + customSearchAttributes: customSearchAttributes, + } +} + func (m NameTypeMap) System() map[string]enumspb.IndexedValueType { - system := sadefs.System() - predefined := sadefs.Predefined() allSystem := make(map[string]enumspb.IndexedValueType, len(system)+len(predefined)) maps.Copy(allSystem, system) maps.Copy(allSystem, predefined) @@ -48,8 +58,6 @@ func (m NameTypeMap) Custom() map[string]enumspb.IndexedValueType { } func (m NameTypeMap) All() map[string]enumspb.IndexedValueType { - system := sadefs.System() - predefined := sadefs.Predefined() allSearchAttributes := make(map[string]enumspb.IndexedValueType, len(system)+len(m.customSearchAttributes)+len(predefined)) maps.Copy(allSearchAttributes, system) maps.Copy(allSearchAttributes, predefined) @@ -76,7 +84,6 @@ func (m NameTypeMap) getType(name string, cat category) (enumspb.IndexedValueTyp } } if cat|systemCategory == cat { - system := sadefs.System() if t, isSystem := system[name]; isSystem { return t, nil } diff --git a/service/frontend/fx.go b/service/frontend/fx.go index 0dbfdff5eb5..54dc986900a 100644 --- a/service/frontend/fx.go +++ b/service/frontend/fx.go @@ -6,6 +6,7 @@ import ( "github.com/gorilla/mux" "go.temporal.io/server/api/adminservice/v1" + "go.temporal.io/server/chasm" "go.temporal.io/server/client" "go.temporal.io/server/common" "go.temporal.io/server/common/archiver" @@ -583,6 +584,7 @@ func VisibilityManagerProvider( searchAttributesMapperProvider searchattribute.MapperProvider, saProvider searchattribute.Provider, namespaceRegistry namespace.Registry, + chasmRegistry *chasm.Registry, ) (manager.VisibilityManager, error) { return visibility.NewManager( *persistenceConfig, @@ -592,6 +594,7 @@ func VisibilityManagerProvider( saProvider, searchAttributesMapperProvider, namespaceRegistry, + chasmRegistry, serviceConfig.VisibilityPersistenceMaxReadQPS, serviceConfig.VisibilityPersistenceMaxWriteQPS, serviceConfig.OperatorRPSRatio, diff --git a/service/history/chasm_engine.go b/service/history/chasm_engine.go index a182fc2dbc1..ff1ced07840 100644 --- a/service/history/chasm_engine.go +++ b/service/history/chasm_engine.go @@ -4,7 +4,9 @@ import ( "context" "errors" "fmt" + "reflect" + commonpb "go.temporal.io/api/common/v1" enumspb "go.temporal.io/api/enums/v1" "go.temporal.io/api/serviceerror" enumsspb "go.temporal.io/server/api/enums/v1" @@ -15,6 +17,7 @@ import ( "go.temporal.io/server/common/locks" "go.temporal.io/server/common/namespace" "go.temporal.io/server/common/persistence" + "go.temporal.io/server/common/persistence/visibility/manager" "go.temporal.io/server/common/primitives" "go.temporal.io/server/service/history/api" "go.temporal.io/server/service/history/configs" @@ -32,6 +35,7 @@ type ( shardController shard.Controller registry *chasm.Registry config *configs.Config + visibilityMgr manager.VisibilityManager } newExecutionParams struct { @@ -67,11 +71,13 @@ func newChasmEngine( executionCache cache.Cache, registry *chasm.Registry, config *configs.Config, + visibilityMgr manager.VisibilityManager, ) *ChasmEngine { return &ChasmEngine{ executionCache: executionCache, registry: registry, config: config, + visibilityMgr: visibilityMgr, } } @@ -624,3 +630,47 @@ func (e *ChasmEngine) getExecutionLease( return shardContext, executionLease, err } + +// ListExecutions implements the Engine interface for visibility queries. +func (e *ChasmEngine) ListExecutions( + ctx context.Context, + archetypeType reflect.Type, + request *chasm.ListExecutionsRequest, +) (*chasm.ListExecutionsResponse[*commonpb.Payload], error) { + archetypeID, ok := e.registry.ArchetypeIDOf(archetypeType) + if !ok { + return nil, serviceerror.NewInternal("unknown chasm component type: " + archetypeType.String()) + } + + visReq := &manager.ListChasmExecutionsRequest{ + ArchetypeID: archetypeID, + NamespaceID: namespace.ID(request.NamespaceID), + Namespace: namespace.Name(request.NamespaceName), + PageSize: request.PageSize, + NextPageToken: request.NextPageToken, + Query: request.Query, + } + + return e.visibilityMgr.ListChasmExecutions(ctx, visReq) +} + +// CountExecutions implements the Engine interface for visibility queries. +func (e *ChasmEngine) CountExecutions( + ctx context.Context, + archetypeType reflect.Type, + request *chasm.CountExecutionsRequest, +) (*chasm.CountExecutionsResponse, error) { + archetypeID, ok := e.registry.ArchetypeIDOf(archetypeType) + if !ok { + return nil, serviceerror.NewInternal("unknown chasm component type: " + archetypeType.String()) + } + + visReq := &manager.CountChasmExecutionsRequest{ + ArchetypeID: archetypeID, + NamespaceID: namespace.ID(request.NamespaceID), + Namespace: namespace.Name(request.NamespaceName), + Query: request.Query, + } + + return e.visibilityMgr.CountChasmExecutions(ctx, visReq) +} diff --git a/service/history/chasm_engine_test.go b/service/history/chasm_engine_test.go index 5fc94b69f04..11e38367fe5 100644 --- a/service/history/chasm_engine_test.go +++ b/service/history/chasm_engine_test.go @@ -115,6 +115,7 @@ func (s *chasmEngineSuite) SetupTest() { s.executionCache, s.registry, s.config, + s.mockShard.Resource.VisibilityManager, ) s.engine.SetShardController(s.mockShardController) } @@ -716,9 +717,9 @@ func (l *testComponent) SearchAttributes(_ chasm.Context) []chasm.SearchAttribut } } -func (l *testComponent) Memo(_ chasm.Context) map[string]chasm.VisibilityValue { - return map[string]chasm.VisibilityValue{ - testComponentPausedMemoName: chasm.VisibilityValueBool(l.ActivityInfo.Paused), +func (l *testComponent) Memo(_ chasm.Context) proto.Message { + return &persistencespb.WorkflowExecutionState{ + RunId: l.ActivityInfo.ActivityId, } } diff --git a/service/history/chasm_visibility_test.go b/service/history/chasm_visibility_test.go new file mode 100644 index 00000000000..cc211c70091 --- /dev/null +++ b/service/history/chasm_visibility_test.go @@ -0,0 +1,410 @@ +package history + +import ( + "context" + "reflect" + "testing" + "time" + + "github.com/stretchr/testify/require" + "github.com/stretchr/testify/suite" + commonpb "go.temporal.io/api/common/v1" + persistencespb "go.temporal.io/server/api/persistence/v1" + "go.temporal.io/server/chasm" + "go.temporal.io/server/common/dynamicconfig" + "go.temporal.io/server/common/log" + "go.temporal.io/server/common/namespace" + "go.temporal.io/server/common/payload" + "go.temporal.io/server/common/persistence/visibility/manager" + "go.temporal.io/server/service/history/configs" + historyi "go.temporal.io/server/service/history/interfaces" + "go.temporal.io/server/service/history/tests" + "go.uber.org/mock/gomock" + "google.golang.org/protobuf/proto" +) + +type ( + ChasmVisibilitySuite struct { + *require.Assertions + suite.Suite + controller *gomock.Controller + + registry *chasm.Registry + visibilityManager *manager.MockVisibilityManager + shardContext *historyi.MockShardContext + config *configs.Config + engine *ChasmEngine + } + + // Test component for mocking CHASM components in visibility tests + visibilityTestComponent struct { + chasm.UnimplementedComponent + Data *persistencespb.WorkflowExecutionState + } +) + +var ( + testChasmNamespaceID = namespace.ID("test-chasm-ns-id") + testChasmNamespace = namespace.Name("test-chasm-namespace") + testBusinessID = "test-business-id" + testRunID = "test-run-id" + testComponentStartTime = time.Now().UTC().Truncate(time.Millisecond) + testComponentCloseTime = testComponentStartTime.Add(5 * time.Minute) + + testChasmSA1Name = "chasm_sa1" + testChasmSA2Name = "chasm_sa2" + testCustomSAName = "custom_sa" + + errTestVisibilityError = &testVisibilityError{message: "test visibility error"} +) + +type testVisibilityError struct { + message string +} + +func (e *testVisibilityError) Error() string { + return e.message +} + +func (tc *visibilityTestComponent) LifecycleState(_ chasm.Context) chasm.LifecycleState { + return chasm.LifecycleStateRunning +} + +func TestChasmVisibilitySuite(t *testing.T) { + suite.Run(t, new(ChasmVisibilitySuite)) +} + +func (s *ChasmVisibilitySuite) SetupTest() { + s.Assertions = require.New(s.T()) + s.controller = gomock.NewController(s.T()) + + // Create a test registry with test component + library := chasm.NewMockLibrary(s.controller) + library.EXPECT().Name().Return("TestLibrary").AnyTimes() + library.EXPECT().Components().Return([]*chasm.RegistrableComponent{ + chasm.NewRegistrableComponent[*visibilityTestComponent]( + "TestComponent", + chasm.WithSearchAttributes( + chasm.NewSearchAttributeInt(testChasmSA1Name, chasm.SearchAttributeFieldInt01), + chasm.NewSearchAttributeKeyword(testChasmSA2Name, chasm.SearchAttributeFieldKeyword01), + ), + ), + }).AnyTimes() + library.EXPECT().Tasks().Return(nil).AnyTimes() + + s.registry = chasm.NewRegistry(log.NewNoopLogger()) + err := s.registry.Register(library) + s.NoError(err) + + s.visibilityManager = manager.NewMockVisibilityManager(s.controller) + s.shardContext = historyi.NewMockShardContext(s.controller) + s.shardContext.EXPECT().ChasmRegistry().Return(s.registry).AnyTimes() + + s.config = tests.NewDynamicConfig() + s.config.HistoryMaxPageSize = dynamicconfig.GetIntPropertyFnFilteredByNamespace(1000) + + s.engine = newChasmEngine( + nil, // entityCache not needed for visibility tests + s.registry, + s.config, + s.visibilityManager, + ) +} + +func (s *ChasmVisibilitySuite) TearDownTest() { + s.controller.Finish() +} + +func (s *ChasmVisibilitySuite) TestListRuns_Success() { + ctx := context.Background() + + query := "StartTime > '2024-01-01T00:00:00Z'" + pageSize := 10 + pageToken := []byte("test-token") + + // Create test search attributes + chasmSA1Payload, err := payload.Encode(int64(123)) + s.NoError(err) + customSAPayload, err := payload.Encode("custom-value") + s.NoError(err) + + // Create test memo + memoData := &commonpb.Memo{ + Fields: map[string]*commonpb.Payload{ + "test-memo-key": chasmSA1Payload, + }, + } + + // Create chasm memo + chasmMemoData := &persistencespb.WorkflowExecutionState{ + RunId: testRunID, + } + chasmMemoBytes, err := proto.Marshal(chasmMemoData) + s.NoError(err) + chasmMemoPayload, err := payload.Encode(chasmMemoBytes) + s.NoError(err) + + // Get archetype ID for the test component + archetypeID, ok := s.registry.ArchetypeIDOf(reflect.TypeFor[*visibilityTestComponent]()) + s.True(ok) + + // Create chasm search attributes map + chasmSearchAttributes := chasm.NewSearchAttributesMap(map[string]chasm.VisibilityValue{ + testChasmSA1Name: chasm.VisibilityValueInt64(123), + testChasmSA2Name: chasm.VisibilityValueString("test-value"), + }) + + // Create custom search attributes map + customSearchAttributes := map[string]*commonpb.Payload{ + testCustomSAName: customSAPayload, + } + + // Setup visibility manager mock + expectedRequest := &manager.ListChasmExecutionsRequest{ + ArchetypeID: archetypeID, + NamespaceID: testChasmNamespaceID, + Namespace: testChasmNamespace, + PageSize: pageSize, + NextPageToken: pageToken, + Query: query, + } + + expectedResponse := &chasm.ListExecutionsResponse[*commonpb.Payload]{ + Executions: []*chasm.ExecutionInfo[*commonpb.Payload]{ + { + BusinessID: testBusinessID, + RunID: testRunID, + StartTime: testComponentStartTime, + CloseTime: testComponentCloseTime, + HistoryLength: 100, + HistorySizeBytes: 5000, + StateTransitionCount: 42, + ChasmSearchAttributes: chasmSearchAttributes, + CustomSearchAttributes: customSearchAttributes, + Memo: memoData, + ChasmMemo: chasmMemoPayload, + }, + }, + NextPageToken: []byte("next-token"), + } + + s.visibilityManager.EXPECT(). + ListChasmExecutions(ctx, gomock.Any()). + DoAndReturn(func(_ context.Context, req *manager.ListChasmExecutionsRequest) (*chasm.ListExecutionsResponse[*commonpb.Payload], error) { + s.Equal(expectedRequest.ArchetypeID, req.ArchetypeID) + s.Equal(expectedRequest.NamespaceID, req.NamespaceID) + s.Equal(expectedRequest.Namespace, req.Namespace) + s.Equal(expectedRequest.PageSize, req.PageSize) + s.Equal(expectedRequest.NextPageToken, req.NextPageToken) + s.Equal(expectedRequest.Query, req.Query) + return expectedResponse, nil + }) + + // Call ListExecutions + request := &chasm.ListExecutionsRequest{ + NamespaceID: string(testChasmNamespaceID), + NamespaceName: string(testChasmNamespace), + Query: query, + PageSize: pageSize, + NextPageToken: pageToken, + } + + response, err := s.engine.ListExecutions( + ctx, + reflect.TypeFor[*visibilityTestComponent](), + request, + ) + + // Verify results + s.NoError(err) + s.NotNil(response) + s.Len(response.Executions, 1) + s.Equal([]byte("next-token"), response.NextPageToken) + + execution := response.Executions[0] + s.Equal(testBusinessID, execution.BusinessID) + s.Equal(testRunID, execution.RunID) + s.Equal(testComponentStartTime, execution.StartTime) + s.Equal(testComponentCloseTime, execution.CloseTime) + s.Equal(int64(100), execution.HistoryLength) + s.Equal(int64(5000), execution.HistorySizeBytes) + s.Equal(int64(42), execution.StateTransitionCount) + + // Verify chasm search attributes - check that the map was created correctly + s.NotNil(execution.ChasmSearchAttributes) + // The SearchAttributesMap wraps the values, so we verify it's not empty + // The actual values are verified through the manager response + + // Verify custom search attributes + s.Equal(customSearchAttributes, execution.CustomSearchAttributes) + + // Verify memo + s.Equal(memoData, execution.Memo) + + // Verify chasm memo + s.Equal(chasmMemoPayload, execution.ChasmMemo) +} + +func (s *ChasmVisibilitySuite) TestCountRuns_Success() { + ctx := context.Background() + + query := "StartTime > '2024-01-01T00:00:00Z'" + + // Get archetype ID for the test component + archetypeID, ok := s.registry.ArchetypeIDOf(reflect.TypeFor[*visibilityTestComponent]()) + s.True(ok) + + expectedCount := int64(42) + + // Setup visibility manager mock + expectedRequest := &manager.CountChasmExecutionsRequest{ + ArchetypeID: archetypeID, + NamespaceID: testChasmNamespaceID, + Namespace: testChasmNamespace, + Query: query, + } + + expectedResponse := &chasm.CountExecutionsResponse{ + Count: expectedCount, + } + + s.visibilityManager.EXPECT(). + CountChasmExecutions(ctx, gomock.Any()). + DoAndReturn(func(_ context.Context, req *manager.CountChasmExecutionsRequest) (*chasm.CountExecutionsResponse, error) { + s.Equal(expectedRequest.ArchetypeID, req.ArchetypeID) + s.Equal(expectedRequest.NamespaceID, req.NamespaceID) + s.Equal(expectedRequest.Namespace, req.Namespace) + s.Equal(expectedRequest.Query, req.Query) + return expectedResponse, nil + }) + + // Call CountExecutions + request := &chasm.CountExecutionsRequest{ + NamespaceID: string(testChasmNamespaceID), + NamespaceName: string(testChasmNamespace), + Query: query, + } + + response, err := s.engine.CountExecutions( + ctx, + reflect.TypeFor[*visibilityTestComponent](), + request, + ) + + // Verify results + s.NoError(err) + s.NotNil(response) + s.Equal(expectedCount, response.Count) +} + +func (s *ChasmVisibilitySuite) TestListRuns_InvalidArchetypeType() { + ctx := context.Background() + + // Use an invalid type that's not registered + invalidType := reflect.TypeFor[struct{ Field string }]() + + request := &chasm.ListExecutionsRequest{ + NamespaceID: string(testChasmNamespaceID), + NamespaceName: string(testChasmNamespace), + Query: "StartTime > '2024-01-01T00:00:00Z'", + } + + response, err := s.engine.ListExecutions( + ctx, + invalidType, + request, + ) + + s.Error(err) + s.Nil(response) +} + +func (s *ChasmVisibilitySuite) TestCountRuns_InvalidArchetypeType() { + ctx := context.Background() + + // Use an invalid type that's not registered + invalidType := reflect.TypeFor[struct{ Field string }]() + + request := &chasm.CountExecutionsRequest{ + NamespaceID: string(testChasmNamespaceID), + NamespaceName: string(testChasmNamespace), + Query: "StartTime > '2024-01-01T00:00:00Z'", + } + + response, err := s.engine.CountExecutions( + ctx, + invalidType, + request, + ) + + s.Error(err) + s.Nil(response) +} + +func (s *ChasmVisibilitySuite) TestListRuns_VisibilityManagerError() { + ctx := context.Background() + + query := "StartTime > '2024-01-01T00:00:00Z'" + + // Get archetype ID for the test component + archetypeID, ok := s.registry.ArchetypeIDOf(reflect.TypeFor[*visibilityTestComponent]()) + s.True(ok) + + // Setup visibility manager mock to return an error + s.visibilityManager.EXPECT(). + ListChasmExecutions(ctx, gomock.Any()). + DoAndReturn(func(_ context.Context, req *manager.ListChasmExecutionsRequest) (*chasm.ListExecutionsResponse[*commonpb.Payload], error) { + s.Equal(archetypeID, req.ArchetypeID) + return nil, errTestVisibilityError + }) + + request := &chasm.ListExecutionsRequest{ + NamespaceID: string(testChasmNamespaceID), + NamespaceName: string(testChasmNamespace), + Query: query, + } + + response, err := s.engine.ListExecutions( + ctx, + reflect.TypeFor[*visibilityTestComponent](), + request, + ) + + s.Error(err) + s.Nil(response) + s.Equal(errTestVisibilityError, err) +} + +func (s *ChasmVisibilitySuite) TestCountRuns_VisibilityManagerError() { + ctx := context.Background() + + query := "StartTime > '2024-01-01T00:00:00Z'" + + // Get archetype ID for the test component + archetypeID, ok := s.registry.ArchetypeIDOf(reflect.TypeFor[*visibilityTestComponent]()) + s.True(ok) + + // Setup visibility manager mock to return an error + s.visibilityManager.EXPECT(). + CountChasmExecutions(ctx, gomock.Any()). + DoAndReturn(func(_ context.Context, req *manager.CountChasmExecutionsRequest) (*chasm.CountExecutionsResponse, error) { + s.Equal(archetypeID, req.ArchetypeID) + return nil, errTestVisibilityError + }) + + request := &chasm.CountExecutionsRequest{ + NamespaceID: string(testChasmNamespaceID), + NamespaceName: string(testChasmNamespace), + Query: query, + } + + response, err := s.engine.CountExecutions( + ctx, + reflect.TypeFor[*visibilityTestComponent](), + request, + ) + + s.Error(err) + s.Nil(response) + s.Equal(errTestVisibilityError, err) +} diff --git a/service/history/fx.go b/service/history/fx.go index b42a7968889..32db310a947 100644 --- a/service/history/fx.go +++ b/service/history/fx.go @@ -291,6 +291,7 @@ func VisibilityManagerProvider( searchAttributesMapperProvider searchattribute.MapperProvider, saProvider searchattribute.Provider, namespaceRegistry namespace.Registry, + chasmRegistry *chasm.Registry, ) (manager.VisibilityManager, error) { return visibility.NewManager( *persistenceConfig, @@ -300,6 +301,7 @@ func VisibilityManagerProvider( saProvider, searchAttributesMapperProvider, namespaceRegistry, + chasmRegistry, serviceConfig.VisibilityPersistenceMaxReadQPS, serviceConfig.VisibilityPersistenceMaxWriteQPS, serviceConfig.OperatorRPSRatio, diff --git a/service/history/visibility_queue_task_executor.go b/service/history/visibility_queue_task_executor.go index 77ef9e5e14f..bb1cabffdb2 100644 --- a/service/history/visibility_queue_task_executor.go +++ b/service/history/visibility_queue_task_executor.go @@ -13,6 +13,7 @@ import ( "go.temporal.io/server/common/definition" "go.temporal.io/server/common/dynamicconfig" "go.temporal.io/server/common/log" + "go.temporal.io/server/common/log/tag" "go.temporal.io/server/common/metrics" "go.temporal.io/server/common/namespace" "go.temporal.io/server/common/payload" @@ -405,20 +406,15 @@ func (t *visibilityQueueTaskExecutor) processChasmTask( searchattributes := make(map[string]*commonpb.Payload) aliasedSearchAttributes := visComponent.GetSearchAttributes(visTaskContext) - for alias, value := range aliasedSearchAttributes { fieldName, err := searchAttributesMapper.GetFieldName(alias, namespaceEntry.Name().String()) if err != nil { - return err + t.logger.Warn("Failed to get field name for alias, ignoring search attribute", tag.NewStringTag("alias", alias), tag.Error(err)) + continue } searchattributes[fieldName] = value } - memo := visComponent.GetMemo(visTaskContext) - if memo == nil { - memo = make(map[string]*commonpb.Payload) - } - rootComponent, err := tree.ComponentByPath(visTaskContext, nil) if err != nil { return err @@ -428,9 +424,25 @@ func (t *visibilityQueueTaskExecutor) processChasmTask( searchattributes[sa.Field] = sa.Value.MustEncode() } } + + combinedMemo := make(map[string]*commonpb.Payload, 2) + userMemoMap := visComponent.GetMemo(visTaskContext) + if len(userMemoMap) > 0 { + userMemoProto := &commonpb.Memo{Fields: userMemoMap} + userMemoPayload, err := payload.Encode(userMemoProto) + if err != nil { + return err + } + combinedMemo[chasm.UserMemoKey] = userMemoPayload + } if memoProvider, ok := rootComponent.(chasm.VisibilityMemoProvider); ok { - for key, value := range memoProvider.Memo(visTaskContext) { - memo[key] = value.MustEncode() + chasmMemo := memoProvider.Memo(visTaskContext) + if chasmMemo != nil { + chasmMemoPayload, err := payload.Encode(chasmMemo) + if err != nil { + return err + } + combinedMemo[chasm.ChasmMemoKey] = chasmMemoPayload } } @@ -438,7 +450,7 @@ func (t *visibilityQueueTaskExecutor) processChasmTask( task, namespaceEntry, mutableState, - memo, + combinedMemo, searchattributes, ) diff --git a/service/history/visibility_queue_task_executor_test.go b/service/history/visibility_queue_task_executor_test.go index 8cc57ae07f5..4b60e45742f 100644 --- a/service/history/visibility_queue_task_executor_test.go +++ b/service/history/visibility_queue_task_executor_test.go @@ -641,9 +641,16 @@ func (s *visibilityQueueTaskExecutorSuite) TestProcessChasmTask_RunningExecution s.True(paused) s.Len(request.Memo.Fields, 1) - err = payload.Decode(request.Memo.Fields[testComponentPausedSAName], &paused) + + // Memo should contain "__chasm__" key with encoded proto message + chasmMemoPayload, ok := request.Memo.Fields[chasm.ChasmMemoKey] + s.True(ok, "Expected %s key in memo", chasm.ChasmMemoKey) + + // Decode the chasm memo proto message + var chasmMemoProto persistencespb.WorkflowExecutionState + err = payload.Decode(chasmMemoPayload, &chasmMemoProto) s.NoError(err) - s.True(paused) + s.NotEmpty(chasmMemoProto.RunId) return nil }, @@ -730,7 +737,10 @@ func (s *visibilityQueueTaskExecutorSuite) buildChasmMutableState( }, }, }, - Data: newTestComponentStateBlob(&persistencespb.ActivityInfo{Paused: true}), + Data: newTestComponentStateBlob(&persistencespb.ActivityInfo{ + Paused: true, + ActivityId: key.RunID, + }), }, "Visibility": { Metadata: &persistencespb.ChasmNodeMetadata{ diff --git a/service/matching/fx.go b/service/matching/fx.go index a9346c501ef..af16f4fb369 100644 --- a/service/matching/fx.go +++ b/service/matching/fx.go @@ -1,6 +1,7 @@ package matching import ( + "go.temporal.io/server/chasm" "go.temporal.io/server/common" "go.temporal.io/server/common/cluster" "go.temporal.io/server/common/config" @@ -161,6 +162,7 @@ func VisibilityManagerProvider( searchAttributesMapperProvider searchattribute.MapperProvider, saProvider searchattribute.Provider, namespaceRegistry namespace.Registry, + chasmRegistry *chasm.Registry, ) (manager.VisibilityManager, error) { return visibility.NewManager( *persistenceConfig, @@ -170,6 +172,7 @@ func VisibilityManagerProvider( saProvider, searchAttributesMapperProvider, namespaceRegistry, + chasmRegistry, serviceConfig.VisibilityPersistenceMaxReadQPS, serviceConfig.VisibilityPersistenceMaxWriteQPS, serviceConfig.OperatorRPSRatio, diff --git a/service/worker/fx.go b/service/worker/fx.go index e9f40f5ba57..083337d88c2 100644 --- a/service/worker/fx.go +++ b/service/worker/fx.go @@ -5,6 +5,7 @@ import ( "os" "go.temporal.io/server/api/adminservice/v1" + "go.temporal.io/server/chasm" "go.temporal.io/server/client" "go.temporal.io/server/common/cluster" "go.temporal.io/server/common/config" @@ -142,6 +143,7 @@ func VisibilityManagerProvider( searchAttributesMapperProvider searchattribute.MapperProvider, saProvider searchattribute.Provider, namespaceRegistry namespace.Registry, + chasmRegistry *chasm.Registry, ) (manager.VisibilityManager, error) { return visibility.NewManager( *persistenceConfig, @@ -151,6 +153,7 @@ func VisibilityManagerProvider( saProvider, searchAttributesMapperProvider, namespaceRegistry, + chasmRegistry, serviceConfig.VisibilityPersistenceMaxReadQPS, serviceConfig.VisibilityPersistenceMaxWriteQPS, serviceConfig.OperatorRPSRatio, diff --git a/service/worker/scheduler/query.go b/service/worker/scheduler/query.go index e7dfdeeaadd..1e6c9ed26c7 100644 --- a/service/worker/scheduler/query.go +++ b/service/worker/scheduler/query.go @@ -34,7 +34,7 @@ func newFieldNameAggInterceptor( saMapperProvider searchattribute.MapperProvider, ) *fieldNameAggInterceptor { return &fieldNameAggInterceptor{ - baseInterceptor: elasticsearch.NewNameInterceptor(namespaceName, saNameType, saMapperProvider), + baseInterceptor: elasticsearch.NewNameInterceptor(namespaceName, saNameType, saMapperProvider, nil), names: make(map[string]bool), } } @@ -125,7 +125,7 @@ func getQueryFieldsLegacy( queryString string, ) ([]string, error) { fnInterceptor := newFieldNameAggInterceptor(namespaceName, saNameType, saMapperProvider) - queryConverter := elasticsearch.NewQueryConverterLegacy(fnInterceptor, nil, saNameType) + queryConverter := elasticsearch.NewQueryConverterLegacy(fnInterceptor, nil, saNameType, nil) _, err := queryConverter.ConvertWhereOrderBy(queryString) if err != nil { var converterErr *query.ConverterError