You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@mynewt.apache.org by cc...@apache.org on 2016/09/25 20:04:47 UTC

incubator-mynewt-newt git commit: newt - Detect syscfg override ambiguities

Repository: incubator-mynewt-newt
Updated Branches:
  refs/heads/develop 4d098d64d -> 9a18359ca


newt - Detect syscfg override ambiguities

Newt reports an error and aborts the build upon detection of an
ambiguous syscfg override.  A syscfg setting is ambiguous if both of the
following are true:

    * The final 2+ overrides are done by packages of the same priority
    * Of the above overrides, not all values are identical

Hypothetical examples:

1. Ambiguous:

    [sys/log/pkg.yml]
        pkg.syscfg_defs:
            LOG_LEVEL:
                description: 'TBD'
                value: 0

    [net/nimble/host/pkg.yml]
        pkg.syscfg_vals:
            LOG_LEVEL: 1

    [libs/os/pkg.yml]
        pkg.syscfg_vals:
            LOG_LEVEL: 0

This is ambiguous because two packages of the same priority
(net/nimble/host and libs/os are both library packages) override a
setting with different values.

2. Unambiguous:

    [sys/log/pkg.yml]
        pkg.syscfg_defs:
            LOG_LEVEL:
                description: 'TBD'
                value: 0

    [net/nimble/host/pkg.yml]
        pkg.syscfg_vals:
            LOG_LEVEL: 1

    [libs/os/pkg.yml]
        pkg.syscfg_vals:
            LOG_LEVEL: 0

    [apps/myapp/pkg.yml]
        pkg.syscfg_vals:
            LOG_LEVEL: 3

The ambiguity is resolved because a higher priority package overrides
the setting.

3. Unambiguous:

    [sys/log/pkg.yml]
        pkg.syscfg_defs:
            LOG_LEVEL:
                description: 'TBD'
                value: 0

    [net/nimble/host/pkg.yml]
        pkg.syscfg_vals:
            LOG_LEVEL: 1

    [libs/os/pkg.yml]
        pkg.syscfg_vals:
            LOG_LEVEL: 1

Both overrides specify identical values, so there is no ambiguity.

4. Unambiguous:

    [sys/log/pkg.yml]
        pkg.syscfg_defs:
            LOG_LEVEL:
                description: 'TBD'
                value: 0

    [net/nimble/host/pkg.yml]
        pkg.syscfg_vals:
            LOG_LEVEL: 1

The original setting definition does not factor into ambiguity
detection.  There is only one override here, so an ambiguity is not
possible.


Project: http://git-wip-us.apache.org/repos/asf/incubator-mynewt-newt/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-mynewt-newt/commit/9a18359c
Tree: http://git-wip-us.apache.org/repos/asf/incubator-mynewt-newt/tree/9a18359c
Diff: http://git-wip-us.apache.org/repos/asf/incubator-mynewt-newt/diff/9a18359c

Branch: refs/heads/develop
Commit: 9a18359ca23ec655983588b0c821220d8e4f2442
Parents: 4d098d6
Author: Christopher Collins <cc...@apache.org>
Authored: Sun Sep 25 12:57:26 2016 -0700
Committer: Christopher Collins <cc...@apache.org>
Committed: Sun Sep 25 12:57:26 2016 -0700

----------------------------------------------------------------------
 newt/builder/build.go        |   9 +-
 newt/builder/buildpackage.go |   4 +-
 newt/builder/buildutil.go    |   3 +-
 newt/builder/load.go         |  14 +--
 newt/builder/targetbuild.go  |   5 +-
 newt/syscfg/syscfg.go        | 247 ++++++++++++++++++++++++++++----------
 6 files changed, 202 insertions(+), 80 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-mynewt-newt/blob/9a18359c/newt/builder/build.go
----------------------------------------------------------------------
diff --git a/newt/builder/build.go b/newt/builder/build.go
index 6e1596f..79b0bca 100644
--- a/newt/builder/build.go
+++ b/newt/builder/build.go
@@ -122,7 +122,7 @@ func (b *Builder) reloadCfg() (bool, error) {
 	// Determine which features have been detected so far.  The feature map is
 	// required for reloading syscfg, as features may unlock additional
 	// settings.
-	features := syscfg.Features(b.Cfg)
+	features := b.Cfg.Features()
 	cfg, err := syscfg.Read(b.sortedLocalPackages(), apis, b.injectedSettings,
 		features)
 	if err != nil {
@@ -200,7 +200,12 @@ func (b *Builder) loadDeps() error {
 	}
 
 	// Log the final syscfg.
-	syscfg.Log(b.Cfg)
+	b.Cfg.Log()
+
+	// Report syscfg errors.
+	if err := b.Cfg.DetectErrors(); err != nil {
+		return err
+	}
 
 	return nil
 }

http://git-wip-us.apache.org/repos/asf/incubator-mynewt-newt/blob/9a18359c/newt/builder/buildpackage.go
----------------------------------------------------------------------
diff --git a/newt/builder/buildpackage.go b/newt/builder/buildpackage.go
index 2cdd75b..596b359 100644
--- a/newt/builder/buildpackage.go
+++ b/newt/builder/buildpackage.go
@@ -140,7 +140,7 @@ func (bpkg *BuildPackage) CompilerInfo(b *Builder) (*toolchain.CompilerInfo, err
 	}
 
 	ci := toolchain.NewCompilerInfo()
-	features := syscfg.FeaturesForLpkg(b.Cfg, bpkg.LocalPackage)
+	features := b.Cfg.FeaturesForLpkg(bpkg.LocalPackage)
 	ci.Cflags = newtutil.GetStringSliceFeatures(bpkg.Viper, features,
 		"pkg.cflags")
 	ci.Lflags = newtutil.GetStringSliceFeatures(bpkg.Viper, features,
@@ -392,7 +392,7 @@ func (bpkg *BuildPackage) Resolve(b *Builder,
 	var err error
 	newDeps := false
 
-	features := syscfg.FeaturesForLpkg(cfg, bpkg.LocalPackage)
+	features := cfg.FeaturesForLpkg(bpkg.LocalPackage)
 
 	if !bpkg.depsResolved {
 		newDeps, err = bpkg.loadDeps(b, features)

http://git-wip-us.apache.org/repos/asf/incubator-mynewt-newt/blob/9a18359c/newt/builder/buildutil.go
----------------------------------------------------------------------
diff --git a/newt/builder/buildutil.go b/newt/builder/buildutil.go
index f35a1db..bc68212 100644
--- a/newt/builder/buildutil.go
+++ b/newt/builder/buildutil.go
@@ -29,7 +29,6 @@ import (
 
 	"mynewt.apache.org/newt/newt/pkg"
 	"mynewt.apache.org/newt/newt/project"
-	"mynewt.apache.org/newt/newt/syscfg"
 	"mynewt.apache.org/newt/util"
 )
 
@@ -91,7 +90,7 @@ func (b *Builder) TestExePath(pkgName string) string {
 func (b *Builder) FeatureString() string {
 	var buffer bytes.Buffer
 
-	featureMap := syscfg.Features(b.Cfg)
+	featureMap := b.Cfg.Features()
 	featureSlice := make([]string, 0, len(featureMap))
 	for k, _ := range featureMap {
 		featureSlice = append(featureSlice, k)

http://git-wip-us.apache.org/repos/asf/incubator-mynewt-newt/blob/9a18359c/newt/builder/load.go
----------------------------------------------------------------------
diff --git a/newt/builder/load.go b/newt/builder/load.go
index 9596532..f3d0bbe 100644
--- a/newt/builder/load.go
+++ b/newt/builder/load.go
@@ -25,7 +25,6 @@ import (
 	"path/filepath"
 
 	"mynewt.apache.org/newt/newt/project"
-	"mynewt.apache.org/newt/newt/syscfg"
 	"mynewt.apache.org/newt/util"
 )
 
@@ -84,11 +83,12 @@ func (b *Builder) Load(image_slot int, extraJtagCmd string) error {
 		envSettings += fmt.Sprintf("EXTRA_JTAG_CMD=\"%s\" ", extraJtagCmd)
 	}
 
-	// bspPath, binBaseName are passed in command line for backwards compatibility
-	downloadCmd := fmt.Sprintf("%s %s %s %s", envSettings, downloadScript, bspPath,
-		binBaseName)
+	// bspPath, binBaseName are passed in command line for backwards
+	// compatibility
+	downloadCmd := fmt.Sprintf("%s %s %s %s", envSettings, downloadScript,
+		bspPath, binBaseName)
 
-	features := syscfg.Features(b.Cfg)
+	features := b.Cfg.Features()
 
 	if _, ok := features["bootloader"]; ok {
 		util.StatusMessage(util.VERBOSITY_DEFAULT,
@@ -150,9 +150,9 @@ func (b *Builder) Debug(extraJtagCmd string, reset bool) error {
 
 	envSettings := []string{
 		fmt.Sprintf("BSP_PATH=%s", bspPath),
-	        fmt.Sprintf("BIN_BASENAME=%s", binBaseName),
+		fmt.Sprintf("BIN_BASENAME=%s", binBaseName),
 		fmt.Sprintf("FEATURES=\"%s\"", featureString),
-		}
+	}
 	if extraJtagCmd != "" {
 		envSettings = append(envSettings,
 			fmt.Sprintf("EXTRA_JTAG_CMD=%s", extraJtagCmd))

http://git-wip-us.apache.org/repos/asf/incubator-mynewt-newt/blob/9a18359c/newt/builder/targetbuild.go
----------------------------------------------------------------------
diff --git a/newt/builder/targetbuild.go b/newt/builder/targetbuild.go
index 96b43f0..5e16139 100644
--- a/newt/builder/targetbuild.go
+++ b/newt/builder/targetbuild.go
@@ -27,7 +27,6 @@ import (
 	"mynewt.apache.org/newt/newt/pkg"
 	"mynewt.apache.org/newt/newt/project"
 	"mynewt.apache.org/newt/newt/symbol"
-	"mynewt.apache.org/newt/newt/syscfg"
 	"mynewt.apache.org/newt/newt/target"
 	"mynewt.apache.org/newt/newt/toolchain"
 	"mynewt.apache.org/newt/util"
@@ -157,7 +156,7 @@ func (t *TargetBuilder) Build() error {
 	/* Build the Apps */
 	project.ResetDeps(t.AppList)
 
-	if err := t.Bsp.Reload(syscfg.Features(t.App.Cfg)); err != nil {
+	if err := t.Bsp.Reload(t.App.Cfg.Features()); err != nil {
 		return err
 	}
 
@@ -182,7 +181,7 @@ func (t *TargetBuilder) Build() error {
 	/* rebuild the loader */
 	project.ResetDeps(t.LoaderList)
 
-	if err = t.Bsp.Reload(syscfg.Features(t.Loader.Cfg)); err != nil {
+	if err = t.Bsp.Reload(t.Loader.Cfg.Features()); err != nil {
 		return err
 	}
 

http://git-wip-us.apache.org/repos/asf/incubator-mynewt-newt/blob/9a18359c/newt/syscfg/syscfg.go
----------------------------------------------------------------------
diff --git a/newt/syscfg/syscfg.go b/newt/syscfg/syscfg.go
index 8efcc8f..e8c7208 100644
--- a/newt/syscfg/syscfg.go
+++ b/newt/syscfg/syscfg.go
@@ -33,6 +33,7 @@ import (
 	log "github.com/Sirupsen/logrus"
 	"github.com/spf13/cast"
 
+	"mynewt.apache.org/newt/newt/interfaces"
 	"mynewt.apache.org/newt/newt/newtutil"
 	"mynewt.apache.org/newt/newt/pkg"
 	"mynewt.apache.org/newt/util"
@@ -88,9 +89,10 @@ type CfgRoster struct {
 }
 
 type Cfg struct {
-	Settings map[string]CfgEntry
-	Roster   CfgRoster
-	Orphans  map[string][]CfgPoint
+	Settings    map[string]CfgEntry
+	Roster      CfgRoster
+	Orphans     map[string][]CfgPoint
+	Ambiguities []CfgEntry
 }
 
 func newRoster() CfgRoster {
@@ -127,7 +129,7 @@ func ValueIsTrue(val string) bool {
 	return true
 }
 
-func Features(cfg Cfg) map[string]bool {
+func (cfg *Cfg) Features() map[string]bool {
 	features := map[string]bool{}
 	for k, v := range cfg.Settings {
 		if v.IsTrue() {
@@ -138,8 +140,8 @@ func Features(cfg Cfg) map[string]bool {
 	return features
 }
 
-func FeaturesForLpkg(cfg Cfg, lpkg *pkg.LocalPackage) map[string]bool {
-	features := Features(cfg)
+func (cfg *Cfg) FeaturesForLpkg(lpkg *pkg.LocalPackage) map[string]bool {
+	features := cfg.Features()
 
 	for k, v := range lpkg.InjectedSettings() {
 		_, ok := features[k]
@@ -168,13 +170,72 @@ func (entry *CfgEntry) IsTrue() bool {
 	return ValueIsTrue(entry.Value)
 }
 
-func appendValue(entry *CfgEntry, lpkg *pkg.LocalPackage, value interface{}) {
+func (entry *CfgEntry) appendValue(lpkg *pkg.LocalPackage, value interface{}) {
 	strval := stringValue(value)
 	point := CfgPoint{Value: strval, Source: lpkg}
 	entry.History = append(entry.History, point)
 	entry.Value = strval
 }
 
+func (entry *CfgEntry) ambiguousCount() int {
+	diffVals := false
+	count := 0
+	for i := 1; i < len(entry.History)-1; i++ {
+		cur := entry.History[len(entry.History)-i-1]
+		next := entry.History[len(entry.History)-i]
+
+		// If either setting is injected, there is no ambiguity
+		if cur.Source == nil || next.Source == nil {
+			break
+		}
+
+		// If the two package have different priorities, there is no ambiguity.
+		if normalizePkgType(cur.Source.Type()) !=
+			normalizePkgType(next.Source.Type()) {
+
+			break
+		}
+
+		if cur.Value != next.Value {
+			diffVals = true
+		}
+
+		count++
+	}
+
+	// Account for final package that was skipped in loop.
+	if count > 0 {
+		count++
+	}
+
+	// If all values are identical, there is no ambiguity
+	if !diffVals {
+		count = 0
+	}
+
+	return count
+}
+
+func (entry *CfgEntry) ambiguousText() string {
+	count := entry.ambiguousCount()
+	if count == 0 {
+		return ""
+	}
+
+	str := fmt.Sprintf("%s [", entry.Name)
+
+	for i := 0; i < count; i++ {
+		cur := entry.History[len(entry.History)-i-1]
+		if i != 0 {
+			str += ", "
+		}
+		str += fmt.Sprintf("%s:%s", cur.Name(), cur.Value)
+	}
+	str += "]"
+
+	return str
+}
+
 func FeatureToCflag(featureName string) string {
 	return fmt.Sprintf("-D%s=1", settingName(featureName))
 }
@@ -203,15 +264,16 @@ func readSetting(name string, lpkg *pkg.LocalPackage,
 		}
 	}
 
-	appendValue(&entry, lpkg, entry.Value)
+	entry.appendValue(lpkg, entry.Value)
 
 	return entry, nil
 }
 
-func readOnce(cfg Cfg, lpkg *pkg.LocalPackage, features map[string]bool) error {
+func (cfg *Cfg) readDefsOnce(lpkg *pkg.LocalPackage,
+	features map[string]bool) error {
 	v := lpkg.Viper
 
-	lfeatures := FeaturesForLpkg(cfg, lpkg)
+	lfeatures := cfg.FeaturesForLpkg(lpkg)
 	for k, _ := range features {
 		lfeatures[k] = true
 	}
@@ -234,12 +296,23 @@ func readOnce(cfg Cfg, lpkg *pkg.LocalPackage, features map[string]bool) error {
 		}
 	}
 
+	return nil
+}
+
+func (cfg *Cfg) readValsOnce(lpkg *pkg.LocalPackage,
+	features map[string]bool) error {
+	v := lpkg.Viper
+
+	lfeatures := cfg.FeaturesForLpkg(lpkg)
+	for k, _ := range features {
+		lfeatures[k] = true
+	}
 	values := newtutil.GetStringMapFeatures(v, lfeatures, "pkg.syscfg_vals")
 	if values != nil {
 		for k, v := range values {
 			entry, ok := cfg.Settings[k]
 			if ok {
-				appendValue(&entry, lpkg, v)
+				entry.appendValue(lpkg, v)
 				cfg.Settings[k] = entry
 			} else {
 				orphan := CfgPoint{
@@ -255,7 +328,7 @@ func readOnce(cfg Cfg, lpkg *pkg.LocalPackage, features map[string]bool) error {
 	return nil
 }
 
-func Log(cfg Cfg) {
+func (cfg *Cfg) Log() {
 	keys := make([]string, len(cfg.Settings))
 	i := 0
 	for k, _ := range cfg.Settings {
@@ -303,6 +376,18 @@ func Log(cfg Cfg) {
 	}
 }
 
+func (cfg *Cfg) DetectErrors() error {
+	if len(cfg.Ambiguities) == 0 {
+		return nil
+	}
+
+	str := "Syscfg ambiguities detected:"
+	for _, entry := range cfg.Ambiguities {
+		str += "\n    " + entry.ambiguousText()
+	}
+	return util.NewNewtError(str)
+}
+
 func escapeStr(s string) string {
 	s = strings.Replace(s, "/", "_", -1)
 	s = strings.Replace(s, "-", "_", -1)
@@ -335,6 +420,69 @@ func apiPresentName(apiName string) string {
 	return SYSCFG_PREFIX_API + strings.ToUpper(apiName)
 }
 
+func normalizePkgType(typ interfaces.PackageType) interfaces.PackageType {
+	switch typ {
+	case pkg.PACKAGE_TYPE_TARGET:
+		return pkg.PACKAGE_TYPE_TARGET
+	case pkg.PACKAGE_TYPE_APP:
+		return pkg.PACKAGE_TYPE_APP
+	case pkg.PACKAGE_TYPE_UNITTEST:
+		return pkg.PACKAGE_TYPE_UNITTEST
+	case pkg.PACKAGE_TYPE_BSP:
+		return pkg.PACKAGE_TYPE_BSP
+	default:
+		return pkg.PACKAGE_TYPE_LIB
+	}
+}
+
+func categorizePkgs(lpkgs []*pkg.LocalPackage) map[interfaces.PackageType][]*pkg.LocalPackage {
+	pmap := map[interfaces.PackageType][]*pkg.LocalPackage{
+		pkg.PACKAGE_TYPE_TARGET:   []*pkg.LocalPackage{},
+		pkg.PACKAGE_TYPE_APP:      []*pkg.LocalPackage{},
+		pkg.PACKAGE_TYPE_UNITTEST: []*pkg.LocalPackage{},
+		pkg.PACKAGE_TYPE_BSP:      []*pkg.LocalPackage{},
+		pkg.PACKAGE_TYPE_LIB:      []*pkg.LocalPackage{},
+	}
+
+	for _, lpkg := range lpkgs {
+		typ := normalizePkgType(lpkg.Type())
+		pmap[typ] = append(pmap[typ], lpkg)
+	}
+
+	for k, v := range pmap {
+		pmap[k] = pkg.SortLclPkgs(v)
+	}
+
+	return pmap
+}
+
+func (cfg *Cfg) readForPkgType(lpkgs []*pkg.LocalPackage,
+	features map[string]bool) error {
+
+	for _, lpkg := range lpkgs {
+		if err := cfg.readDefsOnce(lpkg, features); err != nil {
+			return err
+		}
+	}
+	for _, lpkg := range lpkgs {
+		if err := cfg.readValsOnce(lpkg, features); err != nil {
+			return err
+		}
+	}
+
+	return nil
+}
+
+func detectAmbiguities(cfg Cfg) Cfg {
+	for _, entry := range cfg.Settings {
+		if entry.ambiguousCount() > 0 {
+			cfg.Ambiguities = append(cfg.Ambiguities, entry)
+		}
+	}
+
+	return cfg
+}
+
 func Read(lpkgs []*pkg.LocalPackage, apis []string,
 	injectedSettings map[string]string, features map[string]bool) (Cfg, error) {
 
@@ -362,58 +510,29 @@ func Read(lpkgs []*pkg.LocalPackage, apis []string,
 	//     * app (if present)
 	//     * unittest (if no app)
 	//     * bsp
-
-	var app *pkg.LocalPackage
-	var bsp *pkg.LocalPackage
-	var target *pkg.LocalPackage
-	var unittest *pkg.LocalPackage
-
-	for _, lpkg := range lpkgs {
-		switch lpkg.Type() {
-		case pkg.PACKAGE_TYPE_LIB:
-			if err := readOnce(cfg, lpkg, features); err != nil {
-				return cfg, err
-			}
-
-		case pkg.PACKAGE_TYPE_APP:
-			app = lpkg
-
-		case pkg.PACKAGE_TYPE_BSP:
-			bsp = lpkg
-
-		case pkg.PACKAGE_TYPE_TARGET:
-			target = lpkg
-
-		case pkg.PACKAGE_TYPE_UNITTEST:
-			unittest = lpkg
-		}
-	}
-
-	if bsp != nil {
-		if err := readOnce(cfg, bsp, features); err != nil {
-			return cfg, err
-		}
-	}
-	if app != nil {
-		if err := readOnce(cfg, app, features); err != nil {
-			return cfg, err
-		}
-	} else if unittest != nil {
-		if err := readOnce(cfg, unittest, features); err != nil {
-			return cfg, err
-		}
-	}
-	if target != nil {
-		if err := readOnce(cfg, target, features); err != nil {
+	//     * everything else (lib, sdk, compiler)
+
+	lpkgMap := categorizePkgs(lpkgs)
+
+	for _, ptype := range []interfaces.PackageType{
+		pkg.PACKAGE_TYPE_LIB,
+		pkg.PACKAGE_TYPE_BSP,
+		pkg.PACKAGE_TYPE_UNITTEST,
+		pkg.PACKAGE_TYPE_APP,
+		pkg.PACKAGE_TYPE_TARGET,
+	} {
+		if err := cfg.readForPkgType(lpkgMap[ptype], features); err != nil {
 			return cfg, err
 		}
 	}
 
-	roster := buildCfgRoster(cfg, lpkgs, apis)
-	if err := fixupSettings(cfg, roster); err != nil {
+	buildCfgRoster(cfg, lpkgs, apis)
+	if err := fixupSettings(cfg); err != nil {
 		return cfg, err
 	}
 
+	cfg = detectAmbiguities(cfg)
+
 	return cfg, nil
 }
 
@@ -545,7 +664,7 @@ func specialValues(cfg Cfg) (apis, pkgs, settings []string) {
 }
 
 func buildCfgRoster(cfg Cfg, lpkgs []*pkg.LocalPackage,
-	apis []string) CfgRoster {
+	apis []string) {
 
 	roster := CfgRoster{
 		settings:    make(map[string]string, len(cfg.Settings)),
@@ -581,7 +700,7 @@ func buildCfgRoster(cfg Cfg, lpkgs []*pkg.LocalPackage,
 		}
 	}
 
-	return roster
+	cfg.Roster = roster
 }
 
 func settingValueToConstant(value string,
@@ -627,9 +746,9 @@ func settingValueToConstant(value string,
 	return value, false, nil
 }
 
-func fixupSettings(cfg Cfg, roster CfgRoster) error {
+func fixupSettings(cfg Cfg) error {
 	for k, entry := range cfg.Settings {
-		value, changed, err := settingValueToConstant(entry.Value, roster)
+		value, changed, err := settingValueToConstant(entry.Value, cfg.Roster)
 		if err != nil {
 			return err
 		}
@@ -686,7 +805,7 @@ func writeSettingsOnePkg(cfg Cfg, pkgName string, pkgEntries []CfgEntry,
 
 func writeSettings(cfg Cfg, w io.Writer) {
 	// Group settings by package name so that the generated header file is
-	// easier to readOnce.
+	// easier to read.
 	pkgEntries := EntriesByPkg(cfg)
 	for _, v := range cfg.Settings {
 		name := v.History[0].Name()
@@ -820,8 +939,8 @@ func EnsureWritten(cfg Cfg, lpkgs []*pkg.LocalPackage,
 		return err
 	}
 
-	cfg.Roster = buildCfgRoster(cfg, lpkgs, apis)
-	if err := fixupSettings(cfg, cfg.Roster); err != nil {
+	buildCfgRoster(cfg, lpkgs, apis)
+	if err := fixupSettings(cfg); err != nil {
 		return err
 	}