aboutsummaryrefslogtreecommitdiff
path: root/weed/replication/sub/notification_kafka.go
blob: 622a759eac1e4cc4947de41fa75bec3743fa7f64 (plain)
1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19
20
21
22
23
24
25
26
27
28
29
30
31
32
33
34
35
36
37
38
39
40
41
42
43
44
45
46
47
48
49
50
51
52
53
54
55
56
57
58
59
60
61
62
63
64
65
66
67
68
69
70
71
72
73
74
75
76
77
78
79
80
81
82
83
84
85
86
87
88
89
90
91
92
93
94
95
96
97
98
99
100
101
102
103
104
105
106
107
108
109
110
111
112
113
114
115
116
117
118
119
120
121
122
123
124
125
126
127
128
129
130
131
132
133
134
135
136
137
138
139
140
141
142
143
144
145
146
147
148
149
150
151
152
153
154
155
156
157
158
package sub

import (
	"encoding/json"
	"fmt"
	"io/ioutil"
	"sync"
	"time"

	"github.com/Shopify/sarama"
	"github.com/chrislusf/seaweedfs/weed/glog"
	"github.com/chrislusf/seaweedfs/weed/pb/filer_pb"
	"github.com/chrislusf/seaweedfs/weed/util"
	"github.com/golang/protobuf/proto"
)

func init() {
	NotificationInputs = append(NotificationInputs, &KafkaInput{})
}

type KafkaInput struct {
	topic       string
	consumer    sarama.Consumer
	messageChan chan *sarama.ConsumerMessage
}

func (k *KafkaInput) GetName() string {
	return "kafka"
}

func (k *KafkaInput) Initialize(configuration util.Configuration, prefix string) error {
	glog.V(0).Infof("replication.notification.kafka.hosts: %v\n", configuration.GetStringSlice(prefix+"hosts"))
	glog.V(0).Infof("replication.notification.kafka.topic: %v\n", configuration.GetString(prefix+"topic"))
	return k.initialize(
		configuration.GetStringSlice(prefix+"hosts"),
		configuration.GetString(prefix+"topic"),
		configuration.GetString(prefix+"offsetFile"),
		configuration.GetInt(prefix+"offsetSaveIntervalSeconds"),
	)
}

func (k *KafkaInput) initialize(hosts []string, topic string, offsetFile string, offsetSaveIntervalSeconds int) (err error) {
	config := sarama.NewConfig()
	config.Consumer.Return.Errors = true
	k.consumer, err = sarama.NewConsumer(hosts, config)
	if err != nil {
		panic(err)
	} else {
		glog.V(0).Infof("connected to %v", hosts)
	}

	k.topic = topic
	k.messageChan = make(chan *sarama.ConsumerMessage, 1)

	partitions, err := k.consumer.Partitions(topic)
	if err != nil {
		panic(err)
	}

	progress := loadProgress(offsetFile)
	if progress == nil || progress.Topic != topic {
		progress = &KafkaProgress{
			Topic:            topic,
			PartitionOffsets: make(map[int32]int64),
		}
	}
	progress.lastSaveTime = time.Now()
	progress.offsetFile = offsetFile
	progress.offsetSaveIntervalSeconds = offsetSaveIntervalSeconds

	for _, partition := range partitions {
		offset, found := progress.PartitionOffsets[partition]
		if !found {
			offset = sarama.OffsetOldest
		} else {
			offset += 1
		}
		partitionConsumer, err := k.consumer.ConsumePartition(topic, partition, offset)
		if err != nil {
			panic(err)
		}
		go func() {
			for {
				select {
				case err := <-partitionConsumer.Errors():
					fmt.Println(err)
				case msg := <-partitionConsumer.Messages():
					k.messageChan <- msg
					if err := progress.setOffset(msg.Partition, msg.Offset); err != nil {
						glog.Warningf("set kafka offset: %v", err)
					}
				}
			}
		}()
	}

	return nil
}

func (k *KafkaInput) ReceiveMessage() (key string, message *filer_pb.EventNotification, onSuccessFn func(), onFailureFn func(), err error) {

	msg := <-k.messageChan

	key = string(msg.Key)
	message = &filer_pb.EventNotification{}
	err = proto.Unmarshal(msg.Value, message)

	return
}

type KafkaProgress struct {
	Topic                     string          `json:"topic"`
	PartitionOffsets          map[int32]int64 `json:"partitionOffsets"`
	offsetFile                string
	lastSaveTime              time.Time
	offsetSaveIntervalSeconds int
	sync.Mutex
}

func loadProgress(offsetFile string) *KafkaProgress {
	progress := &KafkaProgress{}
	data, err := ioutil.ReadFile(offsetFile)
	if err != nil {
		glog.Warningf("failed to read kafka progress file: %s", offsetFile)
		return nil
	}
	err = json.Unmarshal(data, progress)
	if err != nil {
		glog.Warningf("failed to read kafka progress message: %s", string(data))
		return nil
	}
	return progress
}

func (progress *KafkaProgress) saveProgress() error {
	data, err := json.Marshal(progress)
	if err != nil {
		return fmt.Errorf("failed to marshal progress: %v", err)
	}
	err = ioutil.WriteFile(progress.offsetFile, data, 0640)
	if err != nil {
		return fmt.Errorf("failed to save progress to %s: %v", progress.offsetFile, err)
	}

	progress.lastSaveTime = time.Now()
	return nil
}

func (progress *KafkaProgress) setOffset(parition int32, offset int64) error {
	progress.Lock()
	defer progress.Unlock()

	progress.PartitionOffsets[parition] = offset
	if int(time.Now().Sub(progress.lastSaveTime).Seconds()) > progress.offsetSaveIntervalSeconds {
		return progress.saveProgress()
	}
	return nil
}