diff --git a/clients/cmd/promtail/promtail-kafka.yaml b/clients/cmd/promtail/promtail-kafka.yaml new file mode 100644 index 000000000000..7b0252acff18 --- /dev/null +++ b/clients/cmd/promtail/promtail-kafka.yaml @@ -0,0 +1,32 @@ +server: + http_listen_port: 9080 + grpc_listen_port: 0 + +clients: + - url: http://localhost:3100/loki/api/v1/push + +scrape_configs: +- job_name: kafka + kafka: + use_incoming_timestamp: false + brokers: + - host.docker.internal:50705 + group_id: some_group + topics: + - foo + - ^promtail.* + labels: + job: kafka + relabel_configs: + - action: replace + source_labels: + - __meta_kafka_topic + target_label: topic + - action: replace + source_labels: + - __meta_kafka_partition + target_label: partition + - action: replace + source_labels: + - __meta_kafka_group_id + target_label: group diff --git a/clients/pkg/promtail/scrapeconfig/scrapeconfig.go b/clients/pkg/promtail/scrapeconfig/scrapeconfig.go index 693cfe7fc5d5..2fc2bec148a5 100644 --- a/clients/pkg/promtail/scrapeconfig/scrapeconfig.go +++ b/clients/pkg/promtail/scrapeconfig/scrapeconfig.go @@ -37,6 +37,7 @@ type Config struct { GcplogConfig *GcplogTargetConfig `yaml:"gcplog,omitempty"` PushConfig *PushTargetConfig `yaml:"loki_push_api,omitempty"` WindowsConfig *WindowsEventsTargetConfig `yaml:"windows_events,omitempty"` + KafkaConfig *KafkaTargetConfig `yaml:"kafka,omitempty"` RelabelConfigs []*relabel.Config `yaml:"relabel_configs,omitempty"` ServiceDiscoveryConfig ServiceDiscoveryConfig `yaml:",inline"` } @@ -221,6 +222,30 @@ type WindowsEventsTargetConfig struct { Labels model.LabelSet `yaml:"labels"` } +type KafkaTargetConfig struct { + // Labels optionally holds labels to associate with each log line. + Labels model.LabelSet `yaml:"labels"` + + // UseIncomingTimestamp sets the timestamp to the incoming kafka messages + // timestamp if it's set. + UseIncomingTimestamp bool `yaml:"use_incoming_timestamp"` + + // The list of brokers to connect to kafka (Required). + Brokers []string `yaml:"brokers"` + + // The consumer group id (Required). + GroupID string `yaml:"group_id"` + + // Kafka Topics to consume (Required). + Topics []string `yaml:"topics"` + + // Kafka version. Default to 2.2.1 + Version string `yaml:"version"` + + // Rebalancing strategy to use. (e.g sticky, roundrobin or range) + Assignor string `yaml:"assignor"` +} + // GcplogTargetConfig describes a scrape config to pull logs from any pubsub topic. type GcplogTargetConfig struct { // ProjectID is the Cloud project id @@ -263,7 +288,6 @@ func (c *Config) HasServiceDiscoveryConfig() bool { // UnmarshalYAML implements the yaml.Unmarshaler interface. func (c *Config) UnmarshalYAML(unmarshal func(interface{}) error) error { - *c = DefaultScrapeConfig type plain Config diff --git a/clients/pkg/promtail/targets/kafka/consumer.go b/clients/pkg/promtail/targets/kafka/consumer.go new file mode 100644 index 000000000000..57eb24245c0e --- /dev/null +++ b/clients/pkg/promtail/targets/kafka/consumer.go @@ -0,0 +1,150 @@ +package kafka + +import ( + "context" + "fmt" + "sync" + "time" + + "github.com/Shopify/sarama" + "github.com/go-kit/log" + "github.com/go-kit/log/level" + "github.com/grafana/dskit/backoff" + "github.com/grafana/loki/clients/pkg/promtail/targets/target" +) + +var defaultBackOff = backoff.Config{ + MinBackoff: 1 * time.Second, + MaxBackoff: 60 * time.Second, + MaxRetries: 20, +} + +type RunnableTarget interface { + target.Target + run() +} + +type TargetDiscoverer interface { + NewTarget(sarama.ConsumerGroupSession, sarama.ConsumerGroupClaim) (RunnableTarget, error) +} + +// consumer handle a group consumer instance. +// It will create a new target for every consumer claim using the `TargetDiscoverer`. +type consumer struct { + sarama.ConsumerGroup + discoverer TargetDiscoverer + logger log.Logger + + ctx context.Context + cancel context.CancelFunc + wg sync.WaitGroup + + mutex sync.Mutex // used during rebalancing setup and tear down + activeTargets []target.Target + droppedTargets []target.Target +} + +// start starts the consumer for a given list of topics. +func (c *consumer) start(ctx context.Context, topics []string) { + c.wg.Wait() + c.wg.Add(1) + + c.ctx, c.cancel = context.WithCancel(ctx) + level.Info(c.logger).Log("msg", "starting consumer", "topics", fmt.Sprintf("%+v", topics)) + + go func() { + defer c.wg.Done() + backoff := backoff.New(c.ctx, defaultBackOff) + for { + // Calling Consume in an infinite loop in case rebalancing is kicking in. + // In which case all claims will be renewed. + err := c.ConsumerGroup.Consume(c.ctx, topics, c) + if err != nil && err != context.Canceled { + level.Error(c.logger).Log("msg", "error from the consumer, retrying...", "err", err) + // backoff before re-trying. + backoff.Wait() + if backoff.Ongoing() { + continue + } + level.Error(c.logger).Log("msg", "maximun error from the consumer reached", "last_err", err) + return + } + if c.ctx.Err() != nil || err == context.Canceled { + level.Info(c.logger).Log("msg", "stopping consumer", "topics", fmt.Sprintf("%+v", topics)) + return + } + backoff.Reset() + } + }() +} + +// ConsumeClaim creates a target for the given received claim and start reading message from it. +func (c *consumer) ConsumeClaim(session sarama.ConsumerGroupSession, claim sarama.ConsumerGroupClaim) error { + c.wg.Add(1) + defer c.wg.Done() + + t, err := c.discoverer.NewTarget(session, claim) + if err != nil { + return err + } + if len(t.Labels()) == 0 { + c.addDroppedTarget(t) + t.run() + return nil + } + c.addTarget(t) + level.Info(c.logger).Log("msg", "consuming topic", "details", t.Details()) + t.run() + + return nil +} + +// Setup is run at the beginning of a new session, before ConsumeClaim +func (c *consumer) Setup(session sarama.ConsumerGroupSession) error { + c.resetTargets() + return nil +} + +// Cleanup is run at the end of a session, once all ConsumeClaim goroutines have exited +func (c *consumer) Cleanup(sarama.ConsumerGroupSession) error { + c.resetTargets() + return nil +} + +// stop stops the consumer. +func (c *consumer) stop() { + c.cancel() + c.wg.Wait() + c.resetTargets() +} + +func (c *consumer) resetTargets() { + c.mutex.Lock() + defer c.mutex.Unlock() + c.activeTargets = nil + c.droppedTargets = nil +} + +func (c *consumer) getActiveTargets() []target.Target { + c.mutex.Lock() + defer c.mutex.Unlock() + return c.activeTargets +} + +func (c *consumer) getDroppedTargets() []target.Target { + c.mutex.Lock() + defer c.mutex.Unlock() + return c.droppedTargets +} + +func (c *consumer) addTarget(t target.Target) { + c.mutex.Lock() + defer c.mutex.Unlock() + c.activeTargets = append(c.activeTargets, t) +} + +func (c *consumer) addDroppedTarget(t target.Target) { + c.mutex.Lock() + defer c.mutex.Unlock() + c.droppedTargets = append(c.droppedTargets, t) +} diff --git a/clients/pkg/promtail/targets/kafka/consumer_test.go b/clients/pkg/promtail/targets/kafka/consumer_test.go new file mode 100644 index 000000000000..c9f591fd2c5c --- /dev/null +++ b/clients/pkg/promtail/targets/kafka/consumer_test.go @@ -0,0 +1,102 @@ +package kafka + +import ( + "context" + "errors" + "testing" + "time" + + "github.com/Shopify/sarama" + "github.com/go-kit/log" + "github.com/grafana/loki/clients/pkg/promtail/targets/target" + "github.com/prometheus/common/model" + "github.com/stretchr/testify/require" +) + +type DiscovererFn func(sarama.ConsumerGroupSession, sarama.ConsumerGroupClaim) (RunnableTarget, error) + +func (d DiscovererFn) NewTarget(session sarama.ConsumerGroupSession, claim sarama.ConsumerGroupClaim) (RunnableTarget, error) { + return d(session, claim) +} + +type fakeTarget struct { + ctx context.Context + lbs model.LabelSet +} + +func (f *fakeTarget) run() { <-f.ctx.Done() } +func (f *fakeTarget) Type() target.TargetType { return "" } +func (f *fakeTarget) DiscoveredLabels() model.LabelSet { return nil } +func (f *fakeTarget) Labels() model.LabelSet { return f.lbs } +func (f *fakeTarget) Ready() bool { return true } +func (f *fakeTarget) Details() interface{} { return nil } + +func Test_ComsumerConsume(t *testing.T) { + var ( + group = &testConsumerGroupHandler{} + session = &testSession{} + ctx, cancel = context.WithCancel(context.Background()) + c = &consumer{ + logger: log.NewNopLogger(), + ctx: context.Background(), + cancel: func() {}, + ConsumerGroup: group, + discoverer: DiscovererFn(func(session sarama.ConsumerGroupSession, claim sarama.ConsumerGroupClaim) (RunnableTarget, error) { + if claim.Topic() != "dropped" { + return &fakeTarget{ + ctx: ctx, + lbs: model.LabelSet{"topic": model.LabelValue(claim.Topic())}, + }, nil + } + return &fakeTarget{ + ctx: ctx, + }, nil + }), + } + ) + + c.start(ctx, []string{"foo"}) + require.Eventually(t, group.consuming.Load, 5*time.Second, 100*time.Microsecond) + require.NoError(t, group.handler.Setup(session)) + go func() { + err := group.handler.ConsumeClaim(session, newTestClaim("foo", 1, 2)) + require.NoError(t, err) + }() + go func() { + err := group.handler.ConsumeClaim(session, newTestClaim("bar", 1, 2)) + require.NoError(t, err) + }() + go func() { + err := group.handler.ConsumeClaim(session, newTestClaim("dropped", 1, 2)) + require.NoError(t, err) + }() + require.Eventually(t, func() bool { + return len(c.getActiveTargets()) == 2 + }, 2*time.Second, 100*time.Millisecond) + require.Eventually(t, func() bool { + return len(c.getDroppedTargets()) == 1 + }, 2*time.Second, 100*time.Millisecond) + err := group.handler.Cleanup(session) + require.NoError(t, err) + cancel() + c.stop() +} + +func Test_ComsumerRetry(t *testing.T) { + var ( + group = &testConsumerGroupHandler{ + returnErr: errors.New("foo"), + } + ctx, cancel = context.WithCancel(context.Background()) + c = &consumer{ + logger: log.NewNopLogger(), + ctx: context.Background(), + cancel: func() {}, + ConsumerGroup: group, + } + ) + defer cancel() + c.start(ctx, []string{"foo"}) + <-time.After(2 * time.Second) + c.stop() +} diff --git a/clients/pkg/promtail/targets/kafka/target.go b/clients/pkg/promtail/targets/kafka/target.go new file mode 100644 index 000000000000..eb05a200b30c --- /dev/null +++ b/clients/pkg/promtail/targets/kafka/target.go @@ -0,0 +1,121 @@ +package kafka + +import ( + "fmt" + "time" + + "github.com/Shopify/sarama" + "github.com/prometheus/common/model" + + "github.com/grafana/loki/clients/pkg/promtail/api" + "github.com/grafana/loki/clients/pkg/promtail/targets/target" + + "github.com/grafana/loki/pkg/logproto" +) + +type runnableDroppedTarget struct { + target.Target + runFn func() +} + +func (d *runnableDroppedTarget) run() { + d.runFn() +} + +type Target struct { + discoveredLabels model.LabelSet + lbs model.LabelSet + details ConsumerDetails + claim sarama.ConsumerGroupClaim + session sarama.ConsumerGroupSession + client api.EntryHandler + useIncomingTimestamp bool +} + +func NewTarget( + session sarama.ConsumerGroupSession, + claim sarama.ConsumerGroupClaim, + discoveredLabels, lbs model.LabelSet, + client api.EntryHandler, + useIncomingTimestamp bool, +) *Target { + return &Target{ + discoveredLabels: discoveredLabels, + lbs: lbs, + details: newDetails(session, claim), + claim: claim, + session: session, + client: client, + useIncomingTimestamp: useIncomingTimestamp, + } +} + +func (t *Target) run() { + defer t.client.Stop() + + for message := range t.claim.Messages() { + t.client.Chan() <- api.Entry{ + Entry: logproto.Entry{ + Line: string(message.Value), + Timestamp: timestamp(t.useIncomingTimestamp, message.Timestamp), + }, + Labels: t.lbs.Clone(), + } + t.session.MarkMessage(message, "") + } +} + +func timestamp(useIncoming bool, incoming time.Time) time.Time { + if useIncoming { + return incoming + } + return time.Now() +} + +func (t *Target) Type() target.TargetType { + return target.KafkaTargetType +} + +func (t *Target) Ready() bool { + return true +} + +func (t *Target) DiscoveredLabels() model.LabelSet { + return t.discoveredLabels +} + +func (t *Target) Labels() model.LabelSet { + return t.lbs +} + +// Details returns target-specific details. +func (t *Target) Details() interface{} { + return t.details +} + +type ConsumerDetails struct { + + // MemberID returns the cluster member ID. + MemberID string + + // GenerationID returns the current generation ID. + GenerationID int32 + + Topic string + Partition int32 + InitialOffset int64 +} + +func (c ConsumerDetails) String() string { + return fmt.Sprintf("member_id=%s generation_id=%d topic=%s partition=%d initial_offset=%d", c.MemberID, c.GenerationID, c.Topic, c.Partition, c.InitialOffset) +} + +func newDetails(session sarama.ConsumerGroupSession, claim sarama.ConsumerGroupClaim) ConsumerDetails { + return ConsumerDetails{ + MemberID: session.MemberID(), + GenerationID: session.GenerationID(), + Topic: claim.Topic(), + Partition: claim.Partition(), + InitialOffset: claim.InitialOffset(), + } +} diff --git a/clients/pkg/promtail/targets/kafka/target_syncer.go b/clients/pkg/promtail/targets/kafka/target_syncer.go new file mode 100644 index 000000000000..94c24b6eb272 --- /dev/null +++ b/clients/pkg/promtail/targets/kafka/target_syncer.go @@ -0,0 +1,257 @@ +package kafka + +import ( + "context" + "errors" + "fmt" + "strings" + "sync" + "time" + + "github.com/Shopify/sarama" + "github.com/go-kit/log" + "github.com/go-kit/log/level" + "github.com/prometheus/client_golang/prometheus" + "github.com/prometheus/common/model" + "github.com/prometheus/prometheus/pkg/labels" + "github.com/prometheus/prometheus/pkg/relabel" + + "github.com/grafana/loki/clients/pkg/logentry/stages" + "github.com/grafana/loki/clients/pkg/promtail/api" + "github.com/grafana/loki/clients/pkg/promtail/scrapeconfig" + "github.com/grafana/loki/clients/pkg/promtail/targets/target" + + "github.com/grafana/loki/pkg/util" +) + +var TopicPollInterval = 30 * time.Second + +type TopicManager interface { + Topics() ([]string, error) +} + +type TargetSyncer struct { + logger log.Logger + cfg scrapeconfig.Config + reg prometheus.Registerer + client api.EntryHandler + + topicManager TopicManager + consumer + close func() error + + ctx context.Context + cancel context.CancelFunc + wg sync.WaitGroup + previousTopics []string +} + +func NewSyncer( + reg prometheus.Registerer, + logger log.Logger, + cfg scrapeconfig.Config, + pushClient api.EntryHandler, +) (*TargetSyncer, error) { + if err := validateConfig(&cfg); err != nil { + return nil, err + } + version, err := sarama.ParseKafkaVersion(cfg.KafkaConfig.Version) + if err != nil { + return nil, err + } + config := sarama.NewConfig() + config.Version = version + config.Consumer.Offsets.Initial = sarama.OffsetOldest + + switch cfg.KafkaConfig.Assignor { + case sarama.StickyBalanceStrategyName: + config.Consumer.Group.Rebalance.Strategy = sarama.BalanceStrategySticky + case sarama.RoundRobinBalanceStrategyName: + config.Consumer.Group.Rebalance.Strategy = sarama.BalanceStrategyRoundRobin + case sarama.RangeBalanceStrategyName, "": + config.Consumer.Group.Rebalance.Strategy = sarama.BalanceStrategyRange + default: + return nil, fmt.Errorf("unrecognized consumer group partition assignor: %s", cfg.KafkaConfig.Assignor) + } + client, err := sarama.NewClient(cfg.KafkaConfig.Brokers, config) + if err != nil { + return nil, fmt.Errorf("error creating kafka client: %w", err) + } + group, err := sarama.NewConsumerGroup(cfg.KafkaConfig.Brokers, cfg.KafkaConfig.GroupID, config) + if err != nil { + return nil, fmt.Errorf("error creating consumer group client: %w", err) + } + topicManager, err := newTopicManager(client, cfg.KafkaConfig.Topics) + if err != nil { + return nil, fmt.Errorf("error creating topic manager: %w", err) + } + ctx, cancel := context.WithCancel(context.Background()) + + t := &TargetSyncer{ + logger: logger, + ctx: ctx, + cancel: cancel, + topicManager: topicManager, + cfg: cfg, + reg: reg, + client: pushClient, + close: func() error { + if err := group.Close(); err != nil { + level.Warn(logger).Log("msg", "error while closing consumer group", "err", err) + } + return client.Close() + }, + consumer: consumer{ + ctx: context.Background(), + cancel: func() {}, + ConsumerGroup: group, + logger: logger, + }, + } + t.discoverer = t + t.loop() + return t, nil +} + +func (ts *TargetSyncer) loop() { + topicChanged := make(chan []string) + ts.wg.Add(2) + go func() { + defer ts.wg.Done() + for { + select { + case <-ts.ctx.Done(): + return + case topics := <-topicChanged: + level.Info(ts.logger).Log("msg", "new topics received", "topics", fmt.Sprintf("%+v", topics)) + ts.stop() + if len(topics) > 0 { // no topics we don't need to start. + ts.start(ts.ctx, topics) + } + } + } + }() + go func() { + defer ts.wg.Done() + ticker := time.NewTicker(TopicPollInterval) + defer ticker.Stop() + + tick := func() { + select { + case <-ts.ctx.Done(): + case <-ticker.C: + } + } + for ; true; tick() { // instant tick. + if ts.ctx.Err() != nil { + ts.stop() + close(topicChanged) + return + } + newTopics, ok, err := ts.fetchTopics() + if err != nil { + level.Warn(ts.logger).Log("msg", "failed to fetch topics", "err", err) + continue + } + if ok { + topicChanged <- newTopics + } + + } + }() +} + +// fetchTopics fetches and return new topics, if there's a difference with previous found topics +// it will return true as second return value. +func (ts *TargetSyncer) fetchTopics() ([]string, bool, error) { + new, err := ts.topicManager.Topics() + if err != nil { + return nil, false, err + } + if len(ts.previousTopics) != len(new) { + ts.previousTopics = new + return new, true, nil + } + for i, v := range ts.previousTopics { + if v != new[i] { + ts.previousTopics = new + return new, true, nil + } + } + return nil, false, nil +} + +func (ts *TargetSyncer) Stop() error { + ts.cancel() + ts.wg.Wait() + return ts.close() +} + +// NewTarget creates a new targets based on the current kafka claim and group session. +func (ts *TargetSyncer) NewTarget(session sarama.ConsumerGroupSession, claim sarama.ConsumerGroupClaim) (RunnableTarget, error) { + discoveredLabels := model.LabelSet{ + "__meta_kafka_topic": model.LabelValue(claim.Topic()), + "__meta_kafka_partition": model.LabelValue(fmt.Sprintf("%d", claim.Partition())), + "__meta_kafka_member_id": model.LabelValue(session.MemberID()), + "__meta_kafka_group_id": model.LabelValue(ts.cfg.KafkaConfig.GroupID), + } + labelMap := make(map[string]string) + for k, v := range discoveredLabels.Clone().Merge(ts.cfg.KafkaConfig.Labels) { + labelMap[string(k)] = string(v) + } + lbs := relabel.Process(labels.FromMap(labelMap), ts.cfg.RelabelConfigs...) + details := newDetails(session, claim) + labelOut := model.LabelSet(util.LabelsToMetric(lbs)) + for k := range labelOut { + if strings.HasPrefix(string(k), "__") { + delete(labelOut, k) + } + } + if len(labelOut) == 0 { + level.Warn(ts.logger).Log("msg", "dropping target", "reason", "no labels", "details", details, "discovered_labels", discoveredLabels.String()) + return &runnableDroppedTarget{ + Target: target.NewDroppedTarget("dropping target, no labels", discoveredLabels), + runFn: func() { + for range claim.Messages() { + } + }, + }, nil + } + + pipeline, err := stages.NewPipeline(log.With(ts.logger, "component", "kafka_pipeline"), ts.cfg.PipelineStages, &ts.cfg.JobName, ts.reg) + if err != nil { + return nil, err + } + + t := NewTarget( + session, + claim, + discoveredLabels, + labelOut, + pipeline.Wrap(ts.client), + ts.cfg.KafkaConfig.UseIncomingTimestamp, + ) + + return t, nil +} + +func validateConfig(cfg *scrapeconfig.Config) error { + if cfg.KafkaConfig == nil { + return errors.New("Kafka configuration is empty") + } + if cfg.KafkaConfig.Version == "" { + cfg.KafkaConfig.Version = "2.1.1" + } + if len(cfg.KafkaConfig.Brokers) == 0 { + return errors.New("no Kafka bootstrap brokers defined") + } + + if len(cfg.KafkaConfig.Topics) == 0 { + return errors.New("no topics given to be consumed") + } + + if cfg.KafkaConfig.GroupID == "" { + cfg.KafkaConfig.GroupID = "promtail" + } + return nil +} diff --git a/clients/pkg/promtail/targets/kafka/target_syncer_test.go b/clients/pkg/promtail/targets/kafka/target_syncer_test.go new file mode 100644 index 000000000000..8099476eaa16 --- /dev/null +++ b/clients/pkg/promtail/targets/kafka/target_syncer_test.go @@ -0,0 +1,210 @@ +package kafka + +import ( + "context" + "fmt" + "testing" + "time" + + "github.com/Shopify/sarama" + "github.com/go-kit/log" + "github.com/grafana/loki/clients/pkg/promtail/client/fake" + "github.com/grafana/loki/clients/pkg/promtail/scrapeconfig" + "github.com/prometheus/client_golang/prometheus" + "github.com/prometheus/common/model" + "github.com/prometheus/prometheus/pkg/relabel" + "github.com/stretchr/testify/assert" + "github.com/stretchr/testify/require" +) + +func Test_TopicDiscovery(t *testing.T) { + ctx, cancel := context.WithCancel(context.Background()) + group := &testConsumerGroupHandler{} + TopicPollInterval = time.Microsecond + var closed bool + client := &mockKafkaClient{ + topics: []string{"topic1"}, + } + ts := &TargetSyncer{ + ctx: ctx, + cancel: cancel, + logger: log.NewNopLogger(), + reg: prometheus.DefaultRegisterer, + topicManager: mustNewTopicsManager(client, []string{"topic1", "topic2"}), + close: func() error { + closed = true + return nil + }, + consumer: consumer{ + ctx: context.Background(), + cancel: func() {}, + ConsumerGroup: group, + logger: log.NewNopLogger(), + discoverer: DiscovererFn(func(s sarama.ConsumerGroupSession, c sarama.ConsumerGroupClaim) (RunnableTarget, error) { + return nil, nil + }), + }, + cfg: scrapeconfig.Config{ + JobName: "foo", + RelabelConfigs: []*relabel.Config{}, + KafkaConfig: &scrapeconfig.KafkaTargetConfig{ + UseIncomingTimestamp: true, + Topics: []string{"topic1", "topic2"}, + }, + }, + } + + ts.loop() + require.Eventually(t, func() bool { + if !group.consuming.Load() { + return false + } + return assert.Equal(t, group.topics, []string{"topic1"}) + }, 200*time.Millisecond, time.Millisecond) + + client.topics = []string{"topic1", "topic2"} // introduce new topics + + require.Eventually(t, func() bool { + if !group.consuming.Load() { + return false + } + return assert.Equal(t, group.topics, []string{"topic1", "topic2"}) + }, 200*time.Millisecond, time.Millisecond) + + require.NoError(t, ts.Stop()) + require.True(t, closed) +} + +func Test_NewTarget(t *testing.T) { + ts := &TargetSyncer{ + logger: log.NewNopLogger(), + reg: prometheus.DefaultRegisterer, + client: fake.New(func() {}), + cfg: scrapeconfig.Config{ + JobName: "foo", + RelabelConfigs: []*relabel.Config{ + { + SourceLabels: model.LabelNames{"__meta_kafka_topic"}, + TargetLabel: "topic", + Replacement: "$1", + Action: relabel.Replace, + Regex: relabel.MustNewRegexp("(.*)"), + }, + }, + KafkaConfig: &scrapeconfig.KafkaTargetConfig{ + UseIncomingTimestamp: true, + GroupID: "group_1", + Topics: []string{"topic1", "topic2"}, + Labels: model.LabelSet{"static": "static1"}, + }, + }, + } + tg, err := ts.NewTarget(&testSession{}, newTestClaim("foo", 10, 1)) + + require.NoError(t, err) + require.Equal(t, ConsumerDetails{ + MemberID: "foo", + GenerationID: 10, + Topic: "foo", + Partition: 10, + InitialOffset: 1, + }, tg.Details()) + require.Equal(t, model.LabelSet{"static": "static1", "topic": "foo"}, tg.Labels()) + require.Equal(t, model.LabelSet{"__meta_kafka_member_id": "foo", "__meta_kafka_partition": "10", "__meta_kafka_topic": "foo", "__meta_kafka_group_id": "group_1"}, tg.DiscoveredLabels()) +} + +func Test_NewDroppedTarget(t *testing.T) { + ts := &TargetSyncer{ + logger: log.NewNopLogger(), + reg: prometheus.DefaultRegisterer, + cfg: scrapeconfig.Config{ + JobName: "foo", + KafkaConfig: &scrapeconfig.KafkaTargetConfig{ + UseIncomingTimestamp: true, + GroupID: "group1", + Topics: []string{"topic1", "topic2"}, + }, + }, + } + tg, err := ts.NewTarget(&testSession{}, newTestClaim("foo", 10, 1)) + + require.NoError(t, err) + require.Equal(t, "dropping target, no labels", tg.Details()) + require.Equal(t, model.LabelSet(nil), tg.Labels()) + require.Equal(t, model.LabelSet{"__meta_kafka_member_id": "foo", "__meta_kafka_partition": "10", "__meta_kafka_topic": "foo", "__meta_kafka_group_id": "group1"}, tg.DiscoveredLabels()) +} + +func Test_validateConfig(t *testing.T) { + tests := []struct { + cfg *scrapeconfig.Config + wantErr bool + expected *scrapeconfig.Config + }{ + { + &scrapeconfig.Config{ + KafkaConfig: nil, + }, + true, + nil, + }, + { + &scrapeconfig.Config{ + KafkaConfig: &scrapeconfig.KafkaTargetConfig{ + GroupID: "foo", + Topics: []string{"bar"}, + }, + }, + true, + nil, + }, + { + &scrapeconfig.Config{ + KafkaConfig: &scrapeconfig.KafkaTargetConfig{ + Brokers: []string{"foo"}, + GroupID: "bar", + }, + }, + true, + nil, + }, + { + &scrapeconfig.Config{ + KafkaConfig: &scrapeconfig.KafkaTargetConfig{ + Brokers: []string{"foo"}, + }, + }, + true, + nil, + }, + { + &scrapeconfig.Config{ + KafkaConfig: &scrapeconfig.KafkaTargetConfig{ + Brokers: []string{"foo"}, + Topics: []string{"bar"}, + }, + }, + false, + &scrapeconfig.Config{ + KafkaConfig: &scrapeconfig.KafkaTargetConfig{ + Brokers: []string{"foo"}, + Topics: []string{"bar"}, + GroupID: "promtail", + Version: "2.1.1", + }, + }, + }, + } + + for i, tt := range tests { + tt := tt + t.Run(fmt.Sprintf("%d", i), func(t *testing.T) { + err := validateConfig(tt.cfg) + if (err != nil) != tt.wantErr { + t.Errorf("validateConfig() error = %v, wantErr %v", err, tt.wantErr) + } + if err == nil { + require.Equal(t, tt.expected, tt.cfg) + } + }) + } +} diff --git a/clients/pkg/promtail/targets/kafka/target_test.go b/clients/pkg/promtail/targets/kafka/target_test.go new file mode 100644 index 000000000000..c85f3ae72822 --- /dev/null +++ b/clients/pkg/promtail/targets/kafka/target_test.go @@ -0,0 +1,121 @@ +package kafka + +import ( + "context" + "fmt" + "sync" + "testing" + "time" + + "github.com/Shopify/sarama" + "github.com/grafana/loki/clients/pkg/promtail/client/fake" + "github.com/prometheus/common/model" + "github.com/stretchr/testify/require" + "go.uber.org/atomic" +) + +// Consumergroup handler +type testConsumerGroupHandler struct { + handler sarama.ConsumerGroupHandler + ctx context.Context + topics []string + + returnErr error + + consuming atomic.Bool +} + +func (c *testConsumerGroupHandler) Consume(ctx context.Context, topics []string, handler sarama.ConsumerGroupHandler) error { + if c.returnErr != nil { + return c.returnErr + } + c.ctx = ctx + c.topics = topics + c.handler = handler + c.consuming.Store(true) + <-ctx.Done() + c.consuming.Store(false) + return nil +} + +func (c testConsumerGroupHandler) Errors() <-chan error { + return nil +} + +func (c testConsumerGroupHandler) Close() error { + return nil +} + +type testSession struct { + markedMessage []*sarama.ConsumerMessage +} + +func (s *testSession) Claims() map[string][]int32 { return nil } +func (s *testSession) MemberID() string { return "foo" } +func (s *testSession) GenerationID() int32 { return 10 } +func (s *testSession) MarkOffset(topic string, partition int32, offset int64, metadata string) {} +func (s *testSession) Commit() {} +func (s *testSession) ResetOffset(topic string, partition int32, offset int64, metadata string) {} +func (s *testSession) MarkMessage(msg *sarama.ConsumerMessage, metadata string) { + s.markedMessage = append(s.markedMessage, msg) +} +func (s *testSession) Context() context.Context { return context.Background() } + +type testClaim struct { + topic string + partition int32 + offset int64 + messages chan *sarama.ConsumerMessage +} + +func newTestClaim(topic string, partition int32, offset int64) *testClaim { + return &testClaim{ + topic: topic, + partition: partition, + offset: offset, + messages: make(chan *sarama.ConsumerMessage), + } +} + +func (t *testClaim) Topic() string { return t.topic } +func (t *testClaim) Partition() int32 { return t.partition } +func (t *testClaim) InitialOffset() int64 { return t.offset } +func (t *testClaim) HighWaterMarkOffset() int64 { return 0 } +func (t *testClaim) Messages() <-chan *sarama.ConsumerMessage { return t.messages } +func (t *testClaim) Send(m *sarama.ConsumerMessage) { + t.messages <- m +} + +func (t *testClaim) Stop() { + close(t.messages) +} + +func Test_TargetRun(t *testing.T) { + session, claim := &testSession{}, newTestClaim("footopic", 10, 12) + var closed bool + fc := fake.New( + func() { + closed = true + }, + ) + tg := NewTarget(session, claim, model.LabelSet{"foo": "bar"}, model.LabelSet{"buzz": "bazz"}, fc, true) + + var wg sync.WaitGroup + wg.Add(1) + go func() { + defer wg.Done() + tg.run() + }() + + for i := 0; i < 10; i++ { + claim.Send(&sarama.ConsumerMessage{ + Timestamp: time.Unix(0, int64(i)), + Value: []byte(fmt.Sprintf("%d", i)), + }) + } + claim.Stop() + wg.Wait() + require.Len(t, session.markedMessage, 10) + require.Len(t, fc.Received(), 10) + require.True(t, closed) +} diff --git a/clients/pkg/promtail/targets/kafka/targetmanager.go b/clients/pkg/promtail/targets/kafka/targetmanager.go new file mode 100644 index 000000000000..887706a3960c --- /dev/null +++ b/clients/pkg/promtail/targets/kafka/targetmanager.go @@ -0,0 +1,73 @@ +package kafka + +import ( + "github.com/go-kit/log" + "github.com/go-kit/log/level" + "github.com/prometheus/client_golang/prometheus" + + "github.com/grafana/loki/clients/pkg/promtail/api" + "github.com/grafana/loki/clients/pkg/promtail/scrapeconfig" + "github.com/grafana/loki/clients/pkg/promtail/targets/target" +) + +// TargetManager manages a series of kafka targets. +type TargetManager struct { + logger log.Logger + targetSyncers map[string]*TargetSyncer +} + +// NewTargetManager creates a new Kafka managers. +func NewTargetManager( + reg prometheus.Registerer, + logger log.Logger, + pushClient api.EntryHandler, + scrapeConfigs []scrapeconfig.Config, +) (*TargetManager, error) { + tm := &TargetManager{ + logger: logger, + targetSyncers: make(map[string]*TargetSyncer), + } + for _, cfg := range scrapeConfigs { + t, err := NewSyncer(reg, logger, cfg, pushClient) + if err != nil { + return nil, err + } + tm.targetSyncers[cfg.JobName] = t + } + + return tm, nil +} + +// Ready returns true if at least one Kafka target is active. +func (tm *TargetManager) Ready() bool { + for _, t := range tm.targetSyncers { + if len(t.getActiveTargets()) > 0 { + return true + } + } + return false +} + +func (tm *TargetManager) Stop() { + for _, t := range tm.targetSyncers { + if err := t.Stop(); err != nil { + level.Error(t.logger).Log("msg", "error stopping kafka target", "err", err) + } + } +} + +func (tm *TargetManager) ActiveTargets() map[string][]target.Target { + result := make(map[string][]target.Target, len(tm.targetSyncers)) + for k, v := range tm.targetSyncers { + result[k] = v.getActiveTargets() + } + return result +} + +func (tm *TargetManager) AllTargets() map[string][]target.Target { + result := make(map[string][]target.Target, len(tm.targetSyncers)) + for k, v := range tm.targetSyncers { + result[k] = append(v.getActiveTargets(), v.getDroppedTargets()...) + } + return result +} diff --git a/clients/pkg/promtail/targets/kafka/topics.go b/clients/pkg/promtail/targets/kafka/topics.go new file mode 100644 index 000000000000..00899e65fca8 --- /dev/null +++ b/clients/pkg/promtail/targets/kafka/topics.go @@ -0,0 +1,78 @@ +package kafka + +import ( + "errors" + "fmt" + "regexp" + "sort" +) + +type topicClient interface { + RefreshMetadata(topics ...string) error + Topics() ([]string, error) +} + +type topicManager struct { + client topicClient + + patterns []*regexp.Regexp + matches []string +} + +// newTopicManager fetches topics and returns matchings one based on list of requested topics. +// If a topic starts with a '^' it is treated as a regexp and can match multiple topics. +func newTopicManager(client topicClient, topics []string) (*topicManager, error) { + var ( + patterns []*regexp.Regexp + matches []string + ) + for _, t := range topics { + if len(t) == 0 { + return nil, errors.New("invalid empty topic") + } + if t[0] != '^' { + matches = append(matches, t) + } + re, err := regexp.Compile(t) + if err != nil { + return nil, fmt.Errorf("invalid topic pattern: %w", err) + } + patterns = append(patterns, re) + } + return &topicManager{ + client: client, + patterns: patterns, + matches: matches, + }, nil +} + +func (tm *topicManager) Topics() ([]string, error) { + if err := tm.client.RefreshMetadata(); err != nil { + return nil, err + } + topics, err := tm.client.Topics() + if err != nil { + return nil, err + } + + result := make([]string, 0, len(topics)) + +Outer: + for _, topic := range topics { + for _, m := range tm.matches { + if m == topic { + result = append(result, topic) + continue Outer + } + } + for _, p := range tm.patterns { + if p.MatchString(topic) { + result = append(result, topic) + continue Outer + } + } + } + + sort.Strings(result) + return result, nil +} diff --git a/clients/pkg/promtail/targets/kafka/topics_test.go b/clients/pkg/promtail/targets/kafka/topics_test.go new file mode 100644 index 000000000000..dcab0c8d9191 --- /dev/null +++ b/clients/pkg/promtail/targets/kafka/topics_test.go @@ -0,0 +1,106 @@ +package kafka + +import ( + "errors" + "strings" + "testing" + + "github.com/stretchr/testify/require" +) + +type mockKafkaClient struct { + topics []string + err error +} + +func (m *mockKafkaClient) RefreshMetadata(topics ...string) error { + return nil +} + +func (m *mockKafkaClient) Topics() ([]string, error) { + return m.topics, m.err +} + +func Test_NewTopicManager(t *testing.T) { + t.Parallel() + + for _, tt := range []struct { + in []string + expectedErr bool + }{ + { + []string{""}, + true, + }, + { + []string{"^("}, + true, + }, + { + []string{"foo"}, + false, + }, + { + []string{"foo", "^foo.*"}, + false, + }, + } { + tt := tt + t.Run(strings.Join(tt.in, ","), func(t *testing.T) { + t.Parallel() + _, err := newTopicManager(&mockKafkaClient{}, tt.in) + if tt.expectedErr { + require.Error(t, err) + return + } + require.NoError(t, err) + }) + } +} + +func Test_Topics(t *testing.T) { + t.Parallel() + + for _, tt := range []struct { + manager *topicManager + expected []string + expectedErr bool + }{ + { + mustNewTopicsManager(&mockKafkaClient{err: errors.New("")}, []string{"foo"}), + []string{}, + true, + }, + { + mustNewTopicsManager(&mockKafkaClient{topics: []string{"foo", "foobar", "buzz"}}, []string{"^foo"}), + []string{"foo", "foobar"}, + false, + }, + { + mustNewTopicsManager(&mockKafkaClient{topics: []string{"foo", "foobar", "buzz"}}, []string{"^foo.*", "buzz"}), + []string{"buzz", "foo", "foobar"}, + false, + }, + } { + tt := tt + t.Run("", func(t *testing.T) { + t.Parallel() + + actual, err := tt.manager.Topics() + if tt.expectedErr { + require.Error(t, err) + return + } + require.NoError(t, err) + require.Equal(t, tt.expected, actual) + }) + } +} + +func mustNewTopicsManager(client topicClient, topics []string) *topicManager { + t, err := newTopicManager(client, topics) + if err != nil { + panic(err) + } + return t +} diff --git a/clients/pkg/promtail/targets/manager.go b/clients/pkg/promtail/targets/manager.go index 9ff0a08859f6..3d3704eb9502 100644 --- a/clients/pkg/promtail/targets/manager.go +++ b/clients/pkg/promtail/targets/manager.go @@ -9,11 +9,13 @@ import ( "github.com/prometheus/client_golang/prometheus" "github.com/grafana/loki/clients/pkg/promtail/api" + "github.com/grafana/loki/clients/pkg/promtail/client" "github.com/grafana/loki/clients/pkg/promtail/positions" "github.com/grafana/loki/clients/pkg/promtail/scrapeconfig" "github.com/grafana/loki/clients/pkg/promtail/targets/file" "github.com/grafana/loki/clients/pkg/promtail/targets/gcplog" "github.com/grafana/loki/clients/pkg/promtail/targets/journal" + "github.com/grafana/loki/clients/pkg/promtail/targets/kafka" "github.com/grafana/loki/clients/pkg/promtail/targets/lokipush" "github.com/grafana/loki/clients/pkg/promtail/targets/stdin" "github.com/grafana/loki/clients/pkg/promtail/targets/syslog" @@ -28,6 +30,7 @@ const ( GcplogScrapeConfigs = "gcplogScrapeConfigs" PushScrapeConfigs = "pushScrapeConfigs" WindowsEventsConfigs = "windowsEventsConfigs" + KafkaConfigs = "KafkaConfigs" ) type targetManager interface { @@ -52,6 +55,7 @@ func NewTargetManagers( client api.EntryHandler, scrapeConfigs []scrapeconfig.Config, targetConfig *file.Config, + clientConfigs ...client.Config, ) (*TargetManagers, error) { var targetManagers []targetManager targetScrapeConfigs := make(map[string][]scrapeconfig.Config, 4) @@ -80,7 +84,8 @@ func NewTargetManagers( targetScrapeConfigs[PushScrapeConfigs] = append(targetScrapeConfigs[PushScrapeConfigs], cfg) case cfg.WindowsConfig != nil: targetScrapeConfigs[WindowsEventsConfigs] = append(targetScrapeConfigs[WindowsEventsConfigs], cfg) - + case cfg.KafkaConfig != nil: + targetScrapeConfigs[KafkaConfigs] = append(targetScrapeConfigs[KafkaConfigs], cfg) default: return nil, fmt.Errorf("no valid target scrape config defined for %q", cfg.JobName) } @@ -189,6 +194,13 @@ func NewTargetManagers( return nil, errors.Wrap(err, "failed to make windows target manager") } targetManagers = append(targetManagers, windowsTargetManager) + case KafkaConfigs: + kafkaTargetManager, err := kafka.NewTargetManager(reg, logger, client, scrapeConfigs) + if err != nil { + return nil, errors.Wrap(err, "failed to make kafka target manager") + } + targetManagers = append(targetManagers, kafkaTargetManager) + default: return nil, errors.New("unknown scrape config") } diff --git a/clients/pkg/promtail/targets/target/target.go b/clients/pkg/promtail/targets/target/target.go index 01f1ef03291f..e8e0e2c168ae 100644 --- a/clients/pkg/promtail/targets/target/target.go +++ b/clients/pkg/promtail/targets/target/target.go @@ -29,6 +29,9 @@ const ( // WindowsTargetType is a Windows event target WindowsTargetType = TargetType("WindowsEvent") + + // KafkaTargetType is a Kafka target + KafkaTargetType = TargetType("Kafka") ) // Target is a promtail scrape target diff --git a/clients/pkg/promtail/targets/windows/targetmanager.go b/clients/pkg/promtail/targets/windows/targetmanager.go index 2980ae93cc11..e281ddde5ad8 100644 --- a/clients/pkg/promtail/targets/windows/targetmanager.go +++ b/clients/pkg/promtail/targets/windows/targetmanager.go @@ -33,7 +33,7 @@ func (tm *TargetManager) Ready() bool { return false } // Stop stops the Windows target manager and all of its targets. func (tm *TargetManager) Stop() {} -// ActiveTargets returns the list of actuve Windows targets. +// ActiveTargets returns the list of active Windows targets. func (tm *TargetManager) ActiveTargets() map[string][]target.Target { return nil } // AllTargets returns the list of all targets. diff --git a/clients/pkg/promtail/targets/windows/targetmanager_windows.go b/clients/pkg/promtail/targets/windows/targetmanager_windows.go index 7eaf7418f507..78793b26c730 100644 --- a/clients/pkg/promtail/targets/windows/targetmanager_windows.go +++ b/clients/pkg/promtail/targets/windows/targetmanager_windows.go @@ -68,7 +68,7 @@ func (tm *TargetManager) Stop() { } } -// ActiveTargets returns the list of actuve Windows targets. +// ActiveTargets returns the list of active Windows targets. func (tm *TargetManager) ActiveTargets() map[string][]target.Target { result := make(map[string][]target.Target, len(tm.targets)) for k, v := range tm.targets { diff --git a/docs/sources/clients/promtail/configuration.md b/docs/sources/clients/promtail/configuration.md index b1e0050bf1d9..0fc11cf979b9 100644 --- a/docs/sources/clients/promtail/configuration.md +++ b/docs/sources/clients/promtail/configuration.md @@ -319,6 +319,12 @@ job_name: # Describes how to receive logs via the Loki push API, (e.g. from other Promtails or the Docker Logging Driver) [loki_push_api: ] +# Describes how to scrape logs from the Windows event logs. +[windows_events: ] + +# Describes how to fetch logs from Kafka via a Consumer group. +[kafka: ] + # Describes how to relabel targets to determine if they should # be processed. relabel_configs: @@ -336,12 +342,12 @@ file_sd_configs: # same host. kubernetes_sd_configs: - [] - -# Describes how to use the Consul Catalog API to discover services registered with the + +# Describes how to use the Consul Catalog API to discover services registered with the # consul cluster. consul_sd_configs: [ - ... ] - + # Describes how to use the Consul Agent API to discover services registered with the consul agent # running on the same host as Promtail. consulagent_sd_configs: @@ -870,7 +876,7 @@ You can add additional labels with the `labels` property. # Allows to exclude the user data of each windows event. [exclude_event_data: | default = false] -# Label map to add to every log line sent to the push API +# Label map to add to every log line read from the windows event log labels: [ : ... ] @@ -879,6 +885,70 @@ labels: [use_incoming_timestamp: | default = false] ``` +### kafka + +The `kafka` block configures Promtail to scrape logs from [Kafka](https://kafka.apache.org/) using a group consumer. + +The `brokers` should list available brokers to communicate with the Kafka cluster. Use multiple brokers when you want to increase availability. + +The `topics` is the list of topics Promtail will subscribe to. If a topic starts with `^` then a regular expression ([RE2](https://github.com/google/re2/wiki/Syntax)) is used to match topics. +For instance `^promtail-.*` will match the topic `promtail-dev` and `promtail-prod`. Topics are refreshed every 30 seconds, so if a new topic matches, it will be automatically added without requiring a Promtail restart. + +The `group_id` defined the unique consumer group id to use for consuming logs. Each log record published to a topic is delivered to one consumer instance within each subscribing consumer group. + +- If all promtail instances have the same consumer group, then the records will effectively be load balanced over the promtail instances. +- If all promtail instances have different consumer groups, then each record will be broadcast to all promtail instances. + +The `group_id` is useful if you want to effectively send the data to multiple loki instances and/or other sinks. + +The `assignor` configuration allow you to select the rebalancing strategy to use for the consumer group. +Rebalancing is the process where a group of consumer instances (belonging to the same group) co-ordinate to own a mutually exclusive set of partitions of topics that the group is subscribed to. + +- `range` the default, assigns partitions as ranges to consumer group members. +- `sticky` assigns partitions to members with an attempt to preserve earlier assignments +- `roundrobin` assigns partitions to members in alternating order. + +The `version` allows to select the kafka version required to connect to the cluster.(default to `2.2.1`) + +By default, timestamps are assigned by Promtail when the message is read, if you want to keep the actual message timestamp from Kafka you can set the `use_incoming_timestamp` to true. + +```yaml +# The list of brokers to connect to kafka (Required). +[brokers: | default = [""]] + +# The list of Kafka topics to consume (Required). +[topics: | default = [""]] + +# The Kafka consumer group id. +[group_id: | default = "promtail"] + +# The consumer group rebalancing strategy to use. (e.g `sticky`, `roundrobin` or `range`) +[assignor: | default = "range"] + +# Kafka version to connect to. +[version: | default = "2.2.1"] + + +# Label map to add to every log line read from kafka +labels: + [ : ... ] + +# If Promtail should pass on the timestamp from the incoming log or not. +# When false Promtail will assign the current timestamp to the log when it was processed +[use_incoming_timestamp: | default = false] +``` + +#### Available Labels + +The list of labels below are discovered when consuming kafka: + +- `__meta_kafka_topic`: The current topic for where the message has been read. +- `__meta_kafka_partition`: The partition id where the message has been read. +- `__meta_kafka_member_id`: the consumer group member id. +- `__meta_kafka_group_id`: the consumer group id. + +To keep discovered labels to your logs use the [relabel_configs](#relabel_configs) section. + ### relabel_configs Relabeling is a powerful tool to dynamically rewrite the label set of a target @@ -1274,7 +1344,7 @@ way to filter services or nodes for a service based on arbitrary labels. For users with thousands of services it can be more efficient to use the Consul API directly which has basic support for filtering nodes (currently by node metadata and a single tag). - + ### consulagent_sd_config Consul Agent SD configurations allow retrieving scrape targets from [Consul's](https://www.consul.io) diff --git a/docs/sources/clients/promtail/scraping.md b/docs/sources/clients/promtail/scraping.md index ccca42445afd..065a743af61c 100644 --- a/docs/sources/clients/promtail/scraping.md +++ b/docs/sources/clients/promtail/scraping.md @@ -182,7 +182,7 @@ You can relabel default labels via [Relabeling](#relabeling) if required. Providing a path to a bookmark is mandatory, it will be used to persist the last event processed and allow resuming the target without skipping logs. -see the [configuration](./configuration.md#windows_events) section for more information. +see the [configuration](https://grafana.com/docs/loki/latest/clients/promtail/configuration/#windows_events) section for more information. ## Gcplog scraping Promtail supports scraping cloud resource logs(say GCS bucket logs, Load Balancer logs, Kubernetes Cluster logs) from GCP. @@ -207,7 +207,7 @@ Here `project_id` and `subscription` are the only required fields. Before using `gcplog` target, GCP should be [configured](../gcplog-cloud) with pubsub subscription to receive logs from. -It also support `relabeling` and `pipeline` stages just like other targets. +It also supports `relabeling` and `pipeline` stages just like other targets. When Promtail receives GCP logs the labels that are set on the GCP resources are available as internal labels. Like in the example above, the `__project_id` label from a GCP resource was transformed into a label called `project` through `relabel_configs`. See [Relabeling](#relabeling) for more information. @@ -271,6 +271,42 @@ destination d_loki { action(type="omfwd" protocol="tcp" port="" Template="RSYSLOG_SyslogProtocol23Format" TCP_Framing="octet-counted") ``` + +## Kafka + +Promtail supports reading message from Kafka using a consumer group. +The Kafka targets can be configured using the `kafka` stanza: + +```yaml +scrape_configs: +- job_name: kafka + kafka: + brokers: + - my-kafka-0.org:50705 + - my-kafka-1.org:50705 + topics: + - ^promtail.* + - some_fixed_topic + labels: + job: kafka + relabel_configs: + - action: replace + source_labels: + - __meta_kafka_topic + target_label: topic + - action: replace + source_labels: + - __meta_kafka_partition + target_label: partition + - action: replace + source_labels: + - __meta_kafka_group_id + target_label: group +``` + +Only the `brokers` and `topics` is required. +see the [configuration](https://grafana.com/docs/loki/latest/clients/promtail/configuration/#kafka) section for more information. + ## Relabeling Each `scrape_configs` entry can contain a `relabel_configs` stanza. diff --git a/go.mod b/go.mod index 9568996b4eb5..2f8e18fcc940 100644 --- a/go.mod +++ b/go.mod @@ -10,6 +10,7 @@ require ( github.com/Azure/azure-storage-blob-go v0.13.0 github.com/Masterminds/sprig/v3 v3.2.2 github.com/NYTimes/gziphandler v1.1.1 + github.com/Shopify/sarama v1.30.0 github.com/Workiva/go-datastructures v1.0.53 github.com/alicebob/miniredis/v2 v2.14.3 github.com/aws/aws-sdk-go v1.40.37 @@ -55,7 +56,7 @@ require ( github.com/jmespath/go-jmespath v0.4.0 github.com/joncrlsn/dque v2.2.1-0.20200515025108-956d14155fa2+incompatible github.com/json-iterator/go v1.1.11 - github.com/klauspost/compress v1.13.1 + github.com/klauspost/compress v1.13.6 github.com/klauspost/pgzip v1.2.5 github.com/minio/minio-go/v7 v7.0.10 github.com/mitchellh/mapstructure v1.4.1 @@ -87,8 +88,8 @@ require ( go.etcd.io/bbolt v1.3.6 go.uber.org/atomic v1.9.0 go.uber.org/goleak v1.1.10 - golang.org/x/crypto v0.0.0-20210616213533-5ff15b29337e - golang.org/x/net v0.0.0-20210903162142-ad29c8ab022f + golang.org/x/crypto v0.0.0-20210921155107-089bfa567519 + golang.org/x/net v0.0.0-20211101193420-4a448f8816b3 golang.org/x/sync v0.0.0-20210220032951-036812b2e83c golang.org/x/sys v0.0.0-20210908233432-aa78b53d3365 golang.org/x/time v0.0.0-20210723032227-1f47c861a9ac @@ -145,6 +146,9 @@ require ( github.com/docker/go-connections v0.4.0 // indirect github.com/docker/go-metrics v0.0.1 // indirect github.com/docker/go-units v0.4.0 // indirect + github.com/eapache/go-resiliency v1.2.0 // indirect + github.com/eapache/go-xerial-snappy v0.0.0-20180814174437-776d5712da21 // indirect + github.com/eapache/queue v1.1.0 // indirect github.com/edsrzf/mmap-go v1.0.0 // indirect github.com/felixge/httpsnoop v1.0.1 // indirect github.com/fsnotify/fsnotify v1.4.9 // indirect @@ -185,9 +189,15 @@ require ( github.com/hashicorp/go-multierror v1.1.0 // indirect github.com/hashicorp/go-rootcerts v1.0.2 // indirect github.com/hashicorp/go-sockaddr v1.0.2 // indirect + github.com/hashicorp/go-uuid v1.0.2 // indirect github.com/hashicorp/memberlist v0.2.4 // indirect github.com/hashicorp/serf v0.9.5 // indirect github.com/huandu/xstrings v1.3.1 // indirect + github.com/jcmturner/aescts/v2 v2.0.0 // indirect + github.com/jcmturner/dnsutils/v2 v2.0.0 // indirect + github.com/jcmturner/gofork v1.0.0 // indirect + github.com/jcmturner/gokrb5/v8 v8.4.2 // indirect + github.com/jcmturner/rpc/v2 v2.0.3 // indirect github.com/jessevdk/go-flags v1.5.0 // indirect github.com/josharian/intern v1.0.0 // indirect github.com/jpillora/backoff v1.0.0 // indirect @@ -213,12 +223,14 @@ require ( github.com/morikuni/aec v1.0.0 // indirect github.com/opencontainers/go-digest v1.0.0 // indirect github.com/opencontainers/image-spec v1.0.1 // indirect + github.com/pierrec/lz4 v2.6.1+incompatible // indirect github.com/pmezard/go-difflib v1.0.0 // indirect github.com/prometheus/alertmanager v0.23.1-0.20210914172521-e35efbddb66a // indirect github.com/prometheus/common/sigv4 v0.1.0 // indirect github.com/prometheus/exporter-toolkit v0.6.1 // indirect github.com/prometheus/node_exporter v1.0.0-rc.0.0.20200428091818-01054558c289 // indirect github.com/prometheus/procfs v0.6.0 // indirect + github.com/rcrowley/go-metrics v0.0.0-20201227073835-cf1acfcdf475 // indirect github.com/rs/cors v1.8.0 // indirect github.com/rs/xid v1.2.1 // indirect github.com/sean-/seed v0.0.0-20170313163322-e2103e2c3529 // indirect @@ -245,7 +257,7 @@ require ( golang.org/x/lint v0.0.0-20210508222113-6edffad5e616 // indirect golang.org/x/oauth2 v0.0.0-20210819190943-2bc19b11175f // indirect golang.org/x/term v0.0.0-20210220032956-6a3ed077a48d // indirect - golang.org/x/text v0.3.6 // indirect + golang.org/x/text v0.3.7 // indirect golang.org/x/tools v0.1.5 // indirect google.golang.org/appengine v1.6.7 // indirect google.golang.org/genproto v0.0.0-20210921142501-181ce0d877f6 // indirect diff --git a/go.sum b/go.sum index 1f1782a9f126..02529b93e9cc 100644 --- a/go.sum +++ b/go.sum @@ -219,7 +219,12 @@ github.com/SermoDigital/jose v0.0.0-20180104203859-803625baeddc/go.mod h1:ARgCUh github.com/Shopify/logrus-bugsnag v0.0.0-20171204204709-577dee27f20d/go.mod h1:HI8ITrYtUY+O+ZhtlqUnD8+KwNPOyugEhfP9fdUIaEQ= github.com/Shopify/sarama v1.19.0/go.mod h1:FVkBWblsNy7DGZRfXLU0O9RCGt5g3g3yEuWXgklEdEo= github.com/Shopify/sarama v1.27.1/go.mod h1:g5s5osgELxgM+Md9Qni9rzo7Rbt+vvFQI4bt/Mc93II= +github.com/Shopify/sarama v1.30.0 h1:TOZL6r37xJBDEMLx4yjB77jxbZYXPaDow08TSK6vIL0= +github.com/Shopify/sarama v1.30.0/go.mod h1:zujlQQx1kzHsh4jfV1USnptCQrHAEZ2Hk8fTKCulPVs= +github.com/Shopify/toxiproxy v2.1.4+incompatible h1:TKdv8HiTLgE5wdJuEML90aBgNWsokNbMijUGhmcoBJc= github.com/Shopify/toxiproxy v2.1.4+incompatible/go.mod h1:OXgGpZ6Cli1/URJOF1DMxUHB2q5Ap20/P/eIdh4G0pI= +github.com/Shopify/toxiproxy/v2 v2.1.6-0.20210914104332-15ea381dcdae h1:ePgznFqEG1v3AjMklnK8H7BSc++FDSo7xfK9K7Af+0Y= +github.com/Shopify/toxiproxy/v2 v2.1.6-0.20210914104332-15ea381dcdae/go.mod h1:/cvHQkZ1fst0EmZnA5dFtiQdWCNCFYzb+uE2vqVgvx0= github.com/StackExchange/wmi v0.0.0-20180116203802-5d049714c4a6/go.mod h1:3eOhrUMpNV+6aFIbp5/iudMxNCF27Vw2OZgy4xEx0Fg= github.com/VividCortex/gohistogram v1.0.0/go.mod h1:Pf5mBqqDxYaXu3hDrrU+w6nw50o/4+TcAqDqk/vUH7g= github.com/Workiva/go-datastructures v1.0.53 h1:J6Y/52yX10Xc5JjXmGtWoSSxs3mZnGSaq37xZZh7Yig= @@ -640,8 +645,11 @@ github.com/dustin/go-humanize v1.0.0 h1:VSnTsYCnlFHaM2/igO1h6X3HA71jcobQuxemgkq4 github.com/dustin/go-humanize v1.0.0/go.mod h1:HtrtbFcZ19U5GC7JDqmcUSB87Iq5E25KnS6fMYU6eOk= github.com/dvyukov/go-fuzz v0.0.0-20210103155950-6a8e9d1f2415/go.mod h1:11Gm+ccJnvAhCNLlf5+cS9KjtbaD5I5zaZpFMsTHWTw= github.com/eapache/go-resiliency v1.1.0/go.mod h1:kFI+JgMyC7bLPUVY133qvEBtVayf5mFgVsvEsIPBvNs= +github.com/eapache/go-resiliency v1.2.0 h1:v7g92e/KSN71Rq7vSThKaWIq68fL4YHvWyiUKorFR1Q= github.com/eapache/go-resiliency v1.2.0/go.mod h1:kFI+JgMyC7bLPUVY133qvEBtVayf5mFgVsvEsIPBvNs= +github.com/eapache/go-xerial-snappy v0.0.0-20180814174437-776d5712da21 h1:YEetp8/yCZMuEPMUDHG0CW/brkkEp8mzqk2+ODEitlw= github.com/eapache/go-xerial-snappy v0.0.0-20180814174437-776d5712da21/go.mod h1:+020luEh2TKB4/GOp8oxxtq0Daoen/Cii55CzbTV6DU= +github.com/eapache/queue v1.1.0 h1:YOEu7KNc61ntiQlcEeUIoDTJ2o8mQznoNvUhiigpIqc= github.com/eapache/queue v1.1.0/go.mod h1:6eCeP0CKFpHLu8blIFXhExK/dRa7WDZfr6jVFPTqq+I= github.com/eclipse/paho.mqtt.golang v1.2.0/go.mod h1:H9keYFcgq3Qr5OUJm/JZI/i6U7joQ8SYLhZwfeOo6Ts= github.com/edsrzf/mmap-go v0.0.0-20170320065105-0bce6a688712/go.mod h1:YO35OhQPt3KJa3ryjFM5Bs14WD66h8eGKpfaBNrHW5M= @@ -695,11 +703,13 @@ github.com/fluent/fluent-bit-go v0.0.0-20190925192703-ea13c021720c/go.mod h1:WQX github.com/fogleman/gg v1.2.1-0.20190220221249-0403632d5b90/go.mod h1:R/bRT+9gY/C5z7JzPU0zXsXHKM4/ayA+zqcVNZzPa1k= github.com/form3tech-oss/jwt-go v3.2.2+incompatible/go.mod h1:pbq4aXjuKjdthFRnoDwaVPLA+WlJuPGy+QneDUgJi2k= github.com/form3tech-oss/jwt-go v3.2.3+incompatible/go.mod h1:pbq4aXjuKjdthFRnoDwaVPLA+WlJuPGy+QneDUgJi2k= +github.com/fortytw2/leaktest v1.3.0 h1:u8491cBMTQ8ft8aeV+adlcytMZylmA5nnwwkRZjI8vw= github.com/fortytw2/leaktest v1.3.0/go.mod h1:jDsjWgpAGjm2CA7WthBh/CdZYEPF31XHquHwclZch5g= github.com/foxcpp/go-mockdns v0.0.0-20201212160233-ede2f9158d15/go.mod h1:tPg4cp4nseejPd+UKxtCVQ2hUxNTZ7qQZJa7CLriIeo= github.com/franela/goblin v0.0.0-20200105215937-c9ffbefa60db/go.mod h1:7dvUGVsVBjqR7JHJk0brhHOZYGmfBYOrK0ZhYMEtBr4= github.com/franela/goreq v0.0.0-20171204163338-bcd34c9993f8/go.mod h1:ZhphrRTfi2rbfLwlschooIH4+wKKDR4Pdxhh+TRoA20= github.com/frankban/quicktest v1.10.2/go.mod h1:K+q6oSqb0W0Ininfk863uOk1lMy69l/P6txr3mVT54s= +github.com/frankban/quicktest v1.11.3 h1:8sXhOn0uLys67V8EsXLc6eszDs8VXWxL3iRvebPhedY= github.com/frankban/quicktest v1.11.3/go.mod h1:wRf/ReqHper53s+kmmSZizM8NamnL3IM0I9ntUbOk+k= github.com/fsnotify/fsnotify v1.4.7/go.mod h1:jwhsz4b93w/PPRr/qN1Yymfu8t87LnFCMoQvtojpjFo= github.com/fsnotify/fsnotify v1.4.9 h1:hsms1Qyu0jgnwNXIxa+/V/PDsU6CfLf6CNO8H7IWoS4= @@ -1110,6 +1120,8 @@ github.com/gorilla/mux v1.7.2/go.mod h1:1lud6UwP+6orDFRuTfBEV8e9/aOM/c4fVVCaMa2z github.com/gorilla/mux v1.7.3/go.mod h1:1lud6UwP+6orDFRuTfBEV8e9/aOM/c4fVVCaMa2zaAs= github.com/gorilla/mux v1.8.0 h1:i40aqfkR1h2SlN9hojwV5ZA91wcXFOvkdNIeFDP5koI= github.com/gorilla/mux v1.8.0/go.mod h1:DVbg23sWSpFRCP0SfiEN6jmj59UnW/n46BH5rLB71So= +github.com/gorilla/securecookie v1.1.1/go.mod h1:ra0sb63/xPlUeL+yeDciTfxMRAA+MP+HVt/4epWDjd4= +github.com/gorilla/sessions v1.2.1/go.mod h1:dk2InVEVJ0sfLlnXv9EAgkf6ecYs/i80K/zI+bUmuGM= github.com/gorilla/websocket v0.0.0-20170926233335-4201258b820c/go.mod h1:E7qHFY5m1UJ88s3WnNqhKjPHQ0heANvMoAMk2YaljkQ= github.com/gorilla/websocket v1.4.0/go.mod h1:E7qHFY5m1UJ88s3WnNqhKjPHQ0heANvMoAMk2YaljkQ= github.com/gorilla/websocket v1.4.2 h1:+/TMaTYc4QFitKJxsQ7Yye35DkWvkdLcvGKqM+x0Ufc= @@ -1314,7 +1326,18 @@ github.com/jackc/fake v0.0.0-20150926172116-812a484cc733/go.mod h1:WrMFNQdiFJ80s github.com/jackc/pgx v3.2.0+incompatible/go.mod h1:0ZGrqGqkRlliWnWB4zKnWtjbSWbGkVEFm4TeybAXq+I= github.com/jackc/pgx v3.6.0+incompatible/go.mod h1:0ZGrqGqkRlliWnWB4zKnWtjbSWbGkVEFm4TeybAXq+I= github.com/jarcoal/httpmock v0.0.0-20180424175123-9c70cfe4a1da/go.mod h1:ks+b9deReOc7jgqp+e7LuFiCBH6Rm5hL32cLcEAArb4= +github.com/jcmturner/aescts/v2 v2.0.0 h1:9YKLH6ey7H4eDBXW8khjYslgyqG2xZikXP0EQFKrle8= +github.com/jcmturner/aescts/v2 v2.0.0/go.mod h1:AiaICIRyfYg35RUkr8yESTqvSy7csK90qZ5xfvvsoNs= +github.com/jcmturner/dnsutils/v2 v2.0.0 h1:lltnkeZGL0wILNvrNiVCR6Ro5PGU/SeBvVO/8c/iPbo= +github.com/jcmturner/dnsutils/v2 v2.0.0/go.mod h1:b0TnjGOvI/n42bZa+hmXL+kFJZsFT7G4t3HTlQ184QM= +github.com/jcmturner/gofork v1.0.0 h1:J7uCkflzTEhUZ64xqKnkDxq3kzc96ajM1Gli5ktUem8= github.com/jcmturner/gofork v1.0.0/go.mod h1:MK8+TM0La+2rjBD4jE12Kj1pCCxK7d2LK/UM3ncEo0o= +github.com/jcmturner/goidentity/v6 v6.0.1 h1:VKnZd2oEIMorCTsFBnJWbExfNN7yZr3EhJAxwOkZg6o= +github.com/jcmturner/goidentity/v6 v6.0.1/go.mod h1:X1YW3bgtvwAXju7V3LCIMpY0Gbxyjn/mY9zx4tFonSg= +github.com/jcmturner/gokrb5/v8 v8.4.2 h1:6ZIM6b/JJN0X8UM43ZOM6Z4SJzla+a/u7scXFJzodkA= +github.com/jcmturner/gokrb5/v8 v8.4.2/go.mod h1:sb+Xq/fTY5yktf/VxLsE3wlfPqQjp0aWNYyvBVK62bc= +github.com/jcmturner/rpc/v2 v2.0.3 h1:7FXXj8Ti1IaVFpSAziCZWNzbNuZmnvw/i6CqLNdWfZY= +github.com/jcmturner/rpc/v2 v2.0.3/go.mod h1:VUJYCIDm3PVOEHw8sgt091/20OJjskO/YJki3ELg/Hc= github.com/jefferai/jsonx v0.0.0-20160721235117-9cc31c3135ee/go.mod h1:N0t2vlmpe8nyZB5ouIbJQPDSR+mH6oe7xHB9VZHSUzM= github.com/jessevdk/go-flags v0.0.0-20180331124232-1c38ed7ad0cc/go.mod h1:4FA24M0QyGHXBuZZK/XkWh8h0e1EYbRYJSGM75WSRxI= github.com/jessevdk/go-flags v1.4.0/go.mod h1:4FA24M0QyGHXBuZZK/XkWh8h0e1EYbRYJSGM75WSRxI= @@ -1381,8 +1404,9 @@ github.com/klauspost/compress v1.11.0/go.mod h1:aoV0uJVorq1K+umq18yTdKaF57EivdYs github.com/klauspost/compress v1.11.3/go.mod h1:aoV0uJVorq1K+umq18yTdKaF57EivdYsUV+/s2qKfXs= github.com/klauspost/compress v1.11.12/go.mod h1:aoV0uJVorq1K+umq18yTdKaF57EivdYsUV+/s2qKfXs= github.com/klauspost/compress v1.11.13/go.mod h1:aoV0uJVorq1K+umq18yTdKaF57EivdYsUV+/s2qKfXs= -github.com/klauspost/compress v1.13.1 h1:wXr2uRxZTJXHLly6qhJabee5JqIhTRoLBhDOA74hDEQ= github.com/klauspost/compress v1.13.1/go.mod h1:8dP1Hq4DHOhN9w426knH3Rhby4rFm6D8eO+e+Dq5Gzg= +github.com/klauspost/compress v1.13.6 h1:P76CopJELS0TiO2mebmnzgWaajssP/EszplttgQxcgc= +github.com/klauspost/compress v1.13.6/go.mod h1:/3/Vjq9QcHkK5uEr5lBEmyoZ1iFhe47etQ6QUkpK6sk= github.com/klauspost/cpuid v0.0.0-20170728055534-ae7887de9fa5/go.mod h1:Pj4uuM528wm8OyEC2QMXAi2YiTZ96dNQPGgoMS4s3ek= github.com/klauspost/cpuid v1.2.3/go.mod h1:Pj4uuM528wm8OyEC2QMXAi2YiTZ96dNQPGgoMS4s3ek= github.com/klauspost/cpuid v1.3.1 h1:5JNjFYYQrZeKRJ0734q51WCEEn2huer72Dc7K+R/b6s= @@ -1604,7 +1628,6 @@ github.com/ncw/swift v1.0.52 h1:ACF3JufDGgeKp/9mrDgQlEgS8kRYC4XKcuzj/8EJjQU= github.com/ncw/swift v1.0.52/go.mod h1:23YIA4yWVnGwv2dQlN4bB7egfYX6YLn0Yo/S6zZO/ZM= github.com/newrelic/newrelic-telemetry-sdk-go v0.2.0/go.mod h1:G9MqE/cHGv3Hx3qpYhfuyFUsGx2DpVcGi1iJIqTg+JQ= github.com/nicolai86/scaleway-sdk v1.10.2-0.20180628010248-798f60e20bb2/go.mod h1:TLb2Sg7HQcgGdloNxkrmtgDNR9uVYF3lfdFIN4Ro6Sk= -github.com/niemeyer/pretty v0.0.0-20200227124842-a10e7caefd8e h1:fD57ERR4JtEqsWbfPhv4DMiApHyliiK5xCTNVSPiaAs= github.com/niemeyer/pretty v0.0.0-20200227124842-a10e7caefd8e/go.mod h1:zD1mROLANZcx1PVRCS0qkT7pwLkGfwJo4zjcN/Tysno= github.com/nsqio/go-nsq v1.0.7/go.mod h1:XP5zaUs3pqf+Q71EqUJs3HYfBIqfK6G83WQMdNN+Ito= github.com/nxadm/tail v1.4.4/go.mod h1:kenIhsEOeOJmVchQTgglprH7qJGnHDVpk1VPCcaMI8A= @@ -1711,8 +1734,9 @@ github.com/philhofer/fwd v1.0.0/go.mod h1:gk3iGcWd9+svBvR0sR+KPcfE+RNWozjowpeBVG github.com/philhofer/fwd v1.1.1/go.mod h1:gk3iGcWd9+svBvR0sR+KPcfE+RNWozjowpeBVG3ZVNU= github.com/pierrec/lz4 v1.0.2-0.20190131084431-473cd7ce01a1/go.mod h1:3/3N9NVKO0jef7pBehbT1qWhCMrIgbYNnFAZCqQ5LRc= github.com/pierrec/lz4 v2.0.5+incompatible/go.mod h1:pdkljMzZIN41W+lC3N2tnIh5sFi+IEE17M5jbnwPHcY= -github.com/pierrec/lz4 v2.5.2+incompatible h1:WCjObylUIOlKy/+7Abdn34TLIkXiA4UWUMhxq9m9ZXI= github.com/pierrec/lz4 v2.5.2+incompatible/go.mod h1:pdkljMzZIN41W+lC3N2tnIh5sFi+IEE17M5jbnwPHcY= +github.com/pierrec/lz4 v2.6.1+incompatible h1:9UY3+iC23yxF0UfGaYrGplQ+79Rg+h/q9FV9ix19jjM= +github.com/pierrec/lz4 v2.6.1+incompatible/go.mod h1:pdkljMzZIN41W+lC3N2tnIh5sFi+IEE17M5jbnwPHcY= github.com/pierrec/lz4/v4 v4.1.7 h1:UDV9geJWhFIufAliH7HQlz9wP3JA0t748w+RwbWMLow= github.com/pierrec/lz4/v4 v4.1.7/go.mod h1:gZWDp/Ze/IJXGXf23ltt2EXimqmTUXEy0GFuRQyBid4= github.com/pkg/browser v0.0.0-20180916011732-0a3d74bf9ce4/go.mod h1:4OwLy04Bl9Ef3GJJCoec+30X3LQs/0/m4HFRt/2LUSA= @@ -1855,6 +1879,8 @@ github.com/prometheus/tsdb v0.7.1/go.mod h1:qhTCs0VvXwvX/y3TZrWD7rabWM+ijKTux40T github.com/rafaeljusto/redigomock v0.0.0-20190202135759-257e089e14a1/go.mod h1:JaY6n2sDr+z2WTsXkOmNRUfDy6FN0L6Nk7x06ndm4tY= github.com/rcrowley/go-metrics v0.0.0-20181016184325-3113b8401b8a/go.mod h1:bCqnVzQkZxMG4s8nGwiZ5l3QUCyqpo9Y+/ZMZ9VjZe4= github.com/rcrowley/go-metrics v0.0.0-20200313005456-10cdbea86bc0/go.mod h1:bCqnVzQkZxMG4s8nGwiZ5l3QUCyqpo9Y+/ZMZ9VjZe4= +github.com/rcrowley/go-metrics v0.0.0-20201227073835-cf1acfcdf475 h1:N/ElC8H3+5XpJzTSTfLsJV/mx9Q9g7kxmchpfZyxgzM= +github.com/rcrowley/go-metrics v0.0.0-20201227073835-cf1acfcdf475/go.mod h1:bCqnVzQkZxMG4s8nGwiZ5l3QUCyqpo9Y+/ZMZ9VjZe4= github.com/remyoudompheng/bigfft v0.0.0-20200410134404-eec4a21b6bb0/go.mod h1:qqbHyh8v60DhA7CoWK5oRCqLrMHRGoxYCSS9EjAz6Eo= github.com/renier/xmlrpc v0.0.0-20170708154548-ce4a1a486c03/go.mod h1:gRAiPF5C5Nd0eyyRdqIu9qTiFSoZzpTq727b5B8fkkU= github.com/retailnext/hllpp v1.0.1-0.20180308014038-101a6d2f8b52/go.mod h1:RDpi1RftBQPUCDRw6SmxeaREsAaRKnOclghuzp/WRzc= @@ -2060,6 +2086,7 @@ github.com/urfave/cli v0.0.0-20171014202726-7bc6a0acffa5/go.mod h1:70zkFmudgCuE/ github.com/urfave/cli v1.20.0/go.mod h1:70zkFmudgCuE/ngEzBv17Jvp/497gISqfk5gWijbERA= github.com/urfave/cli v1.22.1/go.mod h1:Gos4lmkARVdJ6EkW0WaNv/tZAAMe9V7XWyB60NtXRu0= github.com/urfave/cli v1.22.2/go.mod h1:Gos4lmkARVdJ6EkW0WaNv/tZAAMe9V7XWyB60NtXRu0= +github.com/urfave/cli/v2 v2.3.0/go.mod h1:LJmUH05zAU44vOAcrfzZQKsZbVcdbOG8rtL3/XcUArI= github.com/valyala/bytebufferpool v1.0.0/go.mod h1:6bBcMArwyJ5K/AmCkWv1jt77kVWyCJ6HpOuEn7z0Csc= github.com/valyala/fasttemplate v1.0.1/go.mod h1:UQGH1tvbgY+Nz5t2n7tXsz52dQxojPUpymEIMZ47gx8= github.com/valyala/fasttemplate v1.2.1/go.mod h1:KHLXt3tVN2HBp8eijSv/kGJopbvo7S+qRAEEKiv+SiQ= @@ -2265,14 +2292,17 @@ golang.org/x/crypto v0.0.0-20200709230013-948cd5f35899/go.mod h1:LzIPMQfyMNhhGPh golang.org/x/crypto v0.0.0-20200728195943-123391ffb6de/go.mod h1:LzIPMQfyMNhhGPhUkYOs5KpL4U8rLKemX1yGLhDgUto= golang.org/x/crypto v0.0.0-20200820211705-5c72a883971a/go.mod h1:LzIPMQfyMNhhGPhUkYOs5KpL4U8rLKemX1yGLhDgUto= golang.org/x/crypto v0.0.0-20201002170205-7f63de1d35b0/go.mod h1:LzIPMQfyMNhhGPhUkYOs5KpL4U8rLKemX1yGLhDgUto= +golang.org/x/crypto v0.0.0-20201112155050-0c6587e931a9/go.mod h1:LzIPMQfyMNhhGPhUkYOs5KpL4U8rLKemX1yGLhDgUto= golang.org/x/crypto v0.0.0-20201208171446-5f87f3452ae9/go.mod h1:jdWPYTVW3xRLrWPugEBEK3UY2ZEsg3UU495nc5E+M+I= golang.org/x/crypto v0.0.0-20201221181555-eec23a3978ad/go.mod h1:jdWPYTVW3xRLrWPugEBEK3UY2ZEsg3UU495nc5E+M+I= golang.org/x/crypto v0.0.0-20210220033148-5ea612d1eb83/go.mod h1:jdWPYTVW3xRLrWPugEBEK3UY2ZEsg3UU495nc5E+M+I= golang.org/x/crypto v0.0.0-20210314154223-e6e6c4f2bb5b/go.mod h1:T9bdIzuCu7OtxOm1hfPfRQxPLYneinmdGuTeoZ9dtd4= golang.org/x/crypto v0.0.0-20210322153248-0c34fe9e7dc2/go.mod h1:T9bdIzuCu7OtxOm1hfPfRQxPLYneinmdGuTeoZ9dtd4= golang.org/x/crypto v0.0.0-20210513164829-c07d793c2f9a/go.mod h1:P+XmwS30IXTQdn5tA2iutPOUgjI07+tq3H3K9MVA1s8= -golang.org/x/crypto v0.0.0-20210616213533-5ff15b29337e h1:gsTQYXdTw2Gq7RBsWvlQ91b+aEQ6bXFUngBGuR8sPpI= golang.org/x/crypto v0.0.0-20210616213533-5ff15b29337e/go.mod h1:GvvjBRRGRdwPK5ydBHafDWAxML/pGHZbMvKqRZ5+Abc= +golang.org/x/crypto v0.0.0-20210920023735-84f357641f63/go.mod h1:GvvjBRRGRdwPK5ydBHafDWAxML/pGHZbMvKqRZ5+Abc= +golang.org/x/crypto v0.0.0-20210921155107-089bfa567519 h1:7I4JAnoQBe7ZtJcBaYHi5UtiO8tQHbUSXxL+pnGRANg= +golang.org/x/crypto v0.0.0-20210921155107-089bfa567519/go.mod h1:GvvjBRRGRdwPK5ydBHafDWAxML/pGHZbMvKqRZ5+Abc= golang.org/x/exp v0.0.0-20180321215751-8460e604b9de/go.mod h1:CJ0aWSM057203Lf6IL+f9T1iT9GByDxfZKAQTCR3kQA= golang.org/x/exp v0.0.0-20180807140117-3d87b88a115f/go.mod h1:CJ0aWSM057203Lf6IL+f9T1iT9GByDxfZKAQTCR3kQA= golang.org/x/exp v0.0.0-20190121172915-509febef88a4/go.mod h1:CJ0aWSM057203Lf6IL+f9T1iT9GByDxfZKAQTCR3kQA= @@ -2385,6 +2415,7 @@ golang.org/x/net v0.0.0-20210224082022-3d97a244fca7/go.mod h1:m0MpNAwzfU5UDzcl9v golang.org/x/net v0.0.0-20210226172049-e18ecbb05110/go.mod h1:m0MpNAwzfU5UDzcl9v0D8zg8gWTRqZa9RBIspLL5mdg= golang.org/x/net v0.0.0-20210316092652-d523dce5a7f4/go.mod h1:RBQZq4jEuRlivfhVLdyRGr576XBO4/greRjx4P4O3yc= golang.org/x/net v0.0.0-20210324051636-2c4c8ecb7826/go.mod h1:RBQZq4jEuRlivfhVLdyRGr576XBO4/greRjx4P4O3yc= +golang.org/x/net v0.0.0-20210324051636-2c4c8ecb7826/go.mod h1:RBQZq4jEuRlivfhVLdyRGr576XBO4/greRjx4P4O3yc= golang.org/x/net v0.0.0-20210405180319-a5a99cb37ef4/go.mod h1:p54w0d4576C0XHj96bSt6lcn1PtDYWL6XObtHCRCNQM= golang.org/x/net v0.0.0-20210421230115-4e50805a0758/go.mod h1:72T/g9IO56b78aLF+1Kcs5dz7/ng1VjMUvfKvpfy+jM= golang.org/x/net v0.0.0-20210428140749-89ef3d95e781/go.mod h1:OJAsFXCWl8Ukc7SiCT/9KSuxbyM7479/AVlXFRxuMCk= @@ -2394,8 +2425,10 @@ golang.org/x/net v0.0.0-20210525063256-abc453219eb5/go.mod h1:9nx3DQGgdP8bBQD5qx golang.org/x/net v0.0.0-20210610132358-84b48f89b13b/go.mod h1:9nx3DQGgdP8bBQD5qxJ1jj9UTztislL4KSBs9R2vV5Y= golang.org/x/net v0.0.0-20210614182718-04defd469f4e/go.mod h1:9nx3DQGgdP8bBQD5qxJ1jj9UTztislL4KSBs9R2vV5Y= golang.org/x/net v0.0.0-20210726213435-c6fcb2dbf985/go.mod h1:9nx3DQGgdP8bBQD5qxJ1jj9UTztislL4KSBs9R2vV5Y= -golang.org/x/net v0.0.0-20210903162142-ad29c8ab022f h1:w6wWR0H+nyVpbSAQbzVEIACVyr/h8l/BEkY6Sokc7Eg= golang.org/x/net v0.0.0-20210903162142-ad29c8ab022f/go.mod h1:9nx3DQGgdP8bBQD5qxJ1jj9UTztislL4KSBs9R2vV5Y= +golang.org/x/net v0.0.0-20210917221730-978cfadd31cf/go.mod h1:9nx3DQGgdP8bBQD5qxJ1jj9UTztislL4KSBs9R2vV5Y= +golang.org/x/net v0.0.0-20211101193420-4a448f8816b3 h1:VrJZAjbekhoRn7n5FBujY31gboH+iB3pdLxn3gE9FjU= +golang.org/x/net v0.0.0-20211101193420-4a448f8816b3/go.mod h1:9nx3DQGgdP8bBQD5qxJ1jj9UTztislL4KSBs9R2vV5Y= golang.org/x/oauth2 v0.0.0-20170807180024-9a379c6b3e95/go.mod h1:N/0e6XlmueqKjAGxoOufVs8QHGRruUQn6yWY3a++T0U= golang.org/x/oauth2 v0.0.0-20180821212333-d2e6202438be/go.mod h1:N/0e6XlmueqKjAGxoOufVs8QHGRruUQn6yWY3a++T0U= golang.org/x/oauth2 v0.0.0-20181106182150-f42d05182288/go.mod h1:N/0e6XlmueqKjAGxoOufVs8QHGRruUQn6yWY3a++T0U= @@ -2554,10 +2587,12 @@ golang.org/x/sys v0.0.0-20210314195730-07df6a141424/go.mod h1:h1NjWce9XRLGQEsW7w golang.org/x/sys v0.0.0-20210315160823-c6e025ad8005/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20210320140829-1e4c9ba3b0c4/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20210324051608-47abb6519492/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= +golang.org/x/sys v0.0.0-20210324051608-47abb6519492/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20210330210617-4fbd30eecc44/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20210403161142-5e06dd20ab57/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20210420072515-93ed5bcd2bfe/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20210423082822-04245dca01da/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= +golang.org/x/sys v0.0.0-20210423082822-04245dca01da/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20210503080704-8803ae5d1324/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20210503173754-0981d6026fa6/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= golang.org/x/sys v0.0.0-20210510120138-977fb7262007/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= @@ -2585,8 +2620,9 @@ golang.org/x/text v0.3.2/go.mod h1:bEr9sfX3Q8Zfm5fL9x+3itogRgK3+ptLWKqgva+5dAk= golang.org/x/text v0.3.3/go.mod h1:5Zoc/QRtKVWzQhOtBMvqHzDpF6irO9z98xDceosuGiQ= golang.org/x/text v0.3.4/go.mod h1:5Zoc/QRtKVWzQhOtBMvqHzDpF6irO9z98xDceosuGiQ= golang.org/x/text v0.3.5/go.mod h1:5Zoc/QRtKVWzQhOtBMvqHzDpF6irO9z98xDceosuGiQ= -golang.org/x/text v0.3.6 h1:aRYxNxv6iGQlyVaZmk6ZgYEDa+Jg18DxebPSrd6bg1M= golang.org/x/text v0.3.6/go.mod h1:5Zoc/QRtKVWzQhOtBMvqHzDpF6irO9z98xDceosuGiQ= +golang.org/x/text v0.3.7 h1:olpwvP2KacW1ZWvsR7uQhoyTYvKAupfQrRGBFM352Gk= +golang.org/x/text v0.3.7/go.mod h1:u+2+/6zg+i71rQMx5EYifcz6MCKuco9NR6JIITiCfzQ= golang.org/x/time v0.0.0-20180412165947-fbb02b2291d2/go.mod h1:tRJNPiyCQ0inRvYxbN9jk5I+vvW/OXSQhTDSoE431IQ= golang.org/x/time v0.0.0-20181108054448-85acf8d2951c/go.mod h1:tRJNPiyCQ0inRvYxbN9jk5I+vvW/OXSQhTDSoE431IQ= golang.org/x/time v0.0.0-20190308202827-9d24e82272b4/go.mod h1:tRJNPiyCQ0inRvYxbN9jk5I+vvW/OXSQhTDSoE431IQ= @@ -2903,8 +2939,9 @@ gopkg.in/check.v1 v1.0.0-20161208181325-20d25e280405/go.mod h1:Co6ibVJAznAaIkqp8 gopkg.in/check.v1 v1.0.0-20180628173108-788fd7840127/go.mod h1:Co6ibVJAznAaIkqp8huTwlJQCZ016jof/cbN4VW5Yz0= gopkg.in/check.v1 v1.0.0-20190902080502-41f04d3bba15/go.mod h1:Co6ibVJAznAaIkqp8huTwlJQCZ016jof/cbN4VW5Yz0= gopkg.in/check.v1 v1.0.0-20200227125254-8fa46927fb4f/go.mod h1:Co6ibVJAznAaIkqp8huTwlJQCZ016jof/cbN4VW5Yz0= -gopkg.in/check.v1 v1.0.0-20200902074654-038fdea0a05b h1:QRR6H1YWRnHb4Y/HeNFCTJLFVxaq6wH4YuVdsUOr75U= gopkg.in/check.v1 v1.0.0-20200902074654-038fdea0a05b/go.mod h1:Co6ibVJAznAaIkqp8huTwlJQCZ016jof/cbN4VW5Yz0= +gopkg.in/check.v1 v1.0.0-20201130134442-10cb98267c6c h1:Hei/4ADfdWqJk1ZMxUNpqntNwaWcugrBjAiHlqqRiVk= +gopkg.in/check.v1 v1.0.0-20201130134442-10cb98267c6c/go.mod h1:JHkPIbrfpd72SG/EVd6muEfDQjcINNoR0C8j2r3qZ4Q= gopkg.in/cheggaaa/pb.v1 v1.0.25/go.mod h1:V/YB90LKu/1FcN3WVnfiiE5oMCibMjukxqG/qStrOgw= gopkg.in/errgo.v2 v2.1.0/go.mod h1:hNsd1EY+bozCKY1Ytp96fpM3vjJbqLJn88ws8XvfDNI= gopkg.in/fatih/pool.v2 v2.0.0/go.mod h1:8xVGeu1/2jr2wm5V9SPuMht2H5AEmf5aFMGSQixtjTY= diff --git a/tools/kafka/Makefile b/tools/kafka/Makefile new file mode 100644 index 000000000000..a65da1eda610 --- /dev/null +++ b/tools/kafka/Makefile @@ -0,0 +1,34 @@ + +HOST_IP ?= host.docker.internal +TOPIC ?= promtail +RF ?= 1 +PARTS ?= 3 + +BROKER_LIST := $(shell ./broker-list.sh $(HOST_IP)) +DOCKER_RUN := docker run --rm -v /var/run/docker.sock:/var/run/docker.sock -e HOST_IP=$(HOST_IP) -i -t wurstmeister/kafka /bin/bash -c + +start-kafka: + docker-compose up -d + +stop-kafka: + docker-compose down + +print-brokers: + @echo $(BROKER_LIST) + +# TOPIC=bar make create-topic +create-topic: + @$(DOCKER_RUN) "kafka-topics.sh --create -bootstrap-server $(BROKER_LIST) --replication-factor $(RF) --partitions $(PARTS) --topic $(TOPIC)" + +# TOPIC=bar make describe-topic +describe-topic: + @$(DOCKER_RUN) "kafka-topics.sh --describe -bootstrap-server $(BROKER_LIST) --topic $(TOPIC)" + +list-topics: + @$(DOCKER_RUN) "kafka-topics.sh --list -bootstrap-server $(BROKER_LIST)" + +# TOPIC=bar make producer +producer: + @echo "Producing messages to topic $(TOPIC)... " + @echo "Write a message and press Enter" + @$(DOCKER_RUN) "kafka-console-producer.sh --broker-list $(BROKER_LIST) --topic $(TOPIC)" diff --git a/tools/kafka/README.md b/tools/kafka/README.md new file mode 100644 index 000000000000..26f8bf05c761 --- /dev/null +++ b/tools/kafka/README.md @@ -0,0 +1,104 @@ +# Kafka tools + +This folder contains tools for testing promtail <-> kafka integration. + +**Requirements:** + +- docker and docker-compose +- make +- go > 1.15 for consuming using code source of promtail + +## Running kafka locally + +To start kafka use `make start-kafka` this should start `kafka` and a `zookeeper` docker compose stack. +To discover available brokers you can use the `make print-brokers`. + +Finally to stop the compose stack use `make stop-kafka`. This will result in all topics being lost with their messages. + +## Working with Topic + +In Kafka before sending messages you need to create and select the topic you want to use for the exchange. + +To create a new topic use: `make create-topic`, by default the topic name will be `promtail` if you wish to overrides it you can use the `TOPIC` variable as shown below: + +```bash +TOPIC=new-topic make create-topic +``` + +To list all available topics use the `make list-topics` target. + +```bash + make list-topics +__consumer_offsets +promtail +``` + +To describe a topic use the `make describe-topic` target. + +```bash + TOPIC=new-topic make describe-topic + +Topic: new-topic PartitionCount: 3 ReplicationFactor: 1 Configs: segment.bytes=1073741824 + Topic: new-topic Partition: 0 Leader: 1001 Replicas: 1001 Isr: 1001 + Topic: new-topic Partition: 1 Leader: 1001 Replicas: 1001 Isr: 1001 + Topic: new-topic Partition: 2 Leader: 1001 Replicas: 1001 Isr: 1001 +``` + +As you can see by default each topic is assigned by default 3 partitions and a replication factor of 1. You can change those default using respectively `PARTS` and `RF` variable, for example the example below will create a topic with 4 partitions: + +```bash +PARTS=4 TOPIC=new-topic-part make create-topic +``` + +Partitions in kafka are used for scaling reads and writes. Each partitions is replicated on multiple machines. + +> WARNING: Order of messages is guaranteed on a single partition but not across a single topic. + +## Producing messages + +You can start sending message using `make producer` target, it will start reading newlines and send them to the default topic (`promtail`): + +```bash +TOPIC=new-topic make producer + +Producing messages to topic new-topic... +Write a message and press Enter +>hello world ! +> +``` + +## Consuming with promtail + +You can run promtail in `dry-run` mode and connect it to the local kafka to try out the integration. + +Before doing so make sure the brokers list (`make print-broker`) is correctly configured in your promtail configuration file, see for example the [kafka example](../../clients/cmd/promtail/promtail-kafka.yaml). + +```bash +go run ../../clients/cmd/promtail/main.go --dry-run --config.file ../../clients/cmd/promtail/promtail-kafka.yaml + +Clients configured: +---------------------- +url: http://localhost:3100/loki/api/v1/push +batchwait: 1s +batchsize: 1048576 +follow_redirects: false +backoff_config: + min_period: 500ms + max_period: 5m0s + max_retries: 10 +timeout: 10s +tenant_id: "" +stream_lag_labels: filename + +level=info ts=2021-11-02T10:44:14.137894Z caller=server.go:260 http=[::]:9080 grpc=[::]:59237 msg="server listening on addresses" +level=info ts=2021-11-02T10:44:14.138059Z caller=main.go:119 msg="Starting Promtail" version="(version=, branch=, revision=)" +level=info ts=2021-11-02T10:44:14.139308Z caller=target_syncer.go:133 msg="new topics received" topics=[promtail] +level=info ts=2021-11-02T10:44:14.139337Z caller=consumer.go:50 msg="starting consumer" topics=[promtail] +level=info ts=2021-11-02T10:44:14.153164Z caller=consumer.go:92 msg="consuming topic" details="member_id=sarama-8cfa484d-2a04-458a-a0c0-4506c7a0969f generation_id=5 topic=promtail partition=1 initial_offset=12" +level=info ts=2021-11-02T10:44:14.153673Z caller=consumer.go:92 msg="consuming topic" details="member_id=sarama-8cfa484d-2a04-458a-a0c0-4506c7a0969f generation_id=5 topic=promtail partition=0 initial_offset=13" +level=info ts=2021-11-02T10:44:14.153927Z caller=consumer.go:92 msg="consuming topic" details="member_id=sarama-8cfa484d-2a04-458a-a0c0-4506c7a0969f generation_id=5 topic=promtail partition=2 initial_offset=10" + +2021-11-02T11:47:08.849115+0100{group="some_group", job="kafka", partition="1", topic="promtail"} hello world ! +``` + +> Alternatively you can use the binary or docker version of promtail. diff --git a/tools/kafka/broker-list.sh b/tools/kafka/broker-list.sh new file mode 100755 index 000000000000..1e621efafe09 --- /dev/null +++ b/tools/kafka/broker-list.sh @@ -0,0 +1,5 @@ +#!/bin/bash + +CONTAINERS=$(docker ps | grep 9092 | awk '{print $1}') +BROKERS=$(for CONTAINER in ${CONTAINERS}; do docker port "${CONTAINER}" 9092 | sed -e "s/0.0.0.0:/$1:/g"; done) +echo "${BROKERS//$'\n'/,}" diff --git a/tools/kafka/docker-compose.yml b/tools/kafka/docker-compose.yml new file mode 100644 index 000000000000..d06e9f5681c0 --- /dev/null +++ b/tools/kafka/docker-compose.yml @@ -0,0 +1,16 @@ +version: '2' +services: + zookeeper: + image: wurstmeister/zookeeper + ports: + - "2181:2181" + kafka: + image: wurstmeister/kafka + ports: + - "9092" + environment: + DOCKER_API_VERSION: 1.22 + KAFKA_ADVERTISED_HOST_NAME: host.docker.internal + KAFKA_ZOOKEEPER_CONNECT: zookeeper:2181 + volumes: + - /var/run/docker.sock:/var/run/docker.sock diff --git a/vendor/github.com/Shopify/sarama/.gitignore b/vendor/github.com/Shopify/sarama/.gitignore new file mode 100644 index 000000000000..2c9adc20b319 --- /dev/null +++ b/vendor/github.com/Shopify/sarama/.gitignore @@ -0,0 +1,29 @@ +# Compiled Object files, Static and Dynamic libs (Shared Objects) +*.o +*.a +*.so +*.test + +# Folders +_obj +_test +.vagrant + +# Architecture specific extensions/prefixes +*.[568vq] +[568vq].out + +*.cgo1.go +*.cgo2.c +_cgo_defun.c +_cgo_gotypes.go +_cgo_export.* + +_testmain.go + +*.exe + +coverage.txt +profile.out + +simplest-uncommitted-msg-0.1-jar-with-dependencies.jar diff --git a/vendor/github.com/Shopify/sarama/.golangci.yml b/vendor/github.com/Shopify/sarama/.golangci.yml new file mode 100644 index 000000000000..f92a7167c232 --- /dev/null +++ b/vendor/github.com/Shopify/sarama/.golangci.yml @@ -0,0 +1,76 @@ +run: + timeout: 5m + deadline: 10m + +linters-settings: + govet: + check-shadowing: false + golint: + min-confidence: 0 + gocyclo: + min-complexity: 99 + maligned: + suggest-new: true + dupl: + threshold: 100 + goconst: + min-len: 2 + min-occurrences: 3 + misspell: + locale: US + goimports: + local-prefixes: github.com/Shopify/sarama + gocritic: + enabled-tags: + - diagnostic + - experimental + - opinionated + - performance + - style + disabled-checks: + - wrapperFunc + - ifElseChain + funlen: + lines: 300 + statements: 300 + +linters: + disable-all: true + enable: + - bodyclose + - deadcode + - depguard + - exportloopref + - dogsled + # - dupl + - errcheck + - funlen + - gochecknoinits + # - goconst + # - gocritic + - gocyclo + - gofmt + - goimports + # - golint + - gosec + # - gosimple + - govet + # - ineffassign + - misspell + # - nakedret + - nilerr + # - scopelint + - staticcheck + - structcheck + # - stylecheck + - typecheck + - unconvert + - unused + - varcheck + - whitespace + +issues: + exclude: + - "G404: Use of weak random number generator" + # maximum count of issues with the same text. set to 0 for unlimited. default is 3. + max-same-issues: 0 diff --git a/vendor/github.com/Shopify/sarama/CHANGELOG.md b/vendor/github.com/Shopify/sarama/CHANGELOG.md new file mode 100644 index 000000000000..dd826e273df7 --- /dev/null +++ b/vendor/github.com/Shopify/sarama/CHANGELOG.md @@ -0,0 +1,1023 @@ +# Changelog + +#### Unreleased + +# Improvements + +- #1912 - @faillefer Support for --delete-offsets for consumer group topic + +#### Version 1.28.0 (2021-02-15) + +**Note that with this release we change `RoundRobinBalancer` strategy to match Java client behavior. See #1788 for details.** + +- #1870 - @kvch - Update Kerberos library to latest major +- #1876 - @bai - Update docs, reference pkg.go.dev +- #1846 - @wclaeys - Do not ignore Consumer.Offsets.AutoCommit.Enable config on Close +- #1747 - @XSAM - fix: mock sync producer does not handle the offset while sending messages +- #1863 - @bai - Add support for Kafka 2.7.0 + update lz4 and klauspost/compress dependencies +- #1788 - @kzinglzy - feat[balance_strategy]: announcing a new round robin balance strategy +- #1862 - @bai - Fix CI setenv permissions issues +- #1832 - @ilyakaznacheev - Update Godoc link to pkg.go.dev +- #1822 - @danp - KIP-392: Allow consumers to fetch from closest replica + +#### Version 1.27.2 (2020-10-21) + +# Improvements + +#1750 - @krantideep95 Adds missing mock responses for mocking consumer group + +# Fixes + +#1817 - reverts #1785 - Add private method to Client interface to prevent implementation + +#### Version 1.27.1 (2020-10-07) + +# Improvements + +#1775 - @d1egoaz - Adds a Producer Interceptor example +#1781 - @justin-chen - Refresh brokers given list of seed brokers +#1784 - @justin-chen - Add randomize seed broker method +#1790 - @d1egoaz - remove example binary +#1798 - @bai - Test against Go 1.15 +#1785 - @justin-chen - Add private method to Client interface to prevent implementation +#1802 - @uvw - Support Go 1.13 error unwrapping + +# Fixes + +#1791 - @stanislavkozlovski - bump default version to 1.0.0 + +#### Version 1.27.0 (2020-08-11) + +# Improvements + +#1466 - @rubenvp8510 - Expose kerberos fast negotiation configuration +#1695 - @KJTsanaktsidis - Use docker-compose to run the functional tests +#1699 - @wclaeys - Consumer group support for manually comitting offsets +#1714 - @bai - Bump Go to version 1.14.3, golangci-lint to 1.27.0 +#1726 - @d1egoaz - Include zstd on the functional tests +#1730 - @d1egoaz - KIP-42 Add producer and consumer interceptors +#1738 - @varun06 - fixed variable names that are named same as some std lib package names +#1741 - @varun06 - updated zstd dependency to latest v1.10.10 +#1743 - @varun06 - Fixed declaration dependencies and other lint issues in code base +#1763 - @alrs - remove deprecated tls options from test +#1769 - @bai - Add support for Kafka 2.6.0 + +# Fixes + +#1697 - @kvch - Use gofork for encoding/asn1 to fix ASN errors during Kerberos authentication +#1744 - @alrs - Fix isBalanced Function Signature + +#### Version 1.26.4 (2020-05-19) + +# Fixes + +- #1701 - @d1egoaz - Set server name only for the current broker +- #1694 - @dnwe - testfix: set KAFKA_HEAP_OPTS for zk and kafka + +#### Version 1.26.3 (2020-05-07) + +# Fixes + +- #1692 - @d1egoaz - Set tls ServerName to fix issue: either ServerName or InsecureSkipVerify must be specified in the tls.Config + +#### Version 1.26.2 (2020-05-06) + +# ⚠️ Known Issues + +This release has been marked as not ready for production and may be unstable, please use v1.26.4. + +# Improvements + +- #1560 - @iyacontrol - add sync pool for gzip 1-9 +- #1605 - @dnwe - feat: protocol support for V11 fetch w/ rackID +- #1617 - @sladkoff / @dwi-di / @random-dwi - Add support for alter/list partition reassignements APIs +- #1632 - @bai - Add support for Go 1.14 +- #1640 - @random-dwi - Feature/fix list partition reassignments +- #1646 - @mimaison - Add DescribeLogDirs to admin client +- #1667 - @bai - Add support for kafka 2.5.0 + +# Fixes + +- #1594 - @sladkoff - Sets ConfigEntry.Default flag in addition to the ConfigEntry.Source for Kafka versions > V1_1_0_0 +- #1601 - @alrs - fix: remove use of testing.T.FailNow() inside goroutine +- #1602 - @d1egoaz - adds a note about consumer groups Consume method +- #1607 - @darklore - Fix memory leak when Broker.Open and Broker.Close called repeatedly +- #1613 - @wblakecaldwell - Updated "retrying" log message when BackoffFunc implemented +- #1614 - @alrs - produce_response.go: Remove Unused Functions +- #1619 - @alrs - tools/kafka-producer-performance: prune unused flag variables +- #1639 - @agriffaut - Handle errors with no message but error code +- #1643 - @kzinglzy - fix `config.net.keepalive` +- #1644 - @KJTsanaktsidis - Fix brokers continually allocating new Session IDs +- #1645 - @Stephan14 - Remove broker(s) which no longer exist in metadata +- #1650 - @lavoiesl - Return the response error in heartbeatLoop +- #1661 - @KJTsanaktsidis - Fix "broker received out of order sequence" when brokers die +- #1666 - @KevinJCross - Bugfix: Allow TLS connections to work over socks proxy. + +#### Version 1.26.1 (2020-02-04) + +Improvements: +- Add requests-in-flight metric ([1539](https://github.com/Shopify/sarama/pull/1539)) +- Fix misleading example for cluster admin ([1595](https://github.com/Shopify/sarama/pull/1595)) +- Replace Travis with GitHub Actions, linters housekeeping ([1573](https://github.com/Shopify/sarama/pull/1573)) +- Allow BalanceStrategy to provide custom assignment data ([1592](https://github.com/Shopify/sarama/pull/1592)) + +Bug Fixes: +- Adds back Consumer.Offsets.CommitInterval to fix API ([1590](https://github.com/Shopify/sarama/pull/1590)) +- Fix error message s/CommitInterval/AutoCommit.Interval ([1589](https://github.com/Shopify/sarama/pull/1589)) + +#### Version 1.26.0 (2020-01-24) + +New Features: +- Enable zstd compression + ([1574](https://github.com/Shopify/sarama/pull/1574), + [1582](https://github.com/Shopify/sarama/pull/1582)) +- Support headers in tools kafka-console-producer + ([1549](https://github.com/Shopify/sarama/pull/1549)) + +Improvements: +- Add SASL AuthIdentity to SASL frames (authzid) + ([1585](https://github.com/Shopify/sarama/pull/1585)). + +Bug Fixes: +- Sending messages with ZStd compression enabled fails in multiple ways + ([1252](https://github.com/Shopify/sarama/issues/1252)). +- Use the broker for any admin on BrokerConfig + ([1571](https://github.com/Shopify/sarama/pull/1571)). +- Set DescribeConfigRequest Version field + ([1576](https://github.com/Shopify/sarama/pull/1576)). +- ConsumerGroup flooding logs with client/metadata update req + ([1578](https://github.com/Shopify/sarama/pull/1578)). +- MetadataRequest version in DescribeCluster + ([1580](https://github.com/Shopify/sarama/pull/1580)). +- Fix deadlock in consumer group handleError + ([1581](https://github.com/Shopify/sarama/pull/1581)) +- Fill in the Fetch{Request,Response} protocol + ([1582](https://github.com/Shopify/sarama/pull/1582)). +- Retry topic request on ControllerNotAvailable + ([1586](https://github.com/Shopify/sarama/pull/1586)). + +#### Version 1.25.0 (2020-01-13) + +New Features: +- Support TLS protocol in kafka-producer-performance + ([1538](https://github.com/Shopify/sarama/pull/1538)). +- Add support for kafka 2.4.0 + ([1552](https://github.com/Shopify/sarama/pull/1552)). + +Improvements: +- Allow the Consumer to disable auto-commit offsets + ([1164](https://github.com/Shopify/sarama/pull/1164)). +- Produce records with consistent timestamps + ([1455](https://github.com/Shopify/sarama/pull/1455)). + +Bug Fixes: +- Fix incorrect SetTopicMetadata name mentions + ([1534](https://github.com/Shopify/sarama/pull/1534)). +- Fix client.tryRefreshMetadata Println + ([1535](https://github.com/Shopify/sarama/pull/1535)). +- Fix panic on calling updateMetadata on closed client + ([1531](https://github.com/Shopify/sarama/pull/1531)). +- Fix possible faulty metrics in TestFuncProducing + ([1545](https://github.com/Shopify/sarama/pull/1545)). + +#### Version 1.24.1 (2019-10-31) + +New Features: +- Add DescribeLogDirs Request/Response pair + ([1520](https://github.com/Shopify/sarama/pull/1520)). + +Bug Fixes: +- Fix ClusterAdmin returning invalid controller ID on DescribeCluster + ([1518](https://github.com/Shopify/sarama/pull/1518)). +- Fix issue with consumergroup not rebalancing when new partition is added + ([1525](https://github.com/Shopify/sarama/pull/1525)). +- Ensure consistent use of read/write deadlines + ([1529](https://github.com/Shopify/sarama/pull/1529)). + +#### Version 1.24.0 (2019-10-09) + +New Features: +- Add sticky partition assignor + ([1416](https://github.com/Shopify/sarama/pull/1416)). +- Switch from cgo zstd package to pure Go implementation + ([1477](https://github.com/Shopify/sarama/pull/1477)). + +Improvements: +- Allow creating ClusterAdmin from client + ([1415](https://github.com/Shopify/sarama/pull/1415)). +- Set KafkaVersion in ListAcls method + ([1452](https://github.com/Shopify/sarama/pull/1452)). +- Set request version in CreateACL ClusterAdmin method + ([1458](https://github.com/Shopify/sarama/pull/1458)). +- Set request version in DeleteACL ClusterAdmin method + ([1461](https://github.com/Shopify/sarama/pull/1461)). +- Handle missed error codes on TopicMetaDataRequest and GroupCoordinatorRequest + ([1464](https://github.com/Shopify/sarama/pull/1464)). +- Remove direct usage of gofork + ([1465](https://github.com/Shopify/sarama/pull/1465)). +- Add support for Go 1.13 + ([1478](https://github.com/Shopify/sarama/pull/1478)). +- Improve behavior of NewMockListAclsResponse + ([1481](https://github.com/Shopify/sarama/pull/1481)). + +Bug Fixes: +- Fix race condition in consumergroup example + ([1434](https://github.com/Shopify/sarama/pull/1434)). +- Fix brokerProducer goroutine leak + ([1442](https://github.com/Shopify/sarama/pull/1442)). +- Use released version of lz4 library + ([1469](https://github.com/Shopify/sarama/pull/1469)). +- Set correct version in MockDeleteTopicsResponse + ([1484](https://github.com/Shopify/sarama/pull/1484)). +- Fix CLI help message typo + ([1494](https://github.com/Shopify/sarama/pull/1494)). + +Known Issues: +- Please **don't** use Zstd, as it doesn't work right now. + See https://github.com/Shopify/sarama/issues/1252 + +#### Version 1.23.1 (2019-07-22) + +Bug Fixes: +- Fix fetch delete bug record + ([1425](https://github.com/Shopify/sarama/pull/1425)). +- Handle SASL/OAUTHBEARER token rejection + ([1428](https://github.com/Shopify/sarama/pull/1428)). + +#### Version 1.23.0 (2019-07-02) + +New Features: +- Add support for Kafka 2.3.0 + ([1418](https://github.com/Shopify/sarama/pull/1418)). +- Add support for ListConsumerGroupOffsets v2 + ([1374](https://github.com/Shopify/sarama/pull/1374)). +- Add support for DeleteConsumerGroup + ([1417](https://github.com/Shopify/sarama/pull/1417)). +- Add support for SASLVersion configuration + ([1410](https://github.com/Shopify/sarama/pull/1410)). +- Add kerberos support + ([1366](https://github.com/Shopify/sarama/pull/1366)). + +Improvements: +- Improve sasl_scram_client example + ([1406](https://github.com/Shopify/sarama/pull/1406)). +- Fix shutdown and race-condition in consumer-group example + ([1404](https://github.com/Shopify/sarama/pull/1404)). +- Add support for error codes 77—81 + ([1397](https://github.com/Shopify/sarama/pull/1397)). +- Pool internal objects allocated per message + ([1385](https://github.com/Shopify/sarama/pull/1385)). +- Reduce packet decoder allocations + ([1373](https://github.com/Shopify/sarama/pull/1373)). +- Support timeout when fetching metadata + ([1359](https://github.com/Shopify/sarama/pull/1359)). + +Bug Fixes: +- Fix fetch size integer overflow + ([1376](https://github.com/Shopify/sarama/pull/1376)). +- Handle and log throttled FetchResponses + ([1383](https://github.com/Shopify/sarama/pull/1383)). +- Refactor misspelled word Resouce to Resource + ([1368](https://github.com/Shopify/sarama/pull/1368)). + +#### Version 1.22.1 (2019-04-29) + +Improvements: +- Use zstd 1.3.8 + ([1350](https://github.com/Shopify/sarama/pull/1350)). +- Add support for SaslHandshakeRequest v1 + ([1354](https://github.com/Shopify/sarama/pull/1354)). + +Bug Fixes: +- Fix V5 MetadataRequest nullable topics array + ([1353](https://github.com/Shopify/sarama/pull/1353)). +- Use a different SCRAM client for each broker connection + ([1349](https://github.com/Shopify/sarama/pull/1349)). +- Fix AllowAutoTopicCreation for MetadataRequest greater than v3 + ([1344](https://github.com/Shopify/sarama/pull/1344)). + +#### Version 1.22.0 (2019-04-09) + +New Features: +- Add Offline Replicas Operation to Client + ([1318](https://github.com/Shopify/sarama/pull/1318)). +- Allow using proxy when connecting to broker + ([1326](https://github.com/Shopify/sarama/pull/1326)). +- Implement ReadCommitted + ([1307](https://github.com/Shopify/sarama/pull/1307)). +- Add support for Kafka 2.2.0 + ([1331](https://github.com/Shopify/sarama/pull/1331)). +- Add SASL SCRAM-SHA-512 and SCRAM-SHA-256 mechanismes + ([1331](https://github.com/Shopify/sarama/pull/1295)). + +Improvements: +- Unregister all broker metrics on broker stop + ([1232](https://github.com/Shopify/sarama/pull/1232)). +- Add SCRAM authentication example + ([1303](https://github.com/Shopify/sarama/pull/1303)). +- Add consumergroup examples + ([1304](https://github.com/Shopify/sarama/pull/1304)). +- Expose consumer batch size metric + ([1296](https://github.com/Shopify/sarama/pull/1296)). +- Add TLS options to console producer and consumer + ([1300](https://github.com/Shopify/sarama/pull/1300)). +- Reduce client close bookkeeping + ([1297](https://github.com/Shopify/sarama/pull/1297)). +- Satisfy error interface in create responses + ([1154](https://github.com/Shopify/sarama/pull/1154)). +- Please lint gods + ([1346](https://github.com/Shopify/sarama/pull/1346)). + +Bug Fixes: +- Fix multi consumer group instance crash + ([1338](https://github.com/Shopify/sarama/pull/1338)). +- Update lz4 to latest version + ([1347](https://github.com/Shopify/sarama/pull/1347)). +- Retry ErrNotCoordinatorForConsumer in new consumergroup session + ([1231](https://github.com/Shopify/sarama/pull/1231)). +- Fix cleanup error handler + ([1332](https://github.com/Shopify/sarama/pull/1332)). +- Fix rate condition in PartitionConsumer + ([1156](https://github.com/Shopify/sarama/pull/1156)). + +#### Version 1.21.0 (2019-02-24) + +New Features: +- Add CreateAclRequest, DescribeAclRequest, DeleteAclRequest + ([1236](https://github.com/Shopify/sarama/pull/1236)). +- Add DescribeTopic, DescribeConsumerGroup, ListConsumerGroups, ListConsumerGroupOffsets admin requests + ([1178](https://github.com/Shopify/sarama/pull/1178)). +- Implement SASL/OAUTHBEARER + ([1240](https://github.com/Shopify/sarama/pull/1240)). + +Improvements: +- Add Go mod support + ([1282](https://github.com/Shopify/sarama/pull/1282)). +- Add error codes 73—76 + ([1239](https://github.com/Shopify/sarama/pull/1239)). +- Add retry backoff function + ([1160](https://github.com/Shopify/sarama/pull/1160)). +- Maintain metadata in the producer even when retries are disabled + ([1189](https://github.com/Shopify/sarama/pull/1189)). +- Include ReplicaAssignment in ListTopics + ([1274](https://github.com/Shopify/sarama/pull/1274)). +- Add producer performance tool + ([1222](https://github.com/Shopify/sarama/pull/1222)). +- Add support LogAppend timestamps + ([1258](https://github.com/Shopify/sarama/pull/1258)). + +Bug Fixes: +- Fix potential deadlock when a heartbeat request fails + ([1286](https://github.com/Shopify/sarama/pull/1286)). +- Fix consuming compacted topic + ([1227](https://github.com/Shopify/sarama/pull/1227)). +- Set correct Kafka version for DescribeConfigsRequest v1 + ([1277](https://github.com/Shopify/sarama/pull/1277)). +- Update kafka test version + ([1273](https://github.com/Shopify/sarama/pull/1273)). + +#### Version 1.20.1 (2019-01-10) + +New Features: +- Add optional replica id in offset request + ([1100](https://github.com/Shopify/sarama/pull/1100)). + +Improvements: +- Implement DescribeConfigs Request + Response v1 & v2 + ([1230](https://github.com/Shopify/sarama/pull/1230)). +- Reuse compression objects + ([1185](https://github.com/Shopify/sarama/pull/1185)). +- Switch from png to svg for GoDoc link in README + ([1243](https://github.com/Shopify/sarama/pull/1243)). +- Fix typo in deprecation notice for FetchResponseBlock.Records + ([1242](https://github.com/Shopify/sarama/pull/1242)). +- Fix typos in consumer metadata response file + ([1244](https://github.com/Shopify/sarama/pull/1244)). + +Bug Fixes: +- Revert to individual msg retries for non-idempotent + ([1203](https://github.com/Shopify/sarama/pull/1203)). +- Respect MaxMessageBytes limit for uncompressed messages + ([1141](https://github.com/Shopify/sarama/pull/1141)). + +#### Version 1.20.0 (2018-12-10) + +New Features: + - Add support for zstd compression + ([#1170](https://github.com/Shopify/sarama/pull/1170)). + - Add support for Idempotent Producer + ([#1152](https://github.com/Shopify/sarama/pull/1152)). + - Add support support for Kafka 2.1.0 + ([#1229](https://github.com/Shopify/sarama/pull/1229)). + - Add support support for OffsetCommit request/response pairs versions v1 to v5 + ([#1201](https://github.com/Shopify/sarama/pull/1201)). + - Add support support for OffsetFetch request/response pair up to version v5 + ([#1198](https://github.com/Shopify/sarama/pull/1198)). + +Improvements: + - Export broker's Rack setting + ([#1173](https://github.com/Shopify/sarama/pull/1173)). + - Always use latest patch version of Go on CI + ([#1202](https://github.com/Shopify/sarama/pull/1202)). + - Add error codes 61 to 72 + ([#1195](https://github.com/Shopify/sarama/pull/1195)). + +Bug Fixes: + - Fix build without cgo + ([#1182](https://github.com/Shopify/sarama/pull/1182)). + - Fix go vet suggestion in consumer group file + ([#1209](https://github.com/Shopify/sarama/pull/1209)). + - Fix typos in code and comments + ([#1228](https://github.com/Shopify/sarama/pull/1228)). + +#### Version 1.19.0 (2018-09-27) + +New Features: + - Implement a higher-level consumer group + ([#1099](https://github.com/Shopify/sarama/pull/1099)). + +Improvements: + - Add support for Go 1.11 + ([#1176](https://github.com/Shopify/sarama/pull/1176)). + +Bug Fixes: + - Fix encoding of `MetadataResponse` with version 2 and higher + ([#1174](https://github.com/Shopify/sarama/pull/1174)). + - Fix race condition in mock async producer + ([#1174](https://github.com/Shopify/sarama/pull/1174)). + +#### Version 1.18.0 (2018-09-07) + +New Features: + - Make `Partitioner.RequiresConsistency` vary per-message + ([#1112](https://github.com/Shopify/sarama/pull/1112)). + - Add customizable partitioner + ([#1118](https://github.com/Shopify/sarama/pull/1118)). + - Add `ClusterAdmin` support for `CreateTopic`, `DeleteTopic`, `CreatePartitions`, + `DeleteRecords`, `DescribeConfig`, `AlterConfig`, `CreateACL`, `ListAcls`, `DeleteACL` + ([#1055](https://github.com/Shopify/sarama/pull/1055)). + +Improvements: + - Add support for Kafka 2.0.0 + ([#1149](https://github.com/Shopify/sarama/pull/1149)). + - Allow setting `LocalAddr` when dialing an address to support multi-homed hosts + ([#1123](https://github.com/Shopify/sarama/pull/1123)). + - Simpler offset management + ([#1127](https://github.com/Shopify/sarama/pull/1127)). + +Bug Fixes: + - Fix mutation of `ProducerMessage.MetaData` when producing to Kafka + ([#1110](https://github.com/Shopify/sarama/pull/1110)). + - Fix consumer block when response did not contain all the + expected topic/partition blocks + ([#1086](https://github.com/Shopify/sarama/pull/1086)). + - Fix consumer block when response contains only constrol messages + ([#1115](https://github.com/Shopify/sarama/pull/1115)). + - Add timeout config for ClusterAdmin requests + ([#1142](https://github.com/Shopify/sarama/pull/1142)). + - Add version check when producing message with headers + ([#1117](https://github.com/Shopify/sarama/pull/1117)). + - Fix `MetadataRequest` for empty list of topics + ([#1132](https://github.com/Shopify/sarama/pull/1132)). + - Fix producer topic metadata on-demand fetch when topic error happens in metadata response + ([#1125](https://github.com/Shopify/sarama/pull/1125)). + +#### Version 1.17.0 (2018-05-30) + +New Features: + - Add support for gzip compression levels + ([#1044](https://github.com/Shopify/sarama/pull/1044)). + - Add support for Metadata request/response pairs versions v1 to v5 + ([#1047](https://github.com/Shopify/sarama/pull/1047), + [#1069](https://github.com/Shopify/sarama/pull/1069)). + - Add versioning to JoinGroup request/response pairs + ([#1098](https://github.com/Shopify/sarama/pull/1098)) + - Add support for CreatePartitions, DeleteGroups, DeleteRecords request/response pairs + ([#1065](https://github.com/Shopify/sarama/pull/1065), + [#1096](https://github.com/Shopify/sarama/pull/1096), + [#1027](https://github.com/Shopify/sarama/pull/1027)). + - Add `Controller()` method to Client interface + ([#1063](https://github.com/Shopify/sarama/pull/1063)). + +Improvements: + - ConsumerMetadataReq/Resp has been migrated to FindCoordinatorReq/Resp + ([#1010](https://github.com/Shopify/sarama/pull/1010)). + - Expose missing protocol parts: `msgSet` and `recordBatch` + ([#1049](https://github.com/Shopify/sarama/pull/1049)). + - Add support for v1 DeleteTopics Request + ([#1052](https://github.com/Shopify/sarama/pull/1052)). + - Add support for Go 1.10 + ([#1064](https://github.com/Shopify/sarama/pull/1064)). + - Claim support for Kafka 1.1.0 + ([#1073](https://github.com/Shopify/sarama/pull/1073)). + +Bug Fixes: + - Fix FindCoordinatorResponse.encode to allow nil Coordinator + ([#1050](https://github.com/Shopify/sarama/pull/1050), + [#1051](https://github.com/Shopify/sarama/pull/1051)). + - Clear all metadata when we have the latest topic info + ([#1033](https://github.com/Shopify/sarama/pull/1033)). + - Make `PartitionConsumer.Close` idempotent + ([#1092](https://github.com/Shopify/sarama/pull/1092)). + +#### Version 1.16.0 (2018-02-12) + +New Features: + - Add support for the Create/Delete Topics request/response pairs + ([#1007](https://github.com/Shopify/sarama/pull/1007), + [#1008](https://github.com/Shopify/sarama/pull/1008)). + - Add support for the Describe/Create/Delete ACL request/response pairs + ([#1009](https://github.com/Shopify/sarama/pull/1009)). + - Add support for the five transaction-related request/response pairs + ([#1016](https://github.com/Shopify/sarama/pull/1016)). + +Improvements: + - Permit setting version on mock producer responses + ([#999](https://github.com/Shopify/sarama/pull/999)). + - Add `NewMockBrokerListener` helper for testing TLS connections + ([#1019](https://github.com/Shopify/sarama/pull/1019)). + - Changed the default value for `Consumer.Fetch.Default` from 32KiB to 1MiB + which results in much higher throughput in most cases + ([#1024](https://github.com/Shopify/sarama/pull/1024)). + - Reuse the `time.Ticker` across fetch requests in the PartitionConsumer to + reduce CPU and memory usage when processing many partitions + ([#1028](https://github.com/Shopify/sarama/pull/1028)). + - Assign relative offsets to messages in the producer to save the brokers a + recompression pass + ([#1002](https://github.com/Shopify/sarama/pull/1002), + [#1015](https://github.com/Shopify/sarama/pull/1015)). + +Bug Fixes: + - Fix producing uncompressed batches with the new protocol format + ([#1032](https://github.com/Shopify/sarama/issues/1032)). + - Fix consuming compacted topics with the new protocol format + ([#1005](https://github.com/Shopify/sarama/issues/1005)). + - Fix consuming topics with a mix of protocol formats + ([#1021](https://github.com/Shopify/sarama/issues/1021)). + - Fix consuming when the broker includes multiple batches in a single response + ([#1022](https://github.com/Shopify/sarama/issues/1022)). + - Fix detection of `PartialTrailingMessage` when the partial message was + truncated before the magic value indicating its version + ([#1030](https://github.com/Shopify/sarama/pull/1030)). + - Fix expectation-checking in the mock of `SyncProducer.SendMessages` + ([#1035](https://github.com/Shopify/sarama/pull/1035)). + +#### Version 1.15.0 (2017-12-08) + +New Features: + - Claim official support for Kafka 1.0, though it did already work + ([#984](https://github.com/Shopify/sarama/pull/984)). + - Helper methods for Kafka version numbers to/from strings + ([#989](https://github.com/Shopify/sarama/pull/989)). + - Implement CreatePartitions request/response + ([#985](https://github.com/Shopify/sarama/pull/985)). + +Improvements: + - Add error codes 45-60 + ([#986](https://github.com/Shopify/sarama/issues/986)). + +Bug Fixes: + - Fix slow consuming for certain Kafka 0.11/1.0 configurations + ([#982](https://github.com/Shopify/sarama/pull/982)). + - Correctly determine when a FetchResponse contains the new message format + ([#990](https://github.com/Shopify/sarama/pull/990)). + - Fix producing with multiple headers + ([#996](https://github.com/Shopify/sarama/pull/996)). + - Fix handling of truncated record batches + ([#998](https://github.com/Shopify/sarama/pull/998)). + - Fix leaking metrics when closing brokers + ([#991](https://github.com/Shopify/sarama/pull/991)). + +#### Version 1.14.0 (2017-11-13) + +New Features: + - Add support for the new Kafka 0.11 record-batch format, including the wire + protocol and the necessary behavioural changes in the producer and consumer. + Transactions and idempotency are not yet supported, but producing and + consuming should work with all the existing bells and whistles (batching, + compression, etc) as well as the new custom headers. Thanks to Vlad Hanciuta + of Arista Networks for this work. Part of + ([#901](https://github.com/Shopify/sarama/issues/901)). + +Bug Fixes: + - Fix encoding of ProduceResponse versions in test + ([#970](https://github.com/Shopify/sarama/pull/970)). + - Return partial replicas list when we have it + ([#975](https://github.com/Shopify/sarama/pull/975)). + +#### Version 1.13.0 (2017-10-04) + +New Features: + - Support for FetchRequest version 3 + ([#905](https://github.com/Shopify/sarama/pull/905)). + - Permit setting version on mock FetchResponses + ([#939](https://github.com/Shopify/sarama/pull/939)). + - Add a configuration option to support storing only minimal metadata for + extremely large clusters + ([#937](https://github.com/Shopify/sarama/pull/937)). + - Add `PartitionOffsetManager.ResetOffset` for backtracking tracked offsets + ([#932](https://github.com/Shopify/sarama/pull/932)). + +Improvements: + - Provide the block-level timestamp when consuming compressed messages + ([#885](https://github.com/Shopify/sarama/issues/885)). + - `Client.Replicas` and `Client.InSyncReplicas` now respect the order returned + by the broker, which can be meaningful + ([#930](https://github.com/Shopify/sarama/pull/930)). + - Use a `Ticker` to reduce consumer timer overhead at the cost of higher + variance in the actual timeout + ([#933](https://github.com/Shopify/sarama/pull/933)). + +Bug Fixes: + - Gracefully handle messages with negative timestamps + ([#907](https://github.com/Shopify/sarama/pull/907)). + - Raise a proper error when encountering an unknown message version + ([#940](https://github.com/Shopify/sarama/pull/940)). + +#### Version 1.12.0 (2017-05-08) + +New Features: + - Added support for the `ApiVersions` request and response pair, and Kafka + version 0.10.2 ([#867](https://github.com/Shopify/sarama/pull/867)). Note + that you still need to specify the Kafka version in the Sarama configuration + for the time being. + - Added a `Brokers` method to the Client which returns the complete set of + active brokers ([#813](https://github.com/Shopify/sarama/pull/813)). + - Added an `InSyncReplicas` method to the Client which returns the set of all + in-sync broker IDs for the given partition, now that the Kafka versions for + which this was misleading are no longer in our supported set + ([#872](https://github.com/Shopify/sarama/pull/872)). + - Added a `NewCustomHashPartitioner` method which allows constructing a hash + partitioner with a custom hash method in case the default (FNV-1a) is not + suitable + ([#837](https://github.com/Shopify/sarama/pull/837), + [#841](https://github.com/Shopify/sarama/pull/841)). + +Improvements: + - Recognize more Kafka error codes + ([#859](https://github.com/Shopify/sarama/pull/859)). + +Bug Fixes: + - Fix an issue where decoding a malformed FetchRequest would not return the + correct error ([#818](https://github.com/Shopify/sarama/pull/818)). + - Respect ordering of group protocols in JoinGroupRequests. This fix is + transparent if you're using the `AddGroupProtocol` or + `AddGroupProtocolMetadata` helpers; otherwise you will need to switch from + the `GroupProtocols` field (now deprecated) to use `OrderedGroupProtocols` + ([#812](https://github.com/Shopify/sarama/issues/812)). + - Fix an alignment-related issue with atomics on 32-bit architectures + ([#859](https://github.com/Shopify/sarama/pull/859)). + +#### Version 1.11.0 (2016-12-20) + +_Important:_ As of Sarama 1.11 it is necessary to set the config value of +`Producer.Return.Successes` to true in order to use the SyncProducer. Previous +versions would silently override this value when instantiating a SyncProducer +which led to unexpected values and data races. + +New Features: + - Metrics! Thanks to Sébastien Launay for all his work on this feature + ([#701](https://github.com/Shopify/sarama/pull/701), + [#746](https://github.com/Shopify/sarama/pull/746), + [#766](https://github.com/Shopify/sarama/pull/766)). + - Add support for LZ4 compression + ([#786](https://github.com/Shopify/sarama/pull/786)). + - Add support for ListOffsetRequest v1 and Kafka 0.10.1 + ([#775](https://github.com/Shopify/sarama/pull/775)). + - Added a `HighWaterMarks` method to the Consumer which aggregates the + `HighWaterMarkOffset` values of its child topic/partitions + ([#769](https://github.com/Shopify/sarama/pull/769)). + +Bug Fixes: + - Fixed producing when using timestamps, compression and Kafka 0.10 + ([#759](https://github.com/Shopify/sarama/pull/759)). + - Added missing decoder methods to DescribeGroups response + ([#756](https://github.com/Shopify/sarama/pull/756)). + - Fix producer shutdown when `Return.Errors` is disabled + ([#787](https://github.com/Shopify/sarama/pull/787)). + - Don't mutate configuration in SyncProducer + ([#790](https://github.com/Shopify/sarama/pull/790)). + - Fix crash on SASL initialization failure + ([#795](https://github.com/Shopify/sarama/pull/795)). + +#### Version 1.10.1 (2016-08-30) + +Bug Fixes: + - Fix the documentation for `HashPartitioner` which was incorrect + ([#717](https://github.com/Shopify/sarama/pull/717)). + - Permit client creation even when it is limited by ACLs + ([#722](https://github.com/Shopify/sarama/pull/722)). + - Several fixes to the consumer timer optimization code, regressions introduced + in v1.10.0. Go's timers are finicky + ([#730](https://github.com/Shopify/sarama/pull/730), + [#733](https://github.com/Shopify/sarama/pull/733), + [#734](https://github.com/Shopify/sarama/pull/734)). + - Handle consuming compressed relative offsets with Kafka 0.10 + ([#735](https://github.com/Shopify/sarama/pull/735)). + +#### Version 1.10.0 (2016-08-02) + +_Important:_ As of Sarama 1.10 it is necessary to tell Sarama the version of +Kafka you are running against (via the `config.Version` value) in order to use +features that may not be compatible with old Kafka versions. If you don't +specify this value it will default to 0.8.2 (the minimum supported), and trying +to use more recent features (like the offset manager) will fail with an error. + +_Also:_ The offset-manager's behaviour has been changed to match the upstream +java consumer (see [#705](https://github.com/Shopify/sarama/pull/705) and +[#713](https://github.com/Shopify/sarama/pull/713)). If you use the +offset-manager, please ensure that you are committing one *greater* than the +last consumed message offset or else you may end up consuming duplicate +messages. + +New Features: + - Support for Kafka 0.10 + ([#672](https://github.com/Shopify/sarama/pull/672), + [#678](https://github.com/Shopify/sarama/pull/678), + [#681](https://github.com/Shopify/sarama/pull/681), and others). + - Support for configuring the target Kafka version + ([#676](https://github.com/Shopify/sarama/pull/676)). + - Batch producing support in the SyncProducer + ([#677](https://github.com/Shopify/sarama/pull/677)). + - Extend producer mock to allow setting expectations on message contents + ([#667](https://github.com/Shopify/sarama/pull/667)). + +Improvements: + - Support `nil` compressed messages for deleting in compacted topics + ([#634](https://github.com/Shopify/sarama/pull/634)). + - Pre-allocate decoding errors, greatly reducing heap usage and GC time against + misbehaving brokers ([#690](https://github.com/Shopify/sarama/pull/690)). + - Re-use consumer expiry timers, removing one allocation per consumed message + ([#707](https://github.com/Shopify/sarama/pull/707)). + +Bug Fixes: + - Actually default the client ID to "sarama" like we say we do + ([#664](https://github.com/Shopify/sarama/pull/664)). + - Fix a rare issue where `Client.Leader` could return the wrong error + ([#685](https://github.com/Shopify/sarama/pull/685)). + - Fix a possible tight loop in the consumer + ([#693](https://github.com/Shopify/sarama/pull/693)). + - Match upstream's offset-tracking behaviour + ([#705](https://github.com/Shopify/sarama/pull/705)). + - Report UnknownTopicOrPartition errors from the offset manager + ([#706](https://github.com/Shopify/sarama/pull/706)). + - Fix possible negative partition value from the HashPartitioner + ([#709](https://github.com/Shopify/sarama/pull/709)). + +#### Version 1.9.0 (2016-05-16) + +New Features: + - Add support for custom offset manager retention durations + ([#602](https://github.com/Shopify/sarama/pull/602)). + - Publish low-level mocks to enable testing of third-party producer/consumer + implementations ([#570](https://github.com/Shopify/sarama/pull/570)). + - Declare support for Golang 1.6 + ([#611](https://github.com/Shopify/sarama/pull/611)). + - Support for SASL plain-text auth + ([#648](https://github.com/Shopify/sarama/pull/648)). + +Improvements: + - Simplified broker locking scheme slightly + ([#604](https://github.com/Shopify/sarama/pull/604)). + - Documentation cleanup + ([#605](https://github.com/Shopify/sarama/pull/605), + [#621](https://github.com/Shopify/sarama/pull/621), + [#654](https://github.com/Shopify/sarama/pull/654)). + +Bug Fixes: + - Fix race condition shutting down the OffsetManager + ([#658](https://github.com/Shopify/sarama/pull/658)). + +#### Version 1.8.0 (2016-02-01) + +New Features: + - Full support for Kafka 0.9: + - All protocol messages and fields + ([#586](https://github.com/Shopify/sarama/pull/586), + [#588](https://github.com/Shopify/sarama/pull/588), + [#590](https://github.com/Shopify/sarama/pull/590)). + - Verified that TLS support works + ([#581](https://github.com/Shopify/sarama/pull/581)). + - Fixed the OffsetManager compatibility + ([#585](https://github.com/Shopify/sarama/pull/585)). + +Improvements: + - Optimize for fewer system calls when reading from the network + ([#584](https://github.com/Shopify/sarama/pull/584)). + - Automatically retry `InvalidMessage` errors to match upstream behaviour + ([#589](https://github.com/Shopify/sarama/pull/589)). + +#### Version 1.7.0 (2015-12-11) + +New Features: + - Preliminary support for Kafka 0.9 + ([#572](https://github.com/Shopify/sarama/pull/572)). This comes with several + caveats: + - Protocol-layer support is mostly in place + ([#577](https://github.com/Shopify/sarama/pull/577)), however Kafka 0.9 + renamed some messages and fields, which we did not in order to preserve API + compatibility. + - The producer and consumer work against 0.9, but the offset manager does + not ([#573](https://github.com/Shopify/sarama/pull/573)). + - TLS support may or may not work + ([#581](https://github.com/Shopify/sarama/pull/581)). + +Improvements: + - Don't wait for request timeouts on dead brokers, greatly speeding recovery + when the TCP connection is left hanging + ([#548](https://github.com/Shopify/sarama/pull/548)). + - Refactored part of the producer. The new version provides a much more elegant + solution to [#449](https://github.com/Shopify/sarama/pull/449). It is also + slightly more efficient, and much more precise in calculating batch sizes + when compression is used + ([#549](https://github.com/Shopify/sarama/pull/549), + [#550](https://github.com/Shopify/sarama/pull/550), + [#551](https://github.com/Shopify/sarama/pull/551)). + +Bug Fixes: + - Fix race condition in consumer test mock + ([#553](https://github.com/Shopify/sarama/pull/553)). + +#### Version 1.6.1 (2015-09-25) + +Bug Fixes: + - Fix panic that could occur if a user-supplied message value failed to encode + ([#449](https://github.com/Shopify/sarama/pull/449)). + +#### Version 1.6.0 (2015-09-04) + +New Features: + - Implementation of a consumer offset manager using the APIs introduced in + Kafka 0.8.2. The API is designed mainly for integration into a future + high-level consumer, not for direct use, although it is *possible* to use it + directly. + ([#461](https://github.com/Shopify/sarama/pull/461)). + +Improvements: + - CRC32 calculation is much faster on machines with SSE4.2 instructions, + removing a major hotspot from most profiles + ([#255](https://github.com/Shopify/sarama/pull/255)). + +Bug Fixes: + - Make protocol decoding more robust against some malformed packets generated + by go-fuzz ([#523](https://github.com/Shopify/sarama/pull/523), + [#525](https://github.com/Shopify/sarama/pull/525)) or found in other ways + ([#528](https://github.com/Shopify/sarama/pull/528)). + - Fix a potential race condition panic in the consumer on shutdown + ([#529](https://github.com/Shopify/sarama/pull/529)). + +#### Version 1.5.0 (2015-08-17) + +New Features: + - TLS-encrypted network connections are now supported. This feature is subject + to change when Kafka releases built-in TLS support, but for now this is + enough to work with TLS-terminating proxies + ([#154](https://github.com/Shopify/sarama/pull/154)). + +Improvements: + - The consumer will not block if a single partition is not drained by the user; + all other partitions will continue to consume normally + ([#485](https://github.com/Shopify/sarama/pull/485)). + - Formatting of error strings has been much improved + ([#495](https://github.com/Shopify/sarama/pull/495)). + - Internal refactoring of the producer for code cleanliness and to enable + future work ([#300](https://github.com/Shopify/sarama/pull/300)). + +Bug Fixes: + - Fix a potential deadlock in the consumer on shutdown + ([#475](https://github.com/Shopify/sarama/pull/475)). + +#### Version 1.4.3 (2015-07-21) + +Bug Fixes: + - Don't include the partitioner in the producer's "fetch partitions" + circuit-breaker ([#466](https://github.com/Shopify/sarama/pull/466)). + - Don't retry messages until the broker is closed when abandoning a broker in + the producer ([#468](https://github.com/Shopify/sarama/pull/468)). + - Update the import path for snappy-go, it has moved again and the API has + changed slightly ([#486](https://github.com/Shopify/sarama/pull/486)). + +#### Version 1.4.2 (2015-05-27) + +Bug Fixes: + - Update the import path for snappy-go, it has moved from google code to github + ([#456](https://github.com/Shopify/sarama/pull/456)). + +#### Version 1.4.1 (2015-05-25) + +Improvements: + - Optimizations when decoding snappy messages, thanks to John Potocny + ([#446](https://github.com/Shopify/sarama/pull/446)). + +Bug Fixes: + - Fix hypothetical race conditions on producer shutdown + ([#450](https://github.com/Shopify/sarama/pull/450), + [#451](https://github.com/Shopify/sarama/pull/451)). + +#### Version 1.4.0 (2015-05-01) + +New Features: + - The consumer now implements `Topics()` and `Partitions()` methods to enable + users to dynamically choose what topics/partitions to consume without + instantiating a full client + ([#431](https://github.com/Shopify/sarama/pull/431)). + - The partition-consumer now exposes the high water mark offset value returned + by the broker via the `HighWaterMarkOffset()` method ([#339](https://github.com/Shopify/sarama/pull/339)). + - Added a `kafka-console-consumer` tool capable of handling multiple + partitions, and deprecated the now-obsolete `kafka-console-partitionConsumer` + ([#439](https://github.com/Shopify/sarama/pull/439), + [#442](https://github.com/Shopify/sarama/pull/442)). + +Improvements: + - The producer's logging during retry scenarios is more consistent, more + useful, and slightly less verbose + ([#429](https://github.com/Shopify/sarama/pull/429)). + - The client now shuffles its initial list of seed brokers in order to prevent + thundering herd on the first broker in the list + ([#441](https://github.com/Shopify/sarama/pull/441)). + +Bug Fixes: + - The producer now correctly manages its state if retries occur when it is + shutting down, fixing several instances of confusing behaviour and at least + one potential deadlock ([#419](https://github.com/Shopify/sarama/pull/419)). + - The consumer now handles messages for different partitions asynchronously, + making it much more resilient to specific user code ordering + ([#325](https://github.com/Shopify/sarama/pull/325)). + +#### Version 1.3.0 (2015-04-16) + +New Features: + - The client now tracks consumer group coordinators using + ConsumerMetadataRequests similar to how it tracks partition leadership using + regular MetadataRequests ([#411](https://github.com/Shopify/sarama/pull/411)). + This adds two methods to the client API: + - `Coordinator(consumerGroup string) (*Broker, error)` + - `RefreshCoordinator(consumerGroup string) error` + +Improvements: + - ConsumerMetadataResponses now automatically create a Broker object out of the + ID/address/port combination for the Coordinator; accessing the fields + individually has been deprecated + ([#413](https://github.com/Shopify/sarama/pull/413)). + - Much improved handling of `OffsetOutOfRange` errors in the consumer. + Consumers will fail to start if the provided offset is out of range + ([#418](https://github.com/Shopify/sarama/pull/418)) + and they will automatically shut down if the offset falls out of range + ([#424](https://github.com/Shopify/sarama/pull/424)). + - Small performance improvement in encoding and decoding protocol messages + ([#427](https://github.com/Shopify/sarama/pull/427)). + +Bug Fixes: + - Fix a rare race condition in the client's background metadata refresher if + it happens to be activated while the client is being closed + ([#422](https://github.com/Shopify/sarama/pull/422)). + +#### Version 1.2.0 (2015-04-07) + +Improvements: + - The producer's behaviour when `Flush.Frequency` is set is now more intuitive + ([#389](https://github.com/Shopify/sarama/pull/389)). + - The producer is now somewhat more memory-efficient during and after retrying + messages due to an improved queue implementation + ([#396](https://github.com/Shopify/sarama/pull/396)). + - The consumer produces much more useful logging output when leadership + changes ([#385](https://github.com/Shopify/sarama/pull/385)). + - The client's `GetOffset` method will now automatically refresh metadata and + retry once in the event of stale information or similar + ([#394](https://github.com/Shopify/sarama/pull/394)). + - Broker connections now have support for using TCP keepalives + ([#407](https://github.com/Shopify/sarama/issues/407)). + +Bug Fixes: + - The OffsetCommitRequest message now correctly implements all three possible + API versions ([#390](https://github.com/Shopify/sarama/pull/390), + [#400](https://github.com/Shopify/sarama/pull/400)). + +#### Version 1.1.0 (2015-03-20) + +Improvements: + - Wrap the producer's partitioner call in a circuit-breaker so that repeatedly + broken topics don't choke throughput + ([#373](https://github.com/Shopify/sarama/pull/373)). + +Bug Fixes: + - Fix the producer's internal reference counting in certain unusual scenarios + ([#367](https://github.com/Shopify/sarama/pull/367)). + - Fix the consumer's internal reference counting in certain unusual scenarios + ([#369](https://github.com/Shopify/sarama/pull/369)). + - Fix a condition where the producer's internal control messages could have + gotten stuck ([#368](https://github.com/Shopify/sarama/pull/368)). + - Fix an issue where invalid partition lists would be cached when asking for + metadata for a non-existant topic ([#372](https://github.com/Shopify/sarama/pull/372)). + + +#### Version 1.0.0 (2015-03-17) + +Version 1.0.0 is the first tagged version, and is almost a complete rewrite. The primary differences with previous untagged versions are: + +- The producer has been rewritten; there is now a `SyncProducer` with a blocking API, and an `AsyncProducer` that is non-blocking. +- The consumer has been rewritten to only open one connection per broker instead of one connection per partition. +- The main types of Sarama are now interfaces to make depedency injection easy; mock implementations for `Consumer`, `SyncProducer` and `AsyncProducer` are provided in the `github.com/Shopify/sarama/mocks` package. +- For most uses cases, it is no longer necessary to open a `Client`; this will be done for you. +- All the configuration values have been unified in the `Config` struct. +- Much improved test suite. diff --git a/vendor/github.com/Shopify/sarama/LICENSE b/vendor/github.com/Shopify/sarama/LICENSE new file mode 100644 index 000000000000..d2bf4352f4c9 --- /dev/null +++ b/vendor/github.com/Shopify/sarama/LICENSE @@ -0,0 +1,20 @@ +Copyright (c) 2013 Shopify + +Permission is hereby granted, free of charge, to any person obtaining +a copy of this software and associated documentation files (the +"Software"), to deal in the Software without restriction, including +without limitation the rights to use, copy, modify, merge, publish, +distribute, sublicense, and/or sell copies of the Software, and to +permit persons to whom the Software is furnished to do so, subject to +the following conditions: + +The above copyright notice and this permission notice shall be +included in all copies or substantial portions of the Software. + +THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, +EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF +MERCHANTABILITY, FITNESS FOR A PARTICULAR PURPOSE AND +NONINFRINGEMENT. IN NO EVENT SHALL THE AUTHORS OR COPYRIGHT HOLDERS BE +LIABLE FOR ANY CLAIM, DAMAGES OR OTHER LIABILITY, WHETHER IN AN ACTION +OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, OUT OF OR IN CONNECTION +WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE SOFTWARE. diff --git a/vendor/github.com/Shopify/sarama/Makefile b/vendor/github.com/Shopify/sarama/Makefile new file mode 100644 index 000000000000..8f8fc6bdb0eb --- /dev/null +++ b/vendor/github.com/Shopify/sarama/Makefile @@ -0,0 +1,31 @@ +default: fmt get update test lint + +GO := go +GOBUILD := CGO_ENABLED=0 $(GO) build $(BUILD_FLAG) +GOTEST := $(GO) test -gcflags='-l' -p 3 -v -race -timeout 10m -coverprofile=profile.out -covermode=atomic + +FILES := $(shell find . -name '*.go' -type f -not -name '*.pb.go' -not -name '*_generated.go' -not -name '*_test.go') +TESTS := $(shell find . -name '*.go' -type f -not -name '*.pb.go' -not -name '*_generated.go' -name '*_test.go') + +get: + $(GO) get ./... + $(GO) mod verify + $(GO) mod tidy + +update: + $(GO) get -u -v ./... + $(GO) mod verify + $(GO) mod tidy + +fmt: + gofmt -s -l -w $(FILES) $(TESTS) + +lint: + GOFLAGS="-tags=functional" golangci-lint run + +test: + $(GOTEST) ./... + +.PHONY: test_functional +test_functional: + $(GOTEST) -tags=functional ./... diff --git a/vendor/github.com/Shopify/sarama/README.md b/vendor/github.com/Shopify/sarama/README.md new file mode 100644 index 000000000000..f66550a8b063 --- /dev/null +++ b/vendor/github.com/Shopify/sarama/README.md @@ -0,0 +1,35 @@ +# sarama + +[![Go Reference](https://pkg.go.dev/badge/github.com/Shopify/sarama.svg)](https://pkg.go.dev/github.com/Shopify/sarama) +[![Coverage](https://codecov.io/gh/Shopify/sarama/branch/main/graph/badge.svg)](https://codecov.io/gh/Shopify/sarama) + +Sarama is an MIT-licensed Go client library for [Apache Kafka](https://kafka.apache.org/) version 0.8 (and later). + +## Getting started + +- API documentation and examples are available via [pkg.go.dev](https://pkg.go.dev/github.com/Shopify/sarama). +- Mocks for testing are available in the [mocks](./mocks) subpackage. +- The [examples](./examples) directory contains more elaborate example applications. +- The [tools](./tools) directory contains command line tools that can be useful for testing, diagnostics, and instrumentation. + +You might also want to look at the [Frequently Asked Questions](https://github.com/Shopify/sarama/wiki/Frequently-Asked-Questions). + +## Compatibility and API stability + +Sarama provides a "2 releases + 2 months" compatibility guarantee: we support +the two latest stable releases of Kafka and Go, and we provide a two month +grace period for older releases. This means we currently officially support +Go 1.15 through 1.16, and Kafka 2.7 through 2.8, although older releases are +still likely to work. + +Sarama follows semantic versioning and provides API stability via the gopkg.in service. +You can import a version with a guaranteed stable API via http://gopkg.in/Shopify/sarama.v1. +A changelog is available [here](CHANGELOG.md). + +## Contributing + +- Get started by checking our [contribution guidelines](https://github.com/Shopify/sarama/blob/main/.github/CONTRIBUTING.md). +- Read the [Sarama wiki](https://github.com/Shopify/sarama/wiki) for more technical and design details. +- The [Kafka Protocol Specification](https://cwiki.apache.org/confluence/display/KAFKA/A+Guide+To+The+Kafka+Protocol) contains a wealth of useful information. +- For more general issues, there is [a google group](https://groups.google.com/forum/#!forum/kafka-clients) for Kafka client developers. +- If you have any questions, just ask! diff --git a/vendor/github.com/Shopify/sarama/Vagrantfile b/vendor/github.com/Shopify/sarama/Vagrantfile new file mode 100644 index 000000000000..07d7ffb8ff4a --- /dev/null +++ b/vendor/github.com/Shopify/sarama/Vagrantfile @@ -0,0 +1,14 @@ +# We have 5 * 192MB ZK processes and 5 * 320MB Kafka processes => 2560MB +MEMORY = 3072 + +Vagrant.configure("2") do |config| + config.vm.box = "ubuntu/bionic64" + + config.vm.provision :shell, path: "vagrant/provision.sh" + + config.vm.network "private_network", ip: "192.168.100.67" + + config.vm.provider "virtualbox" do |v| + v.memory = MEMORY + end +end diff --git a/vendor/github.com/Shopify/sarama/acl_bindings.go b/vendor/github.com/Shopify/sarama/acl_bindings.go new file mode 100644 index 000000000000..13440be677c1 --- /dev/null +++ b/vendor/github.com/Shopify/sarama/acl_bindings.go @@ -0,0 +1,138 @@ +package sarama + +// Resource holds information about acl resource type +type Resource struct { + ResourceType AclResourceType + ResourceName string + ResourcePatternType AclResourcePatternType +} + +func (r *Resource) encode(pe packetEncoder, version int16) error { + pe.putInt8(int8(r.ResourceType)) + + if err := pe.putString(r.ResourceName); err != nil { + return err + } + + if version == 1 { + if r.ResourcePatternType == AclPatternUnknown { + Logger.Print("Cannot encode an unknown resource pattern type, using Literal instead") + r.ResourcePatternType = AclPatternLiteral + } + pe.putInt8(int8(r.ResourcePatternType)) + } + + return nil +} + +func (r *Resource) decode(pd packetDecoder, version int16) (err error) { + resourceType, err := pd.getInt8() + if err != nil { + return err + } + r.ResourceType = AclResourceType(resourceType) + + if r.ResourceName, err = pd.getString(); err != nil { + return err + } + if version == 1 { + pattern, err := pd.getInt8() + if err != nil { + return err + } + r.ResourcePatternType = AclResourcePatternType(pattern) + } + + return nil +} + +// Acl holds information about acl type +type Acl struct { + Principal string + Host string + Operation AclOperation + PermissionType AclPermissionType +} + +func (a *Acl) encode(pe packetEncoder) error { + if err := pe.putString(a.Principal); err != nil { + return err + } + + if err := pe.putString(a.Host); err != nil { + return err + } + + pe.putInt8(int8(a.Operation)) + pe.putInt8(int8(a.PermissionType)) + + return nil +} + +func (a *Acl) decode(pd packetDecoder, version int16) (err error) { + if a.Principal, err = pd.getString(); err != nil { + return err + } + + if a.Host, err = pd.getString(); err != nil { + return err + } + + operation, err := pd.getInt8() + if err != nil { + return err + } + a.Operation = AclOperation(operation) + + permissionType, err := pd.getInt8() + if err != nil { + return err + } + a.PermissionType = AclPermissionType(permissionType) + + return nil +} + +// ResourceAcls is an acl resource type +type ResourceAcls struct { + Resource + Acls []*Acl +} + +func (r *ResourceAcls) encode(pe packetEncoder, version int16) error { + if err := r.Resource.encode(pe, version); err != nil { + return err + } + + if err := pe.putArrayLength(len(r.Acls)); err != nil { + return err + } + for _, acl := range r.Acls { + if err := acl.encode(pe); err != nil { + return err + } + } + + return nil +} + +func (r *ResourceAcls) decode(pd packetDecoder, version int16) error { + if err := r.Resource.decode(pd, version); err != nil { + return err + } + + n, err := pd.getArrayLength() + if err != nil { + return err + } + + r.Acls = make([]*Acl, n) + for i := 0; i < n; i++ { + r.Acls[i] = new(Acl) + if err := r.Acls[i].decode(pd, version); err != nil { + return err + } + } + + return nil +} diff --git a/vendor/github.com/Shopify/sarama/acl_create_request.go b/vendor/github.com/Shopify/sarama/acl_create_request.go new file mode 100644 index 000000000000..449102f74a7c --- /dev/null +++ b/vendor/github.com/Shopify/sarama/acl_create_request.go @@ -0,0 +1,89 @@ +package sarama + +// CreateAclsRequest is an acl creation request +type CreateAclsRequest struct { + Version int16 + AclCreations []*AclCreation +} + +func (c *CreateAclsRequest) encode(pe packetEncoder) error { + if err := pe.putArrayLength(len(c.AclCreations)); err != nil { + return err + } + + for _, aclCreation := range c.AclCreations { + if err := aclCreation.encode(pe, c.Version); err != nil { + return err + } + } + + return nil +} + +func (c *CreateAclsRequest) decode(pd packetDecoder, version int16) (err error) { + c.Version = version + n, err := pd.getArrayLength() + if err != nil { + return err + } + + c.AclCreations = make([]*AclCreation, n) + + for i := 0; i < n; i++ { + c.AclCreations[i] = new(AclCreation) + if err := c.AclCreations[i].decode(pd, version); err != nil { + return err + } + } + + return nil +} + +func (c *CreateAclsRequest) key() int16 { + return 30 +} + +func (c *CreateAclsRequest) version() int16 { + return c.Version +} + +func (c *CreateAclsRequest) headerVersion() int16 { + return 1 +} + +func (c *CreateAclsRequest) requiredVersion() KafkaVersion { + switch c.Version { + case 1: + return V2_0_0_0 + default: + return V0_11_0_0 + } +} + +// AclCreation is a wrapper around Resource and Acl type +type AclCreation struct { + Resource + Acl +} + +func (a *AclCreation) encode(pe packetEncoder, version int16) error { + if err := a.Resource.encode(pe, version); err != nil { + return err + } + if err := a.Acl.encode(pe); err != nil { + return err + } + + return nil +} + +func (a *AclCreation) decode(pd packetDecoder, version int16) (err error) { + if err := a.Resource.decode(pd, version); err != nil { + return err + } + if err := a.Acl.decode(pd, version); err != nil { + return err + } + + return nil +} diff --git a/vendor/github.com/Shopify/sarama/acl_create_response.go b/vendor/github.com/Shopify/sarama/acl_create_response.go new file mode 100644 index 000000000000..21d6c340cc5d --- /dev/null +++ b/vendor/github.com/Shopify/sarama/acl_create_response.go @@ -0,0 +1,94 @@ +package sarama + +import "time" + +// CreateAclsResponse is a an acl response creation type +type CreateAclsResponse struct { + ThrottleTime time.Duration + AclCreationResponses []*AclCreationResponse +} + +func (c *CreateAclsResponse) encode(pe packetEncoder) error { + pe.putInt32(int32(c.ThrottleTime / time.Millisecond)) + + if err := pe.putArrayLength(len(c.AclCreationResponses)); err != nil { + return err + } + + for _, aclCreationResponse := range c.AclCreationResponses { + if err := aclCreationResponse.encode(pe); err != nil { + return err + } + } + + return nil +} + +func (c *CreateAclsResponse) decode(pd packetDecoder, version int16) (err error) { + throttleTime, err := pd.getInt32() + if err != nil { + return err + } + c.ThrottleTime = time.Duration(throttleTime) * time.Millisecond + + n, err := pd.getArrayLength() + if err != nil { + return err + } + + c.AclCreationResponses = make([]*AclCreationResponse, n) + for i := 0; i < n; i++ { + c.AclCreationResponses[i] = new(AclCreationResponse) + if err := c.AclCreationResponses[i].decode(pd, version); err != nil { + return err + } + } + + return nil +} + +func (c *CreateAclsResponse) key() int16 { + return 30 +} + +func (c *CreateAclsResponse) version() int16 { + return 0 +} + +func (c *CreateAclsResponse) headerVersion() int16 { + return 0 +} + +func (c *CreateAclsResponse) requiredVersion() KafkaVersion { + return V0_11_0_0 +} + +// AclCreationResponse is an acl creation response type +type AclCreationResponse struct { + Err KError + ErrMsg *string +} + +func (a *AclCreationResponse) encode(pe packetEncoder) error { + pe.putInt16(int16(a.Err)) + + if err := pe.putNullableString(a.ErrMsg); err != nil { + return err + } + + return nil +} + +func (a *AclCreationResponse) decode(pd packetDecoder, version int16) (err error) { + kerr, err := pd.getInt16() + if err != nil { + return err + } + a.Err = KError(kerr) + + if a.ErrMsg, err = pd.getNullableString(); err != nil { + return err + } + + return nil +} diff --git a/vendor/github.com/Shopify/sarama/acl_delete_request.go b/vendor/github.com/Shopify/sarama/acl_delete_request.go new file mode 100644 index 000000000000..5e5c03bc2da2 --- /dev/null +++ b/vendor/github.com/Shopify/sarama/acl_delete_request.go @@ -0,0 +1,62 @@ +package sarama + +// DeleteAclsRequest is a delete acl request +type DeleteAclsRequest struct { + Version int + Filters []*AclFilter +} + +func (d *DeleteAclsRequest) encode(pe packetEncoder) error { + if err := pe.putArrayLength(len(d.Filters)); err != nil { + return err + } + + for _, filter := range d.Filters { + filter.Version = d.Version + if err := filter.encode(pe); err != nil { + return err + } + } + + return nil +} + +func (d *DeleteAclsRequest) decode(pd packetDecoder, version int16) (err error) { + d.Version = int(version) + n, err := pd.getArrayLength() + if err != nil { + return err + } + + d.Filters = make([]*AclFilter, n) + for i := 0; i < n; i++ { + d.Filters[i] = new(AclFilter) + d.Filters[i].Version = int(version) + if err := d.Filters[i].decode(pd, version); err != nil { + return err + } + } + + return nil +} + +func (d *DeleteAclsRequest) key() int16 { + return 31 +} + +func (d *DeleteAclsRequest) version() int16 { + return int16(d.Version) +} + +func (d *DeleteAclsRequest) headerVersion() int16 { + return 1 +} + +func (d *DeleteAclsRequest) requiredVersion() KafkaVersion { + switch d.Version { + case 1: + return V2_0_0_0 + default: + return V0_11_0_0 + } +} diff --git a/vendor/github.com/Shopify/sarama/acl_delete_response.go b/vendor/github.com/Shopify/sarama/acl_delete_response.go new file mode 100644 index 000000000000..cd33749d5e56 --- /dev/null +++ b/vendor/github.com/Shopify/sarama/acl_delete_response.go @@ -0,0 +1,163 @@ +package sarama + +import "time" + +// DeleteAclsResponse is a delete acl response +type DeleteAclsResponse struct { + Version int16 + ThrottleTime time.Duration + FilterResponses []*FilterResponse +} + +func (d *DeleteAclsResponse) encode(pe packetEncoder) error { + pe.putInt32(int32(d.ThrottleTime / time.Millisecond)) + + if err := pe.putArrayLength(len(d.FilterResponses)); err != nil { + return err + } + + for _, filterResponse := range d.FilterResponses { + if err := filterResponse.encode(pe, d.Version); err != nil { + return err + } + } + + return nil +} + +func (d *DeleteAclsResponse) decode(pd packetDecoder, version int16) (err error) { + throttleTime, err := pd.getInt32() + if err != nil { + return err + } + d.ThrottleTime = time.Duration(throttleTime) * time.Millisecond + + n, err := pd.getArrayLength() + if err != nil { + return err + } + d.FilterResponses = make([]*FilterResponse, n) + + for i := 0; i < n; i++ { + d.FilterResponses[i] = new(FilterResponse) + if err := d.FilterResponses[i].decode(pd, version); err != nil { + return err + } + } + + return nil +} + +func (d *DeleteAclsResponse) key() int16 { + return 31 +} + +func (d *DeleteAclsResponse) version() int16 { + return d.Version +} + +func (d *DeleteAclsResponse) headerVersion() int16 { + return 0 +} + +func (d *DeleteAclsResponse) requiredVersion() KafkaVersion { + return V0_11_0_0 +} + +// FilterResponse is a filter response type +type FilterResponse struct { + Err KError + ErrMsg *string + MatchingAcls []*MatchingAcl +} + +func (f *FilterResponse) encode(pe packetEncoder, version int16) error { + pe.putInt16(int16(f.Err)) + if err := pe.putNullableString(f.ErrMsg); err != nil { + return err + } + + if err := pe.putArrayLength(len(f.MatchingAcls)); err != nil { + return err + } + for _, matchingAcl := range f.MatchingAcls { + if err := matchingAcl.encode(pe, version); err != nil { + return err + } + } + + return nil +} + +func (f *FilterResponse) decode(pd packetDecoder, version int16) (err error) { + kerr, err := pd.getInt16() + if err != nil { + return err + } + f.Err = KError(kerr) + + if f.ErrMsg, err = pd.getNullableString(); err != nil { + return err + } + + n, err := pd.getArrayLength() + if err != nil { + return err + } + f.MatchingAcls = make([]*MatchingAcl, n) + for i := 0; i < n; i++ { + f.MatchingAcls[i] = new(MatchingAcl) + if err := f.MatchingAcls[i].decode(pd, version); err != nil { + return err + } + } + + return nil +} + +// MatchingAcl is a matching acl type +type MatchingAcl struct { + Err KError + ErrMsg *string + Resource + Acl +} + +func (m *MatchingAcl) encode(pe packetEncoder, version int16) error { + pe.putInt16(int16(m.Err)) + if err := pe.putNullableString(m.ErrMsg); err != nil { + return err + } + + if err := m.Resource.encode(pe, version); err != nil { + return err + } + + if err := m.Acl.encode(pe); err != nil { + return err + } + + return nil +} + +func (m *MatchingAcl) decode(pd packetDecoder, version int16) (err error) { + kerr, err := pd.getInt16() + if err != nil { + return err + } + m.Err = KError(kerr) + + if m.ErrMsg, err = pd.getNullableString(); err != nil { + return err + } + + if err := m.Resource.decode(pd, version); err != nil { + return err + } + + if err := m.Acl.decode(pd, version); err != nil { + return err + } + + return nil +} diff --git a/vendor/github.com/Shopify/sarama/acl_describe_request.go b/vendor/github.com/Shopify/sarama/acl_describe_request.go new file mode 100644 index 000000000000..e0fe9023a28b --- /dev/null +++ b/vendor/github.com/Shopify/sarama/acl_describe_request.go @@ -0,0 +1,39 @@ +package sarama + +// DescribeAclsRequest is a secribe acl request type +type DescribeAclsRequest struct { + Version int + AclFilter +} + +func (d *DescribeAclsRequest) encode(pe packetEncoder) error { + d.AclFilter.Version = d.Version + return d.AclFilter.encode(pe) +} + +func (d *DescribeAclsRequest) decode(pd packetDecoder, version int16) (err error) { + d.Version = int(version) + d.AclFilter.Version = int(version) + return d.AclFilter.decode(pd, version) +} + +func (d *DescribeAclsRequest) key() int16 { + return 29 +} + +func (d *DescribeAclsRequest) version() int16 { + return int16(d.Version) +} + +func (d *DescribeAclsRequest) headerVersion() int16 { + return 1 +} + +func (d *DescribeAclsRequest) requiredVersion() KafkaVersion { + switch d.Version { + case 1: + return V2_0_0_0 + default: + return V0_11_0_0 + } +} diff --git a/vendor/github.com/Shopify/sarama/acl_describe_response.go b/vendor/github.com/Shopify/sarama/acl_describe_response.go new file mode 100644 index 000000000000..3255fd48571a --- /dev/null +++ b/vendor/github.com/Shopify/sarama/acl_describe_response.go @@ -0,0 +1,91 @@ +package sarama + +import "time" + +// DescribeAclsResponse is a describe acl response type +type DescribeAclsResponse struct { + Version int16 + ThrottleTime time.Duration + Err KError + ErrMsg *string + ResourceAcls []*ResourceAcls +} + +func (d *DescribeAclsResponse) encode(pe packetEncoder) error { + pe.putInt32(int32(d.ThrottleTime / time.Millisecond)) + pe.putInt16(int16(d.Err)) + + if err := pe.putNullableString(d.ErrMsg); err != nil { + return err + } + + if err := pe.putArrayLength(len(d.ResourceAcls)); err != nil { + return err + } + + for _, resourceAcl := range d.ResourceAcls { + if err := resourceAcl.encode(pe, d.Version); err != nil { + return err + } + } + + return nil +} + +func (d *DescribeAclsResponse) decode(pd packetDecoder, version int16) (err error) { + throttleTime, err := pd.getInt32() + if err != nil { + return err + } + d.ThrottleTime = time.Duration(throttleTime) * time.Millisecond + + kerr, err := pd.getInt16() + if err != nil { + return err + } + d.Err = KError(kerr) + + errmsg, err := pd.getString() + if err != nil { + return err + } + if errmsg != "" { + d.ErrMsg = &errmsg + } + + n, err := pd.getArrayLength() + if err != nil { + return err + } + d.ResourceAcls = make([]*ResourceAcls, n) + + for i := 0; i < n; i++ { + d.ResourceAcls[i] = new(ResourceAcls) + if err := d.ResourceAcls[i].decode(pd, version); err != nil { + return err + } + } + + return nil +} + +func (d *DescribeAclsResponse) key() int16 { + return 29 +} + +func (d *DescribeAclsResponse) version() int16 { + return d.Version +} + +func (d *DescribeAclsResponse) headerVersion() int16 { + return 0 +} + +func (d *DescribeAclsResponse) requiredVersion() KafkaVersion { + switch d.Version { + case 1: + return V2_0_0_0 + default: + return V0_11_0_0 + } +} diff --git a/vendor/github.com/Shopify/sarama/acl_filter.go b/vendor/github.com/Shopify/sarama/acl_filter.go new file mode 100644 index 000000000000..b380161aa45a --- /dev/null +++ b/vendor/github.com/Shopify/sarama/acl_filter.go @@ -0,0 +1,77 @@ +package sarama + +type AclFilter struct { + Version int + ResourceType AclResourceType + ResourceName *string + ResourcePatternTypeFilter AclResourcePatternType + Principal *string + Host *string + Operation AclOperation + PermissionType AclPermissionType +} + +func (a *AclFilter) encode(pe packetEncoder) error { + pe.putInt8(int8(a.ResourceType)) + if err := pe.putNullableString(a.ResourceName); err != nil { + return err + } + + if a.Version == 1 { + pe.putInt8(int8(a.ResourcePatternTypeFilter)) + } + + if err := pe.putNullableString(a.Principal); err != nil { + return err + } + if err := pe.putNullableString(a.Host); err != nil { + return err + } + pe.putInt8(int8(a.Operation)) + pe.putInt8(int8(a.PermissionType)) + + return nil +} + +func (a *AclFilter) decode(pd packetDecoder, version int16) (err error) { + resourceType, err := pd.getInt8() + if err != nil { + return err + } + a.ResourceType = AclResourceType(resourceType) + + if a.ResourceName, err = pd.getNullableString(); err != nil { + return err + } + + if a.Version == 1 { + pattern, err := pd.getInt8() + if err != nil { + return err + } + + a.ResourcePatternTypeFilter = AclResourcePatternType(pattern) + } + + if a.Principal, err = pd.getNullableString(); err != nil { + return err + } + + if a.Host, err = pd.getNullableString(); err != nil { + return err + } + + operation, err := pd.getInt8() + if err != nil { + return err + } + a.Operation = AclOperation(operation) + + permissionType, err := pd.getInt8() + if err != nil { + return err + } + a.PermissionType = AclPermissionType(permissionType) + + return nil +} diff --git a/vendor/github.com/Shopify/sarama/acl_types.go b/vendor/github.com/Shopify/sarama/acl_types.go new file mode 100644 index 000000000000..c3ba8ddcf644 --- /dev/null +++ b/vendor/github.com/Shopify/sarama/acl_types.go @@ -0,0 +1,238 @@ +package sarama + +import ( + "fmt" + "strings" +) + +type ( + AclOperation int + + AclPermissionType int + + AclResourceType int + + AclResourcePatternType int +) + +// ref: https://github.com/apache/kafka/blob/trunk/clients/src/main/java/org/apache/kafka/common/acl/AclOperation.java +const ( + AclOperationUnknown AclOperation = iota + AclOperationAny + AclOperationAll + AclOperationRead + AclOperationWrite + AclOperationCreate + AclOperationDelete + AclOperationAlter + AclOperationDescribe + AclOperationClusterAction + AclOperationDescribeConfigs + AclOperationAlterConfigs + AclOperationIdempotentWrite +) + +func (a *AclOperation) String() string { + mapping := map[AclOperation]string{ + AclOperationUnknown: "Unknown", + AclOperationAny: "Any", + AclOperationAll: "All", + AclOperationRead: "Read", + AclOperationWrite: "Write", + AclOperationCreate: "Create", + AclOperationDelete: "Delete", + AclOperationAlter: "Alter", + AclOperationDescribe: "Describe", + AclOperationClusterAction: "ClusterAction", + AclOperationDescribeConfigs: "DescribeConfigs", + AclOperationAlterConfigs: "AlterConfigs", + AclOperationIdempotentWrite: "IdempotentWrite", + } + s, ok := mapping[*a] + if !ok { + s = mapping[AclOperationUnknown] + } + return s +} + +// MarshalText returns the text form of the AclOperation (name without prefix) +func (a *AclOperation) MarshalText() ([]byte, error) { + return []byte(a.String()), nil +} + +// UnmarshalText takes a text reprentation of the operation and converts it to an AclOperation +func (a *AclOperation) UnmarshalText(text []byte) error { + normalized := strings.ToLower(string(text)) + mapping := map[string]AclOperation{ + "unknown": AclOperationUnknown, + "any": AclOperationAny, + "all": AclOperationAll, + "read": AclOperationRead, + "write": AclOperationWrite, + "create": AclOperationCreate, + "delete": AclOperationDelete, + "alter": AclOperationAlter, + "describe": AclOperationDescribe, + "clusteraction": AclOperationClusterAction, + "describeconfigs": AclOperationDescribeConfigs, + "alterconfigs": AclOperationAlterConfigs, + "idempotentwrite": AclOperationIdempotentWrite, + } + ao, ok := mapping[normalized] + if !ok { + *a = AclOperationUnknown + return fmt.Errorf("no acl operation with name %s", normalized) + } + *a = ao + return nil +} + +// ref: https://github.com/apache/kafka/blob/trunk/clients/src/main/java/org/apache/kafka/common/acl/AclPermissionType.java +const ( + AclPermissionUnknown AclPermissionType = iota + AclPermissionAny + AclPermissionDeny + AclPermissionAllow +) + +func (a *AclPermissionType) String() string { + mapping := map[AclPermissionType]string{ + AclPermissionUnknown: "Unknown", + AclPermissionAny: "Any", + AclPermissionDeny: "Deny", + AclPermissionAllow: "Allow", + } + s, ok := mapping[*a] + if !ok { + s = mapping[AclPermissionUnknown] + } + return s +} + +// MarshalText returns the text form of the AclPermissionType (name without prefix) +func (a *AclPermissionType) MarshalText() ([]byte, error) { + return []byte(a.String()), nil +} + +// UnmarshalText takes a text reprentation of the permission type and converts it to an AclPermissionType +func (a *AclPermissionType) UnmarshalText(text []byte) error { + normalized := strings.ToLower(string(text)) + mapping := map[string]AclPermissionType{ + "unknown": AclPermissionUnknown, + "any": AclPermissionAny, + "deny": AclPermissionDeny, + "allow": AclPermissionAllow, + } + + apt, ok := mapping[normalized] + if !ok { + *a = AclPermissionUnknown + return fmt.Errorf("no acl permission with name %s", normalized) + } + *a = apt + return nil +} + +// ref: https://github.com/apache/kafka/blob/trunk/clients/src/main/java/org/apache/kafka/common/resource/ResourceType.java +const ( + AclResourceUnknown AclResourceType = iota + AclResourceAny + AclResourceTopic + AclResourceGroup + AclResourceCluster + AclResourceTransactionalID + AclResourceDelegationToken +) + +func (a *AclResourceType) String() string { + mapping := map[AclResourceType]string{ + AclResourceUnknown: "Unknown", + AclResourceAny: "Any", + AclResourceTopic: "Topic", + AclResourceGroup: "Group", + AclResourceCluster: "Cluster", + AclResourceTransactionalID: "TransactionalID", + AclResourceDelegationToken: "DelegationToken", + } + s, ok := mapping[*a] + if !ok { + s = mapping[AclResourceUnknown] + } + return s +} + +// MarshalText returns the text form of the AclResourceType (name without prefix) +func (a *AclResourceType) MarshalText() ([]byte, error) { + return []byte(a.String()), nil +} + +// UnmarshalText takes a text reprentation of the resource type and converts it to an AclResourceType +func (a *AclResourceType) UnmarshalText(text []byte) error { + normalized := strings.ToLower(string(text)) + mapping := map[string]AclResourceType{ + "unknown": AclResourceUnknown, + "any": AclResourceAny, + "topic": AclResourceTopic, + "group": AclResourceGroup, + "cluster": AclResourceCluster, + "transactionalid": AclResourceTransactionalID, + "delegationtoken": AclResourceDelegationToken, + } + + art, ok := mapping[normalized] + if !ok { + *a = AclResourceUnknown + return fmt.Errorf("no acl resource with name %s", normalized) + } + *a = art + return nil +} + +// ref: https://github.com/apache/kafka/blob/trunk/clients/src/main/java/org/apache/kafka/common/resource/PatternType.java +const ( + AclPatternUnknown AclResourcePatternType = iota + AclPatternAny + AclPatternMatch + AclPatternLiteral + AclPatternPrefixed +) + +func (a *AclResourcePatternType) String() string { + mapping := map[AclResourcePatternType]string{ + AclPatternUnknown: "Unknown", + AclPatternAny: "Any", + AclPatternMatch: "Match", + AclPatternLiteral: "Literal", + AclPatternPrefixed: "Prefixed", + } + s, ok := mapping[*a] + if !ok { + s = mapping[AclPatternUnknown] + } + return s +} + +// MarshalText returns the text form of the AclResourcePatternType (name without prefix) +func (a *AclResourcePatternType) MarshalText() ([]byte, error) { + return []byte(a.String()), nil +} + +// UnmarshalText takes a text reprentation of the resource pattern type and converts it to an AclResourcePatternType +func (a *AclResourcePatternType) UnmarshalText(text []byte) error { + normalized := strings.ToLower(string(text)) + mapping := map[string]AclResourcePatternType{ + "unknown": AclPatternUnknown, + "any": AclPatternAny, + "match": AclPatternMatch, + "literal": AclPatternLiteral, + "prefixed": AclPatternPrefixed, + } + + arpt, ok := mapping[normalized] + if !ok { + *a = AclPatternUnknown + return fmt.Errorf("no acl resource pattern with name %s", normalized) + } + *a = arpt + return nil +} diff --git a/vendor/github.com/Shopify/sarama/add_offsets_to_txn_request.go b/vendor/github.com/Shopify/sarama/add_offsets_to_txn_request.go new file mode 100644 index 000000000000..a96af9341788 --- /dev/null +++ b/vendor/github.com/Shopify/sarama/add_offsets_to_txn_request.go @@ -0,0 +1,57 @@ +package sarama + +// AddOffsetsToTxnRequest adds offsets to a transaction request +type AddOffsetsToTxnRequest struct { + TransactionalID string + ProducerID int64 + ProducerEpoch int16 + GroupID string +} + +func (a *AddOffsetsToTxnRequest) encode(pe packetEncoder) error { + if err := pe.putString(a.TransactionalID); err != nil { + return err + } + + pe.putInt64(a.ProducerID) + + pe.putInt16(a.ProducerEpoch) + + if err := pe.putString(a.GroupID); err != nil { + return err + } + + return nil +} + +func (a *AddOffsetsToTxnRequest) decode(pd packetDecoder, version int16) (err error) { + if a.TransactionalID, err = pd.getString(); err != nil { + return err + } + if a.ProducerID, err = pd.getInt64(); err != nil { + return err + } + if a.ProducerEpoch, err = pd.getInt16(); err != nil { + return err + } + if a.GroupID, err = pd.getString(); err != nil { + return err + } + return nil +} + +func (a *AddOffsetsToTxnRequest) key() int16 { + return 25 +} + +func (a *AddOffsetsToTxnRequest) version() int16 { + return 0 +} + +func (a *AddOffsetsToTxnRequest) headerVersion() int16 { + return 1 +} + +func (a *AddOffsetsToTxnRequest) requiredVersion() KafkaVersion { + return V0_11_0_0 +} diff --git a/vendor/github.com/Shopify/sarama/add_offsets_to_txn_response.go b/vendor/github.com/Shopify/sarama/add_offsets_to_txn_response.go new file mode 100644 index 000000000000..bb61973d16b6 --- /dev/null +++ b/vendor/github.com/Shopify/sarama/add_offsets_to_txn_response.go @@ -0,0 +1,49 @@ +package sarama + +import ( + "time" +) + +// AddOffsetsToTxnResponse is a response type for adding offsets to txns +type AddOffsetsToTxnResponse struct { + ThrottleTime time.Duration + Err KError +} + +func (a *AddOffsetsToTxnResponse) encode(pe packetEncoder) error { + pe.putInt32(int32(a.ThrottleTime / time.Millisecond)) + pe.putInt16(int16(a.Err)) + return nil +} + +func (a *AddOffsetsToTxnResponse) decode(pd packetDecoder, version int16) (err error) { + throttleTime, err := pd.getInt32() + if err != nil { + return err + } + a.ThrottleTime = time.Duration(throttleTime) * time.Millisecond + + kerr, err := pd.getInt16() + if err != nil { + return err + } + a.Err = KError(kerr) + + return nil +} + +func (a *AddOffsetsToTxnResponse) key() int16 { + return 25 +} + +func (a *AddOffsetsToTxnResponse) version() int16 { + return 0 +} + +func (a *AddOffsetsToTxnResponse) headerVersion() int16 { + return 0 +} + +func (a *AddOffsetsToTxnResponse) requiredVersion() KafkaVersion { + return V0_11_0_0 +} diff --git a/vendor/github.com/Shopify/sarama/add_partitions_to_txn_request.go b/vendor/github.com/Shopify/sarama/add_partitions_to_txn_request.go new file mode 100644 index 000000000000..57ecf64884de --- /dev/null +++ b/vendor/github.com/Shopify/sarama/add_partitions_to_txn_request.go @@ -0,0 +1,81 @@ +package sarama + +// AddPartitionsToTxnRequest is a add paartition request +type AddPartitionsToTxnRequest struct { + TransactionalID string + ProducerID int64 + ProducerEpoch int16 + TopicPartitions map[string][]int32 +} + +func (a *AddPartitionsToTxnRequest) encode(pe packetEncoder) error { + if err := pe.putString(a.TransactionalID); err != nil { + return err + } + pe.putInt64(a.ProducerID) + pe.putInt16(a.ProducerEpoch) + + if err := pe.putArrayLength(len(a.TopicPartitions)); err != nil { + return err + } + for topic, partitions := range a.TopicPartitions { + if err := pe.putString(topic); err != nil { + return err + } + if err := pe.putInt32Array(partitions); err != nil { + return err + } + } + + return nil +} + +func (a *AddPartitionsToTxnRequest) decode(pd packetDecoder, version int16) (err error) { + if a.TransactionalID, err = pd.getString(); err != nil { + return err + } + if a.ProducerID, err = pd.getInt64(); err != nil { + return err + } + if a.ProducerEpoch, err = pd.getInt16(); err != nil { + return err + } + + n, err := pd.getArrayLength() + if err != nil { + return err + } + + a.TopicPartitions = make(map[string][]int32) + for i := 0; i < n; i++ { + topic, err := pd.getString() + if err != nil { + return err + } + + partitions, err := pd.getInt32Array() + if err != nil { + return err + } + + a.TopicPartitions[topic] = partitions + } + + return nil +} + +func (a *AddPartitionsToTxnRequest) key() int16 { + return 24 +} + +func (a *AddPartitionsToTxnRequest) version() int16 { + return 0 +} + +func (a *AddPartitionsToTxnRequest) headerVersion() int16 { + return 1 +} + +func (a *AddPartitionsToTxnRequest) requiredVersion() KafkaVersion { + return V0_11_0_0 +} diff --git a/vendor/github.com/Shopify/sarama/add_partitions_to_txn_response.go b/vendor/github.com/Shopify/sarama/add_partitions_to_txn_response.go new file mode 100644 index 000000000000..098956507676 --- /dev/null +++ b/vendor/github.com/Shopify/sarama/add_partitions_to_txn_response.go @@ -0,0 +1,114 @@ +package sarama + +import ( + "time" +) + +// AddPartitionsToTxnResponse is a partition errors to transaction type +type AddPartitionsToTxnResponse struct { + ThrottleTime time.Duration + Errors map[string][]*PartitionError +} + +func (a *AddPartitionsToTxnResponse) encode(pe packetEncoder) error { + pe.putInt32(int32(a.ThrottleTime / time.Millisecond)) + if err := pe.putArrayLength(len(a.Errors)); err != nil { + return err + } + + for topic, e := range a.Errors { + if err := pe.putString(topic); err != nil { + return err + } + if err := pe.putArrayLength(len(e)); err != nil { + return err + } + for _, partitionError := range e { + if err := partitionError.encode(pe); err != nil { + return err + } + } + } + + return nil +} + +func (a *AddPartitionsToTxnResponse) decode(pd packetDecoder, version int16) (err error) { + throttleTime, err := pd.getInt32() + if err != nil { + return err + } + a.ThrottleTime = time.Duration(throttleTime) * time.Millisecond + + n, err := pd.getArrayLength() + if err != nil { + return err + } + + a.Errors = make(map[string][]*PartitionError) + + for i := 0; i < n; i++ { + topic, err := pd.getString() + if err != nil { + return err + } + + m, err := pd.getArrayLength() + if err != nil { + return err + } + + a.Errors[topic] = make([]*PartitionError, m) + + for j := 0; j < m; j++ { + a.Errors[topic][j] = new(PartitionError) + if err := a.Errors[topic][j].decode(pd, version); err != nil { + return err + } + } + } + + return nil +} + +func (a *AddPartitionsToTxnResponse) key() int16 { + return 24 +} + +func (a *AddPartitionsToTxnResponse) version() int16 { + return 0 +} + +func (a *AddPartitionsToTxnResponse) headerVersion() int16 { + return 0 +} + +func (a *AddPartitionsToTxnResponse) requiredVersion() KafkaVersion { + return V0_11_0_0 +} + +// PartitionError is a partition error type +type PartitionError struct { + Partition int32 + Err KError +} + +func (p *PartitionError) encode(pe packetEncoder) error { + pe.putInt32(p.Partition) + pe.putInt16(int16(p.Err)) + return nil +} + +func (p *PartitionError) decode(pd packetDecoder, version int16) (err error) { + if p.Partition, err = pd.getInt32(); err != nil { + return err + } + + kerr, err := pd.getInt16() + if err != nil { + return err + } + p.Err = KError(kerr) + + return nil +} diff --git a/vendor/github.com/Shopify/sarama/admin.go b/vendor/github.com/Shopify/sarama/admin.go new file mode 100644 index 000000000000..59fd9983e116 --- /dev/null +++ b/vendor/github.com/Shopify/sarama/admin.go @@ -0,0 +1,1108 @@ +package sarama + +import ( + "errors" + "fmt" + "math/rand" + "strconv" + "sync" + "time" +) + +// ClusterAdmin is the administrative client for Kafka, which supports managing and inspecting topics, +// brokers, configurations and ACLs. The minimum broker version required is 0.10.0.0. +// Methods with stricter requirements will specify the minimum broker version required. +// You MUST call Close() on a client to avoid leaks +type ClusterAdmin interface { + // Creates a new topic. This operation is supported by brokers with version 0.10.1.0 or higher. + // It may take several seconds after CreateTopic returns success for all the brokers + // to become aware that the topic has been created. During this time, listTopics + // may not return information about the new topic.The validateOnly option is supported from version 0.10.2.0. + CreateTopic(topic string, detail *TopicDetail, validateOnly bool) error + + // List the topics available in the cluster with the default options. + ListTopics() (map[string]TopicDetail, error) + + // Describe some topics in the cluster. + DescribeTopics(topics []string) (metadata []*TopicMetadata, err error) + + // Delete a topic. It may take several seconds after the DeleteTopic to returns success + // and for all the brokers to become aware that the topics are gone. + // During this time, listTopics may continue to return information about the deleted topic. + // If delete.topic.enable is false on the brokers, deleteTopic will mark + // the topic for deletion, but not actually delete them. + // This operation is supported by brokers with version 0.10.1.0 or higher. + DeleteTopic(topic string) error + + // Increase the number of partitions of the topics according to the corresponding values. + // If partitions are increased for a topic that has a key, the partition logic or ordering of + // the messages will be affected. It may take several seconds after this method returns + // success for all the brokers to become aware that the partitions have been created. + // During this time, ClusterAdmin#describeTopics may not return information about the + // new partitions. This operation is supported by brokers with version 1.0.0 or higher. + CreatePartitions(topic string, count int32, assignment [][]int32, validateOnly bool) error + + // Alter the replica assignment for partitions. + // This operation is supported by brokers with version 2.4.0.0 or higher. + AlterPartitionReassignments(topic string, assignment [][]int32) error + + // Provides info on ongoing partitions replica reassignments. + // This operation is supported by brokers with version 2.4.0.0 or higher. + ListPartitionReassignments(topics string, partitions []int32) (topicStatus map[string]map[int32]*PartitionReplicaReassignmentsStatus, err error) + + // Delete records whose offset is smaller than the given offset of the corresponding partition. + // This operation is supported by brokers with version 0.11.0.0 or higher. + DeleteRecords(topic string, partitionOffsets map[int32]int64) error + + // Get the configuration for the specified resources. + // The returned configuration includes default values and the Default is true + // can be used to distinguish them from user supplied values. + // Config entries where ReadOnly is true cannot be updated. + // The value of config entries where Sensitive is true is always nil so + // sensitive information is not disclosed. + // This operation is supported by brokers with version 0.11.0.0 or higher. + DescribeConfig(resource ConfigResource) ([]ConfigEntry, error) + + // Update the configuration for the specified resources with the default options. + // This operation is supported by brokers with version 0.11.0.0 or higher. + // The resources with their configs (topic is the only resource type with configs + // that can be updated currently Updates are not transactional so they may succeed + // for some resources while fail for others. The configs for a particular resource are updated automatically. + AlterConfig(resourceType ConfigResourceType, name string, entries map[string]*string, validateOnly bool) error + + // Creates access control lists (ACLs) which are bound to specific resources. + // This operation is not transactional so it may succeed for some ACLs while fail for others. + // If you attempt to add an ACL that duplicates an existing ACL, no error will be raised, but + // no changes will be made. This operation is supported by brokers with version 0.11.0.0 or higher. + CreateACL(resource Resource, acl Acl) error + + // Lists access control lists (ACLs) according to the supplied filter. + // it may take some time for changes made by createAcls or deleteAcls to be reflected in the output of ListAcls + // This operation is supported by brokers with version 0.11.0.0 or higher. + ListAcls(filter AclFilter) ([]ResourceAcls, error) + + // Deletes access control lists (ACLs) according to the supplied filters. + // This operation is not transactional so it may succeed for some ACLs while fail for others. + // This operation is supported by brokers with version 0.11.0.0 or higher. + DeleteACL(filter AclFilter, validateOnly bool) ([]MatchingAcl, error) + + // List the consumer groups available in the cluster. + ListConsumerGroups() (map[string]string, error) + + // Describe the given consumer groups. + DescribeConsumerGroups(groups []string) ([]*GroupDescription, error) + + // List the consumer group offsets available in the cluster. + ListConsumerGroupOffsets(group string, topicPartitions map[string][]int32) (*OffsetFetchResponse, error) + + // Deletes a consumer group offset + DeleteConsumerGroupOffset(group string, topic string, partition int32) error + + // Delete a consumer group. + DeleteConsumerGroup(group string) error + + // Get information about the nodes in the cluster + DescribeCluster() (brokers []*Broker, controllerID int32, err error) + + // Get information about all log directories on the given set of brokers + DescribeLogDirs(brokers []int32) (map[int32][]DescribeLogDirsResponseDirMetadata, error) + + // Get information about SCRAM users + DescribeUserScramCredentials(users []string) ([]*DescribeUserScramCredentialsResult, error) + + // Delete SCRAM users + DeleteUserScramCredentials(delete []AlterUserScramCredentialsDelete) ([]*AlterUserScramCredentialsResult, error) + + // Upsert SCRAM users + UpsertUserScramCredentials(upsert []AlterUserScramCredentialsUpsert) ([]*AlterUserScramCredentialsResult, error) + + // Get client quota configurations corresponding to the specified filter. + // This operation is supported by brokers with version 2.6.0.0 or higher. + DescribeClientQuotas(components []QuotaFilterComponent, strict bool) ([]DescribeClientQuotasEntry, error) + + // Alters client quota configurations with the specified alterations. + // This operation is supported by brokers with version 2.6.0.0 or higher. + AlterClientQuotas(entity []QuotaEntityComponent, op ClientQuotasOp, validateOnly bool) error + + // Controller returns the cluster controller broker. It will return a + // locally cached value if it's available. + Controller() (*Broker, error) + + // Close shuts down the admin and closes underlying client. + Close() error +} + +type clusterAdmin struct { + client Client + conf *Config +} + +// NewClusterAdmin creates a new ClusterAdmin using the given broker addresses and configuration. +func NewClusterAdmin(addrs []string, conf *Config) (ClusterAdmin, error) { + client, err := NewClient(addrs, conf) + if err != nil { + return nil, err + } + return NewClusterAdminFromClient(client) +} + +// NewClusterAdminFromClient creates a new ClusterAdmin using the given client. +// Note that underlying client will also be closed on admin's Close() call. +func NewClusterAdminFromClient(client Client) (ClusterAdmin, error) { + // make sure we can retrieve the controller + _, err := client.Controller() + if err != nil { + return nil, err + } + + ca := &clusterAdmin{ + client: client, + conf: client.Config(), + } + return ca, nil +} + +func (ca *clusterAdmin) Close() error { + return ca.client.Close() +} + +func (ca *clusterAdmin) Controller() (*Broker, error) { + return ca.client.Controller() +} + +func (ca *clusterAdmin) refreshController() (*Broker, error) { + return ca.client.RefreshController() +} + +// isErrNoController returns `true` if the given error type unwraps to an +// `ErrNotController` response from Kafka +func isErrNoController(err error) bool { + switch e := err.(type) { + case *TopicError: + return e.Err == ErrNotController + case *TopicPartitionError: + return e.Err == ErrNotController + case KError: + return e == ErrNotController + } + return false +} + +// retryOnError will repeatedly call the given (error-returning) func in the +// case that its response is non-nil and retryable (as determined by the +// provided retryable func) up to the maximum number of tries permitted by +// the admin client configuration +func (ca *clusterAdmin) retryOnError(retryable func(error) bool, fn func() error) error { + var err error + for attempt := 0; attempt < ca.conf.Admin.Retry.Max; attempt++ { + err = fn() + if err == nil || !retryable(err) { + return err + } + Logger.Printf( + "admin/request retrying after %dms... (%d attempts remaining)\n", + ca.conf.Admin.Retry.Backoff/time.Millisecond, ca.conf.Admin.Retry.Max-attempt) + time.Sleep(ca.conf.Admin.Retry.Backoff) + continue + } + return err +} + +func (ca *clusterAdmin) CreateTopic(topic string, detail *TopicDetail, validateOnly bool) error { + if topic == "" { + return ErrInvalidTopic + } + + if detail == nil { + return errors.New("you must specify topic details") + } + + topicDetails := make(map[string]*TopicDetail) + topicDetails[topic] = detail + + request := &CreateTopicsRequest{ + TopicDetails: topicDetails, + ValidateOnly: validateOnly, + Timeout: ca.conf.Admin.Timeout, + } + + if ca.conf.Version.IsAtLeast(V0_11_0_0) { + request.Version = 1 + } + if ca.conf.Version.IsAtLeast(V1_0_0_0) { + request.Version = 2 + } + + return ca.retryOnError(isErrNoController, func() error { + b, err := ca.Controller() + if err != nil { + return err + } + + rsp, err := b.CreateTopics(request) + if err != nil { + return err + } + + topicErr, ok := rsp.TopicErrors[topic] + if !ok { + return ErrIncompleteResponse + } + + if topicErr.Err != ErrNoError { + if topicErr.Err == ErrNotController { + _, _ = ca.refreshController() + } + return topicErr + } + + return nil + }) +} + +func (ca *clusterAdmin) DescribeTopics(topics []string) (metadata []*TopicMetadata, err error) { + controller, err := ca.Controller() + if err != nil { + return nil, err + } + + request := &MetadataRequest{ + Topics: topics, + AllowAutoTopicCreation: false, + } + + if ca.conf.Version.IsAtLeast(V1_0_0_0) { + request.Version = 5 + } else if ca.conf.Version.IsAtLeast(V0_11_0_0) { + request.Version = 4 + } + + response, err := controller.GetMetadata(request) + if err != nil { + return nil, err + } + return response.Topics, nil +} + +func (ca *clusterAdmin) DescribeCluster() (brokers []*Broker, controllerID int32, err error) { + controller, err := ca.Controller() + if err != nil { + return nil, int32(0), err + } + + request := &MetadataRequest{ + Topics: []string{}, + } + + if ca.conf.Version.IsAtLeast(V0_10_0_0) { + request.Version = 1 + } + + response, err := controller.GetMetadata(request) + if err != nil { + return nil, int32(0), err + } + + return response.Brokers, response.ControllerID, nil +} + +func (ca *clusterAdmin) findBroker(id int32) (*Broker, error) { + brokers := ca.client.Brokers() + for _, b := range brokers { + if b.ID() == id { + return b, nil + } + } + return nil, fmt.Errorf("could not find broker id %d", id) +} + +func (ca *clusterAdmin) findAnyBroker() (*Broker, error) { + brokers := ca.client.Brokers() + if len(brokers) > 0 { + index := rand.Intn(len(brokers)) + return brokers[index], nil + } + return nil, errors.New("no available broker") +} + +func (ca *clusterAdmin) ListTopics() (map[string]TopicDetail, error) { + // In order to build TopicDetails we need to first get the list of all + // topics using a MetadataRequest and then get their configs using a + // DescribeConfigsRequest request. To avoid sending many requests to the + // broker, we use a single DescribeConfigsRequest. + + // Send the all-topic MetadataRequest + b, err := ca.findAnyBroker() + if err != nil { + return nil, err + } + _ = b.Open(ca.client.Config()) + + metadataReq := &MetadataRequest{} + metadataResp, err := b.GetMetadata(metadataReq) + if err != nil { + return nil, err + } + + topicsDetailsMap := make(map[string]TopicDetail) + + var describeConfigsResources []*ConfigResource + + for _, topic := range metadataResp.Topics { + topicDetails := TopicDetail{ + NumPartitions: int32(len(topic.Partitions)), + } + if len(topic.Partitions) > 0 { + topicDetails.ReplicaAssignment = map[int32][]int32{} + for _, partition := range topic.Partitions { + topicDetails.ReplicaAssignment[partition.ID] = partition.Replicas + } + topicDetails.ReplicationFactor = int16(len(topic.Partitions[0].Replicas)) + } + topicsDetailsMap[topic.Name] = topicDetails + + // we populate the resources we want to describe from the MetadataResponse + topicResource := ConfigResource{ + Type: TopicResource, + Name: topic.Name, + } + describeConfigsResources = append(describeConfigsResources, &topicResource) + } + + // Send the DescribeConfigsRequest + describeConfigsReq := &DescribeConfigsRequest{ + Resources: describeConfigsResources, + } + + if ca.conf.Version.IsAtLeast(V1_1_0_0) { + describeConfigsReq.Version = 1 + } + + if ca.conf.Version.IsAtLeast(V2_0_0_0) { + describeConfigsReq.Version = 2 + } + + describeConfigsResp, err := b.DescribeConfigs(describeConfigsReq) + if err != nil { + return nil, err + } + + for _, resource := range describeConfigsResp.Resources { + topicDetails := topicsDetailsMap[resource.Name] + topicDetails.ConfigEntries = make(map[string]*string) + + for _, entry := range resource.Configs { + // only include non-default non-sensitive config + // (don't actually think topic config will ever be sensitive) + if entry.Default || entry.Sensitive { + continue + } + topicDetails.ConfigEntries[entry.Name] = &entry.Value + } + + topicsDetailsMap[resource.Name] = topicDetails + } + + return topicsDetailsMap, nil +} + +func (ca *clusterAdmin) DeleteTopic(topic string) error { + if topic == "" { + return ErrInvalidTopic + } + + request := &DeleteTopicsRequest{ + Topics: []string{topic}, + Timeout: ca.conf.Admin.Timeout, + } + + if ca.conf.Version.IsAtLeast(V0_11_0_0) { + request.Version = 1 + } + + return ca.retryOnError(isErrNoController, func() error { + b, err := ca.Controller() + if err != nil { + return err + } + + rsp, err := b.DeleteTopics(request) + if err != nil { + return err + } + + topicErr, ok := rsp.TopicErrorCodes[topic] + if !ok { + return ErrIncompleteResponse + } + + if topicErr != ErrNoError { + if topicErr == ErrNotController { + _, _ = ca.refreshController() + } + return topicErr + } + + return nil + }) +} + +func (ca *clusterAdmin) CreatePartitions(topic string, count int32, assignment [][]int32, validateOnly bool) error { + if topic == "" { + return ErrInvalidTopic + } + + topicPartitions := make(map[string]*TopicPartition) + topicPartitions[topic] = &TopicPartition{Count: count, Assignment: assignment} + + request := &CreatePartitionsRequest{ + TopicPartitions: topicPartitions, + Timeout: ca.conf.Admin.Timeout, + ValidateOnly: validateOnly, + } + + return ca.retryOnError(isErrNoController, func() error { + b, err := ca.Controller() + if err != nil { + return err + } + + rsp, err := b.CreatePartitions(request) + if err != nil { + return err + } + + topicErr, ok := rsp.TopicPartitionErrors[topic] + if !ok { + return ErrIncompleteResponse + } + + if topicErr.Err != ErrNoError { + if topicErr.Err == ErrNotController { + _, _ = ca.refreshController() + } + return topicErr + } + + return nil + }) +} + +func (ca *clusterAdmin) AlterPartitionReassignments(topic string, assignment [][]int32) error { + if topic == "" { + return ErrInvalidTopic + } + + request := &AlterPartitionReassignmentsRequest{ + TimeoutMs: int32(60000), + Version: int16(0), + } + + for i := 0; i < len(assignment); i++ { + request.AddBlock(topic, int32(i), assignment[i]) + } + + return ca.retryOnError(isErrNoController, func() error { + b, err := ca.Controller() + if err != nil { + return err + } + + errs := make([]error, 0) + + rsp, err := b.AlterPartitionReassignments(request) + + if err != nil { + errs = append(errs, err) + } else { + if rsp.ErrorCode > 0 { + errs = append(errs, errors.New(rsp.ErrorCode.Error())) + } + + for topic, topicErrors := range rsp.Errors { + for partition, partitionError := range topicErrors { + if partitionError.errorCode != ErrNoError { + errStr := fmt.Sprintf("[%s-%d]: %s", topic, partition, partitionError.errorCode.Error()) + errs = append(errs, errors.New(errStr)) + } + } + } + } + + if len(errs) > 0 { + return ErrReassignPartitions{MultiError{&errs}} + } + + return nil + }) +} + +func (ca *clusterAdmin) ListPartitionReassignments(topic string, partitions []int32) (topicStatus map[string]map[int32]*PartitionReplicaReassignmentsStatus, err error) { + if topic == "" { + return nil, ErrInvalidTopic + } + + request := &ListPartitionReassignmentsRequest{ + TimeoutMs: int32(60000), + Version: int16(0), + } + + request.AddBlock(topic, partitions) + + b, err := ca.Controller() + if err != nil { + return nil, err + } + _ = b.Open(ca.client.Config()) + + rsp, err := b.ListPartitionReassignments(request) + + if err == nil && rsp != nil { + return rsp.TopicStatus, nil + } else { + return nil, err + } +} + +func (ca *clusterAdmin) DeleteRecords(topic string, partitionOffsets map[int32]int64) error { + if topic == "" { + return ErrInvalidTopic + } + partitionPerBroker := make(map[*Broker][]int32) + for partition := range partitionOffsets { + broker, err := ca.client.Leader(topic, partition) + if err != nil { + return err + } + partitionPerBroker[broker] = append(partitionPerBroker[broker], partition) + } + errs := make([]error, 0) + for broker, partitions := range partitionPerBroker { + topics := make(map[string]*DeleteRecordsRequestTopic) + recordsToDelete := make(map[int32]int64) + for _, p := range partitions { + recordsToDelete[p] = partitionOffsets[p] + } + topics[topic] = &DeleteRecordsRequestTopic{PartitionOffsets: recordsToDelete} + request := &DeleteRecordsRequest{ + Topics: topics, + Timeout: ca.conf.Admin.Timeout, + } + + rsp, err := broker.DeleteRecords(request) + if err != nil { + errs = append(errs, err) + } else { + deleteRecordsResponseTopic, ok := rsp.Topics[topic] + if !ok { + errs = append(errs, ErrIncompleteResponse) + } else { + for _, deleteRecordsResponsePartition := range deleteRecordsResponseTopic.Partitions { + if deleteRecordsResponsePartition.Err != ErrNoError { + errs = append(errs, errors.New(deleteRecordsResponsePartition.Err.Error())) + } + } + } + } + } + if len(errs) > 0 { + return ErrDeleteRecords{MultiError{&errs}} + } + // todo since we are dealing with couple of partitions it would be good if we return slice of errors + // for each partition instead of one error + return nil +} + +// Returns a bool indicating whether the resource request needs to go to a +// specific broker +func dependsOnSpecificNode(resource ConfigResource) bool { + return (resource.Type == BrokerResource && resource.Name != "") || + resource.Type == BrokerLoggerResource +} + +func (ca *clusterAdmin) DescribeConfig(resource ConfigResource) ([]ConfigEntry, error) { + var entries []ConfigEntry + var resources []*ConfigResource + resources = append(resources, &resource) + + request := &DescribeConfigsRequest{ + Resources: resources, + } + + if ca.conf.Version.IsAtLeast(V1_1_0_0) { + request.Version = 1 + } + + if ca.conf.Version.IsAtLeast(V2_0_0_0) { + request.Version = 2 + } + + var ( + b *Broker + err error + ) + + // DescribeConfig of broker/broker logger must be sent to the broker in question + if dependsOnSpecificNode(resource) { + var id int64 + id, err = strconv.ParseInt(resource.Name, 10, 32) + if err != nil { + return nil, err + } + b, err = ca.findBroker(int32(id)) + } else { + b, err = ca.findAnyBroker() + } + if err != nil { + return nil, err + } + + _ = b.Open(ca.client.Config()) + rsp, err := b.DescribeConfigs(request) + if err != nil { + return nil, err + } + + for _, rspResource := range rsp.Resources { + if rspResource.Name == resource.Name { + if rspResource.ErrorMsg != "" { + return nil, errors.New(rspResource.ErrorMsg) + } + if rspResource.ErrorCode != 0 { + return nil, KError(rspResource.ErrorCode) + } + for _, cfgEntry := range rspResource.Configs { + entries = append(entries, *cfgEntry) + } + } + } + return entries, nil +} + +func (ca *clusterAdmin) AlterConfig(resourceType ConfigResourceType, name string, entries map[string]*string, validateOnly bool) error { + var resources []*AlterConfigsResource + resources = append(resources, &AlterConfigsResource{ + Type: resourceType, + Name: name, + ConfigEntries: entries, + }) + + request := &AlterConfigsRequest{ + Resources: resources, + ValidateOnly: validateOnly, + } + + var ( + b *Broker + err error + ) + + // AlterConfig of broker/broker logger must be sent to the broker in question + if dependsOnSpecificNode(ConfigResource{Name: name, Type: resourceType}) { + var id int64 + id, err = strconv.ParseInt(name, 10, 32) + if err != nil { + return err + } + b, err = ca.findBroker(int32(id)) + } else { + b, err = ca.findAnyBroker() + } + if err != nil { + return err + } + + _ = b.Open(ca.client.Config()) + rsp, err := b.AlterConfigs(request) + if err != nil { + return err + } + + for _, rspResource := range rsp.Resources { + if rspResource.Name == name { + if rspResource.ErrorMsg != "" { + return errors.New(rspResource.ErrorMsg) + } + if rspResource.ErrorCode != 0 { + return KError(rspResource.ErrorCode) + } + } + } + return nil +} + +func (ca *clusterAdmin) CreateACL(resource Resource, acl Acl) error { + var acls []*AclCreation + acls = append(acls, &AclCreation{resource, acl}) + request := &CreateAclsRequest{AclCreations: acls} + + if ca.conf.Version.IsAtLeast(V2_0_0_0) { + request.Version = 1 + } + + b, err := ca.Controller() + if err != nil { + return err + } + + _, err = b.CreateAcls(request) + return err +} + +func (ca *clusterAdmin) ListAcls(filter AclFilter) ([]ResourceAcls, error) { + request := &DescribeAclsRequest{AclFilter: filter} + + if ca.conf.Version.IsAtLeast(V2_0_0_0) { + request.Version = 1 + } + + b, err := ca.Controller() + if err != nil { + return nil, err + } + + rsp, err := b.DescribeAcls(request) + if err != nil { + return nil, err + } + + var lAcls []ResourceAcls + for _, rAcl := range rsp.ResourceAcls { + lAcls = append(lAcls, *rAcl) + } + return lAcls, nil +} + +func (ca *clusterAdmin) DeleteACL(filter AclFilter, validateOnly bool) ([]MatchingAcl, error) { + var filters []*AclFilter + filters = append(filters, &filter) + request := &DeleteAclsRequest{Filters: filters} + + if ca.conf.Version.IsAtLeast(V2_0_0_0) { + request.Version = 1 + } + + b, err := ca.Controller() + if err != nil { + return nil, err + } + + rsp, err := b.DeleteAcls(request) + if err != nil { + return nil, err + } + + var mAcls []MatchingAcl + for _, fr := range rsp.FilterResponses { + for _, mACL := range fr.MatchingAcls { + mAcls = append(mAcls, *mACL) + } + } + return mAcls, nil +} + +func (ca *clusterAdmin) DescribeConsumerGroups(groups []string) (result []*GroupDescription, err error) { + groupsPerBroker := make(map[*Broker][]string) + + for _, group := range groups { + controller, err := ca.client.Coordinator(group) + if err != nil { + return nil, err + } + groupsPerBroker[controller] = append(groupsPerBroker[controller], group) + } + + for broker, brokerGroups := range groupsPerBroker { + response, err := broker.DescribeGroups(&DescribeGroupsRequest{ + Groups: brokerGroups, + }) + if err != nil { + return nil, err + } + + result = append(result, response.Groups...) + } + return result, nil +} + +func (ca *clusterAdmin) ListConsumerGroups() (allGroups map[string]string, err error) { + allGroups = make(map[string]string) + + // Query brokers in parallel, since we have to query *all* brokers + brokers := ca.client.Brokers() + groupMaps := make(chan map[string]string, len(brokers)) + errChan := make(chan error, len(brokers)) + wg := sync.WaitGroup{} + + for _, b := range brokers { + wg.Add(1) + go func(b *Broker, conf *Config) { + defer wg.Done() + _ = b.Open(conf) // Ensure that broker is opened + + response, err := b.ListGroups(&ListGroupsRequest{}) + if err != nil { + errChan <- err + return + } + + groups := make(map[string]string) + for group, typ := range response.Groups { + groups[group] = typ + } + + groupMaps <- groups + }(b, ca.conf) + } + + wg.Wait() + close(groupMaps) + close(errChan) + + for groupMap := range groupMaps { + for group, protocolType := range groupMap { + allGroups[group] = protocolType + } + } + + // Intentionally return only the first error for simplicity + err = <-errChan + return +} + +func (ca *clusterAdmin) ListConsumerGroupOffsets(group string, topicPartitions map[string][]int32) (*OffsetFetchResponse, error) { + coordinator, err := ca.client.Coordinator(group) + if err != nil { + return nil, err + } + + request := &OffsetFetchRequest{ + ConsumerGroup: group, + partitions: topicPartitions, + } + + if ca.conf.Version.IsAtLeast(V0_10_2_0) { + request.Version = 2 + } else if ca.conf.Version.IsAtLeast(V0_8_2_2) { + request.Version = 1 + } + + return coordinator.FetchOffset(request) +} + +func (ca *clusterAdmin) DeleteConsumerGroupOffset(group string, topic string, partition int32) error { + coordinator, err := ca.client.Coordinator(group) + if err != nil { + return err + } + + request := &DeleteOffsetsRequest{ + Group: group, + partitions: map[string][]int32{ + topic: {partition}, + }, + } + + resp, err := coordinator.DeleteOffsets(request) + if err != nil { + return err + } + + if resp.ErrorCode != ErrNoError { + return resp.ErrorCode + } + + if resp.Errors[topic][partition] != ErrNoError { + return resp.Errors[topic][partition] + } + return nil +} + +func (ca *clusterAdmin) DeleteConsumerGroup(group string) error { + coordinator, err := ca.client.Coordinator(group) + if err != nil { + return err + } + + request := &DeleteGroupsRequest{ + Groups: []string{group}, + } + + resp, err := coordinator.DeleteGroups(request) + if err != nil { + return err + } + + groupErr, ok := resp.GroupErrorCodes[group] + if !ok { + return ErrIncompleteResponse + } + + if groupErr != ErrNoError { + return groupErr + } + + return nil +} + +func (ca *clusterAdmin) DescribeLogDirs(brokerIds []int32) (allLogDirs map[int32][]DescribeLogDirsResponseDirMetadata, err error) { + allLogDirs = make(map[int32][]DescribeLogDirsResponseDirMetadata) + + // Query brokers in parallel, since we may have to query multiple brokers + logDirsMaps := make(chan map[int32][]DescribeLogDirsResponseDirMetadata, len(brokerIds)) + errChan := make(chan error, len(brokerIds)) + wg := sync.WaitGroup{} + + for _, b := range brokerIds { + wg.Add(1) + broker, err := ca.findBroker(b) + if err != nil { + Logger.Printf("Unable to find broker with ID = %v\n", b) + continue + } + go func(b *Broker, conf *Config) { + defer wg.Done() + _ = b.Open(conf) // Ensure that broker is opened + + response, err := b.DescribeLogDirs(&DescribeLogDirsRequest{}) + if err != nil { + errChan <- err + return + } + logDirs := make(map[int32][]DescribeLogDirsResponseDirMetadata) + logDirs[b.ID()] = response.LogDirs + logDirsMaps <- logDirs + }(broker, ca.conf) + } + + wg.Wait() + close(logDirsMaps) + close(errChan) + + for logDirsMap := range logDirsMaps { + for id, logDirs := range logDirsMap { + allLogDirs[id] = logDirs + } + } + + // Intentionally return only the first error for simplicity + err = <-errChan + return +} + +func (ca *clusterAdmin) DescribeUserScramCredentials(users []string) ([]*DescribeUserScramCredentialsResult, error) { + req := &DescribeUserScramCredentialsRequest{} + for _, u := range users { + req.DescribeUsers = append(req.DescribeUsers, DescribeUserScramCredentialsRequestUser{ + Name: u, + }) + } + + b, err := ca.Controller() + if err != nil { + return nil, err + } + + rsp, err := b.DescribeUserScramCredentials(req) + if err != nil { + return nil, err + } + + return rsp.Results, nil +} + +func (ca *clusterAdmin) UpsertUserScramCredentials(upsert []AlterUserScramCredentialsUpsert) ([]*AlterUserScramCredentialsResult, error) { + res, err := ca.AlterUserScramCredentials(upsert, nil) + if err != nil { + return nil, err + } + + return res, nil +} + +func (ca *clusterAdmin) DeleteUserScramCredentials(delete []AlterUserScramCredentialsDelete) ([]*AlterUserScramCredentialsResult, error) { + res, err := ca.AlterUserScramCredentials(nil, delete) + if err != nil { + return nil, err + } + + return res, nil +} + +func (ca *clusterAdmin) AlterUserScramCredentials(u []AlterUserScramCredentialsUpsert, d []AlterUserScramCredentialsDelete) ([]*AlterUserScramCredentialsResult, error) { + req := &AlterUserScramCredentialsRequest{ + Deletions: d, + Upsertions: u, + } + + b, err := ca.Controller() + if err != nil { + return nil, err + } + + rsp, err := b.AlterUserScramCredentials(req) + if err != nil { + return nil, err + } + + return rsp.Results, nil +} + +// Describe All : use an empty/nil components slice + strict = false +// Contains components: strict = false +// Contains only components: strict = true +func (ca *clusterAdmin) DescribeClientQuotas(components []QuotaFilterComponent, strict bool) ([]DescribeClientQuotasEntry, error) { + request := &DescribeClientQuotasRequest{ + Components: components, + Strict: strict, + } + + b, err := ca.Controller() + if err != nil { + return nil, err + } + + rsp, err := b.DescribeClientQuotas(request) + if err != nil { + return nil, err + } + + if rsp.ErrorMsg != nil { + return nil, errors.New(*rsp.ErrorMsg) + } + if rsp.ErrorCode != ErrNoError { + return nil, rsp.ErrorCode + } + + return rsp.Entries, nil +} + +func (ca *clusterAdmin) AlterClientQuotas(entity []QuotaEntityComponent, op ClientQuotasOp, validateOnly bool) error { + entry := AlterClientQuotasEntry{ + Entity: entity, + Ops: []ClientQuotasOp{op}, + } + + request := &AlterClientQuotasRequest{ + Entries: []AlterClientQuotasEntry{entry}, + ValidateOnly: validateOnly, + } + + b, err := ca.Controller() + if err != nil { + return err + } + + rsp, err := b.AlterClientQuotas(request) + if err != nil { + return err + } + + for _, entry := range rsp.Entries { + if entry.ErrorCode != ErrNoError { + return entry.ErrorCode + } + } + + return nil +} diff --git a/vendor/github.com/Shopify/sarama/alter_client_quotas_request.go b/vendor/github.com/Shopify/sarama/alter_client_quotas_request.go new file mode 100644 index 000000000000..f528512d024a --- /dev/null +++ b/vendor/github.com/Shopify/sarama/alter_client_quotas_request.go @@ -0,0 +1,194 @@ +package sarama + +// AlterClientQuotas Request (Version: 0) => [entries] validate_only +// entries => [entity] [ops] +// entity => entity_type entity_name +// entity_type => STRING +// entity_name => NULLABLE_STRING +// ops => key value remove +// key => STRING +// value => FLOAT64 +// remove => BOOLEAN +// validate_only => BOOLEAN + +type AlterClientQuotasRequest struct { + Entries []AlterClientQuotasEntry // The quota configuration entries to alter. + ValidateOnly bool // Whether the alteration should be validated, but not performed. +} + +type AlterClientQuotasEntry struct { + Entity []QuotaEntityComponent // The quota entity to alter. + Ops []ClientQuotasOp // An individual quota configuration entry to alter. +} + +type ClientQuotasOp struct { + Key string // The quota configuration key. + Value float64 // The value to set, otherwise ignored if the value is to be removed. + Remove bool // Whether the quota configuration value should be removed, otherwise set. +} + +func (a *AlterClientQuotasRequest) encode(pe packetEncoder) error { + // Entries + if err := pe.putArrayLength(len(a.Entries)); err != nil { + return err + } + for _, e := range a.Entries { + if err := e.encode(pe); err != nil { + return err + } + } + + // ValidateOnly + pe.putBool(a.ValidateOnly) + + return nil +} + +func (a *AlterClientQuotasRequest) decode(pd packetDecoder, version int16) error { + // Entries + entryCount, err := pd.getArrayLength() + if err != nil { + return err + } + if entryCount > 0 { + a.Entries = make([]AlterClientQuotasEntry, entryCount) + for i := range a.Entries { + e := AlterClientQuotasEntry{} + if err = e.decode(pd, version); err != nil { + return err + } + a.Entries[i] = e + } + } else { + a.Entries = []AlterClientQuotasEntry{} + } + + // ValidateOnly + validateOnly, err := pd.getBool() + if err != nil { + return err + } + a.ValidateOnly = validateOnly + + return nil +} + +func (a *AlterClientQuotasEntry) encode(pe packetEncoder) error { + // Entity + if err := pe.putArrayLength(len(a.Entity)); err != nil { + return err + } + for _, component := range a.Entity { + if err := component.encode(pe); err != nil { + return err + } + } + + // Ops + if err := pe.putArrayLength(len(a.Ops)); err != nil { + return err + } + for _, o := range a.Ops { + if err := o.encode(pe); err != nil { + return err + } + } + + return nil +} + +func (a *AlterClientQuotasEntry) decode(pd packetDecoder, version int16) error { + // Entity + componentCount, err := pd.getArrayLength() + if err != nil { + return err + } + if componentCount > 0 { + a.Entity = make([]QuotaEntityComponent, componentCount) + for i := 0; i < componentCount; i++ { + component := QuotaEntityComponent{} + if err := component.decode(pd, version); err != nil { + return err + } + a.Entity[i] = component + } + } else { + a.Entity = []QuotaEntityComponent{} + } + + // Ops + opCount, err := pd.getArrayLength() + if err != nil { + return err + } + if opCount > 0 { + a.Ops = make([]ClientQuotasOp, opCount) + for i := range a.Ops { + c := ClientQuotasOp{} + if err = c.decode(pd, version); err != nil { + return err + } + a.Ops[i] = c + } + } else { + a.Ops = []ClientQuotasOp{} + } + + return nil +} + +func (c *ClientQuotasOp) encode(pe packetEncoder) error { + // Key + if err := pe.putString(c.Key); err != nil { + return err + } + + // Value + pe.putFloat64(c.Value) + + // Remove + pe.putBool(c.Remove) + + return nil +} + +func (c *ClientQuotasOp) decode(pd packetDecoder, version int16) error { + // Key + key, err := pd.getString() + if err != nil { + return err + } + c.Key = key + + // Value + value, err := pd.getFloat64() + if err != nil { + return err + } + c.Value = value + + // Remove + remove, err := pd.getBool() + if err != nil { + return err + } + c.Remove = remove + + return nil +} + +func (a *AlterClientQuotasRequest) key() int16 { + return 49 +} + +func (a *AlterClientQuotasRequest) version() int16 { + return 0 +} + +func (a *AlterClientQuotasRequest) headerVersion() int16 { + return 1 +} + +func (a *AlterClientQuotasRequest) requiredVersion() KafkaVersion { + return V2_6_0_0 +} diff --git a/vendor/github.com/Shopify/sarama/alter_client_quotas_response.go b/vendor/github.com/Shopify/sarama/alter_client_quotas_response.go new file mode 100644 index 000000000000..ccd27d5f5ece --- /dev/null +++ b/vendor/github.com/Shopify/sarama/alter_client_quotas_response.go @@ -0,0 +1,145 @@ +package sarama + +import ( + "time" +) + +// AlterClientQuotas Response (Version: 0) => throttle_time_ms [entries] +// throttle_time_ms => INT32 +// entries => error_code error_message [entity] +// error_code => INT16 +// error_message => NULLABLE_STRING +// entity => entity_type entity_name +// entity_type => STRING +// entity_name => NULLABLE_STRING + +type AlterClientQuotasResponse struct { + ThrottleTime time.Duration // The duration in milliseconds for which the request was throttled due to a quota violation, or zero if the request did not violate any quota. + Entries []AlterClientQuotasEntryResponse // The quota configuration entries altered. +} + +type AlterClientQuotasEntryResponse struct { + ErrorCode KError // The error code, or `0` if the quota alteration succeeded. + ErrorMsg *string // The error message, or `null` if the quota alteration succeeded. + Entity []QuotaEntityComponent // The quota entity altered. +} + +func (a *AlterClientQuotasResponse) encode(pe packetEncoder) error { + // ThrottleTime + pe.putInt32(int32(a.ThrottleTime / time.Millisecond)) + + // Entries + if err := pe.putArrayLength(len(a.Entries)); err != nil { + return err + } + for _, e := range a.Entries { + if err := e.encode(pe); err != nil { + return err + } + } + + return nil +} + +func (a *AlterClientQuotasResponse) decode(pd packetDecoder, version int16) error { + // ThrottleTime + throttleTime, err := pd.getInt32() + if err != nil { + return err + } + a.ThrottleTime = time.Duration(throttleTime) * time.Millisecond + + // Entries + entryCount, err := pd.getArrayLength() + if err != nil { + return err + } + if entryCount > 0 { + a.Entries = make([]AlterClientQuotasEntryResponse, entryCount) + for i := range a.Entries { + e := AlterClientQuotasEntryResponse{} + if err = e.decode(pd, version); err != nil { + return err + } + a.Entries[i] = e + } + } else { + a.Entries = []AlterClientQuotasEntryResponse{} + } + + return nil +} + +func (a *AlterClientQuotasEntryResponse) encode(pe packetEncoder) error { + // ErrorCode + pe.putInt16(int16(a.ErrorCode)) + + // ErrorMsg + if err := pe.putNullableString(a.ErrorMsg); err != nil { + return err + } + + // Entity + if err := pe.putArrayLength(len(a.Entity)); err != nil { + return err + } + for _, component := range a.Entity { + if err := component.encode(pe); err != nil { + return err + } + } + + return nil +} + +func (a *AlterClientQuotasEntryResponse) decode(pd packetDecoder, version int16) error { + // ErrorCode + errCode, err := pd.getInt16() + if err != nil { + return err + } + a.ErrorCode = KError(errCode) + + // ErrorMsg + errMsg, err := pd.getNullableString() + if err != nil { + return err + } + a.ErrorMsg = errMsg + + // Entity + componentCount, err := pd.getArrayLength() + if err != nil { + return err + } + if componentCount > 0 { + a.Entity = make([]QuotaEntityComponent, componentCount) + for i := 0; i < componentCount; i++ { + component := QuotaEntityComponent{} + if err := component.decode(pd, version); err != nil { + return err + } + a.Entity[i] = component + } + } else { + a.Entity = []QuotaEntityComponent{} + } + + return nil +} + +func (a *AlterClientQuotasResponse) key() int16 { + return 49 +} + +func (a *AlterClientQuotasResponse) version() int16 { + return 0 +} + +func (a *AlterClientQuotasResponse) headerVersion() int16 { + return 0 +} + +func (a *AlterClientQuotasResponse) requiredVersion() KafkaVersion { + return V2_6_0_0 +} diff --git a/vendor/github.com/Shopify/sarama/alter_configs_request.go b/vendor/github.com/Shopify/sarama/alter_configs_request.go new file mode 100644 index 000000000000..8b94b1f3fe40 --- /dev/null +++ b/vendor/github.com/Shopify/sarama/alter_configs_request.go @@ -0,0 +1,126 @@ +package sarama + +// AlterConfigsRequest is an alter config request type +type AlterConfigsRequest struct { + Resources []*AlterConfigsResource + ValidateOnly bool +} + +// AlterConfigsResource is an alter config resource type +type AlterConfigsResource struct { + Type ConfigResourceType + Name string + ConfigEntries map[string]*string +} + +func (a *AlterConfigsRequest) encode(pe packetEncoder) error { + if err := pe.putArrayLength(len(a.Resources)); err != nil { + return err + } + + for _, r := range a.Resources { + if err := r.encode(pe); err != nil { + return err + } + } + + pe.putBool(a.ValidateOnly) + return nil +} + +func (a *AlterConfigsRequest) decode(pd packetDecoder, version int16) error { + resourceCount, err := pd.getArrayLength() + if err != nil { + return err + } + + a.Resources = make([]*AlterConfigsResource, resourceCount) + for i := range a.Resources { + r := &AlterConfigsResource{} + err = r.decode(pd, version) + if err != nil { + return err + } + a.Resources[i] = r + } + + validateOnly, err := pd.getBool() + if err != nil { + return err + } + + a.ValidateOnly = validateOnly + + return nil +} + +func (a *AlterConfigsResource) encode(pe packetEncoder) error { + pe.putInt8(int8(a.Type)) + + if err := pe.putString(a.Name); err != nil { + return err + } + + if err := pe.putArrayLength(len(a.ConfigEntries)); err != nil { + return err + } + for configKey, configValue := range a.ConfigEntries { + if err := pe.putString(configKey); err != nil { + return err + } + if err := pe.putNullableString(configValue); err != nil { + return err + } + } + + return nil +} + +func (a *AlterConfigsResource) decode(pd packetDecoder, version int16) error { + t, err := pd.getInt8() + if err != nil { + return err + } + a.Type = ConfigResourceType(t) + + name, err := pd.getString() + if err != nil { + return err + } + a.Name = name + + n, err := pd.getArrayLength() + if err != nil { + return err + } + + if n > 0 { + a.ConfigEntries = make(map[string]*string, n) + for i := 0; i < n; i++ { + configKey, err := pd.getString() + if err != nil { + return err + } + if a.ConfigEntries[configKey], err = pd.getNullableString(); err != nil { + return err + } + } + } + return err +} + +func (a *AlterConfigsRequest) key() int16 { + return 33 +} + +func (a *AlterConfigsRequest) version() int16 { + return 0 +} + +func (a *AlterConfigsRequest) headerVersion() int16 { + return 1 +} + +func (a *AlterConfigsRequest) requiredVersion() KafkaVersion { + return V0_11_0_0 +} diff --git a/vendor/github.com/Shopify/sarama/alter_configs_response.go b/vendor/github.com/Shopify/sarama/alter_configs_response.go new file mode 100644 index 000000000000..84cd86c72920 --- /dev/null +++ b/vendor/github.com/Shopify/sarama/alter_configs_response.go @@ -0,0 +1,116 @@ +package sarama + +import "time" + +// AlterConfigsResponse is a response type for alter config +type AlterConfigsResponse struct { + ThrottleTime time.Duration + Resources []*AlterConfigsResourceResponse +} + +// AlterConfigsResourceResponse is a response type for alter config resource +type AlterConfigsResourceResponse struct { + ErrorCode int16 + ErrorMsg string + Type ConfigResourceType + Name string +} + +func (a *AlterConfigsResponse) encode(pe packetEncoder) error { + pe.putInt32(int32(a.ThrottleTime / time.Millisecond)) + + if err := pe.putArrayLength(len(a.Resources)); err != nil { + return err + } + + for _, v := range a.Resources { + if err := v.encode(pe); err != nil { + return err + } + } + + return nil +} + +func (a *AlterConfigsResponse) decode(pd packetDecoder, version int16) error { + throttleTime, err := pd.getInt32() + if err != nil { + return err + } + a.ThrottleTime = time.Duration(throttleTime) * time.Millisecond + + responseCount, err := pd.getArrayLength() + if err != nil { + return err + } + + a.Resources = make([]*AlterConfigsResourceResponse, responseCount) + + for i := range a.Resources { + a.Resources[i] = new(AlterConfigsResourceResponse) + + if err := a.Resources[i].decode(pd, version); err != nil { + return err + } + } + + return nil +} + +func (a *AlterConfigsResourceResponse) encode(pe packetEncoder) error { + pe.putInt16(a.ErrorCode) + err := pe.putString(a.ErrorMsg) + if err != nil { + return err + } + pe.putInt8(int8(a.Type)) + err = pe.putString(a.Name) + if err != nil { + return err + } + return nil +} + +func (a *AlterConfigsResourceResponse) decode(pd packetDecoder, version int16) error { + errCode, err := pd.getInt16() + if err != nil { + return err + } + a.ErrorCode = errCode + + e, err := pd.getString() + if err != nil { + return err + } + a.ErrorMsg = e + + t, err := pd.getInt8() + if err != nil { + return err + } + a.Type = ConfigResourceType(t) + + name, err := pd.getString() + if err != nil { + return err + } + a.Name = name + + return nil +} + +func (a *AlterConfigsResponse) key() int16 { + return 32 +} + +func (a *AlterConfigsResponse) version() int16 { + return 0 +} + +func (a *AlterConfigsResponse) headerVersion() int16 { + return 0 +} + +func (a *AlterConfigsResponse) requiredVersion() KafkaVersion { + return V0_11_0_0 +} diff --git a/vendor/github.com/Shopify/sarama/alter_partition_reassignments_request.go b/vendor/github.com/Shopify/sarama/alter_partition_reassignments_request.go new file mode 100644 index 000000000000..f0a2f9dd59b1 --- /dev/null +++ b/vendor/github.com/Shopify/sarama/alter_partition_reassignments_request.go @@ -0,0 +1,130 @@ +package sarama + +type alterPartitionReassignmentsBlock struct { + replicas []int32 +} + +func (b *alterPartitionReassignmentsBlock) encode(pe packetEncoder) error { + if err := pe.putNullableCompactInt32Array(b.replicas); err != nil { + return err + } + + pe.putEmptyTaggedFieldArray() + return nil +} + +func (b *alterPartitionReassignmentsBlock) decode(pd packetDecoder) (err error) { + if b.replicas, err = pd.getCompactInt32Array(); err != nil { + return err + } + return nil +} + +type AlterPartitionReassignmentsRequest struct { + TimeoutMs int32 + blocks map[string]map[int32]*alterPartitionReassignmentsBlock + Version int16 +} + +func (r *AlterPartitionReassignmentsRequest) encode(pe packetEncoder) error { + pe.putInt32(r.TimeoutMs) + + pe.putCompactArrayLength(len(r.blocks)) + + for topic, partitions := range r.blocks { + if err := pe.putCompactString(topic); err != nil { + return err + } + pe.putCompactArrayLength(len(partitions)) + for partition, block := range partitions { + pe.putInt32(partition) + if err := block.encode(pe); err != nil { + return err + } + } + pe.putEmptyTaggedFieldArray() + } + + pe.putEmptyTaggedFieldArray() + + return nil +} + +func (r *AlterPartitionReassignmentsRequest) decode(pd packetDecoder, version int16) (err error) { + r.Version = version + + if r.TimeoutMs, err = pd.getInt32(); err != nil { + return err + } + + topicCount, err := pd.getCompactArrayLength() + if err != nil { + return err + } + if topicCount > 0 { + r.blocks = make(map[string]map[int32]*alterPartitionReassignmentsBlock) + for i := 0; i < topicCount; i++ { + topic, err := pd.getCompactString() + if err != nil { + return err + } + partitionCount, err := pd.getCompactArrayLength() + if err != nil { + return err + } + r.blocks[topic] = make(map[int32]*alterPartitionReassignmentsBlock) + for j := 0; j < partitionCount; j++ { + partition, err := pd.getInt32() + if err != nil { + return err + } + block := &alterPartitionReassignmentsBlock{} + if err := block.decode(pd); err != nil { + return err + } + r.blocks[topic][partition] = block + + if _, err := pd.getEmptyTaggedFieldArray(); err != nil { + return err + } + } + if _, err := pd.getEmptyTaggedFieldArray(); err != nil { + return err + } + } + } + + if _, err := pd.getEmptyTaggedFieldArray(); err != nil { + return err + } + + return +} + +func (r *AlterPartitionReassignmentsRequest) key() int16 { + return 45 +} + +func (r *AlterPartitionReassignmentsRequest) version() int16 { + return r.Version +} + +func (r *AlterPartitionReassignmentsRequest) headerVersion() int16 { + return 2 +} + +func (r *AlterPartitionReassignmentsRequest) requiredVersion() KafkaVersion { + return V2_4_0_0 +} + +func (r *AlterPartitionReassignmentsRequest) AddBlock(topic string, partitionID int32, replicas []int32) { + if r.blocks == nil { + r.blocks = make(map[string]map[int32]*alterPartitionReassignmentsBlock) + } + + if r.blocks[topic] == nil { + r.blocks[topic] = make(map[int32]*alterPartitionReassignmentsBlock) + } + + r.blocks[topic][partitionID] = &alterPartitionReassignmentsBlock{replicas} +} diff --git a/vendor/github.com/Shopify/sarama/alter_partition_reassignments_response.go b/vendor/github.com/Shopify/sarama/alter_partition_reassignments_response.go new file mode 100644 index 000000000000..b3f9a15fe7f6 --- /dev/null +++ b/vendor/github.com/Shopify/sarama/alter_partition_reassignments_response.go @@ -0,0 +1,157 @@ +package sarama + +type alterPartitionReassignmentsErrorBlock struct { + errorCode KError + errorMessage *string +} + +func (b *alterPartitionReassignmentsErrorBlock) encode(pe packetEncoder) error { + pe.putInt16(int16(b.errorCode)) + if err := pe.putNullableCompactString(b.errorMessage); err != nil { + return err + } + pe.putEmptyTaggedFieldArray() + + return nil +} + +func (b *alterPartitionReassignmentsErrorBlock) decode(pd packetDecoder) (err error) { + errorCode, err := pd.getInt16() + if err != nil { + return err + } + b.errorCode = KError(errorCode) + b.errorMessage, err = pd.getCompactNullableString() + + if _, err := pd.getEmptyTaggedFieldArray(); err != nil { + return err + } + return err +} + +type AlterPartitionReassignmentsResponse struct { + Version int16 + ThrottleTimeMs int32 + ErrorCode KError + ErrorMessage *string + Errors map[string]map[int32]*alterPartitionReassignmentsErrorBlock +} + +func (r *AlterPartitionReassignmentsResponse) AddError(topic string, partition int32, kerror KError, message *string) { + if r.Errors == nil { + r.Errors = make(map[string]map[int32]*alterPartitionReassignmentsErrorBlock) + } + partitions := r.Errors[topic] + if partitions == nil { + partitions = make(map[int32]*alterPartitionReassignmentsErrorBlock) + r.Errors[topic] = partitions + } + + partitions[partition] = &alterPartitionReassignmentsErrorBlock{errorCode: kerror, errorMessage: message} +} + +func (r *AlterPartitionReassignmentsResponse) encode(pe packetEncoder) error { + pe.putInt32(r.ThrottleTimeMs) + pe.putInt16(int16(r.ErrorCode)) + if err := pe.putNullableCompactString(r.ErrorMessage); err != nil { + return err + } + + pe.putCompactArrayLength(len(r.Errors)) + for topic, partitions := range r.Errors { + if err := pe.putCompactString(topic); err != nil { + return err + } + pe.putCompactArrayLength(len(partitions)) + for partition, block := range partitions { + pe.putInt32(partition) + + if err := block.encode(pe); err != nil { + return err + } + } + pe.putEmptyTaggedFieldArray() + } + + pe.putEmptyTaggedFieldArray() + return nil +} + +func (r *AlterPartitionReassignmentsResponse) decode(pd packetDecoder, version int16) (err error) { + r.Version = version + + if r.ThrottleTimeMs, err = pd.getInt32(); err != nil { + return err + } + + kerr, err := pd.getInt16() + if err != nil { + return err + } + + r.ErrorCode = KError(kerr) + + if r.ErrorMessage, err = pd.getCompactNullableString(); err != nil { + return err + } + + numTopics, err := pd.getCompactArrayLength() + if err != nil { + return err + } + + if numTopics > 0 { + r.Errors = make(map[string]map[int32]*alterPartitionReassignmentsErrorBlock, numTopics) + for i := 0; i < numTopics; i++ { + topic, err := pd.getCompactString() + if err != nil { + return err + } + + ongoingPartitionReassignments, err := pd.getCompactArrayLength() + if err != nil { + return err + } + + r.Errors[topic] = make(map[int32]*alterPartitionReassignmentsErrorBlock, ongoingPartitionReassignments) + + for j := 0; j < ongoingPartitionReassignments; j++ { + partition, err := pd.getInt32() + if err != nil { + return err + } + block := &alterPartitionReassignmentsErrorBlock{} + if err := block.decode(pd); err != nil { + return err + } + + r.Errors[topic][partition] = block + } + if _, err = pd.getEmptyTaggedFieldArray(); err != nil { + return err + } + } + } + + if _, err = pd.getEmptyTaggedFieldArray(); err != nil { + return err + } + + return nil +} + +func (r *AlterPartitionReassignmentsResponse) key() int16 { + return 45 +} + +func (r *AlterPartitionReassignmentsResponse) version() int16 { + return r.Version +} + +func (r *AlterPartitionReassignmentsResponse) headerVersion() int16 { + return 1 +} + +func (r *AlterPartitionReassignmentsResponse) requiredVersion() KafkaVersion { + return V2_4_0_0 +} diff --git a/vendor/github.com/Shopify/sarama/alter_user_scram_credentials_request.go b/vendor/github.com/Shopify/sarama/alter_user_scram_credentials_request.go new file mode 100644 index 000000000000..0530d8946a8f --- /dev/null +++ b/vendor/github.com/Shopify/sarama/alter_user_scram_credentials_request.go @@ -0,0 +1,142 @@ +package sarama + +type AlterUserScramCredentialsRequest struct { + Version int16 + + // Deletions represent list of SCRAM credentials to remove + Deletions []AlterUserScramCredentialsDelete + + // Upsertions represent list of SCRAM credentials to update/insert + Upsertions []AlterUserScramCredentialsUpsert +} + +type AlterUserScramCredentialsDelete struct { + Name string + Mechanism ScramMechanismType +} + +type AlterUserScramCredentialsUpsert struct { + Name string + Mechanism ScramMechanismType + Iterations int32 + Salt []byte + saltedPassword []byte + + // This field is never transmitted over the wire + // @see: https://tools.ietf.org/html/rfc5802 + Password []byte +} + +func (r *AlterUserScramCredentialsRequest) encode(pe packetEncoder) error { + pe.putCompactArrayLength(len(r.Deletions)) + for _, d := range r.Deletions { + if err := pe.putCompactString(d.Name); err != nil { + return err + } + pe.putInt8(int8(d.Mechanism)) + pe.putEmptyTaggedFieldArray() + } + + pe.putCompactArrayLength(len(r.Upsertions)) + for _, u := range r.Upsertions { + if err := pe.putCompactString(u.Name); err != nil { + return err + } + pe.putInt8(int8(u.Mechanism)) + pe.putInt32(u.Iterations) + + if err := pe.putCompactBytes(u.Salt); err != nil { + return err + } + + // do not transmit the password over the wire + formatter := scramFormatter{mechanism: u.Mechanism} + salted, err := formatter.saltedPassword(u.Password, u.Salt, int(u.Iterations)) + if err != nil { + return err + } + + if err := pe.putCompactBytes(salted); err != nil { + return err + } + pe.putEmptyTaggedFieldArray() + } + + pe.putEmptyTaggedFieldArray() + return nil +} + +func (r *AlterUserScramCredentialsRequest) decode(pd packetDecoder, version int16) error { + numDeletions, err := pd.getCompactArrayLength() + if err != nil { + return err + } + + r.Deletions = make([]AlterUserScramCredentialsDelete, numDeletions) + for i := 0; i < numDeletions; i++ { + r.Deletions[i] = AlterUserScramCredentialsDelete{} + if r.Deletions[i].Name, err = pd.getCompactString(); err != nil { + return err + } + mechanism, err := pd.getInt8() + if err != nil { + return err + } + r.Deletions[i].Mechanism = ScramMechanismType(mechanism) + if _, err = pd.getEmptyTaggedFieldArray(); err != nil { + return err + } + } + + numUpsertions, err := pd.getCompactArrayLength() + if err != nil { + return err + } + + r.Upsertions = make([]AlterUserScramCredentialsUpsert, numUpsertions) + for i := 0; i < numUpsertions; i++ { + r.Upsertions[i] = AlterUserScramCredentialsUpsert{} + if r.Upsertions[i].Name, err = pd.getCompactString(); err != nil { + return err + } + mechanism, err := pd.getInt8() + if err != nil { + return err + } + + r.Upsertions[i].Mechanism = ScramMechanismType(mechanism) + if r.Upsertions[i].Iterations, err = pd.getInt32(); err != nil { + return err + } + if r.Upsertions[i].Salt, err = pd.getCompactBytes(); err != nil { + return err + } + if r.Upsertions[i].saltedPassword, err = pd.getCompactBytes(); err != nil { + return err + } + if _, err = pd.getEmptyTaggedFieldArray(); err != nil { + return err + } + } + + if _, err = pd.getEmptyTaggedFieldArray(); err != nil { + return err + } + return nil +} + +func (r *AlterUserScramCredentialsRequest) key() int16 { + return 51 +} + +func (r *AlterUserScramCredentialsRequest) version() int16 { + return r.Version +} + +func (r *AlterUserScramCredentialsRequest) headerVersion() int16 { + return 2 +} + +func (r *AlterUserScramCredentialsRequest) requiredVersion() KafkaVersion { + return V2_7_0_0 +} diff --git a/vendor/github.com/Shopify/sarama/alter_user_scram_credentials_response.go b/vendor/github.com/Shopify/sarama/alter_user_scram_credentials_response.go new file mode 100644 index 000000000000..31e167b5eb79 --- /dev/null +++ b/vendor/github.com/Shopify/sarama/alter_user_scram_credentials_response.go @@ -0,0 +1,94 @@ +package sarama + +import "time" + +type AlterUserScramCredentialsResponse struct { + Version int16 + + ThrottleTime time.Duration + + Results []*AlterUserScramCredentialsResult +} + +type AlterUserScramCredentialsResult struct { + User string + + ErrorCode KError + ErrorMessage *string +} + +func (r *AlterUserScramCredentialsResponse) encode(pe packetEncoder) error { + pe.putInt32(int32(r.ThrottleTime / time.Millisecond)) + pe.putCompactArrayLength(len(r.Results)) + + for _, u := range r.Results { + if err := pe.putCompactString(u.User); err != nil { + return err + } + pe.putInt16(int16(u.ErrorCode)) + if err := pe.putNullableCompactString(u.ErrorMessage); err != nil { + return err + } + pe.putEmptyTaggedFieldArray() + } + + pe.putEmptyTaggedFieldArray() + return nil +} + +func (r *AlterUserScramCredentialsResponse) decode(pd packetDecoder, version int16) error { + throttleTime, err := pd.getInt32() + if err != nil { + return err + } + r.ThrottleTime = time.Duration(throttleTime) * time.Millisecond + + numResults, err := pd.getCompactArrayLength() + if err != nil { + return err + } + + if numResults > 0 { + r.Results = make([]*AlterUserScramCredentialsResult, numResults) + for i := 0; i < numResults; i++ { + r.Results[i] = &AlterUserScramCredentialsResult{} + if r.Results[i].User, err = pd.getCompactString(); err != nil { + return err + } + + kerr, err := pd.getInt16() + if err != nil { + return err + } + + r.Results[i].ErrorCode = KError(kerr) + if r.Results[i].ErrorMessage, err = pd.getCompactNullableString(); err != nil { + return err + } + if _, err := pd.getEmptyTaggedFieldArray(); err != nil { + return err + } + } + } + + if _, err := pd.getEmptyTaggedFieldArray(); err != nil { + return err + } + return nil +} + +func (r *AlterUserScramCredentialsResponse) key() int16 { + return 51 +} + +func (r *AlterUserScramCredentialsResponse) version() int16 { + return r.Version +} + +func (r *AlterUserScramCredentialsResponse) headerVersion() int16 { + return 2 +} + +func (r *AlterUserScramCredentialsResponse) requiredVersion() KafkaVersion { + return V2_7_0_0 +} diff --git a/vendor/github.com/Shopify/sarama/api_versions_request.go b/vendor/github.com/Shopify/sarama/api_versions_request.go new file mode 100644 index 000000000000..e5b3baf646be --- /dev/null +++ b/vendor/github.com/Shopify/sarama/api_versions_request.go @@ -0,0 +1,69 @@ +package sarama + +const defaultClientSoftwareName = "sarama" + +type ApiVersionsRequest struct { + // Version defines the protocol version to use for encode and decode + Version int16 + // ClientSoftwareName contains the name of the client. + ClientSoftwareName string + // ClientSoftwareVersion contains the version of the client. + ClientSoftwareVersion string +} + +func (r *ApiVersionsRequest) encode(pe packetEncoder) (err error) { + if r.Version >= 3 { + if err := pe.putCompactString(r.ClientSoftwareName); err != nil { + return err + } + if err := pe.putCompactString(r.ClientSoftwareVersion); err != nil { + return err + } + pe.putEmptyTaggedFieldArray() + } + + return nil +} + +func (r *ApiVersionsRequest) decode(pd packetDecoder, version int16) (err error) { + r.Version = version + if r.Version >= 3 { + if r.ClientSoftwareName, err = pd.getCompactString(); err != nil { + return err + } + if r.ClientSoftwareVersion, err = pd.getCompactString(); err != nil { + return err + } + if _, err := pd.getEmptyTaggedFieldArray(); err != nil { + return err + } + } + + return nil +} + +func (r *ApiVersionsRequest) key() int16 { + return 18 +} + +func (r *ApiVersionsRequest) version() int16 { + return r.Version +} + +func (r *ApiVersionsRequest) headerVersion() int16 { + if r.Version >= 3 { + return 2 + } + return 1 +} + +func (r *ApiVersionsRequest) requiredVersion() KafkaVersion { + switch r.Version { + case 0: + return V0_10_0_0 + case 3: + return V2_4_0_0 + default: + return V0_10_0_0 + } +} diff --git a/vendor/github.com/Shopify/sarama/api_versions_response.go b/vendor/github.com/Shopify/sarama/api_versions_response.go new file mode 100644 index 000000000000..ade911c59762 --- /dev/null +++ b/vendor/github.com/Shopify/sarama/api_versions_response.go @@ -0,0 +1,156 @@ +package sarama + +// ApiVersionsResponseKey contains the APIs supported by the broker. +type ApiVersionsResponseKey struct { + // Version defines the protocol version to use for encode and decode + Version int16 + // ApiKey contains the API index. + ApiKey int16 + // MinVersion contains the minimum supported version, inclusive. + MinVersion int16 + // MaxVersion contains the maximum supported version, inclusive. + MaxVersion int16 +} + +func (a *ApiVersionsResponseKey) encode(pe packetEncoder, version int16) (err error) { + a.Version = version + pe.putInt16(a.ApiKey) + + pe.putInt16(a.MinVersion) + + pe.putInt16(a.MaxVersion) + + if version >= 3 { + pe.putEmptyTaggedFieldArray() + } + + return nil +} + +func (a *ApiVersionsResponseKey) decode(pd packetDecoder, version int16) (err error) { + a.Version = version + if a.ApiKey, err = pd.getInt16(); err != nil { + return err + } + + if a.MinVersion, err = pd.getInt16(); err != nil { + return err + } + + if a.MaxVersion, err = pd.getInt16(); err != nil { + return err + } + + if version >= 3 { + if _, err := pd.getEmptyTaggedFieldArray(); err != nil { + return err + } + } + + return nil +} + +type ApiVersionsResponse struct { + // Version defines the protocol version to use for encode and decode + Version int16 + // ErrorCode contains the top-level error code. + ErrorCode int16 + // ApiKeys contains the APIs supported by the broker. + ApiKeys []ApiVersionsResponseKey + // ThrottleTimeMs contains the duration in milliseconds for which the request was throttled due to a quota violation, or zero if the request did not violate any quota. + ThrottleTimeMs int32 +} + +func (r *ApiVersionsResponse) encode(pe packetEncoder) (err error) { + pe.putInt16(r.ErrorCode) + + if r.Version >= 3 { + pe.putCompactArrayLength(len(r.ApiKeys)) + } else { + if err := pe.putArrayLength(len(r.ApiKeys)); err != nil { + return err + } + } + for _, block := range r.ApiKeys { + if err := block.encode(pe, r.Version); err != nil { + return err + } + } + + if r.Version >= 1 { + pe.putInt32(r.ThrottleTimeMs) + } + + if r.Version >= 3 { + pe.putEmptyTaggedFieldArray() + } + + return nil +} + +func (r *ApiVersionsResponse) decode(pd packetDecoder, version int16) (err error) { + r.Version = version + if r.ErrorCode, err = pd.getInt16(); err != nil { + return err + } + + var numApiKeys int + if r.Version >= 3 { + numApiKeys, err = pd.getCompactArrayLength() + if err != nil { + return err + } + } else { + numApiKeys, err = pd.getArrayLength() + if err != nil { + return err + } + } + r.ApiKeys = make([]ApiVersionsResponseKey, numApiKeys) + for i := 0; i < numApiKeys; i++ { + var block ApiVersionsResponseKey + if err = block.decode(pd, r.Version); err != nil { + return err + } + r.ApiKeys[i] = block + } + + if r.Version >= 1 { + if r.ThrottleTimeMs, err = pd.getInt32(); err != nil { + return err + } + } + + if r.Version >= 3 { + if _, err = pd.getEmptyTaggedFieldArray(); err != nil { + return err + } + } + + return nil +} + +func (r *ApiVersionsResponse) key() int16 { + return 18 +} + +func (r *ApiVersionsResponse) version() int16 { + return r.Version +} + +func (r *ApiVersionsResponse) headerVersion() int16 { + // ApiVersionsResponse always includes a v0 header. + // See KIP-511 for details + return 0 +} + +func (r *ApiVersionsResponse) requiredVersion() KafkaVersion { + switch r.Version { + case 0: + return V0_10_0_0 + case 3: + return V2_4_0_0 + default: + return V0_10_0_0 + } +} diff --git a/vendor/github.com/Shopify/sarama/async_producer.go b/vendor/github.com/Shopify/sarama/async_producer.go new file mode 100644 index 000000000000..c807aea7208f --- /dev/null +++ b/vendor/github.com/Shopify/sarama/async_producer.go @@ -0,0 +1,1162 @@ +package sarama + +import ( + "encoding/binary" + "fmt" + "sync" + "time" + + "github.com/eapache/go-resiliency/breaker" + "github.com/eapache/queue" +) + +// AsyncProducer publishes Kafka messages using a non-blocking API. It routes messages +// to the correct broker for the provided topic-partition, refreshing metadata as appropriate, +// and parses responses for errors. You must read from the Errors() channel or the +// producer will deadlock. You must call Close() or AsyncClose() on a producer to avoid +// leaks and message lost: it will not be garbage-collected automatically when it passes +// out of scope and buffered messages may not be flushed. +type AsyncProducer interface { + + // AsyncClose triggers a shutdown of the producer. The shutdown has completed + // when both the Errors and Successes channels have been closed. When calling + // AsyncClose, you *must* continue to read from those channels in order to + // drain the results of any messages in flight. + AsyncClose() + + // Close shuts down the producer and waits for any buffered messages to be + // flushed. You must call this function before a producer object passes out of + // scope, as it may otherwise leak memory. You must call this before process + // shutting down, or you may lose messages. You must call this before calling + // Close on the underlying client. + Close() error + + // Input is the input channel for the user to write messages to that they + // wish to send. + Input() chan<- *ProducerMessage + + // Successes is the success output channel back to the user when Return.Successes is + // enabled. If Return.Successes is true, you MUST read from this channel or the + // Producer will deadlock. It is suggested that you send and read messages + // together in a single select statement. + Successes() <-chan *ProducerMessage + + // Errors is the error output channel back to the user. You MUST read from this + // channel or the Producer will deadlock when the channel is full. Alternatively, + // you can set Producer.Return.Errors in your config to false, which prevents + // errors to be returned. + Errors() <-chan *ProducerError +} + +// transactionManager keeps the state necessary to ensure idempotent production +type transactionManager struct { + producerID int64 + producerEpoch int16 + sequenceNumbers map[string]int32 + mutex sync.Mutex +} + +const ( + noProducerID = -1 + noProducerEpoch = -1 +) + +func (t *transactionManager) getAndIncrementSequenceNumber(topic string, partition int32) (int32, int16) { + key := fmt.Sprintf("%s-%d", topic, partition) + t.mutex.Lock() + defer t.mutex.Unlock() + sequence := t.sequenceNumbers[key] + t.sequenceNumbers[key] = sequence + 1 + return sequence, t.producerEpoch +} + +func (t *transactionManager) bumpEpoch() { + t.mutex.Lock() + defer t.mutex.Unlock() + t.producerEpoch++ + for k := range t.sequenceNumbers { + t.sequenceNumbers[k] = 0 + } +} + +func (t *transactionManager) getProducerID() (int64, int16) { + t.mutex.Lock() + defer t.mutex.Unlock() + return t.producerID, t.producerEpoch +} + +func newTransactionManager(conf *Config, client Client) (*transactionManager, error) { + txnmgr := &transactionManager{ + producerID: noProducerID, + producerEpoch: noProducerEpoch, + } + + if conf.Producer.Idempotent { + initProducerIDResponse, err := client.InitProducerID() + if err != nil { + return nil, err + } + txnmgr.producerID = initProducerIDResponse.ProducerID + txnmgr.producerEpoch = initProducerIDResponse.ProducerEpoch + txnmgr.sequenceNumbers = make(map[string]int32) + txnmgr.mutex = sync.Mutex{} + + Logger.Printf("Obtained a ProducerId: %d and ProducerEpoch: %d\n", txnmgr.producerID, txnmgr.producerEpoch) + } + + return txnmgr, nil +} + +type asyncProducer struct { + client Client + conf *Config + + errors chan *ProducerError + input, successes, retries chan *ProducerMessage + inFlight sync.WaitGroup + + brokers map[*Broker]*brokerProducer + brokerRefs map[*brokerProducer]int + brokerLock sync.Mutex + + txnmgr *transactionManager +} + +// NewAsyncProducer creates a new AsyncProducer using the given broker addresses and configuration. +func NewAsyncProducer(addrs []string, conf *Config) (AsyncProducer, error) { + client, err := NewClient(addrs, conf) + if err != nil { + return nil, err + } + return newAsyncProducer(client) +} + +// NewAsyncProducerFromClient creates a new Producer using the given client. It is still +// necessary to call Close() on the underlying client when shutting down this producer. +func NewAsyncProducerFromClient(client Client) (AsyncProducer, error) { + // For clients passed in by the client, ensure we don't + // call Close() on it. + cli := &nopCloserClient{client} + return newAsyncProducer(cli) +} + +func newAsyncProducer(client Client) (AsyncProducer, error) { + // Check that we are not dealing with a closed Client before processing any other arguments + if client.Closed() { + return nil, ErrClosedClient + } + + txnmgr, err := newTransactionManager(client.Config(), client) + if err != nil { + return nil, err + } + + p := &asyncProducer{ + client: client, + conf: client.Config(), + errors: make(chan *ProducerError), + input: make(chan *ProducerMessage), + successes: make(chan *ProducerMessage), + retries: make(chan *ProducerMessage), + brokers: make(map[*Broker]*brokerProducer), + brokerRefs: make(map[*brokerProducer]int), + txnmgr: txnmgr, + } + + // launch our singleton dispatchers + go withRecover(p.dispatcher) + go withRecover(p.retryHandler) + + return p, nil +} + +type flagSet int8 + +const ( + syn flagSet = 1 << iota // first message from partitionProducer to brokerProducer + fin // final message from partitionProducer to brokerProducer and back + shutdown // start the shutdown process +) + +// ProducerMessage is the collection of elements passed to the Producer in order to send a message. +type ProducerMessage struct { + Topic string // The Kafka topic for this message. + // The partitioning key for this message. Pre-existing Encoders include + // StringEncoder and ByteEncoder. + Key Encoder + // The actual message to store in Kafka. Pre-existing Encoders include + // StringEncoder and ByteEncoder. + Value Encoder + + // The headers are key-value pairs that are transparently passed + // by Kafka between producers and consumers. + Headers []RecordHeader + + // This field is used to hold arbitrary data you wish to include so it + // will be available when receiving on the Successes and Errors channels. + // Sarama completely ignores this field and is only to be used for + // pass-through data. + Metadata interface{} + + // Below this point are filled in by the producer as the message is processed + + // Offset is the offset of the message stored on the broker. This is only + // guaranteed to be defined if the message was successfully delivered and + // RequiredAcks is not NoResponse. + Offset int64 + // Partition is the partition that the message was sent to. This is only + // guaranteed to be defined if the message was successfully delivered. + Partition int32 + // Timestamp can vary in behavior depending on broker configuration, being + // in either one of the CreateTime or LogAppendTime modes (default CreateTime), + // and requiring version at least 0.10.0. + // + // When configured to CreateTime, the timestamp is specified by the producer + // either by explicitly setting this field, or when the message is added + // to a produce set. + // + // When configured to LogAppendTime, the timestamp assigned to the message + // by the broker. This is only guaranteed to be defined if the message was + // successfully delivered and RequiredAcks is not NoResponse. + Timestamp time.Time + + retries int + flags flagSet + expectation chan *ProducerError + sequenceNumber int32 + producerEpoch int16 + hasSequence bool +} + +const producerMessageOverhead = 26 // the metadata overhead of CRC, flags, etc. + +func (m *ProducerMessage) byteSize(version int) int { + var size int + if version >= 2 { + size = maximumRecordOverhead + for _, h := range m.Headers { + size += len(h.Key) + len(h.Value) + 2*binary.MaxVarintLen32 + } + } else { + size = producerMessageOverhead + } + if m.Key != nil { + size += m.Key.Length() + } + if m.Value != nil { + size += m.Value.Length() + } + return size +} + +func (m *ProducerMessage) clear() { + m.flags = 0 + m.retries = 0 + m.sequenceNumber = 0 + m.producerEpoch = 0 + m.hasSequence = false +} + +// ProducerError is the type of error generated when the producer fails to deliver a message. +// It contains the original ProducerMessage as well as the actual error value. +type ProducerError struct { + Msg *ProducerMessage + Err error +} + +func (pe ProducerError) Error() string { + return fmt.Sprintf("kafka: Failed to produce message to topic %s: %s", pe.Msg.Topic, pe.Err) +} + +func (pe ProducerError) Unwrap() error { + return pe.Err +} + +// ProducerErrors is a type that wraps a batch of "ProducerError"s and implements the Error interface. +// It can be returned from the Producer's Close method to avoid the need to manually drain the Errors channel +// when closing a producer. +type ProducerErrors []*ProducerError + +func (pe ProducerErrors) Error() string { + return fmt.Sprintf("kafka: Failed to deliver %d messages.", len(pe)) +} + +func (p *asyncProducer) Errors() <-chan *ProducerError { + return p.errors +} + +func (p *asyncProducer) Successes() <-chan *ProducerMessage { + return p.successes +} + +func (p *asyncProducer) Input() chan<- *ProducerMessage { + return p.input +} + +func (p *asyncProducer) Close() error { + p.AsyncClose() + + if p.conf.Producer.Return.Successes { + go withRecover(func() { + for range p.successes { + } + }) + } + + var errors ProducerErrors + if p.conf.Producer.Return.Errors { + for event := range p.errors { + errors = append(errors, event) + } + } else { + <-p.errors + } + + if len(errors) > 0 { + return errors + } + return nil +} + +func (p *asyncProducer) AsyncClose() { + go withRecover(p.shutdown) +} + +// singleton +// dispatches messages by topic +func (p *asyncProducer) dispatcher() { + handlers := make(map[string]chan<- *ProducerMessage) + shuttingDown := false + + for msg := range p.input { + if msg == nil { + Logger.Println("Something tried to send a nil message, it was ignored.") + continue + } + + if msg.flags&shutdown != 0 { + shuttingDown = true + p.inFlight.Done() + continue + } else if msg.retries == 0 { + if shuttingDown { + // we can't just call returnError here because that decrements the wait group, + // which hasn't been incremented yet for this message, and shouldn't be + pErr := &ProducerError{Msg: msg, Err: ErrShuttingDown} + if p.conf.Producer.Return.Errors { + p.errors <- pErr + } else { + Logger.Println(pErr) + } + continue + } + p.inFlight.Add(1) + } + + for _, interceptor := range p.conf.Producer.Interceptors { + msg.safelyApplyInterceptor(interceptor) + } + + version := 1 + if p.conf.Version.IsAtLeast(V0_11_0_0) { + version = 2 + } else if msg.Headers != nil { + p.returnError(msg, ConfigurationError("Producing headers requires Kafka at least v0.11")) + continue + } + if msg.byteSize(version) > p.conf.Producer.MaxMessageBytes { + p.returnError(msg, ErrMessageSizeTooLarge) + continue + } + + handler := handlers[msg.Topic] + if handler == nil { + handler = p.newTopicProducer(msg.Topic) + handlers[msg.Topic] = handler + } + + handler <- msg + } + + for _, handler := range handlers { + close(handler) + } +} + +// one per topic +// partitions messages, then dispatches them by partition +type topicProducer struct { + parent *asyncProducer + topic string + input <-chan *ProducerMessage + + breaker *breaker.Breaker + handlers map[int32]chan<- *ProducerMessage + partitioner Partitioner +} + +func (p *asyncProducer) newTopicProducer(topic string) chan<- *ProducerMessage { + input := make(chan *ProducerMessage, p.conf.ChannelBufferSize) + tp := &topicProducer{ + parent: p, + topic: topic, + input: input, + breaker: breaker.New(3, 1, 10*time.Second), + handlers: make(map[int32]chan<- *ProducerMessage), + partitioner: p.conf.Producer.Partitioner(topic), + } + go withRecover(tp.dispatch) + return input +} + +func (tp *topicProducer) dispatch() { + for msg := range tp.input { + if msg.retries == 0 { + if err := tp.partitionMessage(msg); err != nil { + tp.parent.returnError(msg, err) + continue + } + } + + handler := tp.handlers[msg.Partition] + if handler == nil { + handler = tp.parent.newPartitionProducer(msg.Topic, msg.Partition) + tp.handlers[msg.Partition] = handler + } + + handler <- msg + } + + for _, handler := range tp.handlers { + close(handler) + } +} + +func (tp *topicProducer) partitionMessage(msg *ProducerMessage) error { + var partitions []int32 + + err := tp.breaker.Run(func() (err error) { + requiresConsistency := false + if ep, ok := tp.partitioner.(DynamicConsistencyPartitioner); ok { + requiresConsistency = ep.MessageRequiresConsistency(msg) + } else { + requiresConsistency = tp.partitioner.RequiresConsistency() + } + + if requiresConsistency { + partitions, err = tp.parent.client.Partitions(msg.Topic) + } else { + partitions, err = tp.parent.client.WritablePartitions(msg.Topic) + } + return + }) + if err != nil { + return err + } + + numPartitions := int32(len(partitions)) + + if numPartitions == 0 { + return ErrLeaderNotAvailable + } + + choice, err := tp.partitioner.Partition(msg, numPartitions) + + if err != nil { + return err + } else if choice < 0 || choice >= numPartitions { + return ErrInvalidPartition + } + + msg.Partition = partitions[choice] + + return nil +} + +// one per partition per topic +// dispatches messages to the appropriate broker +// also responsible for maintaining message order during retries +type partitionProducer struct { + parent *asyncProducer + topic string + partition int32 + input <-chan *ProducerMessage + + leader *Broker + breaker *breaker.Breaker + brokerProducer *brokerProducer + + // highWatermark tracks the "current" retry level, which is the only one where we actually let messages through, + // all other messages get buffered in retryState[msg.retries].buf to preserve ordering + // retryState[msg.retries].expectChaser simply tracks whether we've seen a fin message for a given level (and + // therefore whether our buffer is complete and safe to flush) + highWatermark int + retryState []partitionRetryState +} + +type partitionRetryState struct { + buf []*ProducerMessage + expectChaser bool +} + +func (p *asyncProducer) newPartitionProducer(topic string, partition int32) chan<- *ProducerMessage { + input := make(chan *ProducerMessage, p.conf.ChannelBufferSize) + pp := &partitionProducer{ + parent: p, + topic: topic, + partition: partition, + input: input, + + breaker: breaker.New(3, 1, 10*time.Second), + retryState: make([]partitionRetryState, p.conf.Producer.Retry.Max+1), + } + go withRecover(pp.dispatch) + return input +} + +func (pp *partitionProducer) backoff(retries int) { + var backoff time.Duration + if pp.parent.conf.Producer.Retry.BackoffFunc != nil { + maxRetries := pp.parent.conf.Producer.Retry.Max + backoff = pp.parent.conf.Producer.Retry.BackoffFunc(retries, maxRetries) + } else { + backoff = pp.parent.conf.Producer.Retry.Backoff + } + if backoff > 0 { + time.Sleep(backoff) + } +} + +func (pp *partitionProducer) dispatch() { + // try to prefetch the leader; if this doesn't work, we'll do a proper call to `updateLeader` + // on the first message + pp.leader, _ = pp.parent.client.Leader(pp.topic, pp.partition) + if pp.leader != nil { + pp.brokerProducer = pp.parent.getBrokerProducer(pp.leader) + pp.parent.inFlight.Add(1) // we're generating a syn message; track it so we don't shut down while it's still inflight + pp.brokerProducer.input <- &ProducerMessage{Topic: pp.topic, Partition: pp.partition, flags: syn} + } + + defer func() { + if pp.brokerProducer != nil { + pp.parent.unrefBrokerProducer(pp.leader, pp.brokerProducer) + } + }() + + for msg := range pp.input { + if pp.brokerProducer != nil && pp.brokerProducer.abandoned != nil { + select { + case <-pp.brokerProducer.abandoned: + // a message on the abandoned channel means that our current broker selection is out of date + Logger.Printf("producer/leader/%s/%d abandoning broker %d\n", pp.topic, pp.partition, pp.leader.ID()) + pp.parent.unrefBrokerProducer(pp.leader, pp.brokerProducer) + pp.brokerProducer = nil + time.Sleep(pp.parent.conf.Producer.Retry.Backoff) + default: + // producer connection is still open. + } + } + + if msg.retries > pp.highWatermark { + // a new, higher, retry level; handle it and then back off + pp.newHighWatermark(msg.retries) + pp.backoff(msg.retries) + } else if pp.highWatermark > 0 { + // we are retrying something (else highWatermark would be 0) but this message is not a *new* retry level + if msg.retries < pp.highWatermark { + // in fact this message is not even the current retry level, so buffer it for now (unless it's a just a fin) + if msg.flags&fin == fin { + pp.retryState[msg.retries].expectChaser = false + pp.parent.inFlight.Done() // this fin is now handled and will be garbage collected + } else { + pp.retryState[msg.retries].buf = append(pp.retryState[msg.retries].buf, msg) + } + continue + } else if msg.flags&fin == fin { + // this message is of the current retry level (msg.retries == highWatermark) and the fin flag is set, + // meaning this retry level is done and we can go down (at least) one level and flush that + pp.retryState[pp.highWatermark].expectChaser = false + pp.flushRetryBuffers() + pp.parent.inFlight.Done() // this fin is now handled and will be garbage collected + continue + } + } + + // if we made it this far then the current msg contains real data, and can be sent to the next goroutine + // without breaking any of our ordering guarantees + + if pp.brokerProducer == nil { + if err := pp.updateLeader(); err != nil { + pp.parent.returnError(msg, err) + pp.backoff(msg.retries) + continue + } + Logger.Printf("producer/leader/%s/%d selected broker %d\n", pp.topic, pp.partition, pp.leader.ID()) + } + + // Now that we know we have a broker to actually try and send this message to, generate the sequence + // number for it. + // All messages being retried (sent or not) have already had their retry count updated + // Also, ignore "special" syn/fin messages used to sync the brokerProducer and the topicProducer. + if pp.parent.conf.Producer.Idempotent && msg.retries == 0 && msg.flags == 0 { + msg.sequenceNumber, msg.producerEpoch = pp.parent.txnmgr.getAndIncrementSequenceNumber(msg.Topic, msg.Partition) + msg.hasSequence = true + } + + pp.brokerProducer.input <- msg + } +} + +func (pp *partitionProducer) newHighWatermark(hwm int) { + Logger.Printf("producer/leader/%s/%d state change to [retrying-%d]\n", pp.topic, pp.partition, hwm) + pp.highWatermark = hwm + + // send off a fin so that we know when everything "in between" has made it + // back to us and we can safely flush the backlog (otherwise we risk re-ordering messages) + pp.retryState[pp.highWatermark].expectChaser = true + pp.parent.inFlight.Add(1) // we're generating a fin message; track it so we don't shut down while it's still inflight + pp.brokerProducer.input <- &ProducerMessage{Topic: pp.topic, Partition: pp.partition, flags: fin, retries: pp.highWatermark - 1} + + // a new HWM means that our current broker selection is out of date + Logger.Printf("producer/leader/%s/%d abandoning broker %d\n", pp.topic, pp.partition, pp.leader.ID()) + pp.parent.unrefBrokerProducer(pp.leader, pp.brokerProducer) + pp.brokerProducer = nil +} + +func (pp *partitionProducer) flushRetryBuffers() { + Logger.Printf("producer/leader/%s/%d state change to [flushing-%d]\n", pp.topic, pp.partition, pp.highWatermark) + for { + pp.highWatermark-- + + if pp.brokerProducer == nil { + if err := pp.updateLeader(); err != nil { + pp.parent.returnErrors(pp.retryState[pp.highWatermark].buf, err) + goto flushDone + } + Logger.Printf("producer/leader/%s/%d selected broker %d\n", pp.topic, pp.partition, pp.leader.ID()) + } + + for _, msg := range pp.retryState[pp.highWatermark].buf { + pp.brokerProducer.input <- msg + } + + flushDone: + pp.retryState[pp.highWatermark].buf = nil + if pp.retryState[pp.highWatermark].expectChaser { + Logger.Printf("producer/leader/%s/%d state change to [retrying-%d]\n", pp.topic, pp.partition, pp.highWatermark) + break + } else if pp.highWatermark == 0 { + Logger.Printf("producer/leader/%s/%d state change to [normal]\n", pp.topic, pp.partition) + break + } + } +} + +func (pp *partitionProducer) updateLeader() error { + return pp.breaker.Run(func() (err error) { + if err = pp.parent.client.RefreshMetadata(pp.topic); err != nil { + return err + } + + if pp.leader, err = pp.parent.client.Leader(pp.topic, pp.partition); err != nil { + return err + } + + pp.brokerProducer = pp.parent.getBrokerProducer(pp.leader) + pp.parent.inFlight.Add(1) // we're generating a syn message; track it so we don't shut down while it's still inflight + pp.brokerProducer.input <- &ProducerMessage{Topic: pp.topic, Partition: pp.partition, flags: syn} + + return nil + }) +} + +// one per broker; also constructs an associated flusher +func (p *asyncProducer) newBrokerProducer(broker *Broker) *brokerProducer { + var ( + input = make(chan *ProducerMessage) + bridge = make(chan *produceSet) + responses = make(chan *brokerProducerResponse) + ) + + bp := &brokerProducer{ + parent: p, + broker: broker, + input: input, + output: bridge, + responses: responses, + stopchan: make(chan struct{}), + buffer: newProduceSet(p), + currentRetries: make(map[string]map[int32]error), + } + go withRecover(bp.run) + + // minimal bridge to make the network response `select`able + go withRecover(func() { + for set := range bridge { + request := set.buildRequest() + + response, err := broker.Produce(request) + + responses <- &brokerProducerResponse{ + set: set, + err: err, + res: response, + } + } + close(responses) + }) + + if p.conf.Producer.Retry.Max <= 0 { + bp.abandoned = make(chan struct{}) + } + + return bp +} + +type brokerProducerResponse struct { + set *produceSet + err error + res *ProduceResponse +} + +// groups messages together into appropriately-sized batches for sending to the broker +// handles state related to retries etc +type brokerProducer struct { + parent *asyncProducer + broker *Broker + + input chan *ProducerMessage + output chan<- *produceSet + responses <-chan *brokerProducerResponse + abandoned chan struct{} + stopchan chan struct{} + + buffer *produceSet + timer <-chan time.Time + timerFired bool + + closing error + currentRetries map[string]map[int32]error +} + +func (bp *brokerProducer) run() { + var output chan<- *produceSet + Logger.Printf("producer/broker/%d starting up\n", bp.broker.ID()) + + for { + select { + case msg, ok := <-bp.input: + if !ok { + Logger.Printf("producer/broker/%d input chan closed\n", bp.broker.ID()) + bp.shutdown() + return + } + + if msg == nil { + continue + } + + if msg.flags&syn == syn { + Logger.Printf("producer/broker/%d state change to [open] on %s/%d\n", + bp.broker.ID(), msg.Topic, msg.Partition) + if bp.currentRetries[msg.Topic] == nil { + bp.currentRetries[msg.Topic] = make(map[int32]error) + } + bp.currentRetries[msg.Topic][msg.Partition] = nil + bp.parent.inFlight.Done() + continue + } + + if reason := bp.needsRetry(msg); reason != nil { + bp.parent.retryMessage(msg, reason) + + if bp.closing == nil && msg.flags&fin == fin { + // we were retrying this partition but we can start processing again + delete(bp.currentRetries[msg.Topic], msg.Partition) + Logger.Printf("producer/broker/%d state change to [closed] on %s/%d\n", + bp.broker.ID(), msg.Topic, msg.Partition) + } + + continue + } + + if bp.buffer.wouldOverflow(msg) { + Logger.Printf("producer/broker/%d maximum request accumulated, waiting for space\n", bp.broker.ID()) + if err := bp.waitForSpace(msg, false); err != nil { + bp.parent.retryMessage(msg, err) + continue + } + } + + if bp.parent.txnmgr.producerID != noProducerID && bp.buffer.producerEpoch != msg.producerEpoch { + // The epoch was reset, need to roll the buffer over + Logger.Printf("producer/broker/%d detected epoch rollover, waiting for new buffer\n", bp.broker.ID()) + if err := bp.waitForSpace(msg, true); err != nil { + bp.parent.retryMessage(msg, err) + continue + } + } + if err := bp.buffer.add(msg); err != nil { + bp.parent.returnError(msg, err) + continue + } + + if bp.parent.conf.Producer.Flush.Frequency > 0 && bp.timer == nil { + bp.timer = time.After(bp.parent.conf.Producer.Flush.Frequency) + } + case <-bp.timer: + bp.timerFired = true + case output <- bp.buffer: + bp.rollOver() + case response, ok := <-bp.responses: + if ok { + bp.handleResponse(response) + } + case <-bp.stopchan: + Logger.Printf( + "producer/broker/%d run loop asked to stop\n", bp.broker.ID()) + return + } + + if bp.timerFired || bp.buffer.readyToFlush() { + output = bp.output + } else { + output = nil + } + } +} + +func (bp *brokerProducer) shutdown() { + for !bp.buffer.empty() { + select { + case response := <-bp.responses: + bp.handleResponse(response) + case bp.output <- bp.buffer: + bp.rollOver() + } + } + close(bp.output) + for response := range bp.responses { + bp.handleResponse(response) + } + close(bp.stopchan) + Logger.Printf("producer/broker/%d shut down\n", bp.broker.ID()) +} + +func (bp *brokerProducer) needsRetry(msg *ProducerMessage) error { + if bp.closing != nil { + return bp.closing + } + + return bp.currentRetries[msg.Topic][msg.Partition] +} + +func (bp *brokerProducer) waitForSpace(msg *ProducerMessage, forceRollover bool) error { + for { + select { + case response := <-bp.responses: + bp.handleResponse(response) + // handling a response can change our state, so re-check some things + if reason := bp.needsRetry(msg); reason != nil { + return reason + } else if !bp.buffer.wouldOverflow(msg) && !forceRollover { + return nil + } + case bp.output <- bp.buffer: + bp.rollOver() + return nil + } + } +} + +func (bp *brokerProducer) rollOver() { + bp.timer = nil + bp.timerFired = false + bp.buffer = newProduceSet(bp.parent) +} + +func (bp *brokerProducer) handleResponse(response *brokerProducerResponse) { + if response.err != nil { + bp.handleError(response.set, response.err) + } else { + bp.handleSuccess(response.set, response.res) + } + + if bp.buffer.empty() { + bp.rollOver() // this can happen if the response invalidated our buffer + } +} + +func (bp *brokerProducer) handleSuccess(sent *produceSet, response *ProduceResponse) { + // we iterate through the blocks in the request set, not the response, so that we notice + // if the response is missing a block completely + var retryTopics []string + sent.eachPartition(func(topic string, partition int32, pSet *partitionSet) { + if response == nil { + // this only happens when RequiredAcks is NoResponse, so we have to assume success + bp.parent.returnSuccesses(pSet.msgs) + return + } + + block := response.GetBlock(topic, partition) + if block == nil { + bp.parent.returnErrors(pSet.msgs, ErrIncompleteResponse) + return + } + + switch block.Err { + // Success + case ErrNoError: + if bp.parent.conf.Version.IsAtLeast(V0_10_0_0) && !block.Timestamp.IsZero() { + for _, msg := range pSet.msgs { + msg.Timestamp = block.Timestamp + } + } + for i, msg := range pSet.msgs { + msg.Offset = block.Offset + int64(i) + } + bp.parent.returnSuccesses(pSet.msgs) + // Duplicate + case ErrDuplicateSequenceNumber: + bp.parent.returnSuccesses(pSet.msgs) + // Retriable errors + case ErrInvalidMessage, ErrUnknownTopicOrPartition, ErrLeaderNotAvailable, ErrNotLeaderForPartition, + ErrRequestTimedOut, ErrNotEnoughReplicas, ErrNotEnoughReplicasAfterAppend: + if bp.parent.conf.Producer.Retry.Max <= 0 { + bp.parent.abandonBrokerConnection(bp.broker) + bp.parent.returnErrors(pSet.msgs, block.Err) + } else { + retryTopics = append(retryTopics, topic) + } + // Other non-retriable errors + default: + if bp.parent.conf.Producer.Retry.Max <= 0 { + bp.parent.abandonBrokerConnection(bp.broker) + } + bp.parent.returnErrors(pSet.msgs, block.Err) + } + }) + + if len(retryTopics) > 0 { + if bp.parent.conf.Producer.Idempotent { + err := bp.parent.client.RefreshMetadata(retryTopics...) + if err != nil { + Logger.Printf("Failed refreshing metadata because of %v\n", err) + } + } + + sent.eachPartition(func(topic string, partition int32, pSet *partitionSet) { + block := response.GetBlock(topic, partition) + if block == nil { + // handled in the previous "eachPartition" loop + return + } + + switch block.Err { + case ErrInvalidMessage, ErrUnknownTopicOrPartition, ErrLeaderNotAvailable, ErrNotLeaderForPartition, + ErrRequestTimedOut, ErrNotEnoughReplicas, ErrNotEnoughReplicasAfterAppend: + Logger.Printf("producer/broker/%d state change to [retrying] on %s/%d because %v\n", + bp.broker.ID(), topic, partition, block.Err) + if bp.currentRetries[topic] == nil { + bp.currentRetries[topic] = make(map[int32]error) + } + bp.currentRetries[topic][partition] = block.Err + if bp.parent.conf.Producer.Idempotent { + go bp.parent.retryBatch(topic, partition, pSet, block.Err) + } else { + bp.parent.retryMessages(pSet.msgs, block.Err) + } + // dropping the following messages has the side effect of incrementing their retry count + bp.parent.retryMessages(bp.buffer.dropPartition(topic, partition), block.Err) + } + }) + } +} + +func (p *asyncProducer) retryBatch(topic string, partition int32, pSet *partitionSet, kerr KError) { + Logger.Printf("Retrying batch for %v-%d because of %s\n", topic, partition, kerr) + produceSet := newProduceSet(p) + produceSet.msgs[topic] = make(map[int32]*partitionSet) + produceSet.msgs[topic][partition] = pSet + produceSet.bufferBytes += pSet.bufferBytes + produceSet.bufferCount += len(pSet.msgs) + for _, msg := range pSet.msgs { + if msg.retries >= p.conf.Producer.Retry.Max { + p.returnError(msg, kerr) + return + } + msg.retries++ + } + + // it's expected that a metadata refresh has been requested prior to calling retryBatch + leader, err := p.client.Leader(topic, partition) + if err != nil { + Logger.Printf("Failed retrying batch for %v-%d because of %v while looking up for new leader\n", topic, partition, err) + for _, msg := range pSet.msgs { + p.returnError(msg, kerr) + } + return + } + bp := p.getBrokerProducer(leader) + bp.output <- produceSet +} + +func (bp *brokerProducer) handleError(sent *produceSet, err error) { + switch err.(type) { + case PacketEncodingError: + sent.eachPartition(func(topic string, partition int32, pSet *partitionSet) { + bp.parent.returnErrors(pSet.msgs, err) + }) + default: + Logger.Printf("producer/broker/%d state change to [closing] because %s\n", bp.broker.ID(), err) + bp.parent.abandonBrokerConnection(bp.broker) + _ = bp.broker.Close() + bp.closing = err + sent.eachPartition(func(topic string, partition int32, pSet *partitionSet) { + bp.parent.retryMessages(pSet.msgs, err) + }) + bp.buffer.eachPartition(func(topic string, partition int32, pSet *partitionSet) { + bp.parent.retryMessages(pSet.msgs, err) + }) + bp.rollOver() + } +} + +// singleton +// effectively a "bridge" between the flushers and the dispatcher in order to avoid deadlock +// based on https://godoc.org/github.com/eapache/channels#InfiniteChannel +func (p *asyncProducer) retryHandler() { + var msg *ProducerMessage + buf := queue.New() + + for { + if buf.Length() == 0 { + msg = <-p.retries + } else { + select { + case msg = <-p.retries: + case p.input <- buf.Peek().(*ProducerMessage): + buf.Remove() + continue + } + } + + if msg == nil { + return + } + + buf.Add(msg) + } +} + +// utility functions + +func (p *asyncProducer) shutdown() { + Logger.Println("Producer shutting down.") + p.inFlight.Add(1) + p.input <- &ProducerMessage{flags: shutdown} + + p.inFlight.Wait() + + err := p.client.Close() + if err != nil { + Logger.Println("producer/shutdown failed to close the embedded client:", err) + } + + close(p.input) + close(p.retries) + close(p.errors) + close(p.successes) +} + +func (p *asyncProducer) returnError(msg *ProducerMessage, err error) { + // We need to reset the producer ID epoch if we set a sequence number on it, because the broker + // will never see a message with this number, so we can never continue the sequence. + if msg.hasSequence { + Logger.Printf("producer/txnmanager rolling over epoch due to publish failure on %s/%d", msg.Topic, msg.Partition) + p.txnmgr.bumpEpoch() + } + msg.clear() + pErr := &ProducerError{Msg: msg, Err: err} + if p.conf.Producer.Return.Errors { + p.errors <- pErr + } else { + Logger.Println(pErr) + } + p.inFlight.Done() +} + +func (p *asyncProducer) returnErrors(batch []*ProducerMessage, err error) { + for _, msg := range batch { + p.returnError(msg, err) + } +} + +func (p *asyncProducer) returnSuccesses(batch []*ProducerMessage) { + for _, msg := range batch { + if p.conf.Producer.Return.Successes { + msg.clear() + p.successes <- msg + } + p.inFlight.Done() + } +} + +func (p *asyncProducer) retryMessage(msg *ProducerMessage, err error) { + if msg.retries >= p.conf.Producer.Retry.Max { + p.returnError(msg, err) + } else { + msg.retries++ + p.retries <- msg + } +} + +func (p *asyncProducer) retryMessages(batch []*ProducerMessage, err error) { + for _, msg := range batch { + p.retryMessage(msg, err) + } +} + +func (p *asyncProducer) getBrokerProducer(broker *Broker) *brokerProducer { + p.brokerLock.Lock() + defer p.brokerLock.Unlock() + + bp := p.brokers[broker] + + if bp == nil { + bp = p.newBrokerProducer(broker) + p.brokers[broker] = bp + p.brokerRefs[bp] = 0 + } + + p.brokerRefs[bp]++ + + return bp +} + +func (p *asyncProducer) unrefBrokerProducer(broker *Broker, bp *brokerProducer) { + p.brokerLock.Lock() + defer p.brokerLock.Unlock() + + p.brokerRefs[bp]-- + if p.brokerRefs[bp] == 0 { + close(bp.input) + delete(p.brokerRefs, bp) + + if p.brokers[broker] == bp { + delete(p.brokers, broker) + } + } +} + +func (p *asyncProducer) abandonBrokerConnection(broker *Broker) { + p.brokerLock.Lock() + defer p.brokerLock.Unlock() + + bc, ok := p.brokers[broker] + if ok && bc.abandoned != nil { + close(bc.abandoned) + } + + delete(p.brokers, broker) +} diff --git a/vendor/github.com/Shopify/sarama/balance_strategy.go b/vendor/github.com/Shopify/sarama/balance_strategy.go new file mode 100644 index 000000000000..9855bf44398d --- /dev/null +++ b/vendor/github.com/Shopify/sarama/balance_strategy.go @@ -0,0 +1,1136 @@ +package sarama + +import ( + "container/heap" + "errors" + "fmt" + "math" + "sort" + "strings" +) + +const ( + // RangeBalanceStrategyName identifies strategies that use the range partition assignment strategy + RangeBalanceStrategyName = "range" + + // RoundRobinBalanceStrategyName identifies strategies that use the round-robin partition assignment strategy + RoundRobinBalanceStrategyName = "roundrobin" + + // StickyBalanceStrategyName identifies strategies that use the sticky-partition assignment strategy + StickyBalanceStrategyName = "sticky" + + defaultGeneration = -1 +) + +// BalanceStrategyPlan is the results of any BalanceStrategy.Plan attempt. +// It contains an allocation of topic/partitions by memberID in the form of +// a `memberID -> topic -> partitions` map. +type BalanceStrategyPlan map[string]map[string][]int32 + +// Add assigns a topic with a number partitions to a member. +func (p BalanceStrategyPlan) Add(memberID, topic string, partitions ...int32) { + if len(partitions) == 0 { + return + } + if _, ok := p[memberID]; !ok { + p[memberID] = make(map[string][]int32, 1) + } + p[memberID][topic] = append(p[memberID][topic], partitions...) +} + +// -------------------------------------------------------------------- + +// BalanceStrategy is used to balance topics and partitions +// across members of a consumer group +type BalanceStrategy interface { + // Name uniquely identifies the strategy. + Name() string + + // Plan accepts a map of `memberID -> metadata` and a map of `topic -> partitions` + // and returns a distribution plan. + Plan(members map[string]ConsumerGroupMemberMetadata, topics map[string][]int32) (BalanceStrategyPlan, error) + + // AssignmentData returns the serialized assignment data for the specified + // memberID + AssignmentData(memberID string, topics map[string][]int32, generationID int32) ([]byte, error) +} + +// -------------------------------------------------------------------- + +// BalanceStrategyRange is the default and assigns partitions as ranges to consumer group members. +// Example with one topic T with six partitions (0..5) and two members (M1, M2): +// M1: {T: [0, 1, 2]} +// M2: {T: [3, 4, 5]} +var BalanceStrategyRange = &balanceStrategy{ + name: RangeBalanceStrategyName, + coreFn: func(plan BalanceStrategyPlan, memberIDs []string, topic string, partitions []int32) { + step := float64(len(partitions)) / float64(len(memberIDs)) + + for i, memberID := range memberIDs { + pos := float64(i) + min := int(math.Floor(pos*step + 0.5)) + max := int(math.Floor((pos+1)*step + 0.5)) + plan.Add(memberID, topic, partitions[min:max]...) + } + }, +} + +// BalanceStrategySticky assigns partitions to members with an attempt to preserve earlier assignments +// while maintain a balanced partition distribution. +// Example with topic T with six partitions (0..5) and two members (M1, M2): +// M1: {T: [0, 2, 4]} +// M2: {T: [1, 3, 5]} +// +// On reassignment with an additional consumer, you might get an assignment plan like: +// M1: {T: [0, 2]} +// M2: {T: [1, 3]} +// M3: {T: [4, 5]} +// +var BalanceStrategySticky = &stickyBalanceStrategy{} + +// -------------------------------------------------------------------- + +type balanceStrategy struct { + name string + coreFn func(plan BalanceStrategyPlan, memberIDs []string, topic string, partitions []int32) +} + +// Name implements BalanceStrategy. +func (s *balanceStrategy) Name() string { return s.name } + +// Plan implements BalanceStrategy. +func (s *balanceStrategy) Plan(members map[string]ConsumerGroupMemberMetadata, topics map[string][]int32) (BalanceStrategyPlan, error) { + // Build members by topic map + mbt := make(map[string][]string) + for memberID, meta := range members { + for _, topic := range meta.Topics { + mbt[topic] = append(mbt[topic], memberID) + } + } + + // Sort members for each topic + for topic, memberIDs := range mbt { + sort.Sort(&balanceStrategySortable{ + topic: topic, + memberIDs: memberIDs, + }) + } + + // Assemble plan + plan := make(BalanceStrategyPlan, len(members)) + for topic, memberIDs := range mbt { + s.coreFn(plan, memberIDs, topic, topics[topic]) + } + return plan, nil +} + +// AssignmentData simple strategies do not require any shared assignment data +func (s *balanceStrategy) AssignmentData(memberID string, topics map[string][]int32, generationID int32) ([]byte, error) { + return nil, nil +} + +type balanceStrategySortable struct { + topic string + memberIDs []string +} + +func (p balanceStrategySortable) Len() int { return len(p.memberIDs) } +func (p balanceStrategySortable) Swap(i, j int) { + p.memberIDs[i], p.memberIDs[j] = p.memberIDs[j], p.memberIDs[i] +} + +func (p balanceStrategySortable) Less(i, j int) bool { + return balanceStrategyHashValue(p.topic, p.memberIDs[i]) < balanceStrategyHashValue(p.topic, p.memberIDs[j]) +} + +func balanceStrategyHashValue(vv ...string) uint32 { + h := uint32(2166136261) + for _, s := range vv { + for _, c := range s { + h ^= uint32(c) + h *= 16777619 + } + } + return h +} + +type stickyBalanceStrategy struct { + movements partitionMovements +} + +// Name implements BalanceStrategy. +func (s *stickyBalanceStrategy) Name() string { return StickyBalanceStrategyName } + +// Plan implements BalanceStrategy. +func (s *stickyBalanceStrategy) Plan(members map[string]ConsumerGroupMemberMetadata, topics map[string][]int32) (BalanceStrategyPlan, error) { + // track partition movements during generation of the partition assignment plan + s.movements = partitionMovements{ + Movements: make(map[topicPartitionAssignment]consumerPair), + PartitionMovementsByTopic: make(map[string]map[consumerPair]map[topicPartitionAssignment]bool), + } + + // prepopulate the current assignment state from userdata on the consumer group members + currentAssignment, prevAssignment, err := prepopulateCurrentAssignments(members) + if err != nil { + return nil, err + } + + // determine if we're dealing with a completely fresh assignment, or if there's existing assignment state + isFreshAssignment := false + if len(currentAssignment) == 0 { + isFreshAssignment = true + } + + // create a mapping of all current topic partitions and the consumers that can be assigned to them + partition2AllPotentialConsumers := make(map[topicPartitionAssignment][]string) + for topic, partitions := range topics { + for _, partition := range partitions { + partition2AllPotentialConsumers[topicPartitionAssignment{Topic: topic, Partition: partition}] = []string{} + } + } + + // create a mapping of all consumers to all potential topic partitions that can be assigned to them + // also, populate the mapping of partitions to potential consumers + consumer2AllPotentialPartitions := make(map[string][]topicPartitionAssignment, len(members)) + for memberID, meta := range members { + consumer2AllPotentialPartitions[memberID] = make([]topicPartitionAssignment, 0) + for _, topicSubscription := range meta.Topics { + // only evaluate topic subscriptions that are present in the supplied topics map + if _, found := topics[topicSubscription]; found { + for _, partition := range topics[topicSubscription] { + topicPartition := topicPartitionAssignment{Topic: topicSubscription, Partition: partition} + consumer2AllPotentialPartitions[memberID] = append(consumer2AllPotentialPartitions[memberID], topicPartition) + partition2AllPotentialConsumers[topicPartition] = append(partition2AllPotentialConsumers[topicPartition], memberID) + } + } + } + + // add this consumer to currentAssignment (with an empty topic partition assignment) if it does not already exist + if _, exists := currentAssignment[memberID]; !exists { + currentAssignment[memberID] = make([]topicPartitionAssignment, 0) + } + } + + // create a mapping of each partition to its current consumer, where possible + currentPartitionConsumers := make(map[topicPartitionAssignment]string, len(currentAssignment)) + unvisitedPartitions := make(map[topicPartitionAssignment]bool, len(partition2AllPotentialConsumers)) + for partition := range partition2AllPotentialConsumers { + unvisitedPartitions[partition] = true + } + var unassignedPartitions []topicPartitionAssignment + for memberID, partitions := range currentAssignment { + var keepPartitions []topicPartitionAssignment + for _, partition := range partitions { + // If this partition no longer exists at all, likely due to the + // topic being deleted, we remove the partition from the member. + if _, exists := partition2AllPotentialConsumers[partition]; !exists { + continue + } + delete(unvisitedPartitions, partition) + currentPartitionConsumers[partition] = memberID + + if !strsContains(members[memberID].Topics, partition.Topic) { + unassignedPartitions = append(unassignedPartitions, partition) + continue + } + keepPartitions = append(keepPartitions, partition) + } + currentAssignment[memberID] = keepPartitions + } + for unvisited := range unvisitedPartitions { + unassignedPartitions = append(unassignedPartitions, unvisited) + } + + // sort the topic partitions in order of priority for reassignment + sortedPartitions := sortPartitions(currentAssignment, prevAssignment, isFreshAssignment, partition2AllPotentialConsumers, consumer2AllPotentialPartitions) + + // at this point we have preserved all valid topic partition to consumer assignments and removed + // all invalid topic partitions and invalid consumers. Now we need to assign unassignedPartitions + // to consumers so that the topic partition assignments are as balanced as possible. + + // an ascending sorted set of consumers based on how many topic partitions are already assigned to them + sortedCurrentSubscriptions := sortMemberIDsByPartitionAssignments(currentAssignment) + s.balance(currentAssignment, prevAssignment, sortedPartitions, unassignedPartitions, sortedCurrentSubscriptions, consumer2AllPotentialPartitions, partition2AllPotentialConsumers, currentPartitionConsumers) + + // Assemble plan + plan := make(BalanceStrategyPlan, len(currentAssignment)) + for memberID, assignments := range currentAssignment { + if len(assignments) == 0 { + plan[memberID] = make(map[string][]int32) + } else { + for _, assignment := range assignments { + plan.Add(memberID, assignment.Topic, assignment.Partition) + } + } + } + return plan, nil +} + +// AssignmentData serializes the set of topics currently assigned to the +// specified member as part of the supplied balance plan +func (s *stickyBalanceStrategy) AssignmentData(memberID string, topics map[string][]int32, generationID int32) ([]byte, error) { + return encode(&StickyAssignorUserDataV1{ + Topics: topics, + Generation: generationID, + }, nil) +} + +func strsContains(s []string, value string) bool { + for _, entry := range s { + if entry == value { + return true + } + } + return false +} + +// Balance assignments across consumers for maximum fairness and stickiness. +func (s *stickyBalanceStrategy) balance(currentAssignment map[string][]topicPartitionAssignment, prevAssignment map[topicPartitionAssignment]consumerGenerationPair, sortedPartitions []topicPartitionAssignment, unassignedPartitions []topicPartitionAssignment, sortedCurrentSubscriptions []string, consumer2AllPotentialPartitions map[string][]topicPartitionAssignment, partition2AllPotentialConsumers map[topicPartitionAssignment][]string, currentPartitionConsumer map[topicPartitionAssignment]string) { + initializing := false + if len(sortedCurrentSubscriptions) == 0 || len(currentAssignment[sortedCurrentSubscriptions[0]]) == 0 { + initializing = true + } + + // assign all unassigned partitions + for _, partition := range unassignedPartitions { + // skip if there is no potential consumer for the partition + if len(partition2AllPotentialConsumers[partition]) == 0 { + continue + } + sortedCurrentSubscriptions = assignPartition(partition, sortedCurrentSubscriptions, currentAssignment, consumer2AllPotentialPartitions, currentPartitionConsumer) + } + + // narrow down the reassignment scope to only those partitions that can actually be reassigned + for partition := range partition2AllPotentialConsumers { + if !canTopicPartitionParticipateInReassignment(partition, partition2AllPotentialConsumers) { + sortedPartitions = removeTopicPartitionFromMemberAssignments(sortedPartitions, partition) + } + } + + // narrow down the reassignment scope to only those consumers that are subject to reassignment + fixedAssignments := make(map[string][]topicPartitionAssignment) + for memberID := range consumer2AllPotentialPartitions { + if !canConsumerParticipateInReassignment(memberID, currentAssignment, consumer2AllPotentialPartitions, partition2AllPotentialConsumers) { + fixedAssignments[memberID] = currentAssignment[memberID] + delete(currentAssignment, memberID) + sortedCurrentSubscriptions = sortMemberIDsByPartitionAssignments(currentAssignment) + } + } + + // create a deep copy of the current assignment so we can revert to it if we do not get a more balanced assignment later + preBalanceAssignment := deepCopyAssignment(currentAssignment) + preBalancePartitionConsumers := make(map[topicPartitionAssignment]string, len(currentPartitionConsumer)) + for k, v := range currentPartitionConsumer { + preBalancePartitionConsumers[k] = v + } + + reassignmentPerformed := s.performReassignments(sortedPartitions, currentAssignment, prevAssignment, sortedCurrentSubscriptions, consumer2AllPotentialPartitions, partition2AllPotentialConsumers, currentPartitionConsumer) + + // if we are not preserving existing assignments and we have made changes to the current assignment + // make sure we are getting a more balanced assignment; otherwise, revert to previous assignment + if !initializing && reassignmentPerformed && getBalanceScore(currentAssignment) >= getBalanceScore(preBalanceAssignment) { + currentAssignment = deepCopyAssignment(preBalanceAssignment) + currentPartitionConsumer = make(map[topicPartitionAssignment]string, len(preBalancePartitionConsumers)) + for k, v := range preBalancePartitionConsumers { + currentPartitionConsumer[k] = v + } + } + + // add the fixed assignments (those that could not change) back + for consumer, assignments := range fixedAssignments { + currentAssignment[consumer] = assignments + } +} + +// BalanceStrategyRoundRobin assigns partitions to members in alternating order. +// For example, there are two topics (t0, t1) and two consumer (m0, m1), and each topic has three partitions (p0, p1, p2): +// M0: [t0p0, t0p2, t1p1] +// M1: [t0p1, t1p0, t1p2] +var BalanceStrategyRoundRobin = new(roundRobinBalancer) + +type roundRobinBalancer struct{} + +func (b *roundRobinBalancer) Name() string { + return RoundRobinBalanceStrategyName +} + +func (b *roundRobinBalancer) Plan(memberAndMetadata map[string]ConsumerGroupMemberMetadata, topics map[string][]int32) (BalanceStrategyPlan, error) { + if len(memberAndMetadata) == 0 || len(topics) == 0 { + return nil, errors.New("members and topics are not provided") + } + // sort partitions + var topicPartitions []topicAndPartition + for topic, partitions := range topics { + for _, partition := range partitions { + topicPartitions = append(topicPartitions, topicAndPartition{topic: topic, partition: partition}) + } + } + sort.SliceStable(topicPartitions, func(i, j int) bool { + pi := topicPartitions[i] + pj := topicPartitions[j] + return pi.comparedValue() < pj.comparedValue() + }) + + // sort members + var members []memberAndTopic + for memberID, meta := range memberAndMetadata { + m := memberAndTopic{ + memberID: memberID, + topics: make(map[string]struct{}), + } + for _, t := range meta.Topics { + m.topics[t] = struct{}{} + } + members = append(members, m) + } + sort.SliceStable(members, func(i, j int) bool { + mi := members[i] + mj := members[j] + return mi.memberID < mj.memberID + }) + + // assign partitions + plan := make(BalanceStrategyPlan, len(members)) + i := 0 + n := len(members) + for _, tp := range topicPartitions { + m := members[i%n] + for !m.hasTopic(tp.topic) { + i++ + m = members[i%n] + } + plan.Add(m.memberID, tp.topic, tp.partition) + i++ + } + return plan, nil +} + +func (b *roundRobinBalancer) AssignmentData(memberID string, topics map[string][]int32, generationID int32) ([]byte, error) { + return nil, nil // do nothing for now +} + +type topicAndPartition struct { + topic string + partition int32 +} + +func (tp *topicAndPartition) comparedValue() string { + return fmt.Sprintf("%s-%d", tp.topic, tp.partition) +} + +type memberAndTopic struct { + memberID string + topics map[string]struct{} +} + +func (m *memberAndTopic) hasTopic(topic string) bool { + _, isExist := m.topics[topic] + return isExist +} + +// Calculate the balance score of the given assignment, as the sum of assigned partitions size difference of all consumer pairs. +// A perfectly balanced assignment (with all consumers getting the same number of partitions) has a balance score of 0. +// Lower balance score indicates a more balanced assignment. +func getBalanceScore(assignment map[string][]topicPartitionAssignment) int { + consumer2AssignmentSize := make(map[string]int, len(assignment)) + for memberID, partitions := range assignment { + consumer2AssignmentSize[memberID] = len(partitions) + } + + var score float64 + for memberID, consumerAssignmentSize := range consumer2AssignmentSize { + delete(consumer2AssignmentSize, memberID) + for _, otherConsumerAssignmentSize := range consumer2AssignmentSize { + score += math.Abs(float64(consumerAssignmentSize - otherConsumerAssignmentSize)) + } + } + return int(score) +} + +// Determine whether the current assignment plan is balanced. +func isBalanced(currentAssignment map[string][]topicPartitionAssignment, allSubscriptions map[string][]topicPartitionAssignment) bool { + sortedCurrentSubscriptions := sortMemberIDsByPartitionAssignments(currentAssignment) + min := len(currentAssignment[sortedCurrentSubscriptions[0]]) + max := len(currentAssignment[sortedCurrentSubscriptions[len(sortedCurrentSubscriptions)-1]]) + if min >= max-1 { + // if minimum and maximum numbers of partitions assigned to consumers differ by at most one return true + return true + } + + // create a mapping from partitions to the consumer assigned to them + allPartitions := make(map[topicPartitionAssignment]string) + for memberID, partitions := range currentAssignment { + for _, partition := range partitions { + if _, exists := allPartitions[partition]; exists { + Logger.Printf("Topic %s Partition %d is assigned more than one consumer", partition.Topic, partition.Partition) + } + allPartitions[partition] = memberID + } + } + + // for each consumer that does not have all the topic partitions it can get make sure none of the topic partitions it + // could but did not get cannot be moved to it (because that would break the balance) + for _, memberID := range sortedCurrentSubscriptions { + consumerPartitions := currentAssignment[memberID] + consumerPartitionCount := len(consumerPartitions) + + // skip if this consumer already has all the topic partitions it can get + if consumerPartitionCount == len(allSubscriptions[memberID]) { + continue + } + + // otherwise make sure it cannot get any more + potentialTopicPartitions := allSubscriptions[memberID] + for _, partition := range potentialTopicPartitions { + if !memberAssignmentsIncludeTopicPartition(currentAssignment[memberID], partition) { + otherConsumer := allPartitions[partition] + otherConsumerPartitionCount := len(currentAssignment[otherConsumer]) + if consumerPartitionCount < otherConsumerPartitionCount { + return false + } + } + } + } + return true +} + +// Reassign all topic partitions that need reassignment until balanced. +func (s *stickyBalanceStrategy) performReassignments(reassignablePartitions []topicPartitionAssignment, currentAssignment map[string][]topicPartitionAssignment, prevAssignment map[topicPartitionAssignment]consumerGenerationPair, sortedCurrentSubscriptions []string, consumer2AllPotentialPartitions map[string][]topicPartitionAssignment, partition2AllPotentialConsumers map[topicPartitionAssignment][]string, currentPartitionConsumer map[topicPartitionAssignment]string) bool { + reassignmentPerformed := false + modified := false + + // repeat reassignment until no partition can be moved to improve the balance + for { + modified = false + // reassign all reassignable partitions (starting from the partition with least potential consumers and if needed) + // until the full list is processed or a balance is achieved + for _, partition := range reassignablePartitions { + if isBalanced(currentAssignment, consumer2AllPotentialPartitions) { + break + } + + // the partition must have at least two consumers + if len(partition2AllPotentialConsumers[partition]) <= 1 { + Logger.Printf("Expected more than one potential consumer for partition %s topic %d", partition.Topic, partition.Partition) + } + + // the partition must have a consumer + consumer := currentPartitionConsumer[partition] + if consumer == "" { + Logger.Printf("Expected topic %s partition %d to be assigned to a consumer", partition.Topic, partition.Partition) + } + + if _, exists := prevAssignment[partition]; exists { + if len(currentAssignment[consumer]) > (len(currentAssignment[prevAssignment[partition].MemberID]) + 1) { + sortedCurrentSubscriptions = s.reassignPartition(partition, currentAssignment, sortedCurrentSubscriptions, currentPartitionConsumer, prevAssignment[partition].MemberID) + reassignmentPerformed = true + modified = true + continue + } + } + + // check if a better-suited consumer exists for the partition; if so, reassign it + for _, otherConsumer := range partition2AllPotentialConsumers[partition] { + if len(currentAssignment[consumer]) > (len(currentAssignment[otherConsumer]) + 1) { + sortedCurrentSubscriptions = s.reassignPartitionToNewConsumer(partition, currentAssignment, sortedCurrentSubscriptions, currentPartitionConsumer, consumer2AllPotentialPartitions) + reassignmentPerformed = true + modified = true + break + } + } + } + if !modified { + return reassignmentPerformed + } + } +} + +// Identify a new consumer for a topic partition and reassign it. +func (s *stickyBalanceStrategy) reassignPartitionToNewConsumer(partition topicPartitionAssignment, currentAssignment map[string][]topicPartitionAssignment, sortedCurrentSubscriptions []string, currentPartitionConsumer map[topicPartitionAssignment]string, consumer2AllPotentialPartitions map[string][]topicPartitionAssignment) []string { + for _, anotherConsumer := range sortedCurrentSubscriptions { + if memberAssignmentsIncludeTopicPartition(consumer2AllPotentialPartitions[anotherConsumer], partition) { + return s.reassignPartition(partition, currentAssignment, sortedCurrentSubscriptions, currentPartitionConsumer, anotherConsumer) + } + } + return sortedCurrentSubscriptions +} + +// Reassign a specific partition to a new consumer +func (s *stickyBalanceStrategy) reassignPartition(partition topicPartitionAssignment, currentAssignment map[string][]topicPartitionAssignment, sortedCurrentSubscriptions []string, currentPartitionConsumer map[topicPartitionAssignment]string, newConsumer string) []string { + consumer := currentPartitionConsumer[partition] + // find the correct partition movement considering the stickiness requirement + partitionToBeMoved := s.movements.getTheActualPartitionToBeMoved(partition, consumer, newConsumer) + return s.processPartitionMovement(partitionToBeMoved, newConsumer, currentAssignment, sortedCurrentSubscriptions, currentPartitionConsumer) +} + +// Track the movement of a topic partition after assignment +func (s *stickyBalanceStrategy) processPartitionMovement(partition topicPartitionAssignment, newConsumer string, currentAssignment map[string][]topicPartitionAssignment, sortedCurrentSubscriptions []string, currentPartitionConsumer map[topicPartitionAssignment]string) []string { + oldConsumer := currentPartitionConsumer[partition] + s.movements.movePartition(partition, oldConsumer, newConsumer) + + currentAssignment[oldConsumer] = removeTopicPartitionFromMemberAssignments(currentAssignment[oldConsumer], partition) + currentAssignment[newConsumer] = append(currentAssignment[newConsumer], partition) + currentPartitionConsumer[partition] = newConsumer + return sortMemberIDsByPartitionAssignments(currentAssignment) +} + +// Determine whether a specific consumer should be considered for topic partition assignment. +func canConsumerParticipateInReassignment(memberID string, currentAssignment map[string][]topicPartitionAssignment, consumer2AllPotentialPartitions map[string][]topicPartitionAssignment, partition2AllPotentialConsumers map[topicPartitionAssignment][]string) bool { + currentPartitions := currentAssignment[memberID] + currentAssignmentSize := len(currentPartitions) + maxAssignmentSize := len(consumer2AllPotentialPartitions[memberID]) + if currentAssignmentSize > maxAssignmentSize { + Logger.Printf("The consumer %s is assigned more partitions than the maximum possible", memberID) + } + if currentAssignmentSize < maxAssignmentSize { + // if a consumer is not assigned all its potential partitions it is subject to reassignment + return true + } + for _, partition := range currentPartitions { + if canTopicPartitionParticipateInReassignment(partition, partition2AllPotentialConsumers) { + return true + } + } + return false +} + +// Only consider reassigning those topic partitions that have two or more potential consumers. +func canTopicPartitionParticipateInReassignment(partition topicPartitionAssignment, partition2AllPotentialConsumers map[topicPartitionAssignment][]string) bool { + return len(partition2AllPotentialConsumers[partition]) >= 2 +} + +// The assignment should improve the overall balance of the partition assignments to consumers. +func assignPartition(partition topicPartitionAssignment, sortedCurrentSubscriptions []string, currentAssignment map[string][]topicPartitionAssignment, consumer2AllPotentialPartitions map[string][]topicPartitionAssignment, currentPartitionConsumer map[topicPartitionAssignment]string) []string { + for _, memberID := range sortedCurrentSubscriptions { + if memberAssignmentsIncludeTopicPartition(consumer2AllPotentialPartitions[memberID], partition) { + currentAssignment[memberID] = append(currentAssignment[memberID], partition) + currentPartitionConsumer[partition] = memberID + break + } + } + return sortMemberIDsByPartitionAssignments(currentAssignment) +} + +// Deserialize topic partition assignment data to aid with creation of a sticky assignment. +func deserializeTopicPartitionAssignment(userDataBytes []byte) (StickyAssignorUserData, error) { + userDataV1 := &StickyAssignorUserDataV1{} + if err := decode(userDataBytes, userDataV1); err != nil { + userDataV0 := &StickyAssignorUserDataV0{} + if err := decode(userDataBytes, userDataV0); err != nil { + return nil, err + } + return userDataV0, nil + } + return userDataV1, nil +} + +// filterAssignedPartitions returns a map of consumer group members to their list of previously-assigned topic partitions, limited +// to those topic partitions currently reported by the Kafka cluster. +func filterAssignedPartitions(currentAssignment map[string][]topicPartitionAssignment, partition2AllPotentialConsumers map[topicPartitionAssignment][]string) map[string][]topicPartitionAssignment { + assignments := deepCopyAssignment(currentAssignment) + for memberID, partitions := range assignments { + // perform in-place filtering + i := 0 + for _, partition := range partitions { + if _, exists := partition2AllPotentialConsumers[partition]; exists { + partitions[i] = partition + i++ + } + } + assignments[memberID] = partitions[:i] + } + return assignments +} + +func removeTopicPartitionFromMemberAssignments(assignments []topicPartitionAssignment, topic topicPartitionAssignment) []topicPartitionAssignment { + for i, assignment := range assignments { + if assignment == topic { + return append(assignments[:i], assignments[i+1:]...) + } + } + return assignments +} + +func memberAssignmentsIncludeTopicPartition(assignments []topicPartitionAssignment, topic topicPartitionAssignment) bool { + for _, assignment := range assignments { + if assignment == topic { + return true + } + } + return false +} + +func sortPartitions(currentAssignment map[string][]topicPartitionAssignment, partitionsWithADifferentPreviousAssignment map[topicPartitionAssignment]consumerGenerationPair, isFreshAssignment bool, partition2AllPotentialConsumers map[topicPartitionAssignment][]string, consumer2AllPotentialPartitions map[string][]topicPartitionAssignment) []topicPartitionAssignment { + unassignedPartitions := make(map[topicPartitionAssignment]bool, len(partition2AllPotentialConsumers)) + for partition := range partition2AllPotentialConsumers { + unassignedPartitions[partition] = true + } + + sortedPartitions := make([]topicPartitionAssignment, 0) + if !isFreshAssignment && areSubscriptionsIdentical(partition2AllPotentialConsumers, consumer2AllPotentialPartitions) { + // if this is a reassignment and the subscriptions are identical (all consumers can consumer from all topics) + // then we just need to simply list partitions in a round robin fashion (from consumers with + // most assigned partitions to those with least) + assignments := filterAssignedPartitions(currentAssignment, partition2AllPotentialConsumers) + + // use priority-queue to evaluate consumer group members in descending-order based on + // the number of topic partition assignments (i.e. consumers with most assignments first) + pq := make(assignmentPriorityQueue, len(assignments)) + i := 0 + for consumerID, consumerAssignments := range assignments { + pq[i] = &consumerGroupMember{ + id: consumerID, + assignments: consumerAssignments, + } + i++ + } + heap.Init(&pq) + + for { + // loop until no consumer-group members remain + if pq.Len() == 0 { + break + } + member := pq[0] + + // partitions that were assigned to a different consumer last time + var prevPartitionIndex int + for i, partition := range member.assignments { + if _, exists := partitionsWithADifferentPreviousAssignment[partition]; exists { + prevPartitionIndex = i + break + } + } + + if len(member.assignments) > 0 { + partition := member.assignments[prevPartitionIndex] + sortedPartitions = append(sortedPartitions, partition) + delete(unassignedPartitions, partition) + if prevPartitionIndex == 0 { + member.assignments = member.assignments[1:] + } else { + member.assignments = append(member.assignments[:prevPartitionIndex], member.assignments[prevPartitionIndex+1:]...) + } + heap.Fix(&pq, 0) + } else { + heap.Pop(&pq) + } + } + + for partition := range unassignedPartitions { + sortedPartitions = append(sortedPartitions, partition) + } + } else { + // an ascending sorted set of topic partitions based on how many consumers can potentially use them + sortedPartitions = sortPartitionsByPotentialConsumerAssignments(partition2AllPotentialConsumers) + } + return sortedPartitions +} + +func sortMemberIDsByPartitionAssignments(assignments map[string][]topicPartitionAssignment) []string { + // sort the members by the number of partition assignments in ascending order + sortedMemberIDs := make([]string, 0, len(assignments)) + for memberID := range assignments { + sortedMemberIDs = append(sortedMemberIDs, memberID) + } + sort.SliceStable(sortedMemberIDs, func(i, j int) bool { + ret := len(assignments[sortedMemberIDs[i]]) - len(assignments[sortedMemberIDs[j]]) + if ret == 0 { + return sortedMemberIDs[i] < sortedMemberIDs[j] + } + return len(assignments[sortedMemberIDs[i]]) < len(assignments[sortedMemberIDs[j]]) + }) + return sortedMemberIDs +} + +func sortPartitionsByPotentialConsumerAssignments(partition2AllPotentialConsumers map[topicPartitionAssignment][]string) []topicPartitionAssignment { + // sort the members by the number of partition assignments in descending order + sortedPartionIDs := make([]topicPartitionAssignment, len(partition2AllPotentialConsumers)) + i := 0 + for partition := range partition2AllPotentialConsumers { + sortedPartionIDs[i] = partition + i++ + } + sort.Slice(sortedPartionIDs, func(i, j int) bool { + if len(partition2AllPotentialConsumers[sortedPartionIDs[i]]) == len(partition2AllPotentialConsumers[sortedPartionIDs[j]]) { + ret := strings.Compare(sortedPartionIDs[i].Topic, sortedPartionIDs[j].Topic) + if ret == 0 { + return sortedPartionIDs[i].Partition < sortedPartionIDs[j].Partition + } + return ret < 0 + } + return len(partition2AllPotentialConsumers[sortedPartionIDs[i]]) < len(partition2AllPotentialConsumers[sortedPartionIDs[j]]) + }) + return sortedPartionIDs +} + +func deepCopyAssignment(assignment map[string][]topicPartitionAssignment) map[string][]topicPartitionAssignment { + m := make(map[string][]topicPartitionAssignment, len(assignment)) + for memberID, subscriptions := range assignment { + m[memberID] = append(subscriptions[:0:0], subscriptions...) + } + return m +} + +func areSubscriptionsIdentical(partition2AllPotentialConsumers map[topicPartitionAssignment][]string, consumer2AllPotentialPartitions map[string][]topicPartitionAssignment) bool { + curMembers := make(map[string]int) + for _, cur := range partition2AllPotentialConsumers { + if len(curMembers) == 0 { + for _, curMembersElem := range cur { + curMembers[curMembersElem]++ + } + continue + } + + if len(curMembers) != len(cur) { + return false + } + + yMap := make(map[string]int) + for _, yElem := range cur { + yMap[yElem]++ + } + + for curMembersMapKey, curMembersMapVal := range curMembers { + if yMap[curMembersMapKey] != curMembersMapVal { + return false + } + } + } + + curPartitions := make(map[topicPartitionAssignment]int) + for _, cur := range consumer2AllPotentialPartitions { + if len(curPartitions) == 0 { + for _, curPartitionElem := range cur { + curPartitions[curPartitionElem]++ + } + continue + } + + if len(curPartitions) != len(cur) { + return false + } + + yMap := make(map[topicPartitionAssignment]int) + for _, yElem := range cur { + yMap[yElem]++ + } + + for curMembersMapKey, curMembersMapVal := range curPartitions { + if yMap[curMembersMapKey] != curMembersMapVal { + return false + } + } + } + return true +} + +// We need to process subscriptions' user data with each consumer's reported generation in mind +// higher generations overwrite lower generations in case of a conflict +// note that a conflict could exist only if user data is for different generations +func prepopulateCurrentAssignments(members map[string]ConsumerGroupMemberMetadata) (map[string][]topicPartitionAssignment, map[topicPartitionAssignment]consumerGenerationPair, error) { + currentAssignment := make(map[string][]topicPartitionAssignment) + prevAssignment := make(map[topicPartitionAssignment]consumerGenerationPair) + + // for each partition we create a sorted map of its consumers by generation + sortedPartitionConsumersByGeneration := make(map[topicPartitionAssignment]map[int]string) + for memberID, meta := range members { + consumerUserData, err := deserializeTopicPartitionAssignment(meta.UserData) + if err != nil { + return nil, nil, err + } + for _, partition := range consumerUserData.partitions() { + if consumers, exists := sortedPartitionConsumersByGeneration[partition]; exists { + if consumerUserData.hasGeneration() { + if _, generationExists := consumers[consumerUserData.generation()]; generationExists { + // same partition is assigned to two consumers during the same rebalance. + // log a warning and skip this record + Logger.Printf("Topic %s Partition %d is assigned to multiple consumers following sticky assignment generation %d", partition.Topic, partition.Partition, consumerUserData.generation()) + continue + } else { + consumers[consumerUserData.generation()] = memberID + } + } else { + consumers[defaultGeneration] = memberID + } + } else { + generation := defaultGeneration + if consumerUserData.hasGeneration() { + generation = consumerUserData.generation() + } + sortedPartitionConsumersByGeneration[partition] = map[int]string{generation: memberID} + } + } + } + + // prevAssignment holds the prior ConsumerGenerationPair (before current) of each partition + // current and previous consumers are the last two consumers of each partition in the above sorted map + for partition, consumers := range sortedPartitionConsumersByGeneration { + // sort consumers by generation in decreasing order + var generations []int + for generation := range consumers { + generations = append(generations, generation) + } + sort.Sort(sort.Reverse(sort.IntSlice(generations))) + + consumer := consumers[generations[0]] + if _, exists := currentAssignment[consumer]; !exists { + currentAssignment[consumer] = []topicPartitionAssignment{partition} + } else { + currentAssignment[consumer] = append(currentAssignment[consumer], partition) + } + + // check for previous assignment, if any + if len(generations) > 1 { + prevAssignment[partition] = consumerGenerationPair{ + MemberID: consumers[generations[1]], + Generation: generations[1], + } + } + } + return currentAssignment, prevAssignment, nil +} + +type consumerGenerationPair struct { + MemberID string + Generation int +} + +// consumerPair represents a pair of Kafka consumer ids involved in a partition reassignment. +type consumerPair struct { + SrcMemberID string + DstMemberID string +} + +// partitionMovements maintains some data structures to simplify lookup of partition movements among consumers. +type partitionMovements struct { + PartitionMovementsByTopic map[string]map[consumerPair]map[topicPartitionAssignment]bool + Movements map[topicPartitionAssignment]consumerPair +} + +func (p *partitionMovements) removeMovementRecordOfPartition(partition topicPartitionAssignment) consumerPair { + pair := p.Movements[partition] + delete(p.Movements, partition) + + partitionMovementsForThisTopic := p.PartitionMovementsByTopic[partition.Topic] + delete(partitionMovementsForThisTopic[pair], partition) + if len(partitionMovementsForThisTopic[pair]) == 0 { + delete(partitionMovementsForThisTopic, pair) + } + if len(p.PartitionMovementsByTopic[partition.Topic]) == 0 { + delete(p.PartitionMovementsByTopic, partition.Topic) + } + return pair +} + +func (p *partitionMovements) addPartitionMovementRecord(partition topicPartitionAssignment, pair consumerPair) { + p.Movements[partition] = pair + if _, exists := p.PartitionMovementsByTopic[partition.Topic]; !exists { + p.PartitionMovementsByTopic[partition.Topic] = make(map[consumerPair]map[topicPartitionAssignment]bool) + } + partitionMovementsForThisTopic := p.PartitionMovementsByTopic[partition.Topic] + if _, exists := partitionMovementsForThisTopic[pair]; !exists { + partitionMovementsForThisTopic[pair] = make(map[topicPartitionAssignment]bool) + } + partitionMovementsForThisTopic[pair][partition] = true +} + +func (p *partitionMovements) movePartition(partition topicPartitionAssignment, oldConsumer, newConsumer string) { + pair := consumerPair{ + SrcMemberID: oldConsumer, + DstMemberID: newConsumer, + } + if _, exists := p.Movements[partition]; exists { + // this partition has previously moved + existingPair := p.removeMovementRecordOfPartition(partition) + if existingPair.DstMemberID != oldConsumer { + Logger.Printf("Existing pair DstMemberID %s was not equal to the oldConsumer ID %s", existingPair.DstMemberID, oldConsumer) + } + if existingPair.SrcMemberID != newConsumer { + // the partition is not moving back to its previous consumer + p.addPartitionMovementRecord(partition, consumerPair{ + SrcMemberID: existingPair.SrcMemberID, + DstMemberID: newConsumer, + }) + } + } else { + p.addPartitionMovementRecord(partition, pair) + } +} + +func (p *partitionMovements) getTheActualPartitionToBeMoved(partition topicPartitionAssignment, oldConsumer, newConsumer string) topicPartitionAssignment { + if _, exists := p.PartitionMovementsByTopic[partition.Topic]; !exists { + return partition + } + if _, exists := p.Movements[partition]; exists { + // this partition has previously moved + if oldConsumer != p.Movements[partition].DstMemberID { + Logger.Printf("Partition movement DstMemberID %s was not equal to the oldConsumer ID %s", p.Movements[partition].DstMemberID, oldConsumer) + } + oldConsumer = p.Movements[partition].SrcMemberID + } + + partitionMovementsForThisTopic := p.PartitionMovementsByTopic[partition.Topic] + reversePair := consumerPair{ + SrcMemberID: newConsumer, + DstMemberID: oldConsumer, + } + if _, exists := partitionMovementsForThisTopic[reversePair]; !exists { + return partition + } + var reversePairPartition topicPartitionAssignment + for otherPartition := range partitionMovementsForThisTopic[reversePair] { + reversePairPartition = otherPartition + } + return reversePairPartition +} + +func (p *partitionMovements) isLinked(src, dst string, pairs []consumerPair, currentPath []string) ([]string, bool) { + if src == dst { + return currentPath, false + } + if len(pairs) == 0 { + return currentPath, false + } + for _, pair := range pairs { + if src == pair.SrcMemberID && dst == pair.DstMemberID { + currentPath = append(currentPath, src, dst) + return currentPath, true + } + } + + for _, pair := range pairs { + if pair.SrcMemberID == src { + // create a deep copy of the pairs, excluding the current pair + reducedSet := make([]consumerPair, len(pairs)-1) + i := 0 + for _, p := range pairs { + if p != pair { + reducedSet[i] = pair + i++ + } + } + + currentPath = append(currentPath, pair.SrcMemberID) + return p.isLinked(pair.DstMemberID, dst, reducedSet, currentPath) + } + } + return currentPath, false +} + +func (p *partitionMovements) in(cycle []string, cycles [][]string) bool { + superCycle := make([]string, len(cycle)-1) + for i := 0; i < len(cycle)-1; i++ { + superCycle[i] = cycle[i] + } + superCycle = append(superCycle, cycle...) + for _, foundCycle := range cycles { + if len(foundCycle) == len(cycle) && indexOfSubList(superCycle, foundCycle) != -1 { + return true + } + } + return false +} + +func (p *partitionMovements) hasCycles(pairs []consumerPair) bool { + cycles := make([][]string, 0) + for _, pair := range pairs { + // create a deep copy of the pairs, excluding the current pair + reducedPairs := make([]consumerPair, len(pairs)-1) + i := 0 + for _, p := range pairs { + if p != pair { + reducedPairs[i] = pair + i++ + } + } + if path, linked := p.isLinked(pair.DstMemberID, pair.SrcMemberID, reducedPairs, []string{pair.SrcMemberID}); linked { + if !p.in(path, cycles) { + cycles = append(cycles, path) + Logger.Printf("A cycle of length %d was found: %v", len(path)-1, path) + } + } + } + + // for now we want to make sure there is no partition movements of the same topic between a pair of consumers. + // the odds of finding a cycle among more than two consumers seem to be very low (according to various randomized + // tests with the given sticky algorithm) that it should not worth the added complexity of handling those cases. + for _, cycle := range cycles { + if len(cycle) == 3 { + return true + } + } + return false +} + +func (p *partitionMovements) isSticky() bool { + for topic, movements := range p.PartitionMovementsByTopic { + movementPairs := make([]consumerPair, len(movements)) + i := 0 + for pair := range movements { + movementPairs[i] = pair + i++ + } + if p.hasCycles(movementPairs) { + Logger.Printf("Stickiness is violated for topic %s", topic) + Logger.Printf("Partition movements for this topic occurred among the following consumer pairs: %v", movements) + return false + } + } + return true +} + +func indexOfSubList(source []string, target []string) int { + targetSize := len(target) + maxCandidate := len(source) - targetSize +nextCand: + for candidate := 0; candidate <= maxCandidate; candidate++ { + j := candidate + for i := 0; i < targetSize; i++ { + if target[i] != source[j] { + // Element mismatch, try next cand + continue nextCand + } + j++ + } + // All elements of candidate matched target + return candidate + } + return -1 +} + +type consumerGroupMember struct { + id string + assignments []topicPartitionAssignment +} + +// assignmentPriorityQueue is a priority-queue of consumer group members that is sorted +// in descending order (most assignments to least assignments). +type assignmentPriorityQueue []*consumerGroupMember + +func (pq assignmentPriorityQueue) Len() int { return len(pq) } + +func (pq assignmentPriorityQueue) Less(i, j int) bool { + // order asssignment priority queue in descending order using assignment-count/member-id + if len(pq[i].assignments) == len(pq[j].assignments) { + return strings.Compare(pq[i].id, pq[j].id) > 0 + } + return len(pq[i].assignments) > len(pq[j].assignments) +} + +func (pq assignmentPriorityQueue) Swap(i, j int) { + pq[i], pq[j] = pq[j], pq[i] +} + +func (pq *assignmentPriorityQueue) Push(x interface{}) { + member := x.(*consumerGroupMember) + *pq = append(*pq, member) +} + +func (pq *assignmentPriorityQueue) Pop() interface{} { + old := *pq + n := len(old) + member := old[n-1] + *pq = old[0 : n-1] + return member +} diff --git a/vendor/github.com/Shopify/sarama/broker.go b/vendor/github.com/Shopify/sarama/broker.go new file mode 100644 index 000000000000..ed92a669d850 --- /dev/null +++ b/vendor/github.com/Shopify/sarama/broker.go @@ -0,0 +1,1602 @@ +package sarama + +import ( + "crypto/tls" + "encoding/binary" + "fmt" + "io" + "net" + "sort" + "strconv" + "strings" + "sync" + "sync/atomic" + "time" + + "github.com/rcrowley/go-metrics" +) + +// Broker represents a single Kafka broker connection. All operations on this object are entirely concurrency-safe. +type Broker struct { + conf *Config + rack *string + + id int32 + addr string + correlationID int32 + conn net.Conn + connErr error + lock sync.Mutex + opened int32 + responses chan responsePromise + done chan bool + + registeredMetrics []string + + incomingByteRate metrics.Meter + requestRate metrics.Meter + requestSize metrics.Histogram + requestLatency metrics.Histogram + outgoingByteRate metrics.Meter + responseRate metrics.Meter + responseSize metrics.Histogram + requestsInFlight metrics.Counter + brokerIncomingByteRate metrics.Meter + brokerRequestRate metrics.Meter + brokerRequestSize metrics.Histogram + brokerRequestLatency metrics.Histogram + brokerOutgoingByteRate metrics.Meter + brokerResponseRate metrics.Meter + brokerResponseSize metrics.Histogram + brokerRequestsInFlight metrics.Counter + brokerThrottleTime metrics.Histogram + + kerberosAuthenticator GSSAPIKerberosAuth +} + +// SASLMechanism specifies the SASL mechanism the client uses to authenticate with the broker +type SASLMechanism string + +const ( + // SASLTypeOAuth represents the SASL/OAUTHBEARER mechanism (Kafka 2.0.0+) + SASLTypeOAuth = "OAUTHBEARER" + // SASLTypePlaintext represents the SASL/PLAIN mechanism + SASLTypePlaintext = "PLAIN" + // SASLTypeSCRAMSHA256 represents the SCRAM-SHA-256 mechanism. + SASLTypeSCRAMSHA256 = "SCRAM-SHA-256" + // SASLTypeSCRAMSHA512 represents the SCRAM-SHA-512 mechanism. + SASLTypeSCRAMSHA512 = "SCRAM-SHA-512" + SASLTypeGSSAPI = "GSSAPI" + // SASLHandshakeV0 is v0 of the Kafka SASL handshake protocol. Client and + // server negotiate SASL auth using opaque packets. + SASLHandshakeV0 = int16(0) + // SASLHandshakeV1 is v1 of the Kafka SASL handshake protocol. Client and + // server negotiate SASL by wrapping tokens with Kafka protocol headers. + SASLHandshakeV1 = int16(1) + // SASLExtKeyAuth is the reserved extension key name sent as part of the + // SASL/OAUTHBEARER initial client response + SASLExtKeyAuth = "auth" +) + +// AccessToken contains an access token used to authenticate a +// SASL/OAUTHBEARER client along with associated metadata. +type AccessToken struct { + // Token is the access token payload. + Token string + // Extensions is a optional map of arbitrary key-value pairs that can be + // sent with the SASL/OAUTHBEARER initial client response. These values are + // ignored by the SASL server if they are unexpected. This feature is only + // supported by Kafka >= 2.1.0. + Extensions map[string]string +} + +// AccessTokenProvider is the interface that encapsulates how implementors +// can generate access tokens for Kafka broker authentication. +type AccessTokenProvider interface { + // Token returns an access token. The implementation should ensure token + // reuse so that multiple calls at connect time do not create multiple + // tokens. The implementation should also periodically refresh the token in + // order to guarantee that each call returns an unexpired token. This + // method should not block indefinitely--a timeout error should be returned + // after a short period of inactivity so that the broker connection logic + // can log debugging information and retry. + Token() (*AccessToken, error) +} + +// SCRAMClient is a an interface to a SCRAM +// client implementation. +type SCRAMClient interface { + // Begin prepares the client for the SCRAM exchange + // with the server with a user name and a password + Begin(userName, password, authzID string) error + // Step steps client through the SCRAM exchange. It is + // called repeatedly until it errors or `Done` returns true. + Step(challenge string) (response string, err error) + // Done should return true when the SCRAM conversation + // is over. + Done() bool +} + +type responsePromise struct { + requestTime time.Time + correlationID int32 + headerVersion int16 + packets chan []byte + errors chan error +} + +// NewBroker creates and returns a Broker targeting the given host:port address. +// This does not attempt to actually connect, you have to call Open() for that. +func NewBroker(addr string) *Broker { + return &Broker{id: -1, addr: addr} +} + +// Open tries to connect to the Broker if it is not already connected or connecting, but does not block +// waiting for the connection to complete. This means that any subsequent operations on the broker will +// block waiting for the connection to succeed or fail. To get the effect of a fully synchronous Open call, +// follow it by a call to Connected(). The only errors Open will return directly are ConfigurationError or +// AlreadyConnected. If conf is nil, the result of NewConfig() is used. +func (b *Broker) Open(conf *Config) error { + if !atomic.CompareAndSwapInt32(&b.opened, 0, 1) { + return ErrAlreadyConnected + } + + if conf == nil { + conf = NewConfig() + } + + err := conf.Validate() + if err != nil { + return err + } + + usingApiVersionsRequests := conf.Version.IsAtLeast(V2_4_0_0) && conf.ApiVersionsRequest + + b.lock.Lock() + + go withRecover(func() { + defer func() { + b.lock.Unlock() + + // Send an ApiVersionsRequest to identify the client (KIP-511). + // Ideally Sarama would use the response to control protocol versions, + // but for now just fire-and-forget just to send + if usingApiVersionsRequests { + _, err = b.ApiVersions(&ApiVersionsRequest{ + Version: 3, + ClientSoftwareName: defaultClientSoftwareName, + ClientSoftwareVersion: version(), + }) + if err != nil { + Logger.Printf("Error while sending ApiVersionsRequest to broker %s: %s\n", b.addr, err) + } + } + }() + dialer := conf.getDialer() + b.conn, b.connErr = dialer.Dial("tcp", b.addr) + if b.connErr != nil { + Logger.Printf("Failed to connect to broker %s: %s\n", b.addr, b.connErr) + b.conn = nil + atomic.StoreInt32(&b.opened, 0) + return + } + if conf.Net.TLS.Enable { + b.conn = tls.Client(b.conn, validServerNameTLS(b.addr, conf.Net.TLS.Config)) + } + + b.conn = newBufConn(b.conn) + b.conf = conf + + // Create or reuse the global metrics shared between brokers + b.incomingByteRate = metrics.GetOrRegisterMeter("incoming-byte-rate", conf.MetricRegistry) + b.requestRate = metrics.GetOrRegisterMeter("request-rate", conf.MetricRegistry) + b.requestSize = getOrRegisterHistogram("request-size", conf.MetricRegistry) + b.requestLatency = getOrRegisterHistogram("request-latency-in-ms", conf.MetricRegistry) + b.outgoingByteRate = metrics.GetOrRegisterMeter("outgoing-byte-rate", conf.MetricRegistry) + b.responseRate = metrics.GetOrRegisterMeter("response-rate", conf.MetricRegistry) + b.responseSize = getOrRegisterHistogram("response-size", conf.MetricRegistry) + b.requestsInFlight = metrics.GetOrRegisterCounter("requests-in-flight", conf.MetricRegistry) + // Do not gather metrics for seeded broker (only used during bootstrap) because they share + // the same id (-1) and are already exposed through the global metrics above + if b.id >= 0 && !metrics.UseNilMetrics { + b.registerMetrics() + } + + if conf.Net.SASL.Enable { + b.connErr = b.authenticateViaSASL() + + if b.connErr != nil { + err = b.conn.Close() + if err == nil { + DebugLogger.Printf("Closed connection to broker %s\n", b.addr) + } else { + Logger.Printf("Error while closing connection to broker %s: %s\n", b.addr, err) + } + b.conn = nil + atomic.StoreInt32(&b.opened, 0) + return + } + } + + b.done = make(chan bool) + b.responses = make(chan responsePromise, b.conf.Net.MaxOpenRequests-1) + + if b.id >= 0 { + DebugLogger.Printf("Connected to broker at %s (registered as #%d)\n", b.addr, b.id) + } else { + DebugLogger.Printf("Connected to broker at %s (unregistered)\n", b.addr) + } + go withRecover(b.responseReceiver) + }) + + return nil +} + +// Connected returns true if the broker is connected and false otherwise. If the broker is not +// connected but it had tried to connect, the error from that connection attempt is also returned. +func (b *Broker) Connected() (bool, error) { + b.lock.Lock() + defer b.lock.Unlock() + + return b.conn != nil, b.connErr +} + +// Close closes the broker resources +func (b *Broker) Close() error { + b.lock.Lock() + defer b.lock.Unlock() + + if b.conn == nil { + return ErrNotConnected + } + + close(b.responses) + <-b.done + + err := b.conn.Close() + + b.conn = nil + b.connErr = nil + b.done = nil + b.responses = nil + + b.unregisterMetrics() + + if err == nil { + DebugLogger.Printf("Closed connection to broker %s\n", b.addr) + } else { + Logger.Printf("Error while closing connection to broker %s: %s\n", b.addr, err) + } + + atomic.StoreInt32(&b.opened, 0) + + return err +} + +// ID returns the broker ID retrieved from Kafka's metadata, or -1 if that is not known. +func (b *Broker) ID() int32 { + return b.id +} + +// Addr returns the broker address as either retrieved from Kafka's metadata or passed to NewBroker. +func (b *Broker) Addr() string { + return b.addr +} + +// Rack returns the broker's rack as retrieved from Kafka's metadata or the +// empty string if it is not known. The returned value corresponds to the +// broker's broker.rack configuration setting. Requires protocol version to be +// at least v0.10.0.0. +func (b *Broker) Rack() string { + if b.rack == nil { + return "" + } + return *b.rack +} + +// GetMetadata send a metadata request and returns a metadata response or error +func (b *Broker) GetMetadata(request *MetadataRequest) (*MetadataResponse, error) { + response := new(MetadataResponse) + + err := b.sendAndReceive(request, response) + if err != nil { + return nil, err + } + + return response, nil +} + +// GetConsumerMetadata send a consumer metadata request and returns a consumer metadata response or error +func (b *Broker) GetConsumerMetadata(request *ConsumerMetadataRequest) (*ConsumerMetadataResponse, error) { + response := new(ConsumerMetadataResponse) + + err := b.sendAndReceive(request, response) + if err != nil { + return nil, err + } + + return response, nil +} + +// FindCoordinator sends a find coordinate request and returns a response or error +func (b *Broker) FindCoordinator(request *FindCoordinatorRequest) (*FindCoordinatorResponse, error) { + response := new(FindCoordinatorResponse) + + err := b.sendAndReceive(request, response) + if err != nil { + return nil, err + } + + return response, nil +} + +// GetAvailableOffsets return an offset response or error +func (b *Broker) GetAvailableOffsets(request *OffsetRequest) (*OffsetResponse, error) { + response := new(OffsetResponse) + + err := b.sendAndReceive(request, response) + if err != nil { + return nil, err + } + + return response, nil +} + +// Produce returns a produce response or error +func (b *Broker) Produce(request *ProduceRequest) (*ProduceResponse, error) { + var ( + response *ProduceResponse + err error + ) + + if request.RequiredAcks == NoResponse { + err = b.sendAndReceive(request, nil) + } else { + response = new(ProduceResponse) + err = b.sendAndReceive(request, response) + if response.ThrottleTime != time.Duration(0) { + DebugLogger.Printf( + "producer/broker/%d ProduceResponse throttled %v\n", + b.ID(), response.ThrottleTime) + if b.brokerThrottleTime != nil { + throttleTimeInMs := int64(response.ThrottleTime / time.Millisecond) + b.brokerThrottleTime.Update(throttleTimeInMs) + } + } + } + + if err != nil { + return nil, err + } + + return response, nil +} + +// Fetch returns a FetchResponse or error +func (b *Broker) Fetch(request *FetchRequest) (*FetchResponse, error) { + response := new(FetchResponse) + + err := b.sendAndReceive(request, response) + if err != nil { + return nil, err + } + + return response, nil +} + +// CommitOffset return an Offset commit response or error +func (b *Broker) CommitOffset(request *OffsetCommitRequest) (*OffsetCommitResponse, error) { + response := new(OffsetCommitResponse) + + err := b.sendAndReceive(request, response) + if err != nil { + return nil, err + } + + return response, nil +} + +// FetchOffset returns an offset fetch response or error +func (b *Broker) FetchOffset(request *OffsetFetchRequest) (*OffsetFetchResponse, error) { + response := new(OffsetFetchResponse) + response.Version = request.Version // needed to handle the two header versions + + err := b.sendAndReceive(request, response) + if err != nil { + return nil, err + } + + return response, nil +} + +// JoinGroup returns a join group response or error +func (b *Broker) JoinGroup(request *JoinGroupRequest) (*JoinGroupResponse, error) { + response := new(JoinGroupResponse) + + err := b.sendAndReceive(request, response) + if err != nil { + return nil, err + } + + return response, nil +} + +// SyncGroup returns a sync group response or error +func (b *Broker) SyncGroup(request *SyncGroupRequest) (*SyncGroupResponse, error) { + response := new(SyncGroupResponse) + + err := b.sendAndReceive(request, response) + if err != nil { + return nil, err + } + + return response, nil +} + +// LeaveGroup return a leave group response or error +func (b *Broker) LeaveGroup(request *LeaveGroupRequest) (*LeaveGroupResponse, error) { + response := new(LeaveGroupResponse) + + err := b.sendAndReceive(request, response) + if err != nil { + return nil, err + } + + return response, nil +} + +// Heartbeat returns a heartbeat response or error +func (b *Broker) Heartbeat(request *HeartbeatRequest) (*HeartbeatResponse, error) { + response := new(HeartbeatResponse) + + err := b.sendAndReceive(request, response) + if err != nil { + return nil, err + } + + return response, nil +} + +// ListGroups return a list group response or error +func (b *Broker) ListGroups(request *ListGroupsRequest) (*ListGroupsResponse, error) { + response := new(ListGroupsResponse) + + err := b.sendAndReceive(request, response) + if err != nil { + return nil, err + } + + return response, nil +} + +// DescribeGroups return describe group response or error +func (b *Broker) DescribeGroups(request *DescribeGroupsRequest) (*DescribeGroupsResponse, error) { + response := new(DescribeGroupsResponse) + + err := b.sendAndReceive(request, response) + if err != nil { + return nil, err + } + + return response, nil +} + +// ApiVersions return api version response or error +func (b *Broker) ApiVersions(request *ApiVersionsRequest) (*ApiVersionsResponse, error) { + response := new(ApiVersionsResponse) + + err := b.sendAndReceive(request, response) + if err != nil { + return nil, err + } + + return response, nil +} + +// CreateTopics send a create topic request and returns create topic response +func (b *Broker) CreateTopics(request *CreateTopicsRequest) (*CreateTopicsResponse, error) { + response := new(CreateTopicsResponse) + + err := b.sendAndReceive(request, response) + if err != nil { + return nil, err + } + + return response, nil +} + +// DeleteTopics sends a delete topic request and returns delete topic response +func (b *Broker) DeleteTopics(request *DeleteTopicsRequest) (*DeleteTopicsResponse, error) { + response := new(DeleteTopicsResponse) + + err := b.sendAndReceive(request, response) + if err != nil { + return nil, err + } + + return response, nil +} + +// CreatePartitions sends a create partition request and returns create +// partitions response or error +func (b *Broker) CreatePartitions(request *CreatePartitionsRequest) (*CreatePartitionsResponse, error) { + response := new(CreatePartitionsResponse) + + err := b.sendAndReceive(request, response) + if err != nil { + return nil, err + } + + return response, nil +} + +// AlterPartitionReassignments sends a alter partition reassignments request and +// returns alter partition reassignments response +func (b *Broker) AlterPartitionReassignments(request *AlterPartitionReassignmentsRequest) (*AlterPartitionReassignmentsResponse, error) { + response := new(AlterPartitionReassignmentsResponse) + + err := b.sendAndReceive(request, response) + if err != nil { + return nil, err + } + + return response, nil +} + +// ListPartitionReassignments sends a list partition reassignments request and +// returns list partition reassignments response +func (b *Broker) ListPartitionReassignments(request *ListPartitionReassignmentsRequest) (*ListPartitionReassignmentsResponse, error) { + response := new(ListPartitionReassignmentsResponse) + + err := b.sendAndReceive(request, response) + if err != nil { + return nil, err + } + + return response, nil +} + +// DeleteRecords send a request to delete records and return delete record +// response or error +func (b *Broker) DeleteRecords(request *DeleteRecordsRequest) (*DeleteRecordsResponse, error) { + response := new(DeleteRecordsResponse) + + err := b.sendAndReceive(request, response) + if err != nil { + return nil, err + } + + return response, nil +} + +// DescribeAcls sends a describe acl request and returns a response or error +func (b *Broker) DescribeAcls(request *DescribeAclsRequest) (*DescribeAclsResponse, error) { + response := new(DescribeAclsResponse) + + err := b.sendAndReceive(request, response) + if err != nil { + return nil, err + } + + return response, nil +} + +// CreateAcls sends a create acl request and returns a response or error +func (b *Broker) CreateAcls(request *CreateAclsRequest) (*CreateAclsResponse, error) { + response := new(CreateAclsResponse) + + err := b.sendAndReceive(request, response) + if err != nil { + return nil, err + } + + return response, nil +} + +// DeleteAcls sends a delete acl request and returns a response or error +func (b *Broker) DeleteAcls(request *DeleteAclsRequest) (*DeleteAclsResponse, error) { + response := new(DeleteAclsResponse) + + err := b.sendAndReceive(request, response) + if err != nil { + return nil, err + } + + return response, nil +} + +// InitProducerID sends an init producer request and returns a response or error +func (b *Broker) InitProducerID(request *InitProducerIDRequest) (*InitProducerIDResponse, error) { + response := new(InitProducerIDResponse) + + err := b.sendAndReceive(request, response) + if err != nil { + return nil, err + } + + return response, nil +} + +// AddPartitionsToTxn send a request to add partition to txn and returns +// a response or error +func (b *Broker) AddPartitionsToTxn(request *AddPartitionsToTxnRequest) (*AddPartitionsToTxnResponse, error) { + response := new(AddPartitionsToTxnResponse) + + err := b.sendAndReceive(request, response) + if err != nil { + return nil, err + } + + return response, nil +} + +// AddOffsetsToTxn sends a request to add offsets to txn and returns a response +// or error +func (b *Broker) AddOffsetsToTxn(request *AddOffsetsToTxnRequest) (*AddOffsetsToTxnResponse, error) { + response := new(AddOffsetsToTxnResponse) + + err := b.sendAndReceive(request, response) + if err != nil { + return nil, err + } + + return response, nil +} + +// EndTxn sends a request to end txn and returns a response or error +func (b *Broker) EndTxn(request *EndTxnRequest) (*EndTxnResponse, error) { + response := new(EndTxnResponse) + + err := b.sendAndReceive(request, response) + if err != nil { + return nil, err + } + + return response, nil +} + +// TxnOffsetCommit sends a request to commit transaction offsets and returns +// a response or error +func (b *Broker) TxnOffsetCommit(request *TxnOffsetCommitRequest) (*TxnOffsetCommitResponse, error) { + response := new(TxnOffsetCommitResponse) + + err := b.sendAndReceive(request, response) + if err != nil { + return nil, err + } + + return response, nil +} + +// DescribeConfigs sends a request to describe config and returns a response or +// error +func (b *Broker) DescribeConfigs(request *DescribeConfigsRequest) (*DescribeConfigsResponse, error) { + response := new(DescribeConfigsResponse) + + err := b.sendAndReceive(request, response) + if err != nil { + return nil, err + } + + return response, nil +} + +// AlterConfigs sends a request to alter config and return a response or error +func (b *Broker) AlterConfigs(request *AlterConfigsRequest) (*AlterConfigsResponse, error) { + response := new(AlterConfigsResponse) + + err := b.sendAndReceive(request, response) + if err != nil { + return nil, err + } + + return response, nil +} + +// IncrementalAlterConfigs sends a request to incremental alter config and return a response or error +func (b *Broker) IncrementalAlterConfigs(request *IncrementalAlterConfigsRequest) (*IncrementalAlterConfigsResponse, error) { + response := new(IncrementalAlterConfigsResponse) + + err := b.sendAndReceive(request, response) + if err != nil { + return nil, err + } + + return response, nil +} + +// DeleteGroups sends a request to delete groups and returns a response or error +func (b *Broker) DeleteGroups(request *DeleteGroupsRequest) (*DeleteGroupsResponse, error) { + response := new(DeleteGroupsResponse) + + if err := b.sendAndReceive(request, response); err != nil { + return nil, err + } + + return response, nil +} + +// DeleteOffsets sends a request to delete group offsets and returns a response or error +func (b *Broker) DeleteOffsets(request *DeleteOffsetsRequest) (*DeleteOffsetsResponse, error) { + response := new(DeleteOffsetsResponse) + + if err := b.sendAndReceive(request, response); err != nil { + return nil, err + } + + return response, nil +} + +// DescribeLogDirs sends a request to get the broker's log dir paths and sizes +func (b *Broker) DescribeLogDirs(request *DescribeLogDirsRequest) (*DescribeLogDirsResponse, error) { + response := new(DescribeLogDirsResponse) + + err := b.sendAndReceive(request, response) + if err != nil { + return nil, err + } + + return response, nil +} + +// DescribeUserScramCredentials sends a request to get SCRAM users +func (b *Broker) DescribeUserScramCredentials(req *DescribeUserScramCredentialsRequest) (*DescribeUserScramCredentialsResponse, error) { + res := new(DescribeUserScramCredentialsResponse) + + err := b.sendAndReceive(req, res) + if err != nil { + return nil, err + } + + return res, err +} + +func (b *Broker) AlterUserScramCredentials(req *AlterUserScramCredentialsRequest) (*AlterUserScramCredentialsResponse, error) { + res := new(AlterUserScramCredentialsResponse) + + err := b.sendAndReceive(req, res) + if err != nil { + return nil, err + } + + return res, nil +} + +// DescribeClientQuotas sends a request to get the broker's quotas +func (b *Broker) DescribeClientQuotas(request *DescribeClientQuotasRequest) (*DescribeClientQuotasResponse, error) { + response := new(DescribeClientQuotasResponse) + + err := b.sendAndReceive(request, response) + if err != nil { + return nil, err + } + + return response, nil +} + +// AlterClientQuotas sends a request to alter the broker's quotas +func (b *Broker) AlterClientQuotas(request *AlterClientQuotasRequest) (*AlterClientQuotasResponse, error) { + response := new(AlterClientQuotasResponse) + + err := b.sendAndReceive(request, response) + if err != nil { + return nil, err + } + + return response, nil +} + +// readFull ensures the conn ReadDeadline has been setup before making a +// call to io.ReadFull +func (b *Broker) readFull(buf []byte) (n int, err error) { + if err := b.conn.SetReadDeadline(time.Now().Add(b.conf.Net.ReadTimeout)); err != nil { + return 0, err + } + + return io.ReadFull(b.conn, buf) +} + +// write ensures the conn WriteDeadline has been setup before making a +// call to conn.Write +func (b *Broker) write(buf []byte) (n int, err error) { + if err := b.conn.SetWriteDeadline(time.Now().Add(b.conf.Net.WriteTimeout)); err != nil { + return 0, err + } + + return b.conn.Write(buf) +} + +func (b *Broker) send(rb protocolBody, promiseResponse bool, responseHeaderVersion int16) (*responsePromise, error) { + b.lock.Lock() + defer b.lock.Unlock() + + if b.conn == nil { + if b.connErr != nil { + return nil, b.connErr + } + return nil, ErrNotConnected + } + + if !b.conf.Version.IsAtLeast(rb.requiredVersion()) { + return nil, ErrUnsupportedVersion + } + + req := &request{correlationID: b.correlationID, clientID: b.conf.ClientID, body: rb} + buf, err := encode(req, b.conf.MetricRegistry) + if err != nil { + return nil, err + } + + requestTime := time.Now() + // Will be decremented in responseReceiver (except error or request with NoResponse) + b.addRequestInFlightMetrics(1) + bytes, err := b.write(buf) + b.updateOutgoingCommunicationMetrics(bytes) + if err != nil { + b.addRequestInFlightMetrics(-1) + return nil, err + } + b.correlationID++ + + if !promiseResponse { + // Record request latency without the response + b.updateRequestLatencyAndInFlightMetrics(time.Since(requestTime)) + return nil, nil + } + + promise := responsePromise{requestTime, req.correlationID, responseHeaderVersion, make(chan []byte), make(chan error)} + b.responses <- promise + + return &promise, nil +} + +func (b *Broker) sendAndReceive(req protocolBody, res protocolBody) error { + responseHeaderVersion := int16(-1) + if res != nil { + responseHeaderVersion = res.headerVersion() + } + + promise, err := b.send(req, res != nil, responseHeaderVersion) + if err != nil { + return err + } + + if promise == nil { + return nil + } + + select { + case buf := <-promise.packets: + return versionedDecode(buf, res, req.version()) + case err = <-promise.errors: + return err + } +} + +func (b *Broker) decode(pd packetDecoder, version int16) (err error) { + b.id, err = pd.getInt32() + if err != nil { + return err + } + + host, err := pd.getString() + if err != nil { + return err + } + + port, err := pd.getInt32() + if err != nil { + return err + } + + if version >= 1 { + b.rack, err = pd.getNullableString() + if err != nil { + return err + } + } + + b.addr = net.JoinHostPort(host, fmt.Sprint(port)) + if _, _, err := net.SplitHostPort(b.addr); err != nil { + return err + } + + return nil +} + +func (b *Broker) encode(pe packetEncoder, version int16) (err error) { + host, portstr, err := net.SplitHostPort(b.addr) + if err != nil { + return err + } + + port, err := strconv.ParseInt(portstr, 10, 32) + if err != nil { + return err + } + + pe.putInt32(b.id) + + err = pe.putString(host) + if err != nil { + return err + } + + pe.putInt32(int32(port)) + + if version >= 1 { + err = pe.putNullableString(b.rack) + if err != nil { + return err + } + } + + return nil +} + +func (b *Broker) responseReceiver() { + var dead error + + for response := range b.responses { + if dead != nil { + // This was previously incremented in send() and + // we are not calling updateIncomingCommunicationMetrics() + b.addRequestInFlightMetrics(-1) + response.errors <- dead + continue + } + + headerLength := getHeaderLength(response.headerVersion) + header := make([]byte, headerLength) + + bytesReadHeader, err := b.readFull(header) + requestLatency := time.Since(response.requestTime) + if err != nil { + b.updateIncomingCommunicationMetrics(bytesReadHeader, requestLatency) + dead = err + response.errors <- err + continue + } + + decodedHeader := responseHeader{} + err = versionedDecode(header, &decodedHeader, response.headerVersion) + if err != nil { + b.updateIncomingCommunicationMetrics(bytesReadHeader, requestLatency) + dead = err + response.errors <- err + continue + } + if decodedHeader.correlationID != response.correlationID { + b.updateIncomingCommunicationMetrics(bytesReadHeader, requestLatency) + // TODO if decoded ID < cur ID, discard until we catch up + // TODO if decoded ID > cur ID, save it so when cur ID catches up we have a response + dead = PacketDecodingError{fmt.Sprintf("correlation ID didn't match, wanted %d, got %d", response.correlationID, decodedHeader.correlationID)} + response.errors <- dead + continue + } + + buf := make([]byte, decodedHeader.length-int32(headerLength)+4) + bytesReadBody, err := b.readFull(buf) + b.updateIncomingCommunicationMetrics(bytesReadHeader+bytesReadBody, requestLatency) + if err != nil { + dead = err + response.errors <- err + continue + } + + response.packets <- buf + } + close(b.done) +} + +func getHeaderLength(headerVersion int16) int8 { + if headerVersion < 1 { + return 8 + } else { + // header contains additional tagged field length (0), we don't support actual tags yet. + return 9 + } +} + +func (b *Broker) authenticateViaSASL() error { + switch b.conf.Net.SASL.Mechanism { + case SASLTypeOAuth: + return b.sendAndReceiveSASLOAuth(b.conf.Net.SASL.TokenProvider) + case SASLTypeSCRAMSHA256, SASLTypeSCRAMSHA512: + return b.sendAndReceiveSASLSCRAM() + case SASLTypeGSSAPI: + return b.sendAndReceiveKerberos() + default: + return b.sendAndReceiveSASLPlainAuth() + } +} + +func (b *Broker) sendAndReceiveKerberos() error { + b.kerberosAuthenticator.Config = &b.conf.Net.SASL.GSSAPI + if b.kerberosAuthenticator.NewKerberosClientFunc == nil { + b.kerberosAuthenticator.NewKerberosClientFunc = NewKerberosClient + } + return b.kerberosAuthenticator.Authorize(b) +} + +func (b *Broker) sendAndReceiveSASLHandshake(saslType SASLMechanism, version int16) error { + rb := &SaslHandshakeRequest{Mechanism: string(saslType), Version: version} + + req := &request{correlationID: b.correlationID, clientID: b.conf.ClientID, body: rb} + buf, err := encode(req, b.conf.MetricRegistry) + if err != nil { + return err + } + + requestTime := time.Now() + // Will be decremented in updateIncomingCommunicationMetrics (except error) + b.addRequestInFlightMetrics(1) + bytes, err := b.write(buf) + b.updateOutgoingCommunicationMetrics(bytes) + if err != nil { + b.addRequestInFlightMetrics(-1) + Logger.Printf("Failed to send SASL handshake %s: %s\n", b.addr, err.Error()) + return err + } + b.correlationID++ + + header := make([]byte, 8) // response header + _, err = b.readFull(header) + if err != nil { + b.addRequestInFlightMetrics(-1) + Logger.Printf("Failed to read SASL handshake header : %s\n", err.Error()) + return err + } + + length := binary.BigEndian.Uint32(header[:4]) + payload := make([]byte, length-4) + n, err := b.readFull(payload) + if err != nil { + b.addRequestInFlightMetrics(-1) + Logger.Printf("Failed to read SASL handshake payload : %s\n", err.Error()) + return err + } + + b.updateIncomingCommunicationMetrics(n+8, time.Since(requestTime)) + res := &SaslHandshakeResponse{} + + err = versionedDecode(payload, res, 0) + if err != nil { + Logger.Printf("Failed to parse SASL handshake : %s\n", err.Error()) + return err + } + + if res.Err != ErrNoError { + Logger.Printf("Invalid SASL Mechanism : %s\n", res.Err.Error()) + return res.Err + } + + DebugLogger.Print("Successful SASL handshake. Available mechanisms: ", res.EnabledMechanisms) + return nil +} + +// Kafka 0.10.x supported SASL PLAIN/Kerberos via KAFKA-3149 (KIP-43). +// Kafka 1.x.x onward added a SaslAuthenticate request/response message which +// wraps the SASL flow in the Kafka protocol, which allows for returning +// meaningful errors on authentication failure. +// +// In SASL Plain, Kafka expects the auth header to be in the following format +// Message format (from https://tools.ietf.org/html/rfc4616): +// +// message = [authzid] UTF8NUL authcid UTF8NUL passwd +// authcid = 1*SAFE ; MUST accept up to 255 octets +// authzid = 1*SAFE ; MUST accept up to 255 octets +// passwd = 1*SAFE ; MUST accept up to 255 octets +// UTF8NUL = %x00 ; UTF-8 encoded NUL character +// +// SAFE = UTF1 / UTF2 / UTF3 / UTF4 +// ;; any UTF-8 encoded Unicode character except NUL +// +// With SASL v0 handshake and auth then: +// When credentials are valid, Kafka returns a 4 byte array of null characters. +// When credentials are invalid, Kafka closes the connection. +// +// With SASL v1 handshake and auth then: +// When credentials are invalid, Kafka replies with a SaslAuthenticate response +// containing an error code and message detailing the authentication failure. +func (b *Broker) sendAndReceiveSASLPlainAuth() error { + // default to V0 to allow for backward compatibility when SASL is enabled + // but not the handshake + if b.conf.Net.SASL.Handshake { + handshakeErr := b.sendAndReceiveSASLHandshake(SASLTypePlaintext, b.conf.Net.SASL.Version) + if handshakeErr != nil { + Logger.Printf("Error while performing SASL handshake %s\n", b.addr) + return handshakeErr + } + } + + if b.conf.Net.SASL.Version == SASLHandshakeV1 { + return b.sendAndReceiveV1SASLPlainAuth() + } + return b.sendAndReceiveV0SASLPlainAuth() +} + +// sendAndReceiveV0SASLPlainAuth flows the v0 sasl auth NOT wrapped in the kafka protocol +func (b *Broker) sendAndReceiveV0SASLPlainAuth() error { + length := len(b.conf.Net.SASL.AuthIdentity) + 1 + len(b.conf.Net.SASL.User) + 1 + len(b.conf.Net.SASL.Password) + authBytes := make([]byte, length+4) // 4 byte length header + auth data + binary.BigEndian.PutUint32(authBytes, uint32(length)) + copy(authBytes[4:], b.conf.Net.SASL.AuthIdentity+"\x00"+b.conf.Net.SASL.User+"\x00"+b.conf.Net.SASL.Password) + + requestTime := time.Now() + // Will be decremented in updateIncomingCommunicationMetrics (except error) + b.addRequestInFlightMetrics(1) + bytesWritten, err := b.write(authBytes) + b.updateOutgoingCommunicationMetrics(bytesWritten) + if err != nil { + b.addRequestInFlightMetrics(-1) + Logger.Printf("Failed to write SASL auth header to broker %s: %s\n", b.addr, err.Error()) + return err + } + + header := make([]byte, 4) + n, err := b.readFull(header) + b.updateIncomingCommunicationMetrics(n, time.Since(requestTime)) + // If the credentials are valid, we would get a 4 byte response filled with null characters. + // Otherwise, the broker closes the connection and we get an EOF + if err != nil { + Logger.Printf("Failed to read response while authenticating with SASL to broker %s: %s\n", b.addr, err.Error()) + return err + } + + DebugLogger.Printf("SASL authentication successful with broker %s:%v - %v\n", b.addr, n, header) + return nil +} + +// sendAndReceiveV1SASLPlainAuth flows the v1 sasl authentication using the kafka protocol +func (b *Broker) sendAndReceiveV1SASLPlainAuth() error { + correlationID := b.correlationID + + requestTime := time.Now() + + // Will be decremented in updateIncomingCommunicationMetrics (except error) + b.addRequestInFlightMetrics(1) + bytesWritten, err := b.sendSASLPlainAuthClientResponse(correlationID) + b.updateOutgoingCommunicationMetrics(bytesWritten) + + if err != nil { + b.addRequestInFlightMetrics(-1) + Logger.Printf("Failed to write SASL auth header to broker %s: %s\n", b.addr, err.Error()) + return err + } + + b.correlationID++ + + bytesRead, err := b.receiveSASLServerResponse(&SaslAuthenticateResponse{}, correlationID) + b.updateIncomingCommunicationMetrics(bytesRead, time.Since(requestTime)) + + // With v1 sasl we get an error message set in the response we can return + if err != nil { + Logger.Printf("Error returned from broker during SASL flow %s: %s\n", b.addr, err.Error()) + return err + } + + return nil +} + +// sendAndReceiveSASLOAuth performs the authentication flow as described by KIP-255 +// https://cwiki.apache.org/confluence/pages/viewpage.action?pageId=75968876 +func (b *Broker) sendAndReceiveSASLOAuth(provider AccessTokenProvider) error { + if err := b.sendAndReceiveSASLHandshake(SASLTypeOAuth, SASLHandshakeV1); err != nil { + return err + } + + token, err := provider.Token() + if err != nil { + return err + } + + message, err := buildClientFirstMessage(token) + if err != nil { + return err + } + + challenged, err := b.sendClientMessage(message) + if err != nil { + return err + } + + if challenged { + // Abort the token exchange. The broker returns the failure code. + _, err = b.sendClientMessage([]byte(`\x01`)) + } + + return err +} + +// sendClientMessage sends a SASL/OAUTHBEARER client message and returns true +// if the broker responds with a challenge, in which case the token is +// rejected. +func (b *Broker) sendClientMessage(message []byte) (bool, error) { + requestTime := time.Now() + // Will be decremented in updateIncomingCommunicationMetrics (except error) + b.addRequestInFlightMetrics(1) + correlationID := b.correlationID + + bytesWritten, err := b.sendSASLOAuthBearerClientMessage(message, correlationID) + b.updateOutgoingCommunicationMetrics(bytesWritten) + if err != nil { + b.addRequestInFlightMetrics(-1) + return false, err + } + + b.correlationID++ + + res := &SaslAuthenticateResponse{} + bytesRead, err := b.receiveSASLServerResponse(res, correlationID) + + requestLatency := time.Since(requestTime) + b.updateIncomingCommunicationMetrics(bytesRead, requestLatency) + + isChallenge := len(res.SaslAuthBytes) > 0 + + if isChallenge && err != nil { + Logger.Printf("Broker rejected authentication token: %s", res.SaslAuthBytes) + } + + return isChallenge, err +} + +func (b *Broker) sendAndReceiveSASLSCRAM() error { + if b.conf.Net.SASL.Version == SASLHandshakeV0 { + return b.sendAndReceiveSASLSCRAMv0() + } + return b.sendAndReceiveSASLSCRAMv1() +} + +func (b *Broker) sendAndReceiveSASLSCRAMv0() error { + if err := b.sendAndReceiveSASLHandshake(b.conf.Net.SASL.Mechanism, SASLHandshakeV0); err != nil { + return err + } + + scramClient := b.conf.Net.SASL.SCRAMClientGeneratorFunc() + if err := scramClient.Begin(b.conf.Net.SASL.User, b.conf.Net.SASL.Password, b.conf.Net.SASL.SCRAMAuthzID); err != nil { + return fmt.Errorf("failed to start SCRAM exchange with the server: %s", err.Error()) + } + + msg, err := scramClient.Step("") + if err != nil { + return fmt.Errorf("failed to advance the SCRAM exchange: %s", err.Error()) + } + + for !scramClient.Done() { + requestTime := time.Now() + // Will be decremented in updateIncomingCommunicationMetrics (except error) + b.addRequestInFlightMetrics(1) + length := len(msg) + authBytes := make([]byte, length+4) //4 byte length header + auth data + binary.BigEndian.PutUint32(authBytes, uint32(length)) + copy(authBytes[4:], []byte(msg)) + _, err := b.write(authBytes) + b.updateOutgoingCommunicationMetrics(length + 4) + if err != nil { + b.addRequestInFlightMetrics(-1) + Logger.Printf("Failed to write SASL auth header to broker %s: %s\n", b.addr, err.Error()) + return err + } + b.correlationID++ + header := make([]byte, 4) + _, err = b.readFull(header) + if err != nil { + b.addRequestInFlightMetrics(-1) + Logger.Printf("Failed to read response header while authenticating with SASL to broker %s: %s\n", b.addr, err.Error()) + return err + } + payload := make([]byte, int32(binary.BigEndian.Uint32(header))) + n, err := b.readFull(payload) + if err != nil { + b.addRequestInFlightMetrics(-1) + Logger.Printf("Failed to read response payload while authenticating with SASL to broker %s: %s\n", b.addr, err.Error()) + return err + } + b.updateIncomingCommunicationMetrics(n+4, time.Since(requestTime)) + msg, err = scramClient.Step(string(payload)) + if err != nil { + Logger.Println("SASL authentication failed", err) + return err + } + } + + DebugLogger.Println("SASL authentication succeeded") + return nil +} + +func (b *Broker) sendAndReceiveSASLSCRAMv1() error { + if err := b.sendAndReceiveSASLHandshake(b.conf.Net.SASL.Mechanism, SASLHandshakeV1); err != nil { + return err + } + + scramClient := b.conf.Net.SASL.SCRAMClientGeneratorFunc() + if err := scramClient.Begin(b.conf.Net.SASL.User, b.conf.Net.SASL.Password, b.conf.Net.SASL.SCRAMAuthzID); err != nil { + return fmt.Errorf("failed to start SCRAM exchange with the server: %s", err.Error()) + } + + msg, err := scramClient.Step("") + if err != nil { + return fmt.Errorf("failed to advance the SCRAM exchange: %s", err.Error()) + } + + for !scramClient.Done() { + requestTime := time.Now() + // Will be decremented in updateIncomingCommunicationMetrics (except error) + b.addRequestInFlightMetrics(1) + correlationID := b.correlationID + bytesWritten, err := b.sendSaslAuthenticateRequest(correlationID, []byte(msg)) + b.updateOutgoingCommunicationMetrics(bytesWritten) + if err != nil { + b.addRequestInFlightMetrics(-1) + Logger.Printf("Failed to write SASL auth header to broker %s: %s\n", b.addr, err.Error()) + return err + } + + b.correlationID++ + challenge, err := b.receiveSaslAuthenticateResponse(correlationID) + if err != nil { + b.addRequestInFlightMetrics(-1) + Logger.Printf("Failed to read response while authenticating with SASL to broker %s: %s\n", b.addr, err.Error()) + return err + } + + b.updateIncomingCommunicationMetrics(len(challenge), time.Since(requestTime)) + msg, err = scramClient.Step(string(challenge)) + if err != nil { + Logger.Println("SASL authentication failed", err) + return err + } + } + + DebugLogger.Println("SASL authentication succeeded") + return nil +} + +func (b *Broker) sendSaslAuthenticateRequest(correlationID int32, msg []byte) (int, error) { + rb := &SaslAuthenticateRequest{msg} + req := &request{correlationID: correlationID, clientID: b.conf.ClientID, body: rb} + buf, err := encode(req, b.conf.MetricRegistry) + if err != nil { + return 0, err + } + + return b.write(buf) +} + +func (b *Broker) receiveSaslAuthenticateResponse(correlationID int32) ([]byte, error) { + buf := make([]byte, responseLengthSize+correlationIDSize) + _, err := b.readFull(buf) + if err != nil { + return nil, err + } + + header := responseHeader{} + err = versionedDecode(buf, &header, 0) + if err != nil { + return nil, err + } + + if header.correlationID != correlationID { + return nil, fmt.Errorf("correlation ID didn't match, wanted %d, got %d", b.correlationID, header.correlationID) + } + + buf = make([]byte, header.length-correlationIDSize) + _, err = b.readFull(buf) + if err != nil { + return nil, err + } + + res := &SaslAuthenticateResponse{} + if err := versionedDecode(buf, res, 0); err != nil { + return nil, err + } + if res.Err != ErrNoError { + return nil, res.Err + } + return res.SaslAuthBytes, nil +} + +// Build SASL/OAUTHBEARER initial client response as described by RFC-7628 +// https://tools.ietf.org/html/rfc7628 +func buildClientFirstMessage(token *AccessToken) ([]byte, error) { + var ext string + + if token.Extensions != nil && len(token.Extensions) > 0 { + if _, ok := token.Extensions[SASLExtKeyAuth]; ok { + return []byte{}, fmt.Errorf("the extension `%s` is invalid", SASLExtKeyAuth) + } + ext = "\x01" + mapToString(token.Extensions, "=", "\x01") + } + + resp := []byte(fmt.Sprintf("n,,\x01auth=Bearer %s%s\x01\x01", token.Token, ext)) + + return resp, nil +} + +// mapToString returns a list of key-value pairs ordered by key. +// keyValSep separates the key from the value. elemSep separates each pair. +func mapToString(extensions map[string]string, keyValSep string, elemSep string) string { + buf := make([]string, 0, len(extensions)) + + for k, v := range extensions { + buf = append(buf, k+keyValSep+v) + } + + sort.Strings(buf) + + return strings.Join(buf, elemSep) +} + +func (b *Broker) sendSASLPlainAuthClientResponse(correlationID int32) (int, error) { + authBytes := []byte(b.conf.Net.SASL.AuthIdentity + "\x00" + b.conf.Net.SASL.User + "\x00" + b.conf.Net.SASL.Password) + rb := &SaslAuthenticateRequest{authBytes} + req := &request{correlationID: correlationID, clientID: b.conf.ClientID, body: rb} + buf, err := encode(req, b.conf.MetricRegistry) + if err != nil { + return 0, err + } + + return b.write(buf) +} + +func (b *Broker) sendSASLOAuthBearerClientMessage(initialResp []byte, correlationID int32) (int, error) { + rb := &SaslAuthenticateRequest{initialResp} + + req := &request{correlationID: correlationID, clientID: b.conf.ClientID, body: rb} + + buf, err := encode(req, b.conf.MetricRegistry) + if err != nil { + return 0, err + } + + return b.write(buf) +} + +func (b *Broker) receiveSASLServerResponse(res *SaslAuthenticateResponse, correlationID int32) (int, error) { + buf := make([]byte, responseLengthSize+correlationIDSize) + bytesRead, err := b.readFull(buf) + if err != nil { + return bytesRead, err + } + + header := responseHeader{} + err = versionedDecode(buf, &header, 0) + if err != nil { + return bytesRead, err + } + + if header.correlationID != correlationID { + return bytesRead, fmt.Errorf("correlation ID didn't match, wanted %d, got %d", b.correlationID, header.correlationID) + } + + buf = make([]byte, header.length-correlationIDSize) + c, err := b.readFull(buf) + bytesRead += c + if err != nil { + return bytesRead, err + } + + if err := versionedDecode(buf, res, 0); err != nil { + return bytesRead, err + } + + if res.Err != ErrNoError { + return bytesRead, res.Err + } + + return bytesRead, nil +} + +func (b *Broker) updateIncomingCommunicationMetrics(bytes int, requestLatency time.Duration) { + b.updateRequestLatencyAndInFlightMetrics(requestLatency) + b.responseRate.Mark(1) + + if b.brokerResponseRate != nil { + b.brokerResponseRate.Mark(1) + } + + responseSize := int64(bytes) + b.incomingByteRate.Mark(responseSize) + if b.brokerIncomingByteRate != nil { + b.brokerIncomingByteRate.Mark(responseSize) + } + + b.responseSize.Update(responseSize) + if b.brokerResponseSize != nil { + b.brokerResponseSize.Update(responseSize) + } +} + +func (b *Broker) updateRequestLatencyAndInFlightMetrics(requestLatency time.Duration) { + requestLatencyInMs := int64(requestLatency / time.Millisecond) + b.requestLatency.Update(requestLatencyInMs) + + if b.brokerRequestLatency != nil { + b.brokerRequestLatency.Update(requestLatencyInMs) + } + + b.addRequestInFlightMetrics(-1) +} + +func (b *Broker) addRequestInFlightMetrics(i int64) { + b.requestsInFlight.Inc(i) + if b.brokerRequestsInFlight != nil { + b.brokerRequestsInFlight.Inc(i) + } +} + +func (b *Broker) updateOutgoingCommunicationMetrics(bytes int) { + b.requestRate.Mark(1) + if b.brokerRequestRate != nil { + b.brokerRequestRate.Mark(1) + } + + requestSize := int64(bytes) + b.outgoingByteRate.Mark(requestSize) + if b.brokerOutgoingByteRate != nil { + b.brokerOutgoingByteRate.Mark(requestSize) + } + + b.requestSize.Update(requestSize) + if b.brokerRequestSize != nil { + b.brokerRequestSize.Update(requestSize) + } +} + +func (b *Broker) registerMetrics() { + b.brokerIncomingByteRate = b.registerMeter("incoming-byte-rate") + b.brokerRequestRate = b.registerMeter("request-rate") + b.brokerRequestSize = b.registerHistogram("request-size") + b.brokerRequestLatency = b.registerHistogram("request-latency-in-ms") + b.brokerOutgoingByteRate = b.registerMeter("outgoing-byte-rate") + b.brokerResponseRate = b.registerMeter("response-rate") + b.brokerResponseSize = b.registerHistogram("response-size") + b.brokerRequestsInFlight = b.registerCounter("requests-in-flight") + b.brokerThrottleTime = b.registerHistogram("throttle-time-in-ms") +} + +func (b *Broker) unregisterMetrics() { + for _, name := range b.registeredMetrics { + b.conf.MetricRegistry.Unregister(name) + } + b.registeredMetrics = nil +} + +func (b *Broker) registerMeter(name string) metrics.Meter { + nameForBroker := getMetricNameForBroker(name, b) + b.registeredMetrics = append(b.registeredMetrics, nameForBroker) + return metrics.GetOrRegisterMeter(nameForBroker, b.conf.MetricRegistry) +} + +func (b *Broker) registerHistogram(name string) metrics.Histogram { + nameForBroker := getMetricNameForBroker(name, b) + b.registeredMetrics = append(b.registeredMetrics, nameForBroker) + return getOrRegisterHistogram(nameForBroker, b.conf.MetricRegistry) +} + +func (b *Broker) registerCounter(name string) metrics.Counter { + nameForBroker := getMetricNameForBroker(name, b) + b.registeredMetrics = append(b.registeredMetrics, nameForBroker) + return metrics.GetOrRegisterCounter(nameForBroker, b.conf.MetricRegistry) +} + +func validServerNameTLS(addr string, cfg *tls.Config) *tls.Config { + if cfg == nil { + cfg = &tls.Config{ + MinVersion: tls.VersionTLS12, + } + } + if cfg.ServerName != "" { + return cfg + } + + c := cfg.Clone() + sn, _, err := net.SplitHostPort(addr) + if err != nil { + Logger.Println(fmt.Errorf("failed to get ServerName from addr %w", err)) + } + c.ServerName = sn + return c +} diff --git a/vendor/github.com/Shopify/sarama/client.go b/vendor/github.com/Shopify/sarama/client.go new file mode 100644 index 000000000000..1c69cb8d60d2 --- /dev/null +++ b/vendor/github.com/Shopify/sarama/client.go @@ -0,0 +1,1114 @@ +package sarama + +import ( + "math/rand" + "sort" + "sync" + "time" +) + +// Client is a generic Kafka client. It manages connections to one or more Kafka brokers. +// You MUST call Close() on a client to avoid leaks, it will not be garbage-collected +// automatically when it passes out of scope. It is safe to share a client amongst many +// users, however Kafka will process requests from a single client strictly in serial, +// so it is generally more efficient to use the default one client per producer/consumer. +type Client interface { + // Config returns the Config struct of the client. This struct should not be + // altered after it has been created. + Config() *Config + + // Controller returns the cluster controller broker. It will return a + // locally cached value if it's available. You can call RefreshController + // to update the cached value. Requires Kafka 0.10 or higher. + Controller() (*Broker, error) + + // RefreshController retrieves the cluster controller from fresh metadata + // and stores it in the local cache. Requires Kafka 0.10 or higher. + RefreshController() (*Broker, error) + + // Brokers returns the current set of active brokers as retrieved from cluster metadata. + Brokers() []*Broker + + // Broker returns the active Broker if available for the broker ID. + Broker(brokerID int32) (*Broker, error) + + // Topics returns the set of available topics as retrieved from cluster metadata. + Topics() ([]string, error) + + // Partitions returns the sorted list of all partition IDs for the given topic. + Partitions(topic string) ([]int32, error) + + // WritablePartitions returns the sorted list of all writable partition IDs for + // the given topic, where "writable" means "having a valid leader accepting + // writes". + WritablePartitions(topic string) ([]int32, error) + + // Leader returns the broker object that is the leader of the current + // topic/partition, as determined by querying the cluster metadata. + Leader(topic string, partitionID int32) (*Broker, error) + + // Replicas returns the set of all replica IDs for the given partition. + Replicas(topic string, partitionID int32) ([]int32, error) + + // InSyncReplicas returns the set of all in-sync replica IDs for the given + // partition. In-sync replicas are replicas which are fully caught up with + // the partition leader. + InSyncReplicas(topic string, partitionID int32) ([]int32, error) + + // OfflineReplicas returns the set of all offline replica IDs for the given + // partition. Offline replicas are replicas which are offline + OfflineReplicas(topic string, partitionID int32) ([]int32, error) + + // RefreshBrokers takes a list of addresses to be used as seed brokers. + // Existing broker connections are closed and the updated list of seed brokers + // will be used for the next metadata fetch. + RefreshBrokers(addrs []string) error + + // RefreshMetadata takes a list of topics and queries the cluster to refresh the + // available metadata for those topics. If no topics are provided, it will refresh + // metadata for all topics. + RefreshMetadata(topics ...string) error + + // GetOffset queries the cluster to get the most recent available offset at the + // given time (in milliseconds) on the topic/partition combination. + // Time should be OffsetOldest for the earliest available offset, + // OffsetNewest for the offset of the message that will be produced next, or a time. + GetOffset(topic string, partitionID int32, time int64) (int64, error) + + // Coordinator returns the coordinating broker for a consumer group. It will + // return a locally cached value if it's available. You can call + // RefreshCoordinator to update the cached value. This function only works on + // Kafka 0.8.2 and higher. + Coordinator(consumerGroup string) (*Broker, error) + + // RefreshCoordinator retrieves the coordinator for a consumer group and stores it + // in local cache. This function only works on Kafka 0.8.2 and higher. + RefreshCoordinator(consumerGroup string) error + + // InitProducerID retrieves information required for Idempotent Producer + InitProducerID() (*InitProducerIDResponse, error) + + // Close shuts down all broker connections managed by this client. It is required + // to call this function before a client object passes out of scope, as it will + // otherwise leak memory. You must close any Producers or Consumers using a client + // before you close the client. + Close() error + + // Closed returns true if the client has already had Close called on it + Closed() bool +} + +const ( + // OffsetNewest stands for the log head offset, i.e. the offset that will be + // assigned to the next message that will be produced to the partition. You + // can send this to a client's GetOffset method to get this offset, or when + // calling ConsumePartition to start consuming new messages. + OffsetNewest int64 = -1 + // OffsetOldest stands for the oldest offset available on the broker for a + // partition. You can send this to a client's GetOffset method to get this + // offset, or when calling ConsumePartition to start consuming from the + // oldest offset that is still available on the broker. + OffsetOldest int64 = -2 +) + +type client struct { + conf *Config + closer, closed chan none // for shutting down background metadata updater + + // the broker addresses given to us through the constructor are not guaranteed to be returned in + // the cluster metadata (I *think* it only returns brokers who are currently leading partitions?) + // so we store them separately + seedBrokers []*Broker + deadSeeds []*Broker + + controllerID int32 // cluster controller broker id + brokers map[int32]*Broker // maps broker ids to brokers + metadata map[string]map[int32]*PartitionMetadata // maps topics to partition ids to metadata + metadataTopics map[string]none // topics that need to collect metadata + coordinators map[string]int32 // Maps consumer group names to coordinating broker IDs + + // If the number of partitions is large, we can get some churn calling cachedPartitions, + // so the result is cached. It is important to update this value whenever metadata is changed + cachedPartitionsResults map[string][maxPartitionIndex][]int32 + + lock sync.RWMutex // protects access to the maps that hold cluster state. +} + +// NewClient creates a new Client. It connects to one of the given broker addresses +// and uses that broker to automatically fetch metadata on the rest of the kafka cluster. If metadata cannot +// be retrieved from any of the given broker addresses, the client is not created. +func NewClient(addrs []string, conf *Config) (Client, error) { + DebugLogger.Println("Initializing new client") + + if conf == nil { + conf = NewConfig() + } + + if err := conf.Validate(); err != nil { + return nil, err + } + + if len(addrs) < 1 { + return nil, ConfigurationError("You must provide at least one broker address") + } + + client := &client{ + conf: conf, + closer: make(chan none), + closed: make(chan none), + brokers: make(map[int32]*Broker), + metadata: make(map[string]map[int32]*PartitionMetadata), + metadataTopics: make(map[string]none), + cachedPartitionsResults: make(map[string][maxPartitionIndex][]int32), + coordinators: make(map[string]int32), + } + + client.randomizeSeedBrokers(addrs) + + if conf.Metadata.Full { + // do an initial fetch of all cluster metadata by specifying an empty list of topics + err := client.RefreshMetadata() + switch err { + case nil: + break + case ErrLeaderNotAvailable, ErrReplicaNotAvailable, ErrTopicAuthorizationFailed, ErrClusterAuthorizationFailed: + // indicates that maybe part of the cluster is down, but is not fatal to creating the client + Logger.Println(err) + default: + close(client.closed) // we haven't started the background updater yet, so we have to do this manually + _ = client.Close() + return nil, err + } + } + go withRecover(client.backgroundMetadataUpdater) + + DebugLogger.Println("Successfully initialized new client") + + return client, nil +} + +func (client *client) Config() *Config { + return client.conf +} + +func (client *client) Brokers() []*Broker { + client.lock.RLock() + defer client.lock.RUnlock() + brokers := make([]*Broker, 0, len(client.brokers)) + for _, broker := range client.brokers { + brokers = append(brokers, broker) + } + return brokers +} + +func (client *client) Broker(brokerID int32) (*Broker, error) { + client.lock.RLock() + defer client.lock.RUnlock() + broker, ok := client.brokers[brokerID] + if !ok { + return nil, ErrBrokerNotFound + } + _ = broker.Open(client.conf) + return broker, nil +} + +func (client *client) InitProducerID() (*InitProducerIDResponse, error) { + err := ErrOutOfBrokers + for broker := client.any(); broker != nil; broker = client.any() { + var response *InitProducerIDResponse + req := &InitProducerIDRequest{} + + response, err = broker.InitProducerID(req) + switch err.(type) { + case nil: + return response, nil + default: + // some error, remove that broker and try again + Logger.Printf("Client got error from broker %d when issuing InitProducerID : %v\n", broker.ID(), err) + _ = broker.Close() + client.deregisterBroker(broker) + } + } + + return nil, err +} + +func (client *client) Close() error { + if client.Closed() { + // Chances are this is being called from a defer() and the error will go unobserved + // so we go ahead and log the event in this case. + Logger.Printf("Close() called on already closed client") + return ErrClosedClient + } + + // shutdown and wait for the background thread before we take the lock, to avoid races + close(client.closer) + <-client.closed + + client.lock.Lock() + defer client.lock.Unlock() + DebugLogger.Println("Closing Client") + + for _, broker := range client.brokers { + safeAsyncClose(broker) + } + + for _, broker := range client.seedBrokers { + safeAsyncClose(broker) + } + + client.brokers = nil + client.metadata = nil + client.metadataTopics = nil + + return nil +} + +func (client *client) Closed() bool { + client.lock.RLock() + defer client.lock.RUnlock() + + return client.brokers == nil +} + +func (client *client) Topics() ([]string, error) { + if client.Closed() { + return nil, ErrClosedClient + } + + client.lock.RLock() + defer client.lock.RUnlock() + + ret := make([]string, 0, len(client.metadata)) + for topic := range client.metadata { + ret = append(ret, topic) + } + + return ret, nil +} + +func (client *client) MetadataTopics() ([]string, error) { + if client.Closed() { + return nil, ErrClosedClient + } + + client.lock.RLock() + defer client.lock.RUnlock() + + ret := make([]string, 0, len(client.metadataTopics)) + for topic := range client.metadataTopics { + ret = append(ret, topic) + } + + return ret, nil +} + +func (client *client) Partitions(topic string) ([]int32, error) { + if client.Closed() { + return nil, ErrClosedClient + } + + partitions := client.cachedPartitions(topic, allPartitions) + + if len(partitions) == 0 { + err := client.RefreshMetadata(topic) + if err != nil { + return nil, err + } + partitions = client.cachedPartitions(topic, allPartitions) + } + + // no partitions found after refresh metadata + if len(partitions) == 0 { + return nil, ErrUnknownTopicOrPartition + } + + return partitions, nil +} + +func (client *client) WritablePartitions(topic string) ([]int32, error) { + if client.Closed() { + return nil, ErrClosedClient + } + + partitions := client.cachedPartitions(topic, writablePartitions) + + // len==0 catches when it's nil (no such topic) and the odd case when every single + // partition is undergoing leader election simultaneously. Callers have to be able to handle + // this function returning an empty slice (which is a valid return value) but catching it + // here the first time (note we *don't* catch it below where we return ErrUnknownTopicOrPartition) triggers + // a metadata refresh as a nicety so callers can just try again and don't have to manually + // trigger a refresh (otherwise they'd just keep getting a stale cached copy). + if len(partitions) == 0 { + err := client.RefreshMetadata(topic) + if err != nil { + return nil, err + } + partitions = client.cachedPartitions(topic, writablePartitions) + } + + if partitions == nil { + return nil, ErrUnknownTopicOrPartition + } + + return partitions, nil +} + +func (client *client) Replicas(topic string, partitionID int32) ([]int32, error) { + if client.Closed() { + return nil, ErrClosedClient + } + + metadata := client.cachedMetadata(topic, partitionID) + + if metadata == nil { + err := client.RefreshMetadata(topic) + if err != nil { + return nil, err + } + metadata = client.cachedMetadata(topic, partitionID) + } + + if metadata == nil { + return nil, ErrUnknownTopicOrPartition + } + + if metadata.Err == ErrReplicaNotAvailable { + return dupInt32Slice(metadata.Replicas), metadata.Err + } + return dupInt32Slice(metadata.Replicas), nil +} + +func (client *client) InSyncReplicas(topic string, partitionID int32) ([]int32, error) { + if client.Closed() { + return nil, ErrClosedClient + } + + metadata := client.cachedMetadata(topic, partitionID) + + if metadata == nil { + err := client.RefreshMetadata(topic) + if err != nil { + return nil, err + } + metadata = client.cachedMetadata(topic, partitionID) + } + + if metadata == nil { + return nil, ErrUnknownTopicOrPartition + } + + if metadata.Err == ErrReplicaNotAvailable { + return dupInt32Slice(metadata.Isr), metadata.Err + } + return dupInt32Slice(metadata.Isr), nil +} + +func (client *client) OfflineReplicas(topic string, partitionID int32) ([]int32, error) { + if client.Closed() { + return nil, ErrClosedClient + } + + metadata := client.cachedMetadata(topic, partitionID) + + if metadata == nil { + err := client.RefreshMetadata(topic) + if err != nil { + return nil, err + } + metadata = client.cachedMetadata(topic, partitionID) + } + + if metadata == nil { + return nil, ErrUnknownTopicOrPartition + } + + if metadata.Err == ErrReplicaNotAvailable { + return dupInt32Slice(metadata.OfflineReplicas), metadata.Err + } + return dupInt32Slice(metadata.OfflineReplicas), nil +} + +func (client *client) Leader(topic string, partitionID int32) (*Broker, error) { + if client.Closed() { + return nil, ErrClosedClient + } + + leader, err := client.cachedLeader(topic, partitionID) + + if leader == nil { + err = client.RefreshMetadata(topic) + if err != nil { + return nil, err + } + leader, err = client.cachedLeader(topic, partitionID) + } + + return leader, err +} + +func (client *client) RefreshBrokers(addrs []string) error { + if client.Closed() { + return ErrClosedClient + } + + client.lock.Lock() + defer client.lock.Unlock() + + for _, broker := range client.brokers { + _ = broker.Close() + delete(client.brokers, broker.ID()) + } + + client.seedBrokers = nil + client.deadSeeds = nil + + client.randomizeSeedBrokers(addrs) + + return nil +} + +func (client *client) RefreshMetadata(topics ...string) error { + if client.Closed() { + return ErrClosedClient + } + + // Prior to 0.8.2, Kafka will throw exceptions on an empty topic and not return a proper + // error. This handles the case by returning an error instead of sending it + // off to Kafka. See: https://github.com/Shopify/sarama/pull/38#issuecomment-26362310 + for _, topic := range topics { + if topic == "" { + return ErrInvalidTopic // this is the error that 0.8.2 and later correctly return + } + } + + deadline := time.Time{} + if client.conf.Metadata.Timeout > 0 { + deadline = time.Now().Add(client.conf.Metadata.Timeout) + } + return client.tryRefreshMetadata(topics, client.conf.Metadata.Retry.Max, deadline) +} + +func (client *client) GetOffset(topic string, partitionID int32, time int64) (int64, error) { + if client.Closed() { + return -1, ErrClosedClient + } + + offset, err := client.getOffset(topic, partitionID, time) + if err != nil { + if err := client.RefreshMetadata(topic); err != nil { + return -1, err + } + return client.getOffset(topic, partitionID, time) + } + + return offset, err +} + +func (client *client) Controller() (*Broker, error) { + if client.Closed() { + return nil, ErrClosedClient + } + + if !client.conf.Version.IsAtLeast(V0_10_0_0) { + return nil, ErrUnsupportedVersion + } + + controller := client.cachedController() + if controller == nil { + if err := client.refreshMetadata(); err != nil { + return nil, err + } + controller = client.cachedController() + } + + if controller == nil { + return nil, ErrControllerNotAvailable + } + + _ = controller.Open(client.conf) + return controller, nil +} + +// deregisterController removes the cached controllerID +func (client *client) deregisterController() { + client.lock.Lock() + defer client.lock.Unlock() + delete(client.brokers, client.controllerID) +} + +// RefreshController retrieves the cluster controller from fresh metadata +// and stores it in the local cache. Requires Kafka 0.10 or higher. +func (client *client) RefreshController() (*Broker, error) { + if client.Closed() { + return nil, ErrClosedClient + } + + client.deregisterController() + + if err := client.refreshMetadata(); err != nil { + return nil, err + } + + controller := client.cachedController() + if controller == nil { + return nil, ErrControllerNotAvailable + } + + _ = controller.Open(client.conf) + return controller, nil +} + +func (client *client) Coordinator(consumerGroup string) (*Broker, error) { + if client.Closed() { + return nil, ErrClosedClient + } + + coordinator := client.cachedCoordinator(consumerGroup) + + if coordinator == nil { + if err := client.RefreshCoordinator(consumerGroup); err != nil { + return nil, err + } + coordinator = client.cachedCoordinator(consumerGroup) + } + + if coordinator == nil { + return nil, ErrConsumerCoordinatorNotAvailable + } + + _ = coordinator.Open(client.conf) + return coordinator, nil +} + +func (client *client) RefreshCoordinator(consumerGroup string) error { + if client.Closed() { + return ErrClosedClient + } + + response, err := client.getConsumerMetadata(consumerGroup, client.conf.Metadata.Retry.Max) + if err != nil { + return err + } + + client.lock.Lock() + defer client.lock.Unlock() + client.registerBroker(response.Coordinator) + client.coordinators[consumerGroup] = response.Coordinator.ID() + return nil +} + +// private broker management helpers + +func (client *client) randomizeSeedBrokers(addrs []string) { + random := rand.New(rand.NewSource(time.Now().UnixNano())) + for _, index := range random.Perm(len(addrs)) { + client.seedBrokers = append(client.seedBrokers, NewBroker(addrs[index])) + } +} + +func (client *client) updateBroker(brokers []*Broker) { + currentBroker := make(map[int32]*Broker, len(brokers)) + + for _, broker := range brokers { + currentBroker[broker.ID()] = broker + if client.brokers[broker.ID()] == nil { // add new broker + client.brokers[broker.ID()] = broker + DebugLogger.Printf("client/brokers registered new broker #%d at %s", broker.ID(), broker.Addr()) + } else if broker.Addr() != client.brokers[broker.ID()].Addr() { // replace broker with new address + safeAsyncClose(client.brokers[broker.ID()]) + client.brokers[broker.ID()] = broker + Logger.Printf("client/brokers replaced registered broker #%d with %s", broker.ID(), broker.Addr()) + } + } + + for id, broker := range client.brokers { + if _, exist := currentBroker[id]; !exist { // remove old broker + safeAsyncClose(broker) + delete(client.brokers, id) + Logger.Printf("client/broker remove invalid broker #%d with %s", broker.ID(), broker.Addr()) + } + } +} + +// registerBroker makes sure a broker received by a Metadata or Coordinator request is registered +// in the brokers map. It returns the broker that is registered, which may be the provided broker, +// or a previously registered Broker instance. You must hold the write lock before calling this function. +func (client *client) registerBroker(broker *Broker) { + if client.brokers == nil { + Logger.Printf("cannot register broker #%d at %s, client already closed", broker.ID(), broker.Addr()) + return + } + + if client.brokers[broker.ID()] == nil { + client.brokers[broker.ID()] = broker + DebugLogger.Printf("client/brokers registered new broker #%d at %s", broker.ID(), broker.Addr()) + } else if broker.Addr() != client.brokers[broker.ID()].Addr() { + safeAsyncClose(client.brokers[broker.ID()]) + client.brokers[broker.ID()] = broker + Logger.Printf("client/brokers replaced registered broker #%d with %s", broker.ID(), broker.Addr()) + } +} + +// deregisterBroker removes a broker from the seedsBroker list, and if it's +// not the seedbroker, removes it from brokers map completely. +func (client *client) deregisterBroker(broker *Broker) { + client.lock.Lock() + defer client.lock.Unlock() + + if len(client.seedBrokers) > 0 && broker == client.seedBrokers[0] { + client.deadSeeds = append(client.deadSeeds, broker) + client.seedBrokers = client.seedBrokers[1:] + } else { + // we do this so that our loop in `tryRefreshMetadata` doesn't go on forever, + // but we really shouldn't have to; once that loop is made better this case can be + // removed, and the function generally can be renamed from `deregisterBroker` to + // `nextSeedBroker` or something + DebugLogger.Printf("client/brokers deregistered broker #%d at %s", broker.ID(), broker.Addr()) + delete(client.brokers, broker.ID()) + } +} + +func (client *client) resurrectDeadBrokers() { + client.lock.Lock() + defer client.lock.Unlock() + + Logger.Printf("client/brokers resurrecting %d dead seed brokers", len(client.deadSeeds)) + client.seedBrokers = append(client.seedBrokers, client.deadSeeds...) + client.deadSeeds = nil +} + +func (client *client) any() *Broker { + client.lock.RLock() + defer client.lock.RUnlock() + + if len(client.seedBrokers) > 0 { + _ = client.seedBrokers[0].Open(client.conf) + return client.seedBrokers[0] + } + + // not guaranteed to be random *or* deterministic + for _, broker := range client.brokers { + _ = broker.Open(client.conf) + return broker + } + + return nil +} + +// private caching/lazy metadata helpers + +type partitionType int + +const ( + allPartitions partitionType = iota + writablePartitions + // If you add any more types, update the partition cache in update() + + // Ensure this is the last partition type value + maxPartitionIndex +) + +func (client *client) cachedMetadata(topic string, partitionID int32) *PartitionMetadata { + client.lock.RLock() + defer client.lock.RUnlock() + + partitions := client.metadata[topic] + if partitions != nil { + return partitions[partitionID] + } + + return nil +} + +func (client *client) cachedPartitions(topic string, partitionSet partitionType) []int32 { + client.lock.RLock() + defer client.lock.RUnlock() + + partitions, exists := client.cachedPartitionsResults[topic] + + if !exists { + return nil + } + return partitions[partitionSet] +} + +func (client *client) setPartitionCache(topic string, partitionSet partitionType) []int32 { + partitions := client.metadata[topic] + + if partitions == nil { + return nil + } + + ret := make([]int32, 0, len(partitions)) + for _, partition := range partitions { + if partitionSet == writablePartitions && partition.Err == ErrLeaderNotAvailable { + continue + } + ret = append(ret, partition.ID) + } + + sort.Sort(int32Slice(ret)) + return ret +} + +func (client *client) cachedLeader(topic string, partitionID int32) (*Broker, error) { + client.lock.RLock() + defer client.lock.RUnlock() + + partitions := client.metadata[topic] + if partitions != nil { + metadata, ok := partitions[partitionID] + if ok { + if metadata.Err == ErrLeaderNotAvailable { + return nil, ErrLeaderNotAvailable + } + b := client.brokers[metadata.Leader] + if b == nil { + return nil, ErrLeaderNotAvailable + } + _ = b.Open(client.conf) + return b, nil + } + } + + return nil, ErrUnknownTopicOrPartition +} + +func (client *client) getOffset(topic string, partitionID int32, time int64) (int64, error) { + broker, err := client.Leader(topic, partitionID) + if err != nil { + return -1, err + } + + request := &OffsetRequest{} + if client.conf.Version.IsAtLeast(V0_10_1_0) { + request.Version = 1 + } + request.AddBlock(topic, partitionID, time, 1) + + response, err := broker.GetAvailableOffsets(request) + if err != nil { + _ = broker.Close() + return -1, err + } + + block := response.GetBlock(topic, partitionID) + if block == nil { + _ = broker.Close() + return -1, ErrIncompleteResponse + } + if block.Err != ErrNoError { + return -1, block.Err + } + if len(block.Offsets) != 1 { + return -1, ErrOffsetOutOfRange + } + + return block.Offsets[0], nil +} + +// core metadata update logic + +func (client *client) backgroundMetadataUpdater() { + defer close(client.closed) + + if client.conf.Metadata.RefreshFrequency == time.Duration(0) { + return + } + + ticker := time.NewTicker(client.conf.Metadata.RefreshFrequency) + defer ticker.Stop() + + for { + select { + case <-ticker.C: + if err := client.refreshMetadata(); err != nil { + Logger.Println("Client background metadata update:", err) + } + case <-client.closer: + return + } + } +} + +func (client *client) refreshMetadata() error { + var topics []string + + if !client.conf.Metadata.Full { + if specificTopics, err := client.MetadataTopics(); err != nil { + return err + } else if len(specificTopics) == 0 { + return ErrNoTopicsToUpdateMetadata + } else { + topics = specificTopics + } + } + + if err := client.RefreshMetadata(topics...); err != nil { + return err + } + + return nil +} + +func (client *client) tryRefreshMetadata(topics []string, attemptsRemaining int, deadline time.Time) error { + pastDeadline := func(backoff time.Duration) bool { + if !deadline.IsZero() && time.Now().Add(backoff).After(deadline) { + // we are past the deadline + return true + } + return false + } + retry := func(err error) error { + if attemptsRemaining > 0 { + backoff := client.computeBackoff(attemptsRemaining) + if pastDeadline(backoff) { + Logger.Println("client/metadata skipping last retries as we would go past the metadata timeout") + return err + } + Logger.Printf("client/metadata retrying after %dms... (%d attempts remaining)\n", backoff/time.Millisecond, attemptsRemaining) + if backoff > 0 { + time.Sleep(backoff) + } + return client.tryRefreshMetadata(topics, attemptsRemaining-1, deadline) + } + return err + } + + broker := client.any() + for ; broker != nil && !pastDeadline(0); broker = client.any() { + allowAutoTopicCreation := client.conf.Metadata.AllowAutoTopicCreation + if len(topics) > 0 { + DebugLogger.Printf("client/metadata fetching metadata for %v from broker %s\n", topics, broker.addr) + } else { + allowAutoTopicCreation = false + DebugLogger.Printf("client/metadata fetching metadata for all topics from broker %s\n", broker.addr) + } + + req := &MetadataRequest{Topics: topics, AllowAutoTopicCreation: allowAutoTopicCreation} + if client.conf.Version.IsAtLeast(V1_0_0_0) { + req.Version = 5 + } else if client.conf.Version.IsAtLeast(V0_10_0_0) { + req.Version = 1 + } + response, err := broker.GetMetadata(req) + switch err := err.(type) { + case nil: + allKnownMetaData := len(topics) == 0 + // valid response, use it + shouldRetry, err := client.updateMetadata(response, allKnownMetaData) + if shouldRetry { + Logger.Println("client/metadata found some partitions to be leaderless") + return retry(err) // note: err can be nil + } + return err + + case PacketEncodingError: + // didn't even send, return the error + return err + + case KError: + // if SASL auth error return as this _should_ be a non retryable err for all brokers + if err == ErrSASLAuthenticationFailed { + Logger.Println("client/metadata failed SASL authentication") + return err + } + + if err == ErrTopicAuthorizationFailed { + Logger.Println("client is not authorized to access this topic. The topics were: ", topics) + return err + } + // else remove that broker and try again + Logger.Printf("client/metadata got error from broker %d while fetching metadata: %v\n", broker.ID(), err) + _ = broker.Close() + client.deregisterBroker(broker) + + default: + // some other error, remove that broker and try again + Logger.Printf("client/metadata got error from broker %d while fetching metadata: %v\n", broker.ID(), err) + _ = broker.Close() + client.deregisterBroker(broker) + } + } + + if broker != nil { + Logger.Printf("client/metadata not fetching metadata from broker %s as we would go past the metadata timeout\n", broker.addr) + return retry(ErrOutOfBrokers) + } + + Logger.Println("client/metadata no available broker to send metadata request to") + client.resurrectDeadBrokers() + return retry(ErrOutOfBrokers) +} + +// if no fatal error, returns a list of topics that need retrying due to ErrLeaderNotAvailable +func (client *client) updateMetadata(data *MetadataResponse, allKnownMetaData bool) (retry bool, err error) { + if client.Closed() { + return + } + + client.lock.Lock() + defer client.lock.Unlock() + + // For all the brokers we received: + // - if it is a new ID, save it + // - if it is an existing ID, but the address we have is stale, discard the old one and save it + // - if some brokers is not exist in it, remove old broker + // - otherwise ignore it, replacing our existing one would just bounce the connection + client.updateBroker(data.Brokers) + + client.controllerID = data.ControllerID + + if allKnownMetaData { + client.metadata = make(map[string]map[int32]*PartitionMetadata) + client.metadataTopics = make(map[string]none) + client.cachedPartitionsResults = make(map[string][maxPartitionIndex][]int32) + } + for _, topic := range data.Topics { + // topics must be added firstly to `metadataTopics` to guarantee that all + // requested topics must be recorded to keep them trackable for periodically + // metadata refresh. + if _, exists := client.metadataTopics[topic.Name]; !exists { + client.metadataTopics[topic.Name] = none{} + } + delete(client.metadata, topic.Name) + delete(client.cachedPartitionsResults, topic.Name) + + switch topic.Err { + case ErrNoError: + // no-op + case ErrInvalidTopic, ErrTopicAuthorizationFailed: // don't retry, don't store partial results + err = topic.Err + continue + case ErrUnknownTopicOrPartition: // retry, do not store partial partition results + err = topic.Err + retry = true + continue + case ErrLeaderNotAvailable: // retry, but store partial partition results + retry = true + default: // don't retry, don't store partial results + Logger.Printf("Unexpected topic-level metadata error: %s", topic.Err) + err = topic.Err + continue + } + + client.metadata[topic.Name] = make(map[int32]*PartitionMetadata, len(topic.Partitions)) + for _, partition := range topic.Partitions { + client.metadata[topic.Name][partition.ID] = partition + if partition.Err == ErrLeaderNotAvailable { + retry = true + } + } + + var partitionCache [maxPartitionIndex][]int32 + partitionCache[allPartitions] = client.setPartitionCache(topic.Name, allPartitions) + partitionCache[writablePartitions] = client.setPartitionCache(topic.Name, writablePartitions) + client.cachedPartitionsResults[topic.Name] = partitionCache + } + + return +} + +func (client *client) cachedCoordinator(consumerGroup string) *Broker { + client.lock.RLock() + defer client.lock.RUnlock() + if coordinatorID, ok := client.coordinators[consumerGroup]; ok { + return client.brokers[coordinatorID] + } + return nil +} + +func (client *client) cachedController() *Broker { + client.lock.RLock() + defer client.lock.RUnlock() + + return client.brokers[client.controllerID] +} + +func (client *client) computeBackoff(attemptsRemaining int) time.Duration { + if client.conf.Metadata.Retry.BackoffFunc != nil { + maxRetries := client.conf.Metadata.Retry.Max + retries := maxRetries - attemptsRemaining + return client.conf.Metadata.Retry.BackoffFunc(retries, maxRetries) + } + return client.conf.Metadata.Retry.Backoff +} + +func (client *client) getConsumerMetadata(consumerGroup string, attemptsRemaining int) (*FindCoordinatorResponse, error) { + retry := func(err error) (*FindCoordinatorResponse, error) { + if attemptsRemaining > 0 { + backoff := client.computeBackoff(attemptsRemaining) + Logger.Printf("client/coordinator retrying after %dms... (%d attempts remaining)\n", backoff/time.Millisecond, attemptsRemaining) + time.Sleep(backoff) + return client.getConsumerMetadata(consumerGroup, attemptsRemaining-1) + } + return nil, err + } + + for broker := client.any(); broker != nil; broker = client.any() { + DebugLogger.Printf("client/coordinator requesting coordinator for consumergroup %s from %s\n", consumerGroup, broker.Addr()) + + request := new(FindCoordinatorRequest) + request.CoordinatorKey = consumerGroup + request.CoordinatorType = CoordinatorGroup + + response, err := broker.FindCoordinator(request) + if err != nil { + Logger.Printf("client/coordinator request to broker %s failed: %s\n", broker.Addr(), err) + + switch err.(type) { + case PacketEncodingError: + return nil, err + default: + _ = broker.Close() + client.deregisterBroker(broker) + continue + } + } + + switch response.Err { + case ErrNoError: + DebugLogger.Printf("client/coordinator coordinator for consumergroup %s is #%d (%s)\n", consumerGroup, response.Coordinator.ID(), response.Coordinator.Addr()) + return response, nil + + case ErrConsumerCoordinatorNotAvailable: + Logger.Printf("client/coordinator coordinator for consumer group %s is not available\n", consumerGroup) + + // This is very ugly, but this scenario will only happen once per cluster. + // The __consumer_offsets topic only has to be created one time. + // The number of partitions not configurable, but partition 0 should always exist. + if _, err := client.Leader("__consumer_offsets", 0); err != nil { + Logger.Printf("client/coordinator the __consumer_offsets topic is not initialized completely yet. Waiting 2 seconds...\n") + time.Sleep(2 * time.Second) + } + + return retry(ErrConsumerCoordinatorNotAvailable) + case ErrGroupAuthorizationFailed: + Logger.Printf("client was not authorized to access group %s while attempting to find coordinator", consumerGroup) + return retry(ErrGroupAuthorizationFailed) + + default: + return nil, response.Err + } + } + + Logger.Println("client/coordinator no available broker to send consumer metadata request to") + client.resurrectDeadBrokers() + return retry(ErrOutOfBrokers) +} + +// nopCloserClient embeds an existing Client, but disables +// the Close method (yet all other methods pass +// through unchanged). This is for use in larger structs +// where it is undesirable to close the client that was +// passed in by the caller. +type nopCloserClient struct { + Client +} + +// Close intercepts and purposely does not call the underlying +// client's Close() method. +func (ncc *nopCloserClient) Close() error { + return nil +} diff --git a/vendor/github.com/Shopify/sarama/compress.go b/vendor/github.com/Shopify/sarama/compress.go new file mode 100644 index 000000000000..12cd7c3d5104 --- /dev/null +++ b/vendor/github.com/Shopify/sarama/compress.go @@ -0,0 +1,194 @@ +package sarama + +import ( + "bytes" + "compress/gzip" + "fmt" + "sync" + + snappy "github.com/eapache/go-xerial-snappy" + "github.com/pierrec/lz4" +) + +var ( + lz4WriterPool = sync.Pool{ + New: func() interface{} { + return lz4.NewWriter(nil) + }, + } + + gzipWriterPool = sync.Pool{ + New: func() interface{} { + return gzip.NewWriter(nil) + }, + } + gzipWriterPoolForCompressionLevel1 = sync.Pool{ + New: func() interface{} { + gz, err := gzip.NewWriterLevel(nil, 1) + if err != nil { + panic(err) + } + return gz + }, + } + gzipWriterPoolForCompressionLevel2 = sync.Pool{ + New: func() interface{} { + gz, err := gzip.NewWriterLevel(nil, 2) + if err != nil { + panic(err) + } + return gz + }, + } + gzipWriterPoolForCompressionLevel3 = sync.Pool{ + New: func() interface{} { + gz, err := gzip.NewWriterLevel(nil, 3) + if err != nil { + panic(err) + } + return gz + }, + } + gzipWriterPoolForCompressionLevel4 = sync.Pool{ + New: func() interface{} { + gz, err := gzip.NewWriterLevel(nil, 4) + if err != nil { + panic(err) + } + return gz + }, + } + gzipWriterPoolForCompressionLevel5 = sync.Pool{ + New: func() interface{} { + gz, err := gzip.NewWriterLevel(nil, 5) + if err != nil { + panic(err) + } + return gz + }, + } + gzipWriterPoolForCompressionLevel6 = sync.Pool{ + New: func() interface{} { + gz, err := gzip.NewWriterLevel(nil, 6) + if err != nil { + panic(err) + } + return gz + }, + } + gzipWriterPoolForCompressionLevel7 = sync.Pool{ + New: func() interface{} { + gz, err := gzip.NewWriterLevel(nil, 7) + if err != nil { + panic(err) + } + return gz + }, + } + gzipWriterPoolForCompressionLevel8 = sync.Pool{ + New: func() interface{} { + gz, err := gzip.NewWriterLevel(nil, 8) + if err != nil { + panic(err) + } + return gz + }, + } + gzipWriterPoolForCompressionLevel9 = sync.Pool{ + New: func() interface{} { + gz, err := gzip.NewWriterLevel(nil, 9) + if err != nil { + panic(err) + } + return gz + }, + } +) + +func compress(cc CompressionCodec, level int, data []byte) ([]byte, error) { + switch cc { + case CompressionNone: + return data, nil + case CompressionGZIP: + var ( + err error + buf bytes.Buffer + writer *gzip.Writer + ) + + switch level { + case CompressionLevelDefault: + writer = gzipWriterPool.Get().(*gzip.Writer) + defer gzipWriterPool.Put(writer) + writer.Reset(&buf) + case 1: + writer = gzipWriterPoolForCompressionLevel1.Get().(*gzip.Writer) + defer gzipWriterPoolForCompressionLevel1.Put(writer) + writer.Reset(&buf) + case 2: + writer = gzipWriterPoolForCompressionLevel2.Get().(*gzip.Writer) + defer gzipWriterPoolForCompressionLevel2.Put(writer) + writer.Reset(&buf) + case 3: + writer = gzipWriterPoolForCompressionLevel3.Get().(*gzip.Writer) + defer gzipWriterPoolForCompressionLevel3.Put(writer) + writer.Reset(&buf) + case 4: + writer = gzipWriterPoolForCompressionLevel4.Get().(*gzip.Writer) + defer gzipWriterPoolForCompressionLevel4.Put(writer) + writer.Reset(&buf) + case 5: + writer = gzipWriterPoolForCompressionLevel5.Get().(*gzip.Writer) + defer gzipWriterPoolForCompressionLevel5.Put(writer) + writer.Reset(&buf) + case 6: + writer = gzipWriterPoolForCompressionLevel6.Get().(*gzip.Writer) + defer gzipWriterPoolForCompressionLevel6.Put(writer) + writer.Reset(&buf) + case 7: + writer = gzipWriterPoolForCompressionLevel7.Get().(*gzip.Writer) + defer gzipWriterPoolForCompressionLevel7.Put(writer) + writer.Reset(&buf) + case 8: + writer = gzipWriterPoolForCompressionLevel8.Get().(*gzip.Writer) + defer gzipWriterPoolForCompressionLevel8.Put(writer) + writer.Reset(&buf) + case 9: + writer = gzipWriterPoolForCompressionLevel9.Get().(*gzip.Writer) + defer gzipWriterPoolForCompressionLevel9.Put(writer) + writer.Reset(&buf) + default: + writer, err = gzip.NewWriterLevel(&buf, level) + if err != nil { + return nil, err + } + } + if _, err := writer.Write(data); err != nil { + return nil, err + } + if err := writer.Close(); err != nil { + return nil, err + } + return buf.Bytes(), nil + case CompressionSnappy: + return snappy.Encode(data), nil + case CompressionLZ4: + writer := lz4WriterPool.Get().(*lz4.Writer) + defer lz4WriterPool.Put(writer) + + var buf bytes.Buffer + writer.Reset(&buf) + + if _, err := writer.Write(data); err != nil { + return nil, err + } + if err := writer.Close(); err != nil { + return nil, err + } + return buf.Bytes(), nil + case CompressionZSTD: + return zstdCompress(nil, data) + default: + return nil, PacketEncodingError{fmt.Sprintf("unsupported compression codec (%d)", cc)} + } +} diff --git a/vendor/github.com/Shopify/sarama/config.go b/vendor/github.com/Shopify/sarama/config.go new file mode 100644 index 000000000000..3f86f1fb9c5d --- /dev/null +++ b/vendor/github.com/Shopify/sarama/config.go @@ -0,0 +1,777 @@ +package sarama + +import ( + "compress/gzip" + "crypto/tls" + "fmt" + "io" + "net" + "regexp" + "time" + + "github.com/rcrowley/go-metrics" + "golang.org/x/net/proxy" +) + +const defaultClientID = "sarama" + +var validID = regexp.MustCompile(`\A[A-Za-z0-9._-]+\z`) + +// Config is used to pass multiple configuration options to Sarama's constructors. +type Config struct { + // Admin is the namespace for ClusterAdmin properties used by the administrative Kafka client. + Admin struct { + Retry struct { + // The total number of times to retry sending (retriable) admin requests (default 5). + // Similar to the `retries` setting of the JVM AdminClientConfig. + Max int + // Backoff time between retries of a failed request (default 100ms) + Backoff time.Duration + } + // The maximum duration the administrative Kafka client will wait for ClusterAdmin operations, + // including topics, brokers, configurations and ACLs (defaults to 3 seconds). + Timeout time.Duration + } + + // Net is the namespace for network-level properties used by the Broker, and + // shared by the Client/Producer/Consumer. + Net struct { + // How many outstanding requests a connection is allowed to have before + // sending on it blocks (default 5). + MaxOpenRequests int + + // All three of the below configurations are similar to the + // `socket.timeout.ms` setting in JVM kafka. All of them default + // to 30 seconds. + DialTimeout time.Duration // How long to wait for the initial connection. + ReadTimeout time.Duration // How long to wait for a response. + WriteTimeout time.Duration // How long to wait for a transmit. + + TLS struct { + // Whether or not to use TLS when connecting to the broker + // (defaults to false). + Enable bool + // The TLS configuration to use for secure connections if + // enabled (defaults to nil). + Config *tls.Config + } + + // SASL based authentication with broker. While there are multiple SASL authentication methods + // the current implementation is limited to plaintext (SASL/PLAIN) authentication + SASL struct { + // Whether or not to use SASL authentication when connecting to the broker + // (defaults to false). + Enable bool + // SASLMechanism is the name of the enabled SASL mechanism. + // Possible values: OAUTHBEARER, PLAIN (defaults to PLAIN). + Mechanism SASLMechanism + // Version is the SASL Protocol Version to use + // Kafka > 1.x should use V1, except on Azure EventHub which use V0 + Version int16 + // Whether or not to send the Kafka SASL handshake first if enabled + // (defaults to true). You should only set this to false if you're using + // a non-Kafka SASL proxy. + Handshake bool + // AuthIdentity is an (optional) authorization identity (authzid) to + // use for SASL/PLAIN authentication (if different from User) when + // an authenticated user is permitted to act as the presented + // alternative user. See RFC4616 for details. + AuthIdentity string + // User is the authentication identity (authcid) to present for + // SASL/PLAIN or SASL/SCRAM authentication + User string + // Password for SASL/PLAIN authentication + Password string + // authz id used for SASL/SCRAM authentication + SCRAMAuthzID string + // SCRAMClientGeneratorFunc is a generator of a user provided implementation of a SCRAM + // client used to perform the SCRAM exchange with the server. + SCRAMClientGeneratorFunc func() SCRAMClient + // TokenProvider is a user-defined callback for generating + // access tokens for SASL/OAUTHBEARER auth. See the + // AccessTokenProvider interface docs for proper implementation + // guidelines. + TokenProvider AccessTokenProvider + + GSSAPI GSSAPIConfig + } + + // KeepAlive specifies the keep-alive period for an active network connection (defaults to 0). + // If zero or positive, keep-alives are enabled. + // If negative, keep-alives are disabled. + KeepAlive time.Duration + + // LocalAddr is the local address to use when dialing an + // address. The address must be of a compatible type for the + // network being dialed. + // If nil, a local address is automatically chosen. + LocalAddr net.Addr + + Proxy struct { + // Whether or not to use proxy when connecting to the broker + // (defaults to false). + Enable bool + // The proxy dialer to use enabled (defaults to nil). + Dialer proxy.Dialer + } + } + + // Metadata is the namespace for metadata management properties used by the + // Client, and shared by the Producer/Consumer. + Metadata struct { + Retry struct { + // The total number of times to retry a metadata request when the + // cluster is in the middle of a leader election (default 3). + Max int + // How long to wait for leader election to occur before retrying + // (default 250ms). Similar to the JVM's `retry.backoff.ms`. + Backoff time.Duration + // Called to compute backoff time dynamically. Useful for implementing + // more sophisticated backoff strategies. This takes precedence over + // `Backoff` if set. + BackoffFunc func(retries, maxRetries int) time.Duration + } + // How frequently to refresh the cluster metadata in the background. + // Defaults to 10 minutes. Set to 0 to disable. Similar to + // `topic.metadata.refresh.interval.ms` in the JVM version. + RefreshFrequency time.Duration + + // Whether to maintain a full set of metadata for all topics, or just + // the minimal set that has been necessary so far. The full set is simpler + // and usually more convenient, but can take up a substantial amount of + // memory if you have many topics and partitions. Defaults to true. + Full bool + + // How long to wait for a successful metadata response. + // Disabled by default which means a metadata request against an unreachable + // cluster (all brokers are unreachable or unresponsive) can take up to + // `Net.[Dial|Read]Timeout * BrokerCount * (Metadata.Retry.Max + 1) + Metadata.Retry.Backoff * Metadata.Retry.Max` + // to fail. + Timeout time.Duration + + // Whether to allow auto-create topics in metadata refresh. If set to true, + // the broker may auto-create topics that we requested which do not already exist, + // if it is configured to do so (`auto.create.topics.enable` is true). Defaults to true. + AllowAutoTopicCreation bool + } + + // Producer is the namespace for configuration related to producing messages, + // used by the Producer. + Producer struct { + // The maximum permitted size of a message (defaults to 1000000). Should be + // set equal to or smaller than the broker's `message.max.bytes`. + MaxMessageBytes int + // The level of acknowledgement reliability needed from the broker (defaults + // to WaitForLocal). Equivalent to the `request.required.acks` setting of the + // JVM producer. + RequiredAcks RequiredAcks + // The maximum duration the broker will wait the receipt of the number of + // RequiredAcks (defaults to 10 seconds). This is only relevant when + // RequiredAcks is set to WaitForAll or a number > 1. Only supports + // millisecond resolution, nanoseconds will be truncated. Equivalent to + // the JVM producer's `request.timeout.ms` setting. + Timeout time.Duration + // The type of compression to use on messages (defaults to no compression). + // Similar to `compression.codec` setting of the JVM producer. + Compression CompressionCodec + // The level of compression to use on messages. The meaning depends + // on the actual compression type used and defaults to default compression + // level for the codec. + CompressionLevel int + // Generates partitioners for choosing the partition to send messages to + // (defaults to hashing the message key). Similar to the `partitioner.class` + // setting for the JVM producer. + Partitioner PartitionerConstructor + // If enabled, the producer will ensure that exactly one copy of each message is + // written. + Idempotent bool + + // Return specifies what channels will be populated. If they are set to true, + // you must read from the respective channels to prevent deadlock. If, + // however, this config is used to create a `SyncProducer`, both must be set + // to true and you shall not read from the channels since the producer does + // this internally. + Return struct { + // If enabled, successfully delivered messages will be returned on the + // Successes channel (default disabled). + Successes bool + + // If enabled, messages that failed to deliver will be returned on the + // Errors channel, including error (default enabled). + Errors bool + } + + // The following config options control how often messages are batched up and + // sent to the broker. By default, messages are sent as fast as possible, and + // all messages received while the current batch is in-flight are placed + // into the subsequent batch. + Flush struct { + // The best-effort number of bytes needed to trigger a flush. Use the + // global sarama.MaxRequestSize to set a hard upper limit. + Bytes int + // The best-effort number of messages needed to trigger a flush. Use + // `MaxMessages` to set a hard upper limit. + Messages int + // The best-effort frequency of flushes. Equivalent to + // `queue.buffering.max.ms` setting of JVM producer. + Frequency time.Duration + // The maximum number of messages the producer will send in a single + // broker request. Defaults to 0 for unlimited. Similar to + // `queue.buffering.max.messages` in the JVM producer. + MaxMessages int + } + + Retry struct { + // The total number of times to retry sending a message (default 3). + // Similar to the `message.send.max.retries` setting of the JVM producer. + Max int + // How long to wait for the cluster to settle between retries + // (default 100ms). Similar to the `retry.backoff.ms` setting of the + // JVM producer. + Backoff time.Duration + // Called to compute backoff time dynamically. Useful for implementing + // more sophisticated backoff strategies. This takes precedence over + // `Backoff` if set. + BackoffFunc func(retries, maxRetries int) time.Duration + } + + // Interceptors to be called when the producer dispatcher reads the + // message for the first time. Interceptors allows to intercept and + // possible mutate the message before they are published to Kafka + // cluster. *ProducerMessage modified by the first interceptor's + // OnSend() is passed to the second interceptor OnSend(), and so on in + // the interceptor chain. + Interceptors []ProducerInterceptor + } + + // Consumer is the namespace for configuration related to consuming messages, + // used by the Consumer. + Consumer struct { + + // Group is the namespace for configuring consumer group. + Group struct { + Session struct { + // The timeout used to detect consumer failures when using Kafka's group management facility. + // The consumer sends periodic heartbeats to indicate its liveness to the broker. + // If no heartbeats are received by the broker before the expiration of this session timeout, + // then the broker will remove this consumer from the group and initiate a rebalance. + // Note that the value must be in the allowable range as configured in the broker configuration + // by `group.min.session.timeout.ms` and `group.max.session.timeout.ms` (default 10s) + Timeout time.Duration + } + Heartbeat struct { + // The expected time between heartbeats to the consumer coordinator when using Kafka's group + // management facilities. Heartbeats are used to ensure that the consumer's session stays active and + // to facilitate rebalancing when new consumers join or leave the group. + // The value must be set lower than Consumer.Group.Session.Timeout, but typically should be set no + // higher than 1/3 of that value. + // It can be adjusted even lower to control the expected time for normal rebalances (default 3s) + Interval time.Duration + } + Rebalance struct { + // Strategy for allocating topic partitions to members (default BalanceStrategyRange) + Strategy BalanceStrategy + // The maximum allowed time for each worker to join the group once a rebalance has begun. + // This is basically a limit on the amount of time needed for all tasks to flush any pending + // data and commit offsets. If the timeout is exceeded, then the worker will be removed from + // the group, which will cause offset commit failures (default 60s). + Timeout time.Duration + + Retry struct { + // When a new consumer joins a consumer group the set of consumers attempt to "rebalance" + // the load to assign partitions to each consumer. If the set of consumers changes while + // this assignment is taking place the rebalance will fail and retry. This setting controls + // the maximum number of attempts before giving up (default 4). + Max int + // Backoff time between retries during rebalance (default 2s) + Backoff time.Duration + } + } + Member struct { + // Custom metadata to include when joining the group. The user data for all joined members + // can be retrieved by sending a DescribeGroupRequest to the broker that is the + // coordinator for the group. + UserData []byte + } + } + + Retry struct { + // How long to wait after a failing to read from a partition before + // trying again (default 2s). + Backoff time.Duration + // Called to compute backoff time dynamically. Useful for implementing + // more sophisticated backoff strategies. This takes precedence over + // `Backoff` if set. + BackoffFunc func(retries int) time.Duration + } + + // Fetch is the namespace for controlling how many bytes are retrieved by any + // given request. + Fetch struct { + // The minimum number of message bytes to fetch in a request - the broker + // will wait until at least this many are available. The default is 1, + // as 0 causes the consumer to spin when no messages are available. + // Equivalent to the JVM's `fetch.min.bytes`. + Min int32 + // The default number of message bytes to fetch from the broker in each + // request (default 1MB). This should be larger than the majority of + // your messages, or else the consumer will spend a lot of time + // negotiating sizes and not actually consuming. Similar to the JVM's + // `fetch.message.max.bytes`. + Default int32 + // The maximum number of message bytes to fetch from the broker in a + // single request. Messages larger than this will return + // ErrMessageTooLarge and will not be consumable, so you must be sure + // this is at least as large as your largest message. Defaults to 0 + // (no limit). Similar to the JVM's `fetch.message.max.bytes`. The + // global `sarama.MaxResponseSize` still applies. + Max int32 + } + // The maximum amount of time the broker will wait for Consumer.Fetch.Min + // bytes to become available before it returns fewer than that anyways. The + // default is 250ms, since 0 causes the consumer to spin when no events are + // available. 100-500ms is a reasonable range for most cases. Kafka only + // supports precision up to milliseconds; nanoseconds will be truncated. + // Equivalent to the JVM's `fetch.wait.max.ms`. + MaxWaitTime time.Duration + + // The maximum amount of time the consumer expects a message takes to + // process for the user. If writing to the Messages channel takes longer + // than this, that partition will stop fetching more messages until it + // can proceed again. + // Note that, since the Messages channel is buffered, the actual grace time is + // (MaxProcessingTime * ChannelBufferSize). Defaults to 100ms. + // If a message is not written to the Messages channel between two ticks + // of the expiryTicker then a timeout is detected. + // Using a ticker instead of a timer to detect timeouts should typically + // result in many fewer calls to Timer functions which may result in a + // significant performance improvement if many messages are being sent + // and timeouts are infrequent. + // The disadvantage of using a ticker instead of a timer is that + // timeouts will be less accurate. That is, the effective timeout could + // be between `MaxProcessingTime` and `2 * MaxProcessingTime`. For + // example, if `MaxProcessingTime` is 100ms then a delay of 180ms + // between two messages being sent may not be recognized as a timeout. + MaxProcessingTime time.Duration + + // Return specifies what channels will be populated. If they are set to true, + // you must read from them to prevent deadlock. + Return struct { + // If enabled, any errors that occurred while consuming are returned on + // the Errors channel (default disabled). + Errors bool + } + + // Offsets specifies configuration for how and when to commit consumed + // offsets. This currently requires the manual use of an OffsetManager + // but will eventually be automated. + Offsets struct { + // Deprecated: CommitInterval exists for historical compatibility + // and should not be used. Please use Consumer.Offsets.AutoCommit + CommitInterval time.Duration + + // AutoCommit specifies configuration for commit messages automatically. + AutoCommit struct { + // Whether or not to auto-commit updated offsets back to the broker. + // (default enabled). + Enable bool + + // How frequently to commit updated offsets. Ineffective unless + // auto-commit is enabled (default 1s) + Interval time.Duration + } + + // The initial offset to use if no offset was previously committed. + // Should be OffsetNewest or OffsetOldest. Defaults to OffsetNewest. + Initial int64 + + // The retention duration for committed offsets. If zero, disabled + // (in which case the `offsets.retention.minutes` option on the + // broker will be used). Kafka only supports precision up to + // milliseconds; nanoseconds will be truncated. Requires Kafka + // broker version 0.9.0 or later. + // (default is 0: disabled). + Retention time.Duration + + Retry struct { + // The total number of times to retry failing commit + // requests during OffsetManager shutdown (default 3). + Max int + } + } + + // IsolationLevel support 2 mode: + // - use `ReadUncommitted` (default) to consume and return all messages in message channel + // - use `ReadCommitted` to hide messages that are part of an aborted transaction + IsolationLevel IsolationLevel + + // Interceptors to be called just before the record is sent to the + // messages channel. Interceptors allows to intercept and possible + // mutate the message before they are returned to the client. + // *ConsumerMessage modified by the first interceptor's OnConsume() is + // passed to the second interceptor OnConsume(), and so on in the + // interceptor chain. + Interceptors []ConsumerInterceptor + } + + // A user-provided string sent with every request to the brokers for logging, + // debugging, and auditing purposes. Defaults to "sarama", but you should + // probably set it to something specific to your application. + ClientID string + // A rack identifier for this client. This can be any string value which + // indicates where this client is physically located. + // It corresponds with the broker config 'broker.rack' + RackID string + // The number of events to buffer in internal and external channels. This + // permits the producer and consumer to continue processing some messages + // in the background while user code is working, greatly improving throughput. + // Defaults to 256. + ChannelBufferSize int + // ApiVersionsRequest determines whether Sarama should send an + // ApiVersionsRequest message to each broker as part of its initial + // connection. This defaults to `true` to match the official Java client + // and most 3rdparty ones. + ApiVersionsRequest bool + // The version of Kafka that Sarama will assume it is running against. + // Defaults to the oldest supported stable version. Since Kafka provides + // backwards-compatibility, setting it to a version older than you have + // will not break anything, although it may prevent you from using the + // latest features. Setting it to a version greater than you are actually + // running may lead to random breakage. + Version KafkaVersion + // The registry to define metrics into. + // Defaults to a local registry. + // If you want to disable metrics gathering, set "metrics.UseNilMetrics" to "true" + // prior to starting Sarama. + // See Examples on how to use the metrics registry + MetricRegistry metrics.Registry +} + +// NewConfig returns a new configuration instance with sane defaults. +func NewConfig() *Config { + c := &Config{} + + c.Admin.Retry.Max = 5 + c.Admin.Retry.Backoff = 100 * time.Millisecond + c.Admin.Timeout = 3 * time.Second + + c.Net.MaxOpenRequests = 5 + c.Net.DialTimeout = 30 * time.Second + c.Net.ReadTimeout = 30 * time.Second + c.Net.WriteTimeout = 30 * time.Second + c.Net.SASL.Handshake = true + c.Net.SASL.Version = SASLHandshakeV0 + + c.Metadata.Retry.Max = 3 + c.Metadata.Retry.Backoff = 250 * time.Millisecond + c.Metadata.RefreshFrequency = 10 * time.Minute + c.Metadata.Full = true + c.Metadata.AllowAutoTopicCreation = true + + c.Producer.MaxMessageBytes = 1000000 + c.Producer.RequiredAcks = WaitForLocal + c.Producer.Timeout = 10 * time.Second + c.Producer.Partitioner = NewHashPartitioner + c.Producer.Retry.Max = 3 + c.Producer.Retry.Backoff = 100 * time.Millisecond + c.Producer.Return.Errors = true + c.Producer.CompressionLevel = CompressionLevelDefault + + c.Consumer.Fetch.Min = 1 + c.Consumer.Fetch.Default = 1024 * 1024 + c.Consumer.Retry.Backoff = 2 * time.Second + c.Consumer.MaxWaitTime = 250 * time.Millisecond + c.Consumer.MaxProcessingTime = 100 * time.Millisecond + c.Consumer.Return.Errors = false + c.Consumer.Offsets.AutoCommit.Enable = true + c.Consumer.Offsets.AutoCommit.Interval = 1 * time.Second + c.Consumer.Offsets.Initial = OffsetNewest + c.Consumer.Offsets.Retry.Max = 3 + + c.Consumer.Group.Session.Timeout = 10 * time.Second + c.Consumer.Group.Heartbeat.Interval = 3 * time.Second + c.Consumer.Group.Rebalance.Strategy = BalanceStrategyRange + c.Consumer.Group.Rebalance.Timeout = 60 * time.Second + c.Consumer.Group.Rebalance.Retry.Max = 4 + c.Consumer.Group.Rebalance.Retry.Backoff = 2 * time.Second + + c.ClientID = defaultClientID + c.ChannelBufferSize = 256 + c.ApiVersionsRequest = true + c.Version = DefaultVersion + c.MetricRegistry = metrics.NewRegistry() + + return c +} + +// Validate checks a Config instance. It will return a +// ConfigurationError if the specified values don't make sense. +func (c *Config) Validate() error { + // some configuration values should be warned on but not fail completely, do those first + if !c.Net.TLS.Enable && c.Net.TLS.Config != nil { + Logger.Println("Net.TLS is disabled but a non-nil configuration was provided.") + } + if !c.Net.SASL.Enable { + if c.Net.SASL.User != "" { + Logger.Println("Net.SASL is disabled but a non-empty username was provided.") + } + if c.Net.SASL.Password != "" { + Logger.Println("Net.SASL is disabled but a non-empty password was provided.") + } + } + if c.Producer.RequiredAcks > 1 { + Logger.Println("Producer.RequiredAcks > 1 is deprecated and will raise an exception with kafka >= 0.8.2.0.") + } + if c.Producer.MaxMessageBytes >= int(MaxRequestSize) { + Logger.Println("Producer.MaxMessageBytes must be smaller than MaxRequestSize; it will be ignored.") + } + if c.Producer.Flush.Bytes >= int(MaxRequestSize) { + Logger.Println("Producer.Flush.Bytes must be smaller than MaxRequestSize; it will be ignored.") + } + if (c.Producer.Flush.Bytes > 0 || c.Producer.Flush.Messages > 0) && c.Producer.Flush.Frequency == 0 { + Logger.Println("Producer.Flush: Bytes or Messages are set, but Frequency is not; messages may not get flushed.") + } + if c.Producer.Timeout%time.Millisecond != 0 { + Logger.Println("Producer.Timeout only supports millisecond resolution; nanoseconds will be truncated.") + } + if c.Consumer.MaxWaitTime < 100*time.Millisecond { + Logger.Println("Consumer.MaxWaitTime is very low, which can cause high CPU and network usage. See documentation for details.") + } + if c.Consumer.MaxWaitTime%time.Millisecond != 0 { + Logger.Println("Consumer.MaxWaitTime only supports millisecond precision; nanoseconds will be truncated.") + } + if c.Consumer.Offsets.Retention%time.Millisecond != 0 { + Logger.Println("Consumer.Offsets.Retention only supports millisecond precision; nanoseconds will be truncated.") + } + if c.Consumer.Group.Session.Timeout%time.Millisecond != 0 { + Logger.Println("Consumer.Group.Session.Timeout only supports millisecond precision; nanoseconds will be truncated.") + } + if c.Consumer.Group.Heartbeat.Interval%time.Millisecond != 0 { + Logger.Println("Consumer.Group.Heartbeat.Interval only supports millisecond precision; nanoseconds will be truncated.") + } + if c.Consumer.Group.Rebalance.Timeout%time.Millisecond != 0 { + Logger.Println("Consumer.Group.Rebalance.Timeout only supports millisecond precision; nanoseconds will be truncated.") + } + if c.ClientID == defaultClientID { + Logger.Println("ClientID is the default of 'sarama', you should consider setting it to something application-specific.") + } + + // validate Net values + switch { + case c.Net.MaxOpenRequests <= 0: + return ConfigurationError("Net.MaxOpenRequests must be > 0") + case c.Net.DialTimeout <= 0: + return ConfigurationError("Net.DialTimeout must be > 0") + case c.Net.ReadTimeout <= 0: + return ConfigurationError("Net.ReadTimeout must be > 0") + case c.Net.WriteTimeout <= 0: + return ConfigurationError("Net.WriteTimeout must be > 0") + case c.Net.SASL.Enable: + if c.Net.SASL.Mechanism == "" { + c.Net.SASL.Mechanism = SASLTypePlaintext + } + + switch c.Net.SASL.Mechanism { + case SASLTypePlaintext: + if c.Net.SASL.User == "" { + return ConfigurationError("Net.SASL.User must not be empty when SASL is enabled") + } + if c.Net.SASL.Password == "" { + return ConfigurationError("Net.SASL.Password must not be empty when SASL is enabled") + } + case SASLTypeOAuth: + if c.Net.SASL.TokenProvider == nil { + return ConfigurationError("An AccessTokenProvider instance must be provided to Net.SASL.TokenProvider") + } + case SASLTypeSCRAMSHA256, SASLTypeSCRAMSHA512: + if c.Net.SASL.User == "" { + return ConfigurationError("Net.SASL.User must not be empty when SASL is enabled") + } + if c.Net.SASL.Password == "" { + return ConfigurationError("Net.SASL.Password must not be empty when SASL is enabled") + } + if c.Net.SASL.SCRAMClientGeneratorFunc == nil { + return ConfigurationError("A SCRAMClientGeneratorFunc function must be provided to Net.SASL.SCRAMClientGeneratorFunc") + } + case SASLTypeGSSAPI: + if c.Net.SASL.GSSAPI.ServiceName == "" { + return ConfigurationError("Net.SASL.GSSAPI.ServiceName must not be empty when GSS-API mechanism is used") + } + + if c.Net.SASL.GSSAPI.AuthType == KRB5_USER_AUTH { + if c.Net.SASL.GSSAPI.Password == "" { + return ConfigurationError("Net.SASL.GSSAPI.Password must not be empty when GSS-API " + + "mechanism is used and Net.SASL.GSSAPI.AuthType = KRB5_USER_AUTH") + } + } else if c.Net.SASL.GSSAPI.AuthType == KRB5_KEYTAB_AUTH { + if c.Net.SASL.GSSAPI.KeyTabPath == "" { + return ConfigurationError("Net.SASL.GSSAPI.KeyTabPath must not be empty when GSS-API mechanism is used" + + " and Net.SASL.GSSAPI.AuthType = KRB5_KEYTAB_AUTH") + } + } else { + return ConfigurationError("Net.SASL.GSSAPI.AuthType is invalid. Possible values are KRB5_USER_AUTH and KRB5_KEYTAB_AUTH") + } + if c.Net.SASL.GSSAPI.KerberosConfigPath == "" { + return ConfigurationError("Net.SASL.GSSAPI.KerberosConfigPath must not be empty when GSS-API mechanism is used") + } + if c.Net.SASL.GSSAPI.Username == "" { + return ConfigurationError("Net.SASL.GSSAPI.Username must not be empty when GSS-API mechanism is used") + } + if c.Net.SASL.GSSAPI.Realm == "" { + return ConfigurationError("Net.SASL.GSSAPI.Realm must not be empty when GSS-API mechanism is used") + } + default: + msg := fmt.Sprintf("The SASL mechanism configuration is invalid. Possible values are `%s`, `%s`, `%s`, `%s` and `%s`", + SASLTypeOAuth, SASLTypePlaintext, SASLTypeSCRAMSHA256, SASLTypeSCRAMSHA512, SASLTypeGSSAPI) + return ConfigurationError(msg) + } + } + + // validate the Admin values + switch { + case c.Admin.Timeout <= 0: + return ConfigurationError("Admin.Timeout must be > 0") + } + + // validate the Metadata values + switch { + case c.Metadata.Retry.Max < 0: + return ConfigurationError("Metadata.Retry.Max must be >= 0") + case c.Metadata.Retry.Backoff < 0: + return ConfigurationError("Metadata.Retry.Backoff must be >= 0") + case c.Metadata.RefreshFrequency < 0: + return ConfigurationError("Metadata.RefreshFrequency must be >= 0") + } + + // validate the Producer values + switch { + case c.Producer.MaxMessageBytes <= 0: + return ConfigurationError("Producer.MaxMessageBytes must be > 0") + case c.Producer.RequiredAcks < -1: + return ConfigurationError("Producer.RequiredAcks must be >= -1") + case c.Producer.Timeout <= 0: + return ConfigurationError("Producer.Timeout must be > 0") + case c.Producer.Partitioner == nil: + return ConfigurationError("Producer.Partitioner must not be nil") + case c.Producer.Flush.Bytes < 0: + return ConfigurationError("Producer.Flush.Bytes must be >= 0") + case c.Producer.Flush.Messages < 0: + return ConfigurationError("Producer.Flush.Messages must be >= 0") + case c.Producer.Flush.Frequency < 0: + return ConfigurationError("Producer.Flush.Frequency must be >= 0") + case c.Producer.Flush.MaxMessages < 0: + return ConfigurationError("Producer.Flush.MaxMessages must be >= 0") + case c.Producer.Flush.MaxMessages > 0 && c.Producer.Flush.MaxMessages < c.Producer.Flush.Messages: + return ConfigurationError("Producer.Flush.MaxMessages must be >= Producer.Flush.Messages when set") + case c.Producer.Retry.Max < 0: + return ConfigurationError("Producer.Retry.Max must be >= 0") + case c.Producer.Retry.Backoff < 0: + return ConfigurationError("Producer.Retry.Backoff must be >= 0") + } + + if c.Producer.Compression == CompressionLZ4 && !c.Version.IsAtLeast(V0_10_0_0) { + return ConfigurationError("lz4 compression requires Version >= V0_10_0_0") + } + + if c.Producer.Compression == CompressionGZIP { + if c.Producer.CompressionLevel != CompressionLevelDefault { + if _, err := gzip.NewWriterLevel(io.Discard, c.Producer.CompressionLevel); err != nil { + return ConfigurationError(fmt.Sprintf("gzip compression does not work with level %d: %v", c.Producer.CompressionLevel, err)) + } + } + } + + if c.Producer.Compression == CompressionZSTD && !c.Version.IsAtLeast(V2_1_0_0) { + return ConfigurationError("zstd compression requires Version >= V2_1_0_0") + } + + if c.Producer.Idempotent { + if !c.Version.IsAtLeast(V0_11_0_0) { + return ConfigurationError("Idempotent producer requires Version >= V0_11_0_0") + } + if c.Producer.Retry.Max == 0 { + return ConfigurationError("Idempotent producer requires Producer.Retry.Max >= 1") + } + if c.Producer.RequiredAcks != WaitForAll { + return ConfigurationError("Idempotent producer requires Producer.RequiredAcks to be WaitForAll") + } + if c.Net.MaxOpenRequests > 1 { + return ConfigurationError("Idempotent producer requires Net.MaxOpenRequests to be 1") + } + } + + // validate the Consumer values + switch { + case c.Consumer.Fetch.Min <= 0: + return ConfigurationError("Consumer.Fetch.Min must be > 0") + case c.Consumer.Fetch.Default <= 0: + return ConfigurationError("Consumer.Fetch.Default must be > 0") + case c.Consumer.Fetch.Max < 0: + return ConfigurationError("Consumer.Fetch.Max must be >= 0") + case c.Consumer.MaxWaitTime < 1*time.Millisecond: + return ConfigurationError("Consumer.MaxWaitTime must be >= 1ms") + case c.Consumer.MaxProcessingTime <= 0: + return ConfigurationError("Consumer.MaxProcessingTime must be > 0") + case c.Consumer.Retry.Backoff < 0: + return ConfigurationError("Consumer.Retry.Backoff must be >= 0") + case c.Consumer.Offsets.AutoCommit.Interval <= 0: + return ConfigurationError("Consumer.Offsets.AutoCommit.Interval must be > 0") + case c.Consumer.Offsets.Initial != OffsetOldest && c.Consumer.Offsets.Initial != OffsetNewest: + return ConfigurationError("Consumer.Offsets.Initial must be OffsetOldest or OffsetNewest") + case c.Consumer.Offsets.Retry.Max < 0: + return ConfigurationError("Consumer.Offsets.Retry.Max must be >= 0") + case c.Consumer.IsolationLevel != ReadUncommitted && c.Consumer.IsolationLevel != ReadCommitted: + return ConfigurationError("Consumer.IsolationLevel must be ReadUncommitted or ReadCommitted") + } + + if c.Consumer.Offsets.CommitInterval != 0 { + Logger.Println("Deprecation warning: Consumer.Offsets.CommitInterval exists for historical compatibility" + + " and should not be used. Please use Consumer.Offsets.AutoCommit, the current value will be ignored") + } + + // validate IsolationLevel + if c.Consumer.IsolationLevel == ReadCommitted && !c.Version.IsAtLeast(V0_11_0_0) { + return ConfigurationError("ReadCommitted requires Version >= V0_11_0_0") + } + + // validate the Consumer Group values + switch { + case c.Consumer.Group.Session.Timeout <= 2*time.Millisecond: + return ConfigurationError("Consumer.Group.Session.Timeout must be >= 2ms") + case c.Consumer.Group.Heartbeat.Interval < 1*time.Millisecond: + return ConfigurationError("Consumer.Group.Heartbeat.Interval must be >= 1ms") + case c.Consumer.Group.Heartbeat.Interval >= c.Consumer.Group.Session.Timeout: + return ConfigurationError("Consumer.Group.Heartbeat.Interval must be < Consumer.Group.Session.Timeout") + case c.Consumer.Group.Rebalance.Strategy == nil: + return ConfigurationError("Consumer.Group.Rebalance.Strategy must not be empty") + case c.Consumer.Group.Rebalance.Timeout <= time.Millisecond: + return ConfigurationError("Consumer.Group.Rebalance.Timeout must be >= 1ms") + case c.Consumer.Group.Rebalance.Retry.Max < 0: + return ConfigurationError("Consumer.Group.Rebalance.Retry.Max must be >= 0") + case c.Consumer.Group.Rebalance.Retry.Backoff < 0: + return ConfigurationError("Consumer.Group.Rebalance.Retry.Backoff must be >= 0") + } + + // validate misc shared values + switch { + case c.ChannelBufferSize < 0: + return ConfigurationError("ChannelBufferSize must be >= 0") + case !validID.MatchString(c.ClientID): + return ConfigurationError("ClientID is invalid") + } + + return nil +} + +func (c *Config) getDialer() proxy.Dialer { + if c.Net.Proxy.Enable { + Logger.Printf("using proxy %s", c.Net.Proxy.Dialer) + return c.Net.Proxy.Dialer + } else { + return &net.Dialer{ + Timeout: c.Net.DialTimeout, + KeepAlive: c.Net.KeepAlive, + LocalAddr: c.Net.LocalAddr, + } + } +} diff --git a/vendor/github.com/Shopify/sarama/config_resource_type.go b/vendor/github.com/Shopify/sarama/config_resource_type.go new file mode 100644 index 000000000000..bef1053aaeda --- /dev/null +++ b/vendor/github.com/Shopify/sarama/config_resource_type.go @@ -0,0 +1,18 @@ +package sarama + +// ConfigResourceType is a type for resources that have configs. +type ConfigResourceType int8 + +// Taken from: +// https://github.com/apache/kafka/blob/ed7c071e07f1f90e4c2895582f61ca090ced3c42/clients/src/main/java/org/apache/kafka/common/config/ConfigResource.java#L32-L55 + +const ( + // UnknownResource constant type + UnknownResource ConfigResourceType = 0 + // TopicResource constant type + TopicResource ConfigResourceType = 2 + // BrokerResource constant type + BrokerResource ConfigResourceType = 4 + // BrokerLoggerResource constant type + BrokerLoggerResource ConfigResourceType = 8 +) diff --git a/vendor/github.com/Shopify/sarama/consumer.go b/vendor/github.com/Shopify/sarama/consumer.go new file mode 100644 index 000000000000..2d76e8a1e8ee --- /dev/null +++ b/vendor/github.com/Shopify/sarama/consumer.go @@ -0,0 +1,949 @@ +package sarama + +import ( + "errors" + "fmt" + "math" + "sync" + "sync/atomic" + "time" + + "github.com/rcrowley/go-metrics" +) + +// ConsumerMessage encapsulates a Kafka message returned by the consumer. +type ConsumerMessage struct { + Headers []*RecordHeader // only set if kafka is version 0.11+ + Timestamp time.Time // only set if kafka is version 0.10+, inner message timestamp + BlockTimestamp time.Time // only set if kafka is version 0.10+, outer (compressed) block timestamp + + Key, Value []byte + Topic string + Partition int32 + Offset int64 +} + +// ConsumerError is what is provided to the user when an error occurs. +// It wraps an error and includes the topic and partition. +type ConsumerError struct { + Topic string + Partition int32 + Err error +} + +func (ce ConsumerError) Error() string { + return fmt.Sprintf("kafka: error while consuming %s/%d: %s", ce.Topic, ce.Partition, ce.Err) +} + +func (ce ConsumerError) Unwrap() error { + return ce.Err +} + +// ConsumerErrors is a type that wraps a batch of errors and implements the Error interface. +// It can be returned from the PartitionConsumer's Close methods to avoid the need to manually drain errors +// when stopping. +type ConsumerErrors []*ConsumerError + +func (ce ConsumerErrors) Error() string { + return fmt.Sprintf("kafka: %d errors while consuming", len(ce)) +} + +// Consumer manages PartitionConsumers which process Kafka messages from brokers. You MUST call Close() +// on a consumer to avoid leaks, it will not be garbage-collected automatically when it passes out of +// scope. +type Consumer interface { + // Topics returns the set of available topics as retrieved from the cluster + // metadata. This method is the same as Client.Topics(), and is provided for + // convenience. + Topics() ([]string, error) + + // Partitions returns the sorted list of all partition IDs for the given topic. + // This method is the same as Client.Partitions(), and is provided for convenience. + Partitions(topic string) ([]int32, error) + + // ConsumePartition creates a PartitionConsumer on the given topic/partition with + // the given offset. It will return an error if this Consumer is already consuming + // on the given topic/partition. Offset can be a literal offset, or OffsetNewest + // or OffsetOldest + ConsumePartition(topic string, partition int32, offset int64) (PartitionConsumer, error) + + // HighWaterMarks returns the current high water marks for each topic and partition. + // Consistency between partitions is not guaranteed since high water marks are updated separately. + HighWaterMarks() map[string]map[int32]int64 + + // Close shuts down the consumer. It must be called after all child + // PartitionConsumers have already been closed. + Close() error +} + +type consumer struct { + conf *Config + children map[string]map[int32]*partitionConsumer + brokerConsumers map[*Broker]*brokerConsumer + client Client + lock sync.Mutex +} + +// NewConsumer creates a new consumer using the given broker addresses and configuration. +func NewConsumer(addrs []string, config *Config) (Consumer, error) { + client, err := NewClient(addrs, config) + if err != nil { + return nil, err + } + return newConsumer(client) +} + +// NewConsumerFromClient creates a new consumer using the given client. It is still +// necessary to call Close() on the underlying client when shutting down this consumer. +func NewConsumerFromClient(client Client) (Consumer, error) { + // For clients passed in by the client, ensure we don't + // call Close() on it. + cli := &nopCloserClient{client} + return newConsumer(cli) +} + +func newConsumer(client Client) (Consumer, error) { + // Check that we are not dealing with a closed Client before processing any other arguments + if client.Closed() { + return nil, ErrClosedClient + } + + c := &consumer{ + client: client, + conf: client.Config(), + children: make(map[string]map[int32]*partitionConsumer), + brokerConsumers: make(map[*Broker]*brokerConsumer), + } + + return c, nil +} + +func (c *consumer) Close() error { + return c.client.Close() +} + +func (c *consumer) Topics() ([]string, error) { + return c.client.Topics() +} + +func (c *consumer) Partitions(topic string) ([]int32, error) { + return c.client.Partitions(topic) +} + +func (c *consumer) ConsumePartition(topic string, partition int32, offset int64) (PartitionConsumer, error) { + child := &partitionConsumer{ + consumer: c, + conf: c.conf, + topic: topic, + partition: partition, + messages: make(chan *ConsumerMessage, c.conf.ChannelBufferSize), + errors: make(chan *ConsumerError, c.conf.ChannelBufferSize), + feeder: make(chan *FetchResponse, 1), + trigger: make(chan none, 1), + dying: make(chan none), + fetchSize: c.conf.Consumer.Fetch.Default, + } + + if err := child.chooseStartingOffset(offset); err != nil { + return nil, err + } + + var leader *Broker + var err error + if leader, err = c.client.Leader(child.topic, child.partition); err != nil { + return nil, err + } + + if err := c.addChild(child); err != nil { + return nil, err + } + + go withRecover(child.dispatcher) + go withRecover(child.responseFeeder) + + child.broker = c.refBrokerConsumer(leader) + child.broker.input <- child + + return child, nil +} + +func (c *consumer) HighWaterMarks() map[string]map[int32]int64 { + c.lock.Lock() + defer c.lock.Unlock() + + hwms := make(map[string]map[int32]int64) + for topic, p := range c.children { + hwm := make(map[int32]int64, len(p)) + for partition, pc := range p { + hwm[partition] = pc.HighWaterMarkOffset() + } + hwms[topic] = hwm + } + + return hwms +} + +func (c *consumer) addChild(child *partitionConsumer) error { + c.lock.Lock() + defer c.lock.Unlock() + + topicChildren := c.children[child.topic] + if topicChildren == nil { + topicChildren = make(map[int32]*partitionConsumer) + c.children[child.topic] = topicChildren + } + + if topicChildren[child.partition] != nil { + return ConfigurationError("That topic/partition is already being consumed") + } + + topicChildren[child.partition] = child + return nil +} + +func (c *consumer) removeChild(child *partitionConsumer) { + c.lock.Lock() + defer c.lock.Unlock() + + delete(c.children[child.topic], child.partition) +} + +func (c *consumer) refBrokerConsumer(broker *Broker) *brokerConsumer { + c.lock.Lock() + defer c.lock.Unlock() + + bc := c.brokerConsumers[broker] + if bc == nil { + bc = c.newBrokerConsumer(broker) + c.brokerConsumers[broker] = bc + } + + bc.refs++ + + return bc +} + +func (c *consumer) unrefBrokerConsumer(brokerWorker *brokerConsumer) { + c.lock.Lock() + defer c.lock.Unlock() + + brokerWorker.refs-- + + if brokerWorker.refs == 0 { + close(brokerWorker.input) + if c.brokerConsumers[brokerWorker.broker] == brokerWorker { + delete(c.brokerConsumers, brokerWorker.broker) + } + } +} + +func (c *consumer) abandonBrokerConsumer(brokerWorker *brokerConsumer) { + c.lock.Lock() + defer c.lock.Unlock() + + delete(c.brokerConsumers, brokerWorker.broker) +} + +// PartitionConsumer + +// PartitionConsumer processes Kafka messages from a given topic and partition. You MUST call one of Close() or +// AsyncClose() on a PartitionConsumer to avoid leaks; it will not be garbage-collected automatically when it passes out +// of scope. +// +// The simplest way of using a PartitionConsumer is to loop over its Messages channel using a for/range +// loop. The PartitionConsumer will only stop itself in one case: when the offset being consumed is reported +// as out of range by the brokers. In this case you should decide what you want to do (try a different offset, +// notify a human, etc) and handle it appropriately. For all other error cases, it will just keep retrying. +// By default, it logs these errors to sarama.Logger; if you want to be notified directly of all errors, set +// your config's Consumer.Return.Errors to true and read from the Errors channel, using a select statement +// or a separate goroutine. Check out the Consumer examples to see implementations of these different approaches. +// +// To terminate such a for/range loop while the loop is executing, call AsyncClose. This will kick off the process of +// consumer tear-down & return immediately. Continue to loop, servicing the Messages channel until the teardown process +// AsyncClose initiated closes it (thus terminating the for/range loop). If you've already ceased reading Messages, call +// Close; this will signal the PartitionConsumer's goroutines to begin shutting down (just like AsyncClose), but will +// also drain the Messages channel, harvest all errors & return them once cleanup has completed. +type PartitionConsumer interface { + // AsyncClose initiates a shutdown of the PartitionConsumer. This method will return immediately, after which you + // should continue to service the 'Messages' and 'Errors' channels until they are empty. It is required to call this + // function, or Close before a consumer object passes out of scope, as it will otherwise leak memory. You must call + // this before calling Close on the underlying client. + AsyncClose() + + // Close stops the PartitionConsumer from fetching messages. It will initiate a shutdown just like AsyncClose, drain + // the Messages channel, harvest any errors & return them to the caller. Note that if you are continuing to service + // the Messages channel when this function is called, you will be competing with Close for messages; consider + // calling AsyncClose, instead. It is required to call this function (or AsyncClose) before a consumer object passes + // out of scope, as it will otherwise leak memory. You must call this before calling Close on the underlying client. + Close() error + + // Messages returns the read channel for the messages that are returned by + // the broker. + Messages() <-chan *ConsumerMessage + + // Errors returns a read channel of errors that occurred during consuming, if + // enabled. By default, errors are logged and not returned over this channel. + // If you want to implement any custom error handling, set your config's + // Consumer.Return.Errors setting to true, and read from this channel. + Errors() <-chan *ConsumerError + + // HighWaterMarkOffset returns the high water mark offset of the partition, + // i.e. the offset that will be used for the next message that will be produced. + // You can use this to determine how far behind the processing is. + HighWaterMarkOffset() int64 +} + +type partitionConsumer struct { + highWaterMarkOffset int64 // must be at the top of the struct because https://golang.org/pkg/sync/atomic/#pkg-note-BUG + + consumer *consumer + conf *Config + broker *brokerConsumer + messages chan *ConsumerMessage + errors chan *ConsumerError + feeder chan *FetchResponse + + preferredReadReplica int32 + + trigger, dying chan none + closeOnce sync.Once + topic string + partition int32 + responseResult error + fetchSize int32 + offset int64 + retries int32 +} + +var errTimedOut = errors.New("timed out feeding messages to the user") // not user-facing + +func (child *partitionConsumer) sendError(err error) { + cErr := &ConsumerError{ + Topic: child.topic, + Partition: child.partition, + Err: err, + } + + if child.conf.Consumer.Return.Errors { + child.errors <- cErr + } else { + Logger.Println(cErr) + } +} + +func (child *partitionConsumer) computeBackoff() time.Duration { + if child.conf.Consumer.Retry.BackoffFunc != nil { + retries := atomic.AddInt32(&child.retries, 1) + return child.conf.Consumer.Retry.BackoffFunc(int(retries)) + } + return child.conf.Consumer.Retry.Backoff +} + +func (child *partitionConsumer) dispatcher() { + for range child.trigger { + select { + case <-child.dying: + close(child.trigger) + case <-time.After(child.computeBackoff()): + if child.broker != nil { + child.consumer.unrefBrokerConsumer(child.broker) + child.broker = nil + } + + Logger.Printf("consumer/%s/%d finding new broker\n", child.topic, child.partition) + if err := child.dispatch(); err != nil { + child.sendError(err) + child.trigger <- none{} + } + } + } + + if child.broker != nil { + child.consumer.unrefBrokerConsumer(child.broker) + } + child.consumer.removeChild(child) + close(child.feeder) +} + +func (child *partitionConsumer) preferredBroker() (*Broker, error) { + if child.preferredReadReplica >= 0 { + broker, err := child.consumer.client.Broker(child.preferredReadReplica) + if err == nil { + return broker, nil + } + } + + // if preferred replica cannot be found fallback to leader + return child.consumer.client.Leader(child.topic, child.partition) +} + +func (child *partitionConsumer) dispatch() error { + if err := child.consumer.client.RefreshMetadata(child.topic); err != nil { + return err + } + + broker, err := child.preferredBroker() + if err != nil { + return err + } + + child.broker = child.consumer.refBrokerConsumer(broker) + + child.broker.input <- child + + return nil +} + +func (child *partitionConsumer) chooseStartingOffset(offset int64) error { + newestOffset, err := child.consumer.client.GetOffset(child.topic, child.partition, OffsetNewest) + if err != nil { + return err + } + oldestOffset, err := child.consumer.client.GetOffset(child.topic, child.partition, OffsetOldest) + if err != nil { + return err + } + + switch { + case offset == OffsetNewest: + child.offset = newestOffset + case offset == OffsetOldest: + child.offset = oldestOffset + case offset >= oldestOffset && offset <= newestOffset: + child.offset = offset + default: + return ErrOffsetOutOfRange + } + + return nil +} + +func (child *partitionConsumer) Messages() <-chan *ConsumerMessage { + return child.messages +} + +func (child *partitionConsumer) Errors() <-chan *ConsumerError { + return child.errors +} + +func (child *partitionConsumer) AsyncClose() { + // this triggers whatever broker owns this child to abandon it and close its trigger channel, which causes + // the dispatcher to exit its loop, which removes it from the consumer then closes its 'messages' and + // 'errors' channel (alternatively, if the child is already at the dispatcher for some reason, that will + // also just close itself) + child.closeOnce.Do(func() { + close(child.dying) + }) +} + +func (child *partitionConsumer) Close() error { + child.AsyncClose() + + var consumerErrors ConsumerErrors + for err := range child.errors { + consumerErrors = append(consumerErrors, err) + } + + if len(consumerErrors) > 0 { + return consumerErrors + } + return nil +} + +func (child *partitionConsumer) HighWaterMarkOffset() int64 { + return atomic.LoadInt64(&child.highWaterMarkOffset) +} + +func (child *partitionConsumer) responseFeeder() { + var msgs []*ConsumerMessage + expiryTicker := time.NewTicker(child.conf.Consumer.MaxProcessingTime) + firstAttempt := true + +feederLoop: + for response := range child.feeder { + msgs, child.responseResult = child.parseResponse(response) + + if child.responseResult == nil { + atomic.StoreInt32(&child.retries, 0) + } + + for i, msg := range msgs { + child.interceptors(msg) + messageSelect: + select { + case <-child.dying: + child.broker.acks.Done() + continue feederLoop + case child.messages <- msg: + firstAttempt = true + case <-expiryTicker.C: + if !firstAttempt { + child.responseResult = errTimedOut + child.broker.acks.Done() + remainingLoop: + for _, msg = range msgs[i:] { + child.interceptors(msg) + select { + case child.messages <- msg: + case <-child.dying: + break remainingLoop + } + } + child.broker.input <- child + continue feederLoop + } else { + // current message has not been sent, return to select + // statement + firstAttempt = false + goto messageSelect + } + } + } + + child.broker.acks.Done() + } + + expiryTicker.Stop() + close(child.messages) + close(child.errors) +} + +func (child *partitionConsumer) parseMessages(msgSet *MessageSet) ([]*ConsumerMessage, error) { + var messages []*ConsumerMessage + for _, msgBlock := range msgSet.Messages { + for _, msg := range msgBlock.Messages() { + offset := msg.Offset + timestamp := msg.Msg.Timestamp + if msg.Msg.Version >= 1 { + baseOffset := msgBlock.Offset - msgBlock.Messages()[len(msgBlock.Messages())-1].Offset + offset += baseOffset + if msg.Msg.LogAppendTime { + timestamp = msgBlock.Msg.Timestamp + } + } + if offset < child.offset { + continue + } + messages = append(messages, &ConsumerMessage{ + Topic: child.topic, + Partition: child.partition, + Key: msg.Msg.Key, + Value: msg.Msg.Value, + Offset: offset, + Timestamp: timestamp, + BlockTimestamp: msgBlock.Msg.Timestamp, + }) + child.offset = offset + 1 + } + } + if len(messages) == 0 { + child.offset++ + } + return messages, nil +} + +func (child *partitionConsumer) parseRecords(batch *RecordBatch) ([]*ConsumerMessage, error) { + messages := make([]*ConsumerMessage, 0, len(batch.Records)) + + for _, rec := range batch.Records { + offset := batch.FirstOffset + rec.OffsetDelta + if offset < child.offset { + continue + } + timestamp := batch.FirstTimestamp.Add(rec.TimestampDelta) + if batch.LogAppendTime { + timestamp = batch.MaxTimestamp + } + messages = append(messages, &ConsumerMessage{ + Topic: child.topic, + Partition: child.partition, + Key: rec.Key, + Value: rec.Value, + Offset: offset, + Timestamp: timestamp, + Headers: rec.Headers, + }) + child.offset = offset + 1 + } + if len(messages) == 0 { + child.offset++ + } + return messages, nil +} + +func (child *partitionConsumer) parseResponse(response *FetchResponse) ([]*ConsumerMessage, error) { + var ( + metricRegistry = child.conf.MetricRegistry + consumerBatchSizeMetric metrics.Histogram + ) + + if metricRegistry != nil { + consumerBatchSizeMetric = getOrRegisterHistogram("consumer-batch-size", metricRegistry) + } + + // If request was throttled and empty we log and return without error + if response.ThrottleTime != time.Duration(0) && len(response.Blocks) == 0 { + Logger.Printf( + "consumer/broker/%d FetchResponse throttled %v\n", + child.broker.broker.ID(), response.ThrottleTime) + return nil, nil + } + + block := response.GetBlock(child.topic, child.partition) + if block == nil { + return nil, ErrIncompleteResponse + } + + if block.Err != ErrNoError { + return nil, block.Err + } + + nRecs, err := block.numRecords() + if err != nil { + return nil, err + } + + consumerBatchSizeMetric.Update(int64(nRecs)) + + child.preferredReadReplica = block.PreferredReadReplica + + if nRecs == 0 { + partialTrailingMessage, err := block.isPartial() + if err != nil { + return nil, err + } + // We got no messages. If we got a trailing one then we need to ask for more data. + // Otherwise we just poll again and wait for one to be produced... + if partialTrailingMessage { + if child.conf.Consumer.Fetch.Max > 0 && child.fetchSize == child.conf.Consumer.Fetch.Max { + // we can't ask for more data, we've hit the configured limit + child.sendError(ErrMessageTooLarge) + child.offset++ // skip this one so we can keep processing future messages + } else { + child.fetchSize *= 2 + // check int32 overflow + if child.fetchSize < 0 { + child.fetchSize = math.MaxInt32 + } + if child.conf.Consumer.Fetch.Max > 0 && child.fetchSize > child.conf.Consumer.Fetch.Max { + child.fetchSize = child.conf.Consumer.Fetch.Max + } + } + } + + return nil, nil + } + + // we got messages, reset our fetch size in case it was increased for a previous request + child.fetchSize = child.conf.Consumer.Fetch.Default + atomic.StoreInt64(&child.highWaterMarkOffset, block.HighWaterMarkOffset) + + // abortedProducerIDs contains producerID which message should be ignored as uncommitted + // - producerID are added when the partitionConsumer iterate over the offset at which an aborted transaction begins (abortedTransaction.FirstOffset) + // - producerID are removed when partitionConsumer iterate over an aborted controlRecord, meaning the aborted transaction for this producer is over + abortedProducerIDs := make(map[int64]struct{}, len(block.AbortedTransactions)) + abortedTransactions := block.getAbortedTransactions() + + var messages []*ConsumerMessage + for _, records := range block.RecordsSet { + switch records.recordsType { + case legacyRecords: + messageSetMessages, err := child.parseMessages(records.MsgSet) + if err != nil { + return nil, err + } + + messages = append(messages, messageSetMessages...) + case defaultRecords: + // Consume remaining abortedTransaction up to last offset of current batch + for _, txn := range abortedTransactions { + if txn.FirstOffset > records.RecordBatch.LastOffset() { + break + } + abortedProducerIDs[txn.ProducerID] = struct{}{} + // Pop abortedTransactions so that we never add it again + abortedTransactions = abortedTransactions[1:] + } + + recordBatchMessages, err := child.parseRecords(records.RecordBatch) + if err != nil { + return nil, err + } + + // Parse and commit offset but do not expose messages that are: + // - control records + // - part of an aborted transaction when set to `ReadCommitted` + + // control record + isControl, err := records.isControl() + if err != nil { + // I don't know why there is this continue in case of error to begin with + // Safe bet is to ignore control messages if ReadUncommitted + // and block on them in case of error and ReadCommitted + if child.conf.Consumer.IsolationLevel == ReadCommitted { + return nil, err + } + continue + } + if isControl { + controlRecord, err := records.getControlRecord() + if err != nil { + return nil, err + } + + if controlRecord.Type == ControlRecordAbort { + delete(abortedProducerIDs, records.RecordBatch.ProducerID) + } + continue + } + + // filter aborted transactions + if child.conf.Consumer.IsolationLevel == ReadCommitted { + _, isAborted := abortedProducerIDs[records.RecordBatch.ProducerID] + if records.RecordBatch.IsTransactional && isAborted { + continue + } + } + + messages = append(messages, recordBatchMessages...) + default: + return nil, fmt.Errorf("unknown records type: %v", records.recordsType) + } + } + + return messages, nil +} + +func (child *partitionConsumer) interceptors(msg *ConsumerMessage) { + for _, interceptor := range child.conf.Consumer.Interceptors { + msg.safelyApplyInterceptor(interceptor) + } +} + +type brokerConsumer struct { + consumer *consumer + broker *Broker + input chan *partitionConsumer + newSubscriptions chan []*partitionConsumer + subscriptions map[*partitionConsumer]none + wait chan none + acks sync.WaitGroup + refs int +} + +func (c *consumer) newBrokerConsumer(broker *Broker) *brokerConsumer { + bc := &brokerConsumer{ + consumer: c, + broker: broker, + input: make(chan *partitionConsumer), + newSubscriptions: make(chan []*partitionConsumer), + wait: make(chan none), + subscriptions: make(map[*partitionConsumer]none), + refs: 0, + } + + go withRecover(bc.subscriptionManager) + go withRecover(bc.subscriptionConsumer) + + return bc +} + +// The subscriptionManager constantly accepts new subscriptions on `input` (even when the main subscriptionConsumer +// goroutine is in the middle of a network request) and batches it up. The main worker goroutine picks +// up a batch of new subscriptions between every network request by reading from `newSubscriptions`, so we give +// it nil if no new subscriptions are available. We also write to `wait` only when new subscriptions is available, +// so the main goroutine can block waiting for work if it has none. +func (bc *brokerConsumer) subscriptionManager() { + var buffer []*partitionConsumer + + for { + if len(buffer) > 0 { + select { + case event, ok := <-bc.input: + if !ok { + goto done + } + buffer = append(buffer, event) + case bc.newSubscriptions <- buffer: + buffer = nil + case bc.wait <- none{}: + } + } else { + select { + case event, ok := <-bc.input: + if !ok { + goto done + } + buffer = append(buffer, event) + case bc.newSubscriptions <- nil: + } + } + } + +done: + close(bc.wait) + if len(buffer) > 0 { + bc.newSubscriptions <- buffer + } + close(bc.newSubscriptions) +} + +// subscriptionConsumer ensures we will get nil right away if no new subscriptions is available +func (bc *brokerConsumer) subscriptionConsumer() { + <-bc.wait // wait for our first piece of work + + for newSubscriptions := range bc.newSubscriptions { + bc.updateSubscriptions(newSubscriptions) + + if len(bc.subscriptions) == 0 { + // We're about to be shut down or we're about to receive more subscriptions. + // Either way, the signal just hasn't propagated to our goroutine yet. + <-bc.wait + continue + } + + response, err := bc.fetchNewMessages() + if err != nil { + Logger.Printf("consumer/broker/%d disconnecting due to error processing FetchRequest: %s\n", bc.broker.ID(), err) + bc.abort(err) + return + } + + bc.acks.Add(len(bc.subscriptions)) + for child := range bc.subscriptions { + child.feeder <- response + } + bc.acks.Wait() + bc.handleResponses() + } +} + +func (bc *brokerConsumer) updateSubscriptions(newSubscriptions []*partitionConsumer) { + for _, child := range newSubscriptions { + bc.subscriptions[child] = none{} + Logger.Printf("consumer/broker/%d added subscription to %s/%d\n", bc.broker.ID(), child.topic, child.partition) + } + + for child := range bc.subscriptions { + select { + case <-child.dying: + Logger.Printf("consumer/broker/%d closed dead subscription to %s/%d\n", bc.broker.ID(), child.topic, child.partition) + close(child.trigger) + delete(bc.subscriptions, child) + default: + // no-op + } + } +} + +// handleResponses handles the response codes left for us by our subscriptions, and abandons ones that have been closed +func (bc *brokerConsumer) handleResponses() { + for child := range bc.subscriptions { + result := child.responseResult + child.responseResult = nil + + if result == nil { + if preferredBroker, err := child.preferredBroker(); err == nil { + if bc.broker.ID() != preferredBroker.ID() { + // not an error but needs redispatching to consume from preferred replica + child.trigger <- none{} + delete(bc.subscriptions, child) + } + } + continue + } + + // Discard any replica preference. + child.preferredReadReplica = -1 + + switch result { + case errTimedOut: + Logger.Printf("consumer/broker/%d abandoned subscription to %s/%d because consuming was taking too long\n", + bc.broker.ID(), child.topic, child.partition) + delete(bc.subscriptions, child) + case ErrOffsetOutOfRange: + // there's no point in retrying this it will just fail the same way again + // shut it down and force the user to choose what to do + child.sendError(result) + Logger.Printf("consumer/%s/%d shutting down because %s\n", child.topic, child.partition, result) + close(child.trigger) + delete(bc.subscriptions, child) + case ErrUnknownTopicOrPartition, ErrNotLeaderForPartition, ErrLeaderNotAvailable, ErrReplicaNotAvailable: + // not an error, but does need redispatching + Logger.Printf("consumer/broker/%d abandoned subscription to %s/%d because %s\n", + bc.broker.ID(), child.topic, child.partition, result) + child.trigger <- none{} + delete(bc.subscriptions, child) + default: + // dunno, tell the user and try redispatching + child.sendError(result) + Logger.Printf("consumer/broker/%d abandoned subscription to %s/%d because %s\n", + bc.broker.ID(), child.topic, child.partition, result) + child.trigger <- none{} + delete(bc.subscriptions, child) + } + } +} + +func (bc *brokerConsumer) abort(err error) { + bc.consumer.abandonBrokerConsumer(bc) + _ = bc.broker.Close() // we don't care about the error this might return, we already have one + + for child := range bc.subscriptions { + child.sendError(err) + child.trigger <- none{} + } + + for newSubscriptions := range bc.newSubscriptions { + if len(newSubscriptions) == 0 { + <-bc.wait + continue + } + for _, child := range newSubscriptions { + child.sendError(err) + child.trigger <- none{} + } + } +} + +func (bc *brokerConsumer) fetchNewMessages() (*FetchResponse, error) { + request := &FetchRequest{ + MinBytes: bc.consumer.conf.Consumer.Fetch.Min, + MaxWaitTime: int32(bc.consumer.conf.Consumer.MaxWaitTime / time.Millisecond), + } + if bc.consumer.conf.Version.IsAtLeast(V0_9_0_0) { + request.Version = 1 + } + if bc.consumer.conf.Version.IsAtLeast(V0_10_0_0) { + request.Version = 2 + } + if bc.consumer.conf.Version.IsAtLeast(V0_10_1_0) { + request.Version = 3 + request.MaxBytes = MaxResponseSize + } + if bc.consumer.conf.Version.IsAtLeast(V0_11_0_0) { + request.Version = 4 + request.Isolation = bc.consumer.conf.Consumer.IsolationLevel + } + if bc.consumer.conf.Version.IsAtLeast(V1_1_0_0) { + request.Version = 7 + // We do not currently implement KIP-227 FetchSessions. Setting the id to 0 + // and the epoch to -1 tells the broker not to generate as session ID we're going + // to just ignore anyway. + request.SessionID = 0 + request.SessionEpoch = -1 + } + if bc.consumer.conf.Version.IsAtLeast(V2_1_0_0) { + request.Version = 10 + } + if bc.consumer.conf.Version.IsAtLeast(V2_3_0_0) { + request.Version = 11 + request.RackID = bc.consumer.conf.RackID + } + + for child := range bc.subscriptions { + request.AddBlock(child.topic, child.partition, child.offset, child.fetchSize) + } + + return bc.broker.Fetch(request) +} diff --git a/vendor/github.com/Shopify/sarama/consumer_group.go b/vendor/github.com/Shopify/sarama/consumer_group.go new file mode 100644 index 000000000000..b603d1705aba --- /dev/null +++ b/vendor/github.com/Shopify/sarama/consumer_group.go @@ -0,0 +1,923 @@ +package sarama + +import ( + "context" + "errors" + "fmt" + "sort" + "sync" + "time" + + "github.com/rcrowley/go-metrics" +) + +// ErrClosedConsumerGroup is the error returned when a method is called on a consumer group that has been closed. +var ErrClosedConsumerGroup = errors.New("kafka: tried to use a consumer group that was closed") + +// ConsumerGroup is responsible for dividing up processing of topics and partitions +// over a collection of processes (the members of the consumer group). +type ConsumerGroup interface { + // Consume joins a cluster of consumers for a given list of topics and + // starts a blocking ConsumerGroupSession through the ConsumerGroupHandler. + // + // The life-cycle of a session is represented by the following steps: + // + // 1. The consumers join the group (as explained in https://kafka.apache.org/documentation/#intro_consumers) + // and is assigned their "fair share" of partitions, aka 'claims'. + // 2. Before processing starts, the handler's Setup() hook is called to notify the user + // of the claims and allow any necessary preparation or alteration of state. + // 3. For each of the assigned claims the handler's ConsumeClaim() function is then called + // in a separate goroutine which requires it to be thread-safe. Any state must be carefully protected + // from concurrent reads/writes. + // 4. The session will persist until one of the ConsumeClaim() functions exits. This can be either when the + // parent context is canceled or when a server-side rebalance cycle is initiated. + // 5. Once all the ConsumeClaim() loops have exited, the handler's Cleanup() hook is called + // to allow the user to perform any final tasks before a rebalance. + // 6. Finally, marked offsets are committed one last time before claims are released. + // + // Please note, that once a rebalance is triggered, sessions must be completed within + // Config.Consumer.Group.Rebalance.Timeout. This means that ConsumeClaim() functions must exit + // as quickly as possible to allow time for Cleanup() and the final offset commit. If the timeout + // is exceeded, the consumer will be removed from the group by Kafka, which will cause offset + // commit failures. + // This method should be called inside an infinite loop, when a + // server-side rebalance happens, the consumer session will need to be + // recreated to get the new claims. + Consume(ctx context.Context, topics []string, handler ConsumerGroupHandler) error + + // Errors returns a read channel of errors that occurred during the consumer life-cycle. + // By default, errors are logged and not returned over this channel. + // If you want to implement any custom error handling, set your config's + // Consumer.Return.Errors setting to true, and read from this channel. + Errors() <-chan error + + // Close stops the ConsumerGroup and detaches any running sessions. It is required to call + // this function before the object passes out of scope, as it will otherwise leak memory. + Close() error +} + +type consumerGroup struct { + client Client + + config *Config + consumer Consumer + groupID string + memberID string + errors chan error + + lock sync.Mutex + closed chan none + closeOnce sync.Once + + userData []byte +} + +// NewConsumerGroup creates a new consumer group the given broker addresses and configuration. +func NewConsumerGroup(addrs []string, groupID string, config *Config) (ConsumerGroup, error) { + client, err := NewClient(addrs, config) + if err != nil { + return nil, err + } + + c, err := newConsumerGroup(groupID, client) + if err != nil { + _ = client.Close() + } + return c, err +} + +// NewConsumerGroupFromClient creates a new consumer group using the given client. It is still +// necessary to call Close() on the underlying client when shutting down this consumer. +// PLEASE NOTE: consumer groups can only re-use but not share clients. +func NewConsumerGroupFromClient(groupID string, client Client) (ConsumerGroup, error) { + // For clients passed in by the client, ensure we don't + // call Close() on it. + cli := &nopCloserClient{client} + return newConsumerGroup(groupID, cli) +} + +func newConsumerGroup(groupID string, client Client) (ConsumerGroup, error) { + config := client.Config() + if !config.Version.IsAtLeast(V0_10_2_0) { + return nil, ConfigurationError("consumer groups require Version to be >= V0_10_2_0") + } + + consumer, err := NewConsumerFromClient(client) + if err != nil { + return nil, err + } + + return &consumerGroup{ + client: client, + consumer: consumer, + config: config, + groupID: groupID, + errors: make(chan error, config.ChannelBufferSize), + closed: make(chan none), + userData: config.Consumer.Group.Member.UserData, + }, nil +} + +// Errors implements ConsumerGroup. +func (c *consumerGroup) Errors() <-chan error { return c.errors } + +// Close implements ConsumerGroup. +func (c *consumerGroup) Close() (err error) { + c.closeOnce.Do(func() { + close(c.closed) + + // leave group + if e := c.leave(); e != nil { + err = e + } + + // drain errors + go func() { + close(c.errors) + }() + for e := range c.errors { + err = e + } + + if e := c.client.Close(); e != nil { + err = e + } + }) + return +} + +// Consume implements ConsumerGroup. +func (c *consumerGroup) Consume(ctx context.Context, topics []string, handler ConsumerGroupHandler) error { + // Ensure group is not closed + select { + case <-c.closed: + return ErrClosedConsumerGroup + default: + } + + c.lock.Lock() + defer c.lock.Unlock() + + // Quick exit when no topics are provided + if len(topics) == 0 { + return fmt.Errorf("no topics provided") + } + + // Refresh metadata for requested topics + if err := c.client.RefreshMetadata(topics...); err != nil { + return err + } + + // Init session + sess, err := c.newSession(ctx, topics, handler, c.config.Consumer.Group.Rebalance.Retry.Max) + if err == ErrClosedClient { + return ErrClosedConsumerGroup + } else if err != nil { + return err + } + + // loop check topic partition numbers changed + // will trigger rebalance when any topic partitions number had changed + // avoid Consume function called again that will generate more than loopCheckPartitionNumbers coroutine + go c.loopCheckPartitionNumbers(topics, sess) + + // Wait for session exit signal + <-sess.ctx.Done() + + // Gracefully release session claims + return sess.release(true) +} + +func (c *consumerGroup) retryNewSession(ctx context.Context, topics []string, handler ConsumerGroupHandler, retries int, refreshCoordinator bool) (*consumerGroupSession, error) { + select { + case <-c.closed: + return nil, ErrClosedConsumerGroup + case <-time.After(c.config.Consumer.Group.Rebalance.Retry.Backoff): + } + + if refreshCoordinator { + err := c.client.RefreshCoordinator(c.groupID) + if err != nil { + return c.retryNewSession(ctx, topics, handler, retries, true) + } + } + + return c.newSession(ctx, topics, handler, retries-1) +} + +func (c *consumerGroup) newSession(ctx context.Context, topics []string, handler ConsumerGroupHandler, retries int) (*consumerGroupSession, error) { + coordinator, err := c.client.Coordinator(c.groupID) + if err != nil { + if retries <= 0 { + return nil, err + } + + return c.retryNewSession(ctx, topics, handler, retries, true) + } + + var ( + metricRegistry = c.config.MetricRegistry + consumerGroupJoinTotal metrics.Counter + consumerGroupJoinFailed metrics.Counter + consumerGroupSyncTotal metrics.Counter + consumerGroupSyncFailed metrics.Counter + ) + + if metricRegistry != nil { + consumerGroupJoinTotal = metrics.GetOrRegisterCounter(fmt.Sprintf("consumer-group-join-total-%s", c.groupID), metricRegistry) + consumerGroupJoinFailed = metrics.GetOrRegisterCounter(fmt.Sprintf("consumer-group-join-failed-%s", c.groupID), metricRegistry) + consumerGroupSyncTotal = metrics.GetOrRegisterCounter(fmt.Sprintf("consumer-group-sync-total-%s", c.groupID), metricRegistry) + consumerGroupSyncFailed = metrics.GetOrRegisterCounter(fmt.Sprintf("consumer-group-sync-failed-%s", c.groupID), metricRegistry) + } + + // Join consumer group + join, err := c.joinGroupRequest(coordinator, topics) + if consumerGroupJoinTotal != nil { + consumerGroupJoinTotal.Inc(1) + } + if err != nil { + _ = coordinator.Close() + if consumerGroupJoinFailed != nil { + consumerGroupJoinFailed.Inc(1) + } + return nil, err + } + if join.Err != ErrNoError { + if consumerGroupJoinFailed != nil { + consumerGroupJoinFailed.Inc(1) + } + } + switch join.Err { + case ErrNoError: + c.memberID = join.MemberId + case ErrUnknownMemberId, ErrIllegalGeneration: // reset member ID and retry immediately + c.memberID = "" + return c.newSession(ctx, topics, handler, retries) + case ErrNotCoordinatorForConsumer: // retry after backoff with coordinator refresh + if retries <= 0 { + return nil, join.Err + } + + return c.retryNewSession(ctx, topics, handler, retries, true) + case ErrRebalanceInProgress: // retry after backoff + if retries <= 0 { + return nil, join.Err + } + + return c.retryNewSession(ctx, topics, handler, retries, false) + default: + return nil, join.Err + } + + // Prepare distribution plan if we joined as the leader + var plan BalanceStrategyPlan + if join.LeaderId == join.MemberId { + members, err := join.GetMembers() + if err != nil { + return nil, err + } + + plan, err = c.balance(members) + if err != nil { + return nil, err + } + } + + // Sync consumer group + groupRequest, err := c.syncGroupRequest(coordinator, plan, join.GenerationId) + if consumerGroupSyncTotal != nil { + consumerGroupSyncTotal.Inc(1) + } + if err != nil { + _ = coordinator.Close() + if consumerGroupSyncFailed != nil { + consumerGroupSyncFailed.Inc(1) + } + return nil, err + } + if groupRequest.Err != ErrNoError { + if consumerGroupSyncFailed != nil { + consumerGroupSyncFailed.Inc(1) + } + } + + switch groupRequest.Err { + case ErrNoError: + case ErrUnknownMemberId, ErrIllegalGeneration: // reset member ID and retry immediately + c.memberID = "" + return c.newSession(ctx, topics, handler, retries) + case ErrNotCoordinatorForConsumer: // retry after backoff with coordinator refresh + if retries <= 0 { + return nil, groupRequest.Err + } + + return c.retryNewSession(ctx, topics, handler, retries, true) + case ErrRebalanceInProgress: // retry after backoff + if retries <= 0 { + return nil, groupRequest.Err + } + + return c.retryNewSession(ctx, topics, handler, retries, false) + default: + return nil, groupRequest.Err + } + + // Retrieve and sort claims + var claims map[string][]int32 + if len(groupRequest.MemberAssignment) > 0 { + members, err := groupRequest.GetMemberAssignment() + if err != nil { + return nil, err + } + claims = members.Topics + + // in the case of stateful balance strategies, hold on to the returned + // assignment metadata, otherwise, reset the statically defined conusmer + // group metadata + if members.UserData != nil { + c.userData = members.UserData + } else { + c.userData = c.config.Consumer.Group.Member.UserData + } + + for _, partitions := range claims { + sort.Sort(int32Slice(partitions)) + } + } + + return newConsumerGroupSession(ctx, c, claims, join.MemberId, join.GenerationId, handler) +} + +func (c *consumerGroup) joinGroupRequest(coordinator *Broker, topics []string) (*JoinGroupResponse, error) { + req := &JoinGroupRequest{ + GroupId: c.groupID, + MemberId: c.memberID, + SessionTimeout: int32(c.config.Consumer.Group.Session.Timeout / time.Millisecond), + ProtocolType: "consumer", + } + if c.config.Version.IsAtLeast(V0_10_1_0) { + req.Version = 1 + req.RebalanceTimeout = int32(c.config.Consumer.Group.Rebalance.Timeout / time.Millisecond) + } + + meta := &ConsumerGroupMemberMetadata{ + Topics: topics, + UserData: c.userData, + } + strategy := c.config.Consumer.Group.Rebalance.Strategy + if err := req.AddGroupProtocolMetadata(strategy.Name(), meta); err != nil { + return nil, err + } + + return coordinator.JoinGroup(req) +} + +func (c *consumerGroup) syncGroupRequest(coordinator *Broker, plan BalanceStrategyPlan, generationID int32) (*SyncGroupResponse, error) { + req := &SyncGroupRequest{ + GroupId: c.groupID, + MemberId: c.memberID, + GenerationId: generationID, + } + strategy := c.config.Consumer.Group.Rebalance.Strategy + for memberID, topics := range plan { + assignment := &ConsumerGroupMemberAssignment{Topics: topics} + userDataBytes, err := strategy.AssignmentData(memberID, topics, generationID) + if err != nil { + return nil, err + } + assignment.UserData = userDataBytes + if err := req.AddGroupAssignmentMember(memberID, assignment); err != nil { + return nil, err + } + } + return coordinator.SyncGroup(req) +} + +func (c *consumerGroup) heartbeatRequest(coordinator *Broker, memberID string, generationID int32) (*HeartbeatResponse, error) { + req := &HeartbeatRequest{ + GroupId: c.groupID, + MemberId: memberID, + GenerationId: generationID, + } + + return coordinator.Heartbeat(req) +} + +func (c *consumerGroup) balance(members map[string]ConsumerGroupMemberMetadata) (BalanceStrategyPlan, error) { + topics := make(map[string][]int32) + for _, meta := range members { + for _, topic := range meta.Topics { + topics[topic] = nil + } + } + + for topic := range topics { + partitions, err := c.client.Partitions(topic) + if err != nil { + return nil, err + } + topics[topic] = partitions + } + + strategy := c.config.Consumer.Group.Rebalance.Strategy + return strategy.Plan(members, topics) +} + +// Leaves the cluster, called by Close. +func (c *consumerGroup) leave() error { + c.lock.Lock() + defer c.lock.Unlock() + if c.memberID == "" { + return nil + } + + coordinator, err := c.client.Coordinator(c.groupID) + if err != nil { + return err + } + + resp, err := coordinator.LeaveGroup(&LeaveGroupRequest{ + GroupId: c.groupID, + MemberId: c.memberID, + }) + if err != nil { + _ = coordinator.Close() + return err + } + + // Unset memberID + c.memberID = "" + + // Check response + switch resp.Err { + case ErrRebalanceInProgress, ErrUnknownMemberId, ErrNoError: + return nil + default: + return resp.Err + } +} + +func (c *consumerGroup) handleError(err error, topic string, partition int32) { + if _, ok := err.(*ConsumerError); !ok && topic != "" && partition > -1 { + err = &ConsumerError{ + Topic: topic, + Partition: partition, + Err: err, + } + } + + if !c.config.Consumer.Return.Errors { + Logger.Println(err) + return + } + + select { + case <-c.closed: + // consumer is closed + return + default: + } + + select { + case c.errors <- err: + default: + // no error listener + } +} + +func (c *consumerGroup) loopCheckPartitionNumbers(topics []string, session *consumerGroupSession) { + pause := time.NewTicker(c.config.Metadata.RefreshFrequency) + defer session.cancel() + defer pause.Stop() + var oldTopicToPartitionNum map[string]int + var err error + if oldTopicToPartitionNum, err = c.topicToPartitionNumbers(topics); err != nil { + return + } + for { + if newTopicToPartitionNum, err := c.topicToPartitionNumbers(topics); err != nil { + return + } else { + for topic, num := range oldTopicToPartitionNum { + if newTopicToPartitionNum[topic] != num { + return // trigger the end of the session on exit + } + } + } + select { + case <-pause.C: + case <-session.ctx.Done(): + Logger.Printf("loop check partition number coroutine will exit, topics %s", topics) + // if session closed by other, should be exited + return + case <-c.closed: + return + } + } +} + +func (c *consumerGroup) topicToPartitionNumbers(topics []string) (map[string]int, error) { + topicToPartitionNum := make(map[string]int, len(topics)) + for _, topic := range topics { + if partitionNum, err := c.client.Partitions(topic); err != nil { + Logger.Printf("Consumer Group topic %s get partition number failed %v", topic, err) + return nil, err + } else { + topicToPartitionNum[topic] = len(partitionNum) + } + } + return topicToPartitionNum, nil +} + +// -------------------------------------------------------------------- + +// ConsumerGroupSession represents a consumer group member session. +type ConsumerGroupSession interface { + // Claims returns information about the claimed partitions by topic. + Claims() map[string][]int32 + + // MemberID returns the cluster member ID. + MemberID() string + + // GenerationID returns the current generation ID. + GenerationID() int32 + + // MarkOffset marks the provided offset, alongside a metadata string + // that represents the state of the partition consumer at that point in time. The + // metadata string can be used by another consumer to restore that state, so it + // can resume consumption. + // + // To follow upstream conventions, you are expected to mark the offset of the + // next message to read, not the last message read. Thus, when calling `MarkOffset` + // you should typically add one to the offset of the last consumed message. + // + // Note: calling MarkOffset does not necessarily commit the offset to the backend + // store immediately for efficiency reasons, and it may never be committed if + // your application crashes. This means that you may end up processing the same + // message twice, and your processing should ideally be idempotent. + MarkOffset(topic string, partition int32, offset int64, metadata string) + + // Commit the offset to the backend + // + // Note: calling Commit performs a blocking synchronous operation. + Commit() + + // ResetOffset resets to the provided offset, alongside a metadata string that + // represents the state of the partition consumer at that point in time. Reset + // acts as a counterpart to MarkOffset, the difference being that it allows to + // reset an offset to an earlier or smaller value, where MarkOffset only + // allows incrementing the offset. cf MarkOffset for more details. + ResetOffset(topic string, partition int32, offset int64, metadata string) + + // MarkMessage marks a message as consumed. + MarkMessage(msg *ConsumerMessage, metadata string) + + // Context returns the session context. + Context() context.Context +} + +type consumerGroupSession struct { + parent *consumerGroup + memberID string + generationID int32 + handler ConsumerGroupHandler + + claims map[string][]int32 + offsets *offsetManager + ctx context.Context + cancel func() + + waitGroup sync.WaitGroup + releaseOnce sync.Once + hbDying, hbDead chan none +} + +func newConsumerGroupSession(ctx context.Context, parent *consumerGroup, claims map[string][]int32, memberID string, generationID int32, handler ConsumerGroupHandler) (*consumerGroupSession, error) { + // init offset manager + offsets, err := newOffsetManagerFromClient(parent.groupID, memberID, generationID, parent.client) + if err != nil { + return nil, err + } + + // init context + ctx, cancel := context.WithCancel(ctx) + + // init session + sess := &consumerGroupSession{ + parent: parent, + memberID: memberID, + generationID: generationID, + handler: handler, + offsets: offsets, + claims: claims, + ctx: ctx, + cancel: cancel, + hbDying: make(chan none), + hbDead: make(chan none), + } + + // start heartbeat loop + go sess.heartbeatLoop() + + // create a POM for each claim + for topic, partitions := range claims { + for _, partition := range partitions { + pom, err := offsets.ManagePartition(topic, partition) + if err != nil { + _ = sess.release(false) + return nil, err + } + + // handle POM errors + go func(topic string, partition int32) { + for err := range pom.Errors() { + sess.parent.handleError(err, topic, partition) + } + }(topic, partition) + } + } + + // perform setup + if err := handler.Setup(sess); err != nil { + _ = sess.release(true) + return nil, err + } + + // start consuming + for topic, partitions := range claims { + for _, partition := range partitions { + sess.waitGroup.Add(1) + + go func(topic string, partition int32) { + defer sess.waitGroup.Done() + + // cancel the as session as soon as the first + // goroutine exits + defer sess.cancel() + + // consume a single topic/partition, blocking + sess.consume(topic, partition) + }(topic, partition) + } + } + return sess, nil +} + +func (s *consumerGroupSession) Claims() map[string][]int32 { return s.claims } +func (s *consumerGroupSession) MemberID() string { return s.memberID } +func (s *consumerGroupSession) GenerationID() int32 { return s.generationID } + +func (s *consumerGroupSession) MarkOffset(topic string, partition int32, offset int64, metadata string) { + if pom := s.offsets.findPOM(topic, partition); pom != nil { + pom.MarkOffset(offset, metadata) + } +} + +func (s *consumerGroupSession) Commit() { + s.offsets.Commit() +} + +func (s *consumerGroupSession) ResetOffset(topic string, partition int32, offset int64, metadata string) { + if pom := s.offsets.findPOM(topic, partition); pom != nil { + pom.ResetOffset(offset, metadata) + } +} + +func (s *consumerGroupSession) MarkMessage(msg *ConsumerMessage, metadata string) { + s.MarkOffset(msg.Topic, msg.Partition, msg.Offset+1, metadata) +} + +func (s *consumerGroupSession) Context() context.Context { + return s.ctx +} + +func (s *consumerGroupSession) consume(topic string, partition int32) { + // quick exit if rebalance is due + select { + case <-s.ctx.Done(): + return + case <-s.parent.closed: + return + default: + } + + // get next offset + offset := s.parent.config.Consumer.Offsets.Initial + if pom := s.offsets.findPOM(topic, partition); pom != nil { + offset, _ = pom.NextOffset() + } + + // create new claim + claim, err := newConsumerGroupClaim(s, topic, partition, offset) + if err != nil { + s.parent.handleError(err, topic, partition) + return + } + + // handle errors + go func() { + for err := range claim.Errors() { + s.parent.handleError(err, topic, partition) + } + }() + + // trigger close when session is done + go func() { + select { + case <-s.ctx.Done(): + case <-s.parent.closed: + } + claim.AsyncClose() + }() + + // start processing + if err := s.handler.ConsumeClaim(s, claim); err != nil { + s.parent.handleError(err, topic, partition) + } + + // ensure consumer is closed & drained + claim.AsyncClose() + for _, err := range claim.waitClosed() { + s.parent.handleError(err, topic, partition) + } +} + +func (s *consumerGroupSession) release(withCleanup bool) (err error) { + // signal release, stop heartbeat + s.cancel() + + // wait for consumers to exit + s.waitGroup.Wait() + + // perform release + s.releaseOnce.Do(func() { + if withCleanup { + if e := s.handler.Cleanup(s); e != nil { + s.parent.handleError(e, "", -1) + err = e + } + } + + if e := s.offsets.Close(); e != nil { + err = e + } + + close(s.hbDying) + <-s.hbDead + }) + + return +} + +func (s *consumerGroupSession) heartbeatLoop() { + defer close(s.hbDead) + defer s.cancel() // trigger the end of the session on exit + + pause := time.NewTicker(s.parent.config.Consumer.Group.Heartbeat.Interval) + defer pause.Stop() + + retryBackoff := time.NewTimer(s.parent.config.Metadata.Retry.Backoff) + defer retryBackoff.Stop() + + retries := s.parent.config.Metadata.Retry.Max + for { + coordinator, err := s.parent.client.Coordinator(s.parent.groupID) + if err != nil { + if retries <= 0 { + s.parent.handleError(err, "", -1) + return + } + retryBackoff.Reset(s.parent.config.Metadata.Retry.Backoff) + select { + case <-s.hbDying: + return + case <-retryBackoff.C: + retries-- + } + continue + } + + resp, err := s.parent.heartbeatRequest(coordinator, s.memberID, s.generationID) + if err != nil { + _ = coordinator.Close() + + if retries <= 0 { + s.parent.handleError(err, "", -1) + return + } + + retries-- + continue + } + + switch resp.Err { + case ErrNoError: + retries = s.parent.config.Metadata.Retry.Max + case ErrRebalanceInProgress, ErrUnknownMemberId, ErrIllegalGeneration: + return + default: + s.parent.handleError(resp.Err, "", -1) + return + } + + select { + case <-pause.C: + case <-s.hbDying: + return + } + } +} + +// -------------------------------------------------------------------- + +// ConsumerGroupHandler instances are used to handle individual topic/partition claims. +// It also provides hooks for your consumer group session life-cycle and allow you to +// trigger logic before or after the consume loop(s). +// +// PLEASE NOTE that handlers are likely be called from several goroutines concurrently, +// ensure that all state is safely protected against race conditions. +type ConsumerGroupHandler interface { + // Setup is run at the beginning of a new session, before ConsumeClaim. + Setup(ConsumerGroupSession) error + + // Cleanup is run at the end of a session, once all ConsumeClaim goroutines have exited + // but before the offsets are committed for the very last time. + Cleanup(ConsumerGroupSession) error + + // ConsumeClaim must start a consumer loop of ConsumerGroupClaim's Messages(). + // Once the Messages() channel is closed, the Handler must finish its processing + // loop and exit. + ConsumeClaim(ConsumerGroupSession, ConsumerGroupClaim) error +} + +// ConsumerGroupClaim processes Kafka messages from a given topic and partition within a consumer group. +type ConsumerGroupClaim interface { + // Topic returns the consumed topic name. + Topic() string + + // Partition returns the consumed partition. + Partition() int32 + + // InitialOffset returns the initial offset that was used as a starting point for this claim. + InitialOffset() int64 + + // HighWaterMarkOffset returns the high water mark offset of the partition, + // i.e. the offset that will be used for the next message that will be produced. + // You can use this to determine how far behind the processing is. + HighWaterMarkOffset() int64 + + // Messages returns the read channel for the messages that are returned by + // the broker. The messages channel will be closed when a new rebalance cycle + // is due. You must finish processing and mark offsets within + // Config.Consumer.Group.Session.Timeout before the topic/partition is eventually + // re-assigned to another group member. + Messages() <-chan *ConsumerMessage +} + +type consumerGroupClaim struct { + topic string + partition int32 + offset int64 + PartitionConsumer +} + +func newConsumerGroupClaim(sess *consumerGroupSession, topic string, partition int32, offset int64) (*consumerGroupClaim, error) { + pcm, err := sess.parent.consumer.ConsumePartition(topic, partition, offset) + if err == ErrOffsetOutOfRange { + offset = sess.parent.config.Consumer.Offsets.Initial + pcm, err = sess.parent.consumer.ConsumePartition(topic, partition, offset) + } + if err != nil { + return nil, err + } + + go func() { + for err := range pcm.Errors() { + sess.parent.handleError(err, topic, partition) + } + }() + + return &consumerGroupClaim{ + topic: topic, + partition: partition, + offset: offset, + PartitionConsumer: pcm, + }, nil +} + +func (c *consumerGroupClaim) Topic() string { return c.topic } +func (c *consumerGroupClaim) Partition() int32 { return c.partition } +func (c *consumerGroupClaim) InitialOffset() int64 { return c.offset } + +// Drains messages and errors, ensures the claim is fully closed. +func (c *consumerGroupClaim) waitClosed() (errs ConsumerErrors) { + go func() { + for range c.Messages() { + } + }() + + for err := range c.Errors() { + errs = append(errs, err) + } + return +} diff --git a/vendor/github.com/Shopify/sarama/consumer_group_members.go b/vendor/github.com/Shopify/sarama/consumer_group_members.go new file mode 100644 index 000000000000..21b11e944fe5 --- /dev/null +++ b/vendor/github.com/Shopify/sarama/consumer_group_members.go @@ -0,0 +1,96 @@ +package sarama + +// ConsumerGroupMemberMetadata holds the metadata for consumer group +type ConsumerGroupMemberMetadata struct { + Version int16 + Topics []string + UserData []byte +} + +func (m *ConsumerGroupMemberMetadata) encode(pe packetEncoder) error { + pe.putInt16(m.Version) + + if err := pe.putStringArray(m.Topics); err != nil { + return err + } + + if err := pe.putBytes(m.UserData); err != nil { + return err + } + + return nil +} + +func (m *ConsumerGroupMemberMetadata) decode(pd packetDecoder) (err error) { + if m.Version, err = pd.getInt16(); err != nil { + return + } + + if m.Topics, err = pd.getStringArray(); err != nil { + return + } + + if m.UserData, err = pd.getBytes(); err != nil { + return + } + + return nil +} + +// ConsumerGroupMemberAssignment holds the member assignment for a consume group +type ConsumerGroupMemberAssignment struct { + Version int16 + Topics map[string][]int32 + UserData []byte +} + +func (m *ConsumerGroupMemberAssignment) encode(pe packetEncoder) error { + pe.putInt16(m.Version) + + if err := pe.putArrayLength(len(m.Topics)); err != nil { + return err + } + + for topic, partitions := range m.Topics { + if err := pe.putString(topic); err != nil { + return err + } + if err := pe.putInt32Array(partitions); err != nil { + return err + } + } + + if err := pe.putBytes(m.UserData); err != nil { + return err + } + + return nil +} + +func (m *ConsumerGroupMemberAssignment) decode(pd packetDecoder) (err error) { + if m.Version, err = pd.getInt16(); err != nil { + return + } + + var topicLen int + if topicLen, err = pd.getArrayLength(); err != nil { + return + } + + m.Topics = make(map[string][]int32, topicLen) + for i := 0; i < topicLen; i++ { + var topic string + if topic, err = pd.getString(); err != nil { + return + } + if m.Topics[topic], err = pd.getInt32Array(); err != nil { + return + } + } + + if m.UserData, err = pd.getBytes(); err != nil { + return + } + + return nil +} diff --git a/vendor/github.com/Shopify/sarama/consumer_metadata_request.go b/vendor/github.com/Shopify/sarama/consumer_metadata_request.go new file mode 100644 index 000000000000..5c18e048a720 --- /dev/null +++ b/vendor/github.com/Shopify/sarama/consumer_metadata_request.go @@ -0,0 +1,38 @@ +package sarama + +// ConsumerMetadataRequest is used for metadata requests +type ConsumerMetadataRequest struct { + ConsumerGroup string +} + +func (r *ConsumerMetadataRequest) encode(pe packetEncoder) error { + tmp := new(FindCoordinatorRequest) + tmp.CoordinatorKey = r.ConsumerGroup + tmp.CoordinatorType = CoordinatorGroup + return tmp.encode(pe) +} + +func (r *ConsumerMetadataRequest) decode(pd packetDecoder, version int16) (err error) { + tmp := new(FindCoordinatorRequest) + if err := tmp.decode(pd, version); err != nil { + return err + } + r.ConsumerGroup = tmp.CoordinatorKey + return nil +} + +func (r *ConsumerMetadataRequest) key() int16 { + return 10 +} + +func (r *ConsumerMetadataRequest) version() int16 { + return 0 +} + +func (r *ConsumerMetadataRequest) headerVersion() int16 { + return 1 +} + +func (r *ConsumerMetadataRequest) requiredVersion() KafkaVersion { + return V0_8_2_0 +} diff --git a/vendor/github.com/Shopify/sarama/consumer_metadata_response.go b/vendor/github.com/Shopify/sarama/consumer_metadata_response.go new file mode 100644 index 000000000000..7fe0cf9716de --- /dev/null +++ b/vendor/github.com/Shopify/sarama/consumer_metadata_response.go @@ -0,0 +1,82 @@ +package sarama + +import ( + "net" + "strconv" +) + +// ConsumerMetadataResponse holds the response for a consumer group meta data requests +type ConsumerMetadataResponse struct { + Err KError + Coordinator *Broker + CoordinatorID int32 // deprecated: use Coordinator.ID() + CoordinatorHost string // deprecated: use Coordinator.Addr() + CoordinatorPort int32 // deprecated: use Coordinator.Addr() +} + +func (r *ConsumerMetadataResponse) decode(pd packetDecoder, version int16) (err error) { + tmp := new(FindCoordinatorResponse) + + if err := tmp.decode(pd, version); err != nil { + return err + } + + r.Err = tmp.Err + + r.Coordinator = tmp.Coordinator + if tmp.Coordinator == nil { + return nil + } + + // this can all go away in 2.0, but we have to fill in deprecated fields to maintain + // backwards compatibility + host, portstr, err := net.SplitHostPort(r.Coordinator.Addr()) + if err != nil { + return err + } + port, err := strconv.ParseInt(portstr, 10, 32) + if err != nil { + return err + } + r.CoordinatorID = r.Coordinator.ID() + r.CoordinatorHost = host + r.CoordinatorPort = int32(port) + + return nil +} + +func (r *ConsumerMetadataResponse) encode(pe packetEncoder) error { + if r.Coordinator == nil { + r.Coordinator = new(Broker) + r.Coordinator.id = r.CoordinatorID + r.Coordinator.addr = net.JoinHostPort(r.CoordinatorHost, strconv.Itoa(int(r.CoordinatorPort))) + } + + tmp := &FindCoordinatorResponse{ + Version: 0, + Err: r.Err, + Coordinator: r.Coordinator, + } + + if err := tmp.encode(pe); err != nil { + return err + } + + return nil +} + +func (r *ConsumerMetadataResponse) key() int16 { + return 10 +} + +func (r *ConsumerMetadataResponse) version() int16 { + return 0 +} + +func (r *ConsumerMetadataResponse) headerVersion() int16 { + return 0 +} + +func (r *ConsumerMetadataResponse) requiredVersion() KafkaVersion { + return V0_8_2_0 +} diff --git a/vendor/github.com/Shopify/sarama/control_record.go b/vendor/github.com/Shopify/sarama/control_record.go new file mode 100644 index 000000000000..244a821368d7 --- /dev/null +++ b/vendor/github.com/Shopify/sarama/control_record.go @@ -0,0 +1,74 @@ +package sarama + +// ControlRecordType ... +type ControlRecordType int + +const ( + // ControlRecordAbort is a control record for abort + ControlRecordAbort ControlRecordType = iota + // ControlRecordCommit is a control record for commit + ControlRecordCommit + // ControlRecordUnknown is a control record of unknown type + ControlRecordUnknown +) + +// Control records are returned as a record by fetchRequest +// However unlike "normal" records, they mean nothing application wise. +// They only serve internal logic for supporting transactions. +type ControlRecord struct { + Version int16 + CoordinatorEpoch int32 + Type ControlRecordType +} + +func (cr *ControlRecord) decode(key, value packetDecoder) error { + var err error + // There a version for the value part AND the key part. And I have no idea if they are supposed to match or not + // Either way, all these version can only be 0 for now + cr.Version, err = key.getInt16() + if err != nil { + return err + } + + recordType, err := key.getInt16() + if err != nil { + return err + } + + switch recordType { + case 0: + cr.Type = ControlRecordAbort + case 1: + cr.Type = ControlRecordCommit + default: + // from JAVA implementation: + // UNKNOWN is used to indicate a control type which the client is not aware of and should be ignored + cr.Type = ControlRecordUnknown + } + // we want to parse value only if we are decoding control record of known type + if cr.Type != ControlRecordUnknown { + cr.Version, err = value.getInt16() + if err != nil { + return err + } + + cr.CoordinatorEpoch, err = value.getInt32() + if err != nil { + return err + } + } + return nil +} + +func (cr *ControlRecord) encode(key, value packetEncoder) { + value.putInt16(cr.Version) + value.putInt32(cr.CoordinatorEpoch) + key.putInt16(cr.Version) + + switch cr.Type { + case ControlRecordAbort: + key.putInt16(0) + case ControlRecordCommit: + key.putInt16(1) + } +} diff --git a/vendor/github.com/Shopify/sarama/crc32_field.go b/vendor/github.com/Shopify/sarama/crc32_field.go new file mode 100644 index 000000000000..32236e50f038 --- /dev/null +++ b/vendor/github.com/Shopify/sarama/crc32_field.go @@ -0,0 +1,87 @@ +package sarama + +import ( + "encoding/binary" + "fmt" + "hash/crc32" + "sync" +) + +type crcPolynomial int8 + +const ( + crcIEEE crcPolynomial = iota + crcCastagnoli +) + +var crc32FieldPool = sync.Pool{} + +func acquireCrc32Field(polynomial crcPolynomial) *crc32Field { + val := crc32FieldPool.Get() + if val != nil { + c := val.(*crc32Field) + c.polynomial = polynomial + return c + } + return newCRC32Field(polynomial) +} + +func releaseCrc32Field(c *crc32Field) { + crc32FieldPool.Put(c) +} + +var castagnoliTable = crc32.MakeTable(crc32.Castagnoli) + +// crc32Field implements the pushEncoder and pushDecoder interfaces for calculating CRC32s. +type crc32Field struct { + startOffset int + polynomial crcPolynomial +} + +func (c *crc32Field) saveOffset(in int) { + c.startOffset = in +} + +func (c *crc32Field) reserveLength() int { + return 4 +} + +func newCRC32Field(polynomial crcPolynomial) *crc32Field { + return &crc32Field{polynomial: polynomial} +} + +func (c *crc32Field) run(curOffset int, buf []byte) error { + crc, err := c.crc(curOffset, buf) + if err != nil { + return err + } + binary.BigEndian.PutUint32(buf[c.startOffset:], crc) + return nil +} + +func (c *crc32Field) check(curOffset int, buf []byte) error { + crc, err := c.crc(curOffset, buf) + if err != nil { + return err + } + + expected := binary.BigEndian.Uint32(buf[c.startOffset:]) + if crc != expected { + return PacketDecodingError{fmt.Sprintf("CRC didn't match expected %#x got %#x", expected, crc)} + } + + return nil +} + +func (c *crc32Field) crc(curOffset int, buf []byte) (uint32, error) { + var tab *crc32.Table + switch c.polynomial { + case crcIEEE: + tab = crc32.IEEETable + case crcCastagnoli: + tab = castagnoliTable + default: + return 0, PacketDecodingError{"invalid CRC type"} + } + return crc32.Checksum(buf[c.startOffset+4:curOffset], tab), nil +} diff --git a/vendor/github.com/Shopify/sarama/create_partitions_request.go b/vendor/github.com/Shopify/sarama/create_partitions_request.go new file mode 100644 index 000000000000..46fb0440249c --- /dev/null +++ b/vendor/github.com/Shopify/sarama/create_partitions_request.go @@ -0,0 +1,125 @@ +package sarama + +import "time" + +type CreatePartitionsRequest struct { + TopicPartitions map[string]*TopicPartition + Timeout time.Duration + ValidateOnly bool +} + +func (c *CreatePartitionsRequest) encode(pe packetEncoder) error { + if err := pe.putArrayLength(len(c.TopicPartitions)); err != nil { + return err + } + + for topic, partition := range c.TopicPartitions { + if err := pe.putString(topic); err != nil { + return err + } + if err := partition.encode(pe); err != nil { + return err + } + } + + pe.putInt32(int32(c.Timeout / time.Millisecond)) + + pe.putBool(c.ValidateOnly) + + return nil +} + +func (c *CreatePartitionsRequest) decode(pd packetDecoder, version int16) (err error) { + n, err := pd.getArrayLength() + if err != nil { + return err + } + c.TopicPartitions = make(map[string]*TopicPartition, n) + for i := 0; i < n; i++ { + topic, err := pd.getString() + if err != nil { + return err + } + c.TopicPartitions[topic] = new(TopicPartition) + if err := c.TopicPartitions[topic].decode(pd, version); err != nil { + return err + } + } + + timeout, err := pd.getInt32() + if err != nil { + return err + } + c.Timeout = time.Duration(timeout) * time.Millisecond + + if c.ValidateOnly, err = pd.getBool(); err != nil { + return err + } + + return nil +} + +func (r *CreatePartitionsRequest) key() int16 { + return 37 +} + +func (r *CreatePartitionsRequest) version() int16 { + return 0 +} + +func (r *CreatePartitionsRequest) headerVersion() int16 { + return 1 +} + +func (r *CreatePartitionsRequest) requiredVersion() KafkaVersion { + return V1_0_0_0 +} + +type TopicPartition struct { + Count int32 + Assignment [][]int32 +} + +func (t *TopicPartition) encode(pe packetEncoder) error { + pe.putInt32(t.Count) + + if len(t.Assignment) == 0 { + pe.putInt32(-1) + return nil + } + + if err := pe.putArrayLength(len(t.Assignment)); err != nil { + return err + } + + for _, assign := range t.Assignment { + if err := pe.putInt32Array(assign); err != nil { + return err + } + } + + return nil +} + +func (t *TopicPartition) decode(pd packetDecoder, version int16) (err error) { + if t.Count, err = pd.getInt32(); err != nil { + return err + } + + n, err := pd.getInt32() + if err != nil { + return err + } + if n <= 0 { + return nil + } + t.Assignment = make([][]int32, n) + + for i := 0; i < int(n); i++ { + if t.Assignment[i], err = pd.getInt32Array(); err != nil { + return err + } + } + + return nil +} diff --git a/vendor/github.com/Shopify/sarama/create_partitions_response.go b/vendor/github.com/Shopify/sarama/create_partitions_response.go new file mode 100644 index 000000000000..12ce78857bc5 --- /dev/null +++ b/vendor/github.com/Shopify/sarama/create_partitions_response.go @@ -0,0 +1,109 @@ +package sarama + +import ( + "fmt" + "time" +) + +type CreatePartitionsResponse struct { + ThrottleTime time.Duration + TopicPartitionErrors map[string]*TopicPartitionError +} + +func (c *CreatePartitionsResponse) encode(pe packetEncoder) error { + pe.putInt32(int32(c.ThrottleTime / time.Millisecond)) + if err := pe.putArrayLength(len(c.TopicPartitionErrors)); err != nil { + return err + } + + for topic, partitionError := range c.TopicPartitionErrors { + if err := pe.putString(topic); err != nil { + return err + } + if err := partitionError.encode(pe); err != nil { + return err + } + } + + return nil +} + +func (c *CreatePartitionsResponse) decode(pd packetDecoder, version int16) (err error) { + throttleTime, err := pd.getInt32() + if err != nil { + return err + } + c.ThrottleTime = time.Duration(throttleTime) * time.Millisecond + + n, err := pd.getArrayLength() + if err != nil { + return err + } + + c.TopicPartitionErrors = make(map[string]*TopicPartitionError, n) + for i := 0; i < n; i++ { + topic, err := pd.getString() + if err != nil { + return err + } + c.TopicPartitionErrors[topic] = new(TopicPartitionError) + if err := c.TopicPartitionErrors[topic].decode(pd, version); err != nil { + return err + } + } + + return nil +} + +func (r *CreatePartitionsResponse) key() int16 { + return 37 +} + +func (r *CreatePartitionsResponse) version() int16 { + return 0 +} + +func (r *CreatePartitionsResponse) headerVersion() int16 { + return 0 +} + +func (r *CreatePartitionsResponse) requiredVersion() KafkaVersion { + return V1_0_0_0 +} + +type TopicPartitionError struct { + Err KError + ErrMsg *string +} + +func (t *TopicPartitionError) Error() string { + text := t.Err.Error() + if t.ErrMsg != nil { + text = fmt.Sprintf("%s - %s", text, *t.ErrMsg) + } + return text +} + +func (t *TopicPartitionError) encode(pe packetEncoder) error { + pe.putInt16(int16(t.Err)) + + if err := pe.putNullableString(t.ErrMsg); err != nil { + return err + } + + return nil +} + +func (t *TopicPartitionError) decode(pd packetDecoder, version int16) (err error) { + kerr, err := pd.getInt16() + if err != nil { + return err + } + t.Err = KError(kerr) + + if t.ErrMsg, err = pd.getNullableString(); err != nil { + return err + } + + return nil +} diff --git a/vendor/github.com/Shopify/sarama/create_topics_request.go b/vendor/github.com/Shopify/sarama/create_topics_request.go new file mode 100644 index 000000000000..287acd069b6f --- /dev/null +++ b/vendor/github.com/Shopify/sarama/create_topics_request.go @@ -0,0 +1,178 @@ +package sarama + +import ( + "time" +) + +type CreateTopicsRequest struct { + Version int16 + + TopicDetails map[string]*TopicDetail + Timeout time.Duration + ValidateOnly bool +} + +func (c *CreateTopicsRequest) encode(pe packetEncoder) error { + if err := pe.putArrayLength(len(c.TopicDetails)); err != nil { + return err + } + for topic, detail := range c.TopicDetails { + if err := pe.putString(topic); err != nil { + return err + } + if err := detail.encode(pe); err != nil { + return err + } + } + + pe.putInt32(int32(c.Timeout / time.Millisecond)) + + if c.Version >= 1 { + pe.putBool(c.ValidateOnly) + } + + return nil +} + +func (c *CreateTopicsRequest) decode(pd packetDecoder, version int16) (err error) { + n, err := pd.getArrayLength() + if err != nil { + return err + } + + c.TopicDetails = make(map[string]*TopicDetail, n) + + for i := 0; i < n; i++ { + topic, err := pd.getString() + if err != nil { + return err + } + c.TopicDetails[topic] = new(TopicDetail) + if err = c.TopicDetails[topic].decode(pd, version); err != nil { + return err + } + } + + timeout, err := pd.getInt32() + if err != nil { + return err + } + c.Timeout = time.Duration(timeout) * time.Millisecond + + if version >= 1 { + c.ValidateOnly, err = pd.getBool() + if err != nil { + return err + } + + c.Version = version + } + + return nil +} + +func (c *CreateTopicsRequest) key() int16 { + return 19 +} + +func (c *CreateTopicsRequest) version() int16 { + return c.Version +} + +func (r *CreateTopicsRequest) headerVersion() int16 { + return 1 +} + +func (c *CreateTopicsRequest) requiredVersion() KafkaVersion { + switch c.Version { + case 2: + return V1_0_0_0 + case 1: + return V0_11_0_0 + default: + return V0_10_1_0 + } +} + +type TopicDetail struct { + NumPartitions int32 + ReplicationFactor int16 + ReplicaAssignment map[int32][]int32 + ConfigEntries map[string]*string +} + +func (t *TopicDetail) encode(pe packetEncoder) error { + pe.putInt32(t.NumPartitions) + pe.putInt16(t.ReplicationFactor) + + if err := pe.putArrayLength(len(t.ReplicaAssignment)); err != nil { + return err + } + for partition, assignment := range t.ReplicaAssignment { + pe.putInt32(partition) + if err := pe.putInt32Array(assignment); err != nil { + return err + } + } + + if err := pe.putArrayLength(len(t.ConfigEntries)); err != nil { + return err + } + for configKey, configValue := range t.ConfigEntries { + if err := pe.putString(configKey); err != nil { + return err + } + if err := pe.putNullableString(configValue); err != nil { + return err + } + } + + return nil +} + +func (t *TopicDetail) decode(pd packetDecoder, version int16) (err error) { + if t.NumPartitions, err = pd.getInt32(); err != nil { + return err + } + if t.ReplicationFactor, err = pd.getInt16(); err != nil { + return err + } + + n, err := pd.getArrayLength() + if err != nil { + return err + } + + if n > 0 { + t.ReplicaAssignment = make(map[int32][]int32, n) + for i := 0; i < n; i++ { + replica, err := pd.getInt32() + if err != nil { + return err + } + if t.ReplicaAssignment[replica], err = pd.getInt32Array(); err != nil { + return err + } + } + } + + n, err = pd.getArrayLength() + if err != nil { + return err + } + + if n > 0 { + t.ConfigEntries = make(map[string]*string, n) + for i := 0; i < n; i++ { + configKey, err := pd.getString() + if err != nil { + return err + } + if t.ConfigEntries[configKey], err = pd.getNullableString(); err != nil { + return err + } + } + } + + return nil +} diff --git a/vendor/github.com/Shopify/sarama/create_topics_response.go b/vendor/github.com/Shopify/sarama/create_topics_response.go new file mode 100644 index 000000000000..7e1448a6692d --- /dev/null +++ b/vendor/github.com/Shopify/sarama/create_topics_response.go @@ -0,0 +1,127 @@ +package sarama + +import ( + "fmt" + "time" +) + +type CreateTopicsResponse struct { + Version int16 + ThrottleTime time.Duration + TopicErrors map[string]*TopicError +} + +func (c *CreateTopicsResponse) encode(pe packetEncoder) error { + if c.Version >= 2 { + pe.putInt32(int32(c.ThrottleTime / time.Millisecond)) + } + + if err := pe.putArrayLength(len(c.TopicErrors)); err != nil { + return err + } + for topic, topicError := range c.TopicErrors { + if err := pe.putString(topic); err != nil { + return err + } + if err := topicError.encode(pe, c.Version); err != nil { + return err + } + } + + return nil +} + +func (c *CreateTopicsResponse) decode(pd packetDecoder, version int16) (err error) { + c.Version = version + + if version >= 2 { + throttleTime, err := pd.getInt32() + if err != nil { + return err + } + c.ThrottleTime = time.Duration(throttleTime) * time.Millisecond + } + + n, err := pd.getArrayLength() + if err != nil { + return err + } + + c.TopicErrors = make(map[string]*TopicError, n) + for i := 0; i < n; i++ { + topic, err := pd.getString() + if err != nil { + return err + } + c.TopicErrors[topic] = new(TopicError) + if err := c.TopicErrors[topic].decode(pd, version); err != nil { + return err + } + } + + return nil +} + +func (c *CreateTopicsResponse) key() int16 { + return 19 +} + +func (c *CreateTopicsResponse) version() int16 { + return c.Version +} + +func (c *CreateTopicsResponse) headerVersion() int16 { + return 0 +} + +func (c *CreateTopicsResponse) requiredVersion() KafkaVersion { + switch c.Version { + case 2: + return V1_0_0_0 + case 1: + return V0_11_0_0 + default: + return V0_10_1_0 + } +} + +type TopicError struct { + Err KError + ErrMsg *string +} + +func (t *TopicError) Error() string { + text := t.Err.Error() + if t.ErrMsg != nil { + text = fmt.Sprintf("%s - %s", text, *t.ErrMsg) + } + return text +} + +func (t *TopicError) encode(pe packetEncoder, version int16) error { + pe.putInt16(int16(t.Err)) + + if version >= 1 { + if err := pe.putNullableString(t.ErrMsg); err != nil { + return err + } + } + + return nil +} + +func (t *TopicError) decode(pd packetDecoder, version int16) (err error) { + kErr, err := pd.getInt16() + if err != nil { + return err + } + t.Err = KError(kErr) + + if version >= 1 { + if t.ErrMsg, err = pd.getNullableString(); err != nil { + return err + } + } + + return nil +} diff --git a/vendor/github.com/Shopify/sarama/decompress.go b/vendor/github.com/Shopify/sarama/decompress.go new file mode 100644 index 000000000000..5565e36cfa29 --- /dev/null +++ b/vendor/github.com/Shopify/sarama/decompress.go @@ -0,0 +1,61 @@ +package sarama + +import ( + "bytes" + "compress/gzip" + "fmt" + "io" + "sync" + + snappy "github.com/eapache/go-xerial-snappy" + "github.com/pierrec/lz4" +) + +var ( + lz4ReaderPool = sync.Pool{ + New: func() interface{} { + return lz4.NewReader(nil) + }, + } + + gzipReaderPool sync.Pool +) + +func decompress(cc CompressionCodec, data []byte) ([]byte, error) { + switch cc { + case CompressionNone: + return data, nil + case CompressionGZIP: + var err error + reader, ok := gzipReaderPool.Get().(*gzip.Reader) + if !ok { + reader, err = gzip.NewReader(bytes.NewReader(data)) + } else { + err = reader.Reset(bytes.NewReader(data)) + } + + if err != nil { + return nil, err + } + + defer gzipReaderPool.Put(reader) + + return io.ReadAll(reader) + case CompressionSnappy: + return snappy.Decode(data) + case CompressionLZ4: + reader, ok := lz4ReaderPool.Get().(*lz4.Reader) + if !ok { + reader = lz4.NewReader(bytes.NewReader(data)) + } else { + reader.Reset(bytes.NewReader(data)) + } + defer lz4ReaderPool.Put(reader) + + return io.ReadAll(reader) + case CompressionZSTD: + return zstdDecompress(nil, data) + default: + return nil, PacketDecodingError{fmt.Sprintf("invalid compression specified (%d)", cc)} + } +} diff --git a/vendor/github.com/Shopify/sarama/delete_groups_request.go b/vendor/github.com/Shopify/sarama/delete_groups_request.go new file mode 100644 index 000000000000..4ac8bbee4cb1 --- /dev/null +++ b/vendor/github.com/Shopify/sarama/delete_groups_request.go @@ -0,0 +1,34 @@ +package sarama + +type DeleteGroupsRequest struct { + Groups []string +} + +func (r *DeleteGroupsRequest) encode(pe packetEncoder) error { + return pe.putStringArray(r.Groups) +} + +func (r *DeleteGroupsRequest) decode(pd packetDecoder, version int16) (err error) { + r.Groups, err = pd.getStringArray() + return +} + +func (r *DeleteGroupsRequest) key() int16 { + return 42 +} + +func (r *DeleteGroupsRequest) version() int16 { + return 0 +} + +func (r *DeleteGroupsRequest) headerVersion() int16 { + return 1 +} + +func (r *DeleteGroupsRequest) requiredVersion() KafkaVersion { + return V1_1_0_0 +} + +func (r *DeleteGroupsRequest) AddGroup(group string) { + r.Groups = append(r.Groups, group) +} diff --git a/vendor/github.com/Shopify/sarama/delete_groups_response.go b/vendor/github.com/Shopify/sarama/delete_groups_response.go new file mode 100644 index 000000000000..5e7b1ed3681e --- /dev/null +++ b/vendor/github.com/Shopify/sarama/delete_groups_response.go @@ -0,0 +1,74 @@ +package sarama + +import ( + "time" +) + +type DeleteGroupsResponse struct { + ThrottleTime time.Duration + GroupErrorCodes map[string]KError +} + +func (r *DeleteGroupsResponse) encode(pe packetEncoder) error { + pe.putInt32(int32(r.ThrottleTime / time.Millisecond)) + + if err := pe.putArrayLength(len(r.GroupErrorCodes)); err != nil { + return err + } + for groupID, errorCode := range r.GroupErrorCodes { + if err := pe.putString(groupID); err != nil { + return err + } + pe.putInt16(int16(errorCode)) + } + + return nil +} + +func (r *DeleteGroupsResponse) decode(pd packetDecoder, version int16) error { + throttleTime, err := pd.getInt32() + if err != nil { + return err + } + r.ThrottleTime = time.Duration(throttleTime) * time.Millisecond + + n, err := pd.getArrayLength() + if err != nil { + return err + } + if n == 0 { + return nil + } + + r.GroupErrorCodes = make(map[string]KError, n) + for i := 0; i < n; i++ { + groupID, err := pd.getString() + if err != nil { + return err + } + errorCode, err := pd.getInt16() + if err != nil { + return err + } + + r.GroupErrorCodes[groupID] = KError(errorCode) + } + + return nil +} + +func (r *DeleteGroupsResponse) key() int16 { + return 42 +} + +func (r *DeleteGroupsResponse) version() int16 { + return 0 +} + +func (r *DeleteGroupsResponse) headerVersion() int16 { + return 0 +} + +func (r *DeleteGroupsResponse) requiredVersion() KafkaVersion { + return V1_1_0_0 +} diff --git a/vendor/github.com/Shopify/sarama/delete_offsets_request.go b/vendor/github.com/Shopify/sarama/delete_offsets_request.go new file mode 100644 index 000000000000..339c7857cac8 --- /dev/null +++ b/vendor/github.com/Shopify/sarama/delete_offsets_request.go @@ -0,0 +1,92 @@ +package sarama + +type DeleteOffsetsRequest struct { + Group string + partitions map[string][]int32 +} + +func (r *DeleteOffsetsRequest) encode(pe packetEncoder) (err error) { + err = pe.putString(r.Group) + if err != nil { + return err + } + + if r.partitions == nil { + pe.putInt32(0) + } else { + if err = pe.putArrayLength(len(r.partitions)); err != nil { + return err + } + } + for topic, partitions := range r.partitions { + err = pe.putString(topic) + if err != nil { + return err + } + err = pe.putInt32Array(partitions) + if err != nil { + return err + } + } + return +} + +func (r *DeleteOffsetsRequest) decode(pd packetDecoder, version int16) (err error) { + r.Group, err = pd.getString() + if err != nil { + return err + } + var partitionCount int + + partitionCount, err = pd.getArrayLength() + if err != nil { + return err + } + + if (partitionCount == 0 && version < 2) || partitionCount < 0 { + return nil + } + + r.partitions = make(map[string][]int32, partitionCount) + for i := 0; i < partitionCount; i++ { + var topic string + topic, err = pd.getString() + if err != nil { + return err + } + + var partitions []int32 + partitions, err = pd.getInt32Array() + if err != nil { + return err + } + + r.partitions[topic] = partitions + } + + return nil +} + +func (r *DeleteOffsetsRequest) key() int16 { + return 47 +} + +func (r *DeleteOffsetsRequest) version() int16 { + return 0 +} + +func (r *DeleteOffsetsRequest) headerVersion() int16 { + return 1 +} + +func (r *DeleteOffsetsRequest) requiredVersion() KafkaVersion { + return V2_4_0_0 +} + +func (r *DeleteOffsetsRequest) AddPartition(topic string, partitionID int32) { + if r.partitions == nil { + r.partitions = make(map[string][]int32) + } + + r.partitions[topic] = append(r.partitions[topic], partitionID) +} diff --git a/vendor/github.com/Shopify/sarama/delete_offsets_response.go b/vendor/github.com/Shopify/sarama/delete_offsets_response.go new file mode 100644 index 000000000000..00c3eaf906ed --- /dev/null +++ b/vendor/github.com/Shopify/sarama/delete_offsets_response.go @@ -0,0 +1,112 @@ +package sarama + +import ( + "time" +) + +type DeleteOffsetsResponse struct { + //The top-level error code, or 0 if there was no error. + ErrorCode KError + ThrottleTime time.Duration + //The responses for each partition of the topics. + Errors map[string]map[int32]KError +} + +func (r *DeleteOffsetsResponse) AddError(topic string, partition int32, errorCode KError) { + if r.Errors == nil { + r.Errors = make(map[string]map[int32]KError) + } + partitions := r.Errors[topic] + if partitions == nil { + partitions = make(map[int32]KError) + r.Errors[topic] = partitions + } + partitions[partition] = errorCode +} + +func (r *DeleteOffsetsResponse) encode(pe packetEncoder) error { + pe.putInt16(int16(r.ErrorCode)) + pe.putInt32(int32(r.ThrottleTime / time.Millisecond)) + + if err := pe.putArrayLength(len(r.Errors)); err != nil { + return err + } + for topic, partitions := range r.Errors { + if err := pe.putString(topic); err != nil { + return err + } + if err := pe.putArrayLength(len(partitions)); err != nil { + return err + } + for partition, errorCode := range partitions { + pe.putInt32(partition) + pe.putInt16(int16(errorCode)) + } + } + return nil +} + +func (r *DeleteOffsetsResponse) decode(pd packetDecoder, version int16) error { + tmpErr, err := pd.getInt16() + if err != nil { + return err + } + r.ErrorCode = KError(tmpErr) + + throttleTime, err := pd.getInt32() + if err != nil { + return err + } + r.ThrottleTime = time.Duration(throttleTime) * time.Millisecond + + numTopics, err := pd.getArrayLength() + if err != nil || numTopics == 0 { + return err + } + + r.Errors = make(map[string]map[int32]KError, numTopics) + for i := 0; i < numTopics; i++ { + name, err := pd.getString() + if err != nil { + return err + } + + numErrors, err := pd.getArrayLength() + if err != nil { + return err + } + + r.Errors[name] = make(map[int32]KError, numErrors) + + for j := 0; j < numErrors; j++ { + id, err := pd.getInt32() + if err != nil { + return err + } + + tmp, err := pd.getInt16() + if err != nil { + return err + } + r.Errors[name][id] = KError(tmp) + } + } + + return nil +} + +func (r *DeleteOffsetsResponse) key() int16 { + return 47 +} + +func (r *DeleteOffsetsResponse) version() int16 { + return 0 +} + +func (r *DeleteOffsetsResponse) headerVersion() int16 { + return 0 +} + +func (r *DeleteOffsetsResponse) requiredVersion() KafkaVersion { + return V2_4_0_0 +} diff --git a/vendor/github.com/Shopify/sarama/delete_records_request.go b/vendor/github.com/Shopify/sarama/delete_records_request.go new file mode 100644 index 000000000000..dc106b17d62b --- /dev/null +++ b/vendor/github.com/Shopify/sarama/delete_records_request.go @@ -0,0 +1,130 @@ +package sarama + +import ( + "sort" + "time" +) + +// request message format is: +// [topic] timeout(int32) +// where topic is: +// name(string) [partition] +// where partition is: +// id(int32) offset(int64) + +type DeleteRecordsRequest struct { + Topics map[string]*DeleteRecordsRequestTopic + Timeout time.Duration +} + +func (d *DeleteRecordsRequest) encode(pe packetEncoder) error { + if err := pe.putArrayLength(len(d.Topics)); err != nil { + return err + } + keys := make([]string, 0, len(d.Topics)) + for topic := range d.Topics { + keys = append(keys, topic) + } + sort.Strings(keys) + for _, topic := range keys { + if err := pe.putString(topic); err != nil { + return err + } + if err := d.Topics[topic].encode(pe); err != nil { + return err + } + } + pe.putInt32(int32(d.Timeout / time.Millisecond)) + + return nil +} + +func (d *DeleteRecordsRequest) decode(pd packetDecoder, version int16) error { + n, err := pd.getArrayLength() + if err != nil { + return err + } + + if n > 0 { + d.Topics = make(map[string]*DeleteRecordsRequestTopic, n) + for i := 0; i < n; i++ { + topic, err := pd.getString() + if err != nil { + return err + } + details := new(DeleteRecordsRequestTopic) + if err = details.decode(pd, version); err != nil { + return err + } + d.Topics[topic] = details + } + } + + timeout, err := pd.getInt32() + if err != nil { + return err + } + d.Timeout = time.Duration(timeout) * time.Millisecond + + return nil +} + +func (d *DeleteRecordsRequest) key() int16 { + return 21 +} + +func (d *DeleteRecordsRequest) version() int16 { + return 0 +} + +func (d *DeleteRecordsRequest) headerVersion() int16 { + return 1 +} + +func (d *DeleteRecordsRequest) requiredVersion() KafkaVersion { + return V0_11_0_0 +} + +type DeleteRecordsRequestTopic struct { + PartitionOffsets map[int32]int64 // partition => offset +} + +func (t *DeleteRecordsRequestTopic) encode(pe packetEncoder) error { + if err := pe.putArrayLength(len(t.PartitionOffsets)); err != nil { + return err + } + keys := make([]int32, 0, len(t.PartitionOffsets)) + for partition := range t.PartitionOffsets { + keys = append(keys, partition) + } + sort.Slice(keys, func(i, j int) bool { return keys[i] < keys[j] }) + for _, partition := range keys { + pe.putInt32(partition) + pe.putInt64(t.PartitionOffsets[partition]) + } + return nil +} + +func (t *DeleteRecordsRequestTopic) decode(pd packetDecoder, version int16) error { + n, err := pd.getArrayLength() + if err != nil { + return err + } + + if n > 0 { + t.PartitionOffsets = make(map[int32]int64, n) + for i := 0; i < n; i++ { + partition, err := pd.getInt32() + if err != nil { + return err + } + offset, err := pd.getInt64() + if err != nil { + return err + } + t.PartitionOffsets[partition] = offset + } + } + + return nil +} diff --git a/vendor/github.com/Shopify/sarama/delete_records_response.go b/vendor/github.com/Shopify/sarama/delete_records_response.go new file mode 100644 index 000000000000..d530b4c7e912 --- /dev/null +++ b/vendor/github.com/Shopify/sarama/delete_records_response.go @@ -0,0 +1,162 @@ +package sarama + +import ( + "sort" + "time" +) + +// response message format is: +// throttleMs(int32) [topic] +// where topic is: +// name(string) [partition] +// where partition is: +// id(int32) low_watermark(int64) error_code(int16) + +type DeleteRecordsResponse struct { + Version int16 + ThrottleTime time.Duration + Topics map[string]*DeleteRecordsResponseTopic +} + +func (d *DeleteRecordsResponse) encode(pe packetEncoder) error { + pe.putInt32(int32(d.ThrottleTime / time.Millisecond)) + + if err := pe.putArrayLength(len(d.Topics)); err != nil { + return err + } + keys := make([]string, 0, len(d.Topics)) + for topic := range d.Topics { + keys = append(keys, topic) + } + sort.Strings(keys) + for _, topic := range keys { + if err := pe.putString(topic); err != nil { + return err + } + if err := d.Topics[topic].encode(pe); err != nil { + return err + } + } + return nil +} + +func (d *DeleteRecordsResponse) decode(pd packetDecoder, version int16) error { + d.Version = version + + throttleTime, err := pd.getInt32() + if err != nil { + return err + } + d.ThrottleTime = time.Duration(throttleTime) * time.Millisecond + + n, err := pd.getArrayLength() + if err != nil { + return err + } + + if n > 0 { + d.Topics = make(map[string]*DeleteRecordsResponseTopic, n) + for i := 0; i < n; i++ { + topic, err := pd.getString() + if err != nil { + return err + } + details := new(DeleteRecordsResponseTopic) + if err = details.decode(pd, version); err != nil { + return err + } + d.Topics[topic] = details + } + } + + return nil +} + +func (d *DeleteRecordsResponse) key() int16 { + return 21 +} + +func (d *DeleteRecordsResponse) version() int16 { + return 0 +} + +func (d *DeleteRecordsResponse) headerVersion() int16 { + return 0 +} + +func (d *DeleteRecordsResponse) requiredVersion() KafkaVersion { + return V0_11_0_0 +} + +type DeleteRecordsResponseTopic struct { + Partitions map[int32]*DeleteRecordsResponsePartition +} + +func (t *DeleteRecordsResponseTopic) encode(pe packetEncoder) error { + if err := pe.putArrayLength(len(t.Partitions)); err != nil { + return err + } + keys := make([]int32, 0, len(t.Partitions)) + for partition := range t.Partitions { + keys = append(keys, partition) + } + sort.Slice(keys, func(i, j int) bool { return keys[i] < keys[j] }) + for _, partition := range keys { + pe.putInt32(partition) + if err := t.Partitions[partition].encode(pe); err != nil { + return err + } + } + return nil +} + +func (t *DeleteRecordsResponseTopic) decode(pd packetDecoder, version int16) error { + n, err := pd.getArrayLength() + if err != nil { + return err + } + + if n > 0 { + t.Partitions = make(map[int32]*DeleteRecordsResponsePartition, n) + for i := 0; i < n; i++ { + partition, err := pd.getInt32() + if err != nil { + return err + } + details := new(DeleteRecordsResponsePartition) + if err = details.decode(pd, version); err != nil { + return err + } + t.Partitions[partition] = details + } + } + + return nil +} + +type DeleteRecordsResponsePartition struct { + LowWatermark int64 + Err KError +} + +func (t *DeleteRecordsResponsePartition) encode(pe packetEncoder) error { + pe.putInt64(t.LowWatermark) + pe.putInt16(int16(t.Err)) + return nil +} + +func (t *DeleteRecordsResponsePartition) decode(pd packetDecoder, version int16) error { + lowWatermark, err := pd.getInt64() + if err != nil { + return err + } + t.LowWatermark = lowWatermark + + kErr, err := pd.getInt16() + if err != nil { + return err + } + t.Err = KError(kErr) + + return nil +} diff --git a/vendor/github.com/Shopify/sarama/delete_topics_request.go b/vendor/github.com/Shopify/sarama/delete_topics_request.go new file mode 100644 index 000000000000..ba6780a8e39c --- /dev/null +++ b/vendor/github.com/Shopify/sarama/delete_topics_request.go @@ -0,0 +1,52 @@ +package sarama + +import "time" + +type DeleteTopicsRequest struct { + Version int16 + Topics []string + Timeout time.Duration +} + +func (d *DeleteTopicsRequest) encode(pe packetEncoder) error { + if err := pe.putStringArray(d.Topics); err != nil { + return err + } + pe.putInt32(int32(d.Timeout / time.Millisecond)) + + return nil +} + +func (d *DeleteTopicsRequest) decode(pd packetDecoder, version int16) (err error) { + if d.Topics, err = pd.getStringArray(); err != nil { + return err + } + timeout, err := pd.getInt32() + if err != nil { + return err + } + d.Timeout = time.Duration(timeout) * time.Millisecond + d.Version = version + return nil +} + +func (d *DeleteTopicsRequest) key() int16 { + return 20 +} + +func (d *DeleteTopicsRequest) version() int16 { + return d.Version +} + +func (d *DeleteTopicsRequest) headerVersion() int16 { + return 1 +} + +func (d *DeleteTopicsRequest) requiredVersion() KafkaVersion { + switch d.Version { + case 1: + return V0_11_0_0 + default: + return V0_10_1_0 + } +} diff --git a/vendor/github.com/Shopify/sarama/delete_topics_response.go b/vendor/github.com/Shopify/sarama/delete_topics_response.go new file mode 100644 index 000000000000..733961a89a03 --- /dev/null +++ b/vendor/github.com/Shopify/sarama/delete_topics_response.go @@ -0,0 +1,82 @@ +package sarama + +import "time" + +type DeleteTopicsResponse struct { + Version int16 + ThrottleTime time.Duration + TopicErrorCodes map[string]KError +} + +func (d *DeleteTopicsResponse) encode(pe packetEncoder) error { + if d.Version >= 1 { + pe.putInt32(int32(d.ThrottleTime / time.Millisecond)) + } + + if err := pe.putArrayLength(len(d.TopicErrorCodes)); err != nil { + return err + } + for topic, errorCode := range d.TopicErrorCodes { + if err := pe.putString(topic); err != nil { + return err + } + pe.putInt16(int16(errorCode)) + } + + return nil +} + +func (d *DeleteTopicsResponse) decode(pd packetDecoder, version int16) (err error) { + if version >= 1 { + throttleTime, err := pd.getInt32() + if err != nil { + return err + } + d.ThrottleTime = time.Duration(throttleTime) * time.Millisecond + + d.Version = version + } + + n, err := pd.getArrayLength() + if err != nil { + return err + } + + d.TopicErrorCodes = make(map[string]KError, n) + + for i := 0; i < n; i++ { + topic, err := pd.getString() + if err != nil { + return err + } + errorCode, err := pd.getInt16() + if err != nil { + return err + } + + d.TopicErrorCodes[topic] = KError(errorCode) + } + + return nil +} + +func (d *DeleteTopicsResponse) key() int16 { + return 20 +} + +func (d *DeleteTopicsResponse) version() int16 { + return d.Version +} + +func (d *DeleteTopicsResponse) headerVersion() int16 { + return 0 +} + +func (d *DeleteTopicsResponse) requiredVersion() KafkaVersion { + switch d.Version { + case 1: + return V0_11_0_0 + default: + return V0_10_1_0 + } +} diff --git a/vendor/github.com/Shopify/sarama/describe_client_quotas_request.go b/vendor/github.com/Shopify/sarama/describe_client_quotas_request.go new file mode 100644 index 000000000000..17a82051c512 --- /dev/null +++ b/vendor/github.com/Shopify/sarama/describe_client_quotas_request.go @@ -0,0 +1,141 @@ +package sarama + +// DescribeClientQuotas Request (Version: 0) => [components] strict +// components => entity_type match_type match +// entity_type => STRING +// match_type => INT8 +// match => NULLABLE_STRING +// strict => BOOLEAN + +// A filter to be applied to matching client quotas. +// Components: the components to filter on +// Strict: whether the filter only includes specified components +type DescribeClientQuotasRequest struct { + Components []QuotaFilterComponent + Strict bool +} + +// Describe a component for applying a client quota filter. +// EntityType: the entity type the filter component applies to ("user", "client-id", "ip") +// MatchType: the match type of the filter component (any, exact, default) +// Match: the name that's matched exactly (used when MatchType is QuotaMatchExact) +type QuotaFilterComponent struct { + EntityType QuotaEntityType + MatchType QuotaMatchType + Match string +} + +func (d *DescribeClientQuotasRequest) encode(pe packetEncoder) error { + // Components + if err := pe.putArrayLength(len(d.Components)); err != nil { + return err + } + for _, c := range d.Components { + if err := c.encode(pe); err != nil { + return err + } + } + + // Strict + pe.putBool(d.Strict) + + return nil +} + +func (d *DescribeClientQuotasRequest) decode(pd packetDecoder, version int16) error { + // Components + componentCount, err := pd.getArrayLength() + if err != nil { + return err + } + if componentCount > 0 { + d.Components = make([]QuotaFilterComponent, componentCount) + for i := range d.Components { + c := QuotaFilterComponent{} + if err = c.decode(pd, version); err != nil { + return err + } + d.Components[i] = c + } + } else { + d.Components = []QuotaFilterComponent{} + } + + // Strict + strict, err := pd.getBool() + if err != nil { + return err + } + d.Strict = strict + + return nil +} + +func (d *QuotaFilterComponent) encode(pe packetEncoder) error { + // EntityType + if err := pe.putString(string(d.EntityType)); err != nil { + return err + } + + // MatchType + pe.putInt8(int8(d.MatchType)) + + // Match + if d.MatchType == QuotaMatchAny { + if err := pe.putNullableString(nil); err != nil { + return err + } + } else if d.MatchType == QuotaMatchDefault { + if err := pe.putString(""); err != nil { + return err + } + } else { + if err := pe.putString(d.Match); err != nil { + return err + } + } + + return nil +} + +func (d *QuotaFilterComponent) decode(pd packetDecoder, version int16) error { + // EntityType + entityType, err := pd.getString() + if err != nil { + return err + } + d.EntityType = QuotaEntityType(entityType) + + // MatchType + matchType, err := pd.getInt8() + if err != nil { + return err + } + d.MatchType = QuotaMatchType(matchType) + + // Match + match, err := pd.getNullableString() + if err != nil { + return err + } + if match != nil { + d.Match = *match + } + return nil +} + +func (d *DescribeClientQuotasRequest) key() int16 { + return 48 +} + +func (d *DescribeClientQuotasRequest) version() int16 { + return 0 +} + +func (d *DescribeClientQuotasRequest) headerVersion() int16 { + return 1 +} + +func (d *DescribeClientQuotasRequest) requiredVersion() KafkaVersion { + return V2_6_0_0 +} diff --git a/vendor/github.com/Shopify/sarama/describe_client_quotas_response.go b/vendor/github.com/Shopify/sarama/describe_client_quotas_response.go new file mode 100644 index 000000000000..555da0c485d2 --- /dev/null +++ b/vendor/github.com/Shopify/sarama/describe_client_quotas_response.go @@ -0,0 +1,235 @@ +package sarama + +import ( + "time" +) + +// DescribeClientQuotas Response (Version: 0) => throttle_time_ms error_code error_message [entries] +// throttle_time_ms => INT32 +// error_code => INT16 +// error_message => NULLABLE_STRING +// entries => [entity] [values] +// entity => entity_type entity_name +// entity_type => STRING +// entity_name => NULLABLE_STRING +// values => key value +// key => STRING +// value => FLOAT64 + +type DescribeClientQuotasResponse struct { + ThrottleTime time.Duration // The duration in milliseconds for which the request was throttled due to a quota violation, or zero if the request did not violate any quota. + ErrorCode KError // The error code, or `0` if the quota description succeeded. + ErrorMsg *string // The error message, or `null` if the quota description succeeded. + Entries []DescribeClientQuotasEntry // A result entry. +} + +type DescribeClientQuotasEntry struct { + Entity []QuotaEntityComponent // The quota entity description. + Values map[string]float64 // The quota values for the entity. +} + +type QuotaEntityComponent struct { + EntityType QuotaEntityType + MatchType QuotaMatchType + Name string +} + +func (d *DescribeClientQuotasResponse) encode(pe packetEncoder) error { + // ThrottleTime + pe.putInt32(int32(d.ThrottleTime / time.Millisecond)) + + // ErrorCode + pe.putInt16(int16(d.ErrorCode)) + + // ErrorMsg + if err := pe.putNullableString(d.ErrorMsg); err != nil { + return err + } + + // Entries + if err := pe.putArrayLength(len(d.Entries)); err != nil { + return err + } + for _, e := range d.Entries { + if err := e.encode(pe); err != nil { + return err + } + } + + return nil +} + +func (d *DescribeClientQuotasResponse) decode(pd packetDecoder, version int16) error { + // ThrottleTime + throttleTime, err := pd.getInt32() + if err != nil { + return err + } + d.ThrottleTime = time.Duration(throttleTime) * time.Millisecond + + // ErrorCode + errCode, err := pd.getInt16() + if err != nil { + return err + } + d.ErrorCode = KError(errCode) + + // ErrorMsg + errMsg, err := pd.getNullableString() + if err != nil { + return err + } + d.ErrorMsg = errMsg + + // Entries + entryCount, err := pd.getArrayLength() + if err != nil { + return err + } + if entryCount > 0 { + d.Entries = make([]DescribeClientQuotasEntry, entryCount) + for i := range d.Entries { + e := DescribeClientQuotasEntry{} + if err = e.decode(pd, version); err != nil { + return err + } + d.Entries[i] = e + } + } else { + d.Entries = []DescribeClientQuotasEntry{} + } + + return nil +} + +func (d *DescribeClientQuotasEntry) encode(pe packetEncoder) error { + // Entity + if err := pe.putArrayLength(len(d.Entity)); err != nil { + return err + } + for _, e := range d.Entity { + if err := e.encode(pe); err != nil { + return err + } + } + + // Values + if err := pe.putArrayLength(len(d.Values)); err != nil { + return err + } + for key, value := range d.Values { + // key + if err := pe.putString(key); err != nil { + return err + } + // value + pe.putFloat64(value) + } + + return nil +} + +func (d *DescribeClientQuotasEntry) decode(pd packetDecoder, version int16) error { + // Entity + componentCount, err := pd.getArrayLength() + if err != nil { + return err + } + if componentCount > 0 { + d.Entity = make([]QuotaEntityComponent, componentCount) + for i := 0; i < componentCount; i++ { + component := QuotaEntityComponent{} + if err := component.decode(pd, version); err != nil { + return err + } + d.Entity[i] = component + } + } else { + d.Entity = []QuotaEntityComponent{} + } + + // Values + valueCount, err := pd.getArrayLength() + if err != nil { + return err + } + if valueCount > 0 { + d.Values = make(map[string]float64, valueCount) + for i := 0; i < valueCount; i++ { + // key + key, err := pd.getString() + if err != nil { + return err + } + // value + value, err := pd.getFloat64() + if err != nil { + return err + } + d.Values[key] = value + } + } else { + d.Values = map[string]float64{} + } + + return nil +} + +func (c *QuotaEntityComponent) encode(pe packetEncoder) error { + // entity_type + if err := pe.putString(string(c.EntityType)); err != nil { + return err + } + // entity_name + if c.MatchType == QuotaMatchDefault { + if err := pe.putNullableString(nil); err != nil { + return err + } + } else { + if err := pe.putString(c.Name); err != nil { + return err + } + } + + return nil +} + +func (c *QuotaEntityComponent) decode(pd packetDecoder, version int16) error { + // entity_type + entityType, err := pd.getString() + if err != nil { + return err + } + c.EntityType = QuotaEntityType(entityType) + + // entity_name + entityName, err := pd.getNullableString() + if err != nil { + return err + } + + if entityName == nil { + c.MatchType = QuotaMatchDefault + } else { + c.MatchType = QuotaMatchExact + c.Name = *entityName + } + + return nil +} + +func (d *DescribeClientQuotasResponse) key() int16 { + return 48 +} + +func (d *DescribeClientQuotasResponse) version() int16 { + return 0 +} + +func (d *DescribeClientQuotasResponse) headerVersion() int16 { + return 0 +} + +func (d *DescribeClientQuotasResponse) requiredVersion() KafkaVersion { + return V2_6_0_0 +} diff --git a/vendor/github.com/Shopify/sarama/describe_configs_request.go b/vendor/github.com/Shopify/sarama/describe_configs_request.go new file mode 100644 index 000000000000..4c34880318c8 --- /dev/null +++ b/vendor/github.com/Shopify/sarama/describe_configs_request.go @@ -0,0 +1,115 @@ +package sarama + +type DescribeConfigsRequest struct { + Version int16 + Resources []*ConfigResource + IncludeSynonyms bool +} + +type ConfigResource struct { + Type ConfigResourceType + Name string + ConfigNames []string +} + +func (r *DescribeConfigsRequest) encode(pe packetEncoder) error { + if err := pe.putArrayLength(len(r.Resources)); err != nil { + return err + } + + for _, c := range r.Resources { + pe.putInt8(int8(c.Type)) + if err := pe.putString(c.Name); err != nil { + return err + } + + if len(c.ConfigNames) == 0 { + pe.putInt32(-1) + continue + } + if err := pe.putStringArray(c.ConfigNames); err != nil { + return err + } + } + + if r.Version >= 1 { + pe.putBool(r.IncludeSynonyms) + } + + return nil +} + +func (r *DescribeConfigsRequest) decode(pd packetDecoder, version int16) (err error) { + n, err := pd.getArrayLength() + if err != nil { + return err + } + + r.Resources = make([]*ConfigResource, n) + + for i := 0; i < n; i++ { + r.Resources[i] = &ConfigResource{} + t, err := pd.getInt8() + if err != nil { + return err + } + r.Resources[i].Type = ConfigResourceType(t) + name, err := pd.getString() + if err != nil { + return err + } + r.Resources[i].Name = name + + confLength, err := pd.getArrayLength() + if err != nil { + return err + } + + if confLength == -1 { + continue + } + + cfnames := make([]string, confLength) + for i := 0; i < confLength; i++ { + s, err := pd.getString() + if err != nil { + return err + } + cfnames[i] = s + } + r.Resources[i].ConfigNames = cfnames + } + r.Version = version + if r.Version >= 1 { + b, err := pd.getBool() + if err != nil { + return err + } + r.IncludeSynonyms = b + } + + return nil +} + +func (r *DescribeConfigsRequest) key() int16 { + return 32 +} + +func (r *DescribeConfigsRequest) version() int16 { + return r.Version +} + +func (r *DescribeConfigsRequest) headerVersion() int16 { + return 1 +} + +func (r *DescribeConfigsRequest) requiredVersion() KafkaVersion { + switch r.Version { + case 1: + return V1_1_0_0 + case 2: + return V2_0_0_0 + default: + return V0_11_0_0 + } +} diff --git a/vendor/github.com/Shopify/sarama/describe_configs_response.go b/vendor/github.com/Shopify/sarama/describe_configs_response.go new file mode 100644 index 000000000000..4968f4854a62 --- /dev/null +++ b/vendor/github.com/Shopify/sarama/describe_configs_response.go @@ -0,0 +1,327 @@ +package sarama + +import ( + "fmt" + "time" +) + +type ConfigSource int8 + +func (s ConfigSource) String() string { + switch s { + case SourceUnknown: + return "Unknown" + case SourceTopic: + return "Topic" + case SourceDynamicBroker: + return "DynamicBroker" + case SourceDynamicDefaultBroker: + return "DynamicDefaultBroker" + case SourceStaticBroker: + return "StaticBroker" + case SourceDefault: + return "Default" + } + return fmt.Sprintf("Source Invalid: %d", int(s)) +} + +const ( + SourceUnknown ConfigSource = iota + SourceTopic + SourceDynamicBroker + SourceDynamicDefaultBroker + SourceStaticBroker + SourceDefault +) + +type DescribeConfigsResponse struct { + Version int16 + ThrottleTime time.Duration + Resources []*ResourceResponse +} + +type ResourceResponse struct { + ErrorCode int16 + ErrorMsg string + Type ConfigResourceType + Name string + Configs []*ConfigEntry +} + +type ConfigEntry struct { + Name string + Value string + ReadOnly bool + Default bool + Source ConfigSource + Sensitive bool + Synonyms []*ConfigSynonym +} + +type ConfigSynonym struct { + ConfigName string + ConfigValue string + Source ConfigSource +} + +func (r *DescribeConfigsResponse) encode(pe packetEncoder) (err error) { + pe.putInt32(int32(r.ThrottleTime / time.Millisecond)) + if err = pe.putArrayLength(len(r.Resources)); err != nil { + return err + } + + for _, c := range r.Resources { + if err = c.encode(pe, r.Version); err != nil { + return err + } + } + + return nil +} + +func (r *DescribeConfigsResponse) decode(pd packetDecoder, version int16) (err error) { + r.Version = version + throttleTime, err := pd.getInt32() + if err != nil { + return err + } + r.ThrottleTime = time.Duration(throttleTime) * time.Millisecond + + n, err := pd.getArrayLength() + if err != nil { + return err + } + + r.Resources = make([]*ResourceResponse, n) + for i := 0; i < n; i++ { + rr := &ResourceResponse{} + if err := rr.decode(pd, version); err != nil { + return err + } + r.Resources[i] = rr + } + + return nil +} + +func (r *DescribeConfigsResponse) key() int16 { + return 32 +} + +func (r *DescribeConfigsResponse) version() int16 { + return r.Version +} + +func (r *DescribeConfigsResponse) headerVersion() int16 { + return 0 +} + +func (r *DescribeConfigsResponse) requiredVersion() KafkaVersion { + switch r.Version { + case 1: + return V1_0_0_0 + case 2: + return V2_0_0_0 + default: + return V0_11_0_0 + } +} + +func (r *ResourceResponse) encode(pe packetEncoder, version int16) (err error) { + pe.putInt16(r.ErrorCode) + + if err = pe.putString(r.ErrorMsg); err != nil { + return err + } + + pe.putInt8(int8(r.Type)) + + if err = pe.putString(r.Name); err != nil { + return err + } + + if err = pe.putArrayLength(len(r.Configs)); err != nil { + return err + } + + for _, c := range r.Configs { + if err = c.encode(pe, version); err != nil { + return err + } + } + return nil +} + +func (r *ResourceResponse) decode(pd packetDecoder, version int16) (err error) { + ec, err := pd.getInt16() + if err != nil { + return err + } + r.ErrorCode = ec + + em, err := pd.getString() + if err != nil { + return err + } + r.ErrorMsg = em + + t, err := pd.getInt8() + if err != nil { + return err + } + r.Type = ConfigResourceType(t) + + name, err := pd.getString() + if err != nil { + return err + } + r.Name = name + + n, err := pd.getArrayLength() + if err != nil { + return err + } + + r.Configs = make([]*ConfigEntry, n) + for i := 0; i < n; i++ { + c := &ConfigEntry{} + if err := c.decode(pd, version); err != nil { + return err + } + r.Configs[i] = c + } + return nil +} + +func (r *ConfigEntry) encode(pe packetEncoder, version int16) (err error) { + if err = pe.putString(r.Name); err != nil { + return err + } + + if err = pe.putString(r.Value); err != nil { + return err + } + + pe.putBool(r.ReadOnly) + + if version <= 0 { + pe.putBool(r.Default) + pe.putBool(r.Sensitive) + } else { + pe.putInt8(int8(r.Source)) + pe.putBool(r.Sensitive) + + if err := pe.putArrayLength(len(r.Synonyms)); err != nil { + return err + } + for _, c := range r.Synonyms { + if err = c.encode(pe, version); err != nil { + return err + } + } + } + + return nil +} + +// https://cwiki.apache.org/confluence/display/KAFKA/KIP-226+-+Dynamic+Broker+Configuration +func (r *ConfigEntry) decode(pd packetDecoder, version int16) (err error) { + if version == 0 { + r.Source = SourceUnknown + } + name, err := pd.getString() + if err != nil { + return err + } + r.Name = name + + value, err := pd.getString() + if err != nil { + return err + } + r.Value = value + + read, err := pd.getBool() + if err != nil { + return err + } + r.ReadOnly = read + + if version == 0 { + defaultB, err := pd.getBool() + if err != nil { + return err + } + r.Default = defaultB + if defaultB { + r.Source = SourceDefault + } + } else { + source, err := pd.getInt8() + if err != nil { + return err + } + r.Source = ConfigSource(source) + r.Default = r.Source == SourceDefault + } + + sensitive, err := pd.getBool() + if err != nil { + return err + } + r.Sensitive = sensitive + + if version > 0 { + n, err := pd.getArrayLength() + if err != nil { + return err + } + r.Synonyms = make([]*ConfigSynonym, n) + + for i := 0; i < n; i++ { + s := &ConfigSynonym{} + if err := s.decode(pd, version); err != nil { + return err + } + r.Synonyms[i] = s + } + } + return nil +} + +func (c *ConfigSynonym) encode(pe packetEncoder, version int16) (err error) { + err = pe.putString(c.ConfigName) + if err != nil { + return err + } + + err = pe.putString(c.ConfigValue) + if err != nil { + return err + } + + pe.putInt8(int8(c.Source)) + + return nil +} + +func (c *ConfigSynonym) decode(pd packetDecoder, version int16) error { + name, err := pd.getString() + if err != nil { + return err + } + c.ConfigName = name + + value, err := pd.getString() + if err != nil { + return err + } + c.ConfigValue = value + + source, err := pd.getInt8() + if err != nil { + return err + } + c.Source = ConfigSource(source) + return nil +} diff --git a/vendor/github.com/Shopify/sarama/describe_groups_request.go b/vendor/github.com/Shopify/sarama/describe_groups_request.go new file mode 100644 index 000000000000..f8962da58fc0 --- /dev/null +++ b/vendor/github.com/Shopify/sarama/describe_groups_request.go @@ -0,0 +1,34 @@ +package sarama + +type DescribeGroupsRequest struct { + Groups []string +} + +func (r *DescribeGroupsRequest) encode(pe packetEncoder) error { + return pe.putStringArray(r.Groups) +} + +func (r *DescribeGroupsRequest) decode(pd packetDecoder, version int16) (err error) { + r.Groups, err = pd.getStringArray() + return +} + +func (r *DescribeGroupsRequest) key() int16 { + return 15 +} + +func (r *DescribeGroupsRequest) version() int16 { + return 0 +} + +func (r *DescribeGroupsRequest) headerVersion() int16 { + return 1 +} + +func (r *DescribeGroupsRequest) requiredVersion() KafkaVersion { + return V0_9_0_0 +} + +func (r *DescribeGroupsRequest) AddGroup(group string) { + r.Groups = append(r.Groups, group) +} diff --git a/vendor/github.com/Shopify/sarama/describe_groups_response.go b/vendor/github.com/Shopify/sarama/describe_groups_response.go new file mode 100644 index 000000000000..bc242e4217dd --- /dev/null +++ b/vendor/github.com/Shopify/sarama/describe_groups_response.go @@ -0,0 +1,191 @@ +package sarama + +type DescribeGroupsResponse struct { + Groups []*GroupDescription +} + +func (r *DescribeGroupsResponse) encode(pe packetEncoder) error { + if err := pe.putArrayLength(len(r.Groups)); err != nil { + return err + } + + for _, groupDescription := range r.Groups { + if err := groupDescription.encode(pe); err != nil { + return err + } + } + + return nil +} + +func (r *DescribeGroupsResponse) decode(pd packetDecoder, version int16) (err error) { + n, err := pd.getArrayLength() + if err != nil { + return err + } + + r.Groups = make([]*GroupDescription, n) + for i := 0; i < n; i++ { + r.Groups[i] = new(GroupDescription) + if err := r.Groups[i].decode(pd); err != nil { + return err + } + } + + return nil +} + +func (r *DescribeGroupsResponse) key() int16 { + return 15 +} + +func (r *DescribeGroupsResponse) version() int16 { + return 0 +} + +func (r *DescribeGroupsResponse) headerVersion() int16 { + return 0 +} + +func (r *DescribeGroupsResponse) requiredVersion() KafkaVersion { + return V0_9_0_0 +} + +type GroupDescription struct { + Err KError + GroupId string + State string + ProtocolType string + Protocol string + Members map[string]*GroupMemberDescription +} + +func (gd *GroupDescription) encode(pe packetEncoder) error { + pe.putInt16(int16(gd.Err)) + + if err := pe.putString(gd.GroupId); err != nil { + return err + } + if err := pe.putString(gd.State); err != nil { + return err + } + if err := pe.putString(gd.ProtocolType); err != nil { + return err + } + if err := pe.putString(gd.Protocol); err != nil { + return err + } + + if err := pe.putArrayLength(len(gd.Members)); err != nil { + return err + } + + for memberId, groupMemberDescription := range gd.Members { + if err := pe.putString(memberId); err != nil { + return err + } + if err := groupMemberDescription.encode(pe); err != nil { + return err + } + } + + return nil +} + +func (gd *GroupDescription) decode(pd packetDecoder) (err error) { + kerr, err := pd.getInt16() + if err != nil { + return err + } + + gd.Err = KError(kerr) + + if gd.GroupId, err = pd.getString(); err != nil { + return + } + if gd.State, err = pd.getString(); err != nil { + return + } + if gd.ProtocolType, err = pd.getString(); err != nil { + return + } + if gd.Protocol, err = pd.getString(); err != nil { + return + } + + n, err := pd.getArrayLength() + if err != nil { + return err + } + if n == 0 { + return nil + } + + gd.Members = make(map[string]*GroupMemberDescription) + for i := 0; i < n; i++ { + memberId, err := pd.getString() + if err != nil { + return err + } + + gd.Members[memberId] = new(GroupMemberDescription) + if err := gd.Members[memberId].decode(pd); err != nil { + return err + } + } + + return nil +} + +type GroupMemberDescription struct { + ClientId string + ClientHost string + MemberMetadata []byte + MemberAssignment []byte +} + +func (gmd *GroupMemberDescription) encode(pe packetEncoder) error { + if err := pe.putString(gmd.ClientId); err != nil { + return err + } + if err := pe.putString(gmd.ClientHost); err != nil { + return err + } + if err := pe.putBytes(gmd.MemberMetadata); err != nil { + return err + } + if err := pe.putBytes(gmd.MemberAssignment); err != nil { + return err + } + + return nil +} + +func (gmd *GroupMemberDescription) decode(pd packetDecoder) (err error) { + if gmd.ClientId, err = pd.getString(); err != nil { + return + } + if gmd.ClientHost, err = pd.getString(); err != nil { + return + } + if gmd.MemberMetadata, err = pd.getBytes(); err != nil { + return + } + if gmd.MemberAssignment, err = pd.getBytes(); err != nil { + return + } + + return nil +} + +func (gmd *GroupMemberDescription) GetMemberAssignment() (*ConsumerGroupMemberAssignment, error) { + assignment := new(ConsumerGroupMemberAssignment) + err := decode(gmd.MemberAssignment, assignment) + return assignment, err +} + +func (gmd *GroupMemberDescription) GetMemberMetadata() (*ConsumerGroupMemberMetadata, error) { + metadata := new(ConsumerGroupMemberMetadata) + err := decode(gmd.MemberMetadata, metadata) + return metadata, err +} diff --git a/vendor/github.com/Shopify/sarama/describe_log_dirs_request.go b/vendor/github.com/Shopify/sarama/describe_log_dirs_request.go new file mode 100644 index 000000000000..c0bf04e04e27 --- /dev/null +++ b/vendor/github.com/Shopify/sarama/describe_log_dirs_request.go @@ -0,0 +1,87 @@ +package sarama + +// DescribeLogDirsRequest is a describe request to get partitions' log size +type DescribeLogDirsRequest struct { + // Version 0 and 1 are equal + // The version number is bumped to indicate that on quota violation brokers send out responses before throttling. + Version int16 + + // If this is an empty array, all topics will be queried + DescribeTopics []DescribeLogDirsRequestTopic +} + +// DescribeLogDirsRequestTopic is a describe request about the log dir of one or more partitions within a Topic +type DescribeLogDirsRequestTopic struct { + Topic string + PartitionIDs []int32 +} + +func (r *DescribeLogDirsRequest) encode(pe packetEncoder) error { + length := len(r.DescribeTopics) + if length == 0 { + // In order to query all topics we must send null + length = -1 + } + + if err := pe.putArrayLength(length); err != nil { + return err + } + + for _, d := range r.DescribeTopics { + if err := pe.putString(d.Topic); err != nil { + return err + } + + if err := pe.putInt32Array(d.PartitionIDs); err != nil { + return err + } + } + + return nil +} + +func (r *DescribeLogDirsRequest) decode(pd packetDecoder, version int16) error { + n, err := pd.getArrayLength() + if err != nil { + return err + } + if n == -1 { + n = 0 + } + + topics := make([]DescribeLogDirsRequestTopic, n) + for i := 0; i < n; i++ { + topics[i] = DescribeLogDirsRequestTopic{} + + topic, err := pd.getString() + if err != nil { + return err + } + topics[i].Topic = topic + + pIDs, err := pd.getInt32Array() + if err != nil { + return err + } + topics[i].PartitionIDs = pIDs + } + r.DescribeTopics = topics + + return nil +} + +func (r *DescribeLogDirsRequest) key() int16 { + return 35 +} + +func (r *DescribeLogDirsRequest) version() int16 { + return r.Version +} + +func (r *DescribeLogDirsRequest) headerVersion() int16 { + return 1 +} + +func (r *DescribeLogDirsRequest) requiredVersion() KafkaVersion { + return V1_0_0_0 +} diff --git a/vendor/github.com/Shopify/sarama/describe_log_dirs_response.go b/vendor/github.com/Shopify/sarama/describe_log_dirs_response.go new file mode 100644 index 000000000000..411da38ad204 --- /dev/null +++ b/vendor/github.com/Shopify/sarama/describe_log_dirs_response.go @@ -0,0 +1,229 @@ +package sarama + +import "time" + +type DescribeLogDirsResponse struct { + ThrottleTime time.Duration + + // Version 0 and 1 are equal + // The version number is bumped to indicate that on quota violation brokers send out responses before throttling. + Version int16 + + LogDirs []DescribeLogDirsResponseDirMetadata +} + +func (r *DescribeLogDirsResponse) encode(pe packetEncoder) error { + pe.putInt32(int32(r.ThrottleTime / time.Millisecond)) + + if err := pe.putArrayLength(len(r.LogDirs)); err != nil { + return err + } + + for _, dir := range r.LogDirs { + if err := dir.encode(pe); err != nil { + return err + } + } + + return nil +} + +func (r *DescribeLogDirsResponse) decode(pd packetDecoder, version int16) error { + throttleTime, err := pd.getInt32() + if err != nil { + return err + } + r.ThrottleTime = time.Duration(throttleTime) * time.Millisecond + + // Decode array of DescribeLogDirsResponseDirMetadata + n, err := pd.getArrayLength() + if err != nil { + return err + } + + r.LogDirs = make([]DescribeLogDirsResponseDirMetadata, n) + for i := 0; i < n; i++ { + dir := DescribeLogDirsResponseDirMetadata{} + if err := dir.decode(pd, version); err != nil { + return err + } + r.LogDirs[i] = dir + } + + return nil +} + +func (r *DescribeLogDirsResponse) key() int16 { + return 35 +} + +func (r *DescribeLogDirsResponse) version() int16 { + return r.Version +} + +func (r *DescribeLogDirsResponse) headerVersion() int16 { + return 0 +} + +func (r *DescribeLogDirsResponse) requiredVersion() KafkaVersion { + return V1_0_0_0 +} + +type DescribeLogDirsResponseDirMetadata struct { + ErrorCode KError + + // The absolute log directory path + Path string + Topics []DescribeLogDirsResponseTopic +} + +func (r *DescribeLogDirsResponseDirMetadata) encode(pe packetEncoder) error { + pe.putInt16(int16(r.ErrorCode)) + + if err := pe.putString(r.Path); err != nil { + return err + } + + if err := pe.putArrayLength(len(r.Topics)); err != nil { + return err + } + for _, topic := range r.Topics { + if err := topic.encode(pe); err != nil { + return err + } + } + + return nil +} + +func (r *DescribeLogDirsResponseDirMetadata) decode(pd packetDecoder, version int16) error { + errCode, err := pd.getInt16() + if err != nil { + return err + } + r.ErrorCode = KError(errCode) + + path, err := pd.getString() + if err != nil { + return err + } + r.Path = path + + // Decode array of DescribeLogDirsResponseTopic + n, err := pd.getArrayLength() + if err != nil { + return err + } + + r.Topics = make([]DescribeLogDirsResponseTopic, n) + for i := 0; i < n; i++ { + t := DescribeLogDirsResponseTopic{} + + if err := t.decode(pd, version); err != nil { + return err + } + + r.Topics[i] = t + } + + return nil +} + +// DescribeLogDirsResponseTopic contains a topic's partitions descriptions +type DescribeLogDirsResponseTopic struct { + Topic string + Partitions []DescribeLogDirsResponsePartition +} + +func (r *DescribeLogDirsResponseTopic) encode(pe packetEncoder) error { + if err := pe.putString(r.Topic); err != nil { + return err + } + + if err := pe.putArrayLength(len(r.Partitions)); err != nil { + return err + } + for _, partition := range r.Partitions { + if err := partition.encode(pe); err != nil { + return err + } + } + + return nil +} + +func (r *DescribeLogDirsResponseTopic) decode(pd packetDecoder, version int16) error { + t, err := pd.getString() + if err != nil { + return err + } + r.Topic = t + + n, err := pd.getArrayLength() + if err != nil { + return err + } + r.Partitions = make([]DescribeLogDirsResponsePartition, n) + for i := 0; i < n; i++ { + p := DescribeLogDirsResponsePartition{} + if err := p.decode(pd, version); err != nil { + return err + } + r.Partitions[i] = p + } + + return nil +} + +// DescribeLogDirsResponsePartition describes a partition's log directory +type DescribeLogDirsResponsePartition struct { + PartitionID int32 + + // The size of the log segments of the partition in bytes. + Size int64 + + // The lag of the log's LEO w.r.t. partition's HW (if it is the current log for the partition) or + // current replica's LEO (if it is the future log for the partition) + OffsetLag int64 + + // True if this log is created by AlterReplicaLogDirsRequest and will replace the current log of + // the replica in the future. + IsTemporary bool +} + +func (r *DescribeLogDirsResponsePartition) encode(pe packetEncoder) error { + pe.putInt32(r.PartitionID) + pe.putInt64(r.Size) + pe.putInt64(r.OffsetLag) + pe.putBool(r.IsTemporary) + + return nil +} + +func (r *DescribeLogDirsResponsePartition) decode(pd packetDecoder, version int16) error { + pID, err := pd.getInt32() + if err != nil { + return err + } + r.PartitionID = pID + + size, err := pd.getInt64() + if err != nil { + return err + } + r.Size = size + + lag, err := pd.getInt64() + if err != nil { + return err + } + r.OffsetLag = lag + + isTemp, err := pd.getBool() + if err != nil { + return err + } + r.IsTemporary = isTemp + + return nil +} diff --git a/vendor/github.com/Shopify/sarama/describe_user_scram_credentials_request.go b/vendor/github.com/Shopify/sarama/describe_user_scram_credentials_request.go new file mode 100644 index 000000000000..b5b59404bdce --- /dev/null +++ b/vendor/github.com/Shopify/sarama/describe_user_scram_credentials_request.go @@ -0,0 +1,70 @@ +package sarama + +// DescribeUserScramCredentialsRequest is a request to get list of SCRAM user names +type DescribeUserScramCredentialsRequest struct { + // Version 0 is currently only supported + Version int16 + + // If this is an empty array, all users will be queried + DescribeUsers []DescribeUserScramCredentialsRequestUser +} + +// DescribeUserScramCredentialsRequestUser is a describe request about specific user name +type DescribeUserScramCredentialsRequestUser struct { + Name string +} + +func (r *DescribeUserScramCredentialsRequest) encode(pe packetEncoder) error { + pe.putCompactArrayLength(len(r.DescribeUsers)) + for _, d := range r.DescribeUsers { + if err := pe.putCompactString(d.Name); err != nil { + return err + } + pe.putEmptyTaggedFieldArray() + } + + pe.putEmptyTaggedFieldArray() + return nil +} + +func (r *DescribeUserScramCredentialsRequest) decode(pd packetDecoder, version int16) error { + n, err := pd.getCompactArrayLength() + if err != nil { + return err + } + if n == -1 { + n = 0 + } + + r.DescribeUsers = make([]DescribeUserScramCredentialsRequestUser, n) + for i := 0; i < n; i++ { + r.DescribeUsers[i] = DescribeUserScramCredentialsRequestUser{} + if r.DescribeUsers[i].Name, err = pd.getCompactString(); err != nil { + return err + } + if _, err = pd.getEmptyTaggedFieldArray(); err != nil { + return err + } + } + + if _, err = pd.getEmptyTaggedFieldArray(); err != nil { + return err + } + return nil +} + +func (r *DescribeUserScramCredentialsRequest) key() int16 { + return 50 +} + +func (r *DescribeUserScramCredentialsRequest) version() int16 { + return r.Version +} + +func (r *DescribeUserScramCredentialsRequest) headerVersion() int16 { + return 2 +} + +func (r *DescribeUserScramCredentialsRequest) requiredVersion() KafkaVersion { + return V2_7_0_0 +} diff --git a/vendor/github.com/Shopify/sarama/describe_user_scram_credentials_response.go b/vendor/github.com/Shopify/sarama/describe_user_scram_credentials_response.go new file mode 100644 index 000000000000..2656c2faa1c1 --- /dev/null +++ b/vendor/github.com/Shopify/sarama/describe_user_scram_credentials_response.go @@ -0,0 +1,168 @@ +package sarama + +import "time" + +type ScramMechanismType int8 + +const ( + SCRAM_MECHANISM_UNKNOWN ScramMechanismType = iota // 0 + SCRAM_MECHANISM_SHA_256 // 1 + SCRAM_MECHANISM_SHA_512 // 2 +) + +func (s ScramMechanismType) String() string { + switch s { + case 1: + return SASLTypeSCRAMSHA256 + case 2: + return SASLTypeSCRAMSHA512 + default: + return "Unknown" + } +} + +type DescribeUserScramCredentialsResponse struct { + // Version 0 is currently only supported + Version int16 + + ThrottleTime time.Duration + + ErrorCode KError + ErrorMessage *string + + Results []*DescribeUserScramCredentialsResult +} + +type DescribeUserScramCredentialsResult struct { + User string + + ErrorCode KError + ErrorMessage *string + + CredentialInfos []*UserScramCredentialsResponseInfo +} + +type UserScramCredentialsResponseInfo struct { + Mechanism ScramMechanismType + Iterations int32 +} + +func (r *DescribeUserScramCredentialsResponse) encode(pe packetEncoder) error { + pe.putInt32(int32(r.ThrottleTime / time.Millisecond)) + + pe.putInt16(int16(r.ErrorCode)) + if err := pe.putNullableCompactString(r.ErrorMessage); err != nil { + return err + } + + pe.putCompactArrayLength(len(r.Results)) + for _, u := range r.Results { + if err := pe.putCompactString(u.User); err != nil { + return err + } + pe.putInt16(int16(u.ErrorCode)) + if err := pe.putNullableCompactString(u.ErrorMessage); err != nil { + return err + } + + pe.putCompactArrayLength(len(u.CredentialInfos)) + for _, c := range u.CredentialInfos { + pe.putInt8(int8(c.Mechanism)) + pe.putInt32(c.Iterations) + pe.putEmptyTaggedFieldArray() + } + + pe.putEmptyTaggedFieldArray() + } + + pe.putEmptyTaggedFieldArray() + return nil +} + +func (r *DescribeUserScramCredentialsResponse) decode(pd packetDecoder, version int16) error { + throttleTime, err := pd.getInt32() + if err != nil { + return err + } + r.ThrottleTime = time.Duration(throttleTime) * time.Millisecond + + kerr, err := pd.getInt16() + if err != nil { + return err + } + + r.ErrorCode = KError(kerr) + if r.ErrorMessage, err = pd.getCompactNullableString(); err != nil { + return err + } + + numUsers, err := pd.getCompactArrayLength() + if err != nil { + return err + } + + if numUsers > 0 { + r.Results = make([]*DescribeUserScramCredentialsResult, numUsers) + for i := 0; i < numUsers; i++ { + r.Results[i] = &DescribeUserScramCredentialsResult{} + if r.Results[i].User, err = pd.getCompactString(); err != nil { + return err + } + + errorCode, err := pd.getInt16() + if err != nil { + return err + } + r.Results[i].ErrorCode = KError(errorCode) + if r.Results[i].ErrorMessage, err = pd.getCompactNullableString(); err != nil { + return err + } + + numCredentialInfos, err := pd.getCompactArrayLength() + if err != nil { + return err + } + + r.Results[i].CredentialInfos = make([]*UserScramCredentialsResponseInfo, numCredentialInfos) + for j := 0; j < numCredentialInfos; j++ { + r.Results[i].CredentialInfos[j] = &UserScramCredentialsResponseInfo{} + scramMechanism, err := pd.getInt8() + if err != nil { + return err + } + r.Results[i].CredentialInfos[j].Mechanism = ScramMechanismType(scramMechanism) + if r.Results[i].CredentialInfos[j].Iterations, err = pd.getInt32(); err != nil { + return err + } + if _, err = pd.getEmptyTaggedFieldArray(); err != nil { + return err + } + } + + if _, err = pd.getEmptyTaggedFieldArray(); err != nil { + return err + } + } + } + + if _, err = pd.getEmptyTaggedFieldArray(); err != nil { + return err + } + return nil +} + +func (r *DescribeUserScramCredentialsResponse) key() int16 { + return 50 +} + +func (r *DescribeUserScramCredentialsResponse) version() int16 { + return r.Version +} + +func (r *DescribeUserScramCredentialsResponse) headerVersion() int16 { + return 2 +} + +func (r *DescribeUserScramCredentialsResponse) requiredVersion() KafkaVersion { + return V2_7_0_0 +} diff --git a/vendor/github.com/Shopify/sarama/dev.yml b/vendor/github.com/Shopify/sarama/dev.yml new file mode 100644 index 000000000000..05448a884fb3 --- /dev/null +++ b/vendor/github.com/Shopify/sarama/dev.yml @@ -0,0 +1,10 @@ +name: sarama + +up: + - go: + version: '1.17.1' + +commands: + test: + run: make test + desc: 'run unit tests' diff --git a/vendor/github.com/Shopify/sarama/docker-compose.yml b/vendor/github.com/Shopify/sarama/docker-compose.yml new file mode 100644 index 000000000000..744ed3cf8762 --- /dev/null +++ b/vendor/github.com/Shopify/sarama/docker-compose.yml @@ -0,0 +1,134 @@ +version: '3.7' +services: + zookeeper-1: + image: 'confluentinc/cp-zookeeper:${CONFLUENT_PLATFORM_VERSION:-6.2.0}' + restart: always + environment: + ZOOKEEPER_SERVER_ID: '1' + ZOOKEEPER_SERVERS: 'zookeeper-1:2888:3888;zookeeper-2:2888:3888;zookeeper-3:2888:3888' + ZOOKEEPER_CLIENT_PORT: '2181' + ZOOKEEPER_PEER_PORT: '2888' + ZOOKEEPER_LEADER_PORT: '3888' + ZOOKEEPER_INIT_LIMIT: '10' + ZOOKEEPER_SYNC_LIMIT: '5' + ZOOKEEPER_MAX_CLIENT_CONNS: '0' + zookeeper-2: + image: 'confluentinc/cp-zookeeper:${CONFLUENT_PLATFORM_VERSION:-6.2.0}' + restart: always + environment: + ZOOKEEPER_SERVER_ID: '2' + ZOOKEEPER_SERVERS: 'zookeeper-1:2888:3888;zookeeper-2:2888:3888;zookeeper-3:2888:3888' + ZOOKEEPER_CLIENT_PORT: '2181' + ZOOKEEPER_PEER_PORT: '2888' + ZOOKEEPER_LEADER_PORT: '3888' + ZOOKEEPER_INIT_LIMIT: '10' + ZOOKEEPER_SYNC_LIMIT: '5' + ZOOKEEPER_MAX_CLIENT_CONNS: '0' + zookeeper-3: + image: 'confluentinc/cp-zookeeper:${CONFLUENT_PLATFORM_VERSION:-6.2.0}' + restart: always + environment: + ZOOKEEPER_SERVER_ID: '3' + ZOOKEEPER_SERVERS: 'zookeeper-1:2888:3888;zookeeper-2:2888:3888;zookeeper-3:2888:3888' + ZOOKEEPER_CLIENT_PORT: '2181' + ZOOKEEPER_PEER_PORT: '2888' + ZOOKEEPER_LEADER_PORT: '3888' + ZOOKEEPER_INIT_LIMIT: '10' + ZOOKEEPER_SYNC_LIMIT: '5' + ZOOKEEPER_MAX_CLIENT_CONNS: '0' + kafka-1: + image: 'confluentinc/cp-kafka:${CONFLUENT_PLATFORM_VERSION:-6.2.0}' + restart: always + environment: + KAFKA_ZOOKEEPER_CONNECT: 'zookeeper-1:2181,zookeeper-2:2181,zookeeper-3:2181' + KAFKA_LISTENERS: 'LISTENER_INTERNAL://:9091,LISTENER_LOCAL://:29091' + KAFKA_ADVERTISED_LISTENERS: 'LISTENER_INTERNAL://kafka-1:9091,LISTENER_LOCAL://localhost:29091' + KAFKA_INTER_BROKER_LISTENER_NAME: 'LISTENER_INTERNAL' + KAFKA_LISTENER_SECURITY_PROTOCOL_MAP: 'LISTENER_INTERNAL:PLAINTEXT,LISTENER_LOCAL:PLAINTEXT' + KAFKA_DEFAULT_REPLICATION_FACTOR: '2' + KAFKA_BROKER_ID: '1' + KAFKA_BROKER_RACK: '1' + KAFKA_ZOOKEEPER_SESSION_TIMEOUT_MS: '3000' + KAFKA_ZOOKEEPER_CONNECTION_TIMEOUT_MS: '3000' + KAFKA_REPLICA_SELECTOR_CLASS: 'org.apache.kafka.common.replica.RackAwareReplicaSelector' + KAFKA_DELETE_TOPIC_ENABLE: 'true' + KAFKA_AUTO_CREATE_TOPICS_ENABLE: 'false' + kafka-2: + image: 'confluentinc/cp-kafka:${CONFLUENT_PLATFORM_VERSION:-6.2.0}' + restart: always + environment: + KAFKA_ZOOKEEPER_CONNECT: 'zookeeper-1:2181,zookeeper-2:2181,zookeeper-3:2181' + KAFKA_LISTENERS: 'LISTENER_INTERNAL://:9091,LISTENER_LOCAL://:29092' + KAFKA_ADVERTISED_LISTENERS: 'LISTENER_INTERNAL://kafka-2:9091,LISTENER_LOCAL://localhost:29092' + KAFKA_INTER_BROKER_LISTENER_NAME: 'LISTENER_INTERNAL' + KAFKA_LISTENER_SECURITY_PROTOCOL_MAP: 'LISTENER_INTERNAL:PLAINTEXT,LISTENER_LOCAL:PLAINTEXT' + KAFKA_DEFAULT_REPLICATION_FACTOR: '2' + KAFKA_BROKER_ID: '2' + KAFKA_BROKER_RACK: '2' + KAFKA_ZOOKEEPER_SESSION_TIMEOUT_MS: '3000' + KAFKA_ZOOKEEPER_CONNECTION_TIMEOUT_MS: '3000' + KAFKA_REPLICA_SELECTOR_CLASS: 'org.apache.kafka.common.replica.RackAwareReplicaSelector' + KAFKA_DELETE_TOPIC_ENABLE: 'true' + KAFKA_AUTO_CREATE_TOPICS_ENABLE: 'false' + kafka-3: + image: 'confluentinc/cp-kafka:${CONFLUENT_PLATFORM_VERSION:-6.2.0}' + restart: always + environment: + KAFKA_ZOOKEEPER_CONNECT: 'zookeeper-1:2181,zookeeper-2:2181,zookeeper-3:2181' + KAFKA_LISTENERS: 'LISTENER_INTERNAL://:9091,LISTENER_LOCAL://:29093' + KAFKA_ADVERTISED_LISTENERS: 'LISTENER_INTERNAL://kafka-3:9091,LISTENER_LOCAL://localhost:29093' + KAFKA_INTER_BROKER_LISTENER_NAME: 'LISTENER_INTERNAL' + KAFKA_LISTENER_SECURITY_PROTOCOL_MAP: 'LISTENER_INTERNAL:PLAINTEXT,LISTENER_LOCAL:PLAINTEXT' + KAFKA_DEFAULT_REPLICATION_FACTOR: '2' + KAFKA_BROKER_ID: '3' + KAFKA_BROKER_RACK: '3' + KAFKA_ZOOKEEPER_SESSION_TIMEOUT_MS: '3000' + KAFKA_ZOOKEEPER_CONNECTION_TIMEOUT_MS: '3000' + KAFKA_REPLICA_SELECTOR_CLASS: 'org.apache.kafka.common.replica.RackAwareReplicaSelector' + KAFKA_DELETE_TOPIC_ENABLE: 'true' + KAFKA_AUTO_CREATE_TOPICS_ENABLE: 'false' + kafka-4: + image: 'confluentinc/cp-kafka:${CONFLUENT_PLATFORM_VERSION:-6.2.0}' + restart: always + environment: + KAFKA_ZOOKEEPER_CONNECT: 'zookeeper-1:2181,zookeeper-2:2181,zookeeper-3:2181' + KAFKA_LISTENERS: 'LISTENER_INTERNAL://:9091,LISTENER_LOCAL://:29094' + KAFKA_ADVERTISED_LISTENERS: 'LISTENER_INTERNAL://kafka-4:9091,LISTENER_LOCAL://localhost:29094' + KAFKA_INTER_BROKER_LISTENER_NAME: 'LISTENER_INTERNAL' + KAFKA_LISTENER_SECURITY_PROTOCOL_MAP: 'LISTENER_INTERNAL:PLAINTEXT,LISTENER_LOCAL:PLAINTEXT' + KAFKA_DEFAULT_REPLICATION_FACTOR: '2' + KAFKA_BROKER_ID: '4' + KAFKA_BROKER_RACK: '4' + KAFKA_ZOOKEEPER_SESSION_TIMEOUT_MS: '3000' + KAFKA_ZOOKEEPER_CONNECTION_TIMEOUT_MS: '3000' + KAFKA_REPLICA_SELECTOR_CLASS: 'org.apache.kafka.common.replica.RackAwareReplicaSelector' + KAFKA_DELETE_TOPIC_ENABLE: 'true' + KAFKA_AUTO_CREATE_TOPICS_ENABLE: 'false' + kafka-5: + image: 'confluentinc/cp-kafka:${CONFLUENT_PLATFORM_VERSION:-6.2.0}' + restart: always + environment: + KAFKA_ZOOKEEPER_CONNECT: 'zookeeper-1:2181,zookeeper-2:2181,zookeeper-3:2181' + KAFKA_LISTENERS: 'LISTENER_INTERNAL://:9091,LISTENER_LOCAL://:29095' + KAFKA_ADVERTISED_LISTENERS: 'LISTENER_INTERNAL://kafka-5:9091,LISTENER_LOCAL://localhost:29095' + KAFKA_INTER_BROKER_LISTENER_NAME: 'LISTENER_INTERNAL' + KAFKA_LISTENER_SECURITY_PROTOCOL_MAP: 'LISTENER_INTERNAL:PLAINTEXT,LISTENER_LOCAL:PLAINTEXT' + KAFKA_DEFAULT_REPLICATION_FACTOR: '2' + KAFKA_BROKER_ID: '5' + KAFKA_BROKER_RACK: '5' + KAFKA_ZOOKEEPER_SESSION_TIMEOUT_MS: '3000' + KAFKA_ZOOKEEPER_CONNECTION_TIMEOUT_MS: '3000' + KAFKA_REPLICA_SELECTOR_CLASS: 'org.apache.kafka.common.replica.RackAwareReplicaSelector' + KAFKA_DELETE_TOPIC_ENABLE: 'true' + KAFKA_AUTO_CREATE_TOPICS_ENABLE: 'false' + toxiproxy: + image: 'ghcr.io/shopify/toxiproxy:2.1.5' + ports: + # The tests themselves actually start the proxies on these ports + - '29091:29091' + - '29092:29092' + - '29093:29093' + - '29094:29094' + - '29095:29095' + # This is the toxiproxy API port + - '8474:8474' diff --git a/vendor/github.com/Shopify/sarama/encoder_decoder.go b/vendor/github.com/Shopify/sarama/encoder_decoder.go new file mode 100644 index 000000000000..dab54f88cc78 --- /dev/null +++ b/vendor/github.com/Shopify/sarama/encoder_decoder.go @@ -0,0 +1,94 @@ +package sarama + +import ( + "fmt" + + "github.com/rcrowley/go-metrics" +) + +// Encoder is the interface that wraps the basic Encode method. +// Anything implementing Encoder can be turned into bytes using Kafka's encoding rules. +type encoder interface { + encode(pe packetEncoder) error +} + +type encoderWithHeader interface { + encoder + headerVersion() int16 +} + +// Encode takes an Encoder and turns it into bytes while potentially recording metrics. +func encode(e encoder, metricRegistry metrics.Registry) ([]byte, error) { + if e == nil { + return nil, nil + } + + var prepEnc prepEncoder + var realEnc realEncoder + + err := e.encode(&prepEnc) + if err != nil { + return nil, err + } + + if prepEnc.length < 0 || prepEnc.length > int(MaxRequestSize) { + return nil, PacketEncodingError{fmt.Sprintf("invalid request size (%d)", prepEnc.length)} + } + + realEnc.raw = make([]byte, prepEnc.length) + realEnc.registry = metricRegistry + err = e.encode(&realEnc) + if err != nil { + return nil, err + } + + return realEnc.raw, nil +} + +// decoder is the interface that wraps the basic Decode method. +// Anything implementing Decoder can be extracted from bytes using Kafka's encoding rules. +type decoder interface { + decode(pd packetDecoder) error +} + +type versionedDecoder interface { + decode(pd packetDecoder, version int16) error +} + +// decode takes bytes and a decoder and fills the fields of the decoder from the bytes, +// interpreted using Kafka's encoding rules. +func decode(buf []byte, in decoder) error { + if buf == nil { + return nil + } + + helper := realDecoder{raw: buf} + err := in.decode(&helper) + if err != nil { + return err + } + + if helper.off != len(buf) { + return PacketDecodingError{"invalid length"} + } + + return nil +} + +func versionedDecode(buf []byte, in versionedDecoder, version int16) error { + if buf == nil { + return nil + } + + helper := realDecoder{raw: buf} + err := in.decode(&helper, version) + if err != nil { + return err + } + + if helper.off != len(buf) { + return PacketDecodingError{"invalid length"} + } + + return nil +} diff --git a/vendor/github.com/Shopify/sarama/end_txn_request.go b/vendor/github.com/Shopify/sarama/end_txn_request.go new file mode 100644 index 000000000000..6635425ddd62 --- /dev/null +++ b/vendor/github.com/Shopify/sarama/end_txn_request.go @@ -0,0 +1,54 @@ +package sarama + +type EndTxnRequest struct { + TransactionalID string + ProducerID int64 + ProducerEpoch int16 + TransactionResult bool +} + +func (a *EndTxnRequest) encode(pe packetEncoder) error { + if err := pe.putString(a.TransactionalID); err != nil { + return err + } + + pe.putInt64(a.ProducerID) + + pe.putInt16(a.ProducerEpoch) + + pe.putBool(a.TransactionResult) + + return nil +} + +func (a *EndTxnRequest) decode(pd packetDecoder, version int16) (err error) { + if a.TransactionalID, err = pd.getString(); err != nil { + return err + } + if a.ProducerID, err = pd.getInt64(); err != nil { + return err + } + if a.ProducerEpoch, err = pd.getInt16(); err != nil { + return err + } + if a.TransactionResult, err = pd.getBool(); err != nil { + return err + } + return nil +} + +func (a *EndTxnRequest) key() int16 { + return 26 +} + +func (a *EndTxnRequest) version() int16 { + return 0 +} + +func (r *EndTxnRequest) headerVersion() int16 { + return 1 +} + +func (a *EndTxnRequest) requiredVersion() KafkaVersion { + return V0_11_0_0 +} diff --git a/vendor/github.com/Shopify/sarama/end_txn_response.go b/vendor/github.com/Shopify/sarama/end_txn_response.go new file mode 100644 index 000000000000..763976726ccb --- /dev/null +++ b/vendor/github.com/Shopify/sarama/end_txn_response.go @@ -0,0 +1,48 @@ +package sarama + +import ( + "time" +) + +type EndTxnResponse struct { + ThrottleTime time.Duration + Err KError +} + +func (e *EndTxnResponse) encode(pe packetEncoder) error { + pe.putInt32(int32(e.ThrottleTime / time.Millisecond)) + pe.putInt16(int16(e.Err)) + return nil +} + +func (e *EndTxnResponse) decode(pd packetDecoder, version int16) (err error) { + throttleTime, err := pd.getInt32() + if err != nil { + return err + } + e.ThrottleTime = time.Duration(throttleTime) * time.Millisecond + + kerr, err := pd.getInt16() + if err != nil { + return err + } + e.Err = KError(kerr) + + return nil +} + +func (e *EndTxnResponse) key() int16 { + return 25 +} + +func (e *EndTxnResponse) version() int16 { + return 0 +} + +func (r *EndTxnResponse) headerVersion() int16 { + return 0 +} + +func (e *EndTxnResponse) requiredVersion() KafkaVersion { + return V0_11_0_0 +} diff --git a/vendor/github.com/Shopify/sarama/errors.go b/vendor/github.com/Shopify/sarama/errors.go new file mode 100644 index 000000000000..0fca0a30eabd --- /dev/null +++ b/vendor/github.com/Shopify/sarama/errors.go @@ -0,0 +1,409 @@ +package sarama + +import ( + "errors" + "fmt" +) + +// ErrOutOfBrokers is the error returned when the client has run out of brokers to talk to because all of them errored +// or otherwise failed to respond. +var ErrOutOfBrokers = errors.New("kafka: client has run out of available brokers to talk to (Is your cluster reachable?)") + +// ErrBrokerNotFound is the error returned when there's no broker found for the requested ID. +var ErrBrokerNotFound = errors.New("kafka: broker for ID is not found") + +// ErrClosedClient is the error returned when a method is called on a client that has been closed. +var ErrClosedClient = errors.New("kafka: tried to use a client that was closed") + +// ErrIncompleteResponse is the error returned when the server returns a syntactically valid response, but it does +// not contain the expected information. +var ErrIncompleteResponse = errors.New("kafka: response did not contain all the expected topic/partition blocks") + +// ErrInvalidPartition is the error returned when a partitioner returns an invalid partition index +// (meaning one outside of the range [0...numPartitions-1]). +var ErrInvalidPartition = errors.New("kafka: partitioner returned an invalid partition index") + +// ErrAlreadyConnected is the error returned when calling Open() on a Broker that is already connected or connecting. +var ErrAlreadyConnected = errors.New("kafka: broker connection already initiated") + +// ErrNotConnected is the error returned when trying to send or call Close() on a Broker that is not connected. +var ErrNotConnected = errors.New("kafka: broker not connected") + +// ErrInsufficientData is returned when decoding and the packet is truncated. This can be expected +// when requesting messages, since as an optimization the server is allowed to return a partial message at the end +// of the message set. +var ErrInsufficientData = errors.New("kafka: insufficient data to decode packet, more bytes expected") + +// ErrShuttingDown is returned when a producer receives a message during shutdown. +var ErrShuttingDown = errors.New("kafka: message received by producer in process of shutting down") + +// ErrMessageTooLarge is returned when the next message to consume is larger than the configured Consumer.Fetch.Max +var ErrMessageTooLarge = errors.New("kafka: message is larger than Consumer.Fetch.Max") + +// ErrConsumerOffsetNotAdvanced is returned when a partition consumer didn't advance its offset after parsing +// a RecordBatch. +var ErrConsumerOffsetNotAdvanced = errors.New("kafka: consumer offset was not advanced after a RecordBatch") + +// ErrControllerNotAvailable is returned when server didn't give correct controller id. May be kafka server's version +// is lower than 0.10.0.0. +var ErrControllerNotAvailable = errors.New("kafka: controller is not available") + +// ErrNoTopicsToUpdateMetadata is returned when Meta.Full is set to false but no specific topics were found to update +// the metadata. +var ErrNoTopicsToUpdateMetadata = errors.New("kafka: no specific topics to update metadata") + +// ErrUnknownScramMechanism is returned when user tries to AlterUserScramCredentials with unknown SCRAM mechanism +var ErrUnknownScramMechanism = errors.New("kafka: unknown SCRAM mechanism provided") + +// PacketEncodingError is returned from a failure while encoding a Kafka packet. This can happen, for example, +// if you try to encode a string over 2^15 characters in length, since Kafka's encoding rules do not permit that. +type PacketEncodingError struct { + Info string +} + +func (err PacketEncodingError) Error() string { + return fmt.Sprintf("kafka: error encoding packet: %s", err.Info) +} + +// PacketDecodingError is returned when there was an error (other than truncated data) decoding the Kafka broker's response. +// This can be a bad CRC or length field, or any other invalid value. +type PacketDecodingError struct { + Info string +} + +func (err PacketDecodingError) Error() string { + return fmt.Sprintf("kafka: error decoding packet: %s", err.Info) +} + +// ConfigurationError is the type of error returned from a constructor (e.g. NewClient, or NewConsumer) +// when the specified configuration is invalid. +type ConfigurationError string + +func (err ConfigurationError) Error() string { + return "kafka: invalid configuration (" + string(err) + ")" +} + +// KError is the type of error that can be returned directly by the Kafka broker. +// See https://cwiki.apache.org/confluence/display/KAFKA/A+Guide+To+The+Kafka+Protocol#AGuideToTheKafkaProtocol-ErrorCodes +type KError int16 + +// MultiError is used to contain multi error. +type MultiError struct { + Errors *[]error +} + +func (mErr MultiError) Error() string { + errString := "" + for _, err := range *mErr.Errors { + errString += err.Error() + "," + } + return errString +} + +func (mErr MultiError) PrettyError() string { + errString := "" + for _, err := range *mErr.Errors { + errString += err.Error() + "\n" + } + return errString +} + +// ErrDeleteRecords is the type of error returned when fail to delete the required records +type ErrDeleteRecords struct { + MultiError +} + +func (err ErrDeleteRecords) Error() string { + return "kafka server: failed to delete records " + err.MultiError.Error() +} + +type ErrReassignPartitions struct { + MultiError +} + +func (err ErrReassignPartitions) Error() string { + return fmt.Sprintf("failed to reassign partitions for topic: \n%s", err.MultiError.PrettyError()) +} + +// Numeric error codes returned by the Kafka server. +const ( + ErrNoError KError = 0 + ErrUnknown KError = -1 + ErrOffsetOutOfRange KError = 1 + ErrInvalidMessage KError = 2 + ErrUnknownTopicOrPartition KError = 3 + ErrInvalidMessageSize KError = 4 + ErrLeaderNotAvailable KError = 5 + ErrNotLeaderForPartition KError = 6 + ErrRequestTimedOut KError = 7 + ErrBrokerNotAvailable KError = 8 + ErrReplicaNotAvailable KError = 9 + ErrMessageSizeTooLarge KError = 10 + ErrStaleControllerEpochCode KError = 11 + ErrOffsetMetadataTooLarge KError = 12 + ErrNetworkException KError = 13 + ErrOffsetsLoadInProgress KError = 14 + ErrConsumerCoordinatorNotAvailable KError = 15 + ErrNotCoordinatorForConsumer KError = 16 + ErrInvalidTopic KError = 17 + ErrMessageSetSizeTooLarge KError = 18 + ErrNotEnoughReplicas KError = 19 + ErrNotEnoughReplicasAfterAppend KError = 20 + ErrInvalidRequiredAcks KError = 21 + ErrIllegalGeneration KError = 22 + ErrInconsistentGroupProtocol KError = 23 + ErrInvalidGroupId KError = 24 + ErrUnknownMemberId KError = 25 + ErrInvalidSessionTimeout KError = 26 + ErrRebalanceInProgress KError = 27 + ErrInvalidCommitOffsetSize KError = 28 + ErrTopicAuthorizationFailed KError = 29 + ErrGroupAuthorizationFailed KError = 30 + ErrClusterAuthorizationFailed KError = 31 + ErrInvalidTimestamp KError = 32 + ErrUnsupportedSASLMechanism KError = 33 + ErrIllegalSASLState KError = 34 + ErrUnsupportedVersion KError = 35 + ErrTopicAlreadyExists KError = 36 + ErrInvalidPartitions KError = 37 + ErrInvalidReplicationFactor KError = 38 + ErrInvalidReplicaAssignment KError = 39 + ErrInvalidConfig KError = 40 + ErrNotController KError = 41 + ErrInvalidRequest KError = 42 + ErrUnsupportedForMessageFormat KError = 43 + ErrPolicyViolation KError = 44 + ErrOutOfOrderSequenceNumber KError = 45 + ErrDuplicateSequenceNumber KError = 46 + ErrInvalidProducerEpoch KError = 47 + ErrInvalidTxnState KError = 48 + ErrInvalidProducerIDMapping KError = 49 + ErrInvalidTransactionTimeout KError = 50 + ErrConcurrentTransactions KError = 51 + ErrTransactionCoordinatorFenced KError = 52 + ErrTransactionalIDAuthorizationFailed KError = 53 + ErrSecurityDisabled KError = 54 + ErrOperationNotAttempted KError = 55 + ErrKafkaStorageError KError = 56 + ErrLogDirNotFound KError = 57 + ErrSASLAuthenticationFailed KError = 58 + ErrUnknownProducerID KError = 59 + ErrReassignmentInProgress KError = 60 + ErrDelegationTokenAuthDisabled KError = 61 + ErrDelegationTokenNotFound KError = 62 + ErrDelegationTokenOwnerMismatch KError = 63 + ErrDelegationTokenRequestNotAllowed KError = 64 + ErrDelegationTokenAuthorizationFailed KError = 65 + ErrDelegationTokenExpired KError = 66 + ErrInvalidPrincipalType KError = 67 + ErrNonEmptyGroup KError = 68 + ErrGroupIDNotFound KError = 69 + ErrFetchSessionIDNotFound KError = 70 + ErrInvalidFetchSessionEpoch KError = 71 + ErrListenerNotFound KError = 72 + ErrTopicDeletionDisabled KError = 73 + ErrFencedLeaderEpoch KError = 74 + ErrUnknownLeaderEpoch KError = 75 + ErrUnsupportedCompressionType KError = 76 + ErrStaleBrokerEpoch KError = 77 + ErrOffsetNotAvailable KError = 78 + ErrMemberIdRequired KError = 79 + ErrPreferredLeaderNotAvailable KError = 80 + ErrGroupMaxSizeReached KError = 81 + ErrFencedInstancedId KError = 82 + ErrEligibleLeadersNotAvailable KError = 83 + ErrElectionNotNeeded KError = 84 + ErrNoReassignmentInProgress KError = 85 + ErrGroupSubscribedToTopic KError = 86 + ErrInvalidRecord KError = 87 + ErrUnstableOffsetCommit KError = 88 +) + +func (err KError) Error() string { + // Error messages stolen/adapted from + // https://kafka.apache.org/protocol#protocol_error_codes + switch err { + case ErrNoError: + return "kafka server: Not an error, why are you printing me?" + case ErrUnknown: + return "kafka server: Unexpected (unknown?) server error." + case ErrOffsetOutOfRange: + return "kafka server: The requested offset is outside the range of offsets maintained by the server for the given topic/partition." + case ErrInvalidMessage: + return "kafka server: Message contents does not match its CRC." + case ErrUnknownTopicOrPartition: + return "kafka server: Request was for a topic or partition that does not exist on this broker." + case ErrInvalidMessageSize: + return "kafka server: The message has a negative size." + case ErrLeaderNotAvailable: + return "kafka server: In the middle of a leadership election, there is currently no leader for this partition and hence it is unavailable for writes." + case ErrNotLeaderForPartition: + return "kafka server: Tried to send a message to a replica that is not the leader for some partition. Your metadata is out of date." + case ErrRequestTimedOut: + return "kafka server: Request exceeded the user-specified time limit in the request." + case ErrBrokerNotAvailable: + return "kafka server: Broker not available. Not a client facing error, we should never receive this!!!" + case ErrReplicaNotAvailable: + return "kafka server: Replica information not available, one or more brokers are down." + case ErrMessageSizeTooLarge: + return "kafka server: Message was too large, server rejected it to avoid allocation error." + case ErrStaleControllerEpochCode: + return "kafka server: StaleControllerEpochCode (internal error code for broker-to-broker communication)." + case ErrOffsetMetadataTooLarge: + return "kafka server: Specified a string larger than the configured maximum for offset metadata." + case ErrNetworkException: + return "kafka server: The server disconnected before a response was received." + case ErrOffsetsLoadInProgress: + return "kafka server: The broker is still loading offsets after a leader change for that offset's topic partition." + case ErrConsumerCoordinatorNotAvailable: + return "kafka server: Offset's topic has not yet been created." + case ErrNotCoordinatorForConsumer: + return "kafka server: Request was for a consumer group that is not coordinated by this broker." + case ErrInvalidTopic: + return "kafka server: The request attempted to perform an operation on an invalid topic." + case ErrMessageSetSizeTooLarge: + return "kafka server: The request included message batch larger than the configured segment size on the server." + case ErrNotEnoughReplicas: + return "kafka server: Messages are rejected since there are fewer in-sync replicas than required." + case ErrNotEnoughReplicasAfterAppend: + return "kafka server: Messages are written to the log, but to fewer in-sync replicas than required." + case ErrInvalidRequiredAcks: + return "kafka server: The number of required acks is invalid (should be either -1, 0, or 1)." + case ErrIllegalGeneration: + return "kafka server: The provided generation id is not the current generation." + case ErrInconsistentGroupProtocol: + return "kafka server: The provider group protocol type is incompatible with the other members." + case ErrInvalidGroupId: + return "kafka server: The provided group id was empty." + case ErrUnknownMemberId: + return "kafka server: The provided member is not known in the current generation." + case ErrInvalidSessionTimeout: + return "kafka server: The provided session timeout is outside the allowed range." + case ErrRebalanceInProgress: + return "kafka server: A rebalance for the group is in progress. Please re-join the group." + case ErrInvalidCommitOffsetSize: + return "kafka server: The provided commit metadata was too large." + case ErrTopicAuthorizationFailed: + return "kafka server: The client is not authorized to access this topic." + case ErrGroupAuthorizationFailed: + return "kafka server: The client is not authorized to access this group." + case ErrClusterAuthorizationFailed: + return "kafka server: The client is not authorized to send this request type." + case ErrInvalidTimestamp: + return "kafka server: The timestamp of the message is out of acceptable range." + case ErrUnsupportedSASLMechanism: + return "kafka server: The broker does not support the requested SASL mechanism." + case ErrIllegalSASLState: + return "kafka server: Request is not valid given the current SASL state." + case ErrUnsupportedVersion: + return "kafka server: The version of API is not supported." + case ErrTopicAlreadyExists: + return "kafka server: Topic with this name already exists." + case ErrInvalidPartitions: + return "kafka server: Number of partitions is invalid." + case ErrInvalidReplicationFactor: + return "kafka server: Replication-factor is invalid." + case ErrInvalidReplicaAssignment: + return "kafka server: Replica assignment is invalid." + case ErrInvalidConfig: + return "kafka server: Configuration is invalid." + case ErrNotController: + return "kafka server: This is not the correct controller for this cluster." + case ErrInvalidRequest: + return "kafka server: This most likely occurs because of a request being malformed by the client library or the message was sent to an incompatible broker. See the broker logs for more details." + case ErrUnsupportedForMessageFormat: + return "kafka server: The requested operation is not supported by the message format version." + case ErrPolicyViolation: + return "kafka server: Request parameters do not satisfy the configured policy." + case ErrOutOfOrderSequenceNumber: + return "kafka server: The broker received an out of order sequence number." + case ErrDuplicateSequenceNumber: + return "kafka server: The broker received a duplicate sequence number." + case ErrInvalidProducerEpoch: + return "kafka server: Producer attempted an operation with an old epoch." + case ErrInvalidTxnState: + return "kafka server: The producer attempted a transactional operation in an invalid state." + case ErrInvalidProducerIDMapping: + return "kafka server: The producer attempted to use a producer id which is not currently assigned to its transactional id." + case ErrInvalidTransactionTimeout: + return "kafka server: The transaction timeout is larger than the maximum value allowed by the broker (as configured by max.transaction.timeout.ms)." + case ErrConcurrentTransactions: + return "kafka server: The producer attempted to update a transaction while another concurrent operation on the same transaction was ongoing." + case ErrTransactionCoordinatorFenced: + return "kafka server: The transaction coordinator sending a WriteTxnMarker is no longer the current coordinator for a given producer." + case ErrTransactionalIDAuthorizationFailed: + return "kafka server: Transactional ID authorization failed." + case ErrSecurityDisabled: + return "kafka server: Security features are disabled." + case ErrOperationNotAttempted: + return "kafka server: The broker did not attempt to execute this operation." + case ErrKafkaStorageError: + return "kafka server: Disk error when trying to access log file on the disk." + case ErrLogDirNotFound: + return "kafka server: The specified log directory is not found in the broker config." + case ErrSASLAuthenticationFailed: + return "kafka server: SASL Authentication failed." + case ErrUnknownProducerID: + return "kafka server: The broker could not locate the producer metadata associated with the Producer ID." + case ErrReassignmentInProgress: + return "kafka server: A partition reassignment is in progress." + case ErrDelegationTokenAuthDisabled: + return "kafka server: Delegation Token feature is not enabled." + case ErrDelegationTokenNotFound: + return "kafka server: Delegation Token is not found on server." + case ErrDelegationTokenOwnerMismatch: + return "kafka server: Specified Principal is not valid Owner/Renewer." + case ErrDelegationTokenRequestNotAllowed: + return "kafka server: Delegation Token requests are not allowed on PLAINTEXT/1-way SSL channels and on delegation token authenticated channels." + case ErrDelegationTokenAuthorizationFailed: + return "kafka server: Delegation Token authorization failed." + case ErrDelegationTokenExpired: + return "kafka server: Delegation Token is expired." + case ErrInvalidPrincipalType: + return "kafka server: Supplied principalType is not supported." + case ErrNonEmptyGroup: + return "kafka server: The group is not empty." + case ErrGroupIDNotFound: + return "kafka server: The group id does not exist." + case ErrFetchSessionIDNotFound: + return "kafka server: The fetch session ID was not found." + case ErrInvalidFetchSessionEpoch: + return "kafka server: The fetch session epoch is invalid." + case ErrListenerNotFound: + return "kafka server: There is no listener on the leader broker that matches the listener on which metadata request was processed." + case ErrTopicDeletionDisabled: + return "kafka server: Topic deletion is disabled." + case ErrFencedLeaderEpoch: + return "kafka server: The leader epoch in the request is older than the epoch on the broker." + case ErrUnknownLeaderEpoch: + return "kafka server: The leader epoch in the request is newer than the epoch on the broker." + case ErrUnsupportedCompressionType: + return "kafka server: The requesting client does not support the compression type of given partition." + case ErrStaleBrokerEpoch: + return "kafka server: Broker epoch has changed" + case ErrOffsetNotAvailable: + return "kafka server: The leader high watermark has not caught up from a recent leader election so the offsets cannot be guaranteed to be monotonically increasing" + case ErrMemberIdRequired: + return "kafka server: The group member needs to have a valid member id before actually entering a consumer group" + case ErrPreferredLeaderNotAvailable: + return "kafka server: The preferred leader was not available" + case ErrGroupMaxSizeReached: + return "kafka server: Consumer group The consumer group has reached its max size. already has the configured maximum number of members." + case ErrFencedInstancedId: + return "kafka server: The broker rejected this static consumer since another consumer with the same group.instance.id has registered with a different member.id." + case ErrEligibleLeadersNotAvailable: + return "kafka server: Eligible topic partition leaders are not available." + case ErrElectionNotNeeded: + return "kafka server: Leader election not needed for topic partition." + case ErrNoReassignmentInProgress: + return "kafka server: No partition reassignment is in progress." + case ErrGroupSubscribedToTopic: + return "kafka server: Deleting offsets of a topic is forbidden while the consumer group is actively subscribed to it." + case ErrInvalidRecord: + return "kafka server: This record has failed the validation on broker and hence will be rejected." + case ErrUnstableOffsetCommit: + return "kafka server: There are unstable offsets that need to be cleared." + } + + return fmt.Sprintf("Unknown error, how did this happen? Error code = %d", err) +} diff --git a/vendor/github.com/Shopify/sarama/fetch_request.go b/vendor/github.com/Shopify/sarama/fetch_request.go new file mode 100644 index 000000000000..f893aeff7d52 --- /dev/null +++ b/vendor/github.com/Shopify/sarama/fetch_request.go @@ -0,0 +1,295 @@ +package sarama + +type fetchRequestBlock struct { + Version int16 + currentLeaderEpoch int32 + fetchOffset int64 + logStartOffset int64 + maxBytes int32 +} + +func (b *fetchRequestBlock) encode(pe packetEncoder, version int16) error { + b.Version = version + if b.Version >= 9 { + pe.putInt32(b.currentLeaderEpoch) + } + pe.putInt64(b.fetchOffset) + if b.Version >= 5 { + pe.putInt64(b.logStartOffset) + } + pe.putInt32(b.maxBytes) + return nil +} + +func (b *fetchRequestBlock) decode(pd packetDecoder, version int16) (err error) { + b.Version = version + if b.Version >= 9 { + if b.currentLeaderEpoch, err = pd.getInt32(); err != nil { + return err + } + } + if b.fetchOffset, err = pd.getInt64(); err != nil { + return err + } + if b.Version >= 5 { + if b.logStartOffset, err = pd.getInt64(); err != nil { + return err + } + } + if b.maxBytes, err = pd.getInt32(); err != nil { + return err + } + return nil +} + +// FetchRequest (API key 1) will fetch Kafka messages. Version 3 introduced the MaxBytes field. See +// https://issues.apache.org/jira/browse/KAFKA-2063 for a discussion of the issues leading up to that. The KIP is at +// https://cwiki.apache.org/confluence/display/KAFKA/KIP-74%3A+Add+Fetch+Response+Size+Limit+in+Bytes +type FetchRequest struct { + MaxWaitTime int32 + MinBytes int32 + MaxBytes int32 + Version int16 + Isolation IsolationLevel + SessionID int32 + SessionEpoch int32 + blocks map[string]map[int32]*fetchRequestBlock + forgotten map[string][]int32 + RackID string +} + +type IsolationLevel int8 + +const ( + ReadUncommitted IsolationLevel = iota + ReadCommitted +) + +func (r *FetchRequest) encode(pe packetEncoder) (err error) { + pe.putInt32(-1) // replica ID is always -1 for clients + pe.putInt32(r.MaxWaitTime) + pe.putInt32(r.MinBytes) + if r.Version >= 3 { + pe.putInt32(r.MaxBytes) + } + if r.Version >= 4 { + pe.putInt8(int8(r.Isolation)) + } + if r.Version >= 7 { + pe.putInt32(r.SessionID) + pe.putInt32(r.SessionEpoch) + } + err = pe.putArrayLength(len(r.blocks)) + if err != nil { + return err + } + for topic, blocks := range r.blocks { + err = pe.putString(topic) + if err != nil { + return err + } + err = pe.putArrayLength(len(blocks)) + if err != nil { + return err + } + for partition, block := range blocks { + pe.putInt32(partition) + err = block.encode(pe, r.Version) + if err != nil { + return err + } + } + } + if r.Version >= 7 { + err = pe.putArrayLength(len(r.forgotten)) + if err != nil { + return err + } + for topic, partitions := range r.forgotten { + err = pe.putString(topic) + if err != nil { + return err + } + err = pe.putArrayLength(len(partitions)) + if err != nil { + return err + } + for _, partition := range partitions { + pe.putInt32(partition) + } + } + } + if r.Version >= 11 { + err = pe.putString(r.RackID) + if err != nil { + return err + } + } + + return nil +} + +func (r *FetchRequest) decode(pd packetDecoder, version int16) (err error) { + r.Version = version + + if _, err = pd.getInt32(); err != nil { + return err + } + if r.MaxWaitTime, err = pd.getInt32(); err != nil { + return err + } + if r.MinBytes, err = pd.getInt32(); err != nil { + return err + } + if r.Version >= 3 { + if r.MaxBytes, err = pd.getInt32(); err != nil { + return err + } + } + if r.Version >= 4 { + isolation, err := pd.getInt8() + if err != nil { + return err + } + r.Isolation = IsolationLevel(isolation) + } + if r.Version >= 7 { + r.SessionID, err = pd.getInt32() + if err != nil { + return err + } + r.SessionEpoch, err = pd.getInt32() + if err != nil { + return err + } + } + topicCount, err := pd.getArrayLength() + if err != nil { + return err + } + if topicCount == 0 { + return nil + } + r.blocks = make(map[string]map[int32]*fetchRequestBlock) + for i := 0; i < topicCount; i++ { + topic, err := pd.getString() + if err != nil { + return err + } + partitionCount, err := pd.getArrayLength() + if err != nil { + return err + } + r.blocks[topic] = make(map[int32]*fetchRequestBlock) + for j := 0; j < partitionCount; j++ { + partition, err := pd.getInt32() + if err != nil { + return err + } + fetchBlock := &fetchRequestBlock{} + if err = fetchBlock.decode(pd, r.Version); err != nil { + return err + } + r.blocks[topic][partition] = fetchBlock + } + } + + if r.Version >= 7 { + forgottenCount, err := pd.getArrayLength() + if err != nil { + return err + } + r.forgotten = make(map[string][]int32) + for i := 0; i < forgottenCount; i++ { + topic, err := pd.getString() + if err != nil { + return err + } + partitionCount, err := pd.getArrayLength() + if err != nil { + return err + } + r.forgotten[topic] = make([]int32, partitionCount) + + for j := 0; j < partitionCount; j++ { + partition, err := pd.getInt32() + if err != nil { + return err + } + r.forgotten[topic][j] = partition + } + } + } + + if r.Version >= 11 { + r.RackID, err = pd.getString() + if err != nil { + return err + } + } + + return nil +} + +func (r *FetchRequest) key() int16 { + return 1 +} + +func (r *FetchRequest) version() int16 { + return r.Version +} + +func (r *FetchRequest) headerVersion() int16 { + return 1 +} + +func (r *FetchRequest) requiredVersion() KafkaVersion { + switch r.Version { + case 0: + return MinVersion + case 1: + return V0_9_0_0 + case 2: + return V0_10_0_0 + case 3: + return V0_10_1_0 + case 4, 5: + return V0_11_0_0 + case 6: + return V1_0_0_0 + case 7: + return V1_1_0_0 + case 8: + return V2_0_0_0 + case 9, 10: + return V2_1_0_0 + case 11: + return V2_3_0_0 + default: + return MaxVersion + } +} + +func (r *FetchRequest) AddBlock(topic string, partitionID int32, fetchOffset int64, maxBytes int32) { + if r.blocks == nil { + r.blocks = make(map[string]map[int32]*fetchRequestBlock) + } + + if r.Version >= 7 && r.forgotten == nil { + r.forgotten = make(map[string][]int32) + } + + if r.blocks[topic] == nil { + r.blocks[topic] = make(map[int32]*fetchRequestBlock) + } + + tmp := new(fetchRequestBlock) + tmp.Version = r.Version + tmp.maxBytes = maxBytes + tmp.fetchOffset = fetchOffset + if r.Version >= 9 { + tmp.currentLeaderEpoch = int32(-1) + } + + r.blocks[topic][partitionID] = tmp +} diff --git a/vendor/github.com/Shopify/sarama/fetch_response.go b/vendor/github.com/Shopify/sarama/fetch_response.go new file mode 100644 index 000000000000..54b88284ad98 --- /dev/null +++ b/vendor/github.com/Shopify/sarama/fetch_response.go @@ -0,0 +1,548 @@ +package sarama + +import ( + "sort" + "time" +) + +type AbortedTransaction struct { + ProducerID int64 + FirstOffset int64 +} + +func (t *AbortedTransaction) decode(pd packetDecoder) (err error) { + if t.ProducerID, err = pd.getInt64(); err != nil { + return err + } + + if t.FirstOffset, err = pd.getInt64(); err != nil { + return err + } + + return nil +} + +func (t *AbortedTransaction) encode(pe packetEncoder) (err error) { + pe.putInt64(t.ProducerID) + pe.putInt64(t.FirstOffset) + + return nil +} + +type FetchResponseBlock struct { + Err KError + HighWaterMarkOffset int64 + LastStableOffset int64 + LogStartOffset int64 + AbortedTransactions []*AbortedTransaction + PreferredReadReplica int32 + Records *Records // deprecated: use FetchResponseBlock.RecordsSet + RecordsSet []*Records + Partial bool +} + +func (b *FetchResponseBlock) decode(pd packetDecoder, version int16) (err error) { + tmp, err := pd.getInt16() + if err != nil { + return err + } + b.Err = KError(tmp) + + b.HighWaterMarkOffset, err = pd.getInt64() + if err != nil { + return err + } + + if version >= 4 { + b.LastStableOffset, err = pd.getInt64() + if err != nil { + return err + } + + if version >= 5 { + b.LogStartOffset, err = pd.getInt64() + if err != nil { + return err + } + } + + numTransact, err := pd.getArrayLength() + if err != nil { + return err + } + + if numTransact >= 0 { + b.AbortedTransactions = make([]*AbortedTransaction, numTransact) + } + + for i := 0; i < numTransact; i++ { + transact := new(AbortedTransaction) + if err = transact.decode(pd); err != nil { + return err + } + b.AbortedTransactions[i] = transact + } + } + + if version >= 11 { + b.PreferredReadReplica, err = pd.getInt32() + if err != nil { + return err + } + } else { + b.PreferredReadReplica = -1 + } + + recordsSize, err := pd.getInt32() + if err != nil { + return err + } + + recordsDecoder, err := pd.getSubset(int(recordsSize)) + if err != nil { + return err + } + + b.RecordsSet = []*Records{} + + for recordsDecoder.remaining() > 0 { + records := &Records{} + if err := records.decode(recordsDecoder); err != nil { + // If we have at least one decoded records, this is not an error + if err == ErrInsufficientData { + if len(b.RecordsSet) == 0 { + b.Partial = true + } + break + } + return err + } + + partial, err := records.isPartial() + if err != nil { + return err + } + + n, err := records.numRecords() + if err != nil { + return err + } + + if n > 0 || (partial && len(b.RecordsSet) == 0) { + b.RecordsSet = append(b.RecordsSet, records) + + if b.Records == nil { + b.Records = records + } + } + + overflow, err := records.isOverflow() + if err != nil { + return err + } + + if partial || overflow { + break + } + } + + return nil +} + +func (b *FetchResponseBlock) numRecords() (int, error) { + sum := 0 + + for _, records := range b.RecordsSet { + count, err := records.numRecords() + if err != nil { + return 0, err + } + + sum += count + } + + return sum, nil +} + +func (b *FetchResponseBlock) isPartial() (bool, error) { + if b.Partial { + return true, nil + } + + if len(b.RecordsSet) == 1 { + return b.RecordsSet[0].isPartial() + } + + return false, nil +} + +func (b *FetchResponseBlock) encode(pe packetEncoder, version int16) (err error) { + pe.putInt16(int16(b.Err)) + + pe.putInt64(b.HighWaterMarkOffset) + + if version >= 4 { + pe.putInt64(b.LastStableOffset) + + if version >= 5 { + pe.putInt64(b.LogStartOffset) + } + + if err = pe.putArrayLength(len(b.AbortedTransactions)); err != nil { + return err + } + for _, transact := range b.AbortedTransactions { + if err = transact.encode(pe); err != nil { + return err + } + } + } + + if version >= 11 { + pe.putInt32(b.PreferredReadReplica) + } + + pe.push(&lengthField{}) + for _, records := range b.RecordsSet { + err = records.encode(pe) + if err != nil { + return err + } + } + return pe.pop() +} + +func (b *FetchResponseBlock) getAbortedTransactions() []*AbortedTransaction { + // I can't find any doc that guarantee the field `fetchResponse.AbortedTransactions` is ordered + // plus Java implementation use a PriorityQueue based on `FirstOffset`. I guess we have to order it ourself + at := b.AbortedTransactions + sort.Slice( + at, + func(i, j int) bool { return at[i].FirstOffset < at[j].FirstOffset }, + ) + return at +} + +type FetchResponse struct { + Blocks map[string]map[int32]*FetchResponseBlock + ThrottleTime time.Duration + ErrorCode int16 + SessionID int32 + Version int16 + LogAppendTime bool + Timestamp time.Time +} + +func (r *FetchResponse) decode(pd packetDecoder, version int16) (err error) { + r.Version = version + + if r.Version >= 1 { + throttle, err := pd.getInt32() + if err != nil { + return err + } + r.ThrottleTime = time.Duration(throttle) * time.Millisecond + } + + if r.Version >= 7 { + r.ErrorCode, err = pd.getInt16() + if err != nil { + return err + } + r.SessionID, err = pd.getInt32() + if err != nil { + return err + } + } + + numTopics, err := pd.getArrayLength() + if err != nil { + return err + } + + r.Blocks = make(map[string]map[int32]*FetchResponseBlock, numTopics) + for i := 0; i < numTopics; i++ { + name, err := pd.getString() + if err != nil { + return err + } + + numBlocks, err := pd.getArrayLength() + if err != nil { + return err + } + + r.Blocks[name] = make(map[int32]*FetchResponseBlock, numBlocks) + + for j := 0; j < numBlocks; j++ { + id, err := pd.getInt32() + if err != nil { + return err + } + + block := new(FetchResponseBlock) + err = block.decode(pd, version) + if err != nil { + return err + } + r.Blocks[name][id] = block + } + } + + return nil +} + +func (r *FetchResponse) encode(pe packetEncoder) (err error) { + if r.Version >= 1 { + pe.putInt32(int32(r.ThrottleTime / time.Millisecond)) + } + + if r.Version >= 7 { + pe.putInt16(r.ErrorCode) + pe.putInt32(r.SessionID) + } + + err = pe.putArrayLength(len(r.Blocks)) + if err != nil { + return err + } + + for topic, partitions := range r.Blocks { + err = pe.putString(topic) + if err != nil { + return err + } + + err = pe.putArrayLength(len(partitions)) + if err != nil { + return err + } + + for id, block := range partitions { + pe.putInt32(id) + err = block.encode(pe, r.Version) + if err != nil { + return err + } + } + } + return nil +} + +func (r *FetchResponse) key() int16 { + return 1 +} + +func (r *FetchResponse) version() int16 { + return r.Version +} + +func (r *FetchResponse) headerVersion() int16 { + return 0 +} + +func (r *FetchResponse) requiredVersion() KafkaVersion { + switch r.Version { + case 0: + return MinVersion + case 1: + return V0_9_0_0 + case 2: + return V0_10_0_0 + case 3: + return V0_10_1_0 + case 4, 5: + return V0_11_0_0 + case 6: + return V1_0_0_0 + case 7: + return V1_1_0_0 + case 8: + return V2_0_0_0 + case 9, 10: + return V2_1_0_0 + case 11: + return V2_3_0_0 + default: + return MaxVersion + } +} + +func (r *FetchResponse) GetBlock(topic string, partition int32) *FetchResponseBlock { + if r.Blocks == nil { + return nil + } + + if r.Blocks[topic] == nil { + return nil + } + + return r.Blocks[topic][partition] +} + +func (r *FetchResponse) AddError(topic string, partition int32, err KError) { + if r.Blocks == nil { + r.Blocks = make(map[string]map[int32]*FetchResponseBlock) + } + partitions, ok := r.Blocks[topic] + if !ok { + partitions = make(map[int32]*FetchResponseBlock) + r.Blocks[topic] = partitions + } + frb, ok := partitions[partition] + if !ok { + frb = new(FetchResponseBlock) + partitions[partition] = frb + } + frb.Err = err +} + +func (r *FetchResponse) getOrCreateBlock(topic string, partition int32) *FetchResponseBlock { + if r.Blocks == nil { + r.Blocks = make(map[string]map[int32]*FetchResponseBlock) + } + partitions, ok := r.Blocks[topic] + if !ok { + partitions = make(map[int32]*FetchResponseBlock) + r.Blocks[topic] = partitions + } + frb, ok := partitions[partition] + if !ok { + frb = new(FetchResponseBlock) + partitions[partition] = frb + } + + return frb +} + +func encodeKV(key, value Encoder) ([]byte, []byte) { + var kb []byte + var vb []byte + if key != nil { + kb, _ = key.Encode() + } + if value != nil { + vb, _ = value.Encode() + } + + return kb, vb +} + +func (r *FetchResponse) AddMessageWithTimestamp(topic string, partition int32, key, value Encoder, offset int64, timestamp time.Time, version int8) { + frb := r.getOrCreateBlock(topic, partition) + kb, vb := encodeKV(key, value) + if r.LogAppendTime { + timestamp = r.Timestamp + } + msg := &Message{Key: kb, Value: vb, LogAppendTime: r.LogAppendTime, Timestamp: timestamp, Version: version} + msgBlock := &MessageBlock{Msg: msg, Offset: offset} + if len(frb.RecordsSet) == 0 { + records := newLegacyRecords(&MessageSet{}) + frb.RecordsSet = []*Records{&records} + } + set := frb.RecordsSet[0].MsgSet + set.Messages = append(set.Messages, msgBlock) +} + +func (r *FetchResponse) AddRecordWithTimestamp(topic string, partition int32, key, value Encoder, offset int64, timestamp time.Time) { + frb := r.getOrCreateBlock(topic, partition) + kb, vb := encodeKV(key, value) + if len(frb.RecordsSet) == 0 { + records := newDefaultRecords(&RecordBatch{Version: 2, LogAppendTime: r.LogAppendTime, FirstTimestamp: timestamp, MaxTimestamp: r.Timestamp}) + frb.RecordsSet = []*Records{&records} + } + batch := frb.RecordsSet[0].RecordBatch + rec := &Record{Key: kb, Value: vb, OffsetDelta: offset, TimestampDelta: timestamp.Sub(batch.FirstTimestamp)} + batch.addRecord(rec) +} + +// AddRecordBatchWithTimestamp is similar to AddRecordWithTimestamp +// But instead of appending 1 record to a batch, it append a new batch containing 1 record to the fetchResponse +// Since transaction are handled on batch level (the whole batch is either committed or aborted), use this to test transactions +func (r *FetchResponse) AddRecordBatchWithTimestamp(topic string, partition int32, key, value Encoder, offset int64, producerID int64, isTransactional bool, timestamp time.Time) { + frb := r.getOrCreateBlock(topic, partition) + kb, vb := encodeKV(key, value) + + records := newDefaultRecords(&RecordBatch{Version: 2, LogAppendTime: r.LogAppendTime, FirstTimestamp: timestamp, MaxTimestamp: r.Timestamp}) + batch := &RecordBatch{ + Version: 2, + LogAppendTime: r.LogAppendTime, + FirstTimestamp: timestamp, + MaxTimestamp: r.Timestamp, + FirstOffset: offset, + LastOffsetDelta: 0, + ProducerID: producerID, + IsTransactional: isTransactional, + } + rec := &Record{Key: kb, Value: vb, OffsetDelta: 0, TimestampDelta: timestamp.Sub(batch.FirstTimestamp)} + batch.addRecord(rec) + records.RecordBatch = batch + + frb.RecordsSet = append(frb.RecordsSet, &records) +} + +func (r *FetchResponse) AddControlRecordWithTimestamp(topic string, partition int32, offset int64, producerID int64, recordType ControlRecordType, timestamp time.Time) { + frb := r.getOrCreateBlock(topic, partition) + + // batch + batch := &RecordBatch{ + Version: 2, + LogAppendTime: r.LogAppendTime, + FirstTimestamp: timestamp, + MaxTimestamp: r.Timestamp, + FirstOffset: offset, + LastOffsetDelta: 0, + ProducerID: producerID, + IsTransactional: true, + Control: true, + } + + // records + records := newDefaultRecords(nil) + records.RecordBatch = batch + + // record + crAbort := ControlRecord{ + Version: 0, + Type: recordType, + } + crKey := &realEncoder{raw: make([]byte, 4)} + crValue := &realEncoder{raw: make([]byte, 6)} + crAbort.encode(crKey, crValue) + rec := &Record{Key: ByteEncoder(crKey.raw), Value: ByteEncoder(crValue.raw), OffsetDelta: 0, TimestampDelta: timestamp.Sub(batch.FirstTimestamp)} + batch.addRecord(rec) + + frb.RecordsSet = append(frb.RecordsSet, &records) +} + +func (r *FetchResponse) AddMessage(topic string, partition int32, key, value Encoder, offset int64) { + r.AddMessageWithTimestamp(topic, partition, key, value, offset, time.Time{}, 0) +} + +func (r *FetchResponse) AddRecord(topic string, partition int32, key, value Encoder, offset int64) { + r.AddRecordWithTimestamp(topic, partition, key, value, offset, time.Time{}) +} + +func (r *FetchResponse) AddRecordBatch(topic string, partition int32, key, value Encoder, offset int64, producerID int64, isTransactional bool) { + r.AddRecordBatchWithTimestamp(topic, partition, key, value, offset, producerID, isTransactional, time.Time{}) +} + +func (r *FetchResponse) AddControlRecord(topic string, partition int32, offset int64, producerID int64, recordType ControlRecordType) { + // define controlRecord key and value + r.AddControlRecordWithTimestamp(topic, partition, offset, producerID, recordType, time.Time{}) +} + +func (r *FetchResponse) SetLastOffsetDelta(topic string, partition int32, offset int32) { + frb := r.getOrCreateBlock(topic, partition) + if len(frb.RecordsSet) == 0 { + records := newDefaultRecords(&RecordBatch{Version: 2}) + frb.RecordsSet = []*Records{&records} + } + batch := frb.RecordsSet[0].RecordBatch + batch.LastOffsetDelta = offset +} + +func (r *FetchResponse) SetLastStableOffset(topic string, partition int32, offset int64) { + frb := r.getOrCreateBlock(topic, partition) + frb.LastStableOffset = offset +} diff --git a/vendor/github.com/Shopify/sarama/find_coordinator_request.go b/vendor/github.com/Shopify/sarama/find_coordinator_request.go new file mode 100644 index 000000000000..597bcbf786f5 --- /dev/null +++ b/vendor/github.com/Shopify/sarama/find_coordinator_request.go @@ -0,0 +1,65 @@ +package sarama + +type CoordinatorType int8 + +const ( + CoordinatorGroup CoordinatorType = iota + CoordinatorTransaction +) + +type FindCoordinatorRequest struct { + Version int16 + CoordinatorKey string + CoordinatorType CoordinatorType +} + +func (f *FindCoordinatorRequest) encode(pe packetEncoder) error { + if err := pe.putString(f.CoordinatorKey); err != nil { + return err + } + + if f.Version >= 1 { + pe.putInt8(int8(f.CoordinatorType)) + } + + return nil +} + +func (f *FindCoordinatorRequest) decode(pd packetDecoder, version int16) (err error) { + if f.CoordinatorKey, err = pd.getString(); err != nil { + return err + } + + if version >= 1 { + f.Version = version + coordinatorType, err := pd.getInt8() + if err != nil { + return err + } + + f.CoordinatorType = CoordinatorType(coordinatorType) + } + + return nil +} + +func (f *FindCoordinatorRequest) key() int16 { + return 10 +} + +func (f *FindCoordinatorRequest) version() int16 { + return f.Version +} + +func (r *FindCoordinatorRequest) headerVersion() int16 { + return 1 +} + +func (f *FindCoordinatorRequest) requiredVersion() KafkaVersion { + switch f.Version { + case 1: + return V0_11_0_0 + default: + return V0_8_2_0 + } +} diff --git a/vendor/github.com/Shopify/sarama/find_coordinator_response.go b/vendor/github.com/Shopify/sarama/find_coordinator_response.go new file mode 100644 index 000000000000..83a648ad4aec --- /dev/null +++ b/vendor/github.com/Shopify/sarama/find_coordinator_response.go @@ -0,0 +1,96 @@ +package sarama + +import ( + "time" +) + +var NoNode = &Broker{id: -1, addr: ":-1"} + +type FindCoordinatorResponse struct { + Version int16 + ThrottleTime time.Duration + Err KError + ErrMsg *string + Coordinator *Broker +} + +func (f *FindCoordinatorResponse) decode(pd packetDecoder, version int16) (err error) { + if version >= 1 { + f.Version = version + + throttleTime, err := pd.getInt32() + if err != nil { + return err + } + f.ThrottleTime = time.Duration(throttleTime) * time.Millisecond + } + + tmp, err := pd.getInt16() + if err != nil { + return err + } + f.Err = KError(tmp) + + if version >= 1 { + if f.ErrMsg, err = pd.getNullableString(); err != nil { + return err + } + } + + coordinator := new(Broker) + // The version is hardcoded to 0, as version 1 of the Broker-decode + // contains the rack-field which is not present in the FindCoordinatorResponse. + if err := coordinator.decode(pd, 0); err != nil { + return err + } + if coordinator.addr == ":0" { + return nil + } + f.Coordinator = coordinator + + return nil +} + +func (f *FindCoordinatorResponse) encode(pe packetEncoder) error { + if f.Version >= 1 { + pe.putInt32(int32(f.ThrottleTime / time.Millisecond)) + } + + pe.putInt16(int16(f.Err)) + + if f.Version >= 1 { + if err := pe.putNullableString(f.ErrMsg); err != nil { + return err + } + } + + coordinator := f.Coordinator + if coordinator == nil { + coordinator = NoNode + } + if err := coordinator.encode(pe, 0); err != nil { + return err + } + return nil +} + +func (f *FindCoordinatorResponse) key() int16 { + return 10 +} + +func (f *FindCoordinatorResponse) version() int16 { + return f.Version +} + +func (r *FindCoordinatorResponse) headerVersion() int16 { + return 0 +} + +func (f *FindCoordinatorResponse) requiredVersion() KafkaVersion { + switch f.Version { + case 1: + return V0_11_0_0 + default: + return V0_8_2_0 + } +} diff --git a/vendor/github.com/Shopify/sarama/gssapi_kerberos.go b/vendor/github.com/Shopify/sarama/gssapi_kerberos.go new file mode 100644 index 000000000000..ab8b70196f82 --- /dev/null +++ b/vendor/github.com/Shopify/sarama/gssapi_kerberos.go @@ -0,0 +1,253 @@ +package sarama + +import ( + "encoding/binary" + "errors" + "fmt" + "io" + "math" + "strings" + "time" + + "github.com/jcmturner/gofork/encoding/asn1" + "github.com/jcmturner/gokrb5/v8/asn1tools" + "github.com/jcmturner/gokrb5/v8/gssapi" + "github.com/jcmturner/gokrb5/v8/iana/chksumtype" + "github.com/jcmturner/gokrb5/v8/iana/keyusage" + "github.com/jcmturner/gokrb5/v8/messages" + "github.com/jcmturner/gokrb5/v8/types" +) + +const ( + TOK_ID_KRB_AP_REQ = 256 + GSS_API_GENERIC_TAG = 0x60 + KRB5_USER_AUTH = 1 + KRB5_KEYTAB_AUTH = 2 + GSS_API_INITIAL = 1 + GSS_API_VERIFY = 2 + GSS_API_FINISH = 3 +) + +type GSSAPIConfig struct { + AuthType int + KeyTabPath string + KerberosConfigPath string + ServiceName string + Username string + Password string + Realm string + DisablePAFXFAST bool +} + +type GSSAPIKerberosAuth struct { + Config *GSSAPIConfig + ticket messages.Ticket + encKey types.EncryptionKey + NewKerberosClientFunc func(config *GSSAPIConfig) (KerberosClient, error) + step int +} + +type KerberosClient interface { + Login() error + GetServiceTicket(spn string) (messages.Ticket, types.EncryptionKey, error) + Domain() string + CName() types.PrincipalName + Destroy() +} + +// writePackage appends length in big endian before the payload, and sends it to kafka +func (krbAuth *GSSAPIKerberosAuth) writePackage(broker *Broker, payload []byte) (int, error) { + length := uint64(len(payload)) + size := length + 4 // 4 byte length header + payload + if size > math.MaxInt32 { + return 0, errors.New("payload too large, will overflow int32") + } + finalPackage := make([]byte, size) + copy(finalPackage[4:], payload) + binary.BigEndian.PutUint32(finalPackage, uint32(length)) + bytes, err := broker.conn.Write(finalPackage) + if err != nil { + return bytes, err + } + return bytes, nil +} + +// readPackage reads payload length (4 bytes) and then reads the payload into []byte +func (krbAuth *GSSAPIKerberosAuth) readPackage(broker *Broker) ([]byte, int, error) { + bytesRead := 0 + lengthInBytes := make([]byte, 4) + bytes, err := io.ReadFull(broker.conn, lengthInBytes) + if err != nil { + return nil, bytesRead, err + } + bytesRead += bytes + payloadLength := binary.BigEndian.Uint32(lengthInBytes) + payloadBytes := make([]byte, payloadLength) // buffer for read.. + bytes, err = io.ReadFull(broker.conn, payloadBytes) // read bytes + if err != nil { + return payloadBytes, bytesRead, err + } + bytesRead += bytes + return payloadBytes, bytesRead, nil +} + +func (krbAuth *GSSAPIKerberosAuth) newAuthenticatorChecksum() []byte { + a := make([]byte, 24) + flags := []int{gssapi.ContextFlagInteg, gssapi.ContextFlagConf} + binary.LittleEndian.PutUint32(a[:4], 16) + for _, i := range flags { + f := binary.LittleEndian.Uint32(a[20:24]) + f |= uint32(i) + binary.LittleEndian.PutUint32(a[20:24], f) + } + return a +} + +/* +* +* Construct Kerberos AP_REQ package, conforming to RFC-4120 +* https://tools.ietf.org/html/rfc4120#page-84 +* + */ +func (krbAuth *GSSAPIKerberosAuth) createKrb5Token( + domain string, cname types.PrincipalName, + ticket messages.Ticket, + sessionKey types.EncryptionKey) ([]byte, error) { + auth, err := types.NewAuthenticator(domain, cname) + if err != nil { + return nil, err + } + auth.Cksum = types.Checksum{ + CksumType: chksumtype.GSSAPI, + Checksum: krbAuth.newAuthenticatorChecksum(), + } + APReq, err := messages.NewAPReq( + ticket, + sessionKey, + auth, + ) + if err != nil { + return nil, err + } + aprBytes := make([]byte, 2) + binary.BigEndian.PutUint16(aprBytes, TOK_ID_KRB_AP_REQ) + tb, err := APReq.Marshal() + if err != nil { + return nil, err + } + aprBytes = append(aprBytes, tb...) + return aprBytes, nil +} + +/* +* +* Append the GSS-API header to the payload, conforming to RFC-2743 +* Section 3.1, Mechanism-Independent Token Format +* +* https://tools.ietf.org/html/rfc2743#page-81 +* +* GSSAPIHeader + +* + */ +func (krbAuth *GSSAPIKerberosAuth) appendGSSAPIHeader(payload []byte) ([]byte, error) { + oidBytes, err := asn1.Marshal(gssapi.OIDKRB5.OID()) + if err != nil { + return nil, err + } + tkoLengthBytes := asn1tools.MarshalLengthBytes(len(oidBytes) + len(payload)) + GSSHeader := append([]byte{GSS_API_GENERIC_TAG}, tkoLengthBytes...) + GSSHeader = append(GSSHeader, oidBytes...) + GSSPackage := append(GSSHeader, payload...) + return GSSPackage, nil +} + +func (krbAuth *GSSAPIKerberosAuth) initSecContext(bytes []byte, kerberosClient KerberosClient) ([]byte, error) { + switch krbAuth.step { + case GSS_API_INITIAL: + aprBytes, err := krbAuth.createKrb5Token( + kerberosClient.Domain(), + kerberosClient.CName(), + krbAuth.ticket, + krbAuth.encKey) + if err != nil { + return nil, err + } + krbAuth.step = GSS_API_VERIFY + return krbAuth.appendGSSAPIHeader(aprBytes) + case GSS_API_VERIFY: + wrapTokenReq := gssapi.WrapToken{} + if err := wrapTokenReq.Unmarshal(bytes, true); err != nil { + return nil, err + } + // Validate response. + isValid, err := wrapTokenReq.Verify(krbAuth.encKey, keyusage.GSSAPI_ACCEPTOR_SEAL) + if !isValid { + return nil, err + } + + wrapTokenResponse, err := gssapi.NewInitiatorWrapToken(wrapTokenReq.Payload, krbAuth.encKey) + if err != nil { + return nil, err + } + krbAuth.step = GSS_API_FINISH + return wrapTokenResponse.Marshal() + } + return nil, nil +} + +/* This does the handshake for authorization */ +func (krbAuth *GSSAPIKerberosAuth) Authorize(broker *Broker) error { + kerberosClient, err := krbAuth.NewKerberosClientFunc(krbAuth.Config) + if err != nil { + Logger.Printf("Kerberos client error: %s", err) + return err + } + + err = kerberosClient.Login() + if err != nil { + Logger.Printf("Kerberos client error: %s", err) + return err + } + // Construct SPN using serviceName and host + // SPN format: / + + host := strings.SplitN(broker.addr, ":", 2)[0] // Strip port part + spn := fmt.Sprintf("%s/%s", broker.conf.Net.SASL.GSSAPI.ServiceName, host) + + ticket, encKey, err := kerberosClient.GetServiceTicket(spn) + if err != nil { + Logger.Printf("Error getting Kerberos service ticket : %s", err) + return err + } + krbAuth.ticket = ticket + krbAuth.encKey = encKey + krbAuth.step = GSS_API_INITIAL + var receivedBytes []byte = nil + defer kerberosClient.Destroy() + for { + packBytes, err := krbAuth.initSecContext(receivedBytes, kerberosClient) + if err != nil { + Logger.Printf("Error while performing GSSAPI Kerberos Authentication: %s\n", err) + return err + } + requestTime := time.Now() + bytesWritten, err := krbAuth.writePackage(broker, packBytes) + if err != nil { + Logger.Printf("Error while performing GSSAPI Kerberos Authentication: %s\n", err) + return err + } + broker.updateOutgoingCommunicationMetrics(bytesWritten) + if krbAuth.step == GSS_API_VERIFY { + bytesRead := 0 + receivedBytes, bytesRead, err = krbAuth.readPackage(broker) + requestLatency := time.Since(requestTime) + broker.updateIncomingCommunicationMetrics(bytesRead, requestLatency) + if err != nil { + Logger.Printf("Error while performing GSSAPI Kerberos Authentication: %s\n", err) + return err + } + } else if krbAuth.step == GSS_API_FINISH { + return nil + } + } +} diff --git a/vendor/github.com/Shopify/sarama/heartbeat_request.go b/vendor/github.com/Shopify/sarama/heartbeat_request.go new file mode 100644 index 000000000000..e9d9af191102 --- /dev/null +++ b/vendor/github.com/Shopify/sarama/heartbeat_request.go @@ -0,0 +1,51 @@ +package sarama + +type HeartbeatRequest struct { + GroupId string + GenerationId int32 + MemberId string +} + +func (r *HeartbeatRequest) encode(pe packetEncoder) error { + if err := pe.putString(r.GroupId); err != nil { + return err + } + + pe.putInt32(r.GenerationId) + + if err := pe.putString(r.MemberId); err != nil { + return err + } + + return nil +} + +func (r *HeartbeatRequest) decode(pd packetDecoder, version int16) (err error) { + if r.GroupId, err = pd.getString(); err != nil { + return + } + if r.GenerationId, err = pd.getInt32(); err != nil { + return + } + if r.MemberId, err = pd.getString(); err != nil { + return + } + + return nil +} + +func (r *HeartbeatRequest) key() int16 { + return 12 +} + +func (r *HeartbeatRequest) version() int16 { + return 0 +} + +func (r *HeartbeatRequest) headerVersion() int16 { + return 1 +} + +func (r *HeartbeatRequest) requiredVersion() KafkaVersion { + return V0_9_0_0 +} diff --git a/vendor/github.com/Shopify/sarama/heartbeat_response.go b/vendor/github.com/Shopify/sarama/heartbeat_response.go new file mode 100644 index 000000000000..577ab72e5742 --- /dev/null +++ b/vendor/github.com/Shopify/sarama/heartbeat_response.go @@ -0,0 +1,36 @@ +package sarama + +type HeartbeatResponse struct { + Err KError +} + +func (r *HeartbeatResponse) encode(pe packetEncoder) error { + pe.putInt16(int16(r.Err)) + return nil +} + +func (r *HeartbeatResponse) decode(pd packetDecoder, version int16) error { + kerr, err := pd.getInt16() + if err != nil { + return err + } + r.Err = KError(kerr) + + return nil +} + +func (r *HeartbeatResponse) key() int16 { + return 12 +} + +func (r *HeartbeatResponse) version() int16 { + return 0 +} + +func (r *HeartbeatResponse) headerVersion() int16 { + return 0 +} + +func (r *HeartbeatResponse) requiredVersion() KafkaVersion { + return V0_9_0_0 +} diff --git a/vendor/github.com/Shopify/sarama/incremental_alter_configs_request.go b/vendor/github.com/Shopify/sarama/incremental_alter_configs_request.go new file mode 100644 index 000000000000..c4d05a972041 --- /dev/null +++ b/vendor/github.com/Shopify/sarama/incremental_alter_configs_request.go @@ -0,0 +1,173 @@ +package sarama + +type IncrementalAlterConfigsOperation int8 + +const ( + IncrementalAlterConfigsOperationSet IncrementalAlterConfigsOperation = iota + IncrementalAlterConfigsOperationDelete + IncrementalAlterConfigsOperationAppend + IncrementalAlterConfigsOperationSubtract +) + +// IncrementalAlterConfigsRequest is an incremental alter config request type +type IncrementalAlterConfigsRequest struct { + Resources []*IncrementalAlterConfigsResource + ValidateOnly bool +} + +type IncrementalAlterConfigsResource struct { + Type ConfigResourceType + Name string + ConfigEntries map[string]IncrementalAlterConfigsEntry +} + +type IncrementalAlterConfigsEntry struct { + Operation IncrementalAlterConfigsOperation + Value *string +} + +func (a *IncrementalAlterConfigsRequest) encode(pe packetEncoder) error { + if err := pe.putArrayLength(len(a.Resources)); err != nil { + return err + } + + for _, r := range a.Resources { + if err := r.encode(pe); err != nil { + return err + } + } + + pe.putBool(a.ValidateOnly) + return nil +} + +func (a *IncrementalAlterConfigsRequest) decode(pd packetDecoder, version int16) error { + resourceCount, err := pd.getArrayLength() + if err != nil { + return err + } + + a.Resources = make([]*IncrementalAlterConfigsResource, resourceCount) + for i := range a.Resources { + r := &IncrementalAlterConfigsResource{} + err = r.decode(pd, version) + if err != nil { + return err + } + a.Resources[i] = r + } + + validateOnly, err := pd.getBool() + if err != nil { + return err + } + + a.ValidateOnly = validateOnly + + return nil +} + +func (a *IncrementalAlterConfigsResource) encode(pe packetEncoder) error { + pe.putInt8(int8(a.Type)) + + if err := pe.putString(a.Name); err != nil { + return err + } + + if err := pe.putArrayLength(len(a.ConfigEntries)); err != nil { + return err + } + + for name, e := range a.ConfigEntries { + if err := pe.putString(name); err != nil { + return err + } + + if err := e.encode(pe); err != nil { + return err + } + } + + return nil +} + +func (a *IncrementalAlterConfigsResource) decode(pd packetDecoder, version int16) error { + t, err := pd.getInt8() + if err != nil { + return err + } + a.Type = ConfigResourceType(t) + + name, err := pd.getString() + if err != nil { + return err + } + a.Name = name + + n, err := pd.getArrayLength() + if err != nil { + return err + } + + if n > 0 { + a.ConfigEntries = make(map[string]IncrementalAlterConfigsEntry, n) + for i := 0; i < n; i++ { + name, err := pd.getString() + if err != nil { + return err + } + + var v IncrementalAlterConfigsEntry + + if err := v.decode(pd, version); err != nil { + return err + } + + a.ConfigEntries[name] = v + } + } + return err +} + +func (a *IncrementalAlterConfigsEntry) encode(pe packetEncoder) error { + pe.putInt8(int8(a.Operation)) + + if err := pe.putNullableString(a.Value); err != nil { + return err + } + + return nil +} + +func (a *IncrementalAlterConfigsEntry) decode(pd packetDecoder, version int16) error { + t, err := pd.getInt8() + if err != nil { + return err + } + a.Operation = IncrementalAlterConfigsOperation(t) + + s, err := pd.getNullableString() + if err != nil { + return err + } + + a.Value = s + + return nil +} + +func (a *IncrementalAlterConfigsRequest) key() int16 { + return 44 +} + +func (a *IncrementalAlterConfigsRequest) version() int16 { + return 0 +} + +func (a *IncrementalAlterConfigsRequest) headerVersion() int16 { + return 1 +} + +func (a *IncrementalAlterConfigsRequest) requiredVersion() KafkaVersion { + return V2_3_0_0 +} diff --git a/vendor/github.com/Shopify/sarama/incremental_alter_configs_response.go b/vendor/github.com/Shopify/sarama/incremental_alter_configs_response.go new file mode 100644 index 000000000000..3e8c4500c324 --- /dev/null +++ b/vendor/github.com/Shopify/sarama/incremental_alter_configs_response.go @@ -0,0 +1,66 @@ +package sarama + +import "time" + +// IncrementalAlterConfigsResponse is a response type for incremental alter config +type IncrementalAlterConfigsResponse struct { + ThrottleTime time.Duration + Resources []*AlterConfigsResourceResponse +} + +func (a *IncrementalAlterConfigsResponse) encode(pe packetEncoder) error { + pe.putInt32(int32(a.ThrottleTime / time.Millisecond)) + + if err := pe.putArrayLength(len(a.Resources)); err != nil { + return err + } + + for _, v := range a.Resources { + if err := v.encode(pe); err != nil { + return err + } + } + + return nil +} + +func (a *IncrementalAlterConfigsResponse) decode(pd packetDecoder, version int16) error { + throttleTime, err := pd.getInt32() + if err != nil { + return err + } + a.ThrottleTime = time.Duration(throttleTime) * time.Millisecond + + responseCount, err := pd.getArrayLength() + if err != nil { + return err + } + + a.Resources = make([]*AlterConfigsResourceResponse, responseCount) + + for i := range a.Resources { + a.Resources[i] = new(AlterConfigsResourceResponse) + + if err := a.Resources[i].decode(pd, version); err != nil { + return err + } + } + + return nil +} + +func (a *IncrementalAlterConfigsResponse) key() int16 { + return 44 +} + +func (a *IncrementalAlterConfigsResponse) version() int16 { + return 0 +} + +func (a *IncrementalAlterConfigsResponse) headerVersion() int16 { + return 0 +} + +func (a *IncrementalAlterConfigsResponse) requiredVersion() KafkaVersion { + return V2_3_0_0 +} diff --git a/vendor/github.com/Shopify/sarama/init_producer_id_request.go b/vendor/github.com/Shopify/sarama/init_producer_id_request.go new file mode 100644 index 000000000000..689444397d6f --- /dev/null +++ b/vendor/github.com/Shopify/sarama/init_producer_id_request.go @@ -0,0 +1,47 @@ +package sarama + +import "time" + +type InitProducerIDRequest struct { + TransactionalID *string + TransactionTimeout time.Duration +} + +func (i *InitProducerIDRequest) encode(pe packetEncoder) error { + if err := pe.putNullableString(i.TransactionalID); err != nil { + return err + } + pe.putInt32(int32(i.TransactionTimeout / time.Millisecond)) + + return nil +} + +func (i *InitProducerIDRequest) decode(pd packetDecoder, version int16) (err error) { + if i.TransactionalID, err = pd.getNullableString(); err != nil { + return err + } + + timeout, err := pd.getInt32() + if err != nil { + return err + } + i.TransactionTimeout = time.Duration(timeout) * time.Millisecond + + return nil +} + +func (i *InitProducerIDRequest) key() int16 { + return 22 +} + +func (i *InitProducerIDRequest) version() int16 { + return 0 +} + +func (i *InitProducerIDRequest) headerVersion() int16 { + return 1 +} + +func (i *InitProducerIDRequest) requiredVersion() KafkaVersion { + return V0_11_0_0 +} diff --git a/vendor/github.com/Shopify/sarama/init_producer_id_response.go b/vendor/github.com/Shopify/sarama/init_producer_id_response.go new file mode 100644 index 000000000000..3e1242bf6224 --- /dev/null +++ b/vendor/github.com/Shopify/sarama/init_producer_id_response.go @@ -0,0 +1,59 @@ +package sarama + +import "time" + +type InitProducerIDResponse struct { + ThrottleTime time.Duration + Err KError + ProducerID int64 + ProducerEpoch int16 +} + +func (i *InitProducerIDResponse) encode(pe packetEncoder) error { + pe.putInt32(int32(i.ThrottleTime / time.Millisecond)) + pe.putInt16(int16(i.Err)) + pe.putInt64(i.ProducerID) + pe.putInt16(i.ProducerEpoch) + + return nil +} + +func (i *InitProducerIDResponse) decode(pd packetDecoder, version int16) (err error) { + throttleTime, err := pd.getInt32() + if err != nil { + return err + } + i.ThrottleTime = time.Duration(throttleTime) * time.Millisecond + + kerr, err := pd.getInt16() + if err != nil { + return err + } + i.Err = KError(kerr) + + if i.ProducerID, err = pd.getInt64(); err != nil { + return err + } + + if i.ProducerEpoch, err = pd.getInt16(); err != nil { + return err + } + + return nil +} + +func (i *InitProducerIDResponse) key() int16 { + return 22 +} + +func (i *InitProducerIDResponse) version() int16 { + return 0 +} + +func (i *InitProducerIDResponse) headerVersion() int16 { + return 0 +} + +func (i *InitProducerIDResponse) requiredVersion() KafkaVersion { + return V0_11_0_0 +} diff --git a/vendor/github.com/Shopify/sarama/interceptors.go b/vendor/github.com/Shopify/sarama/interceptors.go new file mode 100644 index 000000000000..d0d33e526f8e --- /dev/null +++ b/vendor/github.com/Shopify/sarama/interceptors.go @@ -0,0 +1,43 @@ +package sarama + +// ProducerInterceptor allows you to intercept (and possibly mutate) the records +// received by the producer before they are published to the Kafka cluster. +// https://cwiki.apache.org/confluence/display/KAFKA/KIP-42%3A+Add+Producer+and+Consumer+Interceptors#KIP42:AddProducerandConsumerInterceptors-Motivation +type ProducerInterceptor interface { + + // OnSend is called when the producer message is intercepted. Please avoid + // modifying the message until it's safe to do so, as this is _not_ a copy + // of the message. + OnSend(*ProducerMessage) +} + +// ConsumerInterceptor allows you to intercept (and possibly mutate) the records +// received by the consumer before they are sent to the messages channel. +// https://cwiki.apache.org/confluence/display/KAFKA/KIP-42%3A+Add+Producer+and+Consumer+Interceptors#KIP42:AddProducerandConsumerInterceptors-Motivation +type ConsumerInterceptor interface { + + // OnConsume is called when the consumed message is intercepted. Please + // avoid modifying the message until it's safe to do so, as this is _not_ a + // copy of the message. + OnConsume(*ConsumerMessage) +} + +func (msg *ProducerMessage) safelyApplyInterceptor(interceptor ProducerInterceptor) { + defer func() { + if r := recover(); r != nil { + Logger.Printf("Error when calling producer interceptor: %s, %w\n", interceptor, r) + } + }() + + interceptor.OnSend(msg) +} + +func (msg *ConsumerMessage) safelyApplyInterceptor(interceptor ConsumerInterceptor) { + defer func() { + if r := recover(); r != nil { + Logger.Printf("Error when calling consumer interceptor: %s, %w\n", interceptor, r) + } + }() + + interceptor.OnConsume(msg) +} diff --git a/vendor/github.com/Shopify/sarama/join_group_request.go b/vendor/github.com/Shopify/sarama/join_group_request.go new file mode 100644 index 000000000000..3734e82e4068 --- /dev/null +++ b/vendor/github.com/Shopify/sarama/join_group_request.go @@ -0,0 +1,167 @@ +package sarama + +type GroupProtocol struct { + Name string + Metadata []byte +} + +func (p *GroupProtocol) decode(pd packetDecoder) (err error) { + p.Name, err = pd.getString() + if err != nil { + return err + } + p.Metadata, err = pd.getBytes() + return err +} + +func (p *GroupProtocol) encode(pe packetEncoder) (err error) { + if err := pe.putString(p.Name); err != nil { + return err + } + if err := pe.putBytes(p.Metadata); err != nil { + return err + } + return nil +} + +type JoinGroupRequest struct { + Version int16 + GroupId string + SessionTimeout int32 + RebalanceTimeout int32 + MemberId string + ProtocolType string + GroupProtocols map[string][]byte // deprecated; use OrderedGroupProtocols + OrderedGroupProtocols []*GroupProtocol +} + +func (r *JoinGroupRequest) encode(pe packetEncoder) error { + if err := pe.putString(r.GroupId); err != nil { + return err + } + pe.putInt32(r.SessionTimeout) + if r.Version >= 1 { + pe.putInt32(r.RebalanceTimeout) + } + if err := pe.putString(r.MemberId); err != nil { + return err + } + if err := pe.putString(r.ProtocolType); err != nil { + return err + } + + if len(r.GroupProtocols) > 0 { + if len(r.OrderedGroupProtocols) > 0 { + return PacketDecodingError{"cannot specify both GroupProtocols and OrderedGroupProtocols on JoinGroupRequest"} + } + + if err := pe.putArrayLength(len(r.GroupProtocols)); err != nil { + return err + } + for name, metadata := range r.GroupProtocols { + if err := pe.putString(name); err != nil { + return err + } + if err := pe.putBytes(metadata); err != nil { + return err + } + } + } else { + if err := pe.putArrayLength(len(r.OrderedGroupProtocols)); err != nil { + return err + } + for _, protocol := range r.OrderedGroupProtocols { + if err := protocol.encode(pe); err != nil { + return err + } + } + } + + return nil +} + +func (r *JoinGroupRequest) decode(pd packetDecoder, version int16) (err error) { + r.Version = version + + if r.GroupId, err = pd.getString(); err != nil { + return + } + + if r.SessionTimeout, err = pd.getInt32(); err != nil { + return + } + + if version >= 1 { + if r.RebalanceTimeout, err = pd.getInt32(); err != nil { + return err + } + } + + if r.MemberId, err = pd.getString(); err != nil { + return + } + + if r.ProtocolType, err = pd.getString(); err != nil { + return + } + + n, err := pd.getArrayLength() + if err != nil { + return err + } + if n == 0 { + return nil + } + + r.GroupProtocols = make(map[string][]byte) + for i := 0; i < n; i++ { + protocol := &GroupProtocol{} + if err := protocol.decode(pd); err != nil { + return err + } + r.GroupProtocols[protocol.Name] = protocol.Metadata + r.OrderedGroupProtocols = append(r.OrderedGroupProtocols, protocol) + } + + return nil +} + +func (r *JoinGroupRequest) key() int16 { + return 11 +} + +func (r *JoinGroupRequest) version() int16 { + return r.Version +} + +func (r *JoinGroupRequest) headerVersion() int16 { + return 1 +} + +func (r *JoinGroupRequest) requiredVersion() KafkaVersion { + switch r.Version { + case 2: + return V0_11_0_0 + case 1: + return V0_10_1_0 + default: + return V0_9_0_0 + } +} + +func (r *JoinGroupRequest) AddGroupProtocol(name string, metadata []byte) { + r.OrderedGroupProtocols = append(r.OrderedGroupProtocols, &GroupProtocol{ + Name: name, + Metadata: metadata, + }) +} + +func (r *JoinGroupRequest) AddGroupProtocolMetadata(name string, metadata *ConsumerGroupMemberMetadata) error { + bin, err := encode(metadata, nil) + if err != nil { + return err + } + + r.AddGroupProtocol(name, bin) + return nil +} diff --git a/vendor/github.com/Shopify/sarama/join_group_response.go b/vendor/github.com/Shopify/sarama/join_group_response.go new file mode 100644 index 000000000000..54b0a45c28ea --- /dev/null +++ b/vendor/github.com/Shopify/sarama/join_group_response.go @@ -0,0 +1,139 @@ +package sarama + +type JoinGroupResponse struct { + Version int16 + ThrottleTime int32 + Err KError + GenerationId int32 + GroupProtocol string + LeaderId string + MemberId string + Members map[string][]byte +} + +func (r *JoinGroupResponse) GetMembers() (map[string]ConsumerGroupMemberMetadata, error) { + members := make(map[string]ConsumerGroupMemberMetadata, len(r.Members)) + for id, bin := range r.Members { + meta := new(ConsumerGroupMemberMetadata) + if err := decode(bin, meta); err != nil { + return nil, err + } + members[id] = *meta + } + return members, nil +} + +func (r *JoinGroupResponse) encode(pe packetEncoder) error { + if r.Version >= 2 { + pe.putInt32(r.ThrottleTime) + } + pe.putInt16(int16(r.Err)) + pe.putInt32(r.GenerationId) + + if err := pe.putString(r.GroupProtocol); err != nil { + return err + } + if err := pe.putString(r.LeaderId); err != nil { + return err + } + if err := pe.putString(r.MemberId); err != nil { + return err + } + + if err := pe.putArrayLength(len(r.Members)); err != nil { + return err + } + + for memberId, memberMetadata := range r.Members { + if err := pe.putString(memberId); err != nil { + return err + } + + if err := pe.putBytes(memberMetadata); err != nil { + return err + } + } + + return nil +} + +func (r *JoinGroupResponse) decode(pd packetDecoder, version int16) (err error) { + r.Version = version + + if version >= 2 { + if r.ThrottleTime, err = pd.getInt32(); err != nil { + return + } + } + + kerr, err := pd.getInt16() + if err != nil { + return err + } + + r.Err = KError(kerr) + + if r.GenerationId, err = pd.getInt32(); err != nil { + return + } + + if r.GroupProtocol, err = pd.getString(); err != nil { + return + } + + if r.LeaderId, err = pd.getString(); err != nil { + return + } + + if r.MemberId, err = pd.getString(); err != nil { + return + } + + n, err := pd.getArrayLength() + if err != nil { + return err + } + if n == 0 { + return nil + } + + r.Members = make(map[string][]byte) + for i := 0; i < n; i++ { + memberId, err := pd.getString() + if err != nil { + return err + } + + memberMetadata, err := pd.getBytes() + if err != nil { + return err + } + + r.Members[memberId] = memberMetadata + } + + return nil +} + +func (r *JoinGroupResponse) key() int16 { + return 11 +} + +func (r *JoinGroupResponse) version() int16 { + return r.Version +} + +func (r *JoinGroupResponse) headerVersion() int16 { + return 0 +} + +func (r *JoinGroupResponse) requiredVersion() KafkaVersion { + switch r.Version { + case 2: + return V0_11_0_0 + case 1: + return V0_10_1_0 + default: + return V0_9_0_0 + } +} diff --git a/vendor/github.com/Shopify/sarama/kerberos_client.go b/vendor/github.com/Shopify/sarama/kerberos_client.go new file mode 100644 index 000000000000..01a53193bb95 --- /dev/null +++ b/vendor/github.com/Shopify/sarama/kerberos_client.go @@ -0,0 +1,46 @@ +package sarama + +import ( + krb5client "github.com/jcmturner/gokrb5/v8/client" + krb5config "github.com/jcmturner/gokrb5/v8/config" + "github.com/jcmturner/gokrb5/v8/keytab" + "github.com/jcmturner/gokrb5/v8/types" +) + +type KerberosGoKrb5Client struct { + krb5client.Client +} + +func (c *KerberosGoKrb5Client) Domain() string { + return c.Credentials.Domain() +} + +func (c *KerberosGoKrb5Client) CName() types.PrincipalName { + return c.Credentials.CName() +} + +// NewKerberosClient creates kerberos client used to obtain TGT and TGS tokens. +// It uses pure go Kerberos 5 solution (RFC-4121 and RFC-4120). +// uses gokrb5 library underlying which is a pure go kerberos client with some GSS-API capabilities. +func NewKerberosClient(config *GSSAPIConfig) (KerberosClient, error) { + cfg, err := krb5config.Load(config.KerberosConfigPath) + if err != nil { + return nil, err + } + return createClient(config, cfg) +} + +func createClient(config *GSSAPIConfig, cfg *krb5config.Config) (KerberosClient, error) { + var client *krb5client.Client + if config.AuthType == KRB5_KEYTAB_AUTH { + kt, err := keytab.Load(config.KeyTabPath) + if err != nil { + return nil, err + } + client = krb5client.NewWithKeytab(config.Username, config.Realm, kt, cfg, krb5client.DisablePAFXFAST(config.DisablePAFXFAST)) + } else { + client = krb5client.NewWithPassword(config.Username, + config.Realm, config.Password, cfg, krb5client.DisablePAFXFAST(config.DisablePAFXFAST)) + } + return &KerberosGoKrb5Client{*client}, nil +} diff --git a/vendor/github.com/Shopify/sarama/leave_group_request.go b/vendor/github.com/Shopify/sarama/leave_group_request.go new file mode 100644 index 000000000000..d7789b68dbe2 --- /dev/null +++ b/vendor/github.com/Shopify/sarama/leave_group_request.go @@ -0,0 +1,44 @@ +package sarama + +type LeaveGroupRequest struct { + GroupId string + MemberId string +} + +func (r *LeaveGroupRequest) encode(pe packetEncoder) error { + if err := pe.putString(r.GroupId); err != nil { + return err + } + if err := pe.putString(r.MemberId); err != nil { + return err + } + + return nil +} + +func (r *LeaveGroupRequest) decode(pd packetDecoder, version int16) (err error) { + if r.GroupId, err = pd.getString(); err != nil { + return + } + if r.MemberId, err = pd.getString(); err != nil { + return + } + + return nil +} + +func (r *LeaveGroupRequest) key() int16 { + return 13 +} + +func (r *LeaveGroupRequest) version() int16 { + return 0 +} + +func (r *LeaveGroupRequest) headerVersion() int16 { + return 1 +} + +func (r *LeaveGroupRequest) requiredVersion() KafkaVersion { + return V0_9_0_0 +} diff --git a/vendor/github.com/Shopify/sarama/leave_group_response.go b/vendor/github.com/Shopify/sarama/leave_group_response.go new file mode 100644 index 000000000000..25f8d5eb36b4 --- /dev/null +++ b/vendor/github.com/Shopify/sarama/leave_group_response.go @@ -0,0 +1,36 @@ +package sarama + +type LeaveGroupResponse struct { + Err KError +} + +func (r *LeaveGroupResponse) encode(pe packetEncoder) error { + pe.putInt16(int16(r.Err)) + return nil +} + +func (r *LeaveGroupResponse) decode(pd packetDecoder, version int16) (err error) { + kerr, err := pd.getInt16() + if err != nil { + return err + } + r.Err = KError(kerr) + + return nil +} + +func (r *LeaveGroupResponse) key() int16 { + return 13 +} + +func (r *LeaveGroupResponse) version() int16 { + return 0 +} + +func (r *LeaveGroupResponse) headerVersion() int16 { + return 0 +} + +func (r *LeaveGroupResponse) requiredVersion() KafkaVersion { + return V0_9_0_0 +} diff --git a/vendor/github.com/Shopify/sarama/length_field.go b/vendor/github.com/Shopify/sarama/length_field.go new file mode 100644 index 000000000000..7d864f6bf97a --- /dev/null +++ b/vendor/github.com/Shopify/sarama/length_field.go @@ -0,0 +1,99 @@ +package sarama + +import ( + "encoding/binary" + "sync" +) + +// LengthField implements the PushEncoder and PushDecoder interfaces for calculating 4-byte lengths. +type lengthField struct { + startOffset int + length int32 +} + +var lengthFieldPool = sync.Pool{} + +func acquireLengthField() *lengthField { + val := lengthFieldPool.Get() + if val != nil { + return val.(*lengthField) + } + return &lengthField{} +} + +func releaseLengthField(m *lengthField) { + lengthFieldPool.Put(m) +} + +func (l *lengthField) decode(pd packetDecoder) error { + var err error + l.length, err = pd.getInt32() + if err != nil { + return err + } + if l.length > int32(pd.remaining()) { + return ErrInsufficientData + } + return nil +} + +func (l *lengthField) saveOffset(in int) { + l.startOffset = in +} + +func (l *lengthField) reserveLength() int { + return 4 +} + +func (l *lengthField) run(curOffset int, buf []byte) error { + binary.BigEndian.PutUint32(buf[l.startOffset:], uint32(curOffset-l.startOffset-4)) + return nil +} + +func (l *lengthField) check(curOffset int, buf []byte) error { + if int32(curOffset-l.startOffset-4) != l.length { + return PacketDecodingError{"length field invalid"} + } + + return nil +} + +type varintLengthField struct { + startOffset int + length int64 +} + +func (l *varintLengthField) decode(pd packetDecoder) error { + var err error + l.length, err = pd.getVarint() + return err +} + +func (l *varintLengthField) saveOffset(in int) { + l.startOffset = in +} + +func (l *varintLengthField) adjustLength(currOffset int) int { + oldFieldSize := l.reserveLength() + l.length = int64(currOffset - l.startOffset - oldFieldSize) + + return l.reserveLength() - oldFieldSize +} + +func (l *varintLengthField) reserveLength() int { + var tmp [binary.MaxVarintLen64]byte + return binary.PutVarint(tmp[:], l.length) +} + +func (l *varintLengthField) run(curOffset int, buf []byte) error { + binary.PutVarint(buf[l.startOffset:], l.length) + return nil +} + +func (l *varintLengthField) check(curOffset int, buf []byte) error { + if int64(curOffset-l.startOffset-l.reserveLength()) != l.length { + return PacketDecodingError{"length field invalid"} + } + + return nil +} diff --git a/vendor/github.com/Shopify/sarama/list_groups_request.go b/vendor/github.com/Shopify/sarama/list_groups_request.go new file mode 100644 index 000000000000..4553b2d2ea06 --- /dev/null +++ b/vendor/github.com/Shopify/sarama/list_groups_request.go @@ -0,0 +1,27 @@ +package sarama + +type ListGroupsRequest struct{} + +func (r *ListGroupsRequest) encode(pe packetEncoder) error { + return nil +} + +func (r *ListGroupsRequest) decode(pd packetDecoder, version int16) (err error) { + return nil +} + +func (r *ListGroupsRequest) key() int16 { + return 16 +} + +func (r *ListGroupsRequest) version() int16 { + return 0 +} + +func (r *ListGroupsRequest) headerVersion() int16 { + return 1 +} + +func (r *ListGroupsRequest) requiredVersion() KafkaVersion { + return V0_9_0_0 +} diff --git a/vendor/github.com/Shopify/sarama/list_groups_response.go b/vendor/github.com/Shopify/sarama/list_groups_response.go new file mode 100644 index 000000000000..777bae7e63ea --- /dev/null +++ b/vendor/github.com/Shopify/sarama/list_groups_response.go @@ -0,0 +1,73 @@ +package sarama + +type ListGroupsResponse struct { + Err KError + Groups map[string]string +} + +func (r *ListGroupsResponse) encode(pe packetEncoder) error { + pe.putInt16(int16(r.Err)) + + if err := pe.putArrayLength(len(r.Groups)); err != nil { + return err + } + for groupId, protocolType := range r.Groups { + if err := pe.putString(groupId); err != nil { + return err + } + if err := pe.putString(protocolType); err != nil { + return err + } + } + + return nil +} + +func (r *ListGroupsResponse) decode(pd packetDecoder, version int16) error { + kerr, err := pd.getInt16() + if err != nil { + return err + } + + r.Err = KError(kerr) + + n, err := pd.getArrayLength() + if err != nil { + return err + } + if n == 0 { + return nil + } + + r.Groups = make(map[string]string) + for i := 0; i < n; i++ { + groupId, err := pd.getString() + if err != nil { + return err + } + protocolType, err := pd.getString() + if err != nil { + return err + } + + r.Groups[groupId] = protocolType + } + + return nil +} + +func (r *ListGroupsResponse) key() int16 { + return 16 +} + +func (r *ListGroupsResponse) version() int16 { + return 0 +} + +func (r *ListGroupsResponse) headerVersion() int16 { + return 0 +} + +func (r *ListGroupsResponse) requiredVersion() KafkaVersion { + return V0_9_0_0 +} diff --git a/vendor/github.com/Shopify/sarama/list_partition_reassignments_request.go b/vendor/github.com/Shopify/sarama/list_partition_reassignments_request.go new file mode 100644 index 000000000000..c1ffa9ba02b7 --- /dev/null +++ b/vendor/github.com/Shopify/sarama/list_partition_reassignments_request.go @@ -0,0 +1,98 @@ +package sarama + +type ListPartitionReassignmentsRequest struct { + TimeoutMs int32 + blocks map[string][]int32 + Version int16 +} + +func (r *ListPartitionReassignmentsRequest) encode(pe packetEncoder) error { + pe.putInt32(r.TimeoutMs) + + pe.putCompactArrayLength(len(r.blocks)) + + for topic, partitions := range r.blocks { + if err := pe.putCompactString(topic); err != nil { + return err + } + + if err := pe.putCompactInt32Array(partitions); err != nil { + return err + } + + pe.putEmptyTaggedFieldArray() + } + + pe.putEmptyTaggedFieldArray() + + return nil +} + +func (r *ListPartitionReassignmentsRequest) decode(pd packetDecoder, version int16) (err error) { + r.Version = version + + if r.TimeoutMs, err = pd.getInt32(); err != nil { + return err + } + + topicCount, err := pd.getCompactArrayLength() + if err != nil { + return err + } + if topicCount > 0 { + r.blocks = make(map[string][]int32) + for i := 0; i < topicCount; i++ { + topic, err := pd.getCompactString() + if err != nil { + return err + } + partitionCount, err := pd.getCompactArrayLength() + if err != nil { + return err + } + r.blocks[topic] = make([]int32, partitionCount) + for j := 0; j < partitionCount; j++ { + partition, err := pd.getInt32() + if err != nil { + return err + } + r.blocks[topic][j] = partition + } + if _, err := pd.getEmptyTaggedFieldArray(); err != nil { + return err + } + } + } + + if _, err := pd.getEmptyTaggedFieldArray(); err != nil { + return err + } + + return +} + +func (r *ListPartitionReassignmentsRequest) key() int16 { + return 46 +} + +func (r *ListPartitionReassignmentsRequest) version() int16 { + return r.Version +} + +func (r *ListPartitionReassignmentsRequest) headerVersion() int16 { + return 2 +} + +func (r *ListPartitionReassignmentsRequest) requiredVersion() KafkaVersion { + return V2_4_0_0 +} + +func (r *ListPartitionReassignmentsRequest) AddBlock(topic string, partitionIDs []int32) { + if r.blocks == nil { + r.blocks = make(map[string][]int32) + } + + if r.blocks[topic] == nil { + r.blocks[topic] = partitionIDs + } +} diff --git a/vendor/github.com/Shopify/sarama/list_partition_reassignments_response.go b/vendor/github.com/Shopify/sarama/list_partition_reassignments_response.go new file mode 100644 index 000000000000..4baa6a08e83e --- /dev/null +++ b/vendor/github.com/Shopify/sarama/list_partition_reassignments_response.go @@ -0,0 +1,169 @@ +package sarama + +type PartitionReplicaReassignmentsStatus struct { + Replicas []int32 + AddingReplicas []int32 + RemovingReplicas []int32 +} + +func (b *PartitionReplicaReassignmentsStatus) encode(pe packetEncoder) error { + if err := pe.putCompactInt32Array(b.Replicas); err != nil { + return err + } + if err := pe.putCompactInt32Array(b.AddingReplicas); err != nil { + return err + } + if err := pe.putCompactInt32Array(b.RemovingReplicas); err != nil { + return err + } + + pe.putEmptyTaggedFieldArray() + + return nil +} + +func (b *PartitionReplicaReassignmentsStatus) decode(pd packetDecoder) (err error) { + if b.Replicas, err = pd.getCompactInt32Array(); err != nil { + return err + } + + if b.AddingReplicas, err = pd.getCompactInt32Array(); err != nil { + return err + } + + if b.RemovingReplicas, err = pd.getCompactInt32Array(); err != nil { + return err + } + + if _, err := pd.getEmptyTaggedFieldArray(); err != nil { + return err + } + + return err +} + +type ListPartitionReassignmentsResponse struct { + Version int16 + ThrottleTimeMs int32 + ErrorCode KError + ErrorMessage *string + TopicStatus map[string]map[int32]*PartitionReplicaReassignmentsStatus +} + +func (r *ListPartitionReassignmentsResponse) AddBlock(topic string, partition int32, replicas, addingReplicas, removingReplicas []int32) { + if r.TopicStatus == nil { + r.TopicStatus = make(map[string]map[int32]*PartitionReplicaReassignmentsStatus) + } + partitions := r.TopicStatus[topic] + if partitions == nil { + partitions = make(map[int32]*PartitionReplicaReassignmentsStatus) + r.TopicStatus[topic] = partitions + } + + partitions[partition] = &PartitionReplicaReassignmentsStatus{Replicas: replicas, AddingReplicas: addingReplicas, RemovingReplicas: removingReplicas} +} + +func (r *ListPartitionReassignmentsResponse) encode(pe packetEncoder) error { + pe.putInt32(r.ThrottleTimeMs) + pe.putInt16(int16(r.ErrorCode)) + if err := pe.putNullableCompactString(r.ErrorMessage); err != nil { + return err + } + + pe.putCompactArrayLength(len(r.TopicStatus)) + for topic, partitions := range r.TopicStatus { + if err := pe.putCompactString(topic); err != nil { + return err + } + pe.putCompactArrayLength(len(partitions)) + for partition, block := range partitions { + pe.putInt32(partition) + + if err := block.encode(pe); err != nil { + return err + } + } + pe.putEmptyTaggedFieldArray() + } + + pe.putEmptyTaggedFieldArray() + + return nil +} + +func (r *ListPartitionReassignmentsResponse) decode(pd packetDecoder, version int16) (err error) { + r.Version = version + + if r.ThrottleTimeMs, err = pd.getInt32(); err != nil { + return err + } + + kerr, err := pd.getInt16() + if err != nil { + return err + } + + r.ErrorCode = KError(kerr) + + if r.ErrorMessage, err = pd.getCompactNullableString(); err != nil { + return err + } + + numTopics, err := pd.getCompactArrayLength() + if err != nil { + return err + } + + r.TopicStatus = make(map[string]map[int32]*PartitionReplicaReassignmentsStatus, numTopics) + for i := 0; i < numTopics; i++ { + topic, err := pd.getCompactString() + if err != nil { + return err + } + + ongoingPartitionReassignments, err := pd.getCompactArrayLength() + if err != nil { + return err + } + + r.TopicStatus[topic] = make(map[int32]*PartitionReplicaReassignmentsStatus, ongoingPartitionReassignments) + + for j := 0; j < ongoingPartitionReassignments; j++ { + partition, err := pd.getInt32() + if err != nil { + return err + } + + block := &PartitionReplicaReassignmentsStatus{} + if err := block.decode(pd); err != nil { + return err + } + r.TopicStatus[topic][partition] = block + } + + if _, err := pd.getEmptyTaggedFieldArray(); err != nil { + return err + } + } + if _, err := pd.getEmptyTaggedFieldArray(); err != nil { + return err + } + + return nil +} + +func (r *ListPartitionReassignmentsResponse) key() int16 { + return 46 +} + +func (r *ListPartitionReassignmentsResponse) version() int16 { + return r.Version +} + +func (r *ListPartitionReassignmentsResponse) headerVersion() int16 { + return 1 +} + +func (r *ListPartitionReassignmentsResponse) requiredVersion() KafkaVersion { + return V2_4_0_0 +} diff --git a/vendor/github.com/Shopify/sarama/message.go b/vendor/github.com/Shopify/sarama/message.go new file mode 100644 index 000000000000..fd0d1d90b7fa --- /dev/null +++ b/vendor/github.com/Shopify/sarama/message.go @@ -0,0 +1,168 @@ +package sarama + +import ( + "fmt" + "time" +) + +const ( + // CompressionNone no compression + CompressionNone CompressionCodec = iota + // CompressionGZIP compression using GZIP + CompressionGZIP + // CompressionSnappy compression using snappy + CompressionSnappy + // CompressionLZ4 compression using LZ4 + CompressionLZ4 + // CompressionZSTD compression using ZSTD + CompressionZSTD + + // The lowest 3 bits contain the compression codec used for the message + compressionCodecMask int8 = 0x07 + + // Bit 3 set for "LogAppend" timestamps + timestampTypeMask = 0x08 + + // CompressionLevelDefault is the constant to use in CompressionLevel + // to have the default compression level for any codec. The value is picked + // that we don't use any existing compression levels. + CompressionLevelDefault = -1000 +) + +// CompressionCodec represents the various compression codecs recognized by Kafka in messages. +type CompressionCodec int8 + +func (cc CompressionCodec) String() string { + return []string{ + "none", + "gzip", + "snappy", + "lz4", + "zstd", + }[int(cc)] +} + +// Message is a kafka message type +type Message struct { + Codec CompressionCodec // codec used to compress the message contents + CompressionLevel int // compression level + LogAppendTime bool // the used timestamp is LogAppendTime + Key []byte // the message key, may be nil + Value []byte // the message contents + Set *MessageSet // the message set a message might wrap + Version int8 // v1 requires Kafka 0.10 + Timestamp time.Time // the timestamp of the message (version 1+ only) + + compressedCache []byte + compressedSize int // used for computing the compression ratio metrics +} + +func (m *Message) encode(pe packetEncoder) error { + pe.push(newCRC32Field(crcIEEE)) + + pe.putInt8(m.Version) + + attributes := int8(m.Codec) & compressionCodecMask + if m.LogAppendTime { + attributes |= timestampTypeMask + } + pe.putInt8(attributes) + + if m.Version >= 1 { + if err := (Timestamp{&m.Timestamp}).encode(pe); err != nil { + return err + } + } + + err := pe.putBytes(m.Key) + if err != nil { + return err + } + + var payload []byte + + if m.compressedCache != nil { + payload = m.compressedCache + m.compressedCache = nil + } else if m.Value != nil { + payload, err = compress(m.Codec, m.CompressionLevel, m.Value) + if err != nil { + return err + } + m.compressedCache = payload + // Keep in mind the compressed payload size for metric gathering + m.compressedSize = len(payload) + } + + if err = pe.putBytes(payload); err != nil { + return err + } + + return pe.pop() +} + +func (m *Message) decode(pd packetDecoder) (err error) { + crc32Decoder := acquireCrc32Field(crcIEEE) + defer releaseCrc32Field(crc32Decoder) + + err = pd.push(crc32Decoder) + if err != nil { + return err + } + + m.Version, err = pd.getInt8() + if err != nil { + return err + } + + if m.Version > 1 { + return PacketDecodingError{fmt.Sprintf("unknown magic byte (%v)", m.Version)} + } + + attribute, err := pd.getInt8() + if err != nil { + return err + } + m.Codec = CompressionCodec(attribute & compressionCodecMask) + m.LogAppendTime = attribute×tampTypeMask == timestampTypeMask + + if m.Version == 1 { + if err := (Timestamp{&m.Timestamp}).decode(pd); err != nil { + return err + } + } + + m.Key, err = pd.getBytes() + if err != nil { + return err + } + + m.Value, err = pd.getBytes() + if err != nil { + return err + } + + // Required for deep equal assertion during tests but might be useful + // for future metrics about the compression ratio in fetch requests + m.compressedSize = len(m.Value) + + if m.Value != nil && m.Codec != CompressionNone { + m.Value, err = decompress(m.Codec, m.Value) + if err != nil { + return err + } + + if err := m.decodeSet(); err != nil { + return err + } + } + + return pd.pop() +} + +// decodes a message set from a previously encoded bulk-message +func (m *Message) decodeSet() (err error) { + pd := realDecoder{raw: m.Value} + m.Set = &MessageSet{} + return m.Set.decode(&pd) +} diff --git a/vendor/github.com/Shopify/sarama/message_set.go b/vendor/github.com/Shopify/sarama/message_set.go new file mode 100644 index 000000000000..6523ec2f74d5 --- /dev/null +++ b/vendor/github.com/Shopify/sarama/message_set.go @@ -0,0 +1,111 @@ +package sarama + +type MessageBlock struct { + Offset int64 + Msg *Message +} + +// Messages convenience helper which returns either all the +// messages that are wrapped in this block +func (msb *MessageBlock) Messages() []*MessageBlock { + if msb.Msg.Set != nil { + return msb.Msg.Set.Messages + } + return []*MessageBlock{msb} +} + +func (msb *MessageBlock) encode(pe packetEncoder) error { + pe.putInt64(msb.Offset) + pe.push(&lengthField{}) + err := msb.Msg.encode(pe) + if err != nil { + return err + } + return pe.pop() +} + +func (msb *MessageBlock) decode(pd packetDecoder) (err error) { + if msb.Offset, err = pd.getInt64(); err != nil { + return err + } + + lengthDecoder := acquireLengthField() + defer releaseLengthField(lengthDecoder) + + if err = pd.push(lengthDecoder); err != nil { + return err + } + + msb.Msg = new(Message) + if err = msb.Msg.decode(pd); err != nil { + return err + } + + if err = pd.pop(); err != nil { + return err + } + + return nil +} + +type MessageSet struct { + PartialTrailingMessage bool // whether the set on the wire contained an incomplete trailing MessageBlock + OverflowMessage bool // whether the set on the wire contained an overflow message + Messages []*MessageBlock +} + +func (ms *MessageSet) encode(pe packetEncoder) error { + for i := range ms.Messages { + err := ms.Messages[i].encode(pe) + if err != nil { + return err + } + } + return nil +} + +func (ms *MessageSet) decode(pd packetDecoder) (err error) { + ms.Messages = nil + + for pd.remaining() > 0 { + magic, err := magicValue(pd) + if err != nil { + if err == ErrInsufficientData { + ms.PartialTrailingMessage = true + return nil + } + return err + } + + if magic > 1 { + return nil + } + + msb := new(MessageBlock) + err = msb.decode(pd) + switch err { + case nil: + ms.Messages = append(ms.Messages, msb) + case ErrInsufficientData: + // As an optimization the server is allowed to return a partial message at the + // end of the message set. Clients should handle this case. So we just ignore such things. + if msb.Offset == -1 { + // This is an overflow message caused by chunked down conversion + ms.OverflowMessage = true + } else { + ms.PartialTrailingMessage = true + } + return nil + default: + return err + } + } + + return nil +} + +func (ms *MessageSet) addMessage(msg *Message) { + block := new(MessageBlock) + block.Msg = msg + ms.Messages = append(ms.Messages, block) +} diff --git a/vendor/github.com/Shopify/sarama/metadata_request.go b/vendor/github.com/Shopify/sarama/metadata_request.go new file mode 100644 index 000000000000..e835f5a9c8a5 --- /dev/null +++ b/vendor/github.com/Shopify/sarama/metadata_request.go @@ -0,0 +1,85 @@ +package sarama + +type MetadataRequest struct { + Version int16 + Topics []string + AllowAutoTopicCreation bool +} + +func (r *MetadataRequest) encode(pe packetEncoder) error { + if r.Version < 0 || r.Version > 5 { + return PacketEncodingError{"invalid or unsupported MetadataRequest version field"} + } + if r.Version == 0 || len(r.Topics) > 0 { + err := pe.putArrayLength(len(r.Topics)) + if err != nil { + return err + } + + for i := range r.Topics { + err = pe.putString(r.Topics[i]) + if err != nil { + return err + } + } + } else { + pe.putInt32(-1) + } + if r.Version > 3 { + pe.putBool(r.AllowAutoTopicCreation) + } + return nil +} + +func (r *MetadataRequest) decode(pd packetDecoder, version int16) error { + r.Version = version + size, err := pd.getInt32() + if err != nil { + return err + } + if size > 0 { + r.Topics = make([]string, size) + for i := range r.Topics { + topic, err := pd.getString() + if err != nil { + return err + } + r.Topics[i] = topic + } + } + if r.Version > 3 { + autoCreation, err := pd.getBool() + if err != nil { + return err + } + r.AllowAutoTopicCreation = autoCreation + } + return nil +} + +func (r *MetadataRequest) key() int16 { + return 3 +} + +func (r *MetadataRequest) version() int16 { + return r.Version +} + +func (r *MetadataRequest) headerVersion() int16 { + return 1 +} + +func (r *MetadataRequest) requiredVersion() KafkaVersion { + switch r.Version { + case 1: + return V0_10_0_0 + case 2: + return V0_10_1_0 + case 3, 4: + return V0_11_0_0 + case 5: + return V1_0_0_0 + default: + return MinVersion + } +} diff --git a/vendor/github.com/Shopify/sarama/metadata_response.go b/vendor/github.com/Shopify/sarama/metadata_response.go new file mode 100644 index 000000000000..0bb8702cc37d --- /dev/null +++ b/vendor/github.com/Shopify/sarama/metadata_response.go @@ -0,0 +1,325 @@ +package sarama + +type PartitionMetadata struct { + Err KError + ID int32 + Leader int32 + Replicas []int32 + Isr []int32 + OfflineReplicas []int32 +} + +func (pm *PartitionMetadata) decode(pd packetDecoder, version int16) (err error) { + tmp, err := pd.getInt16() + if err != nil { + return err + } + pm.Err = KError(tmp) + + pm.ID, err = pd.getInt32() + if err != nil { + return err + } + + pm.Leader, err = pd.getInt32() + if err != nil { + return err + } + + pm.Replicas, err = pd.getInt32Array() + if err != nil { + return err + } + + pm.Isr, err = pd.getInt32Array() + if err != nil { + return err + } + + if version >= 5 { + pm.OfflineReplicas, err = pd.getInt32Array() + if err != nil { + return err + } + } + + return nil +} + +func (pm *PartitionMetadata) encode(pe packetEncoder, version int16) (err error) { + pe.putInt16(int16(pm.Err)) + pe.putInt32(pm.ID) + pe.putInt32(pm.Leader) + + err = pe.putInt32Array(pm.Replicas) + if err != nil { + return err + } + + err = pe.putInt32Array(pm.Isr) + if err != nil { + return err + } + + if version >= 5 { + err = pe.putInt32Array(pm.OfflineReplicas) + if err != nil { + return err + } + } + + return nil +} + +type TopicMetadata struct { + Err KError + Name string + IsInternal bool // Only valid for Version >= 1 + Partitions []*PartitionMetadata +} + +func (tm *TopicMetadata) decode(pd packetDecoder, version int16) (err error) { + tmp, err := pd.getInt16() + if err != nil { + return err + } + tm.Err = KError(tmp) + + tm.Name, err = pd.getString() + if err != nil { + return err + } + + if version >= 1 { + tm.IsInternal, err = pd.getBool() + if err != nil { + return err + } + } + + n, err := pd.getArrayLength() + if err != nil { + return err + } + tm.Partitions = make([]*PartitionMetadata, n) + for i := 0; i < n; i++ { + tm.Partitions[i] = new(PartitionMetadata) + err = tm.Partitions[i].decode(pd, version) + if err != nil { + return err + } + } + + return nil +} + +func (tm *TopicMetadata) encode(pe packetEncoder, version int16) (err error) { + pe.putInt16(int16(tm.Err)) + + err = pe.putString(tm.Name) + if err != nil { + return err + } + + if version >= 1 { + pe.putBool(tm.IsInternal) + } + + err = pe.putArrayLength(len(tm.Partitions)) + if err != nil { + return err + } + + for _, pm := range tm.Partitions { + err = pm.encode(pe, version) + if err != nil { + return err + } + } + + return nil +} + +type MetadataResponse struct { + Version int16 + ThrottleTimeMs int32 + Brokers []*Broker + ClusterID *string + ControllerID int32 + Topics []*TopicMetadata +} + +func (r *MetadataResponse) decode(pd packetDecoder, version int16) (err error) { + r.Version = version + + if version >= 3 { + r.ThrottleTimeMs, err = pd.getInt32() + if err != nil { + return err + } + } + + n, err := pd.getArrayLength() + if err != nil { + return err + } + + r.Brokers = make([]*Broker, n) + for i := 0; i < n; i++ { + r.Brokers[i] = new(Broker) + err = r.Brokers[i].decode(pd, version) + if err != nil { + return err + } + } + + if version >= 2 { + r.ClusterID, err = pd.getNullableString() + if err != nil { + return err + } + } + + if version >= 1 { + r.ControllerID, err = pd.getInt32() + if err != nil { + return err + } + } else { + r.ControllerID = -1 + } + + n, err = pd.getArrayLength() + if err != nil { + return err + } + + r.Topics = make([]*TopicMetadata, n) + for i := 0; i < n; i++ { + r.Topics[i] = new(TopicMetadata) + err = r.Topics[i].decode(pd, version) + if err != nil { + return err + } + } + + return nil +} + +func (r *MetadataResponse) encode(pe packetEncoder) error { + if r.Version >= 3 { + pe.putInt32(r.ThrottleTimeMs) + } + + err := pe.putArrayLength(len(r.Brokers)) + if err != nil { + return err + } + for _, broker := range r.Brokers { + err = broker.encode(pe, r.Version) + if err != nil { + return err + } + } + + if r.Version >= 2 { + err := pe.putNullableString(r.ClusterID) + if err != nil { + return err + } + } + + if r.Version >= 1 { + pe.putInt32(r.ControllerID) + } + + err = pe.putArrayLength(len(r.Topics)) + if err != nil { + return err + } + for _, tm := range r.Topics { + err = tm.encode(pe, r.Version) + if err != nil { + return err + } + } + + return nil +} + +func (r *MetadataResponse) key() int16 { + return 3 +} + +func (r *MetadataResponse) version() int16 { + return r.Version +} + +func (r *MetadataResponse) headerVersion() int16 { + return 0 +} + +func (r *MetadataResponse) requiredVersion() KafkaVersion { + switch r.Version { + case 1: + return V0_10_0_0 + case 2: + return V0_10_1_0 + case 3, 4: + return V0_11_0_0 + case 5: + return V1_0_0_0 + default: + return MinVersion + } +} + +// testing API + +func (r *MetadataResponse) AddBroker(addr string, id int32) { + r.Brokers = append(r.Brokers, &Broker{id: id, addr: addr}) +} + +func (r *MetadataResponse) AddTopic(topic string, err KError) *TopicMetadata { + var tmatch *TopicMetadata + + for _, tm := range r.Topics { + if tm.Name == topic { + tmatch = tm + goto foundTopic + } + } + + tmatch = new(TopicMetadata) + tmatch.Name = topic + r.Topics = append(r.Topics, tmatch) + +foundTopic: + + tmatch.Err = err + return tmatch +} + +func (r *MetadataResponse) AddTopicPartition(topic string, partition, brokerID int32, replicas, isr []int32, offline []int32, err KError) { + tmatch := r.AddTopic(topic, ErrNoError) + var pmatch *PartitionMetadata + + for _, pm := range tmatch.Partitions { + if pm.ID == partition { + pmatch = pm + goto foundPartition + } + } + + pmatch = new(PartitionMetadata) + pmatch.ID = partition + tmatch.Partitions = append(tmatch.Partitions, pmatch) + +foundPartition: + + pmatch.Leader = brokerID + pmatch.Replicas = replicas + pmatch.Isr = isr + pmatch.OfflineReplicas = offline + pmatch.Err = err +} diff --git a/vendor/github.com/Shopify/sarama/metrics.go b/vendor/github.com/Shopify/sarama/metrics.go new file mode 100644 index 000000000000..90e5a87f4972 --- /dev/null +++ b/vendor/github.com/Shopify/sarama/metrics.go @@ -0,0 +1,43 @@ +package sarama + +import ( + "fmt" + "strings" + + "github.com/rcrowley/go-metrics" +) + +// Use exponentially decaying reservoir for sampling histograms with the same defaults as the Java library: +// 1028 elements, which offers a 99.9% confidence level with a 5% margin of error assuming a normal distribution, +// and an alpha factor of 0.015, which heavily biases the reservoir to the past 5 minutes of measurements. +// See https://github.com/dropwizard/metrics/blob/v3.1.0/metrics-core/src/main/java/com/codahale/metrics/ExponentiallyDecayingReservoir.java#L38 +const ( + metricsReservoirSize = 1028 + metricsAlphaFactor = 0.015 +) + +func getOrRegisterHistogram(name string, r metrics.Registry) metrics.Histogram { + return r.GetOrRegister(name, func() metrics.Histogram { + return metrics.NewHistogram(metrics.NewExpDecaySample(metricsReservoirSize, metricsAlphaFactor)) + }).(metrics.Histogram) +} + +func getMetricNameForBroker(name string, broker *Broker) string { + // Use broker id like the Java client as it does not contain '.' or ':' characters that + // can be interpreted as special character by monitoring tool (e.g. Graphite) + return fmt.Sprintf(name+"-for-broker-%d", broker.ID()) +} + +func getMetricNameForTopic(name string, topic string) string { + // Convert dot to _ since reporters like Graphite typically use dot to represent hierarchy + // cf. KAFKA-1902 and KAFKA-2337 + return fmt.Sprintf(name+"-for-topic-%s", strings.Replace(topic, ".", "_", -1)) +} + +func getOrRegisterTopicMeter(name string, topic string, r metrics.Registry) metrics.Meter { + return metrics.GetOrRegisterMeter(getMetricNameForTopic(name, topic), r) +} + +func getOrRegisterTopicHistogram(name string, topic string, r metrics.Registry) metrics.Histogram { + return getOrRegisterHistogram(getMetricNameForTopic(name, topic), r) +} diff --git a/vendor/github.com/Shopify/sarama/mockbroker.go b/vendor/github.com/Shopify/sarama/mockbroker.go new file mode 100644 index 000000000000..9166f6efbca9 --- /dev/null +++ b/vendor/github.com/Shopify/sarama/mockbroker.go @@ -0,0 +1,426 @@ +package sarama + +import ( + "bytes" + "encoding/binary" + "fmt" + "io" + "net" + "reflect" + "strconv" + "sync" + "time" + + "github.com/davecgh/go-spew/spew" +) + +const ( + expectationTimeout = 500 * time.Millisecond +) + +type GSSApiHandlerFunc func([]byte) []byte + +type requestHandlerFunc func(req *request) (res encoderWithHeader) + +// RequestNotifierFunc is invoked when a mock broker processes a request successfully +// and will provides the number of bytes read and written. +type RequestNotifierFunc func(bytesRead, bytesWritten int) + +// MockBroker is a mock Kafka broker that is used in unit tests. It is exposed +// to facilitate testing of higher level or specialized consumers and producers +// built on top of Sarama. Note that it does not 'mimic' the Kafka API protocol, +// but rather provides a facility to do that. It takes care of the TCP +// transport, request unmarshalling, response marshaling, and makes it the test +// writer responsibility to program correct according to the Kafka API protocol +// MockBroker behavior. +// +// MockBroker is implemented as a TCP server listening on a kernel-selected +// localhost port that can accept many connections. It reads Kafka requests +// from that connection and returns responses programmed by the SetHandlerByMap +// function. If a MockBroker receives a request that it has no programmed +// response for, then it returns nothing and the request times out. +// +// A set of MockRequest builders to define mappings used by MockBroker is +// provided by Sarama. But users can develop MockRequests of their own and use +// them along with or instead of the standard ones. +// +// When running tests with MockBroker it is strongly recommended to specify +// a timeout to `go test` so that if the broker hangs waiting for a response, +// the test panics. +// +// It is not necessary to prefix message length or correlation ID to your +// response bytes, the server does that automatically as a convenience. +type MockBroker struct { + brokerID int32 + port int32 + closing chan none + stopper chan none + expectations chan encoderWithHeader + listener net.Listener + t TestReporter + latency time.Duration + handler requestHandlerFunc + notifier RequestNotifierFunc + history []RequestResponse + lock sync.Mutex + gssApiHandler GSSApiHandlerFunc +} + +// RequestResponse represents a Request/Response pair processed by MockBroker. +type RequestResponse struct { + Request protocolBody + Response encoder +} + +// SetLatency makes broker pause for the specified period every time before +// replying. +func (b *MockBroker) SetLatency(latency time.Duration) { + b.latency = latency +} + +// SetHandlerByMap defines mapping of Request types to MockResponses. When a +// request is received by the broker, it looks up the request type in the map +// and uses the found MockResponse instance to generate an appropriate reply. +// If the request type is not found in the map then nothing is sent. +func (b *MockBroker) SetHandlerByMap(handlerMap map[string]MockResponse) { + fnMap := make(map[string]MockResponse) + for k, v := range handlerMap { + fnMap[k] = v + } + b.setHandler(func(req *request) (res encoderWithHeader) { + reqTypeName := reflect.TypeOf(req.body).Elem().Name() + mockResponse := fnMap[reqTypeName] + if mockResponse == nil { + return nil + } + return mockResponse.For(req.body) + }) +} + +// SetNotifier set a function that will get invoked whenever a request has been +// processed successfully and will provide the number of bytes read and written +func (b *MockBroker) SetNotifier(notifier RequestNotifierFunc) { + b.lock.Lock() + b.notifier = notifier + b.lock.Unlock() +} + +// BrokerID returns broker ID assigned to the broker. +func (b *MockBroker) BrokerID() int32 { + return b.brokerID +} + +// History returns a slice of RequestResponse pairs in the order they were +// processed by the broker. Note that in case of multiple connections to the +// broker the order expected by a test can be different from the order recorded +// in the history, unless some synchronization is implemented in the test. +func (b *MockBroker) History() []RequestResponse { + b.lock.Lock() + history := make([]RequestResponse, len(b.history)) + copy(history, b.history) + b.lock.Unlock() + return history +} + +// Port returns the TCP port number the broker is listening for requests on. +func (b *MockBroker) Port() int32 { + return b.port +} + +// Addr returns the broker connection string in the form "
:". +func (b *MockBroker) Addr() string { + return b.listener.Addr().String() +} + +// Close terminates the broker blocking until it stops internal goroutines and +// releases all resources. +func (b *MockBroker) Close() { + close(b.expectations) + if len(b.expectations) > 0 { + buf := bytes.NewBufferString(fmt.Sprintf("mockbroker/%d: not all expectations were satisfied! Still waiting on:\n", b.BrokerID())) + for e := range b.expectations { + _, _ = buf.WriteString(spew.Sdump(e)) + } + b.t.Error(buf.String()) + } + close(b.closing) + <-b.stopper +} + +// setHandler sets the specified function as the request handler. Whenever +// a mock broker reads a request from the wire it passes the request to the +// function and sends back whatever the handler function returns. +func (b *MockBroker) setHandler(handler requestHandlerFunc) { + b.lock.Lock() + b.handler = handler + b.lock.Unlock() +} + +func (b *MockBroker) serverLoop() { + defer close(b.stopper) + var err error + var conn net.Conn + + go func() { + <-b.closing + err := b.listener.Close() + if err != nil { + b.t.Error(err) + } + }() + + wg := &sync.WaitGroup{} + i := 0 + for conn, err = b.listener.Accept(); err == nil; conn, err = b.listener.Accept() { + wg.Add(1) + go b.handleRequests(conn, i, wg) + i++ + } + wg.Wait() + Logger.Printf("*** mockbroker/%d: listener closed, err=%v", b.BrokerID(), err) +} + +func (b *MockBroker) SetGSSAPIHandler(handler GSSApiHandlerFunc) { + b.gssApiHandler = handler +} + +func (b *MockBroker) readToBytes(r io.Reader) ([]byte, error) { + var ( + bytesRead int + lengthBytes = make([]byte, 4) + ) + + if _, err := io.ReadFull(r, lengthBytes); err != nil { + return nil, err + } + + bytesRead += len(lengthBytes) + length := int32(binary.BigEndian.Uint32(lengthBytes)) + + if length <= 4 || length > MaxRequestSize { + return nil, PacketDecodingError{fmt.Sprintf("message of length %d too large or too small", length)} + } + + encodedReq := make([]byte, length) + if _, err := io.ReadFull(r, encodedReq); err != nil { + return nil, err + } + + bytesRead += len(encodedReq) + + fullBytes := append(lengthBytes, encodedReq...) + + return fullBytes, nil +} + +func (b *MockBroker) isGSSAPI(buffer []byte) bool { + return buffer[4] == 0x60 || bytes.Equal(buffer[4:6], []byte{0x05, 0x04}) +} + +func (b *MockBroker) handleRequests(conn io.ReadWriteCloser, idx int, wg *sync.WaitGroup) { + defer wg.Done() + defer func() { + _ = conn.Close() + }() + s := spew.NewDefaultConfig() + s.MaxDepth = 1 + Logger.Printf("*** mockbroker/%d/%d: connection opened", b.BrokerID(), idx) + var err error + + abort := make(chan none) + defer close(abort) + go func() { + select { + case <-b.closing: + _ = conn.Close() + case <-abort: + } + }() + + var bytesWritten int + var bytesRead int + for { + buffer, err := b.readToBytes(conn) + if err != nil { + Logger.Printf("*** mockbroker/%d/%d: invalid request: err=%+v, %+v", b.brokerID, idx, err, spew.Sdump(buffer)) + b.serverError(err) + break + } + + bytesWritten = 0 + if !b.isGSSAPI(buffer) { + req, br, err := decodeRequest(bytes.NewReader(buffer)) + bytesRead = br + if err != nil { + Logger.Printf("*** mockbroker/%d/%d: invalid request: err=%+v, %+v", b.brokerID, idx, err, spew.Sdump(req)) + b.serverError(err) + break + } + + if b.latency > 0 { + time.Sleep(b.latency) + } + + b.lock.Lock() + res := b.handler(req) + b.history = append(b.history, RequestResponse{req.body, res}) + b.lock.Unlock() + + if res == nil { + Logger.Printf("*** mockbroker/%d/%d: ignored %v", b.brokerID, idx, spew.Sdump(req)) + continue + } + Logger.Printf( + "*** mockbroker/%d/%d: replied to %T with %T\n-> %s\n-> %s", + b.brokerID, idx, req.body, res, + s.Sprintf("%#v", req.body), + s.Sprintf("%#v", res), + ) + + encodedRes, err := encode(res, nil) + if err != nil { + b.serverError(err) + break + } + if len(encodedRes) == 0 { + b.lock.Lock() + if b.notifier != nil { + b.notifier(bytesRead, 0) + } + b.lock.Unlock() + continue + } + + resHeader := b.encodeHeader(res.headerVersion(), req.correlationID, uint32(len(encodedRes))) + if _, err = conn.Write(resHeader); err != nil { + b.serverError(err) + break + } + if _, err = conn.Write(encodedRes); err != nil { + b.serverError(err) + break + } + bytesWritten = len(resHeader) + len(encodedRes) + } else { + // GSSAPI is not part of kafka protocol, but is supported for authentication proposes. + // Don't support history for this kind of request as is only used for test GSSAPI authentication mechanism + b.lock.Lock() + res := b.gssApiHandler(buffer) + b.lock.Unlock() + if res == nil { + Logger.Printf("*** mockbroker/%d/%d: ignored %v", b.brokerID, idx, spew.Sdump(buffer)) + continue + } + if _, err = conn.Write(res); err != nil { + b.serverError(err) + break + } + bytesWritten = len(res) + } + + b.lock.Lock() + if b.notifier != nil { + b.notifier(bytesRead, bytesWritten) + } + b.lock.Unlock() + } + Logger.Printf("*** mockbroker/%d/%d: connection closed, err=%v", b.BrokerID(), idx, err) +} + +func (b *MockBroker) encodeHeader(headerVersion int16, correlationId int32, payloadLength uint32) []byte { + headerLength := uint32(8) + + if headerVersion >= 1 { + headerLength = 9 + } + + resHeader := make([]byte, headerLength) + binary.BigEndian.PutUint32(resHeader, payloadLength+headerLength-4) + binary.BigEndian.PutUint32(resHeader[4:], uint32(correlationId)) + + if headerVersion >= 1 { + binary.PutUvarint(resHeader[8:], 0) + } + + return resHeader +} + +func (b *MockBroker) defaultRequestHandler(req *request) (res encoderWithHeader) { + select { + case res, ok := <-b.expectations: + if !ok { + return nil + } + return res + case <-time.After(expectationTimeout): + return nil + } +} + +func (b *MockBroker) serverError(err error) { + isConnectionClosedError := false + if _, ok := err.(*net.OpError); ok { + isConnectionClosedError = true + } else if err == io.EOF { + isConnectionClosedError = true + } else if err.Error() == "use of closed network connection" { + isConnectionClosedError = true + } + + if isConnectionClosedError { + return + } + + b.t.Errorf(err.Error()) +} + +// NewMockBroker launches a fake Kafka broker. It takes a TestReporter as provided by the +// test framework and a channel of responses to use. If an error occurs it is +// simply logged to the TestReporter and the broker exits. +func NewMockBroker(t TestReporter, brokerID int32) *MockBroker { + return NewMockBrokerAddr(t, brokerID, "localhost:0") +} + +// NewMockBrokerAddr behaves like newMockBroker but listens on the address you give +// it rather than just some ephemeral port. +func NewMockBrokerAddr(t TestReporter, brokerID int32, addr string) *MockBroker { + listener, err := net.Listen("tcp", addr) + if err != nil { + t.Fatal(err) + } + return NewMockBrokerListener(t, brokerID, listener) +} + +// NewMockBrokerListener behaves like newMockBrokerAddr but accepts connections on the listener specified. +func NewMockBrokerListener(t TestReporter, brokerID int32, listener net.Listener) *MockBroker { + var err error + + broker := &MockBroker{ + closing: make(chan none), + stopper: make(chan none), + t: t, + brokerID: brokerID, + expectations: make(chan encoderWithHeader, 512), + listener: listener, + } + broker.handler = broker.defaultRequestHandler + + Logger.Printf("*** mockbroker/%d listening on %s\n", brokerID, broker.listener.Addr().String()) + _, portStr, err := net.SplitHostPort(broker.listener.Addr().String()) + if err != nil { + t.Fatal(err) + } + tmp, err := strconv.ParseInt(portStr, 10, 32) + if err != nil { + t.Fatal(err) + } + broker.port = int32(tmp) + + go broker.serverLoop() + + return broker +} + +func (b *MockBroker) Returns(e encoderWithHeader) { + b.expectations <- e +} diff --git a/vendor/github.com/Shopify/sarama/mockkerberos.go b/vendor/github.com/Shopify/sarama/mockkerberos.go new file mode 100644 index 000000000000..a43607e1c107 --- /dev/null +++ b/vendor/github.com/Shopify/sarama/mockkerberos.go @@ -0,0 +1,125 @@ +package sarama + +import ( + "encoding/binary" + "encoding/hex" + + "github.com/jcmturner/gokrb5/v8/credentials" + "github.com/jcmturner/gokrb5/v8/gssapi" + "github.com/jcmturner/gokrb5/v8/iana/keyusage" + "github.com/jcmturner/gokrb5/v8/messages" + "github.com/jcmturner/gokrb5/v8/types" +) + +type KafkaGSSAPIHandler struct { + client *MockKerberosClient + badResponse bool + badKeyChecksum bool +} + +func (h *KafkaGSSAPIHandler) MockKafkaGSSAPI(buffer []byte) []byte { + // Default payload used for verify + err := h.client.Login() // Mock client construct keys when login + if err != nil { + return nil + } + if h.badResponse { // Returns trash + return []byte{0x00, 0x00, 0x00, 0x01, 0xAD} + } + + pack := gssapi.WrapToken{ + Flags: KRB5_USER_AUTH, + EC: 12, + RRC: 0, + SndSeqNum: 3398292281, + Payload: []byte{0x11, 0x00}, // 1100 + } + // Compute checksum + if h.badKeyChecksum { + pack.CheckSum = []byte{0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF} + } else { + err = pack.SetCheckSum(h.client.ASRep.DecryptedEncPart.Key, keyusage.GSSAPI_ACCEPTOR_SEAL) + if err != nil { + return nil + } + } + + packBytes, err := pack.Marshal() + if err != nil { + return nil + } + lenBytes := len(packBytes) + response := make([]byte, lenBytes+4) + copy(response[4:], packBytes) + binary.BigEndian.PutUint32(response, uint32(lenBytes)) + return response +} + +type MockKerberosClient struct { + asRepBytes string + ASRep messages.ASRep + credentials *credentials.Credentials + mockError error + errorStage string +} + +func (c *MockKerberosClient) Login() error { + if c.errorStage == "login" && c.mockError != nil { + return c.mockError + } + c.asRepBytes = "6b8202e9308202e5a003020105a10302010ba22b30293027a103020113a220041e301c301aa003020112a1131b114" + + "558414d504c452e434f4d636c69656e74a30d1b0b4558414d504c452e434f4da4133011a003020101a10a30081b06636c69656e7" + + "4a5820156618201523082014ea003020105a10d1b0b4558414d504c452e434f4da220301ea003020102a11730151b066b7262746" + + "7741b0b4558414d504c452e434f4da382011430820110a003020112a103020101a28201020481ffdb9891175d106818e61008c51" + + "d0b3462bca92f3bf9d4cfa82de4c4d7aff9994ec87c573e3a3d54dcb2bb79618c76f2bf4a3d006f90d5bdbd049bc18f48be39203" + + "549ca02acaf63f292b12404f9b74c34b83687119d8f56552ccc0c50ebee2a53bb114c1b4619bb1d5d31f0f49b4d40a08a9b4c046" + + "2e1398d0b648be1c0e50c552ad16e1d8d8e74263dd0bf0ec591e4797dfd40a9a1be4ae830d03a306e053fd7586fef84ffc5e4a83" + + "7c3122bf3e6a40fe87e84019f6283634461b955712b44a5f7386c278bff94ec2c2dc0403247e29c2450e853471ceababf9b8911f" + + "997f2e3010b046d2c49eb438afb0f4c210821e80d4ffa4c9521eb895dcd68610b3feaa682012c30820128a003020112a282011f0" + + "482011bce73cbce3f1dd17661c412005f0f2257c756fe8e98ff97e6ec24b7bab66e5fd3a3827aeeae4757af0c6e892948122d8b2" + + "03c8df48df0ef5d142d0e416d688f11daa0fcd63d96bdd431d02b8e951c664eeff286a2be62383d274a04016d5f0e141da58cb86" + + "331de64063062f4f885e8e9ce5b181ca2fdc67897c5995e0ae1ae0c171a64493ff7bd91bc6d89cd4fce1e2b3ea0a10e34b0d5eda" + + "aa38ee727b50c5632ed1d2f2b457908e616178d0d80b72af209fb8ac9dbaa1768fa45931392b36b6d8c12400f8ded2efaa0654d0" + + "da1db966e8b5aab4706c800f95d559664646041fdb38b411c62fc0fbe0d25083a28562b0e1c8df16e62e9d5626b0addee489835f" + + "eedb0f26c05baa596b69b17f47920aa64b29dc77cfcc97ba47885" + apRepBytes, err := hex.DecodeString(c.asRepBytes) + if err != nil { + return err + } + err = c.ASRep.Unmarshal(apRepBytes) + if err != nil { + return err + } + c.credentials = credentials.New("client", "EXAMPLE.COM").WithPassword("qwerty") + _, err = c.ASRep.DecryptEncPart(c.credentials) + if err != nil { + return err + } + return nil +} + +func (c *MockKerberosClient) GetServiceTicket(spn string) (messages.Ticket, types.EncryptionKey, error) { + if c.errorStage == "service_ticket" && c.mockError != nil { + return messages.Ticket{}, types.EncryptionKey{}, c.mockError + } + return c.ASRep.Ticket, c.ASRep.DecryptedEncPart.Key, nil +} + +func (c *MockKerberosClient) Domain() string { + return "EXAMPLE.COM" +} + +func (c *MockKerberosClient) CName() types.PrincipalName { + p := types.PrincipalName{ + NameType: KRB5_USER_AUTH, + NameString: []string{ + "kafka", + "kafka", + }, + } + return p +} + +func (c *MockKerberosClient) Destroy() { + // Do nothing. +} diff --git a/vendor/github.com/Shopify/sarama/mockresponses.go b/vendor/github.com/Shopify/sarama/mockresponses.go new file mode 100644 index 000000000000..a5e94fb39f07 --- /dev/null +++ b/vendor/github.com/Shopify/sarama/mockresponses.go @@ -0,0 +1,1332 @@ +package sarama + +import ( + "fmt" + "strings" +) + +// TestReporter has methods matching go's testing.T to avoid importing +// `testing` in the main part of the library. +type TestReporter interface { + Error(...interface{}) + Errorf(string, ...interface{}) + Fatal(...interface{}) + Fatalf(string, ...interface{}) +} + +// MockResponse is a response builder interface it defines one method that +// allows generating a response based on a request body. MockResponses are used +// to program behavior of MockBroker in tests. +type MockResponse interface { + For(reqBody versionedDecoder) (res encoderWithHeader) +} + +// MockWrapper is a mock response builder that returns a particular concrete +// response regardless of the actual request passed to the `For` method. +type MockWrapper struct { + res encoderWithHeader +} + +func (mw *MockWrapper) For(reqBody versionedDecoder) (res encoderWithHeader) { + return mw.res +} + +func NewMockWrapper(res encoderWithHeader) *MockWrapper { + return &MockWrapper{res: res} +} + +// MockSequence is a mock response builder that is created from a sequence of +// concrete responses. Every time when a `MockBroker` calls its `For` method +// the next response from the sequence is returned. When the end of the +// sequence is reached the last element from the sequence is returned. +type MockSequence struct { + responses []MockResponse +} + +func NewMockSequence(responses ...interface{}) *MockSequence { + ms := &MockSequence{} + ms.responses = make([]MockResponse, len(responses)) + for i, res := range responses { + switch res := res.(type) { + case MockResponse: + ms.responses[i] = res + case encoderWithHeader: + ms.responses[i] = NewMockWrapper(res) + default: + panic(fmt.Sprintf("Unexpected response type: %T", res)) + } + } + return ms +} + +func (mc *MockSequence) For(reqBody versionedDecoder) (res encoderWithHeader) { + res = mc.responses[0].For(reqBody) + if len(mc.responses) > 1 { + mc.responses = mc.responses[1:] + } + return res +} + +type MockListGroupsResponse struct { + groups map[string]string + t TestReporter +} + +func NewMockListGroupsResponse(t TestReporter) *MockListGroupsResponse { + return &MockListGroupsResponse{ + groups: make(map[string]string), + t: t, + } +} + +func (m *MockListGroupsResponse) For(reqBody versionedDecoder) encoderWithHeader { + request := reqBody.(*ListGroupsRequest) + _ = request + response := &ListGroupsResponse{ + Groups: m.groups, + } + return response +} + +func (m *MockListGroupsResponse) AddGroup(groupID, protocolType string) *MockListGroupsResponse { + m.groups[groupID] = protocolType + return m +} + +type MockDescribeGroupsResponse struct { + groups map[string]*GroupDescription + t TestReporter +} + +func NewMockDescribeGroupsResponse(t TestReporter) *MockDescribeGroupsResponse { + return &MockDescribeGroupsResponse{ + t: t, + groups: make(map[string]*GroupDescription), + } +} + +func (m *MockDescribeGroupsResponse) AddGroupDescription(groupID string, description *GroupDescription) *MockDescribeGroupsResponse { + m.groups[groupID] = description + return m +} + +func (m *MockDescribeGroupsResponse) For(reqBody versionedDecoder) encoderWithHeader { + request := reqBody.(*DescribeGroupsRequest) + + response := &DescribeGroupsResponse{} + for _, requestedGroup := range request.Groups { + if group, ok := m.groups[requestedGroup]; ok { + response.Groups = append(response.Groups, group) + } else { + // Mimic real kafka - if a group doesn't exist, return + // an entry with state "Dead" + response.Groups = append(response.Groups, &GroupDescription{ + GroupId: requestedGroup, + State: "Dead", + }) + } + } + + return response +} + +// MockMetadataResponse is a `MetadataResponse` builder. +type MockMetadataResponse struct { + controllerID int32 + leaders map[string]map[int32]int32 + brokers map[string]int32 + t TestReporter +} + +func NewMockMetadataResponse(t TestReporter) *MockMetadataResponse { + return &MockMetadataResponse{ + leaders: make(map[string]map[int32]int32), + brokers: make(map[string]int32), + t: t, + } +} + +func (mmr *MockMetadataResponse) SetLeader(topic string, partition, brokerID int32) *MockMetadataResponse { + partitions := mmr.leaders[topic] + if partitions == nil { + partitions = make(map[int32]int32) + mmr.leaders[topic] = partitions + } + partitions[partition] = brokerID + return mmr +} + +func (mmr *MockMetadataResponse) SetBroker(addr string, brokerID int32) *MockMetadataResponse { + mmr.brokers[addr] = brokerID + return mmr +} + +func (mmr *MockMetadataResponse) SetController(brokerID int32) *MockMetadataResponse { + mmr.controllerID = brokerID + return mmr +} + +func (mmr *MockMetadataResponse) For(reqBody versionedDecoder) encoderWithHeader { + metadataRequest := reqBody.(*MetadataRequest) + metadataResponse := &MetadataResponse{ + Version: metadataRequest.version(), + ControllerID: mmr.controllerID, + } + for addr, brokerID := range mmr.brokers { + metadataResponse.AddBroker(addr, brokerID) + } + + // Generate set of replicas + var replicas []int32 + var offlineReplicas []int32 + for _, brokerID := range mmr.brokers { + replicas = append(replicas, brokerID) + } + + if len(metadataRequest.Topics) == 0 { + for topic, partitions := range mmr.leaders { + for partition, brokerID := range partitions { + metadataResponse.AddTopicPartition(topic, partition, brokerID, replicas, replicas, offlineReplicas, ErrNoError) + } + } + return metadataResponse + } + for _, topic := range metadataRequest.Topics { + for partition, brokerID := range mmr.leaders[topic] { + metadataResponse.AddTopicPartition(topic, partition, brokerID, replicas, replicas, offlineReplicas, ErrNoError) + } + } + return metadataResponse +} + +// MockOffsetResponse is an `OffsetResponse` builder. +type MockOffsetResponse struct { + offsets map[string]map[int32]map[int64]int64 + t TestReporter + version int16 +} + +func NewMockOffsetResponse(t TestReporter) *MockOffsetResponse { + return &MockOffsetResponse{ + offsets: make(map[string]map[int32]map[int64]int64), + t: t, + } +} + +func (mor *MockOffsetResponse) SetVersion(version int16) *MockOffsetResponse { + mor.version = version + return mor +} + +func (mor *MockOffsetResponse) SetOffset(topic string, partition int32, time, offset int64) *MockOffsetResponse { + partitions := mor.offsets[topic] + if partitions == nil { + partitions = make(map[int32]map[int64]int64) + mor.offsets[topic] = partitions + } + times := partitions[partition] + if times == nil { + times = make(map[int64]int64) + partitions[partition] = times + } + times[time] = offset + return mor +} + +func (mor *MockOffsetResponse) For(reqBody versionedDecoder) encoderWithHeader { + offsetRequest := reqBody.(*OffsetRequest) + offsetResponse := &OffsetResponse{Version: mor.version} + for topic, partitions := range offsetRequest.blocks { + for partition, block := range partitions { + offset := mor.getOffset(topic, partition, block.time) + offsetResponse.AddTopicPartition(topic, partition, offset) + } + } + return offsetResponse +} + +func (mor *MockOffsetResponse) getOffset(topic string, partition int32, time int64) int64 { + partitions := mor.offsets[topic] + if partitions == nil { + mor.t.Errorf("missing topic: %s", topic) + } + times := partitions[partition] + if times == nil { + mor.t.Errorf("missing partition: %d", partition) + } + offset, ok := times[time] + if !ok { + mor.t.Errorf("missing time: %d", time) + } + return offset +} + +// MockFetchResponse is a `FetchResponse` builder. +type MockFetchResponse struct { + messages map[string]map[int32]map[int64]Encoder + highWaterMarks map[string]map[int32]int64 + t TestReporter + batchSize int + version int16 +} + +func NewMockFetchResponse(t TestReporter, batchSize int) *MockFetchResponse { + return &MockFetchResponse{ + messages: make(map[string]map[int32]map[int64]Encoder), + highWaterMarks: make(map[string]map[int32]int64), + t: t, + batchSize: batchSize, + } +} + +func (mfr *MockFetchResponse) SetVersion(version int16) *MockFetchResponse { + mfr.version = version + return mfr +} + +func (mfr *MockFetchResponse) SetMessage(topic string, partition int32, offset int64, msg Encoder) *MockFetchResponse { + partitions := mfr.messages[topic] + if partitions == nil { + partitions = make(map[int32]map[int64]Encoder) + mfr.messages[topic] = partitions + } + messages := partitions[partition] + if messages == nil { + messages = make(map[int64]Encoder) + partitions[partition] = messages + } + messages[offset] = msg + return mfr +} + +func (mfr *MockFetchResponse) SetHighWaterMark(topic string, partition int32, offset int64) *MockFetchResponse { + partitions := mfr.highWaterMarks[topic] + if partitions == nil { + partitions = make(map[int32]int64) + mfr.highWaterMarks[topic] = partitions + } + partitions[partition] = offset + return mfr +} + +func (mfr *MockFetchResponse) For(reqBody versionedDecoder) encoderWithHeader { + fetchRequest := reqBody.(*FetchRequest) + res := &FetchResponse{ + Version: mfr.version, + } + for topic, partitions := range fetchRequest.blocks { + for partition, block := range partitions { + initialOffset := block.fetchOffset + offset := initialOffset + maxOffset := initialOffset + int64(mfr.getMessageCount(topic, partition)) + for i := 0; i < mfr.batchSize && offset < maxOffset; { + msg := mfr.getMessage(topic, partition, offset) + if msg != nil { + res.AddMessage(topic, partition, nil, msg, offset) + i++ + } + offset++ + } + fb := res.GetBlock(topic, partition) + if fb == nil { + res.AddError(topic, partition, ErrNoError) + fb = res.GetBlock(topic, partition) + } + fb.HighWaterMarkOffset = mfr.getHighWaterMark(topic, partition) + } + } + return res +} + +func (mfr *MockFetchResponse) getMessage(topic string, partition int32, offset int64) Encoder { + partitions := mfr.messages[topic] + if partitions == nil { + return nil + } + messages := partitions[partition] + if messages == nil { + return nil + } + return messages[offset] +} + +func (mfr *MockFetchResponse) getMessageCount(topic string, partition int32) int { + partitions := mfr.messages[topic] + if partitions == nil { + return 0 + } + messages := partitions[partition] + if messages == nil { + return 0 + } + return len(messages) +} + +func (mfr *MockFetchResponse) getHighWaterMark(topic string, partition int32) int64 { + partitions := mfr.highWaterMarks[topic] + if partitions == nil { + return 0 + } + return partitions[partition] +} + +// MockConsumerMetadataResponse is a `ConsumerMetadataResponse` builder. +type MockConsumerMetadataResponse struct { + coordinators map[string]interface{} + t TestReporter +} + +func NewMockConsumerMetadataResponse(t TestReporter) *MockConsumerMetadataResponse { + return &MockConsumerMetadataResponse{ + coordinators: make(map[string]interface{}), + t: t, + } +} + +func (mr *MockConsumerMetadataResponse) SetCoordinator(group string, broker *MockBroker) *MockConsumerMetadataResponse { + mr.coordinators[group] = broker + return mr +} + +func (mr *MockConsumerMetadataResponse) SetError(group string, kerror KError) *MockConsumerMetadataResponse { + mr.coordinators[group] = kerror + return mr +} + +func (mr *MockConsumerMetadataResponse) For(reqBody versionedDecoder) encoderWithHeader { + req := reqBody.(*ConsumerMetadataRequest) + group := req.ConsumerGroup + res := &ConsumerMetadataResponse{} + v := mr.coordinators[group] + switch v := v.(type) { + case *MockBroker: + res.Coordinator = &Broker{id: v.BrokerID(), addr: v.Addr()} + case KError: + res.Err = v + } + return res +} + +// MockFindCoordinatorResponse is a `FindCoordinatorResponse` builder. +type MockFindCoordinatorResponse struct { + groupCoordinators map[string]interface{} + transCoordinators map[string]interface{} + t TestReporter +} + +func NewMockFindCoordinatorResponse(t TestReporter) *MockFindCoordinatorResponse { + return &MockFindCoordinatorResponse{ + groupCoordinators: make(map[string]interface{}), + transCoordinators: make(map[string]interface{}), + t: t, + } +} + +func (mr *MockFindCoordinatorResponse) SetCoordinator(coordinatorType CoordinatorType, group string, broker *MockBroker) *MockFindCoordinatorResponse { + switch coordinatorType { + case CoordinatorGroup: + mr.groupCoordinators[group] = broker + case CoordinatorTransaction: + mr.transCoordinators[group] = broker + } + return mr +} + +func (mr *MockFindCoordinatorResponse) SetError(coordinatorType CoordinatorType, group string, kerror KError) *MockFindCoordinatorResponse { + switch coordinatorType { + case CoordinatorGroup: + mr.groupCoordinators[group] = kerror + case CoordinatorTransaction: + mr.transCoordinators[group] = kerror + } + return mr +} + +func (mr *MockFindCoordinatorResponse) For(reqBody versionedDecoder) encoderWithHeader { + req := reqBody.(*FindCoordinatorRequest) + res := &FindCoordinatorResponse{} + var v interface{} + switch req.CoordinatorType { + case CoordinatorGroup: + v = mr.groupCoordinators[req.CoordinatorKey] + case CoordinatorTransaction: + v = mr.transCoordinators[req.CoordinatorKey] + } + switch v := v.(type) { + case *MockBroker: + res.Coordinator = &Broker{id: v.BrokerID(), addr: v.Addr()} + case KError: + res.Err = v + } + return res +} + +// MockOffsetCommitResponse is a `OffsetCommitResponse` builder. +type MockOffsetCommitResponse struct { + errors map[string]map[string]map[int32]KError + t TestReporter +} + +func NewMockOffsetCommitResponse(t TestReporter) *MockOffsetCommitResponse { + return &MockOffsetCommitResponse{t: t} +} + +func (mr *MockOffsetCommitResponse) SetError(group, topic string, partition int32, kerror KError) *MockOffsetCommitResponse { + if mr.errors == nil { + mr.errors = make(map[string]map[string]map[int32]KError) + } + topics := mr.errors[group] + if topics == nil { + topics = make(map[string]map[int32]KError) + mr.errors[group] = topics + } + partitions := topics[topic] + if partitions == nil { + partitions = make(map[int32]KError) + topics[topic] = partitions + } + partitions[partition] = kerror + return mr +} + +func (mr *MockOffsetCommitResponse) For(reqBody versionedDecoder) encoderWithHeader { + req := reqBody.(*OffsetCommitRequest) + group := req.ConsumerGroup + res := &OffsetCommitResponse{} + for topic, partitions := range req.blocks { + for partition := range partitions { + res.AddError(topic, partition, mr.getError(group, topic, partition)) + } + } + return res +} + +func (mr *MockOffsetCommitResponse) getError(group, topic string, partition int32) KError { + topics := mr.errors[group] + if topics == nil { + return ErrNoError + } + partitions := topics[topic] + if partitions == nil { + return ErrNoError + } + kerror, ok := partitions[partition] + if !ok { + return ErrNoError + } + return kerror +} + +// MockProduceResponse is a `ProduceResponse` builder. +type MockProduceResponse struct { + version int16 + errors map[string]map[int32]KError + t TestReporter +} + +func NewMockProduceResponse(t TestReporter) *MockProduceResponse { + return &MockProduceResponse{t: t} +} + +func (mr *MockProduceResponse) SetVersion(version int16) *MockProduceResponse { + mr.version = version + return mr +} + +func (mr *MockProduceResponse) SetError(topic string, partition int32, kerror KError) *MockProduceResponse { + if mr.errors == nil { + mr.errors = make(map[string]map[int32]KError) + } + partitions := mr.errors[topic] + if partitions == nil { + partitions = make(map[int32]KError) + mr.errors[topic] = partitions + } + partitions[partition] = kerror + return mr +} + +func (mr *MockProduceResponse) For(reqBody versionedDecoder) encoderWithHeader { + req := reqBody.(*ProduceRequest) + res := &ProduceResponse{ + Version: mr.version, + } + for topic, partitions := range req.records { + for partition := range partitions { + res.AddTopicPartition(topic, partition, mr.getError(topic, partition)) + } + } + return res +} + +func (mr *MockProduceResponse) getError(topic string, partition int32) KError { + partitions := mr.errors[topic] + if partitions == nil { + return ErrNoError + } + kerror, ok := partitions[partition] + if !ok { + return ErrNoError + } + return kerror +} + +// MockOffsetFetchResponse is a `OffsetFetchResponse` builder. +type MockOffsetFetchResponse struct { + offsets map[string]map[string]map[int32]*OffsetFetchResponseBlock + error KError + t TestReporter +} + +func NewMockOffsetFetchResponse(t TestReporter) *MockOffsetFetchResponse { + return &MockOffsetFetchResponse{t: t} +} + +func (mr *MockOffsetFetchResponse) SetOffset(group, topic string, partition int32, offset int64, metadata string, kerror KError) *MockOffsetFetchResponse { + if mr.offsets == nil { + mr.offsets = make(map[string]map[string]map[int32]*OffsetFetchResponseBlock) + } + topics := mr.offsets[group] + if topics == nil { + topics = make(map[string]map[int32]*OffsetFetchResponseBlock) + mr.offsets[group] = topics + } + partitions := topics[topic] + if partitions == nil { + partitions = make(map[int32]*OffsetFetchResponseBlock) + topics[topic] = partitions + } + partitions[partition] = &OffsetFetchResponseBlock{offset, 0, metadata, kerror} + return mr +} + +func (mr *MockOffsetFetchResponse) SetError(kerror KError) *MockOffsetFetchResponse { + mr.error = kerror + return mr +} + +func (mr *MockOffsetFetchResponse) For(reqBody versionedDecoder) encoderWithHeader { + req := reqBody.(*OffsetFetchRequest) + group := req.ConsumerGroup + res := &OffsetFetchResponse{Version: req.Version} + + for topic, partitions := range mr.offsets[group] { + for partition, block := range partitions { + res.AddBlock(topic, partition, block) + } + } + + if res.Version >= 2 { + res.Err = mr.error + } + return res +} + +type MockCreateTopicsResponse struct { + t TestReporter +} + +func NewMockCreateTopicsResponse(t TestReporter) *MockCreateTopicsResponse { + return &MockCreateTopicsResponse{t: t} +} + +func (mr *MockCreateTopicsResponse) For(reqBody versionedDecoder) encoderWithHeader { + req := reqBody.(*CreateTopicsRequest) + res := &CreateTopicsResponse{ + Version: req.Version, + } + res.TopicErrors = make(map[string]*TopicError) + + for topic := range req.TopicDetails { + if res.Version >= 1 && strings.HasPrefix(topic, "_") { + msg := "insufficient permissions to create topic with reserved prefix" + res.TopicErrors[topic] = &TopicError{ + Err: ErrTopicAuthorizationFailed, + ErrMsg: &msg, + } + continue + } + res.TopicErrors[topic] = &TopicError{Err: ErrNoError} + } + return res +} + +type MockDeleteTopicsResponse struct { + t TestReporter +} + +func NewMockDeleteTopicsResponse(t TestReporter) *MockDeleteTopicsResponse { + return &MockDeleteTopicsResponse{t: t} +} + +func (mr *MockDeleteTopicsResponse) For(reqBody versionedDecoder) encoderWithHeader { + req := reqBody.(*DeleteTopicsRequest) + res := &DeleteTopicsResponse{} + res.TopicErrorCodes = make(map[string]KError) + + for _, topic := range req.Topics { + res.TopicErrorCodes[topic] = ErrNoError + } + res.Version = req.Version + return res +} + +type MockCreatePartitionsResponse struct { + t TestReporter +} + +func NewMockCreatePartitionsResponse(t TestReporter) *MockCreatePartitionsResponse { + return &MockCreatePartitionsResponse{t: t} +} + +func (mr *MockCreatePartitionsResponse) For(reqBody versionedDecoder) encoderWithHeader { + req := reqBody.(*CreatePartitionsRequest) + res := &CreatePartitionsResponse{} + res.TopicPartitionErrors = make(map[string]*TopicPartitionError) + + for topic := range req.TopicPartitions { + if strings.HasPrefix(topic, "_") { + msg := "insufficient permissions to create partition on topic with reserved prefix" + res.TopicPartitionErrors[topic] = &TopicPartitionError{ + Err: ErrTopicAuthorizationFailed, + ErrMsg: &msg, + } + continue + } + res.TopicPartitionErrors[topic] = &TopicPartitionError{Err: ErrNoError} + } + return res +} + +type MockAlterPartitionReassignmentsResponse struct { + t TestReporter +} + +func NewMockAlterPartitionReassignmentsResponse(t TestReporter) *MockAlterPartitionReassignmentsResponse { + return &MockAlterPartitionReassignmentsResponse{t: t} +} + +func (mr *MockAlterPartitionReassignmentsResponse) For(reqBody versionedDecoder) encoderWithHeader { + req := reqBody.(*AlterPartitionReassignmentsRequest) + _ = req + res := &AlterPartitionReassignmentsResponse{} + return res +} + +type MockListPartitionReassignmentsResponse struct { + t TestReporter +} + +func NewMockListPartitionReassignmentsResponse(t TestReporter) *MockListPartitionReassignmentsResponse { + return &MockListPartitionReassignmentsResponse{t: t} +} + +func (mr *MockListPartitionReassignmentsResponse) For(reqBody versionedDecoder) encoderWithHeader { + req := reqBody.(*ListPartitionReassignmentsRequest) + _ = req + res := &ListPartitionReassignmentsResponse{} + + for topic, partitions := range req.blocks { + for _, partition := range partitions { + res.AddBlock(topic, partition, []int32{0}, []int32{1}, []int32{2}) + } + } + + return res +} + +type MockDeleteRecordsResponse struct { + t TestReporter +} + +func NewMockDeleteRecordsResponse(t TestReporter) *MockDeleteRecordsResponse { + return &MockDeleteRecordsResponse{t: t} +} + +func (mr *MockDeleteRecordsResponse) For(reqBody versionedDecoder) encoderWithHeader { + req := reqBody.(*DeleteRecordsRequest) + res := &DeleteRecordsResponse{} + res.Topics = make(map[string]*DeleteRecordsResponseTopic) + + for topic, deleteRecordRequestTopic := range req.Topics { + partitions := make(map[int32]*DeleteRecordsResponsePartition) + for partition := range deleteRecordRequestTopic.PartitionOffsets { + partitions[partition] = &DeleteRecordsResponsePartition{Err: ErrNoError} + } + res.Topics[topic] = &DeleteRecordsResponseTopic{Partitions: partitions} + } + return res +} + +type MockDescribeConfigsResponse struct { + t TestReporter +} + +func NewMockDescribeConfigsResponse(t TestReporter) *MockDescribeConfigsResponse { + return &MockDescribeConfigsResponse{t: t} +} + +func (mr *MockDescribeConfigsResponse) For(reqBody versionedDecoder) encoderWithHeader { + req := reqBody.(*DescribeConfigsRequest) + res := &DescribeConfigsResponse{ + Version: req.Version, + } + + includeSynonyms := req.Version > 0 + includeSource := req.Version > 0 + + for _, r := range req.Resources { + var configEntries []*ConfigEntry + switch r.Type { + case BrokerResource: + configEntries = append(configEntries, + &ConfigEntry{ + Name: "min.insync.replicas", + Value: "2", + ReadOnly: false, + Default: false, + }, + ) + res.Resources = append(res.Resources, &ResourceResponse{ + Name: r.Name, + Configs: configEntries, + }) + case BrokerLoggerResource: + configEntries = append(configEntries, + &ConfigEntry{ + Name: "kafka.controller.KafkaController", + Value: "DEBUG", + ReadOnly: false, + Default: false, + }, + ) + res.Resources = append(res.Resources, &ResourceResponse{ + Name: r.Name, + Configs: configEntries, + }) + case TopicResource: + maxMessageBytes := &ConfigEntry{ + Name: "max.message.bytes", + Value: "1000000", + ReadOnly: false, + Default: !includeSource, + Sensitive: false, + } + if includeSource { + maxMessageBytes.Source = SourceDefault + } + if includeSynonyms { + maxMessageBytes.Synonyms = []*ConfigSynonym{ + { + ConfigName: "max.message.bytes", + ConfigValue: "500000", + }, + } + } + retentionMs := &ConfigEntry{ + Name: "retention.ms", + Value: "5000", + ReadOnly: false, + Default: false, + Sensitive: false, + } + if includeSynonyms { + retentionMs.Synonyms = []*ConfigSynonym{ + { + ConfigName: "log.retention.ms", + ConfigValue: "2500", + }, + } + } + password := &ConfigEntry{ + Name: "password", + Value: "12345", + ReadOnly: false, + Default: false, + Sensitive: true, + } + configEntries = append( + configEntries, maxMessageBytes, retentionMs, password) + res.Resources = append(res.Resources, &ResourceResponse{ + Name: r.Name, + Configs: configEntries, + }) + } + } + return res +} + +type MockDescribeConfigsResponseWithErrorCode struct { + t TestReporter +} + +func NewMockDescribeConfigsResponseWithErrorCode(t TestReporter) *MockDescribeConfigsResponseWithErrorCode { + return &MockDescribeConfigsResponseWithErrorCode{t: t} +} + +func (mr *MockDescribeConfigsResponseWithErrorCode) For(reqBody versionedDecoder) encoderWithHeader { + req := reqBody.(*DescribeConfigsRequest) + res := &DescribeConfigsResponse{ + Version: req.Version, + } + + for _, r := range req.Resources { + res.Resources = append(res.Resources, &ResourceResponse{ + Name: r.Name, + Type: r.Type, + ErrorCode: 83, + ErrorMsg: "", + }) + } + return res +} + +type MockAlterConfigsResponse struct { + t TestReporter +} + +func NewMockAlterConfigsResponse(t TestReporter) *MockAlterConfigsResponse { + return &MockAlterConfigsResponse{t: t} +} + +func (mr *MockAlterConfigsResponse) For(reqBody versionedDecoder) encoderWithHeader { + req := reqBody.(*AlterConfigsRequest) + res := &AlterConfigsResponse{} + + for _, r := range req.Resources { + res.Resources = append(res.Resources, &AlterConfigsResourceResponse{ + Name: r.Name, + Type: r.Type, + ErrorMsg: "", + }) + } + return res +} + +type MockAlterConfigsResponseWithErrorCode struct { + t TestReporter +} + +func NewMockAlterConfigsResponseWithErrorCode(t TestReporter) *MockAlterConfigsResponseWithErrorCode { + return &MockAlterConfigsResponseWithErrorCode{t: t} +} + +func (mr *MockAlterConfigsResponseWithErrorCode) For(reqBody versionedDecoder) encoderWithHeader { + req := reqBody.(*AlterConfigsRequest) + res := &AlterConfigsResponse{} + + for _, r := range req.Resources { + res.Resources = append(res.Resources, &AlterConfigsResourceResponse{ + Name: r.Name, + Type: r.Type, + ErrorCode: 83, + ErrorMsg: "", + }) + } + return res +} + +type MockCreateAclsResponse struct { + t TestReporter +} + +func NewMockCreateAclsResponse(t TestReporter) *MockCreateAclsResponse { + return &MockCreateAclsResponse{t: t} +} + +func (mr *MockCreateAclsResponse) For(reqBody versionedDecoder) encoderWithHeader { + req := reqBody.(*CreateAclsRequest) + res := &CreateAclsResponse{} + + for range req.AclCreations { + res.AclCreationResponses = append(res.AclCreationResponses, &AclCreationResponse{Err: ErrNoError}) + } + return res +} + +type MockListAclsResponse struct { + t TestReporter +} + +func NewMockListAclsResponse(t TestReporter) *MockListAclsResponse { + return &MockListAclsResponse{t: t} +} + +func (mr *MockListAclsResponse) For(reqBody versionedDecoder) encoderWithHeader { + req := reqBody.(*DescribeAclsRequest) + res := &DescribeAclsResponse{} + res.Err = ErrNoError + acl := &ResourceAcls{} + if req.ResourceName != nil { + acl.Resource.ResourceName = *req.ResourceName + } + acl.Resource.ResourcePatternType = req.ResourcePatternTypeFilter + acl.Resource.ResourceType = req.ResourceType + + host := "*" + if req.Host != nil { + host = *req.Host + } + + principal := "User:test" + if req.Principal != nil { + principal = *req.Principal + } + + permissionType := req.PermissionType + if permissionType == AclPermissionAny { + permissionType = AclPermissionAllow + } + + acl.Acls = append(acl.Acls, &Acl{Operation: req.Operation, PermissionType: permissionType, Host: host, Principal: principal}) + res.ResourceAcls = append(res.ResourceAcls, acl) + res.Version = int16(req.Version) + return res +} + +type MockSaslAuthenticateResponse struct { + t TestReporter + kerror KError + saslAuthBytes []byte +} + +func NewMockSaslAuthenticateResponse(t TestReporter) *MockSaslAuthenticateResponse { + return &MockSaslAuthenticateResponse{t: t} +} + +func (msar *MockSaslAuthenticateResponse) For(reqBody versionedDecoder) encoderWithHeader { + res := &SaslAuthenticateResponse{} + res.Err = msar.kerror + res.SaslAuthBytes = msar.saslAuthBytes + return res +} + +func (msar *MockSaslAuthenticateResponse) SetError(kerror KError) *MockSaslAuthenticateResponse { + msar.kerror = kerror + return msar +} + +func (msar *MockSaslAuthenticateResponse) SetAuthBytes(saslAuthBytes []byte) *MockSaslAuthenticateResponse { + msar.saslAuthBytes = saslAuthBytes + return msar +} + +type MockDeleteAclsResponse struct { + t TestReporter +} + +type MockSaslHandshakeResponse struct { + enabledMechanisms []string + kerror KError + t TestReporter +} + +func NewMockSaslHandshakeResponse(t TestReporter) *MockSaslHandshakeResponse { + return &MockSaslHandshakeResponse{t: t} +} + +func (mshr *MockSaslHandshakeResponse) For(reqBody versionedDecoder) encoderWithHeader { + res := &SaslHandshakeResponse{} + res.Err = mshr.kerror + res.EnabledMechanisms = mshr.enabledMechanisms + return res +} + +func (mshr *MockSaslHandshakeResponse) SetError(kerror KError) *MockSaslHandshakeResponse { + mshr.kerror = kerror + return mshr +} + +func (mshr *MockSaslHandshakeResponse) SetEnabledMechanisms(enabledMechanisms []string) *MockSaslHandshakeResponse { + mshr.enabledMechanisms = enabledMechanisms + return mshr +} + +func NewMockDeleteAclsResponse(t TestReporter) *MockDeleteAclsResponse { + return &MockDeleteAclsResponse{t: t} +} + +func (mr *MockDeleteAclsResponse) For(reqBody versionedDecoder) encoderWithHeader { + req := reqBody.(*DeleteAclsRequest) + res := &DeleteAclsResponse{} + + for range req.Filters { + response := &FilterResponse{Err: ErrNoError} + response.MatchingAcls = append(response.MatchingAcls, &MatchingAcl{Err: ErrNoError}) + res.FilterResponses = append(res.FilterResponses, response) + } + res.Version = int16(req.Version) + return res +} + +type MockDeleteGroupsResponse struct { + deletedGroups []string +} + +func NewMockDeleteGroupsRequest(t TestReporter) *MockDeleteGroupsResponse { + return &MockDeleteGroupsResponse{} +} + +func (m *MockDeleteGroupsResponse) SetDeletedGroups(groups []string) *MockDeleteGroupsResponse { + m.deletedGroups = groups + return m +} + +func (m *MockDeleteGroupsResponse) For(reqBody versionedDecoder) encoderWithHeader { + resp := &DeleteGroupsResponse{ + GroupErrorCodes: map[string]KError{}, + } + for _, group := range m.deletedGroups { + resp.GroupErrorCodes[group] = ErrNoError + } + return resp +} + +type MockDeleteOffsetResponse struct { + errorCode KError + topic string + partition int32 + errorPartition KError +} + +func NewMockDeleteOffsetRequest(t TestReporter) *MockDeleteOffsetResponse { + return &MockDeleteOffsetResponse{} +} + +func (m *MockDeleteOffsetResponse) SetDeletedOffset(errorCode KError, topic string, partition int32, errorPartition KError) *MockDeleteOffsetResponse { + m.errorCode = errorCode + m.topic = topic + m.partition = partition + m.errorPartition = errorPartition + return m +} + +func (m *MockDeleteOffsetResponse) For(reqBody versionedDecoder) encoderWithHeader { + resp := &DeleteOffsetsResponse{ + ErrorCode: m.errorCode, + Errors: map[string]map[int32]KError{ + m.topic: {m.partition: m.errorPartition}, + }, + } + return resp +} + +type MockJoinGroupResponse struct { + t TestReporter + + ThrottleTime int32 + Err KError + GenerationId int32 + GroupProtocol string + LeaderId string + MemberId string + Members map[string][]byte +} + +func NewMockJoinGroupResponse(t TestReporter) *MockJoinGroupResponse { + return &MockJoinGroupResponse{ + t: t, + Members: make(map[string][]byte), + } +} + +func (m *MockJoinGroupResponse) For(reqBody versionedDecoder) encoderWithHeader { + req := reqBody.(*JoinGroupRequest) + resp := &JoinGroupResponse{ + Version: req.Version, + ThrottleTime: m.ThrottleTime, + Err: m.Err, + GenerationId: m.GenerationId, + GroupProtocol: m.GroupProtocol, + LeaderId: m.LeaderId, + MemberId: m.MemberId, + Members: m.Members, + } + return resp +} + +func (m *MockJoinGroupResponse) SetThrottleTime(t int32) *MockJoinGroupResponse { + m.ThrottleTime = t + return m +} + +func (m *MockJoinGroupResponse) SetError(kerr KError) *MockJoinGroupResponse { + m.Err = kerr + return m +} + +func (m *MockJoinGroupResponse) SetGenerationId(id int32) *MockJoinGroupResponse { + m.GenerationId = id + return m +} + +func (m *MockJoinGroupResponse) SetGroupProtocol(proto string) *MockJoinGroupResponse { + m.GroupProtocol = proto + return m +} + +func (m *MockJoinGroupResponse) SetLeaderId(id string) *MockJoinGroupResponse { + m.LeaderId = id + return m +} + +func (m *MockJoinGroupResponse) SetMemberId(id string) *MockJoinGroupResponse { + m.MemberId = id + return m +} + +func (m *MockJoinGroupResponse) SetMember(id string, meta *ConsumerGroupMemberMetadata) *MockJoinGroupResponse { + bin, err := encode(meta, nil) + if err != nil { + panic(fmt.Sprintf("error encoding member metadata: %v", err)) + } + m.Members[id] = bin + return m +} + +type MockLeaveGroupResponse struct { + t TestReporter + + Err KError +} + +func NewMockLeaveGroupResponse(t TestReporter) *MockLeaveGroupResponse { + return &MockLeaveGroupResponse{t: t} +} + +func (m *MockLeaveGroupResponse) For(reqBody versionedDecoder) encoderWithHeader { + resp := &LeaveGroupResponse{ + Err: m.Err, + } + return resp +} + +func (m *MockLeaveGroupResponse) SetError(kerr KError) *MockLeaveGroupResponse { + m.Err = kerr + return m +} + +type MockSyncGroupResponse struct { + t TestReporter + + Err KError + MemberAssignment []byte +} + +func NewMockSyncGroupResponse(t TestReporter) *MockSyncGroupResponse { + return &MockSyncGroupResponse{t: t} +} + +func (m *MockSyncGroupResponse) For(reqBody versionedDecoder) encoderWithHeader { + resp := &SyncGroupResponse{ + Err: m.Err, + MemberAssignment: m.MemberAssignment, + } + return resp +} + +func (m *MockSyncGroupResponse) SetError(kerr KError) *MockSyncGroupResponse { + m.Err = kerr + return m +} + +func (m *MockSyncGroupResponse) SetMemberAssignment(assignment *ConsumerGroupMemberAssignment) *MockSyncGroupResponse { + bin, err := encode(assignment, nil) + if err != nil { + panic(fmt.Sprintf("error encoding member assignment: %v", err)) + } + m.MemberAssignment = bin + return m +} + +type MockHeartbeatResponse struct { + t TestReporter + + Err KError +} + +func NewMockHeartbeatResponse(t TestReporter) *MockHeartbeatResponse { + return &MockHeartbeatResponse{t: t} +} + +func (m *MockHeartbeatResponse) For(reqBody versionedDecoder) encoderWithHeader { + resp := &HeartbeatResponse{} + return resp +} + +func (m *MockHeartbeatResponse) SetError(kerr KError) *MockHeartbeatResponse { + m.Err = kerr + return m +} + +type MockDescribeLogDirsResponse struct { + t TestReporter + logDirs []DescribeLogDirsResponseDirMetadata +} + +func NewMockDescribeLogDirsResponse(t TestReporter) *MockDescribeLogDirsResponse { + return &MockDescribeLogDirsResponse{t: t} +} + +func (m *MockDescribeLogDirsResponse) SetLogDirs(logDirPath string, topicPartitions map[string]int) *MockDescribeLogDirsResponse { + var topics []DescribeLogDirsResponseTopic + for topic := range topicPartitions { + var partitions []DescribeLogDirsResponsePartition + for i := 0; i < topicPartitions[topic]; i++ { + partitions = append(partitions, DescribeLogDirsResponsePartition{ + PartitionID: int32(i), + IsTemporary: false, + OffsetLag: int64(0), + Size: int64(1234), + }) + } + topics = append(topics, DescribeLogDirsResponseTopic{ + Topic: topic, + Partitions: partitions, + }) + } + logDir := DescribeLogDirsResponseDirMetadata{ + ErrorCode: ErrNoError, + Path: logDirPath, + Topics: topics, + } + m.logDirs = []DescribeLogDirsResponseDirMetadata{logDir} + return m +} + +func (m *MockDescribeLogDirsResponse) For(reqBody versionedDecoder) encoderWithHeader { + resp := &DescribeLogDirsResponse{ + LogDirs: m.logDirs, + } + return resp +} + +type MockApiVersionsResponse struct { + t TestReporter +} + +func NewMockApiVersionsResponse(t TestReporter) *MockApiVersionsResponse { + return &MockApiVersionsResponse{t: t} +} + +func (mr *MockApiVersionsResponse) For(reqBody versionedDecoder) encoderWithHeader { + req := reqBody.(*ApiVersionsRequest) + res := &ApiVersionsResponse{ + Version: req.Version, + ApiKeys: []ApiVersionsResponseKey{ + { + Version: req.Version, + ApiKey: 0, + MinVersion: 5, + MaxVersion: 8, + }, + { + Version: req.Version, + ApiKey: 1, + MinVersion: 7, + MaxVersion: 11, + }, + }, + } + return res +} diff --git a/vendor/github.com/Shopify/sarama/offset_commit_request.go b/vendor/github.com/Shopify/sarama/offset_commit_request.go new file mode 100644 index 000000000000..9931cade512d --- /dev/null +++ b/vendor/github.com/Shopify/sarama/offset_commit_request.go @@ -0,0 +1,214 @@ +package sarama + +import "errors" + +// ReceiveTime is a special value for the timestamp field of Offset Commit Requests which +// tells the broker to set the timestamp to the time at which the request was received. +// The timestamp is only used if message version 1 is used, which requires kafka 0.8.2. +const ReceiveTime int64 = -1 + +// GroupGenerationUndefined is a special value for the group generation field of +// Offset Commit Requests that should be used when a consumer group does not rely +// on Kafka for partition management. +const GroupGenerationUndefined = -1 + +type offsetCommitRequestBlock struct { + offset int64 + timestamp int64 + metadata string +} + +func (b *offsetCommitRequestBlock) encode(pe packetEncoder, version int16) error { + pe.putInt64(b.offset) + if version == 1 { + pe.putInt64(b.timestamp) + } else if b.timestamp != 0 { + Logger.Println("Non-zero timestamp specified for OffsetCommitRequest not v1, it will be ignored") + } + + return pe.putString(b.metadata) +} + +func (b *offsetCommitRequestBlock) decode(pd packetDecoder, version int16) (err error) { + if b.offset, err = pd.getInt64(); err != nil { + return err + } + if version == 1 { + if b.timestamp, err = pd.getInt64(); err != nil { + return err + } + } + b.metadata, err = pd.getString() + return err +} + +type OffsetCommitRequest struct { + ConsumerGroup string + ConsumerGroupGeneration int32 // v1 or later + ConsumerID string // v1 or later + RetentionTime int64 // v2 or later + + // Version can be: + // - 0 (kafka 0.8.1 and later) + // - 1 (kafka 0.8.2 and later) + // - 2 (kafka 0.9.0 and later) + // - 3 (kafka 0.11.0 and later) + // - 4 (kafka 2.0.0 and later) + Version int16 + blocks map[string]map[int32]*offsetCommitRequestBlock +} + +func (r *OffsetCommitRequest) encode(pe packetEncoder) error { + if r.Version < 0 || r.Version > 4 { + return PacketEncodingError{"invalid or unsupported OffsetCommitRequest version field"} + } + + if err := pe.putString(r.ConsumerGroup); err != nil { + return err + } + + if r.Version >= 1 { + pe.putInt32(r.ConsumerGroupGeneration) + if err := pe.putString(r.ConsumerID); err != nil { + return err + } + } else { + if r.ConsumerGroupGeneration != 0 { + Logger.Println("Non-zero ConsumerGroupGeneration specified for OffsetCommitRequest v0, it will be ignored") + } + if r.ConsumerID != "" { + Logger.Println("Non-empty ConsumerID specified for OffsetCommitRequest v0, it will be ignored") + } + } + + if r.Version >= 2 { + pe.putInt64(r.RetentionTime) + } else if r.RetentionTime != 0 { + Logger.Println("Non-zero RetentionTime specified for OffsetCommitRequest version <2, it will be ignored") + } + + if err := pe.putArrayLength(len(r.blocks)); err != nil { + return err + } + for topic, partitions := range r.blocks { + if err := pe.putString(topic); err != nil { + return err + } + if err := pe.putArrayLength(len(partitions)); err != nil { + return err + } + for partition, block := range partitions { + pe.putInt32(partition) + if err := block.encode(pe, r.Version); err != nil { + return err + } + } + } + return nil +} + +func (r *OffsetCommitRequest) decode(pd packetDecoder, version int16) (err error) { + r.Version = version + + if r.ConsumerGroup, err = pd.getString(); err != nil { + return err + } + + if r.Version >= 1 { + if r.ConsumerGroupGeneration, err = pd.getInt32(); err != nil { + return err + } + if r.ConsumerID, err = pd.getString(); err != nil { + return err + } + } + + if r.Version >= 2 { + if r.RetentionTime, err = pd.getInt64(); err != nil { + return err + } + } + + topicCount, err := pd.getArrayLength() + if err != nil { + return err + } + if topicCount == 0 { + return nil + } + r.blocks = make(map[string]map[int32]*offsetCommitRequestBlock) + for i := 0; i < topicCount; i++ { + topic, err := pd.getString() + if err != nil { + return err + } + partitionCount, err := pd.getArrayLength() + if err != nil { + return err + } + r.blocks[topic] = make(map[int32]*offsetCommitRequestBlock) + for j := 0; j < partitionCount; j++ { + partition, err := pd.getInt32() + if err != nil { + return err + } + block := &offsetCommitRequestBlock{} + if err := block.decode(pd, r.Version); err != nil { + return err + } + r.blocks[topic][partition] = block + } + } + return nil +} + +func (r *OffsetCommitRequest) key() int16 { + return 8 +} + +func (r *OffsetCommitRequest) version() int16 { + return r.Version +} + +func (r *OffsetCommitRequest) headerVersion() int16 { + return 1 +} + +func (r *OffsetCommitRequest) requiredVersion() KafkaVersion { + switch r.Version { + case 1: + return V0_8_2_0 + case 2: + return V0_9_0_0 + case 3: + return V0_11_0_0 + case 4: + return V2_0_0_0 + default: + return MinVersion + } +} + +func (r *OffsetCommitRequest) AddBlock(topic string, partitionID int32, offset int64, timestamp int64, metadata string) { + if r.blocks == nil { + r.blocks = make(map[string]map[int32]*offsetCommitRequestBlock) + } + + if r.blocks[topic] == nil { + r.blocks[topic] = make(map[int32]*offsetCommitRequestBlock) + } + + r.blocks[topic][partitionID] = &offsetCommitRequestBlock{offset, timestamp, metadata} +} + +func (r *OffsetCommitRequest) Offset(topic string, partitionID int32) (int64, string, error) { + partitions := r.blocks[topic] + if partitions == nil { + return 0, "", errors.New("no such offset") + } + block := partitions[partitionID] + if block == nil { + return 0, "", errors.New("no such offset") + } + return block.offset, block.metadata, nil +} diff --git a/vendor/github.com/Shopify/sarama/offset_commit_response.go b/vendor/github.com/Shopify/sarama/offset_commit_response.go new file mode 100644 index 000000000000..342260ef5995 --- /dev/null +++ b/vendor/github.com/Shopify/sarama/offset_commit_response.go @@ -0,0 +1,114 @@ +package sarama + +type OffsetCommitResponse struct { + Version int16 + ThrottleTimeMs int32 + Errors map[string]map[int32]KError +} + +func (r *OffsetCommitResponse) AddError(topic string, partition int32, kerror KError) { + if r.Errors == nil { + r.Errors = make(map[string]map[int32]KError) + } + partitions := r.Errors[topic] + if partitions == nil { + partitions = make(map[int32]KError) + r.Errors[topic] = partitions + } + partitions[partition] = kerror +} + +func (r *OffsetCommitResponse) encode(pe packetEncoder) error { + if r.Version >= 3 { + pe.putInt32(r.ThrottleTimeMs) + } + if err := pe.putArrayLength(len(r.Errors)); err != nil { + return err + } + for topic, partitions := range r.Errors { + if err := pe.putString(topic); err != nil { + return err + } + if err := pe.putArrayLength(len(partitions)); err != nil { + return err + } + for partition, kerror := range partitions { + pe.putInt32(partition) + pe.putInt16(int16(kerror)) + } + } + return nil +} + +func (r *OffsetCommitResponse) decode(pd packetDecoder, version int16) (err error) { + r.Version = version + + if version >= 3 { + r.ThrottleTimeMs, err = pd.getInt32() + if err != nil { + return err + } + } + + numTopics, err := pd.getArrayLength() + if err != nil || numTopics == 0 { + return err + } + + r.Errors = make(map[string]map[int32]KError, numTopics) + for i := 0; i < numTopics; i++ { + name, err := pd.getString() + if err != nil { + return err + } + + numErrors, err := pd.getArrayLength() + if err != nil { + return err + } + + r.Errors[name] = make(map[int32]KError, numErrors) + + for j := 0; j < numErrors; j++ { + id, err := pd.getInt32() + if err != nil { + return err + } + + tmp, err := pd.getInt16() + if err != nil { + return err + } + r.Errors[name][id] = KError(tmp) + } + } + + return nil +} + +func (r *OffsetCommitResponse) key() int16 { + return 8 +} + +func (r *OffsetCommitResponse) version() int16 { + return r.Version +} + +func (r *OffsetCommitResponse) headerVersion() int16 { + return 0 +} + +func (r *OffsetCommitResponse) requiredVersion() KafkaVersion { + switch r.Version { + case 1: + return V0_8_2_0 + case 2: + return V0_9_0_0 + case 3: + return V0_11_0_0 + case 4: + return V2_0_0_0 + default: + return MinVersion + } +} diff --git a/vendor/github.com/Shopify/sarama/offset_fetch_request.go b/vendor/github.com/Shopify/sarama/offset_fetch_request.go new file mode 100644 index 000000000000..7e147eb60c19 --- /dev/null +++ b/vendor/github.com/Shopify/sarama/offset_fetch_request.go @@ -0,0 +1,207 @@ +package sarama + +type OffsetFetchRequest struct { + Version int16 + ConsumerGroup string + RequireStable bool // requires v7+ + partitions map[string][]int32 +} + +func (r *OffsetFetchRequest) encode(pe packetEncoder) (err error) { + if r.Version < 0 || r.Version > 7 { + return PacketEncodingError{"invalid or unsupported OffsetFetchRequest version field"} + } + + isFlexible := r.Version >= 6 + + if isFlexible { + err = pe.putCompactString(r.ConsumerGroup) + } else { + err = pe.putString(r.ConsumerGroup) + } + if err != nil { + return err + } + + if isFlexible { + if r.partitions == nil { + pe.putUVarint(0) + } else { + pe.putCompactArrayLength(len(r.partitions)) + } + } else { + if r.partitions == nil && r.Version >= 2 { + pe.putInt32(-1) + } else { + if err = pe.putArrayLength(len(r.partitions)); err != nil { + return err + } + } + } + + for topic, partitions := range r.partitions { + if isFlexible { + err = pe.putCompactString(topic) + } else { + err = pe.putString(topic) + } + if err != nil { + return err + } + + // + + if isFlexible { + err = pe.putCompactInt32Array(partitions) + } else { + err = pe.putInt32Array(partitions) + } + if err != nil { + return err + } + + if isFlexible { + pe.putEmptyTaggedFieldArray() + } + } + + if r.RequireStable && r.Version < 7 { + return PacketEncodingError{"requireStable is not supported. use version 7 or later"} + } + + if r.Version >= 7 { + pe.putBool(r.RequireStable) + } + + if isFlexible { + pe.putEmptyTaggedFieldArray() + } + + return nil +} + +func (r *OffsetFetchRequest) decode(pd packetDecoder, version int16) (err error) { + r.Version = version + isFlexible := r.Version >= 6 + if isFlexible { + r.ConsumerGroup, err = pd.getCompactString() + } else { + r.ConsumerGroup, err = pd.getString() + } + if err != nil { + return err + } + + var partitionCount int + + if isFlexible { + partitionCount, err = pd.getCompactArrayLength() + } else { + partitionCount, err = pd.getArrayLength() + } + if err != nil { + return err + } + + if (partitionCount == 0 && version < 2) || partitionCount < 0 { + return nil + } + + r.partitions = make(map[string][]int32, partitionCount) + for i := 0; i < partitionCount; i++ { + var topic string + if isFlexible { + topic, err = pd.getCompactString() + } else { + topic, err = pd.getString() + } + if err != nil { + return err + } + + var partitions []int32 + if isFlexible { + partitions, err = pd.getCompactInt32Array() + } else { + partitions, err = pd.getInt32Array() + } + if err != nil { + return err + } + if isFlexible { + _, err = pd.getEmptyTaggedFieldArray() + if err != nil { + return err + } + } + + r.partitions[topic] = partitions + } + + if r.Version >= 7 { + r.RequireStable, err = pd.getBool() + if err != nil { + return err + } + } + + if isFlexible { + _, err = pd.getEmptyTaggedFieldArray() + if err != nil { + return err + } + } + + return nil +} + +func (r *OffsetFetchRequest) key() int16 { + return 9 +} + +func (r *OffsetFetchRequest) version() int16 { + return r.Version +} + +func (r *OffsetFetchRequest) headerVersion() int16 { + if r.Version >= 6 { + return 2 + } + + return 1 +} + +func (r *OffsetFetchRequest) requiredVersion() KafkaVersion { + switch r.Version { + case 1: + return V0_8_2_0 + case 2: + return V0_10_2_0 + case 3: + return V0_11_0_0 + case 4: + return V2_0_0_0 + case 5: + return V2_1_0_0 + case 6: + return V2_4_0_0 + case 7: + return V2_5_0_0 + default: + return MinVersion + } +} + +func (r *OffsetFetchRequest) ZeroPartitions() { + if r.partitions == nil && r.Version >= 2 { + r.partitions = make(map[string][]int32) + } +} + +func (r *OffsetFetchRequest) AddPartition(topic string, partitionID int32) { + if r.partitions == nil { + r.partitions = make(map[string][]int32) + } + + r.partitions[topic] = append(r.partitions[topic], partitionID) +} diff --git a/vendor/github.com/Shopify/sarama/offset_fetch_response.go b/vendor/github.com/Shopify/sarama/offset_fetch_response.go new file mode 100644 index 000000000000..19449220f285 --- /dev/null +++ b/vendor/github.com/Shopify/sarama/offset_fetch_response.go @@ -0,0 +1,280 @@ +package sarama + +type OffsetFetchResponseBlock struct { + Offset int64 + LeaderEpoch int32 + Metadata string + Err KError +} + +func (b *OffsetFetchResponseBlock) decode(pd packetDecoder, version int16) (err error) { + isFlexible := version >= 6 + + b.Offset, err = pd.getInt64() + if err != nil { + return err + } + + if version >= 5 { + b.LeaderEpoch, err = pd.getInt32() + if err != nil { + return err + } + } + + if isFlexible { + b.Metadata, err = pd.getCompactString() + } else { + b.Metadata, err = pd.getString() + } + if err != nil { + return err + } + + tmp, err := pd.getInt16() + if err != nil { + return err + } + b.Err = KError(tmp) + + if isFlexible { + if _, err := pd.getEmptyTaggedFieldArray(); err != nil { + return err + } + } + + return nil +} + +func (b *OffsetFetchResponseBlock) encode(pe packetEncoder, version int16) (err error) { + isFlexible := version >= 6 + pe.putInt64(b.Offset) + + if version >= 5 { + pe.putInt32(b.LeaderEpoch) + } + if isFlexible { + err = pe.putCompactString(b.Metadata) + } else { + err = pe.putString(b.Metadata) + } + if err != nil { + return err + } + + pe.putInt16(int16(b.Err)) + + if isFlexible { + pe.putEmptyTaggedFieldArray() + } + + return nil +} + +type OffsetFetchResponse struct { + Version int16 + ThrottleTimeMs int32 + Blocks map[string]map[int32]*OffsetFetchResponseBlock + Err KError +} + +func (r *OffsetFetchResponse) encode(pe packetEncoder) (err error) { + isFlexible := r.Version >= 6 + + if r.Version >= 3 { + pe.putInt32(r.ThrottleTimeMs) + } + if isFlexible { + pe.putCompactArrayLength(len(r.Blocks)) + } else { + err = pe.putArrayLength(len(r.Blocks)) + } + if err != nil { + return err + } + + for topic, partitions := range r.Blocks { + if isFlexible { + err = pe.putCompactString(topic) + } else { + err = pe.putString(topic) + } + if err != nil { + return err + } + + if isFlexible { + pe.putCompactArrayLength(len(partitions)) + } else { + err = pe.putArrayLength(len(partitions)) + } + if err != nil { + return err + } + for partition, block := range partitions { + pe.putInt32(partition) + if err := block.encode(pe, r.Version); err != nil { + return err + } + } + if isFlexible { + pe.putEmptyTaggedFieldArray() + } + } + if r.Version >= 2 { + pe.putInt16(int16(r.Err)) + } + if isFlexible { + pe.putEmptyTaggedFieldArray() + } + return nil +} + +func (r *OffsetFetchResponse) decode(pd packetDecoder, version int16) (err error) { + r.Version = version + isFlexible := version >= 6 + + if version >= 3 { + r.ThrottleTimeMs, err = pd.getInt32() + if err != nil { + return err + } + } + + var numTopics int + if isFlexible { + numTopics, err = pd.getCompactArrayLength() + } else { + numTopics, err = pd.getArrayLength() + } + if err != nil { + return err + } + + if numTopics > 0 { + r.Blocks = make(map[string]map[int32]*OffsetFetchResponseBlock, numTopics) + for i := 0; i < numTopics; i++ { + var name string + if isFlexible { + name, err = pd.getCompactString() + } else { + name, err = pd.getString() + } + if err != nil { + return err + } + + var numBlocks int + if isFlexible { + numBlocks, err = pd.getCompactArrayLength() + } else { + numBlocks, err = pd.getArrayLength() + } + if err != nil { + return err + } + + r.Blocks[name] = nil + if numBlocks > 0 { + r.Blocks[name] = make(map[int32]*OffsetFetchResponseBlock, numBlocks) + } + for j := 0; j < numBlocks; j++ { + id, err := pd.getInt32() + if err != nil { + return err + } + + block := new(OffsetFetchResponseBlock) + err = block.decode(pd, version) + if err != nil { + return err + } + + r.Blocks[name][id] = block + } + + if isFlexible { + if _, err := pd.getEmptyTaggedFieldArray(); err != nil { + return err + } + } + } + } + + if version >= 2 { + kerr, err := pd.getInt16() + if err != nil { + return err + } + r.Err = KError(kerr) + } + + if isFlexible { + if _, err := pd.getEmptyTaggedFieldArray(); err != nil { + return err + } + } + + return nil +} + +func (r *OffsetFetchResponse) key() int16 { + return 9 +} + +func (r *OffsetFetchResponse) version() int16 { + return r.Version +} + +func (r *OffsetFetchResponse) headerVersion() int16 { + if r.Version >= 6 { + return 1 + } + + return 0 +} + +func (r *OffsetFetchResponse) requiredVersion() KafkaVersion { + switch r.Version { + case 1: + return V0_8_2_0 + case 2: + return V0_10_2_0 + case 3: + return V0_11_0_0 + case 4: + return V2_0_0_0 + case 5: + return V2_1_0_0 + case 6: + return V2_4_0_0 + case 7: + return V2_5_0_0 + default: + return MinVersion + } +} + +func (r *OffsetFetchResponse) GetBlock(topic string, partition int32) *OffsetFetchResponseBlock { + if r.Blocks == nil { + return nil + } + + if r.Blocks[topic] == nil { + return nil + } + + return r.Blocks[topic][partition] +} + +func (r *OffsetFetchResponse) AddBlock(topic string, partition int32, block *OffsetFetchResponseBlock) { + if r.Blocks == nil { + r.Blocks = make(map[string]map[int32]*OffsetFetchResponseBlock) + } + partitions := r.Blocks[topic] + if partitions == nil { + partitions = make(map[int32]*OffsetFetchResponseBlock) + r.Blocks[topic] = partitions + } + partitions[partition] = block +} diff --git a/vendor/github.com/Shopify/sarama/offset_manager.go b/vendor/github.com/Shopify/sarama/offset_manager.go new file mode 100644 index 000000000000..4f480a08b905 --- /dev/null +++ b/vendor/github.com/Shopify/sarama/offset_manager.go @@ -0,0 +1,593 @@ +package sarama + +import ( + "sync" + "time" +) + +// Offset Manager + +// OffsetManager uses Kafka to store and fetch consumed partition offsets. +type OffsetManager interface { + // ManagePartition creates a PartitionOffsetManager on the given topic/partition. + // It will return an error if this OffsetManager is already managing the given + // topic/partition. + ManagePartition(topic string, partition int32) (PartitionOffsetManager, error) + + // Close stops the OffsetManager from managing offsets. It is required to call + // this function before an OffsetManager object passes out of scope, as it + // will otherwise leak memory. You must call this after all the + // PartitionOffsetManagers are closed. + Close() error + + // Commit commits the offsets. This method can be used if AutoCommit.Enable is + // set to false. + Commit() +} + +type offsetManager struct { + client Client + conf *Config + group string + ticker *time.Ticker + + memberID string + generation int32 + + broker *Broker + brokerLock sync.RWMutex + + poms map[string]map[int32]*partitionOffsetManager + pomsLock sync.RWMutex + + closeOnce sync.Once + closing chan none + closed chan none +} + +// NewOffsetManagerFromClient creates a new OffsetManager from the given client. +// It is still necessary to call Close() on the underlying client when finished with the partition manager. +func NewOffsetManagerFromClient(group string, client Client) (OffsetManager, error) { + return newOffsetManagerFromClient(group, "", GroupGenerationUndefined, client) +} + +func newOffsetManagerFromClient(group, memberID string, generation int32, client Client) (*offsetManager, error) { + // Check that we are not dealing with a closed Client before processing any other arguments + if client.Closed() { + return nil, ErrClosedClient + } + + conf := client.Config() + om := &offsetManager{ + client: client, + conf: conf, + group: group, + poms: make(map[string]map[int32]*partitionOffsetManager), + + memberID: memberID, + generation: generation, + + closing: make(chan none), + closed: make(chan none), + } + if conf.Consumer.Offsets.AutoCommit.Enable { + om.ticker = time.NewTicker(conf.Consumer.Offsets.AutoCommit.Interval) + go withRecover(om.mainLoop) + } + + return om, nil +} + +func (om *offsetManager) ManagePartition(topic string, partition int32) (PartitionOffsetManager, error) { + pom, err := om.newPartitionOffsetManager(topic, partition) + if err != nil { + return nil, err + } + + om.pomsLock.Lock() + defer om.pomsLock.Unlock() + + topicManagers := om.poms[topic] + if topicManagers == nil { + topicManagers = make(map[int32]*partitionOffsetManager) + om.poms[topic] = topicManagers + } + + if topicManagers[partition] != nil { + return nil, ConfigurationError("That topic/partition is already being managed") + } + + topicManagers[partition] = pom + return pom, nil +} + +func (om *offsetManager) Close() error { + om.closeOnce.Do(func() { + // exit the mainLoop + close(om.closing) + if om.conf.Consumer.Offsets.AutoCommit.Enable { + <-om.closed + } + + // mark all POMs as closed + om.asyncClosePOMs() + + // flush one last time + if om.conf.Consumer.Offsets.AutoCommit.Enable { + for attempt := 0; attempt <= om.conf.Consumer.Offsets.Retry.Max; attempt++ { + om.flushToBroker() + if om.releasePOMs(false) == 0 { + break + } + } + } + + om.releasePOMs(true) + om.brokerLock.Lock() + om.broker = nil + om.brokerLock.Unlock() + }) + return nil +} + +func (om *offsetManager) computeBackoff(retries int) time.Duration { + if om.conf.Metadata.Retry.BackoffFunc != nil { + return om.conf.Metadata.Retry.BackoffFunc(retries, om.conf.Metadata.Retry.Max) + } else { + return om.conf.Metadata.Retry.Backoff + } +} + +func (om *offsetManager) fetchInitialOffset(topic string, partition int32, retries int) (int64, string, error) { + broker, err := om.coordinator() + if err != nil { + if retries <= 0 { + return 0, "", err + } + return om.fetchInitialOffset(topic, partition, retries-1) + } + + req := new(OffsetFetchRequest) + req.Version = 1 + req.ConsumerGroup = om.group + req.AddPartition(topic, partition) + + resp, err := broker.FetchOffset(req) + if err != nil { + if retries <= 0 { + return 0, "", err + } + om.releaseCoordinator(broker) + return om.fetchInitialOffset(topic, partition, retries-1) + } + + block := resp.GetBlock(topic, partition) + if block == nil { + return 0, "", ErrIncompleteResponse + } + + switch block.Err { + case ErrNoError: + return block.Offset, block.Metadata, nil + case ErrNotCoordinatorForConsumer: + if retries <= 0 { + return 0, "", block.Err + } + om.releaseCoordinator(broker) + return om.fetchInitialOffset(topic, partition, retries-1) + case ErrOffsetsLoadInProgress: + if retries <= 0 { + return 0, "", block.Err + } + backoff := om.computeBackoff(retries) + select { + case <-om.closing: + return 0, "", block.Err + case <-time.After(backoff): + } + return om.fetchInitialOffset(topic, partition, retries-1) + default: + return 0, "", block.Err + } +} + +func (om *offsetManager) coordinator() (*Broker, error) { + om.brokerLock.RLock() + broker := om.broker + om.brokerLock.RUnlock() + + if broker != nil { + return broker, nil + } + + om.brokerLock.Lock() + defer om.brokerLock.Unlock() + + if broker := om.broker; broker != nil { + return broker, nil + } + + if err := om.client.RefreshCoordinator(om.group); err != nil { + return nil, err + } + + broker, err := om.client.Coordinator(om.group) + if err != nil { + return nil, err + } + + om.broker = broker + return broker, nil +} + +func (om *offsetManager) releaseCoordinator(b *Broker) { + om.brokerLock.Lock() + if om.broker == b { + om.broker = nil + } + om.brokerLock.Unlock() +} + +func (om *offsetManager) mainLoop() { + defer om.ticker.Stop() + defer close(om.closed) + + for { + select { + case <-om.ticker.C: + om.Commit() + case <-om.closing: + return + } + } +} + +func (om *offsetManager) Commit() { + om.flushToBroker() + om.releasePOMs(false) +} + +func (om *offsetManager) flushToBroker() { + req := om.constructRequest() + if req == nil { + return + } + + broker, err := om.coordinator() + if err != nil { + om.handleError(err) + return + } + + resp, err := broker.CommitOffset(req) + if err != nil { + om.handleError(err) + om.releaseCoordinator(broker) + _ = broker.Close() + return + } + + om.handleResponse(broker, req, resp) +} + +func (om *offsetManager) constructRequest() *OffsetCommitRequest { + var r *OffsetCommitRequest + var perPartitionTimestamp int64 + if om.conf.Consumer.Offsets.Retention == 0 { + perPartitionTimestamp = ReceiveTime + r = &OffsetCommitRequest{ + Version: 1, + ConsumerGroup: om.group, + ConsumerID: om.memberID, + ConsumerGroupGeneration: om.generation, + } + } else { + r = &OffsetCommitRequest{ + Version: 2, + RetentionTime: int64(om.conf.Consumer.Offsets.Retention / time.Millisecond), + ConsumerGroup: om.group, + ConsumerID: om.memberID, + ConsumerGroupGeneration: om.generation, + } + } + + om.pomsLock.RLock() + defer om.pomsLock.RUnlock() + + for _, topicManagers := range om.poms { + for _, pom := range topicManagers { + pom.lock.Lock() + if pom.dirty { + r.AddBlock(pom.topic, pom.partition, pom.offset, perPartitionTimestamp, pom.metadata) + } + pom.lock.Unlock() + } + } + + if len(r.blocks) > 0 { + return r + } + + return nil +} + +func (om *offsetManager) handleResponse(broker *Broker, req *OffsetCommitRequest, resp *OffsetCommitResponse) { + om.pomsLock.RLock() + defer om.pomsLock.RUnlock() + + for _, topicManagers := range om.poms { + for _, pom := range topicManagers { + if req.blocks[pom.topic] == nil || req.blocks[pom.topic][pom.partition] == nil { + continue + } + + var err KError + var ok bool + + if resp.Errors[pom.topic] == nil { + pom.handleError(ErrIncompleteResponse) + continue + } + if err, ok = resp.Errors[pom.topic][pom.partition]; !ok { + pom.handleError(ErrIncompleteResponse) + continue + } + + switch err { + case ErrNoError: + block := req.blocks[pom.topic][pom.partition] + pom.updateCommitted(block.offset, block.metadata) + case ErrNotLeaderForPartition, ErrLeaderNotAvailable, + ErrConsumerCoordinatorNotAvailable, ErrNotCoordinatorForConsumer: + // not a critical error, we just need to redispatch + om.releaseCoordinator(broker) + case ErrOffsetMetadataTooLarge, ErrInvalidCommitOffsetSize: + // nothing we can do about this, just tell the user and carry on + pom.handleError(err) + case ErrOffsetsLoadInProgress: + // nothing wrong but we didn't commit, we'll get it next time round + case ErrUnknownTopicOrPartition: + // let the user know *and* try redispatching - if topic-auto-create is + // enabled, redispatching should trigger a metadata req and create the + // topic; if not then re-dispatching won't help, but we've let the user + // know and it shouldn't hurt either (see https://github.com/Shopify/sarama/issues/706) + fallthrough + default: + // dunno, tell the user and try redispatching + pom.handleError(err) + om.releaseCoordinator(broker) + } + } + } +} + +func (om *offsetManager) handleError(err error) { + om.pomsLock.RLock() + defer om.pomsLock.RUnlock() + + for _, topicManagers := range om.poms { + for _, pom := range topicManagers { + pom.handleError(err) + } + } +} + +func (om *offsetManager) asyncClosePOMs() { + om.pomsLock.RLock() + defer om.pomsLock.RUnlock() + + for _, topicManagers := range om.poms { + for _, pom := range topicManagers { + pom.AsyncClose() + } + } +} + +// Releases/removes closed POMs once they are clean (or when forced) +func (om *offsetManager) releasePOMs(force bool) (remaining int) { + om.pomsLock.Lock() + defer om.pomsLock.Unlock() + + for topic, topicManagers := range om.poms { + for partition, pom := range topicManagers { + pom.lock.Lock() + releaseDue := pom.done && (force || !pom.dirty) + pom.lock.Unlock() + + if releaseDue { + pom.release() + + delete(om.poms[topic], partition) + if len(om.poms[topic]) == 0 { + delete(om.poms, topic) + } + } + } + remaining += len(om.poms[topic]) + } + return +} + +func (om *offsetManager) findPOM(topic string, partition int32) *partitionOffsetManager { + om.pomsLock.RLock() + defer om.pomsLock.RUnlock() + + if partitions, ok := om.poms[topic]; ok { + if pom, ok := partitions[partition]; ok { + return pom + } + } + return nil +} + +// Partition Offset Manager + +// PartitionOffsetManager uses Kafka to store and fetch consumed partition offsets. You MUST call Close() +// on a partition offset manager to avoid leaks, it will not be garbage-collected automatically when it passes +// out of scope. +type PartitionOffsetManager interface { + // NextOffset returns the next offset that should be consumed for the managed + // partition, accompanied by metadata which can be used to reconstruct the state + // of the partition consumer when it resumes. NextOffset() will return + // `config.Consumer.Offsets.Initial` and an empty metadata string if no offset + // was committed for this partition yet. + NextOffset() (int64, string) + + // MarkOffset marks the provided offset, alongside a metadata string + // that represents the state of the partition consumer at that point in time. The + // metadata string can be used by another consumer to restore that state, so it + // can resume consumption. + // + // To follow upstream conventions, you are expected to mark the offset of the + // next message to read, not the last message read. Thus, when calling `MarkOffset` + // you should typically add one to the offset of the last consumed message. + // + // Note: calling MarkOffset does not necessarily commit the offset to the backend + // store immediately for efficiency reasons, and it may never be committed if + // your application crashes. This means that you may end up processing the same + // message twice, and your processing should ideally be idempotent. + MarkOffset(offset int64, metadata string) + + // ResetOffset resets to the provided offset, alongside a metadata string that + // represents the state of the partition consumer at that point in time. Reset + // acts as a counterpart to MarkOffset, the difference being that it allows to + // reset an offset to an earlier or smaller value, where MarkOffset only + // allows incrementing the offset. cf MarkOffset for more details. + ResetOffset(offset int64, metadata string) + + // Errors returns a read channel of errors that occur during offset management, if + // enabled. By default, errors are logged and not returned over this channel. If + // you want to implement any custom error handling, set your config's + // Consumer.Return.Errors setting to true, and read from this channel. + Errors() <-chan *ConsumerError + + // AsyncClose initiates a shutdown of the PartitionOffsetManager. This method will + // return immediately, after which you should wait until the 'errors' channel has + // been drained and closed. It is required to call this function, or Close before + // a consumer object passes out of scope, as it will otherwise leak memory. You + // must call this before calling Close on the underlying client. + AsyncClose() + + // Close stops the PartitionOffsetManager from managing offsets. It is required to + // call this function (or AsyncClose) before a PartitionOffsetManager object + // passes out of scope, as it will otherwise leak memory. You must call this + // before calling Close on the underlying client. + Close() error +} + +type partitionOffsetManager struct { + parent *offsetManager + topic string + partition int32 + + lock sync.Mutex + offset int64 + metadata string + dirty bool + done bool + + releaseOnce sync.Once + errors chan *ConsumerError +} + +func (om *offsetManager) newPartitionOffsetManager(topic string, partition int32) (*partitionOffsetManager, error) { + offset, metadata, err := om.fetchInitialOffset(topic, partition, om.conf.Metadata.Retry.Max) + if err != nil { + return nil, err + } + + return &partitionOffsetManager{ + parent: om, + topic: topic, + partition: partition, + errors: make(chan *ConsumerError, om.conf.ChannelBufferSize), + offset: offset, + metadata: metadata, + }, nil +} + +func (pom *partitionOffsetManager) Errors() <-chan *ConsumerError { + return pom.errors +} + +func (pom *partitionOffsetManager) MarkOffset(offset int64, metadata string) { + pom.lock.Lock() + defer pom.lock.Unlock() + + if offset > pom.offset { + pom.offset = offset + pom.metadata = metadata + pom.dirty = true + } +} + +func (pom *partitionOffsetManager) ResetOffset(offset int64, metadata string) { + pom.lock.Lock() + defer pom.lock.Unlock() + + if offset <= pom.offset { + pom.offset = offset + pom.metadata = metadata + pom.dirty = true + } +} + +func (pom *partitionOffsetManager) updateCommitted(offset int64, metadata string) { + pom.lock.Lock() + defer pom.lock.Unlock() + + if pom.offset == offset && pom.metadata == metadata { + pom.dirty = false + } +} + +func (pom *partitionOffsetManager) NextOffset() (int64, string) { + pom.lock.Lock() + defer pom.lock.Unlock() + + if pom.offset >= 0 { + return pom.offset, pom.metadata + } + + return pom.parent.conf.Consumer.Offsets.Initial, "" +} + +func (pom *partitionOffsetManager) AsyncClose() { + pom.lock.Lock() + pom.done = true + pom.lock.Unlock() +} + +func (pom *partitionOffsetManager) Close() error { + pom.AsyncClose() + + var errors ConsumerErrors + for err := range pom.errors { + errors = append(errors, err) + } + + if len(errors) > 0 { + return errors + } + return nil +} + +func (pom *partitionOffsetManager) handleError(err error) { + cErr := &ConsumerError{ + Topic: pom.topic, + Partition: pom.partition, + Err: err, + } + + if pom.parent.conf.Consumer.Return.Errors { + pom.errors <- cErr + } else { + Logger.Println(cErr) + } +} + +func (pom *partitionOffsetManager) release() { + pom.releaseOnce.Do(func() { + close(pom.errors) + }) +} diff --git a/vendor/github.com/Shopify/sarama/offset_request.go b/vendor/github.com/Shopify/sarama/offset_request.go new file mode 100644 index 000000000000..4c9ce4df552c --- /dev/null +++ b/vendor/github.com/Shopify/sarama/offset_request.go @@ -0,0 +1,179 @@ +package sarama + +type offsetRequestBlock struct { + time int64 + maxOffsets int32 // Only used in version 0 +} + +func (b *offsetRequestBlock) encode(pe packetEncoder, version int16) error { + pe.putInt64(b.time) + if version == 0 { + pe.putInt32(b.maxOffsets) + } + + return nil +} + +func (b *offsetRequestBlock) decode(pd packetDecoder, version int16) (err error) { + if b.time, err = pd.getInt64(); err != nil { + return err + } + if version == 0 { + if b.maxOffsets, err = pd.getInt32(); err != nil { + return err + } + } + return nil +} + +type OffsetRequest struct { + Version int16 + IsolationLevel IsolationLevel + replicaID int32 + isReplicaIDSet bool + blocks map[string]map[int32]*offsetRequestBlock +} + +func (r *OffsetRequest) encode(pe packetEncoder) error { + if r.isReplicaIDSet { + pe.putInt32(r.replicaID) + } else { + // default replica ID is always -1 for clients + pe.putInt32(-1) + } + + if r.Version >= 2 { + pe.putBool(r.IsolationLevel == ReadCommitted) + } + + err := pe.putArrayLength(len(r.blocks)) + if err != nil { + return err + } + for topic, partitions := range r.blocks { + err = pe.putString(topic) + if err != nil { + return err + } + err = pe.putArrayLength(len(partitions)) + if err != nil { + return err + } + for partition, block := range partitions { + pe.putInt32(partition) + if err = block.encode(pe, r.Version); err != nil { + return err + } + } + } + return nil +} + +func (r *OffsetRequest) decode(pd packetDecoder, version int16) error { + r.Version = version + + replicaID, err := pd.getInt32() + if err != nil { + return err + } + if replicaID >= 0 { + r.SetReplicaID(replicaID) + } + + if r.Version >= 2 { + tmp, err := pd.getBool() + if err != nil { + return err + } + + r.IsolationLevel = ReadUncommitted + if tmp { + r.IsolationLevel = ReadCommitted + } + } + + blockCount, err := pd.getArrayLength() + if err != nil { + return err + } + if blockCount == 0 { + return nil + } + r.blocks = make(map[string]map[int32]*offsetRequestBlock) + for i := 0; i < blockCount; i++ { + topic, err := pd.getString() + if err != nil { + return err + } + partitionCount, err := pd.getArrayLength() + if err != nil { + return err + } + r.blocks[topic] = make(map[int32]*offsetRequestBlock) + for j := 0; j < partitionCount; j++ { + partition, err := pd.getInt32() + if err != nil { + return err + } + block := &offsetRequestBlock{} + if err := block.decode(pd, version); err != nil { + return err + } + r.blocks[topic][partition] = block + } + } + return nil +} + +func (r *OffsetRequest) key() int16 { + return 2 +} + +func (r *OffsetRequest) version() int16 { + return r.Version +} + +func (r *OffsetRequest) headerVersion() int16 { + return 1 +} + +func (r *OffsetRequest) requiredVersion() KafkaVersion { + switch r.Version { + case 1: + return V0_10_1_0 + case 2: + return V0_11_0_0 + default: + return MinVersion + } +} + +func (r *OffsetRequest) SetReplicaID(id int32) { + r.replicaID = id + r.isReplicaIDSet = true +} + +func (r *OffsetRequest) ReplicaID() int32 { + if r.isReplicaIDSet { + return r.replicaID + } + return -1 +} + +func (r *OffsetRequest) AddBlock(topic string, partitionID int32, time int64, maxOffsets int32) { + if r.blocks == nil { + r.blocks = make(map[string]map[int32]*offsetRequestBlock) + } + + if r.blocks[topic] == nil { + r.blocks[topic] = make(map[int32]*offsetRequestBlock) + } + + tmp := new(offsetRequestBlock) + tmp.time = time + if r.Version == 0 { + tmp.maxOffsets = maxOffsets + } + + r.blocks[topic][partitionID] = tmp +} diff --git a/vendor/github.com/Shopify/sarama/offset_response.go b/vendor/github.com/Shopify/sarama/offset_response.go new file mode 100644 index 000000000000..69349efe2ba0 --- /dev/null +++ b/vendor/github.com/Shopify/sarama/offset_response.go @@ -0,0 +1,192 @@ +package sarama + +type OffsetResponseBlock struct { + Err KError + Offsets []int64 // Version 0 + Offset int64 // Version 1 + Timestamp int64 // Version 1 +} + +func (b *OffsetResponseBlock) decode(pd packetDecoder, version int16) (err error) { + tmp, err := pd.getInt16() + if err != nil { + return err + } + b.Err = KError(tmp) + + if version == 0 { + b.Offsets, err = pd.getInt64Array() + + return err + } + + b.Timestamp, err = pd.getInt64() + if err != nil { + return err + } + + b.Offset, err = pd.getInt64() + if err != nil { + return err + } + + // For backwards compatibility put the offset in the offsets array too + b.Offsets = []int64{b.Offset} + + return nil +} + +func (b *OffsetResponseBlock) encode(pe packetEncoder, version int16) (err error) { + pe.putInt16(int16(b.Err)) + + if version == 0 { + return pe.putInt64Array(b.Offsets) + } + + pe.putInt64(b.Timestamp) + pe.putInt64(b.Offset) + + return nil +} + +type OffsetResponse struct { + Version int16 + ThrottleTimeMs int32 + Blocks map[string]map[int32]*OffsetResponseBlock +} + +func (r *OffsetResponse) decode(pd packetDecoder, version int16) (err error) { + if version >= 2 { + r.ThrottleTimeMs, err = pd.getInt32() + if err != nil { + return err + } + } + + numTopics, err := pd.getArrayLength() + if err != nil { + return err + } + + r.Blocks = make(map[string]map[int32]*OffsetResponseBlock, numTopics) + for i := 0; i < numTopics; i++ { + name, err := pd.getString() + if err != nil { + return err + } + + numBlocks, err := pd.getArrayLength() + if err != nil { + return err + } + + r.Blocks[name] = make(map[int32]*OffsetResponseBlock, numBlocks) + + for j := 0; j < numBlocks; j++ { + id, err := pd.getInt32() + if err != nil { + return err + } + + block := new(OffsetResponseBlock) + err = block.decode(pd, version) + if err != nil { + return err + } + r.Blocks[name][id] = block + } + } + + return nil +} + +func (r *OffsetResponse) GetBlock(topic string, partition int32) *OffsetResponseBlock { + if r.Blocks == nil { + return nil + } + + if r.Blocks[topic] == nil { + return nil + } + + return r.Blocks[topic][partition] +} + +/* +// [0 0 0 1 ntopics +0 8 109 121 95 116 111 112 105 99 topic +0 0 0 1 npartitions +0 0 0 0 id +0 0 + +0 0 0 1 0 0 0 0 +0 1 1 1 0 0 0 1 +0 8 109 121 95 116 111 112 +105 99 0 0 0 1 0 0 +0 0 0 0 0 0 0 1 +0 0 0 0 0 1 1 1] + +*/ +func (r *OffsetResponse) encode(pe packetEncoder) (err error) { + if r.Version >= 2 { + pe.putInt32(r.ThrottleTimeMs) + } + + if err = pe.putArrayLength(len(r.Blocks)); err != nil { + return err + } + + for topic, partitions := range r.Blocks { + if err = pe.putString(topic); err != nil { + return err + } + if err = pe.putArrayLength(len(partitions)); err != nil { + return err + } + for partition, block := range partitions { + pe.putInt32(partition) + if err = block.encode(pe, r.version()); err != nil { + return err + } + } + } + + return nil +} + +func (r *OffsetResponse) key() int16 { + return 2 +} + +func (r *OffsetResponse) version() int16 { + return r.Version +} + +func (r *OffsetResponse) headerVersion() int16 { + return 0 +} + +func (r *OffsetResponse) requiredVersion() KafkaVersion { + switch r.Version { + case 1: + return V0_10_1_0 + case 2: + return V0_11_0_0 + default: + return MinVersion + } +} + +// testing API + +func (r *OffsetResponse) AddTopicPartition(topic string, partition int32, offset int64) { + if r.Blocks == nil { + r.Blocks = make(map[string]map[int32]*OffsetResponseBlock) + } + byTopic, ok := r.Blocks[topic] + if !ok { + byTopic = make(map[int32]*OffsetResponseBlock) + r.Blocks[topic] = byTopic + } + byTopic[partition] = &OffsetResponseBlock{Offsets: []int64{offset}, Offset: offset} +} diff --git a/vendor/github.com/Shopify/sarama/packet_decoder.go b/vendor/github.com/Shopify/sarama/packet_decoder.go new file mode 100644 index 000000000000..08b433223409 --- /dev/null +++ b/vendor/github.com/Shopify/sarama/packet_decoder.go @@ -0,0 +1,69 @@ +package sarama + +// PacketDecoder is the interface providing helpers for reading with Kafka's encoding rules. +// Types implementing Decoder only need to worry about calling methods like GetString, +// not about how a string is represented in Kafka. +type packetDecoder interface { + // Primitives + getInt8() (int8, error) + getInt16() (int16, error) + getInt32() (int32, error) + getInt64() (int64, error) + getVarint() (int64, error) + getUVarint() (uint64, error) + getFloat64() (float64, error) + getArrayLength() (int, error) + getCompactArrayLength() (int, error) + getBool() (bool, error) + getEmptyTaggedFieldArray() (int, error) + + // Collections + getBytes() ([]byte, error) + getVarintBytes() ([]byte, error) + getCompactBytes() ([]byte, error) + getRawBytes(length int) ([]byte, error) + getString() (string, error) + getNullableString() (*string, error) + getCompactString() (string, error) + getCompactNullableString() (*string, error) + getCompactInt32Array() ([]int32, error) + getInt32Array() ([]int32, error) + getInt64Array() ([]int64, error) + getStringArray() ([]string, error) + + // Subsets + remaining() int + getSubset(length int) (packetDecoder, error) + peek(offset, length int) (packetDecoder, error) // similar to getSubset, but it doesn't advance the offset + peekInt8(offset int) (int8, error) // similar to peek, but just one byte + + // Stacks, see PushDecoder + push(in pushDecoder) error + pop() error +} + +// PushDecoder is the interface for decoding fields like CRCs and lengths where the validity +// of the field depends on what is after it in the packet. Start them with PacketDecoder.Push() where +// the actual value is located in the packet, then PacketDecoder.Pop() them when all the bytes they +// depend upon have been decoded. +type pushDecoder interface { + // Saves the offset into the input buffer as the location to actually read the calculated value when able. + saveOffset(in int) + + // Returns the length of data to reserve for the input of this encoder (eg 4 bytes for a CRC32). + reserveLength() int + + // Indicates that all required data is now available to calculate and check the field. + // SaveOffset is guaranteed to have been called first. The implementation should read ReserveLength() bytes + // of data from the saved offset, and verify it based on the data between the saved offset and curOffset. + check(curOffset int, buf []byte) error +} + +// dynamicPushDecoder extends the interface of pushDecoder for uses cases where the length of the +// fields itself is unknown until its value was decoded (for instance varint encoded length +// fields). +// During push, dynamicPushDecoder.decode() method will be called instead of reserveLength() +type dynamicPushDecoder interface { + pushDecoder + decoder +} diff --git a/vendor/github.com/Shopify/sarama/packet_encoder.go b/vendor/github.com/Shopify/sarama/packet_encoder.go new file mode 100644 index 000000000000..5016e09a6ceb --- /dev/null +++ b/vendor/github.com/Shopify/sarama/packet_encoder.go @@ -0,0 +1,74 @@ +package sarama + +import "github.com/rcrowley/go-metrics" + +// PacketEncoder is the interface providing helpers for writing with Kafka's encoding rules. +// Types implementing Encoder only need to worry about calling methods like PutString, +// not about how a string is represented in Kafka. +type packetEncoder interface { + // Primitives + putInt8(in int8) + putInt16(in int16) + putInt32(in int32) + putInt64(in int64) + putVarint(in int64) + putUVarint(in uint64) + putFloat64(in float64) + putCompactArrayLength(in int) + putArrayLength(in int) error + putBool(in bool) + + // Collections + putBytes(in []byte) error + putVarintBytes(in []byte) error + putCompactBytes(in []byte) error + putRawBytes(in []byte) error + putCompactString(in string) error + putNullableCompactString(in *string) error + putString(in string) error + putNullableString(in *string) error + putStringArray(in []string) error + putCompactInt32Array(in []int32) error + putNullableCompactInt32Array(in []int32) error + putInt32Array(in []int32) error + putInt64Array(in []int64) error + putEmptyTaggedFieldArray() + + // Provide the current offset to record the batch size metric + offset() int + + // Stacks, see PushEncoder + push(in pushEncoder) + pop() error + + // To record metrics when provided + metricRegistry() metrics.Registry +} + +// PushEncoder is the interface for encoding fields like CRCs and lengths where the value +// of the field depends on what is encoded after it in the packet. Start them with PacketEncoder.Push() where +// the actual value is located in the packet, then PacketEncoder.Pop() them when all the bytes they +// depend upon have been written. +type pushEncoder interface { + // Saves the offset into the input buffer as the location to actually write the calculated value when able. + saveOffset(in int) + + // Returns the length of data to reserve for the output of this encoder (eg 4 bytes for a CRC32). + reserveLength() int + + // Indicates that all required data is now available to calculate and write the field. + // SaveOffset is guaranteed to have been called first. The implementation should write ReserveLength() bytes + // of data to the saved offset, based on the data between the saved offset and curOffset. + run(curOffset int, buf []byte) error +} + +// dynamicPushEncoder extends the interface of pushEncoder for uses cases where the length of the +// fields itself is unknown until its value was computed (for instance varint encoded length +// fields). +type dynamicPushEncoder interface { + pushEncoder + + // Called during pop() to adjust the length of the field. + // It should return the difference in bytes between the last computed length and current length. + adjustLength(currOffset int) int +} diff --git a/vendor/github.com/Shopify/sarama/partitioner.go b/vendor/github.com/Shopify/sarama/partitioner.go new file mode 100644 index 000000000000..57377760a7cb --- /dev/null +++ b/vendor/github.com/Shopify/sarama/partitioner.go @@ -0,0 +1,217 @@ +package sarama + +import ( + "hash" + "hash/fnv" + "math/rand" + "time" +) + +// Partitioner is anything that, given a Kafka message and a number of partitions indexed [0...numPartitions-1], +// decides to which partition to send the message. RandomPartitioner, RoundRobinPartitioner and HashPartitioner are provided +// as simple default implementations. +type Partitioner interface { + // Partition takes a message and partition count and chooses a partition + Partition(message *ProducerMessage, numPartitions int32) (int32, error) + + // RequiresConsistency indicates to the user of the partitioner whether the + // mapping of key->partition is consistent or not. Specifically, if a + // partitioner requires consistency then it must be allowed to choose from all + // partitions (even ones known to be unavailable), and its choice must be + // respected by the caller. The obvious example is the HashPartitioner. + RequiresConsistency() bool +} + +// DynamicConsistencyPartitioner can optionally be implemented by Partitioners +// in order to allow more flexibility than is originally allowed by the +// RequiresConsistency method in the Partitioner interface. This allows +// partitioners to require consistency sometimes, but not all times. It's useful +// for, e.g., the HashPartitioner, which does not require consistency if the +// message key is nil. +type DynamicConsistencyPartitioner interface { + Partitioner + + // MessageRequiresConsistency is similar to Partitioner.RequiresConsistency, + // but takes in the message being partitioned so that the partitioner can + // make a per-message determination. + MessageRequiresConsistency(message *ProducerMessage) bool +} + +// PartitionerConstructor is the type for a function capable of constructing new Partitioners. +type PartitionerConstructor func(topic string) Partitioner + +type manualPartitioner struct{} + +// HashPartitionerOption lets you modify default values of the partitioner +type HashPartitionerOption func(*hashPartitioner) + +// WithAbsFirst means that the partitioner handles absolute values +// in the same way as the reference Java implementation +func WithAbsFirst() HashPartitionerOption { + return func(hp *hashPartitioner) { + hp.referenceAbs = true + } +} + +// WithCustomHashFunction lets you specify what hash function to use for the partitioning +func WithCustomHashFunction(hasher func() hash.Hash32) HashPartitionerOption { + return func(hp *hashPartitioner) { + hp.hasher = hasher() + } +} + +// WithCustomFallbackPartitioner lets you specify what HashPartitioner should be used in case a Distribution Key is empty +func WithCustomFallbackPartitioner(randomHP Partitioner) HashPartitionerOption { + return func(hp *hashPartitioner) { + hp.random = randomHP + } +} + +// NewManualPartitioner returns a Partitioner which uses the partition manually set in the provided +// ProducerMessage's Partition field as the partition to produce to. +func NewManualPartitioner(topic string) Partitioner { + return new(manualPartitioner) +} + +func (p *manualPartitioner) Partition(message *ProducerMessage, numPartitions int32) (int32, error) { + return message.Partition, nil +} + +func (p *manualPartitioner) RequiresConsistency() bool { + return true +} + +type randomPartitioner struct { + generator *rand.Rand +} + +// NewRandomPartitioner returns a Partitioner which chooses a random partition each time. +func NewRandomPartitioner(topic string) Partitioner { + p := new(randomPartitioner) + p.generator = rand.New(rand.NewSource(time.Now().UTC().UnixNano())) + return p +} + +func (p *randomPartitioner) Partition(message *ProducerMessage, numPartitions int32) (int32, error) { + return int32(p.generator.Intn(int(numPartitions))), nil +} + +func (p *randomPartitioner) RequiresConsistency() bool { + return false +} + +type roundRobinPartitioner struct { + partition int32 +} + +// NewRoundRobinPartitioner returns a Partitioner which walks through the available partitions one at a time. +func NewRoundRobinPartitioner(topic string) Partitioner { + return &roundRobinPartitioner{} +} + +func (p *roundRobinPartitioner) Partition(message *ProducerMessage, numPartitions int32) (int32, error) { + if p.partition >= numPartitions { + p.partition = 0 + } + ret := p.partition + p.partition++ + return ret, nil +} + +func (p *roundRobinPartitioner) RequiresConsistency() bool { + return false +} + +type hashPartitioner struct { + random Partitioner + hasher hash.Hash32 + referenceAbs bool +} + +// NewCustomHashPartitioner is a wrapper around NewHashPartitioner, allowing the use of custom hasher. +// The argument is a function providing the instance, implementing the hash.Hash32 interface. This is to ensure that +// each partition dispatcher gets its own hasher, to avoid concurrency issues by sharing an instance. +func NewCustomHashPartitioner(hasher func() hash.Hash32) PartitionerConstructor { + return func(topic string) Partitioner { + p := new(hashPartitioner) + p.random = NewRandomPartitioner(topic) + p.hasher = hasher() + p.referenceAbs = false + return p + } +} + +// NewCustomPartitioner creates a default Partitioner but lets you specify the behavior of each component via options +func NewCustomPartitioner(options ...HashPartitionerOption) PartitionerConstructor { + return func(topic string) Partitioner { + p := new(hashPartitioner) + p.random = NewRandomPartitioner(topic) + p.hasher = fnv.New32a() + p.referenceAbs = false + for _, option := range options { + option(p) + } + return p + } +} + +// NewHashPartitioner returns a Partitioner which behaves as follows. If the message's key is nil then a +// random partition is chosen. Otherwise the FNV-1a hash of the encoded bytes of the message key is used, +// modulus the number of partitions. This ensures that messages with the same key always end up on the +// same partition. +func NewHashPartitioner(topic string) Partitioner { + p := new(hashPartitioner) + p.random = NewRandomPartitioner(topic) + p.hasher = fnv.New32a() + p.referenceAbs = false + return p +} + +// NewReferenceHashPartitioner is like NewHashPartitioner except that it handles absolute values +// in the same way as the reference Java implementation. NewHashPartitioner was supposed to do +// that but it had a mistake and now there are people depending on both behaviors. This will +// all go away on the next major version bump. +func NewReferenceHashPartitioner(topic string) Partitioner { + p := new(hashPartitioner) + p.random = NewRandomPartitioner(topic) + p.hasher = fnv.New32a() + p.referenceAbs = true + return p +} + +func (p *hashPartitioner) Partition(message *ProducerMessage, numPartitions int32) (int32, error) { + if message.Key == nil { + return p.random.Partition(message, numPartitions) + } + bytes, err := message.Key.Encode() + if err != nil { + return -1, err + } + p.hasher.Reset() + _, err = p.hasher.Write(bytes) + if err != nil { + return -1, err + } + var partition int32 + // Turns out we were doing our absolute value in a subtly different way from the upstream + // implementation, but now we need to maintain backwards compat for people who started using + // the old version; if referenceAbs is set we are compatible with the reference java client + // but not past Sarama versions + if p.referenceAbs { + partition = (int32(p.hasher.Sum32()) & 0x7fffffff) % numPartitions + } else { + partition = int32(p.hasher.Sum32()) % numPartitions + if partition < 0 { + partition = -partition + } + } + return partition, nil +} + +func (p *hashPartitioner) RequiresConsistency() bool { + return true +} + +func (p *hashPartitioner) MessageRequiresConsistency(message *ProducerMessage) bool { + return message.Key != nil +} diff --git a/vendor/github.com/Shopify/sarama/prep_encoder.go b/vendor/github.com/Shopify/sarama/prep_encoder.go new file mode 100644 index 000000000000..1602fcb3f6b8 --- /dev/null +++ b/vendor/github.com/Shopify/sarama/prep_encoder.go @@ -0,0 +1,211 @@ +package sarama + +import ( + "encoding/binary" + "errors" + "fmt" + "math" + + "github.com/rcrowley/go-metrics" +) + +type prepEncoder struct { + stack []pushEncoder + length int +} + +// primitives + +func (pe *prepEncoder) putInt8(in int8) { + pe.length++ +} + +func (pe *prepEncoder) putInt16(in int16) { + pe.length += 2 +} + +func (pe *prepEncoder) putInt32(in int32) { + pe.length += 4 +} + +func (pe *prepEncoder) putInt64(in int64) { + pe.length += 8 +} + +func (pe *prepEncoder) putVarint(in int64) { + var buf [binary.MaxVarintLen64]byte + pe.length += binary.PutVarint(buf[:], in) +} + +func (pe *prepEncoder) putUVarint(in uint64) { + var buf [binary.MaxVarintLen64]byte + pe.length += binary.PutUvarint(buf[:], in) +} + +func (pe *prepEncoder) putFloat64(in float64) { + pe.length += 8 +} + +func (pe *prepEncoder) putArrayLength(in int) error { + if in > math.MaxInt32 { + return PacketEncodingError{fmt.Sprintf("array too long (%d)", in)} + } + pe.length += 4 + return nil +} + +func (pe *prepEncoder) putCompactArrayLength(in int) { + pe.putUVarint(uint64(in + 1)) +} + +func (pe *prepEncoder) putBool(in bool) { + pe.length++ +} + +// arrays + +func (pe *prepEncoder) putBytes(in []byte) error { + pe.length += 4 + if in == nil { + return nil + } + return pe.putRawBytes(in) +} + +func (pe *prepEncoder) putVarintBytes(in []byte) error { + if in == nil { + pe.putVarint(-1) + return nil + } + pe.putVarint(int64(len(in))) + return pe.putRawBytes(in) +} + +func (pe *prepEncoder) putCompactBytes(in []byte) error { + pe.putUVarint(uint64(len(in) + 1)) + return pe.putRawBytes(in) +} + +func (pe *prepEncoder) putCompactString(in string) error { + pe.putCompactArrayLength(len(in)) + return pe.putRawBytes([]byte(in)) +} + +func (pe *prepEncoder) putNullableCompactString(in *string) error { + if in == nil { + pe.putUVarint(0) + return nil + } else { + return pe.putCompactString(*in) + } +} + +func (pe *prepEncoder) putRawBytes(in []byte) error { + if len(in) > math.MaxInt32 { + return PacketEncodingError{fmt.Sprintf("byteslice too long (%d)", len(in))} + } + pe.length += len(in) + return nil +} + +func (pe *prepEncoder) putNullableString(in *string) error { + if in == nil { + pe.length += 2 + return nil + } + return pe.putString(*in) +} + +func (pe *prepEncoder) putString(in string) error { + pe.length += 2 + if len(in) > math.MaxInt16 { + return PacketEncodingError{fmt.Sprintf("string too long (%d)", len(in))} + } + pe.length += len(in) + return nil +} + +func (pe *prepEncoder) putStringArray(in []string) error { + err := pe.putArrayLength(len(in)) + if err != nil { + return err + } + + for _, str := range in { + if err := pe.putString(str); err != nil { + return err + } + } + + return nil +} + +func (pe *prepEncoder) putCompactInt32Array(in []int32) error { + if in == nil { + return errors.New("expected int32 array to be non null") + } + + pe.putUVarint(uint64(len(in)) + 1) + pe.length += 4 * len(in) + return nil +} + +func (pe *prepEncoder) putNullableCompactInt32Array(in []int32) error { + if in == nil { + pe.putUVarint(0) + return nil + } + + pe.putUVarint(uint64(len(in)) + 1) + pe.length += 4 * len(in) + return nil +} + +func (pe *prepEncoder) putInt32Array(in []int32) error { + err := pe.putArrayLength(len(in)) + if err != nil { + return err + } + pe.length += 4 * len(in) + return nil +} + +func (pe *prepEncoder) putInt64Array(in []int64) error { + err := pe.putArrayLength(len(in)) + if err != nil { + return err + } + pe.length += 8 * len(in) + return nil +} + +func (pe *prepEncoder) putEmptyTaggedFieldArray() { + pe.putUVarint(0) +} + +func (pe *prepEncoder) offset() int { + return pe.length +} + +// stackable + +func (pe *prepEncoder) push(in pushEncoder) { + in.saveOffset(pe.length) + pe.length += in.reserveLength() + pe.stack = append(pe.stack, in) +} + +func (pe *prepEncoder) pop() error { + in := pe.stack[len(pe.stack)-1] + pe.stack = pe.stack[:len(pe.stack)-1] + if dpe, ok := in.(dynamicPushEncoder); ok { + pe.length += dpe.adjustLength(pe.length) + } + + return nil +} + +// we do not record metrics during the prep encoder pass +func (pe *prepEncoder) metricRegistry() metrics.Registry { + return nil +} diff --git a/vendor/github.com/Shopify/sarama/produce_request.go b/vendor/github.com/Shopify/sarama/produce_request.go new file mode 100644 index 000000000000..0034651e2542 --- /dev/null +++ b/vendor/github.com/Shopify/sarama/produce_request.go @@ -0,0 +1,258 @@ +package sarama + +import "github.com/rcrowley/go-metrics" + +// RequiredAcks is used in Produce Requests to tell the broker how many replica acknowledgements +// it must see before responding. Any of the constants defined here are valid. On broker versions +// prior to 0.8.2.0 any other positive int16 is also valid (the broker will wait for that many +// acknowledgements) but in 0.8.2.0 and later this will raise an exception (it has been replaced +// by setting the `min.isr` value in the brokers configuration). +type RequiredAcks int16 + +const ( + // NoResponse doesn't send any response, the TCP ACK is all you get. + NoResponse RequiredAcks = 0 + // WaitForLocal waits for only the local commit to succeed before responding. + WaitForLocal RequiredAcks = 1 + // WaitForAll waits for all in-sync replicas to commit before responding. + // The minimum number of in-sync replicas is configured on the broker via + // the `min.insync.replicas` configuration key. + WaitForAll RequiredAcks = -1 +) + +type ProduceRequest struct { + TransactionalID *string + RequiredAcks RequiredAcks + Timeout int32 + Version int16 // v1 requires Kafka 0.9, v2 requires Kafka 0.10, v3 requires Kafka 0.11 + records map[string]map[int32]Records +} + +func updateMsgSetMetrics(msgSet *MessageSet, compressionRatioMetric metrics.Histogram, + topicCompressionRatioMetric metrics.Histogram) int64 { + var topicRecordCount int64 + for _, messageBlock := range msgSet.Messages { + // Is this a fake "message" wrapping real messages? + if messageBlock.Msg.Set != nil { + topicRecordCount += int64(len(messageBlock.Msg.Set.Messages)) + } else { + // A single uncompressed message + topicRecordCount++ + } + // Better be safe than sorry when computing the compression ratio + if messageBlock.Msg.compressedSize != 0 { + compressionRatio := float64(len(messageBlock.Msg.Value)) / + float64(messageBlock.Msg.compressedSize) + // Histogram do not support decimal values, let's multiple it by 100 for better precision + intCompressionRatio := int64(100 * compressionRatio) + compressionRatioMetric.Update(intCompressionRatio) + topicCompressionRatioMetric.Update(intCompressionRatio) + } + } + return topicRecordCount +} + +func updateBatchMetrics(recordBatch *RecordBatch, compressionRatioMetric metrics.Histogram, + topicCompressionRatioMetric metrics.Histogram) int64 { + if recordBatch.compressedRecords != nil { + compressionRatio := int64(float64(recordBatch.recordsLen) / float64(len(recordBatch.compressedRecords)) * 100) + compressionRatioMetric.Update(compressionRatio) + topicCompressionRatioMetric.Update(compressionRatio) + } + + return int64(len(recordBatch.Records)) +} + +func (r *ProduceRequest) encode(pe packetEncoder) error { + if r.Version >= 3 { + if err := pe.putNullableString(r.TransactionalID); err != nil { + return err + } + } + pe.putInt16(int16(r.RequiredAcks)) + pe.putInt32(r.Timeout) + metricRegistry := pe.metricRegistry() + var batchSizeMetric metrics.Histogram + var compressionRatioMetric metrics.Histogram + if metricRegistry != nil { + batchSizeMetric = getOrRegisterHistogram("batch-size", metricRegistry) + compressionRatioMetric = getOrRegisterHistogram("compression-ratio", metricRegistry) + } + totalRecordCount := int64(0) + + err := pe.putArrayLength(len(r.records)) + if err != nil { + return err + } + + for topic, partitions := range r.records { + err = pe.putString(topic) + if err != nil { + return err + } + err = pe.putArrayLength(len(partitions)) + if err != nil { + return err + } + topicRecordCount := int64(0) + var topicCompressionRatioMetric metrics.Histogram + if metricRegistry != nil { + topicCompressionRatioMetric = getOrRegisterTopicHistogram("compression-ratio", topic, metricRegistry) + } + for id, records := range partitions { + startOffset := pe.offset() + pe.putInt32(id) + pe.push(&lengthField{}) + err = records.encode(pe) + if err != nil { + return err + } + err = pe.pop() + if err != nil { + return err + } + if metricRegistry != nil { + if r.Version >= 3 { + topicRecordCount += updateBatchMetrics(records.RecordBatch, compressionRatioMetric, topicCompressionRatioMetric) + } else { + topicRecordCount += updateMsgSetMetrics(records.MsgSet, compressionRatioMetric, topicCompressionRatioMetric) + } + batchSize := int64(pe.offset() - startOffset) + batchSizeMetric.Update(batchSize) + getOrRegisterTopicHistogram("batch-size", topic, metricRegistry).Update(batchSize) + } + } + if topicRecordCount > 0 { + getOrRegisterTopicMeter("record-send-rate", topic, metricRegistry).Mark(topicRecordCount) + getOrRegisterTopicHistogram("records-per-request", topic, metricRegistry).Update(topicRecordCount) + totalRecordCount += topicRecordCount + } + } + if totalRecordCount > 0 { + metrics.GetOrRegisterMeter("record-send-rate", metricRegistry).Mark(totalRecordCount) + getOrRegisterHistogram("records-per-request", metricRegistry).Update(totalRecordCount) + } + + return nil +} + +func (r *ProduceRequest) decode(pd packetDecoder, version int16) error { + r.Version = version + + if version >= 3 { + id, err := pd.getNullableString() + if err != nil { + return err + } + r.TransactionalID = id + } + requiredAcks, err := pd.getInt16() + if err != nil { + return err + } + r.RequiredAcks = RequiredAcks(requiredAcks) + if r.Timeout, err = pd.getInt32(); err != nil { + return err + } + topicCount, err := pd.getArrayLength() + if err != nil { + return err + } + if topicCount == 0 { + return nil + } + + r.records = make(map[string]map[int32]Records) + for i := 0; i < topicCount; i++ { + topic, err := pd.getString() + if err != nil { + return err + } + partitionCount, err := pd.getArrayLength() + if err != nil { + return err + } + r.records[topic] = make(map[int32]Records) + + for j := 0; j < partitionCount; j++ { + partition, err := pd.getInt32() + if err != nil { + return err + } + size, err := pd.getInt32() + if err != nil { + return err + } + recordsDecoder, err := pd.getSubset(int(size)) + if err != nil { + return err + } + var records Records + if err := records.decode(recordsDecoder); err != nil { + return err + } + r.records[topic][partition] = records + } + } + + return nil +} + +func (r *ProduceRequest) key() int16 { + return 0 +} + +func (r *ProduceRequest) version() int16 { + return r.Version +} + +func (r *ProduceRequest) headerVersion() int16 { + return 1 +} + +func (r *ProduceRequest) requiredVersion() KafkaVersion { + switch r.Version { + case 1: + return V0_9_0_0 + case 2: + return V0_10_0_0 + case 3: + return V0_11_0_0 + case 7: + return V2_1_0_0 + default: + return MinVersion + } +} + +func (r *ProduceRequest) ensureRecords(topic string, partition int32) { + if r.records == nil { + r.records = make(map[string]map[int32]Records) + } + + if r.records[topic] == nil { + r.records[topic] = make(map[int32]Records) + } +} + +func (r *ProduceRequest) AddMessage(topic string, partition int32, msg *Message) { + r.ensureRecords(topic, partition) + set := r.records[topic][partition].MsgSet + + if set == nil { + set = new(MessageSet) + r.records[topic][partition] = newLegacyRecords(set) + } + + set.addMessage(msg) +} + +func (r *ProduceRequest) AddSet(topic string, partition int32, set *MessageSet) { + r.ensureRecords(topic, partition) + r.records[topic][partition] = newLegacyRecords(set) +} + +func (r *ProduceRequest) AddBatch(topic string, partition int32, batch *RecordBatch) { + r.ensureRecords(topic, partition) + r.records[topic][partition] = newDefaultRecords(batch) +} diff --git a/vendor/github.com/Shopify/sarama/produce_response.go b/vendor/github.com/Shopify/sarama/produce_response.go new file mode 100644 index 000000000000..edf978790c9f --- /dev/null +++ b/vendor/github.com/Shopify/sarama/produce_response.go @@ -0,0 +1,212 @@ +package sarama + +import ( + "fmt" + "time" +) + +// Protocol, http://kafka.apache.org/protocol.html +// v1 +// v2 = v3 = v4 +// v5 = v6 = v7 +// Produce Response (Version: 7) => [responses] throttle_time_ms +// responses => topic [partition_responses] +// topic => STRING +// partition_responses => partition error_code base_offset log_append_time log_start_offset +// partition => INT32 +// error_code => INT16 +// base_offset => INT64 +// log_append_time => INT64 +// log_start_offset => INT64 +// throttle_time_ms => INT32 + +// partition_responses in protocol +type ProduceResponseBlock struct { + Err KError // v0, error_code + Offset int64 // v0, base_offset + Timestamp time.Time // v2, log_append_time, and the broker is configured with `LogAppendTime` + StartOffset int64 // v5, log_start_offset +} + +func (b *ProduceResponseBlock) decode(pd packetDecoder, version int16) (err error) { + tmp, err := pd.getInt16() + if err != nil { + return err + } + b.Err = KError(tmp) + + b.Offset, err = pd.getInt64() + if err != nil { + return err + } + + if version >= 2 { + if millis, err := pd.getInt64(); err != nil { + return err + } else if millis != -1 { + b.Timestamp = time.Unix(millis/1000, (millis%1000)*int64(time.Millisecond)) + } + } + + if version >= 5 { + b.StartOffset, err = pd.getInt64() + if err != nil { + return err + } + } + + return nil +} + +func (b *ProduceResponseBlock) encode(pe packetEncoder, version int16) (err error) { + pe.putInt16(int16(b.Err)) + pe.putInt64(b.Offset) + + if version >= 2 { + timestamp := int64(-1) + if !b.Timestamp.Before(time.Unix(0, 0)) { + timestamp = b.Timestamp.UnixNano() / int64(time.Millisecond) + } else if !b.Timestamp.IsZero() { + return PacketEncodingError{fmt.Sprintf("invalid timestamp (%v)", b.Timestamp)} + } + pe.putInt64(timestamp) + } + + if version >= 5 { + pe.putInt64(b.StartOffset) + } + + return nil +} + +type ProduceResponse struct { + Blocks map[string]map[int32]*ProduceResponseBlock // v0, responses + Version int16 + ThrottleTime time.Duration // v1, throttle_time_ms +} + +func (r *ProduceResponse) decode(pd packetDecoder, version int16) (err error) { + r.Version = version + + numTopics, err := pd.getArrayLength() + if err != nil { + return err + } + + r.Blocks = make(map[string]map[int32]*ProduceResponseBlock, numTopics) + for i := 0; i < numTopics; i++ { + name, err := pd.getString() + if err != nil { + return err + } + + numBlocks, err := pd.getArrayLength() + if err != nil { + return err + } + + r.Blocks[name] = make(map[int32]*ProduceResponseBlock, numBlocks) + + for j := 0; j < numBlocks; j++ { + id, err := pd.getInt32() + if err != nil { + return err + } + + block := new(ProduceResponseBlock) + err = block.decode(pd, version) + if err != nil { + return err + } + r.Blocks[name][id] = block + } + } + + if r.Version >= 1 { + millis, err := pd.getInt32() + if err != nil { + return err + } + + r.ThrottleTime = time.Duration(millis) * time.Millisecond + } + + return nil +} + +func (r *ProduceResponse) encode(pe packetEncoder) error { + err := pe.putArrayLength(len(r.Blocks)) + if err != nil { + return err + } + for topic, partitions := range r.Blocks { + err = pe.putString(topic) + if err != nil { + return err + } + err = pe.putArrayLength(len(partitions)) + if err != nil { + return err + } + for id, prb := range partitions { + pe.putInt32(id) + err = prb.encode(pe, r.Version) + if err != nil { + return err + } + } + } + + if r.Version >= 1 { + pe.putInt32(int32(r.ThrottleTime / time.Millisecond)) + } + return nil +} + +func (r *ProduceResponse) key() int16 { + return 0 +} + +func (r *ProduceResponse) version() int16 { + return r.Version +} + +func (r *ProduceResponse) headerVersion() int16 { + return 0 +} + +func (r *ProduceResponse) requiredVersion() KafkaVersion { + return MinVersion +} + +func (r *ProduceResponse) GetBlock(topic string, partition int32) *ProduceResponseBlock { + if r.Blocks == nil { + return nil + } + + if r.Blocks[topic] == nil { + return nil + } + + return r.Blocks[topic][partition] +} + +// Testing API + +func (r *ProduceResponse) AddTopicPartition(topic string, partition int32, err KError) { + if r.Blocks == nil { + r.Blocks = make(map[string]map[int32]*ProduceResponseBlock) + } + byTopic, ok := r.Blocks[topic] + if !ok { + byTopic = make(map[int32]*ProduceResponseBlock) + r.Blocks[topic] = byTopic + } + block := &ProduceResponseBlock{ + Err: err, + } + if r.Version >= 2 { + block.Timestamp = time.Now() + } + byTopic[partition] = block +} diff --git a/vendor/github.com/Shopify/sarama/produce_set.go b/vendor/github.com/Shopify/sarama/produce_set.go new file mode 100644 index 000000000000..9c70f8180069 --- /dev/null +++ b/vendor/github.com/Shopify/sarama/produce_set.go @@ -0,0 +1,273 @@ +package sarama + +import ( + "encoding/binary" + "errors" + "time" +) + +type partitionSet struct { + msgs []*ProducerMessage + recordsToSend Records + bufferBytes int +} + +type produceSet struct { + parent *asyncProducer + msgs map[string]map[int32]*partitionSet + producerID int64 + producerEpoch int16 + + bufferBytes int + bufferCount int +} + +func newProduceSet(parent *asyncProducer) *produceSet { + pid, epoch := parent.txnmgr.getProducerID() + return &produceSet{ + msgs: make(map[string]map[int32]*partitionSet), + parent: parent, + producerID: pid, + producerEpoch: epoch, + } +} + +func (ps *produceSet) add(msg *ProducerMessage) error { + var err error + var key, val []byte + + if msg.Key != nil { + if key, err = msg.Key.Encode(); err != nil { + return err + } + } + + if msg.Value != nil { + if val, err = msg.Value.Encode(); err != nil { + return err + } + } + + timestamp := msg.Timestamp + if timestamp.IsZero() { + timestamp = time.Now() + } + timestamp = timestamp.Truncate(time.Millisecond) + + partitions := ps.msgs[msg.Topic] + if partitions == nil { + partitions = make(map[int32]*partitionSet) + ps.msgs[msg.Topic] = partitions + } + + var size int + + set := partitions[msg.Partition] + if set == nil { + if ps.parent.conf.Version.IsAtLeast(V0_11_0_0) { + batch := &RecordBatch{ + FirstTimestamp: timestamp, + Version: 2, + Codec: ps.parent.conf.Producer.Compression, + CompressionLevel: ps.parent.conf.Producer.CompressionLevel, + ProducerID: ps.producerID, + ProducerEpoch: ps.producerEpoch, + } + if ps.parent.conf.Producer.Idempotent { + batch.FirstSequence = msg.sequenceNumber + } + set = &partitionSet{recordsToSend: newDefaultRecords(batch)} + size = recordBatchOverhead + } else { + set = &partitionSet{recordsToSend: newLegacyRecords(new(MessageSet))} + } + partitions[msg.Partition] = set + } + + if ps.parent.conf.Version.IsAtLeast(V0_11_0_0) { + if ps.parent.conf.Producer.Idempotent && msg.sequenceNumber < set.recordsToSend.RecordBatch.FirstSequence { + return errors.New("assertion failed: message out of sequence added to a batch") + } + } + + // Past this point we can't return an error, because we've already added the message to the set. + set.msgs = append(set.msgs, msg) + + if ps.parent.conf.Version.IsAtLeast(V0_11_0_0) { + // We are being conservative here to avoid having to prep encode the record + size += maximumRecordOverhead + rec := &Record{ + Key: key, + Value: val, + TimestampDelta: timestamp.Sub(set.recordsToSend.RecordBatch.FirstTimestamp), + } + size += len(key) + len(val) + if len(msg.Headers) > 0 { + rec.Headers = make([]*RecordHeader, len(msg.Headers)) + for i := range msg.Headers { + rec.Headers[i] = &msg.Headers[i] + size += len(rec.Headers[i].Key) + len(rec.Headers[i].Value) + 2*binary.MaxVarintLen32 + } + } + set.recordsToSend.RecordBatch.addRecord(rec) + } else { + msgToSend := &Message{Codec: CompressionNone, Key: key, Value: val} + if ps.parent.conf.Version.IsAtLeast(V0_10_0_0) { + msgToSend.Timestamp = timestamp + msgToSend.Version = 1 + } + set.recordsToSend.MsgSet.addMessage(msgToSend) + size = producerMessageOverhead + len(key) + len(val) + } + + set.bufferBytes += size + ps.bufferBytes += size + ps.bufferCount++ + + return nil +} + +func (ps *produceSet) buildRequest() *ProduceRequest { + req := &ProduceRequest{ + RequiredAcks: ps.parent.conf.Producer.RequiredAcks, + Timeout: int32(ps.parent.conf.Producer.Timeout / time.Millisecond), + } + if ps.parent.conf.Version.IsAtLeast(V0_10_0_0) { + req.Version = 2 + } + if ps.parent.conf.Version.IsAtLeast(V0_11_0_0) { + req.Version = 3 + } + + if ps.parent.conf.Producer.Compression == CompressionZSTD && ps.parent.conf.Version.IsAtLeast(V2_1_0_0) { + req.Version = 7 + } + + for topic, partitionSets := range ps.msgs { + for partition, set := range partitionSets { + if req.Version >= 3 { + // If the API version we're hitting is 3 or greater, we need to calculate + // offsets for each record in the batch relative to FirstOffset. + // Additionally, we must set LastOffsetDelta to the value of the last offset + // in the batch. Since the OffsetDelta of the first record is 0, we know that the + // final record of any batch will have an offset of (# of records in batch) - 1. + // (See https://cwiki.apache.org/confluence/display/KAFKA/A+Guide+To+The+Kafka+Protocol#AGuideToTheKafkaProtocol-Messagesets + // under the RecordBatch section for details.) + rb := set.recordsToSend.RecordBatch + if len(rb.Records) > 0 { + rb.LastOffsetDelta = int32(len(rb.Records) - 1) + for i, record := range rb.Records { + record.OffsetDelta = int64(i) + } + } + req.AddBatch(topic, partition, rb) + continue + } + if ps.parent.conf.Producer.Compression == CompressionNone { + req.AddSet(topic, partition, set.recordsToSend.MsgSet) + } else { + // When compression is enabled, the entire set for each partition is compressed + // and sent as the payload of a single fake "message" with the appropriate codec + // set and no key. When the server sees a message with a compression codec, it + // decompresses the payload and treats the result as its message set. + + if ps.parent.conf.Version.IsAtLeast(V0_10_0_0) { + // If our version is 0.10 or later, assign relative offsets + // to the inner messages. This lets the broker avoid + // recompressing the message set. + // (See https://cwiki.apache.org/confluence/display/KAFKA/KIP-31+-+Move+to+relative+offsets+in+compressed+message+sets + // for details on relative offsets.) + for i, msg := range set.recordsToSend.MsgSet.Messages { + msg.Offset = int64(i) + } + } + payload, err := encode(set.recordsToSend.MsgSet, ps.parent.conf.MetricRegistry) + if err != nil { + Logger.Println(err) // if this happens, it's basically our fault. + panic(err) + } + compMsg := &Message{ + Codec: ps.parent.conf.Producer.Compression, + CompressionLevel: ps.parent.conf.Producer.CompressionLevel, + Key: nil, + Value: payload, + Set: set.recordsToSend.MsgSet, // Provide the underlying message set for accurate metrics + } + if ps.parent.conf.Version.IsAtLeast(V0_10_0_0) { + compMsg.Version = 1 + compMsg.Timestamp = set.recordsToSend.MsgSet.Messages[0].Msg.Timestamp + } + req.AddMessage(topic, partition, compMsg) + } + } + } + + return req +} + +func (ps *produceSet) eachPartition(cb func(topic string, partition int32, pSet *partitionSet)) { + for topic, partitionSet := range ps.msgs { + for partition, set := range partitionSet { + cb(topic, partition, set) + } + } +} + +func (ps *produceSet) dropPartition(topic string, partition int32) []*ProducerMessage { + if ps.msgs[topic] == nil { + return nil + } + set := ps.msgs[topic][partition] + if set == nil { + return nil + } + ps.bufferBytes -= set.bufferBytes + ps.bufferCount -= len(set.msgs) + delete(ps.msgs[topic], partition) + return set.msgs +} + +func (ps *produceSet) wouldOverflow(msg *ProducerMessage) bool { + version := 1 + if ps.parent.conf.Version.IsAtLeast(V0_11_0_0) { + version = 2 + } + + switch { + // Would we overflow our maximum possible size-on-the-wire? 10KiB is arbitrary overhead for safety. + case ps.bufferBytes+msg.byteSize(version) >= int(MaxRequestSize-(10*1024)): + return true + // Would we overflow the size-limit of a message-batch for this partition? + case ps.msgs[msg.Topic] != nil && ps.msgs[msg.Topic][msg.Partition] != nil && + ps.msgs[msg.Topic][msg.Partition].bufferBytes+msg.byteSize(version) >= ps.parent.conf.Producer.MaxMessageBytes: + return true + // Would we overflow simply in number of messages? + case ps.parent.conf.Producer.Flush.MaxMessages > 0 && ps.bufferCount >= ps.parent.conf.Producer.Flush.MaxMessages: + return true + default: + return false + } +} + +func (ps *produceSet) readyToFlush() bool { + switch { + // If we don't have any messages, nothing else matters + case ps.empty(): + return false + // If all three config values are 0, we always flush as-fast-as-possible + case ps.parent.conf.Producer.Flush.Frequency == 0 && ps.parent.conf.Producer.Flush.Bytes == 0 && ps.parent.conf.Producer.Flush.Messages == 0: + return true + // If we've passed the message trigger-point + case ps.parent.conf.Producer.Flush.Messages > 0 && ps.bufferCount >= ps.parent.conf.Producer.Flush.Messages: + return true + // If we've passed the byte trigger-point + case ps.parent.conf.Producer.Flush.Bytes > 0 && ps.bufferBytes >= ps.parent.conf.Producer.Flush.Bytes: + return true + default: + return false + } +} + +func (ps *produceSet) empty() bool { + return ps.bufferCount == 0 +} diff --git a/vendor/github.com/Shopify/sarama/quota_types.go b/vendor/github.com/Shopify/sarama/quota_types.go new file mode 100644 index 000000000000..4f33af0bca45 --- /dev/null +++ b/vendor/github.com/Shopify/sarama/quota_types.go @@ -0,0 +1,21 @@ +package sarama + +type ( + QuotaEntityType string + + QuotaMatchType int +) + +// ref: https://github.com/apache/kafka/blob/trunk/clients/src/main/java/org/apache/kafka/common/quota/ClientQuotaEntity.java +const ( + QuotaEntityUser QuotaEntityType = "user" + QuotaEntityClientID QuotaEntityType = "client-id" + QuotaEntityIP QuotaEntityType = "ip" +) + +// ref: https://github.com/apache/kafka/blob/trunk/clients/src/main/java/org/apache/kafka/common/requests/DescribeClientQuotasRequest.java +const ( + QuotaMatchExact QuotaMatchType = iota + QuotaMatchDefault + QuotaMatchAny +) diff --git a/vendor/github.com/Shopify/sarama/real_decoder.go b/vendor/github.com/Shopify/sarama/real_decoder.go new file mode 100644 index 000000000000..bab6cb486f27 --- /dev/null +++ b/vendor/github.com/Shopify/sarama/real_decoder.go @@ -0,0 +1,449 @@ +package sarama + +import ( + "encoding/binary" + "math" +) + +var ( + errInvalidArrayLength = PacketDecodingError{"invalid array length"} + errInvalidByteSliceLength = PacketDecodingError{"invalid byteslice length"} + errInvalidStringLength = PacketDecodingError{"invalid string length"} + errVarintOverflow = PacketDecodingError{"varint overflow"} + errUVarintOverflow = PacketDecodingError{"uvarint overflow"} + errInvalidBool = PacketDecodingError{"invalid bool"} + errUnsupportedTaggedFields = PacketDecodingError{"non-empty tagged fields are not supported yet"} +) + +type realDecoder struct { + raw []byte + off int + stack []pushDecoder +} + +// primitives + +func (rd *realDecoder) getInt8() (int8, error) { + if rd.remaining() < 1 { + rd.off = len(rd.raw) + return -1, ErrInsufficientData + } + tmp := int8(rd.raw[rd.off]) + rd.off++ + return tmp, nil +} + +func (rd *realDecoder) getInt16() (int16, error) { + if rd.remaining() < 2 { + rd.off = len(rd.raw) + return -1, ErrInsufficientData + } + tmp := int16(binary.BigEndian.Uint16(rd.raw[rd.off:])) + rd.off += 2 + return tmp, nil +} + +func (rd *realDecoder) getInt32() (int32, error) { + if rd.remaining() < 4 { + rd.off = len(rd.raw) + return -1, ErrInsufficientData + } + tmp := int32(binary.BigEndian.Uint32(rd.raw[rd.off:])) + rd.off += 4 + return tmp, nil +} + +func (rd *realDecoder) getInt64() (int64, error) { + if rd.remaining() < 8 { + rd.off = len(rd.raw) + return -1, ErrInsufficientData + } + tmp := int64(binary.BigEndian.Uint64(rd.raw[rd.off:])) + rd.off += 8 + return tmp, nil +} + +func (rd *realDecoder) getVarint() (int64, error) { + tmp, n := binary.Varint(rd.raw[rd.off:]) + if n == 0 { + rd.off = len(rd.raw) + return -1, ErrInsufficientData + } + if n < 0 { + rd.off -= n + return -1, errVarintOverflow + } + rd.off += n + return tmp, nil +} + +func (rd *realDecoder) getUVarint() (uint64, error) { + tmp, n := binary.Uvarint(rd.raw[rd.off:]) + if n == 0 { + rd.off = len(rd.raw) + return 0, ErrInsufficientData + } + + if n < 0 { + rd.off -= n + return 0, errUVarintOverflow + } + + rd.off += n + return tmp, nil +} + +func (rd *realDecoder) getFloat64() (float64, error) { + if rd.remaining() < 8 { + rd.off = len(rd.raw) + return -1, ErrInsufficientData + } + tmp := math.Float64frombits(binary.BigEndian.Uint64(rd.raw[rd.off:])) + rd.off += 8 + return tmp, nil +} + +func (rd *realDecoder) getArrayLength() (int, error) { + if rd.remaining() < 4 { + rd.off = len(rd.raw) + return -1, ErrInsufficientData + } + tmp := int(int32(binary.BigEndian.Uint32(rd.raw[rd.off:]))) + rd.off += 4 + if tmp > rd.remaining() { + rd.off = len(rd.raw) + return -1, ErrInsufficientData + } else if tmp > 2*math.MaxUint16 { + return -1, errInvalidArrayLength + } + return tmp, nil +} + +func (rd *realDecoder) getCompactArrayLength() (int, error) { + n, err := rd.getUVarint() + if err != nil { + return 0, err + } + + if n == 0 { + return 0, nil + } + + return int(n) - 1, nil +} + +func (rd *realDecoder) getBool() (bool, error) { + b, err := rd.getInt8() + if err != nil || b == 0 { + return false, err + } + if b != 1 { + return false, errInvalidBool + } + return true, nil +} + +func (rd *realDecoder) getEmptyTaggedFieldArray() (int, error) { + tagCount, err := rd.getUVarint() + if err != nil { + return 0, err + } + + if tagCount != 0 { + return 0, errUnsupportedTaggedFields + } + + return 0, nil +} + +// collections + +func (rd *realDecoder) getBytes() ([]byte, error) { + tmp, err := rd.getInt32() + if err != nil { + return nil, err + } + if tmp == -1 { + return nil, nil + } + + return rd.getRawBytes(int(tmp)) +} + +func (rd *realDecoder) getVarintBytes() ([]byte, error) { + tmp, err := rd.getVarint() + if err != nil { + return nil, err + } + if tmp == -1 { + return nil, nil + } + + return rd.getRawBytes(int(tmp)) +} + +func (rd *realDecoder) getCompactBytes() ([]byte, error) { + n, err := rd.getUVarint() + if err != nil { + return nil, err + } + + length := int(n - 1) + return rd.getRawBytes(length) +} + +func (rd *realDecoder) getStringLength() (int, error) { + length, err := rd.getInt16() + if err != nil { + return 0, err + } + + n := int(length) + + switch { + case n < -1: + return 0, errInvalidStringLength + case n > rd.remaining(): + rd.off = len(rd.raw) + return 0, ErrInsufficientData + } + + return n, nil +} + +func (rd *realDecoder) getString() (string, error) { + n, err := rd.getStringLength() + if err != nil || n == -1 { + return "", err + } + + tmpStr := string(rd.raw[rd.off : rd.off+n]) + rd.off += n + return tmpStr, nil +} + +func (rd *realDecoder) getNullableString() (*string, error) { + n, err := rd.getStringLength() + if err != nil || n == -1 { + return nil, err + } + + tmpStr := string(rd.raw[rd.off : rd.off+n]) + rd.off += n + return &tmpStr, err +} + +func (rd *realDecoder) getCompactString() (string, error) { + n, err := rd.getUVarint() + if err != nil { + return "", err + } + + length := int(n - 1) + if length < 0 { + return "", errInvalidByteSliceLength + } + tmpStr := string(rd.raw[rd.off : rd.off+length]) + rd.off += length + return tmpStr, nil +} + +func (rd *realDecoder) getCompactNullableString() (*string, error) { + n, err := rd.getUVarint() + if err != nil { + return nil, err + } + + length := int(n - 1) + + if length < 0 { + return nil, err + } + + tmpStr := string(rd.raw[rd.off : rd.off+length]) + rd.off += length + return &tmpStr, err +} + +func (rd *realDecoder) getCompactInt32Array() ([]int32, error) { + n, err := rd.getUVarint() + if err != nil { + return nil, err + } + + if n == 0 { + return nil, nil + } + + arrayLength := int(n) - 1 + + ret := make([]int32, arrayLength) + + for i := range ret { + ret[i] = int32(binary.BigEndian.Uint32(rd.raw[rd.off:])) + rd.off += 4 + } + return ret, nil +} + +func (rd *realDecoder) getInt32Array() ([]int32, error) { + if rd.remaining() < 4 { + rd.off = len(rd.raw) + return nil, ErrInsufficientData + } + n := int(binary.BigEndian.Uint32(rd.raw[rd.off:])) + rd.off += 4 + + if rd.remaining() < 4*n { + rd.off = len(rd.raw) + return nil, ErrInsufficientData + } + + if n == 0 { + return nil, nil + } + + if n < 0 { + return nil, errInvalidArrayLength + } + + ret := make([]int32, n) + for i := range ret { + ret[i] = int32(binary.BigEndian.Uint32(rd.raw[rd.off:])) + rd.off += 4 + } + return ret, nil +} + +func (rd *realDecoder) getInt64Array() ([]int64, error) { + if rd.remaining() < 4 { + rd.off = len(rd.raw) + return nil, ErrInsufficientData + } + n := int(binary.BigEndian.Uint32(rd.raw[rd.off:])) + rd.off += 4 + + if rd.remaining() < 8*n { + rd.off = len(rd.raw) + return nil, ErrInsufficientData + } + + if n == 0 { + return nil, nil + } + + if n < 0 { + return nil, errInvalidArrayLength + } + + ret := make([]int64, n) + for i := range ret { + ret[i] = int64(binary.BigEndian.Uint64(rd.raw[rd.off:])) + rd.off += 8 + } + return ret, nil +} + +func (rd *realDecoder) getStringArray() ([]string, error) { + if rd.remaining() < 4 { + rd.off = len(rd.raw) + return nil, ErrInsufficientData + } + n := int(binary.BigEndian.Uint32(rd.raw[rd.off:])) + rd.off += 4 + + if n == 0 { + return nil, nil + } + + if n < 0 { + return nil, errInvalidArrayLength + } + + ret := make([]string, n) + for i := range ret { + str, err := rd.getString() + if err != nil { + return nil, err + } + + ret[i] = str + } + return ret, nil +} + +// subsets + +func (rd *realDecoder) remaining() int { + return len(rd.raw) - rd.off +} + +func (rd *realDecoder) getSubset(length int) (packetDecoder, error) { + buf, err := rd.getRawBytes(length) + if err != nil { + return nil, err + } + return &realDecoder{raw: buf}, nil +} + +func (rd *realDecoder) getRawBytes(length int) ([]byte, error) { + if length < 0 { + return nil, errInvalidByteSliceLength + } else if length > rd.remaining() { + rd.off = len(rd.raw) + return nil, ErrInsufficientData + } + + start := rd.off + rd.off += length + return rd.raw[start:rd.off], nil +} + +func (rd *realDecoder) peek(offset, length int) (packetDecoder, error) { + if rd.remaining() < offset+length { + return nil, ErrInsufficientData + } + off := rd.off + offset + return &realDecoder{raw: rd.raw[off : off+length]}, nil +} + +func (rd *realDecoder) peekInt8(offset int) (int8, error) { + const byteLen = 1 + if rd.remaining() < offset+byteLen { + return -1, ErrInsufficientData + } + return int8(rd.raw[rd.off+offset]), nil +} + +// stacks + +func (rd *realDecoder) push(in pushDecoder) error { + in.saveOffset(rd.off) + + var reserve int + if dpd, ok := in.(dynamicPushDecoder); ok { + if err := dpd.decode(rd); err != nil { + return err + } + } else { + reserve = in.reserveLength() + if rd.remaining() < reserve { + rd.off = len(rd.raw) + return ErrInsufficientData + } + } + + rd.stack = append(rd.stack, in) + + rd.off += reserve + + return nil +} + +func (rd *realDecoder) pop() error { + // this is go's ugly pop pattern (the inverse of append) + in := rd.stack[len(rd.stack)-1] + rd.stack = rd.stack[:len(rd.stack)-1] + + return in.check(rd.off, rd.raw) +} diff --git a/vendor/github.com/Shopify/sarama/real_encoder.go b/vendor/github.com/Shopify/sarama/real_encoder.go new file mode 100644 index 000000000000..d6a0ddf12476 --- /dev/null +++ b/vendor/github.com/Shopify/sarama/real_encoder.go @@ -0,0 +1,219 @@ +package sarama + +import ( + "encoding/binary" + "errors" + "math" + + "github.com/rcrowley/go-metrics" +) + +type realEncoder struct { + raw []byte + off int + stack []pushEncoder + registry metrics.Registry +} + +// primitives + +func (re *realEncoder) putInt8(in int8) { + re.raw[re.off] = byte(in) + re.off++ +} + +func (re *realEncoder) putInt16(in int16) { + binary.BigEndian.PutUint16(re.raw[re.off:], uint16(in)) + re.off += 2 +} + +func (re *realEncoder) putInt32(in int32) { + binary.BigEndian.PutUint32(re.raw[re.off:], uint32(in)) + re.off += 4 +} + +func (re *realEncoder) putInt64(in int64) { + binary.BigEndian.PutUint64(re.raw[re.off:], uint64(in)) + re.off += 8 +} + +func (re *realEncoder) putVarint(in int64) { + re.off += binary.PutVarint(re.raw[re.off:], in) +} + +func (re *realEncoder) putUVarint(in uint64) { + re.off += binary.PutUvarint(re.raw[re.off:], in) +} + +func (re *realEncoder) putFloat64(in float64) { + binary.BigEndian.PutUint64(re.raw[re.off:], math.Float64bits(in)) + re.off += 8 +} + +func (re *realEncoder) putArrayLength(in int) error { + re.putInt32(int32(in)) + return nil +} + +func (re *realEncoder) putCompactArrayLength(in int) { + // 0 represents a null array, so +1 has to be added + re.putUVarint(uint64(in + 1)) +} + +func (re *realEncoder) putBool(in bool) { + if in { + re.putInt8(1) + return + } + re.putInt8(0) +} + +// collection + +func (re *realEncoder) putRawBytes(in []byte) error { + copy(re.raw[re.off:], in) + re.off += len(in) + return nil +} + +func (re *realEncoder) putBytes(in []byte) error { + if in == nil { + re.putInt32(-1) + return nil + } + re.putInt32(int32(len(in))) + return re.putRawBytes(in) +} + +func (re *realEncoder) putVarintBytes(in []byte) error { + if in == nil { + re.putVarint(-1) + return nil + } + re.putVarint(int64(len(in))) + return re.putRawBytes(in) +} + +func (re *realEncoder) putCompactBytes(in []byte) error { + re.putUVarint(uint64(len(in) + 1)) + return re.putRawBytes(in) +} + +func (re *realEncoder) putCompactString(in string) error { + re.putCompactArrayLength(len(in)) + return re.putRawBytes([]byte(in)) +} + +func (re *realEncoder) putNullableCompactString(in *string) error { + if in == nil { + re.putInt8(0) + return nil + } + return re.putCompactString(*in) +} + +func (re *realEncoder) putString(in string) error { + re.putInt16(int16(len(in))) + copy(re.raw[re.off:], in) + re.off += len(in) + return nil +} + +func (re *realEncoder) putNullableString(in *string) error { + if in == nil { + re.putInt16(-1) + return nil + } + return re.putString(*in) +} + +func (re *realEncoder) putStringArray(in []string) error { + err := re.putArrayLength(len(in)) + if err != nil { + return err + } + + for _, val := range in { + if err := re.putString(val); err != nil { + return err + } + } + + return nil +} + +func (re *realEncoder) putCompactInt32Array(in []int32) error { + if in == nil { + return errors.New("expected int32 array to be non null") + } + // 0 represents a null array, so +1 has to be added + re.putUVarint(uint64(len(in)) + 1) + for _, val := range in { + re.putInt32(val) + } + return nil +} + +func (re *realEncoder) putNullableCompactInt32Array(in []int32) error { + if in == nil { + re.putUVarint(0) + return nil + } + // 0 represents a null array, so +1 has to be added + re.putUVarint(uint64(len(in)) + 1) + for _, val := range in { + re.putInt32(val) + } + return nil +} + +func (re *realEncoder) putInt32Array(in []int32) error { + err := re.putArrayLength(len(in)) + if err != nil { + return err + } + for _, val := range in { + re.putInt32(val) + } + return nil +} + +func (re *realEncoder) putInt64Array(in []int64) error { + err := re.putArrayLength(len(in)) + if err != nil { + return err + } + for _, val := range in { + re.putInt64(val) + } + return nil +} + +func (re *realEncoder) putEmptyTaggedFieldArray() { + re.putUVarint(0) +} + +func (re *realEncoder) offset() int { + return re.off +} + +// stacks + +func (re *realEncoder) push(in pushEncoder) { + in.saveOffset(re.off) + re.off += in.reserveLength() + re.stack = append(re.stack, in) +} + +func (re *realEncoder) pop() error { + // this is go's ugly pop pattern (the inverse of append) + in := re.stack[len(re.stack)-1] + re.stack = re.stack[:len(re.stack)-1] + + return in.run(re.off, re.raw) +} + +// we do record metrics during the real encoder pass +func (re *realEncoder) metricRegistry() metrics.Registry { + return re.registry +} diff --git a/vendor/github.com/Shopify/sarama/record.go b/vendor/github.com/Shopify/sarama/record.go new file mode 100644 index 000000000000..a3fe8c0614eb --- /dev/null +++ b/vendor/github.com/Shopify/sarama/record.go @@ -0,0 +1,116 @@ +package sarama + +import ( + "encoding/binary" + "time" +) + +const ( + isTransactionalMask = 0x10 + controlMask = 0x20 + maximumRecordOverhead = 5*binary.MaxVarintLen32 + binary.MaxVarintLen64 + 1 +) + +// RecordHeader stores key and value for a record header +type RecordHeader struct { + Key []byte + Value []byte +} + +func (h *RecordHeader) encode(pe packetEncoder) error { + if err := pe.putVarintBytes(h.Key); err != nil { + return err + } + return pe.putVarintBytes(h.Value) +} + +func (h *RecordHeader) decode(pd packetDecoder) (err error) { + if h.Key, err = pd.getVarintBytes(); err != nil { + return err + } + + if h.Value, err = pd.getVarintBytes(); err != nil { + return err + } + return nil +} + +// Record is kafka record type +type Record struct { + Headers []*RecordHeader + + Attributes int8 + TimestampDelta time.Duration + OffsetDelta int64 + Key []byte + Value []byte + length varintLengthField +} + +func (r *Record) encode(pe packetEncoder) error { + pe.push(&r.length) + pe.putInt8(r.Attributes) + pe.putVarint(int64(r.TimestampDelta / time.Millisecond)) + pe.putVarint(r.OffsetDelta) + if err := pe.putVarintBytes(r.Key); err != nil { + return err + } + if err := pe.putVarintBytes(r.Value); err != nil { + return err + } + pe.putVarint(int64(len(r.Headers))) + + for _, h := range r.Headers { + if err := h.encode(pe); err != nil { + return err + } + } + + return pe.pop() +} + +func (r *Record) decode(pd packetDecoder) (err error) { + if err = pd.push(&r.length); err != nil { + return err + } + + if r.Attributes, err = pd.getInt8(); err != nil { + return err + } + + timestamp, err := pd.getVarint() + if err != nil { + return err + } + r.TimestampDelta = time.Duration(timestamp) * time.Millisecond + + if r.OffsetDelta, err = pd.getVarint(); err != nil { + return err + } + + if r.Key, err = pd.getVarintBytes(); err != nil { + return err + } + + if r.Value, err = pd.getVarintBytes(); err != nil { + return err + } + + numHeaders, err := pd.getVarint() + if err != nil { + return err + } + + if numHeaders >= 0 { + r.Headers = make([]*RecordHeader, numHeaders) + } + for i := int64(0); i < numHeaders; i++ { + hdr := new(RecordHeader) + if err := hdr.decode(pd); err != nil { + return err + } + r.Headers[i] = hdr + } + + return pd.pop() +} diff --git a/vendor/github.com/Shopify/sarama/record_batch.go b/vendor/github.com/Shopify/sarama/record_batch.go new file mode 100644 index 000000000000..c653763eca83 --- /dev/null +++ b/vendor/github.com/Shopify/sarama/record_batch.go @@ -0,0 +1,225 @@ +package sarama + +import ( + "fmt" + "time" +) + +const recordBatchOverhead = 49 + +type recordsArray []*Record + +func (e recordsArray) encode(pe packetEncoder) error { + for _, r := range e { + if err := r.encode(pe); err != nil { + return err + } + } + return nil +} + +func (e recordsArray) decode(pd packetDecoder) error { + for i := range e { + rec := &Record{} + if err := rec.decode(pd); err != nil { + return err + } + e[i] = rec + } + return nil +} + +type RecordBatch struct { + FirstOffset int64 + PartitionLeaderEpoch int32 + Version int8 + Codec CompressionCodec + CompressionLevel int + Control bool + LogAppendTime bool + LastOffsetDelta int32 + FirstTimestamp time.Time + MaxTimestamp time.Time + ProducerID int64 + ProducerEpoch int16 + FirstSequence int32 + Records []*Record + PartialTrailingRecord bool + IsTransactional bool + + compressedRecords []byte + recordsLen int // uncompressed records size +} + +func (b *RecordBatch) LastOffset() int64 { + return b.FirstOffset + int64(b.LastOffsetDelta) +} + +func (b *RecordBatch) encode(pe packetEncoder) error { + if b.Version != 2 { + return PacketEncodingError{fmt.Sprintf("unsupported compression codec (%d)", b.Codec)} + } + pe.putInt64(b.FirstOffset) + pe.push(&lengthField{}) + pe.putInt32(b.PartitionLeaderEpoch) + pe.putInt8(b.Version) + pe.push(newCRC32Field(crcCastagnoli)) + pe.putInt16(b.computeAttributes()) + pe.putInt32(b.LastOffsetDelta) + + if err := (Timestamp{&b.FirstTimestamp}).encode(pe); err != nil { + return err + } + + if err := (Timestamp{&b.MaxTimestamp}).encode(pe); err != nil { + return err + } + + pe.putInt64(b.ProducerID) + pe.putInt16(b.ProducerEpoch) + pe.putInt32(b.FirstSequence) + + if err := pe.putArrayLength(len(b.Records)); err != nil { + return err + } + + if b.compressedRecords == nil { + if err := b.encodeRecords(pe); err != nil { + return err + } + } + if err := pe.putRawBytes(b.compressedRecords); err != nil { + return err + } + + if err := pe.pop(); err != nil { + return err + } + return pe.pop() +} + +func (b *RecordBatch) decode(pd packetDecoder) (err error) { + if b.FirstOffset, err = pd.getInt64(); err != nil { + return err + } + + batchLen, err := pd.getInt32() + if err != nil { + return err + } + + if b.PartitionLeaderEpoch, err = pd.getInt32(); err != nil { + return err + } + + if b.Version, err = pd.getInt8(); err != nil { + return err + } + + crc32Decoder := acquireCrc32Field(crcCastagnoli) + defer releaseCrc32Field(crc32Decoder) + + if err = pd.push(crc32Decoder); err != nil { + return err + } + + attributes, err := pd.getInt16() + if err != nil { + return err + } + b.Codec = CompressionCodec(int8(attributes) & compressionCodecMask) + b.Control = attributes&controlMask == controlMask + b.LogAppendTime = attributes×tampTypeMask == timestampTypeMask + b.IsTransactional = attributes&isTransactionalMask == isTransactionalMask + + if b.LastOffsetDelta, err = pd.getInt32(); err != nil { + return err + } + + if err = (Timestamp{&b.FirstTimestamp}).decode(pd); err != nil { + return err + } + + if err = (Timestamp{&b.MaxTimestamp}).decode(pd); err != nil { + return err + } + + if b.ProducerID, err = pd.getInt64(); err != nil { + return err + } + + if b.ProducerEpoch, err = pd.getInt16(); err != nil { + return err + } + + if b.FirstSequence, err = pd.getInt32(); err != nil { + return err + } + + numRecs, err := pd.getArrayLength() + if err != nil { + return err + } + if numRecs >= 0 { + b.Records = make([]*Record, numRecs) + } + + bufSize := int(batchLen) - recordBatchOverhead + recBuffer, err := pd.getRawBytes(bufSize) + if err != nil { + if err == ErrInsufficientData { + b.PartialTrailingRecord = true + b.Records = nil + return nil + } + return err + } + + if err = pd.pop(); err != nil { + return err + } + + recBuffer, err = decompress(b.Codec, recBuffer) + if err != nil { + return err + } + + b.recordsLen = len(recBuffer) + err = decode(recBuffer, recordsArray(b.Records)) + if err == ErrInsufficientData { + b.PartialTrailingRecord = true + b.Records = nil + return nil + } + return err +} + +func (b *RecordBatch) encodeRecords(pe packetEncoder) error { + var raw []byte + var err error + if raw, err = encode(recordsArray(b.Records), pe.metricRegistry()); err != nil { + return err + } + b.recordsLen = len(raw) + + b.compressedRecords, err = compress(b.Codec, b.CompressionLevel, raw) + return err +} + +func (b *RecordBatch) computeAttributes() int16 { + attr := int16(b.Codec) & int16(compressionCodecMask) + if b.Control { + attr |= controlMask + } + if b.LogAppendTime { + attr |= timestampTypeMask + } + if b.IsTransactional { + attr |= isTransactionalMask + } + return attr +} + +func (b *RecordBatch) addRecord(r *Record) { + b.Records = append(b.Records, r) +} diff --git a/vendor/github.com/Shopify/sarama/records.go b/vendor/github.com/Shopify/sarama/records.go new file mode 100644 index 000000000000..f4c5e95f1de3 --- /dev/null +++ b/vendor/github.com/Shopify/sarama/records.go @@ -0,0 +1,203 @@ +package sarama + +import "fmt" + +const ( + unknownRecords = iota + legacyRecords + defaultRecords + + magicOffset = 16 +) + +// Records implements a union type containing either a RecordBatch or a legacy MessageSet. +type Records struct { + recordsType int + MsgSet *MessageSet + RecordBatch *RecordBatch +} + +func newLegacyRecords(msgSet *MessageSet) Records { + return Records{recordsType: legacyRecords, MsgSet: msgSet} +} + +func newDefaultRecords(batch *RecordBatch) Records { + return Records{recordsType: defaultRecords, RecordBatch: batch} +} + +// setTypeFromFields sets type of Records depending on which of MsgSet or RecordBatch is not nil. +// The first return value indicates whether both fields are nil (and the type is not set). +// If both fields are not nil, it returns an error. +func (r *Records) setTypeFromFields() (bool, error) { + if r.MsgSet == nil && r.RecordBatch == nil { + return true, nil + } + if r.MsgSet != nil && r.RecordBatch != nil { + return false, fmt.Errorf("both MsgSet and RecordBatch are set, but record type is unknown") + } + r.recordsType = defaultRecords + if r.MsgSet != nil { + r.recordsType = legacyRecords + } + return false, nil +} + +func (r *Records) encode(pe packetEncoder) error { + if r.recordsType == unknownRecords { + if empty, err := r.setTypeFromFields(); err != nil || empty { + return err + } + } + + switch r.recordsType { + case legacyRecords: + if r.MsgSet == nil { + return nil + } + return r.MsgSet.encode(pe) + case defaultRecords: + if r.RecordBatch == nil { + return nil + } + return r.RecordBatch.encode(pe) + } + + return fmt.Errorf("unknown records type: %v", r.recordsType) +} + +func (r *Records) setTypeFromMagic(pd packetDecoder) error { + magic, err := magicValue(pd) + if err != nil { + return err + } + + r.recordsType = defaultRecords + if magic < 2 { + r.recordsType = legacyRecords + } + + return nil +} + +func (r *Records) decode(pd packetDecoder) error { + if r.recordsType == unknownRecords { + if err := r.setTypeFromMagic(pd); err != nil { + return err + } + } + + switch r.recordsType { + case legacyRecords: + r.MsgSet = &MessageSet{} + return r.MsgSet.decode(pd) + case defaultRecords: + r.RecordBatch = &RecordBatch{} + return r.RecordBatch.decode(pd) + } + return fmt.Errorf("unknown records type: %v", r.recordsType) +} + +func (r *Records) numRecords() (int, error) { + if r.recordsType == unknownRecords { + if empty, err := r.setTypeFromFields(); err != nil || empty { + return 0, err + } + } + + switch r.recordsType { + case legacyRecords: + if r.MsgSet == nil { + return 0, nil + } + return len(r.MsgSet.Messages), nil + case defaultRecords: + if r.RecordBatch == nil { + return 0, nil + } + return len(r.RecordBatch.Records), nil + } + return 0, fmt.Errorf("unknown records type: %v", r.recordsType) +} + +func (r *Records) isPartial() (bool, error) { + if r.recordsType == unknownRecords { + if empty, err := r.setTypeFromFields(); err != nil || empty { + return false, err + } + } + + switch r.recordsType { + case unknownRecords: + return false, nil + case legacyRecords: + if r.MsgSet == nil { + return false, nil + } + return r.MsgSet.PartialTrailingMessage, nil + case defaultRecords: + if r.RecordBatch == nil { + return false, nil + } + return r.RecordBatch.PartialTrailingRecord, nil + } + return false, fmt.Errorf("unknown records type: %v", r.recordsType) +} + +func (r *Records) isControl() (bool, error) { + if r.recordsType == unknownRecords { + if empty, err := r.setTypeFromFields(); err != nil || empty { + return false, err + } + } + + switch r.recordsType { + case legacyRecords: + return false, nil + case defaultRecords: + if r.RecordBatch == nil { + return false, nil + } + return r.RecordBatch.Control, nil + } + return false, fmt.Errorf("unknown records type: %v", r.recordsType) +} + +func (r *Records) isOverflow() (bool, error) { + if r.recordsType == unknownRecords { + if empty, err := r.setTypeFromFields(); err != nil || empty { + return false, err + } + } + + switch r.recordsType { + case unknownRecords: + return false, nil + case legacyRecords: + if r.MsgSet == nil { + return false, nil + } + return r.MsgSet.OverflowMessage, nil + case defaultRecords: + return false, nil + } + return false, fmt.Errorf("unknown records type: %v", r.recordsType) +} + +func magicValue(pd packetDecoder) (int8, error) { + return pd.peekInt8(magicOffset) +} + +func (r *Records) getControlRecord() (ControlRecord, error) { + if r.RecordBatch == nil || len(r.RecordBatch.Records) <= 0 { + return ControlRecord{}, fmt.Errorf("cannot get control record, record batch is empty") + } + + firstRecord := r.RecordBatch.Records[0] + controlRecord := ControlRecord{} + err := controlRecord.decode(&realDecoder{raw: firstRecord.Key}, &realDecoder{raw: firstRecord.Value}) + if err != nil { + return ControlRecord{}, err + } + + return controlRecord, nil +} diff --git a/vendor/github.com/Shopify/sarama/request.go b/vendor/github.com/Shopify/sarama/request.go new file mode 100644 index 000000000000..ce90eb8c46e2 --- /dev/null +++ b/vendor/github.com/Shopify/sarama/request.go @@ -0,0 +1,203 @@ +package sarama + +import ( + "encoding/binary" + "fmt" + "io" +) + +type protocolBody interface { + encoder + versionedDecoder + key() int16 + version() int16 + headerVersion() int16 + requiredVersion() KafkaVersion +} + +type request struct { + correlationID int32 + clientID string + body protocolBody +} + +func (r *request) encode(pe packetEncoder) error { + pe.push(&lengthField{}) + pe.putInt16(r.body.key()) + pe.putInt16(r.body.version()) + pe.putInt32(r.correlationID) + + if r.body.headerVersion() >= 1 { + err := pe.putString(r.clientID) + if err != nil { + return err + } + } + + if r.body.headerVersion() >= 2 { + // we don't use tag headers at the moment so we just put an array length of 0 + pe.putUVarint(0) + } + + err := r.body.encode(pe) + if err != nil { + return err + } + + return pe.pop() +} + +func (r *request) decode(pd packetDecoder) (err error) { + key, err := pd.getInt16() + if err != nil { + return err + } + + version, err := pd.getInt16() + if err != nil { + return err + } + + r.correlationID, err = pd.getInt32() + if err != nil { + return err + } + + r.clientID, err = pd.getString() + if err != nil { + return err + } + + r.body = allocateBody(key, version) + if r.body == nil { + return PacketDecodingError{fmt.Sprintf("unknown request key (%d)", key)} + } + + if r.body.headerVersion() >= 2 { + // tagged field + _, err = pd.getUVarint() + if err != nil { + return err + } + } + + return r.body.decode(pd, version) +} + +func decodeRequest(r io.Reader) (*request, int, error) { + var ( + bytesRead int + lengthBytes = make([]byte, 4) + ) + + if _, err := io.ReadFull(r, lengthBytes); err != nil { + return nil, bytesRead, err + } + + bytesRead += len(lengthBytes) + length := int32(binary.BigEndian.Uint32(lengthBytes)) + + if length <= 4 || length > MaxRequestSize { + return nil, bytesRead, PacketDecodingError{fmt.Sprintf("message of length %d too large or too small", length)} + } + + encodedReq := make([]byte, length) + if _, err := io.ReadFull(r, encodedReq); err != nil { + return nil, bytesRead, err + } + + bytesRead += len(encodedReq) + + req := &request{} + if err := decode(encodedReq, req); err != nil { + return nil, bytesRead, err + } + + return req, bytesRead, nil +} + +func allocateBody(key, version int16) protocolBody { + switch key { + case 0: + return &ProduceRequest{} + case 1: + return &FetchRequest{Version: version} + case 2: + return &OffsetRequest{Version: version} + case 3: + return &MetadataRequest{} + case 8: + return &OffsetCommitRequest{Version: version} + case 9: + return &OffsetFetchRequest{Version: version} + case 10: + return &FindCoordinatorRequest{} + case 11: + return &JoinGroupRequest{} + case 12: + return &HeartbeatRequest{} + case 13: + return &LeaveGroupRequest{} + case 14: + return &SyncGroupRequest{} + case 15: + return &DescribeGroupsRequest{} + case 16: + return &ListGroupsRequest{} + case 17: + return &SaslHandshakeRequest{} + case 18: + return &ApiVersionsRequest{Version: version} + case 19: + return &CreateTopicsRequest{} + case 20: + return &DeleteTopicsRequest{} + case 21: + return &DeleteRecordsRequest{} + case 22: + return &InitProducerIDRequest{} + case 24: + return &AddPartitionsToTxnRequest{} + case 25: + return &AddOffsetsToTxnRequest{} + case 26: + return &EndTxnRequest{} + case 28: + return &TxnOffsetCommitRequest{} + case 29: + return &DescribeAclsRequest{} + case 30: + return &CreateAclsRequest{} + case 31: + return &DeleteAclsRequest{} + case 32: + return &DescribeConfigsRequest{} + case 33: + return &AlterConfigsRequest{} + case 35: + return &DescribeLogDirsRequest{} + case 36: + return &SaslAuthenticateRequest{} + case 37: + return &CreatePartitionsRequest{} + case 42: + return &DeleteGroupsRequest{} + case 44: + return &IncrementalAlterConfigsRequest{} + case 45: + return &AlterPartitionReassignmentsRequest{} + case 46: + return &ListPartitionReassignmentsRequest{} + case 47: + return &DeleteOffsetsRequest{} + case 48: + return &DescribeClientQuotasRequest{} + case 49: + return &AlterClientQuotasRequest{} + case 50: + return &DescribeUserScramCredentialsRequest{} + case 51: + return &AlterUserScramCredentialsRequest{} + } + return nil +} diff --git a/vendor/github.com/Shopify/sarama/response_header.go b/vendor/github.com/Shopify/sarama/response_header.go new file mode 100644 index 000000000000..fbcef0bfbea3 --- /dev/null +++ b/vendor/github.com/Shopify/sarama/response_header.go @@ -0,0 +1,33 @@ +package sarama + +import "fmt" + +const ( + responseLengthSize = 4 + correlationIDSize = 4 +) + +type responseHeader struct { + length int32 + correlationID int32 +} + +func (r *responseHeader) decode(pd packetDecoder, version int16) (err error) { + r.length, err = pd.getInt32() + if err != nil { + return err + } + if r.length <= 4 || r.length > MaxResponseSize { + return PacketDecodingError{fmt.Sprintf("message of length %d too large or too small", r.length)} + } + + r.correlationID, err = pd.getInt32() + + if version >= 1 { + if _, err := pd.getEmptyTaggedFieldArray(); err != nil { + return err + } + } + + return err +} diff --git a/vendor/github.com/Shopify/sarama/sarama.go b/vendor/github.com/Shopify/sarama/sarama.go new file mode 100644 index 000000000000..f6e55a395da5 --- /dev/null +++ b/vendor/github.com/Shopify/sarama/sarama.go @@ -0,0 +1,132 @@ +/* +Package sarama is a pure Go client library for dealing with Apache Kafka (versions 0.8 and later). It includes a high-level +API for easily producing and consuming messages, and a low-level API for controlling bytes on the wire when the high-level +API is insufficient. Usage examples for the high-level APIs are provided inline with their full documentation. + +To produce messages, use either the AsyncProducer or the SyncProducer. The AsyncProducer accepts messages on a channel +and produces them asynchronously in the background as efficiently as possible; it is preferred in most cases. +The SyncProducer provides a method which will block until Kafka acknowledges the message as produced. This can be +useful but comes with two caveats: it will generally be less efficient, and the actual durability guarantees +depend on the configured value of `Producer.RequiredAcks`. There are configurations where a message acknowledged by the +SyncProducer can still sometimes be lost. + +To consume messages, use Consumer or Consumer-Group API. + +For lower-level needs, the Broker and Request/Response objects permit precise control over each connection +and message sent on the wire; the Client provides higher-level metadata management that is shared between +the producers and the consumer. The Request/Response objects and properties are mostly undocumented, as they line up +exactly with the protocol fields documented by Kafka at +https://cwiki.apache.org/confluence/display/KAFKA/A+Guide+To+The+Kafka+Protocol + +Metrics are exposed through https://github.com/rcrowley/go-metrics library in a local registry. + +Broker related metrics: + + +----------------------------------------------+------------+---------------------------------------------------------------+ + | Name | Type | Description | + +----------------------------------------------+------------+---------------------------------------------------------------+ + | incoming-byte-rate | meter | Bytes/second read off all brokers | + | incoming-byte-rate-for-broker- | meter | Bytes/second read off a given broker | + | outgoing-byte-rate | meter | Bytes/second written off all brokers | + | outgoing-byte-rate-for-broker- | meter | Bytes/second written off a given broker | + | request-rate | meter | Requests/second sent to all brokers | + | request-rate-for-broker- | meter | Requests/second sent to a given broker | + | request-size | histogram | Distribution of the request size in bytes for all brokers | + | request-size-for-broker- | histogram | Distribution of the request size in bytes for a given broker | + | request-latency-in-ms | histogram | Distribution of the request latency in ms for all brokers | + | request-latency-in-ms-for-broker- | histogram | Distribution of the request latency in ms for a given broker | + | response-rate | meter | Responses/second received from all brokers | + | response-rate-for-broker- | meter | Responses/second received from a given broker | + | response-size | histogram | Distribution of the response size in bytes for all brokers | + | response-size-for-broker- | histogram | Distribution of the response size in bytes for a given broker | + | requests-in-flight | counter | The current number of in-flight requests awaiting a response | + | | | for all brokers | + | requests-in-flight-for-broker- | counter | The current number of in-flight requests awaiting a response | + | | | for a given broker | + +----------------------------------------------+------------+---------------------------------------------------------------+ + +Note that we do not gather specific metrics for seed brokers but they are part of the "all brokers" metrics. + +Producer related metrics: + + +-------------------------------------------+------------+--------------------------------------------------------------------------------------+ + | Name | Type | Description | + +-------------------------------------------+------------+--------------------------------------------------------------------------------------+ + | batch-size | histogram | Distribution of the number of bytes sent per partition per request for all topics | + | batch-size-for-topic- | histogram | Distribution of the number of bytes sent per partition per request for a given topic | + | record-send-rate | meter | Records/second sent to all topics | + | record-send-rate-for-topic- | meter | Records/second sent to a given topic | + | records-per-request | histogram | Distribution of the number of records sent per request for all topics | + | records-per-request-for-topic- | histogram | Distribution of the number of records sent per request for a given topic | + | compression-ratio | histogram | Distribution of the compression ratio times 100 of record batches for all topics | + | compression-ratio-for-topic- | histogram | Distribution of the compression ratio times 100 of record batches for a given topic | + +-------------------------------------------+------------+--------------------------------------------------------------------------------------+ + +Consumer related metrics: + + +-------------------------------------------+------------+--------------------------------------------------------------------------------------+ + | Name | Type | Description | + +-------------------------------------------+------------+--------------------------------------------------------------------------------------+ + | consumer-batch-size | histogram | Distribution of the number of messages in a batch | + | consumer-group-join-total- | counter | Total count of consumer group join attempts | + | consumer-group-join-failed- | counter | Total count of consumer group join failures | + | consumer-group-sync-total- | counter | Total count of consumer group sync attempts | + | consumer-group-sync-failed- | counter | Total count of consumer group sync failures | + +-------------------------------------------+------------+--------------------------------------------------------------------------------------+ + +*/ +package sarama + +import ( + "io" + "log" +) + +var ( + // Logger is the instance of a StdLogger interface that Sarama writes connection + // management events to. By default it is set to discard all log messages via ioutil.Discard, + // but you can set it to redirect wherever you want. + Logger StdLogger = log.New(io.Discard, "[Sarama] ", log.LstdFlags) + + // PanicHandler is called for recovering from panics spawned internally to the library (and thus + // not recoverable by the caller's goroutine). Defaults to nil, which means panics are not recovered. + PanicHandler func(interface{}) + + // MaxRequestSize is the maximum size (in bytes) of any request that Sarama will attempt to send. Trying + // to send a request larger than this will result in an PacketEncodingError. The default of 100 MiB is aligned + // with Kafka's default `socket.request.max.bytes`, which is the largest request the broker will attempt + // to process. + MaxRequestSize int32 = 100 * 1024 * 1024 + + // MaxResponseSize is the maximum size (in bytes) of any response that Sarama will attempt to parse. If + // a broker returns a response message larger than this value, Sarama will return a PacketDecodingError to + // protect the client from running out of memory. Please note that brokers do not have any natural limit on + // the size of responses they send. In particular, they can send arbitrarily large fetch responses to consumers + // (see https://issues.apache.org/jira/browse/KAFKA-2063). + MaxResponseSize int32 = 100 * 1024 * 1024 +) + +// StdLogger is used to log error messages. +type StdLogger interface { + Print(v ...interface{}) + Printf(format string, v ...interface{}) + Println(v ...interface{}) +} + +type debugLogger struct{} + +func (d *debugLogger) Print(v ...interface{}) { + Logger.Print(v) +} +func (d *debugLogger) Printf(format string, v ...interface{}) { + Logger.Printf(format, v) +} +func (d *debugLogger) Println(v ...interface{}) { + Logger.Println(v) +} + +// DebugLogger is the instance of a StdLogger that Sarama writes more verbose +// debug information to. By default it is set to redirect all debug to the +// default Logger above, but you can optionally set it to another StdLogger +// instance to (e.g.,) discard debug information +var DebugLogger StdLogger = &debugLogger{} diff --git a/vendor/github.com/Shopify/sarama/sasl_authenticate_request.go b/vendor/github.com/Shopify/sarama/sasl_authenticate_request.go new file mode 100644 index 000000000000..90504df6f522 --- /dev/null +++ b/vendor/github.com/Shopify/sarama/sasl_authenticate_request.go @@ -0,0 +1,33 @@ +package sarama + +type SaslAuthenticateRequest struct { + SaslAuthBytes []byte +} + +// APIKeySASLAuth is the API key for the SaslAuthenticate Kafka API +const APIKeySASLAuth = 36 + +func (r *SaslAuthenticateRequest) encode(pe packetEncoder) error { + return pe.putBytes(r.SaslAuthBytes) +} + +func (r *SaslAuthenticateRequest) decode(pd packetDecoder, version int16) (err error) { + r.SaslAuthBytes, err = pd.getBytes() + return err +} + +func (r *SaslAuthenticateRequest) key() int16 { + return APIKeySASLAuth +} + +func (r *SaslAuthenticateRequest) version() int16 { + return 0 +} + +func (r *SaslAuthenticateRequest) headerVersion() int16 { + return 1 +} + +func (r *SaslAuthenticateRequest) requiredVersion() KafkaVersion { + return V1_0_0_0 +} diff --git a/vendor/github.com/Shopify/sarama/sasl_authenticate_response.go b/vendor/github.com/Shopify/sarama/sasl_authenticate_response.go new file mode 100644 index 000000000000..3ef57b5afad0 --- /dev/null +++ b/vendor/github.com/Shopify/sarama/sasl_authenticate_response.go @@ -0,0 +1,48 @@ +package sarama + +type SaslAuthenticateResponse struct { + Err KError + ErrorMessage *string + SaslAuthBytes []byte +} + +func (r *SaslAuthenticateResponse) encode(pe packetEncoder) error { + pe.putInt16(int16(r.Err)) + if err := pe.putNullableString(r.ErrorMessage); err != nil { + return err + } + return pe.putBytes(r.SaslAuthBytes) +} + +func (r *SaslAuthenticateResponse) decode(pd packetDecoder, version int16) error { + kerr, err := pd.getInt16() + if err != nil { + return err + } + + r.Err = KError(kerr) + + if r.ErrorMessage, err = pd.getNullableString(); err != nil { + return err + } + + r.SaslAuthBytes, err = pd.getBytes() + + return err +} + +func (r *SaslAuthenticateResponse) key() int16 { + return APIKeySASLAuth +} + +func (r *SaslAuthenticateResponse) version() int16 { + return 0 +} + +func (r *SaslAuthenticateResponse) headerVersion() int16 { + return 0 +} + +func (r *SaslAuthenticateResponse) requiredVersion() KafkaVersion { + return V1_0_0_0 +} diff --git a/vendor/github.com/Shopify/sarama/sasl_handshake_request.go b/vendor/github.com/Shopify/sarama/sasl_handshake_request.go new file mode 100644 index 000000000000..74dc3072f488 --- /dev/null +++ b/vendor/github.com/Shopify/sarama/sasl_handshake_request.go @@ -0,0 +1,38 @@ +package sarama + +type SaslHandshakeRequest struct { + Mechanism string + Version int16 +} + +func (r *SaslHandshakeRequest) encode(pe packetEncoder) error { + if err := pe.putString(r.Mechanism); err != nil { + return err + } + + return nil +} + +func (r *SaslHandshakeRequest) decode(pd packetDecoder, version int16) (err error) { + if r.Mechanism, err = pd.getString(); err != nil { + return err + } + + return nil +} + +func (r *SaslHandshakeRequest) key() int16 { + return 17 +} + +func (r *SaslHandshakeRequest) version() int16 { + return r.Version +} + +func (r *SaslHandshakeRequest) headerVersion() int16 { + return 1 +} + +func (r *SaslHandshakeRequest) requiredVersion() KafkaVersion { + return V0_10_0_0 +} diff --git a/vendor/github.com/Shopify/sarama/sasl_handshake_response.go b/vendor/github.com/Shopify/sarama/sasl_handshake_response.go new file mode 100644 index 000000000000..69dfc3178ec2 --- /dev/null +++ b/vendor/github.com/Shopify/sarama/sasl_handshake_response.go @@ -0,0 +1,42 @@ +package sarama + +type SaslHandshakeResponse struct { + Err KError + EnabledMechanisms []string +} + +func (r *SaslHandshakeResponse) encode(pe packetEncoder) error { + pe.putInt16(int16(r.Err)) + return pe.putStringArray(r.EnabledMechanisms) +} + +func (r *SaslHandshakeResponse) decode(pd packetDecoder, version int16) error { + kerr, err := pd.getInt16() + if err != nil { + return err + } + + r.Err = KError(kerr) + + if r.EnabledMechanisms, err = pd.getStringArray(); err != nil { + return err + } + + return nil +} + +func (r *SaslHandshakeResponse) key() int16 { + return 17 +} + +func (r *SaslHandshakeResponse) version() int16 { + return 0 +} + +func (r *SaslHandshakeResponse) headerVersion() int16 { + return 0 +} + +func (r *SaslHandshakeResponse) requiredVersion() KafkaVersion { + return V0_10_0_0 +} diff --git a/vendor/github.com/Shopify/sarama/scram_formatter.go b/vendor/github.com/Shopify/sarama/scram_formatter.go new file mode 100644 index 000000000000..2af9e4a695f5 --- /dev/null +++ b/vendor/github.com/Shopify/sarama/scram_formatter.go @@ -0,0 +1,78 @@ +package sarama + +import ( + "crypto/hmac" + "crypto/sha256" + "crypto/sha512" + "hash" +) + +// ScramFormatter implementation +// @see: https://github.com/apache/kafka/blob/99b9b3e84f4e98c3f07714e1de6a139a004cbc5b/clients/src/main/java/org/apache/kafka/common/security/scram/internals/ScramFormatter.java#L93 +type scramFormatter struct { + mechanism ScramMechanismType +} + +func (s scramFormatter) mac(key []byte) (hash.Hash, error) { + var m hash.Hash + + switch s.mechanism { + case SCRAM_MECHANISM_SHA_256: + m = hmac.New(sha256.New, key) + + case SCRAM_MECHANISM_SHA_512: + m = hmac.New(sha512.New, key) + default: + return nil, ErrUnknownScramMechanism + } + + return m, nil +} + +func (s scramFormatter) hmac(key []byte, extra []byte) ([]byte, error) { + mac, err := s.mac(key) + if err != nil { + return nil, err + } + + if _, err := mac.Write(extra); err != nil { + return nil, err + } + return mac.Sum(nil), nil +} + +func (s scramFormatter) xor(result []byte, second []byte) { + for i := 0; i < len(result); i++ { + result[i] = result[i] ^ second[i] + } +} + +func (s scramFormatter) saltedPassword(password []byte, salt []byte, iterations int) ([]byte, error) { + mac, err := s.mac(password) + if err != nil { + return nil, err + } + + if _, err := mac.Write(salt); err != nil { + return nil, err + } + if _, err := mac.Write([]byte{0, 0, 0, 1}); err != nil { + return nil, err + } + + u1 := mac.Sum(nil) + prev := u1 + result := u1 + + for i := 2; i <= iterations; i++ { + ui, err := s.hmac(password, prev) + if err != nil { + return nil, err + } + + s.xor(result, ui) + prev = ui + } + + return result, nil +} diff --git a/vendor/github.com/Shopify/sarama/sticky_assignor_user_data.go b/vendor/github.com/Shopify/sarama/sticky_assignor_user_data.go new file mode 100644 index 000000000000..161233fc3578 --- /dev/null +++ b/vendor/github.com/Shopify/sarama/sticky_assignor_user_data.go @@ -0,0 +1,124 @@ +package sarama + +type topicPartitionAssignment struct { + Topic string + Partition int32 +} + +type StickyAssignorUserData interface { + partitions() []topicPartitionAssignment + hasGeneration() bool + generation() int +} + +// StickyAssignorUserDataV0 holds topic partition information for an assignment +type StickyAssignorUserDataV0 struct { + Topics map[string][]int32 + + topicPartitions []topicPartitionAssignment +} + +func (m *StickyAssignorUserDataV0) encode(pe packetEncoder) error { + if err := pe.putArrayLength(len(m.Topics)); err != nil { + return err + } + + for topic, partitions := range m.Topics { + if err := pe.putString(topic); err != nil { + return err + } + if err := pe.putInt32Array(partitions); err != nil { + return err + } + } + return nil +} + +func (m *StickyAssignorUserDataV0) decode(pd packetDecoder) (err error) { + var topicLen int + if topicLen, err = pd.getArrayLength(); err != nil { + return + } + + m.Topics = make(map[string][]int32, topicLen) + for i := 0; i < topicLen; i++ { + var topic string + if topic, err = pd.getString(); err != nil { + return + } + if m.Topics[topic], err = pd.getInt32Array(); err != nil { + return + } + } + m.topicPartitions = populateTopicPartitions(m.Topics) + return nil +} + +func (m *StickyAssignorUserDataV0) partitions() []topicPartitionAssignment { return m.topicPartitions } +func (m *StickyAssignorUserDataV0) hasGeneration() bool { return false } +func (m *StickyAssignorUserDataV0) generation() int { return defaultGeneration } + +// StickyAssignorUserDataV1 holds topic partition information for an assignment +type StickyAssignorUserDataV1 struct { + Topics map[string][]int32 + Generation int32 + + topicPartitions []topicPartitionAssignment +} + +func (m *StickyAssignorUserDataV1) encode(pe packetEncoder) error { + if err := pe.putArrayLength(len(m.Topics)); err != nil { + return err + } + + for topic, partitions := range m.Topics { + if err := pe.putString(topic); err != nil { + return err + } + if err := pe.putInt32Array(partitions); err != nil { + return err + } + } + + pe.putInt32(m.Generation) + return nil +} + +func (m *StickyAssignorUserDataV1) decode(pd packetDecoder) (err error) { + var topicLen int + if topicLen, err = pd.getArrayLength(); err != nil { + return + } + + m.Topics = make(map[string][]int32, topicLen) + for i := 0; i < topicLen; i++ { + var topic string + if topic, err = pd.getString(); err != nil { + return + } + if m.Topics[topic], err = pd.getInt32Array(); err != nil { + return + } + } + + m.Generation, err = pd.getInt32() + if err != nil { + return err + } + m.topicPartitions = populateTopicPartitions(m.Topics) + return nil +} + +func (m *StickyAssignorUserDataV1) partitions() []topicPartitionAssignment { return m.topicPartitions } +func (m *StickyAssignorUserDataV1) hasGeneration() bool { return true } +func (m *StickyAssignorUserDataV1) generation() int { return int(m.Generation) } + +func populateTopicPartitions(topics map[string][]int32) []topicPartitionAssignment { + topicPartitions := make([]topicPartitionAssignment, 0) + for topic, partitions := range topics { + for _, partition := range partitions { + topicPartitions = append(topicPartitions, topicPartitionAssignment{Topic: topic, Partition: partition}) + } + } + return topicPartitions +} diff --git a/vendor/github.com/Shopify/sarama/sync_group_request.go b/vendor/github.com/Shopify/sarama/sync_group_request.go new file mode 100644 index 000000000000..ac6ecb13e045 --- /dev/null +++ b/vendor/github.com/Shopify/sarama/sync_group_request.go @@ -0,0 +1,104 @@ +package sarama + +type SyncGroupRequest struct { + GroupId string + GenerationId int32 + MemberId string + GroupAssignments map[string][]byte +} + +func (r *SyncGroupRequest) encode(pe packetEncoder) error { + if err := pe.putString(r.GroupId); err != nil { + return err + } + + pe.putInt32(r.GenerationId) + + if err := pe.putString(r.MemberId); err != nil { + return err + } + + if err := pe.putArrayLength(len(r.GroupAssignments)); err != nil { + return err + } + for memberId, memberAssignment := range r.GroupAssignments { + if err := pe.putString(memberId); err != nil { + return err + } + if err := pe.putBytes(memberAssignment); err != nil { + return err + } + } + + return nil +} + +func (r *SyncGroupRequest) decode(pd packetDecoder, version int16) (err error) { + if r.GroupId, err = pd.getString(); err != nil { + return + } + if r.GenerationId, err = pd.getInt32(); err != nil { + return + } + if r.MemberId, err = pd.getString(); err != nil { + return + } + + n, err := pd.getArrayLength() + if err != nil { + return err + } + if n == 0 { + return nil + } + + r.GroupAssignments = make(map[string][]byte) + for i := 0; i < n; i++ { + memberId, err := pd.getString() + if err != nil { + return err + } + memberAssignment, err := pd.getBytes() + if err != nil { + return err + } + + r.GroupAssignments[memberId] = memberAssignment + } + + return nil +} + +func (r *SyncGroupRequest) key() int16 { + return 14 +} + +func (r *SyncGroupRequest) version() int16 { + return 0 +} + +func (r *SyncGroupRequest) headerVersion() int16 { + return 1 +} + +func (r *SyncGroupRequest) requiredVersion() KafkaVersion { + return V0_9_0_0 +} + +func (r *SyncGroupRequest) AddGroupAssignment(memberId string, memberAssignment []byte) { + if r.GroupAssignments == nil { + r.GroupAssignments = make(map[string][]byte) + } + + r.GroupAssignments[memberId] = memberAssignment +} + +func (r *SyncGroupRequest) AddGroupAssignmentMember(memberId string, memberAssignment *ConsumerGroupMemberAssignment) error { + bin, err := encode(memberAssignment, nil) + if err != nil { + return err + } + + r.AddGroupAssignment(memberId, bin) + return nil +} diff --git a/vendor/github.com/Shopify/sarama/sync_group_response.go b/vendor/github.com/Shopify/sarama/sync_group_response.go new file mode 100644 index 000000000000..af019c42f97b --- /dev/null +++ b/vendor/github.com/Shopify/sarama/sync_group_response.go @@ -0,0 +1,45 @@ +package sarama + +type SyncGroupResponse struct { + Err KError + MemberAssignment []byte +} + +func (r *SyncGroupResponse) GetMemberAssignment() (*ConsumerGroupMemberAssignment, error) { + assignment := new(ConsumerGroupMemberAssignment) + err := decode(r.MemberAssignment, assignment) + return assignment, err +} + +func (r *SyncGroupResponse) encode(pe packetEncoder) error { + pe.putInt16(int16(r.Err)) + return pe.putBytes(r.MemberAssignment) +} + +func (r *SyncGroupResponse) decode(pd packetDecoder, version int16) (err error) { + kerr, err := pd.getInt16() + if err != nil { + return err + } + + r.Err = KError(kerr) + + r.MemberAssignment, err = pd.getBytes() + return +} + +func (r *SyncGroupResponse) key() int16 { + return 14 +} + +func (r *SyncGroupResponse) version() int16 { + return 0 +} + +func (r *SyncGroupResponse) headerVersion() int16 { + return 0 +} + +func (r *SyncGroupResponse) requiredVersion() KafkaVersion { + return V0_9_0_0 +} diff --git a/vendor/github.com/Shopify/sarama/sync_producer.go b/vendor/github.com/Shopify/sarama/sync_producer.go new file mode 100644 index 000000000000..eedece6b4a97 --- /dev/null +++ b/vendor/github.com/Shopify/sarama/sync_producer.go @@ -0,0 +1,148 @@ +package sarama + +import "sync" + +// SyncProducer publishes Kafka messages, blocking until they have been acknowledged. It routes messages to the correct +// broker, refreshing metadata as appropriate, and parses responses for errors. You must call Close() on a producer +// to avoid leaks, it may not be garbage-collected automatically when it passes out of scope. +// +// The SyncProducer comes with two caveats: it will generally be less efficient than the AsyncProducer, and the actual +// durability guarantee provided when a message is acknowledged depend on the configured value of `Producer.RequiredAcks`. +// There are configurations where a message acknowledged by the SyncProducer can still sometimes be lost. +// +// For implementation reasons, the SyncProducer requires `Producer.Return.Errors` and `Producer.Return.Successes` to +// be set to true in its configuration. +type SyncProducer interface { + + // SendMessage produces a given message, and returns only when it either has + // succeeded or failed to produce. It will return the partition and the offset + // of the produced message, or an error if the message failed to produce. + SendMessage(msg *ProducerMessage) (partition int32, offset int64, err error) + + // SendMessages produces a given set of messages, and returns only when all + // messages in the set have either succeeded or failed. Note that messages + // can succeed and fail individually; if some succeed and some fail, + // SendMessages will return an error. + SendMessages(msgs []*ProducerMessage) error + + // Close shuts down the producer; you must call this function before a producer + // object passes out of scope, as it may otherwise leak memory. + // You must call this before calling Close on the underlying client. + Close() error +} + +type syncProducer struct { + producer *asyncProducer + wg sync.WaitGroup +} + +// NewSyncProducer creates a new SyncProducer using the given broker addresses and configuration. +func NewSyncProducer(addrs []string, config *Config) (SyncProducer, error) { + if config == nil { + config = NewConfig() + config.Producer.Return.Successes = true + } + + if err := verifyProducerConfig(config); err != nil { + return nil, err + } + + p, err := NewAsyncProducer(addrs, config) + if err != nil { + return nil, err + } + return newSyncProducerFromAsyncProducer(p.(*asyncProducer)), nil +} + +// NewSyncProducerFromClient creates a new SyncProducer using the given client. It is still +// necessary to call Close() on the underlying client when shutting down this producer. +func NewSyncProducerFromClient(client Client) (SyncProducer, error) { + if err := verifyProducerConfig(client.Config()); err != nil { + return nil, err + } + + p, err := NewAsyncProducerFromClient(client) + if err != nil { + return nil, err + } + return newSyncProducerFromAsyncProducer(p.(*asyncProducer)), nil +} + +func newSyncProducerFromAsyncProducer(p *asyncProducer) *syncProducer { + sp := &syncProducer{producer: p} + + sp.wg.Add(2) + go withRecover(sp.handleSuccesses) + go withRecover(sp.handleErrors) + + return sp +} + +func verifyProducerConfig(config *Config) error { + if !config.Producer.Return.Errors { + return ConfigurationError("Producer.Return.Errors must be true to be used in a SyncProducer") + } + if !config.Producer.Return.Successes { + return ConfigurationError("Producer.Return.Successes must be true to be used in a SyncProducer") + } + return nil +} + +func (sp *syncProducer) SendMessage(msg *ProducerMessage) (partition int32, offset int64, err error) { + expectation := make(chan *ProducerError, 1) + msg.expectation = expectation + sp.producer.Input() <- msg + + if pErr := <-expectation; pErr != nil { + return -1, -1, pErr.Err + } + + return msg.Partition, msg.Offset, nil +} + +func (sp *syncProducer) SendMessages(msgs []*ProducerMessage) error { + expectations := make(chan chan *ProducerError, len(msgs)) + go func() { + for _, msg := range msgs { + expectation := make(chan *ProducerError, 1) + msg.expectation = expectation + sp.producer.Input() <- msg + expectations <- expectation + } + close(expectations) + }() + + var errors ProducerErrors + for expectation := range expectations { + if pErr := <-expectation; pErr != nil { + errors = append(errors, pErr) + } + } + + if len(errors) > 0 { + return errors + } + return nil +} + +func (sp *syncProducer) handleSuccesses() { + defer sp.wg.Done() + for msg := range sp.producer.Successes() { + expectation := msg.expectation + expectation <- nil + } +} + +func (sp *syncProducer) handleErrors() { + defer sp.wg.Done() + for err := range sp.producer.Errors() { + expectation := err.Msg.expectation + expectation <- err + } +} + +func (sp *syncProducer) Close() error { + sp.producer.AsyncClose() + sp.wg.Wait() + return nil +} diff --git a/vendor/github.com/Shopify/sarama/timestamp.go b/vendor/github.com/Shopify/sarama/timestamp.go new file mode 100644 index 000000000000..372278d0bfa2 --- /dev/null +++ b/vendor/github.com/Shopify/sarama/timestamp.go @@ -0,0 +1,40 @@ +package sarama + +import ( + "fmt" + "time" +) + +type Timestamp struct { + *time.Time +} + +func (t Timestamp) encode(pe packetEncoder) error { + timestamp := int64(-1) + + if !t.Before(time.Unix(0, 0)) { + timestamp = t.UnixNano() / int64(time.Millisecond) + } else if !t.IsZero() { + return PacketEncodingError{fmt.Sprintf("invalid timestamp (%v)", t)} + } + + pe.putInt64(timestamp) + return nil +} + +func (t Timestamp) decode(pd packetDecoder) error { + millis, err := pd.getInt64() + if err != nil { + return err + } + + // negative timestamps are invalid, in these cases we should return + // a zero time + timestamp := time.Time{} + if millis >= 0 { + timestamp = time.Unix(millis/1000, (millis%1000)*int64(time.Millisecond)) + } + + *t.Time = timestamp + return nil +} diff --git a/vendor/github.com/Shopify/sarama/txn_offset_commit_request.go b/vendor/github.com/Shopify/sarama/txn_offset_commit_request.go new file mode 100644 index 000000000000..c4043a33520d --- /dev/null +++ b/vendor/github.com/Shopify/sarama/txn_offset_commit_request.go @@ -0,0 +1,130 @@ +package sarama + +type TxnOffsetCommitRequest struct { + TransactionalID string + GroupID string + ProducerID int64 + ProducerEpoch int16 + Topics map[string][]*PartitionOffsetMetadata +} + +func (t *TxnOffsetCommitRequest) encode(pe packetEncoder) error { + if err := pe.putString(t.TransactionalID); err != nil { + return err + } + if err := pe.putString(t.GroupID); err != nil { + return err + } + pe.putInt64(t.ProducerID) + pe.putInt16(t.ProducerEpoch) + + if err := pe.putArrayLength(len(t.Topics)); err != nil { + return err + } + for topic, partitions := range t.Topics { + if err := pe.putString(topic); err != nil { + return err + } + if err := pe.putArrayLength(len(partitions)); err != nil { + return err + } + for _, partition := range partitions { + if err := partition.encode(pe); err != nil { + return err + } + } + } + + return nil +} + +func (t *TxnOffsetCommitRequest) decode(pd packetDecoder, version int16) (err error) { + if t.TransactionalID, err = pd.getString(); err != nil { + return err + } + if t.GroupID, err = pd.getString(); err != nil { + return err + } + if t.ProducerID, err = pd.getInt64(); err != nil { + return err + } + if t.ProducerEpoch, err = pd.getInt16(); err != nil { + return err + } + + n, err := pd.getArrayLength() + if err != nil { + return err + } + + t.Topics = make(map[string][]*PartitionOffsetMetadata) + for i := 0; i < n; i++ { + topic, err := pd.getString() + if err != nil { + return err + } + + m, err := pd.getArrayLength() + if err != nil { + return err + } + + t.Topics[topic] = make([]*PartitionOffsetMetadata, m) + + for j := 0; j < m; j++ { + partitionOffsetMetadata := new(PartitionOffsetMetadata) + if err := partitionOffsetMetadata.decode(pd, version); err != nil { + return err + } + t.Topics[topic][j] = partitionOffsetMetadata + } + } + + return nil +} + +func (a *TxnOffsetCommitRequest) key() int16 { + return 28 +} + +func (a *TxnOffsetCommitRequest) version() int16 { + return 0 +} + +func (a *TxnOffsetCommitRequest) headerVersion() int16 { + return 1 +} + +func (a *TxnOffsetCommitRequest) requiredVersion() KafkaVersion { + return V0_11_0_0 +} + +type PartitionOffsetMetadata struct { + Partition int32 + Offset int64 + Metadata *string +} + +func (p *PartitionOffsetMetadata) encode(pe packetEncoder) error { + pe.putInt32(p.Partition) + pe.putInt64(p.Offset) + if err := pe.putNullableString(p.Metadata); err != nil { + return err + } + + return nil +} + +func (p *PartitionOffsetMetadata) decode(pd packetDecoder, version int16) (err error) { + if p.Partition, err = pd.getInt32(); err != nil { + return err + } + if p.Offset, err = pd.getInt64(); err != nil { + return err + } + if p.Metadata, err = pd.getNullableString(); err != nil { + return err + } + + return nil +} diff --git a/vendor/github.com/Shopify/sarama/txn_offset_commit_response.go b/vendor/github.com/Shopify/sarama/txn_offset_commit_response.go new file mode 100644 index 000000000000..94d8029dace6 --- /dev/null +++ b/vendor/github.com/Shopify/sarama/txn_offset_commit_response.go @@ -0,0 +1,87 @@ +package sarama + +import ( + "time" +) + +type TxnOffsetCommitResponse struct { + ThrottleTime time.Duration + Topics map[string][]*PartitionError +} + +func (t *TxnOffsetCommitResponse) encode(pe packetEncoder) error { + pe.putInt32(int32(t.ThrottleTime / time.Millisecond)) + if err := pe.putArrayLength(len(t.Topics)); err != nil { + return err + } + + for topic, e := range t.Topics { + if err := pe.putString(topic); err != nil { + return err + } + if err := pe.putArrayLength(len(e)); err != nil { + return err + } + for _, partitionError := range e { + if err := partitionError.encode(pe); err != nil { + return err + } + } + } + + return nil +} + +func (t *TxnOffsetCommitResponse) decode(pd packetDecoder, version int16) (err error) { + throttleTime, err := pd.getInt32() + if err != nil { + return err + } + t.ThrottleTime = time.Duration(throttleTime) * time.Millisecond + + n, err := pd.getArrayLength() + if err != nil { + return err + } + + t.Topics = make(map[string][]*PartitionError) + + for i := 0; i < n; i++ { + topic, err := pd.getString() + if err != nil { + return err + } + + m, err := pd.getArrayLength() + if err != nil { + return err + } + + t.Topics[topic] = make([]*PartitionError, m) + + for j := 0; j < m; j++ { + t.Topics[topic][j] = new(PartitionError) + if err := t.Topics[topic][j].decode(pd, version); err != nil { + return err + } + } + } + + return nil +} + +func (a *TxnOffsetCommitResponse) key() int16 { + return 28 +} + +func (a *TxnOffsetCommitResponse) version() int16 { + return 0 +} + +func (a *TxnOffsetCommitResponse) headerVersion() int16 { + return 0 +} + +func (a *TxnOffsetCommitResponse) requiredVersion() KafkaVersion { + return V0_11_0_0 +} diff --git a/vendor/github.com/Shopify/sarama/utils.go b/vendor/github.com/Shopify/sarama/utils.go new file mode 100644 index 000000000000..4ff973033459 --- /dev/null +++ b/vendor/github.com/Shopify/sarama/utils.go @@ -0,0 +1,262 @@ +package sarama + +import ( + "bufio" + "fmt" + "net" + "regexp" +) + +type none struct{} + +// make []int32 sortable so we can sort partition numbers +type int32Slice []int32 + +func (slice int32Slice) Len() int { + return len(slice) +} + +func (slice int32Slice) Less(i, j int) bool { + return slice[i] < slice[j] +} + +func (slice int32Slice) Swap(i, j int) { + slice[i], slice[j] = slice[j], slice[i] +} + +func dupInt32Slice(input []int32) []int32 { + ret := make([]int32, 0, len(input)) + ret = append(ret, input...) + return ret +} + +func withRecover(fn func()) { + defer func() { + handler := PanicHandler + if handler != nil { + if err := recover(); err != nil { + handler(err) + } + } + }() + + fn() +} + +func safeAsyncClose(b *Broker) { + tmp := b // local var prevents clobbering in goroutine + go withRecover(func() { + if connected, _ := tmp.Connected(); connected { + if err := tmp.Close(); err != nil { + Logger.Println("Error closing broker", tmp.ID(), ":", err) + } + } + }) +} + +// Encoder is a simple interface for any type that can be encoded as an array of bytes +// in order to be sent as the key or value of a Kafka message. Length() is provided as an +// optimization, and must return the same as len() on the result of Encode(). +type Encoder interface { + Encode() ([]byte, error) + Length() int +} + +// make strings and byte slices encodable for convenience so they can be used as keys +// and/or values in kafka messages + +// StringEncoder implements the Encoder interface for Go strings so that they can be used +// as the Key or Value in a ProducerMessage. +type StringEncoder string + +func (s StringEncoder) Encode() ([]byte, error) { + return []byte(s), nil +} + +func (s StringEncoder) Length() int { + return len(s) +} + +// ByteEncoder implements the Encoder interface for Go byte slices so that they can be used +// as the Key or Value in a ProducerMessage. +type ByteEncoder []byte + +func (b ByteEncoder) Encode() ([]byte, error) { + return b, nil +} + +func (b ByteEncoder) Length() int { + return len(b) +} + +// bufConn wraps a net.Conn with a buffer for reads to reduce the number of +// reads that trigger syscalls. +type bufConn struct { + net.Conn + buf *bufio.Reader +} + +func newBufConn(conn net.Conn) *bufConn { + return &bufConn{ + Conn: conn, + buf: bufio.NewReader(conn), + } +} + +func (bc *bufConn) Read(b []byte) (n int, err error) { + return bc.buf.Read(b) +} + +// KafkaVersion instances represent versions of the upstream Kafka broker. +type KafkaVersion struct { + // it's a struct rather than just typing the array directly to make it opaque and stop people + // generating their own arbitrary versions + version [4]uint +} + +func newKafkaVersion(major, minor, veryMinor, patch uint) KafkaVersion { + return KafkaVersion{ + version: [4]uint{major, minor, veryMinor, patch}, + } +} + +// IsAtLeast return true if and only if the version it is called on is +// greater than or equal to the version passed in: +// V1.IsAtLeast(V2) // false +// V2.IsAtLeast(V1) // true +func (v KafkaVersion) IsAtLeast(other KafkaVersion) bool { + for i := range v.version { + if v.version[i] > other.version[i] { + return true + } else if v.version[i] < other.version[i] { + return false + } + } + return true +} + +// Effective constants defining the supported kafka versions. +var ( + V0_8_2_0 = newKafkaVersion(0, 8, 2, 0) + V0_8_2_1 = newKafkaVersion(0, 8, 2, 1) + V0_8_2_2 = newKafkaVersion(0, 8, 2, 2) + V0_9_0_0 = newKafkaVersion(0, 9, 0, 0) + V0_9_0_1 = newKafkaVersion(0, 9, 0, 1) + V0_10_0_0 = newKafkaVersion(0, 10, 0, 0) + V0_10_0_1 = newKafkaVersion(0, 10, 0, 1) + V0_10_1_0 = newKafkaVersion(0, 10, 1, 0) + V0_10_1_1 = newKafkaVersion(0, 10, 1, 1) + V0_10_2_0 = newKafkaVersion(0, 10, 2, 0) + V0_10_2_1 = newKafkaVersion(0, 10, 2, 1) + V0_10_2_2 = newKafkaVersion(0, 10, 2, 2) + V0_11_0_0 = newKafkaVersion(0, 11, 0, 0) + V0_11_0_1 = newKafkaVersion(0, 11, 0, 1) + V0_11_0_2 = newKafkaVersion(0, 11, 0, 2) + V1_0_0_0 = newKafkaVersion(1, 0, 0, 0) + V1_0_1_0 = newKafkaVersion(1, 0, 1, 0) + V1_0_2_0 = newKafkaVersion(1, 0, 2, 0) + V1_1_0_0 = newKafkaVersion(1, 1, 0, 0) + V1_1_1_0 = newKafkaVersion(1, 1, 1, 0) + V2_0_0_0 = newKafkaVersion(2, 0, 0, 0) + V2_0_1_0 = newKafkaVersion(2, 0, 1, 0) + V2_1_0_0 = newKafkaVersion(2, 1, 0, 0) + V2_1_1_0 = newKafkaVersion(2, 1, 1, 0) + V2_2_0_0 = newKafkaVersion(2, 2, 0, 0) + V2_2_1_0 = newKafkaVersion(2, 2, 1, 0) + V2_2_2_0 = newKafkaVersion(2, 2, 2, 0) + V2_3_0_0 = newKafkaVersion(2, 3, 0, 0) + V2_3_1_0 = newKafkaVersion(2, 3, 1, 0) + V2_4_0_0 = newKafkaVersion(2, 4, 0, 0) + V2_4_1_0 = newKafkaVersion(2, 4, 1, 0) + V2_5_0_0 = newKafkaVersion(2, 5, 0, 0) + V2_5_1_0 = newKafkaVersion(2, 5, 1, 0) + V2_6_0_0 = newKafkaVersion(2, 6, 0, 0) + V2_6_1_0 = newKafkaVersion(2, 6, 1, 0) + V2_6_2_0 = newKafkaVersion(2, 6, 2, 0) + V2_7_0_0 = newKafkaVersion(2, 7, 0, 0) + V2_7_1_0 = newKafkaVersion(2, 7, 1, 0) + V2_8_0_0 = newKafkaVersion(2, 8, 0, 0) + V2_8_1_0 = newKafkaVersion(2, 8, 1, 0) + V3_0_0_0 = newKafkaVersion(3, 0, 0, 0) + + SupportedVersions = []KafkaVersion{ + V0_8_2_0, + V0_8_2_1, + V0_8_2_2, + V0_9_0_0, + V0_9_0_1, + V0_10_0_0, + V0_10_0_1, + V0_10_1_0, + V0_10_1_1, + V0_10_2_0, + V0_10_2_1, + V0_10_2_2, + V0_11_0_0, + V0_11_0_1, + V0_11_0_2, + V1_0_0_0, + V1_0_1_0, + V1_0_2_0, + V1_1_0_0, + V1_1_1_0, + V2_0_0_0, + V2_0_1_0, + V2_1_0_0, + V2_1_1_0, + V2_2_0_0, + V2_2_1_0, + V2_2_2_0, + V2_3_0_0, + V2_3_1_0, + V2_4_0_0, + V2_4_1_0, + V2_5_0_0, + V2_5_1_0, + V2_6_0_0, + V2_6_1_0, + V2_6_2_0, + V2_7_0_0, + V2_7_1_0, + V2_8_0_0, + V2_8_1_0, + V3_0_0_0, + } + MinVersion = V0_8_2_0 + MaxVersion = V3_0_0_0 + DefaultVersion = V1_0_0_0 +) + +// ParseKafkaVersion parses and returns kafka version or error from a string +func ParseKafkaVersion(s string) (KafkaVersion, error) { + if len(s) < 5 { + return DefaultVersion, fmt.Errorf("invalid version `%s`", s) + } + var major, minor, veryMinor, patch uint + var err error + if s[0] == '0' { + err = scanKafkaVersion(s, `^0\.\d+\.\d+\.\d+$`, "0.%d.%d.%d", [3]*uint{&minor, &veryMinor, &patch}) + } else { + err = scanKafkaVersion(s, `^\d+\.\d+\.\d+$`, "%d.%d.%d", [3]*uint{&major, &minor, &veryMinor}) + } + if err != nil { + return DefaultVersion, err + } + return newKafkaVersion(major, minor, veryMinor, patch), nil +} + +func scanKafkaVersion(s string, pattern string, format string, v [3]*uint) error { + if !regexp.MustCompile(pattern).MatchString(s) { + return fmt.Errorf("invalid version `%s`", s) + } + _, err := fmt.Sscanf(s, format, v[0], v[1], v[2]) + return err +} + +func (v KafkaVersion) String() string { + if v.version[0] == 0 { + return fmt.Sprintf("0.%d.%d.%d", v.version[1], v.version[2], v.version[3]) + } + + return fmt.Sprintf("%d.%d.%d", v.version[0], v.version[1], v.version[2]) +} diff --git a/vendor/github.com/Shopify/sarama/version.go b/vendor/github.com/Shopify/sarama/version.go new file mode 100644 index 000000000000..590329ef84ec --- /dev/null +++ b/vendor/github.com/Shopify/sarama/version.go @@ -0,0 +1,20 @@ +package sarama + +import "runtime/debug" + +var v string + +func version() string { + if v == "" { + bi, ok := debug.ReadBuildInfo() + if ok { + v = bi.Main.Version + } else { + // if we can't read a go module version then they're using a git + // clone or vendored module so all we can do is report "dev" for + // the version + v = "dev" + } + } + return v +} diff --git a/vendor/github.com/Shopify/sarama/zstd.go b/vendor/github.com/Shopify/sarama/zstd.go new file mode 100644 index 000000000000..e23bfc4772f7 --- /dev/null +++ b/vendor/github.com/Shopify/sarama/zstd.go @@ -0,0 +1,18 @@ +package sarama + +import ( + "github.com/klauspost/compress/zstd" +) + +var ( + zstdDec, _ = zstd.NewReader(nil) + zstdEnc, _ = zstd.NewWriter(nil, zstd.WithZeroFrames(true)) +) + +func zstdDecompress(dst, src []byte) ([]byte, error) { + return zstdDec.DecodeAll(src, dst) +} + +func zstdCompress(dst, src []byte) ([]byte, error) { + return zstdEnc.EncodeAll(src, dst), nil +} diff --git a/vendor/github.com/eapache/go-resiliency/LICENSE b/vendor/github.com/eapache/go-resiliency/LICENSE new file mode 100644 index 000000000000..698a3f51397e --- /dev/null +++ b/vendor/github.com/eapache/go-resiliency/LICENSE @@ -0,0 +1,22 @@ +The MIT License (MIT) + +Copyright (c) 2014 Evan Huus + +Permission is hereby granted, free of charge, to any person obtaining a copy +of this software and associated documentation files (the "Software"), to deal +in the Software without restriction, including without limitation the rights +to use, copy, modify, merge, publish, distribute, sublicense, and/or sell +copies of the Software, and to permit persons to whom the Software is +furnished to do so, subject to the following conditions: + +The above copyright notice and this permission notice shall be included in all +copies or substantial portions of the Software. + +THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR +IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, +FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE +AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER +LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, +OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE +SOFTWARE. + diff --git a/vendor/github.com/eapache/go-resiliency/breaker/README.md b/vendor/github.com/eapache/go-resiliency/breaker/README.md new file mode 100644 index 000000000000..2d1b3d93225d --- /dev/null +++ b/vendor/github.com/eapache/go-resiliency/breaker/README.md @@ -0,0 +1,34 @@ +circuit-breaker +=============== + +[![Build Status](https://travis-ci.org/eapache/go-resiliency.svg?branch=master)](https://travis-ci.org/eapache/go-resiliency) +[![GoDoc](https://godoc.org/github.com/eapache/go-resiliency/breaker?status.svg)](https://godoc.org/github.com/eapache/go-resiliency/breaker) +[![Code of Conduct](https://img.shields.io/badge/code%20of%20conduct-active-blue.svg)](https://eapache.github.io/conduct.html) + +The circuit-breaker resiliency pattern for golang. + +Creating a breaker takes three parameters: +- error threshold (for opening the breaker) +- success threshold (for closing the breaker) +- timeout (how long to keep the breaker open) + +```go +b := breaker.New(3, 1, 5*time.Second) + +for { + result := b.Run(func() error { + // communicate with some external service and + // return an error if the communication failed + return nil + }) + + switch result { + case nil: + // success! + case breaker.ErrBreakerOpen: + // our function wasn't run because the breaker was open + default: + // some other error + } +} +``` diff --git a/vendor/github.com/eapache/go-resiliency/breaker/breaker.go b/vendor/github.com/eapache/go-resiliency/breaker/breaker.go new file mode 100644 index 000000000000..f88ca7248b0f --- /dev/null +++ b/vendor/github.com/eapache/go-resiliency/breaker/breaker.go @@ -0,0 +1,161 @@ +// Package breaker implements the circuit-breaker resiliency pattern for Go. +package breaker + +import ( + "errors" + "sync" + "sync/atomic" + "time" +) + +// ErrBreakerOpen is the error returned from Run() when the function is not executed +// because the breaker is currently open. +var ErrBreakerOpen = errors.New("circuit breaker is open") + +const ( + closed uint32 = iota + open + halfOpen +) + +// Breaker implements the circuit-breaker resiliency pattern +type Breaker struct { + errorThreshold, successThreshold int + timeout time.Duration + + lock sync.Mutex + state uint32 + errors, successes int + lastError time.Time +} + +// New constructs a new circuit-breaker that starts closed. +// From closed, the breaker opens if "errorThreshold" errors are seen +// without an error-free period of at least "timeout". From open, the +// breaker half-closes after "timeout". From half-open, the breaker closes +// after "successThreshold" consecutive successes, or opens on a single error. +func New(errorThreshold, successThreshold int, timeout time.Duration) *Breaker { + return &Breaker{ + errorThreshold: errorThreshold, + successThreshold: successThreshold, + timeout: timeout, + } +} + +// Run will either return ErrBreakerOpen immediately if the circuit-breaker is +// already open, or it will run the given function and pass along its return +// value. It is safe to call Run concurrently on the same Breaker. +func (b *Breaker) Run(work func() error) error { + state := atomic.LoadUint32(&b.state) + + if state == open { + return ErrBreakerOpen + } + + return b.doWork(state, work) +} + +// Go will either return ErrBreakerOpen immediately if the circuit-breaker is +// already open, or it will run the given function in a separate goroutine. +// If the function is run, Go will return nil immediately, and will *not* return +// the return value of the function. It is safe to call Go concurrently on the +// same Breaker. +func (b *Breaker) Go(work func() error) error { + state := atomic.LoadUint32(&b.state) + + if state == open { + return ErrBreakerOpen + } + + // errcheck complains about ignoring the error return value, but + // that's on purpose; if you want an error from a goroutine you have to + // get it over a channel or something + go b.doWork(state, work) + + return nil +} + +func (b *Breaker) doWork(state uint32, work func() error) error { + var panicValue interface{} + + result := func() error { + defer func() { + panicValue = recover() + }() + return work() + }() + + if result == nil && panicValue == nil && state == closed { + // short-circuit the normal, success path without contending + // on the lock + return nil + } + + // oh well, I guess we have to contend on the lock + b.processResult(result, panicValue) + + if panicValue != nil { + // as close as Go lets us come to a "rethrow" although unfortunately + // we lose the original panicing location + panic(panicValue) + } + + return result +} + +func (b *Breaker) processResult(result error, panicValue interface{}) { + b.lock.Lock() + defer b.lock.Unlock() + + if result == nil && panicValue == nil { + if b.state == halfOpen { + b.successes++ + if b.successes == b.successThreshold { + b.closeBreaker() + } + } + } else { + if b.errors > 0 { + expiry := b.lastError.Add(b.timeout) + if time.Now().After(expiry) { + b.errors = 0 + } + } + + switch b.state { + case closed: + b.errors++ + if b.errors == b.errorThreshold { + b.openBreaker() + } else { + b.lastError = time.Now() + } + case halfOpen: + b.openBreaker() + } + } +} + +func (b *Breaker) openBreaker() { + b.changeState(open) + go b.timer() +} + +func (b *Breaker) closeBreaker() { + b.changeState(closed) +} + +func (b *Breaker) timer() { + time.Sleep(b.timeout) + + b.lock.Lock() + defer b.lock.Unlock() + + b.changeState(halfOpen) +} + +func (b *Breaker) changeState(newState uint32) { + b.errors = 0 + b.successes = 0 + atomic.StoreUint32(&b.state, newState) +} diff --git a/vendor/github.com/eapache/go-xerial-snappy/.gitignore b/vendor/github.com/eapache/go-xerial-snappy/.gitignore new file mode 100644 index 000000000000..daf913b1b347 --- /dev/null +++ b/vendor/github.com/eapache/go-xerial-snappy/.gitignore @@ -0,0 +1,24 @@ +# Compiled Object files, Static and Dynamic libs (Shared Objects) +*.o +*.a +*.so + +# Folders +_obj +_test + +# Architecture specific extensions/prefixes +*.[568vq] +[568vq].out + +*.cgo1.go +*.cgo2.c +_cgo_defun.c +_cgo_gotypes.go +_cgo_export.* + +_testmain.go + +*.exe +*.test +*.prof diff --git a/vendor/github.com/eapache/go-xerial-snappy/.travis.yml b/vendor/github.com/eapache/go-xerial-snappy/.travis.yml new file mode 100644 index 000000000000..d6cf4f1fa1b5 --- /dev/null +++ b/vendor/github.com/eapache/go-xerial-snappy/.travis.yml @@ -0,0 +1,7 @@ +language: go + +go: +- 1.5.4 +- 1.6.1 + +sudo: false diff --git a/vendor/github.com/eapache/go-xerial-snappy/LICENSE b/vendor/github.com/eapache/go-xerial-snappy/LICENSE new file mode 100644 index 000000000000..5bf3688d9e41 --- /dev/null +++ b/vendor/github.com/eapache/go-xerial-snappy/LICENSE @@ -0,0 +1,21 @@ +The MIT License (MIT) + +Copyright (c) 2016 Evan Huus + +Permission is hereby granted, free of charge, to any person obtaining a copy +of this software and associated documentation files (the "Software"), to deal +in the Software without restriction, including without limitation the rights +to use, copy, modify, merge, publish, distribute, sublicense, and/or sell +copies of the Software, and to permit persons to whom the Software is +furnished to do so, subject to the following conditions: + +The above copyright notice and this permission notice shall be included in all +copies or substantial portions of the Software. + +THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR +IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, +FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE +AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER +LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, +OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE +SOFTWARE. diff --git a/vendor/github.com/eapache/go-xerial-snappy/README.md b/vendor/github.com/eapache/go-xerial-snappy/README.md new file mode 100644 index 000000000000..3f2695c72826 --- /dev/null +++ b/vendor/github.com/eapache/go-xerial-snappy/README.md @@ -0,0 +1,13 @@ +# go-xerial-snappy + +[![Build Status](https://travis-ci.org/eapache/go-xerial-snappy.svg?branch=master)](https://travis-ci.org/eapache/go-xerial-snappy) + +Xerial-compatible Snappy framing support for golang. + +Packages using Xerial for snappy encoding use a framing format incompatible with +basically everything else in existence. This package wraps Go's built-in snappy +package to support it. + +Apps that use this format include Apache Kafka (see +https://github.com/dpkp/kafka-python/issues/126#issuecomment-35478921 for +details). diff --git a/vendor/github.com/eapache/go-xerial-snappy/fuzz.go b/vendor/github.com/eapache/go-xerial-snappy/fuzz.go new file mode 100644 index 000000000000..6a46f4784e11 --- /dev/null +++ b/vendor/github.com/eapache/go-xerial-snappy/fuzz.go @@ -0,0 +1,16 @@ +// +build gofuzz + +package snappy + +func Fuzz(data []byte) int { + decode, err := Decode(data) + if decode == nil && err == nil { + panic("nil error with nil result") + } + + if err != nil { + return 0 + } + + return 1 +} diff --git a/vendor/github.com/eapache/go-xerial-snappy/snappy.go b/vendor/github.com/eapache/go-xerial-snappy/snappy.go new file mode 100644 index 000000000000..ea8f7afeb335 --- /dev/null +++ b/vendor/github.com/eapache/go-xerial-snappy/snappy.go @@ -0,0 +1,131 @@ +package snappy + +import ( + "bytes" + "encoding/binary" + "errors" + + master "github.com/golang/snappy" +) + +const ( + sizeOffset = 16 + sizeBytes = 4 +) + +var ( + xerialHeader = []byte{130, 83, 78, 65, 80, 80, 89, 0} + + // This is xerial version 1 and minimally compatible with version 1 + xerialVersionInfo = []byte{0, 0, 0, 1, 0, 0, 0, 1} + + // ErrMalformed is returned by the decoder when the xerial framing + // is malformed + ErrMalformed = errors.New("malformed xerial framing") +) + +func min(x, y int) int { + if x < y { + return x + } + return y +} + +// Encode encodes data as snappy with no framing header. +func Encode(src []byte) []byte { + return master.Encode(nil, src) +} + +// EncodeStream *appends* to the specified 'dst' the compressed +// 'src' in xerial framing format. If 'dst' does not have enough +// capacity, then a new slice will be allocated. If 'dst' has +// non-zero length, then if *must* have been built using this function. +func EncodeStream(dst, src []byte) []byte { + if len(dst) == 0 { + dst = append(dst, xerialHeader...) + dst = append(dst, xerialVersionInfo...) + } + + // Snappy encode in blocks of maximum 32KB + var ( + max = len(src) + blockSize = 32 * 1024 + pos = 0 + chunk []byte + ) + + for pos < max { + newPos := min(pos + blockSize, max) + chunk = master.Encode(chunk[:cap(chunk)], src[pos:newPos]) + + // First encode the compressed size (big-endian) + // Put* panics if the buffer is too small, so pad 4 bytes first + origLen := len(dst) + dst = append(dst, dst[0:4]...) + binary.BigEndian.PutUint32(dst[origLen:], uint32(len(chunk))) + + // And now the compressed data + dst = append(dst, chunk...) + pos = newPos + } + return dst +} + +// Decode decodes snappy data whether it is traditional unframed +// or includes the xerial framing format. +func Decode(src []byte) ([]byte, error) { + return DecodeInto(nil, src) +} + +// DecodeInto decodes snappy data whether it is traditional unframed +// or includes the xerial framing format into the specified `dst`. +// It is assumed that the entirety of `dst` including all capacity is available +// for use by this function. If `dst` is nil *or* insufficiently large to hold +// the decoded `src`, new space will be allocated. +func DecodeInto(dst, src []byte) ([]byte, error) { + var max = len(src) + if max < len(xerialHeader) { + return nil, ErrMalformed + } + + if !bytes.Equal(src[:8], xerialHeader) { + return master.Decode(dst[:cap(dst)], src) + } + + if max < sizeOffset+sizeBytes { + return nil, ErrMalformed + } + + if dst == nil { + dst = make([]byte, 0, len(src)) + } + + dst = dst[:0] + var ( + pos = sizeOffset + chunk []byte + err error + ) + + for pos+sizeBytes <= max { + size := int(binary.BigEndian.Uint32(src[pos : pos+sizeBytes])) + pos += sizeBytes + + nextPos := pos + size + // On architectures where int is 32-bytes wide size + pos could + // overflow so we need to check the low bound as well as the + // high + if nextPos < pos || nextPos > max { + return nil, ErrMalformed + } + + chunk, err = master.Decode(chunk[:cap(chunk)], src[pos:nextPos]) + + if err != nil { + return nil, err + } + pos = nextPos + dst = append(dst, chunk...) + } + return dst, nil +} diff --git a/vendor/github.com/eapache/queue/.gitignore b/vendor/github.com/eapache/queue/.gitignore new file mode 100644 index 000000000000..836562412fe8 --- /dev/null +++ b/vendor/github.com/eapache/queue/.gitignore @@ -0,0 +1,23 @@ +# Compiled Object files, Static and Dynamic libs (Shared Objects) +*.o +*.a +*.so + +# Folders +_obj +_test + +# Architecture specific extensions/prefixes +*.[568vq] +[568vq].out + +*.cgo1.go +*.cgo2.c +_cgo_defun.c +_cgo_gotypes.go +_cgo_export.* + +_testmain.go + +*.exe +*.test diff --git a/vendor/github.com/eapache/queue/.travis.yml b/vendor/github.com/eapache/queue/.travis.yml new file mode 100644 index 000000000000..235a40a493ff --- /dev/null +++ b/vendor/github.com/eapache/queue/.travis.yml @@ -0,0 +1,7 @@ +language: go +sudo: false + +go: + - 1.2 + - 1.3 + - 1.4 diff --git a/vendor/github.com/eapache/queue/LICENSE b/vendor/github.com/eapache/queue/LICENSE new file mode 100644 index 000000000000..d5f36dbcaaf6 --- /dev/null +++ b/vendor/github.com/eapache/queue/LICENSE @@ -0,0 +1,21 @@ +The MIT License (MIT) + +Copyright (c) 2014 Evan Huus + +Permission is hereby granted, free of charge, to any person obtaining a copy +of this software and associated documentation files (the "Software"), to deal +in the Software without restriction, including without limitation the rights +to use, copy, modify, merge, publish, distribute, sublicense, and/or sell +copies of the Software, and to permit persons to whom the Software is +furnished to do so, subject to the following conditions: + +The above copyright notice and this permission notice shall be included in all +copies or substantial portions of the Software. + +THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR +IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, +FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE +AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER +LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, +OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE +SOFTWARE. \ No newline at end of file diff --git a/vendor/github.com/eapache/queue/README.md b/vendor/github.com/eapache/queue/README.md new file mode 100644 index 000000000000..8e782335cd77 --- /dev/null +++ b/vendor/github.com/eapache/queue/README.md @@ -0,0 +1,16 @@ +Queue +===== + +[![Build Status](https://travis-ci.org/eapache/queue.svg)](https://travis-ci.org/eapache/queue) +[![GoDoc](https://godoc.org/github.com/eapache/queue?status.png)](https://godoc.org/github.com/eapache/queue) +[![Code of Conduct](https://img.shields.io/badge/code%20of%20conduct-active-blue.svg)](https://eapache.github.io/conduct.html) + +A fast Golang queue using a ring-buffer, based on the version suggested by Dariusz Górecki. +Using this instead of other, simpler, queue implementations (slice+append or linked list) provides +substantial memory and time benefits, and fewer GC pauses. + +The queue implemented here is as fast as it is in part because it is *not* thread-safe. + +Follows semantic versioning using https://gopkg.in/ - import from +[`gopkg.in/eapache/queue.v1`](https://gopkg.in/eapache/queue.v1) +for guaranteed API stability. diff --git a/vendor/github.com/eapache/queue/queue.go b/vendor/github.com/eapache/queue/queue.go new file mode 100644 index 000000000000..71d1acdf27b2 --- /dev/null +++ b/vendor/github.com/eapache/queue/queue.go @@ -0,0 +1,102 @@ +/* +Package queue provides a fast, ring-buffer queue based on the version suggested by Dariusz Górecki. +Using this instead of other, simpler, queue implementations (slice+append or linked list) provides +substantial memory and time benefits, and fewer GC pauses. + +The queue implemented here is as fast as it is for an additional reason: it is *not* thread-safe. +*/ +package queue + +// minQueueLen is smallest capacity that queue may have. +// Must be power of 2 for bitwise modulus: x % n == x & (n - 1). +const minQueueLen = 16 + +// Queue represents a single instance of the queue data structure. +type Queue struct { + buf []interface{} + head, tail, count int +} + +// New constructs and returns a new Queue. +func New() *Queue { + return &Queue{ + buf: make([]interface{}, minQueueLen), + } +} + +// Length returns the number of elements currently stored in the queue. +func (q *Queue) Length() int { + return q.count +} + +// resizes the queue to fit exactly twice its current contents +// this can result in shrinking if the queue is less than half-full +func (q *Queue) resize() { + newBuf := make([]interface{}, q.count<<1) + + if q.tail > q.head { + copy(newBuf, q.buf[q.head:q.tail]) + } else { + n := copy(newBuf, q.buf[q.head:]) + copy(newBuf[n:], q.buf[:q.tail]) + } + + q.head = 0 + q.tail = q.count + q.buf = newBuf +} + +// Add puts an element on the end of the queue. +func (q *Queue) Add(elem interface{}) { + if q.count == len(q.buf) { + q.resize() + } + + q.buf[q.tail] = elem + // bitwise modulus + q.tail = (q.tail + 1) & (len(q.buf) - 1) + q.count++ +} + +// Peek returns the element at the head of the queue. This call panics +// if the queue is empty. +func (q *Queue) Peek() interface{} { + if q.count <= 0 { + panic("queue: Peek() called on empty queue") + } + return q.buf[q.head] +} + +// Get returns the element at index i in the queue. If the index is +// invalid, the call will panic. This method accepts both positive and +// negative index values. Index 0 refers to the first element, and +// index -1 refers to the last. +func (q *Queue) Get(i int) interface{} { + // If indexing backwards, convert to positive index. + if i < 0 { + i += q.count + } + if i < 0 || i >= q.count { + panic("queue: Get() called with index out of range") + } + // bitwise modulus + return q.buf[(q.head+i)&(len(q.buf)-1)] +} + +// Remove removes and returns the element from the front of the queue. If the +// queue is empty, the call will panic. +func (q *Queue) Remove() interface{} { + if q.count <= 0 { + panic("queue: Remove() called on empty queue") + } + ret := q.buf[q.head] + q.buf[q.head] = nil + // bitwise modulus + q.head = (q.head + 1) & (len(q.buf) - 1) + q.count-- + // Resize down if buffer 1/4 full. + if len(q.buf) > minQueueLen && (q.count<<2) == len(q.buf) { + q.resize() + } + return ret +} diff --git a/vendor/github.com/hashicorp/go-uuid/.travis.yml b/vendor/github.com/hashicorp/go-uuid/.travis.yml new file mode 100644 index 000000000000..769849071ed7 --- /dev/null +++ b/vendor/github.com/hashicorp/go-uuid/.travis.yml @@ -0,0 +1,12 @@ +language: go + +sudo: false + +go: + - 1.4 + - 1.5 + - 1.6 + - tip + +script: + - go test -bench . -benchmem -v ./... diff --git a/vendor/github.com/hashicorp/go-uuid/LICENSE b/vendor/github.com/hashicorp/go-uuid/LICENSE new file mode 100644 index 000000000000..e87a115e462e --- /dev/null +++ b/vendor/github.com/hashicorp/go-uuid/LICENSE @@ -0,0 +1,363 @@ +Mozilla Public License, version 2.0 + +1. Definitions + +1.1. "Contributor" + + means each individual or legal entity that creates, contributes to the + creation of, or owns Covered Software. + +1.2. "Contributor Version" + + means the combination of the Contributions of others (if any) used by a + Contributor and that particular Contributor's Contribution. + +1.3. "Contribution" + + means Covered Software of a particular Contributor. + +1.4. "Covered Software" + + means Source Code Form to which the initial Contributor has attached the + notice in Exhibit A, the Executable Form of such Source Code Form, and + Modifications of such Source Code Form, in each case including portions + thereof. + +1.5. "Incompatible With Secondary Licenses" + means + + a. that the initial Contributor has attached the notice described in + Exhibit B to the Covered Software; or + + b. that the Covered Software was made available under the terms of + version 1.1 or earlier of the License, but not also under the terms of + a Secondary License. + +1.6. "Executable Form" + + means any form of the work other than Source Code Form. + +1.7. "Larger Work" + + means a work that combines Covered Software with other material, in a + separate file or files, that is not Covered Software. + +1.8. "License" + + means this document. + +1.9. "Licensable" + + means having the right to grant, to the maximum extent possible, whether + at the time of the initial grant or subsequently, any and all of the + rights conveyed by this License. + +1.10. "Modifications" + + means any of the following: + + a. any file in Source Code Form that results from an addition to, + deletion from, or modification of the contents of Covered Software; or + + b. any new file in Source Code Form that contains any Covered Software. + +1.11. "Patent Claims" of a Contributor + + means any patent claim(s), including without limitation, method, + process, and apparatus claims, in any patent Licensable by such + Contributor that would be infringed, but for the grant of the License, + by the making, using, selling, offering for sale, having made, import, + or transfer of either its Contributions or its Contributor Version. + +1.12. "Secondary License" + + means either the GNU General Public License, Version 2.0, the GNU Lesser + General Public License, Version 2.1, the GNU Affero General Public + License, Version 3.0, or any later versions of those licenses. + +1.13. "Source Code Form" + + means the form of the work preferred for making modifications. + +1.14. "You" (or "Your") + + means an individual or a legal entity exercising rights under this + License. For legal entities, "You" includes any entity that controls, is + controlled by, or is under common control with You. For purposes of this + definition, "control" means (a) the power, direct or indirect, to cause + the direction or management of such entity, whether by contract or + otherwise, or (b) ownership of more than fifty percent (50%) of the + outstanding shares or beneficial ownership of such entity. + + +2. License Grants and Conditions + +2.1. Grants + + Each Contributor hereby grants You a world-wide, royalty-free, + non-exclusive license: + + a. under intellectual property rights (other than patent or trademark) + Licensable by such Contributor to use, reproduce, make available, + modify, display, perform, distribute, and otherwise exploit its + Contributions, either on an unmodified basis, with Modifications, or + as part of a Larger Work; and + + b. under Patent Claims of such Contributor to make, use, sell, offer for + sale, have made, import, and otherwise transfer either its + Contributions or its Contributor Version. + +2.2. Effective Date + + The licenses granted in Section 2.1 with respect to any Contribution + become effective for each Contribution on the date the Contributor first + distributes such Contribution. + +2.3. Limitations on Grant Scope + + The licenses granted in this Section 2 are the only rights granted under + this License. No additional rights or licenses will be implied from the + distribution or licensing of Covered Software under this License. + Notwithstanding Section 2.1(b) above, no patent license is granted by a + Contributor: + + a. for any code that a Contributor has removed from Covered Software; or + + b. for infringements caused by: (i) Your and any other third party's + modifications of Covered Software, or (ii) the combination of its + Contributions with other software (except as part of its Contributor + Version); or + + c. under Patent Claims infringed by Covered Software in the absence of + its Contributions. + + This License does not grant any rights in the trademarks, service marks, + or logos of any Contributor (except as may be necessary to comply with + the notice requirements in Section 3.4). + +2.4. Subsequent Licenses + + No Contributor makes additional grants as a result of Your choice to + distribute the Covered Software under a subsequent version of this + License (see Section 10.2) or under the terms of a Secondary License (if + permitted under the terms of Section 3.3). + +2.5. Representation + + Each Contributor represents that the Contributor believes its + Contributions are its original creation(s) or it has sufficient rights to + grant the rights to its Contributions conveyed by this License. + +2.6. Fair Use + + This License is not intended to limit any rights You have under + applicable copyright doctrines of fair use, fair dealing, or other + equivalents. + +2.7. Conditions + + Sections 3.1, 3.2, 3.3, and 3.4 are conditions of the licenses granted in + Section 2.1. + + +3. Responsibilities + +3.1. Distribution of Source Form + + All distribution of Covered Software in Source Code Form, including any + Modifications that You create or to which You contribute, must be under + the terms of this License. You must inform recipients that the Source + Code Form of the Covered Software is governed by the terms of this + License, and how they can obtain a copy of this License. You may not + attempt to alter or restrict the recipients' rights in the Source Code + Form. + +3.2. Distribution of Executable Form + + If You distribute Covered Software in Executable Form then: + + a. such Covered Software must also be made available in Source Code Form, + as described in Section 3.1, and You must inform recipients of the + Executable Form how they can obtain a copy of such Source Code Form by + reasonable means in a timely manner, at a charge no more than the cost + of distribution to the recipient; and + + b. You may distribute such Executable Form under the terms of this + License, or sublicense it under different terms, provided that the + license for the Executable Form does not attempt to limit or alter the + recipients' rights in the Source Code Form under this License. + +3.3. Distribution of a Larger Work + + You may create and distribute a Larger Work under terms of Your choice, + provided that You also comply with the requirements of this License for + the Covered Software. If the Larger Work is a combination of Covered + Software with a work governed by one or more Secondary Licenses, and the + Covered Software is not Incompatible With Secondary Licenses, this + License permits You to additionally distribute such Covered Software + under the terms of such Secondary License(s), so that the recipient of + the Larger Work may, at their option, further distribute the Covered + Software under the terms of either this License or such Secondary + License(s). + +3.4. Notices + + You may not remove or alter the substance of any license notices + (including copyright notices, patent notices, disclaimers of warranty, or + limitations of liability) contained within the Source Code Form of the + Covered Software, except that You may alter any license notices to the + extent required to remedy known factual inaccuracies. + +3.5. Application of Additional Terms + + You may choose to offer, and to charge a fee for, warranty, support, + indemnity or liability obligations to one or more recipients of Covered + Software. However, You may do so only on Your own behalf, and not on + behalf of any Contributor. You must make it absolutely clear that any + such warranty, support, indemnity, or liability obligation is offered by + You alone, and You hereby agree to indemnify every Contributor for any + liability incurred by such Contributor as a result of warranty, support, + indemnity or liability terms You offer. You may include additional + disclaimers of warranty and limitations of liability specific to any + jurisdiction. + +4. Inability to Comply Due to Statute or Regulation + + If it is impossible for You to comply with any of the terms of this License + with respect to some or all of the Covered Software due to statute, + judicial order, or regulation then You must: (a) comply with the terms of + this License to the maximum extent possible; and (b) describe the + limitations and the code they affect. Such description must be placed in a + text file included with all distributions of the Covered Software under + this License. Except to the extent prohibited by statute or regulation, + such description must be sufficiently detailed for a recipient of ordinary + skill to be able to understand it. + +5. Termination + +5.1. The rights granted under this License will terminate automatically if You + fail to comply with any of its terms. However, if You become compliant, + then the rights granted under this License from a particular Contributor + are reinstated (a) provisionally, unless and until such Contributor + explicitly and finally terminates Your grants, and (b) on an ongoing + basis, if such Contributor fails to notify You of the non-compliance by + some reasonable means prior to 60 days after You have come back into + compliance. Moreover, Your grants from a particular Contributor are + reinstated on an ongoing basis if such Contributor notifies You of the + non-compliance by some reasonable means, this is the first time You have + received notice of non-compliance with this License from such + Contributor, and You become compliant prior to 30 days after Your receipt + of the notice. + +5.2. If You initiate litigation against any entity by asserting a patent + infringement claim (excluding declaratory judgment actions, + counter-claims, and cross-claims) alleging that a Contributor Version + directly or indirectly infringes any patent, then the rights granted to + You by any and all Contributors for the Covered Software under Section + 2.1 of this License shall terminate. + +5.3. In the event of termination under Sections 5.1 or 5.2 above, all end user + license agreements (excluding distributors and resellers) which have been + validly granted by You or Your distributors under this License prior to + termination shall survive termination. + +6. Disclaimer of Warranty + + Covered Software is provided under this License on an "as is" basis, + without warranty of any kind, either expressed, implied, or statutory, + including, without limitation, warranties that the Covered Software is free + of defects, merchantable, fit for a particular purpose or non-infringing. + The entire risk as to the quality and performance of the Covered Software + is with You. Should any Covered Software prove defective in any respect, + You (not any Contributor) assume the cost of any necessary servicing, + repair, or correction. This disclaimer of warranty constitutes an essential + part of this License. No use of any Covered Software is authorized under + this License except under this disclaimer. + +7. Limitation of Liability + + Under no circumstances and under no legal theory, whether tort (including + negligence), contract, or otherwise, shall any Contributor, or anyone who + distributes Covered Software as permitted above, be liable to You for any + direct, indirect, special, incidental, or consequential damages of any + character including, without limitation, damages for lost profits, loss of + goodwill, work stoppage, computer failure or malfunction, or any and all + other commercial damages or losses, even if such party shall have been + informed of the possibility of such damages. This limitation of liability + shall not apply to liability for death or personal injury resulting from + such party's negligence to the extent applicable law prohibits such + limitation. Some jurisdictions do not allow the exclusion or limitation of + incidental or consequential damages, so this exclusion and limitation may + not apply to You. + +8. Litigation + + Any litigation relating to this License may be brought only in the courts + of a jurisdiction where the defendant maintains its principal place of + business and such litigation shall be governed by laws of that + jurisdiction, without reference to its conflict-of-law provisions. Nothing + in this Section shall prevent a party's ability to bring cross-claims or + counter-claims. + +9. Miscellaneous + + This License represents the complete agreement concerning the subject + matter hereof. If any provision of this License is held to be + unenforceable, such provision shall be reformed only to the extent + necessary to make it enforceable. Any law or regulation which provides that + the language of a contract shall be construed against the drafter shall not + be used to construe this License against a Contributor. + + +10. Versions of the License + +10.1. New Versions + + Mozilla Foundation is the license steward. Except as provided in Section + 10.3, no one other than the license steward has the right to modify or + publish new versions of this License. Each version will be given a + distinguishing version number. + +10.2. Effect of New Versions + + You may distribute the Covered Software under the terms of the version + of the License under which You originally received the Covered Software, + or under the terms of any subsequent version published by the license + steward. + +10.3. Modified Versions + + If you create software not governed by this License, and you want to + create a new license for such software, you may create and use a + modified version of this License if you rename the license and remove + any references to the name of the license steward (except to note that + such modified license differs from this License). + +10.4. Distributing Source Code Form that is Incompatible With Secondary + Licenses If You choose to distribute Source Code Form that is + Incompatible With Secondary Licenses under the terms of this version of + the License, the notice described in Exhibit B of this License must be + attached. + +Exhibit A - Source Code Form License Notice + + This Source Code Form is subject to the + terms of the Mozilla Public License, v. + 2.0. If a copy of the MPL was not + distributed with this file, You can + obtain one at + http://mozilla.org/MPL/2.0/. + +If it is not possible or desirable to put the notice in a particular file, +then You may include the notice in a location (such as a LICENSE file in a +relevant directory) where a recipient would be likely to look for such a +notice. + +You may add additional accurate notices of copyright ownership. + +Exhibit B - "Incompatible With Secondary Licenses" Notice + + This Source Code Form is "Incompatible + With Secondary Licenses", as defined by + the Mozilla Public License, v. 2.0. + diff --git a/vendor/github.com/hashicorp/go-uuid/README.md b/vendor/github.com/hashicorp/go-uuid/README.md new file mode 100644 index 000000000000..fbde8b9aef60 --- /dev/null +++ b/vendor/github.com/hashicorp/go-uuid/README.md @@ -0,0 +1,8 @@ +# uuid [![Build Status](https://travis-ci.org/hashicorp/go-uuid.svg?branch=master)](https://travis-ci.org/hashicorp/go-uuid) + +Generates UUID-format strings using high quality, _purely random_ bytes. It is **not** intended to be RFC compliant, merely to use a well-understood string representation of a 128-bit value. It can also parse UUID-format strings into their component bytes. + +Documentation +============= + +The full documentation is available on [Godoc](http://godoc.org/github.com/hashicorp/go-uuid). diff --git a/vendor/github.com/hashicorp/go-uuid/uuid.go b/vendor/github.com/hashicorp/go-uuid/uuid.go new file mode 100644 index 000000000000..0c10c4e9f5fa --- /dev/null +++ b/vendor/github.com/hashicorp/go-uuid/uuid.go @@ -0,0 +1,83 @@ +package uuid + +import ( + "crypto/rand" + "encoding/hex" + "fmt" + "io" +) + +// GenerateRandomBytes is used to generate random bytes of given size. +func GenerateRandomBytes(size int) ([]byte, error) { + return GenerateRandomBytesWithReader(size, rand.Reader) +} + +// GenerateRandomBytesWithReader is used to generate random bytes of given size read from a given reader. +func GenerateRandomBytesWithReader(size int, reader io.Reader) ([]byte, error) { + if reader == nil { + return nil, fmt.Errorf("provided reader is nil") + } + buf := make([]byte, size) + if _, err := io.ReadFull(reader, buf); err != nil { + return nil, fmt.Errorf("failed to read random bytes: %v", err) + } + return buf, nil +} + + +const uuidLen = 16 + +// GenerateUUID is used to generate a random UUID +func GenerateUUID() (string, error) { + return GenerateUUIDWithReader(rand.Reader) +} + +// GenerateUUIDWithReader is used to generate a random UUID with a given Reader +func GenerateUUIDWithReader(reader io.Reader) (string, error) { + if reader == nil { + return "", fmt.Errorf("provided reader is nil") + } + buf, err := GenerateRandomBytesWithReader(uuidLen, reader) + if err != nil { + return "", err + } + return FormatUUID(buf) +} + +func FormatUUID(buf []byte) (string, error) { + if buflen := len(buf); buflen != uuidLen { + return "", fmt.Errorf("wrong length byte slice (%d)", buflen) + } + + return fmt.Sprintf("%x-%x-%x-%x-%x", + buf[0:4], + buf[4:6], + buf[6:8], + buf[8:10], + buf[10:16]), nil +} + +func ParseUUID(uuid string) ([]byte, error) { + if len(uuid) != 2 * uuidLen + 4 { + return nil, fmt.Errorf("uuid string is wrong length") + } + + if uuid[8] != '-' || + uuid[13] != '-' || + uuid[18] != '-' || + uuid[23] != '-' { + return nil, fmt.Errorf("uuid is improperly formatted") + } + + hexStr := uuid[0:8] + uuid[9:13] + uuid[14:18] + uuid[19:23] + uuid[24:36] + + ret, err := hex.DecodeString(hexStr) + if err != nil { + return nil, err + } + if len(ret) != uuidLen { + return nil, fmt.Errorf("decoded hex is the wrong length") + } + + return ret, nil +} diff --git a/vendor/github.com/jcmturner/aescts/v2/LICENSE b/vendor/github.com/jcmturner/aescts/v2/LICENSE new file mode 100644 index 000000000000..8dada3edaf50 --- /dev/null +++ b/vendor/github.com/jcmturner/aescts/v2/LICENSE @@ -0,0 +1,201 @@ + 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. + + END OF TERMS AND CONDITIONS + + APPENDIX: How to apply the Apache License to your work. + + To apply the Apache License to your work, attach the following + boilerplate notice, with the fields enclosed by brackets "{}" + replaced with your own identifying information. (Don't include + the brackets!) The text should be enclosed in the appropriate + comment syntax for the file format. We also recommend that a + file or class name and description of purpose be included on the + same "printed page" as the copyright notice for easier + identification within third-party archives. + + Copyright {yyyy} {name of copyright owner} + + 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. diff --git a/vendor/github.com/jcmturner/aescts/v2/aescts.go b/vendor/github.com/jcmturner/aescts/v2/aescts.go new file mode 100644 index 000000000000..fee3b43e91ef --- /dev/null +++ b/vendor/github.com/jcmturner/aescts/v2/aescts.go @@ -0,0 +1,186 @@ +// Package aescts provides AES CBC CipherText Stealing encryption and decryption methods +package aescts + +import ( + "crypto/aes" + "crypto/cipher" + "errors" + "fmt" +) + +// Encrypt the message with the key and the initial vector. +// Returns: next iv, ciphertext bytes, error +func Encrypt(key, iv, plaintext []byte) ([]byte, []byte, error) { + l := len(plaintext) + + block, err := aes.NewCipher(key) + if err != nil { + return []byte{}, []byte{}, fmt.Errorf("error creating cipher: %v", err) + } + mode := cipher.NewCBCEncrypter(block, iv) + + m := make([]byte, len(plaintext)) + copy(m, plaintext) + + /*For consistency, ciphertext stealing is always used for the last two + blocks of the data to be encrypted, as in [RC5]. If the data length + is a multiple of the block size, this is equivalent to plain CBC mode + with the last two ciphertext blocks swapped.*/ + /*The initial vector carried out from one encryption for use in a + subsequent encryption is the next-to-last block of the encryption + output; this is the encrypted form of the last plaintext block.*/ + if l <= aes.BlockSize { + m, _ = zeroPad(m, aes.BlockSize) + mode.CryptBlocks(m, m) + return m, m, nil + } + if l%aes.BlockSize == 0 { + mode.CryptBlocks(m, m) + iv = m[len(m)-aes.BlockSize:] + rb, _ := swapLastTwoBlocks(m, aes.BlockSize) + return iv, rb, nil + } + m, _ = zeroPad(m, aes.BlockSize) + rb, pb, lb, err := tailBlocks(m, aes.BlockSize) + if err != nil { + return []byte{}, []byte{}, fmt.Errorf("error tailing blocks: %v", err) + } + var ct []byte + if rb != nil { + // Encrpt all but the lats 2 blocks and update the rolling iv + mode.CryptBlocks(rb, rb) + iv = rb[len(rb)-aes.BlockSize:] + mode = cipher.NewCBCEncrypter(block, iv) + ct = append(ct, rb...) + } + mode.CryptBlocks(pb, pb) + mode = cipher.NewCBCEncrypter(block, pb) + mode.CryptBlocks(lb, lb) + // Cipher Text Stealing (CTS) - Ref: https://en.wikipedia.org/wiki/Ciphertext_stealing#CBC_ciphertext_stealing + // Swap the last two cipher blocks + // Truncate the ciphertext to the length of the original plaintext + ct = append(ct, lb...) + ct = append(ct, pb...) + return lb, ct[:l], nil +} + +// Decrypt the ciphertext with the key and the initial vector. +func Decrypt(key, iv, ciphertext []byte) ([]byte, error) { + // Copy the cipher text as golang slices even when passed by value to this method can result in the backing arrays of the calling code value being updated. + ct := make([]byte, len(ciphertext)) + copy(ct, ciphertext) + if len(ct) < aes.BlockSize { + return []byte{}, fmt.Errorf("ciphertext is not large enough. It is less that one block size. Blocksize:%v; Ciphertext:%v", aes.BlockSize, len(ct)) + } + // Configure the CBC + block, err := aes.NewCipher(key) + if err != nil { + return nil, fmt.Errorf("error creating cipher: %v", err) + } + var mode cipher.BlockMode + + //If ciphertext is multiple of blocksize we just need to swap back the last two blocks and then do CBC + //If the ciphertext is just one block we can't swap so we just decrypt + if len(ct)%aes.BlockSize == 0 { + if len(ct) > aes.BlockSize { + ct, _ = swapLastTwoBlocks(ct, aes.BlockSize) + } + mode = cipher.NewCBCDecrypter(block, iv) + message := make([]byte, len(ct)) + mode.CryptBlocks(message, ct) + return message[:len(ct)], nil + } + + // Cipher Text Stealing (CTS) using CBC interface. Ref: https://en.wikipedia.org/wiki/Ciphertext_stealing#CBC_ciphertext_stealing + // Get ciphertext of the 2nd to last (penultimate) block (cpb), the last block (clb) and the rest (crb) + crb, cpb, clb, _ := tailBlocks(ct, aes.BlockSize) + v := make([]byte, len(iv), len(iv)) + copy(v, iv) + var message []byte + if crb != nil { + //If there is more than just the last and the penultimate block we decrypt it and the last bloc of this becomes the iv for later + rb := make([]byte, len(crb)) + mode = cipher.NewCBCDecrypter(block, v) + v = crb[len(crb)-aes.BlockSize:] + mode.CryptBlocks(rb, crb) + message = append(message, rb...) + } + + // We need to modify the cipher text + // Decryt the 2nd to last (penultimate) block with a the original iv + pb := make([]byte, aes.BlockSize) + mode = cipher.NewCBCDecrypter(block, iv) + mode.CryptBlocks(pb, cpb) + // number of byte needed to pad + npb := aes.BlockSize - len(ct)%aes.BlockSize + //pad last block using the number of bytes needed from the tail of the plaintext 2nd to last (penultimate) block + clb = append(clb, pb[len(pb)-npb:]...) + + // Now decrypt the last block in the penultimate position (iv will be from the crb, if the is no crb it's zeros) + // iv for the penultimate block decrypted in the last position becomes the modified last block + lb := make([]byte, aes.BlockSize) + mode = cipher.NewCBCDecrypter(block, v) + v = clb + mode.CryptBlocks(lb, clb) + message = append(message, lb...) + + // Now decrypt the penultimate block in the last position (iv will be from the modified last block) + mode = cipher.NewCBCDecrypter(block, v) + mode.CryptBlocks(cpb, cpb) + message = append(message, cpb...) + + // Truncate to the size of the original cipher text + return message[:len(ct)], nil +} + +func tailBlocks(b []byte, c int) ([]byte, []byte, []byte, error) { + if len(b) <= c { + return []byte{}, []byte{}, []byte{}, errors.New("bytes slice is not larger than one block so cannot tail") + } + // Get size of last block + var lbs int + if l := len(b) % aes.BlockSize; l == 0 { + lbs = aes.BlockSize + } else { + lbs = l + } + // Get last block + lb := b[len(b)-lbs:] + // Get 2nd to last (penultimate) block + pb := b[len(b)-lbs-c : len(b)-lbs] + if len(b) > 2*c { + rb := b[:len(b)-lbs-c] + return rb, pb, lb, nil + } + return nil, pb, lb, nil +} + +func swapLastTwoBlocks(b []byte, c int) ([]byte, error) { + rb, pb, lb, err := tailBlocks(b, c) + if err != nil { + return nil, err + } + var out []byte + if rb != nil { + out = append(out, rb...) + } + out = append(out, lb...) + out = append(out, pb...) + return out, nil +} + +// zeroPad pads bytes with zeros to nearest multiple of message size m. +func zeroPad(b []byte, m int) ([]byte, error) { + if m <= 0 { + return nil, errors.New("invalid message block size when padding") + } + if b == nil || len(b) == 0 { + return nil, errors.New("data not valid to pad: Zero size") + } + if l := len(b) % m; l != 0 { + n := m - l + z := make([]byte, n) + b = append(b, z...) + } + return b, nil +} diff --git a/vendor/github.com/jcmturner/dnsutils/v2/LICENSE b/vendor/github.com/jcmturner/dnsutils/v2/LICENSE new file mode 100644 index 000000000000..261eeb9e9f8b --- /dev/null +++ b/vendor/github.com/jcmturner/dnsutils/v2/LICENSE @@ -0,0 +1,201 @@ + 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. + + END OF TERMS AND CONDITIONS + + APPENDIX: How to apply the Apache License to your work. + + To apply the Apache License to your work, attach the following + boilerplate notice, with the fields enclosed by brackets "[]" + replaced with your own identifying information. (Don't include + the brackets!) The text should be enclosed in the appropriate + comment syntax for the file format. We also recommend that a + file or class name and description of purpose be included on the + same "printed page" as the copyright notice for easier + identification within third-party archives. + + Copyright [yyyy] [name of copyright owner] + + 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. diff --git a/vendor/github.com/jcmturner/dnsutils/v2/srv.go b/vendor/github.com/jcmturner/dnsutils/v2/srv.go new file mode 100644 index 000000000000..15ea912d100d --- /dev/null +++ b/vendor/github.com/jcmturner/dnsutils/v2/srv.go @@ -0,0 +1,95 @@ +package dnsutils + +import ( + "math/rand" + "net" + "sort" +) + +// OrderedSRV returns a count of the results and a map keyed on the order they should be used. +// This based on the records' priority and randomised selection based on their relative weighting. +// The function's inputs are the same as those for net.LookupSRV +// To use in the correct order: +// +// count, orderedSRV, err := OrderedSRV(service, proto, name) +// i := 1 +// for i <= count { +// srv := orderedSRV[i] +// // Do something such as dial this SRV. If fails move on the the next or break if it succeeds. +// i += 1 +// } +func OrderedSRV(service, proto, name string) (int, map[int]*net.SRV, error) { + _, addrs, err := net.LookupSRV(service, proto, name) + if err != nil { + return 0, make(map[int]*net.SRV), err + } + index, osrv := orderSRV(addrs) + return index, osrv, nil +} + +func orderSRV(addrs []*net.SRV) (int, map[int]*net.SRV) { + // Initialise the ordered map + var o int + osrv := make(map[int]*net.SRV) + + prioMap := make(map[int][]*net.SRV, 0) + for _, srv := range addrs { + prioMap[int(srv.Priority)] = append(prioMap[int(srv.Priority)], srv) + } + + priorities := make([]int, 0) + for p := range prioMap { + priorities = append(priorities, p) + } + + var count int + sort.Ints(priorities) + for _, p := range priorities { + tos := weightedOrder(prioMap[p]) + for i, s := range tos { + count += 1 + osrv[o+i] = s + } + o += len(tos) + } + return count, osrv +} + +func weightedOrder(srvs []*net.SRV) map[int]*net.SRV { + // Get the total weight + var tw int + for _, s := range srvs { + tw += int(s.Weight) + } + + // Initialise the ordered map + o := 1 + osrv := make(map[int]*net.SRV) + + // Whilst there are still entries to be ordered + l := len(srvs) + for l > 0 { + i := rand.Intn(l) + s := srvs[i] + var rw int + if tw > 0 { + // Greater the weight the more likely this will be zero or less + rw = rand.Intn(tw) - int(s.Weight) + } + if rw <= 0 { + // Put entry in position + osrv[o] = s + if len(srvs) > 1 { + // Remove the entry from the source slice by swapping with the last entry and truncating + srvs[len(srvs)-1], srvs[i] = srvs[i], srvs[len(srvs)-1] + srvs = srvs[:len(srvs)-1] + l = len(srvs) + } else { + l = 0 + } + o += 1 + tw = tw - int(s.Weight) + } + } + return osrv +} diff --git a/vendor/github.com/jcmturner/gofork/LICENSE b/vendor/github.com/jcmturner/gofork/LICENSE new file mode 100644 index 000000000000..6a66aea5eafe --- /dev/null +++ b/vendor/github.com/jcmturner/gofork/LICENSE @@ -0,0 +1,27 @@ +Copyright (c) 2009 The Go Authors. All rights reserved. + +Redistribution and use in source and binary forms, with or without +modification, are permitted provided that the following conditions are +met: + + * Redistributions of source code must retain the above copyright +notice, this list of conditions and the following disclaimer. + * Redistributions in binary form must reproduce the above +copyright notice, this list of conditions and the following disclaimer +in the documentation and/or other materials provided with the +distribution. + * Neither the name of Google Inc. nor the names of its +contributors may be used to endorse or promote products derived from +this software without specific prior written permission. + +THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS +"AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT +LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR +A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT +OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, +SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT +LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, +DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY +THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT +(INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE +OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. diff --git a/vendor/github.com/jcmturner/gofork/encoding/asn1/README.md b/vendor/github.com/jcmturner/gofork/encoding/asn1/README.md new file mode 100644 index 000000000000..66a2a8cca71b --- /dev/null +++ b/vendor/github.com/jcmturner/gofork/encoding/asn1/README.md @@ -0,0 +1,5 @@ +This is a temporary repository that will be removed when the issues below are fixed in the core golang code. + +## Issues +* [encoding/asn1: cannot marshal into a GeneralString](https://github.com/golang/go/issues/18832) +* [encoding/asn1: cannot marshal into slice of strings and pass stringtype parameter tags to members](https://github.com/golang/go/issues/18834) \ No newline at end of file diff --git a/vendor/github.com/jcmturner/gofork/encoding/asn1/asn1.go b/vendor/github.com/jcmturner/gofork/encoding/asn1/asn1.go new file mode 100644 index 000000000000..f1bb7671795e --- /dev/null +++ b/vendor/github.com/jcmturner/gofork/encoding/asn1/asn1.go @@ -0,0 +1,1003 @@ +// Copyright 2009 The Go Authors. All rights reserved. +// Use of this source code is governed by a BSD-style +// license that can be found in the LICENSE file. + +// Package asn1 implements parsing of DER-encoded ASN.1 data structures, +// as defined in ITU-T Rec X.690. +// +// See also ``A Layman's Guide to a Subset of ASN.1, BER, and DER,'' +// http://luca.ntop.org/Teaching/Appunti/asn1.html. +package asn1 + +// ASN.1 is a syntax for specifying abstract objects and BER, DER, PER, XER etc +// are different encoding formats for those objects. Here, we'll be dealing +// with DER, the Distinguished Encoding Rules. DER is used in X.509 because +// it's fast to parse and, unlike BER, has a unique encoding for every object. +// When calculating hashes over objects, it's important that the resulting +// bytes be the same at both ends and DER removes this margin of error. +// +// ASN.1 is very complex and this package doesn't attempt to implement +// everything by any means. + +import ( + "errors" + "fmt" + "math/big" + "reflect" + "strconv" + "time" + "unicode/utf8" +) + +// A StructuralError suggests that the ASN.1 data is valid, but the Go type +// which is receiving it doesn't match. +type StructuralError struct { + Msg string +} + +func (e StructuralError) Error() string { return "asn1: structure error: " + e.Msg } + +// A SyntaxError suggests that the ASN.1 data is invalid. +type SyntaxError struct { + Msg string +} + +func (e SyntaxError) Error() string { return "asn1: syntax error: " + e.Msg } + +// We start by dealing with each of the primitive types in turn. + +// BOOLEAN + +func parseBool(bytes []byte) (ret bool, err error) { + if len(bytes) != 1 { + err = SyntaxError{"invalid boolean"} + return + } + + // DER demands that "If the encoding represents the boolean value TRUE, + // its single contents octet shall have all eight bits set to one." + // Thus only 0 and 255 are valid encoded values. + switch bytes[0] { + case 0: + ret = false + case 0xff: + ret = true + default: + err = SyntaxError{"invalid boolean"} + } + + return +} + +// INTEGER + +// checkInteger returns nil if the given bytes are a valid DER-encoded +// INTEGER and an error otherwise. +func checkInteger(bytes []byte) error { + if len(bytes) == 0 { + return StructuralError{"empty integer"} + } + if len(bytes) == 1 { + return nil + } + if (bytes[0] == 0 && bytes[1]&0x80 == 0) || (bytes[0] == 0xff && bytes[1]&0x80 == 0x80) { + return StructuralError{"integer not minimally-encoded"} + } + return nil +} + +// parseInt64 treats the given bytes as a big-endian, signed integer and +// returns the result. +func parseInt64(bytes []byte) (ret int64, err error) { + err = checkInteger(bytes) + if err != nil { + return + } + if len(bytes) > 8 { + // We'll overflow an int64 in this case. + err = StructuralError{"integer too large"} + return + } + for bytesRead := 0; bytesRead < len(bytes); bytesRead++ { + ret <<= 8 + ret |= int64(bytes[bytesRead]) + } + + // Shift up and down in order to sign extend the result. + ret <<= 64 - uint8(len(bytes))*8 + ret >>= 64 - uint8(len(bytes))*8 + return +} + +// parseInt treats the given bytes as a big-endian, signed integer and returns +// the result. +func parseInt32(bytes []byte) (int32, error) { + if err := checkInteger(bytes); err != nil { + return 0, err + } + ret64, err := parseInt64(bytes) + if err != nil { + return 0, err + } + if ret64 != int64(int32(ret64)) { + return 0, StructuralError{"integer too large"} + } + return int32(ret64), nil +} + +var bigOne = big.NewInt(1) + +// parseBigInt treats the given bytes as a big-endian, signed integer and returns +// the result. +func parseBigInt(bytes []byte) (*big.Int, error) { + if err := checkInteger(bytes); err != nil { + return nil, err + } + ret := new(big.Int) + if len(bytes) > 0 && bytes[0]&0x80 == 0x80 { + // This is a negative number. + notBytes := make([]byte, len(bytes)) + for i := range notBytes { + notBytes[i] = ^bytes[i] + } + ret.SetBytes(notBytes) + ret.Add(ret, bigOne) + ret.Neg(ret) + return ret, nil + } + ret.SetBytes(bytes) + return ret, nil +} + +// BIT STRING + +// BitString is the structure to use when you want an ASN.1 BIT STRING type. A +// bit string is padded up to the nearest byte in memory and the number of +// valid bits is recorded. Padding bits will be zero. +type BitString struct { + Bytes []byte // bits packed into bytes. + BitLength int // length in bits. +} + +// At returns the bit at the given index. If the index is out of range it +// returns false. +func (b BitString) At(i int) int { + if i < 0 || i >= b.BitLength { + return 0 + } + x := i / 8 + y := 7 - uint(i%8) + return int(b.Bytes[x]>>y) & 1 +} + +// RightAlign returns a slice where the padding bits are at the beginning. The +// slice may share memory with the BitString. +func (b BitString) RightAlign() []byte { + shift := uint(8 - (b.BitLength % 8)) + if shift == 8 || len(b.Bytes) == 0 { + return b.Bytes + } + + a := make([]byte, len(b.Bytes)) + a[0] = b.Bytes[0] >> shift + for i := 1; i < len(b.Bytes); i++ { + a[i] = b.Bytes[i-1] << (8 - shift) + a[i] |= b.Bytes[i] >> shift + } + + return a +} + +// parseBitString parses an ASN.1 bit string from the given byte slice and returns it. +func parseBitString(bytes []byte) (ret BitString, err error) { + if len(bytes) == 0 { + err = SyntaxError{"zero length BIT STRING"} + return + } + paddingBits := int(bytes[0]) + if paddingBits > 7 || + len(bytes) == 1 && paddingBits > 0 || + bytes[len(bytes)-1]&((1< 0 { + s += "." + } + s += strconv.Itoa(v) + } + + return s +} + +// parseObjectIdentifier parses an OBJECT IDENTIFIER from the given bytes and +// returns it. An object identifier is a sequence of variable length integers +// that are assigned in a hierarchy. +func parseObjectIdentifier(bytes []byte) (s []int, err error) { + if len(bytes) == 0 { + err = SyntaxError{"zero length OBJECT IDENTIFIER"} + return + } + + // In the worst case, we get two elements from the first byte (which is + // encoded differently) and then every varint is a single byte long. + s = make([]int, len(bytes)+1) + + // The first varint is 40*value1 + value2: + // According to this packing, value1 can take the values 0, 1 and 2 only. + // When value1 = 0 or value1 = 1, then value2 is <= 39. When value1 = 2, + // then there are no restrictions on value2. + v, offset, err := parseBase128Int(bytes, 0) + if err != nil { + return + } + if v < 80 { + s[0] = v / 40 + s[1] = v % 40 + } else { + s[0] = 2 + s[1] = v - 80 + } + + i := 2 + for ; offset < len(bytes); i++ { + v, offset, err = parseBase128Int(bytes, offset) + if err != nil { + return + } + s[i] = v + } + s = s[0:i] + return +} + +// ENUMERATED + +// An Enumerated is represented as a plain int. +type Enumerated int + +// FLAG + +// A Flag accepts any data and is set to true if present. +type Flag bool + +// parseBase128Int parses a base-128 encoded int from the given offset in the +// given byte slice. It returns the value and the new offset. +func parseBase128Int(bytes []byte, initOffset int) (ret, offset int, err error) { + offset = initOffset + for shifted := 0; offset < len(bytes); shifted++ { + if shifted == 4 { + err = StructuralError{"base 128 integer too large"} + return + } + ret <<= 7 + b := bytes[offset] + ret |= int(b & 0x7f) + offset++ + if b&0x80 == 0 { + return + } + } + err = SyntaxError{"truncated base 128 integer"} + return +} + +// UTCTime + +func parseUTCTime(bytes []byte) (ret time.Time, err error) { + s := string(bytes) + + formatStr := "0601021504Z0700" + ret, err = time.Parse(formatStr, s) + if err != nil { + formatStr = "060102150405Z0700" + ret, err = time.Parse(formatStr, s) + } + if err != nil { + return + } + + if serialized := ret.Format(formatStr); serialized != s { + err = fmt.Errorf("asn1: time did not serialize back to the original value and may be invalid: given %q, but serialized as %q", s, serialized) + return + } + + if ret.Year() >= 2050 { + // UTCTime only encodes times prior to 2050. See https://tools.ietf.org/html/rfc5280#section-4.1.2.5.1 + ret = ret.AddDate(-100, 0, 0) + } + + return +} + +// parseGeneralizedTime parses the GeneralizedTime from the given byte slice +// and returns the resulting time. +func parseGeneralizedTime(bytes []byte) (ret time.Time, err error) { + const formatStr = "20060102150405Z0700" + s := string(bytes) + + if ret, err = time.Parse(formatStr, s); err != nil { + return + } + + if serialized := ret.Format(formatStr); serialized != s { + err = fmt.Errorf("asn1: time did not serialize back to the original value and may be invalid: given %q, but serialized as %q", s, serialized) + } + + return +} + +// PrintableString + +// parsePrintableString parses a ASN.1 PrintableString from the given byte +// array and returns it. +func parsePrintableString(bytes []byte) (ret string, err error) { + for _, b := range bytes { + if !isPrintable(b) { + err = SyntaxError{"PrintableString contains invalid character"} + return + } + } + ret = string(bytes) + return +} + +// isPrintable reports whether the given b is in the ASN.1 PrintableString set. +func isPrintable(b byte) bool { + return 'a' <= b && b <= 'z' || + 'A' <= b && b <= 'Z' || + '0' <= b && b <= '9' || + '\'' <= b && b <= ')' || + '+' <= b && b <= '/' || + b == ' ' || + b == ':' || + b == '=' || + b == '?' || + // This is technically not allowed in a PrintableString. + // However, x509 certificates with wildcard strings don't + // always use the correct string type so we permit it. + b == '*' +} + +// IA5String + +// parseIA5String parses a ASN.1 IA5String (ASCII string) from the given +// byte slice and returns it. +func parseIA5String(bytes []byte) (ret string, err error) { + for _, b := range bytes { + if b >= utf8.RuneSelf { + err = SyntaxError{"IA5String contains invalid character"} + return + } + } + ret = string(bytes) + return +} + +// T61String + +// parseT61String parses a ASN.1 T61String (8-bit clean string) from the given +// byte slice and returns it. +func parseT61String(bytes []byte) (ret string, err error) { + return string(bytes), nil +} + +// UTF8String + +// parseUTF8String parses a ASN.1 UTF8String (raw UTF-8) from the given byte +// array and returns it. +func parseUTF8String(bytes []byte) (ret string, err error) { + if !utf8.Valid(bytes) { + return "", errors.New("asn1: invalid UTF-8 string") + } + return string(bytes), nil +} + +// A RawValue represents an undecoded ASN.1 object. +type RawValue struct { + Class, Tag int + IsCompound bool + Bytes []byte + FullBytes []byte // includes the tag and length +} + +// RawContent is used to signal that the undecoded, DER data needs to be +// preserved for a struct. To use it, the first field of the struct must have +// this type. It's an error for any of the other fields to have this type. +type RawContent []byte + +// Tagging + +// parseTagAndLength parses an ASN.1 tag and length pair from the given offset +// into a byte slice. It returns the parsed data and the new offset. SET and +// SET OF (tag 17) are mapped to SEQUENCE and SEQUENCE OF (tag 16) since we +// don't distinguish between ordered and unordered objects in this code. +func parseTagAndLength(bytes []byte, initOffset int) (ret tagAndLength, offset int, err error) { + offset = initOffset + // parseTagAndLength should not be called without at least a single + // byte to read. Thus this check is for robustness: + if offset >= len(bytes) { + err = errors.New("asn1: internal error in parseTagAndLength") + return + } + b := bytes[offset] + offset++ + ret.class = int(b >> 6) + ret.isCompound = b&0x20 == 0x20 + ret.tag = int(b & 0x1f) + + // If the bottom five bits are set, then the tag number is actually base 128 + // encoded afterwards + if ret.tag == 0x1f { + ret.tag, offset, err = parseBase128Int(bytes, offset) + if err != nil { + return + } + // Tags should be encoded in minimal form. + if ret.tag < 0x1f { + err = SyntaxError{"non-minimal tag"} + return + } + } + if offset >= len(bytes) { + err = SyntaxError{"truncated tag or length"} + return + } + b = bytes[offset] + offset++ + if b&0x80 == 0 { + // The length is encoded in the bottom 7 bits. + ret.length = int(b & 0x7f) + } else { + // Bottom 7 bits give the number of length bytes to follow. + numBytes := int(b & 0x7f) + if numBytes == 0 { + err = SyntaxError{"indefinite length found (not DER)"} + return + } + ret.length = 0 + for i := 0; i < numBytes; i++ { + if offset >= len(bytes) { + err = SyntaxError{"truncated tag or length"} + return + } + b = bytes[offset] + offset++ + if ret.length >= 1<<23 { + // We can't shift ret.length up without + // overflowing. + err = StructuralError{"length too large"} + return + } + ret.length <<= 8 + ret.length |= int(b) + if ret.length == 0 { + // DER requires that lengths be minimal. + err = StructuralError{"superfluous leading zeros in length"} + return + } + } + // Short lengths must be encoded in short form. + if ret.length < 0x80 { + err = StructuralError{"non-minimal length"} + return + } + } + + return +} + +// parseSequenceOf is used for SEQUENCE OF and SET OF values. It tries to parse +// a number of ASN.1 values from the given byte slice and returns them as a +// slice of Go values of the given type. +func parseSequenceOf(bytes []byte, sliceType reflect.Type, elemType reflect.Type) (ret reflect.Value, err error) { + expectedTag, compoundType, ok := getUniversalType(elemType) + if !ok { + err = StructuralError{"unknown Go type for slice"} + return + } + + // First we iterate over the input and count the number of elements, + // checking that the types are correct in each case. + numElements := 0 + for offset := 0; offset < len(bytes); { + var t tagAndLength + t, offset, err = parseTagAndLength(bytes, offset) + if err != nil { + return + } + switch t.tag { + case TagIA5String, TagGeneralString, TagT61String, TagUTF8String: + // We pretend that various other string types are + // PRINTABLE STRINGs so that a sequence of them can be + // parsed into a []string. + t.tag = TagPrintableString + case TagGeneralizedTime, TagUTCTime: + // Likewise, both time types are treated the same. + t.tag = TagUTCTime + } + + if t.class != ClassUniversal || t.isCompound != compoundType || t.tag != expectedTag { + err = StructuralError{"sequence tag mismatch"} + return + } + if invalidLength(offset, t.length, len(bytes)) { + err = SyntaxError{"truncated sequence"} + return + } + offset += t.length + numElements++ + } + ret = reflect.MakeSlice(sliceType, numElements, numElements) + params := fieldParameters{} + offset := 0 + for i := 0; i < numElements; i++ { + offset, err = parseField(ret.Index(i), bytes, offset, params) + if err != nil { + return + } + } + return +} + +var ( + bitStringType = reflect.TypeOf(BitString{}) + objectIdentifierType = reflect.TypeOf(ObjectIdentifier{}) + enumeratedType = reflect.TypeOf(Enumerated(0)) + flagType = reflect.TypeOf(Flag(false)) + timeType = reflect.TypeOf(time.Time{}) + rawValueType = reflect.TypeOf(RawValue{}) + rawContentsType = reflect.TypeOf(RawContent(nil)) + bigIntType = reflect.TypeOf(new(big.Int)) +) + +// invalidLength returns true iff offset + length > sliceLength, or if the +// addition would overflow. +func invalidLength(offset, length, sliceLength int) bool { + return offset+length < offset || offset+length > sliceLength +} + +// parseField is the main parsing function. Given a byte slice and an offset +// into the array, it will try to parse a suitable ASN.1 value out and store it +// in the given Value. +func parseField(v reflect.Value, bytes []byte, initOffset int, params fieldParameters) (offset int, err error) { + offset = initOffset + fieldType := v.Type() + + // If we have run out of data, it may be that there are optional elements at the end. + if offset == len(bytes) { + if !setDefaultValue(v, params) { + err = SyntaxError{"sequence truncated"} + } + return + } + + // Deal with raw values. + if fieldType == rawValueType { + var t tagAndLength + t, offset, err = parseTagAndLength(bytes, offset) + if err != nil { + return + } + if invalidLength(offset, t.length, len(bytes)) { + err = SyntaxError{"data truncated"} + return + } + result := RawValue{t.class, t.tag, t.isCompound, bytes[offset : offset+t.length], bytes[initOffset : offset+t.length]} + offset += t.length + v.Set(reflect.ValueOf(result)) + return + } + + // Deal with the ANY type. + if ifaceType := fieldType; ifaceType.Kind() == reflect.Interface && ifaceType.NumMethod() == 0 { + var t tagAndLength + t, offset, err = parseTagAndLength(bytes, offset) + if err != nil { + return + } + if invalidLength(offset, t.length, len(bytes)) { + err = SyntaxError{"data truncated"} + return + } + var result interface{} + if !t.isCompound && t.class == ClassUniversal { + innerBytes := bytes[offset : offset+t.length] + switch t.tag { + case TagPrintableString: + result, err = parsePrintableString(innerBytes) + case TagIA5String: + result, err = parseIA5String(innerBytes) + // jtasn1 addition of following case + case TagGeneralString: + result, err = parseIA5String(innerBytes) + case TagT61String: + result, err = parseT61String(innerBytes) + case TagUTF8String: + result, err = parseUTF8String(innerBytes) + case TagInteger: + result, err = parseInt64(innerBytes) + case TagBitString: + result, err = parseBitString(innerBytes) + case TagOID: + result, err = parseObjectIdentifier(innerBytes) + case TagUTCTime: + result, err = parseUTCTime(innerBytes) + case TagGeneralizedTime: + result, err = parseGeneralizedTime(innerBytes) + case TagOctetString: + result = innerBytes + default: + // If we don't know how to handle the type, we just leave Value as nil. + } + } + offset += t.length + if err != nil { + return + } + if result != nil { + v.Set(reflect.ValueOf(result)) + } + return + } + universalTag, compoundType, ok1 := getUniversalType(fieldType) + if !ok1 { + err = StructuralError{fmt.Sprintf("unknown Go type: %v", fieldType)} + return + } + + t, offset, err := parseTagAndLength(bytes, offset) + if err != nil { + return + } + if params.explicit { + expectedClass := ClassContextSpecific + if params.application { + expectedClass = ClassApplication + } + if offset == len(bytes) { + err = StructuralError{"explicit tag has no child"} + return + } + if t.class == expectedClass && t.tag == *params.tag && (t.length == 0 || t.isCompound) { + if t.length > 0 { + t, offset, err = parseTagAndLength(bytes, offset) + if err != nil { + return + } + } else { + if fieldType != flagType { + err = StructuralError{"zero length explicit tag was not an asn1.Flag"} + return + } + v.SetBool(true) + return + } + } else { + // The tags didn't match, it might be an optional element. + ok := setDefaultValue(v, params) + if ok { + offset = initOffset + } else { + err = StructuralError{"explicitly tagged member didn't match"} + } + return + } + } + + // Special case for strings: all the ASN.1 string types map to the Go + // type string. getUniversalType returns the tag for PrintableString + // when it sees a string, so if we see a different string type on the + // wire, we change the universal type to match. + if universalTag == TagPrintableString { + if t.class == ClassUniversal { + switch t.tag { + case TagIA5String, TagGeneralString, TagT61String, TagUTF8String: + universalTag = t.tag + } + } else if params.stringType != 0 { + universalTag = params.stringType + } + } + + // Special case for time: UTCTime and GeneralizedTime both map to the + // Go type time.Time. + if universalTag == TagUTCTime && t.tag == TagGeneralizedTime && t.class == ClassUniversal { + universalTag = TagGeneralizedTime + } + + if params.set { + universalTag = TagSet + } + + expectedClass := ClassUniversal + expectedTag := universalTag + + if !params.explicit && params.tag != nil { + expectedClass = ClassContextSpecific + expectedTag = *params.tag + } + + if !params.explicit && params.application && params.tag != nil { + expectedClass = ClassApplication + expectedTag = *params.tag + } + + // We have unwrapped any explicit tagging at this point. + if t.class != expectedClass || t.tag != expectedTag || t.isCompound != compoundType { + // Tags don't match. Again, it could be an optional element. + ok := setDefaultValue(v, params) + if ok { + offset = initOffset + } else { + err = StructuralError{fmt.Sprintf("tags don't match (%d vs %+v) %+v %s @%d", expectedTag, t, params, fieldType.Name(), offset)} + } + return + } + if invalidLength(offset, t.length, len(bytes)) { + err = SyntaxError{"data truncated"} + return + } + innerBytes := bytes[offset : offset+t.length] + offset += t.length + + // We deal with the structures defined in this package first. + switch fieldType { + case objectIdentifierType: + newSlice, err1 := parseObjectIdentifier(innerBytes) + v.Set(reflect.MakeSlice(v.Type(), len(newSlice), len(newSlice))) + if err1 == nil { + reflect.Copy(v, reflect.ValueOf(newSlice)) + } + err = err1 + return + case bitStringType: + bs, err1 := parseBitString(innerBytes) + if err1 == nil { + v.Set(reflect.ValueOf(bs)) + } + err = err1 + return + case timeType: + var time time.Time + var err1 error + if universalTag == TagUTCTime { + time, err1 = parseUTCTime(innerBytes) + } else { + time, err1 = parseGeneralizedTime(innerBytes) + } + if err1 == nil { + v.Set(reflect.ValueOf(time)) + } + err = err1 + return + case enumeratedType: + parsedInt, err1 := parseInt32(innerBytes) + if err1 == nil { + v.SetInt(int64(parsedInt)) + } + err = err1 + return + case flagType: + v.SetBool(true) + return + case bigIntType: + parsedInt, err1 := parseBigInt(innerBytes) + if err1 == nil { + v.Set(reflect.ValueOf(parsedInt)) + } + err = err1 + return + } + switch val := v; val.Kind() { + case reflect.Bool: + parsedBool, err1 := parseBool(innerBytes) + if err1 == nil { + val.SetBool(parsedBool) + } + err = err1 + return + case reflect.Int, reflect.Int32, reflect.Int64: + if val.Type().Size() == 4 { + parsedInt, err1 := parseInt32(innerBytes) + if err1 == nil { + val.SetInt(int64(parsedInt)) + } + err = err1 + } else { + parsedInt, err1 := parseInt64(innerBytes) + if err1 == nil { + val.SetInt(parsedInt) + } + err = err1 + } + return + // TODO(dfc) Add support for the remaining integer types + case reflect.Struct: + structType := fieldType + + if structType.NumField() > 0 && + structType.Field(0).Type == rawContentsType { + bytes := bytes[initOffset:offset] + val.Field(0).Set(reflect.ValueOf(RawContent(bytes))) + } + + innerOffset := 0 + for i := 0; i < structType.NumField(); i++ { + field := structType.Field(i) + if i == 0 && field.Type == rawContentsType { + continue + } + innerOffset, err = parseField(val.Field(i), innerBytes, innerOffset, parseFieldParameters(field.Tag.Get("asn1"))) + if err != nil { + return + } + } + // We allow extra bytes at the end of the SEQUENCE because + // adding elements to the end has been used in X.509 as the + // version numbers have increased. + return + case reflect.Slice: + sliceType := fieldType + if sliceType.Elem().Kind() == reflect.Uint8 { + val.Set(reflect.MakeSlice(sliceType, len(innerBytes), len(innerBytes))) + reflect.Copy(val, reflect.ValueOf(innerBytes)) + return + } + newSlice, err1 := parseSequenceOf(innerBytes, sliceType, sliceType.Elem()) + if err1 == nil { + val.Set(newSlice) + } + err = err1 + return + case reflect.String: + var v string + switch universalTag { + case TagPrintableString: + v, err = parsePrintableString(innerBytes) + case TagIA5String: + v, err = parseIA5String(innerBytes) + case TagT61String: + v, err = parseT61String(innerBytes) + case TagUTF8String: + v, err = parseUTF8String(innerBytes) + case TagGeneralString: + // GeneralString is specified in ISO-2022/ECMA-35, + // A brief review suggests that it includes structures + // that allow the encoding to change midstring and + // such. We give up and pass it as an 8-bit string. + v, err = parseT61String(innerBytes) + default: + err = SyntaxError{fmt.Sprintf("internal error: unknown string type %d", universalTag)} + } + if err == nil { + val.SetString(v) + } + return + } + err = StructuralError{"unsupported: " + v.Type().String()} + return +} + +// canHaveDefaultValue reports whether k is a Kind that we will set a default +// value for. (A signed integer, essentially.) +func canHaveDefaultValue(k reflect.Kind) bool { + switch k { + case reflect.Int, reflect.Int8, reflect.Int16, reflect.Int32, reflect.Int64: + return true + } + + return false +} + +// setDefaultValue is used to install a default value, from a tag string, into +// a Value. It is successful if the field was optional, even if a default value +// wasn't provided or it failed to install it into the Value. +func setDefaultValue(v reflect.Value, params fieldParameters) (ok bool) { + if !params.optional { + return + } + ok = true + if params.defaultValue == nil { + return + } + if canHaveDefaultValue(v.Kind()) { + v.SetInt(*params.defaultValue) + } + return +} + +// Unmarshal parses the DER-encoded ASN.1 data structure b +// and uses the reflect package to fill in an arbitrary value pointed at by val. +// Because Unmarshal uses the reflect package, the structs +// being written to must use upper case field names. +// +// An ASN.1 INTEGER can be written to an int, int32, int64, +// or *big.Int (from the math/big package). +// If the encoded value does not fit in the Go type, +// Unmarshal returns a parse error. +// +// An ASN.1 BIT STRING can be written to a BitString. +// +// An ASN.1 OCTET STRING can be written to a []byte. +// +// An ASN.1 OBJECT IDENTIFIER can be written to an +// ObjectIdentifier. +// +// An ASN.1 ENUMERATED can be written to an Enumerated. +// +// An ASN.1 UTCTIME or GENERALIZEDTIME can be written to a time.Time. +// +// An ASN.1 PrintableString or IA5String can be written to a string. +// +// Any of the above ASN.1 values can be written to an interface{}. +// The value stored in the interface has the corresponding Go type. +// For integers, that type is int64. +// +// An ASN.1 SEQUENCE OF x or SET OF x can be written +// to a slice if an x can be written to the slice's element type. +// +// An ASN.1 SEQUENCE or SET can be written to a struct +// if each of the elements in the sequence can be +// written to the corresponding element in the struct. +// +// The following tags on struct fields have special meaning to Unmarshal: +// +// application specifies that a APPLICATION tag is used +// default:x sets the default value for optional integer fields +// explicit specifies that an additional, explicit tag wraps the implicit one +// optional marks the field as ASN.1 OPTIONAL +// set causes a SET, rather than a SEQUENCE type to be expected +// tag:x specifies the ASN.1 tag number; implies ASN.1 CONTEXT SPECIFIC +// +// If the type of the first field of a structure is RawContent then the raw +// ASN1 contents of the struct will be stored in it. +// +// If the type name of a slice element ends with "SET" then it's treated as if +// the "set" tag was set on it. This can be used with nested slices where a +// struct tag cannot be given. +// +// Other ASN.1 types are not supported; if it encounters them, +// Unmarshal returns a parse error. +func Unmarshal(b []byte, val interface{}) (rest []byte, err error) { + return UnmarshalWithParams(b, val, "") +} + +// UnmarshalWithParams allows field parameters to be specified for the +// top-level element. The form of the params is the same as the field tags. +func UnmarshalWithParams(b []byte, val interface{}, params string) (rest []byte, err error) { + v := reflect.ValueOf(val).Elem() + offset, err := parseField(v, b, 0, parseFieldParameters(params)) + if err != nil { + return nil, err + } + return b[offset:], nil +} diff --git a/vendor/github.com/jcmturner/gofork/encoding/asn1/common.go b/vendor/github.com/jcmturner/gofork/encoding/asn1/common.go new file mode 100644 index 000000000000..7a9da49f396f --- /dev/null +++ b/vendor/github.com/jcmturner/gofork/encoding/asn1/common.go @@ -0,0 +1,173 @@ +// Copyright 2009 The Go Authors. All rights reserved. +// Use of this source code is governed by a BSD-style +// license that can be found in the LICENSE file. + +package asn1 + +import ( + "reflect" + "strconv" + "strings" +) + +// ASN.1 objects have metadata preceding them: +// the tag: the type of the object +// a flag denoting if this object is compound or not +// the class type: the namespace of the tag +// the length of the object, in bytes + +// Here are some standard tags and classes + +// ASN.1 tags represent the type of the following object. +const ( + TagBoolean = 1 + TagInteger = 2 + TagBitString = 3 + TagOctetString = 4 + TagOID = 6 + TagEnum = 10 + TagUTF8String = 12 + TagSequence = 16 + TagSet = 17 + TagPrintableString = 19 + TagT61String = 20 + TagIA5String = 22 + TagUTCTime = 23 + TagGeneralizedTime = 24 + TagGeneralString = 27 +) + +// ASN.1 class types represent the namespace of the tag. +const ( + ClassUniversal = 0 + ClassApplication = 1 + ClassContextSpecific = 2 + ClassPrivate = 3 +) + +type tagAndLength struct { + class, tag, length int + isCompound bool +} + +// ASN.1 has IMPLICIT and EXPLICIT tags, which can be translated as "instead +// of" and "in addition to". When not specified, every primitive type has a +// default tag in the UNIVERSAL class. +// +// For example: a BIT STRING is tagged [UNIVERSAL 3] by default (although ASN.1 +// doesn't actually have a UNIVERSAL keyword). However, by saying [IMPLICIT +// CONTEXT-SPECIFIC 42], that means that the tag is replaced by another. +// +// On the other hand, if it said [EXPLICIT CONTEXT-SPECIFIC 10], then an +// /additional/ tag would wrap the default tag. This explicit tag will have the +// compound flag set. +// +// (This is used in order to remove ambiguity with optional elements.) +// +// You can layer EXPLICIT and IMPLICIT tags to an arbitrary depth, however we +// don't support that here. We support a single layer of EXPLICIT or IMPLICIT +// tagging with tag strings on the fields of a structure. + +// fieldParameters is the parsed representation of tag string from a structure field. +type fieldParameters struct { + optional bool // true iff the field is OPTIONAL + explicit bool // true iff an EXPLICIT tag is in use. + application bool // true iff an APPLICATION tag is in use. + defaultValue *int64 // a default value for INTEGER typed fields (maybe nil). + tag *int // the EXPLICIT or IMPLICIT tag (maybe nil). + stringType int // the string tag to use when marshaling. + timeType int // the time tag to use when marshaling. + set bool // true iff this should be encoded as a SET + omitEmpty bool // true iff this should be omitted if empty when marshaling. + + // Invariants: + // if explicit is set, tag is non-nil. +} + +// Given a tag string with the format specified in the package comment, +// parseFieldParameters will parse it into a fieldParameters structure, +// ignoring unknown parts of the string. +func parseFieldParameters(str string) (ret fieldParameters) { + for _, part := range strings.Split(str, ",") { + switch { + case part == "optional": + ret.optional = true + case part == "explicit": + ret.explicit = true + if ret.tag == nil { + ret.tag = new(int) + } + case part == "generalized": + ret.timeType = TagGeneralizedTime + case part == "utc": + ret.timeType = TagUTCTime + case part == "ia5": + ret.stringType = TagIA5String + // jtasn1 case below added + case part == "generalstring": + ret.stringType = TagGeneralString + case part == "printable": + ret.stringType = TagPrintableString + case part == "utf8": + ret.stringType = TagUTF8String + case strings.HasPrefix(part, "default:"): + i, err := strconv.ParseInt(part[8:], 10, 64) + if err == nil { + ret.defaultValue = new(int64) + *ret.defaultValue = i + } + case strings.HasPrefix(part, "tag:"): + i, err := strconv.Atoi(part[4:]) + if err == nil { + ret.tag = new(int) + *ret.tag = i + } + case part == "set": + ret.set = true + case part == "application": + ret.application = true + if ret.tag == nil { + ret.tag = new(int) + } + case part == "omitempty": + ret.omitEmpty = true + } + } + return +} + +// Given a reflected Go type, getUniversalType returns the default tag number +// and expected compound flag. +func getUniversalType(t reflect.Type) (tagNumber int, isCompound, ok bool) { + switch t { + case objectIdentifierType: + return TagOID, false, true + case bitStringType: + return TagBitString, false, true + case timeType: + return TagUTCTime, false, true + case enumeratedType: + return TagEnum, false, true + case bigIntType: + return TagInteger, false, true + } + switch t.Kind() { + case reflect.Bool: + return TagBoolean, false, true + case reflect.Int, reflect.Int8, reflect.Int16, reflect.Int32, reflect.Int64: + return TagInteger, false, true + case reflect.Struct: + return TagSequence, true, true + case reflect.Slice: + if t.Elem().Kind() == reflect.Uint8 { + return TagOctetString, false, true + } + if strings.HasSuffix(t.Name(), "SET") { + return TagSet, true, true + } + return TagSequence, true, true + case reflect.String: + return TagPrintableString, false, true + } + return 0, false, false +} diff --git a/vendor/github.com/jcmturner/gofork/encoding/asn1/marshal.go b/vendor/github.com/jcmturner/gofork/encoding/asn1/marshal.go new file mode 100644 index 000000000000..f52eee9d2618 --- /dev/null +++ b/vendor/github.com/jcmturner/gofork/encoding/asn1/marshal.go @@ -0,0 +1,659 @@ +// Copyright 2009 The Go Authors. All rights reserved. +// Use of this source code is governed by a BSD-style +// license that can be found in the LICENSE file. + +package asn1 + +import ( + "bytes" + "errors" + "fmt" + "io" + "math/big" + "reflect" + "time" + "unicode/utf8" +) + +// A forkableWriter is an in-memory buffer that can be +// 'forked' to create new forkableWriters that bracket the +// original. After +// pre, post := w.fork() +// the overall sequence of bytes represented is logically w+pre+post. +type forkableWriter struct { + *bytes.Buffer + pre, post *forkableWriter +} + +func newForkableWriter() *forkableWriter { + return &forkableWriter{new(bytes.Buffer), nil, nil} +} + +func (f *forkableWriter) fork() (pre, post *forkableWriter) { + if f.pre != nil || f.post != nil { + panic("have already forked") + } + f.pre = newForkableWriter() + f.post = newForkableWriter() + return f.pre, f.post +} + +func (f *forkableWriter) Len() (l int) { + l += f.Buffer.Len() + if f.pre != nil { + l += f.pre.Len() + } + if f.post != nil { + l += f.post.Len() + } + return +} + +func (f *forkableWriter) writeTo(out io.Writer) (n int, err error) { + n, err = out.Write(f.Bytes()) + if err != nil { + return + } + + var nn int + + if f.pre != nil { + nn, err = f.pre.writeTo(out) + n += nn + if err != nil { + return + } + } + + if f.post != nil { + nn, err = f.post.writeTo(out) + n += nn + } + return +} + +func marshalBase128Int(out *forkableWriter, n int64) (err error) { + if n == 0 { + err = out.WriteByte(0) + return + } + + l := 0 + for i := n; i > 0; i >>= 7 { + l++ + } + + for i := l - 1; i >= 0; i-- { + o := byte(n >> uint(i*7)) + o &= 0x7f + if i != 0 { + o |= 0x80 + } + err = out.WriteByte(o) + if err != nil { + return + } + } + + return nil +} + +func marshalInt64(out *forkableWriter, i int64) (err error) { + n := int64Length(i) + + for ; n > 0; n-- { + err = out.WriteByte(byte(i >> uint((n-1)*8))) + if err != nil { + return + } + } + + return nil +} + +func int64Length(i int64) (numBytes int) { + numBytes = 1 + + for i > 127 { + numBytes++ + i >>= 8 + } + + for i < -128 { + numBytes++ + i >>= 8 + } + + return +} + +func marshalBigInt(out *forkableWriter, n *big.Int) (err error) { + if n.Sign() < 0 { + // A negative number has to be converted to two's-complement + // form. So we'll subtract 1 and invert. If the + // most-significant-bit isn't set then we'll need to pad the + // beginning with 0xff in order to keep the number negative. + nMinus1 := new(big.Int).Neg(n) + nMinus1.Sub(nMinus1, bigOne) + bytes := nMinus1.Bytes() + for i := range bytes { + bytes[i] ^= 0xff + } + if len(bytes) == 0 || bytes[0]&0x80 == 0 { + err = out.WriteByte(0xff) + if err != nil { + return + } + } + _, err = out.Write(bytes) + } else if n.Sign() == 0 { + // Zero is written as a single 0 zero rather than no bytes. + err = out.WriteByte(0x00) + } else { + bytes := n.Bytes() + if len(bytes) > 0 && bytes[0]&0x80 != 0 { + // We'll have to pad this with 0x00 in order to stop it + // looking like a negative number. + err = out.WriteByte(0) + if err != nil { + return + } + } + _, err = out.Write(bytes) + } + return +} + +func marshalLength(out *forkableWriter, i int) (err error) { + n := lengthLength(i) + + for ; n > 0; n-- { + err = out.WriteByte(byte(i >> uint((n-1)*8))) + if err != nil { + return + } + } + + return nil +} + +func lengthLength(i int) (numBytes int) { + numBytes = 1 + for i > 255 { + numBytes++ + i >>= 8 + } + return +} + +func marshalTagAndLength(out *forkableWriter, t tagAndLength) (err error) { + b := uint8(t.class) << 6 + if t.isCompound { + b |= 0x20 + } + if t.tag >= 31 { + b |= 0x1f + err = out.WriteByte(b) + if err != nil { + return + } + err = marshalBase128Int(out, int64(t.tag)) + if err != nil { + return + } + } else { + b |= uint8(t.tag) + err = out.WriteByte(b) + if err != nil { + return + } + } + + if t.length >= 128 { + l := lengthLength(t.length) + err = out.WriteByte(0x80 | byte(l)) + if err != nil { + return + } + err = marshalLength(out, t.length) + if err != nil { + return + } + } else { + err = out.WriteByte(byte(t.length)) + if err != nil { + return + } + } + + return nil +} + +func marshalBitString(out *forkableWriter, b BitString) (err error) { + paddingBits := byte((8 - b.BitLength%8) % 8) + err = out.WriteByte(paddingBits) + if err != nil { + return + } + _, err = out.Write(b.Bytes) + return +} + +func marshalObjectIdentifier(out *forkableWriter, oid []int) (err error) { + if len(oid) < 2 || oid[0] > 2 || (oid[0] < 2 && oid[1] >= 40) { + return StructuralError{"invalid object identifier"} + } + + err = marshalBase128Int(out, int64(oid[0]*40+oid[1])) + if err != nil { + return + } + for i := 2; i < len(oid); i++ { + err = marshalBase128Int(out, int64(oid[i])) + if err != nil { + return + } + } + + return +} + +func marshalPrintableString(out *forkableWriter, s string) (err error) { + b := []byte(s) + for _, c := range b { + if !isPrintable(c) { + return StructuralError{"PrintableString contains invalid character"} + } + } + + _, err = out.Write(b) + return +} + +func marshalIA5String(out *forkableWriter, s string) (err error) { + b := []byte(s) + for _, c := range b { + if c > 127 { + return StructuralError{"IA5String contains invalid character"} + } + } + + _, err = out.Write(b) + return +} + +func marshalUTF8String(out *forkableWriter, s string) (err error) { + _, err = out.Write([]byte(s)) + return +} + +func marshalTwoDigits(out *forkableWriter, v int) (err error) { + err = out.WriteByte(byte('0' + (v/10)%10)) + if err != nil { + return + } + return out.WriteByte(byte('0' + v%10)) +} + +func marshalFourDigits(out *forkableWriter, v int) (err error) { + var bytes [4]byte + for i := range bytes { + bytes[3-i] = '0' + byte(v%10) + v /= 10 + } + _, err = out.Write(bytes[:]) + return +} + +func outsideUTCRange(t time.Time) bool { + year := t.Year() + return year < 1950 || year >= 2050 +} + +func marshalUTCTime(out *forkableWriter, t time.Time) (err error) { + year := t.Year() + + switch { + case 1950 <= year && year < 2000: + err = marshalTwoDigits(out, year-1900) + case 2000 <= year && year < 2050: + err = marshalTwoDigits(out, year-2000) + default: + return StructuralError{"cannot represent time as UTCTime"} + } + if err != nil { + return + } + + return marshalTimeCommon(out, t) +} + +func marshalGeneralizedTime(out *forkableWriter, t time.Time) (err error) { + year := t.Year() + if year < 0 || year > 9999 { + return StructuralError{"cannot represent time as GeneralizedTime"} + } + if err = marshalFourDigits(out, year); err != nil { + return + } + + return marshalTimeCommon(out, t) +} + +func marshalTimeCommon(out *forkableWriter, t time.Time) (err error) { + _, month, day := t.Date() + + err = marshalTwoDigits(out, int(month)) + if err != nil { + return + } + + err = marshalTwoDigits(out, day) + if err != nil { + return + } + + hour, min, sec := t.Clock() + + err = marshalTwoDigits(out, hour) + if err != nil { + return + } + + err = marshalTwoDigits(out, min) + if err != nil { + return + } + + err = marshalTwoDigits(out, sec) + if err != nil { + return + } + + _, offset := t.Zone() + + switch { + case offset/60 == 0: + err = out.WriteByte('Z') + return + case offset > 0: + err = out.WriteByte('+') + case offset < 0: + err = out.WriteByte('-') + } + + if err != nil { + return + } + + offsetMinutes := offset / 60 + if offsetMinutes < 0 { + offsetMinutes = -offsetMinutes + } + + err = marshalTwoDigits(out, offsetMinutes/60) + if err != nil { + return + } + + err = marshalTwoDigits(out, offsetMinutes%60) + return +} + +func stripTagAndLength(in []byte) []byte { + _, offset, err := parseTagAndLength(in, 0) + if err != nil { + return in + } + return in[offset:] +} + +func marshalBody(out *forkableWriter, value reflect.Value, params fieldParameters) (err error) { + switch value.Type() { + case flagType: + return nil + case timeType: + t := value.Interface().(time.Time) + if params.timeType == TagGeneralizedTime || outsideUTCRange(t) { + return marshalGeneralizedTime(out, t) + } else { + return marshalUTCTime(out, t) + } + case bitStringType: + return marshalBitString(out, value.Interface().(BitString)) + case objectIdentifierType: + return marshalObjectIdentifier(out, value.Interface().(ObjectIdentifier)) + case bigIntType: + return marshalBigInt(out, value.Interface().(*big.Int)) + } + + switch v := value; v.Kind() { + case reflect.Bool: + if v.Bool() { + return out.WriteByte(255) + } else { + return out.WriteByte(0) + } + case reflect.Int, reflect.Int8, reflect.Int16, reflect.Int32, reflect.Int64: + return marshalInt64(out, v.Int()) + case reflect.Struct: + t := v.Type() + + startingField := 0 + + // If the first element of the structure is a non-empty + // RawContents, then we don't bother serializing the rest. + if t.NumField() > 0 && t.Field(0).Type == rawContentsType { + s := v.Field(0) + if s.Len() > 0 { + bytes := make([]byte, s.Len()) + for i := 0; i < s.Len(); i++ { + bytes[i] = uint8(s.Index(i).Uint()) + } + /* The RawContents will contain the tag and + * length fields but we'll also be writing + * those ourselves, so we strip them out of + * bytes */ + _, err = out.Write(stripTagAndLength(bytes)) + return + } else { + startingField = 1 + } + } + + for i := startingField; i < t.NumField(); i++ { + var pre *forkableWriter + pre, out = out.fork() + err = marshalField(pre, v.Field(i), parseFieldParameters(t.Field(i).Tag.Get("asn1"))) + if err != nil { + return + } + } + return + case reflect.Slice: + sliceType := v.Type() + if sliceType.Elem().Kind() == reflect.Uint8 { + bytes := make([]byte, v.Len()) + for i := 0; i < v.Len(); i++ { + bytes[i] = uint8(v.Index(i).Uint()) + } + _, err = out.Write(bytes) + return + } + + // jtasn1 Pass on the tags to the members but need to unset explicit switch and implicit value + //var fp fieldParameters + params.explicit = false + params.tag = nil + for i := 0; i < v.Len(); i++ { + var pre *forkableWriter + pre, out = out.fork() + err = marshalField(pre, v.Index(i), params) + if err != nil { + return + } + } + return + case reflect.String: + switch params.stringType { + case TagIA5String: + return marshalIA5String(out, v.String()) + case TagPrintableString: + return marshalPrintableString(out, v.String()) + default: + return marshalUTF8String(out, v.String()) + } + } + + return StructuralError{"unknown Go type"} +} + +func marshalField(out *forkableWriter, v reflect.Value, params fieldParameters) (err error) { + if !v.IsValid() { + return fmt.Errorf("asn1: cannot marshal nil value") + } + // If the field is an interface{} then recurse into it. + if v.Kind() == reflect.Interface && v.Type().NumMethod() == 0 { + return marshalField(out, v.Elem(), params) + } + + if v.Kind() == reflect.Slice && v.Len() == 0 && params.omitEmpty { + return + } + + if params.optional && params.defaultValue != nil && canHaveDefaultValue(v.Kind()) { + defaultValue := reflect.New(v.Type()).Elem() + defaultValue.SetInt(*params.defaultValue) + + if reflect.DeepEqual(v.Interface(), defaultValue.Interface()) { + return + } + } + + // If no default value is given then the zero value for the type is + // assumed to be the default value. This isn't obviously the correct + // behaviour, but it's what Go has traditionally done. + if params.optional && params.defaultValue == nil { + if reflect.DeepEqual(v.Interface(), reflect.Zero(v.Type()).Interface()) { + return + } + } + + if v.Type() == rawValueType { + rv := v.Interface().(RawValue) + if len(rv.FullBytes) != 0 { + _, err = out.Write(rv.FullBytes) + } else { + err = marshalTagAndLength(out, tagAndLength{rv.Class, rv.Tag, len(rv.Bytes), rv.IsCompound}) + if err != nil { + return + } + _, err = out.Write(rv.Bytes) + } + return + } + + tag, isCompound, ok := getUniversalType(v.Type()) + if !ok { + err = StructuralError{fmt.Sprintf("unknown Go type: %v", v.Type())} + return + } + class := ClassUniversal + + if params.timeType != 0 && tag != TagUTCTime { + return StructuralError{"explicit time type given to non-time member"} + } + + // jtasn1 updated to allow slices of strings + if params.stringType != 0 && !(tag == TagPrintableString || (v.Kind() == reflect.Slice && tag == 16 && v.Type().Elem().Kind() == reflect.String)) { + return StructuralError{"explicit string type given to non-string member"} + } + + switch tag { + case TagPrintableString: + if params.stringType == 0 { + // This is a string without an explicit string type. We'll use + // a PrintableString if the character set in the string is + // sufficiently limited, otherwise we'll use a UTF8String. + for _, r := range v.String() { + if r >= utf8.RuneSelf || !isPrintable(byte(r)) { + if !utf8.ValidString(v.String()) { + return errors.New("asn1: string not valid UTF-8") + } + tag = TagUTF8String + break + } + } + } else { + tag = params.stringType + } + case TagUTCTime: + if params.timeType == TagGeneralizedTime || outsideUTCRange(v.Interface().(time.Time)) { + tag = TagGeneralizedTime + } + } + + if params.set { + if tag != TagSequence { + return StructuralError{"non sequence tagged as set"} + } + tag = TagSet + } + + tags, body := out.fork() + + err = marshalBody(body, v, params) + if err != nil { + return + } + + bodyLen := body.Len() + + var explicitTag *forkableWriter + if params.explicit { + explicitTag, tags = tags.fork() + } + + if !params.explicit && params.tag != nil { + // implicit tag. + tag = *params.tag + class = ClassContextSpecific + } + + err = marshalTagAndLength(tags, tagAndLength{class, tag, bodyLen, isCompound}) + if err != nil { + return + } + + if params.explicit { + err = marshalTagAndLength(explicitTag, tagAndLength{ + class: ClassContextSpecific, + tag: *params.tag, + length: bodyLen + tags.Len(), + isCompound: true, + }) + } + + return err +} + +// Marshal returns the ASN.1 encoding of val. +// +// In addition to the struct tags recognised by Unmarshal, the following can be +// used: +// +// ia5: causes strings to be marshaled as ASN.1, IA5 strings +// omitempty: causes empty slices to be skipped +// printable: causes strings to be marshaled as ASN.1, PrintableString strings. +// utf8: causes strings to be marshaled as ASN.1, UTF8 strings +func Marshal(val interface{}) ([]byte, error) { + var out bytes.Buffer + v := reflect.ValueOf(val) + f := newForkableWriter() + err := marshalField(f, v, fieldParameters{}) + if err != nil { + return nil, err + } + _, err = f.writeTo(&out) + return out.Bytes(), err +} diff --git a/vendor/github.com/jcmturner/gofork/x/crypto/pbkdf2/pbkdf2.go b/vendor/github.com/jcmturner/gofork/x/crypto/pbkdf2/pbkdf2.go new file mode 100644 index 000000000000..75d418763db0 --- /dev/null +++ b/vendor/github.com/jcmturner/gofork/x/crypto/pbkdf2/pbkdf2.go @@ -0,0 +1,98 @@ +// Copyright 2012 The Go Authors. All rights reserved. +// Use of this source code is governed by a BSD-style +// license that can be found in the LICENSE file. + +/* +Package pbkdf2 implements the key derivation function PBKDF2 as defined in RFC +2898 / PKCS #5 v2.0. + +A key derivation function is useful when encrypting data based on a password +or any other not-fully-random data. It uses a pseudorandom function to derive +a secure encryption key based on the password. + +While v2.0 of the standard defines only one pseudorandom function to use, +HMAC-SHA1, the drafted v2.1 specification allows use of all five FIPS Approved +Hash Functions SHA-1, SHA-224, SHA-256, SHA-384 and SHA-512 for HMAC. To +choose, you can pass the `New` functions from the different SHA packages to +pbkdf2.Key. +*/ +package pbkdf2 + +import ( + "crypto/hmac" + "hash" +) + +// Key derives a key from the password, salt and iteration count, returning a +// []byte of length keylen that can be used as cryptographic key. The key is +// derived based on the method described as PBKDF2 with the HMAC variant using +// the supplied hash function. +// +// For example, to use a HMAC-SHA-1 based PBKDF2 key derivation function, you +// can get a derived key for e.g. AES-256 (which needs a 32-byte key) by +// doing: +// +// dk := pbkdf2.Key([]byte("some password"), salt, 4096, 32, sha1.New) +// +// Remember to get a good random salt. At least 8 bytes is recommended by the +// RFC. +// +// Using a higher iteration count will increase the cost of an exhaustive +// search but will also make derivation proportionally slower. +func Key(password, salt []byte, iter, keyLen int, h func() hash.Hash) []byte { + return Key64(password, salt, int64(iter), int64(keyLen), h) +} + +// Key64 derives a key from the password, salt and iteration count, returning a +// []byte of length keylen that can be used as cryptographic key. Key64 uses +// int64 for the iteration count and key length to allow larger values. +// The key is derived based on the method described as PBKDF2 with the HMAC +// variant using the supplied hash function. +// +// For example, to use a HMAC-SHA-1 based PBKDF2 key derivation function, you +// can get a derived key for e.g. AES-256 (which needs a 32-byte key) by +// doing: +// +// dk := pbkdf2.Key([]byte("some password"), salt, 4096, 32, sha1.New) +// +// Remember to get a good random salt. At least 8 bytes is recommended by the +// RFC. +// +// Using a higher iteration count will increase the cost of an exhaustive +// search but will also make derivation proportionally slower. +func Key64(password, salt []byte, iter, keyLen int64, h func() hash.Hash) []byte { + prf := hmac.New(h, password) + hashLen := int64(prf.Size()) + numBlocks := (keyLen + hashLen - 1) / hashLen + + var buf [4]byte + dk := make([]byte, 0, numBlocks*hashLen) + U := make([]byte, hashLen) + for block := int64(1); block <= numBlocks; block++ { + // N.B.: || means concatenation, ^ means XOR + // for each block T_i = U_1 ^ U_2 ^ ... ^ U_iter + // U_1 = PRF(password, salt || uint(i)) + prf.Reset() + prf.Write(salt) + buf[0] = byte(block >> 24) + buf[1] = byte(block >> 16) + buf[2] = byte(block >> 8) + buf[3] = byte(block) + prf.Write(buf[:4]) + dk = prf.Sum(dk) + T := dk[int64(len(dk))-hashLen:] + copy(U, T) + + // U_n = PRF(password, U_(n-1)) + for n := int64(2); n <= iter; n++ { + prf.Reset() + prf.Write(U) + U = U[:0] + U = prf.Sum(U) + for x := range U { + T[x] ^= U[x] + } + } + } + return dk[:keyLen] +} diff --git a/vendor/github.com/jcmturner/gokrb5/v8/LICENSE b/vendor/github.com/jcmturner/gokrb5/v8/LICENSE new file mode 100644 index 000000000000..8dada3edaf50 --- /dev/null +++ b/vendor/github.com/jcmturner/gokrb5/v8/LICENSE @@ -0,0 +1,201 @@ + 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. + + END OF TERMS AND CONDITIONS + + APPENDIX: How to apply the Apache License to your work. + + To apply the Apache License to your work, attach the following + boilerplate notice, with the fields enclosed by brackets "{}" + replaced with your own identifying information. (Don't include + the brackets!) The text should be enclosed in the appropriate + comment syntax for the file format. We also recommend that a + file or class name and description of purpose be included on the + same "printed page" as the copyright notice for easier + identification within third-party archives. + + Copyright {yyyy} {name of copyright owner} + + 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. diff --git a/vendor/github.com/jcmturner/gokrb5/v8/asn1tools/tools.go b/vendor/github.com/jcmturner/gokrb5/v8/asn1tools/tools.go new file mode 100644 index 000000000000..f27740b9bd73 --- /dev/null +++ b/vendor/github.com/jcmturner/gokrb5/v8/asn1tools/tools.go @@ -0,0 +1,86 @@ +// Package asn1tools provides tools for managing ASN1 marshaled data. +package asn1tools + +import ( + "github.com/jcmturner/gofork/encoding/asn1" +) + +// MarshalLengthBytes returns the ASN1 encoded bytes for the length 'l' +// +// There are two forms: short (for lengths between 0 and 127), and long definite (for lengths between 0 and 2^1008 -1). +// +// Short form: One octet. Bit 8 has value "0" and bits 7-1 give the length. +// +// Long form: Two to 127 octets. Bit 8 of first octet has value "1" and bits 7-1 give the number of additional length octets. Second and following octets give the length, base 256, most significant digit first. +func MarshalLengthBytes(l int) []byte { + if l <= 127 { + return []byte{byte(l)} + } + var b []byte + p := 1 + for i := 1; i < 127; { + b = append([]byte{byte((l % (p * 256)) / p)}, b...) + p = p * 256 + l = l - l%p + if l <= 0 { + break + } + } + return append([]byte{byte(128 + len(b))}, b...) +} + +// GetLengthFromASN returns the length of a slice of ASN1 encoded bytes from the ASN1 length header it contains. +func GetLengthFromASN(b []byte) int { + if int(b[1]) <= 127 { + return int(b[1]) + } + // The bytes that indicate the length + lb := b[2 : 2+int(b[1])-128] + base := 1 + l := 0 + for i := len(lb) - 1; i >= 0; i-- { + l += int(lb[i]) * base + base = base * 256 + } + return l +} + +// GetNumberBytesInLengthHeader returns the number of bytes in the ASn1 header that indicate the length. +func GetNumberBytesInLengthHeader(b []byte) int { + if int(b[1]) <= 127 { + return 1 + } + // The bytes that indicate the length + return 1 + int(b[1]) - 128 +} + +// AddASNAppTag adds an ASN1 encoding application tag value to the raw bytes provided. +func AddASNAppTag(b []byte, tag int) []byte { + r := asn1.RawValue{ + Class: asn1.ClassApplication, + IsCompound: true, + Tag: tag, + Bytes: b, + } + ab, _ := asn1.Marshal(r) + return ab +} + +/* +// The Marshal method of golang's asn1 package does not enable you to define wrapping the output in an application tag. +// This method adds that wrapping tag. +func AddASNAppTag(b []byte, tag int) []byte { + // The ASN1 wrapping consists of 2 bytes: + // 1st byte -> Identifier Octet - Application Tag + // 2nd byte -> The length (this will be the size indicated in the input bytes + 2 for the additional bytes we add here. + // Application Tag: + //| Bit: | 8 | 7 | 6 | 5 | 4 | 3 | 2 | 1 | + //| Value: | 0 | 1 | 1 | From the RFC spec 4120 | + //| Explanation | Defined by the ASN1 encoding rules for an application tag | A value of 1 indicates a constructed type | The ASN Application tag value | + // Therefore the value of the byte is an integer = ( Application tag value + 96 ) + //b = append(MarshalLengthBytes(int(b[1])+2), b...) + b = append(MarshalLengthBytes(len(b)), b...) + b = append([]byte{byte(96 + tag)}, b...) + return b +} +*/ diff --git a/vendor/github.com/jcmturner/gokrb5/v8/client/ASExchange.go b/vendor/github.com/jcmturner/gokrb5/v8/client/ASExchange.go new file mode 100644 index 000000000000..5becccc4dde5 --- /dev/null +++ b/vendor/github.com/jcmturner/gokrb5/v8/client/ASExchange.go @@ -0,0 +1,182 @@ +package client + +import ( + "github.com/jcmturner/gokrb5/v8/crypto" + "github.com/jcmturner/gokrb5/v8/crypto/etype" + "github.com/jcmturner/gokrb5/v8/iana/errorcode" + "github.com/jcmturner/gokrb5/v8/iana/keyusage" + "github.com/jcmturner/gokrb5/v8/iana/patype" + "github.com/jcmturner/gokrb5/v8/krberror" + "github.com/jcmturner/gokrb5/v8/messages" + "github.com/jcmturner/gokrb5/v8/types" +) + +// ASExchange performs an AS exchange for the client to retrieve a TGT. +func (cl *Client) ASExchange(realm string, ASReq messages.ASReq, referral int) (messages.ASRep, error) { + if ok, err := cl.IsConfigured(); !ok { + return messages.ASRep{}, krberror.Errorf(err, krberror.ConfigError, "AS Exchange cannot be performed") + } + + // Set PAData if required + err := setPAData(cl, nil, &ASReq) + if err != nil { + return messages.ASRep{}, krberror.Errorf(err, krberror.KRBMsgError, "AS Exchange Error: issue with setting PAData on AS_REQ") + } + + b, err := ASReq.Marshal() + if err != nil { + return messages.ASRep{}, krberror.Errorf(err, krberror.EncodingError, "AS Exchange Error: failed marshaling AS_REQ") + } + var ASRep messages.ASRep + + rb, err := cl.sendToKDC(b, realm) + if err != nil { + if e, ok := err.(messages.KRBError); ok { + switch e.ErrorCode { + case errorcode.KDC_ERR_PREAUTH_REQUIRED, errorcode.KDC_ERR_PREAUTH_FAILED: + // From now on assume this client will need to do this pre-auth and set the PAData + cl.settings.assumePreAuthentication = true + err = setPAData(cl, &e, &ASReq) + if err != nil { + return messages.ASRep{}, krberror.Errorf(err, krberror.KRBMsgError, "AS Exchange Error: failed setting AS_REQ PAData for pre-authentication required") + } + b, err := ASReq.Marshal() + if err != nil { + return messages.ASRep{}, krberror.Errorf(err, krberror.EncodingError, "AS Exchange Error: failed marshaling AS_REQ with PAData") + } + rb, err = cl.sendToKDC(b, realm) + if err != nil { + if _, ok := err.(messages.KRBError); ok { + return messages.ASRep{}, krberror.Errorf(err, krberror.KDCError, "AS Exchange Error: kerberos error response from KDC") + } + return messages.ASRep{}, krberror.Errorf(err, krberror.NetworkingError, "AS Exchange Error: failed sending AS_REQ to KDC") + } + case errorcode.KDC_ERR_WRONG_REALM: + // Client referral https://tools.ietf.org/html/rfc6806.html#section-7 + if referral > 5 { + return messages.ASRep{}, krberror.Errorf(err, krberror.KRBMsgError, "maximum number of client referrals exceeded") + } + referral++ + return cl.ASExchange(e.CRealm, ASReq, referral) + default: + return messages.ASRep{}, krberror.Errorf(err, krberror.KDCError, "AS Exchange Error: kerberos error response from KDC") + } + } else { + return messages.ASRep{}, krberror.Errorf(err, krberror.NetworkingError, "AS Exchange Error: failed sending AS_REQ to KDC") + } + } + err = ASRep.Unmarshal(rb) + if err != nil { + return messages.ASRep{}, krberror.Errorf(err, krberror.EncodingError, "AS Exchange Error: failed to process the AS_REP") + } + if ok, err := ASRep.Verify(cl.Config, cl.Credentials, ASReq); !ok { + return messages.ASRep{}, krberror.Errorf(err, krberror.KRBMsgError, "AS Exchange Error: AS_REP is not valid or client password/keytab incorrect") + } + return ASRep, nil +} + +// setPAData adds pre-authentication data to the AS_REQ. +func setPAData(cl *Client, krberr *messages.KRBError, ASReq *messages.ASReq) error { + if !cl.settings.DisablePAFXFAST() { + pa := types.PAData{PADataType: patype.PA_REQ_ENC_PA_REP} + ASReq.PAData = append(ASReq.PAData, pa) + } + if cl.settings.AssumePreAuthentication() { + // Identify the etype to use to encrypt the PA Data + var et etype.EType + var err error + var key types.EncryptionKey + var kvno int + if krberr == nil { + // This is not in response to an error from the KDC. It is preemptive or renewal + // There is no KRB Error that tells us the etype to use + etn := cl.settings.preAuthEType // Use the etype that may have previously been negotiated + if etn == 0 { + etn = int32(cl.Config.LibDefaults.PreferredPreauthTypes[0]) // Resort to config + } + et, err = crypto.GetEtype(etn) + if err != nil { + return krberror.Errorf(err, krberror.EncryptingError, "error getting etype for pre-auth encryption") + } + key, kvno, err = cl.Key(et, 0, nil) + if err != nil { + return krberror.Errorf(err, krberror.EncryptingError, "error getting key from credentials") + } + } else { + // Get the etype to use from the PA data in the KRBError e-data + et, err = preAuthEType(krberr) + if err != nil { + return krberror.Errorf(err, krberror.EncryptingError, "error getting etype for pre-auth encryption") + } + cl.settings.preAuthEType = et.GetETypeID() // Set the etype that has been defined for potential future use + key, kvno, err = cl.Key(et, 0, krberr) + if err != nil { + return krberror.Errorf(err, krberror.EncryptingError, "error getting key from credentials") + } + } + // Generate the PA data + paTSb, err := types.GetPAEncTSEncAsnMarshalled() + if err != nil { + return krberror.Errorf(err, krberror.KRBMsgError, "error creating PAEncTSEnc for Pre-Authentication") + } + paEncTS, err := crypto.GetEncryptedData(paTSb, key, keyusage.AS_REQ_PA_ENC_TIMESTAMP, kvno) + if err != nil { + return krberror.Errorf(err, krberror.EncryptingError, "error encrypting pre-authentication timestamp") + } + pb, err := paEncTS.Marshal() + if err != nil { + return krberror.Errorf(err, krberror.EncodingError, "error marshaling the PAEncTSEnc encrypted data") + } + pa := types.PAData{ + PADataType: patype.PA_ENC_TIMESTAMP, + PADataValue: pb, + } + // Look for and delete any exiting patype.PA_ENC_TIMESTAMP + for i, pa := range ASReq.PAData { + if pa.PADataType == patype.PA_ENC_TIMESTAMP { + ASReq.PAData[i] = ASReq.PAData[len(ASReq.PAData)-1] + ASReq.PAData = ASReq.PAData[:len(ASReq.PAData)-1] + } + } + ASReq.PAData = append(ASReq.PAData, pa) + } + return nil +} + +// preAuthEType establishes what encryption type to use for pre-authentication from the KRBError returned from the KDC. +func preAuthEType(krberr *messages.KRBError) (etype etype.EType, err error) { + //RFC 4120 5.2.7.5 covers the preference order of ETYPE-INFO2 and ETYPE-INFO. + var etypeID int32 + var pas types.PADataSequence + e := pas.Unmarshal(krberr.EData) + if e != nil { + err = krberror.Errorf(e, krberror.EncodingError, "error unmashalling KRBError data") + return + } +Loop: + for _, pa := range pas { + switch pa.PADataType { + case patype.PA_ETYPE_INFO2: + info, e := pa.GetETypeInfo2() + if e != nil { + err = krberror.Errorf(e, krberror.EncodingError, "error unmashalling ETYPE-INFO2 data") + return + } + etypeID = info[0].EType + break Loop + case patype.PA_ETYPE_INFO: + info, e := pa.GetETypeInfo() + if e != nil { + err = krberror.Errorf(e, krberror.EncodingError, "error unmashalling ETYPE-INFO data") + return + } + etypeID = info[0].EType + } + } + etype, e = crypto.GetEtype(etypeID) + if e != nil { + err = krberror.Errorf(e, krberror.EncryptingError, "error creating etype") + return + } + return etype, nil +} diff --git a/vendor/github.com/jcmturner/gokrb5/v8/client/TGSExchange.go b/vendor/github.com/jcmturner/gokrb5/v8/client/TGSExchange.go new file mode 100644 index 000000000000..e4571ce86691 --- /dev/null +++ b/vendor/github.com/jcmturner/gokrb5/v8/client/TGSExchange.go @@ -0,0 +1,103 @@ +package client + +import ( + "github.com/jcmturner/gokrb5/v8/iana/flags" + "github.com/jcmturner/gokrb5/v8/iana/nametype" + "github.com/jcmturner/gokrb5/v8/krberror" + "github.com/jcmturner/gokrb5/v8/messages" + "github.com/jcmturner/gokrb5/v8/types" +) + +// TGSREQGenerateAndExchange generates the TGS_REQ and performs a TGS exchange to retrieve a ticket to the specified SPN. +func (cl *Client) TGSREQGenerateAndExchange(spn types.PrincipalName, kdcRealm string, tgt messages.Ticket, sessionKey types.EncryptionKey, renewal bool) (tgsReq messages.TGSReq, tgsRep messages.TGSRep, err error) { + tgsReq, err = messages.NewTGSReq(cl.Credentials.CName(), kdcRealm, cl.Config, tgt, sessionKey, spn, renewal) + if err != nil { + return tgsReq, tgsRep, krberror.Errorf(err, krberror.KRBMsgError, "TGS Exchange Error: failed to generate a new TGS_REQ") + } + return cl.TGSExchange(tgsReq, kdcRealm, tgsRep.Ticket, sessionKey, 0) +} + +// TGSExchange exchanges the provided TGS_REQ with the KDC to retrieve a TGS_REP. +// Referrals are automatically handled. +// The client's cache is updated with the ticket received. +func (cl *Client) TGSExchange(tgsReq messages.TGSReq, kdcRealm string, tgt messages.Ticket, sessionKey types.EncryptionKey, referral int) (messages.TGSReq, messages.TGSRep, error) { + var tgsRep messages.TGSRep + b, err := tgsReq.Marshal() + if err != nil { + return tgsReq, tgsRep, krberror.Errorf(err, krberror.EncodingError, "TGS Exchange Error: failed to marshal TGS_REQ") + } + r, err := cl.sendToKDC(b, kdcRealm) + if err != nil { + if _, ok := err.(messages.KRBError); ok { + return tgsReq, tgsRep, krberror.Errorf(err, krberror.KDCError, "TGS Exchange Error: kerberos error response from KDC when requesting for %s", tgsReq.ReqBody.SName.PrincipalNameString()) + } + return tgsReq, tgsRep, krberror.Errorf(err, krberror.NetworkingError, "TGS Exchange Error: issue sending TGS_REQ to KDC") + } + err = tgsRep.Unmarshal(r) + if err != nil { + return tgsReq, tgsRep, krberror.Errorf(err, krberror.EncodingError, "TGS Exchange Error: failed to process the TGS_REP") + } + err = tgsRep.DecryptEncPart(sessionKey) + if err != nil { + return tgsReq, tgsRep, krberror.Errorf(err, krberror.EncodingError, "TGS Exchange Error: failed to process the TGS_REP") + } + if ok, err := tgsRep.Verify(cl.Config, tgsReq); !ok { + return tgsReq, tgsRep, krberror.Errorf(err, krberror.EncodingError, "TGS Exchange Error: TGS_REP is not valid") + } + + if tgsRep.Ticket.SName.NameString[0] == "krbtgt" && !tgsRep.Ticket.SName.Equal(tgsReq.ReqBody.SName) { + if referral > 5 { + return tgsReq, tgsRep, krberror.Errorf(err, krberror.KRBMsgError, "TGS Exchange Error: maximum number of referrals exceeded") + } + // Server referral https://tools.ietf.org/html/rfc6806.html#section-8 + // The TGS Rep contains a TGT for another domain as the service resides in that domain. + cl.addSession(tgsRep.Ticket, tgsRep.DecryptedEncPart) + realm := tgsRep.Ticket.SName.NameString[len(tgsRep.Ticket.SName.NameString)-1] + referral++ + if types.IsFlagSet(&tgsReq.ReqBody.KDCOptions, flags.EncTktInSkey) && len(tgsReq.ReqBody.AdditionalTickets) > 0 { + tgsReq, err = messages.NewUser2UserTGSReq(cl.Credentials.CName(), kdcRealm, cl.Config, tgt, sessionKey, tgsReq.ReqBody.SName, tgsReq.Renewal, tgsReq.ReqBody.AdditionalTickets[0]) + if err != nil { + return tgsReq, tgsRep, err + } + } + tgsReq, err = messages.NewTGSReq(cl.Credentials.CName(), realm, cl.Config, tgsRep.Ticket, tgsRep.DecryptedEncPart.Key, tgsReq.ReqBody.SName, tgsReq.Renewal) + if err != nil { + return tgsReq, tgsRep, err + } + return cl.TGSExchange(tgsReq, realm, tgsRep.Ticket, tgsRep.DecryptedEncPart.Key, referral) + } + cl.cache.addEntry( + tgsRep.Ticket, + tgsRep.DecryptedEncPart.AuthTime, + tgsRep.DecryptedEncPart.StartTime, + tgsRep.DecryptedEncPart.EndTime, + tgsRep.DecryptedEncPart.RenewTill, + tgsRep.DecryptedEncPart.Key, + ) + cl.Log("ticket added to cache for %s (EndTime: %v)", tgsRep.Ticket.SName.PrincipalNameString(), tgsRep.DecryptedEncPart.EndTime) + return tgsReq, tgsRep, err +} + +// GetServiceTicket makes a request to get a service ticket for the SPN specified +// SPN format: / Eg. HTTP/www.example.com +// The ticket will be added to the client's ticket cache +func (cl *Client) GetServiceTicket(spn string) (messages.Ticket, types.EncryptionKey, error) { + var tkt messages.Ticket + var skey types.EncryptionKey + if tkt, skey, ok := cl.GetCachedTicket(spn); ok { + // Already a valid ticket in the cache + return tkt, skey, nil + } + princ := types.NewPrincipalName(nametype.KRB_NT_PRINCIPAL, spn) + realm := cl.Config.ResolveRealm(princ.NameString[len(princ.NameString)-1]) + + tgt, skey, err := cl.sessionTGT(realm) + if err != nil { + return tkt, skey, err + } + _, tgsRep, err := cl.TGSREQGenerateAndExchange(princ, realm, tgt, skey, false) + if err != nil { + return tkt, skey, err + } + return tgsRep.Ticket, tgsRep.DecryptedEncPart.Key, nil +} diff --git a/vendor/github.com/jcmturner/gokrb5/v8/client/cache.go b/vendor/github.com/jcmturner/gokrb5/v8/client/cache.go new file mode 100644 index 000000000000..552e73e41e13 --- /dev/null +++ b/vendor/github.com/jcmturner/gokrb5/v8/client/cache.go @@ -0,0 +1,134 @@ +package client + +import ( + "encoding/json" + "errors" + "sort" + "sync" + "time" + + "github.com/jcmturner/gokrb5/v8/messages" + "github.com/jcmturner/gokrb5/v8/types" +) + +// Cache for service tickets held by the client. +type Cache struct { + Entries map[string]CacheEntry + mux sync.RWMutex +} + +// CacheEntry holds details for a cache entry. +type CacheEntry struct { + SPN string + Ticket messages.Ticket `json:"-"` + AuthTime time.Time + StartTime time.Time + EndTime time.Time + RenewTill time.Time + SessionKey types.EncryptionKey `json:"-"` +} + +// NewCache creates a new client ticket cache instance. +func NewCache() *Cache { + return &Cache{ + Entries: map[string]CacheEntry{}, + } +} + +// getEntry returns a cache entry that matches the SPN. +func (c *Cache) getEntry(spn string) (CacheEntry, bool) { + c.mux.RLock() + defer c.mux.RUnlock() + e, ok := (*c).Entries[spn] + return e, ok +} + +// JSON returns information about the cached service tickets in a JSON format. +func (c *Cache) JSON() (string, error) { + c.mux.RLock() + defer c.mux.RUnlock() + var es []CacheEntry + keys := make([]string, 0, len(c.Entries)) + for k := range c.Entries { + keys = append(keys, k) + } + sort.Strings(keys) + for _, k := range keys { + es = append(es, c.Entries[k]) + } + b, err := json.MarshalIndent(&es, "", " ") + if err != nil { + return "", err + } + return string(b), nil +} + +// addEntry adds a ticket to the cache. +func (c *Cache) addEntry(tkt messages.Ticket, authTime, startTime, endTime, renewTill time.Time, sessionKey types.EncryptionKey) CacheEntry { + spn := tkt.SName.PrincipalNameString() + c.mux.Lock() + defer c.mux.Unlock() + (*c).Entries[spn] = CacheEntry{ + SPN: spn, + Ticket: tkt, + AuthTime: authTime, + StartTime: startTime, + EndTime: endTime, + RenewTill: renewTill, + SessionKey: sessionKey, + } + return c.Entries[spn] +} + +// clear deletes all the cache entries +func (c *Cache) clear() { + c.mux.Lock() + defer c.mux.Unlock() + for k := range c.Entries { + delete(c.Entries, k) + } +} + +// RemoveEntry removes the cache entry for the defined SPN. +func (c *Cache) RemoveEntry(spn string) { + c.mux.Lock() + defer c.mux.Unlock() + delete(c.Entries, spn) +} + +// GetCachedTicket returns a ticket from the cache for the SPN. +// Only a ticket that is currently valid will be returned. +func (cl *Client) GetCachedTicket(spn string) (messages.Ticket, types.EncryptionKey, bool) { + if e, ok := cl.cache.getEntry(spn); ok { + //If within time window of ticket return it + if time.Now().UTC().After(e.StartTime) && time.Now().UTC().Before(e.EndTime) { + cl.Log("ticket received from cache for %s", spn) + return e.Ticket, e.SessionKey, true + } else if time.Now().UTC().Before(e.RenewTill) { + e, err := cl.renewTicket(e) + if err != nil { + return e.Ticket, e.SessionKey, false + } + return e.Ticket, e.SessionKey, true + } + } + var tkt messages.Ticket + var key types.EncryptionKey + return tkt, key, false +} + +// renewTicket renews a cache entry ticket. +// To renew from outside the client package use GetCachedTicket +func (cl *Client) renewTicket(e CacheEntry) (CacheEntry, error) { + spn := e.Ticket.SName + _, _, err := cl.TGSREQGenerateAndExchange(spn, e.Ticket.Realm, e.Ticket, e.SessionKey, true) + if err != nil { + return e, err + } + e, ok := cl.cache.getEntry(e.Ticket.SName.PrincipalNameString()) + if !ok { + return e, errors.New("ticket was not added to cache") + } + cl.Log("ticket renewed for %s (EndTime: %v)", spn.PrincipalNameString(), e.EndTime) + return e, nil +} diff --git a/vendor/github.com/jcmturner/gokrb5/v8/client/client.go b/vendor/github.com/jcmturner/gokrb5/v8/client/client.go new file mode 100644 index 000000000000..074e3f1245f7 --- /dev/null +++ b/vendor/github.com/jcmturner/gokrb5/v8/client/client.go @@ -0,0 +1,329 @@ +// Package client provides a client library and methods for Kerberos 5 authentication. +package client + +import ( + "encoding/json" + "errors" + "fmt" + "io" + "strings" + "time" + + "github.com/jcmturner/gokrb5/v8/config" + "github.com/jcmturner/gokrb5/v8/credentials" + "github.com/jcmturner/gokrb5/v8/crypto" + "github.com/jcmturner/gokrb5/v8/crypto/etype" + "github.com/jcmturner/gokrb5/v8/iana/errorcode" + "github.com/jcmturner/gokrb5/v8/iana/nametype" + "github.com/jcmturner/gokrb5/v8/keytab" + "github.com/jcmturner/gokrb5/v8/krberror" + "github.com/jcmturner/gokrb5/v8/messages" + "github.com/jcmturner/gokrb5/v8/types" +) + +// Client side configuration and state. +type Client struct { + Credentials *credentials.Credentials + Config *config.Config + settings *Settings + sessions *sessions + cache *Cache +} + +// NewWithPassword creates a new client from a password credential. +// Set the realm to empty string to use the default realm from config. +func NewWithPassword(username, realm, password string, krb5conf *config.Config, settings ...func(*Settings)) *Client { + creds := credentials.New(username, realm) + return &Client{ + Credentials: creds.WithPassword(password), + Config: krb5conf, + settings: NewSettings(settings...), + sessions: &sessions{ + Entries: make(map[string]*session), + }, + cache: NewCache(), + } +} + +// NewWithKeytab creates a new client from a keytab credential. +func NewWithKeytab(username, realm string, kt *keytab.Keytab, krb5conf *config.Config, settings ...func(*Settings)) *Client { + creds := credentials.New(username, realm) + return &Client{ + Credentials: creds.WithKeytab(kt), + Config: krb5conf, + settings: NewSettings(settings...), + sessions: &sessions{ + Entries: make(map[string]*session), + }, + cache: NewCache(), + } +} + +// NewFromCCache create a client from a populated client cache. +// +// WARNING: A client created from CCache does not automatically renew TGTs and a failure will occur after the TGT expires. +func NewFromCCache(c *credentials.CCache, krb5conf *config.Config, settings ...func(*Settings)) (*Client, error) { + cl := &Client{ + Credentials: c.GetClientCredentials(), + Config: krb5conf, + settings: NewSettings(settings...), + sessions: &sessions{ + Entries: make(map[string]*session), + }, + cache: NewCache(), + } + spn := types.PrincipalName{ + NameType: nametype.KRB_NT_SRV_INST, + NameString: []string{"krbtgt", c.DefaultPrincipal.Realm}, + } + cred, ok := c.GetEntry(spn) + if !ok { + return cl, errors.New("TGT not found in CCache") + } + var tgt messages.Ticket + err := tgt.Unmarshal(cred.Ticket) + if err != nil { + return cl, fmt.Errorf("TGT bytes in cache are not valid: %v", err) + } + cl.sessions.Entries[c.DefaultPrincipal.Realm] = &session{ + realm: c.DefaultPrincipal.Realm, + authTime: cred.AuthTime, + endTime: cred.EndTime, + renewTill: cred.RenewTill, + tgt: tgt, + sessionKey: cred.Key, + } + for _, cred := range c.GetEntries() { + var tkt messages.Ticket + err = tkt.Unmarshal(cred.Ticket) + if err != nil { + return cl, fmt.Errorf("cache entry ticket bytes are not valid: %v", err) + } + cl.cache.addEntry( + tkt, + cred.AuthTime, + cred.StartTime, + cred.EndTime, + cred.RenewTill, + cred.Key, + ) + } + return cl, nil +} + +// Key returns the client's encryption key for the specified encryption type and its kvno (kvno of zero will find latest). +// The key can be retrieved either from the keytab or generated from the client's password. +// If the client has both a keytab and a password defined the keytab is favoured as the source for the key +// A KRBError can be passed in the event the KDC returns one of type KDC_ERR_PREAUTH_REQUIRED and is required to derive +// the key for pre-authentication from the client's password. If a KRBError is not available, pass nil to this argument. +func (cl *Client) Key(etype etype.EType, kvno int, krberr *messages.KRBError) (types.EncryptionKey, int, error) { + if cl.Credentials.HasKeytab() && etype != nil { + return cl.Credentials.Keytab().GetEncryptionKey(cl.Credentials.CName(), cl.Credentials.Domain(), kvno, etype.GetETypeID()) + } else if cl.Credentials.HasPassword() { + if krberr != nil && krberr.ErrorCode == errorcode.KDC_ERR_PREAUTH_REQUIRED { + var pas types.PADataSequence + err := pas.Unmarshal(krberr.EData) + if err != nil { + return types.EncryptionKey{}, 0, fmt.Errorf("could not get PAData from KRBError to generate key from password: %v", err) + } + key, _, err := crypto.GetKeyFromPassword(cl.Credentials.Password(), krberr.CName, krberr.CRealm, etype.GetETypeID(), pas) + return key, 0, err + } + key, _, err := crypto.GetKeyFromPassword(cl.Credentials.Password(), cl.Credentials.CName(), cl.Credentials.Domain(), etype.GetETypeID(), types.PADataSequence{}) + return key, 0, err + } + return types.EncryptionKey{}, 0, errors.New("credential has neither keytab or password to generate key") +} + +// IsConfigured indicates if the client has the values required set. +func (cl *Client) IsConfigured() (bool, error) { + if cl.Credentials.UserName() == "" { + return false, errors.New("client does not have a username") + } + if cl.Credentials.Domain() == "" { + return false, errors.New("client does not have a define realm") + } + // Client needs to have either a password, keytab or a session already (later when loading from CCache) + if !cl.Credentials.HasPassword() && !cl.Credentials.HasKeytab() { + authTime, _, _, _, err := cl.sessionTimes(cl.Credentials.Domain()) + if err != nil || authTime.IsZero() { + return false, errors.New("client has neither a keytab nor a password set and no session") + } + } + if !cl.Config.LibDefaults.DNSLookupKDC { + for _, r := range cl.Config.Realms { + if r.Realm == cl.Credentials.Domain() { + if len(r.KDC) > 0 { + return true, nil + } + return false, errors.New("client krb5 config does not have any defined KDCs for the default realm") + } + } + } + return true, nil +} + +// Login the client with the KDC via an AS exchange. +func (cl *Client) Login() error { + if ok, err := cl.IsConfigured(); !ok { + return err + } + if !cl.Credentials.HasPassword() && !cl.Credentials.HasKeytab() { + _, endTime, _, _, err := cl.sessionTimes(cl.Credentials.Domain()) + if err != nil { + return krberror.Errorf(err, krberror.KRBMsgError, "no user credentials available and error getting any existing session") + } + if time.Now().UTC().After(endTime) { + return krberror.New(krberror.KRBMsgError, "cannot login, no user credentials available and no valid existing session") + } + // no credentials but there is a session with tgt already + return nil + } + ASReq, err := messages.NewASReqForTGT(cl.Credentials.Domain(), cl.Config, cl.Credentials.CName()) + if err != nil { + return krberror.Errorf(err, krberror.KRBMsgError, "error generating new AS_REQ") + } + ASRep, err := cl.ASExchange(cl.Credentials.Domain(), ASReq, 0) + if err != nil { + return err + } + cl.addSession(ASRep.Ticket, ASRep.DecryptedEncPart) + return nil +} + +// AffirmLogin will only perform an AS exchange with the KDC if the client does not already have a TGT. +func (cl *Client) AffirmLogin() error { + _, endTime, _, _, err := cl.sessionTimes(cl.Credentials.Domain()) + if err != nil || time.Now().UTC().After(endTime) { + err := cl.Login() + if err != nil { + return fmt.Errorf("could not get valid TGT for client's realm: %v", err) + } + } + return nil +} + +// realmLogin obtains or renews a TGT and establishes a session for the realm specified. +func (cl *Client) realmLogin(realm string) error { + if realm == cl.Credentials.Domain() { + return cl.Login() + } + _, endTime, _, _, err := cl.sessionTimes(cl.Credentials.Domain()) + if err != nil || time.Now().UTC().After(endTime) { + err := cl.Login() + if err != nil { + return fmt.Errorf("could not get valid TGT for client's realm: %v", err) + } + } + tgt, skey, err := cl.sessionTGT(cl.Credentials.Domain()) + if err != nil { + return err + } + + spn := types.PrincipalName{ + NameType: nametype.KRB_NT_SRV_INST, + NameString: []string{"krbtgt", realm}, + } + + _, tgsRep, err := cl.TGSREQGenerateAndExchange(spn, cl.Credentials.Domain(), tgt, skey, false) + if err != nil { + return err + } + cl.addSession(tgsRep.Ticket, tgsRep.DecryptedEncPart) + + return nil +} + +// Destroy stops the auto-renewal of all sessions and removes the sessions and cache entries from the client. +func (cl *Client) Destroy() { + creds := credentials.New("", "") + cl.sessions.destroy() + cl.cache.clear() + cl.Credentials = creds + cl.Log("client destroyed") +} + +// Diagnostics runs a set of checks that the client is properly configured and writes details to the io.Writer provided. +func (cl *Client) Diagnostics(w io.Writer) error { + cl.Print(w) + var errs []string + if cl.Credentials.HasKeytab() { + var loginRealmEncTypes []int32 + for _, e := range cl.Credentials.Keytab().Entries { + if e.Principal.Realm == cl.Credentials.Realm() { + loginRealmEncTypes = append(loginRealmEncTypes, e.Key.KeyType) + } + } + for _, et := range cl.Config.LibDefaults.DefaultTktEnctypeIDs { + var etInKt bool + for _, val := range loginRealmEncTypes { + if val == et { + etInKt = true + break + } + } + if !etInKt { + errs = append(errs, fmt.Sprintf("default_tkt_enctypes specifies %d but this enctype is not available in the client's keytab", et)) + } + } + for _, et := range cl.Config.LibDefaults.PreferredPreauthTypes { + var etInKt bool + for _, val := range loginRealmEncTypes { + if int(val) == et { + etInKt = true + break + } + } + if !etInKt { + errs = append(errs, fmt.Sprintf("preferred_preauth_types specifies %d but this enctype is not available in the client's keytab", et)) + } + } + } + udpCnt, udpKDC, err := cl.Config.GetKDCs(cl.Credentials.Realm(), false) + if err != nil { + errs = append(errs, fmt.Sprintf("error when resolving KDCs for UDP communication: %v", err)) + } + if udpCnt < 1 { + errs = append(errs, "no KDCs resolved for communication via UDP.") + } else { + b, _ := json.MarshalIndent(&udpKDC, "", " ") + fmt.Fprintf(w, "UDP KDCs: %s\n", string(b)) + } + tcpCnt, tcpKDC, err := cl.Config.GetKDCs(cl.Credentials.Realm(), false) + if err != nil { + errs = append(errs, fmt.Sprintf("error when resolving KDCs for TCP communication: %v", err)) + } + if tcpCnt < 1 { + errs = append(errs, "no KDCs resolved for communication via TCP.") + } else { + b, _ := json.MarshalIndent(&tcpKDC, "", " ") + fmt.Fprintf(w, "TCP KDCs: %s\n", string(b)) + } + + if errs == nil || len(errs) < 1 { + return nil + } + err = fmt.Errorf(strings.Join(errs, "\n")) + return err +} + +// Print writes the details of the client to the io.Writer provided. +func (cl *Client) Print(w io.Writer) { + c, _ := cl.Credentials.JSON() + fmt.Fprintf(w, "Credentials:\n%s\n", c) + + s, _ := cl.sessions.JSON() + fmt.Fprintf(w, "TGT Sessions:\n%s\n", s) + + c, _ = cl.cache.JSON() + fmt.Fprintf(w, "Service ticket cache:\n%s\n", c) + + s, _ = cl.settings.JSON() + fmt.Fprintf(w, "Settings:\n%s\n", s) + + j, _ := cl.Config.JSON() + fmt.Fprintf(w, "Krb5 config:\n%s\n", j) + + k, _ := cl.Credentials.Keytab().JSON() + fmt.Fprintf(w, "Keytab:\n%s\n", k) +} diff --git a/vendor/github.com/jcmturner/gokrb5/v8/client/network.go b/vendor/github.com/jcmturner/gokrb5/v8/client/network.go new file mode 100644 index 000000000000..634f015c214a --- /dev/null +++ b/vendor/github.com/jcmturner/gokrb5/v8/client/network.go @@ -0,0 +1,218 @@ +package client + +import ( + "encoding/binary" + "errors" + "fmt" + "io" + "net" + "strings" + "time" + + "github.com/jcmturner/gokrb5/v8/iana/errorcode" + "github.com/jcmturner/gokrb5/v8/messages" +) + +// SendToKDC performs network actions to send data to the KDC. +func (cl *Client) sendToKDC(b []byte, realm string) ([]byte, error) { + var rb []byte + if cl.Config.LibDefaults.UDPPreferenceLimit == 1 { + //1 means we should always use TCP + rb, errtcp := cl.sendKDCTCP(realm, b) + if errtcp != nil { + if e, ok := errtcp.(messages.KRBError); ok { + return rb, e + } + return rb, fmt.Errorf("communication error with KDC via TCP: %v", errtcp) + } + return rb, nil + } + if len(b) <= cl.Config.LibDefaults.UDPPreferenceLimit { + //Try UDP first, TCP second + rb, errudp := cl.sendKDCUDP(realm, b) + if errudp != nil { + if e, ok := errudp.(messages.KRBError); ok && e.ErrorCode != errorcode.KRB_ERR_RESPONSE_TOO_BIG { + // Got a KRBError from KDC + // If this is not a KRB_ERR_RESPONSE_TOO_BIG we will return immediately otherwise will try TCP. + return rb, e + } + // Try TCP + r, errtcp := cl.sendKDCTCP(realm, b) + if errtcp != nil { + if e, ok := errtcp.(messages.KRBError); ok { + // Got a KRBError + return r, e + } + return r, fmt.Errorf("failed to communicate with KDC. Attempts made with UDP (%v) and then TCP (%v)", errudp, errtcp) + } + rb = r + } + return rb, nil + } + //Try TCP first, UDP second + rb, errtcp := cl.sendKDCTCP(realm, b) + if errtcp != nil { + if e, ok := errtcp.(messages.KRBError); ok { + // Got a KRBError from KDC so returning and not trying UDP. + return rb, e + } + rb, errudp := cl.sendKDCUDP(realm, b) + if errudp != nil { + if e, ok := errudp.(messages.KRBError); ok { + // Got a KRBError + return rb, e + } + return rb, fmt.Errorf("failed to communicate with KDC. Attempts made with TCP (%v) and then UDP (%v)", errtcp, errudp) + } + } + return rb, nil +} + +// sendKDCUDP sends bytes to the KDC via UDP. +func (cl *Client) sendKDCUDP(realm string, b []byte) ([]byte, error) { + var r []byte + _, kdcs, err := cl.Config.GetKDCs(realm, false) + if err != nil { + return r, err + } + r, err = dialSendUDP(kdcs, b) + if err != nil { + return r, err + } + return checkForKRBError(r) +} + +// dialSendUDP establishes a UDP connection to a KDC. +func dialSendUDP(kdcs map[int]string, b []byte) ([]byte, error) { + var errs []string + for i := 1; i <= len(kdcs); i++ { + udpAddr, err := net.ResolveUDPAddr("udp", kdcs[i]) + if err != nil { + errs = append(errs, fmt.Sprintf("error resolving KDC address: %v", err)) + continue + } + + conn, err := net.DialTimeout("udp", udpAddr.String(), 5*time.Second) + if err != nil { + errs = append(errs, fmt.Sprintf("error setting dial timeout on connection to %s: %v", kdcs[i], err)) + continue + } + if err := conn.SetDeadline(time.Now().Add(5 * time.Second)); err != nil { + errs = append(errs, fmt.Sprintf("error setting deadline on connection to %s: %v", kdcs[i], err)) + continue + } + // conn is guaranteed to be a UDPConn + rb, err := sendUDP(conn.(*net.UDPConn), b) + if err != nil { + errs = append(errs, fmt.Sprintf("error sneding to %s: %v", kdcs[i], err)) + continue + } + return rb, nil + } + return nil, fmt.Errorf("error sending to a KDC: %s", strings.Join(errs, "; ")) +} + +// sendUDP sends bytes to connection over UDP. +func sendUDP(conn *net.UDPConn, b []byte) ([]byte, error) { + var r []byte + defer conn.Close() + _, err := conn.Write(b) + if err != nil { + return r, fmt.Errorf("error sending to (%s): %v", conn.RemoteAddr().String(), err) + } + udpbuf := make([]byte, 4096) + n, _, err := conn.ReadFrom(udpbuf) + r = udpbuf[:n] + if err != nil { + return r, fmt.Errorf("sending over UDP failed to %s: %v", conn.RemoteAddr().String(), err) + } + if len(r) < 1 { + return r, fmt.Errorf("no response data from %s", conn.RemoteAddr().String()) + } + return r, nil +} + +// sendKDCTCP sends bytes to the KDC via TCP. +func (cl *Client) sendKDCTCP(realm string, b []byte) ([]byte, error) { + var r []byte + _, kdcs, err := cl.Config.GetKDCs(realm, true) + if err != nil { + return r, err + } + r, err = dialSendTCP(kdcs, b) + if err != nil { + return r, err + } + return checkForKRBError(r) +} + +// dialKDCTCP establishes a TCP connection to a KDC. +func dialSendTCP(kdcs map[int]string, b []byte) ([]byte, error) { + var errs []string + for i := 1; i <= len(kdcs); i++ { + tcpAddr, err := net.ResolveTCPAddr("tcp", kdcs[i]) + if err != nil { + errs = append(errs, fmt.Sprintf("error resolving KDC address: %v", err)) + continue + } + + conn, err := net.DialTimeout("tcp", tcpAddr.String(), 5*time.Second) + if err != nil { + errs = append(errs, fmt.Sprintf("error setting dial timeout on connection to %s: %v", kdcs[i], err)) + continue + } + if err := conn.SetDeadline(time.Now().Add(5 * time.Second)); err != nil { + errs = append(errs, fmt.Sprintf("error setting deadline on connection to %s: %v", kdcs[i], err)) + continue + } + // conn is guaranteed to be a TCPConn + rb, err := sendTCP(conn.(*net.TCPConn), b) + if err != nil { + errs = append(errs, fmt.Sprintf("error sneding to %s: %v", kdcs[i], err)) + continue + } + return rb, nil + } + return nil, errors.New("error in getting a TCP connection to any of the KDCs") +} + +// sendTCP sends bytes to connection over TCP. +func sendTCP(conn *net.TCPConn, b []byte) ([]byte, error) { + defer conn.Close() + var r []byte + // RFC 4120 7.2.2 specifies the first 4 bytes indicate the length of the message in big endian order. + hb := make([]byte, 4, 4) + binary.BigEndian.PutUint32(hb, uint32(len(b))) + b = append(hb, b...) + + _, err := conn.Write(b) + if err != nil { + return r, fmt.Errorf("error sending to KDC (%s): %v", conn.RemoteAddr().String(), err) + } + + sh := make([]byte, 4, 4) + _, err = conn.Read(sh) + if err != nil { + return r, fmt.Errorf("error reading response size header: %v", err) + } + s := binary.BigEndian.Uint32(sh) + + rb := make([]byte, s, s) + _, err = io.ReadFull(conn, rb) + if err != nil { + return r, fmt.Errorf("error reading response: %v", err) + } + if len(rb) < 1 { + return r, fmt.Errorf("no response data from KDC %s", conn.RemoteAddr().String()) + } + return rb, nil +} + +// checkForKRBError checks if the response bytes from the KDC are a KRBError. +func checkForKRBError(b []byte) ([]byte, error) { + var KRBErr messages.KRBError + if err := KRBErr.Unmarshal(b); err == nil { + return b, KRBErr + } + return b, nil +} diff --git a/vendor/github.com/jcmturner/gokrb5/v8/client/passwd.go b/vendor/github.com/jcmturner/gokrb5/v8/client/passwd.go new file mode 100644 index 000000000000..fe20559ca12f --- /dev/null +++ b/vendor/github.com/jcmturner/gokrb5/v8/client/passwd.go @@ -0,0 +1,75 @@ +package client + +import ( + "fmt" + + "github.com/jcmturner/gokrb5/v8/kadmin" + "github.com/jcmturner/gokrb5/v8/messages" +) + +// Kpasswd server response codes. +const ( + KRB5_KPASSWD_SUCCESS = 0 + KRB5_KPASSWD_MALFORMED = 1 + KRB5_KPASSWD_HARDERROR = 2 + KRB5_KPASSWD_AUTHERROR = 3 + KRB5_KPASSWD_SOFTERROR = 4 + KRB5_KPASSWD_ACCESSDENIED = 5 + KRB5_KPASSWD_BAD_VERSION = 6 + KRB5_KPASSWD_INITIAL_FLAG_NEEDED = 7 +) + +// ChangePasswd changes the password of the client to the value provided. +func (cl *Client) ChangePasswd(newPasswd string) (bool, error) { + ASReq, err := messages.NewASReqForChgPasswd(cl.Credentials.Domain(), cl.Config, cl.Credentials.CName()) + if err != nil { + return false, err + } + ASRep, err := cl.ASExchange(cl.Credentials.Domain(), ASReq, 0) + if err != nil { + return false, err + } + + msg, key, err := kadmin.ChangePasswdMsg(cl.Credentials.CName(), cl.Credentials.Domain(), newPasswd, ASRep.Ticket, ASRep.DecryptedEncPart.Key) + if err != nil { + return false, err + } + r, err := cl.sendToKPasswd(msg) + if err != nil { + return false, err + } + err = r.Decrypt(key) + if err != nil { + return false, err + } + if r.ResultCode != KRB5_KPASSWD_SUCCESS { + return false, fmt.Errorf("error response from kadmin: code: %d; result: %s; krberror: %v", r.ResultCode, r.Result, r.KRBError) + } + cl.Credentials.WithPassword(newPasswd) + return true, nil +} + +func (cl *Client) sendToKPasswd(msg kadmin.Request) (r kadmin.Reply, err error) { + _, kps, err := cl.Config.GetKpasswdServers(cl.Credentials.Domain(), true) + if err != nil { + return + } + b, err := msg.Marshal() + if err != nil { + return + } + var rb []byte + if len(b) <= cl.Config.LibDefaults.UDPPreferenceLimit { + rb, err = dialSendUDP(kps, b) + if err != nil { + return + } + } else { + rb, err = dialSendTCP(kps, b) + if err != nil { + return + } + } + err = r.Unmarshal(rb) + return +} diff --git a/vendor/github.com/jcmturner/gokrb5/v8/client/session.go b/vendor/github.com/jcmturner/gokrb5/v8/client/session.go new file mode 100644 index 000000000000..f7654d0d07eb --- /dev/null +++ b/vendor/github.com/jcmturner/gokrb5/v8/client/session.go @@ -0,0 +1,295 @@ +package client + +import ( + "encoding/json" + "fmt" + "sort" + "strings" + "sync" + "time" + + "github.com/jcmturner/gokrb5/v8/iana/nametype" + "github.com/jcmturner/gokrb5/v8/krberror" + "github.com/jcmturner/gokrb5/v8/messages" + "github.com/jcmturner/gokrb5/v8/types" +) + +// sessions hold TGTs and are keyed on the realm name +type sessions struct { + Entries map[string]*session + mux sync.RWMutex +} + +// destroy erases all sessions +func (s *sessions) destroy() { + s.mux.Lock() + defer s.mux.Unlock() + for k, e := range s.Entries { + e.destroy() + delete(s.Entries, k) + } +} + +// update replaces a session with the one provided or adds it as a new one +func (s *sessions) update(sess *session) { + s.mux.Lock() + defer s.mux.Unlock() + // if a session already exists for this, cancel its auto renew. + if i, ok := s.Entries[sess.realm]; ok { + if i != sess { + // Session in the sessions cache is not the same as one provided. + // Cancel the one in the cache and add this one. + i.mux.Lock() + defer i.mux.Unlock() + i.cancel <- true + s.Entries[sess.realm] = sess + return + } + } + // No session for this realm was found so just add it + s.Entries[sess.realm] = sess +} + +// get returns the session for the realm specified +func (s *sessions) get(realm string) (*session, bool) { + s.mux.RLock() + defer s.mux.RUnlock() + sess, ok := s.Entries[realm] + return sess, ok +} + +// session holds the TGT details for a realm +type session struct { + realm string + authTime time.Time + endTime time.Time + renewTill time.Time + tgt messages.Ticket + sessionKey types.EncryptionKey + sessionKeyExpiration time.Time + cancel chan bool + mux sync.RWMutex +} + +// jsonSession is used to enable marshaling some information of a session in a JSON format +type jsonSession struct { + Realm string + AuthTime time.Time + EndTime time.Time + RenewTill time.Time + SessionKeyExpiration time.Time +} + +// AddSession adds a session for a realm with a TGT to the client's session cache. +// A goroutine is started to automatically renew the TGT before expiry. +func (cl *Client) addSession(tgt messages.Ticket, dep messages.EncKDCRepPart) { + if strings.ToLower(tgt.SName.NameString[0]) != "krbtgt" { + // Not a TGT + return + } + realm := tgt.SName.NameString[len(tgt.SName.NameString)-1] + s := &session{ + realm: realm, + authTime: dep.AuthTime, + endTime: dep.EndTime, + renewTill: dep.RenewTill, + tgt: tgt, + sessionKey: dep.Key, + sessionKeyExpiration: dep.KeyExpiration, + } + cl.sessions.update(s) + cl.enableAutoSessionRenewal(s) + cl.Log("TGT session added for %s (EndTime: %v)", realm, dep.EndTime) +} + +// update overwrites the session details with those from the TGT and decrypted encPart +func (s *session) update(tgt messages.Ticket, dep messages.EncKDCRepPart) { + s.mux.Lock() + defer s.mux.Unlock() + s.authTime = dep.AuthTime + s.endTime = dep.EndTime + s.renewTill = dep.RenewTill + s.tgt = tgt + s.sessionKey = dep.Key + s.sessionKeyExpiration = dep.KeyExpiration +} + +// destroy will cancel any auto renewal of the session and set the expiration times to the current time +func (s *session) destroy() { + s.mux.Lock() + defer s.mux.Unlock() + if s.cancel != nil { + s.cancel <- true + } + s.endTime = time.Now().UTC() + s.renewTill = s.endTime + s.sessionKeyExpiration = s.endTime +} + +// valid informs if the TGT is still within the valid time window +func (s *session) valid() bool { + s.mux.RLock() + defer s.mux.RUnlock() + t := time.Now().UTC() + if t.Before(s.endTime) && s.authTime.Before(t) { + return true + } + return false +} + +// tgtDetails is a thread safe way to get the session's realm, TGT and session key values +func (s *session) tgtDetails() (string, messages.Ticket, types.EncryptionKey) { + s.mux.RLock() + defer s.mux.RUnlock() + return s.realm, s.tgt, s.sessionKey +} + +// timeDetails is a thread safe way to get the session's validity time values +func (s *session) timeDetails() (string, time.Time, time.Time, time.Time, time.Time) { + s.mux.RLock() + defer s.mux.RUnlock() + return s.realm, s.authTime, s.endTime, s.renewTill, s.sessionKeyExpiration +} + +// JSON return information about the held sessions in a JSON format. +func (s *sessions) JSON() (string, error) { + s.mux.RLock() + defer s.mux.RUnlock() + var js []jsonSession + keys := make([]string, 0, len(s.Entries)) + for k := range s.Entries { + keys = append(keys, k) + } + sort.Strings(keys) + for _, k := range keys { + r, at, et, rt, kt := s.Entries[k].timeDetails() + j := jsonSession{ + Realm: r, + AuthTime: at, + EndTime: et, + RenewTill: rt, + SessionKeyExpiration: kt, + } + js = append(js, j) + } + b, err := json.MarshalIndent(js, "", " ") + if err != nil { + return "", err + } + return string(b), nil +} + +// enableAutoSessionRenewal turns on the automatic renewal for the client's TGT session. +func (cl *Client) enableAutoSessionRenewal(s *session) { + var timer *time.Timer + s.mux.Lock() + s.cancel = make(chan bool, 1) + s.mux.Unlock() + go func(s *session) { + for { + s.mux.RLock() + w := (s.endTime.Sub(time.Now().UTC()) * 5) / 6 + s.mux.RUnlock() + if w < 0 { + return + } + timer = time.NewTimer(w) + select { + case <-timer.C: + renewal, err := cl.refreshSession(s) + if err != nil { + cl.Log("error refreshing session: %v", err) + } + if !renewal && err == nil { + // end this goroutine as there will have been a new login and new auto renewal goroutine created. + return + } + case <-s.cancel: + // cancel has been called. Stop the timer and exit. + timer.Stop() + return + } + } + }(s) +} + +// renewTGT renews the client's TGT session. +func (cl *Client) renewTGT(s *session) error { + realm, tgt, skey := s.tgtDetails() + spn := types.PrincipalName{ + NameType: nametype.KRB_NT_SRV_INST, + NameString: []string{"krbtgt", realm}, + } + _, tgsRep, err := cl.TGSREQGenerateAndExchange(spn, cl.Credentials.Domain(), tgt, skey, true) + if err != nil { + return krberror.Errorf(err, krberror.KRBMsgError, "error renewing TGT for %s", realm) + } + s.update(tgsRep.Ticket, tgsRep.DecryptedEncPart) + cl.sessions.update(s) + cl.Log("TGT session renewed for %s (EndTime: %v)", realm, tgsRep.DecryptedEncPart.EndTime) + return nil +} + +// refreshSession updates either through renewal or creating a new login. +// The boolean indicates if the update was a renewal. +func (cl *Client) refreshSession(s *session) (bool, error) { + s.mux.RLock() + realm := s.realm + renewTill := s.renewTill + s.mux.RUnlock() + cl.Log("refreshing TGT session for %s", realm) + if time.Now().UTC().Before(renewTill) { + err := cl.renewTGT(s) + return true, err + } + err := cl.realmLogin(realm) + return false, err +} + +// ensureValidSession makes sure there is a valid session for the realm +func (cl *Client) ensureValidSession(realm string) error { + s, ok := cl.sessions.get(realm) + if ok { + s.mux.RLock() + d := s.endTime.Sub(s.authTime) / 6 + if s.endTime.Sub(time.Now().UTC()) > d { + s.mux.RUnlock() + return nil + } + s.mux.RUnlock() + _, err := cl.refreshSession(s) + return err + } + return cl.realmLogin(realm) +} + +// sessionTGTDetails is a thread safe way to get the TGT and session key values for a realm +func (cl *Client) sessionTGT(realm string) (tgt messages.Ticket, sessionKey types.EncryptionKey, err error) { + err = cl.ensureValidSession(realm) + if err != nil { + return + } + s, ok := cl.sessions.get(realm) + if !ok { + err = fmt.Errorf("could not find TGT session for %s", realm) + return + } + _, tgt, sessionKey = s.tgtDetails() + return +} + +// sessionTimes provides the timing information with regards to a session for the realm specified. +func (cl *Client) sessionTimes(realm string) (authTime, endTime, renewTime, sessionExp time.Time, err error) { + s, ok := cl.sessions.get(realm) + if !ok { + err = fmt.Errorf("could not find TGT session for %s", realm) + return + } + _, authTime, endTime, renewTime, sessionExp = s.timeDetails() + return +} + +// spnRealm resolves the realm name of a service principal name +func (cl *Client) spnRealm(spn types.PrincipalName) string { + return cl.Config.ResolveRealm(spn.NameString[len(spn.NameString)-1]) +} diff --git a/vendor/github.com/jcmturner/gokrb5/v8/client/settings.go b/vendor/github.com/jcmturner/gokrb5/v8/client/settings.go new file mode 100644 index 000000000000..bcd3945431eb --- /dev/null +++ b/vendor/github.com/jcmturner/gokrb5/v8/client/settings.go @@ -0,0 +1,93 @@ +package client + +import ( + "encoding/json" + "fmt" + "log" +) + +// Settings holds optional client settings. +type Settings struct { + disablePAFXFast bool + assumePreAuthentication bool + preAuthEType int32 + logger *log.Logger +} + +// jsonSettings is used when marshaling the Settings details to JSON format. +type jsonSettings struct { + DisablePAFXFast bool + AssumePreAuthentication bool +} + +// NewSettings creates a new client settings struct. +func NewSettings(settings ...func(*Settings)) *Settings { + s := new(Settings) + for _, set := range settings { + set(s) + } + return s +} + +// DisablePAFXFAST used to configure the client to not use PA_FX_FAST. +// +// s := NewSettings(DisablePAFXFAST(true)) +func DisablePAFXFAST(b bool) func(*Settings) { + return func(s *Settings) { + s.disablePAFXFast = b + } +} + +// DisablePAFXFAST indicates is the client should disable the use of PA_FX_FAST. +func (s *Settings) DisablePAFXFAST() bool { + return s.disablePAFXFast +} + +// AssumePreAuthentication used to configure the client to assume pre-authentication is required. +// +// s := NewSettings(AssumePreAuthentication(true)) +func AssumePreAuthentication(b bool) func(*Settings) { + return func(s *Settings) { + s.assumePreAuthentication = b + } +} + +// AssumePreAuthentication indicates if the client should proactively assume using pre-authentication. +func (s *Settings) AssumePreAuthentication() bool { + return s.assumePreAuthentication +} + +// Logger used to configure client with a logger. +// +// s := NewSettings(kt, Logger(l)) +func Logger(l *log.Logger) func(*Settings) { + return func(s *Settings) { + s.logger = l + } +} + +// Logger returns the client logger instance. +func (s *Settings) Logger() *log.Logger { + return s.logger +} + +// Log will write to the service's logger if it is configured. +func (cl *Client) Log(format string, v ...interface{}) { + if cl.settings.Logger() != nil { + cl.settings.Logger().Output(2, fmt.Sprintf(format, v...)) + } +} + +// JSON returns a JSON representation of the settings. +func (s *Settings) JSON() (string, error) { + js := jsonSettings{ + DisablePAFXFast: s.disablePAFXFast, + AssumePreAuthentication: s.assumePreAuthentication, + } + b, err := json.MarshalIndent(js, "", " ") + if err != nil { + return "", err + } + return string(b), nil + +} diff --git a/vendor/github.com/jcmturner/gokrb5/v8/config/error.go b/vendor/github.com/jcmturner/gokrb5/v8/config/error.go new file mode 100644 index 000000000000..1fbda51f387a --- /dev/null +++ b/vendor/github.com/jcmturner/gokrb5/v8/config/error.go @@ -0,0 +1,30 @@ +package config + +import "fmt" + +// UnsupportedDirective error. +type UnsupportedDirective struct { + text string +} + +// Error implements the error interface for unsupported directives. +func (e UnsupportedDirective) Error() string { + return e.text +} + +// Invalid config error. +type Invalid struct { + text string +} + +// Error implements the error interface for invalid config error. +func (e Invalid) Error() string { + return e.text +} + +// InvalidErrorf creates a new Invalid error. +func InvalidErrorf(format string, a ...interface{}) Invalid { + return Invalid{ + text: fmt.Sprintf("invalid krb5 config "+format, a...), + } +} diff --git a/vendor/github.com/jcmturner/gokrb5/v8/config/hosts.go b/vendor/github.com/jcmturner/gokrb5/v8/config/hosts.go new file mode 100644 index 000000000000..3f22c70c41c6 --- /dev/null +++ b/vendor/github.com/jcmturner/gokrb5/v8/config/hosts.go @@ -0,0 +1,141 @@ +package config + +import ( + "fmt" + "math/rand" + "net" + "strconv" + "strings" + + "github.com/jcmturner/dnsutils/v2" +) + +// GetKDCs returns the count of KDCs available and a map of KDC host names keyed on preference order. +func (c *Config) GetKDCs(realm string, tcp bool) (int, map[int]string, error) { + if realm == "" { + realm = c.LibDefaults.DefaultRealm + } + kdcs := make(map[int]string) + var count int + + // Get the KDCs from the krb5.conf. + var ks []string + for _, r := range c.Realms { + if r.Realm != realm { + continue + } + ks = r.KDC + } + count = len(ks) + + if count > 0 { + // Order the kdcs randomly for preference. + kdcs = randServOrder(ks) + return count, kdcs, nil + } + + if !c.LibDefaults.DNSLookupKDC { + return count, kdcs, fmt.Errorf("no KDCs defined in configuration for realm %s", realm) + } + + // Use DNS to resolve kerberos SRV records. + proto := "udp" + if tcp { + proto = "tcp" + } + index, addrs, err := dnsutils.OrderedSRV("kerberos", proto, realm) + if err != nil { + return count, kdcs, err + } + if len(addrs) < 1 { + return count, kdcs, fmt.Errorf("no KDC SRV records found for realm %s", realm) + } + count = index + for k, v := range addrs { + kdcs[k] = strings.TrimRight(v.Target, ".") + ":" + strconv.Itoa(int(v.Port)) + } + return count, kdcs, nil +} + +// GetKpasswdServers returns the count of kpasswd servers available and a map of kpasswd host names keyed on preference order. +// https://web.mit.edu/kerberos/krb5-latest/doc/admin/conf_files/krb5_conf.html#realms - see kpasswd_server section +func (c *Config) GetKpasswdServers(realm string, tcp bool) (int, map[int]string, error) { + kdcs := make(map[int]string) + var count int + + // Use DNS to resolve kerberos SRV records if configured to do so in krb5.conf. + if c.LibDefaults.DNSLookupKDC { + proto := "udp" + if tcp { + proto = "tcp" + } + c, addrs, err := dnsutils.OrderedSRV("kpasswd", proto, realm) + if err != nil { + return count, kdcs, err + } + if c < 1 { + c, addrs, err = dnsutils.OrderedSRV("kerberos-adm", proto, realm) + if err != nil { + return count, kdcs, err + } + } + if len(addrs) < 1 { + return count, kdcs, fmt.Errorf("no kpasswd or kadmin SRV records found for realm %s", realm) + } + count = c + for k, v := range addrs { + kdcs[k] = strings.TrimRight(v.Target, ".") + ":" + strconv.Itoa(int(v.Port)) + } + } else { + // Get the KDCs from the krb5.conf an order them randomly for preference. + var ks []string + var ka []string + for _, r := range c.Realms { + if r.Realm == realm { + ks = r.KPasswdServer + ka = r.AdminServer + break + } + } + if len(ks) < 1 { + for _, k := range ka { + h, _, err := net.SplitHostPort(k) + if err != nil { + continue + } + ks = append(ks, h+":464") + } + } + count = len(ks) + if count < 1 { + return count, kdcs, fmt.Errorf("no kpasswd or kadmin defined in configuration for realm %s", realm) + } + kdcs = randServOrder(ks) + } + return count, kdcs, nil +} + +func randServOrder(ks []string) map[int]string { + kdcs := make(map[int]string) + count := len(ks) + i := 1 + if count > 1 { + l := len(ks) + for l > 0 { + ri := rand.Intn(l) + kdcs[i] = ks[ri] + if l > 1 { + // Remove the entry from the source slice by swapping with the last entry and truncating + ks[len(ks)-1], ks[ri] = ks[ri], ks[len(ks)-1] + ks = ks[:len(ks)-1] + l = len(ks) + } else { + l = 0 + } + i++ + } + } else { + kdcs[i] = ks[0] + } + return kdcs +} diff --git a/vendor/github.com/jcmturner/gokrb5/v8/config/krb5conf.go b/vendor/github.com/jcmturner/gokrb5/v8/config/krb5conf.go new file mode 100644 index 000000000000..a7638433d5fa --- /dev/null +++ b/vendor/github.com/jcmturner/gokrb5/v8/config/krb5conf.go @@ -0,0 +1,728 @@ +// Package config implements KRB5 client and service configuration as described at https://web.mit.edu/kerberos/krb5-latest/doc/admin/conf_files/krb5_conf.html +package config + +import ( + "bufio" + "encoding/hex" + "encoding/json" + "errors" + "fmt" + "io" + "net" + "os" + "os/user" + "regexp" + "strconv" + "strings" + "time" + + "github.com/jcmturner/gofork/encoding/asn1" + "github.com/jcmturner/gokrb5/v8/iana/etypeID" +) + +// Config represents the KRB5 configuration. +type Config struct { + LibDefaults LibDefaults + Realms []Realm + DomainRealm DomainRealm + //CaPaths + //AppDefaults + //Plugins +} + +// WeakETypeList is a list of encryption types that have been deemed weak. +const WeakETypeList = "des-cbc-crc des-cbc-md4 des-cbc-md5 des-cbc-raw des3-cbc-raw des-hmac-sha1 arcfour-hmac-exp rc4-hmac-exp arcfour-hmac-md5-exp des" + +// New creates a new config struct instance. +func New() *Config { + d := make(DomainRealm) + return &Config{ + LibDefaults: newLibDefaults(), + DomainRealm: d, + } +} + +// LibDefaults represents the [libdefaults] section of the configuration. +type LibDefaults struct { + AllowWeakCrypto bool //default false + // ap_req_checksum_type int //unlikely to support this + Canonicalize bool //default false + CCacheType int //default is 4. unlikely to implement older + Clockskew time.Duration //max allowed skew in seconds, default 300 + //Default_ccache_name string // default /tmp/krb5cc_%{uid} //Not implementing as will hold in memory + DefaultClientKeytabName string //default /usr/local/var/krb5/user/%{euid}/client.keytab + DefaultKeytabName string //default /etc/krb5.keytab + DefaultRealm string + DefaultTGSEnctypes []string //default aes256-cts-hmac-sha1-96 aes128-cts-hmac-sha1-96 des3-cbc-sha1 arcfour-hmac-md5 camellia256-cts-cmac camellia128-cts-cmac des-cbc-crc des-cbc-md5 des-cbc-md4 + DefaultTktEnctypes []string //default aes256-cts-hmac-sha1-96 aes128-cts-hmac-sha1-96 des3-cbc-sha1 arcfour-hmac-md5 camellia256-cts-cmac camellia128-cts-cmac des-cbc-crc des-cbc-md5 des-cbc-md4 + DefaultTGSEnctypeIDs []int32 //default aes256-cts-hmac-sha1-96 aes128-cts-hmac-sha1-96 des3-cbc-sha1 arcfour-hmac-md5 camellia256-cts-cmac camellia128-cts-cmac des-cbc-crc des-cbc-md5 des-cbc-md4 + DefaultTktEnctypeIDs []int32 //default aes256-cts-hmac-sha1-96 aes128-cts-hmac-sha1-96 des3-cbc-sha1 arcfour-hmac-md5 camellia256-cts-cmac camellia128-cts-cmac des-cbc-crc des-cbc-md5 des-cbc-md4 + DNSCanonicalizeHostname bool //default true + DNSLookupKDC bool //default false + DNSLookupRealm bool + ExtraAddresses []net.IP //Not implementing yet + Forwardable bool //default false + IgnoreAcceptorHostname bool //default false + K5LoginAuthoritative bool //default false + K5LoginDirectory string //default user's home directory. Must be owned by the user or root + KDCDefaultOptions asn1.BitString //default 0x00000010 (KDC_OPT_RENEWABLE_OK) + KDCTimeSync int //default 1 + //kdc_req_checksum_type int //unlikely to implement as for very old KDCs + NoAddresses bool //default true + PermittedEnctypes []string //default aes256-cts-hmac-sha1-96 aes128-cts-hmac-sha1-96 des3-cbc-sha1 arcfour-hmac-md5 camellia256-cts-cmac camellia128-cts-cmac des-cbc-crc des-cbc-md5 des-cbc-md4 + PermittedEnctypeIDs []int32 + //plugin_base_dir string //not supporting plugins + PreferredPreauthTypes []int //default “17, 16, 15, 14”, which forces libkrb5 to attempt to use PKINIT if it is supported + Proxiable bool //default false + RDNS bool //default true + RealmTryDomains int //default -1 + RenewLifetime time.Duration //default 0 + SafeChecksumType int //default 8 + TicketLifetime time.Duration //default 1 day + UDPPreferenceLimit int // 1 means to always use tcp. MIT krb5 has a default value of 1465, and it prevents user setting more than 32700. + VerifyAPReqNofail bool //default false +} + +// Create a new LibDefaults struct. +func newLibDefaults() LibDefaults { + uid := "0" + var hdir string + usr, _ := user.Current() + if usr != nil { + uid = usr.Uid + hdir = usr.HomeDir + } + opts := asn1.BitString{} + opts.Bytes, _ = hex.DecodeString("00000010") + opts.BitLength = len(opts.Bytes) * 8 + return LibDefaults{ + CCacheType: 4, + Clockskew: time.Duration(300) * time.Second, + DefaultClientKeytabName: fmt.Sprintf("/usr/local/var/krb5/user/%s/client.keytab", uid), + DefaultKeytabName: "/etc/krb5.keytab", + DefaultTGSEnctypes: []string{"aes256-cts-hmac-sha1-96", "aes128-cts-hmac-sha1-96", "des3-cbc-sha1", "arcfour-hmac-md5", "camellia256-cts-cmac", "camellia128-cts-cmac", "des-cbc-crc", "des-cbc-md5", "des-cbc-md4"}, + DefaultTktEnctypes: []string{"aes256-cts-hmac-sha1-96", "aes128-cts-hmac-sha1-96", "des3-cbc-sha1", "arcfour-hmac-md5", "camellia256-cts-cmac", "camellia128-cts-cmac", "des-cbc-crc", "des-cbc-md5", "des-cbc-md4"}, + DNSCanonicalizeHostname: true, + K5LoginDirectory: hdir, + KDCDefaultOptions: opts, + KDCTimeSync: 1, + NoAddresses: true, + PermittedEnctypes: []string{"aes256-cts-hmac-sha1-96", "aes128-cts-hmac-sha1-96", "des3-cbc-sha1", "arcfour-hmac-md5", "camellia256-cts-cmac", "camellia128-cts-cmac", "des-cbc-crc", "des-cbc-md5", "des-cbc-md4"}, + RDNS: true, + RealmTryDomains: -1, + SafeChecksumType: 8, + TicketLifetime: time.Duration(24) * time.Hour, + UDPPreferenceLimit: 1465, + PreferredPreauthTypes: []int{17, 16, 15, 14}, + } +} + +// Parse the lines of the [libdefaults] section of the configuration into the LibDefaults struct. +func (l *LibDefaults) parseLines(lines []string) error { + for _, line := range lines { + //Remove comments after the values + if idx := strings.IndexAny(line, "#;"); idx != -1 { + line = line[:idx] + } + line = strings.TrimSpace(line) + if line == "" { + continue + } + if !strings.Contains(line, "=") { + return InvalidErrorf("libdefaults section line (%s)", line) + } + + p := strings.Split(line, "=") + key := strings.TrimSpace(strings.ToLower(p[0])) + switch key { + case "allow_weak_crypto": + v, err := parseBoolean(p[1]) + if err != nil { + return InvalidErrorf("libdefaults section line (%s): %v", line, err) + } + l.AllowWeakCrypto = v + case "canonicalize": + v, err := parseBoolean(p[1]) + if err != nil { + return InvalidErrorf("libdefaults section line (%s): %v", line, err) + } + l.Canonicalize = v + case "ccache_type": + p[1] = strings.TrimSpace(p[1]) + v, err := strconv.ParseUint(p[1], 10, 32) + if err != nil || v < 0 || v > 4 { + return InvalidErrorf("libdefaults section line (%s)", line) + } + l.CCacheType = int(v) + case "clockskew": + d, err := parseDuration(p[1]) + if err != nil { + return InvalidErrorf("libdefaults section line (%s): %v", line, err) + } + l.Clockskew = d + case "default_client_keytab_name": + l.DefaultClientKeytabName = strings.TrimSpace(p[1]) + case "default_keytab_name": + l.DefaultKeytabName = strings.TrimSpace(p[1]) + case "default_realm": + l.DefaultRealm = strings.TrimSpace(p[1]) + case "default_tgs_enctypes": + l.DefaultTGSEnctypes = strings.Fields(p[1]) + case "default_tkt_enctypes": + l.DefaultTktEnctypes = strings.Fields(p[1]) + case "dns_canonicalize_hostname": + v, err := parseBoolean(p[1]) + if err != nil { + return InvalidErrorf("libdefaults section line (%s): %v", line, err) + } + l.DNSCanonicalizeHostname = v + case "dns_lookup_kdc": + v, err := parseBoolean(p[1]) + if err != nil { + return InvalidErrorf("libdefaults section line (%s): %v", line, err) + } + l.DNSLookupKDC = v + case "dns_lookup_realm": + v, err := parseBoolean(p[1]) + if err != nil { + return InvalidErrorf("libdefaults section line (%s): %v", line, err) + } + l.DNSLookupRealm = v + case "extra_addresses": + ipStr := strings.TrimSpace(p[1]) + for _, ip := range strings.Split(ipStr, ",") { + if eip := net.ParseIP(ip); eip != nil { + l.ExtraAddresses = append(l.ExtraAddresses, eip) + } + } + case "forwardable": + v, err := parseBoolean(p[1]) + if err != nil { + return InvalidErrorf("libdefaults section line (%s): %v", line, err) + } + l.Forwardable = v + case "ignore_acceptor_hostname": + v, err := parseBoolean(p[1]) + if err != nil { + return InvalidErrorf("libdefaults section line (%s): %v", line, err) + } + l.IgnoreAcceptorHostname = v + case "k5login_authoritative": + v, err := parseBoolean(p[1]) + if err != nil { + return InvalidErrorf("libdefaults section line (%s): %v", line, err) + } + l.K5LoginAuthoritative = v + case "k5login_directory": + l.K5LoginDirectory = strings.TrimSpace(p[1]) + case "kdc_default_options": + v := strings.TrimSpace(p[1]) + v = strings.Replace(v, "0x", "", -1) + b, err := hex.DecodeString(v) + if err != nil { + return InvalidErrorf("libdefaults section line (%s): %v", line, err) + } + l.KDCDefaultOptions.Bytes = b + l.KDCDefaultOptions.BitLength = len(b) * 8 + case "kdc_timesync": + p[1] = strings.TrimSpace(p[1]) + v, err := strconv.ParseInt(p[1], 10, 32) + if err != nil || v < 0 { + return InvalidErrorf("libdefaults section line (%s)", line) + } + l.KDCTimeSync = int(v) + case "noaddresses": + v, err := parseBoolean(p[1]) + if err != nil { + return InvalidErrorf("libdefaults section line (%s): %v", line, err) + } + l.NoAddresses = v + case "permitted_enctypes": + l.PermittedEnctypes = strings.Fields(p[1]) + case "preferred_preauth_types": + p[1] = strings.TrimSpace(p[1]) + t := strings.Split(p[1], ",") + var v []int + for _, s := range t { + i, err := strconv.ParseInt(s, 10, 32) + if err != nil { + return InvalidErrorf("libdefaults section line (%s): %v", line, err) + } + v = append(v, int(i)) + } + l.PreferredPreauthTypes = v + case "proxiable": + v, err := parseBoolean(p[1]) + if err != nil { + return InvalidErrorf("libdefaults section line (%s): %v", line, err) + } + l.Proxiable = v + case "rdns": + v, err := parseBoolean(p[1]) + if err != nil { + return InvalidErrorf("libdefaults section line (%s): %v", line, err) + } + l.RDNS = v + case "realm_try_domains": + p[1] = strings.TrimSpace(p[1]) + v, err := strconv.ParseInt(p[1], 10, 32) + if err != nil || v < -1 { + return InvalidErrorf("libdefaults section line (%s)", line) + } + l.RealmTryDomains = int(v) + case "renew_lifetime": + d, err := parseDuration(p[1]) + if err != nil { + return InvalidErrorf("libdefaults section line (%s): %v", line, err) + } + l.RenewLifetime = d + case "safe_checksum_type": + p[1] = strings.TrimSpace(p[1]) + v, err := strconv.ParseInt(p[1], 10, 32) + if err != nil || v < 0 { + return InvalidErrorf("libdefaults section line (%s)", line) + } + l.SafeChecksumType = int(v) + case "ticket_lifetime": + d, err := parseDuration(p[1]) + if err != nil { + return InvalidErrorf("libdefaults section line (%s): %v", line, err) + } + l.TicketLifetime = d + case "udp_preference_limit": + p[1] = strings.TrimSpace(p[1]) + v, err := strconv.ParseUint(p[1], 10, 32) + if err != nil || v > 32700 { + return InvalidErrorf("libdefaults section line (%s)", line) + } + l.UDPPreferenceLimit = int(v) + case "verify_ap_req_nofail": + v, err := parseBoolean(p[1]) + if err != nil { + return InvalidErrorf("libdefaults section line (%s): %v", line, err) + } + l.VerifyAPReqNofail = v + } + } + l.DefaultTGSEnctypeIDs = parseETypes(l.DefaultTGSEnctypes, l.AllowWeakCrypto) + l.DefaultTktEnctypeIDs = parseETypes(l.DefaultTktEnctypes, l.AllowWeakCrypto) + l.PermittedEnctypeIDs = parseETypes(l.PermittedEnctypes, l.AllowWeakCrypto) + return nil +} + +// Realm represents an entry in the [realms] section of the configuration. +type Realm struct { + Realm string + AdminServer []string + //auth_to_local //Not implementing for now + //auth_to_local_names //Not implementing for now + DefaultDomain string + KDC []string + KPasswdServer []string //default admin_server:464 + MasterKDC []string +} + +// Parse the lines of a [realms] entry into the Realm struct. +func (r *Realm) parseLines(name string, lines []string) (err error) { + r.Realm = name + var adminServerFinal bool + var KDCFinal bool + var kpasswdServerFinal bool + var masterKDCFinal bool + var ignore bool + var c int // counts the depth of blocks within brackets { } + for _, line := range lines { + if ignore && c > 0 && !strings.Contains(line, "{") && !strings.Contains(line, "}") { + continue + } + //Remove comments after the values + if idx := strings.IndexAny(line, "#;"); idx != -1 { + line = line[:idx] + } + line = strings.TrimSpace(line) + if line == "" { + continue + } + if !strings.Contains(line, "=") && !strings.Contains(line, "}") { + return InvalidErrorf("realms section line (%s)", line) + } + if strings.Contains(line, "v4_") { + ignore = true + err = UnsupportedDirective{"v4 configurations are not supported"} + } + if strings.Contains(line, "{") { + c++ + if ignore { + continue + } + } + if strings.Contains(line, "}") { + c-- + if c < 0 { + return InvalidErrorf("unpaired curly brackets") + } + if ignore { + if c < 1 { + c = 0 + ignore = false + } + continue + } + } + + p := strings.Split(line, "=") + key := strings.TrimSpace(strings.ToLower(p[0])) + v := strings.TrimSpace(p[1]) + switch key { + case "admin_server": + appendUntilFinal(&r.AdminServer, v, &adminServerFinal) + case "default_domain": + r.DefaultDomain = v + case "kdc": + if !strings.Contains(v, ":") { + // No port number specified default to 88 + if strings.HasSuffix(v, `*`) { + v = strings.TrimSpace(strings.TrimSuffix(v, `*`)) + ":88*" + } else { + v = strings.TrimSpace(v) + ":88" + } + } + appendUntilFinal(&r.KDC, v, &KDCFinal) + case "kpasswd_server": + appendUntilFinal(&r.KPasswdServer, v, &kpasswdServerFinal) + case "master_kdc": + appendUntilFinal(&r.MasterKDC, v, &masterKDCFinal) + } + } + //default for Kpasswd_server = admin_server:464 + if len(r.KPasswdServer) < 1 { + for _, a := range r.AdminServer { + s := strings.Split(a, ":") + r.KPasswdServer = append(r.KPasswdServer, s[0]+":464") + } + } + return +} + +// Parse the lines of the [realms] section of the configuration into an slice of Realm structs. +func parseRealms(lines []string) (realms []Realm, err error) { + var name string + var start int + var c int + for i, l := range lines { + //Remove comments after the values + if idx := strings.IndexAny(l, "#;"); idx != -1 { + l = l[:idx] + } + l = strings.TrimSpace(l) + if l == "" { + continue + } + //if strings.Contains(l, "v4_") { + // return nil, errors.New("v4 configurations are not supported in Realms section") + //} + if strings.Contains(l, "{") { + c++ + if !strings.Contains(l, "=") { + return nil, fmt.Errorf("realm configuration line invalid: %s", l) + } + if c == 1 { + start = i + p := strings.Split(l, "=") + name = strings.TrimSpace(p[0]) + } + } + if strings.Contains(l, "}") { + if c < 1 { + // but not started a block!!! + return nil, errors.New("invalid Realms section in configuration") + } + c-- + if c == 0 { + var r Realm + e := r.parseLines(name, lines[start+1:i]) + if e != nil { + if _, ok := e.(UnsupportedDirective); !ok { + err = e + return + } + err = e + } + realms = append(realms, r) + } + } + } + return +} + +// DomainRealm maps the domains to realms representing the [domain_realm] section of the configuration. +type DomainRealm map[string]string + +// Parse the lines of the [domain_realm] section of the configuration and add to the mapping. +func (d *DomainRealm) parseLines(lines []string) error { + for _, line := range lines { + //Remove comments after the values + if idx := strings.IndexAny(line, "#;"); idx != -1 { + line = line[:idx] + } + if strings.TrimSpace(line) == "" { + continue + } + if !strings.Contains(line, "=") { + return InvalidErrorf("realm line (%s)", line) + } + p := strings.Split(line, "=") + domain := strings.TrimSpace(strings.ToLower(p[0])) + realm := strings.TrimSpace(p[1]) + d.addMapping(domain, realm) + } + return nil +} + +// Add a domain to realm mapping. +func (d *DomainRealm) addMapping(domain, realm string) { + (*d)[domain] = realm +} + +// Delete a domain to realm mapping. +func (d *DomainRealm) deleteMapping(domain, realm string) { + delete(*d, domain) +} + +// ResolveRealm resolves the kerberos realm for the specified domain name from the domain to realm mapping. +// The most specific mapping is returned. +func (c *Config) ResolveRealm(domainName string) string { + domainName = strings.TrimSuffix(domainName, ".") + + // Try to match the entire hostname first + if r, ok := c.DomainRealm[domainName]; ok { + return r + } + + // Try to match all DNS domain parts + periods := strings.Count(domainName, ".") + 1 + for i := 2; i <= periods; i++ { + z := strings.SplitN(domainName, ".", i) + if r, ok := c.DomainRealm["."+z[len(z)-1]]; ok { + return r + } + } + return c.LibDefaults.DefaultRealm +} + +// Load the KRB5 configuration from the specified file path. +func Load(cfgPath string) (*Config, error) { + fh, err := os.Open(cfgPath) + if err != nil { + return nil, errors.New("configuration file could not be opened: " + cfgPath + " " + err.Error()) + } + defer fh.Close() + scanner := bufio.NewScanner(fh) + return NewFromScanner(scanner) +} + +// NewFromString creates a new Config struct from a string. +func NewFromString(s string) (*Config, error) { + reader := strings.NewReader(s) + return NewFromReader(reader) +} + +// NewFromReader creates a new Config struct from an io.Reader. +func NewFromReader(r io.Reader) (*Config, error) { + scanner := bufio.NewScanner(r) + return NewFromScanner(scanner) +} + +// NewFromScanner creates a new Config struct from a bufio.Scanner. +func NewFromScanner(scanner *bufio.Scanner) (*Config, error) { + c := New() + var e error + sections := make(map[int]string) + var sectionLineNum []int + var lines []string + for scanner.Scan() { + // Skip comments and blank lines + if matched, _ := regexp.MatchString(`^\s*(#|;|\n)`, scanner.Text()); matched { + continue + } + if matched, _ := regexp.MatchString(`^\s*\[libdefaults\]\s*`, scanner.Text()); matched { + sections[len(lines)] = "libdefaults" + sectionLineNum = append(sectionLineNum, len(lines)) + continue + } + if matched, _ := regexp.MatchString(`^\s*\[realms\]\s*`, scanner.Text()); matched { + sections[len(lines)] = "realms" + sectionLineNum = append(sectionLineNum, len(lines)) + continue + } + if matched, _ := regexp.MatchString(`^\s*\[domain_realm\]\s*`, scanner.Text()); matched { + sections[len(lines)] = "domain_realm" + sectionLineNum = append(sectionLineNum, len(lines)) + continue + } + if matched, _ := regexp.MatchString(`^\s*\[.*\]\s*`, scanner.Text()); matched { + sections[len(lines)] = "unknown_section" + sectionLineNum = append(sectionLineNum, len(lines)) + continue + } + lines = append(lines, scanner.Text()) + } + for i, start := range sectionLineNum { + var end int + if i+1 >= len(sectionLineNum) { + end = len(lines) + } else { + end = sectionLineNum[i+1] + } + switch section := sections[start]; section { + case "libdefaults": + err := c.LibDefaults.parseLines(lines[start:end]) + if err != nil { + if _, ok := err.(UnsupportedDirective); !ok { + return nil, fmt.Errorf("error processing libdefaults section: %v", err) + } + e = err + } + case "realms": + realms, err := parseRealms(lines[start:end]) + if err != nil { + if _, ok := err.(UnsupportedDirective); !ok { + return nil, fmt.Errorf("error processing realms section: %v", err) + } + e = err + } + c.Realms = realms + case "domain_realm": + err := c.DomainRealm.parseLines(lines[start:end]) + if err != nil { + if _, ok := err.(UnsupportedDirective); !ok { + return nil, fmt.Errorf("error processing domaain_realm section: %v", err) + } + e = err + } + } + } + return c, e +} + +// Parse a space delimited list of ETypes into a list of EType numbers optionally filtering out weak ETypes. +func parseETypes(s []string, w bool) []int32 { + var eti []int32 + for _, et := range s { + if !w { + var weak bool + for _, wet := range strings.Fields(WeakETypeList) { + if et == wet { + weak = true + break + } + } + if weak { + continue + } + } + i := etypeID.EtypeSupported(et) + if i != 0 { + eti = append(eti, i) + } + } + return eti +} + +// Parse a time duration string in the configuration to a golang time.Duration. +func parseDuration(s string) (time.Duration, error) { + s = strings.Replace(strings.TrimSpace(s), " ", "", -1) + + // handle Nd[NmNs] + if strings.Contains(s, "d") { + ds := strings.SplitN(s, "d", 2) + dn, err := strconv.ParseUint(ds[0], 10, 32) + if err != nil { + return time.Duration(0), errors.New("invalid time duration") + } + d := time.Duration(dn*24) * time.Hour + if ds[1] != "" { + dp, err := time.ParseDuration(ds[1]) + if err != nil { + return time.Duration(0), errors.New("invalid time duration") + } + d = d + dp + } + return d, nil + } + + // handle Nm[Ns] + d, err := time.ParseDuration(s) + if err == nil { + return d, nil + } + + // handle N + v, err := strconv.ParseUint(s, 10, 32) + if err == nil && v > 0 { + return time.Duration(v) * time.Second, nil + } + + // handle h:m[:s] + if strings.Contains(s, ":") { + t := strings.Split(s, ":") + if 2 > len(t) || len(t) > 3 { + return time.Duration(0), errors.New("invalid time duration value") + } + var i []int + for _, n := range t { + j, err := strconv.ParseInt(n, 10, 16) + if err != nil { + return time.Duration(0), errors.New("invalid time duration value") + } + i = append(i, int(j)) + } + d := time.Duration(i[0])*time.Hour + time.Duration(i[1])*time.Minute + if len(i) == 3 { + d = d + time.Duration(i[2])*time.Second + } + return d, nil + } + return time.Duration(0), errors.New("invalid time duration value") +} + +// Parse possible boolean values to golang bool. +func parseBoolean(s string) (bool, error) { + s = strings.TrimSpace(s) + v, err := strconv.ParseBool(s) + if err == nil { + return v, nil + } + switch strings.ToLower(s) { + case "yes": + return true, nil + case "y": + return true, nil + case "no": + return false, nil + case "n": + return false, nil + } + return false, errors.New("invalid boolean value") +} + +// Parse array of strings but stop if an asterisk is placed at the end of a line. +func appendUntilFinal(s *[]string, value string, final *bool) { + if *final { + return + } + if last := len(value) - 1; last >= 0 && value[last] == '*' { + *final = true + value = value[:len(value)-1] + } + *s = append(*s, value) +} + +// JSON return details of the config in a JSON format. +func (c *Config) JSON() (string, error) { + b, err := json.MarshalIndent(c, "", " ") + if err != nil { + return "", err + } + return string(b), nil +} diff --git a/vendor/github.com/jcmturner/gokrb5/v8/credentials/ccache.go b/vendor/github.com/jcmturner/gokrb5/v8/credentials/ccache.go new file mode 100644 index 000000000000..c3b35c77a12d --- /dev/null +++ b/vendor/github.com/jcmturner/gokrb5/v8/credentials/ccache.go @@ -0,0 +1,333 @@ +package credentials + +import ( + "bytes" + "encoding/binary" + "errors" + "io/ioutil" + "strings" + "time" + "unsafe" + + "github.com/jcmturner/gofork/encoding/asn1" + "github.com/jcmturner/gokrb5/v8/types" +) + +const ( + headerFieldTagKDCOffset = 1 +) + +// CCache is the file credentials cache as define here: https://web.mit.edu/kerberos/krb5-latest/doc/formats/ccache_file_format.html +type CCache struct { + Version uint8 + Header header + DefaultPrincipal principal + Credentials []*Credential + Path string +} + +type header struct { + length uint16 + fields []headerField +} + +type headerField struct { + tag uint16 + length uint16 + value []byte +} + +// Credential cache entry principal struct. +type principal struct { + Realm string + PrincipalName types.PrincipalName +} + +// Credential holds a Kerberos client's ccache credential information. +type Credential struct { + Client principal + Server principal + Key types.EncryptionKey + AuthTime time.Time + StartTime time.Time + EndTime time.Time + RenewTill time.Time + IsSKey bool + TicketFlags asn1.BitString + Addresses []types.HostAddress + AuthData []types.AuthorizationDataEntry + Ticket []byte + SecondTicket []byte +} + +// LoadCCache loads a credential cache file into a CCache type. +func LoadCCache(cpath string) (*CCache, error) { + c := new(CCache) + b, err := ioutil.ReadFile(cpath) + if err != nil { + return c, err + } + err = c.Unmarshal(b) + return c, err +} + +// Unmarshal a byte slice of credential cache data into CCache type. +func (c *CCache) Unmarshal(b []byte) error { + p := 0 + //The first byte of the file always has the value 5 + if int8(b[p]) != 5 { + return errors.New("Invalid credential cache data. First byte does not equal 5") + } + p++ + //Get credential cache version + //The second byte contains the version number (1 to 4) + c.Version = b[p] + if c.Version < 1 || c.Version > 4 { + return errors.New("Invalid credential cache data. Keytab version is not within 1 to 4") + } + p++ + //Version 1 or 2 of the file format uses native byte order for integer representations. Versions 3 & 4 always uses big-endian byte order + var endian binary.ByteOrder + endian = binary.BigEndian + if (c.Version == 1 || c.Version == 2) && isNativeEndianLittle() { + endian = binary.LittleEndian + } + if c.Version == 4 { + err := parseHeader(b, &p, c, &endian) + if err != nil { + return err + } + } + c.DefaultPrincipal = parsePrincipal(b, &p, c, &endian) + for p < len(b) { + cred, err := parseCredential(b, &p, c, &endian) + if err != nil { + return err + } + c.Credentials = append(c.Credentials, cred) + } + return nil +} + +func parseHeader(b []byte, p *int, c *CCache, e *binary.ByteOrder) error { + if c.Version != 4 { + return errors.New("Credentials cache version is not 4 so there is no header to parse.") + } + h := header{} + h.length = uint16(readInt16(b, p, e)) + for *p <= int(h.length) { + f := headerField{} + f.tag = uint16(readInt16(b, p, e)) + f.length = uint16(readInt16(b, p, e)) + f.value = b[*p : *p+int(f.length)] + *p += int(f.length) + if !f.valid() { + return errors.New("Invalid credential cache header found") + } + h.fields = append(h.fields, f) + } + c.Header = h + return nil +} + +// Parse the Keytab bytes of a principal into a Keytab entry's principal. +func parsePrincipal(b []byte, p *int, c *CCache, e *binary.ByteOrder) (princ principal) { + if c.Version != 1 { + //Name Type is omitted in version 1 + princ.PrincipalName.NameType = readInt32(b, p, e) + } + nc := int(readInt32(b, p, e)) + if c.Version == 1 { + //In version 1 the number of components includes the realm. Minus 1 to make consistent with version 2 + nc-- + } + lenRealm := readInt32(b, p, e) + princ.Realm = string(readBytes(b, p, int(lenRealm), e)) + for i := 0; i < nc; i++ { + l := readInt32(b, p, e) + princ.PrincipalName.NameString = append(princ.PrincipalName.NameString, string(readBytes(b, p, int(l), e))) + } + return princ +} + +func parseCredential(b []byte, p *int, c *CCache, e *binary.ByteOrder) (cred *Credential, err error) { + cred = new(Credential) + cred.Client = parsePrincipal(b, p, c, e) + cred.Server = parsePrincipal(b, p, c, e) + key := types.EncryptionKey{} + key.KeyType = int32(readInt16(b, p, e)) + if c.Version == 3 { + //repeated twice in version 3 + key.KeyType = int32(readInt16(b, p, e)) + } + key.KeyValue = readData(b, p, e) + cred.Key = key + cred.AuthTime = readTimestamp(b, p, e) + cred.StartTime = readTimestamp(b, p, e) + cred.EndTime = readTimestamp(b, p, e) + cred.RenewTill = readTimestamp(b, p, e) + if ik := readInt8(b, p, e); ik == 0 { + cred.IsSKey = false + } else { + cred.IsSKey = true + } + cred.TicketFlags = types.NewKrbFlags() + cred.TicketFlags.Bytes = readBytes(b, p, 4, e) + l := int(readInt32(b, p, e)) + cred.Addresses = make([]types.HostAddress, l, l) + for i := range cred.Addresses { + cred.Addresses[i] = readAddress(b, p, e) + } + l = int(readInt32(b, p, e)) + cred.AuthData = make([]types.AuthorizationDataEntry, l, l) + for i := range cred.AuthData { + cred.AuthData[i] = readAuthDataEntry(b, p, e) + } + cred.Ticket = readData(b, p, e) + cred.SecondTicket = readData(b, p, e) + return +} + +// GetClientPrincipalName returns a PrincipalName type for the client the credentials cache is for. +func (c *CCache) GetClientPrincipalName() types.PrincipalName { + return c.DefaultPrincipal.PrincipalName +} + +// GetClientRealm returns the reals of the client the credentials cache is for. +func (c *CCache) GetClientRealm() string { + return c.DefaultPrincipal.Realm +} + +// GetClientCredentials returns a Credentials object representing the client of the credentials cache. +func (c *CCache) GetClientCredentials() *Credentials { + return &Credentials{ + username: c.DefaultPrincipal.PrincipalName.PrincipalNameString(), + realm: c.GetClientRealm(), + cname: c.DefaultPrincipal.PrincipalName, + } +} + +// Contains tests if the cache contains a credential for the provided server PrincipalName +func (c *CCache) Contains(p types.PrincipalName) bool { + for _, cred := range c.Credentials { + if cred.Server.PrincipalName.Equal(p) { + return true + } + } + return false +} + +// GetEntry returns a specific credential for the PrincipalName provided. +func (c *CCache) GetEntry(p types.PrincipalName) (*Credential, bool) { + cred := new(Credential) + var found bool + for i := range c.Credentials { + if c.Credentials[i].Server.PrincipalName.Equal(p) { + cred = c.Credentials[i] + found = true + break + } + } + if !found { + return cred, false + } + return cred, true +} + +// GetEntries filters out configuration entries an returns a slice of credentials. +func (c *CCache) GetEntries() []*Credential { + creds := make([]*Credential, 0) + for _, cred := range c.Credentials { + // Filter out configuration entries + if strings.HasPrefix(cred.Server.Realm, "X-CACHECONF") { + continue + } + creds = append(creds, cred) + } + return creds +} + +func (h *headerField) valid() bool { + // See https://web.mit.edu/kerberos/krb5-latest/doc/formats/ccache_file_format.html - Header format + switch h.tag { + case headerFieldTagKDCOffset: + if h.length != 8 || len(h.value) != 8 { + return false + } + return true + } + return false +} + +func readData(b []byte, p *int, e *binary.ByteOrder) []byte { + l := readInt32(b, p, e) + return readBytes(b, p, int(l), e) +} + +func readAddress(b []byte, p *int, e *binary.ByteOrder) types.HostAddress { + a := types.HostAddress{} + a.AddrType = int32(readInt16(b, p, e)) + a.Address = readData(b, p, e) + return a +} + +func readAuthDataEntry(b []byte, p *int, e *binary.ByteOrder) types.AuthorizationDataEntry { + a := types.AuthorizationDataEntry{} + a.ADType = int32(readInt16(b, p, e)) + a.ADData = readData(b, p, e) + return a +} + +// Read bytes representing a timestamp. +func readTimestamp(b []byte, p *int, e *binary.ByteOrder) time.Time { + return time.Unix(int64(readInt32(b, p, e)), 0) +} + +// Read bytes representing an eight bit integer. +func readInt8(b []byte, p *int, e *binary.ByteOrder) (i int8) { + buf := bytes.NewBuffer(b[*p : *p+1]) + binary.Read(buf, *e, &i) + *p++ + return +} + +// Read bytes representing a sixteen bit integer. +func readInt16(b []byte, p *int, e *binary.ByteOrder) (i int16) { + buf := bytes.NewBuffer(b[*p : *p+2]) + binary.Read(buf, *e, &i) + *p += 2 + return +} + +// Read bytes representing a thirty two bit integer. +func readInt32(b []byte, p *int, e *binary.ByteOrder) (i int32) { + buf := bytes.NewBuffer(b[*p : *p+4]) + binary.Read(buf, *e, &i) + *p += 4 + return +} + +func readBytes(b []byte, p *int, s int, e *binary.ByteOrder) []byte { + buf := bytes.NewBuffer(b[*p : *p+s]) + r := make([]byte, s) + binary.Read(buf, *e, &r) + *p += s + return r +} + +func isNativeEndianLittle() bool { + var x = 0x012345678 + var p = unsafe.Pointer(&x) + var bp = (*[4]byte)(p) + + var endian bool + if 0x01 == bp[0] { + endian = false + } else if (0x78 & 0xff) == (bp[0] & 0xff) { + endian = true + } else { + // Default to big endian + endian = false + } + return endian +} diff --git a/vendor/github.com/jcmturner/gokrb5/v8/credentials/credentials.go b/vendor/github.com/jcmturner/gokrb5/v8/credentials/credentials.go new file mode 100644 index 000000000000..bddbc7e3e3ab --- /dev/null +++ b/vendor/github.com/jcmturner/gokrb5/v8/credentials/credentials.go @@ -0,0 +1,405 @@ +// Package credentials provides credentials management for Kerberos 5 authentication. +package credentials + +import ( + "bytes" + "encoding/gob" + "encoding/json" + "time" + + "github.com/hashicorp/go-uuid" + "github.com/jcmturner/gokrb5/v8/iana/nametype" + "github.com/jcmturner/gokrb5/v8/keytab" + "github.com/jcmturner/gokrb5/v8/types" +) + +const ( + // AttributeKeyADCredentials assigned number for AD credentials. + AttributeKeyADCredentials = "gokrb5AttributeKeyADCredentials" +) + +// Credentials struct for a user. +// Contains either a keytab, password or both. +// Keytabs are used over passwords if both are defined. +type Credentials struct { + username string + displayName string + realm string + cname types.PrincipalName + keytab *keytab.Keytab + password string + attributes map[string]interface{} + validUntil time.Time + authenticated bool + human bool + authTime time.Time + groupMembership map[string]bool + sessionID string +} + +// marshalCredentials is used to enable marshaling and unmarshaling of credentials +// without having exported fields on the Credentials struct +type marshalCredentials struct { + Username string + DisplayName string + Realm string + CName types.PrincipalName `json:"-"` + Keytab bool + Password bool + Attributes map[string]interface{} `json:"-"` + ValidUntil time.Time + Authenticated bool + Human bool + AuthTime time.Time + GroupMembership map[string]bool `json:"-"` + SessionID string +} + +// ADCredentials contains information obtained from the PAC. +type ADCredentials struct { + EffectiveName string + FullName string + UserID int + PrimaryGroupID int + LogOnTime time.Time + LogOffTime time.Time + PasswordLastSet time.Time + GroupMembershipSIDs []string + LogonDomainName string + LogonDomainID string + LogonServer string +} + +// New creates a new Credentials instance. +func New(username string, realm string) *Credentials { + uid, err := uuid.GenerateUUID() + if err != nil { + uid = "00unique-sess-ions-uuid-unavailable0" + } + return &Credentials{ + username: username, + displayName: username, + realm: realm, + cname: types.NewPrincipalName(nametype.KRB_NT_PRINCIPAL, username), + keytab: keytab.New(), + attributes: make(map[string]interface{}), + groupMembership: make(map[string]bool), + sessionID: uid, + human: true, + } +} + +// NewFromPrincipalName creates a new Credentials instance with the user details provides as a PrincipalName type. +func NewFromPrincipalName(cname types.PrincipalName, realm string) *Credentials { + c := New(cname.PrincipalNameString(), realm) + c.cname = cname + return c +} + +// WithKeytab sets the Keytab in the Credentials struct. +func (c *Credentials) WithKeytab(kt *keytab.Keytab) *Credentials { + c.keytab = kt + c.password = "" + return c +} + +// Keytab returns the credential's Keytab. +func (c *Credentials) Keytab() *keytab.Keytab { + return c.keytab +} + +// HasKeytab queries if the Credentials has a keytab defined. +func (c *Credentials) HasKeytab() bool { + if c.keytab != nil && len(c.keytab.Entries) > 0 { + return true + } + return false +} + +// WithPassword sets the password in the Credentials struct. +func (c *Credentials) WithPassword(password string) *Credentials { + c.password = password + c.keytab = keytab.New() // clear any keytab + return c +} + +// Password returns the credential's password. +func (c *Credentials) Password() string { + return c.password +} + +// HasPassword queries if the Credentials has a password defined. +func (c *Credentials) HasPassword() bool { + if c.password != "" { + return true + } + return false +} + +// SetValidUntil sets the expiry time of the credentials +func (c *Credentials) SetValidUntil(t time.Time) { + c.validUntil = t +} + +// SetADCredentials adds ADCredentials attributes to the credentials +func (c *Credentials) SetADCredentials(a ADCredentials) { + c.SetAttribute(AttributeKeyADCredentials, a) + if a.FullName != "" { + c.SetDisplayName(a.FullName) + } + if a.EffectiveName != "" { + c.SetUserName(a.EffectiveName) + } + for i := range a.GroupMembershipSIDs { + c.AddAuthzAttribute(a.GroupMembershipSIDs[i]) + } +} + +// GetADCredentials returns ADCredentials attributes sorted in the credential +func (c *Credentials) GetADCredentials() ADCredentials { + if a, ok := c.attributes[AttributeKeyADCredentials].(ADCredentials); ok { + return a + } + return ADCredentials{} +} + +// Methods to implement goidentity.Identity interface + +// UserName returns the credential's username. +func (c *Credentials) UserName() string { + return c.username +} + +// SetUserName sets the username value on the credential. +func (c *Credentials) SetUserName(s string) { + c.username = s +} + +// CName returns the credential's client principal name. +func (c *Credentials) CName() types.PrincipalName { + return c.cname +} + +// SetCName sets the client principal name on the credential. +func (c *Credentials) SetCName(pn types.PrincipalName) { + c.cname = pn +} + +// Domain returns the credential's domain. +func (c *Credentials) Domain() string { + return c.realm +} + +// SetDomain sets the domain value on the credential. +func (c *Credentials) SetDomain(s string) { + c.realm = s +} + +// Realm returns the credential's realm. Same as the domain. +func (c *Credentials) Realm() string { + return c.Domain() +} + +// SetRealm sets the realm value on the credential. Same as the domain +func (c *Credentials) SetRealm(s string) { + c.SetDomain(s) +} + +// DisplayName returns the credential's display name. +func (c *Credentials) DisplayName() string { + return c.displayName +} + +// SetDisplayName sets the display name value on the credential. +func (c *Credentials) SetDisplayName(s string) { + c.displayName = s +} + +// Human returns if the credential represents a human or not. +func (c *Credentials) Human() bool { + return c.human +} + +// SetHuman sets the credential as human. +func (c *Credentials) SetHuman(b bool) { + c.human = b +} + +// AuthTime returns the time the credential was authenticated. +func (c *Credentials) AuthTime() time.Time { + return c.authTime +} + +// SetAuthTime sets the time the credential was authenticated. +func (c *Credentials) SetAuthTime(t time.Time) { + c.authTime = t +} + +// AuthzAttributes returns the credentials authorizing attributes. +func (c *Credentials) AuthzAttributes() []string { + s := make([]string, len(c.groupMembership)) + i := 0 + for a := range c.groupMembership { + s[i] = a + i++ + } + return s +} + +// Authenticated indicates if the credential has been successfully authenticated or not. +func (c *Credentials) Authenticated() bool { + return c.authenticated +} + +// SetAuthenticated sets the credential as having been successfully authenticated. +func (c *Credentials) SetAuthenticated(b bool) { + c.authenticated = b +} + +// AddAuthzAttribute adds an authorization attribute to the credential. +func (c *Credentials) AddAuthzAttribute(a string) { + c.groupMembership[a] = true +} + +// RemoveAuthzAttribute removes an authorization attribute from the credential. +func (c *Credentials) RemoveAuthzAttribute(a string) { + if _, ok := c.groupMembership[a]; !ok { + return + } + delete(c.groupMembership, a) +} + +// EnableAuthzAttribute toggles an authorization attribute to an enabled state on the credential. +func (c *Credentials) EnableAuthzAttribute(a string) { + if enabled, ok := c.groupMembership[a]; ok && !enabled { + c.groupMembership[a] = true + } +} + +// DisableAuthzAttribute toggles an authorization attribute to a disabled state on the credential. +func (c *Credentials) DisableAuthzAttribute(a string) { + if enabled, ok := c.groupMembership[a]; ok && enabled { + c.groupMembership[a] = false + } +} + +// Authorized indicates if the credential has the specified authorizing attribute. +func (c *Credentials) Authorized(a string) bool { + if enabled, ok := c.groupMembership[a]; ok && enabled { + return true + } + return false +} + +// SessionID returns the credential's session ID. +func (c *Credentials) SessionID() string { + return c.sessionID +} + +// Expired indicates if the credential has expired. +func (c *Credentials) Expired() bool { + if !c.validUntil.IsZero() && time.Now().UTC().After(c.validUntil) { + return true + } + return false +} + +// ValidUntil returns the credential's valid until date +func (c *Credentials) ValidUntil() time.Time { + return c.validUntil +} + +// Attributes returns the Credentials' attributes map. +func (c *Credentials) Attributes() map[string]interface{} { + return c.attributes +} + +// SetAttribute sets the value of an attribute. +func (c *Credentials) SetAttribute(k string, v interface{}) { + c.attributes[k] = v +} + +// SetAttributes replaces the attributes map with the one provided. +func (c *Credentials) SetAttributes(a map[string]interface{}) { + c.attributes = a +} + +// RemoveAttribute deletes an attribute from the attribute map that has the key provided. +func (c *Credentials) RemoveAttribute(k string) { + delete(c.attributes, k) +} + +// Marshal the Credentials into a byte slice +func (c *Credentials) Marshal() ([]byte, error) { + gob.Register(map[string]interface{}{}) + gob.Register(ADCredentials{}) + buf := new(bytes.Buffer) + enc := gob.NewEncoder(buf) + mc := marshalCredentials{ + Username: c.username, + DisplayName: c.displayName, + Realm: c.realm, + CName: c.cname, + Keytab: c.HasKeytab(), + Password: c.HasPassword(), + Attributes: c.attributes, + ValidUntil: c.validUntil, + Authenticated: c.authenticated, + Human: c.human, + AuthTime: c.authTime, + GroupMembership: c.groupMembership, + SessionID: c.sessionID, + } + err := enc.Encode(&mc) + if err != nil { + return []byte{}, err + } + return buf.Bytes(), nil +} + +// Unmarshal a byte slice into Credentials +func (c *Credentials) Unmarshal(b []byte) error { + gob.Register(map[string]interface{}{}) + gob.Register(ADCredentials{}) + mc := new(marshalCredentials) + buf := bytes.NewBuffer(b) + dec := gob.NewDecoder(buf) + err := dec.Decode(mc) + if err != nil { + return err + } + c.username = mc.Username + c.displayName = mc.DisplayName + c.realm = mc.Realm + c.cname = mc.CName + c.attributes = mc.Attributes + c.validUntil = mc.ValidUntil + c.authenticated = mc.Authenticated + c.human = mc.Human + c.authTime = mc.AuthTime + c.groupMembership = mc.GroupMembership + c.sessionID = mc.SessionID + return nil +} + +// JSON return details of the Credentials in a JSON format. +func (c *Credentials) JSON() (string, error) { + mc := marshalCredentials{ + Username: c.username, + DisplayName: c.displayName, + Realm: c.realm, + CName: c.cname, + Keytab: c.HasKeytab(), + Password: c.HasPassword(), + ValidUntil: c.validUntil, + Authenticated: c.authenticated, + Human: c.human, + AuthTime: c.authTime, + SessionID: c.sessionID, + } + b, err := json.MarshalIndent(mc, "", " ") + if err != nil { + return "", err + } + return string(b), nil +} diff --git a/vendor/github.com/jcmturner/gokrb5/v8/crypto/aes128-cts-hmac-sha1-96.go b/vendor/github.com/jcmturner/gokrb5/v8/crypto/aes128-cts-hmac-sha1-96.go new file mode 100644 index 000000000000..dd8babd5df41 --- /dev/null +++ b/vendor/github.com/jcmturner/gokrb5/v8/crypto/aes128-cts-hmac-sha1-96.go @@ -0,0 +1,129 @@ +package crypto + +import ( + "crypto/aes" + "crypto/hmac" + "crypto/sha1" + "hash" + + "github.com/jcmturner/gokrb5/v8/crypto/common" + "github.com/jcmturner/gokrb5/v8/crypto/rfc3961" + "github.com/jcmturner/gokrb5/v8/crypto/rfc3962" + "github.com/jcmturner/gokrb5/v8/iana/chksumtype" + "github.com/jcmturner/gokrb5/v8/iana/etypeID" +) + +// RFC 3962 + +// Aes128CtsHmacSha96 implements Kerberos encryption type aes128-cts-hmac-sha1-96 +type Aes128CtsHmacSha96 struct { +} + +// GetETypeID returns the EType ID number. +func (e Aes128CtsHmacSha96) GetETypeID() int32 { + return etypeID.AES128_CTS_HMAC_SHA1_96 +} + +// GetHashID returns the checksum type ID number. +func (e Aes128CtsHmacSha96) GetHashID() int32 { + return chksumtype.HMAC_SHA1_96_AES128 +} + +// GetKeyByteSize returns the number of bytes for key of this etype. +func (e Aes128CtsHmacSha96) GetKeyByteSize() int { + return 128 / 8 +} + +// GetKeySeedBitLength returns the number of bits for the seed for key generation. +func (e Aes128CtsHmacSha96) GetKeySeedBitLength() int { + return e.GetKeyByteSize() * 8 +} + +// GetHashFunc returns the hash function for this etype. +func (e Aes128CtsHmacSha96) GetHashFunc() func() hash.Hash { + return sha1.New +} + +// GetMessageBlockByteSize returns the block size for the etype's messages. +func (e Aes128CtsHmacSha96) GetMessageBlockByteSize() int { + return 1 +} + +// GetDefaultStringToKeyParams returns the default key derivation parameters in string form. +func (e Aes128CtsHmacSha96) GetDefaultStringToKeyParams() string { + return "00001000" +} + +// GetConfounderByteSize returns the byte count for confounder to be used during cryptographic operations. +func (e Aes128CtsHmacSha96) GetConfounderByteSize() int { + return aes.BlockSize +} + +// GetHMACBitLength returns the bit count size of the integrity hash. +func (e Aes128CtsHmacSha96) GetHMACBitLength() int { + return 96 +} + +// GetCypherBlockBitLength returns the bit count size of the cypher block. +func (e Aes128CtsHmacSha96) GetCypherBlockBitLength() int { + return aes.BlockSize * 8 +} + +// StringToKey returns a key derived from the string provided. +func (e Aes128CtsHmacSha96) StringToKey(secret string, salt string, s2kparams string) ([]byte, error) { + return rfc3962.StringToKey(secret, salt, s2kparams, e) +} + +// RandomToKey returns a key from the bytes provided. +func (e Aes128CtsHmacSha96) RandomToKey(b []byte) []byte { + return rfc3961.RandomToKey(b) +} + +// EncryptData encrypts the data provided. +func (e Aes128CtsHmacSha96) EncryptData(key, data []byte) ([]byte, []byte, error) { + return rfc3962.EncryptData(key, data, e) +} + +// EncryptMessage encrypts the message provided and concatenates it with the integrity hash to create an encrypted message. +func (e Aes128CtsHmacSha96) EncryptMessage(key, message []byte, usage uint32) ([]byte, []byte, error) { + return rfc3962.EncryptMessage(key, message, usage, e) +} + +// DecryptData decrypts the data provided. +func (e Aes128CtsHmacSha96) DecryptData(key, data []byte) ([]byte, error) { + return rfc3962.DecryptData(key, data, e) +} + +// DecryptMessage decrypts the message provided and verifies the integrity of the message. +func (e Aes128CtsHmacSha96) DecryptMessage(key, ciphertext []byte, usage uint32) ([]byte, error) { + return rfc3962.DecryptMessage(key, ciphertext, usage, e) +} + +// DeriveKey derives a key from the protocol key based on the usage value. +func (e Aes128CtsHmacSha96) DeriveKey(protocolKey, usage []byte) ([]byte, error) { + return rfc3961.DeriveKey(protocolKey, usage, e) +} + +// DeriveRandom generates data needed for key generation. +func (e Aes128CtsHmacSha96) DeriveRandom(protocolKey, usage []byte) ([]byte, error) { + return rfc3961.DeriveRandom(protocolKey, usage, e) +} + +// VerifyIntegrity checks the integrity of the plaintext message. +func (e Aes128CtsHmacSha96) VerifyIntegrity(protocolKey, ct, pt []byte, usage uint32) bool { + return rfc3961.VerifyIntegrity(protocolKey, ct, pt, usage, e) +} + +// GetChecksumHash returns a keyed checksum hash of the bytes provided. +func (e Aes128CtsHmacSha96) GetChecksumHash(protocolKey, data []byte, usage uint32) ([]byte, error) { + return common.GetHash(data, protocolKey, common.GetUsageKc(usage), e) +} + +// VerifyChecksum compares the checksum of the message bytes is the same as the checksum provided. +func (e Aes128CtsHmacSha96) VerifyChecksum(protocolKey, data, chksum []byte, usage uint32) bool { + c, err := e.GetChecksumHash(protocolKey, data, usage) + if err != nil { + return false + } + return hmac.Equal(chksum, c) +} diff --git a/vendor/github.com/jcmturner/gokrb5/v8/crypto/aes128-cts-hmac-sha256-128.go b/vendor/github.com/jcmturner/gokrb5/v8/crypto/aes128-cts-hmac-sha256-128.go new file mode 100644 index 000000000000..b05af7d3d45a --- /dev/null +++ b/vendor/github.com/jcmturner/gokrb5/v8/crypto/aes128-cts-hmac-sha256-128.go @@ -0,0 +1,132 @@ +package crypto + +import ( + "crypto/aes" + "crypto/hmac" + "crypto/sha256" + "hash" + + "github.com/jcmturner/gokrb5/v8/crypto/common" + "github.com/jcmturner/gokrb5/v8/crypto/rfc8009" + "github.com/jcmturner/gokrb5/v8/iana/chksumtype" + "github.com/jcmturner/gokrb5/v8/iana/etypeID" +) + +// RFC https://tools.ietf.org/html/rfc8009 + +// Aes128CtsHmacSha256128 implements Kerberos encryption type aes128-cts-hmac-sha256-128 +type Aes128CtsHmacSha256128 struct { +} + +// GetETypeID returns the EType ID number. +func (e Aes128CtsHmacSha256128) GetETypeID() int32 { + return etypeID.AES128_CTS_HMAC_SHA256_128 +} + +// GetHashID returns the checksum type ID number. +func (e Aes128CtsHmacSha256128) GetHashID() int32 { + return chksumtype.HMAC_SHA256_128_AES128 +} + +// GetKeyByteSize returns the number of bytes for key of this etype. +func (e Aes128CtsHmacSha256128) GetKeyByteSize() int { + return 128 / 8 +} + +// GetKeySeedBitLength returns the number of bits for the seed for key generation. +func (e Aes128CtsHmacSha256128) GetKeySeedBitLength() int { + return e.GetKeyByteSize() * 8 +} + +// GetHashFunc returns the hash function for this etype. +func (e Aes128CtsHmacSha256128) GetHashFunc() func() hash.Hash { + return sha256.New +} + +// GetMessageBlockByteSize returns the block size for the etype's messages. +func (e Aes128CtsHmacSha256128) GetMessageBlockByteSize() int { + return 1 +} + +// GetDefaultStringToKeyParams returns the default key derivation parameters in string form. +func (e Aes128CtsHmacSha256128) GetDefaultStringToKeyParams() string { + return "00008000" +} + +// GetConfounderByteSize returns the byte count for confounder to be used during cryptographic operations. +func (e Aes128CtsHmacSha256128) GetConfounderByteSize() int { + return aes.BlockSize +} + +// GetHMACBitLength returns the bit count size of the integrity hash. +func (e Aes128CtsHmacSha256128) GetHMACBitLength() int { + return 128 +} + +// GetCypherBlockBitLength returns the bit count size of the cypher block. +func (e Aes128CtsHmacSha256128) GetCypherBlockBitLength() int { + return aes.BlockSize * 8 +} + +// StringToKey returns a key derived from the string provided. +func (e Aes128CtsHmacSha256128) StringToKey(secret string, salt string, s2kparams string) ([]byte, error) { + saltp := rfc8009.GetSaltP(salt, "aes128-cts-hmac-sha256-128") + return rfc8009.StringToKey(secret, saltp, s2kparams, e) +} + +// RandomToKey returns a key from the bytes provided. +func (e Aes128CtsHmacSha256128) RandomToKey(b []byte) []byte { + return rfc8009.RandomToKey(b) +} + +// EncryptData encrypts the data provided. +func (e Aes128CtsHmacSha256128) EncryptData(key, data []byte) ([]byte, []byte, error) { + return rfc8009.EncryptData(key, data, e) +} + +// EncryptMessage encrypts the message provided and concatenates it with the integrity hash to create an encrypted message. +func (e Aes128CtsHmacSha256128) EncryptMessage(key, message []byte, usage uint32) ([]byte, []byte, error) { + return rfc8009.EncryptMessage(key, message, usage, e) +} + +// DecryptData decrypts the data provided. +func (e Aes128CtsHmacSha256128) DecryptData(key, data []byte) ([]byte, error) { + return rfc8009.DecryptData(key, data, e) +} + +// DecryptMessage decrypts the message provided and verifies the integrity of the message. +func (e Aes128CtsHmacSha256128) DecryptMessage(key, ciphertext []byte, usage uint32) ([]byte, error) { + return rfc8009.DecryptMessage(key, ciphertext, usage, e) +} + +// DeriveKey derives a key from the protocol key based on the usage value. +func (e Aes128CtsHmacSha256128) DeriveKey(protocolKey, usage []byte) ([]byte, error) { + return rfc8009.DeriveKey(protocolKey, usage, e), nil +} + +// DeriveRandom generates data needed for key generation. +func (e Aes128CtsHmacSha256128) DeriveRandom(protocolKey, usage []byte) ([]byte, error) { + return rfc8009.DeriveRandom(protocolKey, usage, e) +} + +// VerifyIntegrity checks the integrity of the ciphertext message. +// As the hash is calculated over the iv concatenated with the AES cipher output not the plaintext the pt value to this +// interface method is not use. Pass any []byte. +func (e Aes128CtsHmacSha256128) VerifyIntegrity(protocolKey, ct, pt []byte, usage uint32) bool { + // We don't need ib just there for the interface + return rfc8009.VerifyIntegrity(protocolKey, ct, usage, e) +} + +// GetChecksumHash returns a keyed checksum hash of the bytes provided. +func (e Aes128CtsHmacSha256128) GetChecksumHash(protocolKey, data []byte, usage uint32) ([]byte, error) { + return common.GetHash(data, protocolKey, common.GetUsageKc(usage), e) +} + +// VerifyChecksum compares the checksum of the message bytes is the same as the checksum provided. +func (e Aes128CtsHmacSha256128) VerifyChecksum(protocolKey, data, chksum []byte, usage uint32) bool { + c, err := e.GetChecksumHash(protocolKey, data, usage) + if err != nil { + return false + } + return hmac.Equal(chksum, c) +} diff --git a/vendor/github.com/jcmturner/gokrb5/v8/crypto/aes256-cts-hmac-sha1-96.go b/vendor/github.com/jcmturner/gokrb5/v8/crypto/aes256-cts-hmac-sha1-96.go new file mode 100644 index 000000000000..45e439a434e3 --- /dev/null +++ b/vendor/github.com/jcmturner/gokrb5/v8/crypto/aes256-cts-hmac-sha1-96.go @@ -0,0 +1,129 @@ +package crypto + +import ( + "crypto/aes" + "crypto/hmac" + "crypto/sha1" + "hash" + + "github.com/jcmturner/gokrb5/v8/crypto/common" + "github.com/jcmturner/gokrb5/v8/crypto/rfc3961" + "github.com/jcmturner/gokrb5/v8/crypto/rfc3962" + "github.com/jcmturner/gokrb5/v8/iana/chksumtype" + "github.com/jcmturner/gokrb5/v8/iana/etypeID" +) + +// RFC 3962 + +// Aes256CtsHmacSha96 implements Kerberos encryption type aes256-cts-hmac-sha1-96 +type Aes256CtsHmacSha96 struct { +} + +// GetETypeID returns the EType ID number. +func (e Aes256CtsHmacSha96) GetETypeID() int32 { + return etypeID.AES256_CTS_HMAC_SHA1_96 +} + +// GetHashID returns the checksum type ID number. +func (e Aes256CtsHmacSha96) GetHashID() int32 { + return chksumtype.HMAC_SHA1_96_AES256 +} + +// GetKeyByteSize returns the number of bytes for key of this etype. +func (e Aes256CtsHmacSha96) GetKeyByteSize() int { + return 256 / 8 +} + +// GetKeySeedBitLength returns the number of bits for the seed for key generation. +func (e Aes256CtsHmacSha96) GetKeySeedBitLength() int { + return e.GetKeyByteSize() * 8 +} + +// GetHashFunc returns the hash function for this etype. +func (e Aes256CtsHmacSha96) GetHashFunc() func() hash.Hash { + return sha1.New +} + +// GetMessageBlockByteSize returns the block size for the etype's messages. +func (e Aes256CtsHmacSha96) GetMessageBlockByteSize() int { + return 1 +} + +// GetDefaultStringToKeyParams returns the default key derivation parameters in string form. +func (e Aes256CtsHmacSha96) GetDefaultStringToKeyParams() string { + return "00001000" +} + +// GetConfounderByteSize returns the byte count for confounder to be used during cryptographic operations. +func (e Aes256CtsHmacSha96) GetConfounderByteSize() int { + return aes.BlockSize +} + +// GetHMACBitLength returns the bit count size of the integrity hash. +func (e Aes256CtsHmacSha96) GetHMACBitLength() int { + return 96 +} + +// GetCypherBlockBitLength returns the bit count size of the cypher block. +func (e Aes256CtsHmacSha96) GetCypherBlockBitLength() int { + return aes.BlockSize * 8 +} + +// StringToKey returns a key derived from the string provided. +func (e Aes256CtsHmacSha96) StringToKey(secret string, salt string, s2kparams string) ([]byte, error) { + return rfc3962.StringToKey(secret, salt, s2kparams, e) +} + +// RandomToKey returns a key from the bytes provided. +func (e Aes256CtsHmacSha96) RandomToKey(b []byte) []byte { + return rfc3961.RandomToKey(b) +} + +// EncryptData encrypts the data provided. +func (e Aes256CtsHmacSha96) EncryptData(key, data []byte) ([]byte, []byte, error) { + return rfc3962.EncryptData(key, data, e) +} + +// EncryptMessage encrypts the message provided and concatenates it with the integrity hash to create an encrypted message. +func (e Aes256CtsHmacSha96) EncryptMessage(key, message []byte, usage uint32) ([]byte, []byte, error) { + return rfc3962.EncryptMessage(key, message, usage, e) +} + +// DecryptData decrypts the data provided. +func (e Aes256CtsHmacSha96) DecryptData(key, data []byte) ([]byte, error) { + return rfc3962.DecryptData(key, data, e) +} + +// DecryptMessage decrypts the message provided and verifies the integrity of the message. +func (e Aes256CtsHmacSha96) DecryptMessage(key, ciphertext []byte, usage uint32) ([]byte, error) { + return rfc3962.DecryptMessage(key, ciphertext, usage, e) +} + +// DeriveKey derives a key from the protocol key based on the usage value. +func (e Aes256CtsHmacSha96) DeriveKey(protocolKey, usage []byte) ([]byte, error) { + return rfc3961.DeriveKey(protocolKey, usage, e) +} + +// DeriveRandom generates data needed for key generation. +func (e Aes256CtsHmacSha96) DeriveRandom(protocolKey, usage []byte) ([]byte, error) { + return rfc3961.DeriveRandom(protocolKey, usage, e) +} + +// VerifyIntegrity checks the integrity of the plaintext message. +func (e Aes256CtsHmacSha96) VerifyIntegrity(protocolKey, ct, pt []byte, usage uint32) bool { + return rfc3961.VerifyIntegrity(protocolKey, ct, pt, usage, e) +} + +// GetChecksumHash returns a keyed checksum hash of the bytes provided. +func (e Aes256CtsHmacSha96) GetChecksumHash(protocolKey, data []byte, usage uint32) ([]byte, error) { + return common.GetHash(data, protocolKey, common.GetUsageKc(usage), e) +} + +// VerifyChecksum compares the checksum of the message bytes is the same as the checksum provided. +func (e Aes256CtsHmacSha96) VerifyChecksum(protocolKey, data, chksum []byte, usage uint32) bool { + c, err := e.GetChecksumHash(protocolKey, data, usage) + if err != nil { + return false + } + return hmac.Equal(chksum, c) +} diff --git a/vendor/github.com/jcmturner/gokrb5/v8/crypto/aes256-cts-hmac-sha384-192.go b/vendor/github.com/jcmturner/gokrb5/v8/crypto/aes256-cts-hmac-sha384-192.go new file mode 100644 index 000000000000..6a54475930e0 --- /dev/null +++ b/vendor/github.com/jcmturner/gokrb5/v8/crypto/aes256-cts-hmac-sha384-192.go @@ -0,0 +1,132 @@ +package crypto + +import ( + "crypto/aes" + "crypto/hmac" + "crypto/sha512" + "hash" + + "github.com/jcmturner/gokrb5/v8/crypto/common" + "github.com/jcmturner/gokrb5/v8/crypto/rfc8009" + "github.com/jcmturner/gokrb5/v8/iana/chksumtype" + "github.com/jcmturner/gokrb5/v8/iana/etypeID" +) + +// RFC https://tools.ietf.org/html/rfc8009 + +// Aes256CtsHmacSha384192 implements Kerberos encryption type aes256-cts-hmac-sha384-192 +type Aes256CtsHmacSha384192 struct { +} + +// GetETypeID returns the EType ID number. +func (e Aes256CtsHmacSha384192) GetETypeID() int32 { + return etypeID.AES256_CTS_HMAC_SHA384_192 +} + +// GetHashID returns the checksum type ID number. +func (e Aes256CtsHmacSha384192) GetHashID() int32 { + return chksumtype.HMAC_SHA384_192_AES256 +} + +// GetKeyByteSize returns the number of bytes for key of this etype. +func (e Aes256CtsHmacSha384192) GetKeyByteSize() int { + return 192 / 8 +} + +// GetKeySeedBitLength returns the number of bits for the seed for key generation. +func (e Aes256CtsHmacSha384192) GetKeySeedBitLength() int { + return e.GetKeyByteSize() * 8 +} + +// GetHashFunc returns the hash function for this etype. +func (e Aes256CtsHmacSha384192) GetHashFunc() func() hash.Hash { + return sha512.New384 +} + +// GetMessageBlockByteSize returns the block size for the etype's messages. +func (e Aes256CtsHmacSha384192) GetMessageBlockByteSize() int { + return 1 +} + +// GetDefaultStringToKeyParams returns the default key derivation parameters in string form. +func (e Aes256CtsHmacSha384192) GetDefaultStringToKeyParams() string { + return "00008000" +} + +// GetConfounderByteSize returns the byte count for confounder to be used during cryptographic operations. +func (e Aes256CtsHmacSha384192) GetConfounderByteSize() int { + return aes.BlockSize +} + +// GetHMACBitLength returns the bit count size of the integrity hash. +func (e Aes256CtsHmacSha384192) GetHMACBitLength() int { + return 192 +} + +// GetCypherBlockBitLength returns the bit count size of the cypher block. +func (e Aes256CtsHmacSha384192) GetCypherBlockBitLength() int { + return aes.BlockSize * 8 +} + +// StringToKey returns a key derived from the string provided. +func (e Aes256CtsHmacSha384192) StringToKey(secret string, salt string, s2kparams string) ([]byte, error) { + saltp := rfc8009.GetSaltP(salt, "aes256-cts-hmac-sha384-192") + return rfc8009.StringToKey(secret, saltp, s2kparams, e) +} + +// RandomToKey returns a key from the bytes provided. +func (e Aes256CtsHmacSha384192) RandomToKey(b []byte) []byte { + return rfc8009.RandomToKey(b) +} + +// EncryptData encrypts the data provided. +func (e Aes256CtsHmacSha384192) EncryptData(key, data []byte) ([]byte, []byte, error) { + return rfc8009.EncryptData(key, data, e) +} + +// EncryptMessage encrypts the message provided and concatenates it with the integrity hash to create an encrypted message. +func (e Aes256CtsHmacSha384192) EncryptMessage(key, message []byte, usage uint32) ([]byte, []byte, error) { + return rfc8009.EncryptMessage(key, message, usage, e) +} + +// DecryptData decrypts the data provided. +func (e Aes256CtsHmacSha384192) DecryptData(key, data []byte) ([]byte, error) { + return rfc8009.DecryptData(key, data, e) +} + +// DecryptMessage decrypts the message provided and verifies the integrity of the message. +func (e Aes256CtsHmacSha384192) DecryptMessage(key, ciphertext []byte, usage uint32) ([]byte, error) { + return rfc8009.DecryptMessage(key, ciphertext, usage, e) +} + +// DeriveKey derives a key from the protocol key based on the usage value. +func (e Aes256CtsHmacSha384192) DeriveKey(protocolKey, usage []byte) ([]byte, error) { + return rfc8009.DeriveKey(protocolKey, usage, e), nil +} + +// DeriveRandom generates data needed for key generation. +func (e Aes256CtsHmacSha384192) DeriveRandom(protocolKey, usage []byte) ([]byte, error) { + return rfc8009.DeriveRandom(protocolKey, usage, e) +} + +// VerifyIntegrity checks the integrity of the ciphertext message. +// As the hash is calculated over the iv concatenated with the AES cipher output not the plaintext the pt value to this +// interface method is not use. Pass any []byte. +func (e Aes256CtsHmacSha384192) VerifyIntegrity(protocolKey, ct, pt []byte, usage uint32) bool { + // We don't need ib just there for the interface + return rfc8009.VerifyIntegrity(protocolKey, ct, usage, e) +} + +// GetChecksumHash returns a keyed checksum hash of the bytes provided. +func (e Aes256CtsHmacSha384192) GetChecksumHash(protocolKey, data []byte, usage uint32) ([]byte, error) { + return common.GetHash(data, protocolKey, common.GetUsageKc(usage), e) +} + +// VerifyChecksum compares the checksum of the message bytes is the same as the checksum provided. +func (e Aes256CtsHmacSha384192) VerifyChecksum(protocolKey, data, chksum []byte, usage uint32) bool { + c, err := e.GetChecksumHash(protocolKey, data, usage) + if err != nil { + return false + } + return hmac.Equal(chksum, c) +} diff --git a/vendor/github.com/jcmturner/gokrb5/v8/crypto/common/common.go b/vendor/github.com/jcmturner/gokrb5/v8/crypto/common/common.go new file mode 100644 index 000000000000..dab55be7583b --- /dev/null +++ b/vendor/github.com/jcmturner/gokrb5/v8/crypto/common/common.go @@ -0,0 +1,132 @@ +// Package common provides encryption methods common across encryption types +package common + +import ( + "bytes" + "crypto/hmac" + "encoding/binary" + "encoding/hex" + "errors" + "fmt" + + "github.com/jcmturner/gokrb5/v8/crypto/etype" +) + +// ZeroPad pads bytes with zeros to nearest multiple of message size m. +func ZeroPad(b []byte, m int) ([]byte, error) { + if m <= 0 { + return nil, errors.New("Invalid message block size when padding") + } + if b == nil || len(b) == 0 { + return nil, errors.New("Data not valid to pad: Zero size") + } + if l := len(b) % m; l != 0 { + n := m - l + z := make([]byte, n) + b = append(b, z...) + } + return b, nil +} + +// PKCS7Pad pads bytes according to RFC 2315 to nearest multiple of message size m. +func PKCS7Pad(b []byte, m int) ([]byte, error) { + if m <= 0 { + return nil, errors.New("Invalid message block size when padding") + } + if b == nil || len(b) == 0 { + return nil, errors.New("Data not valid to pad: Zero size") + } + n := m - (len(b) % m) + pb := make([]byte, len(b)+n) + copy(pb, b) + copy(pb[len(b):], bytes.Repeat([]byte{byte(n)}, n)) + return pb, nil +} + +// PKCS7Unpad removes RFC 2315 padding from byes where message size is m. +func PKCS7Unpad(b []byte, m int) ([]byte, error) { + if m <= 0 { + return nil, errors.New("invalid message block size when unpadding") + } + if b == nil || len(b) == 0 { + return nil, errors.New("padded data not valid: Zero size") + } + if len(b)%m != 0 { + return nil, errors.New("padded data not valid: Not multiple of message block size") + } + c := b[len(b)-1] + n := int(c) + if n == 0 || n > len(b) { + return nil, errors.New("padded data not valid: Data may not have been padded") + } + for i := 0; i < n; i++ { + if b[len(b)-n+i] != c { + return nil, errors.New("padded data not valid") + } + } + return b[:len(b)-n], nil +} + +// GetHash generates the keyed hash value according to the etype's hash function. +func GetHash(pt, key []byte, usage []byte, etype etype.EType) ([]byte, error) { + k, err := etype.DeriveKey(key, usage) + if err != nil { + return nil, fmt.Errorf("unable to derive key for checksum: %v", err) + } + mac := hmac.New(etype.GetHashFunc(), k) + p := make([]byte, len(pt)) + copy(p, pt) + mac.Write(p) + return mac.Sum(nil)[:etype.GetHMACBitLength()/8], nil +} + +// GetChecksumHash returns a keyed checksum hash of the bytes provided. +func GetChecksumHash(b, key []byte, usage uint32, etype etype.EType) ([]byte, error) { + return GetHash(b, key, GetUsageKc(usage), etype) +} + +// GetIntegrityHash returns a keyed integrity hash of the bytes provided. +func GetIntegrityHash(b, key []byte, usage uint32, etype etype.EType) ([]byte, error) { + return GetHash(b, key, GetUsageKi(usage), etype) +} + +// VerifyChecksum compares the checksum of the msg bytes is the same as the checksum provided. +func VerifyChecksum(key, chksum, msg []byte, usage uint32, etype etype.EType) bool { + //The encrypted message is a concatenation of the encrypted output and the hash HMAC. + expectedMAC, _ := GetChecksumHash(msg, key, usage, etype) + return hmac.Equal(chksum, expectedMAC) +} + +// GetUsageKc returns the checksum key usage value for the usage number un. +// +// See RFC 3961 5.3 key-derivation function definition. +func GetUsageKc(un uint32) []byte { + return getUsage(un, 0x99) +} + +// GetUsageKe returns the encryption key usage value for the usage number un +// +// See RFC 3961 5.3 key-derivation function definition. +func GetUsageKe(un uint32) []byte { + return getUsage(un, 0xAA) +} + +// GetUsageKi returns the integrity key usage value for the usage number un +// +// See RFC 3961 5.3 key-derivation function definition. +func GetUsageKi(un uint32) []byte { + return getUsage(un, 0x55) +} + +func getUsage(un uint32, o byte) []byte { + var buf bytes.Buffer + binary.Write(&buf, binary.BigEndian, un) + return append(buf.Bytes(), o) +} + +// IterationsToS2Kparams converts the number of iterations as an integer to a string representation. +func IterationsToS2Kparams(i uint32) string { + b := make([]byte, 4, 4) + binary.BigEndian.PutUint32(b, i) + return hex.EncodeToString(b) +} diff --git a/vendor/github.com/jcmturner/gokrb5/v8/crypto/crypto.go b/vendor/github.com/jcmturner/gokrb5/v8/crypto/crypto.go new file mode 100644 index 000000000000..5c96ddfdf6fb --- /dev/null +++ b/vendor/github.com/jcmturner/gokrb5/v8/crypto/crypto.go @@ -0,0 +1,175 @@ +// Package crypto implements cryptographic functions for Kerberos 5 implementation. +package crypto + +import ( + "encoding/hex" + "fmt" + + "github.com/jcmturner/gokrb5/v8/crypto/etype" + "github.com/jcmturner/gokrb5/v8/iana/chksumtype" + "github.com/jcmturner/gokrb5/v8/iana/etypeID" + "github.com/jcmturner/gokrb5/v8/iana/patype" + "github.com/jcmturner/gokrb5/v8/types" +) + +// GetEtype returns an instances of the required etype struct for the etype ID. +func GetEtype(id int32) (etype.EType, error) { + switch id { + case etypeID.AES128_CTS_HMAC_SHA1_96: + var et Aes128CtsHmacSha96 + return et, nil + case etypeID.AES256_CTS_HMAC_SHA1_96: + var et Aes256CtsHmacSha96 + return et, nil + case etypeID.AES128_CTS_HMAC_SHA256_128: + var et Aes128CtsHmacSha256128 + return et, nil + case etypeID.AES256_CTS_HMAC_SHA384_192: + var et Aes256CtsHmacSha384192 + return et, nil + case etypeID.DES3_CBC_SHA1_KD: + var et Des3CbcSha1Kd + return et, nil + case etypeID.RC4_HMAC: + var et RC4HMAC + return et, nil + default: + return nil, fmt.Errorf("unknown or unsupported EType: %d", id) + } +} + +// GetChksumEtype returns an instances of the required etype struct for the checksum ID. +func GetChksumEtype(id int32) (etype.EType, error) { + switch id { + case chksumtype.HMAC_SHA1_96_AES128: + var et Aes128CtsHmacSha96 + return et, nil + case chksumtype.HMAC_SHA1_96_AES256: + var et Aes256CtsHmacSha96 + return et, nil + case chksumtype.HMAC_SHA256_128_AES128: + var et Aes128CtsHmacSha256128 + return et, nil + case chksumtype.HMAC_SHA384_192_AES256: + var et Aes256CtsHmacSha384192 + return et, nil + case chksumtype.HMAC_SHA1_DES3_KD: + var et Des3CbcSha1Kd + return et, nil + case chksumtype.KERB_CHECKSUM_HMAC_MD5: + var et RC4HMAC + return et, nil + //case chksumtype.KERB_CHECKSUM_HMAC_MD5_UNSIGNED: + // var et RC4HMAC + // return et, nil + default: + return nil, fmt.Errorf("unknown or unsupported checksum type: %d", id) + } +} + +// GetKeyFromPassword generates an encryption key from the principal's password. +func GetKeyFromPassword(passwd string, cname types.PrincipalName, realm string, etypeID int32, pas types.PADataSequence) (types.EncryptionKey, etype.EType, error) { + var key types.EncryptionKey + et, err := GetEtype(etypeID) + if err != nil { + return key, et, fmt.Errorf("error getting encryption type: %v", err) + } + sk2p := et.GetDefaultStringToKeyParams() + var salt string + var paID int32 + for _, pa := range pas { + switch pa.PADataType { + case patype.PA_PW_SALT: + if paID > pa.PADataType { + continue + } + salt = string(pa.PADataValue) + case patype.PA_ETYPE_INFO: + if paID > pa.PADataType { + continue + } + var eti types.ETypeInfo + err := eti.Unmarshal(pa.PADataValue) + if err != nil { + return key, et, fmt.Errorf("error unmashaling PA Data to PA-ETYPE-INFO2: %v", err) + } + if etypeID != eti[0].EType { + et, err = GetEtype(eti[0].EType) + if err != nil { + return key, et, fmt.Errorf("error getting encryption type: %v", err) + } + } + salt = string(eti[0].Salt) + case patype.PA_ETYPE_INFO2: + if paID > pa.PADataType { + continue + } + var et2 types.ETypeInfo2 + err := et2.Unmarshal(pa.PADataValue) + if err != nil { + return key, et, fmt.Errorf("error unmashalling PA Data to PA-ETYPE-INFO2: %v", err) + } + if etypeID != et2[0].EType { + et, err = GetEtype(et2[0].EType) + if err != nil { + return key, et, fmt.Errorf("error getting encryption type: %v", err) + } + } + if len(et2[0].S2KParams) == 4 { + sk2p = hex.EncodeToString(et2[0].S2KParams) + } + salt = et2[0].Salt + } + } + if salt == "" { + salt = cname.GetSalt(realm) + } + k, err := et.StringToKey(passwd, salt, sk2p) + if err != nil { + return key, et, fmt.Errorf("error deriving key from string: %+v", err) + } + key = types.EncryptionKey{ + KeyType: etypeID, + KeyValue: k, + } + return key, et, nil +} + +// GetEncryptedData encrypts the data provided and returns and EncryptedData type. +// Pass a usage value of zero to use the key provided directly rather than deriving one. +func GetEncryptedData(plainBytes []byte, key types.EncryptionKey, usage uint32, kvno int) (types.EncryptedData, error) { + var ed types.EncryptedData + et, err := GetEtype(key.KeyType) + if err != nil { + return ed, fmt.Errorf("error getting etype: %v", err) + } + _, b, err := et.EncryptMessage(key.KeyValue, plainBytes, usage) + if err != nil { + return ed, err + } + + ed = types.EncryptedData{ + EType: key.KeyType, + Cipher: b, + KVNO: kvno, + } + return ed, nil +} + +// DecryptEncPart decrypts the EncryptedData. +func DecryptEncPart(ed types.EncryptedData, key types.EncryptionKey, usage uint32) ([]byte, error) { + return DecryptMessage(ed.Cipher, key, usage) +} + +// DecryptMessage decrypts the ciphertext and verifies the integrity. +func DecryptMessage(ciphertext []byte, key types.EncryptionKey, usage uint32) ([]byte, error) { + et, err := GetEtype(key.KeyType) + if err != nil { + return []byte{}, fmt.Errorf("error decrypting: %v", err) + } + b, err := et.DecryptMessage(key.KeyValue, ciphertext, usage) + if err != nil { + return nil, fmt.Errorf("error decrypting: %v", err) + } + return b, nil +} diff --git a/vendor/github.com/jcmturner/gokrb5/v8/crypto/des3-cbc-sha1-kd.go b/vendor/github.com/jcmturner/gokrb5/v8/crypto/des3-cbc-sha1-kd.go new file mode 100644 index 000000000000..6e650eb6c19d --- /dev/null +++ b/vendor/github.com/jcmturner/gokrb5/v8/crypto/des3-cbc-sha1-kd.go @@ -0,0 +1,139 @@ +package crypto + +import ( + "crypto/des" + "crypto/hmac" + "crypto/sha1" + "errors" + "hash" + + "github.com/jcmturner/gokrb5/v8/crypto/common" + "github.com/jcmturner/gokrb5/v8/crypto/rfc3961" + "github.com/jcmturner/gokrb5/v8/iana/chksumtype" + "github.com/jcmturner/gokrb5/v8/iana/etypeID" +) + +//RFC: 3961 Section 6.3 + +// Des3CbcSha1Kd implements Kerberos encryption type des3-cbc-hmac-sha1-kd +type Des3CbcSha1Kd struct { +} + +// GetETypeID returns the EType ID number. +func (e Des3CbcSha1Kd) GetETypeID() int32 { + return etypeID.DES3_CBC_SHA1_KD +} + +// GetHashID returns the checksum type ID number. +func (e Des3CbcSha1Kd) GetHashID() int32 { + return chksumtype.HMAC_SHA1_DES3_KD +} + +// GetKeyByteSize returns the number of bytes for key of this etype. +func (e Des3CbcSha1Kd) GetKeyByteSize() int { + return 24 +} + +// GetKeySeedBitLength returns the number of bits for the seed for key generation. +func (e Des3CbcSha1Kd) GetKeySeedBitLength() int { + return 21 * 8 +} + +// GetHashFunc returns the hash function for this etype. +func (e Des3CbcSha1Kd) GetHashFunc() func() hash.Hash { + return sha1.New +} + +// GetMessageBlockByteSize returns the block size for the etype's messages. +func (e Des3CbcSha1Kd) GetMessageBlockByteSize() int { + //For traditional CBC mode with padding, it would be the underlying cipher's block size + return des.BlockSize +} + +// GetDefaultStringToKeyParams returns the default key derivation parameters in string form. +func (e Des3CbcSha1Kd) GetDefaultStringToKeyParams() string { + var s string + return s +} + +// GetConfounderByteSize returns the byte count for confounder to be used during cryptographic operations. +func (e Des3CbcSha1Kd) GetConfounderByteSize() int { + return des.BlockSize +} + +// GetHMACBitLength returns the bit count size of the integrity hash. +func (e Des3CbcSha1Kd) GetHMACBitLength() int { + return e.GetHashFunc()().Size() * 8 +} + +// GetCypherBlockBitLength returns the bit count size of the cypher block. +func (e Des3CbcSha1Kd) GetCypherBlockBitLength() int { + return des.BlockSize * 8 +} + +// StringToKey returns a key derived from the string provided. +func (e Des3CbcSha1Kd) StringToKey(secret string, salt string, s2kparams string) ([]byte, error) { + if s2kparams != "" { + return []byte{}, errors.New("s2kparams must be an empty string") + } + return rfc3961.DES3StringToKey(secret, salt, e) +} + +// RandomToKey returns a key from the bytes provided. +func (e Des3CbcSha1Kd) RandomToKey(b []byte) []byte { + return rfc3961.DES3RandomToKey(b) +} + +// DeriveRandom generates data needed for key generation. +func (e Des3CbcSha1Kd) DeriveRandom(protocolKey, usage []byte) ([]byte, error) { + r, err := rfc3961.DeriveRandom(protocolKey, usage, e) + return r, err +} + +// DeriveKey derives a key from the protocol key based on the usage value. +func (e Des3CbcSha1Kd) DeriveKey(protocolKey, usage []byte) ([]byte, error) { + r, err := e.DeriveRandom(protocolKey, usage) + if err != nil { + return nil, err + } + return e.RandomToKey(r), nil +} + +// EncryptData encrypts the data provided. +func (e Des3CbcSha1Kd) EncryptData(key, data []byte) ([]byte, []byte, error) { + return rfc3961.DES3EncryptData(key, data, e) +} + +// EncryptMessage encrypts the message provided and concatenates it with the integrity hash to create an encrypted message. +func (e Des3CbcSha1Kd) EncryptMessage(key, message []byte, usage uint32) ([]byte, []byte, error) { + return rfc3961.DES3EncryptMessage(key, message, usage, e) +} + +// DecryptData decrypts the data provided. +func (e Des3CbcSha1Kd) DecryptData(key, data []byte) ([]byte, error) { + return rfc3961.DES3DecryptData(key, data, e) +} + +// DecryptMessage decrypts the message provided and verifies the integrity of the message. +func (e Des3CbcSha1Kd) DecryptMessage(key, ciphertext []byte, usage uint32) ([]byte, error) { + return rfc3961.DES3DecryptMessage(key, ciphertext, usage, e) +} + +// VerifyIntegrity checks the integrity of the plaintext message. +func (e Des3CbcSha1Kd) VerifyIntegrity(protocolKey, ct, pt []byte, usage uint32) bool { + return rfc3961.VerifyIntegrity(protocolKey, ct, pt, usage, e) +} + +// GetChecksumHash returns a keyed checksum hash of the bytes provided. +func (e Des3CbcSha1Kd) GetChecksumHash(protocolKey, data []byte, usage uint32) ([]byte, error) { + return common.GetHash(data, protocolKey, common.GetUsageKc(usage), e) +} + +// VerifyChecksum compares the checksum of the message bytes is the same as the checksum provided. +func (e Des3CbcSha1Kd) VerifyChecksum(protocolKey, data, chksum []byte, usage uint32) bool { + c, err := e.GetChecksumHash(protocolKey, data, usage) + if err != nil { + return false + } + return hmac.Equal(chksum, c) +} diff --git a/vendor/github.com/jcmturner/gokrb5/v8/crypto/etype/etype.go b/vendor/github.com/jcmturner/gokrb5/v8/crypto/etype/etype.go new file mode 100644 index 000000000000..ab1496d3f677 --- /dev/null +++ b/vendor/github.com/jcmturner/gokrb5/v8/crypto/etype/etype.go @@ -0,0 +1,29 @@ +// Package etype provides the Kerberos Encryption Type interface +package etype + +import "hash" + +// EType is the interface defining the Encryption Type. +type EType interface { + GetETypeID() int32 + GetHashID() int32 + GetKeyByteSize() int + GetKeySeedBitLength() int + GetDefaultStringToKeyParams() string + StringToKey(string, salt, s2kparams string) ([]byte, error) + RandomToKey(b []byte) []byte + GetHMACBitLength() int + GetMessageBlockByteSize() int + EncryptData(key, data []byte) ([]byte, []byte, error) + EncryptMessage(key, message []byte, usage uint32) ([]byte, []byte, error) + DecryptData(key, data []byte) ([]byte, error) + DecryptMessage(key, ciphertext []byte, usage uint32) ([]byte, error) + GetCypherBlockBitLength() int + GetConfounderByteSize() int + DeriveKey(protocolKey, usage []byte) ([]byte, error) + DeriveRandom(protocolKey, usage []byte) ([]byte, error) + VerifyIntegrity(protocolKey, ct, pt []byte, usage uint32) bool + GetChecksumHash(protocolKey, data []byte, usage uint32) ([]byte, error) + VerifyChecksum(protocolKey, data, chksum []byte, usage uint32) bool + GetHashFunc() func() hash.Hash +} diff --git a/vendor/github.com/jcmturner/gokrb5/v8/crypto/rc4-hmac.go b/vendor/github.com/jcmturner/gokrb5/v8/crypto/rc4-hmac.go new file mode 100644 index 000000000000..42f84b850b97 --- /dev/null +++ b/vendor/github.com/jcmturner/gokrb5/v8/crypto/rc4-hmac.go @@ -0,0 +1,133 @@ +package crypto + +import ( + "bytes" + "crypto/hmac" + "crypto/md5" + "hash" + "io" + + "github.com/jcmturner/gokrb5/v8/crypto/rfc3961" + "github.com/jcmturner/gokrb5/v8/crypto/rfc4757" + "github.com/jcmturner/gokrb5/v8/iana/chksumtype" + "github.com/jcmturner/gokrb5/v8/iana/etypeID" + "golang.org/x/crypto/md4" +) + +// RC4HMAC implements Kerberos encryption type rc4-hmac +type RC4HMAC struct { +} + +// GetETypeID returns the EType ID number. +func (e RC4HMAC) GetETypeID() int32 { + return etypeID.RC4_HMAC +} + +// GetHashID returns the checksum type ID number. +func (e RC4HMAC) GetHashID() int32 { + return chksumtype.KERB_CHECKSUM_HMAC_MD5 +} + +// GetKeyByteSize returns the number of bytes for key of this etype. +func (e RC4HMAC) GetKeyByteSize() int { + return 16 +} + +// GetKeySeedBitLength returns the number of bits for the seed for key generation. +func (e RC4HMAC) GetKeySeedBitLength() int { + return e.GetKeyByteSize() * 8 +} + +// GetHashFunc returns the hash function for this etype. +func (e RC4HMAC) GetHashFunc() func() hash.Hash { + return md5.New +} + +// GetMessageBlockByteSize returns the block size for the etype's messages. +func (e RC4HMAC) GetMessageBlockByteSize() int { + return 1 +} + +// GetDefaultStringToKeyParams returns the default key derivation parameters in string form. +func (e RC4HMAC) GetDefaultStringToKeyParams() string { + return "" +} + +// GetConfounderByteSize returns the byte count for confounder to be used during cryptographic operations. +func (e RC4HMAC) GetConfounderByteSize() int { + return 8 +} + +// GetHMACBitLength returns the bit count size of the integrity hash. +func (e RC4HMAC) GetHMACBitLength() int { + return md5.Size * 8 +} + +// GetCypherBlockBitLength returns the bit count size of the cypher block. +func (e RC4HMAC) GetCypherBlockBitLength() int { + return 8 // doesn't really apply +} + +// StringToKey returns a key derived from the string provided. +func (e RC4HMAC) StringToKey(secret string, salt string, s2kparams string) ([]byte, error) { + return rfc4757.StringToKey(secret) +} + +// RandomToKey returns a key from the bytes provided. +func (e RC4HMAC) RandomToKey(b []byte) []byte { + r := bytes.NewReader(b) + h := md4.New() + io.Copy(h, r) + return h.Sum(nil) +} + +// EncryptData encrypts the data provided. +func (e RC4HMAC) EncryptData(key, data []byte) ([]byte, []byte, error) { + b, err := rfc4757.EncryptData(key, data, e) + return []byte{}, b, err +} + +// EncryptMessage encrypts the message provided and concatenates it with the integrity hash to create an encrypted message. +func (e RC4HMAC) EncryptMessage(key, message []byte, usage uint32) ([]byte, []byte, error) { + b, err := rfc4757.EncryptMessage(key, message, usage, false, e) + return []byte{}, b, err +} + +// DecryptData decrypts the data provided. +func (e RC4HMAC) DecryptData(key, data []byte) ([]byte, error) { + return rfc4757.DecryptData(key, data, e) +} + +// DecryptMessage decrypts the message provided and verifies the integrity of the message. +func (e RC4HMAC) DecryptMessage(key, ciphertext []byte, usage uint32) ([]byte, error) { + return rfc4757.DecryptMessage(key, ciphertext, usage, false, e) +} + +// DeriveKey derives a key from the protocol key based on the usage value. +func (e RC4HMAC) DeriveKey(protocolKey, usage []byte) ([]byte, error) { + return rfc4757.HMAC(protocolKey, usage), nil +} + +// DeriveRandom generates data needed for key generation. +func (e RC4HMAC) DeriveRandom(protocolKey, usage []byte) ([]byte, error) { + return rfc3961.DeriveRandom(protocolKey, usage, e) +} + +// VerifyIntegrity checks the integrity of the plaintext message. +func (e RC4HMAC) VerifyIntegrity(protocolKey, ct, pt []byte, usage uint32) bool { + return rfc4757.VerifyIntegrity(protocolKey, pt, ct, e) +} + +// GetChecksumHash returns a keyed checksum hash of the bytes provided. +func (e RC4HMAC) GetChecksumHash(protocolKey, data []byte, usage uint32) ([]byte, error) { + return rfc4757.Checksum(protocolKey, usage, data) +} + +// VerifyChecksum compares the checksum of the message bytes is the same as the checksum provided. +func (e RC4HMAC) VerifyChecksum(protocolKey, data, chksum []byte, usage uint32) bool { + checksum, err := rfc4757.Checksum(protocolKey, usage, data) + if err != nil { + return false + } + return hmac.Equal(checksum, chksum) +} diff --git a/vendor/github.com/jcmturner/gokrb5/v8/crypto/rfc3961/encryption.go b/vendor/github.com/jcmturner/gokrb5/v8/crypto/rfc3961/encryption.go new file mode 100644 index 000000000000..1383258c7f1b --- /dev/null +++ b/vendor/github.com/jcmturner/gokrb5/v8/crypto/rfc3961/encryption.go @@ -0,0 +1,119 @@ +// Package rfc3961 provides encryption and checksum methods as specified in RFC 3961 +package rfc3961 + +import ( + "crypto/cipher" + "crypto/des" + "crypto/hmac" + "crypto/rand" + "errors" + "fmt" + + "github.com/jcmturner/gokrb5/v8/crypto/common" + "github.com/jcmturner/gokrb5/v8/crypto/etype" +) + +// DES3EncryptData encrypts the data provided using DES3 and methods specific to the etype provided. +func DES3EncryptData(key, data []byte, e etype.EType) ([]byte, []byte, error) { + if len(key) != e.GetKeyByteSize() { + return nil, nil, fmt.Errorf("incorrect keysize: expected: %v actual: %v", e.GetKeyByteSize(), len(key)) + } + data, _ = common.ZeroPad(data, e.GetMessageBlockByteSize()) + + block, err := des.NewTripleDESCipher(key) + if err != nil { + return nil, nil, fmt.Errorf("error creating cipher: %v", err) + } + + //RFC 3961: initial cipher state All bits zero + ivz := make([]byte, des.BlockSize) + + ct := make([]byte, len(data)) + mode := cipher.NewCBCEncrypter(block, ivz) + mode.CryptBlocks(ct, data) + return ct[len(ct)-e.GetMessageBlockByteSize():], ct, nil +} + +// DES3EncryptMessage encrypts the message provided using DES3 and methods specific to the etype provided. +// The encrypted data is concatenated with its integrity hash to create an encrypted message. +func DES3EncryptMessage(key, message []byte, usage uint32, e etype.EType) ([]byte, []byte, error) { + //confounder + c := make([]byte, e.GetConfounderByteSize()) + _, err := rand.Read(c) + if err != nil { + return []byte{}, []byte{}, fmt.Errorf("could not generate random confounder: %v", err) + } + plainBytes := append(c, message...) + plainBytes, _ = common.ZeroPad(plainBytes, e.GetMessageBlockByteSize()) + + // Derive key for encryption from usage + var k []byte + if usage != 0 { + k, err = e.DeriveKey(key, common.GetUsageKe(usage)) + if err != nil { + return []byte{}, []byte{}, fmt.Errorf("error deriving key for encryption: %v", err) + } + } + + iv, b, err := e.EncryptData(k, plainBytes) + if err != nil { + return iv, b, fmt.Errorf("error encrypting data: %v", err) + } + + // Generate and append integrity hash + ih, err := common.GetIntegrityHash(plainBytes, key, usage, e) + if err != nil { + return iv, b, fmt.Errorf("error encrypting data: %v", err) + } + b = append(b, ih...) + return iv, b, nil +} + +// DES3DecryptData decrypts the data provided using DES3 and methods specific to the etype provided. +func DES3DecryptData(key, data []byte, e etype.EType) ([]byte, error) { + if len(key) != e.GetKeyByteSize() { + return []byte{}, fmt.Errorf("incorrect keysize: expected: %v actual: %v", e.GetKeyByteSize(), len(key)) + } + + if len(data) < des.BlockSize || len(data)%des.BlockSize != 0 { + return []byte{}, errors.New("ciphertext is not a multiple of the block size") + } + block, err := des.NewTripleDESCipher(key) + if err != nil { + return []byte{}, fmt.Errorf("error creating cipher: %v", err) + } + pt := make([]byte, len(data)) + ivz := make([]byte, des.BlockSize) + mode := cipher.NewCBCDecrypter(block, ivz) + mode.CryptBlocks(pt, data) + return pt, nil +} + +// DES3DecryptMessage decrypts the message provided using DES3 and methods specific to the etype provided. +// The integrity of the message is also verified. +func DES3DecryptMessage(key, ciphertext []byte, usage uint32, e etype.EType) ([]byte, error) { + //Derive the key + k, err := e.DeriveKey(key, common.GetUsageKe(usage)) + if err != nil { + return nil, fmt.Errorf("error deriving key: %v", err) + } + // Strip off the checksum from the end + b, err := e.DecryptData(k, ciphertext[:len(ciphertext)-e.GetHMACBitLength()/8]) + if err != nil { + return nil, fmt.Errorf("error decrypting: %v", err) + } + //Verify checksum + if !e.VerifyIntegrity(key, ciphertext, b, usage) { + return nil, errors.New("error decrypting: integrity verification failed") + } + //Remove the confounder bytes + return b[e.GetConfounderByteSize():], nil +} + +// VerifyIntegrity verifies the integrity of cipertext bytes ct. +func VerifyIntegrity(key, ct, pt []byte, usage uint32, etype etype.EType) bool { + h := make([]byte, etype.GetHMACBitLength()/8) + copy(h, ct[len(ct)-etype.GetHMACBitLength()/8:]) + expectedMAC, _ := common.GetIntegrityHash(pt, key, usage, etype) + return hmac.Equal(h, expectedMAC) +} diff --git a/vendor/github.com/jcmturner/gokrb5/v8/crypto/rfc3961/keyDerivation.go b/vendor/github.com/jcmturner/gokrb5/v8/crypto/rfc3961/keyDerivation.go new file mode 100644 index 000000000000..ed9b169c5d9e --- /dev/null +++ b/vendor/github.com/jcmturner/gokrb5/v8/crypto/rfc3961/keyDerivation.go @@ -0,0 +1,169 @@ +package rfc3961 + +import ( + "bytes" + + "github.com/jcmturner/gokrb5/v8/crypto/etype" +) + +const ( + prfconstant = "prf" +) + +// DeriveRandom implements the RFC 3961 defined function: DR(Key, Constant) = k-truncate(E(Key, Constant, initial-cipher-state)). +// +// key: base key or protocol key. Likely to be a key from a keytab file. +// +// usage: a constant. +// +// n: block size in bits (not bytes) - note if you use something like aes.BlockSize this is in bytes. +// +// k: key length / key seed length in bits. Eg. for AES256 this value is 256. +// +// e: the encryption etype function to use. +func DeriveRandom(key, usage []byte, e etype.EType) ([]byte, error) { + n := e.GetCypherBlockBitLength() + k := e.GetKeySeedBitLength() + //Ensure the usage constant is at least the size of the cypher block size. Pass it through the nfold algorithm that will "stretch" it if needs be. + nFoldUsage := Nfold(usage, n) + //k-truncate implemented by creating a byte array the size of k (k is in bits hence /8) + out := make([]byte, k/8) + // Keep feeding the output back into the encryption function until it is no longer short than k. + _, K, err := e.EncryptData(key, nFoldUsage) + if err != nil { + return out, err + } + for i := copy(out, K); i < len(out); { + _, K, _ = e.EncryptData(key, K) + i = i + copy(out[i:], K) + } + return out, nil +} + +// DeriveKey derives a key from the protocol key based on the usage and the etype's specific methods. +func DeriveKey(protocolKey, usage []byte, e etype.EType) ([]byte, error) { + r, err := e.DeriveRandom(protocolKey, usage) + if err != nil { + return nil, err + } + return e.RandomToKey(r), nil +} + +// RandomToKey returns a key from the bytes provided according to the definition in RFC 3961. +func RandomToKey(b []byte) []byte { + return b +} + +// DES3RandomToKey returns a key from the bytes provided according to the definition in RFC 3961 for DES3 etypes. +func DES3RandomToKey(b []byte) []byte { + r := fixWeakKey(stretch56Bits(b[:7])) + r2 := fixWeakKey(stretch56Bits(b[7:14])) + r = append(r, r2...) + r3 := fixWeakKey(stretch56Bits(b[14:21])) + r = append(r, r3...) + return r +} + +// DES3StringToKey returns a key derived from the string provided according to the definition in RFC 3961 for DES3 etypes. +func DES3StringToKey(secret, salt string, e etype.EType) ([]byte, error) { + s := secret + salt + tkey := e.RandomToKey(Nfold([]byte(s), e.GetKeySeedBitLength())) + return e.DeriveKey(tkey, []byte("kerberos")) +} + +// PseudoRandom function as defined in RFC 3961 +func PseudoRandom(key, b []byte, e etype.EType) ([]byte, error) { + h := e.GetHashFunc()() + h.Write(b) + tmp := h.Sum(nil)[:e.GetMessageBlockByteSize()] + k, err := e.DeriveKey(key, []byte(prfconstant)) + if err != nil { + return []byte{}, err + } + _, prf, err := e.EncryptData(k, tmp) + if err != nil { + return []byte{}, err + } + return prf, nil +} + +func stretch56Bits(b []byte) []byte { + d := make([]byte, len(b), len(b)) + copy(d, b) + var lb byte + for i, v := range d { + bv, nb := calcEvenParity(v) + d[i] = nb + if bv != 0 { + lb = lb | (1 << uint(i+1)) + } else { + lb = lb &^ (1 << uint(i+1)) + } + } + _, lb = calcEvenParity(lb) + d = append(d, lb) + return d +} + +func calcEvenParity(b byte) (uint8, uint8) { + lowestbit := b & 0x01 + // c counter of 1s in the first 7 bits of the byte + var c int + // Iterate over the highest 7 bits (hence p starts at 1 not zero) and count the 1s. + for p := 1; p < 8; p++ { + v := b & (1 << uint(p)) + if v != 0 { + c++ + } + } + if c%2 == 0 { + //Even number of 1s so set parity to 1 + b = b | 1 + } else { + //Odd number of 1s so set parity to 0 + b = b &^ 1 + } + return lowestbit, b +} + +func fixWeakKey(b []byte) []byte { + if weak(b) { + b[7] ^= 0xF0 + } + return b +} + +func weak(b []byte) bool { + // weak keys from https://nvlpubs.nist.gov/nistpubs/Legacy/SP/nistspecialpublication800-67r1.pdf + weakKeys := [4][]byte{ + {0x01, 0x01, 0x01, 0x01, 0x01, 0x01, 0x01, 0x01}, + {0xFE, 0xFE, 0xFE, 0xFE, 0xFE, 0xFE, 0xFE, 0xFE}, + {0xE0, 0xE0, 0xE0, 0xE0, 0xF1, 0xF1, 0xF1, 0xF1}, + {0x1F, 0x1F, 0x1F, 0x1F, 0x0E, 0x0E, 0x0E, 0x0E}, + } + semiWeakKeys := [12][]byte{ + {0x01, 0x1F, 0x01, 0x1F, 0x01, 0x0E, 0x01, 0x0E}, + {0x1F, 0x01, 0x1F, 0x01, 0x0E, 0x01, 0x0E, 0x01}, + {0x01, 0xE0, 0x01, 0xE0, 0x01, 0xF1, 0x01, 0xF1}, + {0xE0, 0x01, 0xE0, 0x01, 0xF1, 0x01, 0xF1, 0x01}, + {0x01, 0xFE, 0x01, 0xFE, 0x01, 0xFE, 0x01, 0xFE}, + {0xFE, 0x01, 0xFE, 0x01, 0xFE, 0x01, 0xFE, 0x01}, + {0x1F, 0xE0, 0x1F, 0xE0, 0x0E, 0xF1, 0x0E, 0xF1}, + {0xE0, 0x1F, 0xE0, 0x1F, 0xF1, 0x0E, 0xF1, 0x0E}, + {0x1F, 0xFE, 0x1F, 0xFE, 0x0E, 0xFE, 0x0E, 0xFE}, + {0xFE, 0x1F, 0xFE, 0x1F, 0xFE, 0x0E, 0xFE, 0x0E}, + {0xE0, 0xFE, 0xE0, 0xFE, 0xF1, 0xFE, 0xF1, 0xFE}, + {0xFE, 0xE0, 0xFE, 0xE0, 0xFE, 0xF1, 0xFE, 0xF1}, + } + for _, k := range weakKeys { + if bytes.Equal(b, k) { + return true + } + } + for _, k := range semiWeakKeys { + if bytes.Equal(b, k) { + return true + } + } + return false +} diff --git a/vendor/github.com/jcmturner/gokrb5/v8/crypto/rfc3961/nfold.go b/vendor/github.com/jcmturner/gokrb5/v8/crypto/rfc3961/nfold.go new file mode 100644 index 000000000000..9536b1e3e353 --- /dev/null +++ b/vendor/github.com/jcmturner/gokrb5/v8/crypto/rfc3961/nfold.go @@ -0,0 +1,107 @@ +package rfc3961 + +// Implementation of the n-fold algorithm as defined in RFC 3961. + +/* Credits +This golang implementation of nfold used the following project for help with implementation detail. +Although their source is in java it was helpful as a reference implementation of the RFC. +You can find the source code of their open source project along with license information below. +We acknowledge and are grateful to these developers for their contributions to open source + +Project: Apache Directory (http://http://directory.apache.org/) +https://svn.apache.org/repos/asf/directory/apacheds/tags/1.5.1/kerberos-shared/src/main/java/org/apache/directory/server/kerberos/shared/crypto/encryption/NFold.java +License: http://www.apache.org/licenses/LICENSE-2.0 +*/ + +// Nfold expands the key to ensure it is not smaller than one cipher block. +// Defined in RFC 3961. +// +// m input bytes that will be "stretched" to the least common multiple of n bits and the bit length of m. +func Nfold(m []byte, n int) []byte { + k := len(m) * 8 + + //Get the lowest common multiple of the two bit sizes + lcm := lcm(n, k) + relicate := lcm / k + var sumBytes []byte + + for i := 0; i < relicate; i++ { + rotation := 13 * i + sumBytes = append(sumBytes, rotateRight(m, rotation)...) + } + + nfold := make([]byte, n/8) + sum := make([]byte, n/8) + for i := 0; i < lcm/n; i++ { + for j := 0; j < n/8; j++ { + sum[j] = sumBytes[j+(i*len(sum))] + } + nfold = onesComplementAddition(nfold, sum) + } + return nfold +} + +func onesComplementAddition(n1, n2 []byte) []byte { + numBits := len(n1) * 8 + out := make([]byte, numBits/8) + carry := 0 + for i := numBits - 1; i > -1; i-- { + n1b := getBit(&n1, i) + n2b := getBit(&n2, i) + s := n1b + n2b + carry + + if s == 0 || s == 1 { + setBit(&out, i, s) + carry = 0 + } else if s == 2 { + carry = 1 + } else if s == 3 { + setBit(&out, i, 1) + carry = 1 + } + } + if carry == 1 { + carryArray := make([]byte, len(n1)) + carryArray[len(carryArray)-1] = 1 + out = onesComplementAddition(out, carryArray) + } + return out +} + +func rotateRight(b []byte, step int) []byte { + out := make([]byte, len(b)) + bitLen := len(b) * 8 + for i := 0; i < bitLen; i++ { + v := getBit(&b, i) + setBit(&out, (i+step)%bitLen, v) + } + return out +} + +func lcm(x, y int) int { + return (x * y) / gcd(x, y) +} + +func gcd(x, y int) int { + for y != 0 { + x, y = y, x%y + } + return x +} + +func getBit(b *[]byte, p int) int { + pByte := p / 8 + pBit := uint(p % 8) + vByte := (*b)[pByte] + vInt := int(vByte >> (8 - (pBit + 1)) & 0x0001) + return vInt +} + +func setBit(b *[]byte, p, v int) { + pByte := p / 8 + pBit := uint(p % 8) + oldByte := (*b)[pByte] + var newByte byte + newByte = byte(v<<(8-(pBit+1))) | oldByte + (*b)[pByte] = newByte +} diff --git a/vendor/github.com/jcmturner/gokrb5/v8/crypto/rfc3962/encryption.go b/vendor/github.com/jcmturner/gokrb5/v8/crypto/rfc3962/encryption.go new file mode 100644 index 000000000000..5ff89e85b03c --- /dev/null +++ b/vendor/github.com/jcmturner/gokrb5/v8/crypto/rfc3962/encryption.go @@ -0,0 +1,89 @@ +// Package rfc3962 provides encryption and checksum methods as specified in RFC 3962 +package rfc3962 + +import ( + "crypto/rand" + "errors" + "fmt" + + "github.com/jcmturner/aescts/v2" + "github.com/jcmturner/gokrb5/v8/crypto/common" + "github.com/jcmturner/gokrb5/v8/crypto/etype" +) + +// EncryptData encrypts the data provided using methods specific to the etype provided as defined in RFC 3962. +func EncryptData(key, data []byte, e etype.EType) ([]byte, []byte, error) { + if len(key) != e.GetKeyByteSize() { + return []byte{}, []byte{}, fmt.Errorf("incorrect keysize: expected: %v actual: %v", e.GetKeyByteSize(), len(key)) + } + ivz := make([]byte, e.GetCypherBlockBitLength()/8) + return aescts.Encrypt(key, ivz, data) +} + +// EncryptMessage encrypts the message provided using the methods specific to the etype provided as defined in RFC 3962. +// The encrypted data is concatenated with its integrity hash to create an encrypted message. +func EncryptMessage(key, message []byte, usage uint32, e etype.EType) ([]byte, []byte, error) { + if len(key) != e.GetKeyByteSize() { + return []byte{}, []byte{}, fmt.Errorf("incorrect keysize: expected: %v actual: %v", e.GetKeyByteSize(), len(key)) + } + //confounder + c := make([]byte, e.GetConfounderByteSize()) + _, err := rand.Read(c) + if err != nil { + return []byte{}, []byte{}, fmt.Errorf("could not generate random confounder: %v", err) + } + plainBytes := append(c, message...) + + // Derive key for encryption from usage + var k []byte + if usage != 0 { + k, err = e.DeriveKey(key, common.GetUsageKe(usage)) + if err != nil { + return []byte{}, []byte{}, fmt.Errorf("error deriving key for encryption: %v", err) + } + } + + // Encrypt the data + iv, b, err := e.EncryptData(k, plainBytes) + if err != nil { + return iv, b, fmt.Errorf("error encrypting data: %v", err) + } + + // Generate and append integrity hash + ih, err := common.GetIntegrityHash(plainBytes, key, usage, e) + if err != nil { + return iv, b, fmt.Errorf("error encrypting data: %v", err) + } + b = append(b, ih...) + return iv, b, nil +} + +// DecryptData decrypts the data provided using the methods specific to the etype provided as defined in RFC 3962. +func DecryptData(key, data []byte, e etype.EType) ([]byte, error) { + if len(key) != e.GetKeyByteSize() { + return []byte{}, fmt.Errorf("incorrect keysize: expected: %v actual: %v", e.GetKeyByteSize(), len(key)) + } + ivz := make([]byte, e.GetCypherBlockBitLength()/8) + return aescts.Decrypt(key, ivz, data) +} + +// DecryptMessage decrypts the message provided using the methods specific to the etype provided as defined in RFC 3962. +// The integrity of the message is also verified. +func DecryptMessage(key, ciphertext []byte, usage uint32, e etype.EType) ([]byte, error) { + //Derive the key + k, err := e.DeriveKey(key, common.GetUsageKe(usage)) + if err != nil { + return nil, fmt.Errorf("error deriving key: %v", err) + } + // Strip off the checksum from the end + b, err := e.DecryptData(k, ciphertext[:len(ciphertext)-e.GetHMACBitLength()/8]) + if err != nil { + return nil, err + } + //Verify checksum + if !e.VerifyIntegrity(key, ciphertext, b, usage) { + return nil, errors.New("integrity verification failed") + } + //Remove the confounder bytes + return b[e.GetConfounderByteSize():], nil +} diff --git a/vendor/github.com/jcmturner/gokrb5/v8/crypto/rfc3962/keyDerivation.go b/vendor/github.com/jcmturner/gokrb5/v8/crypto/rfc3962/keyDerivation.go new file mode 100644 index 000000000000..fb402d97b755 --- /dev/null +++ b/vendor/github.com/jcmturner/gokrb5/v8/crypto/rfc3962/keyDerivation.go @@ -0,0 +1,51 @@ +package rfc3962 + +import ( + "encoding/binary" + "encoding/hex" + "errors" + + "github.com/jcmturner/gofork/x/crypto/pbkdf2" + "github.com/jcmturner/gokrb5/v8/crypto/etype" +) + +const ( + s2kParamsZero = 4294967296 +) + +// StringToKey returns a key derived from the string provided according to the definition in RFC 3961. +func StringToKey(secret, salt, s2kparams string, e etype.EType) ([]byte, error) { + i, err := S2KparamsToItertions(s2kparams) + if err != nil { + return nil, err + } + return StringToKeyIter(secret, salt, i, e) +} + +// StringToPBKDF2 generates an encryption key from a pass phrase and salt string using the PBKDF2 function from PKCS #5 v2.0 +func StringToPBKDF2(secret, salt string, iterations int64, e etype.EType) []byte { + return pbkdf2.Key64([]byte(secret), []byte(salt), iterations, int64(e.GetKeyByteSize()), e.GetHashFunc()) +} + +// StringToKeyIter returns a key derived from the string provided according to the definition in RFC 3961. +func StringToKeyIter(secret, salt string, iterations int64, e etype.EType) ([]byte, error) { + tkey := e.RandomToKey(StringToPBKDF2(secret, salt, iterations, e)) + return e.DeriveKey(tkey, []byte("kerberos")) +} + +// S2KparamsToItertions converts the string representation of iterations to an integer +func S2KparamsToItertions(s2kparams string) (int64, error) { + //The s2kparams string should be hex string representing 4 bytes + //The 4 bytes represent a number in big endian order + //If the value is zero then the number of iterations should be 4,294,967,296 (2^32) + var i uint32 + if len(s2kparams) != 8 { + return int64(s2kParamsZero), errors.New("invalid s2kparams length") + } + b, err := hex.DecodeString(s2kparams) + if err != nil { + return int64(s2kParamsZero), errors.New("invalid s2kparams, cannot decode string to bytes") + } + i = binary.BigEndian.Uint32(b) + return int64(i), nil +} diff --git a/vendor/github.com/jcmturner/gokrb5/v8/crypto/rfc4757/checksum.go b/vendor/github.com/jcmturner/gokrb5/v8/crypto/rfc4757/checksum.go new file mode 100644 index 000000000000..45276e95322f --- /dev/null +++ b/vendor/github.com/jcmturner/gokrb5/v8/crypto/rfc4757/checksum.go @@ -0,0 +1,40 @@ +package rfc4757 + +import ( + "bytes" + "crypto/hmac" + "crypto/md5" + "io" +) + +// Checksum returns a hash of the data in accordance with RFC 4757 +func Checksum(key []byte, usage uint32, data []byte) ([]byte, error) { + // Create hashing key + s := append([]byte(`signaturekey`), byte(0x00)) //includes zero octet at end + mac := hmac.New(md5.New, key) + mac.Write(s) + Ksign := mac.Sum(nil) + + // Format data + tb := UsageToMSMsgType(usage) + p := append(tb, data...) + h := md5.New() + rb := bytes.NewReader(p) + _, err := io.Copy(h, rb) + if err != nil { + return []byte{}, err + } + tmp := h.Sum(nil) + + // Generate HMAC + mac = hmac.New(md5.New, Ksign) + mac.Write(tmp) + return mac.Sum(nil), nil +} + +// HMAC returns a keyed MD5 checksum of the data +func HMAC(key []byte, data []byte) []byte { + mac := hmac.New(md5.New, key) + mac.Write(data) + return mac.Sum(nil) +} diff --git a/vendor/github.com/jcmturner/gokrb5/v8/crypto/rfc4757/encryption.go b/vendor/github.com/jcmturner/gokrb5/v8/crypto/rfc4757/encryption.go new file mode 100644 index 000000000000..fdebe7366855 --- /dev/null +++ b/vendor/github.com/jcmturner/gokrb5/v8/crypto/rfc4757/encryption.go @@ -0,0 +1,80 @@ +// Package rfc4757 provides encryption and checksum methods as specified in RFC 4757 +package rfc4757 + +import ( + "crypto/hmac" + "crypto/rand" + "crypto/rc4" + "errors" + "fmt" + + "github.com/jcmturner/gokrb5/v8/crypto/etype" +) + +// EncryptData encrypts the data provided using methods specific to the etype provided as defined in RFC 4757. +func EncryptData(key, data []byte, e etype.EType) ([]byte, error) { + if len(key) != e.GetKeyByteSize() { + return []byte{}, fmt.Errorf("incorrect keysize: expected: %v actual: %v", e.GetKeyByteSize(), len(key)) + } + rc4Cipher, err := rc4.NewCipher(key) + if err != nil { + return []byte{}, fmt.Errorf("error creating RC4 cipher: %v", err) + } + ed := make([]byte, len(data)) + copy(ed, data) + rc4Cipher.XORKeyStream(ed, ed) + rc4Cipher.Reset() + return ed, nil +} + +// DecryptData decrypts the data provided using the methods specific to the etype provided as defined in RFC 4757. +func DecryptData(key, data []byte, e etype.EType) ([]byte, error) { + return EncryptData(key, data, e) +} + +// EncryptMessage encrypts the message provided using the methods specific to the etype provided as defined in RFC 4757. +// The encrypted data is concatenated with its RC4 header containing integrity checksum and confounder to create an encrypted message. +func EncryptMessage(key, data []byte, usage uint32, export bool, e etype.EType) ([]byte, error) { + confounder := make([]byte, e.GetConfounderByteSize()) // size = 8 + _, err := rand.Read(confounder) + if err != nil { + return []byte{}, fmt.Errorf("error generating confounder: %v", err) + } + k1 := key + k2 := HMAC(k1, UsageToMSMsgType(usage)) + toenc := append(confounder, data...) + chksum := HMAC(k2, toenc) + k3 := HMAC(k2, chksum) + + ed, err := EncryptData(k3, toenc, e) + if err != nil { + return []byte{}, fmt.Errorf("error encrypting data: %v", err) + } + + msg := append(chksum, ed...) + return msg, nil +} + +// DecryptMessage decrypts the message provided using the methods specific to the etype provided as defined in RFC 4757. +// The integrity of the message is also verified. +func DecryptMessage(key, data []byte, usage uint32, export bool, e etype.EType) ([]byte, error) { + checksum := data[:e.GetHMACBitLength()/8] + ct := data[e.GetHMACBitLength()/8:] + _, k2, k3 := deriveKeys(key, checksum, usage, export) + + pt, err := DecryptData(k3, ct, e) + if err != nil { + return []byte{}, fmt.Errorf("error decrypting data: %v", err) + } + + if !VerifyIntegrity(k2, pt, data, e) { + return []byte{}, errors.New("integrity checksum incorrect") + } + return pt[e.GetConfounderByteSize():], nil +} + +// VerifyIntegrity checks the integrity checksum of the data matches that calculated from the decrypted data. +func VerifyIntegrity(key, pt, data []byte, e etype.EType) bool { + chksum := HMAC(key, pt) + return hmac.Equal(chksum, data[:e.GetHMACBitLength()/8]) +} diff --git a/vendor/github.com/jcmturner/gokrb5/v8/crypto/rfc4757/keyDerivation.go b/vendor/github.com/jcmturner/gokrb5/v8/crypto/rfc4757/keyDerivation.go new file mode 100644 index 000000000000..d1f90c077d8d --- /dev/null +++ b/vendor/github.com/jcmturner/gokrb5/v8/crypto/rfc4757/keyDerivation.go @@ -0,0 +1,40 @@ +package rfc4757 + +import ( + "bytes" + "encoding/hex" + "errors" + "fmt" + "io" + + "golang.org/x/crypto/md4" +) + +// StringToKey returns a key derived from the string provided according to the definition in RFC 4757. +func StringToKey(secret string) ([]byte, error) { + b := make([]byte, len(secret)*2, len(secret)*2) + for i, r := range secret { + u := fmt.Sprintf("%04x", r) + c, err := hex.DecodeString(u) + if err != nil { + return []byte{}, errors.New("character could not be encoded") + } + // Swap round the two bytes to make little endian as we put into byte slice + b[2*i] = c[1] + b[2*i+1] = c[0] + } + r := bytes.NewReader(b) + h := md4.New() + _, err := io.Copy(h, r) + if err != nil { + return []byte{}, err + } + return h.Sum(nil), nil +} + +func deriveKeys(key, checksum []byte, usage uint32, export bool) (k1, k2, k3 []byte) { + k1 = key + k2 = HMAC(k1, UsageToMSMsgType(usage)) + k3 = HMAC(k2, checksum) + return +} diff --git a/vendor/github.com/jcmturner/gokrb5/v8/crypto/rfc4757/msgtype.go b/vendor/github.com/jcmturner/gokrb5/v8/crypto/rfc4757/msgtype.go new file mode 100644 index 000000000000..068588d3baa2 --- /dev/null +++ b/vendor/github.com/jcmturner/gokrb5/v8/crypto/rfc4757/msgtype.go @@ -0,0 +1,20 @@ +package rfc4757 + +import "encoding/binary" + +// UsageToMSMsgType converts Kerberos key usage numbers to Microsoft message type encoded as a little-endian four byte slice. +func UsageToMSMsgType(usage uint32) []byte { + // Translate usage numbers to the Microsoft T numbers + switch usage { + case 3: + usage = 8 + case 9: + usage = 8 + case 23: + usage = 13 + } + // Now convert to bytes + tb := make([]byte, 4) // We force an int32 input so we can't go over 4 bytes + binary.PutUvarint(tb, uint64(usage)) + return tb +} diff --git a/vendor/github.com/jcmturner/gokrb5/v8/crypto/rfc8009/encryption.go b/vendor/github.com/jcmturner/gokrb5/v8/crypto/rfc8009/encryption.go new file mode 100644 index 000000000000..54cff7b4629a --- /dev/null +++ b/vendor/github.com/jcmturner/gokrb5/v8/crypto/rfc8009/encryption.go @@ -0,0 +1,125 @@ +// Package rfc8009 provides encryption and checksum methods as specified in RFC 8009 +package rfc8009 + +import ( + "crypto/aes" + "crypto/hmac" + "crypto/rand" + "errors" + "fmt" + + "github.com/jcmturner/aescts/v2" + "github.com/jcmturner/gokrb5/v8/crypto/common" + "github.com/jcmturner/gokrb5/v8/crypto/etype" + "github.com/jcmturner/gokrb5/v8/iana/etypeID" +) + +// EncryptData encrypts the data provided using methods specific to the etype provided as defined in RFC 8009. +func EncryptData(key, data []byte, e etype.EType) ([]byte, []byte, error) { + kl := e.GetKeyByteSize() + if e.GetETypeID() == etypeID.AES256_CTS_HMAC_SHA384_192 { + kl = 32 + } + if len(key) != kl { + return []byte{}, []byte{}, fmt.Errorf("incorrect keysize: expected: %v actual: %v", e.GetKeyByteSize(), len(key)) + } + ivz := make([]byte, aes.BlockSize) + return aescts.Encrypt(key, ivz, data) +} + +// EncryptMessage encrypts the message provided using the methods specific to the etype provided as defined in RFC 8009. +// The encrypted data is concatenated with its integrity hash to create an encrypted message. +func EncryptMessage(key, message []byte, usage uint32, e etype.EType) ([]byte, []byte, error) { + kl := e.GetKeyByteSize() + if e.GetETypeID() == etypeID.AES256_CTS_HMAC_SHA384_192 { + kl = 32 + } + if len(key) != kl { + return []byte{}, []byte{}, fmt.Errorf("incorrect keysize: expected: %v actual: %v", kl, len(key)) + } + if len(key) != e.GetKeyByteSize() { + } + //confounder + c := make([]byte, e.GetConfounderByteSize()) + _, err := rand.Read(c) + if err != nil { + return []byte{}, []byte{}, fmt.Errorf("could not generate random confounder: %v", err) + } + plainBytes := append(c, message...) + + // Derive key for encryption from usage + var k []byte + if usage != 0 { + k, err = e.DeriveKey(key, common.GetUsageKe(usage)) + if err != nil { + return []byte{}, []byte{}, fmt.Errorf("error deriving key for encryption: %v", err) + } + } + + // Encrypt the data + iv, b, err := e.EncryptData(k, plainBytes) + if err != nil { + return iv, b, fmt.Errorf("error encrypting data: %v", err) + } + + ivz := make([]byte, e.GetConfounderByteSize()) + ih, err := GetIntegityHash(ivz, b, key, usage, e) + if err != nil { + return iv, b, fmt.Errorf("error encrypting data: %v", err) + } + b = append(b, ih...) + return iv, b, nil +} + +// DecryptData decrypts the data provided using the methods specific to the etype provided as defined in RFC 8009. +func DecryptData(key, data []byte, e etype.EType) ([]byte, error) { + kl := e.GetKeyByteSize() + if e.GetETypeID() == etypeID.AES256_CTS_HMAC_SHA384_192 { + kl = 32 + } + if len(key) != kl { + return []byte{}, fmt.Errorf("incorrect keysize: expected: %v actual: %v", kl, len(key)) + } + ivz := make([]byte, aes.BlockSize) + return aescts.Decrypt(key, ivz, data) +} + +// DecryptMessage decrypts the message provided using the methods specific to the etype provided as defined in RFC 8009. +// The integrity of the message is also verified. +func DecryptMessage(key, ciphertext []byte, usage uint32, e etype.EType) ([]byte, error) { + //Derive the key + k, err := e.DeriveKey(key, common.GetUsageKe(usage)) + if err != nil { + return nil, fmt.Errorf("error deriving key: %v", err) + } + // Strip off the checksum from the end + b, err := e.DecryptData(k, ciphertext[:len(ciphertext)-e.GetHMACBitLength()/8]) + if err != nil { + return nil, err + } + //Verify checksum + if !e.VerifyIntegrity(key, ciphertext, b, usage) { + return nil, errors.New("integrity verification failed") + } + //Remove the confounder bytes + return b[e.GetConfounderByteSize():], nil +} + +// GetIntegityHash returns a keyed integrity hash of the bytes provided as defined in RFC 8009 +func GetIntegityHash(iv, c, key []byte, usage uint32, e etype.EType) ([]byte, error) { + // Generate and append integrity hash + // Rather than calculating the hash over the confounder and plaintext + // it is calculated over the iv concatenated with the AES cipher output. + ib := append(iv, c...) + return common.GetIntegrityHash(ib, key, usage, e) +} + +// VerifyIntegrity verifies the integrity of cipertext bytes ct. +func VerifyIntegrity(key, ct []byte, usage uint32, etype etype.EType) bool { + h := make([]byte, etype.GetHMACBitLength()/8) + copy(h, ct[len(ct)-etype.GetHMACBitLength()/8:]) + ivz := make([]byte, etype.GetConfounderByteSize()) + ib := append(ivz, ct[:len(ct)-(etype.GetHMACBitLength()/8)]...) + expectedMAC, _ := common.GetIntegrityHash(ib, key, usage, etype) + return hmac.Equal(h, expectedMAC) +} diff --git a/vendor/github.com/jcmturner/gokrb5/v8/crypto/rfc8009/keyDerivation.go b/vendor/github.com/jcmturner/gokrb5/v8/crypto/rfc8009/keyDerivation.go new file mode 100644 index 000000000000..e94732226c6e --- /dev/null +++ b/vendor/github.com/jcmturner/gokrb5/v8/crypto/rfc8009/keyDerivation.go @@ -0,0 +1,135 @@ +package rfc8009 + +import ( + "crypto/hmac" + "encoding/binary" + "encoding/hex" + "errors" + + "github.com/jcmturner/gokrb5/v8/crypto/etype" + "github.com/jcmturner/gokrb5/v8/iana/etypeID" + "golang.org/x/crypto/pbkdf2" +) + +const ( + s2kParamsZero = 32768 +) + +// DeriveRandom for key derivation as defined in RFC 8009 +func DeriveRandom(protocolKey, usage []byte, e etype.EType) ([]byte, error) { + h := e.GetHashFunc()() + return KDF_HMAC_SHA2(protocolKey, []byte("prf"), usage, h.Size(), e), nil +} + +// DeriveKey derives a key from the protocol key based on the usage and the etype's specific methods. +// +// https://tools.ietf.org/html/rfc8009#section-5 +func DeriveKey(protocolKey, label []byte, e etype.EType) []byte { + var context []byte + var kl int + // Key length is longer for aes256-cts-hmac-sha384-192 is it is a Ke or from StringToKey (where label is "kerberos") + if e.GetETypeID() == etypeID.AES256_CTS_HMAC_SHA384_192 { + Swtch: + switch label[len(label)-1] { + case 0x73: + // 0x73 is "s" so label could be kerberos meaning StringToKey so now check if the label is "kerberos" + kerblabel := []byte("kerberos") + if len(label) != len(kerblabel) { + break + } + for i, b := range label { + if b != kerblabel[i] { + kl = e.GetKeySeedBitLength() + break Swtch + } + } + if kl == 0 { + // This is StringToKey + kl = 256 + } + case 0xAA: + // This is a Ke + kl = 256 + } + } + if kl == 0 { + kl = e.GetKeySeedBitLength() + } + return e.RandomToKey(KDF_HMAC_SHA2(protocolKey, label, context, kl, e)) +} + +// RandomToKey returns a key from the bytes provided according to the definition in RFC 8009. +func RandomToKey(b []byte) []byte { + return b +} + +// StringToKey returns a key derived from the string provided according to the definition in RFC 8009. +func StringToKey(secret, salt, s2kparams string, e etype.EType) ([]byte, error) { + i, err := S2KparamsToItertions(s2kparams) + if err != nil { + return nil, err + } + return StringToKeyIter(secret, salt, i, e) +} + +// StringToKeyIter returns a key derived from the string provided according to the definition in RFC 8009. +func StringToKeyIter(secret, salt string, iterations int, e etype.EType) ([]byte, error) { + tkey := e.RandomToKey(StringToPBKDF2(secret, salt, iterations, e)) + return e.DeriveKey(tkey, []byte("kerberos")) +} + +// StringToPBKDF2 generates an encryption key from a pass phrase and salt string using the PBKDF2 function from PKCS #5 v2.0 +func StringToPBKDF2(secret, salt string, iterations int, e etype.EType) []byte { + kl := e.GetKeyByteSize() + if e.GetETypeID() == etypeID.AES256_CTS_HMAC_SHA384_192 { + kl = 32 + } + return pbkdf2.Key([]byte(secret), []byte(salt), iterations, kl, e.GetHashFunc()) +} + +// KDF_HMAC_SHA2 key derivation: https://tools.ietf.org/html/rfc8009#section-3 +func KDF_HMAC_SHA2(protocolKey, label, context []byte, kl int, e etype.EType) []byte { + //k: Length in bits of the key to be outputted, expressed in big-endian binary representation in 4 bytes. + k := make([]byte, 4, 4) + binary.BigEndian.PutUint32(k, uint32(kl)) + + c := make([]byte, 4, 4) + binary.BigEndian.PutUint32(c, uint32(1)) + c = append(c, label...) + c = append(c, byte(0)) + if len(context) > 0 { + c = append(c, context...) + } + c = append(c, k...) + + mac := hmac.New(e.GetHashFunc(), protocolKey) + mac.Write(c) + return mac.Sum(nil)[:(kl / 8)] +} + +// GetSaltP returns the salt value based on the etype name: https://tools.ietf.org/html/rfc8009#section-4 +func GetSaltP(salt, ename string) string { + b := []byte(ename) + b = append(b, byte(0)) + b = append(b, []byte(salt)...) + return string(b) +} + +// S2KparamsToItertions converts the string representation of iterations to an integer for RFC 8009. +func S2KparamsToItertions(s2kparams string) (int, error) { + var i uint32 + if len(s2kparams) != 8 { + return s2kParamsZero, errors.New("Invalid s2kparams length") + } + b, err := hex.DecodeString(s2kparams) + if err != nil { + return s2kParamsZero, errors.New("Invalid s2kparams, cannot decode string to bytes") + } + i = binary.BigEndian.Uint32(b) + //buf := bytes.NewBuffer(b) + //err = binary.Read(buf, binary.BigEndian, &i) + if err != nil { + return s2kParamsZero, errors.New("Invalid s2kparams, cannot convert to big endian int32") + } + return int(i), nil +} diff --git a/vendor/github.com/jcmturner/gokrb5/v8/gssapi/MICToken.go b/vendor/github.com/jcmturner/gokrb5/v8/gssapi/MICToken.go new file mode 100644 index 000000000000..ab8daa2897b6 --- /dev/null +++ b/vendor/github.com/jcmturner/gokrb5/v8/gssapi/MICToken.go @@ -0,0 +1,174 @@ +package gssapi + +import ( + "bytes" + "crypto/hmac" + "encoding/binary" + "encoding/hex" + "errors" + "fmt" + + "github.com/jcmturner/gokrb5/v8/crypto" + "github.com/jcmturner/gokrb5/v8/iana/keyusage" + "github.com/jcmturner/gokrb5/v8/types" +) + +// RFC 4121, section 4.2.6.1 + +const ( + // MICTokenFlagSentByAcceptor - this flag indicates the sender is the context acceptor. When not set, it indicates the sender is the context initiator + MICTokenFlagSentByAcceptor = 1 << iota + // MICTokenFlagSealed - this flag indicates confidentiality is provided for. It SHALL NOT be set in MIC tokens + MICTokenFlagSealed + // MICTokenFlagAcceptorSubkey - a subkey asserted by the context acceptor is used to protect the message + MICTokenFlagAcceptorSubkey +) + +const ( + micHdrLen = 16 // Length of the MIC Token's header +) + +// MICToken represents a GSS API MIC token, as defined in RFC 4121. +// It contains the header fields, the payload (this is not transmitted) and +// the checksum, and provides the logic for converting to/from bytes plus +// computing and verifying checksums +type MICToken struct { + // const GSS Token ID: 0x0404 + Flags byte // contains three flags: acceptor, sealed, acceptor subkey + // const Filler: 0xFF 0xFF 0xFF 0xFF 0xFF + SndSeqNum uint64 // sender's sequence number. big-endian + Payload []byte // your data! :) + Checksum []byte // checksum of { payload | header } +} + +// Return the 2 bytes identifying a GSS API MIC token +func getGSSMICTokenID() *[2]byte { + return &[2]byte{0x04, 0x04} +} + +// Return the filler bytes used in header +func fillerBytes() *[5]byte { + return &[5]byte{0xFF, 0xFF, 0xFF, 0xFF, 0xFF} +} + +// Marshal the MICToken into a byte slice. +// The payload should have been set and the checksum computed, otherwise an error is returned. +func (mt *MICToken) Marshal() ([]byte, error) { + if mt.Checksum == nil { + return nil, errors.New("checksum has not been set") + } + + bytes := make([]byte, micHdrLen+len(mt.Checksum)) + copy(bytes[0:micHdrLen], mt.getMICChecksumHeader()[:]) + copy(bytes[micHdrLen:], mt.Checksum) + + return bytes, nil +} + +// SetChecksum uses the passed encryption key and key usage to compute the checksum over the payload and +// the header, and sets the Checksum field of this MICToken. +// If the payload has not been set or the checksum has already been set, an error is returned. +func (mt *MICToken) SetChecksum(key types.EncryptionKey, keyUsage uint32) error { + if mt.Checksum != nil { + return errors.New("checksum has already been computed") + } + checksum, err := mt.checksum(key, keyUsage) + if err != nil { + return err + } + mt.Checksum = checksum + return nil +} + +// Compute and return the checksum of this token, computed using the passed key and key usage. +// Note: This will NOT update the struct's Checksum field. +func (mt *MICToken) checksum(key types.EncryptionKey, keyUsage uint32) ([]byte, error) { + if mt.Payload == nil { + return nil, errors.New("cannot compute checksum with uninitialized payload") + } + d := make([]byte, micHdrLen+len(mt.Payload)) + copy(d[0:], mt.Payload) + copy(d[len(mt.Payload):], mt.getMICChecksumHeader()) + + encType, err := crypto.GetEtype(key.KeyType) + if err != nil { + return nil, err + } + return encType.GetChecksumHash(key.KeyValue, d, keyUsage) +} + +// Build a header suitable for a checksum computation +func (mt *MICToken) getMICChecksumHeader() []byte { + header := make([]byte, micHdrLen) + copy(header[0:2], getGSSMICTokenID()[:]) + header[2] = mt.Flags + copy(header[3:8], fillerBytes()[:]) + binary.BigEndian.PutUint64(header[8:16], mt.SndSeqNum) + return header +} + +// Verify computes the token's checksum with the provided key and usage, +// and compares it to the checksum present in the token. +// In case of any failure, (false, err) is returned, with err an explanatory error. +func (mt *MICToken) Verify(key types.EncryptionKey, keyUsage uint32) (bool, error) { + computed, err := mt.checksum(key, keyUsage) + if err != nil { + return false, err + } + if !hmac.Equal(computed, mt.Checksum) { + return false, fmt.Errorf( + "checksum mismatch. Computed: %s, Contained in token: %s", + hex.EncodeToString(computed), hex.EncodeToString(mt.Checksum)) + } + return true, nil +} + +// Unmarshal bytes into the corresponding MICToken. +// If expectFromAcceptor is true we expect the token to have been emitted by the gss acceptor, +// and will check the according flag, returning an error if the token does not match the expectation. +func (mt *MICToken) Unmarshal(b []byte, expectFromAcceptor bool) error { + if len(b) < micHdrLen { + return errors.New("bytes shorter than header length") + } + if !bytes.Equal(getGSSMICTokenID()[:], b[0:2]) { + return fmt.Errorf("wrong Token ID, Expected %s, was %s", + hex.EncodeToString(getGSSMICTokenID()[:]), + hex.EncodeToString(b[0:2])) + } + flags := b[2] + isFromAcceptor := flags&MICTokenFlagSentByAcceptor != 0 + if isFromAcceptor && !expectFromAcceptor { + return errors.New("unexpected acceptor flag is set: not expecting a token from the acceptor") + } + if !isFromAcceptor && expectFromAcceptor { + return errors.New("unexpected acceptor flag is not set: expecting a token from the acceptor, not in the initiator") + } + if !bytes.Equal(b[3:8], fillerBytes()[:]) { + return fmt.Errorf("unexpected filler bytes: expecting %s, was %s", + hex.EncodeToString(fillerBytes()[:]), + hex.EncodeToString(b[3:8])) + } + + mt.Flags = flags + mt.SndSeqNum = binary.BigEndian.Uint64(b[8:16]) + mt.Checksum = b[micHdrLen:] + return nil +} + +// NewInitiatorMICToken builds a new initiator token (acceptor flag will be set to 0) and computes the authenticated checksum. +// Other flags are set to 0. +// Note that in certain circumstances you may need to provide a sequence number that has been defined earlier. +// This is currently not supported. +func NewInitiatorMICToken(payload []byte, key types.EncryptionKey) (*MICToken, error) { + token := MICToken{ + Flags: 0x00, + SndSeqNum: 0, + Payload: payload, + } + + if err := token.SetChecksum(key, keyusage.GSSAPI_INITIATOR_SIGN); err != nil { + return nil, err + } + + return &token, nil +} diff --git a/vendor/github.com/jcmturner/gokrb5/v8/gssapi/README.md b/vendor/github.com/jcmturner/gokrb5/v8/gssapi/README.md new file mode 100644 index 000000000000..8fdcf70c3297 --- /dev/null +++ b/vendor/github.com/jcmturner/gokrb5/v8/gssapi/README.md @@ -0,0 +1,20 @@ +# Notes on GSS-API Negotiation Mechanism +https://tools.ietf.org/html/rfc4178 + +Client sends an initial negotiation message to the server which specifies the list of mechanisms +the client can support in order of decreasing preference. +This message is generated with the ``NewNegTokenInitKrb5`` method. +The message generated by this function specifies only a kerberos v5 mechanism is supported. + +The RFC states that this message can optionally contain the initial mechanism token +for the preferred mechanism (KRB5 in this case) of the client. The ``NewNegTokenInitKrb5`` +includes this in the message. + +The server side responds to this message with a one of four messages: + +| Message Type/State | Description | +|--------------------|-------------| +| accept-completed | indicates that the initiator-selected mechanism was acceptable to the target, and that the security mechanism token embedded in the first negotiation message was sufficient to complete the authentication | +| accept-incomplete | At least one more message is needed from the client to establish security context. | +| reject | Negotiation is being terminated. | +| request-mic | (this state can only be present in the first reply message from the target) indicates that the MIC token exchange is REQUIRED if per-message integrity services are available | \ No newline at end of file diff --git a/vendor/github.com/jcmturner/gokrb5/v8/gssapi/contextFlags.go b/vendor/github.com/jcmturner/gokrb5/v8/gssapi/contextFlags.go new file mode 100644 index 000000000000..8c91859b89ff --- /dev/null +++ b/vendor/github.com/jcmturner/gokrb5/v8/gssapi/contextFlags.go @@ -0,0 +1,27 @@ +package gssapi + +import "github.com/jcmturner/gofork/encoding/asn1" + +// GSS-API context flags assigned numbers. +const ( + ContextFlagDeleg = 1 + ContextFlagMutual = 2 + ContextFlagReplay = 4 + ContextFlagSequence = 8 + ContextFlagConf = 16 + ContextFlagInteg = 32 + ContextFlagAnon = 64 +) + +// ContextFlags flags for GSSAPI +// DEPRECATED - do not use +type ContextFlags asn1.BitString + +// NewContextFlags creates a new ContextFlags instance +// DEPRECATED - do not use +func NewContextFlags() ContextFlags { + var c ContextFlags + c.BitLength = 32 + c.Bytes = make([]byte, 4) + return c +} diff --git a/vendor/github.com/jcmturner/gokrb5/v8/gssapi/gssapi.go b/vendor/github.com/jcmturner/gokrb5/v8/gssapi/gssapi.go new file mode 100644 index 000000000000..80822319022a --- /dev/null +++ b/vendor/github.com/jcmturner/gokrb5/v8/gssapi/gssapi.go @@ -0,0 +1,202 @@ +// Package gssapi implements Generic Security Services Application Program Interface required for SPNEGO kerberos authentication. +package gssapi + +import ( + "context" + "fmt" + + "github.com/jcmturner/gofork/encoding/asn1" +) + +// GSS-API OID names +const ( + // GSS-API OID names + OIDKRB5 OIDName = "KRB5" // MechType OID for Kerberos 5 + OIDMSLegacyKRB5 OIDName = "MSLegacyKRB5" // MechType OID for Kerberos 5 + OIDSPNEGO OIDName = "SPNEGO" + OIDGSSIAKerb OIDName = "GSSIAKerb" // Indicates the client cannot get a service ticket and asks the server to serve as an intermediate to the target KDC. http://k5wiki.kerberos.org/wiki/Projects/IAKERB#IAKERB_mech +) + +// GSS-API status values +const ( + StatusBadBindings = 1 << iota + StatusBadMech + StatusBadName + StatusBadNameType + StatusBadStatus + StatusBadSig + StatusBadMIC + StatusContextExpired + StatusCredentialsExpired + StatusDefectiveCredential + StatusDefectiveToken + StatusFailure + StatusNoContext + StatusNoCred + StatusBadQOP + StatusUnauthorized + StatusUnavailable + StatusDuplicateElement + StatusNameNotMN + StatusComplete + StatusContinueNeeded + StatusDuplicateToken + StatusOldToken + StatusUnseqToken + StatusGapToken +) + +// ContextToken is an interface for a GSS-API context token. +type ContextToken interface { + Marshal() ([]byte, error) + Unmarshal(b []byte) error + Verify() (bool, Status) + Context() context.Context +} + +/* +CREDENTIAL MANAGEMENT + +GSS_Acquire_cred acquire credentials for use +GSS_Release_cred release credentials after use +GSS_Inquire_cred display information about credentials +GSS_Add_cred construct credentials incrementally +GSS_Inquire_cred_by_mech display per-mechanism credential information + +CONTEXT-LEVEL CALLS + +GSS_Init_sec_context initiate outbound security context +GSS_Accept_sec_context accept inbound security context +GSS_Delete_sec_context flush context when no longer needed +GSS_Process_context_token process received control token on context +GSS_Context_time indicate validity time remaining on context +GSS_Inquire_context display information about context +GSS_Wrap_size_limit determine GSS_Wrap token size limit +GSS_Export_sec_context transfer context to other process +GSS_Import_sec_context import transferred context + +PER-MESSAGE CALLS + +GSS_GetMIC apply integrity check, receive as token separate from message +GSS_VerifyMIC validate integrity check token along with message +GSS_Wrap sign, optionally encrypt, encapsulate +GSS_Unwrap decapsulate, decrypt if needed, validate integrity check + +SUPPORT CALLS + +GSS_Display_status translate status codes to printable form +GSS_Indicate_mechs indicate mech_types supported on local system +GSS_Compare_name compare two names for equality +GSS_Display_name translate name to printable form +GSS_Import_name convert printable name to normalized form +GSS_Release_name free storage of normalized-form name +GSS_Release_buffer free storage of general GSS-allocated object +GSS_Release_OID_set free storage of OID set object +GSS_Create_empty_OID_set create empty OID set +GSS_Add_OID_set_member add member to OID set +GSS_Test_OID_set_member test if OID is member of OID set +GSS_Inquire_names_for_mech indicate name types supported by mechanism +GSS_Inquire_mechs_for_name indicates mechanisms supporting name type +GSS_Canonicalize_name translate name to per-mechanism form +GSS_Export_name externalize per-mechanism name +GSS_Duplicate_name duplicate name object +*/ + +// Mechanism is the GSS-API interface for authentication mechanisms. +type Mechanism interface { + OID() asn1.ObjectIdentifier + AcquireCred() error // acquire credentials for use (eg. AS exchange for KRB5) + InitSecContext() (ContextToken, error) // initiate outbound security context (eg TGS exchange builds AP_REQ to go into ContextToken to send to service) + AcceptSecContext(ct ContextToken) (bool, context.Context, Status) // service verifies the token server side to establish a context + MIC() MICToken // apply integrity check, receive as token separate from message + VerifyMIC(mt MICToken) (bool, error) // validate integrity check token along with message + Wrap(msg []byte) WrapToken // sign, optionally encrypt, encapsulate + Unwrap(wt WrapToken) []byte // decapsulate, decrypt if needed, validate integrity check +} + +// OIDName is the type for defined GSS-API OIDs. +type OIDName string + +// OID returns the OID for the provided OID name. +func (o OIDName) OID() asn1.ObjectIdentifier { + switch o { + case OIDSPNEGO: + return asn1.ObjectIdentifier{1, 3, 6, 1, 5, 5, 2} + case OIDKRB5: + return asn1.ObjectIdentifier{1, 2, 840, 113554, 1, 2, 2} + case OIDMSLegacyKRB5: + return asn1.ObjectIdentifier{1, 2, 840, 48018, 1, 2, 2} + case OIDGSSIAKerb: + return asn1.ObjectIdentifier{1, 3, 6, 1, 5, 2, 5} + } + return asn1.ObjectIdentifier{} +} + +// Status is the GSS-API status and implements the error interface. +type Status struct { + Code int + Message string +} + +// Error returns the Status description. +func (s Status) Error() string { + var str string + switch s.Code { + case StatusBadBindings: + str = "channel binding mismatch" + case StatusBadMech: + str = "unsupported mechanism requested" + case StatusBadName: + str = "invalid name provided" + case StatusBadNameType: + str = "name of unsupported type provided" + case StatusBadStatus: + str = "invalid input status selector" + case StatusBadSig: + str = "token had invalid integrity check" + case StatusBadMIC: + str = "preferred alias for GSS_S_BAD_SIG" + case StatusContextExpired: + str = "specified security context expired" + case StatusCredentialsExpired: + str = "expired credentials detected" + case StatusDefectiveCredential: + str = "defective credential detected" + case StatusDefectiveToken: + str = "defective token detected" + case StatusFailure: + str = "failure, unspecified at GSS-API level" + case StatusNoContext: + str = "no valid security context specified" + case StatusNoCred: + str = "no valid credentials provided" + case StatusBadQOP: + str = "unsupported QOP valu" + case StatusUnauthorized: + str = "operation unauthorized" + case StatusUnavailable: + str = "operation unavailable" + case StatusDuplicateElement: + str = "duplicate credential element requested" + case StatusNameNotMN: + str = "name contains multi-mechanism elements" + case StatusComplete: + str = "normal completion" + case StatusContinueNeeded: + str = "continuation call to routine required" + case StatusDuplicateToken: + str = "duplicate per-message token detected" + case StatusOldToken: + str = "timed-out per-message token detected" + case StatusUnseqToken: + str = "reordered (early) per-message token detected" + case StatusGapToken: + str = "skipped predecessor token(s) detected" + default: + str = "unknown GSS-API error status" + } + if s.Message != "" { + return fmt.Sprintf("%s: %s", str, s.Message) + } + return str +} diff --git a/vendor/github.com/jcmturner/gokrb5/v8/gssapi/wrapToken.go b/vendor/github.com/jcmturner/gokrb5/v8/gssapi/wrapToken.go new file mode 100644 index 000000000000..ea7d0543e090 --- /dev/null +++ b/vendor/github.com/jcmturner/gokrb5/v8/gssapi/wrapToken.go @@ -0,0 +1,195 @@ +package gssapi + +import ( + "bytes" + "crypto/hmac" + "encoding/binary" + "encoding/hex" + "errors" + "fmt" + + "github.com/jcmturner/gokrb5/v8/crypto" + "github.com/jcmturner/gokrb5/v8/iana/keyusage" + "github.com/jcmturner/gokrb5/v8/types" +) + +// RFC 4121, section 4.2.6.2 + +const ( + // HdrLen is the length of the Wrap Token's header + HdrLen = 16 + // FillerByte is a filler in the WrapToken structure + FillerByte byte = 0xFF +) + +// WrapToken represents a GSS API Wrap token, as defined in RFC 4121. +// It contains the header fields, the payload and the checksum, and provides +// the logic for converting to/from bytes plus computing and verifying checksums +type WrapToken struct { + // const GSS Token ID: 0x0504 + Flags byte // contains three flags: acceptor, sealed, acceptor subkey + // const Filler: 0xFF + EC uint16 // checksum length. big-endian + RRC uint16 // right rotation count. big-endian + SndSeqNum uint64 // sender's sequence number. big-endian + Payload []byte // your data! :) + CheckSum []byte // authenticated checksum of { payload | header } +} + +// Return the 2 bytes identifying a GSS API Wrap token +func getGssWrapTokenId() *[2]byte { + return &[2]byte{0x05, 0x04} +} + +// Marshal the WrapToken into a byte slice. +// The payload should have been set and the checksum computed, otherwise an error is returned. +func (wt *WrapToken) Marshal() ([]byte, error) { + if wt.CheckSum == nil { + return nil, errors.New("checksum has not been set") + } + if wt.Payload == nil { + return nil, errors.New("payload has not been set") + } + + pldOffset := HdrLen // Offset of the payload in the token + chkSOffset := HdrLen + len(wt.Payload) // Offset of the checksum in the token + + bytes := make([]byte, chkSOffset+int(wt.EC)) + copy(bytes[0:], getGssWrapTokenId()[:]) + bytes[2] = wt.Flags + bytes[3] = FillerByte + binary.BigEndian.PutUint16(bytes[4:6], wt.EC) + binary.BigEndian.PutUint16(bytes[6:8], wt.RRC) + binary.BigEndian.PutUint64(bytes[8:16], wt.SndSeqNum) + copy(bytes[pldOffset:], wt.Payload) + copy(bytes[chkSOffset:], wt.CheckSum) + return bytes, nil +} + +// SetCheckSum uses the passed encryption key and key usage to compute the checksum over the payload and +// the header, and sets the CheckSum field of this WrapToken. +// If the payload has not been set or the checksum has already been set, an error is returned. +func (wt *WrapToken) SetCheckSum(key types.EncryptionKey, keyUsage uint32) error { + if wt.Payload == nil { + return errors.New("payload has not been set") + } + if wt.CheckSum != nil { + return errors.New("checksum has already been computed") + } + chkSum, cErr := wt.computeCheckSum(key, keyUsage) + if cErr != nil { + return cErr + } + wt.CheckSum = chkSum + return nil +} + +// ComputeCheckSum computes and returns the checksum of this token, computed using the passed key and key usage. +// Note: This will NOT update the struct's Checksum field. +func (wt *WrapToken) computeCheckSum(key types.EncryptionKey, keyUsage uint32) ([]byte, error) { + if wt.Payload == nil { + return nil, errors.New("cannot compute checksum with uninitialized payload") + } + // Build a slice containing { payload | header } + checksumMe := make([]byte, HdrLen+len(wt.Payload)) + copy(checksumMe[0:], wt.Payload) + copy(checksumMe[len(wt.Payload):], getChecksumHeader(wt.Flags, wt.SndSeqNum)) + + encType, err := crypto.GetEtype(key.KeyType) + if err != nil { + return nil, err + } + return encType.GetChecksumHash(key.KeyValue, checksumMe, keyUsage) +} + +// Build a header suitable for a checksum computation +func getChecksumHeader(flags byte, senderSeqNum uint64) []byte { + header := make([]byte, 16) + copy(header[0:], []byte{0x05, 0x04, flags, 0xFF, 0x00, 0x00, 0x00, 0x00}) + binary.BigEndian.PutUint64(header[8:], senderSeqNum) + return header +} + +// Verify computes the token's checksum with the provided key and usage, +// and compares it to the checksum present in the token. +// In case of any failure, (false, Err) is returned, with Err an explanatory error. +func (wt *WrapToken) Verify(key types.EncryptionKey, keyUsage uint32) (bool, error) { + computed, cErr := wt.computeCheckSum(key, keyUsage) + if cErr != nil { + return false, cErr + } + if !hmac.Equal(computed, wt.CheckSum) { + return false, fmt.Errorf( + "checksum mismatch. Computed: %s, Contained in token: %s", + hex.EncodeToString(computed), hex.EncodeToString(wt.CheckSum)) + } + return true, nil +} + +// Unmarshal bytes into the corresponding WrapToken. +// If expectFromAcceptor is true, we expect the token to have been emitted by the gss acceptor, +// and will check the according flag, returning an error if the token does not match the expectation. +func (wt *WrapToken) Unmarshal(b []byte, expectFromAcceptor bool) error { + // Check if we can read a whole header + if len(b) < 16 { + return errors.New("bytes shorter than header length") + } + // Is the Token ID correct? + if !bytes.Equal(getGssWrapTokenId()[:], b[0:2]) { + return fmt.Errorf("wrong Token ID. Expected %s, was %s", + hex.EncodeToString(getGssWrapTokenId()[:]), + hex.EncodeToString(b[0:2])) + } + // Check the acceptor flag + flags := b[2] + isFromAcceptor := flags&0x01 == 1 + if isFromAcceptor && !expectFromAcceptor { + return errors.New("unexpected acceptor flag is set: not expecting a token from the acceptor") + } + if !isFromAcceptor && expectFromAcceptor { + return errors.New("expected acceptor flag is not set: expecting a token from the acceptor, not the initiator") + } + // Check the filler byte + if b[3] != FillerByte { + return fmt.Errorf("unexpected filler byte: expecting 0xFF, was %s ", hex.EncodeToString(b[3:4])) + } + checksumL := binary.BigEndian.Uint16(b[4:6]) + // Sanity check on the checksum length + if int(checksumL) > len(b)-HdrLen { + return fmt.Errorf("inconsistent checksum length: %d bytes to parse, checksum length is %d", len(b), checksumL) + } + + wt.Flags = flags + wt.EC = checksumL + wt.RRC = binary.BigEndian.Uint16(b[6:8]) + wt.SndSeqNum = binary.BigEndian.Uint64(b[8:16]) + wt.Payload = b[16 : len(b)-int(checksumL)] + wt.CheckSum = b[len(b)-int(checksumL):] + return nil +} + +// NewInitiatorWrapToken builds a new initiator token (acceptor flag will be set to 0) and computes the authenticated checksum. +// Other flags are set to 0, and the RRC and sequence number are initialized to 0. +// Note that in certain circumstances you may need to provide a sequence number that has been defined earlier. +// This is currently not supported. +func NewInitiatorWrapToken(payload []byte, key types.EncryptionKey) (*WrapToken, error) { + encType, err := crypto.GetEtype(key.KeyType) + if err != nil { + return nil, err + } + + token := WrapToken{ + Flags: 0x00, // all zeroed out (this is a token sent by the initiator) + // Checksum size: length of output of the HMAC function, in bytes. + EC: uint16(encType.GetHMACBitLength() / 8), + RRC: 0, + SndSeqNum: 0, + Payload: payload, + } + + if err := token.SetCheckSum(key, keyusage.GSSAPI_INITIATOR_SEAL); err != nil { + return nil, err + } + + return &token, nil +} diff --git a/vendor/github.com/jcmturner/gokrb5/v8/iana/addrtype/constants.go b/vendor/github.com/jcmturner/gokrb5/v8/iana/addrtype/constants.go new file mode 100644 index 000000000000..457b89d7ace7 --- /dev/null +++ b/vendor/github.com/jcmturner/gokrb5/v8/iana/addrtype/constants.go @@ -0,0 +1,15 @@ +// Package addrtype provides Address type assigned numbers. +package addrtype + +// Address type IDs. +const ( + IPv4 int32 = 2 + Directional int32 = 3 + ChaosNet int32 = 5 + XNS int32 = 6 + ISO int32 = 7 + DECNETPhaseIV int32 = 12 + AppleTalkDDP int32 = 16 + NetBios int32 = 20 + IPv6 int32 = 24 +) diff --git a/vendor/github.com/jcmturner/gokrb5/v8/iana/adtype/constants.go b/vendor/github.com/jcmturner/gokrb5/v8/iana/adtype/constants.go new file mode 100644 index 000000000000..e805b7466601 --- /dev/null +++ b/vendor/github.com/jcmturner/gokrb5/v8/iana/adtype/constants.go @@ -0,0 +1,23 @@ +// Package adtype provides Authenticator type assigned numbers. +package adtype + +// Authenticator type IDs. +const ( + ADIfRelevant int32 = 1 + ADIntendedForServer int32 = 2 + ADIntendedForApplicationClass int32 = 3 + ADKDCIssued int32 = 4 + ADAndOr int32 = 5 + ADMandatoryTicketExtensions int32 = 6 + ADInTicketExtensions int32 = 7 + ADMandatoryForKDC int32 = 8 + OSFDCE int32 = 64 + SESAME int32 = 65 + ADOSFDCEPKICertID int32 = 66 + ADAuthenticationStrength int32 = 70 + ADFXFastArmor int32 = 71 + ADFXFastUsed int32 = 72 + ADWin2KPAC int32 = 128 + ADEtypeNegotiation int32 = 129 + //Reserved values 9-63 +) diff --git a/vendor/github.com/jcmturner/gokrb5/v8/iana/asnAppTag/constants.go b/vendor/github.com/jcmturner/gokrb5/v8/iana/asnAppTag/constants.go new file mode 100644 index 000000000000..d74cd60e93eb --- /dev/null +++ b/vendor/github.com/jcmturner/gokrb5/v8/iana/asnAppTag/constants.go @@ -0,0 +1,24 @@ +// Package asnAppTag provides ASN1 application tag numbers. +package asnAppTag + +// ASN1 application tag numbers. +const ( + Ticket = 1 + Authenticator = 2 + EncTicketPart = 3 + ASREQ = 10 + TGSREQ = 12 + ASREP = 11 + TGSREP = 13 + APREQ = 14 + APREP = 15 + KRBSafe = 20 + KRBPriv = 21 + KRBCred = 22 + EncASRepPart = 25 + EncTGSRepPart = 26 + EncAPRepPart = 27 + EncKrbPrivPart = 28 + EncKrbCredPart = 29 + KRBError = 30 +) diff --git a/vendor/github.com/jcmturner/gokrb5/v8/iana/chksumtype/constants.go b/vendor/github.com/jcmturner/gokrb5/v8/iana/chksumtype/constants.go new file mode 100644 index 000000000000..93db952dda1e --- /dev/null +++ b/vendor/github.com/jcmturner/gokrb5/v8/iana/chksumtype/constants.go @@ -0,0 +1,32 @@ +// Package chksumtype provides Kerberos 5 checksum type assigned numbers. +package chksumtype + +// Checksum type IDs. +const ( + //RESERVED : 0 + CRC32 int32 = 1 + RSA_MD4 int32 = 2 + RSA_MD4_DES int32 = 3 + DES_MAC int32 = 4 + DES_MAC_K int32 = 5 + RSA_MD4_DES_K int32 = 6 + RSA_MD5 int32 = 7 + RSA_MD5_DES int32 = 8 + RSA_MD5_DES3 int32 = 9 + SHA1_ID10 int32 = 10 + //UNASSIGNED : 11 + HMAC_SHA1_DES3_KD int32 = 12 + HMAC_SHA1_DES3 int32 = 13 + SHA1_ID14 int32 = 14 + HMAC_SHA1_96_AES128 int32 = 15 + HMAC_SHA1_96_AES256 int32 = 16 + CMAC_CAMELLIA128 int32 = 17 + CMAC_CAMELLIA256 int32 = 18 + HMAC_SHA256_128_AES128 int32 = 19 + HMAC_SHA384_192_AES256 int32 = 20 + //UNASSIGNED : 21-32770 + GSSAPI int32 = 32771 + //UNASSIGNED : 32772-2147483647 + KERB_CHECKSUM_HMAC_MD5_UNSIGNED uint32 = 4294967158 // 0xFFFFFF76 documentation says this is -138 but in an unsigned int this is 4294967158 + KERB_CHECKSUM_HMAC_MD5 int32 = -138 +) diff --git a/vendor/github.com/jcmturner/gokrb5/v8/iana/constants.go b/vendor/github.com/jcmturner/gokrb5/v8/iana/constants.go new file mode 100644 index 000000000000..0b8e916d5b64 --- /dev/null +++ b/vendor/github.com/jcmturner/gokrb5/v8/iana/constants.go @@ -0,0 +1,5 @@ +// Package iana provides Kerberos 5 assigned numbers. +package iana + +// PVNO is the Protocol Version Number. +const PVNO = 5 diff --git a/vendor/github.com/jcmturner/gokrb5/v8/iana/errorcode/constants.go b/vendor/github.com/jcmturner/gokrb5/v8/iana/errorcode/constants.go new file mode 100644 index 000000000000..fd756bc5e367 --- /dev/null +++ b/vendor/github.com/jcmturner/gokrb5/v8/iana/errorcode/constants.go @@ -0,0 +1,155 @@ +// Package errorcode provides Kerberos 5 assigned error codes. +package errorcode + +import "fmt" + +// Kerberos error codes. +const ( + KDC_ERR_NONE int32 = 0 //No error + KDC_ERR_NAME_EXP int32 = 1 //Client's entry in database has expired + KDC_ERR_SERVICE_EXP int32 = 2 //Server's entry in database has expired + KDC_ERR_BAD_PVNO int32 = 3 //Requested protocol version number not supported + KDC_ERR_C_OLD_MAST_KVNO int32 = 4 //Client's key encrypted in old master key + KDC_ERR_S_OLD_MAST_KVNO int32 = 5 //Server's key encrypted in old master key + KDC_ERR_C_PRINCIPAL_UNKNOWN int32 = 6 //Client not found in Kerberos database + KDC_ERR_S_PRINCIPAL_UNKNOWN int32 = 7 //Server not found in Kerberos database + KDC_ERR_PRINCIPAL_NOT_UNIQUE int32 = 8 //Multiple principal entries in database + KDC_ERR_NULL_KEY int32 = 9 //The client or server has a null key + KDC_ERR_CANNOT_POSTDATE int32 = 10 //Ticket not eligible for postdating + KDC_ERR_NEVER_VALID int32 = 11 //Requested starttime is later than end time + KDC_ERR_POLICY int32 = 12 //KDC policy rejects request + KDC_ERR_BADOPTION int32 = 13 //KDC cannot accommodate requested option + KDC_ERR_ETYPE_NOSUPP int32 = 14 //KDC has no support for encryption type + KDC_ERR_SUMTYPE_NOSUPP int32 = 15 //KDC has no support for checksum type + KDC_ERR_PADATA_TYPE_NOSUPP int32 = 16 //KDC has no support for padata type + KDC_ERR_TRTYPE_NOSUPP int32 = 17 //KDC has no support for transited type + KDC_ERR_CLIENT_REVOKED int32 = 18 //Clients credentials have been revoked + KDC_ERR_SERVICE_REVOKED int32 = 19 //Credentials for server have been revoked + KDC_ERR_TGT_REVOKED int32 = 20 //TGT has been revoked + KDC_ERR_CLIENT_NOTYET int32 = 21 //Client not yet valid; try again later + KDC_ERR_SERVICE_NOTYET int32 = 22 //Server not yet valid; try again later + KDC_ERR_KEY_EXPIRED int32 = 23 //Password has expired; change password to reset + KDC_ERR_PREAUTH_FAILED int32 = 24 //Pre-authentication information was invalid + KDC_ERR_PREAUTH_REQUIRED int32 = 25 //Additional pre-authentication required + KDC_ERR_SERVER_NOMATCH int32 = 26 //Requested server and ticket don't match + KDC_ERR_MUST_USE_USER2USER int32 = 27 //Server principal valid for user2user only + KDC_ERR_PATH_NOT_ACCEPTED int32 = 28 //KDC Policy rejects transited path + KDC_ERR_SVC_UNAVAILABLE int32 = 29 //A service is not available + KRB_AP_ERR_BAD_INTEGRITY int32 = 31 //Integrity check on decrypted field failed + KRB_AP_ERR_TKT_EXPIRED int32 = 32 //Ticket expired + KRB_AP_ERR_TKT_NYV int32 = 33 //Ticket not yet valid + KRB_AP_ERR_REPEAT int32 = 34 //Request is a replay + KRB_AP_ERR_NOT_US int32 = 35 //The ticket isn't for us + KRB_AP_ERR_BADMATCH int32 = 36 //Ticket and authenticator don't match + KRB_AP_ERR_SKEW int32 = 37 //Clock skew too great + KRB_AP_ERR_BADADDR int32 = 38 //Incorrect net address + KRB_AP_ERR_BADVERSION int32 = 39 //Protocol version mismatch + KRB_AP_ERR_MSG_TYPE int32 = 40 //Invalid msg type + KRB_AP_ERR_MODIFIED int32 = 41 //Message stream modified + KRB_AP_ERR_BADORDER int32 = 42 //Message out of order + KRB_AP_ERR_BADKEYVER int32 = 44 //Specified version of key is not available + KRB_AP_ERR_NOKEY int32 = 45 //Service key not available + KRB_AP_ERR_MUT_FAIL int32 = 46 //Mutual authentication failed + KRB_AP_ERR_BADDIRECTION int32 = 47 //Incorrect message direction + KRB_AP_ERR_METHOD int32 = 48 //Alternative authentication method required + KRB_AP_ERR_BADSEQ int32 = 49 //Incorrect sequence number in message + KRB_AP_ERR_INAPP_CKSUM int32 = 50 //Inappropriate type of checksum in message + KRB_AP_PATH_NOT_ACCEPTED int32 = 51 //Policy rejects transited path + KRB_ERR_RESPONSE_TOO_BIG int32 = 52 //Response too big for UDP; retry with TCP + KRB_ERR_GENERIC int32 = 60 //Generic error (description in e-text) + KRB_ERR_FIELD_TOOLONG int32 = 61 //Field is too long for this implementation + KDC_ERROR_CLIENT_NOT_TRUSTED int32 = 62 //Reserved for PKINIT + KDC_ERROR_KDC_NOT_TRUSTED int32 = 63 //Reserved for PKINIT + KDC_ERROR_INVALID_SIG int32 = 64 //Reserved for PKINIT + KDC_ERR_KEY_TOO_WEAK int32 = 65 //Reserved for PKINIT + KDC_ERR_CERTIFICATE_MISMATCH int32 = 66 //Reserved for PKINIT + KRB_AP_ERR_NO_TGT int32 = 67 //No TGT available to validate USER-TO-USER + KDC_ERR_WRONG_REALM int32 = 68 //Reserved for future use + KRB_AP_ERR_USER_TO_USER_REQUIRED int32 = 69 //Ticket must be for USER-TO-USER + KDC_ERR_CANT_VERIFY_CERTIFICATE int32 = 70 //Reserved for PKINIT + KDC_ERR_INVALID_CERTIFICATE int32 = 71 //Reserved for PKINIT + KDC_ERR_REVOKED_CERTIFICATE int32 = 72 //Reserved for PKINIT + KDC_ERR_REVOCATION_STATUS_UNKNOWN int32 = 73 //Reserved for PKINIT + KDC_ERR_REVOCATION_STATUS_UNAVAILABLE int32 = 74 //Reserved for PKINIT + KDC_ERR_CLIENT_NAME_MISMATCH int32 = 75 //Reserved for PKINIT + KDC_ERR_KDC_NAME_MISMATCH int32 = 76 //Reserved for PKINIT +) + +// Lookup an error code description. +func Lookup(i int32) string { + if s, ok := errorcodeLookup[i]; ok { + return fmt.Sprintf("(%d) %s", i, s) + } + return fmt.Sprintf("Unknown ErrorCode %d", i) +} + +var errorcodeLookup = map[int32]string{ + KDC_ERR_NONE: "KDC_ERR_NONE No error", + KDC_ERR_NAME_EXP: "KDC_ERR_NAME_EXP Client's entry in database has expired", + KDC_ERR_SERVICE_EXP: "KDC_ERR_SERVICE_EXP Server's entry in database has expired", + KDC_ERR_BAD_PVNO: "KDC_ERR_BAD_PVNO Requested protocol version number not supported", + KDC_ERR_C_OLD_MAST_KVNO: "KDC_ERR_C_OLD_MAST_KVNO Client's key encrypted in old master key", + KDC_ERR_S_OLD_MAST_KVNO: "KDC_ERR_S_OLD_MAST_KVNO Server's key encrypted in old master key", + KDC_ERR_C_PRINCIPAL_UNKNOWN: "KDC_ERR_C_PRINCIPAL_UNKNOWN Client not found in Kerberos database", + KDC_ERR_S_PRINCIPAL_UNKNOWN: "KDC_ERR_S_PRINCIPAL_UNKNOWN Server not found in Kerberos database", + KDC_ERR_PRINCIPAL_NOT_UNIQUE: "KDC_ERR_PRINCIPAL_NOT_UNIQUE Multiple principal entries in database", + KDC_ERR_NULL_KEY: "KDC_ERR_NULL_KEY The client or server has a null key", + KDC_ERR_CANNOT_POSTDATE: "KDC_ERR_CANNOT_POSTDATE Ticket not eligible for postdating", + KDC_ERR_NEVER_VALID: "KDC_ERR_NEVER_VALID Requested starttime is later than end time", + KDC_ERR_POLICY: "KDC_ERR_POLICY KDC policy rejects request", + KDC_ERR_BADOPTION: "KDC_ERR_BADOPTION KDC cannot accommodate requested option", + KDC_ERR_ETYPE_NOSUPP: "KDC_ERR_ETYPE_NOSUPP KDC has no support for encryption type", + KDC_ERR_SUMTYPE_NOSUPP: "KDC_ERR_SUMTYPE_NOSUPP KDC has no support for checksum type", + KDC_ERR_PADATA_TYPE_NOSUPP: "KDC_ERR_PADATA_TYPE_NOSUPP KDC has no support for padata type", + KDC_ERR_TRTYPE_NOSUPP: "KDC_ERR_TRTYPE_NOSUPP KDC has no support for transited type", + KDC_ERR_CLIENT_REVOKED: "KDC_ERR_CLIENT_REVOKED Clients credentials have been revoked", + KDC_ERR_SERVICE_REVOKED: "KDC_ERR_SERVICE_REVOKED Credentials for server have been revoked", + KDC_ERR_TGT_REVOKED: "KDC_ERR_TGT_REVOKED TGT has been revoked", + KDC_ERR_CLIENT_NOTYET: "KDC_ERR_CLIENT_NOTYET Client not yet valid; try again later", + KDC_ERR_SERVICE_NOTYET: "KDC_ERR_SERVICE_NOTYET Server not yet valid; try again later", + KDC_ERR_KEY_EXPIRED: "KDC_ERR_KEY_EXPIRED Password has expired; change password to reset", + KDC_ERR_PREAUTH_FAILED: "KDC_ERR_PREAUTH_FAILED Pre-authentication information was invalid", + KDC_ERR_PREAUTH_REQUIRED: "KDC_ERR_PREAUTH_REQUIRED Additional pre-authentication required", + KDC_ERR_SERVER_NOMATCH: "KDC_ERR_SERVER_NOMATCH Requested server and ticket don't match", + KDC_ERR_MUST_USE_USER2USER: "KDC_ERR_MUST_USE_USER2USER Server principal valid for user2user only", + KDC_ERR_PATH_NOT_ACCEPTED: "KDC_ERR_PATH_NOT_ACCEPTED KDC Policy rejects transited path", + KDC_ERR_SVC_UNAVAILABLE: "KDC_ERR_SVC_UNAVAILABLE A service is not available", + KRB_AP_ERR_BAD_INTEGRITY: "KRB_AP_ERR_BAD_INTEGRITY Integrity check on decrypted field failed", + KRB_AP_ERR_TKT_EXPIRED: "KRB_AP_ERR_TKT_EXPIRED Ticket expired", + KRB_AP_ERR_TKT_NYV: "KRB_AP_ERR_TKT_NYV Ticket not yet valid", + KRB_AP_ERR_REPEAT: "KRB_AP_ERR_REPEAT Request is a replay", + KRB_AP_ERR_NOT_US: "KRB_AP_ERR_NOT_US The ticket isn't for us", + KRB_AP_ERR_BADMATCH: "KRB_AP_ERR_BADMATCH Ticket and authenticator don't match", + KRB_AP_ERR_SKEW: "KRB_AP_ERR_SKEW Clock skew too great", + KRB_AP_ERR_BADADDR: "KRB_AP_ERR_BADADDR Incorrect net address", + KRB_AP_ERR_BADVERSION: "KRB_AP_ERR_BADVERSION Protocol version mismatch", + KRB_AP_ERR_MSG_TYPE: "KRB_AP_ERR_MSG_TYPE Invalid msg type", + KRB_AP_ERR_MODIFIED: "KRB_AP_ERR_MODIFIED Message stream modified", + KRB_AP_ERR_BADORDER: "KRB_AP_ERR_BADORDER Message out of order", + KRB_AP_ERR_BADKEYVER: "KRB_AP_ERR_BADKEYVER Specified version of key is not available", + KRB_AP_ERR_NOKEY: "KRB_AP_ERR_NOKEY Service key not available", + KRB_AP_ERR_MUT_FAIL: "KRB_AP_ERR_MUT_FAIL Mutual authentication failed", + KRB_AP_ERR_BADDIRECTION: "KRB_AP_ERR_BADDIRECTION Incorrect message direction", + KRB_AP_ERR_METHOD: "KRB_AP_ERR_METHOD Alternative authentication method required", + KRB_AP_ERR_BADSEQ: "KRB_AP_ERR_BADSEQ Incorrect sequence number in message", + KRB_AP_ERR_INAPP_CKSUM: "KRB_AP_ERR_INAPP_CKSUM Inappropriate type of checksum in message", + KRB_AP_PATH_NOT_ACCEPTED: "KRB_AP_PATH_NOT_ACCEPTED Policy rejects transited path", + KRB_ERR_RESPONSE_TOO_BIG: "KRB_ERR_RESPONSE_TOO_BIG Response too big for UDP; retry with TCP", + KRB_ERR_GENERIC: "KRB_ERR_GENERIC Generic error (description in e-text)", + KRB_ERR_FIELD_TOOLONG: "KRB_ERR_FIELD_TOOLONG Field is too long for this implementation", + KDC_ERROR_CLIENT_NOT_TRUSTED: "KDC_ERROR_CLIENT_NOT_TRUSTED Reserved for PKINIT", + KDC_ERROR_KDC_NOT_TRUSTED: "KDC_ERROR_KDC_NOT_TRUSTED Reserved for PKINIT", + KDC_ERROR_INVALID_SIG: "KDC_ERROR_INVALID_SIG Reserved for PKINIT", + KDC_ERR_KEY_TOO_WEAK: "KDC_ERR_KEY_TOO_WEAK Reserved for PKINIT", + KDC_ERR_CERTIFICATE_MISMATCH: "KDC_ERR_CERTIFICATE_MISMATCH Reserved for PKINIT", + KRB_AP_ERR_NO_TGT: "KRB_AP_ERR_NO_TGT No TGT available to validate USER-TO-USER", + KDC_ERR_WRONG_REALM: "KDC_ERR_WRONG_REALM Reserved for future use", + KRB_AP_ERR_USER_TO_USER_REQUIRED: "KRB_AP_ERR_USER_TO_USER_REQUIRED Ticket must be for USER-TO-USER", + KDC_ERR_CANT_VERIFY_CERTIFICATE: "KDC_ERR_CANT_VERIFY_CERTIFICATE Reserved for PKINIT", + KDC_ERR_INVALID_CERTIFICATE: "KDC_ERR_INVALID_CERTIFICATE Reserved for PKINIT", + KDC_ERR_REVOKED_CERTIFICATE: "KDC_ERR_REVOKED_CERTIFICATE Reserved for PKINIT", + KDC_ERR_REVOCATION_STATUS_UNKNOWN: "KDC_ERR_REVOCATION_STATUS_UNKNOWN Reserved for PKINIT", + KDC_ERR_REVOCATION_STATUS_UNAVAILABLE: "KDC_ERR_REVOCATION_STATUS_UNAVAILABLE Reserved for PKINIT", + KDC_ERR_CLIENT_NAME_MISMATCH: "KDC_ERR_CLIENT_NAME_MISMATCH Reserved for PKINIT", + KDC_ERR_KDC_NAME_MISMATCH: "KDC_ERR_KDC_NAME_MISMATCH Reserved for PKINIT", +} diff --git a/vendor/github.com/jcmturner/gokrb5/v8/iana/etypeID/constants.go b/vendor/github.com/jcmturner/gokrb5/v8/iana/etypeID/constants.go new file mode 100644 index 000000000000..46a0d748fb76 --- /dev/null +++ b/vendor/github.com/jcmturner/gokrb5/v8/iana/etypeID/constants.go @@ -0,0 +1,101 @@ +// Package etypeID provides Kerberos 5 encryption type assigned numbers. +package etypeID + +// Kerberos encryption type assigned numbers. +const ( + //RESERVED : 0 + DES_CBC_CRC int32 = 1 + DES_CBC_MD4 int32 = 2 + DES_CBC_MD5 int32 = 3 + DES_CBC_RAW int32 = 4 + DES3_CBC_MD5 int32 = 5 + DES3_CBC_RAW int32 = 6 + DES3_CBC_SHA1 int32 = 7 + DES_HMAC_SHA1 int32 = 8 + DSAWITHSHA1_CMSOID int32 = 9 + MD5WITHRSAENCRYPTION_CMSOID int32 = 10 + SHA1WITHRSAENCRYPTION_CMSOID int32 = 11 + RC2CBC_ENVOID int32 = 12 + RSAENCRYPTION_ENVOID int32 = 13 + RSAES_OAEP_ENV_OID int32 = 14 + DES_EDE3_CBC_ENV_OID int32 = 15 + DES3_CBC_SHA1_KD int32 = 16 + AES128_CTS_HMAC_SHA1_96 int32 = 17 + AES256_CTS_HMAC_SHA1_96 int32 = 18 + AES128_CTS_HMAC_SHA256_128 int32 = 19 + AES256_CTS_HMAC_SHA384_192 int32 = 20 + //UNASSIGNED : 21-22 + RC4_HMAC int32 = 23 + RC4_HMAC_EXP int32 = 24 + CAMELLIA128_CTS_CMAC int32 = 25 + CAMELLIA256_CTS_CMAC int32 = 26 + //UNASSIGNED : 27-64 + SUBKEY_KEYMATERIAL int32 = 65 + //UNASSIGNED : 66-2147483647 +) + +// ETypesByName is a map of EncType names to their assigned EncType number. +var ETypesByName = map[string]int32{ + "des-cbc-crc": DES_CBC_CRC, + "des-cbc-md4": DES_CBC_MD4, + "des-cbc-md5": DES_CBC_MD5, + "des-cbc-raw": DES_CBC_RAW, + "des3-cbc-md5": DES3_CBC_MD5, + "des3-cbc-raw": DES3_CBC_RAW, + "des3-cbc-sha1": DES3_CBC_SHA1, + "des3-hmac-sha1": DES_HMAC_SHA1, + "des3-cbc-sha1-kd": DES3_CBC_SHA1_KD, + "des-hmac-sha1": DES_HMAC_SHA1, + "dsaWithSHA1-CmsOID": DSAWITHSHA1_CMSOID, + "md5WithRSAEncryption-CmsOID": MD5WITHRSAENCRYPTION_CMSOID, + "sha1WithRSAEncryption-CmsOID": SHA1WITHRSAENCRYPTION_CMSOID, + "rc2CBC-EnvOID": RC2CBC_ENVOID, + "rsaEncryption-EnvOID": RSAENCRYPTION_ENVOID, + "rsaES-OAEP-ENV-OID": RSAES_OAEP_ENV_OID, + "des-ede3-cbc-Env-OID": DES_EDE3_CBC_ENV_OID, + "aes128-cts-hmac-sha1-96": AES128_CTS_HMAC_SHA1_96, + "aes128-cts": AES128_CTS_HMAC_SHA1_96, + "aes128-sha1": AES128_CTS_HMAC_SHA1_96, + "aes256-cts-hmac-sha1-96": AES256_CTS_HMAC_SHA1_96, + "aes256-cts": AES256_CTS_HMAC_SHA1_96, + "aes256-sha1": AES256_CTS_HMAC_SHA1_96, + "aes128-cts-hmac-sha256-128": AES128_CTS_HMAC_SHA256_128, + "aes128-sha2": AES128_CTS_HMAC_SHA256_128, + "aes256-cts-hmac-sha384-192": AES256_CTS_HMAC_SHA384_192, + "aes256-sha2": AES256_CTS_HMAC_SHA384_192, + "arcfour-hmac": RC4_HMAC, + "rc4-hmac": RC4_HMAC, + "arcfour-hmac-md5": RC4_HMAC, + "arcfour-hmac-exp": RC4_HMAC_EXP, + "rc4-hmac-exp": RC4_HMAC_EXP, + "arcfour-hmac-md5-exp": RC4_HMAC_EXP, + "camellia128-cts-cmac": CAMELLIA128_CTS_CMAC, + "camellia128-cts": CAMELLIA128_CTS_CMAC, + "camellia256-cts-cmac": CAMELLIA256_CTS_CMAC, + "camellia256-cts": CAMELLIA256_CTS_CMAC, + "subkey-keymaterial": SUBKEY_KEYMATERIAL, +} + +// EtypeSupported resolves the etype name string to the etype ID. +// If zero is returned the etype is not supported by gokrb5. +func EtypeSupported(etype string) int32 { + // Slice of supported enctype IDs + s := []int32{ + AES128_CTS_HMAC_SHA1_96, + AES256_CTS_HMAC_SHA1_96, + AES128_CTS_HMAC_SHA256_128, + AES256_CTS_HMAC_SHA384_192, + DES3_CBC_SHA1_KD, + RC4_HMAC, + } + id := ETypesByName[etype] + if id == 0 { + return id + } + for _, sid := range s { + if id == sid { + return id + } + } + return 0 +} diff --git a/vendor/github.com/jcmturner/gokrb5/v8/iana/flags/constants.go b/vendor/github.com/jcmturner/gokrb5/v8/iana/flags/constants.go new file mode 100644 index 000000000000..787801f8f994 --- /dev/null +++ b/vendor/github.com/jcmturner/gokrb5/v8/iana/flags/constants.go @@ -0,0 +1,36 @@ +// Package flags provides Kerberos 5 flag assigned numbers. +package flags + +// Flag values for KRB5 messages and tickets. +const ( + Reserved = 0 + Forwardable = 1 + Forwarded = 2 + Proxiable = 3 + Proxy = 4 + AllowPostDate = 5 + MayPostDate = 5 + PostDated = 6 + Invalid = 7 + Renewable = 8 + Initial = 9 + PreAuthent = 10 + HWAuthent = 11 + OptHardwareAuth = 11 + RequestAnonymous = 12 + TransitedPolicyChecked = 12 + OKAsDelegate = 13 + EncPARep = 15 + Canonicalize = 15 + DisableTransitedCheck = 26 + RenewableOK = 27 + EncTktInSkey = 28 + Renew = 30 + Validate = 31 + + // AP Option Flags + // 0 Reserved for future use. + APOptionUseSessionKey = 1 + APOptionMutualRequired = 2 + // 3-31 Reserved for future use. +) diff --git a/vendor/github.com/jcmturner/gokrb5/v8/iana/keyusage/constants.go b/vendor/github.com/jcmturner/gokrb5/v8/iana/keyusage/constants.go new file mode 100644 index 000000000000..5b232d1d40de --- /dev/null +++ b/vendor/github.com/jcmturner/gokrb5/v8/iana/keyusage/constants.go @@ -0,0 +1,42 @@ +// Package keyusage provides Kerberos 5 key usage assigned numbers. +package keyusage + +// Key usage numbers. +const ( + AS_REQ_PA_ENC_TIMESTAMP = 1 + KDC_REP_TICKET = 2 + AS_REP_ENCPART = 3 + TGS_REQ_KDC_REQ_BODY_AUTHDATA_SESSION_KEY = 4 + TGS_REQ_KDC_REQ_BODY_AUTHDATA_SUB_KEY = 5 + TGS_REQ_PA_TGS_REQ_AP_REQ_AUTHENTICATOR_CHKSUM = 6 + TGS_REQ_PA_TGS_REQ_AP_REQ_AUTHENTICATOR = 7 + TGS_REP_ENCPART_SESSION_KEY = 8 + TGS_REP_ENCPART_AUTHENTICATOR_SUB_KEY = 9 + AP_REQ_AUTHENTICATOR_CHKSUM = 10 + AP_REQ_AUTHENTICATOR = 11 + AP_REP_ENCPART = 12 + KRB_PRIV_ENCPART = 13 + KRB_CRED_ENCPART = 14 + KRB_SAFE_CHKSUM = 15 + KERB_NON_KERB_SALT = 16 + KERB_NON_KERB_CKSUM_SALT = 17 + //18. Reserved for future use in Kerberos and related protocols. + AD_KDC_ISSUED_CHKSUM = 19 + //20-21. Reserved for future use in Kerberos and related protocols. + GSSAPI_ACCEPTOR_SEAL = 22 + GSSAPI_ACCEPTOR_SIGN = 23 + GSSAPI_INITIATOR_SEAL = 24 + GSSAPI_INITIATOR_SIGN = 25 + KEY_USAGE_FAST_REQ_CHKSUM = 50 + KEY_USAGE_FAST_ENC = 51 + KEY_USAGE_FAST_REP = 52 + KEY_USAGE_FAST_FINISHED = 53 + KEY_USAGE_ENC_CHALLENGE_CLIENT = 54 + KEY_USAGE_ENC_CHALLENGE_KDC = 55 + KEY_USAGE_AS_REQ = 56 + //26-511. Reserved for future use in Kerberos and related protocols. + //512-1023. Reserved for uses internal to a Kerberos implementation. + //1024. Encryption for application use in protocols that do not specify key usage values + //1025. Checksums for application use in protocols that do not specify key usage values + //1026-2047. Reserved for application use. +) diff --git a/vendor/github.com/jcmturner/gokrb5/v8/iana/msgtype/constants.go b/vendor/github.com/jcmturner/gokrb5/v8/iana/msgtype/constants.go new file mode 100644 index 000000000000..ad21810b675c --- /dev/null +++ b/vendor/github.com/jcmturner/gokrb5/v8/iana/msgtype/constants.go @@ -0,0 +1,18 @@ +// Package msgtype provides Kerberos 5 message type assigned numbers. +package msgtype + +// KRB message type IDs. +const ( + KRB_AS_REQ = 10 //Request for initial authentication + KRB_AS_REP = 11 //Response to KRB_AS_REQ request + KRB_TGS_REQ = 12 //Request for authentication based on TGT + KRB_TGS_REP = 13 //Response to KRB_TGS_REQ request + KRB_AP_REQ = 14 //Application request to server + KRB_AP_REP = 15 //Response to KRB_AP_REQ_MUTUAL + KRB_RESERVED16 = 16 //Reserved for user-to-user krb_tgt_request + KRB_RESERVED17 = 17 //Reserved for user-to-user krb_tgt_reply + KRB_SAFE = 20 // Safe (checksummed) application message + KRB_PRIV = 21 // Private (encrypted) application message + KRB_CRED = 22 //Private (encrypted) message to forward credentials + KRB_ERROR = 30 //Error response +) diff --git a/vendor/github.com/jcmturner/gokrb5/v8/iana/nametype/constants.go b/vendor/github.com/jcmturner/gokrb5/v8/iana/nametype/constants.go new file mode 100644 index 000000000000..c111a05f9267 --- /dev/null +++ b/vendor/github.com/jcmturner/gokrb5/v8/iana/nametype/constants.go @@ -0,0 +1,15 @@ +// Package nametype provides Kerberos 5 principal name type numbers. +package nametype + +// Kerberos name type IDs. +const ( + KRB_NT_UNKNOWN int32 = 0 //Name type not known + KRB_NT_PRINCIPAL int32 = 1 //Just the name of the principal as in DCE, or for users + KRB_NT_SRV_INST int32 = 2 //Service and other unique instance (krbtgt) + KRB_NT_SRV_HST int32 = 3 //Service with host name as instance (telnet, rcommands) + KRB_NT_SRV_XHST int32 = 4 //Service with host as remaining components + KRB_NT_UID int32 = 5 //Unique ID + KRB_NT_X500_PRINCIPAL int32 = 6 //Encoded X.509 Distinguished name [RFC2253] + KRB_NT_SMTP_NAME int32 = 7 //Name in form of SMTP email name (e.g., user@example.com) + KRB_NT_ENTERPRISE int32 = 10 //Enterprise name; may be mapped to principal name +) diff --git a/vendor/github.com/jcmturner/gokrb5/v8/iana/patype/constants.go b/vendor/github.com/jcmturner/gokrb5/v8/iana/patype/constants.go new file mode 100644 index 000000000000..aa04f63765c0 --- /dev/null +++ b/vendor/github.com/jcmturner/gokrb5/v8/iana/patype/constants.go @@ -0,0 +1,77 @@ +// Package patype provides Kerberos 5 pre-authentication type assigned numbers. +package patype + +// Kerberos pre-authentication type assigned numbers. +const ( + PA_TGS_REQ int32 = 1 + PA_ENC_TIMESTAMP int32 = 2 + PA_PW_SALT int32 = 3 + //RESERVED : 4 + PA_ENC_UNIX_TIME int32 = 5 + PA_SANDIA_SECUREID int32 = 6 + PA_SESAME int32 = 7 + PA_OSF_DCE int32 = 8 + PA_CYBERSAFE_SECUREID int32 = 9 + PA_AFS3_SALT int32 = 10 + PA_ETYPE_INFO int32 = 11 + PA_SAM_CHALLENGE int32 = 12 + PA_SAM_RESPONSE int32 = 13 + PA_PK_AS_REQ_OLD int32 = 14 + PA_PK_AS_REP_OLD int32 = 15 + PA_PK_AS_REQ int32 = 16 + PA_PK_AS_REP int32 = 17 + PA_PK_OCSP_RESPONSE int32 = 18 + PA_ETYPE_INFO2 int32 = 19 + PA_USE_SPECIFIED_KVNO int32 = 20 + PA_SVR_REFERRAL_INFO int32 = 20 + PA_SAM_REDIRECT int32 = 21 + PA_GET_FROM_TYPED_DATA int32 = 22 + TD_PADATA int32 = 22 + PA_SAM_ETYPE_INFO int32 = 23 + PA_ALT_PRINC int32 = 24 + PA_SERVER_REFERRAL int32 = 25 + //UNASSIGNED : 26-29 + PA_SAM_CHALLENGE2 int32 = 30 + PA_SAM_RESPONSE2 int32 = 31 + //UNASSIGNED : 32-40 + PA_EXTRA_TGT int32 = 41 + //UNASSIGNED : 42-100 + TD_PKINIT_CMS_CERTIFICATES int32 = 101 + TD_KRB_PRINCIPAL int32 = 102 + TD_KRB_REALM int32 = 103 + TD_TRUSTED_CERTIFIERS int32 = 104 + TD_CERTIFICATE_INDEX int32 = 105 + TD_APP_DEFINED_ERROR int32 = 106 + TD_REQ_NONCE int32 = 107 + TD_REQ_SEQ int32 = 108 + TD_DH_PARAMETERS int32 = 109 + //UNASSIGNED : 110 + TD_CMS_DIGEST_ALGORITHMS int32 = 111 + TD_CERT_DIGEST_ALGORITHMS int32 = 112 + //UNASSIGNED : 113-127 + PA_PAC_REQUEST int32 = 128 + PA_FOR_USER int32 = 129 + PA_FOR_X509_USER int32 = 130 + PA_FOR_CHECK_DUPS int32 = 131 + PA_AS_CHECKSUM int32 = 132 + PA_FX_COOKIE int32 = 133 + PA_AUTHENTICATION_SET int32 = 134 + PA_AUTH_SET_SELECTED int32 = 135 + PA_FX_FAST int32 = 136 + PA_FX_ERROR int32 = 137 + PA_ENCRYPTED_CHALLENGE int32 = 138 + //UNASSIGNED : 139-140 + PA_OTP_CHALLENGE int32 = 141 + PA_OTP_REQUEST int32 = 142 + PA_OTP_CONFIRM int32 = 143 + PA_OTP_PIN_CHANGE int32 = 144 + PA_EPAK_AS_REQ int32 = 145 + PA_EPAK_AS_REP int32 = 146 + PA_PKINIT_KX int32 = 147 + PA_PKU2U_NAME int32 = 148 + PA_REQ_ENC_PA_REP int32 = 149 + PA_AS_FRESHNESS int32 = 150 + //UNASSIGNED : 151-164 + PA_SUPPORTED_ETYPES int32 = 165 + PA_EXTENDED_ERROR int32 = 166 +) diff --git a/vendor/github.com/jcmturner/gokrb5/v8/kadmin/changepasswddata.go b/vendor/github.com/jcmturner/gokrb5/v8/kadmin/changepasswddata.go new file mode 100644 index 000000000000..2d68eda1924c --- /dev/null +++ b/vendor/github.com/jcmturner/gokrb5/v8/kadmin/changepasswddata.go @@ -0,0 +1,23 @@ +package kadmin + +import ( + "github.com/jcmturner/gofork/encoding/asn1" + "github.com/jcmturner/gokrb5/v8/types" +) + +// ChangePasswdData is the payload to a password change message. +type ChangePasswdData struct { + NewPasswd []byte `asn1:"explicit,tag:0"` + TargName types.PrincipalName `asn1:"explicit,optional,tag:1"` + TargRealm string `asn1:"generalstring,optional,explicit,tag:2"` +} + +// Marshal ChangePasswdData into a byte slice. +func (c *ChangePasswdData) Marshal() ([]byte, error) { + b, err := asn1.Marshal(*c) + if err != nil { + return []byte{}, err + } + //b = asn1tools.AddASNAppTag(b, asnAppTag.) + return b, nil +} diff --git a/vendor/github.com/jcmturner/gokrb5/v8/kadmin/message.go b/vendor/github.com/jcmturner/gokrb5/v8/kadmin/message.go new file mode 100644 index 000000000000..d1864c998be3 --- /dev/null +++ b/vendor/github.com/jcmturner/gokrb5/v8/kadmin/message.go @@ -0,0 +1,114 @@ +package kadmin + +import ( + "bytes" + "encoding/binary" + "errors" + "fmt" + "math" + + "github.com/jcmturner/gokrb5/v8/messages" + "github.com/jcmturner/gokrb5/v8/types" +) + +const ( + verisonHex = "ff80" +) + +// Request message for changing password. +type Request struct { + APREQ messages.APReq + KRBPriv messages.KRBPriv +} + +// Reply message for a password change. +type Reply struct { + MessageLength int + Version int + APREPLength int + APREP messages.APRep + KRBPriv messages.KRBPriv + KRBError messages.KRBError + IsKRBError bool + ResultCode uint16 + Result string +} + +// Marshal a Request into a byte slice. +func (m *Request) Marshal() (b []byte, err error) { + b = []byte{255, 128} // protocol version number: contains the hex constant 0xff80 (big-endian integer). + ab, e := m.APREQ.Marshal() + if e != nil { + err = fmt.Errorf("error marshaling AP_REQ: %v", e) + return + } + if len(ab) > math.MaxUint16 { + err = errors.New("length of AP_REQ greater then max Uint16 size") + return + } + al := make([]byte, 2) + binary.BigEndian.PutUint16(al, uint16(len(ab))) + b = append(b, al...) + b = append(b, ab...) + pb, e := m.KRBPriv.Marshal() + if e != nil { + err = fmt.Errorf("error marshaling KRB_Priv: %v", e) + return + } + b = append(b, pb...) + if len(b)+2 > math.MaxUint16 { + err = errors.New("length of message greater then max Uint16 size") + return + } + ml := make([]byte, 2) + binary.BigEndian.PutUint16(ml, uint16(len(b)+2)) + b = append(ml, b...) + return +} + +// Unmarshal a byte slice into a Reply. +func (m *Reply) Unmarshal(b []byte) error { + m.MessageLength = int(binary.BigEndian.Uint16(b[0:2])) + m.Version = int(binary.BigEndian.Uint16(b[2:4])) + if m.Version != 1 { + return fmt.Errorf("kadmin reply has incorrect protocol version number: %d", m.Version) + } + m.APREPLength = int(binary.BigEndian.Uint16(b[4:6])) + if m.APREPLength != 0 { + err := m.APREP.Unmarshal(b[6 : 6+m.APREPLength]) + if err != nil { + return err + } + err = m.KRBPriv.Unmarshal(b[6+m.APREPLength : m.MessageLength]) + if err != nil { + return err + } + } else { + m.IsKRBError = true + m.KRBError.Unmarshal(b[6:m.MessageLength]) + m.ResultCode, m.Result = parseResponse(m.KRBError.EData) + } + return nil +} + +func parseResponse(b []byte) (c uint16, s string) { + c = binary.BigEndian.Uint16(b[0:2]) + buf := bytes.NewBuffer(b[2:]) + m := make([]byte, len(b)-2) + binary.Read(buf, binary.BigEndian, &m) + s = string(m) + return +} + +// Decrypt the encrypted part of the KRBError within the change password Reply. +func (m *Reply) Decrypt(key types.EncryptionKey) error { + if m.IsKRBError { + return m.KRBError + } + err := m.KRBPriv.DecryptEncPart(key) + if err != nil { + return err + } + m.ResultCode, m.Result = parseResponse(m.KRBPriv.DecryptedEncPart.UserData) + return nil +} diff --git a/vendor/github.com/jcmturner/gokrb5/v8/kadmin/passwd.go b/vendor/github.com/jcmturner/gokrb5/v8/kadmin/passwd.go new file mode 100644 index 000000000000..db199bffcb22 --- /dev/null +++ b/vendor/github.com/jcmturner/gokrb5/v8/kadmin/passwd.go @@ -0,0 +1,68 @@ +// Package kadmin provides Kerberos administration capabilities. +package kadmin + +import ( + "github.com/jcmturner/gokrb5/v8/crypto" + "github.com/jcmturner/gokrb5/v8/krberror" + "github.com/jcmturner/gokrb5/v8/messages" + "github.com/jcmturner/gokrb5/v8/types" +) + +// ChangePasswdMsg generate a change password request and also return the key needed to decrypt the reply. +func ChangePasswdMsg(cname types.PrincipalName, realm, password string, tkt messages.Ticket, sessionKey types.EncryptionKey) (r Request, k types.EncryptionKey, err error) { + // Create change password data struct and marshal to bytes + chgpasswd := ChangePasswdData{ + NewPasswd: []byte(password), + TargName: cname, + TargRealm: realm, + } + chpwdb, err := chgpasswd.Marshal() + if err != nil { + err = krberror.Errorf(err, krberror.KRBMsgError, "error marshaling change passwd data") + return + } + + // Generate authenticator + auth, err := types.NewAuthenticator(realm, cname) + if err != nil { + err = krberror.Errorf(err, krberror.KRBMsgError, "error generating new authenticator") + return + } + etype, err := crypto.GetEtype(sessionKey.KeyType) + if err != nil { + err = krberror.Errorf(err, krberror.KRBMsgError, "error generating subkey etype") + return + } + err = auth.GenerateSeqNumberAndSubKey(etype.GetETypeID(), etype.GetKeyByteSize()) + if err != nil { + err = krberror.Errorf(err, krberror.KRBMsgError, "error generating subkey") + return + } + k = auth.SubKey + + // Generate AP_REQ + APreq, err := messages.NewAPReq(tkt, sessionKey, auth) + if err != nil { + return + } + + // Form the KRBPriv encpart data + kp := messages.EncKrbPrivPart{ + UserData: chpwdb, + Timestamp: auth.CTime, + Usec: auth.Cusec, + SequenceNumber: auth.SeqNumber, + } + kpriv := messages.NewKRBPriv(kp) + err = kpriv.EncryptEncPart(k) + if err != nil { + err = krberror.Errorf(err, krberror.EncryptingError, "error encrypting change passwd data") + return + } + + r = Request{ + APREQ: APreq, + KRBPriv: kpriv, + } + return +} diff --git a/vendor/github.com/jcmturner/gokrb5/v8/keytab/keytab.go b/vendor/github.com/jcmturner/gokrb5/v8/keytab/keytab.go new file mode 100644 index 000000000000..5c2e9d79a1b9 --- /dev/null +++ b/vendor/github.com/jcmturner/gokrb5/v8/keytab/keytab.go @@ -0,0 +1,530 @@ +// Package keytab implements Kerberos keytabs: https://web.mit.edu/kerberos/krb5-devel/doc/formats/keytab_file_format.html. +package keytab + +import ( + "bytes" + "encoding/binary" + "encoding/json" + "errors" + "fmt" + "io" + "io/ioutil" + "strings" + "time" + "unsafe" + + "github.com/jcmturner/gokrb5/v8/crypto" + "github.com/jcmturner/gokrb5/v8/types" +) + +const ( + keytabFirstByte byte = 05 +) + +// Keytab struct. +type Keytab struct { + version uint8 + Entries []entry +} + +// Keytab entry struct. +type entry struct { + Principal principal + Timestamp time.Time + KVNO8 uint8 + Key types.EncryptionKey + KVNO uint32 +} + +func (e entry) String() string { + return fmt.Sprintf("% 4d %s %-56s %2d %-64x", + e.KVNO8, + e.Timestamp.Format("02/01/06 15:04:05"), + e.Principal.String(), + e.Key.KeyType, + e.Key.KeyValue, + ) +} + +// Keytab entry principal struct. +type principal struct { + NumComponents int16 `json:"-"` + Realm string + Components []string + NameType int32 +} + +func (p principal) String() string { + return fmt.Sprintf("%s@%s", strings.Join(p.Components, "/"), p.Realm) +} + +// New creates new, empty Keytab type. +func New() *Keytab { + var e []entry + return &Keytab{ + version: 2, + Entries: e, + } +} + +// GetEncryptionKey returns the EncryptionKey from the Keytab for the newest entry with the required kvno, etype and matching principal. +// If the kvno is zero then the latest kvno will be returned. The kvno is also returned for +func (kt *Keytab) GetEncryptionKey(princName types.PrincipalName, realm string, kvno int, etype int32) (types.EncryptionKey, int, error) { + var key types.EncryptionKey + var t time.Time + var kv int + for _, k := range kt.Entries { + if k.Principal.Realm == realm && len(k.Principal.Components) == len(princName.NameString) && + k.Key.KeyType == etype && + (k.KVNO == uint32(kvno) || kvno == 0) && + k.Timestamp.After(t) { + p := true + for i, n := range k.Principal.Components { + if princName.NameString[i] != n { + p = false + break + } + } + if p { + key = k.Key + kv = int(k.KVNO) + t = k.Timestamp + } + } + } + if len(key.KeyValue) < 1 { + return key, 0, fmt.Errorf("matching key not found in keytab. Looking for %v realm: %v kvno: %v etype: %v", princName.NameString, realm, kvno, etype) + } + return key, kv, nil +} + +// Create a new Keytab entry. +func newEntry() entry { + var b []byte + return entry{ + Principal: newPrincipal(), + Timestamp: time.Time{}, + KVNO8: 0, + Key: types.EncryptionKey{ + KeyType: 0, + KeyValue: b, + }, + KVNO: 0, + } +} + +func (kt Keytab) String() string { + var s string + s = `KVNO Timestamp Principal ET Key +---- ----------------- -------------------------------------------------------- -- ---------------------------------------------------------------- +` + for _, entry := range kt.Entries { + s += entry.String() + "\n" + } + return s +} + +// AddEntry adds an entry to the keytab. The password should be provided in plain text and it will be converted using the defined enctype to be stored. +func (kt *Keytab) AddEntry(principalName, realm, password string, ts time.Time, KVNO uint8, encType int32) error { + // Generate a key from the password + princ, _ := types.ParseSPNString(principalName) + key, _, err := crypto.GetKeyFromPassword(password, princ, realm, encType, types.PADataSequence{}) + if err != nil { + return err + } + + // Populate the keytab entry principal + ktep := newPrincipal() + ktep.NumComponents = int16(len(princ.NameString)) + if kt.version == 1 { + ktep.NumComponents += 1 + } + + ktep.Realm = realm + ktep.Components = princ.NameString + ktep.NameType = princ.NameType + + // Populate the keytab entry + e := newEntry() + e.Principal = ktep + e.Timestamp = ts + e.KVNO8 = KVNO + e.KVNO = uint32(KVNO) + e.Key = key + + kt.Entries = append(kt.Entries, e) + return nil +} + +// Create a new principal. +func newPrincipal() principal { + var c []string + return principal{ + NumComponents: 0, + Realm: "", + Components: c, + NameType: 0, + } +} + +// Load a Keytab file into a Keytab type. +func Load(ktPath string) (*Keytab, error) { + kt := new(Keytab) + b, err := ioutil.ReadFile(ktPath) + if err != nil { + return kt, err + } + err = kt.Unmarshal(b) + return kt, err +} + +// Marshal keytab into byte slice +func (kt *Keytab) Marshal() ([]byte, error) { + b := []byte{keytabFirstByte, kt.version} + for _, e := range kt.Entries { + eb, err := e.marshal(int(kt.version)) + if err != nil { + return b, err + } + b = append(b, eb...) + } + return b, nil +} + +// Write the keytab bytes to io.Writer. +// Returns the number of bytes written +func (kt *Keytab) Write(w io.Writer) (int, error) { + b, err := kt.Marshal() + if err != nil { + return 0, fmt.Errorf("error marshaling keytab: %v", err) + } + return w.Write(b) +} + +// Unmarshal byte slice of Keytab data into Keytab type. +func (kt *Keytab) Unmarshal(b []byte) error { + if len(b) < 2 { + return fmt.Errorf("byte array is less than 2 bytes: %d", len(b)) + } + + //The first byte of the file always has the value 5 + if b[0] != keytabFirstByte { + return errors.New("invalid keytab data. First byte does not equal 5") + } + //Get keytab version + //The 2nd byte contains the version number (1 or 2) + kt.version = b[1] + if kt.version != 1 && kt.version != 2 { + return errors.New("invalid keytab data. Keytab version is neither 1 nor 2") + } + //Version 1 of the file format uses native byte order for integer representations. Version 2 always uses big-endian byte order + var endian binary.ByteOrder + endian = binary.BigEndian + if kt.version == 1 && isNativeEndianLittle() { + endian = binary.LittleEndian + } + // n tracks position in the byte array + n := 2 + l, err := readInt32(b, &n, &endian) + if err != nil { + return err + } + for l != 0 { + if l < 0 { + //Zero padded so skip over + l = l * -1 + n = n + int(l) + } else { + if n < 0 { + return fmt.Errorf("%d can't be less than zero", n) + } + if n+int(l) > len(b) { + return fmt.Errorf("%s's length is less than %d", b, n+int(l)) + } + eb := b[n : n+int(l)] + n = n + int(l) + ke := newEntry() + // p keeps track as to where we are in the byte stream + var p int + var err error + parsePrincipal(eb, &p, kt, &ke, &endian) + ke.Timestamp, err = readTimestamp(eb, &p, &endian) + if err != nil { + return err + } + rei8, err := readInt8(eb, &p, &endian) + if err != nil { + return err + } + ke.KVNO8 = uint8(rei8) + rei16, err := readInt16(eb, &p, &endian) + if err != nil { + return err + } + ke.Key.KeyType = int32(rei16) + rei16, err = readInt16(eb, &p, &endian) + if err != nil { + return err + } + kl := int(rei16) + ke.Key.KeyValue, err = readBytes(eb, &p, kl, &endian) + if err != nil { + return err + } + // The 32-bit key version overrides the 8-bit key version. + // If at least 4 bytes are left after the other fields are read and they are non-zero + // this indicates the 32-bit version is present. + if len(eb)-p >= 4 { + // The 32-bit key may be present + ri32, err := readInt32(eb, &p, &endian) + if err != nil { + return err + } + ke.KVNO = uint32(ri32) + } + if ke.KVNO == 0 { + // Handles if the value from the last 4 bytes was zero and also if there are not the 4 bytes present. Makes sense to put the same value here as KVNO8 + ke.KVNO = uint32(ke.KVNO8) + } + // Add the entry to the keytab + kt.Entries = append(kt.Entries, ke) + } + // Check if there are still 4 bytes left to read + // Also check that n is greater than zero + if n < 0 || n > len(b) || len(b[n:]) < 4 { + break + } + // Read the size of the next entry + l, err = readInt32(b, &n, &endian) + if err != nil { + return err + } + } + return nil +} + +func (e entry) marshal(v int) ([]byte, error) { + var b []byte + pb, err := e.Principal.marshal(v) + if err != nil { + return b, err + } + b = append(b, pb...) + + var endian binary.ByteOrder + endian = binary.BigEndian + if v == 1 && isNativeEndianLittle() { + endian = binary.LittleEndian + } + + t := make([]byte, 9) + endian.PutUint32(t[0:4], uint32(e.Timestamp.Unix())) + t[4] = e.KVNO8 + endian.PutUint16(t[5:7], uint16(e.Key.KeyType)) + endian.PutUint16(t[7:9], uint16(len(e.Key.KeyValue))) + b = append(b, t...) + + buf := new(bytes.Buffer) + err = binary.Write(buf, endian, e.Key.KeyValue) + if err != nil { + return b, err + } + b = append(b, buf.Bytes()...) + + t = make([]byte, 4) + endian.PutUint32(t, e.KVNO) + b = append(b, t...) + + // Add the length header + t = make([]byte, 4) + endian.PutUint32(t, uint32(len(b))) + b = append(t, b...) + return b, nil +} + +// Parse the Keytab bytes of a principal into a Keytab entry's principal. +func parsePrincipal(b []byte, p *int, kt *Keytab, ke *entry, e *binary.ByteOrder) error { + var err error + ke.Principal.NumComponents, err = readInt16(b, p, e) + if err != nil { + return err + } + if kt.version == 1 { + //In version 1 the number of components includes the realm. Minus 1 to make consistent with version 2 + ke.Principal.NumComponents-- + } + lenRealm, err := readInt16(b, p, e) + if err != nil { + return err + } + realmB, err := readBytes(b, p, int(lenRealm), e) + if err != nil { + return err + } + ke.Principal.Realm = string(realmB) + for i := 0; i < int(ke.Principal.NumComponents); i++ { + l, err := readInt16(b, p, e) + if err != nil { + return err + } + compB, err := readBytes(b, p, int(l), e) + if err != nil { + return err + } + ke.Principal.Components = append(ke.Principal.Components, string(compB)) + } + if kt.version != 1 { + //Name Type is omitted in version 1 + ke.Principal.NameType, err = readInt32(b, p, e) + if err != nil { + return err + } + } + return nil +} + +func (p principal) marshal(v int) ([]byte, error) { + //var b []byte + b := make([]byte, 2) + var endian binary.ByteOrder + endian = binary.BigEndian + if v == 1 && isNativeEndianLittle() { + endian = binary.LittleEndian + } + endian.PutUint16(b[0:], uint16(p.NumComponents)) + realm, err := marshalString(p.Realm, v) + if err != nil { + return b, err + } + b = append(b, realm...) + for _, c := range p.Components { + cb, err := marshalString(c, v) + if err != nil { + return b, err + } + b = append(b, cb...) + } + if v != 1 { + t := make([]byte, 4) + endian.PutUint32(t, uint32(p.NameType)) + b = append(b, t...) + } + return b, nil +} + +func marshalString(s string, v int) ([]byte, error) { + sb := []byte(s) + b := make([]byte, 2) + var endian binary.ByteOrder + endian = binary.BigEndian + if v == 1 && isNativeEndianLittle() { + endian = binary.LittleEndian + } + endian.PutUint16(b[0:], uint16(len(sb))) + buf := new(bytes.Buffer) + err := binary.Write(buf, endian, sb) + if err != nil { + return b, err + } + b = append(b, buf.Bytes()...) + return b, err +} + +// Read bytes representing a timestamp. +func readTimestamp(b []byte, p *int, e *binary.ByteOrder) (time.Time, error) { + i32, err := readInt32(b, p, e) + if err != nil { + return time.Time{}, err + } + return time.Unix(int64(i32), 0), nil +} + +// Read bytes representing an eight bit integer. +func readInt8(b []byte, p *int, e *binary.ByteOrder) (i int8, err error) { + if *p < 0 { + return 0, fmt.Errorf("%d cannot be less than zero", *p) + } + + if (*p + 1) > len(b) { + return 0, fmt.Errorf("%s's length is less than %d", b, *p+1) + } + buf := bytes.NewBuffer(b[*p : *p+1]) + binary.Read(buf, *e, &i) + *p++ + return +} + +// Read bytes representing a sixteen bit integer. +func readInt16(b []byte, p *int, e *binary.ByteOrder) (i int16, err error) { + if *p < 0 { + return 0, fmt.Errorf("%d cannot be less than zero", *p) + } + + if (*p + 2) > len(b) { + return 0, fmt.Errorf("%s's length is less than %d", b, *p+2) + } + + buf := bytes.NewBuffer(b[*p : *p+2]) + binary.Read(buf, *e, &i) + *p += 2 + return +} + +// Read bytes representing a thirty two bit integer. +func readInt32(b []byte, p *int, e *binary.ByteOrder) (i int32, err error) { + if *p < 0 { + return 0, fmt.Errorf("%d cannot be less than zero", *p) + } + + if (*p + 4) > len(b) { + return 0, fmt.Errorf("%s's length is less than %d", b, *p+4) + } + + buf := bytes.NewBuffer(b[*p : *p+4]) + binary.Read(buf, *e, &i) + *p += 4 + return +} + +func readBytes(b []byte, p *int, s int, e *binary.ByteOrder) ([]byte, error) { + if s < 0 { + return nil, fmt.Errorf("%d cannot be less than zero", s) + } + i := *p + s + if i > len(b) { + return nil, fmt.Errorf("%s's length is greater than %d", b, i) + } + buf := bytes.NewBuffer(b[*p:i]) + r := make([]byte, s) + if err := binary.Read(buf, *e, &r); err != nil { + return nil, err + } + *p += s + return r, nil +} + +func isNativeEndianLittle() bool { + var x = 0x012345678 + var p = unsafe.Pointer(&x) + var bp = (*[4]byte)(p) + + var endian bool + if 0x01 == bp[0] { + endian = false + } else if (0x78 & 0xff) == (bp[0] & 0xff) { + endian = true + } else { + // Default to big endian + endian = false + } + return endian +} + +// JSON return information about the keys held in the keytab in a JSON format. +func (kt *Keytab) JSON() (string, error) { + b, err := json.MarshalIndent(kt, "", " ") + if err != nil { + return "", err + } + return string(b), nil +} diff --git a/vendor/github.com/jcmturner/gokrb5/v8/krberror/error.go b/vendor/github.com/jcmturner/gokrb5/v8/krberror/error.go new file mode 100644 index 000000000000..01c6d9904c52 --- /dev/null +++ b/vendor/github.com/jcmturner/gokrb5/v8/krberror/error.go @@ -0,0 +1,67 @@ +// Package krberror provides error type and functions for gokrb5. +package krberror + +import ( + "fmt" + "strings" +) + +// Error type descriptions. +const ( + separator = " < " + EncodingError = "Encoding_Error" + NetworkingError = "Networking_Error" + DecryptingError = "Decrypting_Error" + EncryptingError = "Encrypting_Error" + ChksumError = "Checksum_Error" + KRBMsgError = "KRBMessage_Handling_Error" + ConfigError = "Configuration_Error" + KDCError = "KDC_Error" +) + +// Krberror is an error type for gokrb5 +type Krberror struct { + RootCause string + EText []string +} + +// Error function to implement the error interface. +func (e Krberror) Error() string { + return fmt.Sprintf("[Root cause: %s] ", e.RootCause) + strings.Join(e.EText, separator) +} + +// Add another error statement to the error. +func (e *Krberror) Add(et string, s string) { + e.EText = append([]string{fmt.Sprintf("%s: %s", et, s)}, e.EText...) +} + +// New creates a new instance of Krberror. +func New(et, s string) Krberror { + return Krberror{ + RootCause: et, + EText: []string{s}, + } +} + +// Errorf appends to or creates a new Krberror. +func Errorf(err error, et, format string, a ...interface{}) Krberror { + if e, ok := err.(Krberror); ok { + e.Add(et, fmt.Sprintf(format, a...)) + return e + } + return NewErrorf(et, format+": %s", append(a, err)...) +} + +// NewErrorf creates a new Krberror from a formatted string. +func NewErrorf(et, format string, a ...interface{}) Krberror { + var s string + if len(a) > 0 { + s = fmt.Sprintf("%s: %s", et, fmt.Sprintf(format, a...)) + } else { + s = fmt.Sprintf("%s: %s", et, format) + } + return Krberror{ + RootCause: et, + EText: []string{s}, + } +} diff --git a/vendor/github.com/jcmturner/gokrb5/v8/messages/APRep.go b/vendor/github.com/jcmturner/gokrb5/v8/messages/APRep.go new file mode 100644 index 000000000000..555fb8072074 --- /dev/null +++ b/vendor/github.com/jcmturner/gokrb5/v8/messages/APRep.go @@ -0,0 +1,49 @@ +package messages + +import ( + "fmt" + "time" + + "github.com/jcmturner/gofork/encoding/asn1" + "github.com/jcmturner/gokrb5/v8/iana/asnAppTag" + "github.com/jcmturner/gokrb5/v8/iana/msgtype" + "github.com/jcmturner/gokrb5/v8/krberror" + "github.com/jcmturner/gokrb5/v8/types" +) + +// APRep implements RFC 4120 KRB_AP_REP: https://tools.ietf.org/html/rfc4120#section-5.5.2. +type APRep struct { + PVNO int `asn1:"explicit,tag:0"` + MsgType int `asn1:"explicit,tag:1"` + EncPart types.EncryptedData `asn1:"explicit,tag:2"` +} + +// EncAPRepPart is the encrypted part of KRB_AP_REP. +type EncAPRepPart struct { + CTime time.Time `asn1:"generalized,explicit,tag:0"` + Cusec int `asn1:"explicit,tag:1"` + Subkey types.EncryptionKey `asn1:"optional,explicit,tag:2"` + SequenceNumber int64 `asn1:"optional,explicit,tag:3"` +} + +// Unmarshal bytes b into the APRep struct. +func (a *APRep) Unmarshal(b []byte) error { + _, err := asn1.UnmarshalWithParams(b, a, fmt.Sprintf("application,explicit,tag:%v", asnAppTag.APREP)) + if err != nil { + return processUnmarshalReplyError(b, err) + } + expectedMsgType := msgtype.KRB_AP_REP + if a.MsgType != expectedMsgType { + return krberror.NewErrorf(krberror.KRBMsgError, "message ID does not indicate a KRB_AP_REP. Expected: %v; Actual: %v", expectedMsgType, a.MsgType) + } + return nil +} + +// Unmarshal bytes b into the APRep encrypted part struct. +func (a *EncAPRepPart) Unmarshal(b []byte) error { + _, err := asn1.UnmarshalWithParams(b, a, fmt.Sprintf("application,explicit,tag:%v", asnAppTag.EncAPRepPart)) + if err != nil { + return krberror.Errorf(err, krberror.EncodingError, "AP_REP unmarshal error") + } + return nil +} diff --git a/vendor/github.com/jcmturner/gokrb5/v8/messages/APReq.go b/vendor/github.com/jcmturner/gokrb5/v8/messages/APReq.go new file mode 100644 index 000000000000..183607976314 --- /dev/null +++ b/vendor/github.com/jcmturner/gokrb5/v8/messages/APReq.go @@ -0,0 +1,199 @@ +package messages + +import ( + "fmt" + "time" + + "github.com/jcmturner/gofork/encoding/asn1" + "github.com/jcmturner/gokrb5/v8/asn1tools" + "github.com/jcmturner/gokrb5/v8/crypto" + "github.com/jcmturner/gokrb5/v8/iana" + "github.com/jcmturner/gokrb5/v8/iana/asnAppTag" + "github.com/jcmturner/gokrb5/v8/iana/errorcode" + "github.com/jcmturner/gokrb5/v8/iana/keyusage" + "github.com/jcmturner/gokrb5/v8/iana/msgtype" + "github.com/jcmturner/gokrb5/v8/keytab" + "github.com/jcmturner/gokrb5/v8/krberror" + "github.com/jcmturner/gokrb5/v8/types" +) + +type marshalAPReq struct { + PVNO int `asn1:"explicit,tag:0"` + MsgType int `asn1:"explicit,tag:1"` + APOptions asn1.BitString `asn1:"explicit,tag:2"` + // Ticket needs to be a raw value as it is wrapped in an APPLICATION tag + Ticket asn1.RawValue `asn1:"explicit,tag:3"` + EncryptedAuthenticator types.EncryptedData `asn1:"explicit,tag:4"` +} + +// APReq implements RFC 4120 KRB_AP_REQ: https://tools.ietf.org/html/rfc4120#section-5.5.1. +type APReq struct { + PVNO int `asn1:"explicit,tag:0"` + MsgType int `asn1:"explicit,tag:1"` + APOptions asn1.BitString `asn1:"explicit,tag:2"` + Ticket Ticket `asn1:"explicit,tag:3"` + EncryptedAuthenticator types.EncryptedData `asn1:"explicit,tag:4"` + Authenticator types.Authenticator `asn1:"optional"` +} + +// NewAPReq generates a new KRB_AP_REQ struct. +func NewAPReq(tkt Ticket, sessionKey types.EncryptionKey, auth types.Authenticator) (APReq, error) { + var a APReq + ed, err := encryptAuthenticator(auth, sessionKey, tkt) + if err != nil { + return a, krberror.Errorf(err, krberror.KRBMsgError, "error creating Authenticator for AP_REQ") + } + a = APReq{ + PVNO: iana.PVNO, + MsgType: msgtype.KRB_AP_REQ, + APOptions: types.NewKrbFlags(), + Ticket: tkt, + EncryptedAuthenticator: ed, + } + return a, nil +} + +// Encrypt Authenticator +func encryptAuthenticator(a types.Authenticator, sessionKey types.EncryptionKey, tkt Ticket) (types.EncryptedData, error) { + var ed types.EncryptedData + m, err := a.Marshal() + if err != nil { + return ed, krberror.Errorf(err, krberror.EncodingError, "marshaling error of EncryptedData form of Authenticator") + } + usage := authenticatorKeyUsage(tkt.SName) + ed, err = crypto.GetEncryptedData(m, sessionKey, uint32(usage), tkt.EncPart.KVNO) + if err != nil { + return ed, krberror.Errorf(err, krberror.EncryptingError, "error encrypting Authenticator") + } + return ed, nil +} + +// DecryptAuthenticator decrypts the Authenticator within the AP_REQ. +// sessionKey may simply be the key within the decrypted EncPart of the ticket within the AP_REQ. +func (a *APReq) DecryptAuthenticator(sessionKey types.EncryptionKey) error { + usage := authenticatorKeyUsage(a.Ticket.SName) + ab, e := crypto.DecryptEncPart(a.EncryptedAuthenticator, sessionKey, uint32(usage)) + if e != nil { + return fmt.Errorf("error decrypting authenticator: %v", e) + } + err := a.Authenticator.Unmarshal(ab) + if err != nil { + return fmt.Errorf("error unmarshaling authenticator: %v", err) + } + return nil +} + +func authenticatorKeyUsage(pn types.PrincipalName) int { + if pn.NameString[0] == "krbtgt" { + return keyusage.TGS_REQ_PA_TGS_REQ_AP_REQ_AUTHENTICATOR + } + return keyusage.AP_REQ_AUTHENTICATOR +} + +// Unmarshal bytes b into the APReq struct. +func (a *APReq) Unmarshal(b []byte) error { + var m marshalAPReq + _, err := asn1.UnmarshalWithParams(b, &m, fmt.Sprintf("application,explicit,tag:%v", asnAppTag.APREQ)) + if err != nil { + return krberror.Errorf(err, krberror.EncodingError, "unmarshal error of AP_REQ") + } + if m.MsgType != msgtype.KRB_AP_REQ { + return NewKRBError(types.PrincipalName{}, "", errorcode.KRB_AP_ERR_MSG_TYPE, errorcode.Lookup(errorcode.KRB_AP_ERR_MSG_TYPE)) + } + a.PVNO = m.PVNO + a.MsgType = m.MsgType + a.APOptions = m.APOptions + a.EncryptedAuthenticator = m.EncryptedAuthenticator + a.Ticket, err = unmarshalTicket(m.Ticket.Bytes) + if err != nil { + return krberror.Errorf(err, krberror.EncodingError, "unmarshaling error of Ticket within AP_REQ") + } + return nil +} + +// Marshal APReq struct. +func (a *APReq) Marshal() ([]byte, error) { + m := marshalAPReq{ + PVNO: a.PVNO, + MsgType: a.MsgType, + APOptions: a.APOptions, + EncryptedAuthenticator: a.EncryptedAuthenticator, + } + var b []byte + b, err := a.Ticket.Marshal() + if err != nil { + return b, err + } + m.Ticket = asn1.RawValue{ + Class: asn1.ClassContextSpecific, + IsCompound: true, + Tag: 3, + Bytes: b, + } + mk, err := asn1.Marshal(m) + if err != nil { + return mk, krberror.Errorf(err, krberror.EncodingError, "marshaling error of AP_REQ") + } + mk = asn1tools.AddASNAppTag(mk, asnAppTag.APREQ) + return mk, nil +} + +// Verify an AP_REQ using service's keytab, spn and max acceptable clock skew duration. +// The service ticket encrypted part and authenticator will be decrypted as part of this operation. +func (a *APReq) Verify(kt *keytab.Keytab, d time.Duration, cAddr types.HostAddress, snameOverride *types.PrincipalName) (bool, error) { + // Decrypt ticket's encrypted part with service key + //TODO decrypt with service's session key from its TGT is use-to-user. Need to figure out how to get TGT. + //if types.IsFlagSet(&a.APOptions, flags.APOptionUseSessionKey) { + // err := a.Ticket.Decrypt(tgt.DecryptedEncPart.Key) + // if err != nil { + // return false, krberror.Errorf(err, krberror.DecryptingError, "error decrypting encpart of ticket provided using session key") + // } + //} else { + // err := a.Ticket.DecryptEncPart(*kt, &a.Ticket.SName) + // if err != nil { + // return false, krberror.Errorf(err, krberror.DecryptingError, "error decrypting encpart of service ticket provided") + // } + //} + sname := &a.Ticket.SName + if snameOverride != nil { + sname = snameOverride + } + err := a.Ticket.DecryptEncPart(kt, sname) + if err != nil { + return false, krberror.Errorf(err, krberror.DecryptingError, "error decrypting encpart of service ticket provided") + } + + // Check time validity of ticket + ok, err := a.Ticket.Valid(d) + if err != nil || !ok { + return ok, err + } + + // Check client's address is listed in the client addresses in the ticket + if len(a.Ticket.DecryptedEncPart.CAddr) > 0 { + //If client addresses are present check if any of them match the source IP that sent the APReq + //If there is no match return KRB_AP_ERR_BADADDR error. + if !types.HostAddressesContains(a.Ticket.DecryptedEncPart.CAddr, cAddr) { + return false, NewKRBError(a.Ticket.SName, a.Ticket.Realm, errorcode.KRB_AP_ERR_BADADDR, "client address not within the list contained in the service ticket") + } + } + + // Decrypt authenticator with session key from ticket's encrypted part + err = a.DecryptAuthenticator(a.Ticket.DecryptedEncPart.Key) + if err != nil { + return false, NewKRBError(a.Ticket.SName, a.Ticket.Realm, errorcode.KRB_AP_ERR_BAD_INTEGRITY, "could not decrypt authenticator") + } + + // Check CName in authenticator is the same as that in the ticket + if !a.Authenticator.CName.Equal(a.Ticket.DecryptedEncPart.CName) { + return false, NewKRBError(a.Ticket.SName, a.Ticket.Realm, errorcode.KRB_AP_ERR_BADMATCH, "CName in Authenticator does not match that in service ticket") + } + + // Check the clock skew between the client and the service server + ct := a.Authenticator.CTime.Add(time.Duration(a.Authenticator.Cusec) * time.Microsecond) + t := time.Now().UTC() + if t.Sub(ct) > d || ct.Sub(t) > d { + return false, NewKRBError(a.Ticket.SName, a.Ticket.Realm, errorcode.KRB_AP_ERR_SKEW, fmt.Sprintf("clock skew with client too large. greater than %v seconds", d)) + } + return true, nil +} diff --git a/vendor/github.com/jcmturner/gokrb5/v8/messages/KDCRep.go b/vendor/github.com/jcmturner/gokrb5/v8/messages/KDCRep.go new file mode 100644 index 000000000000..69df9f0f6f54 --- /dev/null +++ b/vendor/github.com/jcmturner/gokrb5/v8/messages/KDCRep.go @@ -0,0 +1,360 @@ +package messages + +// Reference: https://www.ietf.org/rfc/rfc4120.txt +// Section: 5.4.2 + +import ( + "fmt" + "time" + + "github.com/jcmturner/gofork/encoding/asn1" + "github.com/jcmturner/gokrb5/v8/asn1tools" + "github.com/jcmturner/gokrb5/v8/config" + "github.com/jcmturner/gokrb5/v8/credentials" + "github.com/jcmturner/gokrb5/v8/crypto" + "github.com/jcmturner/gokrb5/v8/iana/asnAppTag" + "github.com/jcmturner/gokrb5/v8/iana/flags" + "github.com/jcmturner/gokrb5/v8/iana/keyusage" + "github.com/jcmturner/gokrb5/v8/iana/msgtype" + "github.com/jcmturner/gokrb5/v8/iana/patype" + "github.com/jcmturner/gokrb5/v8/krberror" + "github.com/jcmturner/gokrb5/v8/types" +) + +type marshalKDCRep struct { + PVNO int `asn1:"explicit,tag:0"` + MsgType int `asn1:"explicit,tag:1"` + PAData types.PADataSequence `asn1:"explicit,optional,tag:2"` + CRealm string `asn1:"generalstring,explicit,tag:3"` + CName types.PrincipalName `asn1:"explicit,tag:4"` + // Ticket needs to be a raw value as it is wrapped in an APPLICATION tag + Ticket asn1.RawValue `asn1:"explicit,tag:5"` + EncPart types.EncryptedData `asn1:"explicit,tag:6"` +} + +// KDCRepFields represents the KRB_KDC_REP fields. +type KDCRepFields struct { + PVNO int + MsgType int + PAData []types.PAData + CRealm string + CName types.PrincipalName + Ticket Ticket + EncPart types.EncryptedData + DecryptedEncPart EncKDCRepPart +} + +// ASRep implements RFC 4120 KRB_AS_REP: https://tools.ietf.org/html/rfc4120#section-5.4.2. +type ASRep struct { + KDCRepFields +} + +// TGSRep implements RFC 4120 KRB_TGS_REP: https://tools.ietf.org/html/rfc4120#section-5.4.2. +type TGSRep struct { + KDCRepFields +} + +// EncKDCRepPart is the encrypted part of KRB_KDC_REP. +type EncKDCRepPart struct { + Key types.EncryptionKey `asn1:"explicit,tag:0"` + LastReqs []LastReq `asn1:"explicit,tag:1"` + Nonce int `asn1:"explicit,tag:2"` + KeyExpiration time.Time `asn1:"generalized,explicit,optional,tag:3"` + Flags asn1.BitString `asn1:"explicit,tag:4"` + AuthTime time.Time `asn1:"generalized,explicit,tag:5"` + StartTime time.Time `asn1:"generalized,explicit,optional,tag:6"` + EndTime time.Time `asn1:"generalized,explicit,tag:7"` + RenewTill time.Time `asn1:"generalized,explicit,optional,tag:8"` + SRealm string `asn1:"generalstring,explicit,tag:9"` + SName types.PrincipalName `asn1:"explicit,tag:10"` + CAddr []types.HostAddress `asn1:"explicit,optional,tag:11"` + EncPAData types.PADataSequence `asn1:"explicit,optional,tag:12"` +} + +// LastReq part of KRB_KDC_REP. +type LastReq struct { + LRType int32 `asn1:"explicit,tag:0"` + LRValue time.Time `asn1:"generalized,explicit,tag:1"` +} + +// Unmarshal bytes b into the ASRep struct. +func (k *ASRep) Unmarshal(b []byte) error { + var m marshalKDCRep + _, err := asn1.UnmarshalWithParams(b, &m, fmt.Sprintf("application,explicit,tag:%v", asnAppTag.ASREP)) + if err != nil { + return processUnmarshalReplyError(b, err) + } + if m.MsgType != msgtype.KRB_AS_REP { + return krberror.NewErrorf(krberror.KRBMsgError, "message ID does not indicate an AS_REP. Expected: %v; Actual: %v", msgtype.KRB_AS_REP, m.MsgType) + } + //Process the raw ticket within + tkt, err := unmarshalTicket(m.Ticket.Bytes) + if err != nil { + return krberror.Errorf(err, krberror.EncodingError, "error unmarshaling Ticket within AS_REP") + } + k.KDCRepFields = KDCRepFields{ + PVNO: m.PVNO, + MsgType: m.MsgType, + PAData: m.PAData, + CRealm: m.CRealm, + CName: m.CName, + Ticket: tkt, + EncPart: m.EncPart, + } + return nil +} + +// Marshal ASRep struct. +func (k *ASRep) Marshal() ([]byte, error) { + m := marshalKDCRep{ + PVNO: k.PVNO, + MsgType: k.MsgType, + PAData: k.PAData, + CRealm: k.CRealm, + CName: k.CName, + EncPart: k.EncPart, + } + b, err := k.Ticket.Marshal() + if err != nil { + return []byte{}, err + } + m.Ticket = asn1.RawValue{ + Class: asn1.ClassContextSpecific, + IsCompound: true, + Tag: 5, + Bytes: b, + } + mk, err := asn1.Marshal(m) + if err != nil { + return mk, krberror.Errorf(err, krberror.EncodingError, "error marshaling AS_REP") + } + mk = asn1tools.AddASNAppTag(mk, asnAppTag.ASREP) + return mk, nil +} + +// Unmarshal bytes b into the TGSRep struct. +func (k *TGSRep) Unmarshal(b []byte) error { + var m marshalKDCRep + _, err := asn1.UnmarshalWithParams(b, &m, fmt.Sprintf("application,explicit,tag:%v", asnAppTag.TGSREP)) + if err != nil { + return processUnmarshalReplyError(b, err) + } + if m.MsgType != msgtype.KRB_TGS_REP { + return krberror.NewErrorf(krberror.KRBMsgError, "message ID does not indicate an TGS_REP. Expected: %v; Actual: %v", msgtype.KRB_TGS_REP, m.MsgType) + } + //Process the raw ticket within + tkt, err := unmarshalTicket(m.Ticket.Bytes) + if err != nil { + return krberror.Errorf(err, krberror.EncodingError, "error unmarshaling Ticket within TGS_REP") + } + k.KDCRepFields = KDCRepFields{ + PVNO: m.PVNO, + MsgType: m.MsgType, + PAData: m.PAData, + CRealm: m.CRealm, + CName: m.CName, + Ticket: tkt, + EncPart: m.EncPart, + } + return nil +} + +// Marshal TGSRep struct. +func (k *TGSRep) Marshal() ([]byte, error) { + m := marshalKDCRep{ + PVNO: k.PVNO, + MsgType: k.MsgType, + PAData: k.PAData, + CRealm: k.CRealm, + CName: k.CName, + EncPart: k.EncPart, + } + b, err := k.Ticket.Marshal() + if err != nil { + return []byte{}, err + } + m.Ticket = asn1.RawValue{ + Class: asn1.ClassContextSpecific, + IsCompound: true, + Tag: 5, + Bytes: b, + } + mk, err := asn1.Marshal(m) + if err != nil { + return mk, krberror.Errorf(err, krberror.EncodingError, "error marshaling TGS_REP") + } + mk = asn1tools.AddASNAppTag(mk, asnAppTag.TGSREP) + return mk, nil +} + +// Unmarshal bytes b into encrypted part of KRB_KDC_REP. +func (e *EncKDCRepPart) Unmarshal(b []byte) error { + _, err := asn1.UnmarshalWithParams(b, e, fmt.Sprintf("application,explicit,tag:%v", asnAppTag.EncASRepPart)) + if err != nil { + // Try using tag 26 + // Ref: RFC 4120 - mentions that some implementations use application tag number 26 wether or not the reply is + // a AS-REP or a TGS-REP. + _, err = asn1.UnmarshalWithParams(b, e, fmt.Sprintf("application,explicit,tag:%v", asnAppTag.EncTGSRepPart)) + if err != nil { + return krberror.Errorf(err, krberror.EncodingError, "error unmarshaling encrypted part within KDC_REP") + } + } + return nil +} + +// Marshal encrypted part of KRB_KDC_REP. +func (e *EncKDCRepPart) Marshal() ([]byte, error) { + b, err := asn1.Marshal(*e) + if err != nil { + return b, krberror.Errorf(err, krberror.EncodingError, "marshaling error of AS_REP encpart") + } + b = asn1tools.AddASNAppTag(b, asnAppTag.EncASRepPart) + return b, nil +} + +// DecryptEncPart decrypts the encrypted part of an AS_REP. +func (k *ASRep) DecryptEncPart(c *credentials.Credentials) (types.EncryptionKey, error) { + var key types.EncryptionKey + var err error + if c.HasKeytab() { + key, _, err = c.Keytab().GetEncryptionKey(k.CName, k.CRealm, k.EncPart.KVNO, k.EncPart.EType) + if err != nil { + return key, krberror.Errorf(err, krberror.DecryptingError, "error decrypting AS_REP encrypted part") + } + } + if c.HasPassword() { + key, _, err = crypto.GetKeyFromPassword(c.Password(), k.CName, k.CRealm, k.EncPart.EType, k.PAData) + if err != nil { + return key, krberror.Errorf(err, krberror.DecryptingError, "error decrypting AS_REP encrypted part") + } + } + if !c.HasKeytab() && !c.HasPassword() { + return key, krberror.NewErrorf(krberror.DecryptingError, "no secret available in credentials to perform decryption of AS_REP encrypted part") + } + b, err := crypto.DecryptEncPart(k.EncPart, key, keyusage.AS_REP_ENCPART) + if err != nil { + return key, krberror.Errorf(err, krberror.DecryptingError, "error decrypting AS_REP encrypted part") + } + var denc EncKDCRepPart + err = denc.Unmarshal(b) + if err != nil { + return key, krberror.Errorf(err, krberror.EncodingError, "error unmarshaling decrypted encpart of AS_REP") + } + k.DecryptedEncPart = denc + return key, nil +} + +// Verify checks the validity of AS_REP message. +func (k *ASRep) Verify(cfg *config.Config, creds *credentials.Credentials, asReq ASReq) (bool, error) { + //Ref RFC 4120 Section 3.1.5 + if !k.CName.Equal(asReq.ReqBody.CName) { + return false, krberror.NewErrorf(krberror.KRBMsgError, "CName in response does not match what was requested. Requested: %+v; Reply: %+v", asReq.ReqBody.CName, k.CName) + } + if k.CRealm != asReq.ReqBody.Realm { + return false, krberror.NewErrorf(krberror.KRBMsgError, "CRealm in response does not match what was requested. Requested: %s; Reply: %s", asReq.ReqBody.Realm, k.CRealm) + } + key, err := k.DecryptEncPart(creds) + if err != nil { + return false, krberror.Errorf(err, krberror.DecryptingError, "error decrypting EncPart of AS_REP") + } + if k.DecryptedEncPart.Nonce != asReq.ReqBody.Nonce { + return false, krberror.NewErrorf(krberror.KRBMsgError, "possible replay attack, nonce in response does not match that in request") + } + if !k.DecryptedEncPart.SName.Equal(asReq.ReqBody.SName) { + return false, krberror.NewErrorf(krberror.KRBMsgError, "SName in response does not match what was requested. Requested: %v; Reply: %v", asReq.ReqBody.SName, k.DecryptedEncPart.SName) + } + if k.DecryptedEncPart.SRealm != asReq.ReqBody.Realm { + return false, krberror.NewErrorf(krberror.KRBMsgError, "SRealm in response does not match what was requested. Requested: %s; Reply: %s", asReq.ReqBody.Realm, k.DecryptedEncPart.SRealm) + } + if len(asReq.ReqBody.Addresses) > 0 { + if !types.HostAddressesEqual(k.DecryptedEncPart.CAddr, asReq.ReqBody.Addresses) { + return false, krberror.NewErrorf(krberror.KRBMsgError, "addresses listed in the AS_REP does not match those listed in the AS_REQ") + } + } + t := time.Now().UTC() + if t.Sub(k.DecryptedEncPart.AuthTime) > cfg.LibDefaults.Clockskew || k.DecryptedEncPart.AuthTime.Sub(t) > cfg.LibDefaults.Clockskew { + return false, krberror.NewErrorf(krberror.KRBMsgError, "clock skew with KDC too large. Greater than %v seconds", cfg.LibDefaults.Clockskew.Seconds()) + } + // RFC 6806 https://tools.ietf.org/html/rfc6806.html#section-11 + if asReq.PAData.Contains(patype.PA_REQ_ENC_PA_REP) && types.IsFlagSet(&k.DecryptedEncPart.Flags, flags.EncPARep) { + if len(k.DecryptedEncPart.EncPAData) < 2 || !k.DecryptedEncPart.EncPAData.Contains(patype.PA_FX_FAST) { + return false, krberror.NewErrorf(krberror.KRBMsgError, "KDC did not respond appropriately to FAST negotiation") + } + for _, pa := range k.DecryptedEncPart.EncPAData { + if pa.PADataType == patype.PA_REQ_ENC_PA_REP { + var pafast types.PAReqEncPARep + err := pafast.Unmarshal(pa.PADataValue) + if err != nil { + return false, krberror.Errorf(err, krberror.EncodingError, "KDC FAST negotiation response error, could not unmarshal PA_REQ_ENC_PA_REP") + } + etype, err := crypto.GetChksumEtype(pafast.ChksumType) + if err != nil { + return false, krberror.Errorf(err, krberror.ChksumError, "KDC FAST negotiation response error") + } + ab, _ := asReq.Marshal() + if !etype.VerifyChecksum(key.KeyValue, ab, pafast.Chksum, keyusage.KEY_USAGE_AS_REQ) { + return false, krberror.Errorf(err, krberror.ChksumError, "KDC FAST negotiation response checksum invalid") + } + } + } + } + return true, nil +} + +// DecryptEncPart decrypts the encrypted part of an TGS_REP. +func (k *TGSRep) DecryptEncPart(key types.EncryptionKey) error { + b, err := crypto.DecryptEncPart(k.EncPart, key, keyusage.TGS_REP_ENCPART_SESSION_KEY) + if err != nil { + return krberror.Errorf(err, krberror.DecryptingError, "error decrypting TGS_REP EncPart") + } + var denc EncKDCRepPart + err = denc.Unmarshal(b) + if err != nil { + return krberror.Errorf(err, krberror.EncodingError, "error unmarshaling encrypted part") + } + k.DecryptedEncPart = denc + return nil +} + +// Verify checks the validity of the TGS_REP message. +func (k *TGSRep) Verify(cfg *config.Config, tgsReq TGSReq) (bool, error) { + if !k.CName.Equal(tgsReq.ReqBody.CName) { + return false, krberror.NewErrorf(krberror.KRBMsgError, "CName in response does not match what was requested. Requested: %+v; Reply: %+v", tgsReq.ReqBody.CName, k.CName) + } + if k.Ticket.Realm != tgsReq.ReqBody.Realm { + return false, krberror.NewErrorf(krberror.KRBMsgError, "realm in response ticket does not match what was requested. Requested: %s; Reply: %s", tgsReq.ReqBody.Realm, k.Ticket.Realm) + } + if k.DecryptedEncPart.Nonce != tgsReq.ReqBody.Nonce { + return false, krberror.NewErrorf(krberror.KRBMsgError, "possible replay attack, nonce in response does not match that in request") + } + //if k.Ticket.SName.NameType != tgsReq.ReqBody.SName.NameType || k.Ticket.SName.NameString == nil { + // return false, krberror.NewErrorf(krberror.KRBMsgError, "SName in response ticket does not match what was requested. Requested: %v; Reply: %v", tgsReq.ReqBody.SName, k.Ticket.SName) + //} + //for i := range k.Ticket.SName.NameString { + // if k.Ticket.SName.NameString[i] != tgsReq.ReqBody.SName.NameString[i] { + // return false, krberror.NewErrorf(krberror.KRBMsgError, "SName in response ticket does not match what was requested. Requested: %+v; Reply: %+v", tgsReq.ReqBody.SName, k.Ticket.SName) + // } + //} + //if k.DecryptedEncPart.SName.NameType != tgsReq.ReqBody.SName.NameType || k.DecryptedEncPart.SName.NameString == nil { + // return false, krberror.NewErrorf(krberror.KRBMsgError, "SName in response does not match what was requested. Requested: %v; Reply: %v", tgsReq.ReqBody.SName, k.DecryptedEncPart.SName) + //} + //for i := range k.DecryptedEncPart.SName.NameString { + // if k.DecryptedEncPart.SName.NameString[i] != tgsReq.ReqBody.SName.NameString[i] { + // return false, krberror.NewErrorf(krberror.KRBMsgError, "SName in response does not match what was requested. Requested: %+v; Reply: %+v", tgsReq.ReqBody.SName, k.DecryptedEncPart.SName) + // } + //} + if k.DecryptedEncPart.SRealm != tgsReq.ReqBody.Realm { + return false, krberror.NewErrorf(krberror.KRBMsgError, "SRealm in response does not match what was requested. Requested: %s; Reply: %s", tgsReq.ReqBody.Realm, k.DecryptedEncPart.SRealm) + } + if len(k.DecryptedEncPart.CAddr) > 0 { + if !types.HostAddressesEqual(k.DecryptedEncPart.CAddr, tgsReq.ReqBody.Addresses) { + return false, krberror.NewErrorf(krberror.KRBMsgError, "addresses listed in the TGS_REP does not match those listed in the TGS_REQ") + } + } + if time.Since(k.DecryptedEncPart.StartTime) > cfg.LibDefaults.Clockskew || k.DecryptedEncPart.StartTime.Sub(time.Now().UTC()) > cfg.LibDefaults.Clockskew { + if time.Since(k.DecryptedEncPart.AuthTime) > cfg.LibDefaults.Clockskew || k.DecryptedEncPart.AuthTime.Sub(time.Now().UTC()) > cfg.LibDefaults.Clockskew { + return false, krberror.NewErrorf(krberror.KRBMsgError, "clock skew with KDC too large. Greater than %v seconds.", cfg.LibDefaults.Clockskew.Seconds()) + } + } + return true, nil +} diff --git a/vendor/github.com/jcmturner/gokrb5/v8/messages/KDCReq.go b/vendor/github.com/jcmturner/gokrb5/v8/messages/KDCReq.go new file mode 100644 index 000000000000..3745afed2b9e --- /dev/null +++ b/vendor/github.com/jcmturner/gokrb5/v8/messages/KDCReq.go @@ -0,0 +1,432 @@ +package messages + +// Reference: https://www.ietf.org/rfc/rfc4120.txt +// Section: 5.4.1 + +import ( + "crypto/rand" + "fmt" + "math" + "math/big" + "time" + + "github.com/jcmturner/gofork/encoding/asn1" + "github.com/jcmturner/gokrb5/v8/asn1tools" + "github.com/jcmturner/gokrb5/v8/config" + "github.com/jcmturner/gokrb5/v8/crypto" + "github.com/jcmturner/gokrb5/v8/iana" + "github.com/jcmturner/gokrb5/v8/iana/asnAppTag" + "github.com/jcmturner/gokrb5/v8/iana/flags" + "github.com/jcmturner/gokrb5/v8/iana/keyusage" + "github.com/jcmturner/gokrb5/v8/iana/msgtype" + "github.com/jcmturner/gokrb5/v8/iana/nametype" + "github.com/jcmturner/gokrb5/v8/iana/patype" + "github.com/jcmturner/gokrb5/v8/krberror" + "github.com/jcmturner/gokrb5/v8/types" +) + +type marshalKDCReq struct { + PVNO int `asn1:"explicit,tag:1"` + MsgType int `asn1:"explicit,tag:2"` + PAData types.PADataSequence `asn1:"explicit,optional,tag:3"` + ReqBody asn1.RawValue `asn1:"explicit,tag:4"` +} + +// KDCReqFields represents the KRB_KDC_REQ fields. +type KDCReqFields struct { + PVNO int + MsgType int + PAData types.PADataSequence + ReqBody KDCReqBody + Renewal bool +} + +// ASReq implements RFC 4120 KRB_AS_REQ: https://tools.ietf.org/html/rfc4120#section-5.4.1. +type ASReq struct { + KDCReqFields +} + +// TGSReq implements RFC 4120 KRB_TGS_REQ: https://tools.ietf.org/html/rfc4120#section-5.4.1. +type TGSReq struct { + KDCReqFields +} + +type marshalKDCReqBody struct { + KDCOptions asn1.BitString `asn1:"explicit,tag:0"` + CName types.PrincipalName `asn1:"explicit,optional,tag:1"` + Realm string `asn1:"generalstring,explicit,tag:2"` + SName types.PrincipalName `asn1:"explicit,optional,tag:3"` + From time.Time `asn1:"generalized,explicit,optional,tag:4"` + Till time.Time `asn1:"generalized,explicit,tag:5"` + RTime time.Time `asn1:"generalized,explicit,optional,tag:6"` + Nonce int `asn1:"explicit,tag:7"` + EType []int32 `asn1:"explicit,tag:8"` + Addresses []types.HostAddress `asn1:"explicit,optional,tag:9"` + EncAuthData types.EncryptedData `asn1:"explicit,optional,tag:10"` + // Ticket needs to be a raw value as it is wrapped in an APPLICATION tag + AdditionalTickets asn1.RawValue `asn1:"explicit,optional,tag:11"` +} + +// KDCReqBody implements the KRB_KDC_REQ request body. +type KDCReqBody struct { + KDCOptions asn1.BitString `asn1:"explicit,tag:0"` + CName types.PrincipalName `asn1:"explicit,optional,tag:1"` + Realm string `asn1:"generalstring,explicit,tag:2"` + SName types.PrincipalName `asn1:"explicit,optional,tag:3"` + From time.Time `asn1:"generalized,explicit,optional,tag:4"` + Till time.Time `asn1:"generalized,explicit,tag:5"` + RTime time.Time `asn1:"generalized,explicit,optional,tag:6"` + Nonce int `asn1:"explicit,tag:7"` + EType []int32 `asn1:"explicit,tag:8"` + Addresses []types.HostAddress `asn1:"explicit,optional,tag:9"` + EncAuthData types.EncryptedData `asn1:"explicit,optional,tag:10"` + AdditionalTickets []Ticket `asn1:"explicit,optional,tag:11"` +} + +// NewASReqForTGT generates a new KRB_AS_REQ struct for a TGT request. +func NewASReqForTGT(realm string, c *config.Config, cname types.PrincipalName) (ASReq, error) { + sname := types.PrincipalName{ + NameType: nametype.KRB_NT_SRV_INST, + NameString: []string{"krbtgt", realm}, + } + return NewASReq(realm, c, cname, sname) +} + +// NewASReqForChgPasswd generates a new KRB_AS_REQ struct for a change password request. +func NewASReqForChgPasswd(realm string, c *config.Config, cname types.PrincipalName) (ASReq, error) { + sname := types.PrincipalName{ + NameType: nametype.KRB_NT_PRINCIPAL, + NameString: []string{"kadmin", "changepw"}, + } + return NewASReq(realm, c, cname, sname) +} + +// NewASReq generates a new KRB_AS_REQ struct for a given SNAME. +func NewASReq(realm string, c *config.Config, cname, sname types.PrincipalName) (ASReq, error) { + nonce, err := rand.Int(rand.Reader, big.NewInt(math.MaxInt32)) + if err != nil { + return ASReq{}, err + } + t := time.Now().UTC() + // Copy the default options to make this thread safe + kopts := types.NewKrbFlags() + copy(kopts.Bytes, c.LibDefaults.KDCDefaultOptions.Bytes) + kopts.BitLength = c.LibDefaults.KDCDefaultOptions.BitLength + a := ASReq{ + KDCReqFields{ + PVNO: iana.PVNO, + MsgType: msgtype.KRB_AS_REQ, + PAData: types.PADataSequence{}, + ReqBody: KDCReqBody{ + KDCOptions: kopts, + Realm: realm, + CName: cname, + SName: sname, + Till: t.Add(c.LibDefaults.TicketLifetime), + Nonce: int(nonce.Int64()), + EType: c.LibDefaults.DefaultTktEnctypeIDs, + }, + }, + } + if c.LibDefaults.Forwardable { + types.SetFlag(&a.ReqBody.KDCOptions, flags.Forwardable) + } + if c.LibDefaults.Canonicalize { + types.SetFlag(&a.ReqBody.KDCOptions, flags.Canonicalize) + } + if c.LibDefaults.Proxiable { + types.SetFlag(&a.ReqBody.KDCOptions, flags.Proxiable) + } + if c.LibDefaults.RenewLifetime != 0 { + types.SetFlag(&a.ReqBody.KDCOptions, flags.Renewable) + a.ReqBody.RTime = t.Add(c.LibDefaults.RenewLifetime) + a.ReqBody.RTime = t.Add(time.Duration(48) * time.Hour) + } + if !c.LibDefaults.NoAddresses { + ha, err := types.LocalHostAddresses() + if err != nil { + return a, fmt.Errorf("could not get local addresses: %v", err) + } + ha = append(ha, types.HostAddressesFromNetIPs(c.LibDefaults.ExtraAddresses)...) + a.ReqBody.Addresses = ha + } + return a, nil +} + +// NewTGSReq generates a new KRB_TGS_REQ struct. +func NewTGSReq(cname types.PrincipalName, kdcRealm string, c *config.Config, tgt Ticket, sessionKey types.EncryptionKey, sname types.PrincipalName, renewal bool) (TGSReq, error) { + a, err := tgsReq(cname, sname, kdcRealm, renewal, c) + if err != nil { + return a, err + } + err = a.setPAData(tgt, sessionKey) + return a, err +} + +// NewUser2UserTGSReq returns a TGS-REQ suitable for user-to-user authentication (https://tools.ietf.org/html/rfc4120#section-3.7) +func NewUser2UserTGSReq(cname types.PrincipalName, kdcRealm string, c *config.Config, clientTGT Ticket, sessionKey types.EncryptionKey, sname types.PrincipalName, renewal bool, verifyingTGT Ticket) (TGSReq, error) { + a, err := tgsReq(cname, sname, kdcRealm, renewal, c) + if err != nil { + return a, err + } + a.ReqBody.AdditionalTickets = []Ticket{verifyingTGT} + types.SetFlag(&a.ReqBody.KDCOptions, flags.EncTktInSkey) + err = a.setPAData(clientTGT, sessionKey) + return a, err +} + +// tgsReq populates the fields for a TGS_REQ +func tgsReq(cname, sname types.PrincipalName, kdcRealm string, renewal bool, c *config.Config) (TGSReq, error) { + nonce, err := rand.Int(rand.Reader, big.NewInt(math.MaxInt32)) + if err != nil { + return TGSReq{}, err + } + t := time.Now().UTC() + k := KDCReqFields{ + PVNO: iana.PVNO, + MsgType: msgtype.KRB_TGS_REQ, + ReqBody: KDCReqBody{ + KDCOptions: types.NewKrbFlags(), + Realm: kdcRealm, + CName: cname, // Add the CName to make validation of the reply easier + SName: sname, + Till: t.Add(c.LibDefaults.TicketLifetime), + Nonce: int(nonce.Int64()), + EType: c.LibDefaults.DefaultTGSEnctypeIDs, + }, + Renewal: renewal, + } + if c.LibDefaults.Forwardable { + types.SetFlag(&k.ReqBody.KDCOptions, flags.Forwardable) + } + if c.LibDefaults.Canonicalize { + types.SetFlag(&k.ReqBody.KDCOptions, flags.Canonicalize) + } + if c.LibDefaults.Proxiable { + types.SetFlag(&k.ReqBody.KDCOptions, flags.Proxiable) + } + if c.LibDefaults.RenewLifetime > time.Duration(0) { + types.SetFlag(&k.ReqBody.KDCOptions, flags.Renewable) + k.ReqBody.RTime = t.Add(c.LibDefaults.RenewLifetime) + } + if !c.LibDefaults.NoAddresses { + ha, err := types.LocalHostAddresses() + if err != nil { + return TGSReq{}, fmt.Errorf("could not get local addresses: %v", err) + } + ha = append(ha, types.HostAddressesFromNetIPs(c.LibDefaults.ExtraAddresses)...) + k.ReqBody.Addresses = ha + } + if renewal { + types.SetFlag(&k.ReqBody.KDCOptions, flags.Renew) + types.SetFlag(&k.ReqBody.KDCOptions, flags.Renewable) + } + return TGSReq{ + k, + }, nil +} + +func (k *TGSReq) setPAData(tgt Ticket, sessionKey types.EncryptionKey) error { + // Marshal the request and calculate checksum + b, err := k.ReqBody.Marshal() + if err != nil { + return krberror.Errorf(err, krberror.EncodingError, "error marshaling TGS_REQ body") + } + etype, err := crypto.GetEtype(sessionKey.KeyType) + if err != nil { + return krberror.Errorf(err, krberror.EncryptingError, "error getting etype to encrypt authenticator") + } + cb, err := etype.GetChecksumHash(sessionKey.KeyValue, b, keyusage.TGS_REQ_PA_TGS_REQ_AP_REQ_AUTHENTICATOR_CHKSUM) + if err != nil { + return krberror.Errorf(err, krberror.ChksumError, "error getting etype checksum hash") + } + + // Form PAData for TGS_REQ + // Create authenticator + auth, err := types.NewAuthenticator(tgt.Realm, k.ReqBody.CName) + if err != nil { + return krberror.Errorf(err, krberror.KRBMsgError, "error generating new authenticator") + } + auth.Cksum = types.Checksum{ + CksumType: etype.GetHashID(), + Checksum: cb, + } + // Create AP_REQ + apReq, err := NewAPReq(tgt, sessionKey, auth) + if err != nil { + return krberror.Errorf(err, krberror.KRBMsgError, "error generating new AP_REQ") + } + apb, err := apReq.Marshal() + if err != nil { + return krberror.Errorf(err, krberror.EncodingError, "error marshaling AP_REQ for pre-authentication data") + } + k.PAData = types.PADataSequence{ + types.PAData{ + PADataType: patype.PA_TGS_REQ, + PADataValue: apb, + }, + } + return nil +} + +// Unmarshal bytes b into the ASReq struct. +func (k *ASReq) Unmarshal(b []byte) error { + var m marshalKDCReq + _, err := asn1.UnmarshalWithParams(b, &m, fmt.Sprintf("application,explicit,tag:%v", asnAppTag.ASREQ)) + if err != nil { + return krberror.Errorf(err, krberror.EncodingError, "error unmarshaling AS_REQ") + } + expectedMsgType := msgtype.KRB_AS_REQ + if m.MsgType != expectedMsgType { + return krberror.NewErrorf(krberror.KRBMsgError, "message ID does not indicate a AS_REQ. Expected: %v; Actual: %v", expectedMsgType, m.MsgType) + } + var reqb KDCReqBody + err = reqb.Unmarshal(m.ReqBody.Bytes) + if err != nil { + return krberror.Errorf(err, krberror.EncodingError, "error processing AS_REQ body") + } + k.MsgType = m.MsgType + k.PAData = m.PAData + k.PVNO = m.PVNO + k.ReqBody = reqb + return nil +} + +// Unmarshal bytes b into the TGSReq struct. +func (k *TGSReq) Unmarshal(b []byte) error { + var m marshalKDCReq + _, err := asn1.UnmarshalWithParams(b, &m, fmt.Sprintf("application,explicit,tag:%v", asnAppTag.TGSREQ)) + if err != nil { + return krberror.Errorf(err, krberror.EncodingError, "error unmarshaling TGS_REQ") + } + expectedMsgType := msgtype.KRB_TGS_REQ + if m.MsgType != expectedMsgType { + return krberror.NewErrorf(krberror.KRBMsgError, "message ID does not indicate a TGS_REQ. Expected: %v; Actual: %v", expectedMsgType, m.MsgType) + } + var reqb KDCReqBody + err = reqb.Unmarshal(m.ReqBody.Bytes) + if err != nil { + return krberror.Errorf(err, krberror.EncodingError, "error processing TGS_REQ body") + } + k.MsgType = m.MsgType + k.PAData = m.PAData + k.PVNO = m.PVNO + k.ReqBody = reqb + return nil +} + +// Unmarshal bytes b into the KRB_KDC_REQ body struct. +func (k *KDCReqBody) Unmarshal(b []byte) error { + var m marshalKDCReqBody + _, err := asn1.Unmarshal(b, &m) + if err != nil { + return krberror.Errorf(err, krberror.EncodingError, "error unmarshaling KDC_REQ body") + } + k.KDCOptions = m.KDCOptions + if len(k.KDCOptions.Bytes) < 4 { + tb := make([]byte, 4-len(k.KDCOptions.Bytes)) + k.KDCOptions.Bytes = append(tb, k.KDCOptions.Bytes...) + k.KDCOptions.BitLength = len(k.KDCOptions.Bytes) * 8 + } + k.CName = m.CName + k.Realm = m.Realm + k.SName = m.SName + k.From = m.From + k.Till = m.Till + k.RTime = m.RTime + k.Nonce = m.Nonce + k.EType = m.EType + k.Addresses = m.Addresses + k.EncAuthData = m.EncAuthData + if len(m.AdditionalTickets.Bytes) > 0 { + k.AdditionalTickets, err = unmarshalTicketsSequence(m.AdditionalTickets) + if err != nil { + return krberror.Errorf(err, krberror.EncodingError, "error unmarshaling additional tickets") + } + } + return nil +} + +// Marshal ASReq struct. +func (k *ASReq) Marshal() ([]byte, error) { + m := marshalKDCReq{ + PVNO: k.PVNO, + MsgType: k.MsgType, + PAData: k.PAData, + } + b, err := k.ReqBody.Marshal() + if err != nil { + var mk []byte + return mk, err + } + m.ReqBody = asn1.RawValue{ + Class: asn1.ClassContextSpecific, + IsCompound: true, + Tag: 4, + Bytes: b, + } + mk, err := asn1.Marshal(m) + if err != nil { + return mk, krberror.Errorf(err, krberror.EncodingError, "error marshaling AS_REQ") + } + mk = asn1tools.AddASNAppTag(mk, asnAppTag.ASREQ) + return mk, nil +} + +// Marshal TGSReq struct. +func (k *TGSReq) Marshal() ([]byte, error) { + m := marshalKDCReq{ + PVNO: k.PVNO, + MsgType: k.MsgType, + PAData: k.PAData, + } + b, err := k.ReqBody.Marshal() + if err != nil { + var mk []byte + return mk, err + } + m.ReqBody = asn1.RawValue{ + Class: asn1.ClassContextSpecific, + IsCompound: true, + Tag: 4, + Bytes: b, + } + mk, err := asn1.Marshal(m) + if err != nil { + return mk, krberror.Errorf(err, krberror.EncodingError, "error marshaling AS_REQ") + } + mk = asn1tools.AddASNAppTag(mk, asnAppTag.TGSREQ) + return mk, nil +} + +// Marshal KRB_KDC_REQ body struct. +func (k *KDCReqBody) Marshal() ([]byte, error) { + var b []byte + m := marshalKDCReqBody{ + KDCOptions: k.KDCOptions, + CName: k.CName, + Realm: k.Realm, + SName: k.SName, + From: k.From, + Till: k.Till, + RTime: k.RTime, + Nonce: k.Nonce, + EType: k.EType, + Addresses: k.Addresses, + EncAuthData: k.EncAuthData, + } + rawtkts, err := MarshalTicketSequence(k.AdditionalTickets) + if err != nil { + return b, krberror.Errorf(err, krberror.EncodingError, "error in marshaling KDC request body additional tickets") + } + //The asn1.rawValue needs the tag setting on it for where it is in the KDCReqBody + rawtkts.Tag = 11 + if len(rawtkts.Bytes) > 0 { + m.AdditionalTickets = rawtkts + } + b, err = asn1.Marshal(m) + if err != nil { + return b, krberror.Errorf(err, krberror.EncodingError, "error in marshaling KDC request body") + } + return b, nil +} diff --git a/vendor/github.com/jcmturner/gokrb5/v8/messages/KRBCred.go b/vendor/github.com/jcmturner/gokrb5/v8/messages/KRBCred.go new file mode 100644 index 000000000000..536fdb9ec939 --- /dev/null +++ b/vendor/github.com/jcmturner/gokrb5/v8/messages/KRBCred.go @@ -0,0 +1,102 @@ +package messages + +import ( + "fmt" + "time" + + "github.com/jcmturner/gofork/encoding/asn1" + "github.com/jcmturner/gokrb5/v8/crypto" + "github.com/jcmturner/gokrb5/v8/iana/asnAppTag" + "github.com/jcmturner/gokrb5/v8/iana/keyusage" + "github.com/jcmturner/gokrb5/v8/iana/msgtype" + "github.com/jcmturner/gokrb5/v8/krberror" + "github.com/jcmturner/gokrb5/v8/types" +) + +type marshalKRBCred struct { + PVNO int `asn1:"explicit,tag:0"` + MsgType int `asn1:"explicit,tag:1"` + Tickets asn1.RawValue `asn1:"explicit,tag:2"` + EncPart types.EncryptedData `asn1:"explicit,tag:3"` +} + +// KRBCred implements RFC 4120 KRB_CRED: https://tools.ietf.org/html/rfc4120#section-5.8.1. +type KRBCred struct { + PVNO int + MsgType int + Tickets []Ticket + EncPart types.EncryptedData + DecryptedEncPart EncKrbCredPart +} + +// EncKrbCredPart is the encrypted part of KRB_CRED. +type EncKrbCredPart struct { + TicketInfo []KrbCredInfo `asn1:"explicit,tag:0"` + Nouce int `asn1:"optional,explicit,tag:1"` + Timestamp time.Time `asn1:"generalized,optional,explicit,tag:2"` + Usec int `asn1:"optional,explicit,tag:3"` + SAddress types.HostAddress `asn1:"optional,explicit,tag:4"` + RAddress types.HostAddress `asn1:"optional,explicit,tag:5"` +} + +// KrbCredInfo is the KRB_CRED_INFO part of KRB_CRED. +type KrbCredInfo struct { + Key types.EncryptionKey `asn1:"explicit,tag:0"` + PRealm string `asn1:"generalstring,optional,explicit,tag:1"` + PName types.PrincipalName `asn1:"optional,explicit,tag:2"` + Flags asn1.BitString `asn1:"optional,explicit,tag:3"` + AuthTime time.Time `asn1:"generalized,optional,explicit,tag:4"` + StartTime time.Time `asn1:"generalized,optional,explicit,tag:5"` + EndTime time.Time `asn1:"generalized,optional,explicit,tag:6"` + RenewTill time.Time `asn1:"generalized,optional,explicit,tag:7"` + SRealm string `asn1:"optional,explicit,ia5,tag:8"` + SName types.PrincipalName `asn1:"optional,explicit,tag:9"` + CAddr types.HostAddresses `asn1:"optional,explicit,tag:10"` +} + +// Unmarshal bytes b into the KRBCred struct. +func (k *KRBCred) Unmarshal(b []byte) error { + var m marshalKRBCred + _, err := asn1.UnmarshalWithParams(b, &m, fmt.Sprintf("application,explicit,tag:%v", asnAppTag.KRBCred)) + if err != nil { + return processUnmarshalReplyError(b, err) + } + expectedMsgType := msgtype.KRB_CRED + if m.MsgType != expectedMsgType { + return krberror.NewErrorf(krberror.KRBMsgError, "message ID does not indicate a KRB_CRED. Expected: %v; Actual: %v", expectedMsgType, m.MsgType) + } + k.PVNO = m.PVNO + k.MsgType = m.MsgType + k.EncPart = m.EncPart + if len(m.Tickets.Bytes) > 0 { + k.Tickets, err = unmarshalTicketsSequence(m.Tickets) + if err != nil { + return krberror.Errorf(err, krberror.EncodingError, "error unmarshaling tickets within KRB_CRED") + } + } + return nil +} + +// DecryptEncPart decrypts the encrypted part of a KRB_CRED. +func (k *KRBCred) DecryptEncPart(key types.EncryptionKey) error { + b, err := crypto.DecryptEncPart(k.EncPart, key, keyusage.KRB_CRED_ENCPART) + if err != nil { + return krberror.Errorf(err, krberror.DecryptingError, "error decrypting KRB_CRED EncPart") + } + var denc EncKrbCredPart + err = denc.Unmarshal(b) + if err != nil { + return krberror.Errorf(err, krberror.EncodingError, "error unmarshaling encrypted part of KRB_CRED") + } + k.DecryptedEncPart = denc + return nil +} + +// Unmarshal bytes b into the encrypted part of KRB_CRED. +func (k *EncKrbCredPart) Unmarshal(b []byte) error { + _, err := asn1.UnmarshalWithParams(b, k, fmt.Sprintf("application,explicit,tag:%v", asnAppTag.EncKrbCredPart)) + if err != nil { + return krberror.Errorf(err, krberror.EncodingError, "error unmarshaling EncKrbCredPart") + } + return nil +} diff --git a/vendor/github.com/jcmturner/gokrb5/v8/messages/KRBError.go b/vendor/github.com/jcmturner/gokrb5/v8/messages/KRBError.go new file mode 100644 index 000000000000..d2cf32d65819 --- /dev/null +++ b/vendor/github.com/jcmturner/gokrb5/v8/messages/KRBError.go @@ -0,0 +1,94 @@ +// Package messages implements Kerberos 5 message types and methods. +package messages + +import ( + "fmt" + "time" + + "github.com/jcmturner/gofork/encoding/asn1" + "github.com/jcmturner/gokrb5/v8/asn1tools" + "github.com/jcmturner/gokrb5/v8/iana" + "github.com/jcmturner/gokrb5/v8/iana/asnAppTag" + "github.com/jcmturner/gokrb5/v8/iana/errorcode" + "github.com/jcmturner/gokrb5/v8/iana/msgtype" + "github.com/jcmturner/gokrb5/v8/krberror" + "github.com/jcmturner/gokrb5/v8/types" +) + +// KRBError implements RFC 4120 KRB_ERROR: https://tools.ietf.org/html/rfc4120#section-5.9.1. +type KRBError struct { + PVNO int `asn1:"explicit,tag:0"` + MsgType int `asn1:"explicit,tag:1"` + CTime time.Time `asn1:"generalized,optional,explicit,tag:2"` + Cusec int `asn1:"optional,explicit,tag:3"` + STime time.Time `asn1:"generalized,explicit,tag:4"` + Susec int `asn1:"explicit,tag:5"` + ErrorCode int32 `asn1:"explicit,tag:6"` + CRealm string `asn1:"generalstring,optional,explicit,tag:7"` + CName types.PrincipalName `asn1:"optional,explicit,tag:8"` + Realm string `asn1:"generalstring,explicit,tag:9"` + SName types.PrincipalName `asn1:"explicit,tag:10"` + EText string `asn1:"generalstring,optional,explicit,tag:11"` + EData []byte `asn1:"optional,explicit,tag:12"` +} + +// NewKRBError creates a new KRBError. +func NewKRBError(sname types.PrincipalName, realm string, code int32, etext string) KRBError { + t := time.Now().UTC() + return KRBError{ + PVNO: iana.PVNO, + MsgType: msgtype.KRB_ERROR, + STime: t, + Susec: int((t.UnixNano() / int64(time.Microsecond)) - (t.Unix() * 1e6)), + ErrorCode: code, + SName: sname, + Realm: realm, + EText: etext, + } +} + +// Unmarshal bytes b into the KRBError struct. +func (k *KRBError) Unmarshal(b []byte) error { + _, err := asn1.UnmarshalWithParams(b, k, fmt.Sprintf("application,explicit,tag:%v", asnAppTag.KRBError)) + if err != nil { + return krberror.Errorf(err, krberror.EncodingError, "KRB_ERROR unmarshal error") + } + expectedMsgType := msgtype.KRB_ERROR + if k.MsgType != expectedMsgType { + return krberror.NewErrorf(krberror.KRBMsgError, "message ID does not indicate a KRB_ERROR. Expected: %v; Actual: %v", expectedMsgType, k.MsgType) + } + return nil +} + +// Marshal a KRBError into bytes. +func (k *KRBError) Marshal() ([]byte, error) { + b, err := asn1.Marshal(*k) + if err != nil { + return b, krberror.Errorf(err, krberror.EncodingError, "error marshaling KRBError") + } + b = asn1tools.AddASNAppTag(b, asnAppTag.KRBError) + return b, nil +} + +// Error method implementing error interface on KRBError struct. +func (k KRBError) Error() string { + etxt := fmt.Sprintf("KRB Error: %s", errorcode.Lookup(k.ErrorCode)) + if k.EText != "" { + etxt = fmt.Sprintf("%s - %s", etxt, k.EText) + } + return etxt +} + +func processUnmarshalReplyError(b []byte, err error) error { + switch err.(type) { + case asn1.StructuralError: + var krberr KRBError + tmperr := krberr.Unmarshal(b) + if tmperr != nil { + return krberror.Errorf(err, krberror.EncodingError, "failed to unmarshal KDC's reply") + } + return krberr + default: + return krberror.Errorf(err, krberror.EncodingError, "failed to unmarshal KDC's reply") + } +} diff --git a/vendor/github.com/jcmturner/gokrb5/v8/messages/KRBPriv.go b/vendor/github.com/jcmturner/gokrb5/v8/messages/KRBPriv.go new file mode 100644 index 000000000000..0ca614949fd7 --- /dev/null +++ b/vendor/github.com/jcmturner/gokrb5/v8/messages/KRBPriv.go @@ -0,0 +1,108 @@ +package messages + +import ( + "fmt" + "time" + + "github.com/jcmturner/gofork/encoding/asn1" + "github.com/jcmturner/gokrb5/v8/asn1tools" + "github.com/jcmturner/gokrb5/v8/crypto" + "github.com/jcmturner/gokrb5/v8/iana" + "github.com/jcmturner/gokrb5/v8/iana/asnAppTag" + "github.com/jcmturner/gokrb5/v8/iana/keyusage" + "github.com/jcmturner/gokrb5/v8/iana/msgtype" + "github.com/jcmturner/gokrb5/v8/krberror" + "github.com/jcmturner/gokrb5/v8/types" +) + +// KRBPriv implements RFC 4120 type: https://tools.ietf.org/html/rfc4120#section-5.7.1. +type KRBPriv struct { + PVNO int `asn1:"explicit,tag:0"` + MsgType int `asn1:"explicit,tag:1"` + EncPart types.EncryptedData `asn1:"explicit,tag:3"` + DecryptedEncPart EncKrbPrivPart `asn1:"optional,omitempty"` // Not part of ASN1 bytes so marked as optional so unmarshalling works +} + +// EncKrbPrivPart is the encrypted part of KRB_PRIV. +type EncKrbPrivPart struct { + UserData []byte `asn1:"explicit,tag:0"` + Timestamp time.Time `asn1:"generalized,optional,explicit,tag:1"` + Usec int `asn1:"optional,explicit,tag:2"` + SequenceNumber int64 `asn1:"optional,explicit,tag:3"` + SAddress types.HostAddress `asn1:"explicit,tag:4"` + RAddress types.HostAddress `asn1:"optional,explicit,tag:5"` +} + +// NewKRBPriv returns a new KRBPriv type. +func NewKRBPriv(part EncKrbPrivPart) KRBPriv { + return KRBPriv{ + PVNO: iana.PVNO, + MsgType: msgtype.KRB_PRIV, + DecryptedEncPart: part, + } +} + +// Unmarshal bytes b into the KRBPriv struct. +func (k *KRBPriv) Unmarshal(b []byte) error { + _, err := asn1.UnmarshalWithParams(b, k, fmt.Sprintf("application,explicit,tag:%v", asnAppTag.KRBPriv)) + if err != nil { + return processUnmarshalReplyError(b, err) + } + expectedMsgType := msgtype.KRB_PRIV + if k.MsgType != expectedMsgType { + return krberror.NewErrorf(krberror.KRBMsgError, "message ID does not indicate a KRB_PRIV. Expected: %v; Actual: %v", expectedMsgType, k.MsgType) + } + return nil +} + +// Unmarshal bytes b into the EncKrbPrivPart struct. +func (k *EncKrbPrivPart) Unmarshal(b []byte) error { + _, err := asn1.UnmarshalWithParams(b, k, fmt.Sprintf("application,explicit,tag:%v", asnAppTag.EncKrbPrivPart)) + if err != nil { + return krberror.Errorf(err, krberror.EncodingError, "KRB_PRIV unmarshal error") + } + return nil +} + +// Marshal the KRBPriv. +func (k *KRBPriv) Marshal() ([]byte, error) { + tk := KRBPriv{ + PVNO: k.PVNO, + MsgType: k.MsgType, + EncPart: k.EncPart, + } + b, err := asn1.Marshal(tk) + if err != nil { + return []byte{}, err + } + b = asn1tools.AddASNAppTag(b, asnAppTag.KRBPriv) + return b, nil +} + +// EncryptEncPart encrypts the DecryptedEncPart within the KRBPriv. +// Use to prepare for marshaling. +func (k *KRBPriv) EncryptEncPart(key types.EncryptionKey) error { + b, err := asn1.Marshal(k.DecryptedEncPart) + if err != nil { + return err + } + b = asn1tools.AddASNAppTag(b, asnAppTag.EncKrbPrivPart) + k.EncPart, err = crypto.GetEncryptedData(b, key, keyusage.KRB_PRIV_ENCPART, 1) + if err != nil { + return err + } + return nil +} + +// DecryptEncPart decrypts the encrypted part of the KRBPriv message. +func (k *KRBPriv) DecryptEncPart(key types.EncryptionKey) error { + b, err := crypto.DecryptEncPart(k.EncPart, key, keyusage.KRB_PRIV_ENCPART) + if err != nil { + return fmt.Errorf("error decrypting KRBPriv EncPart: %v", err) + } + err = k.DecryptedEncPart.Unmarshal(b) + if err != nil { + return fmt.Errorf("error unmarshaling encrypted part: %v", err) + } + return nil +} diff --git a/vendor/github.com/jcmturner/gokrb5/v8/messages/KRBSafe.go b/vendor/github.com/jcmturner/gokrb5/v8/messages/KRBSafe.go new file mode 100644 index 000000000000..52cd28449360 --- /dev/null +++ b/vendor/github.com/jcmturner/gokrb5/v8/messages/KRBSafe.go @@ -0,0 +1,43 @@ +package messages + +import ( + "fmt" + "time" + + "github.com/jcmturner/gofork/encoding/asn1" + "github.com/jcmturner/gokrb5/v8/iana/asnAppTag" + "github.com/jcmturner/gokrb5/v8/iana/msgtype" + "github.com/jcmturner/gokrb5/v8/krberror" + "github.com/jcmturner/gokrb5/v8/types" +) + +// KRBSafe implements RFC 4120 KRB_SAFE: https://tools.ietf.org/html/rfc4120#section-5.6.1. +type KRBSafe struct { + PVNO int `asn1:"explicit,tag:0"` + MsgType int `asn1:"explicit,tag:1"` + SafeBody KRBSafeBody `asn1:"explicit,tag:2"` + Cksum types.Checksum `asn1:"explicit,tag:3"` +} + +// KRBSafeBody implements the KRB_SAFE_BODY of KRB_SAFE. +type KRBSafeBody struct { + UserData []byte `asn1:"explicit,tag:0"` + Timestamp time.Time `asn1:"generalized,optional,explicit,tag:1"` + Usec int `asn1:"optional,explicit,tag:2"` + SequenceNumber int64 `asn1:"optional,explicit,tag:3"` + SAddress types.HostAddress `asn1:"explicit,tag:4"` + RAddress types.HostAddress `asn1:"optional,explicit,tag:5"` +} + +// Unmarshal bytes b into the KRBSafe struct. +func (s *KRBSafe) Unmarshal(b []byte) error { + _, err := asn1.UnmarshalWithParams(b, s, fmt.Sprintf("application,explicit,tag:%v", asnAppTag.KRBSafe)) + if err != nil { + return processUnmarshalReplyError(b, err) + } + expectedMsgType := msgtype.KRB_SAFE + if s.MsgType != expectedMsgType { + return krberror.NewErrorf(krberror.KRBMsgError, "message ID does not indicate a KRB_SAFE. Expected: %v; Actual: %v", expectedMsgType, s.MsgType) + } + return nil +} diff --git a/vendor/github.com/jcmturner/gokrb5/v8/messages/Ticket.go b/vendor/github.com/jcmturner/gokrb5/v8/messages/Ticket.go new file mode 100644 index 000000000000..11efad62c2c6 --- /dev/null +++ b/vendor/github.com/jcmturner/gokrb5/v8/messages/Ticket.go @@ -0,0 +1,262 @@ +package messages + +import ( + "fmt" + "log" + "time" + + "github.com/jcmturner/gofork/encoding/asn1" + "github.com/jcmturner/gokrb5/v8/asn1tools" + "github.com/jcmturner/gokrb5/v8/crypto" + "github.com/jcmturner/gokrb5/v8/iana" + "github.com/jcmturner/gokrb5/v8/iana/adtype" + "github.com/jcmturner/gokrb5/v8/iana/asnAppTag" + "github.com/jcmturner/gokrb5/v8/iana/errorcode" + "github.com/jcmturner/gokrb5/v8/iana/flags" + "github.com/jcmturner/gokrb5/v8/iana/keyusage" + "github.com/jcmturner/gokrb5/v8/keytab" + "github.com/jcmturner/gokrb5/v8/krberror" + "github.com/jcmturner/gokrb5/v8/pac" + "github.com/jcmturner/gokrb5/v8/types" +) + +// Reference: https://www.ietf.org/rfc/rfc4120.txt +// Section: 5.3 + +// Ticket implements the Kerberos ticket. +type Ticket struct { + TktVNO int `asn1:"explicit,tag:0"` + Realm string `asn1:"generalstring,explicit,tag:1"` + SName types.PrincipalName `asn1:"explicit,tag:2"` + EncPart types.EncryptedData `asn1:"explicit,tag:3"` + DecryptedEncPart EncTicketPart `asn1:"optional"` // Not part of ASN1 bytes so marked as optional so unmarshalling works +} + +// EncTicketPart is the encrypted part of the Ticket. +type EncTicketPart struct { + Flags asn1.BitString `asn1:"explicit,tag:0"` + Key types.EncryptionKey `asn1:"explicit,tag:1"` + CRealm string `asn1:"generalstring,explicit,tag:2"` + CName types.PrincipalName `asn1:"explicit,tag:3"` + Transited TransitedEncoding `asn1:"explicit,tag:4"` + AuthTime time.Time `asn1:"generalized,explicit,tag:5"` + StartTime time.Time `asn1:"generalized,explicit,optional,tag:6"` + EndTime time.Time `asn1:"generalized,explicit,tag:7"` + RenewTill time.Time `asn1:"generalized,explicit,optional,tag:8"` + CAddr types.HostAddresses `asn1:"explicit,optional,tag:9"` + AuthorizationData types.AuthorizationData `asn1:"explicit,optional,tag:10"` +} + +// TransitedEncoding part of the ticket's encrypted part. +type TransitedEncoding struct { + TRType int32 `asn1:"explicit,tag:0"` + Contents []byte `asn1:"explicit,tag:1"` +} + +// NewTicket creates a new Ticket instance. +func NewTicket(cname types.PrincipalName, crealm string, sname types.PrincipalName, srealm string, flags asn1.BitString, sktab *keytab.Keytab, eTypeID int32, kvno int, authTime, startTime, endTime, renewTill time.Time) (Ticket, types.EncryptionKey, error) { + etype, err := crypto.GetEtype(eTypeID) + if err != nil { + return Ticket{}, types.EncryptionKey{}, krberror.Errorf(err, krberror.EncryptingError, "error getting etype for new ticket") + } + sessionKey, err := types.GenerateEncryptionKey(etype) + if err != nil { + return Ticket{}, types.EncryptionKey{}, krberror.Errorf(err, krberror.EncryptingError, "error generating session key") + } + + etp := EncTicketPart{ + Flags: flags, + Key: sessionKey, + CRealm: crealm, + CName: cname, + Transited: TransitedEncoding{}, + AuthTime: authTime, + StartTime: startTime, + EndTime: endTime, + RenewTill: renewTill, + } + b, err := asn1.Marshal(etp) + if err != nil { + return Ticket{}, types.EncryptionKey{}, krberror.Errorf(err, krberror.EncodingError, "error marshalling ticket encpart") + } + b = asn1tools.AddASNAppTag(b, asnAppTag.EncTicketPart) + skey, _, err := sktab.GetEncryptionKey(sname, srealm, kvno, eTypeID) + if err != nil { + return Ticket{}, types.EncryptionKey{}, krberror.Errorf(err, krberror.EncryptingError, "error getting encryption key for new ticket") + } + ed, err := crypto.GetEncryptedData(b, skey, keyusage.KDC_REP_TICKET, kvno) + if err != nil { + return Ticket{}, types.EncryptionKey{}, krberror.Errorf(err, krberror.EncryptingError, "error encrypting ticket encpart") + } + tkt := Ticket{ + TktVNO: iana.PVNO, + Realm: srealm, + SName: sname, + EncPart: ed, + } + return tkt, sessionKey, nil +} + +// Unmarshal bytes b into a Ticket struct. +func (t *Ticket) Unmarshal(b []byte) error { + _, err := asn1.UnmarshalWithParams(b, t, fmt.Sprintf("application,explicit,tag:%d", asnAppTag.Ticket)) + return err +} + +// Marshal the Ticket. +func (t *Ticket) Marshal() ([]byte, error) { + b, err := asn1.Marshal(*t) + if err != nil { + return nil, err + } + b = asn1tools.AddASNAppTag(b, asnAppTag.Ticket) + return b, nil +} + +// Unmarshal bytes b into the EncTicketPart struct. +func (t *EncTicketPart) Unmarshal(b []byte) error { + _, err := asn1.UnmarshalWithParams(b, t, fmt.Sprintf("application,explicit,tag:%d", asnAppTag.EncTicketPart)) + return err +} + +// unmarshalTicket returns a ticket from the bytes provided. +func unmarshalTicket(b []byte) (t Ticket, err error) { + err = t.Unmarshal(b) + return +} + +// UnmarshalTicketsSequence returns a slice of Tickets from a raw ASN1 value. +func unmarshalTicketsSequence(in asn1.RawValue) ([]Ticket, error) { + //This is a workaround to a asn1 decoding issue in golang - https://github.com/golang/go/issues/17321. It's not pretty I'm afraid + //We pull out raw values from the larger raw value (that is actually the data of the sequence of raw values) and track our position moving along the data. + b := in.Bytes + // Ignore the head of the asn1 stream (1 byte for tag and those for the length) as this is what tells us its a sequence but we're handling it ourselves + p := 1 + asn1tools.GetNumberBytesInLengthHeader(in.Bytes) + var tkts []Ticket + var raw asn1.RawValue + for p < (len(b)) { + _, err := asn1.UnmarshalWithParams(b[p:], &raw, fmt.Sprintf("application,tag:%d", asnAppTag.Ticket)) + if err != nil { + return nil, fmt.Errorf("unmarshaling sequence of tickets failed getting length of ticket: %v", err) + } + t, err := unmarshalTicket(b[p:]) + if err != nil { + return nil, fmt.Errorf("unmarshaling sequence of tickets failed: %v", err) + } + p += len(raw.FullBytes) + tkts = append(tkts, t) + } + MarshalTicketSequence(tkts) + return tkts, nil +} + +// MarshalTicketSequence marshals a slice of Tickets returning an ASN1 raw value containing the ticket sequence. +func MarshalTicketSequence(tkts []Ticket) (asn1.RawValue, error) { + raw := asn1.RawValue{ + Class: 2, + IsCompound: true, + } + if len(tkts) < 1 { + // There are no tickets to marshal + return raw, nil + } + var btkts []byte + for i, t := range tkts { + b, err := t.Marshal() + if err != nil { + return raw, fmt.Errorf("error marshaling ticket number %d in sequence of tickets", i+1) + } + btkts = append(btkts, b...) + } + // The ASN1 wrapping consists of 2 bytes: + // 1st byte -> Identifier Octet - In this case an OCTET STRING (ASN TAG + // 2nd byte -> The length (this will be the size indicated in the input bytes + 2 for the additional bytes we add here. + // Application Tag: + //| Byte: | 8 | 7 | 6 | 5 | 4 | 3 | 2 | 1 | + //| Value: | 0 | 1 | 1 | From the RFC spec 4120 | + //| Explanation | Defined by the ASN1 encoding rules for an application tag | A value of 1 indicates a constructed type | The ASN Application tag value | + btkts = append(asn1tools.MarshalLengthBytes(len(btkts)), btkts...) + btkts = append([]byte{byte(32 + asn1.TagSequence)}, btkts...) + raw.Bytes = btkts + // If we need to create the full bytes then identifier octet is "context-specific" = 128 + "constructed" + 32 + the wrapping explicit tag (11) + //fmt.Fprintf(os.Stderr, "mRaw fb: %v\n", raw.FullBytes) + return raw, nil +} + +// DecryptEncPart decrypts the encrypted part of the ticket. +// The sname argument can be used to specify which service principal's key should be used to decrypt the ticket. +// If nil is passed as the sname then the service principal specified within the ticket it used. +func (t *Ticket) DecryptEncPart(keytab *keytab.Keytab, sname *types.PrincipalName) error { + if sname == nil { + sname = &t.SName + } + key, _, err := keytab.GetEncryptionKey(*sname, t.Realm, t.EncPart.KVNO, t.EncPart.EType) + if err != nil { + return NewKRBError(t.SName, t.Realm, errorcode.KRB_AP_ERR_NOKEY, fmt.Sprintf("Could not get key from keytab: %v", err)) + } + return t.Decrypt(key) +} + +// Decrypt decrypts the encrypted part of the ticket using the key provided. +func (t *Ticket) Decrypt(key types.EncryptionKey) error { + b, err := crypto.DecryptEncPart(t.EncPart, key, keyusage.KDC_REP_TICKET) + if err != nil { + return fmt.Errorf("error decrypting Ticket EncPart: %v", err) + } + var denc EncTicketPart + err = denc.Unmarshal(b) + if err != nil { + return fmt.Errorf("error unmarshaling encrypted part: %v", err) + } + t.DecryptedEncPart = denc + return nil +} + +// GetPACType returns a Microsoft PAC that has been extracted from the ticket and processed. +func (t *Ticket) GetPACType(keytab *keytab.Keytab, sname *types.PrincipalName, l *log.Logger) (bool, pac.PACType, error) { + var isPAC bool + for _, ad := range t.DecryptedEncPart.AuthorizationData { + if ad.ADType == adtype.ADIfRelevant { + var ad2 types.AuthorizationData + err := ad2.Unmarshal(ad.ADData) + if err != nil { + l.Printf("PAC authorization data could not be unmarshaled: %v", err) + continue + } + if ad2[0].ADType == adtype.ADWin2KPAC { + isPAC = true + var p pac.PACType + err = p.Unmarshal(ad2[0].ADData) + if err != nil { + return isPAC, p, fmt.Errorf("error unmarshaling PAC: %v", err) + } + if sname == nil { + sname = &t.SName + } + key, _, err := keytab.GetEncryptionKey(*sname, t.Realm, t.EncPart.KVNO, t.EncPart.EType) + if err != nil { + return isPAC, p, NewKRBError(t.SName, t.Realm, errorcode.KRB_AP_ERR_NOKEY, fmt.Sprintf("Could not get key from keytab: %v", err)) + } + err = p.ProcessPACInfoBuffers(key, l) + return isPAC, p, err + } + } + } + return isPAC, pac.PACType{}, nil +} + +// Valid checks it the ticket is currently valid. Max duration passed endtime passed in as argument. +func (t *Ticket) Valid(d time.Duration) (bool, error) { + // Check for future tickets or invalid tickets + time := time.Now().UTC() + if t.DecryptedEncPart.StartTime.Sub(time) > d || types.IsFlagSet(&t.DecryptedEncPart.Flags, flags.Invalid) { + return false, NewKRBError(t.SName, t.Realm, errorcode.KRB_AP_ERR_TKT_NYV, "service ticket provided is not yet valid") + } + + // Check for expired ticket + if time.Sub(t.DecryptedEncPart.EndTime) > d { + return false, NewKRBError(t.SName, t.Realm, errorcode.KRB_AP_ERR_TKT_EXPIRED, "service ticket provided has expired") + } + + return true, nil +} diff --git a/vendor/github.com/jcmturner/gokrb5/v8/pac/client_claims.go b/vendor/github.com/jcmturner/gokrb5/v8/pac/client_claims.go new file mode 100644 index 000000000000..36871e00fb48 --- /dev/null +++ b/vendor/github.com/jcmturner/gokrb5/v8/pac/client_claims.go @@ -0,0 +1,34 @@ +package pac + +import ( + "bytes" + "fmt" + + "github.com/jcmturner/rpc/v2/mstypes" + "github.com/jcmturner/rpc/v2/ndr" +) + +// Claims reference: https://msdn.microsoft.com/en-us/library/hh553895.aspx + +// ClientClaimsInfo implements https://msdn.microsoft.com/en-us/library/hh536365.aspx +type ClientClaimsInfo struct { + ClaimsSetMetadata mstypes.ClaimsSetMetadata + ClaimsSet mstypes.ClaimsSet +} + +// Unmarshal bytes into the ClientClaimsInfo struct +func (k *ClientClaimsInfo) Unmarshal(b []byte) (err error) { + dec := ndr.NewDecoder(bytes.NewReader(b)) + m := new(mstypes.ClaimsSetMetadata) + err = dec.Decode(m) + if err != nil { + err = fmt.Errorf("error unmarshaling ClientClaimsInfo ClaimsSetMetadata: %v", err) + return + } + k.ClaimsSetMetadata = *m + k.ClaimsSet, err = k.ClaimsSetMetadata.ClaimsSet() + if err != nil { + err = fmt.Errorf("error unmarshaling ClientClaimsInfo ClaimsSet: %v", err) + } + return +} diff --git a/vendor/github.com/jcmturner/gokrb5/v8/pac/client_info.go b/vendor/github.com/jcmturner/gokrb5/v8/pac/client_info.go new file mode 100644 index 000000000000..ddd957808a81 --- /dev/null +++ b/vendor/github.com/jcmturner/gokrb5/v8/pac/client_info.go @@ -0,0 +1,31 @@ +package pac + +import ( + "bytes" + + "github.com/jcmturner/rpc/v2/mstypes" +) + +// ClientInfo implements https://msdn.microsoft.com/en-us/library/cc237951.aspx +type ClientInfo struct { + ClientID mstypes.FileTime // A FILETIME structure in little-endian format that contains the Kerberos initial ticket-granting ticket TGT authentication time + NameLength uint16 // An unsigned 16-bit integer in little-endian format that specifies the length, in bytes, of the Name field. + Name string // An array of 16-bit Unicode characters in little-endian format that contains the client's account name. +} + +// Unmarshal bytes into the ClientInfo struct +func (k *ClientInfo) Unmarshal(b []byte) (err error) { + //The PAC_CLIENT_INFO structure is a simple structure that is not NDR-encoded. + r := mstypes.NewReader(bytes.NewReader(b)) + + k.ClientID, err = r.FileTime() + if err != nil { + return + } + k.NameLength, err = r.Uint16() + if err != nil { + return + } + k.Name, err = r.UTF16String(int(k.NameLength)) + return +} diff --git a/vendor/github.com/jcmturner/gokrb5/v8/pac/credentials_info.go b/vendor/github.com/jcmturner/gokrb5/v8/pac/credentials_info.go new file mode 100644 index 000000000000..0c7ccd4263e5 --- /dev/null +++ b/vendor/github.com/jcmturner/gokrb5/v8/pac/credentials_info.go @@ -0,0 +1,86 @@ +package pac + +import ( + "bytes" + "errors" + "fmt" + + "github.com/jcmturner/gokrb5/v8/crypto" + "github.com/jcmturner/gokrb5/v8/iana/keyusage" + "github.com/jcmturner/gokrb5/v8/types" + "github.com/jcmturner/rpc/v2/mstypes" + "github.com/jcmturner/rpc/v2/ndr" +) + +// https://msdn.microsoft.com/en-us/library/cc237931.aspx + +// CredentialsInfo implements https://msdn.microsoft.com/en-us/library/cc237953.aspx +type CredentialsInfo struct { + Version uint32 // A 32-bit unsigned integer in little-endian format that defines the version. MUST be 0x00000000. + EType uint32 + PACCredentialDataEncrypted []byte // Key usage number for encryption: KERB_NON_KERB_SALT (16) + PACCredentialData CredentialData +} + +// Unmarshal bytes into the CredentialsInfo struct +func (c *CredentialsInfo) Unmarshal(b []byte, k types.EncryptionKey) (err error) { + //The CredentialsInfo structure is a simple structure that is not NDR-encoded. + r := mstypes.NewReader(bytes.NewReader(b)) + + c.Version, err = r.Uint32() + if err != nil { + return + } + if c.Version != 0 { + err = errors.New("credentials info version is not zero") + return + } + c.EType, err = r.Uint32() + if err != nil { + return + } + c.PACCredentialDataEncrypted, err = r.ReadBytes(len(b) - 8) + if err != nil { + err = fmt.Errorf("error reading PAC Credetials Data: %v", err) + return + } + + err = c.DecryptEncPart(k) + if err != nil { + err = fmt.Errorf("error decrypting PAC Credentials Data: %v", err) + return + } + return +} + +// DecryptEncPart decrypts the encrypted part of the CredentialsInfo. +func (c *CredentialsInfo) DecryptEncPart(k types.EncryptionKey) error { + if k.KeyType != int32(c.EType) { + return fmt.Errorf("key provided is not the correct type. Type needed: %d, type provided: %d", c.EType, k.KeyType) + } + pt, err := crypto.DecryptMessage(c.PACCredentialDataEncrypted, k, keyusage.KERB_NON_KERB_SALT) + if err != nil { + return err + } + err = c.PACCredentialData.Unmarshal(pt) + if err != nil { + return err + } + return nil +} + +// CredentialData implements https://msdn.microsoft.com/en-us/library/cc237952.aspx +type CredentialData struct { + CredentialCount uint32 + Credentials []SECPKGSupplementalCred // Size is the value of CredentialCount +} + +// Unmarshal converts the bytes provided into a CredentialData type. +func (c *CredentialData) Unmarshal(b []byte) (err error) { + dec := ndr.NewDecoder(bytes.NewReader(b)) + err = dec.Decode(c) + if err != nil { + err = fmt.Errorf("error unmarshaling KerbValidationInfo: %v", err) + } + return +} diff --git a/vendor/github.com/jcmturner/gokrb5/v8/pac/device_claims.go b/vendor/github.com/jcmturner/gokrb5/v8/pac/device_claims.go new file mode 100644 index 000000000000..6eb29260050f --- /dev/null +++ b/vendor/github.com/jcmturner/gokrb5/v8/pac/device_claims.go @@ -0,0 +1,34 @@ +package pac + +import ( + "bytes" + "fmt" + + "github.com/jcmturner/rpc/v2/mstypes" + "github.com/jcmturner/rpc/v2/ndr" +) + +// Claims reference: https://msdn.microsoft.com/en-us/library/hh553895.aspx + +// DeviceClaimsInfo implements https://msdn.microsoft.com/en-us/library/hh554226.aspx +type DeviceClaimsInfo struct { + ClaimsSetMetadata mstypes.ClaimsSetMetadata + ClaimsSet mstypes.ClaimsSet +} + +// Unmarshal bytes into the ClientClaimsInfo struct +func (k *DeviceClaimsInfo) Unmarshal(b []byte) (err error) { + dec := ndr.NewDecoder(bytes.NewReader(b)) + m := new(mstypes.ClaimsSetMetadata) + err = dec.Decode(m) + if err != nil { + err = fmt.Errorf("error unmarshaling ClientClaimsInfo ClaimsSetMetadata: %v", err) + return + } + k.ClaimsSetMetadata = *m + k.ClaimsSet, err = k.ClaimsSetMetadata.ClaimsSet() + if err != nil { + err = fmt.Errorf("error unmarshaling ClientClaimsInfo ClaimsSet: %v", err) + } + return +} diff --git a/vendor/github.com/jcmturner/gokrb5/v8/pac/device_info.go b/vendor/github.com/jcmturner/gokrb5/v8/pac/device_info.go new file mode 100644 index 000000000000..ce82daa58019 --- /dev/null +++ b/vendor/github.com/jcmturner/gokrb5/v8/pac/device_info.go @@ -0,0 +1,32 @@ +package pac + +import ( + "bytes" + "fmt" + + "github.com/jcmturner/rpc/v2/mstypes" + "github.com/jcmturner/rpc/v2/ndr" +) + +// DeviceInfo implements https://msdn.microsoft.com/en-us/library/hh536402.aspx +type DeviceInfo struct { + UserID uint32 // A 32-bit unsigned integer that contains the RID of the account. If the UserId member equals 0x00000000, the first group SID in this member is the SID for this account. + PrimaryGroupID uint32 // A 32-bit unsigned integer that contains the RID for the primary group to which this account belongs. + AccountDomainID mstypes.RPCSID `ndr:"pointer"` // A SID structure that contains the SID for the domain of the account.This member is used in conjunction with the UserId, and GroupIds members to create the user and group SIDs for the client. + AccountGroupCount uint32 // A 32-bit unsigned integer that contains the number of groups within the account domain to which the account belongs + AccountGroupIDs []mstypes.GroupMembership `ndr:"pointer,conformant"` // A pointer to a list of GROUP_MEMBERSHIP (section 2.2.2) structures that contains the groups to which the account belongs in the account domain. The number of groups in this list MUST be equal to GroupCount. + SIDCount uint32 // A 32-bit unsigned integer that contains the total number of SIDs present in the ExtraSids member. + ExtraSIDs []mstypes.KerbSidAndAttributes `ndr:"pointer,conformant"` // A pointer to a list of KERB_SID_AND_ATTRIBUTES structures that contain a list of SIDs corresponding to groups not in domains. If the UserId member equals 0x00000000, the first group SID in this member is the SID for this account. + DomainGroupCount uint32 // A 32-bit unsigned integer that contains the number of domains with groups to which the account belongs. + DomainGroup []mstypes.DomainGroupMembership `ndr:"pointer,conformant"` // A pointer to a list of DOMAIN_GROUP_MEMBERSHIP structures (section 2.2.3) that contains the domains to which the account belongs to a group. The number of sets in this list MUST be equal to DomainCount. +} + +// Unmarshal bytes into the DeviceInfo struct +func (k *DeviceInfo) Unmarshal(b []byte) (err error) { + dec := ndr.NewDecoder(bytes.NewReader(b)) + err = dec.Decode(k) + if err != nil { + err = fmt.Errorf("error unmarshaling DeviceInfo: %v", err) + } + return +} diff --git a/vendor/github.com/jcmturner/gokrb5/v8/pac/kerb_validation_info.go b/vendor/github.com/jcmturner/gokrb5/v8/pac/kerb_validation_info.go new file mode 100644 index 000000000000..dde78614eeeb --- /dev/null +++ b/vendor/github.com/jcmturner/gokrb5/v8/pac/kerb_validation_info.go @@ -0,0 +1,110 @@ +// Package pac implements Microsoft Privilege Attribute Certificate (PAC) processing. +package pac + +import ( + "bytes" + "fmt" + + "github.com/jcmturner/rpc/v2/mstypes" + "github.com/jcmturner/rpc/v2/ndr" +) + +// KERB_VALIDATION_INFO flags. +const ( + USERFLAG_GUEST = 31 // Authentication was done via the GUEST account; no password was used. + USERFLAG_NO_ENCRYPTION_AVAILABLE = 30 // No encryption is available. + USERFLAG_LAN_MANAGER_KEY = 28 // LAN Manager key was used for authentication. + USERFLAG_SUB_AUTH = 25 // Sub-authentication used; session key came from the sub-authentication package. + USERFLAG_EXTRA_SIDS = 26 // Indicates that the ExtraSids field is populated and contains additional SIDs. + USERFLAG_MACHINE_ACCOUNT = 24 // Indicates that the account is a machine account. + USERFLAG_DC_NTLM2 = 23 // Indicates that the domain controller understands NTLMv2. + USERFLAG_RESOURCE_GROUPIDS = 22 // Indicates that the ResourceGroupIds field is populated. + USERFLAG_PROFILEPATH = 21 // Indicates that ProfilePath is populated. + USERFLAG_NTLM2_NTCHALLENGERESP = 20 // The NTLMv2 response from the NtChallengeResponseFields ([MS-NLMP] section 2.2.1.3) was used for authentication and session key generation. + USERFLAG_LM2_LMCHALLENGERESP = 19 // The LMv2 response from the LmChallengeResponseFields ([MS-NLMP] section 2.2.1.3) was used for authentication and session key generation. + USERFLAG_AUTH_LMCHALLENGERESP_KEY_NTCHALLENGERESP = 18 // The LMv2 response from the LmChallengeResponseFields ([MS-NLMP] section 2.2.1.3) was used for authentication and the NTLMv2 response from the NtChallengeResponseFields ([MS-NLMP] section 2.2.1.3) was used session key generation. +) + +// KerbValidationInfo implement https://msdn.microsoft.com/en-us/library/cc237948.aspx +type KerbValidationInfo struct { + LogOnTime mstypes.FileTime + LogOffTime mstypes.FileTime + KickOffTime mstypes.FileTime + PasswordLastSet mstypes.FileTime + PasswordCanChange mstypes.FileTime + PasswordMustChange mstypes.FileTime + EffectiveName mstypes.RPCUnicodeString + FullName mstypes.RPCUnicodeString + LogonScript mstypes.RPCUnicodeString + ProfilePath mstypes.RPCUnicodeString + HomeDirectory mstypes.RPCUnicodeString + HomeDirectoryDrive mstypes.RPCUnicodeString + LogonCount uint16 + BadPasswordCount uint16 + UserID uint32 + PrimaryGroupID uint32 + GroupCount uint32 + GroupIDs []mstypes.GroupMembership `ndr:"pointer,conformant"` + UserFlags uint32 + UserSessionKey mstypes.UserSessionKey + LogonServer mstypes.RPCUnicodeString + LogonDomainName mstypes.RPCUnicodeString + LogonDomainID mstypes.RPCSID `ndr:"pointer"` + Reserved1 [2]uint32 // Has 2 elements + UserAccountControl uint32 + SubAuthStatus uint32 + LastSuccessfulILogon mstypes.FileTime + LastFailedILogon mstypes.FileTime + FailedILogonCount uint32 + Reserved3 uint32 + SIDCount uint32 + ExtraSIDs []mstypes.KerbSidAndAttributes `ndr:"pointer,conformant"` + ResourceGroupDomainSID mstypes.RPCSID `ndr:"pointer"` + ResourceGroupCount uint32 + ResourceGroupIDs []mstypes.GroupMembership `ndr:"pointer,conformant"` +} + +// Unmarshal bytes into the DeviceInfo struct +func (k *KerbValidationInfo) Unmarshal(b []byte) (err error) { + dec := ndr.NewDecoder(bytes.NewReader(b)) + err = dec.Decode(k) + if err != nil { + err = fmt.Errorf("error unmarshaling KerbValidationInfo: %v", err) + } + return +} + +// GetGroupMembershipSIDs returns a slice of strings containing the group membership SIDs found in the PAC. +func (k *KerbValidationInfo) GetGroupMembershipSIDs() []string { + var g []string + lSID := k.LogonDomainID.String() + for i := range k.GroupIDs { + g = append(g, fmt.Sprintf("%s-%d", lSID, k.GroupIDs[i].RelativeID)) + } + for _, s := range k.ExtraSIDs { + var exists = false + for _, es := range g { + if es == s.SID.String() { + exists = true + break + } + } + if !exists { + g = append(g, s.SID.String()) + } + } + for _, r := range k.ResourceGroupIDs { + var exists = false + s := fmt.Sprintf("%s-%d", k.ResourceGroupDomainSID.String(), r.RelativeID) + for _, es := range g { + if es == s { + exists = true + break + } + } + if !exists { + g = append(g, s) + } + } + return g +} diff --git a/vendor/github.com/jcmturner/gokrb5/v8/pac/pac_type.go b/vendor/github.com/jcmturner/gokrb5/v8/pac/pac_type.go new file mode 100644 index 000000000000..fab2ad7cda76 --- /dev/null +++ b/vendor/github.com/jcmturner/gokrb5/v8/pac/pac_type.go @@ -0,0 +1,251 @@ +package pac + +import ( + "bytes" + "errors" + "fmt" + "log" + + "github.com/jcmturner/gokrb5/v8/crypto" + "github.com/jcmturner/gokrb5/v8/iana/keyusage" + "github.com/jcmturner/gokrb5/v8/types" + "github.com/jcmturner/rpc/v2/mstypes" +) + +const ( + infoTypeKerbValidationInfo uint32 = 1 + infoTypeCredentials uint32 = 2 + infoTypePACServerSignatureData uint32 = 6 + infoTypePACKDCSignatureData uint32 = 7 + infoTypePACClientInfo uint32 = 10 + infoTypeS4UDelegationInfo uint32 = 11 + infoTypeUPNDNSInfo uint32 = 12 + infoTypePACClientClaimsInfo uint32 = 13 + infoTypePACDeviceInfo uint32 = 14 + infoTypePACDeviceClaimsInfo uint32 = 15 +) + +// PACType implements: https://msdn.microsoft.com/en-us/library/cc237950.aspx +type PACType struct { + CBuffers uint32 + Version uint32 + Buffers []InfoBuffer + Data []byte + KerbValidationInfo *KerbValidationInfo + CredentialsInfo *CredentialsInfo + ServerChecksum *SignatureData + KDCChecksum *SignatureData + ClientInfo *ClientInfo + S4UDelegationInfo *S4UDelegationInfo + UPNDNSInfo *UPNDNSInfo + ClientClaimsInfo *ClientClaimsInfo + DeviceInfo *DeviceInfo + DeviceClaimsInfo *DeviceClaimsInfo + ZeroSigData []byte +} + +// InfoBuffer implements the PAC Info Buffer: https://msdn.microsoft.com/en-us/library/cc237954.aspx +type InfoBuffer struct { + ULType uint32 // A 32-bit unsigned integer in little-endian format that describes the type of data present in the buffer contained at Offset. + CBBufferSize uint32 // A 32-bit unsigned integer in little-endian format that contains the size, in bytes, of the buffer in the PAC located at Offset. + Offset uint64 // A 64-bit unsigned integer in little-endian format that contains the offset to the beginning of the buffer, in bytes, from the beginning of the PACTYPE structure. The data offset MUST be a multiple of eight. The following sections specify the format of each type of element. +} + +// Unmarshal bytes into the PACType struct +func (pac *PACType) Unmarshal(b []byte) (err error) { + pac.Data = b + zb := make([]byte, len(b), len(b)) + copy(zb, b) + pac.ZeroSigData = zb + r := mstypes.NewReader(bytes.NewReader(b)) + pac.CBuffers, err = r.Uint32() + if err != nil { + return + } + pac.Version, err = r.Uint32() + if err != nil { + return + } + buf := make([]InfoBuffer, pac.CBuffers, pac.CBuffers) + for i := range buf { + buf[i].ULType, err = r.Uint32() + if err != nil { + return + } + buf[i].CBBufferSize, err = r.Uint32() + if err != nil { + return + } + buf[i].Offset, err = r.Uint64() + if err != nil { + return + } + } + pac.Buffers = buf + return nil +} + +// ProcessPACInfoBuffers processes the PAC Info Buffers. +// https://msdn.microsoft.com/en-us/library/cc237954.aspx +func (pac *PACType) ProcessPACInfoBuffers(key types.EncryptionKey, l *log.Logger) error { + for _, buf := range pac.Buffers { + p := make([]byte, buf.CBBufferSize, buf.CBBufferSize) + copy(p, pac.Data[int(buf.Offset):int(buf.Offset)+int(buf.CBBufferSize)]) + switch buf.ULType { + case infoTypeKerbValidationInfo: + if pac.KerbValidationInfo != nil { + //Must ignore subsequent buffers of this type + continue + } + var k KerbValidationInfo + err := k.Unmarshal(p) + if err != nil { + return fmt.Errorf("error processing KerbValidationInfo: %v", err) + } + pac.KerbValidationInfo = &k + case infoTypeCredentials: + // Currently PAC parsing is only useful on the service side in gokrb5 + // The CredentialsInfo are only useful when gokrb5 has implemented RFC4556 and only applied on the client side. + // Skipping CredentialsInfo - will be revisited under RFC4556 implementation. + continue + //if pac.CredentialsInfo != nil { + // //Must ignore subsequent buffers of this type + // continue + //} + //var k CredentialsInfo + //err := k.Unmarshal(p, key) // The encryption key used is the AS reply key only available to the client. + //if err != nil { + // return fmt.Errorf("error processing CredentialsInfo: %v", err) + //} + //pac.CredentialsInfo = &k + case infoTypePACServerSignatureData: + if pac.ServerChecksum != nil { + //Must ignore subsequent buffers of this type + continue + } + var k SignatureData + zb, err := k.Unmarshal(p) + copy(pac.ZeroSigData[int(buf.Offset):int(buf.Offset)+int(buf.CBBufferSize)], zb) + if err != nil { + return fmt.Errorf("error processing ServerChecksum: %v", err) + } + pac.ServerChecksum = &k + case infoTypePACKDCSignatureData: + if pac.KDCChecksum != nil { + //Must ignore subsequent buffers of this type + continue + } + var k SignatureData + zb, err := k.Unmarshal(p) + copy(pac.ZeroSigData[int(buf.Offset):int(buf.Offset)+int(buf.CBBufferSize)], zb) + if err != nil { + return fmt.Errorf("error processing KDCChecksum: %v", err) + } + pac.KDCChecksum = &k + case infoTypePACClientInfo: + if pac.ClientInfo != nil { + //Must ignore subsequent buffers of this type + continue + } + var k ClientInfo + err := k.Unmarshal(p) + if err != nil { + return fmt.Errorf("error processing ClientInfo: %v", err) + } + pac.ClientInfo = &k + case infoTypeS4UDelegationInfo: + if pac.S4UDelegationInfo != nil { + //Must ignore subsequent buffers of this type + continue + } + var k S4UDelegationInfo + err := k.Unmarshal(p) + if err != nil { + l.Printf("could not process S4U_DelegationInfo: %v", err) + continue + } + pac.S4UDelegationInfo = &k + case infoTypeUPNDNSInfo: + if pac.UPNDNSInfo != nil { + //Must ignore subsequent buffers of this type + continue + } + var k UPNDNSInfo + err := k.Unmarshal(p) + if err != nil { + l.Printf("could not process UPN_DNSInfo: %v", err) + continue + } + pac.UPNDNSInfo = &k + case infoTypePACClientClaimsInfo: + if pac.ClientClaimsInfo != nil || len(p) < 1 { + //Must ignore subsequent buffers of this type + continue + } + var k ClientClaimsInfo + err := k.Unmarshal(p) + if err != nil { + l.Printf("could not process ClientClaimsInfo: %v", err) + continue + } + pac.ClientClaimsInfo = &k + case infoTypePACDeviceInfo: + if pac.DeviceInfo != nil { + //Must ignore subsequent buffers of this type + continue + } + var k DeviceInfo + err := k.Unmarshal(p) + if err != nil { + l.Printf("could not process DeviceInfo: %v", err) + continue + } + pac.DeviceInfo = &k + case infoTypePACDeviceClaimsInfo: + if pac.DeviceClaimsInfo != nil { + //Must ignore subsequent buffers of this type + continue + } + var k DeviceClaimsInfo + err := k.Unmarshal(p) + if err != nil { + l.Printf("could not process DeviceClaimsInfo: %v", err) + continue + } + pac.DeviceClaimsInfo = &k + } + } + + if ok, err := pac.verify(key); !ok { + return err + } + + return nil +} + +func (pac *PACType) verify(key types.EncryptionKey) (bool, error) { + if pac.KerbValidationInfo == nil { + return false, errors.New("PAC Info Buffers does not contain a KerbValidationInfo") + } + if pac.ServerChecksum == nil { + return false, errors.New("PAC Info Buffers does not contain a ServerChecksum") + } + if pac.KDCChecksum == nil { + return false, errors.New("PAC Info Buffers does not contain a KDCChecksum") + } + if pac.ClientInfo == nil { + return false, errors.New("PAC Info Buffers does not contain a ClientInfo") + } + etype, err := crypto.GetChksumEtype(int32(pac.ServerChecksum.SignatureType)) + if err != nil { + return false, err + } + if ok := etype.VerifyChecksum(key.KeyValue, + pac.ZeroSigData, + pac.ServerChecksum.Signature, + keyusage.KERB_NON_KERB_CKSUM_SALT); !ok { + return false, errors.New("PAC service checksum verification failed") + } + + return true, nil +} diff --git a/vendor/github.com/jcmturner/gokrb5/v8/pac/s4u_delegation_info.go b/vendor/github.com/jcmturner/gokrb5/v8/pac/s4u_delegation_info.go new file mode 100644 index 000000000000..da837d4b3a49 --- /dev/null +++ b/vendor/github.com/jcmturner/gokrb5/v8/pac/s4u_delegation_info.go @@ -0,0 +1,26 @@ +package pac + +import ( + "bytes" + "fmt" + + "github.com/jcmturner/rpc/v2/mstypes" + "github.com/jcmturner/rpc/v2/ndr" +) + +// S4UDelegationInfo implements https://msdn.microsoft.com/en-us/library/cc237944.aspx +type S4UDelegationInfo struct { + S4U2proxyTarget mstypes.RPCUnicodeString // The name of the principal to whom the application can forward the ticket. + TransitedListSize uint32 + S4UTransitedServices []mstypes.RPCUnicodeString `ndr:"pointer,conformant"` // List of all services that have been delegated through by this client and subsequent services or servers.. Size is value of TransitedListSize +} + +// Unmarshal bytes into the S4UDelegationInfo struct +func (k *S4UDelegationInfo) Unmarshal(b []byte) (err error) { + dec := ndr.NewDecoder(bytes.NewReader(b)) + err = dec.Decode(k) + if err != nil { + err = fmt.Errorf("error unmarshaling S4UDelegationInfo: %v", err) + } + return +} diff --git a/vendor/github.com/jcmturner/gokrb5/v8/pac/signature_data.go b/vendor/github.com/jcmturner/gokrb5/v8/pac/signature_data.go new file mode 100644 index 000000000000..8f6aa58faa88 --- /dev/null +++ b/vendor/github.com/jcmturner/gokrb5/v8/pac/signature_data.go @@ -0,0 +1,67 @@ +package pac + +import ( + "bytes" + + "github.com/jcmturner/gokrb5/v8/iana/chksumtype" + "github.com/jcmturner/rpc/v2/mstypes" +) + +/* +https://msdn.microsoft.com/en-us/library/cc237955.aspx + +The Key Usage Value MUST be KERB_NON_KERB_CKSUM_SALT (17) [MS-KILE] (section 3.1.5.9). + +Server Signature (SignatureType = 0x00000006) +https://msdn.microsoft.com/en-us/library/cc237957.aspx + +KDC Signature (SignatureType = 0x00000007) +https://msdn.microsoft.com/en-us/library/dd357117.aspx +*/ + +// SignatureData implements https://msdn.microsoft.com/en-us/library/cc237955.aspx +type SignatureData struct { + SignatureType uint32 // A 32-bit unsigned integer value in little-endian format that defines the cryptographic system used to calculate the checksum. This MUST be one of the following checksum types: KERB_CHECKSUM_HMAC_MD5 (signature size = 16), HMAC_SHA1_96_AES128 (signature size = 12), HMAC_SHA1_96_AES256 (signature size = 12). + Signature []byte // Size depends on the type. See comment above. + RODCIdentifier uint16 // A 16-bit unsigned integer value in little-endian format that contains the first 16 bits of the key version number ([MS-KILE] section 3.1.5.8) when the KDC is an RODC. When the KDC is not an RODC, this field does not exist. +} + +// Unmarshal bytes into the SignatureData struct +func (k *SignatureData) Unmarshal(b []byte) (rb []byte, err error) { + r := mstypes.NewReader(bytes.NewReader(b)) + + k.SignatureType, err = r.Uint32() + if err != nil { + return + } + + var c int + switch k.SignatureType { + case chksumtype.KERB_CHECKSUM_HMAC_MD5_UNSIGNED: + c = 16 + case uint32(chksumtype.HMAC_SHA1_96_AES128): + c = 12 + case uint32(chksumtype.HMAC_SHA1_96_AES256): + c = 12 + } + k.Signature, err = r.ReadBytes(c) + if err != nil { + return + } + + // When the KDC is not an Read Only Domain Controller (RODC), this field does not exist. + if len(b) >= 4+c+2 { + k.RODCIdentifier, err = r.Uint16() + if err != nil { + return + } + } + + // Create bytes with zeroed signature needed for checksum verification + rb = make([]byte, len(b), len(b)) + copy(rb, b) + z := make([]byte, len(b), len(b)) + copy(rb[4:4+c], z) + + return +} diff --git a/vendor/github.com/jcmturner/gokrb5/v8/pac/supplemental_cred.go b/vendor/github.com/jcmturner/gokrb5/v8/pac/supplemental_cred.go new file mode 100644 index 000000000000..d40679d49b84 --- /dev/null +++ b/vendor/github.com/jcmturner/gokrb5/v8/pac/supplemental_cred.go @@ -0,0 +1,87 @@ +package pac + +import ( + "bytes" + "encoding/binary" + "errors" + "fmt" + + "github.com/jcmturner/rpc/v2/mstypes" + "github.com/jcmturner/rpc/v2/ndr" +) + +const ( + // NTLMSupCredLMOWF indicates that the LM OWF member is present and valid. + NTLMSupCredLMOWF uint32 = 31 + // NTLMSupCredNTOWF indicates that the NT OWF member is present and valid. + NTLMSupCredNTOWF uint32 = 30 +) + +// NTLMSupplementalCred implements https://msdn.microsoft.com/en-us/library/cc237949.aspx +type NTLMSupplementalCred struct { + Version uint32 // A 32-bit unsigned integer that defines the credential version.This field MUST be 0x00000000. + Flags uint32 + LMPassword []byte // A 16-element array of unsigned 8-bit integers that define the LM OWF. The LMPassword member MUST be ignored if the L flag is not set in the Flags member. + NTPassword []byte // A 16-element array of unsigned 8-bit integers that define the NT OWF. The NTPassword member MUST be ignored if the N flag is not set in the Flags member. +} + +// Unmarshal converts the bytes provided into a NTLMSupplementalCred. +func (c *NTLMSupplementalCred) Unmarshal(b []byte) (err error) { + r := mstypes.NewReader(bytes.NewReader(b)) + c.Version, err = r.Uint32() + if err != nil { + return + } + if c.Version != 0 { + err = errors.New("NTLMSupplementalCred version is not zero") + return + } + c.Flags, err = r.Uint32() + if err != nil { + return + } + if isFlagSet(c.Flags, NTLMSupCredLMOWF) { + c.LMPassword, err = r.ReadBytes(16) + if err != nil { + return + } + } + if isFlagSet(c.Flags, NTLMSupCredNTOWF) { + c.NTPassword, err = r.ReadBytes(16) + if err != nil { + return + } + } + return +} + +// isFlagSet tests if a flag is set in the uint32 little endian flag +func isFlagSet(f uint32, i uint32) bool { + //Which byte? + b := int(i / 8) + //Which bit in byte + p := uint(7 - (int(i) - 8*b)) + fb := make([]byte, 4) + binary.LittleEndian.PutUint32(fb, f) + if fb[b]&(1<: +func GetHostAddress(s string) (HostAddress, error) { + var h HostAddress + cAddr, _, err := net.SplitHostPort(s) + if err != nil { + return h, fmt.Errorf("invalid format of client address: %v", err) + } + ip := net.ParseIP(cAddr) + var ht int32 + if ip.To4() != nil { + ht = addrtype.IPv4 + ip = ip.To4() + } else if ip.To16() != nil { + ht = addrtype.IPv6 + ip = ip.To16() + } else { + return h, fmt.Errorf("could not determine client's address types: %v", err) + } + h = HostAddress{ + AddrType: ht, + Address: ip, + } + return h, nil +} + +// GetAddress returns a string representation of the HostAddress. +func (h *HostAddress) GetAddress() (string, error) { + var b []byte + _, err := asn1.Unmarshal(h.Address, &b) + return string(b), err +} + +// LocalHostAddresses returns a HostAddresses struct for the local machines interface IP addresses. +func LocalHostAddresses() (ha HostAddresses, err error) { + ifs, err := net.Interfaces() + if err != nil { + return + } + for _, iface := range ifs { + if iface.Flags&net.FlagLoopback != 0 || iface.Flags&net.FlagUp == 0 { + // Interface is either loopback of not up + continue + } + addrs, err := iface.Addrs() + if err != nil { + continue + } + for _, addr := range addrs { + var ip net.IP + switch v := addr.(type) { + case *net.IPNet: + ip = v.IP + case *net.IPAddr: + ip = v.IP + } + var a HostAddress + if ip.To16() == nil { + //neither IPv4 or IPv6 + continue + } + if ip.To4() != nil { + //Is IPv4 + a.AddrType = addrtype.IPv4 + a.Address = ip.To4() + } else { + a.AddrType = addrtype.IPv6 + a.Address = ip.To16() + } + ha = append(ha, a) + } + } + return ha, nil +} + +// HostAddressesFromNetIPs returns a HostAddresses type from a slice of net.IP +func HostAddressesFromNetIPs(ips []net.IP) (ha HostAddresses) { + for _, ip := range ips { + ha = append(ha, HostAddressFromNetIP(ip)) + } + return ha +} + +// HostAddressFromNetIP returns a HostAddress type from a net.IP +func HostAddressFromNetIP(ip net.IP) HostAddress { + if ip.To4() != nil { + //Is IPv4 + return HostAddress{ + AddrType: addrtype.IPv4, + Address: ip.To4(), + } + } + return HostAddress{ + AddrType: addrtype.IPv6, + Address: ip.To16(), + } +} + +// HostAddressesEqual tests if two HostAddress slices are equal. +func HostAddressesEqual(h, a []HostAddress) bool { + if len(h) != len(a) { + return false + } + for _, e := range a { + var found bool + for _, i := range h { + if e.Equal(i) { + found = true + break + } + } + if !found { + return false + } + } + return true +} + +// HostAddressesContains tests if a HostAddress is contained in a HostAddress slice. +func HostAddressesContains(h []HostAddress, a HostAddress) bool { + for _, e := range h { + if e.Equal(a) { + return true + } + } + return false +} + +// Equal tests if the HostAddress is equal to another HostAddress provided. +func (h *HostAddress) Equal(a HostAddress) bool { + if h.AddrType != a.AddrType { + return false + } + return bytes.Equal(h.Address, a.Address) +} + +// Contains tests if a HostAddress is contained within the HostAddresses struct. +func (h *HostAddresses) Contains(a HostAddress) bool { + for _, e := range *h { + if e.Equal(a) { + return true + } + } + return false +} + +// Equal tests if a HostAddress slice is equal to the HostAddresses struct. +func (h *HostAddresses) Equal(a []HostAddress) bool { + if len(*h) != len(a) { + return false + } + for _, e := range a { + if !h.Contains(e) { + return false + } + } + return true +} diff --git a/vendor/github.com/jcmturner/gokrb5/v8/types/KerberosFlags.go b/vendor/github.com/jcmturner/gokrb5/v8/types/KerberosFlags.go new file mode 100644 index 000000000000..0f2038340dd1 --- /dev/null +++ b/vendor/github.com/jcmturner/gokrb5/v8/types/KerberosFlags.go @@ -0,0 +1,68 @@ +package types + +// Reference: https://www.ietf.org/rfc/rfc4120.txt +// Section: 5.2.8 + +import ( + "github.com/jcmturner/gofork/encoding/asn1" +) + +// NewKrbFlags returns an ASN1 BitString struct of the right size for KrbFlags. +func NewKrbFlags() asn1.BitString { + f := asn1.BitString{} + f.Bytes = make([]byte, 4) + f.BitLength = len(f.Bytes) * 8 + return f +} + +// SetFlags sets the flags of an ASN1 BitString. +func SetFlags(f *asn1.BitString, j []int) { + for _, i := range j { + SetFlag(f, i) + } +} + +// SetFlag sets a flag in an ASN1 BitString. +func SetFlag(f *asn1.BitString, i int) { + for l := len(f.Bytes); l < 4; l++ { + (*f).Bytes = append((*f).Bytes, byte(0)) + (*f).BitLength = len((*f).Bytes) * 8 + } + //Which byte? + b := i / 8 + //Which bit in byte + p := uint(7 - (i - 8*b)) + (*f).Bytes[b] = (*f).Bytes[b] | (1 << p) +} + +// UnsetFlags unsets flags in an ASN1 BitString. +func UnsetFlags(f *asn1.BitString, j []int) { + for _, i := range j { + UnsetFlag(f, i) + } +} + +// UnsetFlag unsets a flag in an ASN1 BitString. +func UnsetFlag(f *asn1.BitString, i int) { + for l := len(f.Bytes); l < 4; l++ { + (*f).Bytes = append((*f).Bytes, byte(0)) + (*f).BitLength = len((*f).Bytes) * 8 + } + //Which byte? + b := i / 8 + //Which bit in byte + p := uint(7 - (i - 8*b)) + (*f).Bytes[b] = (*f).Bytes[b] &^ (1 << p) +} + +// IsFlagSet tests if a flag is set in the ASN1 BitString. +func IsFlagSet(f *asn1.BitString, i int) bool { + //Which byte? + b := i / 8 + //Which bit in byte + p := uint(7 - (i - 8*b)) + if (*f).Bytes[b]&(1</@ +// a PrincipalName type will be returned with the name type set to KRB_NT_PRINCIPAL(1) +// and the realm will be returned as a string. If the "@" suffix +// is not included in the SPN then the value of realm string returned will be "" +func ParseSPNString(spn string) (pn PrincipalName, realm string) { + if strings.Contains(spn, "@") { + s := strings.Split(spn, "@") + realm = s[len(s)-1] + spn = strings.TrimSuffix(spn, "@"+realm) + } + pn = NewPrincipalName(nametype.KRB_NT_PRINCIPAL, spn) + return +} diff --git a/vendor/github.com/jcmturner/gokrb5/v8/types/TypedData.go b/vendor/github.com/jcmturner/gokrb5/v8/types/TypedData.go new file mode 100644 index 000000000000..19e9f4961f4e --- /dev/null +++ b/vendor/github.com/jcmturner/gokrb5/v8/types/TypedData.go @@ -0,0 +1,18 @@ +package types + +import "github.com/jcmturner/gofork/encoding/asn1" + +// TypedData implements RFC 4120 type: https://tools.ietf.org/html/rfc4120#section-5.9.1 +type TypedData struct { + DataType int32 `asn1:"explicit,tag:0"` + DataValue []byte `asn1:"optional,explicit,tag:1"` +} + +// TypedDataSequence implements RFC 4120 type: https://tools.ietf.org/html/rfc4120#section-5.9.1 +type TypedDataSequence []TypedData + +// Unmarshal bytes into the TypedDataSequence. +func (a *TypedDataSequence) Unmarshal(b []byte) error { + _, err := asn1.Unmarshal(b, a) + return err +} diff --git a/vendor/github.com/jcmturner/rpc/v2/LICENSE b/vendor/github.com/jcmturner/rpc/v2/LICENSE new file mode 100644 index 000000000000..261eeb9e9f8b --- /dev/null +++ b/vendor/github.com/jcmturner/rpc/v2/LICENSE @@ -0,0 +1,201 @@ + 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. + + END OF TERMS AND CONDITIONS + + APPENDIX: How to apply the Apache License to your work. + + To apply the Apache License to your work, attach the following + boilerplate notice, with the fields enclosed by brackets "[]" + replaced with your own identifying information. (Don't include + the brackets!) The text should be enclosed in the appropriate + comment syntax for the file format. We also recommend that a + file or class name and description of purpose be included on the + same "printed page" as the copyright notice for easier + identification within third-party archives. + + Copyright [yyyy] [name of copyright owner] + + 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. diff --git a/vendor/github.com/jcmturner/rpc/v2/mstypes/claims.go b/vendor/github.com/jcmturner/rpc/v2/mstypes/claims.go new file mode 100644 index 000000000000..b9f535f5b97a --- /dev/null +++ b/vendor/github.com/jcmturner/rpc/v2/mstypes/claims.go @@ -0,0 +1,152 @@ +package mstypes + +import ( + "bytes" + "encoding/hex" + "errors" + "fmt" + + "github.com/jcmturner/rpc/v2/ndr" + "golang.org/x/net/http2/hpack" +) + +// Compression format assigned numbers. https://docs.microsoft.com/en-us/openspecs/windows_protocols/ms-xca/a8b7cb0a-92a6-4187-a23b-5e14273b96f8 +const ( + CompressionFormatNone uint16 = 0 + CompressionFormatLZNT1 uint16 = 2 // LZNT1 aka ntfs compression + CompressionFormatXPress uint16 = 3 // plain LZ77 + CompressionFormatXPressHuff uint16 = 4 // LZ77+Huffman - The Huffman variant of the XPRESS compression format uses LZ77-style dictionary compression combined with Huffman coding. +) + +// ClaimsSourceTypeAD https://msdn.microsoft.com/en-us/library/hh553809.aspx +const ClaimsSourceTypeAD uint16 = 1 + +// Claim Type assigned numbers +const ( + ClaimTypeIDInt64 uint16 = 1 + ClaimTypeIDUInt64 uint16 = 2 + ClaimTypeIDString uint16 = 3 + ClaimsTypeIDBoolean uint16 = 6 +) + +// ClaimsBlob implements https://msdn.microsoft.com/en-us/library/hh554119.aspx +type ClaimsBlob struct { + Size uint32 + EncodedBlob EncodedBlob +} + +// EncodedBlob are the bytes of the encoded Claims +type EncodedBlob []byte + +// Size returns the size of the bytes of the encoded Claims +func (b EncodedBlob) Size(c interface{}) int { + cb := c.(ClaimsBlob) + return int(cb.Size) +} + +// ClaimsSetMetadata implements https://msdn.microsoft.com/en-us/library/hh554073.aspx +type ClaimsSetMetadata struct { + ClaimsSetSize uint32 + ClaimsSetBytes []byte `ndr:"pointer,conformant"` + CompressionFormat uint16 // Enum see constants for options + UncompressedClaimsSetSize uint32 + ReservedType uint16 + ReservedFieldSize uint32 + ReservedField []byte `ndr:"pointer,conformant"` +} + +// ClaimsSet reads the ClaimsSet type from the NDR encoded ClaimsSetBytes in the ClaimsSetMetadata +func (m *ClaimsSetMetadata) ClaimsSet() (c ClaimsSet, err error) { + if len(m.ClaimsSetBytes) < 1 { + err = errors.New("no bytes available for ClaimsSet") + return + } + // TODO switch statement to decompress ClaimsSetBytes + switch m.CompressionFormat { + case CompressionFormatLZNT1: + s := hex.EncodeToString(m.ClaimsSetBytes) + err = fmt.Errorf("ClaimsSet compressed, format LZNT1 not currently supported: %s", s) + return + case CompressionFormatXPress: + s := hex.EncodeToString(m.ClaimsSetBytes) + err = fmt.Errorf("ClaimsSet compressed, format XPress not currently supported: %s", s) + return + case CompressionFormatXPressHuff: + var b []byte + buff := bytes.NewBuffer(b) + _, e := hpack.HuffmanDecode(buff, m.ClaimsSetBytes) + if e != nil { + err = fmt.Errorf("error deflating: %v", e) + return + } + m.ClaimsSetBytes = buff.Bytes() + } + dec := ndr.NewDecoder(bytes.NewReader(m.ClaimsSetBytes)) + err = dec.Decode(&c) + return +} + +// ClaimsSet implements https://msdn.microsoft.com/en-us/library/hh554122.aspx +type ClaimsSet struct { + ClaimsArrayCount uint32 + ClaimsArrays []ClaimsArray `ndr:"pointer,conformant"` + ReservedType uint16 + ReservedFieldSize uint32 + ReservedField []byte `ndr:"pointer,conformant"` +} + +// ClaimsArray implements https://msdn.microsoft.com/en-us/library/hh536458.aspx +type ClaimsArray struct { + ClaimsSourceType uint16 + ClaimsCount uint32 + ClaimEntries []ClaimEntry `ndr:"pointer,conformant"` +} + +// ClaimEntry is a NDR union that implements https://msdn.microsoft.com/en-us/library/hh536374.aspx +type ClaimEntry struct { + ID string `ndr:"pointer,conformant,varying"` + Type uint16 `ndr:"unionTag"` + TypeInt64 ClaimTypeInt64 `ndr:"unionField"` + TypeUInt64 ClaimTypeUInt64 `ndr:"unionField"` + TypeString ClaimTypeString `ndr:"unionField"` + TypeBool ClaimTypeBoolean `ndr:"unionField"` +} + +// SwitchFunc is the ClaimEntry union field selection function +func (u ClaimEntry) SwitchFunc(_ interface{}) string { + switch u.Type { + case ClaimTypeIDInt64: + return "TypeInt64" + case ClaimTypeIDUInt64: + return "TypeUInt64" + case ClaimTypeIDString: + return "TypeString" + case ClaimsTypeIDBoolean: + return "TypeBool" + } + return "" +} + +// ClaimTypeInt64 is a claim of type int64 +type ClaimTypeInt64 struct { + ValueCount uint32 + Value []int64 `ndr:"pointer,conformant"` +} + +// ClaimTypeUInt64 is a claim of type uint64 +type ClaimTypeUInt64 struct { + ValueCount uint32 + Value []uint64 `ndr:"pointer,conformant"` +} + +// ClaimTypeString is a claim of type string +type ClaimTypeString struct { + ValueCount uint32 + Value []LPWSTR `ndr:"pointer,conformant"` +} + +// ClaimTypeBoolean is a claim of type bool +type ClaimTypeBoolean struct { + ValueCount uint32 + Value []bool `ndr:"pointer,conformant"` +} diff --git a/vendor/github.com/jcmturner/rpc/v2/mstypes/common.go b/vendor/github.com/jcmturner/rpc/v2/mstypes/common.go new file mode 100644 index 000000000000..fb6510d1cbc3 --- /dev/null +++ b/vendor/github.com/jcmturner/rpc/v2/mstypes/common.go @@ -0,0 +1,12 @@ +// Package mstypes provides implemnations of some Microsoft data types [MS-DTYP] https://msdn.microsoft.com/en-us/library/cc230283.aspx +package mstypes + +// LPWSTR implements https://msdn.microsoft.com/en-us/library/cc230355.aspx +type LPWSTR struct { + Value string `ndr:"pointer,conformant,varying"` +} + +// String returns the string representation of LPWSTR data type. +func (s *LPWSTR) String() string { + return s.Value +} diff --git a/vendor/github.com/jcmturner/rpc/v2/mstypes/filetime.go b/vendor/github.com/jcmturner/rpc/v2/mstypes/filetime.go new file mode 100644 index 000000000000..5cc952fa9d11 --- /dev/null +++ b/vendor/github.com/jcmturner/rpc/v2/mstypes/filetime.go @@ -0,0 +1,52 @@ +// Package mstypes implements representations of Microsoft types +package mstypes + +import ( + "time" +) + +/* +FILETIME is a windows data structure. +Ref: https://msdn.microsoft.com/en-us/library/windows/desktop/ms724284%28v=vs.85%29.aspx +It contains two parts that are 32bit integers: + dwLowDateTime + dwHighDateTime +We need to combine these two into one 64bit integer. +This gives the number of 100 nano second period from January 1, 1601, Coordinated Universal Time (UTC) +*/ + +const unixEpochDiff = 116444736000000000 + +// FileTime implements the Microsoft FILETIME type https://msdn.microsoft.com/en-us/library/cc230324.aspx +type FileTime struct { + LowDateTime uint32 + HighDateTime uint32 +} + +// Time return a golang Time type from the FileTime +func (ft FileTime) Time() time.Time { + ns := (ft.MSEpoch() - unixEpochDiff) * 100 + return time.Unix(0, int64(ns)).UTC() +} + +// MSEpoch returns the FileTime as a Microsoft epoch, the number of 100 nano second periods elapsed from January 1, 1601 UTC. +func (ft FileTime) MSEpoch() int64 { + return (int64(ft.HighDateTime) << 32) + int64(ft.LowDateTime) +} + +// Unix returns the FileTime as a Unix time, the number of seconds elapsed since January 1, 1970 UTC. +func (ft FileTime) Unix() int64 { + return (ft.MSEpoch() - unixEpochDiff) / 10000000 +} + +// GetFileTime returns a FileTime type from the provided Golang Time type. +func GetFileTime(t time.Time) FileTime { + ns := t.UnixNano() + fp := (ns / 100) + unixEpochDiff + hd := fp >> 32 + ld := fp - (hd << 32) + return FileTime{ + LowDateTime: uint32(ld), + HighDateTime: uint32(hd), + } +} diff --git a/vendor/github.com/jcmturner/rpc/v2/mstypes/group_membership.go b/vendor/github.com/jcmturner/rpc/v2/mstypes/group_membership.go new file mode 100644 index 000000000000..79151378fcb4 --- /dev/null +++ b/vendor/github.com/jcmturner/rpc/v2/mstypes/group_membership.go @@ -0,0 +1,19 @@ +package mstypes + +// GroupMembership implements https://msdn.microsoft.com/en-us/library/cc237945.aspx +// RelativeID : A 32-bit unsigned integer that contains the RID of a particular group. +// The possible values for the Attributes flags are identical to those specified in KERB_SID_AND_ATTRIBUTES +type GroupMembership struct { + RelativeID uint32 + Attributes uint32 +} + +// DomainGroupMembership implements https://msdn.microsoft.com/en-us/library/hh536344.aspx +// DomainId: A SID structure that contains the SID for the domain.This member is used in conjunction with the GroupIds members to create group SIDs for the device. +// GroupCount: A 32-bit unsigned integer that contains the number of groups within the domain to which the account belongs. +// GroupIds: A pointer to a list of GROUP_MEMBERSHIP structures that contain the groups to which the account belongs in the domain. The number of groups in this list MUST be equal to GroupCount. +type DomainGroupMembership struct { + DomainID RPCSID `ndr:"pointer"` + GroupCount uint32 + GroupIDs []GroupMembership `ndr:"pointer,conformant"` // Size is value of GroupCount +} diff --git a/vendor/github.com/jcmturner/rpc/v2/mstypes/kerb_sid_and_attributes.go b/vendor/github.com/jcmturner/rpc/v2/mstypes/kerb_sid_and_attributes.go new file mode 100644 index 000000000000..61ac39bbf36f --- /dev/null +++ b/vendor/github.com/jcmturner/rpc/v2/mstypes/kerb_sid_and_attributes.go @@ -0,0 +1,23 @@ +package mstypes + +// Attributes of a security group membership and can be combined by using the bitwise OR operation. +// They are used by an access check mechanism to specify whether the membership is to be used in an access check decision. +const ( + SEGroupMandatory = 31 + SEGroupEnabledByDefault = 30 + SEGroupEnabled = 29 + SEGroupOwner = 28 + SEGroupResource = 2 + //All other bits MUST be set to zero and MUST be ignored on receipt. +) + +// KerbSidAndAttributes implements https://msdn.microsoft.com/en-us/library/cc237947.aspx +type KerbSidAndAttributes struct { + SID RPCSID `ndr:"pointer"` // A pointer to an RPC_SID structure. + Attributes uint32 +} + +// SetFlag sets a flag in a uint32 attribute value. +func SetFlag(a *uint32, i uint) { + *a = *a | (1 << (31 - i)) +} diff --git a/vendor/github.com/jcmturner/rpc/v2/mstypes/reader.go b/vendor/github.com/jcmturner/rpc/v2/mstypes/reader.go new file mode 100644 index 000000000000..24495bca0a97 --- /dev/null +++ b/vendor/github.com/jcmturner/rpc/v2/mstypes/reader.go @@ -0,0 +1,109 @@ +package mstypes + +import ( + "bufio" + "encoding/binary" + "fmt" + "io" +) + +// Byte sizes of primitive types +const ( + SizeBool = 1 + SizeChar = 1 + SizeUint8 = 1 + SizeUint16 = 2 + SizeUint32 = 4 + SizeUint64 = 8 + SizeEnum = 2 + SizeSingle = 4 + SizeDouble = 8 + SizePtr = 4 +) + +// Reader reads simple byte stream data into a Go representations +type Reader struct { + r *bufio.Reader // source of the data +} + +// NewReader creates a new instance of a simple Reader. +func NewReader(r io.Reader) *Reader { + reader := new(Reader) + reader.r = bufio.NewReader(r) + return reader +} + +func (r *Reader) Read(p []byte) (n int, err error) { + return r.r.Read(p) +} + +func (r *Reader) Uint8() (uint8, error) { + b, err := r.r.ReadByte() + if err != nil { + return uint8(0), err + } + return uint8(b), nil +} + +func (r *Reader) Uint16() (uint16, error) { + b, err := r.ReadBytes(SizeUint16) + if err != nil { + return uint16(0), err + } + return binary.LittleEndian.Uint16(b), nil +} + +func (r *Reader) Uint32() (uint32, error) { + b, err := r.ReadBytes(SizeUint32) + if err != nil { + return uint32(0), err + } + return binary.LittleEndian.Uint32(b), nil +} + +func (r *Reader) Uint64() (uint64, error) { + b, err := r.ReadBytes(SizeUint64) + if err != nil { + return uint64(0), err + } + return binary.LittleEndian.Uint64(b), nil +} + +func (r *Reader) FileTime() (f FileTime, err error) { + f.LowDateTime, err = r.Uint32() + if err != nil { + return + } + f.HighDateTime, err = r.Uint32() + if err != nil { + return + } + return +} + +// UTF16String returns a string that is UTF16 encoded in a byte slice. n is the number of bytes representing the string +func (r *Reader) UTF16String(n int) (str string, err error) { + //Length divided by 2 as each run is 16bits = 2bytes + s := make([]rune, n/2, n/2) + for i := 0; i < len(s); i++ { + var u uint16 + u, err = r.Uint16() + if err != nil { + return + } + s[i] = rune(u) + } + str = string(s) + return +} + +// readBytes returns a number of bytes from the NDR byte stream. +func (r *Reader) ReadBytes(n int) ([]byte, error) { + //TODO make this take an int64 as input to allow for larger values on all systems? + b := make([]byte, n, n) + m, err := r.r.Read(b) + if err != nil || m != n { + return b, fmt.Errorf("error reading bytes from stream: %v", err) + } + return b, nil +} diff --git a/vendor/github.com/jcmturner/rpc/v2/mstypes/rpc_unicode_string.go b/vendor/github.com/jcmturner/rpc/v2/mstypes/rpc_unicode_string.go new file mode 100644 index 000000000000..4bf02e0ed689 --- /dev/null +++ b/vendor/github.com/jcmturner/rpc/v2/mstypes/rpc_unicode_string.go @@ -0,0 +1,13 @@ +package mstypes + +// RPCUnicodeString implements https://msdn.microsoft.com/en-us/library/cc230365.aspx +type RPCUnicodeString struct { + Length uint16 // The length, in bytes, of the string pointed to by the Buffer member, not including the terminating null character if any. The length MUST be a multiple of 2. The length SHOULD equal the entire size of the Buffer, in which case there is no terminating null character. Any method that accesses this structure MUST use the Length specified instead of relying on the presence or absence of a null character. + MaximumLength uint16 // The maximum size, in bytes, of the string pointed to by Buffer. The size MUST be a multiple of 2. If not, the size MUST be decremented by 1 prior to use. This value MUST not be less than Length. + Value string `ndr:"pointer,conformant,varying"` +} + +// String returns the RPCUnicodeString string value +func (r *RPCUnicodeString) String() string { + return r.Value +} diff --git a/vendor/github.com/jcmturner/rpc/v2/mstypes/sid.go b/vendor/github.com/jcmturner/rpc/v2/mstypes/sid.go new file mode 100644 index 000000000000..8e347058c006 --- /dev/null +++ b/vendor/github.com/jcmturner/rpc/v2/mstypes/sid.go @@ -0,0 +1,36 @@ +package mstypes + +import ( + "encoding/binary" + "encoding/hex" + "fmt" + "math" + "strings" +) + +// RPCSID implements https://msdn.microsoft.com/en-us/library/cc230364.aspx +type RPCSID struct { + Revision uint8 // An 8-bit unsigned integer that specifies the revision level of the SID. This value MUST be set to 0x01. + SubAuthorityCount uint8 // An 8-bit unsigned integer that specifies the number of elements in the SubAuthority array. The maximum number of elements allowed is 15. + IdentifierAuthority [6]byte // An RPC_SID_IDENTIFIER_AUTHORITY structure that indicates the authority under which the SID was created. It describes the entity that created the SID. The Identifier Authority value {0,0,0,0,0,5} denotes SIDs created by the NT SID authority. + SubAuthority []uint32 `ndr:"conformant"` // A variable length array of unsigned 32-bit integers that uniquely identifies a principal relative to the IdentifierAuthority. Its length is determined by SubAuthorityCount. +} + +// String returns the string representation of the RPC_SID. +func (s *RPCSID) String() string { + var strb strings.Builder + strb.WriteString("S-1-") + + b := append(make([]byte, 2, 2), s.IdentifierAuthority[:]...) + // For a strange reason this is read big endian: https://msdn.microsoft.com/en-us/library/dd302645.aspx + i := binary.BigEndian.Uint64(b) + if i > math.MaxUint32 { + fmt.Fprintf(&strb, "0x%s", hex.EncodeToString(s.IdentifierAuthority[:])) + } else { + fmt.Fprintf(&strb, "%d", i) + } + for _, sub := range s.SubAuthority { + fmt.Fprintf(&strb, "-%d", sub) + } + return strb.String() +} diff --git a/vendor/github.com/jcmturner/rpc/v2/mstypes/user_session_key.go b/vendor/github.com/jcmturner/rpc/v2/mstypes/user_session_key.go new file mode 100644 index 000000000000..fcf0a5d9a66e --- /dev/null +++ b/vendor/github.com/jcmturner/rpc/v2/mstypes/user_session_key.go @@ -0,0 +1,11 @@ +package mstypes + +// CypherBlock implements https://msdn.microsoft.com/en-us/library/cc237040.aspx +type CypherBlock struct { + Data [8]byte // size = 8 +} + +// UserSessionKey implements https://msdn.microsoft.com/en-us/library/cc237080.aspx +type UserSessionKey struct { + CypherBlock [2]CypherBlock // size = 2 +} diff --git a/vendor/github.com/jcmturner/rpc/v2/ndr/arrays.go b/vendor/github.com/jcmturner/rpc/v2/ndr/arrays.go new file mode 100644 index 000000000000..5e2def2a8298 --- /dev/null +++ b/vendor/github.com/jcmturner/rpc/v2/ndr/arrays.go @@ -0,0 +1,413 @@ +package ndr + +import ( + "errors" + "fmt" + "reflect" + "strconv" +) + +// intFromTag returns an int that is a value in a struct tag key/value pair +func intFromTag(tag reflect.StructTag, key string) (int, error) { + ndrTag := parseTags(tag) + d := 1 + if n, ok := ndrTag.Map[key]; ok { + i, err := strconv.Atoi(n) + if err != nil { + return d, fmt.Errorf("invalid dimensions tag [%s]: %v", n, err) + } + d = i + } + return d, nil +} + +// parseDimensions returns the a slice of the size of each dimension and type of the member at the deepest level. +func parseDimensions(v reflect.Value) (l []int, tb reflect.Type) { + if v.Kind() == reflect.Ptr { + v = v.Elem() + } + t := v.Type() + if t.Kind() == reflect.Ptr { + t = t.Elem() + } + if t.Kind() != reflect.Array && t.Kind() != reflect.Slice { + return + } + l = append(l, v.Len()) + if t.Elem().Kind() == reflect.Array || t.Elem().Kind() == reflect.Slice { + // contains array or slice + var m []int + m, tb = parseDimensions(v.Index(0)) + l = append(l, m...) + } else { + tb = t.Elem() + } + return +} + +// sliceDimensions returns the count of dimensions a slice has. +func sliceDimensions(t reflect.Type) (d int, tb reflect.Type) { + if t.Kind() == reflect.Ptr { + t = t.Elem() + } + if t.Kind() == reflect.Slice { + d++ + var n int + n, tb = sliceDimensions(t.Elem()) + d += n + } else { + tb = t + } + return +} + +// makeSubSlices is a deep recursive creation/initialisation of multi-dimensional slices. +// Takes the reflect.Value of the 1st dimension and a slice of the lengths of the sub dimensions +func makeSubSlices(v reflect.Value, l []int) { + ty := v.Type().Elem() + if ty.Kind() != reflect.Slice { + return + } + for i := 0; i < v.Len(); i++ { + s := reflect.MakeSlice(ty, l[0], l[0]) + v.Index(i).Set(s) + // Are there more sub dimensions? + if len(l) > 1 { + makeSubSlices(v.Index(i), l[1:]) + } + } + return +} + +// multiDimensionalIndexPermutations returns all the permutations of the indexes of a multi-dimensional slice. +// The input is a slice of integers that indicates the max size/length of each dimension +func multiDimensionalIndexPermutations(l []int) (ps [][]int) { + z := make([]int, len(l), len(l)) // The zeros permutation + ps = append(ps, z) + // for each dimension, in reverse + for i := len(l) - 1; i >= 0; i-- { + ws := make([][]int, len(ps)) + copy(ws, ps) + //create a permutation for each of the iterations of the current dimension + for j := 1; j <= l[i]-1; j++ { + // For each existing permutation + for _, p := range ws { + np := make([]int, len(p), len(p)) + copy(np, p) + np[i] = j + ps = append(ps, np) + } + } + } + return +} + +// precedingMax reads off the next conformant max value +func (dec *Decoder) precedingMax() uint32 { + m := dec.conformantMax[0] + dec.conformantMax = dec.conformantMax[1:] + return m +} + +// fillFixedArray establishes if the fixed array is uni or multi dimensional and then fills it. +func (dec *Decoder) fillFixedArray(v reflect.Value, tag reflect.StructTag, def *[]deferedPtr) error { + l, t := parseDimensions(v) + if t.Kind() == reflect.String { + tag = reflect.StructTag(subStringArrayTag) + } + if len(l) < 1 { + return errors.New("could not establish dimensions of fixed array") + } + if len(l) == 1 { + err := dec.fillUniDimensionalFixedArray(v, tag, def) + if err != nil { + return fmt.Errorf("could not fill uni-dimensional fixed array: %v", err) + } + return nil + } + // Fixed array is multidimensional + ps := multiDimensionalIndexPermutations(l[:len(l)-1]) + for _, p := range ps { + // Get current multi-dimensional index to fill + a := v + for _, i := range p { + a = a.Index(i) + } + // fill with the last dimension array + err := dec.fillUniDimensionalFixedArray(a, tag, def) + if err != nil { + return fmt.Errorf("could not fill dimension %v of multi-dimensional fixed array: %v", p, err) + } + } + return nil +} + +// readUniDimensionalFixedArray reads an array (not slice) from the byte stream. +func (dec *Decoder) fillUniDimensionalFixedArray(v reflect.Value, tag reflect.StructTag, def *[]deferedPtr) error { + for i := 0; i < v.Len(); i++ { + err := dec.fill(v.Index(i), tag, def) + if err != nil { + return fmt.Errorf("could not fill index %d of fixed array: %v", i, err) + } + } + return nil +} + +// fillConformantArray establishes if the conformant array is uni or multi dimensional and then fills the slice. +func (dec *Decoder) fillConformantArray(v reflect.Value, tag reflect.StructTag, def *[]deferedPtr) error { + d, _ := sliceDimensions(v.Type()) + if d > 1 { + err := dec.fillMultiDimensionalConformantArray(v, d, tag, def) + if err != nil { + return err + } + } else { + err := dec.fillUniDimensionalConformantArray(v, tag, def) + if err != nil { + return err + } + } + return nil +} + +// fillUniDimensionalConformantArray fills the uni-dimensional slice value. +func (dec *Decoder) fillUniDimensionalConformantArray(v reflect.Value, tag reflect.StructTag, def *[]deferedPtr) error { + m := dec.precedingMax() + n := int(m) + a := reflect.MakeSlice(v.Type(), n, n) + for i := 0; i < n; i++ { + err := dec.fill(a.Index(i), tag, def) + if err != nil { + return fmt.Errorf("could not fill index %d of uni-dimensional conformant array: %v", i, err) + } + } + v.Set(a) + return nil +} + +// fillMultiDimensionalConformantArray fills the multi-dimensional slice value provided from conformant array data. +// The number of dimensions must be specified. This must be less than or equal to the dimensions in the slice for this +// method not to panic. +func (dec *Decoder) fillMultiDimensionalConformantArray(v reflect.Value, d int, tag reflect.StructTag, def *[]deferedPtr) error { + // Read the max size of each dimensions from the ndr stream + l := make([]int, d, d) + for i := range l { + l[i] = int(dec.precedingMax()) + } + // Initialise size of slices + // Initialise the size of the 1st dimension + ty := v.Type() + v.Set(reflect.MakeSlice(ty, l[0], l[0])) + // Initialise the size of the other dimensions recursively + makeSubSlices(v, l[1:]) + + // Get all permutations of the indexes and go through each and fill + ps := multiDimensionalIndexPermutations(l) + for _, p := range ps { + // Get current multi-dimensional index to fill + a := v + for _, i := range p { + a = a.Index(i) + } + err := dec.fill(a, tag, def) + if err != nil { + return fmt.Errorf("could not fill index %v of slice: %v", p, err) + } + } + return nil +} + +// fillVaryingArray establishes if the varying array is uni or multi dimensional and then fills the slice. +func (dec *Decoder) fillVaryingArray(v reflect.Value, tag reflect.StructTag, def *[]deferedPtr) error { + d, t := sliceDimensions(v.Type()) + if d > 1 { + err := dec.fillMultiDimensionalVaryingArray(v, t, d, tag, def) + if err != nil { + return err + } + } else { + err := dec.fillUniDimensionalVaryingArray(v, tag, def) + if err != nil { + return err + } + } + return nil +} + +// fillUniDimensionalVaryingArray fills the uni-dimensional slice value. +func (dec *Decoder) fillUniDimensionalVaryingArray(v reflect.Value, tag reflect.StructTag, def *[]deferedPtr) error { + o, err := dec.readUint32() + if err != nil { + return fmt.Errorf("could not read offset of uni-dimensional varying array: %v", err) + } + s, err := dec.readUint32() + if err != nil { + return fmt.Errorf("could not establish actual count of uni-dimensional varying array: %v", err) + } + t := v.Type() + // Total size of the array is the offset in the index being passed plus the actual count of elements being passed. + n := int(s + o) + a := reflect.MakeSlice(t, n, n) + // Populate the array starting at the offset specified + for i := int(o); i < n; i++ { + err := dec.fill(a.Index(i), tag, def) + if err != nil { + return fmt.Errorf("could not fill index %d of uni-dimensional varying array: %v", i, err) + } + } + v.Set(a) + return nil +} + +// fillMultiDimensionalVaryingArray fills the multi-dimensional slice value provided from varying array data. +// The number of dimensions must be specified. This must be less than or equal to the dimensions in the slice for this +// method not to panic. +func (dec *Decoder) fillMultiDimensionalVaryingArray(v reflect.Value, t reflect.Type, d int, tag reflect.StructTag, def *[]deferedPtr) error { + // Read the offset and actual count of each dimensions from the ndr stream + o := make([]int, d, d) + l := make([]int, d, d) + for i := range l { + off, err := dec.readUint32() + if err != nil { + return fmt.Errorf("could not read offset of dimension %d: %v", i+1, err) + } + o[i] = int(off) + s, err := dec.readUint32() + if err != nil { + return fmt.Errorf("could not read size of dimension %d: %v", i+1, err) + } + l[i] = int(s) + int(off) + } + // Initialise size of slices + // Initialise the size of the 1st dimension + ty := v.Type() + v.Set(reflect.MakeSlice(ty, l[0], l[0])) + // Initialise the size of the other dimensions recursively + makeSubSlices(v, l[1:]) + + // Get all permutations of the indexes and go through each and fill + ps := multiDimensionalIndexPermutations(l) + for _, p := range ps { + // Get current multi-dimensional index to fill + a := v + var os bool // should this permutation be skipped due to the offset of any of the dimensions? + for i, j := range p { + if j < o[i] { + os = true + break + } + a = a.Index(j) + } + if os { + // This permutation should be skipped as it is less than the offset for one of the dimensions. + continue + } + err := dec.fill(a, tag, def) + if err != nil { + return fmt.Errorf("could not fill index %v of slice: %v", p, err) + } + } + return nil +} + +// fillConformantVaryingArray establishes if the varying array is uni or multi dimensional and then fills the slice. +func (dec *Decoder) fillConformantVaryingArray(v reflect.Value, tag reflect.StructTag, def *[]deferedPtr) error { + d, t := sliceDimensions(v.Type()) + if d > 1 { + err := dec.fillMultiDimensionalConformantVaryingArray(v, t, d, tag, def) + if err != nil { + return err + } + } else { + err := dec.fillUniDimensionalConformantVaryingArray(v, tag, def) + if err != nil { + return err + } + } + return nil +} + +// fillUniDimensionalConformantVaryingArray fills the uni-dimensional slice value. +func (dec *Decoder) fillUniDimensionalConformantVaryingArray(v reflect.Value, tag reflect.StructTag, def *[]deferedPtr) error { + m := dec.precedingMax() + o, err := dec.readUint32() + if err != nil { + return fmt.Errorf("could not read offset of uni-dimensional conformant varying array: %v", err) + } + s, err := dec.readUint32() + if err != nil { + return fmt.Errorf("could not establish actual count of uni-dimensional conformant varying array: %v", err) + } + if m < o+s { + return errors.New("max count is less than the offset plus actual count") + } + t := v.Type() + n := int(s) + a := reflect.MakeSlice(t, n, n) + for i := int(o); i < n; i++ { + err := dec.fill(a.Index(i), tag, def) + if err != nil { + return fmt.Errorf("could not fill index %d of uni-dimensional conformant varying array: %v", i, err) + } + } + v.Set(a) + return nil +} + +// fillMultiDimensionalConformantVaryingArray fills the multi-dimensional slice value provided from conformant varying array data. +// The number of dimensions must be specified. This must be less than or equal to the dimensions in the slice for this +// method not to panic. +func (dec *Decoder) fillMultiDimensionalConformantVaryingArray(v reflect.Value, t reflect.Type, d int, tag reflect.StructTag, def *[]deferedPtr) error { + // Read the offset and actual count of each dimensions from the ndr stream + m := make([]int, d, d) + for i := range m { + m[i] = int(dec.precedingMax()) + } + o := make([]int, d, d) + l := make([]int, d, d) + for i := range l { + off, err := dec.readUint32() + if err != nil { + return fmt.Errorf("could not read offset of dimension %d: %v", i+1, err) + } + o[i] = int(off) + s, err := dec.readUint32() + if err != nil { + return fmt.Errorf("could not read actual count of dimension %d: %v", i+1, err) + } + if m[i] < int(s)+int(off) { + m[i] = int(s) + int(off) + } + l[i] = int(s) + } + // Initialise size of slices + // Initialise the size of the 1st dimension + ty := v.Type() + v.Set(reflect.MakeSlice(ty, m[0], m[0])) + // Initialise the size of the other dimensions recursively + makeSubSlices(v, m[1:]) + + // Get all permutations of the indexes and go through each and fill + ps := multiDimensionalIndexPermutations(m) + for _, p := range ps { + // Get current multi-dimensional index to fill + a := v + var os bool // should this permutation be skipped due to the offset of any of the dimensions or max is higher than the actual count being passed + for i, j := range p { + if j < o[i] || j >= l[i] { + os = true + break + } + a = a.Index(j) + } + if os { + // This permutation should be skipped as it is less than the offset for one of the dimensions. + continue + } + err := dec.fill(a, tag, def) + if err != nil { + return fmt.Errorf("could not fill index %v of slice: %v", p, err) + } + } + return nil +} diff --git a/vendor/github.com/jcmturner/rpc/v2/ndr/decoder.go b/vendor/github.com/jcmturner/rpc/v2/ndr/decoder.go new file mode 100644 index 000000000000..6157b4ef9e29 --- /dev/null +++ b/vendor/github.com/jcmturner/rpc/v2/ndr/decoder.go @@ -0,0 +1,393 @@ +// Package ndr provides the ability to unmarshal NDR encoded byte steams into Go data structures +package ndr + +import ( + "bufio" + "fmt" + "io" + "reflect" + "strings" +) + +// Struct tag values +const ( + TagConformant = "conformant" + TagVarying = "varying" + TagPointer = "pointer" + TagPipe = "pipe" +) + +// Decoder unmarshals NDR byte stream data into a Go struct representation +type Decoder struct { + r *bufio.Reader // source of the data + size int // initial size of bytes in buffer + ch CommonHeader // NDR common header + ph PrivateHeader // NDR private header + conformantMax []uint32 // conformant max values that were moved to the beginning of the structure + s interface{} // pointer to the structure being populated + current []string // keeps track of the current field being populated +} + +type deferedPtr struct { + v reflect.Value + tag reflect.StructTag +} + +// NewDecoder creates a new instance of a NDR Decoder. +func NewDecoder(r io.Reader) *Decoder { + dec := new(Decoder) + dec.r = bufio.NewReader(r) + dec.r.Peek(int(commonHeaderBytes)) // For some reason an operation is needed on the buffer to initialise it so Buffered() != 0 + dec.size = dec.r.Buffered() + return dec +} + +// Decode unmarshals the NDR encoded bytes into the pointer of a struct provided. +func (dec *Decoder) Decode(s interface{}) error { + dec.s = s + err := dec.readCommonHeader() + if err != nil { + return err + } + err = dec.readPrivateHeader() + if err != nil { + return err + } + _, err = dec.r.Discard(4) //The next 4 bytes are an RPC unique pointer referent. We just skip these. + if err != nil { + return Errorf("unable to process byte stream: %v", err) + } + + return dec.process(s, reflect.StructTag("")) +} + +func (dec *Decoder) process(s interface{}, tag reflect.StructTag) error { + // Scan for conformant fields as their max counts are moved to the beginning + // http://pubs.opengroup.org/onlinepubs/9629399/chap14.htm#tagfcjh_37 + err := dec.scanConformantArrays(s, tag) + if err != nil { + return err + } + // Recursively fill the struct fields + var localDef []deferedPtr + err = dec.fill(s, tag, &localDef) + if err != nil { + return Errorf("could not decode: %v", err) + } + // Read any deferred referents associated with pointers + for _, p := range localDef { + err = dec.process(p.v, p.tag) + if err != nil { + return fmt.Errorf("could not decode deferred referent: %v", err) + } + } + return nil +} + +// scanConformantArrays scans the structure for embedded conformant fields and captures the maximum element counts for +// dimensions of the array that are moved to the beginning of the structure. +func (dec *Decoder) scanConformantArrays(s interface{}, tag reflect.StructTag) error { + err := dec.conformantScan(s, tag) + if err != nil { + return fmt.Errorf("failed to scan for embedded conformant arrays: %v", err) + } + for i := range dec.conformantMax { + dec.conformantMax[i], err = dec.readUint32() + if err != nil { + return fmt.Errorf("could not read preceding conformant max count index %d: %v", i, err) + } + } + return nil +} + +// conformantScan inspects the structure's fields for whether they are conformant. +func (dec *Decoder) conformantScan(s interface{}, tag reflect.StructTag) error { + ndrTag := parseTags(tag) + if ndrTag.HasValue(TagPointer) { + return nil + } + v := getReflectValue(s) + switch v.Kind() { + case reflect.Struct: + for i := 0; i < v.NumField(); i++ { + err := dec.conformantScan(v.Field(i), v.Type().Field(i).Tag) + if err != nil { + return err + } + } + case reflect.String: + if !ndrTag.HasValue(TagConformant) { + break + } + dec.conformantMax = append(dec.conformantMax, uint32(0)) + case reflect.Slice: + if !ndrTag.HasValue(TagConformant) { + break + } + d, t := sliceDimensions(v.Type()) + for i := 0; i < d; i++ { + dec.conformantMax = append(dec.conformantMax, uint32(0)) + } + // For string arrays there is a common max for the strings within the array. + if t.Kind() == reflect.String { + dec.conformantMax = append(dec.conformantMax, uint32(0)) + } + } + return nil +} + +func (dec *Decoder) isPointer(v reflect.Value, tag reflect.StructTag, def *[]deferedPtr) (bool, error) { + // Pointer so defer filling the referent + ndrTag := parseTags(tag) + if ndrTag.HasValue(TagPointer) { + p, err := dec.readUint32() + if err != nil { + return true, fmt.Errorf("could not read pointer: %v", err) + } + ndrTag.delete(TagPointer) + if p != 0 { + // if pointer is not zero add to the deferred items at end of stream + *def = append(*def, deferedPtr{v, ndrTag.StructTag()}) + } + return true, nil + } + return false, nil +} + +func getReflectValue(s interface{}) (v reflect.Value) { + if r, ok := s.(reflect.Value); ok { + v = r + } else { + if reflect.ValueOf(s).Kind() == reflect.Ptr { + v = reflect.ValueOf(s).Elem() + } + } + return +} + +// fill populates fields with values from the NDR byte stream. +func (dec *Decoder) fill(s interface{}, tag reflect.StructTag, localDef *[]deferedPtr) error { + v := getReflectValue(s) + + //// Pointer so defer filling the referent + ptr, err := dec.isPointer(v, tag, localDef) + if err != nil { + return fmt.Errorf("could not process struct field(%s): %v", strings.Join(dec.current, "/"), err) + } + if ptr { + return nil + } + + // Populate the value from the byte stream + switch v.Kind() { + case reflect.Struct: + dec.current = append(dec.current, v.Type().Name()) //Track the current field being filled + // in case struct is a union, track this and the selected union field for efficiency + var unionTag reflect.Value + var unionField string // field to fill if struct is a union + // Go through each field in the struct and recursively fill + for i := 0; i < v.NumField(); i++ { + fieldName := v.Type().Field(i).Name + dec.current = append(dec.current, fieldName) //Track the current field being filled + //fmt.Fprintf(os.Stderr, "DEBUG Decoding: %s\n", strings.Join(dec.current, "/")) + structTag := v.Type().Field(i).Tag + ndrTag := parseTags(structTag) + + // Union handling + if !unionTag.IsValid() { + // Is this field a union tag? + unionTag = dec.isUnion(v.Field(i), structTag) + } else { + // What is the selected field value of the union if we don't already know + if unionField == "" { + unionField, err = unionSelectedField(v, unionTag) + if err != nil { + return fmt.Errorf("could not determine selected union value field for %s with discriminat"+ + " tag %s: %v", v.Type().Name(), unionTag, err) + } + } + if ndrTag.HasValue(TagUnionField) && fieldName != unionField { + // is a union and this field has not been selected so will skip it. + dec.current = dec.current[:len(dec.current)-1] //This field has been skipped so remove it from the current field tracker + continue + } + } + + // Check if field is a pointer + if v.Field(i).Type().Implements(reflect.TypeOf(new(RawBytes)).Elem()) && + v.Field(i).Type().Kind() == reflect.Slice && v.Field(i).Type().Elem().Kind() == reflect.Uint8 { + //field is for rawbytes + structTag, err = addSizeToTag(v, v.Field(i), structTag) + if err != nil { + return fmt.Errorf("could not get rawbytes field(%s) size: %v", strings.Join(dec.current, "/"), err) + } + ptr, err := dec.isPointer(v.Field(i), structTag, localDef) + if err != nil { + return fmt.Errorf("could not process struct field(%s): %v", strings.Join(dec.current, "/"), err) + } + if !ptr { + err := dec.readRawBytes(v.Field(i), structTag) + if err != nil { + return fmt.Errorf("could not fill raw bytes struct field(%s): %v", strings.Join(dec.current, "/"), err) + } + } + } else { + err := dec.fill(v.Field(i), structTag, localDef) + if err != nil { + return fmt.Errorf("could not fill struct field(%s): %v", strings.Join(dec.current, "/"), err) + } + } + dec.current = dec.current[:len(dec.current)-1] //This field has been filled so remove it from the current field tracker + } + dec.current = dec.current[:len(dec.current)-1] //This field has been filled so remove it from the current field tracker + case reflect.Bool: + i, err := dec.readBool() + if err != nil { + return fmt.Errorf("could not fill %s: %v", v.Type().Name(), err) + } + v.Set(reflect.ValueOf(i)) + case reflect.Uint8: + i, err := dec.readUint8() + if err != nil { + return fmt.Errorf("could not fill %s: %v", v.Type().Name(), err) + } + v.Set(reflect.ValueOf(i)) + case reflect.Uint16: + i, err := dec.readUint16() + if err != nil { + return fmt.Errorf("could not fill %s: %v", v.Type().Name(), err) + } + v.Set(reflect.ValueOf(i)) + case reflect.Uint32: + i, err := dec.readUint32() + if err != nil { + return fmt.Errorf("could not fill %s: %v", v.Type().Name(), err) + } + v.Set(reflect.ValueOf(i)) + case reflect.Uint64: + i, err := dec.readUint64() + if err != nil { + return fmt.Errorf("could not fill %s: %v", v.Type().Name(), err) + } + v.Set(reflect.ValueOf(i)) + case reflect.Int8: + i, err := dec.readInt8() + if err != nil { + return fmt.Errorf("could not fill %s: %v", v.Type().Name(), err) + } + v.Set(reflect.ValueOf(i)) + case reflect.Int16: + i, err := dec.readInt16() + if err != nil { + return fmt.Errorf("could not fill %s: %v", v.Type().Name(), err) + } + v.Set(reflect.ValueOf(i)) + case reflect.Int32: + i, err := dec.readInt32() + if err != nil { + return fmt.Errorf("could not fill %s: %v", v.Type().Name(), err) + } + v.Set(reflect.ValueOf(i)) + case reflect.Int64: + i, err := dec.readInt64() + if err != nil { + return fmt.Errorf("could not fill %s: %v", v.Type().Name(), err) + } + v.Set(reflect.ValueOf(i)) + case reflect.String: + ndrTag := parseTags(tag) + conformant := ndrTag.HasValue(TagConformant) + // strings are always varying so this is assumed without an explicit tag + var s string + var err error + if conformant { + s, err = dec.readConformantVaryingString(localDef) + if err != nil { + return fmt.Errorf("could not fill with conformant varying string: %v", err) + } + } else { + s, err = dec.readVaryingString(localDef) + if err != nil { + return fmt.Errorf("could not fill with varying string: %v", err) + } + } + v.Set(reflect.ValueOf(s)) + case reflect.Float32: + i, err := dec.readFloat32() + if err != nil { + return fmt.Errorf("could not fill %v: %v", v.Type().Name(), err) + } + v.Set(reflect.ValueOf(i)) + case reflect.Float64: + i, err := dec.readFloat64() + if err != nil { + return fmt.Errorf("could not fill %v: %v", v.Type().Name(), err) + } + v.Set(reflect.ValueOf(i)) + case reflect.Array: + err := dec.fillFixedArray(v, tag, localDef) + if err != nil { + return err + } + case reflect.Slice: + if v.Type().Implements(reflect.TypeOf(new(RawBytes)).Elem()) && v.Type().Elem().Kind() == reflect.Uint8 { + //field is for rawbytes + err := dec.readRawBytes(v, tag) + if err != nil { + return fmt.Errorf("could not fill raw bytes struct field(%s): %v", strings.Join(dec.current, "/"), err) + } + break + } + ndrTag := parseTags(tag) + conformant := ndrTag.HasValue(TagConformant) + varying := ndrTag.HasValue(TagVarying) + if ndrTag.HasValue(TagPipe) { + err := dec.fillPipe(v, tag) + if err != nil { + return err + } + break + } + _, t := sliceDimensions(v.Type()) + if t.Kind() == reflect.String && !ndrTag.HasValue(subStringArrayValue) { + // String array + err := dec.readStringsArray(v, tag, localDef) + if err != nil { + return err + } + break + } + // varying is assumed as fixed arrays use the Go array type rather than slice + if conformant && varying { + err := dec.fillConformantVaryingArray(v, tag, localDef) + if err != nil { + return err + } + } else if !conformant && varying { + err := dec.fillVaryingArray(v, tag, localDef) + if err != nil { + return err + } + } else { + //default to conformant and not varying + err := dec.fillConformantArray(v, tag, localDef) + if err != nil { + return err + } + } + default: + return fmt.Errorf("unsupported type") + } + return nil +} + +// readBytes returns a number of bytes from the NDR byte stream. +func (dec *Decoder) readBytes(n int) ([]byte, error) { + //TODO make this take an int64 as input to allow for larger values on all systems? + b := make([]byte, n, n) + m, err := dec.r.Read(b) + if err != nil || m != n { + return b, fmt.Errorf("error reading bytes from stream: %v", err) + } + return b, nil +} diff --git a/vendor/github.com/jcmturner/rpc/v2/ndr/error.go b/vendor/github.com/jcmturner/rpc/v2/ndr/error.go new file mode 100644 index 000000000000..9971194d0eca --- /dev/null +++ b/vendor/github.com/jcmturner/rpc/v2/ndr/error.go @@ -0,0 +1,18 @@ +package ndr + +import "fmt" + +// Malformed implements the error interface for malformed NDR encoding errors. +type Malformed struct { + EText string +} + +// Error implements the error interface on the Malformed struct. +func (e Malformed) Error() string { + return fmt.Sprintf("malformed NDR stream: %s", e.EText) +} + +// Errorf formats an error message into a malformed NDR error. +func Errorf(format string, a ...interface{}) Malformed { + return Malformed{EText: fmt.Sprintf(format, a...)} +} diff --git a/vendor/github.com/jcmturner/rpc/v2/ndr/header.go b/vendor/github.com/jcmturner/rpc/v2/ndr/header.go new file mode 100644 index 000000000000..1970ddb600ec --- /dev/null +++ b/vendor/github.com/jcmturner/rpc/v2/ndr/header.go @@ -0,0 +1,116 @@ +package ndr + +import ( + "encoding/binary" + "fmt" +) + +/* +Serialization Version 1 +https://msdn.microsoft.com/en-us/library/cc243563.aspx + +Common Header - https://msdn.microsoft.com/en-us/library/cc243890.aspx +8 bytes in total: +- First byte - Version: Must equal 1 +- Second byte - 1st 4 bits: Endianess (0=Big; 1=Little); 2nd 4 bits: Character Encoding (0=ASCII; 1=EBCDIC) +- 3rd - Floating point representation (This does not seem to be the case in examples for Microsoft test sources) +- 4th - Common Header Length: Must equal 8 +- 5th - 8th - Filler: MUST be set to 0xcccccccc on marshaling, and SHOULD be ignored during unmarshaling. + +Private Header - https://msdn.microsoft.com/en-us/library/cc243919.aspx +8 bytes in total: +- First 4 bytes - Indicates the length of a serialized top-level type in the octet stream. It MUST include the padding length and exclude the header itself. +- Second 4 bytes - Filler: MUST be set to 0 (zero) during marshaling, and SHOULD be ignored during unmarshaling. +*/ + +const ( + protocolVersion uint8 = 1 + commonHeaderBytes uint16 = 8 + bigEndian = 0 + littleEndian = 1 + ascii uint8 = 0 + ebcdic uint8 = 1 + ieee uint8 = 0 + vax uint8 = 1 + cray uint8 = 2 + ibm uint8 = 3 +) + +// CommonHeader implements the NDR common header: https://msdn.microsoft.com/en-us/library/cc243889.aspx +type CommonHeader struct { + Version uint8 + Endianness binary.ByteOrder + CharacterEncoding uint8 + FloatRepresentation uint8 + HeaderLength uint16 + Filler []byte +} + +// PrivateHeader implements the NDR private header: https://msdn.microsoft.com/en-us/library/cc243919.aspx +type PrivateHeader struct { + ObjectBufferLength uint32 + Filler []byte +} + +func (dec *Decoder) readCommonHeader() error { + // Version + vb, err := dec.r.ReadByte() + if err != nil { + return Malformed{EText: "could not read first byte of common header for version"} + } + dec.ch.Version = uint8(vb) + if dec.ch.Version != protocolVersion { + return Malformed{EText: fmt.Sprintf("byte stream does not indicate a RPC Type serialization of version %v", protocolVersion)} + } + // Read Endianness & Character Encoding + eb, err := dec.r.ReadByte() + if err != nil { + return Malformed{EText: "could not read second byte of common header for endianness"} + } + endian := int(eb >> 4 & 0xF) + if endian != 0 && endian != 1 { + return Malformed{EText: "common header does not indicate a valid endianness"} + } + dec.ch.CharacterEncoding = uint8(vb & 0xF) + if dec.ch.CharacterEncoding != 0 && dec.ch.CharacterEncoding != 1 { + return Malformed{EText: "common header does not indicate a valid character encoding"} + } + switch endian { + case littleEndian: + dec.ch.Endianness = binary.LittleEndian + case bigEndian: + dec.ch.Endianness = binary.BigEndian + } + // Common header length + lb, err := dec.readBytes(2) + if err != nil { + return Malformed{EText: fmt.Sprintf("could not read common header length: %v", err)} + } + dec.ch.HeaderLength = dec.ch.Endianness.Uint16(lb) + if dec.ch.HeaderLength != commonHeaderBytes { + return Malformed{EText: "common header does not indicate a valid length"} + } + // Filler bytes + dec.ch.Filler, err = dec.readBytes(4) + if err != nil { + return Malformed{EText: fmt.Sprintf("could not read common header filler: %v", err)} + } + return nil +} + +func (dec *Decoder) readPrivateHeader() error { + // The next 8 bytes after the common header comprise the RPC type marshalling private header for constructed types. + err := binary.Read(dec.r, dec.ch.Endianness, &dec.ph.ObjectBufferLength) + if err != nil { + return Malformed{EText: "could not read private header object buffer length"} + } + if dec.ph.ObjectBufferLength%8 != 0 { + return Malformed{EText: "object buffer length not a multiple of 8"} + } + // Filler bytes + dec.ph.Filler, err = dec.readBytes(4) + if err != nil { + return Malformed{EText: fmt.Sprintf("could not read private header filler: %v", err)} + } + return nil +} diff --git a/vendor/github.com/jcmturner/rpc/v2/ndr/pipe.go b/vendor/github.com/jcmturner/rpc/v2/ndr/pipe.go new file mode 100644 index 000000000000..5fd27da0192d --- /dev/null +++ b/vendor/github.com/jcmturner/rpc/v2/ndr/pipe.go @@ -0,0 +1,31 @@ +package ndr + +import ( + "fmt" + "reflect" +) + +func (dec *Decoder) fillPipe(v reflect.Value, tag reflect.StructTag) error { + s, err := dec.readUint32() // read element count of first chunk + if err != nil { + return err + } + a := reflect.MakeSlice(v.Type(), 0, 0) + c := reflect.MakeSlice(v.Type(), int(s), int(s)) + for s != 0 { + for i := 0; i < int(s); i++ { + err := dec.fill(c.Index(i), tag, &[]deferedPtr{}) + if err != nil { + return fmt.Errorf("could not fill element %d of pipe: %v", i, err) + } + } + s, err = dec.readUint32() // read element count of first chunk + if err != nil { + return err + } + a = reflect.AppendSlice(a, c) + c = reflect.MakeSlice(v.Type(), int(s), int(s)) + } + v.Set(a) + return nil +} diff --git a/vendor/github.com/jcmturner/rpc/v2/ndr/primitives.go b/vendor/github.com/jcmturner/rpc/v2/ndr/primitives.go new file mode 100644 index 000000000000..7eb1d1afbf0a --- /dev/null +++ b/vendor/github.com/jcmturner/rpc/v2/ndr/primitives.go @@ -0,0 +1,211 @@ +package ndr + +import ( + "bytes" + "encoding/binary" + "math" +) + +// Byte sizes of primitive types +const ( + SizeBool = 1 + SizeChar = 1 + SizeUint8 = 1 + SizeUint16 = 2 + SizeUint32 = 4 + SizeUint64 = 8 + SizeEnum = 2 + SizeSingle = 4 + SizeDouble = 8 + SizePtr = 4 +) + +// Bool is an NDR Boolean which is a logical quantity that assumes one of two values: TRUE or FALSE. +// NDR represents a Boolean as one octet. +// It represents a value of FALSE as a zero octet, an octet in which every bit is reset. +// It represents a value of TRUE as a non-zero octet, an octet in which one or more bits are set. + +// Char is an NDR character. +// NDR represents a character as one octet. +// Characters have two representation formats: ASCII and EBCDIC. + +// USmall is an unsigned 8 bit integer + +// UShort is an unsigned 16 bit integer + +// ULong is an unsigned 32 bit integer + +// UHyper is an unsigned 64 bit integer + +// Small is an signed 8 bit integer + +// Short is an signed 16 bit integer + +// Long is an signed 32 bit integer + +// Hyper is an signed 64 bit integer + +// Enum is the NDR representation of enumerated types as signed short integers (2 octets) + +// Single is an NDR defined single-precision floating-point data type + +// Double is an NDR defined double-precision floating-point data type + +// readBool reads a byte representing a boolean. +// NDR represents a Boolean as one octet. +// It represents a value of FALSE as a zero octet, an octet in which every bit is reset. +// It represents a value of TRUE as a non-zero octet, an octet in which one or more bits are set. +func (dec *Decoder) readBool() (bool, error) { + i, err := dec.readUint8() + if err != nil { + return false, err + } + if i != 0 { + return true, nil + } + return false, nil +} + +// readChar reads bytes representing a 8bit ASCII integer cast to a rune. +func (dec *Decoder) readChar() (rune, error) { + var r rune + a, err := dec.readUint8() + if err != nil { + return r, err + } + return rune(a), nil +} + +// readUint8 reads bytes representing a 8bit unsigned integer. +func (dec *Decoder) readUint8() (uint8, error) { + b, err := dec.r.ReadByte() + if err != nil { + return uint8(0), err + } + return uint8(b), nil +} + +// readUint16 reads bytes representing a 16bit unsigned integer. +func (dec *Decoder) readUint16() (uint16, error) { + dec.ensureAlignment(SizeUint16) + b, err := dec.readBytes(SizeUint16) + if err != nil { + return uint16(0), err + } + return dec.ch.Endianness.Uint16(b), nil +} + +// readUint32 reads bytes representing a 32bit unsigned integer. +func (dec *Decoder) readUint32() (uint32, error) { + dec.ensureAlignment(SizeUint32) + b, err := dec.readBytes(SizeUint32) + if err != nil { + return uint32(0), err + } + return dec.ch.Endianness.Uint32(b), nil +} + +// readUint32 reads bytes representing a 32bit unsigned integer. +func (dec *Decoder) readUint64() (uint64, error) { + dec.ensureAlignment(SizeUint64) + b, err := dec.readBytes(SizeUint64) + if err != nil { + return uint64(0), err + } + return dec.ch.Endianness.Uint64(b), nil +} + +func (dec *Decoder) readInt8() (int8, error) { + dec.ensureAlignment(SizeUint8) + b, err := dec.readBytes(SizeUint8) + if err != nil { + return 0, err + } + var i int8 + buf := bytes.NewReader(b) + err = binary.Read(buf, dec.ch.Endianness, &i) + if err != nil { + return 0, err + } + return i, nil +} + +func (dec *Decoder) readInt16() (int16, error) { + dec.ensureAlignment(SizeUint16) + b, err := dec.readBytes(SizeUint16) + if err != nil { + return 0, err + } + var i int16 + buf := bytes.NewReader(b) + err = binary.Read(buf, dec.ch.Endianness, &i) + if err != nil { + return 0, err + } + return i, nil +} + +func (dec *Decoder) readInt32() (int32, error) { + dec.ensureAlignment(SizeUint32) + b, err := dec.readBytes(SizeUint32) + if err != nil { + return 0, err + } + var i int32 + buf := bytes.NewReader(b) + err = binary.Read(buf, dec.ch.Endianness, &i) + if err != nil { + return 0, err + } + return i, nil +} + +func (dec *Decoder) readInt64() (int64, error) { + dec.ensureAlignment(SizeUint64) + b, err := dec.readBytes(SizeUint64) + if err != nil { + return 0, err + } + var i int64 + buf := bytes.NewReader(b) + err = binary.Read(buf, dec.ch.Endianness, &i) + if err != nil { + return 0, err + } + return i, nil +} + +// https://en.wikipedia.org/wiki/IEEE_754-1985 +func (dec *Decoder) readFloat32() (f float32, err error) { + dec.ensureAlignment(SizeSingle) + b, err := dec.readBytes(SizeSingle) + if err != nil { + return + } + bits := dec.ch.Endianness.Uint32(b) + f = math.Float32frombits(bits) + return +} + +func (dec *Decoder) readFloat64() (f float64, err error) { + dec.ensureAlignment(SizeDouble) + b, err := dec.readBytes(SizeDouble) + if err != nil { + return + } + bits := dec.ch.Endianness.Uint64(b) + f = math.Float64frombits(bits) + return +} + +// NDR enforces NDR alignment of primitive data; that is, any primitive of size n octets is aligned at a octet stream +// index that is a multiple of n. (In this version of NDR, n is one of {1, 2, 4, 8}.) An octet stream index indicates +// the number of an octet in an octet stream when octets are numbered, beginning with 0, from the first octet in the +// stream. Where necessary, an alignment gap, consisting of octets of unspecified value, precedes the representation +// of a primitive. The gap is of the smallest size sufficient to align the primitive. +func (dec *Decoder) ensureAlignment(n int) { + p := dec.size - dec.r.Buffered() + if s := p % n; s != 0 { + dec.r.Discard(n - s) + } +} diff --git a/vendor/github.com/jcmturner/rpc/v2/ndr/rawbytes.go b/vendor/github.com/jcmturner/rpc/v2/ndr/rawbytes.go new file mode 100644 index 000000000000..9ee59fb10e42 --- /dev/null +++ b/vendor/github.com/jcmturner/rpc/v2/ndr/rawbytes.go @@ -0,0 +1,61 @@ +package ndr + +import ( + "errors" + "fmt" + "reflect" + "strconv" +) + +// type MyBytes []byte +// implement RawBytes interface + +const ( + sizeMethod = "Size" +) + +// RawBytes interface should be implemented if reading just a number of bytes from the NDR stream +type RawBytes interface { + Size(interface{}) int +} + +func rawBytesSize(parent reflect.Value, v reflect.Value) (int, error) { + sf := v.MethodByName(sizeMethod) + if !sf.IsValid() { + return 0, fmt.Errorf("could not find a method called %s on the implementation of RawBytes", sizeMethod) + } + in := []reflect.Value{parent} + f := sf.Call(in) + if f[0].Kind() != reflect.Int { + return 0, errors.New("the RawBytes size function did not return an integer") + } + return int(f[0].Int()), nil +} + +func addSizeToTag(parent reflect.Value, v reflect.Value, tag reflect.StructTag) (reflect.StructTag, error) { + size, err := rawBytesSize(parent, v) + if err != nil { + return tag, err + } + ndrTag := parseTags(tag) + ndrTag.Map["size"] = strconv.Itoa(size) + return ndrTag.StructTag(), nil +} + +func (dec *Decoder) readRawBytes(v reflect.Value, tag reflect.StructTag) error { + ndrTag := parseTags(tag) + sizeStr, ok := ndrTag.Map["size"] + if !ok { + return errors.New("size tag not available") + } + size, err := strconv.Atoi(sizeStr) + if err != nil { + return fmt.Errorf("size not valid: %v", err) + } + b, err := dec.readBytes(size) + if err != nil { + return err + } + v.Set(reflect.ValueOf(b).Convert(v.Type())) + return nil +} diff --git a/vendor/github.com/jcmturner/rpc/v2/ndr/strings.go b/vendor/github.com/jcmturner/rpc/v2/ndr/strings.go new file mode 100644 index 000000000000..b7a910b3d1bd --- /dev/null +++ b/vendor/github.com/jcmturner/rpc/v2/ndr/strings.go @@ -0,0 +1,70 @@ +package ndr + +import ( + "fmt" + "reflect" +) + +const ( + subStringArrayTag = `ndr:"varying,X-subStringArray"` + subStringArrayValue = "X-subStringArray" +) + +func uint16SliceToString(a []uint16) string { + s := make([]rune, len(a), len(a)) + for i := range s { + s[i] = rune(a[i]) + } + if len(s) > 0 { + // Remove any null terminator + if s[len(s)-1] == rune(0) { + s = s[:len(s)-1] + } + } + return string(s) +} + +func (dec *Decoder) readVaryingString(def *[]deferedPtr) (string, error) { + a := new([]uint16) + v := reflect.ValueOf(a) + var t reflect.StructTag + err := dec.fillUniDimensionalVaryingArray(v.Elem(), t, def) + if err != nil { + return "", err + } + s := uint16SliceToString(*a) + return s, nil +} + +func (dec *Decoder) readConformantVaryingString(def *[]deferedPtr) (string, error) { + a := new([]uint16) + v := reflect.ValueOf(a) + var t reflect.StructTag + err := dec.fillUniDimensionalConformantVaryingArray(v.Elem(), t, def) + if err != nil { + return "", err + } + s := uint16SliceToString(*a) + return s, nil +} + +func (dec *Decoder) readStringsArray(v reflect.Value, tag reflect.StructTag, def *[]deferedPtr) error { + d, _ := sliceDimensions(v.Type()) + ndrTag := parseTags(tag) + var m []int + //var ms int + if ndrTag.HasValue(TagConformant) { + for i := 0; i < d; i++ { + m = append(m, int(dec.precedingMax())) + } + //common max size + _ = dec.precedingMax() + //ms = int(n) + } + tag = reflect.StructTag(subStringArrayTag) + err := dec.fillVaryingArray(v, tag, def) + if err != nil { + return fmt.Errorf("could not read string array: %v", err) + } + return nil +} diff --git a/vendor/github.com/jcmturner/rpc/v2/ndr/tags.go b/vendor/github.com/jcmturner/rpc/v2/ndr/tags.go new file mode 100644 index 000000000000..01657e0a1d77 --- /dev/null +++ b/vendor/github.com/jcmturner/rpc/v2/ndr/tags.go @@ -0,0 +1,69 @@ +package ndr + +import ( + "fmt" + "reflect" + "strings" +) + +const ndrNameSpace = "ndr" + +type tags struct { + Values []string + Map map[string]string +} + +// parse the struct field tags and extract the ndr related ones. +// format of tag ndr:"value,key:value1,value2" +func parseTags(st reflect.StructTag) tags { + s := st.Get(ndrNameSpace) + t := tags{ + Values: []string{}, + Map: make(map[string]string), + } + if s != "" { + ndrTags := strings.Trim(s, `"`) + for _, tag := range strings.Split(ndrTags, ",") { + if strings.Contains(tag, ":") { + m := strings.SplitN(tag, ":", 2) + t.Map[m[0]] = m[1] + } else { + t.Values = append(t.Values, tag) + } + } + } + return t +} + +func appendTag(t reflect.StructTag, s string) reflect.StructTag { + ts := t.Get(ndrNameSpace) + ts = fmt.Sprintf(`%s"%s,%s"`, ndrNameSpace, ts, s) + return reflect.StructTag(ts) +} + +func (t *tags) StructTag() reflect.StructTag { + mv := t.Values + for key, val := range t.Map { + mv = append(mv, key+":"+val) + } + s := ndrNameSpace + ":" + `"` + strings.Join(mv, ",") + `"` + return reflect.StructTag(s) +} + +func (t *tags) delete(s string) { + for i, x := range t.Values { + if x == s { + t.Values = append(t.Values[:i], t.Values[i+1:]...) + } + } + delete(t.Map, s) +} + +func (t *tags) HasValue(s string) bool { + for _, v := range t.Values { + if v == s { + return true + } + } + return false +} diff --git a/vendor/github.com/jcmturner/rpc/v2/ndr/union.go b/vendor/github.com/jcmturner/rpc/v2/ndr/union.go new file mode 100644 index 000000000000..6a657fa6f660 --- /dev/null +++ b/vendor/github.com/jcmturner/rpc/v2/ndr/union.go @@ -0,0 +1,57 @@ +package ndr + +import ( + "errors" + "fmt" + "reflect" +) + +// Union interface must be implemented by structs that will be unmarshaled into from the NDR byte stream union representation. +// The union's discriminating tag will be passed to the SwitchFunc method. +// The discriminating tag field must have the struct tag: `ndr:"unionTag"` +// If the union is encapsulated the discriminating tag field must have the struct tag: `ndr:"encapsulated"` +// The possible value fields that can be selected from must have the struct tag: `ndr:"unionField"` +type Union interface { + SwitchFunc(t interface{}) string +} + +// Union related constants such as struct tag values +const ( + unionSelectionFuncName = "SwitchFunc" + TagEncapsulated = "encapsulated" + TagUnionTag = "unionTag" + TagUnionField = "unionField" +) + +func (dec *Decoder) isUnion(field reflect.Value, tag reflect.StructTag) (r reflect.Value) { + ndrTag := parseTags(tag) + if !ndrTag.HasValue(TagUnionTag) { + return + } + r = field + // For a non-encapsulated union, the discriminant is marshalled into the transmitted data stream twice: once as the + // field or parameter, which is referenced by the switch_is construct, in the procedure argument list; and once as + // the first part of the union representation. + if !ndrTag.HasValue(TagEncapsulated) { + dec.r.Discard(int(r.Type().Size())) + } + return +} + +// unionSelectedField returns the field name of which of the union values to fill +func unionSelectedField(union, discriminant reflect.Value) (string, error) { + if !union.Type().Implements(reflect.TypeOf(new(Union)).Elem()) { + return "", errors.New("struct does not implement union interface") + } + args := []reflect.Value{discriminant} + // Call the SelectFunc of the union struct to find the name of the field to fill with the value selected. + sf := union.MethodByName(unionSelectionFuncName) + if !sf.IsValid() { + return "", fmt.Errorf("could not find a selection function called %s in the unions struct representation", unionSelectionFuncName) + } + f := sf.Call(args) + if f[0].Kind() != reflect.String || f[0].String() == "" { + return "", fmt.Errorf("the union select function did not return a string for the name of the field to fill") + } + return f[0].String(), nil +} diff --git a/vendor/github.com/klauspost/compress/.gitattributes b/vendor/github.com/klauspost/compress/.gitattributes new file mode 100644 index 000000000000..402433593c09 --- /dev/null +++ b/vendor/github.com/klauspost/compress/.gitattributes @@ -0,0 +1,2 @@ +* -text +*.bin -text -diff diff --git a/vendor/github.com/klauspost/compress/.gitignore b/vendor/github.com/klauspost/compress/.gitignore new file mode 100644 index 000000000000..b35f8449bf28 --- /dev/null +++ b/vendor/github.com/klauspost/compress/.gitignore @@ -0,0 +1,25 @@ +# Compiled Object files, Static and Dynamic libs (Shared Objects) +*.o +*.a +*.so + +# Folders +_obj +_test + +# Architecture specific extensions/prefixes +*.[568vq] +[568vq].out + +*.cgo1.go +*.cgo2.c +_cgo_defun.c +_cgo_gotypes.go +_cgo_export.* + +_testmain.go + +*.exe +*.test +*.prof +/s2/cmd/_s2sx/sfx-exe diff --git a/vendor/github.com/klauspost/compress/.goreleaser.yml b/vendor/github.com/klauspost/compress/.goreleaser.yml new file mode 100644 index 000000000000..c9014ce1da23 --- /dev/null +++ b/vendor/github.com/klauspost/compress/.goreleaser.yml @@ -0,0 +1,137 @@ +# This is an example goreleaser.yaml file with some sane defaults. +# Make sure to check the documentation at http://goreleaser.com +before: + hooks: + - ./gen.sh + +builds: + - + id: "s2c" + binary: s2c + main: ./s2/cmd/s2c/main.go + flags: + - -trimpath + env: + - CGO_ENABLED=0 + goos: + - aix + - linux + - freebsd + - netbsd + - windows + - darwin + goarch: + - 386 + - amd64 + - arm + - arm64 + - ppc64 + - ppc64le + - mips64 + - mips64le + goarm: + - 7 + - + id: "s2d" + binary: s2d + main: ./s2/cmd/s2d/main.go + flags: + - -trimpath + env: + - CGO_ENABLED=0 + goos: + - aix + - linux + - freebsd + - netbsd + - windows + - darwin + goarch: + - 386 + - amd64 + - arm + - arm64 + - ppc64 + - ppc64le + - mips64 + - mips64le + goarm: + - 7 + - + id: "s2sx" + binary: s2sx + main: ./s2/cmd/_s2sx/main.go + flags: + - -modfile=s2sx.mod + - -trimpath + env: + - CGO_ENABLED=0 + goos: + - aix + - linux + - freebsd + - netbsd + - windows + - darwin + goarch: + - 386 + - amd64 + - arm + - arm64 + - ppc64 + - ppc64le + - mips64 + - mips64le + goarm: + - 7 + +archives: + - + id: s2-binaries + name_template: "s2-{{ .Os }}_{{ .Arch }}_{{ .Version }}" + replacements: + aix: AIX + darwin: OSX + linux: Linux + windows: Windows + 386: i386 + amd64: x86_64 + freebsd: FreeBSD + netbsd: NetBSD + format_overrides: + - goos: windows + format: zip + files: + - unpack/* + - s2/LICENSE + - s2/README.md +checksum: + name_template: 'checksums.txt' +snapshot: + name_template: "{{ .Tag }}-next" +changelog: + sort: asc + filters: + exclude: + - '^doc:' + - '^docs:' + - '^test:' + - '^tests:' + - '^Update\sREADME.md' + +nfpms: + - + file_name_template: "s2_package_{{ .Version }}_{{ .Os }}_{{ .Arch }}" + vendor: Klaus Post + homepage: https://github.com/klauspost/compress + maintainer: Klaus Post + description: S2 Compression Tool + license: BSD 3-Clause + formats: + - deb + - rpm + replacements: + darwin: Darwin + linux: Linux + freebsd: FreeBSD + amd64: x86_64 diff --git a/vendor/github.com/klauspost/compress/LICENSE b/vendor/github.com/klauspost/compress/LICENSE index 1eb75ef68e44..87d55747778c 100644 --- a/vendor/github.com/klauspost/compress/LICENSE +++ b/vendor/github.com/klauspost/compress/LICENSE @@ -26,3 +26,279 @@ DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. + +------------------ + +Files: gzhttp/* + + 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. + + END OF TERMS AND CONDITIONS + + APPENDIX: How to apply the Apache License to your work. + + To apply the Apache License to your work, attach the following + boilerplate notice, with the fields enclosed by brackets "[]" + replaced with your own identifying information. (Don't include + the brackets!) The text should be enclosed in the appropriate + comment syntax for the file format. We also recommend that a + file or class name and description of purpose be included on the + same "printed page" as the copyright notice for easier + identification within third-party archives. + + Copyright 2016-2017 The New York Times Company + + 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. + +------------------ + +Files: s2/cmd/internal/readahead/* + +The MIT License (MIT) + +Copyright (c) 2015 Klaus Post + +Permission is hereby granted, free of charge, to any person obtaining a copy +of this software and associated documentation files (the "Software"), to deal +in the Software without restriction, including without limitation the rights +to use, copy, modify, merge, publish, distribute, sublicense, and/or sell +copies of the Software, and to permit persons to whom the Software is +furnished to do so, subject to the following conditions: + +The above copyright notice and this permission notice shall be included in all +copies or substantial portions of the Software. + +THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR +IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, +FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE +AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER +LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, +OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE +SOFTWARE. + +--------------------- +Files: snappy/* +Files: internal/snapref/* + +Copyright (c) 2011 The Snappy-Go Authors. All rights reserved. + +Redistribution and use in source and binary forms, with or without +modification, are permitted provided that the following conditions are +met: + + * Redistributions of source code must retain the above copyright +notice, this list of conditions and the following disclaimer. + * Redistributions in binary form must reproduce the above +copyright notice, this list of conditions and the following disclaimer +in the documentation and/or other materials provided with the +distribution. + * Neither the name of Google Inc. nor the names of its +contributors may be used to endorse or promote products derived from +this software without specific prior written permission. + +THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS +"AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT +LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR +A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT +OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, +SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT +LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, +DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY +THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT +(INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE +OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. + +----------------- + +Files: s2/cmd/internal/filepathx/* + +Copyright 2016 The filepathx Authors + +Permission is hereby granted, free of charge, to any person obtaining a copy of this software and associated documentation files (the "Software"), to deal in the Software without restriction, including without limitation the rights to use, copy, modify, merge, publish, distribute, sublicense, and/or sell copies of the Software, and to permit persons to whom the Software is furnished to do so, subject to the following conditions: + +The above copyright notice and this permission notice shall be included in all copies or substantial portions of the Software. + +THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE SOFTWARE. diff --git a/vendor/github.com/klauspost/compress/README.md b/vendor/github.com/klauspost/compress/README.md new file mode 100644 index 000000000000..3429879eb69f --- /dev/null +++ b/vendor/github.com/klauspost/compress/README.md @@ -0,0 +1,438 @@ +# compress + +This package provides various compression algorithms. + +* [zstandard](https://github.com/klauspost/compress/tree/master/zstd#zstd) compression and decompression in pure Go. +* [S2](https://github.com/klauspost/compress/tree/master/s2#s2-compression) is a high performance replacement for Snappy. +* Optimized [deflate](https://godoc.org/github.com/klauspost/compress/flate) packages which can be used as a dropin replacement for [gzip](https://godoc.org/github.com/klauspost/compress/gzip), [zip](https://godoc.org/github.com/klauspost/compress/zip) and [zlib](https://godoc.org/github.com/klauspost/compress/zlib). +* [snappy](https://github.com/klauspost/compress/tree/master/snappy) is a drop-in replacement for `github.com/golang/snappy` offering better compression and concurrent streams. +* [huff0](https://github.com/klauspost/compress/tree/master/huff0) and [FSE](https://github.com/klauspost/compress/tree/master/fse) implementations for raw entropy encoding. +* [gzhttp](https://github.com/klauspost/compress/tree/master/gzhttp) Provides client and server wrappers for handling gzipped requests efficiently. +* [pgzip](https://github.com/klauspost/pgzip) is a separate package that provides a very fast parallel gzip implementation. +* [fuzz package](https://github.com/klauspost/compress-fuzz) for fuzz testing all compressors/decompressors here. + +[![Go Reference](https://pkg.go.dev/badge/klauspost/compress.svg)](https://pkg.go.dev/github.com/klauspost/compress?tab=subdirectories) +[![Go](https://github.com/klauspost/compress/actions/workflows/go.yml/badge.svg)](https://github.com/klauspost/compress/actions/workflows/go.yml) +[![Sourcegraph Badge](https://sourcegraph.com/github.com/klauspost/compress/-/badge.svg)](https://sourcegraph.com/github.com/klauspost/compress?badge) + +# changelog + +* Aug 30, 2021 (v1.13.5) + * gz/zlib/flate: Alias stdlib errors [#425](https://github.com/klauspost/compress/pull/425) + * s2: Add block support to commandline tools [#413](https://github.com/klauspost/compress/pull/413) + * zstd: pooledZipWriter should return Writers to the same pool [#426](https://github.com/klauspost/compress/pull/426) + * Removed golang/snappy as external dependency for tests [#421](https://github.com/klauspost/compress/pull/421) + +* Aug 12, 2021 (v1.13.4) + * Add [snappy replacement package](https://github.com/klauspost/compress/tree/master/snappy). + * zstd: Fix incorrect encoding in "best" mode [#415](https://github.com/klauspost/compress/pull/415) + +* Aug 3, 2021 (v1.13.3) + * zstd: Improve Best compression [#404](https://github.com/klauspost/compress/pull/404) + * zstd: Fix WriteTo error forwarding [#411](https://github.com/klauspost/compress/pull/411) + * gzhttp: Return http.HandlerFunc instead of http.Handler. Unlikely breaking change. [#406](https://github.com/klauspost/compress/pull/406) + * s2sx: Fix max size error [#399](https://github.com/klauspost/compress/pull/399) + * zstd: Add optional stream content size on reset [#401](https://github.com/klauspost/compress/pull/401) + * zstd: use SpeedBestCompression for level >= 10 [#410](https://github.com/klauspost/compress/pull/410) + +* Jun 14, 2021 (v1.13.1) + * s2: Add full Snappy output support [#396](https://github.com/klauspost/compress/pull/396) + * zstd: Add configurable [Decoder window](https://pkg.go.dev/github.com/klauspost/compress/zstd#WithDecoderMaxWindow) size [#394](https://github.com/klauspost/compress/pull/394) + * gzhttp: Add header to skip compression [#389](https://github.com/klauspost/compress/pull/389) + * s2: Improve speed with bigger output margin [#395](https://github.com/klauspost/compress/pull/395) + +* Jun 3, 2021 (v1.13.0) + * Added [gzhttp](https://github.com/klauspost/compress/tree/master/gzhttp#gzip-handler) which allows wrapping HTTP servers and clients with GZIP compressors. + * zstd: Detect short invalid signatures [#382](https://github.com/klauspost/compress/pull/382) + * zstd: Spawn decoder goroutine only if needed. [#380](https://github.com/klauspost/compress/pull/380) + +* May 25, 2021 (v1.12.3) + * deflate: Better/faster Huffman encoding [#374](https://github.com/klauspost/compress/pull/374) + * deflate: Allocate less for history. [#375](https://github.com/klauspost/compress/pull/375) + * zstd: Forward read errors [#373](https://github.com/klauspost/compress/pull/373) + +* Apr 27, 2021 (v1.12.2) + * zstd: Improve better/best compression [#360](https://github.com/klauspost/compress/pull/360) [#364](https://github.com/klauspost/compress/pull/364) [#365](https://github.com/klauspost/compress/pull/365) + * zstd: Add helpers to compress/decompress zstd inside zip files [#363](https://github.com/klauspost/compress/pull/363) + * deflate: Improve level 5+6 compression [#367](https://github.com/klauspost/compress/pull/367) + * s2: Improve better/best compression [#358](https://github.com/klauspost/compress/pull/358) [#359](https://github.com/klauspost/compress/pull/358) + * s2: Load after checking src limit on amd64. [#362](https://github.com/klauspost/compress/pull/362) + * s2sx: Limit max executable size [#368](https://github.com/klauspost/compress/pull/368) + +* Apr 14, 2021 (v1.12.1) + * snappy package removed. Upstream added as dependency. + * s2: Better compression in "best" mode [#353](https://github.com/klauspost/compress/pull/353) + * s2sx: Add stdin input and detect pre-compressed from signature [#352](https://github.com/klauspost/compress/pull/352) + * s2c/s2d: Add http as possible input [#348](https://github.com/klauspost/compress/pull/348) + * s2c/s2d/s2sx: Always truncate when writing files [#352](https://github.com/klauspost/compress/pull/352) + * zstd: Reduce memory usage further when using [WithLowerEncoderMem](https://pkg.go.dev/github.com/klauspost/compress/zstd#WithLowerEncoderMem) [#346](https://github.com/klauspost/compress/pull/346) + * s2: Fix potential problem with amd64 assembly and profilers [#349](https://github.com/klauspost/compress/pull/349) + +
+ See changes prior to v1.12.1 + +* Mar 26, 2021 (v1.11.13) + * zstd: Big speedup on small dictionary encodes [#344](https://github.com/klauspost/compress/pull/344) [#345](https://github.com/klauspost/compress/pull/345) + * zstd: Add [WithLowerEncoderMem](https://pkg.go.dev/github.com/klauspost/compress/zstd#WithLowerEncoderMem) encoder option [#336](https://github.com/klauspost/compress/pull/336) + * deflate: Improve entropy compression [#338](https://github.com/klauspost/compress/pull/338) + * s2: Clean up and minor performance improvement in best [#341](https://github.com/klauspost/compress/pull/341) + +* Mar 5, 2021 (v1.11.12) + * s2: Add `s2sx` binary that creates [self extracting archives](https://github.com/klauspost/compress/tree/master/s2#s2sx-self-extracting-archives). + * s2: Speed up decompression on non-assembly platforms [#328](https://github.com/klauspost/compress/pull/328) + +* Mar 1, 2021 (v1.11.9) + * s2: Add ARM64 decompression assembly. Around 2x output speed. [#324](https://github.com/klauspost/compress/pull/324) + * s2: Improve "better" speed and efficiency. [#325](https://github.com/klauspost/compress/pull/325) + * s2: Fix binaries. + +* Feb 25, 2021 (v1.11.8) + * s2: Fixed occational out-of-bounds write on amd64. Upgrade recommended. + * s2: Add AMD64 assembly for better mode. 25-50% faster. [#315](https://github.com/klauspost/compress/pull/315) + * s2: Less upfront decoder allocation. [#322](https://github.com/klauspost/compress/pull/322) + * zstd: Faster "compression" of incompressible data. [#314](https://github.com/klauspost/compress/pull/314) + * zip: Fix zip64 headers. [#313](https://github.com/klauspost/compress/pull/313) + +* Jan 14, 2021 (v1.11.7) + * Use Bytes() interface to get bytes across packages. [#309](https://github.com/klauspost/compress/pull/309) + * s2: Add 'best' compression option. [#310](https://github.com/klauspost/compress/pull/310) + * s2: Add ReaderMaxBlockSize, changes `s2.NewReader` signature to include varargs. [#311](https://github.com/klauspost/compress/pull/311) + * s2: Fix crash on small better buffers. [#308](https://github.com/klauspost/compress/pull/308) + * s2: Clean up decoder. [#312](https://github.com/klauspost/compress/pull/312) + +* Jan 7, 2021 (v1.11.6) + * zstd: Make decoder allocations smaller [#306](https://github.com/klauspost/compress/pull/306) + * zstd: Free Decoder resources when Reset is called with a nil io.Reader [#305](https://github.com/klauspost/compress/pull/305) + +* Dec 20, 2020 (v1.11.4) + * zstd: Add Best compression mode [#304](https://github.com/klauspost/compress/pull/304) + * Add header decoder [#299](https://github.com/klauspost/compress/pull/299) + * s2: Add uncompressed stream option [#297](https://github.com/klauspost/compress/pull/297) + * Simplify/speed up small blocks with known max size. [#300](https://github.com/klauspost/compress/pull/300) + * zstd: Always reset literal dict encoder [#303](https://github.com/klauspost/compress/pull/303) + +* Nov 15, 2020 (v1.11.3) + * inflate: 10-15% faster decompression [#293](https://github.com/klauspost/compress/pull/293) + * zstd: Tweak DecodeAll default allocation [#295](https://github.com/klauspost/compress/pull/295) + +* Oct 11, 2020 (v1.11.2) + * s2: Fix out of bounds read in "better" block compression [#291](https://github.com/klauspost/compress/pull/291) + +* Oct 1, 2020 (v1.11.1) + * zstd: Set allLitEntropy true in default configuration [#286](https://github.com/klauspost/compress/pull/286) + +* Sept 8, 2020 (v1.11.0) + * zstd: Add experimental compression [dictionaries](https://github.com/klauspost/compress/tree/master/zstd#dictionaries) [#281](https://github.com/klauspost/compress/pull/281) + * zstd: Fix mixed Write and ReadFrom calls [#282](https://github.com/klauspost/compress/pull/282) + * inflate/gz: Limit variable shifts, ~5% faster decompression [#274](https://github.com/klauspost/compress/pull/274) +
+ +
+ See changes prior to v1.11.0 + +* July 8, 2020 (v1.10.11) + * zstd: Fix extra block when compressing with ReadFrom. [#278](https://github.com/klauspost/compress/pull/278) + * huff0: Also populate compression table when reading decoding table. [#275](https://github.com/klauspost/compress/pull/275) + +* June 23, 2020 (v1.10.10) + * zstd: Skip entropy compression in fastest mode when no matches. [#270](https://github.com/klauspost/compress/pull/270) + +* June 16, 2020 (v1.10.9): + * zstd: API change for specifying dictionaries. See [#268](https://github.com/klauspost/compress/pull/268) + * zip: update CreateHeaderRaw to handle zip64 fields. [#266](https://github.com/klauspost/compress/pull/266) + * Fuzzit tests removed. The service has been purchased and is no longer available. + +* June 5, 2020 (v1.10.8): + * 1.15x faster zstd block decompression. [#265](https://github.com/klauspost/compress/pull/265) + +* June 1, 2020 (v1.10.7): + * Added zstd decompression [dictionary support](https://github.com/klauspost/compress/tree/master/zstd#dictionaries) + * Increase zstd decompression speed up to 1.19x. [#259](https://github.com/klauspost/compress/pull/259) + * Remove internal reset call in zstd compression and reduce allocations. [#263](https://github.com/klauspost/compress/pull/263) + +* May 21, 2020: (v1.10.6) + * zstd: Reduce allocations while decoding. [#258](https://github.com/klauspost/compress/pull/258), [#252](https://github.com/klauspost/compress/pull/252) + * zstd: Stricter decompression checks. + +* April 12, 2020: (v1.10.5) + * s2-commands: Flush output when receiving SIGINT. [#239](https://github.com/klauspost/compress/pull/239) + +* Apr 8, 2020: (v1.10.4) + * zstd: Minor/special case optimizations. [#251](https://github.com/klauspost/compress/pull/251), [#250](https://github.com/klauspost/compress/pull/250), [#249](https://github.com/klauspost/compress/pull/249), [#247](https://github.com/klauspost/compress/pull/247) +* Mar 11, 2020: (v1.10.3) + * s2: Use S2 encoder in pure Go mode for Snappy output as well. [#245](https://github.com/klauspost/compress/pull/245) + * s2: Fix pure Go block encoder. [#244](https://github.com/klauspost/compress/pull/244) + * zstd: Added "better compression" mode. [#240](https://github.com/klauspost/compress/pull/240) + * zstd: Improve speed of fastest compression mode by 5-10% [#241](https://github.com/klauspost/compress/pull/241) + * zstd: Skip creating encoders when not needed. [#238](https://github.com/klauspost/compress/pull/238) + +* Feb 27, 2020: (v1.10.2) + * Close to 50% speedup in inflate (gzip/zip decompression). [#236](https://github.com/klauspost/compress/pull/236) [#234](https://github.com/klauspost/compress/pull/234) [#232](https://github.com/klauspost/compress/pull/232) + * Reduce deflate level 1-6 memory usage up to 59%. [#227](https://github.com/klauspost/compress/pull/227) + +* Feb 18, 2020: (v1.10.1) + * Fix zstd crash when resetting multiple times without sending data. [#226](https://github.com/klauspost/compress/pull/226) + * deflate: Fix dictionary use on level 1-6. [#224](https://github.com/klauspost/compress/pull/224) + * Remove deflate writer reference when closing. [#224](https://github.com/klauspost/compress/pull/224) + +* Feb 4, 2020: (v1.10.0) + * Add optional dictionary to [stateless deflate](https://pkg.go.dev/github.com/klauspost/compress/flate?tab=doc#StatelessDeflate). Breaking change, send `nil` for previous behaviour. [#216](https://github.com/klauspost/compress/pull/216) + * Fix buffer overflow on repeated small block deflate. [#218](https://github.com/klauspost/compress/pull/218) + * Allow copying content from an existing ZIP file without decompressing+compressing. [#214](https://github.com/klauspost/compress/pull/214) + * Added [S2](https://github.com/klauspost/compress/tree/master/s2#s2-compression) AMD64 assembler and various optimizations. Stream speed >10GB/s. [#186](https://github.com/klauspost/compress/pull/186) + +
+ +
+ See changes prior to v1.10.0 + +* Jan 20,2020 (v1.9.8) Optimize gzip/deflate with better size estimates and faster table generation. [#207](https://github.com/klauspost/compress/pull/207) by [luyu6056](https://github.com/luyu6056), [#206](https://github.com/klauspost/compress/pull/206). +* Jan 11, 2020: S2 Encode/Decode will use provided buffer if capacity is big enough. [#204](https://github.com/klauspost/compress/pull/204) +* Jan 5, 2020: (v1.9.7) Fix another zstd regression in v1.9.5 - v1.9.6 removed. +* Jan 4, 2020: (v1.9.6) Regression in v1.9.5 fixed causing corrupt zstd encodes in rare cases. +* Jan 4, 2020: Faster IO in [s2c + s2d commandline tools](https://github.com/klauspost/compress/tree/master/s2#commandline-tools) compression/decompression. [#192](https://github.com/klauspost/compress/pull/192) +* Dec 29, 2019: Removed v1.9.5 since fuzz tests showed a compatibility problem with the reference zstandard decoder. +* Dec 29, 2019: (v1.9.5) zstd: 10-20% faster block compression. [#199](https://github.com/klauspost/compress/pull/199) +* Dec 29, 2019: [zip](https://godoc.org/github.com/klauspost/compress/zip) package updated with latest Go features +* Dec 29, 2019: zstd: Single segment flag condintions tweaked. [#197](https://github.com/klauspost/compress/pull/197) +* Dec 18, 2019: s2: Faster compression when ReadFrom is used. [#198](https://github.com/klauspost/compress/pull/198) +* Dec 10, 2019: s2: Fix repeat length output when just above at 16MB limit. +* Dec 10, 2019: zstd: Add function to get decoder as io.ReadCloser. [#191](https://github.com/klauspost/compress/pull/191) +* Dec 3, 2019: (v1.9.4) S2: limit max repeat length. [#188](https://github.com/klauspost/compress/pull/188) +* Dec 3, 2019: Add [WithNoEntropyCompression](https://godoc.org/github.com/klauspost/compress/zstd#WithNoEntropyCompression) to zstd [#187](https://github.com/klauspost/compress/pull/187) +* Dec 3, 2019: Reduce memory use for tests. Check for leaked goroutines. +* Nov 28, 2019 (v1.9.3) Less allocations in stateless deflate. +* Nov 28, 2019: 5-20% Faster huff0 decode. Impacts zstd as well. [#184](https://github.com/klauspost/compress/pull/184) +* Nov 12, 2019 (v1.9.2) Added [Stateless Compression](#stateless-compression) for gzip/deflate. +* Nov 12, 2019: Fixed zstd decompression of large single blocks. [#180](https://github.com/klauspost/compress/pull/180) +* Nov 11, 2019: Set default [s2c](https://github.com/klauspost/compress/tree/master/s2#commandline-tools) block size to 4MB. +* Nov 11, 2019: Reduce inflate memory use by 1KB. +* Nov 10, 2019: Less allocations in deflate bit writer. +* Nov 10, 2019: Fix inconsistent error returned by zstd decoder. +* Oct 28, 2019 (v1.9.1) ztsd: Fix crash when compressing blocks. [#174](https://github.com/klauspost/compress/pull/174) +* Oct 24, 2019 (v1.9.0) zstd: Fix rare data corruption [#173](https://github.com/klauspost/compress/pull/173) +* Oct 24, 2019 zstd: Fix huff0 out of buffer write [#171](https://github.com/klauspost/compress/pull/171) and always return errors [#172](https://github.com/klauspost/compress/pull/172) +* Oct 10, 2019: Big deflate rewrite, 30-40% faster with better compression [#105](https://github.com/klauspost/compress/pull/105) + +
+ +
+ See changes prior to v1.9.0 + +* Oct 10, 2019: (v1.8.6) zstd: Allow partial reads to get flushed data. [#169](https://github.com/klauspost/compress/pull/169) +* Oct 3, 2019: Fix inconsistent results on broken zstd streams. +* Sep 25, 2019: Added `-rm` (remove source files) and `-q` (no output except errors) to `s2c` and `s2d` [commands](https://github.com/klauspost/compress/tree/master/s2#commandline-tools) +* Sep 16, 2019: (v1.8.4) Add `s2c` and `s2d` [commandline tools](https://github.com/klauspost/compress/tree/master/s2#commandline-tools). +* Sep 10, 2019: (v1.8.3) Fix s2 decoder [Skip](https://godoc.org/github.com/klauspost/compress/s2#Reader.Skip). +* Sep 7, 2019: zstd: Added [WithWindowSize](https://godoc.org/github.com/klauspost/compress/zstd#WithWindowSize), contributed by [ianwilkes](https://github.com/ianwilkes). +* Sep 5, 2019: (v1.8.2) Add [WithZeroFrames](https://godoc.org/github.com/klauspost/compress/zstd#WithZeroFrames) which adds full zero payload block encoding option. +* Sep 5, 2019: Lazy initialization of zstandard predefined en/decoder tables. +* Aug 26, 2019: (v1.8.1) S2: 1-2% compression increase in "better" compression mode. +* Aug 26, 2019: zstd: Check maximum size of Huffman 1X compressed literals while decoding. +* Aug 24, 2019: (v1.8.0) Added [S2 compression](https://github.com/klauspost/compress/tree/master/s2#s2-compression), a high performance replacement for Snappy. +* Aug 21, 2019: (v1.7.6) Fixed minor issues found by fuzzer. One could lead to zstd not decompressing. +* Aug 18, 2019: Add [fuzzit](https://fuzzit.dev/) continuous fuzzing. +* Aug 14, 2019: zstd: Skip incompressible data 2x faster. [#147](https://github.com/klauspost/compress/pull/147) +* Aug 4, 2019 (v1.7.5): Better literal compression. [#146](https://github.com/klauspost/compress/pull/146) +* Aug 4, 2019: Faster zstd compression. [#143](https://github.com/klauspost/compress/pull/143) [#144](https://github.com/klauspost/compress/pull/144) +* Aug 4, 2019: Faster zstd decompression. [#145](https://github.com/klauspost/compress/pull/145) [#143](https://github.com/klauspost/compress/pull/143) [#142](https://github.com/klauspost/compress/pull/142) +* July 15, 2019 (v1.7.4): Fix double EOF block in rare cases on zstd encoder. +* July 15, 2019 (v1.7.3): Minor speedup/compression increase in default zstd encoder. +* July 14, 2019: zstd decoder: Fix decompression error on multiple uses with mixed content. +* July 7, 2019 (v1.7.2): Snappy update, zstd decoder potential race fix. +* June 17, 2019: zstd decompression bugfix. +* June 17, 2019: fix 32 bit builds. +* June 17, 2019: Easier use in modules (less dependencies). +* June 9, 2019: New stronger "default" [zstd](https://github.com/klauspost/compress/tree/master/zstd#zstd) compression mode. Matches zstd default compression ratio. +* June 5, 2019: 20-40% throughput in [zstandard](https://github.com/klauspost/compress/tree/master/zstd#zstd) compression and better compression. +* June 5, 2019: deflate/gzip compression: Reduce memory usage of lower compression levels. +* June 2, 2019: Added [zstandard](https://github.com/klauspost/compress/tree/master/zstd#zstd) compression! +* May 25, 2019: deflate/gzip: 10% faster bit writer, mostly visible in lower levels. +* Apr 22, 2019: [zstd](https://github.com/klauspost/compress/tree/master/zstd#zstd) decompression added. +* Aug 1, 2018: Added [huff0 README](https://github.com/klauspost/compress/tree/master/huff0#huff0-entropy-compression). +* Jul 8, 2018: Added [Performance Update 2018](#performance-update-2018) below. +* Jun 23, 2018: Merged [Go 1.11 inflate optimizations](https://go-review.googlesource.com/c/go/+/102235). Go 1.9 is now required. Backwards compatible version tagged with [v1.3.0](https://github.com/klauspost/compress/releases/tag/v1.3.0). +* Apr 2, 2018: Added [huff0](https://godoc.org/github.com/klauspost/compress/huff0) en/decoder. Experimental for now, API may change. +* Mar 4, 2018: Added [FSE Entropy](https://godoc.org/github.com/klauspost/compress/fse) en/decoder. Experimental for now, API may change. +* Nov 3, 2017: Add compression [Estimate](https://godoc.org/github.com/klauspost/compress#Estimate) function. +* May 28, 2017: Reduce allocations when resetting decoder. +* Apr 02, 2017: Change back to official crc32, since changes were merged in Go 1.7. +* Jan 14, 2017: Reduce stack pressure due to array copies. See [Issue #18625](https://github.com/golang/go/issues/18625). +* Oct 25, 2016: Level 2-4 have been rewritten and now offers significantly better performance than before. +* Oct 20, 2016: Port zlib changes from Go 1.7 to fix zlib writer issue. Please update. +* Oct 16, 2016: Go 1.7 changes merged. Apples to apples this package is a few percent faster, but has a significantly better balance between speed and compression per level. +* Mar 24, 2016: Always attempt Huffman encoding on level 4-7. This improves base 64 encoded data compression. +* Mar 24, 2016: Small speedup for level 1-3. +* Feb 19, 2016: Faster bit writer, level -2 is 15% faster, level 1 is 4% faster. +* Feb 19, 2016: Handle small payloads faster in level 1-3. +* Feb 19, 2016: Added faster level 2 + 3 compression modes. +* Feb 19, 2016: [Rebalanced compression levels](https://blog.klauspost.com/rebalancing-deflate-compression-levels/), so there is a more even progresssion in terms of compression. New default level is 5. +* Feb 14, 2016: Snappy: Merge upstream changes. +* Feb 14, 2016: Snappy: Fix aggressive skipping. +* Feb 14, 2016: Snappy: Update benchmark. +* Feb 13, 2016: Deflate: Fixed assembler problem that could lead to sub-optimal compression. +* Feb 12, 2016: Snappy: Added AMD64 SSE 4.2 optimizations to matching, which makes easy to compress material run faster. Typical speedup is around 25%. +* Feb 9, 2016: Added Snappy package fork. This version is 5-7% faster, much more on hard to compress content. +* Jan 30, 2016: Optimize level 1 to 3 by not considering static dictionary or storing uncompressed. ~4-5% speedup. +* Jan 16, 2016: Optimization on deflate level 1,2,3 compression. +* Jan 8 2016: Merge [CL 18317](https://go-review.googlesource.com/#/c/18317): fix reading, writing of zip64 archives. +* Dec 8 2015: Make level 1 and -2 deterministic even if write size differs. +* Dec 8 2015: Split encoding functions, so hashing and matching can potentially be inlined. 1-3% faster on AMD64. 5% faster on other platforms. +* Dec 8 2015: Fixed rare [one byte out-of bounds read](https://github.com/klauspost/compress/issues/20). Please update! +* Nov 23 2015: Optimization on token writer. ~2-4% faster. Contributed by [@dsnet](https://github.com/dsnet). +* Nov 20 2015: Small optimization to bit writer on 64 bit systems. +* Nov 17 2015: Fixed out-of-bound errors if the underlying Writer returned an error. See [#15](https://github.com/klauspost/compress/issues/15). +* Nov 12 2015: Added [io.WriterTo](https://golang.org/pkg/io/#WriterTo) support to gzip/inflate. +* Nov 11 2015: Merged [CL 16669](https://go-review.googlesource.com/#/c/16669/4): archive/zip: enable overriding (de)compressors per file +* Oct 15 2015: Added skipping on uncompressible data. Random data speed up >5x. + +
+ +# deflate usage + +* [High Throughput Benchmark](http://blog.klauspost.com/go-gzipdeflate-benchmarks/). +* [Small Payload/Webserver Benchmarks](http://blog.klauspost.com/gzip-performance-for-go-webservers/). +* [Linear Time Compression](http://blog.klauspost.com/constant-time-gzipzip-compression/). +* [Re-balancing Deflate Compression Levels](https://blog.klauspost.com/rebalancing-deflate-compression-levels/) + +The packages are drop-in replacements for standard libraries. Simply replace the import path to use them: + +| old import | new import | Documentation +|--------------------|-----------------------------------------|--------------------| +| `compress/gzip` | `github.com/klauspost/compress/gzip` | [gzip](https://pkg.go.dev/github.com/klauspost/compress/gzip?tab=doc) +| `compress/zlib` | `github.com/klauspost/compress/zlib` | [zlib](https://pkg.go.dev/github.com/klauspost/compress/zlib?tab=doc) +| `archive/zip` | `github.com/klauspost/compress/zip` | [zip](https://pkg.go.dev/github.com/klauspost/compress/zip?tab=doc) +| `compress/flate` | `github.com/klauspost/compress/flate` | [flate](https://pkg.go.dev/github.com/klauspost/compress/flate?tab=doc) + +* Optimized [deflate](https://godoc.org/github.com/klauspost/compress/flate) packages which can be used as a dropin replacement for [gzip](https://godoc.org/github.com/klauspost/compress/gzip), [zip](https://godoc.org/github.com/klauspost/compress/zip) and [zlib](https://godoc.org/github.com/klauspost/compress/zlib). + +You may also be interested in [pgzip](https://github.com/klauspost/pgzip), which is a drop in replacement for gzip, which support multithreaded compression on big files and the optimized [crc32](https://github.com/klauspost/crc32) package used by these packages. + +The packages contains the same as the standard library, so you can use the godoc for that: [gzip](http://golang.org/pkg/compress/gzip/), [zip](http://golang.org/pkg/archive/zip/), [zlib](http://golang.org/pkg/compress/zlib/), [flate](http://golang.org/pkg/compress/flate/). + +Currently there is only minor speedup on decompression (mostly CRC32 calculation). + +Memory usage is typically 1MB for a Writer. stdlib is in the same range. +If you expect to have a lot of concurrently allocated Writers consider using +the stateless compress described below. + +# Stateless compression + +This package offers stateless compression as a special option for gzip/deflate. +It will do compression but without maintaining any state between Write calls. + +This means there will be no memory kept between Write calls, but compression and speed will be suboptimal. + +This is only relevant in cases where you expect to run many thousands of compressors concurrently, +but with very little activity. This is *not* intended for regular web servers serving individual requests. + +Because of this, the size of actual Write calls will affect output size. + +In gzip, specify level `-3` / `gzip.StatelessCompression` to enable. + +For direct deflate use, NewStatelessWriter and StatelessDeflate are available. See [documentation](https://godoc.org/github.com/klauspost/compress/flate#NewStatelessWriter) + +A `bufio.Writer` can of course be used to control write sizes. For example, to use a 4KB buffer: + +``` + // replace 'ioutil.Discard' with your output. + gzw, err := gzip.NewWriterLevel(ioutil.Discard, gzip.StatelessCompression) + if err != nil { + return err + } + defer gzw.Close() + + w := bufio.NewWriterSize(gzw, 4096) + defer w.Flush() + + // Write to 'w' +``` + +This will only use up to 4KB in memory when the writer is idle. + +Compression is almost always worse than the fastest compression level +and each write will allocate (a little) memory. + +# Performance Update 2018 + +It has been a while since we have been looking at the speed of this package compared to the standard library, so I thought I would re-do my tests and give some overall recommendations based on the current state. All benchmarks have been performed with Go 1.10 on my Desktop Intel(R) Core(TM) i7-2600 CPU @3.40GHz. Since I last ran the tests, I have gotten more RAM, which means tests with big files are no longer limited by my SSD. + +The raw results are in my [updated spreadsheet](https://docs.google.com/spreadsheets/d/1nuNE2nPfuINCZJRMt6wFWhKpToF95I47XjSsc-1rbPQ/edit?usp=sharing). Due to cgo changes and upstream updates i could not get the cgo version of gzip to compile. Instead I included the [zstd](https://github.com/datadog/zstd) cgo implementation. If I get cgo gzip to work again, I might replace the results in the sheet. + +The columns to take note of are: *MB/s* - the throughput. *Reduction* - the data size reduction in percent of the original. *Rel Speed* relative speed compared to the standard library at the same level. *Smaller* - how many percent smaller is the compressed output compared to stdlib. Negative means the output was bigger. *Loss* means the loss (or gain) in compression as a percentage difference of the input. + +The `gzstd` (standard library gzip) and `gzkp` (this package gzip) only uses one CPU core. [`pgzip`](https://github.com/klauspost/pgzip), [`bgzf`](https://github.com/biogo/hts/tree/master/bgzf) uses all 4 cores. [`zstd`](https://github.com/DataDog/zstd) uses one core, and is a beast (but not Go, yet). + + +## Overall differences. + +There appears to be a roughly 5-10% speed advantage over the standard library when comparing at similar compression levels. + +The biggest difference you will see is the result of [re-balancing](https://blog.klauspost.com/rebalancing-deflate-compression-levels/) the compression levels. I wanted by library to give a smoother transition between the compression levels than the standard library. + +This package attempts to provide a more smooth transition, where "1" is taking a lot of shortcuts, "5" is the reasonable trade-off and "9" is the "give me the best compression", and the values in between gives something reasonable in between. The standard library has big differences in levels 1-4, but levels 5-9 having no significant gains - often spending a lot more time than can be justified by the achieved compression. + +There are links to all the test data in the [spreadsheet](https://docs.google.com/spreadsheets/d/1nuNE2nPfuINCZJRMt6wFWhKpToF95I47XjSsc-1rbPQ/edit?usp=sharing) in the top left field on each tab. + +## Web Content + +This test set aims to emulate typical use in a web server. The test-set is 4GB data in 53k files, and is a mixture of (mostly) HTML, JS, CSS. + +Since level 1 and 9 are close to being the same code, they are quite close. But looking at the levels in-between the differences are quite big. + +Looking at level 6, this package is 88% faster, but will output about 6% more data. For a web server, this means you can serve 88% more data, but have to pay for 6% more bandwidth. You can draw your own conclusions on what would be the most expensive for your case. + +## Object files + +This test is for typical data files stored on a server. In this case it is a collection of Go precompiled objects. They are very compressible. + +The picture is similar to the web content, but with small differences since this is very compressible. Levels 2-3 offer good speed, but is sacrificing quite a bit of compression. + +The standard library seems suboptimal on level 3 and 4 - offering both worse compression and speed than level 6 & 7 of this package respectively. + +## Highly Compressible File + +This is a JSON file with very high redundancy. The reduction starts at 95% on level 1, so in real life terms we are dealing with something like a highly redundant stream of data, etc. + +It is definitely visible that we are dealing with specialized content here, so the results are very scattered. This package does not do very well at levels 1-4, but picks up significantly at level 5 and levels 7 and 8 offering great speed for the achieved compression. + +So if you know you content is extremely compressible you might want to go slightly higher than the defaults. The standard library has a huge gap between levels 3 and 4 in terms of speed (2.75x slowdown), so it offers little "middle ground". + +## Medium-High Compressible + +This is a pretty common test corpus: [enwik9](http://mattmahoney.net/dc/textdata.html). It contains the first 10^9 bytes of the English Wikipedia dump on Mar. 3, 2006. This is a very good test of typical text based compression and more data heavy streams. + +We see a similar picture here as in "Web Content". On equal levels some compression is sacrificed for more speed. Level 5 seems to be the best trade-off between speed and size, beating stdlib level 3 in both. + +## Medium Compressible + +I will combine two test sets, one [10GB file set](http://mattmahoney.net/dc/10gb.html) and a VM disk image (~8GB). Both contain different data types and represent a typical backup scenario. + +The most notable thing is how quickly the standard library drops to very low compression speeds around level 5-6 without any big gains in compression. Since this type of data is fairly common, this does not seem like good behavior. + + +## Un-compressible Content + +This is mainly a test of how good the algorithms are at detecting un-compressible input. The standard library only offers this feature with very conservative settings at level 1. Obviously there is no reason for the algorithms to try to compress input that cannot be compressed. The only downside is that it might skip some compressible data on false detections. + + +## Huffman only compression + +This compression library adds a special compression level, named `HuffmanOnly`, which allows near linear time compression. This is done by completely disabling matching of previous data, and only reduce the number of bits to represent each character. + +This means that often used characters, like 'e' and ' ' (space) in text use the fewest bits to represent, and rare characters like '¤' takes more bits to represent. For more information see [wikipedia](https://en.wikipedia.org/wiki/Huffman_coding) or this nice [video](https://youtu.be/ZdooBTdW5bM). + +Since this type of compression has much less variance, the compression speed is mostly unaffected by the input data, and is usually more than *180MB/s* for a single core. + +The downside is that the compression ratio is usually considerably worse than even the fastest conventional compression. The compression ratio can never be better than 8:1 (12.5%). + +The linear time compression can be used as a "better than nothing" mode, where you cannot risk the encoder to slow down on some content. For comparison, the size of the "Twain" text is *233460 bytes* (+29% vs. level 1) and encode speed is 144MB/s (4.5x level 1). So in this case you trade a 30% size increase for a 4 times speedup. + +For more information see my blog post on [Fast Linear Time Compression](http://blog.klauspost.com/constant-time-gzipzip-compression/). + +This is implemented on Go 1.7 as "Huffman Only" mode, though not exposed for gzip. + + +# license + +This code is licensed under the same conditions as the original Go code. See LICENSE file. diff --git a/vendor/github.com/klauspost/compress/compressible.go b/vendor/github.com/klauspost/compress/compressible.go new file mode 100644 index 000000000000..ea5a692d5130 --- /dev/null +++ b/vendor/github.com/klauspost/compress/compressible.go @@ -0,0 +1,85 @@ +package compress + +import "math" + +// Estimate returns a normalized compressibility estimate of block b. +// Values close to zero are likely uncompressible. +// Values above 0.1 are likely to be compressible. +// Values above 0.5 are very compressible. +// Very small lengths will return 0. +func Estimate(b []byte) float64 { + if len(b) < 16 { + return 0 + } + + // Correctly predicted order 1 + hits := 0 + lastMatch := false + var o1 [256]byte + var hist [256]int + c1 := byte(0) + for _, c := range b { + if c == o1[c1] { + // We only count a hit if there was two correct predictions in a row. + if lastMatch { + hits++ + } + lastMatch = true + } else { + lastMatch = false + } + o1[c1] = c + c1 = c + hist[c]++ + } + + // Use x^0.6 to give better spread + prediction := math.Pow(float64(hits)/float64(len(b)), 0.6) + + // Calculate histogram distribution + variance := float64(0) + avg := float64(len(b)) / 256 + + for _, v := range hist { + Δ := float64(v) - avg + variance += Δ * Δ + } + + stddev := math.Sqrt(float64(variance)) / float64(len(b)) + exp := math.Sqrt(1 / float64(len(b))) + + // Subtract expected stddev + stddev -= exp + if stddev < 0 { + stddev = 0 + } + stddev *= 1 + exp + + // Use x^0.4 to give better spread + entropy := math.Pow(stddev, 0.4) + + // 50/50 weight between prediction and histogram distribution + return math.Pow((prediction+entropy)/2, 0.9) +} + +// ShannonEntropyBits returns the number of bits minimum required to represent +// an entropy encoding of the input bytes. +// https://en.wiktionary.org/wiki/Shannon_entropy +func ShannonEntropyBits(b []byte) int { + if len(b) == 0 { + return 0 + } + var hist [256]int + for _, c := range b { + hist[c]++ + } + shannon := float64(0) + invTotal := 1.0 / float64(len(b)) + for _, v := range hist[:] { + if v > 0 { + n := float64(v) + shannon += math.Ceil(-math.Log2(n*invTotal) * n) + } + } + return int(math.Ceil(shannon)) +} diff --git a/vendor/github.com/klauspost/compress/flate/fast_encoder.go b/vendor/github.com/klauspost/compress/flate/fast_encoder.go index 347ac2c902e3..a746eb73387f 100644 --- a/vendor/github.com/klauspost/compress/flate/fast_encoder.go +++ b/vendor/github.com/klauspost/compress/flate/fast_encoder.go @@ -215,24 +215,15 @@ func (e *fastGen) Reset() { func matchLen(a, b []byte) int { b = b[:len(a)] var checked int - if len(a) >= 4 { - // Try 4 bytes first - if diff := binary.LittleEndian.Uint32(a) ^ binary.LittleEndian.Uint32(b); diff != 0 { - return bits.TrailingZeros32(diff) >> 3 - } - // Switch to 8 byte matching. - checked = 4 - a = a[4:] - b = b[4:] - for len(a) >= 8 { - b = b[:len(a)] - if diff := binary.LittleEndian.Uint64(a) ^ binary.LittleEndian.Uint64(b); diff != 0 { - return checked + (bits.TrailingZeros64(diff) >> 3) - } - checked += 8 - a = a[8:] - b = b[8:] + + for len(a) >= 8 { + b = b[:len(a)] + if diff := binary.LittleEndian.Uint64(a) ^ binary.LittleEndian.Uint64(b); diff != 0 { + return checked + (bits.TrailingZeros64(diff) >> 3) } + checked += 8 + a = a[8:] + b = b[8:] } b = b[:len(a)] for i := range a { diff --git a/vendor/github.com/klauspost/compress/flate/inflate.go b/vendor/github.com/klauspost/compress/flate/inflate.go index 16bc51408e07..d1edb356c4bc 100644 --- a/vendor/github.com/klauspost/compress/flate/inflate.go +++ b/vendor/github.com/klauspost/compress/flate/inflate.go @@ -9,10 +9,10 @@ package flate import ( "bufio" + "compress/flate" "fmt" "io" "math/bits" - "strconv" "sync" ) @@ -41,11 +41,7 @@ var fixedOnce sync.Once var fixedHuffmanDecoder huffmanDecoder // A CorruptInputError reports the presence of corrupt input at a given offset. -type CorruptInputError int64 - -func (e CorruptInputError) Error() string { - return "flate: corrupt input before offset " + strconv.FormatInt(int64(e), 10) -} +type CorruptInputError = flate.CorruptInputError // An InternalError reports an error in the flate code itself. type InternalError string @@ -55,26 +51,12 @@ func (e InternalError) Error() string { return "flate: internal error: " + strin // A ReadError reports an error encountered while reading input. // // Deprecated: No longer returned. -type ReadError struct { - Offset int64 // byte offset where error occurred - Err error // error returned by underlying Read -} - -func (e *ReadError) Error() string { - return "flate: read error at offset " + strconv.FormatInt(e.Offset, 10) + ": " + e.Err.Error() -} +type ReadError = flate.ReadError // A WriteError reports an error encountered while writing output. // // Deprecated: No longer returned. -type WriteError struct { - Offset int64 // byte offset where error occurred - Err error // error returned by underlying Write -} - -func (e *WriteError) Error() string { - return "flate: write error at offset " + strconv.FormatInt(e.Offset, 10) + ": " + e.Err.Error() -} +type WriteError = flate.WriteError // Resetter resets a ReadCloser returned by NewReader or NewReaderDict to // to switch to a new underlying Reader. This permits reusing a ReadCloser diff --git a/vendor/github.com/klauspost/compress/flate/regmask_other.go b/vendor/github.com/klauspost/compress/flate/regmask_other.go index f477a5d6e5a6..1b7a2cbd7933 100644 --- a/vendor/github.com/klauspost/compress/flate/regmask_other.go +++ b/vendor/github.com/klauspost/compress/flate/regmask_other.go @@ -1,4 +1,5 @@ -//+build !amd64 +//go:build !amd64 +// +build !amd64 package flate diff --git a/vendor/github.com/klauspost/compress/gen.sh b/vendor/github.com/klauspost/compress/gen.sh new file mode 100644 index 000000000000..aff942205f1c --- /dev/null +++ b/vendor/github.com/klauspost/compress/gen.sh @@ -0,0 +1,4 @@ +#!/bin/sh + +cd s2/cmd/_s2sx/ || exit 1 +go generate . diff --git a/vendor/github.com/klauspost/compress/gzip/gunzip.go b/vendor/github.com/klauspost/compress/gzip/gunzip.go index 21e768b3604e..4d7018913e29 100644 --- a/vendor/github.com/klauspost/compress/gzip/gunzip.go +++ b/vendor/github.com/klauspost/compress/gzip/gunzip.go @@ -8,8 +8,8 @@ package gzip import ( "bufio" + "compress/gzip" "encoding/binary" - "errors" "hash/crc32" "io" "time" @@ -30,9 +30,9 @@ const ( var ( // ErrChecksum is returned when reading GZIP data that has an invalid checksum. - ErrChecksum = errors.New("gzip: invalid checksum") + ErrChecksum = gzip.ErrChecksum // ErrHeader is returned when reading GZIP data that has an invalid header. - ErrHeader = errors.New("gzip: invalid header") + ErrHeader = gzip.ErrHeader ) var le = binary.LittleEndian diff --git a/vendor/github.com/klauspost/compress/huff0/compress.go b/vendor/github.com/klauspost/compress/huff0/compress.go index 0823c928cec8..8323dc053890 100644 --- a/vendor/github.com/klauspost/compress/huff0/compress.go +++ b/vendor/github.com/klauspost/compress/huff0/compress.go @@ -161,6 +161,70 @@ func compress(in []byte, s *Scratch, compressor func(src []byte) ([]byte, error) return s.Out, false, nil } +// EstimateSizes will estimate the data sizes +func EstimateSizes(in []byte, s *Scratch) (tableSz, dataSz, reuseSz int, err error) { + s, err = s.prepare(in) + if err != nil { + return 0, 0, 0, err + } + + // Create histogram, if none was provided. + tableSz, dataSz, reuseSz = -1, -1, -1 + maxCount := s.maxCount + var canReuse = false + if maxCount == 0 { + maxCount, canReuse = s.countSimple(in) + } else { + canReuse = s.canUseTable(s.prevTable) + } + + // We want the output size to be less than this: + wantSize := len(in) + if s.WantLogLess > 0 { + wantSize -= wantSize >> s.WantLogLess + } + + // Reset for next run. + s.clearCount = true + s.maxCount = 0 + if maxCount >= len(in) { + if maxCount > len(in) { + return 0, 0, 0, fmt.Errorf("maxCount (%d) > length (%d)", maxCount, len(in)) + } + if len(in) == 1 { + return 0, 0, 0, ErrIncompressible + } + // One symbol, use RLE + return 0, 0, 0, ErrUseRLE + } + if maxCount == 1 || maxCount < (len(in)>>7) { + // Each symbol present maximum once or too well distributed. + return 0, 0, 0, ErrIncompressible + } + + // Calculate new table. + err = s.buildCTable() + if err != nil { + return 0, 0, 0, err + } + + if false && !s.canUseTable(s.cTable) { + panic("invalid table generated") + } + + tableSz, err = s.cTable.estTableSize(s) + if err != nil { + return 0, 0, 0, err + } + if canReuse { + reuseSz = s.prevTable.estimateSize(s.count[:s.symbolLen]) + } + dataSz = s.cTable.estimateSize(s.count[:s.symbolLen]) + + // Restore + return tableSz, dataSz, reuseSz, nil +} + func (s *Scratch) compress1X(src []byte) ([]byte, error) { return s.compress1xDo(s.Out, src) } diff --git a/vendor/github.com/klauspost/compress/huff0/decompress.go b/vendor/github.com/klauspost/compress/huff0/decompress.go index 41703bba4d65..9b7cc8e97bb9 100644 --- a/vendor/github.com/klauspost/compress/huff0/decompress.go +++ b/vendor/github.com/klauspost/compress/huff0/decompress.go @@ -344,35 +344,241 @@ func (d *Decoder) decompress1X8Bit(dst, src []byte) ([]byte, error) { var buf [256]byte var off uint8 - shift := (8 - d.actualTableLog) & 7 - - //fmt.Printf("mask: %b, tl:%d\n", mask, d.actualTableLog) - for br.off >= 4 { - br.fillFast() - v := dt[br.peekByteFast()>>shift] - br.advance(uint8(v.entry)) - buf[off+0] = uint8(v.entry >> 8) - - v = dt[br.peekByteFast()>>shift] - br.advance(uint8(v.entry)) - buf[off+1] = uint8(v.entry >> 8) - - v = dt[br.peekByteFast()>>shift] - br.advance(uint8(v.entry)) - buf[off+2] = uint8(v.entry >> 8) - - v = dt[br.peekByteFast()>>shift] - br.advance(uint8(v.entry)) - buf[off+3] = uint8(v.entry >> 8) - - off += 4 - if off == 0 { - if len(dst)+256 > maxDecodedSize { - br.close() - return nil, ErrMaxDecodedSizeExceeded + switch d.actualTableLog { + case 8: + const shift = 8 - 8 + for br.off >= 4 { + br.fillFast() + v := dt[uint8(br.value>>(56+shift))] + br.advance(uint8(v.entry)) + buf[off+0] = uint8(v.entry >> 8) + + v = dt[uint8(br.value>>(56+shift))] + br.advance(uint8(v.entry)) + buf[off+1] = uint8(v.entry >> 8) + + v = dt[uint8(br.value>>(56+shift))] + br.advance(uint8(v.entry)) + buf[off+2] = uint8(v.entry >> 8) + + v = dt[uint8(br.value>>(56+shift))] + br.advance(uint8(v.entry)) + buf[off+3] = uint8(v.entry >> 8) + + off += 4 + if off == 0 { + if len(dst)+256 > maxDecodedSize { + br.close() + return nil, ErrMaxDecodedSizeExceeded + } + dst = append(dst, buf[:]...) + } + } + case 7: + const shift = 8 - 7 + for br.off >= 4 { + br.fillFast() + v := dt[uint8(br.value>>(56+shift))] + br.advance(uint8(v.entry)) + buf[off+0] = uint8(v.entry >> 8) + + v = dt[uint8(br.value>>(56+shift))] + br.advance(uint8(v.entry)) + buf[off+1] = uint8(v.entry >> 8) + + v = dt[uint8(br.value>>(56+shift))] + br.advance(uint8(v.entry)) + buf[off+2] = uint8(v.entry >> 8) + + v = dt[uint8(br.value>>(56+shift))] + br.advance(uint8(v.entry)) + buf[off+3] = uint8(v.entry >> 8) + + off += 4 + if off == 0 { + if len(dst)+256 > maxDecodedSize { + br.close() + return nil, ErrMaxDecodedSizeExceeded + } + dst = append(dst, buf[:]...) + } + } + case 6: + const shift = 8 - 6 + for br.off >= 4 { + br.fillFast() + v := dt[uint8(br.value>>(56+shift))] + br.advance(uint8(v.entry)) + buf[off+0] = uint8(v.entry >> 8) + + v = dt[uint8(br.value>>(56+shift))] + br.advance(uint8(v.entry)) + buf[off+1] = uint8(v.entry >> 8) + + v = dt[uint8(br.value>>(56+shift))] + br.advance(uint8(v.entry)) + buf[off+2] = uint8(v.entry >> 8) + + v = dt[uint8(br.value>>(56+shift))] + br.advance(uint8(v.entry)) + buf[off+3] = uint8(v.entry >> 8) + + off += 4 + if off == 0 { + if len(dst)+256 > maxDecodedSize { + br.close() + return nil, ErrMaxDecodedSizeExceeded + } + dst = append(dst, buf[:]...) + } + } + case 5: + const shift = 8 - 5 + for br.off >= 4 { + br.fillFast() + v := dt[uint8(br.value>>(56+shift))] + br.advance(uint8(v.entry)) + buf[off+0] = uint8(v.entry >> 8) + + v = dt[uint8(br.value>>(56+shift))] + br.advance(uint8(v.entry)) + buf[off+1] = uint8(v.entry >> 8) + + v = dt[uint8(br.value>>(56+shift))] + br.advance(uint8(v.entry)) + buf[off+2] = uint8(v.entry >> 8) + + v = dt[uint8(br.value>>(56+shift))] + br.advance(uint8(v.entry)) + buf[off+3] = uint8(v.entry >> 8) + + off += 4 + if off == 0 { + if len(dst)+256 > maxDecodedSize { + br.close() + return nil, ErrMaxDecodedSizeExceeded + } + dst = append(dst, buf[:]...) + } + } + case 4: + const shift = 8 - 4 + for br.off >= 4 { + br.fillFast() + v := dt[uint8(br.value>>(56+shift))] + br.advance(uint8(v.entry)) + buf[off+0] = uint8(v.entry >> 8) + + v = dt[uint8(br.value>>(56+shift))] + br.advance(uint8(v.entry)) + buf[off+1] = uint8(v.entry >> 8) + + v = dt[uint8(br.value>>(56+shift))] + br.advance(uint8(v.entry)) + buf[off+2] = uint8(v.entry >> 8) + + v = dt[uint8(br.value>>(56+shift))] + br.advance(uint8(v.entry)) + buf[off+3] = uint8(v.entry >> 8) + + off += 4 + if off == 0 { + if len(dst)+256 > maxDecodedSize { + br.close() + return nil, ErrMaxDecodedSizeExceeded + } + dst = append(dst, buf[:]...) + } + } + case 3: + const shift = 8 - 3 + for br.off >= 4 { + br.fillFast() + v := dt[uint8(br.value>>(56+shift))] + br.advance(uint8(v.entry)) + buf[off+0] = uint8(v.entry >> 8) + + v = dt[uint8(br.value>>(56+shift))] + br.advance(uint8(v.entry)) + buf[off+1] = uint8(v.entry >> 8) + + v = dt[uint8(br.value>>(56+shift))] + br.advance(uint8(v.entry)) + buf[off+2] = uint8(v.entry >> 8) + + v = dt[uint8(br.value>>(56+shift))] + br.advance(uint8(v.entry)) + buf[off+3] = uint8(v.entry >> 8) + + off += 4 + if off == 0 { + if len(dst)+256 > maxDecodedSize { + br.close() + return nil, ErrMaxDecodedSizeExceeded + } + dst = append(dst, buf[:]...) + } + } + case 2: + const shift = 8 - 2 + for br.off >= 4 { + br.fillFast() + v := dt[uint8(br.value>>(56+shift))] + br.advance(uint8(v.entry)) + buf[off+0] = uint8(v.entry >> 8) + + v = dt[uint8(br.value>>(56+shift))] + br.advance(uint8(v.entry)) + buf[off+1] = uint8(v.entry >> 8) + + v = dt[uint8(br.value>>(56+shift))] + br.advance(uint8(v.entry)) + buf[off+2] = uint8(v.entry >> 8) + + v = dt[uint8(br.value>>(56+shift))] + br.advance(uint8(v.entry)) + buf[off+3] = uint8(v.entry >> 8) + + off += 4 + if off == 0 { + if len(dst)+256 > maxDecodedSize { + br.close() + return nil, ErrMaxDecodedSizeExceeded + } + dst = append(dst, buf[:]...) + } + } + case 1: + const shift = 8 - 1 + for br.off >= 4 { + br.fillFast() + v := dt[uint8(br.value>>(56+shift))] + br.advance(uint8(v.entry)) + buf[off+0] = uint8(v.entry >> 8) + + v = dt[uint8(br.value>>(56+shift))] + br.advance(uint8(v.entry)) + buf[off+1] = uint8(v.entry >> 8) + + v = dt[uint8(br.value>>(56+shift))] + br.advance(uint8(v.entry)) + buf[off+2] = uint8(v.entry >> 8) + + v = dt[uint8(br.value>>(56+shift))] + br.advance(uint8(v.entry)) + buf[off+3] = uint8(v.entry >> 8) + + off += 4 + if off == 0 { + if len(dst)+256 > maxDecodedSize { + br.close() + return nil, ErrMaxDecodedSizeExceeded + } + dst = append(dst, buf[:]...) } - dst = append(dst, buf[:]...) } + default: + return nil, fmt.Errorf("invalid tablelog: %d", d.actualTableLog) } if len(dst)+int(off) > maxDecodedSize { @@ -383,6 +589,8 @@ func (d *Decoder) decompress1X8Bit(dst, src []byte) ([]byte, error) { // br < 4, so uint8 is fine bitsLeft := int8(uint8(br.off)*8 + (64 - br.bitsRead)) + shift := (8 - d.actualTableLog) & 7 + for bitsLeft > 0 { if br.bitsRead >= 64-8 { for br.off > 0 { @@ -423,24 +631,24 @@ func (d *Decoder) decompress1X8BitExactly(dst, src []byte) ([]byte, error) { var buf [256]byte var off uint8 - const shift = 0 + const shift = 56 //fmt.Printf("mask: %b, tl:%d\n", mask, d.actualTableLog) for br.off >= 4 { br.fillFast() - v := dt[br.peekByteFast()>>shift] + v := dt[uint8(br.value>>shift)] br.advance(uint8(v.entry)) buf[off+0] = uint8(v.entry >> 8) - v = dt[br.peekByteFast()>>shift] + v = dt[uint8(br.value>>shift)] br.advance(uint8(v.entry)) buf[off+1] = uint8(v.entry >> 8) - v = dt[br.peekByteFast()>>shift] + v = dt[uint8(br.value>>shift)] br.advance(uint8(v.entry)) buf[off+2] = uint8(v.entry >> 8) - v = dt[br.peekByteFast()>>shift] + v = dt[uint8(br.value>>shift)] br.advance(uint8(v.entry)) buf[off+3] = uint8(v.entry >> 8) @@ -474,7 +682,7 @@ func (d *Decoder) decompress1X8BitExactly(dst, src []byte) ([]byte, error) { br.close() return nil, ErrMaxDecodedSizeExceeded } - v := dt[br.peekByteFast()>>shift] + v := dt[br.peekByteFast()] nBits := uint8(v.entry) br.advance(nBits) bitsLeft -= int8(nBits) @@ -709,7 +917,6 @@ func (d *Decoder) decompress4X8bit(dst, src []byte) ([]byte, error) { shift := (8 - d.actualTableLog) & 7 const tlSize = 1 << 8 - const tlMask = tlSize - 1 single := d.dt.single[:tlSize] // Use temp table to avoid bound checks/append penalty. diff --git a/vendor/github.com/klauspost/compress/huff0/huff0.go b/vendor/github.com/klauspost/compress/huff0/huff0.go index 7ec2022b6506..3ee00ecb470a 100644 --- a/vendor/github.com/klauspost/compress/huff0/huff0.go +++ b/vendor/github.com/klauspost/compress/huff0/huff0.go @@ -245,6 +245,68 @@ func (c cTable) write(s *Scratch) error { return nil } +func (c cTable) estTableSize(s *Scratch) (sz int, err error) { + var ( + // precomputed conversion table + bitsToWeight [tableLogMax + 1]byte + huffLog = s.actualTableLog + // last weight is not saved. + maxSymbolValue = uint8(s.symbolLen - 1) + huffWeight = s.huffWeight[:256] + ) + const ( + maxFSETableLog = 6 + ) + // convert to weight + bitsToWeight[0] = 0 + for n := uint8(1); n < huffLog+1; n++ { + bitsToWeight[n] = huffLog + 1 - n + } + + // Acquire histogram for FSE. + hist := s.fse.Histogram() + hist = hist[:256] + for i := range hist[:16] { + hist[i] = 0 + } + for n := uint8(0); n < maxSymbolValue; n++ { + v := bitsToWeight[c[n].nBits] & 15 + huffWeight[n] = v + hist[v]++ + } + + // FSE compress if feasible. + if maxSymbolValue >= 2 { + huffMaxCnt := uint32(0) + huffMax := uint8(0) + for i, v := range hist[:16] { + if v == 0 { + continue + } + huffMax = byte(i) + if v > huffMaxCnt { + huffMaxCnt = v + } + } + s.fse.HistogramFinished(huffMax, int(huffMaxCnt)) + s.fse.TableLog = maxFSETableLog + b, err := fse.Compress(huffWeight[:maxSymbolValue], s.fse) + if err == nil && len(b) < int(s.symbolLen>>1) { + sz += 1 + len(b) + return sz, nil + } + // Unable to compress (RLE/uncompressible) + } + // write raw values as 4-bits (max : 15) + if maxSymbolValue > (256 - 128) { + // should not happen : likely means source cannot be compressed + return 0, ErrIncompressible + } + // special case, pack weights 4 bits/weight. + sz += 1 + int(maxSymbolValue/2) + return sz, nil +} + // estimateSize returns the estimated size in bytes of the input represented in the // histogram supplied. func (c cTable) estimateSize(hist []uint32) int { diff --git a/vendor/github.com/klauspost/compress/internal/snapref/LICENSE b/vendor/github.com/klauspost/compress/internal/snapref/LICENSE new file mode 100644 index 000000000000..6050c10f4c8b --- /dev/null +++ b/vendor/github.com/klauspost/compress/internal/snapref/LICENSE @@ -0,0 +1,27 @@ +Copyright (c) 2011 The Snappy-Go Authors. All rights reserved. + +Redistribution and use in source and binary forms, with or without +modification, are permitted provided that the following conditions are +met: + + * Redistributions of source code must retain the above copyright +notice, this list of conditions and the following disclaimer. + * Redistributions in binary form must reproduce the above +copyright notice, this list of conditions and the following disclaimer +in the documentation and/or other materials provided with the +distribution. + * Neither the name of Google Inc. nor the names of its +contributors may be used to endorse or promote products derived from +this software without specific prior written permission. + +THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS +"AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT +LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR +A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT +OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, +SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT +LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, +DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY +THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT +(INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE +OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. diff --git a/vendor/github.com/klauspost/compress/internal/snapref/decode.go b/vendor/github.com/klauspost/compress/internal/snapref/decode.go new file mode 100644 index 000000000000..40796a49d659 --- /dev/null +++ b/vendor/github.com/klauspost/compress/internal/snapref/decode.go @@ -0,0 +1,264 @@ +// Copyright 2011 The Snappy-Go Authors. All rights reserved. +// Use of this source code is governed by a BSD-style +// license that can be found in the LICENSE file. + +package snapref + +import ( + "encoding/binary" + "errors" + "io" +) + +var ( + // ErrCorrupt reports that the input is invalid. + ErrCorrupt = errors.New("snappy: corrupt input") + // ErrTooLarge reports that the uncompressed length is too large. + ErrTooLarge = errors.New("snappy: decoded block is too large") + // ErrUnsupported reports that the input isn't supported. + ErrUnsupported = errors.New("snappy: unsupported input") + + errUnsupportedLiteralLength = errors.New("snappy: unsupported literal length") +) + +// DecodedLen returns the length of the decoded block. +func DecodedLen(src []byte) (int, error) { + v, _, err := decodedLen(src) + return v, err +} + +// decodedLen returns the length of the decoded block and the number of bytes +// that the length header occupied. +func decodedLen(src []byte) (blockLen, headerLen int, err error) { + v, n := binary.Uvarint(src) + if n <= 0 || v > 0xffffffff { + return 0, 0, ErrCorrupt + } + + const wordSize = 32 << (^uint(0) >> 32 & 1) + if wordSize == 32 && v > 0x7fffffff { + return 0, 0, ErrTooLarge + } + return int(v), n, nil +} + +const ( + decodeErrCodeCorrupt = 1 + decodeErrCodeUnsupportedLiteralLength = 2 +) + +// Decode returns the decoded form of src. The returned slice may be a sub- +// slice of dst if dst was large enough to hold the entire decoded block. +// Otherwise, a newly allocated slice will be returned. +// +// The dst and src must not overlap. It is valid to pass a nil dst. +// +// Decode handles the Snappy block format, not the Snappy stream format. +func Decode(dst, src []byte) ([]byte, error) { + dLen, s, err := decodedLen(src) + if err != nil { + return nil, err + } + if dLen <= len(dst) { + dst = dst[:dLen] + } else { + dst = make([]byte, dLen) + } + switch decode(dst, src[s:]) { + case 0: + return dst, nil + case decodeErrCodeUnsupportedLiteralLength: + return nil, errUnsupportedLiteralLength + } + return nil, ErrCorrupt +} + +// NewReader returns a new Reader that decompresses from r, using the framing +// format described at +// https://github.com/google/snappy/blob/master/framing_format.txt +func NewReader(r io.Reader) *Reader { + return &Reader{ + r: r, + decoded: make([]byte, maxBlockSize), + buf: make([]byte, maxEncodedLenOfMaxBlockSize+checksumSize), + } +} + +// Reader is an io.Reader that can read Snappy-compressed bytes. +// +// Reader handles the Snappy stream format, not the Snappy block format. +type Reader struct { + r io.Reader + err error + decoded []byte + buf []byte + // decoded[i:j] contains decoded bytes that have not yet been passed on. + i, j int + readHeader bool +} + +// Reset discards any buffered data, resets all state, and switches the Snappy +// reader to read from r. This permits reusing a Reader rather than allocating +// a new one. +func (r *Reader) Reset(reader io.Reader) { + r.r = reader + r.err = nil + r.i = 0 + r.j = 0 + r.readHeader = false +} + +func (r *Reader) readFull(p []byte, allowEOF bool) (ok bool) { + if _, r.err = io.ReadFull(r.r, p); r.err != nil { + if r.err == io.ErrUnexpectedEOF || (r.err == io.EOF && !allowEOF) { + r.err = ErrCorrupt + } + return false + } + return true +} + +func (r *Reader) fill() error { + for r.i >= r.j { + if !r.readFull(r.buf[:4], true) { + return r.err + } + chunkType := r.buf[0] + if !r.readHeader { + if chunkType != chunkTypeStreamIdentifier { + r.err = ErrCorrupt + return r.err + } + r.readHeader = true + } + chunkLen := int(r.buf[1]) | int(r.buf[2])<<8 | int(r.buf[3])<<16 + if chunkLen > len(r.buf) { + r.err = ErrUnsupported + return r.err + } + + // The chunk types are specified at + // https://github.com/google/snappy/blob/master/framing_format.txt + switch chunkType { + case chunkTypeCompressedData: + // Section 4.2. Compressed data (chunk type 0x00). + if chunkLen < checksumSize { + r.err = ErrCorrupt + return r.err + } + buf := r.buf[:chunkLen] + if !r.readFull(buf, false) { + return r.err + } + checksum := uint32(buf[0]) | uint32(buf[1])<<8 | uint32(buf[2])<<16 | uint32(buf[3])<<24 + buf = buf[checksumSize:] + + n, err := DecodedLen(buf) + if err != nil { + r.err = err + return r.err + } + if n > len(r.decoded) { + r.err = ErrCorrupt + return r.err + } + if _, err := Decode(r.decoded, buf); err != nil { + r.err = err + return r.err + } + if crc(r.decoded[:n]) != checksum { + r.err = ErrCorrupt + return r.err + } + r.i, r.j = 0, n + continue + + case chunkTypeUncompressedData: + // Section 4.3. Uncompressed data (chunk type 0x01). + if chunkLen < checksumSize { + r.err = ErrCorrupt + return r.err + } + buf := r.buf[:checksumSize] + if !r.readFull(buf, false) { + return r.err + } + checksum := uint32(buf[0]) | uint32(buf[1])<<8 | uint32(buf[2])<<16 | uint32(buf[3])<<24 + // Read directly into r.decoded instead of via r.buf. + n := chunkLen - checksumSize + if n > len(r.decoded) { + r.err = ErrCorrupt + return r.err + } + if !r.readFull(r.decoded[:n], false) { + return r.err + } + if crc(r.decoded[:n]) != checksum { + r.err = ErrCorrupt + return r.err + } + r.i, r.j = 0, n + continue + + case chunkTypeStreamIdentifier: + // Section 4.1. Stream identifier (chunk type 0xff). + if chunkLen != len(magicBody) { + r.err = ErrCorrupt + return r.err + } + if !r.readFull(r.buf[:len(magicBody)], false) { + return r.err + } + for i := 0; i < len(magicBody); i++ { + if r.buf[i] != magicBody[i] { + r.err = ErrCorrupt + return r.err + } + } + continue + } + + if chunkType <= 0x7f { + // Section 4.5. Reserved unskippable chunks (chunk types 0x02-0x7f). + r.err = ErrUnsupported + return r.err + } + // Section 4.4 Padding (chunk type 0xfe). + // Section 4.6. Reserved skippable chunks (chunk types 0x80-0xfd). + if !r.readFull(r.buf[:chunkLen], false) { + return r.err + } + } + + return nil +} + +// Read satisfies the io.Reader interface. +func (r *Reader) Read(p []byte) (int, error) { + if r.err != nil { + return 0, r.err + } + + if err := r.fill(); err != nil { + return 0, err + } + + n := copy(p, r.decoded[r.i:r.j]) + r.i += n + return n, nil +} + +// ReadByte satisfies the io.ByteReader interface. +func (r *Reader) ReadByte() (byte, error) { + if r.err != nil { + return 0, r.err + } + + if err := r.fill(); err != nil { + return 0, err + } + + c := r.decoded[r.i] + r.i++ + return c, nil +} diff --git a/vendor/github.com/klauspost/compress/internal/snapref/decode_other.go b/vendor/github.com/klauspost/compress/internal/snapref/decode_other.go new file mode 100644 index 000000000000..77395a6b8b9e --- /dev/null +++ b/vendor/github.com/klauspost/compress/internal/snapref/decode_other.go @@ -0,0 +1,113 @@ +// Copyright 2016 The Snappy-Go Authors. All rights reserved. +// Use of this source code is governed by a BSD-style +// license that can be found in the LICENSE file. + +package snapref + +// decode writes the decoding of src to dst. It assumes that the varint-encoded +// length of the decompressed bytes has already been read, and that len(dst) +// equals that length. +// +// It returns 0 on success or a decodeErrCodeXxx error code on failure. +func decode(dst, src []byte) int { + var d, s, offset, length int + for s < len(src) { + switch src[s] & 0x03 { + case tagLiteral: + x := uint32(src[s] >> 2) + switch { + case x < 60: + s++ + case x == 60: + s += 2 + if uint(s) > uint(len(src)) { // The uint conversions catch overflow from the previous line. + return decodeErrCodeCorrupt + } + x = uint32(src[s-1]) + case x == 61: + s += 3 + if uint(s) > uint(len(src)) { // The uint conversions catch overflow from the previous line. + return decodeErrCodeCorrupt + } + x = uint32(src[s-2]) | uint32(src[s-1])<<8 + case x == 62: + s += 4 + if uint(s) > uint(len(src)) { // The uint conversions catch overflow from the previous line. + return decodeErrCodeCorrupt + } + x = uint32(src[s-3]) | uint32(src[s-2])<<8 | uint32(src[s-1])<<16 + case x == 63: + s += 5 + if uint(s) > uint(len(src)) { // The uint conversions catch overflow from the previous line. + return decodeErrCodeCorrupt + } + x = uint32(src[s-4]) | uint32(src[s-3])<<8 | uint32(src[s-2])<<16 | uint32(src[s-1])<<24 + } + length = int(x) + 1 + if length <= 0 { + return decodeErrCodeUnsupportedLiteralLength + } + if length > len(dst)-d || length > len(src)-s { + return decodeErrCodeCorrupt + } + copy(dst[d:], src[s:s+length]) + d += length + s += length + continue + + case tagCopy1: + s += 2 + if uint(s) > uint(len(src)) { // The uint conversions catch overflow from the previous line. + return decodeErrCodeCorrupt + } + length = 4 + int(src[s-2])>>2&0x7 + offset = int(uint32(src[s-2])&0xe0<<3 | uint32(src[s-1])) + + case tagCopy2: + s += 3 + if uint(s) > uint(len(src)) { // The uint conversions catch overflow from the previous line. + return decodeErrCodeCorrupt + } + length = 1 + int(src[s-3])>>2 + offset = int(uint32(src[s-2]) | uint32(src[s-1])<<8) + + case tagCopy4: + s += 5 + if uint(s) > uint(len(src)) { // The uint conversions catch overflow from the previous line. + return decodeErrCodeCorrupt + } + length = 1 + int(src[s-5])>>2 + offset = int(uint32(src[s-4]) | uint32(src[s-3])<<8 | uint32(src[s-2])<<16 | uint32(src[s-1])<<24) + } + + if offset <= 0 || d < offset || length > len(dst)-d { + return decodeErrCodeCorrupt + } + // Copy from an earlier sub-slice of dst to a later sub-slice. + // If no overlap, use the built-in copy: + if offset >= length { + copy(dst[d:d+length], dst[d-offset:]) + d += length + continue + } + + // Unlike the built-in copy function, this byte-by-byte copy always runs + // forwards, even if the slices overlap. Conceptually, this is: + // + // d += forwardCopy(dst[d:d+length], dst[d-offset:]) + // + // We align the slices into a and b and show the compiler they are the same size. + // This allows the loop to run without bounds checks. + a := dst[d : d+length] + b := dst[d-offset:] + b = b[:len(a)] + for i := range a { + a[i] = b[i] + } + d += length + } + if d != len(dst) { + return decodeErrCodeCorrupt + } + return 0 +} diff --git a/vendor/github.com/klauspost/compress/internal/snapref/encode.go b/vendor/github.com/klauspost/compress/internal/snapref/encode.go new file mode 100644 index 000000000000..13c6040a5ded --- /dev/null +++ b/vendor/github.com/klauspost/compress/internal/snapref/encode.go @@ -0,0 +1,289 @@ +// Copyright 2011 The Snappy-Go Authors. All rights reserved. +// Use of this source code is governed by a BSD-style +// license that can be found in the LICENSE file. + +package snapref + +import ( + "encoding/binary" + "errors" + "io" +) + +// Encode returns the encoded form of src. The returned slice may be a sub- +// slice of dst if dst was large enough to hold the entire encoded block. +// Otherwise, a newly allocated slice will be returned. +// +// The dst and src must not overlap. It is valid to pass a nil dst. +// +// Encode handles the Snappy block format, not the Snappy stream format. +func Encode(dst, src []byte) []byte { + if n := MaxEncodedLen(len(src)); n < 0 { + panic(ErrTooLarge) + } else if len(dst) < n { + dst = make([]byte, n) + } + + // The block starts with the varint-encoded length of the decompressed bytes. + d := binary.PutUvarint(dst, uint64(len(src))) + + for len(src) > 0 { + p := src + src = nil + if len(p) > maxBlockSize { + p, src = p[:maxBlockSize], p[maxBlockSize:] + } + if len(p) < minNonLiteralBlockSize { + d += emitLiteral(dst[d:], p) + } else { + d += encodeBlock(dst[d:], p) + } + } + return dst[:d] +} + +// inputMargin is the minimum number of extra input bytes to keep, inside +// encodeBlock's inner loop. On some architectures, this margin lets us +// implement a fast path for emitLiteral, where the copy of short (<= 16 byte) +// literals can be implemented as a single load to and store from a 16-byte +// register. That literal's actual length can be as short as 1 byte, so this +// can copy up to 15 bytes too much, but that's OK as subsequent iterations of +// the encoding loop will fix up the copy overrun, and this inputMargin ensures +// that we don't overrun the dst and src buffers. +const inputMargin = 16 - 1 + +// minNonLiteralBlockSize is the minimum size of the input to encodeBlock that +// could be encoded with a copy tag. This is the minimum with respect to the +// algorithm used by encodeBlock, not a minimum enforced by the file format. +// +// The encoded output must start with at least a 1 byte literal, as there are +// no previous bytes to copy. A minimal (1 byte) copy after that, generated +// from an emitCopy call in encodeBlock's main loop, would require at least +// another inputMargin bytes, for the reason above: we want any emitLiteral +// calls inside encodeBlock's main loop to use the fast path if possible, which +// requires being able to overrun by inputMargin bytes. Thus, +// minNonLiteralBlockSize equals 1 + 1 + inputMargin. +// +// The C++ code doesn't use this exact threshold, but it could, as discussed at +// https://groups.google.com/d/topic/snappy-compression/oGbhsdIJSJ8/discussion +// The difference between Go (2+inputMargin) and C++ (inputMargin) is purely an +// optimization. It should not affect the encoded form. This is tested by +// TestSameEncodingAsCppShortCopies. +const minNonLiteralBlockSize = 1 + 1 + inputMargin + +// MaxEncodedLen returns the maximum length of a snappy block, given its +// uncompressed length. +// +// It will return a negative value if srcLen is too large to encode. +func MaxEncodedLen(srcLen int) int { + n := uint64(srcLen) + if n > 0xffffffff { + return -1 + } + // Compressed data can be defined as: + // compressed := item* literal* + // item := literal* copy + // + // The trailing literal sequence has a space blowup of at most 62/60 + // since a literal of length 60 needs one tag byte + one extra byte + // for length information. + // + // Item blowup is trickier to measure. Suppose the "copy" op copies + // 4 bytes of data. Because of a special check in the encoding code, + // we produce a 4-byte copy only if the offset is < 65536. Therefore + // the copy op takes 3 bytes to encode, and this type of item leads + // to at most the 62/60 blowup for representing literals. + // + // Suppose the "copy" op copies 5 bytes of data. If the offset is big + // enough, it will take 5 bytes to encode the copy op. Therefore the + // worst case here is a one-byte literal followed by a five-byte copy. + // That is, 6 bytes of input turn into 7 bytes of "compressed" data. + // + // This last factor dominates the blowup, so the final estimate is: + n = 32 + n + n/6 + if n > 0xffffffff { + return -1 + } + return int(n) +} + +var errClosed = errors.New("snappy: Writer is closed") + +// NewWriter returns a new Writer that compresses to w. +// +// The Writer returned does not buffer writes. There is no need to Flush or +// Close such a Writer. +// +// Deprecated: the Writer returned is not suitable for many small writes, only +// for few large writes. Use NewBufferedWriter instead, which is efficient +// regardless of the frequency and shape of the writes, and remember to Close +// that Writer when done. +func NewWriter(w io.Writer) *Writer { + return &Writer{ + w: w, + obuf: make([]byte, obufLen), + } +} + +// NewBufferedWriter returns a new Writer that compresses to w, using the +// framing format described at +// https://github.com/google/snappy/blob/master/framing_format.txt +// +// The Writer returned buffers writes. Users must call Close to guarantee all +// data has been forwarded to the underlying io.Writer. They may also call +// Flush zero or more times before calling Close. +func NewBufferedWriter(w io.Writer) *Writer { + return &Writer{ + w: w, + ibuf: make([]byte, 0, maxBlockSize), + obuf: make([]byte, obufLen), + } +} + +// Writer is an io.Writer that can write Snappy-compressed bytes. +// +// Writer handles the Snappy stream format, not the Snappy block format. +type Writer struct { + w io.Writer + err error + + // ibuf is a buffer for the incoming (uncompressed) bytes. + // + // Its use is optional. For backwards compatibility, Writers created by the + // NewWriter function have ibuf == nil, do not buffer incoming bytes, and + // therefore do not need to be Flush'ed or Close'd. + ibuf []byte + + // obuf is a buffer for the outgoing (compressed) bytes. + obuf []byte + + // wroteStreamHeader is whether we have written the stream header. + wroteStreamHeader bool +} + +// Reset discards the writer's state and switches the Snappy writer to write to +// w. This permits reusing a Writer rather than allocating a new one. +func (w *Writer) Reset(writer io.Writer) { + w.w = writer + w.err = nil + if w.ibuf != nil { + w.ibuf = w.ibuf[:0] + } + w.wroteStreamHeader = false +} + +// Write satisfies the io.Writer interface. +func (w *Writer) Write(p []byte) (nRet int, errRet error) { + if w.ibuf == nil { + // Do not buffer incoming bytes. This does not perform or compress well + // if the caller of Writer.Write writes many small slices. This + // behavior is therefore deprecated, but still supported for backwards + // compatibility with code that doesn't explicitly Flush or Close. + return w.write(p) + } + + // The remainder of this method is based on bufio.Writer.Write from the + // standard library. + + for len(p) > (cap(w.ibuf)-len(w.ibuf)) && w.err == nil { + var n int + if len(w.ibuf) == 0 { + // Large write, empty buffer. + // Write directly from p to avoid copy. + n, _ = w.write(p) + } else { + n = copy(w.ibuf[len(w.ibuf):cap(w.ibuf)], p) + w.ibuf = w.ibuf[:len(w.ibuf)+n] + w.Flush() + } + nRet += n + p = p[n:] + } + if w.err != nil { + return nRet, w.err + } + n := copy(w.ibuf[len(w.ibuf):cap(w.ibuf)], p) + w.ibuf = w.ibuf[:len(w.ibuf)+n] + nRet += n + return nRet, nil +} + +func (w *Writer) write(p []byte) (nRet int, errRet error) { + if w.err != nil { + return 0, w.err + } + for len(p) > 0 { + obufStart := len(magicChunk) + if !w.wroteStreamHeader { + w.wroteStreamHeader = true + copy(w.obuf, magicChunk) + obufStart = 0 + } + + var uncompressed []byte + if len(p) > maxBlockSize { + uncompressed, p = p[:maxBlockSize], p[maxBlockSize:] + } else { + uncompressed, p = p, nil + } + checksum := crc(uncompressed) + + // Compress the buffer, discarding the result if the improvement + // isn't at least 12.5%. + compressed := Encode(w.obuf[obufHeaderLen:], uncompressed) + chunkType := uint8(chunkTypeCompressedData) + chunkLen := 4 + len(compressed) + obufEnd := obufHeaderLen + len(compressed) + if len(compressed) >= len(uncompressed)-len(uncompressed)/8 { + chunkType = chunkTypeUncompressedData + chunkLen = 4 + len(uncompressed) + obufEnd = obufHeaderLen + } + + // Fill in the per-chunk header that comes before the body. + w.obuf[len(magicChunk)+0] = chunkType + w.obuf[len(magicChunk)+1] = uint8(chunkLen >> 0) + w.obuf[len(magicChunk)+2] = uint8(chunkLen >> 8) + w.obuf[len(magicChunk)+3] = uint8(chunkLen >> 16) + w.obuf[len(magicChunk)+4] = uint8(checksum >> 0) + w.obuf[len(magicChunk)+5] = uint8(checksum >> 8) + w.obuf[len(magicChunk)+6] = uint8(checksum >> 16) + w.obuf[len(magicChunk)+7] = uint8(checksum >> 24) + + if _, err := w.w.Write(w.obuf[obufStart:obufEnd]); err != nil { + w.err = err + return nRet, err + } + if chunkType == chunkTypeUncompressedData { + if _, err := w.w.Write(uncompressed); err != nil { + w.err = err + return nRet, err + } + } + nRet += len(uncompressed) + } + return nRet, nil +} + +// Flush flushes the Writer to its underlying io.Writer. +func (w *Writer) Flush() error { + if w.err != nil { + return w.err + } + if len(w.ibuf) == 0 { + return nil + } + w.write(w.ibuf) + w.ibuf = w.ibuf[:0] + return w.err +} + +// Close calls Flush and then closes the Writer. +func (w *Writer) Close() error { + w.Flush() + ret := w.err + if w.err == nil { + w.err = errClosed + } + return ret +} diff --git a/vendor/github.com/klauspost/compress/internal/snapref/encode_other.go b/vendor/github.com/klauspost/compress/internal/snapref/encode_other.go new file mode 100644 index 000000000000..511bba65db8f --- /dev/null +++ b/vendor/github.com/klauspost/compress/internal/snapref/encode_other.go @@ -0,0 +1,236 @@ +// Copyright 2016 The Snappy-Go Authors. All rights reserved. +// Use of this source code is governed by a BSD-style +// license that can be found in the LICENSE file. + +package snapref + +func load32(b []byte, i int) uint32 { + b = b[i : i+4 : len(b)] // Help the compiler eliminate bounds checks on the next line. + return uint32(b[0]) | uint32(b[1])<<8 | uint32(b[2])<<16 | uint32(b[3])<<24 +} + +func load64(b []byte, i int) uint64 { + b = b[i : i+8 : len(b)] // Help the compiler eliminate bounds checks on the next line. + return uint64(b[0]) | uint64(b[1])<<8 | uint64(b[2])<<16 | uint64(b[3])<<24 | + uint64(b[4])<<32 | uint64(b[5])<<40 | uint64(b[6])<<48 | uint64(b[7])<<56 +} + +// emitLiteral writes a literal chunk and returns the number of bytes written. +// +// It assumes that: +// dst is long enough to hold the encoded bytes +// 1 <= len(lit) && len(lit) <= 65536 +func emitLiteral(dst, lit []byte) int { + i, n := 0, uint(len(lit)-1) + switch { + case n < 60: + dst[0] = uint8(n)<<2 | tagLiteral + i = 1 + case n < 1<<8: + dst[0] = 60<<2 | tagLiteral + dst[1] = uint8(n) + i = 2 + default: + dst[0] = 61<<2 | tagLiteral + dst[1] = uint8(n) + dst[2] = uint8(n >> 8) + i = 3 + } + return i + copy(dst[i:], lit) +} + +// emitCopy writes a copy chunk and returns the number of bytes written. +// +// It assumes that: +// dst is long enough to hold the encoded bytes +// 1 <= offset && offset <= 65535 +// 4 <= length && length <= 65535 +func emitCopy(dst []byte, offset, length int) int { + i := 0 + // The maximum length for a single tagCopy1 or tagCopy2 op is 64 bytes. The + // threshold for this loop is a little higher (at 68 = 64 + 4), and the + // length emitted down below is is a little lower (at 60 = 64 - 4), because + // it's shorter to encode a length 67 copy as a length 60 tagCopy2 followed + // by a length 7 tagCopy1 (which encodes as 3+2 bytes) than to encode it as + // a length 64 tagCopy2 followed by a length 3 tagCopy2 (which encodes as + // 3+3 bytes). The magic 4 in the 64±4 is because the minimum length for a + // tagCopy1 op is 4 bytes, which is why a length 3 copy has to be an + // encodes-as-3-bytes tagCopy2 instead of an encodes-as-2-bytes tagCopy1. + for length >= 68 { + // Emit a length 64 copy, encoded as 3 bytes. + dst[i+0] = 63<<2 | tagCopy2 + dst[i+1] = uint8(offset) + dst[i+2] = uint8(offset >> 8) + i += 3 + length -= 64 + } + if length > 64 { + // Emit a length 60 copy, encoded as 3 bytes. + dst[i+0] = 59<<2 | tagCopy2 + dst[i+1] = uint8(offset) + dst[i+2] = uint8(offset >> 8) + i += 3 + length -= 60 + } + if length >= 12 || offset >= 2048 { + // Emit the remaining copy, encoded as 3 bytes. + dst[i+0] = uint8(length-1)<<2 | tagCopy2 + dst[i+1] = uint8(offset) + dst[i+2] = uint8(offset >> 8) + return i + 3 + } + // Emit the remaining copy, encoded as 2 bytes. + dst[i+0] = uint8(offset>>8)<<5 | uint8(length-4)<<2 | tagCopy1 + dst[i+1] = uint8(offset) + return i + 2 +} + +// extendMatch returns the largest k such that k <= len(src) and that +// src[i:i+k-j] and src[j:k] have the same contents. +// +// It assumes that: +// 0 <= i && i < j && j <= len(src) +func extendMatch(src []byte, i, j int) int { + for ; j < len(src) && src[i] == src[j]; i, j = i+1, j+1 { + } + return j +} + +func hash(u, shift uint32) uint32 { + return (u * 0x1e35a7bd) >> shift +} + +// encodeBlock encodes a non-empty src to a guaranteed-large-enough dst. It +// assumes that the varint-encoded length of the decompressed bytes has already +// been written. +// +// It also assumes that: +// len(dst) >= MaxEncodedLen(len(src)) && +// minNonLiteralBlockSize <= len(src) && len(src) <= maxBlockSize +func encodeBlock(dst, src []byte) (d int) { + // Initialize the hash table. Its size ranges from 1<<8 to 1<<14 inclusive. + // The table element type is uint16, as s < sLimit and sLimit < len(src) + // and len(src) <= maxBlockSize and maxBlockSize == 65536. + const ( + maxTableSize = 1 << 14 + // tableMask is redundant, but helps the compiler eliminate bounds + // checks. + tableMask = maxTableSize - 1 + ) + shift := uint32(32 - 8) + for tableSize := 1 << 8; tableSize < maxTableSize && tableSize < len(src); tableSize *= 2 { + shift-- + } + // In Go, all array elements are zero-initialized, so there is no advantage + // to a smaller tableSize per se. However, it matches the C++ algorithm, + // and in the asm versions of this code, we can get away with zeroing only + // the first tableSize elements. + var table [maxTableSize]uint16 + + // sLimit is when to stop looking for offset/length copies. The inputMargin + // lets us use a fast path for emitLiteral in the main loop, while we are + // looking for copies. + sLimit := len(src) - inputMargin + + // nextEmit is where in src the next emitLiteral should start from. + nextEmit := 0 + + // The encoded form must start with a literal, as there are no previous + // bytes to copy, so we start looking for hash matches at s == 1. + s := 1 + nextHash := hash(load32(src, s), shift) + + for { + // Copied from the C++ snappy implementation: + // + // Heuristic match skipping: If 32 bytes are scanned with no matches + // found, start looking only at every other byte. If 32 more bytes are + // scanned (or skipped), look at every third byte, etc.. When a match + // is found, immediately go back to looking at every byte. This is a + // small loss (~5% performance, ~0.1% density) for compressible data + // due to more bookkeeping, but for non-compressible data (such as + // JPEG) it's a huge win since the compressor quickly "realizes" the + // data is incompressible and doesn't bother looking for matches + // everywhere. + // + // The "skip" variable keeps track of how many bytes there are since + // the last match; dividing it by 32 (ie. right-shifting by five) gives + // the number of bytes to move ahead for each iteration. + skip := 32 + + nextS := s + candidate := 0 + for { + s = nextS + bytesBetweenHashLookups := skip >> 5 + nextS = s + bytesBetweenHashLookups + skip += bytesBetweenHashLookups + if nextS > sLimit { + goto emitRemainder + } + candidate = int(table[nextHash&tableMask]) + table[nextHash&tableMask] = uint16(s) + nextHash = hash(load32(src, nextS), shift) + if load32(src, s) == load32(src, candidate) { + break + } + } + + // A 4-byte match has been found. We'll later see if more than 4 bytes + // match. But, prior to the match, src[nextEmit:s] are unmatched. Emit + // them as literal bytes. + d += emitLiteral(dst[d:], src[nextEmit:s]) + + // Call emitCopy, and then see if another emitCopy could be our next + // move. Repeat until we find no match for the input immediately after + // what was consumed by the last emitCopy call. + // + // If we exit this loop normally then we need to call emitLiteral next, + // though we don't yet know how big the literal will be. We handle that + // by proceeding to the next iteration of the main loop. We also can + // exit this loop via goto if we get close to exhausting the input. + for { + // Invariant: we have a 4-byte match at s, and no need to emit any + // literal bytes prior to s. + base := s + + // Extend the 4-byte match as long as possible. + // + // This is an inlined version of: + // s = extendMatch(src, candidate+4, s+4) + s += 4 + for i := candidate + 4; s < len(src) && src[i] == src[s]; i, s = i+1, s+1 { + } + + d += emitCopy(dst[d:], base-candidate, s-base) + nextEmit = s + if s >= sLimit { + goto emitRemainder + } + + // We could immediately start working at s now, but to improve + // compression we first update the hash table at s-1 and at s. If + // another emitCopy is not our next move, also calculate nextHash + // at s+1. At least on GOARCH=amd64, these three hash calculations + // are faster as one load64 call (with some shifts) instead of + // three load32 calls. + x := load64(src, s-1) + prevHash := hash(uint32(x>>0), shift) + table[prevHash&tableMask] = uint16(s - 1) + currHash := hash(uint32(x>>8), shift) + candidate = int(table[currHash&tableMask]) + table[currHash&tableMask] = uint16(s) + if uint32(x>>8) != load32(src, candidate) { + nextHash = hash(uint32(x>>16), shift) + s++ + break + } + } + } + +emitRemainder: + if nextEmit < len(src) { + d += emitLiteral(dst[d:], src[nextEmit:]) + } + return d +} diff --git a/vendor/github.com/klauspost/compress/internal/snapref/snappy.go b/vendor/github.com/klauspost/compress/internal/snapref/snappy.go new file mode 100644 index 000000000000..34d01f4aa63a --- /dev/null +++ b/vendor/github.com/klauspost/compress/internal/snapref/snappy.go @@ -0,0 +1,98 @@ +// Copyright 2011 The Snappy-Go Authors. All rights reserved. +// Use of this source code is governed by a BSD-style +// license that can be found in the LICENSE file. + +// Package snapref implements the Snappy compression format. It aims for very +// high speeds and reasonable compression. +// +// There are actually two Snappy formats: block and stream. They are related, +// but different: trying to decompress block-compressed data as a Snappy stream +// will fail, and vice versa. The block format is the Decode and Encode +// functions and the stream format is the Reader and Writer types. +// +// The block format, the more common case, is used when the complete size (the +// number of bytes) of the original data is known upfront, at the time +// compression starts. The stream format, also known as the framing format, is +// for when that isn't always true. +// +// The canonical, C++ implementation is at https://github.com/google/snappy and +// it only implements the block format. +package snapref + +import ( + "hash/crc32" +) + +/* +Each encoded block begins with the varint-encoded length of the decoded data, +followed by a sequence of chunks. Chunks begin and end on byte boundaries. The +first byte of each chunk is broken into its 2 least and 6 most significant bits +called l and m: l ranges in [0, 4) and m ranges in [0, 64). l is the chunk tag. +Zero means a literal tag. All other values mean a copy tag. + +For literal tags: + - If m < 60, the next 1 + m bytes are literal bytes. + - Otherwise, let n be the little-endian unsigned integer denoted by the next + m - 59 bytes. The next 1 + n bytes after that are literal bytes. + +For copy tags, length bytes are copied from offset bytes ago, in the style of +Lempel-Ziv compression algorithms. In particular: + - For l == 1, the offset ranges in [0, 1<<11) and the length in [4, 12). + The length is 4 + the low 3 bits of m. The high 3 bits of m form bits 8-10 + of the offset. The next byte is bits 0-7 of the offset. + - For l == 2, the offset ranges in [0, 1<<16) and the length in [1, 65). + The length is 1 + m. The offset is the little-endian unsigned integer + denoted by the next 2 bytes. + - For l == 3, this tag is a legacy format that is no longer issued by most + encoders. Nonetheless, the offset ranges in [0, 1<<32) and the length in + [1, 65). The length is 1 + m. The offset is the little-endian unsigned + integer denoted by the next 4 bytes. +*/ +const ( + tagLiteral = 0x00 + tagCopy1 = 0x01 + tagCopy2 = 0x02 + tagCopy4 = 0x03 +) + +const ( + checksumSize = 4 + chunkHeaderSize = 4 + magicChunk = "\xff\x06\x00\x00" + magicBody + magicBody = "sNaPpY" + + // maxBlockSize is the maximum size of the input to encodeBlock. It is not + // part of the wire format per se, but some parts of the encoder assume + // that an offset fits into a uint16. + // + // Also, for the framing format (Writer type instead of Encode function), + // https://github.com/google/snappy/blob/master/framing_format.txt says + // that "the uncompressed data in a chunk must be no longer than 65536 + // bytes". + maxBlockSize = 65536 + + // maxEncodedLenOfMaxBlockSize equals MaxEncodedLen(maxBlockSize), but is + // hard coded to be a const instead of a variable, so that obufLen can also + // be a const. Their equivalence is confirmed by + // TestMaxEncodedLenOfMaxBlockSize. + maxEncodedLenOfMaxBlockSize = 76490 + + obufHeaderLen = len(magicChunk) + checksumSize + chunkHeaderSize + obufLen = obufHeaderLen + maxEncodedLenOfMaxBlockSize +) + +const ( + chunkTypeCompressedData = 0x00 + chunkTypeUncompressedData = 0x01 + chunkTypePadding = 0xfe + chunkTypeStreamIdentifier = 0xff +) + +var crcTable = crc32.MakeTable(crc32.Castagnoli) + +// crc implements the checksum specified in section 3 of +// https://github.com/google/snappy/blob/master/framing_format.txt +func crc(b []byte) uint32 { + c := crc32.Update(0, crcTable, b) + return uint32(c>>15|c<<17) + 0xa282ead8 +} diff --git a/vendor/github.com/klauspost/compress/s2sx.mod b/vendor/github.com/klauspost/compress/s2sx.mod new file mode 100644 index 000000000000..2263853fcade --- /dev/null +++ b/vendor/github.com/klauspost/compress/s2sx.mod @@ -0,0 +1,4 @@ +module github.com/klauspost/compress + +go 1.16 + diff --git a/vendor/github.com/klauspost/compress/s2sx.sum b/vendor/github.com/klauspost/compress/s2sx.sum new file mode 100644 index 000000000000..e69de29bb2d1 diff --git a/vendor/github.com/klauspost/compress/zstd/README.md b/vendor/github.com/klauspost/compress/zstd/README.md index 787813fa9e2a..c8f0f16fc1ec 100644 --- a/vendor/github.com/klauspost/compress/zstd/README.md +++ b/vendor/github.com/klauspost/compress/zstd/README.md @@ -152,7 +152,7 @@ file out level insize outsize millis mb/s silesia.tar zskp 1 211947520 73101992 643 313.87 silesia.tar zskp 2 211947520 67504318 969 208.38 silesia.tar zskp 3 211947520 64595893 2007 100.68 -silesia.tar zskp 4 211947520 60995370 7691 26.28 +silesia.tar zskp 4 211947520 60995370 8825 22.90 cgo zstd: silesia.tar zstd 1 211947520 73605392 543 371.56 @@ -162,7 +162,7 @@ silesia.tar zstd 9 211947520 60212393 5063 39.92 gzip, stdlib/this package: silesia.tar gzstd 1 211947520 80007735 1654 122.21 -silesia.tar gzkp 1 211947520 80369488 1168 173.06 +silesia.tar gzkp 1 211947520 80136201 1152 175.45 GOB stream of binary data. Highly compressible. https://files.klauspost.com/compress/gob-stream.7z @@ -171,13 +171,15 @@ file out level insize outsize millis mb/s gob-stream zskp 1 1911399616 235022249 3088 590.30 gob-stream zskp 2 1911399616 205669791 3786 481.34 gob-stream zskp 3 1911399616 175034659 9636 189.17 -gob-stream zskp 4 1911399616 167273881 29337 62.13 +gob-stream zskp 4 1911399616 165609838 50369 36.19 + gob-stream zstd 1 1911399616 249810424 2637 691.26 gob-stream zstd 3 1911399616 208192146 3490 522.31 gob-stream zstd 6 1911399616 193632038 6687 272.56 gob-stream zstd 9 1911399616 177620386 16175 112.70 + gob-stream gzstd 1 1911399616 357382641 10251 177.82 -gob-stream gzkp 1 1911399616 362156523 5695 320.08 +gob-stream gzkp 1 1911399616 359753026 5438 335.20 The test data for the Large Text Compression Benchmark is the first 10^9 bytes of the English Wikipedia dump on Mar. 3, 2006. @@ -187,11 +189,13 @@ file out level insize outsize millis mb/s enwik9 zskp 1 1000000000 343848582 3609 264.18 enwik9 zskp 2 1000000000 317276632 5746 165.97 enwik9 zskp 3 1000000000 292243069 12162 78.41 -enwik9 zskp 4 1000000000 275241169 36430 26.18 +enwik9 zskp 4 1000000000 262183768 82837 11.51 + enwik9 zstd 1 1000000000 358072021 3110 306.65 enwik9 zstd 3 1000000000 313734672 4784 199.35 enwik9 zstd 6 1000000000 295138875 10290 92.68 enwik9 zstd 9 1000000000 278348700 28549 33.40 + enwik9 gzstd 1 1000000000 382578136 9604 99.30 enwik9 gzkp 1 1000000000 383825945 6544 145.73 @@ -202,13 +206,15 @@ file out level insize outsize millis mb/s github-june-2days-2019.json zskp 1 6273951764 699045015 10620 563.40 github-june-2days-2019.json zskp 2 6273951764 617881763 11687 511.96 github-june-2days-2019.json zskp 3 6273951764 524340691 34043 175.75 -github-june-2days-2019.json zskp 4 6273951764 503314661 93811 63.78 +github-june-2days-2019.json zskp 4 6273951764 470320075 170190 35.16 + github-june-2days-2019.json zstd 1 6273951764 766284037 8450 708.00 github-june-2days-2019.json zstd 3 6273951764 661889476 10927 547.57 github-june-2days-2019.json zstd 6 6273951764 642756859 22996 260.18 github-june-2days-2019.json zstd 9 6273951764 601974523 52413 114.16 + github-june-2days-2019.json gzstd 1 6273951764 1164400847 29948 199.79 -github-june-2days-2019.json gzkp 1 6273951764 1128755542 19236 311.03 +github-june-2days-2019.json gzkp 1 6273951764 1125417694 21788 274.61 VM Image, Linux mint with a few installed applications: https://files.klauspost.com/compress/rawstudio-mint14.7z @@ -217,13 +223,15 @@ file out level insize outsize millis mb/s rawstudio-mint14.tar zskp 1 8558382592 3667489370 20210 403.84 rawstudio-mint14.tar zskp 2 8558382592 3364592300 31873 256.07 rawstudio-mint14.tar zskp 3 8558382592 3158085214 77675 105.08 -rawstudio-mint14.tar zskp 4 8558382592 3020370044 404956 20.16 +rawstudio-mint14.tar zskp 4 8558382592 2965110639 857750 9.52 + rawstudio-mint14.tar zstd 1 8558382592 3609250104 17136 476.27 rawstudio-mint14.tar zstd 3 8558382592 3341679997 29262 278.92 rawstudio-mint14.tar zstd 6 8558382592 3235846406 77904 104.77 rawstudio-mint14.tar zstd 9 8558382592 3160778861 140946 57.91 + rawstudio-mint14.tar gzstd 1 8558382592 3926257486 57722 141.40 -rawstudio-mint14.tar gzkp 1 8558382592 3970463184 41749 195.49 +rawstudio-mint14.tar gzkp 1 8558382592 3962605659 45113 180.92 CSV data: https://files.klauspost.com/compress/nyc-taxi-data-10M.csv.zst @@ -232,13 +240,15 @@ file out level insize outsize millis mb/s nyc-taxi-data-10M.csv zskp 1 3325605752 641339945 8925 355.35 nyc-taxi-data-10M.csv zskp 2 3325605752 591748091 11268 281.44 nyc-taxi-data-10M.csv zskp 3 3325605752 530289687 25239 125.66 -nyc-taxi-data-10M.csv zskp 4 3325605752 490907191 65939 48.10 +nyc-taxi-data-10M.csv zskp 4 3325605752 476268884 135958 23.33 + nyc-taxi-data-10M.csv zstd 1 3325605752 687399637 8233 385.18 nyc-taxi-data-10M.csv zstd 3 3325605752 598514411 10065 315.07 nyc-taxi-data-10M.csv zstd 6 3325605752 570522953 20038 158.27 nyc-taxi-data-10M.csv zstd 9 3325605752 517554797 64565 49.12 + nyc-taxi-data-10M.csv gzstd 1 3325605752 928656485 23876 132.83 -nyc-taxi-data-10M.csv gzkp 1 3325605752 924718719 16388 193.53 +nyc-taxi-data-10M.csv gzkp 1 3325605752 922257165 16780 189.00 ``` ## Decompressor diff --git a/vendor/github.com/klauspost/compress/zstd/decoder.go b/vendor/github.com/klauspost/compress/zstd/decoder.go index 4d984c3b2633..f430f58b5726 100644 --- a/vendor/github.com/klauspost/compress/zstd/decoder.go +++ b/vendor/github.com/klauspost/compress/zstd/decoder.go @@ -260,9 +260,10 @@ func (d *Decoder) WriteTo(w io.Writer) (int64, error) { if len(d.current.b) > 0 { n2, err2 := w.Write(d.current.b) n += int64(n2) - if err2 != nil && d.current.err == nil { + if err2 != nil && (d.current.err == nil || d.current.err == io.EOF) { d.current.err = err2 - break + } else if n2 != len(d.current.b) { + d.current.err = io.ErrShortWrite } } if d.current.err != nil { diff --git a/vendor/github.com/klauspost/compress/zstd/enc_base.go b/vendor/github.com/klauspost/compress/zstd/enc_base.go index 60f298648644..295cd602a424 100644 --- a/vendor/github.com/klauspost/compress/zstd/enc_base.go +++ b/vendor/github.com/klauspost/compress/zstd/enc_base.go @@ -38,8 +38,8 @@ func (e *fastBase) AppendCRC(dst []byte) []byte { // WindowSize returns the window size of the encoder, // or a window size small enough to contain the input size, if > 0. -func (e *fastBase) WindowSize(size int) int32 { - if size > 0 && size < int(e.maxMatchOff) { +func (e *fastBase) WindowSize(size int64) int32 { + if size > 0 && size < int64(e.maxMatchOff) { b := int32(1) << uint(bits.Len(uint(size))) // Keep minimum window. if b < 1024 { diff --git a/vendor/github.com/klauspost/compress/zstd/enc_best.go b/vendor/github.com/klauspost/compress/zstd/enc_best.go index b7d4b90047f8..96028ecd8366 100644 --- a/vendor/github.com/klauspost/compress/zstd/enc_best.go +++ b/vendor/github.com/klauspost/compress/zstd/enc_best.go @@ -5,22 +5,61 @@ package zstd import ( + "bytes" "fmt" - "math/bits" + + "github.com/klauspost/compress" ) const ( - bestLongTableBits = 20 // Bits used in the long match table + bestLongTableBits = 22 // Bits used in the long match table bestLongTableSize = 1 << bestLongTableBits // Size of the table + bestLongLen = 8 // Bytes used for table hash // Note: Increasing the short table bits or making the hash shorter // can actually lead to compression degradation since it will 'steal' more from the // long match table and match offsets are quite big. // This greatly depends on the type of input. - bestShortTableBits = 16 // Bits used in the short match table + bestShortTableBits = 18 // Bits used in the short match table bestShortTableSize = 1 << bestShortTableBits // Size of the table + bestShortLen = 4 // Bytes used for table hash + ) +type match struct { + offset int32 + s int32 + length int32 + rep int32 + est int32 +} + +const highScore = 25000 + +// estBits will estimate output bits from predefined tables. +func (m *match) estBits(bitsPerByte int32) { + mlc := mlCode(uint32(m.length - zstdMinMatch)) + var ofc uint8 + if m.rep < 0 { + ofc = ofCode(uint32(m.s-m.offset) + 3) + } else { + ofc = ofCode(uint32(m.rep)) + } + // Cost, excluding + ofTT, mlTT := fsePredefEnc[tableOffsets].ct.symbolTT[ofc], fsePredefEnc[tableMatchLengths].ct.symbolTT[mlc] + + // Add cost of match encoding... + m.est = int32(ofTT.outBits + mlTT.outBits) + m.est += int32(ofTT.deltaNbBits>>16 + mlTT.deltaNbBits>>16) + // Subtract savings compared to literal encoding... + m.est -= (m.length * bitsPerByte) >> 10 + if m.est > 0 { + // Unlikely gain.. + m.length = 0 + m.est = highScore + } +} + // bestFastEncoder uses 2 tables, one for short matches (5 bytes) and one for long matches. // The long match table contains the previous entry with the same hash, // effectively making it a "chain" of length 2. @@ -109,6 +148,14 @@ func (e *bestFastEncoder) Encode(blk *blockEnc, src []byte) { return } + // Use this to estimate literal cost. + // Scaled by 10 bits. + bitsPerByte := int32((compress.ShannonEntropyBits(src) * 1024) / len(src)) + // Huffman can never go < 1 bit/byte + if bitsPerByte < 1024 { + bitsPerByte = 1024 + } + // Override src src = e.hist sLimit := int32(len(src)) - inputMargin @@ -145,51 +192,49 @@ encodeLoop: panic("offset0 was 0") } - type match struct { - offset int32 - s int32 - length int32 - rep int32 - } - matchAt := func(offset int32, s int32, first uint32, rep int32) match { - if s-offset >= e.maxMatchOff || load3232(src, offset) != first { - return match{offset: offset, s: s} - } - return match{offset: offset, s: s, length: 4 + e.matchlen(s+4, offset+4, src), rep: rep} - } - bestOf := func(a, b match) match { - aScore := b.s - a.s + a.length - bScore := a.s - b.s + b.length - if a.rep < 0 { - aScore = aScore - int32(bits.Len32(uint32(a.offset)))/8 - } - if b.rep < 0 { - bScore = bScore - int32(bits.Len32(uint32(b.offset)))/8 - } - if aScore >= bScore { + if a.est+(a.s-b.s)*bitsPerByte>>10 < b.est+(b.s-a.s)*bitsPerByte>>10 { return a } return b } const goodEnough = 100 - nextHashL := hash8(cv, bestLongTableBits) - nextHashS := hash4x64(cv, bestShortTableBits) + nextHashL := hashLen(cv, bestLongTableBits, bestLongLen) + nextHashS := hashLen(cv, bestShortTableBits, bestShortLen) candidateL := e.longTable[nextHashL] candidateS := e.table[nextHashS] + matchAt := func(offset int32, s int32, first uint32, rep int32) match { + if s-offset >= e.maxMatchOff || load3232(src, offset) != first { + return match{s: s, est: highScore} + } + if debugAsserts { + if !bytes.Equal(src[s:s+4], src[offset:offset+4]) { + panic(fmt.Sprintf("first match mismatch: %v != %v, first: %08x", src[s:s+4], src[offset:offset+4], first)) + } + } + m := match{offset: offset, s: s, length: 4 + e.matchlen(s+4, offset+4, src), rep: rep} + m.estBits(bitsPerByte) + return m + } + best := bestOf(matchAt(candidateL.offset-e.cur, s, uint32(cv), -1), matchAt(candidateL.prev-e.cur, s, uint32(cv), -1)) best = bestOf(best, matchAt(candidateS.offset-e.cur, s, uint32(cv), -1)) best = bestOf(best, matchAt(candidateS.prev-e.cur, s, uint32(cv), -1)) + if canRepeat && best.length < goodEnough { - best = bestOf(best, matchAt(s-offset1+1, s+1, uint32(cv>>8), 1)) - best = bestOf(best, matchAt(s-offset2+1, s+1, uint32(cv>>8), 2)) - best = bestOf(best, matchAt(s-offset3+1, s+1, uint32(cv>>8), 3)) + cv32 := uint32(cv >> 8) + spp := s + 1 + best = bestOf(best, matchAt(spp-offset1, spp, cv32, 1)) + best = bestOf(best, matchAt(spp-offset2, spp, cv32, 2)) + best = bestOf(best, matchAt(spp-offset3, spp, cv32, 3)) if best.length > 0 { - best = bestOf(best, matchAt(s-offset1+3, s+3, uint32(cv>>24), 1)) - best = bestOf(best, matchAt(s-offset2+3, s+3, uint32(cv>>24), 2)) - best = bestOf(best, matchAt(s-offset3+3, s+3, uint32(cv>>24), 3)) + cv32 = uint32(cv >> 24) + spp += 2 + best = bestOf(best, matchAt(spp-offset1, spp, cv32, 1)) + best = bestOf(best, matchAt(spp-offset2, spp, cv32, 2)) + best = bestOf(best, matchAt(spp-offset3, spp, cv32, 3)) } } // Load next and check... @@ -209,22 +254,28 @@ encodeLoop: } s++ - candidateS = e.table[hash4x64(cv>>8, bestShortTableBits)] + candidateS = e.table[hashLen(cv>>8, bestShortTableBits, bestShortLen)] cv = load6432(src, s) cv2 := load6432(src, s+1) - candidateL = e.longTable[hash8(cv, bestLongTableBits)] - candidateL2 := e.longTable[hash8(cv2, bestLongTableBits)] + candidateL = e.longTable[hashLen(cv, bestLongTableBits, bestLongLen)] + candidateL2 := e.longTable[hashLen(cv2, bestLongTableBits, bestLongLen)] + // Short at s+1 best = bestOf(best, matchAt(candidateS.offset-e.cur, s, uint32(cv), -1)) + // Long at s+1, s+2 best = bestOf(best, matchAt(candidateL.offset-e.cur, s, uint32(cv), -1)) best = bestOf(best, matchAt(candidateL.prev-e.cur, s, uint32(cv), -1)) best = bestOf(best, matchAt(candidateL2.offset-e.cur, s+1, uint32(cv2), -1)) best = bestOf(best, matchAt(candidateL2.prev-e.cur, s+1, uint32(cv2), -1)) - + if false { + // Short at s+3. + // Too often worse... + best = bestOf(best, matchAt(e.table[hashLen(cv2>>8, bestShortTableBits, bestShortLen)].offset-e.cur, s+2, uint32(cv2>>8), -1)) + } // See if we can find a better match by checking where the current best ends. // Use that offset to see if we can find a better full match. if sAt := best.s + best.length; sAt < sLimit { - nextHashL := hash8(load6432(src, sAt), bestLongTableBits) + nextHashL := hashLen(load6432(src, sAt), bestLongTableBits, bestLongLen) candidateEnd := e.longTable[nextHashL] if pos := candidateEnd.offset - e.cur - best.length; pos >= 0 { bestEnd := bestOf(best, matchAt(pos, best.s, load3232(src, best.s), -1)) @@ -236,6 +287,12 @@ encodeLoop: } } + if debugAsserts { + if !bytes.Equal(src[best.s:best.s+best.length], src[best.offset:best.offset+best.length]) { + panic(fmt.Sprintf("match mismatch: %v != %v", src[best.s:best.s+best.length], src[best.offset:best.offset+best.length])) + } + } + // We have a match, we can store the forward value if best.rep > 0 { s = best.s @@ -284,8 +341,8 @@ encodeLoop: off := index0 + e.cur for index0 < s-1 { cv0 := load6432(src, index0) - h0 := hash8(cv0, bestLongTableBits) - h1 := hash4x64(cv0, bestShortTableBits) + h0 := hashLen(cv0, bestLongTableBits, bestLongLen) + h1 := hashLen(cv0, bestShortTableBits, bestShortLen) e.longTable[h0] = prevEntry{offset: off, prev: e.longTable[h0].offset} e.table[h1] = prevEntry{offset: off, prev: e.table[h1].offset} off++ @@ -311,7 +368,7 @@ encodeLoop: panic(fmt.Sprintf("s (%d) <= t (%d)", s, t)) } - if debugAsserts && canRepeat && int(offset1) > len(src) { + if debugAsserts && int(offset1) > len(src) { panic("invalid offset") } @@ -352,8 +409,8 @@ encodeLoop: // every entry for index0 < s-1 { cv0 := load6432(src, index0) - h0 := hash8(cv0, bestLongTableBits) - h1 := hash4x64(cv0, bestShortTableBits) + h0 := hashLen(cv0, bestLongTableBits, bestLongLen) + h1 := hashLen(cv0, bestShortTableBits, bestShortLen) off := index0 + e.cur e.longTable[h0] = prevEntry{offset: off, prev: e.longTable[h0].offset} e.table[h1] = prevEntry{offset: off, prev: e.table[h1].offset} @@ -374,8 +431,8 @@ encodeLoop: } // Store this, since we have it. - nextHashS := hash4x64(cv, bestShortTableBits) - nextHashL := hash8(cv, bestLongTableBits) + nextHashS := hashLen(cv, bestShortTableBits, bestShortLen) + nextHashL := hashLen(cv, bestLongTableBits, bestLongLen) // We have at least 4 byte match. // No need to check backwards. We come straight from a match @@ -425,7 +482,7 @@ func (e *bestFastEncoder) EncodeNoHist(blk *blockEnc, src []byte) { e.Encode(blk, src) } -// ResetDict will reset and set a dictionary if not nil +// Reset will reset and set a dictionary if not nil func (e *bestFastEncoder) Reset(d *dict, singleBlock bool) { e.resetBase(d, singleBlock) if d == nil { @@ -441,10 +498,10 @@ func (e *bestFastEncoder) Reset(d *dict, singleBlock bool) { const hashLog = bestShortTableBits cv := load6432(d.content, i-e.maxMatchOff) - nextHash := hash4x64(cv, hashLog) // 0 -> 4 - nextHash1 := hash4x64(cv>>8, hashLog) // 1 -> 5 - nextHash2 := hash4x64(cv>>16, hashLog) // 2 -> 6 - nextHash3 := hash4x64(cv>>24, hashLog) // 3 -> 7 + nextHash := hashLen(cv, hashLog, bestShortLen) // 0 -> 4 + nextHash1 := hashLen(cv>>8, hashLog, bestShortLen) // 1 -> 5 + nextHash2 := hashLen(cv>>16, hashLog, bestShortLen) // 2 -> 6 + nextHash3 := hashLen(cv>>24, hashLog, bestShortLen) // 3 -> 7 e.dictTable[nextHash] = prevEntry{ prev: e.dictTable[nextHash].offset, offset: i, @@ -472,7 +529,7 @@ func (e *bestFastEncoder) Reset(d *dict, singleBlock bool) { } if len(d.content) >= 8 { cv := load6432(d.content, 0) - h := hash8(cv, bestLongTableBits) + h := hashLen(cv, bestLongTableBits, bestLongLen) e.dictLongTable[h] = prevEntry{ offset: e.maxMatchOff, prev: e.dictLongTable[h].offset, @@ -482,7 +539,7 @@ func (e *bestFastEncoder) Reset(d *dict, singleBlock bool) { off := 8 // First to read for i := e.maxMatchOff + 1; i < end; i++ { cv = cv>>8 | (uint64(d.content[off]) << 56) - h := hash8(cv, bestLongTableBits) + h := hashLen(cv, bestLongTableBits, bestLongLen) e.dictLongTable[h] = prevEntry{ offset: i, prev: e.dictLongTable[h].offset, diff --git a/vendor/github.com/klauspost/compress/zstd/enc_better.go b/vendor/github.com/klauspost/compress/zstd/enc_better.go index eab7b5083e46..602c05ee0c4c 100644 --- a/vendor/github.com/klauspost/compress/zstd/enc_better.go +++ b/vendor/github.com/klauspost/compress/zstd/enc_better.go @@ -9,6 +9,7 @@ import "fmt" const ( betterLongTableBits = 19 // Bits used in the long match table betterLongTableSize = 1 << betterLongTableBits // Size of the table + betterLongLen = 8 // Bytes used for table hash // Note: Increasing the short table bits or making the hash shorter // can actually lead to compression degradation since it will 'steal' more from the @@ -16,6 +17,7 @@ const ( // This greatly depends on the type of input. betterShortTableBits = 13 // Bits used in the short match table betterShortTableSize = 1 << betterShortTableBits // Size of the table + betterShortLen = 5 // Bytes used for table hash betterLongTableShardCnt = 1 << (betterLongTableBits - dictShardBits) // Number of shards in the table betterLongTableShardSize = betterLongTableSize / betterLongTableShardCnt // Size of an individual shard @@ -154,8 +156,8 @@ encodeLoop: panic("offset0 was 0") } - nextHashS := hash5(cv, betterShortTableBits) - nextHashL := hash8(cv, betterLongTableBits) + nextHashS := hashLen(cv, betterShortTableBits, betterShortLen) + nextHashL := hashLen(cv, betterLongTableBits, betterLongLen) candidateL := e.longTable[nextHashL] candidateS := e.table[nextHashS] @@ -214,10 +216,10 @@ encodeLoop: for index0 < s-1 { cv0 := load6432(src, index0) cv1 := cv0 >> 8 - h0 := hash8(cv0, betterLongTableBits) + h0 := hashLen(cv0, betterLongTableBits, betterLongLen) off := index0 + e.cur e.longTable[h0] = prevEntry{offset: off, prev: e.longTable[h0].offset} - e.table[hash5(cv1, betterShortTableBits)] = tableEntry{offset: off + 1, val: uint32(cv1)} + e.table[hashLen(cv1, betterShortTableBits, betterShortLen)] = tableEntry{offset: off + 1, val: uint32(cv1)} index0 += 2 } cv = load6432(src, s) @@ -275,10 +277,10 @@ encodeLoop: for index0 < s-1 { cv0 := load6432(src, index0) cv1 := cv0 >> 8 - h0 := hash8(cv0, betterLongTableBits) + h0 := hashLen(cv0, betterLongTableBits, betterLongLen) off := index0 + e.cur e.longTable[h0] = prevEntry{offset: off, prev: e.longTable[h0].offset} - e.table[hash5(cv1, betterShortTableBits)] = tableEntry{offset: off + 1, val: uint32(cv1)} + e.table[hashLen(cv1, betterShortTableBits, betterShortLen)] = tableEntry{offset: off + 1, val: uint32(cv1)} index0 += 2 } cv = load6432(src, s) @@ -353,7 +355,7 @@ encodeLoop: // See if we can find a long match at s+1 const checkAt = 1 cv := load6432(src, s+checkAt) - nextHashL = hash8(cv, betterLongTableBits) + nextHashL = hashLen(cv, betterLongTableBits, betterLongLen) candidateL = e.longTable[nextHashL] coffsetL = candidateL.offset - e.cur @@ -413,8 +415,8 @@ encodeLoop: } // Try to find a better match by searching for a long match at the end of the current best match - if true && s+matched < sLimit { - nextHashL := hash8(load6432(src, s+matched), betterLongTableBits) + if s+matched < sLimit { + nextHashL := hashLen(load6432(src, s+matched), betterLongTableBits, betterLongLen) cv := load3232(src, s) candidateL := e.longTable[nextHashL] coffsetL := candidateL.offset - e.cur - matched @@ -495,10 +497,10 @@ encodeLoop: for index0 < s-1 { cv0 := load6432(src, index0) cv1 := cv0 >> 8 - h0 := hash8(cv0, betterLongTableBits) + h0 := hashLen(cv0, betterLongTableBits, betterLongLen) off := index0 + e.cur e.longTable[h0] = prevEntry{offset: off, prev: e.longTable[h0].offset} - e.table[hash5(cv1, betterShortTableBits)] = tableEntry{offset: off + 1, val: uint32(cv1)} + e.table[hashLen(cv1, betterShortTableBits, betterShortLen)] = tableEntry{offset: off + 1, val: uint32(cv1)} index0 += 2 } @@ -516,8 +518,8 @@ encodeLoop: } // Store this, since we have it. - nextHashS := hash5(cv, betterShortTableBits) - nextHashL := hash8(cv, betterLongTableBits) + nextHashS := hashLen(cv, betterShortTableBits, betterShortLen) + nextHashL := hashLen(cv, betterLongTableBits, betterLongLen) // We have at least 4 byte match. // No need to check backwards. We come straight from a match @@ -672,8 +674,8 @@ encodeLoop: panic("offset0 was 0") } - nextHashS := hash5(cv, betterShortTableBits) - nextHashL := hash8(cv, betterLongTableBits) + nextHashS := hashLen(cv, betterShortTableBits, betterShortLen) + nextHashL := hashLen(cv, betterLongTableBits, betterLongLen) candidateL := e.longTable[nextHashL] candidateS := e.table[nextHashS] @@ -734,11 +736,11 @@ encodeLoop: for index0 < s-1 { cv0 := load6432(src, index0) cv1 := cv0 >> 8 - h0 := hash8(cv0, betterLongTableBits) + h0 := hashLen(cv0, betterLongTableBits, betterLongLen) off := index0 + e.cur e.longTable[h0] = prevEntry{offset: off, prev: e.longTable[h0].offset} e.markLongShardDirty(h0) - h1 := hash5(cv1, betterShortTableBits) + h1 := hashLen(cv1, betterShortTableBits, betterShortLen) e.table[h1] = tableEntry{offset: off + 1, val: uint32(cv1)} e.markShortShardDirty(h1) index0 += 2 @@ -798,11 +800,11 @@ encodeLoop: for index0 < s-1 { cv0 := load6432(src, index0) cv1 := cv0 >> 8 - h0 := hash8(cv0, betterLongTableBits) + h0 := hashLen(cv0, betterLongTableBits, betterLongLen) off := index0 + e.cur e.longTable[h0] = prevEntry{offset: off, prev: e.longTable[h0].offset} e.markLongShardDirty(h0) - h1 := hash5(cv1, betterShortTableBits) + h1 := hashLen(cv1, betterShortTableBits, betterShortLen) e.table[h1] = tableEntry{offset: off + 1, val: uint32(cv1)} e.markShortShardDirty(h1) index0 += 2 @@ -879,7 +881,7 @@ encodeLoop: // See if we can find a long match at s+1 const checkAt = 1 cv := load6432(src, s+checkAt) - nextHashL = hash8(cv, betterLongTableBits) + nextHashL = hashLen(cv, betterLongTableBits, betterLongLen) candidateL = e.longTable[nextHashL] coffsetL = candidateL.offset - e.cur @@ -940,7 +942,7 @@ encodeLoop: } // Try to find a better match by searching for a long match at the end of the current best match if s+matched < sLimit { - nextHashL := hash8(load6432(src, s+matched), betterLongTableBits) + nextHashL := hashLen(load6432(src, s+matched), betterLongTableBits, betterLongLen) cv := load3232(src, s) candidateL := e.longTable[nextHashL] coffsetL := candidateL.offset - e.cur - matched @@ -1021,11 +1023,11 @@ encodeLoop: for index0 < s-1 { cv0 := load6432(src, index0) cv1 := cv0 >> 8 - h0 := hash8(cv0, betterLongTableBits) + h0 := hashLen(cv0, betterLongTableBits, betterLongLen) off := index0 + e.cur e.longTable[h0] = prevEntry{offset: off, prev: e.longTable[h0].offset} e.markLongShardDirty(h0) - h1 := hash5(cv1, betterShortTableBits) + h1 := hashLen(cv1, betterShortTableBits, betterShortLen) e.table[h1] = tableEntry{offset: off + 1, val: uint32(cv1)} e.markShortShardDirty(h1) index0 += 2 @@ -1045,8 +1047,8 @@ encodeLoop: } // Store this, since we have it. - nextHashS := hash5(cv, betterShortTableBits) - nextHashL := hash8(cv, betterLongTableBits) + nextHashS := hashLen(cv, betterShortTableBits, betterShortLen) + nextHashL := hashLen(cv, betterLongTableBits, betterLongLen) // We have at least 4 byte match. // No need to check backwards. We come straight from a match @@ -1113,10 +1115,10 @@ func (e *betterFastEncoderDict) Reset(d *dict, singleBlock bool) { const hashLog = betterShortTableBits cv := load6432(d.content, i-e.maxMatchOff) - nextHash := hash5(cv, hashLog) // 0 -> 4 - nextHash1 := hash5(cv>>8, hashLog) // 1 -> 5 - nextHash2 := hash5(cv>>16, hashLog) // 2 -> 6 - nextHash3 := hash5(cv>>24, hashLog) // 3 -> 7 + nextHash := hashLen(cv, hashLog, betterShortLen) // 0 -> 4 + nextHash1 := hashLen(cv>>8, hashLog, betterShortLen) // 1 -> 5 + nextHash2 := hashLen(cv>>16, hashLog, betterShortLen) // 2 -> 6 + nextHash3 := hashLen(cv>>24, hashLog, betterShortLen) // 3 -> 7 e.dictTable[nextHash] = tableEntry{ val: uint32(cv), offset: i, @@ -1145,7 +1147,7 @@ func (e *betterFastEncoderDict) Reset(d *dict, singleBlock bool) { } if len(d.content) >= 8 { cv := load6432(d.content, 0) - h := hash8(cv, betterLongTableBits) + h := hashLen(cv, betterLongTableBits, betterLongLen) e.dictLongTable[h] = prevEntry{ offset: e.maxMatchOff, prev: e.dictLongTable[h].offset, @@ -1155,7 +1157,7 @@ func (e *betterFastEncoderDict) Reset(d *dict, singleBlock bool) { off := 8 // First to read for i := e.maxMatchOff + 1; i < end; i++ { cv = cv>>8 | (uint64(d.content[off]) << 56) - h := hash8(cv, betterLongTableBits) + h := hashLen(cv, betterLongTableBits, betterLongLen) e.dictLongTable[h] = prevEntry{ offset: i, prev: e.dictLongTable[h].offset, diff --git a/vendor/github.com/klauspost/compress/zstd/enc_dfast.go b/vendor/github.com/klauspost/compress/zstd/enc_dfast.go index 96b21b90e81d..d6b3104240b0 100644 --- a/vendor/github.com/klauspost/compress/zstd/enc_dfast.go +++ b/vendor/github.com/klauspost/compress/zstd/enc_dfast.go @@ -10,6 +10,7 @@ const ( dFastLongTableBits = 17 // Bits used in the long match table dFastLongTableSize = 1 << dFastLongTableBits // Size of the table dFastLongTableMask = dFastLongTableSize - 1 // Mask for table indices. Redundant, but can eliminate bounds checks. + dFastLongLen = 8 // Bytes used for table hash dLongTableShardCnt = 1 << (dFastLongTableBits - dictShardBits) // Number of shards in the table dLongTableShardSize = dFastLongTableSize / tableShardCnt // Size of an individual shard @@ -17,6 +18,8 @@ const ( dFastShortTableBits = tableBits // Bits used in the short match table dFastShortTableSize = 1 << dFastShortTableBits // Size of the table dFastShortTableMask = dFastShortTableSize - 1 // Mask for table indices. Redundant, but can eliminate bounds checks. + dFastShortLen = 5 // Bytes used for table hash + ) type doubleFastEncoder struct { @@ -124,8 +127,8 @@ encodeLoop: panic("offset0 was 0") } - nextHashS := hash5(cv, dFastShortTableBits) - nextHashL := hash8(cv, dFastLongTableBits) + nextHashS := hashLen(cv, dFastShortTableBits, dFastShortLen) + nextHashL := hashLen(cv, dFastLongTableBits, dFastLongLen) candidateL := e.longTable[nextHashL] candidateS := e.table[nextHashS] @@ -208,7 +211,7 @@ encodeLoop: // See if we can find a long match at s+1 const checkAt = 1 cv := load6432(src, s+checkAt) - nextHashL = hash8(cv, dFastLongTableBits) + nextHashL = hashLen(cv, dFastLongTableBits, dFastLongLen) candidateL = e.longTable[nextHashL] coffsetL = s - (candidateL.offset - e.cur) + checkAt @@ -304,16 +307,16 @@ encodeLoop: cv1 := load6432(src, index1) te0 := tableEntry{offset: index0 + e.cur, val: uint32(cv0)} te1 := tableEntry{offset: index1 + e.cur, val: uint32(cv1)} - e.longTable[hash8(cv0, dFastLongTableBits)] = te0 - e.longTable[hash8(cv1, dFastLongTableBits)] = te1 + e.longTable[hashLen(cv0, dFastLongTableBits, dFastLongLen)] = te0 + e.longTable[hashLen(cv1, dFastLongTableBits, dFastLongLen)] = te1 cv0 >>= 8 cv1 >>= 8 te0.offset++ te1.offset++ te0.val = uint32(cv0) te1.val = uint32(cv1) - e.table[hash5(cv0, dFastShortTableBits)] = te0 - e.table[hash5(cv1, dFastShortTableBits)] = te1 + e.table[hashLen(cv0, dFastShortTableBits, dFastShortLen)] = te0 + e.table[hashLen(cv1, dFastShortTableBits, dFastShortLen)] = te1 cv = load6432(src, s) @@ -330,8 +333,8 @@ encodeLoop: } // Store this, since we have it. - nextHashS := hash5(cv, dFastShortTableBits) - nextHashL := hash8(cv, dFastLongTableBits) + nextHashS := hashLen(cv, dFastShortTableBits, dFastShortLen) + nextHashL := hashLen(cv, dFastLongTableBits, dFastLongLen) // We have at least 4 byte match. // No need to check backwards. We come straight from a match @@ -436,8 +439,8 @@ encodeLoop: var t int32 for { - nextHashS := hash5(cv, dFastShortTableBits) - nextHashL := hash8(cv, dFastLongTableBits) + nextHashS := hashLen(cv, dFastShortTableBits, dFastShortLen) + nextHashL := hashLen(cv, dFastLongTableBits, dFastLongLen) candidateL := e.longTable[nextHashL] candidateS := e.table[nextHashS] @@ -521,7 +524,7 @@ encodeLoop: // See if we can find a long match at s+1 const checkAt = 1 cv := load6432(src, s+checkAt) - nextHashL = hash8(cv, dFastLongTableBits) + nextHashL = hashLen(cv, dFastLongTableBits, dFastLongLen) candidateL = e.longTable[nextHashL] coffsetL = s - (candidateL.offset - e.cur) + checkAt @@ -614,16 +617,16 @@ encodeLoop: cv1 := load6432(src, index1) te0 := tableEntry{offset: index0 + e.cur, val: uint32(cv0)} te1 := tableEntry{offset: index1 + e.cur, val: uint32(cv1)} - e.longTable[hash8(cv0, dFastLongTableBits)] = te0 - e.longTable[hash8(cv1, dFastLongTableBits)] = te1 + e.longTable[hashLen(cv0, dFastLongTableBits, dFastLongLen)] = te0 + e.longTable[hashLen(cv1, dFastLongTableBits, dFastLongLen)] = te1 cv0 >>= 8 cv1 >>= 8 te0.offset++ te1.offset++ te0.val = uint32(cv0) te1.val = uint32(cv1) - e.table[hash5(cv0, dFastShortTableBits)] = te0 - e.table[hash5(cv1, dFastShortTableBits)] = te1 + e.table[hashLen(cv0, dFastShortTableBits, dFastShortLen)] = te0 + e.table[hashLen(cv1, dFastShortTableBits, dFastShortLen)] = te1 cv = load6432(src, s) @@ -640,8 +643,8 @@ encodeLoop: } // Store this, since we have it. - nextHashS := hash5(cv1>>8, dFastShortTableBits) - nextHashL := hash8(cv, dFastLongTableBits) + nextHashS := hashLen(cv1>>8, dFastShortTableBits, dFastShortLen) + nextHashL := hashLen(cv, dFastLongTableBits, dFastLongLen) // We have at least 4 byte match. // No need to check backwards. We come straight from a match @@ -782,8 +785,8 @@ encodeLoop: panic("offset0 was 0") } - nextHashS := hash5(cv, dFastShortTableBits) - nextHashL := hash8(cv, dFastLongTableBits) + nextHashS := hashLen(cv, dFastShortTableBits, dFastShortLen) + nextHashL := hashLen(cv, dFastLongTableBits, dFastLongLen) candidateL := e.longTable[nextHashL] candidateS := e.table[nextHashS] @@ -868,7 +871,7 @@ encodeLoop: // See if we can find a long match at s+1 const checkAt = 1 cv := load6432(src, s+checkAt) - nextHashL = hash8(cv, dFastLongTableBits) + nextHashL = hashLen(cv, dFastLongTableBits, dFastLongLen) candidateL = e.longTable[nextHashL] coffsetL = s - (candidateL.offset - e.cur) + checkAt @@ -965,8 +968,8 @@ encodeLoop: cv1 := load6432(src, index1) te0 := tableEntry{offset: index0 + e.cur, val: uint32(cv0)} te1 := tableEntry{offset: index1 + e.cur, val: uint32(cv1)} - longHash1 := hash8(cv0, dFastLongTableBits) - longHash2 := hash8(cv0, dFastLongTableBits) + longHash1 := hashLen(cv0, dFastLongTableBits, dFastLongLen) + longHash2 := hashLen(cv0, dFastLongTableBits, dFastLongLen) e.longTable[longHash1] = te0 e.longTable[longHash2] = te1 e.markLongShardDirty(longHash1) @@ -977,8 +980,8 @@ encodeLoop: te1.offset++ te0.val = uint32(cv0) te1.val = uint32(cv1) - hashVal1 := hash5(cv0, dFastShortTableBits) - hashVal2 := hash5(cv1, dFastShortTableBits) + hashVal1 := hashLen(cv0, dFastShortTableBits, dFastShortLen) + hashVal2 := hashLen(cv1, dFastShortTableBits, dFastShortLen) e.table[hashVal1] = te0 e.markShardDirty(hashVal1) e.table[hashVal2] = te1 @@ -999,8 +1002,8 @@ encodeLoop: } // Store this, since we have it. - nextHashS := hash5(cv, dFastShortTableBits) - nextHashL := hash8(cv, dFastLongTableBits) + nextHashS := hashLen(cv, dFastShortTableBits, dFastShortLen) + nextHashL := hashLen(cv, dFastLongTableBits, dFastLongLen) // We have at least 4 byte match. // No need to check backwards. We come straight from a match @@ -1071,14 +1074,14 @@ func (e *doubleFastEncoderDict) Reset(d *dict, singleBlock bool) { } if len(d.content) >= 8 { cv := load6432(d.content, 0) - e.dictLongTable[hash8(cv, dFastLongTableBits)] = tableEntry{ + e.dictLongTable[hashLen(cv, dFastLongTableBits, dFastLongLen)] = tableEntry{ val: uint32(cv), offset: e.maxMatchOff, } end := int32(len(d.content)) - 8 + e.maxMatchOff for i := e.maxMatchOff + 1; i < end; i++ { cv = cv>>8 | (uint64(d.content[i-e.maxMatchOff+7]) << 56) - e.dictLongTable[hash8(cv, dFastLongTableBits)] = tableEntry{ + e.dictLongTable[hashLen(cv, dFastLongTableBits, dFastLongLen)] = tableEntry{ val: uint32(cv), offset: i, } diff --git a/vendor/github.com/klauspost/compress/zstd/enc_fast.go b/vendor/github.com/klauspost/compress/zstd/enc_fast.go index 2246d286dc63..f2502629bc55 100644 --- a/vendor/github.com/klauspost/compress/zstd/enc_fast.go +++ b/vendor/github.com/klauspost/compress/zstd/enc_fast.go @@ -11,12 +11,13 @@ import ( ) const ( - tableBits = 15 // Bits used in the table - tableSize = 1 << tableBits // Size of the table - tableShardCnt = 1 << (tableBits - dictShardBits) // Number of shards in the table - tableShardSize = tableSize / tableShardCnt // Size of an individual shard - tableMask = tableSize - 1 // Mask for table indices. Redundant, but can eliminate bounds checks. - maxMatchLength = 131074 + tableBits = 15 // Bits used in the table + tableSize = 1 << tableBits // Size of the table + tableShardCnt = 1 << (tableBits - dictShardBits) // Number of shards in the table + tableShardSize = tableSize / tableShardCnt // Size of an individual shard + tableFastHashLen = 6 + tableMask = tableSize - 1 // Mask for table indices. Redundant, but can eliminate bounds checks. + maxMatchLength = 131074 ) type tableEntry struct { @@ -122,8 +123,8 @@ encodeLoop: panic("offset0 was 0") } - nextHash := hash6(cv, hashLog) - nextHash2 := hash6(cv>>8, hashLog) + nextHash := hashLen(cv, hashLog, tableFastHashLen) + nextHash2 := hashLen(cv>>8, hashLog, tableFastHashLen) candidate := e.table[nextHash] candidate2 := e.table[nextHash2] repIndex := s - offset1 + 2 @@ -301,7 +302,7 @@ encodeLoop: } // Store this, since we have it. - nextHash := hash6(cv, hashLog) + nextHash := hashLen(cv, hashLog, tableFastHashLen) e.table[nextHash] = tableEntry{offset: s + e.cur, val: uint32(cv)} seq.matchLen = uint32(l) - zstdMinMatch seq.litLen = 0 @@ -405,8 +406,8 @@ encodeLoop: // By not using them for the first 3 matches for { - nextHash := hash6(cv, hashLog) - nextHash2 := hash6(cv>>8, hashLog) + nextHash := hashLen(cv, hashLog, tableFastHashLen) + nextHash2 := hashLen(cv>>8, hashLog, tableFastHashLen) candidate := e.table[nextHash] candidate2 := e.table[nextHash2] repIndex := s - offset1 + 2 @@ -589,7 +590,7 @@ encodeLoop: } // Store this, since we have it. - nextHash := hash6(cv, hashLog) + nextHash := hashLen(cv, hashLog, tableFastHashLen) e.table[nextHash] = tableEntry{offset: s + e.cur, val: uint32(cv)} seq.matchLen = uint32(l) - zstdMinMatch seq.litLen = 0 @@ -715,8 +716,8 @@ encodeLoop: panic("offset0 was 0") } - nextHash := hash6(cv, hashLog) - nextHash2 := hash6(cv>>8, hashLog) + nextHash := hashLen(cv, hashLog, tableFastHashLen) + nextHash2 := hashLen(cv>>8, hashLog, tableFastHashLen) candidate := e.table[nextHash] candidate2 := e.table[nextHash2] repIndex := s - offset1 + 2 @@ -896,7 +897,7 @@ encodeLoop: } // Store this, since we have it. - nextHash := hash6(cv, hashLog) + nextHash := hashLen(cv, hashLog, tableFastHashLen) e.table[nextHash] = tableEntry{offset: s + e.cur, val: uint32(cv)} e.markShardDirty(nextHash) seq.matchLen = uint32(l) - zstdMinMatch @@ -957,9 +958,9 @@ func (e *fastEncoderDict) Reset(d *dict, singleBlock bool) { const hashLog = tableBits cv := load6432(d.content, i-e.maxMatchOff) - nextHash := hash6(cv, hashLog) // 0 -> 5 - nextHash1 := hash6(cv>>8, hashLog) // 1 -> 6 - nextHash2 := hash6(cv>>16, hashLog) // 2 -> 7 + nextHash := hashLen(cv, hashLog, tableFastHashLen) // 0 -> 5 + nextHash1 := hashLen(cv>>8, hashLog, tableFastHashLen) // 1 -> 6 + nextHash2 := hashLen(cv>>16, hashLog, tableFastHashLen) // 2 -> 7 e.dictTable[nextHash] = tableEntry{ val: uint32(cv), offset: i, diff --git a/vendor/github.com/klauspost/compress/zstd/encoder.go b/vendor/github.com/klauspost/compress/zstd/encoder.go index ea85548fc9a4..e6e315969b00 100644 --- a/vendor/github.com/klauspost/compress/zstd/encoder.go +++ b/vendor/github.com/klauspost/compress/zstd/encoder.go @@ -33,7 +33,7 @@ type encoder interface { Block() *blockEnc CRC() *xxhash.Digest AppendCRC([]byte) []byte - WindowSize(size int) int32 + WindowSize(size int64) int32 UseBlock(*blockEnc) Reset(d *dict, singleBlock bool) } @@ -48,6 +48,8 @@ type encoderState struct { err error writeErr error nWritten int64 + nInput int64 + frameContentSize int64 headerWritten bool eofWritten bool fullFrameWritten bool @@ -120,7 +122,21 @@ func (e *Encoder) Reset(w io.Writer) { s.w = w s.err = nil s.nWritten = 0 + s.nInput = 0 s.writeErr = nil + s.frameContentSize = 0 +} + +// ResetContentSize will reset and set a content size for the next stream. +// If the bytes written does not match the size given an error will be returned +// when calling Close(). +// This is removed when Reset is called. +// Sizes <= 0 results in no content size set. +func (e *Encoder) ResetContentSize(w io.Writer, size int64) { + e.Reset(w) + if size >= 0 { + e.state.frameContentSize = size + } } // Write data to the encoder. @@ -190,6 +206,7 @@ func (e *Encoder) nextBlock(final bool) error { return s.err } s.nWritten += int64(n2) + s.nInput += int64(len(s.filling)) s.current = s.current[:0] s.filling = s.filling[:0] s.headerWritten = true @@ -200,8 +217,8 @@ func (e *Encoder) nextBlock(final bool) error { var tmp [maxHeaderSize]byte fh := frameHeader{ - ContentSize: 0, - WindowSize: uint32(s.encoder.WindowSize(0)), + ContentSize: uint64(s.frameContentSize), + WindowSize: uint32(s.encoder.WindowSize(s.frameContentSize)), SingleSegment: false, Checksum: e.o.crc, DictID: e.o.dict.ID(), @@ -243,6 +260,7 @@ func (e *Encoder) nextBlock(final bool) error { // Move blocks forward. s.filling, s.current, s.previous = s.previous[:0], s.filling, s.current + s.nInput += int64(len(s.current)) s.wg.Add(1) go func(src []byte) { if debugEncoder { @@ -394,6 +412,11 @@ func (e *Encoder) Close() error { if err != nil { return err } + if s.frameContentSize > 0 { + if s.nInput != s.frameContentSize { + return fmt.Errorf("frame content size %d given, but %d bytes was written", s.frameContentSize, s.nInput) + } + } if e.state.fullFrameWritten { return s.err } @@ -470,7 +493,7 @@ func (e *Encoder) EncodeAll(src, dst []byte) []byte { } fh := frameHeader{ ContentSize: uint64(len(src)), - WindowSize: uint32(enc.WindowSize(len(src))), + WindowSize: uint32(enc.WindowSize(int64(len(src)))), SingleSegment: single, Checksum: e.o.crc, DictID: e.o.dict.ID(), diff --git a/vendor/github.com/klauspost/compress/zstd/encoder_options.go b/vendor/github.com/klauspost/compress/zstd/encoder_options.go index 16d4ab63c19d..7d29e1d689ee 100644 --- a/vendor/github.com/klauspost/compress/zstd/encoder_options.go +++ b/vendor/github.com/klauspost/compress/zstd/encoder_options.go @@ -189,7 +189,7 @@ func EncoderLevelFromZstd(level int) EncoderLevel { case level >= 6 && level < 10: return SpeedBetterCompression case level >= 10: - return SpeedBetterCompression + return SpeedBestCompression } return SpeedDefault } diff --git a/vendor/github.com/klauspost/compress/zstd/hash.go b/vendor/github.com/klauspost/compress/zstd/hash.go index 4a752067fc90..cf33f29a1b48 100644 --- a/vendor/github.com/klauspost/compress/zstd/hash.go +++ b/vendor/github.com/klauspost/compress/zstd/hash.go @@ -13,24 +13,24 @@ const ( prime8bytes = 0xcf1bbcdcb7a56463 ) -// hashLen returns a hash of the lowest l bytes of u for a size size of h bytes. -// l must be >=4 and <=8. Any other value will return hash for 4 bytes. -// h should always be <32. -// Preferably h and l should be a constant. -// FIXME: This does NOT get resolved, if 'mls' is constant, -// so this cannot be used. -func hashLen(u uint64, hashLog, mls uint8) uint32 { +// hashLen returns a hash of the lowest mls bytes of with length output bits. +// mls must be >=3 and <=8. Any other value will return hash for 4 bytes. +// length should always be < 32. +// Preferably length and mls should be a constant for inlining. +func hashLen(u uint64, length, mls uint8) uint32 { switch mls { + case 3: + return (uint32(u<<8) * prime3bytes) >> (32 - length) case 5: - return hash5(u, hashLog) + return uint32(((u << (64 - 40)) * prime5bytes) >> (64 - length)) case 6: - return hash6(u, hashLog) + return uint32(((u << (64 - 48)) * prime6bytes) >> (64 - length)) case 7: - return hash7(u, hashLog) + return uint32(((u << (64 - 56)) * prime7bytes) >> (64 - length)) case 8: - return hash8(u, hashLog) + return uint32((u * prime8bytes) >> (64 - length)) default: - return hash4x64(u, hashLog) + return (uint32(u) * prime4bytes) >> (32 - length) } } @@ -39,39 +39,3 @@ func hashLen(u uint64, hashLog, mls uint8) uint32 { func hash3(u uint32, h uint8) uint32 { return ((u << (32 - 24)) * prime3bytes) >> ((32 - h) & 31) } - -// hash4 returns the hash of u to fit in a hash table with h bits. -// Preferably h should be a constant and should always be <32. -func hash4(u uint32, h uint8) uint32 { - return (u * prime4bytes) >> ((32 - h) & 31) -} - -// hash4x64 returns the hash of the lowest 4 bytes of u to fit in a hash table with h bits. -// Preferably h should be a constant and should always be <32. -func hash4x64(u uint64, h uint8) uint32 { - return (uint32(u) * prime4bytes) >> ((32 - h) & 31) -} - -// hash5 returns the hash of the lowest 5 bytes of u to fit in a hash table with h bits. -// Preferably h should be a constant and should always be <64. -func hash5(u uint64, h uint8) uint32 { - return uint32(((u << (64 - 40)) * prime5bytes) >> ((64 - h) & 63)) -} - -// hash6 returns the hash of the lowest 6 bytes of u to fit in a hash table with h bits. -// Preferably h should be a constant and should always be <64. -func hash6(u uint64, h uint8) uint32 { - return uint32(((u << (64 - 48)) * prime6bytes) >> ((64 - h) & 63)) -} - -// hash7 returns the hash of the lowest 7 bytes of u to fit in a hash table with h bits. -// Preferably h should be a constant and should always be <64. -func hash7(u uint64, h uint8) uint32 { - return uint32(((u << (64 - 56)) * prime7bytes) >> ((64 - h) & 63)) -} - -// hash8 returns the hash of u to fit in a hash table with h bits. -// Preferably h should be a constant and should always be <64. -func hash8(u uint64, h uint8) uint32 { - return uint32((u * prime8bytes) >> ((64 - h) & 63)) -} diff --git a/vendor/github.com/klauspost/compress/zstd/internal/xxhash/xxhash.go b/vendor/github.com/klauspost/compress/zstd/internal/xxhash/xxhash.go index 426b9cac7869..2c112a0ab1c1 100644 --- a/vendor/github.com/klauspost/compress/zstd/internal/xxhash/xxhash.go +++ b/vendor/github.com/klauspost/compress/zstd/internal/xxhash/xxhash.go @@ -195,7 +195,6 @@ func (d *Digest) UnmarshalBinary(b []byte) error { b, d.v4 = consumeUint64(b) b, d.total = consumeUint64(b) copy(d.mem[:], b) - b = b[len(d.mem):] d.n = int(d.total % uint64(len(d.mem))) return nil } diff --git a/vendor/github.com/klauspost/compress/zstd/internal/xxhash/xxhash_amd64.go b/vendor/github.com/klauspost/compress/zstd/internal/xxhash/xxhash_amd64.go index 35318d7c46cb..0ae847f75b05 100644 --- a/vendor/github.com/klauspost/compress/zstd/internal/xxhash/xxhash_amd64.go +++ b/vendor/github.com/klauspost/compress/zstd/internal/xxhash/xxhash_amd64.go @@ -1,6 +1,5 @@ -// +build !appengine -// +build gc -// +build !purego +//go:build !appengine && gc && !purego +// +build !appengine,gc,!purego package xxhash @@ -10,4 +9,4 @@ package xxhash func Sum64(b []byte) uint64 //go:noescape -func writeBlocks(*Digest, []byte) int +func writeBlocks(d *Digest, b []byte) int diff --git a/vendor/github.com/klauspost/compress/zstd/internal/xxhash/xxhash_amd64.s b/vendor/github.com/klauspost/compress/zstd/internal/xxhash/xxhash_amd64.s index 2c9c5357a14b..be8db5bf7960 100644 --- a/vendor/github.com/klauspost/compress/zstd/internal/xxhash/xxhash_amd64.s +++ b/vendor/github.com/klauspost/compress/zstd/internal/xxhash/xxhash_amd64.s @@ -6,7 +6,7 @@ // Register allocation: // AX h -// CX pointer to advance through b +// SI pointer to advance through b // DX n // BX loop end // R8 v1, k1 @@ -16,39 +16,39 @@ // R12 tmp // R13 prime1v // R14 prime2v -// R15 prime4v +// DI prime4v -// round reads from and advances the buffer pointer in CX. +// round reads from and advances the buffer pointer in SI. // It assumes that R13 has prime1v and R14 has prime2v. #define round(r) \ - MOVQ (CX), R12 \ - ADDQ $8, CX \ + MOVQ (SI), R12 \ + ADDQ $8, SI \ IMULQ R14, R12 \ ADDQ R12, r \ ROLQ $31, r \ IMULQ R13, r // mergeRound applies a merge round on the two registers acc and val. -// It assumes that R13 has prime1v, R14 has prime2v, and R15 has prime4v. +// It assumes that R13 has prime1v, R14 has prime2v, and DI has prime4v. #define mergeRound(acc, val) \ IMULQ R14, val \ ROLQ $31, val \ IMULQ R13, val \ XORQ val, acc \ IMULQ R13, acc \ - ADDQ R15, acc + ADDQ DI, acc // func Sum64(b []byte) uint64 TEXT ·Sum64(SB), NOSPLIT, $0-32 // Load fixed primes. MOVQ ·prime1v(SB), R13 MOVQ ·prime2v(SB), R14 - MOVQ ·prime4v(SB), R15 + MOVQ ·prime4v(SB), DI // Load slice. - MOVQ b_base+0(FP), CX + MOVQ b_base+0(FP), SI MOVQ b_len+8(FP), DX - LEAQ (CX)(DX*1), BX + LEAQ (SI)(DX*1), BX // The first loop limit will be len(b)-32. SUBQ $32, BX @@ -65,14 +65,14 @@ TEXT ·Sum64(SB), NOSPLIT, $0-32 XORQ R11, R11 SUBQ R13, R11 - // Loop until CX > BX. + // Loop until SI > BX. blockLoop: round(R8) round(R9) round(R10) round(R11) - CMPQ CX, BX + CMPQ SI, BX JLE blockLoop MOVQ R8, AX @@ -100,16 +100,16 @@ noBlocks: afterBlocks: ADDQ DX, AX - // Right now BX has len(b)-32, and we want to loop until CX > len(b)-8. + // Right now BX has len(b)-32, and we want to loop until SI > len(b)-8. ADDQ $24, BX - CMPQ CX, BX + CMPQ SI, BX JG fourByte wordLoop: // Calculate k1. - MOVQ (CX), R8 - ADDQ $8, CX + MOVQ (SI), R8 + ADDQ $8, SI IMULQ R14, R8 ROLQ $31, R8 IMULQ R13, R8 @@ -117,18 +117,18 @@ wordLoop: XORQ R8, AX ROLQ $27, AX IMULQ R13, AX - ADDQ R15, AX + ADDQ DI, AX - CMPQ CX, BX + CMPQ SI, BX JLE wordLoop fourByte: ADDQ $4, BX - CMPQ CX, BX + CMPQ SI, BX JG singles - MOVL (CX), R8 - ADDQ $4, CX + MOVL (SI), R8 + ADDQ $4, SI IMULQ R13, R8 XORQ R8, AX @@ -138,19 +138,19 @@ fourByte: singles: ADDQ $4, BX - CMPQ CX, BX + CMPQ SI, BX JGE finalize singlesLoop: - MOVBQZX (CX), R12 - ADDQ $1, CX + MOVBQZX (SI), R12 + ADDQ $1, SI IMULQ ·prime5v(SB), R12 XORQ R12, AX ROLQ $11, AX IMULQ R13, AX - CMPQ CX, BX + CMPQ SI, BX JL singlesLoop finalize: @@ -179,13 +179,13 @@ TEXT ·writeBlocks(SB), NOSPLIT, $0-40 MOVQ ·prime2v(SB), R14 // Load slice. - MOVQ arg1_base+8(FP), CX - MOVQ arg1_len+16(FP), DX - LEAQ (CX)(DX*1), BX + MOVQ b_base+8(FP), SI + MOVQ b_len+16(FP), DX + LEAQ (SI)(DX*1), BX SUBQ $32, BX // Load vN from d. - MOVQ arg+0(FP), AX + MOVQ d+0(FP), AX MOVQ 0(AX), R8 // v1 MOVQ 8(AX), R9 // v2 MOVQ 16(AX), R10 // v3 @@ -199,7 +199,7 @@ blockLoop: round(R10) round(R11) - CMPQ CX, BX + CMPQ SI, BX JLE blockLoop // Copy vN back to d. @@ -208,8 +208,8 @@ blockLoop: MOVQ R10, 16(AX) MOVQ R11, 24(AX) - // The number of bytes written is CX minus the old base pointer. - SUBQ arg1_base+8(FP), CX - MOVQ CX, ret+32(FP) + // The number of bytes written is SI minus the old base pointer. + SUBQ b_base+8(FP), SI + MOVQ SI, ret+32(FP) RET diff --git a/vendor/github.com/klauspost/compress/zstd/internal/xxhash/xxhash_other.go b/vendor/github.com/klauspost/compress/zstd/internal/xxhash/xxhash_other.go index 4a5a821603e5..1f52f296e71f 100644 --- a/vendor/github.com/klauspost/compress/zstd/internal/xxhash/xxhash_other.go +++ b/vendor/github.com/klauspost/compress/zstd/internal/xxhash/xxhash_other.go @@ -1,3 +1,4 @@ +//go:build !amd64 || appengine || !gc || purego // +build !amd64 appengine !gc purego package xxhash diff --git a/vendor/github.com/klauspost/compress/zstd/snappy.go b/vendor/github.com/klauspost/compress/zstd/snappy.go index 0372b1714a7a..9e1baad73be8 100644 --- a/vendor/github.com/klauspost/compress/zstd/snappy.go +++ b/vendor/github.com/klauspost/compress/zstd/snappy.go @@ -10,8 +10,8 @@ import ( "hash/crc32" "io" - "github.com/golang/snappy" "github.com/klauspost/compress/huff0" + snappy "github.com/klauspost/compress/internal/snapref" ) const ( diff --git a/vendor/github.com/klauspost/compress/zstd/zip.go b/vendor/github.com/klauspost/compress/zstd/zip.go index 9325b928ae2f..967f29b3120e 100644 --- a/vendor/github.com/klauspost/compress/zstd/zip.go +++ b/vendor/github.com/klauspost/compress/zstd/zip.go @@ -64,8 +64,9 @@ func (r *pooledZipReader) Close() error { } type pooledZipWriter struct { - mu sync.Mutex // guards Close and Read - enc *Encoder + mu sync.Mutex // guards Close and Read + enc *Encoder + pool *sync.Pool } func (w *pooledZipWriter) Write(p []byte) (n int, err error) { @@ -83,7 +84,7 @@ func (w *pooledZipWriter) Close() error { var err error if w.enc != nil { err = w.enc.Close() - zipReaderPool.Put(w.enc) + w.pool.Put(w.enc) w.enc = nil } return err @@ -104,7 +105,7 @@ func ZipCompressor(opts ...EOption) func(w io.Writer) (io.WriteCloser, error) { return nil, err } } - return &pooledZipWriter{enc: enc}, nil + return &pooledZipWriter{enc: enc, pool: &pool}, nil } } diff --git a/vendor/github.com/pierrec/lz4/.gitignore b/vendor/github.com/pierrec/lz4/.gitignore new file mode 100644 index 000000000000..5e987350471d --- /dev/null +++ b/vendor/github.com/pierrec/lz4/.gitignore @@ -0,0 +1,34 @@ +# Created by https://www.gitignore.io/api/macos + +### macOS ### +*.DS_Store +.AppleDouble +.LSOverride + +# Icon must end with two \r +Icon + + +# Thumbnails +._* + +# Files that might appear in the root of a volume +.DocumentRevisions-V100 +.fseventsd +.Spotlight-V100 +.TemporaryItems +.Trashes +.VolumeIcon.icns +.com.apple.timemachine.donotpresent + +# Directories potentially created on remote AFP share +.AppleDB +.AppleDesktop +Network Trash Folder +Temporary Items +.apdisk + +# End of https://www.gitignore.io/api/macos + +cmd/*/*exe +.idea \ No newline at end of file diff --git a/vendor/github.com/pierrec/lz4/.travis.yml b/vendor/github.com/pierrec/lz4/.travis.yml new file mode 100644 index 000000000000..fd6c6db713d3 --- /dev/null +++ b/vendor/github.com/pierrec/lz4/.travis.yml @@ -0,0 +1,24 @@ +language: go + +env: + - GO111MODULE=off + +go: + - 1.9.x + - 1.10.x + - 1.11.x + - 1.12.x + - master + +matrix: + fast_finish: true + allow_failures: + - go: master + +sudo: false + +script: + - go test -v -cpu=2 + - go test -v -cpu=2 -race + - go test -v -cpu=2 -tags noasm + - go test -v -cpu=2 -race -tags noasm diff --git a/vendor/github.com/pierrec/lz4/LICENSE b/vendor/github.com/pierrec/lz4/LICENSE new file mode 100644 index 000000000000..bd899d8353dd --- /dev/null +++ b/vendor/github.com/pierrec/lz4/LICENSE @@ -0,0 +1,28 @@ +Copyright (c) 2015, Pierre Curto +All rights reserved. + +Redistribution and use in source and binary forms, with or without +modification, are permitted provided that the following conditions are met: + +* Redistributions of source code must retain the above copyright notice, this + list of conditions and the following disclaimer. + +* Redistributions in binary form must reproduce the above copyright notice, + this list of conditions and the following disclaimer in the documentation + and/or other materials provided with the distribution. + +* Neither the name of xxHash nor the names of its + contributors may be used to endorse or promote products derived from + this software without specific prior written permission. + +THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS IS" +AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE +IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE ARE +DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT HOLDER OR CONTRIBUTORS BE LIABLE +FOR ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR CONSEQUENTIAL +DAMAGES (INCLUDING, BUT NOT LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR +SERVICES; LOSS OF USE, DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER +CAUSED AND ON ANY THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, +OR TORT (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE +OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. + diff --git a/vendor/github.com/pierrec/lz4/README.md b/vendor/github.com/pierrec/lz4/README.md new file mode 100644 index 000000000000..4ee388e81bfb --- /dev/null +++ b/vendor/github.com/pierrec/lz4/README.md @@ -0,0 +1,90 @@ +# lz4 : LZ4 compression in pure Go + +[![GoDoc](https://godoc.org/github.com/pierrec/lz4?status.svg)](https://godoc.org/github.com/pierrec/lz4) +[![Build Status](https://travis-ci.org/pierrec/lz4.svg?branch=master)](https://travis-ci.org/pierrec/lz4) +[![Go Report Card](https://goreportcard.com/badge/github.com/pierrec/lz4)](https://goreportcard.com/report/github.com/pierrec/lz4) +[![GitHub tag (latest SemVer)](https://img.shields.io/github/tag/pierrec/lz4.svg?style=social)](https://github.com/pierrec/lz4/tags) + +## Overview + +This package provides a streaming interface to [LZ4 data streams](http://fastcompression.blogspot.fr/2013/04/lz4-streaming-format-final.html) as well as low level compress and uncompress functions for LZ4 data blocks. +The implementation is based on the reference C [one](https://github.com/lz4/lz4). + +## Install + +Assuming you have the go toolchain installed: + +``` +go get github.com/pierrec/lz4 +``` + +There is a command line interface tool to compress and decompress LZ4 files. + +``` +go install github.com/pierrec/lz4/cmd/lz4c +``` + +Usage + +``` +Usage of lz4c: + -version + print the program version + +Subcommands: +Compress the given files or from stdin to stdout. +compress [arguments] [ ...] + -bc + enable block checksum + -l int + compression level (0=fastest) + -sc + disable stream checksum + -size string + block max size [64K,256K,1M,4M] (default "4M") + +Uncompress the given files or from stdin to stdout. +uncompress [arguments] [ ...] + +``` + + +## Example + +``` +// Compress and uncompress an input string. +s := "hello world" +r := strings.NewReader(s) + +// The pipe will uncompress the data from the writer. +pr, pw := io.Pipe() +zw := lz4.NewWriter(pw) +zr := lz4.NewReader(pr) + +go func() { + // Compress the input string. + _, _ = io.Copy(zw, r) + _ = zw.Close() // Make sure the writer is closed + _ = pw.Close() // Terminate the pipe +}() + +_, _ = io.Copy(os.Stdout, zr) + +// Output: +// hello world +``` + +## Contributing + +Contributions are very welcome for bug fixing, performance improvements...! + +- Open an issue with a proper description +- Send a pull request with appropriate test case(s) + +## Contributors + +Thanks to all [contributors](https://github.com/pierrec/lz4/graphs/contributors) so far! + +Special thanks to [@Zariel](https://github.com/Zariel) for his asm implementation of the decoder. + +Special thanks to [@klauspost](https://github.com/klauspost) for his work on optimizing the code. diff --git a/vendor/github.com/pierrec/lz4/block.go b/vendor/github.com/pierrec/lz4/block.go new file mode 100644 index 000000000000..664d9be580d4 --- /dev/null +++ b/vendor/github.com/pierrec/lz4/block.go @@ -0,0 +1,413 @@ +package lz4 + +import ( + "encoding/binary" + "math/bits" + "sync" +) + +// blockHash hashes the lower 6 bytes into a value < htSize. +func blockHash(x uint64) uint32 { + const prime6bytes = 227718039650203 + return uint32(((x << (64 - 48)) * prime6bytes) >> (64 - hashLog)) +} + +// CompressBlockBound returns the maximum size of a given buffer of size n, when not compressible. +func CompressBlockBound(n int) int { + return n + n/255 + 16 +} + +// UncompressBlock uncompresses the source buffer into the destination one, +// and returns the uncompressed size. +// +// The destination buffer must be sized appropriately. +// +// An error is returned if the source data is invalid or the destination buffer is too small. +func UncompressBlock(src, dst []byte) (int, error) { + if len(src) == 0 { + return 0, nil + } + if di := decodeBlock(dst, src); di >= 0 { + return di, nil + } + return 0, ErrInvalidSourceShortBuffer +} + +// CompressBlock compresses the source buffer into the destination one. +// This is the fast version of LZ4 compression and also the default one. +// +// The argument hashTable is scratch space for a hash table used by the +// compressor. If provided, it should have length at least 1<<16. If it is +// shorter (or nil), CompressBlock allocates its own hash table. +// +// The size of the compressed data is returned. +// +// If the destination buffer size is lower than CompressBlockBound and +// the compressed size is 0 and no error, then the data is incompressible. +// +// An error is returned if the destination buffer is too small. +func CompressBlock(src, dst []byte, hashTable []int) (_ int, err error) { + defer recoverBlock(&err) + + // Return 0, nil only if the destination buffer size is < CompressBlockBound. + isNotCompressible := len(dst) < CompressBlockBound(len(src)) + + // adaptSkipLog sets how quickly the compressor begins skipping blocks when data is incompressible. + // This significantly speeds up incompressible data and usually has very small impact on compression. + // bytes to skip = 1 + (bytes since last match >> adaptSkipLog) + const adaptSkipLog = 7 + if len(hashTable) < htSize { + htIface := htPool.Get() + defer htPool.Put(htIface) + hashTable = (*(htIface).(*[htSize]int))[:] + } + // Prove to the compiler the table has at least htSize elements. + // The compiler can see that "uint32() >> hashShift" cannot be out of bounds. + hashTable = hashTable[:htSize] + + // si: Current position of the search. + // anchor: Position of the current literals. + var si, di, anchor int + sn := len(src) - mfLimit + if sn <= 0 { + goto lastLiterals + } + + // Fast scan strategy: the hash table only stores the last 4 bytes sequences. + for si < sn { + // Hash the next 6 bytes (sequence)... + match := binary.LittleEndian.Uint64(src[si:]) + h := blockHash(match) + h2 := blockHash(match >> 8) + + // We check a match at s, s+1 and s+2 and pick the first one we get. + // Checking 3 only requires us to load the source one. + ref := hashTable[h] + ref2 := hashTable[h2] + hashTable[h] = si + hashTable[h2] = si + 1 + offset := si - ref + + // If offset <= 0 we got an old entry in the hash table. + if offset <= 0 || offset >= winSize || // Out of window. + uint32(match) != binary.LittleEndian.Uint32(src[ref:]) { // Hash collision on different matches. + // No match. Start calculating another hash. + // The processor can usually do this out-of-order. + h = blockHash(match >> 16) + ref = hashTable[h] + + // Check the second match at si+1 + si += 1 + offset = si - ref2 + + if offset <= 0 || offset >= winSize || + uint32(match>>8) != binary.LittleEndian.Uint32(src[ref2:]) { + // No match. Check the third match at si+2 + si += 1 + offset = si - ref + hashTable[h] = si + + if offset <= 0 || offset >= winSize || + uint32(match>>16) != binary.LittleEndian.Uint32(src[ref:]) { + // Skip one extra byte (at si+3) before we check 3 matches again. + si += 2 + (si-anchor)>>adaptSkipLog + continue + } + } + } + + // Match found. + lLen := si - anchor // Literal length. + // We already matched 4 bytes. + mLen := 4 + + // Extend backwards if we can, reducing literals. + tOff := si - offset - 1 + for lLen > 0 && tOff >= 0 && src[si-1] == src[tOff] { + si-- + tOff-- + lLen-- + mLen++ + } + + // Add the match length, so we continue search at the end. + // Use mLen to store the offset base. + si, mLen = si+mLen, si+minMatch + + // Find the longest match by looking by batches of 8 bytes. + for si+8 < sn { + x := binary.LittleEndian.Uint64(src[si:]) ^ binary.LittleEndian.Uint64(src[si-offset:]) + if x == 0 { + si += 8 + } else { + // Stop is first non-zero byte. + si += bits.TrailingZeros64(x) >> 3 + break + } + } + + mLen = si - mLen + if mLen < 0xF { + dst[di] = byte(mLen) + } else { + dst[di] = 0xF + } + + // Encode literals length. + if lLen < 0xF { + dst[di] |= byte(lLen << 4) + } else { + dst[di] |= 0xF0 + di++ + l := lLen - 0xF + for ; l >= 0xFF; l -= 0xFF { + dst[di] = 0xFF + di++ + } + dst[di] = byte(l) + } + di++ + + // Literals. + copy(dst[di:di+lLen], src[anchor:anchor+lLen]) + di += lLen + 2 + anchor = si + + // Encode offset. + _ = dst[di] // Bound check elimination. + dst[di-2], dst[di-1] = byte(offset), byte(offset>>8) + + // Encode match length part 2. + if mLen >= 0xF { + for mLen -= 0xF; mLen >= 0xFF; mLen -= 0xFF { + dst[di] = 0xFF + di++ + } + dst[di] = byte(mLen) + di++ + } + // Check if we can load next values. + if si >= sn { + break + } + // Hash match end-2 + h = blockHash(binary.LittleEndian.Uint64(src[si-2:])) + hashTable[h] = si - 2 + } + +lastLiterals: + if isNotCompressible && anchor == 0 { + // Incompressible. + return 0, nil + } + + // Last literals. + lLen := len(src) - anchor + if lLen < 0xF { + dst[di] = byte(lLen << 4) + } else { + dst[di] = 0xF0 + di++ + for lLen -= 0xF; lLen >= 0xFF; lLen -= 0xFF { + dst[di] = 0xFF + di++ + } + dst[di] = byte(lLen) + } + di++ + + // Write the last literals. + if isNotCompressible && di >= anchor { + // Incompressible. + return 0, nil + } + di += copy(dst[di:di+len(src)-anchor], src[anchor:]) + return di, nil +} + +// Pool of hash tables for CompressBlock. +var htPool = sync.Pool{ + New: func() interface{} { + return new([htSize]int) + }, +} + +// blockHash hashes 4 bytes into a value < winSize. +func blockHashHC(x uint32) uint32 { + const hasher uint32 = 2654435761 // Knuth multiplicative hash. + return x * hasher >> (32 - winSizeLog) +} + +// CompressBlockHC compresses the source buffer src into the destination dst +// with max search depth (use 0 or negative value for no max). +// +// CompressBlockHC compression ratio is better than CompressBlock but it is also slower. +// +// The size of the compressed data is returned. +// +// If the destination buffer size is lower than CompressBlockBound and +// the compressed size is 0 and no error, then the data is incompressible. +// +// An error is returned if the destination buffer is too small. +func CompressBlockHC(src, dst []byte, depth int) (_ int, err error) { + defer recoverBlock(&err) + + // Return 0, nil only if the destination buffer size is < CompressBlockBound. + isNotCompressible := len(dst) < CompressBlockBound(len(src)) + + // adaptSkipLog sets how quickly the compressor begins skipping blocks when data is incompressible. + // This significantly speeds up incompressible data and usually has very small impact on compression. + // bytes to skip = 1 + (bytes since last match >> adaptSkipLog) + const adaptSkipLog = 7 + + var si, di, anchor int + + // hashTable: stores the last position found for a given hash + // chainTable: stores previous positions for a given hash + var hashTable, chainTable [winSize]int + + if depth <= 0 { + depth = winSize + } + + sn := len(src) - mfLimit + if sn <= 0 { + goto lastLiterals + } + + for si < sn { + // Hash the next 4 bytes (sequence). + match := binary.LittleEndian.Uint32(src[si:]) + h := blockHashHC(match) + + // Follow the chain until out of window and give the longest match. + mLen := 0 + offset := 0 + for next, try := hashTable[h], depth; try > 0 && next > 0 && si-next < winSize; next = chainTable[next&winMask] { + // The first (mLen==0) or next byte (mLen>=minMatch) at current match length + // must match to improve on the match length. + if src[next+mLen] != src[si+mLen] { + continue + } + ml := 0 + // Compare the current position with a previous with the same hash. + for ml < sn-si { + x := binary.LittleEndian.Uint64(src[next+ml:]) ^ binary.LittleEndian.Uint64(src[si+ml:]) + if x == 0 { + ml += 8 + } else { + // Stop is first non-zero byte. + ml += bits.TrailingZeros64(x) >> 3 + break + } + } + if ml < minMatch || ml <= mLen { + // Match too small (>adaptSkipLog + continue + } + + // Match found. + // Update hash/chain tables with overlapping bytes: + // si already hashed, add everything from si+1 up to the match length. + winStart := si + 1 + if ws := si + mLen - winSize; ws > winStart { + winStart = ws + } + for si, ml := winStart, si+mLen; si < ml; { + match >>= 8 + match |= uint32(src[si+3]) << 24 + h := blockHashHC(match) + chainTable[si&winMask] = hashTable[h] + hashTable[h] = si + si++ + } + + lLen := si - anchor + si += mLen + mLen -= minMatch // Match length does not include minMatch. + + if mLen < 0xF { + dst[di] = byte(mLen) + } else { + dst[di] = 0xF + } + + // Encode literals length. + if lLen < 0xF { + dst[di] |= byte(lLen << 4) + } else { + dst[di] |= 0xF0 + di++ + l := lLen - 0xF + for ; l >= 0xFF; l -= 0xFF { + dst[di] = 0xFF + di++ + } + dst[di] = byte(l) + } + di++ + + // Literals. + copy(dst[di:di+lLen], src[anchor:anchor+lLen]) + di += lLen + anchor = si + + // Encode offset. + di += 2 + dst[di-2], dst[di-1] = byte(offset), byte(offset>>8) + + // Encode match length part 2. + if mLen >= 0xF { + for mLen -= 0xF; mLen >= 0xFF; mLen -= 0xFF { + dst[di] = 0xFF + di++ + } + dst[di] = byte(mLen) + di++ + } + } + + if isNotCompressible && anchor == 0 { + // Incompressible. + return 0, nil + } + + // Last literals. +lastLiterals: + lLen := len(src) - anchor + if lLen < 0xF { + dst[di] = byte(lLen << 4) + } else { + dst[di] = 0xF0 + di++ + lLen -= 0xF + for ; lLen >= 0xFF; lLen -= 0xFF { + dst[di] = 0xFF + di++ + } + dst[di] = byte(lLen) + } + di++ + + // Write the last literals. + if isNotCompressible && di >= anchor { + // Incompressible. + return 0, nil + } + di += copy(dst[di:di+len(src)-anchor], src[anchor:]) + return di, nil +} diff --git a/vendor/github.com/pierrec/lz4/debug.go b/vendor/github.com/pierrec/lz4/debug.go new file mode 100644 index 000000000000..bc5e78d40f0a --- /dev/null +++ b/vendor/github.com/pierrec/lz4/debug.go @@ -0,0 +1,23 @@ +// +build lz4debug + +package lz4 + +import ( + "fmt" + "os" + "path/filepath" + "runtime" +) + +const debugFlag = true + +func debug(args ...interface{}) { + _, file, line, _ := runtime.Caller(1) + file = filepath.Base(file) + + f := fmt.Sprintf("LZ4: %s:%d %s", file, line, args[0]) + if f[len(f)-1] != '\n' { + f += "\n" + } + fmt.Fprintf(os.Stderr, f, args[1:]...) +} diff --git a/vendor/github.com/pierrec/lz4/debug_stub.go b/vendor/github.com/pierrec/lz4/debug_stub.go new file mode 100644 index 000000000000..44211ad96453 --- /dev/null +++ b/vendor/github.com/pierrec/lz4/debug_stub.go @@ -0,0 +1,7 @@ +// +build !lz4debug + +package lz4 + +const debugFlag = false + +func debug(args ...interface{}) {} diff --git a/vendor/github.com/pierrec/lz4/decode_amd64.go b/vendor/github.com/pierrec/lz4/decode_amd64.go new file mode 100644 index 000000000000..43cc14fbe2e3 --- /dev/null +++ b/vendor/github.com/pierrec/lz4/decode_amd64.go @@ -0,0 +1,8 @@ +// +build !appengine +// +build gc +// +build !noasm + +package lz4 + +//go:noescape +func decodeBlock(dst, src []byte) int diff --git a/vendor/github.com/pierrec/lz4/decode_amd64.s b/vendor/github.com/pierrec/lz4/decode_amd64.s new file mode 100644 index 000000000000..20fef39759cb --- /dev/null +++ b/vendor/github.com/pierrec/lz4/decode_amd64.s @@ -0,0 +1,375 @@ +// +build !appengine +// +build gc +// +build !noasm + +#include "textflag.h" + +// AX scratch +// BX scratch +// CX scratch +// DX token +// +// DI &dst +// SI &src +// R8 &dst + len(dst) +// R9 &src + len(src) +// R11 &dst +// R12 short output end +// R13 short input end +// func decodeBlock(dst, src []byte) int +// using 50 bytes of stack currently +TEXT ·decodeBlock(SB), NOSPLIT, $64-56 + MOVQ dst_base+0(FP), DI + MOVQ DI, R11 + MOVQ dst_len+8(FP), R8 + ADDQ DI, R8 + + MOVQ src_base+24(FP), SI + MOVQ src_len+32(FP), R9 + ADDQ SI, R9 + + // shortcut ends + // short output end + MOVQ R8, R12 + SUBQ $32, R12 + // short input end + MOVQ R9, R13 + SUBQ $16, R13 + +loop: + // for si < len(src) + CMPQ SI, R9 + JGE end + + // token := uint32(src[si]) + MOVBQZX (SI), DX + INCQ SI + + // lit_len = token >> 4 + // if lit_len > 0 + // CX = lit_len + MOVQ DX, CX + SHRQ $4, CX + + // if lit_len != 0xF + CMPQ CX, $0xF + JEQ lit_len_loop_pre + CMPQ DI, R12 + JGE lit_len_loop_pre + CMPQ SI, R13 + JGE lit_len_loop_pre + + // copy shortcut + + // A two-stage shortcut for the most common case: + // 1) If the literal length is 0..14, and there is enough space, + // enter the shortcut and copy 16 bytes on behalf of the literals + // (in the fast mode, only 8 bytes can be safely copied this way). + // 2) Further if the match length is 4..18, copy 18 bytes in a similar + // manner; but we ensure that there's enough space in the output for + // those 18 bytes earlier, upon entering the shortcut (in other words, + // there is a combined check for both stages). + + // copy literal + MOVOU (SI), X0 + MOVOU X0, (DI) + ADDQ CX, DI + ADDQ CX, SI + + MOVQ DX, CX + ANDQ $0xF, CX + + // The second stage: prepare for match copying, decode full info. + // If it doesn't work out, the info won't be wasted. + // offset := uint16(data[:2]) + MOVWQZX (SI), DX + ADDQ $2, SI + + MOVQ DI, AX + SUBQ DX, AX + CMPQ AX, DI + JGT err_short_buf + + // if we can't do the second stage then jump straight to read the + // match length, we already have the offset. + CMPQ CX, $0xF + JEQ match_len_loop_pre + CMPQ DX, $8 + JLT match_len_loop_pre + CMPQ AX, R11 + JLT err_short_buf + + // memcpy(op + 0, match + 0, 8); + MOVQ (AX), BX + MOVQ BX, (DI) + // memcpy(op + 8, match + 8, 8); + MOVQ 8(AX), BX + MOVQ BX, 8(DI) + // memcpy(op +16, match +16, 2); + MOVW 16(AX), BX + MOVW BX, 16(DI) + + ADDQ $4, DI // minmatch + ADDQ CX, DI + + // shortcut complete, load next token + JMP loop + +lit_len_loop_pre: + // if lit_len > 0 + CMPQ CX, $0 + JEQ offset + CMPQ CX, $0xF + JNE copy_literal + +lit_len_loop: + // for src[si] == 0xFF + CMPB (SI), $0xFF + JNE lit_len_finalise + + // bounds check src[si+1] + MOVQ SI, AX + ADDQ $1, AX + CMPQ AX, R9 + JGT err_short_buf + + // lit_len += 0xFF + ADDQ $0xFF, CX + INCQ SI + JMP lit_len_loop + +lit_len_finalise: + // lit_len += int(src[si]) + // si++ + MOVBQZX (SI), AX + ADDQ AX, CX + INCQ SI + +copy_literal: + // bounds check src and dst + MOVQ SI, AX + ADDQ CX, AX + CMPQ AX, R9 + JGT err_short_buf + + MOVQ DI, AX + ADDQ CX, AX + CMPQ AX, R8 + JGT err_short_buf + + // whats a good cut off to call memmove? + CMPQ CX, $16 + JGT memmove_lit + + // if len(dst[di:]) < 16 + MOVQ R8, AX + SUBQ DI, AX + CMPQ AX, $16 + JLT memmove_lit + + // if len(src[si:]) < 16 + MOVQ R9, AX + SUBQ SI, AX + CMPQ AX, $16 + JLT memmove_lit + + MOVOU (SI), X0 + MOVOU X0, (DI) + + JMP finish_lit_copy + +memmove_lit: + // memmove(to, from, len) + MOVQ DI, 0(SP) + MOVQ SI, 8(SP) + MOVQ CX, 16(SP) + // spill + MOVQ DI, 24(SP) + MOVQ SI, 32(SP) + MOVQ CX, 40(SP) // need len to inc SI, DI after + MOVB DX, 48(SP) + CALL runtime·memmove(SB) + + // restore registers + MOVQ 24(SP), DI + MOVQ 32(SP), SI + MOVQ 40(SP), CX + MOVB 48(SP), DX + + // recalc initial values + MOVQ dst_base+0(FP), R8 + MOVQ R8, R11 + ADDQ dst_len+8(FP), R8 + MOVQ src_base+24(FP), R9 + ADDQ src_len+32(FP), R9 + MOVQ R8, R12 + SUBQ $32, R12 + MOVQ R9, R13 + SUBQ $16, R13 + +finish_lit_copy: + ADDQ CX, SI + ADDQ CX, DI + + CMPQ SI, R9 + JGE end + +offset: + // CX := mLen + // free up DX to use for offset + MOVQ DX, CX + + MOVQ SI, AX + ADDQ $2, AX + CMPQ AX, R9 + JGT err_short_buf + + // offset + // DX := int(src[si]) | int(src[si+1])<<8 + MOVWQZX (SI), DX + ADDQ $2, SI + + // 0 offset is invalid + CMPQ DX, $0 + JEQ err_corrupt + + ANDB $0xF, CX + +match_len_loop_pre: + // if mlen != 0xF + CMPB CX, $0xF + JNE copy_match + +match_len_loop: + // for src[si] == 0xFF + // lit_len += 0xFF + CMPB (SI), $0xFF + JNE match_len_finalise + + // bounds check src[si+1] + MOVQ SI, AX + ADDQ $1, AX + CMPQ AX, R9 + JGT err_short_buf + + ADDQ $0xFF, CX + INCQ SI + JMP match_len_loop + +match_len_finalise: + // lit_len += int(src[si]) + // si++ + MOVBQZX (SI), AX + ADDQ AX, CX + INCQ SI + +copy_match: + // mLen += minMatch + ADDQ $4, CX + + // check we have match_len bytes left in dst + // di+match_len < len(dst) + MOVQ DI, AX + ADDQ CX, AX + CMPQ AX, R8 + JGT err_short_buf + + // DX = offset + // CX = match_len + // BX = &dst + (di - offset) + MOVQ DI, BX + SUBQ DX, BX + + // check BX is within dst + // if BX < &dst + CMPQ BX, R11 + JLT err_short_buf + + // if offset + match_len < di + MOVQ BX, AX + ADDQ CX, AX + CMPQ DI, AX + JGT copy_interior_match + + // AX := len(dst[:di]) + // MOVQ DI, AX + // SUBQ R11, AX + + // copy 16 bytes at a time + // if di-offset < 16 copy 16-(di-offset) bytes to di + // then do the remaining + +copy_match_loop: + // for match_len >= 0 + // dst[di] = dst[i] + // di++ + // i++ + MOVB (BX), AX + MOVB AX, (DI) + INCQ DI + INCQ BX + DECQ CX + + CMPQ CX, $0 + JGT copy_match_loop + + JMP loop + +copy_interior_match: + CMPQ CX, $16 + JGT memmove_match + + // if len(dst[di:]) < 16 + MOVQ R8, AX + SUBQ DI, AX + CMPQ AX, $16 + JLT memmove_match + + MOVOU (BX), X0 + MOVOU X0, (DI) + + ADDQ CX, DI + JMP loop + +memmove_match: + // memmove(to, from, len) + MOVQ DI, 0(SP) + MOVQ BX, 8(SP) + MOVQ CX, 16(SP) + // spill + MOVQ DI, 24(SP) + MOVQ SI, 32(SP) + MOVQ CX, 40(SP) // need len to inc SI, DI after + CALL runtime·memmove(SB) + + // restore registers + MOVQ 24(SP), DI + MOVQ 32(SP), SI + MOVQ 40(SP), CX + + // recalc initial values + MOVQ dst_base+0(FP), R8 + MOVQ R8, R11 // TODO: make these sensible numbers + ADDQ dst_len+8(FP), R8 + MOVQ src_base+24(FP), R9 + ADDQ src_len+32(FP), R9 + MOVQ R8, R12 + SUBQ $32, R12 + MOVQ R9, R13 + SUBQ $16, R13 + + ADDQ CX, DI + JMP loop + +err_corrupt: + MOVQ $-1, ret+48(FP) + RET + +err_short_buf: + MOVQ $-2, ret+48(FP) + RET + +end: + SUBQ R11, DI + MOVQ DI, ret+48(FP) + RET diff --git a/vendor/github.com/pierrec/lz4/decode_other.go b/vendor/github.com/pierrec/lz4/decode_other.go new file mode 100644 index 000000000000..919888edf7dc --- /dev/null +++ b/vendor/github.com/pierrec/lz4/decode_other.go @@ -0,0 +1,98 @@ +// +build !amd64 appengine !gc noasm + +package lz4 + +func decodeBlock(dst, src []byte) (ret int) { + const hasError = -2 + defer func() { + if recover() != nil { + ret = hasError + } + }() + + var si, di int + for { + // Literals and match lengths (token). + b := int(src[si]) + si++ + + // Literals. + if lLen := b >> 4; lLen > 0 { + switch { + case lLen < 0xF && si+16 < len(src): + // Shortcut 1 + // if we have enough room in src and dst, and the literals length + // is small enough (0..14) then copy all 16 bytes, even if not all + // are part of the literals. + copy(dst[di:], src[si:si+16]) + si += lLen + di += lLen + if mLen := b & 0xF; mLen < 0xF { + // Shortcut 2 + // if the match length (4..18) fits within the literals, then copy + // all 18 bytes, even if not all are part of the literals. + mLen += 4 + if offset := int(src[si]) | int(src[si+1])<<8; mLen <= offset { + i := di - offset + end := i + 18 + if end > len(dst) { + // The remaining buffer may not hold 18 bytes. + // See https://github.com/pierrec/lz4/issues/51. + end = len(dst) + } + copy(dst[di:], dst[i:end]) + si += 2 + di += mLen + continue + } + } + case lLen == 0xF: + for src[si] == 0xFF { + lLen += 0xFF + si++ + } + lLen += int(src[si]) + si++ + fallthrough + default: + copy(dst[di:di+lLen], src[si:si+lLen]) + si += lLen + di += lLen + } + } + if si >= len(src) { + return di + } + + offset := int(src[si]) | int(src[si+1])<<8 + if offset == 0 { + return hasError + } + si += 2 + + // Match. + mLen := b & 0xF + if mLen == 0xF { + for src[si] == 0xFF { + mLen += 0xFF + si++ + } + mLen += int(src[si]) + si++ + } + mLen += minMatch + + // Copy the match. + expanded := dst[di-offset:] + if mLen > offset { + // Efficiently copy the match dst[di-offset:di] into the dst slice. + bytesToCopy := offset * (mLen / offset) + for n := offset; n <= bytesToCopy+offset; n *= 2 { + copy(expanded[n:], expanded[:n]) + } + di += bytesToCopy + mLen -= bytesToCopy + } + di += copy(dst[di:di+mLen], expanded[:mLen]) + } +} diff --git a/vendor/github.com/pierrec/lz4/errors.go b/vendor/github.com/pierrec/lz4/errors.go new file mode 100644 index 000000000000..1c45d1813cef --- /dev/null +++ b/vendor/github.com/pierrec/lz4/errors.go @@ -0,0 +1,30 @@ +package lz4 + +import ( + "errors" + "fmt" + "os" + rdebug "runtime/debug" +) + +var ( + // ErrInvalidSourceShortBuffer is returned by UncompressBlock or CompressBLock when a compressed + // block is corrupted or the destination buffer is not large enough for the uncompressed data. + ErrInvalidSourceShortBuffer = errors.New("lz4: invalid source or destination buffer too short") + // ErrInvalid is returned when reading an invalid LZ4 archive. + ErrInvalid = errors.New("lz4: bad magic number") + // ErrBlockDependency is returned when attempting to decompress an archive created with block dependency. + ErrBlockDependency = errors.New("lz4: block dependency not supported") + // ErrUnsupportedSeek is returned when attempting to Seek any way but forward from the current position. + ErrUnsupportedSeek = errors.New("lz4: can only seek forward from io.SeekCurrent") +) + +func recoverBlock(e *error) { + if r := recover(); r != nil && *e == nil { + if debugFlag { + fmt.Fprintln(os.Stderr, r) + rdebug.PrintStack() + } + *e = ErrInvalidSourceShortBuffer + } +} diff --git a/vendor/github.com/pierrec/lz4/internal/xxh32/xxh32zero.go b/vendor/github.com/pierrec/lz4/internal/xxh32/xxh32zero.go new file mode 100644 index 000000000000..7a76a6bce2b5 --- /dev/null +++ b/vendor/github.com/pierrec/lz4/internal/xxh32/xxh32zero.go @@ -0,0 +1,223 @@ +// Package xxh32 implements the very fast XXH hashing algorithm (32 bits version). +// (https://github.com/Cyan4973/XXH/) +package xxh32 + +import ( + "encoding/binary" +) + +const ( + prime1 uint32 = 2654435761 + prime2 uint32 = 2246822519 + prime3 uint32 = 3266489917 + prime4 uint32 = 668265263 + prime5 uint32 = 374761393 + + primeMask = 0xFFFFFFFF + prime1plus2 = uint32((uint64(prime1) + uint64(prime2)) & primeMask) // 606290984 + prime1minus = uint32((-int64(prime1)) & primeMask) // 1640531535 +) + +// XXHZero represents an xxhash32 object with seed 0. +type XXHZero struct { + v1 uint32 + v2 uint32 + v3 uint32 + v4 uint32 + totalLen uint64 + buf [16]byte + bufused int +} + +// Sum appends the current hash to b and returns the resulting slice. +// It does not change the underlying hash state. +func (xxh XXHZero) Sum(b []byte) []byte { + h32 := xxh.Sum32() + return append(b, byte(h32), byte(h32>>8), byte(h32>>16), byte(h32>>24)) +} + +// Reset resets the Hash to its initial state. +func (xxh *XXHZero) Reset() { + xxh.v1 = prime1plus2 + xxh.v2 = prime2 + xxh.v3 = 0 + xxh.v4 = prime1minus + xxh.totalLen = 0 + xxh.bufused = 0 +} + +// Size returns the number of bytes returned by Sum(). +func (xxh *XXHZero) Size() int { + return 4 +} + +// BlockSize gives the minimum number of bytes accepted by Write(). +func (xxh *XXHZero) BlockSize() int { + return 1 +} + +// Write adds input bytes to the Hash. +// It never returns an error. +func (xxh *XXHZero) Write(input []byte) (int, error) { + if xxh.totalLen == 0 { + xxh.Reset() + } + n := len(input) + m := xxh.bufused + + xxh.totalLen += uint64(n) + + r := len(xxh.buf) - m + if n < r { + copy(xxh.buf[m:], input) + xxh.bufused += len(input) + return n, nil + } + + p := 0 + // Causes compiler to work directly from registers instead of stack: + v1, v2, v3, v4 := xxh.v1, xxh.v2, xxh.v3, xxh.v4 + if m > 0 { + // some data left from previous update + copy(xxh.buf[xxh.bufused:], input[:r]) + xxh.bufused += len(input) - r + + // fast rotl(13) + buf := xxh.buf[:16] // BCE hint. + v1 = rol13(v1+binary.LittleEndian.Uint32(buf[:])*prime2) * prime1 + v2 = rol13(v2+binary.LittleEndian.Uint32(buf[4:])*prime2) * prime1 + v3 = rol13(v3+binary.LittleEndian.Uint32(buf[8:])*prime2) * prime1 + v4 = rol13(v4+binary.LittleEndian.Uint32(buf[12:])*prime2) * prime1 + p = r + xxh.bufused = 0 + } + + for n := n - 16; p <= n; p += 16 { + sub := input[p:][:16] //BCE hint for compiler + v1 = rol13(v1+binary.LittleEndian.Uint32(sub[:])*prime2) * prime1 + v2 = rol13(v2+binary.LittleEndian.Uint32(sub[4:])*prime2) * prime1 + v3 = rol13(v3+binary.LittleEndian.Uint32(sub[8:])*prime2) * prime1 + v4 = rol13(v4+binary.LittleEndian.Uint32(sub[12:])*prime2) * prime1 + } + xxh.v1, xxh.v2, xxh.v3, xxh.v4 = v1, v2, v3, v4 + + copy(xxh.buf[xxh.bufused:], input[p:]) + xxh.bufused += len(input) - p + + return n, nil +} + +// Sum32 returns the 32 bits Hash value. +func (xxh *XXHZero) Sum32() uint32 { + h32 := uint32(xxh.totalLen) + if h32 >= 16 { + h32 += rol1(xxh.v1) + rol7(xxh.v2) + rol12(xxh.v3) + rol18(xxh.v4) + } else { + h32 += prime5 + } + + p := 0 + n := xxh.bufused + buf := xxh.buf + for n := n - 4; p <= n; p += 4 { + h32 += binary.LittleEndian.Uint32(buf[p:p+4]) * prime3 + h32 = rol17(h32) * prime4 + } + for ; p < n; p++ { + h32 += uint32(buf[p]) * prime5 + h32 = rol11(h32) * prime1 + } + + h32 ^= h32 >> 15 + h32 *= prime2 + h32 ^= h32 >> 13 + h32 *= prime3 + h32 ^= h32 >> 16 + + return h32 +} + +// ChecksumZero returns the 32bits Hash value. +func ChecksumZero(input []byte) uint32 { + n := len(input) + h32 := uint32(n) + + if n < 16 { + h32 += prime5 + } else { + v1 := prime1plus2 + v2 := prime2 + v3 := uint32(0) + v4 := prime1minus + p := 0 + for n := n - 16; p <= n; p += 16 { + sub := input[p:][:16] //BCE hint for compiler + v1 = rol13(v1+binary.LittleEndian.Uint32(sub[:])*prime2) * prime1 + v2 = rol13(v2+binary.LittleEndian.Uint32(sub[4:])*prime2) * prime1 + v3 = rol13(v3+binary.LittleEndian.Uint32(sub[8:])*prime2) * prime1 + v4 = rol13(v4+binary.LittleEndian.Uint32(sub[12:])*prime2) * prime1 + } + input = input[p:] + n -= p + h32 += rol1(v1) + rol7(v2) + rol12(v3) + rol18(v4) + } + + p := 0 + for n := n - 4; p <= n; p += 4 { + h32 += binary.LittleEndian.Uint32(input[p:p+4]) * prime3 + h32 = rol17(h32) * prime4 + } + for p < n { + h32 += uint32(input[p]) * prime5 + h32 = rol11(h32) * prime1 + p++ + } + + h32 ^= h32 >> 15 + h32 *= prime2 + h32 ^= h32 >> 13 + h32 *= prime3 + h32 ^= h32 >> 16 + + return h32 +} + +// Uint32Zero hashes x with seed 0. +func Uint32Zero(x uint32) uint32 { + h := prime5 + 4 + x*prime3 + h = rol17(h) * prime4 + h ^= h >> 15 + h *= prime2 + h ^= h >> 13 + h *= prime3 + h ^= h >> 16 + return h +} + +func rol1(u uint32) uint32 { + return u<<1 | u>>31 +} + +func rol7(u uint32) uint32 { + return u<<7 | u>>25 +} + +func rol11(u uint32) uint32 { + return u<<11 | u>>21 +} + +func rol12(u uint32) uint32 { + return u<<12 | u>>20 +} + +func rol13(u uint32) uint32 { + return u<<13 | u>>19 +} + +func rol17(u uint32) uint32 { + return u<<17 | u>>15 +} + +func rol18(u uint32) uint32 { + return u<<18 | u>>14 +} diff --git a/vendor/github.com/pierrec/lz4/lz4.go b/vendor/github.com/pierrec/lz4/lz4.go new file mode 100644 index 000000000000..a3284bdf708b --- /dev/null +++ b/vendor/github.com/pierrec/lz4/lz4.go @@ -0,0 +1,116 @@ +// Package lz4 implements reading and writing lz4 compressed data (a frame), +// as specified in http://fastcompression.blogspot.fr/2013/04/lz4-streaming-format-final.html. +// +// Although the block level compression and decompression functions are exposed and are fully compatible +// with the lz4 block format definition, they are low level and should not be used directly. +// For a complete description of an lz4 compressed block, see: +// http://fastcompression.blogspot.fr/2011/05/lz4-explained.html +// +// See https://github.com/Cyan4973/lz4 for the reference C implementation. +// +package lz4 + +import ( + "math/bits" + "sync" +) + +const ( + // Extension is the LZ4 frame file name extension + Extension = ".lz4" + // Version is the LZ4 frame format version + Version = 1 + + frameMagic uint32 = 0x184D2204 + frameSkipMagic uint32 = 0x184D2A50 + frameMagicLegacy uint32 = 0x184C2102 + + // The following constants are used to setup the compression algorithm. + minMatch = 4 // the minimum size of the match sequence size (4 bytes) + winSizeLog = 16 // LZ4 64Kb window size limit + winSize = 1 << winSizeLog + winMask = winSize - 1 // 64Kb window of previous data for dependent blocks + compressedBlockFlag = 1 << 31 + compressedBlockMask = compressedBlockFlag - 1 + + // hashLog determines the size of the hash table used to quickly find a previous match position. + // Its value influences the compression speed and memory usage, the lower the faster, + // but at the expense of the compression ratio. + // 16 seems to be the best compromise for fast compression. + hashLog = 16 + htSize = 1 << hashLog + + mfLimit = 10 + minMatch // The last match cannot start within the last 14 bytes. +) + +// map the block max size id with its value in bytes: 64Kb, 256Kb, 1Mb and 4Mb. +const ( + blockSize64K = 1 << (16 + 2*iota) + blockSize256K + blockSize1M + blockSize4M +) + +var ( + // Keep a pool of buffers for each valid block sizes. + bsMapValue = [...]*sync.Pool{ + newBufferPool(2 * blockSize64K), + newBufferPool(2 * blockSize256K), + newBufferPool(2 * blockSize1M), + newBufferPool(2 * blockSize4M), + } +) + +// newBufferPool returns a pool for buffers of the given size. +func newBufferPool(size int) *sync.Pool { + return &sync.Pool{ + New: func() interface{} { + return make([]byte, size) + }, + } +} + +// getBuffer returns a buffer to its pool. +func getBuffer(size int) []byte { + idx := blockSizeValueToIndex(size) - 4 + return bsMapValue[idx].Get().([]byte) +} + +// putBuffer returns a buffer to its pool. +func putBuffer(size int, buf []byte) { + if cap(buf) > 0 { + idx := blockSizeValueToIndex(size) - 4 + bsMapValue[idx].Put(buf[:cap(buf)]) + } +} +func blockSizeIndexToValue(i byte) int { + return 1 << (16 + 2*uint(i)) +} +func isValidBlockSize(size int) bool { + const blockSizeMask = blockSize64K | blockSize256K | blockSize1M | blockSize4M + + return size&blockSizeMask > 0 && bits.OnesCount(uint(size)) == 1 +} +func blockSizeValueToIndex(size int) byte { + return 4 + byte(bits.TrailingZeros(uint(size)>>16)/2) +} + +// Header describes the various flags that can be set on a Writer or obtained from a Reader. +// The default values match those of the LZ4 frame format definition +// (http://fastcompression.blogspot.com/2013/04/lz4-streaming-format-final.html). +// +// NB. in a Reader, in case of concatenated frames, the Header values may change between Read() calls. +// It is the caller's responsibility to check them if necessary. +type Header struct { + BlockChecksum bool // Compressed blocks checksum flag. + NoChecksum bool // Frame checksum flag. + BlockMaxSize int // Size of the uncompressed data block (one of [64KB, 256KB, 1MB, 4MB]). Default=4MB. + Size uint64 // Frame total size. It is _not_ computed by the Writer. + CompressionLevel int // Compression level (higher is better, use 0 for fastest compression). + done bool // Header processed flag (Read or Write and checked). +} + +// Reset reset internal status +func (h *Header) Reset() { + h.done = false +} diff --git a/vendor/github.com/pierrec/lz4/lz4_go1.10.go b/vendor/github.com/pierrec/lz4/lz4_go1.10.go new file mode 100644 index 000000000000..9a0fb00709d5 --- /dev/null +++ b/vendor/github.com/pierrec/lz4/lz4_go1.10.go @@ -0,0 +1,29 @@ +//+build go1.10 + +package lz4 + +import ( + "fmt" + "strings" +) + +func (h Header) String() string { + var s strings.Builder + + s.WriteString(fmt.Sprintf("%T{", h)) + if h.BlockChecksum { + s.WriteString("BlockChecksum: true ") + } + if h.NoChecksum { + s.WriteString("NoChecksum: true ") + } + if bs := h.BlockMaxSize; bs != 0 && bs != 4<<20 { + s.WriteString(fmt.Sprintf("BlockMaxSize: %d ", bs)) + } + if l := h.CompressionLevel; l != 0 { + s.WriteString(fmt.Sprintf("CompressionLevel: %d ", l)) + } + s.WriteByte('}') + + return s.String() +} diff --git a/vendor/github.com/pierrec/lz4/lz4_notgo1.10.go b/vendor/github.com/pierrec/lz4/lz4_notgo1.10.go new file mode 100644 index 000000000000..12c761a2e7f9 --- /dev/null +++ b/vendor/github.com/pierrec/lz4/lz4_notgo1.10.go @@ -0,0 +1,29 @@ +//+build !go1.10 + +package lz4 + +import ( + "bytes" + "fmt" +) + +func (h Header) String() string { + var s bytes.Buffer + + s.WriteString(fmt.Sprintf("%T{", h)) + if h.BlockChecksum { + s.WriteString("BlockChecksum: true ") + } + if h.NoChecksum { + s.WriteString("NoChecksum: true ") + } + if bs := h.BlockMaxSize; bs != 0 && bs != 4<<20 { + s.WriteString(fmt.Sprintf("BlockMaxSize: %d ", bs)) + } + if l := h.CompressionLevel; l != 0 { + s.WriteString(fmt.Sprintf("CompressionLevel: %d ", l)) + } + s.WriteByte('}') + + return s.String() +} diff --git a/vendor/github.com/pierrec/lz4/reader.go b/vendor/github.com/pierrec/lz4/reader.go new file mode 100644 index 000000000000..87dd72bd0db3 --- /dev/null +++ b/vendor/github.com/pierrec/lz4/reader.go @@ -0,0 +1,335 @@ +package lz4 + +import ( + "encoding/binary" + "fmt" + "io" + "io/ioutil" + + "github.com/pierrec/lz4/internal/xxh32" +) + +// Reader implements the LZ4 frame decoder. +// The Header is set after the first call to Read(). +// The Header may change between Read() calls in case of concatenated frames. +type Reader struct { + Header + // Handler called when a block has been successfully read. + // It provides the number of bytes read. + OnBlockDone func(size int) + + buf [8]byte // Scrap buffer. + pos int64 // Current position in src. + src io.Reader // Source. + zdata []byte // Compressed data. + data []byte // Uncompressed data. + idx int // Index of unread bytes into data. + checksum xxh32.XXHZero // Frame hash. + skip int64 // Bytes to skip before next read. + dpos int64 // Position in dest +} + +// NewReader returns a new LZ4 frame decoder. +// No access to the underlying io.Reader is performed. +func NewReader(src io.Reader) *Reader { + r := &Reader{src: src} + return r +} + +// readHeader checks the frame magic number and parses the frame descriptoz. +// Skippable frames are supported even as a first frame although the LZ4 +// specifications recommends skippable frames not to be used as first frames. +func (z *Reader) readHeader(first bool) error { + defer z.checksum.Reset() + + buf := z.buf[:] + for { + magic, err := z.readUint32() + if err != nil { + z.pos += 4 + if !first && err == io.ErrUnexpectedEOF { + return io.EOF + } + return err + } + if magic == frameMagic { + break + } + if magic>>8 != frameSkipMagic>>8 { + return ErrInvalid + } + skipSize, err := z.readUint32() + if err != nil { + return err + } + z.pos += 4 + m, err := io.CopyN(ioutil.Discard, z.src, int64(skipSize)) + if err != nil { + return err + } + z.pos += m + } + + // Header. + if _, err := io.ReadFull(z.src, buf[:2]); err != nil { + return err + } + z.pos += 8 + + b := buf[0] + if v := b >> 6; v != Version { + return fmt.Errorf("lz4: invalid version: got %d; expected %d", v, Version) + } + if b>>5&1 == 0 { + return ErrBlockDependency + } + z.BlockChecksum = b>>4&1 > 0 + frameSize := b>>3&1 > 0 + z.NoChecksum = b>>2&1 == 0 + + bmsID := buf[1] >> 4 & 0x7 + if bmsID < 4 || bmsID > 7 { + return fmt.Errorf("lz4: invalid block max size ID: %d", bmsID) + } + bSize := blockSizeIndexToValue(bmsID - 4) + z.BlockMaxSize = bSize + + // Allocate the compressed/uncompressed buffers. + // The compressed buffer cannot exceed the uncompressed one. + if n := 2 * bSize; cap(z.zdata) < n { + z.zdata = make([]byte, n, n) + } + if debugFlag { + debug("header block max size id=%d size=%d", bmsID, bSize) + } + z.zdata = z.zdata[:bSize] + z.data = z.zdata[:cap(z.zdata)][bSize:] + z.idx = len(z.data) + + _, _ = z.checksum.Write(buf[0:2]) + + if frameSize { + buf := buf[:8] + if _, err := io.ReadFull(z.src, buf); err != nil { + return err + } + z.Size = binary.LittleEndian.Uint64(buf) + z.pos += 8 + _, _ = z.checksum.Write(buf) + } + + // Header checksum. + if _, err := io.ReadFull(z.src, buf[:1]); err != nil { + return err + } + z.pos++ + if h := byte(z.checksum.Sum32() >> 8 & 0xFF); h != buf[0] { + return fmt.Errorf("lz4: invalid header checksum: got %x; expected %x", buf[0], h) + } + + z.Header.done = true + if debugFlag { + debug("header read: %v", z.Header) + } + + return nil +} + +// Read decompresses data from the underlying source into the supplied buffer. +// +// Since there can be multiple streams concatenated, Header values may +// change between calls to Read(). If that is the case, no data is actually read from +// the underlying io.Reader, to allow for potential input buffer resizing. +func (z *Reader) Read(buf []byte) (int, error) { + if debugFlag { + debug("Read buf len=%d", len(buf)) + } + if !z.Header.done { + if err := z.readHeader(true); err != nil { + return 0, err + } + if debugFlag { + debug("header read OK compressed buffer %d / %d uncompressed buffer %d : %d index=%d", + len(z.zdata), cap(z.zdata), len(z.data), cap(z.data), z.idx) + } + } + + if len(buf) == 0 { + return 0, nil + } + + if z.idx == len(z.data) { + // No data ready for reading, process the next block. + if debugFlag { + debug("reading block from writer") + } + // Reset uncompressed buffer + z.data = z.zdata[:cap(z.zdata)][len(z.zdata):] + + // Block length: 0 = end of frame, highest bit set: uncompressed. + bLen, err := z.readUint32() + if err != nil { + return 0, err + } + z.pos += 4 + + if bLen == 0 { + // End of frame reached. + if !z.NoChecksum { + // Validate the frame checksum. + checksum, err := z.readUint32() + if err != nil { + return 0, err + } + if debugFlag { + debug("frame checksum got=%x / want=%x", z.checksum.Sum32(), checksum) + } + z.pos += 4 + if h := z.checksum.Sum32(); checksum != h { + return 0, fmt.Errorf("lz4: invalid frame checksum: got %x; expected %x", h, checksum) + } + } + + // Get ready for the next concatenated frame and keep the position. + pos := z.pos + z.Reset(z.src) + z.pos = pos + + // Since multiple frames can be concatenated, check for more. + return 0, z.readHeader(false) + } + + if debugFlag { + debug("raw block size %d", bLen) + } + if bLen&compressedBlockFlag > 0 { + // Uncompressed block. + bLen &= compressedBlockMask + if debugFlag { + debug("uncompressed block size %d", bLen) + } + if int(bLen) > cap(z.data) { + return 0, fmt.Errorf("lz4: invalid block size: %d", bLen) + } + z.data = z.data[:bLen] + if _, err := io.ReadFull(z.src, z.data); err != nil { + return 0, err + } + z.pos += int64(bLen) + if z.OnBlockDone != nil { + z.OnBlockDone(int(bLen)) + } + + if z.BlockChecksum { + checksum, err := z.readUint32() + if err != nil { + return 0, err + } + z.pos += 4 + + if h := xxh32.ChecksumZero(z.data); h != checksum { + return 0, fmt.Errorf("lz4: invalid block checksum: got %x; expected %x", h, checksum) + } + } + + } else { + // Compressed block. + if debugFlag { + debug("compressed block size %d", bLen) + } + if int(bLen) > cap(z.data) { + return 0, fmt.Errorf("lz4: invalid block size: %d", bLen) + } + zdata := z.zdata[:bLen] + if _, err := io.ReadFull(z.src, zdata); err != nil { + return 0, err + } + z.pos += int64(bLen) + + if z.BlockChecksum { + checksum, err := z.readUint32() + if err != nil { + return 0, err + } + z.pos += 4 + + if h := xxh32.ChecksumZero(zdata); h != checksum { + return 0, fmt.Errorf("lz4: invalid block checksum: got %x; expected %x", h, checksum) + } + } + + n, err := UncompressBlock(zdata, z.data) + if err != nil { + return 0, err + } + z.data = z.data[:n] + if z.OnBlockDone != nil { + z.OnBlockDone(n) + } + } + + if !z.NoChecksum { + _, _ = z.checksum.Write(z.data) + if debugFlag { + debug("current frame checksum %x", z.checksum.Sum32()) + } + } + z.idx = 0 + } + + if z.skip > int64(len(z.data[z.idx:])) { + z.skip -= int64(len(z.data[z.idx:])) + z.dpos += int64(len(z.data[z.idx:])) + z.idx = len(z.data) + return 0, nil + } + + z.idx += int(z.skip) + z.dpos += z.skip + z.skip = 0 + + n := copy(buf, z.data[z.idx:]) + z.idx += n + z.dpos += int64(n) + if debugFlag { + debug("copied %d bytes to input", n) + } + + return n, nil +} + +// Seek implements io.Seeker, but supports seeking forward from the current +// position only. Any other seek will return an error. Allows skipping output +// bytes which aren't needed, which in some scenarios is faster than reading +// and discarding them. +// Note this may cause future calls to Read() to read 0 bytes if all of the +// data they would have returned is skipped. +func (z *Reader) Seek(offset int64, whence int) (int64, error) { + if offset < 0 || whence != io.SeekCurrent { + return z.dpos + z.skip, ErrUnsupportedSeek + } + z.skip += offset + return z.dpos + z.skip, nil +} + +// Reset discards the Reader's state and makes it equivalent to the +// result of its original state from NewReader, but reading from r instead. +// This permits reusing a Reader rather than allocating a new one. +func (z *Reader) Reset(r io.Reader) { + z.Header = Header{} + z.pos = 0 + z.src = r + z.zdata = z.zdata[:0] + z.data = z.data[:0] + z.idx = 0 + z.checksum.Reset() +} + +// readUint32 reads an uint32 into the supplied buffer. +// The idea is to make use of the already allocated buffers avoiding additional allocations. +func (z *Reader) readUint32() (uint32, error) { + buf := z.buf[:4] + _, err := io.ReadFull(z.src, buf) + x := binary.LittleEndian.Uint32(buf) + return x, err +} diff --git a/vendor/github.com/pierrec/lz4/reader_legacy.go b/vendor/github.com/pierrec/lz4/reader_legacy.go new file mode 100644 index 000000000000..1670a77d02a4 --- /dev/null +++ b/vendor/github.com/pierrec/lz4/reader_legacy.go @@ -0,0 +1,207 @@ +package lz4 + +import ( + "encoding/binary" + "fmt" + "io" +) + +// ReaderLegacy implements the LZ4Demo frame decoder. +// The Header is set after the first call to Read(). +type ReaderLegacy struct { + Header + // Handler called when a block has been successfully read. + // It provides the number of bytes read. + OnBlockDone func(size int) + + lastBlock bool + buf [8]byte // Scrap buffer. + pos int64 // Current position in src. + src io.Reader // Source. + zdata []byte // Compressed data. + data []byte // Uncompressed data. + idx int // Index of unread bytes into data. + skip int64 // Bytes to skip before next read. + dpos int64 // Position in dest +} + +// NewReaderLegacy returns a new LZ4Demo frame decoder. +// No access to the underlying io.Reader is performed. +func NewReaderLegacy(src io.Reader) *ReaderLegacy { + r := &ReaderLegacy{src: src} + return r +} + +// readHeader checks the frame magic number and parses the frame descriptoz. +// Skippable frames are supported even as a first frame although the LZ4 +// specifications recommends skippable frames not to be used as first frames. +func (z *ReaderLegacy) readLegacyHeader() error { + z.lastBlock = false + magic, err := z.readUint32() + if err != nil { + z.pos += 4 + if err == io.ErrUnexpectedEOF { + return io.EOF + } + return err + } + if magic != frameMagicLegacy { + return ErrInvalid + } + z.pos += 4 + + // Legacy has fixed 8MB blocksizes + // https://github.com/lz4/lz4/blob/dev/doc/lz4_Frame_format.md#legacy-frame + bSize := blockSize4M * 2 + + // Allocate the compressed/uncompressed buffers. + // The compressed buffer cannot exceed the uncompressed one. + if n := 2 * bSize; cap(z.zdata) < n { + z.zdata = make([]byte, n, n) + } + if debugFlag { + debug("header block max size size=%d", bSize) + } + z.zdata = z.zdata[:bSize] + z.data = z.zdata[:cap(z.zdata)][bSize:] + z.idx = len(z.data) + + z.Header.done = true + if debugFlag { + debug("header read: %v", z.Header) + } + + return nil +} + +// Read decompresses data from the underlying source into the supplied buffer. +// +// Since there can be multiple streams concatenated, Header values may +// change between calls to Read(). If that is the case, no data is actually read from +// the underlying io.Reader, to allow for potential input buffer resizing. +func (z *ReaderLegacy) Read(buf []byte) (int, error) { + if debugFlag { + debug("Read buf len=%d", len(buf)) + } + if !z.Header.done { + if err := z.readLegacyHeader(); err != nil { + return 0, err + } + if debugFlag { + debug("header read OK compressed buffer %d / %d uncompressed buffer %d : %d index=%d", + len(z.zdata), cap(z.zdata), len(z.data), cap(z.data), z.idx) + } + } + + if len(buf) == 0 { + return 0, nil + } + + if z.idx == len(z.data) { + // No data ready for reading, process the next block. + if debugFlag { + debug(" reading block from writer %d %d", z.idx, blockSize4M*2) + } + + // Reset uncompressed buffer + z.data = z.zdata[:cap(z.zdata)][len(z.zdata):] + + bLen, err := z.readUint32() + if err != nil { + return 0, err + } + if debugFlag { + debug(" bLen %d (0x%x) offset = %d (0x%x)", bLen, bLen, z.pos, z.pos) + } + z.pos += 4 + + // Legacy blocks are always compressed, even when detrimental + if debugFlag { + debug(" compressed block size %d", bLen) + } + + if int(bLen) > cap(z.data) { + return 0, fmt.Errorf("lz4: invalid block size: %d", bLen) + } + zdata := z.zdata[:bLen] + if _, err := io.ReadFull(z.src, zdata); err != nil { + return 0, err + } + z.pos += int64(bLen) + + n, err := UncompressBlock(zdata, z.data) + if err != nil { + return 0, err + } + + z.data = z.data[:n] + if z.OnBlockDone != nil { + z.OnBlockDone(n) + } + + z.idx = 0 + + // Legacy blocks are fixed to 8MB, if we read a decompressed block smaller than this + // it means we've reached the end... + if n < blockSize4M*2 { + z.lastBlock = true + } + } + + if z.skip > int64(len(z.data[z.idx:])) { + z.skip -= int64(len(z.data[z.idx:])) + z.dpos += int64(len(z.data[z.idx:])) + z.idx = len(z.data) + return 0, nil + } + + z.idx += int(z.skip) + z.dpos += z.skip + z.skip = 0 + + n := copy(buf, z.data[z.idx:]) + z.idx += n + z.dpos += int64(n) + if debugFlag { + debug("%v] copied %d bytes to input (%d:%d)", z.lastBlock, n, z.idx, len(z.data)) + } + if z.lastBlock && len(z.data) == z.idx { + return n, io.EOF + } + return n, nil +} + +// Seek implements io.Seeker, but supports seeking forward from the current +// position only. Any other seek will return an error. Allows skipping output +// bytes which aren't needed, which in some scenarios is faster than reading +// and discarding them. +// Note this may cause future calls to Read() to read 0 bytes if all of the +// data they would have returned is skipped. +func (z *ReaderLegacy) Seek(offset int64, whence int) (int64, error) { + if offset < 0 || whence != io.SeekCurrent { + return z.dpos + z.skip, ErrUnsupportedSeek + } + z.skip += offset + return z.dpos + z.skip, nil +} + +// Reset discards the Reader's state and makes it equivalent to the +// result of its original state from NewReader, but reading from r instead. +// This permits reusing a Reader rather than allocating a new one. +func (z *ReaderLegacy) Reset(r io.Reader) { + z.Header = Header{} + z.pos = 0 + z.src = r + z.zdata = z.zdata[:0] + z.data = z.data[:0] + z.idx = 0 +} + +// readUint32 reads an uint32 into the supplied buffer. +// The idea is to make use of the already allocated buffers avoiding additional allocations. +func (z *ReaderLegacy) readUint32() (uint32, error) { + buf := z.buf[:4] + _, err := io.ReadFull(z.src, buf) + x := binary.LittleEndian.Uint32(buf) + return x, err +} diff --git a/vendor/github.com/pierrec/lz4/writer.go b/vendor/github.com/pierrec/lz4/writer.go new file mode 100644 index 000000000000..f066d56305e6 --- /dev/null +++ b/vendor/github.com/pierrec/lz4/writer.go @@ -0,0 +1,422 @@ +package lz4 + +import ( + "encoding/binary" + "fmt" + "io" + "runtime" + + "github.com/pierrec/lz4/internal/xxh32" +) + +// zResult contains the results of compressing a block. +type zResult struct { + size uint32 // Block header + data []byte // Compressed data + checksum uint32 // Data checksum +} + +// Writer implements the LZ4 frame encoder. +type Writer struct { + Header + // Handler called when a block has been successfully written out. + // It provides the number of bytes written. + OnBlockDone func(size int) + + buf [19]byte // magic number(4) + header(flags(2)+[Size(8)+DictID(4)]+checksum(1)) does not exceed 19 bytes + dst io.Writer // Destination. + checksum xxh32.XXHZero // Frame checksum. + data []byte // Data to be compressed + buffer for compressed data. + idx int // Index into data. + hashtable [winSize]int // Hash table used in CompressBlock(). + + // For concurrency. + c chan chan zResult // Channel for block compression goroutines and writer goroutine. + err error // Any error encountered while writing to the underlying destination. +} + +// NewWriter returns a new LZ4 frame encoder. +// No access to the underlying io.Writer is performed. +// The supplied Header is checked at the first Write. +// It is ok to change it before the first Write but then not until a Reset() is performed. +func NewWriter(dst io.Writer) *Writer { + z := new(Writer) + z.Reset(dst) + return z +} + +// WithConcurrency sets the number of concurrent go routines used for compression. +// A negative value sets the concurrency to GOMAXPROCS. +func (z *Writer) WithConcurrency(n int) *Writer { + switch { + case n == 0 || n == 1: + z.c = nil + return z + case n < 0: + n = runtime.GOMAXPROCS(0) + } + z.c = make(chan chan zResult, n) + // Writer goroutine managing concurrent block compression goroutines. + go func() { + // Process next block compression item. + for c := range z.c { + // Read the next compressed block result. + // Waiting here ensures that the blocks are output in the order they were sent. + // The incoming channel is always closed as it indicates to the caller that + // the block has been processed. + res := <-c + n := len(res.data) + if n == 0 { + // Notify the block compression routine that we are done with its result. + // This is used when a sentinel block is sent to terminate the compression. + close(c) + return + } + // Write the block. + if err := z.writeUint32(res.size); err != nil && z.err == nil { + z.err = err + } + if _, err := z.dst.Write(res.data); err != nil && z.err == nil { + z.err = err + } + if z.BlockChecksum { + if err := z.writeUint32(res.checksum); err != nil && z.err == nil { + z.err = err + } + } + // It is now safe to release the buffer as no longer in use by any goroutine. + putBuffer(cap(res.data), res.data) + if h := z.OnBlockDone; h != nil { + h(n) + } + close(c) + } + }() + return z +} + +// newBuffers instantiates new buffers which size matches the one in Header. +// The returned buffers are for decompression and compression respectively. +func (z *Writer) newBuffers() { + bSize := z.Header.BlockMaxSize + buf := getBuffer(bSize) + z.data = buf[:bSize] // Uncompressed buffer is the first half. +} + +// freeBuffers puts the writer's buffers back to the pool. +func (z *Writer) freeBuffers() { + // Put the buffer back into the pool, if any. + putBuffer(z.Header.BlockMaxSize, z.data) + z.data = nil +} + +// writeHeader builds and writes the header (magic+header) to the underlying io.Writer. +func (z *Writer) writeHeader() error { + // Default to 4Mb if BlockMaxSize is not set. + if z.Header.BlockMaxSize == 0 { + z.Header.BlockMaxSize = blockSize4M + } + // The only option that needs to be validated. + bSize := z.Header.BlockMaxSize + if !isValidBlockSize(z.Header.BlockMaxSize) { + return fmt.Errorf("lz4: invalid block max size: %d", bSize) + } + // Allocate the compressed/uncompressed buffers. + // The compressed buffer cannot exceed the uncompressed one. + z.newBuffers() + z.idx = 0 + + // Size is optional. + buf := z.buf[:] + + // Set the fixed size data: magic number, block max size and flags. + binary.LittleEndian.PutUint32(buf[0:], frameMagic) + flg := byte(Version << 6) + flg |= 1 << 5 // No block dependency. + if z.Header.BlockChecksum { + flg |= 1 << 4 + } + if z.Header.Size > 0 { + flg |= 1 << 3 + } + if !z.Header.NoChecksum { + flg |= 1 << 2 + } + buf[4] = flg + buf[5] = blockSizeValueToIndex(z.Header.BlockMaxSize) << 4 + + // Current buffer size: magic(4) + flags(1) + block max size (1). + n := 6 + // Optional items. + if z.Header.Size > 0 { + binary.LittleEndian.PutUint64(buf[n:], z.Header.Size) + n += 8 + } + + // The header checksum includes the flags, block max size and optional Size. + buf[n] = byte(xxh32.ChecksumZero(buf[4:n]) >> 8 & 0xFF) + z.checksum.Reset() + + // Header ready, write it out. + if _, err := z.dst.Write(buf[0 : n+1]); err != nil { + return err + } + z.Header.done = true + if debugFlag { + debug("wrote header %v", z.Header) + } + + return nil +} + +// Write compresses data from the supplied buffer into the underlying io.Writer. +// Write does not return until the data has been written. +func (z *Writer) Write(buf []byte) (int, error) { + if !z.Header.done { + if err := z.writeHeader(); err != nil { + return 0, err + } + } + if debugFlag { + debug("input buffer len=%d index=%d", len(buf), z.idx) + } + + zn := len(z.data) + var n int + for len(buf) > 0 { + if z.idx == 0 && len(buf) >= zn { + // Avoid a copy as there is enough data for a block. + if err := z.compressBlock(buf[:zn]); err != nil { + return n, err + } + n += zn + buf = buf[zn:] + continue + } + // Accumulate the data to be compressed. + m := copy(z.data[z.idx:], buf) + n += m + z.idx += m + buf = buf[m:] + if debugFlag { + debug("%d bytes copied to buf, current index %d", n, z.idx) + } + + if z.idx < len(z.data) { + // Buffer not filled. + if debugFlag { + debug("need more data for compression") + } + return n, nil + } + + // Buffer full. + if err := z.compressBlock(z.data); err != nil { + return n, err + } + z.idx = 0 + } + + return n, nil +} + +// compressBlock compresses a block. +func (z *Writer) compressBlock(data []byte) error { + if !z.NoChecksum { + _, _ = z.checksum.Write(data) + } + + if z.c != nil { + c := make(chan zResult) + z.c <- c // Send now to guarantee order + + // get a buffer from the pool and copy the data over + block := getBuffer(z.Header.BlockMaxSize)[:len(data)] + copy(block, data) + + go writerCompressBlock(c, z.Header, block) + return nil + } + + zdata := z.data[z.Header.BlockMaxSize:cap(z.data)] + // The compressed block size cannot exceed the input's. + var zn int + + if level := z.Header.CompressionLevel; level != 0 { + zn, _ = CompressBlockHC(data, zdata, level) + } else { + zn, _ = CompressBlock(data, zdata, z.hashtable[:]) + } + + var bLen uint32 + if debugFlag { + debug("block compression %d => %d", len(data), zn) + } + if zn > 0 && zn < len(data) { + // Compressible and compressed size smaller than uncompressed: ok! + bLen = uint32(zn) + zdata = zdata[:zn] + } else { + // Uncompressed block. + bLen = uint32(len(data)) | compressedBlockFlag + zdata = data + } + if debugFlag { + debug("block compression to be written len=%d data len=%d", bLen, len(zdata)) + } + + // Write the block. + if err := z.writeUint32(bLen); err != nil { + return err + } + written, err := z.dst.Write(zdata) + if err != nil { + return err + } + if h := z.OnBlockDone; h != nil { + h(written) + } + + if !z.BlockChecksum { + if debugFlag { + debug("current frame checksum %x", z.checksum.Sum32()) + } + return nil + } + checksum := xxh32.ChecksumZero(zdata) + if debugFlag { + debug("block checksum %x", checksum) + defer func() { debug("current frame checksum %x", z.checksum.Sum32()) }() + } + return z.writeUint32(checksum) +} + +// Flush flushes any pending compressed data to the underlying writer. +// Flush does not return until the data has been written. +// If the underlying writer returns an error, Flush returns that error. +func (z *Writer) Flush() error { + if debugFlag { + debug("flush with index %d", z.idx) + } + if z.idx == 0 { + return nil + } + + data := getBuffer(z.Header.BlockMaxSize)[:len(z.data[:z.idx])] + copy(data, z.data[:z.idx]) + + z.idx = 0 + if z.c == nil { + return z.compressBlock(data) + } + if !z.NoChecksum { + _, _ = z.checksum.Write(data) + } + c := make(chan zResult) + z.c <- c + writerCompressBlock(c, z.Header, data) + return nil +} + +func (z *Writer) close() error { + if z.c == nil { + return nil + } + // Send a sentinel block (no data to compress) to terminate the writer main goroutine. + c := make(chan zResult) + z.c <- c + c <- zResult{} + // Wait for the main goroutine to complete. + <-c + // At this point the main goroutine has shut down or is about to return. + z.c = nil + return z.err +} + +// Close closes the Writer, flushing any unwritten data to the underlying io.Writer, but does not close the underlying io.Writer. +func (z *Writer) Close() error { + if !z.Header.done { + if err := z.writeHeader(); err != nil { + return err + } + } + if err := z.Flush(); err != nil { + return err + } + if err := z.close(); err != nil { + return err + } + z.freeBuffers() + + if debugFlag { + debug("writing last empty block") + } + if err := z.writeUint32(0); err != nil { + return err + } + if z.NoChecksum { + return nil + } + checksum := z.checksum.Sum32() + if debugFlag { + debug("stream checksum %x", checksum) + } + return z.writeUint32(checksum) +} + +// Reset clears the state of the Writer z such that it is equivalent to its +// initial state from NewWriter, but instead writing to w. +// No access to the underlying io.Writer is performed. +func (z *Writer) Reset(w io.Writer) { + n := cap(z.c) + _ = z.close() + z.freeBuffers() + z.Header.Reset() + z.dst = w + z.checksum.Reset() + z.idx = 0 + z.err = nil + // reset hashtable to ensure deterministic output. + for i := range z.hashtable { + z.hashtable[i] = 0 + } + z.WithConcurrency(n) +} + +// writeUint32 writes a uint32 to the underlying writer. +func (z *Writer) writeUint32(x uint32) error { + buf := z.buf[:4] + binary.LittleEndian.PutUint32(buf, x) + _, err := z.dst.Write(buf) + return err +} + +// writerCompressBlock compresses data into a pooled buffer and writes its result +// out to the input channel. +func writerCompressBlock(c chan zResult, header Header, data []byte) { + zdata := getBuffer(header.BlockMaxSize) + // The compressed block size cannot exceed the input's. + var zn int + if level := header.CompressionLevel; level != 0 { + zn, _ = CompressBlockHC(data, zdata, level) + } else { + var hashTable [winSize]int + zn, _ = CompressBlock(data, zdata, hashTable[:]) + } + var res zResult + if zn > 0 && zn < len(data) { + res.size = uint32(zn) + res.data = zdata[:zn] + // release the uncompressed block since it is not used anymore + putBuffer(header.BlockMaxSize, data) + } else { + res.size = uint32(len(data)) | compressedBlockFlag + res.data = data + // release the compressed block since it was not used + putBuffer(header.BlockMaxSize, zdata) + } + if header.BlockChecksum { + res.checksum = xxh32.ChecksumZero(res.data) + } + c <- res +} diff --git a/vendor/github.com/pierrec/lz4/writer_legacy.go b/vendor/github.com/pierrec/lz4/writer_legacy.go new file mode 100644 index 000000000000..ca8dc8c7f0c6 --- /dev/null +++ b/vendor/github.com/pierrec/lz4/writer_legacy.go @@ -0,0 +1,182 @@ +package lz4 + +import ( + "encoding/binary" + "io" +) + +// WriterLegacy implements the LZ4Demo frame decoder. +type WriterLegacy struct { + Header + // Handler called when a block has been successfully read. + // It provides the number of bytes read. + OnBlockDone func(size int) + + dst io.Writer // Destination. + data []byte // Data to be compressed + buffer for compressed data. + idx int // Index into data. + hashtable [winSize]int // Hash table used in CompressBlock(). +} + +// NewWriterLegacy returns a new LZ4 encoder for the legacy frame format. +// No access to the underlying io.Writer is performed. +// The supplied Header is checked at the first Write. +// It is ok to change it before the first Write but then not until a Reset() is performed. +func NewWriterLegacy(dst io.Writer) *WriterLegacy { + z := new(WriterLegacy) + z.Reset(dst) + return z +} + +// Write compresses data from the supplied buffer into the underlying io.Writer. +// Write does not return until the data has been written. +func (z *WriterLegacy) Write(buf []byte) (int, error) { + if !z.Header.done { + if err := z.writeHeader(); err != nil { + return 0, err + } + } + if debugFlag { + debug("input buffer len=%d index=%d", len(buf), z.idx) + } + + zn := len(z.data) + var n int + for len(buf) > 0 { + if z.idx == 0 && len(buf) >= zn { + // Avoid a copy as there is enough data for a block. + if err := z.compressBlock(buf[:zn]); err != nil { + return n, err + } + n += zn + buf = buf[zn:] + continue + } + // Accumulate the data to be compressed. + m := copy(z.data[z.idx:], buf) + n += m + z.idx += m + buf = buf[m:] + if debugFlag { + debug("%d bytes copied to buf, current index %d", n, z.idx) + } + + if z.idx < len(z.data) { + // Buffer not filled. + if debugFlag { + debug("need more data for compression") + } + return n, nil + } + + // Buffer full. + if err := z.compressBlock(z.data); err != nil { + return n, err + } + z.idx = 0 + } + + return n, nil +} + +// writeHeader builds and writes the header to the underlying io.Writer. +func (z *WriterLegacy) writeHeader() error { + // Legacy has fixed 8MB blocksizes + // https://github.com/lz4/lz4/blob/dev/doc/lz4_Frame_format.md#legacy-frame + bSize := 2 * blockSize4M + + buf := make([]byte, 2*bSize, 2*bSize) + z.data = buf[:bSize] // Uncompressed buffer is the first half. + + z.idx = 0 + + // Header consists of one mageic number, write it out. + if err := binary.Write(z.dst, binary.LittleEndian, frameMagicLegacy); err != nil { + return err + } + z.Header.done = true + if debugFlag { + debug("wrote header %v", z.Header) + } + + return nil +} + +// compressBlock compresses a block. +func (z *WriterLegacy) compressBlock(data []byte) error { + bSize := 2 * blockSize4M + zdata := z.data[bSize:cap(z.data)] + // The compressed block size cannot exceed the input's. + var zn int + + if level := z.Header.CompressionLevel; level != 0 { + zn, _ = CompressBlockHC(data, zdata, level) + } else { + zn, _ = CompressBlock(data, zdata, z.hashtable[:]) + } + + if debugFlag { + debug("block compression %d => %d", len(data), zn) + } + zdata = zdata[:zn] + + // Write the block. + if err := binary.Write(z.dst, binary.LittleEndian, uint32(zn)); err != nil { + return err + } + written, err := z.dst.Write(zdata) + if err != nil { + return err + } + if h := z.OnBlockDone; h != nil { + h(written) + } + return nil +} + +// Flush flushes any pending compressed data to the underlying writer. +// Flush does not return until the data has been written. +// If the underlying writer returns an error, Flush returns that error. +func (z *WriterLegacy) Flush() error { + if debugFlag { + debug("flush with index %d", z.idx) + } + if z.idx == 0 { + return nil + } + + data := z.data[:z.idx] + z.idx = 0 + return z.compressBlock(data) +} + +// Close closes the WriterLegacy, flushing any unwritten data to the underlying io.Writer, but does not close the underlying io.Writer. +func (z *WriterLegacy) Close() error { + if !z.Header.done { + if err := z.writeHeader(); err != nil { + return err + } + } + if err := z.Flush(); err != nil { + return err + } + + if debugFlag { + debug("writing last empty block") + } + + return nil +} + +// Reset clears the state of the WriterLegacy z such that it is equivalent to its +// initial state from NewWriterLegacy, but instead writing to w. +// No access to the underlying io.Writer is performed. +func (z *WriterLegacy) Reset(w io.Writer) { + z.Header.Reset() + z.dst = w + z.idx = 0 + // reset hashtable to ensure deterministic output. + for i := range z.hashtable { + z.hashtable[i] = 0 + } +} diff --git a/vendor/github.com/rcrowley/go-metrics/.gitignore b/vendor/github.com/rcrowley/go-metrics/.gitignore new file mode 100644 index 000000000000..83c8f82374a2 --- /dev/null +++ b/vendor/github.com/rcrowley/go-metrics/.gitignore @@ -0,0 +1,9 @@ +*.[68] +*.a +*.out +*.swp +_obj +_testmain.go +cmd/metrics-bench/metrics-bench +cmd/metrics-example/metrics-example +cmd/never-read/never-read diff --git a/vendor/github.com/rcrowley/go-metrics/.travis.yml b/vendor/github.com/rcrowley/go-metrics/.travis.yml new file mode 100644 index 000000000000..ce9afeaeeac3 --- /dev/null +++ b/vendor/github.com/rcrowley/go-metrics/.travis.yml @@ -0,0 +1,23 @@ +language: go + +go: + - "1.3" + - "1.4" + - "1.5" + - "1.6" + - "1.7" + - "1.8" + - "1.9" + - "1.10" + - "1.11" + - "1.12" + - "1.13" + - "1.14" + - "1.15" + +script: + - ./validate.sh + +# this should give us faster builds according to +# http://docs.travis-ci.com/user/migrating-from-legacy/ +sudo: false diff --git a/vendor/github.com/rcrowley/go-metrics/LICENSE b/vendor/github.com/rcrowley/go-metrics/LICENSE new file mode 100644 index 000000000000..363fa9ee77b8 --- /dev/null +++ b/vendor/github.com/rcrowley/go-metrics/LICENSE @@ -0,0 +1,29 @@ +Copyright 2012 Richard Crowley. All rights reserved. + +Redistribution and use in source and binary forms, with or without +modification, are permitted provided that the following conditions are +met: + + 1. Redistributions of source code must retain the above copyright + notice, this list of conditions and the following disclaimer. + + 2. Redistributions in binary form must reproduce the above + copyright notice, this list of conditions and the following + disclaimer in the documentation and/or other materials provided + with the distribution. + +THIS SOFTWARE IS PROVIDED BY RICHARD CROWLEY ``AS IS'' AND ANY EXPRESS +OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE IMPLIED +WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE ARE +DISCLAIMED. IN NO EVENT SHALL RICHARD CROWLEY OR CONTRIBUTORS BE LIABLE +FOR ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR +CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT LIMITED TO, PROCUREMENT OF +SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, DATA, OR PROFITS; OR BUSINESS +INTERRUPTION) HOWEVER CAUSED AND ON ANY THEORY OF LIABILITY, WHETHER IN +CONTRACT, STRICT LIABILITY, OR TORT (INCLUDING NEGLIGENCE OR OTHERWISE) +ARISING IN ANY WAY OUT OF THE USE OF THIS SOFTWARE, EVEN IF ADVISED OF +THE POSSIBILITY OF SUCH DAMAGE. + +The views and conclusions contained in the software and documentation +are those of the authors and should not be interpreted as representing +official policies, either expressed or implied, of Richard Crowley. diff --git a/vendor/github.com/rcrowley/go-metrics/README.md b/vendor/github.com/rcrowley/go-metrics/README.md new file mode 100644 index 000000000000..27ddfee8b89f --- /dev/null +++ b/vendor/github.com/rcrowley/go-metrics/README.md @@ -0,0 +1,171 @@ +go-metrics +========== + +![travis build status](https://travis-ci.org/rcrowley/go-metrics.svg?branch=master) + +Go port of Coda Hale's Metrics library: . + +Documentation: . + +Usage +----- + +Create and update metrics: + +```go +c := metrics.NewCounter() +metrics.Register("foo", c) +c.Inc(47) + +g := metrics.NewGauge() +metrics.Register("bar", g) +g.Update(47) + +r := NewRegistry() +g := metrics.NewRegisteredFunctionalGauge("cache-evictions", r, func() int64 { return cache.getEvictionsCount() }) + +s := metrics.NewExpDecaySample(1028, 0.015) // or metrics.NewUniformSample(1028) +h := metrics.NewHistogram(s) +metrics.Register("baz", h) +h.Update(47) + +m := metrics.NewMeter() +metrics.Register("quux", m) +m.Mark(47) + +t := metrics.NewTimer() +metrics.Register("bang", t) +t.Time(func() {}) +t.Update(47) +``` + +Register() is not threadsafe. For threadsafe metric registration use +GetOrRegister: + +```go +t := metrics.GetOrRegisterTimer("account.create.latency", nil) +t.Time(func() {}) +t.Update(47) +``` + +**NOTE:** Be sure to unregister short-lived meters and timers otherwise they will +leak memory: + +```go +// Will call Stop() on the Meter to allow for garbage collection +metrics.Unregister("quux") +// Or similarly for a Timer that embeds a Meter +metrics.Unregister("bang") +``` + +Periodically log every metric in human-readable form to standard error: + +```go +go metrics.Log(metrics.DefaultRegistry, 5 * time.Second, log.New(os.Stderr, "metrics: ", log.Lmicroseconds)) +``` + +Periodically log every metric in slightly-more-parseable form to syslog: + +```go +w, _ := syslog.Dial("unixgram", "/dev/log", syslog.LOG_INFO, "metrics") +go metrics.Syslog(metrics.DefaultRegistry, 60e9, w) +``` + +Periodically emit every metric to Graphite using the [Graphite client](https://github.com/cyberdelia/go-metrics-graphite): + +```go + +import "github.com/cyberdelia/go-metrics-graphite" + +addr, _ := net.ResolveTCPAddr("tcp", "127.0.0.1:2003") +go graphite.Graphite(metrics.DefaultRegistry, 10e9, "metrics", addr) +``` + +Periodically emit every metric into InfluxDB: + +**NOTE:** this has been pulled out of the library due to constant fluctuations +in the InfluxDB API. In fact, all client libraries are on their way out. see +issues [#121](https://github.com/rcrowley/go-metrics/issues/121) and +[#124](https://github.com/rcrowley/go-metrics/issues/124) for progress and details. + +```go +import "github.com/vrischmann/go-metrics-influxdb" + +go influxdb.InfluxDB(metrics.DefaultRegistry, + 10e9, + "127.0.0.1:8086", + "database-name", + "username", + "password" +) +``` + +Periodically upload every metric to Librato using the [Librato client](https://github.com/mihasya/go-metrics-librato): + +**Note**: the client included with this repository under the `librato` package +has been deprecated and moved to the repository linked above. + +```go +import "github.com/mihasya/go-metrics-librato" + +go librato.Librato(metrics.DefaultRegistry, + 10e9, // interval + "example@example.com", // account owner email address + "token", // Librato API token + "hostname", // source + []float64{0.95}, // percentiles to send + time.Millisecond, // time unit +) +``` + +Periodically emit every metric to StatHat: + +```go +import "github.com/rcrowley/go-metrics/stathat" + +go stathat.Stathat(metrics.DefaultRegistry, 10e9, "example@example.com") +``` + +Maintain all metrics along with expvars at `/debug/metrics`: + +This uses the same mechanism as [the official expvar](http://golang.org/pkg/expvar/) +but exposed under `/debug/metrics`, which shows a json representation of all your usual expvars +as well as all your go-metrics. + + +```go +import "github.com/rcrowley/go-metrics/exp" + +exp.Exp(metrics.DefaultRegistry) +``` + +Installation +------------ + +```sh +go get github.com/rcrowley/go-metrics +``` + +StatHat support additionally requires their Go client: + +```sh +go get github.com/stathat/go +``` + +Publishing Metrics +------------------ + +Clients are available for the following destinations: + +* AppOptics - https://github.com/ysamlan/go-metrics-appoptics +* Librato - https://github.com/mihasya/go-metrics-librato +* Graphite - https://github.com/cyberdelia/go-metrics-graphite +* InfluxDB - https://github.com/vrischmann/go-metrics-influxdb +* Ganglia - https://github.com/appscode/metlia +* Prometheus - https://github.com/deathowl/go-metrics-prometheus +* DataDog - https://github.com/syntaqx/go-metrics-datadog +* SignalFX - https://github.com/pascallouisperez/go-metrics-signalfx +* Honeycomb - https://github.com/getspine/go-metrics-honeycomb +* Wavefront - https://github.com/wavefrontHQ/go-metrics-wavefront +* Open-Falcon - https://github.com/g4zhuj/go-metrics-falcon +* AWS CloudWatch - [https://github.com/savaki/cloudmetrics](https://github.com/savaki/cloudmetrics) diff --git a/vendor/github.com/rcrowley/go-metrics/counter.go b/vendor/github.com/rcrowley/go-metrics/counter.go new file mode 100644 index 000000000000..bb7b039cb572 --- /dev/null +++ b/vendor/github.com/rcrowley/go-metrics/counter.go @@ -0,0 +1,112 @@ +package metrics + +import "sync/atomic" + +// Counters hold an int64 value that can be incremented and decremented. +type Counter interface { + Clear() + Count() int64 + Dec(int64) + Inc(int64) + Snapshot() Counter +} + +// GetOrRegisterCounter returns an existing Counter or constructs and registers +// a new StandardCounter. +func GetOrRegisterCounter(name string, r Registry) Counter { + if nil == r { + r = DefaultRegistry + } + return r.GetOrRegister(name, NewCounter).(Counter) +} + +// NewCounter constructs a new StandardCounter. +func NewCounter() Counter { + if UseNilMetrics { + return NilCounter{} + } + return &StandardCounter{0} +} + +// NewRegisteredCounter constructs and registers a new StandardCounter. +func NewRegisteredCounter(name string, r Registry) Counter { + c := NewCounter() + if nil == r { + r = DefaultRegistry + } + r.Register(name, c) + return c +} + +// CounterSnapshot is a read-only copy of another Counter. +type CounterSnapshot int64 + +// Clear panics. +func (CounterSnapshot) Clear() { + panic("Clear called on a CounterSnapshot") +} + +// Count returns the count at the time the snapshot was taken. +func (c CounterSnapshot) Count() int64 { return int64(c) } + +// Dec panics. +func (CounterSnapshot) Dec(int64) { + panic("Dec called on a CounterSnapshot") +} + +// Inc panics. +func (CounterSnapshot) Inc(int64) { + panic("Inc called on a CounterSnapshot") +} + +// Snapshot returns the snapshot. +func (c CounterSnapshot) Snapshot() Counter { return c } + +// NilCounter is a no-op Counter. +type NilCounter struct{} + +// Clear is a no-op. +func (NilCounter) Clear() {} + +// Count is a no-op. +func (NilCounter) Count() int64 { return 0 } + +// Dec is a no-op. +func (NilCounter) Dec(i int64) {} + +// Inc is a no-op. +func (NilCounter) Inc(i int64) {} + +// Snapshot is a no-op. +func (NilCounter) Snapshot() Counter { return NilCounter{} } + +// StandardCounter is the standard implementation of a Counter and uses the +// sync/atomic package to manage a single int64 value. +type StandardCounter struct { + count int64 +} + +// Clear sets the counter to zero. +func (c *StandardCounter) Clear() { + atomic.StoreInt64(&c.count, 0) +} + +// Count returns the current count. +func (c *StandardCounter) Count() int64 { + return atomic.LoadInt64(&c.count) +} + +// Dec decrements the counter by the given amount. +func (c *StandardCounter) Dec(i int64) { + atomic.AddInt64(&c.count, -i) +} + +// Inc increments the counter by the given amount. +func (c *StandardCounter) Inc(i int64) { + atomic.AddInt64(&c.count, i) +} + +// Snapshot returns a read-only copy of the counter. +func (c *StandardCounter) Snapshot() Counter { + return CounterSnapshot(c.Count()) +} diff --git a/vendor/github.com/rcrowley/go-metrics/debug.go b/vendor/github.com/rcrowley/go-metrics/debug.go new file mode 100644 index 000000000000..179e5aae7290 --- /dev/null +++ b/vendor/github.com/rcrowley/go-metrics/debug.go @@ -0,0 +1,80 @@ +package metrics + +import ( + "runtime/debug" + "sync" + "time" +) + +var ( + debugMetrics struct { + GCStats struct { + LastGC Gauge + NumGC Gauge + Pause Histogram + //PauseQuantiles Histogram + PauseTotal Gauge + } + ReadGCStats Timer + } + gcStats debug.GCStats + registerDebugMetricsOnce = sync.Once{} +) + +// Capture new values for the Go garbage collector statistics exported in +// debug.GCStats. This is designed to be called as a goroutine. +func CaptureDebugGCStats(r Registry, d time.Duration) { + for _ = range time.Tick(d) { + CaptureDebugGCStatsOnce(r) + } +} + +// Capture new values for the Go garbage collector statistics exported in +// debug.GCStats. This is designed to be called in a background goroutine. +// Giving a registry which has not been given to RegisterDebugGCStats will +// panic. +// +// Be careful (but much less so) with this because debug.ReadGCStats calls +// the C function runtime·lock(runtime·mheap) which, while not a stop-the-world +// operation, isn't something you want to be doing all the time. +func CaptureDebugGCStatsOnce(r Registry) { + lastGC := gcStats.LastGC + t := time.Now() + debug.ReadGCStats(&gcStats) + debugMetrics.ReadGCStats.UpdateSince(t) + + debugMetrics.GCStats.LastGC.Update(int64(gcStats.LastGC.UnixNano())) + debugMetrics.GCStats.NumGC.Update(int64(gcStats.NumGC)) + if lastGC != gcStats.LastGC && 0 < len(gcStats.Pause) { + debugMetrics.GCStats.Pause.Update(int64(gcStats.Pause[0])) + } + //debugMetrics.GCStats.PauseQuantiles.Update(gcStats.PauseQuantiles) + debugMetrics.GCStats.PauseTotal.Update(int64(gcStats.PauseTotal)) +} + +// Register metrics for the Go garbage collector statistics exported in +// debug.GCStats. The metrics are named by their fully-qualified Go symbols, +// i.e. debug.GCStats.PauseTotal. +func RegisterDebugGCStats(r Registry) { + registerDebugMetricsOnce.Do(func() { + debugMetrics.GCStats.LastGC = NewGauge() + debugMetrics.GCStats.NumGC = NewGauge() + debugMetrics.GCStats.Pause = NewHistogram(NewExpDecaySample(1028, 0.015)) + //debugMetrics.GCStats.PauseQuantiles = NewHistogram(NewExpDecaySample(1028, 0.015)) + debugMetrics.GCStats.PauseTotal = NewGauge() + debugMetrics.ReadGCStats = NewTimer() + + r.Register("debug.GCStats.LastGC", debugMetrics.GCStats.LastGC) + r.Register("debug.GCStats.NumGC", debugMetrics.GCStats.NumGC) + r.Register("debug.GCStats.Pause", debugMetrics.GCStats.Pause) + //r.Register("debug.GCStats.PauseQuantiles", debugMetrics.GCStats.PauseQuantiles) + r.Register("debug.GCStats.PauseTotal", debugMetrics.GCStats.PauseTotal) + r.Register("debug.ReadGCStats", debugMetrics.ReadGCStats) + }) +} + +// Allocate an initial slice for gcStats.Pause to avoid allocations during +// normal operation. +func init() { + gcStats.Pause = make([]time.Duration, 11) +} diff --git a/vendor/github.com/rcrowley/go-metrics/ewma.go b/vendor/github.com/rcrowley/go-metrics/ewma.go new file mode 100644 index 000000000000..a8183dd7e21f --- /dev/null +++ b/vendor/github.com/rcrowley/go-metrics/ewma.go @@ -0,0 +1,138 @@ +package metrics + +import ( + "math" + "sync" + "sync/atomic" +) + +// EWMAs continuously calculate an exponentially-weighted moving average +// based on an outside source of clock ticks. +type EWMA interface { + Rate() float64 + Snapshot() EWMA + Tick() + Update(int64) +} + +// NewEWMA constructs a new EWMA with the given alpha. +func NewEWMA(alpha float64) EWMA { + if UseNilMetrics { + return NilEWMA{} + } + return &StandardEWMA{alpha: alpha} +} + +// NewEWMA1 constructs a new EWMA for a one-minute moving average. +func NewEWMA1() EWMA { + return NewEWMA(1 - math.Exp(-5.0/60.0/1)) +} + +// NewEWMA5 constructs a new EWMA for a five-minute moving average. +func NewEWMA5() EWMA { + return NewEWMA(1 - math.Exp(-5.0/60.0/5)) +} + +// NewEWMA15 constructs a new EWMA for a fifteen-minute moving average. +func NewEWMA15() EWMA { + return NewEWMA(1 - math.Exp(-5.0/60.0/15)) +} + +// EWMASnapshot is a read-only copy of another EWMA. +type EWMASnapshot float64 + +// Rate returns the rate of events per second at the time the snapshot was +// taken. +func (a EWMASnapshot) Rate() float64 { return float64(a) } + +// Snapshot returns the snapshot. +func (a EWMASnapshot) Snapshot() EWMA { return a } + +// Tick panics. +func (EWMASnapshot) Tick() { + panic("Tick called on an EWMASnapshot") +} + +// Update panics. +func (EWMASnapshot) Update(int64) { + panic("Update called on an EWMASnapshot") +} + +// NilEWMA is a no-op EWMA. +type NilEWMA struct{} + +// Rate is a no-op. +func (NilEWMA) Rate() float64 { return 0.0 } + +// Snapshot is a no-op. +func (NilEWMA) Snapshot() EWMA { return NilEWMA{} } + +// Tick is a no-op. +func (NilEWMA) Tick() {} + +// Update is a no-op. +func (NilEWMA) Update(n int64) {} + +// StandardEWMA is the standard implementation of an EWMA and tracks the number +// of uncounted events and processes them on each tick. It uses the +// sync/atomic package to manage uncounted events. +type StandardEWMA struct { + uncounted int64 // /!\ this should be the first member to ensure 64-bit alignment + alpha float64 + rate uint64 + init uint32 + mutex sync.Mutex +} + +// Rate returns the moving average rate of events per second. +func (a *StandardEWMA) Rate() float64 { + currentRate := math.Float64frombits(atomic.LoadUint64(&a.rate)) * float64(1e9) + return currentRate +} + +// Snapshot returns a read-only copy of the EWMA. +func (a *StandardEWMA) Snapshot() EWMA { + return EWMASnapshot(a.Rate()) +} + +// Tick ticks the clock to update the moving average. It assumes it is called +// every five seconds. +func (a *StandardEWMA) Tick() { + // Optimization to avoid mutex locking in the hot-path. + if atomic.LoadUint32(&a.init) == 1 { + a.updateRate(a.fetchInstantRate()) + } else { + // Slow-path: this is only needed on the first Tick() and preserves transactional updating + // of init and rate in the else block. The first conditional is needed below because + // a different thread could have set a.init = 1 between the time of the first atomic load and when + // the lock was acquired. + a.mutex.Lock() + if atomic.LoadUint32(&a.init) == 1 { + // The fetchInstantRate() uses atomic loading, which is unecessary in this critical section + // but again, this section is only invoked on the first successful Tick() operation. + a.updateRate(a.fetchInstantRate()) + } else { + atomic.StoreUint32(&a.init, 1) + atomic.StoreUint64(&a.rate, math.Float64bits(a.fetchInstantRate())) + } + a.mutex.Unlock() + } +} + +func (a *StandardEWMA) fetchInstantRate() float64 { + count := atomic.LoadInt64(&a.uncounted) + atomic.AddInt64(&a.uncounted, -count) + instantRate := float64(count) / float64(5e9) + return instantRate +} + +func (a *StandardEWMA) updateRate(instantRate float64) { + currentRate := math.Float64frombits(atomic.LoadUint64(&a.rate)) + currentRate += a.alpha * (instantRate - currentRate) + atomic.StoreUint64(&a.rate, math.Float64bits(currentRate)) +} + +// Update adds n uncounted events. +func (a *StandardEWMA) Update(n int64) { + atomic.AddInt64(&a.uncounted, n) +} diff --git a/vendor/github.com/rcrowley/go-metrics/gauge.go b/vendor/github.com/rcrowley/go-metrics/gauge.go new file mode 100644 index 000000000000..cb57a93889fc --- /dev/null +++ b/vendor/github.com/rcrowley/go-metrics/gauge.go @@ -0,0 +1,120 @@ +package metrics + +import "sync/atomic" + +// Gauges hold an int64 value that can be set arbitrarily. +type Gauge interface { + Snapshot() Gauge + Update(int64) + Value() int64 +} + +// GetOrRegisterGauge returns an existing Gauge or constructs and registers a +// new StandardGauge. +func GetOrRegisterGauge(name string, r Registry) Gauge { + if nil == r { + r = DefaultRegistry + } + return r.GetOrRegister(name, NewGauge).(Gauge) +} + +// NewGauge constructs a new StandardGauge. +func NewGauge() Gauge { + if UseNilMetrics { + return NilGauge{} + } + return &StandardGauge{0} +} + +// NewRegisteredGauge constructs and registers a new StandardGauge. +func NewRegisteredGauge(name string, r Registry) Gauge { + c := NewGauge() + if nil == r { + r = DefaultRegistry + } + r.Register(name, c) + return c +} + +// NewFunctionalGauge constructs a new FunctionalGauge. +func NewFunctionalGauge(f func() int64) Gauge { + if UseNilMetrics { + return NilGauge{} + } + return &FunctionalGauge{value: f} +} + +// NewRegisteredFunctionalGauge constructs and registers a new StandardGauge. +func NewRegisteredFunctionalGauge(name string, r Registry, f func() int64) Gauge { + c := NewFunctionalGauge(f) + if nil == r { + r = DefaultRegistry + } + r.Register(name, c) + return c +} + +// GaugeSnapshot is a read-only copy of another Gauge. +type GaugeSnapshot int64 + +// Snapshot returns the snapshot. +func (g GaugeSnapshot) Snapshot() Gauge { return g } + +// Update panics. +func (GaugeSnapshot) Update(int64) { + panic("Update called on a GaugeSnapshot") +} + +// Value returns the value at the time the snapshot was taken. +func (g GaugeSnapshot) Value() int64 { return int64(g) } + +// NilGauge is a no-op Gauge. +type NilGauge struct{} + +// Snapshot is a no-op. +func (NilGauge) Snapshot() Gauge { return NilGauge{} } + +// Update is a no-op. +func (NilGauge) Update(v int64) {} + +// Value is a no-op. +func (NilGauge) Value() int64 { return 0 } + +// StandardGauge is the standard implementation of a Gauge and uses the +// sync/atomic package to manage a single int64 value. +type StandardGauge struct { + value int64 +} + +// Snapshot returns a read-only copy of the gauge. +func (g *StandardGauge) Snapshot() Gauge { + return GaugeSnapshot(g.Value()) +} + +// Update updates the gauge's value. +func (g *StandardGauge) Update(v int64) { + atomic.StoreInt64(&g.value, v) +} + +// Value returns the gauge's current value. +func (g *StandardGauge) Value() int64 { + return atomic.LoadInt64(&g.value) +} + +// FunctionalGauge returns value from given function +type FunctionalGauge struct { + value func() int64 +} + +// Value returns the gauge's current value. +func (g FunctionalGauge) Value() int64 { + return g.value() +} + +// Snapshot returns the snapshot. +func (g FunctionalGauge) Snapshot() Gauge { return GaugeSnapshot(g.Value()) } + +// Update panics. +func (FunctionalGauge) Update(int64) { + panic("Update called on a FunctionalGauge") +} diff --git a/vendor/github.com/rcrowley/go-metrics/gauge_float64.go b/vendor/github.com/rcrowley/go-metrics/gauge_float64.go new file mode 100644 index 000000000000..3962e6db09a8 --- /dev/null +++ b/vendor/github.com/rcrowley/go-metrics/gauge_float64.go @@ -0,0 +1,125 @@ +package metrics + +import ( + "math" + "sync/atomic" +) + +// GaugeFloat64s hold a float64 value that can be set arbitrarily. +type GaugeFloat64 interface { + Snapshot() GaugeFloat64 + Update(float64) + Value() float64 +} + +// GetOrRegisterGaugeFloat64 returns an existing GaugeFloat64 or constructs and registers a +// new StandardGaugeFloat64. +func GetOrRegisterGaugeFloat64(name string, r Registry) GaugeFloat64 { + if nil == r { + r = DefaultRegistry + } + return r.GetOrRegister(name, NewGaugeFloat64()).(GaugeFloat64) +} + +// NewGaugeFloat64 constructs a new StandardGaugeFloat64. +func NewGaugeFloat64() GaugeFloat64 { + if UseNilMetrics { + return NilGaugeFloat64{} + } + return &StandardGaugeFloat64{ + value: 0.0, + } +} + +// NewRegisteredGaugeFloat64 constructs and registers a new StandardGaugeFloat64. +func NewRegisteredGaugeFloat64(name string, r Registry) GaugeFloat64 { + c := NewGaugeFloat64() + if nil == r { + r = DefaultRegistry + } + r.Register(name, c) + return c +} + +// NewFunctionalGauge constructs a new FunctionalGauge. +func NewFunctionalGaugeFloat64(f func() float64) GaugeFloat64 { + if UseNilMetrics { + return NilGaugeFloat64{} + } + return &FunctionalGaugeFloat64{value: f} +} + +// NewRegisteredFunctionalGauge constructs and registers a new StandardGauge. +func NewRegisteredFunctionalGaugeFloat64(name string, r Registry, f func() float64) GaugeFloat64 { + c := NewFunctionalGaugeFloat64(f) + if nil == r { + r = DefaultRegistry + } + r.Register(name, c) + return c +} + +// GaugeFloat64Snapshot is a read-only copy of another GaugeFloat64. +type GaugeFloat64Snapshot float64 + +// Snapshot returns the snapshot. +func (g GaugeFloat64Snapshot) Snapshot() GaugeFloat64 { return g } + +// Update panics. +func (GaugeFloat64Snapshot) Update(float64) { + panic("Update called on a GaugeFloat64Snapshot") +} + +// Value returns the value at the time the snapshot was taken. +func (g GaugeFloat64Snapshot) Value() float64 { return float64(g) } + +// NilGauge is a no-op Gauge. +type NilGaugeFloat64 struct{} + +// Snapshot is a no-op. +func (NilGaugeFloat64) Snapshot() GaugeFloat64 { return NilGaugeFloat64{} } + +// Update is a no-op. +func (NilGaugeFloat64) Update(v float64) {} + +// Value is a no-op. +func (NilGaugeFloat64) Value() float64 { return 0.0 } + +// StandardGaugeFloat64 is the standard implementation of a GaugeFloat64 and uses +// sync.Mutex to manage a single float64 value. +type StandardGaugeFloat64 struct { + value uint64 +} + +// Snapshot returns a read-only copy of the gauge. +func (g *StandardGaugeFloat64) Snapshot() GaugeFloat64 { + return GaugeFloat64Snapshot(g.Value()) +} + +// Update updates the gauge's value. +func (g *StandardGaugeFloat64) Update(v float64) { + atomic.StoreUint64(&g.value, math.Float64bits(v)) +} + +// Value returns the gauge's current value. +func (g *StandardGaugeFloat64) Value() float64 { + return math.Float64frombits(atomic.LoadUint64(&g.value)) +} + +// FunctionalGaugeFloat64 returns value from given function +type FunctionalGaugeFloat64 struct { + value func() float64 +} + +// Value returns the gauge's current value. +func (g FunctionalGaugeFloat64) Value() float64 { + return g.value() +} + +// Snapshot returns the snapshot. +func (g FunctionalGaugeFloat64) Snapshot() GaugeFloat64 { return GaugeFloat64Snapshot(g.Value()) } + +// Update panics. +func (FunctionalGaugeFloat64) Update(float64) { + panic("Update called on a FunctionalGaugeFloat64") +} diff --git a/vendor/github.com/rcrowley/go-metrics/graphite.go b/vendor/github.com/rcrowley/go-metrics/graphite.go new file mode 100644 index 000000000000..abd0a7d2918b --- /dev/null +++ b/vendor/github.com/rcrowley/go-metrics/graphite.go @@ -0,0 +1,113 @@ +package metrics + +import ( + "bufio" + "fmt" + "log" + "net" + "strconv" + "strings" + "time" +) + +// GraphiteConfig provides a container with configuration parameters for +// the Graphite exporter +type GraphiteConfig struct { + Addr *net.TCPAddr // Network address to connect to + Registry Registry // Registry to be exported + FlushInterval time.Duration // Flush interval + DurationUnit time.Duration // Time conversion unit for durations + Prefix string // Prefix to be prepended to metric names + Percentiles []float64 // Percentiles to export from timers and histograms +} + +// Graphite is a blocking exporter function which reports metrics in r +// to a graphite server located at addr, flushing them every d duration +// and prepending metric names with prefix. +func Graphite(r Registry, d time.Duration, prefix string, addr *net.TCPAddr) { + GraphiteWithConfig(GraphiteConfig{ + Addr: addr, + Registry: r, + FlushInterval: d, + DurationUnit: time.Nanosecond, + Prefix: prefix, + Percentiles: []float64{0.5, 0.75, 0.95, 0.99, 0.999}, + }) +} + +// GraphiteWithConfig is a blocking exporter function just like Graphite, +// but it takes a GraphiteConfig instead. +func GraphiteWithConfig(c GraphiteConfig) { + log.Printf("WARNING: This go-metrics client has been DEPRECATED! It has been moved to https://github.com/cyberdelia/go-metrics-graphite and will be removed from rcrowley/go-metrics on August 12th 2015") + for _ = range time.Tick(c.FlushInterval) { + if err := graphite(&c); nil != err { + log.Println(err) + } + } +} + +// GraphiteOnce performs a single submission to Graphite, returning a +// non-nil error on failed connections. This can be used in a loop +// similar to GraphiteWithConfig for custom error handling. +func GraphiteOnce(c GraphiteConfig) error { + log.Printf("WARNING: This go-metrics client has been DEPRECATED! It has been moved to https://github.com/cyberdelia/go-metrics-graphite and will be removed from rcrowley/go-metrics on August 12th 2015") + return graphite(&c) +} + +func graphite(c *GraphiteConfig) error { + now := time.Now().Unix() + du := float64(c.DurationUnit) + conn, err := net.DialTCP("tcp", nil, c.Addr) + if nil != err { + return err + } + defer conn.Close() + w := bufio.NewWriter(conn) + c.Registry.Each(func(name string, i interface{}) { + switch metric := i.(type) { + case Counter: + fmt.Fprintf(w, "%s.%s.count %d %d\n", c.Prefix, name, metric.Count(), now) + case Gauge: + fmt.Fprintf(w, "%s.%s.value %d %d\n", c.Prefix, name, metric.Value(), now) + case GaugeFloat64: + fmt.Fprintf(w, "%s.%s.value %f %d\n", c.Prefix, name, metric.Value(), now) + case Histogram: + h := metric.Snapshot() + ps := h.Percentiles(c.Percentiles) + fmt.Fprintf(w, "%s.%s.count %d %d\n", c.Prefix, name, h.Count(), now) + fmt.Fprintf(w, "%s.%s.min %d %d\n", c.Prefix, name, h.Min(), now) + fmt.Fprintf(w, "%s.%s.max %d %d\n", c.Prefix, name, h.Max(), now) + fmt.Fprintf(w, "%s.%s.mean %.2f %d\n", c.Prefix, name, h.Mean(), now) + fmt.Fprintf(w, "%s.%s.std-dev %.2f %d\n", c.Prefix, name, h.StdDev(), now) + for psIdx, psKey := range c.Percentiles { + key := strings.Replace(strconv.FormatFloat(psKey*100.0, 'f', -1, 64), ".", "", 1) + fmt.Fprintf(w, "%s.%s.%s-percentile %.2f %d\n", c.Prefix, name, key, ps[psIdx], now) + } + case Meter: + m := metric.Snapshot() + fmt.Fprintf(w, "%s.%s.count %d %d\n", c.Prefix, name, m.Count(), now) + fmt.Fprintf(w, "%s.%s.one-minute %.2f %d\n", c.Prefix, name, m.Rate1(), now) + fmt.Fprintf(w, "%s.%s.five-minute %.2f %d\n", c.Prefix, name, m.Rate5(), now) + fmt.Fprintf(w, "%s.%s.fifteen-minute %.2f %d\n", c.Prefix, name, m.Rate15(), now) + fmt.Fprintf(w, "%s.%s.mean %.2f %d\n", c.Prefix, name, m.RateMean(), now) + case Timer: + t := metric.Snapshot() + ps := t.Percentiles(c.Percentiles) + fmt.Fprintf(w, "%s.%s.count %d %d\n", c.Prefix, name, t.Count(), now) + fmt.Fprintf(w, "%s.%s.min %d %d\n", c.Prefix, name, t.Min()/int64(du), now) + fmt.Fprintf(w, "%s.%s.max %d %d\n", c.Prefix, name, t.Max()/int64(du), now) + fmt.Fprintf(w, "%s.%s.mean %.2f %d\n", c.Prefix, name, t.Mean()/du, now) + fmt.Fprintf(w, "%s.%s.std-dev %.2f %d\n", c.Prefix, name, t.StdDev()/du, now) + for psIdx, psKey := range c.Percentiles { + key := strings.Replace(strconv.FormatFloat(psKey*100.0, 'f', -1, 64), ".", "", 1) + fmt.Fprintf(w, "%s.%s.%s-percentile %.2f %d\n", c.Prefix, name, key, ps[psIdx], now) + } + fmt.Fprintf(w, "%s.%s.one-minute %.2f %d\n", c.Prefix, name, t.Rate1(), now) + fmt.Fprintf(w, "%s.%s.five-minute %.2f %d\n", c.Prefix, name, t.Rate5(), now) + fmt.Fprintf(w, "%s.%s.fifteen-minute %.2f %d\n", c.Prefix, name, t.Rate15(), now) + fmt.Fprintf(w, "%s.%s.mean-rate %.2f %d\n", c.Prefix, name, t.RateMean(), now) + } + w.Flush() + }) + return nil +} diff --git a/vendor/github.com/rcrowley/go-metrics/healthcheck.go b/vendor/github.com/rcrowley/go-metrics/healthcheck.go new file mode 100644 index 000000000000..445131caee59 --- /dev/null +++ b/vendor/github.com/rcrowley/go-metrics/healthcheck.go @@ -0,0 +1,61 @@ +package metrics + +// Healthchecks hold an error value describing an arbitrary up/down status. +type Healthcheck interface { + Check() + Error() error + Healthy() + Unhealthy(error) +} + +// NewHealthcheck constructs a new Healthcheck which will use the given +// function to update its status. +func NewHealthcheck(f func(Healthcheck)) Healthcheck { + if UseNilMetrics { + return NilHealthcheck{} + } + return &StandardHealthcheck{nil, f} +} + +// NilHealthcheck is a no-op. +type NilHealthcheck struct{} + +// Check is a no-op. +func (NilHealthcheck) Check() {} + +// Error is a no-op. +func (NilHealthcheck) Error() error { return nil } + +// Healthy is a no-op. +func (NilHealthcheck) Healthy() {} + +// Unhealthy is a no-op. +func (NilHealthcheck) Unhealthy(error) {} + +// StandardHealthcheck is the standard implementation of a Healthcheck and +// stores the status and a function to call to update the status. +type StandardHealthcheck struct { + err error + f func(Healthcheck) +} + +// Check runs the healthcheck function to update the healthcheck's status. +func (h *StandardHealthcheck) Check() { + h.f(h) +} + +// Error returns the healthcheck's status, which will be nil if it is healthy. +func (h *StandardHealthcheck) Error() error { + return h.err +} + +// Healthy marks the healthcheck as healthy. +func (h *StandardHealthcheck) Healthy() { + h.err = nil +} + +// Unhealthy marks the healthcheck as unhealthy. The error is stored and +// may be retrieved by the Error method. +func (h *StandardHealthcheck) Unhealthy(err error) { + h.err = err +} diff --git a/vendor/github.com/rcrowley/go-metrics/histogram.go b/vendor/github.com/rcrowley/go-metrics/histogram.go new file mode 100644 index 000000000000..dbc837fe4d95 --- /dev/null +++ b/vendor/github.com/rcrowley/go-metrics/histogram.go @@ -0,0 +1,202 @@ +package metrics + +// Histograms calculate distribution statistics from a series of int64 values. +type Histogram interface { + Clear() + Count() int64 + Max() int64 + Mean() float64 + Min() int64 + Percentile(float64) float64 + Percentiles([]float64) []float64 + Sample() Sample + Snapshot() Histogram + StdDev() float64 + Sum() int64 + Update(int64) + Variance() float64 +} + +// GetOrRegisterHistogram returns an existing Histogram or constructs and +// registers a new StandardHistogram. +func GetOrRegisterHistogram(name string, r Registry, s Sample) Histogram { + if nil == r { + r = DefaultRegistry + } + return r.GetOrRegister(name, func() Histogram { return NewHistogram(s) }).(Histogram) +} + +// NewHistogram constructs a new StandardHistogram from a Sample. +func NewHistogram(s Sample) Histogram { + if UseNilMetrics { + return NilHistogram{} + } + return &StandardHistogram{sample: s} +} + +// NewRegisteredHistogram constructs and registers a new StandardHistogram from +// a Sample. +func NewRegisteredHistogram(name string, r Registry, s Sample) Histogram { + c := NewHistogram(s) + if nil == r { + r = DefaultRegistry + } + r.Register(name, c) + return c +} + +// HistogramSnapshot is a read-only copy of another Histogram. +type HistogramSnapshot struct { + sample *SampleSnapshot +} + +// Clear panics. +func (*HistogramSnapshot) Clear() { + panic("Clear called on a HistogramSnapshot") +} + +// Count returns the number of samples recorded at the time the snapshot was +// taken. +func (h *HistogramSnapshot) Count() int64 { return h.sample.Count() } + +// Max returns the maximum value in the sample at the time the snapshot was +// taken. +func (h *HistogramSnapshot) Max() int64 { return h.sample.Max() } + +// Mean returns the mean of the values in the sample at the time the snapshot +// was taken. +func (h *HistogramSnapshot) Mean() float64 { return h.sample.Mean() } + +// Min returns the minimum value in the sample at the time the snapshot was +// taken. +func (h *HistogramSnapshot) Min() int64 { return h.sample.Min() } + +// Percentile returns an arbitrary percentile of values in the sample at the +// time the snapshot was taken. +func (h *HistogramSnapshot) Percentile(p float64) float64 { + return h.sample.Percentile(p) +} + +// Percentiles returns a slice of arbitrary percentiles of values in the sample +// at the time the snapshot was taken. +func (h *HistogramSnapshot) Percentiles(ps []float64) []float64 { + return h.sample.Percentiles(ps) +} + +// Sample returns the Sample underlying the histogram. +func (h *HistogramSnapshot) Sample() Sample { return h.sample } + +// Snapshot returns the snapshot. +func (h *HistogramSnapshot) Snapshot() Histogram { return h } + +// StdDev returns the standard deviation of the values in the sample at the +// time the snapshot was taken. +func (h *HistogramSnapshot) StdDev() float64 { return h.sample.StdDev() } + +// Sum returns the sum in the sample at the time the snapshot was taken. +func (h *HistogramSnapshot) Sum() int64 { return h.sample.Sum() } + +// Update panics. +func (*HistogramSnapshot) Update(int64) { + panic("Update called on a HistogramSnapshot") +} + +// Variance returns the variance of inputs at the time the snapshot was taken. +func (h *HistogramSnapshot) Variance() float64 { return h.sample.Variance() } + +// NilHistogram is a no-op Histogram. +type NilHistogram struct{} + +// Clear is a no-op. +func (NilHistogram) Clear() {} + +// Count is a no-op. +func (NilHistogram) Count() int64 { return 0 } + +// Max is a no-op. +func (NilHistogram) Max() int64 { return 0 } + +// Mean is a no-op. +func (NilHistogram) Mean() float64 { return 0.0 } + +// Min is a no-op. +func (NilHistogram) Min() int64 { return 0 } + +// Percentile is a no-op. +func (NilHistogram) Percentile(p float64) float64 { return 0.0 } + +// Percentiles is a no-op. +func (NilHistogram) Percentiles(ps []float64) []float64 { + return make([]float64, len(ps)) +} + +// Sample is a no-op. +func (NilHistogram) Sample() Sample { return NilSample{} } + +// Snapshot is a no-op. +func (NilHistogram) Snapshot() Histogram { return NilHistogram{} } + +// StdDev is a no-op. +func (NilHistogram) StdDev() float64 { return 0.0 } + +// Sum is a no-op. +func (NilHistogram) Sum() int64 { return 0 } + +// Update is a no-op. +func (NilHistogram) Update(v int64) {} + +// Variance is a no-op. +func (NilHistogram) Variance() float64 { return 0.0 } + +// StandardHistogram is the standard implementation of a Histogram and uses a +// Sample to bound its memory use. +type StandardHistogram struct { + sample Sample +} + +// Clear clears the histogram and its sample. +func (h *StandardHistogram) Clear() { h.sample.Clear() } + +// Count returns the number of samples recorded since the histogram was last +// cleared. +func (h *StandardHistogram) Count() int64 { return h.sample.Count() } + +// Max returns the maximum value in the sample. +func (h *StandardHistogram) Max() int64 { return h.sample.Max() } + +// Mean returns the mean of the values in the sample. +func (h *StandardHistogram) Mean() float64 { return h.sample.Mean() } + +// Min returns the minimum value in the sample. +func (h *StandardHistogram) Min() int64 { return h.sample.Min() } + +// Percentile returns an arbitrary percentile of the values in the sample. +func (h *StandardHistogram) Percentile(p float64) float64 { + return h.sample.Percentile(p) +} + +// Percentiles returns a slice of arbitrary percentiles of the values in the +// sample. +func (h *StandardHistogram) Percentiles(ps []float64) []float64 { + return h.sample.Percentiles(ps) +} + +// Sample returns the Sample underlying the histogram. +func (h *StandardHistogram) Sample() Sample { return h.sample } + +// Snapshot returns a read-only copy of the histogram. +func (h *StandardHistogram) Snapshot() Histogram { + return &HistogramSnapshot{sample: h.sample.Snapshot().(*SampleSnapshot)} +} + +// StdDev returns the standard deviation of the values in the sample. +func (h *StandardHistogram) StdDev() float64 { return h.sample.StdDev() } + +// Sum returns the sum in the sample. +func (h *StandardHistogram) Sum() int64 { return h.sample.Sum() } + +// Update samples a new value. +func (h *StandardHistogram) Update(v int64) { h.sample.Update(v) } + +// Variance returns the variance of the values in the sample. +func (h *StandardHistogram) Variance() float64 { return h.sample.Variance() } diff --git a/vendor/github.com/rcrowley/go-metrics/json.go b/vendor/github.com/rcrowley/go-metrics/json.go new file mode 100644 index 000000000000..174b9477e92d --- /dev/null +++ b/vendor/github.com/rcrowley/go-metrics/json.go @@ -0,0 +1,31 @@ +package metrics + +import ( + "encoding/json" + "io" + "time" +) + +// MarshalJSON returns a byte slice containing a JSON representation of all +// the metrics in the Registry. +func (r *StandardRegistry) MarshalJSON() ([]byte, error) { + return json.Marshal(r.GetAll()) +} + +// WriteJSON writes metrics from the given registry periodically to the +// specified io.Writer as JSON. +func WriteJSON(r Registry, d time.Duration, w io.Writer) { + for _ = range time.Tick(d) { + WriteJSONOnce(r, w) + } +} + +// WriteJSONOnce writes metrics from the given registry to the specified +// io.Writer as JSON. +func WriteJSONOnce(r Registry, w io.Writer) { + json.NewEncoder(w).Encode(r) +} + +func (p *PrefixedRegistry) MarshalJSON() ([]byte, error) { + return json.Marshal(p.GetAll()) +} diff --git a/vendor/github.com/rcrowley/go-metrics/log.go b/vendor/github.com/rcrowley/go-metrics/log.go new file mode 100644 index 000000000000..2614a0a33eb7 --- /dev/null +++ b/vendor/github.com/rcrowley/go-metrics/log.go @@ -0,0 +1,100 @@ +package metrics + +import ( + "time" +) + +type Logger interface { + Printf(format string, v ...interface{}) +} + +// Log outputs each metric in the given registry periodically using the given logger. +func Log(r Registry, freq time.Duration, l Logger) { + LogScaled(r, freq, time.Nanosecond, l) +} + +// LogOnCue outputs each metric in the given registry on demand through the channel +// using the given logger +func LogOnCue(r Registry, ch chan interface{}, l Logger) { + LogScaledOnCue(r, ch, time.Nanosecond, l) +} + +// LogScaled outputs each metric in the given registry periodically using the given +// logger. Print timings in `scale` units (eg time.Millisecond) rather than nanos. +func LogScaled(r Registry, freq time.Duration, scale time.Duration, l Logger) { + ch := make(chan interface{}) + go func(channel chan interface{}) { + for _ = range time.Tick(freq) { + channel <- struct{}{} + } + }(ch) + LogScaledOnCue(r, ch, scale, l) +} + +// LogScaledOnCue outputs each metric in the given registry on demand through the channel +// using the given logger. Print timings in `scale` units (eg time.Millisecond) rather +// than nanos. +func LogScaledOnCue(r Registry, ch chan interface{}, scale time.Duration, l Logger) { + du := float64(scale) + duSuffix := scale.String()[1:] + + for _ = range ch { + r.Each(func(name string, i interface{}) { + switch metric := i.(type) { + case Counter: + l.Printf("counter %s\n", name) + l.Printf(" count: %9d\n", metric.Count()) + case Gauge: + l.Printf("gauge %s\n", name) + l.Printf(" value: %9d\n", metric.Value()) + case GaugeFloat64: + l.Printf("gauge %s\n", name) + l.Printf(" value: %f\n", metric.Value()) + case Healthcheck: + metric.Check() + l.Printf("healthcheck %s\n", name) + l.Printf(" error: %v\n", metric.Error()) + case Histogram: + h := metric.Snapshot() + ps := h.Percentiles([]float64{0.5, 0.75, 0.95, 0.99, 0.999}) + l.Printf("histogram %s\n", name) + l.Printf(" count: %9d\n", h.Count()) + l.Printf(" min: %9d\n", h.Min()) + l.Printf(" max: %9d\n", h.Max()) + l.Printf(" mean: %12.2f\n", h.Mean()) + l.Printf(" stddev: %12.2f\n", h.StdDev()) + l.Printf(" median: %12.2f\n", ps[0]) + l.Printf(" 75%%: %12.2f\n", ps[1]) + l.Printf(" 95%%: %12.2f\n", ps[2]) + l.Printf(" 99%%: %12.2f\n", ps[3]) + l.Printf(" 99.9%%: %12.2f\n", ps[4]) + case Meter: + m := metric.Snapshot() + l.Printf("meter %s\n", name) + l.Printf(" count: %9d\n", m.Count()) + l.Printf(" 1-min rate: %12.2f\n", m.Rate1()) + l.Printf(" 5-min rate: %12.2f\n", m.Rate5()) + l.Printf(" 15-min rate: %12.2f\n", m.Rate15()) + l.Printf(" mean rate: %12.2f\n", m.RateMean()) + case Timer: + t := metric.Snapshot() + ps := t.Percentiles([]float64{0.5, 0.75, 0.95, 0.99, 0.999}) + l.Printf("timer %s\n", name) + l.Printf(" count: %9d\n", t.Count()) + l.Printf(" min: %12.2f%s\n", float64(t.Min())/du, duSuffix) + l.Printf(" max: %12.2f%s\n", float64(t.Max())/du, duSuffix) + l.Printf(" mean: %12.2f%s\n", t.Mean()/du, duSuffix) + l.Printf(" stddev: %12.2f%s\n", t.StdDev()/du, duSuffix) + l.Printf(" median: %12.2f%s\n", ps[0]/du, duSuffix) + l.Printf(" 75%%: %12.2f%s\n", ps[1]/du, duSuffix) + l.Printf(" 95%%: %12.2f%s\n", ps[2]/du, duSuffix) + l.Printf(" 99%%: %12.2f%s\n", ps[3]/du, duSuffix) + l.Printf(" 99.9%%: %12.2f%s\n", ps[4]/du, duSuffix) + l.Printf(" 1-min rate: %12.2f\n", t.Rate1()) + l.Printf(" 5-min rate: %12.2f\n", t.Rate5()) + l.Printf(" 15-min rate: %12.2f\n", t.Rate15()) + l.Printf(" mean rate: %12.2f\n", t.RateMean()) + } + }) + } +} diff --git a/vendor/github.com/rcrowley/go-metrics/memory.md b/vendor/github.com/rcrowley/go-metrics/memory.md new file mode 100644 index 000000000000..47454f54b640 --- /dev/null +++ b/vendor/github.com/rcrowley/go-metrics/memory.md @@ -0,0 +1,285 @@ +Memory usage +============ + +(Highly unscientific.) + +Command used to gather static memory usage: + +```sh +grep ^Vm "/proc/$(ps fax | grep [m]etrics-bench | awk '{print $1}')/status" +``` + +Program used to gather baseline memory usage: + +```go +package main + +import "time" + +func main() { + time.Sleep(600e9) +} +``` + +Baseline +-------- + +``` +VmPeak: 42604 kB +VmSize: 42604 kB +VmLck: 0 kB +VmHWM: 1120 kB +VmRSS: 1120 kB +VmData: 35460 kB +VmStk: 136 kB +VmExe: 1020 kB +VmLib: 1848 kB +VmPTE: 36 kB +VmSwap: 0 kB +``` + +Program used to gather metric memory usage (with other metrics being similar): + +```go +package main + +import ( + "fmt" + "metrics" + "time" +) + +func main() { + fmt.Sprintf("foo") + metrics.NewRegistry() + time.Sleep(600e9) +} +``` + +1000 counters registered +------------------------ + +``` +VmPeak: 44016 kB +VmSize: 44016 kB +VmLck: 0 kB +VmHWM: 1928 kB +VmRSS: 1928 kB +VmData: 36868 kB +VmStk: 136 kB +VmExe: 1024 kB +VmLib: 1848 kB +VmPTE: 40 kB +VmSwap: 0 kB +``` + +**1.412 kB virtual, TODO 0.808 kB resident per counter.** + +100000 counters registered +-------------------------- + +``` +VmPeak: 55024 kB +VmSize: 55024 kB +VmLck: 0 kB +VmHWM: 12440 kB +VmRSS: 12440 kB +VmData: 47876 kB +VmStk: 136 kB +VmExe: 1024 kB +VmLib: 1848 kB +VmPTE: 64 kB +VmSwap: 0 kB +``` + +**0.1242 kB virtual, 0.1132 kB resident per counter.** + +1000 gauges registered +---------------------- + +``` +VmPeak: 44012 kB +VmSize: 44012 kB +VmLck: 0 kB +VmHWM: 1928 kB +VmRSS: 1928 kB +VmData: 36868 kB +VmStk: 136 kB +VmExe: 1020 kB +VmLib: 1848 kB +VmPTE: 40 kB +VmSwap: 0 kB +``` + +**1.408 kB virtual, 0.808 kB resident per counter.** + +100000 gauges registered +------------------------ + +``` +VmPeak: 55020 kB +VmSize: 55020 kB +VmLck: 0 kB +VmHWM: 12432 kB +VmRSS: 12432 kB +VmData: 47876 kB +VmStk: 136 kB +VmExe: 1020 kB +VmLib: 1848 kB +VmPTE: 60 kB +VmSwap: 0 kB +``` + +**0.12416 kB virtual, 0.11312 resident per gauge.** + +1000 histograms with a uniform sample size of 1028 +-------------------------------------------------- + +``` +VmPeak: 72272 kB +VmSize: 72272 kB +VmLck: 0 kB +VmHWM: 16204 kB +VmRSS: 16204 kB +VmData: 65100 kB +VmStk: 136 kB +VmExe: 1048 kB +VmLib: 1848 kB +VmPTE: 80 kB +VmSwap: 0 kB +``` + +**29.668 kB virtual, TODO 15.084 resident per histogram.** + +10000 histograms with a uniform sample size of 1028 +--------------------------------------------------- + +``` +VmPeak: 256912 kB +VmSize: 256912 kB +VmLck: 0 kB +VmHWM: 146204 kB +VmRSS: 146204 kB +VmData: 249740 kB +VmStk: 136 kB +VmExe: 1048 kB +VmLib: 1848 kB +VmPTE: 448 kB +VmSwap: 0 kB +``` + +**21.4308 kB virtual, 14.5084 kB resident per histogram.** + +50000 histograms with a uniform sample size of 1028 +--------------------------------------------------- + +``` +VmPeak: 908112 kB +VmSize: 908112 kB +VmLck: 0 kB +VmHWM: 645832 kB +VmRSS: 645588 kB +VmData: 900940 kB +VmStk: 136 kB +VmExe: 1048 kB +VmLib: 1848 kB +VmPTE: 1716 kB +VmSwap: 1544 kB +``` + +**17.31016 kB virtual, 12.88936 kB resident per histogram.** + +1000 histograms with an exponentially-decaying sample size of 1028 and alpha of 0.015 +------------------------------------------------------------------------------------- + +``` +VmPeak: 62480 kB +VmSize: 62480 kB +VmLck: 0 kB +VmHWM: 11572 kB +VmRSS: 11572 kB +VmData: 55308 kB +VmStk: 136 kB +VmExe: 1048 kB +VmLib: 1848 kB +VmPTE: 64 kB +VmSwap: 0 kB +``` + +**19.876 kB virtual, 10.452 kB resident per histogram.** + +10000 histograms with an exponentially-decaying sample size of 1028 and alpha of 0.015 +-------------------------------------------------------------------------------------- + +``` +VmPeak: 153296 kB +VmSize: 153296 kB +VmLck: 0 kB +VmHWM: 101176 kB +VmRSS: 101176 kB +VmData: 146124 kB +VmStk: 136 kB +VmExe: 1048 kB +VmLib: 1848 kB +VmPTE: 240 kB +VmSwap: 0 kB +``` + +**11.0692 kB virtual, 10.0056 kB resident per histogram.** + +50000 histograms with an exponentially-decaying sample size of 1028 and alpha of 0.015 +-------------------------------------------------------------------------------------- + +``` +VmPeak: 557264 kB +VmSize: 557264 kB +VmLck: 0 kB +VmHWM: 501056 kB +VmRSS: 501056 kB +VmData: 550092 kB +VmStk: 136 kB +VmExe: 1048 kB +VmLib: 1848 kB +VmPTE: 1032 kB +VmSwap: 0 kB +``` + +**10.2932 kB virtual, 9.99872 kB resident per histogram.** + +1000 meters +----------- + +``` +VmPeak: 74504 kB +VmSize: 74504 kB +VmLck: 0 kB +VmHWM: 24124 kB +VmRSS: 24124 kB +VmData: 67340 kB +VmStk: 136 kB +VmExe: 1040 kB +VmLib: 1848 kB +VmPTE: 92 kB +VmSwap: 0 kB +``` + +**31.9 kB virtual, 23.004 kB resident per meter.** + +10000 meters +------------ + +``` +VmPeak: 278920 kB +VmSize: 278920 kB +VmLck: 0 kB +VmHWM: 227300 kB +VmRSS: 227300 kB +VmData: 271756 kB +VmStk: 136 kB +VmExe: 1040 kB +VmLib: 1848 kB +VmPTE: 488 kB +VmSwap: 0 kB +``` + +**23.6316 kB virtual, 22.618 kB resident per meter.** diff --git a/vendor/github.com/rcrowley/go-metrics/meter.go b/vendor/github.com/rcrowley/go-metrics/meter.go new file mode 100644 index 000000000000..223669bcb292 --- /dev/null +++ b/vendor/github.com/rcrowley/go-metrics/meter.go @@ -0,0 +1,251 @@ +package metrics + +import ( + "math" + "sync" + "sync/atomic" + "time" +) + +// Meters count events to produce exponentially-weighted moving average rates +// at one-, five-, and fifteen-minutes and a mean rate. +type Meter interface { + Count() int64 + Mark(int64) + Rate1() float64 + Rate5() float64 + Rate15() float64 + RateMean() float64 + Snapshot() Meter + Stop() +} + +// GetOrRegisterMeter returns an existing Meter or constructs and registers a +// new StandardMeter. +// Be sure to unregister the meter from the registry once it is of no use to +// allow for garbage collection. +func GetOrRegisterMeter(name string, r Registry) Meter { + if nil == r { + r = DefaultRegistry + } + return r.GetOrRegister(name, NewMeter).(Meter) +} + +// NewMeter constructs a new StandardMeter and launches a goroutine. +// Be sure to call Stop() once the meter is of no use to allow for garbage collection. +func NewMeter() Meter { + if UseNilMetrics { + return NilMeter{} + } + m := newStandardMeter() + arbiter.Lock() + defer arbiter.Unlock() + arbiter.meters[m] = struct{}{} + if !arbiter.started { + arbiter.started = true + go arbiter.tick() + } + return m +} + +// NewMeter constructs and registers a new StandardMeter and launches a +// goroutine. +// Be sure to unregister the meter from the registry once it is of no use to +// allow for garbage collection. +func NewRegisteredMeter(name string, r Registry) Meter { + c := NewMeter() + if nil == r { + r = DefaultRegistry + } + r.Register(name, c) + return c +} + +// MeterSnapshot is a read-only copy of another Meter. +type MeterSnapshot struct { + count int64 + rate1, rate5, rate15, rateMean uint64 +} + +// Count returns the count of events at the time the snapshot was taken. +func (m *MeterSnapshot) Count() int64 { return m.count } + +// Mark panics. +func (*MeterSnapshot) Mark(n int64) { + panic("Mark called on a MeterSnapshot") +} + +// Rate1 returns the one-minute moving average rate of events per second at the +// time the snapshot was taken. +func (m *MeterSnapshot) Rate1() float64 { return math.Float64frombits(m.rate1) } + +// Rate5 returns the five-minute moving average rate of events per second at +// the time the snapshot was taken. +func (m *MeterSnapshot) Rate5() float64 { return math.Float64frombits(m.rate5) } + +// Rate15 returns the fifteen-minute moving average rate of events per second +// at the time the snapshot was taken. +func (m *MeterSnapshot) Rate15() float64 { return math.Float64frombits(m.rate15) } + +// RateMean returns the meter's mean rate of events per second at the time the +// snapshot was taken. +func (m *MeterSnapshot) RateMean() float64 { return math.Float64frombits(m.rateMean) } + +// Snapshot returns the snapshot. +func (m *MeterSnapshot) Snapshot() Meter { return m } + +// Stop is a no-op. +func (m *MeterSnapshot) Stop() {} + +// NilMeter is a no-op Meter. +type NilMeter struct{} + +// Count is a no-op. +func (NilMeter) Count() int64 { return 0 } + +// Mark is a no-op. +func (NilMeter) Mark(n int64) {} + +// Rate1 is a no-op. +func (NilMeter) Rate1() float64 { return 0.0 } + +// Rate5 is a no-op. +func (NilMeter) Rate5() float64 { return 0.0 } + +// Rate15is a no-op. +func (NilMeter) Rate15() float64 { return 0.0 } + +// RateMean is a no-op. +func (NilMeter) RateMean() float64 { return 0.0 } + +// Snapshot is a no-op. +func (NilMeter) Snapshot() Meter { return NilMeter{} } + +// Stop is a no-op. +func (NilMeter) Stop() {} + +// StandardMeter is the standard implementation of a Meter. +type StandardMeter struct { + snapshot *MeterSnapshot + a1, a5, a15 EWMA + startTime time.Time + stopped uint32 +} + +func newStandardMeter() *StandardMeter { + return &StandardMeter{ + snapshot: &MeterSnapshot{}, + a1: NewEWMA1(), + a5: NewEWMA5(), + a15: NewEWMA15(), + startTime: time.Now(), + } +} + +// Stop stops the meter, Mark() will be a no-op if you use it after being stopped. +func (m *StandardMeter) Stop() { + if atomic.CompareAndSwapUint32(&m.stopped, 0, 1) { + arbiter.Lock() + delete(arbiter.meters, m) + arbiter.Unlock() + } +} + +// Count returns the number of events recorded. +func (m *StandardMeter) Count() int64 { + return atomic.LoadInt64(&m.snapshot.count) +} + +// Mark records the occurance of n events. +func (m *StandardMeter) Mark(n int64) { + if atomic.LoadUint32(&m.stopped) == 1 { + return + } + + atomic.AddInt64(&m.snapshot.count, n) + + m.a1.Update(n) + m.a5.Update(n) + m.a15.Update(n) + m.updateSnapshot() +} + +// Rate1 returns the one-minute moving average rate of events per second. +func (m *StandardMeter) Rate1() float64 { + return math.Float64frombits(atomic.LoadUint64(&m.snapshot.rate1)) +} + +// Rate5 returns the five-minute moving average rate of events per second. +func (m *StandardMeter) Rate5() float64 { + return math.Float64frombits(atomic.LoadUint64(&m.snapshot.rate5)) +} + +// Rate15 returns the fifteen-minute moving average rate of events per second. +func (m *StandardMeter) Rate15() float64 { + return math.Float64frombits(atomic.LoadUint64(&m.snapshot.rate15)) +} + +// RateMean returns the meter's mean rate of events per second. +func (m *StandardMeter) RateMean() float64 { + return math.Float64frombits(atomic.LoadUint64(&m.snapshot.rateMean)) +} + +// Snapshot returns a read-only copy of the meter. +func (m *StandardMeter) Snapshot() Meter { + copiedSnapshot := MeterSnapshot{ + count: atomic.LoadInt64(&m.snapshot.count), + rate1: atomic.LoadUint64(&m.snapshot.rate1), + rate5: atomic.LoadUint64(&m.snapshot.rate5), + rate15: atomic.LoadUint64(&m.snapshot.rate15), + rateMean: atomic.LoadUint64(&m.snapshot.rateMean), + } + return &copiedSnapshot +} + +func (m *StandardMeter) updateSnapshot() { + rate1 := math.Float64bits(m.a1.Rate()) + rate5 := math.Float64bits(m.a5.Rate()) + rate15 := math.Float64bits(m.a15.Rate()) + rateMean := math.Float64bits(float64(m.Count()) / time.Since(m.startTime).Seconds()) + + atomic.StoreUint64(&m.snapshot.rate1, rate1) + atomic.StoreUint64(&m.snapshot.rate5, rate5) + atomic.StoreUint64(&m.snapshot.rate15, rate15) + atomic.StoreUint64(&m.snapshot.rateMean, rateMean) +} + +func (m *StandardMeter) tick() { + m.a1.Tick() + m.a5.Tick() + m.a15.Tick() + m.updateSnapshot() +} + +// meterArbiter ticks meters every 5s from a single goroutine. +// meters are references in a set for future stopping. +type meterArbiter struct { + sync.RWMutex + started bool + meters map[*StandardMeter]struct{} + ticker *time.Ticker +} + +var arbiter = meterArbiter{ticker: time.NewTicker(5e9), meters: make(map[*StandardMeter]struct{})} + +// Ticks meters on the scheduled interval +func (ma *meterArbiter) tick() { + for { + select { + case <-ma.ticker.C: + ma.tickMeters() + } + } +} + +func (ma *meterArbiter) tickMeters() { + ma.RLock() + defer ma.RUnlock() + for meter := range ma.meters { + meter.tick() + } +} diff --git a/vendor/github.com/rcrowley/go-metrics/metrics.go b/vendor/github.com/rcrowley/go-metrics/metrics.go new file mode 100644 index 000000000000..b97a49ed123e --- /dev/null +++ b/vendor/github.com/rcrowley/go-metrics/metrics.go @@ -0,0 +1,13 @@ +// Go port of Coda Hale's Metrics library +// +// +// +// Coda Hale's original work: +package metrics + +// UseNilMetrics is checked by the constructor functions for all of the +// standard metrics. If it is true, the metric returned is a stub. +// +// This global kill-switch helps quantify the observer effect and makes +// for less cluttered pprof profiles. +var UseNilMetrics bool = false diff --git a/vendor/github.com/rcrowley/go-metrics/opentsdb.go b/vendor/github.com/rcrowley/go-metrics/opentsdb.go new file mode 100644 index 000000000000..266b6c93d21d --- /dev/null +++ b/vendor/github.com/rcrowley/go-metrics/opentsdb.go @@ -0,0 +1,119 @@ +package metrics + +import ( + "bufio" + "fmt" + "log" + "net" + "os" + "strings" + "time" +) + +var shortHostName string = "" + +// OpenTSDBConfig provides a container with configuration parameters for +// the OpenTSDB exporter +type OpenTSDBConfig struct { + Addr *net.TCPAddr // Network address to connect to + Registry Registry // Registry to be exported + FlushInterval time.Duration // Flush interval + DurationUnit time.Duration // Time conversion unit for durations + Prefix string // Prefix to be prepended to metric names +} + +// OpenTSDB is a blocking exporter function which reports metrics in r +// to a TSDB server located at addr, flushing them every d duration +// and prepending metric names with prefix. +func OpenTSDB(r Registry, d time.Duration, prefix string, addr *net.TCPAddr) { + OpenTSDBWithConfig(OpenTSDBConfig{ + Addr: addr, + Registry: r, + FlushInterval: d, + DurationUnit: time.Nanosecond, + Prefix: prefix, + }) +} + +// OpenTSDBWithConfig is a blocking exporter function just like OpenTSDB, +// but it takes a OpenTSDBConfig instead. +func OpenTSDBWithConfig(c OpenTSDBConfig) { + for _ = range time.Tick(c.FlushInterval) { + if err := openTSDB(&c); nil != err { + log.Println(err) + } + } +} + +func getShortHostname() string { + if shortHostName == "" { + host, _ := os.Hostname() + if index := strings.Index(host, "."); index > 0 { + shortHostName = host[:index] + } else { + shortHostName = host + } + } + return shortHostName +} + +func openTSDB(c *OpenTSDBConfig) error { + shortHostname := getShortHostname() + now := time.Now().Unix() + du := float64(c.DurationUnit) + conn, err := net.DialTCP("tcp", nil, c.Addr) + if nil != err { + return err + } + defer conn.Close() + w := bufio.NewWriter(conn) + c.Registry.Each(func(name string, i interface{}) { + switch metric := i.(type) { + case Counter: + fmt.Fprintf(w, "put %s.%s.count %d %d host=%s\n", c.Prefix, name, now, metric.Count(), shortHostname) + case Gauge: + fmt.Fprintf(w, "put %s.%s.value %d %d host=%s\n", c.Prefix, name, now, metric.Value(), shortHostname) + case GaugeFloat64: + fmt.Fprintf(w, "put %s.%s.value %d %f host=%s\n", c.Prefix, name, now, metric.Value(), shortHostname) + case Histogram: + h := metric.Snapshot() + ps := h.Percentiles([]float64{0.5, 0.75, 0.95, 0.99, 0.999}) + fmt.Fprintf(w, "put %s.%s.count %d %d host=%s\n", c.Prefix, name, now, h.Count(), shortHostname) + fmt.Fprintf(w, "put %s.%s.min %d %d host=%s\n", c.Prefix, name, now, h.Min(), shortHostname) + fmt.Fprintf(w, "put %s.%s.max %d %d host=%s\n", c.Prefix, name, now, h.Max(), shortHostname) + fmt.Fprintf(w, "put %s.%s.mean %d %.2f host=%s\n", c.Prefix, name, now, h.Mean(), shortHostname) + fmt.Fprintf(w, "put %s.%s.std-dev %d %.2f host=%s\n", c.Prefix, name, now, h.StdDev(), shortHostname) + fmt.Fprintf(w, "put %s.%s.50-percentile %d %.2f host=%s\n", c.Prefix, name, now, ps[0], shortHostname) + fmt.Fprintf(w, "put %s.%s.75-percentile %d %.2f host=%s\n", c.Prefix, name, now, ps[1], shortHostname) + fmt.Fprintf(w, "put %s.%s.95-percentile %d %.2f host=%s\n", c.Prefix, name, now, ps[2], shortHostname) + fmt.Fprintf(w, "put %s.%s.99-percentile %d %.2f host=%s\n", c.Prefix, name, now, ps[3], shortHostname) + fmt.Fprintf(w, "put %s.%s.999-percentile %d %.2f host=%s\n", c.Prefix, name, now, ps[4], shortHostname) + case Meter: + m := metric.Snapshot() + fmt.Fprintf(w, "put %s.%s.count %d %d host=%s\n", c.Prefix, name, now, m.Count(), shortHostname) + fmt.Fprintf(w, "put %s.%s.one-minute %d %.2f host=%s\n", c.Prefix, name, now, m.Rate1(), shortHostname) + fmt.Fprintf(w, "put %s.%s.five-minute %d %.2f host=%s\n", c.Prefix, name, now, m.Rate5(), shortHostname) + fmt.Fprintf(w, "put %s.%s.fifteen-minute %d %.2f host=%s\n", c.Prefix, name, now, m.Rate15(), shortHostname) + fmt.Fprintf(w, "put %s.%s.mean %d %.2f host=%s\n", c.Prefix, name, now, m.RateMean(), shortHostname) + case Timer: + t := metric.Snapshot() + ps := t.Percentiles([]float64{0.5, 0.75, 0.95, 0.99, 0.999}) + fmt.Fprintf(w, "put %s.%s.count %d %d host=%s\n", c.Prefix, name, now, t.Count(), shortHostname) + fmt.Fprintf(w, "put %s.%s.min %d %d host=%s\n", c.Prefix, name, now, t.Min()/int64(du), shortHostname) + fmt.Fprintf(w, "put %s.%s.max %d %d host=%s\n", c.Prefix, name, now, t.Max()/int64(du), shortHostname) + fmt.Fprintf(w, "put %s.%s.mean %d %.2f host=%s\n", c.Prefix, name, now, t.Mean()/du, shortHostname) + fmt.Fprintf(w, "put %s.%s.std-dev %d %.2f host=%s\n", c.Prefix, name, now, t.StdDev()/du, shortHostname) + fmt.Fprintf(w, "put %s.%s.50-percentile %d %.2f host=%s\n", c.Prefix, name, now, ps[0]/du, shortHostname) + fmt.Fprintf(w, "put %s.%s.75-percentile %d %.2f host=%s\n", c.Prefix, name, now, ps[1]/du, shortHostname) + fmt.Fprintf(w, "put %s.%s.95-percentile %d %.2f host=%s\n", c.Prefix, name, now, ps[2]/du, shortHostname) + fmt.Fprintf(w, "put %s.%s.99-percentile %d %.2f host=%s\n", c.Prefix, name, now, ps[3]/du, shortHostname) + fmt.Fprintf(w, "put %s.%s.999-percentile %d %.2f host=%s\n", c.Prefix, name, now, ps[4]/du, shortHostname) + fmt.Fprintf(w, "put %s.%s.one-minute %d %.2f host=%s\n", c.Prefix, name, now, t.Rate1(), shortHostname) + fmt.Fprintf(w, "put %s.%s.five-minute %d %.2f host=%s\n", c.Prefix, name, now, t.Rate5(), shortHostname) + fmt.Fprintf(w, "put %s.%s.fifteen-minute %d %.2f host=%s\n", c.Prefix, name, now, t.Rate15(), shortHostname) + fmt.Fprintf(w, "put %s.%s.mean-rate %d %.2f host=%s\n", c.Prefix, name, now, t.RateMean(), shortHostname) + } + w.Flush() + }) + return nil +} diff --git a/vendor/github.com/rcrowley/go-metrics/registry.go b/vendor/github.com/rcrowley/go-metrics/registry.go new file mode 100644 index 000000000000..a8e67228a45b --- /dev/null +++ b/vendor/github.com/rcrowley/go-metrics/registry.go @@ -0,0 +1,373 @@ +package metrics + +import ( + "fmt" + "reflect" + "strings" + "sync" +) + +// DuplicateMetric is the error returned by Registry.Register when a metric +// already exists. If you mean to Register that metric you must first +// Unregister the existing metric. +type DuplicateMetric string + +func (err DuplicateMetric) Error() string { + return fmt.Sprintf("duplicate metric: %s", string(err)) +} + +// A Registry holds references to a set of metrics by name and can iterate +// over them, calling callback functions provided by the user. +// +// This is an interface so as to encourage other structs to implement +// the Registry API as appropriate. +type Registry interface { + + // Call the given function for each registered metric. + Each(func(string, interface{})) + + // Get the metric by the given name or nil if none is registered. + Get(string) interface{} + + // GetAll metrics in the Registry. + GetAll() map[string]map[string]interface{} + + // Gets an existing metric or registers the given one. + // The interface can be the metric to register if not found in registry, + // or a function returning the metric for lazy instantiation. + GetOrRegister(string, interface{}) interface{} + + // Register the given metric under the given name. + Register(string, interface{}) error + + // Run all registered healthchecks. + RunHealthchecks() + + // Unregister the metric with the given name. + Unregister(string) + + // Unregister all metrics. (Mostly for testing.) + UnregisterAll() +} + +// The standard implementation of a Registry is a mutex-protected map +// of names to metrics. +type StandardRegistry struct { + metrics map[string]interface{} + mutex sync.RWMutex +} + +// Create a new registry. +func NewRegistry() Registry { + return &StandardRegistry{metrics: make(map[string]interface{})} +} + +// Call the given function for each registered metric. +func (r *StandardRegistry) Each(f func(string, interface{})) { + metrics := r.registered() + for i := range metrics { + kv := &metrics[i] + f(kv.name, kv.value) + } +} + +// Get the metric by the given name or nil if none is registered. +func (r *StandardRegistry) Get(name string) interface{} { + r.mutex.RLock() + defer r.mutex.RUnlock() + return r.metrics[name] +} + +// Gets an existing metric or creates and registers a new one. Threadsafe +// alternative to calling Get and Register on failure. +// The interface can be the metric to register if not found in registry, +// or a function returning the metric for lazy instantiation. +func (r *StandardRegistry) GetOrRegister(name string, i interface{}) interface{} { + // access the read lock first which should be re-entrant + r.mutex.RLock() + metric, ok := r.metrics[name] + r.mutex.RUnlock() + if ok { + return metric + } + + // only take the write lock if we'll be modifying the metrics map + r.mutex.Lock() + defer r.mutex.Unlock() + if metric, ok := r.metrics[name]; ok { + return metric + } + if v := reflect.ValueOf(i); v.Kind() == reflect.Func { + i = v.Call(nil)[0].Interface() + } + r.register(name, i) + return i +} + +// Register the given metric under the given name. Returns a DuplicateMetric +// if a metric by the given name is already registered. +func (r *StandardRegistry) Register(name string, i interface{}) error { + r.mutex.Lock() + defer r.mutex.Unlock() + return r.register(name, i) +} + +// Run all registered healthchecks. +func (r *StandardRegistry) RunHealthchecks() { + r.mutex.RLock() + defer r.mutex.RUnlock() + for _, i := range r.metrics { + if h, ok := i.(Healthcheck); ok { + h.Check() + } + } +} + +// GetAll metrics in the Registry +func (r *StandardRegistry) GetAll() map[string]map[string]interface{} { + data := make(map[string]map[string]interface{}) + r.Each(func(name string, i interface{}) { + values := make(map[string]interface{}) + switch metric := i.(type) { + case Counter: + values["count"] = metric.Count() + case Gauge: + values["value"] = metric.Value() + case GaugeFloat64: + values["value"] = metric.Value() + case Healthcheck: + values["error"] = nil + metric.Check() + if err := metric.Error(); nil != err { + values["error"] = metric.Error().Error() + } + case Histogram: + h := metric.Snapshot() + ps := h.Percentiles([]float64{0.5, 0.75, 0.95, 0.99, 0.999}) + values["count"] = h.Count() + values["min"] = h.Min() + values["max"] = h.Max() + values["mean"] = h.Mean() + values["stddev"] = h.StdDev() + values["median"] = ps[0] + values["75%"] = ps[1] + values["95%"] = ps[2] + values["99%"] = ps[3] + values["99.9%"] = ps[4] + case Meter: + m := metric.Snapshot() + values["count"] = m.Count() + values["1m.rate"] = m.Rate1() + values["5m.rate"] = m.Rate5() + values["15m.rate"] = m.Rate15() + values["mean.rate"] = m.RateMean() + case Timer: + t := metric.Snapshot() + ps := t.Percentiles([]float64{0.5, 0.75, 0.95, 0.99, 0.999}) + values["count"] = t.Count() + values["min"] = t.Min() + values["max"] = t.Max() + values["mean"] = t.Mean() + values["stddev"] = t.StdDev() + values["median"] = ps[0] + values["75%"] = ps[1] + values["95%"] = ps[2] + values["99%"] = ps[3] + values["99.9%"] = ps[4] + values["1m.rate"] = t.Rate1() + values["5m.rate"] = t.Rate5() + values["15m.rate"] = t.Rate15() + values["mean.rate"] = t.RateMean() + } + data[name] = values + }) + return data +} + +// Unregister the metric with the given name. +func (r *StandardRegistry) Unregister(name string) { + r.mutex.Lock() + defer r.mutex.Unlock() + r.stop(name) + delete(r.metrics, name) +} + +// Unregister all metrics. (Mostly for testing.) +func (r *StandardRegistry) UnregisterAll() { + r.mutex.Lock() + defer r.mutex.Unlock() + for name, _ := range r.metrics { + r.stop(name) + delete(r.metrics, name) + } +} + +func (r *StandardRegistry) register(name string, i interface{}) error { + if _, ok := r.metrics[name]; ok { + return DuplicateMetric(name) + } + switch i.(type) { + case Counter, Gauge, GaugeFloat64, Healthcheck, Histogram, Meter, Timer: + r.metrics[name] = i + } + return nil +} + +type metricKV struct { + name string + value interface{} +} + +func (r *StandardRegistry) registered() []metricKV { + r.mutex.RLock() + defer r.mutex.RUnlock() + metrics := make([]metricKV, 0, len(r.metrics)) + for name, i := range r.metrics { + metrics = append(metrics, metricKV{ + name: name, + value: i, + }) + } + return metrics +} + +func (r *StandardRegistry) stop(name string) { + if i, ok := r.metrics[name]; ok { + if s, ok := i.(Stoppable); ok { + s.Stop() + } + } +} + +// Stoppable defines the metrics which has to be stopped. +type Stoppable interface { + Stop() +} + +type PrefixedRegistry struct { + underlying Registry + prefix string +} + +func NewPrefixedRegistry(prefix string) Registry { + return &PrefixedRegistry{ + underlying: NewRegistry(), + prefix: prefix, + } +} + +func NewPrefixedChildRegistry(parent Registry, prefix string) Registry { + return &PrefixedRegistry{ + underlying: parent, + prefix: prefix, + } +} + +// Call the given function for each registered metric. +func (r *PrefixedRegistry) Each(fn func(string, interface{})) { + wrappedFn := func(prefix string) func(string, interface{}) { + return func(name string, iface interface{}) { + if strings.HasPrefix(name, prefix) { + fn(name, iface) + } else { + return + } + } + } + + baseRegistry, prefix := findPrefix(r, "") + baseRegistry.Each(wrappedFn(prefix)) +} + +func findPrefix(registry Registry, prefix string) (Registry, string) { + switch r := registry.(type) { + case *PrefixedRegistry: + return findPrefix(r.underlying, r.prefix+prefix) + case *StandardRegistry: + return r, prefix + } + return nil, "" +} + +// Get the metric by the given name or nil if none is registered. +func (r *PrefixedRegistry) Get(name string) interface{} { + realName := r.prefix + name + return r.underlying.Get(realName) +} + +// Gets an existing metric or registers the given one. +// The interface can be the metric to register if not found in registry, +// or a function returning the metric for lazy instantiation. +func (r *PrefixedRegistry) GetOrRegister(name string, metric interface{}) interface{} { + realName := r.prefix + name + return r.underlying.GetOrRegister(realName, metric) +} + +// Register the given metric under the given name. The name will be prefixed. +func (r *PrefixedRegistry) Register(name string, metric interface{}) error { + realName := r.prefix + name + return r.underlying.Register(realName, metric) +} + +// Run all registered healthchecks. +func (r *PrefixedRegistry) RunHealthchecks() { + r.underlying.RunHealthchecks() +} + +// GetAll metrics in the Registry +func (r *PrefixedRegistry) GetAll() map[string]map[string]interface{} { + return r.underlying.GetAll() +} + +// Unregister the metric with the given name. The name will be prefixed. +func (r *PrefixedRegistry) Unregister(name string) { + realName := r.prefix + name + r.underlying.Unregister(realName) +} + +// Unregister all metrics. (Mostly for testing.) +func (r *PrefixedRegistry) UnregisterAll() { + r.underlying.UnregisterAll() +} + +var DefaultRegistry Registry = NewRegistry() + +// Call the given function for each registered metric. +func Each(f func(string, interface{})) { + DefaultRegistry.Each(f) +} + +// Get the metric by the given name or nil if none is registered. +func Get(name string) interface{} { + return DefaultRegistry.Get(name) +} + +// Gets an existing metric or creates and registers a new one. Threadsafe +// alternative to calling Get and Register on failure. +func GetOrRegister(name string, i interface{}) interface{} { + return DefaultRegistry.GetOrRegister(name, i) +} + +// Register the given metric under the given name. Returns a DuplicateMetric +// if a metric by the given name is already registered. +func Register(name string, i interface{}) error { + return DefaultRegistry.Register(name, i) +} + +// Register the given metric under the given name. Panics if a metric by the +// given name is already registered. +func MustRegister(name string, i interface{}) { + if err := Register(name, i); err != nil { + panic(err) + } +} + +// Run all registered healthchecks. +func RunHealthchecks() { + DefaultRegistry.RunHealthchecks() +} + +// Unregister the metric with the given name. +func Unregister(name string) { + DefaultRegistry.Unregister(name) +} diff --git a/vendor/github.com/rcrowley/go-metrics/runtime.go b/vendor/github.com/rcrowley/go-metrics/runtime.go new file mode 100644 index 000000000000..4047ab3d3732 --- /dev/null +++ b/vendor/github.com/rcrowley/go-metrics/runtime.go @@ -0,0 +1,216 @@ +package metrics + +import ( + "runtime" + "runtime/pprof" + "sync" + "time" +) + +var ( + memStats runtime.MemStats + runtimeMetrics struct { + MemStats struct { + Alloc Gauge + BuckHashSys Gauge + DebugGC Gauge + EnableGC Gauge + Frees Gauge + HeapAlloc Gauge + HeapIdle Gauge + HeapInuse Gauge + HeapObjects Gauge + HeapReleased Gauge + HeapSys Gauge + LastGC Gauge + Lookups Gauge + Mallocs Gauge + MCacheInuse Gauge + MCacheSys Gauge + MSpanInuse Gauge + MSpanSys Gauge + NextGC Gauge + NumGC Gauge + GCCPUFraction GaugeFloat64 + PauseNs Histogram + PauseTotalNs Gauge + StackInuse Gauge + StackSys Gauge + Sys Gauge + TotalAlloc Gauge + } + NumCgoCall Gauge + NumGoroutine Gauge + NumThread Gauge + ReadMemStats Timer + } + frees uint64 + lookups uint64 + mallocs uint64 + numGC uint32 + numCgoCalls int64 + + threadCreateProfile = pprof.Lookup("threadcreate") + registerRuntimeMetricsOnce = sync.Once{} +) + +// Capture new values for the Go runtime statistics exported in +// runtime.MemStats. This is designed to be called as a goroutine. +func CaptureRuntimeMemStats(r Registry, d time.Duration) { + for _ = range time.Tick(d) { + CaptureRuntimeMemStatsOnce(r) + } +} + +// Capture new values for the Go runtime statistics exported in +// runtime.MemStats. This is designed to be called in a background +// goroutine. Giving a registry which has not been given to +// RegisterRuntimeMemStats will panic. +// +// Be very careful with this because runtime.ReadMemStats calls the C +// functions runtime·semacquire(&runtime·worldsema) and runtime·stoptheworld() +// and that last one does what it says on the tin. +func CaptureRuntimeMemStatsOnce(r Registry) { + t := time.Now() + runtime.ReadMemStats(&memStats) // This takes 50-200us. + runtimeMetrics.ReadMemStats.UpdateSince(t) + + runtimeMetrics.MemStats.Alloc.Update(int64(memStats.Alloc)) + runtimeMetrics.MemStats.BuckHashSys.Update(int64(memStats.BuckHashSys)) + if memStats.DebugGC { + runtimeMetrics.MemStats.DebugGC.Update(1) + } else { + runtimeMetrics.MemStats.DebugGC.Update(0) + } + if memStats.EnableGC { + runtimeMetrics.MemStats.EnableGC.Update(1) + } else { + runtimeMetrics.MemStats.EnableGC.Update(0) + } + + runtimeMetrics.MemStats.Frees.Update(int64(memStats.Frees - frees)) + runtimeMetrics.MemStats.HeapAlloc.Update(int64(memStats.HeapAlloc)) + runtimeMetrics.MemStats.HeapIdle.Update(int64(memStats.HeapIdle)) + runtimeMetrics.MemStats.HeapInuse.Update(int64(memStats.HeapInuse)) + runtimeMetrics.MemStats.HeapObjects.Update(int64(memStats.HeapObjects)) + runtimeMetrics.MemStats.HeapReleased.Update(int64(memStats.HeapReleased)) + runtimeMetrics.MemStats.HeapSys.Update(int64(memStats.HeapSys)) + runtimeMetrics.MemStats.LastGC.Update(int64(memStats.LastGC)) + runtimeMetrics.MemStats.Lookups.Update(int64(memStats.Lookups - lookups)) + runtimeMetrics.MemStats.Mallocs.Update(int64(memStats.Mallocs - mallocs)) + runtimeMetrics.MemStats.MCacheInuse.Update(int64(memStats.MCacheInuse)) + runtimeMetrics.MemStats.MCacheSys.Update(int64(memStats.MCacheSys)) + runtimeMetrics.MemStats.MSpanInuse.Update(int64(memStats.MSpanInuse)) + runtimeMetrics.MemStats.MSpanSys.Update(int64(memStats.MSpanSys)) + runtimeMetrics.MemStats.NextGC.Update(int64(memStats.NextGC)) + runtimeMetrics.MemStats.NumGC.Update(int64(memStats.NumGC - numGC)) + runtimeMetrics.MemStats.GCCPUFraction.Update(gcCPUFraction(&memStats)) + + // + i := numGC % uint32(len(memStats.PauseNs)) + ii := memStats.NumGC % uint32(len(memStats.PauseNs)) + if memStats.NumGC-numGC >= uint32(len(memStats.PauseNs)) { + for i = 0; i < uint32(len(memStats.PauseNs)); i++ { + runtimeMetrics.MemStats.PauseNs.Update(int64(memStats.PauseNs[i])) + } + } else { + if i > ii { + for ; i < uint32(len(memStats.PauseNs)); i++ { + runtimeMetrics.MemStats.PauseNs.Update(int64(memStats.PauseNs[i])) + } + i = 0 + } + for ; i < ii; i++ { + runtimeMetrics.MemStats.PauseNs.Update(int64(memStats.PauseNs[i])) + } + } + frees = memStats.Frees + lookups = memStats.Lookups + mallocs = memStats.Mallocs + numGC = memStats.NumGC + + runtimeMetrics.MemStats.PauseTotalNs.Update(int64(memStats.PauseTotalNs)) + runtimeMetrics.MemStats.StackInuse.Update(int64(memStats.StackInuse)) + runtimeMetrics.MemStats.StackSys.Update(int64(memStats.StackSys)) + runtimeMetrics.MemStats.Sys.Update(int64(memStats.Sys)) + runtimeMetrics.MemStats.TotalAlloc.Update(int64(memStats.TotalAlloc)) + + currentNumCgoCalls := numCgoCall() + runtimeMetrics.NumCgoCall.Update(currentNumCgoCalls - numCgoCalls) + numCgoCalls = currentNumCgoCalls + + runtimeMetrics.NumGoroutine.Update(int64(runtime.NumGoroutine())) + + runtimeMetrics.NumThread.Update(int64(threadCreateProfile.Count())) +} + +// Register runtimeMetrics for the Go runtime statistics exported in runtime and +// specifically runtime.MemStats. The runtimeMetrics are named by their +// fully-qualified Go symbols, i.e. runtime.MemStats.Alloc. +func RegisterRuntimeMemStats(r Registry) { + registerRuntimeMetricsOnce.Do(func() { + runtimeMetrics.MemStats.Alloc = NewGauge() + runtimeMetrics.MemStats.BuckHashSys = NewGauge() + runtimeMetrics.MemStats.DebugGC = NewGauge() + runtimeMetrics.MemStats.EnableGC = NewGauge() + runtimeMetrics.MemStats.Frees = NewGauge() + runtimeMetrics.MemStats.HeapAlloc = NewGauge() + runtimeMetrics.MemStats.HeapIdle = NewGauge() + runtimeMetrics.MemStats.HeapInuse = NewGauge() + runtimeMetrics.MemStats.HeapObjects = NewGauge() + runtimeMetrics.MemStats.HeapReleased = NewGauge() + runtimeMetrics.MemStats.HeapSys = NewGauge() + runtimeMetrics.MemStats.LastGC = NewGauge() + runtimeMetrics.MemStats.Lookups = NewGauge() + runtimeMetrics.MemStats.Mallocs = NewGauge() + runtimeMetrics.MemStats.MCacheInuse = NewGauge() + runtimeMetrics.MemStats.MCacheSys = NewGauge() + runtimeMetrics.MemStats.MSpanInuse = NewGauge() + runtimeMetrics.MemStats.MSpanSys = NewGauge() + runtimeMetrics.MemStats.NextGC = NewGauge() + runtimeMetrics.MemStats.NumGC = NewGauge() + runtimeMetrics.MemStats.GCCPUFraction = NewGaugeFloat64() + runtimeMetrics.MemStats.PauseNs = NewHistogram(NewExpDecaySample(1028, 0.015)) + runtimeMetrics.MemStats.PauseTotalNs = NewGauge() + runtimeMetrics.MemStats.StackInuse = NewGauge() + runtimeMetrics.MemStats.StackSys = NewGauge() + runtimeMetrics.MemStats.Sys = NewGauge() + runtimeMetrics.MemStats.TotalAlloc = NewGauge() + runtimeMetrics.NumCgoCall = NewGauge() + runtimeMetrics.NumGoroutine = NewGauge() + runtimeMetrics.NumThread = NewGauge() + runtimeMetrics.ReadMemStats = NewTimer() + + r.Register("runtime.MemStats.Alloc", runtimeMetrics.MemStats.Alloc) + r.Register("runtime.MemStats.BuckHashSys", runtimeMetrics.MemStats.BuckHashSys) + r.Register("runtime.MemStats.DebugGC", runtimeMetrics.MemStats.DebugGC) + r.Register("runtime.MemStats.EnableGC", runtimeMetrics.MemStats.EnableGC) + r.Register("runtime.MemStats.Frees", runtimeMetrics.MemStats.Frees) + r.Register("runtime.MemStats.HeapAlloc", runtimeMetrics.MemStats.HeapAlloc) + r.Register("runtime.MemStats.HeapIdle", runtimeMetrics.MemStats.HeapIdle) + r.Register("runtime.MemStats.HeapInuse", runtimeMetrics.MemStats.HeapInuse) + r.Register("runtime.MemStats.HeapObjects", runtimeMetrics.MemStats.HeapObjects) + r.Register("runtime.MemStats.HeapReleased", runtimeMetrics.MemStats.HeapReleased) + r.Register("runtime.MemStats.HeapSys", runtimeMetrics.MemStats.HeapSys) + r.Register("runtime.MemStats.LastGC", runtimeMetrics.MemStats.LastGC) + r.Register("runtime.MemStats.Lookups", runtimeMetrics.MemStats.Lookups) + r.Register("runtime.MemStats.Mallocs", runtimeMetrics.MemStats.Mallocs) + r.Register("runtime.MemStats.MCacheInuse", runtimeMetrics.MemStats.MCacheInuse) + r.Register("runtime.MemStats.MCacheSys", runtimeMetrics.MemStats.MCacheSys) + r.Register("runtime.MemStats.MSpanInuse", runtimeMetrics.MemStats.MSpanInuse) + r.Register("runtime.MemStats.MSpanSys", runtimeMetrics.MemStats.MSpanSys) + r.Register("runtime.MemStats.NextGC", runtimeMetrics.MemStats.NextGC) + r.Register("runtime.MemStats.NumGC", runtimeMetrics.MemStats.NumGC) + r.Register("runtime.MemStats.GCCPUFraction", runtimeMetrics.MemStats.GCCPUFraction) + r.Register("runtime.MemStats.PauseNs", runtimeMetrics.MemStats.PauseNs) + r.Register("runtime.MemStats.PauseTotalNs", runtimeMetrics.MemStats.PauseTotalNs) + r.Register("runtime.MemStats.StackInuse", runtimeMetrics.MemStats.StackInuse) + r.Register("runtime.MemStats.StackSys", runtimeMetrics.MemStats.StackSys) + r.Register("runtime.MemStats.Sys", runtimeMetrics.MemStats.Sys) + r.Register("runtime.MemStats.TotalAlloc", runtimeMetrics.MemStats.TotalAlloc) + r.Register("runtime.NumCgoCall", runtimeMetrics.NumCgoCall) + r.Register("runtime.NumGoroutine", runtimeMetrics.NumGoroutine) + r.Register("runtime.NumThread", runtimeMetrics.NumThread) + r.Register("runtime.ReadMemStats", runtimeMetrics.ReadMemStats) + }) +} diff --git a/vendor/github.com/rcrowley/go-metrics/runtime_cgo.go b/vendor/github.com/rcrowley/go-metrics/runtime_cgo.go new file mode 100644 index 000000000000..e3391f4e89fa --- /dev/null +++ b/vendor/github.com/rcrowley/go-metrics/runtime_cgo.go @@ -0,0 +1,10 @@ +// +build cgo +// +build !appengine + +package metrics + +import "runtime" + +func numCgoCall() int64 { + return runtime.NumCgoCall() +} diff --git a/vendor/github.com/rcrowley/go-metrics/runtime_gccpufraction.go b/vendor/github.com/rcrowley/go-metrics/runtime_gccpufraction.go new file mode 100644 index 000000000000..ca12c05bac74 --- /dev/null +++ b/vendor/github.com/rcrowley/go-metrics/runtime_gccpufraction.go @@ -0,0 +1,9 @@ +// +build go1.5 + +package metrics + +import "runtime" + +func gcCPUFraction(memStats *runtime.MemStats) float64 { + return memStats.GCCPUFraction +} diff --git a/vendor/github.com/rcrowley/go-metrics/runtime_no_cgo.go b/vendor/github.com/rcrowley/go-metrics/runtime_no_cgo.go new file mode 100644 index 000000000000..616a3b4751be --- /dev/null +++ b/vendor/github.com/rcrowley/go-metrics/runtime_no_cgo.go @@ -0,0 +1,7 @@ +// +build !cgo appengine + +package metrics + +func numCgoCall() int64 { + return 0 +} diff --git a/vendor/github.com/rcrowley/go-metrics/runtime_no_gccpufraction.go b/vendor/github.com/rcrowley/go-metrics/runtime_no_gccpufraction.go new file mode 100644 index 000000000000..be96aa6f1be9 --- /dev/null +++ b/vendor/github.com/rcrowley/go-metrics/runtime_no_gccpufraction.go @@ -0,0 +1,9 @@ +// +build !go1.5 + +package metrics + +import "runtime" + +func gcCPUFraction(memStats *runtime.MemStats) float64 { + return 0 +} diff --git a/vendor/github.com/rcrowley/go-metrics/sample.go b/vendor/github.com/rcrowley/go-metrics/sample.go new file mode 100644 index 000000000000..fecee5ef68ba --- /dev/null +++ b/vendor/github.com/rcrowley/go-metrics/sample.go @@ -0,0 +1,616 @@ +package metrics + +import ( + "math" + "math/rand" + "sort" + "sync" + "time" +) + +const rescaleThreshold = time.Hour + +// Samples maintain a statistically-significant selection of values from +// a stream. +type Sample interface { + Clear() + Count() int64 + Max() int64 + Mean() float64 + Min() int64 + Percentile(float64) float64 + Percentiles([]float64) []float64 + Size() int + Snapshot() Sample + StdDev() float64 + Sum() int64 + Update(int64) + Values() []int64 + Variance() float64 +} + +// ExpDecaySample is an exponentially-decaying sample using a forward-decaying +// priority reservoir. See Cormode et al's "Forward Decay: A Practical Time +// Decay Model for Streaming Systems". +// +// +type ExpDecaySample struct { + alpha float64 + count int64 + mutex sync.Mutex + reservoirSize int + t0, t1 time.Time + values *expDecaySampleHeap +} + +// NewExpDecaySample constructs a new exponentially-decaying sample with the +// given reservoir size and alpha. +func NewExpDecaySample(reservoirSize int, alpha float64) Sample { + if UseNilMetrics { + return NilSample{} + } + s := &ExpDecaySample{ + alpha: alpha, + reservoirSize: reservoirSize, + t0: time.Now(), + values: newExpDecaySampleHeap(reservoirSize), + } + s.t1 = s.t0.Add(rescaleThreshold) + return s +} + +// Clear clears all samples. +func (s *ExpDecaySample) Clear() { + s.mutex.Lock() + defer s.mutex.Unlock() + s.count = 0 + s.t0 = time.Now() + s.t1 = s.t0.Add(rescaleThreshold) + s.values.Clear() +} + +// Count returns the number of samples recorded, which may exceed the +// reservoir size. +func (s *ExpDecaySample) Count() int64 { + s.mutex.Lock() + defer s.mutex.Unlock() + return s.count +} + +// Max returns the maximum value in the sample, which may not be the maximum +// value ever to be part of the sample. +func (s *ExpDecaySample) Max() int64 { + return SampleMax(s.Values()) +} + +// Mean returns the mean of the values in the sample. +func (s *ExpDecaySample) Mean() float64 { + return SampleMean(s.Values()) +} + +// Min returns the minimum value in the sample, which may not be the minimum +// value ever to be part of the sample. +func (s *ExpDecaySample) Min() int64 { + return SampleMin(s.Values()) +} + +// Percentile returns an arbitrary percentile of values in the sample. +func (s *ExpDecaySample) Percentile(p float64) float64 { + return SamplePercentile(s.Values(), p) +} + +// Percentiles returns a slice of arbitrary percentiles of values in the +// sample. +func (s *ExpDecaySample) Percentiles(ps []float64) []float64 { + return SamplePercentiles(s.Values(), ps) +} + +// Size returns the size of the sample, which is at most the reservoir size. +func (s *ExpDecaySample) Size() int { + s.mutex.Lock() + defer s.mutex.Unlock() + return s.values.Size() +} + +// Snapshot returns a read-only copy of the sample. +func (s *ExpDecaySample) Snapshot() Sample { + s.mutex.Lock() + defer s.mutex.Unlock() + vals := s.values.Values() + values := make([]int64, len(vals)) + for i, v := range vals { + values[i] = v.v + } + return &SampleSnapshot{ + count: s.count, + values: values, + } +} + +// StdDev returns the standard deviation of the values in the sample. +func (s *ExpDecaySample) StdDev() float64 { + return SampleStdDev(s.Values()) +} + +// Sum returns the sum of the values in the sample. +func (s *ExpDecaySample) Sum() int64 { + return SampleSum(s.Values()) +} + +// Update samples a new value. +func (s *ExpDecaySample) Update(v int64) { + s.update(time.Now(), v) +} + +// Values returns a copy of the values in the sample. +func (s *ExpDecaySample) Values() []int64 { + s.mutex.Lock() + defer s.mutex.Unlock() + vals := s.values.Values() + values := make([]int64, len(vals)) + for i, v := range vals { + values[i] = v.v + } + return values +} + +// Variance returns the variance of the values in the sample. +func (s *ExpDecaySample) Variance() float64 { + return SampleVariance(s.Values()) +} + +// update samples a new value at a particular timestamp. This is a method all +// its own to facilitate testing. +func (s *ExpDecaySample) update(t time.Time, v int64) { + s.mutex.Lock() + defer s.mutex.Unlock() + s.count++ + if s.values.Size() == s.reservoirSize { + s.values.Pop() + } + s.values.Push(expDecaySample{ + k: math.Exp(t.Sub(s.t0).Seconds()*s.alpha) / rand.Float64(), + v: v, + }) + if t.After(s.t1) { + values := s.values.Values() + t0 := s.t0 + s.values.Clear() + s.t0 = t + s.t1 = s.t0.Add(rescaleThreshold) + for _, v := range values { + v.k = v.k * math.Exp(-s.alpha*s.t0.Sub(t0).Seconds()) + s.values.Push(v) + } + } +} + +// NilSample is a no-op Sample. +type NilSample struct{} + +// Clear is a no-op. +func (NilSample) Clear() {} + +// Count is a no-op. +func (NilSample) Count() int64 { return 0 } + +// Max is a no-op. +func (NilSample) Max() int64 { return 0 } + +// Mean is a no-op. +func (NilSample) Mean() float64 { return 0.0 } + +// Min is a no-op. +func (NilSample) Min() int64 { return 0 } + +// Percentile is a no-op. +func (NilSample) Percentile(p float64) float64 { return 0.0 } + +// Percentiles is a no-op. +func (NilSample) Percentiles(ps []float64) []float64 { + return make([]float64, len(ps)) +} + +// Size is a no-op. +func (NilSample) Size() int { return 0 } + +// Sample is a no-op. +func (NilSample) Snapshot() Sample { return NilSample{} } + +// StdDev is a no-op. +func (NilSample) StdDev() float64 { return 0.0 } + +// Sum is a no-op. +func (NilSample) Sum() int64 { return 0 } + +// Update is a no-op. +func (NilSample) Update(v int64) {} + +// Values is a no-op. +func (NilSample) Values() []int64 { return []int64{} } + +// Variance is a no-op. +func (NilSample) Variance() float64 { return 0.0 } + +// SampleMax returns the maximum value of the slice of int64. +func SampleMax(values []int64) int64 { + if 0 == len(values) { + return 0 + } + var max int64 = math.MinInt64 + for _, v := range values { + if max < v { + max = v + } + } + return max +} + +// SampleMean returns the mean value of the slice of int64. +func SampleMean(values []int64) float64 { + if 0 == len(values) { + return 0.0 + } + return float64(SampleSum(values)) / float64(len(values)) +} + +// SampleMin returns the minimum value of the slice of int64. +func SampleMin(values []int64) int64 { + if 0 == len(values) { + return 0 + } + var min int64 = math.MaxInt64 + for _, v := range values { + if min > v { + min = v + } + } + return min +} + +// SamplePercentiles returns an arbitrary percentile of the slice of int64. +func SamplePercentile(values int64Slice, p float64) float64 { + return SamplePercentiles(values, []float64{p})[0] +} + +// SamplePercentiles returns a slice of arbitrary percentiles of the slice of +// int64. +func SamplePercentiles(values int64Slice, ps []float64) []float64 { + scores := make([]float64, len(ps)) + size := len(values) + if size > 0 { + sort.Sort(values) + for i, p := range ps { + pos := p * float64(size+1) + if pos < 1.0 { + scores[i] = float64(values[0]) + } else if pos >= float64(size) { + scores[i] = float64(values[size-1]) + } else { + lower := float64(values[int(pos)-1]) + upper := float64(values[int(pos)]) + scores[i] = lower + (pos-math.Floor(pos))*(upper-lower) + } + } + } + return scores +} + +// SampleSnapshot is a read-only copy of another Sample. +type SampleSnapshot struct { + count int64 + values []int64 +} + +func NewSampleSnapshot(count int64, values []int64) *SampleSnapshot { + return &SampleSnapshot{ + count: count, + values: values, + } +} + +// Clear panics. +func (*SampleSnapshot) Clear() { + panic("Clear called on a SampleSnapshot") +} + +// Count returns the count of inputs at the time the snapshot was taken. +func (s *SampleSnapshot) Count() int64 { return s.count } + +// Max returns the maximal value at the time the snapshot was taken. +func (s *SampleSnapshot) Max() int64 { return SampleMax(s.values) } + +// Mean returns the mean value at the time the snapshot was taken. +func (s *SampleSnapshot) Mean() float64 { return SampleMean(s.values) } + +// Min returns the minimal value at the time the snapshot was taken. +func (s *SampleSnapshot) Min() int64 { return SampleMin(s.values) } + +// Percentile returns an arbitrary percentile of values at the time the +// snapshot was taken. +func (s *SampleSnapshot) Percentile(p float64) float64 { + return SamplePercentile(s.values, p) +} + +// Percentiles returns a slice of arbitrary percentiles of values at the time +// the snapshot was taken. +func (s *SampleSnapshot) Percentiles(ps []float64) []float64 { + return SamplePercentiles(s.values, ps) +} + +// Size returns the size of the sample at the time the snapshot was taken. +func (s *SampleSnapshot) Size() int { return len(s.values) } + +// Snapshot returns the snapshot. +func (s *SampleSnapshot) Snapshot() Sample { return s } + +// StdDev returns the standard deviation of values at the time the snapshot was +// taken. +func (s *SampleSnapshot) StdDev() float64 { return SampleStdDev(s.values) } + +// Sum returns the sum of values at the time the snapshot was taken. +func (s *SampleSnapshot) Sum() int64 { return SampleSum(s.values) } + +// Update panics. +func (*SampleSnapshot) Update(int64) { + panic("Update called on a SampleSnapshot") +} + +// Values returns a copy of the values in the sample. +func (s *SampleSnapshot) Values() []int64 { + values := make([]int64, len(s.values)) + copy(values, s.values) + return values +} + +// Variance returns the variance of values at the time the snapshot was taken. +func (s *SampleSnapshot) Variance() float64 { return SampleVariance(s.values) } + +// SampleStdDev returns the standard deviation of the slice of int64. +func SampleStdDev(values []int64) float64 { + return math.Sqrt(SampleVariance(values)) +} + +// SampleSum returns the sum of the slice of int64. +func SampleSum(values []int64) int64 { + var sum int64 + for _, v := range values { + sum += v + } + return sum +} + +// SampleVariance returns the variance of the slice of int64. +func SampleVariance(values []int64) float64 { + if 0 == len(values) { + return 0.0 + } + m := SampleMean(values) + var sum float64 + for _, v := range values { + d := float64(v) - m + sum += d * d + } + return sum / float64(len(values)) +} + +// A uniform sample using Vitter's Algorithm R. +// +// +type UniformSample struct { + count int64 + mutex sync.Mutex + reservoirSize int + values []int64 +} + +// NewUniformSample constructs a new uniform sample with the given reservoir +// size. +func NewUniformSample(reservoirSize int) Sample { + if UseNilMetrics { + return NilSample{} + } + return &UniformSample{ + reservoirSize: reservoirSize, + values: make([]int64, 0, reservoirSize), + } +} + +// Clear clears all samples. +func (s *UniformSample) Clear() { + s.mutex.Lock() + defer s.mutex.Unlock() + s.count = 0 + s.values = make([]int64, 0, s.reservoirSize) +} + +// Count returns the number of samples recorded, which may exceed the +// reservoir size. +func (s *UniformSample) Count() int64 { + s.mutex.Lock() + defer s.mutex.Unlock() + return s.count +} + +// Max returns the maximum value in the sample, which may not be the maximum +// value ever to be part of the sample. +func (s *UniformSample) Max() int64 { + s.mutex.Lock() + defer s.mutex.Unlock() + return SampleMax(s.values) +} + +// Mean returns the mean of the values in the sample. +func (s *UniformSample) Mean() float64 { + s.mutex.Lock() + defer s.mutex.Unlock() + return SampleMean(s.values) +} + +// Min returns the minimum value in the sample, which may not be the minimum +// value ever to be part of the sample. +func (s *UniformSample) Min() int64 { + s.mutex.Lock() + defer s.mutex.Unlock() + return SampleMin(s.values) +} + +// Percentile returns an arbitrary percentile of values in the sample. +func (s *UniformSample) Percentile(p float64) float64 { + s.mutex.Lock() + defer s.mutex.Unlock() + return SamplePercentile(s.values, p) +} + +// Percentiles returns a slice of arbitrary percentiles of values in the +// sample. +func (s *UniformSample) Percentiles(ps []float64) []float64 { + s.mutex.Lock() + defer s.mutex.Unlock() + return SamplePercentiles(s.values, ps) +} + +// Size returns the size of the sample, which is at most the reservoir size. +func (s *UniformSample) Size() int { + s.mutex.Lock() + defer s.mutex.Unlock() + return len(s.values) +} + +// Snapshot returns a read-only copy of the sample. +func (s *UniformSample) Snapshot() Sample { + s.mutex.Lock() + defer s.mutex.Unlock() + values := make([]int64, len(s.values)) + copy(values, s.values) + return &SampleSnapshot{ + count: s.count, + values: values, + } +} + +// StdDev returns the standard deviation of the values in the sample. +func (s *UniformSample) StdDev() float64 { + s.mutex.Lock() + defer s.mutex.Unlock() + return SampleStdDev(s.values) +} + +// Sum returns the sum of the values in the sample. +func (s *UniformSample) Sum() int64 { + s.mutex.Lock() + defer s.mutex.Unlock() + return SampleSum(s.values) +} + +// Update samples a new value. +func (s *UniformSample) Update(v int64) { + s.mutex.Lock() + defer s.mutex.Unlock() + s.count++ + if len(s.values) < s.reservoirSize { + s.values = append(s.values, v) + } else { + r := rand.Int63n(s.count) + if r < int64(len(s.values)) { + s.values[int(r)] = v + } + } +} + +// Values returns a copy of the values in the sample. +func (s *UniformSample) Values() []int64 { + s.mutex.Lock() + defer s.mutex.Unlock() + values := make([]int64, len(s.values)) + copy(values, s.values) + return values +} + +// Variance returns the variance of the values in the sample. +func (s *UniformSample) Variance() float64 { + s.mutex.Lock() + defer s.mutex.Unlock() + return SampleVariance(s.values) +} + +// expDecaySample represents an individual sample in a heap. +type expDecaySample struct { + k float64 + v int64 +} + +func newExpDecaySampleHeap(reservoirSize int) *expDecaySampleHeap { + return &expDecaySampleHeap{make([]expDecaySample, 0, reservoirSize)} +} + +// expDecaySampleHeap is a min-heap of expDecaySamples. +// The internal implementation is copied from the standard library's container/heap +type expDecaySampleHeap struct { + s []expDecaySample +} + +func (h *expDecaySampleHeap) Clear() { + h.s = h.s[:0] +} + +func (h *expDecaySampleHeap) Push(s expDecaySample) { + n := len(h.s) + h.s = h.s[0 : n+1] + h.s[n] = s + h.up(n) +} + +func (h *expDecaySampleHeap) Pop() expDecaySample { + n := len(h.s) - 1 + h.s[0], h.s[n] = h.s[n], h.s[0] + h.down(0, n) + + n = len(h.s) + s := h.s[n-1] + h.s = h.s[0 : n-1] + return s +} + +func (h *expDecaySampleHeap) Size() int { + return len(h.s) +} + +func (h *expDecaySampleHeap) Values() []expDecaySample { + return h.s +} + +func (h *expDecaySampleHeap) up(j int) { + for { + i := (j - 1) / 2 // parent + if i == j || !(h.s[j].k < h.s[i].k) { + break + } + h.s[i], h.s[j] = h.s[j], h.s[i] + j = i + } +} + +func (h *expDecaySampleHeap) down(i, n int) { + for { + j1 := 2*i + 1 + if j1 >= n || j1 < 0 { // j1 < 0 after int overflow + break + } + j := j1 // left child + if j2 := j1 + 1; j2 < n && !(h.s[j1].k < h.s[j2].k) { + j = j2 // = 2*i + 2 // right child + } + if !(h.s[j].k < h.s[i].k) { + break + } + h.s[i], h.s[j] = h.s[j], h.s[i] + i = j + } +} + +type int64Slice []int64 + +func (p int64Slice) Len() int { return len(p) } +func (p int64Slice) Less(i, j int) bool { return p[i] < p[j] } +func (p int64Slice) Swap(i, j int) { p[i], p[j] = p[j], p[i] } diff --git a/vendor/github.com/rcrowley/go-metrics/syslog.go b/vendor/github.com/rcrowley/go-metrics/syslog.go new file mode 100644 index 000000000000..693f190855c1 --- /dev/null +++ b/vendor/github.com/rcrowley/go-metrics/syslog.go @@ -0,0 +1,78 @@ +// +build !windows + +package metrics + +import ( + "fmt" + "log/syslog" + "time" +) + +// Output each metric in the given registry to syslog periodically using +// the given syslogger. +func Syslog(r Registry, d time.Duration, w *syslog.Writer) { + for _ = range time.Tick(d) { + r.Each(func(name string, i interface{}) { + switch metric := i.(type) { + case Counter: + w.Info(fmt.Sprintf("counter %s: count: %d", name, metric.Count())) + case Gauge: + w.Info(fmt.Sprintf("gauge %s: value: %d", name, metric.Value())) + case GaugeFloat64: + w.Info(fmt.Sprintf("gauge %s: value: %f", name, metric.Value())) + case Healthcheck: + metric.Check() + w.Info(fmt.Sprintf("healthcheck %s: error: %v", name, metric.Error())) + case Histogram: + h := metric.Snapshot() + ps := h.Percentiles([]float64{0.5, 0.75, 0.95, 0.99, 0.999}) + w.Info(fmt.Sprintf( + "histogram %s: count: %d min: %d max: %d mean: %.2f stddev: %.2f median: %.2f 75%%: %.2f 95%%: %.2f 99%%: %.2f 99.9%%: %.2f", + name, + h.Count(), + h.Min(), + h.Max(), + h.Mean(), + h.StdDev(), + ps[0], + ps[1], + ps[2], + ps[3], + ps[4], + )) + case Meter: + m := metric.Snapshot() + w.Info(fmt.Sprintf( + "meter %s: count: %d 1-min: %.2f 5-min: %.2f 15-min: %.2f mean: %.2f", + name, + m.Count(), + m.Rate1(), + m.Rate5(), + m.Rate15(), + m.RateMean(), + )) + case Timer: + t := metric.Snapshot() + ps := t.Percentiles([]float64{0.5, 0.75, 0.95, 0.99, 0.999}) + w.Info(fmt.Sprintf( + "timer %s: count: %d min: %d max: %d mean: %.2f stddev: %.2f median: %.2f 75%%: %.2f 95%%: %.2f 99%%: %.2f 99.9%%: %.2f 1-min: %.2f 5-min: %.2f 15-min: %.2f mean-rate: %.2f", + name, + t.Count(), + t.Min(), + t.Max(), + t.Mean(), + t.StdDev(), + ps[0], + ps[1], + ps[2], + ps[3], + ps[4], + t.Rate1(), + t.Rate5(), + t.Rate15(), + t.RateMean(), + )) + } + }) + } +} diff --git a/vendor/github.com/rcrowley/go-metrics/timer.go b/vendor/github.com/rcrowley/go-metrics/timer.go new file mode 100644 index 000000000000..d6ec4c6260fc --- /dev/null +++ b/vendor/github.com/rcrowley/go-metrics/timer.go @@ -0,0 +1,329 @@ +package metrics + +import ( + "sync" + "time" +) + +// Timers capture the duration and rate of events. +type Timer interface { + Count() int64 + Max() int64 + Mean() float64 + Min() int64 + Percentile(float64) float64 + Percentiles([]float64) []float64 + Rate1() float64 + Rate5() float64 + Rate15() float64 + RateMean() float64 + Snapshot() Timer + StdDev() float64 + Stop() + Sum() int64 + Time(func()) + Update(time.Duration) + UpdateSince(time.Time) + Variance() float64 +} + +// GetOrRegisterTimer returns an existing Timer or constructs and registers a +// new StandardTimer. +// Be sure to unregister the meter from the registry once it is of no use to +// allow for garbage collection. +func GetOrRegisterTimer(name string, r Registry) Timer { + if nil == r { + r = DefaultRegistry + } + return r.GetOrRegister(name, NewTimer).(Timer) +} + +// NewCustomTimer constructs a new StandardTimer from a Histogram and a Meter. +// Be sure to call Stop() once the timer is of no use to allow for garbage collection. +func NewCustomTimer(h Histogram, m Meter) Timer { + if UseNilMetrics { + return NilTimer{} + } + return &StandardTimer{ + histogram: h, + meter: m, + } +} + +// NewRegisteredTimer constructs and registers a new StandardTimer. +// Be sure to unregister the meter from the registry once it is of no use to +// allow for garbage collection. +func NewRegisteredTimer(name string, r Registry) Timer { + c := NewTimer() + if nil == r { + r = DefaultRegistry + } + r.Register(name, c) + return c +} + +// NewTimer constructs a new StandardTimer using an exponentially-decaying +// sample with the same reservoir size and alpha as UNIX load averages. +// Be sure to call Stop() once the timer is of no use to allow for garbage collection. +func NewTimer() Timer { + if UseNilMetrics { + return NilTimer{} + } + return &StandardTimer{ + histogram: NewHistogram(NewExpDecaySample(1028, 0.015)), + meter: NewMeter(), + } +} + +// NilTimer is a no-op Timer. +type NilTimer struct { + h Histogram + m Meter +} + +// Count is a no-op. +func (NilTimer) Count() int64 { return 0 } + +// Max is a no-op. +func (NilTimer) Max() int64 { return 0 } + +// Mean is a no-op. +func (NilTimer) Mean() float64 { return 0.0 } + +// Min is a no-op. +func (NilTimer) Min() int64 { return 0 } + +// Percentile is a no-op. +func (NilTimer) Percentile(p float64) float64 { return 0.0 } + +// Percentiles is a no-op. +func (NilTimer) Percentiles(ps []float64) []float64 { + return make([]float64, len(ps)) +} + +// Rate1 is a no-op. +func (NilTimer) Rate1() float64 { return 0.0 } + +// Rate5 is a no-op. +func (NilTimer) Rate5() float64 { return 0.0 } + +// Rate15 is a no-op. +func (NilTimer) Rate15() float64 { return 0.0 } + +// RateMean is a no-op. +func (NilTimer) RateMean() float64 { return 0.0 } + +// Snapshot is a no-op. +func (NilTimer) Snapshot() Timer { return NilTimer{} } + +// StdDev is a no-op. +func (NilTimer) StdDev() float64 { return 0.0 } + +// Stop is a no-op. +func (NilTimer) Stop() {} + +// Sum is a no-op. +func (NilTimer) Sum() int64 { return 0 } + +// Time is a no-op. +func (NilTimer) Time(func()) {} + +// Update is a no-op. +func (NilTimer) Update(time.Duration) {} + +// UpdateSince is a no-op. +func (NilTimer) UpdateSince(time.Time) {} + +// Variance is a no-op. +func (NilTimer) Variance() float64 { return 0.0 } + +// StandardTimer is the standard implementation of a Timer and uses a Histogram +// and Meter. +type StandardTimer struct { + histogram Histogram + meter Meter + mutex sync.Mutex +} + +// Count returns the number of events recorded. +func (t *StandardTimer) Count() int64 { + return t.histogram.Count() +} + +// Max returns the maximum value in the sample. +func (t *StandardTimer) Max() int64 { + return t.histogram.Max() +} + +// Mean returns the mean of the values in the sample. +func (t *StandardTimer) Mean() float64 { + return t.histogram.Mean() +} + +// Min returns the minimum value in the sample. +func (t *StandardTimer) Min() int64 { + return t.histogram.Min() +} + +// Percentile returns an arbitrary percentile of the values in the sample. +func (t *StandardTimer) Percentile(p float64) float64 { + return t.histogram.Percentile(p) +} + +// Percentiles returns a slice of arbitrary percentiles of the values in the +// sample. +func (t *StandardTimer) Percentiles(ps []float64) []float64 { + return t.histogram.Percentiles(ps) +} + +// Rate1 returns the one-minute moving average rate of events per second. +func (t *StandardTimer) Rate1() float64 { + return t.meter.Rate1() +} + +// Rate5 returns the five-minute moving average rate of events per second. +func (t *StandardTimer) Rate5() float64 { + return t.meter.Rate5() +} + +// Rate15 returns the fifteen-minute moving average rate of events per second. +func (t *StandardTimer) Rate15() float64 { + return t.meter.Rate15() +} + +// RateMean returns the meter's mean rate of events per second. +func (t *StandardTimer) RateMean() float64 { + return t.meter.RateMean() +} + +// Snapshot returns a read-only copy of the timer. +func (t *StandardTimer) Snapshot() Timer { + t.mutex.Lock() + defer t.mutex.Unlock() + return &TimerSnapshot{ + histogram: t.histogram.Snapshot().(*HistogramSnapshot), + meter: t.meter.Snapshot().(*MeterSnapshot), + } +} + +// StdDev returns the standard deviation of the values in the sample. +func (t *StandardTimer) StdDev() float64 { + return t.histogram.StdDev() +} + +// Stop stops the meter. +func (t *StandardTimer) Stop() { + t.meter.Stop() +} + +// Sum returns the sum in the sample. +func (t *StandardTimer) Sum() int64 { + return t.histogram.Sum() +} + +// Record the duration of the execution of the given function. +func (t *StandardTimer) Time(f func()) { + ts := time.Now() + f() + t.Update(time.Since(ts)) +} + +// Record the duration of an event. +func (t *StandardTimer) Update(d time.Duration) { + t.mutex.Lock() + defer t.mutex.Unlock() + t.histogram.Update(int64(d)) + t.meter.Mark(1) +} + +// Record the duration of an event that started at a time and ends now. +func (t *StandardTimer) UpdateSince(ts time.Time) { + t.mutex.Lock() + defer t.mutex.Unlock() + t.histogram.Update(int64(time.Since(ts))) + t.meter.Mark(1) +} + +// Variance returns the variance of the values in the sample. +func (t *StandardTimer) Variance() float64 { + return t.histogram.Variance() +} + +// TimerSnapshot is a read-only copy of another Timer. +type TimerSnapshot struct { + histogram *HistogramSnapshot + meter *MeterSnapshot +} + +// Count returns the number of events recorded at the time the snapshot was +// taken. +func (t *TimerSnapshot) Count() int64 { return t.histogram.Count() } + +// Max returns the maximum value at the time the snapshot was taken. +func (t *TimerSnapshot) Max() int64 { return t.histogram.Max() } + +// Mean returns the mean value at the time the snapshot was taken. +func (t *TimerSnapshot) Mean() float64 { return t.histogram.Mean() } + +// Min returns the minimum value at the time the snapshot was taken. +func (t *TimerSnapshot) Min() int64 { return t.histogram.Min() } + +// Percentile returns an arbitrary percentile of sampled values at the time the +// snapshot was taken. +func (t *TimerSnapshot) Percentile(p float64) float64 { + return t.histogram.Percentile(p) +} + +// Percentiles returns a slice of arbitrary percentiles of sampled values at +// the time the snapshot was taken. +func (t *TimerSnapshot) Percentiles(ps []float64) []float64 { + return t.histogram.Percentiles(ps) +} + +// Rate1 returns the one-minute moving average rate of events per second at the +// time the snapshot was taken. +func (t *TimerSnapshot) Rate1() float64 { return t.meter.Rate1() } + +// Rate5 returns the five-minute moving average rate of events per second at +// the time the snapshot was taken. +func (t *TimerSnapshot) Rate5() float64 { return t.meter.Rate5() } + +// Rate15 returns the fifteen-minute moving average rate of events per second +// at the time the snapshot was taken. +func (t *TimerSnapshot) Rate15() float64 { return t.meter.Rate15() } + +// RateMean returns the meter's mean rate of events per second at the time the +// snapshot was taken. +func (t *TimerSnapshot) RateMean() float64 { return t.meter.RateMean() } + +// Snapshot returns the snapshot. +func (t *TimerSnapshot) Snapshot() Timer { return t } + +// StdDev returns the standard deviation of the values at the time the snapshot +// was taken. +func (t *TimerSnapshot) StdDev() float64 { return t.histogram.StdDev() } + +// Stop is a no-op. +func (t *TimerSnapshot) Stop() {} + +// Sum returns the sum at the time the snapshot was taken. +func (t *TimerSnapshot) Sum() int64 { return t.histogram.Sum() } + +// Time panics. +func (*TimerSnapshot) Time(func()) { + panic("Time called on a TimerSnapshot") +} + +// Update panics. +func (*TimerSnapshot) Update(time.Duration) { + panic("Update called on a TimerSnapshot") +} + +// UpdateSince panics. +func (*TimerSnapshot) UpdateSince(time.Time) { + panic("UpdateSince called on a TimerSnapshot") +} + +// Variance returns the variance of the values at the time the snapshot was +// taken. +func (t *TimerSnapshot) Variance() float64 { return t.histogram.Variance() } diff --git a/vendor/github.com/rcrowley/go-metrics/validate.sh b/vendor/github.com/rcrowley/go-metrics/validate.sh new file mode 100644 index 000000000000..c4ae91e642d6 --- /dev/null +++ b/vendor/github.com/rcrowley/go-metrics/validate.sh @@ -0,0 +1,10 @@ +#!/bin/bash + +set -e + +# check there are no formatting issues +GOFMT_LINES=`gofmt -l . | wc -l | xargs` +test $GOFMT_LINES -eq 0 || echo "gofmt needs to be run, ${GOFMT_LINES} files have issues" + +# run the tests for the root package +go test -race . diff --git a/vendor/github.com/rcrowley/go-metrics/writer.go b/vendor/github.com/rcrowley/go-metrics/writer.go new file mode 100644 index 000000000000..091e971d2e6f --- /dev/null +++ b/vendor/github.com/rcrowley/go-metrics/writer.go @@ -0,0 +1,100 @@ +package metrics + +import ( + "fmt" + "io" + "sort" + "time" +) + +// Write sorts writes each metric in the given registry periodically to the +// given io.Writer. +func Write(r Registry, d time.Duration, w io.Writer) { + for _ = range time.Tick(d) { + WriteOnce(r, w) + } +} + +// WriteOnce sorts and writes metrics in the given registry to the given +// io.Writer. +func WriteOnce(r Registry, w io.Writer) { + var namedMetrics namedMetricSlice + r.Each(func(name string, i interface{}) { + namedMetrics = append(namedMetrics, namedMetric{name, i}) + }) + + sort.Sort(namedMetrics) + for _, namedMetric := range namedMetrics { + switch metric := namedMetric.m.(type) { + case Counter: + fmt.Fprintf(w, "counter %s\n", namedMetric.name) + fmt.Fprintf(w, " count: %9d\n", metric.Count()) + case Gauge: + fmt.Fprintf(w, "gauge %s\n", namedMetric.name) + fmt.Fprintf(w, " value: %9d\n", metric.Value()) + case GaugeFloat64: + fmt.Fprintf(w, "gauge %s\n", namedMetric.name) + fmt.Fprintf(w, " value: %f\n", metric.Value()) + case Healthcheck: + metric.Check() + fmt.Fprintf(w, "healthcheck %s\n", namedMetric.name) + fmt.Fprintf(w, " error: %v\n", metric.Error()) + case Histogram: + h := metric.Snapshot() + ps := h.Percentiles([]float64{0.5, 0.75, 0.95, 0.99, 0.999}) + fmt.Fprintf(w, "histogram %s\n", namedMetric.name) + fmt.Fprintf(w, " count: %9d\n", h.Count()) + fmt.Fprintf(w, " min: %9d\n", h.Min()) + fmt.Fprintf(w, " max: %9d\n", h.Max()) + fmt.Fprintf(w, " mean: %12.2f\n", h.Mean()) + fmt.Fprintf(w, " stddev: %12.2f\n", h.StdDev()) + fmt.Fprintf(w, " median: %12.2f\n", ps[0]) + fmt.Fprintf(w, " 75%%: %12.2f\n", ps[1]) + fmt.Fprintf(w, " 95%%: %12.2f\n", ps[2]) + fmt.Fprintf(w, " 99%%: %12.2f\n", ps[3]) + fmt.Fprintf(w, " 99.9%%: %12.2f\n", ps[4]) + case Meter: + m := metric.Snapshot() + fmt.Fprintf(w, "meter %s\n", namedMetric.name) + fmt.Fprintf(w, " count: %9d\n", m.Count()) + fmt.Fprintf(w, " 1-min rate: %12.2f\n", m.Rate1()) + fmt.Fprintf(w, " 5-min rate: %12.2f\n", m.Rate5()) + fmt.Fprintf(w, " 15-min rate: %12.2f\n", m.Rate15()) + fmt.Fprintf(w, " mean rate: %12.2f\n", m.RateMean()) + case Timer: + t := metric.Snapshot() + ps := t.Percentiles([]float64{0.5, 0.75, 0.95, 0.99, 0.999}) + fmt.Fprintf(w, "timer %s\n", namedMetric.name) + fmt.Fprintf(w, " count: %9d\n", t.Count()) + fmt.Fprintf(w, " min: %9d\n", t.Min()) + fmt.Fprintf(w, " max: %9d\n", t.Max()) + fmt.Fprintf(w, " mean: %12.2f\n", t.Mean()) + fmt.Fprintf(w, " stddev: %12.2f\n", t.StdDev()) + fmt.Fprintf(w, " median: %12.2f\n", ps[0]) + fmt.Fprintf(w, " 75%%: %12.2f\n", ps[1]) + fmt.Fprintf(w, " 95%%: %12.2f\n", ps[2]) + fmt.Fprintf(w, " 99%%: %12.2f\n", ps[3]) + fmt.Fprintf(w, " 99.9%%: %12.2f\n", ps[4]) + fmt.Fprintf(w, " 1-min rate: %12.2f\n", t.Rate1()) + fmt.Fprintf(w, " 5-min rate: %12.2f\n", t.Rate5()) + fmt.Fprintf(w, " 15-min rate: %12.2f\n", t.Rate15()) + fmt.Fprintf(w, " mean rate: %12.2f\n", t.RateMean()) + } + } +} + +type namedMetric struct { + name string + m interface{} +} + +// namedMetricSlice is a slice of namedMetrics that implements sort.Interface. +type namedMetricSlice []namedMetric + +func (nms namedMetricSlice) Len() int { return len(nms) } + +func (nms namedMetricSlice) Swap(i, j int) { nms[i], nms[j] = nms[j], nms[i] } + +func (nms namedMetricSlice) Less(i, j int) bool { + return nms[i].name < nms[j].name +} diff --git a/vendor/golang.org/x/crypto/md4/md4.go b/vendor/golang.org/x/crypto/md4/md4.go new file mode 100644 index 000000000000..59d348069305 --- /dev/null +++ b/vendor/golang.org/x/crypto/md4/md4.go @@ -0,0 +1,122 @@ +// Copyright 2009 The Go Authors. All rights reserved. +// Use of this source code is governed by a BSD-style +// license that can be found in the LICENSE file. + +// Package md4 implements the MD4 hash algorithm as defined in RFC 1320. +// +// Deprecated: MD4 is cryptographically broken and should should only be used +// where compatibility with legacy systems, not security, is the goal. Instead, +// use a secure hash like SHA-256 (from crypto/sha256). +package md4 // import "golang.org/x/crypto/md4" + +import ( + "crypto" + "hash" +) + +func init() { + crypto.RegisterHash(crypto.MD4, New) +} + +// The size of an MD4 checksum in bytes. +const Size = 16 + +// The blocksize of MD4 in bytes. +const BlockSize = 64 + +const ( + _Chunk = 64 + _Init0 = 0x67452301 + _Init1 = 0xEFCDAB89 + _Init2 = 0x98BADCFE + _Init3 = 0x10325476 +) + +// digest represents the partial evaluation of a checksum. +type digest struct { + s [4]uint32 + x [_Chunk]byte + nx int + len uint64 +} + +func (d *digest) Reset() { + d.s[0] = _Init0 + d.s[1] = _Init1 + d.s[2] = _Init2 + d.s[3] = _Init3 + d.nx = 0 + d.len = 0 +} + +// New returns a new hash.Hash computing the MD4 checksum. +func New() hash.Hash { + d := new(digest) + d.Reset() + return d +} + +func (d *digest) Size() int { return Size } + +func (d *digest) BlockSize() int { return BlockSize } + +func (d *digest) Write(p []byte) (nn int, err error) { + nn = len(p) + d.len += uint64(nn) + if d.nx > 0 { + n := len(p) + if n > _Chunk-d.nx { + n = _Chunk - d.nx + } + for i := 0; i < n; i++ { + d.x[d.nx+i] = p[i] + } + d.nx += n + if d.nx == _Chunk { + _Block(d, d.x[0:]) + d.nx = 0 + } + p = p[n:] + } + n := _Block(d, p) + p = p[n:] + if len(p) > 0 { + d.nx = copy(d.x[:], p) + } + return +} + +func (d0 *digest) Sum(in []byte) []byte { + // Make a copy of d0, so that caller can keep writing and summing. + d := new(digest) + *d = *d0 + + // Padding. Add a 1 bit and 0 bits until 56 bytes mod 64. + len := d.len + var tmp [64]byte + tmp[0] = 0x80 + if len%64 < 56 { + d.Write(tmp[0 : 56-len%64]) + } else { + d.Write(tmp[0 : 64+56-len%64]) + } + + // Length in bits. + len <<= 3 + for i := uint(0); i < 8; i++ { + tmp[i] = byte(len >> (8 * i)) + } + d.Write(tmp[0:8]) + + if d.nx != 0 { + panic("d.nx != 0") + } + + for _, s := range d.s { + in = append(in, byte(s>>0)) + in = append(in, byte(s>>8)) + in = append(in, byte(s>>16)) + in = append(in, byte(s>>24)) + } + return in +} diff --git a/vendor/golang.org/x/crypto/md4/md4block.go b/vendor/golang.org/x/crypto/md4/md4block.go new file mode 100644 index 000000000000..3fed475f3f60 --- /dev/null +++ b/vendor/golang.org/x/crypto/md4/md4block.go @@ -0,0 +1,89 @@ +// Copyright 2009 The Go Authors. All rights reserved. +// Use of this source code is governed by a BSD-style +// license that can be found in the LICENSE file. + +// MD4 block step. +// In its own file so that a faster assembly or C version +// can be substituted easily. + +package md4 + +var shift1 = []uint{3, 7, 11, 19} +var shift2 = []uint{3, 5, 9, 13} +var shift3 = []uint{3, 9, 11, 15} + +var xIndex2 = []uint{0, 4, 8, 12, 1, 5, 9, 13, 2, 6, 10, 14, 3, 7, 11, 15} +var xIndex3 = []uint{0, 8, 4, 12, 2, 10, 6, 14, 1, 9, 5, 13, 3, 11, 7, 15} + +func _Block(dig *digest, p []byte) int { + a := dig.s[0] + b := dig.s[1] + c := dig.s[2] + d := dig.s[3] + n := 0 + var X [16]uint32 + for len(p) >= _Chunk { + aa, bb, cc, dd := a, b, c, d + + j := 0 + for i := 0; i < 16; i++ { + X[i] = uint32(p[j]) | uint32(p[j+1])<<8 | uint32(p[j+2])<<16 | uint32(p[j+3])<<24 + j += 4 + } + + // If this needs to be made faster in the future, + // the usual trick is to unroll each of these + // loops by a factor of 4; that lets you replace + // the shift[] lookups with constants and, + // with suitable variable renaming in each + // unrolled body, delete the a, b, c, d = d, a, b, c + // (or you can let the optimizer do the renaming). + // + // The index variables are uint so that % by a power + // of two can be optimized easily by a compiler. + + // Round 1. + for i := uint(0); i < 16; i++ { + x := i + s := shift1[i%4] + f := ((c ^ d) & b) ^ d + a += f + X[x] + a = a<>(32-s) + a, b, c, d = d, a, b, c + } + + // Round 2. + for i := uint(0); i < 16; i++ { + x := xIndex2[i] + s := shift2[i%4] + g := (b & c) | (b & d) | (c & d) + a += g + X[x] + 0x5a827999 + a = a<>(32-s) + a, b, c, d = d, a, b, c + } + + // Round 3. + for i := uint(0); i < 16; i++ { + x := xIndex3[i] + s := shift3[i%4] + h := b ^ c ^ d + a += h + X[x] + 0x6ed9eba1 + a = a<>(32-s) + a, b, c, d = d, a, b, c + } + + a += aa + b += bb + c += cc + d += dd + + p = p[_Chunk:] + n += _Chunk + } + + dig.s[0] = a + dig.s[1] = b + dig.s[2] = c + dig.s[3] = d + return n +} diff --git a/vendor/golang.org/x/crypto/sha3/xor_generic.go b/vendor/golang.org/x/crypto/sha3/xor_generic.go index fd35f02ef6e0..8d947711272c 100644 --- a/vendor/golang.org/x/crypto/sha3/xor_generic.go +++ b/vendor/golang.org/x/crypto/sha3/xor_generic.go @@ -19,7 +19,7 @@ func xorInGeneric(d *state, buf []byte) { } } -// copyOutGeneric copies ulint64s to a byte buffer. +// copyOutGeneric copies uint64s to a byte buffer. func copyOutGeneric(d *state, b []byte) { for i := 0; len(b) >= 8; i++ { binary.LittleEndian.PutUint64(b, d.a[i]) diff --git a/vendor/golang.org/x/net/http/httpproxy/proxy.go b/vendor/golang.org/x/net/http/httpproxy/proxy.go index 1415b077912f..d2c8c87eab91 100644 --- a/vendor/golang.org/x/net/http/httpproxy/proxy.go +++ b/vendor/golang.org/x/net/http/httpproxy/proxy.go @@ -113,8 +113,8 @@ func getEnvAny(names ...string) string { // environment, or a proxy should not be used for the given request, as // defined by NO_PROXY. // -// As a special case, if req.URL.Host is "localhost" (with or without a -// port number), then a nil URL and nil error will be returned. +// As a special case, if req.URL.Host is "localhost" or a loopback address +// (with or without a port number), then a nil URL and nil error will be returned. func (cfg *Config) ProxyFunc() func(reqURL *url.URL) (*url.URL, error) { // Preprocess the Config settings for more efficient evaluation. cfg1 := &config{ diff --git a/vendor/golang.org/x/net/http2/client_conn_pool.go b/vendor/golang.org/x/net/http2/client_conn_pool.go index 652bc11a029b..c936843eafa1 100644 --- a/vendor/golang.org/x/net/http2/client_conn_pool.go +++ b/vendor/golang.org/x/net/http2/client_conn_pool.go @@ -16,6 +16,12 @@ import ( // ClientConnPool manages a pool of HTTP/2 client connections. type ClientConnPool interface { + // GetClientConn returns a specific HTTP/2 connection (usually + // a TLS-TCP connection) to an HTTP/2 server. On success, the + // returned ClientConn accounts for the upcoming RoundTrip + // call, so the caller should not omit it. If the caller needs + // to, ClientConn.RoundTrip can be called with a bogus + // new(http.Request) to release the stream reservation. GetClientConn(req *http.Request, addr string) (*ClientConn, error) MarkDead(*ClientConn) } @@ -42,7 +48,7 @@ type clientConnPool struct { conns map[string][]*ClientConn // key is host:port dialing map[string]*dialCall // currently in-flight dials keys map[*ClientConn][]string - addConnCalls map[string]*addConnCall // in-flight addConnIfNeede calls + addConnCalls map[string]*addConnCall // in-flight addConnIfNeeded calls } func (p *clientConnPool) GetClientConn(req *http.Request, addr string) (*ClientConn, error) { @@ -54,28 +60,8 @@ const ( noDialOnMiss = false ) -// shouldTraceGetConn reports whether getClientConn should call any -// ClientTrace.GetConn hook associated with the http.Request. -// -// This complexity is needed to avoid double calls of the GetConn hook -// during the back-and-forth between net/http and x/net/http2 (when the -// net/http.Transport is upgraded to also speak http2), as well as support -// the case where x/net/http2 is being used directly. -func (p *clientConnPool) shouldTraceGetConn(st clientConnIdleState) bool { - // If our Transport wasn't made via ConfigureTransport, always - // trace the GetConn hook if provided, because that means the - // http2 package is being used directly and it's the one - // dialing, as opposed to net/http. - if _, ok := p.t.ConnPool.(noDialClientConnPool); !ok { - return true - } - // Otherwise, only use the GetConn hook if this connection has - // been used previously for other requests. For fresh - // connections, the net/http package does the dialing. - return !st.freshConn -} - func (p *clientConnPool) getClientConn(req *http.Request, addr string, dialOnMiss bool) (*ClientConn, error) { + // TODO(dneil): Dial a new connection when t.DisableKeepAlives is set? if isConnectionCloseRequest(req) && dialOnMiss { // It gets its own connection. traceGetConn(req, addr) @@ -89,10 +75,14 @@ func (p *clientConnPool) getClientConn(req *http.Request, addr string, dialOnMis for { p.mu.Lock() for _, cc := range p.conns[addr] { - if st := cc.idleState(); st.canTakeNewRequest { - if p.shouldTraceGetConn(st) { + if cc.ReserveNewRequest() { + // When a connection is presented to us by the net/http package, + // the GetConn hook has already been called. + // Don't call it a second time here. + if !cc.getConnCalled { traceGetConn(req, addr) } + cc.getConnCalled = false p.mu.Unlock() return cc, nil } @@ -108,7 +98,13 @@ func (p *clientConnPool) getClientConn(req *http.Request, addr string, dialOnMis if shouldRetryDial(call, req) { continue } - return call.res, call.err + cc, err := call.res, call.err + if err != nil { + return nil, err + } + if cc.ReserveNewRequest() { + return cc, nil + } } } @@ -205,6 +201,7 @@ func (c *addConnCall) run(t *Transport, key string, tc *tls.Conn) { if err != nil { c.err = err } else { + cc.getConnCalled = true // already called by the net/http package p.addConnLocked(key, cc) } delete(p.addConnCalls, key) diff --git a/vendor/golang.org/x/net/http2/errors.go b/vendor/golang.org/x/net/http2/errors.go index c789fa338cd5..2663e5d287ee 100644 --- a/vendor/golang.org/x/net/http2/errors.go +++ b/vendor/golang.org/x/net/http2/errors.go @@ -53,6 +53,13 @@ func (e ErrCode) String() string { return fmt.Sprintf("unknown error code 0x%x", uint32(e)) } +func (e ErrCode) stringToken() string { + if s, ok := errCodeName[e]; ok { + return s + } + return fmt.Sprintf("ERR_UNKNOWN_%d", uint32(e)) +} + // ConnectionError is an error that results in the termination of the // entire connection. type ConnectionError ErrCode diff --git a/vendor/golang.org/x/net/http2/frame.go b/vendor/golang.org/x/net/http2/frame.go index b95d6f2df2f9..96a747905241 100644 --- a/vendor/golang.org/x/net/http2/frame.go +++ b/vendor/golang.org/x/net/http2/frame.go @@ -122,7 +122,7 @@ var flagName = map[FrameType]map[Flags]string{ // a frameParser parses a frame given its FrameHeader and payload // bytes. The length of payload will always equal fh.Length (which // might be 0). -type frameParser func(fc *frameCache, fh FrameHeader, payload []byte) (Frame, error) +type frameParser func(fc *frameCache, fh FrameHeader, countError func(string), payload []byte) (Frame, error) var frameParsers = map[FrameType]frameParser{ FrameData: parseDataFrame, @@ -267,6 +267,11 @@ type Framer struct { lastFrame Frame errDetail error + // countError is a non-nil func that's called on a frame parse + // error with some unique error path token. It's initialized + // from Transport.CountError or Server.CountError. + countError func(errToken string) + // lastHeaderStream is non-zero if the last frame was an // unfinished HEADERS/CONTINUATION. lastHeaderStream uint32 @@ -426,6 +431,7 @@ func NewFramer(w io.Writer, r io.Reader) *Framer { fr := &Framer{ w: w, r: r, + countError: func(string) {}, logReads: logFrameReads, logWrites: logFrameWrites, debugReadLoggerf: log.Printf, @@ -500,7 +506,7 @@ func (fr *Framer) ReadFrame() (Frame, error) { if _, err := io.ReadFull(fr.r, payload); err != nil { return nil, err } - f, err := typeFrameParser(fh.Type)(fr.frameCache, fh, payload) + f, err := typeFrameParser(fh.Type)(fr.frameCache, fh, fr.countError, payload) if err != nil { if ce, ok := err.(connError); ok { return nil, fr.connError(ce.Code, ce.Reason) @@ -588,13 +594,14 @@ func (f *DataFrame) Data() []byte { return f.data } -func parseDataFrame(fc *frameCache, fh FrameHeader, payload []byte) (Frame, error) { +func parseDataFrame(fc *frameCache, fh FrameHeader, countError func(string), payload []byte) (Frame, error) { if fh.StreamID == 0 { // DATA frames MUST be associated with a stream. If a // DATA frame is received whose stream identifier // field is 0x0, the recipient MUST respond with a // connection error (Section 5.4.1) of type // PROTOCOL_ERROR. + countError("frame_data_stream_0") return nil, connError{ErrCodeProtocol, "DATA frame with stream ID 0"} } f := fc.getDataFrame() @@ -605,6 +612,7 @@ func parseDataFrame(fc *frameCache, fh FrameHeader, payload []byte) (Frame, erro var err error payload, padSize, err = readByte(payload) if err != nil { + countError("frame_data_pad_byte_short") return nil, err } } @@ -613,6 +621,7 @@ func parseDataFrame(fc *frameCache, fh FrameHeader, payload []byte) (Frame, erro // length of the frame payload, the recipient MUST // treat this as a connection error. // Filed: https://github.com/http2/http2-spec/issues/610 + countError("frame_data_pad_too_big") return nil, connError{ErrCodeProtocol, "pad size larger than data payload"} } f.data = payload[:len(payload)-int(padSize)] @@ -695,7 +704,7 @@ type SettingsFrame struct { p []byte } -func parseSettingsFrame(_ *frameCache, fh FrameHeader, p []byte) (Frame, error) { +func parseSettingsFrame(_ *frameCache, fh FrameHeader, countError func(string), p []byte) (Frame, error) { if fh.Flags.Has(FlagSettingsAck) && fh.Length > 0 { // When this (ACK 0x1) bit is set, the payload of the // SETTINGS frame MUST be empty. Receipt of a @@ -703,6 +712,7 @@ func parseSettingsFrame(_ *frameCache, fh FrameHeader, p []byte) (Frame, error) // field value other than 0 MUST be treated as a // connection error (Section 5.4.1) of type // FRAME_SIZE_ERROR. + countError("frame_settings_ack_with_length") return nil, ConnectionError(ErrCodeFrameSize) } if fh.StreamID != 0 { @@ -713,14 +723,17 @@ func parseSettingsFrame(_ *frameCache, fh FrameHeader, p []byte) (Frame, error) // field is anything other than 0x0, the endpoint MUST // respond with a connection error (Section 5.4.1) of // type PROTOCOL_ERROR. + countError("frame_settings_has_stream") return nil, ConnectionError(ErrCodeProtocol) } if len(p)%6 != 0 { + countError("frame_settings_mod_6") // Expecting even number of 6 byte settings. return nil, ConnectionError(ErrCodeFrameSize) } f := &SettingsFrame{FrameHeader: fh, p: p} if v, ok := f.Value(SettingInitialWindowSize); ok && v > (1<<31)-1 { + countError("frame_settings_window_size_too_big") // Values above the maximum flow control window size of 2^31 - 1 MUST // be treated as a connection error (Section 5.4.1) of type // FLOW_CONTROL_ERROR. @@ -832,11 +845,13 @@ type PingFrame struct { func (f *PingFrame) IsAck() bool { return f.Flags.Has(FlagPingAck) } -func parsePingFrame(_ *frameCache, fh FrameHeader, payload []byte) (Frame, error) { +func parsePingFrame(_ *frameCache, fh FrameHeader, countError func(string), payload []byte) (Frame, error) { if len(payload) != 8 { + countError("frame_ping_length") return nil, ConnectionError(ErrCodeFrameSize) } if fh.StreamID != 0 { + countError("frame_ping_has_stream") return nil, ConnectionError(ErrCodeProtocol) } f := &PingFrame{FrameHeader: fh} @@ -872,11 +887,13 @@ func (f *GoAwayFrame) DebugData() []byte { return f.debugData } -func parseGoAwayFrame(_ *frameCache, fh FrameHeader, p []byte) (Frame, error) { +func parseGoAwayFrame(_ *frameCache, fh FrameHeader, countError func(string), p []byte) (Frame, error) { if fh.StreamID != 0 { + countError("frame_goaway_has_stream") return nil, ConnectionError(ErrCodeProtocol) } if len(p) < 8 { + countError("frame_goaway_short") return nil, ConnectionError(ErrCodeFrameSize) } return &GoAwayFrame{ @@ -912,7 +929,7 @@ func (f *UnknownFrame) Payload() []byte { return f.p } -func parseUnknownFrame(_ *frameCache, fh FrameHeader, p []byte) (Frame, error) { +func parseUnknownFrame(_ *frameCache, fh FrameHeader, countError func(string), p []byte) (Frame, error) { return &UnknownFrame{fh, p}, nil } @@ -923,8 +940,9 @@ type WindowUpdateFrame struct { Increment uint32 // never read with high bit set } -func parseWindowUpdateFrame(_ *frameCache, fh FrameHeader, p []byte) (Frame, error) { +func parseWindowUpdateFrame(_ *frameCache, fh FrameHeader, countError func(string), p []byte) (Frame, error) { if len(p) != 4 { + countError("frame_windowupdate_bad_len") return nil, ConnectionError(ErrCodeFrameSize) } inc := binary.BigEndian.Uint32(p[:4]) & 0x7fffffff // mask off high reserved bit @@ -936,8 +954,10 @@ func parseWindowUpdateFrame(_ *frameCache, fh FrameHeader, p []byte) (Frame, err // control window MUST be treated as a connection // error (Section 5.4.1). if fh.StreamID == 0 { + countError("frame_windowupdate_zero_inc_conn") return nil, ConnectionError(ErrCodeProtocol) } + countError("frame_windowupdate_zero_inc_stream") return nil, streamError(fh.StreamID, ErrCodeProtocol) } return &WindowUpdateFrame{ @@ -988,7 +1008,7 @@ func (f *HeadersFrame) HasPriority() bool { return f.FrameHeader.Flags.Has(FlagHeadersPriority) } -func parseHeadersFrame(_ *frameCache, fh FrameHeader, p []byte) (_ Frame, err error) { +func parseHeadersFrame(_ *frameCache, fh FrameHeader, countError func(string), p []byte) (_ Frame, err error) { hf := &HeadersFrame{ FrameHeader: fh, } @@ -997,11 +1017,13 @@ func parseHeadersFrame(_ *frameCache, fh FrameHeader, p []byte) (_ Frame, err er // is received whose stream identifier field is 0x0, the recipient MUST // respond with a connection error (Section 5.4.1) of type // PROTOCOL_ERROR. + countError("frame_headers_zero_stream") return nil, connError{ErrCodeProtocol, "HEADERS frame with stream ID 0"} } var padLength uint8 if fh.Flags.Has(FlagHeadersPadded) { if p, padLength, err = readByte(p); err != nil { + countError("frame_headers_pad_short") return } } @@ -1009,16 +1031,19 @@ func parseHeadersFrame(_ *frameCache, fh FrameHeader, p []byte) (_ Frame, err er var v uint32 p, v, err = readUint32(p) if err != nil { + countError("frame_headers_prio_short") return nil, err } hf.Priority.StreamDep = v & 0x7fffffff hf.Priority.Exclusive = (v != hf.Priority.StreamDep) // high bit was set p, hf.Priority.Weight, err = readByte(p) if err != nil { + countError("frame_headers_prio_weight_short") return nil, err } } if len(p)-int(padLength) < 0 { + countError("frame_headers_pad_too_big") return nil, streamError(fh.StreamID, ErrCodeProtocol) } hf.headerFragBuf = p[:len(p)-int(padLength)] @@ -1125,11 +1150,13 @@ func (p PriorityParam) IsZero() bool { return p == PriorityParam{} } -func parsePriorityFrame(_ *frameCache, fh FrameHeader, payload []byte) (Frame, error) { +func parsePriorityFrame(_ *frameCache, fh FrameHeader, countError func(string), payload []byte) (Frame, error) { if fh.StreamID == 0 { + countError("frame_priority_zero_stream") return nil, connError{ErrCodeProtocol, "PRIORITY frame with stream ID 0"} } if len(payload) != 5 { + countError("frame_priority_bad_length") return nil, connError{ErrCodeFrameSize, fmt.Sprintf("PRIORITY frame payload size was %d; want 5", len(payload))} } v := binary.BigEndian.Uint32(payload[:4]) @@ -1172,11 +1199,13 @@ type RSTStreamFrame struct { ErrCode ErrCode } -func parseRSTStreamFrame(_ *frameCache, fh FrameHeader, p []byte) (Frame, error) { +func parseRSTStreamFrame(_ *frameCache, fh FrameHeader, countError func(string), p []byte) (Frame, error) { if len(p) != 4 { + countError("frame_rststream_bad_len") return nil, ConnectionError(ErrCodeFrameSize) } if fh.StreamID == 0 { + countError("frame_rststream_zero_stream") return nil, ConnectionError(ErrCodeProtocol) } return &RSTStreamFrame{fh, ErrCode(binary.BigEndian.Uint32(p[:4]))}, nil @@ -1202,8 +1231,9 @@ type ContinuationFrame struct { headerFragBuf []byte } -func parseContinuationFrame(_ *frameCache, fh FrameHeader, p []byte) (Frame, error) { +func parseContinuationFrame(_ *frameCache, fh FrameHeader, countError func(string), p []byte) (Frame, error) { if fh.StreamID == 0 { + countError("frame_continuation_zero_stream") return nil, connError{ErrCodeProtocol, "CONTINUATION frame with stream ID 0"} } return &ContinuationFrame{fh, p}, nil @@ -1252,7 +1282,7 @@ func (f *PushPromiseFrame) HeadersEnded() bool { return f.FrameHeader.Flags.Has(FlagPushPromiseEndHeaders) } -func parsePushPromise(_ *frameCache, fh FrameHeader, p []byte) (_ Frame, err error) { +func parsePushPromise(_ *frameCache, fh FrameHeader, countError func(string), p []byte) (_ Frame, err error) { pp := &PushPromiseFrame{ FrameHeader: fh, } @@ -1263,6 +1293,7 @@ func parsePushPromise(_ *frameCache, fh FrameHeader, p []byte) (_ Frame, err err // with. If the stream identifier field specifies the value // 0x0, a recipient MUST respond with a connection error // (Section 5.4.1) of type PROTOCOL_ERROR. + countError("frame_pushpromise_zero_stream") return nil, ConnectionError(ErrCodeProtocol) } // The PUSH_PROMISE frame includes optional padding. @@ -1270,18 +1301,21 @@ func parsePushPromise(_ *frameCache, fh FrameHeader, p []byte) (_ Frame, err err var padLength uint8 if fh.Flags.Has(FlagPushPromisePadded) { if p, padLength, err = readByte(p); err != nil { + countError("frame_pushpromise_pad_short") return } } p, pp.PromiseID, err = readUint32(p) if err != nil { + countError("frame_pushpromise_promiseid_short") return } pp.PromiseID = pp.PromiseID & (1<<31 - 1) if int(padLength) > len(p) { // like the DATA frame, error out if padding is longer than the body. + countError("frame_pushpromise_pad_too_big") return nil, ConnectionError(ErrCodeProtocol) } pp.headerFragBuf = p[:len(p)-int(padLength)] diff --git a/vendor/golang.org/x/net/http2/hpack/huffman.go b/vendor/golang.org/x/net/http2/hpack/huffman.go index a1ab2f056794..fe0b84ccd467 100644 --- a/vendor/golang.org/x/net/http2/hpack/huffman.go +++ b/vendor/golang.org/x/net/http2/hpack/huffman.go @@ -140,25 +140,29 @@ func buildRootHuffmanNode() { panic("unexpected size") } lazyRootHuffmanNode = newInternalNode() - for i, code := range huffmanCodes { - addDecoderNode(byte(i), code, huffmanCodeLen[i]) - } -} + // allocate a leaf node for each of the 256 symbols + leaves := new([256]node) + + for sym, code := range huffmanCodes { + codeLen := huffmanCodeLen[sym] + + cur := lazyRootHuffmanNode + for codeLen > 8 { + codeLen -= 8 + i := uint8(code >> codeLen) + if cur.children[i] == nil { + cur.children[i] = newInternalNode() + } + cur = cur.children[i] + } + shift := 8 - codeLen + start, end := int(uint8(code< 8 { - codeLen -= 8 - i := uint8(code >> codeLen) - if cur.children[i] == nil { - cur.children[i] = newInternalNode() + leaves[sym].sym = byte(sym) + leaves[sym].codeLen = codeLen + for i := start; i < start+end; i++ { + cur.children[i] = &leaves[sym] } - cur = cur.children[i] - } - shift := 8 - codeLen - start, end := int(uint8(code< 0 { // Check whether the client has flow control quota. if st.inflow.available() < int32(f.Length) { - return streamError(id, ErrCodeFlowControl) + return sc.countError("flow_on_data_length", streamError(id, ErrCodeFlowControl)) } st.inflow.take(int32(f.Length)) @@ -1710,7 +1719,7 @@ func (sc *serverConn) processData(f *DataFrame) error { wrote, err := st.body.Write(data) if err != nil { sc.sendWindowUpdate(nil, int(f.Length)-wrote) - return streamError(id, ErrCodeStreamClosed) + return sc.countError("body_write_err", streamError(id, ErrCodeStreamClosed)) } if wrote != len(data) { panic("internal error: bad Writer") @@ -1796,7 +1805,7 @@ func (sc *serverConn) processHeaders(f *MetaHeadersFrame) error { // stream identifier MUST respond with a connection error // (Section 5.4.1) of type PROTOCOL_ERROR. if id%2 != 1 { - return ConnectionError(ErrCodeProtocol) + return sc.countError("headers_even", ConnectionError(ErrCodeProtocol)) } // A HEADERS frame can be used to create a new stream or // send a trailer for an open one. If we already have a stream @@ -1813,7 +1822,7 @@ func (sc *serverConn) processHeaders(f *MetaHeadersFrame) error { // this state, it MUST respond with a stream error (Section 5.4.2) of // type STREAM_CLOSED. if st.state == stateHalfClosedRemote { - return streamError(id, ErrCodeStreamClosed) + return sc.countError("headers_half_closed", streamError(id, ErrCodeStreamClosed)) } return st.processTrailerHeaders(f) } @@ -1824,7 +1833,7 @@ func (sc *serverConn) processHeaders(f *MetaHeadersFrame) error { // receives an unexpected stream identifier MUST respond with // a connection error (Section 5.4.1) of type PROTOCOL_ERROR. if id <= sc.maxClientStreamID { - return ConnectionError(ErrCodeProtocol) + return sc.countError("stream_went_down", ConnectionError(ErrCodeProtocol)) } sc.maxClientStreamID = id @@ -1841,14 +1850,14 @@ func (sc *serverConn) processHeaders(f *MetaHeadersFrame) error { if sc.curClientStreams+1 > sc.advMaxStreams { if sc.unackedSettings == 0 { // They should know better. - return streamError(id, ErrCodeProtocol) + return sc.countError("over_max_streams", streamError(id, ErrCodeProtocol)) } // Assume it's a network race, where they just haven't // received our last SETTINGS update. But actually // this can't happen yet, because we don't yet provide // a way for users to adjust server parameters at // runtime. - return streamError(id, ErrCodeRefusedStream) + return sc.countError("over_max_streams_race", streamError(id, ErrCodeRefusedStream)) } initialState := stateOpen @@ -1858,7 +1867,7 @@ func (sc *serverConn) processHeaders(f *MetaHeadersFrame) error { st := sc.newStream(id, 0, initialState) if f.HasPriority() { - if err := checkPriority(f.StreamID, f.Priority); err != nil { + if err := sc.checkPriority(f.StreamID, f.Priority); err != nil { return err } sc.writeSched.AdjustStream(st.id, f.Priority) @@ -1902,15 +1911,15 @@ func (st *stream) processTrailerHeaders(f *MetaHeadersFrame) error { sc := st.sc sc.serveG.check() if st.gotTrailerHeader { - return ConnectionError(ErrCodeProtocol) + return sc.countError("dup_trailers", ConnectionError(ErrCodeProtocol)) } st.gotTrailerHeader = true if !f.StreamEnded() { - return streamError(st.id, ErrCodeProtocol) + return sc.countError("trailers_not_ended", streamError(st.id, ErrCodeProtocol)) } if len(f.PseudoFields()) > 0 { - return streamError(st.id, ErrCodeProtocol) + return sc.countError("trailers_pseudo", streamError(st.id, ErrCodeProtocol)) } if st.trailer != nil { for _, hf := range f.RegularFields() { @@ -1919,7 +1928,7 @@ func (st *stream) processTrailerHeaders(f *MetaHeadersFrame) error { // TODO: send more details to the peer somehow. But http2 has // no way to send debug data at a stream level. Discuss with // HTTP folk. - return streamError(st.id, ErrCodeProtocol) + return sc.countError("trailers_bogus", streamError(st.id, ErrCodeProtocol)) } st.trailer[key] = append(st.trailer[key], hf.Value) } @@ -1928,13 +1937,13 @@ func (st *stream) processTrailerHeaders(f *MetaHeadersFrame) error { return nil } -func checkPriority(streamID uint32, p PriorityParam) error { +func (sc *serverConn) checkPriority(streamID uint32, p PriorityParam) error { if streamID == p.StreamDep { // Section 5.3.1: "A stream cannot depend on itself. An endpoint MUST treat // this as a stream error (Section 5.4.2) of type PROTOCOL_ERROR." // Section 5.3.3 says that a stream can depend on one of its dependencies, // so it's only self-dependencies that are forbidden. - return streamError(streamID, ErrCodeProtocol) + return sc.countError("priority", streamError(streamID, ErrCodeProtocol)) } return nil } @@ -1943,7 +1952,7 @@ func (sc *serverConn) processPriority(f *PriorityFrame) error { if sc.inGoAway { return nil } - if err := checkPriority(f.StreamID, f.PriorityParam); err != nil { + if err := sc.checkPriority(f.StreamID, f.PriorityParam); err != nil { return err } sc.writeSched.AdjustStream(f.StreamID, f.PriorityParam) @@ -2000,7 +2009,7 @@ func (sc *serverConn) newWriterAndRequest(st *stream, f *MetaHeadersFrame) (*res isConnect := rp.method == "CONNECT" if isConnect { if rp.path != "" || rp.scheme != "" || rp.authority == "" { - return nil, nil, streamError(f.StreamID, ErrCodeProtocol) + return nil, nil, sc.countError("bad_connect", streamError(f.StreamID, ErrCodeProtocol)) } } else if rp.method == "" || rp.path == "" || (rp.scheme != "https" && rp.scheme != "http") { // See 8.1.2.6 Malformed Requests and Responses: @@ -2013,13 +2022,13 @@ func (sc *serverConn) newWriterAndRequest(st *stream, f *MetaHeadersFrame) (*res // "All HTTP/2 requests MUST include exactly one valid // value for the :method, :scheme, and :path // pseudo-header fields" - return nil, nil, streamError(f.StreamID, ErrCodeProtocol) + return nil, nil, sc.countError("bad_path_method", streamError(f.StreamID, ErrCodeProtocol)) } bodyOpen := !f.StreamEnded() if rp.method == "HEAD" && bodyOpen { // HEAD requests can't have bodies - return nil, nil, streamError(f.StreamID, ErrCodeProtocol) + return nil, nil, sc.countError("head_body", streamError(f.StreamID, ErrCodeProtocol)) } rp.header = make(http.Header) @@ -2102,7 +2111,7 @@ func (sc *serverConn) newWriterAndRequestNoBody(st *stream, rp requestParam) (*r var err error url_, err = url.ParseRequestURI(rp.path) if err != nil { - return nil, nil, streamError(st.id, ErrCodeProtocol) + return nil, nil, sc.countError("bad_path", streamError(st.id, ErrCodeProtocol)) } requestURI = rp.path } @@ -2985,3 +2994,31 @@ func h1ServerKeepAlivesDisabled(hs *http.Server) bool { } return false } + +func (sc *serverConn) countError(name string, err error) error { + if sc == nil || sc.srv == nil { + return err + } + f := sc.srv.CountError + if f == nil { + return err + } + var typ string + var code ErrCode + switch e := err.(type) { + case ConnectionError: + typ = "conn" + code = ErrCode(e) + case StreamError: + typ = "stream" + code = ErrCode(e.Code) + default: + return err + } + codeStr := errCodeName[code] + if codeStr == "" { + codeStr = strconv.Itoa(int(code)) + } + f(fmt.Sprintf("%s_%s_%s", typ, codeStr, name)) + return err +} diff --git a/vendor/golang.org/x/net/http2/transport.go b/vendor/golang.org/x/net/http2/transport.go index dc31cfd752d6..1cecf98b23fe 100644 --- a/vendor/golang.org/x/net/http2/transport.go +++ b/vendor/golang.org/x/net/http2/transport.go @@ -24,6 +24,7 @@ import ( "net/http" "net/http/httptrace" "net/textproto" + "os" "sort" "strconv" "strings" @@ -51,6 +52,15 @@ const ( transportDefaultStreamMinRefresh = 4 << 10 defaultUserAgent = "Go-http-client/2.0" + + // initialMaxConcurrentStreams is a connections maxConcurrentStreams until + // it's received servers initial SETTINGS frame, which corresponds with the + // spec's minimum recommended value. + initialMaxConcurrentStreams = 100 + + // defaultMaxConcurrentStreams is a connections default maxConcurrentStreams + // if the server doesn't include one in its initial SETTINGS frame. + defaultMaxConcurrentStreams = 1000 ) // Transport is an HTTP/2 Transport. @@ -121,6 +131,17 @@ type Transport struct { // Defaults to 15s. PingTimeout time.Duration + // WriteByteTimeout is the timeout after which the connection will be + // closed no data can be written to it. The timeout begins when data is + // available to write, and is extended whenever any bytes are written. + WriteByteTimeout time.Duration + + // CountError, if non-nil, is called on HTTP/2 transport errors. + // It's intended to increment a metric for monitoring, such + // as an expvar or Prometheus metric. + // The errType consists of only ASCII word characters. + CountError func(errType string) + // t1, if non-nil, is the standard library Transport using // this transport. Its settings are used (but not its // RoundTrip method, etc). @@ -227,11 +248,12 @@ func (t *Transport) initConnPool() { // ClientConn is the state of a single HTTP/2 client connection to an // HTTP/2 server. type ClientConn struct { - t *Transport - tconn net.Conn // usually *tls.Conn, except specialized impls - tlsState *tls.ConnectionState // nil only for specialized impls - reused uint32 // whether conn is being reused; atomic - singleUse bool // whether being used for a single http.Request + t *Transport + tconn net.Conn // usually *tls.Conn, except specialized impls + tlsState *tls.ConnectionState // nil only for specialized impls + reused uint32 // whether conn is being reused; atomic + singleUse bool // whether being used for a single http.Request + getConnCalled bool // used by clientConnPool // readLoop goroutine fields: readerDone chan struct{} // closed on error @@ -247,84 +269,91 @@ type ClientConn struct { doNotReuse bool // whether conn is marked to not be reused for any future requests closing bool closed bool + seenSettings bool // true if we've seen a settings frame, false otherwise wantSettingsAck bool // we sent a SETTINGS frame and haven't heard back goAway *GoAwayFrame // if non-nil, the GoAwayFrame we received goAwayDebug string // goAway frame's debug data, retained as a string streams map[uint32]*clientStream // client-initiated + streamsReserved int // incr by ReserveNewRequest; decr on RoundTrip nextStreamID uint32 pendingRequests int // requests blocked and waiting to be sent because len(streams) == maxConcurrentStreams pings map[[8]byte]chan struct{} // in flight ping data to notification channel - bw *bufio.Writer br *bufio.Reader - fr *Framer lastActive time.Time lastIdle time.Time // time last idle - // Settings from peer: (also guarded by mu) + // Settings from peer: (also guarded by wmu) maxFrameSize uint32 maxConcurrentStreams uint32 peerMaxHeaderListSize uint64 initialWindowSize uint32 + // reqHeaderMu is a 1-element semaphore channel controlling access to sending new requests. + // Write to reqHeaderMu to lock it, read from it to unlock. + // Lock reqmu BEFORE mu or wmu. + reqHeaderMu chan struct{} + + // wmu is held while writing. + // Acquire BEFORE mu when holding both, to avoid blocking mu on network writes. + // Only acquire both at the same time when changing peer settings. + wmu sync.Mutex + bw *bufio.Writer + fr *Framer + werr error // first write error that has occurred hbuf bytes.Buffer // HPACK encoder writes into this henc *hpack.Encoder - - wmu sync.Mutex // held while writing; acquire AFTER mu if holding both - werr error // first write error that has occurred } // clientStream is the state for a single HTTP/2 stream. One of these // is created for each Transport.RoundTrip call. type clientStream struct { - cc *ClientConn - req *http.Request + cc *ClientConn + + // Fields of Request that we may access even after the response body is closed. + ctx context.Context + reqCancel <-chan struct{} + trace *httptrace.ClientTrace // or nil ID uint32 - resc chan resAndError bufPipe pipe // buffered pipe with the flow-controlled response payload - startedWrite bool // started request body write; guarded by cc.mu requestedGzip bool - on100 func() // optional code to run if get a 100 continue response + isHead bool + + abortOnce sync.Once + abort chan struct{} // closed to signal stream should end immediately + abortErr error // set if abort is closed + + peerClosed chan struct{} // closed when the peer sends an END_STREAM flag + donec chan struct{} // closed after the stream is in the closed state + on100 chan struct{} // buffered; written to if a 100 is received + + respHeaderRecv chan struct{} // closed when headers are received + res *http.Response // set if respHeaderRecv is closed flow flow // guarded by cc.mu inflow flow // guarded by cc.mu bytesRemain int64 // -1 means unknown; owned by transportResponseBody.Read readErr error // sticky read error; owned by transportResponseBody.Read - stopReqBody error // if non-nil, stop writing req body; guarded by cc.mu - didReset bool // whether we sent a RST_STREAM to the server; guarded by cc.mu - peerReset chan struct{} // closed on peer reset - resetErr error // populated before peerReset is closed + reqBody io.ReadCloser + reqBodyContentLength int64 // -1 means unknown + reqBodyClosed bool // body has been closed; guarded by cc.mu - done chan struct{} // closed when stream remove from cc.streams map; close calls guarded by cc.mu + // owned by writeRequest: + sentEndStream bool // sent an END_STREAM flag to the peer + sentHeaders bool // owned by clientConnReadLoop: firstByte bool // got the first response byte pastHeaders bool // got first MetaHeadersFrame (actual headers) pastTrailers bool // got optional second MetaHeadersFrame (trailers) num1xx uint8 // number of 1xx responses seen + readClosed bool // peer sent an END_STREAM flag + readAborted bool // read loop reset the stream trailer http.Header // accumulated trailers resTrailer *http.Header // client's Response.Trailer } -// awaitRequestCancel waits for the user to cancel a request or for the done -// channel to be signaled. A non-nil error is returned only if the request was -// canceled. -func awaitRequestCancel(req *http.Request, done <-chan struct{}) error { - ctx := req.Context() - if req.Cancel == nil && ctx.Done() == nil { - return nil - } - select { - case <-req.Cancel: - return errRequestCanceled - case <-ctx.Done(): - return ctx.Err() - case <-done: - return nil - } -} - var got1xxFuncForTests func(int, textproto.MIMEHeader) error // get1xxTraceFunc returns the value of request's httptrace.ClientTrace.Got1xxResponse func, @@ -336,78 +365,65 @@ func (cs *clientStream) get1xxTraceFunc() func(int, textproto.MIMEHeader) error return traceGot1xxResponseFunc(cs.trace) } -// awaitRequestCancel waits for the user to cancel a request, its context to -// expire, or for the request to be done (any way it might be removed from the -// cc.streams map: peer reset, successful completion, TCP connection breakage, -// etc). If the request is canceled, then cs will be canceled and closed. -func (cs *clientStream) awaitRequestCancel(req *http.Request) { - if err := awaitRequestCancel(req, cs.done); err != nil { - cs.cancelStream() - cs.bufPipe.CloseWithError(err) - } +func (cs *clientStream) abortStream(err error) { + cs.cc.mu.Lock() + defer cs.cc.mu.Unlock() + cs.abortStreamLocked(err) } -func (cs *clientStream) cancelStream() { - cc := cs.cc - cc.mu.Lock() - didReset := cs.didReset - cs.didReset = true - cc.mu.Unlock() - - if !didReset { - cc.writeStreamReset(cs.ID, ErrCodeCancel, nil) - cc.forgetStreamID(cs.ID) +func (cs *clientStream) abortStreamLocked(err error) { + cs.abortOnce.Do(func() { + cs.abortErr = err + close(cs.abort) + }) + if cs.reqBody != nil && !cs.reqBodyClosed { + cs.reqBody.Close() + cs.reqBodyClosed = true } -} - -// checkResetOrDone reports any error sent in a RST_STREAM frame by the -// server, or errStreamClosed if the stream is complete. -func (cs *clientStream) checkResetOrDone() error { - select { - case <-cs.peerReset: - return cs.resetErr - case <-cs.done: - return errStreamClosed - default: - return nil + // TODO(dneil): Clean up tests where cs.cc.cond is nil. + if cs.cc.cond != nil { + // Wake up writeRequestBody if it is waiting on flow control. + cs.cc.cond.Broadcast() } } -func (cs *clientStream) getStartedWrite() bool { +func (cs *clientStream) abortRequestBodyWrite() { cc := cs.cc cc.mu.Lock() defer cc.mu.Unlock() - return cs.startedWrite -} - -func (cs *clientStream) abortRequestBodyWrite(err error) { - if err == nil { - panic("nil error") - } - cc := cs.cc - cc.mu.Lock() - if cs.stopReqBody == nil { - cs.stopReqBody = err - if cs.req.Body != nil { - cs.req.Body.Close() - } + if cs.reqBody != nil && !cs.reqBodyClosed { + cs.reqBody.Close() + cs.reqBodyClosed = true cc.cond.Broadcast() } - cc.mu.Unlock() } type stickyErrWriter struct { - w io.Writer - err *error + conn net.Conn + timeout time.Duration + err *error } func (sew stickyErrWriter) Write(p []byte) (n int, err error) { if *sew.err != nil { return 0, *sew.err } - n, err = sew.w.Write(p) - *sew.err = err - return + for { + if sew.timeout != 0 { + sew.conn.SetWriteDeadline(time.Now().Add(sew.timeout)) + } + nn, err := sew.conn.Write(p[n:]) + n += nn + if n < len(p) && nn > 0 && errors.Is(err, os.ErrDeadlineExceeded) { + // Keep extending the deadline so long as we're making progress. + continue + } + if sew.timeout != 0 { + sew.conn.SetWriteDeadline(time.Time{}) + } + *sew.err = err + return n, err + } } // noCachedConnError is the concrete type of ErrNoCachedConn, which @@ -480,9 +496,9 @@ func (t *Transport) RoundTripOpt(req *http.Request, opt RoundTripOpt) (*http.Res } reused := !atomic.CompareAndSwapUint32(&cc.reused, 0, 1) traceGotConn(req, cc, reused) - res, gotErrAfterReqBodyWrite, err := cc.roundTrip(req) + res, err := cc.RoundTrip(req) if err != nil && retry <= 6 { - if req, err = shouldRetryRequest(req, err, gotErrAfterReqBodyWrite); err == nil { + if req, err = shouldRetryRequest(req, err); err == nil { // After the first retry, do exponential backoff with 10% jitter. if retry == 0 { continue @@ -493,7 +509,7 @@ func (t *Transport) RoundTripOpt(req *http.Request, opt RoundTripOpt) (*http.Res case <-time.After(time.Second * time.Duration(backoff)): continue case <-req.Context().Done(): - return nil, req.Context().Err() + err = req.Context().Err() } } } @@ -524,7 +540,7 @@ var ( // response headers. It is always called with a non-nil error. // It returns either a request to retry (either the same request, or a // modified clone), or an error if the request can't be replayed. -func shouldRetryRequest(req *http.Request, err error, afterBodyWrite bool) (*http.Request, error) { +func shouldRetryRequest(req *http.Request, err error) (*http.Request, error) { if !canRetryError(err) { return nil, err } @@ -537,7 +553,6 @@ func shouldRetryRequest(req *http.Request, err error, afterBodyWrite bool) (*htt // If the request body can be reset back to its original // state via the optional req.GetBody, do that. if req.GetBody != nil { - // TODO: consider a req.Body.Close here? or audit that all caller paths do? body, err := req.GetBody() if err != nil { return nil, err @@ -549,10 +564,8 @@ func shouldRetryRequest(req *http.Request, err error, afterBodyWrite bool) (*htt // The Request.Body can't reset back to the beginning, but we // don't seem to have started to read from it yet, so reuse - // the request directly. The "afterBodyWrite" means the - // bodyWrite process has started, which becomes true before - // the first Read. - if !afterBodyWrite { + // the request directly. + if err == errClientConnUnusable { return req, nil } @@ -642,14 +655,15 @@ func (t *Transport) newClientConn(c net.Conn, singleUse bool) (*ClientConn, erro tconn: c, readerDone: make(chan struct{}), nextStreamID: 1, - maxFrameSize: 16 << 10, // spec default - initialWindowSize: 65535, // spec default - maxConcurrentStreams: 1000, // "infinite", per spec. 1000 seems good enough. - peerMaxHeaderListSize: 0xffffffffffffffff, // "infinite", per spec. Use 2^64-1 instead. + maxFrameSize: 16 << 10, // spec default + initialWindowSize: 65535, // spec default + maxConcurrentStreams: initialMaxConcurrentStreams, // "infinite", per spec. Use a smaller value until we have received server settings. + peerMaxHeaderListSize: 0xffffffffffffffff, // "infinite", per spec. Use 2^64-1 instead. streams: make(map[uint32]*clientStream), singleUse: singleUse, wantSettingsAck: true, pings: make(map[[8]byte]chan struct{}), + reqHeaderMu: make(chan struct{}, 1), } if d := t.idleConnTimeout(); d != 0 { cc.idleTimeout = d @@ -664,9 +678,16 @@ func (t *Transport) newClientConn(c net.Conn, singleUse bool) (*ClientConn, erro // TODO: adjust this writer size to account for frame size + // MTU + crypto/tls record padding. - cc.bw = bufio.NewWriter(stickyErrWriter{c, &cc.werr}) + cc.bw = bufio.NewWriter(stickyErrWriter{ + conn: c, + timeout: t.WriteByteTimeout, + err: &cc.werr, + }) cc.br = bufio.NewReader(c) cc.fr = NewFramer(cc.bw, cc.br) + if t.CountError != nil { + cc.fr.countError = t.CountError + } cc.fr.ReadMetaHeaders = hpack.NewDecoder(initialHeaderTableSize, nil) cc.fr.MaxHeaderListSize = t.maxHeaderListSize() @@ -743,27 +764,94 @@ func (cc *ClientConn) setGoAway(f *GoAwayFrame) { last := f.LastStreamID for streamID, cs := range cc.streams { if streamID > last { - select { - case cs.resc <- resAndError{err: errClientConnGotGoAway}: - default: - } + cs.abortStreamLocked(errClientConnGotGoAway) } } } // CanTakeNewRequest reports whether the connection can take a new request, // meaning it has not been closed or received or sent a GOAWAY. +// +// If the caller is going to immediately make a new request on this +// connection, use ReserveNewRequest instead. func (cc *ClientConn) CanTakeNewRequest() bool { cc.mu.Lock() defer cc.mu.Unlock() return cc.canTakeNewRequestLocked() } +// ReserveNewRequest is like CanTakeNewRequest but also reserves a +// concurrent stream in cc. The reservation is decremented on the +// next call to RoundTrip. +func (cc *ClientConn) ReserveNewRequest() bool { + cc.mu.Lock() + defer cc.mu.Unlock() + if st := cc.idleStateLocked(); !st.canTakeNewRequest { + return false + } + cc.streamsReserved++ + return true +} + +// ClientConnState describes the state of a ClientConn. +type ClientConnState struct { + // Closed is whether the connection is closed. + Closed bool + + // Closing is whether the connection is in the process of + // closing. It may be closing due to shutdown, being a + // single-use connection, being marked as DoNotReuse, or + // having received a GOAWAY frame. + Closing bool + + // StreamsActive is how many streams are active. + StreamsActive int + + // StreamsReserved is how many streams have been reserved via + // ClientConn.ReserveNewRequest. + StreamsReserved int + + // StreamsPending is how many requests have been sent in excess + // of the peer's advertised MaxConcurrentStreams setting and + // are waiting for other streams to complete. + StreamsPending int + + // MaxConcurrentStreams is how many concurrent streams the + // peer advertised as acceptable. Zero means no SETTINGS + // frame has been received yet. + MaxConcurrentStreams uint32 + + // LastIdle, if non-zero, is when the connection last + // transitioned to idle state. + LastIdle time.Time +} + +// State returns a snapshot of cc's state. +func (cc *ClientConn) State() ClientConnState { + cc.wmu.Lock() + maxConcurrent := cc.maxConcurrentStreams + if !cc.seenSettings { + maxConcurrent = 0 + } + cc.wmu.Unlock() + + cc.mu.Lock() + defer cc.mu.Unlock() + return ClientConnState{ + Closed: cc.closed, + Closing: cc.closing || cc.singleUse || cc.doNotReuse || cc.goAway != nil, + StreamsActive: len(cc.streams), + StreamsReserved: cc.streamsReserved, + StreamsPending: cc.pendingRequests, + LastIdle: cc.lastIdle, + MaxConcurrentStreams: maxConcurrent, + } +} + // clientConnIdleState describes the suitability of a client // connection to initiate a new RoundTrip request. type clientConnIdleState struct { canTakeNewRequest bool - freshConn bool // whether it's unused by any previous request } func (cc *ClientConn) idleState() clientConnIdleState { @@ -784,14 +872,13 @@ func (cc *ClientConn) idleStateLocked() (st clientConnIdleState) { // writing it. maxConcurrentOkay = true } else { - maxConcurrentOkay = int64(len(cc.streams)+1) < int64(cc.maxConcurrentStreams) + maxConcurrentOkay = int64(len(cc.streams)+cc.streamsReserved+1) <= int64(cc.maxConcurrentStreams) } st.canTakeNewRequest = cc.goAway == nil && !cc.closed && !cc.closing && maxConcurrentOkay && !cc.doNotReuse && int64(cc.nextStreamID)+2*int64(cc.pendingRequests) < math.MaxInt32 && !cc.tooIdleLocked() - st.freshConn = cc.nextStreamID == 1 && st.canTakeNewRequest return } @@ -822,7 +909,7 @@ func (cc *ClientConn) onIdleTimeout() { func (cc *ClientConn) closeIfIdle() { cc.mu.Lock() - if len(cc.streams) > 0 { + if len(cc.streams) > 0 || cc.streamsReserved > 0 { cc.mu.Unlock() return } @@ -837,9 +924,15 @@ func (cc *ClientConn) closeIfIdle() { cc.tconn.Close() } +func (cc *ClientConn) isDoNotReuseAndIdle() bool { + cc.mu.Lock() + defer cc.mu.Unlock() + return cc.doNotReuse && len(cc.streams) == 0 +} + var shutdownEnterWaitStateHook = func() {} -// Shutdown gracefully close the client connection, waiting for running streams to complete. +// Shutdown gracefully closes the client connection, waiting for running streams to complete. func (cc *ClientConn) Shutdown(ctx context.Context) error { if err := cc.sendGoAway(); err != nil { return err @@ -878,15 +971,18 @@ func (cc *ClientConn) Shutdown(ctx context.Context) error { func (cc *ClientConn) sendGoAway() error { cc.mu.Lock() - defer cc.mu.Unlock() - cc.wmu.Lock() - defer cc.wmu.Unlock() - if cc.closing { + closing := cc.closing + cc.closing = true + maxStreamID := cc.nextStreamID + cc.mu.Unlock() + if closing { // GOAWAY sent already return nil } + + cc.wmu.Lock() + defer cc.wmu.Unlock() // Send a graceful shutdown frame to server - maxStreamID := cc.nextStreamID if err := cc.fr.WriteGoAway(maxStreamID, ErrCodeNo, nil); err != nil { return err } @@ -894,7 +990,6 @@ func (cc *ClientConn) sendGoAway() error { return err } // Prevent new requests - cc.closing = true return nil } @@ -902,17 +997,12 @@ func (cc *ClientConn) sendGoAway() error { // err is sent to streams. func (cc *ClientConn) closeForError(err error) error { cc.mu.Lock() + cc.closed = true + for _, cs := range cc.streams { + cs.abortStreamLocked(err) + } defer cc.cond.Broadcast() defer cc.mu.Unlock() - for id, cs := range cc.streams { - select { - case cs.resc <- resAndError{err: err}: - default: - } - cs.bufPipe.CloseWithError(err) - delete(cc.streams, id) - } - cc.closed = true return cc.tconn.Close() } @@ -927,6 +1017,9 @@ func (cc *ClientConn) Close() error { // closes the client connection immediately. In-flight requests are interrupted. func (cc *ClientConn) closeForLostPing() error { err := errors.New("http2: client connection lost") + if f := cc.t.CountError; f != nil { + f("conn_close_lost_ping") + } return cc.closeForError(err) } @@ -991,41 +1084,142 @@ func actualContentLength(req *http.Request) int64 { return -1 } +func (cc *ClientConn) decrStreamReservations() { + cc.mu.Lock() + defer cc.mu.Unlock() + cc.decrStreamReservationsLocked() +} + +func (cc *ClientConn) decrStreamReservationsLocked() { + if cc.streamsReserved > 0 { + cc.streamsReserved-- + } +} + func (cc *ClientConn) RoundTrip(req *http.Request) (*http.Response, error) { - resp, _, err := cc.roundTrip(req) - return resp, err + ctx := req.Context() + cs := &clientStream{ + cc: cc, + ctx: ctx, + reqCancel: req.Cancel, + isHead: req.Method == "HEAD", + reqBody: req.Body, + reqBodyContentLength: actualContentLength(req), + trace: httptrace.ContextClientTrace(ctx), + peerClosed: make(chan struct{}), + abort: make(chan struct{}), + respHeaderRecv: make(chan struct{}), + donec: make(chan struct{}), + } + go cs.doRequest(req) + + waitDone := func() error { + select { + case <-cs.donec: + return nil + case <-ctx.Done(): + return ctx.Err() + case <-cs.reqCancel: + return errRequestCanceled + } + } + + for { + select { + case <-cs.respHeaderRecv: + res := cs.res + if res.StatusCode > 299 { + // On error or status code 3xx, 4xx, 5xx, etc abort any + // ongoing write, assuming that the server doesn't care + // about our request body. If the server replied with 1xx or + // 2xx, however, then assume the server DOES potentially + // want our body (e.g. full-duplex streaming: + // golang.org/issue/13444). If it turns out the server + // doesn't, they'll RST_STREAM us soon enough. This is a + // heuristic to avoid adding knobs to Transport. Hopefully + // we can keep it. + cs.abortRequestBodyWrite() + } + res.Request = req + res.TLS = cc.tlsState + if res.Body == noBody && actualContentLength(req) == 0 { + // If there isn't a request or response body still being + // written, then wait for the stream to be closed before + // RoundTrip returns. + if err := waitDone(); err != nil { + return nil, err + } + } + return res, nil + case <-cs.abort: + waitDone() + return nil, cs.abortErr + case <-ctx.Done(): + err := ctx.Err() + cs.abortStream(err) + return nil, err + case <-cs.reqCancel: + cs.abortStream(errRequestCanceled) + return nil, errRequestCanceled + } + } +} + +// doRequest runs for the duration of the request lifetime. +// +// It sends the request and performs post-request cleanup (closing Request.Body, etc.). +func (cs *clientStream) doRequest(req *http.Request) { + err := cs.writeRequest(req) + cs.cleanupWriteRequest(err) } -func (cc *ClientConn) roundTrip(req *http.Request) (res *http.Response, gotErrAfterReqBodyWrite bool, err error) { +// writeRequest sends a request. +// +// It returns nil after the request is written, the response read, +// and the request stream is half-closed by the peer. +// +// It returns non-nil if the request ends otherwise. +// If the returned error is StreamError, the error Code may be used in resetting the stream. +func (cs *clientStream) writeRequest(req *http.Request) (err error) { + cc := cs.cc + ctx := cs.ctx + if err := checkConnHeaders(req); err != nil { - return nil, false, err - } - if cc.idleTimer != nil { - cc.idleTimer.Stop() + return err } - trailers, err := commaSeparatedTrailers(req) - if err != nil { - return nil, false, err + // Acquire the new-request lock by writing to reqHeaderMu. + // This lock guards the critical section covering allocating a new stream ID + // (requires mu) and creating the stream (requires wmu). + if cc.reqHeaderMu == nil { + panic("RoundTrip on uninitialized ClientConn") // for tests + } + select { + case cc.reqHeaderMu <- struct{}{}: + case <-cs.reqCancel: + return errRequestCanceled + case <-ctx.Done(): + return ctx.Err() } - hasTrailers := trailers != "" cc.mu.Lock() - if err := cc.awaitOpenSlotForRequest(req); err != nil { + if cc.idleTimer != nil { + cc.idleTimer.Stop() + } + cc.decrStreamReservationsLocked() + if err := cc.awaitOpenSlotForStreamLocked(cs); err != nil { cc.mu.Unlock() - return nil, false, err + <-cc.reqHeaderMu + return err } - - body := req.Body - contentLen := actualContentLength(req) - hasBody := contentLen != 0 + cc.addStreamLocked(cs) // assigns stream ID + cc.mu.Unlock() // TODO(bradfitz): this is a copy of the logic in net/http. Unify somewhere? - var requestedGzip bool if !cc.t.disableCompression() && req.Header.Get("Accept-Encoding") == "" && req.Header.Get("Range") == "" && - req.Method != "HEAD" { + !cs.isHead { // Request gzip only, not deflate. Deflate is ambiguous and // not as universally supported anyway. // See: https://zlib.net/zlib_faq.html#faq39 @@ -1038,183 +1232,223 @@ func (cc *ClientConn) roundTrip(req *http.Request) (res *http.Response, gotErrAf // We don't request gzip if the request is for a range, since // auto-decoding a portion of a gzipped document will just fail // anyway. See https://golang.org/issue/8923 - requestedGzip = true + cs.requestedGzip = true } - // we send: HEADERS{1}, CONTINUATION{0,} + DATA{0,} (DATA is - // sent by writeRequestBody below, along with any Trailers, - // again in form HEADERS{1}, CONTINUATION{0,}) - hdrs, err := cc.encodeHeaders(req, requestedGzip, trailers, contentLen) + continueTimeout := cc.t.expectContinueTimeout() + if continueTimeout != 0 && + !httpguts.HeaderValuesContainsToken( + req.Header["Expect"], + "100-continue") { + continueTimeout = 0 + cs.on100 = make(chan struct{}, 1) + } + + // Past this point (where we send request headers), it is possible for + // RoundTrip to return successfully. Since the RoundTrip contract permits + // the caller to "mutate or reuse" the Request after closing the Response's Body, + // we must take care when referencing the Request from here on. + err = cs.encodeAndWriteHeaders(req) + <-cc.reqHeaderMu if err != nil { - cc.mu.Unlock() - return nil, false, err + return err } - cs := cc.newStream() - cs.req = req - cs.trace = httptrace.ContextClientTrace(req.Context()) - cs.requestedGzip = requestedGzip - bodyWriter := cc.t.getBodyWriterState(cs, body) - cs.on100 = bodyWriter.on100 + hasBody := cs.reqBodyContentLength != 0 + if !hasBody { + cs.sentEndStream = true + } else { + if continueTimeout != 0 { + traceWait100Continue(cs.trace) + timer := time.NewTimer(continueTimeout) + select { + case <-timer.C: + err = nil + case <-cs.on100: + err = nil + case <-cs.abort: + err = cs.abortErr + case <-ctx.Done(): + err = ctx.Err() + case <-cs.reqCancel: + err = errRequestCanceled + } + timer.Stop() + if err != nil { + traceWroteRequest(cs.trace, err) + return err + } + } - defer func() { - cc.wmu.Lock() - werr := cc.werr - cc.wmu.Unlock() - if werr != nil { - cc.Close() + if err = cs.writeRequestBody(req); err != nil { + if err != errStopReqBodyWrite { + traceWroteRequest(cs.trace, err) + return err + } + } else { + cs.sentEndStream = true } - }() + } + + traceWroteRequest(cs.trace, err) + + var respHeaderTimer <-chan time.Time + var respHeaderRecv chan struct{} + if d := cc.responseHeaderTimeout(); d != 0 { + timer := time.NewTimer(d) + defer timer.Stop() + respHeaderTimer = timer.C + respHeaderRecv = cs.respHeaderRecv + } + // Wait until the peer half-closes its end of the stream, + // or until the request is aborted (via context, error, or otherwise), + // whichever comes first. + for { + select { + case <-cs.peerClosed: + return nil + case <-respHeaderTimer: + return errTimeout + case <-respHeaderRecv: + respHeaderTimer = nil // keep waiting for END_STREAM + case <-cs.abort: + return cs.abortErr + case <-ctx.Done(): + return ctx.Err() + case <-cs.reqCancel: + return errRequestCanceled + } + } +} + +func (cs *clientStream) encodeAndWriteHeaders(req *http.Request) error { + cc := cs.cc + ctx := cs.ctx cc.wmu.Lock() + defer cc.wmu.Unlock() + + // If the request was canceled while waiting for cc.mu, just quit. + select { + case <-cs.abort: + return cs.abortErr + case <-ctx.Done(): + return ctx.Err() + case <-cs.reqCancel: + return errRequestCanceled + default: + } + + // Encode headers. + // + // we send: HEADERS{1}, CONTINUATION{0,} + DATA{0,} (DATA is + // sent by writeRequestBody below, along with any Trailers, + // again in form HEADERS{1}, CONTINUATION{0,}) + trailers, err := commaSeparatedTrailers(req) + if err != nil { + return err + } + hasTrailers := trailers != "" + contentLen := actualContentLength(req) + hasBody := contentLen != 0 + hdrs, err := cc.encodeHeaders(req, cs.requestedGzip, trailers, contentLen) + if err != nil { + return err + } + + // Write the request. endStream := !hasBody && !hasTrailers - werr := cc.writeHeaders(cs.ID, endStream, int(cc.maxFrameSize), hdrs) - cc.wmu.Unlock() + cs.sentHeaders = true + err = cc.writeHeaders(cs.ID, endStream, int(cc.maxFrameSize), hdrs) traceWroteHeaders(cs.trace) - cc.mu.Unlock() + return err +} - if werr != nil { - if hasBody { - req.Body.Close() // per RoundTripper contract - bodyWriter.cancel() - } - cc.forgetStreamID(cs.ID) - // Don't bother sending a RST_STREAM (our write already failed; - // no need to keep writing) - traceWroteRequest(cs.trace, werr) - return nil, false, werr - } +// cleanupWriteRequest performs post-request tasks. +// +// If err (the result of writeRequest) is non-nil and the stream is not closed, +// cleanupWriteRequest will send a reset to the peer. +func (cs *clientStream) cleanupWriteRequest(err error) { + cc := cs.cc - var respHeaderTimer <-chan time.Time - if hasBody { - bodyWriter.scheduleBodyWrite() - } else { - traceWroteRequest(cs.trace, nil) - if d := cc.responseHeaderTimeout(); d != 0 { - timer := time.NewTimer(d) - defer timer.Stop() - respHeaderTimer = timer.C - } + if cs.ID == 0 { + // We were canceled before creating the stream, so return our reservation. + cc.decrStreamReservations() } - readLoopResCh := cs.resc - bodyWritten := false - ctx := req.Context() + // TODO: write h12Compare test showing whether + // Request.Body is closed by the Transport, + // and in multiple cases: server replies <=299 and >299 + // while still writing request body + cc.mu.Lock() + bodyClosed := cs.reqBodyClosed + cs.reqBodyClosed = true + cc.mu.Unlock() + if !bodyClosed && cs.reqBody != nil { + cs.reqBody.Close() + } - handleReadLoopResponse := func(re resAndError) (*http.Response, bool, error) { - res := re.res - if re.err != nil || res.StatusCode > 299 { - // On error or status code 3xx, 4xx, 5xx, etc abort any - // ongoing write, assuming that the server doesn't care - // about our request body. If the server replied with 1xx or - // 2xx, however, then assume the server DOES potentially - // want our body (e.g. full-duplex streaming: - // golang.org/issue/13444). If it turns out the server - // doesn't, they'll RST_STREAM us soon enough. This is a - // heuristic to avoid adding knobs to Transport. Hopefully - // we can keep it. - bodyWriter.cancel() - cs.abortRequestBodyWrite(errStopReqBodyWrite) - if hasBody && !bodyWritten { - <-bodyWriter.resc + if err != nil && cs.sentEndStream { + // If the connection is closed immediately after the response is read, + // we may be aborted before finishing up here. If the stream was closed + // cleanly on both sides, there is no error. + select { + case <-cs.peerClosed: + err = nil + default: + } + } + if err != nil { + cs.abortStream(err) // possibly redundant, but harmless + if cs.sentHeaders { + if se, ok := err.(StreamError); ok { + if se.Cause != errFromPeer { + cc.writeStreamReset(cs.ID, se.Code, err) + } + } else { + cc.writeStreamReset(cs.ID, ErrCodeCancel, err) } } - if re.err != nil { - cc.forgetStreamID(cs.ID) - return nil, cs.getStartedWrite(), re.err + cs.bufPipe.CloseWithError(err) // no-op if already closed + } else { + if cs.sentHeaders && !cs.sentEndStream { + cc.writeStreamReset(cs.ID, ErrCodeNo, nil) } - res.Request = req - res.TLS = cc.tlsState - return res, false, nil + cs.bufPipe.CloseWithError(errRequestCanceled) } - - handleError := func(err error) (*http.Response, bool, error) { - if !hasBody || bodyWritten { - cc.writeStreamReset(cs.ID, ErrCodeCancel, nil) - } else { - bodyWriter.cancel() - cs.abortRequestBodyWrite(errStopReqBodyWriteAndCancel) - <-bodyWriter.resc - } + if cs.ID != 0 { cc.forgetStreamID(cs.ID) - return nil, cs.getStartedWrite(), err } - for { - select { - case re := <-readLoopResCh: - return handleReadLoopResponse(re) - case <-respHeaderTimer: - return handleError(errTimeout) - case <-ctx.Done(): - return handleError(ctx.Err()) - case <-req.Cancel: - return handleError(errRequestCanceled) - case <-cs.peerReset: - // processResetStream already removed the - // stream from the streams map; no need for - // forgetStreamID. - return nil, cs.getStartedWrite(), cs.resetErr - case err := <-bodyWriter.resc: - bodyWritten = true - // Prefer the read loop's response, if available. Issue 16102. - select { - case re := <-readLoopResCh: - return handleReadLoopResponse(re) - default: - } - if err != nil { - cc.forgetStreamID(cs.ID) - return nil, cs.getStartedWrite(), err - } - if d := cc.responseHeaderTimeout(); d != 0 { - timer := time.NewTimer(d) - defer timer.Stop() - respHeaderTimer = timer.C - } - } + cc.wmu.Lock() + werr := cc.werr + cc.wmu.Unlock() + if werr != nil { + cc.Close() } + + close(cs.donec) } -// awaitOpenSlotForRequest waits until len(streams) < maxConcurrentStreams. +// awaitOpenSlotForStream waits until len(streams) < maxConcurrentStreams. // Must hold cc.mu. -func (cc *ClientConn) awaitOpenSlotForRequest(req *http.Request) error { - var waitingForConn chan struct{} - var waitingForConnErr error // guarded by cc.mu +func (cc *ClientConn) awaitOpenSlotForStreamLocked(cs *clientStream) error { for { cc.lastActive = time.Now() if cc.closed || !cc.canTakeNewRequestLocked() { - if waitingForConn != nil { - close(waitingForConn) - } return errClientConnUnusable } cc.lastIdle = time.Time{} - if int64(len(cc.streams))+1 <= int64(cc.maxConcurrentStreams) { - if waitingForConn != nil { - close(waitingForConn) - } + if int64(len(cc.streams)) < int64(cc.maxConcurrentStreams) { return nil } - // Unfortunately, we cannot wait on a condition variable and channel at - // the same time, so instead, we spin up a goroutine to check if the - // request is canceled while we wait for a slot to open in the connection. - if waitingForConn == nil { - waitingForConn = make(chan struct{}) - go func() { - if err := awaitRequestCancel(req, waitingForConn); err != nil { - cc.mu.Lock() - waitingForConnErr = err - cc.cond.Broadcast() - cc.mu.Unlock() - } - }() - } cc.pendingRequests++ cc.cond.Wait() cc.pendingRequests-- - if waitingForConnErr != nil { - return waitingForConnErr + select { + case <-cs.abort: + return cs.abortErr + default: } } } @@ -1241,10 +1475,6 @@ func (cc *ClientConn) writeHeaders(streamID uint32, endStream bool, maxFrameSize cc.fr.WriteContinuation(streamID, endHeaders, chunk) } } - // TODO(bradfitz): this Flush could potentially block (as - // could the WriteHeaders call(s) above), which means they - // wouldn't respond to Request.Cancel being readable. That's - // rare, but this should probably be in a goroutine. cc.bw.Flush() return cc.werr } @@ -1271,7 +1501,7 @@ func (cs *clientStream) frameScratchBufferLen(maxFrameSize int) int { if n > max { n = max } - if cl := actualContentLength(cs.req); cl != -1 && cl+1 < n { + if cl := cs.reqBodyContentLength; cl != -1 && cl+1 < n { // Add an extra byte past the declared content-length to // give the caller's Request.Body io.Reader a chance to // give us more bytes than they declared, so we can catch it @@ -1286,31 +1516,13 @@ func (cs *clientStream) frameScratchBufferLen(maxFrameSize int) int { var bufPool sync.Pool // of *[]byte -func (cs *clientStream) writeRequestBody(body io.Reader, bodyCloser io.Closer) (err error) { +func (cs *clientStream) writeRequestBody(req *http.Request) (err error) { cc := cs.cc + body := cs.reqBody sentEnd := false // whether we sent the final DATA frame w/ END_STREAM - defer func() { - traceWroteRequest(cs.trace, err) - // TODO: write h12Compare test showing whether - // Request.Body is closed by the Transport, - // and in multiple cases: server replies <=299 and >299 - // while still writing request body - var cerr error - cc.mu.Lock() - if cs.stopReqBody == nil { - cs.stopReqBody = errStopReqBodyWrite - cerr = bodyCloser.Close() - } - cc.mu.Unlock() - if err == nil { - err = cerr - } - }() - - req := cs.req hasTrailers := req.Trailer != nil - remainLen := actualContentLength(req) + remainLen := cs.reqBodyContentLength hasContentLen := remainLen != -1 cc.mu.Lock() @@ -1348,29 +1560,29 @@ func (cs *clientStream) writeRequestBody(body io.Reader, bodyCloser io.Closer) ( } if remainLen < 0 { err = errReqBodyTooLong - cc.writeStreamReset(cs.ID, ErrCodeCancel, err) return err } } - if err == io.EOF { - sawEOF = true - err = nil - } else if err != nil { - cc.writeStreamReset(cs.ID, ErrCodeCancel, err) - return err + if err != nil { + cc.mu.Lock() + bodyClosed := cs.reqBodyClosed + cc.mu.Unlock() + switch { + case bodyClosed: + return errStopReqBodyWrite + case err == io.EOF: + sawEOF = true + err = nil + default: + return err + } } remain := buf[:n] for len(remain) > 0 && err == nil { var allowed int32 allowed, err = cs.awaitFlowControl(len(remain)) - switch { - case err == errStopReqBodyWrite: - return err - case err == errStopReqBodyWriteAndCancel: - cc.writeStreamReset(cs.ID, ErrCodeCancel, nil) - return err - case err != nil: + if err != nil { return err } cc.wmu.Lock() @@ -1401,21 +1613,27 @@ func (cs *clientStream) writeRequestBody(body io.Reader, bodyCloser io.Closer) ( return nil } + // Since the RoundTrip contract permits the caller to "mutate or reuse" + // a request after the Response's Body is closed, verify that this hasn't + // happened before accessing the trailers. + cc.mu.Lock() + trailer := req.Trailer + err = cs.abortErr + cc.mu.Unlock() + if err != nil { + return err + } + + cc.wmu.Lock() + defer cc.wmu.Unlock() var trls []byte - if hasTrailers { - cc.mu.Lock() - trls, err = cc.encodeTrailers(req) - cc.mu.Unlock() + if len(trailer) > 0 { + trls, err = cc.encodeTrailers(trailer) if err != nil { - cc.writeStreamReset(cs.ID, ErrCodeInternal, err) - cc.forgetStreamID(cs.ID) return err } } - cc.wmu.Lock() - defer cc.wmu.Unlock() - // Two ways to send END_STREAM: either with trailers, or // with an empty DATA frame. if len(trls) > 0 { @@ -1435,17 +1653,24 @@ func (cs *clientStream) writeRequestBody(body io.Reader, bodyCloser io.Closer) ( // if the stream is dead. func (cs *clientStream) awaitFlowControl(maxBytes int) (taken int32, err error) { cc := cs.cc + ctx := cs.ctx cc.mu.Lock() defer cc.mu.Unlock() for { if cc.closed { return 0, errClientConnClosed } - if cs.stopReqBody != nil { - return 0, cs.stopReqBody + if cs.reqBodyClosed { + return 0, errStopReqBodyWrite } - if err := cs.checkResetOrDone(); err != nil { - return 0, err + select { + case <-cs.abort: + return 0, cs.abortErr + case <-ctx.Done(): + return 0, ctx.Err() + case <-cs.reqCancel: + return 0, errRequestCanceled + default: } if a := cs.flow.available(); a > 0 { take := a @@ -1463,9 +1688,14 @@ func (cs *clientStream) awaitFlowControl(maxBytes int) (taken int32, err error) } } -// requires cc.mu be held. +var errNilRequestURL = errors.New("http2: Request.URI is nil") + +// requires cc.wmu be held. func (cc *ClientConn) encodeHeaders(req *http.Request, addGzipHeader bool, trailers string, contentLength int64) ([]byte, error) { cc.hbuf.Reset() + if req.URL == nil { + return nil, errNilRequestURL + } host := req.Host if host == "" { @@ -1651,12 +1881,12 @@ func shouldSendReqContentLength(method string, contentLength int64) bool { } } -// requires cc.mu be held. -func (cc *ClientConn) encodeTrailers(req *http.Request) ([]byte, error) { +// requires cc.wmu be held. +func (cc *ClientConn) encodeTrailers(trailer http.Header) ([]byte, error) { cc.hbuf.Reset() hlSize := uint64(0) - for k, vv := range req.Trailer { + for k, vv := range trailer { for _, v := range vv { hf := hpack.HeaderField{Name: k, Value: v} hlSize += uint64(hf.Size()) @@ -1666,7 +1896,7 @@ func (cc *ClientConn) encodeTrailers(req *http.Request) ([]byte, error) { return nil, errRequestHeaderListSize } - for k, vv := range req.Trailer { + for k, vv := range trailer { lowKey, ascii := asciiToLower(k) if !ascii { // Skip writing invalid headers. Per RFC 7540, Section 8.1.2, header @@ -1696,51 +1926,51 @@ type resAndError struct { } // requires cc.mu be held. -func (cc *ClientConn) newStream() *clientStream { - cs := &clientStream{ - cc: cc, - ID: cc.nextStreamID, - resc: make(chan resAndError, 1), - peerReset: make(chan struct{}), - done: make(chan struct{}), - } +func (cc *ClientConn) addStreamLocked(cs *clientStream) { cs.flow.add(int32(cc.initialWindowSize)) cs.flow.setConnFlow(&cc.flow) cs.inflow.add(transportDefaultStreamFlow) cs.inflow.setConnFlow(&cc.inflow) + cs.ID = cc.nextStreamID cc.nextStreamID += 2 cc.streams[cs.ID] = cs - return cs + if cs.ID == 0 { + panic("assigned stream ID 0") + } } func (cc *ClientConn) forgetStreamID(id uint32) { - cc.streamByID(id, true) -} - -func (cc *ClientConn) streamByID(id uint32, andRemove bool) *clientStream { cc.mu.Lock() - defer cc.mu.Unlock() - cs := cc.streams[id] - if andRemove && cs != nil && !cc.closed { - cc.lastActive = time.Now() - delete(cc.streams, id) - if len(cc.streams) == 0 && cc.idleTimer != nil { - cc.idleTimer.Reset(cc.idleTimeout) - cc.lastIdle = time.Now() - } - close(cs.done) - // Wake up checkResetOrDone via clientStream.awaitFlowControl and - // wake up RoundTrip if there is a pending request. - cc.cond.Broadcast() + slen := len(cc.streams) + delete(cc.streams, id) + if len(cc.streams) != slen-1 { + panic("forgetting unknown stream id") + } + cc.lastActive = time.Now() + if len(cc.streams) == 0 && cc.idleTimer != nil { + cc.idleTimer.Reset(cc.idleTimeout) + cc.lastIdle = time.Now() + } + // Wake up writeRequestBody via clientStream.awaitFlowControl and + // wake up RoundTrip if there is a pending request. + cc.cond.Broadcast() + + closeOnIdle := cc.singleUse || cc.doNotReuse || cc.t.disableKeepAlives() + if closeOnIdle && cc.streamsReserved == 0 && len(cc.streams) == 0 { + if VerboseLogs { + cc.vlogf("http2: Transport closing idle conn %p (forSingleUse=%v, maxStream=%v)", cc, cc.singleUse, cc.nextStreamID-2) + } + cc.closed = true + defer cc.tconn.Close() } - return cs + + cc.mu.Unlock() } // clientConnReadLoop is the state owned by the clientConn's frame-reading readLoop. type clientConnReadLoop struct { - _ incomparable - cc *ClientConn - closeWhenIdle bool + _ incomparable + cc *ClientConn } // readLoop runs in its own goroutine and reads and dispatches frames. @@ -1800,23 +2030,49 @@ func (rl *clientConnReadLoop) cleanup() { } else if err == io.EOF { err = io.ErrUnexpectedEOF } + cc.closed = true for _, cs := range cc.streams { - cs.bufPipe.CloseWithError(err) // no-op if already closed select { - case cs.resc <- resAndError{err: err}: + case <-cs.peerClosed: + // The server closed the stream before closing the conn, + // so no need to interrupt it. default: + cs.abortStreamLocked(err) } - close(cs.done) } - cc.closed = true cc.cond.Broadcast() cc.mu.Unlock() } +// countReadFrameError calls Transport.CountError with a string +// representing err. +func (cc *ClientConn) countReadFrameError(err error) { + f := cc.t.CountError + if f == nil || err == nil { + return + } + if ce, ok := err.(ConnectionError); ok { + errCode := ErrCode(ce) + f(fmt.Sprintf("read_frame_conn_error_%s", errCode.stringToken())) + return + } + if errors.Is(err, io.EOF) { + f("read_frame_eof") + return + } + if errors.Is(err, io.ErrUnexpectedEOF) { + f("read_frame_unexpected_eof") + return + } + if errors.Is(err, ErrFrameTooLarge) { + f("read_frame_too_large") + return + } + f("read_frame_other") +} + func (rl *clientConnReadLoop) run() error { cc := rl.cc - rl.closeWhenIdle = cc.t.disableKeepAlives() || cc.singleUse - gotReply := false // ever saw a HEADERS reply gotSettings := false readIdleTimeout := cc.t.ReadIdleTimeout var t *time.Timer @@ -1833,9 +2089,7 @@ func (rl *clientConnReadLoop) run() error { cc.vlogf("http2: Transport readFrame error on conn %p: (%T) %v", cc, err, err) } if se, ok := err.(StreamError); ok { - if cs := cc.streamByID(se.StreamID, false); cs != nil { - cs.cc.writeStreamReset(cs.ID, se.Code, err) - cs.cc.forgetStreamID(cs.ID) + if cs := rl.streamByID(se.StreamID); cs != nil { if se.Cause == nil { se.Cause = cc.fr.errDetail } @@ -1843,6 +2097,7 @@ func (rl *clientConnReadLoop) run() error { } continue } else if err != nil { + cc.countReadFrameError(err) return err } if VerboseLogs { @@ -1855,22 +2110,16 @@ func (rl *clientConnReadLoop) run() error { } gotSettings = true } - maybeIdle := false // whether frame might transition us to idle switch f := f.(type) { case *MetaHeadersFrame: err = rl.processHeaders(f) - maybeIdle = true - gotReply = true case *DataFrame: err = rl.processData(f) - maybeIdle = true case *GoAwayFrame: err = rl.processGoAway(f) - maybeIdle = true case *RSTStreamFrame: err = rl.processResetStream(f) - maybeIdle = true case *SettingsFrame: err = rl.processSettings(f) case *PushPromiseFrame: @@ -1888,38 +2137,24 @@ func (rl *clientConnReadLoop) run() error { } return err } - if rl.closeWhenIdle && gotReply && maybeIdle { - cc.closeIfIdle() - } } } func (rl *clientConnReadLoop) processHeaders(f *MetaHeadersFrame) error { - cc := rl.cc - cs := cc.streamByID(f.StreamID, false) + cs := rl.streamByID(f.StreamID) if cs == nil { // We'd get here if we canceled a request while the // server had its response still in flight. So if this // was just something we canceled, ignore it. return nil } - if f.StreamEnded() { - // Issue 20521: If the stream has ended, streamByID() causes - // clientStream.done to be closed, which causes the request's bodyWriter - // to be closed with an errStreamClosed, which may be received by - // clientConn.RoundTrip before the result of processing these headers. - // Deferring stream closure allows the header processing to occur first. - // clientConn.RoundTrip may still receive the bodyWriter error first, but - // the fix for issue 16102 prioritises any response. - // - // Issue 22413: If there is no request body, we should close the - // stream before writing to cs.resc so that the stream is closed - // immediately once RoundTrip returns. - if cs.req.Body != nil { - defer cc.forgetStreamID(f.StreamID) - } else { - cc.forgetStreamID(f.StreamID) - } + if cs.readClosed { + rl.endStreamError(cs, StreamError{ + StreamID: f.StreamID, + Code: ErrCodeProtocol, + Cause: errors.New("protocol error: headers after END_STREAM"), + }) + return nil } if !cs.firstByte { if cs.trace != nil { @@ -1943,9 +2178,11 @@ func (rl *clientConnReadLoop) processHeaders(f *MetaHeadersFrame) error { return err } // Any other error type is a stream error. - cs.cc.writeStreamReset(f.StreamID, ErrCodeProtocol, err) - cc.forgetStreamID(cs.ID) - cs.resc <- resAndError{err: err} + rl.endStreamError(cs, StreamError{ + StreamID: f.StreamID, + Code: ErrCodeProtocol, + Cause: err, + }) return nil // return nil from process* funcs to keep conn alive } if res == nil { @@ -1953,7 +2190,11 @@ func (rl *clientConnReadLoop) processHeaders(f *MetaHeadersFrame) error { return nil } cs.resTrailer = &res.Trailer - cs.resc <- resAndError{res: res} + cs.res = res + close(cs.respHeaderRecv) + if f.StreamEnded() { + rl.endStream(cs) + } return nil } @@ -2015,6 +2256,9 @@ func (rl *clientConnReadLoop) handleResponse(cs *clientStream, f *MetaHeadersFra } if statusCode >= 100 && statusCode <= 199 { + if f.StreamEnded() { + return nil, errors.New("1xx informational response with END_STREAM flag") + } cs.num1xx++ const max1xxResponses = 5 // arbitrary bound on number of informational responses, same as net/http if cs.num1xx > max1xxResponses { @@ -2027,40 +2271,47 @@ func (rl *clientConnReadLoop) handleResponse(cs *clientStream, f *MetaHeadersFra } if statusCode == 100 { traceGot100Continue(cs.trace) - if cs.on100 != nil { - cs.on100() // forces any write delay timer to fire + select { + case cs.on100 <- struct{}{}: + default: } } cs.pastHeaders = false // do it all again return nil, nil } - streamEnded := f.StreamEnded() - isHead := cs.req.Method == "HEAD" - if !streamEnded || isHead { - res.ContentLength = -1 - if clens := res.Header["Content-Length"]; len(clens) == 1 { - if cl, err := strconv.ParseUint(clens[0], 10, 63); err == nil { - res.ContentLength = int64(cl) - } else { - // TODO: care? unlike http/1, it won't mess up our framing, so it's - // more safe smuggling-wise to ignore. - } - } else if len(clens) > 1 { + res.ContentLength = -1 + if clens := res.Header["Content-Length"]; len(clens) == 1 { + if cl, err := strconv.ParseUint(clens[0], 10, 63); err == nil { + res.ContentLength = int64(cl) + } else { // TODO: care? unlike http/1, it won't mess up our framing, so it's // more safe smuggling-wise to ignore. } + } else if len(clens) > 1 { + // TODO: care? unlike http/1, it won't mess up our framing, so it's + // more safe smuggling-wise to ignore. + } else if f.StreamEnded() && !cs.isHead { + res.ContentLength = 0 } - if streamEnded || isHead { + if cs.isHead { res.Body = noBody return res, nil } - cs.bufPipe = pipe{b: &dataBuffer{expected: res.ContentLength}} + if f.StreamEnded() { + if res.ContentLength > 0 { + res.Body = missingBody{} + } else { + res.Body = noBody + } + return res, nil + } + + cs.bufPipe.setBuffer(&dataBuffer{expected: res.ContentLength}) cs.bytesRemain = res.ContentLength res.Body = transportResponseBody{cs} - go cs.awaitRequestCancel(cs.req) if cs.requestedGzip && res.Header.Get("Content-Encoding") == "gzip" { res.Header.Del("Content-Encoding") @@ -2101,8 +2352,7 @@ func (rl *clientConnReadLoop) processTrailers(cs *clientStream, f *MetaHeadersFr } // transportResponseBody is the concrete type of Transport.RoundTrip's -// Response.Body. It is an io.ReadCloser. On Read, it reads from cs.body. -// On Close it sends RST_STREAM if EOF wasn't already seen. +// Response.Body. It is an io.ReadCloser. type transportResponseBody struct { cs *clientStream } @@ -2120,7 +2370,7 @@ func (b transportResponseBody) Read(p []byte) (n int, err error) { n = int(cs.bytesRemain) if err == nil { err = errors.New("net/http: server replied with more than declared Content-Length; truncated") - cc.writeStreamReset(cs.ID, ErrCodeProtocol, err) + cs.abortStream(err) } cs.readErr = err return int(cs.bytesRemain), err @@ -2138,8 +2388,6 @@ func (b transportResponseBody) Read(p []byte) (n int, err error) { } cc.mu.Lock() - defer cc.mu.Unlock() - var connAdd, streamAdd int32 // Check the conn-level first, before the stream-level. if v := cc.inflow.available(); v < transportDefaultConnFlow/2 { @@ -2156,6 +2404,8 @@ func (b transportResponseBody) Read(p []byte) (n int, err error) { cs.inflow.add(streamAdd) } } + cc.mu.Unlock() + if connAdd != 0 || streamAdd != 0 { cc.wmu.Lock() defer cc.wmu.Unlock() @@ -2176,34 +2426,42 @@ func (b transportResponseBody) Close() error { cs := b.cs cc := cs.cc - serverSentStreamEnd := cs.bufPipe.Err() == io.EOF unread := cs.bufPipe.Len() - - if unread > 0 || !serverSentStreamEnd { + if unread > 0 { cc.mu.Lock() - cc.wmu.Lock() - if !serverSentStreamEnd { - cc.fr.WriteRSTStream(cs.ID, ErrCodeCancel) - cs.didReset = true - } // Return connection-level flow control. if unread > 0 { cc.inflow.add(int32(unread)) + } + cc.mu.Unlock() + + // TODO(dneil): Acquiring this mutex can block indefinitely. + // Move flow control return to a goroutine? + cc.wmu.Lock() + // Return connection-level flow control. + if unread > 0 { cc.fr.WriteWindowUpdate(0, uint32(unread)) } cc.bw.Flush() cc.wmu.Unlock() - cc.mu.Unlock() } cs.bufPipe.BreakWithError(errClosedResponseBody) - cc.forgetStreamID(cs.ID) + cs.abortStream(errClosedResponseBody) + + select { + case <-cs.donec: + case <-cs.ctx.Done(): + return cs.ctx.Err() + case <-cs.reqCancel: + return errRequestCanceled + } return nil } func (rl *clientConnReadLoop) processData(f *DataFrame) error { cc := rl.cc - cs := cc.streamByID(f.StreamID, f.StreamEnded()) + cs := rl.streamByID(f.StreamID) data := f.Data() if cs == nil { cc.mu.Lock() @@ -2232,6 +2490,14 @@ func (rl *clientConnReadLoop) processData(f *DataFrame) error { } return nil } + if cs.readClosed { + cc.logf("protocol error: received DATA after END_STREAM") + rl.endStreamError(cs, StreamError{ + StreamID: f.StreamID, + Code: ErrCodeProtocol, + }) + return nil + } if !cs.firstByte { cc.logf("protocol error: received DATA before a HEADERS frame") rl.endStreamError(cs, StreamError{ @@ -2241,7 +2507,7 @@ func (rl *clientConnReadLoop) processData(f *DataFrame) error { return nil } if f.Length > 0 { - if cs.req.Method == "HEAD" && len(data) > 0 { + if cs.isHead && len(data) > 0 { cc.logf("protocol error: received DATA on a HEAD request") rl.endStreamError(cs, StreamError{ StreamID: f.StreamID, @@ -2263,30 +2529,39 @@ func (rl *clientConnReadLoop) processData(f *DataFrame) error { if pad := int(f.Length) - len(data); pad > 0 { refund += pad } - // Return len(data) now if the stream is already closed, - // since data will never be read. - didReset := cs.didReset - if didReset { - refund += len(data) + + didReset := false + var err error + if len(data) > 0 { + if _, err = cs.bufPipe.Write(data); err != nil { + // Return len(data) now if the stream is already closed, + // since data will never be read. + didReset = true + refund += len(data) + } } + if refund > 0 { cc.inflow.add(int32(refund)) + if !didReset { + cs.inflow.add(int32(refund)) + } + } + cc.mu.Unlock() + + if refund > 0 { cc.wmu.Lock() cc.fr.WriteWindowUpdate(0, uint32(refund)) if !didReset { - cs.inflow.add(int32(refund)) cc.fr.WriteWindowUpdate(cs.ID, uint32(refund)) } cc.bw.Flush() cc.wmu.Unlock() } - cc.mu.Unlock() - if len(data) > 0 && !didReset { - if _, err := cs.bufPipe.Write(data); err != nil { - rl.endStreamError(cs, err) - return err - } + if err != nil { + rl.endStreamError(cs, err) + return nil } } @@ -2299,24 +2574,26 @@ func (rl *clientConnReadLoop) processData(f *DataFrame) error { func (rl *clientConnReadLoop) endStream(cs *clientStream) { // TODO: check that any declared content-length matches, like // server.go's (*stream).endStream method. - rl.endStreamError(cs, nil) + if !cs.readClosed { + cs.readClosed = true + cs.bufPipe.closeWithErrorAndCode(io.EOF, cs.copyTrailers) + close(cs.peerClosed) + } } func (rl *clientConnReadLoop) endStreamError(cs *clientStream, err error) { - var code func() - if err == nil { - err = io.EOF - code = cs.copyTrailers - } - if isConnectionCloseRequest(cs.req) { - rl.closeWhenIdle = true - } - cs.bufPipe.closeWithErrorAndCode(err, code) + cs.readAborted = true + cs.abortStream(err) +} - select { - case cs.resc <- resAndError{err: err}: - default: +func (rl *clientConnReadLoop) streamByID(id uint32) *clientStream { + rl.cc.mu.Lock() + defer rl.cc.mu.Unlock() + cs := rl.cc.streams[id] + if cs != nil && !cs.readAborted { + return cs } + return nil } func (cs *clientStream) copyTrailers() { @@ -2335,12 +2612,33 @@ func (rl *clientConnReadLoop) processGoAway(f *GoAwayFrame) error { if f.ErrCode != 0 { // TODO: deal with GOAWAY more. particularly the error code cc.vlogf("transport got GOAWAY with error code = %v", f.ErrCode) + if fn := cc.t.CountError; fn != nil { + fn("recv_goaway_" + f.ErrCode.stringToken()) + } + } cc.setGoAway(f) return nil } func (rl *clientConnReadLoop) processSettings(f *SettingsFrame) error { + cc := rl.cc + // Locking both mu and wmu here allows frame encoding to read settings with only wmu held. + // Acquiring wmu when f.IsAck() is unnecessary, but convenient and mostly harmless. + cc.wmu.Lock() + defer cc.wmu.Unlock() + + if err := rl.processSettingsNoWrite(f); err != nil { + return err + } + if !f.IsAck() { + cc.fr.WriteSettingsAck() + cc.bw.Flush() + } + return nil +} + +func (rl *clientConnReadLoop) processSettingsNoWrite(f *SettingsFrame) error { cc := rl.cc cc.mu.Lock() defer cc.mu.Unlock() @@ -2353,12 +2651,14 @@ func (rl *clientConnReadLoop) processSettings(f *SettingsFrame) error { return ConnectionError(ErrCodeProtocol) } + var seenMaxConcurrentStreams bool err := f.ForeachSetting(func(s Setting) error { switch s.ID { case SettingMaxFrameSize: cc.maxFrameSize = s.Val case SettingMaxConcurrentStreams: cc.maxConcurrentStreams = s.Val + seenMaxConcurrentStreams = true case SettingMaxHeaderListSize: cc.peerMaxHeaderListSize = uint64(s.Val) case SettingInitialWindowSize: @@ -2390,17 +2690,23 @@ func (rl *clientConnReadLoop) processSettings(f *SettingsFrame) error { return err } - cc.wmu.Lock() - defer cc.wmu.Unlock() + if !cc.seenSettings { + if !seenMaxConcurrentStreams { + // This was the servers initial SETTINGS frame and it + // didn't contain a MAX_CONCURRENT_STREAMS field so + // increase the number of concurrent streams this + // connection can establish to our default. + cc.maxConcurrentStreams = defaultMaxConcurrentStreams + } + cc.seenSettings = true + } - cc.fr.WriteSettingsAck() - cc.bw.Flush() - return cc.werr + return nil } func (rl *clientConnReadLoop) processWindowUpdate(f *WindowUpdateFrame) error { cc := rl.cc - cs := cc.streamByID(f.StreamID, false) + cs := rl.streamByID(f.StreamID) if f.StreamID != 0 && cs == nil { return nil } @@ -2420,31 +2726,22 @@ func (rl *clientConnReadLoop) processWindowUpdate(f *WindowUpdateFrame) error { } func (rl *clientConnReadLoop) processResetStream(f *RSTStreamFrame) error { - cs := rl.cc.streamByID(f.StreamID, true) + cs := rl.streamByID(f.StreamID) if cs == nil { - // TODO: return error if server tries to RST_STEAM an idle stream + // TODO: return error if server tries to RST_STREAM an idle stream return nil } - select { - case <-cs.peerReset: - // Already reset. - // This is the only goroutine - // which closes this, so there - // isn't a race. - default: - serr := streamError(cs.ID, f.ErrCode) - if f.ErrCode == ErrCodeProtocol { - rl.cc.SetDoNotReuse() - serr.Cause = errFromPeer - // TODO(bradfitz): increment a varz here, once Transport - // takes an optional interface-typed field that expvar.Map.Add - // implements. - } - cs.resetErr = serr - close(cs.peerReset) - cs.bufPipe.CloseWithError(serr) - cs.cc.cond.Broadcast() // wake up checkResetOrDone via clientStream.awaitFlowControl + serr := streamError(cs.ID, f.ErrCode) + serr.Cause = errFromPeer + if f.ErrCode == ErrCodeProtocol { + rl.cc.SetDoNotReuse() + } + if fn := cs.cc.t.CountError; fn != nil { + fn("recv_rststream_" + f.ErrCode.stringToken()) } + cs.abortStream(serr) + + cs.bufPipe.CloseWithError(serr) return nil } @@ -2466,19 +2763,24 @@ func (cc *ClientConn) Ping(ctx context.Context) error { } cc.mu.Unlock() } - cc.wmu.Lock() - if err := cc.fr.WritePing(false, p); err != nil { - cc.wmu.Unlock() - return err - } - if err := cc.bw.Flush(); err != nil { - cc.wmu.Unlock() - return err - } - cc.wmu.Unlock() + errc := make(chan error, 1) + go func() { + cc.wmu.Lock() + defer cc.wmu.Unlock() + if err := cc.fr.WritePing(false, p); err != nil { + errc <- err + return + } + if err := cc.bw.Flush(); err != nil { + errc <- err + return + } + }() select { case <-c: return nil + case err := <-errc: + return err case <-ctx.Done(): return ctx.Err() case <-cc.readerDone: @@ -2555,6 +2857,11 @@ func (t *Transport) logf(format string, args ...interface{}) { var noBody io.ReadCloser = ioutil.NopCloser(bytes.NewReader(nil)) +type missingBody struct{} + +func (missingBody) Close() error { return nil } +func (missingBody) Read([]byte) (int, error) { return 0, io.ErrUnexpectedEOF } + func strSliceContains(ss []string, s string) bool { for _, v := range ss { if v == s { @@ -2600,87 +2907,6 @@ type errorReader struct{ err error } func (r errorReader) Read(p []byte) (int, error) { return 0, r.err } -// bodyWriterState encapsulates various state around the Transport's writing -// of the request body, particularly regarding doing delayed writes of the body -// when the request contains "Expect: 100-continue". -type bodyWriterState struct { - cs *clientStream - timer *time.Timer // if non-nil, we're doing a delayed write - fnonce *sync.Once // to call fn with - fn func() // the code to run in the goroutine, writing the body - resc chan error // result of fn's execution - delay time.Duration // how long we should delay a delayed write for -} - -func (t *Transport) getBodyWriterState(cs *clientStream, body io.Reader) (s bodyWriterState) { - s.cs = cs - if body == nil { - return - } - resc := make(chan error, 1) - s.resc = resc - s.fn = func() { - cs.cc.mu.Lock() - cs.startedWrite = true - cs.cc.mu.Unlock() - resc <- cs.writeRequestBody(body, cs.req.Body) - } - s.delay = t.expectContinueTimeout() - if s.delay == 0 || - !httpguts.HeaderValuesContainsToken( - cs.req.Header["Expect"], - "100-continue") { - return - } - s.fnonce = new(sync.Once) - - // Arm the timer with a very large duration, which we'll - // intentionally lower later. It has to be large now because - // we need a handle to it before writing the headers, but the - // s.delay value is defined to not start until after the - // request headers were written. - const hugeDuration = 365 * 24 * time.Hour - s.timer = time.AfterFunc(hugeDuration, func() { - s.fnonce.Do(s.fn) - }) - return -} - -func (s bodyWriterState) cancel() { - if s.timer != nil { - if s.timer.Stop() { - s.resc <- nil - } - } -} - -func (s bodyWriterState) on100() { - if s.timer == nil { - // If we didn't do a delayed write, ignore the server's - // bogus 100 continue response. - return - } - s.timer.Stop() - go func() { s.fnonce.Do(s.fn) }() -} - -// scheduleBodyWrite starts writing the body, either immediately (in -// the common case) or after the delay timeout. It should not be -// called until after the headers have been written. -func (s bodyWriterState) scheduleBodyWrite() { - if s.timer == nil { - // We're not doing a delayed write (see - // getBodyWriterState), so just start the writing - // goroutine immediately. - go s.fn() - return - } - traceWait100Continue(s.cs.trace) - if s.timer.Stop() { - s.timer.Reset(s.delay) - } -} - // isConnectionCloseRequest reports whether req should use its own // connection for a single request and then close the connection. func isConnectionCloseRequest(req *http.Request) bool { diff --git a/vendor/modules.txt b/vendor/modules.txt index c3ca4ff48567..e1f339f133f6 100644 --- a/vendor/modules.txt +++ b/vendor/modules.txt @@ -97,6 +97,9 @@ github.com/PuerkitoBio/purell # github.com/PuerkitoBio/urlesc v0.0.0-20170810143723-de5bf2ad4578 ## explicit github.com/PuerkitoBio/urlesc +# github.com/Shopify/sarama v1.30.0 +## explicit; go 1.13 +github.com/Shopify/sarama # github.com/Workiva/go-datastructures v1.0.53 ## explicit; go 1.15 github.com/Workiva/go-datastructures/rangetree @@ -418,6 +421,15 @@ github.com/drone/envsubst/path # github.com/dustin/go-humanize v1.0.0 ## explicit github.com/dustin/go-humanize +# github.com/eapache/go-resiliency v1.2.0 +## explicit +github.com/eapache/go-resiliency/breaker +# github.com/eapache/go-xerial-snappy v0.0.0-20180814174437-776d5712da21 +## explicit +github.com/eapache/go-xerial-snappy +# github.com/eapache/queue v1.1.0 +## explicit +github.com/eapache/queue # github.com/edsrzf/mmap-go v1.0.0 ## explicit github.com/edsrzf/mmap-go @@ -684,6 +696,9 @@ github.com/hashicorp/go-rootcerts # github.com/hashicorp/go-sockaddr v1.0.2 ## explicit github.com/hashicorp/go-sockaddr +# github.com/hashicorp/go-uuid v1.0.2 +## explicit +github.com/hashicorp/go-uuid # github.com/hashicorp/golang-lru v0.5.4 ## explicit; go 1.12 github.com/hashicorp/golang-lru @@ -718,6 +733,52 @@ github.com/influxdata/go-syslog/v3/rfc5424 ## explicit; go 1.15 github.com/influxdata/telegraf github.com/influxdata/telegraf/plugins/inputs +# github.com/jcmturner/aescts/v2 v2.0.0 +## explicit; go 1.13 +github.com/jcmturner/aescts/v2 +# github.com/jcmturner/dnsutils/v2 v2.0.0 +## explicit; go 1.13 +github.com/jcmturner/dnsutils/v2 +# github.com/jcmturner/gofork v1.0.0 +## explicit +github.com/jcmturner/gofork/encoding/asn1 +github.com/jcmturner/gofork/x/crypto/pbkdf2 +# github.com/jcmturner/gokrb5/v8 v8.4.2 +## explicit; go 1.14 +github.com/jcmturner/gokrb5/v8/asn1tools +github.com/jcmturner/gokrb5/v8/client +github.com/jcmturner/gokrb5/v8/config +github.com/jcmturner/gokrb5/v8/credentials +github.com/jcmturner/gokrb5/v8/crypto +github.com/jcmturner/gokrb5/v8/crypto/common +github.com/jcmturner/gokrb5/v8/crypto/etype +github.com/jcmturner/gokrb5/v8/crypto/rfc3961 +github.com/jcmturner/gokrb5/v8/crypto/rfc3962 +github.com/jcmturner/gokrb5/v8/crypto/rfc4757 +github.com/jcmturner/gokrb5/v8/crypto/rfc8009 +github.com/jcmturner/gokrb5/v8/gssapi +github.com/jcmturner/gokrb5/v8/iana +github.com/jcmturner/gokrb5/v8/iana/addrtype +github.com/jcmturner/gokrb5/v8/iana/adtype +github.com/jcmturner/gokrb5/v8/iana/asnAppTag +github.com/jcmturner/gokrb5/v8/iana/chksumtype +github.com/jcmturner/gokrb5/v8/iana/errorcode +github.com/jcmturner/gokrb5/v8/iana/etypeID +github.com/jcmturner/gokrb5/v8/iana/flags +github.com/jcmturner/gokrb5/v8/iana/keyusage +github.com/jcmturner/gokrb5/v8/iana/msgtype +github.com/jcmturner/gokrb5/v8/iana/nametype +github.com/jcmturner/gokrb5/v8/iana/patype +github.com/jcmturner/gokrb5/v8/kadmin +github.com/jcmturner/gokrb5/v8/keytab +github.com/jcmturner/gokrb5/v8/krberror +github.com/jcmturner/gokrb5/v8/messages +github.com/jcmturner/gokrb5/v8/pac +github.com/jcmturner/gokrb5/v8/types +# github.com/jcmturner/rpc/v2 v2.0.3 +## explicit; go 1.13 +github.com/jcmturner/rpc/v2/mstypes +github.com/jcmturner/rpc/v2/ndr # github.com/jessevdk/go-flags v1.5.0 ## explicit; go 1.15 github.com/jessevdk/go-flags @@ -739,12 +800,14 @@ github.com/json-iterator/go # github.com/julienschmidt/httprouter v1.3.0 ## explicit; go 1.7 github.com/julienschmidt/httprouter -# github.com/klauspost/compress v1.13.1 -## explicit; go 1.13 +# github.com/klauspost/compress v1.13.6 +## explicit; go 1.15 +github.com/klauspost/compress github.com/klauspost/compress/flate github.com/klauspost/compress/fse github.com/klauspost/compress/gzip github.com/klauspost/compress/huff0 +github.com/klauspost/compress/internal/snapref github.com/klauspost/compress/zstd github.com/klauspost/compress/zstd/internal/xxhash # github.com/klauspost/cpuid v1.3.1 @@ -862,6 +925,10 @@ github.com/opentracing-contrib/go-stdlib/nethttp github.com/opentracing/opentracing-go github.com/opentracing/opentracing-go/ext github.com/opentracing/opentracing-go/log +# github.com/pierrec/lz4 v2.6.1+incompatible +## explicit +github.com/pierrec/lz4 +github.com/pierrec/lz4/internal/xxh32 # github.com/pierrec/lz4/v4 v4.1.7 ## explicit; go 1.14 github.com/pierrec/lz4/v4 @@ -1013,6 +1080,9 @@ github.com/prometheus/prometheus/util/teststorage github.com/prometheus/prometheus/util/testutil github.com/prometheus/prometheus/util/treecache github.com/prometheus/prometheus/web/api/v1 +# github.com/rcrowley/go-metrics v0.0.0-20201227073835-cf1acfcdf475 +## explicit +github.com/rcrowley/go-metrics # github.com/rs/cors v1.8.0 ## explicit; go 1.13 github.com/rs/cors @@ -1240,12 +1310,13 @@ go4.org/intern # go4.org/unsafe/assume-no-moving-gc v0.0.0-20201222180813-1025295fd063 ## explicit; go 1.11 go4.org/unsafe/assume-no-moving-gc -# golang.org/x/crypto v0.0.0-20210616213533-5ff15b29337e +# golang.org/x/crypto v0.0.0-20210921155107-089bfa567519 ## explicit; go 1.17 golang.org/x/crypto/argon2 golang.org/x/crypto/bcrypt golang.org/x/crypto/blake2b golang.org/x/crypto/blowfish +golang.org/x/crypto/md4 golang.org/x/crypto/pbkdf2 golang.org/x/crypto/pkcs12 golang.org/x/crypto/pkcs12/internal/rc2 @@ -1255,7 +1326,7 @@ golang.org/x/crypto/sha3 ## explicit; go 1.11 golang.org/x/lint golang.org/x/lint/golint -# golang.org/x/net v0.0.0-20210903162142-ad29c8ab022f +# golang.org/x/net v0.0.0-20211101193420-4a448f8816b3 ## explicit; go 1.17 golang.org/x/net/bpf golang.org/x/net/context @@ -1301,8 +1372,8 @@ golang.org/x/sys/windows/svc/eventlog # golang.org/x/term v0.0.0-20210220032956-6a3ed077a48d ## explicit; go 1.11 golang.org/x/term -# golang.org/x/text v0.3.6 -## explicit; go 1.11 +# golang.org/x/text v0.3.7 +## explicit; go 1.17 golang.org/x/text/secure/bidirule golang.org/x/text/transform golang.org/x/text/unicode/bidi