-
Notifications
You must be signed in to change notification settings - Fork 226
/
backuper.go
451 lines (417 loc) · 15.6 KB
/
backuper.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
248
249
250
251
252
253
254
255
256
257
258
259
260
261
262
263
264
265
266
267
268
269
270
271
272
273
274
275
276
277
278
279
280
281
282
283
284
285
286
287
288
289
290
291
292
293
294
295
296
297
298
299
300
301
302
303
304
305
306
307
308
309
310
311
312
313
314
315
316
317
318
319
320
321
322
323
324
325
326
327
328
329
330
331
332
333
334
335
336
337
338
339
340
341
342
343
344
345
346
347
348
349
350
351
352
353
354
355
356
357
358
359
360
361
362
363
364
365
366
367
368
369
370
371
372
373
374
375
376
377
378
379
380
381
382
383
384
385
386
387
388
389
390
391
392
393
394
395
396
397
398
399
400
401
402
403
404
405
406
407
408
409
410
411
412
413
414
415
416
417
418
419
420
421
422
423
424
425
426
427
428
429
430
431
432
433
434
435
436
437
438
439
440
441
442
443
444
445
446
447
448
449
450
451
package backup
import (
"context"
"errors"
"fmt"
"github.com/Altinity/clickhouse-backup/v2/pkg/metadata"
"net/url"
"os"
"path"
"strings"
"github.com/Altinity/clickhouse-backup/v2/pkg/clickhouse"
"github.com/Altinity/clickhouse-backup/v2/pkg/config"
"github.com/Altinity/clickhouse-backup/v2/pkg/resumable"
"github.com/Altinity/clickhouse-backup/v2/pkg/storage"
"github.com/rs/zerolog/log"
)
const DirectoryFormat = "directory"
var errShardOperationUnsupported = errors.New("sharded operations are not supported")
// versioner is an interface for determining the version of Clickhouse
type versioner interface {
CanShardOperation(ctx context.Context) error
}
type BackuperOpt func(*Backuper)
type Backuper struct {
cfg *config.Config
ch *clickhouse.ClickHouse
vers versioner
bs backupSharder
dst *storage.BackupDestination
DiskToPathMap map[string]string
DefaultDataPath string
EmbeddedBackupDataPath string
isEmbedded bool
resume bool
resumableState *resumable.State
}
func NewBackuper(cfg *config.Config, opts ...BackuperOpt) *Backuper {
ch := &clickhouse.ClickHouse{
Config: &cfg.ClickHouse,
}
b := &Backuper{
cfg: cfg,
ch: ch,
vers: ch,
bs: nil,
}
for _, opt := range opts {
opt(b)
}
return b
}
func WithVersioner(v versioner) BackuperOpt {
return func(b *Backuper) {
b.vers = v
}
}
func WithBackupSharder(s backupSharder) BackuperOpt {
return func(b *Backuper) {
b.bs = s
}
}
func (b *Backuper) initDisksPathsAndBackupDestination(ctx context.Context, disks []clickhouse.Disk, backupName string) error {
var err error
if disks == nil {
disks, err = b.ch.GetDisks(ctx, true)
if err != nil {
return err
}
}
b.DefaultDataPath, err = b.ch.GetDefaultPath(disks)
if err != nil {
return ErrUnknownClickhouseDataPath
}
diskMap := map[string]string{}
for _, disk := range disks {
diskMap[disk.Name] = disk.Path
if b.cfg.ClickHouse.UseEmbeddedBackupRestore && (disk.IsBackup || disk.Name == b.cfg.ClickHouse.EmbeddedBackupDisk) {
b.EmbeddedBackupDataPath = disk.Path
}
}
if b.cfg.ClickHouse.UseEmbeddedBackupRestore && b.EmbeddedBackupDataPath == "" {
b.EmbeddedBackupDataPath = b.DefaultDataPath
}
b.DiskToPathMap = diskMap
if b.cfg.General.RemoteStorage != "none" && b.cfg.General.RemoteStorage != "custom" {
if err = b.CalculateMaxSize(ctx); err != nil {
return err
}
b.dst, err = storage.NewBackupDestination(ctx, b.cfg, b.ch, backupName)
if err != nil {
return err
}
if err := b.dst.Connect(ctx); err != nil {
return fmt.Errorf("can't connect to %s: %v", b.dst.Kind(), err)
}
}
return nil
}
// CalculateMaxSize https://github.com/Altinity/clickhouse-backup/issues/404
func (b *Backuper) CalculateMaxSize(ctx context.Context) error {
maxFileSize, err := b.ch.CalculateMaxFileSize(ctx, b.cfg)
if err != nil {
return err
}
if b.cfg.General.MaxFileSize > 0 && b.cfg.General.MaxFileSize < maxFileSize {
log.Warn().Msgf("MAX_FILE_SIZE=%d is less than actual %d, please remove general->max_file_size section from your config", b.cfg.General.MaxFileSize, maxFileSize)
}
if b.cfg.General.MaxFileSize <= 0 || b.cfg.General.MaxFileSize < maxFileSize {
b.cfg.General.MaxFileSize = maxFileSize
}
return nil
}
func (b *Backuper) getLocalBackupDataPathForTable(backupName string, disk string, dbAndTablePath string) string {
backupPath := path.Join(b.DiskToPathMap[disk], "backup", backupName, "shadow", dbAndTablePath, disk)
if b.isEmbedded {
backupPath = path.Join(b.DiskToPathMap[disk], backupName, "data", dbAndTablePath)
}
return backupPath
}
// populateBackupShardField populates the BackupShard field for a slice of Table structs
func (b *Backuper) populateBackupShardField(ctx context.Context, tables []clickhouse.Table) error {
// By default, have all fields populated to full backup unless the table is to be skipped
for i := range tables {
tables[i].BackupType = clickhouse.ShardBackupFull
if tables[i].Skip {
tables[i].BackupType = clickhouse.ShardBackupNone
}
}
if !doesShard(b.cfg.General.ShardedOperationMode) {
return nil
}
if err := b.vers.CanShardOperation(ctx); err != nil {
return err
}
if b.bs == nil {
// Parse shard config here to avoid error return in NewBackuper
shardFunc, err := shardFuncByName(b.cfg.General.ShardedOperationMode)
if err != nil {
return fmt.Errorf("could not determine shards for tables: %w", err)
}
b.bs = newReplicaDeterminer(b.ch, shardFunc)
}
assignment, err := b.bs.determineShards(ctx)
if err != nil {
return err
}
for i, t := range tables {
if t.Skip {
continue
}
fullBackup, err := assignment.inShard(t.Database, t.Name)
if err != nil {
return err
}
if !fullBackup {
tables[i].BackupType = clickhouse.ShardBackupSchema
}
}
return nil
}
func (b *Backuper) isDiskTypeObject(diskType string) bool {
return diskType == "s3" || diskType == "azure_blob_storage" || diskType == "azure"
}
func (b *Backuper) isDiskTypeEncryptedObject(disk clickhouse.Disk, disks []clickhouse.Disk) bool {
if disk.Type != "encrypted" {
return false
}
underlyingIdx := -1
underlyingPath := ""
for i, d := range disks {
if d.Name != disk.Name && strings.HasPrefix(disk.Path, d.Path) && b.isDiskTypeObject(d.Type) {
if d.Path > underlyingPath {
underlyingIdx = i
underlyingPath = d.Path
}
}
}
return underlyingIdx >= 0
}
// getEmbeddedRestoreSettings - different with getEmbeddedBackupSettings, cause https://github.com/ClickHouse/ClickHouse/issues/69053
func (b *Backuper) getEmbeddedRestoreSettings(version int) []string {
settings := []string{}
if (b.cfg.General.RemoteStorage == "s3" || b.cfg.General.RemoteStorage == "gcs") && version >= 23007000 {
settings = append(settings, "allow_s3_native_copy=1")
if err := b.ch.Query("SET s3_request_timeout_ms=600000"); err != nil {
log.Fatal().Msgf("SET s3_request_timeout_ms=600000 error: %v", err)
}
}
if (b.cfg.General.RemoteStorage == "s3" || b.cfg.General.RemoteStorage == "gcs") && version >= 23011000 {
if err := b.ch.Query("SET s3_use_adaptive_timeouts=0"); err != nil {
log.Fatal().Msgf("SET s3_use_adaptive_timeouts=0 error: %v", err)
}
}
return settings
}
func (b *Backuper) getEmbeddedBackupSettings(version int) []string {
settings := []string{}
if (b.cfg.General.RemoteStorage == "s3" || b.cfg.General.RemoteStorage == "gcs") && version >= 23007000 {
settings = append(settings, "allow_s3_native_copy=1")
if err := b.ch.Query("SET s3_request_timeout_ms=600000"); err != nil {
log.Fatal().Msgf("SET s3_request_timeout_ms=600000 error: %v", err)
}
}
if (b.cfg.General.RemoteStorage == "s3" || b.cfg.General.RemoteStorage == "gcs") && version >= 23011000 {
if err := b.ch.Query("SET s3_use_adaptive_timeouts=0"); err != nil {
log.Fatal().Msgf("SET s3_use_adaptive_timeouts=0 error: %v", err)
}
}
if b.cfg.General.RemoteStorage == "azblob" && version >= 24005000 && b.cfg.ClickHouse.EmbeddedBackupDisk == "" {
settings = append(settings, "allow_azure_native_copy=1")
}
return settings
}
func (b *Backuper) getEmbeddedBackupLocation(ctx context.Context, backupName string) (string, error) {
if b.cfg.ClickHouse.EmbeddedBackupDisk != "" {
return fmt.Sprintf("Disk('%s','%s')", b.cfg.ClickHouse.EmbeddedBackupDisk, backupName), nil
}
if err := b.applyMacrosToObjectDiskPath(ctx); err != nil {
return "", err
}
if b.cfg.General.RemoteStorage == "s3" {
s3Endpoint, err := b.ch.ApplyMacros(ctx, b.buildEmbeddedLocationS3())
if err != nil {
return "", err
}
if b.cfg.S3.AccessKey != "" {
return fmt.Sprintf("S3('%s/%s/','%s','%s')", s3Endpoint, backupName, b.cfg.S3.AccessKey, b.cfg.S3.SecretKey), nil
}
if os.Getenv("AWS_ACCESS_KEY_ID") != "" {
return fmt.Sprintf("S3('%s/%s/','%s','%s')", s3Endpoint, backupName, os.Getenv("AWS_ACCESS_KEY_ID"), os.Getenv("AWS_SECRET_ACCESS_KEY")), nil
}
return "", fmt.Errorf("provide s3->access_key and s3->secret_key in config to allow embedded backup without `clickhouse->embedded_backup_disk`")
}
if b.cfg.General.RemoteStorage == "gcs" {
gcsEndpoint, err := b.ch.ApplyMacros(ctx, b.buildEmbeddedLocationGCS())
if err != nil {
return "", err
}
if b.cfg.GCS.EmbeddedAccessKey != "" {
return fmt.Sprintf("S3('%s/%s/','%s','%s')", gcsEndpoint, backupName, b.cfg.GCS.EmbeddedAccessKey, b.cfg.GCS.EmbeddedSecretKey), nil
}
if os.Getenv("AWS_ACCESS_KEY_ID") != "" {
return fmt.Sprintf("S3('%s/%s/','%s','%s')", gcsEndpoint, backupName, os.Getenv("AWS_ACCESS_KEY_ID"), os.Getenv("AWS_SECRET_ACCESS_KEY")), nil
}
return "", fmt.Errorf("provide gcs->embedded_access_key and gcs->embedded_secret_key in config to allow embedded backup without `clickhouse->embedded_backup_disk`")
}
if b.cfg.General.RemoteStorage == "azblob" {
azblobEndpoint, err := b.ch.ApplyMacros(ctx, b.buildEmbeddedLocationAZBLOB())
if err != nil {
return "", err
}
if b.cfg.AzureBlob.Container != "" {
return fmt.Sprintf("AzureBlobStorage('%s','%s','%s/%s/')", azblobEndpoint, b.cfg.AzureBlob.Container, b.cfg.AzureBlob.ObjectDiskPath, backupName), nil
}
return "", fmt.Errorf("provide azblob->container and azblob->account_name, azblob->account_key in config to allow embedded backup without `clickhouse->embedded_backup_disk`")
}
return "", fmt.Errorf("empty clickhouse->embedded_backup_disk and invalid general->remote_storage: %s", b.cfg.General.RemoteStorage)
}
func (b *Backuper) applyMacrosToObjectDiskPath(ctx context.Context) error {
var err error
if b.cfg.General.RemoteStorage == "s3" {
b.cfg.S3.ObjectDiskPath, err = b.ch.ApplyMacros(ctx, b.cfg.S3.ObjectDiskPath)
} else if b.cfg.General.RemoteStorage == "gcs" {
b.cfg.GCS.ObjectDiskPath, err = b.ch.ApplyMacros(ctx, b.cfg.GCS.ObjectDiskPath)
} else if b.cfg.General.RemoteStorage == "azblob" {
b.cfg.AzureBlob.ObjectDiskPath, err = b.ch.ApplyMacros(ctx, b.cfg.AzureBlob.ObjectDiskPath)
} else if b.cfg.General.RemoteStorage == "ftp" {
b.cfg.FTP.ObjectDiskPath, err = b.ch.ApplyMacros(ctx, b.cfg.FTP.ObjectDiskPath)
} else if b.cfg.General.RemoteStorage == "sftp" {
b.cfg.SFTP.ObjectDiskPath, err = b.ch.ApplyMacros(ctx, b.cfg.SFTP.ObjectDiskPath)
} else if b.cfg.General.RemoteStorage == "cos" {
b.cfg.COS.ObjectDiskPath, err = b.ch.ApplyMacros(ctx, b.cfg.COS.ObjectDiskPath)
}
return err
}
func (b *Backuper) buildEmbeddedLocationS3() string {
s3backupURL := url.URL{}
s3backupURL.Scheme = "https"
if strings.HasPrefix(b.cfg.S3.Endpoint, "http") {
newUrl, _ := s3backupURL.Parse(b.cfg.S3.Endpoint)
s3backupURL = *newUrl
s3backupURL.Path = path.Join(b.cfg.S3.Bucket, b.cfg.S3.ObjectDiskPath)
} else {
s3backupURL.Host = b.cfg.S3.Endpoint
s3backupURL.Path = path.Join(b.cfg.S3.Bucket, b.cfg.S3.ObjectDiskPath)
}
if b.cfg.S3.DisableSSL {
s3backupURL.Scheme = "http"
}
if s3backupURL.Host == "" && b.cfg.S3.Region != "" && b.cfg.S3.ForcePathStyle {
s3backupURL.Host = "s3." + b.cfg.S3.Region + ".amazonaws.com"
s3backupURL.Path = path.Join(b.cfg.S3.Bucket, b.cfg.S3.ObjectDiskPath)
}
if s3backupURL.Host == "" && b.cfg.S3.Bucket != "" && !b.cfg.S3.ForcePathStyle {
s3backupURL.Host = b.cfg.S3.Bucket + "." + "s3." + b.cfg.S3.Region + ".amazonaws.com"
s3backupURL.Path = b.cfg.S3.ObjectDiskPath
}
return s3backupURL.String()
}
func (b *Backuper) buildEmbeddedLocationGCS() string {
gcsBackupURL := url.URL{}
gcsBackupURL.Scheme = "https"
if b.cfg.GCS.ForceHttp {
gcsBackupURL.Scheme = "http"
}
if b.cfg.GCS.Endpoint != "" {
if !strings.HasPrefix(b.cfg.GCS.Endpoint, "http") {
gcsBackupURL.Host = b.cfg.GCS.Endpoint
} else {
newUrl, _ := gcsBackupURL.Parse(b.cfg.GCS.Endpoint)
gcsBackupURL = *newUrl
}
}
if gcsBackupURL.Host == "" {
gcsBackupURL.Host = "storage.googleapis.com"
}
gcsBackupURL.Path = path.Join(b.cfg.GCS.Bucket, b.cfg.GCS.ObjectDiskPath)
return gcsBackupURL.String()
}
func (b *Backuper) buildEmbeddedLocationAZBLOB() string {
azblobBackupURL := url.URL{}
azblobBackupURL.Scheme = b.cfg.AzureBlob.EndpointSchema
// https://github.com/Altinity/clickhouse-backup/issues/1031
if b.cfg.AzureBlob.EndpointSuffix == "core.windows.net" {
azblobBackupURL.Host = b.cfg.AzureBlob.AccountName + "." + b.cfg.AzureBlob.EndpointSuffix
} else {
azblobBackupURL.Host = b.cfg.AzureBlob.EndpointSuffix
azblobBackupURL.Path = b.cfg.AzureBlob.AccountName
}
return fmt.Sprintf("DefaultEndpointsProtocol=%s;AccountName=%s;AccountKey=%s;BlobEndpoint=%s;", b.cfg.AzureBlob.EndpointSchema, b.cfg.AzureBlob.AccountName, b.cfg.AzureBlob.AccountKey, azblobBackupURL.String())
}
func (b *Backuper) getObjectDiskPath() (string, error) {
if b.cfg.General.RemoteStorage == "s3" {
return b.cfg.S3.ObjectDiskPath, nil
} else if b.cfg.General.RemoteStorage == "azblob" {
return b.cfg.AzureBlob.ObjectDiskPath, nil
} else if b.cfg.General.RemoteStorage == "gcs" {
return b.cfg.GCS.ObjectDiskPath, nil
} else if b.cfg.General.RemoteStorage == "cos" {
return b.cfg.COS.ObjectDiskPath, nil
} else if b.cfg.General.RemoteStorage == "ftp" {
return b.cfg.FTP.ObjectDiskPath, nil
} else if b.cfg.General.RemoteStorage == "sftp" {
return b.cfg.SFTP.ObjectDiskPath, nil
} else {
return "", fmt.Errorf("cleanBackupObjectDisks: requesst object disks path but have unsupported remote_storage: %s", b.cfg.General.RemoteStorage)
}
}
func (b *Backuper) getTablesDiffFromLocal(ctx context.Context, diffFrom string, tablePattern string) (tablesForUploadFromDiff map[metadata.TableTitle]metadata.TableMetadata, err error) {
tablesForUploadFromDiff = make(map[metadata.TableTitle]metadata.TableMetadata)
diffFromBackup, err := b.ReadBackupMetadataLocal(ctx, diffFrom)
if err != nil {
return nil, err
}
if len(diffFromBackup.Tables) != 0 {
metadataPath := path.Join(b.DefaultDataPath, "backup", diffFrom, "metadata")
// empty partitions, because we don't want filter
diffTablesList, _, err := b.getTableListByPatternLocal(ctx, metadataPath, tablePattern, false, []string{})
if err != nil {
return nil, err
}
for _, t := range diffTablesList {
tablesForUploadFromDiff[metadata.TableTitle{
Database: t.Database,
Table: t.Table,
}] = t
}
}
return tablesForUploadFromDiff, nil
}
func (b *Backuper) getTablesDiffFromRemote(ctx context.Context, diffFromRemote string, tablePattern string) (tablesForUploadFromDiff map[metadata.TableTitle]metadata.TableMetadata, err error) {
tablesForUploadFromDiff = make(map[metadata.TableTitle]metadata.TableMetadata)
backupList, err := b.dst.BackupList(ctx, true, diffFromRemote)
if err != nil {
return nil, err
}
var diffRemoteMetadata *metadata.BackupMetadata
for _, backup := range backupList {
if backup.BackupName == diffFromRemote {
diffRemoteMetadata = &backup.BackupMetadata
break
}
}
if diffRemoteMetadata == nil {
return nil, fmt.Errorf("%s not found on remote storage", diffFromRemote)
}
if len(diffRemoteMetadata.Tables) != 0 {
diffTablesList, err := getTableListByPatternRemote(ctx, b, diffRemoteMetadata, tablePattern, false)
if err != nil {
return nil, err
}
for _, t := range diffTablesList {
tablesForUploadFromDiff[metadata.TableTitle{
Database: t.Database,
Table: t.Table,
}] = t
}
}
return tablesForUploadFromDiff, nil
}
func (b *Backuper) GetLocalDataSize(ctx context.Context) (float64, error) {
localDataSize := float64(0)
err := b.ch.SelectSingleRow(ctx, &localDataSize, "SELECT value FROM system.asynchronous_metrics WHERE metric='TotalBytesOfMergeTreeTables'")
return localDataSize, err
}
func (b *Backuper) GetStateDir() string {
if b.isEmbedded && b.EmbeddedBackupDataPath != "" {
return b.EmbeddedBackupDataPath
}
return b.DefaultDataPath
}
func (b *Backuper) adjustResumeFlag(resume bool) {
if !resume && b.cfg.General.UseResumableState {
resume = true
}
b.resume = resume
}