 Documentation
      ¶
      Documentation
      ¶
    
    
  
    
  
    Index ¶
- Constants
- Variables
- func ApplyFileIncrement(fileName string, increment io.Reader, createNewIncrementalFiles bool, ...) error
- func BackupCopyingInfo(backup Backup, from storage.Folder, to storage.Folder) ([]copy.InfoProvider, error)
- func CleanupPrefetchDirectories(walFileName string, location string, cleaner Cleaner)
- func Connect(configOptions ...func(config *pgx.ConnConfig) error) (*pgx.Conn, error)
- func CreateFileFromIncrement(increment io.Reader, target ReadWriterAt) (int64, error)
- func DeduceBackupName(object storage.Object) string
- func ExtractDeleteGarbagePredicate(args []string) func(storage.Object) bool
- func GetBaseFilesToUnwrap(backupFileStates internal.BackupFileList, currentFilesToUnwrap map[string]bool) (map[string]bool, error)
- func GetDeltaFilenameFor(walFilename string) (string, error)
- func GetIncrementHeaderFields(increment io.Reader) (uint64, uint32, []byte, error)
- func GetLastWalFilename(backup Backup) (string, error)
- func GetMissingWals(lastSeg uint64, lastTl, currentTl uint32, ...) ([]string, error)
- func GetNextWalFilename(name string) (string, error)
- func GetPermanentBackupsAndWals(folder storage.Folder) (map[string]bool, map[string]bool)
- func GetPgFetcherNew(dbDataDirectory, fileMask, restoreSpecPath string, skipRedundantTars bool, ...) func(folder storage.Folder, backup internal.Backup)
- func GetPgFetcherOld(dbDataDirectory, fileMask, restoreSpecPath string, extractProv ExtractProvider) func(rootFolder storage.Folder, backup internal.Backup)
- func GetPositionInDelta(walFilename string) int
- func GetRelFileIDFrom(filePath string) (int, error)
- func GetRelFileNodeFrom(filePath string) (*walparser.RelFileNode, error)
- func HandleCatchupFetch(folder storage.Folder, dbDirectory, backupName string, useNewUnwrap bool)
- func HandleCatchupPush(pgDataDirectory string, fromLSN LSN)
- func HandleCopy(fromConfigFile string, toConfigFile string, backupName string, ...)
- func HandleDaemon(uploader *WalUploader, pathToSocket string)
- func HandleDetailedBackupList(folder storage.Folder, pretty bool, json bool)
- func HandleWALFetch(folder storage.Folder, walFileName string, location string, ...)
- func HandleWALPrefetch(uploader *WalUploader, walFileName string, location string)
- func HandleWALPush(uploader *WalUploader, walFilePath string) error
- func HandleWALReceive(uploader *WalUploader)
- func HandleWALRestore(targetPath, sourcePath string, cloudFolder storage.Folder)
- func HandleWalShow(rootFolder storage.Folder, showBackups bool, outputWriter WalShowOutputWriter)
- func HandleWalVerify(checkTypes []WalVerifyCheckType, rootFolder storage.Folder, ...)
- func HistoryCopyingInfo(backup Backup, from storage.Folder, to storage.Folder) ([]copy.InfoProvider, error)
- func IsPermanent(objectName string, permanentBackups, permanentWals map[string]bool) bool
- func IsPgControlRequired(backup Backup) bool
- func Listen(c net.Conn, uploader *WalUploader)
- func ParseTimelineFromBackupName(backupName string) (uint32, error)
- func ParseTimelineFromString(timelineString string) (uint32, error)
- func ParseWALFilename(name string) (timelineID uint32, logSegNo uint64, err error)
- func PrepareDirs(fileName string, targetPath string) error
- func ReadIncrementFileHeader(reader io.Reader) error
- func ReadIncrementLocations(filePath string, fileSize int64, lsn LSN) ([]walparser.BlockLocation, error)
- func ReadIncrementalFile(filePath string, fileSize int64, lsn LSN, deltaBitmap *roaring.Bitmap) (fileReader io.ReadCloser, size int64, err error)
- func RestoreMissingPages(base io.Reader, target ReadWriterAt) error
- func SdNotify(state string) error
- func SelectRelFileBlocks(bitmap *roaring.Bitmap, relFileID int) *roaring.Bitmap
- func SetWalSize(sizeMb uint64)
- func SortBackupDetails(backupDetails []BackupDetail)
- func ToPartFilename(deltaFilename string) string
- func TryFetchTimelineAndLogSegNo(objectName string) (uint32, uint64, bool)
- func ValidateSlotName(pgSlotName string) (err error)
- func VerifyPagedFileBase(path string, fileInfo os.FileInfo, pagedFile io.Reader) ([]uint32, error)
- func VerifyPagedFileIncrement(path string, fileInfo os.FileInfo, increment io.Reader) ([]uint32, error)
- func WildcardInfo(from storage.Folder, to storage.Folder) ([]copy.InfoProvider, error)
- func WriteBackupListDetails(backupDetails []BackupDetail, output io.Writer) error
- func WritePagesFromIncrement(increment io.Reader, target ReadWriterAt, overwriteExisting bool) (int64, error)
- func WritePrettyBackupListDetails(backupDetails []BackupDetail, output io.Writer)
- type ArchiveMessageHandler
- type Backup
- func (backup *Backup) FetchMeta() (ExtendedMetadataDto, error)
- func (backup *Backup) GetFilesToUnwrap(fileMask string) (map[string]bool, error)
- func (backup *Backup) GetSentinel() (BackupSentinelDto, error)
- func (backup *Backup) GetSentinelAndFilesMetadata() (BackupSentinelDto, FilesMetadataDto, error)
- func (backup *Backup) GetTarNames() ([]string, error)
 
- type BackupArguments
- type BackupDetail
- type BackupFileOptions
- type BackupFileUnwrapper
- type BackupHandler
- type BackupObject
- type BackupPgInfo
- type BackupSentinelDto
- type BackupSentinelDtoV2
- type BackupTerminator
- type BackupTimeSlicesOrder
- type BackupWorkers
- type BgUploader
- type Bundle
- func (bundle *Bundle) DownloadDeltaMap(folder storage.Folder, backupStartLSN LSN) error
- func (bundle *Bundle) FinishTarComposer() (internal.TarFileSets, error)
- func (bundle *Bundle) GetFiles() *sync.Map
- func (bundle *Bundle) HandleWalkedFSObject(path string, info os.FileInfo, err error) error
- func (bundle *Bundle) NewTarBall(dedicatedUploader bool) internal.TarBall
- func (bundle *Bundle) SetupComposer(composerMaker TarBallComposerMaker) (err error)
- func (bundle *Bundle) StartBackup(queryRunner *PgQueryRunner, backup string) (backupName string, lsn LSN, err error)
- func (bundle *Bundle) UploadPgControl(compressorFileExtension string) error
 
- type BytesPerWalSegmentError
- type CantDiscardWalDataError
- type CantOverwriteWalFileError
- type CatchupDeltaBackupConfigurator
- type CatchupFileUnwrapper
- type CheckMessageHandler
- type Cleaner
- type CopyTarBallComposer
- func (c *CopyTarBallComposer) AddFile(info *internal.ComposeFileInfo)
- func (c *CopyTarBallComposer) AddHeader(fileInfoHeader *tar.Header, info os.FileInfo) error
- func (c *CopyTarBallComposer) FinishComposing() (internal.TarFileSets, error)
- func (c *CopyTarBallComposer) GetFiles() internal.BundleFiles
- func (c *CopyTarBallComposer) SkipFile(tarHeader *tar.Header, fileInfo os.FileInfo)
 
- type CopyTarBallComposerMaker
- type CurBackupInfo
- type DefaultFileUnwrapper
- type DeleteHandler
- type DeltaBackupConfigurator
- type DeltaFile
- type DeltaFileChanWriter
- type DeltaFileManager
- func (manager *DeltaFileManager) CancelRecording(walFilename string)
- func (manager *DeltaFileManager) CombinePartFile(deltaFilename string, partFile *WalPartFile) error
- func (manager *DeltaFileManager) FlushDeltaFiles(uploader *internal.Uploader, completedPartFiles map[string]bool)
- func (manager *DeltaFileManager) FlushFiles(uploader *internal.Uploader)
- func (manager *DeltaFileManager) FlushPartFiles() (completedPartFiles map[string]bool)
- func (manager *DeltaFileManager) GetBlockLocationConsumer(deltaFilename string) (chan walparser.BlockLocation, error)
- func (manager *DeltaFileManager) GetPartFile(deltaFilename string) (*WalPartFile, error)
 
- type DeltaFileWriterNotFoundError
- type DeltaNo
- type DeprecatedSentinelFields
- type DirDatabaseTarBallComposer
- func (d DirDatabaseTarBallComposer) AddFile(info *internal.ComposeFileInfo)
- func (d DirDatabaseTarBallComposer) AddHeader(header *tar.Header, fileInfo os.FileInfo) error
- func (d DirDatabaseTarBallComposer) FinishComposing() (internal.TarFileSets, error)
- func (d DirDatabaseTarBallComposer) GetFiles() internal.BundleFiles
- func (d DirDatabaseTarBallComposer) SkipFile(tarHeader *tar.Header, fileInfo os.FileInfo)
 
- type DirDatabaseTarBallComposerMaker
- type ExtendedMetadataDto
- type ExtractProvider
- type ExtractProviderDBSpec
- type ExtractProviderImpl
- type FetchConfig
- type FileTarInterpreter
- type FileUnwrapResult
- type FileUnwrapResultType
- type FileUnwrapperType
- type FilesMetadataDto
- type FilesToExtractProvider
- type FilesToExtractProviderImpl
- type GenericMetaFetcher
- type GenericMetaInteractor
- type GenericMetaSetter
- type HistoryFileNotFoundError
- type IBackupFileUnwrapper
- type IncorrectBackupNameError
- type IncorrectLogSegNoError
- type IncrementDetailsFetcher
- type IncrementalPageReader
- func (pageReader *IncrementalPageReader) AdvanceFileReader() error
- func (pageReader *IncrementalPageReader) Close() error
- func (pageReader *IncrementalPageReader) DeltaBitmapInitialize(deltaBitmap *roaring.Bitmap)
- func (pageReader *IncrementalPageReader) DrainMoreData() (succeed bool, err error)
- func (pageReader *IncrementalPageReader) FullScanInitialize() error
- func (pageReader *IncrementalPageReader) Read(p []byte) (n int, err error)
- func (pageReader *IncrementalPageReader) SelectNewValidPage(pageBytes []byte, blockNo uint32) (valid bool)
- func (pageReader *IncrementalPageReader) WriteDiffMapToHeader(headerWriter io.Writer)
 
- type IncrementalTarInterpreter
- type IntegrityCheckDetails
- type IntegrityCheckRunner
- type IntegrityScanSegmentSequence
- type InvalidBlockError
- type InvalidIncrementFileHeaderError
- type InvalidWalFileMagicError
- type LSN
- type NilWalParserError
- type NoBitmapFoundError
- type NoCorrectBackupFoundError
- type NoPostgresVersionError
- type NonEmptyDBDataDirectoryError
- type NotWalFilenameError
- type PageHeader
- type PagedFileDeltaMap
- type PgAliveWatcher
- type PgChecksummablePage
- type PgControlData
- type PgControlNotFoundError
- type PgDatabaseInfo
- type PgDatabasePage
- type PgQueryRunner
- func (queryRunner *PgQueryRunner) BuildGetDatabasesQuery() (string, error)
- func (queryRunner *PgQueryRunner) BuildStartBackup() (string, error)
- func (queryRunner *PgQueryRunner) BuildStatisticsQuery() (string, error)
- func (queryRunner *PgQueryRunner) BuildStopBackup() (string, error)
- func (queryRunner *PgQueryRunner) GetDataDir() (dataDir string, err error)
- func (queryRunner *PgQueryRunner) GetDatabaseInfos() ([]PgDatabaseInfo, error)
- func (queryRunner *PgQueryRunner) GetParameter(parameterName string) (string, error)
- func (queryRunner *PgQueryRunner) GetPhysicalSlotInfo(slotName string) (PhysicalSlot, error)
- func (queryRunner *PgQueryRunner) GetWalSegmentBytes() (segBlocks uint64, err error)
- func (queryRunner *PgQueryRunner) IsTablespaceMapExists() bool
- func (queryRunner *PgQueryRunner) Ping() error
 
- type PgRelationStat
- type PhysicalSlot
- type PrevBackupInfo
- type ProcessMessageResult
- type QueryRunner
- type RatedComposeFileInfo
- type RatingTarBallComposer
- func (c *RatingTarBallComposer) AddFile(info *internal.ComposeFileInfo)
- func (c *RatingTarBallComposer) AddHeader(fileInfoHeader *tar.Header, info os.FileInfo) error
- func (c *RatingTarBallComposer) FinishComposing() (internal.TarFileSets, error)
- func (c *RatingTarBallComposer) GetFiles() internal.BundleFiles
- func (c *RatingTarBallComposer) SkipFile(tarHeader *tar.Header, fileInfo os.FileInfo)
 
- type RatingTarBallComposerMaker
- type ReachedStopSegmentError
- type ReadWriterAt
- type ReadWriterAtFileImpl
- type RegularDeltaBackupConfigurator
- type RegularTarBallComposer
- func (c *RegularTarBallComposer) AddFile(info *internal.ComposeFileInfo)
- func (c *RegularTarBallComposer) AddHeader(fileInfoHeader *tar.Header, info os.FileInfo) error
- func (c *RegularTarBallComposer) FinishComposing() (internal.TarFileSets, error)
- func (c *RegularTarBallComposer) GetFiles() internal.BundleFiles
- func (c *RegularTarBallComposer) SkipFile(tarHeader *tar.Header, fileInfo os.FileInfo)
 
- type RegularTarBallComposerMaker
- type RelFileStatistics
- type ScannedSegmentDescription
- type ScannedSegmentStatus
- type SegmentScanConfig
- type SkippedFileError
- type SocketMessageHandler
- type SocketMessageReader
- type SocketMessageType
- type StatBundleFiles
- func (files *StatBundleFiles) AddFile(tarHeader *tar.Header, fileInfo os.FileInfo, isIncremented bool)
- func (files *StatBundleFiles) AddFileDescription(name string, backupFileDescription internal.BackupFileDescription)
- func (files *StatBundleFiles) AddFileWithCorruptBlocks(tarHeader *tar.Header, fileInfo os.FileInfo, isIncremented bool, ...)
- func (files *StatBundleFiles) AddSkippedFile(tarHeader *tar.Header, fileInfo os.FileInfo)
- func (files *StatBundleFiles) GetUnderlyingMap() *sync.Map
 
- type StreamingBaseBackup
- func (bb *StreamingBaseBackup) BackupName() string
- func (bb *StreamingBaseBackup) FileName() string
- func (bb *StreamingBaseBackup) Finish() (err error)
- func (bb *StreamingBaseBackup) GetTablespaceSpec() *TablespaceSpec
- func (bb *StreamingBaseBackup) Path() string
- func (bb *StreamingBaseBackup) Read(p []byte) (n int, err error)
- func (bb *StreamingBaseBackup) Start(verifyChecksum bool, diskLimit int32) (err error)
- func (bb *StreamingBaseBackup) Upload(uploader *WalUploader, bundleFiles internal.BundleFiles) (err error)
 
- type TablespaceLocation
- type TablespaceSpec
- func (spec *TablespaceSpec) BasePrefix() (string, bool)
- func (spec *TablespaceSpec) EnsureSymlinkExist(location TablespaceLocation) error
- func (spec *TablespaceSpec) MarshalJSON() ([]byte, error)
- func (spec *TablespaceSpec) TablespaceNames() []string
- func (spec *TablespaceSpec) UnmarshalJSON(b []byte) error
 
- type TarBallComposerMaker
- type TarBallComposerType
- type TarBallFilePackerImpl
- type TarBallFilePackerOptions
- type TarFilesCollection
- type TarSizeError
- type TarballStreamer
- type TarballStreamerRemap
- type TarballStreamerRemaps
- type TimeLineHistFile
- type TimeLineHistFileRow
- type TimelineCheckDetails
- type TimelineCheckRunner
- type TimelineHistoryRecord
- type TimelineInfo
- type TimelineWithSegmentNo
- type UnexpectedTarDataError
- type UnknownIncrementFileHeaderError
- type UnknownTableSpaceError
- type UnknownWalVerifyCheckError
- type UnsupportedPostgresVersionError
- type UnwrapResult
- type WalDeltaRecorder
- type WalDeltaRecordingReader
- type WalMetadataDescription
- type WalMetadataUploader
- type WalPart
- type WalPartDataType
- type WalPartFile
- type WalPartRecorder
- type WalSegment
- type WalSegmentDescription
- type WalSegmentNo
- type WalSegmentNotFoundError
- type WalSegmentRunner
- type WalSegmentScanner
- type WalSegmentsSequence
- type WalShowJSONOutputWriter
- type WalShowOutputType
- type WalShowOutputWriter
- type WalShowTableOutputWriter
- type WalUploader
- type WalVerifyCheckDetails
- type WalVerifyCheckResult
- type WalVerifyCheckRunner
- type WalVerifyCheckStatus
- type WalVerifyCheckType
- type WalVerifyJSONOutputWriter
- type WalVerifyOutputType
- type WalVerifyOutputWriter
- type WalVerifyTableOutputWriter
Constants ¶
const ( PgControlPath = "/global/pg_control" FilesMetadataName = "files_metadata.json" )
const ( PgControl = "pg_control" BackupLabelFilename = "backup_label" TablespaceMapFilename = "tablespace_map" TablespaceFolder = "pg_tblspc" )
const ( DeleteGarbageArchivesModifier = "ARCHIVES" DeleteGarbageBackupsModifier = "BACKUPS" )
const ( RelFileSizeBound = 1 << 30 BlocksInRelFile = int(RelFileSizeBound / DatabasePageSize) DefaultSpcNode walparser.Oid = 1663 )
const ( // number of checksums to calculate in parallel NSums int = 32 // prime multiplier of FNV-1a hash FnvPrime uint32 = 16777619 // page header checksum offset PdChecksumOffset = 8 // page header checksum length (in bytes) PdChecksumLen = 2 )
const ( DatabasePageSize = int64(walparser.BlockSize) SignatureMagicNumber byte = 0x55 DefaultTablespace = "base" GlobalTablespace = "global" NonDefaultTablespace = "pg_tblspc" )
const ( BasePrefix = "base_prefix" Tablespaces = "tablespaces" )
const ( WalFileInDelta uint64 = 16 DeltaFilenameSuffix string = "_delta" PartFilenameSuffix string = "_part" )
const ( WalBulkMetadataLevel = "BULK" WalIndividualMetadataLevel = "INDIVIDUAL" WalNoMetadataLevel = "NOMETADATA" )
const ( TimelineOkStatus = "OK" TimelineLostSegmentStatus = "LOST_SEGMENTS" )
const ( WalVerifyIntegrityCheck = iota + 1 WalVerifyTimelineCheck )
const MetadataDatetimeFormat = "%Y-%m-%dT%H:%M:%S.%fZ"
    const PatternLSN = "[0-9A-F]{8}"
    const PatternTimelineAndLogSegNo = "[0-9A-F]{24}"
    const (
	RecordPartFilename = "currentRecord.part"
)
    const ( // Sets standbyMessageTimeout in Streaming Replication Protocol. StandbyMessageTimeout = time.Second * 10 )
Variables ¶
var ErrorBasePrefixMissing = fmt.Errorf("base prefix not set while working with tablespaces")
    var ExcludedFilenames = make(map[string]utility.Empty)
    ExcludedFilenames is a list of excluded members from the bundled backup.
var IncrementFileHeader = []byte{'w', 'i', '1', SignatureMagicNumber}
    IncrementFileHeader contains "wi" at the head which stands for "wal-g increment" format version "1", signature magic number
var UnwrapAll map[string]bool
    var UtilityFilePaths = map[string]bool{ PgControlPath: true, BackupLabelFilename: true, TablespaceMapFilename: true, }
var WalMetadataLevels = []string{WalBulkMetadataLevel, WalIndividualMetadataLevel, WalNoMetadataLevel}
    var ( // WalSegmentSize is the size of one WAL file WalSegmentSize = uint64(16 * 1024 * 1024) )
Functions ¶
func ApplyFileIncrement ¶
func ApplyFileIncrement(fileName string, increment io.Reader, createNewIncrementalFiles bool, fsync bool) error
ApplyFileIncrement changes pages according to supplied change map file
func BackupCopyingInfo ¶
func Connect ¶
Connect establishes a connection to postgres using a UNIX socket. Must export PGHOST and run with `sudo -E -u postgres`. If PGHOST is not set or if the connection fails, an error is returned and the connection is `<nil>`.
Example: PGHOST=/var/run/postgresql or PGHOST=10.0.0.1
func CreateFileFromIncrement ¶
func CreateFileFromIncrement(increment io.Reader, target ReadWriterAt) (int64, error)
CreateFileFromIncrement writes the pages from the increment to local file and write empty blocks in place of pages which are not present in the increment
func DeduceBackupName ¶
func ExtractDeleteGarbagePredicate ¶
ExtractDeleteGarbagePredicate extracts delete modifier the "delete garbage" command
func GetBaseFilesToUnwrap ¶
func GetDeltaFilenameFor ¶
func GetLastWalFilename ¶
func GetMissingWals ¶
func GetMissingWals(lastSeg uint64, lastTl, currentTl uint32, tlToSeg map[uint32]*TimelineWithSegmentNo, walsByTimelines map[uint32]*WalSegmentsSequence, ) ([]string, error)
GetMissingWals collect the slice of WAL filenames by last LSN, last timeline, current timeline, history records and folder
func GetNextWalFilename ¶
GetNextWalFilename computes name of next WAL segment
func GetPgFetcherNew ¶
func GetPgFetcherOld ¶
func GetPositionInDelta ¶
func GetRelFileIDFrom ¶
func GetRelFileNodeFrom ¶
func GetRelFileNodeFrom(filePath string) (*walparser.RelFileNode, error)
func HandleCatchupFetch ¶
HandleCatchupFetch is invoked to perform wal-g catchup-fetch
func HandleCatchupPush ¶
HandleCatchupPush is invoked to perform a wal-g catchup-push
func HandleCopy ¶
HandleCopy copy specific or all backups from one storage to another
func HandleDaemon ¶
func HandleDaemon(uploader *WalUploader, pathToSocket string)
HandleDaemon is invoked to perform daemon mode
func HandleDetailedBackupList ¶
TODO : unit tests
func HandleWALFetch ¶
func HandleWALFetch(folder storage.Folder, walFileName string, location string, triggerPrefetch bool)
TODO : unit tests HandleWALFetch is invoked to performa wal-g wal-fetch
func HandleWALPrefetch ¶
func HandleWALPrefetch(uploader *WalUploader, walFileName string, location string)
TODO : unit tests HandleWALPrefetch is invoked by wal-fetch command to speed up database restoration
func HandleWALPush ¶
func HandleWALPush(uploader *WalUploader, walFilePath string) error
TODO : unit tests HandleWALPush is invoked to perform wal-g wal-push
func HandleWALReceive ¶
func HandleWALReceive(uploader *WalUploader)
HandleWALReceive is invoked to receive wal with a replication connection and push
func HandleWALRestore ¶
HandleWALRestore is invoked to perform wal-g wal-restore
func HandleWalShow ¶
func HandleWalShow(rootFolder storage.Folder, showBackups bool, outputWriter WalShowOutputWriter)
HandleWalShow gets the list of files inside WAL folder, detects the available WAL segments, groups WAL segments by the timeline and shows detailed info about each timeline stored in storage
func HandleWalVerify ¶
func HandleWalVerify( checkTypes []WalVerifyCheckType, rootFolder storage.Folder, currentWalSegment WalSegmentDescription, outputWriter WalVerifyOutputWriter, )
HandleWalVerify builds a check runner for each check type and writes the check results to the provided output writer
func HistoryCopyingInfo ¶
func IsPermanent ¶
func IsPgControlRequired ¶
func Listen ¶
func Listen(c net.Conn, uploader *WalUploader)
Listen is used for listening connection and processing messages
func ParseTimelineFromString ¶
func ParseWALFilename ¶
ParseWALFilename extracts numeric parts from WAL file name
func PrepareDirs ¶
PrepareDirs makes sure all dirs exist
func ReadIncrementFileHeader ¶
func ReadIncrementLocations ¶
func ReadIncrementalFile ¶
func RestoreMissingPages ¶
func RestoreMissingPages(base io.Reader, target ReadWriterAt) error
RestoreMissingPages restores missing pages (zero blocks) of local file with their base backup version
func SelectRelFileBlocks ¶
func SetWalSize ¶
func SetWalSize(sizeMb uint64)
func SortBackupDetails ¶
func SortBackupDetails(backupDetails []BackupDetail)
func ToPartFilename ¶
func ValidateSlotName ¶
ValidateSlotName validates pgSlotName to be a valid slot name
func VerifyPagedFileBase ¶
VerifyPagedFileBase verifies pages of a standard paged file
func VerifyPagedFileIncrement ¶
func VerifyPagedFileIncrement(path string, fileInfo os.FileInfo, increment io.Reader) ([]uint32, error)
VerifyPagedFileIncrement verifies pages of an increment
func WildcardInfo ¶
func WriteBackupListDetails ¶
func WriteBackupListDetails(backupDetails []BackupDetail, output io.Writer) error
TODO : unit tests
func WritePagesFromIncrement ¶
func WritePagesFromIncrement(increment io.Reader, target ReadWriterAt, overwriteExisting bool) (int64, error)
WritePagesFromIncrement writes pages from delta backup according to diffMap
func WritePrettyBackupListDetails ¶
func WritePrettyBackupListDetails(backupDetails []BackupDetail, output io.Writer)
TODO : unit tests
Types ¶
type ArchiveMessageHandler ¶
type ArchiveMessageHandler struct {
	// contains filtered or unexported fields
}
    func (*ArchiveMessageHandler) Handle ¶
func (h *ArchiveMessageHandler) Handle(messageBody []byte) error
type Backup ¶
type Backup struct {
	internal.Backup
	SentinelDto      *BackupSentinelDto // used for storage query caching
	FilesMetadataDto *FilesMetadataDto
}
    Backup contains information about a valid Postgres backup generated and uploaded by WAL-G.
func ToPgBackup ¶
func (*Backup) FetchMeta ¶
func (backup *Backup) FetchMeta() (ExtendedMetadataDto, error)
func (*Backup) GetFilesToUnwrap ¶
func (*Backup) GetSentinel ¶
func (backup *Backup) GetSentinel() (BackupSentinelDto, error)
func (*Backup) GetSentinelAndFilesMetadata ¶
func (backup *Backup) GetSentinelAndFilesMetadata() (BackupSentinelDto, FilesMetadataDto, error)
func (*Backup) GetTarNames ¶
type BackupArguments ¶
type BackupArguments struct {
	// contains filtered or unexported fields
}
    BackupArguments holds all arguments parsed from cmd to this handler class
func NewBackupArguments ¶
func NewBackupArguments(pgDataDirectory string, backupsFolder string, isPermanent bool, verifyPageChecksums bool, isFullBackup bool, storeAllCorruptBlocks bool, tarBallComposerType TarBallComposerType, deltaConfigurator DeltaBackupConfigurator, userData interface{}, withoutFilesMetadata bool) BackupArguments
NewBackupArguments creates a BackupArgument object to hold the arguments from the cmd
type BackupDetail ¶
type BackupDetail struct {
	internal.BackupTime
	ExtendedMetadataDto
}
    BackupDetails is used to append ExtendedMetadataDto details to BackupTime struct
func GetBackupDetails ¶
func GetBackupDetails(folder storage.Folder, backupTime internal.BackupTime) (BackupDetail, error)
func GetBackupsDetails ¶
func GetBackupsDetails(folder storage.Folder, backups []internal.BackupTime) ([]BackupDetail, error)
type BackupFileOptions ¶
type BackupFileOptions struct {
	// contains filtered or unexported fields
}
    type BackupFileUnwrapper ¶
type BackupFileUnwrapper struct {
	// contains filtered or unexported fields
}
    type BackupHandler ¶
type BackupHandler struct {
	CurBackupInfo CurBackupInfo
	Arguments BackupArguments
	Workers   BackupWorkers
	PgInfo    BackupPgInfo
	// contains filtered or unexported fields
}
    BackupHandler is the main struct which is handling the backup process
func NewBackupHandler ¶
func NewBackupHandler(arguments BackupArguments) (bh *BackupHandler, err error)
NewBackupHandler returns a backup handler object, which can handle the backup
func (*BackupHandler) HandleBackupPush ¶
func (bh *BackupHandler) HandleBackupPush()
HandleBackupPush handles the backup being read from Postgres or filesystem and being pushed to the repository TODO : unit tests
func (*BackupHandler) SetComposerInitFunc ¶
func (bh *BackupHandler) SetComposerInitFunc(initFunc func(handler *BackupHandler) error)
type BackupObject ¶
type BackupObject struct {
	storage.Object
	BackupName string
	// contains filtered or unexported fields
}
    func (BackupObject) GetBackupName ¶
func (o BackupObject) GetBackupName() string
func (BackupObject) GetBackupTime ¶
func (o BackupObject) GetBackupTime() time.Time
func (BackupObject) GetBaseBackupName ¶
func (o BackupObject) GetBaseBackupName() string
func (BackupObject) GetIncrementFromName ¶
func (o BackupObject) GetIncrementFromName() string
func (BackupObject) IsFullBackup ¶
func (o BackupObject) IsFullBackup() bool
type BackupPgInfo ¶
type BackupPgInfo struct {
	PgDataDirectory string
	// contains filtered or unexported fields
}
    BackupPgInfo holds the PostgreSQL info that the handler queries before running the backup
type BackupSentinelDto ¶
type BackupSentinelDto struct {
	BackupStartLSN    *LSN    `json:"LSN"`
	IncrementFromLSN  *LSN    `json:"DeltaLSN,omitempty"`
	IncrementFrom     *string `json:"DeltaFrom,omitempty"`
	IncrementFullName *string `json:"DeltaFullName,omitempty"`
	IncrementCount    *int    `json:"DeltaCount,omitempty"`
	PgVersion        int     `json:"PgVersion"`
	BackupFinishLSN  *LSN    `json:"FinishLSN"`
	SystemIdentifier *uint64 `json:"SystemIdentifier,omitempty"`
	UncompressedSize int64           `json:"UncompressedSize"`
	CompressedSize   int64           `json:"CompressedSize"`
	DataCatalogSize  int64           `json:"DataCatalogSize,omitempty"`
	TablespaceSpec   *TablespaceSpec `json:"Spec"`
	UserData interface{} `json:"UserData,omitempty"`
	FilesMetadataDisabled bool `json:"FilesMetadataDisabled,omitempty"`
}
    BackupSentinelDto describes file structure of json sentinel
func NewBackupSentinelDto ¶
func NewBackupSentinelDto(bh *BackupHandler, tbsSpec *TablespaceSpec) BackupSentinelDto
func (*BackupSentinelDto) IsIncremental ¶
func (dto *BackupSentinelDto) IsIncremental() (isIncremental bool)
TODO : unit tests TODO : get rid of panic here IsIncremental checks that sentinel represents delta backup
type BackupSentinelDtoV2 ¶
type BackupSentinelDtoV2 struct {
	BackupSentinelDto
	Version        int       `json:"Version"`
	StartTime      time.Time `json:"StartTime"`
	FinishTime     time.Time `json:"FinishTime"`
	DatetimeFormat string    `json:"DateFmt"`
	Hostname       string    `json:"Hostname"`
	DataDir        string    `json:"DataDir"`
	IsPermanent    bool      `json:"IsPermanent"`
}
    BackupSentinelDtoV2 is the future version of the backup sentinel. Basically, it is a union of BackupSentinelDto and ExtendedMetadataDto. Currently, WAL-G only uploads it, but use as the regular BackupSentinelDto. WAL-G will switch to the BackupSentinelDtoV2 in the next major release.
func NewBackupSentinelDtoV2 ¶
func NewBackupSentinelDtoV2(sentinel BackupSentinelDto, meta ExtendedMetadataDto) BackupSentinelDtoV2
type BackupTerminator ¶
type BackupTerminator struct {
	// contains filtered or unexported fields
}
    func NewBackupTerminator ¶
func NewBackupTerminator(queryRunner *PgQueryRunner, pgVersion int, pgDataDir string) *BackupTerminator
func (*BackupTerminator) TerminateBackup ¶
func (t *BackupTerminator) TerminateBackup()
type BackupTimeSlicesOrder ¶
type BackupTimeSlicesOrder int
const ( ByCreationTime BackupTimeSlicesOrder = iota ByModificationTime )
type BackupWorkers ¶
type BackupWorkers struct {
	Uploader    *WalUploader
	Bundle      *Bundle
	QueryRunner *PgQueryRunner
}
    BackupWorkers holds the external objects that the handler uses to get the backup data / write the backup data
type BgUploader ¶
type BgUploader struct {
	// contains filtered or unexported fields
}
    BgUploader represents the state of concurrent WAL upload
func NewBgUploader ¶
func NewBgUploader(walFilePath string, maxParallelWorkers int32, maxNumUploaded int32, uploader *WalUploader, preventWalOverwrite bool, readyRename bool) *BgUploader
NewBgUploader creates a new BgUploader which looks for WAL files adjacent to walFilePath. maxParallelWorkers and maxNumUploaded limits maximum concurrency and total work done by this BgUploader respectively.
func (*BgUploader) Start ¶
func (b *BgUploader) Start()
Start up checking what's inside archive_status
func (*BgUploader) Stop ¶
func (b *BgUploader) Stop() error
Stop pipeline. Stop can be safely called concurrently and repeatedly.
type Bundle ¶
type Bundle struct {
	internal.Bundle
	Timeline           uint32
	Replica            bool
	IncrementFromLsn   *LSN
	IncrementFromFiles internal.BackupFileList
	IncrementFromName  string
	DeltaMap           PagedFileDeltaMap
	TablespaceSpec     TablespaceSpec
	DataCatalogSize    *int64
	// contains filtered or unexported fields
}
    A Bundle represents the directory to be walked. Contains at least one TarBall if walk has started. Each TarBall except for the last one will be at least TarSizeThreshold bytes. The Sentinel is used to ensure complete uploaded backups; in this case, pg_control is used as the sentinel.
func NewBundle ¶
func NewBundle( directory string, crypter crypto.Crypter, incrementFromName string, incrementFromLsn *LSN, incrementFromFiles internal.BackupFileList, forceIncremental bool, tarSizeThreshold int64, ) *Bundle
TODO: use DiskDataFolder
func (*Bundle) DownloadDeltaMap ¶
func (*Bundle) FinishTarComposer ¶
func (bundle *Bundle) FinishTarComposer() (internal.TarFileSets, error)
func (*Bundle) HandleWalkedFSObject ¶
TODO : unit tests HandleWalkedFSObject walks files provided by the passed in directory and creates compressed tar members labeled as `part_00i.tar.*`, where '*' is compressor file extension.
To see which files and directories are Skipped, please consult ExcludedFilenames. Excluded directories will be created but their contents will not be included in the tar bundle.
func (*Bundle) NewTarBall ¶
NewTarBall starts writing new tarball
func (*Bundle) SetupComposer ¶
func (bundle *Bundle) SetupComposer(composerMaker TarBallComposerMaker) (err error)
func (*Bundle) StartBackup ¶
func (bundle *Bundle) StartBackup(queryRunner *PgQueryRunner, backup string) (backupName string, lsn LSN, err error)
TODO : unit tests StartBackup starts a non-exclusive base backup immediately. When finishing the backup, `backup_label` and `tablespace_map` contents are not immediately written to a file but returned instead. Returns empty string and an error if backup fails.
func (*Bundle) UploadPgControl ¶
TODO : unit tests UploadPgControl should only be called after the rest of the backup is successfully uploaded to S3.
type BytesPerWalSegmentError ¶
type BytesPerWalSegmentError struct {
	// contains filtered or unexported fields
}
    func (BytesPerWalSegmentError) Error ¶
func (err BytesPerWalSegmentError) Error() string
type CantDiscardWalDataError ¶
type CantDiscardWalDataError struct {
	// contains filtered or unexported fields
}
    func (CantDiscardWalDataError) Error ¶
func (err CantDiscardWalDataError) Error() string
type CantOverwriteWalFileError ¶
type CantOverwriteWalFileError struct {
	// contains filtered or unexported fields
}
    func (CantOverwriteWalFileError) Error ¶
func (err CantOverwriteWalFileError) Error() string
type CatchupDeltaBackupConfigurator ¶
type CatchupDeltaBackupConfigurator struct {
	// contains filtered or unexported fields
}
    func NewCatchupDeltaBackupConfigurator ¶
func NewCatchupDeltaBackupConfigurator(fakePreviousBackupSentinelDto BackupSentinelDto) CatchupDeltaBackupConfigurator
func (CatchupDeltaBackupConfigurator) Configure ¶
func (c CatchupDeltaBackupConfigurator) Configure(storage.Folder, bool) (prevBackupInfo PrevBackupInfo, incrementCount int, err error)
type CatchupFileUnwrapper ¶
type CatchupFileUnwrapper struct {
	BackupFileUnwrapper
}
    CatchupFileUnwrapper is used for catchup (catchup-push) backups
func (*CatchupFileUnwrapper) UnwrapExistingFile ¶
func (u *CatchupFileUnwrapper) UnwrapExistingFile(reader io.Reader, header *tar.Header, file *os.File, fsync bool) (*FileUnwrapResult, error)
func (*CatchupFileUnwrapper) UnwrapNewFile ¶
func (u *CatchupFileUnwrapper) UnwrapNewFile(reader io.Reader, header *tar.Header, file *os.File, fsync bool) (*FileUnwrapResult, error)
type CheckMessageHandler ¶
type CheckMessageHandler struct {
	// contains filtered or unexported fields
}
    func (*CheckMessageHandler) Handle ¶
func (h *CheckMessageHandler) Handle(messageBody []byte) error
type Cleaner ¶
Cleaner interface serves to separate file system logic from prefetch clean logic to make it testable
type CopyTarBallComposer ¶
type CopyTarBallComposer struct {
	// contains filtered or unexported fields
}
    func NewCopyTarBallComposer ¶
func NewCopyTarBallComposer( tarBallQueue *internal.TarBallQueue, tarBallFilePacker *TarBallFilePackerImpl, files *internal.RegularBundleFiles, crypter crypto.Crypter, prevBackup Backup, newBackupName string, tarUnchangedFilesCount map[string]int, prevFileTar map[string]string, prevTarFileSets internal.TarFileSets, ) (*CopyTarBallComposer, error)
func (*CopyTarBallComposer) AddFile ¶
func (c *CopyTarBallComposer) AddFile(info *internal.ComposeFileInfo)
func (*CopyTarBallComposer) FinishComposing ¶
func (c *CopyTarBallComposer) FinishComposing() (internal.TarFileSets, error)
func (*CopyTarBallComposer) GetFiles ¶
func (c *CopyTarBallComposer) GetFiles() internal.BundleFiles
type CopyTarBallComposerMaker ¶
type CopyTarBallComposerMaker struct {
	// contains filtered or unexported fields
}
    func NewCopyTarBallComposerMaker ¶
func NewCopyTarBallComposerMaker(previousBackup Backup, newBackupName string, filePackerOptions TarBallFilePackerOptions) *CopyTarBallComposerMaker
func (*CopyTarBallComposerMaker) Make ¶
func (maker *CopyTarBallComposerMaker) Make(bundle *Bundle) (internal.TarBallComposer, error)
type CurBackupInfo ¶
type CurBackupInfo struct {
	Name      string
	StartTime time.Time
	// contains filtered or unexported fields
}
    CurBackupInfo holds all information that is harvest during the backup process
type DefaultFileUnwrapper ¶
type DefaultFileUnwrapper struct {
	BackupFileUnwrapper
}
    DefaultFileUnwrapper is used for default (backup-push) backups
func (*DefaultFileUnwrapper) UnwrapExistingFile ¶
func (u *DefaultFileUnwrapper) UnwrapExistingFile(reader io.Reader, header *tar.Header, file *os.File, fsync bool) (*FileUnwrapResult, error)
func (*DefaultFileUnwrapper) UnwrapNewFile ¶
func (u *DefaultFileUnwrapper) UnwrapNewFile(reader io.Reader, header *tar.Header, file *os.File, fsync bool) (*FileUnwrapResult, error)
type DeleteHandler ¶
type DeleteHandler struct {
	internal.DeleteHandler
}
    func NewDeleteHandler ¶
func (*DeleteHandler) HandleDeleteGarbage ¶
func (dh *DeleteHandler) HandleDeleteGarbage(args []string, confirm bool) error
HandleDeleteGarbage delete outdated WAL archives and leftover backup files
type DeltaBackupConfigurator ¶
type DeltaFile ¶
type DeltaFile struct {
	Locations []walparser.BlockLocation
	WalParser *walparser.WalParser
}
    type DeltaFileChanWriter ¶
type DeltaFileChanWriter struct {
	DeltaFile             *DeltaFile
	BlockLocationConsumer chan walparser.BlockLocation
}
    func NewDeltaFileChanWriter ¶
func NewDeltaFileChanWriter(deltaFile *DeltaFile) *DeltaFileChanWriter
func (*DeltaFileChanWriter) Consume ¶
func (writer *DeltaFileChanWriter) Consume(waitGroup *sync.WaitGroup)
type DeltaFileManager ¶
type DeltaFileManager struct {
	PartFiles        *internal.LazyCache[string, *WalPartFile]
	DeltaFileWriters *internal.LazyCache[string, *DeltaFileChanWriter]
	CanceledDeltaFiles map[string]bool
	// contains filtered or unexported fields
}
    func NewDeltaFileManager ¶
func NewDeltaFileManager(dataFolder fsutil.DataFolder) *DeltaFileManager
func (*DeltaFileManager) CancelRecording ¶
func (manager *DeltaFileManager) CancelRecording(walFilename string)
func (*DeltaFileManager) CombinePartFile ¶
func (manager *DeltaFileManager) CombinePartFile(deltaFilename string, partFile *WalPartFile) error
func (*DeltaFileManager) FlushDeltaFiles ¶
func (manager *DeltaFileManager) FlushDeltaFiles(uploader *internal.Uploader, completedPartFiles map[string]bool)
func (*DeltaFileManager) FlushFiles ¶
func (manager *DeltaFileManager) FlushFiles(uploader *internal.Uploader)
func (*DeltaFileManager) FlushPartFiles ¶
func (manager *DeltaFileManager) FlushPartFiles() (completedPartFiles map[string]bool)
func (*DeltaFileManager) GetBlockLocationConsumer ¶
func (manager *DeltaFileManager) GetBlockLocationConsumer(deltaFilename string) (chan walparser.BlockLocation, error)
func (*DeltaFileManager) GetPartFile ¶
func (manager *DeltaFileManager) GetPartFile(deltaFilename string) (*WalPartFile, error)
type DeltaFileWriterNotFoundError ¶
type DeltaFileWriterNotFoundError struct {
	// contains filtered or unexported fields
}
    func (DeltaFileWriterNotFoundError) Error ¶
func (err DeltaFileWriterNotFoundError) Error() string
type DeprecatedSentinelFields ¶
type DeprecatedSentinelFields struct {
	FilesMetadataDto
	DeltaFromLSN *LSN `json:"DeltaFromLSN,omitempty"`
}
    type DirDatabaseTarBallComposer ¶
type DirDatabaseTarBallComposer struct {
	// contains filtered or unexported fields
}
    func (DirDatabaseTarBallComposer) AddFile ¶
func (d DirDatabaseTarBallComposer) AddFile(info *internal.ComposeFileInfo)
func (DirDatabaseTarBallComposer) FinishComposing ¶
func (d DirDatabaseTarBallComposer) FinishComposing() (internal.TarFileSets, error)
func (DirDatabaseTarBallComposer) GetFiles ¶
func (d DirDatabaseTarBallComposer) GetFiles() internal.BundleFiles
type DirDatabaseTarBallComposerMaker ¶
type DirDatabaseTarBallComposerMaker struct {
	// contains filtered or unexported fields
}
    func NewDirDatabaseTarBallComposerMaker ¶
func NewDirDatabaseTarBallComposerMaker(files internal.BundleFiles, filePackerOptions TarBallFilePackerOptions, tarFileSets internal.TarFileSets) *DirDatabaseTarBallComposerMaker
func (DirDatabaseTarBallComposerMaker) Make ¶
func (m DirDatabaseTarBallComposerMaker) Make(bundle *Bundle) (internal.TarBallComposer, error)
type ExtendedMetadataDto ¶
type ExtendedMetadataDto struct {
	StartTime        time.Time `json:"start_time"`
	FinishTime       time.Time `json:"finish_time"`
	DatetimeFormat   string    `json:"date_fmt"`
	Hostname         string    `json:"hostname"`
	DataDir          string    `json:"data_dir"`
	PgVersion        int       `json:"pg_version"`
	StartLsn         LSN       `json:"start_lsn"`
	FinishLsn        LSN       `json:"finish_lsn"`
	IsPermanent      bool      `json:"is_permanent"`
	SystemIdentifier *uint64   `json:"system_identifier"`
	UncompressedSize int64 `json:"uncompressed_size"`
	CompressedSize   int64 `json:"compressed_size"`
	UserData interface{} `json:"user_data,omitempty"`
}
    Extended metadata should describe backup in more details, but be small enough to be downloaded often
func NewExtendedMetadataDto ¶
func NewExtendedMetadataDto(isPermanent bool, dataDir string, startTime time.Time, sentinelDto BackupSentinelDto) (meta ExtendedMetadataDto)
type ExtractProvider ¶
type ExtractProviderDBSpec ¶
type ExtractProviderDBSpec struct {
	ExtractProviderImpl
	// contains filtered or unexported fields
}
    func NewExtractProviderDBSpec ¶
func NewExtractProviderDBSpec(onlyDatabases []int) *ExtractProviderDBSpec
func (ExtractProviderDBSpec) Get ¶
func (t ExtractProviderDBSpec) Get( backup Backup, filesToUnwrap map[string]bool, skipRedundantTars bool, dbDataDir string, createNewIncrementalFiles bool, ) (IncrementalTarInterpreter, []internal.ReaderMaker, string, error)
type ExtractProviderImpl ¶
type ExtractProviderImpl struct {
	FilesToExtractProviderImpl
}
    func (ExtractProviderImpl) Get ¶
func (t ExtractProviderImpl) Get( backup Backup, filesToUnwrap map[string]bool, skipRedundantTars bool, dbDataDir string, createNewIncrementalFiles bool, ) (IncrementalTarInterpreter, []internal.ReaderMaker, string, error)
type FetchConfig ¶
type FetchConfig struct {
	// contains filtered or unexported fields
}
    func NewFetchConfig ¶
func NewFetchConfig(backupName, dbDataDirectory string, folder storage.Folder, spec *TablespaceSpec, filesToUnwrap map[string]bool, skipRedundantTars bool, manager ExtractProvider) *FetchConfig
func (*FetchConfig) SkipRedundantFiles ¶
func (fc *FetchConfig) SkipRedundantFiles(unwrapResult *UnwrapResult)
type FileTarInterpreter ¶
type FileTarInterpreter struct {
	DBDataDirectory string
	Sentinel        BackupSentinelDto
	FilesMetadata   FilesMetadataDto
	FilesToUnwrap   map[string]bool
	UnwrapResult    *UnwrapResult
	// contains filtered or unexported fields
}
    FileTarInterpreter extracts input to disk.
func NewFileTarInterpreter ¶
func NewFileTarInterpreter( dbDataDirectory string, sentinel BackupSentinelDto, filesMetadata FilesMetadataDto, filesToUnwrap map[string]bool, createNewIncrementalFiles bool, ) *FileTarInterpreter
func (*FileTarInterpreter) AddFileUnwrapResult ¶
func (tarInterpreter *FileTarInterpreter) AddFileUnwrapResult(result *FileUnwrapResult, fileName string)
func (*FileTarInterpreter) GetUnwrapResult ¶
func (tarInterpreter *FileTarInterpreter) GetUnwrapResult() *UnwrapResult
func (*FileTarInterpreter) Interpret ¶
func (tarInterpreter *FileTarInterpreter) Interpret(fileReader io.Reader, fileInfo *tar.Header) error
Interpret extracts a tar file to disk and creates needed directories. Returns the first error encountered. Calls fsync after each file is written successfully.
type FileUnwrapResult ¶
type FileUnwrapResult struct {
	FileUnwrapResultType
	// contains filtered or unexported fields
}
    func NewCompletedResult ¶
func NewCompletedResult() *FileUnwrapResult
func NewCreatedFromIncrementResult ¶
func NewCreatedFromIncrementResult(missingBlockCount int64) *FileUnwrapResult
func NewSkippedResult ¶
func NewSkippedResult() *FileUnwrapResult
func NewWroteIncrementBlocksResult ¶
func NewWroteIncrementBlocksResult(restoredBlockCount int64) *FileUnwrapResult
type FileUnwrapResultType ¶
type FileUnwrapResultType int
const ( Completed FileUnwrapResultType = iota + 1 CreatedFromIncrement WroteIncrementBlocks Skipped )
type FileUnwrapperType ¶
type FileUnwrapperType int
const ( DefaultBackupFileUnwrapper FileUnwrapperType = iota + 1 CatchupBackupFileUnwrapper )
type FilesMetadataDto ¶
type FilesMetadataDto struct {
	Files       internal.BackupFileList `json:"Files,omitempty"`
	TarFileSets map[string][]string     `json:"TarFileSets,omitempty"`
}
    FilesMetadataDto contains the information about the backup files. It can be pretty large on some databases, sometimes more than 1GB
func NewFilesMetadataDto ¶
func NewFilesMetadataDto(files internal.BackupFileList, tarFileSets internal.TarFileSets) FilesMetadataDto
type FilesToExtractProvider ¶
type FilesToExtractProviderImpl ¶
type FilesToExtractProviderImpl struct {
}
    type GenericMetaFetcher ¶
type GenericMetaFetcher struct{}
    func NewGenericMetaFetcher ¶
func NewGenericMetaFetcher() GenericMetaFetcher
func (GenericMetaFetcher) Fetch ¶
func (mf GenericMetaFetcher) Fetch(backupName string, backupFolder storage.Folder) (internal.GenericMetadata, error)
TODO: Unit tests
type GenericMetaInteractor ¶
type GenericMetaInteractor struct {
	GenericMetaFetcher
	GenericMetaSetter
}
    func NewGenericMetaInteractor ¶
func NewGenericMetaInteractor() GenericMetaInteractor
type GenericMetaSetter ¶
type GenericMetaSetter struct{}
    func NewGenericMetaSetter ¶
func NewGenericMetaSetter() GenericMetaSetter
func (GenericMetaSetter) SetIsPermanent ¶
func (ms GenericMetaSetter) SetIsPermanent(backupName string, backupFolder storage.Folder, isPermanent bool) error
TODO: Unit tests
func (GenericMetaSetter) SetUserData ¶
func (ms GenericMetaSetter) SetUserData(backupName string, backupFolder storage.Folder, userData interface{}) error
TODO: Unit tests
type HistoryFileNotFoundError ¶
type HistoryFileNotFoundError struct {
	// contains filtered or unexported fields
}
    func (HistoryFileNotFoundError) Error ¶
func (err HistoryFileNotFoundError) Error() string
type IBackupFileUnwrapper ¶
type IBackupFileUnwrapper interface {
	UnwrapNewFile(reader io.Reader, header *tar.Header, file *os.File, fsync bool) (*FileUnwrapResult, error)
	UnwrapExistingFile(reader io.Reader, header *tar.Header, file *os.File, fsync bool) (*FileUnwrapResult, error)
}
    func NewFileUnwrapper ¶
func NewFileUnwrapper(unwrapperType FileUnwrapperType, options *BackupFileOptions) IBackupFileUnwrapper
type IncorrectBackupNameError ¶
type IncorrectBackupNameError struct {
	// contains filtered or unexported fields
}
    func (IncorrectBackupNameError) Error ¶
func (err IncorrectBackupNameError) Error() string
type IncorrectLogSegNoError ¶
type IncorrectLogSegNoError struct {
	// contains filtered or unexported fields
}
    func (IncorrectLogSegNoError) Error ¶
func (err IncorrectLogSegNoError) Error() string
type IncrementDetailsFetcher ¶
type IncrementDetailsFetcher struct {
	// contains filtered or unexported fields
}
    func NewIncrementDetailsFetcher ¶
func NewIncrementDetailsFetcher(backup Backup) *IncrementDetailsFetcher
func (*IncrementDetailsFetcher) Fetch ¶
func (idf *IncrementDetailsFetcher) Fetch() (bool, internal.IncrementDetails, error)
type IncrementalPageReader ¶
type IncrementalPageReader struct {
	PagedFile ioextensions.ReadSeekCloser
	FileSize  int64
	Lsn       LSN
	Next      []byte
	Blocks    []uint32
}
    IncrementalPageReader constructs difference map during initialization and than re-read file Diff map may consist of 1Gb/PostgresBlockSize elements == 512Kb
func (*IncrementalPageReader) AdvanceFileReader ¶
func (pageReader *IncrementalPageReader) AdvanceFileReader() error
func (*IncrementalPageReader) Close ¶
func (pageReader *IncrementalPageReader) Close() error
Close IncrementalPageReader
func (*IncrementalPageReader) DeltaBitmapInitialize ¶
func (pageReader *IncrementalPageReader) DeltaBitmapInitialize(deltaBitmap *roaring.Bitmap)
func (*IncrementalPageReader) DrainMoreData ¶
func (pageReader *IncrementalPageReader) DrainMoreData() (succeed bool, err error)
func (*IncrementalPageReader) FullScanInitialize ¶
func (pageReader *IncrementalPageReader) FullScanInitialize() error
func (*IncrementalPageReader) Read ¶
func (pageReader *IncrementalPageReader) Read(p []byte) (n int, err error)
func (*IncrementalPageReader) SelectNewValidPage ¶
func (pageReader *IncrementalPageReader) SelectNewValidPage(pageBytes []byte, blockNo uint32) (valid bool)
SelectNewValidPage checks whether page is valid and if it so, then blockNo is appended to Blocks list
func (*IncrementalPageReader) WriteDiffMapToHeader ¶
func (pageReader *IncrementalPageReader) WriteDiffMapToHeader(headerWriter io.Writer)
WriteDiffMapToHeader is currently used only with buffers, so we don't handle any writing errors
type IncrementalTarInterpreter ¶
type IncrementalTarInterpreter interface {
	internal.TarInterpreter
	GetUnwrapResult() *UnwrapResult
}
    type IntegrityCheckDetails ¶
type IntegrityCheckDetails []*IntegrityScanSegmentSequence
func (IntegrityCheckDetails) NewPlainTextReader ¶
func (sequences IntegrityCheckDetails) NewPlainTextReader() (io.Reader, error)
type IntegrityCheckRunner ¶
type IntegrityCheckRunner struct {
	// contains filtered or unexported fields
}
    IntegrityCheckRunner queries the current cluster WAL segment and timeline and travels through WAL segments in storage in reversed chronological order (starting from that segment) to find any missing WAL segments that could potentially fail the PITR procedure
func NewIntegrityCheckRunner ¶
func NewIntegrityCheckRunner( rootFolder storage.Folder, walFolderFilenames []string, currentWalSegment WalSegmentDescription, ) (IntegrityCheckRunner, error)
func (IntegrityCheckRunner) Run ¶
func (check IntegrityCheckRunner) Run() (WalVerifyCheckResult, error)
func (IntegrityCheckRunner) Type ¶
func (check IntegrityCheckRunner) Type() WalVerifyCheckType
type IntegrityScanSegmentSequence ¶
type IntegrityScanSegmentSequence struct {
	TimelineID    uint32               `json:"timeline_id"`
	StartSegment  string               `json:"start_segment"`
	EndSegment    string               `json:"end_segment"`
	SegmentsCount int                  `json:"segments_count"`
	Status        ScannedSegmentStatus `json:"status"`
}
    IntegrityScanSegmentSequence is a continuous sequence of segments with the same timeline and Status
type InvalidBlockError ¶
type InvalidBlockError struct {
	// contains filtered or unexported fields
}
    InvalidBlockError indicates that file contain invalid page and cannot be archived incrementally
func (InvalidBlockError) Error ¶
func (err InvalidBlockError) Error() string
type InvalidIncrementFileHeaderError ¶
type InvalidIncrementFileHeaderError struct {
	// contains filtered or unexported fields
}
    func (InvalidIncrementFileHeaderError) Error ¶
func (err InvalidIncrementFileHeaderError) Error() string
type InvalidWalFileMagicError ¶
type InvalidWalFileMagicError struct {
	// contains filtered or unexported fields
}
    func (InvalidWalFileMagicError) Error ¶
func (err InvalidWalFileMagicError) Error() string
type LSN ¶
type LSN uint64
func FindLastCommonPoint ¶
func FindLastCommonPoint(target, source []*TimelineHistoryRecord) (LSN, uint32, error)
FindLastCommonPoint get the last common LSN and timeline between two slices of history records. Nil input is not handle
type NilWalParserError ¶
type NilWalParserError struct {
	// contains filtered or unexported fields
}
    func (NilWalParserError) Error ¶
func (err NilWalParserError) Error() string
type NoBitmapFoundError ¶
type NoBitmapFoundError struct {
	// contains filtered or unexported fields
}
    func (NoBitmapFoundError) Error ¶
func (err NoBitmapFoundError) Error() string
type NoCorrectBackupFoundError ¶
type NoCorrectBackupFoundError struct {
	// contains filtered or unexported fields
}
    func (NoCorrectBackupFoundError) Error ¶
func (err NoCorrectBackupFoundError) Error() string
type NoPostgresVersionError ¶
type NoPostgresVersionError struct {
	// contains filtered or unexported fields
}
    func NewNoPostgresVersionError ¶
func NewNoPostgresVersionError() NoPostgresVersionError
func (NoPostgresVersionError) Error ¶
func (err NoPostgresVersionError) Error() string
type NonEmptyDBDataDirectoryError ¶
type NonEmptyDBDataDirectoryError struct {
	// contains filtered or unexported fields
}
    func NewNonEmptyDBDataDirectoryError ¶
func NewNonEmptyDBDataDirectoryError(dbDataDirectory string) NonEmptyDBDataDirectoryError
func (NonEmptyDBDataDirectoryError) Error ¶
func (err NonEmptyDBDataDirectoryError) Error() string
type NotWalFilenameError ¶
type NotWalFilenameError struct {
	// contains filtered or unexported fields
}
    func (NotWalFilenameError) Error ¶
func (err NotWalFilenameError) Error() string
type PageHeader ¶
type PageHeader struct {
	// contains filtered or unexported fields
}
    type PagedFileDeltaMap ¶
type PagedFileDeltaMap map[walparser.RelFileNode]*roaring.Bitmap
func NewPagedFileDeltaMap ¶
func NewPagedFileDeltaMap() PagedFileDeltaMap
func (*PagedFileDeltaMap) AddLocationToDelta ¶
func (deltaMap *PagedFileDeltaMap) AddLocationToDelta(location walparser.BlockLocation)
func (*PagedFileDeltaMap) AddLocationsToDelta ¶
func (deltaMap *PagedFileDeltaMap) AddLocationsToDelta(locations []walparser.BlockLocation)
func (*PagedFileDeltaMap) GetDeltaBitmapFor ¶
func (deltaMap *PagedFileDeltaMap) GetDeltaBitmapFor(filePath string) (*roaring.Bitmap, error)
TODO : unit test no bitmap found
type PgAliveWatcher ¶
type PgAliveWatcher struct {
	Err <-chan error
}
    func NewPgWatcher ¶
func NewPgWatcher(queryRunner *PgQueryRunner, aliveCheckInterval time.Duration) *PgAliveWatcher
type PgChecksummablePage ¶
type PgChecksummablePage [DatabasePageSize / int64(NSums*sizeofInt32)][NSums]uint32
PgChecksummablePage represents single database page divided by NSums blocks for checksum calculation
type PgControlData ¶
type PgControlData struct {
	// contains filtered or unexported fields
}
    PgControlData represents data contained in pg_control file
func ExtractPgControl ¶
func ExtractPgControl(folder string) (*PgControlData, error)
ExtractPgControl extract pg_control data of cluster by storage
func (*PgControlData) GetCurrentTimeline ¶
func (data *PgControlData) GetCurrentTimeline() uint32
func (*PgControlData) GetSystemIdentifier ¶
func (data *PgControlData) GetSystemIdentifier() uint64
type PgControlNotFoundError ¶
type PgControlNotFoundError struct {
	// contains filtered or unexported fields
}
    func (PgControlNotFoundError) Error ¶
func (err PgControlNotFoundError) Error() string
type PgDatabaseInfo ¶
type PgDatabasePage ¶
type PgDatabasePage [DatabasePageSize]byte
PgDatabasePage represents single database page
type PgQueryRunner ¶
type PgQueryRunner struct {
	Connection       *pgx.Conn
	Version          int
	SystemIdentifier *uint64
	Mu sync.Mutex
	// contains filtered or unexported fields
}
    PgQueryRunner is implementation for controlling PostgreSQL 9.0+
func NewPgQueryRunner ¶
func NewPgQueryRunner(conn *pgx.Conn) (*PgQueryRunner, error)
NewPgQueryRunner builds QueryRunner from available connection
func (*PgQueryRunner) BuildGetDatabasesQuery ¶
func (queryRunner *PgQueryRunner) BuildGetDatabasesQuery() (string, error)
BuildGetDatabasesQuery formats a query to get all databases in cluster which are allowed to connect
func (*PgQueryRunner) BuildStartBackup ¶
func (queryRunner *PgQueryRunner) BuildStartBackup() (string, error)
BuildStartBackup formats a query that starts backup according to server features and version
func (*PgQueryRunner) BuildStatisticsQuery ¶
func (queryRunner *PgQueryRunner) BuildStatisticsQuery() (string, error)
BuildStatisticsQuery formats a query that fetch relations statistics from database
func (*PgQueryRunner) BuildStopBackup ¶
func (queryRunner *PgQueryRunner) BuildStopBackup() (string, error)
BuildStopBackup formats a query that stops backup according to server features and version
func (*PgQueryRunner) GetDataDir ¶
func (queryRunner *PgQueryRunner) GetDataDir() (dataDir string, err error)
GetDataDir reads the wals segment size (in bytes) and converts it to uint64 TODO: Unittest
func (*PgQueryRunner) GetDatabaseInfos ¶
func (queryRunner *PgQueryRunner) GetDatabaseInfos() ([]PgDatabaseInfo, error)
GetDatabaseInfos fetches a list of all databases in cluster which are allowed to connect
func (*PgQueryRunner) GetParameter ¶
func (queryRunner *PgQueryRunner) GetParameter(parameterName string) (string, error)
GetParameter reads a Postgres setting TODO: Unittest
func (*PgQueryRunner) GetPhysicalSlotInfo ¶
func (queryRunner *PgQueryRunner) GetPhysicalSlotInfo(slotName string) (PhysicalSlot, error)
GetPhysicalSlotInfo reads information on a physical replication slot TODO: Unittest
func (*PgQueryRunner) GetWalSegmentBytes ¶
func (queryRunner *PgQueryRunner) GetWalSegmentBytes() (segBlocks uint64, err error)
GetWalSegmentBytes reads the wals segment size (in bytes) and converts it to uint64 TODO: Unittest
func (*PgQueryRunner) IsTablespaceMapExists ¶
func (queryRunner *PgQueryRunner) IsTablespaceMapExists() bool
tablespace map does not exist in < 9.6 TODO: Unittest
func (*PgQueryRunner) Ping ¶
func (queryRunner *PgQueryRunner) Ping() error
type PgRelationStat ¶
type PgRelationStat struct {
	// contains filtered or unexported fields
}
    type PhysicalSlot ¶
The PhysicalSlot represents a Physical Replication Slot.
func NewPhysicalSlot ¶
func NewPhysicalSlot(name string, exists bool, active bool, restartLSN string) (slot PhysicalSlot, err error)
NewPhysicalSlot is a helper function to declare a new PhysicalSlot object and set vaues from the parsed arguments
type PrevBackupInfo ¶
type PrevBackupInfo struct {
	// contains filtered or unexported fields
}
    PrevBackupInfo holds all information that is harvest during the backup process
func NewPrevBackupInfo ¶
func NewPrevBackupInfo(name string, sentinel BackupSentinelDto, filesMeta FilesMetadataDto) PrevBackupInfo
type ProcessMessageResult ¶
type ProcessMessageResult int
The ProcessMessageResult is an enum representing possible results from the methods processing the messages as received from Postgres into the wal segment.
const ( ProcessMessageOK ProcessMessageResult = iota ProcessMessageUnknown ProcessMessageCopyDone ProcessMessageReplyRequested ProcessMessageSegmentGap ProcessMessageMismatch )
These are the multiple results that the methods can return
type QueryRunner ¶
type QueryRunner interface {
	// This call should inform the database that we are going to copy cluster's contents
	// Should fail if backup is currently impossible
	StartBackup(backup string) (string, string, bool, error)
	// Inform database that contents are copied, get information on backup
	StopBackup() (string, string, string, error)
}
    The QueryRunner interface for controlling database during backup
type RatedComposeFileInfo ¶
type RatedComposeFileInfo struct {
	internal.ComposeFileInfo
	// contains filtered or unexported fields
}
    type RatingTarBallComposer ¶
type RatingTarBallComposer struct {
	// contains filtered or unexported fields
}
    RatingTarBallComposer receives all files and tar headers that are going to be written to the backup, and composes the tarballs by placing the files with similar update rating in the same tarballs
func NewRatingTarBallComposer ¶
func NewRatingTarBallComposer( tarSizeThreshold uint64, updateRatingEvaluator internal.ComposeRatingEvaluator, incrementBaseLsn *LSN, deltaMap PagedFileDeltaMap, tarBallQueue *internal.TarBallQueue, crypter crypto.Crypter, fileStats RelFileStatistics, bundleFiles internal.BundleFiles, packer *TarBallFilePackerImpl, ) (*RatingTarBallComposer, error)
func (*RatingTarBallComposer) AddFile ¶
func (c *RatingTarBallComposer) AddFile(info *internal.ComposeFileInfo)
func (*RatingTarBallComposer) FinishComposing ¶
func (c *RatingTarBallComposer) FinishComposing() (internal.TarFileSets, error)
func (*RatingTarBallComposer) GetFiles ¶
func (c *RatingTarBallComposer) GetFiles() internal.BundleFiles
type RatingTarBallComposerMaker ¶
type RatingTarBallComposerMaker struct {
	// contains filtered or unexported fields
}
    func NewRatingTarBallComposerMaker ¶
func NewRatingTarBallComposerMaker(relFileStats RelFileStatistics, filePackerOptions TarBallFilePackerOptions) (*RatingTarBallComposerMaker, error)
func (*RatingTarBallComposerMaker) Make ¶
func (maker *RatingTarBallComposerMaker) Make(bundle *Bundle) (internal.TarBallComposer, error)
type ReachedStopSegmentError ¶
type ReachedStopSegmentError struct {
	// contains filtered or unexported fields
}
    func (ReachedStopSegmentError) Error ¶
func (err ReachedStopSegmentError) Error() string
type ReadWriterAt ¶
func NewReadWriterAtFrom ¶
func NewReadWriterAtFrom(file *os.File) (ReadWriterAt, error)
type ReadWriterAtFileImpl ¶
func (*ReadWriterAtFileImpl) Size ¶
func (rw *ReadWriterAtFileImpl) Size() int64
type RegularDeltaBackupConfigurator ¶
type RegularDeltaBackupConfigurator struct {
	// contains filtered or unexported fields
}
    func NewRegularDeltaBackupConfigurator ¶
func NewRegularDeltaBackupConfigurator(deltaBaseSelector internal.BackupSelector) RegularDeltaBackupConfigurator
func (RegularDeltaBackupConfigurator) Configure ¶
func (c RegularDeltaBackupConfigurator) Configure( folder storage.Folder, isPermanent bool, ) (prevBackupInfo PrevBackupInfo, incrementCount int, err error)
type RegularTarBallComposer ¶
type RegularTarBallComposer struct {
	// contains filtered or unexported fields
}
    func NewRegularTarBallComposer ¶
func NewRegularTarBallComposer( tarBallQueue *internal.TarBallQueue, tarBallFilePacker *TarBallFilePackerImpl, files internal.BundleFiles, tarFileSets internal.TarFileSets, crypter crypto.Crypter, ) *RegularTarBallComposer
func (*RegularTarBallComposer) AddFile ¶
func (c *RegularTarBallComposer) AddFile(info *internal.ComposeFileInfo)
func (*RegularTarBallComposer) FinishComposing ¶
func (c *RegularTarBallComposer) FinishComposing() (internal.TarFileSets, error)
func (*RegularTarBallComposer) GetFiles ¶
func (c *RegularTarBallComposer) GetFiles() internal.BundleFiles
type RegularTarBallComposerMaker ¶
type RegularTarBallComposerMaker struct {
	// contains filtered or unexported fields
}
    func NewRegularTarBallComposerMaker ¶
func NewRegularTarBallComposerMaker( filePackerOptions TarBallFilePackerOptions, files internal.BundleFiles, tarFileSets internal.TarFileSets, ) *RegularTarBallComposerMaker
func (*RegularTarBallComposerMaker) Make ¶
func (maker *RegularTarBallComposerMaker) Make(bundle *Bundle) (internal.TarBallComposer, error)
type RelFileStatistics ¶
type RelFileStatistics map[walparser.RelFileNode]PgRelationStat
type ScannedSegmentDescription ¶
type ScannedSegmentDescription struct {
	WalSegmentDescription
	// contains filtered or unexported fields
}
    type ScannedSegmentStatus ¶
type ScannedSegmentStatus int
const ( // Surely lost missing segment Lost ScannedSegmentStatus = iota + 1 // Missing but probably still uploading segment ProbablyUploading // Missing but probably delayed segment ProbablyDelayed // Segment exists in storage Found )
func (ScannedSegmentStatus) MarshalText ¶
func (status ScannedSegmentStatus) MarshalText() ([]byte, error)
MarshalText marshals the ScannedSegmentStatus enum as a string
func (ScannedSegmentStatus) String ¶
func (status ScannedSegmentStatus) String() string
type SegmentScanConfig ¶
type SegmentScanConfig struct {
	UnlimitedScan bool
	// ScanSegmentsLimit is used in case of UnlimitedScan is set to false
	ScanSegmentsLimit       int
	StopOnFirstFoundSegment bool
	// MissingSegmentStatus is set to all missing segments encountered during scan
	MissingSegmentStatus ScannedSegmentStatus
}
    SegmentScanConfig is used to configure the single Scan() call of the WalSegmentScanner
type SkippedFileError ¶
type SkippedFileError struct {
	// contains filtered or unexported fields
}
    func (SkippedFileError) Error ¶
func (err SkippedFileError) Error() string
type SocketMessageHandler ¶
func NewMessageHandler ¶
func NewMessageHandler(messageType SocketMessageType, c net.Conn, uploader *WalUploader) SocketMessageHandler
type SocketMessageReader ¶
type SocketMessageReader struct {
	// contains filtered or unexported fields
}
    func NewMessageReader ¶
func NewMessageReader(c net.Conn) *SocketMessageReader
func (SocketMessageReader) Next ¶
func (r SocketMessageReader) Next() (messageType SocketMessageType, messageBody []byte, err error)
Next method reads messages sequentially from the Reader
type SocketMessageType ¶
type SocketMessageType byte
const ( SdNotifyWatchdog = "WATCHDOG=1" CheckType SocketMessageType = 'C' FileNameType SocketMessageType = 'F' OkType SocketMessageType = 'O' ErrorType SocketMessageType = 'E' )
func (SocketMessageType) ToBytes ¶
func (msg SocketMessageType) ToBytes() []byte
type StatBundleFiles ¶
StatBundleFiles contains the bundle files. Additionally, it calculates and stores the updates count for each added file
func (*StatBundleFiles) AddFileDescription ¶
func (files *StatBundleFiles) AddFileDescription(name string, backupFileDescription internal.BackupFileDescription)
func (*StatBundleFiles) AddFileWithCorruptBlocks ¶
func (*StatBundleFiles) AddSkippedFile ¶
func (files *StatBundleFiles) AddSkippedFile(tarHeader *tar.Header, fileInfo os.FileInfo)
func (*StatBundleFiles) GetUnderlyingMap ¶
func (files *StatBundleFiles) GetUnderlyingMap() *sync.Map
type StreamingBaseBackup ¶
type StreamingBaseBackup struct {
	TimeLine uint32
	StartLSN pglogrepl.LSN
	EndLSN   pglogrepl.LSN
	UncompressedSize int64
	Files internal.BackupFileList
	// contains filtered or unexported fields
}
    The StreamingBaseBackup object represents a Postgres BASE_BACKUP, connecting to Postgres, and streaming backup data. For every tablespace, all files are combined in a tar format and streamed in a CopyData stream.
func NewStreamingBaseBackup ¶
func NewStreamingBaseBackup(pgDataDir string, maxTarSize int64, pgConn *pgconn.PgConn) (bb *StreamingBaseBackup)
NewStreamingBaseBackup will define a new StreamingBaseBackup object
func (*StreamingBaseBackup) BackupName ¶
func (bb *StreamingBaseBackup) BackupName() string
BackupName returns the name of the folder where the backup should be stored.
func (*StreamingBaseBackup) FileName ¶
func (bb *StreamingBaseBackup) FileName() string
FileName returns the filename of a tablespace backup file. This is used by the WalUploader to set the name of the destination file during upload of the backup file.
func (*StreamingBaseBackup) Finish ¶
func (bb *StreamingBaseBackup) Finish() (err error)
Finish will wrap up a backup after finalizing upload.
func (*StreamingBaseBackup) GetTablespaceSpec ¶
func (bb *StreamingBaseBackup) GetTablespaceSpec() *TablespaceSpec
GetTablespaceSpec returns the tablespace specifications.
func (*StreamingBaseBackup) Path ¶
func (bb *StreamingBaseBackup) Path() string
Path returns the name of the folder where the backup should be stored.
func (*StreamingBaseBackup) Read ¶
func (bb *StreamingBaseBackup) Read(p []byte) (n int, err error)
Read makes the StreamingBaseBackup an io.Reader, to be handled by WalUploader.UploadWalFile written to a file.
func (*StreamingBaseBackup) Start ¶
func (bb *StreamingBaseBackup) Start(verifyChecksum bool, diskLimit int32) (err error)
Start will start a base_backup read the backup info, and prepare for uploading tar files
func (*StreamingBaseBackup) Upload ¶
func (bb *StreamingBaseBackup) Upload(uploader *WalUploader, bundleFiles internal.BundleFiles) (err error)
Upload will read all tar files from Postgres, and use the uploader to upload to the backup location
type TablespaceLocation ¶
type TablespaceSpec ¶
type TablespaceSpec struct {
	// contains filtered or unexported fields
}
    The mandatory keys for this map are "base_prefix" and "tablespaces". "base_prefix" contains Location of pg_data folder. "tablespaces" contains array of keys, which also happen to be names of tablespace folders. The rest keys should be these names of tablespace folders and values should be TablespaceLocation structs.
func NewTablespaceSpec ¶
func NewTablespaceSpec(basePrefix string) TablespaceSpec
func (*TablespaceSpec) BasePrefix ¶
func (spec *TablespaceSpec) BasePrefix() (string, bool)
func (*TablespaceSpec) EnsureSymlinkExist ¶
func (spec *TablespaceSpec) EnsureSymlinkExist(location TablespaceLocation) error
func (*TablespaceSpec) MarshalJSON ¶
func (spec *TablespaceSpec) MarshalJSON() ([]byte, error)
func (*TablespaceSpec) TablespaceNames ¶
func (spec *TablespaceSpec) TablespaceNames() []string
func (*TablespaceSpec) UnmarshalJSON ¶
func (spec *TablespaceSpec) UnmarshalJSON(b []byte) error
type TarBallComposerMaker ¶
type TarBallComposerMaker interface {
	Make(bundle *Bundle) (internal.TarBallComposer, error)
}
    TarBallComposerMaker is used to make an instance of TarBallComposer
func NewTarBallComposerMaker ¶
func NewTarBallComposerMaker(composerType TarBallComposerType, queryRunner *PgQueryRunner, uploader *internal.Uploader, newBackupName string, filePackOptions TarBallFilePackerOptions, withoutFilesMetadata bool) (TarBallComposerMaker, error)
type TarBallComposerType ¶
type TarBallComposerType int
const ( RegularComposer TarBallComposerType = iota + 1 RatingComposer CopyComposer DatabaseComposer )
type TarBallFilePackerImpl ¶
type TarBallFilePackerImpl struct {
	// contains filtered or unexported fields
}
    TarBallFilePackerImpl is used to pack bundle file into tarball.
func NewTarBallFilePacker ¶
func NewTarBallFilePacker(deltaMap PagedFileDeltaMap, incrementFromLsn *LSN, files internal.BundleFiles, options TarBallFilePackerOptions) *TarBallFilePackerImpl
func (*TarBallFilePackerImpl) PackFileIntoTar ¶
func (p *TarBallFilePackerImpl) PackFileIntoTar(cfi *internal.ComposeFileInfo, tarBall internal.TarBall) error
TODO : unit tests
func (*TarBallFilePackerImpl) UpdateDeltaMap ¶
func (p *TarBallFilePackerImpl) UpdateDeltaMap(deltaMap PagedFileDeltaMap)
type TarBallFilePackerOptions ¶
type TarBallFilePackerOptions struct {
	// contains filtered or unexported fields
}
    func NewTarBallFilePackerOptions ¶
func NewTarBallFilePackerOptions(verifyPageChecksums, storeAllCorruptBlocks bool) TarBallFilePackerOptions
type TarFilesCollection ¶
type TarFilesCollection struct {
	// contains filtered or unexported fields
}
    TarFilesCollection stores the files which are going to be written to the same tarball
func (*TarFilesCollection) AddFile ¶
func (collection *TarFilesCollection) AddFile(file *RatedComposeFileInfo)
type TarSizeError ¶
type TarSizeError struct {
	// contains filtered or unexported fields
}
    func (TarSizeError) Error ¶
func (err TarSizeError) Error() string
type TarballStreamer ¶
type TarballStreamer struct {
	// files to write to extra tar (teeTar)
	Tee []string
	// io buffer where writer writes data to
	TeeIo *bytes.Buffer
	// list of remaps, remapping input file names to output file names
	Remaps TarballStreamerRemaps
	// list of processed files
	Files internal.BundleFiles
	// contains filtered or unexported fields
}
    TarballStreamer is used to modify tar files which are received streaming. Two modifications are: * remap: change (some of) the paths for files in the tar file, and * tee: copy some files to a second tar file In addition TarballStreamer maintains a list of files with their info
func NewTarballStreamer ¶
func NewTarballStreamer(input io.Reader, maxTarSize int64, bundleFiles internal.BundleFiles) (streamer *TarballStreamer)
func (*TarballStreamer) NextInputFile ¶
func (streamer *TarballStreamer) NextInputFile() (err error)
NextInputFile is what makes the TarballStreamer move to the next file.
type TarballStreamerRemap ¶
type TarballStreamerRemap struct {
	// contains filtered or unexported fields
}
    func NewTarballStreamerRemap ¶
func NewTarballStreamerRemap(from string, to string) (tsr *TarballStreamerRemap, err error)
type TarballStreamerRemaps ¶
type TarballStreamerRemaps []TarballStreamerRemap
type TimeLineHistFile ¶
type TimeLineHistFile struct {
	TimeLineID uint32
	Filename   string
	// contains filtered or unexported fields
}
    The TimeLineHistFile struct represents a TimeLineHistory file containing TimeLineHistFileRows. Since TimeLineHistFileRows are only parsed 0 or 1 rimes, the data is only preserved as []byte and parsed to TimeLineHistFileRows when required.
func NewTimeLineHistFile ¶
func NewTimeLineHistFile(timelineid uint32, filename string, body []byte) (TimeLineHistFile, error)
NewTimeLineHistFile is a helper function to define a new TimeLineHistFile
func (TimeLineHistFile) LSNToTimeLine ¶
func (tlh TimeLineHistFile) LSNToTimeLine(lsn pglogrepl.LSN) (uint32, error)
LSNToTimeLine uses rows() to get all TimeLineHistFileRows and from those rows get the timeline that a LS belongs too.
func (TimeLineHistFile) Name ¶
func (tlh TimeLineHistFile) Name() string
Name returns the filename of this wal segment. This is a convenience function used by the WalUploader.
type TimeLineHistFileRow ¶
The TimeLineHistFileRow struct represents one line in the TimeLineHistory file
type TimelineCheckDetails ¶
type TimelineCheckDetails struct {
	CurrentTimelineID        uint32 `json:"current_timeline_id"`
	HighestStorageTimelineID uint32 `json:"highest_storage_timeline_id"`
}
    func (TimelineCheckDetails) NewPlainTextReader ¶
func (details TimelineCheckDetails) NewPlainTextReader() (io.Reader, error)
type TimelineCheckRunner ¶
type TimelineCheckRunner struct {
	// contains filtered or unexported fields
}
    TimelineCheckRunner is used to verify that the current timeline is the highest among the storage timelines
func NewTimelineCheckRunner ¶
func NewTimelineCheckRunner(walFolderFilenames []string, currentSegment WalSegmentDescription) (TimelineCheckRunner, error)
func (TimelineCheckRunner) Name ¶
func (check TimelineCheckRunner) Name() string
func (TimelineCheckRunner) Run ¶
func (check TimelineCheckRunner) Run() (WalVerifyCheckResult, error)
func (TimelineCheckRunner) Type ¶
func (check TimelineCheckRunner) Type() WalVerifyCheckType
type TimelineHistoryRecord ¶
type TimelineHistoryRecord struct {
	// contains filtered or unexported fields
}
    TimelineHistoryRecord represents entry in .history file
func GetTimeLineHistoryRecords ¶
func GetTimeLineHistoryRecords(startTimeline uint32, walFolder storage.Folder) ([]*TimelineHistoryRecord, error)
func NewTimelineHistoryRecord ¶
func NewTimelineHistoryRecord(timeline uint32, lsn LSN, comment string) *TimelineHistoryRecord
type TimelineInfo ¶
type TimelineInfo struct {
	ID               uint32          `json:"id"`
	ParentID         uint32          `json:"parent_id"`
	SwitchPointLsn   LSN             `json:"switch_point_lsn"`
	StartSegment     string          `json:"start_segment"`
	EndSegment       string          `json:"end_segment"`
	SegmentsCount    int             `json:"segments_count"`
	MissingSegments  []string        `json:"missing_segments"`
	Backups          []*BackupDetail `json:"backups,omitempty"`
	SegmentRangeSize uint64          `json:"segment_range_size"`
	Status           string          `json:"status"`
}
    TimelineInfo contains information about some timeline in storage
func NewTimelineInfo ¶
func NewTimelineInfo(walSegments *WalSegmentsSequence, historyRecords []*TimelineHistoryRecord) (*TimelineInfo, error)
type TimelineWithSegmentNo ¶
type TimelineWithSegmentNo struct {
	// contains filtered or unexported fields
}
    func NewTimelineWithSegmentNo ¶
func NewTimelineWithSegmentNo(tl uint32, seg uint64) *TimelineWithSegmentNo
func NewTimelineWithSegmentNoBy ¶
func NewTimelineWithSegmentNoBy(record *TimelineHistoryRecord) *TimelineWithSegmentNo
type UnexpectedTarDataError ¶
type UnexpectedTarDataError struct {
	// contains filtered or unexported fields
}
    func (UnexpectedTarDataError) Error ¶
func (err UnexpectedTarDataError) Error() string
type UnknownIncrementFileHeaderError ¶
type UnknownIncrementFileHeaderError struct {
	// contains filtered or unexported fields
}
    func (UnknownIncrementFileHeaderError) Error ¶
func (err UnknownIncrementFileHeaderError) Error() string
type UnknownTableSpaceError ¶
type UnknownTableSpaceError struct {
	// contains filtered or unexported fields
}
    func (UnknownTableSpaceError) Error ¶
func (err UnknownTableSpaceError) Error() string
type UnknownWalVerifyCheckError ¶
type UnknownWalVerifyCheckError struct {
	// contains filtered or unexported fields
}
    func NewUnknownWalVerifyCheckError ¶
func NewUnknownWalVerifyCheckError(checkType WalVerifyCheckType) UnknownWalVerifyCheckError
func (UnknownWalVerifyCheckError) Error ¶
func (err UnknownWalVerifyCheckError) Error() string
type UnsupportedPostgresVersionError ¶
type UnsupportedPostgresVersionError struct {
	// contains filtered or unexported fields
}
    func NewUnsupportedPostgresVersionError ¶
func NewUnsupportedPostgresVersionError(version int) UnsupportedPostgresVersionError
func (UnsupportedPostgresVersionError) Error ¶
func (err UnsupportedPostgresVersionError) Error() string
type UnwrapResult ¶
type UnwrapResult struct {
	// contains filtered or unexported fields
}
    UnwrapResult stores information about the result of single backup unwrap operation
type WalDeltaRecorder ¶
type WalDeltaRecorder struct {
	// contains filtered or unexported fields
}
    func NewWalDeltaRecorder ¶
func NewWalDeltaRecorder(blockLocationConsumer chan walparser.BlockLocation) *WalDeltaRecorder
type WalDeltaRecordingReader ¶
type WalDeltaRecordingReader struct {
	PageReader       walparser.WalPageReader
	WalParser        walparser.WalParser
	PageDataLeftover []byte
	Recorder         *WalDeltaRecorder
	// contains filtered or unexported fields
}
    In case of recording error WalDeltaRecordingReader stops recording, but continues reading data correctly
func NewWalDeltaRecordingReader ¶
func NewWalDeltaRecordingReader(walFileReader io.Reader, walFilename string, manager *DeltaFileManager) (*WalDeltaRecordingReader, error)
func (*WalDeltaRecordingReader) Close ¶
func (reader *WalDeltaRecordingReader) Close() error
func (*WalDeltaRecordingReader) Read ¶
func (reader *WalDeltaRecordingReader) Read(p []byte) (n int, err error)
func (*WalDeltaRecordingReader) RecordBlockLocationsFromPage ¶
func (reader *WalDeltaRecordingReader) RecordBlockLocationsFromPage() error
type WalMetadataDescription ¶
type WalMetadataUploader ¶
type WalMetadataUploader struct {
	// contains filtered or unexported fields
}
    func NewWalMetadataUploader ¶
func NewWalMetadataUploader(walMetadataSetting string) (*WalMetadataUploader, error)
func (*WalMetadataUploader) UploadWalMetadata ¶
type WalPart ¶
type WalPart struct {
	// contains filtered or unexported fields
}
    func NewWalPart ¶
func NewWalPart(dataType WalPartDataType, id uint8, data []byte) *WalPart
type WalPartDataType ¶
type WalPartDataType uint8
const ( PreviousWalHeadType WalPartDataType = 0 WalTailType WalPartDataType = 1 WalHeadType WalPartDataType = 2 )
type WalPartFile ¶
func LoadPartFile ¶
func LoadPartFile(reader io.Reader) (*WalPartFile, error)
func NewWalPartFile ¶
func NewWalPartFile() *WalPartFile
func (*WalPartFile) CombineRecords ¶
func (partFile *WalPartFile) CombineRecords() ([]walparser.XLogRecord, error)
func (*WalPartFile) IsComplete ¶
func (partFile *WalPartFile) IsComplete() bool
type WalPartRecorder ¶
type WalPartRecorder struct {
	// contains filtered or unexported fields
}
    func NewWalPartRecorder ¶
func NewWalPartRecorder(walFilename string, manager *DeltaFileManager) (*WalPartRecorder, error)
func (*WalPartRecorder) SaveNextWalHead ¶
func (recorder *WalPartRecorder) SaveNextWalHead(head []byte) error
func (*WalPartRecorder) SavePreviousWalTail ¶
func (recorder *WalPartRecorder) SavePreviousWalTail(tailData []byte) error
type WalSegment ¶
type WalSegment struct {
	TimeLine uint32
	StartLSN pglogrepl.LSN
	// contains filtered or unexported fields
}
    The WalSegment object represents a Postgres Wal Segment, holding all wal data for a wal file.
func NewWalSegment ¶
func NewWalSegment(timeline uint32, location pglogrepl.LSN, walSegmentBytes uint64) *WalSegment
NewWalSegment is a helper function to declare a new WalSegment.
func (*WalSegment) Name ¶
func (seg *WalSegment) Name() string
Name returns the filename of this wal segment. This is also used by the WalUploader to set the name of the destination file during upload of the wal segment.
func (*WalSegment) NextWalSegment ¶
func (seg *WalSegment) NextWalSegment() (*WalSegment, error)
NextWalSegment is a helper function to create the next wal segment which comes after this wal segment. Note that this will be on the same timeline. the convenience is that it also automatically processes a message that crosses the boundary between the two segments.
func (*WalSegment) Read ¶
func (seg *WalSegment) Read(p []byte) (n int, err error)
Read is what makes the WalSegment an io.Reader, which can be handled by WalUploader.UploadWalFile to write to a file.
func (*WalSegment) Stream ¶
func (seg *WalSegment) Stream(conn *pgconn.PgConn, standbyMessageTimeout time.Duration) (ProcessMessageResult, error)
Stream is a helper function to retrieve messages from Postgres and have them processed by processMessage().
type WalSegmentDescription ¶
type WalSegmentDescription struct {
	Number   WalSegmentNo
	Timeline uint32
}
    func NewWalSegmentDescription ¶
func NewWalSegmentDescription(name string) (WalSegmentDescription, error)
func QueryCurrentWalSegment ¶
func QueryCurrentWalSegment() WalSegmentDescription
QueryCurrentWalSegment() gets start WAL segment from Postgres cluster
func (WalSegmentDescription) GetFileName ¶
func (desc WalSegmentDescription) GetFileName() string
type WalSegmentNo ¶
type WalSegmentNo uint64
type WalSegmentNotFoundError ¶
type WalSegmentNotFoundError struct {
	// contains filtered or unexported fields
}
    func (WalSegmentNotFoundError) Error ¶
func (err WalSegmentNotFoundError) Error() string
type WalSegmentRunner ¶
type WalSegmentRunner struct {
	// contains filtered or unexported fields
}
    WalSegmentRunner is used for sequential iteration over WAL segments in the storage
func NewWalSegmentRunner ¶
func NewWalSegmentRunner( startWalSegment WalSegmentDescription, segments map[WalSegmentDescription]bool, stopSegmentNo WalSegmentNo, timelineSwitchMap map[WalSegmentNo]*TimelineHistoryRecord, ) *WalSegmentRunner
func (*WalSegmentRunner) Current ¶
func (r *WalSegmentRunner) Current() WalSegmentDescription
func (*WalSegmentRunner) ForceMoveNext ¶
func (r *WalSegmentRunner) ForceMoveNext()
ForceMoveNext do a force-switch to the next segment without accessing storage
func (*WalSegmentRunner) Next ¶
func (r *WalSegmentRunner) Next() (WalSegmentDescription, error)
Next tries to get the next segment from storage
type WalSegmentScanner ¶
type WalSegmentScanner struct {
	ScannedSegments []ScannedSegmentDescription
	// contains filtered or unexported fields
}
    WalSegmentScanner is used to scan the WAL segments storage
func NewWalSegmentScanner ¶
func NewWalSegmentScanner(walSegmentRunner *WalSegmentRunner) *WalSegmentScanner
func (*WalSegmentScanner) AddScannedSegment ¶
func (sc *WalSegmentScanner) AddScannedSegment(description WalSegmentDescription, status ScannedSegmentStatus)
func (*WalSegmentScanner) GetMissingSegmentsDescriptions ¶
func (sc *WalSegmentScanner) GetMissingSegmentsDescriptions() []WalSegmentDescription
GetMissingSegmentsDescriptions returns a slice containing WalSegmentDescription of each missing segment
func (*WalSegmentScanner) Scan ¶
func (sc *WalSegmentScanner) Scan(config SegmentScanConfig) error
Scan traverse the WAL storage with WalSegmentRunner. Scan starts from the WalSegmentRunner's current position, so in case of subsequent Scan() call it will continue from the position where it stopped previously.
Scan always stops if: - Stop segment is reached OR - Unknown error encountered Also, it may be configured to stop after: - Scanning the ScanSegmentsLimit of segments - Finding the first segment which exists in WAL storage
type WalSegmentsSequence ¶
type WalSegmentsSequence struct {
	TimelineID        uint32
	WalSegmentNumbers map[WalSegmentNo]bool
	MinSegmentNo      WalSegmentNo
	MaxSegmentNo      WalSegmentNo
}
    WalSegmentsSequence represents some collection of wal segments with the same timeline
func NewSegmentsSequence ¶
func NewSegmentsSequence(id uint32, segmentNo WalSegmentNo) *WalSegmentsSequence
func (*WalSegmentsSequence) AddWalSegmentNo ¶
func (seq *WalSegmentsSequence) AddWalSegmentNo(number WalSegmentNo)
AddWalSegmentNo adds the provided segment number to collection
func (*WalSegmentsSequence) FindMissingSegments ¶
func (seq *WalSegmentsSequence) FindMissingSegments() ([]WalSegmentDescription, error)
FindMissingSegments finds missing segments in range [minSegmentNo, maxSegmentNo]
type WalShowJSONOutputWriter ¶
type WalShowJSONOutputWriter struct {
	// contains filtered or unexported fields
}
    WalShowJsonOutputWriter writes the detailed JSON output
func (*WalShowJSONOutputWriter) Write ¶
func (writer *WalShowJSONOutputWriter) Write(timelineInfos []*TimelineInfo) error
type WalShowOutputType ¶
type WalShowOutputType int
const ( TableOutput WalShowOutputType = iota + 1 JSONOutput )
type WalShowOutputWriter ¶
type WalShowOutputWriter interface {
	Write(timelineInfos []*TimelineInfo) error
}
    WalShowOutputWriter writes the output of wal-show command execution result
func NewWalShowOutputWriter ¶
func NewWalShowOutputWriter(outputType WalShowOutputType, output io.Writer, includeBackups bool) WalShowOutputWriter
type WalShowTableOutputWriter ¶
type WalShowTableOutputWriter struct {
	// contains filtered or unexported fields
}
    WalShowTableOutputWriter writes the output in compact pretty table
func (*WalShowTableOutputWriter) Write ¶
func (writer *WalShowTableOutputWriter) Write(timelineInfos []*TimelineInfo) error
type WalUploader ¶
type WalUploader struct {
	*internal.Uploader
	*DeltaFileManager
}
    WalUploader extends uploader with wal specific functionality.
func ConfigureWalUploader ¶
func ConfigureWalUploader() (uploader *WalUploader, err error)
ConfigureWalUploader connects to storage and creates an uploader. It makes sure that a valid session has started; if invalid, returns AWS error and `<nil>` values.
func ConfigureWalUploaderWithoutCompressMethod ¶
func ConfigureWalUploaderWithoutCompressMethod() (uploader *WalUploader, err error)
func NewWalUploader ¶
func NewWalUploader( compressor compression.Compressor, uploadingLocation storage.Folder, deltaFileManager *DeltaFileManager, ) *WalUploader
func (*WalUploader) FlushFiles ¶
func (walUploader *WalUploader) FlushFiles()
func (*WalUploader) UploadWalFile ¶
func (walUploader *WalUploader) UploadWalFile(file ioextensions.NamedReader) error
TODO : unit tests
type WalVerifyCheckDetails ¶
type WalVerifyCheckResult ¶
type WalVerifyCheckResult struct {
	Status  WalVerifyCheckStatus  `json:"status"`
	Details WalVerifyCheckDetails `json:"details"`
}
    WalVerifyCheckResult contains the result of some WalVerifyCheckRunner run
type WalVerifyCheckRunner ¶
type WalVerifyCheckRunner interface {
	Type() WalVerifyCheckType
	Run() (WalVerifyCheckResult, error)
}
    WalVerifyCheckRunner performs the check of WAL storage
func BuildWalVerifyCheckRunner ¶
func BuildWalVerifyCheckRunner( checkType WalVerifyCheckType, rootFolder storage.Folder, walFolderFilenames []string, currentWalSegment WalSegmentDescription, ) (WalVerifyCheckRunner, error)
type WalVerifyCheckStatus ¶
type WalVerifyCheckStatus int
const ( StatusOk WalVerifyCheckStatus = iota + 1 StatusWarning StatusFailure )
func (WalVerifyCheckStatus) MarshalText ¶
func (status WalVerifyCheckStatus) MarshalText() ([]byte, error)
MarshalText marshals the WalVerifyCheckStatus enum as a string
func (WalVerifyCheckStatus) String ¶
func (status WalVerifyCheckStatus) String() string
type WalVerifyCheckType ¶
type WalVerifyCheckType int
func (WalVerifyCheckType) MarshalText ¶
func (checkType WalVerifyCheckType) MarshalText() (text []byte, err error)
func (WalVerifyCheckType) String ¶
func (checkType WalVerifyCheckType) String() string
type WalVerifyJSONOutputWriter ¶
type WalVerifyJSONOutputWriter struct {
	// contains filtered or unexported fields
}
    WalVerifyJsonOutputWriter writes the detailed JSON output
func (*WalVerifyJSONOutputWriter) Write ¶
func (writer *WalVerifyJSONOutputWriter) Write(results map[WalVerifyCheckType]WalVerifyCheckResult) error
type WalVerifyOutputType ¶
type WalVerifyOutputType int
const ( WalVerifyTableOutput WalVerifyOutputType = iota + 1 WalVerifyJSONOutput )
type WalVerifyOutputWriter ¶
type WalVerifyOutputWriter interface {
	Write(results map[WalVerifyCheckType]WalVerifyCheckResult) error
}
    WalVerifyOutputWriter writes the output of wal-verify command execution result
func NewWalVerifyOutputWriter ¶
func NewWalVerifyOutputWriter(outputType WalVerifyOutputType, output io.Writer) WalVerifyOutputWriter
type WalVerifyTableOutputWriter ¶
type WalVerifyTableOutputWriter struct {
	// contains filtered or unexported fields
}
    WalVerifyTableOutputWriter writes the output as pretty table
func (*WalVerifyTableOutputWriter) Write ¶
func (writer *WalVerifyTableOutputWriter) Write(result map[WalVerifyCheckType]WalVerifyCheckResult) error
       Source Files
      ¶
      Source Files
      ¶
    
- backup.go
- backup_detail.go
- backup_fetch_handler.go
- backup_fetch_handler_new.go
- backup_list_handler.go
- backup_new.go
- backup_push_handler.go
- backup_sentinel_dto.go
- backup_terminator.go
- backup_util.go
- bguploader.go
- bundle.go
- bundle_files.go
- catchup_fetch_handler.go
- catchup_file_unwrapper.go
- catchup_push_handler.go
- cleaner.go
- configure.go
- connect.go
- copy.go
- copy_tar_ball_composer.go
- daemon_handler.go
- default_file_unwrapper.go
- delete.go
- delete_util.go
- delta_backup_configurator.go
- delta_file.go
- delta_file_chan_writer.go
- delta_file_manager.go
- delta_map_downloader.go
- delta_no.go
- dir_database_tar_ball_composer.go
- extract_probider_with_spec.go
- extract_provider.go
- fetch_config.go
- file_unwrapper.go
- generic_meta_interactor.go
- incremental_page_reader.go
- integrity_check_runner.go
- lsn.go
- paged_file_delta_map.go
- paged_file_verifier.go
- pagefile.go
- pagefile_new.go
- pg_alive_watcher.go
- pg_control_data.go
- physical_slot.go
- postgres_page_header.go
- prefetch.go
- query_runner.go
- rating_tar_ball_composer.go
- regular_tar_ball_composer.go
- streaming_base_backup.go
- tablespace_spec.go
- tar_ball_composer.go
- tar_ball_file_packer.go
- tar_interpreter.go
- tar_interpreter_new.go
- tarball_streamer.go
- tars_to_extract_provider.go
- timeline.go
- timeline_check_runner.go
- timeline_history.go
- timeline_history_record.go
- wal_delta_recorder.go
- wal_delta_recording_reader.go
- wal_delta_util.go
- wal_fetch_handler.go
- wal_metadata_uploader.go
- wal_part.go
- wal_part_file.go
- wal_part_recorder.go
- wal_push_handler.go
- wal_receive_handler.go
- wal_restore_handler.go
- wal_segment.go
- wal_segment_no.go
- wal_segment_runner.go
- wal_segment_scanner.go
- wal_show_handler.go
- wal_show_output_writer.go
- wal_uploader.go
- wal_verify_handler.go
- wal_verify_output_writer.go