mirror of https://github.com/databricks/cli.git
Add reflection/dyn-based tests
This commit is contained in:
parent
35bed3f549
commit
3e1e169225
|
@ -244,23 +244,23 @@ func (m *applyPresets) Apply(ctx context.Context, b *bundle.Bundle) diag.Diagnos
|
||||||
// Quality monitors presets
|
// Quality monitors presets
|
||||||
// Supported: Schedule, Catalog, Schema
|
// Supported: Schedule, Catalog, Schema
|
||||||
// Not supported: Tags (not in API as of 2024-10)
|
// Not supported: Tags (not in API as of 2024-10)
|
||||||
if t.TriggerPauseStatus == config.Paused {
|
for key, q := range r.QualityMonitors {
|
||||||
for key, q := range r.QualityMonitors {
|
if q.CreateMonitor == nil {
|
||||||
if q.CreateMonitor == nil {
|
diags = diags.Extend(diag.Errorf("quality monitor %s is not defined", key))
|
||||||
diags = diags.Extend(diag.Errorf("quality monitor %s is not defined", key))
|
continue
|
||||||
continue
|
}
|
||||||
}
|
// Remove all schedules from monitors, since they don't support pausing/unpausing.
|
||||||
// Remove all schedules from monitors, since they don't support pausing/unpausing.
|
// Quality monitors might support the "pause" property in the future, so at the
|
||||||
// Quality monitors might support the "pause" property in the future, so at the
|
// CLI level we do respect that property if it is set to "unpaused."
|
||||||
// CLI level we do respect that property if it is set to "unpaused."
|
if t.TriggerPauseStatus == config.Paused {
|
||||||
if q.Schedule != nil && q.Schedule.PauseStatus != catalog.MonitorCronSchedulePauseStatusUnpaused {
|
if q.Schedule != nil && q.Schedule.PauseStatus != catalog.MonitorCronSchedulePauseStatusUnpaused {
|
||||||
q.Schedule = nil
|
q.Schedule = nil
|
||||||
}
|
}
|
||||||
if t.Catalog != "" && t.Schema != "" {
|
}
|
||||||
parts := strings.Split(q.TableName, ".")
|
if t.Catalog != "" && t.Schema != "" {
|
||||||
if len(parts) != 3 {
|
q.TableName = fullyQualifyName(q.TableName, t.Catalog, t.Schema)
|
||||||
q.TableName = fmt.Sprintf("%s.%s.%s", t.Catalog, t.Schema, q.TableName)
|
if q.OutputSchemaName == "" {
|
||||||
}
|
q.OutputSchemaName = t.Catalog + "." + t.Schema
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -2,10 +2,8 @@ package mutator_test
|
||||||
|
|
||||||
import (
|
import (
|
||||||
"context"
|
"context"
|
||||||
"fmt"
|
|
||||||
"reflect"
|
"reflect"
|
||||||
"runtime"
|
"runtime"
|
||||||
"strconv"
|
|
||||||
"strings"
|
"strings"
|
||||||
"testing"
|
"testing"
|
||||||
|
|
||||||
|
@ -17,18 +15,17 @@ import (
|
||||||
"github.com/databricks/cli/libs/dbr"
|
"github.com/databricks/cli/libs/dbr"
|
||||||
"github.com/databricks/cli/libs/dyn"
|
"github.com/databricks/cli/libs/dyn"
|
||||||
"github.com/databricks/databricks-sdk-go/service/catalog"
|
"github.com/databricks/databricks-sdk-go/service/catalog"
|
||||||
"github.com/databricks/databricks-sdk-go/service/compute"
|
|
||||||
"github.com/databricks/databricks-sdk-go/service/dashboards"
|
|
||||||
"github.com/databricks/databricks-sdk-go/service/jobs"
|
"github.com/databricks/databricks-sdk-go/service/jobs"
|
||||||
"github.com/databricks/databricks-sdk-go/service/ml"
|
|
||||||
"github.com/databricks/databricks-sdk-go/service/pipelines"
|
"github.com/databricks/databricks-sdk-go/service/pipelines"
|
||||||
"github.com/databricks/databricks-sdk-go/service/serving"
|
"github.com/databricks/databricks-sdk-go/service/serving"
|
||||||
"github.com/stretchr/testify/require"
|
"github.com/stretchr/testify/require"
|
||||||
)
|
)
|
||||||
|
|
||||||
type RecordedField struct {
|
type recordedField struct {
|
||||||
Path string
|
Path dyn.Path
|
||||||
Value string
|
PathString string
|
||||||
|
Placeholder string
|
||||||
|
Expected string
|
||||||
}
|
}
|
||||||
|
|
||||||
func TestApplyPresetsPrefix(t *testing.T) {
|
func TestApplyPresetsPrefix(t *testing.T) {
|
||||||
|
@ -473,7 +470,7 @@ func TestApplyPresetsCatalogSchema(t *testing.T) {
|
||||||
Config: config.Root{
|
Config: config.Root{
|
||||||
Resources: config.Resources{
|
Resources: config.Resources{
|
||||||
Jobs: map[string]*resources.Job{
|
Jobs: map[string]*resources.Job{
|
||||||
"object": {
|
"key": {
|
||||||
JobSettings: &jobs.JobSettings{
|
JobSettings: &jobs.JobSettings{
|
||||||
Name: "job",
|
Name: "job",
|
||||||
Parameters: []jobs.JobParameterDefinition{
|
Parameters: []jobs.JobParameterDefinition{
|
||||||
|
@ -484,7 +481,7 @@ func TestApplyPresetsCatalogSchema(t *testing.T) {
|
||||||
},
|
},
|
||||||
},
|
},
|
||||||
Pipelines: map[string]*resources.Pipeline{
|
Pipelines: map[string]*resources.Pipeline{
|
||||||
"object": {
|
"key": {
|
||||||
PipelineSpec: &pipelines.PipelineSpec{
|
PipelineSpec: &pipelines.PipelineSpec{
|
||||||
Name: "pipeline",
|
Name: "pipeline",
|
||||||
Catalog: "<catalog>",
|
Catalog: "<catalog>",
|
||||||
|
@ -493,7 +490,7 @@ func TestApplyPresetsCatalogSchema(t *testing.T) {
|
||||||
},
|
},
|
||||||
},
|
},
|
||||||
ModelServingEndpoints: map[string]*resources.ModelServingEndpoint{
|
ModelServingEndpoints: map[string]*resources.ModelServingEndpoint{
|
||||||
"object": {
|
"key": {
|
||||||
CreateServingEndpoint: &serving.CreateServingEndpoint{
|
CreateServingEndpoint: &serving.CreateServingEndpoint{
|
||||||
Name: "serving",
|
Name: "serving",
|
||||||
AiGateway: &serving.AiGatewayConfig{
|
AiGateway: &serving.AiGatewayConfig{
|
||||||
|
@ -518,7 +515,7 @@ func TestApplyPresetsCatalogSchema(t *testing.T) {
|
||||||
},
|
},
|
||||||
},
|
},
|
||||||
RegisteredModels: map[string]*resources.RegisteredModel{
|
RegisteredModels: map[string]*resources.RegisteredModel{
|
||||||
"object": {
|
"key": {
|
||||||
CreateRegisteredModelRequest: &catalog.CreateRegisteredModelRequest{
|
CreateRegisteredModelRequest: &catalog.CreateRegisteredModelRequest{
|
||||||
Name: "registered_model",
|
Name: "registered_model",
|
||||||
CatalogName: "<catalog>",
|
CatalogName: "<catalog>",
|
||||||
|
@ -527,49 +524,21 @@ func TestApplyPresetsCatalogSchema(t *testing.T) {
|
||||||
},
|
},
|
||||||
},
|
},
|
||||||
QualityMonitors: map[string]*resources.QualityMonitor{
|
QualityMonitors: map[string]*resources.QualityMonitor{
|
||||||
"object": {
|
"key": {
|
||||||
TableName: "table",
|
TableName: "<catalog>.<schema>.table",
|
||||||
CreateMonitor: &catalog.CreateMonitor{
|
CreateMonitor: &catalog.CreateMonitor{
|
||||||
OutputSchemaName: "<catalog>.<schema>",
|
OutputSchemaName: "<catalog>.<schema>",
|
||||||
},
|
},
|
||||||
},
|
},
|
||||||
},
|
},
|
||||||
Schemas: map[string]*resources.Schema{
|
Schemas: map[string]*resources.Schema{
|
||||||
"object": {
|
"key": {
|
||||||
CreateSchema: &catalog.CreateSchema{
|
CreateSchema: &catalog.CreateSchema{
|
||||||
Name: "<schema>",
|
Name: "<schema>",
|
||||||
CatalogName: "<catalog>",
|
CatalogName: "<catalog>",
|
||||||
},
|
},
|
||||||
},
|
},
|
||||||
},
|
},
|
||||||
Models: map[string]*resources.MlflowModel{
|
|
||||||
"object": {
|
|
||||||
Model: &ml.Model{
|
|
||||||
Name: "<catalog>.<schema>.model",
|
|
||||||
},
|
|
||||||
},
|
|
||||||
},
|
|
||||||
Experiments: map[string]*resources.MlflowExperiment{
|
|
||||||
"object": {
|
|
||||||
Experiment: &ml.Experiment{
|
|
||||||
Name: "<catalog>.<schema>.experiment",
|
|
||||||
},
|
|
||||||
},
|
|
||||||
},
|
|
||||||
Clusters: map[string]*resources.Cluster{
|
|
||||||
"object": {
|
|
||||||
ClusterSpec: &compute.ClusterSpec{
|
|
||||||
ClusterName: "cluster",
|
|
||||||
},
|
|
||||||
},
|
|
||||||
},
|
|
||||||
Dashboards: map[string]*resources.Dashboard{
|
|
||||||
"object": {
|
|
||||||
Dashboard: &dashboards.Dashboard{
|
|
||||||
DisplayName: "dashboard",
|
|
||||||
},
|
|
||||||
},
|
|
||||||
},
|
|
||||||
},
|
},
|
||||||
},
|
},
|
||||||
}
|
}
|
||||||
|
@ -577,243 +546,194 @@ func TestApplyPresetsCatalogSchema(t *testing.T) {
|
||||||
Catalog: "my_catalog",
|
Catalog: "my_catalog",
|
||||||
Schema: "my_schema",
|
Schema: "my_schema",
|
||||||
}
|
}
|
||||||
|
|
||||||
// Stage 1: Apply presets BEFORE cleanup.
|
|
||||||
// Because all fields are already set to placeholders, Apply should NOT overwrite them (no-op).
|
|
||||||
ctx := context.Background()
|
ctx := context.Background()
|
||||||
|
|
||||||
|
// Initial scan: record all fields that contain placeholders.
|
||||||
|
// We do this before the first apply so we can verify no changes occur.
|
||||||
|
var recordedFields []recordedField
|
||||||
|
require.NoError(t, b.Config.Mutate(func(root dyn.Value) (dyn.Value, error) {
|
||||||
|
_, err := dyn.Walk(b.Config.Value(), func(p dyn.Path, v dyn.Value) (dyn.Value, error) {
|
||||||
|
if v.Kind() == dyn.KindString {
|
||||||
|
val := v.MustString()
|
||||||
|
if strings.Contains(val, "<catalog>") || strings.Contains(val, "<schema>") {
|
||||||
|
pathCopy := make(dyn.Path, len(p))
|
||||||
|
copy(pathCopy, p)
|
||||||
|
recordedFields = append(recordedFields, recordedField{
|
||||||
|
Path: pathCopy,
|
||||||
|
PathString: pathCopy.String(),
|
||||||
|
Placeholder: val,
|
||||||
|
Expected: replacePlaceholders(val, "my_catalog", "my_schema"),
|
||||||
|
})
|
||||||
|
}
|
||||||
|
}
|
||||||
|
return v, nil
|
||||||
|
})
|
||||||
|
return root, err
|
||||||
|
}))
|
||||||
|
|
||||||
|
// Stage 1: Apply presets before cleanup, should be no-op.
|
||||||
diags := bundle.Apply(ctx, b, mutator.ApplyPresets())
|
diags := bundle.Apply(ctx, b, mutator.ApplyPresets())
|
||||||
require.False(t, diags.HasError(), "unexpected error before cleanup: %v", diags.Error())
|
require.False(t, diags.HasError(), "unexpected error before cleanup: %v", diags.Error())
|
||||||
verifyNoChangesBeforeCleanup(t, b.Config)
|
|
||||||
|
|
||||||
// Stage 2: Cleanup all "<catalog>" and "<schema>" placeholders
|
// Verify that no recorded fields changed
|
||||||
// and record where they were.
|
verifyNoChangesBeforeCleanup(t, b.Config.Value(), recordedFields)
|
||||||
b.Config.MarkMutatorEntry(ctx)
|
|
||||||
resources := reflect.ValueOf(&b.Config.Resources).Elem()
|
// Stage 2: Cleanup: Walk over rootVal and remove placeholders, adjusting recordedFields Expected values.
|
||||||
recordedFields := recordAndCleanupFields(resources, "Resources")
|
require.NoError(t, b.Config.Mutate(func(root dyn.Value) (dyn.Value, error) {
|
||||||
b.Config.Resources.Jobs["object"].Parameters = nil
|
for _, f := range recordedFields {
|
||||||
b.Config.MarkMutatorExit(ctx)
|
value, err := dyn.GetByPath(root, f.Path)
|
||||||
|
require.NoError(t, err)
|
||||||
|
|
||||||
|
val := value.MustString()
|
||||||
|
cleanedVal := removePlaceholders(val)
|
||||||
|
root, err = dyn.SetByPath(root, f.Path, dyn.V(cleanedVal))
|
||||||
|
require.NoError(t, err)
|
||||||
|
}
|
||||||
|
root, err := dyn.Set(root, "resources.jobs.key.parameters", dyn.NilValue)
|
||||||
|
require.NoError(t, err)
|
||||||
|
return root, nil
|
||||||
|
}))
|
||||||
|
|
||||||
// Stage 3: Apply presets after cleanup.
|
// Stage 3: Apply presets after cleanup.
|
||||||
diags = bundle.Apply(ctx, b, mutator.ApplyPresets())
|
diags = bundle.Apply(ctx, b, mutator.ApplyPresets())
|
||||||
require.False(t, diags.HasError(), "unexpected error after cleanup: %v", diags.Error())
|
require.False(t, diags.HasError(), "unexpected error after cleanup: %v", diags.Error())
|
||||||
verifyAllFields(t, b.Config, recordedFields)
|
|
||||||
|
|
||||||
// Stage 4: Verify that all known fields in config.Resources have been processed.
|
// Verify that fields have the expected replacements
|
||||||
checkCompleteness(t, &b.Config.Resources, "Resources", recordedFields)
|
config := b.Config.Value()
|
||||||
}
|
|
||||||
|
|
||||||
func verifyNoChangesBeforeCleanup(t *testing.T, cfg config.Root) {
|
|
||||||
t.Helper()
|
|
||||||
|
|
||||||
// Just check a few representative fields to ensure they are still placeholders.
|
|
||||||
// For example: Job parameter defaults should still have "<catalog>" and "<schema>"
|
|
||||||
jobParams := cfg.Resources.Jobs["object"].Parameters
|
|
||||||
require.Len(t, jobParams, 2, "job parameters count mismatch")
|
|
||||||
require.Equal(t, "<catalog>", jobParams[0].Default, "expected no changes before cleanup")
|
|
||||||
require.Equal(t, "<schema>", jobParams[1].Default, "expected no changes before cleanup")
|
|
||||||
|
|
||||||
pipeline := cfg.Resources.Pipelines["object"]
|
|
||||||
require.Equal(t, "<catalog>", pipeline.Catalog, "expected no changes before cleanup")
|
|
||||||
require.Equal(t, "<schema>", pipeline.Target, "expected no changes before cleanup")
|
|
||||||
}
|
|
||||||
|
|
||||||
// recordAndCleanupFields recursively finds all Catalog/CatalogName/Schema/SchemaName fields,
|
|
||||||
// records their original values, and replaces them with empty strings.
|
|
||||||
func recordAndCleanupFields(rv reflect.Value, path string) []RecordedField {
|
|
||||||
var recordedFields []RecordedField
|
|
||||||
|
|
||||||
switch rv.Kind() {
|
|
||||||
case reflect.Ptr, reflect.Interface:
|
|
||||||
if !rv.IsNil() {
|
|
||||||
recordedFields = append(recordedFields, recordAndCleanupFields(rv.Elem(), path)...)
|
|
||||||
}
|
|
||||||
|
|
||||||
case reflect.Struct:
|
|
||||||
tp := rv.Type()
|
|
||||||
for i := 0; i < rv.NumField(); i++ {
|
|
||||||
ft := tp.Field(i)
|
|
||||||
fv := rv.Field(i)
|
|
||||||
fPath := path + "." + ft.Name
|
|
||||||
|
|
||||||
if fv.Kind() == reflect.String {
|
|
||||||
original := fv.String()
|
|
||||||
newVal := cleanedValue(original)
|
|
||||||
if newVal != original {
|
|
||||||
fv.SetString(newVal)
|
|
||||||
recordedFields = append(recordedFields, RecordedField{fPath, original})
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
recordedFields = append(recordedFields, recordAndCleanupFieldsRecursive(fv, fPath)...)
|
|
||||||
}
|
|
||||||
|
|
||||||
case reflect.Map:
|
|
||||||
for _, mk := range rv.MapKeys() {
|
|
||||||
mVal := rv.MapIndex(mk)
|
|
||||||
recordedFields = append(recordedFields, recordAndCleanupFieldsRecursive(mVal, path+"."+mk.String())...)
|
|
||||||
}
|
|
||||||
|
|
||||||
case reflect.Slice, reflect.Array:
|
|
||||||
for i := 0; i < rv.Len(); i++ {
|
|
||||||
recordedFields = append(recordedFields, recordAndCleanupFieldsRecursive(rv.Index(i), fmt.Sprintf("%s[%d]", path, i))...)
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
return recordedFields
|
|
||||||
}
|
|
||||||
|
|
||||||
// verifyAllFields checks if all collected fields are now properly replaced after ApplyPresets.
|
|
||||||
func verifyAllFields(t *testing.T, cfg config.Root, recordedFields []RecordedField) {
|
|
||||||
t.Helper()
|
|
||||||
for _, f := range recordedFields {
|
for _, f := range recordedFields {
|
||||||
expected := replaceCatalogSchemaPlaceholders(f.Value)
|
val, err := dyn.GetByPath(config, f.Path)
|
||||||
got := getStringValueAtPath(t, reflect.ValueOf(cfg), f.Path)
|
require.NoError(t, err, "failed to get path %s", f.Path)
|
||||||
require.Equal(t, expected, got, "expected catalog/schema to be replaced by preset values at %s", f.Path)
|
require.Equal(t, f.Expected, val.MustString(), "preset value expected for %s based on placeholder %s", f.Path, f.Placeholder)
|
||||||
|
}
|
||||||
|
|
||||||
|
// Stage 4: Check completeness
|
||||||
|
ignoredFields := map[string]string{
|
||||||
|
// Any fields that should be ignored in the completeness check
|
||||||
|
// Example:
|
||||||
|
// "resources.jobs.object.schema_something": "this property doesn't relate to the catalog/schema",
|
||||||
|
}
|
||||||
|
checkCompleteness(t, recordedFields, ignoredFields)
|
||||||
|
}
|
||||||
|
|
||||||
|
func verifyNoChangesBeforeCleanup(t *testing.T, rootVal dyn.Value, recordedFields []recordedField) {
|
||||||
|
t.Helper()
|
||||||
|
|
||||||
|
for _, f := range recordedFields {
|
||||||
|
val, err := dyn.GetByPath(rootVal, f.Path)
|
||||||
|
require.NoError(t, err, "failed to get path %s", f.Path)
|
||||||
|
require.Equal(t, f.Placeholder, val.MustString(),
|
||||||
|
"expected placeholder '%s' at %s to remain unchanged before cleanup", f.Placeholder, f.Path)
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
// checkCompleteness ensures that all catalog/schema fields have been processed.
|
func checkCompleteness(t *testing.T, recordedFields []recordedField, ignoredFields map[string]string) {
|
||||||
func checkCompleteness(t *testing.T, root interface{}, rootPath string, recordedFields []RecordedField) {
|
|
||||||
t.Helper()
|
t.Helper()
|
||||||
recordedSet := make(map[string]bool)
|
|
||||||
for _, f := range recordedFields {
|
// Build a set for recorded fields
|
||||||
recordedSet[f.Path] = true
|
recordedSet := make(map[string]struct{})
|
||||||
|
for _, field := range recordedFields {
|
||||||
|
recordedSet[field.PathString] = struct{}{}
|
||||||
}
|
}
|
||||||
|
|
||||||
var check func(rv reflect.Value, path string)
|
// Obtain the type of config.Resources
|
||||||
check = func(rv reflect.Value, path string) {
|
var r config.Resources
|
||||||
switch rv.Kind() {
|
resourcesType := reflect.TypeOf(r)
|
||||||
case reflect.Ptr, reflect.Interface:
|
|
||||||
if !rv.IsNil() {
|
// Track missing fields
|
||||||
check(rv.Elem(), path)
|
var missingFields []string
|
||||||
}
|
|
||||||
case reflect.Struct:
|
// Keep track of visited types to prevent infinite loops (cycles)
|
||||||
tp := rv.Type()
|
visited := make(map[reflect.Type]struct{})
|
||||||
for i := 0; i < rv.NumField(); i++ {
|
|
||||||
ft := tp.Field(i)
|
// Helper function to handle maps, slices, arrays, and nested pointers/interfaces
|
||||||
fv := rv.Field(i)
|
verifyFieldType := func(fieldType reflect.Type, path string, fn func(reflect.Type, string)) {
|
||||||
fPath := path + "." + ft.Name
|
switch fieldType.Kind() {
|
||||||
if isCatalogOrSchemaField(ft.Name) {
|
|
||||||
require.Truef(t, recordedSet[fPath],
|
|
||||||
"Field %s was not recorded in recordedFields (completeness check failed)", fPath)
|
|
||||||
}
|
|
||||||
check(fv, fPath)
|
|
||||||
}
|
|
||||||
case reflect.Map:
|
|
||||||
for _, mk := range rv.MapKeys() {
|
|
||||||
mVal := rv.MapIndex(mk)
|
|
||||||
check(mVal, path+"."+mk.String())
|
|
||||||
}
|
|
||||||
case reflect.Slice, reflect.Array:
|
case reflect.Slice, reflect.Array:
|
||||||
for i := 0; i < rv.Len(); i++ {
|
// For arrays/slices, inspect the element type
|
||||||
check(rv.Index(i), fmt.Sprintf("%s[%d]", path, i))
|
fn(fieldType.Elem(), path+"[0]")
|
||||||
}
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
rv := reflect.ValueOf(root)
|
|
||||||
if rv.Kind() == reflect.Ptr {
|
|
||||||
rv = rv.Elem()
|
|
||||||
}
|
|
||||||
check(rv, rootPath)
|
|
||||||
}
|
|
||||||
|
|
||||||
// getStringValueAtPath navigates the given path and returns the string value at that path.
|
|
||||||
func getStringValueAtPath(t *testing.T, root reflect.Value, path string) string {
|
|
||||||
t.Helper()
|
|
||||||
parts := strings.Split(path, ".")
|
|
||||||
return navigatePath(t, root, parts)
|
|
||||||
}
|
|
||||||
|
|
||||||
func navigatePath(t *testing.T, rv reflect.Value, parts []string) string {
|
|
||||||
t.Helper()
|
|
||||||
|
|
||||||
// Trim empty parts if any
|
|
||||||
for len(parts) > 0 && parts[0] == "" {
|
|
||||||
parts = parts[1:]
|
|
||||||
}
|
|
||||||
|
|
||||||
for len(parts) > 0 {
|
|
||||||
part := parts[0]
|
|
||||||
parts = parts[1:]
|
|
||||||
|
|
||||||
// Dereference pointers/interfaces before proceeding
|
|
||||||
for rv.Kind() == reflect.Ptr || rv.Kind() == reflect.Interface {
|
|
||||||
require.Falsef(t, rv.IsNil(), "nil pointer or interface encountered at part '%s'", part)
|
|
||||||
rv = rv.Elem()
|
|
||||||
}
|
|
||||||
|
|
||||||
// If the part has indexing like "Parameters[0]", split it into "Parameters" and "[0]"
|
|
||||||
var indexPart string
|
|
||||||
fieldName := part
|
|
||||||
if idx := strings.IndexRune(part, '['); idx != -1 {
|
|
||||||
// e.g. part = "Parameters[0]"
|
|
||||||
fieldName = part[:idx] // "Parameters"
|
|
||||||
indexPart = part[idx:] // "[0]"
|
|
||||||
require.Truef(t, strings.HasPrefix(indexPart, "["), "expected '[' in indexing")
|
|
||||||
require.Truef(t, strings.HasSuffix(indexPart, "]"), "expected ']' at end of indexing")
|
|
||||||
}
|
|
||||||
|
|
||||||
// Navigate down structures/maps
|
|
||||||
switch rv.Kind() {
|
|
||||||
case reflect.Struct:
|
|
||||||
// Find the struct field by name
|
|
||||||
ft, ok := rv.Type().FieldByName(fieldName)
|
|
||||||
if !ok {
|
|
||||||
t.Fatalf("Could not find field '%s' in struct at path", fieldName)
|
|
||||||
}
|
|
||||||
rv = rv.FieldByIndex(ft.Index)
|
|
||||||
|
|
||||||
case reflect.Map:
|
case reflect.Map:
|
||||||
// Use fieldName as map key
|
// For maps, inspect the value type
|
||||||
mapVal := rv.MapIndex(reflect.ValueOf(fieldName))
|
fn(fieldType.Elem(), path+".key")
|
||||||
require.Truef(t, mapVal.IsValid(), "no map entry '%s' found in path", fieldName)
|
case reflect.Ptr, reflect.Interface:
|
||||||
rv = mapVal
|
// For pointers/interfaces, inspect the element if it's a pointer
|
||||||
|
if fieldType.Kind() == reflect.Ptr {
|
||||||
default:
|
fn(fieldType.Elem(), path)
|
||||||
// If we're here, maybe we expected a struct or map but got something else
|
|
||||||
t.Fatalf("Unexpected kind '%s' when looking for '%s'", rv.Kind(), fieldName)
|
|
||||||
}
|
|
||||||
|
|
||||||
// If there's an index part, apply it now
|
|
||||||
if indexPart != "" {
|
|
||||||
// Dereference again if needed
|
|
||||||
for rv.Kind() == reflect.Ptr || rv.Kind() == reflect.Interface {
|
|
||||||
require.False(t, rv.IsNil(), "nil pointer or interface when indexing")
|
|
||||||
rv = rv.Elem()
|
|
||||||
}
|
}
|
||||||
|
case reflect.Struct:
|
||||||
require.Truef(t, rv.Kind() == reflect.Slice || rv.Kind() == reflect.Array, "expected slice/array for indexing but got %s", rv.Kind())
|
// For structs, directly recurse into their fields
|
||||||
|
fn(fieldType, path)
|
||||||
idxStr := indexPart[1 : len(indexPart)-1] // remove [ and ]
|
default:
|
||||||
idx, err := strconv.Atoi(idxStr)
|
// For basic or unknown kinds, do nothing
|
||||||
require.NoError(t, err, "invalid slice index %s", indexPart)
|
|
||||||
|
|
||||||
require.Truef(t, idx < rv.Len(), "index %d out of range in slice/array of length %d", idx, rv.Len())
|
|
||||||
rv = rv.Index(idx)
|
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
// Dereference if needed at the leaf
|
// Recursive function to verify the fields of a given type.
|
||||||
for rv.Kind() == reflect.Ptr || rv.Kind() == reflect.Interface {
|
var verifyTypeFields func(rt reflect.Type, path string)
|
||||||
require.False(t, rv.IsNil(), "nil pointer or interface at leaf")
|
verifyTypeFields = func(rt reflect.Type, path string) {
|
||||||
rv = rv.Elem()
|
// Avoid cycles by skipping already visited types
|
||||||
|
if _, seen := visited[rt]; seen {
|
||||||
|
return
|
||||||
|
}
|
||||||
|
visited[rt] = struct{}{}
|
||||||
|
|
||||||
|
switch rt.Kind() {
|
||||||
|
case reflect.Ptr, reflect.Interface:
|
||||||
|
// For pointers/interfaces, inspect the element type if available
|
||||||
|
if rt.Kind() == reflect.Ptr {
|
||||||
|
verifyTypeFields(rt.Elem(), path)
|
||||||
|
}
|
||||||
|
case reflect.Struct:
|
||||||
|
for i := 0; i < rt.NumField(); i++ {
|
||||||
|
ft := rt.Field(i)
|
||||||
|
jsonTag := ft.Tag.Get("json")
|
||||||
|
if jsonTag == "" || jsonTag == "-" {
|
||||||
|
// Ignore field names when there's no JSON tag,
|
||||||
|
// e.g. for Jobs.JobSettings
|
||||||
|
verifyFieldType(ft.Type, path, verifyTypeFields)
|
||||||
|
continue
|
||||||
|
}
|
||||||
|
|
||||||
|
fieldName := strings.Split(jsonTag, ",")[0]
|
||||||
|
fieldPath := path + "." + fieldName
|
||||||
|
|
||||||
|
if isCatalogOrSchemaField(fieldName) {
|
||||||
|
// Only check if the field is a string
|
||||||
|
if ft.Type.Kind() == reflect.String {
|
||||||
|
if _, recorded := recordedSet[fieldPath]; !recorded {
|
||||||
|
if _, ignored := ignoredFields[fieldPath]; !ignored {
|
||||||
|
missingFields = append(missingFields, fieldPath)
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
verifyFieldType(ft.Type, fieldPath, verifyTypeFields)
|
||||||
|
}
|
||||||
|
default:
|
||||||
|
// For other kinds at this level, do nothing
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
require.Equal(t, reflect.String, rv.Kind(), "expected a string at the final path")
|
// Start from "resources"
|
||||||
return rv.String()
|
verifyTypeFields(resourcesType, "resources")
|
||||||
|
|
||||||
|
// Report all missing fields
|
||||||
|
for _, field := range missingFields {
|
||||||
|
t.Errorf("Field %s was not included in the test (should be covered in 'recordedFields' or 'ignoredFields')", field)
|
||||||
|
}
|
||||||
|
|
||||||
|
// Fail the test if there were any missing fields
|
||||||
|
if len(missingFields) > 0 {
|
||||||
|
t.FailNow()
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
// isCatalogOrSchemaField returns true for a field names in config.Resources that we suspect could contain a catalog or schema name
|
||||||
func isCatalogOrSchemaField(name string) bool {
|
func isCatalogOrSchemaField(name string) bool {
|
||||||
switch name {
|
return strings.Contains(name, "catalog") || strings.Contains(name, "schema")
|
||||||
case "Catalog", "CatalogName", "Schema", "SchemaName", "Target":
|
|
||||||
return true
|
|
||||||
default:
|
|
||||||
return false
|
|
||||||
}
|
|
||||||
}
|
}
|
||||||
|
|
||||||
func cleanedValue(value string) string {
|
func removePlaceholders(value string) string {
|
||||||
value = strings.ReplaceAll(value, "<catalog>.", "")
|
value = strings.ReplaceAll(value, "<catalog>.", "")
|
||||||
value = strings.ReplaceAll(value, "<schema>.", "")
|
value = strings.ReplaceAll(value, "<schema>.", "")
|
||||||
value = strings.ReplaceAll(value, "<catalog>", "")
|
value = strings.ReplaceAll(value, "<catalog>", "")
|
||||||
|
@ -821,9 +741,8 @@ func cleanedValue(value string) string {
|
||||||
return value
|
return value
|
||||||
}
|
}
|
||||||
|
|
||||||
// replaceCatalogSchemaPlaceholders replaces placeholders with the final expected values.
|
func replacePlaceholders(placeholder, catalog, schema string) string {
|
||||||
func replaceCatalogSchemaPlaceholders(value string) string {
|
expected := strings.ReplaceAll(placeholder, "<catalog>", catalog)
|
||||||
value = strings.ReplaceAll(value, "<catalog>", "my_catalog")
|
expected = strings.ReplaceAll(expected, "<schema>", schema)
|
||||||
value = strings.ReplaceAll(value, "<schema>", "my_schema")
|
return expected
|
||||||
return value
|
|
||||||
}
|
}
|
||||||
|
|
Loading…
Reference in New Issue