Merge pull request #2844 from Gouthamve/cobra

Move CLI commander to cobra
This commit is contained in:
Fabian Reinartz 2017-06-19 11:59:52 +02:00 committed by GitHub
commit 98c2d8477a
80 changed files with 5156 additions and 491 deletions

View file

@ -34,12 +34,13 @@ import (
"github.com/prometheus/prometheus/promql"
"github.com/prometheus/prometheus/storage/tsdb"
"github.com/prometheus/prometheus/web"
"github.com/spf13/pflag"
)
// cfg contains immutable configuration parameters for a running Prometheus
// server. It is populated by its flag set.
var cfg = struct {
fs *flag.FlagSet
fs *pflag.FlagSet
printVersion bool
configFile string
@ -47,133 +48,36 @@ var cfg = struct {
localStoragePath string
localStorageEngine string
notifier notifier.Options
notifierTimeout time.Duration
notifierTimeout model.Duration
queryEngine promql.EngineOptions
web web.Options
tsdb tsdb.Options
lookbackDelta model.Duration
webTimeout model.Duration
queryTimeout model.Duration
alertmanagerURLs stringset
prometheusURL string
// Deprecated storage flags, kept for backwards compatibility.
deprecatedMemoryChunks uint64
deprecatedMaxChunksToPersist uint64
logFormat string
logLevel string
}{
// The defaults for model.Duration flag parsing.
notifierTimeout: model.Duration(10 * time.Second),
tsdb: tsdb.Options{
MinBlockDuration: model.Duration(2 * time.Hour),
Retention: model.Duration(15 * 24 * time.Hour),
},
lookbackDelta: model.Duration(5 * time.Minute),
webTimeout: model.Duration(30 * time.Second),
queryTimeout: model.Duration(2 * time.Minute),
alertmanagerURLs: stringset{},
notifier: notifier.Options{
Registerer: prometheus.DefaultRegisterer,
},
}
func init() {
cfg.fs = flag.NewFlagSet(os.Args[0], flag.ContinueOnError)
cfg.fs.Usage = usage
cfg.fs.BoolVar(
&cfg.printVersion, "version", false,
"Print version information.",
)
cfg.fs.StringVar(
&cfg.configFile, "config.file", "prometheus.yml",
"Prometheus configuration file name.",
)
// Web.
cfg.fs.StringVar(
&cfg.web.ListenAddress, "web.listen-address", ":9090",
"Address to listen on for the web interface, API, and telemetry.",
)
cfg.fs.DurationVar(
&cfg.web.ReadTimeout, "web.read-timeout", 30*time.Second,
"Maximum duration before timing out read of the request, and closing idle connections.",
)
cfg.fs.IntVar(
&cfg.web.MaxConnections, "web.max-connections", 512,
"Maximum number of simultaneous connections.",
)
cfg.fs.StringVar(
&cfg.prometheusURL, "web.external-url", "",
"The URL under which Prometheus is externally reachable (for example, if Prometheus is served via a reverse proxy). Used for generating relative and absolute links back to Prometheus itself. If the URL has a path portion, it will be used to prefix all HTTP endpoints served by Prometheus. If omitted, relevant URL components will be derived automatically.",
)
cfg.fs.StringVar(
&cfg.web.RoutePrefix, "web.route-prefix", "",
"Prefix for the internal routes of web endpoints. Defaults to path of -web.external-url.",
)
cfg.fs.StringVar(
&cfg.web.MetricsPath, "web.telemetry-path", "/metrics",
"Path under which to expose metrics.",
)
cfg.fs.StringVar(
&cfg.web.UserAssetsPath, "web.user-assets", "",
"Path to static asset directory, available at /user.",
)
cfg.fs.BoolVar(
&cfg.web.EnableQuit, "web.enable-remote-shutdown", false,
"Enable remote service shutdown.",
)
cfg.fs.StringVar(
&cfg.web.ConsoleTemplatesPath, "web.console.templates", "consoles",
"Path to the console template directory, available at /consoles.",
)
cfg.fs.StringVar(
&cfg.web.ConsoleLibrariesPath, "web.console.libraries", "console_libraries",
"Path to the console library directory.",
)
// Storage.
cfg.fs.StringVar(
&cfg.localStoragePath, "storage.local.path", "data",
"Base path for metrics storage.",
)
cfg.fs.BoolVar(
&cfg.tsdb.NoLockfile, "storage.tsdb.no-lockfile", false,
"Disable lock file usage.",
)
cfg.fs.DurationVar(
&cfg.tsdb.MinBlockDuration, "storage.tsdb.min-block-duration", 2*time.Hour,
"Minimum duration of a data block before being persisted.",
)
cfg.fs.DurationVar(
&cfg.tsdb.MaxBlockDuration, "storage.tsdb.max-block-duration", 0,
"Maximum duration compacted blocks may span. (Defaults to 10% of the retention period)",
)
cfg.fs.DurationVar(
&cfg.tsdb.Retention, "storage.tsdb.retention", 15*24*time.Hour,
"How long to retain samples in the storage.",
)
cfg.fs.StringVar(
&cfg.localStorageEngine, "storage.local.engine", "persisted",
"Local storage engine. Supported values are: 'persisted' (full local storage with on-disk persistence) and 'none' (no local storage).",
)
// Alertmanager.
cfg.fs.IntVar(
&cfg.notifier.QueueCapacity, "alertmanager.notification-queue-capacity", 10000,
"The capacity of the queue for pending alert manager notifications.",
)
cfg.fs.DurationVar(
&cfg.notifierTimeout, "alertmanager.timeout", 10*time.Second,
"Alert manager HTTP API timeout.",
)
// Query engine.
cfg.fs.DurationVar(
&promql.StalenessDelta, "query.staleness-delta", promql.StalenessDelta,
"Staleness delta allowance during expression evaluations.",
)
cfg.fs.DurationVar(
&cfg.queryEngine.Timeout, "query.timeout", 2*time.Minute,
"Maximum time a query may take before being aborted.",
)
cfg.fs.IntVar(
&cfg.queryEngine.MaxConcurrentQueries, "query.max-concurrency", 20,
"Maximum number of queries executed concurrently.",
)
// Flags from the log package have to be added explicitly to our custom flag set.
log.AddFlags(cfg.fs)
}
func parse(args []string) error {
err := cfg.fs.Parse(args)
if err != nil || len(cfg.fs.Args()) != 0 {
@ -186,13 +90,11 @@ func parse(args []string) error {
return err
}
if promql.StalenessDelta < 0 {
return fmt.Errorf("negative staleness delta: %s", promql.StalenessDelta)
}
if err := parsePrometheusURL(); err != nil {
return err
}
cfg.web.ReadTimeout = time.Duration(cfg.webTimeout)
// Default -web.route-prefix to path of -web.external-url.
if cfg.web.RoutePrefix == "" {
cfg.web.RoutePrefix = cfg.web.ExternalURL.Path
@ -210,6 +112,12 @@ func parse(args []string) error {
cfg.tsdb.MaxBlockDuration = cfg.tsdb.Retention / 10
}
if cfg.lookbackDelta > 0 {
promql.LookbackDelta = time.Duration(cfg.lookbackDelta)
}
cfg.queryEngine.Timeout = time.Duration(cfg.queryTimeout)
return nil
}
@ -269,7 +177,7 @@ func parseAlertmanagerURLToConfig(us string) (*config.AlertmanagerConfig, error)
acfg := &config.AlertmanagerConfig{
Scheme: u.Scheme,
PathPrefix: u.Path,
Timeout: cfg.notifierTimeout,
Timeout: time.Duration(cfg.notifierTimeout),
ServiceDiscoveryConfig: config.ServiceDiscoveryConfig{
StaticConfigs: []*config.TargetGroup{
{
@ -335,10 +243,10 @@ func usage() {
})
t = template.Must(t.Parse(helpTmpl))
groups := make(map[string][]*flag.Flag)
groups := make(map[string][]*pflag.Flag)
// Bucket flags into groups based on the first of their dot-separated levels.
cfg.fs.VisitAll(func(fl *flag.Flag) {
cfg.fs.VisitAll(func(fl *pflag.Flag) {
parts := strings.SplitN(fl.Name, ".", 2)
if len(parts) == 1 {
groups["."] = append(groups["."], fl)

View file

@ -25,15 +25,15 @@ func TestParse(t *testing.T) {
valid: true,
},
{
input: []string{"-web.external-url", ""},
input: []string{"--web.external-url", ""},
valid: true,
},
{
input: []string{"-web.external-url", "http://proxy.com/prometheus"},
input: []string{"--web.external-url", "http://proxy.com/prometheus"},
valid: true,
},
{
input: []string{"-web.external-url", "'https://url/prometheus'"},
input: []string{"--web.external-url", "'https://url/prometheus'"},
valid: false,
},
}
@ -43,6 +43,8 @@ func TestParse(t *testing.T) {
cfg.prometheusURL = ""
cfg.alertmanagerURLs = stringset{}
newRootCmd() // To register the flags and flagset.
err := parse(test.input)
if test.valid && err != nil {
t.Errorf("%d. expected input to be valid, got %s", i, err)

View file

@ -15,7 +15,6 @@
package main
import (
"flag"
"fmt"
_ "net/http/pprof" // Comment this line to disable pprof endpoint.
"os"
@ -26,6 +25,8 @@ import (
"github.com/prometheus/client_golang/prometheus"
"github.com/prometheus/common/log"
"github.com/prometheus/common/version"
"github.com/spf13/cobra"
"github.com/spf13/pflag"
"golang.org/x/net/context"
"github.com/prometheus/prometheus/config"
@ -38,7 +39,7 @@ import (
)
func main() {
os.Exit(Main())
newRootCmd().Execute()
}
var (
@ -58,6 +59,123 @@ func init() {
prometheus.MustRegister(version.NewCollector("prometheus"))
}
func newRootCmd() *cobra.Command {
rootCmd := &cobra.Command{
Use: "prometheus",
Short: "prometheus --config.file=prometheus.yaml",
Run: func(cmd *cobra.Command, args []string) {
os.Exit(Main())
},
}
rootCmd.PersistentFlags().BoolVar(
&cfg.printVersion, "version", false,
"Print version information.",
)
rootCmd.PersistentFlags().StringVar(
&cfg.configFile, "config.file", "prometheus.yml",
"Prometheus configuration file name.",
)
// Web.
rootCmd.PersistentFlags().StringVar(
&cfg.web.ListenAddress, "web.listen-address", ":9090",
"Address to listen on for the web interface, API, and telemetry.",
)
rootCmd.PersistentFlags().Var(
&cfg.webTimeout, "web.read-timeout",
"Maximum duration before timing out read of the request, and closing idle connections.",
)
rootCmd.PersistentFlags().IntVar(
&cfg.web.MaxConnections, "web.max-connections", 512,
"Maximum number of simultaneous connections.",
)
rootCmd.PersistentFlags().StringVar(
&cfg.prometheusURL, "web.external-url", "",
"The URL under which Prometheus is externally reachable (for example, if Prometheus is served via a reverse proxy). Used for generating relative and absolute links back to Prometheus itself. If the URL has a path portion, it will be used to prefix all HTTP endpoints served by Prometheus. If omitted, relevant URL components will be derived automatically.",
)
rootCmd.PersistentFlags().StringVar(
&cfg.web.RoutePrefix, "web.route-prefix", "",
"Prefix for the internal routes of web endpoints. Defaults to path of -web.external-url.",
)
rootCmd.PersistentFlags().StringVar(
&cfg.web.UserAssetsPath, "web.user-assets", "",
"Path to static asset directory, available at /user.",
)
rootCmd.PersistentFlags().BoolVar(
&cfg.web.EnableQuit, "web.enable-remote-shutdown", false,
"Enable remote service shutdown.",
)
rootCmd.PersistentFlags().StringVar(
&cfg.web.ConsoleTemplatesPath, "web.console.templates", "consoles",
"Path to the console template directory, available at /consoles.",
)
rootCmd.PersistentFlags().StringVar(
&cfg.web.ConsoleLibrariesPath, "web.console.libraries", "console_libraries",
"Path to the console library directory.",
)
// Storage.
rootCmd.PersistentFlags().StringVar(
&cfg.localStoragePath, "storage.tsdb.path", "data",
"Base path for metrics storage.",
)
rootCmd.PersistentFlags().BoolVar(
&cfg.tsdb.NoLockfile, "storage.tsdb.no-lockfile", false,
"Disable lock file usage.",
)
rootCmd.PersistentFlags().Var(
&cfg.tsdb.MinBlockDuration, "storage.tsdb.min-block-duration",
"Minimum duration of a data block before being persisted.",
)
rootCmd.PersistentFlags().Var(
&cfg.tsdb.MaxBlockDuration, "storage.tsdb.max-block-duration",
"Maximum duration compacted blocks may span. (Defaults to 10% of the retention period)",
)
rootCmd.PersistentFlags().Var(
&cfg.tsdb.Retention, "storage.tsdb.retention",
"How long to retain samples in the storage.",
)
// Alertmanager.
rootCmd.PersistentFlags().IntVar(
&cfg.notifier.QueueCapacity, "alertmanager.notification-queue-capacity", 10000,
"The capacity of the queue for pending alert manager notifications.",
)
rootCmd.PersistentFlags().Var(
&cfg.notifierTimeout, "alertmanager.timeout",
"Alert manager HTTP API timeout.",
)
// Query engine.
rootCmd.PersistentFlags().Var(
&cfg.lookbackDelta, "query.lookback-delta",
"The delta difference allowed for retrieving metrics during expression evaluations.",
)
rootCmd.PersistentFlags().Var(
&cfg.queryTimeout, "query.timeout",
"Maximum time a query may take before being aborted.",
)
rootCmd.PersistentFlags().IntVar(
&cfg.queryEngine.MaxConcurrentQueries, "query.max-concurrency", 20,
"Maximum number of queries executed concurrently.",
)
// Logging.
rootCmd.PersistentFlags().StringVar(
&cfg.logLevel, "log.level", "info",
"Only log messages with the given severity or above. Valid levels: [debug, info, warn, error, fatal]",
)
rootCmd.PersistentFlags().StringVar(
&cfg.logFormat, "log.format", "logger:stderr",
`Set the log target and format. Example: "logger:syslog?appname=bob&local=7" or "logger:stdout?json=true"`,
)
cfg.fs = rootCmd.PersistentFlags()
return rootCmd
}
// Main manages the stup and shutdown lifecycle of the entire Prometheus server.
func Main() int {
if err := parse(os.Args[1:]); err != nil {
@ -65,14 +183,18 @@ func Main() int {
return 2
}
logger := log.NewLogger(os.Stdout)
logger.SetLevel(cfg.logLevel)
logger.SetFormat(cfg.logFormat)
if cfg.printVersion {
fmt.Fprintln(os.Stdout, version.Print("prometheus"))
return 0
}
log.Infoln("Starting prometheus", version.Info())
log.Infoln("Build context", version.BuildContext())
log.Infoln("Host details", Uname())
logger.Infoln("Starting prometheus", version.Info())
logger.Infoln("Build context", version.BuildContext())
logger.Infoln("Host details", Uname())
var (
// sampleAppender = storage.Fanout{}
@ -84,19 +206,22 @@ func Main() int {
hup := make(chan os.Signal)
hupReady := make(chan bool)
signal.Notify(hup, syscall.SIGHUP)
logger.Infoln("Starting tsdb")
localStorage, err := tsdb.Open(cfg.localStoragePath, prometheus.DefaultRegisterer, &cfg.tsdb)
if err != nil {
log.Errorf("Opening storage failed: %s", err)
return 1
}
logger.Infoln("tsdb started")
// remoteStorage := &remote.Storage{}
// sampleAppender = append(sampleAppender, remoteStorage)
// reloadables = append(reloadables, remoteStorage)
cfg.queryEngine.Logger = logger
var (
notifier = notifier.New(&cfg.notifier, log.Base())
targetManager = retrieval.NewTargetManager(localStorage, log.Base())
notifier = notifier.New(&cfg.notifier, logger)
targetManager = retrieval.NewTargetManager(localStorage, logger)
queryEngine = promql.NewEngine(localStorage, &cfg.queryEngine)
ctx, cancelCtx = context.WithCancel(context.Background())
)
@ -107,6 +232,7 @@ func Main() int {
QueryEngine: queryEngine,
Context: ctx,
ExternalURL: cfg.web.ExternalURL,
Logger: logger,
})
cfg.web.Context = ctx
@ -126,7 +252,7 @@ func Main() int {
}
cfg.web.Flags = map[string]string{}
cfg.fs.VisitAll(func(f *flag.Flag) {
cfg.fs.VisitAll(func(f *pflag.Flag) {
cfg.web.Flags[f.Name] = f.Value.String()
})
@ -134,8 +260,8 @@ func Main() int {
reloadables = append(reloadables, targetManager, ruleManager, webHandler, notifier)
if err := reloadConfig(cfg.configFile, reloadables...); err != nil {
log.Errorf("Error loading config: %s", err)
if err := reloadConfig(cfg.configFile, logger, reloadables...); err != nil {
logger.Errorf("Error loading config: %s", err)
return 1
}
@ -147,12 +273,12 @@ func Main() int {
for {
select {
case <-hup:
if err := reloadConfig(cfg.configFile, reloadables...); err != nil {
log.Errorf("Error reloading config: %s", err)
if err := reloadConfig(cfg.configFile, logger, reloadables...); err != nil {
logger.Errorf("Error reloading config: %s", err)
}
case rc := <-webHandler.Reload():
if err := reloadConfig(cfg.configFile, reloadables...); err != nil {
log.Errorf("Error reloading config: %s", err)
if err := reloadConfig(cfg.configFile, logger, reloadables...); err != nil {
logger.Errorf("Error reloading config: %s", err)
rc <- err
} else {
rc <- nil
@ -164,7 +290,7 @@ func Main() int {
// Start all components. The order is NOT arbitrary.
defer func() {
if err := localStorage.Close(); err != nil {
log.Errorln("Error stopping storage:", err)
logger.Errorln("Error stopping storage:", err)
}
}()
@ -197,14 +323,14 @@ func Main() int {
signal.Notify(term, os.Interrupt, syscall.SIGTERM)
select {
case <-term:
log.Warn("Received SIGTERM, exiting gracefully...")
logger.Warn("Received SIGTERM, exiting gracefully...")
case <-webHandler.Quit():
log.Warn("Received termination request via web service, exiting gracefully...")
logger.Warn("Received termination request via web service, exiting gracefully...")
case err := <-webHandler.ListenError():
log.Errorln("Error starting web server, exiting gracefully:", err)
logger.Errorln("Error starting web server, exiting gracefully:", err)
}
log.Info("See you next time!")
logger.Info("See you next time!")
return 0
}
@ -214,8 +340,8 @@ type Reloadable interface {
ApplyConfig(*config.Config) error
}
func reloadConfig(filename string, rls ...Reloadable) (err error) {
log.Infof("Loading configuration file %s", filename)
func reloadConfig(filename string, logger log.Logger, rls ...Reloadable) (err error) {
logger.Infof("Loading configuration file %s", filename)
defer func() {
if err == nil {
configSuccess.Set(1)
@ -233,7 +359,7 @@ func reloadConfig(filename string, rls ...Reloadable) (err error) {
failed := false
for _, rl := range rls {
if err := rl.ApplyConfig(conf); err != nil {
log.Error("Failed to apply configuration: ", err)
logger.Error("Failed to apply configuration: ", err)
failed = true
}
}

View file

@ -77,7 +77,7 @@ func NewDiscovery(
logger: logger,
}
for _, path := range paths {
sd.treeCaches = append(sd.treeCaches, treecache.NewZookeeperTreeCache(conn, path, updates))
sd.treeCaches = append(sd.treeCaches, treecache.NewZookeeperTreeCache(conn, path, updates, logger))
}
return sd
}

View file

@ -205,6 +205,8 @@ type Engine struct {
// The gate limiting the maximum number of concurrent and waiting queries.
gate *queryGate
options *EngineOptions
logger log.Logger
}
// Queryable allows opening a storage querier.
@ -222,6 +224,7 @@ func NewEngine(queryable Queryable, o *EngineOptions) *Engine {
queryable: queryable,
gate: newQueryGate(o.MaxConcurrentQueries),
options: o,
logger: o.Logger,
}
}
@ -229,12 +232,14 @@ func NewEngine(queryable Queryable, o *EngineOptions) *Engine {
type EngineOptions struct {
MaxConcurrentQueries int
Timeout time.Duration
Logger log.Logger
}
// DefaultEngineOptions are the default engine options.
var DefaultEngineOptions = &EngineOptions{
MaxConcurrentQueries: 20,
Timeout: 2 * time.Minute,
Logger: log.Base(),
}
// NewInstantQuery returns an evaluation query for the given expression at the given time.
@ -374,6 +379,7 @@ func (ng *Engine) execEvalStmt(ctx context.Context, query *query, s *EvalStmt) (
evaluator := &evaluator{
Timestamp: start,
ctx: ctx,
logger: ng.logger,
}
val, err := evaluator.Eval(s.Expr)
if err != nil {
@ -409,6 +415,7 @@ func (ng *Engine) execEvalStmt(ctx context.Context, query *query, s *EvalStmt) (
evaluator := &evaluator{
Timestamp: t,
ctx: ctx,
logger: ng.logger,
}
val, err := evaluator.Eval(s.Expr)
if err != nil {
@ -480,11 +487,11 @@ func (ng *Engine) populateIterators(ctx context.Context, s *EvalStmt) (storage.Q
Inspect(s.Expr, func(node Node) bool {
switch n := node.(type) {
case *VectorSelector:
if maxOffset < StalenessDelta {
maxOffset = StalenessDelta
if maxOffset < LookbackDelta {
maxOffset = LookbackDelta
}
if n.Offset+StalenessDelta > maxOffset {
maxOffset = n.Offset + StalenessDelta
if n.Offset+LookbackDelta > maxOffset {
maxOffset = n.Offset + LookbackDelta
}
case *MatrixSelector:
if maxOffset < n.Range {
@ -510,18 +517,18 @@ func (ng *Engine) populateIterators(ctx context.Context, s *EvalStmt) (storage.Q
n.series, err = expandSeriesSet(querier.Select(n.LabelMatchers...))
if err != nil {
// TODO(fabxc): use multi-error.
log.Errorln("expand series set:", err)
ng.logger.Errorln("expand series set:", err)
return false
}
for _, s := range n.series {
it := storage.NewBuffer(s.Iterator(), durationMilliseconds(StalenessDelta))
it := storage.NewBuffer(s.Iterator(), durationMilliseconds(LookbackDelta))
n.iterators = append(n.iterators, it)
}
case *MatrixSelector:
n.series, err = expandSeriesSet(querier.Select(n.LabelMatchers...))
if err != nil {
log.Errorln("expand series set:", err)
ng.logger.Errorln("expand series set:", err)
return false
}
for _, s := range n.series {
@ -550,6 +557,8 @@ type evaluator struct {
Timestamp int64 // time in milliseconds
finalizers []func()
logger log.Logger
}
func (ev *evaluator) close() {
@ -577,7 +586,7 @@ func (ev *evaluator) recover(errp *error) {
buf := make([]byte, 64<<10)
buf = buf[:runtime.Stack(buf, false)]
log.Errorf("parser panic: %v\n%s", e, buf)
ev.logger.Errorf("parser panic: %v\n%s", e, buf)
*errp = fmt.Errorf("unexpected error")
} else {
*errp = e.(error)
@ -754,8 +763,8 @@ func (ev *evaluator) vectorSelector(node *VectorSelector) Vector {
peek := 1
if !ok || t > refTime {
t, v, ok = it.PeekBack(peek)
peek += 1
if !ok || t < refTime-durationMilliseconds(StalenessDelta) {
peek++
if !ok || t < refTime-durationMilliseconds(LookbackDelta) {
continue
}
}
@ -764,7 +773,7 @@ func (ev *evaluator) vectorSelector(node *VectorSelector) Vector {
}
// Find timestamp before this point, within the staleness delta.
prevT, _, ok := it.PeekBack(peek)
if ok && prevT >= refTime-durationMilliseconds(StalenessDelta) {
if ok && prevT >= refTime-durationMilliseconds(LookbackDelta) {
interval := t - prevT
if interval*4+interval/10 < refTime-t {
// It is more than 4 (+10% for safety) intervals
@ -1451,9 +1460,9 @@ func shouldDropMetricName(op itemType) bool {
}
}
// StalenessDelta determines the time since the last sample after which a time
// LookbackDelta determines the time since the last sample after which a time
// series is considered stale.
var StalenessDelta = 5 * time.Minute
var LookbackDelta = 5 * time.Minute
// A queryGate controls the maximum number of concurrently running and waiting queries.
type queryGate struct {

View file

@ -21,6 +21,7 @@ import (
"golang.org/x/net/context"
"github.com/prometheus/common/log"
"github.com/prometheus/prometheus/pkg/labels"
)
@ -295,7 +296,9 @@ load 10s
}
func TestRecoverEvaluatorRuntime(t *testing.T) {
var ev *evaluator
ev := &evaluator{
logger: log.Base(),
}
var err error
defer ev.recover(&err)
@ -309,7 +312,7 @@ func TestRecoverEvaluatorRuntime(t *testing.T) {
}
func TestRecoverEvaluatorError(t *testing.T) {
var ev *evaluator
ev := &evaluator{logger: log.Base()}
var err error
e := fmt.Errorf("custom error")

View file

@ -112,13 +112,15 @@ type scrapePool struct {
// Constructor for new scrape loops. This is settable for testing convenience.
newLoop func(context.Context, scraper, func() storage.Appender, func() storage.Appender, log.Logger) loop
logger log.Logger
}
func newScrapePool(ctx context.Context, cfg *config.ScrapeConfig, app Appendable) *scrapePool {
func newScrapePool(ctx context.Context, cfg *config.ScrapeConfig, app Appendable, logger log.Logger) *scrapePool {
client, err := httputil.NewClientFromConfig(cfg.HTTPClientConfig)
if err != nil {
// Any errors that could occur here should be caught during config validation.
log.Errorf("Error creating HTTP client for job %q: %s", cfg.JobName, err)
logger.Errorf("Error creating HTTP client for job %q: %s", cfg.JobName, err)
}
newLoop := func(
@ -138,6 +140,7 @@ func newScrapePool(ctx context.Context, cfg *config.ScrapeConfig, app Appendable
targets: map[uint64]*Target{},
loops: map[uint64]loop{},
newLoop: newLoop,
logger: logger,
}
}
@ -175,7 +178,7 @@ func (sp *scrapePool) reload(cfg *config.ScrapeConfig) {
client, err := httputil.NewClientFromConfig(cfg.HTTPClientConfig)
if err != nil {
// Any errors that could occur here should be caught during config validation.
log.Errorf("Error creating HTTP client for job %q: %s", cfg.JobName, err)
sp.logger.Errorf("Error creating HTTP client for job %q: %s", cfg.JobName, err)
}
sp.config = cfg
sp.client = client
@ -197,7 +200,7 @@ func (sp *scrapePool) reload(cfg *config.ScrapeConfig) {
func() storage.Appender {
return sp.reportAppender(t)
},
log.With("target", t.labels.String()),
sp.logger.With("target", t.labels.String()),
)
)
wg.Add(1)
@ -227,7 +230,7 @@ func (sp *scrapePool) Sync(tgs []*config.TargetGroup) {
for _, tg := range tgs {
targets, err := targetsFromGroup(tg, sp.config)
if err != nil {
log.With("err", err).Error("creating targets failed")
sp.logger.With("err", err).Error("creating targets failed")
continue
}
all = append(all, targets...)
@ -267,7 +270,7 @@ func (sp *scrapePool) sync(targets []*Target) {
func() storage.Appender {
return sp.reportAppender(t)
},
log.With("target", t.labels.String()),
sp.logger.With("target", t.labels.String()),
)
sp.targets[hash] = t

View file

@ -44,7 +44,7 @@ func TestNewScrapePool(t *testing.T) {
var (
app = &nopAppendable{}
cfg = &config.ScrapeConfig{}
sp = newScrapePool(context.Background(), cfg, app)
sp = newScrapePool(context.Background(), cfg, app, log.Base())
)
if a, ok := sp.appendable.(*nopAppendable); !ok || a != app {
@ -167,6 +167,7 @@ func TestScrapePoolReload(t *testing.T) {
targets: map[uint64]*Target{},
loops: map[uint64]loop{},
newLoop: newLoop,
logger: log.Base(),
}
// Reloading a scrape pool with a new scrape configuration must stop all scrape
@ -236,7 +237,7 @@ func TestScrapePoolReportAppender(t *testing.T) {
target := newTestTarget("example.com:80", 10*time.Millisecond, nil)
app := &nopAppendable{}
sp := newScrapePool(context.Background(), cfg, app)
sp := newScrapePool(context.Background(), cfg, app, log.Base())
cfg.HonorLabels = false
wrapped := sp.reportAppender(target)
@ -271,7 +272,7 @@ func TestScrapePoolSampleAppender(t *testing.T) {
target := newTestTarget("example.com:80", 10*time.Millisecond, nil)
app := &nopAppendable{}
sp := newScrapePool(context.Background(), cfg, app)
sp := newScrapePool(context.Background(), cfg, app, log.Base())
cfg.HonorLabels = false
wrapped := sp.sampleAppender(target)

View file

@ -106,7 +106,7 @@ func (tm *TargetManager) reload() {
ts = &targetSet{
ctx: ctx,
cancel: cancel,
sp: newScrapePool(ctx, scfg, tm.append),
sp: newScrapePool(ctx, scfg, tm.append, tm.logger),
}
ts.ts = discovery.NewTargetSet(ts.sp)

View file

@ -102,10 +102,12 @@ type AlertingRule struct {
// A map of alerts which are currently active (Pending or Firing), keyed by
// the fingerprint of the labelset they correspond to.
active map[uint64]*Alert
logger log.Logger
}
// NewAlertingRule constructs a new AlertingRule.
func NewAlertingRule(name string, vec promql.Expr, hold time.Duration, lbls, anns labels.Labels) *AlertingRule {
func NewAlertingRule(name string, vec promql.Expr, hold time.Duration, lbls, anns labels.Labels, logger log.Logger) *AlertingRule {
return &AlertingRule{
name: name,
vector: vec,
@ -113,6 +115,7 @@ func NewAlertingRule(name string, vec promql.Expr, hold time.Duration, lbls, ann
labels: lbls,
annotations: anns,
active: map[uint64]*Alert{},
logger: logger.With("alert", name),
}
}
@ -197,7 +200,7 @@ func (r *AlertingRule) Eval(ctx context.Context, ts time.Time, engine *promql.En
result, err := tmpl.Expand()
if err != nil {
result = fmt.Sprintf("<error expanding template: %s>", err)
log.Warnf("Error expanding alert template %v with data '%v': %s", r.Name(), tmplData, err)
r.logger.Warnf("Error expanding alert template %v with data '%v': %s", r.Name(), tmplData, err)
}
return result
}

View file

@ -16,6 +16,7 @@ package rules
import (
"testing"
"github.com/prometheus/common/log"
"github.com/prometheus/prometheus/pkg/labels"
"github.com/prometheus/prometheus/promql"
)
@ -25,7 +26,7 @@ func TestAlertingRuleHTMLSnippet(t *testing.T) {
if err != nil {
t.Fatal(err)
}
rule := NewAlertingRule("testrule", expr, 0, labels.FromStrings("html", "<b>BOLD</b>"), labels.FromStrings("html", "<b>BOLD</b>"))
rule := NewAlertingRule("testrule", expr, 0, labels.FromStrings("html", "<b>BOLD</b>"), labels.FromStrings("html", "<b>BOLD</b>"), log.Base())
const want = `ALERT <a href="/test/prefix/graph?g0.expr=ALERTS%7Balertname%3D%22testrule%22%7D&g0.tab=0">testrule</a>
IF <a href="/test/prefix/graph?g0.expr=foo%7Bhtml%3D%22%3Cb%3EBOLD%3Cb%3E%22%7D&g0.tab=0">foo{html=&#34;&lt;b&gt;BOLD&lt;b&gt;&#34;}</a>

View file

@ -134,6 +134,8 @@ type Group struct {
done chan struct{}
terminated chan struct{}
logger log.Logger
}
// NewGroup makes a new Group with the given name, options, and rules.
@ -146,6 +148,7 @@ func NewGroup(name string, interval time.Duration, rules []Rule, opts *ManagerOp
seriesInPreviousEval: make([]map[string]labels.Labels, len(rules)),
done: make(chan struct{}),
terminated: make(chan struct{}),
logger: opts.Logger.With("group", name),
}
}
@ -293,7 +296,7 @@ func (g *Group) Eval(ts time.Time) {
// Canceled queries are intentional termination of queries. This normally
// happens on shutdown and thus we skip logging of any errors here.
if _, ok := err.(promql.ErrQueryCanceled); !ok {
log.Warnf("Error while evaluating rule %q: %s", rule, err)
g.logger.Warnf("Error while evaluating rule %q: %s", rule, err)
}
evalFailures.WithLabelValues(rtyp).Inc()
return
@ -309,7 +312,7 @@ func (g *Group) Eval(ts time.Time) {
app, err := g.opts.Appendable.Appender()
if err != nil {
log.With("err", err).Warn("creating appender failed")
g.logger.With("err", err).Warn("creating appender failed")
return
}
@ -319,22 +322,22 @@ func (g *Group) Eval(ts time.Time) {
switch err {
case storage.ErrOutOfOrderSample:
numOutOfOrder++
log.With("sample", s).With("err", err).Debug("Rule evaluation result discarded")
g.logger.With("sample", s).With("err", err).Debug("Rule evaluation result discarded")
case storage.ErrDuplicateSampleForTimestamp:
numDuplicates++
log.With("sample", s).With("err", err).Debug("Rule evaluation result discarded")
g.logger.With("sample", s).With("err", err).Debug("Rule evaluation result discarded")
default:
log.With("sample", s).With("err", err).Warn("Rule evaluation result discarded")
g.logger.With("sample", s).With("err", err).Warn("Rule evaluation result discarded")
}
} else {
seriesReturned[s.Metric.String()] = s.Metric
}
}
if numOutOfOrder > 0 {
log.With("numDropped", numOutOfOrder).Warn("Error on ingesting out-of-order result from rule evaluation")
g.logger.With("numDropped", numOutOfOrder).Warn("Error on ingesting out-of-order result from rule evaluation")
}
if numDuplicates > 0 {
log.With("numDropped", numDuplicates).Warn("Error on ingesting results from rule evaluation with different value but same timestamp")
g.logger.With("numDropped", numDuplicates).Warn("Error on ingesting results from rule evaluation with different value but same timestamp")
}
for metric, lset := range g.seriesInPreviousEval[i] {
@ -347,12 +350,12 @@ func (g *Group) Eval(ts time.Time) {
// Do not count these in logging, as this is expected if series
// is exposed from a different rule.
default:
log.With("sample", metric).With("err", err).Warn("adding stale sample failed")
g.logger.With("sample", metric).With("err", err).Warn("adding stale sample failed")
}
}
}
if err := app.Commit(); err != nil {
log.With("err", err).Warn("rule sample appending failed")
g.logger.With("err", err).Warn("rule sample appending failed")
} else {
g.seriesInPreviousEval[i] = seriesReturned
}
@ -397,6 +400,8 @@ type Manager struct {
groups map[string]*Group
mtx sync.RWMutex
block chan struct{}
logger log.Logger
}
type Appendable interface {
@ -410,17 +415,18 @@ type ManagerOptions struct {
Context context.Context
Notifier *notifier.Notifier
Appendable Appendable
Logger log.Logger
}
// NewManager returns an implementation of Manager, ready to be started
// by calling the Run method.
func NewManager(o *ManagerOptions) *Manager {
manager := &Manager{
return &Manager{
groups: map[string]*Group{},
opts: o,
block: make(chan struct{}),
logger: o.Logger,
}
return manager
}
// Run starts processing of the rule manager.
@ -433,13 +439,13 @@ func (m *Manager) Stop() {
m.mtx.Lock()
defer m.mtx.Unlock()
log.Info("Stopping rule manager...")
m.logger.Info("Stopping rule manager...")
for _, eg := range m.groups {
eg.stop()
}
log.Info("Rule manager stopped.")
m.logger.Info("Rule manager stopped.")
}
// ApplyConfig updates the rule manager's state as the config requires. If
@ -522,7 +528,7 @@ func (m *Manager) loadGroups(interval time.Duration, filenames ...string) (map[s
switch r := stmt.(type) {
case *promql.AlertStmt:
rule = NewAlertingRule(r.Name, r.Expr, r.Duration, r.Labels, r.Annotations)
rule = NewAlertingRule(r.Name, r.Expr, r.Duration, r.Labels, r.Annotations, m.logger)
case *promql.RecordStmt:
rule = NewRecordingRule(r.Name, r.Expr, r.Labels)

View file

@ -22,6 +22,7 @@ import (
"testing"
"time"
"github.com/prometheus/common/log"
"github.com/prometheus/common/model"
"github.com/prometheus/prometheus/pkg/labels"
@ -58,6 +59,7 @@ func TestAlertingRule(t *testing.T) {
time.Minute,
labels.FromStrings("severity", "{{\"c\"}}ritical"),
nil,
log.Base(),
)
baseTime := time.Unix(0, 0)
@ -165,6 +167,7 @@ func TestStaleness(t *testing.T) {
QueryEngine: engine,
Appendable: storage,
Context: context.Background(),
Logger: log.Base(),
}
expr, err := promql.ParseExpr("a + 1")
@ -241,7 +244,7 @@ func readSeriesSet(ss storage.SeriesSet) (map[string][]promql.Point, error) {
func TestCopyState(t *testing.T) {
oldGroup := &Group{
rules: []Rule{
NewAlertingRule("alert", nil, 0, nil, nil),
NewAlertingRule("alert", nil, 0, nil, nil, log.Base()),
NewRecordingRule("rule1", nil, nil),
NewRecordingRule("rule2", nil, nil),
NewRecordingRule("rule3", nil, nil),
@ -261,7 +264,7 @@ func TestCopyState(t *testing.T) {
NewRecordingRule("rule3", nil, nil),
NewRecordingRule("rule3", nil, nil),
NewRecordingRule("rule3", nil, nil),
NewAlertingRule("alert", nil, 0, nil, nil),
NewAlertingRule("alert", nil, 0, nil, nil, log.Base()),
NewRecordingRule("rule1", nil, nil),
NewRecordingRule("rule4", nil, nil),
},

View file

@ -19,6 +19,7 @@ import (
"github.com/pkg/errors"
"github.com/prometheus/client_golang/prometheus"
"github.com/prometheus/common/model"
"github.com/prometheus/prometheus/pkg/labels"
"github.com/prometheus/prometheus/storage"
"github.com/prometheus/tsdb"
@ -37,13 +38,13 @@ type Options struct {
// The timestamp range of head blocks after which they get persisted.
// It's the minimum duration of any persisted block.
MinBlockDuration time.Duration
MinBlockDuration model.Duration
// The maximum timestamp range of compacted blocks.
MaxBlockDuration time.Duration
MaxBlockDuration model.Duration
// Duration for how long to retain data.
Retention time.Duration
Retention model.Duration
// Disable creation and consideration of lockfile.
NoLockfile bool
@ -53,9 +54,9 @@ type Options struct {
func Open(path string, r prometheus.Registerer, opts *Options) (storage.Storage, error) {
db, err := tsdb.Open(path, nil, r, &tsdb.Options{
WALFlushInterval: 10 * time.Second,
MinBlockDuration: uint64(opts.MinBlockDuration.Seconds() * 1000),
MaxBlockDuration: uint64(opts.MaxBlockDuration.Seconds() * 1000),
RetentionDuration: uint64(opts.Retention.Seconds() * 1000),
MinBlockDuration: uint64(time.Duration(opts.MinBlockDuration).Seconds() * 1000),
MaxBlockDuration: uint64(time.Duration(opts.MaxBlockDuration).Seconds() * 1000),
RetentionDuration: uint64(time.Duration(opts.Retention).Seconds() * 1000),
NoLockfile: opts.NoLockfile,
})
if err != nil {

View file

@ -19,6 +19,7 @@ import (
"time"
"github.com/prometheus/common/log"
"github.com/prometheus/common/model"
"github.com/prometheus/prometheus/storage"
"github.com/prometheus/prometheus/storage/tsdb"
)
@ -36,8 +37,8 @@ func NewStorage(t T) storage.Storage {
// Tests just load data for a series sequentially. Thus we
// need a long appendable window.
db, err := tsdb.Open(dir, nil, &tsdb.Options{
MinBlockDuration: 24 * time.Hour,
MaxBlockDuration: 24 * time.Hour,
MinBlockDuration: model.Duration(24 * time.Hour),
MaxBlockDuration: model.Duration(24 * time.Hour),
})
if err != nil {
t.Fatalf("Opening test storage failed: %s", err)

View file

@ -45,11 +45,12 @@ func init() {
}
type ZookeeperLogger struct {
logger log.Logger
}
// Implements zk.Logger
func (zl ZookeeperLogger) Printf(s string, i ...interface{}) {
log.Infof(s, i...)
zl.logger.Infof(s, i...)
}
type ZookeeperTreeCache struct {
@ -59,6 +60,8 @@ type ZookeeperTreeCache struct {
zkEvents chan zk.Event
stop chan struct{}
head *zookeeperTreeCacheNode
logger log.Logger
}
type ZookeeperTreeCacheEvent struct {
@ -74,12 +77,14 @@ type zookeeperTreeCacheNode struct {
children map[string]*zookeeperTreeCacheNode
}
func NewZookeeperTreeCache(conn *zk.Conn, path string, events chan ZookeeperTreeCacheEvent) *ZookeeperTreeCache {
func NewZookeeperTreeCache(conn *zk.Conn, path string, events chan ZookeeperTreeCacheEvent, logger log.Logger) *ZookeeperTreeCache {
tc := &ZookeeperTreeCache{
conn: conn,
prefix: path,
events: events,
stop: make(chan struct{}),
logger: logger,
}
tc.head = &zookeeperTreeCacheNode{
events: make(chan zk.Event),
@ -108,20 +113,20 @@ func (tc *ZookeeperTreeCache) loop(path string) {
err := tc.recursiveNodeUpdate(path, tc.head)
if err != nil {
log.Errorf("Error during initial read of Zookeeper: %s", err)
tc.logger.Errorf("Error during initial read of Zookeeper: %s", err)
failure()
}
for {
select {
case ev := <-tc.head.events:
log.Debugf("Received Zookeeper event: %s", ev)
tc.logger.Debugf("Received Zookeeper event: %s", ev)
if failureMode {
continue
}
if ev.Type == zk.EventNotWatching {
log.Infof("Lost connection to Zookeeper.")
tc.logger.Infof("Lost connection to Zookeeper.")
failure()
} else {
path := strings.TrimPrefix(ev.Path, tc.prefix)
@ -142,15 +147,15 @@ func (tc *ZookeeperTreeCache) loop(path string) {
err := tc.recursiveNodeUpdate(ev.Path, node)
if err != nil {
log.Errorf("Error during processing of Zookeeper event: %s", err)
tc.logger.Errorf("Error during processing of Zookeeper event: %s", err)
failure()
} else if tc.head.data == nil {
log.Errorf("Error during processing of Zookeeper event: path %s no longer exists", tc.prefix)
tc.logger.Errorf("Error during processing of Zookeeper event: path %s no longer exists", tc.prefix)
failure()
}
}
case <-retryChan:
log.Infof("Attempting to resync state with Zookeeper")
tc.logger.Infof("Attempting to resync state with Zookeeper")
previousState := &zookeeperTreeCacheNode{
children: tc.head.children,
}
@ -158,13 +163,13 @@ func (tc *ZookeeperTreeCache) loop(path string) {
tc.head.children = make(map[string]*zookeeperTreeCacheNode)
if err := tc.recursiveNodeUpdate(tc.prefix, tc.head); err != nil {
log.Errorf("Error during Zookeeper resync: %s", err)
tc.logger.Errorf("Error during Zookeeper resync: %s", err)
// Revert to our previous state.
tc.head.children = previousState.children
failure()
} else {
tc.resyncState(tc.prefix, tc.head, previousState)
log.Infof("Zookeeper resync successful")
tc.logger.Infof("Zookeeper resync successful")
failureMode = false
}
case <-tc.stop:

13
vendor/github.com/inconshreveable/mousetrap/LICENSE generated vendored Normal file
View file

@ -0,0 +1,13 @@
Copyright 2014 Alan Shreve
Licensed under the Apache License, Version 2.0 (the "License");
you may not use this file except in compliance with the License.
You may obtain a copy of the License at
http://www.apache.org/licenses/LICENSE-2.0
Unless required by applicable law or agreed to in writing, software
distributed under the License is distributed on an "AS IS" BASIS,
WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
See the License for the specific language governing permissions and
limitations under the License.

23
vendor/github.com/inconshreveable/mousetrap/README.md generated vendored Normal file
View file

@ -0,0 +1,23 @@
# mousetrap
mousetrap is a tiny library that answers a single question.
On a Windows machine, was the process invoked by someone double clicking on
the executable file while browsing in explorer?
### Motivation
Windows developers unfamiliar with command line tools will often "double-click"
the executable for a tool. Because most CLI tools print the help and then exit
when invoked without arguments, this is often very frustrating for those users.
mousetrap provides a way to detect these invocations so that you can provide
more helpful behavior and instructions on how to run the CLI tool. To see what
this looks like, both from an organizational and a technical perspective, see
https://inconshreveable.com/09-09-2014/sweat-the-small-stuff/
### The interface
The library exposes a single interface:
func StartedByExplorer() (bool)

View file

@ -0,0 +1,15 @@
// +build !windows
package mousetrap
// StartedByExplorer returns true if the program was invoked by the user
// double-clicking on the executable from explorer.exe
//
// It is conservative and returns false if any of the internal calls fail.
// It does not guarantee that the program was run from a terminal. It only can tell you
// whether it was launched from explorer.exe
//
// On non-Windows platforms, it always returns false.
func StartedByExplorer() bool {
return false
}

View file

@ -0,0 +1,98 @@
// +build windows
// +build !go1.4
package mousetrap
import (
"fmt"
"os"
"syscall"
"unsafe"
)
const (
// defined by the Win32 API
th32cs_snapprocess uintptr = 0x2
)
var (
kernel = syscall.MustLoadDLL("kernel32.dll")
CreateToolhelp32Snapshot = kernel.MustFindProc("CreateToolhelp32Snapshot")
Process32First = kernel.MustFindProc("Process32FirstW")
Process32Next = kernel.MustFindProc("Process32NextW")
)
// ProcessEntry32 structure defined by the Win32 API
type processEntry32 struct {
dwSize uint32
cntUsage uint32
th32ProcessID uint32
th32DefaultHeapID int
th32ModuleID uint32
cntThreads uint32
th32ParentProcessID uint32
pcPriClassBase int32
dwFlags uint32
szExeFile [syscall.MAX_PATH]uint16
}
func getProcessEntry(pid int) (pe *processEntry32, err error) {
snapshot, _, e1 := CreateToolhelp32Snapshot.Call(th32cs_snapprocess, uintptr(0))
if snapshot == uintptr(syscall.InvalidHandle) {
err = fmt.Errorf("CreateToolhelp32Snapshot: %v", e1)
return
}
defer syscall.CloseHandle(syscall.Handle(snapshot))
var processEntry processEntry32
processEntry.dwSize = uint32(unsafe.Sizeof(processEntry))
ok, _, e1 := Process32First.Call(snapshot, uintptr(unsafe.Pointer(&processEntry)))
if ok == 0 {
err = fmt.Errorf("Process32First: %v", e1)
return
}
for {
if processEntry.th32ProcessID == uint32(pid) {
pe = &processEntry
return
}
ok, _, e1 = Process32Next.Call(snapshot, uintptr(unsafe.Pointer(&processEntry)))
if ok == 0 {
err = fmt.Errorf("Process32Next: %v", e1)
return
}
}
}
func getppid() (pid int, err error) {
pe, err := getProcessEntry(os.Getpid())
if err != nil {
return
}
pid = int(pe.th32ParentProcessID)
return
}
// StartedByExplorer returns true if the program was invoked by the user double-clicking
// on the executable from explorer.exe
//
// It is conservative and returns false if any of the internal calls fail.
// It does not guarantee that the program was run from a terminal. It only can tell you
// whether it was launched from explorer.exe
func StartedByExplorer() bool {
ppid, err := getppid()
if err != nil {
return false
}
pe, err := getProcessEntry(ppid)
if err != nil {
return false
}
name := syscall.UTF16ToString(pe.szExeFile[:])
return name == "explorer.exe"
}

View file

@ -0,0 +1,46 @@
// +build windows
// +build go1.4
package mousetrap
import (
"os"
"syscall"
"unsafe"
)
func getProcessEntry(pid int) (*syscall.ProcessEntry32, error) {
snapshot, err := syscall.CreateToolhelp32Snapshot(syscall.TH32CS_SNAPPROCESS, 0)
if err != nil {
return nil, err
}
defer syscall.CloseHandle(snapshot)
var procEntry syscall.ProcessEntry32
procEntry.Size = uint32(unsafe.Sizeof(procEntry))
if err = syscall.Process32First(snapshot, &procEntry); err != nil {
return nil, err
}
for {
if procEntry.ProcessID == uint32(pid) {
return &procEntry, nil
}
err = syscall.Process32Next(snapshot, &procEntry)
if err != nil {
return nil, err
}
}
}
// StartedByExplorer returns true if the program was invoked by the user double-clicking
// on the executable from explorer.exe
//
// It is conservative and returns false if any of the internal calls fail.
// It does not guarantee that the program was run from a terminal. It only can tell you
// whether it was launched from explorer.exe
func StartedByExplorer() bool {
pe, err := getProcessEntry(os.Getppid())
if err != nil {
return false
}
return "explorer.exe" == syscall.UTF16ToString(pe.ExeFile[:])
}

View file

@ -21,22 +21,22 @@ import (
"golang.org/x/sys/windows/svc/eventlog"
"github.com/Sirupsen/logrus"
"github.com/sirupsen/logrus"
)
func init() {
setEventlogFormatter = func(name string, debugAsInfo bool) error {
setEventlogFormatter = func(l logger, name string, debugAsInfo bool) error {
if name == "" {
return fmt.Errorf("missing name parameter")
}
fmter, err := newEventlogger(name, debugAsInfo, origLogger.Formatter)
fmter, err := newEventlogger(name, debugAsInfo, l.entry.Logger.Formatter)
if err != nil {
fmt.Fprintf(os.Stderr, "error creating eventlog formatter: %v\n", err)
origLogger.Errorf("can't connect logger to eventlog: %v", err)
l.Errorf("can't connect logger to eventlog: %v", err)
return err
}
origLogger.Formatter = fmter
l.entry.Logger.Formatter = fmter
return nil
}
}

View file

@ -25,7 +25,7 @@ import (
"strconv"
"strings"
"github.com/Sirupsen/logrus"
"github.com/sirupsen/logrus"
)
type levelFlag string
@ -46,10 +46,10 @@ func (f levelFlag) Set(level string) error {
}
// setSyslogFormatter is nil if the target architecture does not support syslog.
var setSyslogFormatter func(string, string) error
var setSyslogFormatter func(logger, string, string) error
// setEventlogFormatter is nil if the target OS does not support Eventlog (i.e., is not Windows).
var setEventlogFormatter func(string, bool) error
var setEventlogFormatter func(logger, string, bool) error
func setJSONFormatter() {
origLogger.Formatter = &logrus.JSONFormatter{}
@ -65,45 +65,7 @@ func (f logFormatFlag) String() string {
// Set implements flag.Value.
func (f logFormatFlag) Set(format string) error {
u, err := url.Parse(format)
if err != nil {
return err
}
if u.Scheme != "logger" {
return fmt.Errorf("invalid scheme %s", u.Scheme)
}
jsonq := u.Query().Get("json")
if jsonq == "true" {
setJSONFormatter()
}
switch u.Opaque {
case "syslog":
if setSyslogFormatter == nil {
return fmt.Errorf("system does not support syslog")
}
appname := u.Query().Get("appname")
facility := u.Query().Get("local")
return setSyslogFormatter(appname, facility)
case "eventlog":
if setEventlogFormatter == nil {
return fmt.Errorf("system does not support eventlog")
}
name := u.Query().Get("name")
debugAsInfo := false
debugAsInfoRaw := u.Query().Get("debugAsInfo")
if parsedDebugAsInfo, err := strconv.ParseBool(debugAsInfoRaw); err == nil {
debugAsInfo = parsedDebugAsInfo
}
return setEventlogFormatter(name, debugAsInfo)
case "stdout":
origLogger.Out = os.Stdout
case "stderr":
origLogger.Out = os.Stderr
default:
return fmt.Errorf("unsupported logger %q", u.Opaque)
}
return nil
return baseLogger.SetFormat(format)
}
func init() {
@ -150,6 +112,9 @@ type Logger interface {
Fatalf(string, ...interface{})
With(key string, value interface{}) Logger
SetFormat(string) error
SetLevel(string) error
}
type logger struct {
@ -235,6 +200,58 @@ func (l logger) Fatalf(format string, args ...interface{}) {
l.sourced().Fatalf(format, args...)
}
func (l logger) SetLevel(level string) error {
lvl, err := logrus.ParseLevel(level)
if err != nil {
return err
}
l.entry.Logger.Level = lvl
return nil
}
func (l logger) SetFormat(format string) error {
u, err := url.Parse(format)
if err != nil {
return err
}
if u.Scheme != "logger" {
return fmt.Errorf("invalid scheme %s", u.Scheme)
}
jsonq := u.Query().Get("json")
if jsonq == "true" {
setJSONFormatter()
}
switch u.Opaque {
case "syslog":
if setSyslogFormatter == nil {
return fmt.Errorf("system does not support syslog")
}
appname := u.Query().Get("appname")
facility := u.Query().Get("local")
return setSyslogFormatter(l, appname, facility)
case "eventlog":
if setEventlogFormatter == nil {
return fmt.Errorf("system does not support eventlog")
}
name := u.Query().Get("name")
debugAsInfo := false
debugAsInfoRaw := u.Query().Get("debugAsInfo")
if parsedDebugAsInfo, err := strconv.ParseBool(debugAsInfoRaw); err == nil {
debugAsInfo = parsedDebugAsInfo
}
return setEventlogFormatter(l, name, debugAsInfo)
case "stdout":
l.entry.Logger.Out = os.Stdout
case "stderr":
l.entry.Logger.Out = os.Stderr
default:
return fmt.Errorf("unsupported logger %q", u.Opaque)
}
return nil
}
// sourced adds a source field to the logger that contains
// the file name and line where the logging happened.
func (l logger) sourced() *logrus.Entry {

View file

@ -20,13 +20,13 @@ import (
"log/syslog"
"os"
"github.com/Sirupsen/logrus"
"github.com/sirupsen/logrus"
)
var _ logrus.Formatter = (*syslogger)(nil)
func init() {
setSyslogFormatter = func(appname, local string) error {
setSyslogFormatter = func(l logger, appname, local string) error {
if appname == "" {
return fmt.Errorf("missing appname parameter")
}
@ -34,13 +34,13 @@ func init() {
return fmt.Errorf("missing local parameter")
}
fmter, err := newSyslogger(appname, local, origLogger.Formatter)
fmter, err := newSyslogger(appname, local, l.entry.Logger.Formatter)
if err != nil {
fmt.Fprintf(os.Stderr, "error creating syslog formatter: %v\n", err)
origLogger.Errorf("can't connect logger to syslog: %v", err)
l.entry.Errorf("can't connect logger to syslog: %v", err)
return err
}
origLogger.Formatter = fmter
l.entry.Logger.Formatter = fmter
return nil
}
}

View file

@ -163,9 +163,21 @@ func (t *Time) UnmarshalJSON(b []byte) error {
// This type should not propagate beyond the scope of input/output processing.
type Duration time.Duration
// Set implements pflag/flag.Value
func (d *Duration) Set(s string) error {
var err error
*d, err = ParseDuration(s)
return err
}
// Type implements pflag.Value
func (d *Duration) Type() string {
return "duration"
}
var durationRE = regexp.MustCompile("^([0-9]+)(y|w|d|h|m|s|ms)$")
// StringToDuration parses a string into a time.Duration, assuming that a year
// ParseDuration parses a string into a time.Duration, assuming that a year
// always has 365d, a week always has 7d, and a day always has 24h.
func ParseDuration(durationStr string) (Duration, error) {
matches := durationRE.FindStringSubmatch(durationStr)

View file

@ -1,3 +1,9 @@
# 1.0.0
* Officially changed name to lower-case
* bug: colors on Windows 10 (#541)
* bug: fix race in accessing level (#512)
# 0.11.5
* feature: add writer and writerlevel to entry (#372)

View file

@ -1,11 +1,5 @@
# Logrus <img src="http://i.imgur.com/hTeVwmJ.png" width="40" height="40" alt=":walrus:" class="emoji" title=":walrus:"/>&nbsp;[![Build Status](https://travis-ci.org/sirupsen/logrus.svg?branch=master)](https://travis-ci.org/sirupsen/logrus)&nbsp;[![GoDoc](https://godoc.org/github.com/sirupsen/logrus?status.svg)](https://godoc.org/github.com/sirupsen/logrus)
**Seeing weird case-sensitive problems?** See [this
issue](https://github.com/sirupsen/logrus/issues/451#issuecomment-264332021).
This change has been reverted. I apologize for causing this. I greatly
underestimated the impact this would have. Logrus strives for stability and
backwards compatibility and failed to provide that.
Logrus is a structured logger for Go (golang), completely API compatible with
the standard library logger. [Godoc][godoc]. **Please note the Logrus API is not
yet stable (pre 1.0). Logrus itself is completely stable and has been used in
@ -13,6 +7,17 @@ many large deployments. The core API is unlikely to change much but please
version control your Logrus to make sure you aren't fetching latest `master` on
every build.**
**Seeing weird case-sensitive problems?** Unfortunately, the author failed to
realize the consequences of renaming to lower-case. Due to the Go package
environment, this caused issues. Regretfully, there's no turning back now.
Everything using `logrus` will need to use the lower-case:
`github.com/sirupsen/logrus`. Any package that isn't, should be changed.
I am terribly sorry for this inconvenience. Logrus strives hard for backwards
compatibility, and the author failed to realize the cascading consequences of
such a name-change. To fix Glide, see [these
comments](https://github.com/sirupsen/logrus/issues/553#issuecomment-306591437).
Nicely color-coded in development (when a TTY is attached, otherwise just
plain text):
@ -276,6 +281,7 @@ Note: Syslog hook also support connecting to local syslog (Ex. "/dev/log" or "/v
| [Stackdriver](https://github.com/knq/sdhook) | Hook for logging to [Google Stackdriver](https://cloud.google.com/logging/) |
| [Sumorus](https://github.com/doublefree/sumorus) | Hook for logging to [SumoLogic](https://www.sumologic.com/)|
| [Syslog](https://github.com/Sirupsen/logrus/blob/master/hooks/syslog/syslog.go) | Send errors to remote syslog server. Uses standard library `log/syslog` behind the scenes. |
| [Syslog TLS](https://github.com/shinji62/logrus-syslog-ng) | Send errors to remote syslog server with TLS support. |
| [TraceView](https://github.com/evalphobia/logrus_appneta) | Hook for logging to [AppNeta TraceView](https://www.appneta.com/products/traceview/) |
| [Typetalk](https://github.com/dragon3/logrus-typetalk-hook) | Hook for logging to [Typetalk](https://www.typetalk.in/) |
| [logz.io](https://github.com/ripcurld00d/logrus-logzio-hook) | Hook for logging to [logz.io](https://logz.io), a Log as a Service using Logstash |

174
vendor/github.com/spf13/cobra/LICENSE.txt generated vendored Normal file
View file

@ -0,0 +1,174 @@
Apache License
Version 2.0, January 2004
http://www.apache.org/licenses/
TERMS AND CONDITIONS FOR USE, REPRODUCTION, AND DISTRIBUTION
1. Definitions.
"License" shall mean the terms and conditions for use, reproduction,
and distribution as defined by Sections 1 through 9 of this document.
"Licensor" shall mean the copyright owner or entity authorized by
the copyright owner that is granting the License.
"Legal Entity" shall mean the union of the acting entity and all
other entities that control, are controlled by, or are under common
control with that entity. For the purposes of this definition,
"control" means (i) the power, direct or indirect, to cause the
direction or management of such entity, whether by contract or
otherwise, or (ii) ownership of fifty percent (50%) or more of the
outstanding shares, or (iii) beneficial ownership of such entity.
"You" (or "Your") shall mean an individual or Legal Entity
exercising permissions granted by this License.
"Source" form shall mean the preferred form for making modifications,
including but not limited to software source code, documentation
source, and configuration files.
"Object" form shall mean any form resulting from mechanical
transformation or translation of a Source form, including but
not limited to compiled object code, generated documentation,
and conversions to other media types.
"Work" shall mean the work of authorship, whether in Source or
Object form, made available under the License, as indicated by a
copyright notice that is included in or attached to the work
(an example is provided in the Appendix below).
"Derivative Works" shall mean any work, whether in Source or Object
form, that is based on (or derived from) the Work and for which the
editorial revisions, annotations, elaborations, or other modifications
represent, as a whole, an original work of authorship. For the purposes
of this License, Derivative Works shall not include works that remain
separable from, or merely link (or bind by name) to the interfaces of,
the Work and Derivative Works thereof.
"Contribution" shall mean any work of authorship, including
the original version of the Work and any modifications or additions
to that Work or Derivative Works thereof, that is intentionally
submitted to Licensor for inclusion in the Work by the copyright owner
or by an individual or Legal Entity authorized to submit on behalf of
the copyright owner. For the purposes of this definition, "submitted"
means any form of electronic, verbal, or written communication sent
to the Licensor or its representatives, including but not limited to
communication on electronic mailing lists, source code control systems,
and issue tracking systems that are managed by, or on behalf of, the
Licensor for the purpose of discussing and improving the Work, but
excluding communication that is conspicuously marked or otherwise
designated in writing by the copyright owner as "Not a Contribution."
"Contributor" shall mean Licensor and any individual or Legal Entity
on behalf of whom a Contribution has been received by Licensor and
subsequently incorporated within the Work.
2. Grant of Copyright License. Subject to the terms and conditions of
this License, each Contributor hereby grants to You a perpetual,
worldwide, non-exclusive, no-charge, royalty-free, irrevocable
copyright license to reproduce, prepare Derivative Works of,
publicly display, publicly perform, sublicense, and distribute the
Work and such Derivative Works in Source or Object form.
3. Grant of Patent License. Subject to the terms and conditions of
this License, each Contributor hereby grants to You a perpetual,
worldwide, non-exclusive, no-charge, royalty-free, irrevocable
(except as stated in this section) patent license to make, have made,
use, offer to sell, sell, import, and otherwise transfer the Work,
where such license applies only to those patent claims licensable
by such Contributor that are necessarily infringed by their
Contribution(s) alone or by combination of their Contribution(s)
with the Work to which such Contribution(s) was submitted. If You
institute patent litigation against any entity (including a
cross-claim or counterclaim in a lawsuit) alleging that the Work
or a Contribution incorporated within the Work constitutes direct
or contributory patent infringement, then any patent licenses
granted to You under this License for that Work shall terminate
as of the date such litigation is filed.
4. Redistribution. You may reproduce and distribute copies of the
Work or Derivative Works thereof in any medium, with or without
modifications, and in Source or Object form, provided that You
meet the following conditions:
(a) You must give any other recipients of the Work or
Derivative Works a copy of this License; and
(b) You must cause any modified files to carry prominent notices
stating that You changed the files; and
(c) You must retain, in the Source form of any Derivative Works
that You distribute, all copyright, patent, trademark, and
attribution notices from the Source form of the Work,
excluding those notices that do not pertain to any part of
the Derivative Works; and
(d) If the Work includes a "NOTICE" text file as part of its
distribution, then any Derivative Works that You distribute must
include a readable copy of the attribution notices contained
within such NOTICE file, excluding those notices that do not
pertain to any part of the Derivative Works, in at least one
of the following places: within a NOTICE text file distributed
as part of the Derivative Works; within the Source form or
documentation, if provided along with the Derivative Works; or,
within a display generated by the Derivative Works, if and
wherever such third-party notices normally appear. The contents
of the NOTICE file are for informational purposes only and
do not modify the License. You may add Your own attribution
notices within Derivative Works that You distribute, alongside
or as an addendum to the NOTICE text from the Work, provided
that such additional attribution notices cannot be construed
as modifying the License.
You may add Your own copyright statement to Your modifications and
may provide additional or different license terms and conditions
for use, reproduction, or distribution of Your modifications, or
for any such Derivative Works as a whole, provided Your use,
reproduction, and distribution of the Work otherwise complies with
the conditions stated in this License.
5. Submission of Contributions. Unless You explicitly state otherwise,
any Contribution intentionally submitted for inclusion in the Work
by You to the Licensor shall be under the terms and conditions of
this License, without any additional terms or conditions.
Notwithstanding the above, nothing herein shall supersede or modify
the terms of any separate license agreement you may have executed
with Licensor regarding such Contributions.
6. Trademarks. This License does not grant permission to use the trade
names, trademarks, service marks, or product names of the Licensor,
except as required for reasonable and customary use in describing the
origin of the Work and reproducing the content of the NOTICE file.
7. Disclaimer of Warranty. Unless required by applicable law or
agreed to in writing, Licensor provides the Work (and each
Contributor provides its Contributions) on an "AS IS" BASIS,
WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or
implied, including, without limitation, any warranties or conditions
of TITLE, NON-INFRINGEMENT, MERCHANTABILITY, or FITNESS FOR A
PARTICULAR PURPOSE. You are solely responsible for determining the
appropriateness of using or redistributing the Work and assume any
risks associated with Your exercise of permissions under this License.
8. Limitation of Liability. In no event and under no legal theory,
whether in tort (including negligence), contract, or otherwise,
unless required by applicable law (such as deliberate and grossly
negligent acts) or agreed to in writing, shall any Contributor be
liable to You for damages, including any direct, indirect, special,
incidental, or consequential damages of any character arising as a
result of this License or out of the use or inability to use the
Work (including but not limited to damages for loss of goodwill,
work stoppage, computer failure or malfunction, or any and all
other commercial damages or losses), even if such Contributor
has been advised of the possibility of such damages.
9. Accepting Warranty or Additional Liability. While redistributing
the Work or Derivative Works thereof, You may choose to offer,
and charge a fee for, acceptance of support, warranty, indemnity,
or other liability obligations and/or rights consistent with this
License. However, in accepting such obligations, You may act only
on Your own behalf and on Your sole responsibility, not on behalf
of any other Contributor, and only if You agree to indemnify,
defend, and hold each Contributor harmless for any liability
incurred by, or claims asserted against, such Contributor by reason
of your accepting any such warranty or additional liability.

939
vendor/github.com/spf13/cobra/README.md generated vendored Normal file
View file

@ -0,0 +1,939 @@
![cobra logo](https://cloud.githubusercontent.com/assets/173412/10886352/ad566232-814f-11e5-9cd0-aa101788c117.png)
Cobra is both a library for creating powerful modern CLI applications as well as a program to generate applications and command files.
Many of the most widely used Go projects are built using Cobra including:
* [Kubernetes](http://kubernetes.io/)
* [Hugo](http://gohugo.io)
* [rkt](https://github.com/coreos/rkt)
* [etcd](https://github.com/coreos/etcd)
* [Moby (former Docker)](https://github.com/moby/moby)
* [Docker (distribution)](https://github.com/docker/distribution)
* [OpenShift](https://www.openshift.com/)
* [Delve](https://github.com/derekparker/delve)
* [GopherJS](http://www.gopherjs.org/)
* [CockroachDB](http://www.cockroachlabs.com/)
* [Bleve](http://www.blevesearch.com/)
* [ProjectAtomic (enterprise)](http://www.projectatomic.io/)
* [GiantSwarm's swarm](https://github.com/giantswarm/cli)
* [Nanobox](https://github.com/nanobox-io/nanobox)/[Nanopack](https://github.com/nanopack)
* [rclone](http://rclone.org/)
[![Build Status](https://travis-ci.org/spf13/cobra.svg "Travis CI status")](https://travis-ci.org/spf13/cobra)
[![CircleCI status](https://circleci.com/gh/spf13/cobra.png?circle-token=:circle-token "CircleCI status")](https://circleci.com/gh/spf13/cobra)
[![GoDoc](https://godoc.org/github.com/spf13/cobra?status.svg)](https://godoc.org/github.com/spf13/cobra)
![cobra](https://cloud.githubusercontent.com/assets/173412/10911369/84832a8e-8212-11e5-9f82-cc96660a4794.gif)
# Overview
Cobra is a library providing a simple interface to create powerful modern CLI
interfaces similar to git & go tools.
Cobra is also an application that will generate your application scaffolding to rapidly
develop a Cobra-based application.
Cobra provides:
* Easy subcommand-based CLIs: `app server`, `app fetch`, etc.
* Fully POSIX-compliant flags (including short & long versions)
* Nested subcommands
* Global, local and cascading flags
* Easy generation of applications & commands with `cobra init appname` & `cobra add cmdname`
* Intelligent suggestions (`app srver`... did you mean `app server`?)
* Automatic help generation for commands and flags
* Automatic detailed help for `app help [command]`
* Automatic help flag recognition of `-h`, `--help`, etc.
* Automatically generated bash autocomplete for your application
* Automatically generated man pages for your application
* Command aliases so you can change things without breaking them
* The flexibility to define your own help, usage, etc.
* Optional tight integration with [viper](http://github.com/spf13/viper) for 12-factor apps
Cobra has an exceptionally clean interface and simple design without needless
constructors or initialization methods.
Applications built with Cobra commands are designed to be as user-friendly as
possible. Flags can be placed before or after the command (as long as a
confusing space isnt provided). Both short and long flags can be used. A
command need not even be fully typed. Help is automatically generated and
available for the application or for a specific command using either the help
command or the `--help` flag.
# Concepts
Cobra is built on a structure of commands, arguments & flags.
**Commands** represent actions, **Args** are things and **Flags** are modifiers for those actions.
The best applications will read like sentences when used. Users will know how
to use the application because they will natively understand how to use it.
The pattern to follow is
`APPNAME VERB NOUN --ADJECTIVE.`
or
`APPNAME COMMAND ARG --FLAG`
A few good real world examples may better illustrate this point.
In the following example, 'server' is a command, and 'port' is a flag:
hugo server --port=1313
In this command we are telling Git to clone the url bare.
git clone URL --bare
## Commands
Command is the central point of the application. Each interaction that
the application supports will be contained in a Command. A command can
have children commands and optionally run an action.
In the example above, 'server' is the command.
A Command has the following structure:
```go
type Command struct {
Use string // The one-line usage message.
Short string // The short description shown in the 'help' output.
Long string // The long message shown in the 'help <this-command>' output.
Run func(cmd *Command, args []string) // Run runs the command.
}
```
## Flags
A Flag is a way to modify the behavior of a command. Cobra supports
fully POSIX-compliant flags as well as the Go [flag package](https://golang.org/pkg/flag/).
A Cobra command can define flags that persist through to children commands
and flags that are only available to that command.
In the example above, 'port' is the flag.
Flag functionality is provided by the [pflag
library](https://github.com/spf13/pflag), a fork of the flag standard library
which maintains the same interface while adding POSIX compliance.
## Usage
Cobra works by creating a set of commands and then organizing them into a tree.
The tree defines the structure of the application.
Once each command is defined with its corresponding flags, then the
tree is assigned to the commander which is finally executed.
# Installing
Using Cobra is easy. First, use `go get` to install the latest version
of the library. This command will install the `cobra` generator executable
along with the library and its dependencies:
go get -u github.com/spf13/cobra/cobra
Next, include Cobra in your application:
```go
import "github.com/spf13/cobra"
```
# Getting Started
While you are welcome to provide your own organization, typically a Cobra based
application will follow the following organizational structure.
```
▾ appName/
▾ cmd/
add.go
your.go
commands.go
here.go
main.go
```
In a Cobra app, typically the main.go file is very bare. It serves, one purpose, to initialize Cobra.
```go
package main
import (
"fmt"
"os"
"{pathToYourApp}/cmd"
)
func main() {
if err := cmd.RootCmd.Execute(); err != nil {
fmt.Println(err)
os.Exit(1)
}
}
```
## Using the Cobra Generator
Cobra provides its own program that will create your application and add any
commands you want. It's the easiest way to incorporate Cobra into your application.
In order to use the cobra command, compile it using the following command:
go get github.com/spf13/cobra/cobra
This will create the cobra executable under your `$GOPATH/bin` directory.
### cobra init
The `cobra init [yourApp]` command will create your initial application code
for you. It is a very powerful application that will populate your program with
the right structure so you can immediately enjoy all the benefits of Cobra. It
will also automatically apply the license you specify to your application.
Cobra init is pretty smart. You can provide it a full path, or simply a path
similar to what is expected in the import.
```
cobra init github.com/spf13/newAppName
```
### cobra add
Once an application is initialized Cobra can create additional commands for you.
Let's say you created an app and you wanted the following commands for it:
* app serve
* app config
* app config create
In your project directory (where your main.go file is) you would run the following:
```
cobra add serve
cobra add config
cobra add create -p 'configCmd'
```
*Note: Use camelCase (not snake_case/snake-case) for command names.
Otherwise, you will become unexpected errors.
For example, `cobra add add-user` is incorrect, but `cobra add addUser` is valid.*
Once you have run these three commands you would have an app structure that would look like:
```
▾ app/
▾ cmd/
serve.go
config.go
create.go
main.go
```
At this point you can run `go run main.go` and it would run your app. `go run
main.go serve`, `go run main.go config`, `go run main.go config create` along
with `go run main.go help serve`, etc would all work.
Obviously you haven't added your own code to these yet, the commands are ready
for you to give them their tasks. Have fun!
### Configuring the cobra generator
The cobra generator will be easier to use if you provide a simple configuration
file which will help you eliminate providing a bunch of repeated information in
flags over and over.
An example ~/.cobra.yaml file:
```yaml
author: Steve Francia <spf@spf13.com>
license: MIT
```
You can specify no license by setting `license` to `none` or you can specify
a custom license:
```yaml
license:
header: This file is part of {{ .appName }}.
text: |
{{ .copyright }}
This is my license. There are many like it, but this one is mine.
My license is my best friend. It is my life. I must master it as I must
master my life.
```
You can also use built-in licenses. For example, **GPLv2**, **GPLv3**, **LGPL**,
**AGPL**, **MIT**, **2-Clause BSD** or **3-Clause BSD**.
## Manually implementing Cobra
To manually implement cobra you need to create a bare main.go file and a RootCmd file.
You will optionally provide additional commands as you see fit.
### Create the root command
The root command represents your binary itself.
#### Manually create rootCmd
Cobra doesn't require any special constructors. Simply create your commands.
Ideally you place this in app/cmd/root.go:
```go
var RootCmd = &cobra.Command{
Use: "hugo",
Short: "Hugo is a very fast static site generator",
Long: `A Fast and Flexible Static Site Generator built with
love by spf13 and friends in Go.
Complete documentation is available at http://hugo.spf13.com`,
Run: func(cmd *cobra.Command, args []string) {
// Do Stuff Here
},
}
```
You will additionally define flags and handle configuration in your init() function.
For example cmd/root.go:
```go
import (
"fmt"
"os"
homedir "github.com/mitchellh/go-homedir"
"github.com/spf13/cobra"
"github.com/spf13/viper"
)
func init() {
cobra.OnInitialize(initConfig)
RootCmd.PersistentFlags().StringVar(&cfgFile, "config", "", "config file (default is $HOME/.cobra.yaml)")
RootCmd.PersistentFlags().StringVarP(&projectBase, "projectbase", "b", "", "base project directory eg. github.com/spf13/")
RootCmd.PersistentFlags().StringP("author", "a", "YOUR NAME", "Author name for copyright attribution")
RootCmd.PersistentFlags().StringVarP(&userLicense, "license", "l", "", "Name of license for the project (can provide `licensetext` in config)")
RootCmd.PersistentFlags().Bool("viper", true, "Use Viper for configuration")
viper.BindPFlag("author", RootCmd.PersistentFlags().Lookup("author"))
viper.BindPFlag("projectbase", RootCmd.PersistentFlags().Lookup("projectbase"))
viper.BindPFlag("useViper", RootCmd.PersistentFlags().Lookup("viper"))
viper.SetDefault("author", "NAME HERE <EMAIL ADDRESS>")
viper.SetDefault("license", "apache")
}
func Execute() {
rootCmd.Execute()
}
func main() {
// Don't forget to read config either from cfgFile or from home directory!
if cfgFile != "" {
// Use config file from the flag.
viper.SetConfigFile(cfgFile)
} else {
// Find home directory.
home, err := homedir.Dir()
if err != nil {
fmt.Println(home)
os.Exit(1)
}
// Search config in home directory with name ".cobra" (without extension).
viper.AddConfigPath(home)
viper.SetConfigName(".cobra")
}
if err := viper.ReadInConfig(); err != nil {
fmt.Println("Can't read config:", err)
os.Exit(1)
}
}
```
### Create your main.go
With the root command you need to have your main function execute it.
Execute should be run on the root for clarity, though it can be called on any command.
In a Cobra app, typically the main.go file is very bare. It serves, one purpose, to initialize Cobra.
```go
package main
import (
"fmt"
"os"
"{pathToYourApp}/cmd"
)
func main() {
if err := cmd.RootCmd.Execute(); err != nil {
fmt.Println(err)
os.Exit(1)
}
}
```
### Create additional commands
Additional commands can be defined and typically are each given their own file
inside of the cmd/ directory.
If you wanted to create a version command you would create cmd/version.go and
populate it with the following:
```go
package cmd
import (
"github.com/spf13/cobra"
"fmt"
)
func init() {
RootCmd.AddCommand(versionCmd)
}
var versionCmd = &cobra.Command{
Use: "version",
Short: "Print the version number of Hugo",
Long: `All software has versions. This is Hugo's`,
Run: func(cmd *cobra.Command, args []string) {
fmt.Println("Hugo Static Site Generator v0.9 -- HEAD")
},
}
```
### Attach command to its parent
If you notice in the above example we attach the command to its parent. In
this case the parent is the rootCmd. In this example we are attaching it to the
root, but commands can be attached at any level.
```go
RootCmd.AddCommand(versionCmd)
```
### Remove a command from its parent
Removing a command is not a common action in simple programs, but it allows 3rd
parties to customize an existing command tree.
In this example, we remove the existing `VersionCmd` command of an existing
root command, and we replace it with our own version:
```go
mainlib.RootCmd.RemoveCommand(mainlib.VersionCmd)
mainlib.RootCmd.AddCommand(versionCmd)
```
## Working with Flags
Flags provide modifiers to control how the action command operates.
### Assign flags to a command
Since the flags are defined and used in different locations, we need to
define a variable outside with the correct scope to assign the flag to
work with.
```go
var Verbose bool
var Source string
```
There are two different approaches to assign a flag.
### Persistent Flags
A flag can be 'persistent' meaning that this flag will be available to the
command it's assigned to as well as every command under that command. For
global flags, assign a flag as a persistent flag on the root.
```go
RootCmd.PersistentFlags().BoolVarP(&Verbose, "verbose", "v", false, "verbose output")
```
### Local Flags
A flag can also be assigned locally which will only apply to that specific command.
```go
RootCmd.Flags().StringVarP(&Source, "source", "s", "", "Source directory to read from")
```
### Bind Flags with Config
You can also bind your flags with [viper](https://github.com/spf13/viper):
```go
var author string
func init() {
RootCmd.PersistentFlags().StringVar(&author, "author", "YOUR NAME", "Author name for copyright attribution")
viper.BindPFlag("author", RootCmd.PersistentFlags().Lookup("author"))
}
```
In this example the persistent flag `author` is bound with `viper`.
**Note**, that the variable `author` will not be set to the value from config,
when the `--author` flag is not provided by user.
More in [viper documentation](https://github.com/spf13/viper#working-with-flags).
## Example
In the example below, we have defined three commands. Two are at the top level
and one (cmdTimes) is a child of one of the top commands. In this case the root
is not executable meaning that a subcommand is required. This is accomplished
by not providing a 'Run' for the 'rootCmd'.
We have only defined one flag for a single command.
More documentation about flags is available at https://github.com/spf13/pflag
```go
package main
import (
"fmt"
"strings"
"github.com/spf13/cobra"
)
func main() {
var echoTimes int
var cmdPrint = &cobra.Command{
Use: "print [string to print]",
Short: "Print anything to the screen",
Long: `print is for printing anything back to the screen.
For many years people have printed back to the screen.
`,
Run: func(cmd *cobra.Command, args []string) {
fmt.Println("Print: " + strings.Join(args, " "))
},
}
var cmdEcho = &cobra.Command{
Use: "echo [string to echo]",
Short: "Echo anything to the screen",
Long: `echo is for echoing anything back.
Echo works a lot like print, except it has a child command.
`,
Run: func(cmd *cobra.Command, args []string) {
fmt.Println("Print: " + strings.Join(args, " "))
},
}
var cmdTimes = &cobra.Command{
Use: "times [# times] [string to echo]",
Short: "Echo anything to the screen more times",
Long: `echo things multiple times back to the user by providing
a count and a string.`,
Run: func(cmd *cobra.Command, args []string) {
for i := 0; i < echoTimes; i++ {
fmt.Println("Echo: " + strings.Join(args, " "))
}
},
}
cmdTimes.Flags().IntVarP(&echoTimes, "times", "t", 1, "times to echo the input")
var rootCmd = &cobra.Command{Use: "app"}
rootCmd.AddCommand(cmdPrint, cmdEcho)
cmdEcho.AddCommand(cmdTimes)
rootCmd.Execute()
}
```
For a more complete example of a larger application, please checkout [Hugo](http://gohugo.io/).
## The Help Command
Cobra automatically adds a help command to your application when you have subcommands.
This will be called when a user runs 'app help'. Additionally, help will also
support all other commands as input. Say, for instance, you have a command called
'create' without any additional configuration; Cobra will work when 'app help
create' is called. Every command will automatically have the '--help' flag added.
### Example
The following output is automatically generated by Cobra. Nothing beyond the
command and flag definitions are needed.
> hugo help
hugo is the main command, used to build your Hugo site.
Hugo is a Fast and Flexible Static Site Generator
built with love by spf13 and friends in Go.
Complete documentation is available at http://gohugo.io/.
Usage:
hugo [flags]
hugo [command]
Available Commands:
server Hugo runs its own webserver to render the files
version Print the version number of Hugo
config Print the site configuration
check Check content in the source directory
benchmark Benchmark hugo by building a site a number of times.
convert Convert your content to different formats
new Create new content for your site
list Listing out various types of content
undraft Undraft changes the content's draft status from 'True' to 'False'
genautocomplete Generate shell autocompletion script for Hugo
gendoc Generate Markdown documentation for the Hugo CLI.
genman Generate man page for Hugo
import Import your site from others.
Flags:
-b, --baseURL="": hostname (and path) to the root, e.g. http://spf13.com/
-D, --buildDrafts[=false]: include content marked as draft
-F, --buildFuture[=false]: include content with publishdate in the future
--cacheDir="": filesystem path to cache directory. Defaults: $TMPDIR/hugo_cache/
--canonifyURLs[=false]: if true, all relative URLs will be canonicalized using baseURL
--config="": config file (default is path/config.yaml|json|toml)
-d, --destination="": filesystem path to write files to
--disableRSS[=false]: Do not build RSS files
--disableSitemap[=false]: Do not build Sitemap file
--editor="": edit new content with this editor, if provided
--ignoreCache[=false]: Ignores the cache directory for reading but still writes to it
--log[=false]: Enable Logging
--logFile="": Log File path (if set, logging enabled automatically)
--noTimes[=false]: Don't sync modification time of files
--pluralizeListTitles[=true]: Pluralize titles in lists using inflect
--preserveTaxonomyNames[=false]: Preserve taxonomy names as written ("Gérard Depardieu" vs "gerard-depardieu")
-s, --source="": filesystem path to read files relative from
--stepAnalysis[=false]: display memory and timing of different steps of the program
-t, --theme="": theme to use (located in /themes/THEMENAME/)
--uglyURLs[=false]: if true, use /filename.html instead of /filename/
-v, --verbose[=false]: verbose output
--verboseLog[=false]: verbose logging
-w, --watch[=false]: watch filesystem for changes and recreate as needed
Use "hugo [command] --help" for more information about a command.
Help is just a command like any other. There is no special logic or behavior
around it. In fact, you can provide your own if you want.
### Defining your own help
You can provide your own Help command or your own template for the default command to use.
The default help command is
```go
func (c *Command) initHelp() {
if c.helpCommand == nil {
c.helpCommand = &Command{
Use: "help [command]",
Short: "Help about any command",
Long: `Help provides help for any command in the application.
Simply type ` + c.Name() + ` help [path to command] for full details.`,
Run: c.HelpFunc(),
}
}
c.AddCommand(c.helpCommand)
}
```
You can provide your own command, function or template through the following methods:
```go
command.SetHelpCommand(cmd *Command)
command.SetHelpFunc(f func(*Command, []string))
command.SetHelpTemplate(s string)
```
The latter two will also apply to any children commands.
## Usage
When the user provides an invalid flag or invalid command, Cobra responds by
showing the user the 'usage'.
### Example
You may recognize this from the help above. That's because the default help
embeds the usage as part of its output.
Usage:
hugo [flags]
hugo [command]
Available Commands:
server Hugo runs its own webserver to render the files
version Print the version number of Hugo
config Print the site configuration
check Check content in the source directory
benchmark Benchmark hugo by building a site a number of times.
convert Convert your content to different formats
new Create new content for your site
list Listing out various types of content
undraft Undraft changes the content's draft status from 'True' to 'False'
genautocomplete Generate shell autocompletion script for Hugo
gendoc Generate Markdown documentation for the Hugo CLI.
genman Generate man page for Hugo
import Import your site from others.
Flags:
-b, --baseURL="": hostname (and path) to the root, e.g. http://spf13.com/
-D, --buildDrafts[=false]: include content marked as draft
-F, --buildFuture[=false]: include content with publishdate in the future
--cacheDir="": filesystem path to cache directory. Defaults: $TMPDIR/hugo_cache/
--canonifyURLs[=false]: if true, all relative URLs will be canonicalized using baseURL
--config="": config file (default is path/config.yaml|json|toml)
-d, --destination="": filesystem path to write files to
--disableRSS[=false]: Do not build RSS files
--disableSitemap[=false]: Do not build Sitemap file
--editor="": edit new content with this editor, if provided
--ignoreCache[=false]: Ignores the cache directory for reading but still writes to it
--log[=false]: Enable Logging
--logFile="": Log File path (if set, logging enabled automatically)
--noTimes[=false]: Don't sync modification time of files
--pluralizeListTitles[=true]: Pluralize titles in lists using inflect
--preserveTaxonomyNames[=false]: Preserve taxonomy names as written ("Gérard Depardieu" vs "gerard-depardieu")
-s, --source="": filesystem path to read files relative from
--stepAnalysis[=false]: display memory and timing of different steps of the program
-t, --theme="": theme to use (located in /themes/THEMENAME/)
--uglyURLs[=false]: if true, use /filename.html instead of /filename/
-v, --verbose[=false]: verbose output
--verboseLog[=false]: verbose logging
-w, --watch[=false]: watch filesystem for changes and recreate as needed
### Defining your own usage
You can provide your own usage function or template for Cobra to use.
The default usage function is:
```go
return func(c *Command) error {
err := tmpl(c.Out(), c.UsageTemplate(), c)
return err
}
```
Like help, the function and template are overridable through public methods:
```go
command.SetUsageFunc(f func(*Command) error)
command.SetUsageTemplate(s string)
```
## PreRun or PostRun Hooks
It is possible to run functions before or after the main `Run` function of your command. The `PersistentPreRun` and `PreRun` functions will be executed before `Run`. `PersistentPostRun` and `PostRun` will be executed after `Run`. The `Persistent*Run` functions will be inherited by children if they do not declare their own. These functions are run in the following order:
- `PersistentPreRun`
- `PreRun`
- `Run`
- `PostRun`
- `PersistentPostRun`
An example of two commands which use all of these features is below. When the subcommand is executed, it will run the root command's `PersistentPreRun` but not the root command's `PersistentPostRun`:
```go
package main
import (
"fmt"
"github.com/spf13/cobra"
)
func main() {
var rootCmd = &cobra.Command{
Use: "root [sub]",
Short: "My root command",
PersistentPreRun: func(cmd *cobra.Command, args []string) {
fmt.Printf("Inside rootCmd PersistentPreRun with args: %v\n", args)
},
PreRun: func(cmd *cobra.Command, args []string) {
fmt.Printf("Inside rootCmd PreRun with args: %v\n", args)
},
Run: func(cmd *cobra.Command, args []string) {
fmt.Printf("Inside rootCmd Run with args: %v\n", args)
},
PostRun: func(cmd *cobra.Command, args []string) {
fmt.Printf("Inside rootCmd PostRun with args: %v\n", args)
},
PersistentPostRun: func(cmd *cobra.Command, args []string) {
fmt.Printf("Inside rootCmd PersistentPostRun with args: %v\n", args)
},
}
var subCmd = &cobra.Command{
Use: "sub [no options!]",
Short: "My subcommand",
PreRun: func(cmd *cobra.Command, args []string) {
fmt.Printf("Inside subCmd PreRun with args: %v\n", args)
},
Run: func(cmd *cobra.Command, args []string) {
fmt.Printf("Inside subCmd Run with args: %v\n", args)
},
PostRun: func(cmd *cobra.Command, args []string) {
fmt.Printf("Inside subCmd PostRun with args: %v\n", args)
},
PersistentPostRun: func(cmd *cobra.Command, args []string) {
fmt.Printf("Inside subCmd PersistentPostRun with args: %v\n", args)
},
}
rootCmd.AddCommand(subCmd)
rootCmd.SetArgs([]string{""})
_ = rootCmd.Execute()
fmt.Print("\n")
rootCmd.SetArgs([]string{"sub", "arg1", "arg2"})
_ = rootCmd.Execute()
}
```
## Alternative Error Handling
Cobra also has functions where the return signature is an error. This allows for errors to bubble up to the top,
providing a way to handle the errors in one location. The current list of functions that return an error is:
* PersistentPreRunE
* PreRunE
* RunE
* PostRunE
* PersistentPostRunE
If you would like to silence the default `error` and `usage` output in favor of your own, you can set `SilenceUsage`
and `SilenceErrors` to `true` on the command. A child command respects these flags if they are set on the parent
command.
**Example Usage using RunE:**
```go
package main
import (
"errors"
"log"
"github.com/spf13/cobra"
)
func main() {
var rootCmd = &cobra.Command{
Use: "hugo",
Short: "Hugo is a very fast static site generator",
Long: `A Fast and Flexible Static Site Generator built with
love by spf13 and friends in Go.
Complete documentation is available at http://hugo.spf13.com`,
RunE: func(cmd *cobra.Command, args []string) error {
// Do Stuff Here
return errors.New("some random error")
},
}
if err := rootCmd.Execute(); err != nil {
log.Fatal(err)
}
}
```
## Suggestions when "unknown command" happens
Cobra will print automatic suggestions when "unknown command" errors happen. This allows Cobra to behave similarly to the `git` command when a typo happens. For example:
```
$ hugo srever
Error: unknown command "srever" for "hugo"
Did you mean this?
server
Run 'hugo --help' for usage.
```
Suggestions are automatic based on every subcommand registered and use an implementation of [Levenshtein distance](http://en.wikipedia.org/wiki/Levenshtein_distance). Every registered command that matches a minimum distance of 2 (ignoring case) will be displayed as a suggestion.
If you need to disable suggestions or tweak the string distance in your command, use:
```go
command.DisableSuggestions = true
```
or
```go
command.SuggestionsMinimumDistance = 1
```
You can also explicitly set names for which a given command will be suggested using the `SuggestFor` attribute. This allows suggestions for strings that are not close in terms of string distance, but makes sense in your set of commands and for some which you don't want aliases. Example:
```
$ kubectl remove
Error: unknown command "remove" for "kubectl"
Did you mean this?
delete
Run 'kubectl help' for usage.
```
## Generating Markdown-formatted documentation for your command
Cobra can generate a Markdown-formatted document based on the subcommands, flags, etc. A simple example of how to do this for your command can be found in [Markdown Docs](doc/md_docs.md).
## Generating man pages for your command
Cobra can generate a man page based on the subcommands, flags, etc. A simple example of how to do this for your command can be found in [Man Docs](doc/man_docs.md).
## Generating bash completions for your command
Cobra can generate a bash-completion file. If you add more information to your command, these completions can be amazingly powerful and flexible. Read more about it in [Bash Completions](bash_completions.md).
## Debugging
Cobra provides a DebugFlags method on a command which, when called, will print
out everything Cobra knows about the flags for each command.
### Example
```go
command.DebugFlags()
```
## Extensions
Libraries for extending Cobra:
* [cmdns](https://github.com/gosuri/cmdns): Enables name spacing a command's immediate children. It provides an alternative way to structure subcommands, similar to `heroku apps:create` and `ovrclk clusters:launch`.
## Contributing
1. Fork it
2. Create your feature branch (`git checkout -b my-new-feature`)
3. Commit your changes (`git commit -am 'Add some feature'`)
4. Push to the branch (`git push origin my-new-feature`)
5. Create new Pull Request
## Contributors
Names in no particular order:
* [spf13](https://github.com/spf13),
[eparis](https://github.com/eparis),
[bep](https://github.com/bep), and many more!
## License
Cobra is released under the Apache 2.0 license. See [LICENSE.txt](https://github.com/spf13/cobra/blob/master/LICENSE.txt)

537
vendor/github.com/spf13/cobra/bash_completions.go generated vendored Normal file
View file

@ -0,0 +1,537 @@
package cobra
import (
"bytes"
"fmt"
"io"
"os"
"sort"
"strings"
"github.com/spf13/pflag"
)
// Annotations for Bash completion.
const (
BashCompFilenameExt = "cobra_annotation_bash_completion_filename_extensions"
BashCompCustom = "cobra_annotation_bash_completion_custom"
BashCompOneRequiredFlag = "cobra_annotation_bash_completion_one_required_flag"
BashCompSubdirsInDir = "cobra_annotation_bash_completion_subdirs_in_dir"
)
func writePreamble(buf *bytes.Buffer, name string) {
buf.WriteString(fmt.Sprintf("# bash completion for %-36s -*- shell-script -*-\n", name))
buf.WriteString(`
__debug()
{
if [[ -n ${BASH_COMP_DEBUG_FILE} ]]; then
echo "$*" >> "${BASH_COMP_DEBUG_FILE}"
fi
}
# Homebrew on Macs have version 1.3 of bash-completion which doesn't include
# _init_completion. This is a very minimal version of that function.
__my_init_completion()
{
COMPREPLY=()
_get_comp_words_by_ref "$@" cur prev words cword
}
__index_of_word()
{
local w word=$1
shift
index=0
for w in "$@"; do
[[ $w = "$word" ]] && return
index=$((index+1))
done
index=-1
}
__contains_word()
{
local w word=$1; shift
for w in "$@"; do
[[ $w = "$word" ]] && return
done
return 1
}
__handle_reply()
{
__debug "${FUNCNAME[0]}"
case $cur in
-*)
if [[ $(type -t compopt) = "builtin" ]]; then
compopt -o nospace
fi
local allflags
if [ ${#must_have_one_flag[@]} -ne 0 ]; then
allflags=("${must_have_one_flag[@]}")
else
allflags=("${flags[*]} ${two_word_flags[*]}")
fi
COMPREPLY=( $(compgen -W "${allflags[*]}" -- "$cur") )
if [[ $(type -t compopt) = "builtin" ]]; then
[[ "${COMPREPLY[0]}" == *= ]] || compopt +o nospace
fi
# complete after --flag=abc
if [[ $cur == *=* ]]; then
if [[ $(type -t compopt) = "builtin" ]]; then
compopt +o nospace
fi
local index flag
flag="${cur%%=*}"
__index_of_word "${flag}" "${flags_with_completion[@]}"
COMPREPLY=()
if [[ ${index} -ge 0 ]]; then
PREFIX=""
cur="${cur#*=}"
${flags_completion[${index}]}
if [ -n "${ZSH_VERSION}" ]; then
# zfs completion needs --flag= prefix
eval "COMPREPLY=( \"\${COMPREPLY[@]/#/${flag}=}\" )"
fi
fi
fi
return 0;
;;
esac
# check if we are handling a flag with special work handling
local index
__index_of_word "${prev}" "${flags_with_completion[@]}"
if [[ ${index} -ge 0 ]]; then
${flags_completion[${index}]}
return
fi
# we are parsing a flag and don't have a special handler, no completion
if [[ ${cur} != "${words[cword]}" ]]; then
return
fi
local completions
completions=("${commands[@]}")
if [[ ${#must_have_one_noun[@]} -ne 0 ]]; then
completions=("${must_have_one_noun[@]}")
fi
if [[ ${#must_have_one_flag[@]} -ne 0 ]]; then
completions+=("${must_have_one_flag[@]}")
fi
COMPREPLY=( $(compgen -W "${completions[*]}" -- "$cur") )
if [[ ${#COMPREPLY[@]} -eq 0 && ${#noun_aliases[@]} -gt 0 && ${#must_have_one_noun[@]} -ne 0 ]]; then
COMPREPLY=( $(compgen -W "${noun_aliases[*]}" -- "$cur") )
fi
if [[ ${#COMPREPLY[@]} -eq 0 ]]; then
declare -F __custom_func >/dev/null && __custom_func
fi
# available in bash-completion >= 2, not always present on macOS
if declare -F __ltrim_colon_completions >/dev/null; then
__ltrim_colon_completions "$cur"
fi
}
# The arguments should be in the form "ext1|ext2|extn"
__handle_filename_extension_flag()
{
local ext="$1"
_filedir "@(${ext})"
}
__handle_subdirs_in_dir_flag()
{
local dir="$1"
pushd "${dir}" >/dev/null 2>&1 && _filedir -d && popd >/dev/null 2>&1
}
__handle_flag()
{
__debug "${FUNCNAME[0]}: c is $c words[c] is ${words[c]}"
# if a command required a flag, and we found it, unset must_have_one_flag()
local flagname=${words[c]}
local flagvalue
# if the word contained an =
if [[ ${words[c]} == *"="* ]]; then
flagvalue=${flagname#*=} # take in as flagvalue after the =
flagname=${flagname%%=*} # strip everything after the =
flagname="${flagname}=" # but put the = back
fi
__debug "${FUNCNAME[0]}: looking for ${flagname}"
if __contains_word "${flagname}" "${must_have_one_flag[@]}"; then
must_have_one_flag=()
fi
# if you set a flag which only applies to this command, don't show subcommands
if __contains_word "${flagname}" "${local_nonpersistent_flags[@]}"; then
commands=()
fi
# keep flag value with flagname as flaghash
if [ -n "${flagvalue}" ] ; then
flaghash[${flagname}]=${flagvalue}
elif [ -n "${words[ $((c+1)) ]}" ] ; then
flaghash[${flagname}]=${words[ $((c+1)) ]}
else
flaghash[${flagname}]="true" # pad "true" for bool flag
fi
# skip the argument to a two word flag
if __contains_word "${words[c]}" "${two_word_flags[@]}"; then
c=$((c+1))
# if we are looking for a flags value, don't show commands
if [[ $c -eq $cword ]]; then
commands=()
fi
fi
c=$((c+1))
}
__handle_noun()
{
__debug "${FUNCNAME[0]}: c is $c words[c] is ${words[c]}"
if __contains_word "${words[c]}" "${must_have_one_noun[@]}"; then
must_have_one_noun=()
elif __contains_word "${words[c]}" "${noun_aliases[@]}"; then
must_have_one_noun=()
fi
nouns+=("${words[c]}")
c=$((c+1))
}
__handle_command()
{
__debug "${FUNCNAME[0]}: c is $c words[c] is ${words[c]}"
local next_command
if [[ -n ${last_command} ]]; then
next_command="_${last_command}_${words[c]//:/__}"
else
if [[ $c -eq 0 ]]; then
next_command="_$(basename "${words[c]//:/__}")"
else
next_command="_${words[c]//:/__}"
fi
fi
c=$((c+1))
__debug "${FUNCNAME[0]}: looking for ${next_command}"
declare -F "$next_command" >/dev/null && $next_command
}
__handle_word()
{
if [[ $c -ge $cword ]]; then
__handle_reply
return
fi
__debug "${FUNCNAME[0]}: c is $c words[c] is ${words[c]}"
if [[ "${words[c]}" == -* ]]; then
__handle_flag
elif __contains_word "${words[c]}" "${commands[@]}"; then
__handle_command
elif [[ $c -eq 0 ]] && __contains_word "$(basename "${words[c]}")" "${commands[@]}"; then
__handle_command
else
__handle_noun
fi
__handle_word
}
`)
}
func writePostscript(buf *bytes.Buffer, name string) {
name = strings.Replace(name, ":", "__", -1)
buf.WriteString(fmt.Sprintf("__start_%s()\n", name))
buf.WriteString(fmt.Sprintf(`{
local cur prev words cword
declare -A flaghash 2>/dev/null || :
if declare -F _init_completion >/dev/null 2>&1; then
_init_completion -s || return
else
__my_init_completion -n "=" || return
fi
local c=0
local flags=()
local two_word_flags=()
local local_nonpersistent_flags=()
local flags_with_completion=()
local flags_completion=()
local commands=("%s")
local must_have_one_flag=()
local must_have_one_noun=()
local last_command
local nouns=()
__handle_word
}
`, name))
buf.WriteString(fmt.Sprintf(`if [[ $(type -t compopt) = "builtin" ]]; then
complete -o default -F __start_%s %s
else
complete -o default -o nospace -F __start_%s %s
fi
`, name, name, name, name))
buf.WriteString("# ex: ts=4 sw=4 et filetype=sh\n")
}
func writeCommands(buf *bytes.Buffer, cmd *Command) {
buf.WriteString(" commands=()\n")
for _, c := range cmd.Commands() {
if !c.IsAvailableCommand() || c == cmd.helpCommand {
continue
}
buf.WriteString(fmt.Sprintf(" commands+=(%q)\n", c.Name()))
}
buf.WriteString("\n")
}
func writeFlagHandler(buf *bytes.Buffer, name string, annotations map[string][]string) {
for key, value := range annotations {
switch key {
case BashCompFilenameExt:
buf.WriteString(fmt.Sprintf(" flags_with_completion+=(%q)\n", name))
var ext string
if len(value) > 0 {
ext = "__handle_filename_extension_flag " + strings.Join(value, "|")
} else {
ext = "_filedir"
}
buf.WriteString(fmt.Sprintf(" flags_completion+=(%q)\n", ext))
case BashCompCustom:
buf.WriteString(fmt.Sprintf(" flags_with_completion+=(%q)\n", name))
if len(value) > 0 {
handlers := strings.Join(value, "; ")
buf.WriteString(fmt.Sprintf(" flags_completion+=(%q)\n", handlers))
} else {
buf.WriteString(" flags_completion+=(:)\n")
}
case BashCompSubdirsInDir:
buf.WriteString(fmt.Sprintf(" flags_with_completion+=(%q)\n", name))
var ext string
if len(value) == 1 {
ext = "__handle_subdirs_in_dir_flag " + value[0]
} else {
ext = "_filedir -d"
}
buf.WriteString(fmt.Sprintf(" flags_completion+=(%q)\n", ext))
}
}
}
func writeShortFlag(buf *bytes.Buffer, flag *pflag.Flag) {
name := flag.Shorthand
format := " "
if len(flag.NoOptDefVal) == 0 {
format += "two_word_"
}
format += "flags+=(\"-%s\")\n"
buf.WriteString(fmt.Sprintf(format, name))
writeFlagHandler(buf, "-"+name, flag.Annotations)
}
func writeFlag(buf *bytes.Buffer, flag *pflag.Flag) {
name := flag.Name
format := " flags+=(\"--%s"
if len(flag.NoOptDefVal) == 0 {
format += "="
}
format += "\")\n"
buf.WriteString(fmt.Sprintf(format, name))
writeFlagHandler(buf, "--"+name, flag.Annotations)
}
func writeLocalNonPersistentFlag(buf *bytes.Buffer, flag *pflag.Flag) {
name := flag.Name
format := " local_nonpersistent_flags+=(\"--%s"
if len(flag.NoOptDefVal) == 0 {
format += "="
}
format += "\")\n"
buf.WriteString(fmt.Sprintf(format, name))
}
func writeFlags(buf *bytes.Buffer, cmd *Command) {
buf.WriteString(` flags=()
two_word_flags=()
local_nonpersistent_flags=()
flags_with_completion=()
flags_completion=()
`)
localNonPersistentFlags := cmd.LocalNonPersistentFlags()
cmd.NonInheritedFlags().VisitAll(func(flag *pflag.Flag) {
if nonCompletableFlag(flag) {
return
}
writeFlag(buf, flag)
if len(flag.Shorthand) > 0 {
writeShortFlag(buf, flag)
}
if localNonPersistentFlags.Lookup(flag.Name) != nil {
writeLocalNonPersistentFlag(buf, flag)
}
})
cmd.InheritedFlags().VisitAll(func(flag *pflag.Flag) {
if nonCompletableFlag(flag) {
return
}
writeFlag(buf, flag)
if len(flag.Shorthand) > 0 {
writeShortFlag(buf, flag)
}
})
buf.WriteString("\n")
}
func writeRequiredFlag(buf *bytes.Buffer, cmd *Command) {
buf.WriteString(" must_have_one_flag=()\n")
flags := cmd.NonInheritedFlags()
flags.VisitAll(func(flag *pflag.Flag) {
if nonCompletableFlag(flag) {
return
}
for key := range flag.Annotations {
switch key {
case BashCompOneRequiredFlag:
format := " must_have_one_flag+=(\"--%s"
if flag.Value.Type() != "bool" {
format += "="
}
format += "\")\n"
buf.WriteString(fmt.Sprintf(format, flag.Name))
if len(flag.Shorthand) > 0 {
buf.WriteString(fmt.Sprintf(" must_have_one_flag+=(\"-%s\")\n", flag.Shorthand))
}
}
}
})
}
func writeRequiredNouns(buf *bytes.Buffer, cmd *Command) {
buf.WriteString(" must_have_one_noun=()\n")
sort.Sort(sort.StringSlice(cmd.ValidArgs))
for _, value := range cmd.ValidArgs {
buf.WriteString(fmt.Sprintf(" must_have_one_noun+=(%q)\n", value))
}
}
func writeArgAliases(buf *bytes.Buffer, cmd *Command) {
buf.WriteString(" noun_aliases=()\n")
sort.Sort(sort.StringSlice(cmd.ArgAliases))
for _, value := range cmd.ArgAliases {
buf.WriteString(fmt.Sprintf(" noun_aliases+=(%q)\n", value))
}
}
func gen(buf *bytes.Buffer, cmd *Command) {
for _, c := range cmd.Commands() {
if !c.IsAvailableCommand() || c == cmd.helpCommand {
continue
}
gen(buf, c)
}
commandName := cmd.CommandPath()
commandName = strings.Replace(commandName, " ", "_", -1)
commandName = strings.Replace(commandName, ":", "__", -1)
buf.WriteString(fmt.Sprintf("_%s()\n{\n", commandName))
buf.WriteString(fmt.Sprintf(" last_command=%q\n", commandName))
writeCommands(buf, cmd)
writeFlags(buf, cmd)
writeRequiredFlag(buf, cmd)
writeRequiredNouns(buf, cmd)
writeArgAliases(buf, cmd)
buf.WriteString("}\n\n")
}
// GenBashCompletion generates bash completion file and writes to the passed writer.
func (cmd *Command) GenBashCompletion(w io.Writer) error {
buf := new(bytes.Buffer)
writePreamble(buf, cmd.Name())
if len(cmd.BashCompletionFunction) > 0 {
buf.WriteString(cmd.BashCompletionFunction + "\n")
}
gen(buf, cmd)
writePostscript(buf, cmd.Name())
_, err := buf.WriteTo(w)
return err
}
func nonCompletableFlag(flag *pflag.Flag) bool {
return flag.Hidden || len(flag.Deprecated) > 0
}
// GenBashCompletionFile generates bash completion file.
func (cmd *Command) GenBashCompletionFile(filename string) error {
outFile, err := os.Create(filename)
if err != nil {
return err
}
defer outFile.Close()
return cmd.GenBashCompletion(outFile)
}
// MarkFlagRequired adds the BashCompOneRequiredFlag annotation to the named flag, if it exists.
func (cmd *Command) MarkFlagRequired(name string) error {
return MarkFlagRequired(cmd.Flags(), name)
}
// MarkPersistentFlagRequired adds the BashCompOneRequiredFlag annotation to the named persistent flag, if it exists.
func (cmd *Command) MarkPersistentFlagRequired(name string) error {
return MarkFlagRequired(cmd.PersistentFlags(), name)
}
// MarkFlagRequired adds the BashCompOneRequiredFlag annotation to the named flag in the flag set, if it exists.
func MarkFlagRequired(flags *pflag.FlagSet, name string) error {
return flags.SetAnnotation(name, BashCompOneRequiredFlag, []string{"true"})
}
// MarkFlagFilename adds the BashCompFilenameExt annotation to the named flag, if it exists.
// Generated bash autocompletion will select filenames for the flag, limiting to named extensions if provided.
func (cmd *Command) MarkFlagFilename(name string, extensions ...string) error {
return MarkFlagFilename(cmd.Flags(), name, extensions...)
}
// MarkFlagCustom adds the BashCompCustom annotation to the named flag, if it exists.
// Generated bash autocompletion will call the bash function f for the flag.
func (cmd *Command) MarkFlagCustom(name string, f string) error {
return MarkFlagCustom(cmd.Flags(), name, f)
}
// MarkPersistentFlagFilename adds the BashCompFilenameExt annotation to the named persistent flag, if it exists.
// Generated bash autocompletion will select filenames for the flag, limiting to named extensions if provided.
func (cmd *Command) MarkPersistentFlagFilename(name string, extensions ...string) error {
return MarkFlagFilename(cmd.PersistentFlags(), name, extensions...)
}
// MarkFlagFilename adds the BashCompFilenameExt annotation to the named flag in the flag set, if it exists.
// Generated bash autocompletion will select filenames for the flag, limiting to named extensions if provided.
func MarkFlagFilename(flags *pflag.FlagSet, name string, extensions ...string) error {
return flags.SetAnnotation(name, BashCompFilenameExt, extensions)
}
// MarkFlagCustom adds the BashCompCustom annotation to the named flag in the flag set, if it exists.
// Generated bash autocompletion will call the bash function f for the flag.
func MarkFlagCustom(flags *pflag.FlagSet, name string, f string) error {
return flags.SetAnnotation(name, BashCompCustom, []string{f})
}

206
vendor/github.com/spf13/cobra/bash_completions.md generated vendored Normal file
View file

@ -0,0 +1,206 @@
# Generating Bash Completions For Your Own cobra.Command
Generating bash completions from a cobra command is incredibly easy. An actual program which does so for the kubernetes kubectl binary is as follows:
```go
package main
import (
"io/ioutil"
"os"
"github.com/GoogleCloudPlatform/kubernetes/pkg/kubectl/cmd"
)
func main() {
kubectl := cmd.NewFactory(nil).NewKubectlCommand(os.Stdin, ioutil.Discard, ioutil.Discard)
kubectl.GenBashCompletionFile("out.sh")
}
```
`out.sh` will get you completions of subcommands and flags. Copy it to `/etc/bash_completion.d/` as described [here](https://debian-administration.org/article/316/An_introduction_to_bash_completion_part_1) and reset your terminal to use autocompletion. If you make additional annotations to your code, you can get even more intelligent and flexible behavior.
## Creating your own custom functions
Some more actual code that works in kubernetes:
```bash
const (
bash_completion_func = `__kubectl_parse_get()
{
local kubectl_output out
if kubectl_output=$(kubectl get --no-headers "$1" 2>/dev/null); then
out=($(echo "${kubectl_output}" | awk '{print $1}'))
COMPREPLY=( $( compgen -W "${out[*]}" -- "$cur" ) )
fi
}
__kubectl_get_resource()
{
if [[ ${#nouns[@]} -eq 0 ]]; then
return 1
fi
__kubectl_parse_get ${nouns[${#nouns[@]} -1]}
if [[ $? -eq 0 ]]; then
return 0
fi
}
__custom_func() {
case ${last_command} in
kubectl_get | kubectl_describe | kubectl_delete | kubectl_stop)
__kubectl_get_resource
return
;;
*)
;;
esac
}
`)
```
And then I set that in my command definition:
```go
cmds := &cobra.Command{
Use: "kubectl",
Short: "kubectl controls the Kubernetes cluster manager",
Long: `kubectl controls the Kubernetes cluster manager.
Find more information at https://github.com/GoogleCloudPlatform/kubernetes.`,
Run: runHelp,
BashCompletionFunction: bash_completion_func,
}
```
The `BashCompletionFunction` option is really only valid/useful on the root command. Doing the above will cause `__custom_func()` to be called when the built in processor was unable to find a solution. In the case of kubernetes a valid command might look something like `kubectl get pod [mypod]`. If you type `kubectl get pod [tab][tab]` the `__customc_func()` will run because the cobra.Command only understood "kubectl" and "get." `__custom_func()` will see that the cobra.Command is "kubectl_get" and will thus call another helper `__kubectl_get_resource()`. `__kubectl_get_resource` will look at the 'nouns' collected. In our example the only noun will be `pod`. So it will call `__kubectl_parse_get pod`. `__kubectl_parse_get` will actually call out to kubernetes and get any pods. It will then set `COMPREPLY` to valid pods!
## Have the completions code complete your 'nouns'
In the above example "pod" was assumed to already be typed. But if you want `kubectl get [tab][tab]` to show a list of valid "nouns" you have to set them. Simplified code from `kubectl get` looks like:
```go
validArgs []string = { "pod", "node", "service", "replicationcontroller" }
cmd := &cobra.Command{
Use: "get [(-o|--output=)json|yaml|template|...] (RESOURCE [NAME] | RESOURCE/NAME ...)",
Short: "Display one or many resources",
Long: get_long,
Example: get_example,
Run: func(cmd *cobra.Command, args []string) {
err := RunGet(f, out, cmd, args)
util.CheckErr(err)
},
ValidArgs: validArgs,
}
```
Notice we put the "ValidArgs" on the "get" subcommand. Doing so will give results like
```bash
# kubectl get [tab][tab]
node pod replicationcontroller service
```
## Plural form and shortcuts for nouns
If your nouns have a number of aliases, you can define them alongside `ValidArgs` using `ArgAliases`:
```go
argAliases []string = { "pods", "nodes", "services", "svc", "replicationcontrollers", "rc" }
cmd := &cobra.Command{
...
ValidArgs: validArgs,
ArgAliases: argAliases
}
```
The aliases are not shown to the user on tab completion, but they are accepted as valid nouns by
the completion algorithm if entered manually, e.g. in:
```bash
# kubectl get rc [tab][tab]
backend frontend database
```
Note that without declaring `rc` as an alias, the completion algorithm would show the list of nouns
in this example again instead of the replication controllers.
## Mark flags as required
Most of the time completions will only show subcommands. But if a flag is required to make a subcommand work, you probably want it to show up when the user types [tab][tab]. Marking a flag as 'Required' is incredibly easy.
```go
cmd.MarkFlagRequired("pod")
cmd.MarkFlagRequired("container")
```
and you'll get something like
```bash
# kubectl exec [tab][tab][tab]
-c --container= -p --pod=
```
# Specify valid filename extensions for flags that take a filename
In this example we use --filename= and expect to get a json or yaml file as the argument. To make this easier we annotate the --filename flag with valid filename extensions.
```go
annotations := []string{"json", "yaml", "yml"}
annotation := make(map[string][]string)
annotation[cobra.BashCompFilenameExt] = annotations
flag := &pflag.Flag{
Name: "filename",
Shorthand: "f",
Usage: usage,
Value: value,
DefValue: value.String(),
Annotations: annotation,
}
cmd.Flags().AddFlag(flag)
```
Now when you run a command with this filename flag you'll get something like
```bash
# kubectl create -f
test/ example/ rpmbuild/
hello.yml test.json
```
So while there are many other files in the CWD it only shows me subdirs and those with valid extensions.
# Specifiy custom flag completion
Similar to the filename completion and filtering using cobra.BashCompFilenameExt, you can specifiy
a custom flag completion function with cobra.BashCompCustom:
```go
annotation := make(map[string][]string)
annotation[cobra.BashCompFilenameExt] = []string{"__kubectl_get_namespaces"}
flag := &pflag.Flag{
Name: "namespace",
Usage: usage,
Annotations: annotation,
}
cmd.Flags().AddFlag(flag)
```
In addition add the `__handle_namespace_flag` implementation in the `BashCompletionFunction`
value, e.g.:
```bash
__kubectl_get_namespaces()
{
local template
template="{{ range .items }}{{ .metadata.name }} {{ end }}"
local kubectl_out
if kubectl_out=$(kubectl get -o template --template="${template}" namespace 2>/dev/null); then
COMPREPLY=( $( compgen -W "${kubectl_out}[*]" -- "$cur" ) )
fi
}
```

181
vendor/github.com/spf13/cobra/cobra.go generated vendored Normal file
View file

@ -0,0 +1,181 @@
// Copyright © 2013 Steve Francia <spf@spf13.com>.
//
// Licensed under the Apache License, Version 2.0 (the "License");
// you may not use this file except in compliance with the License.
// You may obtain a copy of the License at
// http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing, software
// distributed under the License is distributed on an "AS IS" BASIS,
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
// See the License for the specific language governing permissions and
// limitations under the License.
// Commands similar to git, go tools and other modern CLI tools
// inspired by go, go-Commander, gh and subcommand
package cobra
import (
"fmt"
"io"
"reflect"
"strconv"
"strings"
"text/template"
"unicode"
)
var templateFuncs = template.FuncMap{
"trim": strings.TrimSpace,
"trimRightSpace": trimRightSpace,
"trimTrailingWhitespaces": trimRightSpace,
"appendIfNotPresent": appendIfNotPresent,
"rpad": rpad,
"gt": Gt,
"eq": Eq,
}
var initializers []func()
// EnablePrefixMatching allows to set automatic prefix matching. Automatic prefix matching can be a dangerous thing
// to automatically enable in CLI tools.
// Set this to true to enable it.
var EnablePrefixMatching = false
// EnableCommandSorting controls sorting of the slice of commands, which is turned on by default.
// To disable sorting, set it to false.
var EnableCommandSorting = true
// AddTemplateFunc adds a template function that's available to Usage and Help
// template generation.
func AddTemplateFunc(name string, tmplFunc interface{}) {
templateFuncs[name] = tmplFunc
}
// AddTemplateFuncs adds multiple template functions that are available to Usage and
// Help template generation.
func AddTemplateFuncs(tmplFuncs template.FuncMap) {
for k, v := range tmplFuncs {
templateFuncs[k] = v
}
}
// OnInitialize takes a series of func() arguments and appends them to a slice of func().
func OnInitialize(y ...func()) {
initializers = append(initializers, y...)
}
// FIXME Gt is unused by cobra and should be removed in a version 2. It exists only for compatibility with users of cobra.
// Gt takes two types and checks whether the first type is greater than the second. In case of types Arrays, Chans,
// Maps and Slices, Gt will compare their lengths. Ints are compared directly while strings are first parsed as
// ints and then compared.
func Gt(a interface{}, b interface{}) bool {
var left, right int64
av := reflect.ValueOf(a)
switch av.Kind() {
case reflect.Array, reflect.Chan, reflect.Map, reflect.Slice:
left = int64(av.Len())
case reflect.Int, reflect.Int8, reflect.Int16, reflect.Int32, reflect.Int64:
left = av.Int()
case reflect.String:
left, _ = strconv.ParseInt(av.String(), 10, 64)
}
bv := reflect.ValueOf(b)
switch bv.Kind() {
case reflect.Array, reflect.Chan, reflect.Map, reflect.Slice:
right = int64(bv.Len())
case reflect.Int, reflect.Int8, reflect.Int16, reflect.Int32, reflect.Int64:
right = bv.Int()
case reflect.String:
right, _ = strconv.ParseInt(bv.String(), 10, 64)
}
return left > right
}
// FIXME Eq is unused by cobra and should be removed in a version 2. It exists only for compatibility with users of cobra.
// Eq takes two types and checks whether they are equal. Supported types are int and string. Unsupported types will panic.
func Eq(a interface{}, b interface{}) bool {
av := reflect.ValueOf(a)
bv := reflect.ValueOf(b)
switch av.Kind() {
case reflect.Array, reflect.Chan, reflect.Map, reflect.Slice:
panic("Eq called on unsupported type")
case reflect.Int, reflect.Int8, reflect.Int16, reflect.Int32, reflect.Int64:
return av.Int() == bv.Int()
case reflect.String:
return av.String() == bv.String()
}
return false
}
func trimRightSpace(s string) string {
return strings.TrimRightFunc(s, unicode.IsSpace)
}
// FIXME appendIfNotPresent is unused by cobra and should be removed in a version 2. It exists only for compatibility with users of cobra.
// appendIfNotPresent will append stringToAppend to the end of s, but only if it's not yet present in s.
func appendIfNotPresent(s, stringToAppend string) string {
if strings.Contains(s, stringToAppend) {
return s
}
return s + " " + stringToAppend
}
// rpad adds padding to the right of a string.
func rpad(s string, padding int) string {
template := fmt.Sprintf("%%-%ds", padding)
return fmt.Sprintf(template, s)
}
// tmpl executes the given template text on data, writing the result to w.
func tmpl(w io.Writer, text string, data interface{}) error {
t := template.New("top")
t.Funcs(templateFuncs)
template.Must(t.Parse(text))
return t.Execute(w, data)
}
// ld compares two strings and returns the levenshtein distance between them.
func ld(s, t string, ignoreCase bool) int {
if ignoreCase {
s = strings.ToLower(s)
t = strings.ToLower(t)
}
d := make([][]int, len(s)+1)
for i := range d {
d[i] = make([]int, len(t)+1)
}
for i := range d {
d[i][0] = i
}
for j := range d[0] {
d[0][j] = j
}
for j := 1; j <= len(t); j++ {
for i := 1; i <= len(s); i++ {
if s[i-1] == t[j-1] {
d[i][j] = d[i-1][j-1]
} else {
min := d[i-1][j]
if d[i][j-1] < min {
min = d[i][j-1]
}
if d[i-1][j-1] < min {
min = d[i-1][j-1]
}
d[i][j] = min + 1
}
}
}
return d[len(s)][len(t)]
}

1289
vendor/github.com/spf13/cobra/command.go generated vendored Normal file

File diff suppressed because it is too large Load diff

5
vendor/github.com/spf13/cobra/command_notwin.go generated vendored Normal file
View file

@ -0,0 +1,5 @@
// +build !windows
package cobra
var preExecHookFn func(*Command)

26
vendor/github.com/spf13/cobra/command_win.go generated vendored Normal file
View file

@ -0,0 +1,26 @@
// +build windows
package cobra
import (
"os"
"time"
"github.com/inconshreveable/mousetrap"
)
var preExecHookFn = preExecHook
// enables an information splash screen on Windows if the CLI is started from explorer.exe.
var MousetrapHelpText string = `This is a command line tool
You need to open cmd.exe and run it from there.
`
func preExecHook(c *Command) {
if mousetrap.StartedByExplorer() {
c.Print(MousetrapHelpText)
time.Sleep(5 * time.Second)
os.Exit(1)
}
}

296
vendor/github.com/spf13/pflag/README.md generated vendored Normal file
View file

@ -0,0 +1,296 @@
[![Build Status](https://travis-ci.org/spf13/pflag.svg?branch=master)](https://travis-ci.org/spf13/pflag)
[![Go Report Card](https://goreportcard.com/badge/github.com/spf13/pflag)](https://goreportcard.com/report/github.com/spf13/pflag)
[![GoDoc](https://godoc.org/github.com/spf13/pflag?status.svg)](https://godoc.org/github.com/spf13/pflag)
## Description
pflag is a drop-in replacement for Go's flag package, implementing
POSIX/GNU-style --flags.
pflag is compatible with the [GNU extensions to the POSIX recommendations
for command-line options][1]. For a more precise description, see the
"Command-line flag syntax" section below.
[1]: http://www.gnu.org/software/libc/manual/html_node/Argument-Syntax.html
pflag is available under the same style of BSD license as the Go language,
which can be found in the LICENSE file.
## Installation
pflag is available using the standard `go get` command.
Install by running:
go get github.com/spf13/pflag
Run tests by running:
go test github.com/spf13/pflag
## Usage
pflag is a drop-in replacement of Go's native flag package. If you import
pflag under the name "flag" then all code should continue to function
with no changes.
``` go
import flag "github.com/spf13/pflag"
```
There is one exception to this: if you directly instantiate the Flag struct
there is one more field "Shorthand" that you will need to set.
Most code never instantiates this struct directly, and instead uses
functions such as String(), BoolVar(), and Var(), and is therefore
unaffected.
Define flags using flag.String(), Bool(), Int(), etc.
This declares an integer flag, -flagname, stored in the pointer ip, with type *int.
``` go
var ip *int = flag.Int("flagname", 1234, "help message for flagname")
```
If you like, you can bind the flag to a variable using the Var() functions.
``` go
var flagvar int
func init() {
flag.IntVar(&flagvar, "flagname", 1234, "help message for flagname")
}
```
Or you can create custom flags that satisfy the Value interface (with
pointer receivers) and couple them to flag parsing by
``` go
flag.Var(&flagVal, "name", "help message for flagname")
```
For such flags, the default value is just the initial value of the variable.
After all flags are defined, call
``` go
flag.Parse()
```
to parse the command line into the defined flags.
Flags may then be used directly. If you're using the flags themselves,
they are all pointers; if you bind to variables, they're values.
``` go
fmt.Println("ip has value ", *ip)
fmt.Println("flagvar has value ", flagvar)
```
There are helpers function to get values later if you have the FlagSet but
it was difficult to keep up with all of the flag pointers in your code.
If you have a pflag.FlagSet with a flag called 'flagname' of type int you
can use GetInt() to get the int value. But notice that 'flagname' must exist
and it must be an int. GetString("flagname") will fail.
``` go
i, err := flagset.GetInt("flagname")
```
After parsing, the arguments after the flag are available as the
slice flag.Args() or individually as flag.Arg(i).
The arguments are indexed from 0 through flag.NArg()-1.
The pflag package also defines some new functions that are not in flag,
that give one-letter shorthands for flags. You can use these by appending
'P' to the name of any function that defines a flag.
``` go
var ip = flag.IntP("flagname", "f", 1234, "help message")
var flagvar bool
func init() {
flag.BoolVarP(&flagvar, "boolname", "b", true, "help message")
}
flag.VarP(&flagVal, "varname", "v", "help message")
```
Shorthand letters can be used with single dashes on the command line.
Boolean shorthand flags can be combined with other shorthand flags.
The default set of command-line flags is controlled by
top-level functions. The FlagSet type allows one to define
independent sets of flags, such as to implement subcommands
in a command-line interface. The methods of FlagSet are
analogous to the top-level functions for the command-line
flag set.
## Setting no option default values for flags
After you create a flag it is possible to set the pflag.NoOptDefVal for
the given flag. Doing this changes the meaning of the flag slightly. If
a flag has a NoOptDefVal and the flag is set on the command line without
an option the flag will be set to the NoOptDefVal. For example given:
``` go
var ip = flag.IntP("flagname", "f", 1234, "help message")
flag.Lookup("flagname").NoOptDefVal = "4321"
```
Would result in something like
| Parsed Arguments | Resulting Value |
| ------------- | ------------- |
| --flagname=1357 | ip=1357 |
| --flagname | ip=4321 |
| [nothing] | ip=1234 |
## Command line flag syntax
```
--flag // boolean flags, or flags with no option default values
--flag x // only on flags without a default value
--flag=x
```
Unlike the flag package, a single dash before an option means something
different than a double dash. Single dashes signify a series of shorthand
letters for flags. All but the last shorthand letter must be boolean flags
or a flag with a default value
```
// boolean or flags where the 'no option default value' is set
-f
-f=true
-abc
but
-b true is INVALID
// non-boolean and flags without a 'no option default value'
-n 1234
-n=1234
-n1234
// mixed
-abcs "hello"
-absd="hello"
-abcs1234
```
Flag parsing stops after the terminator "--". Unlike the flag package,
flags can be interspersed with arguments anywhere on the command line
before this terminator.
Integer flags accept 1234, 0664, 0x1234 and may be negative.
Boolean flags (in their long form) accept 1, 0, t, f, true, false,
TRUE, FALSE, True, False.
Duration flags accept any input valid for time.ParseDuration.
## Mutating or "Normalizing" Flag names
It is possible to set a custom flag name 'normalization function.' It allows flag names to be mutated both when created in the code and when used on the command line to some 'normalized' form. The 'normalized' form is used for comparison. Two examples of using the custom normalization func follow.
**Example #1**: You want -, _, and . in flags to compare the same. aka --my-flag == --my_flag == --my.flag
``` go
func wordSepNormalizeFunc(f *pflag.FlagSet, name string) pflag.NormalizedName {
from := []string{"-", "_"}
to := "."
for _, sep := range from {
name = strings.Replace(name, sep, to, -1)
}
return pflag.NormalizedName(name)
}
myFlagSet.SetNormalizeFunc(wordSepNormalizeFunc)
```
**Example #2**: You want to alias two flags. aka --old-flag-name == --new-flag-name
``` go
func aliasNormalizeFunc(f *pflag.FlagSet, name string) pflag.NormalizedName {
switch name {
case "old-flag-name":
name = "new-flag-name"
break
}
return pflag.NormalizedName(name)
}
myFlagSet.SetNormalizeFunc(aliasNormalizeFunc)
```
## Deprecating a flag or its shorthand
It is possible to deprecate a flag, or just its shorthand. Deprecating a flag/shorthand hides it from help text and prints a usage message when the deprecated flag/shorthand is used.
**Example #1**: You want to deprecate a flag named "badflag" as well as inform the users what flag they should use instead.
```go
// deprecate a flag by specifying its name and a usage message
flags.MarkDeprecated("badflag", "please use --good-flag instead")
```
This hides "badflag" from help text, and prints `Flag --badflag has been deprecated, please use --good-flag instead` when "badflag" is used.
**Example #2**: You want to keep a flag name "noshorthandflag" but deprecate its shortname "n".
```go
// deprecate a flag shorthand by specifying its flag name and a usage message
flags.MarkShorthandDeprecated("noshorthandflag", "please use --noshorthandflag only")
```
This hides the shortname "n" from help text, and prints `Flag shorthand -n has been deprecated, please use --noshorthandflag only` when the shorthand "n" is used.
Note that usage message is essential here, and it should not be empty.
## Hidden flags
It is possible to mark a flag as hidden, meaning it will still function as normal, however will not show up in usage/help text.
**Example**: You have a flag named "secretFlag" that you need for internal use only and don't want it showing up in help text, or for its usage text to be available.
```go
// hide a flag by specifying its name
flags.MarkHidden("secretFlag")
```
## Disable sorting of flags
`pflag` allows you to disable sorting of flags for help and usage message.
**Example**:
```go
flags.BoolP("verbose", "v", false, "verbose output")
flags.String("coolflag", "yeaah", "it's really cool flag")
flags.Int("usefulflag", 777, "sometimes it's very useful")
flags.SortFlags = false
flags.PrintDefaults()
```
**Output**:
```
-v, --verbose verbose output
--coolflag string it's really cool flag (default "yeaah")
--usefulflag int sometimes it's very useful (default 777)
```
## Supporting Go flags when using pflag
In order to support flags defined using Go's `flag` package, they must be added to the `pflag` flagset. This is usually necessary
to support flags defined by third-party dependencies (e.g. `golang/glog`).
**Example**: You want to add the Go flags to the `CommandLine` flagset
```go
import (
goflag "flag"
flag "github.com/spf13/pflag"
)
var ip *int = flag.Int("flagname", 1234, "help message for flagname")
func main() {
flag.CommandLine.AddGoFlagSet(goflag.CommandLine)
flag.Parse()
}
```
## More info
You can see the full reference documentation of the pflag package
[at godoc.org][3], or through go's standard documentation system by
running `godoc -http=:6060` and browsing to
[http://localhost:6060/pkg/github.com/spf13/pflag][2] after
installation.
[2]: http://localhost:6060/pkg/github.com/spf13/pflag
[3]: http://godoc.org/github.com/spf13/pflag

View file

@ -1,9 +1,6 @@
package pflag
import (
"fmt"
"strconv"
)
import "strconv"
// optional interface to indicate boolean flags that can be
// supplied without "=value" text
@ -30,7 +27,7 @@ func (b *boolValue) Type() string {
return "bool"
}
func (b *boolValue) String() string { return fmt.Sprintf("%v", *b) }
func (b *boolValue) String() string { return strconv.FormatBool(bool(*b)) }
func (b *boolValue) IsBoolFlag() bool { return true }

147
vendor/github.com/spf13/pflag/bool_slice.go generated vendored Normal file
View file

@ -0,0 +1,147 @@
package pflag
import (
"io"
"strconv"
"strings"
)
// -- boolSlice Value
type boolSliceValue struct {
value *[]bool
changed bool
}
func newBoolSliceValue(val []bool, p *[]bool) *boolSliceValue {
bsv := new(boolSliceValue)
bsv.value = p
*bsv.value = val
return bsv
}
// Set converts, and assigns, the comma-separated boolean argument string representation as the []bool value of this flag.
// If Set is called on a flag that already has a []bool assigned, the newly converted values will be appended.
func (s *boolSliceValue) Set(val string) error {
// remove all quote characters
rmQuote := strings.NewReplacer(`"`, "", `'`, "", "`", "")
// read flag arguments with CSV parser
boolStrSlice, err := readAsCSV(rmQuote.Replace(val))
if err != nil && err != io.EOF {
return err
}
// parse boolean values into slice
out := make([]bool, 0, len(boolStrSlice))
for _, boolStr := range boolStrSlice {
b, err := strconv.ParseBool(strings.TrimSpace(boolStr))
if err != nil {
return err
}
out = append(out, b)
}
if !s.changed {
*s.value = out
} else {
*s.value = append(*s.value, out...)
}
s.changed = true
return nil
}
// Type returns a string that uniquely represents this flag's type.
func (s *boolSliceValue) Type() string {
return "boolSlice"
}
// String defines a "native" format for this boolean slice flag value.
func (s *boolSliceValue) String() string {
boolStrSlice := make([]string, len(*s.value))
for i, b := range *s.value {
boolStrSlice[i] = strconv.FormatBool(b)
}
out, _ := writeAsCSV(boolStrSlice)
return "[" + out + "]"
}
func boolSliceConv(val string) (interface{}, error) {
val = strings.Trim(val, "[]")
// Empty string would cause a slice with one (empty) entry
if len(val) == 0 {
return []bool{}, nil
}
ss := strings.Split(val, ",")
out := make([]bool, len(ss))
for i, t := range ss {
var err error
out[i], err = strconv.ParseBool(t)
if err != nil {
return nil, err
}
}
return out, nil
}
// GetBoolSlice returns the []bool value of a flag with the given name.
func (f *FlagSet) GetBoolSlice(name string) ([]bool, error) {
val, err := f.getFlagType(name, "boolSlice", boolSliceConv)
if err != nil {
return []bool{}, err
}
return val.([]bool), nil
}
// BoolSliceVar defines a boolSlice flag with specified name, default value, and usage string.
// The argument p points to a []bool variable in which to store the value of the flag.
func (f *FlagSet) BoolSliceVar(p *[]bool, name string, value []bool, usage string) {
f.VarP(newBoolSliceValue(value, p), name, "", usage)
}
// BoolSliceVarP is like BoolSliceVar, but accepts a shorthand letter that can be used after a single dash.
func (f *FlagSet) BoolSliceVarP(p *[]bool, name, shorthand string, value []bool, usage string) {
f.VarP(newBoolSliceValue(value, p), name, shorthand, usage)
}
// BoolSliceVar defines a []bool flag with specified name, default value, and usage string.
// The argument p points to a []bool variable in which to store the value of the flag.
func BoolSliceVar(p *[]bool, name string, value []bool, usage string) {
CommandLine.VarP(newBoolSliceValue(value, p), name, "", usage)
}
// BoolSliceVarP is like BoolSliceVar, but accepts a shorthand letter that can be used after a single dash.
func BoolSliceVarP(p *[]bool, name, shorthand string, value []bool, usage string) {
CommandLine.VarP(newBoolSliceValue(value, p), name, shorthand, usage)
}
// BoolSlice defines a []bool flag with specified name, default value, and usage string.
// The return value is the address of a []bool variable that stores the value of the flag.
func (f *FlagSet) BoolSlice(name string, value []bool, usage string) *[]bool {
p := []bool{}
f.BoolSliceVarP(&p, name, "", value, usage)
return &p
}
// BoolSliceP is like BoolSlice, but accepts a shorthand letter that can be used after a single dash.
func (f *FlagSet) BoolSliceP(name, shorthand string, value []bool, usage string) *[]bool {
p := []bool{}
f.BoolSliceVarP(&p, name, shorthand, value, usage)
return &p
}
// BoolSlice defines a []bool flag with specified name, default value, and usage string.
// The return value is the address of a []bool variable that stores the value of the flag.
func BoolSlice(name string, value []bool, usage string) *[]bool {
return CommandLine.BoolSliceP(name, "", value, usage)
}
// BoolSliceP is like BoolSlice, but accepts a shorthand letter that can be used after a single dash.
func BoolSliceP(name, shorthand string, value []bool, usage string) *[]bool {
return CommandLine.BoolSliceP(name, shorthand, value, usage)
}

View file

@ -1,9 +1,6 @@
package pflag
import (
"fmt"
"strconv"
)
import "strconv"
// -- count Value
type countValue int
@ -28,7 +25,7 @@ func (i *countValue) Type() string {
return "count"
}
func (i *countValue) String() string { return fmt.Sprintf("%v", *i) }
func (i *countValue) String() string { return strconv.Itoa(int(*i)) }
func countConv(sval string) (interface{}, error) {
i, err := strconv.Atoi(sval)
@ -86,7 +83,9 @@ func (f *FlagSet) CountP(name, shorthand string, usage string) *int {
return p
}
// Count like Count only the flag is placed on the CommandLine isntead of a given flag set
// Count defines a count flag with specified name, default value, and usage string.
// The return value is the address of an int variable that stores the value of the flag.
// A count flag will add 1 to its value evey time it is found on the command line
func Count(name string, usage string) *int {
return CommandLine.CountP(name, "", usage)
}

428
vendor/github.com/spf13/pflag/flag.go generated vendored
View file

@ -16,9 +16,9 @@ pflag is a drop-in replacement of Go's native flag package. If you import
pflag under the name "flag" then all code should continue to function
with no changes.
import flag "github.com/ogier/pflag"
import flag "github.com/spf13/pflag"
There is one exception to this: if you directly instantiate the Flag struct
There is one exception to this: if you directly instantiate the Flag struct
there is one more field "Shorthand" that you will need to set.
Most code never instantiates this struct directly, and instead uses
functions such as String(), BoolVar(), and Var(), and is therefore
@ -134,14 +134,21 @@ type FlagSet struct {
// a custom error handler.
Usage func()
// SortFlags is used to indicate, if user wants to have sorted flags in
// help/usage messages.
SortFlags bool
name string
parsed bool
actual map[NormalizedName]*Flag
orderedActual []*Flag
sortedActual []*Flag
formal map[NormalizedName]*Flag
orderedFormal []*Flag
sortedFormal []*Flag
shorthands map[byte]*Flag
args []string // arguments after flags
argsLenAtDash int // len(args) when a '--' was located when parsing, or -1 if no --
exitOnError bool // does the program exit if there's an error?
errorHandling ErrorHandling
output io.Writer // nil means stderr; use out() accessor
interspersed bool // allow interspersed option/non-option args
@ -156,7 +163,7 @@ type Flag struct {
Value Value // value as set
DefValue string // default value (as text); for usage message
Changed bool // If the user set the value (or if left to default)
NoOptDefVal string //default value (as text); if the flag is on the command line without any options
NoOptDefVal string // default value (as text); if the flag is on the command line without any options
Deprecated string // If this flag is deprecated, this string is the new or now thing to use
Hidden bool // used by cobra.Command to allow flags to be hidden from help/usage text
ShorthandDeprecated string // If the shorthand of this flag is deprecated, this string is the new or now thing to use
@ -194,11 +201,13 @@ func sortFlags(flags map[NormalizedName]*Flag) []*Flag {
// "--getUrl" which may also be translated to "geturl" and everything will work.
func (f *FlagSet) SetNormalizeFunc(n func(f *FlagSet, name string) NormalizedName) {
f.normalizeNameFunc = n
for k, v := range f.formal {
delete(f.formal, k)
nname := f.normalizeFlagName(string(k))
f.formal[nname] = v
f.sortedFormal = f.sortedFormal[:0]
for k, v := range f.orderedFormal {
delete(f.formal, NormalizedName(v.Name))
nname := f.normalizeFlagName(v.Name)
v.Name = string(nname)
f.formal[nname] = v
f.orderedFormal[k] = v
}
}
@ -229,10 +238,25 @@ func (f *FlagSet) SetOutput(output io.Writer) {
f.output = output
}
// VisitAll visits the flags in lexicographical order, calling fn for each.
// VisitAll visits the flags in lexicographical order or
// in primordial order if f.SortFlags is false, calling fn for each.
// It visits all flags, even those not set.
func (f *FlagSet) VisitAll(fn func(*Flag)) {
for _, flag := range sortFlags(f.formal) {
if len(f.formal) == 0 {
return
}
var flags []*Flag
if f.SortFlags {
if len(f.formal) != len(f.sortedFormal) {
f.sortedFormal = sortFlags(f.formal)
}
flags = f.sortedFormal
} else {
flags = f.orderedFormal
}
for _, flag := range flags {
fn(flag)
}
}
@ -253,22 +277,39 @@ func (f *FlagSet) HasAvailableFlags() bool {
return false
}
// VisitAll visits the command-line flags in lexicographical order, calling
// fn for each. It visits all flags, even those not set.
// VisitAll visits the command-line flags in lexicographical order or
// in primordial order if f.SortFlags is false, calling fn for each.
// It visits all flags, even those not set.
func VisitAll(fn func(*Flag)) {
CommandLine.VisitAll(fn)
}
// Visit visits the flags in lexicographical order, calling fn for each.
// Visit visits the flags in lexicographical order or
// in primordial order if f.SortFlags is false, calling fn for each.
// It visits only those flags that have been set.
func (f *FlagSet) Visit(fn func(*Flag)) {
for _, flag := range sortFlags(f.actual) {
if len(f.actual) == 0 {
return
}
var flags []*Flag
if f.SortFlags {
if len(f.actual) != len(f.sortedActual) {
f.sortedActual = sortFlags(f.actual)
}
flags = f.sortedActual
} else {
flags = f.orderedActual
}
for _, flag := range flags {
fn(flag)
}
}
// Visit visits the command-line flags in lexicographical order, calling fn
// for each. It visits only those flags that have been set.
// Visit visits the command-line flags in lexicographical order or
// in primordial order if f.SortFlags is false, calling fn for each.
// It visits only those flags that have been set.
func Visit(fn func(*Flag)) {
CommandLine.Visit(fn)
}
@ -278,6 +319,22 @@ func (f *FlagSet) Lookup(name string) *Flag {
return f.lookup(f.normalizeFlagName(name))
}
// ShorthandLookup returns the Flag structure of the short handed flag,
// returning nil if none exists.
// It panics, if len(name) > 1.
func (f *FlagSet) ShorthandLookup(name string) *Flag {
if name == "" {
return nil
}
if len(name) > 1 {
msg := fmt.Sprintf("can not look up shorthand which is more than one ASCII character: %q", name)
fmt.Fprintf(f.out(), msg)
panic(msg)
}
c := name[0]
return f.shorthands[c]
}
// lookup returns the Flag structure of the named flag, returning nil if none exists.
func (f *FlagSet) lookup(name NormalizedName) *Flag {
return f.formal[name]
@ -319,7 +376,7 @@ func (f *FlagSet) MarkDeprecated(name string, usageMessage string) error {
if flag == nil {
return fmt.Errorf("flag %q does not exist", name)
}
if len(usageMessage) == 0 {
if usageMessage == "" {
return fmt.Errorf("deprecated message for flag %q must be set", name)
}
flag.Deprecated = usageMessage
@ -334,7 +391,7 @@ func (f *FlagSet) MarkShorthandDeprecated(name string, usageMessage string) erro
if flag == nil {
return fmt.Errorf("flag %q does not exist", name)
}
if len(usageMessage) == 0 {
if usageMessage == "" {
return fmt.Errorf("deprecated message for flag %q must be set", name)
}
flag.ShorthandDeprecated = usageMessage
@ -358,6 +415,12 @@ func Lookup(name string) *Flag {
return CommandLine.Lookup(name)
}
// ShorthandLookup returns the Flag structure of the short handed flag,
// returning nil if none exists.
func ShorthandLookup(name string) *Flag {
return CommandLine.ShorthandLookup(name)
}
// Set sets the value of the named flag.
func (f *FlagSet) Set(name, value string) error {
normalName := f.normalizeFlagName(name)
@ -365,17 +428,28 @@ func (f *FlagSet) Set(name, value string) error {
if !ok {
return fmt.Errorf("no such flag -%v", name)
}
err := flag.Value.Set(value)
if err != nil {
return err
var flagName string
if flag.Shorthand != "" && flag.ShorthandDeprecated == "" {
flagName = fmt.Sprintf("-%s, --%s", flag.Shorthand, flag.Name)
} else {
flagName = fmt.Sprintf("--%s", flag.Name)
}
return fmt.Errorf("invalid argument %q for %q flag: %v", value, flagName, err)
}
if f.actual == nil {
f.actual = make(map[NormalizedName]*Flag)
}
f.actual[normalName] = flag
f.orderedActual = append(f.orderedActual, flag)
flag.Changed = true
if len(flag.Deprecated) > 0 {
fmt.Fprintf(os.Stderr, "Flag --%s has been deprecated, %s\n", flag.Name, flag.Deprecated)
if flag.Deprecated != "" {
fmt.Fprintf(f.out(), "Flag --%s has been deprecated, %s\n", flag.Name, flag.Deprecated)
}
return nil
}
@ -416,23 +490,39 @@ func Set(name, value string) error {
// otherwise, the default values of all defined flags in the set.
func (f *FlagSet) PrintDefaults() {
usages := f.FlagUsages()
fmt.Fprintf(f.out(), "%s", usages)
fmt.Fprint(f.out(), usages)
}
// isZeroValue guesses whether the string represents the zero
// value for a flag. It is not accurate but in practice works OK.
func isZeroValue(value string) bool {
switch value {
case "false":
return true
case "<nil>":
return true
case "":
return true
case "0":
return true
// defaultIsZeroValue returns true if the default value for this flag represents
// a zero value.
func (f *Flag) defaultIsZeroValue() bool {
switch f.Value.(type) {
case boolFlag:
return f.DefValue == "false"
case *durationValue:
// Beginning in Go 1.7, duration zero values are "0s"
return f.DefValue == "0" || f.DefValue == "0s"
case *intValue, *int8Value, *int32Value, *int64Value, *uintValue, *uint8Value, *uint16Value, *uint32Value, *uint64Value, *countValue, *float32Value, *float64Value:
return f.DefValue == "0"
case *stringValue:
return f.DefValue == ""
case *ipValue, *ipMaskValue, *ipNetValue:
return f.DefValue == "<nil>"
case *intSliceValue, *stringSliceValue, *stringArrayValue:
return f.DefValue == "[]"
default:
switch f.Value.String() {
case "false":
return true
case "<nil>":
return true
case "":
return true
case "0":
return true
}
return false
}
return false
}
// UnquoteUsage extracts a back-quoted name from the usage
@ -455,53 +545,117 @@ func UnquoteUsage(flag *Flag) (name string, usage string) {
break // Only one back quote; use type name.
}
}
// No explicit name, so use type if we can find one.
name = "value"
switch flag.Value.(type) {
case boolFlag:
name = flag.Value.Type()
switch name {
case "bool":
name = ""
case *durationValue:
name = "duration"
case *float64Value:
case "float64":
name = "float"
case *intValue, *int64Value:
case "int64":
name = "int"
case *stringValue:
name = "string"
case *uintValue, *uint64Value:
case "uint64":
name = "uint"
}
return
}
// FlagUsages Returns a string containing the usage information for all flags in
// the FlagSet
func (f *FlagSet) FlagUsages() string {
x := new(bytes.Buffer)
// Splits the string `s` on whitespace into an initial substring up to
// `i` runes in length and the remainder. Will go `slop` over `i` if
// that encompasses the entire string (which allows the caller to
// avoid short orphan words on the final line).
func wrapN(i, slop int, s string) (string, string) {
if i+slop > len(s) {
return s, ""
}
w := strings.LastIndexAny(s[:i], " \t")
if w <= 0 {
return s, ""
}
return s[:w], s[w+1:]
}
// Wraps the string `s` to a maximum width `w` with leading indent
// `i`. The first line is not indented (this is assumed to be done by
// caller). Pass `w` == 0 to do no wrapping
func wrap(i, w int, s string) string {
if w == 0 {
return s
}
// space between indent i and end of line width w into which
// we should wrap the text.
wrap := w - i
var r, l string
// Not enough space for sensible wrapping. Wrap as a block on
// the next line instead.
if wrap < 24 {
i = 16
wrap = w - i
r += "\n" + strings.Repeat(" ", i)
}
// If still not enough space then don't even try to wrap.
if wrap < 24 {
return s
}
// Try to avoid short orphan words on the final line, by
// allowing wrapN to go a bit over if that would fit in the
// remainder of the line.
slop := 5
wrap = wrap - slop
// Handle first line, which is indented by the caller (or the
// special case above)
l, s = wrapN(wrap, slop, s)
r = r + l
// Now wrap the rest
for s != "" {
var t string
t, s = wrapN(wrap, slop, s)
r = r + "\n" + strings.Repeat(" ", i) + t
}
return r
}
// FlagUsagesWrapped returns a string containing the usage information
// for all flags in the FlagSet. Wrapped to `cols` columns (0 for no
// wrapping)
func (f *FlagSet) FlagUsagesWrapped(cols int) string {
buf := new(bytes.Buffer)
lines := make([]string, 0, len(f.formal))
maxlen := 0
f.VisitAll(func(flag *Flag) {
if len(flag.Deprecated) > 0 || flag.Hidden {
if flag.Deprecated != "" || flag.Hidden {
return
}
line := ""
if len(flag.Shorthand) > 0 && len(flag.ShorthandDeprecated) == 0 {
if flag.Shorthand != "" && flag.ShorthandDeprecated == "" {
line = fmt.Sprintf(" -%s, --%s", flag.Shorthand, flag.Name)
} else {
line = fmt.Sprintf(" --%s", flag.Name)
}
varname, usage := UnquoteUsage(flag)
if len(varname) > 0 {
if varname != "" {
line += " " + varname
}
if len(flag.NoOptDefVal) > 0 {
if flag.NoOptDefVal != "" {
switch flag.Value.Type() {
case "string":
line += fmt.Sprintf("[=%q]", flag.NoOptDefVal)
line += fmt.Sprintf("[=\"%s\"]", flag.NoOptDefVal)
case "bool":
if flag.NoOptDefVal != "true" {
line += fmt.Sprintf("[=%s]", flag.NoOptDefVal)
@ -519,7 +673,7 @@ func (f *FlagSet) FlagUsages() string {
}
line += usage
if !isZeroValue(flag.DefValue) {
if !flag.defaultIsZeroValue() {
if flag.Value.Type() == "string" {
line += fmt.Sprintf(" (default %q)", flag.DefValue)
} else {
@ -533,10 +687,17 @@ func (f *FlagSet) FlagUsages() string {
for _, line := range lines {
sidx := strings.Index(line, "\x00")
spacing := strings.Repeat(" ", maxlen-sidx)
fmt.Fprintln(x, line[:sidx], spacing, line[sidx+1:])
// maxlen + 2 comes from + 1 for the \x00 and + 1 for the (deliberate) off-by-one in maxlen-sidx
fmt.Fprintln(buf, line[:sidx], spacing, wrap(maxlen+2, cols, line[sidx+1:]))
}
return x.String()
return buf.String()
}
// FlagUsages returns a string containing the usage information for all flags in
// the FlagSet
func (f *FlagSet) FlagUsages() string {
return f.FlagUsagesWrapped(0)
}
// PrintDefaults prints to standard error the default values of all defined command-line flags.
@ -622,16 +783,15 @@ func (f *FlagSet) VarPF(value Value, name, shorthand, usage string) *Flag {
// VarP is like Var, but accepts a shorthand letter that can be used after a single dash.
func (f *FlagSet) VarP(value Value, name, shorthand, usage string) {
_ = f.VarPF(value, name, shorthand, usage)
f.VarPF(value, name, shorthand, usage)
}
// AddFlag will add the flag to the FlagSet
func (f *FlagSet) AddFlag(flag *Flag) {
// Call normalizeFlagName function only once
normalizedFlagName := f.normalizeFlagName(flag.Name)
_, alreadythere := f.formal[normalizedFlagName]
if alreadythere {
_, alreadyThere := f.formal[normalizedFlagName]
if alreadyThere {
msg := fmt.Sprintf("%s flag redefined: %s", f.name, flag.Name)
fmt.Fprintln(f.out(), msg)
panic(msg) // Happens only if flags are declared with identical names
@ -642,28 +802,31 @@ func (f *FlagSet) AddFlag(flag *Flag) {
flag.Name = string(normalizedFlagName)
f.formal[normalizedFlagName] = flag
f.orderedFormal = append(f.orderedFormal, flag)
if len(flag.Shorthand) == 0 {
if flag.Shorthand == "" {
return
}
if len(flag.Shorthand) > 1 {
fmt.Fprintf(f.out(), "%s shorthand more than ASCII character: %s\n", f.name, flag.Shorthand)
panic("shorthand is more than one character")
msg := fmt.Sprintf("%q shorthand is more than one ASCII character", flag.Shorthand)
fmt.Fprintf(f.out(), msg)
panic(msg)
}
if f.shorthands == nil {
f.shorthands = make(map[byte]*Flag)
}
c := flag.Shorthand[0]
old, alreadythere := f.shorthands[c]
if alreadythere {
fmt.Fprintf(f.out(), "%s shorthand reused: %q for %s already used for %s\n", f.name, c, flag.Name, old.Name)
panic("shorthand redefinition")
used, alreadyThere := f.shorthands[c]
if alreadyThere {
msg := fmt.Sprintf("unable to redefine %q shorthand in %q flagset: it's already used for %q flag", c, f.name, used.Name)
fmt.Fprintf(f.out(), msg)
panic(msg)
}
f.shorthands[c] = flag
}
// AddFlagSet adds one FlagSet to another. If a flag is already present in f
// the flag from newSet will be ignored
// the flag from newSet will be ignored.
func (f *FlagSet) AddFlagSet(newSet *FlagSet) {
if newSet == nil {
return
@ -711,45 +874,18 @@ func (f *FlagSet) usage() {
}
}
func (f *FlagSet) setFlag(flag *Flag, value string, origArg string) error {
if err := flag.Value.Set(value); err != nil {
return f.failf("invalid argument %q for %s: %v", value, origArg, err)
}
// mark as visited for Visit()
if f.actual == nil {
f.actual = make(map[NormalizedName]*Flag)
}
f.actual[f.normalizeFlagName(flag.Name)] = flag
flag.Changed = true
if len(flag.Deprecated) > 0 {
fmt.Fprintf(os.Stderr, "Flag --%s has been deprecated, %s\n", flag.Name, flag.Deprecated)
}
if len(flag.ShorthandDeprecated) > 0 && containsShorthand(origArg, flag.Shorthand) {
fmt.Fprintf(os.Stderr, "Flag shorthand -%s has been deprecated, %s\n", flag.Shorthand, flag.ShorthandDeprecated)
}
return nil
}
func containsShorthand(arg, shorthand string) bool {
// filter out flags --<flag_name>
if strings.HasPrefix(arg, "-") {
return false
}
arg = strings.SplitN(arg, "=", 2)[0]
return strings.Contains(arg, shorthand)
}
func (f *FlagSet) parseLongArg(s string, args []string) (a []string, err error) {
func (f *FlagSet) parseLongArg(s string, args []string, fn parseFunc) (a []string, err error) {
a = args
name := s[2:]
if len(name) == 0 || name[0] == '-' || name[0] == '=' {
err = f.failf("bad flag syntax: %s", s)
return
}
split := strings.SplitN(name, "=", 2)
name = split[0]
flag, alreadythere := f.formal[f.normalizeFlagName(name)]
if !alreadythere {
flag, exists := f.formal[f.normalizeFlagName(name)]
if !exists {
if name == "help" { // special case for nice help message.
f.usage()
return a, ErrHelp
@ -757,11 +893,12 @@ func (f *FlagSet) parseLongArg(s string, args []string) (a []string, err error)
err = f.failf("unknown flag: --%s", name)
return
}
var value string
if len(split) == 2 {
// '--flag=arg'
value = split[1]
} else if len(flag.NoOptDefVal) > 0 {
} else if flag.NoOptDefVal != "" {
// '--flag' (arg was optional)
value = flag.NoOptDefVal
} else if len(a) > 0 {
@ -773,55 +910,68 @@ func (f *FlagSet) parseLongArg(s string, args []string) (a []string, err error)
err = f.failf("flag needs an argument: %s", s)
return
}
err = f.setFlag(flag, value, s)
err = fn(flag, value)
return
}
func (f *FlagSet) parseSingleShortArg(shorthands string, args []string) (outShorts string, outArgs []string, err error) {
func (f *FlagSet) parseSingleShortArg(shorthands string, args []string, fn parseFunc) (outShorts string, outArgs []string, err error) {
if strings.HasPrefix(shorthands, "test.") {
return
}
outArgs = args
outShorts = shorthands[1:]
c := shorthands[0]
flag, alreadythere := f.shorthands[c]
if !alreadythere {
flag, exists := f.shorthands[c]
if !exists {
if c == 'h' { // special case for nice help message.
f.usage()
err = ErrHelp
return
}
//TODO continue on error
err = f.failf("unknown shorthand flag: %q in -%s", c, shorthands)
return
}
var value string
if len(shorthands) > 2 && shorthands[1] == '=' {
// '-f=arg'
value = shorthands[2:]
outShorts = ""
} else if len(flag.NoOptDefVal) > 0 {
} else if flag.NoOptDefVal != "" {
// '-f' (arg was optional)
value = flag.NoOptDefVal
} else if len(shorthands) > 1 {
// '-farg'
value = shorthands[1:]
outShorts = ""
} else if len(args) > 0 {
// '-f arg'
value = args[0]
outArgs = args[1:]
} else {
// '-f' (arg was required)
err = f.failf("flag needs an argument: %q in -%s", c, shorthands)
return
}
err = f.setFlag(flag, value, shorthands)
if flag.ShorthandDeprecated != "" {
fmt.Fprintf(f.out(), "Flag shorthand -%s has been deprecated, %s\n", flag.Shorthand, flag.ShorthandDeprecated)
}
err = fn(flag, value)
return
}
func (f *FlagSet) parseShortArg(s string, args []string) (a []string, err error) {
func (f *FlagSet) parseShortArg(s string, args []string, fn parseFunc) (a []string, err error) {
a = args
shorthands := s[1:]
// "shorthands" can be a series of shorthand letters of flags (e.g. "-vvv").
for len(shorthands) > 0 {
shorthands, a, err = f.parseSingleShortArg(shorthands, args)
shorthands, a, err = f.parseSingleShortArg(shorthands, args, fn)
if err != nil {
return
}
@ -830,7 +980,7 @@ func (f *FlagSet) parseShortArg(s string, args []string) (a []string, err error)
return
}
func (f *FlagSet) parseArgs(args []string) (err error) {
func (f *FlagSet) parseArgs(args []string, fn parseFunc) (err error) {
for len(args) > 0 {
s := args[0]
args = args[1:]
@ -850,9 +1000,9 @@ func (f *FlagSet) parseArgs(args []string) (err error) {
f.args = append(f.args, args...)
break
}
args, err = f.parseLongArg(s, args)
args, err = f.parseLongArg(s, args, fn)
} else {
args, err = f.parseShortArg(s, args)
args, err = f.parseShortArg(s, args, fn)
}
if err != nil {
return
@ -867,8 +1017,43 @@ func (f *FlagSet) parseArgs(args []string) (err error) {
// The return value will be ErrHelp if -help was set but not defined.
func (f *FlagSet) Parse(arguments []string) error {
f.parsed = true
if len(arguments) < 0 {
return nil
}
f.args = make([]string, 0, len(arguments))
err := f.parseArgs(arguments)
set := func(flag *Flag, value string) error {
return f.Set(flag.Name, value)
}
err := f.parseArgs(arguments, set)
if err != nil {
switch f.errorHandling {
case ContinueOnError:
return err
case ExitOnError:
os.Exit(2)
case PanicOnError:
panic(err)
}
}
return nil
}
type parseFunc func(flag *Flag, value string) error
// ParseAll parses flag definitions from the argument list, which should not
// include the command name. The arguments for fn are flag and value. Must be
// called after all flags in the FlagSet are defined and before flags are
// accessed by the program. The return value will be ErrHelp if -help was set
// but not defined.
func (f *FlagSet) ParseAll(arguments []string, fn func(flag *Flag, value string) error) error {
f.parsed = true
f.args = make([]string, 0, len(arguments))
err := f.parseArgs(arguments, fn)
if err != nil {
switch f.errorHandling {
case ContinueOnError:
@ -894,6 +1079,14 @@ func Parse() {
CommandLine.Parse(os.Args[1:])
}
// ParseAll parses the command-line flags from os.Args[1:] and called fn for each.
// The arguments for fn are flag and value. Must be called after all flags are
// defined and before flags are accessed by the program.
func ParseAll(fn func(flag *Flag, value string) error) {
// Ignore errors; CommandLine is set for ExitOnError.
CommandLine.ParseAll(os.Args[1:], fn)
}
// SetInterspersed sets whether to support interspersed option/non-option arguments.
func SetInterspersed(interspersed bool) {
CommandLine.SetInterspersed(interspersed)
@ -907,14 +1100,15 @@ func Parsed() bool {
// CommandLine is the default set of command-line flags, parsed from os.Args.
var CommandLine = NewFlagSet(os.Args[0], ExitOnError)
// NewFlagSet returns a new, empty flag set with the specified name and
// error handling property.
// NewFlagSet returns a new, empty flag set with the specified name,
// error handling property and SortFlags set to true.
func NewFlagSet(name string, errorHandling ErrorHandling) *FlagSet {
f := &FlagSet{
name: name,
errorHandling: errorHandling,
argsLenAtDash: -1,
interspersed: true,
SortFlags: true,
}
return f
}

View file

@ -1,9 +1,6 @@
package pflag
import (
"fmt"
"strconv"
)
import "strconv"
// -- float32 Value
type float32Value float32
@ -23,7 +20,7 @@ func (f *float32Value) Type() string {
return "float32"
}
func (f *float32Value) String() string { return fmt.Sprintf("%v", *f) }
func (f *float32Value) String() string { return strconv.FormatFloat(float64(*f), 'g', -1, 32) }
func float32Conv(sval string) (interface{}, error) {
v, err := strconv.ParseFloat(sval, 32)

View file

@ -1,9 +1,6 @@
package pflag
import (
"fmt"
"strconv"
)
import "strconv"
// -- float64 Value
type float64Value float64
@ -23,7 +20,7 @@ func (f *float64Value) Type() string {
return "float64"
}
func (f *float64Value) String() string { return fmt.Sprintf("%v", *f) }
func (f *float64Value) String() string { return strconv.FormatFloat(float64(*f), 'g', -1, 64) }
func float64Conv(sval string) (interface{}, error) {
return strconv.ParseFloat(sval, 64)

View file

@ -6,13 +6,10 @@ package pflag
import (
goflag "flag"
"fmt"
"reflect"
"strings"
)
var _ = fmt.Print
// flagValueWrapper implements pflag.Value around a flag.Value. The main
// difference here is the addition of the Type method that returns a string
// name of the type. As this is generally unknown, we approximate that with

View file

@ -1,9 +1,6 @@
package pflag
import (
"fmt"
"strconv"
)
import "strconv"
// -- int Value
type intValue int
@ -23,7 +20,7 @@ func (i *intValue) Type() string {
return "int"
}
func (i *intValue) String() string { return fmt.Sprintf("%v", *i) }
func (i *intValue) String() string { return strconv.Itoa(int(*i)) }
func intConv(sval string) (interface{}, error) {
return strconv.Atoi(sval)

View file

@ -1,9 +1,6 @@
package pflag
import (
"fmt"
"strconv"
)
import "strconv"
// -- int32 Value
type int32Value int32
@ -23,7 +20,7 @@ func (i *int32Value) Type() string {
return "int32"
}
func (i *int32Value) String() string { return fmt.Sprintf("%v", *i) }
func (i *int32Value) String() string { return strconv.FormatInt(int64(*i), 10) }
func int32Conv(sval string) (interface{}, error) {
v, err := strconv.ParseInt(sval, 0, 32)

View file

@ -1,9 +1,6 @@
package pflag
import (
"fmt"
"strconv"
)
import "strconv"
// -- int64 Value
type int64Value int64
@ -23,7 +20,7 @@ func (i *int64Value) Type() string {
return "int64"
}
func (i *int64Value) String() string { return fmt.Sprintf("%v", *i) }
func (i *int64Value) String() string { return strconv.FormatInt(int64(*i), 10) }
func int64Conv(sval string) (interface{}, error) {
return strconv.ParseInt(sval, 0, 64)

View file

@ -1,9 +1,6 @@
package pflag
import (
"fmt"
"strconv"
)
import "strconv"
// -- int8 Value
type int8Value int8
@ -23,7 +20,7 @@ func (i *int8Value) Type() string {
return "int8"
}
func (i *int8Value) String() string { return fmt.Sprintf("%v", *i) }
func (i *int8Value) String() string { return strconv.FormatInt(int64(*i), 10) }
func int8Conv(sval string) (interface{}, error) {
v, err := strconv.ParseInt(sval, 0, 8)

View file

@ -6,8 +6,6 @@ import (
"strings"
)
var _ = strings.TrimSpace
// -- net.IP value
type ipValue net.IP

148
vendor/github.com/spf13/pflag/ip_slice.go generated vendored Normal file
View file

@ -0,0 +1,148 @@
package pflag
import (
"fmt"
"io"
"net"
"strings"
)
// -- ipSlice Value
type ipSliceValue struct {
value *[]net.IP
changed bool
}
func newIPSliceValue(val []net.IP, p *[]net.IP) *ipSliceValue {
ipsv := new(ipSliceValue)
ipsv.value = p
*ipsv.value = val
return ipsv
}
// Set converts, and assigns, the comma-separated IP argument string representation as the []net.IP value of this flag.
// If Set is called on a flag that already has a []net.IP assigned, the newly converted values will be appended.
func (s *ipSliceValue) Set(val string) error {
// remove all quote characters
rmQuote := strings.NewReplacer(`"`, "", `'`, "", "`", "")
// read flag arguments with CSV parser
ipStrSlice, err := readAsCSV(rmQuote.Replace(val))
if err != nil && err != io.EOF {
return err
}
// parse ip values into slice
out := make([]net.IP, 0, len(ipStrSlice))
for _, ipStr := range ipStrSlice {
ip := net.ParseIP(strings.TrimSpace(ipStr))
if ip == nil {
return fmt.Errorf("invalid string being converted to IP address: %s", ipStr)
}
out = append(out, ip)
}
if !s.changed {
*s.value = out
} else {
*s.value = append(*s.value, out...)
}
s.changed = true
return nil
}
// Type returns a string that uniquely represents this flag's type.
func (s *ipSliceValue) Type() string {
return "ipSlice"
}
// String defines a "native" format for this net.IP slice flag value.
func (s *ipSliceValue) String() string {
ipStrSlice := make([]string, len(*s.value))
for i, ip := range *s.value {
ipStrSlice[i] = ip.String()
}
out, _ := writeAsCSV(ipStrSlice)
return "[" + out + "]"
}
func ipSliceConv(val string) (interface{}, error) {
val = strings.Trim(val, "[]")
// Emtpy string would cause a slice with one (empty) entry
if len(val) == 0 {
return []net.IP{}, nil
}
ss := strings.Split(val, ",")
out := make([]net.IP, len(ss))
for i, sval := range ss {
ip := net.ParseIP(strings.TrimSpace(sval))
if ip == nil {
return nil, fmt.Errorf("invalid string being converted to IP address: %s", sval)
}
out[i] = ip
}
return out, nil
}
// GetIPSlice returns the []net.IP value of a flag with the given name
func (f *FlagSet) GetIPSlice(name string) ([]net.IP, error) {
val, err := f.getFlagType(name, "ipSlice", ipSliceConv)
if err != nil {
return []net.IP{}, err
}
return val.([]net.IP), nil
}
// IPSliceVar defines a ipSlice flag with specified name, default value, and usage string.
// The argument p points to a []net.IP variable in which to store the value of the flag.
func (f *FlagSet) IPSliceVar(p *[]net.IP, name string, value []net.IP, usage string) {
f.VarP(newIPSliceValue(value, p), name, "", usage)
}
// IPSliceVarP is like IPSliceVar, but accepts a shorthand letter that can be used after a single dash.
func (f *FlagSet) IPSliceVarP(p *[]net.IP, name, shorthand string, value []net.IP, usage string) {
f.VarP(newIPSliceValue(value, p), name, shorthand, usage)
}
// IPSliceVar defines a []net.IP flag with specified name, default value, and usage string.
// The argument p points to a []net.IP variable in which to store the value of the flag.
func IPSliceVar(p *[]net.IP, name string, value []net.IP, usage string) {
CommandLine.VarP(newIPSliceValue(value, p), name, "", usage)
}
// IPSliceVarP is like IPSliceVar, but accepts a shorthand letter that can be used after a single dash.
func IPSliceVarP(p *[]net.IP, name, shorthand string, value []net.IP, usage string) {
CommandLine.VarP(newIPSliceValue(value, p), name, shorthand, usage)
}
// IPSlice defines a []net.IP flag with specified name, default value, and usage string.
// The return value is the address of a []net.IP variable that stores the value of that flag.
func (f *FlagSet) IPSlice(name string, value []net.IP, usage string) *[]net.IP {
p := []net.IP{}
f.IPSliceVarP(&p, name, "", value, usage)
return &p
}
// IPSliceP is like IPSlice, but accepts a shorthand letter that can be used after a single dash.
func (f *FlagSet) IPSliceP(name, shorthand string, value []net.IP, usage string) *[]net.IP {
p := []net.IP{}
f.IPSliceVarP(&p, name, shorthand, value, usage)
return &p
}
// IPSlice defines a []net.IP flag with specified name, default value, and usage string.
// The return value is the address of a []net.IP variable that stores the value of the flag.
func IPSlice(name string, value []net.IP, usage string) *[]net.IP {
return CommandLine.IPSliceP(name, "", value, usage)
}
// IPSliceP is like IPSlice, but accepts a shorthand letter that can be used after a single dash.
func IPSliceP(name, shorthand string, value []net.IP, usage string) *[]net.IP {
return CommandLine.IPSliceP(name, shorthand, value, usage)
}

View file

@ -27,8 +27,6 @@ func (*ipNetValue) Type() string {
return "ipNet"
}
var _ = strings.TrimSpace
func newIPNetValue(val net.IPNet, p *net.IPNet) *ipNetValue {
*p = val
return (*ipNetValue)(p)

View file

@ -1,7 +1,5 @@
package pflag
import "fmt"
// -- string Value
type stringValue string
@ -18,7 +16,7 @@ func (s *stringValue) Type() string {
return "string"
}
func (s *stringValue) String() string { return fmt.Sprintf("%s", *s) }
func (s *stringValue) String() string { return string(*s) }
func stringConv(sval string) (interface{}, error) {
return sval, nil

103
vendor/github.com/spf13/pflag/string_array.go generated vendored Normal file
View file

@ -0,0 +1,103 @@
package pflag
// -- stringArray Value
type stringArrayValue struct {
value *[]string
changed bool
}
func newStringArrayValue(val []string, p *[]string) *stringArrayValue {
ssv := new(stringArrayValue)
ssv.value = p
*ssv.value = val
return ssv
}
func (s *stringArrayValue) Set(val string) error {
if !s.changed {
*s.value = []string{val}
s.changed = true
} else {
*s.value = append(*s.value, val)
}
return nil
}
func (s *stringArrayValue) Type() string {
return "stringArray"
}
func (s *stringArrayValue) String() string {
str, _ := writeAsCSV(*s.value)
return "[" + str + "]"
}
func stringArrayConv(sval string) (interface{}, error) {
sval = sval[1 : len(sval)-1]
// An empty string would cause a array with one (empty) string
if len(sval) == 0 {
return []string{}, nil
}
return readAsCSV(sval)
}
// GetStringArray return the []string value of a flag with the given name
func (f *FlagSet) GetStringArray(name string) ([]string, error) {
val, err := f.getFlagType(name, "stringArray", stringArrayConv)
if err != nil {
return []string{}, err
}
return val.([]string), nil
}
// StringArrayVar defines a string flag with specified name, default value, and usage string.
// The argument p points to a []string variable in which to store the values of the multiple flags.
// The value of each argument will not try to be separated by comma
func (f *FlagSet) StringArrayVar(p *[]string, name string, value []string, usage string) {
f.VarP(newStringArrayValue(value, p), name, "", usage)
}
// StringArrayVarP is like StringArrayVar, but accepts a shorthand letter that can be used after a single dash.
func (f *FlagSet) StringArrayVarP(p *[]string, name, shorthand string, value []string, usage string) {
f.VarP(newStringArrayValue(value, p), name, shorthand, usage)
}
// StringArrayVar defines a string flag with specified name, default value, and usage string.
// The argument p points to a []string variable in which to store the value of the flag.
// The value of each argument will not try to be separated by comma
func StringArrayVar(p *[]string, name string, value []string, usage string) {
CommandLine.VarP(newStringArrayValue(value, p), name, "", usage)
}
// StringArrayVarP is like StringArrayVar, but accepts a shorthand letter that can be used after a single dash.
func StringArrayVarP(p *[]string, name, shorthand string, value []string, usage string) {
CommandLine.VarP(newStringArrayValue(value, p), name, shorthand, usage)
}
// StringArray defines a string flag with specified name, default value, and usage string.
// The return value is the address of a []string variable that stores the value of the flag.
// The value of each argument will not try to be separated by comma
func (f *FlagSet) StringArray(name string, value []string, usage string) *[]string {
p := []string{}
f.StringArrayVarP(&p, name, "", value, usage)
return &p
}
// StringArrayP is like StringArray, but accepts a shorthand letter that can be used after a single dash.
func (f *FlagSet) StringArrayP(name, shorthand string, value []string, usage string) *[]string {
p := []string{}
f.StringArrayVarP(&p, name, shorthand, value, usage)
return &p
}
// StringArray defines a string flag with specified name, default value, and usage string.
// The return value is the address of a []string variable that stores the value of the flag.
// The value of each argument will not try to be separated by comma
func StringArray(name string, value []string, usage string) *[]string {
return CommandLine.StringArrayP(name, "", value, usage)
}
// StringArrayP is like StringArray, but accepts a shorthand letter that can be used after a single dash.
func StringArrayP(name, shorthand string, value []string, usage string) *[]string {
return CommandLine.StringArrayP(name, shorthand, value, usage)
}

View file

@ -1,13 +1,11 @@
package pflag
import (
"bytes"
"encoding/csv"
"fmt"
"strings"
)
var _ = fmt.Fprint
// -- stringSlice Value
type stringSliceValue struct {
value *[]string
@ -21,10 +19,28 @@ func newStringSliceValue(val []string, p *[]string) *stringSliceValue {
return ssv
}
func (s *stringSliceValue) Set(val string) error {
func readAsCSV(val string) ([]string, error) {
if val == "" {
return []string{}, nil
}
stringReader := strings.NewReader(val)
csvReader := csv.NewReader(stringReader)
v, err := csvReader.Read()
return csvReader.Read()
}
func writeAsCSV(vals []string) (string, error) {
b := &bytes.Buffer{}
w := csv.NewWriter(b)
err := w.Write(vals)
if err != nil {
return "", err
}
w.Flush()
return strings.TrimSuffix(b.String(), "\n"), nil
}
func (s *stringSliceValue) Set(val string) error {
v, err := readAsCSV(val)
if err != nil {
return err
}
@ -41,16 +57,18 @@ func (s *stringSliceValue) Type() string {
return "stringSlice"
}
func (s *stringSliceValue) String() string { return "[" + strings.Join(*s.value, ",") + "]" }
func (s *stringSliceValue) String() string {
str, _ := writeAsCSV(*s.value)
return "[" + str + "]"
}
func stringSliceConv(sval string) (interface{}, error) {
sval = strings.Trim(sval, "[]")
sval = sval[1 : len(sval)-1]
// An empty string would cause a slice with one (empty) string
if len(sval) == 0 {
return []string{}, nil
}
v := strings.Split(sval, ",")
return v, nil
return readAsCSV(sval)
}
// GetStringSlice return the []string value of a flag with the given name

View file

@ -1,9 +1,6 @@
package pflag
import (
"fmt"
"strconv"
)
import "strconv"
// -- uint Value
type uintValue uint
@ -23,7 +20,7 @@ func (i *uintValue) Type() string {
return "uint"
}
func (i *uintValue) String() string { return fmt.Sprintf("%v", *i) }
func (i *uintValue) String() string { return strconv.FormatUint(uint64(*i), 10) }
func uintConv(sval string) (interface{}, error) {
v, err := strconv.ParseUint(sval, 0, 0)

View file

@ -1,9 +1,6 @@
package pflag
import (
"fmt"
"strconv"
)
import "strconv"
// -- uint16 value
type uint16Value uint16
@ -12,7 +9,7 @@ func newUint16Value(val uint16, p *uint16) *uint16Value {
*p = val
return (*uint16Value)(p)
}
func (i *uint16Value) String() string { return fmt.Sprintf("%d", *i) }
func (i *uint16Value) Set(s string) error {
v, err := strconv.ParseUint(s, 0, 16)
*i = uint16Value(v)
@ -23,6 +20,8 @@ func (i *uint16Value) Type() string {
return "uint16"
}
func (i *uint16Value) String() string { return strconv.FormatUint(uint64(*i), 10) }
func uint16Conv(sval string) (interface{}, error) {
v, err := strconv.ParseUint(sval, 0, 16)
if err != nil {

View file

@ -1,18 +1,15 @@
package pflag
import (
"fmt"
"strconv"
)
import "strconv"
// -- uint16 value
// -- uint32 value
type uint32Value uint32
func newUint32Value(val uint32, p *uint32) *uint32Value {
*p = val
return (*uint32Value)(p)
}
func (i *uint32Value) String() string { return fmt.Sprintf("%d", *i) }
func (i *uint32Value) Set(s string) error {
v, err := strconv.ParseUint(s, 0, 32)
*i = uint32Value(v)
@ -23,6 +20,8 @@ func (i *uint32Value) Type() string {
return "uint32"
}
func (i *uint32Value) String() string { return strconv.FormatUint(uint64(*i), 10) }
func uint32Conv(sval string) (interface{}, error) {
v, err := strconv.ParseUint(sval, 0, 32)
if err != nil {

View file

@ -1,9 +1,6 @@
package pflag
import (
"fmt"
"strconv"
)
import "strconv"
// -- uint64 Value
type uint64Value uint64
@ -23,7 +20,7 @@ func (i *uint64Value) Type() string {
return "uint64"
}
func (i *uint64Value) String() string { return fmt.Sprintf("%v", *i) }
func (i *uint64Value) String() string { return strconv.FormatUint(uint64(*i), 10) }
func uint64Conv(sval string) (interface{}, error) {
v, err := strconv.ParseUint(sval, 0, 64)

View file

@ -1,9 +1,6 @@
package pflag
import (
"fmt"
"strconv"
)
import "strconv"
// -- uint8 Value
type uint8Value uint8
@ -23,7 +20,7 @@ func (i *uint8Value) Type() string {
return "uint8"
}
func (i *uint8Value) String() string { return fmt.Sprintf("%v", *i) }
func (i *uint8Value) String() string { return strconv.FormatUint(uint64(*i), 10) }
func uint8Conv(sval string) (interface{}, error) {
v, err := strconv.ParseUint(sval, 0, 8)

126
vendor/github.com/spf13/pflag/uint_slice.go generated vendored Normal file
View file

@ -0,0 +1,126 @@
package pflag
import (
"fmt"
"strconv"
"strings"
)
// -- uintSlice Value
type uintSliceValue struct {
value *[]uint
changed bool
}
func newUintSliceValue(val []uint, p *[]uint) *uintSliceValue {
uisv := new(uintSliceValue)
uisv.value = p
*uisv.value = val
return uisv
}
func (s *uintSliceValue) Set(val string) error {
ss := strings.Split(val, ",")
out := make([]uint, len(ss))
for i, d := range ss {
u, err := strconv.ParseUint(d, 10, 0)
if err != nil {
return err
}
out[i] = uint(u)
}
if !s.changed {
*s.value = out
} else {
*s.value = append(*s.value, out...)
}
s.changed = true
return nil
}
func (s *uintSliceValue) Type() string {
return "uintSlice"
}
func (s *uintSliceValue) String() string {
out := make([]string, len(*s.value))
for i, d := range *s.value {
out[i] = fmt.Sprintf("%d", d)
}
return "[" + strings.Join(out, ",") + "]"
}
func uintSliceConv(val string) (interface{}, error) {
val = strings.Trim(val, "[]")
// Empty string would cause a slice with one (empty) entry
if len(val) == 0 {
return []uint{}, nil
}
ss := strings.Split(val, ",")
out := make([]uint, len(ss))
for i, d := range ss {
u, err := strconv.ParseUint(d, 10, 0)
if err != nil {
return nil, err
}
out[i] = uint(u)
}
return out, nil
}
// GetUintSlice returns the []uint value of a flag with the given name.
func (f *FlagSet) GetUintSlice(name string) ([]uint, error) {
val, err := f.getFlagType(name, "uintSlice", uintSliceConv)
if err != nil {
return []uint{}, err
}
return val.([]uint), nil
}
// UintSliceVar defines a uintSlice flag with specified name, default value, and usage string.
// The argument p points to a []uint variable in which to store the value of the flag.
func (f *FlagSet) UintSliceVar(p *[]uint, name string, value []uint, usage string) {
f.VarP(newUintSliceValue(value, p), name, "", usage)
}
// UintSliceVarP is like UintSliceVar, but accepts a shorthand letter that can be used after a single dash.
func (f *FlagSet) UintSliceVarP(p *[]uint, name, shorthand string, value []uint, usage string) {
f.VarP(newUintSliceValue(value, p), name, shorthand, usage)
}
// UintSliceVar defines a uint[] flag with specified name, default value, and usage string.
// The argument p points to a uint[] variable in which to store the value of the flag.
func UintSliceVar(p *[]uint, name string, value []uint, usage string) {
CommandLine.VarP(newUintSliceValue(value, p), name, "", usage)
}
// UintSliceVarP is like the UintSliceVar, but accepts a shorthand letter that can be used after a single dash.
func UintSliceVarP(p *[]uint, name, shorthand string, value []uint, usage string) {
CommandLine.VarP(newUintSliceValue(value, p), name, shorthand, usage)
}
// UintSlice defines a []uint flag with specified name, default value, and usage string.
// The return value is the address of a []uint variable that stores the value of the flag.
func (f *FlagSet) UintSlice(name string, value []uint, usage string) *[]uint {
p := []uint{}
f.UintSliceVarP(&p, name, "", value, usage)
return &p
}
// UintSliceP is like UintSlice, but accepts a shorthand letter that can be used after a single dash.
func (f *FlagSet) UintSliceP(name, shorthand string, value []uint, usage string) *[]uint {
p := []uint{}
f.UintSliceVarP(&p, name, shorthand, value, usage)
return &p
}
// UintSlice defines a []uint flag with specified name, default value, and usage string.
// The return value is the address of a []uint variable that stores the value of the flag.
func UintSlice(name string, value []uint, usage string) *[]uint {
return CommandLine.UintSliceP(name, "", value, usage)
}
// UintSliceP is like UintSlice, but accepts a shorthand letter that can be used after a single dash.
func UintSliceP(name, shorthand string, value []uint, usage string) *[]uint {
return CommandLine.UintSliceP(name, shorthand, value, usage)
}

43
vendor/vendor.json vendored
View file

@ -58,12 +58,6 @@
"revision": "c589d0c9f0d81640c518354c7bcae77d99820aa3",
"revisionTime": "2016-09-30T00:14:02Z"
},
{
"checksumSHA1": "DYv6Q1+VfnUVxMwvk5IshAClLvw=",
"path": "github.com/Sirupsen/logrus",
"revision": "5e5dc898656f695e2a086b8e12559febbfc01562",
"revisionTime": "2017-05-15T10:45:16Z"
},
{
"checksumSHA1": "ddYc7mKe3g1x1UUKBrGR4vArJs8=",
"path": "github.com/asaskevich/govalidator",
@ -567,6 +561,12 @@
"revision": "1d4fa605f6ff3ed628d7ae5eda7c0e56803e72a5",
"revisionTime": "2016-10-07T00:41:22Z"
},
{
"checksumSHA1": "40vJyUB4ezQSn/NSadsKEOrudMc=",
"path": "github.com/inconshreveable/mousetrap",
"revision": "76626ae9c91c4f2a10f34cad8ce83ea42c93bb75",
"revisionTime": "2014-10-17T20:07:13Z"
},
{
"path": "github.com/influxdata/influxdb/client/v2",
"revision": "15e594fc09f112cb696c084a20beaca25538a5fa",
@ -721,16 +721,16 @@
"revisionTime": "2017-04-18T15:52:10Z"
},
{
"checksumSHA1": "vfkLfDs6hXtxdJNGdWQglsxFu40=",
"checksumSHA1": "esljoY35OOHttvpjBctA6WGZc70=",
"path": "github.com/prometheus/common/log",
"revision": "75882ff176da1a10e2705adadd81edfd3a50d743",
"revisionTime": "2017-04-27T09:48:40Z"
"revision": "94eb3fe3a55f1b5d12d335b27a4f6e37cd73ed7e",
"revisionTime": "2017-06-16T09:39:39Z"
},
{
"checksumSHA1": "0LL9u9tfv1KPBjNEiMDP6q7lpog=",
"checksumSHA1": "3VoqH7TFfzA6Ds0zFzIbKCUvBmw=",
"path": "github.com/prometheus/common/model",
"revision": "9e0844febd9e2856f839c9cb974fbd676d1755a8",
"revisionTime": "2017-04-18T15:52:10Z"
"revision": "0d0c3d572886e0f2323ed376557f9eb99b97d25b",
"revisionTime": "2017-06-16T14:41:04Z"
},
{
"checksumSHA1": "9aDxDuzZt1l7FQJ9qpn2kPcF7NU=",
@ -775,11 +775,22 @@
"revisionTime": "2016-10-28T23:23:40Z"
},
{
"checksumSHA1": "YuPBOVkkE3uuBh4RcRUTF0n+frs=",
"origin": "k8s.io/client-go/1.5/vendor/github.com/spf13/pflag",
"checksumSHA1": "VHULZdEZKBxJvlZnlhX7Ug19lgY=",
"path": "github.com/sirupsen/logrus",
"revision": "85b1699d505667d13f8ac4478c1debbf85d6c5de",
"revisionTime": "2017-06-08T22:14:41Z"
},
{
"checksumSHA1": "suXMIWAx0XtlQR2zippibpr3Yjg=",
"path": "github.com/spf13/cobra",
"revision": "b4dbd37a01839e0653eec12aa4bbb2a2ce7b2a37",
"revisionTime": "2017-06-12T06:36:10Z"
},
{
"checksumSHA1": "STxYqRb4gnlSr3mRpT+Igfdz/kM=",
"path": "github.com/spf13/pflag",
"revision": "c589d0c9f0d81640c518354c7bcae77d99820aa3",
"revisionTime": "2016-09-30T00:14:02Z"
"revision": "e57e3eeb33f795204c1ca35f56c44f83227c6e66",
"revisionTime": "2017-05-08T18:43:26Z"
},
{
"checksumSHA1": "iydUphwYqZRq3WhstEdGsbvBAKs=",

View file

@ -55,7 +55,7 @@ func (h *Handler) federation(w http.ResponseWriter, req *http.Request) {
}
var (
mint = timestamp.FromTime(h.now().Time().Add(-promql.StalenessDelta))
mint = timestamp.FromTime(h.now().Time().Add(-promql.LookbackDelta))
maxt = timestamp.FromTime(h.now().Time())
format = expfmt.Negotiate(req.Header)
enc = expfmt.NewEncoder(w, format)
@ -86,7 +86,7 @@ func (h *Handler) federation(w http.ResponseWriter, req *http.Request) {
// TODO(fabxc): allow fast path for most recent sample either
// in the storage itself or caching layer in Prometheus.
it := storage.NewBuffer(s.Iterator(), int64(promql.StalenessDelta/1e6))
it := storage.NewBuffer(s.Iterator(), int64(promql.LookbackDelta/1e6))
var t int64
var v float64

View file

@ -121,7 +121,6 @@ type Options struct {
MaxConnections int
ExternalURL *url.URL
RoutePrefix string
MetricsPath string
UseLocalAssets bool
UserAssetsPath string
ConsoleTemplatesPath string
@ -186,7 +185,7 @@ func New(o *Options) *Handler {
router.Get("/heap", instrf("heap", dumpHeap))
router.Get(o.MetricsPath, prometheus.Handler().ServeHTTP)
router.Get("/metrics", prometheus.Handler().ServeHTTP)
router.Get("/federate", instrh("federate", httputil.CompressionHandler{
Handler: http.HandlerFunc(h.federation),