Conformance test: move loader to each component type's folder (#3162)

Signed-off-by: ItalyPaleAle <43508+ItalyPaleAle@users.noreply.github.com>
This commit is contained in:
Alessandro (Ale) Segala 2023-10-03 16:13:28 -07:00 committed by GitHub
parent 5a00d1c210
commit 1208b3e3c0
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
11 changed files with 521 additions and 456 deletions

View File

@ -15,6 +15,7 @@ run:
# list of build tags, all linters use it. Default is empty list.
build-tags:
- certtests
- conftests
- metadata
# which dirs to skip: they won't be analyzed;
@ -301,3 +302,4 @@ linters:
- deadcode
- nosnakecase
- varcheck
- goconst

View File

@ -4,15 +4,15 @@
1. `tests/` directory contains the configuration and the test definition for conformance tests.
2. All the conformance tests are within the `tests/conformance` directory.
3. All the configurations are in the `tests/config` directory.
4. Each of the component specific `component` definition are in their specific `component type` folder in the `tests/config` folder. For example, the `redis` statestore component definition within `state` directory.
- The component types are: `bindings`, `configuration`, `crypto`, `pubsub`, `state`, `secretstores`, `workflows`.
- Cloud specific components will be within their own `cloud` directory within the `component type` folder, e.g. `pubsub/azure/servicebus`.
3. All the components' configurations are in the `tests/config` directory.
4. Each of the component specific component's definition are in their specific `<component type>` folder in the `tests/config` folder. For example, the SQLite state store component definition is in the `tests/config/state/sqlite/` directory.
- The component types are: `bindings`, `configuration`, `crypto`, `lock`, `pubsub`, `state`, `secretstores`, `workflows`.
- Cloud-specific components will be within their own `<cloud>` directory within the `<component type>` folder, e.g. `tests/config/pubsub/azure/servicebus`.
- If a component has multiple variants, they are defined in sub-folders. For example, for the MySQL state store, `tests/config/state/mysql/mysql` contains the configuration for testing against MySQL, while `tests/config/state/mysql/mariadb` uses MariaDB.
5. Similar to the component definitions, each component type has its own set of the conformance tests definitions.
6. Each `component type` contains a `tests.yml` definition that defines the component to be tested along with component specific test configuration. Nested folder names have their `/` in path replaced by `.` in the component name in `tests.yml`, e.g. `azure/servicebus/topics` should be `azure.servicebus.topics`
7. All the tests configurations are defined in `common.go` file.
8. Each `component type` has its own `_test` file to trigger the conformance tests. E.g. `bindings_test.go`.
9. Each test added will also need to be added to the `component type/tests.yml` workflow file.
6. Each component type contains a `tests.yml` definition that defines the component to be tested along with component-specific test configuration. Nested folder names have their `/` in path replaced by `.` in the component name in `tests.yml`, e.g. `azure/servicebus/topics` should be `azure.servicebus.topics`
7. Each component type has its own `_test` file to trigger the conformance tests, e.g. `bindings_test.go`. This file contains also the test configurations.
9. Each test added will also need to be added to the `<component type>/tests.yml` workflow file.
## Conformance test workflow
@ -24,21 +24,21 @@
1. Add the component specific YAML to `tests/config/<COMPONENT-TYPE>/<COMPONENT>/<FILE>.yaml`.
2. All passwords will be of the form `${{PASSWORD_KEY}}` so that it is injected via environment variables.
3. Register the component `New**` function in `common.go`. For example:
3. Register the component `New**` function in `<COMPONENT-TYPE>_test.go`. For example:
```go
...
switch tc.Component {
switch name {
case "azure.servicebusqueues":
binding = b_azure_servicebusqueues.NewAzureServiceBusQueues(testLogger)
return b_azure_servicebusqueues.NewAzureServiceBusQueues(testLogger)
case "azure.storagequeues":
binding = b_azure_storagequeues.NewAzureStorageQueues(testLogger)
return b_azure_storagequeues.NewAzureStorageQueues(testLogger)
case "azure.eventgrid":
binding = b_azure_eventgrid.NewAzureEventGrid(testLogger)
return b_azure_eventgrid.NewAzureEventGrid(testLogger)
case "kafka":
binding = b_kafka.NewKafka(testLogger)
return b_kafka.NewKafka(testLogger)
case "new-component":
binding = b_new_component.NewComponent(testLogger)
return b_new_component.NewComponent(testLogger)
default:
return nil
}

View File

@ -17,14 +17,123 @@ limitations under the License.
package conformance
import (
"path/filepath"
"testing"
"github.com/stretchr/testify/require"
"github.com/dapr/components-contrib/bindings"
b_aws_s3 "github.com/dapr/components-contrib/bindings/aws/s3"
b_azure_blobstorage "github.com/dapr/components-contrib/bindings/azure/blobstorage"
b_azure_cosmosdb "github.com/dapr/components-contrib/bindings/azure/cosmosdb"
b_azure_eventgrid "github.com/dapr/components-contrib/bindings/azure/eventgrid"
b_azure_eventhubs "github.com/dapr/components-contrib/bindings/azure/eventhubs"
b_azure_servicebusqueues "github.com/dapr/components-contrib/bindings/azure/servicebusqueues"
b_azure_storagequeues "github.com/dapr/components-contrib/bindings/azure/storagequeues"
b_cron "github.com/dapr/components-contrib/bindings/cron"
b_http "github.com/dapr/components-contrib/bindings/http"
b_influx "github.com/dapr/components-contrib/bindings/influx"
b_kafka "github.com/dapr/components-contrib/bindings/kafka"
b_kubemq "github.com/dapr/components-contrib/bindings/kubemq"
b_mqtt3 "github.com/dapr/components-contrib/bindings/mqtt3"
b_postgres "github.com/dapr/components-contrib/bindings/postgres"
b_rabbitmq "github.com/dapr/components-contrib/bindings/rabbitmq"
b_redis "github.com/dapr/components-contrib/bindings/redis"
conf_bindings "github.com/dapr/components-contrib/tests/conformance/bindings"
)
func TestBindingsConformance(t *testing.T) {
tc, err := NewTestConfiguration("../config/bindings/tests.yml")
const configPath = "../config/bindings/"
tc, err := NewTestConfiguration(filepath.Join(configPath, "tests.yml"))
require.NoError(t, err)
require.NotNil(t, tc)
tc.TestFn = func(comp *TestComponent) func(t *testing.T) {
return func(t *testing.T) {
ParseConfigurationMap(t, comp.Config)
componentConfigPath := convertComponentNameToPath(comp.Component, comp.Profile)
props, err := loadComponentsAndProperties(t, filepath.Join(configPath, componentConfigPath))
require.NoErrorf(t, err, "error running conformance test for component %s", comp.Component)
inputBinding := loadInputBindings(comp.Component)
outputBinding := loadOutputBindings(comp.Component)
require.True(t, inputBinding != nil || outputBinding != nil)
bindingsConfig, err := conf_bindings.NewTestConfig(comp.Component, comp.Operations, comp.Config)
require.NoErrorf(t, err, "error running conformance test for component %s", comp.Component)
conf_bindings.ConformanceTests(t, props, inputBinding, outputBinding, bindingsConfig)
}
}
tc.Run(t)
}
func loadOutputBindings(name string) bindings.OutputBinding {
switch name {
case "redis.v6":
return b_redis.NewRedis(testLogger)
case "redis.v7":
return b_redis.NewRedis(testLogger)
case "azure.blobstorage":
return b_azure_blobstorage.NewAzureBlobStorage(testLogger)
case "azure.storagequeues":
return b_azure_storagequeues.NewAzureStorageQueues(testLogger)
case "azure.servicebusqueues":
return b_azure_servicebusqueues.NewAzureServiceBusQueues(testLogger)
case "azure.eventgrid":
return b_azure_eventgrid.NewAzureEventGrid(testLogger)
case "azure.eventhubs":
return b_azure_eventhubs.NewAzureEventHubs(testLogger)
case "azure.cosmosdb":
return b_azure_cosmosdb.NewCosmosDB(testLogger)
case "kafka":
return b_kafka.NewKafka(testLogger)
case "http":
return b_http.NewHTTP(testLogger)
case "influx":
return b_influx.NewInflux(testLogger)
case "mqtt3":
return b_mqtt3.NewMQTT(testLogger)
case "rabbitmq":
return b_rabbitmq.NewRabbitMQ(testLogger)
case "kubemq":
return b_kubemq.NewKubeMQ(testLogger)
case "postgresql.docker":
return b_postgres.NewPostgres(testLogger)
case "postgresql.azure":
return b_postgres.NewPostgres(testLogger)
case "aws.s3.docker":
return b_aws_s3.NewAWSS3(testLogger)
case "aws.s3.terraform":
return b_aws_s3.NewAWSS3(testLogger)
default:
return nil
}
}
func loadInputBindings(name string) bindings.InputBinding {
switch name {
case "azure.servicebusqueues":
return b_azure_servicebusqueues.NewAzureServiceBusQueues(testLogger)
case "azure.storagequeues":
return b_azure_storagequeues.NewAzureStorageQueues(testLogger)
case "azure.eventgrid":
return b_azure_eventgrid.NewAzureEventGrid(testLogger)
case "cron":
return b_cron.NewCron(testLogger)
case "azure.eventhubs":
return b_azure_eventhubs.NewAzureEventHubs(testLogger)
case "kafka":
return b_kafka.NewKafka(testLogger)
case "mqtt3":
return b_mqtt3.NewMQTT(testLogger)
case "rabbitmq":
return b_rabbitmq.NewRabbitMQ(testLogger)
case "kubemq":
return b_kubemq.NewKubeMQ(testLogger)
default:
return nil
}
}

View File

@ -33,95 +33,10 @@ import (
"github.com/stretchr/testify/require"
"gopkg.in/yaml.v3"
"github.com/dapr/components-contrib/bindings"
"github.com/dapr/components-contrib/configuration"
contribCrypto "github.com/dapr/components-contrib/crypto"
"github.com/dapr/components-contrib/lock"
"github.com/dapr/components-contrib/pubsub"
"github.com/dapr/components-contrib/secretstores"
"github.com/dapr/components-contrib/state"
"github.com/dapr/components-contrib/workflows"
"github.com/dapr/kit/logger"
b_aws_s3 "github.com/dapr/components-contrib/bindings/aws/s3"
b_azure_blobstorage "github.com/dapr/components-contrib/bindings/azure/blobstorage"
b_azure_cosmosdb "github.com/dapr/components-contrib/bindings/azure/cosmosdb"
b_azure_eventgrid "github.com/dapr/components-contrib/bindings/azure/eventgrid"
b_azure_eventhubs "github.com/dapr/components-contrib/bindings/azure/eventhubs"
b_azure_servicebusqueues "github.com/dapr/components-contrib/bindings/azure/servicebusqueues"
b_azure_storagequeues "github.com/dapr/components-contrib/bindings/azure/storagequeues"
b_cron "github.com/dapr/components-contrib/bindings/cron"
b_http "github.com/dapr/components-contrib/bindings/http"
b_influx "github.com/dapr/components-contrib/bindings/influx"
b_kafka "github.com/dapr/components-contrib/bindings/kafka"
b_kubemq "github.com/dapr/components-contrib/bindings/kubemq"
b_mqtt3 "github.com/dapr/components-contrib/bindings/mqtt3"
b_postgres "github.com/dapr/components-contrib/bindings/postgres"
b_rabbitmq "github.com/dapr/components-contrib/bindings/rabbitmq"
b_redis "github.com/dapr/components-contrib/bindings/redis"
c_postgres "github.com/dapr/components-contrib/configuration/postgres"
c_redis "github.com/dapr/components-contrib/configuration/redis"
cr_azurekeyvault "github.com/dapr/components-contrib/crypto/azure/keyvault"
cr_jwks "github.com/dapr/components-contrib/crypto/jwks"
cr_localstorage "github.com/dapr/components-contrib/crypto/localstorage"
l_redis "github.com/dapr/components-contrib/lock/redis"
p_snssqs "github.com/dapr/components-contrib/pubsub/aws/snssqs"
p_eventhubs "github.com/dapr/components-contrib/pubsub/azure/eventhubs"
p_servicebusqueues "github.com/dapr/components-contrib/pubsub/azure/servicebus/queues"
p_servicebustopics "github.com/dapr/components-contrib/pubsub/azure/servicebus/topics"
p_gcppubsub "github.com/dapr/components-contrib/pubsub/gcp/pubsub"
p_inmemory "github.com/dapr/components-contrib/pubsub/in-memory"
p_jetstream "github.com/dapr/components-contrib/pubsub/jetstream"
p_kafka "github.com/dapr/components-contrib/pubsub/kafka"
p_kubemq "github.com/dapr/components-contrib/pubsub/kubemq"
p_mqtt3 "github.com/dapr/components-contrib/pubsub/mqtt3"
p_pulsar "github.com/dapr/components-contrib/pubsub/pulsar"
p_rabbitmq "github.com/dapr/components-contrib/pubsub/rabbitmq"
p_redis "github.com/dapr/components-contrib/pubsub/redis"
p_solaceamqp "github.com/dapr/components-contrib/pubsub/solace/amqp"
ss_azure "github.com/dapr/components-contrib/secretstores/azure/keyvault"
ss_hashicorp_vault "github.com/dapr/components-contrib/secretstores/hashicorp/vault"
ss_kubernetes "github.com/dapr/components-contrib/secretstores/kubernetes"
ss_local_env "github.com/dapr/components-contrib/secretstores/local/env"
ss_local_file "github.com/dapr/components-contrib/secretstores/local/file"
s_awsdynamodb "github.com/dapr/components-contrib/state/aws/dynamodb"
s_blobstorage "github.com/dapr/components-contrib/state/azure/blobstorage"
s_cosmosdb "github.com/dapr/components-contrib/state/azure/cosmosdb"
s_azuretablestorage "github.com/dapr/components-contrib/state/azure/tablestorage"
s_cassandra "github.com/dapr/components-contrib/state/cassandra"
s_cloudflareworkerskv "github.com/dapr/components-contrib/state/cloudflare/workerskv"
s_cockroachdb "github.com/dapr/components-contrib/state/cockroachdb"
s_etcd "github.com/dapr/components-contrib/state/etcd"
s_gcpfirestore "github.com/dapr/components-contrib/state/gcp/firestore"
s_inmemory "github.com/dapr/components-contrib/state/in-memory"
s_memcached "github.com/dapr/components-contrib/state/memcached"
s_mongodb "github.com/dapr/components-contrib/state/mongodb"
s_mysql "github.com/dapr/components-contrib/state/mysql"
s_oracledatabase "github.com/dapr/components-contrib/state/oracledatabase"
s_postgresql "github.com/dapr/components-contrib/state/postgresql"
s_redis "github.com/dapr/components-contrib/state/redis"
s_rethinkdb "github.com/dapr/components-contrib/state/rethinkdb"
s_sqlite "github.com/dapr/components-contrib/state/sqlite"
s_sqlserver "github.com/dapr/components-contrib/state/sqlserver"
conf_bindings "github.com/dapr/components-contrib/tests/conformance/bindings"
conf_configuration "github.com/dapr/components-contrib/tests/conformance/configuration"
conf_crypto "github.com/dapr/components-contrib/tests/conformance/crypto"
conf_lock "github.com/dapr/components-contrib/tests/conformance/lock"
conf_pubsub "github.com/dapr/components-contrib/tests/conformance/pubsub"
conf_secret "github.com/dapr/components-contrib/tests/conformance/secretstores"
conf_state "github.com/dapr/components-contrib/tests/conformance/state"
conf_workflows "github.com/dapr/components-contrib/tests/conformance/workflows"
"github.com/dapr/components-contrib/tests/utils/configupdater"
cu_postgres "github.com/dapr/components-contrib/tests/utils/configupdater/postgres"
cu_redis "github.com/dapr/components-contrib/tests/utils/configupdater/redis"
wf_temporal "github.com/dapr/components-contrib/workflows/temporal"
)
const (
eventhubs = "azure.eventhubs"
redisv6 = "redis.v6"
redisv7 = "redis.v7"
kafka = "kafka"
generateUUID = "$((uuid))"
generateEd25519PrivateKey = "$((ed25519PrivateKey))"
)
@ -137,13 +52,14 @@ func init() {
type TestConfiguration struct {
ComponentType string `yaml:"componentType,omitempty"`
Components []TestComponent `yaml:"components,omitempty"`
TestFn func(comp *TestComponent) func(t *testing.T)
}
type TestComponent struct {
Component string `yaml:"component,omitempty"`
Profile string `yaml:"profile,omitempty"`
Operations []string `yaml:"operations,omitempty"`
Config map[string]interface{} `yaml:"config,omitempty"`
Component string `yaml:"component,omitempty"`
Profile string `yaml:"profile,omitempty"`
Operations []string `yaml:"operations,omitempty"`
Config map[string]any `yaml:"config,omitempty"`
}
// NewTestConfiguration reads the tests.yml and loads the TestConfiguration.
@ -324,10 +240,10 @@ func decodeYaml(b []byte) (TestConfiguration, error) {
return testConfig, nil
}
func (tc *TestConfiguration) loadComponentsAndProperties(t *testing.T, filepath string) (map[string]string, error) {
func loadComponentsAndProperties(t *testing.T, filepath string) (map[string]string, error) {
comps, err := LoadComponents(filepath)
require.NoError(t, err)
require.Equal(t, 1, len(comps)) // We only expect a single component per file
require.Len(t, comps, 1) // We only expect a single component per file
c := comps[0]
props, err := ConvertMetadataToProperties(c.Spec.Metadata)
return props, err
@ -350,351 +266,13 @@ func (tc *TestConfiguration) Run(t *testing.T) {
// Increase verbosity of tests to allow troubleshooting of runs.
testLogger.SetOutputLevel(logger.DebugLevel)
// For each component in the tests file run the conformance test
for _, comp := range tc.Components {
for i := range tc.Components {
comp := tc.Components[i]
testName := comp.Component
if comp.Profile != "" {
testName += "-" + comp.Profile
}
t.Run(testName, func(t *testing.T) {
// Parse and generate any keys
ParseConfigurationMap(t, comp.Config)
componentConfigPath := convertComponentNameToPath(comp.Component, comp.Profile)
switch tc.ComponentType {
case "state":
filepath := fmt.Sprintf("../config/state/%s", componentConfigPath)
props, err := tc.loadComponentsAndProperties(t, filepath)
require.NoErrorf(t, err, "error running conformance test for component %s", comp.Component)
store := loadStateStore(comp)
require.NotNilf(t, store, "error running conformance test for component %s", comp.Component)
storeConfig, err := conf_state.NewTestConfig(comp.Component, comp.Operations, comp.Config)
require.NoErrorf(t, err, "error running conformance test for component %s", comp.Component)
conf_state.ConformanceTests(t, props, store, storeConfig)
case "secretstores":
filepath := fmt.Sprintf("../config/secretstores/%s", componentConfigPath)
props, err := tc.loadComponentsAndProperties(t, filepath)
require.NoErrorf(t, err, "error running conformance test for component %s", comp.Component)
store := loadSecretStore(comp)
require.NotNilf(t, store, "error running conformance test for component %s", comp.Component)
storeConfig := conf_secret.NewTestConfig(comp.Component, comp.Operations)
conf_secret.ConformanceTests(t, props, store, storeConfig)
case "pubsub":
filepath := fmt.Sprintf("../config/pubsub/%s", componentConfigPath)
props, err := tc.loadComponentsAndProperties(t, filepath)
require.NoErrorf(t, err, "error running conformance test for component %s", comp.Component)
pubsub := loadPubSub(comp)
require.NotNil(t, pubsub, "error running conformance test for component %s", comp.Component)
pubsubConfig, err := conf_pubsub.NewTestConfig(comp.Component, comp.Operations, comp.Config)
require.NoErrorf(t, err, "error running conformance test for component %s", comp.Component)
conf_pubsub.ConformanceTests(t, props, pubsub, pubsubConfig)
case "bindings":
filepath := fmt.Sprintf("../config/bindings/%s", componentConfigPath)
props, err := tc.loadComponentsAndProperties(t, filepath)
require.NoErrorf(t, err, "error running conformance test for component %s", comp.Component)
inputBinding := loadInputBindings(comp)
outputBinding := loadOutputBindings(comp)
require.True(t, inputBinding != nil || outputBinding != nil)
bindingsConfig, err := conf_bindings.NewTestConfig(comp.Component, comp.Operations, comp.Config)
require.NoErrorf(t, err, "error running conformance test for component %s", comp.Component)
conf_bindings.ConformanceTests(t, props, inputBinding, outputBinding, bindingsConfig)
case "workflows":
filepath := fmt.Sprintf("../config/workflows/%s", componentConfigPath)
props, err := tc.loadComponentsAndProperties(t, filepath)
require.NoErrorf(t, err, "error running conformance test for component %s", comp.Component)
wf := loadWorkflow(comp)
wfConfig := conf_workflows.NewTestConfig(comp.Component, comp.Operations, comp.Config)
conf_workflows.ConformanceTests(t, props, wf, wfConfig)
case "lock":
filepath := fmt.Sprintf("../config/lock/%s", componentConfigPath)
props, err := tc.loadComponentsAndProperties(t, filepath)
require.NoErrorf(t, err, "error running conformance test for component %s", comp.Component)
component := loadLockStore(comp)
require.NotNil(t, component, "error running conformance test for component %s", comp.Component)
lockConfig, err := conf_lock.NewTestConfig(comp.Component, comp.Operations, comp.Config)
require.NoErrorf(t, err, "error running conformance test for component %s", comp.Component)
conf_lock.ConformanceTests(t, props, component, lockConfig)
case "crypto":
filepath := fmt.Sprintf("../config/crypto/%s", componentConfigPath)
props, err := tc.loadComponentsAndProperties(t, filepath)
require.NoErrorf(t, err, "error running conformance test for component %s", comp.Component)
component := loadCryptoProvider(comp)
require.NotNil(t, component, "error running conformance test for component %s", comp.Component)
cryptoConfig, err := conf_crypto.NewTestConfig(comp.Component, comp.Operations, comp.Config)
require.NoErrorf(t, err, "error running conformance test for component %s", comp.Component)
conf_crypto.ConformanceTests(t, props, component, cryptoConfig)
case "configuration":
filepath := fmt.Sprintf("../config/configuration/%s", componentConfigPath)
props, err := tc.loadComponentsAndProperties(t, filepath)
require.NoErrorf(t, err, "error running conformance test for component %s", comp.Component)
store, updater := loadConfigurationStore(comp)
require.NotNil(t, store, "error running conformance test for component %s", comp.Component)
require.NotNil(t, updater, "error running conformance test for component %s", comp.Component)
configurationConfig := conf_configuration.NewTestConfig(comp.Component, comp.Operations, comp.Config)
conf_configuration.ConformanceTests(t, props, store, updater, configurationConfig, comp.Component)
default:
t.Fatalf("unknown component type %s", tc.ComponentType)
}
})
t.Run(testName, tc.TestFn(&comp))
}
}
func loadConfigurationStore(tc TestComponent) (configuration.Store, configupdater.Updater) {
var store configuration.Store
var updater configupdater.Updater
switch tc.Component {
case redisv6, redisv7:
store = c_redis.NewRedisConfigurationStore(testLogger)
updater = cu_redis.NewRedisConfigUpdater(testLogger)
case "postgresql.docker", "postgresql.azure":
store = c_postgres.NewPostgresConfigurationStore(testLogger)
updater = cu_postgres.NewPostgresConfigUpdater(testLogger)
default:
return nil, nil
}
return store, updater
}
func loadPubSub(tc TestComponent) pubsub.PubSub {
var pubsub pubsub.PubSub
switch tc.Component {
case redisv6:
pubsub = p_redis.NewRedisStreams(testLogger)
case redisv7:
pubsub = p_redis.NewRedisStreams(testLogger)
case eventhubs:
pubsub = p_eventhubs.NewAzureEventHubs(testLogger)
case "azure.servicebus.topics":
pubsub = p_servicebustopics.NewAzureServiceBusTopics(testLogger)
case "azure.servicebus.queues":
pubsub = p_servicebusqueues.NewAzureServiceBusQueues(testLogger)
case "jetstream":
pubsub = p_jetstream.NewJetStream(testLogger)
case kafka:
pubsub = p_kafka.NewKafka(testLogger)
case "pulsar":
pubsub = p_pulsar.NewPulsar(testLogger)
case "mqtt3":
pubsub = p_mqtt3.NewMQTTPubSub(testLogger)
case "rabbitmq":
pubsub = p_rabbitmq.NewRabbitMQ(testLogger)
case "in-memory":
pubsub = p_inmemory.New(testLogger)
case "aws.snssqs.terraform":
pubsub = p_snssqs.NewSnsSqs(testLogger)
case "aws.snssqs.docker":
pubsub = p_snssqs.NewSnsSqs(testLogger)
case "gcp.pubsub.terraform":
pubsub = p_gcppubsub.NewGCPPubSub(testLogger)
case "gcp.pubsub.docker":
pubsub = p_gcppubsub.NewGCPPubSub(testLogger)
case "kubemq":
pubsub = p_kubemq.NewKubeMQ(testLogger)
case "solace.amqp":
pubsub = p_solaceamqp.NewAMQPPubsub(testLogger)
default:
return nil
}
return pubsub
}
func loadSecretStore(tc TestComponent) secretstores.SecretStore {
var store secretstores.SecretStore
switch tc.Component {
case "azure.keyvault.certificate":
store = ss_azure.NewAzureKeyvaultSecretStore(testLogger)
case "azure.keyvault.serviceprincipal":
store = ss_azure.NewAzureKeyvaultSecretStore(testLogger)
case "kubernetes":
store = ss_kubernetes.NewKubernetesSecretStore(testLogger)
case "local.env":
store = ss_local_env.NewEnvSecretStore(testLogger)
case "local.file":
store = ss_local_file.NewLocalSecretStore(testLogger)
case "hashicorp.vault":
store = ss_hashicorp_vault.NewHashiCorpVaultSecretStore(testLogger)
default:
return nil
}
return store
}
func loadCryptoProvider(tc TestComponent) contribCrypto.SubtleCrypto {
var component contribCrypto.SubtleCrypto
switch tc.Component {
case "azure.keyvault":
component = cr_azurekeyvault.NewAzureKeyvaultCrypto(testLogger)
case "localstorage":
component = cr_localstorage.NewLocalStorageCrypto(testLogger)
case "jwks":
component = cr_jwks.NewJWKSCrypto(testLogger)
}
return component
}
func loadLockStore(tc TestComponent) lock.Store {
var component lock.Store
switch tc.Component {
case redisv6:
component = l_redis.NewStandaloneRedisLock(testLogger)
case redisv7:
component = l_redis.NewStandaloneRedisLock(testLogger)
}
return component
}
func loadStateStore(tc TestComponent) state.Store {
var store state.Store
switch tc.Component {
case redisv6:
store = s_redis.NewRedisStateStore(testLogger)
case redisv7:
store = s_redis.NewRedisStateStore(testLogger)
case "azure.blobstorage":
store = s_blobstorage.NewAzureBlobStorageStore(testLogger)
case "azure.cosmosdb":
store = s_cosmosdb.NewCosmosDBStateStore(testLogger)
case "mongodb":
store = s_mongodb.NewMongoDB(testLogger)
case "azure.sql":
store = s_sqlserver.New(testLogger)
case "sqlserver":
store = s_sqlserver.New(testLogger)
case "postgresql.docker":
store = s_postgresql.NewPostgreSQLStateStore(testLogger)
case "postgresql.azure":
store = s_postgresql.NewPostgreSQLStateStore(testLogger)
case "sqlite":
store = s_sqlite.NewSQLiteStateStore(testLogger)
case "mysql.mysql":
store = s_mysql.NewMySQLStateStore(testLogger)
case "mysql.mariadb":
store = s_mysql.NewMySQLStateStore(testLogger)
case "oracledatabase":
store = s_oracledatabase.NewOracleDatabaseStateStore(testLogger)
case "azure.tablestorage.storage":
store = s_azuretablestorage.NewAzureTablesStateStore(testLogger)
case "azure.tablestorage.cosmosdb":
store = s_azuretablestorage.NewAzureTablesStateStore(testLogger)
case "cassandra":
store = s_cassandra.NewCassandraStateStore(testLogger)
case "cloudflare.workerskv":
store = s_cloudflareworkerskv.NewCFWorkersKV(testLogger)
case "cockroachdb":
store = s_cockroachdb.New(testLogger)
case "memcached":
store = s_memcached.NewMemCacheStateStore(testLogger)
case "rethinkdb":
store = s_rethinkdb.NewRethinkDBStateStore(testLogger)
case "in-memory":
store = s_inmemory.NewInMemoryStateStore(testLogger)
case "aws.dynamodb.docker":
store = s_awsdynamodb.NewDynamoDBStateStore(testLogger)
case "aws.dynamodb.terraform":
store = s_awsdynamodb.NewDynamoDBStateStore(testLogger)
case "etcd.v1":
store = s_etcd.NewEtcdStateStoreV1(testLogger)
case "etcd.v2":
store = s_etcd.NewEtcdStateStoreV2(testLogger)
case "gcp.firestore.docker":
store = s_gcpfirestore.NewFirestoreStateStore(testLogger)
case "gcp.firestore.cloud":
store = s_gcpfirestore.NewFirestoreStateStore(testLogger)
default:
return nil
}
return store
}
func loadOutputBindings(tc TestComponent) bindings.OutputBinding {
var binding bindings.OutputBinding
switch tc.Component {
case redisv6:
binding = b_redis.NewRedis(testLogger)
case redisv7:
binding = b_redis.NewRedis(testLogger)
case "azure.blobstorage":
binding = b_azure_blobstorage.NewAzureBlobStorage(testLogger)
case "azure.storagequeues":
binding = b_azure_storagequeues.NewAzureStorageQueues(testLogger)
case "azure.servicebusqueues":
binding = b_azure_servicebusqueues.NewAzureServiceBusQueues(testLogger)
case "azure.eventgrid":
binding = b_azure_eventgrid.NewAzureEventGrid(testLogger)
case eventhubs:
binding = b_azure_eventhubs.NewAzureEventHubs(testLogger)
case "azure.cosmosdb":
binding = b_azure_cosmosdb.NewCosmosDB(testLogger)
case kafka:
binding = b_kafka.NewKafka(testLogger)
case "http":
binding = b_http.NewHTTP(testLogger)
case "influx":
binding = b_influx.NewInflux(testLogger)
case "mqtt3":
binding = b_mqtt3.NewMQTT(testLogger)
case "rabbitmq":
binding = b_rabbitmq.NewRabbitMQ(testLogger)
case "kubemq":
binding = b_kubemq.NewKubeMQ(testLogger)
case "postgresql.docker":
binding = b_postgres.NewPostgres(testLogger)
case "postgresql.azure":
binding = b_postgres.NewPostgres(testLogger)
case "aws.s3.docker":
binding = b_aws_s3.NewAWSS3(testLogger)
case "aws.s3.terraform":
binding = b_aws_s3.NewAWSS3(testLogger)
default:
return nil
}
return binding
}
func loadInputBindings(tc TestComponent) bindings.InputBinding {
var binding bindings.InputBinding
switch tc.Component {
case "azure.servicebusqueues":
binding = b_azure_servicebusqueues.NewAzureServiceBusQueues(testLogger)
case "azure.storagequeues":
binding = b_azure_storagequeues.NewAzureStorageQueues(testLogger)
case "azure.eventgrid":
binding = b_azure_eventgrid.NewAzureEventGrid(testLogger)
case "cron":
binding = b_cron.NewCron(testLogger)
case eventhubs:
binding = b_azure_eventhubs.NewAzureEventHubs(testLogger)
case kafka:
binding = b_kafka.NewKafka(testLogger)
case "mqtt3":
binding = b_mqtt3.NewMQTT(testLogger)
case "rabbitmq":
binding = b_rabbitmq.NewRabbitMQ(testLogger)
case "kubemq":
binding = b_kubemq.NewKubeMQ(testLogger)
default:
return nil
}
return binding
}
func loadWorkflow(tc TestComponent) workflows.Workflow {
var wf workflows.Workflow
switch tc.Component {
case "temporal":
wf = wf_temporal.NewTemporalWorkflow(testLogger)
default:
return nil
}
return wf
}

View File

@ -17,14 +17,53 @@ limitations under the License.
package conformance
import (
"path/filepath"
"testing"
"github.com/stretchr/testify/require"
"github.com/dapr/components-contrib/configuration"
c_postgres "github.com/dapr/components-contrib/configuration/postgres"
c_redis "github.com/dapr/components-contrib/configuration/redis"
conf_configuration "github.com/dapr/components-contrib/tests/conformance/configuration"
"github.com/dapr/components-contrib/tests/utils/configupdater"
cu_postgres "github.com/dapr/components-contrib/tests/utils/configupdater/postgres"
cu_redis "github.com/dapr/components-contrib/tests/utils/configupdater/redis"
)
func TestConfigurationConformance(t *testing.T) {
tc, err := NewTestConfiguration("../config/configuration/tests.yml")
const configPath = "../config/configuration/"
tc, err := NewTestConfiguration(filepath.Join(configPath, "tests.yml"))
require.NoError(t, err)
require.NotNil(t, tc)
tc.Run(t)
tc.TestFn = func(comp *TestComponent) func(t *testing.T) {
return func(t *testing.T) {
ParseConfigurationMap(t, comp.Config)
componentConfigPath := convertComponentNameToPath(comp.Component, comp.Profile)
props, err := loadComponentsAndProperties(t, filepath.Join(configPath, componentConfigPath))
require.NoErrorf(t, err, "error running conformance test for component %s", comp.Component)
store, updater := loadConfigurationStore(comp.Component)
require.NotNil(t, store, "error running conformance test for component %s", comp.Component)
require.NotNil(t, updater, "error running conformance test for component %s", comp.Component)
configurationConfig := conf_configuration.NewTestConfig(comp.Component, comp.Operations, comp.Config)
conf_configuration.ConformanceTests(t, props, store, updater, configurationConfig, comp.Component)
}
}
}
func loadConfigurationStore(name string) (configuration.Store, configupdater.Updater) {
switch name {
case "redis.v6", "redis.v7":
return c_redis.NewRedisConfigurationStore(testLogger),
cu_redis.NewRedisConfigUpdater(testLogger)
case "postgresql.docker", "postgresql.azure":
return c_postgres.NewPostgresConfigurationStore(testLogger),
cu_postgres.NewPostgresConfigUpdater(testLogger)
default:
return nil, nil
}
}

View File

@ -17,14 +17,54 @@ limitations under the License.
package conformance
import (
"path/filepath"
"testing"
"github.com/stretchr/testify/require"
contribCrypto "github.com/dapr/components-contrib/crypto"
cr_azurekeyvault "github.com/dapr/components-contrib/crypto/azure/keyvault"
cr_jwks "github.com/dapr/components-contrib/crypto/jwks"
cr_localstorage "github.com/dapr/components-contrib/crypto/localstorage"
conf_crypto "github.com/dapr/components-contrib/tests/conformance/crypto"
)
func TestCryptoConformance(t *testing.T) {
tc, err := NewTestConfiguration("../config/crypto/tests.yml")
const configPath = "../config/crypto/"
tc, err := NewTestConfiguration(filepath.Join(configPath, "tests.yml"))
require.NoError(t, err)
require.NotNil(t, tc)
tc.TestFn = func(comp *TestComponent) func(t *testing.T) {
return func(t *testing.T) {
ParseConfigurationMap(t, comp.Config)
componentConfigPath := convertComponentNameToPath(comp.Component, comp.Profile)
props, err := loadComponentsAndProperties(t, filepath.Join(configPath, componentConfigPath))
require.NoErrorf(t, err, "error running conformance test for component %s", comp.Component)
component := loadCryptoProvider(comp.Component)
require.NotNil(t, component, "error running conformance test for component %s", comp.Component)
cryptoConfig, err := conf_crypto.NewTestConfig(comp.Component, comp.Operations, comp.Config)
require.NoErrorf(t, err, "error running conformance test for component %s", comp.Component)
conf_crypto.ConformanceTests(t, props, component, cryptoConfig)
}
}
tc.Run(t)
}
func loadCryptoProvider(name string) contribCrypto.SubtleCrypto {
switch name {
case "azure.keyvault":
return cr_azurekeyvault.NewAzureKeyvaultCrypto(testLogger)
case "localstorage":
return cr_localstorage.NewLocalStorageCrypto(testLogger)
case "jwks":
return cr_jwks.NewJWKSCrypto(testLogger)
default:
return nil
}
}

View File

@ -17,14 +17,50 @@ limitations under the License.
package conformance
import (
"path/filepath"
"testing"
"github.com/stretchr/testify/require"
"github.com/dapr/components-contrib/lock"
l_redis "github.com/dapr/components-contrib/lock/redis"
conf_lock "github.com/dapr/components-contrib/tests/conformance/lock"
)
func TestLockConformance(t *testing.T) {
tc, err := NewTestConfiguration("../config/lock/tests.yml")
const configPath = "../config/lock/"
tc, err := NewTestConfiguration(filepath.Join(configPath, "tests.yml"))
require.NoError(t, err)
require.NotNil(t, tc)
tc.TestFn = func(comp *TestComponent) func(t *testing.T) {
return func(t *testing.T) {
ParseConfigurationMap(t, comp.Config)
componentConfigPath := convertComponentNameToPath(comp.Component, comp.Profile)
props, err := loadComponentsAndProperties(t, filepath.Join(configPath, componentConfigPath))
require.NoErrorf(t, err, "error running conformance test for component %s", comp.Component)
component := loadLockStore(comp.Component)
require.NotNil(t, component, "error running conformance test for component %s", comp.Component)
lockConfig, err := conf_lock.NewTestConfig(comp.Component, comp.Operations, comp.Config)
require.NoErrorf(t, err, "error running conformance test for component %s", comp.Component)
conf_lock.ConformanceTests(t, props, component, lockConfig)
}
}
tc.Run(t)
}
func loadLockStore(name string) lock.Store {
switch name {
case "redis.v6":
return l_redis.NewStandaloneRedisLock(testLogger)
case "redis.v7":
return l_redis.NewStandaloneRedisLock(testLogger)
default:
return nil
}
}

View File

@ -17,14 +17,93 @@ limitations under the License.
package conformance
import (
"path/filepath"
"testing"
"github.com/stretchr/testify/require"
"github.com/dapr/components-contrib/pubsub"
p_snssqs "github.com/dapr/components-contrib/pubsub/aws/snssqs"
p_eventhubs "github.com/dapr/components-contrib/pubsub/azure/eventhubs"
p_servicebusqueues "github.com/dapr/components-contrib/pubsub/azure/servicebus/queues"
p_servicebustopics "github.com/dapr/components-contrib/pubsub/azure/servicebus/topics"
p_gcppubsub "github.com/dapr/components-contrib/pubsub/gcp/pubsub"
p_inmemory "github.com/dapr/components-contrib/pubsub/in-memory"
p_jetstream "github.com/dapr/components-contrib/pubsub/jetstream"
p_kafka "github.com/dapr/components-contrib/pubsub/kafka"
p_kubemq "github.com/dapr/components-contrib/pubsub/kubemq"
p_mqtt3 "github.com/dapr/components-contrib/pubsub/mqtt3"
p_pulsar "github.com/dapr/components-contrib/pubsub/pulsar"
p_rabbitmq "github.com/dapr/components-contrib/pubsub/rabbitmq"
p_redis "github.com/dapr/components-contrib/pubsub/redis"
p_solaceamqp "github.com/dapr/components-contrib/pubsub/solace/amqp"
conf_pubsub "github.com/dapr/components-contrib/tests/conformance/pubsub"
)
func TestPubsubConformance(t *testing.T) {
tc, err := NewTestConfiguration("../config/pubsub/tests.yml")
const configPath = "../config/pubsub/"
tc, err := NewTestConfiguration(filepath.Join(configPath, "tests.yml"))
require.NoError(t, err)
require.NotNil(t, tc)
tc.TestFn = func(comp *TestComponent) func(t *testing.T) {
return func(t *testing.T) {
ParseConfigurationMap(t, comp.Config)
componentConfigPath := convertComponentNameToPath(comp.Component, comp.Profile)
props, err := loadComponentsAndProperties(t, filepath.Join(configPath, componentConfigPath))
require.NoErrorf(t, err, "error running conformance test for component %s", comp.Component)
pubsub := loadPubSub(comp.Component)
require.NotNil(t, pubsub, "error running conformance test for component %s", comp.Component)
pubsubConfig, err := conf_pubsub.NewTestConfig(comp.Component, comp.Operations, comp.Config)
require.NoErrorf(t, err, "error running conformance test for component %s", comp.Component)
conf_pubsub.ConformanceTests(t, props, pubsub, pubsubConfig)
}
}
tc.Run(t)
}
func loadPubSub(name string) pubsub.PubSub {
switch name {
case "redis.v6":
return p_redis.NewRedisStreams(testLogger)
case "redis.v7":
return p_redis.NewRedisStreams(testLogger)
case "azure.eventhubs":
return p_eventhubs.NewAzureEventHubs(testLogger)
case "azure.servicebus.topics":
return p_servicebustopics.NewAzureServiceBusTopics(testLogger)
case "azure.servicebus.queues":
return p_servicebusqueues.NewAzureServiceBusQueues(testLogger)
case "jetstream":
return p_jetstream.NewJetStream(testLogger)
case "kafka":
return p_kafka.NewKafka(testLogger)
case "pulsar":
return p_pulsar.NewPulsar(testLogger)
case "mqtt3":
return p_mqtt3.NewMQTTPubSub(testLogger)
case "rabbitmq":
return p_rabbitmq.NewRabbitMQ(testLogger)
case "in-memory":
return p_inmemory.New(testLogger)
case "aws.snssqs.terraform":
return p_snssqs.NewSnsSqs(testLogger)
case "aws.snssqs.docker":
return p_snssqs.NewSnsSqs(testLogger)
case "gcp.pubsub.terraform":
return p_gcppubsub.NewGCPPubSub(testLogger)
case "gcp.pubsub.docker":
return p_gcppubsub.NewGCPPubSub(testLogger)
case "kubemq":
return p_kubemq.NewKubeMQ(testLogger)
case "solace.amqp":
return p_solaceamqp.NewAMQPPubsub(testLogger)
default:
return nil
}
}

View File

@ -17,14 +17,61 @@ limitations under the License.
package conformance
import (
"path/filepath"
"testing"
"github.com/stretchr/testify/require"
"github.com/dapr/components-contrib/secretstores"
ss_azure "github.com/dapr/components-contrib/secretstores/azure/keyvault"
ss_hashicorp_vault "github.com/dapr/components-contrib/secretstores/hashicorp/vault"
ss_kubernetes "github.com/dapr/components-contrib/secretstores/kubernetes"
ss_local_env "github.com/dapr/components-contrib/secretstores/local/env"
ss_local_file "github.com/dapr/components-contrib/secretstores/local/file"
conf_secret "github.com/dapr/components-contrib/tests/conformance/secretstores"
)
func TestSecretStoreConformance(t *testing.T) {
tc, err := NewTestConfiguration("../config/secretstores/tests.yml")
const configPath = "../config/secretstores/"
tc, err := NewTestConfiguration(filepath.Join(configPath, "tests.yml"))
require.NoError(t, err)
require.NotNil(t, tc)
tc.TestFn = func(comp *TestComponent) func(t *testing.T) {
return func(t *testing.T) {
ParseConfigurationMap(t, comp.Config)
componentConfigPath := convertComponentNameToPath(comp.Component, comp.Profile)
props, err := loadComponentsAndProperties(t, filepath.Join(configPath, componentConfigPath))
require.NoErrorf(t, err, "error running conformance test for component %s", comp.Component)
store := loadSecretStore(comp.Component)
require.NotNilf(t, store, "error running conformance test for component %s", comp.Component)
storeConfig := conf_secret.NewTestConfig(comp.Component, comp.Operations)
conf_secret.ConformanceTests(t, props, store, storeConfig)
}
}
tc.Run(t)
}
func loadSecretStore(name string) secretstores.SecretStore {
switch name {
case "azure.keyvault.certificate":
return ss_azure.NewAzureKeyvaultSecretStore(testLogger)
case "azure.keyvault.serviceprincipal":
return ss_azure.NewAzureKeyvaultSecretStore(testLogger)
case "kubernetes":
return ss_kubernetes.NewKubernetesSecretStore(testLogger)
case "local.env":
return ss_local_env.NewEnvSecretStore(testLogger)
case "local.file":
return ss_local_file.NewLocalSecretStore(testLogger)
case "hashicorp.vault":
return ss_hashicorp_vault.NewHashiCorpVaultSecretStore(testLogger)
default:
return nil
}
}

View File

@ -17,14 +17,118 @@ limitations under the License.
package conformance
import (
"path/filepath"
"testing"
"github.com/stretchr/testify/require"
"github.com/dapr/components-contrib/state"
s_awsdynamodb "github.com/dapr/components-contrib/state/aws/dynamodb"
s_blobstorage "github.com/dapr/components-contrib/state/azure/blobstorage"
s_cosmosdb "github.com/dapr/components-contrib/state/azure/cosmosdb"
s_azuretablestorage "github.com/dapr/components-contrib/state/azure/tablestorage"
s_cassandra "github.com/dapr/components-contrib/state/cassandra"
s_cloudflareworkerskv "github.com/dapr/components-contrib/state/cloudflare/workerskv"
s_cockroachdb "github.com/dapr/components-contrib/state/cockroachdb"
s_etcd "github.com/dapr/components-contrib/state/etcd"
s_gcpfirestore "github.com/dapr/components-contrib/state/gcp/firestore"
s_inmemory "github.com/dapr/components-contrib/state/in-memory"
s_memcached "github.com/dapr/components-contrib/state/memcached"
s_mongodb "github.com/dapr/components-contrib/state/mongodb"
s_mysql "github.com/dapr/components-contrib/state/mysql"
s_oracledatabase "github.com/dapr/components-contrib/state/oracledatabase"
s_postgresql "github.com/dapr/components-contrib/state/postgresql"
s_redis "github.com/dapr/components-contrib/state/redis"
s_rethinkdb "github.com/dapr/components-contrib/state/rethinkdb"
s_sqlite "github.com/dapr/components-contrib/state/sqlite"
s_sqlserver "github.com/dapr/components-contrib/state/sqlserver"
conf_state "github.com/dapr/components-contrib/tests/conformance/state"
)
func TestStateConformance(t *testing.T) {
tc, err := NewTestConfiguration("../config/state/tests.yml")
const configPath = "../config/state/"
tc, err := NewTestConfiguration(filepath.Join(configPath, "tests.yml"))
require.NoError(t, err)
require.NotNil(t, tc)
tc.TestFn = func(comp *TestComponent) func(t *testing.T) {
return func(t *testing.T) {
ParseConfigurationMap(t, comp.Config)
componentConfigPath := convertComponentNameToPath(comp.Component, comp.Profile)
props, err := loadComponentsAndProperties(t, filepath.Join(configPath, componentConfigPath))
require.NoErrorf(t, err, "error running conformance test for component %s", comp.Component)
store := loadStateStore(comp.Component)
require.NotNilf(t, store, "error running conformance test for component %s", comp.Component)
storeConfig, err := conf_state.NewTestConfig(comp.Component, comp.Operations, comp.Config)
require.NoErrorf(t, err, "error running conformance test for component %s", comp.Component)
conf_state.ConformanceTests(t, props, store, storeConfig)
}
}
tc.Run(t)
}
func loadStateStore(name string) state.Store {
switch name {
case "redis.v6":
return s_redis.NewRedisStateStore(testLogger)
case "redis.v7":
return s_redis.NewRedisStateStore(testLogger)
case "azure.blobstorage":
return s_blobstorage.NewAzureBlobStorageStore(testLogger)
case "azure.cosmosdb":
return s_cosmosdb.NewCosmosDBStateStore(testLogger)
case "mongodb":
return s_mongodb.NewMongoDB(testLogger)
case "azure.sql":
return s_sqlserver.New(testLogger)
case "sqlserver":
return s_sqlserver.New(testLogger)
case "postgresql.docker":
return s_postgresql.NewPostgreSQLStateStore(testLogger)
case "postgresql.azure":
return s_postgresql.NewPostgreSQLStateStore(testLogger)
case "sqlite":
return s_sqlite.NewSQLiteStateStore(testLogger)
case "mysql.mysql":
return s_mysql.NewMySQLStateStore(testLogger)
case "mysql.mariadb":
return s_mysql.NewMySQLStateStore(testLogger)
case "oracledatabase":
return s_oracledatabase.NewOracleDatabaseStateStore(testLogger)
case "azure.tablestorage.storage":
return s_azuretablestorage.NewAzureTablesStateStore(testLogger)
case "azure.tablestorage.cosmosdb":
return s_azuretablestorage.NewAzureTablesStateStore(testLogger)
case "cassandra":
return s_cassandra.NewCassandraStateStore(testLogger)
case "cloudflare.workerskv":
return s_cloudflareworkerskv.NewCFWorkersKV(testLogger)
case "cockroachdb":
return s_cockroachdb.New(testLogger)
case "memcached":
return s_memcached.NewMemCacheStateStore(testLogger)
case "rethinkdb":
return s_rethinkdb.NewRethinkDBStateStore(testLogger)
case "in-memory":
return s_inmemory.NewInMemoryStateStore(testLogger)
case "aws.dynamodb.docker":
return s_awsdynamodb.NewDynamoDBStateStore(testLogger)
case "aws.dynamodb.terraform":
return s_awsdynamodb.NewDynamoDBStateStore(testLogger)
case "etcd.v1":
return s_etcd.NewEtcdStateStoreV1(testLogger)
case "etcd.v2":
return s_etcd.NewEtcdStateStoreV2(testLogger)
case "gcp.firestore.docker":
return s_gcpfirestore.NewFirestoreStateStore(testLogger)
case "gcp.firestore.cloud":
return s_gcpfirestore.NewFirestoreStateStore(testLogger)
default:
return nil
}
}

View File

@ -17,14 +17,45 @@ limitations under the License.
package conformance
import (
"path/filepath"
"testing"
"github.com/stretchr/testify/require"
conf_workflows "github.com/dapr/components-contrib/tests/conformance/workflows"
"github.com/dapr/components-contrib/workflows"
wf_temporal "github.com/dapr/components-contrib/workflows/temporal"
)
func TestWorkflowsConformance(t *testing.T) {
tc, err := NewTestConfiguration("../config/workflows/tests.yml")
const configPath = "../config/workflows/"
tc, err := NewTestConfiguration(filepath.Join(configPath, "tests.yml"))
require.NoError(t, err)
require.NotNil(t, tc)
tc.TestFn = func(comp *TestComponent) func(t *testing.T) {
return func(t *testing.T) {
ParseConfigurationMap(t, comp.Config)
componentConfigPath := convertComponentNameToPath(comp.Component, comp.Profile)
props, err := loadComponentsAndProperties(t, filepath.Join(configPath, componentConfigPath))
require.NoErrorf(t, err, "error running conformance test for component %s", comp.Component)
wf := loadWorkflow(comp.Component)
wfConfig := conf_workflows.NewTestConfig(comp.Component, comp.Operations, comp.Config)
conf_workflows.ConformanceTests(t, props, wf, wfConfig)
}
}
tc.Run(t)
}
func loadWorkflow(name string) workflows.Workflow {
switch name {
case "temporal":
return wf_temporal.NewTemporalWorkflow(testLogger)
default:
return nil
}
}