Skip to content

Commit

Permalink
restore RBAC / configs, refactoring restart clickhouse-server via SYS…
Browse files Browse the repository at this point in the history
…TEM SHUTDOWN, add `--rbac-only` and `--configs-only` options to `create`, `upload`, `download`, `restore` command. fix #706

refactoring integration_test.go, split containers to clickhouse-backup + clickhouse
fix TestSkipNotExistsTable for 19.1+ version
  • Loading branch information
Slach committed Jul 31, 2023
1 parent b2cb946 commit 99c8361
Show file tree
Hide file tree
Showing 47 changed files with 582 additions and 382 deletions.
1 change: 1 addition & 0 deletions ChangeLog.md
Original file line number Diff line number Diff line change
Expand Up @@ -2,6 +2,7 @@
IMPROVEMENTS
- first implementation for properly backup S3/GCS/Azure disks, support server-side copy to back up bucket during `clickhouse-backup` create and during `clickhouse-backup restore`, requires add `object_disk_path` to `s3`,`gcs`,`azblob` section, fix [447](https://github.com/Altinity/clickhouse-backup/issues/447)
- Implementation blacklist for table engines during backup / download / upload / restore [537](https://github.com/Altinity/clickhouse-backup/issues/537)
- restore RBAC / configs, refactoring restart clickhouse-server via `sql:SYSTEM SHUTDOWN` or `exec:systemctl restart clickhouse-server`, add `--rbac-only` and `--configs-only` options to `create`, `upload`, `download`, `restore` command. fix [706]https://github.com/Altinity/clickhouse-backup/issues/706

BUG FIXES
- fix possible create backup failures during UNFREEZE not exists tables, affected 2.2.7+ version, fix [704](https://github.com/Altinity/clickhouse-backup/issues/704)
Expand Down
1 change: 1 addition & 0 deletions Dockerfile
Original file line number Diff line number Diff line change
Expand Up @@ -104,6 +104,7 @@ MAINTAINER Eugene Klimov <[email protected]>

RUN apt-get update && apt-get install -y gpg && wget -qO- https://kopia.io/signing-key | gpg --dearmor -o /usr/share/keyrings/kopia-keyring.gpg && \
echo "deb [signed-by=/usr/share/keyrings/kopia-keyring.gpg] http://packages.kopia.io/apt/ stable main" > /etc/apt/sources.list.d/kopia.list && \
wget -c "https://github.com/mikefarah/yq/releases/latest/download/yq_linux_$(dpkg --print-architecture)" -O /usr/bin/yq && chmod +x /usr/bin/yq && \
apt-get update -y && \
apt-get install -y ca-certificates tzdata bash curl restic rsync rclone jq gpg kopia && \
update-ca-certificates && \
Expand Down
7 changes: 6 additions & 1 deletion ReadMe.md
Original file line number Diff line number Diff line change
Expand Up @@ -409,7 +409,12 @@ clickhouse:
log_sql_queries: true # CLICKHOUSE_LOG_SQL_QUERIES, enable logging `clickhouse-backup` SQL queries on `system.query_log` table inside clickhouse-server
debug: false # CLICKHOUSE_DEBUG
config_dir: "/etc/clickhouse-server" # CLICKHOUSE_CONFIG_DIR
restart_command: "systemctl restart clickhouse-server" # CLICKHOUSE_RESTART_COMMAND, use this command when restoring with --rbac or --configs options
# CLICKHOUSE_RESTART_COMMAND, use this command when restoring with --rbac, --rbac-only or --configs, --configs-only options
# will split command by ; and execute one by one, all errors will logged and ignore
# available prefixes
# - sql: will execute SQL query
# - exec: will execute command via shell
restart_command: "sql:SYSTEM SHUTDOWN"
ignore_not_exists_error_during_freeze: true # CLICKHOUSE_IGNORE_NOT_EXISTS_ERROR_DURING_FREEZE, helps to avoid backup failures when running frequent CREATE / DROP tables and databases during backup, `clickhouse-backup` will ignore `code: 60` and `code: 81` errors during execution of `ALTER TABLE ... FREEZE`
check_replicas_before_attach: true # CLICKHOUSE_CHECK_REPLICAS_BEFORE_ATTACH, helps avoiding concurrent ATTACH PART execution when restoring ReplicatedMergeTree tables
use_embedded_backup_restore: false # CLICKHOUSE_USE_EMBEDDED_BACKUP_RESTORE, use BACKUP / RESTORE SQL statements instead of regular SQL queries to use features of modern ClickHouse server versions
Expand Down
52 changes: 46 additions & 6 deletions cmd/clickhouse-backup/main.go
Original file line number Diff line number Diff line change
Expand Up @@ -90,7 +90,7 @@ func main() {
Description: "Create new backup",
Action: func(c *cli.Context) error {
b := backup.NewBackuper(config.GetConfigFromCli(c))
return b.CreateBackup(c.Args().First(), c.String("t"), c.StringSlice("partitions"), c.Bool("s"), c.Bool("rbac"), c.Bool("configs"), c.Bool("skip-check-parts-columns"), version, c.Int("command-id"))
return b.CreateBackup(c.Args().First(), c.String("t"), c.StringSlice("partitions"), c.Bool("s"), c.Bool("rbac"), c.Bool("rbac-only"), c.Bool("configs"), c.Bool("configs-only"), c.Bool("skip-check-parts-columns"), version, c.Int("command-id"))
},
Flags: append(cliapp.Flags,
cli.StringFlag{
Expand All @@ -111,7 +111,7 @@ func main() {
cli.BoolFlag{
Name: "schema, s",
Hidden: false,
Usage: "Backup schemas only",
Usage: "Backup schemas only, will skip data",
},
cli.BoolFlag{
Name: "rbac, backup-rbac, do-backup-rbac",
Expand All @@ -123,6 +123,16 @@ func main() {
Hidden: false,
Usage: "Backup 'clickhouse-server' configuration files",
},
cli.BoolFlag{
Name: "rbac-only",
Hidden: false,
Usage: "Backup RBAC related objects only, will skip backup data, will backup schema only if --schema added",
},
cli.BoolFlag{
Name: "configs-only",
Hidden: false,
Usage: "Backup 'clickhouse-server' configuration files only, will skip backup data, will backup schema only if --schema added",
},
cli.BoolFlag{
Name: "skip-check-parts-columns",
Hidden: false,
Expand All @@ -137,7 +147,7 @@ func main() {
Description: "Create and upload",
Action: func(c *cli.Context) error {
b := backup.NewBackuper(config.GetConfigFromCli(c))
return b.CreateToRemote(c.Args().First(), c.String("diff-from"), c.String("diff-from-remote"), c.String("t"), c.StringSlice("partitions"), c.Bool("s"), c.Bool("rbac"), c.Bool("configs"), c.Bool("resume"), c.Bool("skip-check-parts-columns"), version, c.Int("command-id"))
return b.CreateToRemote(c.Args().First(), c.String("diff-from"), c.String("diff-from-remote"), c.String("t"), c.StringSlice("partitions"), c.Bool("s"), c.Bool("rbac"), c.Bool("rbac-only"), c.Bool("configs"), c.Bool("configs-only"), c.Bool("resume"), c.Bool("skip-check-parts-columns"), version, c.Int("command-id"))
},
Flags: append(cliapp.Flags,
cli.StringFlag{
Expand Down Expand Up @@ -168,7 +178,7 @@ func main() {
cli.BoolFlag{
Name: "schema, s",
Hidden: false,
Usage: "Backup and upload metadata schema only",
Usage: "Backup and upload metadata schema only, will skip data backup",
},
cli.BoolFlag{
Name: "rbac, backup-rbac, do-backup-rbac",
Expand All @@ -180,6 +190,16 @@ func main() {
Hidden: false,
Usage: "Backup and upload 'clickhouse-server' configuration files",
},
cli.BoolFlag{
Name: "rbac-only",
Hidden: false,
Usage: "Backup RBAC related objects only, will skip backup data, will backup schema only if --schema added",
},
cli.BoolFlag{
Name: "configs-only",
Hidden: false,
Usage: "Backup 'clickhouse-server' configuration files only, will skip backup data, will backup schema only if --schema added",
},
cli.BoolFlag{
Name: "resume, resumable",
Hidden: false,
Expand Down Expand Up @@ -290,7 +310,7 @@ func main() {
UsageText: "clickhouse-backup restore [-t, --tables=<db>.<table>] [-m, --restore-database-mapping=<originDB>:<targetDB>[,<...>]] [--partitions=<partitions_names>] [-s, --schema] [-d, --data] [--rm, --drop] [-i, --ignore-dependencies] [--rbac] [--configs] <backup_name>",
Action: func(c *cli.Context) error {
b := backup.NewBackuper(config.GetConfigFromCli(c))
return b.Restore(c.Args().First(), c.String("t"), c.StringSlice("restore-database-mapping"), c.StringSlice("partitions"), c.Bool("s"), c.Bool("d"), c.Bool("rm"), c.Bool("ignore-dependencies"), c.Bool("rbac"), c.Bool("configs"), c.Int("command-id"))
return b.Restore(c.Args().First(), c.String("t"), c.StringSlice("restore-database-mapping"), c.StringSlice("partitions"), c.Bool("s"), c.Bool("d"), c.Bool("rm"), c.Bool("ignore-dependencies"), c.Bool("rbac"), c.Bool("rbac-only"), c.Bool("configs"), c.Bool("configs-only"), c.Int("command-id"))
},
Flags: append(cliapp.Flags,
cli.StringFlag{
Expand Down Expand Up @@ -343,6 +363,16 @@ func main() {
Hidden: false,
Usage: "Restore 'clickhouse-server' CONFIG related files",
},
cli.BoolFlag{
Name: "rbac-only",
Hidden: false,
Usage: "Restore RBAC related objects only, will skip backup data, will backup schema only if --schema added",
},
cli.BoolFlag{
Name: "configs-only",
Hidden: false,
Usage: "Restore 'clickhouse-server' configuration files only, will skip backup data, will backup schema only if --schema added",
},
),
},
{
Expand All @@ -351,7 +381,7 @@ func main() {
UsageText: "clickhouse-backup restore_remote [--schema] [--data] [-t, --tables=<db>.<table>] [-m, --restore-database-mapping=<originDB>:<targetDB>[,<...>]] [--partitions=<partitions_names>] [--rm, --drop] [-i, --ignore-dependencies] [--rbac] [--configs] [--skip-rbac] [--skip-configs] [--resumable] <backup_name>",
Action: func(c *cli.Context) error {
b := backup.NewBackuper(config.GetConfigFromCli(c))
return b.RestoreFromRemote(c.Args().First(), c.String("t"), c.StringSlice("restore-database-mapping"), c.StringSlice("partitions"), c.Bool("s"), c.Bool("d"), c.Bool("rm"), c.Bool("i"), c.Bool("rbac"), c.Bool("configs"), c.Bool("resume"), c.Int("command-id"))
return b.RestoreFromRemote(c.Args().First(), c.String("t"), c.StringSlice("restore-database-mapping"), c.StringSlice("partitions"), c.Bool("s"), c.Bool("d"), c.Bool("rm"), c.Bool("i"), c.Bool("rbac"), c.Bool("rbac-only"), c.Bool("configs"), c.Bool("configs-only"), c.Bool("resume"), c.Int("command-id"))
},
Flags: append(cliapp.Flags,
cli.StringFlag{
Expand Down Expand Up @@ -404,6 +434,16 @@ func main() {
Hidden: false,
Usage: "Download and Restore 'clickhouse-server' CONFIG related files",
},
cli.BoolFlag{
Name: "rbac-only",
Hidden: false,
Usage: "Restore RBAC related objects only, will skip backup data, will backup schema only if --schema added",
},
cli.BoolFlag{
Name: "configs-only",
Hidden: false,
Usage: "Restore 'clickhouse-server' configuration files only, will skip backup data, will backup schema only if --schema added",
},
cli.BoolFlag{
Name: "resume, resumable",
Hidden: false,
Expand Down
75 changes: 38 additions & 37 deletions pkg/backup/create.go
Original file line number Diff line number Diff line change
Expand Up @@ -50,7 +50,7 @@ func NewBackupName() string {

// CreateBackup - create new backup of all tables matched by tablePattern
// If backupName is empty string will use default backup name
func (b *Backuper) CreateBackup(backupName, tablePattern string, partitions []string, schemaOnly, rbacOnly, configsOnly, skipCheckPartsColumns bool, version string, commandId int) error {
func (b *Backuper) CreateBackup(backupName, tablePattern string, partitions []string, schemaOnly, createRBAC, rbacOnly, createConfigs, configsOnly, skipCheckPartsColumns bool, version string, commandId int) error {
ctx, cancel, err := status.Current.GetContextWithCancel(commandId)
if err != nil {
return err
Expand All @@ -59,7 +59,7 @@ func (b *Backuper) CreateBackup(backupName, tablePattern string, partitions []st
defer cancel()

startBackup := time.Now()
doBackupData := !schemaOnly
doBackupData := !schemaOnly && !rbacOnly && !configsOnly
if backupName == "" {
backupName = NewBackupName()
}
Expand Down Expand Up @@ -115,9 +115,9 @@ func (b *Backuper) CreateBackup(backupName, tablePattern string, partitions []st
partitionsIdMap, partitionsNameList := partition.ConvertPartitionsToIdsMapAndNamesList(ctx, b.ch, tables, nil, partitions)
// create
if b.cfg.ClickHouse.UseEmbeddedBackupRestore {
err = b.createBackupEmbedded(ctx, backupName, tablePattern, partitionsNameList, partitionsIdMap, schemaOnly, rbacOnly, configsOnly, tables, allDatabases, allFunctions, disks, diskMap, diskTypes, log, startBackup, version)
err = b.createBackupEmbedded(ctx, backupName, tablePattern, partitionsNameList, partitionsIdMap, schemaOnly, createRBAC, createConfigs, tables, allDatabases, allFunctions, disks, diskMap, diskTypes, log, startBackup, version)
} else {
err = b.createBackupLocal(ctx, backupName, partitionsIdMap, tables, doBackupData, schemaOnly, rbacOnly, configsOnly, version, disks, diskMap, diskTypes, allDatabases, allFunctions, log, startBackup)
err = b.createBackupLocal(ctx, backupName, partitionsIdMap, tables, doBackupData, schemaOnly, createRBAC, rbacOnly, createConfigs, configsOnly, version, disks, diskMap, diskTypes, allDatabases, allFunctions, log, startBackup)
}
if err != nil {
return err
Expand All @@ -130,7 +130,7 @@ func (b *Backuper) CreateBackup(backupName, tablePattern string, partitions []st
return nil
}

func (b *Backuper) createBackupLocal(ctx context.Context, backupName string, partitionsIdMap map[metadata.TableTitle]common.EmptyMap, tables []clickhouse.Table, doBackupData bool, schemaOnly bool, rbacOnly bool, configsOnly bool, version string, disks []clickhouse.Disk, diskMap, diskTypes map[string]string, allDatabases []clickhouse.Database, allFunctions []clickhouse.Function, log *apexLog.Entry, startBackup time.Time) error {
func (b *Backuper) createBackupLocal(ctx context.Context, backupName string, partitionsIdMap map[metadata.TableTitle]common.EmptyMap, tables []clickhouse.Table, doBackupData bool, schemaOnly bool, createRBAC, rbacOnly bool, createConfigs, configsOnly bool, version string, disks []clickhouse.Disk, diskMap, diskTypes map[string]string, allDatabases []clickhouse.Database, allFunctions []clickhouse.Function, log *apexLog.Entry, startBackup time.Time) error {
// Create backup dir on all clickhouse disks
for _, disk := range disks {
if err := filesystemhelper.Mkdir(path.Join(disk.Path, "backup"), b.ch, disks); err != nil {
Expand Down Expand Up @@ -199,45 +199,46 @@ func (b *Backuper) createBackupLocal(ctx context.Context, backupName string, par
}
}
log.Debug("create metadata")

metadataSize, err := b.createTableMetadata(path.Join(backupPath, "metadata"), metadata.TableMetadata{
Table: table.Name,
Database: table.Database,
Query: table.CreateTableQuery,
TotalBytes: table.TotalBytes,
Size: realSize,
Parts: disksToPartsMap,
Mutations: inProgressMutations,
MetadataOnly: schemaOnly,
}, disks)
if err != nil {
if removeBackupErr := b.RemoveBackupLocal(ctx, backupName, disks); removeBackupErr != nil {
log.Error(removeBackupErr.Error())
if schemaOnly || doBackupData {
metadataSize, err := b.createTableMetadata(path.Join(backupPath, "metadata"), metadata.TableMetadata{
Table: table.Name,
Database: table.Database,
Query: table.CreateTableQuery,
TotalBytes: table.TotalBytes,
Size: realSize,
Parts: disksToPartsMap,
Mutations: inProgressMutations,
MetadataOnly: schemaOnly,
}, disks)
if err != nil {
if removeBackupErr := b.RemoveBackupLocal(ctx, backupName, disks); removeBackupErr != nil {
log.Error(removeBackupErr.Error())
}
return err
}
return err
backupMetadataSize += metadataSize
tableMetas = append(tableMetas, metadata.TableTitle{
Database: table.Database,
Table: table.Name,
})
}
backupMetadataSize += metadataSize
tableMetas = append(tableMetas, metadata.TableTitle{
Database: table.Database,
Table: table.Name,
})
log.Infof("done")
}
}
backupRBACSize, backupConfigSize := uint64(0), uint64(0)

if rbacOnly {
if backupRBACSize, err = b.createRBACBackup(ctx, backupPath, disks); err != nil {
if createRBAC || rbacOnly {
if backupRBACSize, err = b.createBackupRBAC(ctx, backupPath, disks); err != nil {
log.Errorf("error during do RBAC backup: %v", err)
} else {
log.WithField("size", utils.FormatBytes(backupRBACSize)).Info("done createRBACBackup")
log.WithField("size", utils.FormatBytes(backupRBACSize)).Info("done createBackupRBAC")
}
}
if configsOnly {
if backupConfigSize, err = b.createConfigBackup(ctx, backupPath); err != nil {
if createConfigs || configsOnly {
if backupConfigSize, err = b.createBackupConfigs(ctx, backupPath); err != nil {
log.Errorf("error during do CONFIG backup: %v", err)
} else {
log.WithField("size", utils.FormatBytes(backupConfigSize)).Info("done createConfigBackup")
log.WithField("size", utils.FormatBytes(backupConfigSize)).Info("done createBackupConfigs")
}
}

Expand All @@ -249,11 +250,11 @@ func (b *Backuper) createBackupLocal(ctx context.Context, backupName string, par
return nil
}

func (b *Backuper) createBackupEmbedded(ctx context.Context, backupName, tablePattern string, partitionsNameList map[metadata.TableTitle][]string, partitionsIdMap map[metadata.TableTitle]common.EmptyMap, schemaOnly, rbacOnly, configsOnly bool, tables []clickhouse.Table, allDatabases []clickhouse.Database, allFunctions []clickhouse.Function, disks []clickhouse.Disk, diskMap, diskTypes map[string]string, log *apexLog.Entry, startBackup time.Time, backupVersion string) error {
func (b *Backuper) createBackupEmbedded(ctx context.Context, backupName, tablePattern string, partitionsNameList map[metadata.TableTitle][]string, partitionsIdMap map[metadata.TableTitle]common.EmptyMap, schemaOnly, createRBAC, createConfigs bool, tables []clickhouse.Table, allDatabases []clickhouse.Database, allFunctions []clickhouse.Function, disks []clickhouse.Disk, diskMap, diskTypes map[string]string, log *apexLog.Entry, startBackup time.Time, backupVersion string) error {
if _, isBackupDiskExists := diskMap[b.cfg.ClickHouse.EmbeddedBackupDisk]; !isBackupDiskExists {
return fmt.Errorf("backup disk `%s` not exists in system.disks", b.cfg.ClickHouse.EmbeddedBackupDisk)
}
if rbacOnly || configsOnly {
if createRBAC || createConfigs {
return fmt.Errorf("`use_embedded_backup_restore: true` doesn't support --rbac, --configs parameters")
}
l := 0
Expand Down Expand Up @@ -412,8 +413,8 @@ func (b *Backuper) getPartsFromBackupDisk(backupPath string, table clickhouse.Ta
return parts, nil
}

func (b *Backuper) createConfigBackup(ctx context.Context, backupPath string) (uint64, error) {
log := b.log.WithField("logger", "createConfigBackup")
func (b *Backuper) createBackupConfigs(ctx context.Context, backupPath string) (uint64, error) {
log := b.log.WithField("logger", "createBackupConfigs")
select {
case <-ctx.Done():
return 0, ctx.Err()
Expand All @@ -431,8 +432,8 @@ func (b *Backuper) createConfigBackup(ctx context.Context, backupPath string) (u
}
}

func (b *Backuper) createRBACBackup(ctx context.Context, backupPath string, disks []clickhouse.Disk) (uint64, error) {
log := b.log.WithField("logger", "createRBACBackup")
func (b *Backuper) createBackupRBAC(ctx context.Context, backupPath string, disks []clickhouse.Disk) (uint64, error) {
log := b.log.WithField("logger", "createBackupRBAC")
select {
case <-ctx.Done():
return 0, ctx.Err()
Expand Down
4 changes: 2 additions & 2 deletions pkg/backup/create_remote.go
Original file line number Diff line number Diff line change
Expand Up @@ -6,7 +6,7 @@ import (
"github.com/Altinity/clickhouse-backup/pkg/status"
)

func (b *Backuper) CreateToRemote(backupName, diffFrom, diffFromRemote, tablePattern string, partitions []string, schemaOnly, rbac, backupConfig, skipCheckPartsColumns, resume bool, version string, commandId int) error {
func (b *Backuper) CreateToRemote(backupName, diffFrom, diffFromRemote, tablePattern string, partitions []string, schemaOnly, backupRBAC, rbacOnly, backupConfigs, configsOnly, skipCheckPartsColumns, resume bool, version string, commandId int) error {
ctx, cancel, err := status.Current.GetContextWithCancel(commandId)
if err != nil {
return err
Expand All @@ -16,7 +16,7 @@ func (b *Backuper) CreateToRemote(backupName, diffFrom, diffFromRemote, tablePat
if backupName == "" {
backupName = NewBackupName()
}
if err := b.CreateBackup(backupName, tablePattern, partitions, schemaOnly, rbac, backupConfig, skipCheckPartsColumns, version, commandId); err != nil {
if err := b.CreateBackup(backupName, tablePattern, partitions, schemaOnly, backupRBAC, rbacOnly, backupConfigs, configsOnly, skipCheckPartsColumns, version, commandId); err != nil {
return err
}
if err := b.Upload(backupName, diffFrom, diffFromRemote, tablePattern, partitions, schemaOnly, resume, commandId); err != nil {
Expand Down
Loading

0 comments on commit 99c8361

Please sign in to comment.