-
Notifications
You must be signed in to change notification settings - Fork 119
/
clickhouse.go
247 lines (213 loc) · 6.01 KB
/
clickhouse.go
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
159
160
161
162
163
164
165
166
167
168
169
170
171
172
173
174
175
176
177
178
179
180
181
182
183
184
185
186
187
188
189
190
191
192
193
194
195
196
197
198
199
200
201
202
203
204
205
206
207
208
209
210
211
212
213
214
215
216
217
218
219
220
221
222
223
224
225
226
227
228
229
230
231
232
233
234
235
236
237
238
239
240
241
242
243
244
245
246
247
/*Copyright [2019] housepower
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.
*/
package output
import (
"fmt"
"regexp"
"strings"
"time"
"github.com/housepower/clickhouse_sinker/model"
"github.com/housepower/clickhouse_sinker/pool"
"github.com/housepower/clickhouse_sinker/statistics"
"github.com/housepower/clickhouse_sinker/util"
"github.com/housepower/clickhouse_sinker/prom"
"github.com/sundy-li/go_commons/log"
"github.com/sundy-li/go_commons/utils"
)
// ClickHouse is an output service consumers from kafka messages
type ClickHouse struct {
Name string
TableName string
Db string
Host string
Port int
// Clickhouse database config
Clickhouse string
Username string
Password string
DsnParams string
MaxLifeTime time.Duration
RetryTimes int
AutoSchema bool
ExcludeColumns []string
// Table Configs
Dims []*model.ColumnWithType
Metrics []*model.ColumnWithType
prepareSQL string
dmMap map[string]*model.ColumnWithType
dms []string
}
// NewClickHouse new a clickhouse instance
func NewClickHouse() *ClickHouse {
return &ClickHouse{}
}
// Init the clickhouse intance
func (c *ClickHouse) Init() error {
return c.initAll()
}
// Write kvs to clickhouse
func (c *ClickHouse) Write(metrics []model.Metric) (err error) {
if len(metrics) == 0 {
return
}
conn := pool.GetConn(c.Host)
tx, err := conn.Begin()
if err != nil {
if shouldReconnect(err) {
_ = conn.ReConnect()
}
return err
}
stmt, err := tx.Prepare(c.prepareSQL)
if err != nil {
log.Error("prepareSQL:", err.Error())
if shouldReconnect(err) {
_ = conn.ReConnect()
}
return err
}
defer stmt.Close()
for _, metric := range metrics {
prom.ClickhouseEventsTotal.WithLabelValues(c.Db, c.TableName).Inc()
var args = make([]interface{}, len(c.dmMap))
for i, name := range c.dms {
args[i] = util.GetValueByType(metric, c.dmMap[name])
}
if _, err := stmt.Exec(args...); err != nil {
prom.ClickhouseEventsErrors.WithLabelValues(c.Db, c.TableName).Inc()
log.Error("execSQL:", err.Error())
return err
}
prom.ClickhouseEventsSuccess.WithLabelValues(c.Db, c.TableName).Inc()
}
if err = tx.Commit(); err != nil {
if shouldReconnect(err) {
_ = conn.ReConnect()
}
return nil
}
return nil
}
func shouldReconnect(err error) bool {
if strings.Contains(err.Error(), "connection refused") || strings.Contains(err.Error(), "bad connection") {
return true
}
log.Info("not match reconnect rules", err.Error())
return false
}
// LoopWrite will dead loop to write the records
func (c *ClickHouse) LoopWrite(metrics []model.Metric) {
err := c.Write(metrics)
times := c.RetryTimes
for err != nil && times > 0 {
log.Error("saving msg error", err.Error(), "will loop to write the data")
statistics.UpdateFlushErrorsTotal(c.Name, 1)
time.Sleep(1 * time.Second)
err = c.Write(metrics)
times--
}
}
// Close does nothing, place holder for handling close
func (c *ClickHouse) Close() error {
return nil
}
// GetName return the name of this instance of clickhouse client
func (c *ClickHouse) GetName() string {
return c.Name
}
// Description describes this instance
func (c *ClickHouse) Description() string {
return "clickhouse desc"
}
// initAll initialises schema and connections for clickhouse
func (c *ClickHouse) initAll() error {
if err := c.initConn(); err != nil {
return err
}
if err := c.initSchema(); err != nil {
return err
}
return nil
}
func (c *ClickHouse) initSchema() (err error) {
if c.AutoSchema {
conn := pool.GetConn(c.Host)
rs, err := conn.Query(fmt.Sprintf(
"select name, type from system.columns where database = '%s' and table = '%s'", c.Db, c.TableName))
if err != nil {
return err
}
c.Dims = make([]*model.ColumnWithType, 0, 10)
c.Metrics = make([]*model.ColumnWithType, 0, 10)
var name, typ string
for rs.Next() {
_ = rs.Scan(&name, &typ)
typ = lowCardinalityRegexp.ReplaceAllString(typ, "$1")
if !util.StringContains(c.ExcludeColumns, name) {
c.Dims = append(c.Dims, &model.ColumnWithType{Name: name, Type: typ})
}
}
}
//根据 dms 生成prepare的sql语句
c.dmMap = make(map[string]*model.ColumnWithType)
c.dms = make([]string, 0, len(c.Dims)+len(c.Metrics))
for i, d := range c.Dims {
c.dmMap[d.Name] = c.Dims[i]
c.dms = append(c.dms, d.Name)
}
for i, m := range c.Metrics {
c.dmMap[m.Name] = c.Metrics[i]
c.dms = append(c.dms, m.Name)
}
var params = make([]string, len(c.dmMap))
for i := range params {
params[i] = "?"
}
c.prepareSQL = "INSERT INTO " + c.Db + "." + c.TableName + " (" + strings.Join(c.dms, ",") + ") " +
"VALUES (" + strings.Join(params, ",") + ")"
log.Info("Prepare sql=>", c.prepareSQL)
return nil
}
func (c *ClickHouse) initConn() (err error) {
var hosts []string
// if contains ',', that means it's a ip list
if strings.Contains(c.Host, ",") {
hosts = strings.Split(strings.TrimSpace(c.Host), ",")
} else {
ips, err := utils.GetIp4Byname(c.Host)
if err != nil {
// fallback to ip
ips = []string{c.Host}
}
for _, ip := range ips {
hosts = append(hosts, fmt.Sprintf("%s:%d", ip, c.Port))
}
}
var dsn = fmt.Sprintf("tcp://%s?database=%s&username=%s&password=%s", hosts[0], c.Db, c.Username, c.Password)
if len(hosts) > 1 {
otherHosts := hosts[1:]
dsn += "&alt_hosts="
dsn += strings.Join(otherHosts, ",")
dsn += "&connection_open_strategy=random"
}
if c.DsnParams != "" {
dsn += "&" + c.DsnParams
}
// dsn += "&debug=1"
for i := 0; i < len(hosts); i++ {
pool.SetDsn(c.Host, dsn, c.MaxLifeTime)
}
return nil
}
var (
lowCardinalityRegexp = regexp.MustCompile(`LowCardinality\((.+)\)`)
)