github.com/Jeffail/benthos/v3@v3.65.0/lib/input/dynamic.go (about)

     1  package input
     2  
     3  import (
     4  	"encoding/json"
     5  	"fmt"
     6  	"path"
     7  	"sync"
     8  	"time"
     9  
    10  	"github.com/Jeffail/benthos/v3/internal/docs"
    11  	"github.com/Jeffail/benthos/v3/internal/interop"
    12  	"github.com/Jeffail/benthos/v3/lib/api"
    13  	"github.com/Jeffail/benthos/v3/lib/broker"
    14  	"github.com/Jeffail/benthos/v3/lib/log"
    15  	"github.com/Jeffail/benthos/v3/lib/metrics"
    16  	"github.com/Jeffail/benthos/v3/lib/types"
    17  	"gopkg.in/yaml.v3"
    18  )
    19  
    20  //------------------------------------------------------------------------------
    21  
    22  func init() {
    23  	Constructors[TypeDynamic] = TypeSpec{
    24  		constructor: func(
    25  			hasBatchProc bool,
    26  			conf Config,
    27  			mgr types.Manager,
    28  			log log.Modular,
    29  			stats metrics.Type,
    30  			pipelines ...types.PipelineConstructorFunc,
    31  		) (Type, error) {
    32  			_, pipelines = appendProcessorsFromConfigBatchAware(hasBatchProc, conf, mgr, log, stats, pipelines...)
    33  			return NewDynamic(conf, mgr, log, stats, pipelines...)
    34  		},
    35  		Summary: `
    36  A special broker type where the inputs are identified by unique labels and can
    37  be created, changed and removed during runtime via a REST HTTP interface.`,
    38  		Description: `
    39  To GET a JSON map of input identifiers with their current uptimes use the
    40  ` + "`/inputs`" + ` endpoint.
    41  
    42  To perform CRUD actions on the inputs themselves use POST, DELETE, and GET
    43  methods on the ` + "`/inputs/{input_id}`" + ` endpoint. When using POST the body
    44  of the request should be a YAML configuration for the input, if the input
    45  already exists it will be changed.`,
    46  		Categories: []Category{
    47  			CategoryUtility,
    48  		},
    49  		FieldSpecs: docs.FieldSpecs{
    50  			docs.FieldCommon("inputs", "A map of inputs to statically create.").Map().HasType(docs.FieldTypeInput),
    51  			docs.FieldCommon("prefix", "A path prefix for HTTP endpoints that are registered."),
    52  			docs.FieldCommon("timeout", "The server side timeout of HTTP requests."),
    53  		},
    54  	}
    55  }
    56  
    57  //------------------------------------------------------------------------------
    58  
    59  // DynamicConfig contains configuration for the Dynamic input type.
    60  type DynamicConfig struct {
    61  	Inputs  map[string]Config `json:"inputs" yaml:"inputs"`
    62  	Prefix  string            `json:"prefix" yaml:"prefix"`
    63  	Timeout string            `json:"timeout" yaml:"timeout"`
    64  }
    65  
    66  // NewDynamicConfig creates a new DynamicConfig with default values.
    67  func NewDynamicConfig() DynamicConfig {
    68  	return DynamicConfig{
    69  		Inputs:  map[string]Config{},
    70  		Prefix:  "",
    71  		Timeout: "5s",
    72  	}
    73  }
    74  
    75  //------------------------------------------------------------------------------
    76  
    77  // NewDynamic creates a new Dynamic input type.
    78  func NewDynamic(
    79  	conf Config,
    80  	mgr types.Manager,
    81  	log log.Modular,
    82  	stats metrics.Type,
    83  	pipelines ...types.PipelineConstructorFunc,
    84  ) (Type, error) {
    85  	dynAPI := api.NewDynamic()
    86  
    87  	inputs := map[string]broker.DynamicInput{}
    88  	for k, v := range conf.Dynamic.Inputs {
    89  		newInput, err := New(v, mgr, log, stats, pipelines...)
    90  		if err != nil {
    91  			return nil, err
    92  		}
    93  		inputs[k] = newInput
    94  	}
    95  
    96  	var timeout time.Duration
    97  	if tout := conf.Dynamic.Timeout; len(tout) > 0 {
    98  		var err error
    99  		if timeout, err = time.ParseDuration(tout); err != nil {
   100  			return nil, fmt.Errorf("failed to parse timeout string: %v", err)
   101  		}
   102  	}
   103  
   104  	inputConfigs := conf.Dynamic.Inputs
   105  	inputConfigsMut := sync.RWMutex{}
   106  
   107  	fanIn, err := broker.NewDynamicFanIn(
   108  		inputs, log, stats,
   109  		broker.OptDynamicFanInSetOnAdd(func(l string) {
   110  			inputConfigsMut.Lock()
   111  			defer inputConfigsMut.Unlock()
   112  
   113  			uConf, exists := inputConfigs[l]
   114  			if !exists {
   115  				return
   116  			}
   117  			sConf, bErr := SanitiseConfig(uConf)
   118  			if bErr != nil {
   119  				log.Errorf("Failed to sanitise config: %v\n", bErr)
   120  			}
   121  
   122  			confBytes, _ := json.Marshal(sConf)
   123  			dynAPI.Started(l, confBytes)
   124  			delete(inputConfigs, l)
   125  		}),
   126  		broker.OptDynamicFanInSetOnRemove(func(l string) {
   127  			dynAPI.Stopped(l)
   128  		}),
   129  	)
   130  	if err != nil {
   131  		return nil, err
   132  	}
   133  
   134  	dynAPI.OnUpdate(func(id string, c []byte) error {
   135  		newConf := NewConfig()
   136  		if err := yaml.Unmarshal(c, &newConf); err != nil {
   137  			return err
   138  		}
   139  		iMgr, iLog, iStats := interop.LabelChild(fmt.Sprintf("dynamic.inputs.%v", id), mgr, log, stats)
   140  		iStats = metrics.Combine(stats, iStats)
   141  		newInput, err := New(newConf, iMgr, iLog, iStats, pipelines...)
   142  		if err != nil {
   143  			return err
   144  		}
   145  		inputConfigsMut.Lock()
   146  		inputConfigs[id] = newConf
   147  		inputConfigsMut.Unlock()
   148  		if err = fanIn.SetInput(id, newInput, timeout); err != nil {
   149  			log.Errorf("Failed to set input '%v': %v", id, err)
   150  			inputConfigsMut.Lock()
   151  			delete(inputConfigs, id)
   152  			inputConfigsMut.Unlock()
   153  		}
   154  		return err
   155  	})
   156  	dynAPI.OnDelete(func(id string) error {
   157  		err := fanIn.SetInput(id, nil, timeout)
   158  		if err != nil {
   159  			log.Errorf("Failed to close input '%v': %v", id, err)
   160  		}
   161  		return err
   162  	})
   163  
   164  	mgr.RegisterEndpoint(
   165  		path.Join(conf.Dynamic.Prefix, "/inputs/{id}"),
   166  		"Perform CRUD operations on the configuration of dynamic inputs. For"+
   167  			" more information read the `dynamic` input type documentation.",
   168  		dynAPI.HandleCRUD,
   169  	)
   170  	mgr.RegisterEndpoint(
   171  		path.Join(conf.Dynamic.Prefix, "/inputs"),
   172  		"Get a map of running input identifiers with their current uptimes.",
   173  		dynAPI.HandleList,
   174  	)
   175  
   176  	return fanIn, nil
   177  }
   178  
   179  //------------------------------------------------------------------------------