Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

Implement --restore-table-mapping for restore and restore_remote #944

Merged
merged 9 commits into from
Jul 4, 2024
Next Next commit
refactor: fixes #937, implement --restore-table-mapping
nithin-vunet committed Jul 1, 2024
commit 18bc91649c3bbd7e77189d51b1109ae2cafe276c
6 changes: 4 additions & 2 deletions Manual.md
Original file line number Diff line number Diff line change
@@ -147,13 +147,14 @@ NAME:
clickhouse-backup restore - Create schema and restore data from backup
USAGE:
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>
clickhouse-backup restore [-t, --tables=<db>.<table>] [-m, --restore-database-mapping=<originDB>:<targetDB>[,<...>]] [-tm, --restore-table-mapping=<originTable>:<targetTable>[,<...>]] [--partitions=<partitions_names>] [-s, --schema] [-d, --data] [--rm, --drop] [-i, --ignore-dependencies] [--rbac] [--configs] <backup_name>
OPTIONS:
--config value, -c value Config 'FILE' name. (default: "/etc/clickhouse-backup/config.yml") [$CLICKHOUSE_BACKUP_CONFIG]
--environment-override value, --env value override any environment variable via CLI parameter
--table value, --tables value, -t value Restore only database and objects which matched with table name patterns, separated by comma, allow ? and * as wildcard
--restore-database-mapping value, -m value Define the rule to restore data. For the database not defined in this struct, the program will not deal with it.
--restore-table-mapping value, -tm value Define the rule to restore data. For the table not defined in this struct, the program will not deal with it.
--partitions partition_id Restore backup only for selected partition names, separated by comma
If PARTITION BY clause returns numeric not hashed values for partition_id field in system.parts table, then use --partitions=partition_id1,partition_id2 format
If PARTITION BY clause returns hashed string values, then use --partitions=('non_numeric_field_value_for_part1'),('non_numeric_field_value_for_part2') format
@@ -177,13 +178,14 @@ NAME:
clickhouse-backup restore_remote - Download and restore
USAGE:
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>
clickhouse-backup restore_remote [--schema] [--data] [-t, --tables=<db>.<table>] [-m, --restore-database-mapping=<originDB>:<targetDB>[,<...>]] [-tm, --restore-table-mapping=<originTable>:<targetTable>[,<...>]] [--partitions=<partitions_names>] [--rm, --drop] [-i, --ignore-dependencies] [--rbac] [--configs] [--skip-rbac] [--skip-configs] [--resumable] <backup_name>
OPTIONS:
--config value, -c value Config 'FILE' name. (default: "/etc/clickhouse-backup/config.yml") [$CLICKHOUSE_BACKUP_CONFIG]
--environment-override value, --env value override any environment variable via CLI parameter
--table value, --tables value, -t value Download and restore objects which matched with table name patterns, separated by comma, allow ? and * as wildcard
--restore-database-mapping value, -m value Define the rule to restore data. For the database not defined in this struct, the program will not deal with it.
--restore-table-mapping value, -tm value Define the rule to restore data. For the table not defined in this struct, the program will not deal with it.
--partitions partition_id Download and restore backup only for selected partition names, separated by comma
If PARTITION BY clause returns numeric not hashed values for partition_id field in system.parts table, then use --partitions=partition_id1,partition_id2 format
If PARTITION BY clause returns hashed string values, then use --partitions=('non_numeric_field_value_for_part1'),('non_numeric_field_value_for_part2') format
11 changes: 9 additions & 2 deletions ReadMe.md
Original file line number Diff line number Diff line change
@@ -119,6 +119,10 @@ general:
# RESTORE_DATABASE_MAPPING, restore rules from backup databases to target databases, which is useful when changing destination database, all atomic tables will be created with new UUIDs.
# The format for this env variable is "src_db1:target_db1,src_db2:target_db2". For YAML please continue using map syntax
restore_database_mapping: {}

# RESTORE_TABLE_MAPPING, restore rules from backup tables to target tables, which is useful when changing destination tables.
# The format for this env variable is "src_table1:target_table1,src_table2:target_table2". For YAML please continue using map syntax
restore_table_mapping: {}
retries_on_failure: 3 # RETRIES_ON_FAILURE, how many times to retry after a failure during upload or download
retries_pause: 30s # RETRIES_PAUSE, duration time to pause after each download or upload failure

@@ -476,6 +480,7 @@ Create schema and restore data from backup: `curl -s localhost:7171/backup/resto
- Optional query argument `rbac` works the same as the `--rbac` CLI argument (restore RBAC).
- Optional query argument `configs` works the same as the `--configs` CLI argument (restore configs).
- Optional query argument `restore_database_mapping` works the same as the `--restore-database-mapping` CLI argument.
- Optional query argument `restore_table_mapping` works the same as the `--restore-table-mapping` CLI argument.
- Optional query argument `callback` allow pass callback URL which will call with POST with `application/json` with payload `{"status":"error|success","error":"not empty when error happens"}`.

### POST /backup/delete
@@ -705,13 +710,14 @@ NAME:
clickhouse-backup restore - Create schema and restore data from backup
USAGE:
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>
clickhouse-backup restore [-t, --tables=<db>.<table>] [-m, --restore-database-mapping=<originDB>:<targetDB>[,<...>]] [-tm, --restore-table-mapping=<originTable>:<targetTable>[,<...>]] [--partitions=<partitions_names>] [-s, --schema] [-d, --data] [--rm, --drop] [-i, --ignore-dependencies] [--rbac] [--configs] <backup_name>
OPTIONS:
--config value, -c value Config 'FILE' name. (default: "/etc/clickhouse-backup/config.yml") [$CLICKHOUSE_BACKUP_CONFIG]
--environment-override value, --env value override any environment variable via CLI parameter
--table value, --tables value, -t value Restore only database and objects which matched with table name patterns, separated by comma, allow ? and * as wildcard
--restore-database-mapping value, -m value Define the rule to restore data. For the database not defined in this struct, the program will not deal with it.
--restore-table-mapping value, -tm value Define the rule to restore data. For the table not defined in this struct, the program will not deal with it.
--partitions partition_id Restore backup only for selected partition names, separated by comma
If PARTITION BY clause returns numeric not hashed values for partition_id field in system.parts table, then use --partitions=partition_id1,partition_id2 format
If PARTITION BY clause returns hashed string values, then use --partitions=('non_numeric_field_value_for_part1'),('non_numeric_field_value_for_part2') format
@@ -735,13 +741,14 @@ NAME:
clickhouse-backup restore_remote - Download and restore
USAGE:
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>
clickhouse-backup restore_remote [--schema] [--data] [-t, --tables=<db>.<table>] [-m, --restore-database-mapping=<originDB>:<targetDB>[,<...>]] [-tm, --restore-table-mapping=<originTable>:<targetTable>[,<...>]] [--partitions=<partitions_names>] [--rm, --drop] [-i, --ignore-dependencies] [--rbac] [--configs] [--skip-rbac] [--skip-configs] [--resumable] <backup_name>
OPTIONS:
--config value, -c value Config 'FILE' name. (default: "/etc/clickhouse-backup/config.yml") [$CLICKHOUSE_BACKUP_CONFIG]
--environment-override value, --env value override any environment variable via CLI parameter
--table value, --tables value, -t value Download and restore objects which matched with table name patterns, separated by comma, allow ? and * as wildcard
--restore-database-mapping value, -m value Define the rule to restore data. For the database not defined in this struct, the program will not deal with it.
--restore-table-mapping value, -tm value Define the rule to restore data. For the table not defined in this struct, the program will not deal with it.
--partitions partition_id Download and restore backup only for selected partition names, separated by comma
If PARTITION BY clause returns numeric not hashed values for partition_id field in system.parts table, then use --partitions=partition_id1,partition_id2 format
If PARTITION BY clause returns hashed string values, then use --partitions=('non_numeric_field_value_for_part1'),('non_numeric_field_value_for_part2') format
25 changes: 17 additions & 8 deletions cmd/clickhouse-backup/main.go
Original file line number Diff line number Diff line change
@@ -6,15 +6,14 @@ import (
"os"
"strings"

"github.com/Altinity/clickhouse-backup/v2/pkg/config"
"github.com/Altinity/clickhouse-backup/v2/pkg/logcli"
"github.com/Altinity/clickhouse-backup/v2/pkg/status"
"github.com/apex/log"
"github.com/urfave/cli"

"github.com/Altinity/clickhouse-backup/v2/pkg/backup"
"github.com/Altinity/clickhouse-backup/v2/pkg/config"
"github.com/Altinity/clickhouse-backup/v2/pkg/logcli"
"github.com/Altinity/clickhouse-backup/v2/pkg/server"

"github.com/apex/log"
"github.com/urfave/cli"
"github.com/Altinity/clickhouse-backup/v2/pkg/status"
)

var (
@@ -340,7 +339,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("schema"), c.Bool("data"), c.Bool("drop"), c.Bool("ignore-dependencies"), c.Bool("rbac"), c.Bool("rbac-only"), c.Bool("configs"), c.Bool("configs-only"), version, c.Int("command-id"))
return b.Restore(c.Args().First(), c.String("t"), c.StringSlice("restore-database-mapping"), c.StringSlice("restore-table-mapping"), c.StringSlice("partitions"), c.Bool("schema"), c.Bool("data"), c.Bool("drop"), c.Bool("ignore-dependencies"), c.Bool("rbac"), c.Bool("rbac-only"), c.Bool("configs"), c.Bool("configs-only"), version, c.Int("command-id"))
},
Flags: append(cliapp.Flags,
cli.StringFlag{
@@ -353,6 +352,11 @@ func main() {
Usage: "Define the rule to restore data. For the database not defined in this struct, the program will not deal with it.",
Hidden: false,
},
cli.StringSliceFlag{
Name: "restore-table-mapping, tm",
Usage: "Define the rule to restore data. For the table not defined in this struct, the program will not deal with it.",
Hidden: false,
},
cli.StringSliceFlag{
Name: "partitions",
Hidden: false,
@@ -412,7 +416,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("rbac-only"), c.Bool("configs"), c.Bool("configs-only"), c.Bool("resume"), version, c.Int("command-id"))
return b.RestoreFromRemote(c.Args().First(), c.String("t"), c.StringSlice("restore-database-mapping"), c.StringSlice("restore-table-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"), version, c.Int("command-id"))
},
Flags: append(cliapp.Flags,
cli.StringFlag{
@@ -425,6 +429,11 @@ func main() {
Usage: "Define the rule to restore data. For the database not defined in this struct, the program will not deal with it.",
Hidden: false,
},
cli.StringSliceFlag{
Name: "restore-table-mapping, tm",
Usage: "Define the rule to restore data. For the database not defined in this struct, the program will not deal with it.",
Hidden: false,
},
cli.StringSliceFlag{
Name: "partitions",
Hidden: false,
16 changes: 8 additions & 8 deletions pkg/backup/create.go
Original file line number Diff line number Diff line change
@@ -5,9 +5,6 @@ import (
"encoding/json"
"errors"
"fmt"
"github.com/Altinity/clickhouse-backup/v2/pkg/config"
"github.com/Altinity/clickhouse-backup/v2/pkg/storage"
"golang.org/x/sync/errgroup"
"os"
"path"
"path/filepath"
@@ -17,19 +14,22 @@ import (
"sync/atomic"
"time"

apexLog "github.com/apex/log"
"github.com/google/uuid"
recursiveCopy "github.com/otiai10/copy"
"golang.org/x/sync/errgroup"

"github.com/Altinity/clickhouse-backup/v2/pkg/clickhouse"
"github.com/Altinity/clickhouse-backup/v2/pkg/common"
"github.com/Altinity/clickhouse-backup/v2/pkg/config"
"github.com/Altinity/clickhouse-backup/v2/pkg/filesystemhelper"
"github.com/Altinity/clickhouse-backup/v2/pkg/keeper"
"github.com/Altinity/clickhouse-backup/v2/pkg/metadata"
"github.com/Altinity/clickhouse-backup/v2/pkg/partition"
"github.com/Altinity/clickhouse-backup/v2/pkg/status"
"github.com/Altinity/clickhouse-backup/v2/pkg/storage"
"github.com/Altinity/clickhouse-backup/v2/pkg/storage/object_disk"
"github.com/Altinity/clickhouse-backup/v2/pkg/utils"

apexLog "github.com/apex/log"
"github.com/google/uuid"
recursiveCopy "github.com/otiai10/copy"
)

const (
@@ -255,7 +255,7 @@ func (b *Backuper) createBackupLocal(ctx context.Context, backupName, diffFromRe
var backupDataSize, backupObjectDiskSize, backupMetadataSize uint64
var metaMutex sync.Mutex
createBackupWorkingGroup, createCtx := errgroup.WithContext(ctx)
createBackupWorkingGroup.SetLimit(max(b.cfg.ClickHouse.MaxConnections,1))
createBackupWorkingGroup.SetLimit(max(b.cfg.ClickHouse.MaxConnections, 1))

var tableMetas []metadata.TableTitle
for tableIdx, tableItem := range tables {
119 changes: 85 additions & 34 deletions pkg/backup/restore.go
Original file line number Diff line number Diff line change
@@ -5,12 +5,6 @@ import (
"context"
"encoding/json"
"fmt"
"github.com/Altinity/clickhouse-backup/v2/pkg/config"
"github.com/Altinity/clickhouse-backup/v2/pkg/keeper"
"github.com/Altinity/clickhouse-backup/v2/pkg/status"
"github.com/Altinity/clickhouse-backup/v2/pkg/storage"
"github.com/Altinity/clickhouse-backup/v2/pkg/storage/object_disk"
"golang.org/x/sync/errgroup"
"io"
"io/fs"
"net/url"
@@ -23,23 +17,30 @@ import (
"sync/atomic"
"time"

"github.com/Altinity/clickhouse-backup/v2/pkg/common"

apexLog "github.com/apex/log"
"github.com/mattn/go-shellwords"
recursiveCopy "github.com/otiai10/copy"
"github.com/yargevad/filepathx"
"golang.org/x/sync/errgroup"
"golang.org/x/text/cases"
"golang.org/x/text/language"

"github.com/Altinity/clickhouse-backup/v2/pkg/clickhouse"
"github.com/Altinity/clickhouse-backup/v2/pkg/common"
"github.com/Altinity/clickhouse-backup/v2/pkg/config"
"github.com/Altinity/clickhouse-backup/v2/pkg/filesystemhelper"
"github.com/Altinity/clickhouse-backup/v2/pkg/keeper"
"github.com/Altinity/clickhouse-backup/v2/pkg/metadata"
"github.com/Altinity/clickhouse-backup/v2/pkg/status"
"github.com/Altinity/clickhouse-backup/v2/pkg/storage"
"github.com/Altinity/clickhouse-backup/v2/pkg/storage/object_disk"
"github.com/Altinity/clickhouse-backup/v2/pkg/utils"
apexLog "github.com/apex/log"
recursiveCopy "github.com/otiai10/copy"
"github.com/yargevad/filepathx"
)

var CreateDatabaseRE = regexp.MustCompile(`(?m)^CREATE DATABASE (\s*)(\S+)(\s*)`)

// Restore - restore tables matched by tablePattern from backupName
func (b *Backuper) Restore(backupName, tablePattern string, databaseMapping, partitions []string, schemaOnly, dataOnly, dropExists, ignoreDependencies, restoreRBAC, rbacOnly, restoreConfigs, configsOnly bool, backupVersion string, commandId int) error {
func (b *Backuper) Restore(backupName, tablePattern string, databaseMapping, tableMapping, partitions []string, schemaOnly, dataOnly, dropExists, ignoreDependencies, restoreRBAC, rbacOnly, restoreConfigs, configsOnly bool, backupVersion string, commandId int) error {
ctx, cancel, err := status.Current.GetContextWithCancel(commandId)
if err != nil {
return err
@@ -48,7 +49,10 @@ func (b *Backuper) Restore(backupName, tablePattern string, databaseMapping, par
defer cancel()
startRestore := time.Now()
backupName = utils.CleanBackupNameRE.ReplaceAllString(backupName, "")
if err := b.prepareRestoreDatabaseMapping(databaseMapping); err != nil {
if err := b.prepareRestoreMapping(databaseMapping, "database"); err != nil {
return err
}
if err := b.prepareRestoreMapping(tableMapping, "table"); err != nil {
return err
}

@@ -247,13 +251,23 @@ func (b *Backuper) getTablesForRestoreLocal(ctx context.Context, backupName stri
if err != nil {
return nil, nil, err
}
// if restore-database-mapping specified, create database in mapping rules instead of in backup files.
// if restore-database-mapping is specified, create database in mapping rules instead of in backup files.
if len(b.cfg.General.RestoreDatabaseMapping) > 0 {
err = changeTableQueryToAdjustDatabaseMapping(&tablesForRestore, b.cfg.General.RestoreDatabaseMapping)
if err != nil {
return nil, nil, err
}
}

// if restore-table-mapping is specified, create table in mapping rules instead of in backup files.
// https://github.com/Altinity/clickhouse-backup/issues/937
if len(b.cfg.General.RestoreTableMapping) > 0 {
err = changeTableQueryToAdjustTableMapping(&tablesForRestore, b.cfg.General.RestoreTableMapping)
if err != nil {
return nil, nil, err
}
}

if len(tablesForRestore) == 0 {
return nil, nil, fmt.Errorf("not found schemas by %s in %s, also check skip_tables and skip_table_engines setting", tablePattern, backupName)
}
@@ -356,15 +370,23 @@ func (b *Backuper) restoreEmptyDatabase(ctx context.Context, targetDB, tablePatt
return nil
}

func (b *Backuper) prepareRestoreDatabaseMapping(databaseMapping []string) error {
for i := 0; i < len(databaseMapping); i++ {
splitByCommas := strings.Split(databaseMapping[i], ",")
func (b *Backuper) prepareRestoreMapping(objectMapping []string, objectType string) error {
if objectType != "database" && objectType != "table" {
return fmt.Errorf("objectType must be one of `database` or `table`")
}
for i := 0; i < len(objectMapping); i++ {
splitByCommas := strings.Split(objectMapping[i], ",")
for _, m := range splitByCommas {
splitByColon := strings.Split(m, ":")
if len(splitByColon) != 2 {
return fmt.Errorf("restore-database-mapping %s should only have srcDatabase:destinationDatabase format for each map rule", m)
objectTypeTitleCase := cases.Title(language.Und).String(objectType)
return fmt.Errorf("restore-%s-mapping %s should only have src%s:destination%s format for each map rule", objectType, m, objectTypeTitleCase, objectTypeTitleCase)
}
if objectType == "database" {
b.cfg.General.RestoreDatabaseMapping[splitByColon[0]] = splitByColon[1]
} else {
b.cfg.General.RestoreTableMapping[splitByColon[0]] = splitByColon[1]
}
b.cfg.General.RestoreDatabaseMapping[splitByColon[0]] = splitByColon[1]
}
}
return nil
@@ -1179,8 +1201,13 @@ func (b *Backuper) restoreDataEmbedded(ctx context.Context, backupName string, d

func (b *Backuper) restoreDataRegular(ctx context.Context, backupName string, backupMetadata metadata.BackupMetadata, tablePattern string, tablesForRestore ListOfTables, diskMap, diskTypes map[string]string, disks []clickhouse.Disk, log *apexLog.Entry) error {
if len(b.cfg.General.RestoreDatabaseMapping) > 0 {
tablePattern = b.changeTablePatternFromRestoreDatabaseMapping(tablePattern)
tablePattern = b.changeTablePatternFromRestoreMapping(tablePattern, "database")
}
// https://github.com/Altinity/clickhouse-backup/issues/937
if len(b.cfg.General.RestoreTableMapping) > 0 {
tablePattern = b.changeTablePatternFromRestoreMapping(tablePattern, "table")
}

if err := b.applyMacrosToObjectDiskPath(ctx); err != nil {
return err
}
@@ -1196,23 +1223,32 @@ func (b *Backuper) restoreDataRegular(ctx context.Context, backupName string, ba
return fmt.Errorf("%s is not created. Restore schema first or create missing tables manually", strings.Join(missingTables, ", "))
}
restoreBackupWorkingGroup, restoreCtx := errgroup.WithContext(ctx)
restoreBackupWorkingGroup.SetLimit(max(b.cfg.ClickHouse.MaxConnections,1))
restoreBackupWorkingGroup.SetLimit(max(b.cfg.ClickHouse.MaxConnections, 1))

for i := range tablesForRestore {
tableRestoreStartTime := time.Now()
table := tablesForRestore[i]
// need mapped database path and original table.Database for HardlinkBackupPartsToStorage
// need mapped database path and original table.Database for HardlinkBackupPartsToStorage.
dstDatabase := table.Database
// The same goes for the table
dstTableName := table.Table
if len(b.cfg.General.RestoreDatabaseMapping) > 0 {
if targetDB, isMapped := b.cfg.General.RestoreDatabaseMapping[table.Database]; isMapped {
dstDatabase = targetDB
tablesForRestore[i].Database = targetDB
}
}
log := log.WithField("table", fmt.Sprintf("%s.%s", dstDatabase, table.Table))
// https://github.com/Altinity/clickhouse-backup/issues/937
if len(b.cfg.General.RestoreTableMapping) > 0 {
if targetTable, isMapped := b.cfg.General.RestoreTableMapping[table.Table]; isMapped {
dstTableName = targetTable
tablesForRestore[i].Table = targetTable
}
}
log := log.WithField("table", fmt.Sprintf("%s.%s", dstDatabase, dstTableName))
dstTable, ok := dstTablesMap[metadata.TableTitle{
Database: dstDatabase,
Table: table.Table}]
Table: dstTableName}]
if !ok {
return fmt.Errorf("can't find '%s.%s' in current system.tables", dstDatabase, table.Table)
}
@@ -1454,14 +1490,20 @@ func (b *Backuper) checkMissingTables(tablesForRestore ListOfTables, chTables []
var missingTables []string
for _, table := range tablesForRestore {
dstDatabase := table.Database
dstTable := table.Table
if len(b.cfg.General.RestoreDatabaseMapping) > 0 {
if targetDB, isMapped := b.cfg.General.RestoreDatabaseMapping[table.Database]; isMapped {
dstDatabase = targetDB
}
}
if len(b.cfg.General.RestoreTableMapping) > 0 {
if targetTable, isMapped := b.cfg.General.RestoreTableMapping[table.Table]; isMapped {
dstTable = targetTable
}
}
found := false
for _, chTable := range chTables {
if (dstDatabase == chTable.Database) && (table.Table == chTable.Name) {
if (dstDatabase == chTable.Database) && (dstTable == chTable.Name) {
found = true
break
}
@@ -1484,22 +1526,31 @@ func (b *Backuper) prepareDstTablesMap(chTables []clickhouse.Table) map[metadata
return dstTablesMap
}

func (b *Backuper) changeTablePatternFromRestoreDatabaseMapping(tablePattern string) string {
for sourceDb, targetDb := range b.cfg.General.RestoreDatabaseMapping {
func (b *Backuper) changeTablePatternFromRestoreMapping(tablePattern, objType string) string {
var mapping map[string]string
switch objType {
case "database":
mapping = b.cfg.General.RestoreDatabaseMapping
case "table":
mapping = b.cfg.General.RestoreDatabaseMapping
default:
return ""
}
for sourceObj, targetObj := range mapping {
if tablePattern != "" {
sourceDbRE := regexp.MustCompile(fmt.Sprintf("(^%s.*)|(,%s.*)", sourceDb, sourceDb))
if sourceDbRE.MatchString(tablePattern) {
matches := sourceDbRE.FindAllStringSubmatch(tablePattern, -1)
substitution := targetDb + ".*"
sourceObjRE := regexp.MustCompile(fmt.Sprintf("(^%s.*)|(,%s.*)", sourceObj, sourceObj))
if sourceObjRE.MatchString(tablePattern) {
matches := sourceObjRE.FindAllStringSubmatch(tablePattern, -1)
substitution := targetObj + ".*"
if strings.HasPrefix(matches[0][1], ",") {
substitution = "," + substitution
}
tablePattern = sourceDbRE.ReplaceAllString(tablePattern, substitution)
tablePattern = sourceObjRE.ReplaceAllString(tablePattern, substitution)
} else {
tablePattern += "," + targetDb + ".*"
tablePattern += "," + targetObj + ".*"
}
} else {
tablePattern += targetDb + ".*"
tablePattern += targetObj + ".*"
}
}
return tablePattern
4 changes: 2 additions & 2 deletions pkg/backup/restore_remote.go
Original file line number Diff line number Diff line change
@@ -2,12 +2,12 @@ package backup

import "errors"

func (b *Backuper) RestoreFromRemote(backupName, tablePattern string, databaseMapping, partitions []string, schemaOnly, dataOnly, dropExists, ignoreDependencies, restoreRBAC, rbacOnly, restoreConfigs, configsOnly, resume bool, version string, commandId int) error {
func (b *Backuper) RestoreFromRemote(backupName, tablePattern string, databaseMapping, tableMapping, partitions []string, schemaOnly, dataOnly, dropExists, ignoreDependencies, restoreRBAC, rbacOnly, restoreConfigs, configsOnly, resume bool, version string, commandId int) error {
if err := b.Download(backupName, tablePattern, partitions, schemaOnly, resume, version, commandId); err != nil {
// https://github.com/Altinity/clickhouse-backup/issues/625
if !errors.Is(err, ErrBackupIsAlreadyExists) {
return err
}
}
return b.Restore(backupName, tablePattern, databaseMapping, partitions, schemaOnly, dataOnly, dropExists, ignoreDependencies, restoreRBAC, rbacOnly, restoreConfigs, configsOnly, version, commandId)
return b.Restore(backupName, tablePattern, databaseMapping, tableMapping, partitions, schemaOnly, dataOnly, dropExists, ignoreDependencies, restoreRBAC, rbacOnly, restoreConfigs, configsOnly, version, commandId)
}
83 changes: 76 additions & 7 deletions pkg/backup/table_pattern.go
Original file line number Diff line number Diff line change
@@ -4,10 +4,6 @@ import (
"context"
"encoding/json"
"fmt"
"github.com/Altinity/clickhouse-backup/v2/pkg/config"
"github.com/Altinity/clickhouse-backup/v2/pkg/partition"
apexLog "github.com/apex/log"
"github.com/google/uuid"
"io"
"net/url"
"os"
@@ -17,10 +13,14 @@ import (
"sort"
"strings"

apexLog "github.com/apex/log"
"github.com/google/uuid"

"github.com/Altinity/clickhouse-backup/v2/pkg/common"
"github.com/Altinity/clickhouse-backup/v2/pkg/config"
"github.com/Altinity/clickhouse-backup/v2/pkg/filesystemhelper"

"github.com/Altinity/clickhouse-backup/v2/pkg/metadata"
"github.com/Altinity/clickhouse-backup/v2/pkg/partition"
)

type ListOfTables []metadata.TableMetadata
@@ -301,7 +301,7 @@ var uuidRE = regexp.MustCompile(`UUID '([a-f\d\-]+)'`)

var usualIdentifier = regexp.MustCompile(`^[a-zA-Z0-9_]+$`)
var replicatedRE = regexp.MustCompile(`(Replicated[a-zA-Z]*MergeTree)\('([^']+)'([^)]+)\)`)
var distributedRE = regexp.MustCompile(`(Distributed)\(([^,]+),([^,]+),([^)]+)\)`)
var distributedRE = regexp.MustCompile(`(Distributed)\(([^,]+),([^,]+),([^,]+),([^)]+)\)`)

func changeTableQueryToAdjustDatabaseMapping(originTables *ListOfTables, dbMapRule map[string]string) error {
for i := 0; i < len(*originTables); i++ {
@@ -360,7 +360,7 @@ func changeTableQueryToAdjustDatabaseMapping(originTables *ListOfTables, dbMapRu
underlyingDB := matches[0][3]
underlyingDBClean := strings.NewReplacer(" ", "", "'", "").Replace(underlyingDB)
if underlyingTargetDB, isUnderlyingMapped := dbMapRule[underlyingDBClean]; isUnderlyingMapped {
substitution = fmt.Sprintf("${1}(${2},%s,${4})", strings.Replace(underlyingDB, underlyingDBClean, underlyingTargetDB, 1))
substitution = fmt.Sprintf("${1}(${2},%s,${4},${5})", strings.Replace(underlyingDB, underlyingDBClean, underlyingTargetDB, 1))
originTable.Query = distributedRE.ReplaceAllString(originTable.Query, substitution)
}
}
@@ -371,6 +371,75 @@ func changeTableQueryToAdjustDatabaseMapping(originTables *ListOfTables, dbMapRu
return nil
}

func changeTableQueryToAdjustTableMapping(originTables *ListOfTables, tableMapRule map[string]string) error {
for i := 0; i < len(*originTables); i++ {
originTable := (*originTables)[i]
if targetTable, isMapped := tableMapRule[originTable.Table]; isMapped {
// substitute table in the table create query
var substitution string

if createOrAttachRE.MatchString(originTable.Query) {
matches := queryRE.FindAllStringSubmatch(originTable.Query, -1)
if matches[0][6] != originTable.Table {
return fmt.Errorf("invalid SQL: %s for restore-table-mapping[%s]=%s", originTable.Query, originTable.Table, targetTable)
}
setMatchedDb := func(clauseTargetTable string) string {
if clauseMappedTable, isClauseMapped := tableMapRule[clauseTargetTable]; isClauseMapped {
clauseTargetTable = clauseMappedTable
if !usualIdentifier.MatchString(clauseTargetTable) {
clauseTargetTable = "`" + clauseTargetTable + "`"
}
}
return clauseTargetTable
}
createTargetTable := targetTable
if !usualIdentifier.MatchString(createTargetTable) {
createTargetTable = "`" + createTargetTable + "`"
}
toClauseTargetTable := setMatchedDb(matches[0][12])
fromClauseTargetTable := setMatchedDb(matches[0][17])
// matching CREATE|ATTACH ... TO .. SELECT ... FROM ... command
substitution = fmt.Sprintf("${1} ${2} ${3}${4}${5}.%v${7}${8}${9}${10}${11}%v${13}${14}${15}${16}%v", createTargetTable, toClauseTargetTable, fromClauseTargetTable)
} else {
if originTable.Query == "" {
continue
}
return fmt.Errorf("error when try to replace table `%s` to `%s` in query: %s", originTable.Table, targetTable, originTable.Query)
}
originTable.Query = queryRE.ReplaceAllString(originTable.Query, substitution)
if uuidRE.MatchString(originTable.Query) {
newUUID, _ := uuid.NewUUID()
substitution = fmt.Sprintf("UUID '%s'", newUUID.String())
originTable.Query = uuidRE.ReplaceAllString(originTable.Query, substitution)
}
// https://github.com/Altinity/clickhouse-backup/issues/547
if replicatedRE.MatchString(originTable.Query) {
matches := replicatedRE.FindAllStringSubmatch(originTable.Query, -1)
originPath := matches[0][2]
tableReplicatedPattern := "/" + originTable.Table + "/"
if strings.Contains(originPath, tableReplicatedPattern) {
substitution = fmt.Sprintf("${1}('%s'${3})", strings.Replace(originPath, tableReplicatedPattern, "/"+targetTable+"/", 1))
originTable.Query = replicatedRE.ReplaceAllString(originTable.Query, substitution)
}
}
// https://github.com/Altinity/clickhouse-backup/issues/547
if distributedRE.MatchString(originTable.Query) {
matches := distributedRE.FindAllStringSubmatch(originTable.Query, -1)
underlyingTable := matches[0][4]
underlyingTableClean := strings.NewReplacer(" ", "", "'", "").Replace(underlyingTable)
underlyingTableClean = underlyingTableClean[:len(underlyingTableClean)-5]
if underlyingTargetTable, isUnderlyingMapped := tableMapRule[underlyingTableClean]; isUnderlyingMapped {
substitution = fmt.Sprintf("${1}(${2},${3},%s,${5})", strings.Replace(underlyingTable, underlyingTableClean, underlyingTargetTable, 1))
originTable.Query = distributedRE.ReplaceAllString(originTable.Query, substitution)
}
}
originTable.Table = targetTable
(*originTables)[i] = originTable
}
}
return nil
}

func filterPartsAndFilesByPartitionsFilter(tableMetadata metadata.TableMetadata, partitionsFilter common.EmptyMap) {
if len(partitionsFilter) > 0 {
for disk, parts := range tableMetadata.Parts {
13 changes: 10 additions & 3 deletions pkg/clickhouse/clickhouse.go
Original file line number Diff line number Diff line change
@@ -15,14 +15,15 @@ import (
"strings"
"time"

"github.com/Altinity/clickhouse-backup/v2/pkg/common"
"github.com/Altinity/clickhouse-backup/v2/pkg/config"
"github.com/Altinity/clickhouse-backup/v2/pkg/metadata"
"github.com/ClickHouse/clickhouse-go/v2"
"github.com/ClickHouse/clickhouse-go/v2/lib/driver"
"github.com/antchfx/xmlquery"
apexLog "github.com/apex/log"
"github.com/ricochet2200/go-disk-usage/du"

"github.com/Altinity/clickhouse-backup/v2/pkg/common"
"github.com/Altinity/clickhouse-backup/v2/pkg/config"
"github.com/Altinity/clickhouse-backup/v2/pkg/metadata"
)

// ClickHouse - provide
@@ -751,6 +752,9 @@ func (ch *ClickHouse) AttachDataParts(table metadata.TableMetadata, dstTable Tab
if dstTable.Database != "" && dstTable.Database != table.Database {
table.Database = dstTable.Database
}
if dstTable.Name != "" && dstTable.Name != table.Table {
table.Table = dstTable.Name
}
canContinue, err := ch.CheckReplicationInProgress(table)
if err != nil {
return err
@@ -784,6 +788,9 @@ func (ch *ClickHouse) AttachTable(ctx context.Context, table metadata.TableMetad
if dstTable.Database != "" && dstTable.Database != table.Database {
table.Database = dstTable.Database
}
if dstTable.Name != "" && dstTable.Name != table.Table {
table.Table = dstTable.Name
}
canContinue, err := ch.CheckReplicationInProgress(table)
if err != nil {
return err
5 changes: 3 additions & 2 deletions pkg/config/config.go
Original file line number Diff line number Diff line change
@@ -10,9 +10,8 @@ import (
"strings"
"time"

s3types "github.com/aws/aws-sdk-go-v2/service/s3/types"

"github.com/apex/log"
s3types "github.com/aws/aws-sdk-go-v2/service/s3/types"
"github.com/kelseyhightower/envconfig"
"github.com/urfave/cli"
"gopkg.in/yaml.v3"
@@ -54,6 +53,7 @@ type GeneralConfig struct {
UploadByPart bool `yaml:"upload_by_part" envconfig:"UPLOAD_BY_PART"`
DownloadByPart bool `yaml:"download_by_part" envconfig:"DOWNLOAD_BY_PART"`
RestoreDatabaseMapping map[string]string `yaml:"restore_database_mapping" envconfig:"RESTORE_DATABASE_MAPPING"`
RestoreTableMapping map[string]string `yaml:"restore_table_mapping" envconfig:"RESTORE_TABLE_MAPPING"`
RetriesOnFailure int `yaml:"retries_on_failure" envconfig:"RETRIES_ON_FAILURE"`
RetriesPause string `yaml:"retries_pause" envconfig:"RETRIES_PAUSE"`
WatchInterval string `yaml:"watch_interval" envconfig:"WATCH_INTERVAL"`
@@ -529,6 +529,7 @@ func DefaultConfig() *Config {
FullDuration: 24 * time.Hour,
WatchBackupNameTemplate: "shard{shard}-{type}-{time:20060102150405}",
RestoreDatabaseMapping: make(map[string]string, 0),
RestoreTableMapping: make(map[string]string, 0),
IONicePriority: "idle",
CPUNicePriority: 15,
RBACBackupAlways: true,
34 changes: 27 additions & 7 deletions pkg/server/server.go
Original file line number Diff line number Diff line change
@@ -22,6 +22,12 @@ import (
"syscall"
"time"

apexLog "github.com/apex/log"
"github.com/google/shlex"
"github.com/gorilla/mux"
"github.com/prometheus/client_golang/prometheus/promhttp"
"github.com/urfave/cli"

"github.com/Altinity/clickhouse-backup/v2/pkg/backup"
"github.com/Altinity/clickhouse-backup/v2/pkg/clickhouse"
"github.com/Altinity/clickhouse-backup/v2/pkg/common"
@@ -30,12 +36,6 @@ import (
"github.com/Altinity/clickhouse-backup/v2/pkg/server/metrics"
"github.com/Altinity/clickhouse-backup/v2/pkg/status"
"github.com/Altinity/clickhouse-backup/v2/pkg/utils"

apexLog "github.com/apex/log"
"github.com/google/shlex"
"github.com/gorilla/mux"
"github.com/prometheus/client_golang/prometheus/promhttp"
"github.com/urfave/cli"
)

type APIServer struct {
@@ -1162,6 +1162,7 @@ func (api *APIServer) httpUploadHandler(w http.ResponseWriter, r *http.Request)
}

var databaseMappingRE = regexp.MustCompile(`[\w+]:[\w+]`)
var tableMappingRE = regexp.MustCompile(`[\w+]:[\w+]`)

// httpRestoreHandler - restore a backup from local storage
func (api *APIServer) httpRestoreHandler(w http.ResponseWriter, r *http.Request) {
@@ -1177,6 +1178,7 @@ func (api *APIServer) httpRestoreHandler(w http.ResponseWriter, r *http.Request)
vars := mux.Vars(r)
tablePattern := ""
databaseMappingToRestore := make([]string, 0)
tableMappingToRestore := make([]string, 0)
partitionsToBackup := make([]string, 0)
schemaOnly := false
dataOnly := false
@@ -1206,6 +1208,24 @@ func (api *APIServer) httpRestoreHandler(w http.ResponseWriter, r *http.Request)

fullCommand = fmt.Sprintf("%s --restore-database-mapping=\"%s\"", fullCommand, strings.Join(databaseMappingToRestore, ","))
}

// https://github.com/Altinity/clickhouse-backup/issues/937
if tableMappingQuery, exist := query["restore_table_mapping"]; exist {
for _, tableMapping := range tableMappingQuery {
mappingItems := strings.Split(tableMapping, ",")
for _, m := range mappingItems {
if strings.Count(m, ":") != 1 || !tableMappingRE.MatchString(m) {
api.writeError(w, http.StatusInternalServerError, "restore", fmt.Errorf("invalid values in restore_table_mapping %s", m))
return

}
}
tableMappingToRestore = append(tableMappingToRestore, mappingItems...)
}

fullCommand = fmt.Sprintf("%s --restore-table-mapping=\"%s\"", fullCommand, strings.Join(tableMappingToRestore, ","))
}

if partitions, exist := query["partitions"]; exist {
partitionsToBackup = append(partitionsToBackup, partitions...)
fullCommand = fmt.Sprintf("%s --partitions=\"%s\"", fullCommand, strings.Join(partitions, "\" --partitions=\""))
@@ -1253,7 +1273,7 @@ func (api *APIServer) httpRestoreHandler(w http.ResponseWriter, r *http.Request)
go func() {
err, _ := api.metrics.ExecuteWithMetrics("restore", 0, func() error {
b := backup.NewBackuper(api.config)
return b.Restore(name, tablePattern, databaseMappingToRestore, partitionsToBackup, schemaOnly, dataOnly, dropExists, ignoreDependencies, restoreRBAC, false, restoreConfigs, false, api.cliApp.Version, commandId)
return b.Restore(name, tablePattern, databaseMappingToRestore, tableMappingToRestore, partitionsToBackup, schemaOnly, dataOnly, dropExists, ignoreDependencies, restoreRBAC, false, restoreConfigs, false, api.cliApp.Version, commandId)
})
status.Current.Stop(commandId, err)
if err != nil {