You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@trafficcontrol.apache.org by GitBox <gi...@apache.org> on 2021/06/08 19:41:28 UTC

[GitHub] [trafficcontrol] shamrickus opened a new pull request #5924: Add tool to migrate data between TV backends

shamrickus opened a new pull request #5924:
URL: https://github.com/apache/trafficcontrol/pull/5924


   <!--
   ************ STOP!! ************
   If this Pull Request is intended to fix a security vulnerability, DO NOT submit it! Instead, contact
   the Apache Software Foundation Security Team at security@trafficcontrol.apache.org and follow the
   guidelines at https://www.apache.org/security/ regarding vulnerability disclosure.
   -->
   ## What does this PR (Pull Request) do?
   <!-- Explain the changes you made here. If this fixes an Issue, identify it by
   replacing the text in the checkbox item with the Issue number e.g.
   
   - [x] This PR fixes #9001 OR is not related to any Issue
   
   ^ This will automatically close Issue number 9001 when the Pull Request is
   merged (The '#' is important).
   
   Be sure you check the box properly, see the "The following criteria are ALL
   met by this PR" section for details.
   -->
   
   - [x] This PR is not related to any Issue <!-- You can check for an issue here: https://github.com/apache/trafficcontrol/issues -->
   
   It adds a new go tool in `tools/traffic_vault_migrate` which can be used to migrate TV data from different backends. It is intended to connect directly to the backends such that a running TO/TV service is not needed.
   
   It also moves the Postgres encryption functions to `lib/go-util` as well as adding godocs and tests.
   
   
   ## Which Traffic Control components are affected by this PR?
   <!-- Please delete all components from this list that are NOT affected by this
   Pull Request. Also, feel free to add the name of a tool or script that is
   affected but not on the list.
   
   Additionally, if this Pull Request does NOT affect documentation, please
   explain why documentation is not required. -->
   
   - Documentation
   - Traffic Ops
   - Traffic Vault
   
   ## What is the best way to verify this PR?
   Run the tool in both PSQL -> Riak and Riak -> PSQL mode. Verify that all the command line options work and return as intended. Also make sure the keys are actually migrated appropriately. Also make sure that this tool operates in a reasonable time frame when migrating a large amount of keys.
   Read the documentation and confirm it makes sense.
   Run the tests (for both the tool and `go-util/encrypt`) and confirm they pass.
   
   *NOTE* until #5914 is fixed, the PSQL -> Riak path will only bring half the SSL Keys as the objects are completely identical without the version change.
   <!-- Please include here ALL the steps necessary to test your Pull Request. If
   it includes tests (and most should), outline here the steps needed to run the
   tests. If not, lay out the manual testing procedure and please explain why
   tests are unnecessary for this Pull Request. -->
   
   <!-- If this PR fixes a bug, please list here all of the affected versions - to
   the best of your knowledge. It's also pretty helpful to include a commit hash
   of where 'master' is at the time this PR is opened (if it affects master),
   because what 'master' means will change over time. For example, if this PR
   fixes a bug that's present in master (at commit hash '1df853c8'), in v4.0.0,
   and in the current 4.0.1 Release candidate (e.g. RC1), then this list would
   look like:
   
   - master (1df853c8)
   - 4.0.0
   - 4.0.1 (RC1)
   
   If you don't know what other versions might have this bug, AND don't know how
   to find the commit hash of 'master', then feel free to leave this section
   blank (or, preferably, delete it entirely).
    -->
   
   
   ## The following criteria are ALL met by this PR
   <!-- Check the boxes to signify that the associated statement is true. To
   "check a box", replace the space inside of the square brackets with an 'x'.
   e.g.
   
   - [ x] <- Wrong
   - [x ] <- Wrong
   - [] <- Wrong
   - [*] <- Wrong
   - [x] <- Correct!
   
   -->
   
   - [x] This PR includes tests OR I have explained why tests are unnecessary
   - [x] This PR includes documentation OR I have explained why documentation is unnecessary
   - [x] This PR includes an update to CHANGELOG.md OR such an update is not necessary
   - [x] This PR includes any and all required license headers
   - [x] This PR **DOES NOT FIX A SERIOUS SECURITY VULNERABILITY** (see [the Apache Software Foundation's security guidelines](https://www.apache.org/security/) for details)
   
   
   ## Additional Information
   If testing PSQL -> Riak, I suggest using CiaB as both are already setup. The AES key can be got by running `docker exec -it cdn-in-a-box_trafficops_1 /bin/cat /opt/traffic_ops/app/conf/aes.key`
   Riak -> PSQL, is a bit more work as it requires standing up your own psql db with the `app/db/admin` tool.
   <!-- If you would like to include any additional information on the PR for
   potential reviewers please put it here.
   
   Some examples of this would be:
   
   - Before and after screenshots/gifs of the Traffic Portal if it is affected
   - Links to other dependent Pull Requests
   - References to relevant context (e.g. new/updates to dependent libraries,
   mailing list records, blueprints)
   
   Feel free to leave this section blank (or, preferably, delete it entirely).
   -->
   
   <!--
   Licensed to the Apache Software Foundation (ASF) under one
   or more contributor license agreements.  See the NOTICE file
   distributed with this work for additional information
   regarding copyright ownership.  The ASF licenses this file
   to you under the Apache License, Version 2.0 (the
   "License"); you may not use this file except in compliance
   with the License.  You may obtain a copy of the License at
   
       http://www.apache.org/licenses/LICENSE-2.0
   
   Unless required by applicable law or agreed to in writing,
   software distributed under the License is distributed on an
   "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
   KIND, either express or implied.  See the License for the
   specific language governing permissions and limitations
   under the License.
   -->
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [trafficcontrol] rawlinp commented on a change in pull request #5924: Add tool to migrate data between TV backends

Posted by GitBox <gi...@apache.org>.
rawlinp commented on a change in pull request #5924:
URL: https://github.com/apache/trafficcontrol/pull/5924#discussion_r652029857



##########
File path: tools/traffic_vault_migrate/postgres.go
##########
@@ -0,0 +1,743 @@
+package main
+
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+import (
+	"database/sql"
+	"encoding/base64"
+	"encoding/json"
+	"errors"
+	"fmt"
+	"log"
+	"strconv"
+	"strings"
+
+	"github.com/apache/trafficcontrol/lib/go-tc"
+	util "github.com/apache/trafficcontrol/lib/go-util"
+
+	_ "github.com/lib/pq"
+)
+
+// PGConfig represents the configuration options available to the PG backend
+type PGConfig struct {
+	Host     string `json:"host"`
+	Port     string `json:"port"`
+	User     string `json:"user"`
+	Password string `json:"password"`
+	SSLMode  string `json:"sslmode"`
+	Database string `json:"database"`
+	Key      string `json:"aesKey"`
+	AESKey   []byte
+}
+
+// PGBackend is the Postgres implementation of TVBackend
+type PGBackend struct {
+	sslKey pgSSLKeyTable
+	dnssec pgDNSSecTable
+	uri    pgURISignKeyTable
+	url    pgURLSigKeyTable
+	cfg    PGConfig
+	db     *sql.DB
+}
+
+// String returns a high level overview of the backend and its keys
+func (pg *PGBackend) String() string {
+	data := fmt.Sprintf("PG server %v@%v:%v\n", pg.cfg.User, pg.cfg.Host, pg.cfg.Port)
+	data += fmt.Sprintf("\tSSL Keys: %v\n", len(pg.sslKey.Records))
+	data += fmt.Sprintf("\tDNSSec Keys: %v\n", len(pg.dnssec.Records))
+	data += fmt.Sprintf("\tURI Keys: %v\n", len(pg.uri.Records))
+	data += fmt.Sprintf("\tURL Keys: %v\n", len(pg.url.Records))
+	return data
+}
+
+// Name returns the name for this backend
+func (pg *PGBackend) Name() string {
+	return "PG"
+}
+
+// ReadConfig takes in a filename and will read it into the backends config
+func (pg *PGBackend) ReadConfig(s string) error {
+	err := UnmarshalConfig(s, &pg.cfg)
+	if err != nil {
+		return err
+	}
+
+	pg.cfg.AESKey, err = base64.StdEncoding.DecodeString(pg.cfg.Key)
+	if err != nil {
+		return fmt.Errorf("unable to decode PG AESKey: %w", err)
+	}
+	return nil
+}
+
+// Insert takes the current keys and inserts them into the backend DB
+func (pg *PGBackend) Insert() error {
+	if err := pg.sslKey.insertKeys(pg.db); err != nil {
+		return err
+	}
+	if err := pg.dnssec.insertKeys(pg.db); err != nil {
+		return err
+	}
+	if err := pg.url.insertKeys(pg.db); err != nil {
+		return err
+	}
+	if err := pg.uri.insertKeys(pg.db); err != nil {
+		return err
+	}
+	return nil
+}
+
+// Start initiates the connection to the backend DB
+func (pg *PGBackend) Start() error {
+	sqlStr := fmt.Sprintf("postgres://%s:%s@%s:%s/%s?sslmode=%s", pg.cfg.User, pg.cfg.Password, pg.cfg.Host, pg.cfg.Port, pg.cfg.Database, pg.cfg.SSLMode)
+	db, err := sql.Open("postgres", sqlStr)
+	if err != nil {
+		return fmt.Errorf("unable to start PG client: %w", err)
+	}
+
+	pg.db = db
+	pg.sslKey = pgSSLKeyTable{}
+	pg.dnssec = pgDNSSecTable{}
+	pg.url = pgURLSigKeyTable{}
+	pg.uri = pgURISignKeyTable{}
+
+	return nil
+}
+
+// ValidateKey validates that the keys are valid (in most cases, certain fields are not null)
+func (pg *PGBackend) ValidateKey() []string {
+	var errors []string
+	if errs := pg.sslKey.validate(); errs != nil {
+		errors = append(errors, errs...)
+	}
+	if errs := pg.dnssec.validate(); errs != nil {
+		errors = append(errors, errs...)
+	}
+	if errs := pg.uri.validate(); errs != nil {
+		errors = append(errors, errs...)
+	}
+	if errs := pg.url.validate(); errs != nil {
+		errors = append(errors, errs...)
+	}
+	return errors
+}
+
+// Close terminates the connection to the backend DB
+func (pg *PGBackend) Close() error {
+	return pg.db.Close()
+}
+
+// Ping checks the connection to the backend DB
+func (pg *PGBackend) Ping() error {
+	return pg.db.Ping()
+}
+
+// Fetch gets all of the keys from the backend DB
+func (pg *PGBackend) Fetch() error {
+	if err := pg.sslKey.gatherKeys(pg.db); err != nil {
+		return err
+	}
+
+	if err := pg.dnssec.gatherKeys(pg.db); err != nil {
+		return err
+	}
+
+	if err := pg.url.gatherKeys(pg.db); err != nil {
+		return err
+	}
+
+	if err := pg.uri.gatherKeys(pg.db); err != nil {
+		return err
+	}
+
+	return nil
+}
+
+// GetSSLKeys converts the backends internal key representation into the common representation (SSLKey)
+func (pg *PGBackend) GetSSLKeys() ([]SSLKey, error) {
+	if err := pg.sslKey.decrypt(pg.cfg.AESKey); err != nil {
+		return nil, err
+	}
+	return pg.sslKey.toGeneric(), nil
+}
+
+// SetSSLKeys takes in keys and converts & encrypts the data into the backends internal format
+func (pg *PGBackend) SetSSLKeys(keys []SSLKey) error {
+	pg.sslKey.fromGeneric(keys)
+	return pg.sslKey.encrypt(pg.cfg.AESKey)
+}
+
+// GetDNSSecKeys converts the backends internal key representation into the common representation (DNSSecKey)
+func (pg *PGBackend) GetDNSSecKeys() ([]DNSSecKey, error) {
+	if err := pg.dnssec.decrypt(pg.cfg.AESKey); err != nil {
+		return nil, err
+	}
+	return pg.dnssec.toGeneric(), nil
+}
+
+// SetDNSSecKeys takes in keys and converts & encrypts the data into the backends internal format
+func (pg *PGBackend) SetDNSSecKeys(keys []DNSSecKey) error {
+	pg.dnssec.fromGeneric(keys)
+	return pg.dnssec.encrypt(pg.cfg.AESKey)
+}
+
+// GetURISignKeys converts the pg internal key representation into the common representation (URISignKey)
+func (pg *PGBackend) GetURISignKeys() ([]URISignKey, error) {
+	if err := pg.uri.decrypt(pg.cfg.AESKey); err != nil {
+		return nil, err
+	}
+	return pg.uri.toGeneric(), nil
+}
+
+// SetURISignKeys takes in keys and converts & encrypts the data into the backends internal format
+func (pg *PGBackend) SetURISignKeys(keys []URISignKey) error {
+	pg.uri.fromGeneric(keys)
+	return pg.uri.encrypt(pg.cfg.AESKey)
+}
+
+// GetURLSigKeys converts the backends internal key representation into the common representation (URLSigKey)
+func (pg *PGBackend) GetURLSigKeys() ([]URLSigKey, error) {
+	if err := pg.url.decrypt(pg.cfg.AESKey); err != nil {
+		return nil, err
+	}
+	return pg.url.toGeneric(), nil
+}
+
+// SetURLSigKeys takes in keys and converts & encrypts the data into the backends internal format
+func (pg *PGBackend) SetURLSigKeys(keys []URLSigKey) error {
+	pg.url.fromGeneric(keys)
+	return pg.url.encrypt(pg.cfg.AESKey)
+}
+
+type pgCommonRecord struct {
+	DataEncrypted []byte
+}
+
+type pgDNSSecRecord struct {
+	Key tc.DNSSECKeysTrafficVault
+	CDN string
+	pgCommonRecord
+}
+type pgDNSSecTable struct {
+	Records []pgDNSSecRecord
+}
+
+func (tbl *pgDNSSecTable) gatherKeys(db *sql.DB) error {
+	sz, err := getSize(db, "dnssec")
+	if err != nil {
+		log.Println("PGDNSSec gatherKeys: unable to determine size of dnssec table")
+	}
+	tbl.Records = make([]pgDNSSecRecord, sz)
+
+	rows, err := db.Query("SELECT cdn, data from dnssec")
+	if err != nil {
+		return fmt.Errorf("PGDNSSec gatherKeys: unable to query: %w", err)
+	}
+	defer rows.Close()
+	i := 0
+	for rows.Next() {
+		if i > len(tbl.Records)-1 {
+			return fmt.Errorf("PGDNSSec gatherKeys got more results than expected %v", len(tbl.Records))
+		}
+		err := rows.Scan(&tbl.Records[i].CDN, &tbl.Records[i].DataEncrypted)
+		if err != nil {
+			return fmt.Errorf("PGDNSSec gatherKeys unable to scan row: %w", err)
+		}
+		i += 1
+	}
+	return nil
+}
+func (tbl *pgDNSSecTable) decrypt(aesKey []byte) error {
+	for i, _ := range tbl.Records {
+		err := decryptInto(aesKey, tbl.Records[i].DataEncrypted, &tbl.Records[i].Key)
+		if err != nil {
+			return fmt.Errorf("unable to decrypt into keys: %w", err)
+		}
+	}
+	return nil
+}
+func (tbl *pgDNSSecTable) encrypt(aesKey []byte) error {
+	for i, dns := range tbl.Records {
+		data, err := json.Marshal(&dns.Key)
+		if err != nil {
+			return fmt.Errorf("encrypt issue marshalling keys: %w", err)
+		}
+		dat, err := encrypt(data, aesKey)
+		if err != nil {
+			return fmt.Errorf("encrypt error: %w", err)
+		}
+		tbl.Records[i].DataEncrypted = dat
+	}
+	return nil
+}
+func (tbl *pgDNSSecTable) toGeneric() []DNSSecKey {
+	keys := make([]DNSSecKey, len(tbl.Records))
+
+	for i, record := range tbl.Records {
+		keys[i] = DNSSecKey{
+			CDN:                    record.CDN,
+			DNSSECKeysTrafficVault: record.Key,
+		}
+	}
+
+	return keys
+}
+func (tbl *pgDNSSecTable) fromGeneric(keys []DNSSecKey) {
+	tbl.Records = make([]pgDNSSecRecord, len(keys))
+
+	for i, key := range keys {
+		tbl.Records[i] = pgDNSSecRecord{
+			Key: key.DNSSECKeysTrafficVault,
+			CDN: key.CDN,
+			pgCommonRecord: pgCommonRecord{
+				DataEncrypted: nil,
+			},
+		}
+	}
+}
+func (tbl *pgDNSSecTable) validate() []string {
+	for i, record := range tbl.Records {
+		if record.DataEncrypted == nil && len(record.Key) > 0 {
+			return []string{fmt.Sprintf("DNSSEC Key %v: DataEncrypted is blank!", i)}
+		}
+	}
+	return nil
+}
+func (tbl *pgDNSSecTable) insertKeys(db *sql.DB) error {
+	queryBase := "INSERT INTO dnssec (cdn, data) VALUES "
+	stride := 2
+	queryArgs := make([]interface{}, len(tbl.Records)*stride)
+	for i, record := range tbl.Records {
+		j := i * stride
+		queryArgs[j] = record.CDN
+		queryArgs[j+1] = record.DataEncrypted
+	}
+	return insertIntoTable(db, queryBase, stride, queryArgs)
+}
+
+type pgSSLKeyRecord struct {
+	Keys tc.DeliveryServiceSSLKeys
+	pgCommonRecord
+
+	// These records are stored on the table but are duplicated
+	DeliveryService string
+	CDN             string
+}
+type pgSSLKeyTable struct {
+	Records []pgSSLKeyRecord
+}
+
+func (tbl *pgSSLKeyTable) insertKeys(db *sql.DB) error {
+	queryBase := "INSERT INTO sslkey (deliveryservice, data, cdn, version) VALUES "
+	duplicateKeys := 2
+	stride := 4
+	queryArgs := make([]interface{}, len(tbl.Records)*stride*duplicateKeys)
+	for i, record := range tbl.Records {
+		j := i * duplicateKeys * stride
+
+		queryArgs[j] = record.DeliveryService
+		queryArgs[j+1] = record.DataEncrypted
+		queryArgs[j+2] = record.CDN
+		queryArgs[j+3] = record.Keys.Version.String()
+
+		queryArgs[j+4] = record.DeliveryService
+		queryArgs[j+5] = record.DataEncrypted
+		queryArgs[j+6] = record.CDN
+		queryArgs[j+7] = "latest"
+	}
+	return insertIntoTable(db, queryBase, 4, queryArgs)
+}
+func (tbl *pgSSLKeyTable) gatherKeys(db *sql.DB) error {
+	sz, err := getSize(db, "sslkey WHERE version='latest'")
+	if err != nil {
+		return fmt.Errorf("PGSSLKey gatherKeys unable to determine size of sslkey table: %w", err)
+	}
+	tbl.Records = make([]pgSSLKeyRecord, sz)
+
+	rows, err := db.Query("SELECT data, deliveryservice, cdn from sslkey WHERE version = 'latest'")
+	if err != nil {
+		return fmt.Errorf("PGSSLKey gatherKeys unable to query: %w", err)
+	}
+	defer rows.Close()
+	i := 0
+	for rows.Next() {
+		if i > len(tbl.Records)-1 {
+			return errors.New("PGSSLKey gatherKeys: got more results than expected")
+		}
+		err := rows.Scan(&tbl.Records[i].DataEncrypted, &tbl.Records[i].DeliveryService, &tbl.Records[i].CDN)
+		if err != nil {
+			return fmt.Errorf("PGSSLKey gatherKeys unable to scan row: %w", err)
+		}
+		i += 1
+	}
+	return nil
+}
+func (tbl *pgSSLKeyTable) decrypt(aesKey []byte) error {
+	for i, dns := range tbl.Records {
+		err := decryptInto(aesKey, dns.DataEncrypted, &tbl.Records[i].Keys)
+		if err != nil {
+			return fmt.Errorf("unable to decrypt into keys: %w", err)
+		}
+	}
+	return nil
+}
+func (tbl *pgSSLKeyTable) encrypt(aesKey []byte) error {
+	for i, dns := range tbl.Records {
+		data, err := json.Marshal(dns.Keys)
+		if err != nil {
+			return fmt.Errorf("encrypt issue marshalling keys: %w", err)
+		}
+		dat, err := encrypt(data, aesKey)
+		if err != nil {
+			return fmt.Errorf("encrypt error: %w", err)
+		}
+		tbl.Records[i].DataEncrypted = dat
+	}
+	return nil
+}
+func (tbl *pgSSLKeyTable) toGeneric() []SSLKey {
+	keys := make([]SSLKey, len(tbl.Records))
+
+	for i, record := range tbl.Records {
+		keys[i] = SSLKey{
+			DeliveryServiceSSLKeys: record.Keys,
+		}
+	}
+	return keys
+}
+func (tbl *pgSSLKeyTable) fromGeneric(keys []SSLKey) {
+	tbl.Records = make([]pgSSLKeyRecord, len(keys))
+
+	for i, key := range keys {
+		tbl.Records[i] = pgSSLKeyRecord{
+			Keys: key.DeliveryServiceSSLKeys,
+			pgCommonRecord: pgCommonRecord{
+				DataEncrypted: nil,
+			},
+			DeliveryService: key.DeliveryService,
+			CDN:             key.CDN,
+		}
+	}
+}
+func (tbl *pgSSLKeyTable) validate() []string {
+	defaultKey := tc.DeliveryServiceSSLKeys{}
+	var errors []string
+	fmtStr := "SSL Key %v: %v"
+	for i, record := range tbl.Records {
+		if record.Keys == defaultKey {
+			errors = append(errors, fmt.Sprintf(fmtStr, i, "DS SSL Keys are default!"))
+		} else if record.Keys.Key == "" {
+			errors = append(errors, fmt.Sprintf(fmtStr, i, "Key is blank!"))
+		} else if record.Keys.CDN == "" {
+			errors = append(errors, fmt.Sprintf(fmtStr, i, "CDN is blank!"))
+		} else if record.Keys.DeliveryService == "" {
+			errors = append(errors, fmt.Sprintf(fmtStr, i, "DS is blank!"))
+		} else if record.DataEncrypted == nil {
+			errors = append(errors, fmt.Sprintf(fmtStr, i, "DataEncrypted is blank!"))
+		} else if record.Keys.Version.String() == "" {
+			errors = append(errors, fmt.Sprintf(fmtStr, i, "Version is blank!"))
+		}
+	}
+	return errors
+}
+
+type pgURLSigKeyRecord struct {
+	Keys            tc.URLSigKeys
+	DeliveryService string
+	pgCommonRecord
+}
+type pgURLSigKeyTable struct {
+	Records []pgURLSigKeyRecord
+}
+
+func (tbl *pgURLSigKeyTable) insertKeys(db *sql.DB) error {
+	queryBase := "INSERT INTO url_sig_key (deliveryservice, data) VALUES "
+	stride := 2
+	queryArgs := make([]interface{}, len(tbl.Records)*stride)
+	for i, record := range tbl.Records {
+		j := i * stride
+		queryArgs[j] = record.DeliveryService
+		queryArgs[j+1] = record.DataEncrypted
+	}
+	return insertIntoTable(db, queryBase, stride, queryArgs)
+}
+func (tbl *pgURLSigKeyTable) gatherKeys(db *sql.DB) error {
+	sz, err := getSize(db, "url_sig_key")
+	if err != nil {
+		log.Println("PGURLSigKey gatherKeys: unable to determine url_sig_key table size")
+	}
+	tbl.Records = make([]pgURLSigKeyRecord, sz)
+
+	rows, err := db.Query("SELECT deliveryservice, data from url_sig_key")
+	if err != nil {
+		return fmt.Errorf("PGURLSigKey gatherKeys error creating query: %w", err)
+	}
+	defer rows.Close()
+	i := 0
+	for rows.Next() {
+		if i > len(tbl.Records)-1 {
+			return fmt.Errorf("PGURLSigKey gatherKeys: got more results than expected %v", len(tbl.Records))
+		}
+		err := rows.Scan(&tbl.Records[i].DeliveryService, &tbl.Records[i].DataEncrypted)
+		if err != nil {
+			return fmt.Errorf("PGURLSigKey gatherKeys: unable to scan row: %w", err)
+		}
+		i += 1
+	}
+	return nil
+}
+func (tbl *pgURLSigKeyTable) decrypt(aesKey []byte) error {
+	for i, sig := range tbl.Records {
+		err := decryptInto(aesKey, sig.DataEncrypted, &tbl.Records[i].Keys)
+		if err != nil {
+			return fmt.Errorf("unable to decrypt into keys: %w", err)
+		}
+	}
+	return nil
+}
+func (tbl *pgURLSigKeyTable) encrypt(aesKey []byte) error {
+	for i, sig := range tbl.Records {
+		data, err := json.Marshal(&sig.Keys)
+		if err != nil {
+			return fmt.Errorf("encrypt issue marshalling keys: %w", err)
+		}
+
+		dat, err := encrypt(data, aesKey)
+		if err != nil {
+			return fmt.Errorf("encrypt error: %w", err)
+		}
+		tbl.Records[i].DataEncrypted = dat
+	}
+	return nil
+}
+func (tbl *pgURLSigKeyTable) toGeneric() []URLSigKey {
+	keys := make([]URLSigKey, len(tbl.Records))
+
+	for i, record := range tbl.Records {
+		keys[i] = URLSigKey{
+			DeliveryService: record.DeliveryService,
+			URLSigKeys:      record.Keys,
+		}
+	}
+	return keys
+}
+func (tbl *pgURLSigKeyTable) fromGeneric(keys []URLSigKey) {
+	tbl.Records = make([]pgURLSigKeyRecord, len(keys))
+
+	for i, key := range keys {
+		tbl.Records[i] = pgURLSigKeyRecord{
+			Keys:            key.URLSigKeys,
+			DeliveryService: key.DeliveryService,
+			pgCommonRecord: pgCommonRecord{
+				DataEncrypted: nil,
+			},
+		}
+	}
+}
+func (tbl *pgURLSigKeyTable) validate() []string {
+	for i, record := range tbl.Records {
+		if record.DataEncrypted == nil && len(record.Keys) > 0 {
+			return []string{fmt.Sprintf("URl Sig Key %v: DataEncrypted is blank!", i)}
+		}
+	}
+	return nil
+}
+
+type pgURISignKeyRecord struct {
+	Keys            map[string]tc.URISignerKeyset
+	DeliveryService string
+	pgCommonRecord
+}
+type pgURISignKeyTable struct {
+	Records []pgURISignKeyRecord
+}
+
+func (tbl *pgURISignKeyTable) insertKeys(db *sql.DB) error {
+	queryBase := "INSERT INTO uri_signing_key (deliveryservice, data) VALUES "

Review comment:
       Well, we can still change the schema if you'd like -- since it's unreleased we don't necessarily need a migration and can just update the create_schema.sql file.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [trafficcontrol] shamrickus commented on a change in pull request #5924: Add tool to migrate data between TV backends

Posted by GitBox <gi...@apache.org>.
shamrickus commented on a change in pull request #5924:
URL: https://github.com/apache/trafficcontrol/pull/5924#discussion_r648791718



##########
File path: docs/source/tools/traffic_vault_migrate.rst
##########
@@ -0,0 +1,107 @@
+..
+..
+.. Licensed under the Apache License, Version 2.0 (the "License");
+.. you may not use this file except in compliance with the License.
+.. You may obtain a copy of the License at
+..
+..     http://www.apache.org/licenses/LICENSE-2.0
+..
+.. Unless required by applicable law or agreed to in writing, software
+.. distributed under the License is distributed on an "AS IS" BASIS,
+.. WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+.. See the License for the specific language governing permissions and
+.. limitations under the License.
+..
+
+.. _traffic_vault_migrate:
+
+#########################
+Traffic Vault Migrate
+#########################
+The ``traffic_vault_migrate`` tool - located at :file:`tools/traffic_vault_migrate/traffic_vault_migrate.go` in the `Apache Traffic Control repository <https://github.com/apache/trafficcontrol>`_ -
+is used to transfer TV keys between database servers. It interfaces directly with each backend so Traffic Ops/Vault being available is not a requirement.
+The tool assumes that the schema for each backend is already setup as according to the :ref:`admin setup <traffic_vault_admin>`.
+
+.. program:: traffic_vault_migrate
+
+Usage
+===========
+``traffic_vault_migrate -from_cfg CFG -to_cfg CFG -from_type TYP -to_type TYP [-confirm] [-compare] [-dry] [-dump]``
+
+.. option:: -compare
+
+		Compare 'to' and 'from' backend keys. Will fetch keys from the dbs of both 'to' and 'from', sorts them by cdn/ds/version and does a deep comparison.
+
+.. option:: -confirm
+
+		Requires confirmation before inserting records (default true)
+
+.. option:: -dry
+
+		Do not perform writes. Will do a basic output of the keys on the 'from' backend.
+
+.. option:: -dump
+
+		Write keys (from 'from' server) to disk in the folder 'dump' with the unix permissions 0640.
+
+		.. warning:: This can write potentially sensitive information to disk, use with care.
+
+.. option:: -from_cfg
+
+		From server config file (default "riak.json")
+
+.. option:: -from_type
+
+		From server types (Riak|PG) (default "Riak")
+
+.. option:: -to_cfg
+
+		To server config file (default "pg.json")
+
+.. option:: -to_type
+
+		From server types (Riak|PG) (default "PG")

Review comment:
       Changed




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [trafficcontrol] shamrickus commented on a change in pull request #5924: Add tool to migrate data between TV backends

Posted by GitBox <gi...@apache.org>.
shamrickus commented on a change in pull request #5924:
URL: https://github.com/apache/trafficcontrol/pull/5924#discussion_r651160143



##########
File path: tools/traffic_vault_migrate/postgres.go
##########
@@ -0,0 +1,716 @@
+package main
+
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+import (
+	"database/sql"
+	"encoding/base64"
+	"encoding/json"
+	"errors"
+	"fmt"
+	"github.com/apache/trafficcontrol/lib/go-tc"
+	util "github.com/apache/trafficcontrol/lib/go-util"
+	_ "github.com/lib/pq"
+	"log"
+	"reflect"
+	"strconv"
+	"strings"
+)
+
+type PGConfig struct {
+	Host     string `json:"host"`
+	Port     string `json:"port"`
+	User     string `json:"user"`
+	Password string `json:"password"`
+	SSLMode  string `json:"sslmode"`
+	Database string `json:"database"`
+	Key      string `json:"aesKey"`
+	AESKey   []byte
+}
+type PGBackend struct {
+	sslKey PGSSLKeyTable
+	dnssec PGDNSSecTable
+	uri    PGURISignKeyTable
+	url    PGURLSigKeyTable
+	cfg    PGConfig
+	db     *sql.DB
+}
+
+func (pg *PGBackend) String() string {
+	data := fmt.Sprintf("PG server %v@%v:%v\n", pg.cfg.User, pg.cfg.Host, pg.cfg.Port)
+	data += fmt.Sprintf("\tSSL Keys: %v\n", len(pg.sslKey.Records))
+	data += fmt.Sprintf("\tDNSSec Keys: %v\n", len(pg.dnssec.Records))
+	data += fmt.Sprintf("\tURI Keys: %v\n", len(pg.uri.Records))
+	data += fmt.Sprintf("\tURL Keys: %v\n", len(pg.url.Records))
+	return data
+}
+func (pg *PGBackend) Name() string {
+	return "PG"
+}
+func (pg *PGBackend) ReadConfig(s string) error {
+	genericCfg, err := UnmarshalConfig(s, reflect.TypeOf(pg.cfg))
+	if err != nil {
+		return err
+	}
+
+	pg.cfg = *genericCfg.Interface().(*PGConfig)
+
+	pg.cfg.AESKey, err = base64.StdEncoding.DecodeString(pg.cfg.Key)
+	if err != nil {
+		return fmt.Errorf("unable to decode PG AESKey: %v", err)
+	}
+	return nil
+}
+func (pg *PGBackend) Insert() error {
+	if err := pg.sslKey.insertKeys(pg.db); err != nil {
+		return err
+	}
+	if err := pg.dnssec.insertKeys(pg.db); err != nil {
+		return err
+	}
+	if err := pg.url.insertKeys(pg.db); err != nil {
+		return err
+	}
+	if err := pg.uri.insertKeys(pg.db); err != nil {
+		return err
+	}
+	return nil
+}
+func (pg *PGBackend) Start() error {
+	sqlStr := fmt.Sprintf("postgres://%s:%s@%s:%s/%s?sslmode=%s", pg.cfg.User, pg.cfg.Password, pg.cfg.Host, pg.cfg.Port, pg.cfg.Database, pg.cfg.SSLMode)
+	db, err := sql.Open("postgres", sqlStr)
+	if err != nil {
+		return fmt.Errorf("Unable to start PG client: %v", err)
+	}
+
+	pg.db = db
+	pg.sslKey = PGSSLKeyTable{}
+	pg.dnssec = PGDNSSecTable{}
+	pg.url = PGURLSigKeyTable{}
+	pg.uri = PGURISignKeyTable{}
+
+	return nil
+}
+func (pg *PGBackend) ValidateKey() []string {
+	var errors []string
+	if errs := pg.sslKey.validate(); errs != nil {
+		errors = append(errors, errs...)
+	}
+	if errs := pg.dnssec.validate(); errs != nil {
+		errors = append(errors, errs...)
+	}
+	if errs := pg.uri.validate(); errs != nil {
+		errors = append(errors, errs...)
+	}
+	if errs := pg.url.validate(); errs != nil {
+		errors = append(errors, errs...)
+	}
+	return errors
+}
+func (pg *PGBackend) Stop() error {
+	return pg.db.Close()
+}
+func (pg *PGBackend) Ping() error {
+	return pg.db.Ping()
+}
+func (pg *PGBackend) Fetch() error {
+	if err := pg.sslKey.gatherKeys(pg.db); err != nil {
+		return err
+	}
+
+	if err := pg.dnssec.gatherKeys(pg.db); err != nil {
+		return err
+	}
+
+	if err := pg.url.gatherKeys(pg.db); err != nil {
+		return err
+	}
+
+	if err := pg.uri.gatherKeys(pg.db); err != nil {
+		return err
+	}
+
+	return nil
+}
+func (pg *PGBackend) GetSSLKeys() ([]SSLKey, error) {
+	if err := pg.sslKey.decrypt(pg.cfg.AESKey); err != nil {
+		return nil, err
+	}
+	return pg.sslKey.toGeneric(), nil
+}
+func (pg *PGBackend) SetSSLKeys(keys []SSLKey) error {
+	pg.sslKey.fromGeneric(keys)
+	return pg.sslKey.encrypt(pg.cfg.AESKey)
+}
+func (pg *PGBackend) GetDNSSecKeys() ([]DNSSecKey, error) {
+	if err := pg.dnssec.decrypt(pg.cfg.AESKey); err != nil {
+		return nil, err
+	}
+	return pg.dnssec.toGeneric(), nil
+}
+func (pg *PGBackend) SetDNSSecKeys(keys []DNSSecKey) error {
+	pg.dnssec.fromGeneric(keys)
+	return pg.dnssec.encrypt(pg.cfg.AESKey)
+}
+func (pg *PGBackend) GetURISignKeys() ([]URISignKey, error) {
+	if err := pg.uri.decrypt(pg.cfg.AESKey); err != nil {
+		return nil, err
+	}
+	return pg.uri.toGeneric(), nil
+}
+func (pg *PGBackend) SetURISignKeys(keys []URISignKey) error {
+	pg.uri.fromGeneric(keys)
+	return pg.uri.encrypt(pg.cfg.AESKey)
+}
+func (pg *PGBackend) GetURLSigKeys() ([]URLSigKey, error) {
+	if err := pg.url.decrypt(pg.cfg.AESKey); err != nil {
+		return nil, err
+	}
+	return pg.url.toGeneric(), nil
+}
+func (pg *PGBackend) SetURLSigKeys(keys []URLSigKey) error {
+	pg.url.fromGeneric(keys)
+	return pg.url.encrypt(pg.cfg.AESKey)
+}
+
+type PGCommonRecord struct {
+	DataEncrypted []byte
+	CommonRecord
+}
+
+type PGDNSSecRecord struct {
+	Key tc.DNSSECKeysTrafficVault
+	CDN string
+	PGCommonRecord
+}
+type PGDNSSecTable struct {
+	Records []PGDNSSecRecord
+}
+
+func (tbl *PGDNSSecTable) gatherKeys(db *sql.DB) error {
+	sz, err := getSize(db, "dnssec")
+	if err != nil {
+		log.Println("PGDNSSec gatherKeys: unable to determine size of dnssec table")
+	}
+	tbl.Records = make([]PGDNSSecRecord, sz)
+
+	rows, err := db.Query("SELECT cdn, data from dnssec")
+	if err != nil {
+		return fmt.Errorf("PGDNSSec gatherKeys: unable to query: %v", err)
+	}
+	defer rows.Close()

Review comment:
       Switched to `go-log`




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [trafficcontrol] shamrickus commented on a change in pull request #5924: Add tool to migrate data between TV backends

Posted by GitBox <gi...@apache.org>.
shamrickus commented on a change in pull request #5924:
URL: https://github.com/apache/trafficcontrol/pull/5924#discussion_r651160443



##########
File path: tools/traffic_vault_migrate/traffic_vault_migrate.go
##########
@@ -423,6 +422,17 @@ type URLSigKey struct {
 	tc.URLSigKeys
 }
 
+func writeKeys(filename string, data interface{}) error {
+	bytes, err := json.Marshal(&data)
+	if err != nil {
+		return err
+	}
+	if err = ioutil.WriteFile(filename, bytes, 0640); err != nil {

Review comment:
       Removed all group perms




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [trafficcontrol] rawlinp commented on a change in pull request #5924: Add tool to migrate data between TV backends

Posted by GitBox <gi...@apache.org>.
rawlinp commented on a change in pull request #5924:
URL: https://github.com/apache/trafficcontrol/pull/5924#discussion_r649267525



##########
File path: tools/traffic_vault_migrate/riak.go
##########
@@ -166,7 +166,7 @@ func (rb *RiakBackend) SetURLSigKeys(keys []URLSigKey) error {
 // Start initiates the connection to the backend DB
 func (rb *RiakBackend) Start() error {
 	tlsConfig := &tls.Config{
-		InsecureSkipVerify: !rb.cfg.VerifyTLS,
+		InsecureSkipVerify: !rb.cfg.Insecure,

Review comment:
       I think this no longer needs to be negated

##########
File path: tools/traffic_vault_migrate/traffic_vault_migrate.go
##########
@@ -423,6 +422,17 @@ type URLSigKey struct {
 	tc.URLSigKeys
 }
 
+func writeKeys(filename string, data interface{}) error {
+	bytes, err := json.Marshal(&data)
+	if err != nil {
+		return err
+	}
+	if err = ioutil.WriteFile(filename, bytes, 0640); err != nil {

Review comment:
       Does the group need read permissions?

##########
File path: tools/traffic_vault_migrate/postgres.go
##########
@@ -0,0 +1,743 @@
+package main
+
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+import (
+	"database/sql"
+	"encoding/base64"
+	"encoding/json"
+	"errors"
+	"fmt"
+	"log"
+	"strconv"
+	"strings"
+
+	"github.com/apache/trafficcontrol/lib/go-tc"
+	util "github.com/apache/trafficcontrol/lib/go-util"
+
+	_ "github.com/lib/pq"
+)
+
+// PGConfig represents the configuration options available to the PG backend
+type PGConfig struct {
+	Host     string `json:"host"`
+	Port     string `json:"port"`
+	User     string `json:"user"`
+	Password string `json:"password"`
+	SSLMode  string `json:"sslmode"`
+	Database string `json:"database"`
+	Key      string `json:"aesKey"`
+	AESKey   []byte
+}
+
+// PGBackend is the Postgres implementation of TVBackend
+type PGBackend struct {
+	sslKey pgSSLKeyTable
+	dnssec pgDNSSecTable
+	uri    pgURISignKeyTable
+	url    pgURLSigKeyTable
+	cfg    PGConfig
+	db     *sql.DB
+}
+
+// String returns a high level overview of the backend and its keys
+func (pg *PGBackend) String() string {
+	data := fmt.Sprintf("PG server %v@%v:%v\n", pg.cfg.User, pg.cfg.Host, pg.cfg.Port)
+	data += fmt.Sprintf("\tSSL Keys: %v\n", len(pg.sslKey.Records))
+	data += fmt.Sprintf("\tDNSSec Keys: %v\n", len(pg.dnssec.Records))
+	data += fmt.Sprintf("\tURI Keys: %v\n", len(pg.uri.Records))
+	data += fmt.Sprintf("\tURL Keys: %v\n", len(pg.url.Records))
+	return data
+}
+
+// Name returns the name for this backend
+func (pg *PGBackend) Name() string {
+	return "PG"
+}
+
+// ReadConfig takes in a filename and will read it into the backends config
+func (pg *PGBackend) ReadConfig(s string) error {
+	err := UnmarshalConfig(s, &pg.cfg)
+	if err != nil {
+		return err
+	}
+
+	pg.cfg.AESKey, err = base64.StdEncoding.DecodeString(pg.cfg.Key)
+	if err != nil {
+		return fmt.Errorf("unable to decode PG AESKey: %w", err)
+	}
+	return nil
+}
+
+// Insert takes the current keys and inserts them into the backend DB
+func (pg *PGBackend) Insert() error {
+	if err := pg.sslKey.insertKeys(pg.db); err != nil {
+		return err
+	}
+	if err := pg.dnssec.insertKeys(pg.db); err != nil {
+		return err
+	}
+	if err := pg.url.insertKeys(pg.db); err != nil {
+		return err
+	}
+	if err := pg.uri.insertKeys(pg.db); err != nil {
+		return err
+	}
+	return nil
+}
+
+// Start initiates the connection to the backend DB
+func (pg *PGBackend) Start() error {
+	sqlStr := fmt.Sprintf("postgres://%s:%s@%s:%s/%s?sslmode=%s", pg.cfg.User, pg.cfg.Password, pg.cfg.Host, pg.cfg.Port, pg.cfg.Database, pg.cfg.SSLMode)
+	db, err := sql.Open("postgres", sqlStr)
+	if err != nil {
+		return fmt.Errorf("unable to start PG client: %w", err)
+	}
+
+	pg.db = db
+	pg.sslKey = pgSSLKeyTable{}
+	pg.dnssec = pgDNSSecTable{}
+	pg.url = pgURLSigKeyTable{}
+	pg.uri = pgURISignKeyTable{}
+
+	return nil
+}
+
+// ValidateKey validates that the keys are valid (in most cases, certain fields are not null)
+func (pg *PGBackend) ValidateKey() []string {
+	var errors []string
+	if errs := pg.sslKey.validate(); errs != nil {
+		errors = append(errors, errs...)
+	}
+	if errs := pg.dnssec.validate(); errs != nil {
+		errors = append(errors, errs...)
+	}
+	if errs := pg.uri.validate(); errs != nil {
+		errors = append(errors, errs...)
+	}
+	if errs := pg.url.validate(); errs != nil {
+		errors = append(errors, errs...)
+	}
+	return errors
+}
+
+// Close terminates the connection to the backend DB
+func (pg *PGBackend) Close() error {
+	return pg.db.Close()
+}
+
+// Ping checks the connection to the backend DB
+func (pg *PGBackend) Ping() error {
+	return pg.db.Ping()
+}
+
+// Fetch gets all of the keys from the backend DB
+func (pg *PGBackend) Fetch() error {
+	if err := pg.sslKey.gatherKeys(pg.db); err != nil {
+		return err
+	}
+
+	if err := pg.dnssec.gatherKeys(pg.db); err != nil {
+		return err
+	}
+
+	if err := pg.url.gatherKeys(pg.db); err != nil {
+		return err
+	}
+
+	if err := pg.uri.gatherKeys(pg.db); err != nil {
+		return err
+	}
+
+	return nil
+}
+
+// GetSSLKeys converts the backends internal key representation into the common representation (SSLKey)
+func (pg *PGBackend) GetSSLKeys() ([]SSLKey, error) {
+	if err := pg.sslKey.decrypt(pg.cfg.AESKey); err != nil {
+		return nil, err
+	}
+	return pg.sslKey.toGeneric(), nil
+}
+
+// SetSSLKeys takes in keys and converts & encrypts the data into the backends internal format
+func (pg *PGBackend) SetSSLKeys(keys []SSLKey) error {
+	pg.sslKey.fromGeneric(keys)
+	return pg.sslKey.encrypt(pg.cfg.AESKey)
+}
+
+// GetDNSSecKeys converts the backends internal key representation into the common representation (DNSSecKey)
+func (pg *PGBackend) GetDNSSecKeys() ([]DNSSecKey, error) {
+	if err := pg.dnssec.decrypt(pg.cfg.AESKey); err != nil {
+		return nil, err
+	}
+	return pg.dnssec.toGeneric(), nil
+}
+
+// SetDNSSecKeys takes in keys and converts & encrypts the data into the backends internal format
+func (pg *PGBackend) SetDNSSecKeys(keys []DNSSecKey) error {
+	pg.dnssec.fromGeneric(keys)
+	return pg.dnssec.encrypt(pg.cfg.AESKey)
+}
+
+// GetURISignKeys converts the pg internal key representation into the common representation (URISignKey)
+func (pg *PGBackend) GetURISignKeys() ([]URISignKey, error) {
+	if err := pg.uri.decrypt(pg.cfg.AESKey); err != nil {
+		return nil, err
+	}
+	return pg.uri.toGeneric(), nil
+}
+
+// SetURISignKeys takes in keys and converts & encrypts the data into the backends internal format
+func (pg *PGBackend) SetURISignKeys(keys []URISignKey) error {
+	pg.uri.fromGeneric(keys)
+	return pg.uri.encrypt(pg.cfg.AESKey)
+}
+
+// GetURLSigKeys converts the backends internal key representation into the common representation (URLSigKey)
+func (pg *PGBackend) GetURLSigKeys() ([]URLSigKey, error) {
+	if err := pg.url.decrypt(pg.cfg.AESKey); err != nil {
+		return nil, err
+	}
+	return pg.url.toGeneric(), nil
+}
+
+// SetURLSigKeys takes in keys and converts & encrypts the data into the backends internal format
+func (pg *PGBackend) SetURLSigKeys(keys []URLSigKey) error {
+	pg.url.fromGeneric(keys)
+	return pg.url.encrypt(pg.cfg.AESKey)
+}
+
+type pgCommonRecord struct {
+	DataEncrypted []byte
+}
+
+type pgDNSSecRecord struct {
+	Key tc.DNSSECKeysTrafficVault
+	CDN string
+	pgCommonRecord
+}
+type pgDNSSecTable struct {
+	Records []pgDNSSecRecord
+}
+
+func (tbl *pgDNSSecTable) gatherKeys(db *sql.DB) error {
+	sz, err := getSize(db, "dnssec")
+	if err != nil {
+		log.Println("PGDNSSec gatherKeys: unable to determine size of dnssec table")
+	}
+	tbl.Records = make([]pgDNSSecRecord, sz)
+
+	rows, err := db.Query("SELECT cdn, data from dnssec")
+	if err != nil {
+		return fmt.Errorf("PGDNSSec gatherKeys: unable to query: %w", err)
+	}
+	defer rows.Close()
+	i := 0
+	for rows.Next() {
+		if i > len(tbl.Records)-1 {
+			return fmt.Errorf("PGDNSSec gatherKeys got more results than expected %v", len(tbl.Records))
+		}
+		err := rows.Scan(&tbl.Records[i].CDN, &tbl.Records[i].DataEncrypted)
+		if err != nil {
+			return fmt.Errorf("PGDNSSec gatherKeys unable to scan row: %w", err)
+		}
+		i += 1
+	}
+	return nil
+}
+func (tbl *pgDNSSecTable) decrypt(aesKey []byte) error {
+	for i, _ := range tbl.Records {
+		err := decryptInto(aesKey, tbl.Records[i].DataEncrypted, &tbl.Records[i].Key)
+		if err != nil {
+			return fmt.Errorf("unable to decrypt into keys: %w", err)
+		}
+	}
+	return nil
+}
+func (tbl *pgDNSSecTable) encrypt(aesKey []byte) error {
+	for i, dns := range tbl.Records {
+		data, err := json.Marshal(&dns.Key)
+		if err != nil {
+			return fmt.Errorf("encrypt issue marshalling keys: %w", err)
+		}
+		dat, err := encrypt(data, aesKey)
+		if err != nil {
+			return fmt.Errorf("encrypt error: %w", err)
+		}
+		tbl.Records[i].DataEncrypted = dat
+	}
+	return nil
+}
+func (tbl *pgDNSSecTable) toGeneric() []DNSSecKey {
+	keys := make([]DNSSecKey, len(tbl.Records))
+
+	for i, record := range tbl.Records {
+		keys[i] = DNSSecKey{
+			CDN:                    record.CDN,
+			DNSSECKeysTrafficVault: record.Key,
+		}
+	}
+
+	return keys
+}
+func (tbl *pgDNSSecTable) fromGeneric(keys []DNSSecKey) {
+	tbl.Records = make([]pgDNSSecRecord, len(keys))
+
+	for i, key := range keys {
+		tbl.Records[i] = pgDNSSecRecord{
+			Key: key.DNSSECKeysTrafficVault,
+			CDN: key.CDN,
+			pgCommonRecord: pgCommonRecord{
+				DataEncrypted: nil,
+			},
+		}
+	}
+}
+func (tbl *pgDNSSecTable) validate() []string {
+	for i, record := range tbl.Records {
+		if record.DataEncrypted == nil && len(record.Key) > 0 {
+			return []string{fmt.Sprintf("DNSSEC Key %v: DataEncrypted is blank!", i)}
+		}
+	}
+	return nil
+}
+func (tbl *pgDNSSecTable) insertKeys(db *sql.DB) error {
+	queryBase := "INSERT INTO dnssec (cdn, data) VALUES "
+	stride := 2
+	queryArgs := make([]interface{}, len(tbl.Records)*stride)
+	for i, record := range tbl.Records {
+		j := i * stride
+		queryArgs[j] = record.CDN
+		queryArgs[j+1] = record.DataEncrypted
+	}
+	return insertIntoTable(db, queryBase, stride, queryArgs)
+}
+
+type pgSSLKeyRecord struct {
+	Keys tc.DeliveryServiceSSLKeys
+	pgCommonRecord
+
+	// These records are stored on the table but are duplicated
+	DeliveryService string
+	CDN             string
+}
+type pgSSLKeyTable struct {
+	Records []pgSSLKeyRecord
+}
+
+func (tbl *pgSSLKeyTable) insertKeys(db *sql.DB) error {
+	queryBase := "INSERT INTO sslkey (deliveryservice, data, cdn, version) VALUES "
+	duplicateKeys := 2
+	stride := 4
+	queryArgs := make([]interface{}, len(tbl.Records)*stride*duplicateKeys)
+	for i, record := range tbl.Records {
+		j := i * duplicateKeys * stride
+
+		queryArgs[j] = record.DeliveryService
+		queryArgs[j+1] = record.DataEncrypted
+		queryArgs[j+2] = record.CDN
+		queryArgs[j+3] = record.Keys.Version.String()
+
+		queryArgs[j+4] = record.DeliveryService
+		queryArgs[j+5] = record.DataEncrypted
+		queryArgs[j+6] = record.CDN
+		queryArgs[j+7] = "latest"
+	}
+	return insertIntoTable(db, queryBase, 4, queryArgs)
+}
+func (tbl *pgSSLKeyTable) gatherKeys(db *sql.DB) error {
+	sz, err := getSize(db, "sslkey WHERE version='latest'")
+	if err != nil {
+		return fmt.Errorf("PGSSLKey gatherKeys unable to determine size of sslkey table: %w", err)
+	}
+	tbl.Records = make([]pgSSLKeyRecord, sz)
+
+	rows, err := db.Query("SELECT data, deliveryservice, cdn from sslkey WHERE version = 'latest'")
+	if err != nil {
+		return fmt.Errorf("PGSSLKey gatherKeys unable to query: %w", err)
+	}
+	defer rows.Close()
+	i := 0
+	for rows.Next() {
+		if i > len(tbl.Records)-1 {
+			return errors.New("PGSSLKey gatherKeys: got more results than expected")
+		}
+		err := rows.Scan(&tbl.Records[i].DataEncrypted, &tbl.Records[i].DeliveryService, &tbl.Records[i].CDN)

Review comment:
       nit: this can be combined with the line below like `if err := function(); err != nil {`

##########
File path: tools/traffic_vault_migrate/postgres.go
##########
@@ -0,0 +1,743 @@
+package main
+
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+import (
+	"database/sql"
+	"encoding/base64"
+	"encoding/json"
+	"errors"
+	"fmt"
+	"log"
+	"strconv"
+	"strings"
+
+	"github.com/apache/trafficcontrol/lib/go-tc"
+	util "github.com/apache/trafficcontrol/lib/go-util"
+
+	_ "github.com/lib/pq"
+)
+
+// PGConfig represents the configuration options available to the PG backend
+type PGConfig struct {
+	Host     string `json:"host"`
+	Port     string `json:"port"`
+	User     string `json:"user"`
+	Password string `json:"password"`
+	SSLMode  string `json:"sslmode"`
+	Database string `json:"database"`
+	Key      string `json:"aesKey"`
+	AESKey   []byte
+}
+
+// PGBackend is the Postgres implementation of TVBackend
+type PGBackend struct {
+	sslKey pgSSLKeyTable
+	dnssec pgDNSSecTable
+	uri    pgURISignKeyTable
+	url    pgURLSigKeyTable
+	cfg    PGConfig
+	db     *sql.DB
+}
+
+// String returns a high level overview of the backend and its keys
+func (pg *PGBackend) String() string {
+	data := fmt.Sprintf("PG server %v@%v:%v\n", pg.cfg.User, pg.cfg.Host, pg.cfg.Port)
+	data += fmt.Sprintf("\tSSL Keys: %v\n", len(pg.sslKey.Records))
+	data += fmt.Sprintf("\tDNSSec Keys: %v\n", len(pg.dnssec.Records))
+	data += fmt.Sprintf("\tURI Keys: %v\n", len(pg.uri.Records))
+	data += fmt.Sprintf("\tURL Keys: %v\n", len(pg.url.Records))
+	return data
+}
+
+// Name returns the name for this backend
+func (pg *PGBackend) Name() string {
+	return "PG"
+}
+
+// ReadConfig takes in a filename and will read it into the backends config
+func (pg *PGBackend) ReadConfig(s string) error {
+	err := UnmarshalConfig(s, &pg.cfg)
+	if err != nil {
+		return err
+	}
+
+	pg.cfg.AESKey, err = base64.StdEncoding.DecodeString(pg.cfg.Key)
+	if err != nil {
+		return fmt.Errorf("unable to decode PG AESKey: %w", err)
+	}
+	return nil
+}
+
+// Insert takes the current keys and inserts them into the backend DB
+func (pg *PGBackend) Insert() error {
+	if err := pg.sslKey.insertKeys(pg.db); err != nil {
+		return err
+	}
+	if err := pg.dnssec.insertKeys(pg.db); err != nil {
+		return err
+	}
+	if err := pg.url.insertKeys(pg.db); err != nil {
+		return err
+	}
+	if err := pg.uri.insertKeys(pg.db); err != nil {
+		return err
+	}
+	return nil
+}
+
+// Start initiates the connection to the backend DB
+func (pg *PGBackend) Start() error {
+	sqlStr := fmt.Sprintf("postgres://%s:%s@%s:%s/%s?sslmode=%s", pg.cfg.User, pg.cfg.Password, pg.cfg.Host, pg.cfg.Port, pg.cfg.Database, pg.cfg.SSLMode)
+	db, err := sql.Open("postgres", sqlStr)
+	if err != nil {
+		return fmt.Errorf("unable to start PG client: %w", err)
+	}
+
+	pg.db = db
+	pg.sslKey = pgSSLKeyTable{}
+	pg.dnssec = pgDNSSecTable{}
+	pg.url = pgURLSigKeyTable{}
+	pg.uri = pgURISignKeyTable{}
+
+	return nil
+}
+
+// ValidateKey validates that the keys are valid (in most cases, certain fields are not null)
+func (pg *PGBackend) ValidateKey() []string {
+	var errors []string
+	if errs := pg.sslKey.validate(); errs != nil {
+		errors = append(errors, errs...)
+	}
+	if errs := pg.dnssec.validate(); errs != nil {
+		errors = append(errors, errs...)
+	}
+	if errs := pg.uri.validate(); errs != nil {
+		errors = append(errors, errs...)
+	}
+	if errs := pg.url.validate(); errs != nil {
+		errors = append(errors, errs...)
+	}
+	return errors
+}
+
+// Close terminates the connection to the backend DB
+func (pg *PGBackend) Close() error {
+	return pg.db.Close()
+}
+
+// Ping checks the connection to the backend DB
+func (pg *PGBackend) Ping() error {
+	return pg.db.Ping()
+}
+
+// Fetch gets all of the keys from the backend DB
+func (pg *PGBackend) Fetch() error {
+	if err := pg.sslKey.gatherKeys(pg.db); err != nil {
+		return err
+	}
+
+	if err := pg.dnssec.gatherKeys(pg.db); err != nil {
+		return err
+	}
+
+	if err := pg.url.gatherKeys(pg.db); err != nil {
+		return err
+	}
+
+	if err := pg.uri.gatherKeys(pg.db); err != nil {
+		return err
+	}
+
+	return nil
+}
+
+// GetSSLKeys converts the backends internal key representation into the common representation (SSLKey)
+func (pg *PGBackend) GetSSLKeys() ([]SSLKey, error) {
+	if err := pg.sslKey.decrypt(pg.cfg.AESKey); err != nil {
+		return nil, err
+	}
+	return pg.sslKey.toGeneric(), nil
+}
+
+// SetSSLKeys takes in keys and converts & encrypts the data into the backends internal format
+func (pg *PGBackend) SetSSLKeys(keys []SSLKey) error {
+	pg.sslKey.fromGeneric(keys)
+	return pg.sslKey.encrypt(pg.cfg.AESKey)
+}
+
+// GetDNSSecKeys converts the backends internal key representation into the common representation (DNSSecKey)
+func (pg *PGBackend) GetDNSSecKeys() ([]DNSSecKey, error) {
+	if err := pg.dnssec.decrypt(pg.cfg.AESKey); err != nil {
+		return nil, err
+	}
+	return pg.dnssec.toGeneric(), nil
+}
+
+// SetDNSSecKeys takes in keys and converts & encrypts the data into the backends internal format
+func (pg *PGBackend) SetDNSSecKeys(keys []DNSSecKey) error {
+	pg.dnssec.fromGeneric(keys)
+	return pg.dnssec.encrypt(pg.cfg.AESKey)
+}
+
+// GetURISignKeys converts the pg internal key representation into the common representation (URISignKey)
+func (pg *PGBackend) GetURISignKeys() ([]URISignKey, error) {
+	if err := pg.uri.decrypt(pg.cfg.AESKey); err != nil {
+		return nil, err
+	}
+	return pg.uri.toGeneric(), nil
+}
+
+// SetURISignKeys takes in keys and converts & encrypts the data into the backends internal format
+func (pg *PGBackend) SetURISignKeys(keys []URISignKey) error {
+	pg.uri.fromGeneric(keys)
+	return pg.uri.encrypt(pg.cfg.AESKey)
+}
+
+// GetURLSigKeys converts the backends internal key representation into the common representation (URLSigKey)
+func (pg *PGBackend) GetURLSigKeys() ([]URLSigKey, error) {
+	if err := pg.url.decrypt(pg.cfg.AESKey); err != nil {
+		return nil, err
+	}
+	return pg.url.toGeneric(), nil
+}
+
+// SetURLSigKeys takes in keys and converts & encrypts the data into the backends internal format
+func (pg *PGBackend) SetURLSigKeys(keys []URLSigKey) error {
+	pg.url.fromGeneric(keys)
+	return pg.url.encrypt(pg.cfg.AESKey)
+}
+
+type pgCommonRecord struct {
+	DataEncrypted []byte
+}
+
+type pgDNSSecRecord struct {
+	Key tc.DNSSECKeysTrafficVault
+	CDN string
+	pgCommonRecord
+}
+type pgDNSSecTable struct {
+	Records []pgDNSSecRecord
+}
+
+func (tbl *pgDNSSecTable) gatherKeys(db *sql.DB) error {
+	sz, err := getSize(db, "dnssec")
+	if err != nil {
+		log.Println("PGDNSSec gatherKeys: unable to determine size of dnssec table")
+	}
+	tbl.Records = make([]pgDNSSecRecord, sz)
+
+	rows, err := db.Query("SELECT cdn, data from dnssec")
+	if err != nil {
+		return fmt.Errorf("PGDNSSec gatherKeys: unable to query: %w", err)
+	}
+	defer rows.Close()
+	i := 0
+	for rows.Next() {
+		if i > len(tbl.Records)-1 {
+			return fmt.Errorf("PGDNSSec gatherKeys got more results than expected %v", len(tbl.Records))
+		}
+		err := rows.Scan(&tbl.Records[i].CDN, &tbl.Records[i].DataEncrypted)
+		if err != nil {
+			return fmt.Errorf("PGDNSSec gatherKeys unable to scan row: %w", err)
+		}
+		i += 1
+	}
+	return nil
+}
+func (tbl *pgDNSSecTable) decrypt(aesKey []byte) error {
+	for i, _ := range tbl.Records {
+		err := decryptInto(aesKey, tbl.Records[i].DataEncrypted, &tbl.Records[i].Key)
+		if err != nil {
+			return fmt.Errorf("unable to decrypt into keys: %w", err)
+		}
+	}
+	return nil
+}
+func (tbl *pgDNSSecTable) encrypt(aesKey []byte) error {
+	for i, dns := range tbl.Records {
+		data, err := json.Marshal(&dns.Key)
+		if err != nil {
+			return fmt.Errorf("encrypt issue marshalling keys: %w", err)
+		}
+		dat, err := encrypt(data, aesKey)
+		if err != nil {
+			return fmt.Errorf("encrypt error: %w", err)
+		}
+		tbl.Records[i].DataEncrypted = dat
+	}
+	return nil
+}
+func (tbl *pgDNSSecTable) toGeneric() []DNSSecKey {
+	keys := make([]DNSSecKey, len(tbl.Records))
+
+	for i, record := range tbl.Records {
+		keys[i] = DNSSecKey{
+			CDN:                    record.CDN,
+			DNSSECKeysTrafficVault: record.Key,
+		}
+	}
+
+	return keys
+}
+func (tbl *pgDNSSecTable) fromGeneric(keys []DNSSecKey) {
+	tbl.Records = make([]pgDNSSecRecord, len(keys))
+
+	for i, key := range keys {
+		tbl.Records[i] = pgDNSSecRecord{
+			Key: key.DNSSECKeysTrafficVault,
+			CDN: key.CDN,
+			pgCommonRecord: pgCommonRecord{
+				DataEncrypted: nil,
+			},
+		}
+	}
+}
+func (tbl *pgDNSSecTable) validate() []string {
+	for i, record := range tbl.Records {
+		if record.DataEncrypted == nil && len(record.Key) > 0 {
+			return []string{fmt.Sprintf("DNSSEC Key %v: DataEncrypted is blank!", i)}
+		}
+	}
+	return nil
+}
+func (tbl *pgDNSSecTable) insertKeys(db *sql.DB) error {
+	queryBase := "INSERT INTO dnssec (cdn, data) VALUES "
+	stride := 2
+	queryArgs := make([]interface{}, len(tbl.Records)*stride)
+	for i, record := range tbl.Records {
+		j := i * stride
+		queryArgs[j] = record.CDN
+		queryArgs[j+1] = record.DataEncrypted
+	}
+	return insertIntoTable(db, queryBase, stride, queryArgs)
+}
+
+type pgSSLKeyRecord struct {
+	Keys tc.DeliveryServiceSSLKeys
+	pgCommonRecord
+
+	// These records are stored on the table but are duplicated
+	DeliveryService string
+	CDN             string
+}
+type pgSSLKeyTable struct {
+	Records []pgSSLKeyRecord
+}
+
+func (tbl *pgSSLKeyTable) insertKeys(db *sql.DB) error {
+	queryBase := "INSERT INTO sslkey (deliveryservice, data, cdn, version) VALUES "
+	duplicateKeys := 2
+	stride := 4
+	queryArgs := make([]interface{}, len(tbl.Records)*stride*duplicateKeys)
+	for i, record := range tbl.Records {
+		j := i * duplicateKeys * stride
+
+		queryArgs[j] = record.DeliveryService
+		queryArgs[j+1] = record.DataEncrypted
+		queryArgs[j+2] = record.CDN
+		queryArgs[j+3] = record.Keys.Version.String()
+
+		queryArgs[j+4] = record.DeliveryService
+		queryArgs[j+5] = record.DataEncrypted
+		queryArgs[j+6] = record.CDN
+		queryArgs[j+7] = "latest"
+	}
+	return insertIntoTable(db, queryBase, 4, queryArgs)
+}
+func (tbl *pgSSLKeyTable) gatherKeys(db *sql.DB) error {
+	sz, err := getSize(db, "sslkey WHERE version='latest'")
+	if err != nil {
+		return fmt.Errorf("PGSSLKey gatherKeys unable to determine size of sslkey table: %w", err)
+	}
+	tbl.Records = make([]pgSSLKeyRecord, sz)
+
+	rows, err := db.Query("SELECT data, deliveryservice, cdn from sslkey WHERE version = 'latest'")
+	if err != nil {
+		return fmt.Errorf("PGSSLKey gatherKeys unable to query: %w", err)
+	}
+	defer rows.Close()
+	i := 0
+	for rows.Next() {
+		if i > len(tbl.Records)-1 {
+			return errors.New("PGSSLKey gatherKeys: got more results than expected")
+		}
+		err := rows.Scan(&tbl.Records[i].DataEncrypted, &tbl.Records[i].DeliveryService, &tbl.Records[i].CDN)
+		if err != nil {
+			return fmt.Errorf("PGSSLKey gatherKeys unable to scan row: %w", err)
+		}
+		i += 1
+	}
+	return nil
+}
+func (tbl *pgSSLKeyTable) decrypt(aesKey []byte) error {
+	for i, dns := range tbl.Records {
+		err := decryptInto(aesKey, dns.DataEncrypted, &tbl.Records[i].Keys)
+		if err != nil {
+			return fmt.Errorf("unable to decrypt into keys: %w", err)
+		}
+	}
+	return nil
+}
+func (tbl *pgSSLKeyTable) encrypt(aesKey []byte) error {
+	for i, dns := range tbl.Records {
+		data, err := json.Marshal(dns.Keys)
+		if err != nil {
+			return fmt.Errorf("encrypt issue marshalling keys: %w", err)
+		}
+		dat, err := encrypt(data, aesKey)
+		if err != nil {
+			return fmt.Errorf("encrypt error: %w", err)
+		}
+		tbl.Records[i].DataEncrypted = dat
+	}
+	return nil
+}
+func (tbl *pgSSLKeyTable) toGeneric() []SSLKey {
+	keys := make([]SSLKey, len(tbl.Records))
+
+	for i, record := range tbl.Records {
+		keys[i] = SSLKey{
+			DeliveryServiceSSLKeys: record.Keys,
+		}
+	}
+	return keys
+}
+func (tbl *pgSSLKeyTable) fromGeneric(keys []SSLKey) {
+	tbl.Records = make([]pgSSLKeyRecord, len(keys))
+
+	for i, key := range keys {
+		tbl.Records[i] = pgSSLKeyRecord{
+			Keys: key.DeliveryServiceSSLKeys,
+			pgCommonRecord: pgCommonRecord{
+				DataEncrypted: nil,
+			},
+			DeliveryService: key.DeliveryService,
+			CDN:             key.CDN,
+		}
+	}
+}
+func (tbl *pgSSLKeyTable) validate() []string {
+	defaultKey := tc.DeliveryServiceSSLKeys{}
+	var errors []string
+	fmtStr := "SSL Key %v: %v"
+	for i, record := range tbl.Records {
+		if record.Keys == defaultKey {
+			errors = append(errors, fmt.Sprintf(fmtStr, i, "DS SSL Keys are default!"))
+		} else if record.Keys.Key == "" {
+			errors = append(errors, fmt.Sprintf(fmtStr, i, "Key is blank!"))
+		} else if record.Keys.CDN == "" {
+			errors = append(errors, fmt.Sprintf(fmtStr, i, "CDN is blank!"))
+		} else if record.Keys.DeliveryService == "" {
+			errors = append(errors, fmt.Sprintf(fmtStr, i, "DS is blank!"))
+		} else if record.DataEncrypted == nil {
+			errors = append(errors, fmt.Sprintf(fmtStr, i, "DataEncrypted is blank!"))
+		} else if record.Keys.Version.String() == "" {
+			errors = append(errors, fmt.Sprintf(fmtStr, i, "Version is blank!"))
+		}
+	}
+	return errors
+}
+
+type pgURLSigKeyRecord struct {
+	Keys            tc.URLSigKeys
+	DeliveryService string
+	pgCommonRecord
+}
+type pgURLSigKeyTable struct {
+	Records []pgURLSigKeyRecord
+}
+
+func (tbl *pgURLSigKeyTable) insertKeys(db *sql.DB) error {
+	queryBase := "INSERT INTO url_sig_key (deliveryservice, data) VALUES "
+	stride := 2
+	queryArgs := make([]interface{}, len(tbl.Records)*stride)
+	for i, record := range tbl.Records {
+		j := i * stride
+		queryArgs[j] = record.DeliveryService
+		queryArgs[j+1] = record.DataEncrypted
+	}
+	return insertIntoTable(db, queryBase, stride, queryArgs)
+}
+func (tbl *pgURLSigKeyTable) gatherKeys(db *sql.DB) error {
+	sz, err := getSize(db, "url_sig_key")
+	if err != nil {
+		log.Println("PGURLSigKey gatherKeys: unable to determine url_sig_key table size")
+	}
+	tbl.Records = make([]pgURLSigKeyRecord, sz)
+
+	rows, err := db.Query("SELECT deliveryservice, data from url_sig_key")
+	if err != nil {
+		return fmt.Errorf("PGURLSigKey gatherKeys error creating query: %w", err)
+	}
+	defer rows.Close()
+	i := 0
+	for rows.Next() {
+		if i > len(tbl.Records)-1 {
+			return fmt.Errorf("PGURLSigKey gatherKeys: got more results than expected %v", len(tbl.Records))

Review comment:
       nit: %d instead of %v

##########
File path: tools/traffic_vault_migrate/postgres.go
##########
@@ -0,0 +1,743 @@
+package main
+
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+import (
+	"database/sql"
+	"encoding/base64"
+	"encoding/json"
+	"errors"
+	"fmt"
+	"log"
+	"strconv"
+	"strings"
+
+	"github.com/apache/trafficcontrol/lib/go-tc"
+	util "github.com/apache/trafficcontrol/lib/go-util"
+
+	_ "github.com/lib/pq"
+)
+
+// PGConfig represents the configuration options available to the PG backend
+type PGConfig struct {
+	Host     string `json:"host"`
+	Port     string `json:"port"`
+	User     string `json:"user"`
+	Password string `json:"password"`
+	SSLMode  string `json:"sslmode"`
+	Database string `json:"database"`
+	Key      string `json:"aesKey"`
+	AESKey   []byte
+}
+
+// PGBackend is the Postgres implementation of TVBackend
+type PGBackend struct {
+	sslKey pgSSLKeyTable
+	dnssec pgDNSSecTable
+	uri    pgURISignKeyTable
+	url    pgURLSigKeyTable
+	cfg    PGConfig
+	db     *sql.DB
+}
+
+// String returns a high level overview of the backend and its keys
+func (pg *PGBackend) String() string {
+	data := fmt.Sprintf("PG server %v@%v:%v\n", pg.cfg.User, pg.cfg.Host, pg.cfg.Port)
+	data += fmt.Sprintf("\tSSL Keys: %v\n", len(pg.sslKey.Records))
+	data += fmt.Sprintf("\tDNSSec Keys: %v\n", len(pg.dnssec.Records))
+	data += fmt.Sprintf("\tURI Keys: %v\n", len(pg.uri.Records))
+	data += fmt.Sprintf("\tURL Keys: %v\n", len(pg.url.Records))
+	return data
+}
+
+// Name returns the name for this backend
+func (pg *PGBackend) Name() string {
+	return "PG"
+}
+
+// ReadConfig takes in a filename and will read it into the backends config
+func (pg *PGBackend) ReadConfig(s string) error {
+	err := UnmarshalConfig(s, &pg.cfg)
+	if err != nil {
+		return err
+	}
+
+	pg.cfg.AESKey, err = base64.StdEncoding.DecodeString(pg.cfg.Key)
+	if err != nil {
+		return fmt.Errorf("unable to decode PG AESKey: %w", err)
+	}
+	return nil
+}
+
+// Insert takes the current keys and inserts them into the backend DB
+func (pg *PGBackend) Insert() error {
+	if err := pg.sslKey.insertKeys(pg.db); err != nil {
+		return err
+	}
+	if err := pg.dnssec.insertKeys(pg.db); err != nil {
+		return err
+	}
+	if err := pg.url.insertKeys(pg.db); err != nil {
+		return err
+	}
+	if err := pg.uri.insertKeys(pg.db); err != nil {
+		return err
+	}
+	return nil
+}
+
+// Start initiates the connection to the backend DB
+func (pg *PGBackend) Start() error {
+	sqlStr := fmt.Sprintf("postgres://%s:%s@%s:%s/%s?sslmode=%s", pg.cfg.User, pg.cfg.Password, pg.cfg.Host, pg.cfg.Port, pg.cfg.Database, pg.cfg.SSLMode)
+	db, err := sql.Open("postgres", sqlStr)
+	if err != nil {
+		return fmt.Errorf("unable to start PG client: %w", err)
+	}
+
+	pg.db = db
+	pg.sslKey = pgSSLKeyTable{}
+	pg.dnssec = pgDNSSecTable{}
+	pg.url = pgURLSigKeyTable{}
+	pg.uri = pgURISignKeyTable{}
+
+	return nil
+}
+
+// ValidateKey validates that the keys are valid (in most cases, certain fields are not null)
+func (pg *PGBackend) ValidateKey() []string {
+	var errors []string
+	if errs := pg.sslKey.validate(); errs != nil {
+		errors = append(errors, errs...)
+	}
+	if errs := pg.dnssec.validate(); errs != nil {
+		errors = append(errors, errs...)
+	}
+	if errs := pg.uri.validate(); errs != nil {
+		errors = append(errors, errs...)
+	}
+	if errs := pg.url.validate(); errs != nil {
+		errors = append(errors, errs...)
+	}
+	return errors
+}
+
+// Close terminates the connection to the backend DB
+func (pg *PGBackend) Close() error {
+	return pg.db.Close()
+}
+
+// Ping checks the connection to the backend DB
+func (pg *PGBackend) Ping() error {
+	return pg.db.Ping()
+}
+
+// Fetch gets all of the keys from the backend DB
+func (pg *PGBackend) Fetch() error {
+	if err := pg.sslKey.gatherKeys(pg.db); err != nil {
+		return err
+	}
+
+	if err := pg.dnssec.gatherKeys(pg.db); err != nil {
+		return err
+	}
+
+	if err := pg.url.gatherKeys(pg.db); err != nil {
+		return err
+	}
+
+	if err := pg.uri.gatherKeys(pg.db); err != nil {
+		return err
+	}
+
+	return nil
+}
+
+// GetSSLKeys converts the backends internal key representation into the common representation (SSLKey)
+func (pg *PGBackend) GetSSLKeys() ([]SSLKey, error) {
+	if err := pg.sslKey.decrypt(pg.cfg.AESKey); err != nil {
+		return nil, err
+	}
+	return pg.sslKey.toGeneric(), nil
+}
+
+// SetSSLKeys takes in keys and converts & encrypts the data into the backends internal format
+func (pg *PGBackend) SetSSLKeys(keys []SSLKey) error {
+	pg.sslKey.fromGeneric(keys)
+	return pg.sslKey.encrypt(pg.cfg.AESKey)
+}
+
+// GetDNSSecKeys converts the backends internal key representation into the common representation (DNSSecKey)
+func (pg *PGBackend) GetDNSSecKeys() ([]DNSSecKey, error) {
+	if err := pg.dnssec.decrypt(pg.cfg.AESKey); err != nil {
+		return nil, err
+	}
+	return pg.dnssec.toGeneric(), nil
+}
+
+// SetDNSSecKeys takes in keys and converts & encrypts the data into the backends internal format
+func (pg *PGBackend) SetDNSSecKeys(keys []DNSSecKey) error {
+	pg.dnssec.fromGeneric(keys)
+	return pg.dnssec.encrypt(pg.cfg.AESKey)
+}
+
+// GetURISignKeys converts the pg internal key representation into the common representation (URISignKey)
+func (pg *PGBackend) GetURISignKeys() ([]URISignKey, error) {
+	if err := pg.uri.decrypt(pg.cfg.AESKey); err != nil {
+		return nil, err
+	}
+	return pg.uri.toGeneric(), nil
+}
+
+// SetURISignKeys takes in keys and converts & encrypts the data into the backends internal format
+func (pg *PGBackend) SetURISignKeys(keys []URISignKey) error {
+	pg.uri.fromGeneric(keys)
+	return pg.uri.encrypt(pg.cfg.AESKey)
+}
+
+// GetURLSigKeys converts the backends internal key representation into the common representation (URLSigKey)
+func (pg *PGBackend) GetURLSigKeys() ([]URLSigKey, error) {
+	if err := pg.url.decrypt(pg.cfg.AESKey); err != nil {
+		return nil, err
+	}
+	return pg.url.toGeneric(), nil
+}
+
+// SetURLSigKeys takes in keys and converts & encrypts the data into the backends internal format
+func (pg *PGBackend) SetURLSigKeys(keys []URLSigKey) error {
+	pg.url.fromGeneric(keys)
+	return pg.url.encrypt(pg.cfg.AESKey)
+}
+
+type pgCommonRecord struct {
+	DataEncrypted []byte
+}
+
+type pgDNSSecRecord struct {
+	Key tc.DNSSECKeysTrafficVault
+	CDN string
+	pgCommonRecord
+}
+type pgDNSSecTable struct {
+	Records []pgDNSSecRecord
+}
+
+func (tbl *pgDNSSecTable) gatherKeys(db *sql.DB) error {
+	sz, err := getSize(db, "dnssec")
+	if err != nil {
+		log.Println("PGDNSSec gatherKeys: unable to determine size of dnssec table")
+	}
+	tbl.Records = make([]pgDNSSecRecord, sz)
+
+	rows, err := db.Query("SELECT cdn, data from dnssec")
+	if err != nil {
+		return fmt.Errorf("PGDNSSec gatherKeys: unable to query: %w", err)
+	}
+	defer rows.Close()
+	i := 0
+	for rows.Next() {
+		if i > len(tbl.Records)-1 {
+			return fmt.Errorf("PGDNSSec gatherKeys got more results than expected %v", len(tbl.Records))
+		}
+		err := rows.Scan(&tbl.Records[i].CDN, &tbl.Records[i].DataEncrypted)
+		if err != nil {
+			return fmt.Errorf("PGDNSSec gatherKeys unable to scan row: %w", err)
+		}
+		i += 1
+	}
+	return nil
+}
+func (tbl *pgDNSSecTable) decrypt(aesKey []byte) error {
+	for i, _ := range tbl.Records {
+		err := decryptInto(aesKey, tbl.Records[i].DataEncrypted, &tbl.Records[i].Key)
+		if err != nil {
+			return fmt.Errorf("unable to decrypt into keys: %w", err)
+		}
+	}
+	return nil
+}
+func (tbl *pgDNSSecTable) encrypt(aesKey []byte) error {
+	for i, dns := range tbl.Records {
+		data, err := json.Marshal(&dns.Key)
+		if err != nil {
+			return fmt.Errorf("encrypt issue marshalling keys: %w", err)
+		}
+		dat, err := encrypt(data, aesKey)
+		if err != nil {
+			return fmt.Errorf("encrypt error: %w", err)
+		}
+		tbl.Records[i].DataEncrypted = dat
+	}
+	return nil
+}
+func (tbl *pgDNSSecTable) toGeneric() []DNSSecKey {
+	keys := make([]DNSSecKey, len(tbl.Records))
+
+	for i, record := range tbl.Records {
+		keys[i] = DNSSecKey{
+			CDN:                    record.CDN,
+			DNSSECKeysTrafficVault: record.Key,
+		}
+	}
+
+	return keys
+}
+func (tbl *pgDNSSecTable) fromGeneric(keys []DNSSecKey) {
+	tbl.Records = make([]pgDNSSecRecord, len(keys))
+
+	for i, key := range keys {
+		tbl.Records[i] = pgDNSSecRecord{
+			Key: key.DNSSECKeysTrafficVault,
+			CDN: key.CDN,
+			pgCommonRecord: pgCommonRecord{
+				DataEncrypted: nil,
+			},
+		}
+	}
+}
+func (tbl *pgDNSSecTable) validate() []string {
+	for i, record := range tbl.Records {
+		if record.DataEncrypted == nil && len(record.Key) > 0 {
+			return []string{fmt.Sprintf("DNSSEC Key %v: DataEncrypted is blank!", i)}
+		}
+	}
+	return nil
+}
+func (tbl *pgDNSSecTable) insertKeys(db *sql.DB) error {
+	queryBase := "INSERT INTO dnssec (cdn, data) VALUES "
+	stride := 2
+	queryArgs := make([]interface{}, len(tbl.Records)*stride)
+	for i, record := range tbl.Records {
+		j := i * stride
+		queryArgs[j] = record.CDN
+		queryArgs[j+1] = record.DataEncrypted
+	}
+	return insertIntoTable(db, queryBase, stride, queryArgs)
+}
+
+type pgSSLKeyRecord struct {
+	Keys tc.DeliveryServiceSSLKeys
+	pgCommonRecord
+
+	// These records are stored on the table but are duplicated
+	DeliveryService string
+	CDN             string
+}
+type pgSSLKeyTable struct {
+	Records []pgSSLKeyRecord
+}
+
+func (tbl *pgSSLKeyTable) insertKeys(db *sql.DB) error {
+	queryBase := "INSERT INTO sslkey (deliveryservice, data, cdn, version) VALUES "
+	duplicateKeys := 2
+	stride := 4
+	queryArgs := make([]interface{}, len(tbl.Records)*stride*duplicateKeys)
+	for i, record := range tbl.Records {
+		j := i * duplicateKeys * stride
+
+		queryArgs[j] = record.DeliveryService
+		queryArgs[j+1] = record.DataEncrypted
+		queryArgs[j+2] = record.CDN
+		queryArgs[j+3] = record.Keys.Version.String()
+
+		queryArgs[j+4] = record.DeliveryService
+		queryArgs[j+5] = record.DataEncrypted
+		queryArgs[j+6] = record.CDN
+		queryArgs[j+7] = "latest"
+	}
+	return insertIntoTable(db, queryBase, 4, queryArgs)
+}
+func (tbl *pgSSLKeyTable) gatherKeys(db *sql.DB) error {
+	sz, err := getSize(db, "sslkey WHERE version='latest'")
+	if err != nil {
+		return fmt.Errorf("PGSSLKey gatherKeys unable to determine size of sslkey table: %w", err)
+	}
+	tbl.Records = make([]pgSSLKeyRecord, sz)
+
+	rows, err := db.Query("SELECT data, deliveryservice, cdn from sslkey WHERE version = 'latest'")
+	if err != nil {
+		return fmt.Errorf("PGSSLKey gatherKeys unable to query: %w", err)
+	}
+	defer rows.Close()
+	i := 0
+	for rows.Next() {
+		if i > len(tbl.Records)-1 {
+			return errors.New("PGSSLKey gatherKeys: got more results than expected")
+		}
+		err := rows.Scan(&tbl.Records[i].DataEncrypted, &tbl.Records[i].DeliveryService, &tbl.Records[i].CDN)
+		if err != nil {
+			return fmt.Errorf("PGSSLKey gatherKeys unable to scan row: %w", err)
+		}
+		i += 1
+	}
+	return nil
+}
+func (tbl *pgSSLKeyTable) decrypt(aesKey []byte) error {
+	for i, dns := range tbl.Records {
+		err := decryptInto(aesKey, dns.DataEncrypted, &tbl.Records[i].Keys)
+		if err != nil {
+			return fmt.Errorf("unable to decrypt into keys: %w", err)
+		}
+	}
+	return nil
+}
+func (tbl *pgSSLKeyTable) encrypt(aesKey []byte) error {
+	for i, dns := range tbl.Records {
+		data, err := json.Marshal(dns.Keys)
+		if err != nil {
+			return fmt.Errorf("encrypt issue marshalling keys: %w", err)
+		}
+		dat, err := encrypt(data, aesKey)
+		if err != nil {
+			return fmt.Errorf("encrypt error: %w", err)
+		}
+		tbl.Records[i].DataEncrypted = dat
+	}
+	return nil
+}
+func (tbl *pgSSLKeyTable) toGeneric() []SSLKey {
+	keys := make([]SSLKey, len(tbl.Records))
+
+	for i, record := range tbl.Records {
+		keys[i] = SSLKey{
+			DeliveryServiceSSLKeys: record.Keys,
+		}
+	}
+	return keys
+}
+func (tbl *pgSSLKeyTable) fromGeneric(keys []SSLKey) {
+	tbl.Records = make([]pgSSLKeyRecord, len(keys))
+
+	for i, key := range keys {
+		tbl.Records[i] = pgSSLKeyRecord{
+			Keys: key.DeliveryServiceSSLKeys,
+			pgCommonRecord: pgCommonRecord{
+				DataEncrypted: nil,
+			},
+			DeliveryService: key.DeliveryService,
+			CDN:             key.CDN,
+		}
+	}
+}
+func (tbl *pgSSLKeyTable) validate() []string {
+	defaultKey := tc.DeliveryServiceSSLKeys{}
+	var errors []string
+	fmtStr := "SSL Key %v: %v"
+	for i, record := range tbl.Records {
+		if record.Keys == defaultKey {
+			errors = append(errors, fmt.Sprintf(fmtStr, i, "DS SSL Keys are default!"))
+		} else if record.Keys.Key == "" {
+			errors = append(errors, fmt.Sprintf(fmtStr, i, "Key is blank!"))
+		} else if record.Keys.CDN == "" {
+			errors = append(errors, fmt.Sprintf(fmtStr, i, "CDN is blank!"))
+		} else if record.Keys.DeliveryService == "" {
+			errors = append(errors, fmt.Sprintf(fmtStr, i, "DS is blank!"))
+		} else if record.DataEncrypted == nil {
+			errors = append(errors, fmt.Sprintf(fmtStr, i, "DataEncrypted is blank!"))
+		} else if record.Keys.Version.String() == "" {
+			errors = append(errors, fmt.Sprintf(fmtStr, i, "Version is blank!"))
+		}
+	}
+	return errors
+}
+
+type pgURLSigKeyRecord struct {
+	Keys            tc.URLSigKeys
+	DeliveryService string
+	pgCommonRecord
+}
+type pgURLSigKeyTable struct {
+	Records []pgURLSigKeyRecord
+}
+
+func (tbl *pgURLSigKeyTable) insertKeys(db *sql.DB) error {
+	queryBase := "INSERT INTO url_sig_key (deliveryservice, data) VALUES "
+	stride := 2
+	queryArgs := make([]interface{}, len(tbl.Records)*stride)
+	for i, record := range tbl.Records {
+		j := i * stride
+		queryArgs[j] = record.DeliveryService
+		queryArgs[j+1] = record.DataEncrypted
+	}
+	return insertIntoTable(db, queryBase, stride, queryArgs)
+}
+func (tbl *pgURLSigKeyTable) gatherKeys(db *sql.DB) error {
+	sz, err := getSize(db, "url_sig_key")
+	if err != nil {
+		log.Println("PGURLSigKey gatherKeys: unable to determine url_sig_key table size")
+	}
+	tbl.Records = make([]pgURLSigKeyRecord, sz)
+
+	rows, err := db.Query("SELECT deliveryservice, data from url_sig_key")
+	if err != nil {
+		return fmt.Errorf("PGURLSigKey gatherKeys error creating query: %w", err)
+	}
+	defer rows.Close()
+	i := 0
+	for rows.Next() {
+		if i > len(tbl.Records)-1 {
+			return fmt.Errorf("PGURLSigKey gatherKeys: got more results than expected %v", len(tbl.Records))
+		}
+		err := rows.Scan(&tbl.Records[i].DeliveryService, &tbl.Records[i].DataEncrypted)
+		if err != nil {
+			return fmt.Errorf("PGURLSigKey gatherKeys: unable to scan row: %w", err)
+		}
+		i += 1
+	}
+	return nil
+}
+func (tbl *pgURLSigKeyTable) decrypt(aesKey []byte) error {
+	for i, sig := range tbl.Records {
+		err := decryptInto(aesKey, sig.DataEncrypted, &tbl.Records[i].Keys)
+		if err != nil {
+			return fmt.Errorf("unable to decrypt into keys: %w", err)
+		}
+	}
+	return nil
+}
+func (tbl *pgURLSigKeyTable) encrypt(aesKey []byte) error {
+	for i, sig := range tbl.Records {
+		data, err := json.Marshal(&sig.Keys)
+		if err != nil {
+			return fmt.Errorf("encrypt issue marshalling keys: %w", err)
+		}
+
+		dat, err := encrypt(data, aesKey)
+		if err != nil {
+			return fmt.Errorf("encrypt error: %w", err)
+		}
+		tbl.Records[i].DataEncrypted = dat
+	}
+	return nil
+}
+func (tbl *pgURLSigKeyTable) toGeneric() []URLSigKey {
+	keys := make([]URLSigKey, len(tbl.Records))
+
+	for i, record := range tbl.Records {
+		keys[i] = URLSigKey{
+			DeliveryService: record.DeliveryService,
+			URLSigKeys:      record.Keys,
+		}
+	}
+	return keys
+}
+func (tbl *pgURLSigKeyTable) fromGeneric(keys []URLSigKey) {
+	tbl.Records = make([]pgURLSigKeyRecord, len(keys))
+
+	for i, key := range keys {
+		tbl.Records[i] = pgURLSigKeyRecord{
+			Keys:            key.URLSigKeys,
+			DeliveryService: key.DeliveryService,
+			pgCommonRecord: pgCommonRecord{
+				DataEncrypted: nil,
+			},
+		}
+	}
+}
+func (tbl *pgURLSigKeyTable) validate() []string {
+	for i, record := range tbl.Records {
+		if record.DataEncrypted == nil && len(record.Keys) > 0 {
+			return []string{fmt.Sprintf("URl Sig Key %v: DataEncrypted is blank!", i)}
+		}
+	}
+	return nil
+}
+
+type pgURISignKeyRecord struct {
+	Keys            map[string]tc.URISignerKeyset
+	DeliveryService string
+	pgCommonRecord
+}
+type pgURISignKeyTable struct {
+	Records []pgURISignKeyRecord
+}
+
+func (tbl *pgURISignKeyTable) insertKeys(db *sql.DB) error {
+	queryBase := "INSERT INTO uri_signing_key (deliveryservice, data) VALUES "
+	stride := 2
+	queryArgs := make([]interface{}, len(tbl.Records)*stride)
+	for i, record := range tbl.Records {
+		j := i * stride
+		queryArgs[j] = record.DeliveryService
+		queryArgs[j+1] = record.DataEncrypted
+	}
+	return insertIntoTable(db, queryBase, stride, queryArgs)
+}
+func (tbl *pgURISignKeyTable) gatherKeys(db *sql.DB) error {
+	sz, err := getSize(db, "uri_signing_key")
+	if err != nil {
+		log.Println("PGURISignKey gatherKeys: unable to determine size of uri_signing_key table")
+	}
+	tbl.Records = make([]pgURISignKeyRecord, sz)
+
+	rows, err := db.Query("SELECT deliveryservice, data from uri_signing_key")
+	if err != nil {
+		return fmt.Errorf("PGURISignKey gatherKeys error while query: %w", err)
+	}
+	defer rows.Close()
+	i := 0
+	for rows.Next() {
+		if i > len(tbl.Records)-1 {
+			return fmt.Errorf("PGURISignKey gatherKeys: got more results than expected %v", len(tbl.Records))
+		}
+		err := rows.Scan(&tbl.Records[i].DeliveryService, &tbl.Records[i].DataEncrypted)
+		if err != nil {
+			return fmt.Errorf("PGURISignKey gatherKeys: unable to scan row: %w", err)
+		}
+		i += 1
+	}
+	return nil
+}
+func (tbl *pgURISignKeyTable) decrypt(aesKey []byte) error {
+	for i, sign := range tbl.Records {
+		err := decryptInto(aesKey, sign.DataEncrypted, &tbl.Records[i].Keys)
+		if err != nil {
+			return fmt.Errorf("unable to decrypt into keys: %w", err)
+		}
+	}
+	return nil
+}
+func (tbl *pgURISignKeyTable) encrypt(aesKey []byte) error {
+	for i, sign := range tbl.Records {
+		data, err := json.Marshal(sign.Keys)
+		if err != nil {
+			return fmt.Errorf("encrypt issue marshalling keys: %w", err)
+		}
+
+		dat, err := encrypt(data, aesKey)
+		if err != nil {
+			return fmt.Errorf("encrypt error: %w", err)
+		}
+		tbl.Records[i].DataEncrypted = dat
+	}
+	return nil
+}
+func (tbl *pgURISignKeyTable) toGeneric() []URISignKey {
+	keys := make([]URISignKey, len(tbl.Records))
+
+	for i, record := range tbl.Records {
+		keys[i] = URISignKey{
+			DeliveryService: record.DeliveryService,
+			Keys:            record.Keys,
+		}
+	}
+
+	return keys
+}
+func (tbl *pgURISignKeyTable) fromGeneric(keys []URISignKey) {
+	tbl.Records = make([]pgURISignKeyRecord, len(keys))
+
+	for i, key := range keys {
+		tbl.Records[i] = pgURISignKeyRecord{
+			Keys:            key.Keys,
+			DeliveryService: key.DeliveryService,
+			pgCommonRecord: pgCommonRecord{
+				DataEncrypted: nil,
+			},
+		}
+	}
+}
+func (tbl *pgURISignKeyTable) validate() []string {
+	for i, record := range tbl.Records {
+		if record.DataEncrypted == nil && len(record.Keys) > 0 {
+			return []string{fmt.Sprintf("URI Sign Key %v: DataEncrypted is blank!", i)}

Review comment:
       nit: %d insteaf of %v

##########
File path: tools/traffic_vault_migrate/riak.go
##########
@@ -0,0 +1,746 @@
+package main
+
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+import (
+	"crypto/tls"
+	"encoding/json"
+	"errors"
+	"fmt"
+	"time"
+
+	"github.com/apache/trafficcontrol/lib/go-rfc"
+	"github.com/apache/trafficcontrol/lib/go-tc"
+
+	"github.com/basho/riak-go-client"
+)
+
+const (
+	BUCKET_SSL     = "ssl"
+	BUCKET_DNSSEC  = "dnssec"
+	BUCKET_URL_SIG = "url_sig_keys"
+	BUCKET_URI_SIG = "cdn_uri_sig_keys"
+
+	INDEX_SSL = "sslkeys"
+
+	SCHEMA_RIAK_KEY    = "_yz_rk"
+	SCHEMA_RIAK_BUCKET = "_yz_rb"
+)
+
+var (
+	SCHEMA_SSL_FIELDS = [...]string{SCHEMA_RIAK_KEY, SCHEMA_RIAK_BUCKET}
+)
+
+// RiakConfig  represents the configuration options available to the Riak backend
+type RiakConfig struct {
+	Host          string `json:"host"`
+	Port          string `json:"port"`
+	User          string `json:"user"`
+	Password      string `json:"password"`
+	Insecure      bool   `json:"insecure"`
+	TLSVersionRaw string `json:"tlsVersion"`
+	TLSVersion    uint16
+}
+
+// RiakBackend is the Riak implementation of TVBackend
+type RiakBackend struct {
+	sslKeys     riakSSLKeyTable
+	dnssecKeys  riakDNSSecKeyTable
+	uriSignKeys riakURISignKeyTable
+	urlSigKeys  riakURLSigKeyTable
+	cfg         RiakConfig
+	cluster     *riak.Cluster
+}
+
+// String returns a high level overview of the backend and its keys
+func (rb *RiakBackend) String() string {
+	data := fmt.Sprintf("Riak server %v@%v:%v\n", rb.cfg.User, rb.cfg.Host, rb.cfg.Port)
+	data += fmt.Sprintf("\tSSL Keys: %v\n", len(rb.sslKeys.Records))
+	data += fmt.Sprintf("\tDNSSec Keys: %v\n", len(rb.dnssecKeys.Records))
+	data += fmt.Sprintf("\tURI Keys: %v\n", len(rb.uriSignKeys.Records))
+	data += fmt.Sprintf("\tURL Keys: %v\n", len(rb.urlSigKeys.Records))
+	return data
+}
+
+// Name returns the name for this backend
+func (rb *RiakBackend) Name() string {
+	return "Riak"
+}
+
+// ReadConfig takes in a filename and will read it into the backends config
+func (rb *RiakBackend) ReadConfig(s string) error {
+	err := UnmarshalConfig(s, &rb.cfg)
+	if err != nil {
+		return err
+	}
+
+	if rb.cfg.TLSVersionRaw == "10" {
+		rb.cfg.TLSVersion = tls.VersionTLS10
+	} else if rb.cfg.TLSVersionRaw == "11" {
+		rb.cfg.TLSVersion = tls.VersionTLS11
+	} else if rb.cfg.TLSVersionRaw == "12" {
+		rb.cfg.TLSVersion = tls.VersionTLS12
+	} else if rb.cfg.TLSVersionRaw == "13" {
+		rb.cfg.TLSVersion = tls.VersionTLS13
+	}
+	return nil
+}
+
+// Insert takes the current keys and inserts them into the backend DB
+func (rb *RiakBackend) Insert() error {
+	if err := rb.sslKeys.insertKeys(rb.cluster); err != nil {
+		return err
+	}
+	if err := rb.dnssecKeys.insertKeys(rb.cluster); err != nil {
+		return err
+	}
+	if err := rb.urlSigKeys.insertKeys(rb.cluster); err != nil {
+		return err
+	}
+	if err := rb.uriSignKeys.insertKeys(rb.cluster); err != nil {
+		return err
+	}
+	return nil
+}
+
+// ValidateKey validates that the keys are valid (in most cases, certain fields are not null)
+func (rb *RiakBackend) ValidateKey() []string {
+	errors := []string{}
+	if errs := rb.sslKeys.validate(); errs != nil {
+		errors = append(errors, errs...)
+	}
+	if errs := rb.dnssecKeys.validate(); errs != nil {
+		errors = append(errors, errs...)
+	}
+	if errs := rb.uriSignKeys.validate(); errs != nil {
+		errors = append(errors, errs...)
+	}
+	if errs := rb.urlSigKeys.validate(); errs != nil {
+		errors = append(errors, errs...)
+	}
+
+	return errors
+}
+
+// SetSSLKeys takes in keys and converts & encrypts the data into the backends internal format
+func (rb *RiakBackend) SetSSLKeys(keys []SSLKey) error {
+	rb.sslKeys.fromGeneric(keys)
+	return nil
+}
+
+// SetDNSSecKeys takes in keys and converts & encrypts the data into the backends internal format
+func (rb *RiakBackend) SetDNSSecKeys(keys []DNSSecKey) error {
+	rb.dnssecKeys.fromGeneric(keys)
+	return nil
+}
+
+// SetURISignKeys takes in keys and converts & encrypts the data into the backends internal format
+func (rb *RiakBackend) SetURISignKeys(keys []URISignKey) error {
+	rb.uriSignKeys.fromGeneric(keys)
+	return nil
+}
+
+// SetURLSigKeys takes in keys and converts & encrypts the data into the backends internal format
+func (rb *RiakBackend) SetURLSigKeys(keys []URLSigKey) error {
+	rb.urlSigKeys.fromGeneric(keys)
+	return nil
+}
+
+// Start initiates the connection to the backend DB
+func (rb *RiakBackend) Start() error {
+	tlsConfig := &tls.Config{
+		InsecureSkipVerify: !rb.cfg.Insecure,
+		MaxVersion:         rb.cfg.TLSVersion,
+	}
+	auth := &riak.AuthOptions{
+		User:      rb.cfg.User,
+		Password:  rb.cfg.Password,
+		TlsConfig: tlsConfig,
+	}
+
+	cluster, err := getRiakCluster(rb.cfg, auth)
+	if err != nil {
+		return err
+	}
+	if err := cluster.Start(); err != nil {
+		return fmt.Errorf("unable to start riak cluster: %w", err)
+	}
+
+	rb.cluster = cluster
+	rb.sslKeys = riakSSLKeyTable{}
+	rb.dnssecKeys = riakDNSSecKeyTable{}
+	rb.urlSigKeys = riakURLSigKeyTable{}
+	rb.uriSignKeys = riakURISignKeyTable{}
+	return nil
+}
+
+// Close terminates the connection to the backend DB
+func (rb *RiakBackend) Close() error {
+	if err := rb.cluster.Stop(); err != nil {
+		return err
+	}
+	return nil
+}
+
+// Ping checks the connection to the backend DB
+func (rb *RiakBackend) Ping() error {
+	return ping(rb.cluster)
+}
+
+// GetSSLKeys converts the backends internal key representation into the common representation (SSLKey)
+func (rb *RiakBackend) GetSSLKeys() ([]SSLKey, error) {
+	return rb.sslKeys.toGeneric(), nil
+}
+
+// GetDNSSecKeys converts the backends internal key representation into the common representation (DNSSecKey)
+func (rb *RiakBackend) GetDNSSecKeys() ([]DNSSecKey, error) {
+	return rb.dnssecKeys.toGeneric(), nil
+}
+
+// GetURISignKeys converts the pg internal key representation into the common representation (URISignKey)
+func (rb *RiakBackend) GetURISignKeys() ([]URISignKey, error) {
+	return rb.uriSignKeys.toGeneric(), nil
+}
+
+// GetURLSigKeys converts the backends internal key representation into the common representation (URLSigKey)
+func (rb *RiakBackend) GetURLSigKeys() ([]URLSigKey, error) {
+	return rb.urlSigKeys.toGeneric(), nil
+}
+
+// Fetch gets all of the keys from the backend DB
+func (rb *RiakBackend) Fetch() error {
+	if err := rb.sslKeys.gatherKeys(rb.cluster); err != nil {
+		return err
+	}
+	if err := rb.dnssecKeys.gatherKeys(rb.cluster); err != nil {
+		return err
+	}
+	if err := rb.urlSigKeys.gatherKeys(rb.cluster); err != nil {
+		return err
+	}
+	if err := rb.uriSignKeys.gatherKeys(rb.cluster); err != nil {
+		return err
+	}
+
+	return nil
+}
+
+type riakSSLKeyRecord struct {
+	tc.DeliveryServiceSSLKeys
+}
+type riakSSLKeyTable struct {
+	Records []riakSSLKeyRecord
+}
+
+func (tbl *riakSSLKeyTable) gatherKeys(cluster *riak.Cluster) error {
+	searchDocs, err := search(cluster, INDEX_SSL, "cdn:*", fmt.Sprintf("%v:*latest", SCHEMA_RIAK_KEY), 1000, SCHEMA_SSL_FIELDS[:])
+	if err != nil {
+		return fmt.Errorf("RiakSSLKey gatherKeys: %w", err)
+	}
+
+	tbl.Records = make([]riakSSLKeyRecord, len(searchDocs))
+	for i, doc := range searchDocs {
+		objs, err := getObject(cluster, doc.Bucket, doc.Key)
+		if err != nil {
+			return err
+		}
+		if len(objs) < 1 {
+			return fmt.Errorf("RiakSSLKey gatherKeys unable to find any objects with key %v and bucket %v, but search results were returned", doc.Key, doc.Bucket)
+		}
+		if len(objs) > 1 {
+			return fmt.Errorf("RiakSSLKey gatherKeys more than 1 ssl key record found %v\n", len(objs))
+		}
+		var obj tc.DeliveryServiceSSLKeys
+		if err = json.Unmarshal(objs[0].Value, &obj); err != nil {
+			return fmt.Errorf("RiakSSLKey gatherKeys unable to unmarshal object into tc.DeliveryServiceSSLKeys: %w", err)
+		}
+		tbl.Records[i] = riakSSLKeyRecord{
+			DeliveryServiceSSLKeys: obj,
+		}
+	}
+	return nil
+}
+func (tbl *riakSSLKeyTable) toGeneric() []SSLKey {
+	keys := make([]SSLKey, len(tbl.Records))
+
+	for i, record := range tbl.Records {
+		keys[i] = SSLKey{
+			DeliveryServiceSSLKeys: record.DeliveryServiceSSLKeys,
+		}
+	}
+
+	return keys
+}
+func (tbl *riakSSLKeyTable) fromGeneric(keys []SSLKey) {
+	tbl.Records = make([]riakSSLKeyRecord, len(keys))
+
+	for i, record := range keys {
+		tbl.Records[i] = riakSSLKeyRecord{
+			DeliveryServiceSSLKeys: record.DeliveryServiceSSLKeys,
+		}
+	}
+}
+func (tbl *riakSSLKeyTable) insertKeys(cluster *riak.Cluster) error {
+	for _, record := range tbl.Records {
+		objBytes, err := json.Marshal(record.DeliveryServiceSSLKeys)
+		if err != nil {
+			return fmt.Errorf("RiakSSLKey insertKeys failed to unmarshal keys: %w", err)
+		}
+		err = setObject(cluster, makeRiakObject(objBytes, record.DeliveryService+"-"+record.Version.String()), BUCKET_SSL)
+		if err != nil {
+			return fmt.Errorf("RiakSSLKey insertKeys: %w", err)
+		}
+		err = setObject(cluster, makeRiakObject(objBytes, record.DeliveryService+"-latest"), BUCKET_SSL)
+		if err != nil {
+			return fmt.Errorf("RiakSSLKey insertKeys: %w", err)
+		}
+	}
+	return nil
+}
+func (tbl *riakSSLKeyTable) validate() []string {
+	errs := []string{}
+	for i, record := range tbl.Records {
+		if record.DeliveryService == "" {
+			errs = append(errs, fmt.Sprintf("SSL Key #%v: Delivery Service is blank!", i))
+		}
+		if record.CDN == "" {
+			errs = append(errs, fmt.Sprintf("SSL Key #%v: CDN is blank!", i))

Review comment:
       nit: type-specific formatting instead of %v

##########
File path: tools/traffic_vault_migrate/riak.go
##########
@@ -0,0 +1,746 @@
+package main
+
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+import (
+	"crypto/tls"
+	"encoding/json"
+	"errors"
+	"fmt"
+	"time"
+
+	"github.com/apache/trafficcontrol/lib/go-rfc"
+	"github.com/apache/trafficcontrol/lib/go-tc"
+
+	"github.com/basho/riak-go-client"
+)
+
+const (
+	BUCKET_SSL     = "ssl"
+	BUCKET_DNSSEC  = "dnssec"
+	BUCKET_URL_SIG = "url_sig_keys"
+	BUCKET_URI_SIG = "cdn_uri_sig_keys"
+
+	INDEX_SSL = "sslkeys"
+
+	SCHEMA_RIAK_KEY    = "_yz_rk"
+	SCHEMA_RIAK_BUCKET = "_yz_rb"
+)
+
+var (
+	SCHEMA_SSL_FIELDS = [...]string{SCHEMA_RIAK_KEY, SCHEMA_RIAK_BUCKET}
+)
+
+// RiakConfig  represents the configuration options available to the Riak backend
+type RiakConfig struct {
+	Host          string `json:"host"`
+	Port          string `json:"port"`
+	User          string `json:"user"`
+	Password      string `json:"password"`
+	Insecure      bool   `json:"insecure"`
+	TLSVersionRaw string `json:"tlsVersion"`
+	TLSVersion    uint16
+}
+
+// RiakBackend is the Riak implementation of TVBackend
+type RiakBackend struct {
+	sslKeys     riakSSLKeyTable
+	dnssecKeys  riakDNSSecKeyTable
+	uriSignKeys riakURISignKeyTable
+	urlSigKeys  riakURLSigKeyTable
+	cfg         RiakConfig
+	cluster     *riak.Cluster
+}
+
+// String returns a high level overview of the backend and its keys
+func (rb *RiakBackend) String() string {
+	data := fmt.Sprintf("Riak server %v@%v:%v\n", rb.cfg.User, rb.cfg.Host, rb.cfg.Port)
+	data += fmt.Sprintf("\tSSL Keys: %v\n", len(rb.sslKeys.Records))
+	data += fmt.Sprintf("\tDNSSec Keys: %v\n", len(rb.dnssecKeys.Records))
+	data += fmt.Sprintf("\tURI Keys: %v\n", len(rb.uriSignKeys.Records))
+	data += fmt.Sprintf("\tURL Keys: %v\n", len(rb.urlSigKeys.Records))
+	return data
+}
+
+// Name returns the name for this backend
+func (rb *RiakBackend) Name() string {
+	return "Riak"
+}
+
+// ReadConfig takes in a filename and will read it into the backends config
+func (rb *RiakBackend) ReadConfig(s string) error {
+	err := UnmarshalConfig(s, &rb.cfg)
+	if err != nil {
+		return err
+	}
+
+	if rb.cfg.TLSVersionRaw == "10" {
+		rb.cfg.TLSVersion = tls.VersionTLS10
+	} else if rb.cfg.TLSVersionRaw == "11" {
+		rb.cfg.TLSVersion = tls.VersionTLS11
+	} else if rb.cfg.TLSVersionRaw == "12" {
+		rb.cfg.TLSVersion = tls.VersionTLS12
+	} else if rb.cfg.TLSVersionRaw == "13" {
+		rb.cfg.TLSVersion = tls.VersionTLS13
+	}
+	return nil
+}
+
+// Insert takes the current keys and inserts them into the backend DB
+func (rb *RiakBackend) Insert() error {
+	if err := rb.sslKeys.insertKeys(rb.cluster); err != nil {
+		return err
+	}
+	if err := rb.dnssecKeys.insertKeys(rb.cluster); err != nil {
+		return err
+	}
+	if err := rb.urlSigKeys.insertKeys(rb.cluster); err != nil {
+		return err
+	}
+	if err := rb.uriSignKeys.insertKeys(rb.cluster); err != nil {
+		return err
+	}
+	return nil
+}
+
+// ValidateKey validates that the keys are valid (in most cases, certain fields are not null)
+func (rb *RiakBackend) ValidateKey() []string {
+	errors := []string{}
+	if errs := rb.sslKeys.validate(); errs != nil {
+		errors = append(errors, errs...)
+	}
+	if errs := rb.dnssecKeys.validate(); errs != nil {
+		errors = append(errors, errs...)
+	}
+	if errs := rb.uriSignKeys.validate(); errs != nil {
+		errors = append(errors, errs...)
+	}
+	if errs := rb.urlSigKeys.validate(); errs != nil {
+		errors = append(errors, errs...)
+	}
+
+	return errors
+}
+
+// SetSSLKeys takes in keys and converts & encrypts the data into the backends internal format
+func (rb *RiakBackend) SetSSLKeys(keys []SSLKey) error {
+	rb.sslKeys.fromGeneric(keys)
+	return nil
+}
+
+// SetDNSSecKeys takes in keys and converts & encrypts the data into the backends internal format
+func (rb *RiakBackend) SetDNSSecKeys(keys []DNSSecKey) error {
+	rb.dnssecKeys.fromGeneric(keys)
+	return nil
+}
+
+// SetURISignKeys takes in keys and converts & encrypts the data into the backends internal format
+func (rb *RiakBackend) SetURISignKeys(keys []URISignKey) error {
+	rb.uriSignKeys.fromGeneric(keys)
+	return nil
+}
+
+// SetURLSigKeys takes in keys and converts & encrypts the data into the backends internal format
+func (rb *RiakBackend) SetURLSigKeys(keys []URLSigKey) error {
+	rb.urlSigKeys.fromGeneric(keys)
+	return nil
+}
+
+// Start initiates the connection to the backend DB
+func (rb *RiakBackend) Start() error {
+	tlsConfig := &tls.Config{
+		InsecureSkipVerify: !rb.cfg.Insecure,
+		MaxVersion:         rb.cfg.TLSVersion,
+	}
+	auth := &riak.AuthOptions{
+		User:      rb.cfg.User,
+		Password:  rb.cfg.Password,
+		TlsConfig: tlsConfig,
+	}
+
+	cluster, err := getRiakCluster(rb.cfg, auth)
+	if err != nil {
+		return err
+	}
+	if err := cluster.Start(); err != nil {
+		return fmt.Errorf("unable to start riak cluster: %w", err)
+	}
+
+	rb.cluster = cluster
+	rb.sslKeys = riakSSLKeyTable{}
+	rb.dnssecKeys = riakDNSSecKeyTable{}
+	rb.urlSigKeys = riakURLSigKeyTable{}
+	rb.uriSignKeys = riakURISignKeyTable{}
+	return nil
+}
+
+// Close terminates the connection to the backend DB
+func (rb *RiakBackend) Close() error {
+	if err := rb.cluster.Stop(); err != nil {
+		return err
+	}
+	return nil
+}
+
+// Ping checks the connection to the backend DB
+func (rb *RiakBackend) Ping() error {
+	return ping(rb.cluster)
+}
+
+// GetSSLKeys converts the backends internal key representation into the common representation (SSLKey)
+func (rb *RiakBackend) GetSSLKeys() ([]SSLKey, error) {
+	return rb.sslKeys.toGeneric(), nil
+}
+
+// GetDNSSecKeys converts the backends internal key representation into the common representation (DNSSecKey)
+func (rb *RiakBackend) GetDNSSecKeys() ([]DNSSecKey, error) {
+	return rb.dnssecKeys.toGeneric(), nil
+}
+
+// GetURISignKeys converts the pg internal key representation into the common representation (URISignKey)
+func (rb *RiakBackend) GetURISignKeys() ([]URISignKey, error) {
+	return rb.uriSignKeys.toGeneric(), nil
+}
+
+// GetURLSigKeys converts the backends internal key representation into the common representation (URLSigKey)
+func (rb *RiakBackend) GetURLSigKeys() ([]URLSigKey, error) {
+	return rb.urlSigKeys.toGeneric(), nil
+}
+
+// Fetch gets all of the keys from the backend DB
+func (rb *RiakBackend) Fetch() error {
+	if err := rb.sslKeys.gatherKeys(rb.cluster); err != nil {
+		return err
+	}
+	if err := rb.dnssecKeys.gatherKeys(rb.cluster); err != nil {
+		return err
+	}
+	if err := rb.urlSigKeys.gatherKeys(rb.cluster); err != nil {
+		return err
+	}
+	if err := rb.uriSignKeys.gatherKeys(rb.cluster); err != nil {
+		return err
+	}
+
+	return nil
+}
+
+type riakSSLKeyRecord struct {
+	tc.DeliveryServiceSSLKeys
+}
+type riakSSLKeyTable struct {
+	Records []riakSSLKeyRecord
+}
+
+func (tbl *riakSSLKeyTable) gatherKeys(cluster *riak.Cluster) error {
+	searchDocs, err := search(cluster, INDEX_SSL, "cdn:*", fmt.Sprintf("%v:*latest", SCHEMA_RIAK_KEY), 1000, SCHEMA_SSL_FIELDS[:])
+	if err != nil {
+		return fmt.Errorf("RiakSSLKey gatherKeys: %w", err)
+	}
+
+	tbl.Records = make([]riakSSLKeyRecord, len(searchDocs))
+	for i, doc := range searchDocs {
+		objs, err := getObject(cluster, doc.Bucket, doc.Key)
+		if err != nil {
+			return err
+		}
+		if len(objs) < 1 {
+			return fmt.Errorf("RiakSSLKey gatherKeys unable to find any objects with key %v and bucket %v, but search results were returned", doc.Key, doc.Bucket)
+		}
+		if len(objs) > 1 {
+			return fmt.Errorf("RiakSSLKey gatherKeys more than 1 ssl key record found %v\n", len(objs))
+		}
+		var obj tc.DeliveryServiceSSLKeys
+		if err = json.Unmarshal(objs[0].Value, &obj); err != nil {
+			return fmt.Errorf("RiakSSLKey gatherKeys unable to unmarshal object into tc.DeliveryServiceSSLKeys: %w", err)
+		}
+		tbl.Records[i] = riakSSLKeyRecord{
+			DeliveryServiceSSLKeys: obj,
+		}
+	}
+	return nil
+}
+func (tbl *riakSSLKeyTable) toGeneric() []SSLKey {
+	keys := make([]SSLKey, len(tbl.Records))
+
+	for i, record := range tbl.Records {
+		keys[i] = SSLKey{
+			DeliveryServiceSSLKeys: record.DeliveryServiceSSLKeys,
+		}
+	}
+
+	return keys
+}
+func (tbl *riakSSLKeyTable) fromGeneric(keys []SSLKey) {
+	tbl.Records = make([]riakSSLKeyRecord, len(keys))
+
+	for i, record := range keys {
+		tbl.Records[i] = riakSSLKeyRecord{
+			DeliveryServiceSSLKeys: record.DeliveryServiceSSLKeys,
+		}
+	}
+}
+func (tbl *riakSSLKeyTable) insertKeys(cluster *riak.Cluster) error {
+	for _, record := range tbl.Records {
+		objBytes, err := json.Marshal(record.DeliveryServiceSSLKeys)
+		if err != nil {
+			return fmt.Errorf("RiakSSLKey insertKeys failed to unmarshal keys: %w", err)
+		}
+		err = setObject(cluster, makeRiakObject(objBytes, record.DeliveryService+"-"+record.Version.String()), BUCKET_SSL)
+		if err != nil {
+			return fmt.Errorf("RiakSSLKey insertKeys: %w", err)
+		}
+		err = setObject(cluster, makeRiakObject(objBytes, record.DeliveryService+"-latest"), BUCKET_SSL)
+		if err != nil {
+			return fmt.Errorf("RiakSSLKey insertKeys: %w", err)
+		}
+	}
+	return nil
+}
+func (tbl *riakSSLKeyTable) validate() []string {
+	errs := []string{}
+	for i, record := range tbl.Records {
+		if record.DeliveryService == "" {
+			errs = append(errs, fmt.Sprintf("SSL Key #%v: Delivery Service is blank!", i))
+		}
+		if record.CDN == "" {
+			errs = append(errs, fmt.Sprintf("SSL Key #%v: CDN is blank!", i))
+		}
+		if record.Version.String() == "" {
+			errs = append(errs, fmt.Sprintf("SSL Key #%v: Version is blank!", i))
+		}
+	}
+	return errs
+}
+
+type riakDNSSecKeyRecord struct {
+	CDN string
+	Key tc.DNSSECKeysRiak
+}
+type riakDNSSecKeyTable struct {
+	Records []riakDNSSecKeyRecord
+}
+
+func (tbl *riakDNSSecKeyTable) gatherKeys(cluster *riak.Cluster) error {
+	tbl.Records = []riakDNSSecKeyRecord{}
+	objs, err := getObjects(cluster, BUCKET_DNSSEC)
+	if err != nil {
+		return fmt.Errorf("RiakDNSSecKey gatherKeys: %w", err)
+	}
+	for _, obj := range objs {
+		key := tc.DNSSECKeysRiak{}
+		if err := json.Unmarshal(obj.Value, &key); err != nil {
+			return fmt.Errorf("RiakDNSSecKey gatherKeys unable to unmarshal object to tc.DNSSECKeysRiak: %w", err)
+		}
+		tbl.Records = append(tbl.Records, riakDNSSecKeyRecord{
+			CDN: obj.Key,
+			Key: key,
+		})
+	}
+	return nil
+}
+func (tbl *riakDNSSecKeyTable) toGeneric() []DNSSecKey {
+	keys := make([]DNSSecKey, len(tbl.Records))
+
+	for i, record := range tbl.Records {
+		keys[i] = DNSSecKey{
+			CDN:                    record.CDN,
+			DNSSECKeysTrafficVault: tc.DNSSECKeysTrafficVault(record.Key),
+		}
+	}
+
+	return keys
+}
+func (tbl *riakDNSSecKeyTable) fromGeneric(keys []DNSSecKey) {
+	tbl.Records = make([]riakDNSSecKeyRecord, len(keys))
+
+	for i, record := range keys {
+		tbl.Records[i] = riakDNSSecKeyRecord{
+			CDN: record.CDN,
+			Key: tc.DNSSECKeysRiak(record.DNSSECKeysTrafficVault),
+		}
+	}
+}
+func (tbl *riakDNSSecKeyTable) insertKeys(cluster *riak.Cluster) error {
+	for _, record := range tbl.Records {
+		objBytes, err := json.Marshal(record.Key)
+		if err != nil {
+			return fmt.Errorf("RiakDNSSecKey insertKeys error marshalling keys: %w", err)
+		}
+
+		err = setObject(cluster, makeRiakObject(objBytes, record.CDN), BUCKET_DNSSEC)
+		if err != nil {
+			return fmt.Errorf("RiakDNSSecKey insertKeys: %w", err)
+		}
+	}
+	return nil
+}
+func (tbl *riakDNSSecKeyTable) validate() []string {
+	errs := []string{}
+	for i, record := range tbl.Records {
+		if record.CDN == "" {
+			errs = append(errs, fmt.Sprintf("DNSSec Key #%v: CDN is blank!", i))
+		}
+	}
+	return errs
+}
+
+type riakURLSigKeyRecord struct {
+	Key             tc.URLSigKeys
+	DeliveryService string
+}
+type riakURLSigKeyTable struct {
+	Records []riakURLSigKeyRecord
+}
+
+func (tbl *riakURLSigKeyTable) gatherKeys(cluster *riak.Cluster) error {
+	tbl.Records = []riakURLSigKeyRecord{}
+	objs, err := getObjects(cluster, BUCKET_URL_SIG)
+	if err != nil {
+		return fmt.Errorf("RiakURLSigKey gatherKeys: %w", err)
+	}
+	for _, obj := range objs {
+		key := tc.URLSigKeys{}
+		if err := json.Unmarshal(obj.Value, &key); err != nil {
+			return fmt.Errorf("RiakURLSigKey gatherKeys unable to unamrshal object into tc.URLSigKeys: %w", err)
+		}
+		tbl.Records = append(tbl.Records, riakURLSigKeyRecord{
+			DeliveryService: obj.Key,
+			Key:             key,
+		})
+	}
+	return nil
+}
+func (tbl *riakURLSigKeyTable) toGeneric() []URLSigKey {
+	keys := make([]URLSigKey, len(tbl.Records))
+
+	for i, record := range tbl.Records {
+		keys[i] = URLSigKey{
+			URLSigKeys:      record.Key,
+			DeliveryService: record.DeliveryService,
+		}
+	}
+
+	return keys
+}
+func (tbl *riakURLSigKeyTable) fromGeneric(keys []URLSigKey) {
+	tbl.Records = make([]riakURLSigKeyRecord, len(keys))
+	for i, key := range keys {
+		tbl.Records[i] = riakURLSigKeyRecord{
+			Key:             key.URLSigKeys,
+			DeliveryService: key.DeliveryService,
+		}
+	}
+}
+func (tbl *riakURLSigKeyTable) insertKeys(cluster *riak.Cluster) error {
+	for _, record := range tbl.Records {
+		objBytes, err := json.Marshal(record.Key)
+		if err != nil {
+			return fmt.Errorf("RiakURLSigKey insertKeys unable to marshal keys: %w", err)
+		}
+
+		err = setObject(cluster, makeRiakObject(objBytes, "url_sig_"+record.DeliveryService+".config"), BUCKET_URL_SIG)
+		if err != nil {
+			return fmt.Errorf("RiakURLSigKey insertKeys: %w", err)
+		}
+	}
+	return nil
+}
+func (tbl *riakURLSigKeyTable) validate() []string {
+	errs := []string{}
+	for i, record := range tbl.Records {
+		if record.DeliveryService == "" {
+			errs = append(errs, fmt.Sprintf("URL Key #%v: Delivery Service is blank!", i))
+		}
+	}
+	return errs
+}
+
+type riakURISignKeyRecord struct {
+	Key             map[string]tc.URISignerKeyset
+	DeliveryService string
+}
+type riakURISignKeyTable struct {
+	Records []riakURISignKeyRecord
+}
+
+func (tbl *riakURISignKeyTable) gatherKeys(cluster *riak.Cluster) error {
+	tbl.Records = []riakURISignKeyRecord{}
+	objs, err := getObjects(cluster, BUCKET_URI_SIG)
+	if err != nil {
+		return fmt.Errorf("RiakURISignKey gatherKeys: %w", err)
+	}
+	for _, obj := range objs {
+		key := map[string]tc.URISignerKeyset{}
+		if err := json.Unmarshal(obj.Value, &key); err != nil {
+			return fmt.Errorf("RiakURISignKey gatherKeys unable to unmarshal object into map[string]tc.URISignerKeySet: %w", err)
+		}
+
+		tbl.Records = append(tbl.Records, riakURISignKeyRecord{
+			DeliveryService: obj.Key,
+			Key:             key,
+		})
+	}
+	return nil
+}
+func (tbl *riakURISignKeyTable) toGeneric() []URISignKey {
+	keys := make([]URISignKey, len(tbl.Records))
+
+	for i, record := range tbl.Records {
+		keys[i] = URISignKey{
+			DeliveryService: record.DeliveryService,
+			Keys:            record.Key,
+		}
+	}
+
+	return keys
+}
+func (tbl *riakURISignKeyTable) fromGeneric(keys []URISignKey) {
+	tbl.Records = make([]riakURISignKeyRecord, len(keys))
+
+	for i, record := range keys {
+		tbl.Records[i] = riakURISignKeyRecord{
+			Key:             record.Keys,
+			DeliveryService: record.DeliveryService,
+		}
+	}
+}
+func (tbl *riakURISignKeyTable) insertKeys(cluster *riak.Cluster) error {
+	for _, record := range tbl.Records {
+		objBytes, err := json.Marshal(record.Key)
+		if err != nil {
+			return fmt.Errorf("RiakURISignKey insertKeys: unable to marshal key: %w", err)

Review comment:
       this error message should probably include the `record.DeliveryService` for troubleshooting

##########
File path: tools/traffic_vault_migrate/riak.go
##########
@@ -0,0 +1,746 @@
+package main
+
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+import (
+	"crypto/tls"
+	"encoding/json"
+	"errors"
+	"fmt"
+	"time"
+
+	"github.com/apache/trafficcontrol/lib/go-rfc"
+	"github.com/apache/trafficcontrol/lib/go-tc"
+
+	"github.com/basho/riak-go-client"
+)
+
+const (
+	BUCKET_SSL     = "ssl"
+	BUCKET_DNSSEC  = "dnssec"
+	BUCKET_URL_SIG = "url_sig_keys"
+	BUCKET_URI_SIG = "cdn_uri_sig_keys"
+
+	INDEX_SSL = "sslkeys"
+
+	SCHEMA_RIAK_KEY    = "_yz_rk"
+	SCHEMA_RIAK_BUCKET = "_yz_rb"
+)
+
+var (
+	SCHEMA_SSL_FIELDS = [...]string{SCHEMA_RIAK_KEY, SCHEMA_RIAK_BUCKET}
+)
+
+// RiakConfig  represents the configuration options available to the Riak backend
+type RiakConfig struct {
+	Host          string `json:"host"`
+	Port          string `json:"port"`
+	User          string `json:"user"`
+	Password      string `json:"password"`
+	Insecure      bool   `json:"insecure"`
+	TLSVersionRaw string `json:"tlsVersion"`
+	TLSVersion    uint16
+}
+
+// RiakBackend is the Riak implementation of TVBackend
+type RiakBackend struct {
+	sslKeys     riakSSLKeyTable
+	dnssecKeys  riakDNSSecKeyTable
+	uriSignKeys riakURISignKeyTable
+	urlSigKeys  riakURLSigKeyTable
+	cfg         RiakConfig
+	cluster     *riak.Cluster
+}
+
+// String returns a high level overview of the backend and its keys
+func (rb *RiakBackend) String() string {
+	data := fmt.Sprintf("Riak server %v@%v:%v\n", rb.cfg.User, rb.cfg.Host, rb.cfg.Port)
+	data += fmt.Sprintf("\tSSL Keys: %v\n", len(rb.sslKeys.Records))
+	data += fmt.Sprintf("\tDNSSec Keys: %v\n", len(rb.dnssecKeys.Records))
+	data += fmt.Sprintf("\tURI Keys: %v\n", len(rb.uriSignKeys.Records))
+	data += fmt.Sprintf("\tURL Keys: %v\n", len(rb.urlSigKeys.Records))
+	return data
+}
+
+// Name returns the name for this backend
+func (rb *RiakBackend) Name() string {
+	return "Riak"
+}
+
+// ReadConfig takes in a filename and will read it into the backends config
+func (rb *RiakBackend) ReadConfig(s string) error {
+	err := UnmarshalConfig(s, &rb.cfg)
+	if err != nil {
+		return err
+	}
+
+	if rb.cfg.TLSVersionRaw == "10" {
+		rb.cfg.TLSVersion = tls.VersionTLS10
+	} else if rb.cfg.TLSVersionRaw == "11" {
+		rb.cfg.TLSVersion = tls.VersionTLS11
+	} else if rb.cfg.TLSVersionRaw == "12" {
+		rb.cfg.TLSVersion = tls.VersionTLS12
+	} else if rb.cfg.TLSVersionRaw == "13" {
+		rb.cfg.TLSVersion = tls.VersionTLS13
+	}
+	return nil
+}
+
+// Insert takes the current keys and inserts them into the backend DB
+func (rb *RiakBackend) Insert() error {
+	if err := rb.sslKeys.insertKeys(rb.cluster); err != nil {
+		return err
+	}
+	if err := rb.dnssecKeys.insertKeys(rb.cluster); err != nil {
+		return err
+	}
+	if err := rb.urlSigKeys.insertKeys(rb.cluster); err != nil {
+		return err
+	}
+	if err := rb.uriSignKeys.insertKeys(rb.cluster); err != nil {
+		return err
+	}
+	return nil
+}
+
+// ValidateKey validates that the keys are valid (in most cases, certain fields are not null)
+func (rb *RiakBackend) ValidateKey() []string {
+	errors := []string{}
+	if errs := rb.sslKeys.validate(); errs != nil {
+		errors = append(errors, errs...)
+	}
+	if errs := rb.dnssecKeys.validate(); errs != nil {
+		errors = append(errors, errs...)
+	}
+	if errs := rb.uriSignKeys.validate(); errs != nil {
+		errors = append(errors, errs...)
+	}
+	if errs := rb.urlSigKeys.validate(); errs != nil {
+		errors = append(errors, errs...)
+	}
+
+	return errors
+}
+
+// SetSSLKeys takes in keys and converts & encrypts the data into the backends internal format
+func (rb *RiakBackend) SetSSLKeys(keys []SSLKey) error {
+	rb.sslKeys.fromGeneric(keys)
+	return nil
+}
+
+// SetDNSSecKeys takes in keys and converts & encrypts the data into the backends internal format
+func (rb *RiakBackend) SetDNSSecKeys(keys []DNSSecKey) error {
+	rb.dnssecKeys.fromGeneric(keys)
+	return nil
+}
+
+// SetURISignKeys takes in keys and converts & encrypts the data into the backends internal format
+func (rb *RiakBackend) SetURISignKeys(keys []URISignKey) error {
+	rb.uriSignKeys.fromGeneric(keys)
+	return nil
+}
+
+// SetURLSigKeys takes in keys and converts & encrypts the data into the backends internal format
+func (rb *RiakBackend) SetURLSigKeys(keys []URLSigKey) error {
+	rb.urlSigKeys.fromGeneric(keys)
+	return nil
+}
+
+// Start initiates the connection to the backend DB
+func (rb *RiakBackend) Start() error {
+	tlsConfig := &tls.Config{
+		InsecureSkipVerify: !rb.cfg.Insecure,
+		MaxVersion:         rb.cfg.TLSVersion,
+	}
+	auth := &riak.AuthOptions{
+		User:      rb.cfg.User,
+		Password:  rb.cfg.Password,
+		TlsConfig: tlsConfig,
+	}
+
+	cluster, err := getRiakCluster(rb.cfg, auth)
+	if err != nil {
+		return err
+	}
+	if err := cluster.Start(); err != nil {
+		return fmt.Errorf("unable to start riak cluster: %w", err)
+	}
+
+	rb.cluster = cluster
+	rb.sslKeys = riakSSLKeyTable{}
+	rb.dnssecKeys = riakDNSSecKeyTable{}
+	rb.urlSigKeys = riakURLSigKeyTable{}
+	rb.uriSignKeys = riakURISignKeyTable{}
+	return nil
+}
+
+// Close terminates the connection to the backend DB
+func (rb *RiakBackend) Close() error {
+	if err := rb.cluster.Stop(); err != nil {
+		return err
+	}
+	return nil
+}
+
+// Ping checks the connection to the backend DB
+func (rb *RiakBackend) Ping() error {
+	return ping(rb.cluster)
+}
+
+// GetSSLKeys converts the backends internal key representation into the common representation (SSLKey)
+func (rb *RiakBackend) GetSSLKeys() ([]SSLKey, error) {
+	return rb.sslKeys.toGeneric(), nil
+}
+
+// GetDNSSecKeys converts the backends internal key representation into the common representation (DNSSecKey)
+func (rb *RiakBackend) GetDNSSecKeys() ([]DNSSecKey, error) {
+	return rb.dnssecKeys.toGeneric(), nil
+}
+
+// GetURISignKeys converts the pg internal key representation into the common representation (URISignKey)
+func (rb *RiakBackend) GetURISignKeys() ([]URISignKey, error) {
+	return rb.uriSignKeys.toGeneric(), nil
+}
+
+// GetURLSigKeys converts the backends internal key representation into the common representation (URLSigKey)
+func (rb *RiakBackend) GetURLSigKeys() ([]URLSigKey, error) {
+	return rb.urlSigKeys.toGeneric(), nil
+}
+
+// Fetch gets all of the keys from the backend DB
+func (rb *RiakBackend) Fetch() error {
+	if err := rb.sslKeys.gatherKeys(rb.cluster); err != nil {
+		return err
+	}
+	if err := rb.dnssecKeys.gatherKeys(rb.cluster); err != nil {
+		return err
+	}
+	if err := rb.urlSigKeys.gatherKeys(rb.cluster); err != nil {
+		return err
+	}
+	if err := rb.uriSignKeys.gatherKeys(rb.cluster); err != nil {
+		return err
+	}
+
+	return nil
+}
+
+type riakSSLKeyRecord struct {
+	tc.DeliveryServiceSSLKeys
+}
+type riakSSLKeyTable struct {
+	Records []riakSSLKeyRecord
+}
+
+func (tbl *riakSSLKeyTable) gatherKeys(cluster *riak.Cluster) error {
+	searchDocs, err := search(cluster, INDEX_SSL, "cdn:*", fmt.Sprintf("%v:*latest", SCHEMA_RIAK_KEY), 1000, SCHEMA_SSL_FIELDS[:])
+	if err != nil {
+		return fmt.Errorf("RiakSSLKey gatherKeys: %w", err)
+	}
+
+	tbl.Records = make([]riakSSLKeyRecord, len(searchDocs))
+	for i, doc := range searchDocs {
+		objs, err := getObject(cluster, doc.Bucket, doc.Key)
+		if err != nil {
+			return err
+		}
+		if len(objs) < 1 {
+			return fmt.Errorf("RiakSSLKey gatherKeys unable to find any objects with key %v and bucket %v, but search results were returned", doc.Key, doc.Bucket)
+		}
+		if len(objs) > 1 {
+			return fmt.Errorf("RiakSSLKey gatherKeys more than 1 ssl key record found %v\n", len(objs))
+		}
+		var obj tc.DeliveryServiceSSLKeys
+		if err = json.Unmarshal(objs[0].Value, &obj); err != nil {
+			return fmt.Errorf("RiakSSLKey gatherKeys unable to unmarshal object into tc.DeliveryServiceSSLKeys: %w", err)
+		}
+		tbl.Records[i] = riakSSLKeyRecord{
+			DeliveryServiceSSLKeys: obj,
+		}
+	}
+	return nil
+}
+func (tbl *riakSSLKeyTable) toGeneric() []SSLKey {
+	keys := make([]SSLKey, len(tbl.Records))
+
+	for i, record := range tbl.Records {
+		keys[i] = SSLKey{
+			DeliveryServiceSSLKeys: record.DeliveryServiceSSLKeys,
+		}
+	}
+
+	return keys
+}
+func (tbl *riakSSLKeyTable) fromGeneric(keys []SSLKey) {
+	tbl.Records = make([]riakSSLKeyRecord, len(keys))
+
+	for i, record := range keys {
+		tbl.Records[i] = riakSSLKeyRecord{
+			DeliveryServiceSSLKeys: record.DeliveryServiceSSLKeys,
+		}
+	}
+}
+func (tbl *riakSSLKeyTable) insertKeys(cluster *riak.Cluster) error {
+	for _, record := range tbl.Records {
+		objBytes, err := json.Marshal(record.DeliveryServiceSSLKeys)
+		if err != nil {
+			return fmt.Errorf("RiakSSLKey insertKeys failed to unmarshal keys: %w", err)
+		}
+		err = setObject(cluster, makeRiakObject(objBytes, record.DeliveryService+"-"+record.Version.String()), BUCKET_SSL)
+		if err != nil {
+			return fmt.Errorf("RiakSSLKey insertKeys: %w", err)
+		}
+		err = setObject(cluster, makeRiakObject(objBytes, record.DeliveryService+"-latest"), BUCKET_SSL)
+		if err != nil {
+			return fmt.Errorf("RiakSSLKey insertKeys: %w", err)
+		}
+	}
+	return nil
+}
+func (tbl *riakSSLKeyTable) validate() []string {
+	errs := []string{}
+	for i, record := range tbl.Records {
+		if record.DeliveryService == "" {
+			errs = append(errs, fmt.Sprintf("SSL Key #%v: Delivery Service is blank!", i))
+		}
+		if record.CDN == "" {
+			errs = append(errs, fmt.Sprintf("SSL Key #%v: CDN is blank!", i))
+		}
+		if record.Version.String() == "" {
+			errs = append(errs, fmt.Sprintf("SSL Key #%v: Version is blank!", i))
+		}
+	}
+	return errs
+}
+
+type riakDNSSecKeyRecord struct {
+	CDN string
+	Key tc.DNSSECKeysRiak
+}
+type riakDNSSecKeyTable struct {
+	Records []riakDNSSecKeyRecord
+}
+
+func (tbl *riakDNSSecKeyTable) gatherKeys(cluster *riak.Cluster) error {
+	tbl.Records = []riakDNSSecKeyRecord{}
+	objs, err := getObjects(cluster, BUCKET_DNSSEC)
+	if err != nil {
+		return fmt.Errorf("RiakDNSSecKey gatherKeys: %w", err)
+	}
+	for _, obj := range objs {
+		key := tc.DNSSECKeysRiak{}
+		if err := json.Unmarshal(obj.Value, &key); err != nil {
+			return fmt.Errorf("RiakDNSSecKey gatherKeys unable to unmarshal object to tc.DNSSECKeysRiak: %w", err)
+		}
+		tbl.Records = append(tbl.Records, riakDNSSecKeyRecord{
+			CDN: obj.Key,
+			Key: key,
+		})
+	}
+	return nil
+}
+func (tbl *riakDNSSecKeyTable) toGeneric() []DNSSecKey {
+	keys := make([]DNSSecKey, len(tbl.Records))
+
+	for i, record := range tbl.Records {
+		keys[i] = DNSSecKey{
+			CDN:                    record.CDN,
+			DNSSECKeysTrafficVault: tc.DNSSECKeysTrafficVault(record.Key),
+		}
+	}
+
+	return keys
+}
+func (tbl *riakDNSSecKeyTable) fromGeneric(keys []DNSSecKey) {
+	tbl.Records = make([]riakDNSSecKeyRecord, len(keys))
+
+	for i, record := range keys {
+		tbl.Records[i] = riakDNSSecKeyRecord{
+			CDN: record.CDN,
+			Key: tc.DNSSECKeysRiak(record.DNSSECKeysTrafficVault),
+		}
+	}
+}
+func (tbl *riakDNSSecKeyTable) insertKeys(cluster *riak.Cluster) error {
+	for _, record := range tbl.Records {
+		objBytes, err := json.Marshal(record.Key)
+		if err != nil {
+			return fmt.Errorf("RiakDNSSecKey insertKeys error marshalling keys: %w", err)
+		}
+
+		err = setObject(cluster, makeRiakObject(objBytes, record.CDN), BUCKET_DNSSEC)
+		if err != nil {
+			return fmt.Errorf("RiakDNSSecKey insertKeys: %w", err)
+		}
+	}
+	return nil
+}
+func (tbl *riakDNSSecKeyTable) validate() []string {
+	errs := []string{}
+	for i, record := range tbl.Records {
+		if record.CDN == "" {
+			errs = append(errs, fmt.Sprintf("DNSSec Key #%v: CDN is blank!", i))
+		}
+	}
+	return errs
+}
+
+type riakURLSigKeyRecord struct {
+	Key             tc.URLSigKeys
+	DeliveryService string
+}
+type riakURLSigKeyTable struct {
+	Records []riakURLSigKeyRecord
+}
+
+func (tbl *riakURLSigKeyTable) gatherKeys(cluster *riak.Cluster) error {
+	tbl.Records = []riakURLSigKeyRecord{}
+	objs, err := getObjects(cluster, BUCKET_URL_SIG)
+	if err != nil {
+		return fmt.Errorf("RiakURLSigKey gatherKeys: %w", err)
+	}
+	for _, obj := range objs {
+		key := tc.URLSigKeys{}
+		if err := json.Unmarshal(obj.Value, &key); err != nil {
+			return fmt.Errorf("RiakURLSigKey gatherKeys unable to unamrshal object into tc.URLSigKeys: %w", err)
+		}
+		tbl.Records = append(tbl.Records, riakURLSigKeyRecord{
+			DeliveryService: obj.Key,
+			Key:             key,
+		})
+	}
+	return nil
+}
+func (tbl *riakURLSigKeyTable) toGeneric() []URLSigKey {
+	keys := make([]URLSigKey, len(tbl.Records))
+
+	for i, record := range tbl.Records {
+		keys[i] = URLSigKey{
+			URLSigKeys:      record.Key,
+			DeliveryService: record.DeliveryService,
+		}
+	}
+
+	return keys
+}
+func (tbl *riakURLSigKeyTable) fromGeneric(keys []URLSigKey) {
+	tbl.Records = make([]riakURLSigKeyRecord, len(keys))
+	for i, key := range keys {
+		tbl.Records[i] = riakURLSigKeyRecord{
+			Key:             key.URLSigKeys,
+			DeliveryService: key.DeliveryService,
+		}
+	}
+}
+func (tbl *riakURLSigKeyTable) insertKeys(cluster *riak.Cluster) error {
+	for _, record := range tbl.Records {
+		objBytes, err := json.Marshal(record.Key)
+		if err != nil {
+			return fmt.Errorf("RiakURLSigKey insertKeys unable to marshal keys: %w", err)
+		}
+
+		err = setObject(cluster, makeRiakObject(objBytes, "url_sig_"+record.DeliveryService+".config"), BUCKET_URL_SIG)
+		if err != nil {
+			return fmt.Errorf("RiakURLSigKey insertKeys: %w", err)
+		}
+	}
+	return nil
+}
+func (tbl *riakURLSigKeyTable) validate() []string {
+	errs := []string{}
+	for i, record := range tbl.Records {
+		if record.DeliveryService == "" {
+			errs = append(errs, fmt.Sprintf("URL Key #%v: Delivery Service is blank!", i))
+		}
+	}
+	return errs
+}
+
+type riakURISignKeyRecord struct {
+	Key             map[string]tc.URISignerKeyset
+	DeliveryService string
+}
+type riakURISignKeyTable struct {
+	Records []riakURISignKeyRecord
+}
+
+func (tbl *riakURISignKeyTable) gatherKeys(cluster *riak.Cluster) error {
+	tbl.Records = []riakURISignKeyRecord{}
+	objs, err := getObjects(cluster, BUCKET_URI_SIG)
+	if err != nil {
+		return fmt.Errorf("RiakURISignKey gatherKeys: %w", err)
+	}
+	for _, obj := range objs {
+		key := map[string]tc.URISignerKeyset{}
+		if err := json.Unmarshal(obj.Value, &key); err != nil {
+			return fmt.Errorf("RiakURISignKey gatherKeys unable to unmarshal object into map[string]tc.URISignerKeySet: %w", err)
+		}
+
+		tbl.Records = append(tbl.Records, riakURISignKeyRecord{
+			DeliveryService: obj.Key,
+			Key:             key,
+		})
+	}
+	return nil
+}
+func (tbl *riakURISignKeyTable) toGeneric() []URISignKey {
+	keys := make([]URISignKey, len(tbl.Records))
+
+	for i, record := range tbl.Records {
+		keys[i] = URISignKey{
+			DeliveryService: record.DeliveryService,
+			Keys:            record.Key,
+		}
+	}
+
+	return keys
+}
+func (tbl *riakURISignKeyTable) fromGeneric(keys []URISignKey) {
+	tbl.Records = make([]riakURISignKeyRecord, len(keys))
+
+	for i, record := range keys {
+		tbl.Records[i] = riakURISignKeyRecord{
+			Key:             record.Keys,
+			DeliveryService: record.DeliveryService,
+		}
+	}
+}
+func (tbl *riakURISignKeyTable) insertKeys(cluster *riak.Cluster) error {
+	for _, record := range tbl.Records {
+		objBytes, err := json.Marshal(record.Key)
+		if err != nil {
+			return fmt.Errorf("RiakURISignKey insertKeys: unable to marshal key: %w", err)
+		}
+
+		err = setObject(cluster, makeRiakObject(objBytes, record.DeliveryService), BUCKET_URI_SIG)
+		if err != nil {
+			return fmt.Errorf("RiakURISignKey insertKeys: %w", err)
+		}
+	}
+	return nil
+}
+func (tbl *riakURISignKeyTable) validate() []string {
+	errs := []string{}
+	for i, record := range tbl.Records {
+		if record.DeliveryService == "" {
+			errs = append(errs, fmt.Sprintf("URI Key #%v: Delivery Service is blank!", i))
+		}
+	}
+	return errs
+}
+
+// Riak functions
+func makeRiakObject(data []byte, key string) *riak.Object {
+	return &riak.Object{
+		ContentType:     rfc.ApplicationJSON,
+		Charset:         "utf-8",
+		ContentEncoding: "utf-8",
+		Key:             key,
+		Value:           data,
+	}
+}
+func getObjects(cluster *riak.Cluster, bucket string) ([]*riak.Object, error) {
+	objs := []*riak.Object{}
+	keys, err := listKeys(cluster, bucket)
+	if err != nil {
+		return nil, err
+	}
+	for _, key := range keys {
+		objects, err := getObject(cluster, bucket, key)
+		if err != nil {
+			return nil, err
+		}
+		if len(objects) > 1 {
+			return nil, fmt.Errorf("Unexpected number of objects %v, ignoring\n", len(objects))
+		}
+
+		objs = append(objs, objects[0])
+	}
+
+	return objs, nil
+}
+func getObject(cluster *riak.Cluster, bucket string, key string) ([]*riak.Object, error) {
+	cmd, err := riak.NewFetchValueCommandBuilder().
+		WithBucket(bucket).
+		WithKey(key).
+		WithTimeout(time.Second * 60).
+		Build()
+	if err != nil {
+		return nil, fmt.Errorf("error building riak fetch value command: %w", err)
+	}
+
+	if err := cluster.Execute(cmd); err != nil {
+		return nil, fmt.Errorf("error executing riak fetch value command: %w", err)
+	}
+
+	fvc := cmd.(*riak.FetchValueCommand)
+	rsp := fvc.Response
+
+	if rsp.IsNotFound {
+		return nil, fmt.Errorf("errors executing riak fetch value command; key not found: %v:%v", bucket, key)

Review comment:
       nit: type-specific formatting instead of %v

##########
File path: tools/traffic_vault_migrate/riak.go
##########
@@ -0,0 +1,746 @@
+package main
+
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+import (
+	"crypto/tls"
+	"encoding/json"
+	"errors"
+	"fmt"
+	"time"
+
+	"github.com/apache/trafficcontrol/lib/go-rfc"
+	"github.com/apache/trafficcontrol/lib/go-tc"
+
+	"github.com/basho/riak-go-client"
+)
+
+const (
+	BUCKET_SSL     = "ssl"
+	BUCKET_DNSSEC  = "dnssec"
+	BUCKET_URL_SIG = "url_sig_keys"
+	BUCKET_URI_SIG = "cdn_uri_sig_keys"
+
+	INDEX_SSL = "sslkeys"
+
+	SCHEMA_RIAK_KEY    = "_yz_rk"
+	SCHEMA_RIAK_BUCKET = "_yz_rb"
+)
+
+var (
+	SCHEMA_SSL_FIELDS = [...]string{SCHEMA_RIAK_KEY, SCHEMA_RIAK_BUCKET}
+)
+
+// RiakConfig  represents the configuration options available to the Riak backend
+type RiakConfig struct {
+	Host          string `json:"host"`
+	Port          string `json:"port"`
+	User          string `json:"user"`
+	Password      string `json:"password"`
+	Insecure      bool   `json:"insecure"`
+	TLSVersionRaw string `json:"tlsVersion"`
+	TLSVersion    uint16
+}
+
+// RiakBackend is the Riak implementation of TVBackend
+type RiakBackend struct {
+	sslKeys     riakSSLKeyTable
+	dnssecKeys  riakDNSSecKeyTable
+	uriSignKeys riakURISignKeyTable

Review comment:
       Can we lengthen this name to `uriSigningKeys`?

##########
File path: tools/traffic_vault_migrate/postgres.go
##########
@@ -0,0 +1,743 @@
+package main
+
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+import (
+	"database/sql"
+	"encoding/base64"
+	"encoding/json"
+	"errors"
+	"fmt"
+	"log"
+	"strconv"
+	"strings"
+
+	"github.com/apache/trafficcontrol/lib/go-tc"
+	util "github.com/apache/trafficcontrol/lib/go-util"
+
+	_ "github.com/lib/pq"
+)
+
+// PGConfig represents the configuration options available to the PG backend
+type PGConfig struct {
+	Host     string `json:"host"`
+	Port     string `json:"port"`
+	User     string `json:"user"`
+	Password string `json:"password"`
+	SSLMode  string `json:"sslmode"`
+	Database string `json:"database"`
+	Key      string `json:"aesKey"`
+	AESKey   []byte
+}
+
+// PGBackend is the Postgres implementation of TVBackend
+type PGBackend struct {
+	sslKey pgSSLKeyTable
+	dnssec pgDNSSecTable
+	uri    pgURISignKeyTable
+	url    pgURLSigKeyTable
+	cfg    PGConfig
+	db     *sql.DB
+}
+
+// String returns a high level overview of the backend and its keys
+func (pg *PGBackend) String() string {
+	data := fmt.Sprintf("PG server %v@%v:%v\n", pg.cfg.User, pg.cfg.Host, pg.cfg.Port)
+	data += fmt.Sprintf("\tSSL Keys: %v\n", len(pg.sslKey.Records))
+	data += fmt.Sprintf("\tDNSSec Keys: %v\n", len(pg.dnssec.Records))
+	data += fmt.Sprintf("\tURI Keys: %v\n", len(pg.uri.Records))
+	data += fmt.Sprintf("\tURL Keys: %v\n", len(pg.url.Records))
+	return data
+}
+
+// Name returns the name for this backend
+func (pg *PGBackend) Name() string {
+	return "PG"
+}
+
+// ReadConfig takes in a filename and will read it into the backends config

Review comment:
       This seems like it takes a file's _contents_ (as a string) and unmarshals it into the config struct.

##########
File path: tools/traffic_vault_migrate/traffic_vault_migrate.go
##########
@@ -400,6 +356,49 @@ type TVBackend interface {
 	SetURLSigKeys([]URLSigKey) error
 }
 
+// Secrets contains every key to be migrated
+type Secrets struct {
+	sslkeys    []SSLKey
+	dnssecKeys []DNSSecKey
+	uriKeys    []URISignKey
+	urlKeys    []URLSigKey
+}
+
+func (s *Secrets) sort() {
+	sort.Slice(s.sslkeys[:], func(a, b int) bool {
+		return s.sslkeys[a].CDN < s.sslkeys[b].CDN ||
+			s.sslkeys[a].CDN == s.sslkeys[b].CDN && s.sslkeys[a].DeliveryService < s.sslkeys[b].DeliveryService
+	})
+	sort.Slice(s.dnssecKeys[:], func(a, b int) bool {
+		return s.dnssecKeys[a].CDN < s.dnssecKeys[b].CDN
+	})
+	sort.Slice(s.uriKeys[:], func(a, b int) bool {
+		return s.uriKeys[a].DeliveryService < s.uriKeys[b].DeliveryService
+	})
+	sort.Slice(s.urlKeys[:], func(a, b int) bool {
+		return s.urlKeys[a].DeliveryService < s.urlKeys[b].DeliveryService
+	})
+}
+func (s *Secrets) dump(directory string) {

Review comment:
       Since we have `dump`, should we also have a way to read a dump and write it to a backend? Would that be easy option to add? If so, it might be handy.

##########
File path: tools/traffic_vault_migrate/postgres.go
##########
@@ -0,0 +1,743 @@
+package main
+
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+import (
+	"database/sql"
+	"encoding/base64"
+	"encoding/json"
+	"errors"
+	"fmt"
+	"log"
+	"strconv"
+	"strings"
+
+	"github.com/apache/trafficcontrol/lib/go-tc"
+	util "github.com/apache/trafficcontrol/lib/go-util"
+
+	_ "github.com/lib/pq"
+)
+
+// PGConfig represents the configuration options available to the PG backend
+type PGConfig struct {
+	Host     string `json:"host"`
+	Port     string `json:"port"`
+	User     string `json:"user"`
+	Password string `json:"password"`
+	SSLMode  string `json:"sslmode"`
+	Database string `json:"database"`
+	Key      string `json:"aesKey"`
+	AESKey   []byte
+}
+
+// PGBackend is the Postgres implementation of TVBackend
+type PGBackend struct {
+	sslKey pgSSLKeyTable
+	dnssec pgDNSSecTable
+	uri    pgURISignKeyTable
+	url    pgURLSigKeyTable
+	cfg    PGConfig
+	db     *sql.DB
+}
+
+// String returns a high level overview of the backend and its keys
+func (pg *PGBackend) String() string {
+	data := fmt.Sprintf("PG server %v@%v:%v\n", pg.cfg.User, pg.cfg.Host, pg.cfg.Port)
+	data += fmt.Sprintf("\tSSL Keys: %v\n", len(pg.sslKey.Records))
+	data += fmt.Sprintf("\tDNSSec Keys: %v\n", len(pg.dnssec.Records))
+	data += fmt.Sprintf("\tURI Keys: %v\n", len(pg.uri.Records))
+	data += fmt.Sprintf("\tURL Keys: %v\n", len(pg.url.Records))
+	return data
+}
+
+// Name returns the name for this backend
+func (pg *PGBackend) Name() string {
+	return "PG"
+}
+
+// ReadConfig takes in a filename and will read it into the backends config
+func (pg *PGBackend) ReadConfig(s string) error {
+	err := UnmarshalConfig(s, &pg.cfg)
+	if err != nil {
+		return err
+	}
+
+	pg.cfg.AESKey, err = base64.StdEncoding.DecodeString(pg.cfg.Key)
+	if err != nil {
+		return fmt.Errorf("unable to decode PG AESKey: %w", err)
+	}
+	return nil

Review comment:
       similar to [this](https://github.com/apache/trafficcontrol/pull/5913/files#diff-743ace595fab07cb3646600fc01d9fcf4aa6a70bd1476d2712641513d5211605R162-R165), we should validate that the given AES key is a valid AES key

##########
File path: tools/traffic_vault_migrate/postgres.go
##########
@@ -0,0 +1,743 @@
+package main
+
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+import (
+	"database/sql"
+	"encoding/base64"
+	"encoding/json"
+	"errors"
+	"fmt"
+	"log"
+	"strconv"
+	"strings"
+
+	"github.com/apache/trafficcontrol/lib/go-tc"
+	util "github.com/apache/trafficcontrol/lib/go-util"
+
+	_ "github.com/lib/pq"
+)
+
+// PGConfig represents the configuration options available to the PG backend
+type PGConfig struct {
+	Host     string `json:"host"`
+	Port     string `json:"port"`
+	User     string `json:"user"`
+	Password string `json:"password"`
+	SSLMode  string `json:"sslmode"`
+	Database string `json:"database"`
+	Key      string `json:"aesKey"`
+	AESKey   []byte
+}
+
+// PGBackend is the Postgres implementation of TVBackend
+type PGBackend struct {
+	sslKey pgSSLKeyTable
+	dnssec pgDNSSecTable
+	uri    pgURISignKeyTable

Review comment:
       Since `uri` and `url` are so close yet are two different things, could we make them names slightly longer, e.g. `uriSigningKeys` and `urlSigKeys`, to help differentiate? The same would apply to the `String()` method below and any log messages generally.

##########
File path: tools/traffic_vault_migrate/postgres.go
##########
@@ -0,0 +1,743 @@
+package main
+
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+import (
+	"database/sql"
+	"encoding/base64"
+	"encoding/json"
+	"errors"
+	"fmt"
+	"log"
+	"strconv"
+	"strings"
+
+	"github.com/apache/trafficcontrol/lib/go-tc"
+	util "github.com/apache/trafficcontrol/lib/go-util"
+
+	_ "github.com/lib/pq"
+)
+
+// PGConfig represents the configuration options available to the PG backend
+type PGConfig struct {
+	Host     string `json:"host"`
+	Port     string `json:"port"`
+	User     string `json:"user"`
+	Password string `json:"password"`
+	SSLMode  string `json:"sslmode"`
+	Database string `json:"database"`
+	Key      string `json:"aesKey"`
+	AESKey   []byte
+}
+
+// PGBackend is the Postgres implementation of TVBackend
+type PGBackend struct {
+	sslKey pgSSLKeyTable
+	dnssec pgDNSSecTable
+	uri    pgURISignKeyTable
+	url    pgURLSigKeyTable
+	cfg    PGConfig
+	db     *sql.DB
+}
+
+// String returns a high level overview of the backend and its keys
+func (pg *PGBackend) String() string {
+	data := fmt.Sprintf("PG server %v@%v:%v\n", pg.cfg.User, pg.cfg.Host, pg.cfg.Port)
+	data += fmt.Sprintf("\tSSL Keys: %v\n", len(pg.sslKey.Records))
+	data += fmt.Sprintf("\tDNSSec Keys: %v\n", len(pg.dnssec.Records))
+	data += fmt.Sprintf("\tURI Keys: %v\n", len(pg.uri.Records))
+	data += fmt.Sprintf("\tURL Keys: %v\n", len(pg.url.Records))
+	return data
+}
+
+// Name returns the name for this backend
+func (pg *PGBackend) Name() string {
+	return "PG"
+}
+
+// ReadConfig takes in a filename and will read it into the backends config
+func (pg *PGBackend) ReadConfig(s string) error {
+	err := UnmarshalConfig(s, &pg.cfg)
+	if err != nil {
+		return err
+	}
+
+	pg.cfg.AESKey, err = base64.StdEncoding.DecodeString(pg.cfg.Key)
+	if err != nil {
+		return fmt.Errorf("unable to decode PG AESKey: %w", err)
+	}
+	return nil
+}
+
+// Insert takes the current keys and inserts them into the backend DB
+func (pg *PGBackend) Insert() error {
+	if err := pg.sslKey.insertKeys(pg.db); err != nil {
+		return err
+	}
+	if err := pg.dnssec.insertKeys(pg.db); err != nil {
+		return err
+	}
+	if err := pg.url.insertKeys(pg.db); err != nil {
+		return err
+	}
+	if err := pg.uri.insertKeys(pg.db); err != nil {
+		return err
+	}
+	return nil
+}
+
+// Start initiates the connection to the backend DB
+func (pg *PGBackend) Start() error {
+	sqlStr := fmt.Sprintf("postgres://%s:%s@%s:%s/%s?sslmode=%s", pg.cfg.User, pg.cfg.Password, pg.cfg.Host, pg.cfg.Port, pg.cfg.Database, pg.cfg.SSLMode)
+	db, err := sql.Open("postgres", sqlStr)
+	if err != nil {
+		return fmt.Errorf("unable to start PG client: %w", err)
+	}
+
+	pg.db = db
+	pg.sslKey = pgSSLKeyTable{}
+	pg.dnssec = pgDNSSecTable{}
+	pg.url = pgURLSigKeyTable{}
+	pg.uri = pgURISignKeyTable{}
+
+	return nil
+}
+
+// ValidateKey validates that the keys are valid (in most cases, certain fields are not null)
+func (pg *PGBackend) ValidateKey() []string {
+	var errors []string
+	if errs := pg.sslKey.validate(); errs != nil {
+		errors = append(errors, errs...)
+	}
+	if errs := pg.dnssec.validate(); errs != nil {
+		errors = append(errors, errs...)
+	}
+	if errs := pg.uri.validate(); errs != nil {
+		errors = append(errors, errs...)
+	}
+	if errs := pg.url.validate(); errs != nil {
+		errors = append(errors, errs...)
+	}
+	return errors
+}
+
+// Close terminates the connection to the backend DB
+func (pg *PGBackend) Close() error {
+	return pg.db.Close()
+}
+
+// Ping checks the connection to the backend DB
+func (pg *PGBackend) Ping() error {
+	return pg.db.Ping()
+}
+
+// Fetch gets all of the keys from the backend DB
+func (pg *PGBackend) Fetch() error {
+	if err := pg.sslKey.gatherKeys(pg.db); err != nil {
+		return err
+	}
+
+	if err := pg.dnssec.gatherKeys(pg.db); err != nil {
+		return err
+	}
+
+	if err := pg.url.gatherKeys(pg.db); err != nil {
+		return err
+	}
+
+	if err := pg.uri.gatherKeys(pg.db); err != nil {
+		return err
+	}
+
+	return nil
+}
+
+// GetSSLKeys converts the backends internal key representation into the common representation (SSLKey)
+func (pg *PGBackend) GetSSLKeys() ([]SSLKey, error) {
+	if err := pg.sslKey.decrypt(pg.cfg.AESKey); err != nil {
+		return nil, err
+	}
+	return pg.sslKey.toGeneric(), nil
+}
+
+// SetSSLKeys takes in keys and converts & encrypts the data into the backends internal format
+func (pg *PGBackend) SetSSLKeys(keys []SSLKey) error {
+	pg.sslKey.fromGeneric(keys)
+	return pg.sslKey.encrypt(pg.cfg.AESKey)
+}
+
+// GetDNSSecKeys converts the backends internal key representation into the common representation (DNSSecKey)
+func (pg *PGBackend) GetDNSSecKeys() ([]DNSSecKey, error) {
+	if err := pg.dnssec.decrypt(pg.cfg.AESKey); err != nil {
+		return nil, err
+	}
+	return pg.dnssec.toGeneric(), nil
+}
+
+// SetDNSSecKeys takes in keys and converts & encrypts the data into the backends internal format
+func (pg *PGBackend) SetDNSSecKeys(keys []DNSSecKey) error {
+	pg.dnssec.fromGeneric(keys)
+	return pg.dnssec.encrypt(pg.cfg.AESKey)
+}
+
+// GetURISignKeys converts the pg internal key representation into the common representation (URISignKey)
+func (pg *PGBackend) GetURISignKeys() ([]URISignKey, error) {
+	if err := pg.uri.decrypt(pg.cfg.AESKey); err != nil {
+		return nil, err
+	}
+	return pg.uri.toGeneric(), nil
+}
+
+// SetURISignKeys takes in keys and converts & encrypts the data into the backends internal format
+func (pg *PGBackend) SetURISignKeys(keys []URISignKey) error {
+	pg.uri.fromGeneric(keys)
+	return pg.uri.encrypt(pg.cfg.AESKey)
+}
+
+// GetURLSigKeys converts the backends internal key representation into the common representation (URLSigKey)
+func (pg *PGBackend) GetURLSigKeys() ([]URLSigKey, error) {
+	if err := pg.url.decrypt(pg.cfg.AESKey); err != nil {
+		return nil, err
+	}
+	return pg.url.toGeneric(), nil
+}
+
+// SetURLSigKeys takes in keys and converts & encrypts the data into the backends internal format
+func (pg *PGBackend) SetURLSigKeys(keys []URLSigKey) error {
+	pg.url.fromGeneric(keys)
+	return pg.url.encrypt(pg.cfg.AESKey)
+}
+
+type pgCommonRecord struct {
+	DataEncrypted []byte
+}
+
+type pgDNSSecRecord struct {
+	Key tc.DNSSECKeysTrafficVault
+	CDN string
+	pgCommonRecord
+}
+type pgDNSSecTable struct {
+	Records []pgDNSSecRecord
+}
+
+func (tbl *pgDNSSecTable) gatherKeys(db *sql.DB) error {
+	sz, err := getSize(db, "dnssec")
+	if err != nil {
+		log.Println("PGDNSSec gatherKeys: unable to determine size of dnssec table")
+	}
+	tbl.Records = make([]pgDNSSecRecord, sz)
+
+	rows, err := db.Query("SELECT cdn, data from dnssec")
+	if err != nil {
+		return fmt.Errorf("PGDNSSec gatherKeys: unable to query: %w", err)
+	}
+	defer rows.Close()
+	i := 0
+	for rows.Next() {
+		if i > len(tbl.Records)-1 {
+			return fmt.Errorf("PGDNSSec gatherKeys got more results than expected %v", len(tbl.Records))
+		}
+		err := rows.Scan(&tbl.Records[i].CDN, &tbl.Records[i].DataEncrypted)
+		if err != nil {
+			return fmt.Errorf("PGDNSSec gatherKeys unable to scan row: %w", err)
+		}
+		i += 1
+	}
+	return nil
+}
+func (tbl *pgDNSSecTable) decrypt(aesKey []byte) error {
+	for i, _ := range tbl.Records {
+		err := decryptInto(aesKey, tbl.Records[i].DataEncrypted, &tbl.Records[i].Key)
+		if err != nil {
+			return fmt.Errorf("unable to decrypt into keys: %w", err)
+		}
+	}
+	return nil
+}
+func (tbl *pgDNSSecTable) encrypt(aesKey []byte) error {
+	for i, dns := range tbl.Records {
+		data, err := json.Marshal(&dns.Key)
+		if err != nil {
+			return fmt.Errorf("encrypt issue marshalling keys: %w", err)
+		}
+		dat, err := encrypt(data, aesKey)
+		if err != nil {
+			return fmt.Errorf("encrypt error: %w", err)
+		}
+		tbl.Records[i].DataEncrypted = dat
+	}
+	return nil
+}
+func (tbl *pgDNSSecTable) toGeneric() []DNSSecKey {
+	keys := make([]DNSSecKey, len(tbl.Records))
+
+	for i, record := range tbl.Records {
+		keys[i] = DNSSecKey{
+			CDN:                    record.CDN,
+			DNSSECKeysTrafficVault: record.Key,
+		}
+	}
+
+	return keys
+}
+func (tbl *pgDNSSecTable) fromGeneric(keys []DNSSecKey) {
+	tbl.Records = make([]pgDNSSecRecord, len(keys))
+
+	for i, key := range keys {
+		tbl.Records[i] = pgDNSSecRecord{
+			Key: key.DNSSECKeysTrafficVault,
+			CDN: key.CDN,
+			pgCommonRecord: pgCommonRecord{
+				DataEncrypted: nil,
+			},
+		}
+	}
+}
+func (tbl *pgDNSSecTable) validate() []string {
+	for i, record := range tbl.Records {
+		if record.DataEncrypted == nil && len(record.Key) > 0 {
+			return []string{fmt.Sprintf("DNSSEC Key %v: DataEncrypted is blank!", i)}
+		}
+	}
+	return nil
+}
+func (tbl *pgDNSSecTable) insertKeys(db *sql.DB) error {
+	queryBase := "INSERT INTO dnssec (cdn, data) VALUES "
+	stride := 2
+	queryArgs := make([]interface{}, len(tbl.Records)*stride)
+	for i, record := range tbl.Records {
+		j := i * stride
+		queryArgs[j] = record.CDN
+		queryArgs[j+1] = record.DataEncrypted
+	}
+	return insertIntoTable(db, queryBase, stride, queryArgs)
+}
+
+type pgSSLKeyRecord struct {
+	Keys tc.DeliveryServiceSSLKeys
+	pgCommonRecord
+
+	// These records are stored on the table but are duplicated
+	DeliveryService string
+	CDN             string
+}
+type pgSSLKeyTable struct {
+	Records []pgSSLKeyRecord
+}
+
+func (tbl *pgSSLKeyTable) insertKeys(db *sql.DB) error {
+	queryBase := "INSERT INTO sslkey (deliveryservice, data, cdn, version) VALUES "
+	duplicateKeys := 2
+	stride := 4
+	queryArgs := make([]interface{}, len(tbl.Records)*stride*duplicateKeys)
+	for i, record := range tbl.Records {
+		j := i * duplicateKeys * stride
+
+		queryArgs[j] = record.DeliveryService
+		queryArgs[j+1] = record.DataEncrypted
+		queryArgs[j+2] = record.CDN
+		queryArgs[j+3] = record.Keys.Version.String()
+
+		queryArgs[j+4] = record.DeliveryService
+		queryArgs[j+5] = record.DataEncrypted
+		queryArgs[j+6] = record.CDN
+		queryArgs[j+7] = "latest"
+	}
+	return insertIntoTable(db, queryBase, 4, queryArgs)
+}
+func (tbl *pgSSLKeyTable) gatherKeys(db *sql.DB) error {
+	sz, err := getSize(db, "sslkey WHERE version='latest'")
+	if err != nil {
+		return fmt.Errorf("PGSSLKey gatherKeys unable to determine size of sslkey table: %w", err)
+	}
+	tbl.Records = make([]pgSSLKeyRecord, sz)
+
+	rows, err := db.Query("SELECT data, deliveryservice, cdn from sslkey WHERE version = 'latest'")
+	if err != nil {
+		return fmt.Errorf("PGSSLKey gatherKeys unable to query: %w", err)
+	}
+	defer rows.Close()
+	i := 0
+	for rows.Next() {
+		if i > len(tbl.Records)-1 {
+			return errors.New("PGSSLKey gatherKeys: got more results than expected")
+		}
+		err := rows.Scan(&tbl.Records[i].DataEncrypted, &tbl.Records[i].DeliveryService, &tbl.Records[i].CDN)
+		if err != nil {
+			return fmt.Errorf("PGSSLKey gatherKeys unable to scan row: %w", err)
+		}
+		i += 1
+	}
+	return nil
+}
+func (tbl *pgSSLKeyTable) decrypt(aesKey []byte) error {
+	for i, dns := range tbl.Records {
+		err := decryptInto(aesKey, dns.DataEncrypted, &tbl.Records[i].Keys)

Review comment:
       nit: this can be combined with the line below like `if err := function(); err != nil {`

##########
File path: tools/traffic_vault_migrate/postgres.go
##########
@@ -0,0 +1,743 @@
+package main
+
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+import (
+	"database/sql"
+	"encoding/base64"
+	"encoding/json"
+	"errors"
+	"fmt"
+	"log"
+	"strconv"
+	"strings"
+
+	"github.com/apache/trafficcontrol/lib/go-tc"
+	util "github.com/apache/trafficcontrol/lib/go-util"
+
+	_ "github.com/lib/pq"
+)
+
+// PGConfig represents the configuration options available to the PG backend
+type PGConfig struct {
+	Host     string `json:"host"`
+	Port     string `json:"port"`
+	User     string `json:"user"`
+	Password string `json:"password"`
+	SSLMode  string `json:"sslmode"`
+	Database string `json:"database"`
+	Key      string `json:"aesKey"`
+	AESKey   []byte
+}
+
+// PGBackend is the Postgres implementation of TVBackend
+type PGBackend struct {
+	sslKey pgSSLKeyTable
+	dnssec pgDNSSecTable
+	uri    pgURISignKeyTable
+	url    pgURLSigKeyTable
+	cfg    PGConfig
+	db     *sql.DB
+}
+
+// String returns a high level overview of the backend and its keys
+func (pg *PGBackend) String() string {
+	data := fmt.Sprintf("PG server %v@%v:%v\n", pg.cfg.User, pg.cfg.Host, pg.cfg.Port)
+	data += fmt.Sprintf("\tSSL Keys: %v\n", len(pg.sslKey.Records))
+	data += fmt.Sprintf("\tDNSSec Keys: %v\n", len(pg.dnssec.Records))
+	data += fmt.Sprintf("\tURI Keys: %v\n", len(pg.uri.Records))
+	data += fmt.Sprintf("\tURL Keys: %v\n", len(pg.url.Records))
+	return data
+}
+
+// Name returns the name for this backend
+func (pg *PGBackend) Name() string {
+	return "PG"
+}
+
+// ReadConfig takes in a filename and will read it into the backends config
+func (pg *PGBackend) ReadConfig(s string) error {
+	err := UnmarshalConfig(s, &pg.cfg)
+	if err != nil {
+		return err
+	}
+
+	pg.cfg.AESKey, err = base64.StdEncoding.DecodeString(pg.cfg.Key)
+	if err != nil {
+		return fmt.Errorf("unable to decode PG AESKey: %w", err)
+	}
+	return nil
+}
+
+// Insert takes the current keys and inserts them into the backend DB
+func (pg *PGBackend) Insert() error {
+	if err := pg.sslKey.insertKeys(pg.db); err != nil {
+		return err
+	}
+	if err := pg.dnssec.insertKeys(pg.db); err != nil {
+		return err
+	}
+	if err := pg.url.insertKeys(pg.db); err != nil {
+		return err
+	}
+	if err := pg.uri.insertKeys(pg.db); err != nil {
+		return err
+	}
+	return nil
+}
+
+// Start initiates the connection to the backend DB
+func (pg *PGBackend) Start() error {
+	sqlStr := fmt.Sprintf("postgres://%s:%s@%s:%s/%s?sslmode=%s", pg.cfg.User, pg.cfg.Password, pg.cfg.Host, pg.cfg.Port, pg.cfg.Database, pg.cfg.SSLMode)
+	db, err := sql.Open("postgres", sqlStr)
+	if err != nil {
+		return fmt.Errorf("unable to start PG client: %w", err)
+	}
+
+	pg.db = db
+	pg.sslKey = pgSSLKeyTable{}
+	pg.dnssec = pgDNSSecTable{}
+	pg.url = pgURLSigKeyTable{}
+	pg.uri = pgURISignKeyTable{}
+
+	return nil
+}
+
+// ValidateKey validates that the keys are valid (in most cases, certain fields are not null)
+func (pg *PGBackend) ValidateKey() []string {
+	var errors []string
+	if errs := pg.sslKey.validate(); errs != nil {
+		errors = append(errors, errs...)
+	}
+	if errs := pg.dnssec.validate(); errs != nil {
+		errors = append(errors, errs...)
+	}
+	if errs := pg.uri.validate(); errs != nil {
+		errors = append(errors, errs...)
+	}
+	if errs := pg.url.validate(); errs != nil {
+		errors = append(errors, errs...)
+	}
+	return errors
+}
+
+// Close terminates the connection to the backend DB
+func (pg *PGBackend) Close() error {
+	return pg.db.Close()
+}
+
+// Ping checks the connection to the backend DB
+func (pg *PGBackend) Ping() error {
+	return pg.db.Ping()
+}
+
+// Fetch gets all of the keys from the backend DB
+func (pg *PGBackend) Fetch() error {
+	if err := pg.sslKey.gatherKeys(pg.db); err != nil {
+		return err
+	}
+
+	if err := pg.dnssec.gatherKeys(pg.db); err != nil {
+		return err
+	}
+
+	if err := pg.url.gatherKeys(pg.db); err != nil {
+		return err
+	}
+
+	if err := pg.uri.gatherKeys(pg.db); err != nil {
+		return err
+	}
+
+	return nil
+}
+
+// GetSSLKeys converts the backends internal key representation into the common representation (SSLKey)
+func (pg *PGBackend) GetSSLKeys() ([]SSLKey, error) {
+	if err := pg.sslKey.decrypt(pg.cfg.AESKey); err != nil {
+		return nil, err
+	}
+	return pg.sslKey.toGeneric(), nil
+}
+
+// SetSSLKeys takes in keys and converts & encrypts the data into the backends internal format
+func (pg *PGBackend) SetSSLKeys(keys []SSLKey) error {
+	pg.sslKey.fromGeneric(keys)
+	return pg.sslKey.encrypt(pg.cfg.AESKey)
+}
+
+// GetDNSSecKeys converts the backends internal key representation into the common representation (DNSSecKey)
+func (pg *PGBackend) GetDNSSecKeys() ([]DNSSecKey, error) {
+	if err := pg.dnssec.decrypt(pg.cfg.AESKey); err != nil {
+		return nil, err
+	}
+	return pg.dnssec.toGeneric(), nil
+}
+
+// SetDNSSecKeys takes in keys and converts & encrypts the data into the backends internal format
+func (pg *PGBackend) SetDNSSecKeys(keys []DNSSecKey) error {
+	pg.dnssec.fromGeneric(keys)
+	return pg.dnssec.encrypt(pg.cfg.AESKey)
+}
+
+// GetURISignKeys converts the pg internal key representation into the common representation (URISignKey)
+func (pg *PGBackend) GetURISignKeys() ([]URISignKey, error) {
+	if err := pg.uri.decrypt(pg.cfg.AESKey); err != nil {
+		return nil, err
+	}
+	return pg.uri.toGeneric(), nil
+}
+
+// SetURISignKeys takes in keys and converts & encrypts the data into the backends internal format
+func (pg *PGBackend) SetURISignKeys(keys []URISignKey) error {
+	pg.uri.fromGeneric(keys)
+	return pg.uri.encrypt(pg.cfg.AESKey)
+}
+
+// GetURLSigKeys converts the backends internal key representation into the common representation (URLSigKey)
+func (pg *PGBackend) GetURLSigKeys() ([]URLSigKey, error) {
+	if err := pg.url.decrypt(pg.cfg.AESKey); err != nil {
+		return nil, err
+	}
+	return pg.url.toGeneric(), nil
+}
+
+// SetURLSigKeys takes in keys and converts & encrypts the data into the backends internal format
+func (pg *PGBackend) SetURLSigKeys(keys []URLSigKey) error {
+	pg.url.fromGeneric(keys)
+	return pg.url.encrypt(pg.cfg.AESKey)
+}
+
+type pgCommonRecord struct {
+	DataEncrypted []byte
+}
+
+type pgDNSSecRecord struct {
+	Key tc.DNSSECKeysTrafficVault
+	CDN string
+	pgCommonRecord
+}
+type pgDNSSecTable struct {
+	Records []pgDNSSecRecord
+}
+
+func (tbl *pgDNSSecTable) gatherKeys(db *sql.DB) error {
+	sz, err := getSize(db, "dnssec")
+	if err != nil {
+		log.Println("PGDNSSec gatherKeys: unable to determine size of dnssec table")
+	}
+	tbl.Records = make([]pgDNSSecRecord, sz)
+
+	rows, err := db.Query("SELECT cdn, data from dnssec")
+	if err != nil {
+		return fmt.Errorf("PGDNSSec gatherKeys: unable to query: %w", err)
+	}
+	defer rows.Close()
+	i := 0
+	for rows.Next() {
+		if i > len(tbl.Records)-1 {
+			return fmt.Errorf("PGDNSSec gatherKeys got more results than expected %v", len(tbl.Records))
+		}
+		err := rows.Scan(&tbl.Records[i].CDN, &tbl.Records[i].DataEncrypted)

Review comment:
       nit: this can be combined with the line below like `if err := function(); err != nil {`

##########
File path: tools/traffic_vault_migrate/postgres.go
##########
@@ -0,0 +1,743 @@
+package main
+
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+import (
+	"database/sql"
+	"encoding/base64"
+	"encoding/json"
+	"errors"
+	"fmt"
+	"log"
+	"strconv"
+	"strings"
+
+	"github.com/apache/trafficcontrol/lib/go-tc"
+	util "github.com/apache/trafficcontrol/lib/go-util"
+
+	_ "github.com/lib/pq"
+)
+
+// PGConfig represents the configuration options available to the PG backend
+type PGConfig struct {
+	Host     string `json:"host"`
+	Port     string `json:"port"`
+	User     string `json:"user"`
+	Password string `json:"password"`
+	SSLMode  string `json:"sslmode"`
+	Database string `json:"database"`
+	Key      string `json:"aesKey"`
+	AESKey   []byte
+}
+
+// PGBackend is the Postgres implementation of TVBackend
+type PGBackend struct {
+	sslKey pgSSLKeyTable
+	dnssec pgDNSSecTable
+	uri    pgURISignKeyTable
+	url    pgURLSigKeyTable
+	cfg    PGConfig
+	db     *sql.DB
+}
+
+// String returns a high level overview of the backend and its keys
+func (pg *PGBackend) String() string {
+	data := fmt.Sprintf("PG server %v@%v:%v\n", pg.cfg.User, pg.cfg.Host, pg.cfg.Port)
+	data += fmt.Sprintf("\tSSL Keys: %v\n", len(pg.sslKey.Records))
+	data += fmt.Sprintf("\tDNSSec Keys: %v\n", len(pg.dnssec.Records))
+	data += fmt.Sprintf("\tURI Keys: %v\n", len(pg.uri.Records))
+	data += fmt.Sprintf("\tURL Keys: %v\n", len(pg.url.Records))
+	return data
+}
+
+// Name returns the name for this backend
+func (pg *PGBackend) Name() string {
+	return "PG"
+}
+
+// ReadConfig takes in a filename and will read it into the backends config
+func (pg *PGBackend) ReadConfig(s string) error {
+	err := UnmarshalConfig(s, &pg.cfg)
+	if err != nil {
+		return err
+	}
+
+	pg.cfg.AESKey, err = base64.StdEncoding.DecodeString(pg.cfg.Key)
+	if err != nil {
+		return fmt.Errorf("unable to decode PG AESKey: %w", err)
+	}
+	return nil
+}
+
+// Insert takes the current keys and inserts them into the backend DB
+func (pg *PGBackend) Insert() error {
+	if err := pg.sslKey.insertKeys(pg.db); err != nil {
+		return err
+	}
+	if err := pg.dnssec.insertKeys(pg.db); err != nil {
+		return err
+	}
+	if err := pg.url.insertKeys(pg.db); err != nil {
+		return err
+	}
+	if err := pg.uri.insertKeys(pg.db); err != nil {
+		return err
+	}
+	return nil
+}
+
+// Start initiates the connection to the backend DB
+func (pg *PGBackend) Start() error {
+	sqlStr := fmt.Sprintf("postgres://%s:%s@%s:%s/%s?sslmode=%s", pg.cfg.User, pg.cfg.Password, pg.cfg.Host, pg.cfg.Port, pg.cfg.Database, pg.cfg.SSLMode)
+	db, err := sql.Open("postgres", sqlStr)
+	if err != nil {
+		return fmt.Errorf("unable to start PG client: %w", err)
+	}
+
+	pg.db = db
+	pg.sslKey = pgSSLKeyTable{}
+	pg.dnssec = pgDNSSecTable{}
+	pg.url = pgURLSigKeyTable{}
+	pg.uri = pgURISignKeyTable{}
+
+	return nil
+}
+
+// ValidateKey validates that the keys are valid (in most cases, certain fields are not null)
+func (pg *PGBackend) ValidateKey() []string {
+	var errors []string
+	if errs := pg.sslKey.validate(); errs != nil {
+		errors = append(errors, errs...)
+	}
+	if errs := pg.dnssec.validate(); errs != nil {
+		errors = append(errors, errs...)
+	}
+	if errs := pg.uri.validate(); errs != nil {
+		errors = append(errors, errs...)
+	}
+	if errs := pg.url.validate(); errs != nil {
+		errors = append(errors, errs...)
+	}
+	return errors
+}
+
+// Close terminates the connection to the backend DB
+func (pg *PGBackend) Close() error {
+	return pg.db.Close()
+}
+
+// Ping checks the connection to the backend DB
+func (pg *PGBackend) Ping() error {
+	return pg.db.Ping()
+}
+
+// Fetch gets all of the keys from the backend DB
+func (pg *PGBackend) Fetch() error {
+	if err := pg.sslKey.gatherKeys(pg.db); err != nil {
+		return err
+	}
+
+	if err := pg.dnssec.gatherKeys(pg.db); err != nil {
+		return err
+	}
+
+	if err := pg.url.gatherKeys(pg.db); err != nil {
+		return err
+	}
+
+	if err := pg.uri.gatherKeys(pg.db); err != nil {
+		return err
+	}
+
+	return nil
+}
+
+// GetSSLKeys converts the backends internal key representation into the common representation (SSLKey)
+func (pg *PGBackend) GetSSLKeys() ([]SSLKey, error) {
+	if err := pg.sslKey.decrypt(pg.cfg.AESKey); err != nil {
+		return nil, err
+	}
+	return pg.sslKey.toGeneric(), nil
+}
+
+// SetSSLKeys takes in keys and converts & encrypts the data into the backends internal format
+func (pg *PGBackend) SetSSLKeys(keys []SSLKey) error {
+	pg.sslKey.fromGeneric(keys)
+	return pg.sslKey.encrypt(pg.cfg.AESKey)
+}
+
+// GetDNSSecKeys converts the backends internal key representation into the common representation (DNSSecKey)
+func (pg *PGBackend) GetDNSSecKeys() ([]DNSSecKey, error) {
+	if err := pg.dnssec.decrypt(pg.cfg.AESKey); err != nil {
+		return nil, err
+	}
+	return pg.dnssec.toGeneric(), nil
+}
+
+// SetDNSSecKeys takes in keys and converts & encrypts the data into the backends internal format
+func (pg *PGBackend) SetDNSSecKeys(keys []DNSSecKey) error {
+	pg.dnssec.fromGeneric(keys)
+	return pg.dnssec.encrypt(pg.cfg.AESKey)
+}
+
+// GetURISignKeys converts the pg internal key representation into the common representation (URISignKey)
+func (pg *PGBackend) GetURISignKeys() ([]URISignKey, error) {
+	if err := pg.uri.decrypt(pg.cfg.AESKey); err != nil {
+		return nil, err
+	}
+	return pg.uri.toGeneric(), nil
+}
+
+// SetURISignKeys takes in keys and converts & encrypts the data into the backends internal format
+func (pg *PGBackend) SetURISignKeys(keys []URISignKey) error {
+	pg.uri.fromGeneric(keys)
+	return pg.uri.encrypt(pg.cfg.AESKey)
+}
+
+// GetURLSigKeys converts the backends internal key representation into the common representation (URLSigKey)
+func (pg *PGBackend) GetURLSigKeys() ([]URLSigKey, error) {
+	if err := pg.url.decrypt(pg.cfg.AESKey); err != nil {
+		return nil, err
+	}
+	return pg.url.toGeneric(), nil
+}
+
+// SetURLSigKeys takes in keys and converts & encrypts the data into the backends internal format
+func (pg *PGBackend) SetURLSigKeys(keys []URLSigKey) error {
+	pg.url.fromGeneric(keys)
+	return pg.url.encrypt(pg.cfg.AESKey)
+}
+
+type pgCommonRecord struct {
+	DataEncrypted []byte
+}
+
+type pgDNSSecRecord struct {
+	Key tc.DNSSECKeysTrafficVault
+	CDN string
+	pgCommonRecord
+}
+type pgDNSSecTable struct {
+	Records []pgDNSSecRecord
+}
+
+func (tbl *pgDNSSecTable) gatherKeys(db *sql.DB) error {
+	sz, err := getSize(db, "dnssec")
+	if err != nil {
+		log.Println("PGDNSSec gatherKeys: unable to determine size of dnssec table")
+	}
+	tbl.Records = make([]pgDNSSecRecord, sz)
+
+	rows, err := db.Query("SELECT cdn, data from dnssec")
+	if err != nil {
+		return fmt.Errorf("PGDNSSec gatherKeys: unable to query: %w", err)
+	}
+	defer rows.Close()
+	i := 0
+	for rows.Next() {
+		if i > len(tbl.Records)-1 {
+			return fmt.Errorf("PGDNSSec gatherKeys got more results than expected %v", len(tbl.Records))

Review comment:
       nit: %d instead of %v

##########
File path: tools/traffic_vault_migrate/postgres.go
##########
@@ -0,0 +1,743 @@
+package main
+
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+import (
+	"database/sql"
+	"encoding/base64"
+	"encoding/json"
+	"errors"
+	"fmt"
+	"log"
+	"strconv"
+	"strings"
+
+	"github.com/apache/trafficcontrol/lib/go-tc"
+	util "github.com/apache/trafficcontrol/lib/go-util"
+
+	_ "github.com/lib/pq"
+)
+
+// PGConfig represents the configuration options available to the PG backend
+type PGConfig struct {
+	Host     string `json:"host"`
+	Port     string `json:"port"`
+	User     string `json:"user"`
+	Password string `json:"password"`
+	SSLMode  string `json:"sslmode"`
+	Database string `json:"database"`
+	Key      string `json:"aesKey"`
+	AESKey   []byte
+}
+
+// PGBackend is the Postgres implementation of TVBackend
+type PGBackend struct {
+	sslKey pgSSLKeyTable
+	dnssec pgDNSSecTable
+	uri    pgURISignKeyTable
+	url    pgURLSigKeyTable
+	cfg    PGConfig
+	db     *sql.DB
+}
+
+// String returns a high level overview of the backend and its keys
+func (pg *PGBackend) String() string {
+	data := fmt.Sprintf("PG server %v@%v:%v\n", pg.cfg.User, pg.cfg.Host, pg.cfg.Port)
+	data += fmt.Sprintf("\tSSL Keys: %v\n", len(pg.sslKey.Records))
+	data += fmt.Sprintf("\tDNSSec Keys: %v\n", len(pg.dnssec.Records))
+	data += fmt.Sprintf("\tURI Keys: %v\n", len(pg.uri.Records))
+	data += fmt.Sprintf("\tURL Keys: %v\n", len(pg.url.Records))
+	return data
+}
+
+// Name returns the name for this backend
+func (pg *PGBackend) Name() string {
+	return "PG"
+}
+
+// ReadConfig takes in a filename and will read it into the backends config
+func (pg *PGBackend) ReadConfig(s string) error {
+	err := UnmarshalConfig(s, &pg.cfg)
+	if err != nil {
+		return err
+	}
+
+	pg.cfg.AESKey, err = base64.StdEncoding.DecodeString(pg.cfg.Key)
+	if err != nil {
+		return fmt.Errorf("unable to decode PG AESKey: %w", err)
+	}
+	return nil
+}
+
+// Insert takes the current keys and inserts them into the backend DB
+func (pg *PGBackend) Insert() error {
+	if err := pg.sslKey.insertKeys(pg.db); err != nil {
+		return err
+	}
+	if err := pg.dnssec.insertKeys(pg.db); err != nil {
+		return err
+	}
+	if err := pg.url.insertKeys(pg.db); err != nil {
+		return err
+	}
+	if err := pg.uri.insertKeys(pg.db); err != nil {
+		return err
+	}
+	return nil
+}
+
+// Start initiates the connection to the backend DB
+func (pg *PGBackend) Start() error {
+	sqlStr := fmt.Sprintf("postgres://%s:%s@%s:%s/%s?sslmode=%s", pg.cfg.User, pg.cfg.Password, pg.cfg.Host, pg.cfg.Port, pg.cfg.Database, pg.cfg.SSLMode)
+	db, err := sql.Open("postgres", sqlStr)
+	if err != nil {
+		return fmt.Errorf("unable to start PG client: %w", err)
+	}
+
+	pg.db = db
+	pg.sslKey = pgSSLKeyTable{}
+	pg.dnssec = pgDNSSecTable{}
+	pg.url = pgURLSigKeyTable{}
+	pg.uri = pgURISignKeyTable{}
+
+	return nil
+}
+
+// ValidateKey validates that the keys are valid (in most cases, certain fields are not null)
+func (pg *PGBackend) ValidateKey() []string {
+	var errors []string
+	if errs := pg.sslKey.validate(); errs != nil {
+		errors = append(errors, errs...)
+	}
+	if errs := pg.dnssec.validate(); errs != nil {
+		errors = append(errors, errs...)
+	}
+	if errs := pg.uri.validate(); errs != nil {
+		errors = append(errors, errs...)
+	}
+	if errs := pg.url.validate(); errs != nil {
+		errors = append(errors, errs...)
+	}
+	return errors
+}
+
+// Close terminates the connection to the backend DB
+func (pg *PGBackend) Close() error {
+	return pg.db.Close()
+}
+
+// Ping checks the connection to the backend DB
+func (pg *PGBackend) Ping() error {
+	return pg.db.Ping()
+}
+
+// Fetch gets all of the keys from the backend DB
+func (pg *PGBackend) Fetch() error {
+	if err := pg.sslKey.gatherKeys(pg.db); err != nil {
+		return err
+	}
+
+	if err := pg.dnssec.gatherKeys(pg.db); err != nil {
+		return err
+	}
+
+	if err := pg.url.gatherKeys(pg.db); err != nil {
+		return err
+	}
+
+	if err := pg.uri.gatherKeys(pg.db); err != nil {
+		return err
+	}
+
+	return nil
+}
+
+// GetSSLKeys converts the backends internal key representation into the common representation (SSLKey)
+func (pg *PGBackend) GetSSLKeys() ([]SSLKey, error) {
+	if err := pg.sslKey.decrypt(pg.cfg.AESKey); err != nil {
+		return nil, err
+	}
+	return pg.sslKey.toGeneric(), nil
+}
+
+// SetSSLKeys takes in keys and converts & encrypts the data into the backends internal format
+func (pg *PGBackend) SetSSLKeys(keys []SSLKey) error {
+	pg.sslKey.fromGeneric(keys)
+	return pg.sslKey.encrypt(pg.cfg.AESKey)
+}
+
+// GetDNSSecKeys converts the backends internal key representation into the common representation (DNSSecKey)
+func (pg *PGBackend) GetDNSSecKeys() ([]DNSSecKey, error) {
+	if err := pg.dnssec.decrypt(pg.cfg.AESKey); err != nil {
+		return nil, err
+	}
+	return pg.dnssec.toGeneric(), nil
+}
+
+// SetDNSSecKeys takes in keys and converts & encrypts the data into the backends internal format
+func (pg *PGBackend) SetDNSSecKeys(keys []DNSSecKey) error {
+	pg.dnssec.fromGeneric(keys)
+	return pg.dnssec.encrypt(pg.cfg.AESKey)
+}
+
+// GetURISignKeys converts the pg internal key representation into the common representation (URISignKey)
+func (pg *PGBackend) GetURISignKeys() ([]URISignKey, error) {
+	if err := pg.uri.decrypt(pg.cfg.AESKey); err != nil {
+		return nil, err
+	}
+	return pg.uri.toGeneric(), nil
+}
+
+// SetURISignKeys takes in keys and converts & encrypts the data into the backends internal format
+func (pg *PGBackend) SetURISignKeys(keys []URISignKey) error {
+	pg.uri.fromGeneric(keys)
+	return pg.uri.encrypt(pg.cfg.AESKey)
+}
+
+// GetURLSigKeys converts the backends internal key representation into the common representation (URLSigKey)
+func (pg *PGBackend) GetURLSigKeys() ([]URLSigKey, error) {
+	if err := pg.url.decrypt(pg.cfg.AESKey); err != nil {
+		return nil, err
+	}
+	return pg.url.toGeneric(), nil
+}
+
+// SetURLSigKeys takes in keys and converts & encrypts the data into the backends internal format
+func (pg *PGBackend) SetURLSigKeys(keys []URLSigKey) error {
+	pg.url.fromGeneric(keys)
+	return pg.url.encrypt(pg.cfg.AESKey)
+}
+
+type pgCommonRecord struct {
+	DataEncrypted []byte
+}
+
+type pgDNSSecRecord struct {
+	Key tc.DNSSECKeysTrafficVault
+	CDN string
+	pgCommonRecord
+}
+type pgDNSSecTable struct {
+	Records []pgDNSSecRecord
+}
+
+func (tbl *pgDNSSecTable) gatherKeys(db *sql.DB) error {
+	sz, err := getSize(db, "dnssec")
+	if err != nil {
+		log.Println("PGDNSSec gatherKeys: unable to determine size of dnssec table")
+	}
+	tbl.Records = make([]pgDNSSecRecord, sz)
+
+	rows, err := db.Query("SELECT cdn, data from dnssec")
+	if err != nil {
+		return fmt.Errorf("PGDNSSec gatherKeys: unable to query: %w", err)
+	}
+	defer rows.Close()
+	i := 0
+	for rows.Next() {
+		if i > len(tbl.Records)-1 {
+			return fmt.Errorf("PGDNSSec gatherKeys got more results than expected %v", len(tbl.Records))
+		}
+		err := rows.Scan(&tbl.Records[i].CDN, &tbl.Records[i].DataEncrypted)
+		if err != nil {
+			return fmt.Errorf("PGDNSSec gatherKeys unable to scan row: %w", err)
+		}
+		i += 1
+	}
+	return nil
+}
+func (tbl *pgDNSSecTable) decrypt(aesKey []byte) error {
+	for i, _ := range tbl.Records {
+		err := decryptInto(aesKey, tbl.Records[i].DataEncrypted, &tbl.Records[i].Key)

Review comment:
       nit: this can be combined with the line below like `if err := function(); err != nil {`

##########
File path: tools/traffic_vault_migrate/postgres.go
##########
@@ -0,0 +1,743 @@
+package main
+
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+import (
+	"database/sql"
+	"encoding/base64"
+	"encoding/json"
+	"errors"
+	"fmt"
+	"log"
+	"strconv"
+	"strings"
+
+	"github.com/apache/trafficcontrol/lib/go-tc"
+	util "github.com/apache/trafficcontrol/lib/go-util"
+
+	_ "github.com/lib/pq"
+)
+
+// PGConfig represents the configuration options available to the PG backend
+type PGConfig struct {
+	Host     string `json:"host"`
+	Port     string `json:"port"`
+	User     string `json:"user"`
+	Password string `json:"password"`
+	SSLMode  string `json:"sslmode"`
+	Database string `json:"database"`
+	Key      string `json:"aesKey"`
+	AESKey   []byte
+}
+
+// PGBackend is the Postgres implementation of TVBackend
+type PGBackend struct {
+	sslKey pgSSLKeyTable
+	dnssec pgDNSSecTable
+	uri    pgURISignKeyTable
+	url    pgURLSigKeyTable
+	cfg    PGConfig
+	db     *sql.DB
+}
+
+// String returns a high level overview of the backend and its keys
+func (pg *PGBackend) String() string {
+	data := fmt.Sprintf("PG server %v@%v:%v\n", pg.cfg.User, pg.cfg.Host, pg.cfg.Port)
+	data += fmt.Sprintf("\tSSL Keys: %v\n", len(pg.sslKey.Records))
+	data += fmt.Sprintf("\tDNSSec Keys: %v\n", len(pg.dnssec.Records))
+	data += fmt.Sprintf("\tURI Keys: %v\n", len(pg.uri.Records))
+	data += fmt.Sprintf("\tURL Keys: %v\n", len(pg.url.Records))
+	return data
+}
+
+// Name returns the name for this backend
+func (pg *PGBackend) Name() string {
+	return "PG"
+}
+
+// ReadConfig takes in a filename and will read it into the backends config
+func (pg *PGBackend) ReadConfig(s string) error {
+	err := UnmarshalConfig(s, &pg.cfg)
+	if err != nil {
+		return err
+	}
+
+	pg.cfg.AESKey, err = base64.StdEncoding.DecodeString(pg.cfg.Key)
+	if err != nil {
+		return fmt.Errorf("unable to decode PG AESKey: %w", err)
+	}
+	return nil
+}
+
+// Insert takes the current keys and inserts them into the backend DB
+func (pg *PGBackend) Insert() error {
+	if err := pg.sslKey.insertKeys(pg.db); err != nil {
+		return err
+	}
+	if err := pg.dnssec.insertKeys(pg.db); err != nil {
+		return err
+	}
+	if err := pg.url.insertKeys(pg.db); err != nil {
+		return err
+	}
+	if err := pg.uri.insertKeys(pg.db); err != nil {
+		return err
+	}
+	return nil
+}
+
+// Start initiates the connection to the backend DB
+func (pg *PGBackend) Start() error {
+	sqlStr := fmt.Sprintf("postgres://%s:%s@%s:%s/%s?sslmode=%s", pg.cfg.User, pg.cfg.Password, pg.cfg.Host, pg.cfg.Port, pg.cfg.Database, pg.cfg.SSLMode)
+	db, err := sql.Open("postgres", sqlStr)
+	if err != nil {
+		return fmt.Errorf("unable to start PG client: %w", err)
+	}
+
+	pg.db = db
+	pg.sslKey = pgSSLKeyTable{}
+	pg.dnssec = pgDNSSecTable{}
+	pg.url = pgURLSigKeyTable{}
+	pg.uri = pgURISignKeyTable{}
+
+	return nil
+}
+
+// ValidateKey validates that the keys are valid (in most cases, certain fields are not null)
+func (pg *PGBackend) ValidateKey() []string {
+	var errors []string
+	if errs := pg.sslKey.validate(); errs != nil {
+		errors = append(errors, errs...)
+	}
+	if errs := pg.dnssec.validate(); errs != nil {
+		errors = append(errors, errs...)
+	}
+	if errs := pg.uri.validate(); errs != nil {
+		errors = append(errors, errs...)
+	}
+	if errs := pg.url.validate(); errs != nil {
+		errors = append(errors, errs...)
+	}
+	return errors
+}
+
+// Close terminates the connection to the backend DB
+func (pg *PGBackend) Close() error {
+	return pg.db.Close()
+}
+
+// Ping checks the connection to the backend DB
+func (pg *PGBackend) Ping() error {
+	return pg.db.Ping()
+}
+
+// Fetch gets all of the keys from the backend DB
+func (pg *PGBackend) Fetch() error {
+	if err := pg.sslKey.gatherKeys(pg.db); err != nil {
+		return err
+	}
+
+	if err := pg.dnssec.gatherKeys(pg.db); err != nil {
+		return err
+	}
+
+	if err := pg.url.gatherKeys(pg.db); err != nil {
+		return err
+	}
+
+	if err := pg.uri.gatherKeys(pg.db); err != nil {
+		return err
+	}
+
+	return nil
+}
+
+// GetSSLKeys converts the backends internal key representation into the common representation (SSLKey)
+func (pg *PGBackend) GetSSLKeys() ([]SSLKey, error) {
+	if err := pg.sslKey.decrypt(pg.cfg.AESKey); err != nil {
+		return nil, err
+	}
+	return pg.sslKey.toGeneric(), nil
+}
+
+// SetSSLKeys takes in keys and converts & encrypts the data into the backends internal format
+func (pg *PGBackend) SetSSLKeys(keys []SSLKey) error {
+	pg.sslKey.fromGeneric(keys)
+	return pg.sslKey.encrypt(pg.cfg.AESKey)
+}
+
+// GetDNSSecKeys converts the backends internal key representation into the common representation (DNSSecKey)
+func (pg *PGBackend) GetDNSSecKeys() ([]DNSSecKey, error) {
+	if err := pg.dnssec.decrypt(pg.cfg.AESKey); err != nil {
+		return nil, err
+	}
+	return pg.dnssec.toGeneric(), nil
+}
+
+// SetDNSSecKeys takes in keys and converts & encrypts the data into the backends internal format
+func (pg *PGBackend) SetDNSSecKeys(keys []DNSSecKey) error {
+	pg.dnssec.fromGeneric(keys)
+	return pg.dnssec.encrypt(pg.cfg.AESKey)
+}
+
+// GetURISignKeys converts the pg internal key representation into the common representation (URISignKey)
+func (pg *PGBackend) GetURISignKeys() ([]URISignKey, error) {
+	if err := pg.uri.decrypt(pg.cfg.AESKey); err != nil {
+		return nil, err
+	}
+	return pg.uri.toGeneric(), nil
+}
+
+// SetURISignKeys takes in keys and converts & encrypts the data into the backends internal format
+func (pg *PGBackend) SetURISignKeys(keys []URISignKey) error {
+	pg.uri.fromGeneric(keys)
+	return pg.uri.encrypt(pg.cfg.AESKey)
+}
+
+// GetURLSigKeys converts the backends internal key representation into the common representation (URLSigKey)
+func (pg *PGBackend) GetURLSigKeys() ([]URLSigKey, error) {
+	if err := pg.url.decrypt(pg.cfg.AESKey); err != nil {
+		return nil, err
+	}
+	return pg.url.toGeneric(), nil
+}
+
+// SetURLSigKeys takes in keys and converts & encrypts the data into the backends internal format
+func (pg *PGBackend) SetURLSigKeys(keys []URLSigKey) error {
+	pg.url.fromGeneric(keys)
+	return pg.url.encrypt(pg.cfg.AESKey)
+}
+
+type pgCommonRecord struct {
+	DataEncrypted []byte
+}
+
+type pgDNSSecRecord struct {
+	Key tc.DNSSECKeysTrafficVault
+	CDN string
+	pgCommonRecord
+}
+type pgDNSSecTable struct {
+	Records []pgDNSSecRecord
+}
+
+func (tbl *pgDNSSecTable) gatherKeys(db *sql.DB) error {
+	sz, err := getSize(db, "dnssec")
+	if err != nil {
+		log.Println("PGDNSSec gatherKeys: unable to determine size of dnssec table")
+	}
+	tbl.Records = make([]pgDNSSecRecord, sz)
+
+	rows, err := db.Query("SELECT cdn, data from dnssec")
+	if err != nil {
+		return fmt.Errorf("PGDNSSec gatherKeys: unable to query: %w", err)
+	}
+	defer rows.Close()
+	i := 0
+	for rows.Next() {
+		if i > len(tbl.Records)-1 {
+			return fmt.Errorf("PGDNSSec gatherKeys got more results than expected %v", len(tbl.Records))
+		}
+		err := rows.Scan(&tbl.Records[i].CDN, &tbl.Records[i].DataEncrypted)
+		if err != nil {
+			return fmt.Errorf("PGDNSSec gatherKeys unable to scan row: %w", err)
+		}
+		i += 1
+	}
+	return nil
+}
+func (tbl *pgDNSSecTable) decrypt(aesKey []byte) error {
+	for i, _ := range tbl.Records {
+		err := decryptInto(aesKey, tbl.Records[i].DataEncrypted, &tbl.Records[i].Key)
+		if err != nil {
+			return fmt.Errorf("unable to decrypt into keys: %w", err)
+		}
+	}
+	return nil
+}
+func (tbl *pgDNSSecTable) encrypt(aesKey []byte) error {
+	for i, dns := range tbl.Records {
+		data, err := json.Marshal(&dns.Key)
+		if err != nil {
+			return fmt.Errorf("encrypt issue marshalling keys: %w", err)
+		}
+		dat, err := encrypt(data, aesKey)
+		if err != nil {
+			return fmt.Errorf("encrypt error: %w", err)
+		}
+		tbl.Records[i].DataEncrypted = dat
+	}
+	return nil
+}
+func (tbl *pgDNSSecTable) toGeneric() []DNSSecKey {
+	keys := make([]DNSSecKey, len(tbl.Records))
+
+	for i, record := range tbl.Records {
+		keys[i] = DNSSecKey{
+			CDN:                    record.CDN,
+			DNSSECKeysTrafficVault: record.Key,
+		}
+	}
+
+	return keys
+}
+func (tbl *pgDNSSecTable) fromGeneric(keys []DNSSecKey) {
+	tbl.Records = make([]pgDNSSecRecord, len(keys))
+
+	for i, key := range keys {
+		tbl.Records[i] = pgDNSSecRecord{
+			Key: key.DNSSECKeysTrafficVault,
+			CDN: key.CDN,
+			pgCommonRecord: pgCommonRecord{
+				DataEncrypted: nil,
+			},
+		}
+	}
+}
+func (tbl *pgDNSSecTable) validate() []string {
+	for i, record := range tbl.Records {
+		if record.DataEncrypted == nil && len(record.Key) > 0 {
+			return []string{fmt.Sprintf("DNSSEC Key %v: DataEncrypted is blank!", i)}
+		}
+	}
+	return nil
+}
+func (tbl *pgDNSSecTable) insertKeys(db *sql.DB) error {
+	queryBase := "INSERT INTO dnssec (cdn, data) VALUES "
+	stride := 2
+	queryArgs := make([]interface{}, len(tbl.Records)*stride)
+	for i, record := range tbl.Records {
+		j := i * stride
+		queryArgs[j] = record.CDN
+		queryArgs[j+1] = record.DataEncrypted
+	}
+	return insertIntoTable(db, queryBase, stride, queryArgs)
+}
+
+type pgSSLKeyRecord struct {
+	Keys tc.DeliveryServiceSSLKeys
+	pgCommonRecord
+
+	// These records are stored on the table but are duplicated
+	DeliveryService string
+	CDN             string
+}
+type pgSSLKeyTable struct {
+	Records []pgSSLKeyRecord
+}
+
+func (tbl *pgSSLKeyTable) insertKeys(db *sql.DB) error {
+	queryBase := "INSERT INTO sslkey (deliveryservice, data, cdn, version) VALUES "
+	duplicateKeys := 2
+	stride := 4
+	queryArgs := make([]interface{}, len(tbl.Records)*stride*duplicateKeys)
+	for i, record := range tbl.Records {
+		j := i * duplicateKeys * stride
+
+		queryArgs[j] = record.DeliveryService
+		queryArgs[j+1] = record.DataEncrypted
+		queryArgs[j+2] = record.CDN
+		queryArgs[j+3] = record.Keys.Version.String()
+
+		queryArgs[j+4] = record.DeliveryService
+		queryArgs[j+5] = record.DataEncrypted
+		queryArgs[j+6] = record.CDN
+		queryArgs[j+7] = "latest"
+	}
+	return insertIntoTable(db, queryBase, 4, queryArgs)
+}
+func (tbl *pgSSLKeyTable) gatherKeys(db *sql.DB) error {
+	sz, err := getSize(db, "sslkey WHERE version='latest'")
+	if err != nil {
+		return fmt.Errorf("PGSSLKey gatherKeys unable to determine size of sslkey table: %w", err)
+	}
+	tbl.Records = make([]pgSSLKeyRecord, sz)
+
+	rows, err := db.Query("SELECT data, deliveryservice, cdn from sslkey WHERE version = 'latest'")
+	if err != nil {
+		return fmt.Errorf("PGSSLKey gatherKeys unable to query: %w", err)
+	}
+	defer rows.Close()
+	i := 0
+	for rows.Next() {
+		if i > len(tbl.Records)-1 {
+			return errors.New("PGSSLKey gatherKeys: got more results than expected")
+		}
+		err := rows.Scan(&tbl.Records[i].DataEncrypted, &tbl.Records[i].DeliveryService, &tbl.Records[i].CDN)
+		if err != nil {
+			return fmt.Errorf("PGSSLKey gatherKeys unable to scan row: %w", err)
+		}
+		i += 1
+	}
+	return nil
+}
+func (tbl *pgSSLKeyTable) decrypt(aesKey []byte) error {
+	for i, dns := range tbl.Records {
+		err := decryptInto(aesKey, dns.DataEncrypted, &tbl.Records[i].Keys)
+		if err != nil {
+			return fmt.Errorf("unable to decrypt into keys: %w", err)
+		}
+	}
+	return nil
+}
+func (tbl *pgSSLKeyTable) encrypt(aesKey []byte) error {
+	for i, dns := range tbl.Records {
+		data, err := json.Marshal(dns.Keys)
+		if err != nil {
+			return fmt.Errorf("encrypt issue marshalling keys: %w", err)
+		}
+		dat, err := encrypt(data, aesKey)
+		if err != nil {
+			return fmt.Errorf("encrypt error: %w", err)
+		}
+		tbl.Records[i].DataEncrypted = dat
+	}
+	return nil
+}
+func (tbl *pgSSLKeyTable) toGeneric() []SSLKey {
+	keys := make([]SSLKey, len(tbl.Records))
+
+	for i, record := range tbl.Records {
+		keys[i] = SSLKey{
+			DeliveryServiceSSLKeys: record.Keys,
+		}
+	}
+	return keys
+}
+func (tbl *pgSSLKeyTable) fromGeneric(keys []SSLKey) {
+	tbl.Records = make([]pgSSLKeyRecord, len(keys))
+
+	for i, key := range keys {
+		tbl.Records[i] = pgSSLKeyRecord{
+			Keys: key.DeliveryServiceSSLKeys,
+			pgCommonRecord: pgCommonRecord{
+				DataEncrypted: nil,
+			},
+			DeliveryService: key.DeliveryService,
+			CDN:             key.CDN,
+		}
+	}
+}
+func (tbl *pgSSLKeyTable) validate() []string {
+	defaultKey := tc.DeliveryServiceSSLKeys{}
+	var errors []string
+	fmtStr := "SSL Key %v: %v"

Review comment:
       nit: %d and %s

##########
File path: tools/traffic_vault_migrate/postgres.go
##########
@@ -0,0 +1,743 @@
+package main
+
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+import (
+	"database/sql"
+	"encoding/base64"
+	"encoding/json"
+	"errors"
+	"fmt"
+	"log"
+	"strconv"
+	"strings"
+
+	"github.com/apache/trafficcontrol/lib/go-tc"
+	util "github.com/apache/trafficcontrol/lib/go-util"
+
+	_ "github.com/lib/pq"
+)
+
+// PGConfig represents the configuration options available to the PG backend
+type PGConfig struct {
+	Host     string `json:"host"`
+	Port     string `json:"port"`
+	User     string `json:"user"`
+	Password string `json:"password"`
+	SSLMode  string `json:"sslmode"`
+	Database string `json:"database"`
+	Key      string `json:"aesKey"`
+	AESKey   []byte
+}
+
+// PGBackend is the Postgres implementation of TVBackend
+type PGBackend struct {
+	sslKey pgSSLKeyTable
+	dnssec pgDNSSecTable
+	uri    pgURISignKeyTable
+	url    pgURLSigKeyTable
+	cfg    PGConfig
+	db     *sql.DB
+}
+
+// String returns a high level overview of the backend and its keys
+func (pg *PGBackend) String() string {
+	data := fmt.Sprintf("PG server %v@%v:%v\n", pg.cfg.User, pg.cfg.Host, pg.cfg.Port)
+	data += fmt.Sprintf("\tSSL Keys: %v\n", len(pg.sslKey.Records))
+	data += fmt.Sprintf("\tDNSSec Keys: %v\n", len(pg.dnssec.Records))
+	data += fmt.Sprintf("\tURI Keys: %v\n", len(pg.uri.Records))
+	data += fmt.Sprintf("\tURL Keys: %v\n", len(pg.url.Records))
+	return data
+}
+
+// Name returns the name for this backend
+func (pg *PGBackend) Name() string {
+	return "PG"
+}
+
+// ReadConfig takes in a filename and will read it into the backends config
+func (pg *PGBackend) ReadConfig(s string) error {
+	err := UnmarshalConfig(s, &pg.cfg)
+	if err != nil {
+		return err
+	}
+
+	pg.cfg.AESKey, err = base64.StdEncoding.DecodeString(pg.cfg.Key)
+	if err != nil {
+		return fmt.Errorf("unable to decode PG AESKey: %w", err)
+	}
+	return nil
+}
+
+// Insert takes the current keys and inserts them into the backend DB
+func (pg *PGBackend) Insert() error {
+	if err := pg.sslKey.insertKeys(pg.db); err != nil {
+		return err
+	}
+	if err := pg.dnssec.insertKeys(pg.db); err != nil {
+		return err
+	}
+	if err := pg.url.insertKeys(pg.db); err != nil {
+		return err
+	}
+	if err := pg.uri.insertKeys(pg.db); err != nil {
+		return err
+	}
+	return nil
+}
+
+// Start initiates the connection to the backend DB
+func (pg *PGBackend) Start() error {
+	sqlStr := fmt.Sprintf("postgres://%s:%s@%s:%s/%s?sslmode=%s", pg.cfg.User, pg.cfg.Password, pg.cfg.Host, pg.cfg.Port, pg.cfg.Database, pg.cfg.SSLMode)
+	db, err := sql.Open("postgres", sqlStr)
+	if err != nil {
+		return fmt.Errorf("unable to start PG client: %w", err)
+	}
+
+	pg.db = db
+	pg.sslKey = pgSSLKeyTable{}
+	pg.dnssec = pgDNSSecTable{}
+	pg.url = pgURLSigKeyTable{}
+	pg.uri = pgURISignKeyTable{}
+
+	return nil
+}
+
+// ValidateKey validates that the keys are valid (in most cases, certain fields are not null)
+func (pg *PGBackend) ValidateKey() []string {
+	var errors []string
+	if errs := pg.sslKey.validate(); errs != nil {
+		errors = append(errors, errs...)
+	}
+	if errs := pg.dnssec.validate(); errs != nil {
+		errors = append(errors, errs...)
+	}
+	if errs := pg.uri.validate(); errs != nil {
+		errors = append(errors, errs...)
+	}
+	if errs := pg.url.validate(); errs != nil {
+		errors = append(errors, errs...)
+	}
+	return errors
+}
+
+// Close terminates the connection to the backend DB
+func (pg *PGBackend) Close() error {
+	return pg.db.Close()
+}
+
+// Ping checks the connection to the backend DB
+func (pg *PGBackend) Ping() error {
+	return pg.db.Ping()
+}
+
+// Fetch gets all of the keys from the backend DB
+func (pg *PGBackend) Fetch() error {
+	if err := pg.sslKey.gatherKeys(pg.db); err != nil {
+		return err
+	}
+
+	if err := pg.dnssec.gatherKeys(pg.db); err != nil {
+		return err
+	}
+
+	if err := pg.url.gatherKeys(pg.db); err != nil {
+		return err
+	}
+
+	if err := pg.uri.gatherKeys(pg.db); err != nil {
+		return err
+	}
+
+	return nil
+}
+
+// GetSSLKeys converts the backends internal key representation into the common representation (SSLKey)
+func (pg *PGBackend) GetSSLKeys() ([]SSLKey, error) {
+	if err := pg.sslKey.decrypt(pg.cfg.AESKey); err != nil {
+		return nil, err
+	}
+	return pg.sslKey.toGeneric(), nil
+}
+
+// SetSSLKeys takes in keys and converts & encrypts the data into the backends internal format
+func (pg *PGBackend) SetSSLKeys(keys []SSLKey) error {
+	pg.sslKey.fromGeneric(keys)
+	return pg.sslKey.encrypt(pg.cfg.AESKey)
+}
+
+// GetDNSSecKeys converts the backends internal key representation into the common representation (DNSSecKey)
+func (pg *PGBackend) GetDNSSecKeys() ([]DNSSecKey, error) {
+	if err := pg.dnssec.decrypt(pg.cfg.AESKey); err != nil {
+		return nil, err
+	}
+	return pg.dnssec.toGeneric(), nil
+}
+
+// SetDNSSecKeys takes in keys and converts & encrypts the data into the backends internal format
+func (pg *PGBackend) SetDNSSecKeys(keys []DNSSecKey) error {
+	pg.dnssec.fromGeneric(keys)
+	return pg.dnssec.encrypt(pg.cfg.AESKey)
+}
+
+// GetURISignKeys converts the pg internal key representation into the common representation (URISignKey)
+func (pg *PGBackend) GetURISignKeys() ([]URISignKey, error) {
+	if err := pg.uri.decrypt(pg.cfg.AESKey); err != nil {
+		return nil, err
+	}
+	return pg.uri.toGeneric(), nil
+}
+
+// SetURISignKeys takes in keys and converts & encrypts the data into the backends internal format
+func (pg *PGBackend) SetURISignKeys(keys []URISignKey) error {
+	pg.uri.fromGeneric(keys)
+	return pg.uri.encrypt(pg.cfg.AESKey)
+}
+
+// GetURLSigKeys converts the backends internal key representation into the common representation (URLSigKey)
+func (pg *PGBackend) GetURLSigKeys() ([]URLSigKey, error) {
+	if err := pg.url.decrypt(pg.cfg.AESKey); err != nil {
+		return nil, err
+	}
+	return pg.url.toGeneric(), nil
+}
+
+// SetURLSigKeys takes in keys and converts & encrypts the data into the backends internal format
+func (pg *PGBackend) SetURLSigKeys(keys []URLSigKey) error {
+	pg.url.fromGeneric(keys)
+	return pg.url.encrypt(pg.cfg.AESKey)
+}
+
+type pgCommonRecord struct {
+	DataEncrypted []byte
+}
+
+type pgDNSSecRecord struct {
+	Key tc.DNSSECKeysTrafficVault
+	CDN string
+	pgCommonRecord
+}
+type pgDNSSecTable struct {
+	Records []pgDNSSecRecord
+}
+
+func (tbl *pgDNSSecTable) gatherKeys(db *sql.DB) error {
+	sz, err := getSize(db, "dnssec")
+	if err != nil {
+		log.Println("PGDNSSec gatherKeys: unable to determine size of dnssec table")
+	}
+	tbl.Records = make([]pgDNSSecRecord, sz)
+
+	rows, err := db.Query("SELECT cdn, data from dnssec")
+	if err != nil {
+		return fmt.Errorf("PGDNSSec gatherKeys: unable to query: %w", err)
+	}
+	defer rows.Close()
+	i := 0
+	for rows.Next() {
+		if i > len(tbl.Records)-1 {
+			return fmt.Errorf("PGDNSSec gatherKeys got more results than expected %v", len(tbl.Records))
+		}
+		err := rows.Scan(&tbl.Records[i].CDN, &tbl.Records[i].DataEncrypted)
+		if err != nil {
+			return fmt.Errorf("PGDNSSec gatherKeys unable to scan row: %w", err)
+		}
+		i += 1
+	}
+	return nil
+}
+func (tbl *pgDNSSecTable) decrypt(aesKey []byte) error {
+	for i, _ := range tbl.Records {
+		err := decryptInto(aesKey, tbl.Records[i].DataEncrypted, &tbl.Records[i].Key)
+		if err != nil {
+			return fmt.Errorf("unable to decrypt into keys: %w", err)
+		}
+	}
+	return nil
+}
+func (tbl *pgDNSSecTable) encrypt(aesKey []byte) error {
+	for i, dns := range tbl.Records {
+		data, err := json.Marshal(&dns.Key)
+		if err != nil {
+			return fmt.Errorf("encrypt issue marshalling keys: %w", err)
+		}
+		dat, err := encrypt(data, aesKey)
+		if err != nil {
+			return fmt.Errorf("encrypt error: %w", err)
+		}
+		tbl.Records[i].DataEncrypted = dat
+	}
+	return nil
+}
+func (tbl *pgDNSSecTable) toGeneric() []DNSSecKey {
+	keys := make([]DNSSecKey, len(tbl.Records))
+
+	for i, record := range tbl.Records {
+		keys[i] = DNSSecKey{
+			CDN:                    record.CDN,
+			DNSSECKeysTrafficVault: record.Key,
+		}
+	}
+
+	return keys
+}
+func (tbl *pgDNSSecTable) fromGeneric(keys []DNSSecKey) {
+	tbl.Records = make([]pgDNSSecRecord, len(keys))
+
+	for i, key := range keys {
+		tbl.Records[i] = pgDNSSecRecord{
+			Key: key.DNSSECKeysTrafficVault,
+			CDN: key.CDN,
+			pgCommonRecord: pgCommonRecord{
+				DataEncrypted: nil,
+			},
+		}
+	}
+}
+func (tbl *pgDNSSecTable) validate() []string {
+	for i, record := range tbl.Records {
+		if record.DataEncrypted == nil && len(record.Key) > 0 {
+			return []string{fmt.Sprintf("DNSSEC Key %v: DataEncrypted is blank!", i)}
+		}
+	}
+	return nil
+}
+func (tbl *pgDNSSecTable) insertKeys(db *sql.DB) error {
+	queryBase := "INSERT INTO dnssec (cdn, data) VALUES "
+	stride := 2
+	queryArgs := make([]interface{}, len(tbl.Records)*stride)
+	for i, record := range tbl.Records {
+		j := i * stride
+		queryArgs[j] = record.CDN
+		queryArgs[j+1] = record.DataEncrypted
+	}
+	return insertIntoTable(db, queryBase, stride, queryArgs)
+}
+
+type pgSSLKeyRecord struct {
+	Keys tc.DeliveryServiceSSLKeys
+	pgCommonRecord
+
+	// These records are stored on the table but are duplicated
+	DeliveryService string
+	CDN             string
+}
+type pgSSLKeyTable struct {
+	Records []pgSSLKeyRecord
+}
+
+func (tbl *pgSSLKeyTable) insertKeys(db *sql.DB) error {
+	queryBase := "INSERT INTO sslkey (deliveryservice, data, cdn, version) VALUES "
+	duplicateKeys := 2
+	stride := 4
+	queryArgs := make([]interface{}, len(tbl.Records)*stride*duplicateKeys)
+	for i, record := range tbl.Records {
+		j := i * duplicateKeys * stride
+
+		queryArgs[j] = record.DeliveryService
+		queryArgs[j+1] = record.DataEncrypted
+		queryArgs[j+2] = record.CDN
+		queryArgs[j+3] = record.Keys.Version.String()
+
+		queryArgs[j+4] = record.DeliveryService
+		queryArgs[j+5] = record.DataEncrypted
+		queryArgs[j+6] = record.CDN
+		queryArgs[j+7] = "latest"
+	}
+	return insertIntoTable(db, queryBase, 4, queryArgs)
+}
+func (tbl *pgSSLKeyTable) gatherKeys(db *sql.DB) error {
+	sz, err := getSize(db, "sslkey WHERE version='latest'")

Review comment:
       Why get only the `latest` version?

##########
File path: tools/traffic_vault_migrate/postgres.go
##########
@@ -0,0 +1,743 @@
+package main
+
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+import (
+	"database/sql"
+	"encoding/base64"
+	"encoding/json"
+	"errors"
+	"fmt"
+	"log"
+	"strconv"
+	"strings"
+
+	"github.com/apache/trafficcontrol/lib/go-tc"
+	util "github.com/apache/trafficcontrol/lib/go-util"
+
+	_ "github.com/lib/pq"
+)
+
+// PGConfig represents the configuration options available to the PG backend
+type PGConfig struct {
+	Host     string `json:"host"`
+	Port     string `json:"port"`
+	User     string `json:"user"`
+	Password string `json:"password"`
+	SSLMode  string `json:"sslmode"`
+	Database string `json:"database"`
+	Key      string `json:"aesKey"`
+	AESKey   []byte
+}
+
+// PGBackend is the Postgres implementation of TVBackend
+type PGBackend struct {
+	sslKey pgSSLKeyTable
+	dnssec pgDNSSecTable
+	uri    pgURISignKeyTable
+	url    pgURLSigKeyTable
+	cfg    PGConfig
+	db     *sql.DB
+}
+
+// String returns a high level overview of the backend and its keys
+func (pg *PGBackend) String() string {
+	data := fmt.Sprintf("PG server %v@%v:%v\n", pg.cfg.User, pg.cfg.Host, pg.cfg.Port)
+	data += fmt.Sprintf("\tSSL Keys: %v\n", len(pg.sslKey.Records))
+	data += fmt.Sprintf("\tDNSSec Keys: %v\n", len(pg.dnssec.Records))
+	data += fmt.Sprintf("\tURI Keys: %v\n", len(pg.uri.Records))
+	data += fmt.Sprintf("\tURL Keys: %v\n", len(pg.url.Records))
+	return data
+}
+
+// Name returns the name for this backend
+func (pg *PGBackend) Name() string {
+	return "PG"
+}
+
+// ReadConfig takes in a filename and will read it into the backends config
+func (pg *PGBackend) ReadConfig(s string) error {
+	err := UnmarshalConfig(s, &pg.cfg)
+	if err != nil {
+		return err
+	}
+
+	pg.cfg.AESKey, err = base64.StdEncoding.DecodeString(pg.cfg.Key)
+	if err != nil {
+		return fmt.Errorf("unable to decode PG AESKey: %w", err)
+	}
+	return nil
+}
+
+// Insert takes the current keys and inserts them into the backend DB
+func (pg *PGBackend) Insert() error {
+	if err := pg.sslKey.insertKeys(pg.db); err != nil {
+		return err
+	}
+	if err := pg.dnssec.insertKeys(pg.db); err != nil {
+		return err
+	}
+	if err := pg.url.insertKeys(pg.db); err != nil {
+		return err
+	}
+	if err := pg.uri.insertKeys(pg.db); err != nil {
+		return err
+	}
+	return nil
+}
+
+// Start initiates the connection to the backend DB
+func (pg *PGBackend) Start() error {
+	sqlStr := fmt.Sprintf("postgres://%s:%s@%s:%s/%s?sslmode=%s", pg.cfg.User, pg.cfg.Password, pg.cfg.Host, pg.cfg.Port, pg.cfg.Database, pg.cfg.SSLMode)
+	db, err := sql.Open("postgres", sqlStr)
+	if err != nil {
+		return fmt.Errorf("unable to start PG client: %w", err)
+	}
+
+	pg.db = db
+	pg.sslKey = pgSSLKeyTable{}
+	pg.dnssec = pgDNSSecTable{}
+	pg.url = pgURLSigKeyTable{}
+	pg.uri = pgURISignKeyTable{}
+
+	return nil
+}
+
+// ValidateKey validates that the keys are valid (in most cases, certain fields are not null)
+func (pg *PGBackend) ValidateKey() []string {
+	var errors []string
+	if errs := pg.sslKey.validate(); errs != nil {
+		errors = append(errors, errs...)
+	}
+	if errs := pg.dnssec.validate(); errs != nil {
+		errors = append(errors, errs...)
+	}
+	if errs := pg.uri.validate(); errs != nil {
+		errors = append(errors, errs...)
+	}
+	if errs := pg.url.validate(); errs != nil {
+		errors = append(errors, errs...)
+	}
+	return errors
+}
+
+// Close terminates the connection to the backend DB
+func (pg *PGBackend) Close() error {
+	return pg.db.Close()
+}
+
+// Ping checks the connection to the backend DB
+func (pg *PGBackend) Ping() error {
+	return pg.db.Ping()
+}
+
+// Fetch gets all of the keys from the backend DB
+func (pg *PGBackend) Fetch() error {
+	if err := pg.sslKey.gatherKeys(pg.db); err != nil {
+		return err
+	}
+
+	if err := pg.dnssec.gatherKeys(pg.db); err != nil {
+		return err
+	}
+
+	if err := pg.url.gatherKeys(pg.db); err != nil {
+		return err
+	}
+
+	if err := pg.uri.gatherKeys(pg.db); err != nil {
+		return err
+	}
+
+	return nil
+}
+
+// GetSSLKeys converts the backends internal key representation into the common representation (SSLKey)
+func (pg *PGBackend) GetSSLKeys() ([]SSLKey, error) {
+	if err := pg.sslKey.decrypt(pg.cfg.AESKey); err != nil {
+		return nil, err
+	}
+	return pg.sslKey.toGeneric(), nil
+}
+
+// SetSSLKeys takes in keys and converts & encrypts the data into the backends internal format
+func (pg *PGBackend) SetSSLKeys(keys []SSLKey) error {
+	pg.sslKey.fromGeneric(keys)
+	return pg.sslKey.encrypt(pg.cfg.AESKey)
+}
+
+// GetDNSSecKeys converts the backends internal key representation into the common representation (DNSSecKey)
+func (pg *PGBackend) GetDNSSecKeys() ([]DNSSecKey, error) {
+	if err := pg.dnssec.decrypt(pg.cfg.AESKey); err != nil {
+		return nil, err
+	}
+	return pg.dnssec.toGeneric(), nil
+}
+
+// SetDNSSecKeys takes in keys and converts & encrypts the data into the backends internal format
+func (pg *PGBackend) SetDNSSecKeys(keys []DNSSecKey) error {
+	pg.dnssec.fromGeneric(keys)
+	return pg.dnssec.encrypt(pg.cfg.AESKey)
+}
+
+// GetURISignKeys converts the pg internal key representation into the common representation (URISignKey)
+func (pg *PGBackend) GetURISignKeys() ([]URISignKey, error) {
+	if err := pg.uri.decrypt(pg.cfg.AESKey); err != nil {
+		return nil, err
+	}
+	return pg.uri.toGeneric(), nil
+}
+
+// SetURISignKeys takes in keys and converts & encrypts the data into the backends internal format
+func (pg *PGBackend) SetURISignKeys(keys []URISignKey) error {
+	pg.uri.fromGeneric(keys)
+	return pg.uri.encrypt(pg.cfg.AESKey)
+}
+
+// GetURLSigKeys converts the backends internal key representation into the common representation (URLSigKey)
+func (pg *PGBackend) GetURLSigKeys() ([]URLSigKey, error) {
+	if err := pg.url.decrypt(pg.cfg.AESKey); err != nil {
+		return nil, err
+	}
+	return pg.url.toGeneric(), nil
+}
+
+// SetURLSigKeys takes in keys and converts & encrypts the data into the backends internal format
+func (pg *PGBackend) SetURLSigKeys(keys []URLSigKey) error {
+	pg.url.fromGeneric(keys)
+	return pg.url.encrypt(pg.cfg.AESKey)
+}
+
+type pgCommonRecord struct {
+	DataEncrypted []byte
+}
+
+type pgDNSSecRecord struct {
+	Key tc.DNSSECKeysTrafficVault
+	CDN string
+	pgCommonRecord
+}
+type pgDNSSecTable struct {
+	Records []pgDNSSecRecord
+}
+
+func (tbl *pgDNSSecTable) gatherKeys(db *sql.DB) error {
+	sz, err := getSize(db, "dnssec")
+	if err != nil {
+		log.Println("PGDNSSec gatherKeys: unable to determine size of dnssec table")
+	}
+	tbl.Records = make([]pgDNSSecRecord, sz)
+
+	rows, err := db.Query("SELECT cdn, data from dnssec")
+	if err != nil {
+		return fmt.Errorf("PGDNSSec gatherKeys: unable to query: %w", err)
+	}
+	defer rows.Close()
+	i := 0
+	for rows.Next() {
+		if i > len(tbl.Records)-1 {
+			return fmt.Errorf("PGDNSSec gatherKeys got more results than expected %v", len(tbl.Records))
+		}
+		err := rows.Scan(&tbl.Records[i].CDN, &tbl.Records[i].DataEncrypted)
+		if err != nil {
+			return fmt.Errorf("PGDNSSec gatherKeys unable to scan row: %w", err)
+		}
+		i += 1
+	}
+	return nil
+}
+func (tbl *pgDNSSecTable) decrypt(aesKey []byte) error {
+	for i, _ := range tbl.Records {
+		err := decryptInto(aesKey, tbl.Records[i].DataEncrypted, &tbl.Records[i].Key)
+		if err != nil {
+			return fmt.Errorf("unable to decrypt into keys: %w", err)
+		}
+	}
+	return nil
+}
+func (tbl *pgDNSSecTable) encrypt(aesKey []byte) error {
+	for i, dns := range tbl.Records {
+		data, err := json.Marshal(&dns.Key)
+		if err != nil {
+			return fmt.Errorf("encrypt issue marshalling keys: %w", err)
+		}
+		dat, err := encrypt(data, aesKey)
+		if err != nil {
+			return fmt.Errorf("encrypt error: %w", err)
+		}
+		tbl.Records[i].DataEncrypted = dat
+	}
+	return nil
+}
+func (tbl *pgDNSSecTable) toGeneric() []DNSSecKey {
+	keys := make([]DNSSecKey, len(tbl.Records))
+
+	for i, record := range tbl.Records {
+		keys[i] = DNSSecKey{
+			CDN:                    record.CDN,
+			DNSSECKeysTrafficVault: record.Key,
+		}
+	}
+
+	return keys
+}
+func (tbl *pgDNSSecTable) fromGeneric(keys []DNSSecKey) {
+	tbl.Records = make([]pgDNSSecRecord, len(keys))
+
+	for i, key := range keys {
+		tbl.Records[i] = pgDNSSecRecord{
+			Key: key.DNSSECKeysTrafficVault,
+			CDN: key.CDN,
+			pgCommonRecord: pgCommonRecord{
+				DataEncrypted: nil,
+			},
+		}
+	}
+}
+func (tbl *pgDNSSecTable) validate() []string {
+	for i, record := range tbl.Records {
+		if record.DataEncrypted == nil && len(record.Key) > 0 {
+			return []string{fmt.Sprintf("DNSSEC Key %v: DataEncrypted is blank!", i)}
+		}
+	}
+	return nil
+}
+func (tbl *pgDNSSecTable) insertKeys(db *sql.DB) error {
+	queryBase := "INSERT INTO dnssec (cdn, data) VALUES "
+	stride := 2
+	queryArgs := make([]interface{}, len(tbl.Records)*stride)
+	for i, record := range tbl.Records {
+		j := i * stride
+		queryArgs[j] = record.CDN
+		queryArgs[j+1] = record.DataEncrypted
+	}
+	return insertIntoTable(db, queryBase, stride, queryArgs)
+}
+
+type pgSSLKeyRecord struct {
+	Keys tc.DeliveryServiceSSLKeys
+	pgCommonRecord
+
+	// These records are stored on the table but are duplicated
+	DeliveryService string
+	CDN             string
+}
+type pgSSLKeyTable struct {
+	Records []pgSSLKeyRecord
+}
+
+func (tbl *pgSSLKeyTable) insertKeys(db *sql.DB) error {
+	queryBase := "INSERT INTO sslkey (deliveryservice, data, cdn, version) VALUES "
+	duplicateKeys := 2
+	stride := 4
+	queryArgs := make([]interface{}, len(tbl.Records)*stride*duplicateKeys)
+	for i, record := range tbl.Records {
+		j := i * duplicateKeys * stride
+
+		queryArgs[j] = record.DeliveryService
+		queryArgs[j+1] = record.DataEncrypted
+		queryArgs[j+2] = record.CDN
+		queryArgs[j+3] = record.Keys.Version.String()
+
+		queryArgs[j+4] = record.DeliveryService
+		queryArgs[j+5] = record.DataEncrypted
+		queryArgs[j+6] = record.CDN
+		queryArgs[j+7] = "latest"
+	}
+	return insertIntoTable(db, queryBase, 4, queryArgs)
+}
+func (tbl *pgSSLKeyTable) gatherKeys(db *sql.DB) error {
+	sz, err := getSize(db, "sslkey WHERE version='latest'")
+	if err != nil {
+		return fmt.Errorf("PGSSLKey gatherKeys unable to determine size of sslkey table: %w", err)
+	}
+	tbl.Records = make([]pgSSLKeyRecord, sz)
+
+	rows, err := db.Query("SELECT data, deliveryservice, cdn from sslkey WHERE version = 'latest'")
+	if err != nil {
+		return fmt.Errorf("PGSSLKey gatherKeys unable to query: %w", err)
+	}
+	defer rows.Close()
+	i := 0
+	for rows.Next() {
+		if i > len(tbl.Records)-1 {
+			return errors.New("PGSSLKey gatherKeys: got more results than expected")
+		}
+		err := rows.Scan(&tbl.Records[i].DataEncrypted, &tbl.Records[i].DeliveryService, &tbl.Records[i].CDN)
+		if err != nil {
+			return fmt.Errorf("PGSSLKey gatherKeys unable to scan row: %w", err)
+		}
+		i += 1
+	}
+	return nil
+}
+func (tbl *pgSSLKeyTable) decrypt(aesKey []byte) error {
+	for i, dns := range tbl.Records {
+		err := decryptInto(aesKey, dns.DataEncrypted, &tbl.Records[i].Keys)
+		if err != nil {
+			return fmt.Errorf("unable to decrypt into keys: %w", err)
+		}
+	}
+	return nil
+}
+func (tbl *pgSSLKeyTable) encrypt(aesKey []byte) error {
+	for i, dns := range tbl.Records {
+		data, err := json.Marshal(dns.Keys)
+		if err != nil {
+			return fmt.Errorf("encrypt issue marshalling keys: %w", err)
+		}
+		dat, err := encrypt(data, aesKey)
+		if err != nil {
+			return fmt.Errorf("encrypt error: %w", err)
+		}
+		tbl.Records[i].DataEncrypted = dat
+	}
+	return nil
+}
+func (tbl *pgSSLKeyTable) toGeneric() []SSLKey {
+	keys := make([]SSLKey, len(tbl.Records))
+
+	for i, record := range tbl.Records {
+		keys[i] = SSLKey{
+			DeliveryServiceSSLKeys: record.Keys,
+		}
+	}
+	return keys
+}
+func (tbl *pgSSLKeyTable) fromGeneric(keys []SSLKey) {
+	tbl.Records = make([]pgSSLKeyRecord, len(keys))
+
+	for i, key := range keys {
+		tbl.Records[i] = pgSSLKeyRecord{
+			Keys: key.DeliveryServiceSSLKeys,
+			pgCommonRecord: pgCommonRecord{
+				DataEncrypted: nil,
+			},
+			DeliveryService: key.DeliveryService,
+			CDN:             key.CDN,
+		}
+	}
+}
+func (tbl *pgSSLKeyTable) validate() []string {
+	defaultKey := tc.DeliveryServiceSSLKeys{}
+	var errors []string
+	fmtStr := "SSL Key %v: %v"
+	for i, record := range tbl.Records {
+		if record.Keys == defaultKey {
+			errors = append(errors, fmt.Sprintf(fmtStr, i, "DS SSL Keys are default!"))
+		} else if record.Keys.Key == "" {
+			errors = append(errors, fmt.Sprintf(fmtStr, i, "Key is blank!"))
+		} else if record.Keys.CDN == "" {
+			errors = append(errors, fmt.Sprintf(fmtStr, i, "CDN is blank!"))
+		} else if record.Keys.DeliveryService == "" {
+			errors = append(errors, fmt.Sprintf(fmtStr, i, "DS is blank!"))
+		} else if record.DataEncrypted == nil {
+			errors = append(errors, fmt.Sprintf(fmtStr, i, "DataEncrypted is blank!"))
+		} else if record.Keys.Version.String() == "" {
+			errors = append(errors, fmt.Sprintf(fmtStr, i, "Version is blank!"))
+		}
+	}
+	return errors
+}
+
+type pgURLSigKeyRecord struct {
+	Keys            tc.URLSigKeys
+	DeliveryService string
+	pgCommonRecord
+}
+type pgURLSigKeyTable struct {
+	Records []pgURLSigKeyRecord
+}
+
+func (tbl *pgURLSigKeyTable) insertKeys(db *sql.DB) error {
+	queryBase := "INSERT INTO url_sig_key (deliveryservice, data) VALUES "
+	stride := 2
+	queryArgs := make([]interface{}, len(tbl.Records)*stride)
+	for i, record := range tbl.Records {
+		j := i * stride
+		queryArgs[j] = record.DeliveryService
+		queryArgs[j+1] = record.DataEncrypted
+	}
+	return insertIntoTable(db, queryBase, stride, queryArgs)
+}
+func (tbl *pgURLSigKeyTable) gatherKeys(db *sql.DB) error {
+	sz, err := getSize(db, "url_sig_key")
+	if err != nil {
+		log.Println("PGURLSigKey gatherKeys: unable to determine url_sig_key table size")
+	}
+	tbl.Records = make([]pgURLSigKeyRecord, sz)
+
+	rows, err := db.Query("SELECT deliveryservice, data from url_sig_key")
+	if err != nil {
+		return fmt.Errorf("PGURLSigKey gatherKeys error creating query: %w", err)
+	}
+	defer rows.Close()
+	i := 0
+	for rows.Next() {
+		if i > len(tbl.Records)-1 {
+			return fmt.Errorf("PGURLSigKey gatherKeys: got more results than expected %v", len(tbl.Records))
+		}
+		err := rows.Scan(&tbl.Records[i].DeliveryService, &tbl.Records[i].DataEncrypted)

Review comment:
       nit: this can be combined with the line below like `if err := function(); err != nil {`

##########
File path: tools/traffic_vault_migrate/postgres.go
##########
@@ -0,0 +1,743 @@
+package main
+
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+import (
+	"database/sql"
+	"encoding/base64"
+	"encoding/json"
+	"errors"
+	"fmt"
+	"log"
+	"strconv"
+	"strings"
+
+	"github.com/apache/trafficcontrol/lib/go-tc"
+	util "github.com/apache/trafficcontrol/lib/go-util"
+
+	_ "github.com/lib/pq"
+)
+
+// PGConfig represents the configuration options available to the PG backend
+type PGConfig struct {
+	Host     string `json:"host"`
+	Port     string `json:"port"`
+	User     string `json:"user"`
+	Password string `json:"password"`
+	SSLMode  string `json:"sslmode"`
+	Database string `json:"database"`
+	Key      string `json:"aesKey"`
+	AESKey   []byte
+}
+
+// PGBackend is the Postgres implementation of TVBackend
+type PGBackend struct {
+	sslKey pgSSLKeyTable
+	dnssec pgDNSSecTable
+	uri    pgURISignKeyTable
+	url    pgURLSigKeyTable
+	cfg    PGConfig
+	db     *sql.DB
+}
+
+// String returns a high level overview of the backend and its keys
+func (pg *PGBackend) String() string {
+	data := fmt.Sprintf("PG server %v@%v:%v\n", pg.cfg.User, pg.cfg.Host, pg.cfg.Port)
+	data += fmt.Sprintf("\tSSL Keys: %v\n", len(pg.sslKey.Records))
+	data += fmt.Sprintf("\tDNSSec Keys: %v\n", len(pg.dnssec.Records))
+	data += fmt.Sprintf("\tURI Keys: %v\n", len(pg.uri.Records))
+	data += fmt.Sprintf("\tURL Keys: %v\n", len(pg.url.Records))
+	return data
+}
+
+// Name returns the name for this backend
+func (pg *PGBackend) Name() string {
+	return "PG"
+}
+
+// ReadConfig takes in a filename and will read it into the backends config
+func (pg *PGBackend) ReadConfig(s string) error {
+	err := UnmarshalConfig(s, &pg.cfg)
+	if err != nil {
+		return err
+	}
+
+	pg.cfg.AESKey, err = base64.StdEncoding.DecodeString(pg.cfg.Key)
+	if err != nil {
+		return fmt.Errorf("unable to decode PG AESKey: %w", err)
+	}
+	return nil
+}
+
+// Insert takes the current keys and inserts them into the backend DB
+func (pg *PGBackend) Insert() error {
+	if err := pg.sslKey.insertKeys(pg.db); err != nil {
+		return err
+	}
+	if err := pg.dnssec.insertKeys(pg.db); err != nil {
+		return err
+	}
+	if err := pg.url.insertKeys(pg.db); err != nil {
+		return err
+	}
+	if err := pg.uri.insertKeys(pg.db); err != nil {
+		return err
+	}
+	return nil
+}
+
+// Start initiates the connection to the backend DB
+func (pg *PGBackend) Start() error {
+	sqlStr := fmt.Sprintf("postgres://%s:%s@%s:%s/%s?sslmode=%s", pg.cfg.User, pg.cfg.Password, pg.cfg.Host, pg.cfg.Port, pg.cfg.Database, pg.cfg.SSLMode)
+	db, err := sql.Open("postgres", sqlStr)
+	if err != nil {
+		return fmt.Errorf("unable to start PG client: %w", err)
+	}
+
+	pg.db = db
+	pg.sslKey = pgSSLKeyTable{}
+	pg.dnssec = pgDNSSecTable{}
+	pg.url = pgURLSigKeyTable{}
+	pg.uri = pgURISignKeyTable{}
+
+	return nil
+}
+
+// ValidateKey validates that the keys are valid (in most cases, certain fields are not null)
+func (pg *PGBackend) ValidateKey() []string {
+	var errors []string
+	if errs := pg.sslKey.validate(); errs != nil {
+		errors = append(errors, errs...)
+	}
+	if errs := pg.dnssec.validate(); errs != nil {
+		errors = append(errors, errs...)
+	}
+	if errs := pg.uri.validate(); errs != nil {
+		errors = append(errors, errs...)
+	}
+	if errs := pg.url.validate(); errs != nil {
+		errors = append(errors, errs...)
+	}
+	return errors
+}
+
+// Close terminates the connection to the backend DB
+func (pg *PGBackend) Close() error {
+	return pg.db.Close()
+}
+
+// Ping checks the connection to the backend DB
+func (pg *PGBackend) Ping() error {
+	return pg.db.Ping()
+}
+
+// Fetch gets all of the keys from the backend DB
+func (pg *PGBackend) Fetch() error {
+	if err := pg.sslKey.gatherKeys(pg.db); err != nil {
+		return err
+	}
+
+	if err := pg.dnssec.gatherKeys(pg.db); err != nil {
+		return err
+	}
+
+	if err := pg.url.gatherKeys(pg.db); err != nil {
+		return err
+	}
+
+	if err := pg.uri.gatherKeys(pg.db); err != nil {
+		return err
+	}
+
+	return nil
+}
+
+// GetSSLKeys converts the backends internal key representation into the common representation (SSLKey)
+func (pg *PGBackend) GetSSLKeys() ([]SSLKey, error) {
+	if err := pg.sslKey.decrypt(pg.cfg.AESKey); err != nil {
+		return nil, err
+	}
+	return pg.sslKey.toGeneric(), nil
+}
+
+// SetSSLKeys takes in keys and converts & encrypts the data into the backends internal format
+func (pg *PGBackend) SetSSLKeys(keys []SSLKey) error {
+	pg.sslKey.fromGeneric(keys)
+	return pg.sslKey.encrypt(pg.cfg.AESKey)
+}
+
+// GetDNSSecKeys converts the backends internal key representation into the common representation (DNSSecKey)
+func (pg *PGBackend) GetDNSSecKeys() ([]DNSSecKey, error) {
+	if err := pg.dnssec.decrypt(pg.cfg.AESKey); err != nil {
+		return nil, err
+	}
+	return pg.dnssec.toGeneric(), nil
+}
+
+// SetDNSSecKeys takes in keys and converts & encrypts the data into the backends internal format
+func (pg *PGBackend) SetDNSSecKeys(keys []DNSSecKey) error {
+	pg.dnssec.fromGeneric(keys)
+	return pg.dnssec.encrypt(pg.cfg.AESKey)
+}
+
+// GetURISignKeys converts the pg internal key representation into the common representation (URISignKey)
+func (pg *PGBackend) GetURISignKeys() ([]URISignKey, error) {
+	if err := pg.uri.decrypt(pg.cfg.AESKey); err != nil {
+		return nil, err
+	}
+	return pg.uri.toGeneric(), nil
+}
+
+// SetURISignKeys takes in keys and converts & encrypts the data into the backends internal format
+func (pg *PGBackend) SetURISignKeys(keys []URISignKey) error {
+	pg.uri.fromGeneric(keys)
+	return pg.uri.encrypt(pg.cfg.AESKey)
+}
+
+// GetURLSigKeys converts the backends internal key representation into the common representation (URLSigKey)
+func (pg *PGBackend) GetURLSigKeys() ([]URLSigKey, error) {
+	if err := pg.url.decrypt(pg.cfg.AESKey); err != nil {
+		return nil, err
+	}
+	return pg.url.toGeneric(), nil
+}
+
+// SetURLSigKeys takes in keys and converts & encrypts the data into the backends internal format
+func (pg *PGBackend) SetURLSigKeys(keys []URLSigKey) error {
+	pg.url.fromGeneric(keys)
+	return pg.url.encrypt(pg.cfg.AESKey)
+}
+
+type pgCommonRecord struct {
+	DataEncrypted []byte
+}
+
+type pgDNSSecRecord struct {
+	Key tc.DNSSECKeysTrafficVault
+	CDN string
+	pgCommonRecord
+}
+type pgDNSSecTable struct {
+	Records []pgDNSSecRecord
+}
+
+func (tbl *pgDNSSecTable) gatherKeys(db *sql.DB) error {
+	sz, err := getSize(db, "dnssec")
+	if err != nil {
+		log.Println("PGDNSSec gatherKeys: unable to determine size of dnssec table")
+	}
+	tbl.Records = make([]pgDNSSecRecord, sz)
+
+	rows, err := db.Query("SELECT cdn, data from dnssec")
+	if err != nil {
+		return fmt.Errorf("PGDNSSec gatherKeys: unable to query: %w", err)
+	}
+	defer rows.Close()
+	i := 0
+	for rows.Next() {
+		if i > len(tbl.Records)-1 {
+			return fmt.Errorf("PGDNSSec gatherKeys got more results than expected %v", len(tbl.Records))
+		}
+		err := rows.Scan(&tbl.Records[i].CDN, &tbl.Records[i].DataEncrypted)
+		if err != nil {
+			return fmt.Errorf("PGDNSSec gatherKeys unable to scan row: %w", err)
+		}
+		i += 1
+	}
+	return nil
+}
+func (tbl *pgDNSSecTable) decrypt(aesKey []byte) error {
+	for i, _ := range tbl.Records {
+		err := decryptInto(aesKey, tbl.Records[i].DataEncrypted, &tbl.Records[i].Key)
+		if err != nil {
+			return fmt.Errorf("unable to decrypt into keys: %w", err)
+		}
+	}
+	return nil
+}
+func (tbl *pgDNSSecTable) encrypt(aesKey []byte) error {
+	for i, dns := range tbl.Records {
+		data, err := json.Marshal(&dns.Key)
+		if err != nil {
+			return fmt.Errorf("encrypt issue marshalling keys: %w", err)
+		}
+		dat, err := encrypt(data, aesKey)
+		if err != nil {
+			return fmt.Errorf("encrypt error: %w", err)
+		}
+		tbl.Records[i].DataEncrypted = dat
+	}
+	return nil
+}
+func (tbl *pgDNSSecTable) toGeneric() []DNSSecKey {
+	keys := make([]DNSSecKey, len(tbl.Records))
+
+	for i, record := range tbl.Records {
+		keys[i] = DNSSecKey{
+			CDN:                    record.CDN,
+			DNSSECKeysTrafficVault: record.Key,
+		}
+	}
+
+	return keys
+}
+func (tbl *pgDNSSecTable) fromGeneric(keys []DNSSecKey) {
+	tbl.Records = make([]pgDNSSecRecord, len(keys))
+
+	for i, key := range keys {
+		tbl.Records[i] = pgDNSSecRecord{
+			Key: key.DNSSECKeysTrafficVault,
+			CDN: key.CDN,
+			pgCommonRecord: pgCommonRecord{
+				DataEncrypted: nil,
+			},
+		}
+	}
+}
+func (tbl *pgDNSSecTable) validate() []string {
+	for i, record := range tbl.Records {
+		if record.DataEncrypted == nil && len(record.Key) > 0 {
+			return []string{fmt.Sprintf("DNSSEC Key %v: DataEncrypted is blank!", i)}
+		}
+	}
+	return nil
+}
+func (tbl *pgDNSSecTable) insertKeys(db *sql.DB) error {
+	queryBase := "INSERT INTO dnssec (cdn, data) VALUES "
+	stride := 2
+	queryArgs := make([]interface{}, len(tbl.Records)*stride)
+	for i, record := range tbl.Records {
+		j := i * stride
+		queryArgs[j] = record.CDN
+		queryArgs[j+1] = record.DataEncrypted
+	}
+	return insertIntoTable(db, queryBase, stride, queryArgs)
+}
+
+type pgSSLKeyRecord struct {
+	Keys tc.DeliveryServiceSSLKeys
+	pgCommonRecord
+
+	// These records are stored on the table but are duplicated
+	DeliveryService string
+	CDN             string
+}
+type pgSSLKeyTable struct {
+	Records []pgSSLKeyRecord
+}
+
+func (tbl *pgSSLKeyTable) insertKeys(db *sql.DB) error {
+	queryBase := "INSERT INTO sslkey (deliveryservice, data, cdn, version) VALUES "
+	duplicateKeys := 2
+	stride := 4
+	queryArgs := make([]interface{}, len(tbl.Records)*stride*duplicateKeys)
+	for i, record := range tbl.Records {
+		j := i * duplicateKeys * stride
+
+		queryArgs[j] = record.DeliveryService
+		queryArgs[j+1] = record.DataEncrypted
+		queryArgs[j+2] = record.CDN
+		queryArgs[j+3] = record.Keys.Version.String()
+
+		queryArgs[j+4] = record.DeliveryService
+		queryArgs[j+5] = record.DataEncrypted
+		queryArgs[j+6] = record.CDN
+		queryArgs[j+7] = "latest"
+	}
+	return insertIntoTable(db, queryBase, 4, queryArgs)
+}
+func (tbl *pgSSLKeyTable) gatherKeys(db *sql.DB) error {
+	sz, err := getSize(db, "sslkey WHERE version='latest'")
+	if err != nil {
+		return fmt.Errorf("PGSSLKey gatherKeys unable to determine size of sslkey table: %w", err)
+	}
+	tbl.Records = make([]pgSSLKeyRecord, sz)
+
+	rows, err := db.Query("SELECT data, deliveryservice, cdn from sslkey WHERE version = 'latest'")
+	if err != nil {
+		return fmt.Errorf("PGSSLKey gatherKeys unable to query: %w", err)
+	}
+	defer rows.Close()
+	i := 0
+	for rows.Next() {
+		if i > len(tbl.Records)-1 {
+			return errors.New("PGSSLKey gatherKeys: got more results than expected")
+		}
+		err := rows.Scan(&tbl.Records[i].DataEncrypted, &tbl.Records[i].DeliveryService, &tbl.Records[i].CDN)
+		if err != nil {
+			return fmt.Errorf("PGSSLKey gatherKeys unable to scan row: %w", err)
+		}
+		i += 1
+	}
+	return nil
+}
+func (tbl *pgSSLKeyTable) decrypt(aesKey []byte) error {
+	for i, dns := range tbl.Records {
+		err := decryptInto(aesKey, dns.DataEncrypted, &tbl.Records[i].Keys)
+		if err != nil {
+			return fmt.Errorf("unable to decrypt into keys: %w", err)
+		}
+	}
+	return nil
+}
+func (tbl *pgSSLKeyTable) encrypt(aesKey []byte) error {
+	for i, dns := range tbl.Records {
+		data, err := json.Marshal(dns.Keys)
+		if err != nil {
+			return fmt.Errorf("encrypt issue marshalling keys: %w", err)
+		}
+		dat, err := encrypt(data, aesKey)
+		if err != nil {
+			return fmt.Errorf("encrypt error: %w", err)
+		}
+		tbl.Records[i].DataEncrypted = dat
+	}
+	return nil
+}
+func (tbl *pgSSLKeyTable) toGeneric() []SSLKey {
+	keys := make([]SSLKey, len(tbl.Records))
+
+	for i, record := range tbl.Records {
+		keys[i] = SSLKey{
+			DeliveryServiceSSLKeys: record.Keys,
+		}
+	}
+	return keys
+}
+func (tbl *pgSSLKeyTable) fromGeneric(keys []SSLKey) {
+	tbl.Records = make([]pgSSLKeyRecord, len(keys))
+
+	for i, key := range keys {
+		tbl.Records[i] = pgSSLKeyRecord{
+			Keys: key.DeliveryServiceSSLKeys,
+			pgCommonRecord: pgCommonRecord{
+				DataEncrypted: nil,
+			},
+			DeliveryService: key.DeliveryService,
+			CDN:             key.CDN,
+		}
+	}
+}
+func (tbl *pgSSLKeyTable) validate() []string {
+	defaultKey := tc.DeliveryServiceSSLKeys{}
+	var errors []string
+	fmtStr := "SSL Key %v: %v"
+	for i, record := range tbl.Records {
+		if record.Keys == defaultKey {
+			errors = append(errors, fmt.Sprintf(fmtStr, i, "DS SSL Keys are default!"))
+		} else if record.Keys.Key == "" {
+			errors = append(errors, fmt.Sprintf(fmtStr, i, "Key is blank!"))
+		} else if record.Keys.CDN == "" {
+			errors = append(errors, fmt.Sprintf(fmtStr, i, "CDN is blank!"))
+		} else if record.Keys.DeliveryService == "" {
+			errors = append(errors, fmt.Sprintf(fmtStr, i, "DS is blank!"))
+		} else if record.DataEncrypted == nil {
+			errors = append(errors, fmt.Sprintf(fmtStr, i, "DataEncrypted is blank!"))
+		} else if record.Keys.Version.String() == "" {
+			errors = append(errors, fmt.Sprintf(fmtStr, i, "Version is blank!"))
+		}
+	}
+	return errors
+}
+
+type pgURLSigKeyRecord struct {
+	Keys            tc.URLSigKeys
+	DeliveryService string
+	pgCommonRecord
+}
+type pgURLSigKeyTable struct {
+	Records []pgURLSigKeyRecord
+}
+
+func (tbl *pgURLSigKeyTable) insertKeys(db *sql.DB) error {
+	queryBase := "INSERT INTO url_sig_key (deliveryservice, data) VALUES "
+	stride := 2
+	queryArgs := make([]interface{}, len(tbl.Records)*stride)
+	for i, record := range tbl.Records {
+		j := i * stride
+		queryArgs[j] = record.DeliveryService
+		queryArgs[j+1] = record.DataEncrypted
+	}
+	return insertIntoTable(db, queryBase, stride, queryArgs)
+}
+func (tbl *pgURLSigKeyTable) gatherKeys(db *sql.DB) error {
+	sz, err := getSize(db, "url_sig_key")
+	if err != nil {
+		log.Println("PGURLSigKey gatherKeys: unable to determine url_sig_key table size")
+	}
+	tbl.Records = make([]pgURLSigKeyRecord, sz)
+
+	rows, err := db.Query("SELECT deliveryservice, data from url_sig_key")
+	if err != nil {
+		return fmt.Errorf("PGURLSigKey gatherKeys error creating query: %w", err)
+	}
+	defer rows.Close()
+	i := 0
+	for rows.Next() {
+		if i > len(tbl.Records)-1 {
+			return fmt.Errorf("PGURLSigKey gatherKeys: got more results than expected %v", len(tbl.Records))
+		}
+		err := rows.Scan(&tbl.Records[i].DeliveryService, &tbl.Records[i].DataEncrypted)
+		if err != nil {
+			return fmt.Errorf("PGURLSigKey gatherKeys: unable to scan row: %w", err)
+		}
+		i += 1
+	}
+	return nil
+}
+func (tbl *pgURLSigKeyTable) decrypt(aesKey []byte) error {
+	for i, sig := range tbl.Records {
+		err := decryptInto(aesKey, sig.DataEncrypted, &tbl.Records[i].Keys)
+		if err != nil {
+			return fmt.Errorf("unable to decrypt into keys: %w", err)
+		}
+	}
+	return nil
+}
+func (tbl *pgURLSigKeyTable) encrypt(aesKey []byte) error {
+	for i, sig := range tbl.Records {
+		data, err := json.Marshal(&sig.Keys)
+		if err != nil {
+			return fmt.Errorf("encrypt issue marshalling keys: %w", err)
+		}
+
+		dat, err := encrypt(data, aesKey)
+		if err != nil {
+			return fmt.Errorf("encrypt error: %w", err)
+		}
+		tbl.Records[i].DataEncrypted = dat
+	}
+	return nil
+}
+func (tbl *pgURLSigKeyTable) toGeneric() []URLSigKey {
+	keys := make([]URLSigKey, len(tbl.Records))
+
+	for i, record := range tbl.Records {
+		keys[i] = URLSigKey{
+			DeliveryService: record.DeliveryService,
+			URLSigKeys:      record.Keys,
+		}
+	}
+	return keys
+}
+func (tbl *pgURLSigKeyTable) fromGeneric(keys []URLSigKey) {
+	tbl.Records = make([]pgURLSigKeyRecord, len(keys))
+
+	for i, key := range keys {
+		tbl.Records[i] = pgURLSigKeyRecord{
+			Keys:            key.URLSigKeys,
+			DeliveryService: key.DeliveryService,
+			pgCommonRecord: pgCommonRecord{
+				DataEncrypted: nil,
+			},
+		}
+	}
+}
+func (tbl *pgURLSigKeyTable) validate() []string {
+	for i, record := range tbl.Records {
+		if record.DataEncrypted == nil && len(record.Keys) > 0 {
+			return []string{fmt.Sprintf("URl Sig Key %v: DataEncrypted is blank!", i)}
+		}
+	}
+	return nil
+}
+
+type pgURISignKeyRecord struct {
+	Keys            map[string]tc.URISignerKeyset
+	DeliveryService string
+	pgCommonRecord
+}
+type pgURISignKeyTable struct {
+	Records []pgURISignKeyRecord
+}
+
+func (tbl *pgURISignKeyTable) insertKeys(db *sql.DB) error {
+	queryBase := "INSERT INTO uri_signing_key (deliveryservice, data) VALUES "
+	stride := 2
+	queryArgs := make([]interface{}, len(tbl.Records)*stride)
+	for i, record := range tbl.Records {
+		j := i * stride
+		queryArgs[j] = record.DeliveryService
+		queryArgs[j+1] = record.DataEncrypted
+	}
+	return insertIntoTable(db, queryBase, stride, queryArgs)
+}
+func (tbl *pgURISignKeyTable) gatherKeys(db *sql.DB) error {
+	sz, err := getSize(db, "uri_signing_key")
+	if err != nil {
+		log.Println("PGURISignKey gatherKeys: unable to determine size of uri_signing_key table")
+	}
+	tbl.Records = make([]pgURISignKeyRecord, sz)
+
+	rows, err := db.Query("SELECT deliveryservice, data from uri_signing_key")
+	if err != nil {
+		return fmt.Errorf("PGURISignKey gatherKeys error while query: %w", err)
+	}
+	defer rows.Close()
+	i := 0
+	for rows.Next() {
+		if i > len(tbl.Records)-1 {
+			return fmt.Errorf("PGURISignKey gatherKeys: got more results than expected %v", len(tbl.Records))
+		}
+		err := rows.Scan(&tbl.Records[i].DeliveryService, &tbl.Records[i].DataEncrypted)
+		if err != nil {
+			return fmt.Errorf("PGURISignKey gatherKeys: unable to scan row: %w", err)
+		}
+		i += 1
+	}
+	return nil
+}
+func (tbl *pgURISignKeyTable) decrypt(aesKey []byte) error {
+	for i, sign := range tbl.Records {
+		err := decryptInto(aesKey, sign.DataEncrypted, &tbl.Records[i].Keys)

Review comment:
       nit: this can be combined with the line below like `if err := function(); err != nil {`

##########
File path: tools/traffic_vault_migrate/postgres.go
##########
@@ -0,0 +1,743 @@
+package main
+
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+import (
+	"database/sql"
+	"encoding/base64"
+	"encoding/json"
+	"errors"
+	"fmt"
+	"log"
+	"strconv"
+	"strings"
+
+	"github.com/apache/trafficcontrol/lib/go-tc"
+	util "github.com/apache/trafficcontrol/lib/go-util"
+
+	_ "github.com/lib/pq"
+)
+
+// PGConfig represents the configuration options available to the PG backend
+type PGConfig struct {
+	Host     string `json:"host"`
+	Port     string `json:"port"`
+	User     string `json:"user"`
+	Password string `json:"password"`
+	SSLMode  string `json:"sslmode"`
+	Database string `json:"database"`
+	Key      string `json:"aesKey"`
+	AESKey   []byte
+}
+
+// PGBackend is the Postgres implementation of TVBackend
+type PGBackend struct {
+	sslKey pgSSLKeyTable
+	dnssec pgDNSSecTable
+	uri    pgURISignKeyTable
+	url    pgURLSigKeyTable
+	cfg    PGConfig
+	db     *sql.DB
+}
+
+// String returns a high level overview of the backend and its keys
+func (pg *PGBackend) String() string {
+	data := fmt.Sprintf("PG server %v@%v:%v\n", pg.cfg.User, pg.cfg.Host, pg.cfg.Port)
+	data += fmt.Sprintf("\tSSL Keys: %v\n", len(pg.sslKey.Records))
+	data += fmt.Sprintf("\tDNSSec Keys: %v\n", len(pg.dnssec.Records))
+	data += fmt.Sprintf("\tURI Keys: %v\n", len(pg.uri.Records))
+	data += fmt.Sprintf("\tURL Keys: %v\n", len(pg.url.Records))
+	return data
+}
+
+// Name returns the name for this backend
+func (pg *PGBackend) Name() string {
+	return "PG"
+}
+
+// ReadConfig takes in a filename and will read it into the backends config
+func (pg *PGBackend) ReadConfig(s string) error {
+	err := UnmarshalConfig(s, &pg.cfg)
+	if err != nil {
+		return err
+	}
+
+	pg.cfg.AESKey, err = base64.StdEncoding.DecodeString(pg.cfg.Key)
+	if err != nil {
+		return fmt.Errorf("unable to decode PG AESKey: %w", err)
+	}
+	return nil
+}
+
+// Insert takes the current keys and inserts them into the backend DB
+func (pg *PGBackend) Insert() error {
+	if err := pg.sslKey.insertKeys(pg.db); err != nil {
+		return err
+	}
+	if err := pg.dnssec.insertKeys(pg.db); err != nil {
+		return err
+	}
+	if err := pg.url.insertKeys(pg.db); err != nil {
+		return err
+	}
+	if err := pg.uri.insertKeys(pg.db); err != nil {
+		return err
+	}
+	return nil
+}
+
+// Start initiates the connection to the backend DB
+func (pg *PGBackend) Start() error {
+	sqlStr := fmt.Sprintf("postgres://%s:%s@%s:%s/%s?sslmode=%s", pg.cfg.User, pg.cfg.Password, pg.cfg.Host, pg.cfg.Port, pg.cfg.Database, pg.cfg.SSLMode)
+	db, err := sql.Open("postgres", sqlStr)
+	if err != nil {
+		return fmt.Errorf("unable to start PG client: %w", err)
+	}
+
+	pg.db = db
+	pg.sslKey = pgSSLKeyTable{}
+	pg.dnssec = pgDNSSecTable{}
+	pg.url = pgURLSigKeyTable{}
+	pg.uri = pgURISignKeyTable{}
+
+	return nil
+}
+
+// ValidateKey validates that the keys are valid (in most cases, certain fields are not null)
+func (pg *PGBackend) ValidateKey() []string {
+	var errors []string
+	if errs := pg.sslKey.validate(); errs != nil {
+		errors = append(errors, errs...)
+	}
+	if errs := pg.dnssec.validate(); errs != nil {
+		errors = append(errors, errs...)
+	}
+	if errs := pg.uri.validate(); errs != nil {
+		errors = append(errors, errs...)
+	}
+	if errs := pg.url.validate(); errs != nil {
+		errors = append(errors, errs...)
+	}
+	return errors
+}
+
+// Close terminates the connection to the backend DB
+func (pg *PGBackend) Close() error {
+	return pg.db.Close()
+}
+
+// Ping checks the connection to the backend DB
+func (pg *PGBackend) Ping() error {
+	return pg.db.Ping()
+}
+
+// Fetch gets all of the keys from the backend DB
+func (pg *PGBackend) Fetch() error {
+	if err := pg.sslKey.gatherKeys(pg.db); err != nil {
+		return err
+	}
+
+	if err := pg.dnssec.gatherKeys(pg.db); err != nil {
+		return err
+	}
+
+	if err := pg.url.gatherKeys(pg.db); err != nil {
+		return err
+	}
+
+	if err := pg.uri.gatherKeys(pg.db); err != nil {
+		return err
+	}
+
+	return nil
+}
+
+// GetSSLKeys converts the backends internal key representation into the common representation (SSLKey)
+func (pg *PGBackend) GetSSLKeys() ([]SSLKey, error) {
+	if err := pg.sslKey.decrypt(pg.cfg.AESKey); err != nil {
+		return nil, err
+	}
+	return pg.sslKey.toGeneric(), nil
+}
+
+// SetSSLKeys takes in keys and converts & encrypts the data into the backends internal format
+func (pg *PGBackend) SetSSLKeys(keys []SSLKey) error {
+	pg.sslKey.fromGeneric(keys)
+	return pg.sslKey.encrypt(pg.cfg.AESKey)
+}
+
+// GetDNSSecKeys converts the backends internal key representation into the common representation (DNSSecKey)
+func (pg *PGBackend) GetDNSSecKeys() ([]DNSSecKey, error) {
+	if err := pg.dnssec.decrypt(pg.cfg.AESKey); err != nil {
+		return nil, err
+	}
+	return pg.dnssec.toGeneric(), nil
+}
+
+// SetDNSSecKeys takes in keys and converts & encrypts the data into the backends internal format
+func (pg *PGBackend) SetDNSSecKeys(keys []DNSSecKey) error {
+	pg.dnssec.fromGeneric(keys)
+	return pg.dnssec.encrypt(pg.cfg.AESKey)
+}
+
+// GetURISignKeys converts the pg internal key representation into the common representation (URISignKey)
+func (pg *PGBackend) GetURISignKeys() ([]URISignKey, error) {
+	if err := pg.uri.decrypt(pg.cfg.AESKey); err != nil {
+		return nil, err
+	}
+	return pg.uri.toGeneric(), nil
+}
+
+// SetURISignKeys takes in keys and converts & encrypts the data into the backends internal format
+func (pg *PGBackend) SetURISignKeys(keys []URISignKey) error {
+	pg.uri.fromGeneric(keys)
+	return pg.uri.encrypt(pg.cfg.AESKey)
+}
+
+// GetURLSigKeys converts the backends internal key representation into the common representation (URLSigKey)
+func (pg *PGBackend) GetURLSigKeys() ([]URLSigKey, error) {
+	if err := pg.url.decrypt(pg.cfg.AESKey); err != nil {
+		return nil, err
+	}
+	return pg.url.toGeneric(), nil
+}
+
+// SetURLSigKeys takes in keys and converts & encrypts the data into the backends internal format
+func (pg *PGBackend) SetURLSigKeys(keys []URLSigKey) error {
+	pg.url.fromGeneric(keys)
+	return pg.url.encrypt(pg.cfg.AESKey)
+}
+
+type pgCommonRecord struct {
+	DataEncrypted []byte
+}
+
+type pgDNSSecRecord struct {
+	Key tc.DNSSECKeysTrafficVault
+	CDN string
+	pgCommonRecord
+}
+type pgDNSSecTable struct {
+	Records []pgDNSSecRecord
+}
+
+func (tbl *pgDNSSecTable) gatherKeys(db *sql.DB) error {
+	sz, err := getSize(db, "dnssec")
+	if err != nil {
+		log.Println("PGDNSSec gatherKeys: unable to determine size of dnssec table")
+	}
+	tbl.Records = make([]pgDNSSecRecord, sz)
+
+	rows, err := db.Query("SELECT cdn, data from dnssec")
+	if err != nil {
+		return fmt.Errorf("PGDNSSec gatherKeys: unable to query: %w", err)
+	}
+	defer rows.Close()
+	i := 0
+	for rows.Next() {
+		if i > len(tbl.Records)-1 {
+			return fmt.Errorf("PGDNSSec gatherKeys got more results than expected %v", len(tbl.Records))
+		}
+		err := rows.Scan(&tbl.Records[i].CDN, &tbl.Records[i].DataEncrypted)
+		if err != nil {
+			return fmt.Errorf("PGDNSSec gatherKeys unable to scan row: %w", err)
+		}
+		i += 1
+	}
+	return nil
+}
+func (tbl *pgDNSSecTable) decrypt(aesKey []byte) error {
+	for i, _ := range tbl.Records {
+		err := decryptInto(aesKey, tbl.Records[i].DataEncrypted, &tbl.Records[i].Key)
+		if err != nil {
+			return fmt.Errorf("unable to decrypt into keys: %w", err)
+		}
+	}
+	return nil
+}
+func (tbl *pgDNSSecTable) encrypt(aesKey []byte) error {
+	for i, dns := range tbl.Records {
+		data, err := json.Marshal(&dns.Key)
+		if err != nil {
+			return fmt.Errorf("encrypt issue marshalling keys: %w", err)
+		}
+		dat, err := encrypt(data, aesKey)
+		if err != nil {
+			return fmt.Errorf("encrypt error: %w", err)
+		}
+		tbl.Records[i].DataEncrypted = dat
+	}
+	return nil
+}
+func (tbl *pgDNSSecTable) toGeneric() []DNSSecKey {
+	keys := make([]DNSSecKey, len(tbl.Records))
+
+	for i, record := range tbl.Records {
+		keys[i] = DNSSecKey{
+			CDN:                    record.CDN,
+			DNSSECKeysTrafficVault: record.Key,
+		}
+	}
+
+	return keys
+}
+func (tbl *pgDNSSecTable) fromGeneric(keys []DNSSecKey) {
+	tbl.Records = make([]pgDNSSecRecord, len(keys))
+
+	for i, key := range keys {
+		tbl.Records[i] = pgDNSSecRecord{
+			Key: key.DNSSECKeysTrafficVault,
+			CDN: key.CDN,
+			pgCommonRecord: pgCommonRecord{
+				DataEncrypted: nil,
+			},
+		}
+	}
+}
+func (tbl *pgDNSSecTable) validate() []string {
+	for i, record := range tbl.Records {
+		if record.DataEncrypted == nil && len(record.Key) > 0 {
+			return []string{fmt.Sprintf("DNSSEC Key %v: DataEncrypted is blank!", i)}
+		}
+	}
+	return nil
+}
+func (tbl *pgDNSSecTable) insertKeys(db *sql.DB) error {
+	queryBase := "INSERT INTO dnssec (cdn, data) VALUES "
+	stride := 2
+	queryArgs := make([]interface{}, len(tbl.Records)*stride)
+	for i, record := range tbl.Records {
+		j := i * stride
+		queryArgs[j] = record.CDN
+		queryArgs[j+1] = record.DataEncrypted
+	}
+	return insertIntoTable(db, queryBase, stride, queryArgs)
+}
+
+type pgSSLKeyRecord struct {
+	Keys tc.DeliveryServiceSSLKeys
+	pgCommonRecord
+
+	// These records are stored on the table but are duplicated
+	DeliveryService string
+	CDN             string
+}
+type pgSSLKeyTable struct {
+	Records []pgSSLKeyRecord
+}
+
+func (tbl *pgSSLKeyTable) insertKeys(db *sql.DB) error {
+	queryBase := "INSERT INTO sslkey (deliveryservice, data, cdn, version) VALUES "
+	duplicateKeys := 2
+	stride := 4
+	queryArgs := make([]interface{}, len(tbl.Records)*stride*duplicateKeys)
+	for i, record := range tbl.Records {
+		j := i * duplicateKeys * stride
+
+		queryArgs[j] = record.DeliveryService
+		queryArgs[j+1] = record.DataEncrypted
+		queryArgs[j+2] = record.CDN
+		queryArgs[j+3] = record.Keys.Version.String()
+
+		queryArgs[j+4] = record.DeliveryService
+		queryArgs[j+5] = record.DataEncrypted
+		queryArgs[j+6] = record.CDN
+		queryArgs[j+7] = "latest"
+	}
+	return insertIntoTable(db, queryBase, 4, queryArgs)
+}
+func (tbl *pgSSLKeyTable) gatherKeys(db *sql.DB) error {
+	sz, err := getSize(db, "sslkey WHERE version='latest'")
+	if err != nil {
+		return fmt.Errorf("PGSSLKey gatherKeys unable to determine size of sslkey table: %w", err)
+	}
+	tbl.Records = make([]pgSSLKeyRecord, sz)
+
+	rows, err := db.Query("SELECT data, deliveryservice, cdn from sslkey WHERE version = 'latest'")
+	if err != nil {
+		return fmt.Errorf("PGSSLKey gatherKeys unable to query: %w", err)
+	}
+	defer rows.Close()
+	i := 0
+	for rows.Next() {
+		if i > len(tbl.Records)-1 {
+			return errors.New("PGSSLKey gatherKeys: got more results than expected")
+		}
+		err := rows.Scan(&tbl.Records[i].DataEncrypted, &tbl.Records[i].DeliveryService, &tbl.Records[i].CDN)
+		if err != nil {
+			return fmt.Errorf("PGSSLKey gatherKeys unable to scan row: %w", err)
+		}
+		i += 1
+	}
+	return nil
+}
+func (tbl *pgSSLKeyTable) decrypt(aesKey []byte) error {
+	for i, dns := range tbl.Records {
+		err := decryptInto(aesKey, dns.DataEncrypted, &tbl.Records[i].Keys)
+		if err != nil {
+			return fmt.Errorf("unable to decrypt into keys: %w", err)
+		}
+	}
+	return nil
+}
+func (tbl *pgSSLKeyTable) encrypt(aesKey []byte) error {
+	for i, dns := range tbl.Records {
+		data, err := json.Marshal(dns.Keys)
+		if err != nil {
+			return fmt.Errorf("encrypt issue marshalling keys: %w", err)
+		}
+		dat, err := encrypt(data, aesKey)
+		if err != nil {
+			return fmt.Errorf("encrypt error: %w", err)
+		}
+		tbl.Records[i].DataEncrypted = dat
+	}
+	return nil
+}
+func (tbl *pgSSLKeyTable) toGeneric() []SSLKey {
+	keys := make([]SSLKey, len(tbl.Records))
+
+	for i, record := range tbl.Records {
+		keys[i] = SSLKey{
+			DeliveryServiceSSLKeys: record.Keys,
+		}
+	}
+	return keys
+}
+func (tbl *pgSSLKeyTable) fromGeneric(keys []SSLKey) {
+	tbl.Records = make([]pgSSLKeyRecord, len(keys))
+
+	for i, key := range keys {
+		tbl.Records[i] = pgSSLKeyRecord{
+			Keys: key.DeliveryServiceSSLKeys,
+			pgCommonRecord: pgCommonRecord{
+				DataEncrypted: nil,
+			},
+			DeliveryService: key.DeliveryService,
+			CDN:             key.CDN,
+		}
+	}
+}
+func (tbl *pgSSLKeyTable) validate() []string {
+	defaultKey := tc.DeliveryServiceSSLKeys{}
+	var errors []string
+	fmtStr := "SSL Key %v: %v"
+	for i, record := range tbl.Records {
+		if record.Keys == defaultKey {
+			errors = append(errors, fmt.Sprintf(fmtStr, i, "DS SSL Keys are default!"))
+		} else if record.Keys.Key == "" {
+			errors = append(errors, fmt.Sprintf(fmtStr, i, "Key is blank!"))
+		} else if record.Keys.CDN == "" {
+			errors = append(errors, fmt.Sprintf(fmtStr, i, "CDN is blank!"))
+		} else if record.Keys.DeliveryService == "" {
+			errors = append(errors, fmt.Sprintf(fmtStr, i, "DS is blank!"))
+		} else if record.DataEncrypted == nil {
+			errors = append(errors, fmt.Sprintf(fmtStr, i, "DataEncrypted is blank!"))
+		} else if record.Keys.Version.String() == "" {
+			errors = append(errors, fmt.Sprintf(fmtStr, i, "Version is blank!"))
+		}
+	}
+	return errors
+}
+
+type pgURLSigKeyRecord struct {
+	Keys            tc.URLSigKeys
+	DeliveryService string
+	pgCommonRecord
+}
+type pgURLSigKeyTable struct {
+	Records []pgURLSigKeyRecord
+}
+
+func (tbl *pgURLSigKeyTable) insertKeys(db *sql.DB) error {
+	queryBase := "INSERT INTO url_sig_key (deliveryservice, data) VALUES "
+	stride := 2
+	queryArgs := make([]interface{}, len(tbl.Records)*stride)
+	for i, record := range tbl.Records {
+		j := i * stride
+		queryArgs[j] = record.DeliveryService
+		queryArgs[j+1] = record.DataEncrypted
+	}
+	return insertIntoTable(db, queryBase, stride, queryArgs)
+}
+func (tbl *pgURLSigKeyTable) gatherKeys(db *sql.DB) error {
+	sz, err := getSize(db, "url_sig_key")
+	if err != nil {
+		log.Println("PGURLSigKey gatherKeys: unable to determine url_sig_key table size")
+	}
+	tbl.Records = make([]pgURLSigKeyRecord, sz)
+
+	rows, err := db.Query("SELECT deliveryservice, data from url_sig_key")
+	if err != nil {
+		return fmt.Errorf("PGURLSigKey gatherKeys error creating query: %w", err)
+	}
+	defer rows.Close()
+	i := 0
+	for rows.Next() {
+		if i > len(tbl.Records)-1 {
+			return fmt.Errorf("PGURLSigKey gatherKeys: got more results than expected %v", len(tbl.Records))
+		}
+		err := rows.Scan(&tbl.Records[i].DeliveryService, &tbl.Records[i].DataEncrypted)
+		if err != nil {
+			return fmt.Errorf("PGURLSigKey gatherKeys: unable to scan row: %w", err)
+		}
+		i += 1
+	}
+	return nil
+}
+func (tbl *pgURLSigKeyTable) decrypt(aesKey []byte) error {
+	for i, sig := range tbl.Records {
+		err := decryptInto(aesKey, sig.DataEncrypted, &tbl.Records[i].Keys)
+		if err != nil {
+			return fmt.Errorf("unable to decrypt into keys: %w", err)
+		}
+	}
+	return nil
+}
+func (tbl *pgURLSigKeyTable) encrypt(aesKey []byte) error {
+	for i, sig := range tbl.Records {
+		data, err := json.Marshal(&sig.Keys)
+		if err != nil {
+			return fmt.Errorf("encrypt issue marshalling keys: %w", err)
+		}
+
+		dat, err := encrypt(data, aesKey)
+		if err != nil {
+			return fmt.Errorf("encrypt error: %w", err)
+		}
+		tbl.Records[i].DataEncrypted = dat
+	}
+	return nil
+}
+func (tbl *pgURLSigKeyTable) toGeneric() []URLSigKey {
+	keys := make([]URLSigKey, len(tbl.Records))
+
+	for i, record := range tbl.Records {
+		keys[i] = URLSigKey{
+			DeliveryService: record.DeliveryService,
+			URLSigKeys:      record.Keys,
+		}
+	}
+	return keys
+}
+func (tbl *pgURLSigKeyTable) fromGeneric(keys []URLSigKey) {
+	tbl.Records = make([]pgURLSigKeyRecord, len(keys))
+
+	for i, key := range keys {
+		tbl.Records[i] = pgURLSigKeyRecord{
+			Keys:            key.URLSigKeys,
+			DeliveryService: key.DeliveryService,
+			pgCommonRecord: pgCommonRecord{
+				DataEncrypted: nil,
+			},
+		}
+	}
+}
+func (tbl *pgURLSigKeyTable) validate() []string {
+	for i, record := range tbl.Records {
+		if record.DataEncrypted == nil && len(record.Keys) > 0 {
+			return []string{fmt.Sprintf("URl Sig Key %v: DataEncrypted is blank!", i)}
+		}
+	}
+	return nil
+}
+
+type pgURISignKeyRecord struct {
+	Keys            map[string]tc.URISignerKeyset
+	DeliveryService string
+	pgCommonRecord
+}
+type pgURISignKeyTable struct {
+	Records []pgURISignKeyRecord
+}
+
+func (tbl *pgURISignKeyTable) insertKeys(db *sql.DB) error {
+	queryBase := "INSERT INTO uri_signing_key (deliveryservice, data) VALUES "

Review comment:
       So this (and the other inserts) will fail if the TVDB already has those keys. I haven't gotten to riak.go yet, but I imagine the behavior when migrating into Riak is that existing data with the same key is just overwritten. Should we run some kind of pre-check query before doing these `INSERT`s, and instead do `UPDATE`s for pre-existing keys?

##########
File path: tools/traffic_vault_migrate/riak.go
##########
@@ -0,0 +1,746 @@
+package main
+
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+import (
+	"crypto/tls"
+	"encoding/json"
+	"errors"
+	"fmt"
+	"time"
+
+	"github.com/apache/trafficcontrol/lib/go-rfc"
+	"github.com/apache/trafficcontrol/lib/go-tc"
+
+	"github.com/basho/riak-go-client"
+)
+
+const (
+	BUCKET_SSL     = "ssl"
+	BUCKET_DNSSEC  = "dnssec"
+	BUCKET_URL_SIG = "url_sig_keys"
+	BUCKET_URI_SIG = "cdn_uri_sig_keys"
+
+	INDEX_SSL = "sslkeys"
+
+	SCHEMA_RIAK_KEY    = "_yz_rk"
+	SCHEMA_RIAK_BUCKET = "_yz_rb"
+)
+
+var (
+	SCHEMA_SSL_FIELDS = [...]string{SCHEMA_RIAK_KEY, SCHEMA_RIAK_BUCKET}
+)
+
+// RiakConfig  represents the configuration options available to the Riak backend
+type RiakConfig struct {
+	Host          string `json:"host"`
+	Port          string `json:"port"`
+	User          string `json:"user"`
+	Password      string `json:"password"`
+	Insecure      bool   `json:"insecure"`
+	TLSVersionRaw string `json:"tlsVersion"`
+	TLSVersion    uint16
+}
+
+// RiakBackend is the Riak implementation of TVBackend
+type RiakBackend struct {
+	sslKeys     riakSSLKeyTable
+	dnssecKeys  riakDNSSecKeyTable
+	uriSignKeys riakURISignKeyTable
+	urlSigKeys  riakURLSigKeyTable
+	cfg         RiakConfig
+	cluster     *riak.Cluster
+}
+
+// String returns a high level overview of the backend and its keys
+func (rb *RiakBackend) String() string {
+	data := fmt.Sprintf("Riak server %v@%v:%v\n", rb.cfg.User, rb.cfg.Host, rb.cfg.Port)
+	data += fmt.Sprintf("\tSSL Keys: %v\n", len(rb.sslKeys.Records))
+	data += fmt.Sprintf("\tDNSSec Keys: %v\n", len(rb.dnssecKeys.Records))
+	data += fmt.Sprintf("\tURI Keys: %v\n", len(rb.uriSignKeys.Records))
+	data += fmt.Sprintf("\tURL Keys: %v\n", len(rb.urlSigKeys.Records))
+	return data
+}
+
+// Name returns the name for this backend
+func (rb *RiakBackend) Name() string {
+	return "Riak"
+}
+
+// ReadConfig takes in a filename and will read it into the backends config
+func (rb *RiakBackend) ReadConfig(s string) error {
+	err := UnmarshalConfig(s, &rb.cfg)
+	if err != nil {
+		return err
+	}
+
+	if rb.cfg.TLSVersionRaw == "10" {
+		rb.cfg.TLSVersion = tls.VersionTLS10
+	} else if rb.cfg.TLSVersionRaw == "11" {
+		rb.cfg.TLSVersion = tls.VersionTLS11
+	} else if rb.cfg.TLSVersionRaw == "12" {
+		rb.cfg.TLSVersion = tls.VersionTLS12
+	} else if rb.cfg.TLSVersionRaw == "13" {
+		rb.cfg.TLSVersion = tls.VersionTLS13
+	}
+	return nil
+}
+
+// Insert takes the current keys and inserts them into the backend DB
+func (rb *RiakBackend) Insert() error {
+	if err := rb.sslKeys.insertKeys(rb.cluster); err != nil {
+		return err
+	}
+	if err := rb.dnssecKeys.insertKeys(rb.cluster); err != nil {
+		return err
+	}
+	if err := rb.urlSigKeys.insertKeys(rb.cluster); err != nil {
+		return err
+	}
+	if err := rb.uriSignKeys.insertKeys(rb.cluster); err != nil {
+		return err
+	}
+	return nil
+}
+
+// ValidateKey validates that the keys are valid (in most cases, certain fields are not null)
+func (rb *RiakBackend) ValidateKey() []string {
+	errors := []string{}
+	if errs := rb.sslKeys.validate(); errs != nil {
+		errors = append(errors, errs...)
+	}
+	if errs := rb.dnssecKeys.validate(); errs != nil {
+		errors = append(errors, errs...)
+	}
+	if errs := rb.uriSignKeys.validate(); errs != nil {
+		errors = append(errors, errs...)
+	}
+	if errs := rb.urlSigKeys.validate(); errs != nil {
+		errors = append(errors, errs...)
+	}
+
+	return errors
+}
+
+// SetSSLKeys takes in keys and converts & encrypts the data into the backends internal format
+func (rb *RiakBackend) SetSSLKeys(keys []SSLKey) error {
+	rb.sslKeys.fromGeneric(keys)
+	return nil
+}
+
+// SetDNSSecKeys takes in keys and converts & encrypts the data into the backends internal format
+func (rb *RiakBackend) SetDNSSecKeys(keys []DNSSecKey) error {
+	rb.dnssecKeys.fromGeneric(keys)
+	return nil
+}
+
+// SetURISignKeys takes in keys and converts & encrypts the data into the backends internal format
+func (rb *RiakBackend) SetURISignKeys(keys []URISignKey) error {
+	rb.uriSignKeys.fromGeneric(keys)
+	return nil
+}
+
+// SetURLSigKeys takes in keys and converts & encrypts the data into the backends internal format
+func (rb *RiakBackend) SetURLSigKeys(keys []URLSigKey) error {
+	rb.urlSigKeys.fromGeneric(keys)
+	return nil
+}
+
+// Start initiates the connection to the backend DB
+func (rb *RiakBackend) Start() error {
+	tlsConfig := &tls.Config{
+		InsecureSkipVerify: !rb.cfg.Insecure,
+		MaxVersion:         rb.cfg.TLSVersion,
+	}
+	auth := &riak.AuthOptions{
+		User:      rb.cfg.User,
+		Password:  rb.cfg.Password,
+		TlsConfig: tlsConfig,
+	}
+
+	cluster, err := getRiakCluster(rb.cfg, auth)
+	if err != nil {
+		return err
+	}
+	if err := cluster.Start(); err != nil {
+		return fmt.Errorf("unable to start riak cluster: %w", err)
+	}
+
+	rb.cluster = cluster
+	rb.sslKeys = riakSSLKeyTable{}
+	rb.dnssecKeys = riakDNSSecKeyTable{}
+	rb.urlSigKeys = riakURLSigKeyTable{}
+	rb.uriSignKeys = riakURISignKeyTable{}
+	return nil
+}
+
+// Close terminates the connection to the backend DB
+func (rb *RiakBackend) Close() error {
+	if err := rb.cluster.Stop(); err != nil {
+		return err
+	}
+	return nil
+}
+
+// Ping checks the connection to the backend DB
+func (rb *RiakBackend) Ping() error {
+	return ping(rb.cluster)
+}
+
+// GetSSLKeys converts the backends internal key representation into the common representation (SSLKey)
+func (rb *RiakBackend) GetSSLKeys() ([]SSLKey, error) {
+	return rb.sslKeys.toGeneric(), nil
+}
+
+// GetDNSSecKeys converts the backends internal key representation into the common representation (DNSSecKey)
+func (rb *RiakBackend) GetDNSSecKeys() ([]DNSSecKey, error) {
+	return rb.dnssecKeys.toGeneric(), nil
+}
+
+// GetURISignKeys converts the pg internal key representation into the common representation (URISignKey)
+func (rb *RiakBackend) GetURISignKeys() ([]URISignKey, error) {
+	return rb.uriSignKeys.toGeneric(), nil
+}
+
+// GetURLSigKeys converts the backends internal key representation into the common representation (URLSigKey)
+func (rb *RiakBackend) GetURLSigKeys() ([]URLSigKey, error) {
+	return rb.urlSigKeys.toGeneric(), nil
+}
+
+// Fetch gets all of the keys from the backend DB
+func (rb *RiakBackend) Fetch() error {
+	if err := rb.sslKeys.gatherKeys(rb.cluster); err != nil {
+		return err
+	}
+	if err := rb.dnssecKeys.gatherKeys(rb.cluster); err != nil {
+		return err
+	}
+	if err := rb.urlSigKeys.gatherKeys(rb.cluster); err != nil {
+		return err
+	}
+	if err := rb.uriSignKeys.gatherKeys(rb.cluster); err != nil {
+		return err
+	}
+
+	return nil
+}
+
+type riakSSLKeyRecord struct {
+	tc.DeliveryServiceSSLKeys
+}
+type riakSSLKeyTable struct {
+	Records []riakSSLKeyRecord
+}
+
+func (tbl *riakSSLKeyTable) gatherKeys(cluster *riak.Cluster) error {
+	searchDocs, err := search(cluster, INDEX_SSL, "cdn:*", fmt.Sprintf("%v:*latest", SCHEMA_RIAK_KEY), 1000, SCHEMA_SSL_FIELDS[:])

Review comment:
       RE: my earlier comment about only gathering the `latest` keys from Postgres, I guess this would also only get the `latest` keys from Riak. However, we might want to fetch and migrate all versions if possible (everything in the `ssl` bucket basically).

##########
File path: tools/traffic_vault_migrate/postgres.go
##########
@@ -0,0 +1,743 @@
+package main
+
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+import (
+	"database/sql"
+	"encoding/base64"
+	"encoding/json"
+	"errors"
+	"fmt"
+	"log"
+	"strconv"
+	"strings"
+
+	"github.com/apache/trafficcontrol/lib/go-tc"
+	util "github.com/apache/trafficcontrol/lib/go-util"
+
+	_ "github.com/lib/pq"
+)
+
+// PGConfig represents the configuration options available to the PG backend
+type PGConfig struct {
+	Host     string `json:"host"`
+	Port     string `json:"port"`
+	User     string `json:"user"`
+	Password string `json:"password"`
+	SSLMode  string `json:"sslmode"`
+	Database string `json:"database"`
+	Key      string `json:"aesKey"`
+	AESKey   []byte
+}
+
+// PGBackend is the Postgres implementation of TVBackend
+type PGBackend struct {
+	sslKey pgSSLKeyTable
+	dnssec pgDNSSecTable
+	uri    pgURISignKeyTable
+	url    pgURLSigKeyTable
+	cfg    PGConfig
+	db     *sql.DB
+}
+
+// String returns a high level overview of the backend and its keys
+func (pg *PGBackend) String() string {
+	data := fmt.Sprintf("PG server %v@%v:%v\n", pg.cfg.User, pg.cfg.Host, pg.cfg.Port)
+	data += fmt.Sprintf("\tSSL Keys: %v\n", len(pg.sslKey.Records))
+	data += fmt.Sprintf("\tDNSSec Keys: %v\n", len(pg.dnssec.Records))
+	data += fmt.Sprintf("\tURI Keys: %v\n", len(pg.uri.Records))
+	data += fmt.Sprintf("\tURL Keys: %v\n", len(pg.url.Records))
+	return data
+}
+
+// Name returns the name for this backend
+func (pg *PGBackend) Name() string {
+	return "PG"
+}
+
+// ReadConfig takes in a filename and will read it into the backends config
+func (pg *PGBackend) ReadConfig(s string) error {
+	err := UnmarshalConfig(s, &pg.cfg)
+	if err != nil {
+		return err
+	}
+
+	pg.cfg.AESKey, err = base64.StdEncoding.DecodeString(pg.cfg.Key)
+	if err != nil {
+		return fmt.Errorf("unable to decode PG AESKey: %w", err)
+	}
+	return nil
+}
+
+// Insert takes the current keys and inserts them into the backend DB
+func (pg *PGBackend) Insert() error {
+	if err := pg.sslKey.insertKeys(pg.db); err != nil {
+		return err
+	}
+	if err := pg.dnssec.insertKeys(pg.db); err != nil {
+		return err
+	}
+	if err := pg.url.insertKeys(pg.db); err != nil {
+		return err
+	}
+	if err := pg.uri.insertKeys(pg.db); err != nil {
+		return err
+	}
+	return nil
+}
+
+// Start initiates the connection to the backend DB
+func (pg *PGBackend) Start() error {
+	sqlStr := fmt.Sprintf("postgres://%s:%s@%s:%s/%s?sslmode=%s", pg.cfg.User, pg.cfg.Password, pg.cfg.Host, pg.cfg.Port, pg.cfg.Database, pg.cfg.SSLMode)
+	db, err := sql.Open("postgres", sqlStr)
+	if err != nil {
+		return fmt.Errorf("unable to start PG client: %w", err)
+	}
+
+	pg.db = db
+	pg.sslKey = pgSSLKeyTable{}
+	pg.dnssec = pgDNSSecTable{}
+	pg.url = pgURLSigKeyTable{}
+	pg.uri = pgURISignKeyTable{}
+
+	return nil
+}
+
+// ValidateKey validates that the keys are valid (in most cases, certain fields are not null)
+func (pg *PGBackend) ValidateKey() []string {
+	var errors []string
+	if errs := pg.sslKey.validate(); errs != nil {
+		errors = append(errors, errs...)
+	}
+	if errs := pg.dnssec.validate(); errs != nil {
+		errors = append(errors, errs...)
+	}
+	if errs := pg.uri.validate(); errs != nil {
+		errors = append(errors, errs...)
+	}
+	if errs := pg.url.validate(); errs != nil {
+		errors = append(errors, errs...)
+	}
+	return errors
+}
+
+// Close terminates the connection to the backend DB
+func (pg *PGBackend) Close() error {
+	return pg.db.Close()
+}
+
+// Ping checks the connection to the backend DB
+func (pg *PGBackend) Ping() error {
+	return pg.db.Ping()
+}
+
+// Fetch gets all of the keys from the backend DB
+func (pg *PGBackend) Fetch() error {
+	if err := pg.sslKey.gatherKeys(pg.db); err != nil {
+		return err
+	}
+
+	if err := pg.dnssec.gatherKeys(pg.db); err != nil {
+		return err
+	}
+
+	if err := pg.url.gatherKeys(pg.db); err != nil {
+		return err
+	}
+
+	if err := pg.uri.gatherKeys(pg.db); err != nil {
+		return err
+	}
+
+	return nil
+}
+
+// GetSSLKeys converts the backends internal key representation into the common representation (SSLKey)
+func (pg *PGBackend) GetSSLKeys() ([]SSLKey, error) {
+	if err := pg.sslKey.decrypt(pg.cfg.AESKey); err != nil {
+		return nil, err
+	}
+	return pg.sslKey.toGeneric(), nil
+}
+
+// SetSSLKeys takes in keys and converts & encrypts the data into the backends internal format
+func (pg *PGBackend) SetSSLKeys(keys []SSLKey) error {
+	pg.sslKey.fromGeneric(keys)
+	return pg.sslKey.encrypt(pg.cfg.AESKey)
+}
+
+// GetDNSSecKeys converts the backends internal key representation into the common representation (DNSSecKey)
+func (pg *PGBackend) GetDNSSecKeys() ([]DNSSecKey, error) {
+	if err := pg.dnssec.decrypt(pg.cfg.AESKey); err != nil {
+		return nil, err
+	}
+	return pg.dnssec.toGeneric(), nil
+}
+
+// SetDNSSecKeys takes in keys and converts & encrypts the data into the backends internal format
+func (pg *PGBackend) SetDNSSecKeys(keys []DNSSecKey) error {
+	pg.dnssec.fromGeneric(keys)
+	return pg.dnssec.encrypt(pg.cfg.AESKey)
+}
+
+// GetURISignKeys converts the pg internal key representation into the common representation (URISignKey)
+func (pg *PGBackend) GetURISignKeys() ([]URISignKey, error) {
+	if err := pg.uri.decrypt(pg.cfg.AESKey); err != nil {
+		return nil, err
+	}
+	return pg.uri.toGeneric(), nil
+}
+
+// SetURISignKeys takes in keys and converts & encrypts the data into the backends internal format
+func (pg *PGBackend) SetURISignKeys(keys []URISignKey) error {
+	pg.uri.fromGeneric(keys)
+	return pg.uri.encrypt(pg.cfg.AESKey)
+}
+
+// GetURLSigKeys converts the backends internal key representation into the common representation (URLSigKey)
+func (pg *PGBackend) GetURLSigKeys() ([]URLSigKey, error) {
+	if err := pg.url.decrypt(pg.cfg.AESKey); err != nil {
+		return nil, err
+	}
+	return pg.url.toGeneric(), nil
+}
+
+// SetURLSigKeys takes in keys and converts & encrypts the data into the backends internal format
+func (pg *PGBackend) SetURLSigKeys(keys []URLSigKey) error {
+	pg.url.fromGeneric(keys)
+	return pg.url.encrypt(pg.cfg.AESKey)
+}
+
+type pgCommonRecord struct {
+	DataEncrypted []byte
+}
+
+type pgDNSSecRecord struct {
+	Key tc.DNSSECKeysTrafficVault
+	CDN string
+	pgCommonRecord
+}
+type pgDNSSecTable struct {
+	Records []pgDNSSecRecord
+}
+
+func (tbl *pgDNSSecTable) gatherKeys(db *sql.DB) error {
+	sz, err := getSize(db, "dnssec")
+	if err != nil {
+		log.Println("PGDNSSec gatherKeys: unable to determine size of dnssec table")
+	}
+	tbl.Records = make([]pgDNSSecRecord, sz)
+
+	rows, err := db.Query("SELECT cdn, data from dnssec")
+	if err != nil {
+		return fmt.Errorf("PGDNSSec gatherKeys: unable to query: %w", err)
+	}
+	defer rows.Close()
+	i := 0
+	for rows.Next() {
+		if i > len(tbl.Records)-1 {
+			return fmt.Errorf("PGDNSSec gatherKeys got more results than expected %v", len(tbl.Records))
+		}
+		err := rows.Scan(&tbl.Records[i].CDN, &tbl.Records[i].DataEncrypted)
+		if err != nil {
+			return fmt.Errorf("PGDNSSec gatherKeys unable to scan row: %w", err)
+		}
+		i += 1
+	}
+	return nil
+}
+func (tbl *pgDNSSecTable) decrypt(aesKey []byte) error {
+	for i, _ := range tbl.Records {
+		err := decryptInto(aesKey, tbl.Records[i].DataEncrypted, &tbl.Records[i].Key)
+		if err != nil {
+			return fmt.Errorf("unable to decrypt into keys: %w", err)
+		}
+	}
+	return nil
+}
+func (tbl *pgDNSSecTable) encrypt(aesKey []byte) error {
+	for i, dns := range tbl.Records {
+		data, err := json.Marshal(&dns.Key)
+		if err != nil {
+			return fmt.Errorf("encrypt issue marshalling keys: %w", err)
+		}
+		dat, err := encrypt(data, aesKey)
+		if err != nil {
+			return fmt.Errorf("encrypt error: %w", err)
+		}
+		tbl.Records[i].DataEncrypted = dat
+	}
+	return nil
+}
+func (tbl *pgDNSSecTable) toGeneric() []DNSSecKey {
+	keys := make([]DNSSecKey, len(tbl.Records))
+
+	for i, record := range tbl.Records {
+		keys[i] = DNSSecKey{
+			CDN:                    record.CDN,
+			DNSSECKeysTrafficVault: record.Key,
+		}
+	}
+
+	return keys
+}
+func (tbl *pgDNSSecTable) fromGeneric(keys []DNSSecKey) {
+	tbl.Records = make([]pgDNSSecRecord, len(keys))
+
+	for i, key := range keys {
+		tbl.Records[i] = pgDNSSecRecord{
+			Key: key.DNSSECKeysTrafficVault,
+			CDN: key.CDN,
+			pgCommonRecord: pgCommonRecord{
+				DataEncrypted: nil,
+			},
+		}
+	}
+}
+func (tbl *pgDNSSecTable) validate() []string {
+	for i, record := range tbl.Records {
+		if record.DataEncrypted == nil && len(record.Key) > 0 {
+			return []string{fmt.Sprintf("DNSSEC Key %v: DataEncrypted is blank!", i)}
+		}
+	}
+	return nil
+}
+func (tbl *pgDNSSecTable) insertKeys(db *sql.DB) error {
+	queryBase := "INSERT INTO dnssec (cdn, data) VALUES "
+	stride := 2
+	queryArgs := make([]interface{}, len(tbl.Records)*stride)
+	for i, record := range tbl.Records {
+		j := i * stride
+		queryArgs[j] = record.CDN
+		queryArgs[j+1] = record.DataEncrypted
+	}
+	return insertIntoTable(db, queryBase, stride, queryArgs)
+}
+
+type pgSSLKeyRecord struct {
+	Keys tc.DeliveryServiceSSLKeys
+	pgCommonRecord
+
+	// These records are stored on the table but are duplicated
+	DeliveryService string
+	CDN             string
+}
+type pgSSLKeyTable struct {
+	Records []pgSSLKeyRecord
+}
+
+func (tbl *pgSSLKeyTable) insertKeys(db *sql.DB) error {
+	queryBase := "INSERT INTO sslkey (deliveryservice, data, cdn, version) VALUES "
+	duplicateKeys := 2
+	stride := 4
+	queryArgs := make([]interface{}, len(tbl.Records)*stride*duplicateKeys)
+	for i, record := range tbl.Records {
+		j := i * duplicateKeys * stride
+
+		queryArgs[j] = record.DeliveryService
+		queryArgs[j+1] = record.DataEncrypted
+		queryArgs[j+2] = record.CDN
+		queryArgs[j+3] = record.Keys.Version.String()
+
+		queryArgs[j+4] = record.DeliveryService
+		queryArgs[j+5] = record.DataEncrypted
+		queryArgs[j+6] = record.CDN
+		queryArgs[j+7] = "latest"
+	}
+	return insertIntoTable(db, queryBase, 4, queryArgs)
+}
+func (tbl *pgSSLKeyTable) gatherKeys(db *sql.DB) error {
+	sz, err := getSize(db, "sslkey WHERE version='latest'")
+	if err != nil {
+		return fmt.Errorf("PGSSLKey gatherKeys unable to determine size of sslkey table: %w", err)
+	}
+	tbl.Records = make([]pgSSLKeyRecord, sz)
+
+	rows, err := db.Query("SELECT data, deliveryservice, cdn from sslkey WHERE version = 'latest'")
+	if err != nil {
+		return fmt.Errorf("PGSSLKey gatherKeys unable to query: %w", err)
+	}
+	defer rows.Close()
+	i := 0
+	for rows.Next() {
+		if i > len(tbl.Records)-1 {
+			return errors.New("PGSSLKey gatherKeys: got more results than expected")
+		}
+		err := rows.Scan(&tbl.Records[i].DataEncrypted, &tbl.Records[i].DeliveryService, &tbl.Records[i].CDN)
+		if err != nil {
+			return fmt.Errorf("PGSSLKey gatherKeys unable to scan row: %w", err)
+		}
+		i += 1
+	}
+	return nil
+}
+func (tbl *pgSSLKeyTable) decrypt(aesKey []byte) error {
+	for i, dns := range tbl.Records {
+		err := decryptInto(aesKey, dns.DataEncrypted, &tbl.Records[i].Keys)
+		if err != nil {
+			return fmt.Errorf("unable to decrypt into keys: %w", err)
+		}
+	}
+	return nil
+}
+func (tbl *pgSSLKeyTable) encrypt(aesKey []byte) error {
+	for i, dns := range tbl.Records {
+		data, err := json.Marshal(dns.Keys)
+		if err != nil {
+			return fmt.Errorf("encrypt issue marshalling keys: %w", err)
+		}
+		dat, err := encrypt(data, aesKey)
+		if err != nil {
+			return fmt.Errorf("encrypt error: %w", err)
+		}
+		tbl.Records[i].DataEncrypted = dat
+	}
+	return nil
+}
+func (tbl *pgSSLKeyTable) toGeneric() []SSLKey {
+	keys := make([]SSLKey, len(tbl.Records))
+
+	for i, record := range tbl.Records {
+		keys[i] = SSLKey{
+			DeliveryServiceSSLKeys: record.Keys,
+		}
+	}
+	return keys
+}
+func (tbl *pgSSLKeyTable) fromGeneric(keys []SSLKey) {
+	tbl.Records = make([]pgSSLKeyRecord, len(keys))
+
+	for i, key := range keys {
+		tbl.Records[i] = pgSSLKeyRecord{
+			Keys: key.DeliveryServiceSSLKeys,
+			pgCommonRecord: pgCommonRecord{
+				DataEncrypted: nil,
+			},
+			DeliveryService: key.DeliveryService,
+			CDN:             key.CDN,
+		}
+	}
+}
+func (tbl *pgSSLKeyTable) validate() []string {
+	defaultKey := tc.DeliveryServiceSSLKeys{}
+	var errors []string
+	fmtStr := "SSL Key %v: %v"
+	for i, record := range tbl.Records {
+		if record.Keys == defaultKey {
+			errors = append(errors, fmt.Sprintf(fmtStr, i, "DS SSL Keys are default!"))
+		} else if record.Keys.Key == "" {
+			errors = append(errors, fmt.Sprintf(fmtStr, i, "Key is blank!"))
+		} else if record.Keys.CDN == "" {
+			errors = append(errors, fmt.Sprintf(fmtStr, i, "CDN is blank!"))
+		} else if record.Keys.DeliveryService == "" {
+			errors = append(errors, fmt.Sprintf(fmtStr, i, "DS is blank!"))
+		} else if record.DataEncrypted == nil {
+			errors = append(errors, fmt.Sprintf(fmtStr, i, "DataEncrypted is blank!"))
+		} else if record.Keys.Version.String() == "" {
+			errors = append(errors, fmt.Sprintf(fmtStr, i, "Version is blank!"))
+		}
+	}
+	return errors
+}
+
+type pgURLSigKeyRecord struct {
+	Keys            tc.URLSigKeys
+	DeliveryService string
+	pgCommonRecord
+}
+type pgURLSigKeyTable struct {
+	Records []pgURLSigKeyRecord
+}
+
+func (tbl *pgURLSigKeyTable) insertKeys(db *sql.DB) error {
+	queryBase := "INSERT INTO url_sig_key (deliveryservice, data) VALUES "
+	stride := 2
+	queryArgs := make([]interface{}, len(tbl.Records)*stride)
+	for i, record := range tbl.Records {
+		j := i * stride
+		queryArgs[j] = record.DeliveryService
+		queryArgs[j+1] = record.DataEncrypted
+	}
+	return insertIntoTable(db, queryBase, stride, queryArgs)
+}
+func (tbl *pgURLSigKeyTable) gatherKeys(db *sql.DB) error {
+	sz, err := getSize(db, "url_sig_key")
+	if err != nil {
+		log.Println("PGURLSigKey gatherKeys: unable to determine url_sig_key table size")
+	}
+	tbl.Records = make([]pgURLSigKeyRecord, sz)
+
+	rows, err := db.Query("SELECT deliveryservice, data from url_sig_key")
+	if err != nil {
+		return fmt.Errorf("PGURLSigKey gatherKeys error creating query: %w", err)
+	}
+	defer rows.Close()
+	i := 0
+	for rows.Next() {
+		if i > len(tbl.Records)-1 {
+			return fmt.Errorf("PGURLSigKey gatherKeys: got more results than expected %v", len(tbl.Records))
+		}
+		err := rows.Scan(&tbl.Records[i].DeliveryService, &tbl.Records[i].DataEncrypted)
+		if err != nil {
+			return fmt.Errorf("PGURLSigKey gatherKeys: unable to scan row: %w", err)
+		}
+		i += 1
+	}
+	return nil
+}
+func (tbl *pgURLSigKeyTable) decrypt(aesKey []byte) error {
+	for i, sig := range tbl.Records {
+		err := decryptInto(aesKey, sig.DataEncrypted, &tbl.Records[i].Keys)
+		if err != nil {
+			return fmt.Errorf("unable to decrypt into keys: %w", err)
+		}
+	}
+	return nil
+}
+func (tbl *pgURLSigKeyTable) encrypt(aesKey []byte) error {
+	for i, sig := range tbl.Records {
+		data, err := json.Marshal(&sig.Keys)
+		if err != nil {
+			return fmt.Errorf("encrypt issue marshalling keys: %w", err)
+		}
+
+		dat, err := encrypt(data, aesKey)
+		if err != nil {
+			return fmt.Errorf("encrypt error: %w", err)
+		}
+		tbl.Records[i].DataEncrypted = dat
+	}
+	return nil
+}
+func (tbl *pgURLSigKeyTable) toGeneric() []URLSigKey {
+	keys := make([]URLSigKey, len(tbl.Records))
+
+	for i, record := range tbl.Records {
+		keys[i] = URLSigKey{
+			DeliveryService: record.DeliveryService,
+			URLSigKeys:      record.Keys,
+		}
+	}
+	return keys
+}
+func (tbl *pgURLSigKeyTable) fromGeneric(keys []URLSigKey) {
+	tbl.Records = make([]pgURLSigKeyRecord, len(keys))
+
+	for i, key := range keys {
+		tbl.Records[i] = pgURLSigKeyRecord{
+			Keys:            key.URLSigKeys,
+			DeliveryService: key.DeliveryService,
+			pgCommonRecord: pgCommonRecord{
+				DataEncrypted: nil,
+			},
+		}
+	}
+}
+func (tbl *pgURLSigKeyTable) validate() []string {
+	for i, record := range tbl.Records {
+		if record.DataEncrypted == nil && len(record.Keys) > 0 {
+			return []string{fmt.Sprintf("URl Sig Key %v: DataEncrypted is blank!", i)}
+		}
+	}
+	return nil
+}
+
+type pgURISignKeyRecord struct {
+	Keys            map[string]tc.URISignerKeyset
+	DeliveryService string
+	pgCommonRecord
+}
+type pgURISignKeyTable struct {
+	Records []pgURISignKeyRecord
+}
+
+func (tbl *pgURISignKeyTable) insertKeys(db *sql.DB) error {
+	queryBase := "INSERT INTO uri_signing_key (deliveryservice, data) VALUES "
+	stride := 2
+	queryArgs := make([]interface{}, len(tbl.Records)*stride)
+	for i, record := range tbl.Records {
+		j := i * stride
+		queryArgs[j] = record.DeliveryService
+		queryArgs[j+1] = record.DataEncrypted
+	}
+	return insertIntoTable(db, queryBase, stride, queryArgs)
+}
+func (tbl *pgURISignKeyTable) gatherKeys(db *sql.DB) error {
+	sz, err := getSize(db, "uri_signing_key")
+	if err != nil {
+		log.Println("PGURISignKey gatherKeys: unable to determine size of uri_signing_key table")
+	}
+	tbl.Records = make([]pgURISignKeyRecord, sz)
+
+	rows, err := db.Query("SELECT deliveryservice, data from uri_signing_key")
+	if err != nil {
+		return fmt.Errorf("PGURISignKey gatherKeys error while query: %w", err)
+	}
+	defer rows.Close()
+	i := 0
+	for rows.Next() {
+		if i > len(tbl.Records)-1 {
+			return fmt.Errorf("PGURISignKey gatherKeys: got more results than expected %v", len(tbl.Records))
+		}
+		err := rows.Scan(&tbl.Records[i].DeliveryService, &tbl.Records[i].DataEncrypted)

Review comment:
       nit: this can be combined with the line below like `if err := function(); err != nil {`

##########
File path: tools/traffic_vault_migrate/riak.go
##########
@@ -0,0 +1,746 @@
+package main
+
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+import (
+	"crypto/tls"
+	"encoding/json"
+	"errors"
+	"fmt"
+	"time"
+
+	"github.com/apache/trafficcontrol/lib/go-rfc"
+	"github.com/apache/trafficcontrol/lib/go-tc"
+
+	"github.com/basho/riak-go-client"
+)
+
+const (
+	BUCKET_SSL     = "ssl"
+	BUCKET_DNSSEC  = "dnssec"
+	BUCKET_URL_SIG = "url_sig_keys"
+	BUCKET_URI_SIG = "cdn_uri_sig_keys"
+
+	INDEX_SSL = "sslkeys"
+
+	SCHEMA_RIAK_KEY    = "_yz_rk"
+	SCHEMA_RIAK_BUCKET = "_yz_rb"
+)
+
+var (
+	SCHEMA_SSL_FIELDS = [...]string{SCHEMA_RIAK_KEY, SCHEMA_RIAK_BUCKET}
+)
+
+// RiakConfig  represents the configuration options available to the Riak backend
+type RiakConfig struct {
+	Host          string `json:"host"`
+	Port          string `json:"port"`
+	User          string `json:"user"`
+	Password      string `json:"password"`
+	Insecure      bool   `json:"insecure"`
+	TLSVersionRaw string `json:"tlsVersion"`
+	TLSVersion    uint16
+}
+
+// RiakBackend is the Riak implementation of TVBackend
+type RiakBackend struct {
+	sslKeys     riakSSLKeyTable
+	dnssecKeys  riakDNSSecKeyTable
+	uriSignKeys riakURISignKeyTable
+	urlSigKeys  riakURLSigKeyTable
+	cfg         RiakConfig
+	cluster     *riak.Cluster
+}
+
+// String returns a high level overview of the backend and its keys
+func (rb *RiakBackend) String() string {
+	data := fmt.Sprintf("Riak server %v@%v:%v\n", rb.cfg.User, rb.cfg.Host, rb.cfg.Port)
+	data += fmt.Sprintf("\tSSL Keys: %v\n", len(rb.sslKeys.Records))
+	data += fmt.Sprintf("\tDNSSec Keys: %v\n", len(rb.dnssecKeys.Records))
+	data += fmt.Sprintf("\tURI Keys: %v\n", len(rb.uriSignKeys.Records))
+	data += fmt.Sprintf("\tURL Keys: %v\n", len(rb.urlSigKeys.Records))
+	return data
+}
+
+// Name returns the name for this backend
+func (rb *RiakBackend) Name() string {
+	return "Riak"
+}
+
+// ReadConfig takes in a filename and will read it into the backends config

Review comment:
       Similar to the other GoDoc comment in postgres.go, this looks like it actually takes in the file contents (as a string) rather than a filename.

##########
File path: tools/traffic_vault_migrate/riak.go
##########
@@ -0,0 +1,746 @@
+package main
+
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+import (
+	"crypto/tls"
+	"encoding/json"
+	"errors"
+	"fmt"
+	"time"
+
+	"github.com/apache/trafficcontrol/lib/go-rfc"
+	"github.com/apache/trafficcontrol/lib/go-tc"
+
+	"github.com/basho/riak-go-client"
+)
+
+const (
+	BUCKET_SSL     = "ssl"
+	BUCKET_DNSSEC  = "dnssec"
+	BUCKET_URL_SIG = "url_sig_keys"
+	BUCKET_URI_SIG = "cdn_uri_sig_keys"
+
+	INDEX_SSL = "sslkeys"
+
+	SCHEMA_RIAK_KEY    = "_yz_rk"
+	SCHEMA_RIAK_BUCKET = "_yz_rb"
+)
+
+var (
+	SCHEMA_SSL_FIELDS = [...]string{SCHEMA_RIAK_KEY, SCHEMA_RIAK_BUCKET}
+)
+
+// RiakConfig  represents the configuration options available to the Riak backend
+type RiakConfig struct {
+	Host          string `json:"host"`
+	Port          string `json:"port"`
+	User          string `json:"user"`
+	Password      string `json:"password"`
+	Insecure      bool   `json:"insecure"`
+	TLSVersionRaw string `json:"tlsVersion"`
+	TLSVersion    uint16
+}
+
+// RiakBackend is the Riak implementation of TVBackend
+type RiakBackend struct {
+	sslKeys     riakSSLKeyTable
+	dnssecKeys  riakDNSSecKeyTable
+	uriSignKeys riakURISignKeyTable
+	urlSigKeys  riakURLSigKeyTable
+	cfg         RiakConfig
+	cluster     *riak.Cluster
+}
+
+// String returns a high level overview of the backend and its keys
+func (rb *RiakBackend) String() string {
+	data := fmt.Sprintf("Riak server %v@%v:%v\n", rb.cfg.User, rb.cfg.Host, rb.cfg.Port)
+	data += fmt.Sprintf("\tSSL Keys: %v\n", len(rb.sslKeys.Records))
+	data += fmt.Sprintf("\tDNSSec Keys: %v\n", len(rb.dnssecKeys.Records))
+	data += fmt.Sprintf("\tURI Keys: %v\n", len(rb.uriSignKeys.Records))
+	data += fmt.Sprintf("\tURL Keys: %v\n", len(rb.urlSigKeys.Records))
+	return data
+}
+
+// Name returns the name for this backend
+func (rb *RiakBackend) Name() string {
+	return "Riak"
+}
+
+// ReadConfig takes in a filename and will read it into the backends config
+func (rb *RiakBackend) ReadConfig(s string) error {
+	err := UnmarshalConfig(s, &rb.cfg)
+	if err != nil {
+		return err
+	}
+
+	if rb.cfg.TLSVersionRaw == "10" {
+		rb.cfg.TLSVersion = tls.VersionTLS10
+	} else if rb.cfg.TLSVersionRaw == "11" {
+		rb.cfg.TLSVersion = tls.VersionTLS11
+	} else if rb.cfg.TLSVersionRaw == "12" {
+		rb.cfg.TLSVersion = tls.VersionTLS12
+	} else if rb.cfg.TLSVersionRaw == "13" {
+		rb.cfg.TLSVersion = tls.VersionTLS13
+	}
+	return nil
+}
+
+// Insert takes the current keys and inserts them into the backend DB
+func (rb *RiakBackend) Insert() error {
+	if err := rb.sslKeys.insertKeys(rb.cluster); err != nil {
+		return err
+	}
+	if err := rb.dnssecKeys.insertKeys(rb.cluster); err != nil {
+		return err
+	}
+	if err := rb.urlSigKeys.insertKeys(rb.cluster); err != nil {
+		return err
+	}
+	if err := rb.uriSignKeys.insertKeys(rb.cluster); err != nil {
+		return err
+	}
+	return nil
+}
+
+// ValidateKey validates that the keys are valid (in most cases, certain fields are not null)
+func (rb *RiakBackend) ValidateKey() []string {
+	errors := []string{}

Review comment:
       nit: `errors` shadows the `errors` package

##########
File path: tools/traffic_vault_migrate/riak.go
##########
@@ -0,0 +1,746 @@
+package main
+
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+import (
+	"crypto/tls"
+	"encoding/json"
+	"errors"
+	"fmt"
+	"time"
+
+	"github.com/apache/trafficcontrol/lib/go-rfc"
+	"github.com/apache/trafficcontrol/lib/go-tc"
+
+	"github.com/basho/riak-go-client"
+)
+
+const (
+	BUCKET_SSL     = "ssl"
+	BUCKET_DNSSEC  = "dnssec"
+	BUCKET_URL_SIG = "url_sig_keys"
+	BUCKET_URI_SIG = "cdn_uri_sig_keys"
+
+	INDEX_SSL = "sslkeys"
+
+	SCHEMA_RIAK_KEY    = "_yz_rk"
+	SCHEMA_RIAK_BUCKET = "_yz_rb"
+)
+
+var (
+	SCHEMA_SSL_FIELDS = [...]string{SCHEMA_RIAK_KEY, SCHEMA_RIAK_BUCKET}
+)
+
+// RiakConfig  represents the configuration options available to the Riak backend
+type RiakConfig struct {
+	Host          string `json:"host"`
+	Port          string `json:"port"`
+	User          string `json:"user"`
+	Password      string `json:"password"`
+	Insecure      bool   `json:"insecure"`
+	TLSVersionRaw string `json:"tlsVersion"`
+	TLSVersion    uint16
+}
+
+// RiakBackend is the Riak implementation of TVBackend
+type RiakBackend struct {
+	sslKeys     riakSSLKeyTable
+	dnssecKeys  riakDNSSecKeyTable
+	uriSignKeys riakURISignKeyTable
+	urlSigKeys  riakURLSigKeyTable
+	cfg         RiakConfig
+	cluster     *riak.Cluster
+}
+
+// String returns a high level overview of the backend and its keys
+func (rb *RiakBackend) String() string {
+	data := fmt.Sprintf("Riak server %v@%v:%v\n", rb.cfg.User, rb.cfg.Host, rb.cfg.Port)

Review comment:
       nit: type-specific formatting instead of %v

##########
File path: tools/traffic_vault_migrate/postgres.go
##########
@@ -0,0 +1,743 @@
+package main
+
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+import (
+	"database/sql"
+	"encoding/base64"
+	"encoding/json"
+	"errors"
+	"fmt"
+	"log"
+	"strconv"
+	"strings"
+
+	"github.com/apache/trafficcontrol/lib/go-tc"
+	util "github.com/apache/trafficcontrol/lib/go-util"
+
+	_ "github.com/lib/pq"
+)
+
+// PGConfig represents the configuration options available to the PG backend
+type PGConfig struct {
+	Host     string `json:"host"`
+	Port     string `json:"port"`
+	User     string `json:"user"`
+	Password string `json:"password"`
+	SSLMode  string `json:"sslmode"`
+	Database string `json:"database"`
+	Key      string `json:"aesKey"`
+	AESKey   []byte
+}
+
+// PGBackend is the Postgres implementation of TVBackend
+type PGBackend struct {
+	sslKey pgSSLKeyTable
+	dnssec pgDNSSecTable
+	uri    pgURISignKeyTable
+	url    pgURLSigKeyTable
+	cfg    PGConfig
+	db     *sql.DB
+}
+
+// String returns a high level overview of the backend and its keys
+func (pg *PGBackend) String() string {
+	data := fmt.Sprintf("PG server %v@%v:%v\n", pg.cfg.User, pg.cfg.Host, pg.cfg.Port)
+	data += fmt.Sprintf("\tSSL Keys: %v\n", len(pg.sslKey.Records))
+	data += fmt.Sprintf("\tDNSSec Keys: %v\n", len(pg.dnssec.Records))
+	data += fmt.Sprintf("\tURI Keys: %v\n", len(pg.uri.Records))
+	data += fmt.Sprintf("\tURL Keys: %v\n", len(pg.url.Records))
+	return data
+}
+
+// Name returns the name for this backend
+func (pg *PGBackend) Name() string {
+	return "PG"
+}
+
+// ReadConfig takes in a filename and will read it into the backends config
+func (pg *PGBackend) ReadConfig(s string) error {
+	err := UnmarshalConfig(s, &pg.cfg)
+	if err != nil {
+		return err
+	}
+
+	pg.cfg.AESKey, err = base64.StdEncoding.DecodeString(pg.cfg.Key)
+	if err != nil {
+		return fmt.Errorf("unable to decode PG AESKey: %w", err)
+	}
+	return nil
+}
+
+// Insert takes the current keys and inserts them into the backend DB
+func (pg *PGBackend) Insert() error {
+	if err := pg.sslKey.insertKeys(pg.db); err != nil {
+		return err
+	}
+	if err := pg.dnssec.insertKeys(pg.db); err != nil {
+		return err
+	}
+	if err := pg.url.insertKeys(pg.db); err != nil {
+		return err
+	}
+	if err := pg.uri.insertKeys(pg.db); err != nil {
+		return err
+	}
+	return nil
+}
+
+// Start initiates the connection to the backend DB
+func (pg *PGBackend) Start() error {
+	sqlStr := fmt.Sprintf("postgres://%s:%s@%s:%s/%s?sslmode=%s", pg.cfg.User, pg.cfg.Password, pg.cfg.Host, pg.cfg.Port, pg.cfg.Database, pg.cfg.SSLMode)
+	db, err := sql.Open("postgres", sqlStr)
+	if err != nil {
+		return fmt.Errorf("unable to start PG client: %w", err)
+	}
+
+	pg.db = db
+	pg.sslKey = pgSSLKeyTable{}
+	pg.dnssec = pgDNSSecTable{}
+	pg.url = pgURLSigKeyTable{}
+	pg.uri = pgURISignKeyTable{}
+
+	return nil
+}
+
+// ValidateKey validates that the keys are valid (in most cases, certain fields are not null)
+func (pg *PGBackend) ValidateKey() []string {
+	var errors []string
+	if errs := pg.sslKey.validate(); errs != nil {
+		errors = append(errors, errs...)
+	}
+	if errs := pg.dnssec.validate(); errs != nil {
+		errors = append(errors, errs...)
+	}
+	if errs := pg.uri.validate(); errs != nil {
+		errors = append(errors, errs...)
+	}
+	if errs := pg.url.validate(); errs != nil {
+		errors = append(errors, errs...)
+	}
+	return errors
+}
+
+// Close terminates the connection to the backend DB
+func (pg *PGBackend) Close() error {
+	return pg.db.Close()
+}
+
+// Ping checks the connection to the backend DB
+func (pg *PGBackend) Ping() error {
+	return pg.db.Ping()
+}
+
+// Fetch gets all of the keys from the backend DB
+func (pg *PGBackend) Fetch() error {
+	if err := pg.sslKey.gatherKeys(pg.db); err != nil {
+		return err
+	}
+
+	if err := pg.dnssec.gatherKeys(pg.db); err != nil {
+		return err
+	}
+
+	if err := pg.url.gatherKeys(pg.db); err != nil {
+		return err
+	}
+
+	if err := pg.uri.gatherKeys(pg.db); err != nil {
+		return err
+	}
+
+	return nil
+}
+
+// GetSSLKeys converts the backends internal key representation into the common representation (SSLKey)
+func (pg *PGBackend) GetSSLKeys() ([]SSLKey, error) {
+	if err := pg.sslKey.decrypt(pg.cfg.AESKey); err != nil {
+		return nil, err
+	}
+	return pg.sslKey.toGeneric(), nil
+}
+
+// SetSSLKeys takes in keys and converts & encrypts the data into the backends internal format
+func (pg *PGBackend) SetSSLKeys(keys []SSLKey) error {
+	pg.sslKey.fromGeneric(keys)
+	return pg.sslKey.encrypt(pg.cfg.AESKey)
+}
+
+// GetDNSSecKeys converts the backends internal key representation into the common representation (DNSSecKey)
+func (pg *PGBackend) GetDNSSecKeys() ([]DNSSecKey, error) {
+	if err := pg.dnssec.decrypt(pg.cfg.AESKey); err != nil {
+		return nil, err
+	}
+	return pg.dnssec.toGeneric(), nil
+}
+
+// SetDNSSecKeys takes in keys and converts & encrypts the data into the backends internal format
+func (pg *PGBackend) SetDNSSecKeys(keys []DNSSecKey) error {
+	pg.dnssec.fromGeneric(keys)
+	return pg.dnssec.encrypt(pg.cfg.AESKey)
+}
+
+// GetURISignKeys converts the pg internal key representation into the common representation (URISignKey)
+func (pg *PGBackend) GetURISignKeys() ([]URISignKey, error) {
+	if err := pg.uri.decrypt(pg.cfg.AESKey); err != nil {
+		return nil, err
+	}
+	return pg.uri.toGeneric(), nil
+}
+
+// SetURISignKeys takes in keys and converts & encrypts the data into the backends internal format
+func (pg *PGBackend) SetURISignKeys(keys []URISignKey) error {
+	pg.uri.fromGeneric(keys)
+	return pg.uri.encrypt(pg.cfg.AESKey)
+}
+
+// GetURLSigKeys converts the backends internal key representation into the common representation (URLSigKey)
+func (pg *PGBackend) GetURLSigKeys() ([]URLSigKey, error) {
+	if err := pg.url.decrypt(pg.cfg.AESKey); err != nil {
+		return nil, err
+	}
+	return pg.url.toGeneric(), nil
+}
+
+// SetURLSigKeys takes in keys and converts & encrypts the data into the backends internal format
+func (pg *PGBackend) SetURLSigKeys(keys []URLSigKey) error {
+	pg.url.fromGeneric(keys)
+	return pg.url.encrypt(pg.cfg.AESKey)
+}
+
+type pgCommonRecord struct {
+	DataEncrypted []byte
+}
+
+type pgDNSSecRecord struct {
+	Key tc.DNSSECKeysTrafficVault
+	CDN string
+	pgCommonRecord
+}
+type pgDNSSecTable struct {
+	Records []pgDNSSecRecord
+}
+
+func (tbl *pgDNSSecTable) gatherKeys(db *sql.DB) error {
+	sz, err := getSize(db, "dnssec")
+	if err != nil {
+		log.Println("PGDNSSec gatherKeys: unable to determine size of dnssec table")
+	}
+	tbl.Records = make([]pgDNSSecRecord, sz)
+
+	rows, err := db.Query("SELECT cdn, data from dnssec")
+	if err != nil {
+		return fmt.Errorf("PGDNSSec gatherKeys: unable to query: %w", err)
+	}
+	defer rows.Close()
+	i := 0
+	for rows.Next() {
+		if i > len(tbl.Records)-1 {
+			return fmt.Errorf("PGDNSSec gatherKeys got more results than expected %v", len(tbl.Records))
+		}
+		err := rows.Scan(&tbl.Records[i].CDN, &tbl.Records[i].DataEncrypted)
+		if err != nil {
+			return fmt.Errorf("PGDNSSec gatherKeys unable to scan row: %w", err)
+		}
+		i += 1
+	}
+	return nil
+}
+func (tbl *pgDNSSecTable) decrypt(aesKey []byte) error {
+	for i, _ := range tbl.Records {
+		err := decryptInto(aesKey, tbl.Records[i].DataEncrypted, &tbl.Records[i].Key)
+		if err != nil {
+			return fmt.Errorf("unable to decrypt into keys: %w", err)
+		}
+	}
+	return nil
+}
+func (tbl *pgDNSSecTable) encrypt(aesKey []byte) error {
+	for i, dns := range tbl.Records {
+		data, err := json.Marshal(&dns.Key)
+		if err != nil {
+			return fmt.Errorf("encrypt issue marshalling keys: %w", err)
+		}
+		dat, err := encrypt(data, aesKey)
+		if err != nil {
+			return fmt.Errorf("encrypt error: %w", err)
+		}
+		tbl.Records[i].DataEncrypted = dat
+	}
+	return nil
+}
+func (tbl *pgDNSSecTable) toGeneric() []DNSSecKey {
+	keys := make([]DNSSecKey, len(tbl.Records))
+
+	for i, record := range tbl.Records {
+		keys[i] = DNSSecKey{
+			CDN:                    record.CDN,
+			DNSSECKeysTrafficVault: record.Key,
+		}
+	}
+
+	return keys
+}
+func (tbl *pgDNSSecTable) fromGeneric(keys []DNSSecKey) {
+	tbl.Records = make([]pgDNSSecRecord, len(keys))
+
+	for i, key := range keys {
+		tbl.Records[i] = pgDNSSecRecord{
+			Key: key.DNSSECKeysTrafficVault,
+			CDN: key.CDN,
+			pgCommonRecord: pgCommonRecord{
+				DataEncrypted: nil,
+			},
+		}
+	}
+}
+func (tbl *pgDNSSecTable) validate() []string {
+	for i, record := range tbl.Records {
+		if record.DataEncrypted == nil && len(record.Key) > 0 {
+			return []string{fmt.Sprintf("DNSSEC Key %v: DataEncrypted is blank!", i)}
+		}
+	}
+	return nil
+}
+func (tbl *pgDNSSecTable) insertKeys(db *sql.DB) error {
+	queryBase := "INSERT INTO dnssec (cdn, data) VALUES "
+	stride := 2
+	queryArgs := make([]interface{}, len(tbl.Records)*stride)
+	for i, record := range tbl.Records {
+		j := i * stride
+		queryArgs[j] = record.CDN
+		queryArgs[j+1] = record.DataEncrypted
+	}
+	return insertIntoTable(db, queryBase, stride, queryArgs)
+}
+
+type pgSSLKeyRecord struct {
+	Keys tc.DeliveryServiceSSLKeys
+	pgCommonRecord
+
+	// These records are stored on the table but are duplicated
+	DeliveryService string
+	CDN             string
+}
+type pgSSLKeyTable struct {
+	Records []pgSSLKeyRecord
+}
+
+func (tbl *pgSSLKeyTable) insertKeys(db *sql.DB) error {
+	queryBase := "INSERT INTO sslkey (deliveryservice, data, cdn, version) VALUES "
+	duplicateKeys := 2
+	stride := 4
+	queryArgs := make([]interface{}, len(tbl.Records)*stride*duplicateKeys)
+	for i, record := range tbl.Records {
+		j := i * duplicateKeys * stride
+
+		queryArgs[j] = record.DeliveryService
+		queryArgs[j+1] = record.DataEncrypted
+		queryArgs[j+2] = record.CDN
+		queryArgs[j+3] = record.Keys.Version.String()
+
+		queryArgs[j+4] = record.DeliveryService
+		queryArgs[j+5] = record.DataEncrypted
+		queryArgs[j+6] = record.CDN
+		queryArgs[j+7] = "latest"
+	}
+	return insertIntoTable(db, queryBase, 4, queryArgs)
+}
+func (tbl *pgSSLKeyTable) gatherKeys(db *sql.DB) error {
+	sz, err := getSize(db, "sslkey WHERE version='latest'")
+	if err != nil {
+		return fmt.Errorf("PGSSLKey gatherKeys unable to determine size of sslkey table: %w", err)
+	}
+	tbl.Records = make([]pgSSLKeyRecord, sz)
+
+	rows, err := db.Query("SELECT data, deliveryservice, cdn from sslkey WHERE version = 'latest'")
+	if err != nil {
+		return fmt.Errorf("PGSSLKey gatherKeys unable to query: %w", err)
+	}
+	defer rows.Close()
+	i := 0
+	for rows.Next() {
+		if i > len(tbl.Records)-1 {
+			return errors.New("PGSSLKey gatherKeys: got more results than expected")
+		}
+		err := rows.Scan(&tbl.Records[i].DataEncrypted, &tbl.Records[i].DeliveryService, &tbl.Records[i].CDN)
+		if err != nil {
+			return fmt.Errorf("PGSSLKey gatherKeys unable to scan row: %w", err)
+		}
+		i += 1
+	}
+	return nil
+}
+func (tbl *pgSSLKeyTable) decrypt(aesKey []byte) error {
+	for i, dns := range tbl.Records {
+		err := decryptInto(aesKey, dns.DataEncrypted, &tbl.Records[i].Keys)
+		if err != nil {
+			return fmt.Errorf("unable to decrypt into keys: %w", err)
+		}
+	}
+	return nil
+}
+func (tbl *pgSSLKeyTable) encrypt(aesKey []byte) error {
+	for i, dns := range tbl.Records {
+		data, err := json.Marshal(dns.Keys)
+		if err != nil {
+			return fmt.Errorf("encrypt issue marshalling keys: %w", err)
+		}
+		dat, err := encrypt(data, aesKey)
+		if err != nil {
+			return fmt.Errorf("encrypt error: %w", err)
+		}
+		tbl.Records[i].DataEncrypted = dat
+	}
+	return nil
+}
+func (tbl *pgSSLKeyTable) toGeneric() []SSLKey {
+	keys := make([]SSLKey, len(tbl.Records))
+
+	for i, record := range tbl.Records {
+		keys[i] = SSLKey{
+			DeliveryServiceSSLKeys: record.Keys,
+		}
+	}
+	return keys
+}
+func (tbl *pgSSLKeyTable) fromGeneric(keys []SSLKey) {
+	tbl.Records = make([]pgSSLKeyRecord, len(keys))
+
+	for i, key := range keys {
+		tbl.Records[i] = pgSSLKeyRecord{
+			Keys: key.DeliveryServiceSSLKeys,
+			pgCommonRecord: pgCommonRecord{
+				DataEncrypted: nil,
+			},
+			DeliveryService: key.DeliveryService,
+			CDN:             key.CDN,
+		}
+	}
+}
+func (tbl *pgSSLKeyTable) validate() []string {
+	defaultKey := tc.DeliveryServiceSSLKeys{}
+	var errors []string
+	fmtStr := "SSL Key %v: %v"
+	for i, record := range tbl.Records {
+		if record.Keys == defaultKey {
+			errors = append(errors, fmt.Sprintf(fmtStr, i, "DS SSL Keys are default!"))
+		} else if record.Keys.Key == "" {
+			errors = append(errors, fmt.Sprintf(fmtStr, i, "Key is blank!"))
+		} else if record.Keys.CDN == "" {
+			errors = append(errors, fmt.Sprintf(fmtStr, i, "CDN is blank!"))
+		} else if record.Keys.DeliveryService == "" {
+			errors = append(errors, fmt.Sprintf(fmtStr, i, "DS is blank!"))
+		} else if record.DataEncrypted == nil {
+			errors = append(errors, fmt.Sprintf(fmtStr, i, "DataEncrypted is blank!"))
+		} else if record.Keys.Version.String() == "" {
+			errors = append(errors, fmt.Sprintf(fmtStr, i, "Version is blank!"))
+		}
+	}
+	return errors
+}
+
+type pgURLSigKeyRecord struct {
+	Keys            tc.URLSigKeys
+	DeliveryService string
+	pgCommonRecord
+}
+type pgURLSigKeyTable struct {
+	Records []pgURLSigKeyRecord
+}
+
+func (tbl *pgURLSigKeyTable) insertKeys(db *sql.DB) error {
+	queryBase := "INSERT INTO url_sig_key (deliveryservice, data) VALUES "
+	stride := 2
+	queryArgs := make([]interface{}, len(tbl.Records)*stride)
+	for i, record := range tbl.Records {
+		j := i * stride
+		queryArgs[j] = record.DeliveryService
+		queryArgs[j+1] = record.DataEncrypted
+	}
+	return insertIntoTable(db, queryBase, stride, queryArgs)
+}
+func (tbl *pgURLSigKeyTable) gatherKeys(db *sql.DB) error {
+	sz, err := getSize(db, "url_sig_key")
+	if err != nil {
+		log.Println("PGURLSigKey gatherKeys: unable to determine url_sig_key table size")
+	}
+	tbl.Records = make([]pgURLSigKeyRecord, sz)
+
+	rows, err := db.Query("SELECT deliveryservice, data from url_sig_key")
+	if err != nil {
+		return fmt.Errorf("PGURLSigKey gatherKeys error creating query: %w", err)
+	}
+	defer rows.Close()
+	i := 0
+	for rows.Next() {
+		if i > len(tbl.Records)-1 {
+			return fmt.Errorf("PGURLSigKey gatherKeys: got more results than expected %v", len(tbl.Records))
+		}
+		err := rows.Scan(&tbl.Records[i].DeliveryService, &tbl.Records[i].DataEncrypted)
+		if err != nil {
+			return fmt.Errorf("PGURLSigKey gatherKeys: unable to scan row: %w", err)
+		}
+		i += 1
+	}
+	return nil
+}
+func (tbl *pgURLSigKeyTable) decrypt(aesKey []byte) error {
+	for i, sig := range tbl.Records {
+		err := decryptInto(aesKey, sig.DataEncrypted, &tbl.Records[i].Keys)
+		if err != nil {
+			return fmt.Errorf("unable to decrypt into keys: %w", err)
+		}
+	}
+	return nil
+}
+func (tbl *pgURLSigKeyTable) encrypt(aesKey []byte) error {
+	for i, sig := range tbl.Records {
+		data, err := json.Marshal(&sig.Keys)
+		if err != nil {
+			return fmt.Errorf("encrypt issue marshalling keys: %w", err)
+		}
+
+		dat, err := encrypt(data, aesKey)
+		if err != nil {
+			return fmt.Errorf("encrypt error: %w", err)
+		}
+		tbl.Records[i].DataEncrypted = dat
+	}
+	return nil
+}
+func (tbl *pgURLSigKeyTable) toGeneric() []URLSigKey {
+	keys := make([]URLSigKey, len(tbl.Records))
+
+	for i, record := range tbl.Records {
+		keys[i] = URLSigKey{
+			DeliveryService: record.DeliveryService,
+			URLSigKeys:      record.Keys,
+		}
+	}
+	return keys
+}
+func (tbl *pgURLSigKeyTable) fromGeneric(keys []URLSigKey) {
+	tbl.Records = make([]pgURLSigKeyRecord, len(keys))
+
+	for i, key := range keys {
+		tbl.Records[i] = pgURLSigKeyRecord{
+			Keys:            key.URLSigKeys,
+			DeliveryService: key.DeliveryService,
+			pgCommonRecord: pgCommonRecord{
+				DataEncrypted: nil,
+			},
+		}
+	}
+}
+func (tbl *pgURLSigKeyTable) validate() []string {
+	for i, record := range tbl.Records {
+		if record.DataEncrypted == nil && len(record.Keys) > 0 {
+			return []string{fmt.Sprintf("URl Sig Key %v: DataEncrypted is blank!", i)}
+		}
+	}
+	return nil
+}
+
+type pgURISignKeyRecord struct {
+	Keys            map[string]tc.URISignerKeyset
+	DeliveryService string
+	pgCommonRecord
+}
+type pgURISignKeyTable struct {
+	Records []pgURISignKeyRecord
+}
+
+func (tbl *pgURISignKeyTable) insertKeys(db *sql.DB) error {
+	queryBase := "INSERT INTO uri_signing_key (deliveryservice, data) VALUES "
+	stride := 2
+	queryArgs := make([]interface{}, len(tbl.Records)*stride)
+	for i, record := range tbl.Records {
+		j := i * stride
+		queryArgs[j] = record.DeliveryService
+		queryArgs[j+1] = record.DataEncrypted
+	}
+	return insertIntoTable(db, queryBase, stride, queryArgs)
+}
+func (tbl *pgURISignKeyTable) gatherKeys(db *sql.DB) error {
+	sz, err := getSize(db, "uri_signing_key")
+	if err != nil {
+		log.Println("PGURISignKey gatherKeys: unable to determine size of uri_signing_key table")
+	}
+	tbl.Records = make([]pgURISignKeyRecord, sz)
+
+	rows, err := db.Query("SELECT deliveryservice, data from uri_signing_key")
+	if err != nil {
+		return fmt.Errorf("PGURISignKey gatherKeys error while query: %w", err)
+	}
+	defer rows.Close()
+	i := 0
+	for rows.Next() {
+		if i > len(tbl.Records)-1 {
+			return fmt.Errorf("PGURISignKey gatherKeys: got more results than expected %v", len(tbl.Records))
+		}
+		err := rows.Scan(&tbl.Records[i].DeliveryService, &tbl.Records[i].DataEncrypted)
+		if err != nil {
+			return fmt.Errorf("PGURISignKey gatherKeys: unable to scan row: %w", err)
+		}
+		i += 1
+	}
+	return nil
+}
+func (tbl *pgURISignKeyTable) decrypt(aesKey []byte) error {
+	for i, sign := range tbl.Records {
+		err := decryptInto(aesKey, sign.DataEncrypted, &tbl.Records[i].Keys)
+		if err != nil {
+			return fmt.Errorf("unable to decrypt into keys: %w", err)
+		}
+	}
+	return nil
+}
+func (tbl *pgURISignKeyTable) encrypt(aesKey []byte) error {
+	for i, sign := range tbl.Records {
+		data, err := json.Marshal(sign.Keys)
+		if err != nil {
+			return fmt.Errorf("encrypt issue marshalling keys: %w", err)
+		}
+
+		dat, err := encrypt(data, aesKey)
+		if err != nil {
+			return fmt.Errorf("encrypt error: %w", err)
+		}
+		tbl.Records[i].DataEncrypted = dat
+	}
+	return nil
+}
+func (tbl *pgURISignKeyTable) toGeneric() []URISignKey {
+	keys := make([]URISignKey, len(tbl.Records))
+
+	for i, record := range tbl.Records {
+		keys[i] = URISignKey{
+			DeliveryService: record.DeliveryService,
+			Keys:            record.Keys,
+		}
+	}
+
+	return keys
+}
+func (tbl *pgURISignKeyTable) fromGeneric(keys []URISignKey) {
+	tbl.Records = make([]pgURISignKeyRecord, len(keys))
+
+	for i, key := range keys {
+		tbl.Records[i] = pgURISignKeyRecord{
+			Keys:            key.Keys,
+			DeliveryService: key.DeliveryService,
+			pgCommonRecord: pgCommonRecord{
+				DataEncrypted: nil,
+			},
+		}
+	}
+}
+func (tbl *pgURISignKeyTable) validate() []string {
+	for i, record := range tbl.Records {
+		if record.DataEncrypted == nil && len(record.Keys) > 0 {
+			return []string{fmt.Sprintf("URI Sign Key %v: DataEncrypted is blank!", i)}

Review comment:
       We should also try to include the DS XMLID in the error message, otherwise I'm not sure how we'd know what index matched what DS for troubleshooting. This applies to the error messages in the other `validate()` methods as well.

##########
File path: tools/traffic_vault_migrate/postgres.go
##########
@@ -0,0 +1,743 @@
+package main
+
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+import (
+	"database/sql"
+	"encoding/base64"
+	"encoding/json"
+	"errors"
+	"fmt"
+	"log"
+	"strconv"
+	"strings"
+
+	"github.com/apache/trafficcontrol/lib/go-tc"
+	util "github.com/apache/trafficcontrol/lib/go-util"
+
+	_ "github.com/lib/pq"
+)
+
+// PGConfig represents the configuration options available to the PG backend
+type PGConfig struct {
+	Host     string `json:"host"`
+	Port     string `json:"port"`
+	User     string `json:"user"`
+	Password string `json:"password"`
+	SSLMode  string `json:"sslmode"`
+	Database string `json:"database"`
+	Key      string `json:"aesKey"`
+	AESKey   []byte
+}
+
+// PGBackend is the Postgres implementation of TVBackend
+type PGBackend struct {
+	sslKey pgSSLKeyTable
+	dnssec pgDNSSecTable
+	uri    pgURISignKeyTable
+	url    pgURLSigKeyTable
+	cfg    PGConfig
+	db     *sql.DB
+}
+
+// String returns a high level overview of the backend and its keys
+func (pg *PGBackend) String() string {
+	data := fmt.Sprintf("PG server %v@%v:%v\n", pg.cfg.User, pg.cfg.Host, pg.cfg.Port)
+	data += fmt.Sprintf("\tSSL Keys: %v\n", len(pg.sslKey.Records))
+	data += fmt.Sprintf("\tDNSSec Keys: %v\n", len(pg.dnssec.Records))
+	data += fmt.Sprintf("\tURI Keys: %v\n", len(pg.uri.Records))
+	data += fmt.Sprintf("\tURL Keys: %v\n", len(pg.url.Records))
+	return data
+}
+
+// Name returns the name for this backend
+func (pg *PGBackend) Name() string {
+	return "PG"
+}
+
+// ReadConfig takes in a filename and will read it into the backends config
+func (pg *PGBackend) ReadConfig(s string) error {
+	err := UnmarshalConfig(s, &pg.cfg)
+	if err != nil {
+		return err
+	}
+
+	pg.cfg.AESKey, err = base64.StdEncoding.DecodeString(pg.cfg.Key)
+	if err != nil {
+		return fmt.Errorf("unable to decode PG AESKey: %w", err)
+	}
+	return nil
+}
+
+// Insert takes the current keys and inserts them into the backend DB
+func (pg *PGBackend) Insert() error {
+	if err := pg.sslKey.insertKeys(pg.db); err != nil {
+		return err
+	}
+	if err := pg.dnssec.insertKeys(pg.db); err != nil {
+		return err
+	}
+	if err := pg.url.insertKeys(pg.db); err != nil {
+		return err
+	}
+	if err := pg.uri.insertKeys(pg.db); err != nil {
+		return err
+	}
+	return nil
+}
+
+// Start initiates the connection to the backend DB
+func (pg *PGBackend) Start() error {
+	sqlStr := fmt.Sprintf("postgres://%s:%s@%s:%s/%s?sslmode=%s", pg.cfg.User, pg.cfg.Password, pg.cfg.Host, pg.cfg.Port, pg.cfg.Database, pg.cfg.SSLMode)
+	db, err := sql.Open("postgres", sqlStr)
+	if err != nil {
+		return fmt.Errorf("unable to start PG client: %w", err)
+	}
+
+	pg.db = db
+	pg.sslKey = pgSSLKeyTable{}
+	pg.dnssec = pgDNSSecTable{}
+	pg.url = pgURLSigKeyTable{}
+	pg.uri = pgURISignKeyTable{}
+
+	return nil
+}
+
+// ValidateKey validates that the keys are valid (in most cases, certain fields are not null)
+func (pg *PGBackend) ValidateKey() []string {
+	var errors []string
+	if errs := pg.sslKey.validate(); errs != nil {
+		errors = append(errors, errs...)
+	}
+	if errs := pg.dnssec.validate(); errs != nil {
+		errors = append(errors, errs...)
+	}
+	if errs := pg.uri.validate(); errs != nil {
+		errors = append(errors, errs...)
+	}
+	if errs := pg.url.validate(); errs != nil {
+		errors = append(errors, errs...)
+	}
+	return errors
+}
+
+// Close terminates the connection to the backend DB
+func (pg *PGBackend) Close() error {
+	return pg.db.Close()
+}
+
+// Ping checks the connection to the backend DB
+func (pg *PGBackend) Ping() error {
+	return pg.db.Ping()
+}
+
+// Fetch gets all of the keys from the backend DB
+func (pg *PGBackend) Fetch() error {
+	if err := pg.sslKey.gatherKeys(pg.db); err != nil {
+		return err
+	}
+
+	if err := pg.dnssec.gatherKeys(pg.db); err != nil {
+		return err
+	}
+
+	if err := pg.url.gatherKeys(pg.db); err != nil {
+		return err
+	}
+
+	if err := pg.uri.gatherKeys(pg.db); err != nil {
+		return err
+	}
+
+	return nil
+}
+
+// GetSSLKeys converts the backends internal key representation into the common representation (SSLKey)
+func (pg *PGBackend) GetSSLKeys() ([]SSLKey, error) {
+	if err := pg.sslKey.decrypt(pg.cfg.AESKey); err != nil {
+		return nil, err
+	}
+	return pg.sslKey.toGeneric(), nil
+}
+
+// SetSSLKeys takes in keys and converts & encrypts the data into the backends internal format
+func (pg *PGBackend) SetSSLKeys(keys []SSLKey) error {
+	pg.sslKey.fromGeneric(keys)
+	return pg.sslKey.encrypt(pg.cfg.AESKey)
+}
+
+// GetDNSSecKeys converts the backends internal key representation into the common representation (DNSSecKey)
+func (pg *PGBackend) GetDNSSecKeys() ([]DNSSecKey, error) {
+	if err := pg.dnssec.decrypt(pg.cfg.AESKey); err != nil {
+		return nil, err
+	}
+	return pg.dnssec.toGeneric(), nil
+}
+
+// SetDNSSecKeys takes in keys and converts & encrypts the data into the backends internal format
+func (pg *PGBackend) SetDNSSecKeys(keys []DNSSecKey) error {
+	pg.dnssec.fromGeneric(keys)
+	return pg.dnssec.encrypt(pg.cfg.AESKey)
+}
+
+// GetURISignKeys converts the pg internal key representation into the common representation (URISignKey)
+func (pg *PGBackend) GetURISignKeys() ([]URISignKey, error) {
+	if err := pg.uri.decrypt(pg.cfg.AESKey); err != nil {
+		return nil, err
+	}
+	return pg.uri.toGeneric(), nil
+}
+
+// SetURISignKeys takes in keys and converts & encrypts the data into the backends internal format
+func (pg *PGBackend) SetURISignKeys(keys []URISignKey) error {
+	pg.uri.fromGeneric(keys)
+	return pg.uri.encrypt(pg.cfg.AESKey)
+}
+
+// GetURLSigKeys converts the backends internal key representation into the common representation (URLSigKey)
+func (pg *PGBackend) GetURLSigKeys() ([]URLSigKey, error) {
+	if err := pg.url.decrypt(pg.cfg.AESKey); err != nil {
+		return nil, err
+	}
+	return pg.url.toGeneric(), nil
+}
+
+// SetURLSigKeys takes in keys and converts & encrypts the data into the backends internal format
+func (pg *PGBackend) SetURLSigKeys(keys []URLSigKey) error {
+	pg.url.fromGeneric(keys)
+	return pg.url.encrypt(pg.cfg.AESKey)
+}
+
+type pgCommonRecord struct {
+	DataEncrypted []byte
+}
+
+type pgDNSSecRecord struct {
+	Key tc.DNSSECKeysTrafficVault
+	CDN string
+	pgCommonRecord
+}
+type pgDNSSecTable struct {
+	Records []pgDNSSecRecord
+}
+
+func (tbl *pgDNSSecTable) gatherKeys(db *sql.DB) error {
+	sz, err := getSize(db, "dnssec")
+	if err != nil {
+		log.Println("PGDNSSec gatherKeys: unable to determine size of dnssec table")
+	}
+	tbl.Records = make([]pgDNSSecRecord, sz)
+
+	rows, err := db.Query("SELECT cdn, data from dnssec")
+	if err != nil {
+		return fmt.Errorf("PGDNSSec gatherKeys: unable to query: %w", err)
+	}
+	defer rows.Close()
+	i := 0
+	for rows.Next() {
+		if i > len(tbl.Records)-1 {
+			return fmt.Errorf("PGDNSSec gatherKeys got more results than expected %v", len(tbl.Records))
+		}
+		err := rows.Scan(&tbl.Records[i].CDN, &tbl.Records[i].DataEncrypted)
+		if err != nil {
+			return fmt.Errorf("PGDNSSec gatherKeys unable to scan row: %w", err)
+		}
+		i += 1
+	}
+	return nil
+}
+func (tbl *pgDNSSecTable) decrypt(aesKey []byte) error {
+	for i, _ := range tbl.Records {
+		err := decryptInto(aesKey, tbl.Records[i].DataEncrypted, &tbl.Records[i].Key)
+		if err != nil {
+			return fmt.Errorf("unable to decrypt into keys: %w", err)
+		}
+	}
+	return nil
+}
+func (tbl *pgDNSSecTable) encrypt(aesKey []byte) error {
+	for i, dns := range tbl.Records {
+		data, err := json.Marshal(&dns.Key)
+		if err != nil {
+			return fmt.Errorf("encrypt issue marshalling keys: %w", err)
+		}
+		dat, err := encrypt(data, aesKey)
+		if err != nil {
+			return fmt.Errorf("encrypt error: %w", err)
+		}
+		tbl.Records[i].DataEncrypted = dat
+	}
+	return nil
+}
+func (tbl *pgDNSSecTable) toGeneric() []DNSSecKey {
+	keys := make([]DNSSecKey, len(tbl.Records))
+
+	for i, record := range tbl.Records {
+		keys[i] = DNSSecKey{
+			CDN:                    record.CDN,
+			DNSSECKeysTrafficVault: record.Key,
+		}
+	}
+
+	return keys
+}
+func (tbl *pgDNSSecTable) fromGeneric(keys []DNSSecKey) {
+	tbl.Records = make([]pgDNSSecRecord, len(keys))
+
+	for i, key := range keys {
+		tbl.Records[i] = pgDNSSecRecord{
+			Key: key.DNSSECKeysTrafficVault,
+			CDN: key.CDN,
+			pgCommonRecord: pgCommonRecord{
+				DataEncrypted: nil,
+			},
+		}
+	}
+}
+func (tbl *pgDNSSecTable) validate() []string {
+	for i, record := range tbl.Records {
+		if record.DataEncrypted == nil && len(record.Key) > 0 {
+			return []string{fmt.Sprintf("DNSSEC Key %v: DataEncrypted is blank!", i)}
+		}
+	}
+	return nil
+}
+func (tbl *pgDNSSecTable) insertKeys(db *sql.DB) error {
+	queryBase := "INSERT INTO dnssec (cdn, data) VALUES "
+	stride := 2
+	queryArgs := make([]interface{}, len(tbl.Records)*stride)
+	for i, record := range tbl.Records {
+		j := i * stride
+		queryArgs[j] = record.CDN
+		queryArgs[j+1] = record.DataEncrypted
+	}
+	return insertIntoTable(db, queryBase, stride, queryArgs)
+}
+
+type pgSSLKeyRecord struct {
+	Keys tc.DeliveryServiceSSLKeys
+	pgCommonRecord
+
+	// These records are stored on the table but are duplicated
+	DeliveryService string
+	CDN             string
+}
+type pgSSLKeyTable struct {
+	Records []pgSSLKeyRecord
+}
+
+func (tbl *pgSSLKeyTable) insertKeys(db *sql.DB) error {
+	queryBase := "INSERT INTO sslkey (deliveryservice, data, cdn, version) VALUES "
+	duplicateKeys := 2
+	stride := 4
+	queryArgs := make([]interface{}, len(tbl.Records)*stride*duplicateKeys)
+	for i, record := range tbl.Records {
+		j := i * duplicateKeys * stride
+
+		queryArgs[j] = record.DeliveryService
+		queryArgs[j+1] = record.DataEncrypted
+		queryArgs[j+2] = record.CDN
+		queryArgs[j+3] = record.Keys.Version.String()
+
+		queryArgs[j+4] = record.DeliveryService
+		queryArgs[j+5] = record.DataEncrypted
+		queryArgs[j+6] = record.CDN
+		queryArgs[j+7] = "latest"
+	}
+	return insertIntoTable(db, queryBase, 4, queryArgs)
+}
+func (tbl *pgSSLKeyTable) gatherKeys(db *sql.DB) error {
+	sz, err := getSize(db, "sslkey WHERE version='latest'")
+	if err != nil {
+		return fmt.Errorf("PGSSLKey gatherKeys unable to determine size of sslkey table: %w", err)
+	}
+	tbl.Records = make([]pgSSLKeyRecord, sz)
+
+	rows, err := db.Query("SELECT data, deliveryservice, cdn from sslkey WHERE version = 'latest'")
+	if err != nil {
+		return fmt.Errorf("PGSSLKey gatherKeys unable to query: %w", err)
+	}
+	defer rows.Close()
+	i := 0
+	for rows.Next() {
+		if i > len(tbl.Records)-1 {
+			return errors.New("PGSSLKey gatherKeys: got more results than expected")
+		}
+		err := rows.Scan(&tbl.Records[i].DataEncrypted, &tbl.Records[i].DeliveryService, &tbl.Records[i].CDN)
+		if err != nil {
+			return fmt.Errorf("PGSSLKey gatherKeys unable to scan row: %w", err)
+		}
+		i += 1
+	}
+	return nil
+}
+func (tbl *pgSSLKeyTable) decrypt(aesKey []byte) error {
+	for i, dns := range tbl.Records {
+		err := decryptInto(aesKey, dns.DataEncrypted, &tbl.Records[i].Keys)
+		if err != nil {
+			return fmt.Errorf("unable to decrypt into keys: %w", err)
+		}
+	}
+	return nil
+}
+func (tbl *pgSSLKeyTable) encrypt(aesKey []byte) error {
+	for i, dns := range tbl.Records {
+		data, err := json.Marshal(dns.Keys)
+		if err != nil {
+			return fmt.Errorf("encrypt issue marshalling keys: %w", err)
+		}
+		dat, err := encrypt(data, aesKey)
+		if err != nil {
+			return fmt.Errorf("encrypt error: %w", err)
+		}
+		tbl.Records[i].DataEncrypted = dat
+	}
+	return nil
+}
+func (tbl *pgSSLKeyTable) toGeneric() []SSLKey {
+	keys := make([]SSLKey, len(tbl.Records))
+
+	for i, record := range tbl.Records {
+		keys[i] = SSLKey{
+			DeliveryServiceSSLKeys: record.Keys,
+		}
+	}
+	return keys
+}
+func (tbl *pgSSLKeyTable) fromGeneric(keys []SSLKey) {
+	tbl.Records = make([]pgSSLKeyRecord, len(keys))
+
+	for i, key := range keys {
+		tbl.Records[i] = pgSSLKeyRecord{
+			Keys: key.DeliveryServiceSSLKeys,
+			pgCommonRecord: pgCommonRecord{
+				DataEncrypted: nil,
+			},
+			DeliveryService: key.DeliveryService,
+			CDN:             key.CDN,
+		}
+	}
+}
+func (tbl *pgSSLKeyTable) validate() []string {
+	defaultKey := tc.DeliveryServiceSSLKeys{}
+	var errors []string
+	fmtStr := "SSL Key %v: %v"
+	for i, record := range tbl.Records {
+		if record.Keys == defaultKey {
+			errors = append(errors, fmt.Sprintf(fmtStr, i, "DS SSL Keys are default!"))
+		} else if record.Keys.Key == "" {
+			errors = append(errors, fmt.Sprintf(fmtStr, i, "Key is blank!"))
+		} else if record.Keys.CDN == "" {
+			errors = append(errors, fmt.Sprintf(fmtStr, i, "CDN is blank!"))
+		} else if record.Keys.DeliveryService == "" {
+			errors = append(errors, fmt.Sprintf(fmtStr, i, "DS is blank!"))
+		} else if record.DataEncrypted == nil {
+			errors = append(errors, fmt.Sprintf(fmtStr, i, "DataEncrypted is blank!"))
+		} else if record.Keys.Version.String() == "" {
+			errors = append(errors, fmt.Sprintf(fmtStr, i, "Version is blank!"))
+		}
+	}
+	return errors
+}
+
+type pgURLSigKeyRecord struct {
+	Keys            tc.URLSigKeys
+	DeliveryService string
+	pgCommonRecord
+}
+type pgURLSigKeyTable struct {
+	Records []pgURLSigKeyRecord
+}
+
+func (tbl *pgURLSigKeyTable) insertKeys(db *sql.DB) error {
+	queryBase := "INSERT INTO url_sig_key (deliveryservice, data) VALUES "
+	stride := 2
+	queryArgs := make([]interface{}, len(tbl.Records)*stride)
+	for i, record := range tbl.Records {
+		j := i * stride
+		queryArgs[j] = record.DeliveryService
+		queryArgs[j+1] = record.DataEncrypted
+	}
+	return insertIntoTable(db, queryBase, stride, queryArgs)
+}
+func (tbl *pgURLSigKeyTable) gatherKeys(db *sql.DB) error {
+	sz, err := getSize(db, "url_sig_key")
+	if err != nil {
+		log.Println("PGURLSigKey gatherKeys: unable to determine url_sig_key table size")
+	}
+	tbl.Records = make([]pgURLSigKeyRecord, sz)
+
+	rows, err := db.Query("SELECT deliveryservice, data from url_sig_key")
+	if err != nil {
+		return fmt.Errorf("PGURLSigKey gatherKeys error creating query: %w", err)
+	}
+	defer rows.Close()
+	i := 0
+	for rows.Next() {
+		if i > len(tbl.Records)-1 {
+			return fmt.Errorf("PGURLSigKey gatherKeys: got more results than expected %v", len(tbl.Records))
+		}
+		err := rows.Scan(&tbl.Records[i].DeliveryService, &tbl.Records[i].DataEncrypted)
+		if err != nil {
+			return fmt.Errorf("PGURLSigKey gatherKeys: unable to scan row: %w", err)
+		}
+		i += 1
+	}
+	return nil
+}
+func (tbl *pgURLSigKeyTable) decrypt(aesKey []byte) error {
+	for i, sig := range tbl.Records {
+		err := decryptInto(aesKey, sig.DataEncrypted, &tbl.Records[i].Keys)

Review comment:
       nit: this can be combined with the line below like `if err := function(); err != nil {`

##########
File path: tools/traffic_vault_migrate/riak.go
##########
@@ -0,0 +1,746 @@
+package main
+
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+import (
+	"crypto/tls"
+	"encoding/json"
+	"errors"
+	"fmt"
+	"time"
+
+	"github.com/apache/trafficcontrol/lib/go-rfc"
+	"github.com/apache/trafficcontrol/lib/go-tc"
+
+	"github.com/basho/riak-go-client"
+)
+
+const (
+	BUCKET_SSL     = "ssl"
+	BUCKET_DNSSEC  = "dnssec"
+	BUCKET_URL_SIG = "url_sig_keys"
+	BUCKET_URI_SIG = "cdn_uri_sig_keys"
+
+	INDEX_SSL = "sslkeys"
+
+	SCHEMA_RIAK_KEY    = "_yz_rk"
+	SCHEMA_RIAK_BUCKET = "_yz_rb"
+)
+
+var (
+	SCHEMA_SSL_FIELDS = [...]string{SCHEMA_RIAK_KEY, SCHEMA_RIAK_BUCKET}
+)
+
+// RiakConfig  represents the configuration options available to the Riak backend
+type RiakConfig struct {
+	Host          string `json:"host"`
+	Port          string `json:"port"`
+	User          string `json:"user"`
+	Password      string `json:"password"`
+	Insecure      bool   `json:"insecure"`
+	TLSVersionRaw string `json:"tlsVersion"`
+	TLSVersion    uint16
+}
+
+// RiakBackend is the Riak implementation of TVBackend
+type RiakBackend struct {
+	sslKeys     riakSSLKeyTable
+	dnssecKeys  riakDNSSecKeyTable
+	uriSignKeys riakURISignKeyTable
+	urlSigKeys  riakURLSigKeyTable
+	cfg         RiakConfig
+	cluster     *riak.Cluster
+}
+
+// String returns a high level overview of the backend and its keys
+func (rb *RiakBackend) String() string {
+	data := fmt.Sprintf("Riak server %v@%v:%v\n", rb.cfg.User, rb.cfg.Host, rb.cfg.Port)
+	data += fmt.Sprintf("\tSSL Keys: %v\n", len(rb.sslKeys.Records))
+	data += fmt.Sprintf("\tDNSSec Keys: %v\n", len(rb.dnssecKeys.Records))
+	data += fmt.Sprintf("\tURI Keys: %v\n", len(rb.uriSignKeys.Records))
+	data += fmt.Sprintf("\tURL Keys: %v\n", len(rb.urlSigKeys.Records))
+	return data
+}
+
+// Name returns the name for this backend
+func (rb *RiakBackend) Name() string {
+	return "Riak"
+}
+
+// ReadConfig takes in a filename and will read it into the backends config
+func (rb *RiakBackend) ReadConfig(s string) error {
+	err := UnmarshalConfig(s, &rb.cfg)
+	if err != nil {
+		return err
+	}
+
+	if rb.cfg.TLSVersionRaw == "10" {
+		rb.cfg.TLSVersion = tls.VersionTLS10
+	} else if rb.cfg.TLSVersionRaw == "11" {
+		rb.cfg.TLSVersion = tls.VersionTLS11
+	} else if rb.cfg.TLSVersionRaw == "12" {
+		rb.cfg.TLSVersion = tls.VersionTLS12
+	} else if rb.cfg.TLSVersionRaw == "13" {
+		rb.cfg.TLSVersion = tls.VersionTLS13
+	}
+	return nil
+}
+
+// Insert takes the current keys and inserts them into the backend DB
+func (rb *RiakBackend) Insert() error {
+	if err := rb.sslKeys.insertKeys(rb.cluster); err != nil {
+		return err
+	}
+	if err := rb.dnssecKeys.insertKeys(rb.cluster); err != nil {
+		return err
+	}
+	if err := rb.urlSigKeys.insertKeys(rb.cluster); err != nil {
+		return err
+	}
+	if err := rb.uriSignKeys.insertKeys(rb.cluster); err != nil {
+		return err
+	}
+	return nil
+}
+
+// ValidateKey validates that the keys are valid (in most cases, certain fields are not null)
+func (rb *RiakBackend) ValidateKey() []string {
+	errors := []string{}
+	if errs := rb.sslKeys.validate(); errs != nil {
+		errors = append(errors, errs...)
+	}
+	if errs := rb.dnssecKeys.validate(); errs != nil {
+		errors = append(errors, errs...)
+	}
+	if errs := rb.uriSignKeys.validate(); errs != nil {
+		errors = append(errors, errs...)
+	}
+	if errs := rb.urlSigKeys.validate(); errs != nil {
+		errors = append(errors, errs...)
+	}
+
+	return errors
+}
+
+// SetSSLKeys takes in keys and converts & encrypts the data into the backends internal format
+func (rb *RiakBackend) SetSSLKeys(keys []SSLKey) error {
+	rb.sslKeys.fromGeneric(keys)
+	return nil
+}
+
+// SetDNSSecKeys takes in keys and converts & encrypts the data into the backends internal format
+func (rb *RiakBackend) SetDNSSecKeys(keys []DNSSecKey) error {
+	rb.dnssecKeys.fromGeneric(keys)
+	return nil
+}
+
+// SetURISignKeys takes in keys and converts & encrypts the data into the backends internal format
+func (rb *RiakBackend) SetURISignKeys(keys []URISignKey) error {
+	rb.uriSignKeys.fromGeneric(keys)
+	return nil
+}
+
+// SetURLSigKeys takes in keys and converts & encrypts the data into the backends internal format
+func (rb *RiakBackend) SetURLSigKeys(keys []URLSigKey) error {
+	rb.urlSigKeys.fromGeneric(keys)
+	return nil
+}
+
+// Start initiates the connection to the backend DB
+func (rb *RiakBackend) Start() error {
+	tlsConfig := &tls.Config{
+		InsecureSkipVerify: !rb.cfg.Insecure,
+		MaxVersion:         rb.cfg.TLSVersion,
+	}
+	auth := &riak.AuthOptions{
+		User:      rb.cfg.User,
+		Password:  rb.cfg.Password,
+		TlsConfig: tlsConfig,
+	}
+
+	cluster, err := getRiakCluster(rb.cfg, auth)
+	if err != nil {
+		return err
+	}
+	if err := cluster.Start(); err != nil {
+		return fmt.Errorf("unable to start riak cluster: %w", err)
+	}
+
+	rb.cluster = cluster
+	rb.sslKeys = riakSSLKeyTable{}
+	rb.dnssecKeys = riakDNSSecKeyTable{}
+	rb.urlSigKeys = riakURLSigKeyTable{}
+	rb.uriSignKeys = riakURISignKeyTable{}
+	return nil
+}
+
+// Close terminates the connection to the backend DB
+func (rb *RiakBackend) Close() error {
+	if err := rb.cluster.Stop(); err != nil {
+		return err
+	}
+	return nil
+}
+
+// Ping checks the connection to the backend DB
+func (rb *RiakBackend) Ping() error {
+	return ping(rb.cluster)
+}
+
+// GetSSLKeys converts the backends internal key representation into the common representation (SSLKey)
+func (rb *RiakBackend) GetSSLKeys() ([]SSLKey, error) {
+	return rb.sslKeys.toGeneric(), nil
+}
+
+// GetDNSSecKeys converts the backends internal key representation into the common representation (DNSSecKey)
+func (rb *RiakBackend) GetDNSSecKeys() ([]DNSSecKey, error) {
+	return rb.dnssecKeys.toGeneric(), nil
+}
+
+// GetURISignKeys converts the pg internal key representation into the common representation (URISignKey)
+func (rb *RiakBackend) GetURISignKeys() ([]URISignKey, error) {
+	return rb.uriSignKeys.toGeneric(), nil
+}
+
+// GetURLSigKeys converts the backends internal key representation into the common representation (URLSigKey)
+func (rb *RiakBackend) GetURLSigKeys() ([]URLSigKey, error) {
+	return rb.urlSigKeys.toGeneric(), nil
+}
+
+// Fetch gets all of the keys from the backend DB
+func (rb *RiakBackend) Fetch() error {
+	if err := rb.sslKeys.gatherKeys(rb.cluster); err != nil {
+		return err
+	}
+	if err := rb.dnssecKeys.gatherKeys(rb.cluster); err != nil {
+		return err
+	}
+	if err := rb.urlSigKeys.gatherKeys(rb.cluster); err != nil {
+		return err
+	}
+	if err := rb.uriSignKeys.gatherKeys(rb.cluster); err != nil {
+		return err
+	}
+
+	return nil
+}
+
+type riakSSLKeyRecord struct {
+	tc.DeliveryServiceSSLKeys
+}
+type riakSSLKeyTable struct {
+	Records []riakSSLKeyRecord
+}
+
+func (tbl *riakSSLKeyTable) gatherKeys(cluster *riak.Cluster) error {
+	searchDocs, err := search(cluster, INDEX_SSL, "cdn:*", fmt.Sprintf("%v:*latest", SCHEMA_RIAK_KEY), 1000, SCHEMA_SSL_FIELDS[:])
+	if err != nil {
+		return fmt.Errorf("RiakSSLKey gatherKeys: %w", err)
+	}
+
+	tbl.Records = make([]riakSSLKeyRecord, len(searchDocs))
+	for i, doc := range searchDocs {
+		objs, err := getObject(cluster, doc.Bucket, doc.Key)
+		if err != nil {
+			return err
+		}
+		if len(objs) < 1 {
+			return fmt.Errorf("RiakSSLKey gatherKeys unable to find any objects with key %v and bucket %v, but search results were returned", doc.Key, doc.Bucket)
+		}
+		if len(objs) > 1 {
+			return fmt.Errorf("RiakSSLKey gatherKeys more than 1 ssl key record found %v\n", len(objs))

Review comment:
       nit: type-specific formatting instead of %v

##########
File path: tools/traffic_vault_migrate/postgres.go
##########
@@ -0,0 +1,743 @@
+package main
+
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+import (
+	"database/sql"
+	"encoding/base64"
+	"encoding/json"
+	"errors"
+	"fmt"
+	"log"
+	"strconv"
+	"strings"
+
+	"github.com/apache/trafficcontrol/lib/go-tc"
+	util "github.com/apache/trafficcontrol/lib/go-util"
+
+	_ "github.com/lib/pq"
+)
+
+// PGConfig represents the configuration options available to the PG backend
+type PGConfig struct {
+	Host     string `json:"host"`
+	Port     string `json:"port"`
+	User     string `json:"user"`
+	Password string `json:"password"`
+	SSLMode  string `json:"sslmode"`
+	Database string `json:"database"`
+	Key      string `json:"aesKey"`
+	AESKey   []byte
+}
+
+// PGBackend is the Postgres implementation of TVBackend
+type PGBackend struct {
+	sslKey pgSSLKeyTable
+	dnssec pgDNSSecTable
+	uri    pgURISignKeyTable
+	url    pgURLSigKeyTable
+	cfg    PGConfig
+	db     *sql.DB
+}
+
+// String returns a high level overview of the backend and its keys
+func (pg *PGBackend) String() string {
+	data := fmt.Sprintf("PG server %v@%v:%v\n", pg.cfg.User, pg.cfg.Host, pg.cfg.Port)
+	data += fmt.Sprintf("\tSSL Keys: %v\n", len(pg.sslKey.Records))
+	data += fmt.Sprintf("\tDNSSec Keys: %v\n", len(pg.dnssec.Records))
+	data += fmt.Sprintf("\tURI Keys: %v\n", len(pg.uri.Records))
+	data += fmt.Sprintf("\tURL Keys: %v\n", len(pg.url.Records))
+	return data
+}
+
+// Name returns the name for this backend
+func (pg *PGBackend) Name() string {
+	return "PG"
+}
+
+// ReadConfig takes in a filename and will read it into the backends config
+func (pg *PGBackend) ReadConfig(s string) error {
+	err := UnmarshalConfig(s, &pg.cfg)
+	if err != nil {
+		return err
+	}
+
+	pg.cfg.AESKey, err = base64.StdEncoding.DecodeString(pg.cfg.Key)
+	if err != nil {
+		return fmt.Errorf("unable to decode PG AESKey: %w", err)
+	}
+	return nil
+}
+
+// Insert takes the current keys and inserts them into the backend DB
+func (pg *PGBackend) Insert() error {
+	if err := pg.sslKey.insertKeys(pg.db); err != nil {
+		return err
+	}
+	if err := pg.dnssec.insertKeys(pg.db); err != nil {
+		return err
+	}
+	if err := pg.url.insertKeys(pg.db); err != nil {
+		return err
+	}
+	if err := pg.uri.insertKeys(pg.db); err != nil {
+		return err
+	}
+	return nil
+}
+
+// Start initiates the connection to the backend DB
+func (pg *PGBackend) Start() error {
+	sqlStr := fmt.Sprintf("postgres://%s:%s@%s:%s/%s?sslmode=%s", pg.cfg.User, pg.cfg.Password, pg.cfg.Host, pg.cfg.Port, pg.cfg.Database, pg.cfg.SSLMode)
+	db, err := sql.Open("postgres", sqlStr)
+	if err != nil {
+		return fmt.Errorf("unable to start PG client: %w", err)
+	}
+
+	pg.db = db
+	pg.sslKey = pgSSLKeyTable{}
+	pg.dnssec = pgDNSSecTable{}
+	pg.url = pgURLSigKeyTable{}
+	pg.uri = pgURISignKeyTable{}
+
+	return nil
+}
+
+// ValidateKey validates that the keys are valid (in most cases, certain fields are not null)
+func (pg *PGBackend) ValidateKey() []string {
+	var errors []string
+	if errs := pg.sslKey.validate(); errs != nil {
+		errors = append(errors, errs...)
+	}
+	if errs := pg.dnssec.validate(); errs != nil {
+		errors = append(errors, errs...)
+	}
+	if errs := pg.uri.validate(); errs != nil {
+		errors = append(errors, errs...)
+	}
+	if errs := pg.url.validate(); errs != nil {
+		errors = append(errors, errs...)
+	}
+	return errors
+}
+
+// Close terminates the connection to the backend DB
+func (pg *PGBackend) Close() error {
+	return pg.db.Close()
+}
+
+// Ping checks the connection to the backend DB
+func (pg *PGBackend) Ping() error {
+	return pg.db.Ping()
+}
+
+// Fetch gets all of the keys from the backend DB
+func (pg *PGBackend) Fetch() error {
+	if err := pg.sslKey.gatherKeys(pg.db); err != nil {
+		return err
+	}
+
+	if err := pg.dnssec.gatherKeys(pg.db); err != nil {
+		return err
+	}
+
+	if err := pg.url.gatherKeys(pg.db); err != nil {
+		return err
+	}
+
+	if err := pg.uri.gatherKeys(pg.db); err != nil {
+		return err
+	}
+
+	return nil
+}
+
+// GetSSLKeys converts the backends internal key representation into the common representation (SSLKey)
+func (pg *PGBackend) GetSSLKeys() ([]SSLKey, error) {
+	if err := pg.sslKey.decrypt(pg.cfg.AESKey); err != nil {
+		return nil, err
+	}
+	return pg.sslKey.toGeneric(), nil
+}
+
+// SetSSLKeys takes in keys and converts & encrypts the data into the backends internal format
+func (pg *PGBackend) SetSSLKeys(keys []SSLKey) error {
+	pg.sslKey.fromGeneric(keys)
+	return pg.sslKey.encrypt(pg.cfg.AESKey)
+}
+
+// GetDNSSecKeys converts the backends internal key representation into the common representation (DNSSecKey)
+func (pg *PGBackend) GetDNSSecKeys() ([]DNSSecKey, error) {
+	if err := pg.dnssec.decrypt(pg.cfg.AESKey); err != nil {
+		return nil, err
+	}
+	return pg.dnssec.toGeneric(), nil
+}
+
+// SetDNSSecKeys takes in keys and converts & encrypts the data into the backends internal format
+func (pg *PGBackend) SetDNSSecKeys(keys []DNSSecKey) error {
+	pg.dnssec.fromGeneric(keys)
+	return pg.dnssec.encrypt(pg.cfg.AESKey)
+}
+
+// GetURISignKeys converts the pg internal key representation into the common representation (URISignKey)
+func (pg *PGBackend) GetURISignKeys() ([]URISignKey, error) {
+	if err := pg.uri.decrypt(pg.cfg.AESKey); err != nil {
+		return nil, err
+	}
+	return pg.uri.toGeneric(), nil
+}
+
+// SetURISignKeys takes in keys and converts & encrypts the data into the backends internal format
+func (pg *PGBackend) SetURISignKeys(keys []URISignKey) error {
+	pg.uri.fromGeneric(keys)
+	return pg.uri.encrypt(pg.cfg.AESKey)
+}
+
+// GetURLSigKeys converts the backends internal key representation into the common representation (URLSigKey)
+func (pg *PGBackend) GetURLSigKeys() ([]URLSigKey, error) {
+	if err := pg.url.decrypt(pg.cfg.AESKey); err != nil {
+		return nil, err
+	}
+	return pg.url.toGeneric(), nil
+}
+
+// SetURLSigKeys takes in keys and converts & encrypts the data into the backends internal format
+func (pg *PGBackend) SetURLSigKeys(keys []URLSigKey) error {
+	pg.url.fromGeneric(keys)
+	return pg.url.encrypt(pg.cfg.AESKey)
+}
+
+type pgCommonRecord struct {
+	DataEncrypted []byte
+}
+
+type pgDNSSecRecord struct {
+	Key tc.DNSSECKeysTrafficVault
+	CDN string
+	pgCommonRecord
+}
+type pgDNSSecTable struct {
+	Records []pgDNSSecRecord
+}
+
+func (tbl *pgDNSSecTable) gatherKeys(db *sql.DB) error {
+	sz, err := getSize(db, "dnssec")
+	if err != nil {
+		log.Println("PGDNSSec gatherKeys: unable to determine size of dnssec table")
+	}
+	tbl.Records = make([]pgDNSSecRecord, sz)
+
+	rows, err := db.Query("SELECT cdn, data from dnssec")
+	if err != nil {
+		return fmt.Errorf("PGDNSSec gatherKeys: unable to query: %w", err)
+	}
+	defer rows.Close()
+	i := 0
+	for rows.Next() {
+		if i > len(tbl.Records)-1 {
+			return fmt.Errorf("PGDNSSec gatherKeys got more results than expected %v", len(tbl.Records))
+		}
+		err := rows.Scan(&tbl.Records[i].CDN, &tbl.Records[i].DataEncrypted)
+		if err != nil {
+			return fmt.Errorf("PGDNSSec gatherKeys unable to scan row: %w", err)
+		}
+		i += 1
+	}
+	return nil
+}
+func (tbl *pgDNSSecTable) decrypt(aesKey []byte) error {
+	for i, _ := range tbl.Records {
+		err := decryptInto(aesKey, tbl.Records[i].DataEncrypted, &tbl.Records[i].Key)
+		if err != nil {
+			return fmt.Errorf("unable to decrypt into keys: %w", err)
+		}
+	}
+	return nil
+}
+func (tbl *pgDNSSecTable) encrypt(aesKey []byte) error {
+	for i, dns := range tbl.Records {
+		data, err := json.Marshal(&dns.Key)
+		if err != nil {
+			return fmt.Errorf("encrypt issue marshalling keys: %w", err)
+		}
+		dat, err := encrypt(data, aesKey)
+		if err != nil {
+			return fmt.Errorf("encrypt error: %w", err)
+		}
+		tbl.Records[i].DataEncrypted = dat
+	}
+	return nil
+}
+func (tbl *pgDNSSecTable) toGeneric() []DNSSecKey {
+	keys := make([]DNSSecKey, len(tbl.Records))
+
+	for i, record := range tbl.Records {
+		keys[i] = DNSSecKey{
+			CDN:                    record.CDN,
+			DNSSECKeysTrafficVault: record.Key,
+		}
+	}
+
+	return keys
+}
+func (tbl *pgDNSSecTable) fromGeneric(keys []DNSSecKey) {
+	tbl.Records = make([]pgDNSSecRecord, len(keys))
+
+	for i, key := range keys {
+		tbl.Records[i] = pgDNSSecRecord{
+			Key: key.DNSSECKeysTrafficVault,
+			CDN: key.CDN,
+			pgCommonRecord: pgCommonRecord{
+				DataEncrypted: nil,
+			},
+		}
+	}
+}
+func (tbl *pgDNSSecTable) validate() []string {
+	for i, record := range tbl.Records {
+		if record.DataEncrypted == nil && len(record.Key) > 0 {
+			return []string{fmt.Sprintf("DNSSEC Key %v: DataEncrypted is blank!", i)}
+		}
+	}
+	return nil
+}
+func (tbl *pgDNSSecTable) insertKeys(db *sql.DB) error {
+	queryBase := "INSERT INTO dnssec (cdn, data) VALUES "
+	stride := 2
+	queryArgs := make([]interface{}, len(tbl.Records)*stride)
+	for i, record := range tbl.Records {
+		j := i * stride
+		queryArgs[j] = record.CDN
+		queryArgs[j+1] = record.DataEncrypted
+	}
+	return insertIntoTable(db, queryBase, stride, queryArgs)
+}
+
+type pgSSLKeyRecord struct {
+	Keys tc.DeliveryServiceSSLKeys
+	pgCommonRecord
+
+	// These records are stored on the table but are duplicated
+	DeliveryService string
+	CDN             string
+}
+type pgSSLKeyTable struct {
+	Records []pgSSLKeyRecord
+}
+
+func (tbl *pgSSLKeyTable) insertKeys(db *sql.DB) error {
+	queryBase := "INSERT INTO sslkey (deliveryservice, data, cdn, version) VALUES "
+	duplicateKeys := 2
+	stride := 4
+	queryArgs := make([]interface{}, len(tbl.Records)*stride*duplicateKeys)
+	for i, record := range tbl.Records {
+		j := i * duplicateKeys * stride
+
+		queryArgs[j] = record.DeliveryService
+		queryArgs[j+1] = record.DataEncrypted
+		queryArgs[j+2] = record.CDN
+		queryArgs[j+3] = record.Keys.Version.String()
+
+		queryArgs[j+4] = record.DeliveryService
+		queryArgs[j+5] = record.DataEncrypted
+		queryArgs[j+6] = record.CDN
+		queryArgs[j+7] = "latest"
+	}
+	return insertIntoTable(db, queryBase, 4, queryArgs)
+}
+func (tbl *pgSSLKeyTable) gatherKeys(db *sql.DB) error {
+	sz, err := getSize(db, "sslkey WHERE version='latest'")
+	if err != nil {
+		return fmt.Errorf("PGSSLKey gatherKeys unable to determine size of sslkey table: %w", err)
+	}
+	tbl.Records = make([]pgSSLKeyRecord, sz)
+
+	rows, err := db.Query("SELECT data, deliveryservice, cdn from sslkey WHERE version = 'latest'")
+	if err != nil {
+		return fmt.Errorf("PGSSLKey gatherKeys unable to query: %w", err)
+	}
+	defer rows.Close()
+	i := 0
+	for rows.Next() {
+		if i > len(tbl.Records)-1 {
+			return errors.New("PGSSLKey gatherKeys: got more results than expected")
+		}
+		err := rows.Scan(&tbl.Records[i].DataEncrypted, &tbl.Records[i].DeliveryService, &tbl.Records[i].CDN)
+		if err != nil {
+			return fmt.Errorf("PGSSLKey gatherKeys unable to scan row: %w", err)
+		}
+		i += 1
+	}
+	return nil
+}
+func (tbl *pgSSLKeyTable) decrypt(aesKey []byte) error {
+	for i, dns := range tbl.Records {
+		err := decryptInto(aesKey, dns.DataEncrypted, &tbl.Records[i].Keys)
+		if err != nil {
+			return fmt.Errorf("unable to decrypt into keys: %w", err)
+		}
+	}
+	return nil
+}
+func (tbl *pgSSLKeyTable) encrypt(aesKey []byte) error {
+	for i, dns := range tbl.Records {
+		data, err := json.Marshal(dns.Keys)
+		if err != nil {
+			return fmt.Errorf("encrypt issue marshalling keys: %w", err)
+		}
+		dat, err := encrypt(data, aesKey)
+		if err != nil {
+			return fmt.Errorf("encrypt error: %w", err)
+		}
+		tbl.Records[i].DataEncrypted = dat
+	}
+	return nil
+}
+func (tbl *pgSSLKeyTable) toGeneric() []SSLKey {
+	keys := make([]SSLKey, len(tbl.Records))
+
+	for i, record := range tbl.Records {
+		keys[i] = SSLKey{
+			DeliveryServiceSSLKeys: record.Keys,
+		}
+	}
+	return keys
+}
+func (tbl *pgSSLKeyTable) fromGeneric(keys []SSLKey) {
+	tbl.Records = make([]pgSSLKeyRecord, len(keys))
+
+	for i, key := range keys {
+		tbl.Records[i] = pgSSLKeyRecord{
+			Keys: key.DeliveryServiceSSLKeys,
+			pgCommonRecord: pgCommonRecord{
+				DataEncrypted: nil,
+			},
+			DeliveryService: key.DeliveryService,
+			CDN:             key.CDN,
+		}
+	}
+}
+func (tbl *pgSSLKeyTable) validate() []string {
+	defaultKey := tc.DeliveryServiceSSLKeys{}
+	var errors []string
+	fmtStr := "SSL Key %v: %v"
+	for i, record := range tbl.Records {
+		if record.Keys == defaultKey {
+			errors = append(errors, fmt.Sprintf(fmtStr, i, "DS SSL Keys are default!"))
+		} else if record.Keys.Key == "" {
+			errors = append(errors, fmt.Sprintf(fmtStr, i, "Key is blank!"))
+		} else if record.Keys.CDN == "" {
+			errors = append(errors, fmt.Sprintf(fmtStr, i, "CDN is blank!"))
+		} else if record.Keys.DeliveryService == "" {
+			errors = append(errors, fmt.Sprintf(fmtStr, i, "DS is blank!"))
+		} else if record.DataEncrypted == nil {
+			errors = append(errors, fmt.Sprintf(fmtStr, i, "DataEncrypted is blank!"))
+		} else if record.Keys.Version.String() == "" {
+			errors = append(errors, fmt.Sprintf(fmtStr, i, "Version is blank!"))
+		}
+	}
+	return errors
+}
+
+type pgURLSigKeyRecord struct {
+	Keys            tc.URLSigKeys
+	DeliveryService string
+	pgCommonRecord
+}
+type pgURLSigKeyTable struct {
+	Records []pgURLSigKeyRecord
+}
+
+func (tbl *pgURLSigKeyTable) insertKeys(db *sql.DB) error {
+	queryBase := "INSERT INTO url_sig_key (deliveryservice, data) VALUES "
+	stride := 2
+	queryArgs := make([]interface{}, len(tbl.Records)*stride)
+	for i, record := range tbl.Records {
+		j := i * stride
+		queryArgs[j] = record.DeliveryService
+		queryArgs[j+1] = record.DataEncrypted
+	}
+	return insertIntoTable(db, queryBase, stride, queryArgs)
+}
+func (tbl *pgURLSigKeyTable) gatherKeys(db *sql.DB) error {
+	sz, err := getSize(db, "url_sig_key")
+	if err != nil {
+		log.Println("PGURLSigKey gatherKeys: unable to determine url_sig_key table size")
+	}
+	tbl.Records = make([]pgURLSigKeyRecord, sz)
+
+	rows, err := db.Query("SELECT deliveryservice, data from url_sig_key")
+	if err != nil {
+		return fmt.Errorf("PGURLSigKey gatherKeys error creating query: %w", err)
+	}
+	defer rows.Close()
+	i := 0
+	for rows.Next() {
+		if i > len(tbl.Records)-1 {
+			return fmt.Errorf("PGURLSigKey gatherKeys: got more results than expected %v", len(tbl.Records))
+		}
+		err := rows.Scan(&tbl.Records[i].DeliveryService, &tbl.Records[i].DataEncrypted)
+		if err != nil {
+			return fmt.Errorf("PGURLSigKey gatherKeys: unable to scan row: %w", err)
+		}
+		i += 1
+	}
+	return nil
+}
+func (tbl *pgURLSigKeyTable) decrypt(aesKey []byte) error {
+	for i, sig := range tbl.Records {
+		err := decryptInto(aesKey, sig.DataEncrypted, &tbl.Records[i].Keys)
+		if err != nil {
+			return fmt.Errorf("unable to decrypt into keys: %w", err)
+		}
+	}
+	return nil
+}
+func (tbl *pgURLSigKeyTable) encrypt(aesKey []byte) error {
+	for i, sig := range tbl.Records {
+		data, err := json.Marshal(&sig.Keys)
+		if err != nil {
+			return fmt.Errorf("encrypt issue marshalling keys: %w", err)
+		}
+
+		dat, err := encrypt(data, aesKey)
+		if err != nil {
+			return fmt.Errorf("encrypt error: %w", err)
+		}
+		tbl.Records[i].DataEncrypted = dat
+	}
+	return nil
+}
+func (tbl *pgURLSigKeyTable) toGeneric() []URLSigKey {
+	keys := make([]URLSigKey, len(tbl.Records))
+
+	for i, record := range tbl.Records {
+		keys[i] = URLSigKey{
+			DeliveryService: record.DeliveryService,
+			URLSigKeys:      record.Keys,
+		}
+	}
+	return keys
+}
+func (tbl *pgURLSigKeyTable) fromGeneric(keys []URLSigKey) {
+	tbl.Records = make([]pgURLSigKeyRecord, len(keys))
+
+	for i, key := range keys {
+		tbl.Records[i] = pgURLSigKeyRecord{
+			Keys:            key.URLSigKeys,
+			DeliveryService: key.DeliveryService,
+			pgCommonRecord: pgCommonRecord{
+				DataEncrypted: nil,
+			},
+		}
+	}
+}
+func (tbl *pgURLSigKeyTable) validate() []string {
+	for i, record := range tbl.Records {
+		if record.DataEncrypted == nil && len(record.Keys) > 0 {
+			return []string{fmt.Sprintf("URl Sig Key %v: DataEncrypted is blank!", i)}

Review comment:
       nit: `l -> L` and %d instead of %v

##########
File path: tools/traffic_vault_migrate/riak.go
##########
@@ -0,0 +1,746 @@
+package main
+
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+import (
+	"crypto/tls"
+	"encoding/json"
+	"errors"
+	"fmt"
+	"time"
+
+	"github.com/apache/trafficcontrol/lib/go-rfc"
+	"github.com/apache/trafficcontrol/lib/go-tc"
+
+	"github.com/basho/riak-go-client"
+)
+
+const (
+	BUCKET_SSL     = "ssl"
+	BUCKET_DNSSEC  = "dnssec"
+	BUCKET_URL_SIG = "url_sig_keys"
+	BUCKET_URI_SIG = "cdn_uri_sig_keys"
+
+	INDEX_SSL = "sslkeys"
+
+	SCHEMA_RIAK_KEY    = "_yz_rk"
+	SCHEMA_RIAK_BUCKET = "_yz_rb"
+)
+
+var (
+	SCHEMA_SSL_FIELDS = [...]string{SCHEMA_RIAK_KEY, SCHEMA_RIAK_BUCKET}
+)
+
+// RiakConfig  represents the configuration options available to the Riak backend
+type RiakConfig struct {
+	Host          string `json:"host"`
+	Port          string `json:"port"`
+	User          string `json:"user"`
+	Password      string `json:"password"`
+	Insecure      bool   `json:"insecure"`
+	TLSVersionRaw string `json:"tlsVersion"`
+	TLSVersion    uint16
+}
+
+// RiakBackend is the Riak implementation of TVBackend
+type RiakBackend struct {
+	sslKeys     riakSSLKeyTable
+	dnssecKeys  riakDNSSecKeyTable
+	uriSignKeys riakURISignKeyTable
+	urlSigKeys  riakURLSigKeyTable
+	cfg         RiakConfig
+	cluster     *riak.Cluster
+}
+
+// String returns a high level overview of the backend and its keys
+func (rb *RiakBackend) String() string {
+	data := fmt.Sprintf("Riak server %v@%v:%v\n", rb.cfg.User, rb.cfg.Host, rb.cfg.Port)
+	data += fmt.Sprintf("\tSSL Keys: %v\n", len(rb.sslKeys.Records))
+	data += fmt.Sprintf("\tDNSSec Keys: %v\n", len(rb.dnssecKeys.Records))
+	data += fmt.Sprintf("\tURI Keys: %v\n", len(rb.uriSignKeys.Records))
+	data += fmt.Sprintf("\tURL Keys: %v\n", len(rb.urlSigKeys.Records))
+	return data
+}
+
+// Name returns the name for this backend
+func (rb *RiakBackend) Name() string {
+	return "Riak"
+}
+
+// ReadConfig takes in a filename and will read it into the backends config
+func (rb *RiakBackend) ReadConfig(s string) error {
+	err := UnmarshalConfig(s, &rb.cfg)
+	if err != nil {
+		return err
+	}
+
+	if rb.cfg.TLSVersionRaw == "10" {
+		rb.cfg.TLSVersion = tls.VersionTLS10
+	} else if rb.cfg.TLSVersionRaw == "11" {
+		rb.cfg.TLSVersion = tls.VersionTLS11
+	} else if rb.cfg.TLSVersionRaw == "12" {
+		rb.cfg.TLSVersion = tls.VersionTLS12
+	} else if rb.cfg.TLSVersionRaw == "13" {
+		rb.cfg.TLSVersion = tls.VersionTLS13
+	}
+	return nil
+}
+
+// Insert takes the current keys and inserts them into the backend DB
+func (rb *RiakBackend) Insert() error {
+	if err := rb.sslKeys.insertKeys(rb.cluster); err != nil {
+		return err
+	}
+	if err := rb.dnssecKeys.insertKeys(rb.cluster); err != nil {
+		return err
+	}
+	if err := rb.urlSigKeys.insertKeys(rb.cluster); err != nil {
+		return err
+	}
+	if err := rb.uriSignKeys.insertKeys(rb.cluster); err != nil {
+		return err
+	}
+	return nil
+}
+
+// ValidateKey validates that the keys are valid (in most cases, certain fields are not null)
+func (rb *RiakBackend) ValidateKey() []string {
+	errors := []string{}
+	if errs := rb.sslKeys.validate(); errs != nil {
+		errors = append(errors, errs...)
+	}
+	if errs := rb.dnssecKeys.validate(); errs != nil {
+		errors = append(errors, errs...)
+	}
+	if errs := rb.uriSignKeys.validate(); errs != nil {
+		errors = append(errors, errs...)
+	}
+	if errs := rb.urlSigKeys.validate(); errs != nil {
+		errors = append(errors, errs...)
+	}
+
+	return errors
+}
+
+// SetSSLKeys takes in keys and converts & encrypts the data into the backends internal format
+func (rb *RiakBackend) SetSSLKeys(keys []SSLKey) error {
+	rb.sslKeys.fromGeneric(keys)
+	return nil
+}
+
+// SetDNSSecKeys takes in keys and converts & encrypts the data into the backends internal format
+func (rb *RiakBackend) SetDNSSecKeys(keys []DNSSecKey) error {
+	rb.dnssecKeys.fromGeneric(keys)
+	return nil
+}
+
+// SetURISignKeys takes in keys and converts & encrypts the data into the backends internal format
+func (rb *RiakBackend) SetURISignKeys(keys []URISignKey) error {
+	rb.uriSignKeys.fromGeneric(keys)
+	return nil
+}
+
+// SetURLSigKeys takes in keys and converts & encrypts the data into the backends internal format
+func (rb *RiakBackend) SetURLSigKeys(keys []URLSigKey) error {
+	rb.urlSigKeys.fromGeneric(keys)
+	return nil
+}
+
+// Start initiates the connection to the backend DB
+func (rb *RiakBackend) Start() error {
+	tlsConfig := &tls.Config{
+		InsecureSkipVerify: !rb.cfg.Insecure,
+		MaxVersion:         rb.cfg.TLSVersion,
+	}
+	auth := &riak.AuthOptions{
+		User:      rb.cfg.User,
+		Password:  rb.cfg.Password,
+		TlsConfig: tlsConfig,
+	}
+
+	cluster, err := getRiakCluster(rb.cfg, auth)
+	if err != nil {
+		return err
+	}
+	if err := cluster.Start(); err != nil {
+		return fmt.Errorf("unable to start riak cluster: %w", err)
+	}
+
+	rb.cluster = cluster
+	rb.sslKeys = riakSSLKeyTable{}
+	rb.dnssecKeys = riakDNSSecKeyTable{}
+	rb.urlSigKeys = riakURLSigKeyTable{}
+	rb.uriSignKeys = riakURISignKeyTable{}
+	return nil
+}
+
+// Close terminates the connection to the backend DB
+func (rb *RiakBackend) Close() error {
+	if err := rb.cluster.Stop(); err != nil {
+		return err
+	}
+	return nil
+}
+
+// Ping checks the connection to the backend DB
+func (rb *RiakBackend) Ping() error {
+	return ping(rb.cluster)
+}
+
+// GetSSLKeys converts the backends internal key representation into the common representation (SSLKey)
+func (rb *RiakBackend) GetSSLKeys() ([]SSLKey, error) {
+	return rb.sslKeys.toGeneric(), nil
+}
+
+// GetDNSSecKeys converts the backends internal key representation into the common representation (DNSSecKey)
+func (rb *RiakBackend) GetDNSSecKeys() ([]DNSSecKey, error) {
+	return rb.dnssecKeys.toGeneric(), nil
+}
+
+// GetURISignKeys converts the pg internal key representation into the common representation (URISignKey)
+func (rb *RiakBackend) GetURISignKeys() ([]URISignKey, error) {
+	return rb.uriSignKeys.toGeneric(), nil
+}
+
+// GetURLSigKeys converts the backends internal key representation into the common representation (URLSigKey)
+func (rb *RiakBackend) GetURLSigKeys() ([]URLSigKey, error) {
+	return rb.urlSigKeys.toGeneric(), nil
+}
+
+// Fetch gets all of the keys from the backend DB
+func (rb *RiakBackend) Fetch() error {
+	if err := rb.sslKeys.gatherKeys(rb.cluster); err != nil {
+		return err
+	}
+	if err := rb.dnssecKeys.gatherKeys(rb.cluster); err != nil {
+		return err
+	}
+	if err := rb.urlSigKeys.gatherKeys(rb.cluster); err != nil {
+		return err
+	}
+	if err := rb.uriSignKeys.gatherKeys(rb.cluster); err != nil {
+		return err
+	}
+
+	return nil
+}
+
+type riakSSLKeyRecord struct {
+	tc.DeliveryServiceSSLKeys
+}
+type riakSSLKeyTable struct {
+	Records []riakSSLKeyRecord
+}
+
+func (tbl *riakSSLKeyTable) gatherKeys(cluster *riak.Cluster) error {
+	searchDocs, err := search(cluster, INDEX_SSL, "cdn:*", fmt.Sprintf("%v:*latest", SCHEMA_RIAK_KEY), 1000, SCHEMA_SSL_FIELDS[:])
+	if err != nil {
+		return fmt.Errorf("RiakSSLKey gatherKeys: %w", err)
+	}
+
+	tbl.Records = make([]riakSSLKeyRecord, len(searchDocs))
+	for i, doc := range searchDocs {
+		objs, err := getObject(cluster, doc.Bucket, doc.Key)
+		if err != nil {
+			return err
+		}
+		if len(objs) < 1 {
+			return fmt.Errorf("RiakSSLKey gatherKeys unable to find any objects with key %v and bucket %v, but search results were returned", doc.Key, doc.Bucket)

Review comment:
       nit: type-specific formatting instead of %v

##########
File path: tools/traffic_vault_migrate/postgres.go
##########
@@ -0,0 +1,743 @@
+package main
+
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+import (
+	"database/sql"
+	"encoding/base64"
+	"encoding/json"
+	"errors"
+	"fmt"
+	"log"
+	"strconv"
+	"strings"
+
+	"github.com/apache/trafficcontrol/lib/go-tc"
+	util "github.com/apache/trafficcontrol/lib/go-util"
+
+	_ "github.com/lib/pq"
+)
+
+// PGConfig represents the configuration options available to the PG backend
+type PGConfig struct {
+	Host     string `json:"host"`
+	Port     string `json:"port"`
+	User     string `json:"user"`
+	Password string `json:"password"`
+	SSLMode  string `json:"sslmode"`
+	Database string `json:"database"`
+	Key      string `json:"aesKey"`
+	AESKey   []byte
+}
+
+// PGBackend is the Postgres implementation of TVBackend
+type PGBackend struct {
+	sslKey pgSSLKeyTable
+	dnssec pgDNSSecTable
+	uri    pgURISignKeyTable
+	url    pgURLSigKeyTable
+	cfg    PGConfig
+	db     *sql.DB
+}
+
+// String returns a high level overview of the backend and its keys
+func (pg *PGBackend) String() string {
+	data := fmt.Sprintf("PG server %v@%v:%v\n", pg.cfg.User, pg.cfg.Host, pg.cfg.Port)
+	data += fmt.Sprintf("\tSSL Keys: %v\n", len(pg.sslKey.Records))
+	data += fmt.Sprintf("\tDNSSec Keys: %v\n", len(pg.dnssec.Records))
+	data += fmt.Sprintf("\tURI Keys: %v\n", len(pg.uri.Records))
+	data += fmt.Sprintf("\tURL Keys: %v\n", len(pg.url.Records))
+	return data
+}
+
+// Name returns the name for this backend
+func (pg *PGBackend) Name() string {
+	return "PG"
+}
+
+// ReadConfig takes in a filename and will read it into the backends config
+func (pg *PGBackend) ReadConfig(s string) error {
+	err := UnmarshalConfig(s, &pg.cfg)
+	if err != nil {
+		return err
+	}
+
+	pg.cfg.AESKey, err = base64.StdEncoding.DecodeString(pg.cfg.Key)
+	if err != nil {
+		return fmt.Errorf("unable to decode PG AESKey: %w", err)
+	}
+	return nil
+}
+
+// Insert takes the current keys and inserts them into the backend DB
+func (pg *PGBackend) Insert() error {
+	if err := pg.sslKey.insertKeys(pg.db); err != nil {
+		return err
+	}
+	if err := pg.dnssec.insertKeys(pg.db); err != nil {
+		return err
+	}
+	if err := pg.url.insertKeys(pg.db); err != nil {
+		return err
+	}
+	if err := pg.uri.insertKeys(pg.db); err != nil {
+		return err
+	}
+	return nil
+}
+
+// Start initiates the connection to the backend DB
+func (pg *PGBackend) Start() error {
+	sqlStr := fmt.Sprintf("postgres://%s:%s@%s:%s/%s?sslmode=%s", pg.cfg.User, pg.cfg.Password, pg.cfg.Host, pg.cfg.Port, pg.cfg.Database, pg.cfg.SSLMode)
+	db, err := sql.Open("postgres", sqlStr)
+	if err != nil {
+		return fmt.Errorf("unable to start PG client: %w", err)
+	}
+
+	pg.db = db
+	pg.sslKey = pgSSLKeyTable{}
+	pg.dnssec = pgDNSSecTable{}
+	pg.url = pgURLSigKeyTable{}
+	pg.uri = pgURISignKeyTable{}
+
+	return nil
+}
+
+// ValidateKey validates that the keys are valid (in most cases, certain fields are not null)
+func (pg *PGBackend) ValidateKey() []string {
+	var errors []string
+	if errs := pg.sslKey.validate(); errs != nil {
+		errors = append(errors, errs...)
+	}
+	if errs := pg.dnssec.validate(); errs != nil {
+		errors = append(errors, errs...)
+	}
+	if errs := pg.uri.validate(); errs != nil {
+		errors = append(errors, errs...)
+	}
+	if errs := pg.url.validate(); errs != nil {
+		errors = append(errors, errs...)
+	}
+	return errors
+}
+
+// Close terminates the connection to the backend DB
+func (pg *PGBackend) Close() error {
+	return pg.db.Close()
+}
+
+// Ping checks the connection to the backend DB
+func (pg *PGBackend) Ping() error {
+	return pg.db.Ping()
+}
+
+// Fetch gets all of the keys from the backend DB
+func (pg *PGBackend) Fetch() error {
+	if err := pg.sslKey.gatherKeys(pg.db); err != nil {
+		return err
+	}
+
+	if err := pg.dnssec.gatherKeys(pg.db); err != nil {
+		return err
+	}
+
+	if err := pg.url.gatherKeys(pg.db); err != nil {
+		return err
+	}
+
+	if err := pg.uri.gatherKeys(pg.db); err != nil {
+		return err
+	}
+
+	return nil
+}
+
+// GetSSLKeys converts the backends internal key representation into the common representation (SSLKey)
+func (pg *PGBackend) GetSSLKeys() ([]SSLKey, error) {
+	if err := pg.sslKey.decrypt(pg.cfg.AESKey); err != nil {
+		return nil, err
+	}
+	return pg.sslKey.toGeneric(), nil
+}
+
+// SetSSLKeys takes in keys and converts & encrypts the data into the backends internal format
+func (pg *PGBackend) SetSSLKeys(keys []SSLKey) error {
+	pg.sslKey.fromGeneric(keys)
+	return pg.sslKey.encrypt(pg.cfg.AESKey)
+}
+
+// GetDNSSecKeys converts the backends internal key representation into the common representation (DNSSecKey)
+func (pg *PGBackend) GetDNSSecKeys() ([]DNSSecKey, error) {
+	if err := pg.dnssec.decrypt(pg.cfg.AESKey); err != nil {
+		return nil, err
+	}
+	return pg.dnssec.toGeneric(), nil
+}
+
+// SetDNSSecKeys takes in keys and converts & encrypts the data into the backends internal format
+func (pg *PGBackend) SetDNSSecKeys(keys []DNSSecKey) error {
+	pg.dnssec.fromGeneric(keys)
+	return pg.dnssec.encrypt(pg.cfg.AESKey)
+}
+
+// GetURISignKeys converts the pg internal key representation into the common representation (URISignKey)
+func (pg *PGBackend) GetURISignKeys() ([]URISignKey, error) {
+	if err := pg.uri.decrypt(pg.cfg.AESKey); err != nil {
+		return nil, err
+	}
+	return pg.uri.toGeneric(), nil
+}
+
+// SetURISignKeys takes in keys and converts & encrypts the data into the backends internal format
+func (pg *PGBackend) SetURISignKeys(keys []URISignKey) error {
+	pg.uri.fromGeneric(keys)
+	return pg.uri.encrypt(pg.cfg.AESKey)
+}
+
+// GetURLSigKeys converts the backends internal key representation into the common representation (URLSigKey)
+func (pg *PGBackend) GetURLSigKeys() ([]URLSigKey, error) {
+	if err := pg.url.decrypt(pg.cfg.AESKey); err != nil {
+		return nil, err
+	}
+	return pg.url.toGeneric(), nil
+}
+
+// SetURLSigKeys takes in keys and converts & encrypts the data into the backends internal format
+func (pg *PGBackend) SetURLSigKeys(keys []URLSigKey) error {
+	pg.url.fromGeneric(keys)
+	return pg.url.encrypt(pg.cfg.AESKey)
+}
+
+type pgCommonRecord struct {
+	DataEncrypted []byte
+}
+
+type pgDNSSecRecord struct {
+	Key tc.DNSSECKeysTrafficVault
+	CDN string
+	pgCommonRecord
+}
+type pgDNSSecTable struct {
+	Records []pgDNSSecRecord
+}
+
+func (tbl *pgDNSSecTable) gatherKeys(db *sql.DB) error {
+	sz, err := getSize(db, "dnssec")
+	if err != nil {
+		log.Println("PGDNSSec gatherKeys: unable to determine size of dnssec table")
+	}
+	tbl.Records = make([]pgDNSSecRecord, sz)
+
+	rows, err := db.Query("SELECT cdn, data from dnssec")
+	if err != nil {
+		return fmt.Errorf("PGDNSSec gatherKeys: unable to query: %w", err)
+	}
+	defer rows.Close()
+	i := 0
+	for rows.Next() {
+		if i > len(tbl.Records)-1 {
+			return fmt.Errorf("PGDNSSec gatherKeys got more results than expected %v", len(tbl.Records))
+		}
+		err := rows.Scan(&tbl.Records[i].CDN, &tbl.Records[i].DataEncrypted)
+		if err != nil {
+			return fmt.Errorf("PGDNSSec gatherKeys unable to scan row: %w", err)
+		}
+		i += 1
+	}
+	return nil
+}
+func (tbl *pgDNSSecTable) decrypt(aesKey []byte) error {
+	for i, _ := range tbl.Records {
+		err := decryptInto(aesKey, tbl.Records[i].DataEncrypted, &tbl.Records[i].Key)
+		if err != nil {
+			return fmt.Errorf("unable to decrypt into keys: %w", err)
+		}
+	}
+	return nil
+}
+func (tbl *pgDNSSecTable) encrypt(aesKey []byte) error {
+	for i, dns := range tbl.Records {
+		data, err := json.Marshal(&dns.Key)
+		if err != nil {
+			return fmt.Errorf("encrypt issue marshalling keys: %w", err)
+		}
+		dat, err := encrypt(data, aesKey)
+		if err != nil {
+			return fmt.Errorf("encrypt error: %w", err)
+		}
+		tbl.Records[i].DataEncrypted = dat
+	}
+	return nil
+}
+func (tbl *pgDNSSecTable) toGeneric() []DNSSecKey {
+	keys := make([]DNSSecKey, len(tbl.Records))
+
+	for i, record := range tbl.Records {
+		keys[i] = DNSSecKey{
+			CDN:                    record.CDN,
+			DNSSECKeysTrafficVault: record.Key,
+		}
+	}
+
+	return keys
+}
+func (tbl *pgDNSSecTable) fromGeneric(keys []DNSSecKey) {
+	tbl.Records = make([]pgDNSSecRecord, len(keys))
+
+	for i, key := range keys {
+		tbl.Records[i] = pgDNSSecRecord{
+			Key: key.DNSSECKeysTrafficVault,
+			CDN: key.CDN,
+			pgCommonRecord: pgCommonRecord{
+				DataEncrypted: nil,
+			},
+		}
+	}
+}
+func (tbl *pgDNSSecTable) validate() []string {
+	for i, record := range tbl.Records {
+		if record.DataEncrypted == nil && len(record.Key) > 0 {
+			return []string{fmt.Sprintf("DNSSEC Key %v: DataEncrypted is blank!", i)}
+		}
+	}
+	return nil
+}
+func (tbl *pgDNSSecTable) insertKeys(db *sql.DB) error {
+	queryBase := "INSERT INTO dnssec (cdn, data) VALUES "
+	stride := 2
+	queryArgs := make([]interface{}, len(tbl.Records)*stride)
+	for i, record := range tbl.Records {
+		j := i * stride
+		queryArgs[j] = record.CDN
+		queryArgs[j+1] = record.DataEncrypted
+	}
+	return insertIntoTable(db, queryBase, stride, queryArgs)
+}
+
+type pgSSLKeyRecord struct {
+	Keys tc.DeliveryServiceSSLKeys
+	pgCommonRecord
+
+	// These records are stored on the table but are duplicated
+	DeliveryService string
+	CDN             string
+}
+type pgSSLKeyTable struct {
+	Records []pgSSLKeyRecord
+}
+
+func (tbl *pgSSLKeyTable) insertKeys(db *sql.DB) error {
+	queryBase := "INSERT INTO sslkey (deliveryservice, data, cdn, version) VALUES "
+	duplicateKeys := 2
+	stride := 4
+	queryArgs := make([]interface{}, len(tbl.Records)*stride*duplicateKeys)
+	for i, record := range tbl.Records {
+		j := i * duplicateKeys * stride
+
+		queryArgs[j] = record.DeliveryService
+		queryArgs[j+1] = record.DataEncrypted
+		queryArgs[j+2] = record.CDN
+		queryArgs[j+3] = record.Keys.Version.String()
+
+		queryArgs[j+4] = record.DeliveryService
+		queryArgs[j+5] = record.DataEncrypted
+		queryArgs[j+6] = record.CDN
+		queryArgs[j+7] = "latest"
+	}
+	return insertIntoTable(db, queryBase, 4, queryArgs)
+}
+func (tbl *pgSSLKeyTable) gatherKeys(db *sql.DB) error {
+	sz, err := getSize(db, "sslkey WHERE version='latest'")
+	if err != nil {
+		return fmt.Errorf("PGSSLKey gatherKeys unable to determine size of sslkey table: %w", err)
+	}
+	tbl.Records = make([]pgSSLKeyRecord, sz)
+
+	rows, err := db.Query("SELECT data, deliveryservice, cdn from sslkey WHERE version = 'latest'")
+	if err != nil {
+		return fmt.Errorf("PGSSLKey gatherKeys unable to query: %w", err)
+	}
+	defer rows.Close()
+	i := 0
+	for rows.Next() {
+		if i > len(tbl.Records)-1 {
+			return errors.New("PGSSLKey gatherKeys: got more results than expected")
+		}
+		err := rows.Scan(&tbl.Records[i].DataEncrypted, &tbl.Records[i].DeliveryService, &tbl.Records[i].CDN)
+		if err != nil {
+			return fmt.Errorf("PGSSLKey gatherKeys unable to scan row: %w", err)
+		}
+		i += 1
+	}
+	return nil
+}
+func (tbl *pgSSLKeyTable) decrypt(aesKey []byte) error {
+	for i, dns := range tbl.Records {
+		err := decryptInto(aesKey, dns.DataEncrypted, &tbl.Records[i].Keys)
+		if err != nil {
+			return fmt.Errorf("unable to decrypt into keys: %w", err)
+		}
+	}
+	return nil
+}
+func (tbl *pgSSLKeyTable) encrypt(aesKey []byte) error {
+	for i, dns := range tbl.Records {
+		data, err := json.Marshal(dns.Keys)
+		if err != nil {
+			return fmt.Errorf("encrypt issue marshalling keys: %w", err)
+		}
+		dat, err := encrypt(data, aesKey)
+		if err != nil {
+			return fmt.Errorf("encrypt error: %w", err)
+		}
+		tbl.Records[i].DataEncrypted = dat
+	}
+	return nil
+}
+func (tbl *pgSSLKeyTable) toGeneric() []SSLKey {
+	keys := make([]SSLKey, len(tbl.Records))
+
+	for i, record := range tbl.Records {
+		keys[i] = SSLKey{
+			DeliveryServiceSSLKeys: record.Keys,
+		}
+	}
+	return keys
+}
+func (tbl *pgSSLKeyTable) fromGeneric(keys []SSLKey) {
+	tbl.Records = make([]pgSSLKeyRecord, len(keys))
+
+	for i, key := range keys {
+		tbl.Records[i] = pgSSLKeyRecord{
+			Keys: key.DeliveryServiceSSLKeys,
+			pgCommonRecord: pgCommonRecord{
+				DataEncrypted: nil,
+			},
+			DeliveryService: key.DeliveryService,
+			CDN:             key.CDN,
+		}
+	}
+}
+func (tbl *pgSSLKeyTable) validate() []string {
+	defaultKey := tc.DeliveryServiceSSLKeys{}
+	var errors []string
+	fmtStr := "SSL Key %v: %v"
+	for i, record := range tbl.Records {
+		if record.Keys == defaultKey {
+			errors = append(errors, fmt.Sprintf(fmtStr, i, "DS SSL Keys are default!"))
+		} else if record.Keys.Key == "" {
+			errors = append(errors, fmt.Sprintf(fmtStr, i, "Key is blank!"))
+		} else if record.Keys.CDN == "" {
+			errors = append(errors, fmt.Sprintf(fmtStr, i, "CDN is blank!"))
+		} else if record.Keys.DeliveryService == "" {
+			errors = append(errors, fmt.Sprintf(fmtStr, i, "DS is blank!"))
+		} else if record.DataEncrypted == nil {
+			errors = append(errors, fmt.Sprintf(fmtStr, i, "DataEncrypted is blank!"))
+		} else if record.Keys.Version.String() == "" {
+			errors = append(errors, fmt.Sprintf(fmtStr, i, "Version is blank!"))
+		}
+	}
+	return errors
+}
+
+type pgURLSigKeyRecord struct {
+	Keys            tc.URLSigKeys
+	DeliveryService string
+	pgCommonRecord
+}
+type pgURLSigKeyTable struct {
+	Records []pgURLSigKeyRecord
+}
+
+func (tbl *pgURLSigKeyTable) insertKeys(db *sql.DB) error {
+	queryBase := "INSERT INTO url_sig_key (deliveryservice, data) VALUES "
+	stride := 2
+	queryArgs := make([]interface{}, len(tbl.Records)*stride)
+	for i, record := range tbl.Records {
+		j := i * stride
+		queryArgs[j] = record.DeliveryService
+		queryArgs[j+1] = record.DataEncrypted
+	}
+	return insertIntoTable(db, queryBase, stride, queryArgs)
+}
+func (tbl *pgURLSigKeyTable) gatherKeys(db *sql.DB) error {
+	sz, err := getSize(db, "url_sig_key")
+	if err != nil {
+		log.Println("PGURLSigKey gatherKeys: unable to determine url_sig_key table size")
+	}
+	tbl.Records = make([]pgURLSigKeyRecord, sz)
+
+	rows, err := db.Query("SELECT deliveryservice, data from url_sig_key")
+	if err != nil {
+		return fmt.Errorf("PGURLSigKey gatherKeys error creating query: %w", err)
+	}
+	defer rows.Close()
+	i := 0
+	for rows.Next() {
+		if i > len(tbl.Records)-1 {
+			return fmt.Errorf("PGURLSigKey gatherKeys: got more results than expected %v", len(tbl.Records))
+		}
+		err := rows.Scan(&tbl.Records[i].DeliveryService, &tbl.Records[i].DataEncrypted)
+		if err != nil {
+			return fmt.Errorf("PGURLSigKey gatherKeys: unable to scan row: %w", err)
+		}
+		i += 1
+	}
+	return nil
+}
+func (tbl *pgURLSigKeyTable) decrypt(aesKey []byte) error {
+	for i, sig := range tbl.Records {
+		err := decryptInto(aesKey, sig.DataEncrypted, &tbl.Records[i].Keys)
+		if err != nil {
+			return fmt.Errorf("unable to decrypt into keys: %w", err)
+		}
+	}
+	return nil
+}
+func (tbl *pgURLSigKeyTable) encrypt(aesKey []byte) error {
+	for i, sig := range tbl.Records {
+		data, err := json.Marshal(&sig.Keys)
+		if err != nil {
+			return fmt.Errorf("encrypt issue marshalling keys: %w", err)
+		}
+
+		dat, err := encrypt(data, aesKey)
+		if err != nil {
+			return fmt.Errorf("encrypt error: %w", err)
+		}
+		tbl.Records[i].DataEncrypted = dat
+	}
+	return nil
+}
+func (tbl *pgURLSigKeyTable) toGeneric() []URLSigKey {
+	keys := make([]URLSigKey, len(tbl.Records))
+
+	for i, record := range tbl.Records {
+		keys[i] = URLSigKey{
+			DeliveryService: record.DeliveryService,
+			URLSigKeys:      record.Keys,
+		}
+	}
+	return keys
+}
+func (tbl *pgURLSigKeyTable) fromGeneric(keys []URLSigKey) {
+	tbl.Records = make([]pgURLSigKeyRecord, len(keys))
+
+	for i, key := range keys {
+		tbl.Records[i] = pgURLSigKeyRecord{
+			Keys:            key.URLSigKeys,
+			DeliveryService: key.DeliveryService,
+			pgCommonRecord: pgCommonRecord{
+				DataEncrypted: nil,
+			},
+		}
+	}
+}
+func (tbl *pgURLSigKeyTable) validate() []string {
+	for i, record := range tbl.Records {
+		if record.DataEncrypted == nil && len(record.Keys) > 0 {
+			return []string{fmt.Sprintf("URl Sig Key %v: DataEncrypted is blank!", i)}
+		}
+	}
+	return nil
+}
+
+type pgURISignKeyRecord struct {
+	Keys            map[string]tc.URISignerKeyset
+	DeliveryService string
+	pgCommonRecord
+}
+type pgURISignKeyTable struct {
+	Records []pgURISignKeyRecord
+}
+
+func (tbl *pgURISignKeyTable) insertKeys(db *sql.DB) error {
+	queryBase := "INSERT INTO uri_signing_key (deliveryservice, data) VALUES "
+	stride := 2
+	queryArgs := make([]interface{}, len(tbl.Records)*stride)
+	for i, record := range tbl.Records {
+		j := i * stride
+		queryArgs[j] = record.DeliveryService
+		queryArgs[j+1] = record.DataEncrypted
+	}
+	return insertIntoTable(db, queryBase, stride, queryArgs)
+}
+func (tbl *pgURISignKeyTable) gatherKeys(db *sql.DB) error {
+	sz, err := getSize(db, "uri_signing_key")
+	if err != nil {
+		log.Println("PGURISignKey gatherKeys: unable to determine size of uri_signing_key table")
+	}
+	tbl.Records = make([]pgURISignKeyRecord, sz)
+
+	rows, err := db.Query("SELECT deliveryservice, data from uri_signing_key")
+	if err != nil {
+		return fmt.Errorf("PGURISignKey gatherKeys error while query: %w", err)
+	}
+	defer rows.Close()
+	i := 0
+	for rows.Next() {
+		if i > len(tbl.Records)-1 {
+			return fmt.Errorf("PGURISignKey gatherKeys: got more results than expected %v", len(tbl.Records))
+		}
+		err := rows.Scan(&tbl.Records[i].DeliveryService, &tbl.Records[i].DataEncrypted)
+		if err != nil {
+			return fmt.Errorf("PGURISignKey gatherKeys: unable to scan row: %w", err)
+		}
+		i += 1
+	}
+	return nil
+}
+func (tbl *pgURISignKeyTable) decrypt(aesKey []byte) error {
+	for i, sign := range tbl.Records {
+		err := decryptInto(aesKey, sign.DataEncrypted, &tbl.Records[i].Keys)
+		if err != nil {
+			return fmt.Errorf("unable to decrypt into keys: %w", err)
+		}
+	}
+	return nil
+}
+func (tbl *pgURISignKeyTable) encrypt(aesKey []byte) error {
+	for i, sign := range tbl.Records {
+		data, err := json.Marshal(sign.Keys)
+		if err != nil {
+			return fmt.Errorf("encrypt issue marshalling keys: %w", err)
+		}
+
+		dat, err := encrypt(data, aesKey)
+		if err != nil {
+			return fmt.Errorf("encrypt error: %w", err)
+		}
+		tbl.Records[i].DataEncrypted = dat
+	}
+	return nil
+}
+func (tbl *pgURISignKeyTable) toGeneric() []URISignKey {
+	keys := make([]URISignKey, len(tbl.Records))
+
+	for i, record := range tbl.Records {
+		keys[i] = URISignKey{
+			DeliveryService: record.DeliveryService,
+			Keys:            record.Keys,
+		}
+	}
+
+	return keys
+}
+func (tbl *pgURISignKeyTable) fromGeneric(keys []URISignKey) {
+	tbl.Records = make([]pgURISignKeyRecord, len(keys))
+
+	for i, key := range keys {
+		tbl.Records[i] = pgURISignKeyRecord{
+			Keys:            key.Keys,
+			DeliveryService: key.DeliveryService,
+			pgCommonRecord: pgCommonRecord{
+				DataEncrypted: nil,
+			},
+		}
+	}
+}
+func (tbl *pgURISignKeyTable) validate() []string {
+	for i, record := range tbl.Records {
+		if record.DataEncrypted == nil && len(record.Keys) > 0 {
+			return []string{fmt.Sprintf("URI Sign Key %v: DataEncrypted is blank!", i)}
+		}
+	}
+	return nil
+}
+
+func getSize(db *sql.DB, table string) (int64, error) {
+	rows, err := db.Query("SELECT COUNT(*) FROM " + table)
+	if err != nil {
+		return 0, err
+	}
+	var numRows int64
+	if !rows.Next() {
+		return 0, errors.New("no results returned for: " + table)
+	}
+	err = rows.Scan(&numRows)

Review comment:
       nit: this can be combined with the line below like `if err := function(); err != nil {`

##########
File path: tools/traffic_vault_migrate/postgres.go
##########
@@ -0,0 +1,743 @@
+package main
+
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+import (
+	"database/sql"
+	"encoding/base64"
+	"encoding/json"
+	"errors"
+	"fmt"
+	"log"
+	"strconv"
+	"strings"
+
+	"github.com/apache/trafficcontrol/lib/go-tc"
+	util "github.com/apache/trafficcontrol/lib/go-util"
+
+	_ "github.com/lib/pq"
+)
+
+// PGConfig represents the configuration options available to the PG backend
+type PGConfig struct {
+	Host     string `json:"host"`
+	Port     string `json:"port"`
+	User     string `json:"user"`
+	Password string `json:"password"`
+	SSLMode  string `json:"sslmode"`
+	Database string `json:"database"`
+	Key      string `json:"aesKey"`
+	AESKey   []byte
+}
+
+// PGBackend is the Postgres implementation of TVBackend
+type PGBackend struct {
+	sslKey pgSSLKeyTable
+	dnssec pgDNSSecTable
+	uri    pgURISignKeyTable
+	url    pgURLSigKeyTable
+	cfg    PGConfig
+	db     *sql.DB
+}
+
+// String returns a high level overview of the backend and its keys
+func (pg *PGBackend) String() string {
+	data := fmt.Sprintf("PG server %v@%v:%v\n", pg.cfg.User, pg.cfg.Host, pg.cfg.Port)
+	data += fmt.Sprintf("\tSSL Keys: %v\n", len(pg.sslKey.Records))
+	data += fmt.Sprintf("\tDNSSec Keys: %v\n", len(pg.dnssec.Records))
+	data += fmt.Sprintf("\tURI Keys: %v\n", len(pg.uri.Records))
+	data += fmt.Sprintf("\tURL Keys: %v\n", len(pg.url.Records))
+	return data
+}
+
+// Name returns the name for this backend
+func (pg *PGBackend) Name() string {
+	return "PG"
+}
+
+// ReadConfig takes in a filename and will read it into the backends config
+func (pg *PGBackend) ReadConfig(s string) error {
+	err := UnmarshalConfig(s, &pg.cfg)
+	if err != nil {
+		return err
+	}
+
+	pg.cfg.AESKey, err = base64.StdEncoding.DecodeString(pg.cfg.Key)
+	if err != nil {
+		return fmt.Errorf("unable to decode PG AESKey: %w", err)
+	}
+	return nil
+}
+
+// Insert takes the current keys and inserts them into the backend DB
+func (pg *PGBackend) Insert() error {
+	if err := pg.sslKey.insertKeys(pg.db); err != nil {
+		return err
+	}
+	if err := pg.dnssec.insertKeys(pg.db); err != nil {
+		return err
+	}
+	if err := pg.url.insertKeys(pg.db); err != nil {
+		return err
+	}
+	if err := pg.uri.insertKeys(pg.db); err != nil {
+		return err
+	}
+	return nil
+}
+
+// Start initiates the connection to the backend DB
+func (pg *PGBackend) Start() error {
+	sqlStr := fmt.Sprintf("postgres://%s:%s@%s:%s/%s?sslmode=%s", pg.cfg.User, pg.cfg.Password, pg.cfg.Host, pg.cfg.Port, pg.cfg.Database, pg.cfg.SSLMode)
+	db, err := sql.Open("postgres", sqlStr)
+	if err != nil {
+		return fmt.Errorf("unable to start PG client: %w", err)
+	}
+
+	pg.db = db
+	pg.sslKey = pgSSLKeyTable{}
+	pg.dnssec = pgDNSSecTable{}
+	pg.url = pgURLSigKeyTable{}
+	pg.uri = pgURISignKeyTable{}
+
+	return nil
+}
+
+// ValidateKey validates that the keys are valid (in most cases, certain fields are not null)
+func (pg *PGBackend) ValidateKey() []string {
+	var errors []string
+	if errs := pg.sslKey.validate(); errs != nil {
+		errors = append(errors, errs...)
+	}
+	if errs := pg.dnssec.validate(); errs != nil {
+		errors = append(errors, errs...)
+	}
+	if errs := pg.uri.validate(); errs != nil {
+		errors = append(errors, errs...)
+	}
+	if errs := pg.url.validate(); errs != nil {
+		errors = append(errors, errs...)
+	}
+	return errors
+}
+
+// Close terminates the connection to the backend DB
+func (pg *PGBackend) Close() error {
+	return pg.db.Close()
+}
+
+// Ping checks the connection to the backend DB
+func (pg *PGBackend) Ping() error {
+	return pg.db.Ping()
+}
+
+// Fetch gets all of the keys from the backend DB
+func (pg *PGBackend) Fetch() error {
+	if err := pg.sslKey.gatherKeys(pg.db); err != nil {
+		return err
+	}
+
+	if err := pg.dnssec.gatherKeys(pg.db); err != nil {
+		return err
+	}
+
+	if err := pg.url.gatherKeys(pg.db); err != nil {
+		return err
+	}
+
+	if err := pg.uri.gatherKeys(pg.db); err != nil {
+		return err
+	}
+
+	return nil
+}
+
+// GetSSLKeys converts the backends internal key representation into the common representation (SSLKey)
+func (pg *PGBackend) GetSSLKeys() ([]SSLKey, error) {
+	if err := pg.sslKey.decrypt(pg.cfg.AESKey); err != nil {
+		return nil, err
+	}
+	return pg.sslKey.toGeneric(), nil
+}
+
+// SetSSLKeys takes in keys and converts & encrypts the data into the backends internal format
+func (pg *PGBackend) SetSSLKeys(keys []SSLKey) error {
+	pg.sslKey.fromGeneric(keys)
+	return pg.sslKey.encrypt(pg.cfg.AESKey)
+}
+
+// GetDNSSecKeys converts the backends internal key representation into the common representation (DNSSecKey)
+func (pg *PGBackend) GetDNSSecKeys() ([]DNSSecKey, error) {
+	if err := pg.dnssec.decrypt(pg.cfg.AESKey); err != nil {
+		return nil, err
+	}
+	return pg.dnssec.toGeneric(), nil
+}
+
+// SetDNSSecKeys takes in keys and converts & encrypts the data into the backends internal format
+func (pg *PGBackend) SetDNSSecKeys(keys []DNSSecKey) error {
+	pg.dnssec.fromGeneric(keys)
+	return pg.dnssec.encrypt(pg.cfg.AESKey)
+}
+
+// GetURISignKeys converts the pg internal key representation into the common representation (URISignKey)
+func (pg *PGBackend) GetURISignKeys() ([]URISignKey, error) {
+	if err := pg.uri.decrypt(pg.cfg.AESKey); err != nil {
+		return nil, err
+	}
+	return pg.uri.toGeneric(), nil
+}
+
+// SetURISignKeys takes in keys and converts & encrypts the data into the backends internal format
+func (pg *PGBackend) SetURISignKeys(keys []URISignKey) error {
+	pg.uri.fromGeneric(keys)
+	return pg.uri.encrypt(pg.cfg.AESKey)
+}
+
+// GetURLSigKeys converts the backends internal key representation into the common representation (URLSigKey)
+func (pg *PGBackend) GetURLSigKeys() ([]URLSigKey, error) {
+	if err := pg.url.decrypt(pg.cfg.AESKey); err != nil {
+		return nil, err
+	}
+	return pg.url.toGeneric(), nil
+}
+
+// SetURLSigKeys takes in keys and converts & encrypts the data into the backends internal format
+func (pg *PGBackend) SetURLSigKeys(keys []URLSigKey) error {
+	pg.url.fromGeneric(keys)
+	return pg.url.encrypt(pg.cfg.AESKey)
+}
+
+type pgCommonRecord struct {
+	DataEncrypted []byte
+}
+
+type pgDNSSecRecord struct {
+	Key tc.DNSSECKeysTrafficVault
+	CDN string
+	pgCommonRecord
+}
+type pgDNSSecTable struct {
+	Records []pgDNSSecRecord
+}
+
+func (tbl *pgDNSSecTable) gatherKeys(db *sql.DB) error {
+	sz, err := getSize(db, "dnssec")
+	if err != nil {
+		log.Println("PGDNSSec gatherKeys: unable to determine size of dnssec table")
+	}
+	tbl.Records = make([]pgDNSSecRecord, sz)
+
+	rows, err := db.Query("SELECT cdn, data from dnssec")
+	if err != nil {
+		return fmt.Errorf("PGDNSSec gatherKeys: unable to query: %w", err)
+	}
+	defer rows.Close()
+	i := 0
+	for rows.Next() {
+		if i > len(tbl.Records)-1 {
+			return fmt.Errorf("PGDNSSec gatherKeys got more results than expected %v", len(tbl.Records))
+		}
+		err := rows.Scan(&tbl.Records[i].CDN, &tbl.Records[i].DataEncrypted)
+		if err != nil {
+			return fmt.Errorf("PGDNSSec gatherKeys unable to scan row: %w", err)
+		}
+		i += 1
+	}
+	return nil
+}
+func (tbl *pgDNSSecTable) decrypt(aesKey []byte) error {
+	for i, _ := range tbl.Records {
+		err := decryptInto(aesKey, tbl.Records[i].DataEncrypted, &tbl.Records[i].Key)
+		if err != nil {
+			return fmt.Errorf("unable to decrypt into keys: %w", err)
+		}
+	}
+	return nil
+}
+func (tbl *pgDNSSecTable) encrypt(aesKey []byte) error {
+	for i, dns := range tbl.Records {
+		data, err := json.Marshal(&dns.Key)
+		if err != nil {
+			return fmt.Errorf("encrypt issue marshalling keys: %w", err)
+		}
+		dat, err := encrypt(data, aesKey)
+		if err != nil {
+			return fmt.Errorf("encrypt error: %w", err)
+		}
+		tbl.Records[i].DataEncrypted = dat
+	}
+	return nil
+}
+func (tbl *pgDNSSecTable) toGeneric() []DNSSecKey {
+	keys := make([]DNSSecKey, len(tbl.Records))
+
+	for i, record := range tbl.Records {
+		keys[i] = DNSSecKey{
+			CDN:                    record.CDN,
+			DNSSECKeysTrafficVault: record.Key,
+		}
+	}
+
+	return keys
+}
+func (tbl *pgDNSSecTable) fromGeneric(keys []DNSSecKey) {
+	tbl.Records = make([]pgDNSSecRecord, len(keys))
+
+	for i, key := range keys {
+		tbl.Records[i] = pgDNSSecRecord{
+			Key: key.DNSSECKeysTrafficVault,
+			CDN: key.CDN,
+			pgCommonRecord: pgCommonRecord{
+				DataEncrypted: nil,
+			},
+		}
+	}
+}
+func (tbl *pgDNSSecTable) validate() []string {
+	for i, record := range tbl.Records {
+		if record.DataEncrypted == nil && len(record.Key) > 0 {
+			return []string{fmt.Sprintf("DNSSEC Key %v: DataEncrypted is blank!", i)}
+		}
+	}
+	return nil
+}
+func (tbl *pgDNSSecTable) insertKeys(db *sql.DB) error {
+	queryBase := "INSERT INTO dnssec (cdn, data) VALUES "
+	stride := 2
+	queryArgs := make([]interface{}, len(tbl.Records)*stride)
+	for i, record := range tbl.Records {
+		j := i * stride
+		queryArgs[j] = record.CDN
+		queryArgs[j+1] = record.DataEncrypted
+	}
+	return insertIntoTable(db, queryBase, stride, queryArgs)
+}
+
+type pgSSLKeyRecord struct {
+	Keys tc.DeliveryServiceSSLKeys
+	pgCommonRecord
+
+	// These records are stored on the table but are duplicated
+	DeliveryService string
+	CDN             string
+}
+type pgSSLKeyTable struct {
+	Records []pgSSLKeyRecord
+}
+
+func (tbl *pgSSLKeyTable) insertKeys(db *sql.DB) error {
+	queryBase := "INSERT INTO sslkey (deliveryservice, data, cdn, version) VALUES "
+	duplicateKeys := 2
+	stride := 4
+	queryArgs := make([]interface{}, len(tbl.Records)*stride*duplicateKeys)
+	for i, record := range tbl.Records {
+		j := i * duplicateKeys * stride
+
+		queryArgs[j] = record.DeliveryService
+		queryArgs[j+1] = record.DataEncrypted
+		queryArgs[j+2] = record.CDN
+		queryArgs[j+3] = record.Keys.Version.String()
+
+		queryArgs[j+4] = record.DeliveryService
+		queryArgs[j+5] = record.DataEncrypted
+		queryArgs[j+6] = record.CDN
+		queryArgs[j+7] = "latest"
+	}
+	return insertIntoTable(db, queryBase, 4, queryArgs)
+}
+func (tbl *pgSSLKeyTable) gatherKeys(db *sql.DB) error {
+	sz, err := getSize(db, "sslkey WHERE version='latest'")
+	if err != nil {
+		return fmt.Errorf("PGSSLKey gatherKeys unable to determine size of sslkey table: %w", err)
+	}
+	tbl.Records = make([]pgSSLKeyRecord, sz)
+
+	rows, err := db.Query("SELECT data, deliveryservice, cdn from sslkey WHERE version = 'latest'")
+	if err != nil {
+		return fmt.Errorf("PGSSLKey gatherKeys unable to query: %w", err)
+	}
+	defer rows.Close()
+	i := 0
+	for rows.Next() {
+		if i > len(tbl.Records)-1 {
+			return errors.New("PGSSLKey gatherKeys: got more results than expected")
+		}
+		err := rows.Scan(&tbl.Records[i].DataEncrypted, &tbl.Records[i].DeliveryService, &tbl.Records[i].CDN)
+		if err != nil {
+			return fmt.Errorf("PGSSLKey gatherKeys unable to scan row: %w", err)
+		}
+		i += 1
+	}
+	return nil
+}
+func (tbl *pgSSLKeyTable) decrypt(aesKey []byte) error {
+	for i, dns := range tbl.Records {
+		err := decryptInto(aesKey, dns.DataEncrypted, &tbl.Records[i].Keys)
+		if err != nil {
+			return fmt.Errorf("unable to decrypt into keys: %w", err)
+		}
+	}
+	return nil
+}
+func (tbl *pgSSLKeyTable) encrypt(aesKey []byte) error {
+	for i, dns := range tbl.Records {
+		data, err := json.Marshal(dns.Keys)
+		if err != nil {
+			return fmt.Errorf("encrypt issue marshalling keys: %w", err)
+		}
+		dat, err := encrypt(data, aesKey)
+		if err != nil {
+			return fmt.Errorf("encrypt error: %w", err)
+		}
+		tbl.Records[i].DataEncrypted = dat
+	}
+	return nil
+}
+func (tbl *pgSSLKeyTable) toGeneric() []SSLKey {
+	keys := make([]SSLKey, len(tbl.Records))
+
+	for i, record := range tbl.Records {
+		keys[i] = SSLKey{
+			DeliveryServiceSSLKeys: record.Keys,
+		}
+	}
+	return keys
+}
+func (tbl *pgSSLKeyTable) fromGeneric(keys []SSLKey) {
+	tbl.Records = make([]pgSSLKeyRecord, len(keys))
+
+	for i, key := range keys {
+		tbl.Records[i] = pgSSLKeyRecord{
+			Keys: key.DeliveryServiceSSLKeys,
+			pgCommonRecord: pgCommonRecord{
+				DataEncrypted: nil,
+			},
+			DeliveryService: key.DeliveryService,
+			CDN:             key.CDN,
+		}
+	}
+}
+func (tbl *pgSSLKeyTable) validate() []string {
+	defaultKey := tc.DeliveryServiceSSLKeys{}
+	var errors []string
+	fmtStr := "SSL Key %v: %v"
+	for i, record := range tbl.Records {
+		if record.Keys == defaultKey {
+			errors = append(errors, fmt.Sprintf(fmtStr, i, "DS SSL Keys are default!"))
+		} else if record.Keys.Key == "" {
+			errors = append(errors, fmt.Sprintf(fmtStr, i, "Key is blank!"))
+		} else if record.Keys.CDN == "" {
+			errors = append(errors, fmt.Sprintf(fmtStr, i, "CDN is blank!"))
+		} else if record.Keys.DeliveryService == "" {
+			errors = append(errors, fmt.Sprintf(fmtStr, i, "DS is blank!"))
+		} else if record.DataEncrypted == nil {
+			errors = append(errors, fmt.Sprintf(fmtStr, i, "DataEncrypted is blank!"))
+		} else if record.Keys.Version.String() == "" {
+			errors = append(errors, fmt.Sprintf(fmtStr, i, "Version is blank!"))
+		}
+	}
+	return errors
+}
+
+type pgURLSigKeyRecord struct {
+	Keys            tc.URLSigKeys
+	DeliveryService string
+	pgCommonRecord
+}
+type pgURLSigKeyTable struct {
+	Records []pgURLSigKeyRecord
+}
+
+func (tbl *pgURLSigKeyTable) insertKeys(db *sql.DB) error {
+	queryBase := "INSERT INTO url_sig_key (deliveryservice, data) VALUES "
+	stride := 2
+	queryArgs := make([]interface{}, len(tbl.Records)*stride)
+	for i, record := range tbl.Records {
+		j := i * stride
+		queryArgs[j] = record.DeliveryService
+		queryArgs[j+1] = record.DataEncrypted
+	}
+	return insertIntoTable(db, queryBase, stride, queryArgs)
+}
+func (tbl *pgURLSigKeyTable) gatherKeys(db *sql.DB) error {
+	sz, err := getSize(db, "url_sig_key")
+	if err != nil {
+		log.Println("PGURLSigKey gatherKeys: unable to determine url_sig_key table size")
+	}
+	tbl.Records = make([]pgURLSigKeyRecord, sz)
+
+	rows, err := db.Query("SELECT deliveryservice, data from url_sig_key")
+	if err != nil {
+		return fmt.Errorf("PGURLSigKey gatherKeys error creating query: %w", err)
+	}
+	defer rows.Close()
+	i := 0
+	for rows.Next() {
+		if i > len(tbl.Records)-1 {
+			return fmt.Errorf("PGURLSigKey gatherKeys: got more results than expected %v", len(tbl.Records))
+		}
+		err := rows.Scan(&tbl.Records[i].DeliveryService, &tbl.Records[i].DataEncrypted)
+		if err != nil {
+			return fmt.Errorf("PGURLSigKey gatherKeys: unable to scan row: %w", err)
+		}
+		i += 1
+	}
+	return nil
+}
+func (tbl *pgURLSigKeyTable) decrypt(aesKey []byte) error {
+	for i, sig := range tbl.Records {
+		err := decryptInto(aesKey, sig.DataEncrypted, &tbl.Records[i].Keys)
+		if err != nil {
+			return fmt.Errorf("unable to decrypt into keys: %w", err)
+		}
+	}
+	return nil
+}
+func (tbl *pgURLSigKeyTable) encrypt(aesKey []byte) error {
+	for i, sig := range tbl.Records {
+		data, err := json.Marshal(&sig.Keys)
+		if err != nil {
+			return fmt.Errorf("encrypt issue marshalling keys: %w", err)
+		}
+
+		dat, err := encrypt(data, aesKey)
+		if err != nil {
+			return fmt.Errorf("encrypt error: %w", err)
+		}
+		tbl.Records[i].DataEncrypted = dat
+	}
+	return nil
+}
+func (tbl *pgURLSigKeyTable) toGeneric() []URLSigKey {
+	keys := make([]URLSigKey, len(tbl.Records))
+
+	for i, record := range tbl.Records {
+		keys[i] = URLSigKey{
+			DeliveryService: record.DeliveryService,
+			URLSigKeys:      record.Keys,
+		}
+	}
+	return keys
+}
+func (tbl *pgURLSigKeyTable) fromGeneric(keys []URLSigKey) {
+	tbl.Records = make([]pgURLSigKeyRecord, len(keys))
+
+	for i, key := range keys {
+		tbl.Records[i] = pgURLSigKeyRecord{
+			Keys:            key.URLSigKeys,
+			DeliveryService: key.DeliveryService,
+			pgCommonRecord: pgCommonRecord{
+				DataEncrypted: nil,
+			},
+		}
+	}
+}
+func (tbl *pgURLSigKeyTable) validate() []string {
+	for i, record := range tbl.Records {
+		if record.DataEncrypted == nil && len(record.Keys) > 0 {
+			return []string{fmt.Sprintf("URl Sig Key %v: DataEncrypted is blank!", i)}
+		}
+	}
+	return nil
+}
+
+type pgURISignKeyRecord struct {
+	Keys            map[string]tc.URISignerKeyset
+	DeliveryService string
+	pgCommonRecord
+}
+type pgURISignKeyTable struct {
+	Records []pgURISignKeyRecord
+}
+
+func (tbl *pgURISignKeyTable) insertKeys(db *sql.DB) error {
+	queryBase := "INSERT INTO uri_signing_key (deliveryservice, data) VALUES "
+	stride := 2
+	queryArgs := make([]interface{}, len(tbl.Records)*stride)
+	for i, record := range tbl.Records {
+		j := i * stride
+		queryArgs[j] = record.DeliveryService
+		queryArgs[j+1] = record.DataEncrypted
+	}
+	return insertIntoTable(db, queryBase, stride, queryArgs)
+}
+func (tbl *pgURISignKeyTable) gatherKeys(db *sql.DB) error {
+	sz, err := getSize(db, "uri_signing_key")
+	if err != nil {
+		log.Println("PGURISignKey gatherKeys: unable to determine size of uri_signing_key table")
+	}
+	tbl.Records = make([]pgURISignKeyRecord, sz)
+
+	rows, err := db.Query("SELECT deliveryservice, data from uri_signing_key")
+	if err != nil {
+		return fmt.Errorf("PGURISignKey gatherKeys error while query: %w", err)
+	}
+	defer rows.Close()
+	i := 0
+	for rows.Next() {
+		if i > len(tbl.Records)-1 {
+			return fmt.Errorf("PGURISignKey gatherKeys: got more results than expected %v", len(tbl.Records))
+		}
+		err := rows.Scan(&tbl.Records[i].DeliveryService, &tbl.Records[i].DataEncrypted)
+		if err != nil {
+			return fmt.Errorf("PGURISignKey gatherKeys: unable to scan row: %w", err)
+		}
+		i += 1
+	}
+	return nil
+}
+func (tbl *pgURISignKeyTable) decrypt(aesKey []byte) error {
+	for i, sign := range tbl.Records {
+		err := decryptInto(aesKey, sign.DataEncrypted, &tbl.Records[i].Keys)
+		if err != nil {
+			return fmt.Errorf("unable to decrypt into keys: %w", err)
+		}
+	}
+	return nil
+}
+func (tbl *pgURISignKeyTable) encrypt(aesKey []byte) error {
+	for i, sign := range tbl.Records {
+		data, err := json.Marshal(sign.Keys)
+		if err != nil {
+			return fmt.Errorf("encrypt issue marshalling keys: %w", err)
+		}
+
+		dat, err := encrypt(data, aesKey)
+		if err != nil {
+			return fmt.Errorf("encrypt error: %w", err)
+		}
+		tbl.Records[i].DataEncrypted = dat
+	}
+	return nil
+}
+func (tbl *pgURISignKeyTable) toGeneric() []URISignKey {
+	keys := make([]URISignKey, len(tbl.Records))
+
+	for i, record := range tbl.Records {
+		keys[i] = URISignKey{
+			DeliveryService: record.DeliveryService,
+			Keys:            record.Keys,
+		}
+	}
+
+	return keys
+}
+func (tbl *pgURISignKeyTable) fromGeneric(keys []URISignKey) {
+	tbl.Records = make([]pgURISignKeyRecord, len(keys))
+
+	for i, key := range keys {
+		tbl.Records[i] = pgURISignKeyRecord{
+			Keys:            key.Keys,
+			DeliveryService: key.DeliveryService,
+			pgCommonRecord: pgCommonRecord{
+				DataEncrypted: nil,
+			},
+		}
+	}
+}
+func (tbl *pgURISignKeyTable) validate() []string {
+	for i, record := range tbl.Records {
+		if record.DataEncrypted == nil && len(record.Keys) > 0 {
+			return []string{fmt.Sprintf("URI Sign Key %v: DataEncrypted is blank!", i)}
+		}
+	}
+	return nil
+}
+
+func getSize(db *sql.DB, table string) (int64, error) {
+	rows, err := db.Query("SELECT COUNT(*) FROM " + table)
+	if err != nil {
+		return 0, err
+	}
+	var numRows int64
+	if !rows.Next() {
+		return 0, errors.New("no results returned for: " + table)
+	}
+	err = rows.Scan(&numRows)
+	if err != nil {
+		return 0, fmt.Errorf("error reading number of results for %v: %w", table, err)
+	}
+	return numRows, nil
+}
+func decrypt(record []byte, aesKey []byte) ([]byte, error) {
+	unencrypted, err := util.AESDecrypt(record, aesKey)
+	if err != nil {
+		return nil, fmt.Errorf("unable to decrypt: %w", err)
+	}
+	return unencrypted, nil
+}
+func encrypt(record []byte, aesKey []byte) ([]byte, error) {
+	encrypted, err := util.AESEncrypt(record, aesKey)
+	if err != nil {
+		return nil, err
+	}
+	return encrypted, nil
+}
+func decryptInto(aesKey []byte, encData []byte, value interface{}) error {
+	data, err := decrypt(encData, aesKey)
+	if err != nil {
+		return err
+	}
+	err = json.Unmarshal(data, &value)

Review comment:
       nit: this can be combined with the line below like `if err := function(); err != nil {`

##########
File path: tools/traffic_vault_migrate/riak.go
##########
@@ -0,0 +1,746 @@
+package main
+
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+import (
+	"crypto/tls"
+	"encoding/json"
+	"errors"
+	"fmt"
+	"time"
+
+	"github.com/apache/trafficcontrol/lib/go-rfc"
+	"github.com/apache/trafficcontrol/lib/go-tc"
+
+	"github.com/basho/riak-go-client"
+)
+
+const (
+	BUCKET_SSL     = "ssl"
+	BUCKET_DNSSEC  = "dnssec"
+	BUCKET_URL_SIG = "url_sig_keys"
+	BUCKET_URI_SIG = "cdn_uri_sig_keys"
+
+	INDEX_SSL = "sslkeys"
+
+	SCHEMA_RIAK_KEY    = "_yz_rk"
+	SCHEMA_RIAK_BUCKET = "_yz_rb"
+)
+
+var (
+	SCHEMA_SSL_FIELDS = [...]string{SCHEMA_RIAK_KEY, SCHEMA_RIAK_BUCKET}
+)
+
+// RiakConfig  represents the configuration options available to the Riak backend
+type RiakConfig struct {
+	Host          string `json:"host"`
+	Port          string `json:"port"`
+	User          string `json:"user"`
+	Password      string `json:"password"`
+	Insecure      bool   `json:"insecure"`
+	TLSVersionRaw string `json:"tlsVersion"`
+	TLSVersion    uint16
+}
+
+// RiakBackend is the Riak implementation of TVBackend
+type RiakBackend struct {
+	sslKeys     riakSSLKeyTable
+	dnssecKeys  riakDNSSecKeyTable
+	uriSignKeys riakURISignKeyTable
+	urlSigKeys  riakURLSigKeyTable
+	cfg         RiakConfig
+	cluster     *riak.Cluster
+}
+
+// String returns a high level overview of the backend and its keys
+func (rb *RiakBackend) String() string {
+	data := fmt.Sprintf("Riak server %v@%v:%v\n", rb.cfg.User, rb.cfg.Host, rb.cfg.Port)
+	data += fmt.Sprintf("\tSSL Keys: %v\n", len(rb.sslKeys.Records))
+	data += fmt.Sprintf("\tDNSSec Keys: %v\n", len(rb.dnssecKeys.Records))
+	data += fmt.Sprintf("\tURI Keys: %v\n", len(rb.uriSignKeys.Records))
+	data += fmt.Sprintf("\tURL Keys: %v\n", len(rb.urlSigKeys.Records))
+	return data
+}
+
+// Name returns the name for this backend
+func (rb *RiakBackend) Name() string {
+	return "Riak"
+}
+
+// ReadConfig takes in a filename and will read it into the backends config
+func (rb *RiakBackend) ReadConfig(s string) error {
+	err := UnmarshalConfig(s, &rb.cfg)
+	if err != nil {
+		return err
+	}
+
+	if rb.cfg.TLSVersionRaw == "10" {
+		rb.cfg.TLSVersion = tls.VersionTLS10
+	} else if rb.cfg.TLSVersionRaw == "11" {
+		rb.cfg.TLSVersion = tls.VersionTLS11
+	} else if rb.cfg.TLSVersionRaw == "12" {
+		rb.cfg.TLSVersion = tls.VersionTLS12
+	} else if rb.cfg.TLSVersionRaw == "13" {
+		rb.cfg.TLSVersion = tls.VersionTLS13
+	}
+	return nil
+}
+
+// Insert takes the current keys and inserts them into the backend DB
+func (rb *RiakBackend) Insert() error {
+	if err := rb.sslKeys.insertKeys(rb.cluster); err != nil {
+		return err
+	}
+	if err := rb.dnssecKeys.insertKeys(rb.cluster); err != nil {
+		return err
+	}
+	if err := rb.urlSigKeys.insertKeys(rb.cluster); err != nil {
+		return err
+	}
+	if err := rb.uriSignKeys.insertKeys(rb.cluster); err != nil {
+		return err
+	}
+	return nil
+}
+
+// ValidateKey validates that the keys are valid (in most cases, certain fields are not null)
+func (rb *RiakBackend) ValidateKey() []string {
+	errors := []string{}
+	if errs := rb.sslKeys.validate(); errs != nil {
+		errors = append(errors, errs...)
+	}
+	if errs := rb.dnssecKeys.validate(); errs != nil {
+		errors = append(errors, errs...)
+	}
+	if errs := rb.uriSignKeys.validate(); errs != nil {
+		errors = append(errors, errs...)
+	}
+	if errs := rb.urlSigKeys.validate(); errs != nil {
+		errors = append(errors, errs...)
+	}
+
+	return errors
+}
+
+// SetSSLKeys takes in keys and converts & encrypts the data into the backends internal format
+func (rb *RiakBackend) SetSSLKeys(keys []SSLKey) error {
+	rb.sslKeys.fromGeneric(keys)
+	return nil
+}
+
+// SetDNSSecKeys takes in keys and converts & encrypts the data into the backends internal format
+func (rb *RiakBackend) SetDNSSecKeys(keys []DNSSecKey) error {
+	rb.dnssecKeys.fromGeneric(keys)
+	return nil
+}
+
+// SetURISignKeys takes in keys and converts & encrypts the data into the backends internal format
+func (rb *RiakBackend) SetURISignKeys(keys []URISignKey) error {
+	rb.uriSignKeys.fromGeneric(keys)
+	return nil
+}
+
+// SetURLSigKeys takes in keys and converts & encrypts the data into the backends internal format
+func (rb *RiakBackend) SetURLSigKeys(keys []URLSigKey) error {
+	rb.urlSigKeys.fromGeneric(keys)
+	return nil
+}
+
+// Start initiates the connection to the backend DB
+func (rb *RiakBackend) Start() error {
+	tlsConfig := &tls.Config{
+		InsecureSkipVerify: !rb.cfg.Insecure,
+		MaxVersion:         rb.cfg.TLSVersion,
+	}
+	auth := &riak.AuthOptions{
+		User:      rb.cfg.User,
+		Password:  rb.cfg.Password,
+		TlsConfig: tlsConfig,
+	}
+
+	cluster, err := getRiakCluster(rb.cfg, auth)
+	if err != nil {
+		return err
+	}
+	if err := cluster.Start(); err != nil {
+		return fmt.Errorf("unable to start riak cluster: %w", err)
+	}
+
+	rb.cluster = cluster
+	rb.sslKeys = riakSSLKeyTable{}
+	rb.dnssecKeys = riakDNSSecKeyTable{}
+	rb.urlSigKeys = riakURLSigKeyTable{}
+	rb.uriSignKeys = riakURISignKeyTable{}
+	return nil
+}
+
+// Close terminates the connection to the backend DB
+func (rb *RiakBackend) Close() error {
+	if err := rb.cluster.Stop(); err != nil {
+		return err
+	}
+	return nil
+}
+
+// Ping checks the connection to the backend DB
+func (rb *RiakBackend) Ping() error {
+	return ping(rb.cluster)
+}
+
+// GetSSLKeys converts the backends internal key representation into the common representation (SSLKey)
+func (rb *RiakBackend) GetSSLKeys() ([]SSLKey, error) {
+	return rb.sslKeys.toGeneric(), nil
+}
+
+// GetDNSSecKeys converts the backends internal key representation into the common representation (DNSSecKey)
+func (rb *RiakBackend) GetDNSSecKeys() ([]DNSSecKey, error) {
+	return rb.dnssecKeys.toGeneric(), nil
+}
+
+// GetURISignKeys converts the pg internal key representation into the common representation (URISignKey)
+func (rb *RiakBackend) GetURISignKeys() ([]URISignKey, error) {
+	return rb.uriSignKeys.toGeneric(), nil
+}
+
+// GetURLSigKeys converts the backends internal key representation into the common representation (URLSigKey)
+func (rb *RiakBackend) GetURLSigKeys() ([]URLSigKey, error) {
+	return rb.urlSigKeys.toGeneric(), nil
+}
+
+// Fetch gets all of the keys from the backend DB
+func (rb *RiakBackend) Fetch() error {
+	if err := rb.sslKeys.gatherKeys(rb.cluster); err != nil {
+		return err
+	}
+	if err := rb.dnssecKeys.gatherKeys(rb.cluster); err != nil {
+		return err
+	}
+	if err := rb.urlSigKeys.gatherKeys(rb.cluster); err != nil {
+		return err
+	}
+	if err := rb.uriSignKeys.gatherKeys(rb.cluster); err != nil {
+		return err
+	}
+
+	return nil
+}
+
+type riakSSLKeyRecord struct {
+	tc.DeliveryServiceSSLKeys
+}
+type riakSSLKeyTable struct {
+	Records []riakSSLKeyRecord
+}
+
+func (tbl *riakSSLKeyTable) gatherKeys(cluster *riak.Cluster) error {
+	searchDocs, err := search(cluster, INDEX_SSL, "cdn:*", fmt.Sprintf("%v:*latest", SCHEMA_RIAK_KEY), 1000, SCHEMA_SSL_FIELDS[:])
+	if err != nil {
+		return fmt.Errorf("RiakSSLKey gatherKeys: %w", err)
+	}
+
+	tbl.Records = make([]riakSSLKeyRecord, len(searchDocs))
+	for i, doc := range searchDocs {
+		objs, err := getObject(cluster, doc.Bucket, doc.Key)
+		if err != nil {
+			return err
+		}
+		if len(objs) < 1 {
+			return fmt.Errorf("RiakSSLKey gatherKeys unable to find any objects with key %v and bucket %v, but search results were returned", doc.Key, doc.Bucket)
+		}
+		if len(objs) > 1 {
+			return fmt.Errorf("RiakSSLKey gatherKeys more than 1 ssl key record found %v\n", len(objs))

Review comment:
       this error message should probably include the key

##########
File path: tools/traffic_vault_migrate/postgres.go
##########
@@ -0,0 +1,743 @@
+package main
+
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+import (
+	"database/sql"
+	"encoding/base64"
+	"encoding/json"
+	"errors"
+	"fmt"
+	"log"
+	"strconv"
+	"strings"
+
+	"github.com/apache/trafficcontrol/lib/go-tc"
+	util "github.com/apache/trafficcontrol/lib/go-util"
+
+	_ "github.com/lib/pq"
+)
+
+// PGConfig represents the configuration options available to the PG backend
+type PGConfig struct {
+	Host     string `json:"host"`
+	Port     string `json:"port"`
+	User     string `json:"user"`
+	Password string `json:"password"`
+	SSLMode  string `json:"sslmode"`
+	Database string `json:"database"`
+	Key      string `json:"aesKey"`
+	AESKey   []byte
+}
+
+// PGBackend is the Postgres implementation of TVBackend
+type PGBackend struct {
+	sslKey pgSSLKeyTable
+	dnssec pgDNSSecTable
+	uri    pgURISignKeyTable
+	url    pgURLSigKeyTable
+	cfg    PGConfig
+	db     *sql.DB
+}
+
+// String returns a high level overview of the backend and its keys
+func (pg *PGBackend) String() string {
+	data := fmt.Sprintf("PG server %v@%v:%v\n", pg.cfg.User, pg.cfg.Host, pg.cfg.Port)
+	data += fmt.Sprintf("\tSSL Keys: %v\n", len(pg.sslKey.Records))
+	data += fmt.Sprintf("\tDNSSec Keys: %v\n", len(pg.dnssec.Records))
+	data += fmt.Sprintf("\tURI Keys: %v\n", len(pg.uri.Records))
+	data += fmt.Sprintf("\tURL Keys: %v\n", len(pg.url.Records))
+	return data
+}
+
+// Name returns the name for this backend
+func (pg *PGBackend) Name() string {
+	return "PG"
+}
+
+// ReadConfig takes in a filename and will read it into the backends config
+func (pg *PGBackend) ReadConfig(s string) error {
+	err := UnmarshalConfig(s, &pg.cfg)
+	if err != nil {
+		return err
+	}
+
+	pg.cfg.AESKey, err = base64.StdEncoding.DecodeString(pg.cfg.Key)
+	if err != nil {
+		return fmt.Errorf("unable to decode PG AESKey: %w", err)
+	}
+	return nil
+}
+
+// Insert takes the current keys and inserts them into the backend DB
+func (pg *PGBackend) Insert() error {
+	if err := pg.sslKey.insertKeys(pg.db); err != nil {
+		return err
+	}
+	if err := pg.dnssec.insertKeys(pg.db); err != nil {
+		return err
+	}
+	if err := pg.url.insertKeys(pg.db); err != nil {
+		return err
+	}
+	if err := pg.uri.insertKeys(pg.db); err != nil {
+		return err
+	}
+	return nil
+}
+
+// Start initiates the connection to the backend DB
+func (pg *PGBackend) Start() error {
+	sqlStr := fmt.Sprintf("postgres://%s:%s@%s:%s/%s?sslmode=%s", pg.cfg.User, pg.cfg.Password, pg.cfg.Host, pg.cfg.Port, pg.cfg.Database, pg.cfg.SSLMode)
+	db, err := sql.Open("postgres", sqlStr)
+	if err != nil {
+		return fmt.Errorf("unable to start PG client: %w", err)
+	}
+
+	pg.db = db
+	pg.sslKey = pgSSLKeyTable{}
+	pg.dnssec = pgDNSSecTable{}
+	pg.url = pgURLSigKeyTable{}
+	pg.uri = pgURISignKeyTable{}
+
+	return nil
+}
+
+// ValidateKey validates that the keys are valid (in most cases, certain fields are not null)
+func (pg *PGBackend) ValidateKey() []string {
+	var errors []string
+	if errs := pg.sslKey.validate(); errs != nil {
+		errors = append(errors, errs...)
+	}
+	if errs := pg.dnssec.validate(); errs != nil {
+		errors = append(errors, errs...)
+	}
+	if errs := pg.uri.validate(); errs != nil {
+		errors = append(errors, errs...)
+	}
+	if errs := pg.url.validate(); errs != nil {
+		errors = append(errors, errs...)
+	}
+	return errors
+}
+
+// Close terminates the connection to the backend DB
+func (pg *PGBackend) Close() error {
+	return pg.db.Close()
+}
+
+// Ping checks the connection to the backend DB
+func (pg *PGBackend) Ping() error {
+	return pg.db.Ping()
+}
+
+// Fetch gets all of the keys from the backend DB
+func (pg *PGBackend) Fetch() error {
+	if err := pg.sslKey.gatherKeys(pg.db); err != nil {
+		return err
+	}
+
+	if err := pg.dnssec.gatherKeys(pg.db); err != nil {
+		return err
+	}
+
+	if err := pg.url.gatherKeys(pg.db); err != nil {
+		return err
+	}
+
+	if err := pg.uri.gatherKeys(pg.db); err != nil {
+		return err
+	}
+
+	return nil
+}
+
+// GetSSLKeys converts the backends internal key representation into the common representation (SSLKey)
+func (pg *PGBackend) GetSSLKeys() ([]SSLKey, error) {
+	if err := pg.sslKey.decrypt(pg.cfg.AESKey); err != nil {
+		return nil, err
+	}
+	return pg.sslKey.toGeneric(), nil
+}
+
+// SetSSLKeys takes in keys and converts & encrypts the data into the backends internal format
+func (pg *PGBackend) SetSSLKeys(keys []SSLKey) error {
+	pg.sslKey.fromGeneric(keys)
+	return pg.sslKey.encrypt(pg.cfg.AESKey)
+}
+
+// GetDNSSecKeys converts the backends internal key representation into the common representation (DNSSecKey)
+func (pg *PGBackend) GetDNSSecKeys() ([]DNSSecKey, error) {
+	if err := pg.dnssec.decrypt(pg.cfg.AESKey); err != nil {
+		return nil, err
+	}
+	return pg.dnssec.toGeneric(), nil
+}
+
+// SetDNSSecKeys takes in keys and converts & encrypts the data into the backends internal format
+func (pg *PGBackend) SetDNSSecKeys(keys []DNSSecKey) error {
+	pg.dnssec.fromGeneric(keys)
+	return pg.dnssec.encrypt(pg.cfg.AESKey)
+}
+
+// GetURISignKeys converts the pg internal key representation into the common representation (URISignKey)
+func (pg *PGBackend) GetURISignKeys() ([]URISignKey, error) {
+	if err := pg.uri.decrypt(pg.cfg.AESKey); err != nil {
+		return nil, err
+	}
+	return pg.uri.toGeneric(), nil
+}
+
+// SetURISignKeys takes in keys and converts & encrypts the data into the backends internal format
+func (pg *PGBackend) SetURISignKeys(keys []URISignKey) error {
+	pg.uri.fromGeneric(keys)
+	return pg.uri.encrypt(pg.cfg.AESKey)
+}
+
+// GetURLSigKeys converts the backends internal key representation into the common representation (URLSigKey)
+func (pg *PGBackend) GetURLSigKeys() ([]URLSigKey, error) {
+	if err := pg.url.decrypt(pg.cfg.AESKey); err != nil {
+		return nil, err
+	}
+	return pg.url.toGeneric(), nil
+}
+
+// SetURLSigKeys takes in keys and converts & encrypts the data into the backends internal format
+func (pg *PGBackend) SetURLSigKeys(keys []URLSigKey) error {
+	pg.url.fromGeneric(keys)
+	return pg.url.encrypt(pg.cfg.AESKey)
+}
+
+type pgCommonRecord struct {
+	DataEncrypted []byte
+}
+
+type pgDNSSecRecord struct {
+	Key tc.DNSSECKeysTrafficVault
+	CDN string
+	pgCommonRecord
+}
+type pgDNSSecTable struct {
+	Records []pgDNSSecRecord
+}
+
+func (tbl *pgDNSSecTable) gatherKeys(db *sql.DB) error {
+	sz, err := getSize(db, "dnssec")
+	if err != nil {
+		log.Println("PGDNSSec gatherKeys: unable to determine size of dnssec table")
+	}
+	tbl.Records = make([]pgDNSSecRecord, sz)
+
+	rows, err := db.Query("SELECT cdn, data from dnssec")
+	if err != nil {
+		return fmt.Errorf("PGDNSSec gatherKeys: unable to query: %w", err)
+	}
+	defer rows.Close()
+	i := 0
+	for rows.Next() {
+		if i > len(tbl.Records)-1 {
+			return fmt.Errorf("PGDNSSec gatherKeys got more results than expected %v", len(tbl.Records))
+		}
+		err := rows.Scan(&tbl.Records[i].CDN, &tbl.Records[i].DataEncrypted)
+		if err != nil {
+			return fmt.Errorf("PGDNSSec gatherKeys unable to scan row: %w", err)
+		}
+		i += 1
+	}
+	return nil
+}
+func (tbl *pgDNSSecTable) decrypt(aesKey []byte) error {
+	for i, _ := range tbl.Records {
+		err := decryptInto(aesKey, tbl.Records[i].DataEncrypted, &tbl.Records[i].Key)
+		if err != nil {
+			return fmt.Errorf("unable to decrypt into keys: %w", err)
+		}
+	}
+	return nil
+}
+func (tbl *pgDNSSecTable) encrypt(aesKey []byte) error {
+	for i, dns := range tbl.Records {
+		data, err := json.Marshal(&dns.Key)
+		if err != nil {
+			return fmt.Errorf("encrypt issue marshalling keys: %w", err)
+		}
+		dat, err := encrypt(data, aesKey)
+		if err != nil {
+			return fmt.Errorf("encrypt error: %w", err)
+		}
+		tbl.Records[i].DataEncrypted = dat
+	}
+	return nil
+}
+func (tbl *pgDNSSecTable) toGeneric() []DNSSecKey {
+	keys := make([]DNSSecKey, len(tbl.Records))
+
+	for i, record := range tbl.Records {
+		keys[i] = DNSSecKey{
+			CDN:                    record.CDN,
+			DNSSECKeysTrafficVault: record.Key,
+		}
+	}
+
+	return keys
+}
+func (tbl *pgDNSSecTable) fromGeneric(keys []DNSSecKey) {
+	tbl.Records = make([]pgDNSSecRecord, len(keys))
+
+	for i, key := range keys {
+		tbl.Records[i] = pgDNSSecRecord{
+			Key: key.DNSSECKeysTrafficVault,
+			CDN: key.CDN,
+			pgCommonRecord: pgCommonRecord{
+				DataEncrypted: nil,
+			},
+		}
+	}
+}
+func (tbl *pgDNSSecTable) validate() []string {
+	for i, record := range tbl.Records {
+		if record.DataEncrypted == nil && len(record.Key) > 0 {
+			return []string{fmt.Sprintf("DNSSEC Key %v: DataEncrypted is blank!", i)}
+		}
+	}
+	return nil
+}
+func (tbl *pgDNSSecTable) insertKeys(db *sql.DB) error {
+	queryBase := "INSERT INTO dnssec (cdn, data) VALUES "
+	stride := 2
+	queryArgs := make([]interface{}, len(tbl.Records)*stride)
+	for i, record := range tbl.Records {
+		j := i * stride
+		queryArgs[j] = record.CDN
+		queryArgs[j+1] = record.DataEncrypted
+	}
+	return insertIntoTable(db, queryBase, stride, queryArgs)
+}
+
+type pgSSLKeyRecord struct {
+	Keys tc.DeliveryServiceSSLKeys
+	pgCommonRecord
+
+	// These records are stored on the table but are duplicated
+	DeliveryService string
+	CDN             string
+}
+type pgSSLKeyTable struct {
+	Records []pgSSLKeyRecord
+}
+
+func (tbl *pgSSLKeyTable) insertKeys(db *sql.DB) error {
+	queryBase := "INSERT INTO sslkey (deliveryservice, data, cdn, version) VALUES "
+	duplicateKeys := 2
+	stride := 4
+	queryArgs := make([]interface{}, len(tbl.Records)*stride*duplicateKeys)
+	for i, record := range tbl.Records {
+		j := i * duplicateKeys * stride
+
+		queryArgs[j] = record.DeliveryService
+		queryArgs[j+1] = record.DataEncrypted
+		queryArgs[j+2] = record.CDN
+		queryArgs[j+3] = record.Keys.Version.String()
+
+		queryArgs[j+4] = record.DeliveryService
+		queryArgs[j+5] = record.DataEncrypted
+		queryArgs[j+6] = record.CDN
+		queryArgs[j+7] = "latest"
+	}
+	return insertIntoTable(db, queryBase, 4, queryArgs)
+}
+func (tbl *pgSSLKeyTable) gatherKeys(db *sql.DB) error {
+	sz, err := getSize(db, "sslkey WHERE version='latest'")
+	if err != nil {
+		return fmt.Errorf("PGSSLKey gatherKeys unable to determine size of sslkey table: %w", err)
+	}
+	tbl.Records = make([]pgSSLKeyRecord, sz)
+
+	rows, err := db.Query("SELECT data, deliveryservice, cdn from sslkey WHERE version = 'latest'")
+	if err != nil {
+		return fmt.Errorf("PGSSLKey gatherKeys unable to query: %w", err)
+	}
+	defer rows.Close()
+	i := 0
+	for rows.Next() {
+		if i > len(tbl.Records)-1 {
+			return errors.New("PGSSLKey gatherKeys: got more results than expected")
+		}
+		err := rows.Scan(&tbl.Records[i].DataEncrypted, &tbl.Records[i].DeliveryService, &tbl.Records[i].CDN)
+		if err != nil {
+			return fmt.Errorf("PGSSLKey gatherKeys unable to scan row: %w", err)
+		}
+		i += 1
+	}
+	return nil
+}
+func (tbl *pgSSLKeyTable) decrypt(aesKey []byte) error {
+	for i, dns := range tbl.Records {
+		err := decryptInto(aesKey, dns.DataEncrypted, &tbl.Records[i].Keys)
+		if err != nil {
+			return fmt.Errorf("unable to decrypt into keys: %w", err)
+		}
+	}
+	return nil
+}
+func (tbl *pgSSLKeyTable) encrypt(aesKey []byte) error {
+	for i, dns := range tbl.Records {
+		data, err := json.Marshal(dns.Keys)
+		if err != nil {
+			return fmt.Errorf("encrypt issue marshalling keys: %w", err)
+		}
+		dat, err := encrypt(data, aesKey)
+		if err != nil {
+			return fmt.Errorf("encrypt error: %w", err)
+		}
+		tbl.Records[i].DataEncrypted = dat
+	}
+	return nil
+}
+func (tbl *pgSSLKeyTable) toGeneric() []SSLKey {
+	keys := make([]SSLKey, len(tbl.Records))
+
+	for i, record := range tbl.Records {
+		keys[i] = SSLKey{
+			DeliveryServiceSSLKeys: record.Keys,
+		}
+	}
+	return keys
+}
+func (tbl *pgSSLKeyTable) fromGeneric(keys []SSLKey) {
+	tbl.Records = make([]pgSSLKeyRecord, len(keys))
+
+	for i, key := range keys {
+		tbl.Records[i] = pgSSLKeyRecord{
+			Keys: key.DeliveryServiceSSLKeys,
+			pgCommonRecord: pgCommonRecord{
+				DataEncrypted: nil,
+			},
+			DeliveryService: key.DeliveryService,
+			CDN:             key.CDN,
+		}
+	}
+}
+func (tbl *pgSSLKeyTable) validate() []string {
+	defaultKey := tc.DeliveryServiceSSLKeys{}
+	var errors []string
+	fmtStr := "SSL Key %v: %v"
+	for i, record := range tbl.Records {
+		if record.Keys == defaultKey {
+			errors = append(errors, fmt.Sprintf(fmtStr, i, "DS SSL Keys are default!"))
+		} else if record.Keys.Key == "" {
+			errors = append(errors, fmt.Sprintf(fmtStr, i, "Key is blank!"))
+		} else if record.Keys.CDN == "" {
+			errors = append(errors, fmt.Sprintf(fmtStr, i, "CDN is blank!"))
+		} else if record.Keys.DeliveryService == "" {
+			errors = append(errors, fmt.Sprintf(fmtStr, i, "DS is blank!"))
+		} else if record.DataEncrypted == nil {
+			errors = append(errors, fmt.Sprintf(fmtStr, i, "DataEncrypted is blank!"))
+		} else if record.Keys.Version.String() == "" {
+			errors = append(errors, fmt.Sprintf(fmtStr, i, "Version is blank!"))
+		}
+	}
+	return errors
+}
+
+type pgURLSigKeyRecord struct {
+	Keys            tc.URLSigKeys
+	DeliveryService string
+	pgCommonRecord
+}
+type pgURLSigKeyTable struct {
+	Records []pgURLSigKeyRecord
+}
+
+func (tbl *pgURLSigKeyTable) insertKeys(db *sql.DB) error {
+	queryBase := "INSERT INTO url_sig_key (deliveryservice, data) VALUES "
+	stride := 2
+	queryArgs := make([]interface{}, len(tbl.Records)*stride)
+	for i, record := range tbl.Records {
+		j := i * stride
+		queryArgs[j] = record.DeliveryService
+		queryArgs[j+1] = record.DataEncrypted
+	}
+	return insertIntoTable(db, queryBase, stride, queryArgs)
+}
+func (tbl *pgURLSigKeyTable) gatherKeys(db *sql.DB) error {
+	sz, err := getSize(db, "url_sig_key")
+	if err != nil {
+		log.Println("PGURLSigKey gatherKeys: unable to determine url_sig_key table size")
+	}
+	tbl.Records = make([]pgURLSigKeyRecord, sz)
+
+	rows, err := db.Query("SELECT deliveryservice, data from url_sig_key")
+	if err != nil {
+		return fmt.Errorf("PGURLSigKey gatherKeys error creating query: %w", err)
+	}
+	defer rows.Close()
+	i := 0
+	for rows.Next() {
+		if i > len(tbl.Records)-1 {
+			return fmt.Errorf("PGURLSigKey gatherKeys: got more results than expected %v", len(tbl.Records))
+		}
+		err := rows.Scan(&tbl.Records[i].DeliveryService, &tbl.Records[i].DataEncrypted)
+		if err != nil {
+			return fmt.Errorf("PGURLSigKey gatherKeys: unable to scan row: %w", err)
+		}
+		i += 1
+	}
+	return nil
+}
+func (tbl *pgURLSigKeyTable) decrypt(aesKey []byte) error {
+	for i, sig := range tbl.Records {
+		err := decryptInto(aesKey, sig.DataEncrypted, &tbl.Records[i].Keys)
+		if err != nil {
+			return fmt.Errorf("unable to decrypt into keys: %w", err)
+		}
+	}
+	return nil
+}
+func (tbl *pgURLSigKeyTable) encrypt(aesKey []byte) error {
+	for i, sig := range tbl.Records {
+		data, err := json.Marshal(&sig.Keys)
+		if err != nil {
+			return fmt.Errorf("encrypt issue marshalling keys: %w", err)
+		}
+
+		dat, err := encrypt(data, aesKey)
+		if err != nil {
+			return fmt.Errorf("encrypt error: %w", err)
+		}
+		tbl.Records[i].DataEncrypted = dat
+	}
+	return nil
+}
+func (tbl *pgURLSigKeyTable) toGeneric() []URLSigKey {
+	keys := make([]URLSigKey, len(tbl.Records))
+
+	for i, record := range tbl.Records {
+		keys[i] = URLSigKey{
+			DeliveryService: record.DeliveryService,
+			URLSigKeys:      record.Keys,
+		}
+	}
+	return keys
+}
+func (tbl *pgURLSigKeyTable) fromGeneric(keys []URLSigKey) {
+	tbl.Records = make([]pgURLSigKeyRecord, len(keys))
+
+	for i, key := range keys {
+		tbl.Records[i] = pgURLSigKeyRecord{
+			Keys:            key.URLSigKeys,
+			DeliveryService: key.DeliveryService,
+			pgCommonRecord: pgCommonRecord{
+				DataEncrypted: nil,
+			},
+		}
+	}
+}
+func (tbl *pgURLSigKeyTable) validate() []string {
+	for i, record := range tbl.Records {
+		if record.DataEncrypted == nil && len(record.Keys) > 0 {
+			return []string{fmt.Sprintf("URl Sig Key %v: DataEncrypted is blank!", i)}
+		}
+	}
+	return nil
+}
+
+type pgURISignKeyRecord struct {
+	Keys            map[string]tc.URISignerKeyset
+	DeliveryService string
+	pgCommonRecord
+}
+type pgURISignKeyTable struct {
+	Records []pgURISignKeyRecord
+}
+
+func (tbl *pgURISignKeyTable) insertKeys(db *sql.DB) error {
+	queryBase := "INSERT INTO uri_signing_key (deliveryservice, data) VALUES "
+	stride := 2
+	queryArgs := make([]interface{}, len(tbl.Records)*stride)
+	for i, record := range tbl.Records {
+		j := i * stride
+		queryArgs[j] = record.DeliveryService
+		queryArgs[j+1] = record.DataEncrypted
+	}
+	return insertIntoTable(db, queryBase, stride, queryArgs)
+}
+func (tbl *pgURISignKeyTable) gatherKeys(db *sql.DB) error {
+	sz, err := getSize(db, "uri_signing_key")
+	if err != nil {
+		log.Println("PGURISignKey gatherKeys: unable to determine size of uri_signing_key table")
+	}
+	tbl.Records = make([]pgURISignKeyRecord, sz)
+
+	rows, err := db.Query("SELECT deliveryservice, data from uri_signing_key")
+	if err != nil {
+		return fmt.Errorf("PGURISignKey gatherKeys error while query: %w", err)
+	}
+	defer rows.Close()
+	i := 0
+	for rows.Next() {
+		if i > len(tbl.Records)-1 {
+			return fmt.Errorf("PGURISignKey gatherKeys: got more results than expected %v", len(tbl.Records))
+		}
+		err := rows.Scan(&tbl.Records[i].DeliveryService, &tbl.Records[i].DataEncrypted)
+		if err != nil {
+			return fmt.Errorf("PGURISignKey gatherKeys: unable to scan row: %w", err)
+		}
+		i += 1
+	}
+	return nil
+}
+func (tbl *pgURISignKeyTable) decrypt(aesKey []byte) error {
+	for i, sign := range tbl.Records {
+		err := decryptInto(aesKey, sign.DataEncrypted, &tbl.Records[i].Keys)
+		if err != nil {
+			return fmt.Errorf("unable to decrypt into keys: %w", err)
+		}
+	}
+	return nil
+}
+func (tbl *pgURISignKeyTable) encrypt(aesKey []byte) error {
+	for i, sign := range tbl.Records {
+		data, err := json.Marshal(sign.Keys)
+		if err != nil {
+			return fmt.Errorf("encrypt issue marshalling keys: %w", err)
+		}
+
+		dat, err := encrypt(data, aesKey)
+		if err != nil {
+			return fmt.Errorf("encrypt error: %w", err)
+		}
+		tbl.Records[i].DataEncrypted = dat
+	}
+	return nil
+}
+func (tbl *pgURISignKeyTable) toGeneric() []URISignKey {
+	keys := make([]URISignKey, len(tbl.Records))
+
+	for i, record := range tbl.Records {
+		keys[i] = URISignKey{
+			DeliveryService: record.DeliveryService,
+			Keys:            record.Keys,
+		}
+	}
+
+	return keys
+}
+func (tbl *pgURISignKeyTable) fromGeneric(keys []URISignKey) {
+	tbl.Records = make([]pgURISignKeyRecord, len(keys))
+
+	for i, key := range keys {
+		tbl.Records[i] = pgURISignKeyRecord{
+			Keys:            key.Keys,
+			DeliveryService: key.DeliveryService,
+			pgCommonRecord: pgCommonRecord{
+				DataEncrypted: nil,
+			},
+		}
+	}
+}
+func (tbl *pgURISignKeyTable) validate() []string {
+	for i, record := range tbl.Records {
+		if record.DataEncrypted == nil && len(record.Keys) > 0 {
+			return []string{fmt.Sprintf("URI Sign Key %v: DataEncrypted is blank!", i)}
+		}
+	}
+	return nil
+}
+
+func getSize(db *sql.DB, table string) (int64, error) {
+	rows, err := db.Query("SELECT COUNT(*) FROM " + table)
+	if err != nil {
+		return 0, err
+	}
+	var numRows int64
+	if !rows.Next() {
+		return 0, errors.New("no results returned for: " + table)
+	}
+	err = rows.Scan(&numRows)
+	if err != nil {
+		return 0, fmt.Errorf("error reading number of results for %v: %w", table, err)

Review comment:
       nit: %s instead of %v

##########
File path: tools/traffic_vault_migrate/riak.go
##########
@@ -0,0 +1,746 @@
+package main
+
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+import (
+	"crypto/tls"
+	"encoding/json"
+	"errors"
+	"fmt"
+	"time"
+
+	"github.com/apache/trafficcontrol/lib/go-rfc"
+	"github.com/apache/trafficcontrol/lib/go-tc"
+
+	"github.com/basho/riak-go-client"
+)
+
+const (
+	BUCKET_SSL     = "ssl"
+	BUCKET_DNSSEC  = "dnssec"
+	BUCKET_URL_SIG = "url_sig_keys"
+	BUCKET_URI_SIG = "cdn_uri_sig_keys"
+
+	INDEX_SSL = "sslkeys"
+
+	SCHEMA_RIAK_KEY    = "_yz_rk"
+	SCHEMA_RIAK_BUCKET = "_yz_rb"
+)
+
+var (
+	SCHEMA_SSL_FIELDS = [...]string{SCHEMA_RIAK_KEY, SCHEMA_RIAK_BUCKET}
+)
+
+// RiakConfig  represents the configuration options available to the Riak backend
+type RiakConfig struct {
+	Host          string `json:"host"`
+	Port          string `json:"port"`
+	User          string `json:"user"`
+	Password      string `json:"password"`
+	Insecure      bool   `json:"insecure"`
+	TLSVersionRaw string `json:"tlsVersion"`
+	TLSVersion    uint16
+}
+
+// RiakBackend is the Riak implementation of TVBackend
+type RiakBackend struct {
+	sslKeys     riakSSLKeyTable
+	dnssecKeys  riakDNSSecKeyTable
+	uriSignKeys riakURISignKeyTable
+	urlSigKeys  riakURLSigKeyTable
+	cfg         RiakConfig
+	cluster     *riak.Cluster
+}
+
+// String returns a high level overview of the backend and its keys
+func (rb *RiakBackend) String() string {
+	data := fmt.Sprintf("Riak server %v@%v:%v\n", rb.cfg.User, rb.cfg.Host, rb.cfg.Port)
+	data += fmt.Sprintf("\tSSL Keys: %v\n", len(rb.sslKeys.Records))
+	data += fmt.Sprintf("\tDNSSec Keys: %v\n", len(rb.dnssecKeys.Records))
+	data += fmt.Sprintf("\tURI Keys: %v\n", len(rb.uriSignKeys.Records))
+	data += fmt.Sprintf("\tURL Keys: %v\n", len(rb.urlSigKeys.Records))
+	return data
+}
+
+// Name returns the name for this backend
+func (rb *RiakBackend) Name() string {
+	return "Riak"
+}
+
+// ReadConfig takes in a filename and will read it into the backends config
+func (rb *RiakBackend) ReadConfig(s string) error {
+	err := UnmarshalConfig(s, &rb.cfg)
+	if err != nil {
+		return err
+	}
+
+	if rb.cfg.TLSVersionRaw == "10" {
+		rb.cfg.TLSVersion = tls.VersionTLS10
+	} else if rb.cfg.TLSVersionRaw == "11" {
+		rb.cfg.TLSVersion = tls.VersionTLS11
+	} else if rb.cfg.TLSVersionRaw == "12" {
+		rb.cfg.TLSVersion = tls.VersionTLS12
+	} else if rb.cfg.TLSVersionRaw == "13" {
+		rb.cfg.TLSVersion = tls.VersionTLS13
+	}
+	return nil
+}
+
+// Insert takes the current keys and inserts them into the backend DB
+func (rb *RiakBackend) Insert() error {
+	if err := rb.sslKeys.insertKeys(rb.cluster); err != nil {
+		return err
+	}
+	if err := rb.dnssecKeys.insertKeys(rb.cluster); err != nil {
+		return err
+	}
+	if err := rb.urlSigKeys.insertKeys(rb.cluster); err != nil {
+		return err
+	}
+	if err := rb.uriSignKeys.insertKeys(rb.cluster); err != nil {
+		return err
+	}
+	return nil
+}
+
+// ValidateKey validates that the keys are valid (in most cases, certain fields are not null)
+func (rb *RiakBackend) ValidateKey() []string {
+	errors := []string{}
+	if errs := rb.sslKeys.validate(); errs != nil {
+		errors = append(errors, errs...)
+	}
+	if errs := rb.dnssecKeys.validate(); errs != nil {
+		errors = append(errors, errs...)
+	}
+	if errs := rb.uriSignKeys.validate(); errs != nil {
+		errors = append(errors, errs...)
+	}
+	if errs := rb.urlSigKeys.validate(); errs != nil {
+		errors = append(errors, errs...)
+	}
+
+	return errors
+}
+
+// SetSSLKeys takes in keys and converts & encrypts the data into the backends internal format
+func (rb *RiakBackend) SetSSLKeys(keys []SSLKey) error {
+	rb.sslKeys.fromGeneric(keys)
+	return nil
+}
+
+// SetDNSSecKeys takes in keys and converts & encrypts the data into the backends internal format
+func (rb *RiakBackend) SetDNSSecKeys(keys []DNSSecKey) error {
+	rb.dnssecKeys.fromGeneric(keys)
+	return nil
+}
+
+// SetURISignKeys takes in keys and converts & encrypts the data into the backends internal format
+func (rb *RiakBackend) SetURISignKeys(keys []URISignKey) error {
+	rb.uriSignKeys.fromGeneric(keys)
+	return nil
+}
+
+// SetURLSigKeys takes in keys and converts & encrypts the data into the backends internal format
+func (rb *RiakBackend) SetURLSigKeys(keys []URLSigKey) error {
+	rb.urlSigKeys.fromGeneric(keys)
+	return nil
+}
+
+// Start initiates the connection to the backend DB
+func (rb *RiakBackend) Start() error {
+	tlsConfig := &tls.Config{
+		InsecureSkipVerify: !rb.cfg.Insecure,
+		MaxVersion:         rb.cfg.TLSVersion,
+	}
+	auth := &riak.AuthOptions{
+		User:      rb.cfg.User,
+		Password:  rb.cfg.Password,
+		TlsConfig: tlsConfig,
+	}
+
+	cluster, err := getRiakCluster(rb.cfg, auth)
+	if err != nil {
+		return err
+	}
+	if err := cluster.Start(); err != nil {
+		return fmt.Errorf("unable to start riak cluster: %w", err)
+	}
+
+	rb.cluster = cluster
+	rb.sslKeys = riakSSLKeyTable{}
+	rb.dnssecKeys = riakDNSSecKeyTable{}
+	rb.urlSigKeys = riakURLSigKeyTable{}
+	rb.uriSignKeys = riakURISignKeyTable{}
+	return nil
+}
+
+// Close terminates the connection to the backend DB
+func (rb *RiakBackend) Close() error {
+	if err := rb.cluster.Stop(); err != nil {
+		return err
+	}
+	return nil
+}
+
+// Ping checks the connection to the backend DB
+func (rb *RiakBackend) Ping() error {
+	return ping(rb.cluster)
+}
+
+// GetSSLKeys converts the backends internal key representation into the common representation (SSLKey)
+func (rb *RiakBackend) GetSSLKeys() ([]SSLKey, error) {
+	return rb.sslKeys.toGeneric(), nil
+}
+
+// GetDNSSecKeys converts the backends internal key representation into the common representation (DNSSecKey)
+func (rb *RiakBackend) GetDNSSecKeys() ([]DNSSecKey, error) {
+	return rb.dnssecKeys.toGeneric(), nil
+}
+
+// GetURISignKeys converts the pg internal key representation into the common representation (URISignKey)
+func (rb *RiakBackend) GetURISignKeys() ([]URISignKey, error) {
+	return rb.uriSignKeys.toGeneric(), nil
+}
+
+// GetURLSigKeys converts the backends internal key representation into the common representation (URLSigKey)
+func (rb *RiakBackend) GetURLSigKeys() ([]URLSigKey, error) {
+	return rb.urlSigKeys.toGeneric(), nil
+}
+
+// Fetch gets all of the keys from the backend DB
+func (rb *RiakBackend) Fetch() error {
+	if err := rb.sslKeys.gatherKeys(rb.cluster); err != nil {
+		return err
+	}
+	if err := rb.dnssecKeys.gatherKeys(rb.cluster); err != nil {
+		return err
+	}
+	if err := rb.urlSigKeys.gatherKeys(rb.cluster); err != nil {
+		return err
+	}
+	if err := rb.uriSignKeys.gatherKeys(rb.cluster); err != nil {
+		return err
+	}
+
+	return nil
+}
+
+type riakSSLKeyRecord struct {
+	tc.DeliveryServiceSSLKeys
+}
+type riakSSLKeyTable struct {
+	Records []riakSSLKeyRecord
+}
+
+func (tbl *riakSSLKeyTable) gatherKeys(cluster *riak.Cluster) error {
+	searchDocs, err := search(cluster, INDEX_SSL, "cdn:*", fmt.Sprintf("%v:*latest", SCHEMA_RIAK_KEY), 1000, SCHEMA_SSL_FIELDS[:])
+	if err != nil {
+		return fmt.Errorf("RiakSSLKey gatherKeys: %w", err)
+	}
+
+	tbl.Records = make([]riakSSLKeyRecord, len(searchDocs))
+	for i, doc := range searchDocs {
+		objs, err := getObject(cluster, doc.Bucket, doc.Key)
+		if err != nil {
+			return err
+		}
+		if len(objs) < 1 {
+			return fmt.Errorf("RiakSSLKey gatherKeys unable to find any objects with key %v and bucket %v, but search results were returned", doc.Key, doc.Bucket)
+		}
+		if len(objs) > 1 {
+			return fmt.Errorf("RiakSSLKey gatherKeys more than 1 ssl key record found %v\n", len(objs))
+		}
+		var obj tc.DeliveryServiceSSLKeys
+		if err = json.Unmarshal(objs[0].Value, &obj); err != nil {
+			return fmt.Errorf("RiakSSLKey gatherKeys unable to unmarshal object into tc.DeliveryServiceSSLKeys: %w", err)
+		}
+		tbl.Records[i] = riakSSLKeyRecord{
+			DeliveryServiceSSLKeys: obj,
+		}
+	}
+	return nil
+}
+func (tbl *riakSSLKeyTable) toGeneric() []SSLKey {
+	keys := make([]SSLKey, len(tbl.Records))
+
+	for i, record := range tbl.Records {
+		keys[i] = SSLKey{
+			DeliveryServiceSSLKeys: record.DeliveryServiceSSLKeys,
+		}
+	}
+
+	return keys
+}
+func (tbl *riakSSLKeyTable) fromGeneric(keys []SSLKey) {
+	tbl.Records = make([]riakSSLKeyRecord, len(keys))
+
+	for i, record := range keys {
+		tbl.Records[i] = riakSSLKeyRecord{
+			DeliveryServiceSSLKeys: record.DeliveryServiceSSLKeys,
+		}
+	}
+}
+func (tbl *riakSSLKeyTable) insertKeys(cluster *riak.Cluster) error {
+	for _, record := range tbl.Records {
+		objBytes, err := json.Marshal(record.DeliveryServiceSSLKeys)
+		if err != nil {
+			return fmt.Errorf("RiakSSLKey insertKeys failed to unmarshal keys: %w", err)
+		}
+		err = setObject(cluster, makeRiakObject(objBytes, record.DeliveryService+"-"+record.Version.String()), BUCKET_SSL)
+		if err != nil {
+			return fmt.Errorf("RiakSSLKey insertKeys: %w", err)
+		}
+		err = setObject(cluster, makeRiakObject(objBytes, record.DeliveryService+"-latest"), BUCKET_SSL)
+		if err != nil {
+			return fmt.Errorf("RiakSSLKey insertKeys: %w", err)
+		}
+	}
+	return nil
+}
+func (tbl *riakSSLKeyTable) validate() []string {
+	errs := []string{}
+	for i, record := range tbl.Records {
+		if record.DeliveryService == "" {
+			errs = append(errs, fmt.Sprintf("SSL Key #%v: Delivery Service is blank!", i))
+		}
+		if record.CDN == "" {
+			errs = append(errs, fmt.Sprintf("SSL Key #%v: CDN is blank!", i))
+		}
+		if record.Version.String() == "" {
+			errs = append(errs, fmt.Sprintf("SSL Key #%v: Version is blank!", i))

Review comment:
       nit: type-specific formatting instead of %v

##########
File path: tools/traffic_vault_migrate/riak.go
##########
@@ -0,0 +1,746 @@
+package main
+
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+import (
+	"crypto/tls"
+	"encoding/json"
+	"errors"
+	"fmt"
+	"time"
+
+	"github.com/apache/trafficcontrol/lib/go-rfc"
+	"github.com/apache/trafficcontrol/lib/go-tc"
+
+	"github.com/basho/riak-go-client"
+)
+
+const (
+	BUCKET_SSL     = "ssl"
+	BUCKET_DNSSEC  = "dnssec"
+	BUCKET_URL_SIG = "url_sig_keys"
+	BUCKET_URI_SIG = "cdn_uri_sig_keys"
+
+	INDEX_SSL = "sslkeys"
+
+	SCHEMA_RIAK_KEY    = "_yz_rk"
+	SCHEMA_RIAK_BUCKET = "_yz_rb"
+)
+
+var (
+	SCHEMA_SSL_FIELDS = [...]string{SCHEMA_RIAK_KEY, SCHEMA_RIAK_BUCKET}
+)
+
+// RiakConfig  represents the configuration options available to the Riak backend
+type RiakConfig struct {
+	Host          string `json:"host"`
+	Port          string `json:"port"`
+	User          string `json:"user"`
+	Password      string `json:"password"`
+	Insecure      bool   `json:"insecure"`
+	TLSVersionRaw string `json:"tlsVersion"`
+	TLSVersion    uint16
+}
+
+// RiakBackend is the Riak implementation of TVBackend
+type RiakBackend struct {
+	sslKeys     riakSSLKeyTable
+	dnssecKeys  riakDNSSecKeyTable
+	uriSignKeys riakURISignKeyTable
+	urlSigKeys  riakURLSigKeyTable
+	cfg         RiakConfig
+	cluster     *riak.Cluster
+}
+
+// String returns a high level overview of the backend and its keys
+func (rb *RiakBackend) String() string {
+	data := fmt.Sprintf("Riak server %v@%v:%v\n", rb.cfg.User, rb.cfg.Host, rb.cfg.Port)
+	data += fmt.Sprintf("\tSSL Keys: %v\n", len(rb.sslKeys.Records))
+	data += fmt.Sprintf("\tDNSSec Keys: %v\n", len(rb.dnssecKeys.Records))
+	data += fmt.Sprintf("\tURI Keys: %v\n", len(rb.uriSignKeys.Records))
+	data += fmt.Sprintf("\tURL Keys: %v\n", len(rb.urlSigKeys.Records))
+	return data
+}
+
+// Name returns the name for this backend
+func (rb *RiakBackend) Name() string {
+	return "Riak"
+}
+
+// ReadConfig takes in a filename and will read it into the backends config
+func (rb *RiakBackend) ReadConfig(s string) error {
+	err := UnmarshalConfig(s, &rb.cfg)
+	if err != nil {
+		return err
+	}
+
+	if rb.cfg.TLSVersionRaw == "10" {
+		rb.cfg.TLSVersion = tls.VersionTLS10
+	} else if rb.cfg.TLSVersionRaw == "11" {
+		rb.cfg.TLSVersion = tls.VersionTLS11
+	} else if rb.cfg.TLSVersionRaw == "12" {
+		rb.cfg.TLSVersion = tls.VersionTLS12
+	} else if rb.cfg.TLSVersionRaw == "13" {
+		rb.cfg.TLSVersion = tls.VersionTLS13
+	}
+	return nil
+}
+
+// Insert takes the current keys and inserts them into the backend DB
+func (rb *RiakBackend) Insert() error {
+	if err := rb.sslKeys.insertKeys(rb.cluster); err != nil {
+		return err
+	}
+	if err := rb.dnssecKeys.insertKeys(rb.cluster); err != nil {
+		return err
+	}
+	if err := rb.urlSigKeys.insertKeys(rb.cluster); err != nil {
+		return err
+	}
+	if err := rb.uriSignKeys.insertKeys(rb.cluster); err != nil {
+		return err
+	}
+	return nil
+}
+
+// ValidateKey validates that the keys are valid (in most cases, certain fields are not null)
+func (rb *RiakBackend) ValidateKey() []string {
+	errors := []string{}
+	if errs := rb.sslKeys.validate(); errs != nil {
+		errors = append(errors, errs...)
+	}
+	if errs := rb.dnssecKeys.validate(); errs != nil {
+		errors = append(errors, errs...)
+	}
+	if errs := rb.uriSignKeys.validate(); errs != nil {
+		errors = append(errors, errs...)
+	}
+	if errs := rb.urlSigKeys.validate(); errs != nil {
+		errors = append(errors, errs...)
+	}
+
+	return errors
+}
+
+// SetSSLKeys takes in keys and converts & encrypts the data into the backends internal format
+func (rb *RiakBackend) SetSSLKeys(keys []SSLKey) error {
+	rb.sslKeys.fromGeneric(keys)
+	return nil
+}
+
+// SetDNSSecKeys takes in keys and converts & encrypts the data into the backends internal format
+func (rb *RiakBackend) SetDNSSecKeys(keys []DNSSecKey) error {
+	rb.dnssecKeys.fromGeneric(keys)
+	return nil
+}
+
+// SetURISignKeys takes in keys and converts & encrypts the data into the backends internal format
+func (rb *RiakBackend) SetURISignKeys(keys []URISignKey) error {
+	rb.uriSignKeys.fromGeneric(keys)
+	return nil
+}
+
+// SetURLSigKeys takes in keys and converts & encrypts the data into the backends internal format
+func (rb *RiakBackend) SetURLSigKeys(keys []URLSigKey) error {
+	rb.urlSigKeys.fromGeneric(keys)
+	return nil
+}
+
+// Start initiates the connection to the backend DB
+func (rb *RiakBackend) Start() error {
+	tlsConfig := &tls.Config{
+		InsecureSkipVerify: !rb.cfg.Insecure,
+		MaxVersion:         rb.cfg.TLSVersion,
+	}
+	auth := &riak.AuthOptions{
+		User:      rb.cfg.User,
+		Password:  rb.cfg.Password,
+		TlsConfig: tlsConfig,
+	}
+
+	cluster, err := getRiakCluster(rb.cfg, auth)
+	if err != nil {
+		return err
+	}
+	if err := cluster.Start(); err != nil {
+		return fmt.Errorf("unable to start riak cluster: %w", err)
+	}
+
+	rb.cluster = cluster
+	rb.sslKeys = riakSSLKeyTable{}
+	rb.dnssecKeys = riakDNSSecKeyTable{}
+	rb.urlSigKeys = riakURLSigKeyTable{}
+	rb.uriSignKeys = riakURISignKeyTable{}
+	return nil
+}
+
+// Close terminates the connection to the backend DB
+func (rb *RiakBackend) Close() error {
+	if err := rb.cluster.Stop(); err != nil {
+		return err
+	}
+	return nil
+}
+
+// Ping checks the connection to the backend DB
+func (rb *RiakBackend) Ping() error {
+	return ping(rb.cluster)
+}
+
+// GetSSLKeys converts the backends internal key representation into the common representation (SSLKey)
+func (rb *RiakBackend) GetSSLKeys() ([]SSLKey, error) {
+	return rb.sslKeys.toGeneric(), nil
+}
+
+// GetDNSSecKeys converts the backends internal key representation into the common representation (DNSSecKey)
+func (rb *RiakBackend) GetDNSSecKeys() ([]DNSSecKey, error) {
+	return rb.dnssecKeys.toGeneric(), nil
+}
+
+// GetURISignKeys converts the pg internal key representation into the common representation (URISignKey)
+func (rb *RiakBackend) GetURISignKeys() ([]URISignKey, error) {
+	return rb.uriSignKeys.toGeneric(), nil
+}
+
+// GetURLSigKeys converts the backends internal key representation into the common representation (URLSigKey)
+func (rb *RiakBackend) GetURLSigKeys() ([]URLSigKey, error) {
+	return rb.urlSigKeys.toGeneric(), nil
+}
+
+// Fetch gets all of the keys from the backend DB
+func (rb *RiakBackend) Fetch() error {
+	if err := rb.sslKeys.gatherKeys(rb.cluster); err != nil {
+		return err
+	}
+	if err := rb.dnssecKeys.gatherKeys(rb.cluster); err != nil {
+		return err
+	}
+	if err := rb.urlSigKeys.gatherKeys(rb.cluster); err != nil {
+		return err
+	}
+	if err := rb.uriSignKeys.gatherKeys(rb.cluster); err != nil {
+		return err
+	}
+
+	return nil
+}
+
+type riakSSLKeyRecord struct {
+	tc.DeliveryServiceSSLKeys
+}
+type riakSSLKeyTable struct {
+	Records []riakSSLKeyRecord
+}
+
+func (tbl *riakSSLKeyTable) gatherKeys(cluster *riak.Cluster) error {
+	searchDocs, err := search(cluster, INDEX_SSL, "cdn:*", fmt.Sprintf("%v:*latest", SCHEMA_RIAK_KEY), 1000, SCHEMA_SSL_FIELDS[:])
+	if err != nil {
+		return fmt.Errorf("RiakSSLKey gatherKeys: %w", err)
+	}
+
+	tbl.Records = make([]riakSSLKeyRecord, len(searchDocs))
+	for i, doc := range searchDocs {
+		objs, err := getObject(cluster, doc.Bucket, doc.Key)
+		if err != nil {
+			return err
+		}
+		if len(objs) < 1 {
+			return fmt.Errorf("RiakSSLKey gatherKeys unable to find any objects with key %v and bucket %v, but search results were returned", doc.Key, doc.Bucket)
+		}
+		if len(objs) > 1 {
+			return fmt.Errorf("RiakSSLKey gatherKeys more than 1 ssl key record found %v\n", len(objs))
+		}
+		var obj tc.DeliveryServiceSSLKeys
+		if err = json.Unmarshal(objs[0].Value, &obj); err != nil {
+			return fmt.Errorf("RiakSSLKey gatherKeys unable to unmarshal object into tc.DeliveryServiceSSLKeys: %w", err)
+		}
+		tbl.Records[i] = riakSSLKeyRecord{
+			DeliveryServiceSSLKeys: obj,
+		}
+	}
+	return nil
+}
+func (tbl *riakSSLKeyTable) toGeneric() []SSLKey {
+	keys := make([]SSLKey, len(tbl.Records))
+
+	for i, record := range tbl.Records {
+		keys[i] = SSLKey{
+			DeliveryServiceSSLKeys: record.DeliveryServiceSSLKeys,
+		}
+	}
+
+	return keys
+}
+func (tbl *riakSSLKeyTable) fromGeneric(keys []SSLKey) {
+	tbl.Records = make([]riakSSLKeyRecord, len(keys))
+
+	for i, record := range keys {
+		tbl.Records[i] = riakSSLKeyRecord{
+			DeliveryServiceSSLKeys: record.DeliveryServiceSSLKeys,
+		}
+	}
+}
+func (tbl *riakSSLKeyTable) insertKeys(cluster *riak.Cluster) error {
+	for _, record := range tbl.Records {
+		objBytes, err := json.Marshal(record.DeliveryServiceSSLKeys)
+		if err != nil {
+			return fmt.Errorf("RiakSSLKey insertKeys failed to unmarshal keys: %w", err)

Review comment:
       ~un~marshal, also, these error messages should probably contain the key (xmlid) being inserted. Otherwise, troubleshooting these errors might be a pain.

##########
File path: tools/traffic_vault_migrate/riak.go
##########
@@ -0,0 +1,746 @@
+package main
+
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+import (
+	"crypto/tls"
+	"encoding/json"
+	"errors"
+	"fmt"
+	"time"
+
+	"github.com/apache/trafficcontrol/lib/go-rfc"
+	"github.com/apache/trafficcontrol/lib/go-tc"
+
+	"github.com/basho/riak-go-client"
+)
+
+const (
+	BUCKET_SSL     = "ssl"
+	BUCKET_DNSSEC  = "dnssec"
+	BUCKET_URL_SIG = "url_sig_keys"
+	BUCKET_URI_SIG = "cdn_uri_sig_keys"
+
+	INDEX_SSL = "sslkeys"
+
+	SCHEMA_RIAK_KEY    = "_yz_rk"
+	SCHEMA_RIAK_BUCKET = "_yz_rb"
+)
+
+var (
+	SCHEMA_SSL_FIELDS = [...]string{SCHEMA_RIAK_KEY, SCHEMA_RIAK_BUCKET}
+)
+
+// RiakConfig  represents the configuration options available to the Riak backend
+type RiakConfig struct {
+	Host          string `json:"host"`
+	Port          string `json:"port"`
+	User          string `json:"user"`
+	Password      string `json:"password"`
+	Insecure      bool   `json:"insecure"`
+	TLSVersionRaw string `json:"tlsVersion"`
+	TLSVersion    uint16
+}
+
+// RiakBackend is the Riak implementation of TVBackend
+type RiakBackend struct {
+	sslKeys     riakSSLKeyTable
+	dnssecKeys  riakDNSSecKeyTable
+	uriSignKeys riakURISignKeyTable
+	urlSigKeys  riakURLSigKeyTable
+	cfg         RiakConfig
+	cluster     *riak.Cluster
+}
+
+// String returns a high level overview of the backend and its keys
+func (rb *RiakBackend) String() string {
+	data := fmt.Sprintf("Riak server %v@%v:%v\n", rb.cfg.User, rb.cfg.Host, rb.cfg.Port)
+	data += fmt.Sprintf("\tSSL Keys: %v\n", len(rb.sslKeys.Records))
+	data += fmt.Sprintf("\tDNSSec Keys: %v\n", len(rb.dnssecKeys.Records))
+	data += fmt.Sprintf("\tURI Keys: %v\n", len(rb.uriSignKeys.Records))
+	data += fmt.Sprintf("\tURL Keys: %v\n", len(rb.urlSigKeys.Records))
+	return data
+}
+
+// Name returns the name for this backend
+func (rb *RiakBackend) Name() string {
+	return "Riak"
+}
+
+// ReadConfig takes in a filename and will read it into the backends config
+func (rb *RiakBackend) ReadConfig(s string) error {
+	err := UnmarshalConfig(s, &rb.cfg)
+	if err != nil {
+		return err
+	}
+
+	if rb.cfg.TLSVersionRaw == "10" {
+		rb.cfg.TLSVersion = tls.VersionTLS10
+	} else if rb.cfg.TLSVersionRaw == "11" {
+		rb.cfg.TLSVersion = tls.VersionTLS11
+	} else if rb.cfg.TLSVersionRaw == "12" {
+		rb.cfg.TLSVersion = tls.VersionTLS12
+	} else if rb.cfg.TLSVersionRaw == "13" {
+		rb.cfg.TLSVersion = tls.VersionTLS13
+	}
+	return nil
+}
+
+// Insert takes the current keys and inserts them into the backend DB
+func (rb *RiakBackend) Insert() error {
+	if err := rb.sslKeys.insertKeys(rb.cluster); err != nil {
+		return err
+	}
+	if err := rb.dnssecKeys.insertKeys(rb.cluster); err != nil {
+		return err
+	}
+	if err := rb.urlSigKeys.insertKeys(rb.cluster); err != nil {
+		return err
+	}
+	if err := rb.uriSignKeys.insertKeys(rb.cluster); err != nil {
+		return err
+	}
+	return nil
+}
+
+// ValidateKey validates that the keys are valid (in most cases, certain fields are not null)
+func (rb *RiakBackend) ValidateKey() []string {
+	errors := []string{}
+	if errs := rb.sslKeys.validate(); errs != nil {
+		errors = append(errors, errs...)
+	}
+	if errs := rb.dnssecKeys.validate(); errs != nil {
+		errors = append(errors, errs...)
+	}
+	if errs := rb.uriSignKeys.validate(); errs != nil {
+		errors = append(errors, errs...)
+	}
+	if errs := rb.urlSigKeys.validate(); errs != nil {
+		errors = append(errors, errs...)
+	}
+
+	return errors
+}
+
+// SetSSLKeys takes in keys and converts & encrypts the data into the backends internal format
+func (rb *RiakBackend) SetSSLKeys(keys []SSLKey) error {
+	rb.sslKeys.fromGeneric(keys)
+	return nil
+}
+
+// SetDNSSecKeys takes in keys and converts & encrypts the data into the backends internal format
+func (rb *RiakBackend) SetDNSSecKeys(keys []DNSSecKey) error {
+	rb.dnssecKeys.fromGeneric(keys)
+	return nil
+}
+
+// SetURISignKeys takes in keys and converts & encrypts the data into the backends internal format
+func (rb *RiakBackend) SetURISignKeys(keys []URISignKey) error {
+	rb.uriSignKeys.fromGeneric(keys)
+	return nil
+}
+
+// SetURLSigKeys takes in keys and converts & encrypts the data into the backends internal format
+func (rb *RiakBackend) SetURLSigKeys(keys []URLSigKey) error {
+	rb.urlSigKeys.fromGeneric(keys)
+	return nil
+}
+
+// Start initiates the connection to the backend DB
+func (rb *RiakBackend) Start() error {
+	tlsConfig := &tls.Config{
+		InsecureSkipVerify: !rb.cfg.Insecure,
+		MaxVersion:         rb.cfg.TLSVersion,
+	}
+	auth := &riak.AuthOptions{
+		User:      rb.cfg.User,
+		Password:  rb.cfg.Password,
+		TlsConfig: tlsConfig,
+	}
+
+	cluster, err := getRiakCluster(rb.cfg, auth)
+	if err != nil {
+		return err
+	}
+	if err := cluster.Start(); err != nil {
+		return fmt.Errorf("unable to start riak cluster: %w", err)
+	}
+
+	rb.cluster = cluster
+	rb.sslKeys = riakSSLKeyTable{}
+	rb.dnssecKeys = riakDNSSecKeyTable{}
+	rb.urlSigKeys = riakURLSigKeyTable{}
+	rb.uriSignKeys = riakURISignKeyTable{}
+	return nil
+}
+
+// Close terminates the connection to the backend DB
+func (rb *RiakBackend) Close() error {
+	if err := rb.cluster.Stop(); err != nil {
+		return err
+	}
+	return nil
+}
+
+// Ping checks the connection to the backend DB
+func (rb *RiakBackend) Ping() error {
+	return ping(rb.cluster)
+}
+
+// GetSSLKeys converts the backends internal key representation into the common representation (SSLKey)
+func (rb *RiakBackend) GetSSLKeys() ([]SSLKey, error) {
+	return rb.sslKeys.toGeneric(), nil
+}
+
+// GetDNSSecKeys converts the backends internal key representation into the common representation (DNSSecKey)
+func (rb *RiakBackend) GetDNSSecKeys() ([]DNSSecKey, error) {
+	return rb.dnssecKeys.toGeneric(), nil
+}
+
+// GetURISignKeys converts the pg internal key representation into the common representation (URISignKey)
+func (rb *RiakBackend) GetURISignKeys() ([]URISignKey, error) {
+	return rb.uriSignKeys.toGeneric(), nil
+}
+
+// GetURLSigKeys converts the backends internal key representation into the common representation (URLSigKey)
+func (rb *RiakBackend) GetURLSigKeys() ([]URLSigKey, error) {
+	return rb.urlSigKeys.toGeneric(), nil
+}
+
+// Fetch gets all of the keys from the backend DB
+func (rb *RiakBackend) Fetch() error {
+	if err := rb.sslKeys.gatherKeys(rb.cluster); err != nil {
+		return err
+	}
+	if err := rb.dnssecKeys.gatherKeys(rb.cluster); err != nil {
+		return err
+	}
+	if err := rb.urlSigKeys.gatherKeys(rb.cluster); err != nil {
+		return err
+	}
+	if err := rb.uriSignKeys.gatherKeys(rb.cluster); err != nil {
+		return err
+	}
+
+	return nil
+}
+
+type riakSSLKeyRecord struct {
+	tc.DeliveryServiceSSLKeys
+}
+type riakSSLKeyTable struct {
+	Records []riakSSLKeyRecord
+}
+
+func (tbl *riakSSLKeyTable) gatherKeys(cluster *riak.Cluster) error {
+	searchDocs, err := search(cluster, INDEX_SSL, "cdn:*", fmt.Sprintf("%v:*latest", SCHEMA_RIAK_KEY), 1000, SCHEMA_SSL_FIELDS[:])
+	if err != nil {
+		return fmt.Errorf("RiakSSLKey gatherKeys: %w", err)
+	}
+
+	tbl.Records = make([]riakSSLKeyRecord, len(searchDocs))
+	for i, doc := range searchDocs {
+		objs, err := getObject(cluster, doc.Bucket, doc.Key)
+		if err != nil {
+			return err
+		}
+		if len(objs) < 1 {
+			return fmt.Errorf("RiakSSLKey gatherKeys unable to find any objects with key %v and bucket %v, but search results were returned", doc.Key, doc.Bucket)
+		}
+		if len(objs) > 1 {
+			return fmt.Errorf("RiakSSLKey gatherKeys more than 1 ssl key record found %v\n", len(objs))
+		}
+		var obj tc.DeliveryServiceSSLKeys
+		if err = json.Unmarshal(objs[0].Value, &obj); err != nil {
+			return fmt.Errorf("RiakSSLKey gatherKeys unable to unmarshal object into tc.DeliveryServiceSSLKeys: %w", err)
+		}
+		tbl.Records[i] = riakSSLKeyRecord{
+			DeliveryServiceSSLKeys: obj,
+		}
+	}
+	return nil
+}
+func (tbl *riakSSLKeyTable) toGeneric() []SSLKey {
+	keys := make([]SSLKey, len(tbl.Records))
+
+	for i, record := range tbl.Records {
+		keys[i] = SSLKey{
+			DeliveryServiceSSLKeys: record.DeliveryServiceSSLKeys,
+		}
+	}
+
+	return keys
+}
+func (tbl *riakSSLKeyTable) fromGeneric(keys []SSLKey) {
+	tbl.Records = make([]riakSSLKeyRecord, len(keys))
+
+	for i, record := range keys {
+		tbl.Records[i] = riakSSLKeyRecord{
+			DeliveryServiceSSLKeys: record.DeliveryServiceSSLKeys,
+		}
+	}
+}
+func (tbl *riakSSLKeyTable) insertKeys(cluster *riak.Cluster) error {
+	for _, record := range tbl.Records {
+		objBytes, err := json.Marshal(record.DeliveryServiceSSLKeys)
+		if err != nil {
+			return fmt.Errorf("RiakSSLKey insertKeys failed to unmarshal keys: %w", err)
+		}
+		err = setObject(cluster, makeRiakObject(objBytes, record.DeliveryService+"-"+record.Version.String()), BUCKET_SSL)
+		if err != nil {
+			return fmt.Errorf("RiakSSLKey insertKeys: %w", err)
+		}
+		err = setObject(cluster, makeRiakObject(objBytes, record.DeliveryService+"-latest"), BUCKET_SSL)
+		if err != nil {
+			return fmt.Errorf("RiakSSLKey insertKeys: %w", err)
+		}
+	}
+	return nil
+}
+func (tbl *riakSSLKeyTable) validate() []string {
+	errs := []string{}
+	for i, record := range tbl.Records {
+		if record.DeliveryService == "" {
+			errs = append(errs, fmt.Sprintf("SSL Key #%v: Delivery Service is blank!", i))

Review comment:
       nit: type-specific formatting instead of %v

##########
File path: tools/traffic_vault_migrate/riak.go
##########
@@ -0,0 +1,746 @@
+package main
+
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+import (
+	"crypto/tls"
+	"encoding/json"
+	"errors"
+	"fmt"
+	"time"
+
+	"github.com/apache/trafficcontrol/lib/go-rfc"
+	"github.com/apache/trafficcontrol/lib/go-tc"
+
+	"github.com/basho/riak-go-client"
+)
+
+const (
+	BUCKET_SSL     = "ssl"
+	BUCKET_DNSSEC  = "dnssec"
+	BUCKET_URL_SIG = "url_sig_keys"
+	BUCKET_URI_SIG = "cdn_uri_sig_keys"
+
+	INDEX_SSL = "sslkeys"
+
+	SCHEMA_RIAK_KEY    = "_yz_rk"
+	SCHEMA_RIAK_BUCKET = "_yz_rb"
+)
+
+var (
+	SCHEMA_SSL_FIELDS = [...]string{SCHEMA_RIAK_KEY, SCHEMA_RIAK_BUCKET}
+)
+
+// RiakConfig  represents the configuration options available to the Riak backend
+type RiakConfig struct {
+	Host          string `json:"host"`
+	Port          string `json:"port"`
+	User          string `json:"user"`
+	Password      string `json:"password"`
+	Insecure      bool   `json:"insecure"`
+	TLSVersionRaw string `json:"tlsVersion"`
+	TLSVersion    uint16
+}
+
+// RiakBackend is the Riak implementation of TVBackend
+type RiakBackend struct {
+	sslKeys     riakSSLKeyTable
+	dnssecKeys  riakDNSSecKeyTable
+	uriSignKeys riakURISignKeyTable
+	urlSigKeys  riakURLSigKeyTable
+	cfg         RiakConfig
+	cluster     *riak.Cluster
+}
+
+// String returns a high level overview of the backend and its keys
+func (rb *RiakBackend) String() string {
+	data := fmt.Sprintf("Riak server %v@%v:%v\n", rb.cfg.User, rb.cfg.Host, rb.cfg.Port)
+	data += fmt.Sprintf("\tSSL Keys: %v\n", len(rb.sslKeys.Records))
+	data += fmt.Sprintf("\tDNSSec Keys: %v\n", len(rb.dnssecKeys.Records))
+	data += fmt.Sprintf("\tURI Keys: %v\n", len(rb.uriSignKeys.Records))
+	data += fmt.Sprintf("\tURL Keys: %v\n", len(rb.urlSigKeys.Records))
+	return data
+}
+
+// Name returns the name for this backend
+func (rb *RiakBackend) Name() string {
+	return "Riak"
+}
+
+// ReadConfig takes in a filename and will read it into the backends config
+func (rb *RiakBackend) ReadConfig(s string) error {
+	err := UnmarshalConfig(s, &rb.cfg)
+	if err != nil {
+		return err
+	}
+
+	if rb.cfg.TLSVersionRaw == "10" {
+		rb.cfg.TLSVersion = tls.VersionTLS10
+	} else if rb.cfg.TLSVersionRaw == "11" {
+		rb.cfg.TLSVersion = tls.VersionTLS11
+	} else if rb.cfg.TLSVersionRaw == "12" {
+		rb.cfg.TLSVersion = tls.VersionTLS12
+	} else if rb.cfg.TLSVersionRaw == "13" {
+		rb.cfg.TLSVersion = tls.VersionTLS13
+	}
+	return nil
+}
+
+// Insert takes the current keys and inserts them into the backend DB
+func (rb *RiakBackend) Insert() error {
+	if err := rb.sslKeys.insertKeys(rb.cluster); err != nil {
+		return err
+	}
+	if err := rb.dnssecKeys.insertKeys(rb.cluster); err != nil {
+		return err
+	}
+	if err := rb.urlSigKeys.insertKeys(rb.cluster); err != nil {
+		return err
+	}
+	if err := rb.uriSignKeys.insertKeys(rb.cluster); err != nil {
+		return err
+	}
+	return nil
+}
+
+// ValidateKey validates that the keys are valid (in most cases, certain fields are not null)
+func (rb *RiakBackend) ValidateKey() []string {
+	errors := []string{}
+	if errs := rb.sslKeys.validate(); errs != nil {
+		errors = append(errors, errs...)
+	}
+	if errs := rb.dnssecKeys.validate(); errs != nil {
+		errors = append(errors, errs...)
+	}
+	if errs := rb.uriSignKeys.validate(); errs != nil {
+		errors = append(errors, errs...)
+	}
+	if errs := rb.urlSigKeys.validate(); errs != nil {
+		errors = append(errors, errs...)
+	}
+
+	return errors
+}
+
+// SetSSLKeys takes in keys and converts & encrypts the data into the backends internal format
+func (rb *RiakBackend) SetSSLKeys(keys []SSLKey) error {
+	rb.sslKeys.fromGeneric(keys)
+	return nil
+}
+
+// SetDNSSecKeys takes in keys and converts & encrypts the data into the backends internal format
+func (rb *RiakBackend) SetDNSSecKeys(keys []DNSSecKey) error {
+	rb.dnssecKeys.fromGeneric(keys)
+	return nil
+}
+
+// SetURISignKeys takes in keys and converts & encrypts the data into the backends internal format
+func (rb *RiakBackend) SetURISignKeys(keys []URISignKey) error {
+	rb.uriSignKeys.fromGeneric(keys)
+	return nil
+}
+
+// SetURLSigKeys takes in keys and converts & encrypts the data into the backends internal format
+func (rb *RiakBackend) SetURLSigKeys(keys []URLSigKey) error {
+	rb.urlSigKeys.fromGeneric(keys)
+	return nil
+}
+
+// Start initiates the connection to the backend DB
+func (rb *RiakBackend) Start() error {
+	tlsConfig := &tls.Config{
+		InsecureSkipVerify: !rb.cfg.Insecure,
+		MaxVersion:         rb.cfg.TLSVersion,
+	}
+	auth := &riak.AuthOptions{
+		User:      rb.cfg.User,
+		Password:  rb.cfg.Password,
+		TlsConfig: tlsConfig,
+	}
+
+	cluster, err := getRiakCluster(rb.cfg, auth)
+	if err != nil {
+		return err
+	}
+	if err := cluster.Start(); err != nil {
+		return fmt.Errorf("unable to start riak cluster: %w", err)
+	}
+
+	rb.cluster = cluster
+	rb.sslKeys = riakSSLKeyTable{}
+	rb.dnssecKeys = riakDNSSecKeyTable{}
+	rb.urlSigKeys = riakURLSigKeyTable{}
+	rb.uriSignKeys = riakURISignKeyTable{}
+	return nil
+}
+
+// Close terminates the connection to the backend DB
+func (rb *RiakBackend) Close() error {
+	if err := rb.cluster.Stop(); err != nil {
+		return err
+	}
+	return nil
+}
+
+// Ping checks the connection to the backend DB
+func (rb *RiakBackend) Ping() error {
+	return ping(rb.cluster)
+}
+
+// GetSSLKeys converts the backends internal key representation into the common representation (SSLKey)
+func (rb *RiakBackend) GetSSLKeys() ([]SSLKey, error) {
+	return rb.sslKeys.toGeneric(), nil
+}
+
+// GetDNSSecKeys converts the backends internal key representation into the common representation (DNSSecKey)
+func (rb *RiakBackend) GetDNSSecKeys() ([]DNSSecKey, error) {
+	return rb.dnssecKeys.toGeneric(), nil
+}
+
+// GetURISignKeys converts the pg internal key representation into the common representation (URISignKey)
+func (rb *RiakBackend) GetURISignKeys() ([]URISignKey, error) {
+	return rb.uriSignKeys.toGeneric(), nil
+}
+
+// GetURLSigKeys converts the backends internal key representation into the common representation (URLSigKey)
+func (rb *RiakBackend) GetURLSigKeys() ([]URLSigKey, error) {
+	return rb.urlSigKeys.toGeneric(), nil
+}
+
+// Fetch gets all of the keys from the backend DB
+func (rb *RiakBackend) Fetch() error {
+	if err := rb.sslKeys.gatherKeys(rb.cluster); err != nil {
+		return err
+	}
+	if err := rb.dnssecKeys.gatherKeys(rb.cluster); err != nil {
+		return err
+	}
+	if err := rb.urlSigKeys.gatherKeys(rb.cluster); err != nil {
+		return err
+	}
+	if err := rb.uriSignKeys.gatherKeys(rb.cluster); err != nil {
+		return err
+	}
+
+	return nil
+}
+
+type riakSSLKeyRecord struct {
+	tc.DeliveryServiceSSLKeys
+}
+type riakSSLKeyTable struct {
+	Records []riakSSLKeyRecord
+}
+
+func (tbl *riakSSLKeyTable) gatherKeys(cluster *riak.Cluster) error {
+	searchDocs, err := search(cluster, INDEX_SSL, "cdn:*", fmt.Sprintf("%v:*latest", SCHEMA_RIAK_KEY), 1000, SCHEMA_SSL_FIELDS[:])
+	if err != nil {
+		return fmt.Errorf("RiakSSLKey gatherKeys: %w", err)
+	}
+
+	tbl.Records = make([]riakSSLKeyRecord, len(searchDocs))
+	for i, doc := range searchDocs {
+		objs, err := getObject(cluster, doc.Bucket, doc.Key)
+		if err != nil {
+			return err
+		}
+		if len(objs) < 1 {
+			return fmt.Errorf("RiakSSLKey gatherKeys unable to find any objects with key %v and bucket %v, but search results were returned", doc.Key, doc.Bucket)
+		}
+		if len(objs) > 1 {
+			return fmt.Errorf("RiakSSLKey gatherKeys more than 1 ssl key record found %v\n", len(objs))
+		}
+		var obj tc.DeliveryServiceSSLKeys
+		if err = json.Unmarshal(objs[0].Value, &obj); err != nil {
+			return fmt.Errorf("RiakSSLKey gatherKeys unable to unmarshal object into tc.DeliveryServiceSSLKeys: %w", err)
+		}
+		tbl.Records[i] = riakSSLKeyRecord{
+			DeliveryServiceSSLKeys: obj,
+		}
+	}
+	return nil
+}
+func (tbl *riakSSLKeyTable) toGeneric() []SSLKey {
+	keys := make([]SSLKey, len(tbl.Records))
+
+	for i, record := range tbl.Records {
+		keys[i] = SSLKey{
+			DeliveryServiceSSLKeys: record.DeliveryServiceSSLKeys,
+		}
+	}
+
+	return keys
+}
+func (tbl *riakSSLKeyTable) fromGeneric(keys []SSLKey) {
+	tbl.Records = make([]riakSSLKeyRecord, len(keys))
+
+	for i, record := range keys {
+		tbl.Records[i] = riakSSLKeyRecord{
+			DeliveryServiceSSLKeys: record.DeliveryServiceSSLKeys,
+		}
+	}
+}
+func (tbl *riakSSLKeyTable) insertKeys(cluster *riak.Cluster) error {
+	for _, record := range tbl.Records {
+		objBytes, err := json.Marshal(record.DeliveryServiceSSLKeys)
+		if err != nil {
+			return fmt.Errorf("RiakSSLKey insertKeys failed to unmarshal keys: %w", err)
+		}
+		err = setObject(cluster, makeRiakObject(objBytes, record.DeliveryService+"-"+record.Version.String()), BUCKET_SSL)
+		if err != nil {
+			return fmt.Errorf("RiakSSLKey insertKeys: %w", err)
+		}
+		err = setObject(cluster, makeRiakObject(objBytes, record.DeliveryService+"-latest"), BUCKET_SSL)
+		if err != nil {
+			return fmt.Errorf("RiakSSLKey insertKeys: %w", err)
+		}
+	}
+	return nil
+}
+func (tbl *riakSSLKeyTable) validate() []string {
+	errs := []string{}
+	for i, record := range tbl.Records {
+		if record.DeliveryService == "" {
+			errs = append(errs, fmt.Sprintf("SSL Key #%v: Delivery Service is blank!", i))
+		}
+		if record.CDN == "" {
+			errs = append(errs, fmt.Sprintf("SSL Key #%v: CDN is blank!", i))
+		}
+		if record.Version.String() == "" {
+			errs = append(errs, fmt.Sprintf("SSL Key #%v: Version is blank!", i))
+		}
+	}
+	return errs
+}
+
+type riakDNSSecKeyRecord struct {
+	CDN string
+	Key tc.DNSSECKeysRiak
+}
+type riakDNSSecKeyTable struct {
+	Records []riakDNSSecKeyRecord
+}
+
+func (tbl *riakDNSSecKeyTable) gatherKeys(cluster *riak.Cluster) error {
+	tbl.Records = []riakDNSSecKeyRecord{}
+	objs, err := getObjects(cluster, BUCKET_DNSSEC)
+	if err != nil {
+		return fmt.Errorf("RiakDNSSecKey gatherKeys: %w", err)
+	}
+	for _, obj := range objs {
+		key := tc.DNSSECKeysRiak{}
+		if err := json.Unmarshal(obj.Value, &key); err != nil {
+			return fmt.Errorf("RiakDNSSecKey gatherKeys unable to unmarshal object to tc.DNSSECKeysRiak: %w", err)
+		}
+		tbl.Records = append(tbl.Records, riakDNSSecKeyRecord{
+			CDN: obj.Key,
+			Key: key,
+		})
+	}
+	return nil
+}
+func (tbl *riakDNSSecKeyTable) toGeneric() []DNSSecKey {
+	keys := make([]DNSSecKey, len(tbl.Records))
+
+	for i, record := range tbl.Records {
+		keys[i] = DNSSecKey{
+			CDN:                    record.CDN,
+			DNSSECKeysTrafficVault: tc.DNSSECKeysTrafficVault(record.Key),
+		}
+	}
+
+	return keys
+}
+func (tbl *riakDNSSecKeyTable) fromGeneric(keys []DNSSecKey) {
+	tbl.Records = make([]riakDNSSecKeyRecord, len(keys))
+
+	for i, record := range keys {
+		tbl.Records[i] = riakDNSSecKeyRecord{
+			CDN: record.CDN,
+			Key: tc.DNSSECKeysRiak(record.DNSSECKeysTrafficVault),
+		}
+	}
+}
+func (tbl *riakDNSSecKeyTable) insertKeys(cluster *riak.Cluster) error {
+	for _, record := range tbl.Records {
+		objBytes, err := json.Marshal(record.Key)
+		if err != nil {
+			return fmt.Errorf("RiakDNSSecKey insertKeys error marshalling keys: %w", err)
+		}
+
+		err = setObject(cluster, makeRiakObject(objBytes, record.CDN), BUCKET_DNSSEC)
+		if err != nil {
+			return fmt.Errorf("RiakDNSSecKey insertKeys: %w", err)
+		}
+	}
+	return nil
+}
+func (tbl *riakDNSSecKeyTable) validate() []string {
+	errs := []string{}
+	for i, record := range tbl.Records {
+		if record.CDN == "" {
+			errs = append(errs, fmt.Sprintf("DNSSec Key #%v: CDN is blank!", i))

Review comment:
       nit: type-specific formatting instead of %v

##########
File path: tools/traffic_vault_migrate/riak.go
##########
@@ -0,0 +1,746 @@
+package main
+
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+import (
+	"crypto/tls"
+	"encoding/json"
+	"errors"
+	"fmt"
+	"time"
+
+	"github.com/apache/trafficcontrol/lib/go-rfc"
+	"github.com/apache/trafficcontrol/lib/go-tc"
+
+	"github.com/basho/riak-go-client"
+)
+
+const (
+	BUCKET_SSL     = "ssl"
+	BUCKET_DNSSEC  = "dnssec"
+	BUCKET_URL_SIG = "url_sig_keys"
+	BUCKET_URI_SIG = "cdn_uri_sig_keys"
+
+	INDEX_SSL = "sslkeys"
+
+	SCHEMA_RIAK_KEY    = "_yz_rk"
+	SCHEMA_RIAK_BUCKET = "_yz_rb"
+)
+
+var (
+	SCHEMA_SSL_FIELDS = [...]string{SCHEMA_RIAK_KEY, SCHEMA_RIAK_BUCKET}
+)
+
+// RiakConfig  represents the configuration options available to the Riak backend
+type RiakConfig struct {
+	Host          string `json:"host"`
+	Port          string `json:"port"`
+	User          string `json:"user"`
+	Password      string `json:"password"`
+	Insecure      bool   `json:"insecure"`
+	TLSVersionRaw string `json:"tlsVersion"`
+	TLSVersion    uint16
+}
+
+// RiakBackend is the Riak implementation of TVBackend
+type RiakBackend struct {
+	sslKeys     riakSSLKeyTable
+	dnssecKeys  riakDNSSecKeyTable
+	uriSignKeys riakURISignKeyTable
+	urlSigKeys  riakURLSigKeyTable
+	cfg         RiakConfig
+	cluster     *riak.Cluster
+}
+
+// String returns a high level overview of the backend and its keys
+func (rb *RiakBackend) String() string {
+	data := fmt.Sprintf("Riak server %v@%v:%v\n", rb.cfg.User, rb.cfg.Host, rb.cfg.Port)
+	data += fmt.Sprintf("\tSSL Keys: %v\n", len(rb.sslKeys.Records))
+	data += fmt.Sprintf("\tDNSSec Keys: %v\n", len(rb.dnssecKeys.Records))
+	data += fmt.Sprintf("\tURI Keys: %v\n", len(rb.uriSignKeys.Records))
+	data += fmt.Sprintf("\tURL Keys: %v\n", len(rb.urlSigKeys.Records))
+	return data
+}
+
+// Name returns the name for this backend
+func (rb *RiakBackend) Name() string {
+	return "Riak"
+}
+
+// ReadConfig takes in a filename and will read it into the backends config
+func (rb *RiakBackend) ReadConfig(s string) error {
+	err := UnmarshalConfig(s, &rb.cfg)
+	if err != nil {
+		return err
+	}
+
+	if rb.cfg.TLSVersionRaw == "10" {
+		rb.cfg.TLSVersion = tls.VersionTLS10
+	} else if rb.cfg.TLSVersionRaw == "11" {
+		rb.cfg.TLSVersion = tls.VersionTLS11
+	} else if rb.cfg.TLSVersionRaw == "12" {
+		rb.cfg.TLSVersion = tls.VersionTLS12
+	} else if rb.cfg.TLSVersionRaw == "13" {
+		rb.cfg.TLSVersion = tls.VersionTLS13
+	}
+	return nil
+}
+
+// Insert takes the current keys and inserts them into the backend DB
+func (rb *RiakBackend) Insert() error {
+	if err := rb.sslKeys.insertKeys(rb.cluster); err != nil {
+		return err
+	}
+	if err := rb.dnssecKeys.insertKeys(rb.cluster); err != nil {
+		return err
+	}
+	if err := rb.urlSigKeys.insertKeys(rb.cluster); err != nil {
+		return err
+	}
+	if err := rb.uriSignKeys.insertKeys(rb.cluster); err != nil {
+		return err
+	}
+	return nil
+}
+
+// ValidateKey validates that the keys are valid (in most cases, certain fields are not null)
+func (rb *RiakBackend) ValidateKey() []string {
+	errors := []string{}
+	if errs := rb.sslKeys.validate(); errs != nil {
+		errors = append(errors, errs...)
+	}
+	if errs := rb.dnssecKeys.validate(); errs != nil {
+		errors = append(errors, errs...)
+	}
+	if errs := rb.uriSignKeys.validate(); errs != nil {
+		errors = append(errors, errs...)
+	}
+	if errs := rb.urlSigKeys.validate(); errs != nil {
+		errors = append(errors, errs...)
+	}
+
+	return errors
+}
+
+// SetSSLKeys takes in keys and converts & encrypts the data into the backends internal format
+func (rb *RiakBackend) SetSSLKeys(keys []SSLKey) error {
+	rb.sslKeys.fromGeneric(keys)
+	return nil
+}
+
+// SetDNSSecKeys takes in keys and converts & encrypts the data into the backends internal format
+func (rb *RiakBackend) SetDNSSecKeys(keys []DNSSecKey) error {
+	rb.dnssecKeys.fromGeneric(keys)
+	return nil
+}
+
+// SetURISignKeys takes in keys and converts & encrypts the data into the backends internal format
+func (rb *RiakBackend) SetURISignKeys(keys []URISignKey) error {
+	rb.uriSignKeys.fromGeneric(keys)
+	return nil
+}
+
+// SetURLSigKeys takes in keys and converts & encrypts the data into the backends internal format
+func (rb *RiakBackend) SetURLSigKeys(keys []URLSigKey) error {
+	rb.urlSigKeys.fromGeneric(keys)
+	return nil
+}
+
+// Start initiates the connection to the backend DB
+func (rb *RiakBackend) Start() error {
+	tlsConfig := &tls.Config{
+		InsecureSkipVerify: !rb.cfg.Insecure,
+		MaxVersion:         rb.cfg.TLSVersion,
+	}
+	auth := &riak.AuthOptions{
+		User:      rb.cfg.User,
+		Password:  rb.cfg.Password,
+		TlsConfig: tlsConfig,
+	}
+
+	cluster, err := getRiakCluster(rb.cfg, auth)
+	if err != nil {
+		return err
+	}
+	if err := cluster.Start(); err != nil {
+		return fmt.Errorf("unable to start riak cluster: %w", err)
+	}
+
+	rb.cluster = cluster
+	rb.sslKeys = riakSSLKeyTable{}
+	rb.dnssecKeys = riakDNSSecKeyTable{}
+	rb.urlSigKeys = riakURLSigKeyTable{}
+	rb.uriSignKeys = riakURISignKeyTable{}
+	return nil
+}
+
+// Close terminates the connection to the backend DB
+func (rb *RiakBackend) Close() error {
+	if err := rb.cluster.Stop(); err != nil {
+		return err
+	}
+	return nil
+}
+
+// Ping checks the connection to the backend DB
+func (rb *RiakBackend) Ping() error {
+	return ping(rb.cluster)
+}
+
+// GetSSLKeys converts the backends internal key representation into the common representation (SSLKey)
+func (rb *RiakBackend) GetSSLKeys() ([]SSLKey, error) {
+	return rb.sslKeys.toGeneric(), nil
+}
+
+// GetDNSSecKeys converts the backends internal key representation into the common representation (DNSSecKey)
+func (rb *RiakBackend) GetDNSSecKeys() ([]DNSSecKey, error) {
+	return rb.dnssecKeys.toGeneric(), nil
+}
+
+// GetURISignKeys converts the pg internal key representation into the common representation (URISignKey)
+func (rb *RiakBackend) GetURISignKeys() ([]URISignKey, error) {
+	return rb.uriSignKeys.toGeneric(), nil
+}
+
+// GetURLSigKeys converts the backends internal key representation into the common representation (URLSigKey)
+func (rb *RiakBackend) GetURLSigKeys() ([]URLSigKey, error) {
+	return rb.urlSigKeys.toGeneric(), nil
+}
+
+// Fetch gets all of the keys from the backend DB
+func (rb *RiakBackend) Fetch() error {
+	if err := rb.sslKeys.gatherKeys(rb.cluster); err != nil {
+		return err
+	}
+	if err := rb.dnssecKeys.gatherKeys(rb.cluster); err != nil {
+		return err
+	}
+	if err := rb.urlSigKeys.gatherKeys(rb.cluster); err != nil {
+		return err
+	}
+	if err := rb.uriSignKeys.gatherKeys(rb.cluster); err != nil {
+		return err
+	}
+
+	return nil
+}
+
+type riakSSLKeyRecord struct {
+	tc.DeliveryServiceSSLKeys
+}
+type riakSSLKeyTable struct {
+	Records []riakSSLKeyRecord
+}
+
+func (tbl *riakSSLKeyTable) gatherKeys(cluster *riak.Cluster) error {
+	searchDocs, err := search(cluster, INDEX_SSL, "cdn:*", fmt.Sprintf("%v:*latest", SCHEMA_RIAK_KEY), 1000, SCHEMA_SSL_FIELDS[:])
+	if err != nil {
+		return fmt.Errorf("RiakSSLKey gatherKeys: %w", err)
+	}
+
+	tbl.Records = make([]riakSSLKeyRecord, len(searchDocs))
+	for i, doc := range searchDocs {
+		objs, err := getObject(cluster, doc.Bucket, doc.Key)
+		if err != nil {
+			return err
+		}
+		if len(objs) < 1 {
+			return fmt.Errorf("RiakSSLKey gatherKeys unable to find any objects with key %v and bucket %v, but search results were returned", doc.Key, doc.Bucket)
+		}
+		if len(objs) > 1 {
+			return fmt.Errorf("RiakSSLKey gatherKeys more than 1 ssl key record found %v\n", len(objs))
+		}
+		var obj tc.DeliveryServiceSSLKeys
+		if err = json.Unmarshal(objs[0].Value, &obj); err != nil {
+			return fmt.Errorf("RiakSSLKey gatherKeys unable to unmarshal object into tc.DeliveryServiceSSLKeys: %w", err)

Review comment:
       this error message should probably include the key

##########
File path: tools/traffic_vault_migrate/riak.go
##########
@@ -0,0 +1,746 @@
+package main
+
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+import (
+	"crypto/tls"
+	"encoding/json"
+	"errors"
+	"fmt"
+	"time"
+
+	"github.com/apache/trafficcontrol/lib/go-rfc"
+	"github.com/apache/trafficcontrol/lib/go-tc"
+
+	"github.com/basho/riak-go-client"
+)
+
+const (
+	BUCKET_SSL     = "ssl"
+	BUCKET_DNSSEC  = "dnssec"
+	BUCKET_URL_SIG = "url_sig_keys"
+	BUCKET_URI_SIG = "cdn_uri_sig_keys"
+
+	INDEX_SSL = "sslkeys"
+
+	SCHEMA_RIAK_KEY    = "_yz_rk"
+	SCHEMA_RIAK_BUCKET = "_yz_rb"
+)
+
+var (
+	SCHEMA_SSL_FIELDS = [...]string{SCHEMA_RIAK_KEY, SCHEMA_RIAK_BUCKET}
+)
+
+// RiakConfig  represents the configuration options available to the Riak backend
+type RiakConfig struct {
+	Host          string `json:"host"`
+	Port          string `json:"port"`
+	User          string `json:"user"`
+	Password      string `json:"password"`
+	Insecure      bool   `json:"insecure"`
+	TLSVersionRaw string `json:"tlsVersion"`
+	TLSVersion    uint16
+}
+
+// RiakBackend is the Riak implementation of TVBackend
+type RiakBackend struct {
+	sslKeys     riakSSLKeyTable
+	dnssecKeys  riakDNSSecKeyTable
+	uriSignKeys riakURISignKeyTable
+	urlSigKeys  riakURLSigKeyTable
+	cfg         RiakConfig
+	cluster     *riak.Cluster
+}
+
+// String returns a high level overview of the backend and its keys
+func (rb *RiakBackend) String() string {
+	data := fmt.Sprintf("Riak server %v@%v:%v\n", rb.cfg.User, rb.cfg.Host, rb.cfg.Port)
+	data += fmt.Sprintf("\tSSL Keys: %v\n", len(rb.sslKeys.Records))
+	data += fmt.Sprintf("\tDNSSec Keys: %v\n", len(rb.dnssecKeys.Records))
+	data += fmt.Sprintf("\tURI Keys: %v\n", len(rb.uriSignKeys.Records))
+	data += fmt.Sprintf("\tURL Keys: %v\n", len(rb.urlSigKeys.Records))
+	return data
+}
+
+// Name returns the name for this backend
+func (rb *RiakBackend) Name() string {
+	return "Riak"
+}
+
+// ReadConfig takes in a filename and will read it into the backends config
+func (rb *RiakBackend) ReadConfig(s string) error {
+	err := UnmarshalConfig(s, &rb.cfg)
+	if err != nil {
+		return err
+	}
+
+	if rb.cfg.TLSVersionRaw == "10" {
+		rb.cfg.TLSVersion = tls.VersionTLS10
+	} else if rb.cfg.TLSVersionRaw == "11" {
+		rb.cfg.TLSVersion = tls.VersionTLS11
+	} else if rb.cfg.TLSVersionRaw == "12" {
+		rb.cfg.TLSVersion = tls.VersionTLS12
+	} else if rb.cfg.TLSVersionRaw == "13" {
+		rb.cfg.TLSVersion = tls.VersionTLS13
+	}
+	return nil
+}
+
+// Insert takes the current keys and inserts them into the backend DB
+func (rb *RiakBackend) Insert() error {
+	if err := rb.sslKeys.insertKeys(rb.cluster); err != nil {
+		return err
+	}
+	if err := rb.dnssecKeys.insertKeys(rb.cluster); err != nil {
+		return err
+	}
+	if err := rb.urlSigKeys.insertKeys(rb.cluster); err != nil {
+		return err
+	}
+	if err := rb.uriSignKeys.insertKeys(rb.cluster); err != nil {
+		return err
+	}
+	return nil
+}
+
+// ValidateKey validates that the keys are valid (in most cases, certain fields are not null)
+func (rb *RiakBackend) ValidateKey() []string {
+	errors := []string{}
+	if errs := rb.sslKeys.validate(); errs != nil {
+		errors = append(errors, errs...)
+	}
+	if errs := rb.dnssecKeys.validate(); errs != nil {
+		errors = append(errors, errs...)
+	}
+	if errs := rb.uriSignKeys.validate(); errs != nil {
+		errors = append(errors, errs...)
+	}
+	if errs := rb.urlSigKeys.validate(); errs != nil {
+		errors = append(errors, errs...)
+	}
+
+	return errors
+}
+
+// SetSSLKeys takes in keys and converts & encrypts the data into the backends internal format
+func (rb *RiakBackend) SetSSLKeys(keys []SSLKey) error {
+	rb.sslKeys.fromGeneric(keys)
+	return nil
+}
+
+// SetDNSSecKeys takes in keys and converts & encrypts the data into the backends internal format
+func (rb *RiakBackend) SetDNSSecKeys(keys []DNSSecKey) error {
+	rb.dnssecKeys.fromGeneric(keys)
+	return nil
+}
+
+// SetURISignKeys takes in keys and converts & encrypts the data into the backends internal format
+func (rb *RiakBackend) SetURISignKeys(keys []URISignKey) error {
+	rb.uriSignKeys.fromGeneric(keys)
+	return nil
+}
+
+// SetURLSigKeys takes in keys and converts & encrypts the data into the backends internal format
+func (rb *RiakBackend) SetURLSigKeys(keys []URLSigKey) error {
+	rb.urlSigKeys.fromGeneric(keys)
+	return nil
+}
+
+// Start initiates the connection to the backend DB
+func (rb *RiakBackend) Start() error {
+	tlsConfig := &tls.Config{
+		InsecureSkipVerify: !rb.cfg.Insecure,
+		MaxVersion:         rb.cfg.TLSVersion,
+	}
+	auth := &riak.AuthOptions{
+		User:      rb.cfg.User,
+		Password:  rb.cfg.Password,
+		TlsConfig: tlsConfig,
+	}
+
+	cluster, err := getRiakCluster(rb.cfg, auth)
+	if err != nil {
+		return err
+	}
+	if err := cluster.Start(); err != nil {
+		return fmt.Errorf("unable to start riak cluster: %w", err)
+	}
+
+	rb.cluster = cluster
+	rb.sslKeys = riakSSLKeyTable{}
+	rb.dnssecKeys = riakDNSSecKeyTable{}
+	rb.urlSigKeys = riakURLSigKeyTable{}
+	rb.uriSignKeys = riakURISignKeyTable{}
+	return nil
+}
+
+// Close terminates the connection to the backend DB
+func (rb *RiakBackend) Close() error {
+	if err := rb.cluster.Stop(); err != nil {
+		return err
+	}
+	return nil
+}
+
+// Ping checks the connection to the backend DB
+func (rb *RiakBackend) Ping() error {
+	return ping(rb.cluster)
+}
+
+// GetSSLKeys converts the backends internal key representation into the common representation (SSLKey)
+func (rb *RiakBackend) GetSSLKeys() ([]SSLKey, error) {
+	return rb.sslKeys.toGeneric(), nil
+}
+
+// GetDNSSecKeys converts the backends internal key representation into the common representation (DNSSecKey)
+func (rb *RiakBackend) GetDNSSecKeys() ([]DNSSecKey, error) {
+	return rb.dnssecKeys.toGeneric(), nil
+}
+
+// GetURISignKeys converts the pg internal key representation into the common representation (URISignKey)
+func (rb *RiakBackend) GetURISignKeys() ([]URISignKey, error) {
+	return rb.uriSignKeys.toGeneric(), nil
+}
+
+// GetURLSigKeys converts the backends internal key representation into the common representation (URLSigKey)
+func (rb *RiakBackend) GetURLSigKeys() ([]URLSigKey, error) {
+	return rb.urlSigKeys.toGeneric(), nil
+}
+
+// Fetch gets all of the keys from the backend DB
+func (rb *RiakBackend) Fetch() error {
+	if err := rb.sslKeys.gatherKeys(rb.cluster); err != nil {
+		return err
+	}
+	if err := rb.dnssecKeys.gatherKeys(rb.cluster); err != nil {
+		return err
+	}
+	if err := rb.urlSigKeys.gatherKeys(rb.cluster); err != nil {
+		return err
+	}
+	if err := rb.uriSignKeys.gatherKeys(rb.cluster); err != nil {
+		return err
+	}
+
+	return nil
+}
+
+type riakSSLKeyRecord struct {
+	tc.DeliveryServiceSSLKeys
+}
+type riakSSLKeyTable struct {
+	Records []riakSSLKeyRecord
+}
+
+func (tbl *riakSSLKeyTable) gatherKeys(cluster *riak.Cluster) error {
+	searchDocs, err := search(cluster, INDEX_SSL, "cdn:*", fmt.Sprintf("%v:*latest", SCHEMA_RIAK_KEY), 1000, SCHEMA_SSL_FIELDS[:])
+	if err != nil {
+		return fmt.Errorf("RiakSSLKey gatherKeys: %w", err)
+	}
+
+	tbl.Records = make([]riakSSLKeyRecord, len(searchDocs))
+	for i, doc := range searchDocs {
+		objs, err := getObject(cluster, doc.Bucket, doc.Key)
+		if err != nil {
+			return err
+		}
+		if len(objs) < 1 {
+			return fmt.Errorf("RiakSSLKey gatherKeys unable to find any objects with key %v and bucket %v, but search results were returned", doc.Key, doc.Bucket)
+		}
+		if len(objs) > 1 {
+			return fmt.Errorf("RiakSSLKey gatherKeys more than 1 ssl key record found %v\n", len(objs))
+		}
+		var obj tc.DeliveryServiceSSLKeys
+		if err = json.Unmarshal(objs[0].Value, &obj); err != nil {
+			return fmt.Errorf("RiakSSLKey gatherKeys unable to unmarshal object into tc.DeliveryServiceSSLKeys: %w", err)
+		}
+		tbl.Records[i] = riakSSLKeyRecord{
+			DeliveryServiceSSLKeys: obj,
+		}
+	}
+	return nil
+}
+func (tbl *riakSSLKeyTable) toGeneric() []SSLKey {
+	keys := make([]SSLKey, len(tbl.Records))
+
+	for i, record := range tbl.Records {
+		keys[i] = SSLKey{
+			DeliveryServiceSSLKeys: record.DeliveryServiceSSLKeys,
+		}
+	}
+
+	return keys
+}
+func (tbl *riakSSLKeyTable) fromGeneric(keys []SSLKey) {
+	tbl.Records = make([]riakSSLKeyRecord, len(keys))
+
+	for i, record := range keys {
+		tbl.Records[i] = riakSSLKeyRecord{
+			DeliveryServiceSSLKeys: record.DeliveryServiceSSLKeys,
+		}
+	}
+}
+func (tbl *riakSSLKeyTable) insertKeys(cluster *riak.Cluster) error {
+	for _, record := range tbl.Records {
+		objBytes, err := json.Marshal(record.DeliveryServiceSSLKeys)
+		if err != nil {
+			return fmt.Errorf("RiakSSLKey insertKeys failed to unmarshal keys: %w", err)
+		}
+		err = setObject(cluster, makeRiakObject(objBytes, record.DeliveryService+"-"+record.Version.String()), BUCKET_SSL)
+		if err != nil {
+			return fmt.Errorf("RiakSSLKey insertKeys: %w", err)
+		}
+		err = setObject(cluster, makeRiakObject(objBytes, record.DeliveryService+"-latest"), BUCKET_SSL)
+		if err != nil {
+			return fmt.Errorf("RiakSSLKey insertKeys: %w", err)
+		}
+	}
+	return nil
+}
+func (tbl *riakSSLKeyTable) validate() []string {
+	errs := []string{}
+	for i, record := range tbl.Records {
+		if record.DeliveryService == "" {
+			errs = append(errs, fmt.Sprintf("SSL Key #%v: Delivery Service is blank!", i))
+		}
+		if record.CDN == "" {
+			errs = append(errs, fmt.Sprintf("SSL Key #%v: CDN is blank!", i))
+		}
+		if record.Version.String() == "" {
+			errs = append(errs, fmt.Sprintf("SSL Key #%v: Version is blank!", i))
+		}
+	}
+	return errs
+}
+
+type riakDNSSecKeyRecord struct {
+	CDN string
+	Key tc.DNSSECKeysRiak
+}
+type riakDNSSecKeyTable struct {
+	Records []riakDNSSecKeyRecord
+}
+
+func (tbl *riakDNSSecKeyTable) gatherKeys(cluster *riak.Cluster) error {
+	tbl.Records = []riakDNSSecKeyRecord{}
+	objs, err := getObjects(cluster, BUCKET_DNSSEC)
+	if err != nil {
+		return fmt.Errorf("RiakDNSSecKey gatherKeys: %w", err)
+	}
+	for _, obj := range objs {
+		key := tc.DNSSECKeysRiak{}
+		if err := json.Unmarshal(obj.Value, &key); err != nil {
+			return fmt.Errorf("RiakDNSSecKey gatherKeys unable to unmarshal object to tc.DNSSECKeysRiak: %w", err)
+		}
+		tbl.Records = append(tbl.Records, riakDNSSecKeyRecord{
+			CDN: obj.Key,
+			Key: key,
+		})
+	}
+	return nil
+}
+func (tbl *riakDNSSecKeyTable) toGeneric() []DNSSecKey {
+	keys := make([]DNSSecKey, len(tbl.Records))
+
+	for i, record := range tbl.Records {
+		keys[i] = DNSSecKey{
+			CDN:                    record.CDN,
+			DNSSECKeysTrafficVault: tc.DNSSECKeysTrafficVault(record.Key),
+		}
+	}
+
+	return keys
+}
+func (tbl *riakDNSSecKeyTable) fromGeneric(keys []DNSSecKey) {
+	tbl.Records = make([]riakDNSSecKeyRecord, len(keys))
+
+	for i, record := range keys {
+		tbl.Records[i] = riakDNSSecKeyRecord{
+			CDN: record.CDN,
+			Key: tc.DNSSECKeysRiak(record.DNSSECKeysTrafficVault),
+		}
+	}
+}
+func (tbl *riakDNSSecKeyTable) insertKeys(cluster *riak.Cluster) error {
+	for _, record := range tbl.Records {
+		objBytes, err := json.Marshal(record.Key)
+		if err != nil {
+			return fmt.Errorf("RiakDNSSecKey insertKeys error marshalling keys: %w", err)
+		}
+
+		err = setObject(cluster, makeRiakObject(objBytes, record.CDN), BUCKET_DNSSEC)
+		if err != nil {
+			return fmt.Errorf("RiakDNSSecKey insertKeys: %w", err)
+		}
+	}
+	return nil
+}
+func (tbl *riakDNSSecKeyTable) validate() []string {
+	errs := []string{}
+	for i, record := range tbl.Records {
+		if record.CDN == "" {
+			errs = append(errs, fmt.Sprintf("DNSSec Key #%v: CDN is blank!", i))
+		}
+	}
+	return errs
+}
+
+type riakURLSigKeyRecord struct {
+	Key             tc.URLSigKeys
+	DeliveryService string
+}
+type riakURLSigKeyTable struct {
+	Records []riakURLSigKeyRecord
+}
+
+func (tbl *riakURLSigKeyTable) gatherKeys(cluster *riak.Cluster) error {
+	tbl.Records = []riakURLSigKeyRecord{}
+	objs, err := getObjects(cluster, BUCKET_URL_SIG)
+	if err != nil {
+		return fmt.Errorf("RiakURLSigKey gatherKeys: %w", err)
+	}
+	for _, obj := range objs {
+		key := tc.URLSigKeys{}
+		if err := json.Unmarshal(obj.Value, &key); err != nil {
+			return fmt.Errorf("RiakURLSigKey gatherKeys unable to unamrshal object into tc.URLSigKeys: %w", err)
+		}
+		tbl.Records = append(tbl.Records, riakURLSigKeyRecord{
+			DeliveryService: obj.Key,
+			Key:             key,
+		})
+	}
+	return nil
+}
+func (tbl *riakURLSigKeyTable) toGeneric() []URLSigKey {
+	keys := make([]URLSigKey, len(tbl.Records))
+
+	for i, record := range tbl.Records {
+		keys[i] = URLSigKey{
+			URLSigKeys:      record.Key,
+			DeliveryService: record.DeliveryService,
+		}
+	}
+
+	return keys
+}
+func (tbl *riakURLSigKeyTable) fromGeneric(keys []URLSigKey) {
+	tbl.Records = make([]riakURLSigKeyRecord, len(keys))
+	for i, key := range keys {
+		tbl.Records[i] = riakURLSigKeyRecord{
+			Key:             key.URLSigKeys,
+			DeliveryService: key.DeliveryService,
+		}
+	}
+}
+func (tbl *riakURLSigKeyTable) insertKeys(cluster *riak.Cluster) error {
+	for _, record := range tbl.Records {
+		objBytes, err := json.Marshal(record.Key)
+		if err != nil {
+			return fmt.Errorf("RiakURLSigKey insertKeys unable to marshal keys: %w", err)

Review comment:
       this error message should probably contain the `record.DeliveryService` for troubleshooting

##########
File path: tools/traffic_vault_migrate/riak.go
##########
@@ -0,0 +1,746 @@
+package main
+
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+import (
+	"crypto/tls"
+	"encoding/json"
+	"errors"
+	"fmt"
+	"time"
+
+	"github.com/apache/trafficcontrol/lib/go-rfc"
+	"github.com/apache/trafficcontrol/lib/go-tc"
+
+	"github.com/basho/riak-go-client"
+)
+
+const (
+	BUCKET_SSL     = "ssl"
+	BUCKET_DNSSEC  = "dnssec"
+	BUCKET_URL_SIG = "url_sig_keys"
+	BUCKET_URI_SIG = "cdn_uri_sig_keys"
+
+	INDEX_SSL = "sslkeys"
+
+	SCHEMA_RIAK_KEY    = "_yz_rk"
+	SCHEMA_RIAK_BUCKET = "_yz_rb"
+)
+
+var (
+	SCHEMA_SSL_FIELDS = [...]string{SCHEMA_RIAK_KEY, SCHEMA_RIAK_BUCKET}
+)
+
+// RiakConfig  represents the configuration options available to the Riak backend
+type RiakConfig struct {
+	Host          string `json:"host"`
+	Port          string `json:"port"`
+	User          string `json:"user"`
+	Password      string `json:"password"`
+	Insecure      bool   `json:"insecure"`
+	TLSVersionRaw string `json:"tlsVersion"`
+	TLSVersion    uint16
+}
+
+// RiakBackend is the Riak implementation of TVBackend
+type RiakBackend struct {
+	sslKeys     riakSSLKeyTable
+	dnssecKeys  riakDNSSecKeyTable
+	uriSignKeys riakURISignKeyTable
+	urlSigKeys  riakURLSigKeyTable
+	cfg         RiakConfig
+	cluster     *riak.Cluster
+}
+
+// String returns a high level overview of the backend and its keys
+func (rb *RiakBackend) String() string {
+	data := fmt.Sprintf("Riak server %v@%v:%v\n", rb.cfg.User, rb.cfg.Host, rb.cfg.Port)
+	data += fmt.Sprintf("\tSSL Keys: %v\n", len(rb.sslKeys.Records))
+	data += fmt.Sprintf("\tDNSSec Keys: %v\n", len(rb.dnssecKeys.Records))
+	data += fmt.Sprintf("\tURI Keys: %v\n", len(rb.uriSignKeys.Records))
+	data += fmt.Sprintf("\tURL Keys: %v\n", len(rb.urlSigKeys.Records))
+	return data
+}
+
+// Name returns the name for this backend
+func (rb *RiakBackend) Name() string {
+	return "Riak"
+}
+
+// ReadConfig takes in a filename and will read it into the backends config
+func (rb *RiakBackend) ReadConfig(s string) error {
+	err := UnmarshalConfig(s, &rb.cfg)
+	if err != nil {
+		return err
+	}
+
+	if rb.cfg.TLSVersionRaw == "10" {
+		rb.cfg.TLSVersion = tls.VersionTLS10
+	} else if rb.cfg.TLSVersionRaw == "11" {
+		rb.cfg.TLSVersion = tls.VersionTLS11
+	} else if rb.cfg.TLSVersionRaw == "12" {
+		rb.cfg.TLSVersion = tls.VersionTLS12
+	} else if rb.cfg.TLSVersionRaw == "13" {
+		rb.cfg.TLSVersion = tls.VersionTLS13
+	}
+	return nil
+}
+
+// Insert takes the current keys and inserts them into the backend DB
+func (rb *RiakBackend) Insert() error {
+	if err := rb.sslKeys.insertKeys(rb.cluster); err != nil {
+		return err
+	}
+	if err := rb.dnssecKeys.insertKeys(rb.cluster); err != nil {
+		return err
+	}
+	if err := rb.urlSigKeys.insertKeys(rb.cluster); err != nil {
+		return err
+	}
+	if err := rb.uriSignKeys.insertKeys(rb.cluster); err != nil {
+		return err
+	}
+	return nil
+}
+
+// ValidateKey validates that the keys are valid (in most cases, certain fields are not null)
+func (rb *RiakBackend) ValidateKey() []string {
+	errors := []string{}
+	if errs := rb.sslKeys.validate(); errs != nil {
+		errors = append(errors, errs...)
+	}
+	if errs := rb.dnssecKeys.validate(); errs != nil {
+		errors = append(errors, errs...)
+	}
+	if errs := rb.uriSignKeys.validate(); errs != nil {
+		errors = append(errors, errs...)
+	}
+	if errs := rb.urlSigKeys.validate(); errs != nil {
+		errors = append(errors, errs...)
+	}
+
+	return errors
+}
+
+// SetSSLKeys takes in keys and converts & encrypts the data into the backends internal format
+func (rb *RiakBackend) SetSSLKeys(keys []SSLKey) error {
+	rb.sslKeys.fromGeneric(keys)
+	return nil
+}
+
+// SetDNSSecKeys takes in keys and converts & encrypts the data into the backends internal format
+func (rb *RiakBackend) SetDNSSecKeys(keys []DNSSecKey) error {
+	rb.dnssecKeys.fromGeneric(keys)
+	return nil
+}
+
+// SetURISignKeys takes in keys and converts & encrypts the data into the backends internal format
+func (rb *RiakBackend) SetURISignKeys(keys []URISignKey) error {
+	rb.uriSignKeys.fromGeneric(keys)
+	return nil
+}
+
+// SetURLSigKeys takes in keys and converts & encrypts the data into the backends internal format
+func (rb *RiakBackend) SetURLSigKeys(keys []URLSigKey) error {
+	rb.urlSigKeys.fromGeneric(keys)
+	return nil
+}
+
+// Start initiates the connection to the backend DB
+func (rb *RiakBackend) Start() error {
+	tlsConfig := &tls.Config{
+		InsecureSkipVerify: !rb.cfg.Insecure,
+		MaxVersion:         rb.cfg.TLSVersion,
+	}
+	auth := &riak.AuthOptions{
+		User:      rb.cfg.User,
+		Password:  rb.cfg.Password,
+		TlsConfig: tlsConfig,
+	}
+
+	cluster, err := getRiakCluster(rb.cfg, auth)
+	if err != nil {
+		return err
+	}
+	if err := cluster.Start(); err != nil {
+		return fmt.Errorf("unable to start riak cluster: %w", err)
+	}
+
+	rb.cluster = cluster
+	rb.sslKeys = riakSSLKeyTable{}
+	rb.dnssecKeys = riakDNSSecKeyTable{}
+	rb.urlSigKeys = riakURLSigKeyTable{}
+	rb.uriSignKeys = riakURISignKeyTable{}
+	return nil
+}
+
+// Close terminates the connection to the backend DB
+func (rb *RiakBackend) Close() error {
+	if err := rb.cluster.Stop(); err != nil {
+		return err
+	}
+	return nil
+}
+
+// Ping checks the connection to the backend DB
+func (rb *RiakBackend) Ping() error {
+	return ping(rb.cluster)
+}
+
+// GetSSLKeys converts the backends internal key representation into the common representation (SSLKey)
+func (rb *RiakBackend) GetSSLKeys() ([]SSLKey, error) {
+	return rb.sslKeys.toGeneric(), nil
+}
+
+// GetDNSSecKeys converts the backends internal key representation into the common representation (DNSSecKey)
+func (rb *RiakBackend) GetDNSSecKeys() ([]DNSSecKey, error) {
+	return rb.dnssecKeys.toGeneric(), nil
+}
+
+// GetURISignKeys converts the pg internal key representation into the common representation (URISignKey)
+func (rb *RiakBackend) GetURISignKeys() ([]URISignKey, error) {
+	return rb.uriSignKeys.toGeneric(), nil
+}
+
+// GetURLSigKeys converts the backends internal key representation into the common representation (URLSigKey)
+func (rb *RiakBackend) GetURLSigKeys() ([]URLSigKey, error) {
+	return rb.urlSigKeys.toGeneric(), nil
+}
+
+// Fetch gets all of the keys from the backend DB
+func (rb *RiakBackend) Fetch() error {
+	if err := rb.sslKeys.gatherKeys(rb.cluster); err != nil {
+		return err
+	}
+	if err := rb.dnssecKeys.gatherKeys(rb.cluster); err != nil {
+		return err
+	}
+	if err := rb.urlSigKeys.gatherKeys(rb.cluster); err != nil {
+		return err
+	}
+	if err := rb.uriSignKeys.gatherKeys(rb.cluster); err != nil {
+		return err
+	}
+
+	return nil
+}
+
+type riakSSLKeyRecord struct {
+	tc.DeliveryServiceSSLKeys
+}
+type riakSSLKeyTable struct {
+	Records []riakSSLKeyRecord
+}
+
+func (tbl *riakSSLKeyTable) gatherKeys(cluster *riak.Cluster) error {
+	searchDocs, err := search(cluster, INDEX_SSL, "cdn:*", fmt.Sprintf("%v:*latest", SCHEMA_RIAK_KEY), 1000, SCHEMA_SSL_FIELDS[:])
+	if err != nil {
+		return fmt.Errorf("RiakSSLKey gatherKeys: %w", err)
+	}
+
+	tbl.Records = make([]riakSSLKeyRecord, len(searchDocs))
+	for i, doc := range searchDocs {
+		objs, err := getObject(cluster, doc.Bucket, doc.Key)
+		if err != nil {
+			return err
+		}
+		if len(objs) < 1 {
+			return fmt.Errorf("RiakSSLKey gatherKeys unable to find any objects with key %v and bucket %v, but search results were returned", doc.Key, doc.Bucket)
+		}
+		if len(objs) > 1 {
+			return fmt.Errorf("RiakSSLKey gatherKeys more than 1 ssl key record found %v\n", len(objs))
+		}
+		var obj tc.DeliveryServiceSSLKeys
+		if err = json.Unmarshal(objs[0].Value, &obj); err != nil {
+			return fmt.Errorf("RiakSSLKey gatherKeys unable to unmarshal object into tc.DeliveryServiceSSLKeys: %w", err)
+		}
+		tbl.Records[i] = riakSSLKeyRecord{
+			DeliveryServiceSSLKeys: obj,
+		}
+	}
+	return nil
+}
+func (tbl *riakSSLKeyTable) toGeneric() []SSLKey {
+	keys := make([]SSLKey, len(tbl.Records))
+
+	for i, record := range tbl.Records {
+		keys[i] = SSLKey{
+			DeliveryServiceSSLKeys: record.DeliveryServiceSSLKeys,
+		}
+	}
+
+	return keys
+}
+func (tbl *riakSSLKeyTable) fromGeneric(keys []SSLKey) {
+	tbl.Records = make([]riakSSLKeyRecord, len(keys))
+
+	for i, record := range keys {
+		tbl.Records[i] = riakSSLKeyRecord{
+			DeliveryServiceSSLKeys: record.DeliveryServiceSSLKeys,
+		}
+	}
+}
+func (tbl *riakSSLKeyTable) insertKeys(cluster *riak.Cluster) error {
+	for _, record := range tbl.Records {
+		objBytes, err := json.Marshal(record.DeliveryServiceSSLKeys)
+		if err != nil {
+			return fmt.Errorf("RiakSSLKey insertKeys failed to unmarshal keys: %w", err)
+		}
+		err = setObject(cluster, makeRiakObject(objBytes, record.DeliveryService+"-"+record.Version.String()), BUCKET_SSL)
+		if err != nil {
+			return fmt.Errorf("RiakSSLKey insertKeys: %w", err)
+		}
+		err = setObject(cluster, makeRiakObject(objBytes, record.DeliveryService+"-latest"), BUCKET_SSL)
+		if err != nil {
+			return fmt.Errorf("RiakSSLKey insertKeys: %w", err)
+		}
+	}
+	return nil
+}
+func (tbl *riakSSLKeyTable) validate() []string {
+	errs := []string{}
+	for i, record := range tbl.Records {
+		if record.DeliveryService == "" {
+			errs = append(errs, fmt.Sprintf("SSL Key #%v: Delivery Service is blank!", i))
+		}
+		if record.CDN == "" {
+			errs = append(errs, fmt.Sprintf("SSL Key #%v: CDN is blank!", i))
+		}
+		if record.Version.String() == "" {
+			errs = append(errs, fmt.Sprintf("SSL Key #%v: Version is blank!", i))
+		}
+	}
+	return errs
+}
+
+type riakDNSSecKeyRecord struct {
+	CDN string
+	Key tc.DNSSECKeysRiak
+}
+type riakDNSSecKeyTable struct {
+	Records []riakDNSSecKeyRecord
+}
+
+func (tbl *riakDNSSecKeyTable) gatherKeys(cluster *riak.Cluster) error {
+	tbl.Records = []riakDNSSecKeyRecord{}
+	objs, err := getObjects(cluster, BUCKET_DNSSEC)
+	if err != nil {
+		return fmt.Errorf("RiakDNSSecKey gatherKeys: %w", err)
+	}
+	for _, obj := range objs {
+		key := tc.DNSSECKeysRiak{}
+		if err := json.Unmarshal(obj.Value, &key); err != nil {
+			return fmt.Errorf("RiakDNSSecKey gatherKeys unable to unmarshal object to tc.DNSSECKeysRiak: %w", err)
+		}
+		tbl.Records = append(tbl.Records, riakDNSSecKeyRecord{
+			CDN: obj.Key,
+			Key: key,
+		})
+	}
+	return nil
+}
+func (tbl *riakDNSSecKeyTable) toGeneric() []DNSSecKey {
+	keys := make([]DNSSecKey, len(tbl.Records))
+
+	for i, record := range tbl.Records {
+		keys[i] = DNSSecKey{
+			CDN:                    record.CDN,
+			DNSSECKeysTrafficVault: tc.DNSSECKeysTrafficVault(record.Key),
+		}
+	}
+
+	return keys
+}
+func (tbl *riakDNSSecKeyTable) fromGeneric(keys []DNSSecKey) {
+	tbl.Records = make([]riakDNSSecKeyRecord, len(keys))
+
+	for i, record := range keys {
+		tbl.Records[i] = riakDNSSecKeyRecord{
+			CDN: record.CDN,
+			Key: tc.DNSSECKeysRiak(record.DNSSECKeysTrafficVault),
+		}
+	}
+}
+func (tbl *riakDNSSecKeyTable) insertKeys(cluster *riak.Cluster) error {
+	for _, record := range tbl.Records {
+		objBytes, err := json.Marshal(record.Key)
+		if err != nil {
+			return fmt.Errorf("RiakDNSSecKey insertKeys error marshalling keys: %w", err)
+		}
+
+		err = setObject(cluster, makeRiakObject(objBytes, record.CDN), BUCKET_DNSSEC)
+		if err != nil {
+			return fmt.Errorf("RiakDNSSecKey insertKeys: %w", err)
+		}
+	}
+	return nil
+}
+func (tbl *riakDNSSecKeyTable) validate() []string {
+	errs := []string{}
+	for i, record := range tbl.Records {
+		if record.CDN == "" {
+			errs = append(errs, fmt.Sprintf("DNSSec Key #%v: CDN is blank!", i))
+		}
+	}
+	return errs
+}
+
+type riakURLSigKeyRecord struct {
+	Key             tc.URLSigKeys
+	DeliveryService string
+}
+type riakURLSigKeyTable struct {
+	Records []riakURLSigKeyRecord
+}
+
+func (tbl *riakURLSigKeyTable) gatherKeys(cluster *riak.Cluster) error {
+	tbl.Records = []riakURLSigKeyRecord{}
+	objs, err := getObjects(cluster, BUCKET_URL_SIG)
+	if err != nil {
+		return fmt.Errorf("RiakURLSigKey gatherKeys: %w", err)
+	}
+	for _, obj := range objs {
+		key := tc.URLSigKeys{}
+		if err := json.Unmarshal(obj.Value, &key); err != nil {
+			return fmt.Errorf("RiakURLSigKey gatherKeys unable to unamrshal object into tc.URLSigKeys: %w", err)
+		}
+		tbl.Records = append(tbl.Records, riakURLSigKeyRecord{
+			DeliveryService: obj.Key,
+			Key:             key,
+		})
+	}
+	return nil
+}
+func (tbl *riakURLSigKeyTable) toGeneric() []URLSigKey {
+	keys := make([]URLSigKey, len(tbl.Records))
+
+	for i, record := range tbl.Records {
+		keys[i] = URLSigKey{
+			URLSigKeys:      record.Key,
+			DeliveryService: record.DeliveryService,
+		}
+	}
+
+	return keys
+}
+func (tbl *riakURLSigKeyTable) fromGeneric(keys []URLSigKey) {
+	tbl.Records = make([]riakURLSigKeyRecord, len(keys))
+	for i, key := range keys {
+		tbl.Records[i] = riakURLSigKeyRecord{
+			Key:             key.URLSigKeys,
+			DeliveryService: key.DeliveryService,
+		}
+	}
+}
+func (tbl *riakURLSigKeyTable) insertKeys(cluster *riak.Cluster) error {
+	for _, record := range tbl.Records {
+		objBytes, err := json.Marshal(record.Key)
+		if err != nil {
+			return fmt.Errorf("RiakURLSigKey insertKeys unable to marshal keys: %w", err)
+		}
+
+		err = setObject(cluster, makeRiakObject(objBytes, "url_sig_"+record.DeliveryService+".config"), BUCKET_URL_SIG)
+		if err != nil {
+			return fmt.Errorf("RiakURLSigKey insertKeys: %w", err)

Review comment:
       this error message should probably contain the `record.DeliveryService` for troubleshooting

##########
File path: tools/traffic_vault_migrate/riak.go
##########
@@ -0,0 +1,746 @@
+package main
+
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+import (
+	"crypto/tls"
+	"encoding/json"
+	"errors"
+	"fmt"
+	"time"
+
+	"github.com/apache/trafficcontrol/lib/go-rfc"
+	"github.com/apache/trafficcontrol/lib/go-tc"
+
+	"github.com/basho/riak-go-client"
+)
+
+const (
+	BUCKET_SSL     = "ssl"
+	BUCKET_DNSSEC  = "dnssec"
+	BUCKET_URL_SIG = "url_sig_keys"
+	BUCKET_URI_SIG = "cdn_uri_sig_keys"
+
+	INDEX_SSL = "sslkeys"
+
+	SCHEMA_RIAK_KEY    = "_yz_rk"
+	SCHEMA_RIAK_BUCKET = "_yz_rb"
+)
+
+var (
+	SCHEMA_SSL_FIELDS = [...]string{SCHEMA_RIAK_KEY, SCHEMA_RIAK_BUCKET}
+)
+
+// RiakConfig  represents the configuration options available to the Riak backend
+type RiakConfig struct {
+	Host          string `json:"host"`
+	Port          string `json:"port"`
+	User          string `json:"user"`
+	Password      string `json:"password"`
+	Insecure      bool   `json:"insecure"`
+	TLSVersionRaw string `json:"tlsVersion"`
+	TLSVersion    uint16
+}
+
+// RiakBackend is the Riak implementation of TVBackend
+type RiakBackend struct {
+	sslKeys     riakSSLKeyTable
+	dnssecKeys  riakDNSSecKeyTable
+	uriSignKeys riakURISignKeyTable
+	urlSigKeys  riakURLSigKeyTable
+	cfg         RiakConfig
+	cluster     *riak.Cluster
+}
+
+// String returns a high level overview of the backend and its keys
+func (rb *RiakBackend) String() string {
+	data := fmt.Sprintf("Riak server %v@%v:%v\n", rb.cfg.User, rb.cfg.Host, rb.cfg.Port)
+	data += fmt.Sprintf("\tSSL Keys: %v\n", len(rb.sslKeys.Records))
+	data += fmt.Sprintf("\tDNSSec Keys: %v\n", len(rb.dnssecKeys.Records))
+	data += fmt.Sprintf("\tURI Keys: %v\n", len(rb.uriSignKeys.Records))
+	data += fmt.Sprintf("\tURL Keys: %v\n", len(rb.urlSigKeys.Records))
+	return data
+}
+
+// Name returns the name for this backend
+func (rb *RiakBackend) Name() string {
+	return "Riak"
+}
+
+// ReadConfig takes in a filename and will read it into the backends config
+func (rb *RiakBackend) ReadConfig(s string) error {
+	err := UnmarshalConfig(s, &rb.cfg)
+	if err != nil {
+		return err
+	}
+
+	if rb.cfg.TLSVersionRaw == "10" {
+		rb.cfg.TLSVersion = tls.VersionTLS10
+	} else if rb.cfg.TLSVersionRaw == "11" {
+		rb.cfg.TLSVersion = tls.VersionTLS11
+	} else if rb.cfg.TLSVersionRaw == "12" {
+		rb.cfg.TLSVersion = tls.VersionTLS12
+	} else if rb.cfg.TLSVersionRaw == "13" {
+		rb.cfg.TLSVersion = tls.VersionTLS13
+	}
+	return nil
+}
+
+// Insert takes the current keys and inserts them into the backend DB
+func (rb *RiakBackend) Insert() error {
+	if err := rb.sslKeys.insertKeys(rb.cluster); err != nil {
+		return err
+	}
+	if err := rb.dnssecKeys.insertKeys(rb.cluster); err != nil {
+		return err
+	}
+	if err := rb.urlSigKeys.insertKeys(rb.cluster); err != nil {
+		return err
+	}
+	if err := rb.uriSignKeys.insertKeys(rb.cluster); err != nil {
+		return err
+	}
+	return nil
+}
+
+// ValidateKey validates that the keys are valid (in most cases, certain fields are not null)
+func (rb *RiakBackend) ValidateKey() []string {
+	errors := []string{}
+	if errs := rb.sslKeys.validate(); errs != nil {
+		errors = append(errors, errs...)
+	}
+	if errs := rb.dnssecKeys.validate(); errs != nil {
+		errors = append(errors, errs...)
+	}
+	if errs := rb.uriSignKeys.validate(); errs != nil {
+		errors = append(errors, errs...)
+	}
+	if errs := rb.urlSigKeys.validate(); errs != nil {
+		errors = append(errors, errs...)
+	}
+
+	return errors
+}
+
+// SetSSLKeys takes in keys and converts & encrypts the data into the backends internal format
+func (rb *RiakBackend) SetSSLKeys(keys []SSLKey) error {
+	rb.sslKeys.fromGeneric(keys)
+	return nil
+}
+
+// SetDNSSecKeys takes in keys and converts & encrypts the data into the backends internal format
+func (rb *RiakBackend) SetDNSSecKeys(keys []DNSSecKey) error {
+	rb.dnssecKeys.fromGeneric(keys)
+	return nil
+}
+
+// SetURISignKeys takes in keys and converts & encrypts the data into the backends internal format
+func (rb *RiakBackend) SetURISignKeys(keys []URISignKey) error {
+	rb.uriSignKeys.fromGeneric(keys)
+	return nil
+}
+
+// SetURLSigKeys takes in keys and converts & encrypts the data into the backends internal format
+func (rb *RiakBackend) SetURLSigKeys(keys []URLSigKey) error {
+	rb.urlSigKeys.fromGeneric(keys)
+	return nil
+}
+
+// Start initiates the connection to the backend DB
+func (rb *RiakBackend) Start() error {
+	tlsConfig := &tls.Config{
+		InsecureSkipVerify: !rb.cfg.Insecure,
+		MaxVersion:         rb.cfg.TLSVersion,
+	}
+	auth := &riak.AuthOptions{
+		User:      rb.cfg.User,
+		Password:  rb.cfg.Password,
+		TlsConfig: tlsConfig,
+	}
+
+	cluster, err := getRiakCluster(rb.cfg, auth)
+	if err != nil {
+		return err
+	}
+	if err := cluster.Start(); err != nil {
+		return fmt.Errorf("unable to start riak cluster: %w", err)
+	}
+
+	rb.cluster = cluster
+	rb.sslKeys = riakSSLKeyTable{}
+	rb.dnssecKeys = riakDNSSecKeyTable{}
+	rb.urlSigKeys = riakURLSigKeyTable{}
+	rb.uriSignKeys = riakURISignKeyTable{}
+	return nil
+}
+
+// Close terminates the connection to the backend DB
+func (rb *RiakBackend) Close() error {
+	if err := rb.cluster.Stop(); err != nil {
+		return err
+	}
+	return nil
+}
+
+// Ping checks the connection to the backend DB
+func (rb *RiakBackend) Ping() error {
+	return ping(rb.cluster)
+}
+
+// GetSSLKeys converts the backends internal key representation into the common representation (SSLKey)
+func (rb *RiakBackend) GetSSLKeys() ([]SSLKey, error) {
+	return rb.sslKeys.toGeneric(), nil
+}
+
+// GetDNSSecKeys converts the backends internal key representation into the common representation (DNSSecKey)
+func (rb *RiakBackend) GetDNSSecKeys() ([]DNSSecKey, error) {
+	return rb.dnssecKeys.toGeneric(), nil
+}
+
+// GetURISignKeys converts the pg internal key representation into the common representation (URISignKey)
+func (rb *RiakBackend) GetURISignKeys() ([]URISignKey, error) {
+	return rb.uriSignKeys.toGeneric(), nil
+}
+
+// GetURLSigKeys converts the backends internal key representation into the common representation (URLSigKey)
+func (rb *RiakBackend) GetURLSigKeys() ([]URLSigKey, error) {
+	return rb.urlSigKeys.toGeneric(), nil
+}
+
+// Fetch gets all of the keys from the backend DB
+func (rb *RiakBackend) Fetch() error {
+	if err := rb.sslKeys.gatherKeys(rb.cluster); err != nil {
+		return err
+	}
+	if err := rb.dnssecKeys.gatherKeys(rb.cluster); err != nil {
+		return err
+	}
+	if err := rb.urlSigKeys.gatherKeys(rb.cluster); err != nil {
+		return err
+	}
+	if err := rb.uriSignKeys.gatherKeys(rb.cluster); err != nil {
+		return err
+	}
+
+	return nil
+}
+
+type riakSSLKeyRecord struct {
+	tc.DeliveryServiceSSLKeys
+}
+type riakSSLKeyTable struct {
+	Records []riakSSLKeyRecord
+}
+
+func (tbl *riakSSLKeyTable) gatherKeys(cluster *riak.Cluster) error {
+	searchDocs, err := search(cluster, INDEX_SSL, "cdn:*", fmt.Sprintf("%v:*latest", SCHEMA_RIAK_KEY), 1000, SCHEMA_SSL_FIELDS[:])
+	if err != nil {
+		return fmt.Errorf("RiakSSLKey gatherKeys: %w", err)
+	}
+
+	tbl.Records = make([]riakSSLKeyRecord, len(searchDocs))
+	for i, doc := range searchDocs {
+		objs, err := getObject(cluster, doc.Bucket, doc.Key)
+		if err != nil {
+			return err
+		}
+		if len(objs) < 1 {
+			return fmt.Errorf("RiakSSLKey gatherKeys unable to find any objects with key %v and bucket %v, but search results were returned", doc.Key, doc.Bucket)
+		}
+		if len(objs) > 1 {
+			return fmt.Errorf("RiakSSLKey gatherKeys more than 1 ssl key record found %v\n", len(objs))
+		}
+		var obj tc.DeliveryServiceSSLKeys
+		if err = json.Unmarshal(objs[0].Value, &obj); err != nil {
+			return fmt.Errorf("RiakSSLKey gatherKeys unable to unmarshal object into tc.DeliveryServiceSSLKeys: %w", err)
+		}
+		tbl.Records[i] = riakSSLKeyRecord{
+			DeliveryServiceSSLKeys: obj,
+		}
+	}
+	return nil
+}
+func (tbl *riakSSLKeyTable) toGeneric() []SSLKey {
+	keys := make([]SSLKey, len(tbl.Records))
+
+	for i, record := range tbl.Records {
+		keys[i] = SSLKey{
+			DeliveryServiceSSLKeys: record.DeliveryServiceSSLKeys,
+		}
+	}
+
+	return keys
+}
+func (tbl *riakSSLKeyTable) fromGeneric(keys []SSLKey) {
+	tbl.Records = make([]riakSSLKeyRecord, len(keys))
+
+	for i, record := range keys {
+		tbl.Records[i] = riakSSLKeyRecord{
+			DeliveryServiceSSLKeys: record.DeliveryServiceSSLKeys,
+		}
+	}
+}
+func (tbl *riakSSLKeyTable) insertKeys(cluster *riak.Cluster) error {
+	for _, record := range tbl.Records {
+		objBytes, err := json.Marshal(record.DeliveryServiceSSLKeys)
+		if err != nil {
+			return fmt.Errorf("RiakSSLKey insertKeys failed to unmarshal keys: %w", err)
+		}
+		err = setObject(cluster, makeRiakObject(objBytes, record.DeliveryService+"-"+record.Version.String()), BUCKET_SSL)
+		if err != nil {
+			return fmt.Errorf("RiakSSLKey insertKeys: %w", err)
+		}
+		err = setObject(cluster, makeRiakObject(objBytes, record.DeliveryService+"-latest"), BUCKET_SSL)
+		if err != nil {
+			return fmt.Errorf("RiakSSLKey insertKeys: %w", err)
+		}
+	}
+	return nil
+}
+func (tbl *riakSSLKeyTable) validate() []string {
+	errs := []string{}
+	for i, record := range tbl.Records {
+		if record.DeliveryService == "" {
+			errs = append(errs, fmt.Sprintf("SSL Key #%v: Delivery Service is blank!", i))
+		}
+		if record.CDN == "" {
+			errs = append(errs, fmt.Sprintf("SSL Key #%v: CDN is blank!", i))
+		}
+		if record.Version.String() == "" {
+			errs = append(errs, fmt.Sprintf("SSL Key #%v: Version is blank!", i))
+		}
+	}
+	return errs
+}
+
+type riakDNSSecKeyRecord struct {
+	CDN string
+	Key tc.DNSSECKeysRiak
+}
+type riakDNSSecKeyTable struct {
+	Records []riakDNSSecKeyRecord
+}
+
+func (tbl *riakDNSSecKeyTable) gatherKeys(cluster *riak.Cluster) error {
+	tbl.Records = []riakDNSSecKeyRecord{}
+	objs, err := getObjects(cluster, BUCKET_DNSSEC)
+	if err != nil {
+		return fmt.Errorf("RiakDNSSecKey gatherKeys: %w", err)
+	}
+	for _, obj := range objs {
+		key := tc.DNSSECKeysRiak{}
+		if err := json.Unmarshal(obj.Value, &key); err != nil {
+			return fmt.Errorf("RiakDNSSecKey gatherKeys unable to unmarshal object to tc.DNSSECKeysRiak: %w", err)
+		}
+		tbl.Records = append(tbl.Records, riakDNSSecKeyRecord{
+			CDN: obj.Key,
+			Key: key,
+		})
+	}
+	return nil
+}
+func (tbl *riakDNSSecKeyTable) toGeneric() []DNSSecKey {
+	keys := make([]DNSSecKey, len(tbl.Records))
+
+	for i, record := range tbl.Records {
+		keys[i] = DNSSecKey{
+			CDN:                    record.CDN,
+			DNSSECKeysTrafficVault: tc.DNSSECKeysTrafficVault(record.Key),
+		}
+	}
+
+	return keys
+}
+func (tbl *riakDNSSecKeyTable) fromGeneric(keys []DNSSecKey) {
+	tbl.Records = make([]riakDNSSecKeyRecord, len(keys))
+
+	for i, record := range keys {
+		tbl.Records[i] = riakDNSSecKeyRecord{
+			CDN: record.CDN,
+			Key: tc.DNSSECKeysRiak(record.DNSSECKeysTrafficVault),
+		}
+	}
+}
+func (tbl *riakDNSSecKeyTable) insertKeys(cluster *riak.Cluster) error {
+	for _, record := range tbl.Records {
+		objBytes, err := json.Marshal(record.Key)
+		if err != nil {
+			return fmt.Errorf("RiakDNSSecKey insertKeys error marshalling keys: %w", err)
+		}
+
+		err = setObject(cluster, makeRiakObject(objBytes, record.CDN), BUCKET_DNSSEC)
+		if err != nil {
+			return fmt.Errorf("RiakDNSSecKey insertKeys: %w", err)
+		}
+	}
+	return nil
+}
+func (tbl *riakDNSSecKeyTable) validate() []string {
+	errs := []string{}
+	for i, record := range tbl.Records {
+		if record.CDN == "" {
+			errs = append(errs, fmt.Sprintf("DNSSec Key #%v: CDN is blank!", i))
+		}
+	}
+	return errs
+}
+
+type riakURLSigKeyRecord struct {
+	Key             tc.URLSigKeys
+	DeliveryService string
+}
+type riakURLSigKeyTable struct {
+	Records []riakURLSigKeyRecord
+}
+
+func (tbl *riakURLSigKeyTable) gatherKeys(cluster *riak.Cluster) error {
+	tbl.Records = []riakURLSigKeyRecord{}
+	objs, err := getObjects(cluster, BUCKET_URL_SIG)
+	if err != nil {
+		return fmt.Errorf("RiakURLSigKey gatherKeys: %w", err)
+	}
+	for _, obj := range objs {
+		key := tc.URLSigKeys{}
+		if err := json.Unmarshal(obj.Value, &key); err != nil {
+			return fmt.Errorf("RiakURLSigKey gatherKeys unable to unamrshal object into tc.URLSigKeys: %w", err)

Review comment:
       this error message should probably include the `obj.Key` for troubleshooting

##########
File path: tools/traffic_vault_migrate/riak.go
##########
@@ -0,0 +1,746 @@
+package main
+
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+import (
+	"crypto/tls"
+	"encoding/json"
+	"errors"
+	"fmt"
+	"time"
+
+	"github.com/apache/trafficcontrol/lib/go-rfc"
+	"github.com/apache/trafficcontrol/lib/go-tc"
+
+	"github.com/basho/riak-go-client"
+)
+
+const (
+	BUCKET_SSL     = "ssl"
+	BUCKET_DNSSEC  = "dnssec"
+	BUCKET_URL_SIG = "url_sig_keys"
+	BUCKET_URI_SIG = "cdn_uri_sig_keys"
+
+	INDEX_SSL = "sslkeys"
+
+	SCHEMA_RIAK_KEY    = "_yz_rk"
+	SCHEMA_RIAK_BUCKET = "_yz_rb"
+)
+
+var (
+	SCHEMA_SSL_FIELDS = [...]string{SCHEMA_RIAK_KEY, SCHEMA_RIAK_BUCKET}
+)
+
+// RiakConfig  represents the configuration options available to the Riak backend
+type RiakConfig struct {
+	Host          string `json:"host"`
+	Port          string `json:"port"`
+	User          string `json:"user"`
+	Password      string `json:"password"`
+	Insecure      bool   `json:"insecure"`
+	TLSVersionRaw string `json:"tlsVersion"`
+	TLSVersion    uint16
+}
+
+// RiakBackend is the Riak implementation of TVBackend
+type RiakBackend struct {
+	sslKeys     riakSSLKeyTable
+	dnssecKeys  riakDNSSecKeyTable
+	uriSignKeys riakURISignKeyTable
+	urlSigKeys  riakURLSigKeyTable
+	cfg         RiakConfig
+	cluster     *riak.Cluster
+}
+
+// String returns a high level overview of the backend and its keys
+func (rb *RiakBackend) String() string {
+	data := fmt.Sprintf("Riak server %v@%v:%v\n", rb.cfg.User, rb.cfg.Host, rb.cfg.Port)
+	data += fmt.Sprintf("\tSSL Keys: %v\n", len(rb.sslKeys.Records))
+	data += fmt.Sprintf("\tDNSSec Keys: %v\n", len(rb.dnssecKeys.Records))
+	data += fmt.Sprintf("\tURI Keys: %v\n", len(rb.uriSignKeys.Records))
+	data += fmt.Sprintf("\tURL Keys: %v\n", len(rb.urlSigKeys.Records))
+	return data
+}
+
+// Name returns the name for this backend
+func (rb *RiakBackend) Name() string {
+	return "Riak"
+}
+
+// ReadConfig takes in a filename and will read it into the backends config
+func (rb *RiakBackend) ReadConfig(s string) error {
+	err := UnmarshalConfig(s, &rb.cfg)
+	if err != nil {
+		return err
+	}
+
+	if rb.cfg.TLSVersionRaw == "10" {
+		rb.cfg.TLSVersion = tls.VersionTLS10
+	} else if rb.cfg.TLSVersionRaw == "11" {
+		rb.cfg.TLSVersion = tls.VersionTLS11
+	} else if rb.cfg.TLSVersionRaw == "12" {
+		rb.cfg.TLSVersion = tls.VersionTLS12
+	} else if rb.cfg.TLSVersionRaw == "13" {
+		rb.cfg.TLSVersion = tls.VersionTLS13
+	}
+	return nil
+}
+
+// Insert takes the current keys and inserts them into the backend DB
+func (rb *RiakBackend) Insert() error {
+	if err := rb.sslKeys.insertKeys(rb.cluster); err != nil {
+		return err
+	}
+	if err := rb.dnssecKeys.insertKeys(rb.cluster); err != nil {
+		return err
+	}
+	if err := rb.urlSigKeys.insertKeys(rb.cluster); err != nil {
+		return err
+	}
+	if err := rb.uriSignKeys.insertKeys(rb.cluster); err != nil {
+		return err
+	}
+	return nil
+}
+
+// ValidateKey validates that the keys are valid (in most cases, certain fields are not null)
+func (rb *RiakBackend) ValidateKey() []string {
+	errors := []string{}
+	if errs := rb.sslKeys.validate(); errs != nil {
+		errors = append(errors, errs...)
+	}
+	if errs := rb.dnssecKeys.validate(); errs != nil {
+		errors = append(errors, errs...)
+	}
+	if errs := rb.uriSignKeys.validate(); errs != nil {
+		errors = append(errors, errs...)
+	}
+	if errs := rb.urlSigKeys.validate(); errs != nil {
+		errors = append(errors, errs...)
+	}
+
+	return errors
+}
+
+// SetSSLKeys takes in keys and converts & encrypts the data into the backends internal format
+func (rb *RiakBackend) SetSSLKeys(keys []SSLKey) error {
+	rb.sslKeys.fromGeneric(keys)
+	return nil
+}
+
+// SetDNSSecKeys takes in keys and converts & encrypts the data into the backends internal format
+func (rb *RiakBackend) SetDNSSecKeys(keys []DNSSecKey) error {
+	rb.dnssecKeys.fromGeneric(keys)
+	return nil
+}
+
+// SetURISignKeys takes in keys and converts & encrypts the data into the backends internal format
+func (rb *RiakBackend) SetURISignKeys(keys []URISignKey) error {
+	rb.uriSignKeys.fromGeneric(keys)
+	return nil
+}
+
+// SetURLSigKeys takes in keys and converts & encrypts the data into the backends internal format
+func (rb *RiakBackend) SetURLSigKeys(keys []URLSigKey) error {
+	rb.urlSigKeys.fromGeneric(keys)
+	return nil
+}
+
+// Start initiates the connection to the backend DB
+func (rb *RiakBackend) Start() error {
+	tlsConfig := &tls.Config{
+		InsecureSkipVerify: !rb.cfg.Insecure,
+		MaxVersion:         rb.cfg.TLSVersion,
+	}
+	auth := &riak.AuthOptions{
+		User:      rb.cfg.User,
+		Password:  rb.cfg.Password,
+		TlsConfig: tlsConfig,
+	}
+
+	cluster, err := getRiakCluster(rb.cfg, auth)
+	if err != nil {
+		return err
+	}
+	if err := cluster.Start(); err != nil {
+		return fmt.Errorf("unable to start riak cluster: %w", err)
+	}
+
+	rb.cluster = cluster
+	rb.sslKeys = riakSSLKeyTable{}
+	rb.dnssecKeys = riakDNSSecKeyTable{}
+	rb.urlSigKeys = riakURLSigKeyTable{}
+	rb.uriSignKeys = riakURISignKeyTable{}
+	return nil
+}
+
+// Close terminates the connection to the backend DB
+func (rb *RiakBackend) Close() error {
+	if err := rb.cluster.Stop(); err != nil {
+		return err
+	}
+	return nil
+}
+
+// Ping checks the connection to the backend DB
+func (rb *RiakBackend) Ping() error {
+	return ping(rb.cluster)
+}
+
+// GetSSLKeys converts the backends internal key representation into the common representation (SSLKey)
+func (rb *RiakBackend) GetSSLKeys() ([]SSLKey, error) {
+	return rb.sslKeys.toGeneric(), nil
+}
+
+// GetDNSSecKeys converts the backends internal key representation into the common representation (DNSSecKey)
+func (rb *RiakBackend) GetDNSSecKeys() ([]DNSSecKey, error) {
+	return rb.dnssecKeys.toGeneric(), nil
+}
+
+// GetURISignKeys converts the pg internal key representation into the common representation (URISignKey)
+func (rb *RiakBackend) GetURISignKeys() ([]URISignKey, error) {
+	return rb.uriSignKeys.toGeneric(), nil
+}
+
+// GetURLSigKeys converts the backends internal key representation into the common representation (URLSigKey)
+func (rb *RiakBackend) GetURLSigKeys() ([]URLSigKey, error) {
+	return rb.urlSigKeys.toGeneric(), nil
+}
+
+// Fetch gets all of the keys from the backend DB
+func (rb *RiakBackend) Fetch() error {
+	if err := rb.sslKeys.gatherKeys(rb.cluster); err != nil {
+		return err
+	}
+	if err := rb.dnssecKeys.gatherKeys(rb.cluster); err != nil {
+		return err
+	}
+	if err := rb.urlSigKeys.gatherKeys(rb.cluster); err != nil {
+		return err
+	}
+	if err := rb.uriSignKeys.gatherKeys(rb.cluster); err != nil {
+		return err
+	}
+
+	return nil
+}
+
+type riakSSLKeyRecord struct {
+	tc.DeliveryServiceSSLKeys
+}
+type riakSSLKeyTable struct {
+	Records []riakSSLKeyRecord
+}
+
+func (tbl *riakSSLKeyTable) gatherKeys(cluster *riak.Cluster) error {
+	searchDocs, err := search(cluster, INDEX_SSL, "cdn:*", fmt.Sprintf("%v:*latest", SCHEMA_RIAK_KEY), 1000, SCHEMA_SSL_FIELDS[:])
+	if err != nil {
+		return fmt.Errorf("RiakSSLKey gatherKeys: %w", err)
+	}
+
+	tbl.Records = make([]riakSSLKeyRecord, len(searchDocs))
+	for i, doc := range searchDocs {
+		objs, err := getObject(cluster, doc.Bucket, doc.Key)
+		if err != nil {
+			return err
+		}
+		if len(objs) < 1 {
+			return fmt.Errorf("RiakSSLKey gatherKeys unable to find any objects with key %v and bucket %v, but search results were returned", doc.Key, doc.Bucket)
+		}
+		if len(objs) > 1 {
+			return fmt.Errorf("RiakSSLKey gatherKeys more than 1 ssl key record found %v\n", len(objs))
+		}
+		var obj tc.DeliveryServiceSSLKeys
+		if err = json.Unmarshal(objs[0].Value, &obj); err != nil {
+			return fmt.Errorf("RiakSSLKey gatherKeys unable to unmarshal object into tc.DeliveryServiceSSLKeys: %w", err)
+		}
+		tbl.Records[i] = riakSSLKeyRecord{
+			DeliveryServiceSSLKeys: obj,
+		}
+	}
+	return nil
+}
+func (tbl *riakSSLKeyTable) toGeneric() []SSLKey {
+	keys := make([]SSLKey, len(tbl.Records))
+
+	for i, record := range tbl.Records {
+		keys[i] = SSLKey{
+			DeliveryServiceSSLKeys: record.DeliveryServiceSSLKeys,
+		}
+	}
+
+	return keys
+}
+func (tbl *riakSSLKeyTable) fromGeneric(keys []SSLKey) {
+	tbl.Records = make([]riakSSLKeyRecord, len(keys))
+
+	for i, record := range keys {
+		tbl.Records[i] = riakSSLKeyRecord{
+			DeliveryServiceSSLKeys: record.DeliveryServiceSSLKeys,
+		}
+	}
+}
+func (tbl *riakSSLKeyTable) insertKeys(cluster *riak.Cluster) error {
+	for _, record := range tbl.Records {
+		objBytes, err := json.Marshal(record.DeliveryServiceSSLKeys)
+		if err != nil {
+			return fmt.Errorf("RiakSSLKey insertKeys failed to unmarshal keys: %w", err)
+		}
+		err = setObject(cluster, makeRiakObject(objBytes, record.DeliveryService+"-"+record.Version.String()), BUCKET_SSL)
+		if err != nil {
+			return fmt.Errorf("RiakSSLKey insertKeys: %w", err)
+		}
+		err = setObject(cluster, makeRiakObject(objBytes, record.DeliveryService+"-latest"), BUCKET_SSL)
+		if err != nil {
+			return fmt.Errorf("RiakSSLKey insertKeys: %w", err)
+		}
+	}
+	return nil
+}
+func (tbl *riakSSLKeyTable) validate() []string {
+	errs := []string{}
+	for i, record := range tbl.Records {
+		if record.DeliveryService == "" {
+			errs = append(errs, fmt.Sprintf("SSL Key #%v: Delivery Service is blank!", i))
+		}
+		if record.CDN == "" {
+			errs = append(errs, fmt.Sprintf("SSL Key #%v: CDN is blank!", i))
+		}
+		if record.Version.String() == "" {
+			errs = append(errs, fmt.Sprintf("SSL Key #%v: Version is blank!", i))
+		}
+	}
+	return errs
+}
+
+type riakDNSSecKeyRecord struct {
+	CDN string
+	Key tc.DNSSECKeysRiak
+}
+type riakDNSSecKeyTable struct {
+	Records []riakDNSSecKeyRecord
+}
+
+func (tbl *riakDNSSecKeyTable) gatherKeys(cluster *riak.Cluster) error {
+	tbl.Records = []riakDNSSecKeyRecord{}
+	objs, err := getObjects(cluster, BUCKET_DNSSEC)
+	if err != nil {
+		return fmt.Errorf("RiakDNSSecKey gatherKeys: %w", err)
+	}
+	for _, obj := range objs {
+		key := tc.DNSSECKeysRiak{}
+		if err := json.Unmarshal(obj.Value, &key); err != nil {
+			return fmt.Errorf("RiakDNSSecKey gatherKeys unable to unmarshal object to tc.DNSSECKeysRiak: %w", err)
+		}
+		tbl.Records = append(tbl.Records, riakDNSSecKeyRecord{
+			CDN: obj.Key,
+			Key: key,
+		})
+	}
+	return nil
+}
+func (tbl *riakDNSSecKeyTable) toGeneric() []DNSSecKey {
+	keys := make([]DNSSecKey, len(tbl.Records))
+
+	for i, record := range tbl.Records {
+		keys[i] = DNSSecKey{
+			CDN:                    record.CDN,
+			DNSSECKeysTrafficVault: tc.DNSSECKeysTrafficVault(record.Key),
+		}
+	}
+
+	return keys
+}
+func (tbl *riakDNSSecKeyTable) fromGeneric(keys []DNSSecKey) {
+	tbl.Records = make([]riakDNSSecKeyRecord, len(keys))
+
+	for i, record := range keys {
+		tbl.Records[i] = riakDNSSecKeyRecord{
+			CDN: record.CDN,
+			Key: tc.DNSSECKeysRiak(record.DNSSECKeysTrafficVault),
+		}
+	}
+}
+func (tbl *riakDNSSecKeyTable) insertKeys(cluster *riak.Cluster) error {
+	for _, record := range tbl.Records {
+		objBytes, err := json.Marshal(record.Key)
+		if err != nil {
+			return fmt.Errorf("RiakDNSSecKey insertKeys error marshalling keys: %w", err)
+		}
+
+		err = setObject(cluster, makeRiakObject(objBytes, record.CDN), BUCKET_DNSSEC)
+		if err != nil {
+			return fmt.Errorf("RiakDNSSecKey insertKeys: %w", err)
+		}
+	}
+	return nil
+}
+func (tbl *riakDNSSecKeyTable) validate() []string {
+	errs := []string{}
+	for i, record := range tbl.Records {
+		if record.CDN == "" {
+			errs = append(errs, fmt.Sprintf("DNSSec Key #%v: CDN is blank!", i))
+		}
+	}
+	return errs
+}
+
+type riakURLSigKeyRecord struct {
+	Key             tc.URLSigKeys
+	DeliveryService string
+}
+type riakURLSigKeyTable struct {
+	Records []riakURLSigKeyRecord
+}
+
+func (tbl *riakURLSigKeyTable) gatherKeys(cluster *riak.Cluster) error {
+	tbl.Records = []riakURLSigKeyRecord{}
+	objs, err := getObjects(cluster, BUCKET_URL_SIG)
+	if err != nil {
+		return fmt.Errorf("RiakURLSigKey gatherKeys: %w", err)
+	}
+	for _, obj := range objs {
+		key := tc.URLSigKeys{}
+		if err := json.Unmarshal(obj.Value, &key); err != nil {
+			return fmt.Errorf("RiakURLSigKey gatherKeys unable to unamrshal object into tc.URLSigKeys: %w", err)
+		}
+		tbl.Records = append(tbl.Records, riakURLSigKeyRecord{
+			DeliveryService: obj.Key,
+			Key:             key,
+		})
+	}
+	return nil
+}
+func (tbl *riakURLSigKeyTable) toGeneric() []URLSigKey {
+	keys := make([]URLSigKey, len(tbl.Records))
+
+	for i, record := range tbl.Records {
+		keys[i] = URLSigKey{
+			URLSigKeys:      record.Key,
+			DeliveryService: record.DeliveryService,
+		}
+	}
+
+	return keys
+}
+func (tbl *riakURLSigKeyTable) fromGeneric(keys []URLSigKey) {
+	tbl.Records = make([]riakURLSigKeyRecord, len(keys))
+	for i, key := range keys {
+		tbl.Records[i] = riakURLSigKeyRecord{
+			Key:             key.URLSigKeys,
+			DeliveryService: key.DeliveryService,
+		}
+	}
+}
+func (tbl *riakURLSigKeyTable) insertKeys(cluster *riak.Cluster) error {
+	for _, record := range tbl.Records {
+		objBytes, err := json.Marshal(record.Key)
+		if err != nil {
+			return fmt.Errorf("RiakURLSigKey insertKeys unable to marshal keys: %w", err)
+		}
+
+		err = setObject(cluster, makeRiakObject(objBytes, "url_sig_"+record.DeliveryService+".config"), BUCKET_URL_SIG)
+		if err != nil {
+			return fmt.Errorf("RiakURLSigKey insertKeys: %w", err)
+		}
+	}
+	return nil
+}
+func (tbl *riakURLSigKeyTable) validate() []string {
+	errs := []string{}
+	for i, record := range tbl.Records {
+		if record.DeliveryService == "" {
+			errs = append(errs, fmt.Sprintf("URL Key #%v: Delivery Service is blank!", i))

Review comment:
       nit: type-specific formatting instead of %v

##########
File path: tools/traffic_vault_migrate/riak.go
##########
@@ -0,0 +1,746 @@
+package main
+
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+import (
+	"crypto/tls"
+	"encoding/json"
+	"errors"
+	"fmt"
+	"time"
+
+	"github.com/apache/trafficcontrol/lib/go-rfc"
+	"github.com/apache/trafficcontrol/lib/go-tc"
+
+	"github.com/basho/riak-go-client"
+)
+
+const (
+	BUCKET_SSL     = "ssl"
+	BUCKET_DNSSEC  = "dnssec"
+	BUCKET_URL_SIG = "url_sig_keys"
+	BUCKET_URI_SIG = "cdn_uri_sig_keys"
+
+	INDEX_SSL = "sslkeys"
+
+	SCHEMA_RIAK_KEY    = "_yz_rk"
+	SCHEMA_RIAK_BUCKET = "_yz_rb"
+)
+
+var (
+	SCHEMA_SSL_FIELDS = [...]string{SCHEMA_RIAK_KEY, SCHEMA_RIAK_BUCKET}
+)
+
+// RiakConfig  represents the configuration options available to the Riak backend
+type RiakConfig struct {
+	Host          string `json:"host"`
+	Port          string `json:"port"`
+	User          string `json:"user"`
+	Password      string `json:"password"`
+	Insecure      bool   `json:"insecure"`
+	TLSVersionRaw string `json:"tlsVersion"`
+	TLSVersion    uint16
+}
+
+// RiakBackend is the Riak implementation of TVBackend
+type RiakBackend struct {
+	sslKeys     riakSSLKeyTable
+	dnssecKeys  riakDNSSecKeyTable
+	uriSignKeys riakURISignKeyTable
+	urlSigKeys  riakURLSigKeyTable
+	cfg         RiakConfig
+	cluster     *riak.Cluster
+}
+
+// String returns a high level overview of the backend and its keys
+func (rb *RiakBackend) String() string {
+	data := fmt.Sprintf("Riak server %v@%v:%v\n", rb.cfg.User, rb.cfg.Host, rb.cfg.Port)
+	data += fmt.Sprintf("\tSSL Keys: %v\n", len(rb.sslKeys.Records))
+	data += fmt.Sprintf("\tDNSSec Keys: %v\n", len(rb.dnssecKeys.Records))
+	data += fmt.Sprintf("\tURI Keys: %v\n", len(rb.uriSignKeys.Records))
+	data += fmt.Sprintf("\tURL Keys: %v\n", len(rb.urlSigKeys.Records))
+	return data
+}
+
+// Name returns the name for this backend
+func (rb *RiakBackend) Name() string {
+	return "Riak"
+}
+
+// ReadConfig takes in a filename and will read it into the backends config
+func (rb *RiakBackend) ReadConfig(s string) error {
+	err := UnmarshalConfig(s, &rb.cfg)
+	if err != nil {
+		return err
+	}
+
+	if rb.cfg.TLSVersionRaw == "10" {
+		rb.cfg.TLSVersion = tls.VersionTLS10
+	} else if rb.cfg.TLSVersionRaw == "11" {
+		rb.cfg.TLSVersion = tls.VersionTLS11
+	} else if rb.cfg.TLSVersionRaw == "12" {
+		rb.cfg.TLSVersion = tls.VersionTLS12
+	} else if rb.cfg.TLSVersionRaw == "13" {
+		rb.cfg.TLSVersion = tls.VersionTLS13
+	}
+	return nil
+}
+
+// Insert takes the current keys and inserts them into the backend DB
+func (rb *RiakBackend) Insert() error {
+	if err := rb.sslKeys.insertKeys(rb.cluster); err != nil {
+		return err
+	}
+	if err := rb.dnssecKeys.insertKeys(rb.cluster); err != nil {
+		return err
+	}
+	if err := rb.urlSigKeys.insertKeys(rb.cluster); err != nil {
+		return err
+	}
+	if err := rb.uriSignKeys.insertKeys(rb.cluster); err != nil {
+		return err
+	}
+	return nil
+}
+
+// ValidateKey validates that the keys are valid (in most cases, certain fields are not null)
+func (rb *RiakBackend) ValidateKey() []string {
+	errors := []string{}
+	if errs := rb.sslKeys.validate(); errs != nil {
+		errors = append(errors, errs...)
+	}
+	if errs := rb.dnssecKeys.validate(); errs != nil {
+		errors = append(errors, errs...)
+	}
+	if errs := rb.uriSignKeys.validate(); errs != nil {
+		errors = append(errors, errs...)
+	}
+	if errs := rb.urlSigKeys.validate(); errs != nil {
+		errors = append(errors, errs...)
+	}
+
+	return errors
+}
+
+// SetSSLKeys takes in keys and converts & encrypts the data into the backends internal format
+func (rb *RiakBackend) SetSSLKeys(keys []SSLKey) error {
+	rb.sslKeys.fromGeneric(keys)
+	return nil
+}
+
+// SetDNSSecKeys takes in keys and converts & encrypts the data into the backends internal format
+func (rb *RiakBackend) SetDNSSecKeys(keys []DNSSecKey) error {
+	rb.dnssecKeys.fromGeneric(keys)
+	return nil
+}
+
+// SetURISignKeys takes in keys and converts & encrypts the data into the backends internal format
+func (rb *RiakBackend) SetURISignKeys(keys []URISignKey) error {
+	rb.uriSignKeys.fromGeneric(keys)
+	return nil
+}
+
+// SetURLSigKeys takes in keys and converts & encrypts the data into the backends internal format
+func (rb *RiakBackend) SetURLSigKeys(keys []URLSigKey) error {
+	rb.urlSigKeys.fromGeneric(keys)
+	return nil
+}
+
+// Start initiates the connection to the backend DB
+func (rb *RiakBackend) Start() error {
+	tlsConfig := &tls.Config{
+		InsecureSkipVerify: !rb.cfg.Insecure,
+		MaxVersion:         rb.cfg.TLSVersion,
+	}
+	auth := &riak.AuthOptions{
+		User:      rb.cfg.User,
+		Password:  rb.cfg.Password,
+		TlsConfig: tlsConfig,
+	}
+
+	cluster, err := getRiakCluster(rb.cfg, auth)
+	if err != nil {
+		return err
+	}
+	if err := cluster.Start(); err != nil {
+		return fmt.Errorf("unable to start riak cluster: %w", err)
+	}
+
+	rb.cluster = cluster
+	rb.sslKeys = riakSSLKeyTable{}
+	rb.dnssecKeys = riakDNSSecKeyTable{}
+	rb.urlSigKeys = riakURLSigKeyTable{}
+	rb.uriSignKeys = riakURISignKeyTable{}
+	return nil
+}
+
+// Close terminates the connection to the backend DB
+func (rb *RiakBackend) Close() error {
+	if err := rb.cluster.Stop(); err != nil {
+		return err
+	}
+	return nil
+}
+
+// Ping checks the connection to the backend DB
+func (rb *RiakBackend) Ping() error {
+	return ping(rb.cluster)
+}
+
+// GetSSLKeys converts the backends internal key representation into the common representation (SSLKey)
+func (rb *RiakBackend) GetSSLKeys() ([]SSLKey, error) {
+	return rb.sslKeys.toGeneric(), nil
+}
+
+// GetDNSSecKeys converts the backends internal key representation into the common representation (DNSSecKey)
+func (rb *RiakBackend) GetDNSSecKeys() ([]DNSSecKey, error) {
+	return rb.dnssecKeys.toGeneric(), nil
+}
+
+// GetURISignKeys converts the pg internal key representation into the common representation (URISignKey)
+func (rb *RiakBackend) GetURISignKeys() ([]URISignKey, error) {
+	return rb.uriSignKeys.toGeneric(), nil
+}
+
+// GetURLSigKeys converts the backends internal key representation into the common representation (URLSigKey)
+func (rb *RiakBackend) GetURLSigKeys() ([]URLSigKey, error) {
+	return rb.urlSigKeys.toGeneric(), nil
+}
+
+// Fetch gets all of the keys from the backend DB
+func (rb *RiakBackend) Fetch() error {
+	if err := rb.sslKeys.gatherKeys(rb.cluster); err != nil {
+		return err
+	}
+	if err := rb.dnssecKeys.gatherKeys(rb.cluster); err != nil {
+		return err
+	}
+	if err := rb.urlSigKeys.gatherKeys(rb.cluster); err != nil {
+		return err
+	}
+	if err := rb.uriSignKeys.gatherKeys(rb.cluster); err != nil {
+		return err
+	}
+
+	return nil
+}
+
+type riakSSLKeyRecord struct {
+	tc.DeliveryServiceSSLKeys
+}
+type riakSSLKeyTable struct {
+	Records []riakSSLKeyRecord
+}
+
+func (tbl *riakSSLKeyTable) gatherKeys(cluster *riak.Cluster) error {
+	searchDocs, err := search(cluster, INDEX_SSL, "cdn:*", fmt.Sprintf("%v:*latest", SCHEMA_RIAK_KEY), 1000, SCHEMA_SSL_FIELDS[:])
+	if err != nil {
+		return fmt.Errorf("RiakSSLKey gatherKeys: %w", err)
+	}
+
+	tbl.Records = make([]riakSSLKeyRecord, len(searchDocs))
+	for i, doc := range searchDocs {
+		objs, err := getObject(cluster, doc.Bucket, doc.Key)
+		if err != nil {
+			return err
+		}
+		if len(objs) < 1 {
+			return fmt.Errorf("RiakSSLKey gatherKeys unable to find any objects with key %v and bucket %v, but search results were returned", doc.Key, doc.Bucket)
+		}
+		if len(objs) > 1 {
+			return fmt.Errorf("RiakSSLKey gatherKeys more than 1 ssl key record found %v\n", len(objs))
+		}
+		var obj tc.DeliveryServiceSSLKeys
+		if err = json.Unmarshal(objs[0].Value, &obj); err != nil {
+			return fmt.Errorf("RiakSSLKey gatherKeys unable to unmarshal object into tc.DeliveryServiceSSLKeys: %w", err)
+		}
+		tbl.Records[i] = riakSSLKeyRecord{
+			DeliveryServiceSSLKeys: obj,
+		}
+	}
+	return nil
+}
+func (tbl *riakSSLKeyTable) toGeneric() []SSLKey {
+	keys := make([]SSLKey, len(tbl.Records))
+
+	for i, record := range tbl.Records {
+		keys[i] = SSLKey{
+			DeliveryServiceSSLKeys: record.DeliveryServiceSSLKeys,
+		}
+	}
+
+	return keys
+}
+func (tbl *riakSSLKeyTable) fromGeneric(keys []SSLKey) {
+	tbl.Records = make([]riakSSLKeyRecord, len(keys))
+
+	for i, record := range keys {
+		tbl.Records[i] = riakSSLKeyRecord{
+			DeliveryServiceSSLKeys: record.DeliveryServiceSSLKeys,
+		}
+	}
+}
+func (tbl *riakSSLKeyTable) insertKeys(cluster *riak.Cluster) error {
+	for _, record := range tbl.Records {
+		objBytes, err := json.Marshal(record.DeliveryServiceSSLKeys)
+		if err != nil {
+			return fmt.Errorf("RiakSSLKey insertKeys failed to unmarshal keys: %w", err)
+		}
+		err = setObject(cluster, makeRiakObject(objBytes, record.DeliveryService+"-"+record.Version.String()), BUCKET_SSL)
+		if err != nil {
+			return fmt.Errorf("RiakSSLKey insertKeys: %w", err)
+		}
+		err = setObject(cluster, makeRiakObject(objBytes, record.DeliveryService+"-latest"), BUCKET_SSL)
+		if err != nil {
+			return fmt.Errorf("RiakSSLKey insertKeys: %w", err)
+		}
+	}
+	return nil
+}
+func (tbl *riakSSLKeyTable) validate() []string {
+	errs := []string{}
+	for i, record := range tbl.Records {
+		if record.DeliveryService == "" {
+			errs = append(errs, fmt.Sprintf("SSL Key #%v: Delivery Service is blank!", i))
+		}
+		if record.CDN == "" {
+			errs = append(errs, fmt.Sprintf("SSL Key #%v: CDN is blank!", i))
+		}
+		if record.Version.String() == "" {
+			errs = append(errs, fmt.Sprintf("SSL Key #%v: Version is blank!", i))
+		}
+	}
+	return errs
+}
+
+type riakDNSSecKeyRecord struct {
+	CDN string
+	Key tc.DNSSECKeysRiak
+}
+type riakDNSSecKeyTable struct {
+	Records []riakDNSSecKeyRecord
+}
+
+func (tbl *riakDNSSecKeyTable) gatherKeys(cluster *riak.Cluster) error {
+	tbl.Records = []riakDNSSecKeyRecord{}
+	objs, err := getObjects(cluster, BUCKET_DNSSEC)
+	if err != nil {
+		return fmt.Errorf("RiakDNSSecKey gatherKeys: %w", err)
+	}
+	for _, obj := range objs {
+		key := tc.DNSSECKeysRiak{}
+		if err := json.Unmarshal(obj.Value, &key); err != nil {
+			return fmt.Errorf("RiakDNSSecKey gatherKeys unable to unmarshal object to tc.DNSSECKeysRiak: %w", err)
+		}
+		tbl.Records = append(tbl.Records, riakDNSSecKeyRecord{
+			CDN: obj.Key,
+			Key: key,
+		})
+	}
+	return nil
+}
+func (tbl *riakDNSSecKeyTable) toGeneric() []DNSSecKey {
+	keys := make([]DNSSecKey, len(tbl.Records))
+
+	for i, record := range tbl.Records {
+		keys[i] = DNSSecKey{
+			CDN:                    record.CDN,
+			DNSSECKeysTrafficVault: tc.DNSSECKeysTrafficVault(record.Key),
+		}
+	}
+
+	return keys
+}
+func (tbl *riakDNSSecKeyTable) fromGeneric(keys []DNSSecKey) {
+	tbl.Records = make([]riakDNSSecKeyRecord, len(keys))
+
+	for i, record := range keys {
+		tbl.Records[i] = riakDNSSecKeyRecord{
+			CDN: record.CDN,
+			Key: tc.DNSSECKeysRiak(record.DNSSECKeysTrafficVault),
+		}
+	}
+}
+func (tbl *riakDNSSecKeyTable) insertKeys(cluster *riak.Cluster) error {
+	for _, record := range tbl.Records {
+		objBytes, err := json.Marshal(record.Key)
+		if err != nil {
+			return fmt.Errorf("RiakDNSSecKey insertKeys error marshalling keys: %w", err)
+		}
+
+		err = setObject(cluster, makeRiakObject(objBytes, record.CDN), BUCKET_DNSSEC)
+		if err != nil {
+			return fmt.Errorf("RiakDNSSecKey insertKeys: %w", err)
+		}
+	}
+	return nil
+}
+func (tbl *riakDNSSecKeyTable) validate() []string {
+	errs := []string{}
+	for i, record := range tbl.Records {
+		if record.CDN == "" {
+			errs = append(errs, fmt.Sprintf("DNSSec Key #%v: CDN is blank!", i))
+		}
+	}
+	return errs
+}
+
+type riakURLSigKeyRecord struct {
+	Key             tc.URLSigKeys
+	DeliveryService string
+}
+type riakURLSigKeyTable struct {
+	Records []riakURLSigKeyRecord
+}
+
+func (tbl *riakURLSigKeyTable) gatherKeys(cluster *riak.Cluster) error {
+	tbl.Records = []riakURLSigKeyRecord{}
+	objs, err := getObjects(cluster, BUCKET_URL_SIG)
+	if err != nil {
+		return fmt.Errorf("RiakURLSigKey gatherKeys: %w", err)
+	}
+	for _, obj := range objs {
+		key := tc.URLSigKeys{}
+		if err := json.Unmarshal(obj.Value, &key); err != nil {
+			return fmt.Errorf("RiakURLSigKey gatherKeys unable to unamrshal object into tc.URLSigKeys: %w", err)
+		}
+		tbl.Records = append(tbl.Records, riakURLSigKeyRecord{
+			DeliveryService: obj.Key,
+			Key:             key,
+		})
+	}
+	return nil
+}
+func (tbl *riakURLSigKeyTable) toGeneric() []URLSigKey {
+	keys := make([]URLSigKey, len(tbl.Records))
+
+	for i, record := range tbl.Records {
+		keys[i] = URLSigKey{
+			URLSigKeys:      record.Key,
+			DeliveryService: record.DeliveryService,
+		}
+	}
+
+	return keys
+}
+func (tbl *riakURLSigKeyTable) fromGeneric(keys []URLSigKey) {
+	tbl.Records = make([]riakURLSigKeyRecord, len(keys))
+	for i, key := range keys {
+		tbl.Records[i] = riakURLSigKeyRecord{
+			Key:             key.URLSigKeys,
+			DeliveryService: key.DeliveryService,
+		}
+	}
+}
+func (tbl *riakURLSigKeyTable) insertKeys(cluster *riak.Cluster) error {
+	for _, record := range tbl.Records {
+		objBytes, err := json.Marshal(record.Key)
+		if err != nil {
+			return fmt.Errorf("RiakURLSigKey insertKeys unable to marshal keys: %w", err)
+		}
+
+		err = setObject(cluster, makeRiakObject(objBytes, "url_sig_"+record.DeliveryService+".config"), BUCKET_URL_SIG)
+		if err != nil {
+			return fmt.Errorf("RiakURLSigKey insertKeys: %w", err)
+		}
+	}
+	return nil
+}
+func (tbl *riakURLSigKeyTable) validate() []string {
+	errs := []string{}
+	for i, record := range tbl.Records {
+		if record.DeliveryService == "" {
+			errs = append(errs, fmt.Sprintf("URL Key #%v: Delivery Service is blank!", i))
+		}
+	}
+	return errs
+}
+
+type riakURISignKeyRecord struct {
+	Key             map[string]tc.URISignerKeyset
+	DeliveryService string
+}
+type riakURISignKeyTable struct {
+	Records []riakURISignKeyRecord
+}
+
+func (tbl *riakURISignKeyTable) gatherKeys(cluster *riak.Cluster) error {
+	tbl.Records = []riakURISignKeyRecord{}
+	objs, err := getObjects(cluster, BUCKET_URI_SIG)
+	if err != nil {
+		return fmt.Errorf("RiakURISignKey gatherKeys: %w", err)
+	}
+	for _, obj := range objs {
+		key := map[string]tc.URISignerKeyset{}
+		if err := json.Unmarshal(obj.Value, &key); err != nil {
+			return fmt.Errorf("RiakURISignKey gatherKeys unable to unmarshal object into map[string]tc.URISignerKeySet: %w", err)
+		}
+
+		tbl.Records = append(tbl.Records, riakURISignKeyRecord{
+			DeliveryService: obj.Key,
+			Key:             key,
+		})
+	}
+	return nil
+}
+func (tbl *riakURISignKeyTable) toGeneric() []URISignKey {
+	keys := make([]URISignKey, len(tbl.Records))
+
+	for i, record := range tbl.Records {
+		keys[i] = URISignKey{
+			DeliveryService: record.DeliveryService,
+			Keys:            record.Key,
+		}
+	}
+
+	return keys
+}
+func (tbl *riakURISignKeyTable) fromGeneric(keys []URISignKey) {
+	tbl.Records = make([]riakURISignKeyRecord, len(keys))
+
+	for i, record := range keys {
+		tbl.Records[i] = riakURISignKeyRecord{
+			Key:             record.Keys,
+			DeliveryService: record.DeliveryService,
+		}
+	}
+}
+func (tbl *riakURISignKeyTable) insertKeys(cluster *riak.Cluster) error {
+	for _, record := range tbl.Records {
+		objBytes, err := json.Marshal(record.Key)
+		if err != nil {
+			return fmt.Errorf("RiakURISignKey insertKeys: unable to marshal key: %w", err)
+		}
+
+		err = setObject(cluster, makeRiakObject(objBytes, record.DeliveryService), BUCKET_URI_SIG)
+		if err != nil {
+			return fmt.Errorf("RiakURISignKey insertKeys: %w", err)

Review comment:
       this error message should probably include the `record.DeliveryService` for troubleshooting

##########
File path: tools/traffic_vault_migrate/riak.go
##########
@@ -0,0 +1,746 @@
+package main
+
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+import (
+	"crypto/tls"
+	"encoding/json"
+	"errors"
+	"fmt"
+	"time"
+
+	"github.com/apache/trafficcontrol/lib/go-rfc"
+	"github.com/apache/trafficcontrol/lib/go-tc"
+
+	"github.com/basho/riak-go-client"
+)
+
+const (
+	BUCKET_SSL     = "ssl"
+	BUCKET_DNSSEC  = "dnssec"
+	BUCKET_URL_SIG = "url_sig_keys"
+	BUCKET_URI_SIG = "cdn_uri_sig_keys"
+
+	INDEX_SSL = "sslkeys"
+
+	SCHEMA_RIAK_KEY    = "_yz_rk"
+	SCHEMA_RIAK_BUCKET = "_yz_rb"
+)
+
+var (
+	SCHEMA_SSL_FIELDS = [...]string{SCHEMA_RIAK_KEY, SCHEMA_RIAK_BUCKET}
+)
+
+// RiakConfig  represents the configuration options available to the Riak backend
+type RiakConfig struct {
+	Host          string `json:"host"`
+	Port          string `json:"port"`
+	User          string `json:"user"`
+	Password      string `json:"password"`
+	Insecure      bool   `json:"insecure"`
+	TLSVersionRaw string `json:"tlsVersion"`
+	TLSVersion    uint16
+}
+
+// RiakBackend is the Riak implementation of TVBackend
+type RiakBackend struct {
+	sslKeys     riakSSLKeyTable
+	dnssecKeys  riakDNSSecKeyTable
+	uriSignKeys riakURISignKeyTable
+	urlSigKeys  riakURLSigKeyTable
+	cfg         RiakConfig
+	cluster     *riak.Cluster
+}
+
+// String returns a high level overview of the backend and its keys
+func (rb *RiakBackend) String() string {
+	data := fmt.Sprintf("Riak server %v@%v:%v\n", rb.cfg.User, rb.cfg.Host, rb.cfg.Port)
+	data += fmt.Sprintf("\tSSL Keys: %v\n", len(rb.sslKeys.Records))
+	data += fmt.Sprintf("\tDNSSec Keys: %v\n", len(rb.dnssecKeys.Records))
+	data += fmt.Sprintf("\tURI Keys: %v\n", len(rb.uriSignKeys.Records))
+	data += fmt.Sprintf("\tURL Keys: %v\n", len(rb.urlSigKeys.Records))
+	return data
+}
+
+// Name returns the name for this backend
+func (rb *RiakBackend) Name() string {
+	return "Riak"
+}
+
+// ReadConfig takes in a filename and will read it into the backends config
+func (rb *RiakBackend) ReadConfig(s string) error {
+	err := UnmarshalConfig(s, &rb.cfg)
+	if err != nil {
+		return err
+	}
+
+	if rb.cfg.TLSVersionRaw == "10" {
+		rb.cfg.TLSVersion = tls.VersionTLS10
+	} else if rb.cfg.TLSVersionRaw == "11" {
+		rb.cfg.TLSVersion = tls.VersionTLS11
+	} else if rb.cfg.TLSVersionRaw == "12" {
+		rb.cfg.TLSVersion = tls.VersionTLS12
+	} else if rb.cfg.TLSVersionRaw == "13" {
+		rb.cfg.TLSVersion = tls.VersionTLS13
+	}
+	return nil
+}
+
+// Insert takes the current keys and inserts them into the backend DB
+func (rb *RiakBackend) Insert() error {
+	if err := rb.sslKeys.insertKeys(rb.cluster); err != nil {
+		return err
+	}
+	if err := rb.dnssecKeys.insertKeys(rb.cluster); err != nil {
+		return err
+	}
+	if err := rb.urlSigKeys.insertKeys(rb.cluster); err != nil {
+		return err
+	}
+	if err := rb.uriSignKeys.insertKeys(rb.cluster); err != nil {
+		return err
+	}
+	return nil
+}
+
+// ValidateKey validates that the keys are valid (in most cases, certain fields are not null)
+func (rb *RiakBackend) ValidateKey() []string {
+	errors := []string{}
+	if errs := rb.sslKeys.validate(); errs != nil {
+		errors = append(errors, errs...)
+	}
+	if errs := rb.dnssecKeys.validate(); errs != nil {
+		errors = append(errors, errs...)
+	}
+	if errs := rb.uriSignKeys.validate(); errs != nil {
+		errors = append(errors, errs...)
+	}
+	if errs := rb.urlSigKeys.validate(); errs != nil {
+		errors = append(errors, errs...)
+	}
+
+	return errors
+}
+
+// SetSSLKeys takes in keys and converts & encrypts the data into the backends internal format
+func (rb *RiakBackend) SetSSLKeys(keys []SSLKey) error {
+	rb.sslKeys.fromGeneric(keys)
+	return nil
+}
+
+// SetDNSSecKeys takes in keys and converts & encrypts the data into the backends internal format
+func (rb *RiakBackend) SetDNSSecKeys(keys []DNSSecKey) error {
+	rb.dnssecKeys.fromGeneric(keys)
+	return nil
+}
+
+// SetURISignKeys takes in keys and converts & encrypts the data into the backends internal format
+func (rb *RiakBackend) SetURISignKeys(keys []URISignKey) error {
+	rb.uriSignKeys.fromGeneric(keys)
+	return nil
+}
+
+// SetURLSigKeys takes in keys and converts & encrypts the data into the backends internal format
+func (rb *RiakBackend) SetURLSigKeys(keys []URLSigKey) error {
+	rb.urlSigKeys.fromGeneric(keys)
+	return nil
+}
+
+// Start initiates the connection to the backend DB
+func (rb *RiakBackend) Start() error {
+	tlsConfig := &tls.Config{
+		InsecureSkipVerify: !rb.cfg.Insecure,
+		MaxVersion:         rb.cfg.TLSVersion,
+	}
+	auth := &riak.AuthOptions{
+		User:      rb.cfg.User,
+		Password:  rb.cfg.Password,
+		TlsConfig: tlsConfig,
+	}
+
+	cluster, err := getRiakCluster(rb.cfg, auth)
+	if err != nil {
+		return err
+	}
+	if err := cluster.Start(); err != nil {
+		return fmt.Errorf("unable to start riak cluster: %w", err)
+	}
+
+	rb.cluster = cluster
+	rb.sslKeys = riakSSLKeyTable{}
+	rb.dnssecKeys = riakDNSSecKeyTable{}
+	rb.urlSigKeys = riakURLSigKeyTable{}
+	rb.uriSignKeys = riakURISignKeyTable{}
+	return nil
+}
+
+// Close terminates the connection to the backend DB
+func (rb *RiakBackend) Close() error {
+	if err := rb.cluster.Stop(); err != nil {
+		return err
+	}
+	return nil
+}
+
+// Ping checks the connection to the backend DB
+func (rb *RiakBackend) Ping() error {
+	return ping(rb.cluster)
+}
+
+// GetSSLKeys converts the backends internal key representation into the common representation (SSLKey)
+func (rb *RiakBackend) GetSSLKeys() ([]SSLKey, error) {
+	return rb.sslKeys.toGeneric(), nil
+}
+
+// GetDNSSecKeys converts the backends internal key representation into the common representation (DNSSecKey)
+func (rb *RiakBackend) GetDNSSecKeys() ([]DNSSecKey, error) {
+	return rb.dnssecKeys.toGeneric(), nil
+}
+
+// GetURISignKeys converts the pg internal key representation into the common representation (URISignKey)
+func (rb *RiakBackend) GetURISignKeys() ([]URISignKey, error) {
+	return rb.uriSignKeys.toGeneric(), nil
+}
+
+// GetURLSigKeys converts the backends internal key representation into the common representation (URLSigKey)
+func (rb *RiakBackend) GetURLSigKeys() ([]URLSigKey, error) {
+	return rb.urlSigKeys.toGeneric(), nil
+}
+
+// Fetch gets all of the keys from the backend DB
+func (rb *RiakBackend) Fetch() error {
+	if err := rb.sslKeys.gatherKeys(rb.cluster); err != nil {
+		return err
+	}
+	if err := rb.dnssecKeys.gatherKeys(rb.cluster); err != nil {
+		return err
+	}
+	if err := rb.urlSigKeys.gatherKeys(rb.cluster); err != nil {
+		return err
+	}
+	if err := rb.uriSignKeys.gatherKeys(rb.cluster); err != nil {
+		return err
+	}
+
+	return nil
+}
+
+type riakSSLKeyRecord struct {
+	tc.DeliveryServiceSSLKeys
+}
+type riakSSLKeyTable struct {
+	Records []riakSSLKeyRecord
+}
+
+func (tbl *riakSSLKeyTable) gatherKeys(cluster *riak.Cluster) error {
+	searchDocs, err := search(cluster, INDEX_SSL, "cdn:*", fmt.Sprintf("%v:*latest", SCHEMA_RIAK_KEY), 1000, SCHEMA_SSL_FIELDS[:])
+	if err != nil {
+		return fmt.Errorf("RiakSSLKey gatherKeys: %w", err)
+	}
+
+	tbl.Records = make([]riakSSLKeyRecord, len(searchDocs))
+	for i, doc := range searchDocs {
+		objs, err := getObject(cluster, doc.Bucket, doc.Key)
+		if err != nil {
+			return err
+		}
+		if len(objs) < 1 {
+			return fmt.Errorf("RiakSSLKey gatherKeys unable to find any objects with key %v and bucket %v, but search results were returned", doc.Key, doc.Bucket)
+		}
+		if len(objs) > 1 {
+			return fmt.Errorf("RiakSSLKey gatherKeys more than 1 ssl key record found %v\n", len(objs))
+		}
+		var obj tc.DeliveryServiceSSLKeys
+		if err = json.Unmarshal(objs[0].Value, &obj); err != nil {
+			return fmt.Errorf("RiakSSLKey gatherKeys unable to unmarshal object into tc.DeliveryServiceSSLKeys: %w", err)
+		}
+		tbl.Records[i] = riakSSLKeyRecord{
+			DeliveryServiceSSLKeys: obj,
+		}
+	}
+	return nil
+}
+func (tbl *riakSSLKeyTable) toGeneric() []SSLKey {
+	keys := make([]SSLKey, len(tbl.Records))
+
+	for i, record := range tbl.Records {
+		keys[i] = SSLKey{
+			DeliveryServiceSSLKeys: record.DeliveryServiceSSLKeys,
+		}
+	}
+
+	return keys
+}
+func (tbl *riakSSLKeyTable) fromGeneric(keys []SSLKey) {
+	tbl.Records = make([]riakSSLKeyRecord, len(keys))
+
+	for i, record := range keys {
+		tbl.Records[i] = riakSSLKeyRecord{
+			DeliveryServiceSSLKeys: record.DeliveryServiceSSLKeys,
+		}
+	}
+}
+func (tbl *riakSSLKeyTable) insertKeys(cluster *riak.Cluster) error {
+	for _, record := range tbl.Records {
+		objBytes, err := json.Marshal(record.DeliveryServiceSSLKeys)
+		if err != nil {
+			return fmt.Errorf("RiakSSLKey insertKeys failed to unmarshal keys: %w", err)
+		}
+		err = setObject(cluster, makeRiakObject(objBytes, record.DeliveryService+"-"+record.Version.String()), BUCKET_SSL)
+		if err != nil {
+			return fmt.Errorf("RiakSSLKey insertKeys: %w", err)
+		}
+		err = setObject(cluster, makeRiakObject(objBytes, record.DeliveryService+"-latest"), BUCKET_SSL)
+		if err != nil {
+			return fmt.Errorf("RiakSSLKey insertKeys: %w", err)
+		}
+	}
+	return nil
+}
+func (tbl *riakSSLKeyTable) validate() []string {
+	errs := []string{}
+	for i, record := range tbl.Records {
+		if record.DeliveryService == "" {
+			errs = append(errs, fmt.Sprintf("SSL Key #%v: Delivery Service is blank!", i))
+		}
+		if record.CDN == "" {
+			errs = append(errs, fmt.Sprintf("SSL Key #%v: CDN is blank!", i))
+		}
+		if record.Version.String() == "" {
+			errs = append(errs, fmt.Sprintf("SSL Key #%v: Version is blank!", i))
+		}
+	}
+	return errs
+}
+
+type riakDNSSecKeyRecord struct {
+	CDN string
+	Key tc.DNSSECKeysRiak
+}
+type riakDNSSecKeyTable struct {
+	Records []riakDNSSecKeyRecord
+}
+
+func (tbl *riakDNSSecKeyTable) gatherKeys(cluster *riak.Cluster) error {
+	tbl.Records = []riakDNSSecKeyRecord{}
+	objs, err := getObjects(cluster, BUCKET_DNSSEC)
+	if err != nil {
+		return fmt.Errorf("RiakDNSSecKey gatherKeys: %w", err)
+	}
+	for _, obj := range objs {
+		key := tc.DNSSECKeysRiak{}
+		if err := json.Unmarshal(obj.Value, &key); err != nil {
+			return fmt.Errorf("RiakDNSSecKey gatherKeys unable to unmarshal object to tc.DNSSECKeysRiak: %w", err)
+		}
+		tbl.Records = append(tbl.Records, riakDNSSecKeyRecord{
+			CDN: obj.Key,
+			Key: key,
+		})
+	}
+	return nil
+}
+func (tbl *riakDNSSecKeyTable) toGeneric() []DNSSecKey {
+	keys := make([]DNSSecKey, len(tbl.Records))
+
+	for i, record := range tbl.Records {
+		keys[i] = DNSSecKey{
+			CDN:                    record.CDN,
+			DNSSECKeysTrafficVault: tc.DNSSECKeysTrafficVault(record.Key),
+		}
+	}
+
+	return keys
+}
+func (tbl *riakDNSSecKeyTable) fromGeneric(keys []DNSSecKey) {
+	tbl.Records = make([]riakDNSSecKeyRecord, len(keys))
+
+	for i, record := range keys {
+		tbl.Records[i] = riakDNSSecKeyRecord{
+			CDN: record.CDN,
+			Key: tc.DNSSECKeysRiak(record.DNSSECKeysTrafficVault),
+		}
+	}
+}
+func (tbl *riakDNSSecKeyTable) insertKeys(cluster *riak.Cluster) error {
+	for _, record := range tbl.Records {
+		objBytes, err := json.Marshal(record.Key)
+		if err != nil {
+			return fmt.Errorf("RiakDNSSecKey insertKeys error marshalling keys: %w", err)
+		}
+
+		err = setObject(cluster, makeRiakObject(objBytes, record.CDN), BUCKET_DNSSEC)
+		if err != nil {
+			return fmt.Errorf("RiakDNSSecKey insertKeys: %w", err)
+		}
+	}
+	return nil
+}
+func (tbl *riakDNSSecKeyTable) validate() []string {
+	errs := []string{}
+	for i, record := range tbl.Records {
+		if record.CDN == "" {
+			errs = append(errs, fmt.Sprintf("DNSSec Key #%v: CDN is blank!", i))
+		}
+	}
+	return errs
+}
+
+type riakURLSigKeyRecord struct {
+	Key             tc.URLSigKeys
+	DeliveryService string
+}
+type riakURLSigKeyTable struct {
+	Records []riakURLSigKeyRecord
+}
+
+func (tbl *riakURLSigKeyTable) gatherKeys(cluster *riak.Cluster) error {
+	tbl.Records = []riakURLSigKeyRecord{}
+	objs, err := getObjects(cluster, BUCKET_URL_SIG)
+	if err != nil {
+		return fmt.Errorf("RiakURLSigKey gatherKeys: %w", err)
+	}
+	for _, obj := range objs {
+		key := tc.URLSigKeys{}
+		if err := json.Unmarshal(obj.Value, &key); err != nil {
+			return fmt.Errorf("RiakURLSigKey gatherKeys unable to unamrshal object into tc.URLSigKeys: %w", err)
+		}
+		tbl.Records = append(tbl.Records, riakURLSigKeyRecord{
+			DeliveryService: obj.Key,
+			Key:             key,
+		})
+	}
+	return nil
+}
+func (tbl *riakURLSigKeyTable) toGeneric() []URLSigKey {
+	keys := make([]URLSigKey, len(tbl.Records))
+
+	for i, record := range tbl.Records {
+		keys[i] = URLSigKey{
+			URLSigKeys:      record.Key,
+			DeliveryService: record.DeliveryService,
+		}
+	}
+
+	return keys
+}
+func (tbl *riakURLSigKeyTable) fromGeneric(keys []URLSigKey) {
+	tbl.Records = make([]riakURLSigKeyRecord, len(keys))
+	for i, key := range keys {
+		tbl.Records[i] = riakURLSigKeyRecord{
+			Key:             key.URLSigKeys,
+			DeliveryService: key.DeliveryService,
+		}
+	}
+}
+func (tbl *riakURLSigKeyTable) insertKeys(cluster *riak.Cluster) error {
+	for _, record := range tbl.Records {
+		objBytes, err := json.Marshal(record.Key)
+		if err != nil {
+			return fmt.Errorf("RiakURLSigKey insertKeys unable to marshal keys: %w", err)
+		}
+
+		err = setObject(cluster, makeRiakObject(objBytes, "url_sig_"+record.DeliveryService+".config"), BUCKET_URL_SIG)
+		if err != nil {
+			return fmt.Errorf("RiakURLSigKey insertKeys: %w", err)
+		}
+	}
+	return nil
+}
+func (tbl *riakURLSigKeyTable) validate() []string {
+	errs := []string{}
+	for i, record := range tbl.Records {
+		if record.DeliveryService == "" {
+			errs = append(errs, fmt.Sprintf("URL Key #%v: Delivery Service is blank!", i))
+		}
+	}
+	return errs
+}
+
+type riakURISignKeyRecord struct {
+	Key             map[string]tc.URISignerKeyset
+	DeliveryService string
+}
+type riakURISignKeyTable struct {
+	Records []riakURISignKeyRecord
+}
+
+func (tbl *riakURISignKeyTable) gatherKeys(cluster *riak.Cluster) error {
+	tbl.Records = []riakURISignKeyRecord{}
+	objs, err := getObjects(cluster, BUCKET_URI_SIG)
+	if err != nil {
+		return fmt.Errorf("RiakURISignKey gatherKeys: %w", err)
+	}
+	for _, obj := range objs {
+		key := map[string]tc.URISignerKeyset{}
+		if err := json.Unmarshal(obj.Value, &key); err != nil {
+			return fmt.Errorf("RiakURISignKey gatherKeys unable to unmarshal object into map[string]tc.URISignerKeySet: %w", err)

Review comment:
       this error message should probably include the `obj.Key` for troubleshooting

##########
File path: tools/traffic_vault_migrate/riak.go
##########
@@ -0,0 +1,746 @@
+package main
+
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+import (
+	"crypto/tls"
+	"encoding/json"
+	"errors"
+	"fmt"
+	"time"
+
+	"github.com/apache/trafficcontrol/lib/go-rfc"
+	"github.com/apache/trafficcontrol/lib/go-tc"
+
+	"github.com/basho/riak-go-client"
+)
+
+const (
+	BUCKET_SSL     = "ssl"
+	BUCKET_DNSSEC  = "dnssec"
+	BUCKET_URL_SIG = "url_sig_keys"
+	BUCKET_URI_SIG = "cdn_uri_sig_keys"
+
+	INDEX_SSL = "sslkeys"
+
+	SCHEMA_RIAK_KEY    = "_yz_rk"
+	SCHEMA_RIAK_BUCKET = "_yz_rb"
+)
+
+var (
+	SCHEMA_SSL_FIELDS = [...]string{SCHEMA_RIAK_KEY, SCHEMA_RIAK_BUCKET}
+)
+
+// RiakConfig  represents the configuration options available to the Riak backend
+type RiakConfig struct {
+	Host          string `json:"host"`
+	Port          string `json:"port"`
+	User          string `json:"user"`
+	Password      string `json:"password"`
+	Insecure      bool   `json:"insecure"`
+	TLSVersionRaw string `json:"tlsVersion"`
+	TLSVersion    uint16
+}
+
+// RiakBackend is the Riak implementation of TVBackend
+type RiakBackend struct {
+	sslKeys     riakSSLKeyTable
+	dnssecKeys  riakDNSSecKeyTable
+	uriSignKeys riakURISignKeyTable
+	urlSigKeys  riakURLSigKeyTable
+	cfg         RiakConfig
+	cluster     *riak.Cluster
+}
+
+// String returns a high level overview of the backend and its keys
+func (rb *RiakBackend) String() string {
+	data := fmt.Sprintf("Riak server %v@%v:%v\n", rb.cfg.User, rb.cfg.Host, rb.cfg.Port)
+	data += fmt.Sprintf("\tSSL Keys: %v\n", len(rb.sslKeys.Records))
+	data += fmt.Sprintf("\tDNSSec Keys: %v\n", len(rb.dnssecKeys.Records))
+	data += fmt.Sprintf("\tURI Keys: %v\n", len(rb.uriSignKeys.Records))
+	data += fmt.Sprintf("\tURL Keys: %v\n", len(rb.urlSigKeys.Records))
+	return data
+}
+
+// Name returns the name for this backend
+func (rb *RiakBackend) Name() string {
+	return "Riak"
+}
+
+// ReadConfig takes in a filename and will read it into the backends config
+func (rb *RiakBackend) ReadConfig(s string) error {
+	err := UnmarshalConfig(s, &rb.cfg)
+	if err != nil {
+		return err
+	}
+
+	if rb.cfg.TLSVersionRaw == "10" {
+		rb.cfg.TLSVersion = tls.VersionTLS10
+	} else if rb.cfg.TLSVersionRaw == "11" {
+		rb.cfg.TLSVersion = tls.VersionTLS11
+	} else if rb.cfg.TLSVersionRaw == "12" {
+		rb.cfg.TLSVersion = tls.VersionTLS12
+	} else if rb.cfg.TLSVersionRaw == "13" {
+		rb.cfg.TLSVersion = tls.VersionTLS13
+	}
+	return nil
+}
+
+// Insert takes the current keys and inserts them into the backend DB
+func (rb *RiakBackend) Insert() error {
+	if err := rb.sslKeys.insertKeys(rb.cluster); err != nil {
+		return err
+	}
+	if err := rb.dnssecKeys.insertKeys(rb.cluster); err != nil {
+		return err
+	}
+	if err := rb.urlSigKeys.insertKeys(rb.cluster); err != nil {
+		return err
+	}
+	if err := rb.uriSignKeys.insertKeys(rb.cluster); err != nil {
+		return err
+	}
+	return nil
+}
+
+// ValidateKey validates that the keys are valid (in most cases, certain fields are not null)
+func (rb *RiakBackend) ValidateKey() []string {
+	errors := []string{}
+	if errs := rb.sslKeys.validate(); errs != nil {
+		errors = append(errors, errs...)
+	}
+	if errs := rb.dnssecKeys.validate(); errs != nil {
+		errors = append(errors, errs...)
+	}
+	if errs := rb.uriSignKeys.validate(); errs != nil {
+		errors = append(errors, errs...)
+	}
+	if errs := rb.urlSigKeys.validate(); errs != nil {
+		errors = append(errors, errs...)
+	}
+
+	return errors
+}
+
+// SetSSLKeys takes in keys and converts & encrypts the data into the backends internal format
+func (rb *RiakBackend) SetSSLKeys(keys []SSLKey) error {
+	rb.sslKeys.fromGeneric(keys)
+	return nil
+}
+
+// SetDNSSecKeys takes in keys and converts & encrypts the data into the backends internal format
+func (rb *RiakBackend) SetDNSSecKeys(keys []DNSSecKey) error {
+	rb.dnssecKeys.fromGeneric(keys)
+	return nil
+}
+
+// SetURISignKeys takes in keys and converts & encrypts the data into the backends internal format
+func (rb *RiakBackend) SetURISignKeys(keys []URISignKey) error {
+	rb.uriSignKeys.fromGeneric(keys)
+	return nil
+}
+
+// SetURLSigKeys takes in keys and converts & encrypts the data into the backends internal format
+func (rb *RiakBackend) SetURLSigKeys(keys []URLSigKey) error {
+	rb.urlSigKeys.fromGeneric(keys)
+	return nil
+}
+
+// Start initiates the connection to the backend DB
+func (rb *RiakBackend) Start() error {
+	tlsConfig := &tls.Config{
+		InsecureSkipVerify: !rb.cfg.Insecure,
+		MaxVersion:         rb.cfg.TLSVersion,
+	}
+	auth := &riak.AuthOptions{
+		User:      rb.cfg.User,
+		Password:  rb.cfg.Password,
+		TlsConfig: tlsConfig,
+	}
+
+	cluster, err := getRiakCluster(rb.cfg, auth)
+	if err != nil {
+		return err
+	}
+	if err := cluster.Start(); err != nil {
+		return fmt.Errorf("unable to start riak cluster: %w", err)
+	}
+
+	rb.cluster = cluster
+	rb.sslKeys = riakSSLKeyTable{}
+	rb.dnssecKeys = riakDNSSecKeyTable{}
+	rb.urlSigKeys = riakURLSigKeyTable{}
+	rb.uriSignKeys = riakURISignKeyTable{}
+	return nil
+}
+
+// Close terminates the connection to the backend DB
+func (rb *RiakBackend) Close() error {
+	if err := rb.cluster.Stop(); err != nil {
+		return err
+	}
+	return nil
+}
+
+// Ping checks the connection to the backend DB
+func (rb *RiakBackend) Ping() error {
+	return ping(rb.cluster)
+}
+
+// GetSSLKeys converts the backends internal key representation into the common representation (SSLKey)
+func (rb *RiakBackend) GetSSLKeys() ([]SSLKey, error) {
+	return rb.sslKeys.toGeneric(), nil
+}
+
+// GetDNSSecKeys converts the backends internal key representation into the common representation (DNSSecKey)
+func (rb *RiakBackend) GetDNSSecKeys() ([]DNSSecKey, error) {
+	return rb.dnssecKeys.toGeneric(), nil
+}
+
+// GetURISignKeys converts the pg internal key representation into the common representation (URISignKey)
+func (rb *RiakBackend) GetURISignKeys() ([]URISignKey, error) {
+	return rb.uriSignKeys.toGeneric(), nil
+}
+
+// GetURLSigKeys converts the backends internal key representation into the common representation (URLSigKey)
+func (rb *RiakBackend) GetURLSigKeys() ([]URLSigKey, error) {
+	return rb.urlSigKeys.toGeneric(), nil
+}
+
+// Fetch gets all of the keys from the backend DB
+func (rb *RiakBackend) Fetch() error {
+	if err := rb.sslKeys.gatherKeys(rb.cluster); err != nil {
+		return err
+	}
+	if err := rb.dnssecKeys.gatherKeys(rb.cluster); err != nil {
+		return err
+	}
+	if err := rb.urlSigKeys.gatherKeys(rb.cluster); err != nil {
+		return err
+	}
+	if err := rb.uriSignKeys.gatherKeys(rb.cluster); err != nil {
+		return err
+	}
+
+	return nil
+}
+
+type riakSSLKeyRecord struct {
+	tc.DeliveryServiceSSLKeys
+}
+type riakSSLKeyTable struct {
+	Records []riakSSLKeyRecord
+}
+
+func (tbl *riakSSLKeyTable) gatherKeys(cluster *riak.Cluster) error {
+	searchDocs, err := search(cluster, INDEX_SSL, "cdn:*", fmt.Sprintf("%v:*latest", SCHEMA_RIAK_KEY), 1000, SCHEMA_SSL_FIELDS[:])
+	if err != nil {
+		return fmt.Errorf("RiakSSLKey gatherKeys: %w", err)
+	}
+
+	tbl.Records = make([]riakSSLKeyRecord, len(searchDocs))
+	for i, doc := range searchDocs {
+		objs, err := getObject(cluster, doc.Bucket, doc.Key)
+		if err != nil {
+			return err
+		}
+		if len(objs) < 1 {
+			return fmt.Errorf("RiakSSLKey gatherKeys unable to find any objects with key %v and bucket %v, but search results were returned", doc.Key, doc.Bucket)
+		}
+		if len(objs) > 1 {
+			return fmt.Errorf("RiakSSLKey gatherKeys more than 1 ssl key record found %v\n", len(objs))
+		}
+		var obj tc.DeliveryServiceSSLKeys
+		if err = json.Unmarshal(objs[0].Value, &obj); err != nil {
+			return fmt.Errorf("RiakSSLKey gatherKeys unable to unmarshal object into tc.DeliveryServiceSSLKeys: %w", err)
+		}
+		tbl.Records[i] = riakSSLKeyRecord{
+			DeliveryServiceSSLKeys: obj,
+		}
+	}
+	return nil
+}
+func (tbl *riakSSLKeyTable) toGeneric() []SSLKey {
+	keys := make([]SSLKey, len(tbl.Records))
+
+	for i, record := range tbl.Records {
+		keys[i] = SSLKey{
+			DeliveryServiceSSLKeys: record.DeliveryServiceSSLKeys,
+		}
+	}
+
+	return keys
+}
+func (tbl *riakSSLKeyTable) fromGeneric(keys []SSLKey) {
+	tbl.Records = make([]riakSSLKeyRecord, len(keys))
+
+	for i, record := range keys {
+		tbl.Records[i] = riakSSLKeyRecord{
+			DeliveryServiceSSLKeys: record.DeliveryServiceSSLKeys,
+		}
+	}
+}
+func (tbl *riakSSLKeyTable) insertKeys(cluster *riak.Cluster) error {
+	for _, record := range tbl.Records {
+		objBytes, err := json.Marshal(record.DeliveryServiceSSLKeys)
+		if err != nil {
+			return fmt.Errorf("RiakSSLKey insertKeys failed to unmarshal keys: %w", err)
+		}
+		err = setObject(cluster, makeRiakObject(objBytes, record.DeliveryService+"-"+record.Version.String()), BUCKET_SSL)
+		if err != nil {
+			return fmt.Errorf("RiakSSLKey insertKeys: %w", err)
+		}
+		err = setObject(cluster, makeRiakObject(objBytes, record.DeliveryService+"-latest"), BUCKET_SSL)
+		if err != nil {
+			return fmt.Errorf("RiakSSLKey insertKeys: %w", err)
+		}
+	}
+	return nil
+}
+func (tbl *riakSSLKeyTable) validate() []string {
+	errs := []string{}
+	for i, record := range tbl.Records {
+		if record.DeliveryService == "" {
+			errs = append(errs, fmt.Sprintf("SSL Key #%v: Delivery Service is blank!", i))
+		}
+		if record.CDN == "" {
+			errs = append(errs, fmt.Sprintf("SSL Key #%v: CDN is blank!", i))
+		}
+		if record.Version.String() == "" {
+			errs = append(errs, fmt.Sprintf("SSL Key #%v: Version is blank!", i))
+		}
+	}
+	return errs
+}
+
+type riakDNSSecKeyRecord struct {
+	CDN string
+	Key tc.DNSSECKeysRiak
+}
+type riakDNSSecKeyTable struct {
+	Records []riakDNSSecKeyRecord
+}
+
+func (tbl *riakDNSSecKeyTable) gatherKeys(cluster *riak.Cluster) error {
+	tbl.Records = []riakDNSSecKeyRecord{}
+	objs, err := getObjects(cluster, BUCKET_DNSSEC)
+	if err != nil {
+		return fmt.Errorf("RiakDNSSecKey gatherKeys: %w", err)
+	}
+	for _, obj := range objs {
+		key := tc.DNSSECKeysRiak{}
+		if err := json.Unmarshal(obj.Value, &key); err != nil {
+			return fmt.Errorf("RiakDNSSecKey gatherKeys unable to unmarshal object to tc.DNSSECKeysRiak: %w", err)
+		}
+		tbl.Records = append(tbl.Records, riakDNSSecKeyRecord{
+			CDN: obj.Key,
+			Key: key,
+		})
+	}
+	return nil
+}
+func (tbl *riakDNSSecKeyTable) toGeneric() []DNSSecKey {
+	keys := make([]DNSSecKey, len(tbl.Records))
+
+	for i, record := range tbl.Records {
+		keys[i] = DNSSecKey{
+			CDN:                    record.CDN,
+			DNSSECKeysTrafficVault: tc.DNSSECKeysTrafficVault(record.Key),
+		}
+	}
+
+	return keys
+}
+func (tbl *riakDNSSecKeyTable) fromGeneric(keys []DNSSecKey) {
+	tbl.Records = make([]riakDNSSecKeyRecord, len(keys))
+
+	for i, record := range keys {
+		tbl.Records[i] = riakDNSSecKeyRecord{
+			CDN: record.CDN,
+			Key: tc.DNSSECKeysRiak(record.DNSSECKeysTrafficVault),
+		}
+	}
+}
+func (tbl *riakDNSSecKeyTable) insertKeys(cluster *riak.Cluster) error {
+	for _, record := range tbl.Records {
+		objBytes, err := json.Marshal(record.Key)
+		if err != nil {
+			return fmt.Errorf("RiakDNSSecKey insertKeys error marshalling keys: %w", err)
+		}
+
+		err = setObject(cluster, makeRiakObject(objBytes, record.CDN), BUCKET_DNSSEC)
+		if err != nil {
+			return fmt.Errorf("RiakDNSSecKey insertKeys: %w", err)
+		}
+	}
+	return nil
+}
+func (tbl *riakDNSSecKeyTable) validate() []string {
+	errs := []string{}
+	for i, record := range tbl.Records {
+		if record.CDN == "" {
+			errs = append(errs, fmt.Sprintf("DNSSec Key #%v: CDN is blank!", i))
+		}
+	}
+	return errs
+}
+
+type riakURLSigKeyRecord struct {
+	Key             tc.URLSigKeys
+	DeliveryService string
+}
+type riakURLSigKeyTable struct {
+	Records []riakURLSigKeyRecord
+}
+
+func (tbl *riakURLSigKeyTable) gatherKeys(cluster *riak.Cluster) error {
+	tbl.Records = []riakURLSigKeyRecord{}
+	objs, err := getObjects(cluster, BUCKET_URL_SIG)
+	if err != nil {
+		return fmt.Errorf("RiakURLSigKey gatherKeys: %w", err)
+	}
+	for _, obj := range objs {
+		key := tc.URLSigKeys{}
+		if err := json.Unmarshal(obj.Value, &key); err != nil {
+			return fmt.Errorf("RiakURLSigKey gatherKeys unable to unamrshal object into tc.URLSigKeys: %w", err)
+		}
+		tbl.Records = append(tbl.Records, riakURLSigKeyRecord{
+			DeliveryService: obj.Key,
+			Key:             key,
+		})
+	}
+	return nil
+}
+func (tbl *riakURLSigKeyTable) toGeneric() []URLSigKey {
+	keys := make([]URLSigKey, len(tbl.Records))
+
+	for i, record := range tbl.Records {
+		keys[i] = URLSigKey{
+			URLSigKeys:      record.Key,
+			DeliveryService: record.DeliveryService,
+		}
+	}
+
+	return keys
+}
+func (tbl *riakURLSigKeyTable) fromGeneric(keys []URLSigKey) {
+	tbl.Records = make([]riakURLSigKeyRecord, len(keys))
+	for i, key := range keys {
+		tbl.Records[i] = riakURLSigKeyRecord{
+			Key:             key.URLSigKeys,
+			DeliveryService: key.DeliveryService,
+		}
+	}
+}
+func (tbl *riakURLSigKeyTable) insertKeys(cluster *riak.Cluster) error {
+	for _, record := range tbl.Records {
+		objBytes, err := json.Marshal(record.Key)
+		if err != nil {
+			return fmt.Errorf("RiakURLSigKey insertKeys unable to marshal keys: %w", err)
+		}
+
+		err = setObject(cluster, makeRiakObject(objBytes, "url_sig_"+record.DeliveryService+".config"), BUCKET_URL_SIG)
+		if err != nil {
+			return fmt.Errorf("RiakURLSigKey insertKeys: %w", err)
+		}
+	}
+	return nil
+}
+func (tbl *riakURLSigKeyTable) validate() []string {
+	errs := []string{}
+	for i, record := range tbl.Records {
+		if record.DeliveryService == "" {
+			errs = append(errs, fmt.Sprintf("URL Key #%v: Delivery Service is blank!", i))
+		}
+	}
+	return errs
+}
+
+type riakURISignKeyRecord struct {
+	Key             map[string]tc.URISignerKeyset
+	DeliveryService string
+}
+type riakURISignKeyTable struct {
+	Records []riakURISignKeyRecord
+}
+
+func (tbl *riakURISignKeyTable) gatherKeys(cluster *riak.Cluster) error {
+	tbl.Records = []riakURISignKeyRecord{}
+	objs, err := getObjects(cluster, BUCKET_URI_SIG)
+	if err != nil {
+		return fmt.Errorf("RiakURISignKey gatherKeys: %w", err)
+	}
+	for _, obj := range objs {
+		key := map[string]tc.URISignerKeyset{}
+		if err := json.Unmarshal(obj.Value, &key); err != nil {
+			return fmt.Errorf("RiakURISignKey gatherKeys unable to unmarshal object into map[string]tc.URISignerKeySet: %w", err)
+		}
+
+		tbl.Records = append(tbl.Records, riakURISignKeyRecord{
+			DeliveryService: obj.Key,
+			Key:             key,
+		})
+	}
+	return nil
+}
+func (tbl *riakURISignKeyTable) toGeneric() []URISignKey {
+	keys := make([]URISignKey, len(tbl.Records))
+
+	for i, record := range tbl.Records {
+		keys[i] = URISignKey{
+			DeliveryService: record.DeliveryService,
+			Keys:            record.Key,
+		}
+	}
+
+	return keys
+}
+func (tbl *riakURISignKeyTable) fromGeneric(keys []URISignKey) {
+	tbl.Records = make([]riakURISignKeyRecord, len(keys))
+
+	for i, record := range keys {
+		tbl.Records[i] = riakURISignKeyRecord{
+			Key:             record.Keys,
+			DeliveryService: record.DeliveryService,
+		}
+	}
+}
+func (tbl *riakURISignKeyTable) insertKeys(cluster *riak.Cluster) error {
+	for _, record := range tbl.Records {
+		objBytes, err := json.Marshal(record.Key)
+		if err != nil {
+			return fmt.Errorf("RiakURISignKey insertKeys: unable to marshal key: %w", err)
+		}
+
+		err = setObject(cluster, makeRiakObject(objBytes, record.DeliveryService), BUCKET_URI_SIG)
+		if err != nil {
+			return fmt.Errorf("RiakURISignKey insertKeys: %w", err)
+		}
+	}
+	return nil
+}
+func (tbl *riakURISignKeyTable) validate() []string {
+	errs := []string{}
+	for i, record := range tbl.Records {
+		if record.DeliveryService == "" {
+			errs = append(errs, fmt.Sprintf("URI Key #%v: Delivery Service is blank!", i))
+		}
+	}
+	return errs
+}
+
+// Riak functions
+func makeRiakObject(data []byte, key string) *riak.Object {
+	return &riak.Object{
+		ContentType:     rfc.ApplicationJSON,
+		Charset:         "utf-8",
+		ContentEncoding: "utf-8",
+		Key:             key,
+		Value:           data,
+	}
+}
+func getObjects(cluster *riak.Cluster, bucket string) ([]*riak.Object, error) {
+	objs := []*riak.Object{}
+	keys, err := listKeys(cluster, bucket)
+	if err != nil {
+		return nil, err
+	}
+	for _, key := range keys {
+		objects, err := getObject(cluster, bucket, key)
+		if err != nil {
+			return nil, err
+		}
+		if len(objects) > 1 {
+			return nil, fmt.Errorf("Unexpected number of objects %v, ignoring\n", len(objects))

Review comment:
       nit: type-specific formatting instead of %v
   
   Also,`ignoring` doesn't really make sense in this error message, because instead of getting ignored, the error gets passed up, eventually causing the program to exit.

##########
File path: tools/traffic_vault_migrate/riak.go
##########
@@ -0,0 +1,746 @@
+package main
+
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+import (
+	"crypto/tls"
+	"encoding/json"
+	"errors"
+	"fmt"
+	"time"
+
+	"github.com/apache/trafficcontrol/lib/go-rfc"
+	"github.com/apache/trafficcontrol/lib/go-tc"
+
+	"github.com/basho/riak-go-client"
+)
+
+const (
+	BUCKET_SSL     = "ssl"
+	BUCKET_DNSSEC  = "dnssec"
+	BUCKET_URL_SIG = "url_sig_keys"
+	BUCKET_URI_SIG = "cdn_uri_sig_keys"
+
+	INDEX_SSL = "sslkeys"
+
+	SCHEMA_RIAK_KEY    = "_yz_rk"
+	SCHEMA_RIAK_BUCKET = "_yz_rb"
+)
+
+var (
+	SCHEMA_SSL_FIELDS = [...]string{SCHEMA_RIAK_KEY, SCHEMA_RIAK_BUCKET}
+)
+
+// RiakConfig  represents the configuration options available to the Riak backend
+type RiakConfig struct {
+	Host          string `json:"host"`
+	Port          string `json:"port"`
+	User          string `json:"user"`
+	Password      string `json:"password"`
+	Insecure      bool   `json:"insecure"`
+	TLSVersionRaw string `json:"tlsVersion"`
+	TLSVersion    uint16
+}
+
+// RiakBackend is the Riak implementation of TVBackend
+type RiakBackend struct {
+	sslKeys     riakSSLKeyTable
+	dnssecKeys  riakDNSSecKeyTable
+	uriSignKeys riakURISignKeyTable
+	urlSigKeys  riakURLSigKeyTable
+	cfg         RiakConfig
+	cluster     *riak.Cluster
+}
+
+// String returns a high level overview of the backend and its keys
+func (rb *RiakBackend) String() string {
+	data := fmt.Sprintf("Riak server %v@%v:%v\n", rb.cfg.User, rb.cfg.Host, rb.cfg.Port)
+	data += fmt.Sprintf("\tSSL Keys: %v\n", len(rb.sslKeys.Records))
+	data += fmt.Sprintf("\tDNSSec Keys: %v\n", len(rb.dnssecKeys.Records))
+	data += fmt.Sprintf("\tURI Keys: %v\n", len(rb.uriSignKeys.Records))
+	data += fmt.Sprintf("\tURL Keys: %v\n", len(rb.urlSigKeys.Records))
+	return data
+}
+
+// Name returns the name for this backend
+func (rb *RiakBackend) Name() string {
+	return "Riak"
+}
+
+// ReadConfig takes in a filename and will read it into the backends config
+func (rb *RiakBackend) ReadConfig(s string) error {
+	err := UnmarshalConfig(s, &rb.cfg)
+	if err != nil {
+		return err
+	}
+
+	if rb.cfg.TLSVersionRaw == "10" {
+		rb.cfg.TLSVersion = tls.VersionTLS10
+	} else if rb.cfg.TLSVersionRaw == "11" {
+		rb.cfg.TLSVersion = tls.VersionTLS11
+	} else if rb.cfg.TLSVersionRaw == "12" {
+		rb.cfg.TLSVersion = tls.VersionTLS12
+	} else if rb.cfg.TLSVersionRaw == "13" {
+		rb.cfg.TLSVersion = tls.VersionTLS13
+	}
+	return nil
+}
+
+// Insert takes the current keys and inserts them into the backend DB
+func (rb *RiakBackend) Insert() error {
+	if err := rb.sslKeys.insertKeys(rb.cluster); err != nil {
+		return err
+	}
+	if err := rb.dnssecKeys.insertKeys(rb.cluster); err != nil {
+		return err
+	}
+	if err := rb.urlSigKeys.insertKeys(rb.cluster); err != nil {
+		return err
+	}
+	if err := rb.uriSignKeys.insertKeys(rb.cluster); err != nil {
+		return err
+	}
+	return nil
+}
+
+// ValidateKey validates that the keys are valid (in most cases, certain fields are not null)
+func (rb *RiakBackend) ValidateKey() []string {
+	errors := []string{}
+	if errs := rb.sslKeys.validate(); errs != nil {
+		errors = append(errors, errs...)
+	}
+	if errs := rb.dnssecKeys.validate(); errs != nil {
+		errors = append(errors, errs...)
+	}
+	if errs := rb.uriSignKeys.validate(); errs != nil {
+		errors = append(errors, errs...)
+	}
+	if errs := rb.urlSigKeys.validate(); errs != nil {
+		errors = append(errors, errs...)
+	}
+
+	return errors
+}
+
+// SetSSLKeys takes in keys and converts & encrypts the data into the backends internal format
+func (rb *RiakBackend) SetSSLKeys(keys []SSLKey) error {
+	rb.sslKeys.fromGeneric(keys)
+	return nil
+}
+
+// SetDNSSecKeys takes in keys and converts & encrypts the data into the backends internal format
+func (rb *RiakBackend) SetDNSSecKeys(keys []DNSSecKey) error {
+	rb.dnssecKeys.fromGeneric(keys)
+	return nil
+}
+
+// SetURISignKeys takes in keys and converts & encrypts the data into the backends internal format
+func (rb *RiakBackend) SetURISignKeys(keys []URISignKey) error {
+	rb.uriSignKeys.fromGeneric(keys)
+	return nil
+}
+
+// SetURLSigKeys takes in keys and converts & encrypts the data into the backends internal format
+func (rb *RiakBackend) SetURLSigKeys(keys []URLSigKey) error {
+	rb.urlSigKeys.fromGeneric(keys)
+	return nil
+}
+
+// Start initiates the connection to the backend DB
+func (rb *RiakBackend) Start() error {
+	tlsConfig := &tls.Config{
+		InsecureSkipVerify: !rb.cfg.Insecure,
+		MaxVersion:         rb.cfg.TLSVersion,
+	}
+	auth := &riak.AuthOptions{
+		User:      rb.cfg.User,
+		Password:  rb.cfg.Password,
+		TlsConfig: tlsConfig,
+	}
+
+	cluster, err := getRiakCluster(rb.cfg, auth)
+	if err != nil {
+		return err
+	}
+	if err := cluster.Start(); err != nil {
+		return fmt.Errorf("unable to start riak cluster: %w", err)
+	}
+
+	rb.cluster = cluster
+	rb.sslKeys = riakSSLKeyTable{}
+	rb.dnssecKeys = riakDNSSecKeyTable{}
+	rb.urlSigKeys = riakURLSigKeyTable{}
+	rb.uriSignKeys = riakURISignKeyTable{}
+	return nil
+}
+
+// Close terminates the connection to the backend DB
+func (rb *RiakBackend) Close() error {
+	if err := rb.cluster.Stop(); err != nil {
+		return err
+	}
+	return nil
+}
+
+// Ping checks the connection to the backend DB
+func (rb *RiakBackend) Ping() error {
+	return ping(rb.cluster)
+}
+
+// GetSSLKeys converts the backends internal key representation into the common representation (SSLKey)
+func (rb *RiakBackend) GetSSLKeys() ([]SSLKey, error) {
+	return rb.sslKeys.toGeneric(), nil
+}
+
+// GetDNSSecKeys converts the backends internal key representation into the common representation (DNSSecKey)
+func (rb *RiakBackend) GetDNSSecKeys() ([]DNSSecKey, error) {
+	return rb.dnssecKeys.toGeneric(), nil
+}
+
+// GetURISignKeys converts the pg internal key representation into the common representation (URISignKey)
+func (rb *RiakBackend) GetURISignKeys() ([]URISignKey, error) {
+	return rb.uriSignKeys.toGeneric(), nil
+}
+
+// GetURLSigKeys converts the backends internal key representation into the common representation (URLSigKey)
+func (rb *RiakBackend) GetURLSigKeys() ([]URLSigKey, error) {
+	return rb.urlSigKeys.toGeneric(), nil
+}
+
+// Fetch gets all of the keys from the backend DB
+func (rb *RiakBackend) Fetch() error {
+	if err := rb.sslKeys.gatherKeys(rb.cluster); err != nil {
+		return err
+	}
+	if err := rb.dnssecKeys.gatherKeys(rb.cluster); err != nil {
+		return err
+	}
+	if err := rb.urlSigKeys.gatherKeys(rb.cluster); err != nil {
+		return err
+	}
+	if err := rb.uriSignKeys.gatherKeys(rb.cluster); err != nil {
+		return err
+	}
+
+	return nil
+}
+
+type riakSSLKeyRecord struct {
+	tc.DeliveryServiceSSLKeys
+}
+type riakSSLKeyTable struct {
+	Records []riakSSLKeyRecord
+}
+
+func (tbl *riakSSLKeyTable) gatherKeys(cluster *riak.Cluster) error {
+	searchDocs, err := search(cluster, INDEX_SSL, "cdn:*", fmt.Sprintf("%v:*latest", SCHEMA_RIAK_KEY), 1000, SCHEMA_SSL_FIELDS[:])
+	if err != nil {
+		return fmt.Errorf("RiakSSLKey gatherKeys: %w", err)
+	}
+
+	tbl.Records = make([]riakSSLKeyRecord, len(searchDocs))
+	for i, doc := range searchDocs {
+		objs, err := getObject(cluster, doc.Bucket, doc.Key)
+		if err != nil {
+			return err
+		}
+		if len(objs) < 1 {
+			return fmt.Errorf("RiakSSLKey gatherKeys unable to find any objects with key %v and bucket %v, but search results were returned", doc.Key, doc.Bucket)
+		}
+		if len(objs) > 1 {
+			return fmt.Errorf("RiakSSLKey gatherKeys more than 1 ssl key record found %v\n", len(objs))
+		}
+		var obj tc.DeliveryServiceSSLKeys
+		if err = json.Unmarshal(objs[0].Value, &obj); err != nil {
+			return fmt.Errorf("RiakSSLKey gatherKeys unable to unmarshal object into tc.DeliveryServiceSSLKeys: %w", err)
+		}
+		tbl.Records[i] = riakSSLKeyRecord{
+			DeliveryServiceSSLKeys: obj,
+		}
+	}
+	return nil
+}
+func (tbl *riakSSLKeyTable) toGeneric() []SSLKey {
+	keys := make([]SSLKey, len(tbl.Records))
+
+	for i, record := range tbl.Records {
+		keys[i] = SSLKey{
+			DeliveryServiceSSLKeys: record.DeliveryServiceSSLKeys,
+		}
+	}
+
+	return keys
+}
+func (tbl *riakSSLKeyTable) fromGeneric(keys []SSLKey) {
+	tbl.Records = make([]riakSSLKeyRecord, len(keys))
+
+	for i, record := range keys {
+		tbl.Records[i] = riakSSLKeyRecord{
+			DeliveryServiceSSLKeys: record.DeliveryServiceSSLKeys,
+		}
+	}
+}
+func (tbl *riakSSLKeyTable) insertKeys(cluster *riak.Cluster) error {
+	for _, record := range tbl.Records {
+		objBytes, err := json.Marshal(record.DeliveryServiceSSLKeys)
+		if err != nil {
+			return fmt.Errorf("RiakSSLKey insertKeys failed to unmarshal keys: %w", err)
+		}
+		err = setObject(cluster, makeRiakObject(objBytes, record.DeliveryService+"-"+record.Version.String()), BUCKET_SSL)
+		if err != nil {
+			return fmt.Errorf("RiakSSLKey insertKeys: %w", err)
+		}
+		err = setObject(cluster, makeRiakObject(objBytes, record.DeliveryService+"-latest"), BUCKET_SSL)
+		if err != nil {
+			return fmt.Errorf("RiakSSLKey insertKeys: %w", err)
+		}
+	}
+	return nil
+}
+func (tbl *riakSSLKeyTable) validate() []string {
+	errs := []string{}
+	for i, record := range tbl.Records {
+		if record.DeliveryService == "" {
+			errs = append(errs, fmt.Sprintf("SSL Key #%v: Delivery Service is blank!", i))
+		}
+		if record.CDN == "" {
+			errs = append(errs, fmt.Sprintf("SSL Key #%v: CDN is blank!", i))
+		}
+		if record.Version.String() == "" {
+			errs = append(errs, fmt.Sprintf("SSL Key #%v: Version is blank!", i))
+		}
+	}
+	return errs
+}
+
+type riakDNSSecKeyRecord struct {
+	CDN string
+	Key tc.DNSSECKeysRiak
+}
+type riakDNSSecKeyTable struct {
+	Records []riakDNSSecKeyRecord
+}
+
+func (tbl *riakDNSSecKeyTable) gatherKeys(cluster *riak.Cluster) error {
+	tbl.Records = []riakDNSSecKeyRecord{}
+	objs, err := getObjects(cluster, BUCKET_DNSSEC)
+	if err != nil {
+		return fmt.Errorf("RiakDNSSecKey gatherKeys: %w", err)
+	}
+	for _, obj := range objs {
+		key := tc.DNSSECKeysRiak{}
+		if err := json.Unmarshal(obj.Value, &key); err != nil {
+			return fmt.Errorf("RiakDNSSecKey gatherKeys unable to unmarshal object to tc.DNSSECKeysRiak: %w", err)
+		}
+		tbl.Records = append(tbl.Records, riakDNSSecKeyRecord{
+			CDN: obj.Key,
+			Key: key,
+		})
+	}
+	return nil
+}
+func (tbl *riakDNSSecKeyTable) toGeneric() []DNSSecKey {
+	keys := make([]DNSSecKey, len(tbl.Records))
+
+	for i, record := range tbl.Records {
+		keys[i] = DNSSecKey{
+			CDN:                    record.CDN,
+			DNSSECKeysTrafficVault: tc.DNSSECKeysTrafficVault(record.Key),
+		}
+	}
+
+	return keys
+}
+func (tbl *riakDNSSecKeyTable) fromGeneric(keys []DNSSecKey) {
+	tbl.Records = make([]riakDNSSecKeyRecord, len(keys))
+
+	for i, record := range keys {
+		tbl.Records[i] = riakDNSSecKeyRecord{
+			CDN: record.CDN,
+			Key: tc.DNSSECKeysRiak(record.DNSSECKeysTrafficVault),
+		}
+	}
+}
+func (tbl *riakDNSSecKeyTable) insertKeys(cluster *riak.Cluster) error {
+	for _, record := range tbl.Records {
+		objBytes, err := json.Marshal(record.Key)
+		if err != nil {
+			return fmt.Errorf("RiakDNSSecKey insertKeys error marshalling keys: %w", err)
+		}
+
+		err = setObject(cluster, makeRiakObject(objBytes, record.CDN), BUCKET_DNSSEC)
+		if err != nil {
+			return fmt.Errorf("RiakDNSSecKey insertKeys: %w", err)
+		}
+	}
+	return nil
+}
+func (tbl *riakDNSSecKeyTable) validate() []string {
+	errs := []string{}
+	for i, record := range tbl.Records {
+		if record.CDN == "" {
+			errs = append(errs, fmt.Sprintf("DNSSec Key #%v: CDN is blank!", i))
+		}
+	}
+	return errs
+}
+
+type riakURLSigKeyRecord struct {
+	Key             tc.URLSigKeys
+	DeliveryService string
+}
+type riakURLSigKeyTable struct {
+	Records []riakURLSigKeyRecord
+}
+
+func (tbl *riakURLSigKeyTable) gatherKeys(cluster *riak.Cluster) error {
+	tbl.Records = []riakURLSigKeyRecord{}
+	objs, err := getObjects(cluster, BUCKET_URL_SIG)
+	if err != nil {
+		return fmt.Errorf("RiakURLSigKey gatherKeys: %w", err)
+	}
+	for _, obj := range objs {
+		key := tc.URLSigKeys{}
+		if err := json.Unmarshal(obj.Value, &key); err != nil {
+			return fmt.Errorf("RiakURLSigKey gatherKeys unable to unamrshal object into tc.URLSigKeys: %w", err)
+		}
+		tbl.Records = append(tbl.Records, riakURLSigKeyRecord{
+			DeliveryService: obj.Key,
+			Key:             key,
+		})
+	}
+	return nil
+}
+func (tbl *riakURLSigKeyTable) toGeneric() []URLSigKey {
+	keys := make([]URLSigKey, len(tbl.Records))
+
+	for i, record := range tbl.Records {
+		keys[i] = URLSigKey{
+			URLSigKeys:      record.Key,
+			DeliveryService: record.DeliveryService,
+		}
+	}
+
+	return keys
+}
+func (tbl *riakURLSigKeyTable) fromGeneric(keys []URLSigKey) {
+	tbl.Records = make([]riakURLSigKeyRecord, len(keys))
+	for i, key := range keys {
+		tbl.Records[i] = riakURLSigKeyRecord{
+			Key:             key.URLSigKeys,
+			DeliveryService: key.DeliveryService,
+		}
+	}
+}
+func (tbl *riakURLSigKeyTable) insertKeys(cluster *riak.Cluster) error {
+	for _, record := range tbl.Records {
+		objBytes, err := json.Marshal(record.Key)
+		if err != nil {
+			return fmt.Errorf("RiakURLSigKey insertKeys unable to marshal keys: %w", err)
+		}
+
+		err = setObject(cluster, makeRiakObject(objBytes, "url_sig_"+record.DeliveryService+".config"), BUCKET_URL_SIG)
+		if err != nil {
+			return fmt.Errorf("RiakURLSigKey insertKeys: %w", err)
+		}
+	}
+	return nil
+}
+func (tbl *riakURLSigKeyTable) validate() []string {
+	errs := []string{}
+	for i, record := range tbl.Records {
+		if record.DeliveryService == "" {
+			errs = append(errs, fmt.Sprintf("URL Key #%v: Delivery Service is blank!", i))
+		}
+	}
+	return errs
+}
+
+type riakURISignKeyRecord struct {
+	Key             map[string]tc.URISignerKeyset
+	DeliveryService string
+}
+type riakURISignKeyTable struct {
+	Records []riakURISignKeyRecord
+}
+
+func (tbl *riakURISignKeyTable) gatherKeys(cluster *riak.Cluster) error {
+	tbl.Records = []riakURISignKeyRecord{}
+	objs, err := getObjects(cluster, BUCKET_URI_SIG)
+	if err != nil {
+		return fmt.Errorf("RiakURISignKey gatherKeys: %w", err)
+	}
+	for _, obj := range objs {
+		key := map[string]tc.URISignerKeyset{}
+		if err := json.Unmarshal(obj.Value, &key); err != nil {
+			return fmt.Errorf("RiakURISignKey gatherKeys unable to unmarshal object into map[string]tc.URISignerKeySet: %w", err)
+		}
+
+		tbl.Records = append(tbl.Records, riakURISignKeyRecord{
+			DeliveryService: obj.Key,
+			Key:             key,
+		})
+	}
+	return nil
+}
+func (tbl *riakURISignKeyTable) toGeneric() []URISignKey {
+	keys := make([]URISignKey, len(tbl.Records))
+
+	for i, record := range tbl.Records {
+		keys[i] = URISignKey{
+			DeliveryService: record.DeliveryService,
+			Keys:            record.Key,
+		}
+	}
+
+	return keys
+}
+func (tbl *riakURISignKeyTable) fromGeneric(keys []URISignKey) {
+	tbl.Records = make([]riakURISignKeyRecord, len(keys))
+
+	for i, record := range keys {
+		tbl.Records[i] = riakURISignKeyRecord{
+			Key:             record.Keys,
+			DeliveryService: record.DeliveryService,
+		}
+	}
+}
+func (tbl *riakURISignKeyTable) insertKeys(cluster *riak.Cluster) error {
+	for _, record := range tbl.Records {
+		objBytes, err := json.Marshal(record.Key)
+		if err != nil {
+			return fmt.Errorf("RiakURISignKey insertKeys: unable to marshal key: %w", err)
+		}
+
+		err = setObject(cluster, makeRiakObject(objBytes, record.DeliveryService), BUCKET_URI_SIG)

Review comment:
       nit: this can be combined with the next line like `if err := foo(); err != nil {...}`

##########
File path: tools/traffic_vault_migrate/riak.go
##########
@@ -0,0 +1,746 @@
+package main
+
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+import (
+	"crypto/tls"
+	"encoding/json"
+	"errors"
+	"fmt"
+	"time"
+
+	"github.com/apache/trafficcontrol/lib/go-rfc"
+	"github.com/apache/trafficcontrol/lib/go-tc"
+
+	"github.com/basho/riak-go-client"
+)
+
+const (
+	BUCKET_SSL     = "ssl"
+	BUCKET_DNSSEC  = "dnssec"
+	BUCKET_URL_SIG = "url_sig_keys"
+	BUCKET_URI_SIG = "cdn_uri_sig_keys"
+
+	INDEX_SSL = "sslkeys"
+
+	SCHEMA_RIAK_KEY    = "_yz_rk"
+	SCHEMA_RIAK_BUCKET = "_yz_rb"
+)
+
+var (
+	SCHEMA_SSL_FIELDS = [...]string{SCHEMA_RIAK_KEY, SCHEMA_RIAK_BUCKET}
+)
+
+// RiakConfig  represents the configuration options available to the Riak backend
+type RiakConfig struct {
+	Host          string `json:"host"`
+	Port          string `json:"port"`
+	User          string `json:"user"`
+	Password      string `json:"password"`
+	Insecure      bool   `json:"insecure"`
+	TLSVersionRaw string `json:"tlsVersion"`
+	TLSVersion    uint16
+}
+
+// RiakBackend is the Riak implementation of TVBackend
+type RiakBackend struct {
+	sslKeys     riakSSLKeyTable
+	dnssecKeys  riakDNSSecKeyTable
+	uriSignKeys riakURISignKeyTable
+	urlSigKeys  riakURLSigKeyTable
+	cfg         RiakConfig
+	cluster     *riak.Cluster
+}
+
+// String returns a high level overview of the backend and its keys
+func (rb *RiakBackend) String() string {
+	data := fmt.Sprintf("Riak server %v@%v:%v\n", rb.cfg.User, rb.cfg.Host, rb.cfg.Port)
+	data += fmt.Sprintf("\tSSL Keys: %v\n", len(rb.sslKeys.Records))
+	data += fmt.Sprintf("\tDNSSec Keys: %v\n", len(rb.dnssecKeys.Records))
+	data += fmt.Sprintf("\tURI Keys: %v\n", len(rb.uriSignKeys.Records))
+	data += fmt.Sprintf("\tURL Keys: %v\n", len(rb.urlSigKeys.Records))
+	return data
+}
+
+// Name returns the name for this backend
+func (rb *RiakBackend) Name() string {
+	return "Riak"
+}
+
+// ReadConfig takes in a filename and will read it into the backends config
+func (rb *RiakBackend) ReadConfig(s string) error {
+	err := UnmarshalConfig(s, &rb.cfg)
+	if err != nil {
+		return err
+	}
+
+	if rb.cfg.TLSVersionRaw == "10" {
+		rb.cfg.TLSVersion = tls.VersionTLS10
+	} else if rb.cfg.TLSVersionRaw == "11" {
+		rb.cfg.TLSVersion = tls.VersionTLS11
+	} else if rb.cfg.TLSVersionRaw == "12" {
+		rb.cfg.TLSVersion = tls.VersionTLS12
+	} else if rb.cfg.TLSVersionRaw == "13" {
+		rb.cfg.TLSVersion = tls.VersionTLS13
+	}
+	return nil
+}
+
+// Insert takes the current keys and inserts them into the backend DB
+func (rb *RiakBackend) Insert() error {
+	if err := rb.sslKeys.insertKeys(rb.cluster); err != nil {
+		return err
+	}
+	if err := rb.dnssecKeys.insertKeys(rb.cluster); err != nil {
+		return err
+	}
+	if err := rb.urlSigKeys.insertKeys(rb.cluster); err != nil {
+		return err
+	}
+	if err := rb.uriSignKeys.insertKeys(rb.cluster); err != nil {
+		return err
+	}
+	return nil
+}
+
+// ValidateKey validates that the keys are valid (in most cases, certain fields are not null)
+func (rb *RiakBackend) ValidateKey() []string {
+	errors := []string{}
+	if errs := rb.sslKeys.validate(); errs != nil {
+		errors = append(errors, errs...)
+	}
+	if errs := rb.dnssecKeys.validate(); errs != nil {
+		errors = append(errors, errs...)
+	}
+	if errs := rb.uriSignKeys.validate(); errs != nil {
+		errors = append(errors, errs...)
+	}
+	if errs := rb.urlSigKeys.validate(); errs != nil {
+		errors = append(errors, errs...)
+	}
+
+	return errors
+}
+
+// SetSSLKeys takes in keys and converts & encrypts the data into the backends internal format
+func (rb *RiakBackend) SetSSLKeys(keys []SSLKey) error {
+	rb.sslKeys.fromGeneric(keys)
+	return nil
+}
+
+// SetDNSSecKeys takes in keys and converts & encrypts the data into the backends internal format
+func (rb *RiakBackend) SetDNSSecKeys(keys []DNSSecKey) error {
+	rb.dnssecKeys.fromGeneric(keys)
+	return nil
+}
+
+// SetURISignKeys takes in keys and converts & encrypts the data into the backends internal format
+func (rb *RiakBackend) SetURISignKeys(keys []URISignKey) error {
+	rb.uriSignKeys.fromGeneric(keys)
+	return nil
+}
+
+// SetURLSigKeys takes in keys and converts & encrypts the data into the backends internal format
+func (rb *RiakBackend) SetURLSigKeys(keys []URLSigKey) error {
+	rb.urlSigKeys.fromGeneric(keys)
+	return nil
+}
+
+// Start initiates the connection to the backend DB
+func (rb *RiakBackend) Start() error {
+	tlsConfig := &tls.Config{
+		InsecureSkipVerify: !rb.cfg.Insecure,
+		MaxVersion:         rb.cfg.TLSVersion,
+	}
+	auth := &riak.AuthOptions{
+		User:      rb.cfg.User,
+		Password:  rb.cfg.Password,
+		TlsConfig: tlsConfig,
+	}
+
+	cluster, err := getRiakCluster(rb.cfg, auth)
+	if err != nil {
+		return err
+	}
+	if err := cluster.Start(); err != nil {
+		return fmt.Errorf("unable to start riak cluster: %w", err)
+	}
+
+	rb.cluster = cluster
+	rb.sslKeys = riakSSLKeyTable{}
+	rb.dnssecKeys = riakDNSSecKeyTable{}
+	rb.urlSigKeys = riakURLSigKeyTable{}
+	rb.uriSignKeys = riakURISignKeyTable{}
+	return nil
+}
+
+// Close terminates the connection to the backend DB
+func (rb *RiakBackend) Close() error {
+	if err := rb.cluster.Stop(); err != nil {
+		return err
+	}
+	return nil
+}
+
+// Ping checks the connection to the backend DB
+func (rb *RiakBackend) Ping() error {
+	return ping(rb.cluster)
+}
+
+// GetSSLKeys converts the backends internal key representation into the common representation (SSLKey)
+func (rb *RiakBackend) GetSSLKeys() ([]SSLKey, error) {
+	return rb.sslKeys.toGeneric(), nil
+}
+
+// GetDNSSecKeys converts the backends internal key representation into the common representation (DNSSecKey)
+func (rb *RiakBackend) GetDNSSecKeys() ([]DNSSecKey, error) {
+	return rb.dnssecKeys.toGeneric(), nil
+}
+
+// GetURISignKeys converts the pg internal key representation into the common representation (URISignKey)
+func (rb *RiakBackend) GetURISignKeys() ([]URISignKey, error) {
+	return rb.uriSignKeys.toGeneric(), nil
+}
+
+// GetURLSigKeys converts the backends internal key representation into the common representation (URLSigKey)
+func (rb *RiakBackend) GetURLSigKeys() ([]URLSigKey, error) {
+	return rb.urlSigKeys.toGeneric(), nil
+}
+
+// Fetch gets all of the keys from the backend DB
+func (rb *RiakBackend) Fetch() error {
+	if err := rb.sslKeys.gatherKeys(rb.cluster); err != nil {
+		return err
+	}
+	if err := rb.dnssecKeys.gatherKeys(rb.cluster); err != nil {
+		return err
+	}
+	if err := rb.urlSigKeys.gatherKeys(rb.cluster); err != nil {
+		return err
+	}
+	if err := rb.uriSignKeys.gatherKeys(rb.cluster); err != nil {
+		return err
+	}
+
+	return nil
+}
+
+type riakSSLKeyRecord struct {
+	tc.DeliveryServiceSSLKeys
+}
+type riakSSLKeyTable struct {
+	Records []riakSSLKeyRecord
+}
+
+func (tbl *riakSSLKeyTable) gatherKeys(cluster *riak.Cluster) error {
+	searchDocs, err := search(cluster, INDEX_SSL, "cdn:*", fmt.Sprintf("%v:*latest", SCHEMA_RIAK_KEY), 1000, SCHEMA_SSL_FIELDS[:])
+	if err != nil {
+		return fmt.Errorf("RiakSSLKey gatherKeys: %w", err)
+	}
+
+	tbl.Records = make([]riakSSLKeyRecord, len(searchDocs))
+	for i, doc := range searchDocs {
+		objs, err := getObject(cluster, doc.Bucket, doc.Key)
+		if err != nil {
+			return err
+		}
+		if len(objs) < 1 {
+			return fmt.Errorf("RiakSSLKey gatherKeys unable to find any objects with key %v and bucket %v, but search results were returned", doc.Key, doc.Bucket)
+		}
+		if len(objs) > 1 {
+			return fmt.Errorf("RiakSSLKey gatherKeys more than 1 ssl key record found %v\n", len(objs))
+		}
+		var obj tc.DeliveryServiceSSLKeys
+		if err = json.Unmarshal(objs[0].Value, &obj); err != nil {
+			return fmt.Errorf("RiakSSLKey gatherKeys unable to unmarshal object into tc.DeliveryServiceSSLKeys: %w", err)
+		}
+		tbl.Records[i] = riakSSLKeyRecord{
+			DeliveryServiceSSLKeys: obj,
+		}
+	}
+	return nil
+}
+func (tbl *riakSSLKeyTable) toGeneric() []SSLKey {
+	keys := make([]SSLKey, len(tbl.Records))
+
+	for i, record := range tbl.Records {
+		keys[i] = SSLKey{
+			DeliveryServiceSSLKeys: record.DeliveryServiceSSLKeys,
+		}
+	}
+
+	return keys
+}
+func (tbl *riakSSLKeyTable) fromGeneric(keys []SSLKey) {
+	tbl.Records = make([]riakSSLKeyRecord, len(keys))
+
+	for i, record := range keys {
+		tbl.Records[i] = riakSSLKeyRecord{
+			DeliveryServiceSSLKeys: record.DeliveryServiceSSLKeys,
+		}
+	}
+}
+func (tbl *riakSSLKeyTable) insertKeys(cluster *riak.Cluster) error {
+	for _, record := range tbl.Records {
+		objBytes, err := json.Marshal(record.DeliveryServiceSSLKeys)
+		if err != nil {
+			return fmt.Errorf("RiakSSLKey insertKeys failed to unmarshal keys: %w", err)
+		}
+		err = setObject(cluster, makeRiakObject(objBytes, record.DeliveryService+"-"+record.Version.String()), BUCKET_SSL)
+		if err != nil {
+			return fmt.Errorf("RiakSSLKey insertKeys: %w", err)
+		}
+		err = setObject(cluster, makeRiakObject(objBytes, record.DeliveryService+"-latest"), BUCKET_SSL)
+		if err != nil {
+			return fmt.Errorf("RiakSSLKey insertKeys: %w", err)
+		}
+	}
+	return nil
+}
+func (tbl *riakSSLKeyTable) validate() []string {
+	errs := []string{}
+	for i, record := range tbl.Records {
+		if record.DeliveryService == "" {
+			errs = append(errs, fmt.Sprintf("SSL Key #%v: Delivery Service is blank!", i))
+		}
+		if record.CDN == "" {
+			errs = append(errs, fmt.Sprintf("SSL Key #%v: CDN is blank!", i))
+		}
+		if record.Version.String() == "" {
+			errs = append(errs, fmt.Sprintf("SSL Key #%v: Version is blank!", i))
+		}
+	}
+	return errs
+}
+
+type riakDNSSecKeyRecord struct {
+	CDN string
+	Key tc.DNSSECKeysRiak
+}
+type riakDNSSecKeyTable struct {
+	Records []riakDNSSecKeyRecord
+}
+
+func (tbl *riakDNSSecKeyTable) gatherKeys(cluster *riak.Cluster) error {
+	tbl.Records = []riakDNSSecKeyRecord{}
+	objs, err := getObjects(cluster, BUCKET_DNSSEC)
+	if err != nil {
+		return fmt.Errorf("RiakDNSSecKey gatherKeys: %w", err)
+	}
+	for _, obj := range objs {
+		key := tc.DNSSECKeysRiak{}
+		if err := json.Unmarshal(obj.Value, &key); err != nil {
+			return fmt.Errorf("RiakDNSSecKey gatherKeys unable to unmarshal object to tc.DNSSECKeysRiak: %w", err)
+		}
+		tbl.Records = append(tbl.Records, riakDNSSecKeyRecord{
+			CDN: obj.Key,
+			Key: key,
+		})
+	}
+	return nil
+}
+func (tbl *riakDNSSecKeyTable) toGeneric() []DNSSecKey {
+	keys := make([]DNSSecKey, len(tbl.Records))
+
+	for i, record := range tbl.Records {
+		keys[i] = DNSSecKey{
+			CDN:                    record.CDN,
+			DNSSECKeysTrafficVault: tc.DNSSECKeysTrafficVault(record.Key),
+		}
+	}
+
+	return keys
+}
+func (tbl *riakDNSSecKeyTable) fromGeneric(keys []DNSSecKey) {
+	tbl.Records = make([]riakDNSSecKeyRecord, len(keys))
+
+	for i, record := range keys {
+		tbl.Records[i] = riakDNSSecKeyRecord{
+			CDN: record.CDN,
+			Key: tc.DNSSECKeysRiak(record.DNSSECKeysTrafficVault),
+		}
+	}
+}
+func (tbl *riakDNSSecKeyTable) insertKeys(cluster *riak.Cluster) error {
+	for _, record := range tbl.Records {
+		objBytes, err := json.Marshal(record.Key)
+		if err != nil {
+			return fmt.Errorf("RiakDNSSecKey insertKeys error marshalling keys: %w", err)
+		}
+
+		err = setObject(cluster, makeRiakObject(objBytes, record.CDN), BUCKET_DNSSEC)
+		if err != nil {
+			return fmt.Errorf("RiakDNSSecKey insertKeys: %w", err)
+		}
+	}
+	return nil
+}
+func (tbl *riakDNSSecKeyTable) validate() []string {
+	errs := []string{}
+	for i, record := range tbl.Records {
+		if record.CDN == "" {
+			errs = append(errs, fmt.Sprintf("DNSSec Key #%v: CDN is blank!", i))
+		}
+	}
+	return errs
+}
+
+type riakURLSigKeyRecord struct {
+	Key             tc.URLSigKeys
+	DeliveryService string
+}
+type riakURLSigKeyTable struct {
+	Records []riakURLSigKeyRecord
+}
+
+func (tbl *riakURLSigKeyTable) gatherKeys(cluster *riak.Cluster) error {
+	tbl.Records = []riakURLSigKeyRecord{}
+	objs, err := getObjects(cluster, BUCKET_URL_SIG)
+	if err != nil {
+		return fmt.Errorf("RiakURLSigKey gatherKeys: %w", err)
+	}
+	for _, obj := range objs {
+		key := tc.URLSigKeys{}
+		if err := json.Unmarshal(obj.Value, &key); err != nil {
+			return fmt.Errorf("RiakURLSigKey gatherKeys unable to unamrshal object into tc.URLSigKeys: %w", err)
+		}
+		tbl.Records = append(tbl.Records, riakURLSigKeyRecord{
+			DeliveryService: obj.Key,
+			Key:             key,
+		})
+	}
+	return nil
+}
+func (tbl *riakURLSigKeyTable) toGeneric() []URLSigKey {
+	keys := make([]URLSigKey, len(tbl.Records))
+
+	for i, record := range tbl.Records {
+		keys[i] = URLSigKey{
+			URLSigKeys:      record.Key,
+			DeliveryService: record.DeliveryService,
+		}
+	}
+
+	return keys
+}
+func (tbl *riakURLSigKeyTable) fromGeneric(keys []URLSigKey) {
+	tbl.Records = make([]riakURLSigKeyRecord, len(keys))
+	for i, key := range keys {
+		tbl.Records[i] = riakURLSigKeyRecord{
+			Key:             key.URLSigKeys,
+			DeliveryService: key.DeliveryService,
+		}
+	}
+}
+func (tbl *riakURLSigKeyTable) insertKeys(cluster *riak.Cluster) error {
+	for _, record := range tbl.Records {
+		objBytes, err := json.Marshal(record.Key)
+		if err != nil {
+			return fmt.Errorf("RiakURLSigKey insertKeys unable to marshal keys: %w", err)
+		}
+
+		err = setObject(cluster, makeRiakObject(objBytes, "url_sig_"+record.DeliveryService+".config"), BUCKET_URL_SIG)
+		if err != nil {
+			return fmt.Errorf("RiakURLSigKey insertKeys: %w", err)
+		}
+	}
+	return nil
+}
+func (tbl *riakURLSigKeyTable) validate() []string {
+	errs := []string{}
+	for i, record := range tbl.Records {
+		if record.DeliveryService == "" {
+			errs = append(errs, fmt.Sprintf("URL Key #%v: Delivery Service is blank!", i))
+		}
+	}
+	return errs
+}
+
+type riakURISignKeyRecord struct {
+	Key             map[string]tc.URISignerKeyset
+	DeliveryService string
+}
+type riakURISignKeyTable struct {
+	Records []riakURISignKeyRecord
+}
+
+func (tbl *riakURISignKeyTable) gatherKeys(cluster *riak.Cluster) error {
+	tbl.Records = []riakURISignKeyRecord{}
+	objs, err := getObjects(cluster, BUCKET_URI_SIG)
+	if err != nil {
+		return fmt.Errorf("RiakURISignKey gatherKeys: %w", err)
+	}
+	for _, obj := range objs {
+		key := map[string]tc.URISignerKeyset{}
+		if err := json.Unmarshal(obj.Value, &key); err != nil {
+			return fmt.Errorf("RiakURISignKey gatherKeys unable to unmarshal object into map[string]tc.URISignerKeySet: %w", err)
+		}
+
+		tbl.Records = append(tbl.Records, riakURISignKeyRecord{
+			DeliveryService: obj.Key,
+			Key:             key,
+		})
+	}
+	return nil
+}
+func (tbl *riakURISignKeyTable) toGeneric() []URISignKey {
+	keys := make([]URISignKey, len(tbl.Records))
+
+	for i, record := range tbl.Records {
+		keys[i] = URISignKey{
+			DeliveryService: record.DeliveryService,
+			Keys:            record.Key,
+		}
+	}
+
+	return keys
+}
+func (tbl *riakURISignKeyTable) fromGeneric(keys []URISignKey) {
+	tbl.Records = make([]riakURISignKeyRecord, len(keys))
+
+	for i, record := range keys {
+		tbl.Records[i] = riakURISignKeyRecord{
+			Key:             record.Keys,
+			DeliveryService: record.DeliveryService,
+		}
+	}
+}
+func (tbl *riakURISignKeyTable) insertKeys(cluster *riak.Cluster) error {
+	for _, record := range tbl.Records {
+		objBytes, err := json.Marshal(record.Key)
+		if err != nil {
+			return fmt.Errorf("RiakURISignKey insertKeys: unable to marshal key: %w", err)
+		}
+
+		err = setObject(cluster, makeRiakObject(objBytes, record.DeliveryService), BUCKET_URI_SIG)
+		if err != nil {
+			return fmt.Errorf("RiakURISignKey insertKeys: %w", err)
+		}
+	}
+	return nil
+}
+func (tbl *riakURISignKeyTable) validate() []string {
+	errs := []string{}
+	for i, record := range tbl.Records {
+		if record.DeliveryService == "" {
+			errs = append(errs, fmt.Sprintf("URI Key #%v: Delivery Service is blank!", i))
+		}
+	}
+	return errs
+}
+
+// Riak functions
+func makeRiakObject(data []byte, key string) *riak.Object {
+	return &riak.Object{
+		ContentType:     rfc.ApplicationJSON,
+		Charset:         "utf-8",
+		ContentEncoding: "utf-8",
+		Key:             key,
+		Value:           data,
+	}
+}
+func getObjects(cluster *riak.Cluster, bucket string) ([]*riak.Object, error) {
+	objs := []*riak.Object{}
+	keys, err := listKeys(cluster, bucket)
+	if err != nil {
+		return nil, err
+	}
+	for _, key := range keys {
+		objects, err := getObject(cluster, bucket, key)
+		if err != nil {
+			return nil, err
+		}
+		if len(objects) > 1 {
+			return nil, fmt.Errorf("Unexpected number of objects %v, ignoring\n", len(objects))
+		}
+
+		objs = append(objs, objects[0])
+	}
+
+	return objs, nil
+}
+func getObject(cluster *riak.Cluster, bucket string, key string) ([]*riak.Object, error) {
+	cmd, err := riak.NewFetchValueCommandBuilder().
+		WithBucket(bucket).
+		WithKey(key).
+		WithTimeout(time.Second * 60).

Review comment:
       Should this timeout be configurable (with a sane default)?

##########
File path: tools/traffic_vault_migrate/riak.go
##########
@@ -0,0 +1,746 @@
+package main
+
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+import (
+	"crypto/tls"
+	"encoding/json"
+	"errors"
+	"fmt"
+	"time"
+
+	"github.com/apache/trafficcontrol/lib/go-rfc"
+	"github.com/apache/trafficcontrol/lib/go-tc"
+
+	"github.com/basho/riak-go-client"
+)
+
+const (
+	BUCKET_SSL     = "ssl"
+	BUCKET_DNSSEC  = "dnssec"
+	BUCKET_URL_SIG = "url_sig_keys"
+	BUCKET_URI_SIG = "cdn_uri_sig_keys"
+
+	INDEX_SSL = "sslkeys"
+
+	SCHEMA_RIAK_KEY    = "_yz_rk"
+	SCHEMA_RIAK_BUCKET = "_yz_rb"
+)
+
+var (
+	SCHEMA_SSL_FIELDS = [...]string{SCHEMA_RIAK_KEY, SCHEMA_RIAK_BUCKET}
+)
+
+// RiakConfig  represents the configuration options available to the Riak backend
+type RiakConfig struct {
+	Host          string `json:"host"`
+	Port          string `json:"port"`
+	User          string `json:"user"`
+	Password      string `json:"password"`
+	Insecure      bool   `json:"insecure"`
+	TLSVersionRaw string `json:"tlsVersion"`
+	TLSVersion    uint16
+}
+
+// RiakBackend is the Riak implementation of TVBackend
+type RiakBackend struct {
+	sslKeys     riakSSLKeyTable
+	dnssecKeys  riakDNSSecKeyTable
+	uriSignKeys riakURISignKeyTable
+	urlSigKeys  riakURLSigKeyTable
+	cfg         RiakConfig
+	cluster     *riak.Cluster
+}
+
+// String returns a high level overview of the backend and its keys
+func (rb *RiakBackend) String() string {
+	data := fmt.Sprintf("Riak server %v@%v:%v\n", rb.cfg.User, rb.cfg.Host, rb.cfg.Port)
+	data += fmt.Sprintf("\tSSL Keys: %v\n", len(rb.sslKeys.Records))
+	data += fmt.Sprintf("\tDNSSec Keys: %v\n", len(rb.dnssecKeys.Records))
+	data += fmt.Sprintf("\tURI Keys: %v\n", len(rb.uriSignKeys.Records))
+	data += fmt.Sprintf("\tURL Keys: %v\n", len(rb.urlSigKeys.Records))
+	return data
+}
+
+// Name returns the name for this backend
+func (rb *RiakBackend) Name() string {
+	return "Riak"
+}
+
+// ReadConfig takes in a filename and will read it into the backends config
+func (rb *RiakBackend) ReadConfig(s string) error {
+	err := UnmarshalConfig(s, &rb.cfg)
+	if err != nil {
+		return err
+	}
+
+	if rb.cfg.TLSVersionRaw == "10" {
+		rb.cfg.TLSVersion = tls.VersionTLS10
+	} else if rb.cfg.TLSVersionRaw == "11" {
+		rb.cfg.TLSVersion = tls.VersionTLS11
+	} else if rb.cfg.TLSVersionRaw == "12" {
+		rb.cfg.TLSVersion = tls.VersionTLS12
+	} else if rb.cfg.TLSVersionRaw == "13" {
+		rb.cfg.TLSVersion = tls.VersionTLS13
+	}
+	return nil
+}
+
+// Insert takes the current keys and inserts them into the backend DB
+func (rb *RiakBackend) Insert() error {
+	if err := rb.sslKeys.insertKeys(rb.cluster); err != nil {
+		return err
+	}
+	if err := rb.dnssecKeys.insertKeys(rb.cluster); err != nil {
+		return err
+	}
+	if err := rb.urlSigKeys.insertKeys(rb.cluster); err != nil {
+		return err
+	}
+	if err := rb.uriSignKeys.insertKeys(rb.cluster); err != nil {
+		return err
+	}
+	return nil
+}
+
+// ValidateKey validates that the keys are valid (in most cases, certain fields are not null)
+func (rb *RiakBackend) ValidateKey() []string {
+	errors := []string{}
+	if errs := rb.sslKeys.validate(); errs != nil {
+		errors = append(errors, errs...)
+	}
+	if errs := rb.dnssecKeys.validate(); errs != nil {
+		errors = append(errors, errs...)
+	}
+	if errs := rb.uriSignKeys.validate(); errs != nil {
+		errors = append(errors, errs...)
+	}
+	if errs := rb.urlSigKeys.validate(); errs != nil {
+		errors = append(errors, errs...)
+	}
+
+	return errors
+}
+
+// SetSSLKeys takes in keys and converts & encrypts the data into the backends internal format
+func (rb *RiakBackend) SetSSLKeys(keys []SSLKey) error {
+	rb.sslKeys.fromGeneric(keys)
+	return nil
+}
+
+// SetDNSSecKeys takes in keys and converts & encrypts the data into the backends internal format
+func (rb *RiakBackend) SetDNSSecKeys(keys []DNSSecKey) error {
+	rb.dnssecKeys.fromGeneric(keys)
+	return nil
+}
+
+// SetURISignKeys takes in keys and converts & encrypts the data into the backends internal format
+func (rb *RiakBackend) SetURISignKeys(keys []URISignKey) error {
+	rb.uriSignKeys.fromGeneric(keys)
+	return nil
+}
+
+// SetURLSigKeys takes in keys and converts & encrypts the data into the backends internal format
+func (rb *RiakBackend) SetURLSigKeys(keys []URLSigKey) error {
+	rb.urlSigKeys.fromGeneric(keys)
+	return nil
+}
+
+// Start initiates the connection to the backend DB
+func (rb *RiakBackend) Start() error {
+	tlsConfig := &tls.Config{
+		InsecureSkipVerify: !rb.cfg.Insecure,
+		MaxVersion:         rb.cfg.TLSVersion,
+	}
+	auth := &riak.AuthOptions{
+		User:      rb.cfg.User,
+		Password:  rb.cfg.Password,
+		TlsConfig: tlsConfig,
+	}
+
+	cluster, err := getRiakCluster(rb.cfg, auth)
+	if err != nil {
+		return err
+	}
+	if err := cluster.Start(); err != nil {
+		return fmt.Errorf("unable to start riak cluster: %w", err)
+	}
+
+	rb.cluster = cluster
+	rb.sslKeys = riakSSLKeyTable{}
+	rb.dnssecKeys = riakDNSSecKeyTable{}
+	rb.urlSigKeys = riakURLSigKeyTable{}
+	rb.uriSignKeys = riakURISignKeyTable{}
+	return nil
+}
+
+// Close terminates the connection to the backend DB
+func (rb *RiakBackend) Close() error {
+	if err := rb.cluster.Stop(); err != nil {
+		return err
+	}
+	return nil
+}
+
+// Ping checks the connection to the backend DB
+func (rb *RiakBackend) Ping() error {
+	return ping(rb.cluster)
+}
+
+// GetSSLKeys converts the backends internal key representation into the common representation (SSLKey)
+func (rb *RiakBackend) GetSSLKeys() ([]SSLKey, error) {
+	return rb.sslKeys.toGeneric(), nil
+}
+
+// GetDNSSecKeys converts the backends internal key representation into the common representation (DNSSecKey)
+func (rb *RiakBackend) GetDNSSecKeys() ([]DNSSecKey, error) {
+	return rb.dnssecKeys.toGeneric(), nil
+}
+
+// GetURISignKeys converts the pg internal key representation into the common representation (URISignKey)
+func (rb *RiakBackend) GetURISignKeys() ([]URISignKey, error) {
+	return rb.uriSignKeys.toGeneric(), nil
+}
+
+// GetURLSigKeys converts the backends internal key representation into the common representation (URLSigKey)
+func (rb *RiakBackend) GetURLSigKeys() ([]URLSigKey, error) {
+	return rb.urlSigKeys.toGeneric(), nil
+}
+
+// Fetch gets all of the keys from the backend DB
+func (rb *RiakBackend) Fetch() error {
+	if err := rb.sslKeys.gatherKeys(rb.cluster); err != nil {
+		return err
+	}
+	if err := rb.dnssecKeys.gatherKeys(rb.cluster); err != nil {
+		return err
+	}
+	if err := rb.urlSigKeys.gatherKeys(rb.cluster); err != nil {
+		return err
+	}
+	if err := rb.uriSignKeys.gatherKeys(rb.cluster); err != nil {
+		return err
+	}
+
+	return nil
+}
+
+type riakSSLKeyRecord struct {
+	tc.DeliveryServiceSSLKeys
+}
+type riakSSLKeyTable struct {
+	Records []riakSSLKeyRecord
+}
+
+func (tbl *riakSSLKeyTable) gatherKeys(cluster *riak.Cluster) error {
+	searchDocs, err := search(cluster, INDEX_SSL, "cdn:*", fmt.Sprintf("%v:*latest", SCHEMA_RIAK_KEY), 1000, SCHEMA_SSL_FIELDS[:])
+	if err != nil {
+		return fmt.Errorf("RiakSSLKey gatherKeys: %w", err)
+	}
+
+	tbl.Records = make([]riakSSLKeyRecord, len(searchDocs))
+	for i, doc := range searchDocs {
+		objs, err := getObject(cluster, doc.Bucket, doc.Key)
+		if err != nil {
+			return err
+		}
+		if len(objs) < 1 {
+			return fmt.Errorf("RiakSSLKey gatherKeys unable to find any objects with key %v and bucket %v, but search results were returned", doc.Key, doc.Bucket)
+		}
+		if len(objs) > 1 {
+			return fmt.Errorf("RiakSSLKey gatherKeys more than 1 ssl key record found %v\n", len(objs))
+		}
+		var obj tc.DeliveryServiceSSLKeys
+		if err = json.Unmarshal(objs[0].Value, &obj); err != nil {
+			return fmt.Errorf("RiakSSLKey gatherKeys unable to unmarshal object into tc.DeliveryServiceSSLKeys: %w", err)
+		}
+		tbl.Records[i] = riakSSLKeyRecord{
+			DeliveryServiceSSLKeys: obj,
+		}
+	}
+	return nil
+}
+func (tbl *riakSSLKeyTable) toGeneric() []SSLKey {
+	keys := make([]SSLKey, len(tbl.Records))
+
+	for i, record := range tbl.Records {
+		keys[i] = SSLKey{
+			DeliveryServiceSSLKeys: record.DeliveryServiceSSLKeys,
+		}
+	}
+
+	return keys
+}
+func (tbl *riakSSLKeyTable) fromGeneric(keys []SSLKey) {
+	tbl.Records = make([]riakSSLKeyRecord, len(keys))
+
+	for i, record := range keys {
+		tbl.Records[i] = riakSSLKeyRecord{
+			DeliveryServiceSSLKeys: record.DeliveryServiceSSLKeys,
+		}
+	}
+}
+func (tbl *riakSSLKeyTable) insertKeys(cluster *riak.Cluster) error {
+	for _, record := range tbl.Records {
+		objBytes, err := json.Marshal(record.DeliveryServiceSSLKeys)
+		if err != nil {
+			return fmt.Errorf("RiakSSLKey insertKeys failed to unmarshal keys: %w", err)
+		}
+		err = setObject(cluster, makeRiakObject(objBytes, record.DeliveryService+"-"+record.Version.String()), BUCKET_SSL)
+		if err != nil {
+			return fmt.Errorf("RiakSSLKey insertKeys: %w", err)
+		}
+		err = setObject(cluster, makeRiakObject(objBytes, record.DeliveryService+"-latest"), BUCKET_SSL)
+		if err != nil {
+			return fmt.Errorf("RiakSSLKey insertKeys: %w", err)
+		}
+	}
+	return nil
+}
+func (tbl *riakSSLKeyTable) validate() []string {
+	errs := []string{}
+	for i, record := range tbl.Records {
+		if record.DeliveryService == "" {
+			errs = append(errs, fmt.Sprintf("SSL Key #%v: Delivery Service is blank!", i))
+		}
+		if record.CDN == "" {
+			errs = append(errs, fmt.Sprintf("SSL Key #%v: CDN is blank!", i))
+		}
+		if record.Version.String() == "" {
+			errs = append(errs, fmt.Sprintf("SSL Key #%v: Version is blank!", i))
+		}
+	}
+	return errs
+}
+
+type riakDNSSecKeyRecord struct {
+	CDN string
+	Key tc.DNSSECKeysRiak
+}
+type riakDNSSecKeyTable struct {
+	Records []riakDNSSecKeyRecord
+}
+
+func (tbl *riakDNSSecKeyTable) gatherKeys(cluster *riak.Cluster) error {
+	tbl.Records = []riakDNSSecKeyRecord{}
+	objs, err := getObjects(cluster, BUCKET_DNSSEC)
+	if err != nil {
+		return fmt.Errorf("RiakDNSSecKey gatherKeys: %w", err)
+	}
+	for _, obj := range objs {
+		key := tc.DNSSECKeysRiak{}
+		if err := json.Unmarshal(obj.Value, &key); err != nil {
+			return fmt.Errorf("RiakDNSSecKey gatherKeys unable to unmarshal object to tc.DNSSECKeysRiak: %w", err)
+		}
+		tbl.Records = append(tbl.Records, riakDNSSecKeyRecord{
+			CDN: obj.Key,
+			Key: key,
+		})
+	}
+	return nil
+}
+func (tbl *riakDNSSecKeyTable) toGeneric() []DNSSecKey {
+	keys := make([]DNSSecKey, len(tbl.Records))
+
+	for i, record := range tbl.Records {
+		keys[i] = DNSSecKey{
+			CDN:                    record.CDN,
+			DNSSECKeysTrafficVault: tc.DNSSECKeysTrafficVault(record.Key),
+		}
+	}
+
+	return keys
+}
+func (tbl *riakDNSSecKeyTable) fromGeneric(keys []DNSSecKey) {
+	tbl.Records = make([]riakDNSSecKeyRecord, len(keys))
+
+	for i, record := range keys {
+		tbl.Records[i] = riakDNSSecKeyRecord{
+			CDN: record.CDN,
+			Key: tc.DNSSECKeysRiak(record.DNSSECKeysTrafficVault),
+		}
+	}
+}
+func (tbl *riakDNSSecKeyTable) insertKeys(cluster *riak.Cluster) error {
+	for _, record := range tbl.Records {
+		objBytes, err := json.Marshal(record.Key)
+		if err != nil {
+			return fmt.Errorf("RiakDNSSecKey insertKeys error marshalling keys: %w", err)
+		}
+
+		err = setObject(cluster, makeRiakObject(objBytes, record.CDN), BUCKET_DNSSEC)
+		if err != nil {
+			return fmt.Errorf("RiakDNSSecKey insertKeys: %w", err)
+		}
+	}
+	return nil
+}
+func (tbl *riakDNSSecKeyTable) validate() []string {
+	errs := []string{}
+	for i, record := range tbl.Records {
+		if record.CDN == "" {
+			errs = append(errs, fmt.Sprintf("DNSSec Key #%v: CDN is blank!", i))
+		}
+	}
+	return errs
+}
+
+type riakURLSigKeyRecord struct {
+	Key             tc.URLSigKeys
+	DeliveryService string
+}
+type riakURLSigKeyTable struct {
+	Records []riakURLSigKeyRecord
+}
+
+func (tbl *riakURLSigKeyTable) gatherKeys(cluster *riak.Cluster) error {
+	tbl.Records = []riakURLSigKeyRecord{}
+	objs, err := getObjects(cluster, BUCKET_URL_SIG)
+	if err != nil {
+		return fmt.Errorf("RiakURLSigKey gatherKeys: %w", err)
+	}
+	for _, obj := range objs {
+		key := tc.URLSigKeys{}
+		if err := json.Unmarshal(obj.Value, &key); err != nil {
+			return fmt.Errorf("RiakURLSigKey gatherKeys unable to unamrshal object into tc.URLSigKeys: %w", err)
+		}
+		tbl.Records = append(tbl.Records, riakURLSigKeyRecord{
+			DeliveryService: obj.Key,
+			Key:             key,
+		})
+	}
+	return nil
+}
+func (tbl *riakURLSigKeyTable) toGeneric() []URLSigKey {
+	keys := make([]URLSigKey, len(tbl.Records))
+
+	for i, record := range tbl.Records {
+		keys[i] = URLSigKey{
+			URLSigKeys:      record.Key,
+			DeliveryService: record.DeliveryService,
+		}
+	}
+
+	return keys
+}
+func (tbl *riakURLSigKeyTable) fromGeneric(keys []URLSigKey) {
+	tbl.Records = make([]riakURLSigKeyRecord, len(keys))
+	for i, key := range keys {
+		tbl.Records[i] = riakURLSigKeyRecord{
+			Key:             key.URLSigKeys,
+			DeliveryService: key.DeliveryService,
+		}
+	}
+}
+func (tbl *riakURLSigKeyTable) insertKeys(cluster *riak.Cluster) error {
+	for _, record := range tbl.Records {
+		objBytes, err := json.Marshal(record.Key)
+		if err != nil {
+			return fmt.Errorf("RiakURLSigKey insertKeys unable to marshal keys: %w", err)
+		}
+
+		err = setObject(cluster, makeRiakObject(objBytes, "url_sig_"+record.DeliveryService+".config"), BUCKET_URL_SIG)
+		if err != nil {
+			return fmt.Errorf("RiakURLSigKey insertKeys: %w", err)
+		}
+	}
+	return nil
+}
+func (tbl *riakURLSigKeyTable) validate() []string {
+	errs := []string{}
+	for i, record := range tbl.Records {
+		if record.DeliveryService == "" {
+			errs = append(errs, fmt.Sprintf("URL Key #%v: Delivery Service is blank!", i))
+		}
+	}
+	return errs
+}
+
+type riakURISignKeyRecord struct {
+	Key             map[string]tc.URISignerKeyset
+	DeliveryService string
+}
+type riakURISignKeyTable struct {
+	Records []riakURISignKeyRecord
+}
+
+func (tbl *riakURISignKeyTable) gatherKeys(cluster *riak.Cluster) error {
+	tbl.Records = []riakURISignKeyRecord{}
+	objs, err := getObjects(cluster, BUCKET_URI_SIG)
+	if err != nil {
+		return fmt.Errorf("RiakURISignKey gatherKeys: %w", err)
+	}
+	for _, obj := range objs {
+		key := map[string]tc.URISignerKeyset{}
+		if err := json.Unmarshal(obj.Value, &key); err != nil {
+			return fmt.Errorf("RiakURISignKey gatherKeys unable to unmarshal object into map[string]tc.URISignerKeySet: %w", err)
+		}
+
+		tbl.Records = append(tbl.Records, riakURISignKeyRecord{
+			DeliveryService: obj.Key,
+			Key:             key,
+		})
+	}
+	return nil
+}
+func (tbl *riakURISignKeyTable) toGeneric() []URISignKey {
+	keys := make([]URISignKey, len(tbl.Records))
+
+	for i, record := range tbl.Records {
+		keys[i] = URISignKey{
+			DeliveryService: record.DeliveryService,
+			Keys:            record.Key,
+		}
+	}
+
+	return keys
+}
+func (tbl *riakURISignKeyTable) fromGeneric(keys []URISignKey) {
+	tbl.Records = make([]riakURISignKeyRecord, len(keys))
+
+	for i, record := range keys {
+		tbl.Records[i] = riakURISignKeyRecord{
+			Key:             record.Keys,
+			DeliveryService: record.DeliveryService,
+		}
+	}
+}
+func (tbl *riakURISignKeyTable) insertKeys(cluster *riak.Cluster) error {
+	for _, record := range tbl.Records {
+		objBytes, err := json.Marshal(record.Key)
+		if err != nil {
+			return fmt.Errorf("RiakURISignKey insertKeys: unable to marshal key: %w", err)
+		}
+
+		err = setObject(cluster, makeRiakObject(objBytes, record.DeliveryService), BUCKET_URI_SIG)
+		if err != nil {
+			return fmt.Errorf("RiakURISignKey insertKeys: %w", err)
+		}
+	}
+	return nil
+}
+func (tbl *riakURISignKeyTable) validate() []string {
+	errs := []string{}
+	for i, record := range tbl.Records {
+		if record.DeliveryService == "" {
+			errs = append(errs, fmt.Sprintf("URI Key #%v: Delivery Service is blank!", i))
+		}
+	}
+	return errs
+}
+
+// Riak functions
+func makeRiakObject(data []byte, key string) *riak.Object {
+	return &riak.Object{
+		ContentType:     rfc.ApplicationJSON,
+		Charset:         "utf-8",
+		ContentEncoding: "utf-8",
+		Key:             key,
+		Value:           data,
+	}
+}
+func getObjects(cluster *riak.Cluster, bucket string) ([]*riak.Object, error) {
+	objs := []*riak.Object{}
+	keys, err := listKeys(cluster, bucket)
+	if err != nil {
+		return nil, err
+	}
+	for _, key := range keys {
+		objects, err := getObject(cluster, bucket, key)
+		if err != nil {
+			return nil, err
+		}
+		if len(objects) > 1 {
+			return nil, fmt.Errorf("Unexpected number of objects %v, ignoring\n", len(objects))
+		}
+
+		objs = append(objs, objects[0])
+	}
+
+	return objs, nil
+}
+func getObject(cluster *riak.Cluster, bucket string, key string) ([]*riak.Object, error) {
+	cmd, err := riak.NewFetchValueCommandBuilder().
+		WithBucket(bucket).
+		WithKey(key).
+		WithTimeout(time.Second * 60).
+		Build()
+	if err != nil {
+		return nil, fmt.Errorf("error building riak fetch value command: %w", err)
+	}
+
+	if err := cluster.Execute(cmd); err != nil {
+		return nil, fmt.Errorf("error executing riak fetch value command: %w", err)
+	}
+
+	fvc := cmd.(*riak.FetchValueCommand)
+	rsp := fvc.Response
+
+	if rsp.IsNotFound {
+		return nil, fmt.Errorf("errors executing riak fetch value command; key not found: %v:%v", bucket, key)
+	}
+
+	return rsp.Values, nil
+}
+func setObject(cluster *riak.Cluster, obj *riak.Object, bucket string) error {
+	cmd, err := riak.NewStoreValueCommandBuilder().
+		WithBucket(bucket).
+		WithContent(obj).
+		WithTimeout(time.Second * 5).
+		Build()
+	if err != nil {
+		return fmt.Errorf("error building riak store value command: %w", err)
+	}
+
+	return cluster.Execute(cmd)
+}
+func search(cluster *riak.Cluster, index string, query string, filterQuery string, numRows uint32, fields []string) ([]*riak.SearchDoc, error) {
+	var searchDocs []*riak.SearchDoc
+	start := uint32(0)
+	for i := uint32(0); ; i += 1 {
+		riakCmd := riak.NewSearchCommandBuilder().
+			WithQuery(query).
+			WithNumRows(numRows).
+			WithStart(start)
+		if len(index) > 0 {
+			riakCmd = riakCmd.WithIndexName(index)
+		}
+		if len(filterQuery) > 0 {
+			riakCmd = riakCmd.WithFilterQuery(filterQuery)
+		}
+		if len(fields) > 0 {
+			riakCmd = riakCmd.WithReturnFields(fields...)
+		}
+		iCmd, err := riakCmd.Build()
+
+		if err != nil {
+			return nil, fmt.Errorf("building Riak search command: %w", err)
+		}
+		if err = cluster.Execute(iCmd); err != nil {
+			return nil, fmt.Errorf("executing Riak search command index '%v' query '%v': %w", index, query, err)

Review comment:
       nit: type-specific formatting instead of %v

##########
File path: tools/traffic_vault_migrate/riak.go
##########
@@ -0,0 +1,746 @@
+package main
+
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+import (
+	"crypto/tls"
+	"encoding/json"
+	"errors"
+	"fmt"
+	"time"
+
+	"github.com/apache/trafficcontrol/lib/go-rfc"
+	"github.com/apache/trafficcontrol/lib/go-tc"
+
+	"github.com/basho/riak-go-client"
+)
+
+const (
+	BUCKET_SSL     = "ssl"
+	BUCKET_DNSSEC  = "dnssec"
+	BUCKET_URL_SIG = "url_sig_keys"
+	BUCKET_URI_SIG = "cdn_uri_sig_keys"
+
+	INDEX_SSL = "sslkeys"
+
+	SCHEMA_RIAK_KEY    = "_yz_rk"
+	SCHEMA_RIAK_BUCKET = "_yz_rb"
+)
+
+var (
+	SCHEMA_SSL_FIELDS = [...]string{SCHEMA_RIAK_KEY, SCHEMA_RIAK_BUCKET}
+)
+
+// RiakConfig  represents the configuration options available to the Riak backend
+type RiakConfig struct {
+	Host          string `json:"host"`
+	Port          string `json:"port"`
+	User          string `json:"user"`
+	Password      string `json:"password"`
+	Insecure      bool   `json:"insecure"`
+	TLSVersionRaw string `json:"tlsVersion"`
+	TLSVersion    uint16
+}
+
+// RiakBackend is the Riak implementation of TVBackend
+type RiakBackend struct {
+	sslKeys     riakSSLKeyTable
+	dnssecKeys  riakDNSSecKeyTable
+	uriSignKeys riakURISignKeyTable
+	urlSigKeys  riakURLSigKeyTable
+	cfg         RiakConfig
+	cluster     *riak.Cluster
+}
+
+// String returns a high level overview of the backend and its keys
+func (rb *RiakBackend) String() string {
+	data := fmt.Sprintf("Riak server %v@%v:%v\n", rb.cfg.User, rb.cfg.Host, rb.cfg.Port)
+	data += fmt.Sprintf("\tSSL Keys: %v\n", len(rb.sslKeys.Records))
+	data += fmt.Sprintf("\tDNSSec Keys: %v\n", len(rb.dnssecKeys.Records))
+	data += fmt.Sprintf("\tURI Keys: %v\n", len(rb.uriSignKeys.Records))
+	data += fmt.Sprintf("\tURL Keys: %v\n", len(rb.urlSigKeys.Records))
+	return data
+}
+
+// Name returns the name for this backend
+func (rb *RiakBackend) Name() string {
+	return "Riak"
+}
+
+// ReadConfig takes in a filename and will read it into the backends config
+func (rb *RiakBackend) ReadConfig(s string) error {
+	err := UnmarshalConfig(s, &rb.cfg)
+	if err != nil {
+		return err
+	}
+
+	if rb.cfg.TLSVersionRaw == "10" {
+		rb.cfg.TLSVersion = tls.VersionTLS10
+	} else if rb.cfg.TLSVersionRaw == "11" {
+		rb.cfg.TLSVersion = tls.VersionTLS11
+	} else if rb.cfg.TLSVersionRaw == "12" {
+		rb.cfg.TLSVersion = tls.VersionTLS12
+	} else if rb.cfg.TLSVersionRaw == "13" {
+		rb.cfg.TLSVersion = tls.VersionTLS13
+	}
+	return nil
+}
+
+// Insert takes the current keys and inserts them into the backend DB
+func (rb *RiakBackend) Insert() error {
+	if err := rb.sslKeys.insertKeys(rb.cluster); err != nil {
+		return err
+	}
+	if err := rb.dnssecKeys.insertKeys(rb.cluster); err != nil {
+		return err
+	}
+	if err := rb.urlSigKeys.insertKeys(rb.cluster); err != nil {
+		return err
+	}
+	if err := rb.uriSignKeys.insertKeys(rb.cluster); err != nil {
+		return err
+	}
+	return nil
+}
+
+// ValidateKey validates that the keys are valid (in most cases, certain fields are not null)
+func (rb *RiakBackend) ValidateKey() []string {
+	errors := []string{}
+	if errs := rb.sslKeys.validate(); errs != nil {
+		errors = append(errors, errs...)
+	}
+	if errs := rb.dnssecKeys.validate(); errs != nil {
+		errors = append(errors, errs...)
+	}
+	if errs := rb.uriSignKeys.validate(); errs != nil {
+		errors = append(errors, errs...)
+	}
+	if errs := rb.urlSigKeys.validate(); errs != nil {
+		errors = append(errors, errs...)
+	}
+
+	return errors
+}
+
+// SetSSLKeys takes in keys and converts & encrypts the data into the backends internal format
+func (rb *RiakBackend) SetSSLKeys(keys []SSLKey) error {
+	rb.sslKeys.fromGeneric(keys)
+	return nil
+}
+
+// SetDNSSecKeys takes in keys and converts & encrypts the data into the backends internal format
+func (rb *RiakBackend) SetDNSSecKeys(keys []DNSSecKey) error {
+	rb.dnssecKeys.fromGeneric(keys)
+	return nil
+}
+
+// SetURISignKeys takes in keys and converts & encrypts the data into the backends internal format
+func (rb *RiakBackend) SetURISignKeys(keys []URISignKey) error {
+	rb.uriSignKeys.fromGeneric(keys)
+	return nil
+}
+
+// SetURLSigKeys takes in keys and converts & encrypts the data into the backends internal format
+func (rb *RiakBackend) SetURLSigKeys(keys []URLSigKey) error {
+	rb.urlSigKeys.fromGeneric(keys)
+	return nil
+}
+
+// Start initiates the connection to the backend DB
+func (rb *RiakBackend) Start() error {
+	tlsConfig := &tls.Config{
+		InsecureSkipVerify: !rb.cfg.Insecure,
+		MaxVersion:         rb.cfg.TLSVersion,
+	}
+	auth := &riak.AuthOptions{
+		User:      rb.cfg.User,
+		Password:  rb.cfg.Password,
+		TlsConfig: tlsConfig,
+	}
+
+	cluster, err := getRiakCluster(rb.cfg, auth)
+	if err != nil {
+		return err
+	}
+	if err := cluster.Start(); err != nil {
+		return fmt.Errorf("unable to start riak cluster: %w", err)
+	}
+
+	rb.cluster = cluster
+	rb.sslKeys = riakSSLKeyTable{}
+	rb.dnssecKeys = riakDNSSecKeyTable{}
+	rb.urlSigKeys = riakURLSigKeyTable{}
+	rb.uriSignKeys = riakURISignKeyTable{}
+	return nil
+}
+
+// Close terminates the connection to the backend DB
+func (rb *RiakBackend) Close() error {
+	if err := rb.cluster.Stop(); err != nil {
+		return err
+	}
+	return nil
+}
+
+// Ping checks the connection to the backend DB
+func (rb *RiakBackend) Ping() error {
+	return ping(rb.cluster)
+}
+
+// GetSSLKeys converts the backends internal key representation into the common representation (SSLKey)
+func (rb *RiakBackend) GetSSLKeys() ([]SSLKey, error) {
+	return rb.sslKeys.toGeneric(), nil
+}
+
+// GetDNSSecKeys converts the backends internal key representation into the common representation (DNSSecKey)
+func (rb *RiakBackend) GetDNSSecKeys() ([]DNSSecKey, error) {
+	return rb.dnssecKeys.toGeneric(), nil
+}
+
+// GetURISignKeys converts the pg internal key representation into the common representation (URISignKey)
+func (rb *RiakBackend) GetURISignKeys() ([]URISignKey, error) {
+	return rb.uriSignKeys.toGeneric(), nil
+}
+
+// GetURLSigKeys converts the backends internal key representation into the common representation (URLSigKey)
+func (rb *RiakBackend) GetURLSigKeys() ([]URLSigKey, error) {
+	return rb.urlSigKeys.toGeneric(), nil
+}
+
+// Fetch gets all of the keys from the backend DB
+func (rb *RiakBackend) Fetch() error {
+	if err := rb.sslKeys.gatherKeys(rb.cluster); err != nil {
+		return err
+	}
+	if err := rb.dnssecKeys.gatherKeys(rb.cluster); err != nil {
+		return err
+	}
+	if err := rb.urlSigKeys.gatherKeys(rb.cluster); err != nil {
+		return err
+	}
+	if err := rb.uriSignKeys.gatherKeys(rb.cluster); err != nil {
+		return err
+	}
+
+	return nil
+}
+
+type riakSSLKeyRecord struct {
+	tc.DeliveryServiceSSLKeys
+}
+type riakSSLKeyTable struct {
+	Records []riakSSLKeyRecord
+}
+
+func (tbl *riakSSLKeyTable) gatherKeys(cluster *riak.Cluster) error {
+	searchDocs, err := search(cluster, INDEX_SSL, "cdn:*", fmt.Sprintf("%v:*latest", SCHEMA_RIAK_KEY), 1000, SCHEMA_SSL_FIELDS[:])
+	if err != nil {
+		return fmt.Errorf("RiakSSLKey gatherKeys: %w", err)
+	}
+
+	tbl.Records = make([]riakSSLKeyRecord, len(searchDocs))
+	for i, doc := range searchDocs {
+		objs, err := getObject(cluster, doc.Bucket, doc.Key)
+		if err != nil {
+			return err
+		}
+		if len(objs) < 1 {
+			return fmt.Errorf("RiakSSLKey gatherKeys unable to find any objects with key %v and bucket %v, but search results were returned", doc.Key, doc.Bucket)
+		}
+		if len(objs) > 1 {
+			return fmt.Errorf("RiakSSLKey gatherKeys more than 1 ssl key record found %v\n", len(objs))
+		}
+		var obj tc.DeliveryServiceSSLKeys
+		if err = json.Unmarshal(objs[0].Value, &obj); err != nil {
+			return fmt.Errorf("RiakSSLKey gatherKeys unable to unmarshal object into tc.DeliveryServiceSSLKeys: %w", err)
+		}
+		tbl.Records[i] = riakSSLKeyRecord{
+			DeliveryServiceSSLKeys: obj,
+		}
+	}
+	return nil
+}
+func (tbl *riakSSLKeyTable) toGeneric() []SSLKey {
+	keys := make([]SSLKey, len(tbl.Records))
+
+	for i, record := range tbl.Records {
+		keys[i] = SSLKey{
+			DeliveryServiceSSLKeys: record.DeliveryServiceSSLKeys,
+		}
+	}
+
+	return keys
+}
+func (tbl *riakSSLKeyTable) fromGeneric(keys []SSLKey) {
+	tbl.Records = make([]riakSSLKeyRecord, len(keys))
+
+	for i, record := range keys {
+		tbl.Records[i] = riakSSLKeyRecord{
+			DeliveryServiceSSLKeys: record.DeliveryServiceSSLKeys,
+		}
+	}
+}
+func (tbl *riakSSLKeyTable) insertKeys(cluster *riak.Cluster) error {
+	for _, record := range tbl.Records {
+		objBytes, err := json.Marshal(record.DeliveryServiceSSLKeys)
+		if err != nil {
+			return fmt.Errorf("RiakSSLKey insertKeys failed to unmarshal keys: %w", err)
+		}
+		err = setObject(cluster, makeRiakObject(objBytes, record.DeliveryService+"-"+record.Version.String()), BUCKET_SSL)
+		if err != nil {
+			return fmt.Errorf("RiakSSLKey insertKeys: %w", err)
+		}
+		err = setObject(cluster, makeRiakObject(objBytes, record.DeliveryService+"-latest"), BUCKET_SSL)
+		if err != nil {
+			return fmt.Errorf("RiakSSLKey insertKeys: %w", err)
+		}
+	}
+	return nil
+}
+func (tbl *riakSSLKeyTable) validate() []string {
+	errs := []string{}
+	for i, record := range tbl.Records {
+		if record.DeliveryService == "" {
+			errs = append(errs, fmt.Sprintf("SSL Key #%v: Delivery Service is blank!", i))
+		}
+		if record.CDN == "" {
+			errs = append(errs, fmt.Sprintf("SSL Key #%v: CDN is blank!", i))
+		}
+		if record.Version.String() == "" {
+			errs = append(errs, fmt.Sprintf("SSL Key #%v: Version is blank!", i))
+		}
+	}
+	return errs
+}
+
+type riakDNSSecKeyRecord struct {
+	CDN string
+	Key tc.DNSSECKeysRiak
+}
+type riakDNSSecKeyTable struct {
+	Records []riakDNSSecKeyRecord
+}
+
+func (tbl *riakDNSSecKeyTable) gatherKeys(cluster *riak.Cluster) error {
+	tbl.Records = []riakDNSSecKeyRecord{}
+	objs, err := getObjects(cluster, BUCKET_DNSSEC)
+	if err != nil {
+		return fmt.Errorf("RiakDNSSecKey gatherKeys: %w", err)
+	}
+	for _, obj := range objs {
+		key := tc.DNSSECKeysRiak{}
+		if err := json.Unmarshal(obj.Value, &key); err != nil {
+			return fmt.Errorf("RiakDNSSecKey gatherKeys unable to unmarshal object to tc.DNSSECKeysRiak: %w", err)
+		}
+		tbl.Records = append(tbl.Records, riakDNSSecKeyRecord{
+			CDN: obj.Key,
+			Key: key,
+		})
+	}
+	return nil
+}
+func (tbl *riakDNSSecKeyTable) toGeneric() []DNSSecKey {
+	keys := make([]DNSSecKey, len(tbl.Records))
+
+	for i, record := range tbl.Records {
+		keys[i] = DNSSecKey{
+			CDN:                    record.CDN,
+			DNSSECKeysTrafficVault: tc.DNSSECKeysTrafficVault(record.Key),
+		}
+	}
+
+	return keys
+}
+func (tbl *riakDNSSecKeyTable) fromGeneric(keys []DNSSecKey) {
+	tbl.Records = make([]riakDNSSecKeyRecord, len(keys))
+
+	for i, record := range keys {
+		tbl.Records[i] = riakDNSSecKeyRecord{
+			CDN: record.CDN,
+			Key: tc.DNSSECKeysRiak(record.DNSSECKeysTrafficVault),
+		}
+	}
+}
+func (tbl *riakDNSSecKeyTable) insertKeys(cluster *riak.Cluster) error {
+	for _, record := range tbl.Records {
+		objBytes, err := json.Marshal(record.Key)
+		if err != nil {
+			return fmt.Errorf("RiakDNSSecKey insertKeys error marshalling keys: %w", err)
+		}
+
+		err = setObject(cluster, makeRiakObject(objBytes, record.CDN), BUCKET_DNSSEC)
+		if err != nil {
+			return fmt.Errorf("RiakDNSSecKey insertKeys: %w", err)
+		}
+	}
+	return nil
+}
+func (tbl *riakDNSSecKeyTable) validate() []string {
+	errs := []string{}
+	for i, record := range tbl.Records {
+		if record.CDN == "" {
+			errs = append(errs, fmt.Sprintf("DNSSec Key #%v: CDN is blank!", i))
+		}
+	}
+	return errs
+}
+
+type riakURLSigKeyRecord struct {
+	Key             tc.URLSigKeys
+	DeliveryService string
+}
+type riakURLSigKeyTable struct {
+	Records []riakURLSigKeyRecord
+}
+
+func (tbl *riakURLSigKeyTable) gatherKeys(cluster *riak.Cluster) error {
+	tbl.Records = []riakURLSigKeyRecord{}
+	objs, err := getObjects(cluster, BUCKET_URL_SIG)
+	if err != nil {
+		return fmt.Errorf("RiakURLSigKey gatherKeys: %w", err)
+	}
+	for _, obj := range objs {
+		key := tc.URLSigKeys{}
+		if err := json.Unmarshal(obj.Value, &key); err != nil {
+			return fmt.Errorf("RiakURLSigKey gatherKeys unable to unamrshal object into tc.URLSigKeys: %w", err)
+		}
+		tbl.Records = append(tbl.Records, riakURLSigKeyRecord{
+			DeliveryService: obj.Key,
+			Key:             key,
+		})
+	}
+	return nil
+}
+func (tbl *riakURLSigKeyTable) toGeneric() []URLSigKey {
+	keys := make([]URLSigKey, len(tbl.Records))
+
+	for i, record := range tbl.Records {
+		keys[i] = URLSigKey{
+			URLSigKeys:      record.Key,
+			DeliveryService: record.DeliveryService,
+		}
+	}
+
+	return keys
+}
+func (tbl *riakURLSigKeyTable) fromGeneric(keys []URLSigKey) {
+	tbl.Records = make([]riakURLSigKeyRecord, len(keys))
+	for i, key := range keys {
+		tbl.Records[i] = riakURLSigKeyRecord{
+			Key:             key.URLSigKeys,
+			DeliveryService: key.DeliveryService,
+		}
+	}
+}
+func (tbl *riakURLSigKeyTable) insertKeys(cluster *riak.Cluster) error {
+	for _, record := range tbl.Records {
+		objBytes, err := json.Marshal(record.Key)
+		if err != nil {
+			return fmt.Errorf("RiakURLSigKey insertKeys unable to marshal keys: %w", err)
+		}
+
+		err = setObject(cluster, makeRiakObject(objBytes, "url_sig_"+record.DeliveryService+".config"), BUCKET_URL_SIG)
+		if err != nil {
+			return fmt.Errorf("RiakURLSigKey insertKeys: %w", err)
+		}
+	}
+	return nil
+}
+func (tbl *riakURLSigKeyTable) validate() []string {
+	errs := []string{}
+	for i, record := range tbl.Records {
+		if record.DeliveryService == "" {
+			errs = append(errs, fmt.Sprintf("URL Key #%v: Delivery Service is blank!", i))
+		}
+	}
+	return errs
+}
+
+type riakURISignKeyRecord struct {
+	Key             map[string]tc.URISignerKeyset
+	DeliveryService string
+}
+type riakURISignKeyTable struct {
+	Records []riakURISignKeyRecord
+}
+
+func (tbl *riakURISignKeyTable) gatherKeys(cluster *riak.Cluster) error {
+	tbl.Records = []riakURISignKeyRecord{}
+	objs, err := getObjects(cluster, BUCKET_URI_SIG)
+	if err != nil {
+		return fmt.Errorf("RiakURISignKey gatherKeys: %w", err)
+	}
+	for _, obj := range objs {
+		key := map[string]tc.URISignerKeyset{}
+		if err := json.Unmarshal(obj.Value, &key); err != nil {
+			return fmt.Errorf("RiakURISignKey gatherKeys unable to unmarshal object into map[string]tc.URISignerKeySet: %w", err)
+		}
+
+		tbl.Records = append(tbl.Records, riakURISignKeyRecord{
+			DeliveryService: obj.Key,
+			Key:             key,
+		})
+	}
+	return nil
+}
+func (tbl *riakURISignKeyTable) toGeneric() []URISignKey {
+	keys := make([]URISignKey, len(tbl.Records))
+
+	for i, record := range tbl.Records {
+		keys[i] = URISignKey{
+			DeliveryService: record.DeliveryService,
+			Keys:            record.Key,
+		}
+	}
+
+	return keys
+}
+func (tbl *riakURISignKeyTable) fromGeneric(keys []URISignKey) {
+	tbl.Records = make([]riakURISignKeyRecord, len(keys))
+
+	for i, record := range keys {
+		tbl.Records[i] = riakURISignKeyRecord{
+			Key:             record.Keys,
+			DeliveryService: record.DeliveryService,
+		}
+	}
+}
+func (tbl *riakURISignKeyTable) insertKeys(cluster *riak.Cluster) error {
+	for _, record := range tbl.Records {
+		objBytes, err := json.Marshal(record.Key)
+		if err != nil {
+			return fmt.Errorf("RiakURISignKey insertKeys: unable to marshal key: %w", err)
+		}
+
+		err = setObject(cluster, makeRiakObject(objBytes, record.DeliveryService), BUCKET_URI_SIG)
+		if err != nil {
+			return fmt.Errorf("RiakURISignKey insertKeys: %w", err)
+		}
+	}
+	return nil
+}
+func (tbl *riakURISignKeyTable) validate() []string {
+	errs := []string{}
+	for i, record := range tbl.Records {
+		if record.DeliveryService == "" {
+			errs = append(errs, fmt.Sprintf("URI Key #%v: Delivery Service is blank!", i))
+		}
+	}
+	return errs
+}
+
+// Riak functions
+func makeRiakObject(data []byte, key string) *riak.Object {
+	return &riak.Object{
+		ContentType:     rfc.ApplicationJSON,
+		Charset:         "utf-8",
+		ContentEncoding: "utf-8",
+		Key:             key,
+		Value:           data,
+	}
+}
+func getObjects(cluster *riak.Cluster, bucket string) ([]*riak.Object, error) {
+	objs := []*riak.Object{}
+	keys, err := listKeys(cluster, bucket)
+	if err != nil {
+		return nil, err
+	}
+	for _, key := range keys {
+		objects, err := getObject(cluster, bucket, key)
+		if err != nil {
+			return nil, err
+		}
+		if len(objects) > 1 {
+			return nil, fmt.Errorf("Unexpected number of objects %v, ignoring\n", len(objects))
+		}
+
+		objs = append(objs, objects[0])
+	}
+
+	return objs, nil
+}
+func getObject(cluster *riak.Cluster, bucket string, key string) ([]*riak.Object, error) {
+	cmd, err := riak.NewFetchValueCommandBuilder().
+		WithBucket(bucket).
+		WithKey(key).
+		WithTimeout(time.Second * 60).
+		Build()
+	if err != nil {
+		return nil, fmt.Errorf("error building riak fetch value command: %w", err)
+	}
+
+	if err := cluster.Execute(cmd); err != nil {
+		return nil, fmt.Errorf("error executing riak fetch value command: %w", err)
+	}
+
+	fvc := cmd.(*riak.FetchValueCommand)
+	rsp := fvc.Response
+
+	if rsp.IsNotFound {
+		return nil, fmt.Errorf("errors executing riak fetch value command; key not found: %v:%v", bucket, key)
+	}
+
+	return rsp.Values, nil
+}
+func setObject(cluster *riak.Cluster, obj *riak.Object, bucket string) error {
+	cmd, err := riak.NewStoreValueCommandBuilder().
+		WithBucket(bucket).
+		WithContent(obj).
+		WithTimeout(time.Second * 5).
+		Build()
+	if err != nil {
+		return fmt.Errorf("error building riak store value command: %w", err)
+	}
+
+	return cluster.Execute(cmd)
+}
+func search(cluster *riak.Cluster, index string, query string, filterQuery string, numRows uint32, fields []string) ([]*riak.SearchDoc, error) {
+	var searchDocs []*riak.SearchDoc
+	start := uint32(0)
+	for i := uint32(0); ; i += 1 {
+		riakCmd := riak.NewSearchCommandBuilder().
+			WithQuery(query).
+			WithNumRows(numRows).
+			WithStart(start)
+		if len(index) > 0 {
+			riakCmd = riakCmd.WithIndexName(index)
+		}
+		if len(filterQuery) > 0 {
+			riakCmd = riakCmd.WithFilterQuery(filterQuery)
+		}
+		if len(fields) > 0 {
+			riakCmd = riakCmd.WithReturnFields(fields...)
+		}
+		iCmd, err := riakCmd.Build()
+
+		if err != nil {
+			return nil, fmt.Errorf("building Riak search command: %w", err)
+		}
+		if err = cluster.Execute(iCmd); err != nil {
+			return nil, fmt.Errorf("executing Riak search command index '%v' query '%v': %w", index, query, err)
+		}
+		cmd, ok := iCmd.(*riak.SearchCommand)
+		if !ok {
+			return nil, fmt.Errorf("riak search command unexpected type %T", iCmd)
+		}
+		if start == 0 {
+			if cmd.Response == nil || cmd.Response.NumFound == 0 {
+				return nil, nil
+			}
+			if cmd.Response.NumFound <= numRows {
+				return cmd.Response.Docs, nil
+			} else if numRows < cmd.Response.NumFound {
+				searchDocs = make([]*riak.SearchDoc, cmd.Response.NumFound)
+			}
+			if cmd.Response.NumFound > 10000 {
+				fmt.Printf("WARNING: found %v rows, press enter to continue", cmd.Response.NumFound)
+				_, _ = fmt.Scanln()
+			}
+		}
+
+		// If the total number of docs is not evenly divisible by 1000
+		if uint32(len(cmd.Response.Docs)) < numRows {
+			numRows = uint32(len(cmd.Response.Docs))
+		}
+
+		for responseIndex := uint32(0); responseIndex < numRows; responseIndex += 1 {
+			returnIndex := responseIndex + start
+			searchDocs[returnIndex] = cmd.Response.Docs[responseIndex]
+		}
+		if cmd.Response.NumFound == numRows+start {
+			return searchDocs, nil
+		}
+		start += numRows
+	}
+}
+func listKeys(cluster *riak.Cluster, bucket string) ([]string, error) {
+	cmd, err := riak.NewListKeysCommandBuilder().
+		WithBucket(bucket).
+		WithTimeout(time.Second * 60).
+		WithAllowListing().
+		Build()
+	if err != nil {
+		return nil, errors.New("building riak list keys command failed: " + err.Error())
+	}
+
+	if err := cluster.Execute(cmd); err != nil {
+		return nil, errors.New("error executing riak list keys command: " + err.Error())
+	}
+
+	lkc := cmd.(*riak.ListKeysCommand)
+	rsp := lkc.Response
+	return rsp.Keys, nil
+}
+func ping(cluster *riak.Cluster) error {
+	ping := riak.PingCommandBuilder{}
+	cmd, err := ping.Build()
+	if err != nil {
+		return errors.New("failed to build riak ping command: " + err.Error())
+	}
+
+	if err = cluster.Execute(cmd); err != nil {
+		return err
+	}
+	response, ok := cmd.(*riak.PingCommand)
+	if !ok {
+		return errors.New(fmt.Sprintf("unexpected riak command type for ping: %v", cmd))

Review comment:
       nit: `fmt.Errorf`

##########
File path: tools/traffic_vault_migrate/riak.go
##########
@@ -0,0 +1,746 @@
+package main
+
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+import (
+	"crypto/tls"
+	"encoding/json"
+	"errors"
+	"fmt"
+	"time"
+
+	"github.com/apache/trafficcontrol/lib/go-rfc"
+	"github.com/apache/trafficcontrol/lib/go-tc"
+
+	"github.com/basho/riak-go-client"
+)
+
+const (
+	BUCKET_SSL     = "ssl"
+	BUCKET_DNSSEC  = "dnssec"
+	BUCKET_URL_SIG = "url_sig_keys"
+	BUCKET_URI_SIG = "cdn_uri_sig_keys"
+
+	INDEX_SSL = "sslkeys"
+
+	SCHEMA_RIAK_KEY    = "_yz_rk"
+	SCHEMA_RIAK_BUCKET = "_yz_rb"
+)
+
+var (
+	SCHEMA_SSL_FIELDS = [...]string{SCHEMA_RIAK_KEY, SCHEMA_RIAK_BUCKET}
+)
+
+// RiakConfig  represents the configuration options available to the Riak backend
+type RiakConfig struct {
+	Host          string `json:"host"`
+	Port          string `json:"port"`
+	User          string `json:"user"`
+	Password      string `json:"password"`
+	Insecure      bool   `json:"insecure"`
+	TLSVersionRaw string `json:"tlsVersion"`
+	TLSVersion    uint16
+}
+
+// RiakBackend is the Riak implementation of TVBackend
+type RiakBackend struct {
+	sslKeys     riakSSLKeyTable
+	dnssecKeys  riakDNSSecKeyTable
+	uriSignKeys riakURISignKeyTable
+	urlSigKeys  riakURLSigKeyTable
+	cfg         RiakConfig
+	cluster     *riak.Cluster
+}
+
+// String returns a high level overview of the backend and its keys
+func (rb *RiakBackend) String() string {
+	data := fmt.Sprintf("Riak server %v@%v:%v\n", rb.cfg.User, rb.cfg.Host, rb.cfg.Port)
+	data += fmt.Sprintf("\tSSL Keys: %v\n", len(rb.sslKeys.Records))
+	data += fmt.Sprintf("\tDNSSec Keys: %v\n", len(rb.dnssecKeys.Records))
+	data += fmt.Sprintf("\tURI Keys: %v\n", len(rb.uriSignKeys.Records))
+	data += fmt.Sprintf("\tURL Keys: %v\n", len(rb.urlSigKeys.Records))
+	return data
+}
+
+// Name returns the name for this backend
+func (rb *RiakBackend) Name() string {
+	return "Riak"
+}
+
+// ReadConfig takes in a filename and will read it into the backends config
+func (rb *RiakBackend) ReadConfig(s string) error {
+	err := UnmarshalConfig(s, &rb.cfg)
+	if err != nil {
+		return err
+	}
+
+	if rb.cfg.TLSVersionRaw == "10" {
+		rb.cfg.TLSVersion = tls.VersionTLS10
+	} else if rb.cfg.TLSVersionRaw == "11" {
+		rb.cfg.TLSVersion = tls.VersionTLS11
+	} else if rb.cfg.TLSVersionRaw == "12" {
+		rb.cfg.TLSVersion = tls.VersionTLS12
+	} else if rb.cfg.TLSVersionRaw == "13" {
+		rb.cfg.TLSVersion = tls.VersionTLS13
+	}
+	return nil
+}
+
+// Insert takes the current keys and inserts them into the backend DB
+func (rb *RiakBackend) Insert() error {
+	if err := rb.sslKeys.insertKeys(rb.cluster); err != nil {
+		return err
+	}
+	if err := rb.dnssecKeys.insertKeys(rb.cluster); err != nil {
+		return err
+	}
+	if err := rb.urlSigKeys.insertKeys(rb.cluster); err != nil {
+		return err
+	}
+	if err := rb.uriSignKeys.insertKeys(rb.cluster); err != nil {
+		return err
+	}
+	return nil
+}
+
+// ValidateKey validates that the keys are valid (in most cases, certain fields are not null)
+func (rb *RiakBackend) ValidateKey() []string {
+	errors := []string{}
+	if errs := rb.sslKeys.validate(); errs != nil {
+		errors = append(errors, errs...)
+	}
+	if errs := rb.dnssecKeys.validate(); errs != nil {
+		errors = append(errors, errs...)
+	}
+	if errs := rb.uriSignKeys.validate(); errs != nil {
+		errors = append(errors, errs...)
+	}
+	if errs := rb.urlSigKeys.validate(); errs != nil {
+		errors = append(errors, errs...)
+	}
+
+	return errors
+}
+
+// SetSSLKeys takes in keys and converts & encrypts the data into the backends internal format
+func (rb *RiakBackend) SetSSLKeys(keys []SSLKey) error {
+	rb.sslKeys.fromGeneric(keys)
+	return nil
+}
+
+// SetDNSSecKeys takes in keys and converts & encrypts the data into the backends internal format
+func (rb *RiakBackend) SetDNSSecKeys(keys []DNSSecKey) error {
+	rb.dnssecKeys.fromGeneric(keys)
+	return nil
+}
+
+// SetURISignKeys takes in keys and converts & encrypts the data into the backends internal format
+func (rb *RiakBackend) SetURISignKeys(keys []URISignKey) error {
+	rb.uriSignKeys.fromGeneric(keys)
+	return nil
+}
+
+// SetURLSigKeys takes in keys and converts & encrypts the data into the backends internal format
+func (rb *RiakBackend) SetURLSigKeys(keys []URLSigKey) error {
+	rb.urlSigKeys.fromGeneric(keys)
+	return nil
+}
+
+// Start initiates the connection to the backend DB
+func (rb *RiakBackend) Start() error {
+	tlsConfig := &tls.Config{
+		InsecureSkipVerify: !rb.cfg.Insecure,
+		MaxVersion:         rb.cfg.TLSVersion,
+	}
+	auth := &riak.AuthOptions{
+		User:      rb.cfg.User,
+		Password:  rb.cfg.Password,
+		TlsConfig: tlsConfig,
+	}
+
+	cluster, err := getRiakCluster(rb.cfg, auth)
+	if err != nil {
+		return err
+	}
+	if err := cluster.Start(); err != nil {
+		return fmt.Errorf("unable to start riak cluster: %w", err)
+	}
+
+	rb.cluster = cluster
+	rb.sslKeys = riakSSLKeyTable{}
+	rb.dnssecKeys = riakDNSSecKeyTable{}
+	rb.urlSigKeys = riakURLSigKeyTable{}
+	rb.uriSignKeys = riakURISignKeyTable{}
+	return nil
+}
+
+// Close terminates the connection to the backend DB
+func (rb *RiakBackend) Close() error {
+	if err := rb.cluster.Stop(); err != nil {
+		return err
+	}
+	return nil
+}
+
+// Ping checks the connection to the backend DB
+func (rb *RiakBackend) Ping() error {
+	return ping(rb.cluster)
+}
+
+// GetSSLKeys converts the backends internal key representation into the common representation (SSLKey)
+func (rb *RiakBackend) GetSSLKeys() ([]SSLKey, error) {
+	return rb.sslKeys.toGeneric(), nil
+}
+
+// GetDNSSecKeys converts the backends internal key representation into the common representation (DNSSecKey)
+func (rb *RiakBackend) GetDNSSecKeys() ([]DNSSecKey, error) {
+	return rb.dnssecKeys.toGeneric(), nil
+}
+
+// GetURISignKeys converts the pg internal key representation into the common representation (URISignKey)
+func (rb *RiakBackend) GetURISignKeys() ([]URISignKey, error) {
+	return rb.uriSignKeys.toGeneric(), nil
+}
+
+// GetURLSigKeys converts the backends internal key representation into the common representation (URLSigKey)
+func (rb *RiakBackend) GetURLSigKeys() ([]URLSigKey, error) {
+	return rb.urlSigKeys.toGeneric(), nil
+}
+
+// Fetch gets all of the keys from the backend DB
+func (rb *RiakBackend) Fetch() error {
+	if err := rb.sslKeys.gatherKeys(rb.cluster); err != nil {
+		return err
+	}
+	if err := rb.dnssecKeys.gatherKeys(rb.cluster); err != nil {
+		return err
+	}
+	if err := rb.urlSigKeys.gatherKeys(rb.cluster); err != nil {
+		return err
+	}
+	if err := rb.uriSignKeys.gatherKeys(rb.cluster); err != nil {
+		return err
+	}
+
+	return nil
+}
+
+type riakSSLKeyRecord struct {
+	tc.DeliveryServiceSSLKeys
+}
+type riakSSLKeyTable struct {
+	Records []riakSSLKeyRecord
+}
+
+func (tbl *riakSSLKeyTable) gatherKeys(cluster *riak.Cluster) error {
+	searchDocs, err := search(cluster, INDEX_SSL, "cdn:*", fmt.Sprintf("%v:*latest", SCHEMA_RIAK_KEY), 1000, SCHEMA_SSL_FIELDS[:])
+	if err != nil {
+		return fmt.Errorf("RiakSSLKey gatherKeys: %w", err)
+	}
+
+	tbl.Records = make([]riakSSLKeyRecord, len(searchDocs))
+	for i, doc := range searchDocs {
+		objs, err := getObject(cluster, doc.Bucket, doc.Key)
+		if err != nil {
+			return err
+		}
+		if len(objs) < 1 {
+			return fmt.Errorf("RiakSSLKey gatherKeys unable to find any objects with key %v and bucket %v, but search results were returned", doc.Key, doc.Bucket)
+		}
+		if len(objs) > 1 {
+			return fmt.Errorf("RiakSSLKey gatherKeys more than 1 ssl key record found %v\n", len(objs))
+		}
+		var obj tc.DeliveryServiceSSLKeys
+		if err = json.Unmarshal(objs[0].Value, &obj); err != nil {
+			return fmt.Errorf("RiakSSLKey gatherKeys unable to unmarshal object into tc.DeliveryServiceSSLKeys: %w", err)
+		}
+		tbl.Records[i] = riakSSLKeyRecord{
+			DeliveryServiceSSLKeys: obj,
+		}
+	}
+	return nil
+}
+func (tbl *riakSSLKeyTable) toGeneric() []SSLKey {
+	keys := make([]SSLKey, len(tbl.Records))
+
+	for i, record := range tbl.Records {
+		keys[i] = SSLKey{
+			DeliveryServiceSSLKeys: record.DeliveryServiceSSLKeys,
+		}
+	}
+
+	return keys
+}
+func (tbl *riakSSLKeyTable) fromGeneric(keys []SSLKey) {
+	tbl.Records = make([]riakSSLKeyRecord, len(keys))
+
+	for i, record := range keys {
+		tbl.Records[i] = riakSSLKeyRecord{
+			DeliveryServiceSSLKeys: record.DeliveryServiceSSLKeys,
+		}
+	}
+}
+func (tbl *riakSSLKeyTable) insertKeys(cluster *riak.Cluster) error {
+	for _, record := range tbl.Records {
+		objBytes, err := json.Marshal(record.DeliveryServiceSSLKeys)
+		if err != nil {
+			return fmt.Errorf("RiakSSLKey insertKeys failed to unmarshal keys: %w", err)
+		}
+		err = setObject(cluster, makeRiakObject(objBytes, record.DeliveryService+"-"+record.Version.String()), BUCKET_SSL)
+		if err != nil {
+			return fmt.Errorf("RiakSSLKey insertKeys: %w", err)
+		}
+		err = setObject(cluster, makeRiakObject(objBytes, record.DeliveryService+"-latest"), BUCKET_SSL)
+		if err != nil {
+			return fmt.Errorf("RiakSSLKey insertKeys: %w", err)
+		}
+	}
+	return nil
+}
+func (tbl *riakSSLKeyTable) validate() []string {
+	errs := []string{}
+	for i, record := range tbl.Records {
+		if record.DeliveryService == "" {
+			errs = append(errs, fmt.Sprintf("SSL Key #%v: Delivery Service is blank!", i))
+		}
+		if record.CDN == "" {
+			errs = append(errs, fmt.Sprintf("SSL Key #%v: CDN is blank!", i))
+		}
+		if record.Version.String() == "" {
+			errs = append(errs, fmt.Sprintf("SSL Key #%v: Version is blank!", i))
+		}
+	}
+	return errs
+}
+
+type riakDNSSecKeyRecord struct {
+	CDN string
+	Key tc.DNSSECKeysRiak
+}
+type riakDNSSecKeyTable struct {
+	Records []riakDNSSecKeyRecord
+}
+
+func (tbl *riakDNSSecKeyTable) gatherKeys(cluster *riak.Cluster) error {
+	tbl.Records = []riakDNSSecKeyRecord{}
+	objs, err := getObjects(cluster, BUCKET_DNSSEC)
+	if err != nil {
+		return fmt.Errorf("RiakDNSSecKey gatherKeys: %w", err)
+	}
+	for _, obj := range objs {
+		key := tc.DNSSECKeysRiak{}
+		if err := json.Unmarshal(obj.Value, &key); err != nil {
+			return fmt.Errorf("RiakDNSSecKey gatherKeys unable to unmarshal object to tc.DNSSECKeysRiak: %w", err)
+		}
+		tbl.Records = append(tbl.Records, riakDNSSecKeyRecord{
+			CDN: obj.Key,
+			Key: key,
+		})
+	}
+	return nil
+}
+func (tbl *riakDNSSecKeyTable) toGeneric() []DNSSecKey {
+	keys := make([]DNSSecKey, len(tbl.Records))
+
+	for i, record := range tbl.Records {
+		keys[i] = DNSSecKey{
+			CDN:                    record.CDN,
+			DNSSECKeysTrafficVault: tc.DNSSECKeysTrafficVault(record.Key),
+		}
+	}
+
+	return keys
+}
+func (tbl *riakDNSSecKeyTable) fromGeneric(keys []DNSSecKey) {
+	tbl.Records = make([]riakDNSSecKeyRecord, len(keys))
+
+	for i, record := range keys {
+		tbl.Records[i] = riakDNSSecKeyRecord{
+			CDN: record.CDN,
+			Key: tc.DNSSECKeysRiak(record.DNSSECKeysTrafficVault),
+		}
+	}
+}
+func (tbl *riakDNSSecKeyTable) insertKeys(cluster *riak.Cluster) error {
+	for _, record := range tbl.Records {
+		objBytes, err := json.Marshal(record.Key)
+		if err != nil {
+			return fmt.Errorf("RiakDNSSecKey insertKeys error marshalling keys: %w", err)
+		}
+
+		err = setObject(cluster, makeRiakObject(objBytes, record.CDN), BUCKET_DNSSEC)
+		if err != nil {
+			return fmt.Errorf("RiakDNSSecKey insertKeys: %w", err)
+		}
+	}
+	return nil
+}
+func (tbl *riakDNSSecKeyTable) validate() []string {
+	errs := []string{}
+	for i, record := range tbl.Records {
+		if record.CDN == "" {
+			errs = append(errs, fmt.Sprintf("DNSSec Key #%v: CDN is blank!", i))
+		}
+	}
+	return errs
+}
+
+type riakURLSigKeyRecord struct {
+	Key             tc.URLSigKeys
+	DeliveryService string
+}
+type riakURLSigKeyTable struct {
+	Records []riakURLSigKeyRecord
+}
+
+func (tbl *riakURLSigKeyTable) gatherKeys(cluster *riak.Cluster) error {
+	tbl.Records = []riakURLSigKeyRecord{}
+	objs, err := getObjects(cluster, BUCKET_URL_SIG)
+	if err != nil {
+		return fmt.Errorf("RiakURLSigKey gatherKeys: %w", err)
+	}
+	for _, obj := range objs {
+		key := tc.URLSigKeys{}
+		if err := json.Unmarshal(obj.Value, &key); err != nil {
+			return fmt.Errorf("RiakURLSigKey gatherKeys unable to unamrshal object into tc.URLSigKeys: %w", err)
+		}
+		tbl.Records = append(tbl.Records, riakURLSigKeyRecord{
+			DeliveryService: obj.Key,
+			Key:             key,
+		})
+	}
+	return nil
+}
+func (tbl *riakURLSigKeyTable) toGeneric() []URLSigKey {
+	keys := make([]URLSigKey, len(tbl.Records))
+
+	for i, record := range tbl.Records {
+		keys[i] = URLSigKey{
+			URLSigKeys:      record.Key,
+			DeliveryService: record.DeliveryService,
+		}
+	}
+
+	return keys
+}
+func (tbl *riakURLSigKeyTable) fromGeneric(keys []URLSigKey) {
+	tbl.Records = make([]riakURLSigKeyRecord, len(keys))
+	for i, key := range keys {
+		tbl.Records[i] = riakURLSigKeyRecord{
+			Key:             key.URLSigKeys,
+			DeliveryService: key.DeliveryService,
+		}
+	}
+}
+func (tbl *riakURLSigKeyTable) insertKeys(cluster *riak.Cluster) error {
+	for _, record := range tbl.Records {
+		objBytes, err := json.Marshal(record.Key)
+		if err != nil {
+			return fmt.Errorf("RiakURLSigKey insertKeys unable to marshal keys: %w", err)
+		}
+
+		err = setObject(cluster, makeRiakObject(objBytes, "url_sig_"+record.DeliveryService+".config"), BUCKET_URL_SIG)
+		if err != nil {
+			return fmt.Errorf("RiakURLSigKey insertKeys: %w", err)
+		}
+	}
+	return nil
+}
+func (tbl *riakURLSigKeyTable) validate() []string {
+	errs := []string{}
+	for i, record := range tbl.Records {
+		if record.DeliveryService == "" {
+			errs = append(errs, fmt.Sprintf("URL Key #%v: Delivery Service is blank!", i))
+		}
+	}
+	return errs
+}
+
+type riakURISignKeyRecord struct {
+	Key             map[string]tc.URISignerKeyset
+	DeliveryService string
+}
+type riakURISignKeyTable struct {
+	Records []riakURISignKeyRecord
+}
+
+func (tbl *riakURISignKeyTable) gatherKeys(cluster *riak.Cluster) error {
+	tbl.Records = []riakURISignKeyRecord{}
+	objs, err := getObjects(cluster, BUCKET_URI_SIG)
+	if err != nil {
+		return fmt.Errorf("RiakURISignKey gatherKeys: %w", err)
+	}
+	for _, obj := range objs {
+		key := map[string]tc.URISignerKeyset{}
+		if err := json.Unmarshal(obj.Value, &key); err != nil {
+			return fmt.Errorf("RiakURISignKey gatherKeys unable to unmarshal object into map[string]tc.URISignerKeySet: %w", err)
+		}
+
+		tbl.Records = append(tbl.Records, riakURISignKeyRecord{
+			DeliveryService: obj.Key,
+			Key:             key,
+		})
+	}
+	return nil
+}
+func (tbl *riakURISignKeyTable) toGeneric() []URISignKey {
+	keys := make([]URISignKey, len(tbl.Records))
+
+	for i, record := range tbl.Records {
+		keys[i] = URISignKey{
+			DeliveryService: record.DeliveryService,
+			Keys:            record.Key,
+		}
+	}
+
+	return keys
+}
+func (tbl *riakURISignKeyTable) fromGeneric(keys []URISignKey) {
+	tbl.Records = make([]riakURISignKeyRecord, len(keys))
+
+	for i, record := range keys {
+		tbl.Records[i] = riakURISignKeyRecord{
+			Key:             record.Keys,
+			DeliveryService: record.DeliveryService,
+		}
+	}
+}
+func (tbl *riakURISignKeyTable) insertKeys(cluster *riak.Cluster) error {
+	for _, record := range tbl.Records {
+		objBytes, err := json.Marshal(record.Key)
+		if err != nil {
+			return fmt.Errorf("RiakURISignKey insertKeys: unable to marshal key: %w", err)
+		}
+
+		err = setObject(cluster, makeRiakObject(objBytes, record.DeliveryService), BUCKET_URI_SIG)
+		if err != nil {
+			return fmt.Errorf("RiakURISignKey insertKeys: %w", err)
+		}
+	}
+	return nil
+}
+func (tbl *riakURISignKeyTable) validate() []string {
+	errs := []string{}
+	for i, record := range tbl.Records {
+		if record.DeliveryService == "" {
+			errs = append(errs, fmt.Sprintf("URI Key #%v: Delivery Service is blank!", i))
+		}
+	}
+	return errs
+}
+
+// Riak functions
+func makeRiakObject(data []byte, key string) *riak.Object {
+	return &riak.Object{
+		ContentType:     rfc.ApplicationJSON,
+		Charset:         "utf-8",
+		ContentEncoding: "utf-8",
+		Key:             key,
+		Value:           data,
+	}
+}
+func getObjects(cluster *riak.Cluster, bucket string) ([]*riak.Object, error) {
+	objs := []*riak.Object{}
+	keys, err := listKeys(cluster, bucket)
+	if err != nil {
+		return nil, err
+	}
+	for _, key := range keys {
+		objects, err := getObject(cluster, bucket, key)
+		if err != nil {
+			return nil, err
+		}
+		if len(objects) > 1 {
+			return nil, fmt.Errorf("Unexpected number of objects %v, ignoring\n", len(objects))
+		}
+
+		objs = append(objs, objects[0])
+	}
+
+	return objs, nil
+}
+func getObject(cluster *riak.Cluster, bucket string, key string) ([]*riak.Object, error) {
+	cmd, err := riak.NewFetchValueCommandBuilder().
+		WithBucket(bucket).
+		WithKey(key).
+		WithTimeout(time.Second * 60).
+		Build()
+	if err != nil {
+		return nil, fmt.Errorf("error building riak fetch value command: %w", err)
+	}
+
+	if err := cluster.Execute(cmd); err != nil {
+		return nil, fmt.Errorf("error executing riak fetch value command: %w", err)
+	}
+
+	fvc := cmd.(*riak.FetchValueCommand)
+	rsp := fvc.Response
+
+	if rsp.IsNotFound {
+		return nil, fmt.Errorf("errors executing riak fetch value command; key not found: %v:%v", bucket, key)
+	}
+
+	return rsp.Values, nil
+}
+func setObject(cluster *riak.Cluster, obj *riak.Object, bucket string) error {
+	cmd, err := riak.NewStoreValueCommandBuilder().
+		WithBucket(bucket).
+		WithContent(obj).
+		WithTimeout(time.Second * 5).
+		Build()
+	if err != nil {
+		return fmt.Errorf("error building riak store value command: %w", err)
+	}
+
+	return cluster.Execute(cmd)
+}
+func search(cluster *riak.Cluster, index string, query string, filterQuery string, numRows uint32, fields []string) ([]*riak.SearchDoc, error) {
+	var searchDocs []*riak.SearchDoc
+	start := uint32(0)
+	for i := uint32(0); ; i += 1 {
+		riakCmd := riak.NewSearchCommandBuilder().
+			WithQuery(query).
+			WithNumRows(numRows).
+			WithStart(start)
+		if len(index) > 0 {
+			riakCmd = riakCmd.WithIndexName(index)
+		}
+		if len(filterQuery) > 0 {
+			riakCmd = riakCmd.WithFilterQuery(filterQuery)
+		}
+		if len(fields) > 0 {
+			riakCmd = riakCmd.WithReturnFields(fields...)
+		}
+		iCmd, err := riakCmd.Build()
+
+		if err != nil {
+			return nil, fmt.Errorf("building Riak search command: %w", err)
+		}
+		if err = cluster.Execute(iCmd); err != nil {
+			return nil, fmt.Errorf("executing Riak search command index '%v' query '%v': %w", index, query, err)
+		}
+		cmd, ok := iCmd.(*riak.SearchCommand)
+		if !ok {
+			return nil, fmt.Errorf("riak search command unexpected type %T", iCmd)
+		}
+		if start == 0 {
+			if cmd.Response == nil || cmd.Response.NumFound == 0 {
+				return nil, nil
+			}
+			if cmd.Response.NumFound <= numRows {
+				return cmd.Response.Docs, nil
+			} else if numRows < cmd.Response.NumFound {
+				searchDocs = make([]*riak.SearchDoc, cmd.Response.NumFound)
+			}
+			if cmd.Response.NumFound > 10000 {
+				fmt.Printf("WARNING: found %v rows, press enter to continue", cmd.Response.NumFound)
+				_, _ = fmt.Scanln()
+			}
+		}
+
+		// If the total number of docs is not evenly divisible by 1000
+		if uint32(len(cmd.Response.Docs)) < numRows {
+			numRows = uint32(len(cmd.Response.Docs))
+		}
+
+		for responseIndex := uint32(0); responseIndex < numRows; responseIndex += 1 {
+			returnIndex := responseIndex + start
+			searchDocs[returnIndex] = cmd.Response.Docs[responseIndex]
+		}
+		if cmd.Response.NumFound == numRows+start {
+			return searchDocs, nil
+		}
+		start += numRows
+	}
+}
+func listKeys(cluster *riak.Cluster, bucket string) ([]string, error) {
+	cmd, err := riak.NewListKeysCommandBuilder().
+		WithBucket(bucket).
+		WithTimeout(time.Second * 60).

Review comment:
       Should this timeout be configurable (with a sane default)?

##########
File path: tools/traffic_vault_migrate/riak.go
##########
@@ -0,0 +1,746 @@
+package main
+
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+import (
+	"crypto/tls"
+	"encoding/json"
+	"errors"
+	"fmt"
+	"time"
+
+	"github.com/apache/trafficcontrol/lib/go-rfc"
+	"github.com/apache/trafficcontrol/lib/go-tc"
+
+	"github.com/basho/riak-go-client"
+)
+
+const (
+	BUCKET_SSL     = "ssl"
+	BUCKET_DNSSEC  = "dnssec"
+	BUCKET_URL_SIG = "url_sig_keys"
+	BUCKET_URI_SIG = "cdn_uri_sig_keys"
+
+	INDEX_SSL = "sslkeys"
+
+	SCHEMA_RIAK_KEY    = "_yz_rk"
+	SCHEMA_RIAK_BUCKET = "_yz_rb"
+)
+
+var (
+	SCHEMA_SSL_FIELDS = [...]string{SCHEMA_RIAK_KEY, SCHEMA_RIAK_BUCKET}
+)
+
+// RiakConfig  represents the configuration options available to the Riak backend
+type RiakConfig struct {
+	Host          string `json:"host"`
+	Port          string `json:"port"`
+	User          string `json:"user"`
+	Password      string `json:"password"`
+	Insecure      bool   `json:"insecure"`
+	TLSVersionRaw string `json:"tlsVersion"`
+	TLSVersion    uint16
+}
+
+// RiakBackend is the Riak implementation of TVBackend
+type RiakBackend struct {
+	sslKeys     riakSSLKeyTable
+	dnssecKeys  riakDNSSecKeyTable
+	uriSignKeys riakURISignKeyTable
+	urlSigKeys  riakURLSigKeyTable
+	cfg         RiakConfig
+	cluster     *riak.Cluster
+}
+
+// String returns a high level overview of the backend and its keys
+func (rb *RiakBackend) String() string {
+	data := fmt.Sprintf("Riak server %v@%v:%v\n", rb.cfg.User, rb.cfg.Host, rb.cfg.Port)
+	data += fmt.Sprintf("\tSSL Keys: %v\n", len(rb.sslKeys.Records))
+	data += fmt.Sprintf("\tDNSSec Keys: %v\n", len(rb.dnssecKeys.Records))
+	data += fmt.Sprintf("\tURI Keys: %v\n", len(rb.uriSignKeys.Records))
+	data += fmt.Sprintf("\tURL Keys: %v\n", len(rb.urlSigKeys.Records))
+	return data
+}
+
+// Name returns the name for this backend
+func (rb *RiakBackend) Name() string {
+	return "Riak"
+}
+
+// ReadConfig takes in a filename and will read it into the backends config
+func (rb *RiakBackend) ReadConfig(s string) error {
+	err := UnmarshalConfig(s, &rb.cfg)
+	if err != nil {
+		return err
+	}
+
+	if rb.cfg.TLSVersionRaw == "10" {
+		rb.cfg.TLSVersion = tls.VersionTLS10
+	} else if rb.cfg.TLSVersionRaw == "11" {
+		rb.cfg.TLSVersion = tls.VersionTLS11
+	} else if rb.cfg.TLSVersionRaw == "12" {
+		rb.cfg.TLSVersion = tls.VersionTLS12
+	} else if rb.cfg.TLSVersionRaw == "13" {
+		rb.cfg.TLSVersion = tls.VersionTLS13
+	}
+	return nil
+}
+
+// Insert takes the current keys and inserts them into the backend DB
+func (rb *RiakBackend) Insert() error {
+	if err := rb.sslKeys.insertKeys(rb.cluster); err != nil {
+		return err
+	}
+	if err := rb.dnssecKeys.insertKeys(rb.cluster); err != nil {
+		return err
+	}
+	if err := rb.urlSigKeys.insertKeys(rb.cluster); err != nil {
+		return err
+	}
+	if err := rb.uriSignKeys.insertKeys(rb.cluster); err != nil {
+		return err
+	}
+	return nil
+}
+
+// ValidateKey validates that the keys are valid (in most cases, certain fields are not null)
+func (rb *RiakBackend) ValidateKey() []string {
+	errors := []string{}
+	if errs := rb.sslKeys.validate(); errs != nil {
+		errors = append(errors, errs...)
+	}
+	if errs := rb.dnssecKeys.validate(); errs != nil {
+		errors = append(errors, errs...)
+	}
+	if errs := rb.uriSignKeys.validate(); errs != nil {
+		errors = append(errors, errs...)
+	}
+	if errs := rb.urlSigKeys.validate(); errs != nil {
+		errors = append(errors, errs...)
+	}
+
+	return errors
+}
+
+// SetSSLKeys takes in keys and converts & encrypts the data into the backends internal format
+func (rb *RiakBackend) SetSSLKeys(keys []SSLKey) error {
+	rb.sslKeys.fromGeneric(keys)
+	return nil
+}
+
+// SetDNSSecKeys takes in keys and converts & encrypts the data into the backends internal format
+func (rb *RiakBackend) SetDNSSecKeys(keys []DNSSecKey) error {
+	rb.dnssecKeys.fromGeneric(keys)
+	return nil
+}
+
+// SetURISignKeys takes in keys and converts & encrypts the data into the backends internal format
+func (rb *RiakBackend) SetURISignKeys(keys []URISignKey) error {
+	rb.uriSignKeys.fromGeneric(keys)
+	return nil
+}
+
+// SetURLSigKeys takes in keys and converts & encrypts the data into the backends internal format
+func (rb *RiakBackend) SetURLSigKeys(keys []URLSigKey) error {
+	rb.urlSigKeys.fromGeneric(keys)
+	return nil
+}
+
+// Start initiates the connection to the backend DB
+func (rb *RiakBackend) Start() error {
+	tlsConfig := &tls.Config{
+		InsecureSkipVerify: !rb.cfg.Insecure,
+		MaxVersion:         rb.cfg.TLSVersion,
+	}
+	auth := &riak.AuthOptions{
+		User:      rb.cfg.User,
+		Password:  rb.cfg.Password,
+		TlsConfig: tlsConfig,
+	}
+
+	cluster, err := getRiakCluster(rb.cfg, auth)
+	if err != nil {
+		return err
+	}
+	if err := cluster.Start(); err != nil {
+		return fmt.Errorf("unable to start riak cluster: %w", err)
+	}
+
+	rb.cluster = cluster
+	rb.sslKeys = riakSSLKeyTable{}
+	rb.dnssecKeys = riakDNSSecKeyTable{}
+	rb.urlSigKeys = riakURLSigKeyTable{}
+	rb.uriSignKeys = riakURISignKeyTable{}
+	return nil
+}
+
+// Close terminates the connection to the backend DB
+func (rb *RiakBackend) Close() error {
+	if err := rb.cluster.Stop(); err != nil {
+		return err
+	}
+	return nil
+}
+
+// Ping checks the connection to the backend DB
+func (rb *RiakBackend) Ping() error {
+	return ping(rb.cluster)
+}
+
+// GetSSLKeys converts the backends internal key representation into the common representation (SSLKey)
+func (rb *RiakBackend) GetSSLKeys() ([]SSLKey, error) {
+	return rb.sslKeys.toGeneric(), nil
+}
+
+// GetDNSSecKeys converts the backends internal key representation into the common representation (DNSSecKey)
+func (rb *RiakBackend) GetDNSSecKeys() ([]DNSSecKey, error) {
+	return rb.dnssecKeys.toGeneric(), nil
+}
+
+// GetURISignKeys converts the pg internal key representation into the common representation (URISignKey)
+func (rb *RiakBackend) GetURISignKeys() ([]URISignKey, error) {
+	return rb.uriSignKeys.toGeneric(), nil
+}
+
+// GetURLSigKeys converts the backends internal key representation into the common representation (URLSigKey)
+func (rb *RiakBackend) GetURLSigKeys() ([]URLSigKey, error) {
+	return rb.urlSigKeys.toGeneric(), nil
+}
+
+// Fetch gets all of the keys from the backend DB
+func (rb *RiakBackend) Fetch() error {
+	if err := rb.sslKeys.gatherKeys(rb.cluster); err != nil {
+		return err
+	}
+	if err := rb.dnssecKeys.gatherKeys(rb.cluster); err != nil {
+		return err
+	}
+	if err := rb.urlSigKeys.gatherKeys(rb.cluster); err != nil {
+		return err
+	}
+	if err := rb.uriSignKeys.gatherKeys(rb.cluster); err != nil {
+		return err
+	}
+
+	return nil
+}
+
+type riakSSLKeyRecord struct {
+	tc.DeliveryServiceSSLKeys
+}
+type riakSSLKeyTable struct {
+	Records []riakSSLKeyRecord
+}
+
+func (tbl *riakSSLKeyTable) gatherKeys(cluster *riak.Cluster) error {
+	searchDocs, err := search(cluster, INDEX_SSL, "cdn:*", fmt.Sprintf("%v:*latest", SCHEMA_RIAK_KEY), 1000, SCHEMA_SSL_FIELDS[:])
+	if err != nil {
+		return fmt.Errorf("RiakSSLKey gatherKeys: %w", err)
+	}
+
+	tbl.Records = make([]riakSSLKeyRecord, len(searchDocs))
+	for i, doc := range searchDocs {
+		objs, err := getObject(cluster, doc.Bucket, doc.Key)
+		if err != nil {
+			return err
+		}
+		if len(objs) < 1 {
+			return fmt.Errorf("RiakSSLKey gatherKeys unable to find any objects with key %v and bucket %v, but search results were returned", doc.Key, doc.Bucket)
+		}
+		if len(objs) > 1 {
+			return fmt.Errorf("RiakSSLKey gatherKeys more than 1 ssl key record found %v\n", len(objs))
+		}
+		var obj tc.DeliveryServiceSSLKeys
+		if err = json.Unmarshal(objs[0].Value, &obj); err != nil {
+			return fmt.Errorf("RiakSSLKey gatherKeys unable to unmarshal object into tc.DeliveryServiceSSLKeys: %w", err)
+		}
+		tbl.Records[i] = riakSSLKeyRecord{
+			DeliveryServiceSSLKeys: obj,
+		}
+	}
+	return nil
+}
+func (tbl *riakSSLKeyTable) toGeneric() []SSLKey {
+	keys := make([]SSLKey, len(tbl.Records))
+
+	for i, record := range tbl.Records {
+		keys[i] = SSLKey{
+			DeliveryServiceSSLKeys: record.DeliveryServiceSSLKeys,
+		}
+	}
+
+	return keys
+}
+func (tbl *riakSSLKeyTable) fromGeneric(keys []SSLKey) {
+	tbl.Records = make([]riakSSLKeyRecord, len(keys))
+
+	for i, record := range keys {
+		tbl.Records[i] = riakSSLKeyRecord{
+			DeliveryServiceSSLKeys: record.DeliveryServiceSSLKeys,
+		}
+	}
+}
+func (tbl *riakSSLKeyTable) insertKeys(cluster *riak.Cluster) error {
+	for _, record := range tbl.Records {
+		objBytes, err := json.Marshal(record.DeliveryServiceSSLKeys)
+		if err != nil {
+			return fmt.Errorf("RiakSSLKey insertKeys failed to unmarshal keys: %w", err)
+		}
+		err = setObject(cluster, makeRiakObject(objBytes, record.DeliveryService+"-"+record.Version.String()), BUCKET_SSL)
+		if err != nil {
+			return fmt.Errorf("RiakSSLKey insertKeys: %w", err)
+		}
+		err = setObject(cluster, makeRiakObject(objBytes, record.DeliveryService+"-latest"), BUCKET_SSL)
+		if err != nil {
+			return fmt.Errorf("RiakSSLKey insertKeys: %w", err)
+		}
+	}
+	return nil
+}
+func (tbl *riakSSLKeyTable) validate() []string {
+	errs := []string{}
+	for i, record := range tbl.Records {
+		if record.DeliveryService == "" {
+			errs = append(errs, fmt.Sprintf("SSL Key #%v: Delivery Service is blank!", i))
+		}
+		if record.CDN == "" {
+			errs = append(errs, fmt.Sprintf("SSL Key #%v: CDN is blank!", i))
+		}
+		if record.Version.String() == "" {
+			errs = append(errs, fmt.Sprintf("SSL Key #%v: Version is blank!", i))
+		}
+	}
+	return errs
+}
+
+type riakDNSSecKeyRecord struct {
+	CDN string
+	Key tc.DNSSECKeysRiak
+}
+type riakDNSSecKeyTable struct {
+	Records []riakDNSSecKeyRecord
+}
+
+func (tbl *riakDNSSecKeyTable) gatherKeys(cluster *riak.Cluster) error {
+	tbl.Records = []riakDNSSecKeyRecord{}
+	objs, err := getObjects(cluster, BUCKET_DNSSEC)
+	if err != nil {
+		return fmt.Errorf("RiakDNSSecKey gatherKeys: %w", err)
+	}
+	for _, obj := range objs {
+		key := tc.DNSSECKeysRiak{}
+		if err := json.Unmarshal(obj.Value, &key); err != nil {
+			return fmt.Errorf("RiakDNSSecKey gatherKeys unable to unmarshal object to tc.DNSSECKeysRiak: %w", err)
+		}
+		tbl.Records = append(tbl.Records, riakDNSSecKeyRecord{
+			CDN: obj.Key,
+			Key: key,
+		})
+	}
+	return nil
+}
+func (tbl *riakDNSSecKeyTable) toGeneric() []DNSSecKey {
+	keys := make([]DNSSecKey, len(tbl.Records))
+
+	for i, record := range tbl.Records {
+		keys[i] = DNSSecKey{
+			CDN:                    record.CDN,
+			DNSSECKeysTrafficVault: tc.DNSSECKeysTrafficVault(record.Key),
+		}
+	}
+
+	return keys
+}
+func (tbl *riakDNSSecKeyTable) fromGeneric(keys []DNSSecKey) {
+	tbl.Records = make([]riakDNSSecKeyRecord, len(keys))
+
+	for i, record := range keys {
+		tbl.Records[i] = riakDNSSecKeyRecord{
+			CDN: record.CDN,
+			Key: tc.DNSSECKeysRiak(record.DNSSECKeysTrafficVault),
+		}
+	}
+}
+func (tbl *riakDNSSecKeyTable) insertKeys(cluster *riak.Cluster) error {
+	for _, record := range tbl.Records {
+		objBytes, err := json.Marshal(record.Key)
+		if err != nil {
+			return fmt.Errorf("RiakDNSSecKey insertKeys error marshalling keys: %w", err)
+		}
+
+		err = setObject(cluster, makeRiakObject(objBytes, record.CDN), BUCKET_DNSSEC)
+		if err != nil {
+			return fmt.Errorf("RiakDNSSecKey insertKeys: %w", err)
+		}
+	}
+	return nil
+}
+func (tbl *riakDNSSecKeyTable) validate() []string {
+	errs := []string{}
+	for i, record := range tbl.Records {
+		if record.CDN == "" {
+			errs = append(errs, fmt.Sprintf("DNSSec Key #%v: CDN is blank!", i))
+		}
+	}
+	return errs
+}
+
+type riakURLSigKeyRecord struct {
+	Key             tc.URLSigKeys
+	DeliveryService string
+}
+type riakURLSigKeyTable struct {
+	Records []riakURLSigKeyRecord
+}
+
+func (tbl *riakURLSigKeyTable) gatherKeys(cluster *riak.Cluster) error {
+	tbl.Records = []riakURLSigKeyRecord{}
+	objs, err := getObjects(cluster, BUCKET_URL_SIG)
+	if err != nil {
+		return fmt.Errorf("RiakURLSigKey gatherKeys: %w", err)
+	}
+	for _, obj := range objs {
+		key := tc.URLSigKeys{}
+		if err := json.Unmarshal(obj.Value, &key); err != nil {
+			return fmt.Errorf("RiakURLSigKey gatherKeys unable to unamrshal object into tc.URLSigKeys: %w", err)
+		}
+		tbl.Records = append(tbl.Records, riakURLSigKeyRecord{
+			DeliveryService: obj.Key,
+			Key:             key,
+		})
+	}
+	return nil
+}
+func (tbl *riakURLSigKeyTable) toGeneric() []URLSigKey {
+	keys := make([]URLSigKey, len(tbl.Records))
+
+	for i, record := range tbl.Records {
+		keys[i] = URLSigKey{
+			URLSigKeys:      record.Key,
+			DeliveryService: record.DeliveryService,
+		}
+	}
+
+	return keys
+}
+func (tbl *riakURLSigKeyTable) fromGeneric(keys []URLSigKey) {
+	tbl.Records = make([]riakURLSigKeyRecord, len(keys))
+	for i, key := range keys {
+		tbl.Records[i] = riakURLSigKeyRecord{
+			Key:             key.URLSigKeys,
+			DeliveryService: key.DeliveryService,
+		}
+	}
+}
+func (tbl *riakURLSigKeyTable) insertKeys(cluster *riak.Cluster) error {
+	for _, record := range tbl.Records {
+		objBytes, err := json.Marshal(record.Key)
+		if err != nil {
+			return fmt.Errorf("RiakURLSigKey insertKeys unable to marshal keys: %w", err)
+		}
+
+		err = setObject(cluster, makeRiakObject(objBytes, "url_sig_"+record.DeliveryService+".config"), BUCKET_URL_SIG)
+		if err != nil {
+			return fmt.Errorf("RiakURLSigKey insertKeys: %w", err)
+		}
+	}
+	return nil
+}
+func (tbl *riakURLSigKeyTable) validate() []string {
+	errs := []string{}
+	for i, record := range tbl.Records {
+		if record.DeliveryService == "" {
+			errs = append(errs, fmt.Sprintf("URL Key #%v: Delivery Service is blank!", i))
+		}
+	}
+	return errs
+}
+
+type riakURISignKeyRecord struct {
+	Key             map[string]tc.URISignerKeyset
+	DeliveryService string
+}
+type riakURISignKeyTable struct {
+	Records []riakURISignKeyRecord
+}
+
+func (tbl *riakURISignKeyTable) gatherKeys(cluster *riak.Cluster) error {
+	tbl.Records = []riakURISignKeyRecord{}
+	objs, err := getObjects(cluster, BUCKET_URI_SIG)
+	if err != nil {
+		return fmt.Errorf("RiakURISignKey gatherKeys: %w", err)
+	}
+	for _, obj := range objs {
+		key := map[string]tc.URISignerKeyset{}
+		if err := json.Unmarshal(obj.Value, &key); err != nil {
+			return fmt.Errorf("RiakURISignKey gatherKeys unable to unmarshal object into map[string]tc.URISignerKeySet: %w", err)
+		}
+
+		tbl.Records = append(tbl.Records, riakURISignKeyRecord{
+			DeliveryService: obj.Key,
+			Key:             key,
+		})
+	}
+	return nil
+}
+func (tbl *riakURISignKeyTable) toGeneric() []URISignKey {
+	keys := make([]URISignKey, len(tbl.Records))
+
+	for i, record := range tbl.Records {
+		keys[i] = URISignKey{
+			DeliveryService: record.DeliveryService,
+			Keys:            record.Key,
+		}
+	}
+
+	return keys
+}
+func (tbl *riakURISignKeyTable) fromGeneric(keys []URISignKey) {
+	tbl.Records = make([]riakURISignKeyRecord, len(keys))
+
+	for i, record := range keys {
+		tbl.Records[i] = riakURISignKeyRecord{
+			Key:             record.Keys,
+			DeliveryService: record.DeliveryService,
+		}
+	}
+}
+func (tbl *riakURISignKeyTable) insertKeys(cluster *riak.Cluster) error {
+	for _, record := range tbl.Records {
+		objBytes, err := json.Marshal(record.Key)
+		if err != nil {
+			return fmt.Errorf("RiakURISignKey insertKeys: unable to marshal key: %w", err)
+		}
+
+		err = setObject(cluster, makeRiakObject(objBytes, record.DeliveryService), BUCKET_URI_SIG)
+		if err != nil {
+			return fmt.Errorf("RiakURISignKey insertKeys: %w", err)
+		}
+	}
+	return nil
+}
+func (tbl *riakURISignKeyTable) validate() []string {
+	errs := []string{}
+	for i, record := range tbl.Records {
+		if record.DeliveryService == "" {
+			errs = append(errs, fmt.Sprintf("URI Key #%v: Delivery Service is blank!", i))
+		}
+	}
+	return errs
+}
+
+// Riak functions
+func makeRiakObject(data []byte, key string) *riak.Object {
+	return &riak.Object{
+		ContentType:     rfc.ApplicationJSON,
+		Charset:         "utf-8",
+		ContentEncoding: "utf-8",
+		Key:             key,
+		Value:           data,
+	}
+}
+func getObjects(cluster *riak.Cluster, bucket string) ([]*riak.Object, error) {
+	objs := []*riak.Object{}
+	keys, err := listKeys(cluster, bucket)
+	if err != nil {
+		return nil, err
+	}
+	for _, key := range keys {
+		objects, err := getObject(cluster, bucket, key)
+		if err != nil {
+			return nil, err
+		}
+		if len(objects) > 1 {
+			return nil, fmt.Errorf("Unexpected number of objects %v, ignoring\n", len(objects))
+		}
+
+		objs = append(objs, objects[0])
+	}
+
+	return objs, nil
+}
+func getObject(cluster *riak.Cluster, bucket string, key string) ([]*riak.Object, error) {
+	cmd, err := riak.NewFetchValueCommandBuilder().
+		WithBucket(bucket).
+		WithKey(key).
+		WithTimeout(time.Second * 60).
+		Build()
+	if err != nil {
+		return nil, fmt.Errorf("error building riak fetch value command: %w", err)
+	}
+
+	if err := cluster.Execute(cmd); err != nil {
+		return nil, fmt.Errorf("error executing riak fetch value command: %w", err)
+	}
+
+	fvc := cmd.(*riak.FetchValueCommand)
+	rsp := fvc.Response
+
+	if rsp.IsNotFound {
+		return nil, fmt.Errorf("errors executing riak fetch value command; key not found: %v:%v", bucket, key)
+	}
+
+	return rsp.Values, nil
+}
+func setObject(cluster *riak.Cluster, obj *riak.Object, bucket string) error {
+	cmd, err := riak.NewStoreValueCommandBuilder().
+		WithBucket(bucket).
+		WithContent(obj).
+		WithTimeout(time.Second * 5).
+		Build()
+	if err != nil {
+		return fmt.Errorf("error building riak store value command: %w", err)
+	}
+
+	return cluster.Execute(cmd)
+}
+func search(cluster *riak.Cluster, index string, query string, filterQuery string, numRows uint32, fields []string) ([]*riak.SearchDoc, error) {
+	var searchDocs []*riak.SearchDoc
+	start := uint32(0)
+	for i := uint32(0); ; i += 1 {
+		riakCmd := riak.NewSearchCommandBuilder().
+			WithQuery(query).
+			WithNumRows(numRows).
+			WithStart(start)
+		if len(index) > 0 {
+			riakCmd = riakCmd.WithIndexName(index)
+		}
+		if len(filterQuery) > 0 {
+			riakCmd = riakCmd.WithFilterQuery(filterQuery)
+		}
+		if len(fields) > 0 {
+			riakCmd = riakCmd.WithReturnFields(fields...)
+		}
+		iCmd, err := riakCmd.Build()
+
+		if err != nil {
+			return nil, fmt.Errorf("building Riak search command: %w", err)
+		}
+		if err = cluster.Execute(iCmd); err != nil {
+			return nil, fmt.Errorf("executing Riak search command index '%v' query '%v': %w", index, query, err)
+		}
+		cmd, ok := iCmd.(*riak.SearchCommand)
+		if !ok {
+			return nil, fmt.Errorf("riak search command unexpected type %T", iCmd)
+		}
+		if start == 0 {
+			if cmd.Response == nil || cmd.Response.NumFound == 0 {
+				return nil, nil
+			}
+			if cmd.Response.NumFound <= numRows {
+				return cmd.Response.Docs, nil
+			} else if numRows < cmd.Response.NumFound {
+				searchDocs = make([]*riak.SearchDoc, cmd.Response.NumFound)
+			}
+			if cmd.Response.NumFound > 10000 {

Review comment:
       Should this be based on `numRows` instead of hard-coded to 10000?

##########
File path: tools/traffic_vault_migrate/riak.go
##########
@@ -0,0 +1,746 @@
+package main
+
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+import (
+	"crypto/tls"
+	"encoding/json"
+	"errors"
+	"fmt"
+	"time"
+
+	"github.com/apache/trafficcontrol/lib/go-rfc"
+	"github.com/apache/trafficcontrol/lib/go-tc"
+
+	"github.com/basho/riak-go-client"
+)
+
+const (
+	BUCKET_SSL     = "ssl"
+	BUCKET_DNSSEC  = "dnssec"
+	BUCKET_URL_SIG = "url_sig_keys"
+	BUCKET_URI_SIG = "cdn_uri_sig_keys"
+
+	INDEX_SSL = "sslkeys"
+
+	SCHEMA_RIAK_KEY    = "_yz_rk"
+	SCHEMA_RIAK_BUCKET = "_yz_rb"
+)
+
+var (
+	SCHEMA_SSL_FIELDS = [...]string{SCHEMA_RIAK_KEY, SCHEMA_RIAK_BUCKET}
+)
+
+// RiakConfig  represents the configuration options available to the Riak backend
+type RiakConfig struct {
+	Host          string `json:"host"`
+	Port          string `json:"port"`
+	User          string `json:"user"`
+	Password      string `json:"password"`
+	Insecure      bool   `json:"insecure"`
+	TLSVersionRaw string `json:"tlsVersion"`
+	TLSVersion    uint16
+}
+
+// RiakBackend is the Riak implementation of TVBackend
+type RiakBackend struct {
+	sslKeys     riakSSLKeyTable
+	dnssecKeys  riakDNSSecKeyTable
+	uriSignKeys riakURISignKeyTable
+	urlSigKeys  riakURLSigKeyTable
+	cfg         RiakConfig
+	cluster     *riak.Cluster
+}
+
+// String returns a high level overview of the backend and its keys
+func (rb *RiakBackend) String() string {
+	data := fmt.Sprintf("Riak server %v@%v:%v\n", rb.cfg.User, rb.cfg.Host, rb.cfg.Port)
+	data += fmt.Sprintf("\tSSL Keys: %v\n", len(rb.sslKeys.Records))
+	data += fmt.Sprintf("\tDNSSec Keys: %v\n", len(rb.dnssecKeys.Records))
+	data += fmt.Sprintf("\tURI Keys: %v\n", len(rb.uriSignKeys.Records))
+	data += fmt.Sprintf("\tURL Keys: %v\n", len(rb.urlSigKeys.Records))
+	return data
+}
+
+// Name returns the name for this backend
+func (rb *RiakBackend) Name() string {
+	return "Riak"
+}
+
+// ReadConfig takes in a filename and will read it into the backends config
+func (rb *RiakBackend) ReadConfig(s string) error {
+	err := UnmarshalConfig(s, &rb.cfg)
+	if err != nil {
+		return err
+	}
+
+	if rb.cfg.TLSVersionRaw == "10" {
+		rb.cfg.TLSVersion = tls.VersionTLS10
+	} else if rb.cfg.TLSVersionRaw == "11" {
+		rb.cfg.TLSVersion = tls.VersionTLS11
+	} else if rb.cfg.TLSVersionRaw == "12" {
+		rb.cfg.TLSVersion = tls.VersionTLS12
+	} else if rb.cfg.TLSVersionRaw == "13" {
+		rb.cfg.TLSVersion = tls.VersionTLS13
+	}
+	return nil
+}
+
+// Insert takes the current keys and inserts them into the backend DB
+func (rb *RiakBackend) Insert() error {
+	if err := rb.sslKeys.insertKeys(rb.cluster); err != nil {
+		return err
+	}
+	if err := rb.dnssecKeys.insertKeys(rb.cluster); err != nil {
+		return err
+	}
+	if err := rb.urlSigKeys.insertKeys(rb.cluster); err != nil {
+		return err
+	}
+	if err := rb.uriSignKeys.insertKeys(rb.cluster); err != nil {
+		return err
+	}
+	return nil
+}
+
+// ValidateKey validates that the keys are valid (in most cases, certain fields are not null)
+func (rb *RiakBackend) ValidateKey() []string {
+	errors := []string{}
+	if errs := rb.sslKeys.validate(); errs != nil {
+		errors = append(errors, errs...)
+	}
+	if errs := rb.dnssecKeys.validate(); errs != nil {
+		errors = append(errors, errs...)
+	}
+	if errs := rb.uriSignKeys.validate(); errs != nil {
+		errors = append(errors, errs...)
+	}
+	if errs := rb.urlSigKeys.validate(); errs != nil {
+		errors = append(errors, errs...)
+	}
+
+	return errors
+}
+
+// SetSSLKeys takes in keys and converts & encrypts the data into the backends internal format
+func (rb *RiakBackend) SetSSLKeys(keys []SSLKey) error {
+	rb.sslKeys.fromGeneric(keys)
+	return nil
+}
+
+// SetDNSSecKeys takes in keys and converts & encrypts the data into the backends internal format
+func (rb *RiakBackend) SetDNSSecKeys(keys []DNSSecKey) error {
+	rb.dnssecKeys.fromGeneric(keys)
+	return nil
+}
+
+// SetURISignKeys takes in keys and converts & encrypts the data into the backends internal format
+func (rb *RiakBackend) SetURISignKeys(keys []URISignKey) error {
+	rb.uriSignKeys.fromGeneric(keys)
+	return nil
+}
+
+// SetURLSigKeys takes in keys and converts & encrypts the data into the backends internal format
+func (rb *RiakBackend) SetURLSigKeys(keys []URLSigKey) error {
+	rb.urlSigKeys.fromGeneric(keys)
+	return nil
+}
+
+// Start initiates the connection to the backend DB
+func (rb *RiakBackend) Start() error {
+	tlsConfig := &tls.Config{
+		InsecureSkipVerify: !rb.cfg.Insecure,
+		MaxVersion:         rb.cfg.TLSVersion,
+	}
+	auth := &riak.AuthOptions{
+		User:      rb.cfg.User,
+		Password:  rb.cfg.Password,
+		TlsConfig: tlsConfig,
+	}
+
+	cluster, err := getRiakCluster(rb.cfg, auth)
+	if err != nil {
+		return err
+	}
+	if err := cluster.Start(); err != nil {
+		return fmt.Errorf("unable to start riak cluster: %w", err)
+	}
+
+	rb.cluster = cluster
+	rb.sslKeys = riakSSLKeyTable{}
+	rb.dnssecKeys = riakDNSSecKeyTable{}
+	rb.urlSigKeys = riakURLSigKeyTable{}
+	rb.uriSignKeys = riakURISignKeyTable{}
+	return nil
+}
+
+// Close terminates the connection to the backend DB
+func (rb *RiakBackend) Close() error {
+	if err := rb.cluster.Stop(); err != nil {
+		return err
+	}
+	return nil
+}
+
+// Ping checks the connection to the backend DB
+func (rb *RiakBackend) Ping() error {
+	return ping(rb.cluster)
+}
+
+// GetSSLKeys converts the backends internal key representation into the common representation (SSLKey)
+func (rb *RiakBackend) GetSSLKeys() ([]SSLKey, error) {
+	return rb.sslKeys.toGeneric(), nil
+}
+
+// GetDNSSecKeys converts the backends internal key representation into the common representation (DNSSecKey)
+func (rb *RiakBackend) GetDNSSecKeys() ([]DNSSecKey, error) {
+	return rb.dnssecKeys.toGeneric(), nil
+}
+
+// GetURISignKeys converts the pg internal key representation into the common representation (URISignKey)
+func (rb *RiakBackend) GetURISignKeys() ([]URISignKey, error) {
+	return rb.uriSignKeys.toGeneric(), nil
+}
+
+// GetURLSigKeys converts the backends internal key representation into the common representation (URLSigKey)
+func (rb *RiakBackend) GetURLSigKeys() ([]URLSigKey, error) {
+	return rb.urlSigKeys.toGeneric(), nil
+}
+
+// Fetch gets all of the keys from the backend DB
+func (rb *RiakBackend) Fetch() error {
+	if err := rb.sslKeys.gatherKeys(rb.cluster); err != nil {
+		return err
+	}
+	if err := rb.dnssecKeys.gatherKeys(rb.cluster); err != nil {
+		return err
+	}
+	if err := rb.urlSigKeys.gatherKeys(rb.cluster); err != nil {
+		return err
+	}
+	if err := rb.uriSignKeys.gatherKeys(rb.cluster); err != nil {
+		return err
+	}
+
+	return nil
+}
+
+type riakSSLKeyRecord struct {
+	tc.DeliveryServiceSSLKeys
+}
+type riakSSLKeyTable struct {
+	Records []riakSSLKeyRecord
+}
+
+func (tbl *riakSSLKeyTable) gatherKeys(cluster *riak.Cluster) error {
+	searchDocs, err := search(cluster, INDEX_SSL, "cdn:*", fmt.Sprintf("%v:*latest", SCHEMA_RIAK_KEY), 1000, SCHEMA_SSL_FIELDS[:])
+	if err != nil {
+		return fmt.Errorf("RiakSSLKey gatherKeys: %w", err)
+	}
+
+	tbl.Records = make([]riakSSLKeyRecord, len(searchDocs))
+	for i, doc := range searchDocs {
+		objs, err := getObject(cluster, doc.Bucket, doc.Key)
+		if err != nil {
+			return err
+		}
+		if len(objs) < 1 {
+			return fmt.Errorf("RiakSSLKey gatherKeys unable to find any objects with key %v and bucket %v, but search results were returned", doc.Key, doc.Bucket)
+		}
+		if len(objs) > 1 {
+			return fmt.Errorf("RiakSSLKey gatherKeys more than 1 ssl key record found %v\n", len(objs))
+		}
+		var obj tc.DeliveryServiceSSLKeys
+		if err = json.Unmarshal(objs[0].Value, &obj); err != nil {
+			return fmt.Errorf("RiakSSLKey gatherKeys unable to unmarshal object into tc.DeliveryServiceSSLKeys: %w", err)
+		}
+		tbl.Records[i] = riakSSLKeyRecord{
+			DeliveryServiceSSLKeys: obj,
+		}
+	}
+	return nil
+}
+func (tbl *riakSSLKeyTable) toGeneric() []SSLKey {
+	keys := make([]SSLKey, len(tbl.Records))
+
+	for i, record := range tbl.Records {
+		keys[i] = SSLKey{
+			DeliveryServiceSSLKeys: record.DeliveryServiceSSLKeys,
+		}
+	}
+
+	return keys
+}
+func (tbl *riakSSLKeyTable) fromGeneric(keys []SSLKey) {
+	tbl.Records = make([]riakSSLKeyRecord, len(keys))
+
+	for i, record := range keys {
+		tbl.Records[i] = riakSSLKeyRecord{
+			DeliveryServiceSSLKeys: record.DeliveryServiceSSLKeys,
+		}
+	}
+}
+func (tbl *riakSSLKeyTable) insertKeys(cluster *riak.Cluster) error {
+	for _, record := range tbl.Records {
+		objBytes, err := json.Marshal(record.DeliveryServiceSSLKeys)
+		if err != nil {
+			return fmt.Errorf("RiakSSLKey insertKeys failed to unmarshal keys: %w", err)
+		}
+		err = setObject(cluster, makeRiakObject(objBytes, record.DeliveryService+"-"+record.Version.String()), BUCKET_SSL)
+		if err != nil {
+			return fmt.Errorf("RiakSSLKey insertKeys: %w", err)
+		}
+		err = setObject(cluster, makeRiakObject(objBytes, record.DeliveryService+"-latest"), BUCKET_SSL)
+		if err != nil {
+			return fmt.Errorf("RiakSSLKey insertKeys: %w", err)
+		}
+	}
+	return nil
+}
+func (tbl *riakSSLKeyTable) validate() []string {
+	errs := []string{}
+	for i, record := range tbl.Records {
+		if record.DeliveryService == "" {
+			errs = append(errs, fmt.Sprintf("SSL Key #%v: Delivery Service is blank!", i))
+		}
+		if record.CDN == "" {
+			errs = append(errs, fmt.Sprintf("SSL Key #%v: CDN is blank!", i))
+		}
+		if record.Version.String() == "" {
+			errs = append(errs, fmt.Sprintf("SSL Key #%v: Version is blank!", i))
+		}
+	}
+	return errs
+}
+
+type riakDNSSecKeyRecord struct {
+	CDN string
+	Key tc.DNSSECKeysRiak
+}
+type riakDNSSecKeyTable struct {
+	Records []riakDNSSecKeyRecord
+}
+
+func (tbl *riakDNSSecKeyTable) gatherKeys(cluster *riak.Cluster) error {
+	tbl.Records = []riakDNSSecKeyRecord{}
+	objs, err := getObjects(cluster, BUCKET_DNSSEC)
+	if err != nil {
+		return fmt.Errorf("RiakDNSSecKey gatherKeys: %w", err)
+	}
+	for _, obj := range objs {
+		key := tc.DNSSECKeysRiak{}
+		if err := json.Unmarshal(obj.Value, &key); err != nil {
+			return fmt.Errorf("RiakDNSSecKey gatherKeys unable to unmarshal object to tc.DNSSECKeysRiak: %w", err)
+		}
+		tbl.Records = append(tbl.Records, riakDNSSecKeyRecord{
+			CDN: obj.Key,
+			Key: key,
+		})
+	}
+	return nil
+}
+func (tbl *riakDNSSecKeyTable) toGeneric() []DNSSecKey {
+	keys := make([]DNSSecKey, len(tbl.Records))
+
+	for i, record := range tbl.Records {
+		keys[i] = DNSSecKey{
+			CDN:                    record.CDN,
+			DNSSECKeysTrafficVault: tc.DNSSECKeysTrafficVault(record.Key),
+		}
+	}
+
+	return keys
+}
+func (tbl *riakDNSSecKeyTable) fromGeneric(keys []DNSSecKey) {
+	tbl.Records = make([]riakDNSSecKeyRecord, len(keys))
+
+	for i, record := range keys {
+		tbl.Records[i] = riakDNSSecKeyRecord{
+			CDN: record.CDN,
+			Key: tc.DNSSECKeysRiak(record.DNSSECKeysTrafficVault),
+		}
+	}
+}
+func (tbl *riakDNSSecKeyTable) insertKeys(cluster *riak.Cluster) error {
+	for _, record := range tbl.Records {
+		objBytes, err := json.Marshal(record.Key)
+		if err != nil {
+			return fmt.Errorf("RiakDNSSecKey insertKeys error marshalling keys: %w", err)
+		}
+
+		err = setObject(cluster, makeRiakObject(objBytes, record.CDN), BUCKET_DNSSEC)
+		if err != nil {
+			return fmt.Errorf("RiakDNSSecKey insertKeys: %w", err)
+		}
+	}
+	return nil
+}
+func (tbl *riakDNSSecKeyTable) validate() []string {
+	errs := []string{}
+	for i, record := range tbl.Records {
+		if record.CDN == "" {
+			errs = append(errs, fmt.Sprintf("DNSSec Key #%v: CDN is blank!", i))
+		}
+	}
+	return errs
+}
+
+type riakURLSigKeyRecord struct {
+	Key             tc.URLSigKeys
+	DeliveryService string
+}
+type riakURLSigKeyTable struct {
+	Records []riakURLSigKeyRecord
+}
+
+func (tbl *riakURLSigKeyTable) gatherKeys(cluster *riak.Cluster) error {
+	tbl.Records = []riakURLSigKeyRecord{}
+	objs, err := getObjects(cluster, BUCKET_URL_SIG)
+	if err != nil {
+		return fmt.Errorf("RiakURLSigKey gatherKeys: %w", err)
+	}
+	for _, obj := range objs {
+		key := tc.URLSigKeys{}
+		if err := json.Unmarshal(obj.Value, &key); err != nil {
+			return fmt.Errorf("RiakURLSigKey gatherKeys unable to unamrshal object into tc.URLSigKeys: %w", err)
+		}
+		tbl.Records = append(tbl.Records, riakURLSigKeyRecord{
+			DeliveryService: obj.Key,
+			Key:             key,
+		})
+	}
+	return nil
+}
+func (tbl *riakURLSigKeyTable) toGeneric() []URLSigKey {
+	keys := make([]URLSigKey, len(tbl.Records))
+
+	for i, record := range tbl.Records {
+		keys[i] = URLSigKey{
+			URLSigKeys:      record.Key,
+			DeliveryService: record.DeliveryService,
+		}
+	}
+
+	return keys
+}
+func (tbl *riakURLSigKeyTable) fromGeneric(keys []URLSigKey) {
+	tbl.Records = make([]riakURLSigKeyRecord, len(keys))
+	for i, key := range keys {
+		tbl.Records[i] = riakURLSigKeyRecord{
+			Key:             key.URLSigKeys,
+			DeliveryService: key.DeliveryService,
+		}
+	}
+}
+func (tbl *riakURLSigKeyTable) insertKeys(cluster *riak.Cluster) error {
+	for _, record := range tbl.Records {
+		objBytes, err := json.Marshal(record.Key)
+		if err != nil {
+			return fmt.Errorf("RiakURLSigKey insertKeys unable to marshal keys: %w", err)
+		}
+
+		err = setObject(cluster, makeRiakObject(objBytes, "url_sig_"+record.DeliveryService+".config"), BUCKET_URL_SIG)
+		if err != nil {
+			return fmt.Errorf("RiakURLSigKey insertKeys: %w", err)
+		}
+	}
+	return nil
+}
+func (tbl *riakURLSigKeyTable) validate() []string {
+	errs := []string{}
+	for i, record := range tbl.Records {
+		if record.DeliveryService == "" {
+			errs = append(errs, fmt.Sprintf("URL Key #%v: Delivery Service is blank!", i))
+		}
+	}
+	return errs
+}
+
+type riakURISignKeyRecord struct {
+	Key             map[string]tc.URISignerKeyset
+	DeliveryService string
+}
+type riakURISignKeyTable struct {
+	Records []riakURISignKeyRecord
+}
+
+func (tbl *riakURISignKeyTable) gatherKeys(cluster *riak.Cluster) error {
+	tbl.Records = []riakURISignKeyRecord{}
+	objs, err := getObjects(cluster, BUCKET_URI_SIG)
+	if err != nil {
+		return fmt.Errorf("RiakURISignKey gatherKeys: %w", err)
+	}
+	for _, obj := range objs {
+		key := map[string]tc.URISignerKeyset{}
+		if err := json.Unmarshal(obj.Value, &key); err != nil {
+			return fmt.Errorf("RiakURISignKey gatherKeys unable to unmarshal object into map[string]tc.URISignerKeySet: %w", err)
+		}
+
+		tbl.Records = append(tbl.Records, riakURISignKeyRecord{
+			DeliveryService: obj.Key,
+			Key:             key,
+		})
+	}
+	return nil
+}
+func (tbl *riakURISignKeyTable) toGeneric() []URISignKey {
+	keys := make([]URISignKey, len(tbl.Records))
+
+	for i, record := range tbl.Records {
+		keys[i] = URISignKey{
+			DeliveryService: record.DeliveryService,
+			Keys:            record.Key,
+		}
+	}
+
+	return keys
+}
+func (tbl *riakURISignKeyTable) fromGeneric(keys []URISignKey) {
+	tbl.Records = make([]riakURISignKeyRecord, len(keys))
+
+	for i, record := range keys {
+		tbl.Records[i] = riakURISignKeyRecord{
+			Key:             record.Keys,
+			DeliveryService: record.DeliveryService,
+		}
+	}
+}
+func (tbl *riakURISignKeyTable) insertKeys(cluster *riak.Cluster) error {
+	for _, record := range tbl.Records {
+		objBytes, err := json.Marshal(record.Key)
+		if err != nil {
+			return fmt.Errorf("RiakURISignKey insertKeys: unable to marshal key: %w", err)
+		}
+
+		err = setObject(cluster, makeRiakObject(objBytes, record.DeliveryService), BUCKET_URI_SIG)
+		if err != nil {
+			return fmt.Errorf("RiakURISignKey insertKeys: %w", err)
+		}
+	}
+	return nil
+}
+func (tbl *riakURISignKeyTable) validate() []string {
+	errs := []string{}
+	for i, record := range tbl.Records {
+		if record.DeliveryService == "" {
+			errs = append(errs, fmt.Sprintf("URI Key #%v: Delivery Service is blank!", i))
+		}
+	}
+	return errs
+}
+
+// Riak functions
+func makeRiakObject(data []byte, key string) *riak.Object {
+	return &riak.Object{
+		ContentType:     rfc.ApplicationJSON,
+		Charset:         "utf-8",
+		ContentEncoding: "utf-8",
+		Key:             key,
+		Value:           data,
+	}
+}
+func getObjects(cluster *riak.Cluster, bucket string) ([]*riak.Object, error) {
+	objs := []*riak.Object{}
+	keys, err := listKeys(cluster, bucket)
+	if err != nil {
+		return nil, err
+	}
+	for _, key := range keys {
+		objects, err := getObject(cluster, bucket, key)
+		if err != nil {
+			return nil, err
+		}
+		if len(objects) > 1 {
+			return nil, fmt.Errorf("Unexpected number of objects %v, ignoring\n", len(objects))
+		}
+
+		objs = append(objs, objects[0])
+	}
+
+	return objs, nil
+}
+func getObject(cluster *riak.Cluster, bucket string, key string) ([]*riak.Object, error) {
+	cmd, err := riak.NewFetchValueCommandBuilder().
+		WithBucket(bucket).
+		WithKey(key).
+		WithTimeout(time.Second * 60).
+		Build()
+	if err != nil {
+		return nil, fmt.Errorf("error building riak fetch value command: %w", err)
+	}
+
+	if err := cluster.Execute(cmd); err != nil {
+		return nil, fmt.Errorf("error executing riak fetch value command: %w", err)
+	}
+
+	fvc := cmd.(*riak.FetchValueCommand)
+	rsp := fvc.Response
+
+	if rsp.IsNotFound {
+		return nil, fmt.Errorf("errors executing riak fetch value command; key not found: %v:%v", bucket, key)
+	}
+
+	return rsp.Values, nil
+}
+func setObject(cluster *riak.Cluster, obj *riak.Object, bucket string) error {
+	cmd, err := riak.NewStoreValueCommandBuilder().
+		WithBucket(bucket).
+		WithContent(obj).
+		WithTimeout(time.Second * 5).

Review comment:
       Should this timeout be configurable (with a sane default)?

##########
File path: tools/traffic_vault_migrate/riak.go
##########
@@ -0,0 +1,746 @@
+package main
+
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+import (
+	"crypto/tls"
+	"encoding/json"
+	"errors"
+	"fmt"
+	"time"
+
+	"github.com/apache/trafficcontrol/lib/go-rfc"
+	"github.com/apache/trafficcontrol/lib/go-tc"
+
+	"github.com/basho/riak-go-client"
+)
+
+const (
+	BUCKET_SSL     = "ssl"
+	BUCKET_DNSSEC  = "dnssec"
+	BUCKET_URL_SIG = "url_sig_keys"
+	BUCKET_URI_SIG = "cdn_uri_sig_keys"
+
+	INDEX_SSL = "sslkeys"
+
+	SCHEMA_RIAK_KEY    = "_yz_rk"
+	SCHEMA_RIAK_BUCKET = "_yz_rb"
+)
+
+var (
+	SCHEMA_SSL_FIELDS = [...]string{SCHEMA_RIAK_KEY, SCHEMA_RIAK_BUCKET}
+)
+
+// RiakConfig  represents the configuration options available to the Riak backend
+type RiakConfig struct {
+	Host          string `json:"host"`
+	Port          string `json:"port"`
+	User          string `json:"user"`
+	Password      string `json:"password"`
+	Insecure      bool   `json:"insecure"`
+	TLSVersionRaw string `json:"tlsVersion"`
+	TLSVersion    uint16
+}
+
+// RiakBackend is the Riak implementation of TVBackend
+type RiakBackend struct {
+	sslKeys     riakSSLKeyTable
+	dnssecKeys  riakDNSSecKeyTable
+	uriSignKeys riakURISignKeyTable
+	urlSigKeys  riakURLSigKeyTable
+	cfg         RiakConfig
+	cluster     *riak.Cluster
+}
+
+// String returns a high level overview of the backend and its keys
+func (rb *RiakBackend) String() string {
+	data := fmt.Sprintf("Riak server %v@%v:%v\n", rb.cfg.User, rb.cfg.Host, rb.cfg.Port)
+	data += fmt.Sprintf("\tSSL Keys: %v\n", len(rb.sslKeys.Records))
+	data += fmt.Sprintf("\tDNSSec Keys: %v\n", len(rb.dnssecKeys.Records))
+	data += fmt.Sprintf("\tURI Keys: %v\n", len(rb.uriSignKeys.Records))
+	data += fmt.Sprintf("\tURL Keys: %v\n", len(rb.urlSigKeys.Records))
+	return data
+}
+
+// Name returns the name for this backend
+func (rb *RiakBackend) Name() string {
+	return "Riak"
+}
+
+// ReadConfig takes in a filename and will read it into the backends config
+func (rb *RiakBackend) ReadConfig(s string) error {
+	err := UnmarshalConfig(s, &rb.cfg)
+	if err != nil {
+		return err
+	}
+
+	if rb.cfg.TLSVersionRaw == "10" {
+		rb.cfg.TLSVersion = tls.VersionTLS10
+	} else if rb.cfg.TLSVersionRaw == "11" {
+		rb.cfg.TLSVersion = tls.VersionTLS11
+	} else if rb.cfg.TLSVersionRaw == "12" {
+		rb.cfg.TLSVersion = tls.VersionTLS12
+	} else if rb.cfg.TLSVersionRaw == "13" {
+		rb.cfg.TLSVersion = tls.VersionTLS13
+	}
+	return nil
+}
+
+// Insert takes the current keys and inserts them into the backend DB
+func (rb *RiakBackend) Insert() error {
+	if err := rb.sslKeys.insertKeys(rb.cluster); err != nil {
+		return err
+	}
+	if err := rb.dnssecKeys.insertKeys(rb.cluster); err != nil {
+		return err
+	}
+	if err := rb.urlSigKeys.insertKeys(rb.cluster); err != nil {
+		return err
+	}
+	if err := rb.uriSignKeys.insertKeys(rb.cluster); err != nil {
+		return err
+	}
+	return nil
+}
+
+// ValidateKey validates that the keys are valid (in most cases, certain fields are not null)
+func (rb *RiakBackend) ValidateKey() []string {
+	errors := []string{}
+	if errs := rb.sslKeys.validate(); errs != nil {
+		errors = append(errors, errs...)
+	}
+	if errs := rb.dnssecKeys.validate(); errs != nil {
+		errors = append(errors, errs...)
+	}
+	if errs := rb.uriSignKeys.validate(); errs != nil {
+		errors = append(errors, errs...)
+	}
+	if errs := rb.urlSigKeys.validate(); errs != nil {
+		errors = append(errors, errs...)
+	}
+
+	return errors
+}
+
+// SetSSLKeys takes in keys and converts & encrypts the data into the backends internal format
+func (rb *RiakBackend) SetSSLKeys(keys []SSLKey) error {
+	rb.sslKeys.fromGeneric(keys)
+	return nil
+}
+
+// SetDNSSecKeys takes in keys and converts & encrypts the data into the backends internal format
+func (rb *RiakBackend) SetDNSSecKeys(keys []DNSSecKey) error {
+	rb.dnssecKeys.fromGeneric(keys)
+	return nil
+}
+
+// SetURISignKeys takes in keys and converts & encrypts the data into the backends internal format
+func (rb *RiakBackend) SetURISignKeys(keys []URISignKey) error {
+	rb.uriSignKeys.fromGeneric(keys)
+	return nil
+}
+
+// SetURLSigKeys takes in keys and converts & encrypts the data into the backends internal format
+func (rb *RiakBackend) SetURLSigKeys(keys []URLSigKey) error {
+	rb.urlSigKeys.fromGeneric(keys)
+	return nil
+}
+
+// Start initiates the connection to the backend DB
+func (rb *RiakBackend) Start() error {
+	tlsConfig := &tls.Config{
+		InsecureSkipVerify: !rb.cfg.Insecure,
+		MaxVersion:         rb.cfg.TLSVersion,
+	}
+	auth := &riak.AuthOptions{
+		User:      rb.cfg.User,
+		Password:  rb.cfg.Password,
+		TlsConfig: tlsConfig,
+	}
+
+	cluster, err := getRiakCluster(rb.cfg, auth)
+	if err != nil {
+		return err
+	}
+	if err := cluster.Start(); err != nil {
+		return fmt.Errorf("unable to start riak cluster: %w", err)
+	}
+
+	rb.cluster = cluster
+	rb.sslKeys = riakSSLKeyTable{}
+	rb.dnssecKeys = riakDNSSecKeyTable{}
+	rb.urlSigKeys = riakURLSigKeyTable{}
+	rb.uriSignKeys = riakURISignKeyTable{}
+	return nil
+}
+
+// Close terminates the connection to the backend DB
+func (rb *RiakBackend) Close() error {
+	if err := rb.cluster.Stop(); err != nil {
+		return err
+	}
+	return nil
+}
+
+// Ping checks the connection to the backend DB
+func (rb *RiakBackend) Ping() error {
+	return ping(rb.cluster)
+}
+
+// GetSSLKeys converts the backends internal key representation into the common representation (SSLKey)
+func (rb *RiakBackend) GetSSLKeys() ([]SSLKey, error) {
+	return rb.sslKeys.toGeneric(), nil
+}
+
+// GetDNSSecKeys converts the backends internal key representation into the common representation (DNSSecKey)
+func (rb *RiakBackend) GetDNSSecKeys() ([]DNSSecKey, error) {
+	return rb.dnssecKeys.toGeneric(), nil
+}
+
+// GetURISignKeys converts the pg internal key representation into the common representation (URISignKey)
+func (rb *RiakBackend) GetURISignKeys() ([]URISignKey, error) {
+	return rb.uriSignKeys.toGeneric(), nil
+}
+
+// GetURLSigKeys converts the backends internal key representation into the common representation (URLSigKey)
+func (rb *RiakBackend) GetURLSigKeys() ([]URLSigKey, error) {
+	return rb.urlSigKeys.toGeneric(), nil
+}
+
+// Fetch gets all of the keys from the backend DB
+func (rb *RiakBackend) Fetch() error {
+	if err := rb.sslKeys.gatherKeys(rb.cluster); err != nil {
+		return err
+	}
+	if err := rb.dnssecKeys.gatherKeys(rb.cluster); err != nil {
+		return err
+	}
+	if err := rb.urlSigKeys.gatherKeys(rb.cluster); err != nil {
+		return err
+	}
+	if err := rb.uriSignKeys.gatherKeys(rb.cluster); err != nil {
+		return err
+	}
+
+	return nil
+}
+
+type riakSSLKeyRecord struct {
+	tc.DeliveryServiceSSLKeys
+}
+type riakSSLKeyTable struct {
+	Records []riakSSLKeyRecord
+}
+
+func (tbl *riakSSLKeyTable) gatherKeys(cluster *riak.Cluster) error {
+	searchDocs, err := search(cluster, INDEX_SSL, "cdn:*", fmt.Sprintf("%v:*latest", SCHEMA_RIAK_KEY), 1000, SCHEMA_SSL_FIELDS[:])
+	if err != nil {
+		return fmt.Errorf("RiakSSLKey gatherKeys: %w", err)
+	}
+
+	tbl.Records = make([]riakSSLKeyRecord, len(searchDocs))
+	for i, doc := range searchDocs {
+		objs, err := getObject(cluster, doc.Bucket, doc.Key)
+		if err != nil {
+			return err
+		}
+		if len(objs) < 1 {
+			return fmt.Errorf("RiakSSLKey gatherKeys unable to find any objects with key %v and bucket %v, but search results were returned", doc.Key, doc.Bucket)
+		}
+		if len(objs) > 1 {
+			return fmt.Errorf("RiakSSLKey gatherKeys more than 1 ssl key record found %v\n", len(objs))
+		}
+		var obj tc.DeliveryServiceSSLKeys
+		if err = json.Unmarshal(objs[0].Value, &obj); err != nil {
+			return fmt.Errorf("RiakSSLKey gatherKeys unable to unmarshal object into tc.DeliveryServiceSSLKeys: %w", err)
+		}
+		tbl.Records[i] = riakSSLKeyRecord{
+			DeliveryServiceSSLKeys: obj,
+		}
+	}
+	return nil
+}
+func (tbl *riakSSLKeyTable) toGeneric() []SSLKey {
+	keys := make([]SSLKey, len(tbl.Records))
+
+	for i, record := range tbl.Records {
+		keys[i] = SSLKey{
+			DeliveryServiceSSLKeys: record.DeliveryServiceSSLKeys,
+		}
+	}
+
+	return keys
+}
+func (tbl *riakSSLKeyTable) fromGeneric(keys []SSLKey) {
+	tbl.Records = make([]riakSSLKeyRecord, len(keys))
+
+	for i, record := range keys {
+		tbl.Records[i] = riakSSLKeyRecord{
+			DeliveryServiceSSLKeys: record.DeliveryServiceSSLKeys,
+		}
+	}
+}
+func (tbl *riakSSLKeyTable) insertKeys(cluster *riak.Cluster) error {
+	for _, record := range tbl.Records {
+		objBytes, err := json.Marshal(record.DeliveryServiceSSLKeys)
+		if err != nil {
+			return fmt.Errorf("RiakSSLKey insertKeys failed to unmarshal keys: %w", err)
+		}
+		err = setObject(cluster, makeRiakObject(objBytes, record.DeliveryService+"-"+record.Version.String()), BUCKET_SSL)
+		if err != nil {
+			return fmt.Errorf("RiakSSLKey insertKeys: %w", err)
+		}
+		err = setObject(cluster, makeRiakObject(objBytes, record.DeliveryService+"-latest"), BUCKET_SSL)
+		if err != nil {
+			return fmt.Errorf("RiakSSLKey insertKeys: %w", err)
+		}
+	}
+	return nil
+}
+func (tbl *riakSSLKeyTable) validate() []string {
+	errs := []string{}
+	for i, record := range tbl.Records {
+		if record.DeliveryService == "" {
+			errs = append(errs, fmt.Sprintf("SSL Key #%v: Delivery Service is blank!", i))
+		}
+		if record.CDN == "" {
+			errs = append(errs, fmt.Sprintf("SSL Key #%v: CDN is blank!", i))
+		}
+		if record.Version.String() == "" {
+			errs = append(errs, fmt.Sprintf("SSL Key #%v: Version is blank!", i))
+		}
+	}
+	return errs
+}
+
+type riakDNSSecKeyRecord struct {
+	CDN string
+	Key tc.DNSSECKeysRiak
+}
+type riakDNSSecKeyTable struct {
+	Records []riakDNSSecKeyRecord
+}
+
+func (tbl *riakDNSSecKeyTable) gatherKeys(cluster *riak.Cluster) error {
+	tbl.Records = []riakDNSSecKeyRecord{}
+	objs, err := getObjects(cluster, BUCKET_DNSSEC)
+	if err != nil {
+		return fmt.Errorf("RiakDNSSecKey gatherKeys: %w", err)
+	}
+	for _, obj := range objs {
+		key := tc.DNSSECKeysRiak{}
+		if err := json.Unmarshal(obj.Value, &key); err != nil {
+			return fmt.Errorf("RiakDNSSecKey gatherKeys unable to unmarshal object to tc.DNSSECKeysRiak: %w", err)
+		}
+		tbl.Records = append(tbl.Records, riakDNSSecKeyRecord{
+			CDN: obj.Key,
+			Key: key,
+		})
+	}
+	return nil
+}
+func (tbl *riakDNSSecKeyTable) toGeneric() []DNSSecKey {
+	keys := make([]DNSSecKey, len(tbl.Records))
+
+	for i, record := range tbl.Records {
+		keys[i] = DNSSecKey{
+			CDN:                    record.CDN,
+			DNSSECKeysTrafficVault: tc.DNSSECKeysTrafficVault(record.Key),
+		}
+	}
+
+	return keys
+}
+func (tbl *riakDNSSecKeyTable) fromGeneric(keys []DNSSecKey) {
+	tbl.Records = make([]riakDNSSecKeyRecord, len(keys))
+
+	for i, record := range keys {
+		tbl.Records[i] = riakDNSSecKeyRecord{
+			CDN: record.CDN,
+			Key: tc.DNSSECKeysRiak(record.DNSSECKeysTrafficVault),
+		}
+	}
+}
+func (tbl *riakDNSSecKeyTable) insertKeys(cluster *riak.Cluster) error {
+	for _, record := range tbl.Records {
+		objBytes, err := json.Marshal(record.Key)
+		if err != nil {
+			return fmt.Errorf("RiakDNSSecKey insertKeys error marshalling keys: %w", err)
+		}
+
+		err = setObject(cluster, makeRiakObject(objBytes, record.CDN), BUCKET_DNSSEC)
+		if err != nil {
+			return fmt.Errorf("RiakDNSSecKey insertKeys: %w", err)
+		}
+	}
+	return nil
+}
+func (tbl *riakDNSSecKeyTable) validate() []string {
+	errs := []string{}
+	for i, record := range tbl.Records {
+		if record.CDN == "" {
+			errs = append(errs, fmt.Sprintf("DNSSec Key #%v: CDN is blank!", i))
+		}
+	}
+	return errs
+}
+
+type riakURLSigKeyRecord struct {
+	Key             tc.URLSigKeys
+	DeliveryService string
+}
+type riakURLSigKeyTable struct {
+	Records []riakURLSigKeyRecord
+}
+
+func (tbl *riakURLSigKeyTable) gatherKeys(cluster *riak.Cluster) error {
+	tbl.Records = []riakURLSigKeyRecord{}
+	objs, err := getObjects(cluster, BUCKET_URL_SIG)
+	if err != nil {
+		return fmt.Errorf("RiakURLSigKey gatherKeys: %w", err)
+	}
+	for _, obj := range objs {
+		key := tc.URLSigKeys{}
+		if err := json.Unmarshal(obj.Value, &key); err != nil {
+			return fmt.Errorf("RiakURLSigKey gatherKeys unable to unamrshal object into tc.URLSigKeys: %w", err)
+		}
+		tbl.Records = append(tbl.Records, riakURLSigKeyRecord{
+			DeliveryService: obj.Key,
+			Key:             key,
+		})
+	}
+	return nil
+}
+func (tbl *riakURLSigKeyTable) toGeneric() []URLSigKey {
+	keys := make([]URLSigKey, len(tbl.Records))
+
+	for i, record := range tbl.Records {
+		keys[i] = URLSigKey{
+			URLSigKeys:      record.Key,
+			DeliveryService: record.DeliveryService,
+		}
+	}
+
+	return keys
+}
+func (tbl *riakURLSigKeyTable) fromGeneric(keys []URLSigKey) {
+	tbl.Records = make([]riakURLSigKeyRecord, len(keys))
+	for i, key := range keys {
+		tbl.Records[i] = riakURLSigKeyRecord{
+			Key:             key.URLSigKeys,
+			DeliveryService: key.DeliveryService,
+		}
+	}
+}
+func (tbl *riakURLSigKeyTable) insertKeys(cluster *riak.Cluster) error {
+	for _, record := range tbl.Records {
+		objBytes, err := json.Marshal(record.Key)
+		if err != nil {
+			return fmt.Errorf("RiakURLSigKey insertKeys unable to marshal keys: %w", err)
+		}
+
+		err = setObject(cluster, makeRiakObject(objBytes, "url_sig_"+record.DeliveryService+".config"), BUCKET_URL_SIG)
+		if err != nil {
+			return fmt.Errorf("RiakURLSigKey insertKeys: %w", err)
+		}
+	}
+	return nil
+}
+func (tbl *riakURLSigKeyTable) validate() []string {
+	errs := []string{}
+	for i, record := range tbl.Records {
+		if record.DeliveryService == "" {
+			errs = append(errs, fmt.Sprintf("URL Key #%v: Delivery Service is blank!", i))
+		}
+	}
+	return errs
+}
+
+type riakURISignKeyRecord struct {
+	Key             map[string]tc.URISignerKeyset
+	DeliveryService string
+}
+type riakURISignKeyTable struct {
+	Records []riakURISignKeyRecord
+}
+
+func (tbl *riakURISignKeyTable) gatherKeys(cluster *riak.Cluster) error {
+	tbl.Records = []riakURISignKeyRecord{}
+	objs, err := getObjects(cluster, BUCKET_URI_SIG)
+	if err != nil {
+		return fmt.Errorf("RiakURISignKey gatherKeys: %w", err)
+	}
+	for _, obj := range objs {
+		key := map[string]tc.URISignerKeyset{}
+		if err := json.Unmarshal(obj.Value, &key); err != nil {
+			return fmt.Errorf("RiakURISignKey gatherKeys unable to unmarshal object into map[string]tc.URISignerKeySet: %w", err)
+		}
+
+		tbl.Records = append(tbl.Records, riakURISignKeyRecord{
+			DeliveryService: obj.Key,
+			Key:             key,
+		})
+	}
+	return nil
+}
+func (tbl *riakURISignKeyTable) toGeneric() []URISignKey {
+	keys := make([]URISignKey, len(tbl.Records))
+
+	for i, record := range tbl.Records {
+		keys[i] = URISignKey{
+			DeliveryService: record.DeliveryService,
+			Keys:            record.Key,
+		}
+	}
+
+	return keys
+}
+func (tbl *riakURISignKeyTable) fromGeneric(keys []URISignKey) {
+	tbl.Records = make([]riakURISignKeyRecord, len(keys))
+
+	for i, record := range keys {
+		tbl.Records[i] = riakURISignKeyRecord{
+			Key:             record.Keys,
+			DeliveryService: record.DeliveryService,
+		}
+	}
+}
+func (tbl *riakURISignKeyTable) insertKeys(cluster *riak.Cluster) error {
+	for _, record := range tbl.Records {
+		objBytes, err := json.Marshal(record.Key)
+		if err != nil {
+			return fmt.Errorf("RiakURISignKey insertKeys: unable to marshal key: %w", err)
+		}
+
+		err = setObject(cluster, makeRiakObject(objBytes, record.DeliveryService), BUCKET_URI_SIG)
+		if err != nil {
+			return fmt.Errorf("RiakURISignKey insertKeys: %w", err)
+		}
+	}
+	return nil
+}
+func (tbl *riakURISignKeyTable) validate() []string {
+	errs := []string{}
+	for i, record := range tbl.Records {
+		if record.DeliveryService == "" {
+			errs = append(errs, fmt.Sprintf("URI Key #%v: Delivery Service is blank!", i))

Review comment:
       nit: type-specific formatting instead of %v




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [trafficcontrol] ocket8888 commented on a change in pull request #5924: Add tool to migrate data between TV backends

Posted by GitBox <gi...@apache.org>.
ocket8888 commented on a change in pull request #5924:
URL: https://github.com/apache/trafficcontrol/pull/5924#discussion_r648527066



##########
File path: tools/traffic_vault_migrate/postgres.go
##########
@@ -0,0 +1,716 @@
+package main
+
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+import (
+	"database/sql"
+	"encoding/base64"
+	"encoding/json"
+	"errors"
+	"fmt"
+	"github.com/apache/trafficcontrol/lib/go-tc"
+	util "github.com/apache/trafficcontrol/lib/go-util"
+	_ "github.com/lib/pq"
+	"log"
+	"reflect"
+	"strconv"
+	"strings"
+)
+
+type PGConfig struct {
+	Host     string `json:"host"`
+	Port     string `json:"port"`
+	User     string `json:"user"`
+	Password string `json:"password"`
+	SSLMode  string `json:"sslmode"`
+	Database string `json:"database"`
+	Key      string `json:"aesKey"`
+	AESKey   []byte
+}
+type PGBackend struct {
+	sslKey PGSSLKeyTable
+	dnssec PGDNSSecTable
+	uri    PGURISignKeyTable
+	url    PGURLSigKeyTable
+	cfg    PGConfig
+	db     *sql.DB
+}
+
+func (pg *PGBackend) String() string {
+	data := fmt.Sprintf("PG server %v@%v:%v\n", pg.cfg.User, pg.cfg.Host, pg.cfg.Port)
+	data += fmt.Sprintf("\tSSL Keys: %v\n", len(pg.sslKey.Records))
+	data += fmt.Sprintf("\tDNSSec Keys: %v\n", len(pg.dnssec.Records))
+	data += fmt.Sprintf("\tURI Keys: %v\n", len(pg.uri.Records))
+	data += fmt.Sprintf("\tURL Keys: %v\n", len(pg.url.Records))
+	return data
+}
+func (pg *PGBackend) Name() string {
+	return "PG"
+}
+func (pg *PGBackend) ReadConfig(s string) error {
+	genericCfg, err := UnmarshalConfig(s, reflect.TypeOf(pg.cfg))

Review comment:
       It looks like the Riak backend calls it's own `ReadConfig` method that doesn't seem to be re-using this code - or vice-versa




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [trafficcontrol] shamrickus commented on a change in pull request #5924: Add tool to migrate data between TV backends

Posted by GitBox <gi...@apache.org>.
shamrickus commented on a change in pull request #5924:
URL: https://github.com/apache/trafficcontrol/pull/5924#discussion_r648792209



##########
File path: tools/traffic_vault_migrate/traffic_vault_migrate.go
##########
@@ -0,0 +1,448 @@
+package main
+
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+import (
+	"encoding/json"
+	"errors"
+	"flag"
+	"fmt"
+	"io/ioutil"
+	"log"
+	"os"
+	"reflect"
+	"sort"
+	"strings"
+	"time"
+
+	"github.com/apache/trafficcontrol/lib/go-tc"
+)
+
+var (
+	fromType string
+	toType   string
+	fromCfg  string
+	toCfg    string
+	dry      bool
+	compare  bool
+	confirm  bool
+	dump     bool
+
+	riakBE RiakBackend = RiakBackend{}
+	pgBE   PGBackend   = PGBackend{}
+)
+
+func init() {
+	flag.StringVar(&fromType, "from_type", riakBE.Name(), fmt.Sprintf("From server types (%v)", strings.Join(supportedTypes(), "|")))
+	flag.StringVar(&toType, "to_type", pgBE.Name(), fmt.Sprintf("From server types (%v)", strings.Join(supportedTypes(), "|")))
+	flag.StringVar(&toCfg, "to_cfg", "pg.json", "To server config file")
+	flag.StringVar(&fromCfg, "from_cfg", "riak.json", "From server config file")
+	flag.BoolVar(&dry, "dry", false, "Do not perform writes")
+	flag.BoolVar(&compare, "compare", false, "Compare to and from server records")
+	flag.BoolVar(&confirm, "confirm", true, "Requires confirmation before inserting records")
+	flag.BoolVar(&dump, "dump", false, "Write keys (from 'from' server) to disk")
+}
+
+func supportedBackends() []TVBackend {
+	return []TVBackend{
+		&riakBE, &pgBE,
+	}
+}
+
+func main() {
+	flag.Parse()
+	var fromSrv TVBackend
+	var toSrv TVBackend
+
+	toSrvUsed := !dump && !dry
+
+	if !validateType(fromType) {
+		log.Fatal("Unknown fromType " + fromType)
+	}
+	if toSrvUsed && !validateType(toType) {
+		log.Fatal("Unknown toType " + toType)
+	}
+
+	fromSrv = getBackendFromType(fromType)
+	if toSrvUsed {
+		toSrv = getBackendFromType(toType)
+	}
+
+	var toTimer time.Time
+	var toTime float64
+	var fromTimer time.Time
+	var fromTime float64
+
+	log.Println("Reading configs...")
+	fromTimer = time.Now()
+	if err := fromSrv.ReadConfig(fromCfg); err != nil {
+		log.Fatalf("Unable to read fromSrv cfg: %v", err)
+	}
+	fromTime = time.Now().Sub(fromTimer).Seconds()
+
+	if toSrvUsed {
+		toTimer = time.Now()
+		if err := toSrv.ReadConfig(toCfg); err != nil {
+			log.Fatalf("Unable to read toSrv cfg: %v", err)
+		}
+		toTime := time.Now().Sub(toTimer).Seconds()

Review comment:
       You are correct, I removed them.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [trafficcontrol] shamrickus commented on a change in pull request #5924: Add tool to migrate data between TV backends

Posted by GitBox <gi...@apache.org>.
shamrickus commented on a change in pull request #5924:
URL: https://github.com/apache/trafficcontrol/pull/5924#discussion_r648545931



##########
File path: tools/traffic_vault_migrate/postgres.go
##########
@@ -0,0 +1,716 @@
+package main
+
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+import (
+	"database/sql"
+	"encoding/base64"
+	"encoding/json"
+	"errors"
+	"fmt"
+	"github.com/apache/trafficcontrol/lib/go-tc"
+	util "github.com/apache/trafficcontrol/lib/go-util"
+	_ "github.com/lib/pq"
+	"log"
+	"reflect"
+	"strconv"
+	"strings"
+)
+
+type PGConfig struct {
+	Host     string `json:"host"`
+	Port     string `json:"port"`
+	User     string `json:"user"`
+	Password string `json:"password"`
+	SSLMode  string `json:"sslmode"`
+	Database string `json:"database"`
+	Key      string `json:"aesKey"`
+	AESKey   []byte
+}
+type PGBackend struct {
+	sslKey PGSSLKeyTable
+	dnssec PGDNSSecTable
+	uri    PGURISignKeyTable
+	url    PGURLSigKeyTable
+	cfg    PGConfig
+	db     *sql.DB
+}
+
+func (pg *PGBackend) String() string {
+	data := fmt.Sprintf("PG server %v@%v:%v\n", pg.cfg.User, pg.cfg.Host, pg.cfg.Port)
+	data += fmt.Sprintf("\tSSL Keys: %v\n", len(pg.sslKey.Records))
+	data += fmt.Sprintf("\tDNSSec Keys: %v\n", len(pg.dnssec.Records))
+	data += fmt.Sprintf("\tURI Keys: %v\n", len(pg.uri.Records))
+	data += fmt.Sprintf("\tURL Keys: %v\n", len(pg.url.Records))
+	return data
+}
+func (pg *PGBackend) Name() string {
+	return "PG"
+}
+func (pg *PGBackend) ReadConfig(s string) error {
+	genericCfg, err := UnmarshalConfig(s, reflect.TypeOf(pg.cfg))

Review comment:
       ahhh...I misread your earlier comment, nope there is no reason.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [trafficcontrol] rawlinp commented on a change in pull request #5924: Add tool to migrate data between TV backends

Posted by GitBox <gi...@apache.org>.
rawlinp commented on a change in pull request #5924:
URL: https://github.com/apache/trafficcontrol/pull/5924#discussion_r651980497



##########
File path: tools/traffic_vault_migrate/postgres.go
##########
@@ -0,0 +1,743 @@
+package main
+
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+import (
+	"database/sql"
+	"encoding/base64"
+	"encoding/json"
+	"errors"
+	"fmt"
+	"log"
+	"strconv"
+	"strings"
+
+	"github.com/apache/trafficcontrol/lib/go-tc"
+	util "github.com/apache/trafficcontrol/lib/go-util"
+
+	_ "github.com/lib/pq"
+)
+
+// PGConfig represents the configuration options available to the PG backend
+type PGConfig struct {
+	Host     string `json:"host"`
+	Port     string `json:"port"`
+	User     string `json:"user"`
+	Password string `json:"password"`
+	SSLMode  string `json:"sslmode"`
+	Database string `json:"database"`
+	Key      string `json:"aesKey"`
+	AESKey   []byte
+}
+
+// PGBackend is the Postgres implementation of TVBackend
+type PGBackend struct {
+	sslKey pgSSLKeyTable
+	dnssec pgDNSSecTable
+	uri    pgURISignKeyTable
+	url    pgURLSigKeyTable
+	cfg    PGConfig
+	db     *sql.DB
+}
+
+// String returns a high level overview of the backend and its keys
+func (pg *PGBackend) String() string {
+	data := fmt.Sprintf("PG server %v@%v:%v\n", pg.cfg.User, pg.cfg.Host, pg.cfg.Port)
+	data += fmt.Sprintf("\tSSL Keys: %v\n", len(pg.sslKey.Records))
+	data += fmt.Sprintf("\tDNSSec Keys: %v\n", len(pg.dnssec.Records))
+	data += fmt.Sprintf("\tURI Keys: %v\n", len(pg.uri.Records))
+	data += fmt.Sprintf("\tURL Keys: %v\n", len(pg.url.Records))
+	return data
+}
+
+// Name returns the name for this backend
+func (pg *PGBackend) Name() string {
+	return "PG"
+}
+
+// ReadConfig takes in a filename and will read it into the backends config
+func (pg *PGBackend) ReadConfig(s string) error {
+	err := UnmarshalConfig(s, &pg.cfg)
+	if err != nil {
+		return err
+	}
+
+	pg.cfg.AESKey, err = base64.StdEncoding.DecodeString(pg.cfg.Key)
+	if err != nil {
+		return fmt.Errorf("unable to decode PG AESKey: %w", err)
+	}
+	return nil
+}
+
+// Insert takes the current keys and inserts them into the backend DB
+func (pg *PGBackend) Insert() error {
+	if err := pg.sslKey.insertKeys(pg.db); err != nil {
+		return err
+	}
+	if err := pg.dnssec.insertKeys(pg.db); err != nil {
+		return err
+	}
+	if err := pg.url.insertKeys(pg.db); err != nil {
+		return err
+	}
+	if err := pg.uri.insertKeys(pg.db); err != nil {
+		return err
+	}
+	return nil
+}
+
+// Start initiates the connection to the backend DB
+func (pg *PGBackend) Start() error {
+	sqlStr := fmt.Sprintf("postgres://%s:%s@%s:%s/%s?sslmode=%s", pg.cfg.User, pg.cfg.Password, pg.cfg.Host, pg.cfg.Port, pg.cfg.Database, pg.cfg.SSLMode)
+	db, err := sql.Open("postgres", sqlStr)
+	if err != nil {
+		return fmt.Errorf("unable to start PG client: %w", err)
+	}
+
+	pg.db = db
+	pg.sslKey = pgSSLKeyTable{}
+	pg.dnssec = pgDNSSecTable{}
+	pg.url = pgURLSigKeyTable{}
+	pg.uri = pgURISignKeyTable{}
+
+	return nil
+}
+
+// ValidateKey validates that the keys are valid (in most cases, certain fields are not null)
+func (pg *PGBackend) ValidateKey() []string {
+	var errors []string
+	if errs := pg.sslKey.validate(); errs != nil {
+		errors = append(errors, errs...)
+	}
+	if errs := pg.dnssec.validate(); errs != nil {
+		errors = append(errors, errs...)
+	}
+	if errs := pg.uri.validate(); errs != nil {
+		errors = append(errors, errs...)
+	}
+	if errs := pg.url.validate(); errs != nil {
+		errors = append(errors, errs...)
+	}
+	return errors
+}
+
+// Close terminates the connection to the backend DB
+func (pg *PGBackend) Close() error {
+	return pg.db.Close()
+}
+
+// Ping checks the connection to the backend DB
+func (pg *PGBackend) Ping() error {
+	return pg.db.Ping()
+}
+
+// Fetch gets all of the keys from the backend DB
+func (pg *PGBackend) Fetch() error {
+	if err := pg.sslKey.gatherKeys(pg.db); err != nil {
+		return err
+	}
+
+	if err := pg.dnssec.gatherKeys(pg.db); err != nil {
+		return err
+	}
+
+	if err := pg.url.gatherKeys(pg.db); err != nil {
+		return err
+	}
+
+	if err := pg.uri.gatherKeys(pg.db); err != nil {
+		return err
+	}
+
+	return nil
+}
+
+// GetSSLKeys converts the backends internal key representation into the common representation (SSLKey)
+func (pg *PGBackend) GetSSLKeys() ([]SSLKey, error) {
+	if err := pg.sslKey.decrypt(pg.cfg.AESKey); err != nil {
+		return nil, err
+	}
+	return pg.sslKey.toGeneric(), nil
+}
+
+// SetSSLKeys takes in keys and converts & encrypts the data into the backends internal format
+func (pg *PGBackend) SetSSLKeys(keys []SSLKey) error {
+	pg.sslKey.fromGeneric(keys)
+	return pg.sslKey.encrypt(pg.cfg.AESKey)
+}
+
+// GetDNSSecKeys converts the backends internal key representation into the common representation (DNSSecKey)
+func (pg *PGBackend) GetDNSSecKeys() ([]DNSSecKey, error) {
+	if err := pg.dnssec.decrypt(pg.cfg.AESKey); err != nil {
+		return nil, err
+	}
+	return pg.dnssec.toGeneric(), nil
+}
+
+// SetDNSSecKeys takes in keys and converts & encrypts the data into the backends internal format
+func (pg *PGBackend) SetDNSSecKeys(keys []DNSSecKey) error {
+	pg.dnssec.fromGeneric(keys)
+	return pg.dnssec.encrypt(pg.cfg.AESKey)
+}
+
+// GetURISignKeys converts the pg internal key representation into the common representation (URISignKey)
+func (pg *PGBackend) GetURISignKeys() ([]URISignKey, error) {
+	if err := pg.uri.decrypt(pg.cfg.AESKey); err != nil {
+		return nil, err
+	}
+	return pg.uri.toGeneric(), nil
+}
+
+// SetURISignKeys takes in keys and converts & encrypts the data into the backends internal format
+func (pg *PGBackend) SetURISignKeys(keys []URISignKey) error {
+	pg.uri.fromGeneric(keys)
+	return pg.uri.encrypt(pg.cfg.AESKey)
+}
+
+// GetURLSigKeys converts the backends internal key representation into the common representation (URLSigKey)
+func (pg *PGBackend) GetURLSigKeys() ([]URLSigKey, error) {
+	if err := pg.url.decrypt(pg.cfg.AESKey); err != nil {
+		return nil, err
+	}
+	return pg.url.toGeneric(), nil
+}
+
+// SetURLSigKeys takes in keys and converts & encrypts the data into the backends internal format
+func (pg *PGBackend) SetURLSigKeys(keys []URLSigKey) error {
+	pg.url.fromGeneric(keys)
+	return pg.url.encrypt(pg.cfg.AESKey)
+}
+
+type pgCommonRecord struct {
+	DataEncrypted []byte
+}
+
+type pgDNSSecRecord struct {
+	Key tc.DNSSECKeysTrafficVault
+	CDN string
+	pgCommonRecord
+}
+type pgDNSSecTable struct {
+	Records []pgDNSSecRecord
+}
+
+func (tbl *pgDNSSecTable) gatherKeys(db *sql.DB) error {
+	sz, err := getSize(db, "dnssec")
+	if err != nil {
+		log.Println("PGDNSSec gatherKeys: unable to determine size of dnssec table")
+	}
+	tbl.Records = make([]pgDNSSecRecord, sz)
+
+	rows, err := db.Query("SELECT cdn, data from dnssec")
+	if err != nil {
+		return fmt.Errorf("PGDNSSec gatherKeys: unable to query: %w", err)
+	}
+	defer rows.Close()
+	i := 0
+	for rows.Next() {
+		if i > len(tbl.Records)-1 {
+			return fmt.Errorf("PGDNSSec gatherKeys got more results than expected %v", len(tbl.Records))
+		}
+		err := rows.Scan(&tbl.Records[i].CDN, &tbl.Records[i].DataEncrypted)
+		if err != nil {
+			return fmt.Errorf("PGDNSSec gatherKeys unable to scan row: %w", err)
+		}
+		i += 1
+	}
+	return nil
+}
+func (tbl *pgDNSSecTable) decrypt(aesKey []byte) error {
+	for i, _ := range tbl.Records {
+		err := decryptInto(aesKey, tbl.Records[i].DataEncrypted, &tbl.Records[i].Key)
+		if err != nil {
+			return fmt.Errorf("unable to decrypt into keys: %w", err)
+		}
+	}
+	return nil
+}
+func (tbl *pgDNSSecTable) encrypt(aesKey []byte) error {
+	for i, dns := range tbl.Records {
+		data, err := json.Marshal(&dns.Key)
+		if err != nil {
+			return fmt.Errorf("encrypt issue marshalling keys: %w", err)
+		}
+		dat, err := encrypt(data, aesKey)
+		if err != nil {
+			return fmt.Errorf("encrypt error: %w", err)
+		}
+		tbl.Records[i].DataEncrypted = dat
+	}
+	return nil
+}
+func (tbl *pgDNSSecTable) toGeneric() []DNSSecKey {
+	keys := make([]DNSSecKey, len(tbl.Records))
+
+	for i, record := range tbl.Records {
+		keys[i] = DNSSecKey{
+			CDN:                    record.CDN,
+			DNSSECKeysTrafficVault: record.Key,
+		}
+	}
+
+	return keys
+}
+func (tbl *pgDNSSecTable) fromGeneric(keys []DNSSecKey) {
+	tbl.Records = make([]pgDNSSecRecord, len(keys))
+
+	for i, key := range keys {
+		tbl.Records[i] = pgDNSSecRecord{
+			Key: key.DNSSECKeysTrafficVault,
+			CDN: key.CDN,
+			pgCommonRecord: pgCommonRecord{
+				DataEncrypted: nil,
+			},
+		}
+	}
+}
+func (tbl *pgDNSSecTable) validate() []string {
+	for i, record := range tbl.Records {
+		if record.DataEncrypted == nil && len(record.Key) > 0 {
+			return []string{fmt.Sprintf("DNSSEC Key %v: DataEncrypted is blank!", i)}
+		}
+	}
+	return nil
+}
+func (tbl *pgDNSSecTable) insertKeys(db *sql.DB) error {
+	queryBase := "INSERT INTO dnssec (cdn, data) VALUES "
+	stride := 2
+	queryArgs := make([]interface{}, len(tbl.Records)*stride)
+	for i, record := range tbl.Records {
+		j := i * stride
+		queryArgs[j] = record.CDN
+		queryArgs[j+1] = record.DataEncrypted
+	}
+	return insertIntoTable(db, queryBase, stride, queryArgs)
+}
+
+type pgSSLKeyRecord struct {
+	Keys tc.DeliveryServiceSSLKeys
+	pgCommonRecord
+
+	// These records are stored on the table but are duplicated
+	DeliveryService string
+	CDN             string
+}
+type pgSSLKeyTable struct {
+	Records []pgSSLKeyRecord
+}
+
+func (tbl *pgSSLKeyTable) insertKeys(db *sql.DB) error {
+	queryBase := "INSERT INTO sslkey (deliveryservice, data, cdn, version) VALUES "
+	duplicateKeys := 2
+	stride := 4
+	queryArgs := make([]interface{}, len(tbl.Records)*stride*duplicateKeys)
+	for i, record := range tbl.Records {
+		j := i * duplicateKeys * stride
+
+		queryArgs[j] = record.DeliveryService
+		queryArgs[j+1] = record.DataEncrypted
+		queryArgs[j+2] = record.CDN
+		queryArgs[j+3] = record.Keys.Version.String()
+
+		queryArgs[j+4] = record.DeliveryService
+		queryArgs[j+5] = record.DataEncrypted
+		queryArgs[j+6] = record.CDN
+		queryArgs[j+7] = "latest"
+	}
+	return insertIntoTable(db, queryBase, 4, queryArgs)
+}
+func (tbl *pgSSLKeyTable) gatherKeys(db *sql.DB) error {
+	sz, err := getSize(db, "sslkey WHERE version='latest'")
+	if err != nil {
+		return fmt.Errorf("PGSSLKey gatherKeys unable to determine size of sslkey table: %w", err)
+	}
+	tbl.Records = make([]pgSSLKeyRecord, sz)
+
+	rows, err := db.Query("SELECT data, deliveryservice, cdn from sslkey WHERE version = 'latest'")
+	if err != nil {
+		return fmt.Errorf("PGSSLKey gatherKeys unable to query: %w", err)
+	}
+	defer rows.Close()
+	i := 0
+	for rows.Next() {
+		if i > len(tbl.Records)-1 {
+			return errors.New("PGSSLKey gatherKeys: got more results than expected")
+		}
+		err := rows.Scan(&tbl.Records[i].DataEncrypted, &tbl.Records[i].DeliveryService, &tbl.Records[i].CDN)
+		if err != nil {
+			return fmt.Errorf("PGSSLKey gatherKeys unable to scan row: %w", err)
+		}
+		i += 1
+	}
+	return nil
+}
+func (tbl *pgSSLKeyTable) decrypt(aesKey []byte) error {
+	for i, dns := range tbl.Records {
+		err := decryptInto(aesKey, dns.DataEncrypted, &tbl.Records[i].Keys)
+		if err != nil {
+			return fmt.Errorf("unable to decrypt into keys: %w", err)
+		}
+	}
+	return nil
+}
+func (tbl *pgSSLKeyTable) encrypt(aesKey []byte) error {
+	for i, dns := range tbl.Records {
+		data, err := json.Marshal(dns.Keys)
+		if err != nil {
+			return fmt.Errorf("encrypt issue marshalling keys: %w", err)
+		}
+		dat, err := encrypt(data, aesKey)
+		if err != nil {
+			return fmt.Errorf("encrypt error: %w", err)
+		}
+		tbl.Records[i].DataEncrypted = dat
+	}
+	return nil
+}
+func (tbl *pgSSLKeyTable) toGeneric() []SSLKey {
+	keys := make([]SSLKey, len(tbl.Records))
+
+	for i, record := range tbl.Records {
+		keys[i] = SSLKey{
+			DeliveryServiceSSLKeys: record.Keys,
+		}
+	}
+	return keys
+}
+func (tbl *pgSSLKeyTable) fromGeneric(keys []SSLKey) {
+	tbl.Records = make([]pgSSLKeyRecord, len(keys))
+
+	for i, key := range keys {
+		tbl.Records[i] = pgSSLKeyRecord{
+			Keys: key.DeliveryServiceSSLKeys,
+			pgCommonRecord: pgCommonRecord{
+				DataEncrypted: nil,
+			},
+			DeliveryService: key.DeliveryService,
+			CDN:             key.CDN,
+		}
+	}
+}
+func (tbl *pgSSLKeyTable) validate() []string {
+	defaultKey := tc.DeliveryServiceSSLKeys{}
+	var errors []string
+	fmtStr := "SSL Key %v: %v"
+	for i, record := range tbl.Records {
+		if record.Keys == defaultKey {
+			errors = append(errors, fmt.Sprintf(fmtStr, i, "DS SSL Keys are default!"))
+		} else if record.Keys.Key == "" {
+			errors = append(errors, fmt.Sprintf(fmtStr, i, "Key is blank!"))
+		} else if record.Keys.CDN == "" {
+			errors = append(errors, fmt.Sprintf(fmtStr, i, "CDN is blank!"))
+		} else if record.Keys.DeliveryService == "" {
+			errors = append(errors, fmt.Sprintf(fmtStr, i, "DS is blank!"))
+		} else if record.DataEncrypted == nil {
+			errors = append(errors, fmt.Sprintf(fmtStr, i, "DataEncrypted is blank!"))
+		} else if record.Keys.Version.String() == "" {
+			errors = append(errors, fmt.Sprintf(fmtStr, i, "Version is blank!"))
+		}
+	}
+	return errors
+}
+
+type pgURLSigKeyRecord struct {
+	Keys            tc.URLSigKeys
+	DeliveryService string
+	pgCommonRecord
+}
+type pgURLSigKeyTable struct {
+	Records []pgURLSigKeyRecord
+}
+
+func (tbl *pgURLSigKeyTable) insertKeys(db *sql.DB) error {
+	queryBase := "INSERT INTO url_sig_key (deliveryservice, data) VALUES "
+	stride := 2
+	queryArgs := make([]interface{}, len(tbl.Records)*stride)
+	for i, record := range tbl.Records {
+		j := i * stride
+		queryArgs[j] = record.DeliveryService
+		queryArgs[j+1] = record.DataEncrypted
+	}
+	return insertIntoTable(db, queryBase, stride, queryArgs)
+}
+func (tbl *pgURLSigKeyTable) gatherKeys(db *sql.DB) error {
+	sz, err := getSize(db, "url_sig_key")
+	if err != nil {
+		log.Println("PGURLSigKey gatherKeys: unable to determine url_sig_key table size")
+	}
+	tbl.Records = make([]pgURLSigKeyRecord, sz)
+
+	rows, err := db.Query("SELECT deliveryservice, data from url_sig_key")
+	if err != nil {
+		return fmt.Errorf("PGURLSigKey gatherKeys error creating query: %w", err)
+	}
+	defer rows.Close()
+	i := 0
+	for rows.Next() {
+		if i > len(tbl.Records)-1 {
+			return fmt.Errorf("PGURLSigKey gatherKeys: got more results than expected %v", len(tbl.Records))
+		}
+		err := rows.Scan(&tbl.Records[i].DeliveryService, &tbl.Records[i].DataEncrypted)
+		if err != nil {
+			return fmt.Errorf("PGURLSigKey gatherKeys: unable to scan row: %w", err)
+		}
+		i += 1
+	}
+	return nil
+}
+func (tbl *pgURLSigKeyTable) decrypt(aesKey []byte) error {
+	for i, sig := range tbl.Records {
+		err := decryptInto(aesKey, sig.DataEncrypted, &tbl.Records[i].Keys)
+		if err != nil {
+			return fmt.Errorf("unable to decrypt into keys: %w", err)
+		}
+	}
+	return nil
+}
+func (tbl *pgURLSigKeyTable) encrypt(aesKey []byte) error {
+	for i, sig := range tbl.Records {
+		data, err := json.Marshal(&sig.Keys)
+		if err != nil {
+			return fmt.Errorf("encrypt issue marshalling keys: %w", err)
+		}
+
+		dat, err := encrypt(data, aesKey)
+		if err != nil {
+			return fmt.Errorf("encrypt error: %w", err)
+		}
+		tbl.Records[i].DataEncrypted = dat
+	}
+	return nil
+}
+func (tbl *pgURLSigKeyTable) toGeneric() []URLSigKey {
+	keys := make([]URLSigKey, len(tbl.Records))
+
+	for i, record := range tbl.Records {
+		keys[i] = URLSigKey{
+			DeliveryService: record.DeliveryService,
+			URLSigKeys:      record.Keys,
+		}
+	}
+	return keys
+}
+func (tbl *pgURLSigKeyTable) fromGeneric(keys []URLSigKey) {
+	tbl.Records = make([]pgURLSigKeyRecord, len(keys))
+
+	for i, key := range keys {
+		tbl.Records[i] = pgURLSigKeyRecord{
+			Keys:            key.URLSigKeys,
+			DeliveryService: key.DeliveryService,
+			pgCommonRecord: pgCommonRecord{
+				DataEncrypted: nil,
+			},
+		}
+	}
+}
+func (tbl *pgURLSigKeyTable) validate() []string {
+	for i, record := range tbl.Records {
+		if record.DataEncrypted == nil && len(record.Keys) > 0 {
+			return []string{fmt.Sprintf("URl Sig Key %v: DataEncrypted is blank!", i)}
+		}
+	}
+	return nil
+}
+
+type pgURISignKeyRecord struct {
+	Keys            map[string]tc.URISignerKeyset
+	DeliveryService string
+	pgCommonRecord
+}
+type pgURISignKeyTable struct {
+	Records []pgURISignKeyRecord
+}
+
+func (tbl *pgURISignKeyTable) insertKeys(db *sql.DB) error {
+	queryBase := "INSERT INTO uri_signing_key (deliveryservice, data) VALUES "

Review comment:
       Oh, I see what you're saying. We could currently have multiple sslkeys at the same version in the DB, because `version` is not part of the primary key?




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [trafficcontrol] rawlinp commented on a change in pull request #5924: Add tool to migrate data between TV backends

Posted by GitBox <gi...@apache.org>.
rawlinp commented on a change in pull request #5924:
URL: https://github.com/apache/trafficcontrol/pull/5924#discussion_r651062385



##########
File path: tools/traffic_vault_migrate/postgres.go
##########
@@ -0,0 +1,743 @@
+package main
+
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+import (
+	"database/sql"
+	"encoding/base64"
+	"encoding/json"
+	"errors"
+	"fmt"
+	"log"
+	"strconv"
+	"strings"
+
+	"github.com/apache/trafficcontrol/lib/go-tc"
+	util "github.com/apache/trafficcontrol/lib/go-util"
+
+	_ "github.com/lib/pq"
+)
+
+// PGConfig represents the configuration options available to the PG backend
+type PGConfig struct {
+	Host     string `json:"host"`
+	Port     string `json:"port"`
+	User     string `json:"user"`
+	Password string `json:"password"`
+	SSLMode  string `json:"sslmode"`
+	Database string `json:"database"`
+	Key      string `json:"aesKey"`
+	AESKey   []byte
+}
+
+// PGBackend is the Postgres implementation of TVBackend
+type PGBackend struct {
+	sslKey pgSSLKeyTable
+	dnssec pgDNSSecTable
+	uri    pgURISignKeyTable
+	url    pgURLSigKeyTable
+	cfg    PGConfig
+	db     *sql.DB
+}
+
+// String returns a high level overview of the backend and its keys
+func (pg *PGBackend) String() string {
+	data := fmt.Sprintf("PG server %v@%v:%v\n", pg.cfg.User, pg.cfg.Host, pg.cfg.Port)
+	data += fmt.Sprintf("\tSSL Keys: %v\n", len(pg.sslKey.Records))
+	data += fmt.Sprintf("\tDNSSec Keys: %v\n", len(pg.dnssec.Records))
+	data += fmt.Sprintf("\tURI Keys: %v\n", len(pg.uri.Records))
+	data += fmt.Sprintf("\tURL Keys: %v\n", len(pg.url.Records))
+	return data
+}
+
+// Name returns the name for this backend
+func (pg *PGBackend) Name() string {
+	return "PG"
+}
+
+// ReadConfig takes in a filename and will read it into the backends config
+func (pg *PGBackend) ReadConfig(s string) error {
+	err := UnmarshalConfig(s, &pg.cfg)
+	if err != nil {
+		return err
+	}
+
+	pg.cfg.AESKey, err = base64.StdEncoding.DecodeString(pg.cfg.Key)
+	if err != nil {
+		return fmt.Errorf("unable to decode PG AESKey: %w", err)
+	}
+	return nil
+}
+
+// Insert takes the current keys and inserts them into the backend DB
+func (pg *PGBackend) Insert() error {
+	if err := pg.sslKey.insertKeys(pg.db); err != nil {
+		return err
+	}
+	if err := pg.dnssec.insertKeys(pg.db); err != nil {
+		return err
+	}
+	if err := pg.url.insertKeys(pg.db); err != nil {
+		return err
+	}
+	if err := pg.uri.insertKeys(pg.db); err != nil {
+		return err
+	}
+	return nil
+}
+
+// Start initiates the connection to the backend DB
+func (pg *PGBackend) Start() error {
+	sqlStr := fmt.Sprintf("postgres://%s:%s@%s:%s/%s?sslmode=%s", pg.cfg.User, pg.cfg.Password, pg.cfg.Host, pg.cfg.Port, pg.cfg.Database, pg.cfg.SSLMode)
+	db, err := sql.Open("postgres", sqlStr)
+	if err != nil {
+		return fmt.Errorf("unable to start PG client: %w", err)
+	}
+
+	pg.db = db
+	pg.sslKey = pgSSLKeyTable{}
+	pg.dnssec = pgDNSSecTable{}
+	pg.url = pgURLSigKeyTable{}
+	pg.uri = pgURISignKeyTable{}
+
+	return nil
+}
+
+// ValidateKey validates that the keys are valid (in most cases, certain fields are not null)
+func (pg *PGBackend) ValidateKey() []string {
+	var errors []string
+	if errs := pg.sslKey.validate(); errs != nil {
+		errors = append(errors, errs...)
+	}
+	if errs := pg.dnssec.validate(); errs != nil {
+		errors = append(errors, errs...)
+	}
+	if errs := pg.uri.validate(); errs != nil {
+		errors = append(errors, errs...)
+	}
+	if errs := pg.url.validate(); errs != nil {
+		errors = append(errors, errs...)
+	}
+	return errors
+}
+
+// Close terminates the connection to the backend DB
+func (pg *PGBackend) Close() error {
+	return pg.db.Close()
+}
+
+// Ping checks the connection to the backend DB
+func (pg *PGBackend) Ping() error {
+	return pg.db.Ping()
+}
+
+// Fetch gets all of the keys from the backend DB
+func (pg *PGBackend) Fetch() error {
+	if err := pg.sslKey.gatherKeys(pg.db); err != nil {
+		return err
+	}
+
+	if err := pg.dnssec.gatherKeys(pg.db); err != nil {
+		return err
+	}
+
+	if err := pg.url.gatherKeys(pg.db); err != nil {
+		return err
+	}
+
+	if err := pg.uri.gatherKeys(pg.db); err != nil {
+		return err
+	}
+
+	return nil
+}
+
+// GetSSLKeys converts the backends internal key representation into the common representation (SSLKey)
+func (pg *PGBackend) GetSSLKeys() ([]SSLKey, error) {
+	if err := pg.sslKey.decrypt(pg.cfg.AESKey); err != nil {
+		return nil, err
+	}
+	return pg.sslKey.toGeneric(), nil
+}
+
+// SetSSLKeys takes in keys and converts & encrypts the data into the backends internal format
+func (pg *PGBackend) SetSSLKeys(keys []SSLKey) error {
+	pg.sslKey.fromGeneric(keys)
+	return pg.sslKey.encrypt(pg.cfg.AESKey)
+}
+
+// GetDNSSecKeys converts the backends internal key representation into the common representation (DNSSecKey)
+func (pg *PGBackend) GetDNSSecKeys() ([]DNSSecKey, error) {
+	if err := pg.dnssec.decrypt(pg.cfg.AESKey); err != nil {
+		return nil, err
+	}
+	return pg.dnssec.toGeneric(), nil
+}
+
+// SetDNSSecKeys takes in keys and converts & encrypts the data into the backends internal format
+func (pg *PGBackend) SetDNSSecKeys(keys []DNSSecKey) error {
+	pg.dnssec.fromGeneric(keys)
+	return pg.dnssec.encrypt(pg.cfg.AESKey)
+}
+
+// GetURISignKeys converts the pg internal key representation into the common representation (URISignKey)
+func (pg *PGBackend) GetURISignKeys() ([]URISignKey, error) {
+	if err := pg.uri.decrypt(pg.cfg.AESKey); err != nil {
+		return nil, err
+	}
+	return pg.uri.toGeneric(), nil
+}
+
+// SetURISignKeys takes in keys and converts & encrypts the data into the backends internal format
+func (pg *PGBackend) SetURISignKeys(keys []URISignKey) error {
+	pg.uri.fromGeneric(keys)
+	return pg.uri.encrypt(pg.cfg.AESKey)
+}
+
+// GetURLSigKeys converts the backends internal key representation into the common representation (URLSigKey)
+func (pg *PGBackend) GetURLSigKeys() ([]URLSigKey, error) {
+	if err := pg.url.decrypt(pg.cfg.AESKey); err != nil {
+		return nil, err
+	}
+	return pg.url.toGeneric(), nil
+}
+
+// SetURLSigKeys takes in keys and converts & encrypts the data into the backends internal format
+func (pg *PGBackend) SetURLSigKeys(keys []URLSigKey) error {
+	pg.url.fromGeneric(keys)
+	return pg.url.encrypt(pg.cfg.AESKey)
+}
+
+type pgCommonRecord struct {
+	DataEncrypted []byte
+}
+
+type pgDNSSecRecord struct {
+	Key tc.DNSSECKeysTrafficVault
+	CDN string
+	pgCommonRecord
+}
+type pgDNSSecTable struct {
+	Records []pgDNSSecRecord
+}
+
+func (tbl *pgDNSSecTable) gatherKeys(db *sql.DB) error {
+	sz, err := getSize(db, "dnssec")
+	if err != nil {
+		log.Println("PGDNSSec gatherKeys: unable to determine size of dnssec table")
+	}
+	tbl.Records = make([]pgDNSSecRecord, sz)
+
+	rows, err := db.Query("SELECT cdn, data from dnssec")
+	if err != nil {
+		return fmt.Errorf("PGDNSSec gatherKeys: unable to query: %w", err)
+	}
+	defer rows.Close()
+	i := 0
+	for rows.Next() {
+		if i > len(tbl.Records)-1 {
+			return fmt.Errorf("PGDNSSec gatherKeys got more results than expected %v", len(tbl.Records))
+		}
+		err := rows.Scan(&tbl.Records[i].CDN, &tbl.Records[i].DataEncrypted)
+		if err != nil {
+			return fmt.Errorf("PGDNSSec gatherKeys unable to scan row: %w", err)
+		}
+		i += 1
+	}
+	return nil
+}
+func (tbl *pgDNSSecTable) decrypt(aesKey []byte) error {
+	for i, _ := range tbl.Records {
+		err := decryptInto(aesKey, tbl.Records[i].DataEncrypted, &tbl.Records[i].Key)
+		if err != nil {
+			return fmt.Errorf("unable to decrypt into keys: %w", err)
+		}
+	}
+	return nil
+}
+func (tbl *pgDNSSecTable) encrypt(aesKey []byte) error {
+	for i, dns := range tbl.Records {
+		data, err := json.Marshal(&dns.Key)
+		if err != nil {
+			return fmt.Errorf("encrypt issue marshalling keys: %w", err)
+		}
+		dat, err := encrypt(data, aesKey)
+		if err != nil {
+			return fmt.Errorf("encrypt error: %w", err)
+		}
+		tbl.Records[i].DataEncrypted = dat
+	}
+	return nil
+}
+func (tbl *pgDNSSecTable) toGeneric() []DNSSecKey {
+	keys := make([]DNSSecKey, len(tbl.Records))
+
+	for i, record := range tbl.Records {
+		keys[i] = DNSSecKey{
+			CDN:                    record.CDN,
+			DNSSECKeysTrafficVault: record.Key,
+		}
+	}
+
+	return keys
+}
+func (tbl *pgDNSSecTable) fromGeneric(keys []DNSSecKey) {
+	tbl.Records = make([]pgDNSSecRecord, len(keys))
+
+	for i, key := range keys {
+		tbl.Records[i] = pgDNSSecRecord{
+			Key: key.DNSSECKeysTrafficVault,
+			CDN: key.CDN,
+			pgCommonRecord: pgCommonRecord{
+				DataEncrypted: nil,
+			},
+		}
+	}
+}
+func (tbl *pgDNSSecTable) validate() []string {
+	for i, record := range tbl.Records {
+		if record.DataEncrypted == nil && len(record.Key) > 0 {
+			return []string{fmt.Sprintf("DNSSEC Key %v: DataEncrypted is blank!", i)}
+		}
+	}
+	return nil
+}
+func (tbl *pgDNSSecTable) insertKeys(db *sql.DB) error {
+	queryBase := "INSERT INTO dnssec (cdn, data) VALUES "
+	stride := 2
+	queryArgs := make([]interface{}, len(tbl.Records)*stride)
+	for i, record := range tbl.Records {
+		j := i * stride
+		queryArgs[j] = record.CDN
+		queryArgs[j+1] = record.DataEncrypted
+	}
+	return insertIntoTable(db, queryBase, stride, queryArgs)
+}
+
+type pgSSLKeyRecord struct {
+	Keys tc.DeliveryServiceSSLKeys
+	pgCommonRecord
+
+	// These records are stored on the table but are duplicated
+	DeliveryService string
+	CDN             string
+}
+type pgSSLKeyTable struct {
+	Records []pgSSLKeyRecord
+}
+
+func (tbl *pgSSLKeyTable) insertKeys(db *sql.DB) error {
+	queryBase := "INSERT INTO sslkey (deliveryservice, data, cdn, version) VALUES "
+	duplicateKeys := 2
+	stride := 4
+	queryArgs := make([]interface{}, len(tbl.Records)*stride*duplicateKeys)
+	for i, record := range tbl.Records {
+		j := i * duplicateKeys * stride
+
+		queryArgs[j] = record.DeliveryService
+		queryArgs[j+1] = record.DataEncrypted
+		queryArgs[j+2] = record.CDN
+		queryArgs[j+3] = record.Keys.Version.String()
+
+		queryArgs[j+4] = record.DeliveryService
+		queryArgs[j+5] = record.DataEncrypted
+		queryArgs[j+6] = record.CDN
+		queryArgs[j+7] = "latest"
+	}
+	return insertIntoTable(db, queryBase, 4, queryArgs)
+}
+func (tbl *pgSSLKeyTable) gatherKeys(db *sql.DB) error {
+	sz, err := getSize(db, "sslkey WHERE version='latest'")

Review comment:
       Right, if we only care to get the latest version of an sslkey, then getting `latest` is fine. But I think we might want to migrate all _prior_ versions too.
   
   For instance, let's say a DS is on ssl_key_version 5. So it currently has all these sslkey versions stored:
   - 0
   - 1
   - 2
   - 3
   - 4
   - 5
   - latest
   
   Versions `5` and `latest` are the same sslkey, but we might also want to migrate versions `0` through `4`.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [trafficcontrol] shamrickus commented on a change in pull request #5924: Add tool to migrate data between TV backends

Posted by GitBox <gi...@apache.org>.
shamrickus commented on a change in pull request #5924:
URL: https://github.com/apache/trafficcontrol/pull/5924#discussion_r652170775



##########
File path: tools/traffic_vault_migrate/riak.go
##########
@@ -0,0 +1,746 @@
+package main
+
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+import (
+	"crypto/tls"
+	"encoding/json"
+	"errors"
+	"fmt"
+	"time"
+
+	"github.com/apache/trafficcontrol/lib/go-rfc"
+	"github.com/apache/trafficcontrol/lib/go-tc"
+
+	"github.com/basho/riak-go-client"
+)
+
+const (
+	BUCKET_SSL     = "ssl"
+	BUCKET_DNSSEC  = "dnssec"
+	BUCKET_URL_SIG = "url_sig_keys"
+	BUCKET_URI_SIG = "cdn_uri_sig_keys"
+
+	INDEX_SSL = "sslkeys"
+
+	SCHEMA_RIAK_KEY    = "_yz_rk"
+	SCHEMA_RIAK_BUCKET = "_yz_rb"
+)
+
+var (
+	SCHEMA_SSL_FIELDS = [...]string{SCHEMA_RIAK_KEY, SCHEMA_RIAK_BUCKET}
+)
+
+// RiakConfig  represents the configuration options available to the Riak backend
+type RiakConfig struct {
+	Host          string `json:"host"`
+	Port          string `json:"port"`
+	User          string `json:"user"`
+	Password      string `json:"password"`
+	Insecure      bool   `json:"insecure"`
+	TLSVersionRaw string `json:"tlsVersion"`
+	TLSVersion    uint16
+}
+
+// RiakBackend is the Riak implementation of TVBackend
+type RiakBackend struct {
+	sslKeys     riakSSLKeyTable
+	dnssecKeys  riakDNSSecKeyTable
+	uriSignKeys riakURISignKeyTable
+	urlSigKeys  riakURLSigKeyTable
+	cfg         RiakConfig
+	cluster     *riak.Cluster
+}
+
+// String returns a high level overview of the backend and its keys
+func (rb *RiakBackend) String() string {
+	data := fmt.Sprintf("Riak server %v@%v:%v\n", rb.cfg.User, rb.cfg.Host, rb.cfg.Port)
+	data += fmt.Sprintf("\tSSL Keys: %v\n", len(rb.sslKeys.Records))
+	data += fmt.Sprintf("\tDNSSec Keys: %v\n", len(rb.dnssecKeys.Records))
+	data += fmt.Sprintf("\tURI Keys: %v\n", len(rb.uriSignKeys.Records))
+	data += fmt.Sprintf("\tURL Keys: %v\n", len(rb.urlSigKeys.Records))
+	return data
+}
+
+// Name returns the name for this backend
+func (rb *RiakBackend) Name() string {
+	return "Riak"
+}
+
+// ReadConfig takes in a filename and will read it into the backends config
+func (rb *RiakBackend) ReadConfig(s string) error {
+	err := UnmarshalConfig(s, &rb.cfg)
+	if err != nil {
+		return err
+	}
+
+	if rb.cfg.TLSVersionRaw == "10" {
+		rb.cfg.TLSVersion = tls.VersionTLS10
+	} else if rb.cfg.TLSVersionRaw == "11" {
+		rb.cfg.TLSVersion = tls.VersionTLS11
+	} else if rb.cfg.TLSVersionRaw == "12" {
+		rb.cfg.TLSVersion = tls.VersionTLS12
+	} else if rb.cfg.TLSVersionRaw == "13" {
+		rb.cfg.TLSVersion = tls.VersionTLS13
+	}
+	return nil
+}
+
+// Insert takes the current keys and inserts them into the backend DB
+func (rb *RiakBackend) Insert() error {
+	if err := rb.sslKeys.insertKeys(rb.cluster); err != nil {
+		return err
+	}
+	if err := rb.dnssecKeys.insertKeys(rb.cluster); err != nil {
+		return err
+	}
+	if err := rb.urlSigKeys.insertKeys(rb.cluster); err != nil {
+		return err
+	}
+	if err := rb.uriSignKeys.insertKeys(rb.cluster); err != nil {
+		return err
+	}
+	return nil
+}
+
+// ValidateKey validates that the keys are valid (in most cases, certain fields are not null)
+func (rb *RiakBackend) ValidateKey() []string {
+	errors := []string{}
+	if errs := rb.sslKeys.validate(); errs != nil {
+		errors = append(errors, errs...)
+	}
+	if errs := rb.dnssecKeys.validate(); errs != nil {
+		errors = append(errors, errs...)
+	}
+	if errs := rb.uriSignKeys.validate(); errs != nil {
+		errors = append(errors, errs...)
+	}
+	if errs := rb.urlSigKeys.validate(); errs != nil {
+		errors = append(errors, errs...)
+	}
+
+	return errors
+}
+
+// SetSSLKeys takes in keys and converts & encrypts the data into the backends internal format
+func (rb *RiakBackend) SetSSLKeys(keys []SSLKey) error {
+	rb.sslKeys.fromGeneric(keys)
+	return nil
+}
+
+// SetDNSSecKeys takes in keys and converts & encrypts the data into the backends internal format
+func (rb *RiakBackend) SetDNSSecKeys(keys []DNSSecKey) error {
+	rb.dnssecKeys.fromGeneric(keys)
+	return nil
+}
+
+// SetURISignKeys takes in keys and converts & encrypts the data into the backends internal format
+func (rb *RiakBackend) SetURISignKeys(keys []URISignKey) error {
+	rb.uriSignKeys.fromGeneric(keys)
+	return nil
+}
+
+// SetURLSigKeys takes in keys and converts & encrypts the data into the backends internal format
+func (rb *RiakBackend) SetURLSigKeys(keys []URLSigKey) error {
+	rb.urlSigKeys.fromGeneric(keys)
+	return nil
+}
+
+// Start initiates the connection to the backend DB
+func (rb *RiakBackend) Start() error {
+	tlsConfig := &tls.Config{
+		InsecureSkipVerify: !rb.cfg.Insecure,
+		MaxVersion:         rb.cfg.TLSVersion,
+	}
+	auth := &riak.AuthOptions{
+		User:      rb.cfg.User,
+		Password:  rb.cfg.Password,
+		TlsConfig: tlsConfig,
+	}
+
+	cluster, err := getRiakCluster(rb.cfg, auth)
+	if err != nil {
+		return err
+	}
+	if err := cluster.Start(); err != nil {
+		return fmt.Errorf("unable to start riak cluster: %w", err)
+	}
+
+	rb.cluster = cluster
+	rb.sslKeys = riakSSLKeyTable{}
+	rb.dnssecKeys = riakDNSSecKeyTable{}
+	rb.urlSigKeys = riakURLSigKeyTable{}
+	rb.uriSignKeys = riakURISignKeyTable{}
+	return nil
+}
+
+// Close terminates the connection to the backend DB
+func (rb *RiakBackend) Close() error {
+	if err := rb.cluster.Stop(); err != nil {
+		return err
+	}
+	return nil
+}
+
+// Ping checks the connection to the backend DB
+func (rb *RiakBackend) Ping() error {
+	return ping(rb.cluster)
+}
+
+// GetSSLKeys converts the backends internal key representation into the common representation (SSLKey)
+func (rb *RiakBackend) GetSSLKeys() ([]SSLKey, error) {
+	return rb.sslKeys.toGeneric(), nil
+}
+
+// GetDNSSecKeys converts the backends internal key representation into the common representation (DNSSecKey)
+func (rb *RiakBackend) GetDNSSecKeys() ([]DNSSecKey, error) {
+	return rb.dnssecKeys.toGeneric(), nil
+}
+
+// GetURISignKeys converts the pg internal key representation into the common representation (URISignKey)
+func (rb *RiakBackend) GetURISignKeys() ([]URISignKey, error) {
+	return rb.uriSignKeys.toGeneric(), nil
+}
+
+// GetURLSigKeys converts the backends internal key representation into the common representation (URLSigKey)
+func (rb *RiakBackend) GetURLSigKeys() ([]URLSigKey, error) {
+	return rb.urlSigKeys.toGeneric(), nil
+}
+
+// Fetch gets all of the keys from the backend DB
+func (rb *RiakBackend) Fetch() error {
+	if err := rb.sslKeys.gatherKeys(rb.cluster); err != nil {
+		return err
+	}
+	if err := rb.dnssecKeys.gatherKeys(rb.cluster); err != nil {
+		return err
+	}
+	if err := rb.urlSigKeys.gatherKeys(rb.cluster); err != nil {
+		return err
+	}
+	if err := rb.uriSignKeys.gatherKeys(rb.cluster); err != nil {
+		return err
+	}
+
+	return nil
+}
+
+type riakSSLKeyRecord struct {
+	tc.DeliveryServiceSSLKeys
+}
+type riakSSLKeyTable struct {
+	Records []riakSSLKeyRecord
+}
+
+func (tbl *riakSSLKeyTable) gatherKeys(cluster *riak.Cluster) error {
+	searchDocs, err := search(cluster, INDEX_SSL, "cdn:*", fmt.Sprintf("%v:*latest", SCHEMA_RIAK_KEY), 1000, SCHEMA_SSL_FIELDS[:])
+	if err != nil {
+		return fmt.Errorf("RiakSSLKey gatherKeys: %w", err)
+	}
+
+	tbl.Records = make([]riakSSLKeyRecord, len(searchDocs))
+	for i, doc := range searchDocs {
+		objs, err := getObject(cluster, doc.Bucket, doc.Key)
+		if err != nil {
+			return err
+		}
+		if len(objs) < 1 {
+			return fmt.Errorf("RiakSSLKey gatherKeys unable to find any objects with key %v and bucket %v, but search results were returned", doc.Key, doc.Bucket)
+		}
+		if len(objs) > 1 {
+			return fmt.Errorf("RiakSSLKey gatherKeys more than 1 ssl key record found %v\n", len(objs))
+		}
+		var obj tc.DeliveryServiceSSLKeys
+		if err = json.Unmarshal(objs[0].Value, &obj); err != nil {
+			return fmt.Errorf("RiakSSLKey gatherKeys unable to unmarshal object into tc.DeliveryServiceSSLKeys: %w", err)
+		}
+		tbl.Records[i] = riakSSLKeyRecord{
+			DeliveryServiceSSLKeys: obj,
+		}
+	}
+	return nil
+}
+func (tbl *riakSSLKeyTable) toGeneric() []SSLKey {
+	keys := make([]SSLKey, len(tbl.Records))
+
+	for i, record := range tbl.Records {
+		keys[i] = SSLKey{
+			DeliveryServiceSSLKeys: record.DeliveryServiceSSLKeys,
+		}
+	}
+
+	return keys
+}
+func (tbl *riakSSLKeyTable) fromGeneric(keys []SSLKey) {
+	tbl.Records = make([]riakSSLKeyRecord, len(keys))
+
+	for i, record := range keys {
+		tbl.Records[i] = riakSSLKeyRecord{
+			DeliveryServiceSSLKeys: record.DeliveryServiceSSLKeys,
+		}
+	}
+}
+func (tbl *riakSSLKeyTable) insertKeys(cluster *riak.Cluster) error {
+	for _, record := range tbl.Records {
+		objBytes, err := json.Marshal(record.DeliveryServiceSSLKeys)
+		if err != nil {
+			return fmt.Errorf("RiakSSLKey insertKeys failed to unmarshal keys: %w", err)
+		}
+		err = setObject(cluster, makeRiakObject(objBytes, record.DeliveryService+"-"+record.Version.String()), BUCKET_SSL)
+		if err != nil {
+			return fmt.Errorf("RiakSSLKey insertKeys: %w", err)
+		}
+		err = setObject(cluster, makeRiakObject(objBytes, record.DeliveryService+"-latest"), BUCKET_SSL)
+		if err != nil {
+			return fmt.Errorf("RiakSSLKey insertKeys: %w", err)
+		}
+	}
+	return nil
+}
+func (tbl *riakSSLKeyTable) validate() []string {
+	errs := []string{}
+	for i, record := range tbl.Records {
+		if record.DeliveryService == "" {
+			errs = append(errs, fmt.Sprintf("SSL Key #%v: Delivery Service is blank!", i))
+		}
+		if record.CDN == "" {
+			errs = append(errs, fmt.Sprintf("SSL Key #%v: CDN is blank!", i))
+		}
+		if record.Version.String() == "" {
+			errs = append(errs, fmt.Sprintf("SSL Key #%v: Version is blank!", i))
+		}
+	}
+	return errs
+}
+
+type riakDNSSecKeyRecord struct {
+	CDN string
+	Key tc.DNSSECKeysRiak
+}
+type riakDNSSecKeyTable struct {
+	Records []riakDNSSecKeyRecord
+}
+
+func (tbl *riakDNSSecKeyTable) gatherKeys(cluster *riak.Cluster) error {
+	tbl.Records = []riakDNSSecKeyRecord{}
+	objs, err := getObjects(cluster, BUCKET_DNSSEC)
+	if err != nil {
+		return fmt.Errorf("RiakDNSSecKey gatherKeys: %w", err)
+	}
+	for _, obj := range objs {
+		key := tc.DNSSECKeysRiak{}
+		if err := json.Unmarshal(obj.Value, &key); err != nil {
+			return fmt.Errorf("RiakDNSSecKey gatherKeys unable to unmarshal object to tc.DNSSECKeysRiak: %w", err)
+		}
+		tbl.Records = append(tbl.Records, riakDNSSecKeyRecord{
+			CDN: obj.Key,
+			Key: key,
+		})
+	}
+	return nil
+}
+func (tbl *riakDNSSecKeyTable) toGeneric() []DNSSecKey {
+	keys := make([]DNSSecKey, len(tbl.Records))
+
+	for i, record := range tbl.Records {
+		keys[i] = DNSSecKey{
+			CDN:                    record.CDN,
+			DNSSECKeysTrafficVault: tc.DNSSECKeysTrafficVault(record.Key),
+		}
+	}
+
+	return keys
+}
+func (tbl *riakDNSSecKeyTable) fromGeneric(keys []DNSSecKey) {
+	tbl.Records = make([]riakDNSSecKeyRecord, len(keys))
+
+	for i, record := range keys {
+		tbl.Records[i] = riakDNSSecKeyRecord{
+			CDN: record.CDN,
+			Key: tc.DNSSECKeysRiak(record.DNSSECKeysTrafficVault),
+		}
+	}
+}
+func (tbl *riakDNSSecKeyTable) insertKeys(cluster *riak.Cluster) error {
+	for _, record := range tbl.Records {
+		objBytes, err := json.Marshal(record.Key)
+		if err != nil {
+			return fmt.Errorf("RiakDNSSecKey insertKeys error marshalling keys: %w", err)
+		}
+
+		err = setObject(cluster, makeRiakObject(objBytes, record.CDN), BUCKET_DNSSEC)
+		if err != nil {
+			return fmt.Errorf("RiakDNSSecKey insertKeys: %w", err)
+		}
+	}
+	return nil
+}
+func (tbl *riakDNSSecKeyTable) validate() []string {
+	errs := []string{}
+	for i, record := range tbl.Records {
+		if record.CDN == "" {
+			errs = append(errs, fmt.Sprintf("DNSSec Key #%v: CDN is blank!", i))
+		}
+	}
+	return errs
+}
+
+type riakURLSigKeyRecord struct {
+	Key             tc.URLSigKeys
+	DeliveryService string
+}
+type riakURLSigKeyTable struct {
+	Records []riakURLSigKeyRecord
+}
+
+func (tbl *riakURLSigKeyTable) gatherKeys(cluster *riak.Cluster) error {
+	tbl.Records = []riakURLSigKeyRecord{}
+	objs, err := getObjects(cluster, BUCKET_URL_SIG)
+	if err != nil {
+		return fmt.Errorf("RiakURLSigKey gatherKeys: %w", err)
+	}
+	for _, obj := range objs {
+		key := tc.URLSigKeys{}
+		if err := json.Unmarshal(obj.Value, &key); err != nil {
+			return fmt.Errorf("RiakURLSigKey gatherKeys unable to unamrshal object into tc.URLSigKeys: %w", err)
+		}
+		tbl.Records = append(tbl.Records, riakURLSigKeyRecord{
+			DeliveryService: obj.Key,
+			Key:             key,
+		})
+	}
+	return nil
+}
+func (tbl *riakURLSigKeyTable) toGeneric() []URLSigKey {
+	keys := make([]URLSigKey, len(tbl.Records))
+
+	for i, record := range tbl.Records {
+		keys[i] = URLSigKey{
+			URLSigKeys:      record.Key,
+			DeliveryService: record.DeliveryService,
+		}
+	}
+
+	return keys
+}
+func (tbl *riakURLSigKeyTable) fromGeneric(keys []URLSigKey) {
+	tbl.Records = make([]riakURLSigKeyRecord, len(keys))
+	for i, key := range keys {
+		tbl.Records[i] = riakURLSigKeyRecord{
+			Key:             key.URLSigKeys,
+			DeliveryService: key.DeliveryService,
+		}
+	}
+}
+func (tbl *riakURLSigKeyTable) insertKeys(cluster *riak.Cluster) error {
+	for _, record := range tbl.Records {
+		objBytes, err := json.Marshal(record.Key)
+		if err != nil {
+			return fmt.Errorf("RiakURLSigKey insertKeys unable to marshal keys: %w", err)
+		}
+
+		err = setObject(cluster, makeRiakObject(objBytes, "url_sig_"+record.DeliveryService+".config"), BUCKET_URL_SIG)
+		if err != nil {
+			return fmt.Errorf("RiakURLSigKey insertKeys: %w", err)
+		}
+	}
+	return nil
+}
+func (tbl *riakURLSigKeyTable) validate() []string {
+	errs := []string{}
+	for i, record := range tbl.Records {
+		if record.DeliveryService == "" {
+			errs = append(errs, fmt.Sprintf("URL Key #%v: Delivery Service is blank!", i))
+		}
+	}
+	return errs
+}
+
+type riakURISignKeyRecord struct {
+	Key             map[string]tc.URISignerKeyset
+	DeliveryService string
+}
+type riakURISignKeyTable struct {
+	Records []riakURISignKeyRecord
+}
+
+func (tbl *riakURISignKeyTable) gatherKeys(cluster *riak.Cluster) error {
+	tbl.Records = []riakURISignKeyRecord{}
+	objs, err := getObjects(cluster, BUCKET_URI_SIG)
+	if err != nil {
+		return fmt.Errorf("RiakURISignKey gatherKeys: %w", err)
+	}
+	for _, obj := range objs {
+		key := map[string]tc.URISignerKeyset{}
+		if err := json.Unmarshal(obj.Value, &key); err != nil {
+			return fmt.Errorf("RiakURISignKey gatherKeys unable to unmarshal object into map[string]tc.URISignerKeySet: %w", err)
+		}
+
+		tbl.Records = append(tbl.Records, riakURISignKeyRecord{
+			DeliveryService: obj.Key,
+			Key:             key,
+		})
+	}
+	return nil
+}
+func (tbl *riakURISignKeyTable) toGeneric() []URISignKey {
+	keys := make([]URISignKey, len(tbl.Records))
+
+	for i, record := range tbl.Records {
+		keys[i] = URISignKey{
+			DeliveryService: record.DeliveryService,
+			Keys:            record.Key,
+		}
+	}
+
+	return keys
+}
+func (tbl *riakURISignKeyTable) fromGeneric(keys []URISignKey) {
+	tbl.Records = make([]riakURISignKeyRecord, len(keys))
+
+	for i, record := range keys {
+		tbl.Records[i] = riakURISignKeyRecord{
+			Key:             record.Keys,
+			DeliveryService: record.DeliveryService,
+		}
+	}
+}
+func (tbl *riakURISignKeyTable) insertKeys(cluster *riak.Cluster) error {
+	for _, record := range tbl.Records {
+		objBytes, err := json.Marshal(record.Key)
+		if err != nil {
+			return fmt.Errorf("RiakURISignKey insertKeys: unable to marshal key: %w", err)
+		}
+
+		err = setObject(cluster, makeRiakObject(objBytes, record.DeliveryService), BUCKET_URI_SIG)
+		if err != nil {
+			return fmt.Errorf("RiakURISignKey insertKeys: %w", err)
+		}
+	}
+	return nil
+}
+func (tbl *riakURISignKeyTable) validate() []string {
+	errs := []string{}
+	for i, record := range tbl.Records {
+		if record.DeliveryService == "" {
+			errs = append(errs, fmt.Sprintf("URI Key #%v: Delivery Service is blank!", i))
+		}
+	}
+	return errs
+}
+
+// Riak functions
+func makeRiakObject(data []byte, key string) *riak.Object {
+	return &riak.Object{
+		ContentType:     rfc.ApplicationJSON,
+		Charset:         "utf-8",
+		ContentEncoding: "utf-8",
+		Key:             key,
+		Value:           data,
+	}
+}
+func getObjects(cluster *riak.Cluster, bucket string) ([]*riak.Object, error) {
+	objs := []*riak.Object{}
+	keys, err := listKeys(cluster, bucket)
+	if err != nil {
+		return nil, err
+	}
+	for _, key := range keys {
+		objects, err := getObject(cluster, bucket, key)
+		if err != nil {
+			return nil, err
+		}
+		if len(objects) > 1 {
+			return nil, fmt.Errorf("Unexpected number of objects %v, ignoring\n", len(objects))
+		}
+
+		objs = append(objs, objects[0])
+	}
+
+	return objs, nil
+}
+func getObject(cluster *riak.Cluster, bucket string, key string) ([]*riak.Object, error) {
+	cmd, err := riak.NewFetchValueCommandBuilder().
+		WithBucket(bucket).
+		WithKey(key).
+		WithTimeout(time.Second * 60).
+		Build()
+	if err != nil {
+		return nil, fmt.Errorf("error building riak fetch value command: %w", err)
+	}
+
+	if err := cluster.Execute(cmd); err != nil {
+		return nil, fmt.Errorf("error executing riak fetch value command: %w", err)
+	}
+
+	fvc := cmd.(*riak.FetchValueCommand)
+	rsp := fvc.Response
+
+	if rsp.IsNotFound {
+		return nil, fmt.Errorf("errors executing riak fetch value command; key not found: %v:%v", bucket, key)
+	}
+
+	return rsp.Values, nil
+}
+func setObject(cluster *riak.Cluster, obj *riak.Object, bucket string) error {
+	cmd, err := riak.NewStoreValueCommandBuilder().
+		WithBucket(bucket).
+		WithContent(obj).
+		WithTimeout(time.Second * 5).
+		Build()
+	if err != nil {
+		return fmt.Errorf("error building riak store value command: %w", err)
+	}
+
+	return cluster.Execute(cmd)
+}
+func search(cluster *riak.Cluster, index string, query string, filterQuery string, numRows uint32, fields []string) ([]*riak.SearchDoc, error) {
+	var searchDocs []*riak.SearchDoc
+	start := uint32(0)
+	for i := uint32(0); ; i += 1 {
+		riakCmd := riak.NewSearchCommandBuilder().
+			WithQuery(query).
+			WithNumRows(numRows).
+			WithStart(start)
+		if len(index) > 0 {
+			riakCmd = riakCmd.WithIndexName(index)
+		}
+		if len(filterQuery) > 0 {
+			riakCmd = riakCmd.WithFilterQuery(filterQuery)
+		}
+		if len(fields) > 0 {
+			riakCmd = riakCmd.WithReturnFields(fields...)
+		}
+		iCmd, err := riakCmd.Build()
+
+		if err != nil {
+			return nil, fmt.Errorf("building Riak search command: %w", err)
+		}
+		if err = cluster.Execute(iCmd); err != nil {
+			return nil, fmt.Errorf("executing Riak search command index '%v' query '%v': %w", index, query, err)
+		}
+		cmd, ok := iCmd.(*riak.SearchCommand)
+		if !ok {
+			return nil, fmt.Errorf("riak search command unexpected type %T", iCmd)
+		}
+		if start == 0 {
+			if cmd.Response == nil || cmd.Response.NumFound == 0 {
+				return nil, nil
+			}
+			if cmd.Response.NumFound <= numRows {
+				return cmd.Response.Docs, nil
+			} else if numRows < cmd.Response.NumFound {
+				searchDocs = make([]*riak.SearchDoc, cmd.Response.NumFound)
+			}
+			if cmd.Response.NumFound > 10000 {

Review comment:
       Set it to `numRows*10`




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [trafficcontrol] ocket8888 commented on a change in pull request #5924: Add tool to migrate data between TV backends

Posted by GitBox <gi...@apache.org>.
ocket8888 commented on a change in pull request #5924:
URL: https://github.com/apache/trafficcontrol/pull/5924#discussion_r648321337



##########
File path: tools/traffic_vault_migrate/postgres.go
##########
@@ -0,0 +1,716 @@
+package main
+
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+import (
+	"database/sql"
+	"encoding/base64"
+	"encoding/json"
+	"errors"
+	"fmt"
+	"github.com/apache/trafficcontrol/lib/go-tc"
+	util "github.com/apache/trafficcontrol/lib/go-util"
+	_ "github.com/lib/pq"
+	"log"
+	"reflect"
+	"strconv"
+	"strings"
+)
+
+type PGConfig struct {

Review comment:
       GoDoc?

##########
File path: docs/source/tools/traffic_vault_migrate.rst
##########
@@ -0,0 +1,107 @@
+..
+..
+.. Licensed under the Apache License, Version 2.0 (the "License");
+.. you may not use this file except in compliance with the License.
+.. You may obtain a copy of the License at
+..
+..     http://www.apache.org/licenses/LICENSE-2.0
+..
+.. Unless required by applicable law or agreed to in writing, software
+.. distributed under the License is distributed on an "AS IS" BASIS,
+.. WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+.. See the License for the specific language governing permissions and
+.. limitations under the License.
+..
+
+.. _traffic_vault_migrate:
+
+#########################
+Traffic Vault Migrate
+#########################
+The ``traffic_vault_migrate`` tool - located at :file:`tools/traffic_vault_migrate/traffic_vault_migrate.go` in the `Apache Traffic Control repository <https://github.com/apache/trafficcontrol>`_ -
+is used to transfer TV keys between database servers. It interfaces directly with each backend so Traffic Ops/Vault being available is not a requirement.
+The tool assumes that the schema for each backend is already setup as according to the :ref:`admin setup <traffic_vault_admin>`.
+
+.. program:: traffic_vault_migrate
+
+Usage
+===========
+``traffic_vault_migrate -from_cfg CFG -to_cfg CFG -from_type TYP -to_type TYP [-confirm] [-compare] [-dry] [-dump]``

Review comment:
       `-h`/`--help` doesn't appear to be documented

##########
File path: tools/traffic_vault_migrate/postgres.go
##########
@@ -0,0 +1,716 @@
+package main
+
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+import (
+	"database/sql"
+	"encoding/base64"
+	"encoding/json"
+	"errors"
+	"fmt"
+	"github.com/apache/trafficcontrol/lib/go-tc"
+	util "github.com/apache/trafficcontrol/lib/go-util"
+	_ "github.com/lib/pq"
+	"log"
+	"reflect"
+	"strconv"
+	"strings"
+)
+
+type PGConfig struct {
+	Host     string `json:"host"`
+	Port     string `json:"port"`
+	User     string `json:"user"`
+	Password string `json:"password"`
+	SSLMode  string `json:"sslmode"`
+	Database string `json:"database"`
+	Key      string `json:"aesKey"`
+	AESKey   []byte
+}
+type PGBackend struct {
+	sslKey PGSSLKeyTable
+	dnssec PGDNSSecTable
+	uri    PGURISignKeyTable
+	url    PGURLSigKeyTable
+	cfg    PGConfig
+	db     *sql.DB
+}
+
+func (pg *PGBackend) String() string {
+	data := fmt.Sprintf("PG server %v@%v:%v\n", pg.cfg.User, pg.cfg.Host, pg.cfg.Port)
+	data += fmt.Sprintf("\tSSL Keys: %v\n", len(pg.sslKey.Records))
+	data += fmt.Sprintf("\tDNSSec Keys: %v\n", len(pg.dnssec.Records))
+	data += fmt.Sprintf("\tURI Keys: %v\n", len(pg.uri.Records))
+	data += fmt.Sprintf("\tURL Keys: %v\n", len(pg.url.Records))
+	return data
+}
+func (pg *PGBackend) Name() string {
+	return "PG"
+}
+func (pg *PGBackend) ReadConfig(s string) error {
+	genericCfg, err := UnmarshalConfig(s, reflect.TypeOf(pg.cfg))
+	if err != nil {
+		return err
+	}
+
+	pg.cfg = *genericCfg.Interface().(*PGConfig)
+
+	pg.cfg.AESKey, err = base64.StdEncoding.DecodeString(pg.cfg.Key)
+	if err != nil {
+		return fmt.Errorf("unable to decode PG AESKey: %v", err)
+	}
+	return nil
+}
+func (pg *PGBackend) Insert() error {
+	if err := pg.sslKey.insertKeys(pg.db); err != nil {
+		return err
+	}
+	if err := pg.dnssec.insertKeys(pg.db); err != nil {
+		return err
+	}
+	if err := pg.url.insertKeys(pg.db); err != nil {
+		return err
+	}
+	if err := pg.uri.insertKeys(pg.db); err != nil {
+		return err
+	}
+	return nil
+}
+func (pg *PGBackend) Start() error {
+	sqlStr := fmt.Sprintf("postgres://%s:%s@%s:%s/%s?sslmode=%s", pg.cfg.User, pg.cfg.Password, pg.cfg.Host, pg.cfg.Port, pg.cfg.Database, pg.cfg.SSLMode)
+	db, err := sql.Open("postgres", sqlStr)
+	if err != nil {
+		return fmt.Errorf("Unable to start PG client: %v", err)

Review comment:
       `fmt.Errorf` format arg for errors should use `%w` to wrap errors

##########
File path: tools/traffic_vault_migrate/postgres.go
##########
@@ -0,0 +1,716 @@
+package main
+
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+import (
+	"database/sql"
+	"encoding/base64"
+	"encoding/json"
+	"errors"
+	"fmt"
+	"github.com/apache/trafficcontrol/lib/go-tc"
+	util "github.com/apache/trafficcontrol/lib/go-util"
+	_ "github.com/lib/pq"
+	"log"
+	"reflect"
+	"strconv"
+	"strings"
+)
+
+type PGConfig struct {
+	Host     string `json:"host"`
+	Port     string `json:"port"`
+	User     string `json:"user"`
+	Password string `json:"password"`
+	SSLMode  string `json:"sslmode"`
+	Database string `json:"database"`
+	Key      string `json:"aesKey"`
+	AESKey   []byte
+}
+type PGBackend struct {
+	sslKey PGSSLKeyTable
+	dnssec PGDNSSecTable
+	uri    PGURISignKeyTable
+	url    PGURLSigKeyTable
+	cfg    PGConfig
+	db     *sql.DB
+}
+
+func (pg *PGBackend) String() string {
+	data := fmt.Sprintf("PG server %v@%v:%v\n", pg.cfg.User, pg.cfg.Host, pg.cfg.Port)
+	data += fmt.Sprintf("\tSSL Keys: %v\n", len(pg.sslKey.Records))
+	data += fmt.Sprintf("\tDNSSec Keys: %v\n", len(pg.dnssec.Records))
+	data += fmt.Sprintf("\tURI Keys: %v\n", len(pg.uri.Records))
+	data += fmt.Sprintf("\tURL Keys: %v\n", len(pg.url.Records))
+	return data
+}
+func (pg *PGBackend) Name() string {
+	return "PG"
+}
+func (pg *PGBackend) ReadConfig(s string) error {
+	genericCfg, err := UnmarshalConfig(s, reflect.TypeOf(pg.cfg))
+	if err != nil {
+		return err
+	}
+
+	pg.cfg = *genericCfg.Interface().(*PGConfig)
+
+	pg.cfg.AESKey, err = base64.StdEncoding.DecodeString(pg.cfg.Key)
+	if err != nil {
+		return fmt.Errorf("unable to decode PG AESKey: %v", err)
+	}
+	return nil
+}
+func (pg *PGBackend) Insert() error {
+	if err := pg.sslKey.insertKeys(pg.db); err != nil {
+		return err
+	}
+	if err := pg.dnssec.insertKeys(pg.db); err != nil {
+		return err
+	}
+	if err := pg.url.insertKeys(pg.db); err != nil {
+		return err
+	}
+	if err := pg.uri.insertKeys(pg.db); err != nil {
+		return err
+	}
+	return nil
+}
+func (pg *PGBackend) Start() error {
+	sqlStr := fmt.Sprintf("postgres://%s:%s@%s:%s/%s?sslmode=%s", pg.cfg.User, pg.cfg.Password, pg.cfg.Host, pg.cfg.Port, pg.cfg.Database, pg.cfg.SSLMode)
+	db, err := sql.Open("postgres", sqlStr)
+	if err != nil {
+		return fmt.Errorf("Unable to start PG client: %v", err)
+	}
+
+	pg.db = db
+	pg.sslKey = PGSSLKeyTable{}
+	pg.dnssec = PGDNSSecTable{}
+	pg.url = PGURLSigKeyTable{}
+	pg.uri = PGURISignKeyTable{}
+
+	return nil
+}
+func (pg *PGBackend) ValidateKey() []string {

Review comment:
       GoDoc?

##########
File path: tools/traffic_vault_migrate/postgres.go
##########
@@ -0,0 +1,716 @@
+package main
+
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+import (
+	"database/sql"
+	"encoding/base64"
+	"encoding/json"
+	"errors"
+	"fmt"
+	"github.com/apache/trafficcontrol/lib/go-tc"
+	util "github.com/apache/trafficcontrol/lib/go-util"
+	_ "github.com/lib/pq"
+	"log"
+	"reflect"
+	"strconv"
+	"strings"
+)
+
+type PGConfig struct {
+	Host     string `json:"host"`
+	Port     string `json:"port"`
+	User     string `json:"user"`
+	Password string `json:"password"`
+	SSLMode  string `json:"sslmode"`
+	Database string `json:"database"`
+	Key      string `json:"aesKey"`
+	AESKey   []byte
+}
+type PGBackend struct {
+	sslKey PGSSLKeyTable
+	dnssec PGDNSSecTable
+	uri    PGURISignKeyTable
+	url    PGURLSigKeyTable
+	cfg    PGConfig
+	db     *sql.DB
+}
+
+func (pg *PGBackend) String() string {
+	data := fmt.Sprintf("PG server %v@%v:%v\n", pg.cfg.User, pg.cfg.Host, pg.cfg.Port)
+	data += fmt.Sprintf("\tSSL Keys: %v\n", len(pg.sslKey.Records))
+	data += fmt.Sprintf("\tDNSSec Keys: %v\n", len(pg.dnssec.Records))
+	data += fmt.Sprintf("\tURI Keys: %v\n", len(pg.uri.Records))
+	data += fmt.Sprintf("\tURL Keys: %v\n", len(pg.url.Records))
+	return data
+}
+func (pg *PGBackend) Name() string {
+	return "PG"
+}
+func (pg *PGBackend) ReadConfig(s string) error {
+	genericCfg, err := UnmarshalConfig(s, reflect.TypeOf(pg.cfg))
+	if err != nil {
+		return err
+	}
+
+	pg.cfg = *genericCfg.Interface().(*PGConfig)
+
+	pg.cfg.AESKey, err = base64.StdEncoding.DecodeString(pg.cfg.Key)
+	if err != nil {
+		return fmt.Errorf("unable to decode PG AESKey: %v", err)
+	}
+	return nil
+}
+func (pg *PGBackend) Insert() error {
+	if err := pg.sslKey.insertKeys(pg.db); err != nil {
+		return err
+	}
+	if err := pg.dnssec.insertKeys(pg.db); err != nil {
+		return err
+	}
+	if err := pg.url.insertKeys(pg.db); err != nil {
+		return err
+	}
+	if err := pg.uri.insertKeys(pg.db); err != nil {
+		return err
+	}
+	return nil
+}
+func (pg *PGBackend) Start() error {
+	sqlStr := fmt.Sprintf("postgres://%s:%s@%s:%s/%s?sslmode=%s", pg.cfg.User, pg.cfg.Password, pg.cfg.Host, pg.cfg.Port, pg.cfg.Database, pg.cfg.SSLMode)
+	db, err := sql.Open("postgres", sqlStr)
+	if err != nil {
+		return fmt.Errorf("Unable to start PG client: %v", err)
+	}
+
+	pg.db = db
+	pg.sslKey = PGSSLKeyTable{}
+	pg.dnssec = PGDNSSecTable{}
+	pg.url = PGURLSigKeyTable{}
+	pg.uri = PGURISignKeyTable{}
+
+	return nil
+}
+func (pg *PGBackend) ValidateKey() []string {
+	var errors []string
+	if errs := pg.sslKey.validate(); errs != nil {
+		errors = append(errors, errs...)
+	}
+	if errs := pg.dnssec.validate(); errs != nil {
+		errors = append(errors, errs...)
+	}
+	if errs := pg.uri.validate(); errs != nil {
+		errors = append(errors, errs...)
+	}
+	if errs := pg.url.validate(); errs != nil {
+		errors = append(errors, errs...)
+	}
+	return errors
+}
+func (pg *PGBackend) Stop() error {
+	return pg.db.Close()
+}
+func (pg *PGBackend) Ping() error {
+	return pg.db.Ping()
+}
+func (pg *PGBackend) Fetch() error {
+	if err := pg.sslKey.gatherKeys(pg.db); err != nil {
+		return err
+	}
+
+	if err := pg.dnssec.gatherKeys(pg.db); err != nil {
+		return err
+	}
+
+	if err := pg.url.gatherKeys(pg.db); err != nil {
+		return err
+	}
+
+	if err := pg.uri.gatherKeys(pg.db); err != nil {
+		return err
+	}
+
+	return nil
+}
+func (pg *PGBackend) GetSSLKeys() ([]SSLKey, error) {
+	if err := pg.sslKey.decrypt(pg.cfg.AESKey); err != nil {
+		return nil, err
+	}
+	return pg.sslKey.toGeneric(), nil
+}
+func (pg *PGBackend) SetSSLKeys(keys []SSLKey) error {
+	pg.sslKey.fromGeneric(keys)
+	return pg.sslKey.encrypt(pg.cfg.AESKey)
+}
+func (pg *PGBackend) GetDNSSecKeys() ([]DNSSecKey, error) {
+	if err := pg.dnssec.decrypt(pg.cfg.AESKey); err != nil {
+		return nil, err
+	}
+	return pg.dnssec.toGeneric(), nil
+}
+func (pg *PGBackend) SetDNSSecKeys(keys []DNSSecKey) error {
+	pg.dnssec.fromGeneric(keys)
+	return pg.dnssec.encrypt(pg.cfg.AESKey)
+}
+func (pg *PGBackend) GetURISignKeys() ([]URISignKey, error) {
+	if err := pg.uri.decrypt(pg.cfg.AESKey); err != nil {
+		return nil, err
+	}
+	return pg.uri.toGeneric(), nil
+}
+func (pg *PGBackend) SetURISignKeys(keys []URISignKey) error {
+	pg.uri.fromGeneric(keys)
+	return pg.uri.encrypt(pg.cfg.AESKey)
+}
+func (pg *PGBackend) GetURLSigKeys() ([]URLSigKey, error) {
+	if err := pg.url.decrypt(pg.cfg.AESKey); err != nil {
+		return nil, err
+	}
+	return pg.url.toGeneric(), nil
+}
+func (pg *PGBackend) SetURLSigKeys(keys []URLSigKey) error {
+	pg.url.fromGeneric(keys)
+	return pg.url.encrypt(pg.cfg.AESKey)
+}
+
+type PGCommonRecord struct {
+	DataEncrypted []byte
+	CommonRecord
+}
+
+type PGDNSSecRecord struct {
+	Key tc.DNSSECKeysTrafficVault
+	CDN string
+	PGCommonRecord
+}
+type PGDNSSecTable struct {
+	Records []PGDNSSecRecord
+}
+
+func (tbl *PGDNSSecTable) gatherKeys(db *sql.DB) error {
+	sz, err := getSize(db, "dnssec")
+	if err != nil {
+		log.Println("PGDNSSec gatherKeys: unable to determine size of dnssec table")
+	}
+	tbl.Records = make([]PGDNSSecRecord, sz)
+
+	rows, err := db.Query("SELECT cdn, data from dnssec")
+	if err != nil {
+		return fmt.Errorf("PGDNSSec gatherKeys: unable to query: %v", err)
+	}
+	defer rows.Close()
+	i := 0
+	for rows.Next() {
+		if i > len(tbl.Records)-1 {
+			return fmt.Errorf("PGDNSSec gatherKeys got more results than expected %v", len(tbl.Records))
+		}
+		err := rows.Scan(&tbl.Records[i].CDN, &tbl.Records[i].DataEncrypted)
+		if err != nil {
+			return fmt.Errorf("PGDNSSec gatherKeys unable to scan row: %v", err)
+		}
+		i += 1
+	}
+	return nil
+}
+func (tbl *PGDNSSecTable) decrypt(aesKey []byte) error {
+	for i, _ := range tbl.Records {
+		data, err := decryptInto(aesKey, tbl.Records[i].DataEncrypted, reflect.TypeOf(tbl.Records[i].Key))
+		if err != nil {
+			return fmt.Errorf("unable to decrypt into keys: %v", err)
+		}
+		tbl.Records[i].Key = *data.Interface().(*tc.DNSSECKeysTrafficVault)
+	}
+	return nil
+}
+func (tbl *PGDNSSecTable) encrypt(aesKey []byte) error {
+	for i, dns := range tbl.Records {
+		data, err := json.Marshal(&dns.Key)
+		if err != nil {
+			return fmt.Errorf("encrypt issue marshalling keys: %v", err)
+		}
+		dat, err := encrypt(data, aesKey)
+		if err != nil {
+			return fmt.Errorf("encrypt error: %v", err)
+		}
+		tbl.Records[i].DataEncrypted = dat
+	}
+	return nil
+}
+func (tbl *PGDNSSecTable) toGeneric() []DNSSecKey {
+	keys := make([]DNSSecKey, len(tbl.Records))
+
+	for i, record := range tbl.Records {
+		keys[i] = DNSSecKey{
+			CDN:                    record.CDN,
+			DNSSECKeysTrafficVault: record.Key,
+			CommonRecord:           record.CommonRecord,
+		}
+	}
+
+	return keys
+}
+func (tbl *PGDNSSecTable) fromGeneric(keys []DNSSecKey) {
+	tbl.Records = make([]PGDNSSecRecord, len(keys))
+
+	for i, key := range keys {
+		tbl.Records[i] = PGDNSSecRecord{
+			Key: key.DNSSECKeysTrafficVault,
+			CDN: key.CDN,
+			PGCommonRecord: PGCommonRecord{
+				DataEncrypted: nil,
+				CommonRecord:  CommonRecord{},
+			},
+		}
+	}
+}
+func (tbl *PGDNSSecTable) validate() []string {
+	for i, record := range tbl.Records {
+		if record.DataEncrypted == nil && len(record.Key) > 0 {
+			return []string{fmt.Sprintf("DNSSEC Key %v: DataEncrypted is blank!", i)}
+		}
+	}
+	return nil
+}
+func (tbl *PGDNSSecTable) insertKeys(db *sql.DB) error {
+	queryFmt := "INSERT INTO dnssec (cdn, data) VALUES "
+	stride := 2
+	queryArgs := make([]interface{}, len(tbl.Records)*stride)
+	for i, record := range tbl.Records {
+		j := i * stride
+		queryArgs[j] = record.CDN
+		queryArgs[j+1] = record.DataEncrypted
+	}
+	return insertIntoTable(db, queryFmt, stride, queryArgs)
+}
+
+type PGSSLKeyRecord struct {
+	Keys tc.DeliveryServiceSSLKeys
+	PGCommonRecord
+
+	// These records are stored on the table but are duplicated
+	DeliveryService string
+	Version         string
+	CDN             string
+}
+type PGSSLKeyTable struct {
+	Records []PGSSLKeyRecord
+}

Review comment:
       GoDoc?

##########
File path: tools/traffic_vault_migrate/postgres.go
##########
@@ -0,0 +1,716 @@
+package main
+
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+import (
+	"database/sql"
+	"encoding/base64"
+	"encoding/json"
+	"errors"
+	"fmt"
+	"github.com/apache/trafficcontrol/lib/go-tc"
+	util "github.com/apache/trafficcontrol/lib/go-util"
+	_ "github.com/lib/pq"
+	"log"
+	"reflect"
+	"strconv"
+	"strings"
+)
+
+type PGConfig struct {
+	Host     string `json:"host"`
+	Port     string `json:"port"`
+	User     string `json:"user"`
+	Password string `json:"password"`
+	SSLMode  string `json:"sslmode"`
+	Database string `json:"database"`
+	Key      string `json:"aesKey"`
+	AESKey   []byte
+}
+type PGBackend struct {
+	sslKey PGSSLKeyTable
+	dnssec PGDNSSecTable
+	uri    PGURISignKeyTable
+	url    PGURLSigKeyTable
+	cfg    PGConfig
+	db     *sql.DB
+}
+
+func (pg *PGBackend) String() string {
+	data := fmt.Sprintf("PG server %v@%v:%v\n", pg.cfg.User, pg.cfg.Host, pg.cfg.Port)
+	data += fmt.Sprintf("\tSSL Keys: %v\n", len(pg.sslKey.Records))
+	data += fmt.Sprintf("\tDNSSec Keys: %v\n", len(pg.dnssec.Records))
+	data += fmt.Sprintf("\tURI Keys: %v\n", len(pg.uri.Records))
+	data += fmt.Sprintf("\tURL Keys: %v\n", len(pg.url.Records))
+	return data
+}
+func (pg *PGBackend) Name() string {
+	return "PG"
+}
+func (pg *PGBackend) ReadConfig(s string) error {
+	genericCfg, err := UnmarshalConfig(s, reflect.TypeOf(pg.cfg))
+	if err != nil {
+		return err
+	}
+
+	pg.cfg = *genericCfg.Interface().(*PGConfig)
+
+	pg.cfg.AESKey, err = base64.StdEncoding.DecodeString(pg.cfg.Key)
+	if err != nil {
+		return fmt.Errorf("unable to decode PG AESKey: %v", err)
+	}
+	return nil
+}
+func (pg *PGBackend) Insert() error {
+	if err := pg.sslKey.insertKeys(pg.db); err != nil {
+		return err
+	}
+	if err := pg.dnssec.insertKeys(pg.db); err != nil {
+		return err
+	}
+	if err := pg.url.insertKeys(pg.db); err != nil {
+		return err
+	}
+	if err := pg.uri.insertKeys(pg.db); err != nil {
+		return err
+	}
+	return nil
+}
+func (pg *PGBackend) Start() error {
+	sqlStr := fmt.Sprintf("postgres://%s:%s@%s:%s/%s?sslmode=%s", pg.cfg.User, pg.cfg.Password, pg.cfg.Host, pg.cfg.Port, pg.cfg.Database, pg.cfg.SSLMode)
+	db, err := sql.Open("postgres", sqlStr)
+	if err != nil {
+		return fmt.Errorf("Unable to start PG client: %v", err)
+	}
+
+	pg.db = db
+	pg.sslKey = PGSSLKeyTable{}
+	pg.dnssec = PGDNSSecTable{}
+	pg.url = PGURLSigKeyTable{}
+	pg.uri = PGURISignKeyTable{}
+
+	return nil
+}
+func (pg *PGBackend) ValidateKey() []string {
+	var errors []string
+	if errs := pg.sslKey.validate(); errs != nil {
+		errors = append(errors, errs...)
+	}
+	if errs := pg.dnssec.validate(); errs != nil {
+		errors = append(errors, errs...)
+	}
+	if errs := pg.uri.validate(); errs != nil {
+		errors = append(errors, errs...)
+	}
+	if errs := pg.url.validate(); errs != nil {
+		errors = append(errors, errs...)
+	}
+	return errors
+}
+func (pg *PGBackend) Stop() error {
+	return pg.db.Close()
+}
+func (pg *PGBackend) Ping() error {
+	return pg.db.Ping()
+}
+func (pg *PGBackend) Fetch() error {
+	if err := pg.sslKey.gatherKeys(pg.db); err != nil {
+		return err
+	}
+
+	if err := pg.dnssec.gatherKeys(pg.db); err != nil {
+		return err
+	}
+
+	if err := pg.url.gatherKeys(pg.db); err != nil {
+		return err
+	}
+
+	if err := pg.uri.gatherKeys(pg.db); err != nil {
+		return err
+	}
+
+	return nil
+}
+func (pg *PGBackend) GetSSLKeys() ([]SSLKey, error) {
+	if err := pg.sslKey.decrypt(pg.cfg.AESKey); err != nil {
+		return nil, err
+	}
+	return pg.sslKey.toGeneric(), nil
+}
+func (pg *PGBackend) SetSSLKeys(keys []SSLKey) error {
+	pg.sslKey.fromGeneric(keys)
+	return pg.sslKey.encrypt(pg.cfg.AESKey)
+}
+func (pg *PGBackend) GetDNSSecKeys() ([]DNSSecKey, error) {
+	if err := pg.dnssec.decrypt(pg.cfg.AESKey); err != nil {
+		return nil, err
+	}
+	return pg.dnssec.toGeneric(), nil
+}
+func (pg *PGBackend) SetDNSSecKeys(keys []DNSSecKey) error {
+	pg.dnssec.fromGeneric(keys)
+	return pg.dnssec.encrypt(pg.cfg.AESKey)
+}
+func (pg *PGBackend) GetURISignKeys() ([]URISignKey, error) {
+	if err := pg.uri.decrypt(pg.cfg.AESKey); err != nil {
+		return nil, err
+	}
+	return pg.uri.toGeneric(), nil
+}
+func (pg *PGBackend) SetURISignKeys(keys []URISignKey) error {
+	pg.uri.fromGeneric(keys)
+	return pg.uri.encrypt(pg.cfg.AESKey)
+}
+func (pg *PGBackend) GetURLSigKeys() ([]URLSigKey, error) {
+	if err := pg.url.decrypt(pg.cfg.AESKey); err != nil {
+		return nil, err
+	}
+	return pg.url.toGeneric(), nil
+}
+func (pg *PGBackend) SetURLSigKeys(keys []URLSigKey) error {
+	pg.url.fromGeneric(keys)
+	return pg.url.encrypt(pg.cfg.AESKey)
+}
+
+type PGCommonRecord struct {
+	DataEncrypted []byte
+	CommonRecord
+}
+
+type PGDNSSecRecord struct {
+	Key tc.DNSSECKeysTrafficVault
+	CDN string
+	PGCommonRecord
+}
+type PGDNSSecTable struct {
+	Records []PGDNSSecRecord
+}
+
+func (tbl *PGDNSSecTable) gatherKeys(db *sql.DB) error {
+	sz, err := getSize(db, "dnssec")
+	if err != nil {
+		log.Println("PGDNSSec gatherKeys: unable to determine size of dnssec table")
+	}
+	tbl.Records = make([]PGDNSSecRecord, sz)
+
+	rows, err := db.Query("SELECT cdn, data from dnssec")
+	if err != nil {
+		return fmt.Errorf("PGDNSSec gatherKeys: unable to query: %v", err)
+	}
+	defer rows.Close()
+	i := 0
+	for rows.Next() {
+		if i > len(tbl.Records)-1 {
+			return fmt.Errorf("PGDNSSec gatherKeys got more results than expected %v", len(tbl.Records))
+		}
+		err := rows.Scan(&tbl.Records[i].CDN, &tbl.Records[i].DataEncrypted)
+		if err != nil {
+			return fmt.Errorf("PGDNSSec gatherKeys unable to scan row: %v", err)
+		}
+		i += 1
+	}
+	return nil
+}
+func (tbl *PGDNSSecTable) decrypt(aesKey []byte) error {
+	for i, _ := range tbl.Records {
+		data, err := decryptInto(aesKey, tbl.Records[i].DataEncrypted, reflect.TypeOf(tbl.Records[i].Key))
+		if err != nil {
+			return fmt.Errorf("unable to decrypt into keys: %v", err)
+		}
+		tbl.Records[i].Key = *data.Interface().(*tc.DNSSECKeysTrafficVault)
+	}
+	return nil
+}
+func (tbl *PGDNSSecTable) encrypt(aesKey []byte) error {
+	for i, dns := range tbl.Records {
+		data, err := json.Marshal(&dns.Key)
+		if err != nil {
+			return fmt.Errorf("encrypt issue marshalling keys: %v", err)
+		}
+		dat, err := encrypt(data, aesKey)
+		if err != nil {
+			return fmt.Errorf("encrypt error: %v", err)
+		}
+		tbl.Records[i].DataEncrypted = dat
+	}
+	return nil
+}
+func (tbl *PGDNSSecTable) toGeneric() []DNSSecKey {
+	keys := make([]DNSSecKey, len(tbl.Records))
+
+	for i, record := range tbl.Records {
+		keys[i] = DNSSecKey{
+			CDN:                    record.CDN,
+			DNSSECKeysTrafficVault: record.Key,
+			CommonRecord:           record.CommonRecord,
+		}
+	}
+
+	return keys
+}
+func (tbl *PGDNSSecTable) fromGeneric(keys []DNSSecKey) {
+	tbl.Records = make([]PGDNSSecRecord, len(keys))
+
+	for i, key := range keys {
+		tbl.Records[i] = PGDNSSecRecord{
+			Key: key.DNSSECKeysTrafficVault,
+			CDN: key.CDN,
+			PGCommonRecord: PGCommonRecord{
+				DataEncrypted: nil,
+				CommonRecord:  CommonRecord{},
+			},
+		}
+	}
+}
+func (tbl *PGDNSSecTable) validate() []string {
+	for i, record := range tbl.Records {
+		if record.DataEncrypted == nil && len(record.Key) > 0 {
+			return []string{fmt.Sprintf("DNSSEC Key %v: DataEncrypted is blank!", i)}
+		}
+	}
+	return nil
+}
+func (tbl *PGDNSSecTable) insertKeys(db *sql.DB) error {
+	queryFmt := "INSERT INTO dnssec (cdn, data) VALUES "
+	stride := 2
+	queryArgs := make([]interface{}, len(tbl.Records)*stride)
+	for i, record := range tbl.Records {
+		j := i * stride
+		queryArgs[j] = record.CDN
+		queryArgs[j+1] = record.DataEncrypted
+	}
+	return insertIntoTable(db, queryFmt, stride, queryArgs)
+}
+
+type PGSSLKeyRecord struct {
+	Keys tc.DeliveryServiceSSLKeys
+	PGCommonRecord
+
+	// These records are stored on the table but are duplicated
+	DeliveryService string
+	Version         string
+	CDN             string
+}
+type PGSSLKeyTable struct {
+	Records []PGSSLKeyRecord
+}
+
+func (tbl *PGSSLKeyTable) insertKeys(db *sql.DB) error {
+	queryFmt := "INSERT INTO sslkey (deliveryservice, data, cdn, version) VALUES "
+	stride := 4
+	queryArgs := make([]interface{}, len(tbl.Records)*stride)
+	for i, record := range tbl.Records {
+		j := i * stride
+		queryArgs[j] = record.DeliveryService
+		queryArgs[j+1] = record.DataEncrypted
+		queryArgs[j+2] = record.CDN
+		queryArgs[j+3] = record.Version
+	}
+	return insertIntoTable(db, queryFmt, 4, queryArgs)
+}
+func (tbl *PGSSLKeyTable) gatherKeys(db *sql.DB) error {
+	sz, err := getSize(db, "sslkey")
+	if err != nil {
+		return fmt.Errorf("PGSSLKey gatherKeys unable to determine size of sslkey table: %v", err)
+	}
+	tbl.Records = make([]PGSSLKeyRecord, sz)
+
+	rows, err := db.Query("SELECT data, deliveryservice, cdn, version from sslkey")
+	if err != nil {
+		return fmt.Errorf("PGSSLKey gatherKeys unable to query: %v", err)
+	}
+	defer rows.Close()
+	i := 0
+	for rows.Next() {
+		if i > len(tbl.Records)-1 {
+			return errors.New("PGSSLKey gatherKeys: got more results than expected")
+		}
+		err := rows.Scan(&tbl.Records[i].DataEncrypted, &tbl.Records[i].DeliveryService, &tbl.Records[i].CDN, &tbl.Records[i].Version)
+		if err != nil {
+			return fmt.Errorf("PGSSLKey gatherKeys unable to scan row: %v", err)
+		}
+		i += 1
+	}
+	return nil
+}
+func (tbl *PGSSLKeyTable) decrypt(aesKey []byte) error {
+	for i, dns := range tbl.Records {
+		data, err := decryptInto(aesKey, dns.DataEncrypted, reflect.TypeOf(dns.Keys))
+		if err != nil {
+			return fmt.Errorf("unable to decrypt into keys: %v", err)
+		}
+		tbl.Records[i].Keys = *data.Interface().(*tc.DeliveryServiceSSLKeys)
+	}
+	return nil
+}
+func (tbl *PGSSLKeyTable) encrypt(aesKey []byte) error {
+	for i, dns := range tbl.Records {
+		data, err := json.Marshal(dns.Keys)
+		if err != nil {
+			return fmt.Errorf("encrypt issue marshalling keys: %v", err)
+		}
+		dat, err := encrypt(data, aesKey)
+		if err != nil {
+			return fmt.Errorf("encrypt error: %v", err)
+		}
+		tbl.Records[i].DataEncrypted = dat
+	}
+	return nil
+}
+func (tbl *PGSSLKeyTable) toGeneric() []SSLKey {
+	keys := make([]SSLKey, len(tbl.Records))
+
+	for i, record := range tbl.Records {
+		keys[i] = SSLKey{
+			DeliveryServiceSSLKeys: record.Keys,
+			CommonRecord:           record.CommonRecord,
+		}
+	}
+	return keys
+}
+func (tbl *PGSSLKeyTable) fromGeneric(keys []SSLKey) {
+	tbl.Records = make([]PGSSLKeyRecord, len(keys))
+
+	for i, key := range keys {
+		tbl.Records[i] = PGSSLKeyRecord{
+			Keys: key.DeliveryServiceSSLKeys,
+			PGCommonRecord: PGCommonRecord{
+				DataEncrypted: nil,
+				CommonRecord:  CommonRecord{},
+			},
+			DeliveryService: key.DeliveryService,
+			CDN:             key.CDN,
+		}
+	}
+}
+func (tbl *PGSSLKeyTable) validate() []string {
+	defaultKey := tc.DeliveryServiceSSLKeys{}
+	var errors []string
+	fmtStr := "SSL Key %v: %v"
+	for i, record := range tbl.Records {
+		if record.Keys == defaultKey {
+			errors = append(errors, fmt.Sprintf(fmtStr, i, "DS SSL Keys are default!"))
+		} else if record.Keys.Key == "" {
+			errors = append(errors, fmt.Sprintf(fmtStr, i, "Key is blank!"))
+		} else if record.Keys.CDN == "" {
+			errors = append(errors, fmt.Sprintf(fmtStr, i, "CDN is blank!"))
+		} else if record.Keys.DeliveryService == "" {
+			errors = append(errors, fmt.Sprintf(fmtStr, i, "DS is blank!"))
+		} else if record.DataEncrypted == nil {
+			errors = append(errors, fmt.Sprintf(fmtStr, i, "DataEncrypted is blank!"))
+		}
+	}
+	return errors
+}
+
+type PGURLSigKeyRecord struct {
+	Keys            tc.URLSigKeys
+	DeliveryService string
+	PGCommonRecord
+}
+type PGURLSigKeyTable struct {
+	Records []PGURLSigKeyRecord
+}
+
+func (tbl *PGURLSigKeyTable) insertKeys(db *sql.DB) error {
+	queryBase := "INSERT INTO url_sig_key (deliveryservice, data) VALUES "
+	stride := 2
+	queryArgs := make([]interface{}, len(tbl.Records)*stride)
+	for i, record := range tbl.Records {
+		j := i * stride
+		queryArgs[j] = record.DeliveryService
+		queryArgs[j+1] = record.DataEncrypted
+	}
+	return insertIntoTable(db, queryBase, stride, queryArgs)
+}
+func (tbl *PGURLSigKeyTable) gatherKeys(db *sql.DB) error {
+	sz, err := getSize(db, "url_sig_key")
+	if err != nil {
+		log.Println("PGURLSigKey gatherKeys: unable to determine url_sig_key table size")
+	}
+	tbl.Records = make([]PGURLSigKeyRecord, sz)
+
+	rows, err := db.Query("SELECT deliveryservice, data from url_sig_key")
+	if err != nil {
+		return fmt.Errorf("PGURLSigKey gatherKeys error creating query: %v", err)
+	}
+	defer rows.Close()
+	i := 0
+	for rows.Next() {
+		if i > len(tbl.Records)-1 {
+			return fmt.Errorf("PGURLSigKey gatherKeys: got more results than expected %v", len(tbl.Records))
+		}
+		err := rows.Scan(&tbl.Records[i].DeliveryService, &tbl.Records[i].DataEncrypted)
+		if err != nil {
+			return fmt.Errorf("PGURLSigKey gatherKeys: unable to scan row: %v", err)
+		}
+		i += 1
+	}
+	return nil
+}
+func (tbl *PGURLSigKeyTable) decrypt(aesKey []byte) error {
+	for i, sig := range tbl.Records {
+		data, err := decryptInto(aesKey, sig.DataEncrypted, reflect.TypeOf(sig.Keys))
+		if err != nil {
+			return fmt.Errorf("unable to decrypt into keys: %v", err)
+		}
+
+		tbl.Records[i].Keys = *data.Interface().(*tc.URLSigKeys)
+	}
+	return nil
+}
+func (tbl *PGURLSigKeyTable) encrypt(aesKey []byte) error {
+	for i, sig := range tbl.Records {
+		data, err := json.Marshal(&sig.Keys)
+		if err != nil {
+			return fmt.Errorf("encrypt issue marshalling keys: %v", err)
+		}
+
+		dat, err := encrypt(data, aesKey)
+		if err != nil {
+			return fmt.Errorf("encrypt error: %v", err)
+		}
+		tbl.Records[i].DataEncrypted = dat
+	}
+	return nil
+}
+func (tbl *PGURLSigKeyTable) toGeneric() []URLSigKey {
+	keys := make([]URLSigKey, len(tbl.Records))
+
+	for i, record := range tbl.Records {
+		keys[i] = URLSigKey{
+			DeliveryService: record.DeliveryService,
+			URLSigKeys:      record.Keys,
+			CommonRecord:    record.CommonRecord,
+		}
+	}
+	return keys
+}
+func (tbl *PGURLSigKeyTable) fromGeneric(keys []URLSigKey) {
+	tbl.Records = make([]PGURLSigKeyRecord, len(keys))
+
+	for i, key := range keys {
+		tbl.Records[i] = PGURLSigKeyRecord{
+			Keys:            key.URLSigKeys,
+			DeliveryService: key.DeliveryService,
+			PGCommonRecord: PGCommonRecord{
+				DataEncrypted: nil,
+				CommonRecord:  CommonRecord{},
+			},
+		}
+	}
+}
+func (tbl *PGURLSigKeyTable) validate() []string {
+	for i, record := range tbl.Records {
+		if record.DataEncrypted == nil && len(record.Keys) > 0 {
+			return []string{fmt.Sprintf("URl Sig Key %v: DataEncrypted is blank!", i)}
+		}
+	}
+	return nil
+}
+
+type PGURISignKeyRecord struct {
+	Keys            map[string]tc.URISignerKeyset
+	DeliveryService string
+	PGCommonRecord
+}
+type PGURISignKeyTable struct {
+	Records []PGURISignKeyRecord
+}

Review comment:
       GoDoc?

##########
File path: tools/traffic_vault_migrate/postgres.go
##########
@@ -0,0 +1,716 @@
+package main
+
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+import (
+	"database/sql"
+	"encoding/base64"
+	"encoding/json"
+	"errors"
+	"fmt"
+	"github.com/apache/trafficcontrol/lib/go-tc"
+	util "github.com/apache/trafficcontrol/lib/go-util"
+	_ "github.com/lib/pq"
+	"log"
+	"reflect"
+	"strconv"
+	"strings"
+)
+
+type PGConfig struct {
+	Host     string `json:"host"`
+	Port     string `json:"port"`
+	User     string `json:"user"`
+	Password string `json:"password"`
+	SSLMode  string `json:"sslmode"`
+	Database string `json:"database"`
+	Key      string `json:"aesKey"`
+	AESKey   []byte
+}
+type PGBackend struct {
+	sslKey PGSSLKeyTable
+	dnssec PGDNSSecTable
+	uri    PGURISignKeyTable
+	url    PGURLSigKeyTable
+	cfg    PGConfig
+	db     *sql.DB
+}
+
+func (pg *PGBackend) String() string {
+	data := fmt.Sprintf("PG server %v@%v:%v\n", pg.cfg.User, pg.cfg.Host, pg.cfg.Port)
+	data += fmt.Sprintf("\tSSL Keys: %v\n", len(pg.sslKey.Records))
+	data += fmt.Sprintf("\tDNSSec Keys: %v\n", len(pg.dnssec.Records))
+	data += fmt.Sprintf("\tURI Keys: %v\n", len(pg.uri.Records))
+	data += fmt.Sprintf("\tURL Keys: %v\n", len(pg.url.Records))
+	return data
+}
+func (pg *PGBackend) Name() string {
+	return "PG"
+}
+func (pg *PGBackend) ReadConfig(s string) error {
+	genericCfg, err := UnmarshalConfig(s, reflect.TypeOf(pg.cfg))
+	if err != nil {
+		return err
+	}
+
+	pg.cfg = *genericCfg.Interface().(*PGConfig)
+
+	pg.cfg.AESKey, err = base64.StdEncoding.DecodeString(pg.cfg.Key)
+	if err != nil {
+		return fmt.Errorf("unable to decode PG AESKey: %v", err)
+	}
+	return nil
+}
+func (pg *PGBackend) Insert() error {
+	if err := pg.sslKey.insertKeys(pg.db); err != nil {
+		return err
+	}
+	if err := pg.dnssec.insertKeys(pg.db); err != nil {
+		return err
+	}
+	if err := pg.url.insertKeys(pg.db); err != nil {
+		return err
+	}
+	if err := pg.uri.insertKeys(pg.db); err != nil {
+		return err
+	}
+	return nil
+}
+func (pg *PGBackend) Start() error {
+	sqlStr := fmt.Sprintf("postgres://%s:%s@%s:%s/%s?sslmode=%s", pg.cfg.User, pg.cfg.Password, pg.cfg.Host, pg.cfg.Port, pg.cfg.Database, pg.cfg.SSLMode)
+	db, err := sql.Open("postgres", sqlStr)
+	if err != nil {
+		return fmt.Errorf("Unable to start PG client: %v", err)
+	}
+
+	pg.db = db
+	pg.sslKey = PGSSLKeyTable{}
+	pg.dnssec = PGDNSSecTable{}
+	pg.url = PGURLSigKeyTable{}
+	pg.uri = PGURISignKeyTable{}
+
+	return nil
+}
+func (pg *PGBackend) ValidateKey() []string {
+	var errors []string
+	if errs := pg.sslKey.validate(); errs != nil {
+		errors = append(errors, errs...)
+	}
+	if errs := pg.dnssec.validate(); errs != nil {
+		errors = append(errors, errs...)
+	}
+	if errs := pg.uri.validate(); errs != nil {
+		errors = append(errors, errs...)
+	}
+	if errs := pg.url.validate(); errs != nil {
+		errors = append(errors, errs...)
+	}
+	return errors
+}
+func (pg *PGBackend) Stop() error {
+	return pg.db.Close()
+}
+func (pg *PGBackend) Ping() error {
+	return pg.db.Ping()
+}
+func (pg *PGBackend) Fetch() error {
+	if err := pg.sslKey.gatherKeys(pg.db); err != nil {
+		return err
+	}
+
+	if err := pg.dnssec.gatherKeys(pg.db); err != nil {
+		return err
+	}
+
+	if err := pg.url.gatherKeys(pg.db); err != nil {
+		return err
+	}
+
+	if err := pg.uri.gatherKeys(pg.db); err != nil {
+		return err
+	}
+
+	return nil
+}
+func (pg *PGBackend) GetSSLKeys() ([]SSLKey, error) {
+	if err := pg.sslKey.decrypt(pg.cfg.AESKey); err != nil {
+		return nil, err
+	}
+	return pg.sslKey.toGeneric(), nil
+}
+func (pg *PGBackend) SetSSLKeys(keys []SSLKey) error {
+	pg.sslKey.fromGeneric(keys)
+	return pg.sslKey.encrypt(pg.cfg.AESKey)
+}
+func (pg *PGBackend) GetDNSSecKeys() ([]DNSSecKey, error) {
+	if err := pg.dnssec.decrypt(pg.cfg.AESKey); err != nil {
+		return nil, err
+	}
+	return pg.dnssec.toGeneric(), nil
+}
+func (pg *PGBackend) SetDNSSecKeys(keys []DNSSecKey) error {
+	pg.dnssec.fromGeneric(keys)
+	return pg.dnssec.encrypt(pg.cfg.AESKey)
+}
+func (pg *PGBackend) GetURISignKeys() ([]URISignKey, error) {
+	if err := pg.uri.decrypt(pg.cfg.AESKey); err != nil {
+		return nil, err
+	}
+	return pg.uri.toGeneric(), nil
+}
+func (pg *PGBackend) SetURISignKeys(keys []URISignKey) error {
+	pg.uri.fromGeneric(keys)
+	return pg.uri.encrypt(pg.cfg.AESKey)
+}
+func (pg *PGBackend) GetURLSigKeys() ([]URLSigKey, error) {
+	if err := pg.url.decrypt(pg.cfg.AESKey); err != nil {
+		return nil, err
+	}
+	return pg.url.toGeneric(), nil
+}
+func (pg *PGBackend) SetURLSigKeys(keys []URLSigKey) error {
+	pg.url.fromGeneric(keys)
+	return pg.url.encrypt(pg.cfg.AESKey)
+}
+
+type PGCommonRecord struct {
+	DataEncrypted []byte
+	CommonRecord
+}
+
+type PGDNSSecRecord struct {
+	Key tc.DNSSECKeysTrafficVault
+	CDN string
+	PGCommonRecord
+}
+type PGDNSSecTable struct {
+	Records []PGDNSSecRecord
+}
+
+func (tbl *PGDNSSecTable) gatherKeys(db *sql.DB) error {
+	sz, err := getSize(db, "dnssec")
+	if err != nil {
+		log.Println("PGDNSSec gatherKeys: unable to determine size of dnssec table")
+	}
+	tbl.Records = make([]PGDNSSecRecord, sz)
+
+	rows, err := db.Query("SELECT cdn, data from dnssec")
+	if err != nil {
+		return fmt.Errorf("PGDNSSec gatherKeys: unable to query: %v", err)
+	}
+	defer rows.Close()
+	i := 0
+	for rows.Next() {
+		if i > len(tbl.Records)-1 {
+			return fmt.Errorf("PGDNSSec gatherKeys got more results than expected %v", len(tbl.Records))
+		}
+		err := rows.Scan(&tbl.Records[i].CDN, &tbl.Records[i].DataEncrypted)
+		if err != nil {
+			return fmt.Errorf("PGDNSSec gatherKeys unable to scan row: %v", err)
+		}
+		i += 1
+	}
+	return nil
+}
+func (tbl *PGDNSSecTable) decrypt(aesKey []byte) error {
+	for i, _ := range tbl.Records {
+		data, err := decryptInto(aesKey, tbl.Records[i].DataEncrypted, reflect.TypeOf(tbl.Records[i].Key))
+		if err != nil {
+			return fmt.Errorf("unable to decrypt into keys: %v", err)
+		}
+		tbl.Records[i].Key = *data.Interface().(*tc.DNSSECKeysTrafficVault)
+	}
+	return nil
+}
+func (tbl *PGDNSSecTable) encrypt(aesKey []byte) error {
+	for i, dns := range tbl.Records {
+		data, err := json.Marshal(&dns.Key)
+		if err != nil {
+			return fmt.Errorf("encrypt issue marshalling keys: %v", err)
+		}
+		dat, err := encrypt(data, aesKey)
+		if err != nil {
+			return fmt.Errorf("encrypt error: %v", err)
+		}
+		tbl.Records[i].DataEncrypted = dat
+	}
+	return nil
+}
+func (tbl *PGDNSSecTable) toGeneric() []DNSSecKey {
+	keys := make([]DNSSecKey, len(tbl.Records))
+
+	for i, record := range tbl.Records {
+		keys[i] = DNSSecKey{
+			CDN:                    record.CDN,
+			DNSSECKeysTrafficVault: record.Key,
+			CommonRecord:           record.CommonRecord,
+		}
+	}
+
+	return keys
+}
+func (tbl *PGDNSSecTable) fromGeneric(keys []DNSSecKey) {
+	tbl.Records = make([]PGDNSSecRecord, len(keys))
+
+	for i, key := range keys {
+		tbl.Records[i] = PGDNSSecRecord{
+			Key: key.DNSSECKeysTrafficVault,
+			CDN: key.CDN,
+			PGCommonRecord: PGCommonRecord{
+				DataEncrypted: nil,
+				CommonRecord:  CommonRecord{},
+			},
+		}
+	}
+}
+func (tbl *PGDNSSecTable) validate() []string {
+	for i, record := range tbl.Records {
+		if record.DataEncrypted == nil && len(record.Key) > 0 {
+			return []string{fmt.Sprintf("DNSSEC Key %v: DataEncrypted is blank!", i)}
+		}
+	}
+	return nil
+}
+func (tbl *PGDNSSecTable) insertKeys(db *sql.DB) error {
+	queryFmt := "INSERT INTO dnssec (cdn, data) VALUES "
+	stride := 2
+	queryArgs := make([]interface{}, len(tbl.Records)*stride)
+	for i, record := range tbl.Records {
+		j := i * stride
+		queryArgs[j] = record.CDN
+		queryArgs[j+1] = record.DataEncrypted
+	}
+	return insertIntoTable(db, queryFmt, stride, queryArgs)
+}
+
+type PGSSLKeyRecord struct {
+	Keys tc.DeliveryServiceSSLKeys
+	PGCommonRecord
+
+	// These records are stored on the table but are duplicated
+	DeliveryService string
+	Version         string
+	CDN             string
+}
+type PGSSLKeyTable struct {
+	Records []PGSSLKeyRecord
+}
+
+func (tbl *PGSSLKeyTable) insertKeys(db *sql.DB) error {
+	queryFmt := "INSERT INTO sslkey (deliveryservice, data, cdn, version) VALUES "
+	stride := 4
+	queryArgs := make([]interface{}, len(tbl.Records)*stride)
+	for i, record := range tbl.Records {
+		j := i * stride
+		queryArgs[j] = record.DeliveryService
+		queryArgs[j+1] = record.DataEncrypted
+		queryArgs[j+2] = record.CDN
+		queryArgs[j+3] = record.Version
+	}
+	return insertIntoTable(db, queryFmt, 4, queryArgs)
+}
+func (tbl *PGSSLKeyTable) gatherKeys(db *sql.DB) error {
+	sz, err := getSize(db, "sslkey")
+	if err != nil {
+		return fmt.Errorf("PGSSLKey gatherKeys unable to determine size of sslkey table: %v", err)
+	}
+	tbl.Records = make([]PGSSLKeyRecord, sz)
+
+	rows, err := db.Query("SELECT data, deliveryservice, cdn, version from sslkey")
+	if err != nil {
+		return fmt.Errorf("PGSSLKey gatherKeys unable to query: %v", err)
+	}
+	defer rows.Close()
+	i := 0
+	for rows.Next() {
+		if i > len(tbl.Records)-1 {
+			return errors.New("PGSSLKey gatherKeys: got more results than expected")
+		}
+		err := rows.Scan(&tbl.Records[i].DataEncrypted, &tbl.Records[i].DeliveryService, &tbl.Records[i].CDN, &tbl.Records[i].Version)
+		if err != nil {
+			return fmt.Errorf("PGSSLKey gatherKeys unable to scan row: %v", err)
+		}
+		i += 1
+	}
+	return nil
+}
+func (tbl *PGSSLKeyTable) decrypt(aesKey []byte) error {
+	for i, dns := range tbl.Records {
+		data, err := decryptInto(aesKey, dns.DataEncrypted, reflect.TypeOf(dns.Keys))
+		if err != nil {
+			return fmt.Errorf("unable to decrypt into keys: %v", err)
+		}
+		tbl.Records[i].Keys = *data.Interface().(*tc.DeliveryServiceSSLKeys)
+	}
+	return nil
+}
+func (tbl *PGSSLKeyTable) encrypt(aesKey []byte) error {
+	for i, dns := range tbl.Records {
+		data, err := json.Marshal(dns.Keys)
+		if err != nil {
+			return fmt.Errorf("encrypt issue marshalling keys: %v", err)
+		}
+		dat, err := encrypt(data, aesKey)
+		if err != nil {
+			return fmt.Errorf("encrypt error: %v", err)
+		}
+		tbl.Records[i].DataEncrypted = dat
+	}
+	return nil
+}
+func (tbl *PGSSLKeyTable) toGeneric() []SSLKey {
+	keys := make([]SSLKey, len(tbl.Records))
+
+	for i, record := range tbl.Records {
+		keys[i] = SSLKey{
+			DeliveryServiceSSLKeys: record.Keys,
+			CommonRecord:           record.CommonRecord,
+		}
+	}
+	return keys
+}
+func (tbl *PGSSLKeyTable) fromGeneric(keys []SSLKey) {
+	tbl.Records = make([]PGSSLKeyRecord, len(keys))
+
+	for i, key := range keys {
+		tbl.Records[i] = PGSSLKeyRecord{
+			Keys: key.DeliveryServiceSSLKeys,
+			PGCommonRecord: PGCommonRecord{
+				DataEncrypted: nil,
+				CommonRecord:  CommonRecord{},
+			},
+			DeliveryService: key.DeliveryService,
+			CDN:             key.CDN,
+		}
+	}
+}
+func (tbl *PGSSLKeyTable) validate() []string {
+	defaultKey := tc.DeliveryServiceSSLKeys{}
+	var errors []string
+	fmtStr := "SSL Key %v: %v"
+	for i, record := range tbl.Records {
+		if record.Keys == defaultKey {
+			errors = append(errors, fmt.Sprintf(fmtStr, i, "DS SSL Keys are default!"))
+		} else if record.Keys.Key == "" {
+			errors = append(errors, fmt.Sprintf(fmtStr, i, "Key is blank!"))
+		} else if record.Keys.CDN == "" {
+			errors = append(errors, fmt.Sprintf(fmtStr, i, "CDN is blank!"))
+		} else if record.Keys.DeliveryService == "" {
+			errors = append(errors, fmt.Sprintf(fmtStr, i, "DS is blank!"))
+		} else if record.DataEncrypted == nil {
+			errors = append(errors, fmt.Sprintf(fmtStr, i, "DataEncrypted is blank!"))
+		}
+	}
+	return errors
+}
+
+type PGURLSigKeyRecord struct {
+	Keys            tc.URLSigKeys
+	DeliveryService string
+	PGCommonRecord
+}
+type PGURLSigKeyTable struct {
+	Records []PGURLSigKeyRecord
+}
+
+func (tbl *PGURLSigKeyTable) insertKeys(db *sql.DB) error {
+	queryBase := "INSERT INTO url_sig_key (deliveryservice, data) VALUES "
+	stride := 2
+	queryArgs := make([]interface{}, len(tbl.Records)*stride)
+	for i, record := range tbl.Records {
+		j := i * stride
+		queryArgs[j] = record.DeliveryService
+		queryArgs[j+1] = record.DataEncrypted
+	}
+	return insertIntoTable(db, queryBase, stride, queryArgs)
+}
+func (tbl *PGURLSigKeyTable) gatherKeys(db *sql.DB) error {
+	sz, err := getSize(db, "url_sig_key")
+	if err != nil {
+		log.Println("PGURLSigKey gatherKeys: unable to determine url_sig_key table size")
+	}
+	tbl.Records = make([]PGURLSigKeyRecord, sz)
+
+	rows, err := db.Query("SELECT deliveryservice, data from url_sig_key")
+	if err != nil {
+		return fmt.Errorf("PGURLSigKey gatherKeys error creating query: %v", err)

Review comment:
       same as above RE: `%w` vs `%v`

##########
File path: tools/traffic_vault_migrate/postgres.go
##########
@@ -0,0 +1,716 @@
+package main
+
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+import (
+	"database/sql"
+	"encoding/base64"
+	"encoding/json"
+	"errors"
+	"fmt"
+	"github.com/apache/trafficcontrol/lib/go-tc"
+	util "github.com/apache/trafficcontrol/lib/go-util"
+	_ "github.com/lib/pq"
+	"log"
+	"reflect"
+	"strconv"
+	"strings"
+)
+
+type PGConfig struct {
+	Host     string `json:"host"`
+	Port     string `json:"port"`
+	User     string `json:"user"`
+	Password string `json:"password"`
+	SSLMode  string `json:"sslmode"`
+	Database string `json:"database"`
+	Key      string `json:"aesKey"`
+	AESKey   []byte
+}
+type PGBackend struct {
+	sslKey PGSSLKeyTable
+	dnssec PGDNSSecTable
+	uri    PGURISignKeyTable
+	url    PGURLSigKeyTable
+	cfg    PGConfig
+	db     *sql.DB
+}
+
+func (pg *PGBackend) String() string {
+	data := fmt.Sprintf("PG server %v@%v:%v\n", pg.cfg.User, pg.cfg.Host, pg.cfg.Port)
+	data += fmt.Sprintf("\tSSL Keys: %v\n", len(pg.sslKey.Records))
+	data += fmt.Sprintf("\tDNSSec Keys: %v\n", len(pg.dnssec.Records))
+	data += fmt.Sprintf("\tURI Keys: %v\n", len(pg.uri.Records))
+	data += fmt.Sprintf("\tURL Keys: %v\n", len(pg.url.Records))
+	return data
+}
+func (pg *PGBackend) Name() string {
+	return "PG"
+}
+func (pg *PGBackend) ReadConfig(s string) error {
+	genericCfg, err := UnmarshalConfig(s, reflect.TypeOf(pg.cfg))
+	if err != nil {
+		return err
+	}
+
+	pg.cfg = *genericCfg.Interface().(*PGConfig)
+
+	pg.cfg.AESKey, err = base64.StdEncoding.DecodeString(pg.cfg.Key)
+	if err != nil {
+		return fmt.Errorf("unable to decode PG AESKey: %v", err)
+	}
+	return nil
+}
+func (pg *PGBackend) Insert() error {
+	if err := pg.sslKey.insertKeys(pg.db); err != nil {
+		return err
+	}
+	if err := pg.dnssec.insertKeys(pg.db); err != nil {
+		return err
+	}
+	if err := pg.url.insertKeys(pg.db); err != nil {
+		return err
+	}
+	if err := pg.uri.insertKeys(pg.db); err != nil {
+		return err
+	}
+	return nil
+}
+func (pg *PGBackend) Start() error {
+	sqlStr := fmt.Sprintf("postgres://%s:%s@%s:%s/%s?sslmode=%s", pg.cfg.User, pg.cfg.Password, pg.cfg.Host, pg.cfg.Port, pg.cfg.Database, pg.cfg.SSLMode)
+	db, err := sql.Open("postgres", sqlStr)
+	if err != nil {
+		return fmt.Errorf("Unable to start PG client: %v", err)
+	}
+
+	pg.db = db
+	pg.sslKey = PGSSLKeyTable{}
+	pg.dnssec = PGDNSSecTable{}
+	pg.url = PGURLSigKeyTable{}
+	pg.uri = PGURISignKeyTable{}
+
+	return nil
+}
+func (pg *PGBackend) ValidateKey() []string {
+	var errors []string
+	if errs := pg.sslKey.validate(); errs != nil {
+		errors = append(errors, errs...)
+	}
+	if errs := pg.dnssec.validate(); errs != nil {
+		errors = append(errors, errs...)
+	}
+	if errs := pg.uri.validate(); errs != nil {
+		errors = append(errors, errs...)
+	}
+	if errs := pg.url.validate(); errs != nil {
+		errors = append(errors, errs...)
+	}
+	return errors
+}
+func (pg *PGBackend) Stop() error {
+	return pg.db.Close()
+}
+func (pg *PGBackend) Ping() error {
+	return pg.db.Ping()
+}
+func (pg *PGBackend) Fetch() error {
+	if err := pg.sslKey.gatherKeys(pg.db); err != nil {
+		return err
+	}
+
+	if err := pg.dnssec.gatherKeys(pg.db); err != nil {
+		return err
+	}
+
+	if err := pg.url.gatherKeys(pg.db); err != nil {
+		return err
+	}
+
+	if err := pg.uri.gatherKeys(pg.db); err != nil {
+		return err
+	}
+
+	return nil
+}
+func (pg *PGBackend) GetSSLKeys() ([]SSLKey, error) {
+	if err := pg.sslKey.decrypt(pg.cfg.AESKey); err != nil {
+		return nil, err
+	}
+	return pg.sslKey.toGeneric(), nil
+}
+func (pg *PGBackend) SetSSLKeys(keys []SSLKey) error {
+	pg.sslKey.fromGeneric(keys)
+	return pg.sslKey.encrypt(pg.cfg.AESKey)
+}
+func (pg *PGBackend) GetDNSSecKeys() ([]DNSSecKey, error) {
+	if err := pg.dnssec.decrypt(pg.cfg.AESKey); err != nil {
+		return nil, err
+	}
+	return pg.dnssec.toGeneric(), nil
+}
+func (pg *PGBackend) SetDNSSecKeys(keys []DNSSecKey) error {
+	pg.dnssec.fromGeneric(keys)
+	return pg.dnssec.encrypt(pg.cfg.AESKey)
+}
+func (pg *PGBackend) GetURISignKeys() ([]URISignKey, error) {
+	if err := pg.uri.decrypt(pg.cfg.AESKey); err != nil {
+		return nil, err
+	}
+	return pg.uri.toGeneric(), nil
+}
+func (pg *PGBackend) SetURISignKeys(keys []URISignKey) error {
+	pg.uri.fromGeneric(keys)
+	return pg.uri.encrypt(pg.cfg.AESKey)
+}
+func (pg *PGBackend) GetURLSigKeys() ([]URLSigKey, error) {
+	if err := pg.url.decrypt(pg.cfg.AESKey); err != nil {
+		return nil, err
+	}
+	return pg.url.toGeneric(), nil
+}
+func (pg *PGBackend) SetURLSigKeys(keys []URLSigKey) error {
+	pg.url.fromGeneric(keys)
+	return pg.url.encrypt(pg.cfg.AESKey)
+}
+
+type PGCommonRecord struct {
+	DataEncrypted []byte
+	CommonRecord
+}
+
+type PGDNSSecRecord struct {
+	Key tc.DNSSECKeysTrafficVault
+	CDN string
+	PGCommonRecord
+}
+type PGDNSSecTable struct {
+	Records []PGDNSSecRecord
+}
+
+func (tbl *PGDNSSecTable) gatherKeys(db *sql.DB) error {
+	sz, err := getSize(db, "dnssec")
+	if err != nil {
+		log.Println("PGDNSSec gatherKeys: unable to determine size of dnssec table")
+	}
+	tbl.Records = make([]PGDNSSecRecord, sz)
+
+	rows, err := db.Query("SELECT cdn, data from dnssec")
+	if err != nil {
+		return fmt.Errorf("PGDNSSec gatherKeys: unable to query: %v", err)
+	}
+	defer rows.Close()
+	i := 0
+	for rows.Next() {
+		if i > len(tbl.Records)-1 {
+			return fmt.Errorf("PGDNSSec gatherKeys got more results than expected %v", len(tbl.Records))
+		}
+		err := rows.Scan(&tbl.Records[i].CDN, &tbl.Records[i].DataEncrypted)
+		if err != nil {
+			return fmt.Errorf("PGDNSSec gatherKeys unable to scan row: %v", err)
+		}
+		i += 1
+	}
+	return nil
+}
+func (tbl *PGDNSSecTable) decrypt(aesKey []byte) error {
+	for i, _ := range tbl.Records {
+		data, err := decryptInto(aesKey, tbl.Records[i].DataEncrypted, reflect.TypeOf(tbl.Records[i].Key))
+		if err != nil {
+			return fmt.Errorf("unable to decrypt into keys: %v", err)
+		}
+		tbl.Records[i].Key = *data.Interface().(*tc.DNSSECKeysTrafficVault)
+	}
+	return nil
+}
+func (tbl *PGDNSSecTable) encrypt(aesKey []byte) error {
+	for i, dns := range tbl.Records {
+		data, err := json.Marshal(&dns.Key)
+		if err != nil {
+			return fmt.Errorf("encrypt issue marshalling keys: %v", err)
+		}
+		dat, err := encrypt(data, aesKey)
+		if err != nil {
+			return fmt.Errorf("encrypt error: %v", err)
+		}
+		tbl.Records[i].DataEncrypted = dat
+	}
+	return nil
+}
+func (tbl *PGDNSSecTable) toGeneric() []DNSSecKey {
+	keys := make([]DNSSecKey, len(tbl.Records))
+
+	for i, record := range tbl.Records {
+		keys[i] = DNSSecKey{
+			CDN:                    record.CDN,
+			DNSSECKeysTrafficVault: record.Key,
+			CommonRecord:           record.CommonRecord,
+		}
+	}
+
+	return keys
+}
+func (tbl *PGDNSSecTable) fromGeneric(keys []DNSSecKey) {
+	tbl.Records = make([]PGDNSSecRecord, len(keys))
+
+	for i, key := range keys {
+		tbl.Records[i] = PGDNSSecRecord{
+			Key: key.DNSSECKeysTrafficVault,
+			CDN: key.CDN,
+			PGCommonRecord: PGCommonRecord{
+				DataEncrypted: nil,
+				CommonRecord:  CommonRecord{},
+			},
+		}
+	}
+}
+func (tbl *PGDNSSecTable) validate() []string {
+	for i, record := range tbl.Records {
+		if record.DataEncrypted == nil && len(record.Key) > 0 {
+			return []string{fmt.Sprintf("DNSSEC Key %v: DataEncrypted is blank!", i)}
+		}
+	}
+	return nil
+}
+func (tbl *PGDNSSecTable) insertKeys(db *sql.DB) error {
+	queryFmt := "INSERT INTO dnssec (cdn, data) VALUES "
+	stride := 2
+	queryArgs := make([]interface{}, len(tbl.Records)*stride)
+	for i, record := range tbl.Records {
+		j := i * stride
+		queryArgs[j] = record.CDN
+		queryArgs[j+1] = record.DataEncrypted
+	}
+	return insertIntoTable(db, queryFmt, stride, queryArgs)
+}
+
+type PGSSLKeyRecord struct {
+	Keys tc.DeliveryServiceSSLKeys
+	PGCommonRecord
+
+	// These records are stored on the table but are duplicated
+	DeliveryService string
+	Version         string
+	CDN             string
+}
+type PGSSLKeyTable struct {
+	Records []PGSSLKeyRecord
+}
+
+func (tbl *PGSSLKeyTable) insertKeys(db *sql.DB) error {
+	queryFmt := "INSERT INTO sslkey (deliveryservice, data, cdn, version) VALUES "
+	stride := 4
+	queryArgs := make([]interface{}, len(tbl.Records)*stride)
+	for i, record := range tbl.Records {
+		j := i * stride
+		queryArgs[j] = record.DeliveryService
+		queryArgs[j+1] = record.DataEncrypted
+		queryArgs[j+2] = record.CDN
+		queryArgs[j+3] = record.Version
+	}
+	return insertIntoTable(db, queryFmt, 4, queryArgs)
+}
+func (tbl *PGSSLKeyTable) gatherKeys(db *sql.DB) error {
+	sz, err := getSize(db, "sslkey")
+	if err != nil {
+		return fmt.Errorf("PGSSLKey gatherKeys unable to determine size of sslkey table: %v", err)
+	}
+	tbl.Records = make([]PGSSLKeyRecord, sz)
+
+	rows, err := db.Query("SELECT data, deliveryservice, cdn, version from sslkey")
+	if err != nil {
+		return fmt.Errorf("PGSSLKey gatherKeys unable to query: %v", err)
+	}
+	defer rows.Close()
+	i := 0
+	for rows.Next() {
+		if i > len(tbl.Records)-1 {
+			return errors.New("PGSSLKey gatherKeys: got more results than expected")
+		}
+		err := rows.Scan(&tbl.Records[i].DataEncrypted, &tbl.Records[i].DeliveryService, &tbl.Records[i].CDN, &tbl.Records[i].Version)
+		if err != nil {
+			return fmt.Errorf("PGSSLKey gatherKeys unable to scan row: %v", err)
+		}
+		i += 1
+	}
+	return nil
+}
+func (tbl *PGSSLKeyTable) decrypt(aesKey []byte) error {
+	for i, dns := range tbl.Records {
+		data, err := decryptInto(aesKey, dns.DataEncrypted, reflect.TypeOf(dns.Keys))
+		if err != nil {
+			return fmt.Errorf("unable to decrypt into keys: %v", err)
+		}
+		tbl.Records[i].Keys = *data.Interface().(*tc.DeliveryServiceSSLKeys)
+	}
+	return nil
+}
+func (tbl *PGSSLKeyTable) encrypt(aesKey []byte) error {
+	for i, dns := range tbl.Records {
+		data, err := json.Marshal(dns.Keys)
+		if err != nil {
+			return fmt.Errorf("encrypt issue marshalling keys: %v", err)
+		}
+		dat, err := encrypt(data, aesKey)
+		if err != nil {
+			return fmt.Errorf("encrypt error: %v", err)
+		}
+		tbl.Records[i].DataEncrypted = dat
+	}
+	return nil
+}
+func (tbl *PGSSLKeyTable) toGeneric() []SSLKey {
+	keys := make([]SSLKey, len(tbl.Records))
+
+	for i, record := range tbl.Records {
+		keys[i] = SSLKey{
+			DeliveryServiceSSLKeys: record.Keys,
+			CommonRecord:           record.CommonRecord,
+		}
+	}
+	return keys
+}
+func (tbl *PGSSLKeyTable) fromGeneric(keys []SSLKey) {
+	tbl.Records = make([]PGSSLKeyRecord, len(keys))
+
+	for i, key := range keys {
+		tbl.Records[i] = PGSSLKeyRecord{
+			Keys: key.DeliveryServiceSSLKeys,
+			PGCommonRecord: PGCommonRecord{
+				DataEncrypted: nil,
+				CommonRecord:  CommonRecord{},
+			},
+			DeliveryService: key.DeliveryService,
+			CDN:             key.CDN,
+		}
+	}
+}
+func (tbl *PGSSLKeyTable) validate() []string {
+	defaultKey := tc.DeliveryServiceSSLKeys{}
+	var errors []string
+	fmtStr := "SSL Key %v: %v"
+	for i, record := range tbl.Records {
+		if record.Keys == defaultKey {
+			errors = append(errors, fmt.Sprintf(fmtStr, i, "DS SSL Keys are default!"))
+		} else if record.Keys.Key == "" {
+			errors = append(errors, fmt.Sprintf(fmtStr, i, "Key is blank!"))
+		} else if record.Keys.CDN == "" {
+			errors = append(errors, fmt.Sprintf(fmtStr, i, "CDN is blank!"))
+		} else if record.Keys.DeliveryService == "" {
+			errors = append(errors, fmt.Sprintf(fmtStr, i, "DS is blank!"))
+		} else if record.DataEncrypted == nil {
+			errors = append(errors, fmt.Sprintf(fmtStr, i, "DataEncrypted is blank!"))
+		}
+	}
+	return errors
+}
+
+type PGURLSigKeyRecord struct {
+	Keys            tc.URLSigKeys
+	DeliveryService string
+	PGCommonRecord
+}
+type PGURLSigKeyTable struct {
+	Records []PGURLSigKeyRecord
+}
+
+func (tbl *PGURLSigKeyTable) insertKeys(db *sql.DB) error {
+	queryBase := "INSERT INTO url_sig_key (deliveryservice, data) VALUES "
+	stride := 2
+	queryArgs := make([]interface{}, len(tbl.Records)*stride)
+	for i, record := range tbl.Records {
+		j := i * stride
+		queryArgs[j] = record.DeliveryService
+		queryArgs[j+1] = record.DataEncrypted
+	}
+	return insertIntoTable(db, queryBase, stride, queryArgs)
+}
+func (tbl *PGURLSigKeyTable) gatherKeys(db *sql.DB) error {
+	sz, err := getSize(db, "url_sig_key")
+	if err != nil {
+		log.Println("PGURLSigKey gatherKeys: unable to determine url_sig_key table size")
+	}
+	tbl.Records = make([]PGURLSigKeyRecord, sz)
+
+	rows, err := db.Query("SELECT deliveryservice, data from url_sig_key")
+	if err != nil {
+		return fmt.Errorf("PGURLSigKey gatherKeys error creating query: %v", err)
+	}
+	defer rows.Close()
+	i := 0
+	for rows.Next() {
+		if i > len(tbl.Records)-1 {
+			return fmt.Errorf("PGURLSigKey gatherKeys: got more results than expected %v", len(tbl.Records))
+		}
+		err := rows.Scan(&tbl.Records[i].DeliveryService, &tbl.Records[i].DataEncrypted)
+		if err != nil {
+			return fmt.Errorf("PGURLSigKey gatherKeys: unable to scan row: %v", err)
+		}
+		i += 1
+	}
+	return nil
+}
+func (tbl *PGURLSigKeyTable) decrypt(aesKey []byte) error {
+	for i, sig := range tbl.Records {
+		data, err := decryptInto(aesKey, sig.DataEncrypted, reflect.TypeOf(sig.Keys))
+		if err != nil {
+			return fmt.Errorf("unable to decrypt into keys: %v", err)

Review comment:
       same as above RE: `%w` vs `%v`

##########
File path: tools/traffic_vault_migrate/postgres.go
##########
@@ -0,0 +1,716 @@
+package main
+
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+import (
+	"database/sql"
+	"encoding/base64"
+	"encoding/json"
+	"errors"
+	"fmt"
+	"github.com/apache/trafficcontrol/lib/go-tc"
+	util "github.com/apache/trafficcontrol/lib/go-util"
+	_ "github.com/lib/pq"
+	"log"
+	"reflect"
+	"strconv"
+	"strings"
+)
+
+type PGConfig struct {
+	Host     string `json:"host"`
+	Port     string `json:"port"`
+	User     string `json:"user"`
+	Password string `json:"password"`
+	SSLMode  string `json:"sslmode"`
+	Database string `json:"database"`
+	Key      string `json:"aesKey"`
+	AESKey   []byte
+}
+type PGBackend struct {
+	sslKey PGSSLKeyTable
+	dnssec PGDNSSecTable
+	uri    PGURISignKeyTable
+	url    PGURLSigKeyTable
+	cfg    PGConfig
+	db     *sql.DB
+}
+
+func (pg *PGBackend) String() string {
+	data := fmt.Sprintf("PG server %v@%v:%v\n", pg.cfg.User, pg.cfg.Host, pg.cfg.Port)
+	data += fmt.Sprintf("\tSSL Keys: %v\n", len(pg.sslKey.Records))
+	data += fmt.Sprintf("\tDNSSec Keys: %v\n", len(pg.dnssec.Records))
+	data += fmt.Sprintf("\tURI Keys: %v\n", len(pg.uri.Records))
+	data += fmt.Sprintf("\tURL Keys: %v\n", len(pg.url.Records))
+	return data
+}
+func (pg *PGBackend) Name() string {
+	return "PG"
+}
+func (pg *PGBackend) ReadConfig(s string) error {
+	genericCfg, err := UnmarshalConfig(s, reflect.TypeOf(pg.cfg))
+	if err != nil {
+		return err
+	}
+
+	pg.cfg = *genericCfg.Interface().(*PGConfig)
+
+	pg.cfg.AESKey, err = base64.StdEncoding.DecodeString(pg.cfg.Key)
+	if err != nil {
+		return fmt.Errorf("unable to decode PG AESKey: %v", err)
+	}
+	return nil
+}
+func (pg *PGBackend) Insert() error {
+	if err := pg.sslKey.insertKeys(pg.db); err != nil {
+		return err
+	}
+	if err := pg.dnssec.insertKeys(pg.db); err != nil {
+		return err
+	}
+	if err := pg.url.insertKeys(pg.db); err != nil {
+		return err
+	}
+	if err := pg.uri.insertKeys(pg.db); err != nil {
+		return err
+	}
+	return nil
+}
+func (pg *PGBackend) Start() error {
+	sqlStr := fmt.Sprintf("postgres://%s:%s@%s:%s/%s?sslmode=%s", pg.cfg.User, pg.cfg.Password, pg.cfg.Host, pg.cfg.Port, pg.cfg.Database, pg.cfg.SSLMode)
+	db, err := sql.Open("postgres", sqlStr)
+	if err != nil {
+		return fmt.Errorf("Unable to start PG client: %v", err)
+	}
+
+	pg.db = db
+	pg.sslKey = PGSSLKeyTable{}
+	pg.dnssec = PGDNSSecTable{}
+	pg.url = PGURLSigKeyTable{}
+	pg.uri = PGURISignKeyTable{}
+
+	return nil
+}
+func (pg *PGBackend) ValidateKey() []string {
+	var errors []string
+	if errs := pg.sslKey.validate(); errs != nil {
+		errors = append(errors, errs...)
+	}
+	if errs := pg.dnssec.validate(); errs != nil {
+		errors = append(errors, errs...)
+	}
+	if errs := pg.uri.validate(); errs != nil {
+		errors = append(errors, errs...)
+	}
+	if errs := pg.url.validate(); errs != nil {
+		errors = append(errors, errs...)
+	}
+	return errors
+}
+func (pg *PGBackend) Stop() error {
+	return pg.db.Close()
+}
+func (pg *PGBackend) Ping() error {
+	return pg.db.Ping()
+}
+func (pg *PGBackend) Fetch() error {
+	if err := pg.sslKey.gatherKeys(pg.db); err != nil {
+		return err
+	}
+
+	if err := pg.dnssec.gatherKeys(pg.db); err != nil {
+		return err
+	}
+
+	if err := pg.url.gatherKeys(pg.db); err != nil {
+		return err
+	}
+
+	if err := pg.uri.gatherKeys(pg.db); err != nil {
+		return err
+	}
+
+	return nil
+}
+func (pg *PGBackend) GetSSLKeys() ([]SSLKey, error) {
+	if err := pg.sslKey.decrypt(pg.cfg.AESKey); err != nil {
+		return nil, err
+	}
+	return pg.sslKey.toGeneric(), nil
+}
+func (pg *PGBackend) SetSSLKeys(keys []SSLKey) error {
+	pg.sslKey.fromGeneric(keys)
+	return pg.sslKey.encrypt(pg.cfg.AESKey)
+}
+func (pg *PGBackend) GetDNSSecKeys() ([]DNSSecKey, error) {
+	if err := pg.dnssec.decrypt(pg.cfg.AESKey); err != nil {
+		return nil, err
+	}
+	return pg.dnssec.toGeneric(), nil
+}
+func (pg *PGBackend) SetDNSSecKeys(keys []DNSSecKey) error {
+	pg.dnssec.fromGeneric(keys)
+	return pg.dnssec.encrypt(pg.cfg.AESKey)
+}
+func (pg *PGBackend) GetURISignKeys() ([]URISignKey, error) {
+	if err := pg.uri.decrypt(pg.cfg.AESKey); err != nil {
+		return nil, err
+	}
+	return pg.uri.toGeneric(), nil
+}
+func (pg *PGBackend) SetURISignKeys(keys []URISignKey) error {
+	pg.uri.fromGeneric(keys)
+	return pg.uri.encrypt(pg.cfg.AESKey)
+}
+func (pg *PGBackend) GetURLSigKeys() ([]URLSigKey, error) {
+	if err := pg.url.decrypt(pg.cfg.AESKey); err != nil {
+		return nil, err
+	}
+	return pg.url.toGeneric(), nil
+}
+func (pg *PGBackend) SetURLSigKeys(keys []URLSigKey) error {
+	pg.url.fromGeneric(keys)
+	return pg.url.encrypt(pg.cfg.AESKey)
+}
+
+type PGCommonRecord struct {
+	DataEncrypted []byte
+	CommonRecord
+}
+
+type PGDNSSecRecord struct {
+	Key tc.DNSSECKeysTrafficVault
+	CDN string
+	PGCommonRecord
+}
+type PGDNSSecTable struct {
+	Records []PGDNSSecRecord
+}
+
+func (tbl *PGDNSSecTable) gatherKeys(db *sql.DB) error {
+	sz, err := getSize(db, "dnssec")
+	if err != nil {
+		log.Println("PGDNSSec gatherKeys: unable to determine size of dnssec table")
+	}
+	tbl.Records = make([]PGDNSSecRecord, sz)
+
+	rows, err := db.Query("SELECT cdn, data from dnssec")
+	if err != nil {
+		return fmt.Errorf("PGDNSSec gatherKeys: unable to query: %v", err)
+	}
+	defer rows.Close()
+	i := 0
+	for rows.Next() {
+		if i > len(tbl.Records)-1 {
+			return fmt.Errorf("PGDNSSec gatherKeys got more results than expected %v", len(tbl.Records))
+		}
+		err := rows.Scan(&tbl.Records[i].CDN, &tbl.Records[i].DataEncrypted)
+		if err != nil {
+			return fmt.Errorf("PGDNSSec gatherKeys unable to scan row: %v", err)
+		}
+		i += 1
+	}
+	return nil
+}
+func (tbl *PGDNSSecTable) decrypt(aesKey []byte) error {
+	for i, _ := range tbl.Records {
+		data, err := decryptInto(aesKey, tbl.Records[i].DataEncrypted, reflect.TypeOf(tbl.Records[i].Key))
+		if err != nil {
+			return fmt.Errorf("unable to decrypt into keys: %v", err)
+		}
+		tbl.Records[i].Key = *data.Interface().(*tc.DNSSECKeysTrafficVault)
+	}
+	return nil
+}
+func (tbl *PGDNSSecTable) encrypt(aesKey []byte) error {
+	for i, dns := range tbl.Records {
+		data, err := json.Marshal(&dns.Key)
+		if err != nil {
+			return fmt.Errorf("encrypt issue marshalling keys: %v", err)
+		}
+		dat, err := encrypt(data, aesKey)
+		if err != nil {
+			return fmt.Errorf("encrypt error: %v", err)
+		}
+		tbl.Records[i].DataEncrypted = dat
+	}
+	return nil
+}
+func (tbl *PGDNSSecTable) toGeneric() []DNSSecKey {
+	keys := make([]DNSSecKey, len(tbl.Records))
+
+	for i, record := range tbl.Records {
+		keys[i] = DNSSecKey{
+			CDN:                    record.CDN,
+			DNSSECKeysTrafficVault: record.Key,
+			CommonRecord:           record.CommonRecord,
+		}
+	}
+
+	return keys
+}
+func (tbl *PGDNSSecTable) fromGeneric(keys []DNSSecKey) {
+	tbl.Records = make([]PGDNSSecRecord, len(keys))
+
+	for i, key := range keys {
+		tbl.Records[i] = PGDNSSecRecord{
+			Key: key.DNSSECKeysTrafficVault,
+			CDN: key.CDN,
+			PGCommonRecord: PGCommonRecord{
+				DataEncrypted: nil,
+				CommonRecord:  CommonRecord{},
+			},
+		}
+	}
+}
+func (tbl *PGDNSSecTable) validate() []string {
+	for i, record := range tbl.Records {
+		if record.DataEncrypted == nil && len(record.Key) > 0 {
+			return []string{fmt.Sprintf("DNSSEC Key %v: DataEncrypted is blank!", i)}
+		}
+	}
+	return nil
+}
+func (tbl *PGDNSSecTable) insertKeys(db *sql.DB) error {
+	queryFmt := "INSERT INTO dnssec (cdn, data) VALUES "
+	stride := 2
+	queryArgs := make([]interface{}, len(tbl.Records)*stride)
+	for i, record := range tbl.Records {
+		j := i * stride
+		queryArgs[j] = record.CDN
+		queryArgs[j+1] = record.DataEncrypted
+	}
+	return insertIntoTable(db, queryFmt, stride, queryArgs)
+}
+
+type PGSSLKeyRecord struct {
+	Keys tc.DeliveryServiceSSLKeys
+	PGCommonRecord
+
+	// These records are stored on the table but are duplicated
+	DeliveryService string
+	Version         string
+	CDN             string
+}
+type PGSSLKeyTable struct {
+	Records []PGSSLKeyRecord
+}
+
+func (tbl *PGSSLKeyTable) insertKeys(db *sql.DB) error {
+	queryFmt := "INSERT INTO sslkey (deliveryservice, data, cdn, version) VALUES "
+	stride := 4
+	queryArgs := make([]interface{}, len(tbl.Records)*stride)
+	for i, record := range tbl.Records {
+		j := i * stride
+		queryArgs[j] = record.DeliveryService
+		queryArgs[j+1] = record.DataEncrypted
+		queryArgs[j+2] = record.CDN
+		queryArgs[j+3] = record.Version
+	}
+	return insertIntoTable(db, queryFmt, 4, queryArgs)
+}
+func (tbl *PGSSLKeyTable) gatherKeys(db *sql.DB) error {
+	sz, err := getSize(db, "sslkey")
+	if err != nil {
+		return fmt.Errorf("PGSSLKey gatherKeys unable to determine size of sslkey table: %v", err)
+	}
+	tbl.Records = make([]PGSSLKeyRecord, sz)
+
+	rows, err := db.Query("SELECT data, deliveryservice, cdn, version from sslkey")
+	if err != nil {
+		return fmt.Errorf("PGSSLKey gatherKeys unable to query: %v", err)
+	}
+	defer rows.Close()
+	i := 0
+	for rows.Next() {
+		if i > len(tbl.Records)-1 {
+			return errors.New("PGSSLKey gatherKeys: got more results than expected")
+		}
+		err := rows.Scan(&tbl.Records[i].DataEncrypted, &tbl.Records[i].DeliveryService, &tbl.Records[i].CDN, &tbl.Records[i].Version)
+		if err != nil {
+			return fmt.Errorf("PGSSLKey gatherKeys unable to scan row: %v", err)
+		}
+		i += 1
+	}
+	return nil
+}
+func (tbl *PGSSLKeyTable) decrypt(aesKey []byte) error {
+	for i, dns := range tbl.Records {
+		data, err := decryptInto(aesKey, dns.DataEncrypted, reflect.TypeOf(dns.Keys))
+		if err != nil {
+			return fmt.Errorf("unable to decrypt into keys: %v", err)
+		}
+		tbl.Records[i].Keys = *data.Interface().(*tc.DeliveryServiceSSLKeys)
+	}
+	return nil
+}
+func (tbl *PGSSLKeyTable) encrypt(aesKey []byte) error {
+	for i, dns := range tbl.Records {
+		data, err := json.Marshal(dns.Keys)
+		if err != nil {
+			return fmt.Errorf("encrypt issue marshalling keys: %v", err)
+		}
+		dat, err := encrypt(data, aesKey)
+		if err != nil {
+			return fmt.Errorf("encrypt error: %v", err)
+		}
+		tbl.Records[i].DataEncrypted = dat
+	}
+	return nil
+}
+func (tbl *PGSSLKeyTable) toGeneric() []SSLKey {
+	keys := make([]SSLKey, len(tbl.Records))
+
+	for i, record := range tbl.Records {
+		keys[i] = SSLKey{
+			DeliveryServiceSSLKeys: record.Keys,
+			CommonRecord:           record.CommonRecord,
+		}
+	}
+	return keys
+}
+func (tbl *PGSSLKeyTable) fromGeneric(keys []SSLKey) {
+	tbl.Records = make([]PGSSLKeyRecord, len(keys))
+
+	for i, key := range keys {
+		tbl.Records[i] = PGSSLKeyRecord{
+			Keys: key.DeliveryServiceSSLKeys,
+			PGCommonRecord: PGCommonRecord{
+				DataEncrypted: nil,
+				CommonRecord:  CommonRecord{},
+			},
+			DeliveryService: key.DeliveryService,
+			CDN:             key.CDN,
+		}
+	}
+}
+func (tbl *PGSSLKeyTable) validate() []string {
+	defaultKey := tc.DeliveryServiceSSLKeys{}
+	var errors []string
+	fmtStr := "SSL Key %v: %v"
+	for i, record := range tbl.Records {
+		if record.Keys == defaultKey {
+			errors = append(errors, fmt.Sprintf(fmtStr, i, "DS SSL Keys are default!"))
+		} else if record.Keys.Key == "" {
+			errors = append(errors, fmt.Sprintf(fmtStr, i, "Key is blank!"))
+		} else if record.Keys.CDN == "" {
+			errors = append(errors, fmt.Sprintf(fmtStr, i, "CDN is blank!"))
+		} else if record.Keys.DeliveryService == "" {
+			errors = append(errors, fmt.Sprintf(fmtStr, i, "DS is blank!"))
+		} else if record.DataEncrypted == nil {
+			errors = append(errors, fmt.Sprintf(fmtStr, i, "DataEncrypted is blank!"))
+		}
+	}
+	return errors
+}
+
+type PGURLSigKeyRecord struct {
+	Keys            tc.URLSigKeys
+	DeliveryService string
+	PGCommonRecord
+}
+type PGURLSigKeyTable struct {
+	Records []PGURLSigKeyRecord
+}
+
+func (tbl *PGURLSigKeyTable) insertKeys(db *sql.DB) error {
+	queryBase := "INSERT INTO url_sig_key (deliveryservice, data) VALUES "
+	stride := 2
+	queryArgs := make([]interface{}, len(tbl.Records)*stride)
+	for i, record := range tbl.Records {
+		j := i * stride
+		queryArgs[j] = record.DeliveryService
+		queryArgs[j+1] = record.DataEncrypted
+	}
+	return insertIntoTable(db, queryBase, stride, queryArgs)
+}
+func (tbl *PGURLSigKeyTable) gatherKeys(db *sql.DB) error {
+	sz, err := getSize(db, "url_sig_key")
+	if err != nil {
+		log.Println("PGURLSigKey gatherKeys: unable to determine url_sig_key table size")
+	}
+	tbl.Records = make([]PGURLSigKeyRecord, sz)
+
+	rows, err := db.Query("SELECT deliveryservice, data from url_sig_key")
+	if err != nil {
+		return fmt.Errorf("PGURLSigKey gatherKeys error creating query: %v", err)
+	}
+	defer rows.Close()
+	i := 0
+	for rows.Next() {
+		if i > len(tbl.Records)-1 {
+			return fmt.Errorf("PGURLSigKey gatherKeys: got more results than expected %v", len(tbl.Records))
+		}
+		err := rows.Scan(&tbl.Records[i].DeliveryService, &tbl.Records[i].DataEncrypted)
+		if err != nil {
+			return fmt.Errorf("PGURLSigKey gatherKeys: unable to scan row: %v", err)
+		}
+		i += 1
+	}
+	return nil
+}
+func (tbl *PGURLSigKeyTable) decrypt(aesKey []byte) error {
+	for i, sig := range tbl.Records {
+		data, err := decryptInto(aesKey, sig.DataEncrypted, reflect.TypeOf(sig.Keys))
+		if err != nil {
+			return fmt.Errorf("unable to decrypt into keys: %v", err)
+		}
+
+		tbl.Records[i].Keys = *data.Interface().(*tc.URLSigKeys)
+	}
+	return nil
+}
+func (tbl *PGURLSigKeyTable) encrypt(aesKey []byte) error {
+	for i, sig := range tbl.Records {
+		data, err := json.Marshal(&sig.Keys)
+		if err != nil {
+			return fmt.Errorf("encrypt issue marshalling keys: %v", err)
+		}
+
+		dat, err := encrypt(data, aesKey)
+		if err != nil {
+			return fmt.Errorf("encrypt error: %v", err)
+		}
+		tbl.Records[i].DataEncrypted = dat
+	}
+	return nil
+}
+func (tbl *PGURLSigKeyTable) toGeneric() []URLSigKey {
+	keys := make([]URLSigKey, len(tbl.Records))
+
+	for i, record := range tbl.Records {
+		keys[i] = URLSigKey{
+			DeliveryService: record.DeliveryService,
+			URLSigKeys:      record.Keys,
+			CommonRecord:    record.CommonRecord,
+		}
+	}
+	return keys
+}
+func (tbl *PGURLSigKeyTable) fromGeneric(keys []URLSigKey) {
+	tbl.Records = make([]PGURLSigKeyRecord, len(keys))
+
+	for i, key := range keys {
+		tbl.Records[i] = PGURLSigKeyRecord{
+			Keys:            key.URLSigKeys,
+			DeliveryService: key.DeliveryService,
+			PGCommonRecord: PGCommonRecord{
+				DataEncrypted: nil,
+				CommonRecord:  CommonRecord{},
+			},
+		}
+	}
+}
+func (tbl *PGURLSigKeyTable) validate() []string {
+	for i, record := range tbl.Records {
+		if record.DataEncrypted == nil && len(record.Keys) > 0 {
+			return []string{fmt.Sprintf("URl Sig Key %v: DataEncrypted is blank!", i)}
+		}
+	}
+	return nil
+}
+
+type PGURISignKeyRecord struct {
+	Keys            map[string]tc.URISignerKeyset
+	DeliveryService string
+	PGCommonRecord
+}
+type PGURISignKeyTable struct {
+	Records []PGURISignKeyRecord
+}
+
+func (tbl *PGURISignKeyTable) insertKeys(db *sql.DB) error {
+	queryFmt := "INSERT INTO uri_signing_key (deliveryservice, data) VALUES "
+	stride := 2
+	queryArgs := make([]interface{}, len(tbl.Records)*stride)
+	for i, record := range tbl.Records {
+		j := i * stride
+		queryArgs[j] = record.DeliveryService
+		queryArgs[j+1] = record.DataEncrypted
+	}
+	return insertIntoTable(db, queryFmt, stride, queryArgs)
+}
+func (tbl *PGURISignKeyTable) gatherKeys(db *sql.DB) error {
+	sz, err := getSize(db, "uri_signing_key")
+	if err != nil {
+		log.Println("PGURISignKey gatherKeys: unable to determine size of uri_signing_key table")
+	}
+	tbl.Records = make([]PGURISignKeyRecord, sz)
+
+	rows, err := db.Query("SELECT deliveryservice, data from uri_signing_key")
+	if err != nil {
+		return fmt.Errorf("PGURISignKey gatherKeys error while query: %v", err)
+	}
+	defer rows.Close()
+	i := 0
+	for rows.Next() {
+		if i > len(tbl.Records)-1 {
+			return fmt.Errorf("PGURISignKey gatherKeys: got more results than expected %v", len(tbl.Records))
+		}
+		err := rows.Scan(&tbl.Records[i].DeliveryService, &tbl.Records[i].DataEncrypted)
+		if err != nil {
+			return fmt.Errorf("PGURISignKey gatherKeys: unable to scan row: %v", err)
+		}
+		i += 1
+	}
+	return nil
+}
+func (tbl *PGURISignKeyTable) decrypt(aesKey []byte) error {
+	for i, sign := range tbl.Records {
+		data, err := decryptInto(aesKey, sign.DataEncrypted, reflect.TypeOf(sign.Keys))
+		if err != nil {
+			return fmt.Errorf("unable to decrypt into keys: %v", err)

Review comment:
       same as above RE: `%w` vs `%v`

##########
File path: tools/traffic_vault_migrate/postgres.go
##########
@@ -0,0 +1,716 @@
+package main
+
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+import (
+	"database/sql"
+	"encoding/base64"
+	"encoding/json"
+	"errors"
+	"fmt"
+	"github.com/apache/trafficcontrol/lib/go-tc"
+	util "github.com/apache/trafficcontrol/lib/go-util"
+	_ "github.com/lib/pq"
+	"log"
+	"reflect"
+	"strconv"
+	"strings"
+)
+
+type PGConfig struct {
+	Host     string `json:"host"`
+	Port     string `json:"port"`
+	User     string `json:"user"`
+	Password string `json:"password"`
+	SSLMode  string `json:"sslmode"`
+	Database string `json:"database"`
+	Key      string `json:"aesKey"`
+	AESKey   []byte
+}
+type PGBackend struct {
+	sslKey PGSSLKeyTable
+	dnssec PGDNSSecTable
+	uri    PGURISignKeyTable
+	url    PGURLSigKeyTable
+	cfg    PGConfig
+	db     *sql.DB
+}
+
+func (pg *PGBackend) String() string {

Review comment:
       GoDoc? Also, does this receiver need to be a pointer?
   
   (Typically for `func (this Thing) String() string` methods I just write "String implements the fmt.Stringer interface.")

##########
File path: tools/traffic_vault_migrate/postgres.go
##########
@@ -0,0 +1,716 @@
+package main
+
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+import (
+	"database/sql"
+	"encoding/base64"
+	"encoding/json"
+	"errors"
+	"fmt"
+	"github.com/apache/trafficcontrol/lib/go-tc"
+	util "github.com/apache/trafficcontrol/lib/go-util"
+	_ "github.com/lib/pq"
+	"log"
+	"reflect"
+	"strconv"
+	"strings"
+)
+
+type PGConfig struct {
+	Host     string `json:"host"`
+	Port     string `json:"port"`
+	User     string `json:"user"`
+	Password string `json:"password"`
+	SSLMode  string `json:"sslmode"`
+	Database string `json:"database"`
+	Key      string `json:"aesKey"`
+	AESKey   []byte
+}
+type PGBackend struct {
+	sslKey PGSSLKeyTable
+	dnssec PGDNSSecTable
+	uri    PGURISignKeyTable
+	url    PGURLSigKeyTable
+	cfg    PGConfig
+	db     *sql.DB
+}
+
+func (pg *PGBackend) String() string {
+	data := fmt.Sprintf("PG server %v@%v:%v\n", pg.cfg.User, pg.cfg.Host, pg.cfg.Port)
+	data += fmt.Sprintf("\tSSL Keys: %v\n", len(pg.sslKey.Records))
+	data += fmt.Sprintf("\tDNSSec Keys: %v\n", len(pg.dnssec.Records))
+	data += fmt.Sprintf("\tURI Keys: %v\n", len(pg.uri.Records))
+	data += fmt.Sprintf("\tURL Keys: %v\n", len(pg.url.Records))
+	return data
+}
+func (pg *PGBackend) Name() string {
+	return "PG"
+}
+func (pg *PGBackend) ReadConfig(s string) error {

Review comment:
       GoDoc?

##########
File path: tools/traffic_vault_migrate/postgres.go
##########
@@ -0,0 +1,716 @@
+package main
+
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+import (
+	"database/sql"
+	"encoding/base64"
+	"encoding/json"
+	"errors"
+	"fmt"
+	"github.com/apache/trafficcontrol/lib/go-tc"
+	util "github.com/apache/trafficcontrol/lib/go-util"
+	_ "github.com/lib/pq"
+	"log"
+	"reflect"
+	"strconv"
+	"strings"
+)
+
+type PGConfig struct {
+	Host     string `json:"host"`
+	Port     string `json:"port"`
+	User     string `json:"user"`
+	Password string `json:"password"`
+	SSLMode  string `json:"sslmode"`
+	Database string `json:"database"`
+	Key      string `json:"aesKey"`
+	AESKey   []byte
+}
+type PGBackend struct {
+	sslKey PGSSLKeyTable
+	dnssec PGDNSSecTable
+	uri    PGURISignKeyTable
+	url    PGURLSigKeyTable
+	cfg    PGConfig
+	db     *sql.DB
+}
+
+func (pg *PGBackend) String() string {
+	data := fmt.Sprintf("PG server %v@%v:%v\n", pg.cfg.User, pg.cfg.Host, pg.cfg.Port)
+	data += fmt.Sprintf("\tSSL Keys: %v\n", len(pg.sslKey.Records))
+	data += fmt.Sprintf("\tDNSSec Keys: %v\n", len(pg.dnssec.Records))
+	data += fmt.Sprintf("\tURI Keys: %v\n", len(pg.uri.Records))
+	data += fmt.Sprintf("\tURL Keys: %v\n", len(pg.url.Records))
+	return data
+}
+func (pg *PGBackend) Name() string {
+	return "PG"
+}
+func (pg *PGBackend) ReadConfig(s string) error {
+	genericCfg, err := UnmarshalConfig(s, reflect.TypeOf(pg.cfg))

Review comment:
       The type is known; we don't need to use reflection here, do we?

##########
File path: tools/traffic_vault_migrate/postgres.go
##########
@@ -0,0 +1,716 @@
+package main
+
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+import (
+	"database/sql"
+	"encoding/base64"
+	"encoding/json"
+	"errors"
+	"fmt"
+	"github.com/apache/trafficcontrol/lib/go-tc"
+	util "github.com/apache/trafficcontrol/lib/go-util"
+	_ "github.com/lib/pq"
+	"log"
+	"reflect"
+	"strconv"
+	"strings"
+)
+
+type PGConfig struct {
+	Host     string `json:"host"`
+	Port     string `json:"port"`
+	User     string `json:"user"`
+	Password string `json:"password"`
+	SSLMode  string `json:"sslmode"`
+	Database string `json:"database"`
+	Key      string `json:"aesKey"`
+	AESKey   []byte
+}
+type PGBackend struct {
+	sslKey PGSSLKeyTable
+	dnssec PGDNSSecTable
+	uri    PGURISignKeyTable
+	url    PGURLSigKeyTable
+	cfg    PGConfig
+	db     *sql.DB
+}
+
+func (pg *PGBackend) String() string {
+	data := fmt.Sprintf("PG server %v@%v:%v\n", pg.cfg.User, pg.cfg.Host, pg.cfg.Port)
+	data += fmt.Sprintf("\tSSL Keys: %v\n", len(pg.sslKey.Records))
+	data += fmt.Sprintf("\tDNSSec Keys: %v\n", len(pg.dnssec.Records))
+	data += fmt.Sprintf("\tURI Keys: %v\n", len(pg.uri.Records))
+	data += fmt.Sprintf("\tURL Keys: %v\n", len(pg.url.Records))
+	return data
+}
+func (pg *PGBackend) Name() string {
+	return "PG"
+}
+func (pg *PGBackend) ReadConfig(s string) error {
+	genericCfg, err := UnmarshalConfig(s, reflect.TypeOf(pg.cfg))
+	if err != nil {
+		return err
+	}
+
+	pg.cfg = *genericCfg.Interface().(*PGConfig)
+
+	pg.cfg.AESKey, err = base64.StdEncoding.DecodeString(pg.cfg.Key)
+	if err != nil {
+		return fmt.Errorf("unable to decode PG AESKey: %v", err)
+	}
+	return nil
+}
+func (pg *PGBackend) Insert() error {
+	if err := pg.sslKey.insertKeys(pg.db); err != nil {
+		return err
+	}
+	if err := pg.dnssec.insertKeys(pg.db); err != nil {
+		return err
+	}
+	if err := pg.url.insertKeys(pg.db); err != nil {
+		return err
+	}
+	if err := pg.uri.insertKeys(pg.db); err != nil {
+		return err
+	}
+	return nil
+}
+func (pg *PGBackend) Start() error {
+	sqlStr := fmt.Sprintf("postgres://%s:%s@%s:%s/%s?sslmode=%s", pg.cfg.User, pg.cfg.Password, pg.cfg.Host, pg.cfg.Port, pg.cfg.Database, pg.cfg.SSLMode)
+	db, err := sql.Open("postgres", sqlStr)
+	if err != nil {
+		return fmt.Errorf("Unable to start PG client: %v", err)
+	}
+
+	pg.db = db
+	pg.sslKey = PGSSLKeyTable{}
+	pg.dnssec = PGDNSSecTable{}
+	pg.url = PGURLSigKeyTable{}
+	pg.uri = PGURISignKeyTable{}
+
+	return nil
+}
+func (pg *PGBackend) ValidateKey() []string {
+	var errors []string
+	if errs := pg.sslKey.validate(); errs != nil {
+		errors = append(errors, errs...)
+	}
+	if errs := pg.dnssec.validate(); errs != nil {
+		errors = append(errors, errs...)
+	}
+	if errs := pg.uri.validate(); errs != nil {
+		errors = append(errors, errs...)
+	}
+	if errs := pg.url.validate(); errs != nil {
+		errors = append(errors, errs...)
+	}
+	return errors
+}
+func (pg *PGBackend) Stop() error {

Review comment:
       GoDoc?

##########
File path: tools/traffic_vault_migrate/postgres.go
##########
@@ -0,0 +1,716 @@
+package main
+
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+import (
+	"database/sql"
+	"encoding/base64"
+	"encoding/json"
+	"errors"
+	"fmt"
+	"github.com/apache/trafficcontrol/lib/go-tc"
+	util "github.com/apache/trafficcontrol/lib/go-util"
+	_ "github.com/lib/pq"
+	"log"
+	"reflect"
+	"strconv"
+	"strings"
+)
+
+type PGConfig struct {
+	Host     string `json:"host"`
+	Port     string `json:"port"`
+	User     string `json:"user"`
+	Password string `json:"password"`
+	SSLMode  string `json:"sslmode"`
+	Database string `json:"database"`
+	Key      string `json:"aesKey"`
+	AESKey   []byte
+}
+type PGBackend struct {
+	sslKey PGSSLKeyTable
+	dnssec PGDNSSecTable
+	uri    PGURISignKeyTable
+	url    PGURLSigKeyTable
+	cfg    PGConfig
+	db     *sql.DB
+}
+
+func (pg *PGBackend) String() string {
+	data := fmt.Sprintf("PG server %v@%v:%v\n", pg.cfg.User, pg.cfg.Host, pg.cfg.Port)
+	data += fmt.Sprintf("\tSSL Keys: %v\n", len(pg.sslKey.Records))
+	data += fmt.Sprintf("\tDNSSec Keys: %v\n", len(pg.dnssec.Records))
+	data += fmt.Sprintf("\tURI Keys: %v\n", len(pg.uri.Records))
+	data += fmt.Sprintf("\tURL Keys: %v\n", len(pg.url.Records))
+	return data
+}
+func (pg *PGBackend) Name() string {
+	return "PG"
+}
+func (pg *PGBackend) ReadConfig(s string) error {
+	genericCfg, err := UnmarshalConfig(s, reflect.TypeOf(pg.cfg))
+	if err != nil {
+		return err
+	}
+
+	pg.cfg = *genericCfg.Interface().(*PGConfig)
+
+	pg.cfg.AESKey, err = base64.StdEncoding.DecodeString(pg.cfg.Key)
+	if err != nil {
+		return fmt.Errorf("unable to decode PG AESKey: %v", err)
+	}
+	return nil
+}
+func (pg *PGBackend) Insert() error {
+	if err := pg.sslKey.insertKeys(pg.db); err != nil {
+		return err
+	}
+	if err := pg.dnssec.insertKeys(pg.db); err != nil {
+		return err
+	}
+	if err := pg.url.insertKeys(pg.db); err != nil {
+		return err
+	}
+	if err := pg.uri.insertKeys(pg.db); err != nil {
+		return err
+	}
+	return nil
+}
+func (pg *PGBackend) Start() error {
+	sqlStr := fmt.Sprintf("postgres://%s:%s@%s:%s/%s?sslmode=%s", pg.cfg.User, pg.cfg.Password, pg.cfg.Host, pg.cfg.Port, pg.cfg.Database, pg.cfg.SSLMode)
+	db, err := sql.Open("postgres", sqlStr)
+	if err != nil {
+		return fmt.Errorf("Unable to start PG client: %v", err)
+	}
+
+	pg.db = db
+	pg.sslKey = PGSSLKeyTable{}
+	pg.dnssec = PGDNSSecTable{}
+	pg.url = PGURLSigKeyTable{}
+	pg.uri = PGURISignKeyTable{}
+
+	return nil
+}
+func (pg *PGBackend) ValidateKey() []string {
+	var errors []string
+	if errs := pg.sslKey.validate(); errs != nil {
+		errors = append(errors, errs...)
+	}
+	if errs := pg.dnssec.validate(); errs != nil {
+		errors = append(errors, errs...)
+	}
+	if errs := pg.uri.validate(); errs != nil {
+		errors = append(errors, errs...)
+	}
+	if errs := pg.url.validate(); errs != nil {
+		errors = append(errors, errs...)
+	}
+	return errors
+}
+func (pg *PGBackend) Stop() error {

Review comment:
       Do you think this would benefit from being an [`io.Closer`](https://pkg.go.dev/io#Closer) interface instead of using `Stop`?

##########
File path: tools/traffic_vault_migrate/postgres.go
##########
@@ -0,0 +1,716 @@
+package main
+
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+import (
+	"database/sql"
+	"encoding/base64"
+	"encoding/json"
+	"errors"
+	"fmt"
+	"github.com/apache/trafficcontrol/lib/go-tc"
+	util "github.com/apache/trafficcontrol/lib/go-util"
+	_ "github.com/lib/pq"
+	"log"
+	"reflect"
+	"strconv"
+	"strings"
+)
+
+type PGConfig struct {
+	Host     string `json:"host"`
+	Port     string `json:"port"`
+	User     string `json:"user"`
+	Password string `json:"password"`
+	SSLMode  string `json:"sslmode"`
+	Database string `json:"database"`
+	Key      string `json:"aesKey"`
+	AESKey   []byte
+}
+type PGBackend struct {

Review comment:
       GoDoc?

##########
File path: tools/traffic_vault_migrate/postgres.go
##########
@@ -0,0 +1,716 @@
+package main
+
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+import (
+	"database/sql"
+	"encoding/base64"
+	"encoding/json"
+	"errors"
+	"fmt"
+	"github.com/apache/trafficcontrol/lib/go-tc"
+	util "github.com/apache/trafficcontrol/lib/go-util"
+	_ "github.com/lib/pq"
+	"log"
+	"reflect"
+	"strconv"
+	"strings"
+)
+
+type PGConfig struct {
+	Host     string `json:"host"`
+	Port     string `json:"port"`
+	User     string `json:"user"`
+	Password string `json:"password"`
+	SSLMode  string `json:"sslmode"`
+	Database string `json:"database"`
+	Key      string `json:"aesKey"`
+	AESKey   []byte
+}
+type PGBackend struct {
+	sslKey PGSSLKeyTable
+	dnssec PGDNSSecTable
+	uri    PGURISignKeyTable
+	url    PGURLSigKeyTable
+	cfg    PGConfig
+	db     *sql.DB
+}
+
+func (pg *PGBackend) String() string {
+	data := fmt.Sprintf("PG server %v@%v:%v\n", pg.cfg.User, pg.cfg.Host, pg.cfg.Port)
+	data += fmt.Sprintf("\tSSL Keys: %v\n", len(pg.sslKey.Records))
+	data += fmt.Sprintf("\tDNSSec Keys: %v\n", len(pg.dnssec.Records))
+	data += fmt.Sprintf("\tURI Keys: %v\n", len(pg.uri.Records))
+	data += fmt.Sprintf("\tURL Keys: %v\n", len(pg.url.Records))
+	return data
+}
+func (pg *PGBackend) Name() string {

Review comment:
       GoDoc?

##########
File path: tools/traffic_vault_migrate/postgres.go
##########
@@ -0,0 +1,716 @@
+package main
+
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+import (
+	"database/sql"
+	"encoding/base64"
+	"encoding/json"
+	"errors"
+	"fmt"
+	"github.com/apache/trafficcontrol/lib/go-tc"
+	util "github.com/apache/trafficcontrol/lib/go-util"
+	_ "github.com/lib/pq"
+	"log"
+	"reflect"
+	"strconv"
+	"strings"
+)
+
+type PGConfig struct {
+	Host     string `json:"host"`
+	Port     string `json:"port"`
+	User     string `json:"user"`
+	Password string `json:"password"`
+	SSLMode  string `json:"sslmode"`
+	Database string `json:"database"`
+	Key      string `json:"aesKey"`
+	AESKey   []byte
+}
+type PGBackend struct {
+	sslKey PGSSLKeyTable
+	dnssec PGDNSSecTable
+	uri    PGURISignKeyTable
+	url    PGURLSigKeyTable
+	cfg    PGConfig
+	db     *sql.DB
+}
+
+func (pg *PGBackend) String() string {
+	data := fmt.Sprintf("PG server %v@%v:%v\n", pg.cfg.User, pg.cfg.Host, pg.cfg.Port)
+	data += fmt.Sprintf("\tSSL Keys: %v\n", len(pg.sslKey.Records))
+	data += fmt.Sprintf("\tDNSSec Keys: %v\n", len(pg.dnssec.Records))
+	data += fmt.Sprintf("\tURI Keys: %v\n", len(pg.uri.Records))
+	data += fmt.Sprintf("\tURL Keys: %v\n", len(pg.url.Records))
+	return data
+}
+func (pg *PGBackend) Name() string {
+	return "PG"
+}
+func (pg *PGBackend) ReadConfig(s string) error {
+	genericCfg, err := UnmarshalConfig(s, reflect.TypeOf(pg.cfg))
+	if err != nil {
+		return err
+	}
+
+	pg.cfg = *genericCfg.Interface().(*PGConfig)
+
+	pg.cfg.AESKey, err = base64.StdEncoding.DecodeString(pg.cfg.Key)
+	if err != nil {
+		return fmt.Errorf("unable to decode PG AESKey: %v", err)
+	}
+	return nil
+}
+func (pg *PGBackend) Insert() error {
+	if err := pg.sslKey.insertKeys(pg.db); err != nil {
+		return err
+	}
+	if err := pg.dnssec.insertKeys(pg.db); err != nil {
+		return err
+	}
+	if err := pg.url.insertKeys(pg.db); err != nil {
+		return err
+	}
+	if err := pg.uri.insertKeys(pg.db); err != nil {
+		return err
+	}
+	return nil
+}
+func (pg *PGBackend) Start() error {

Review comment:
       GoDoc?

##########
File path: tools/traffic_vault_migrate/postgres.go
##########
@@ -0,0 +1,716 @@
+package main
+
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+import (
+	"database/sql"
+	"encoding/base64"
+	"encoding/json"
+	"errors"
+	"fmt"
+	"github.com/apache/trafficcontrol/lib/go-tc"
+	util "github.com/apache/trafficcontrol/lib/go-util"
+	_ "github.com/lib/pq"
+	"log"
+	"reflect"
+	"strconv"
+	"strings"
+)
+
+type PGConfig struct {
+	Host     string `json:"host"`
+	Port     string `json:"port"`
+	User     string `json:"user"`
+	Password string `json:"password"`
+	SSLMode  string `json:"sslmode"`
+	Database string `json:"database"`
+	Key      string `json:"aesKey"`
+	AESKey   []byte
+}
+type PGBackend struct {
+	sslKey PGSSLKeyTable
+	dnssec PGDNSSecTable
+	uri    PGURISignKeyTable
+	url    PGURLSigKeyTable
+	cfg    PGConfig
+	db     *sql.DB
+}
+
+func (pg *PGBackend) String() string {
+	data := fmt.Sprintf("PG server %v@%v:%v\n", pg.cfg.User, pg.cfg.Host, pg.cfg.Port)
+	data += fmt.Sprintf("\tSSL Keys: %v\n", len(pg.sslKey.Records))
+	data += fmt.Sprintf("\tDNSSec Keys: %v\n", len(pg.dnssec.Records))
+	data += fmt.Sprintf("\tURI Keys: %v\n", len(pg.uri.Records))
+	data += fmt.Sprintf("\tURL Keys: %v\n", len(pg.url.Records))
+	return data
+}
+func (pg *PGBackend) Name() string {
+	return "PG"
+}
+func (pg *PGBackend) ReadConfig(s string) error {
+	genericCfg, err := UnmarshalConfig(s, reflect.TypeOf(pg.cfg))
+	if err != nil {
+		return err
+	}
+
+	pg.cfg = *genericCfg.Interface().(*PGConfig)
+
+	pg.cfg.AESKey, err = base64.StdEncoding.DecodeString(pg.cfg.Key)
+	if err != nil {
+		return fmt.Errorf("unable to decode PG AESKey: %v", err)
+	}
+	return nil
+}
+func (pg *PGBackend) Insert() error {
+	if err := pg.sslKey.insertKeys(pg.db); err != nil {
+		return err
+	}
+	if err := pg.dnssec.insertKeys(pg.db); err != nil {
+		return err
+	}
+	if err := pg.url.insertKeys(pg.db); err != nil {
+		return err
+	}
+	if err := pg.uri.insertKeys(pg.db); err != nil {
+		return err
+	}
+	return nil
+}
+func (pg *PGBackend) Start() error {
+	sqlStr := fmt.Sprintf("postgres://%s:%s@%s:%s/%s?sslmode=%s", pg.cfg.User, pg.cfg.Password, pg.cfg.Host, pg.cfg.Port, pg.cfg.Database, pg.cfg.SSLMode)
+	db, err := sql.Open("postgres", sqlStr)
+	if err != nil {
+		return fmt.Errorf("Unable to start PG client: %v", err)
+	}
+
+	pg.db = db
+	pg.sslKey = PGSSLKeyTable{}
+	pg.dnssec = PGDNSSecTable{}
+	pg.url = PGURLSigKeyTable{}
+	pg.uri = PGURISignKeyTable{}
+
+	return nil
+}
+func (pg *PGBackend) ValidateKey() []string {
+	var errors []string
+	if errs := pg.sslKey.validate(); errs != nil {
+		errors = append(errors, errs...)
+	}
+	if errs := pg.dnssec.validate(); errs != nil {
+		errors = append(errors, errs...)
+	}
+	if errs := pg.uri.validate(); errs != nil {
+		errors = append(errors, errs...)
+	}
+	if errs := pg.url.validate(); errs != nil {
+		errors = append(errors, errs...)
+	}
+	return errors
+}
+func (pg *PGBackend) Stop() error {
+	return pg.db.Close()
+}
+func (pg *PGBackend) Ping() error {
+	return pg.db.Ping()
+}
+func (pg *PGBackend) Fetch() error {

Review comment:
       GoDoc?

##########
File path: tools/traffic_vault_migrate/postgres.go
##########
@@ -0,0 +1,716 @@
+package main
+
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+import (
+	"database/sql"
+	"encoding/base64"
+	"encoding/json"
+	"errors"
+	"fmt"
+	"github.com/apache/trafficcontrol/lib/go-tc"
+	util "github.com/apache/trafficcontrol/lib/go-util"
+	_ "github.com/lib/pq"
+	"log"
+	"reflect"
+	"strconv"
+	"strings"
+)
+
+type PGConfig struct {
+	Host     string `json:"host"`
+	Port     string `json:"port"`
+	User     string `json:"user"`
+	Password string `json:"password"`
+	SSLMode  string `json:"sslmode"`
+	Database string `json:"database"`
+	Key      string `json:"aesKey"`
+	AESKey   []byte
+}
+type PGBackend struct {
+	sslKey PGSSLKeyTable
+	dnssec PGDNSSecTable
+	uri    PGURISignKeyTable
+	url    PGURLSigKeyTable
+	cfg    PGConfig
+	db     *sql.DB
+}
+
+func (pg *PGBackend) String() string {
+	data := fmt.Sprintf("PG server %v@%v:%v\n", pg.cfg.User, pg.cfg.Host, pg.cfg.Port)
+	data += fmt.Sprintf("\tSSL Keys: %v\n", len(pg.sslKey.Records))
+	data += fmt.Sprintf("\tDNSSec Keys: %v\n", len(pg.dnssec.Records))
+	data += fmt.Sprintf("\tURI Keys: %v\n", len(pg.uri.Records))
+	data += fmt.Sprintf("\tURL Keys: %v\n", len(pg.url.Records))
+	return data
+}
+func (pg *PGBackend) Name() string {
+	return "PG"
+}
+func (pg *PGBackend) ReadConfig(s string) error {
+	genericCfg, err := UnmarshalConfig(s, reflect.TypeOf(pg.cfg))
+	if err != nil {
+		return err
+	}
+
+	pg.cfg = *genericCfg.Interface().(*PGConfig)
+
+	pg.cfg.AESKey, err = base64.StdEncoding.DecodeString(pg.cfg.Key)
+	if err != nil {
+		return fmt.Errorf("unable to decode PG AESKey: %v", err)
+	}
+	return nil
+}
+func (pg *PGBackend) Insert() error {
+	if err := pg.sslKey.insertKeys(pg.db); err != nil {
+		return err
+	}
+	if err := pg.dnssec.insertKeys(pg.db); err != nil {
+		return err
+	}
+	if err := pg.url.insertKeys(pg.db); err != nil {
+		return err
+	}
+	if err := pg.uri.insertKeys(pg.db); err != nil {
+		return err
+	}
+	return nil
+}
+func (pg *PGBackend) Start() error {
+	sqlStr := fmt.Sprintf("postgres://%s:%s@%s:%s/%s?sslmode=%s", pg.cfg.User, pg.cfg.Password, pg.cfg.Host, pg.cfg.Port, pg.cfg.Database, pg.cfg.SSLMode)
+	db, err := sql.Open("postgres", sqlStr)
+	if err != nil {
+		return fmt.Errorf("Unable to start PG client: %v", err)
+	}
+
+	pg.db = db
+	pg.sslKey = PGSSLKeyTable{}
+	pg.dnssec = PGDNSSecTable{}
+	pg.url = PGURLSigKeyTable{}
+	pg.uri = PGURISignKeyTable{}
+
+	return nil
+}
+func (pg *PGBackend) ValidateKey() []string {
+	var errors []string
+	if errs := pg.sslKey.validate(); errs != nil {
+		errors = append(errors, errs...)
+	}
+	if errs := pg.dnssec.validate(); errs != nil {
+		errors = append(errors, errs...)
+	}
+	if errs := pg.uri.validate(); errs != nil {
+		errors = append(errors, errs...)
+	}
+	if errs := pg.url.validate(); errs != nil {
+		errors = append(errors, errs...)
+	}
+	return errors
+}
+func (pg *PGBackend) Stop() error {
+	return pg.db.Close()
+}
+func (pg *PGBackend) Ping() error {

Review comment:
       GoDoc?

##########
File path: tools/traffic_vault_migrate/postgres.go
##########
@@ -0,0 +1,716 @@
+package main
+
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+import (
+	"database/sql"
+	"encoding/base64"
+	"encoding/json"
+	"errors"
+	"fmt"
+	"github.com/apache/trafficcontrol/lib/go-tc"
+	util "github.com/apache/trafficcontrol/lib/go-util"
+	_ "github.com/lib/pq"
+	"log"
+	"reflect"
+	"strconv"
+	"strings"
+)
+
+type PGConfig struct {
+	Host     string `json:"host"`
+	Port     string `json:"port"`
+	User     string `json:"user"`
+	Password string `json:"password"`
+	SSLMode  string `json:"sslmode"`
+	Database string `json:"database"`
+	Key      string `json:"aesKey"`
+	AESKey   []byte
+}
+type PGBackend struct {
+	sslKey PGSSLKeyTable
+	dnssec PGDNSSecTable
+	uri    PGURISignKeyTable
+	url    PGURLSigKeyTable
+	cfg    PGConfig
+	db     *sql.DB
+}
+
+func (pg *PGBackend) String() string {
+	data := fmt.Sprintf("PG server %v@%v:%v\n", pg.cfg.User, pg.cfg.Host, pg.cfg.Port)
+	data += fmt.Sprintf("\tSSL Keys: %v\n", len(pg.sslKey.Records))
+	data += fmt.Sprintf("\tDNSSec Keys: %v\n", len(pg.dnssec.Records))
+	data += fmt.Sprintf("\tURI Keys: %v\n", len(pg.uri.Records))
+	data += fmt.Sprintf("\tURL Keys: %v\n", len(pg.url.Records))
+	return data
+}
+func (pg *PGBackend) Name() string {
+	return "PG"
+}
+func (pg *PGBackend) ReadConfig(s string) error {
+	genericCfg, err := UnmarshalConfig(s, reflect.TypeOf(pg.cfg))
+	if err != nil {
+		return err
+	}
+
+	pg.cfg = *genericCfg.Interface().(*PGConfig)
+
+	pg.cfg.AESKey, err = base64.StdEncoding.DecodeString(pg.cfg.Key)
+	if err != nil {
+		return fmt.Errorf("unable to decode PG AESKey: %v", err)
+	}
+	return nil
+}
+func (pg *PGBackend) Insert() error {

Review comment:
       GoDoc?

##########
File path: tools/traffic_vault_migrate/postgres.go
##########
@@ -0,0 +1,716 @@
+package main
+
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+import (
+	"database/sql"
+	"encoding/base64"
+	"encoding/json"
+	"errors"
+	"fmt"
+	"github.com/apache/trafficcontrol/lib/go-tc"
+	util "github.com/apache/trafficcontrol/lib/go-util"
+	_ "github.com/lib/pq"
+	"log"
+	"reflect"
+	"strconv"
+	"strings"
+)
+
+type PGConfig struct {
+	Host     string `json:"host"`
+	Port     string `json:"port"`
+	User     string `json:"user"`
+	Password string `json:"password"`
+	SSLMode  string `json:"sslmode"`
+	Database string `json:"database"`
+	Key      string `json:"aesKey"`
+	AESKey   []byte
+}
+type PGBackend struct {
+	sslKey PGSSLKeyTable
+	dnssec PGDNSSecTable
+	uri    PGURISignKeyTable
+	url    PGURLSigKeyTable
+	cfg    PGConfig
+	db     *sql.DB
+}
+
+func (pg *PGBackend) String() string {
+	data := fmt.Sprintf("PG server %v@%v:%v\n", pg.cfg.User, pg.cfg.Host, pg.cfg.Port)
+	data += fmt.Sprintf("\tSSL Keys: %v\n", len(pg.sslKey.Records))
+	data += fmt.Sprintf("\tDNSSec Keys: %v\n", len(pg.dnssec.Records))
+	data += fmt.Sprintf("\tURI Keys: %v\n", len(pg.uri.Records))
+	data += fmt.Sprintf("\tURL Keys: %v\n", len(pg.url.Records))
+	return data
+}
+func (pg *PGBackend) Name() string {
+	return "PG"
+}
+func (pg *PGBackend) ReadConfig(s string) error {
+	genericCfg, err := UnmarshalConfig(s, reflect.TypeOf(pg.cfg))
+	if err != nil {
+		return err
+	}
+
+	pg.cfg = *genericCfg.Interface().(*PGConfig)
+
+	pg.cfg.AESKey, err = base64.StdEncoding.DecodeString(pg.cfg.Key)
+	if err != nil {
+		return fmt.Errorf("unable to decode PG AESKey: %v", err)
+	}
+	return nil
+}
+func (pg *PGBackend) Insert() error {
+	if err := pg.sslKey.insertKeys(pg.db); err != nil {
+		return err
+	}
+	if err := pg.dnssec.insertKeys(pg.db); err != nil {
+		return err
+	}
+	if err := pg.url.insertKeys(pg.db); err != nil {
+		return err
+	}
+	if err := pg.uri.insertKeys(pg.db); err != nil {
+		return err
+	}
+	return nil
+}
+func (pg *PGBackend) Start() error {
+	sqlStr := fmt.Sprintf("postgres://%s:%s@%s:%s/%s?sslmode=%s", pg.cfg.User, pg.cfg.Password, pg.cfg.Host, pg.cfg.Port, pg.cfg.Database, pg.cfg.SSLMode)
+	db, err := sql.Open("postgres", sqlStr)
+	if err != nil {
+		return fmt.Errorf("Unable to start PG client: %v", err)
+	}
+
+	pg.db = db
+	pg.sslKey = PGSSLKeyTable{}
+	pg.dnssec = PGDNSSecTable{}
+	pg.url = PGURLSigKeyTable{}
+	pg.uri = PGURISignKeyTable{}
+
+	return nil
+}
+func (pg *PGBackend) ValidateKey() []string {
+	var errors []string
+	if errs := pg.sslKey.validate(); errs != nil {
+		errors = append(errors, errs...)
+	}
+	if errs := pg.dnssec.validate(); errs != nil {
+		errors = append(errors, errs...)
+	}
+	if errs := pg.uri.validate(); errs != nil {
+		errors = append(errors, errs...)
+	}
+	if errs := pg.url.validate(); errs != nil {
+		errors = append(errors, errs...)
+	}
+	return errors
+}
+func (pg *PGBackend) Stop() error {
+	return pg.db.Close()
+}
+func (pg *PGBackend) Ping() error {
+	return pg.db.Ping()
+}
+func (pg *PGBackend) Fetch() error {
+	if err := pg.sslKey.gatherKeys(pg.db); err != nil {
+		return err
+	}
+
+	if err := pg.dnssec.gatherKeys(pg.db); err != nil {
+		return err
+	}
+
+	if err := pg.url.gatherKeys(pg.db); err != nil {
+		return err
+	}
+
+	if err := pg.uri.gatherKeys(pg.db); err != nil {
+		return err
+	}
+
+	return nil
+}
+func (pg *PGBackend) GetSSLKeys() ([]SSLKey, error) {
+	if err := pg.sslKey.decrypt(pg.cfg.AESKey); err != nil {
+		return nil, err
+	}
+	return pg.sslKey.toGeneric(), nil
+}
+func (pg *PGBackend) SetSSLKeys(keys []SSLKey) error {
+	pg.sslKey.fromGeneric(keys)
+	return pg.sslKey.encrypt(pg.cfg.AESKey)
+}
+func (pg *PGBackend) GetDNSSecKeys() ([]DNSSecKey, error) {
+	if err := pg.dnssec.decrypt(pg.cfg.AESKey); err != nil {
+		return nil, err
+	}
+	return pg.dnssec.toGeneric(), nil
+}
+func (pg *PGBackend) SetDNSSecKeys(keys []DNSSecKey) error {
+	pg.dnssec.fromGeneric(keys)
+	return pg.dnssec.encrypt(pg.cfg.AESKey)
+}
+func (pg *PGBackend) GetURISignKeys() ([]URISignKey, error) {
+	if err := pg.uri.decrypt(pg.cfg.AESKey); err != nil {
+		return nil, err
+	}
+	return pg.uri.toGeneric(), nil
+}
+func (pg *PGBackend) SetURISignKeys(keys []URISignKey) error {
+	pg.uri.fromGeneric(keys)
+	return pg.uri.encrypt(pg.cfg.AESKey)
+}
+func (pg *PGBackend) GetURLSigKeys() ([]URLSigKey, error) {
+	if err := pg.url.decrypt(pg.cfg.AESKey); err != nil {
+		return nil, err
+	}
+	return pg.url.toGeneric(), nil
+}
+func (pg *PGBackend) SetURLSigKeys(keys []URLSigKey) error {
+	pg.url.fromGeneric(keys)
+	return pg.url.encrypt(pg.cfg.AESKey)
+}
+
+type PGCommonRecord struct {
+	DataEncrypted []byte
+	CommonRecord
+}
+
+type PGDNSSecRecord struct {
+	Key tc.DNSSECKeysTrafficVault
+	CDN string
+	PGCommonRecord
+}
+type PGDNSSecTable struct {
+	Records []PGDNSSecRecord
+}

Review comment:
       GoDoc?

##########
File path: tools/traffic_vault_migrate/postgres.go
##########
@@ -0,0 +1,716 @@
+package main
+
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+import (
+	"database/sql"
+	"encoding/base64"
+	"encoding/json"
+	"errors"
+	"fmt"
+	"github.com/apache/trafficcontrol/lib/go-tc"
+	util "github.com/apache/trafficcontrol/lib/go-util"
+	_ "github.com/lib/pq"
+	"log"
+	"reflect"
+	"strconv"
+	"strings"
+)
+
+type PGConfig struct {
+	Host     string `json:"host"`
+	Port     string `json:"port"`
+	User     string `json:"user"`
+	Password string `json:"password"`
+	SSLMode  string `json:"sslmode"`
+	Database string `json:"database"`
+	Key      string `json:"aesKey"`
+	AESKey   []byte
+}
+type PGBackend struct {
+	sslKey PGSSLKeyTable
+	dnssec PGDNSSecTable
+	uri    PGURISignKeyTable
+	url    PGURLSigKeyTable
+	cfg    PGConfig
+	db     *sql.DB
+}
+
+func (pg *PGBackend) String() string {
+	data := fmt.Sprintf("PG server %v@%v:%v\n", pg.cfg.User, pg.cfg.Host, pg.cfg.Port)
+	data += fmt.Sprintf("\tSSL Keys: %v\n", len(pg.sslKey.Records))
+	data += fmt.Sprintf("\tDNSSec Keys: %v\n", len(pg.dnssec.Records))
+	data += fmt.Sprintf("\tURI Keys: %v\n", len(pg.uri.Records))
+	data += fmt.Sprintf("\tURL Keys: %v\n", len(pg.url.Records))
+	return data
+}
+func (pg *PGBackend) Name() string {
+	return "PG"
+}
+func (pg *PGBackend) ReadConfig(s string) error {
+	genericCfg, err := UnmarshalConfig(s, reflect.TypeOf(pg.cfg))
+	if err != nil {
+		return err
+	}
+
+	pg.cfg = *genericCfg.Interface().(*PGConfig)
+
+	pg.cfg.AESKey, err = base64.StdEncoding.DecodeString(pg.cfg.Key)
+	if err != nil {
+		return fmt.Errorf("unable to decode PG AESKey: %v", err)
+	}
+	return nil
+}
+func (pg *PGBackend) Insert() error {
+	if err := pg.sslKey.insertKeys(pg.db); err != nil {
+		return err
+	}
+	if err := pg.dnssec.insertKeys(pg.db); err != nil {
+		return err
+	}
+	if err := pg.url.insertKeys(pg.db); err != nil {
+		return err
+	}
+	if err := pg.uri.insertKeys(pg.db); err != nil {
+		return err
+	}
+	return nil
+}
+func (pg *PGBackend) Start() error {
+	sqlStr := fmt.Sprintf("postgres://%s:%s@%s:%s/%s?sslmode=%s", pg.cfg.User, pg.cfg.Password, pg.cfg.Host, pg.cfg.Port, pg.cfg.Database, pg.cfg.SSLMode)
+	db, err := sql.Open("postgres", sqlStr)
+	if err != nil {
+		return fmt.Errorf("Unable to start PG client: %v", err)
+	}
+
+	pg.db = db
+	pg.sslKey = PGSSLKeyTable{}
+	pg.dnssec = PGDNSSecTable{}
+	pg.url = PGURLSigKeyTable{}
+	pg.uri = PGURISignKeyTable{}
+
+	return nil
+}
+func (pg *PGBackend) ValidateKey() []string {
+	var errors []string
+	if errs := pg.sslKey.validate(); errs != nil {
+		errors = append(errors, errs...)
+	}
+	if errs := pg.dnssec.validate(); errs != nil {
+		errors = append(errors, errs...)
+	}
+	if errs := pg.uri.validate(); errs != nil {
+		errors = append(errors, errs...)
+	}
+	if errs := pg.url.validate(); errs != nil {
+		errors = append(errors, errs...)
+	}
+	return errors
+}
+func (pg *PGBackend) Stop() error {
+	return pg.db.Close()
+}
+func (pg *PGBackend) Ping() error {
+	return pg.db.Ping()
+}
+func (pg *PGBackend) Fetch() error {
+	if err := pg.sslKey.gatherKeys(pg.db); err != nil {
+		return err
+	}
+
+	if err := pg.dnssec.gatherKeys(pg.db); err != nil {
+		return err
+	}
+
+	if err := pg.url.gatherKeys(pg.db); err != nil {
+		return err
+	}
+
+	if err := pg.uri.gatherKeys(pg.db); err != nil {
+		return err
+	}
+
+	return nil
+}
+func (pg *PGBackend) GetSSLKeys() ([]SSLKey, error) {
+	if err := pg.sslKey.decrypt(pg.cfg.AESKey); err != nil {
+		return nil, err
+	}
+	return pg.sslKey.toGeneric(), nil
+}
+func (pg *PGBackend) SetSSLKeys(keys []SSLKey) error {
+	pg.sslKey.fromGeneric(keys)
+	return pg.sslKey.encrypt(pg.cfg.AESKey)
+}
+func (pg *PGBackend) GetDNSSecKeys() ([]DNSSecKey, error) {
+	if err := pg.dnssec.decrypt(pg.cfg.AESKey); err != nil {
+		return nil, err
+	}
+	return pg.dnssec.toGeneric(), nil
+}
+func (pg *PGBackend) SetDNSSecKeys(keys []DNSSecKey) error {
+	pg.dnssec.fromGeneric(keys)
+	return pg.dnssec.encrypt(pg.cfg.AESKey)
+}
+func (pg *PGBackend) GetURISignKeys() ([]URISignKey, error) {
+	if err := pg.uri.decrypt(pg.cfg.AESKey); err != nil {
+		return nil, err
+	}
+	return pg.uri.toGeneric(), nil
+}
+func (pg *PGBackend) SetURISignKeys(keys []URISignKey) error {
+	pg.uri.fromGeneric(keys)
+	return pg.uri.encrypt(pg.cfg.AESKey)
+}
+func (pg *PGBackend) GetURLSigKeys() ([]URLSigKey, error) {
+	if err := pg.url.decrypt(pg.cfg.AESKey); err != nil {
+		return nil, err
+	}
+	return pg.url.toGeneric(), nil
+}
+func (pg *PGBackend) SetURLSigKeys(keys []URLSigKey) error {
+	pg.url.fromGeneric(keys)
+	return pg.url.encrypt(pg.cfg.AESKey)
+}
+
+type PGCommonRecord struct {
+	DataEncrypted []byte
+	CommonRecord
+}
+
+type PGDNSSecRecord struct {
+	Key tc.DNSSECKeysTrafficVault
+	CDN string
+	PGCommonRecord
+}
+type PGDNSSecTable struct {
+	Records []PGDNSSecRecord
+}
+
+func (tbl *PGDNSSecTable) gatherKeys(db *sql.DB) error {
+	sz, err := getSize(db, "dnssec")
+	if err != nil {
+		log.Println("PGDNSSec gatherKeys: unable to determine size of dnssec table")

Review comment:
       Should this log using `log.Errorln`?

##########
File path: tools/traffic_vault_migrate/postgres.go
##########
@@ -0,0 +1,716 @@
+package main
+
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+import (
+	"database/sql"
+	"encoding/base64"
+	"encoding/json"
+	"errors"
+	"fmt"
+	"github.com/apache/trafficcontrol/lib/go-tc"
+	util "github.com/apache/trafficcontrol/lib/go-util"
+	_ "github.com/lib/pq"
+	"log"
+	"reflect"
+	"strconv"
+	"strings"
+)
+
+type PGConfig struct {
+	Host     string `json:"host"`
+	Port     string `json:"port"`
+	User     string `json:"user"`
+	Password string `json:"password"`
+	SSLMode  string `json:"sslmode"`
+	Database string `json:"database"`
+	Key      string `json:"aesKey"`
+	AESKey   []byte
+}
+type PGBackend struct {
+	sslKey PGSSLKeyTable
+	dnssec PGDNSSecTable
+	uri    PGURISignKeyTable
+	url    PGURLSigKeyTable
+	cfg    PGConfig
+	db     *sql.DB
+}
+
+func (pg *PGBackend) String() string {
+	data := fmt.Sprintf("PG server %v@%v:%v\n", pg.cfg.User, pg.cfg.Host, pg.cfg.Port)
+	data += fmt.Sprintf("\tSSL Keys: %v\n", len(pg.sslKey.Records))
+	data += fmt.Sprintf("\tDNSSec Keys: %v\n", len(pg.dnssec.Records))
+	data += fmt.Sprintf("\tURI Keys: %v\n", len(pg.uri.Records))
+	data += fmt.Sprintf("\tURL Keys: %v\n", len(pg.url.Records))
+	return data
+}
+func (pg *PGBackend) Name() string {
+	return "PG"
+}
+func (pg *PGBackend) ReadConfig(s string) error {
+	genericCfg, err := UnmarshalConfig(s, reflect.TypeOf(pg.cfg))
+	if err != nil {
+		return err
+	}
+
+	pg.cfg = *genericCfg.Interface().(*PGConfig)
+
+	pg.cfg.AESKey, err = base64.StdEncoding.DecodeString(pg.cfg.Key)
+	if err != nil {
+		return fmt.Errorf("unable to decode PG AESKey: %v", err)
+	}
+	return nil
+}
+func (pg *PGBackend) Insert() error {
+	if err := pg.sslKey.insertKeys(pg.db); err != nil {
+		return err
+	}
+	if err := pg.dnssec.insertKeys(pg.db); err != nil {
+		return err
+	}
+	if err := pg.url.insertKeys(pg.db); err != nil {
+		return err
+	}
+	if err := pg.uri.insertKeys(pg.db); err != nil {
+		return err
+	}
+	return nil
+}
+func (pg *PGBackend) Start() error {
+	sqlStr := fmt.Sprintf("postgres://%s:%s@%s:%s/%s?sslmode=%s", pg.cfg.User, pg.cfg.Password, pg.cfg.Host, pg.cfg.Port, pg.cfg.Database, pg.cfg.SSLMode)
+	db, err := sql.Open("postgres", sqlStr)
+	if err != nil {
+		return fmt.Errorf("Unable to start PG client: %v", err)
+	}
+
+	pg.db = db
+	pg.sslKey = PGSSLKeyTable{}
+	pg.dnssec = PGDNSSecTable{}
+	pg.url = PGURLSigKeyTable{}
+	pg.uri = PGURISignKeyTable{}
+
+	return nil
+}
+func (pg *PGBackend) ValidateKey() []string {
+	var errors []string
+	if errs := pg.sslKey.validate(); errs != nil {
+		errors = append(errors, errs...)
+	}
+	if errs := pg.dnssec.validate(); errs != nil {
+		errors = append(errors, errs...)
+	}
+	if errs := pg.uri.validate(); errs != nil {
+		errors = append(errors, errs...)
+	}
+	if errs := pg.url.validate(); errs != nil {
+		errors = append(errors, errs...)
+	}
+	return errors
+}
+func (pg *PGBackend) Stop() error {
+	return pg.db.Close()
+}
+func (pg *PGBackend) Ping() error {
+	return pg.db.Ping()
+}
+func (pg *PGBackend) Fetch() error {
+	if err := pg.sslKey.gatherKeys(pg.db); err != nil {
+		return err
+	}
+
+	if err := pg.dnssec.gatherKeys(pg.db); err != nil {
+		return err
+	}
+
+	if err := pg.url.gatherKeys(pg.db); err != nil {
+		return err
+	}
+
+	if err := pg.uri.gatherKeys(pg.db); err != nil {
+		return err
+	}
+
+	return nil
+}
+func (pg *PGBackend) GetSSLKeys() ([]SSLKey, error) {
+	if err := pg.sslKey.decrypt(pg.cfg.AESKey); err != nil {
+		return nil, err
+	}
+	return pg.sslKey.toGeneric(), nil
+}
+func (pg *PGBackend) SetSSLKeys(keys []SSLKey) error {
+	pg.sslKey.fromGeneric(keys)
+	return pg.sslKey.encrypt(pg.cfg.AESKey)
+}
+func (pg *PGBackend) GetDNSSecKeys() ([]DNSSecKey, error) {
+	if err := pg.dnssec.decrypt(pg.cfg.AESKey); err != nil {
+		return nil, err
+	}
+	return pg.dnssec.toGeneric(), nil
+}
+func (pg *PGBackend) SetDNSSecKeys(keys []DNSSecKey) error {
+	pg.dnssec.fromGeneric(keys)
+	return pg.dnssec.encrypt(pg.cfg.AESKey)
+}
+func (pg *PGBackend) GetURISignKeys() ([]URISignKey, error) {
+	if err := pg.uri.decrypt(pg.cfg.AESKey); err != nil {
+		return nil, err
+	}
+	return pg.uri.toGeneric(), nil
+}
+func (pg *PGBackend) SetURISignKeys(keys []URISignKey) error {
+	pg.uri.fromGeneric(keys)
+	return pg.uri.encrypt(pg.cfg.AESKey)
+}
+func (pg *PGBackend) GetURLSigKeys() ([]URLSigKey, error) {
+	if err := pg.url.decrypt(pg.cfg.AESKey); err != nil {
+		return nil, err
+	}
+	return pg.url.toGeneric(), nil
+}
+func (pg *PGBackend) SetURLSigKeys(keys []URLSigKey) error {
+	pg.url.fromGeneric(keys)
+	return pg.url.encrypt(pg.cfg.AESKey)
+}
+
+type PGCommonRecord struct {
+	DataEncrypted []byte
+	CommonRecord
+}
+
+type PGDNSSecRecord struct {
+	Key tc.DNSSECKeysTrafficVault
+	CDN string
+	PGCommonRecord
+}
+type PGDNSSecTable struct {
+	Records []PGDNSSecRecord
+}
+
+func (tbl *PGDNSSecTable) gatherKeys(db *sql.DB) error {
+	sz, err := getSize(db, "dnssec")
+	if err != nil {
+		log.Println("PGDNSSec gatherKeys: unable to determine size of dnssec table")
+	}
+	tbl.Records = make([]PGDNSSecRecord, sz)
+
+	rows, err := db.Query("SELECT cdn, data from dnssec")
+	if err != nil {
+		return fmt.Errorf("PGDNSSec gatherKeys: unable to query: %v", err)

Review comment:
       same as above RE: `%w` vs `%v`

##########
File path: tools/traffic_vault_migrate/postgres.go
##########
@@ -0,0 +1,716 @@
+package main
+
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+import (
+	"database/sql"
+	"encoding/base64"
+	"encoding/json"
+	"errors"
+	"fmt"
+	"github.com/apache/trafficcontrol/lib/go-tc"
+	util "github.com/apache/trafficcontrol/lib/go-util"
+	_ "github.com/lib/pq"
+	"log"
+	"reflect"
+	"strconv"
+	"strings"
+)
+
+type PGConfig struct {
+	Host     string `json:"host"`
+	Port     string `json:"port"`
+	User     string `json:"user"`
+	Password string `json:"password"`
+	SSLMode  string `json:"sslmode"`
+	Database string `json:"database"`
+	Key      string `json:"aesKey"`
+	AESKey   []byte
+}
+type PGBackend struct {
+	sslKey PGSSLKeyTable
+	dnssec PGDNSSecTable
+	uri    PGURISignKeyTable
+	url    PGURLSigKeyTable
+	cfg    PGConfig
+	db     *sql.DB
+}
+
+func (pg *PGBackend) String() string {
+	data := fmt.Sprintf("PG server %v@%v:%v\n", pg.cfg.User, pg.cfg.Host, pg.cfg.Port)
+	data += fmt.Sprintf("\tSSL Keys: %v\n", len(pg.sslKey.Records))
+	data += fmt.Sprintf("\tDNSSec Keys: %v\n", len(pg.dnssec.Records))
+	data += fmt.Sprintf("\tURI Keys: %v\n", len(pg.uri.Records))
+	data += fmt.Sprintf("\tURL Keys: %v\n", len(pg.url.Records))
+	return data
+}
+func (pg *PGBackend) Name() string {
+	return "PG"
+}
+func (pg *PGBackend) ReadConfig(s string) error {
+	genericCfg, err := UnmarshalConfig(s, reflect.TypeOf(pg.cfg))
+	if err != nil {
+		return err
+	}
+
+	pg.cfg = *genericCfg.Interface().(*PGConfig)
+
+	pg.cfg.AESKey, err = base64.StdEncoding.DecodeString(pg.cfg.Key)
+	if err != nil {
+		return fmt.Errorf("unable to decode PG AESKey: %v", err)
+	}
+	return nil
+}
+func (pg *PGBackend) Insert() error {
+	if err := pg.sslKey.insertKeys(pg.db); err != nil {
+		return err
+	}
+	if err := pg.dnssec.insertKeys(pg.db); err != nil {
+		return err
+	}
+	if err := pg.url.insertKeys(pg.db); err != nil {
+		return err
+	}
+	if err := pg.uri.insertKeys(pg.db); err != nil {
+		return err
+	}
+	return nil
+}
+func (pg *PGBackend) Start() error {
+	sqlStr := fmt.Sprintf("postgres://%s:%s@%s:%s/%s?sslmode=%s", pg.cfg.User, pg.cfg.Password, pg.cfg.Host, pg.cfg.Port, pg.cfg.Database, pg.cfg.SSLMode)
+	db, err := sql.Open("postgres", sqlStr)
+	if err != nil {
+		return fmt.Errorf("Unable to start PG client: %v", err)
+	}
+
+	pg.db = db
+	pg.sslKey = PGSSLKeyTable{}
+	pg.dnssec = PGDNSSecTable{}
+	pg.url = PGURLSigKeyTable{}
+	pg.uri = PGURISignKeyTable{}
+
+	return nil
+}
+func (pg *PGBackend) ValidateKey() []string {
+	var errors []string
+	if errs := pg.sslKey.validate(); errs != nil {
+		errors = append(errors, errs...)
+	}
+	if errs := pg.dnssec.validate(); errs != nil {
+		errors = append(errors, errs...)
+	}
+	if errs := pg.uri.validate(); errs != nil {
+		errors = append(errors, errs...)
+	}
+	if errs := pg.url.validate(); errs != nil {
+		errors = append(errors, errs...)
+	}
+	return errors
+}
+func (pg *PGBackend) Stop() error {
+	return pg.db.Close()
+}
+func (pg *PGBackend) Ping() error {
+	return pg.db.Ping()
+}
+func (pg *PGBackend) Fetch() error {
+	if err := pg.sslKey.gatherKeys(pg.db); err != nil {
+		return err
+	}
+
+	if err := pg.dnssec.gatherKeys(pg.db); err != nil {
+		return err
+	}
+
+	if err := pg.url.gatherKeys(pg.db); err != nil {
+		return err
+	}
+
+	if err := pg.uri.gatherKeys(pg.db); err != nil {
+		return err
+	}
+
+	return nil
+}
+func (pg *PGBackend) GetSSLKeys() ([]SSLKey, error) {
+	if err := pg.sslKey.decrypt(pg.cfg.AESKey); err != nil {
+		return nil, err
+	}
+	return pg.sslKey.toGeneric(), nil
+}
+func (pg *PGBackend) SetSSLKeys(keys []SSLKey) error {
+	pg.sslKey.fromGeneric(keys)
+	return pg.sslKey.encrypt(pg.cfg.AESKey)
+}
+func (pg *PGBackend) GetDNSSecKeys() ([]DNSSecKey, error) {
+	if err := pg.dnssec.decrypt(pg.cfg.AESKey); err != nil {
+		return nil, err
+	}
+	return pg.dnssec.toGeneric(), nil
+}
+func (pg *PGBackend) SetDNSSecKeys(keys []DNSSecKey) error {
+	pg.dnssec.fromGeneric(keys)
+	return pg.dnssec.encrypt(pg.cfg.AESKey)
+}
+func (pg *PGBackend) GetURISignKeys() ([]URISignKey, error) {
+	if err := pg.uri.decrypt(pg.cfg.AESKey); err != nil {
+		return nil, err
+	}
+	return pg.uri.toGeneric(), nil
+}
+func (pg *PGBackend) SetURISignKeys(keys []URISignKey) error {
+	pg.uri.fromGeneric(keys)
+	return pg.uri.encrypt(pg.cfg.AESKey)
+}
+func (pg *PGBackend) GetURLSigKeys() ([]URLSigKey, error) {
+	if err := pg.url.decrypt(pg.cfg.AESKey); err != nil {
+		return nil, err
+	}
+	return pg.url.toGeneric(), nil
+}
+func (pg *PGBackend) SetURLSigKeys(keys []URLSigKey) error {
+	pg.url.fromGeneric(keys)
+	return pg.url.encrypt(pg.cfg.AESKey)
+}
+
+type PGCommonRecord struct {
+	DataEncrypted []byte
+	CommonRecord
+}
+
+type PGDNSSecRecord struct {
+	Key tc.DNSSECKeysTrafficVault
+	CDN string
+	PGCommonRecord
+}
+type PGDNSSecTable struct {
+	Records []PGDNSSecRecord
+}
+
+func (tbl *PGDNSSecTable) gatherKeys(db *sql.DB) error {
+	sz, err := getSize(db, "dnssec")
+	if err != nil {
+		log.Println("PGDNSSec gatherKeys: unable to determine size of dnssec table")
+	}
+	tbl.Records = make([]PGDNSSecRecord, sz)
+
+	rows, err := db.Query("SELECT cdn, data from dnssec")
+	if err != nil {
+		return fmt.Errorf("PGDNSSec gatherKeys: unable to query: %v", err)
+	}
+	defer rows.Close()
+	i := 0
+	for rows.Next() {
+		if i > len(tbl.Records)-1 {
+			return fmt.Errorf("PGDNSSec gatherKeys got more results than expected %v", len(tbl.Records))
+		}
+		err := rows.Scan(&tbl.Records[i].CDN, &tbl.Records[i].DataEncrypted)
+		if err != nil {
+			return fmt.Errorf("PGDNSSec gatherKeys unable to scan row: %v", err)

Review comment:
       same as above RE: `%w` vs `%v`

##########
File path: tools/traffic_vault_migrate/postgres.go
##########
@@ -0,0 +1,716 @@
+package main
+
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+import (
+	"database/sql"
+	"encoding/base64"
+	"encoding/json"
+	"errors"
+	"fmt"
+	"github.com/apache/trafficcontrol/lib/go-tc"
+	util "github.com/apache/trafficcontrol/lib/go-util"
+	_ "github.com/lib/pq"
+	"log"
+	"reflect"
+	"strconv"
+	"strings"
+)
+
+type PGConfig struct {
+	Host     string `json:"host"`
+	Port     string `json:"port"`
+	User     string `json:"user"`
+	Password string `json:"password"`
+	SSLMode  string `json:"sslmode"`
+	Database string `json:"database"`
+	Key      string `json:"aesKey"`
+	AESKey   []byte
+}
+type PGBackend struct {
+	sslKey PGSSLKeyTable
+	dnssec PGDNSSecTable
+	uri    PGURISignKeyTable
+	url    PGURLSigKeyTable
+	cfg    PGConfig
+	db     *sql.DB
+}
+
+func (pg *PGBackend) String() string {
+	data := fmt.Sprintf("PG server %v@%v:%v\n", pg.cfg.User, pg.cfg.Host, pg.cfg.Port)
+	data += fmt.Sprintf("\tSSL Keys: %v\n", len(pg.sslKey.Records))
+	data += fmt.Sprintf("\tDNSSec Keys: %v\n", len(pg.dnssec.Records))
+	data += fmt.Sprintf("\tURI Keys: %v\n", len(pg.uri.Records))
+	data += fmt.Sprintf("\tURL Keys: %v\n", len(pg.url.Records))
+	return data
+}
+func (pg *PGBackend) Name() string {
+	return "PG"
+}
+func (pg *PGBackend) ReadConfig(s string) error {
+	genericCfg, err := UnmarshalConfig(s, reflect.TypeOf(pg.cfg))
+	if err != nil {
+		return err
+	}
+
+	pg.cfg = *genericCfg.Interface().(*PGConfig)
+
+	pg.cfg.AESKey, err = base64.StdEncoding.DecodeString(pg.cfg.Key)
+	if err != nil {
+		return fmt.Errorf("unable to decode PG AESKey: %v", err)
+	}
+	return nil
+}
+func (pg *PGBackend) Insert() error {
+	if err := pg.sslKey.insertKeys(pg.db); err != nil {
+		return err
+	}
+	if err := pg.dnssec.insertKeys(pg.db); err != nil {
+		return err
+	}
+	if err := pg.url.insertKeys(pg.db); err != nil {
+		return err
+	}
+	if err := pg.uri.insertKeys(pg.db); err != nil {
+		return err
+	}
+	return nil
+}
+func (pg *PGBackend) Start() error {
+	sqlStr := fmt.Sprintf("postgres://%s:%s@%s:%s/%s?sslmode=%s", pg.cfg.User, pg.cfg.Password, pg.cfg.Host, pg.cfg.Port, pg.cfg.Database, pg.cfg.SSLMode)
+	db, err := sql.Open("postgres", sqlStr)
+	if err != nil {
+		return fmt.Errorf("Unable to start PG client: %v", err)
+	}
+
+	pg.db = db
+	pg.sslKey = PGSSLKeyTable{}
+	pg.dnssec = PGDNSSecTable{}
+	pg.url = PGURLSigKeyTable{}
+	pg.uri = PGURISignKeyTable{}
+
+	return nil
+}
+func (pg *PGBackend) ValidateKey() []string {
+	var errors []string
+	if errs := pg.sslKey.validate(); errs != nil {
+		errors = append(errors, errs...)
+	}
+	if errs := pg.dnssec.validate(); errs != nil {
+		errors = append(errors, errs...)
+	}
+	if errs := pg.uri.validate(); errs != nil {
+		errors = append(errors, errs...)
+	}
+	if errs := pg.url.validate(); errs != nil {
+		errors = append(errors, errs...)
+	}
+	return errors
+}
+func (pg *PGBackend) Stop() error {
+	return pg.db.Close()
+}
+func (pg *PGBackend) Ping() error {
+	return pg.db.Ping()
+}
+func (pg *PGBackend) Fetch() error {
+	if err := pg.sslKey.gatherKeys(pg.db); err != nil {
+		return err
+	}
+
+	if err := pg.dnssec.gatherKeys(pg.db); err != nil {
+		return err
+	}
+
+	if err := pg.url.gatherKeys(pg.db); err != nil {
+		return err
+	}
+
+	if err := pg.uri.gatherKeys(pg.db); err != nil {
+		return err
+	}
+
+	return nil
+}
+func (pg *PGBackend) GetSSLKeys() ([]SSLKey, error) {
+	if err := pg.sslKey.decrypt(pg.cfg.AESKey); err != nil {
+		return nil, err
+	}
+	return pg.sslKey.toGeneric(), nil
+}
+func (pg *PGBackend) SetSSLKeys(keys []SSLKey) error {
+	pg.sslKey.fromGeneric(keys)
+	return pg.sslKey.encrypt(pg.cfg.AESKey)
+}
+func (pg *PGBackend) GetDNSSecKeys() ([]DNSSecKey, error) {
+	if err := pg.dnssec.decrypt(pg.cfg.AESKey); err != nil {
+		return nil, err
+	}
+	return pg.dnssec.toGeneric(), nil
+}
+func (pg *PGBackend) SetDNSSecKeys(keys []DNSSecKey) error {
+	pg.dnssec.fromGeneric(keys)
+	return pg.dnssec.encrypt(pg.cfg.AESKey)
+}
+func (pg *PGBackend) GetURISignKeys() ([]URISignKey, error) {
+	if err := pg.uri.decrypt(pg.cfg.AESKey); err != nil {
+		return nil, err
+	}
+	return pg.uri.toGeneric(), nil
+}
+func (pg *PGBackend) SetURISignKeys(keys []URISignKey) error {
+	pg.uri.fromGeneric(keys)
+	return pg.uri.encrypt(pg.cfg.AESKey)
+}
+func (pg *PGBackend) GetURLSigKeys() ([]URLSigKey, error) {
+	if err := pg.url.decrypt(pg.cfg.AESKey); err != nil {
+		return nil, err
+	}
+	return pg.url.toGeneric(), nil
+}
+func (pg *PGBackend) SetURLSigKeys(keys []URLSigKey) error {
+	pg.url.fromGeneric(keys)
+	return pg.url.encrypt(pg.cfg.AESKey)
+}
+
+type PGCommonRecord struct {
+	DataEncrypted []byte
+	CommonRecord
+}
+
+type PGDNSSecRecord struct {
+	Key tc.DNSSECKeysTrafficVault
+	CDN string
+	PGCommonRecord
+}
+type PGDNSSecTable struct {
+	Records []PGDNSSecRecord
+}
+
+func (tbl *PGDNSSecTable) gatherKeys(db *sql.DB) error {
+	sz, err := getSize(db, "dnssec")
+	if err != nil {
+		log.Println("PGDNSSec gatherKeys: unable to determine size of dnssec table")
+	}
+	tbl.Records = make([]PGDNSSecRecord, sz)
+
+	rows, err := db.Query("SELECT cdn, data from dnssec")
+	if err != nil {
+		return fmt.Errorf("PGDNSSec gatherKeys: unable to query: %v", err)
+	}
+	defer rows.Close()

Review comment:
       nit: this is done commonly in our codebase, but it is throwing away an error return value. There's a convenience method provided by `github.com/apache/trafficcontrol/lib/go-log/` to log the error using `log.Error` if an error occurs - [`log.Close`](https://pkg.go.dev/github.com/apache/trafficcontrol/lib/go-log#Close) (or `log.Closef` to use format strings). Some static analysis tools won't detect that as deferring a Close on a Closer resource, though.
   
   e.g.
   ```go
   defer log.Close(rows, "Closing PGDNSSEC rows from gathering keys")
   ```

##########
File path: tools/traffic_vault_migrate/postgres.go
##########
@@ -0,0 +1,716 @@
+package main
+
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+import (
+	"database/sql"
+	"encoding/base64"
+	"encoding/json"
+	"errors"
+	"fmt"
+	"github.com/apache/trafficcontrol/lib/go-tc"
+	util "github.com/apache/trafficcontrol/lib/go-util"
+	_ "github.com/lib/pq"
+	"log"
+	"reflect"
+	"strconv"
+	"strings"
+)
+
+type PGConfig struct {
+	Host     string `json:"host"`
+	Port     string `json:"port"`
+	User     string `json:"user"`
+	Password string `json:"password"`
+	SSLMode  string `json:"sslmode"`
+	Database string `json:"database"`
+	Key      string `json:"aesKey"`
+	AESKey   []byte
+}
+type PGBackend struct {
+	sslKey PGSSLKeyTable
+	dnssec PGDNSSecTable
+	uri    PGURISignKeyTable
+	url    PGURLSigKeyTable
+	cfg    PGConfig
+	db     *sql.DB
+}
+
+func (pg *PGBackend) String() string {
+	data := fmt.Sprintf("PG server %v@%v:%v\n", pg.cfg.User, pg.cfg.Host, pg.cfg.Port)
+	data += fmt.Sprintf("\tSSL Keys: %v\n", len(pg.sslKey.Records))
+	data += fmt.Sprintf("\tDNSSec Keys: %v\n", len(pg.dnssec.Records))
+	data += fmt.Sprintf("\tURI Keys: %v\n", len(pg.uri.Records))
+	data += fmt.Sprintf("\tURL Keys: %v\n", len(pg.url.Records))
+	return data
+}
+func (pg *PGBackend) Name() string {
+	return "PG"
+}
+func (pg *PGBackend) ReadConfig(s string) error {
+	genericCfg, err := UnmarshalConfig(s, reflect.TypeOf(pg.cfg))
+	if err != nil {
+		return err
+	}
+
+	pg.cfg = *genericCfg.Interface().(*PGConfig)
+
+	pg.cfg.AESKey, err = base64.StdEncoding.DecodeString(pg.cfg.Key)
+	if err != nil {
+		return fmt.Errorf("unable to decode PG AESKey: %v", err)
+	}
+	return nil
+}
+func (pg *PGBackend) Insert() error {
+	if err := pg.sslKey.insertKeys(pg.db); err != nil {
+		return err
+	}
+	if err := pg.dnssec.insertKeys(pg.db); err != nil {
+		return err
+	}
+	if err := pg.url.insertKeys(pg.db); err != nil {
+		return err
+	}
+	if err := pg.uri.insertKeys(pg.db); err != nil {
+		return err
+	}
+	return nil
+}
+func (pg *PGBackend) Start() error {
+	sqlStr := fmt.Sprintf("postgres://%s:%s@%s:%s/%s?sslmode=%s", pg.cfg.User, pg.cfg.Password, pg.cfg.Host, pg.cfg.Port, pg.cfg.Database, pg.cfg.SSLMode)
+	db, err := sql.Open("postgres", sqlStr)
+	if err != nil {
+		return fmt.Errorf("Unable to start PG client: %v", err)
+	}
+
+	pg.db = db
+	pg.sslKey = PGSSLKeyTable{}
+	pg.dnssec = PGDNSSecTable{}
+	pg.url = PGURLSigKeyTable{}
+	pg.uri = PGURISignKeyTable{}
+
+	return nil
+}
+func (pg *PGBackend) ValidateKey() []string {
+	var errors []string
+	if errs := pg.sslKey.validate(); errs != nil {
+		errors = append(errors, errs...)
+	}
+	if errs := pg.dnssec.validate(); errs != nil {
+		errors = append(errors, errs...)
+	}
+	if errs := pg.uri.validate(); errs != nil {
+		errors = append(errors, errs...)
+	}
+	if errs := pg.url.validate(); errs != nil {
+		errors = append(errors, errs...)
+	}
+	return errors
+}
+func (pg *PGBackend) Stop() error {
+	return pg.db.Close()
+}
+func (pg *PGBackend) Ping() error {
+	return pg.db.Ping()
+}
+func (pg *PGBackend) Fetch() error {
+	if err := pg.sslKey.gatherKeys(pg.db); err != nil {
+		return err
+	}
+
+	if err := pg.dnssec.gatherKeys(pg.db); err != nil {
+		return err
+	}
+
+	if err := pg.url.gatherKeys(pg.db); err != nil {
+		return err
+	}
+
+	if err := pg.uri.gatherKeys(pg.db); err != nil {
+		return err
+	}
+
+	return nil
+}
+func (pg *PGBackend) GetSSLKeys() ([]SSLKey, error) {
+	if err := pg.sslKey.decrypt(pg.cfg.AESKey); err != nil {
+		return nil, err
+	}
+	return pg.sslKey.toGeneric(), nil
+}
+func (pg *PGBackend) SetSSLKeys(keys []SSLKey) error {
+	pg.sslKey.fromGeneric(keys)
+	return pg.sslKey.encrypt(pg.cfg.AESKey)
+}
+func (pg *PGBackend) GetDNSSecKeys() ([]DNSSecKey, error) {
+	if err := pg.dnssec.decrypt(pg.cfg.AESKey); err != nil {
+		return nil, err
+	}
+	return pg.dnssec.toGeneric(), nil
+}
+func (pg *PGBackend) SetDNSSecKeys(keys []DNSSecKey) error {
+	pg.dnssec.fromGeneric(keys)
+	return pg.dnssec.encrypt(pg.cfg.AESKey)
+}
+func (pg *PGBackend) GetURISignKeys() ([]URISignKey, error) {
+	if err := pg.uri.decrypt(pg.cfg.AESKey); err != nil {
+		return nil, err
+	}
+	return pg.uri.toGeneric(), nil
+}
+func (pg *PGBackend) SetURISignKeys(keys []URISignKey) error {
+	pg.uri.fromGeneric(keys)
+	return pg.uri.encrypt(pg.cfg.AESKey)
+}
+func (pg *PGBackend) GetURLSigKeys() ([]URLSigKey, error) {
+	if err := pg.url.decrypt(pg.cfg.AESKey); err != nil {
+		return nil, err
+	}
+	return pg.url.toGeneric(), nil
+}
+func (pg *PGBackend) SetURLSigKeys(keys []URLSigKey) error {
+	pg.url.fromGeneric(keys)
+	return pg.url.encrypt(pg.cfg.AESKey)
+}
+
+type PGCommonRecord struct {
+	DataEncrypted []byte
+	CommonRecord
+}
+
+type PGDNSSecRecord struct {
+	Key tc.DNSSECKeysTrafficVault
+	CDN string
+	PGCommonRecord
+}
+type PGDNSSecTable struct {
+	Records []PGDNSSecRecord
+}
+
+func (tbl *PGDNSSecTable) gatherKeys(db *sql.DB) error {
+	sz, err := getSize(db, "dnssec")
+	if err != nil {
+		log.Println("PGDNSSec gatherKeys: unable to determine size of dnssec table")
+	}
+	tbl.Records = make([]PGDNSSecRecord, sz)
+
+	rows, err := db.Query("SELECT cdn, data from dnssec")
+	if err != nil {
+		return fmt.Errorf("PGDNSSec gatherKeys: unable to query: %v", err)
+	}
+	defer rows.Close()
+	i := 0
+	for rows.Next() {
+		if i > len(tbl.Records)-1 {
+			return fmt.Errorf("PGDNSSec gatherKeys got more results than expected %v", len(tbl.Records))
+		}
+		err := rows.Scan(&tbl.Records[i].CDN, &tbl.Records[i].DataEncrypted)
+		if err != nil {
+			return fmt.Errorf("PGDNSSec gatherKeys unable to scan row: %v", err)
+		}
+		i += 1
+	}
+	return nil
+}
+func (tbl *PGDNSSecTable) decrypt(aesKey []byte) error {
+	for i, _ := range tbl.Records {
+		data, err := decryptInto(aesKey, tbl.Records[i].DataEncrypted, reflect.TypeOf(tbl.Records[i].Key))
+		if err != nil {
+			return fmt.Errorf("unable to decrypt into keys: %v", err)

Review comment:
       same as above RE: `%w` vs `%v`

##########
File path: tools/traffic_vault_migrate/postgres.go
##########
@@ -0,0 +1,716 @@
+package main
+
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+import (
+	"database/sql"
+	"encoding/base64"
+	"encoding/json"
+	"errors"
+	"fmt"
+	"github.com/apache/trafficcontrol/lib/go-tc"
+	util "github.com/apache/trafficcontrol/lib/go-util"
+	_ "github.com/lib/pq"
+	"log"
+	"reflect"
+	"strconv"
+	"strings"
+)
+
+type PGConfig struct {
+	Host     string `json:"host"`
+	Port     string `json:"port"`
+	User     string `json:"user"`
+	Password string `json:"password"`
+	SSLMode  string `json:"sslmode"`
+	Database string `json:"database"`
+	Key      string `json:"aesKey"`
+	AESKey   []byte
+}
+type PGBackend struct {
+	sslKey PGSSLKeyTable
+	dnssec PGDNSSecTable
+	uri    PGURISignKeyTable
+	url    PGURLSigKeyTable
+	cfg    PGConfig
+	db     *sql.DB
+}
+
+func (pg *PGBackend) String() string {
+	data := fmt.Sprintf("PG server %v@%v:%v\n", pg.cfg.User, pg.cfg.Host, pg.cfg.Port)
+	data += fmt.Sprintf("\tSSL Keys: %v\n", len(pg.sslKey.Records))
+	data += fmt.Sprintf("\tDNSSec Keys: %v\n", len(pg.dnssec.Records))
+	data += fmt.Sprintf("\tURI Keys: %v\n", len(pg.uri.Records))
+	data += fmt.Sprintf("\tURL Keys: %v\n", len(pg.url.Records))
+	return data
+}
+func (pg *PGBackend) Name() string {
+	return "PG"
+}
+func (pg *PGBackend) ReadConfig(s string) error {
+	genericCfg, err := UnmarshalConfig(s, reflect.TypeOf(pg.cfg))
+	if err != nil {
+		return err
+	}
+
+	pg.cfg = *genericCfg.Interface().(*PGConfig)
+
+	pg.cfg.AESKey, err = base64.StdEncoding.DecodeString(pg.cfg.Key)
+	if err != nil {
+		return fmt.Errorf("unable to decode PG AESKey: %v", err)
+	}
+	return nil
+}
+func (pg *PGBackend) Insert() error {
+	if err := pg.sslKey.insertKeys(pg.db); err != nil {
+		return err
+	}
+	if err := pg.dnssec.insertKeys(pg.db); err != nil {
+		return err
+	}
+	if err := pg.url.insertKeys(pg.db); err != nil {
+		return err
+	}
+	if err := pg.uri.insertKeys(pg.db); err != nil {
+		return err
+	}
+	return nil
+}
+func (pg *PGBackend) Start() error {
+	sqlStr := fmt.Sprintf("postgres://%s:%s@%s:%s/%s?sslmode=%s", pg.cfg.User, pg.cfg.Password, pg.cfg.Host, pg.cfg.Port, pg.cfg.Database, pg.cfg.SSLMode)
+	db, err := sql.Open("postgres", sqlStr)
+	if err != nil {
+		return fmt.Errorf("Unable to start PG client: %v", err)
+	}
+
+	pg.db = db
+	pg.sslKey = PGSSLKeyTable{}
+	pg.dnssec = PGDNSSecTable{}
+	pg.url = PGURLSigKeyTable{}
+	pg.uri = PGURISignKeyTable{}
+
+	return nil
+}
+func (pg *PGBackend) ValidateKey() []string {
+	var errors []string
+	if errs := pg.sslKey.validate(); errs != nil {
+		errors = append(errors, errs...)
+	}
+	if errs := pg.dnssec.validate(); errs != nil {
+		errors = append(errors, errs...)
+	}
+	if errs := pg.uri.validate(); errs != nil {
+		errors = append(errors, errs...)
+	}
+	if errs := pg.url.validate(); errs != nil {
+		errors = append(errors, errs...)
+	}
+	return errors
+}
+func (pg *PGBackend) Stop() error {
+	return pg.db.Close()
+}
+func (pg *PGBackend) Ping() error {
+	return pg.db.Ping()
+}
+func (pg *PGBackend) Fetch() error {
+	if err := pg.sslKey.gatherKeys(pg.db); err != nil {
+		return err
+	}
+
+	if err := pg.dnssec.gatherKeys(pg.db); err != nil {
+		return err
+	}
+
+	if err := pg.url.gatherKeys(pg.db); err != nil {
+		return err
+	}
+
+	if err := pg.uri.gatherKeys(pg.db); err != nil {
+		return err
+	}
+
+	return nil
+}
+func (pg *PGBackend) GetSSLKeys() ([]SSLKey, error) {
+	if err := pg.sslKey.decrypt(pg.cfg.AESKey); err != nil {
+		return nil, err
+	}
+	return pg.sslKey.toGeneric(), nil
+}
+func (pg *PGBackend) SetSSLKeys(keys []SSLKey) error {
+	pg.sslKey.fromGeneric(keys)
+	return pg.sslKey.encrypt(pg.cfg.AESKey)
+}
+func (pg *PGBackend) GetDNSSecKeys() ([]DNSSecKey, error) {
+	if err := pg.dnssec.decrypt(pg.cfg.AESKey); err != nil {
+		return nil, err
+	}
+	return pg.dnssec.toGeneric(), nil
+}
+func (pg *PGBackend) SetDNSSecKeys(keys []DNSSecKey) error {
+	pg.dnssec.fromGeneric(keys)
+	return pg.dnssec.encrypt(pg.cfg.AESKey)
+}
+func (pg *PGBackend) GetURISignKeys() ([]URISignKey, error) {
+	if err := pg.uri.decrypt(pg.cfg.AESKey); err != nil {
+		return nil, err
+	}
+	return pg.uri.toGeneric(), nil
+}
+func (pg *PGBackend) SetURISignKeys(keys []URISignKey) error {
+	pg.uri.fromGeneric(keys)
+	return pg.uri.encrypt(pg.cfg.AESKey)
+}
+func (pg *PGBackend) GetURLSigKeys() ([]URLSigKey, error) {
+	if err := pg.url.decrypt(pg.cfg.AESKey); err != nil {
+		return nil, err
+	}
+	return pg.url.toGeneric(), nil
+}
+func (pg *PGBackend) SetURLSigKeys(keys []URLSigKey) error {
+	pg.url.fromGeneric(keys)
+	return pg.url.encrypt(pg.cfg.AESKey)
+}
+
+type PGCommonRecord struct {
+	DataEncrypted []byte
+	CommonRecord
+}
+
+type PGDNSSecRecord struct {
+	Key tc.DNSSECKeysTrafficVault
+	CDN string
+	PGCommonRecord
+}
+type PGDNSSecTable struct {
+	Records []PGDNSSecRecord
+}
+
+func (tbl *PGDNSSecTable) gatherKeys(db *sql.DB) error {
+	sz, err := getSize(db, "dnssec")
+	if err != nil {
+		log.Println("PGDNSSec gatherKeys: unable to determine size of dnssec table")
+	}
+	tbl.Records = make([]PGDNSSecRecord, sz)
+
+	rows, err := db.Query("SELECT cdn, data from dnssec")
+	if err != nil {
+		return fmt.Errorf("PGDNSSec gatherKeys: unable to query: %v", err)
+	}
+	defer rows.Close()
+	i := 0
+	for rows.Next() {
+		if i > len(tbl.Records)-1 {
+			return fmt.Errorf("PGDNSSec gatherKeys got more results than expected %v", len(tbl.Records))
+		}
+		err := rows.Scan(&tbl.Records[i].CDN, &tbl.Records[i].DataEncrypted)
+		if err != nil {
+			return fmt.Errorf("PGDNSSec gatherKeys unable to scan row: %v", err)
+		}
+		i += 1
+	}
+	return nil
+}
+func (tbl *PGDNSSecTable) decrypt(aesKey []byte) error {
+	for i, _ := range tbl.Records {
+		data, err := decryptInto(aesKey, tbl.Records[i].DataEncrypted, reflect.TypeOf(tbl.Records[i].Key))
+		if err != nil {
+			return fmt.Errorf("unable to decrypt into keys: %v", err)
+		}
+		tbl.Records[i].Key = *data.Interface().(*tc.DNSSECKeysTrafficVault)
+	}
+	return nil
+}
+func (tbl *PGDNSSecTable) encrypt(aesKey []byte) error {
+	for i, dns := range tbl.Records {
+		data, err := json.Marshal(&dns.Key)
+		if err != nil {
+			return fmt.Errorf("encrypt issue marshalling keys: %v", err)
+		}
+		dat, err := encrypt(data, aesKey)
+		if err != nil {
+			return fmt.Errorf("encrypt error: %v", err)
+		}
+		tbl.Records[i].DataEncrypted = dat
+	}
+	return nil
+}
+func (tbl *PGDNSSecTable) toGeneric() []DNSSecKey {
+	keys := make([]DNSSecKey, len(tbl.Records))
+
+	for i, record := range tbl.Records {
+		keys[i] = DNSSecKey{
+			CDN:                    record.CDN,
+			DNSSECKeysTrafficVault: record.Key,
+			CommonRecord:           record.CommonRecord,
+		}
+	}
+
+	return keys
+}
+func (tbl *PGDNSSecTable) fromGeneric(keys []DNSSecKey) {
+	tbl.Records = make([]PGDNSSecRecord, len(keys))
+
+	for i, key := range keys {
+		tbl.Records[i] = PGDNSSecRecord{
+			Key: key.DNSSECKeysTrafficVault,
+			CDN: key.CDN,
+			PGCommonRecord: PGCommonRecord{
+				DataEncrypted: nil,
+				CommonRecord:  CommonRecord{},
+			},
+		}
+	}
+}
+func (tbl *PGDNSSecTable) validate() []string {
+	for i, record := range tbl.Records {
+		if record.DataEncrypted == nil && len(record.Key) > 0 {
+			return []string{fmt.Sprintf("DNSSEC Key %v: DataEncrypted is blank!", i)}
+		}
+	}
+	return nil
+}
+func (tbl *PGDNSSecTable) insertKeys(db *sql.DB) error {
+	queryFmt := "INSERT INTO dnssec (cdn, data) VALUES "
+	stride := 2
+	queryArgs := make([]interface{}, len(tbl.Records)*stride)
+	for i, record := range tbl.Records {
+		j := i * stride
+		queryArgs[j] = record.CDN
+		queryArgs[j+1] = record.DataEncrypted
+	}
+	return insertIntoTable(db, queryFmt, stride, queryArgs)
+}
+
+type PGSSLKeyRecord struct {
+	Keys tc.DeliveryServiceSSLKeys
+	PGCommonRecord
+
+	// These records are stored on the table but are duplicated
+	DeliveryService string
+	Version         string
+	CDN             string
+}
+type PGSSLKeyTable struct {
+	Records []PGSSLKeyRecord
+}
+
+func (tbl *PGSSLKeyTable) insertKeys(db *sql.DB) error {
+	queryFmt := "INSERT INTO sslkey (deliveryservice, data, cdn, version) VALUES "
+	stride := 4
+	queryArgs := make([]interface{}, len(tbl.Records)*stride)
+	for i, record := range tbl.Records {
+		j := i * stride
+		queryArgs[j] = record.DeliveryService
+		queryArgs[j+1] = record.DataEncrypted
+		queryArgs[j+2] = record.CDN
+		queryArgs[j+3] = record.Version
+	}
+	return insertIntoTable(db, queryFmt, 4, queryArgs)
+}
+func (tbl *PGSSLKeyTable) gatherKeys(db *sql.DB) error {
+	sz, err := getSize(db, "sslkey")
+	if err != nil {
+		return fmt.Errorf("PGSSLKey gatherKeys unable to determine size of sslkey table: %v", err)
+	}
+	tbl.Records = make([]PGSSLKeyRecord, sz)
+
+	rows, err := db.Query("SELECT data, deliveryservice, cdn, version from sslkey")
+	if err != nil {
+		return fmt.Errorf("PGSSLKey gatherKeys unable to query: %v", err)
+	}
+	defer rows.Close()
+	i := 0
+	for rows.Next() {
+		if i > len(tbl.Records)-1 {
+			return errors.New("PGSSLKey gatherKeys: got more results than expected")
+		}
+		err := rows.Scan(&tbl.Records[i].DataEncrypted, &tbl.Records[i].DeliveryService, &tbl.Records[i].CDN, &tbl.Records[i].Version)
+		if err != nil {
+			return fmt.Errorf("PGSSLKey gatherKeys unable to scan row: %v", err)
+		}
+		i += 1
+	}
+	return nil
+}
+func (tbl *PGSSLKeyTable) decrypt(aesKey []byte) error {
+	for i, dns := range tbl.Records {
+		data, err := decryptInto(aesKey, dns.DataEncrypted, reflect.TypeOf(dns.Keys))
+		if err != nil {
+			return fmt.Errorf("unable to decrypt into keys: %v", err)
+		}
+		tbl.Records[i].Keys = *data.Interface().(*tc.DeliveryServiceSSLKeys)
+	}
+	return nil
+}
+func (tbl *PGSSLKeyTable) encrypt(aesKey []byte) error {
+	for i, dns := range tbl.Records {
+		data, err := json.Marshal(dns.Keys)
+		if err != nil {
+			return fmt.Errorf("encrypt issue marshalling keys: %v", err)
+		}
+		dat, err := encrypt(data, aesKey)
+		if err != nil {
+			return fmt.Errorf("encrypt error: %v", err)
+		}
+		tbl.Records[i].DataEncrypted = dat
+	}
+	return nil
+}
+func (tbl *PGSSLKeyTable) toGeneric() []SSLKey {
+	keys := make([]SSLKey, len(tbl.Records))
+
+	for i, record := range tbl.Records {
+		keys[i] = SSLKey{
+			DeliveryServiceSSLKeys: record.Keys,
+			CommonRecord:           record.CommonRecord,
+		}
+	}
+	return keys
+}
+func (tbl *PGSSLKeyTable) fromGeneric(keys []SSLKey) {
+	tbl.Records = make([]PGSSLKeyRecord, len(keys))
+
+	for i, key := range keys {
+		tbl.Records[i] = PGSSLKeyRecord{
+			Keys: key.DeliveryServiceSSLKeys,
+			PGCommonRecord: PGCommonRecord{
+				DataEncrypted: nil,
+				CommonRecord:  CommonRecord{},
+			},
+			DeliveryService: key.DeliveryService,
+			CDN:             key.CDN,
+		}
+	}
+}
+func (tbl *PGSSLKeyTable) validate() []string {
+	defaultKey := tc.DeliveryServiceSSLKeys{}
+	var errors []string
+	fmtStr := "SSL Key %v: %v"
+	for i, record := range tbl.Records {
+		if record.Keys == defaultKey {
+			errors = append(errors, fmt.Sprintf(fmtStr, i, "DS SSL Keys are default!"))
+		} else if record.Keys.Key == "" {
+			errors = append(errors, fmt.Sprintf(fmtStr, i, "Key is blank!"))
+		} else if record.Keys.CDN == "" {
+			errors = append(errors, fmt.Sprintf(fmtStr, i, "CDN is blank!"))
+		} else if record.Keys.DeliveryService == "" {
+			errors = append(errors, fmt.Sprintf(fmtStr, i, "DS is blank!"))
+		} else if record.DataEncrypted == nil {
+			errors = append(errors, fmt.Sprintf(fmtStr, i, "DataEncrypted is blank!"))
+		}
+	}
+	return errors
+}
+
+type PGURLSigKeyRecord struct {
+	Keys            tc.URLSigKeys
+	DeliveryService string
+	PGCommonRecord
+}
+type PGURLSigKeyTable struct {
+	Records []PGURLSigKeyRecord
+}

Review comment:
       GoDoc?

##########
File path: tools/traffic_vault_migrate/postgres.go
##########
@@ -0,0 +1,716 @@
+package main
+
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+import (
+	"database/sql"
+	"encoding/base64"
+	"encoding/json"
+	"errors"
+	"fmt"
+	"github.com/apache/trafficcontrol/lib/go-tc"
+	util "github.com/apache/trafficcontrol/lib/go-util"
+	_ "github.com/lib/pq"
+	"log"
+	"reflect"
+	"strconv"
+	"strings"
+)
+
+type PGConfig struct {
+	Host     string `json:"host"`
+	Port     string `json:"port"`
+	User     string `json:"user"`
+	Password string `json:"password"`
+	SSLMode  string `json:"sslmode"`
+	Database string `json:"database"`
+	Key      string `json:"aesKey"`
+	AESKey   []byte
+}
+type PGBackend struct {
+	sslKey PGSSLKeyTable
+	dnssec PGDNSSecTable
+	uri    PGURISignKeyTable
+	url    PGURLSigKeyTable
+	cfg    PGConfig
+	db     *sql.DB
+}
+
+func (pg *PGBackend) String() string {
+	data := fmt.Sprintf("PG server %v@%v:%v\n", pg.cfg.User, pg.cfg.Host, pg.cfg.Port)
+	data += fmt.Sprintf("\tSSL Keys: %v\n", len(pg.sslKey.Records))
+	data += fmt.Sprintf("\tDNSSec Keys: %v\n", len(pg.dnssec.Records))
+	data += fmt.Sprintf("\tURI Keys: %v\n", len(pg.uri.Records))
+	data += fmt.Sprintf("\tURL Keys: %v\n", len(pg.url.Records))
+	return data
+}
+func (pg *PGBackend) Name() string {
+	return "PG"
+}
+func (pg *PGBackend) ReadConfig(s string) error {
+	genericCfg, err := UnmarshalConfig(s, reflect.TypeOf(pg.cfg))
+	if err != nil {
+		return err
+	}
+
+	pg.cfg = *genericCfg.Interface().(*PGConfig)
+
+	pg.cfg.AESKey, err = base64.StdEncoding.DecodeString(pg.cfg.Key)
+	if err != nil {
+		return fmt.Errorf("unable to decode PG AESKey: %v", err)
+	}
+	return nil
+}
+func (pg *PGBackend) Insert() error {
+	if err := pg.sslKey.insertKeys(pg.db); err != nil {
+		return err
+	}
+	if err := pg.dnssec.insertKeys(pg.db); err != nil {
+		return err
+	}
+	if err := pg.url.insertKeys(pg.db); err != nil {
+		return err
+	}
+	if err := pg.uri.insertKeys(pg.db); err != nil {
+		return err
+	}
+	return nil
+}
+func (pg *PGBackend) Start() error {
+	sqlStr := fmt.Sprintf("postgres://%s:%s@%s:%s/%s?sslmode=%s", pg.cfg.User, pg.cfg.Password, pg.cfg.Host, pg.cfg.Port, pg.cfg.Database, pg.cfg.SSLMode)
+	db, err := sql.Open("postgres", sqlStr)
+	if err != nil {
+		return fmt.Errorf("Unable to start PG client: %v", err)
+	}
+
+	pg.db = db
+	pg.sslKey = PGSSLKeyTable{}
+	pg.dnssec = PGDNSSecTable{}
+	pg.url = PGURLSigKeyTable{}
+	pg.uri = PGURISignKeyTable{}
+
+	return nil
+}
+func (pg *PGBackend) ValidateKey() []string {
+	var errors []string
+	if errs := pg.sslKey.validate(); errs != nil {
+		errors = append(errors, errs...)
+	}
+	if errs := pg.dnssec.validate(); errs != nil {
+		errors = append(errors, errs...)
+	}
+	if errs := pg.uri.validate(); errs != nil {
+		errors = append(errors, errs...)
+	}
+	if errs := pg.url.validate(); errs != nil {
+		errors = append(errors, errs...)
+	}
+	return errors
+}
+func (pg *PGBackend) Stop() error {
+	return pg.db.Close()
+}
+func (pg *PGBackend) Ping() error {
+	return pg.db.Ping()
+}
+func (pg *PGBackend) Fetch() error {
+	if err := pg.sslKey.gatherKeys(pg.db); err != nil {
+		return err
+	}
+
+	if err := pg.dnssec.gatherKeys(pg.db); err != nil {
+		return err
+	}
+
+	if err := pg.url.gatherKeys(pg.db); err != nil {
+		return err
+	}
+
+	if err := pg.uri.gatherKeys(pg.db); err != nil {
+		return err
+	}
+
+	return nil
+}
+func (pg *PGBackend) GetSSLKeys() ([]SSLKey, error) {
+	if err := pg.sslKey.decrypt(pg.cfg.AESKey); err != nil {
+		return nil, err
+	}
+	return pg.sslKey.toGeneric(), nil
+}
+func (pg *PGBackend) SetSSLKeys(keys []SSLKey) error {
+	pg.sslKey.fromGeneric(keys)
+	return pg.sslKey.encrypt(pg.cfg.AESKey)
+}
+func (pg *PGBackend) GetDNSSecKeys() ([]DNSSecKey, error) {
+	if err := pg.dnssec.decrypt(pg.cfg.AESKey); err != nil {
+		return nil, err
+	}
+	return pg.dnssec.toGeneric(), nil
+}
+func (pg *PGBackend) SetDNSSecKeys(keys []DNSSecKey) error {
+	pg.dnssec.fromGeneric(keys)
+	return pg.dnssec.encrypt(pg.cfg.AESKey)
+}
+func (pg *PGBackend) GetURISignKeys() ([]URISignKey, error) {
+	if err := pg.uri.decrypt(pg.cfg.AESKey); err != nil {
+		return nil, err
+	}
+	return pg.uri.toGeneric(), nil
+}
+func (pg *PGBackend) SetURISignKeys(keys []URISignKey) error {
+	pg.uri.fromGeneric(keys)
+	return pg.uri.encrypt(pg.cfg.AESKey)
+}
+func (pg *PGBackend) GetURLSigKeys() ([]URLSigKey, error) {
+	if err := pg.url.decrypt(pg.cfg.AESKey); err != nil {
+		return nil, err
+	}
+	return pg.url.toGeneric(), nil
+}
+func (pg *PGBackend) SetURLSigKeys(keys []URLSigKey) error {
+	pg.url.fromGeneric(keys)
+	return pg.url.encrypt(pg.cfg.AESKey)
+}
+
+type PGCommonRecord struct {
+	DataEncrypted []byte
+	CommonRecord
+}
+
+type PGDNSSecRecord struct {
+	Key tc.DNSSECKeysTrafficVault
+	CDN string
+	PGCommonRecord
+}
+type PGDNSSecTable struct {
+	Records []PGDNSSecRecord
+}
+
+func (tbl *PGDNSSecTable) gatherKeys(db *sql.DB) error {
+	sz, err := getSize(db, "dnssec")
+	if err != nil {
+		log.Println("PGDNSSec gatherKeys: unable to determine size of dnssec table")
+	}
+	tbl.Records = make([]PGDNSSecRecord, sz)
+
+	rows, err := db.Query("SELECT cdn, data from dnssec")
+	if err != nil {
+		return fmt.Errorf("PGDNSSec gatherKeys: unable to query: %v", err)
+	}
+	defer rows.Close()
+	i := 0
+	for rows.Next() {
+		if i > len(tbl.Records)-1 {
+			return fmt.Errorf("PGDNSSec gatherKeys got more results than expected %v", len(tbl.Records))
+		}
+		err := rows.Scan(&tbl.Records[i].CDN, &tbl.Records[i].DataEncrypted)
+		if err != nil {
+			return fmt.Errorf("PGDNSSec gatherKeys unable to scan row: %v", err)
+		}
+		i += 1
+	}
+	return nil
+}
+func (tbl *PGDNSSecTable) decrypt(aesKey []byte) error {
+	for i, _ := range tbl.Records {
+		data, err := decryptInto(aesKey, tbl.Records[i].DataEncrypted, reflect.TypeOf(tbl.Records[i].Key))
+		if err != nil {
+			return fmt.Errorf("unable to decrypt into keys: %v", err)
+		}
+		tbl.Records[i].Key = *data.Interface().(*tc.DNSSECKeysTrafficVault)
+	}
+	return nil
+}
+func (tbl *PGDNSSecTable) encrypt(aesKey []byte) error {
+	for i, dns := range tbl.Records {
+		data, err := json.Marshal(&dns.Key)
+		if err != nil {
+			return fmt.Errorf("encrypt issue marshalling keys: %v", err)
+		}
+		dat, err := encrypt(data, aesKey)
+		if err != nil {
+			return fmt.Errorf("encrypt error: %v", err)

Review comment:
       same as above RE: `%w` vs `%v`

##########
File path: tools/traffic_vault_migrate/postgres.go
##########
@@ -0,0 +1,716 @@
+package main
+
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+import (
+	"database/sql"
+	"encoding/base64"
+	"encoding/json"
+	"errors"
+	"fmt"
+	"github.com/apache/trafficcontrol/lib/go-tc"
+	util "github.com/apache/trafficcontrol/lib/go-util"
+	_ "github.com/lib/pq"
+	"log"
+	"reflect"
+	"strconv"
+	"strings"
+)
+
+type PGConfig struct {
+	Host     string `json:"host"`
+	Port     string `json:"port"`
+	User     string `json:"user"`
+	Password string `json:"password"`
+	SSLMode  string `json:"sslmode"`
+	Database string `json:"database"`
+	Key      string `json:"aesKey"`
+	AESKey   []byte
+}
+type PGBackend struct {
+	sslKey PGSSLKeyTable
+	dnssec PGDNSSecTable
+	uri    PGURISignKeyTable
+	url    PGURLSigKeyTable
+	cfg    PGConfig
+	db     *sql.DB
+}
+
+func (pg *PGBackend) String() string {
+	data := fmt.Sprintf("PG server %v@%v:%v\n", pg.cfg.User, pg.cfg.Host, pg.cfg.Port)
+	data += fmt.Sprintf("\tSSL Keys: %v\n", len(pg.sslKey.Records))
+	data += fmt.Sprintf("\tDNSSec Keys: %v\n", len(pg.dnssec.Records))
+	data += fmt.Sprintf("\tURI Keys: %v\n", len(pg.uri.Records))
+	data += fmt.Sprintf("\tURL Keys: %v\n", len(pg.url.Records))
+	return data
+}
+func (pg *PGBackend) Name() string {
+	return "PG"
+}
+func (pg *PGBackend) ReadConfig(s string) error {
+	genericCfg, err := UnmarshalConfig(s, reflect.TypeOf(pg.cfg))
+	if err != nil {
+		return err
+	}
+
+	pg.cfg = *genericCfg.Interface().(*PGConfig)
+
+	pg.cfg.AESKey, err = base64.StdEncoding.DecodeString(pg.cfg.Key)
+	if err != nil {
+		return fmt.Errorf("unable to decode PG AESKey: %v", err)
+	}
+	return nil
+}
+func (pg *PGBackend) Insert() error {
+	if err := pg.sslKey.insertKeys(pg.db); err != nil {
+		return err
+	}
+	if err := pg.dnssec.insertKeys(pg.db); err != nil {
+		return err
+	}
+	if err := pg.url.insertKeys(pg.db); err != nil {
+		return err
+	}
+	if err := pg.uri.insertKeys(pg.db); err != nil {
+		return err
+	}
+	return nil
+}
+func (pg *PGBackend) Start() error {
+	sqlStr := fmt.Sprintf("postgres://%s:%s@%s:%s/%s?sslmode=%s", pg.cfg.User, pg.cfg.Password, pg.cfg.Host, pg.cfg.Port, pg.cfg.Database, pg.cfg.SSLMode)
+	db, err := sql.Open("postgres", sqlStr)
+	if err != nil {
+		return fmt.Errorf("Unable to start PG client: %v", err)
+	}
+
+	pg.db = db
+	pg.sslKey = PGSSLKeyTable{}
+	pg.dnssec = PGDNSSecTable{}
+	pg.url = PGURLSigKeyTable{}
+	pg.uri = PGURISignKeyTable{}
+
+	return nil
+}
+func (pg *PGBackend) ValidateKey() []string {
+	var errors []string
+	if errs := pg.sslKey.validate(); errs != nil {
+		errors = append(errors, errs...)
+	}
+	if errs := pg.dnssec.validate(); errs != nil {
+		errors = append(errors, errs...)
+	}
+	if errs := pg.uri.validate(); errs != nil {
+		errors = append(errors, errs...)
+	}
+	if errs := pg.url.validate(); errs != nil {
+		errors = append(errors, errs...)
+	}
+	return errors
+}
+func (pg *PGBackend) Stop() error {
+	return pg.db.Close()
+}
+func (pg *PGBackend) Ping() error {
+	return pg.db.Ping()
+}
+func (pg *PGBackend) Fetch() error {
+	if err := pg.sslKey.gatherKeys(pg.db); err != nil {
+		return err
+	}
+
+	if err := pg.dnssec.gatherKeys(pg.db); err != nil {
+		return err
+	}
+
+	if err := pg.url.gatherKeys(pg.db); err != nil {
+		return err
+	}
+
+	if err := pg.uri.gatherKeys(pg.db); err != nil {
+		return err
+	}
+
+	return nil
+}
+func (pg *PGBackend) GetSSLKeys() ([]SSLKey, error) {
+	if err := pg.sslKey.decrypt(pg.cfg.AESKey); err != nil {
+		return nil, err
+	}
+	return pg.sslKey.toGeneric(), nil
+}
+func (pg *PGBackend) SetSSLKeys(keys []SSLKey) error {
+	pg.sslKey.fromGeneric(keys)
+	return pg.sslKey.encrypt(pg.cfg.AESKey)
+}
+func (pg *PGBackend) GetDNSSecKeys() ([]DNSSecKey, error) {
+	if err := pg.dnssec.decrypt(pg.cfg.AESKey); err != nil {
+		return nil, err
+	}
+	return pg.dnssec.toGeneric(), nil
+}
+func (pg *PGBackend) SetDNSSecKeys(keys []DNSSecKey) error {
+	pg.dnssec.fromGeneric(keys)
+	return pg.dnssec.encrypt(pg.cfg.AESKey)
+}
+func (pg *PGBackend) GetURISignKeys() ([]URISignKey, error) {
+	if err := pg.uri.decrypt(pg.cfg.AESKey); err != nil {
+		return nil, err
+	}
+	return pg.uri.toGeneric(), nil
+}
+func (pg *PGBackend) SetURISignKeys(keys []URISignKey) error {
+	pg.uri.fromGeneric(keys)
+	return pg.uri.encrypt(pg.cfg.AESKey)
+}
+func (pg *PGBackend) GetURLSigKeys() ([]URLSigKey, error) {
+	if err := pg.url.decrypt(pg.cfg.AESKey); err != nil {
+		return nil, err
+	}
+	return pg.url.toGeneric(), nil
+}
+func (pg *PGBackend) SetURLSigKeys(keys []URLSigKey) error {
+	pg.url.fromGeneric(keys)
+	return pg.url.encrypt(pg.cfg.AESKey)
+}
+
+type PGCommonRecord struct {
+	DataEncrypted []byte
+	CommonRecord
+}
+
+type PGDNSSecRecord struct {
+	Key tc.DNSSECKeysTrafficVault
+	CDN string
+	PGCommonRecord
+}
+type PGDNSSecTable struct {
+	Records []PGDNSSecRecord
+}
+
+func (tbl *PGDNSSecTable) gatherKeys(db *sql.DB) error {
+	sz, err := getSize(db, "dnssec")
+	if err != nil {
+		log.Println("PGDNSSec gatherKeys: unable to determine size of dnssec table")
+	}
+	tbl.Records = make([]PGDNSSecRecord, sz)
+
+	rows, err := db.Query("SELECT cdn, data from dnssec")
+	if err != nil {
+		return fmt.Errorf("PGDNSSec gatherKeys: unable to query: %v", err)
+	}
+	defer rows.Close()
+	i := 0
+	for rows.Next() {
+		if i > len(tbl.Records)-1 {
+			return fmt.Errorf("PGDNSSec gatherKeys got more results than expected %v", len(tbl.Records))
+		}
+		err := rows.Scan(&tbl.Records[i].CDN, &tbl.Records[i].DataEncrypted)
+		if err != nil {
+			return fmt.Errorf("PGDNSSec gatherKeys unable to scan row: %v", err)
+		}
+		i += 1
+	}
+	return nil
+}
+func (tbl *PGDNSSecTable) decrypt(aesKey []byte) error {
+	for i, _ := range tbl.Records {
+		data, err := decryptInto(aesKey, tbl.Records[i].DataEncrypted, reflect.TypeOf(tbl.Records[i].Key))
+		if err != nil {
+			return fmt.Errorf("unable to decrypt into keys: %v", err)
+		}
+		tbl.Records[i].Key = *data.Interface().(*tc.DNSSECKeysTrafficVault)
+	}
+	return nil
+}
+func (tbl *PGDNSSecTable) encrypt(aesKey []byte) error {
+	for i, dns := range tbl.Records {
+		data, err := json.Marshal(&dns.Key)
+		if err != nil {
+			return fmt.Errorf("encrypt issue marshalling keys: %v", err)

Review comment:
       same as above RE: `%w` vs `%v`

##########
File path: tools/traffic_vault_migrate/postgres.go
##########
@@ -0,0 +1,716 @@
+package main
+
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+import (
+	"database/sql"
+	"encoding/base64"
+	"encoding/json"
+	"errors"
+	"fmt"
+	"github.com/apache/trafficcontrol/lib/go-tc"
+	util "github.com/apache/trafficcontrol/lib/go-util"
+	_ "github.com/lib/pq"
+	"log"
+	"reflect"
+	"strconv"
+	"strings"
+)
+
+type PGConfig struct {
+	Host     string `json:"host"`
+	Port     string `json:"port"`
+	User     string `json:"user"`
+	Password string `json:"password"`
+	SSLMode  string `json:"sslmode"`
+	Database string `json:"database"`
+	Key      string `json:"aesKey"`
+	AESKey   []byte
+}
+type PGBackend struct {
+	sslKey PGSSLKeyTable
+	dnssec PGDNSSecTable
+	uri    PGURISignKeyTable
+	url    PGURLSigKeyTable
+	cfg    PGConfig
+	db     *sql.DB
+}
+
+func (pg *PGBackend) String() string {
+	data := fmt.Sprintf("PG server %v@%v:%v\n", pg.cfg.User, pg.cfg.Host, pg.cfg.Port)
+	data += fmt.Sprintf("\tSSL Keys: %v\n", len(pg.sslKey.Records))
+	data += fmt.Sprintf("\tDNSSec Keys: %v\n", len(pg.dnssec.Records))
+	data += fmt.Sprintf("\tURI Keys: %v\n", len(pg.uri.Records))
+	data += fmt.Sprintf("\tURL Keys: %v\n", len(pg.url.Records))
+	return data
+}
+func (pg *PGBackend) Name() string {
+	return "PG"
+}
+func (pg *PGBackend) ReadConfig(s string) error {
+	genericCfg, err := UnmarshalConfig(s, reflect.TypeOf(pg.cfg))
+	if err != nil {
+		return err
+	}
+
+	pg.cfg = *genericCfg.Interface().(*PGConfig)
+
+	pg.cfg.AESKey, err = base64.StdEncoding.DecodeString(pg.cfg.Key)
+	if err != nil {
+		return fmt.Errorf("unable to decode PG AESKey: %v", err)
+	}
+	return nil
+}
+func (pg *PGBackend) Insert() error {
+	if err := pg.sslKey.insertKeys(pg.db); err != nil {
+		return err
+	}
+	if err := pg.dnssec.insertKeys(pg.db); err != nil {
+		return err
+	}
+	if err := pg.url.insertKeys(pg.db); err != nil {
+		return err
+	}
+	if err := pg.uri.insertKeys(pg.db); err != nil {
+		return err
+	}
+	return nil
+}
+func (pg *PGBackend) Start() error {
+	sqlStr := fmt.Sprintf("postgres://%s:%s@%s:%s/%s?sslmode=%s", pg.cfg.User, pg.cfg.Password, pg.cfg.Host, pg.cfg.Port, pg.cfg.Database, pg.cfg.SSLMode)
+	db, err := sql.Open("postgres", sqlStr)
+	if err != nil {
+		return fmt.Errorf("Unable to start PG client: %v", err)
+	}
+
+	pg.db = db
+	pg.sslKey = PGSSLKeyTable{}
+	pg.dnssec = PGDNSSecTable{}
+	pg.url = PGURLSigKeyTable{}
+	pg.uri = PGURISignKeyTable{}
+
+	return nil
+}
+func (pg *PGBackend) ValidateKey() []string {
+	var errors []string
+	if errs := pg.sslKey.validate(); errs != nil {
+		errors = append(errors, errs...)
+	}
+	if errs := pg.dnssec.validate(); errs != nil {
+		errors = append(errors, errs...)
+	}
+	if errs := pg.uri.validate(); errs != nil {
+		errors = append(errors, errs...)
+	}
+	if errs := pg.url.validate(); errs != nil {
+		errors = append(errors, errs...)
+	}
+	return errors
+}
+func (pg *PGBackend) Stop() error {
+	return pg.db.Close()
+}
+func (pg *PGBackend) Ping() error {
+	return pg.db.Ping()
+}
+func (pg *PGBackend) Fetch() error {
+	if err := pg.sslKey.gatherKeys(pg.db); err != nil {
+		return err
+	}
+
+	if err := pg.dnssec.gatherKeys(pg.db); err != nil {
+		return err
+	}
+
+	if err := pg.url.gatherKeys(pg.db); err != nil {
+		return err
+	}
+
+	if err := pg.uri.gatherKeys(pg.db); err != nil {
+		return err
+	}
+
+	return nil
+}
+func (pg *PGBackend) GetSSLKeys() ([]SSLKey, error) {
+	if err := pg.sslKey.decrypt(pg.cfg.AESKey); err != nil {
+		return nil, err
+	}
+	return pg.sslKey.toGeneric(), nil
+}
+func (pg *PGBackend) SetSSLKeys(keys []SSLKey) error {
+	pg.sslKey.fromGeneric(keys)
+	return pg.sslKey.encrypt(pg.cfg.AESKey)
+}
+func (pg *PGBackend) GetDNSSecKeys() ([]DNSSecKey, error) {
+	if err := pg.dnssec.decrypt(pg.cfg.AESKey); err != nil {
+		return nil, err
+	}
+	return pg.dnssec.toGeneric(), nil
+}
+func (pg *PGBackend) SetDNSSecKeys(keys []DNSSecKey) error {
+	pg.dnssec.fromGeneric(keys)
+	return pg.dnssec.encrypt(pg.cfg.AESKey)
+}
+func (pg *PGBackend) GetURISignKeys() ([]URISignKey, error) {
+	if err := pg.uri.decrypt(pg.cfg.AESKey); err != nil {
+		return nil, err
+	}
+	return pg.uri.toGeneric(), nil
+}
+func (pg *PGBackend) SetURISignKeys(keys []URISignKey) error {
+	pg.uri.fromGeneric(keys)
+	return pg.uri.encrypt(pg.cfg.AESKey)
+}
+func (pg *PGBackend) GetURLSigKeys() ([]URLSigKey, error) {
+	if err := pg.url.decrypt(pg.cfg.AESKey); err != nil {
+		return nil, err
+	}
+	return pg.url.toGeneric(), nil
+}
+func (pg *PGBackend) SetURLSigKeys(keys []URLSigKey) error {
+	pg.url.fromGeneric(keys)
+	return pg.url.encrypt(pg.cfg.AESKey)
+}
+
+type PGCommonRecord struct {
+	DataEncrypted []byte
+	CommonRecord
+}
+
+type PGDNSSecRecord struct {
+	Key tc.DNSSECKeysTrafficVault
+	CDN string
+	PGCommonRecord
+}
+type PGDNSSecTable struct {
+	Records []PGDNSSecRecord
+}
+
+func (tbl *PGDNSSecTable) gatherKeys(db *sql.DB) error {
+	sz, err := getSize(db, "dnssec")
+	if err != nil {
+		log.Println("PGDNSSec gatherKeys: unable to determine size of dnssec table")
+	}
+	tbl.Records = make([]PGDNSSecRecord, sz)
+
+	rows, err := db.Query("SELECT cdn, data from dnssec")
+	if err != nil {
+		return fmt.Errorf("PGDNSSec gatherKeys: unable to query: %v", err)
+	}
+	defer rows.Close()
+	i := 0
+	for rows.Next() {
+		if i > len(tbl.Records)-1 {
+			return fmt.Errorf("PGDNSSec gatherKeys got more results than expected %v", len(tbl.Records))
+		}
+		err := rows.Scan(&tbl.Records[i].CDN, &tbl.Records[i].DataEncrypted)
+		if err != nil {
+			return fmt.Errorf("PGDNSSec gatherKeys unable to scan row: %v", err)
+		}
+		i += 1
+	}
+	return nil
+}
+func (tbl *PGDNSSecTable) decrypt(aesKey []byte) error {
+	for i, _ := range tbl.Records {
+		data, err := decryptInto(aesKey, tbl.Records[i].DataEncrypted, reflect.TypeOf(tbl.Records[i].Key))
+		if err != nil {
+			return fmt.Errorf("unable to decrypt into keys: %v", err)
+		}
+		tbl.Records[i].Key = *data.Interface().(*tc.DNSSECKeysTrafficVault)
+	}
+	return nil
+}
+func (tbl *PGDNSSecTable) encrypt(aesKey []byte) error {
+	for i, dns := range tbl.Records {
+		data, err := json.Marshal(&dns.Key)
+		if err != nil {
+			return fmt.Errorf("encrypt issue marshalling keys: %v", err)
+		}
+		dat, err := encrypt(data, aesKey)
+		if err != nil {
+			return fmt.Errorf("encrypt error: %v", err)
+		}
+		tbl.Records[i].DataEncrypted = dat
+	}
+	return nil
+}
+func (tbl *PGDNSSecTable) toGeneric() []DNSSecKey {
+	keys := make([]DNSSecKey, len(tbl.Records))
+
+	for i, record := range tbl.Records {
+		keys[i] = DNSSecKey{
+			CDN:                    record.CDN,
+			DNSSECKeysTrafficVault: record.Key,
+			CommonRecord:           record.CommonRecord,
+		}
+	}
+
+	return keys
+}
+func (tbl *PGDNSSecTable) fromGeneric(keys []DNSSecKey) {
+	tbl.Records = make([]PGDNSSecRecord, len(keys))
+
+	for i, key := range keys {
+		tbl.Records[i] = PGDNSSecRecord{
+			Key: key.DNSSECKeysTrafficVault,
+			CDN: key.CDN,
+			PGCommonRecord: PGCommonRecord{
+				DataEncrypted: nil,
+				CommonRecord:  CommonRecord{},
+			},
+		}
+	}
+}
+func (tbl *PGDNSSecTable) validate() []string {
+	for i, record := range tbl.Records {
+		if record.DataEncrypted == nil && len(record.Key) > 0 {
+			return []string{fmt.Sprintf("DNSSEC Key %v: DataEncrypted is blank!", i)}
+		}
+	}
+	return nil
+}
+func (tbl *PGDNSSecTable) insertKeys(db *sql.DB) error {
+	queryFmt := "INSERT INTO dnssec (cdn, data) VALUES "
+	stride := 2
+	queryArgs := make([]interface{}, len(tbl.Records)*stride)
+	for i, record := range tbl.Records {
+		j := i * stride
+		queryArgs[j] = record.CDN
+		queryArgs[j+1] = record.DataEncrypted
+	}
+	return insertIntoTable(db, queryFmt, stride, queryArgs)
+}
+
+type PGSSLKeyRecord struct {
+	Keys tc.DeliveryServiceSSLKeys
+	PGCommonRecord
+
+	// These records are stored on the table but are duplicated
+	DeliveryService string
+	Version         string
+	CDN             string
+}
+type PGSSLKeyTable struct {
+	Records []PGSSLKeyRecord
+}
+
+func (tbl *PGSSLKeyTable) insertKeys(db *sql.DB) error {
+	queryFmt := "INSERT INTO sslkey (deliveryservice, data, cdn, version) VALUES "
+	stride := 4
+	queryArgs := make([]interface{}, len(tbl.Records)*stride)
+	for i, record := range tbl.Records {
+		j := i * stride
+		queryArgs[j] = record.DeliveryService
+		queryArgs[j+1] = record.DataEncrypted
+		queryArgs[j+2] = record.CDN
+		queryArgs[j+3] = record.Version
+	}
+	return insertIntoTable(db, queryFmt, 4, queryArgs)
+}
+func (tbl *PGSSLKeyTable) gatherKeys(db *sql.DB) error {
+	sz, err := getSize(db, "sslkey")
+	if err != nil {
+		return fmt.Errorf("PGSSLKey gatherKeys unable to determine size of sslkey table: %v", err)
+	}
+	tbl.Records = make([]PGSSLKeyRecord, sz)
+
+	rows, err := db.Query("SELECT data, deliveryservice, cdn, version from sslkey")
+	if err != nil {
+		return fmt.Errorf("PGSSLKey gatherKeys unable to query: %v", err)

Review comment:
       same as above RE: `%w` vs `%v`

##########
File path: tools/traffic_vault_migrate/postgres.go
##########
@@ -0,0 +1,716 @@
+package main
+
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+import (
+	"database/sql"
+	"encoding/base64"
+	"encoding/json"
+	"errors"
+	"fmt"
+	"github.com/apache/trafficcontrol/lib/go-tc"
+	util "github.com/apache/trafficcontrol/lib/go-util"
+	_ "github.com/lib/pq"
+	"log"
+	"reflect"
+	"strconv"
+	"strings"
+)
+
+type PGConfig struct {
+	Host     string `json:"host"`
+	Port     string `json:"port"`
+	User     string `json:"user"`
+	Password string `json:"password"`
+	SSLMode  string `json:"sslmode"`
+	Database string `json:"database"`
+	Key      string `json:"aesKey"`
+	AESKey   []byte
+}
+type PGBackend struct {
+	sslKey PGSSLKeyTable
+	dnssec PGDNSSecTable
+	uri    PGURISignKeyTable
+	url    PGURLSigKeyTable
+	cfg    PGConfig
+	db     *sql.DB
+}
+
+func (pg *PGBackend) String() string {
+	data := fmt.Sprintf("PG server %v@%v:%v\n", pg.cfg.User, pg.cfg.Host, pg.cfg.Port)
+	data += fmt.Sprintf("\tSSL Keys: %v\n", len(pg.sslKey.Records))
+	data += fmt.Sprintf("\tDNSSec Keys: %v\n", len(pg.dnssec.Records))
+	data += fmt.Sprintf("\tURI Keys: %v\n", len(pg.uri.Records))
+	data += fmt.Sprintf("\tURL Keys: %v\n", len(pg.url.Records))
+	return data
+}
+func (pg *PGBackend) Name() string {
+	return "PG"
+}
+func (pg *PGBackend) ReadConfig(s string) error {
+	genericCfg, err := UnmarshalConfig(s, reflect.TypeOf(pg.cfg))
+	if err != nil {
+		return err
+	}
+
+	pg.cfg = *genericCfg.Interface().(*PGConfig)
+
+	pg.cfg.AESKey, err = base64.StdEncoding.DecodeString(pg.cfg.Key)
+	if err != nil {
+		return fmt.Errorf("unable to decode PG AESKey: %v", err)
+	}
+	return nil
+}
+func (pg *PGBackend) Insert() error {
+	if err := pg.sslKey.insertKeys(pg.db); err != nil {
+		return err
+	}
+	if err := pg.dnssec.insertKeys(pg.db); err != nil {
+		return err
+	}
+	if err := pg.url.insertKeys(pg.db); err != nil {
+		return err
+	}
+	if err := pg.uri.insertKeys(pg.db); err != nil {
+		return err
+	}
+	return nil
+}
+func (pg *PGBackend) Start() error {
+	sqlStr := fmt.Sprintf("postgres://%s:%s@%s:%s/%s?sslmode=%s", pg.cfg.User, pg.cfg.Password, pg.cfg.Host, pg.cfg.Port, pg.cfg.Database, pg.cfg.SSLMode)
+	db, err := sql.Open("postgres", sqlStr)
+	if err != nil {
+		return fmt.Errorf("Unable to start PG client: %v", err)
+	}
+
+	pg.db = db
+	pg.sslKey = PGSSLKeyTable{}
+	pg.dnssec = PGDNSSecTable{}
+	pg.url = PGURLSigKeyTable{}
+	pg.uri = PGURISignKeyTable{}
+
+	return nil
+}
+func (pg *PGBackend) ValidateKey() []string {
+	var errors []string
+	if errs := pg.sslKey.validate(); errs != nil {
+		errors = append(errors, errs...)
+	}
+	if errs := pg.dnssec.validate(); errs != nil {
+		errors = append(errors, errs...)
+	}
+	if errs := pg.uri.validate(); errs != nil {
+		errors = append(errors, errs...)
+	}
+	if errs := pg.url.validate(); errs != nil {
+		errors = append(errors, errs...)
+	}
+	return errors
+}
+func (pg *PGBackend) Stop() error {
+	return pg.db.Close()
+}
+func (pg *PGBackend) Ping() error {
+	return pg.db.Ping()
+}
+func (pg *PGBackend) Fetch() error {
+	if err := pg.sslKey.gatherKeys(pg.db); err != nil {
+		return err
+	}
+
+	if err := pg.dnssec.gatherKeys(pg.db); err != nil {
+		return err
+	}
+
+	if err := pg.url.gatherKeys(pg.db); err != nil {
+		return err
+	}
+
+	if err := pg.uri.gatherKeys(pg.db); err != nil {
+		return err
+	}
+
+	return nil
+}
+func (pg *PGBackend) GetSSLKeys() ([]SSLKey, error) {
+	if err := pg.sslKey.decrypt(pg.cfg.AESKey); err != nil {
+		return nil, err
+	}
+	return pg.sslKey.toGeneric(), nil
+}
+func (pg *PGBackend) SetSSLKeys(keys []SSLKey) error {
+	pg.sslKey.fromGeneric(keys)
+	return pg.sslKey.encrypt(pg.cfg.AESKey)
+}
+func (pg *PGBackend) GetDNSSecKeys() ([]DNSSecKey, error) {
+	if err := pg.dnssec.decrypt(pg.cfg.AESKey); err != nil {
+		return nil, err
+	}
+	return pg.dnssec.toGeneric(), nil
+}
+func (pg *PGBackend) SetDNSSecKeys(keys []DNSSecKey) error {
+	pg.dnssec.fromGeneric(keys)
+	return pg.dnssec.encrypt(pg.cfg.AESKey)
+}
+func (pg *PGBackend) GetURISignKeys() ([]URISignKey, error) {
+	if err := pg.uri.decrypt(pg.cfg.AESKey); err != nil {
+		return nil, err
+	}
+	return pg.uri.toGeneric(), nil
+}
+func (pg *PGBackend) SetURISignKeys(keys []URISignKey) error {
+	pg.uri.fromGeneric(keys)
+	return pg.uri.encrypt(pg.cfg.AESKey)
+}
+func (pg *PGBackend) GetURLSigKeys() ([]URLSigKey, error) {
+	if err := pg.url.decrypt(pg.cfg.AESKey); err != nil {
+		return nil, err
+	}
+	return pg.url.toGeneric(), nil
+}
+func (pg *PGBackend) SetURLSigKeys(keys []URLSigKey) error {
+	pg.url.fromGeneric(keys)
+	return pg.url.encrypt(pg.cfg.AESKey)
+}
+
+type PGCommonRecord struct {
+	DataEncrypted []byte
+	CommonRecord
+}
+
+type PGDNSSecRecord struct {
+	Key tc.DNSSECKeysTrafficVault
+	CDN string
+	PGCommonRecord
+}
+type PGDNSSecTable struct {
+	Records []PGDNSSecRecord
+}
+
+func (tbl *PGDNSSecTable) gatherKeys(db *sql.DB) error {
+	sz, err := getSize(db, "dnssec")
+	if err != nil {
+		log.Println("PGDNSSec gatherKeys: unable to determine size of dnssec table")
+	}
+	tbl.Records = make([]PGDNSSecRecord, sz)
+
+	rows, err := db.Query("SELECT cdn, data from dnssec")
+	if err != nil {
+		return fmt.Errorf("PGDNSSec gatherKeys: unable to query: %v", err)
+	}
+	defer rows.Close()
+	i := 0
+	for rows.Next() {
+		if i > len(tbl.Records)-1 {
+			return fmt.Errorf("PGDNSSec gatherKeys got more results than expected %v", len(tbl.Records))
+		}
+		err := rows.Scan(&tbl.Records[i].CDN, &tbl.Records[i].DataEncrypted)
+		if err != nil {
+			return fmt.Errorf("PGDNSSec gatherKeys unable to scan row: %v", err)
+		}
+		i += 1
+	}
+	return nil
+}
+func (tbl *PGDNSSecTable) decrypt(aesKey []byte) error {
+	for i, _ := range tbl.Records {
+		data, err := decryptInto(aesKey, tbl.Records[i].DataEncrypted, reflect.TypeOf(tbl.Records[i].Key))
+		if err != nil {
+			return fmt.Errorf("unable to decrypt into keys: %v", err)
+		}
+		tbl.Records[i].Key = *data.Interface().(*tc.DNSSECKeysTrafficVault)
+	}
+	return nil
+}
+func (tbl *PGDNSSecTable) encrypt(aesKey []byte) error {
+	for i, dns := range tbl.Records {
+		data, err := json.Marshal(&dns.Key)
+		if err != nil {
+			return fmt.Errorf("encrypt issue marshalling keys: %v", err)
+		}
+		dat, err := encrypt(data, aesKey)
+		if err != nil {
+			return fmt.Errorf("encrypt error: %v", err)
+		}
+		tbl.Records[i].DataEncrypted = dat
+	}
+	return nil
+}
+func (tbl *PGDNSSecTable) toGeneric() []DNSSecKey {
+	keys := make([]DNSSecKey, len(tbl.Records))
+
+	for i, record := range tbl.Records {
+		keys[i] = DNSSecKey{
+			CDN:                    record.CDN,
+			DNSSECKeysTrafficVault: record.Key,
+			CommonRecord:           record.CommonRecord,
+		}
+	}
+
+	return keys
+}
+func (tbl *PGDNSSecTable) fromGeneric(keys []DNSSecKey) {
+	tbl.Records = make([]PGDNSSecRecord, len(keys))
+
+	for i, key := range keys {
+		tbl.Records[i] = PGDNSSecRecord{
+			Key: key.DNSSECKeysTrafficVault,
+			CDN: key.CDN,
+			PGCommonRecord: PGCommonRecord{
+				DataEncrypted: nil,
+				CommonRecord:  CommonRecord{},
+			},
+		}
+	}
+}
+func (tbl *PGDNSSecTable) validate() []string {
+	for i, record := range tbl.Records {
+		if record.DataEncrypted == nil && len(record.Key) > 0 {
+			return []string{fmt.Sprintf("DNSSEC Key %v: DataEncrypted is blank!", i)}
+		}
+	}
+	return nil
+}
+func (tbl *PGDNSSecTable) insertKeys(db *sql.DB) error {
+	queryFmt := "INSERT INTO dnssec (cdn, data) VALUES "
+	stride := 2
+	queryArgs := make([]interface{}, len(tbl.Records)*stride)
+	for i, record := range tbl.Records {
+		j := i * stride
+		queryArgs[j] = record.CDN
+		queryArgs[j+1] = record.DataEncrypted
+	}
+	return insertIntoTable(db, queryFmt, stride, queryArgs)
+}
+
+type PGSSLKeyRecord struct {
+	Keys tc.DeliveryServiceSSLKeys
+	PGCommonRecord
+
+	// These records are stored on the table but are duplicated
+	DeliveryService string
+	Version         string
+	CDN             string
+}
+type PGSSLKeyTable struct {
+	Records []PGSSLKeyRecord
+}
+
+func (tbl *PGSSLKeyTable) insertKeys(db *sql.DB) error {
+	queryFmt := "INSERT INTO sslkey (deliveryservice, data, cdn, version) VALUES "
+	stride := 4
+	queryArgs := make([]interface{}, len(tbl.Records)*stride)
+	for i, record := range tbl.Records {
+		j := i * stride
+		queryArgs[j] = record.DeliveryService
+		queryArgs[j+1] = record.DataEncrypted
+		queryArgs[j+2] = record.CDN
+		queryArgs[j+3] = record.Version
+	}
+	return insertIntoTable(db, queryFmt, 4, queryArgs)
+}
+func (tbl *PGSSLKeyTable) gatherKeys(db *sql.DB) error {
+	sz, err := getSize(db, "sslkey")
+	if err != nil {
+		return fmt.Errorf("PGSSLKey gatherKeys unable to determine size of sslkey table: %v", err)
+	}
+	tbl.Records = make([]PGSSLKeyRecord, sz)
+
+	rows, err := db.Query("SELECT data, deliveryservice, cdn, version from sslkey")
+	if err != nil {
+		return fmt.Errorf("PGSSLKey gatherKeys unable to query: %v", err)
+	}
+	defer rows.Close()
+	i := 0
+	for rows.Next() {
+		if i > len(tbl.Records)-1 {
+			return errors.New("PGSSLKey gatherKeys: got more results than expected")
+		}
+		err := rows.Scan(&tbl.Records[i].DataEncrypted, &tbl.Records[i].DeliveryService, &tbl.Records[i].CDN, &tbl.Records[i].Version)
+		if err != nil {
+			return fmt.Errorf("PGSSLKey gatherKeys unable to scan row: %v", err)
+		}
+		i += 1
+	}
+	return nil
+}
+func (tbl *PGSSLKeyTable) decrypt(aesKey []byte) error {
+	for i, dns := range tbl.Records {
+		data, err := decryptInto(aesKey, dns.DataEncrypted, reflect.TypeOf(dns.Keys))
+		if err != nil {
+			return fmt.Errorf("unable to decrypt into keys: %v", err)
+		}
+		tbl.Records[i].Keys = *data.Interface().(*tc.DeliveryServiceSSLKeys)
+	}
+	return nil
+}
+func (tbl *PGSSLKeyTable) encrypt(aesKey []byte) error {
+	for i, dns := range tbl.Records {
+		data, err := json.Marshal(dns.Keys)
+		if err != nil {
+			return fmt.Errorf("encrypt issue marshalling keys: %v", err)

Review comment:
       same as above RE: `%w` vs `%v`

##########
File path: tools/traffic_vault_migrate/postgres.go
##########
@@ -0,0 +1,716 @@
+package main
+
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+import (
+	"database/sql"
+	"encoding/base64"
+	"encoding/json"
+	"errors"
+	"fmt"
+	"github.com/apache/trafficcontrol/lib/go-tc"
+	util "github.com/apache/trafficcontrol/lib/go-util"
+	_ "github.com/lib/pq"
+	"log"
+	"reflect"
+	"strconv"
+	"strings"
+)
+
+type PGConfig struct {
+	Host     string `json:"host"`
+	Port     string `json:"port"`
+	User     string `json:"user"`
+	Password string `json:"password"`
+	SSLMode  string `json:"sslmode"`
+	Database string `json:"database"`
+	Key      string `json:"aesKey"`
+	AESKey   []byte
+}
+type PGBackend struct {
+	sslKey PGSSLKeyTable
+	dnssec PGDNSSecTable
+	uri    PGURISignKeyTable
+	url    PGURLSigKeyTable
+	cfg    PGConfig
+	db     *sql.DB
+}
+
+func (pg *PGBackend) String() string {
+	data := fmt.Sprintf("PG server %v@%v:%v\n", pg.cfg.User, pg.cfg.Host, pg.cfg.Port)
+	data += fmt.Sprintf("\tSSL Keys: %v\n", len(pg.sslKey.Records))
+	data += fmt.Sprintf("\tDNSSec Keys: %v\n", len(pg.dnssec.Records))
+	data += fmt.Sprintf("\tURI Keys: %v\n", len(pg.uri.Records))
+	data += fmt.Sprintf("\tURL Keys: %v\n", len(pg.url.Records))
+	return data
+}
+func (pg *PGBackend) Name() string {
+	return "PG"
+}
+func (pg *PGBackend) ReadConfig(s string) error {
+	genericCfg, err := UnmarshalConfig(s, reflect.TypeOf(pg.cfg))
+	if err != nil {
+		return err
+	}
+
+	pg.cfg = *genericCfg.Interface().(*PGConfig)
+
+	pg.cfg.AESKey, err = base64.StdEncoding.DecodeString(pg.cfg.Key)
+	if err != nil {
+		return fmt.Errorf("unable to decode PG AESKey: %v", err)
+	}
+	return nil
+}
+func (pg *PGBackend) Insert() error {
+	if err := pg.sslKey.insertKeys(pg.db); err != nil {
+		return err
+	}
+	if err := pg.dnssec.insertKeys(pg.db); err != nil {
+		return err
+	}
+	if err := pg.url.insertKeys(pg.db); err != nil {
+		return err
+	}
+	if err := pg.uri.insertKeys(pg.db); err != nil {
+		return err
+	}
+	return nil
+}
+func (pg *PGBackend) Start() error {
+	sqlStr := fmt.Sprintf("postgres://%s:%s@%s:%s/%s?sslmode=%s", pg.cfg.User, pg.cfg.Password, pg.cfg.Host, pg.cfg.Port, pg.cfg.Database, pg.cfg.SSLMode)
+	db, err := sql.Open("postgres", sqlStr)
+	if err != nil {
+		return fmt.Errorf("Unable to start PG client: %v", err)
+	}
+
+	pg.db = db
+	pg.sslKey = PGSSLKeyTable{}
+	pg.dnssec = PGDNSSecTable{}
+	pg.url = PGURLSigKeyTable{}
+	pg.uri = PGURISignKeyTable{}
+
+	return nil
+}
+func (pg *PGBackend) ValidateKey() []string {
+	var errors []string
+	if errs := pg.sslKey.validate(); errs != nil {
+		errors = append(errors, errs...)
+	}
+	if errs := pg.dnssec.validate(); errs != nil {
+		errors = append(errors, errs...)
+	}
+	if errs := pg.uri.validate(); errs != nil {
+		errors = append(errors, errs...)
+	}
+	if errs := pg.url.validate(); errs != nil {
+		errors = append(errors, errs...)
+	}
+	return errors
+}
+func (pg *PGBackend) Stop() error {
+	return pg.db.Close()
+}
+func (pg *PGBackend) Ping() error {
+	return pg.db.Ping()
+}
+func (pg *PGBackend) Fetch() error {
+	if err := pg.sslKey.gatherKeys(pg.db); err != nil {
+		return err
+	}
+
+	if err := pg.dnssec.gatherKeys(pg.db); err != nil {
+		return err
+	}
+
+	if err := pg.url.gatherKeys(pg.db); err != nil {
+		return err
+	}
+
+	if err := pg.uri.gatherKeys(pg.db); err != nil {
+		return err
+	}
+
+	return nil
+}
+func (pg *PGBackend) GetSSLKeys() ([]SSLKey, error) {
+	if err := pg.sslKey.decrypt(pg.cfg.AESKey); err != nil {
+		return nil, err
+	}
+	return pg.sslKey.toGeneric(), nil
+}
+func (pg *PGBackend) SetSSLKeys(keys []SSLKey) error {
+	pg.sslKey.fromGeneric(keys)
+	return pg.sslKey.encrypt(pg.cfg.AESKey)
+}
+func (pg *PGBackend) GetDNSSecKeys() ([]DNSSecKey, error) {
+	if err := pg.dnssec.decrypt(pg.cfg.AESKey); err != nil {
+		return nil, err
+	}
+	return pg.dnssec.toGeneric(), nil
+}
+func (pg *PGBackend) SetDNSSecKeys(keys []DNSSecKey) error {
+	pg.dnssec.fromGeneric(keys)
+	return pg.dnssec.encrypt(pg.cfg.AESKey)
+}
+func (pg *PGBackend) GetURISignKeys() ([]URISignKey, error) {
+	if err := pg.uri.decrypt(pg.cfg.AESKey); err != nil {
+		return nil, err
+	}
+	return pg.uri.toGeneric(), nil
+}
+func (pg *PGBackend) SetURISignKeys(keys []URISignKey) error {
+	pg.uri.fromGeneric(keys)
+	return pg.uri.encrypt(pg.cfg.AESKey)
+}
+func (pg *PGBackend) GetURLSigKeys() ([]URLSigKey, error) {
+	if err := pg.url.decrypt(pg.cfg.AESKey); err != nil {
+		return nil, err
+	}
+	return pg.url.toGeneric(), nil
+}
+func (pg *PGBackend) SetURLSigKeys(keys []URLSigKey) error {
+	pg.url.fromGeneric(keys)
+	return pg.url.encrypt(pg.cfg.AESKey)
+}
+
+type PGCommonRecord struct {
+	DataEncrypted []byte
+	CommonRecord
+}
+
+type PGDNSSecRecord struct {
+	Key tc.DNSSECKeysTrafficVault
+	CDN string
+	PGCommonRecord
+}
+type PGDNSSecTable struct {
+	Records []PGDNSSecRecord
+}
+
+func (tbl *PGDNSSecTable) gatherKeys(db *sql.DB) error {
+	sz, err := getSize(db, "dnssec")
+	if err != nil {
+		log.Println("PGDNSSec gatherKeys: unable to determine size of dnssec table")
+	}
+	tbl.Records = make([]PGDNSSecRecord, sz)
+
+	rows, err := db.Query("SELECT cdn, data from dnssec")
+	if err != nil {
+		return fmt.Errorf("PGDNSSec gatherKeys: unable to query: %v", err)
+	}
+	defer rows.Close()
+	i := 0
+	for rows.Next() {
+		if i > len(tbl.Records)-1 {
+			return fmt.Errorf("PGDNSSec gatherKeys got more results than expected %v", len(tbl.Records))
+		}
+		err := rows.Scan(&tbl.Records[i].CDN, &tbl.Records[i].DataEncrypted)
+		if err != nil {
+			return fmt.Errorf("PGDNSSec gatherKeys unable to scan row: %v", err)
+		}
+		i += 1
+	}
+	return nil
+}
+func (tbl *PGDNSSecTable) decrypt(aesKey []byte) error {
+	for i, _ := range tbl.Records {
+		data, err := decryptInto(aesKey, tbl.Records[i].DataEncrypted, reflect.TypeOf(tbl.Records[i].Key))
+		if err != nil {
+			return fmt.Errorf("unable to decrypt into keys: %v", err)
+		}
+		tbl.Records[i].Key = *data.Interface().(*tc.DNSSECKeysTrafficVault)
+	}
+	return nil
+}
+func (tbl *PGDNSSecTable) encrypt(aesKey []byte) error {
+	for i, dns := range tbl.Records {
+		data, err := json.Marshal(&dns.Key)
+		if err != nil {
+			return fmt.Errorf("encrypt issue marshalling keys: %v", err)
+		}
+		dat, err := encrypt(data, aesKey)
+		if err != nil {
+			return fmt.Errorf("encrypt error: %v", err)
+		}
+		tbl.Records[i].DataEncrypted = dat
+	}
+	return nil
+}
+func (tbl *PGDNSSecTable) toGeneric() []DNSSecKey {
+	keys := make([]DNSSecKey, len(tbl.Records))
+
+	for i, record := range tbl.Records {
+		keys[i] = DNSSecKey{
+			CDN:                    record.CDN,
+			DNSSECKeysTrafficVault: record.Key,
+			CommonRecord:           record.CommonRecord,
+		}
+	}
+
+	return keys
+}
+func (tbl *PGDNSSecTable) fromGeneric(keys []DNSSecKey) {
+	tbl.Records = make([]PGDNSSecRecord, len(keys))
+
+	for i, key := range keys {
+		tbl.Records[i] = PGDNSSecRecord{
+			Key: key.DNSSECKeysTrafficVault,
+			CDN: key.CDN,
+			PGCommonRecord: PGCommonRecord{
+				DataEncrypted: nil,
+				CommonRecord:  CommonRecord{},
+			},
+		}
+	}
+}
+func (tbl *PGDNSSecTable) validate() []string {
+	for i, record := range tbl.Records {
+		if record.DataEncrypted == nil && len(record.Key) > 0 {
+			return []string{fmt.Sprintf("DNSSEC Key %v: DataEncrypted is blank!", i)}
+		}
+	}
+	return nil
+}
+func (tbl *PGDNSSecTable) insertKeys(db *sql.DB) error {
+	queryFmt := "INSERT INTO dnssec (cdn, data) VALUES "
+	stride := 2
+	queryArgs := make([]interface{}, len(tbl.Records)*stride)
+	for i, record := range tbl.Records {
+		j := i * stride
+		queryArgs[j] = record.CDN
+		queryArgs[j+1] = record.DataEncrypted
+	}
+	return insertIntoTable(db, queryFmt, stride, queryArgs)
+}
+
+type PGSSLKeyRecord struct {
+	Keys tc.DeliveryServiceSSLKeys
+	PGCommonRecord
+
+	// These records are stored on the table but are duplicated
+	DeliveryService string
+	Version         string
+	CDN             string
+}
+type PGSSLKeyTable struct {
+	Records []PGSSLKeyRecord
+}
+
+func (tbl *PGSSLKeyTable) insertKeys(db *sql.DB) error {
+	queryFmt := "INSERT INTO sslkey (deliveryservice, data, cdn, version) VALUES "
+	stride := 4
+	queryArgs := make([]interface{}, len(tbl.Records)*stride)
+	for i, record := range tbl.Records {
+		j := i * stride
+		queryArgs[j] = record.DeliveryService
+		queryArgs[j+1] = record.DataEncrypted
+		queryArgs[j+2] = record.CDN
+		queryArgs[j+3] = record.Version
+	}
+	return insertIntoTable(db, queryFmt, 4, queryArgs)
+}
+func (tbl *PGSSLKeyTable) gatherKeys(db *sql.DB) error {
+	sz, err := getSize(db, "sslkey")
+	if err != nil {
+		return fmt.Errorf("PGSSLKey gatherKeys unable to determine size of sslkey table: %v", err)

Review comment:
       same as above RE: `%w` vs `%v`

##########
File path: tools/traffic_vault_migrate/postgres.go
##########
@@ -0,0 +1,716 @@
+package main
+
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+import (
+	"database/sql"
+	"encoding/base64"
+	"encoding/json"
+	"errors"
+	"fmt"
+	"github.com/apache/trafficcontrol/lib/go-tc"
+	util "github.com/apache/trafficcontrol/lib/go-util"
+	_ "github.com/lib/pq"
+	"log"
+	"reflect"
+	"strconv"
+	"strings"
+)
+
+type PGConfig struct {
+	Host     string `json:"host"`
+	Port     string `json:"port"`
+	User     string `json:"user"`
+	Password string `json:"password"`
+	SSLMode  string `json:"sslmode"`
+	Database string `json:"database"`
+	Key      string `json:"aesKey"`
+	AESKey   []byte
+}
+type PGBackend struct {
+	sslKey PGSSLKeyTable
+	dnssec PGDNSSecTable
+	uri    PGURISignKeyTable
+	url    PGURLSigKeyTable
+	cfg    PGConfig
+	db     *sql.DB
+}
+
+func (pg *PGBackend) String() string {
+	data := fmt.Sprintf("PG server %v@%v:%v\n", pg.cfg.User, pg.cfg.Host, pg.cfg.Port)
+	data += fmt.Sprintf("\tSSL Keys: %v\n", len(pg.sslKey.Records))
+	data += fmt.Sprintf("\tDNSSec Keys: %v\n", len(pg.dnssec.Records))
+	data += fmt.Sprintf("\tURI Keys: %v\n", len(pg.uri.Records))
+	data += fmt.Sprintf("\tURL Keys: %v\n", len(pg.url.Records))
+	return data
+}
+func (pg *PGBackend) Name() string {
+	return "PG"
+}
+func (pg *PGBackend) ReadConfig(s string) error {
+	genericCfg, err := UnmarshalConfig(s, reflect.TypeOf(pg.cfg))
+	if err != nil {
+		return err
+	}
+
+	pg.cfg = *genericCfg.Interface().(*PGConfig)
+
+	pg.cfg.AESKey, err = base64.StdEncoding.DecodeString(pg.cfg.Key)
+	if err != nil {
+		return fmt.Errorf("unable to decode PG AESKey: %v", err)
+	}
+	return nil
+}
+func (pg *PGBackend) Insert() error {
+	if err := pg.sslKey.insertKeys(pg.db); err != nil {
+		return err
+	}
+	if err := pg.dnssec.insertKeys(pg.db); err != nil {
+		return err
+	}
+	if err := pg.url.insertKeys(pg.db); err != nil {
+		return err
+	}
+	if err := pg.uri.insertKeys(pg.db); err != nil {
+		return err
+	}
+	return nil
+}
+func (pg *PGBackend) Start() error {
+	sqlStr := fmt.Sprintf("postgres://%s:%s@%s:%s/%s?sslmode=%s", pg.cfg.User, pg.cfg.Password, pg.cfg.Host, pg.cfg.Port, pg.cfg.Database, pg.cfg.SSLMode)
+	db, err := sql.Open("postgres", sqlStr)
+	if err != nil {
+		return fmt.Errorf("Unable to start PG client: %v", err)
+	}
+
+	pg.db = db
+	pg.sslKey = PGSSLKeyTable{}
+	pg.dnssec = PGDNSSecTable{}
+	pg.url = PGURLSigKeyTable{}
+	pg.uri = PGURISignKeyTable{}
+
+	return nil
+}
+func (pg *PGBackend) ValidateKey() []string {
+	var errors []string
+	if errs := pg.sslKey.validate(); errs != nil {
+		errors = append(errors, errs...)
+	}
+	if errs := pg.dnssec.validate(); errs != nil {
+		errors = append(errors, errs...)
+	}
+	if errs := pg.uri.validate(); errs != nil {
+		errors = append(errors, errs...)
+	}
+	if errs := pg.url.validate(); errs != nil {
+		errors = append(errors, errs...)
+	}
+	return errors
+}
+func (pg *PGBackend) Stop() error {
+	return pg.db.Close()
+}
+func (pg *PGBackend) Ping() error {
+	return pg.db.Ping()
+}
+func (pg *PGBackend) Fetch() error {
+	if err := pg.sslKey.gatherKeys(pg.db); err != nil {
+		return err
+	}
+
+	if err := pg.dnssec.gatherKeys(pg.db); err != nil {
+		return err
+	}
+
+	if err := pg.url.gatherKeys(pg.db); err != nil {
+		return err
+	}
+
+	if err := pg.uri.gatherKeys(pg.db); err != nil {
+		return err
+	}
+
+	return nil
+}
+func (pg *PGBackend) GetSSLKeys() ([]SSLKey, error) {
+	if err := pg.sslKey.decrypt(pg.cfg.AESKey); err != nil {
+		return nil, err
+	}
+	return pg.sslKey.toGeneric(), nil
+}
+func (pg *PGBackend) SetSSLKeys(keys []SSLKey) error {
+	pg.sslKey.fromGeneric(keys)
+	return pg.sslKey.encrypt(pg.cfg.AESKey)
+}
+func (pg *PGBackend) GetDNSSecKeys() ([]DNSSecKey, error) {
+	if err := pg.dnssec.decrypt(pg.cfg.AESKey); err != nil {
+		return nil, err
+	}
+	return pg.dnssec.toGeneric(), nil
+}
+func (pg *PGBackend) SetDNSSecKeys(keys []DNSSecKey) error {
+	pg.dnssec.fromGeneric(keys)
+	return pg.dnssec.encrypt(pg.cfg.AESKey)
+}
+func (pg *PGBackend) GetURISignKeys() ([]URISignKey, error) {
+	if err := pg.uri.decrypt(pg.cfg.AESKey); err != nil {
+		return nil, err
+	}
+	return pg.uri.toGeneric(), nil
+}
+func (pg *PGBackend) SetURISignKeys(keys []URISignKey) error {
+	pg.uri.fromGeneric(keys)
+	return pg.uri.encrypt(pg.cfg.AESKey)
+}
+func (pg *PGBackend) GetURLSigKeys() ([]URLSigKey, error) {
+	if err := pg.url.decrypt(pg.cfg.AESKey); err != nil {
+		return nil, err
+	}
+	return pg.url.toGeneric(), nil
+}
+func (pg *PGBackend) SetURLSigKeys(keys []URLSigKey) error {
+	pg.url.fromGeneric(keys)
+	return pg.url.encrypt(pg.cfg.AESKey)
+}
+
+type PGCommonRecord struct {
+	DataEncrypted []byte
+	CommonRecord
+}
+
+type PGDNSSecRecord struct {
+	Key tc.DNSSECKeysTrafficVault
+	CDN string
+	PGCommonRecord
+}
+type PGDNSSecTable struct {
+	Records []PGDNSSecRecord
+}
+
+func (tbl *PGDNSSecTable) gatherKeys(db *sql.DB) error {
+	sz, err := getSize(db, "dnssec")
+	if err != nil {
+		log.Println("PGDNSSec gatherKeys: unable to determine size of dnssec table")
+	}
+	tbl.Records = make([]PGDNSSecRecord, sz)
+
+	rows, err := db.Query("SELECT cdn, data from dnssec")
+	if err != nil {
+		return fmt.Errorf("PGDNSSec gatherKeys: unable to query: %v", err)
+	}
+	defer rows.Close()
+	i := 0
+	for rows.Next() {
+		if i > len(tbl.Records)-1 {
+			return fmt.Errorf("PGDNSSec gatherKeys got more results than expected %v", len(tbl.Records))
+		}
+		err := rows.Scan(&tbl.Records[i].CDN, &tbl.Records[i].DataEncrypted)
+		if err != nil {
+			return fmt.Errorf("PGDNSSec gatherKeys unable to scan row: %v", err)
+		}
+		i += 1
+	}
+	return nil
+}
+func (tbl *PGDNSSecTable) decrypt(aesKey []byte) error {
+	for i, _ := range tbl.Records {
+		data, err := decryptInto(aesKey, tbl.Records[i].DataEncrypted, reflect.TypeOf(tbl.Records[i].Key))
+		if err != nil {
+			return fmt.Errorf("unable to decrypt into keys: %v", err)
+		}
+		tbl.Records[i].Key = *data.Interface().(*tc.DNSSECKeysTrafficVault)
+	}
+	return nil
+}
+func (tbl *PGDNSSecTable) encrypt(aesKey []byte) error {
+	for i, dns := range tbl.Records {
+		data, err := json.Marshal(&dns.Key)
+		if err != nil {
+			return fmt.Errorf("encrypt issue marshalling keys: %v", err)
+		}
+		dat, err := encrypt(data, aesKey)
+		if err != nil {
+			return fmt.Errorf("encrypt error: %v", err)
+		}
+		tbl.Records[i].DataEncrypted = dat
+	}
+	return nil
+}
+func (tbl *PGDNSSecTable) toGeneric() []DNSSecKey {
+	keys := make([]DNSSecKey, len(tbl.Records))
+
+	for i, record := range tbl.Records {
+		keys[i] = DNSSecKey{
+			CDN:                    record.CDN,
+			DNSSECKeysTrafficVault: record.Key,
+			CommonRecord:           record.CommonRecord,
+		}
+	}
+
+	return keys
+}
+func (tbl *PGDNSSecTable) fromGeneric(keys []DNSSecKey) {
+	tbl.Records = make([]PGDNSSecRecord, len(keys))
+
+	for i, key := range keys {
+		tbl.Records[i] = PGDNSSecRecord{
+			Key: key.DNSSECKeysTrafficVault,
+			CDN: key.CDN,
+			PGCommonRecord: PGCommonRecord{
+				DataEncrypted: nil,
+				CommonRecord:  CommonRecord{},
+			},
+		}
+	}
+}
+func (tbl *PGDNSSecTable) validate() []string {
+	for i, record := range tbl.Records {
+		if record.DataEncrypted == nil && len(record.Key) > 0 {
+			return []string{fmt.Sprintf("DNSSEC Key %v: DataEncrypted is blank!", i)}
+		}
+	}
+	return nil
+}
+func (tbl *PGDNSSecTable) insertKeys(db *sql.DB) error {
+	queryFmt := "INSERT INTO dnssec (cdn, data) VALUES "
+	stride := 2
+	queryArgs := make([]interface{}, len(tbl.Records)*stride)
+	for i, record := range tbl.Records {
+		j := i * stride
+		queryArgs[j] = record.CDN
+		queryArgs[j+1] = record.DataEncrypted
+	}
+	return insertIntoTable(db, queryFmt, stride, queryArgs)
+}
+
+type PGSSLKeyRecord struct {
+	Keys tc.DeliveryServiceSSLKeys
+	PGCommonRecord
+
+	// These records are stored on the table but are duplicated
+	DeliveryService string
+	Version         string
+	CDN             string
+}
+type PGSSLKeyTable struct {
+	Records []PGSSLKeyRecord
+}
+
+func (tbl *PGSSLKeyTable) insertKeys(db *sql.DB) error {
+	queryFmt := "INSERT INTO sslkey (deliveryservice, data, cdn, version) VALUES "
+	stride := 4
+	queryArgs := make([]interface{}, len(tbl.Records)*stride)
+	for i, record := range tbl.Records {
+		j := i * stride
+		queryArgs[j] = record.DeliveryService
+		queryArgs[j+1] = record.DataEncrypted
+		queryArgs[j+2] = record.CDN
+		queryArgs[j+3] = record.Version
+	}
+	return insertIntoTable(db, queryFmt, 4, queryArgs)
+}
+func (tbl *PGSSLKeyTable) gatherKeys(db *sql.DB) error {
+	sz, err := getSize(db, "sslkey")
+	if err != nil {
+		return fmt.Errorf("PGSSLKey gatherKeys unable to determine size of sslkey table: %v", err)
+	}
+	tbl.Records = make([]PGSSLKeyRecord, sz)
+
+	rows, err := db.Query("SELECT data, deliveryservice, cdn, version from sslkey")
+	if err != nil {
+		return fmt.Errorf("PGSSLKey gatherKeys unable to query: %v", err)
+	}
+	defer rows.Close()
+	i := 0
+	for rows.Next() {
+		if i > len(tbl.Records)-1 {
+			return errors.New("PGSSLKey gatherKeys: got more results than expected")
+		}
+		err := rows.Scan(&tbl.Records[i].DataEncrypted, &tbl.Records[i].DeliveryService, &tbl.Records[i].CDN, &tbl.Records[i].Version)
+		if err != nil {
+			return fmt.Errorf("PGSSLKey gatherKeys unable to scan row: %v", err)
+		}
+		i += 1
+	}
+	return nil
+}
+func (tbl *PGSSLKeyTable) decrypt(aesKey []byte) error {
+	for i, dns := range tbl.Records {
+		data, err := decryptInto(aesKey, dns.DataEncrypted, reflect.TypeOf(dns.Keys))
+		if err != nil {
+			return fmt.Errorf("unable to decrypt into keys: %v", err)

Review comment:
       same as above RE: `%w` vs `%v`

##########
File path: tools/traffic_vault_migrate/postgres.go
##########
@@ -0,0 +1,716 @@
+package main
+
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+import (
+	"database/sql"
+	"encoding/base64"
+	"encoding/json"
+	"errors"
+	"fmt"
+	"github.com/apache/trafficcontrol/lib/go-tc"
+	util "github.com/apache/trafficcontrol/lib/go-util"
+	_ "github.com/lib/pq"
+	"log"
+	"reflect"
+	"strconv"
+	"strings"
+)
+
+type PGConfig struct {
+	Host     string `json:"host"`
+	Port     string `json:"port"`
+	User     string `json:"user"`
+	Password string `json:"password"`
+	SSLMode  string `json:"sslmode"`
+	Database string `json:"database"`
+	Key      string `json:"aesKey"`
+	AESKey   []byte
+}
+type PGBackend struct {
+	sslKey PGSSLKeyTable
+	dnssec PGDNSSecTable
+	uri    PGURISignKeyTable
+	url    PGURLSigKeyTable
+	cfg    PGConfig
+	db     *sql.DB
+}
+
+func (pg *PGBackend) String() string {
+	data := fmt.Sprintf("PG server %v@%v:%v\n", pg.cfg.User, pg.cfg.Host, pg.cfg.Port)
+	data += fmt.Sprintf("\tSSL Keys: %v\n", len(pg.sslKey.Records))
+	data += fmt.Sprintf("\tDNSSec Keys: %v\n", len(pg.dnssec.Records))
+	data += fmt.Sprintf("\tURI Keys: %v\n", len(pg.uri.Records))
+	data += fmt.Sprintf("\tURL Keys: %v\n", len(pg.url.Records))
+	return data
+}
+func (pg *PGBackend) Name() string {
+	return "PG"
+}
+func (pg *PGBackend) ReadConfig(s string) error {
+	genericCfg, err := UnmarshalConfig(s, reflect.TypeOf(pg.cfg))
+	if err != nil {
+		return err
+	}
+
+	pg.cfg = *genericCfg.Interface().(*PGConfig)
+
+	pg.cfg.AESKey, err = base64.StdEncoding.DecodeString(pg.cfg.Key)
+	if err != nil {
+		return fmt.Errorf("unable to decode PG AESKey: %v", err)
+	}
+	return nil
+}
+func (pg *PGBackend) Insert() error {
+	if err := pg.sslKey.insertKeys(pg.db); err != nil {
+		return err
+	}
+	if err := pg.dnssec.insertKeys(pg.db); err != nil {
+		return err
+	}
+	if err := pg.url.insertKeys(pg.db); err != nil {
+		return err
+	}
+	if err := pg.uri.insertKeys(pg.db); err != nil {
+		return err
+	}
+	return nil
+}
+func (pg *PGBackend) Start() error {
+	sqlStr := fmt.Sprintf("postgres://%s:%s@%s:%s/%s?sslmode=%s", pg.cfg.User, pg.cfg.Password, pg.cfg.Host, pg.cfg.Port, pg.cfg.Database, pg.cfg.SSLMode)
+	db, err := sql.Open("postgres", sqlStr)
+	if err != nil {
+		return fmt.Errorf("Unable to start PG client: %v", err)
+	}
+
+	pg.db = db
+	pg.sslKey = PGSSLKeyTable{}
+	pg.dnssec = PGDNSSecTable{}
+	pg.url = PGURLSigKeyTable{}
+	pg.uri = PGURISignKeyTable{}
+
+	return nil
+}
+func (pg *PGBackend) ValidateKey() []string {
+	var errors []string
+	if errs := pg.sslKey.validate(); errs != nil {
+		errors = append(errors, errs...)
+	}
+	if errs := pg.dnssec.validate(); errs != nil {
+		errors = append(errors, errs...)
+	}
+	if errs := pg.uri.validate(); errs != nil {
+		errors = append(errors, errs...)
+	}
+	if errs := pg.url.validate(); errs != nil {
+		errors = append(errors, errs...)
+	}
+	return errors
+}
+func (pg *PGBackend) Stop() error {
+	return pg.db.Close()
+}
+func (pg *PGBackend) Ping() error {
+	return pg.db.Ping()
+}
+func (pg *PGBackend) Fetch() error {
+	if err := pg.sslKey.gatherKeys(pg.db); err != nil {
+		return err
+	}
+
+	if err := pg.dnssec.gatherKeys(pg.db); err != nil {
+		return err
+	}
+
+	if err := pg.url.gatherKeys(pg.db); err != nil {
+		return err
+	}
+
+	if err := pg.uri.gatherKeys(pg.db); err != nil {
+		return err
+	}
+
+	return nil
+}
+func (pg *PGBackend) GetSSLKeys() ([]SSLKey, error) {
+	if err := pg.sslKey.decrypt(pg.cfg.AESKey); err != nil {
+		return nil, err
+	}
+	return pg.sslKey.toGeneric(), nil
+}
+func (pg *PGBackend) SetSSLKeys(keys []SSLKey) error {
+	pg.sslKey.fromGeneric(keys)
+	return pg.sslKey.encrypt(pg.cfg.AESKey)
+}
+func (pg *PGBackend) GetDNSSecKeys() ([]DNSSecKey, error) {
+	if err := pg.dnssec.decrypt(pg.cfg.AESKey); err != nil {
+		return nil, err
+	}
+	return pg.dnssec.toGeneric(), nil
+}
+func (pg *PGBackend) SetDNSSecKeys(keys []DNSSecKey) error {
+	pg.dnssec.fromGeneric(keys)
+	return pg.dnssec.encrypt(pg.cfg.AESKey)
+}
+func (pg *PGBackend) GetURISignKeys() ([]URISignKey, error) {
+	if err := pg.uri.decrypt(pg.cfg.AESKey); err != nil {
+		return nil, err
+	}
+	return pg.uri.toGeneric(), nil
+}
+func (pg *PGBackend) SetURISignKeys(keys []URISignKey) error {
+	pg.uri.fromGeneric(keys)
+	return pg.uri.encrypt(pg.cfg.AESKey)
+}
+func (pg *PGBackend) GetURLSigKeys() ([]URLSigKey, error) {
+	if err := pg.url.decrypt(pg.cfg.AESKey); err != nil {
+		return nil, err
+	}
+	return pg.url.toGeneric(), nil
+}
+func (pg *PGBackend) SetURLSigKeys(keys []URLSigKey) error {
+	pg.url.fromGeneric(keys)
+	return pg.url.encrypt(pg.cfg.AESKey)
+}
+
+type PGCommonRecord struct {
+	DataEncrypted []byte
+	CommonRecord
+}
+
+type PGDNSSecRecord struct {
+	Key tc.DNSSECKeysTrafficVault
+	CDN string
+	PGCommonRecord
+}
+type PGDNSSecTable struct {
+	Records []PGDNSSecRecord
+}
+
+func (tbl *PGDNSSecTable) gatherKeys(db *sql.DB) error {
+	sz, err := getSize(db, "dnssec")
+	if err != nil {
+		log.Println("PGDNSSec gatherKeys: unable to determine size of dnssec table")
+	}
+	tbl.Records = make([]PGDNSSecRecord, sz)
+
+	rows, err := db.Query("SELECT cdn, data from dnssec")
+	if err != nil {
+		return fmt.Errorf("PGDNSSec gatherKeys: unable to query: %v", err)
+	}
+	defer rows.Close()
+	i := 0
+	for rows.Next() {
+		if i > len(tbl.Records)-1 {
+			return fmt.Errorf("PGDNSSec gatherKeys got more results than expected %v", len(tbl.Records))
+		}
+		err := rows.Scan(&tbl.Records[i].CDN, &tbl.Records[i].DataEncrypted)
+		if err != nil {
+			return fmt.Errorf("PGDNSSec gatherKeys unable to scan row: %v", err)
+		}
+		i += 1
+	}
+	return nil
+}
+func (tbl *PGDNSSecTable) decrypt(aesKey []byte) error {
+	for i, _ := range tbl.Records {
+		data, err := decryptInto(aesKey, tbl.Records[i].DataEncrypted, reflect.TypeOf(tbl.Records[i].Key))
+		if err != nil {
+			return fmt.Errorf("unable to decrypt into keys: %v", err)
+		}
+		tbl.Records[i].Key = *data.Interface().(*tc.DNSSECKeysTrafficVault)
+	}
+	return nil
+}
+func (tbl *PGDNSSecTable) encrypt(aesKey []byte) error {
+	for i, dns := range tbl.Records {
+		data, err := json.Marshal(&dns.Key)
+		if err != nil {
+			return fmt.Errorf("encrypt issue marshalling keys: %v", err)
+		}
+		dat, err := encrypt(data, aesKey)
+		if err != nil {
+			return fmt.Errorf("encrypt error: %v", err)
+		}
+		tbl.Records[i].DataEncrypted = dat
+	}
+	return nil
+}
+func (tbl *PGDNSSecTable) toGeneric() []DNSSecKey {
+	keys := make([]DNSSecKey, len(tbl.Records))
+
+	for i, record := range tbl.Records {
+		keys[i] = DNSSecKey{
+			CDN:                    record.CDN,
+			DNSSECKeysTrafficVault: record.Key,
+			CommonRecord:           record.CommonRecord,
+		}
+	}
+
+	return keys
+}
+func (tbl *PGDNSSecTable) fromGeneric(keys []DNSSecKey) {
+	tbl.Records = make([]PGDNSSecRecord, len(keys))
+
+	for i, key := range keys {
+		tbl.Records[i] = PGDNSSecRecord{
+			Key: key.DNSSECKeysTrafficVault,
+			CDN: key.CDN,
+			PGCommonRecord: PGCommonRecord{
+				DataEncrypted: nil,
+				CommonRecord:  CommonRecord{},
+			},
+		}
+	}
+}
+func (tbl *PGDNSSecTable) validate() []string {
+	for i, record := range tbl.Records {
+		if record.DataEncrypted == nil && len(record.Key) > 0 {
+			return []string{fmt.Sprintf("DNSSEC Key %v: DataEncrypted is blank!", i)}
+		}
+	}
+	return nil
+}
+func (tbl *PGDNSSecTable) insertKeys(db *sql.DB) error {
+	queryFmt := "INSERT INTO dnssec (cdn, data) VALUES "
+	stride := 2
+	queryArgs := make([]interface{}, len(tbl.Records)*stride)
+	for i, record := range tbl.Records {
+		j := i * stride
+		queryArgs[j] = record.CDN
+		queryArgs[j+1] = record.DataEncrypted
+	}
+	return insertIntoTable(db, queryFmt, stride, queryArgs)
+}
+
+type PGSSLKeyRecord struct {
+	Keys tc.DeliveryServiceSSLKeys
+	PGCommonRecord
+
+	// These records are stored on the table but are duplicated
+	DeliveryService string
+	Version         string
+	CDN             string
+}
+type PGSSLKeyTable struct {
+	Records []PGSSLKeyRecord
+}
+
+func (tbl *PGSSLKeyTable) insertKeys(db *sql.DB) error {
+	queryFmt := "INSERT INTO sslkey (deliveryservice, data, cdn, version) VALUES "
+	stride := 4
+	queryArgs := make([]interface{}, len(tbl.Records)*stride)
+	for i, record := range tbl.Records {
+		j := i * stride
+		queryArgs[j] = record.DeliveryService
+		queryArgs[j+1] = record.DataEncrypted
+		queryArgs[j+2] = record.CDN
+		queryArgs[j+3] = record.Version
+	}
+	return insertIntoTable(db, queryFmt, 4, queryArgs)
+}
+func (tbl *PGSSLKeyTable) gatherKeys(db *sql.DB) error {
+	sz, err := getSize(db, "sslkey")
+	if err != nil {
+		return fmt.Errorf("PGSSLKey gatherKeys unable to determine size of sslkey table: %v", err)
+	}
+	tbl.Records = make([]PGSSLKeyRecord, sz)
+
+	rows, err := db.Query("SELECT data, deliveryservice, cdn, version from sslkey")
+	if err != nil {
+		return fmt.Errorf("PGSSLKey gatherKeys unable to query: %v", err)
+	}
+	defer rows.Close()
+	i := 0
+	for rows.Next() {
+		if i > len(tbl.Records)-1 {
+			return errors.New("PGSSLKey gatherKeys: got more results than expected")
+		}
+		err := rows.Scan(&tbl.Records[i].DataEncrypted, &tbl.Records[i].DeliveryService, &tbl.Records[i].CDN, &tbl.Records[i].Version)
+		if err != nil {
+			return fmt.Errorf("PGSSLKey gatherKeys unable to scan row: %v", err)

Review comment:
       same as above RE: `%w` vs `%v`

##########
File path: tools/traffic_vault_migrate/postgres.go
##########
@@ -0,0 +1,716 @@
+package main
+
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+import (
+	"database/sql"
+	"encoding/base64"
+	"encoding/json"
+	"errors"
+	"fmt"
+	"github.com/apache/trafficcontrol/lib/go-tc"
+	util "github.com/apache/trafficcontrol/lib/go-util"
+	_ "github.com/lib/pq"
+	"log"
+	"reflect"
+	"strconv"
+	"strings"
+)
+
+type PGConfig struct {
+	Host     string `json:"host"`
+	Port     string `json:"port"`
+	User     string `json:"user"`
+	Password string `json:"password"`
+	SSLMode  string `json:"sslmode"`
+	Database string `json:"database"`
+	Key      string `json:"aesKey"`
+	AESKey   []byte
+}
+type PGBackend struct {
+	sslKey PGSSLKeyTable
+	dnssec PGDNSSecTable
+	uri    PGURISignKeyTable
+	url    PGURLSigKeyTable
+	cfg    PGConfig
+	db     *sql.DB
+}
+
+func (pg *PGBackend) String() string {
+	data := fmt.Sprintf("PG server %v@%v:%v\n", pg.cfg.User, pg.cfg.Host, pg.cfg.Port)
+	data += fmt.Sprintf("\tSSL Keys: %v\n", len(pg.sslKey.Records))
+	data += fmt.Sprintf("\tDNSSec Keys: %v\n", len(pg.dnssec.Records))
+	data += fmt.Sprintf("\tURI Keys: %v\n", len(pg.uri.Records))
+	data += fmt.Sprintf("\tURL Keys: %v\n", len(pg.url.Records))
+	return data
+}
+func (pg *PGBackend) Name() string {
+	return "PG"
+}
+func (pg *PGBackend) ReadConfig(s string) error {
+	genericCfg, err := UnmarshalConfig(s, reflect.TypeOf(pg.cfg))
+	if err != nil {
+		return err
+	}
+
+	pg.cfg = *genericCfg.Interface().(*PGConfig)
+
+	pg.cfg.AESKey, err = base64.StdEncoding.DecodeString(pg.cfg.Key)
+	if err != nil {
+		return fmt.Errorf("unable to decode PG AESKey: %v", err)
+	}
+	return nil
+}
+func (pg *PGBackend) Insert() error {
+	if err := pg.sslKey.insertKeys(pg.db); err != nil {
+		return err
+	}
+	if err := pg.dnssec.insertKeys(pg.db); err != nil {
+		return err
+	}
+	if err := pg.url.insertKeys(pg.db); err != nil {
+		return err
+	}
+	if err := pg.uri.insertKeys(pg.db); err != nil {
+		return err
+	}
+	return nil
+}
+func (pg *PGBackend) Start() error {
+	sqlStr := fmt.Sprintf("postgres://%s:%s@%s:%s/%s?sslmode=%s", pg.cfg.User, pg.cfg.Password, pg.cfg.Host, pg.cfg.Port, pg.cfg.Database, pg.cfg.SSLMode)
+	db, err := sql.Open("postgres", sqlStr)
+	if err != nil {
+		return fmt.Errorf("Unable to start PG client: %v", err)
+	}
+
+	pg.db = db
+	pg.sslKey = PGSSLKeyTable{}
+	pg.dnssec = PGDNSSecTable{}
+	pg.url = PGURLSigKeyTable{}
+	pg.uri = PGURISignKeyTable{}
+
+	return nil
+}
+func (pg *PGBackend) ValidateKey() []string {
+	var errors []string
+	if errs := pg.sslKey.validate(); errs != nil {
+		errors = append(errors, errs...)
+	}
+	if errs := pg.dnssec.validate(); errs != nil {
+		errors = append(errors, errs...)
+	}
+	if errs := pg.uri.validate(); errs != nil {
+		errors = append(errors, errs...)
+	}
+	if errs := pg.url.validate(); errs != nil {
+		errors = append(errors, errs...)
+	}
+	return errors
+}
+func (pg *PGBackend) Stop() error {
+	return pg.db.Close()
+}
+func (pg *PGBackend) Ping() error {
+	return pg.db.Ping()
+}
+func (pg *PGBackend) Fetch() error {
+	if err := pg.sslKey.gatherKeys(pg.db); err != nil {
+		return err
+	}
+
+	if err := pg.dnssec.gatherKeys(pg.db); err != nil {
+		return err
+	}
+
+	if err := pg.url.gatherKeys(pg.db); err != nil {
+		return err
+	}
+
+	if err := pg.uri.gatherKeys(pg.db); err != nil {
+		return err
+	}
+
+	return nil
+}
+func (pg *PGBackend) GetSSLKeys() ([]SSLKey, error) {
+	if err := pg.sslKey.decrypt(pg.cfg.AESKey); err != nil {
+		return nil, err
+	}
+	return pg.sslKey.toGeneric(), nil
+}
+func (pg *PGBackend) SetSSLKeys(keys []SSLKey) error {
+	pg.sslKey.fromGeneric(keys)
+	return pg.sslKey.encrypt(pg.cfg.AESKey)
+}
+func (pg *PGBackend) GetDNSSecKeys() ([]DNSSecKey, error) {
+	if err := pg.dnssec.decrypt(pg.cfg.AESKey); err != nil {
+		return nil, err
+	}
+	return pg.dnssec.toGeneric(), nil
+}
+func (pg *PGBackend) SetDNSSecKeys(keys []DNSSecKey) error {
+	pg.dnssec.fromGeneric(keys)
+	return pg.dnssec.encrypt(pg.cfg.AESKey)
+}
+func (pg *PGBackend) GetURISignKeys() ([]URISignKey, error) {
+	if err := pg.uri.decrypt(pg.cfg.AESKey); err != nil {
+		return nil, err
+	}
+	return pg.uri.toGeneric(), nil
+}
+func (pg *PGBackend) SetURISignKeys(keys []URISignKey) error {
+	pg.uri.fromGeneric(keys)
+	return pg.uri.encrypt(pg.cfg.AESKey)
+}
+func (pg *PGBackend) GetURLSigKeys() ([]URLSigKey, error) {
+	if err := pg.url.decrypt(pg.cfg.AESKey); err != nil {
+		return nil, err
+	}
+	return pg.url.toGeneric(), nil
+}
+func (pg *PGBackend) SetURLSigKeys(keys []URLSigKey) error {
+	pg.url.fromGeneric(keys)
+	return pg.url.encrypt(pg.cfg.AESKey)
+}
+
+type PGCommonRecord struct {
+	DataEncrypted []byte
+	CommonRecord
+}
+
+type PGDNSSecRecord struct {
+	Key tc.DNSSECKeysTrafficVault
+	CDN string
+	PGCommonRecord
+}
+type PGDNSSecTable struct {
+	Records []PGDNSSecRecord
+}
+
+func (tbl *PGDNSSecTable) gatherKeys(db *sql.DB) error {
+	sz, err := getSize(db, "dnssec")
+	if err != nil {
+		log.Println("PGDNSSec gatherKeys: unable to determine size of dnssec table")
+	}
+	tbl.Records = make([]PGDNSSecRecord, sz)
+
+	rows, err := db.Query("SELECT cdn, data from dnssec")
+	if err != nil {
+		return fmt.Errorf("PGDNSSec gatherKeys: unable to query: %v", err)
+	}
+	defer rows.Close()
+	i := 0
+	for rows.Next() {
+		if i > len(tbl.Records)-1 {
+			return fmt.Errorf("PGDNSSec gatherKeys got more results than expected %v", len(tbl.Records))
+		}
+		err := rows.Scan(&tbl.Records[i].CDN, &tbl.Records[i].DataEncrypted)
+		if err != nil {
+			return fmt.Errorf("PGDNSSec gatherKeys unable to scan row: %v", err)
+		}
+		i += 1
+	}
+	return nil
+}
+func (tbl *PGDNSSecTable) decrypt(aesKey []byte) error {
+	for i, _ := range tbl.Records {
+		data, err := decryptInto(aesKey, tbl.Records[i].DataEncrypted, reflect.TypeOf(tbl.Records[i].Key))
+		if err != nil {
+			return fmt.Errorf("unable to decrypt into keys: %v", err)
+		}
+		tbl.Records[i].Key = *data.Interface().(*tc.DNSSECKeysTrafficVault)
+	}
+	return nil
+}
+func (tbl *PGDNSSecTable) encrypt(aesKey []byte) error {
+	for i, dns := range tbl.Records {
+		data, err := json.Marshal(&dns.Key)
+		if err != nil {
+			return fmt.Errorf("encrypt issue marshalling keys: %v", err)
+		}
+		dat, err := encrypt(data, aesKey)
+		if err != nil {
+			return fmt.Errorf("encrypt error: %v", err)
+		}
+		tbl.Records[i].DataEncrypted = dat
+	}
+	return nil
+}
+func (tbl *PGDNSSecTable) toGeneric() []DNSSecKey {
+	keys := make([]DNSSecKey, len(tbl.Records))
+
+	for i, record := range tbl.Records {
+		keys[i] = DNSSecKey{
+			CDN:                    record.CDN,
+			DNSSECKeysTrafficVault: record.Key,
+			CommonRecord:           record.CommonRecord,
+		}
+	}
+
+	return keys
+}
+func (tbl *PGDNSSecTable) fromGeneric(keys []DNSSecKey) {
+	tbl.Records = make([]PGDNSSecRecord, len(keys))
+
+	for i, key := range keys {
+		tbl.Records[i] = PGDNSSecRecord{
+			Key: key.DNSSECKeysTrafficVault,
+			CDN: key.CDN,
+			PGCommonRecord: PGCommonRecord{
+				DataEncrypted: nil,
+				CommonRecord:  CommonRecord{},
+			},
+		}
+	}
+}
+func (tbl *PGDNSSecTable) validate() []string {
+	for i, record := range tbl.Records {
+		if record.DataEncrypted == nil && len(record.Key) > 0 {
+			return []string{fmt.Sprintf("DNSSEC Key %v: DataEncrypted is blank!", i)}
+		}
+	}
+	return nil
+}
+func (tbl *PGDNSSecTable) insertKeys(db *sql.DB) error {
+	queryFmt := "INSERT INTO dnssec (cdn, data) VALUES "
+	stride := 2
+	queryArgs := make([]interface{}, len(tbl.Records)*stride)
+	for i, record := range tbl.Records {
+		j := i * stride
+		queryArgs[j] = record.CDN
+		queryArgs[j+1] = record.DataEncrypted
+	}
+	return insertIntoTable(db, queryFmt, stride, queryArgs)
+}
+
+type PGSSLKeyRecord struct {
+	Keys tc.DeliveryServiceSSLKeys
+	PGCommonRecord
+
+	// These records are stored on the table but are duplicated
+	DeliveryService string
+	Version         string
+	CDN             string
+}
+type PGSSLKeyTable struct {
+	Records []PGSSLKeyRecord
+}
+
+func (tbl *PGSSLKeyTable) insertKeys(db *sql.DB) error {
+	queryFmt := "INSERT INTO sslkey (deliveryservice, data, cdn, version) VALUES "
+	stride := 4
+	queryArgs := make([]interface{}, len(tbl.Records)*stride)
+	for i, record := range tbl.Records {
+		j := i * stride
+		queryArgs[j] = record.DeliveryService
+		queryArgs[j+1] = record.DataEncrypted
+		queryArgs[j+2] = record.CDN
+		queryArgs[j+3] = record.Version
+	}
+	return insertIntoTable(db, queryFmt, 4, queryArgs)
+}
+func (tbl *PGSSLKeyTable) gatherKeys(db *sql.DB) error {
+	sz, err := getSize(db, "sslkey")
+	if err != nil {
+		return fmt.Errorf("PGSSLKey gatherKeys unable to determine size of sslkey table: %v", err)
+	}
+	tbl.Records = make([]PGSSLKeyRecord, sz)
+
+	rows, err := db.Query("SELECT data, deliveryservice, cdn, version from sslkey")
+	if err != nil {
+		return fmt.Errorf("PGSSLKey gatherKeys unable to query: %v", err)
+	}
+	defer rows.Close()
+	i := 0
+	for rows.Next() {
+		if i > len(tbl.Records)-1 {
+			return errors.New("PGSSLKey gatherKeys: got more results than expected")
+		}
+		err := rows.Scan(&tbl.Records[i].DataEncrypted, &tbl.Records[i].DeliveryService, &tbl.Records[i].CDN, &tbl.Records[i].Version)
+		if err != nil {
+			return fmt.Errorf("PGSSLKey gatherKeys unable to scan row: %v", err)
+		}
+		i += 1
+	}
+	return nil
+}
+func (tbl *PGSSLKeyTable) decrypt(aesKey []byte) error {
+	for i, dns := range tbl.Records {
+		data, err := decryptInto(aesKey, dns.DataEncrypted, reflect.TypeOf(dns.Keys))
+		if err != nil {
+			return fmt.Errorf("unable to decrypt into keys: %v", err)
+		}
+		tbl.Records[i].Keys = *data.Interface().(*tc.DeliveryServiceSSLKeys)
+	}
+	return nil
+}
+func (tbl *PGSSLKeyTable) encrypt(aesKey []byte) error {
+	for i, dns := range tbl.Records {
+		data, err := json.Marshal(dns.Keys)
+		if err != nil {
+			return fmt.Errorf("encrypt issue marshalling keys: %v", err)
+		}
+		dat, err := encrypt(data, aesKey)
+		if err != nil {
+			return fmt.Errorf("encrypt error: %v", err)
+		}
+		tbl.Records[i].DataEncrypted = dat
+	}
+	return nil
+}
+func (tbl *PGSSLKeyTable) toGeneric() []SSLKey {
+	keys := make([]SSLKey, len(tbl.Records))
+
+	for i, record := range tbl.Records {
+		keys[i] = SSLKey{
+			DeliveryServiceSSLKeys: record.Keys,
+			CommonRecord:           record.CommonRecord,
+		}
+	}
+	return keys
+}
+func (tbl *PGSSLKeyTable) fromGeneric(keys []SSLKey) {
+	tbl.Records = make([]PGSSLKeyRecord, len(keys))
+
+	for i, key := range keys {
+		tbl.Records[i] = PGSSLKeyRecord{
+			Keys: key.DeliveryServiceSSLKeys,
+			PGCommonRecord: PGCommonRecord{
+				DataEncrypted: nil,
+				CommonRecord:  CommonRecord{},
+			},
+			DeliveryService: key.DeliveryService,
+			CDN:             key.CDN,
+		}
+	}
+}
+func (tbl *PGSSLKeyTable) validate() []string {
+	defaultKey := tc.DeliveryServiceSSLKeys{}
+	var errors []string
+	fmtStr := "SSL Key %v: %v"
+	for i, record := range tbl.Records {
+		if record.Keys == defaultKey {
+			errors = append(errors, fmt.Sprintf(fmtStr, i, "DS SSL Keys are default!"))
+		} else if record.Keys.Key == "" {
+			errors = append(errors, fmt.Sprintf(fmtStr, i, "Key is blank!"))
+		} else if record.Keys.CDN == "" {
+			errors = append(errors, fmt.Sprintf(fmtStr, i, "CDN is blank!"))
+		} else if record.Keys.DeliveryService == "" {
+			errors = append(errors, fmt.Sprintf(fmtStr, i, "DS is blank!"))
+		} else if record.DataEncrypted == nil {
+			errors = append(errors, fmt.Sprintf(fmtStr, i, "DataEncrypted is blank!"))
+		}
+	}
+	return errors
+}
+
+type PGURLSigKeyRecord struct {
+	Keys            tc.URLSigKeys
+	DeliveryService string
+	PGCommonRecord
+}
+type PGURLSigKeyTable struct {
+	Records []PGURLSigKeyRecord
+}
+
+func (tbl *PGURLSigKeyTable) insertKeys(db *sql.DB) error {
+	queryBase := "INSERT INTO url_sig_key (deliveryservice, data) VALUES "
+	stride := 2
+	queryArgs := make([]interface{}, len(tbl.Records)*stride)
+	for i, record := range tbl.Records {
+		j := i * stride
+		queryArgs[j] = record.DeliveryService
+		queryArgs[j+1] = record.DataEncrypted
+	}
+	return insertIntoTable(db, queryBase, stride, queryArgs)
+}
+func (tbl *PGURLSigKeyTable) gatherKeys(db *sql.DB) error {
+	sz, err := getSize(db, "url_sig_key")
+	if err != nil {
+		log.Println("PGURLSigKey gatherKeys: unable to determine url_sig_key table size")
+	}
+	tbl.Records = make([]PGURLSigKeyRecord, sz)
+
+	rows, err := db.Query("SELECT deliveryservice, data from url_sig_key")
+	if err != nil {
+		return fmt.Errorf("PGURLSigKey gatherKeys error creating query: %v", err)
+	}
+	defer rows.Close()
+	i := 0
+	for rows.Next() {
+		if i > len(tbl.Records)-1 {
+			return fmt.Errorf("PGURLSigKey gatherKeys: got more results than expected %v", len(tbl.Records))
+		}
+		err := rows.Scan(&tbl.Records[i].DeliveryService, &tbl.Records[i].DataEncrypted)
+		if err != nil {
+			return fmt.Errorf("PGURLSigKey gatherKeys: unable to scan row: %v", err)

Review comment:
       same as above RE: `%w` vs `%v`

##########
File path: tools/traffic_vault_migrate/postgres.go
##########
@@ -0,0 +1,716 @@
+package main
+
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+import (
+	"database/sql"
+	"encoding/base64"
+	"encoding/json"
+	"errors"
+	"fmt"
+	"github.com/apache/trafficcontrol/lib/go-tc"
+	util "github.com/apache/trafficcontrol/lib/go-util"
+	_ "github.com/lib/pq"
+	"log"
+	"reflect"
+	"strconv"
+	"strings"
+)
+
+type PGConfig struct {
+	Host     string `json:"host"`
+	Port     string `json:"port"`
+	User     string `json:"user"`
+	Password string `json:"password"`
+	SSLMode  string `json:"sslmode"`
+	Database string `json:"database"`
+	Key      string `json:"aesKey"`
+	AESKey   []byte
+}
+type PGBackend struct {
+	sslKey PGSSLKeyTable
+	dnssec PGDNSSecTable
+	uri    PGURISignKeyTable
+	url    PGURLSigKeyTable
+	cfg    PGConfig
+	db     *sql.DB
+}
+
+func (pg *PGBackend) String() string {
+	data := fmt.Sprintf("PG server %v@%v:%v\n", pg.cfg.User, pg.cfg.Host, pg.cfg.Port)
+	data += fmt.Sprintf("\tSSL Keys: %v\n", len(pg.sslKey.Records))
+	data += fmt.Sprintf("\tDNSSec Keys: %v\n", len(pg.dnssec.Records))
+	data += fmt.Sprintf("\tURI Keys: %v\n", len(pg.uri.Records))
+	data += fmt.Sprintf("\tURL Keys: %v\n", len(pg.url.Records))
+	return data
+}
+func (pg *PGBackend) Name() string {
+	return "PG"
+}
+func (pg *PGBackend) ReadConfig(s string) error {
+	genericCfg, err := UnmarshalConfig(s, reflect.TypeOf(pg.cfg))
+	if err != nil {
+		return err
+	}
+
+	pg.cfg = *genericCfg.Interface().(*PGConfig)
+
+	pg.cfg.AESKey, err = base64.StdEncoding.DecodeString(pg.cfg.Key)
+	if err != nil {
+		return fmt.Errorf("unable to decode PG AESKey: %v", err)
+	}
+	return nil
+}
+func (pg *PGBackend) Insert() error {
+	if err := pg.sslKey.insertKeys(pg.db); err != nil {
+		return err
+	}
+	if err := pg.dnssec.insertKeys(pg.db); err != nil {
+		return err
+	}
+	if err := pg.url.insertKeys(pg.db); err != nil {
+		return err
+	}
+	if err := pg.uri.insertKeys(pg.db); err != nil {
+		return err
+	}
+	return nil
+}
+func (pg *PGBackend) Start() error {
+	sqlStr := fmt.Sprintf("postgres://%s:%s@%s:%s/%s?sslmode=%s", pg.cfg.User, pg.cfg.Password, pg.cfg.Host, pg.cfg.Port, pg.cfg.Database, pg.cfg.SSLMode)
+	db, err := sql.Open("postgres", sqlStr)
+	if err != nil {
+		return fmt.Errorf("Unable to start PG client: %v", err)
+	}
+
+	pg.db = db
+	pg.sslKey = PGSSLKeyTable{}
+	pg.dnssec = PGDNSSecTable{}
+	pg.url = PGURLSigKeyTable{}
+	pg.uri = PGURISignKeyTable{}
+
+	return nil
+}
+func (pg *PGBackend) ValidateKey() []string {
+	var errors []string
+	if errs := pg.sslKey.validate(); errs != nil {
+		errors = append(errors, errs...)
+	}
+	if errs := pg.dnssec.validate(); errs != nil {
+		errors = append(errors, errs...)
+	}
+	if errs := pg.uri.validate(); errs != nil {
+		errors = append(errors, errs...)
+	}
+	if errs := pg.url.validate(); errs != nil {
+		errors = append(errors, errs...)
+	}
+	return errors
+}
+func (pg *PGBackend) Stop() error {
+	return pg.db.Close()
+}
+func (pg *PGBackend) Ping() error {
+	return pg.db.Ping()
+}
+func (pg *PGBackend) Fetch() error {
+	if err := pg.sslKey.gatherKeys(pg.db); err != nil {
+		return err
+	}
+
+	if err := pg.dnssec.gatherKeys(pg.db); err != nil {
+		return err
+	}
+
+	if err := pg.url.gatherKeys(pg.db); err != nil {
+		return err
+	}
+
+	if err := pg.uri.gatherKeys(pg.db); err != nil {
+		return err
+	}
+
+	return nil
+}
+func (pg *PGBackend) GetSSLKeys() ([]SSLKey, error) {
+	if err := pg.sslKey.decrypt(pg.cfg.AESKey); err != nil {
+		return nil, err
+	}
+	return pg.sslKey.toGeneric(), nil
+}
+func (pg *PGBackend) SetSSLKeys(keys []SSLKey) error {
+	pg.sslKey.fromGeneric(keys)
+	return pg.sslKey.encrypt(pg.cfg.AESKey)
+}
+func (pg *PGBackend) GetDNSSecKeys() ([]DNSSecKey, error) {
+	if err := pg.dnssec.decrypt(pg.cfg.AESKey); err != nil {
+		return nil, err
+	}
+	return pg.dnssec.toGeneric(), nil
+}
+func (pg *PGBackend) SetDNSSecKeys(keys []DNSSecKey) error {
+	pg.dnssec.fromGeneric(keys)
+	return pg.dnssec.encrypt(pg.cfg.AESKey)
+}
+func (pg *PGBackend) GetURISignKeys() ([]URISignKey, error) {
+	if err := pg.uri.decrypt(pg.cfg.AESKey); err != nil {
+		return nil, err
+	}
+	return pg.uri.toGeneric(), nil
+}
+func (pg *PGBackend) SetURISignKeys(keys []URISignKey) error {
+	pg.uri.fromGeneric(keys)
+	return pg.uri.encrypt(pg.cfg.AESKey)
+}
+func (pg *PGBackend) GetURLSigKeys() ([]URLSigKey, error) {
+	if err := pg.url.decrypt(pg.cfg.AESKey); err != nil {
+		return nil, err
+	}
+	return pg.url.toGeneric(), nil
+}
+func (pg *PGBackend) SetURLSigKeys(keys []URLSigKey) error {
+	pg.url.fromGeneric(keys)
+	return pg.url.encrypt(pg.cfg.AESKey)
+}
+
+type PGCommonRecord struct {
+	DataEncrypted []byte
+	CommonRecord
+}
+
+type PGDNSSecRecord struct {
+	Key tc.DNSSECKeysTrafficVault
+	CDN string
+	PGCommonRecord
+}
+type PGDNSSecTable struct {
+	Records []PGDNSSecRecord
+}
+
+func (tbl *PGDNSSecTable) gatherKeys(db *sql.DB) error {
+	sz, err := getSize(db, "dnssec")
+	if err != nil {
+		log.Println("PGDNSSec gatherKeys: unable to determine size of dnssec table")
+	}
+	tbl.Records = make([]PGDNSSecRecord, sz)
+
+	rows, err := db.Query("SELECT cdn, data from dnssec")
+	if err != nil {
+		return fmt.Errorf("PGDNSSec gatherKeys: unable to query: %v", err)
+	}
+	defer rows.Close()
+	i := 0
+	for rows.Next() {
+		if i > len(tbl.Records)-1 {
+			return fmt.Errorf("PGDNSSec gatherKeys got more results than expected %v", len(tbl.Records))
+		}
+		err := rows.Scan(&tbl.Records[i].CDN, &tbl.Records[i].DataEncrypted)
+		if err != nil {
+			return fmt.Errorf("PGDNSSec gatherKeys unable to scan row: %v", err)
+		}
+		i += 1
+	}
+	return nil
+}
+func (tbl *PGDNSSecTable) decrypt(aesKey []byte) error {
+	for i, _ := range tbl.Records {
+		data, err := decryptInto(aesKey, tbl.Records[i].DataEncrypted, reflect.TypeOf(tbl.Records[i].Key))
+		if err != nil {
+			return fmt.Errorf("unable to decrypt into keys: %v", err)
+		}
+		tbl.Records[i].Key = *data.Interface().(*tc.DNSSECKeysTrafficVault)
+	}
+	return nil
+}
+func (tbl *PGDNSSecTable) encrypt(aesKey []byte) error {
+	for i, dns := range tbl.Records {
+		data, err := json.Marshal(&dns.Key)
+		if err != nil {
+			return fmt.Errorf("encrypt issue marshalling keys: %v", err)
+		}
+		dat, err := encrypt(data, aesKey)
+		if err != nil {
+			return fmt.Errorf("encrypt error: %v", err)
+		}
+		tbl.Records[i].DataEncrypted = dat
+	}
+	return nil
+}
+func (tbl *PGDNSSecTable) toGeneric() []DNSSecKey {
+	keys := make([]DNSSecKey, len(tbl.Records))
+
+	for i, record := range tbl.Records {
+		keys[i] = DNSSecKey{
+			CDN:                    record.CDN,
+			DNSSECKeysTrafficVault: record.Key,
+			CommonRecord:           record.CommonRecord,
+		}
+	}
+
+	return keys
+}
+func (tbl *PGDNSSecTable) fromGeneric(keys []DNSSecKey) {
+	tbl.Records = make([]PGDNSSecRecord, len(keys))
+
+	for i, key := range keys {
+		tbl.Records[i] = PGDNSSecRecord{
+			Key: key.DNSSECKeysTrafficVault,
+			CDN: key.CDN,
+			PGCommonRecord: PGCommonRecord{
+				DataEncrypted: nil,
+				CommonRecord:  CommonRecord{},
+			},
+		}
+	}
+}
+func (tbl *PGDNSSecTable) validate() []string {
+	for i, record := range tbl.Records {
+		if record.DataEncrypted == nil && len(record.Key) > 0 {
+			return []string{fmt.Sprintf("DNSSEC Key %v: DataEncrypted is blank!", i)}
+		}
+	}
+	return nil
+}
+func (tbl *PGDNSSecTable) insertKeys(db *sql.DB) error {
+	queryFmt := "INSERT INTO dnssec (cdn, data) VALUES "
+	stride := 2
+	queryArgs := make([]interface{}, len(tbl.Records)*stride)
+	for i, record := range tbl.Records {
+		j := i * stride
+		queryArgs[j] = record.CDN
+		queryArgs[j+1] = record.DataEncrypted
+	}
+	return insertIntoTable(db, queryFmt, stride, queryArgs)
+}
+
+type PGSSLKeyRecord struct {
+	Keys tc.DeliveryServiceSSLKeys
+	PGCommonRecord
+
+	// These records are stored on the table but are duplicated
+	DeliveryService string
+	Version         string
+	CDN             string
+}
+type PGSSLKeyTable struct {
+	Records []PGSSLKeyRecord
+}
+
+func (tbl *PGSSLKeyTable) insertKeys(db *sql.DB) error {
+	queryFmt := "INSERT INTO sslkey (deliveryservice, data, cdn, version) VALUES "
+	stride := 4
+	queryArgs := make([]interface{}, len(tbl.Records)*stride)
+	for i, record := range tbl.Records {
+		j := i * stride
+		queryArgs[j] = record.DeliveryService
+		queryArgs[j+1] = record.DataEncrypted
+		queryArgs[j+2] = record.CDN
+		queryArgs[j+3] = record.Version
+	}
+	return insertIntoTable(db, queryFmt, 4, queryArgs)
+}
+func (tbl *PGSSLKeyTable) gatherKeys(db *sql.DB) error {
+	sz, err := getSize(db, "sslkey")
+	if err != nil {
+		return fmt.Errorf("PGSSLKey gatherKeys unable to determine size of sslkey table: %v", err)
+	}
+	tbl.Records = make([]PGSSLKeyRecord, sz)
+
+	rows, err := db.Query("SELECT data, deliveryservice, cdn, version from sslkey")
+	if err != nil {
+		return fmt.Errorf("PGSSLKey gatherKeys unable to query: %v", err)
+	}
+	defer rows.Close()
+	i := 0
+	for rows.Next() {
+		if i > len(tbl.Records)-1 {
+			return errors.New("PGSSLKey gatherKeys: got more results than expected")
+		}
+		err := rows.Scan(&tbl.Records[i].DataEncrypted, &tbl.Records[i].DeliveryService, &tbl.Records[i].CDN, &tbl.Records[i].Version)
+		if err != nil {
+			return fmt.Errorf("PGSSLKey gatherKeys unable to scan row: %v", err)
+		}
+		i += 1
+	}
+	return nil
+}
+func (tbl *PGSSLKeyTable) decrypt(aesKey []byte) error {
+	for i, dns := range tbl.Records {
+		data, err := decryptInto(aesKey, dns.DataEncrypted, reflect.TypeOf(dns.Keys))
+		if err != nil {
+			return fmt.Errorf("unable to decrypt into keys: %v", err)
+		}
+		tbl.Records[i].Keys = *data.Interface().(*tc.DeliveryServiceSSLKeys)
+	}
+	return nil
+}
+func (tbl *PGSSLKeyTable) encrypt(aesKey []byte) error {
+	for i, dns := range tbl.Records {
+		data, err := json.Marshal(dns.Keys)
+		if err != nil {
+			return fmt.Errorf("encrypt issue marshalling keys: %v", err)
+		}
+		dat, err := encrypt(data, aesKey)
+		if err != nil {
+			return fmt.Errorf("encrypt error: %v", err)

Review comment:
       same as above RE: `%w` vs `%v`

##########
File path: tools/traffic_vault_migrate/postgres.go
##########
@@ -0,0 +1,716 @@
+package main
+
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+import (
+	"database/sql"
+	"encoding/base64"
+	"encoding/json"
+	"errors"
+	"fmt"
+	"github.com/apache/trafficcontrol/lib/go-tc"
+	util "github.com/apache/trafficcontrol/lib/go-util"
+	_ "github.com/lib/pq"
+	"log"
+	"reflect"
+	"strconv"
+	"strings"
+)

Review comment:
       import order should be
   
   1. Standard Library
   2. Project-internal (those under `github.com/apache/trafficcontrol/`)
   3. Third-party libraries

##########
File path: tools/traffic_vault_migrate/postgres.go
##########
@@ -0,0 +1,716 @@
+package main
+
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+import (
+	"database/sql"
+	"encoding/base64"
+	"encoding/json"
+	"errors"
+	"fmt"
+	"github.com/apache/trafficcontrol/lib/go-tc"
+	util "github.com/apache/trafficcontrol/lib/go-util"
+	_ "github.com/lib/pq"
+	"log"
+	"reflect"
+	"strconv"
+	"strings"
+)
+
+type PGConfig struct {
+	Host     string `json:"host"`
+	Port     string `json:"port"`
+	User     string `json:"user"`
+	Password string `json:"password"`
+	SSLMode  string `json:"sslmode"`
+	Database string `json:"database"`
+	Key      string `json:"aesKey"`
+	AESKey   []byte
+}
+type PGBackend struct {
+	sslKey PGSSLKeyTable
+	dnssec PGDNSSecTable
+	uri    PGURISignKeyTable
+	url    PGURLSigKeyTable
+	cfg    PGConfig
+	db     *sql.DB
+}
+
+func (pg *PGBackend) String() string {
+	data := fmt.Sprintf("PG server %v@%v:%v\n", pg.cfg.User, pg.cfg.Host, pg.cfg.Port)
+	data += fmt.Sprintf("\tSSL Keys: %v\n", len(pg.sslKey.Records))
+	data += fmt.Sprintf("\tDNSSec Keys: %v\n", len(pg.dnssec.Records))
+	data += fmt.Sprintf("\tURI Keys: %v\n", len(pg.uri.Records))
+	data += fmt.Sprintf("\tURL Keys: %v\n", len(pg.url.Records))
+	return data
+}
+func (pg *PGBackend) Name() string {
+	return "PG"
+}
+func (pg *PGBackend) ReadConfig(s string) error {
+	genericCfg, err := UnmarshalConfig(s, reflect.TypeOf(pg.cfg))
+	if err != nil {
+		return err
+	}
+
+	pg.cfg = *genericCfg.Interface().(*PGConfig)
+
+	pg.cfg.AESKey, err = base64.StdEncoding.DecodeString(pg.cfg.Key)
+	if err != nil {
+		return fmt.Errorf("unable to decode PG AESKey: %v", err)
+	}
+	return nil
+}
+func (pg *PGBackend) Insert() error {
+	if err := pg.sslKey.insertKeys(pg.db); err != nil {
+		return err
+	}
+	if err := pg.dnssec.insertKeys(pg.db); err != nil {
+		return err
+	}
+	if err := pg.url.insertKeys(pg.db); err != nil {
+		return err
+	}
+	if err := pg.uri.insertKeys(pg.db); err != nil {
+		return err
+	}
+	return nil
+}
+func (pg *PGBackend) Start() error {
+	sqlStr := fmt.Sprintf("postgres://%s:%s@%s:%s/%s?sslmode=%s", pg.cfg.User, pg.cfg.Password, pg.cfg.Host, pg.cfg.Port, pg.cfg.Database, pg.cfg.SSLMode)
+	db, err := sql.Open("postgres", sqlStr)
+	if err != nil {
+		return fmt.Errorf("Unable to start PG client: %v", err)
+	}
+
+	pg.db = db
+	pg.sslKey = PGSSLKeyTable{}
+	pg.dnssec = PGDNSSecTable{}
+	pg.url = PGURLSigKeyTable{}
+	pg.uri = PGURISignKeyTable{}
+
+	return nil
+}
+func (pg *PGBackend) ValidateKey() []string {
+	var errors []string
+	if errs := pg.sslKey.validate(); errs != nil {
+		errors = append(errors, errs...)
+	}
+	if errs := pg.dnssec.validate(); errs != nil {
+		errors = append(errors, errs...)
+	}
+	if errs := pg.uri.validate(); errs != nil {
+		errors = append(errors, errs...)
+	}
+	if errs := pg.url.validate(); errs != nil {
+		errors = append(errors, errs...)
+	}
+	return errors
+}
+func (pg *PGBackend) Stop() error {
+	return pg.db.Close()
+}
+func (pg *PGBackend) Ping() error {
+	return pg.db.Ping()
+}
+func (pg *PGBackend) Fetch() error {
+	if err := pg.sslKey.gatherKeys(pg.db); err != nil {
+		return err
+	}
+
+	if err := pg.dnssec.gatherKeys(pg.db); err != nil {
+		return err
+	}
+
+	if err := pg.url.gatherKeys(pg.db); err != nil {
+		return err
+	}
+
+	if err := pg.uri.gatherKeys(pg.db); err != nil {
+		return err
+	}
+
+	return nil
+}
+func (pg *PGBackend) GetSSLKeys() ([]SSLKey, error) {
+	if err := pg.sslKey.decrypt(pg.cfg.AESKey); err != nil {
+		return nil, err
+	}
+	return pg.sslKey.toGeneric(), nil
+}
+func (pg *PGBackend) SetSSLKeys(keys []SSLKey) error {
+	pg.sslKey.fromGeneric(keys)
+	return pg.sslKey.encrypt(pg.cfg.AESKey)
+}
+func (pg *PGBackend) GetDNSSecKeys() ([]DNSSecKey, error) {
+	if err := pg.dnssec.decrypt(pg.cfg.AESKey); err != nil {
+		return nil, err
+	}
+	return pg.dnssec.toGeneric(), nil
+}
+func (pg *PGBackend) SetDNSSecKeys(keys []DNSSecKey) error {
+	pg.dnssec.fromGeneric(keys)
+	return pg.dnssec.encrypt(pg.cfg.AESKey)
+}
+func (pg *PGBackend) GetURISignKeys() ([]URISignKey, error) {
+	if err := pg.uri.decrypt(pg.cfg.AESKey); err != nil {
+		return nil, err
+	}
+	return pg.uri.toGeneric(), nil
+}
+func (pg *PGBackend) SetURISignKeys(keys []URISignKey) error {
+	pg.uri.fromGeneric(keys)
+	return pg.uri.encrypt(pg.cfg.AESKey)
+}
+func (pg *PGBackend) GetURLSigKeys() ([]URLSigKey, error) {
+	if err := pg.url.decrypt(pg.cfg.AESKey); err != nil {
+		return nil, err
+	}
+	return pg.url.toGeneric(), nil
+}
+func (pg *PGBackend) SetURLSigKeys(keys []URLSigKey) error {
+	pg.url.fromGeneric(keys)
+	return pg.url.encrypt(pg.cfg.AESKey)
+}
+
+type PGCommonRecord struct {
+	DataEncrypted []byte
+	CommonRecord
+}
+
+type PGDNSSecRecord struct {
+	Key tc.DNSSECKeysTrafficVault
+	CDN string
+	PGCommonRecord
+}
+type PGDNSSecTable struct {
+	Records []PGDNSSecRecord
+}
+
+func (tbl *PGDNSSecTable) gatherKeys(db *sql.DB) error {
+	sz, err := getSize(db, "dnssec")
+	if err != nil {
+		log.Println("PGDNSSec gatherKeys: unable to determine size of dnssec table")
+	}
+	tbl.Records = make([]PGDNSSecRecord, sz)
+
+	rows, err := db.Query("SELECT cdn, data from dnssec")
+	if err != nil {
+		return fmt.Errorf("PGDNSSec gatherKeys: unable to query: %v", err)
+	}
+	defer rows.Close()
+	i := 0
+	for rows.Next() {
+		if i > len(tbl.Records)-1 {
+			return fmt.Errorf("PGDNSSec gatherKeys got more results than expected %v", len(tbl.Records))
+		}
+		err := rows.Scan(&tbl.Records[i].CDN, &tbl.Records[i].DataEncrypted)
+		if err != nil {
+			return fmt.Errorf("PGDNSSec gatherKeys unable to scan row: %v", err)
+		}
+		i += 1
+	}
+	return nil
+}
+func (tbl *PGDNSSecTable) decrypt(aesKey []byte) error {
+	for i, _ := range tbl.Records {
+		data, err := decryptInto(aesKey, tbl.Records[i].DataEncrypted, reflect.TypeOf(tbl.Records[i].Key))
+		if err != nil {
+			return fmt.Errorf("unable to decrypt into keys: %v", err)
+		}
+		tbl.Records[i].Key = *data.Interface().(*tc.DNSSECKeysTrafficVault)
+	}
+	return nil
+}
+func (tbl *PGDNSSecTable) encrypt(aesKey []byte) error {
+	for i, dns := range tbl.Records {
+		data, err := json.Marshal(&dns.Key)
+		if err != nil {
+			return fmt.Errorf("encrypt issue marshalling keys: %v", err)
+		}
+		dat, err := encrypt(data, aesKey)
+		if err != nil {
+			return fmt.Errorf("encrypt error: %v", err)
+		}
+		tbl.Records[i].DataEncrypted = dat
+	}
+	return nil
+}
+func (tbl *PGDNSSecTable) toGeneric() []DNSSecKey {
+	keys := make([]DNSSecKey, len(tbl.Records))
+
+	for i, record := range tbl.Records {
+		keys[i] = DNSSecKey{
+			CDN:                    record.CDN,
+			DNSSECKeysTrafficVault: record.Key,
+			CommonRecord:           record.CommonRecord,
+		}
+	}
+
+	return keys
+}
+func (tbl *PGDNSSecTable) fromGeneric(keys []DNSSecKey) {
+	tbl.Records = make([]PGDNSSecRecord, len(keys))
+
+	for i, key := range keys {
+		tbl.Records[i] = PGDNSSecRecord{
+			Key: key.DNSSECKeysTrafficVault,
+			CDN: key.CDN,
+			PGCommonRecord: PGCommonRecord{
+				DataEncrypted: nil,
+				CommonRecord:  CommonRecord{},
+			},
+		}
+	}
+}
+func (tbl *PGDNSSecTable) validate() []string {
+	for i, record := range tbl.Records {
+		if record.DataEncrypted == nil && len(record.Key) > 0 {
+			return []string{fmt.Sprintf("DNSSEC Key %v: DataEncrypted is blank!", i)}
+		}
+	}
+	return nil
+}
+func (tbl *PGDNSSecTable) insertKeys(db *sql.DB) error {
+	queryFmt := "INSERT INTO dnssec (cdn, data) VALUES "
+	stride := 2
+	queryArgs := make([]interface{}, len(tbl.Records)*stride)
+	for i, record := range tbl.Records {
+		j := i * stride
+		queryArgs[j] = record.CDN
+		queryArgs[j+1] = record.DataEncrypted
+	}
+	return insertIntoTable(db, queryFmt, stride, queryArgs)
+}
+
+type PGSSLKeyRecord struct {
+	Keys tc.DeliveryServiceSSLKeys
+	PGCommonRecord
+
+	// These records are stored on the table but are duplicated
+	DeliveryService string
+	Version         string
+	CDN             string
+}
+type PGSSLKeyTable struct {
+	Records []PGSSLKeyRecord
+}
+
+func (tbl *PGSSLKeyTable) insertKeys(db *sql.DB) error {
+	queryFmt := "INSERT INTO sslkey (deliveryservice, data, cdn, version) VALUES "
+	stride := 4
+	queryArgs := make([]interface{}, len(tbl.Records)*stride)
+	for i, record := range tbl.Records {
+		j := i * stride
+		queryArgs[j] = record.DeliveryService
+		queryArgs[j+1] = record.DataEncrypted
+		queryArgs[j+2] = record.CDN
+		queryArgs[j+3] = record.Version
+	}
+	return insertIntoTable(db, queryFmt, 4, queryArgs)
+}
+func (tbl *PGSSLKeyTable) gatherKeys(db *sql.DB) error {
+	sz, err := getSize(db, "sslkey")
+	if err != nil {
+		return fmt.Errorf("PGSSLKey gatherKeys unable to determine size of sslkey table: %v", err)
+	}
+	tbl.Records = make([]PGSSLKeyRecord, sz)
+
+	rows, err := db.Query("SELECT data, deliveryservice, cdn, version from sslkey")
+	if err != nil {
+		return fmt.Errorf("PGSSLKey gatherKeys unable to query: %v", err)
+	}
+	defer rows.Close()
+	i := 0
+	for rows.Next() {
+		if i > len(tbl.Records)-1 {
+			return errors.New("PGSSLKey gatherKeys: got more results than expected")
+		}
+		err := rows.Scan(&tbl.Records[i].DataEncrypted, &tbl.Records[i].DeliveryService, &tbl.Records[i].CDN, &tbl.Records[i].Version)
+		if err != nil {
+			return fmt.Errorf("PGSSLKey gatherKeys unable to scan row: %v", err)
+		}
+		i += 1
+	}
+	return nil
+}
+func (tbl *PGSSLKeyTable) decrypt(aesKey []byte) error {
+	for i, dns := range tbl.Records {
+		data, err := decryptInto(aesKey, dns.DataEncrypted, reflect.TypeOf(dns.Keys))
+		if err != nil {
+			return fmt.Errorf("unable to decrypt into keys: %v", err)
+		}
+		tbl.Records[i].Keys = *data.Interface().(*tc.DeliveryServiceSSLKeys)
+	}
+	return nil
+}
+func (tbl *PGSSLKeyTable) encrypt(aesKey []byte) error {
+	for i, dns := range tbl.Records {
+		data, err := json.Marshal(dns.Keys)
+		if err != nil {
+			return fmt.Errorf("encrypt issue marshalling keys: %v", err)
+		}
+		dat, err := encrypt(data, aesKey)
+		if err != nil {
+			return fmt.Errorf("encrypt error: %v", err)
+		}
+		tbl.Records[i].DataEncrypted = dat
+	}
+	return nil
+}
+func (tbl *PGSSLKeyTable) toGeneric() []SSLKey {
+	keys := make([]SSLKey, len(tbl.Records))
+
+	for i, record := range tbl.Records {
+		keys[i] = SSLKey{
+			DeliveryServiceSSLKeys: record.Keys,
+			CommonRecord:           record.CommonRecord,
+		}
+	}
+	return keys
+}
+func (tbl *PGSSLKeyTable) fromGeneric(keys []SSLKey) {
+	tbl.Records = make([]PGSSLKeyRecord, len(keys))
+
+	for i, key := range keys {
+		tbl.Records[i] = PGSSLKeyRecord{
+			Keys: key.DeliveryServiceSSLKeys,
+			PGCommonRecord: PGCommonRecord{
+				DataEncrypted: nil,
+				CommonRecord:  CommonRecord{},
+			},
+			DeliveryService: key.DeliveryService,
+			CDN:             key.CDN,
+		}
+	}
+}
+func (tbl *PGSSLKeyTable) validate() []string {
+	defaultKey := tc.DeliveryServiceSSLKeys{}
+	var errors []string
+	fmtStr := "SSL Key %v: %v"
+	for i, record := range tbl.Records {
+		if record.Keys == defaultKey {
+			errors = append(errors, fmt.Sprintf(fmtStr, i, "DS SSL Keys are default!"))
+		} else if record.Keys.Key == "" {
+			errors = append(errors, fmt.Sprintf(fmtStr, i, "Key is blank!"))
+		} else if record.Keys.CDN == "" {
+			errors = append(errors, fmt.Sprintf(fmtStr, i, "CDN is blank!"))
+		} else if record.Keys.DeliveryService == "" {
+			errors = append(errors, fmt.Sprintf(fmtStr, i, "DS is blank!"))
+		} else if record.DataEncrypted == nil {
+			errors = append(errors, fmt.Sprintf(fmtStr, i, "DataEncrypted is blank!"))
+		}
+	}
+	return errors
+}
+
+type PGURLSigKeyRecord struct {
+	Keys            tc.URLSigKeys
+	DeliveryService string
+	PGCommonRecord
+}
+type PGURLSigKeyTable struct {
+	Records []PGURLSigKeyRecord
+}
+
+func (tbl *PGURLSigKeyTable) insertKeys(db *sql.DB) error {
+	queryBase := "INSERT INTO url_sig_key (deliveryservice, data) VALUES "
+	stride := 2
+	queryArgs := make([]interface{}, len(tbl.Records)*stride)
+	for i, record := range tbl.Records {
+		j := i * stride
+		queryArgs[j] = record.DeliveryService
+		queryArgs[j+1] = record.DataEncrypted
+	}
+	return insertIntoTable(db, queryBase, stride, queryArgs)
+}
+func (tbl *PGURLSigKeyTable) gatherKeys(db *sql.DB) error {
+	sz, err := getSize(db, "url_sig_key")
+	if err != nil {
+		log.Println("PGURLSigKey gatherKeys: unable to determine url_sig_key table size")
+	}
+	tbl.Records = make([]PGURLSigKeyRecord, sz)
+
+	rows, err := db.Query("SELECT deliveryservice, data from url_sig_key")
+	if err != nil {
+		return fmt.Errorf("PGURLSigKey gatherKeys error creating query: %v", err)
+	}
+	defer rows.Close()
+	i := 0
+	for rows.Next() {
+		if i > len(tbl.Records)-1 {
+			return fmt.Errorf("PGURLSigKey gatherKeys: got more results than expected %v", len(tbl.Records))
+		}
+		err := rows.Scan(&tbl.Records[i].DeliveryService, &tbl.Records[i].DataEncrypted)
+		if err != nil {
+			return fmt.Errorf("PGURLSigKey gatherKeys: unable to scan row: %v", err)
+		}
+		i += 1
+	}
+	return nil
+}
+func (tbl *PGURLSigKeyTable) decrypt(aesKey []byte) error {
+	for i, sig := range tbl.Records {
+		data, err := decryptInto(aesKey, sig.DataEncrypted, reflect.TypeOf(sig.Keys))
+		if err != nil {
+			return fmt.Errorf("unable to decrypt into keys: %v", err)
+		}
+
+		tbl.Records[i].Keys = *data.Interface().(*tc.URLSigKeys)
+	}
+	return nil
+}
+func (tbl *PGURLSigKeyTable) encrypt(aesKey []byte) error {
+	for i, sig := range tbl.Records {
+		data, err := json.Marshal(&sig.Keys)
+		if err != nil {
+			return fmt.Errorf("encrypt issue marshalling keys: %v", err)

Review comment:
       same as above RE: `%w` vs `%v`

##########
File path: tools/traffic_vault_migrate/postgres.go
##########
@@ -0,0 +1,716 @@
+package main
+
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+import (
+	"database/sql"
+	"encoding/base64"
+	"encoding/json"
+	"errors"
+	"fmt"
+	"github.com/apache/trafficcontrol/lib/go-tc"
+	util "github.com/apache/trafficcontrol/lib/go-util"
+	_ "github.com/lib/pq"
+	"log"
+	"reflect"
+	"strconv"
+	"strings"
+)
+
+type PGConfig struct {
+	Host     string `json:"host"`
+	Port     string `json:"port"`
+	User     string `json:"user"`
+	Password string `json:"password"`
+	SSLMode  string `json:"sslmode"`
+	Database string `json:"database"`
+	Key      string `json:"aesKey"`
+	AESKey   []byte
+}
+type PGBackend struct {
+	sslKey PGSSLKeyTable
+	dnssec PGDNSSecTable
+	uri    PGURISignKeyTable
+	url    PGURLSigKeyTable
+	cfg    PGConfig
+	db     *sql.DB
+}
+
+func (pg *PGBackend) String() string {
+	data := fmt.Sprintf("PG server %v@%v:%v\n", pg.cfg.User, pg.cfg.Host, pg.cfg.Port)
+	data += fmt.Sprintf("\tSSL Keys: %v\n", len(pg.sslKey.Records))
+	data += fmt.Sprintf("\tDNSSec Keys: %v\n", len(pg.dnssec.Records))
+	data += fmt.Sprintf("\tURI Keys: %v\n", len(pg.uri.Records))
+	data += fmt.Sprintf("\tURL Keys: %v\n", len(pg.url.Records))
+	return data
+}
+func (pg *PGBackend) Name() string {
+	return "PG"
+}
+func (pg *PGBackend) ReadConfig(s string) error {
+	genericCfg, err := UnmarshalConfig(s, reflect.TypeOf(pg.cfg))
+	if err != nil {
+		return err
+	}
+
+	pg.cfg = *genericCfg.Interface().(*PGConfig)
+
+	pg.cfg.AESKey, err = base64.StdEncoding.DecodeString(pg.cfg.Key)
+	if err != nil {
+		return fmt.Errorf("unable to decode PG AESKey: %v", err)
+	}
+	return nil
+}
+func (pg *PGBackend) Insert() error {
+	if err := pg.sslKey.insertKeys(pg.db); err != nil {
+		return err
+	}
+	if err := pg.dnssec.insertKeys(pg.db); err != nil {
+		return err
+	}
+	if err := pg.url.insertKeys(pg.db); err != nil {
+		return err
+	}
+	if err := pg.uri.insertKeys(pg.db); err != nil {
+		return err
+	}
+	return nil
+}
+func (pg *PGBackend) Start() error {
+	sqlStr := fmt.Sprintf("postgres://%s:%s@%s:%s/%s?sslmode=%s", pg.cfg.User, pg.cfg.Password, pg.cfg.Host, pg.cfg.Port, pg.cfg.Database, pg.cfg.SSLMode)
+	db, err := sql.Open("postgres", sqlStr)
+	if err != nil {
+		return fmt.Errorf("Unable to start PG client: %v", err)
+	}
+
+	pg.db = db
+	pg.sslKey = PGSSLKeyTable{}
+	pg.dnssec = PGDNSSecTable{}
+	pg.url = PGURLSigKeyTable{}
+	pg.uri = PGURISignKeyTable{}
+
+	return nil
+}
+func (pg *PGBackend) ValidateKey() []string {
+	var errors []string
+	if errs := pg.sslKey.validate(); errs != nil {
+		errors = append(errors, errs...)
+	}
+	if errs := pg.dnssec.validate(); errs != nil {
+		errors = append(errors, errs...)
+	}
+	if errs := pg.uri.validate(); errs != nil {
+		errors = append(errors, errs...)
+	}
+	if errs := pg.url.validate(); errs != nil {
+		errors = append(errors, errs...)
+	}
+	return errors
+}
+func (pg *PGBackend) Stop() error {
+	return pg.db.Close()
+}
+func (pg *PGBackend) Ping() error {
+	return pg.db.Ping()
+}
+func (pg *PGBackend) Fetch() error {
+	if err := pg.sslKey.gatherKeys(pg.db); err != nil {
+		return err
+	}
+
+	if err := pg.dnssec.gatherKeys(pg.db); err != nil {
+		return err
+	}
+
+	if err := pg.url.gatherKeys(pg.db); err != nil {
+		return err
+	}
+
+	if err := pg.uri.gatherKeys(pg.db); err != nil {
+		return err
+	}
+
+	return nil
+}
+func (pg *PGBackend) GetSSLKeys() ([]SSLKey, error) {
+	if err := pg.sslKey.decrypt(pg.cfg.AESKey); err != nil {
+		return nil, err
+	}
+	return pg.sslKey.toGeneric(), nil
+}
+func (pg *PGBackend) SetSSLKeys(keys []SSLKey) error {
+	pg.sslKey.fromGeneric(keys)
+	return pg.sslKey.encrypt(pg.cfg.AESKey)
+}
+func (pg *PGBackend) GetDNSSecKeys() ([]DNSSecKey, error) {
+	if err := pg.dnssec.decrypt(pg.cfg.AESKey); err != nil {
+		return nil, err
+	}
+	return pg.dnssec.toGeneric(), nil
+}
+func (pg *PGBackend) SetDNSSecKeys(keys []DNSSecKey) error {
+	pg.dnssec.fromGeneric(keys)
+	return pg.dnssec.encrypt(pg.cfg.AESKey)
+}
+func (pg *PGBackend) GetURISignKeys() ([]URISignKey, error) {
+	if err := pg.uri.decrypt(pg.cfg.AESKey); err != nil {
+		return nil, err
+	}
+	return pg.uri.toGeneric(), nil
+}
+func (pg *PGBackend) SetURISignKeys(keys []URISignKey) error {
+	pg.uri.fromGeneric(keys)
+	return pg.uri.encrypt(pg.cfg.AESKey)
+}
+func (pg *PGBackend) GetURLSigKeys() ([]URLSigKey, error) {
+	if err := pg.url.decrypt(pg.cfg.AESKey); err != nil {
+		return nil, err
+	}
+	return pg.url.toGeneric(), nil
+}
+func (pg *PGBackend) SetURLSigKeys(keys []URLSigKey) error {
+	pg.url.fromGeneric(keys)
+	return pg.url.encrypt(pg.cfg.AESKey)
+}
+
+type PGCommonRecord struct {
+	DataEncrypted []byte
+	CommonRecord
+}
+
+type PGDNSSecRecord struct {
+	Key tc.DNSSECKeysTrafficVault
+	CDN string
+	PGCommonRecord
+}
+type PGDNSSecTable struct {
+	Records []PGDNSSecRecord
+}
+
+func (tbl *PGDNSSecTable) gatherKeys(db *sql.DB) error {
+	sz, err := getSize(db, "dnssec")
+	if err != nil {
+		log.Println("PGDNSSec gatherKeys: unable to determine size of dnssec table")
+	}
+	tbl.Records = make([]PGDNSSecRecord, sz)
+
+	rows, err := db.Query("SELECT cdn, data from dnssec")
+	if err != nil {
+		return fmt.Errorf("PGDNSSec gatherKeys: unable to query: %v", err)
+	}
+	defer rows.Close()
+	i := 0
+	for rows.Next() {
+		if i > len(tbl.Records)-1 {
+			return fmt.Errorf("PGDNSSec gatherKeys got more results than expected %v", len(tbl.Records))
+		}
+		err := rows.Scan(&tbl.Records[i].CDN, &tbl.Records[i].DataEncrypted)
+		if err != nil {
+			return fmt.Errorf("PGDNSSec gatherKeys unable to scan row: %v", err)
+		}
+		i += 1
+	}
+	return nil
+}
+func (tbl *PGDNSSecTable) decrypt(aesKey []byte) error {
+	for i, _ := range tbl.Records {
+		data, err := decryptInto(aesKey, tbl.Records[i].DataEncrypted, reflect.TypeOf(tbl.Records[i].Key))
+		if err != nil {
+			return fmt.Errorf("unable to decrypt into keys: %v", err)
+		}
+		tbl.Records[i].Key = *data.Interface().(*tc.DNSSECKeysTrafficVault)
+	}
+	return nil
+}
+func (tbl *PGDNSSecTable) encrypt(aesKey []byte) error {
+	for i, dns := range tbl.Records {
+		data, err := json.Marshal(&dns.Key)
+		if err != nil {
+			return fmt.Errorf("encrypt issue marshalling keys: %v", err)
+		}
+		dat, err := encrypt(data, aesKey)
+		if err != nil {
+			return fmt.Errorf("encrypt error: %v", err)
+		}
+		tbl.Records[i].DataEncrypted = dat
+	}
+	return nil
+}
+func (tbl *PGDNSSecTable) toGeneric() []DNSSecKey {
+	keys := make([]DNSSecKey, len(tbl.Records))
+
+	for i, record := range tbl.Records {
+		keys[i] = DNSSecKey{
+			CDN:                    record.CDN,
+			DNSSECKeysTrafficVault: record.Key,
+			CommonRecord:           record.CommonRecord,
+		}
+	}
+
+	return keys
+}
+func (tbl *PGDNSSecTable) fromGeneric(keys []DNSSecKey) {
+	tbl.Records = make([]PGDNSSecRecord, len(keys))
+
+	for i, key := range keys {
+		tbl.Records[i] = PGDNSSecRecord{
+			Key: key.DNSSECKeysTrafficVault,
+			CDN: key.CDN,
+			PGCommonRecord: PGCommonRecord{
+				DataEncrypted: nil,
+				CommonRecord:  CommonRecord{},
+			},
+		}
+	}
+}
+func (tbl *PGDNSSecTable) validate() []string {
+	for i, record := range tbl.Records {
+		if record.DataEncrypted == nil && len(record.Key) > 0 {
+			return []string{fmt.Sprintf("DNSSEC Key %v: DataEncrypted is blank!", i)}
+		}
+	}
+	return nil
+}
+func (tbl *PGDNSSecTable) insertKeys(db *sql.DB) error {
+	queryFmt := "INSERT INTO dnssec (cdn, data) VALUES "
+	stride := 2
+	queryArgs := make([]interface{}, len(tbl.Records)*stride)
+	for i, record := range tbl.Records {
+		j := i * stride
+		queryArgs[j] = record.CDN
+		queryArgs[j+1] = record.DataEncrypted
+	}
+	return insertIntoTable(db, queryFmt, stride, queryArgs)
+}
+
+type PGSSLKeyRecord struct {
+	Keys tc.DeliveryServiceSSLKeys
+	PGCommonRecord
+
+	// These records are stored on the table but are duplicated
+	DeliveryService string
+	Version         string
+	CDN             string
+}
+type PGSSLKeyTable struct {
+	Records []PGSSLKeyRecord
+}
+
+func (tbl *PGSSLKeyTable) insertKeys(db *sql.DB) error {
+	queryFmt := "INSERT INTO sslkey (deliveryservice, data, cdn, version) VALUES "
+	stride := 4
+	queryArgs := make([]interface{}, len(tbl.Records)*stride)
+	for i, record := range tbl.Records {
+		j := i * stride
+		queryArgs[j] = record.DeliveryService
+		queryArgs[j+1] = record.DataEncrypted
+		queryArgs[j+2] = record.CDN
+		queryArgs[j+3] = record.Version
+	}
+	return insertIntoTable(db, queryFmt, 4, queryArgs)
+}
+func (tbl *PGSSLKeyTable) gatherKeys(db *sql.DB) error {
+	sz, err := getSize(db, "sslkey")
+	if err != nil {
+		return fmt.Errorf("PGSSLKey gatherKeys unable to determine size of sslkey table: %v", err)
+	}
+	tbl.Records = make([]PGSSLKeyRecord, sz)
+
+	rows, err := db.Query("SELECT data, deliveryservice, cdn, version from sslkey")
+	if err != nil {
+		return fmt.Errorf("PGSSLKey gatherKeys unable to query: %v", err)
+	}
+	defer rows.Close()
+	i := 0
+	for rows.Next() {
+		if i > len(tbl.Records)-1 {
+			return errors.New("PGSSLKey gatherKeys: got more results than expected")
+		}
+		err := rows.Scan(&tbl.Records[i].DataEncrypted, &tbl.Records[i].DeliveryService, &tbl.Records[i].CDN, &tbl.Records[i].Version)
+		if err != nil {
+			return fmt.Errorf("PGSSLKey gatherKeys unable to scan row: %v", err)
+		}
+		i += 1
+	}
+	return nil
+}
+func (tbl *PGSSLKeyTable) decrypt(aesKey []byte) error {
+	for i, dns := range tbl.Records {
+		data, err := decryptInto(aesKey, dns.DataEncrypted, reflect.TypeOf(dns.Keys))
+		if err != nil {
+			return fmt.Errorf("unable to decrypt into keys: %v", err)
+		}
+		tbl.Records[i].Keys = *data.Interface().(*tc.DeliveryServiceSSLKeys)
+	}
+	return nil
+}
+func (tbl *PGSSLKeyTable) encrypt(aesKey []byte) error {
+	for i, dns := range tbl.Records {
+		data, err := json.Marshal(dns.Keys)
+		if err != nil {
+			return fmt.Errorf("encrypt issue marshalling keys: %v", err)
+		}
+		dat, err := encrypt(data, aesKey)
+		if err != nil {
+			return fmt.Errorf("encrypt error: %v", err)
+		}
+		tbl.Records[i].DataEncrypted = dat
+	}
+	return nil
+}
+func (tbl *PGSSLKeyTable) toGeneric() []SSLKey {
+	keys := make([]SSLKey, len(tbl.Records))
+
+	for i, record := range tbl.Records {
+		keys[i] = SSLKey{
+			DeliveryServiceSSLKeys: record.Keys,
+			CommonRecord:           record.CommonRecord,
+		}
+	}
+	return keys
+}
+func (tbl *PGSSLKeyTable) fromGeneric(keys []SSLKey) {
+	tbl.Records = make([]PGSSLKeyRecord, len(keys))
+
+	for i, key := range keys {
+		tbl.Records[i] = PGSSLKeyRecord{
+			Keys: key.DeliveryServiceSSLKeys,
+			PGCommonRecord: PGCommonRecord{
+				DataEncrypted: nil,
+				CommonRecord:  CommonRecord{},
+			},
+			DeliveryService: key.DeliveryService,
+			CDN:             key.CDN,
+		}
+	}
+}
+func (tbl *PGSSLKeyTable) validate() []string {
+	defaultKey := tc.DeliveryServiceSSLKeys{}
+	var errors []string
+	fmtStr := "SSL Key %v: %v"
+	for i, record := range tbl.Records {
+		if record.Keys == defaultKey {
+			errors = append(errors, fmt.Sprintf(fmtStr, i, "DS SSL Keys are default!"))
+		} else if record.Keys.Key == "" {
+			errors = append(errors, fmt.Sprintf(fmtStr, i, "Key is blank!"))
+		} else if record.Keys.CDN == "" {
+			errors = append(errors, fmt.Sprintf(fmtStr, i, "CDN is blank!"))
+		} else if record.Keys.DeliveryService == "" {
+			errors = append(errors, fmt.Sprintf(fmtStr, i, "DS is blank!"))
+		} else if record.DataEncrypted == nil {
+			errors = append(errors, fmt.Sprintf(fmtStr, i, "DataEncrypted is blank!"))
+		}
+	}
+	return errors
+}
+
+type PGURLSigKeyRecord struct {
+	Keys            tc.URLSigKeys
+	DeliveryService string
+	PGCommonRecord
+}
+type PGURLSigKeyTable struct {
+	Records []PGURLSigKeyRecord
+}
+
+func (tbl *PGURLSigKeyTable) insertKeys(db *sql.DB) error {
+	queryBase := "INSERT INTO url_sig_key (deliveryservice, data) VALUES "
+	stride := 2
+	queryArgs := make([]interface{}, len(tbl.Records)*stride)
+	for i, record := range tbl.Records {
+		j := i * stride
+		queryArgs[j] = record.DeliveryService
+		queryArgs[j+1] = record.DataEncrypted
+	}
+	return insertIntoTable(db, queryBase, stride, queryArgs)
+}
+func (tbl *PGURLSigKeyTable) gatherKeys(db *sql.DB) error {
+	sz, err := getSize(db, "url_sig_key")
+	if err != nil {
+		log.Println("PGURLSigKey gatherKeys: unable to determine url_sig_key table size")
+	}
+	tbl.Records = make([]PGURLSigKeyRecord, sz)
+
+	rows, err := db.Query("SELECT deliveryservice, data from url_sig_key")
+	if err != nil {
+		return fmt.Errorf("PGURLSigKey gatherKeys error creating query: %v", err)
+	}
+	defer rows.Close()
+	i := 0
+	for rows.Next() {
+		if i > len(tbl.Records)-1 {
+			return fmt.Errorf("PGURLSigKey gatherKeys: got more results than expected %v", len(tbl.Records))
+		}
+		err := rows.Scan(&tbl.Records[i].DeliveryService, &tbl.Records[i].DataEncrypted)
+		if err != nil {
+			return fmt.Errorf("PGURLSigKey gatherKeys: unable to scan row: %v", err)
+		}
+		i += 1
+	}
+	return nil
+}
+func (tbl *PGURLSigKeyTable) decrypt(aesKey []byte) error {
+	for i, sig := range tbl.Records {
+		data, err := decryptInto(aesKey, sig.DataEncrypted, reflect.TypeOf(sig.Keys))
+		if err != nil {
+			return fmt.Errorf("unable to decrypt into keys: %v", err)
+		}
+
+		tbl.Records[i].Keys = *data.Interface().(*tc.URLSigKeys)
+	}
+	return nil
+}
+func (tbl *PGURLSigKeyTable) encrypt(aesKey []byte) error {
+	for i, sig := range tbl.Records {
+		data, err := json.Marshal(&sig.Keys)
+		if err != nil {
+			return fmt.Errorf("encrypt issue marshalling keys: %v", err)
+		}
+
+		dat, err := encrypt(data, aesKey)
+		if err != nil {
+			return fmt.Errorf("encrypt error: %v", err)

Review comment:
       same as above RE: `%w` vs `%v`

##########
File path: tools/traffic_vault_migrate/traffic_vault_migrate.go
##########
@@ -0,0 +1,453 @@
+package main
+
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+import (
+	"encoding/json"
+	"errors"
+	"flag"
+	"fmt"
+	"github.com/apache/trafficcontrol/lib/go-tc"
+	"io/ioutil"
+	"log"
+	"os"
+	"reflect"
+	"sort"
+	"strings"
+	"time"
+)
+
+var (
+	fromType string
+	toType   string
+	fromCfg  string
+	toCfg    string
+	dry      bool
+	compare  bool
+	confirm  bool
+	dump     bool
+
+	riakBE RiakBackend = RiakBackend{}
+	pgBE   PGBackend   = PGBackend{}
+)
+
+func init() {
+	flag.StringVar(&fromType, "from_type", riakBE.Name(), fmt.Sprintf("From server types (%v)", strings.Join(supportedTypes(), "|")))
+	flag.StringVar(&toType, "to_type", pgBE.Name(), fmt.Sprintf("From server types (%v)", strings.Join(supportedTypes(), "|")))
+	flag.StringVar(&toCfg, "to_cfg", "pg.json", "To server config file")
+	flag.StringVar(&fromCfg, "from_cfg", "riak.json", "From server config file")
+	flag.BoolVar(&dry, "dry", false, "Do not perform writes")
+	flag.BoolVar(&compare, "compare", false, "Compare to and from server records")
+	flag.BoolVar(&confirm, "confirm", true, "Requires confirmation before inserting records")
+	flag.BoolVar(&dump, "dump", false, "Write keys (from 'from' server) to disk")
+}
+
+func supportedBackends() []TVBackend {
+	return []TVBackend{
+		&riakBE, &pgBE,
+	}
+}
+
+func main() {
+	flag.Parse()
+	var fromSrv TVBackend
+	var toSrv TVBackend
+
+	toSrvUsed := !dump && !dry
+
+	if !validateType(fromType) {
+		log.Fatal("Unknown fromType " + fromType)
+	}
+	if toSrvUsed && !validateType(toType) {
+		log.Fatal("Unknown toType " + toType)
+	}
+
+	fromSrv = getBackendFromType(fromType)
+	if toSrvUsed {
+		toSrv = getBackendFromType(toType)
+	}
+
+	var toTimer time.Time
+	var toTime float64
+	var fromTimer time.Time
+	var fromTime float64
+
+	log.Println("Reading configs...")
+	fromTimer = time.Now()
+	if err := fromSrv.ReadConfig(fromCfg); err != nil {
+		log.Fatalf("Unable to read fromSrv cfg: %v", err)
+	}
+	fromTime = time.Now().Sub(fromTimer).Seconds()
+
+	if toSrvUsed {
+		toTimer = time.Now()
+		if err := toSrv.ReadConfig(toCfg); err != nil {
+			log.Fatalf("Unable to read toSrv cfg: %v", err)
+		}
+		toTime := time.Now().Sub(toTimer).Seconds()
+		log.Printf("Done [%v seconds]\n\tto: [%v seconds]\n\tfrom: [%v seconds]\n", toTime+fromTime, toTime, fromTime)
+	} else {
+		log.Printf("Done [%v seconds]\n", fromTime)
+	}
+
+	log.Println("Starting servers...")
+	fromTimer = time.Now()
+	if err := fromSrv.Start(); err != nil {
+		log.Fatalf("issue starting fromSrv: %v", err)
+	}
+	fromTime = time.Now().Sub(fromTimer).Seconds()
+	defer func() {
+		fromSrv.Stop()
+	}()
+	if toSrvUsed {
+		toTimer = time.Now()
+		if err := toSrv.Start(); err != nil {
+			log.Fatalf("issue starting toSrv: %v", err)
+		}
+		toTime = time.Now().Sub(toTimer).Seconds()
+		defer func() {
+			toSrv.Stop()
+		}()
+		log.Printf("Done [%v seconds]\n\tto: [%v seconds]\n\tfrom: [%v seconds]\n", toTime+fromTime, toTime, fromTime)
+	} else {
+		log.Printf("Done [%v seconds]\n", fromTime)
+	}
+
+	log.Println("Pinging servers...")
+	fromTimer = time.Now()
+	if err := fromSrv.Ping(); err != nil {
+		log.Fatalf("Unable to ping fromSrv: %v", err)
+	}
+	fromTime = time.Now().Sub(fromTimer).Seconds()
+	if toSrvUsed {
+		toTimer = time.Now()
+		if err := toSrv.Ping(); err != nil {
+			log.Fatalf("Unable to ping toSrv: %v", err)
+		}
+		toTime = time.Now().Sub(toTimer).Seconds()
+		log.Printf("Done [%v seconds]\n\tto: [%v seconds]\n\tfrom: [%v seconds]\n", toTime+fromTime, toTime, fromTime)
+	} else {
+		log.Printf("Done [%v seconds]\n", fromTime)
+	}
+
+	log.Printf("Fetching data from %v...\n", fromSrv.Name())
+	fromTimer = time.Now()
+	if err := fromSrv.Fetch(); err != nil {
+		log.Fatalf("Unable to fetch fromSrv data: %v", err)
+	}
+
+	fromSSLKeys, fromDNSSecKeys, fromURIKeys, fromURLKeys, err := GetKeys(fromSrv)
+	if err != nil {
+		log.Fatal(err)
+	}
+
+	if err := Validate(fromSrv); err != nil {
+		log.Fatal(err)
+	}
+	log.Printf("Done [%v seconds]\n", time.Now().Sub(fromTimer).Seconds())
+
+	if dump {
+		log.Printf("Dumping data from %v...\n", fromSrv.Name())
+		fromTimer = time.Now()
+		sslKeysBytes, err := json.Marshal(&fromSSLKeys)
+		if err != nil {
+			log.Fatal(err)
+		}
+		dnssecKeyBytes, err := json.Marshal(&fromDNSSecKeys)
+		if err != nil {
+			log.Fatal(err)
+		}
+		uriKeyBytes, err := json.Marshal(&fromURIKeys)
+		if err != nil {
+			log.Fatal(err)
+		}
+		urlKeyBytes, err := json.Marshal(&fromURLKeys)
+		if err != nil {
+			log.Fatal(err)
+		}
+
+		if err := os.Mkdir("dump", 0750); err != nil {
+			if !os.IsExist(err) {
+				log.Fatal(err)
+			}
+		}
+		if err = ioutil.WriteFile("dump/sslkeys.json", sslKeysBytes, 0640); err != nil {
+			log.Println(err)
+		}
+		if err = ioutil.WriteFile("dump/dnsseckeys.json", dnssecKeyBytes, 0640); err != nil {
+			log.Println(err)
+		}
+		if err = ioutil.WriteFile("dump/urlkeys.json", urlKeyBytes, 0640); err != nil {
+			log.Println(err)
+		}
+		if err = ioutil.WriteFile("dump/urikeys.json", uriKeyBytes, 0640); err != nil {
+			log.Println(err)
+		}
+		log.Printf("Done [%v seconds]\n", time.Now().Sub(fromTimer).Seconds())
+		return
+	}
+
+	if compare {
+		log.Printf("Fetching data from %v...\n", toSrv.Name())
+		toTimer = time.Now()
+		if err := toSrv.Fetch(); err != nil {
+			log.Fatalf("Unable to fetch toSrv data: %v\n", err)
+		}
+
+		toSSLKeys, toDNSSecKeys, toURIKeys, toURLKeys, err := GetKeys(toSrv)
+		if err != nil {
+			log.Fatal(err)
+		}
+		log.Println("Validating " + toSrv.Name())
+		if err := toSrv.ValidateKey(); err != nil && len(err) > 0 {
+			log.Fatal(strings.Join(err, "\n"))
+		}
+		log.Printf("Done [%v seconds]\n", time.Now().Sub(toTimer).Seconds())
+		log.Println(fromSrv.String())
+		log.Println(toSrv.String())
+
+		if !reflect.DeepEqual(fromSSLKeys, toSSLKeys) {
+			log.Fatal("from sslkeys and to sslkeys don't match")
+		}
+		if !reflect.DeepEqual(fromDNSSecKeys, toDNSSecKeys) {
+			log.Fatal("from dnssec and to dnssec don't match")
+		}
+		if !reflect.DeepEqual(fromURIKeys, toURIKeys) {
+			log.Fatal("from uri and to uri don't match")
+		}
+		if !reflect.DeepEqual(fromURLKeys, toURLKeys) {
+			log.Fatal("from url and to url don't match")
+		}
+		log.Println("Both datasources have the same keys!")
+		return
+	}
+
+	log.Printf("Setting %v keys...\n", toSrv.Name())
+	toTimer = time.Now()
+	if err := SetKeys(toSrv, fromSSLKeys, fromDNSSecKeys, fromURIKeys, fromURLKeys); err != nil {
+		log.Fatal(err)
+	}
+
+	if err := Validate(toSrv); err != nil {
+		log.Fatal(err)
+	}
+	log.Printf("Done [%v seconds]\n", time.Now().Sub(toTimer).Seconds())
+
+	log.Println(fromSrv.String())
+
+	if dry {
+		return
+	}
+
+	if confirm {
+		ans := "q"
+		for {
+			fmt.Print("Confirm data insertion (y/n): ")
+			if _, err := fmt.Scanln(&ans); err != nil {
+				log.Fatal("unable to get user input")
+			}
+
+			if ans == "y" {
+				break
+			} else if ans == "n" {
+				return
+			}
+		}
+	}
+	log.Printf("Inserting data into %v...\n", toSrv.Name())
+	toTimer = time.Now()
+	if err := toSrv.Insert(); err != nil {
+		log.Fatal(err)
+	}
+	log.Printf("Done [%v seconds]\n", time.Now().Sub(toTimer).Seconds())
+
+}
+
+// Validate runs the ValidateKey method on the backend
+func Validate(be TVBackend) error {
+	if errs := be.ValidateKey(); errs != nil && len(errs) > 0 {
+		return errors.New(fmt.Sprintf("Validation Errors (%v): \n%v", be.Name(), strings.Join(errs, "\n")))
+	} else {
+		log.Println("Validated " + be.Name())
+	}
+	return nil
+}
+
+// SetKeys will set all of the keys for a backend
+func SetKeys(be TVBackend, sslkeys []SSLKey, dnssecKeys []DNSSecKey, uriKeys []URISignKey, urlKeys []URLSigKey) error {
+	if err := be.SetSSLKeys(sslkeys); err != nil {
+		return errors.New(fmt.Sprintf("Unable to set %v ssl keys: %v", be.Name(), err))
+	}
+	if err := be.SetDNSSecKeys(dnssecKeys); err != nil {
+		return errors.New(fmt.Sprintf("Unable to set %v dnssec keys: %v", be.Name(), err))
+	}
+	if err := be.SetURLSigKeys(urlKeys); err != nil {
+		return errors.New(fmt.Sprintf("Unable to set %v url keys: %v", be.Name(), err))
+	}
+	if err := be.SetURISignKeys(uriKeys); err != nil {
+		return errors.New(fmt.Sprintf("Unable to set %v uri keys: %v", be.Name(), err))
+	}
+	return nil
+}
+
+// GetKeys will get all of the keys for a backend
+func GetKeys(be TVBackend) ([]SSLKey, []DNSSecKey, []URISignKey, []URLSigKey, error) {
+	var sslkeys []SSLKey
+	var dnssec []DNSSecKey
+	var uri []URISignKey
+	var url []URLSigKey
+	var err error
+	if sslkeys, err = be.GetSSLKeys(); err != nil {
+		return nil, nil, nil, nil, errors.New(fmt.Sprintf("Unable to get %v sslkeys: %v", be.Name(), err))
+	}
+	if dnssec, err = be.GetDNSSecKeys(); err != nil {
+		return nil, nil, nil, nil, errors.New(fmt.Sprintf("Unable to get %v dnssec keys: %v", be.Name(), err))
+	}
+	if uri, err = be.GetURISignKeys(); err != nil {
+		return nil, nil, nil, nil, errors.New(fmt.Sprintf("Unable to get %v uri keys: %v", be.Name(), err))
+	}
+	if url, err = be.GetURLSigKeys(); err != nil {
+		return nil, nil, nil, nil, errors.New(fmt.Sprintf("Unable to %v url keys: %v", be.Name(), err))
+	}
+	sort.Slice(sslkeys[:], func(a, b int) bool {
+		return sslkeys[a].CDN < sslkeys[b].CDN && sslkeys[a].DeliveryService < sslkeys[b].DeliveryService && sslkeys[a].Version < sslkeys[b].Version
+	})
+	sort.Slice(dnssec[:], func(a, b int) bool {
+		return dnssec[a].CDN < dnssec[b].CDN
+	})
+	sort.Slice(uri[:], func(a, b int) bool {
+		return uri[a].DeliveryService < uri[b].DeliveryService
+	})
+	sort.Slice(url[:], func(a, b int) bool {
+		return url[a].DeliveryService < url[b].DeliveryService
+	})
+	return sslkeys, dnssec, uri, url, nil
+}
+
+// UnmarshalConfig takes in a config file and a type and will read the config file into the reflected type
+func UnmarshalConfig(configFile string, t reflect.Type) (reflect.Value, error) {
+	data, err := ioutil.ReadFile(configFile)
+	if err != nil {
+		return reflect.Value{}, err
+	}
+	val := reflect.New(t)
+	err = json.Unmarshal(data, val.Interface())
+	if err != nil {
+		return reflect.Value{}, err
+	}
+
+	return val, nil
+}
+
+// TVBackend represents a TV backend that can be have data migrated to/from
+type TVBackend interface {
+	// Start initiates the connection to the backend DB
+	Start() error
+	// Stop terminates the connection to the backend DB
+	Stop() error
+	// Ping checks the connection to the backend DB
+	Ping() error
+	// ValidateKey validates that the keys are valid (in most cases, certain fields are not null)
+	ValidateKey() []string
+	// Name returns the name for this backend
+	Name() string
+	// ReadConfig takes in a filename and will read it into the backends config
+	ReadConfig(string) error
+	// String returns a high level overview of the backend and its keys
+	String() string
+
+	// Fetch gets all of the keys from the backend DB
+	Fetch() error
+	// Insert takes the current keys and inserts them into the backend DB
+	Insert() error
+
+	// GetSSLKeys converts the backends internal key representation into the common representation (SSLKey)
+	GetSSLKeys() ([]SSLKey, error)
+	// SetSSLKeys takes in keys and converts & encrypts the data into the backends internal format
+	SetSSLKeys([]SSLKey) error
+
+	// GetDNSSecKeys converts the backends internal key representation into the common representation (DNSSecKey)
+	GetDNSSecKeys() ([]DNSSecKey, error)
+	// SetDNSSecKeys takes in keys and converts & encrypts the data into the backends internal format
+	SetDNSSecKeys([]DNSSecKey) error
+
+	// GetURISignKeys converts the pg internal key representation into the common representation (URISignKey)
+	GetURISignKeys() ([]URISignKey, error)
+	// SetURISignKeys takes in keys and converts & encrypts the data into the backends internal format
+	SetURISignKeys([]URISignKey) error
+
+	// GetURLSigKeys converts the backends internal key representation into the common representation (URLSigKey)
+	GetURLSigKeys() ([]URLSigKey, error)
+	// SetURLSigKeys takes in keys and converts & encrypts the data into the backends internal format
+	SetURLSigKeys([]URLSigKey) error
+}
+
+type CommonRecord struct{}

Review comment:
       What's the point of this type?

##########
File path: traffic_ops/traffic_ops_golang/trafficvault/backends/postgres/url_sig_keys.go
##########
@@ -24,6 +24,7 @@ import (
 	"database/sql"
 	"encoding/json"
 	"errors"
+	"github.com/apache/trafficcontrol/lib/go-util"

Review comment:
       this should be grouped below with the other `github.com/apache/trafficcontrol/` import

##########
File path: traffic_ops/traffic_ops_golang/trafficvault/backends/riaksvc/dsutil.go
##########
@@ -23,6 +23,7 @@ import (
 	"database/sql"
 	"encoding/json"
 	"errors"
+	"github.com/apache/trafficcontrol/lib/go-rfc"

Review comment:
       This should be grouped below with the other `github.com/apache/trafficcontrol/` import

##########
File path: tools/traffic_vault_migrate/postgres.go
##########
@@ -0,0 +1,716 @@
+package main
+
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+import (
+	"database/sql"
+	"encoding/base64"
+	"encoding/json"
+	"errors"
+	"fmt"
+	"github.com/apache/trafficcontrol/lib/go-tc"
+	util "github.com/apache/trafficcontrol/lib/go-util"
+	_ "github.com/lib/pq"
+	"log"
+	"reflect"
+	"strconv"
+	"strings"
+)
+
+type PGConfig struct {
+	Host     string `json:"host"`
+	Port     string `json:"port"`
+	User     string `json:"user"`
+	Password string `json:"password"`
+	SSLMode  string `json:"sslmode"`
+	Database string `json:"database"`
+	Key      string `json:"aesKey"`
+	AESKey   []byte
+}
+type PGBackend struct {
+	sslKey PGSSLKeyTable
+	dnssec PGDNSSecTable
+	uri    PGURISignKeyTable
+	url    PGURLSigKeyTable
+	cfg    PGConfig
+	db     *sql.DB
+}
+
+func (pg *PGBackend) String() string {
+	data := fmt.Sprintf("PG server %v@%v:%v\n", pg.cfg.User, pg.cfg.Host, pg.cfg.Port)
+	data += fmt.Sprintf("\tSSL Keys: %v\n", len(pg.sslKey.Records))
+	data += fmt.Sprintf("\tDNSSec Keys: %v\n", len(pg.dnssec.Records))
+	data += fmt.Sprintf("\tURI Keys: %v\n", len(pg.uri.Records))
+	data += fmt.Sprintf("\tURL Keys: %v\n", len(pg.url.Records))
+	return data
+}
+func (pg *PGBackend) Name() string {
+	return "PG"
+}
+func (pg *PGBackend) ReadConfig(s string) error {
+	genericCfg, err := UnmarshalConfig(s, reflect.TypeOf(pg.cfg))
+	if err != nil {
+		return err
+	}
+
+	pg.cfg = *genericCfg.Interface().(*PGConfig)
+
+	pg.cfg.AESKey, err = base64.StdEncoding.DecodeString(pg.cfg.Key)
+	if err != nil {
+		return fmt.Errorf("unable to decode PG AESKey: %v", err)
+	}
+	return nil
+}
+func (pg *PGBackend) Insert() error {
+	if err := pg.sslKey.insertKeys(pg.db); err != nil {
+		return err
+	}
+	if err := pg.dnssec.insertKeys(pg.db); err != nil {
+		return err
+	}
+	if err := pg.url.insertKeys(pg.db); err != nil {
+		return err
+	}
+	if err := pg.uri.insertKeys(pg.db); err != nil {
+		return err
+	}
+	return nil
+}
+func (pg *PGBackend) Start() error {
+	sqlStr := fmt.Sprintf("postgres://%s:%s@%s:%s/%s?sslmode=%s", pg.cfg.User, pg.cfg.Password, pg.cfg.Host, pg.cfg.Port, pg.cfg.Database, pg.cfg.SSLMode)
+	db, err := sql.Open("postgres", sqlStr)
+	if err != nil {
+		return fmt.Errorf("Unable to start PG client: %v", err)
+	}
+
+	pg.db = db
+	pg.sslKey = PGSSLKeyTable{}
+	pg.dnssec = PGDNSSecTable{}
+	pg.url = PGURLSigKeyTable{}
+	pg.uri = PGURISignKeyTable{}
+
+	return nil
+}
+func (pg *PGBackend) ValidateKey() []string {
+	var errors []string
+	if errs := pg.sslKey.validate(); errs != nil {
+		errors = append(errors, errs...)
+	}
+	if errs := pg.dnssec.validate(); errs != nil {
+		errors = append(errors, errs...)
+	}
+	if errs := pg.uri.validate(); errs != nil {
+		errors = append(errors, errs...)
+	}
+	if errs := pg.url.validate(); errs != nil {
+		errors = append(errors, errs...)
+	}
+	return errors
+}
+func (pg *PGBackend) Stop() error {
+	return pg.db.Close()
+}
+func (pg *PGBackend) Ping() error {
+	return pg.db.Ping()
+}
+func (pg *PGBackend) Fetch() error {
+	if err := pg.sslKey.gatherKeys(pg.db); err != nil {
+		return err
+	}
+
+	if err := pg.dnssec.gatherKeys(pg.db); err != nil {
+		return err
+	}
+
+	if err := pg.url.gatherKeys(pg.db); err != nil {
+		return err
+	}
+
+	if err := pg.uri.gatherKeys(pg.db); err != nil {
+		return err
+	}
+
+	return nil
+}
+func (pg *PGBackend) GetSSLKeys() ([]SSLKey, error) {
+	if err := pg.sslKey.decrypt(pg.cfg.AESKey); err != nil {
+		return nil, err
+	}
+	return pg.sslKey.toGeneric(), nil
+}
+func (pg *PGBackend) SetSSLKeys(keys []SSLKey) error {
+	pg.sslKey.fromGeneric(keys)
+	return pg.sslKey.encrypt(pg.cfg.AESKey)
+}
+func (pg *PGBackend) GetDNSSecKeys() ([]DNSSecKey, error) {
+	if err := pg.dnssec.decrypt(pg.cfg.AESKey); err != nil {
+		return nil, err
+	}
+	return pg.dnssec.toGeneric(), nil
+}
+func (pg *PGBackend) SetDNSSecKeys(keys []DNSSecKey) error {
+	pg.dnssec.fromGeneric(keys)
+	return pg.dnssec.encrypt(pg.cfg.AESKey)
+}
+func (pg *PGBackend) GetURISignKeys() ([]URISignKey, error) {
+	if err := pg.uri.decrypt(pg.cfg.AESKey); err != nil {
+		return nil, err
+	}
+	return pg.uri.toGeneric(), nil
+}
+func (pg *PGBackend) SetURISignKeys(keys []URISignKey) error {
+	pg.uri.fromGeneric(keys)
+	return pg.uri.encrypt(pg.cfg.AESKey)
+}
+func (pg *PGBackend) GetURLSigKeys() ([]URLSigKey, error) {
+	if err := pg.url.decrypt(pg.cfg.AESKey); err != nil {
+		return nil, err
+	}
+	return pg.url.toGeneric(), nil
+}
+func (pg *PGBackend) SetURLSigKeys(keys []URLSigKey) error {
+	pg.url.fromGeneric(keys)
+	return pg.url.encrypt(pg.cfg.AESKey)
+}
+
+type PGCommonRecord struct {
+	DataEncrypted []byte
+	CommonRecord
+}
+
+type PGDNSSecRecord struct {
+	Key tc.DNSSECKeysTrafficVault
+	CDN string
+	PGCommonRecord
+}
+type PGDNSSecTable struct {
+	Records []PGDNSSecRecord
+}
+
+func (tbl *PGDNSSecTable) gatherKeys(db *sql.DB) error {
+	sz, err := getSize(db, "dnssec")
+	if err != nil {
+		log.Println("PGDNSSec gatherKeys: unable to determine size of dnssec table")
+	}
+	tbl.Records = make([]PGDNSSecRecord, sz)
+
+	rows, err := db.Query("SELECT cdn, data from dnssec")
+	if err != nil {
+		return fmt.Errorf("PGDNSSec gatherKeys: unable to query: %v", err)
+	}
+	defer rows.Close()
+	i := 0
+	for rows.Next() {
+		if i > len(tbl.Records)-1 {
+			return fmt.Errorf("PGDNSSec gatherKeys got more results than expected %v", len(tbl.Records))
+		}
+		err := rows.Scan(&tbl.Records[i].CDN, &tbl.Records[i].DataEncrypted)
+		if err != nil {
+			return fmt.Errorf("PGDNSSec gatherKeys unable to scan row: %v", err)
+		}
+		i += 1
+	}
+	return nil
+}
+func (tbl *PGDNSSecTable) decrypt(aesKey []byte) error {
+	for i, _ := range tbl.Records {
+		data, err := decryptInto(aesKey, tbl.Records[i].DataEncrypted, reflect.TypeOf(tbl.Records[i].Key))
+		if err != nil {
+			return fmt.Errorf("unable to decrypt into keys: %v", err)
+		}
+		tbl.Records[i].Key = *data.Interface().(*tc.DNSSECKeysTrafficVault)
+	}
+	return nil
+}
+func (tbl *PGDNSSecTable) encrypt(aesKey []byte) error {
+	for i, dns := range tbl.Records {
+		data, err := json.Marshal(&dns.Key)
+		if err != nil {
+			return fmt.Errorf("encrypt issue marshalling keys: %v", err)
+		}
+		dat, err := encrypt(data, aesKey)
+		if err != nil {
+			return fmt.Errorf("encrypt error: %v", err)
+		}
+		tbl.Records[i].DataEncrypted = dat
+	}
+	return nil
+}
+func (tbl *PGDNSSecTable) toGeneric() []DNSSecKey {
+	keys := make([]DNSSecKey, len(tbl.Records))
+
+	for i, record := range tbl.Records {
+		keys[i] = DNSSecKey{
+			CDN:                    record.CDN,
+			DNSSECKeysTrafficVault: record.Key,
+			CommonRecord:           record.CommonRecord,
+		}
+	}
+
+	return keys
+}
+func (tbl *PGDNSSecTable) fromGeneric(keys []DNSSecKey) {
+	tbl.Records = make([]PGDNSSecRecord, len(keys))
+
+	for i, key := range keys {
+		tbl.Records[i] = PGDNSSecRecord{
+			Key: key.DNSSECKeysTrafficVault,
+			CDN: key.CDN,
+			PGCommonRecord: PGCommonRecord{
+				DataEncrypted: nil,
+				CommonRecord:  CommonRecord{},
+			},
+		}
+	}
+}
+func (tbl *PGDNSSecTable) validate() []string {
+	for i, record := range tbl.Records {
+		if record.DataEncrypted == nil && len(record.Key) > 0 {
+			return []string{fmt.Sprintf("DNSSEC Key %v: DataEncrypted is blank!", i)}
+		}
+	}
+	return nil
+}
+func (tbl *PGDNSSecTable) insertKeys(db *sql.DB) error {
+	queryFmt := "INSERT INTO dnssec (cdn, data) VALUES "
+	stride := 2
+	queryArgs := make([]interface{}, len(tbl.Records)*stride)
+	for i, record := range tbl.Records {
+		j := i * stride
+		queryArgs[j] = record.CDN
+		queryArgs[j+1] = record.DataEncrypted
+	}
+	return insertIntoTable(db, queryFmt, stride, queryArgs)
+}
+
+type PGSSLKeyRecord struct {
+	Keys tc.DeliveryServiceSSLKeys
+	PGCommonRecord
+
+	// These records are stored on the table but are duplicated
+	DeliveryService string
+	Version         string
+	CDN             string
+}
+type PGSSLKeyTable struct {
+	Records []PGSSLKeyRecord
+}
+
+func (tbl *PGSSLKeyTable) insertKeys(db *sql.DB) error {
+	queryFmt := "INSERT INTO sslkey (deliveryservice, data, cdn, version) VALUES "
+	stride := 4
+	queryArgs := make([]interface{}, len(tbl.Records)*stride)
+	for i, record := range tbl.Records {
+		j := i * stride
+		queryArgs[j] = record.DeliveryService
+		queryArgs[j+1] = record.DataEncrypted
+		queryArgs[j+2] = record.CDN
+		queryArgs[j+3] = record.Version
+	}
+	return insertIntoTable(db, queryFmt, 4, queryArgs)
+}
+func (tbl *PGSSLKeyTable) gatherKeys(db *sql.DB) error {
+	sz, err := getSize(db, "sslkey")
+	if err != nil {
+		return fmt.Errorf("PGSSLKey gatherKeys unable to determine size of sslkey table: %v", err)
+	}
+	tbl.Records = make([]PGSSLKeyRecord, sz)
+
+	rows, err := db.Query("SELECT data, deliveryservice, cdn, version from sslkey")
+	if err != nil {
+		return fmt.Errorf("PGSSLKey gatherKeys unable to query: %v", err)
+	}
+	defer rows.Close()
+	i := 0
+	for rows.Next() {
+		if i > len(tbl.Records)-1 {
+			return errors.New("PGSSLKey gatherKeys: got more results than expected")
+		}
+		err := rows.Scan(&tbl.Records[i].DataEncrypted, &tbl.Records[i].DeliveryService, &tbl.Records[i].CDN, &tbl.Records[i].Version)
+		if err != nil {
+			return fmt.Errorf("PGSSLKey gatherKeys unable to scan row: %v", err)
+		}
+		i += 1
+	}
+	return nil
+}
+func (tbl *PGSSLKeyTable) decrypt(aesKey []byte) error {
+	for i, dns := range tbl.Records {
+		data, err := decryptInto(aesKey, dns.DataEncrypted, reflect.TypeOf(dns.Keys))
+		if err != nil {
+			return fmt.Errorf("unable to decrypt into keys: %v", err)
+		}
+		tbl.Records[i].Keys = *data.Interface().(*tc.DeliveryServiceSSLKeys)
+	}
+	return nil
+}
+func (tbl *PGSSLKeyTable) encrypt(aesKey []byte) error {
+	for i, dns := range tbl.Records {
+		data, err := json.Marshal(dns.Keys)
+		if err != nil {
+			return fmt.Errorf("encrypt issue marshalling keys: %v", err)
+		}
+		dat, err := encrypt(data, aesKey)
+		if err != nil {
+			return fmt.Errorf("encrypt error: %v", err)
+		}
+		tbl.Records[i].DataEncrypted = dat
+	}
+	return nil
+}
+func (tbl *PGSSLKeyTable) toGeneric() []SSLKey {
+	keys := make([]SSLKey, len(tbl.Records))
+
+	for i, record := range tbl.Records {
+		keys[i] = SSLKey{
+			DeliveryServiceSSLKeys: record.Keys,
+			CommonRecord:           record.CommonRecord,
+		}
+	}
+	return keys
+}
+func (tbl *PGSSLKeyTable) fromGeneric(keys []SSLKey) {
+	tbl.Records = make([]PGSSLKeyRecord, len(keys))
+
+	for i, key := range keys {
+		tbl.Records[i] = PGSSLKeyRecord{
+			Keys: key.DeliveryServiceSSLKeys,
+			PGCommonRecord: PGCommonRecord{
+				DataEncrypted: nil,
+				CommonRecord:  CommonRecord{},
+			},
+			DeliveryService: key.DeliveryService,
+			CDN:             key.CDN,
+		}
+	}
+}
+func (tbl *PGSSLKeyTable) validate() []string {
+	defaultKey := tc.DeliveryServiceSSLKeys{}
+	var errors []string
+	fmtStr := "SSL Key %v: %v"
+	for i, record := range tbl.Records {
+		if record.Keys == defaultKey {
+			errors = append(errors, fmt.Sprintf(fmtStr, i, "DS SSL Keys are default!"))
+		} else if record.Keys.Key == "" {
+			errors = append(errors, fmt.Sprintf(fmtStr, i, "Key is blank!"))
+		} else if record.Keys.CDN == "" {
+			errors = append(errors, fmt.Sprintf(fmtStr, i, "CDN is blank!"))
+		} else if record.Keys.DeliveryService == "" {
+			errors = append(errors, fmt.Sprintf(fmtStr, i, "DS is blank!"))
+		} else if record.DataEncrypted == nil {
+			errors = append(errors, fmt.Sprintf(fmtStr, i, "DataEncrypted is blank!"))
+		}
+	}
+	return errors
+}
+
+type PGURLSigKeyRecord struct {
+	Keys            tc.URLSigKeys
+	DeliveryService string
+	PGCommonRecord
+}
+type PGURLSigKeyTable struct {
+	Records []PGURLSigKeyRecord
+}
+
+func (tbl *PGURLSigKeyTable) insertKeys(db *sql.DB) error {
+	queryBase := "INSERT INTO url_sig_key (deliveryservice, data) VALUES "
+	stride := 2
+	queryArgs := make([]interface{}, len(tbl.Records)*stride)
+	for i, record := range tbl.Records {
+		j := i * stride
+		queryArgs[j] = record.DeliveryService
+		queryArgs[j+1] = record.DataEncrypted
+	}
+	return insertIntoTable(db, queryBase, stride, queryArgs)
+}
+func (tbl *PGURLSigKeyTable) gatherKeys(db *sql.DB) error {
+	sz, err := getSize(db, "url_sig_key")
+	if err != nil {
+		log.Println("PGURLSigKey gatherKeys: unable to determine url_sig_key table size")
+	}
+	tbl.Records = make([]PGURLSigKeyRecord, sz)
+
+	rows, err := db.Query("SELECT deliveryservice, data from url_sig_key")
+	if err != nil {
+		return fmt.Errorf("PGURLSigKey gatherKeys error creating query: %v", err)
+	}
+	defer rows.Close()
+	i := 0
+	for rows.Next() {
+		if i > len(tbl.Records)-1 {
+			return fmt.Errorf("PGURLSigKey gatherKeys: got more results than expected %v", len(tbl.Records))
+		}
+		err := rows.Scan(&tbl.Records[i].DeliveryService, &tbl.Records[i].DataEncrypted)
+		if err != nil {
+			return fmt.Errorf("PGURLSigKey gatherKeys: unable to scan row: %v", err)
+		}
+		i += 1
+	}
+	return nil
+}
+func (tbl *PGURLSigKeyTable) decrypt(aesKey []byte) error {
+	for i, sig := range tbl.Records {
+		data, err := decryptInto(aesKey, sig.DataEncrypted, reflect.TypeOf(sig.Keys))
+		if err != nil {
+			return fmt.Errorf("unable to decrypt into keys: %v", err)
+		}
+
+		tbl.Records[i].Keys = *data.Interface().(*tc.URLSigKeys)
+	}
+	return nil
+}
+func (tbl *PGURLSigKeyTable) encrypt(aesKey []byte) error {
+	for i, sig := range tbl.Records {
+		data, err := json.Marshal(&sig.Keys)
+		if err != nil {
+			return fmt.Errorf("encrypt issue marshalling keys: %v", err)
+		}
+
+		dat, err := encrypt(data, aesKey)
+		if err != nil {
+			return fmt.Errorf("encrypt error: %v", err)
+		}
+		tbl.Records[i].DataEncrypted = dat
+	}
+	return nil
+}
+func (tbl *PGURLSigKeyTable) toGeneric() []URLSigKey {
+	keys := make([]URLSigKey, len(tbl.Records))
+
+	for i, record := range tbl.Records {
+		keys[i] = URLSigKey{
+			DeliveryService: record.DeliveryService,
+			URLSigKeys:      record.Keys,
+			CommonRecord:    record.CommonRecord,
+		}
+	}
+	return keys
+}
+func (tbl *PGURLSigKeyTable) fromGeneric(keys []URLSigKey) {
+	tbl.Records = make([]PGURLSigKeyRecord, len(keys))
+
+	for i, key := range keys {
+		tbl.Records[i] = PGURLSigKeyRecord{
+			Keys:            key.URLSigKeys,
+			DeliveryService: key.DeliveryService,
+			PGCommonRecord: PGCommonRecord{
+				DataEncrypted: nil,
+				CommonRecord:  CommonRecord{},
+			},
+		}
+	}
+}
+func (tbl *PGURLSigKeyTable) validate() []string {
+	for i, record := range tbl.Records {
+		if record.DataEncrypted == nil && len(record.Keys) > 0 {
+			return []string{fmt.Sprintf("URl Sig Key %v: DataEncrypted is blank!", i)}
+		}
+	}
+	return nil
+}
+
+type PGURISignKeyRecord struct {
+	Keys            map[string]tc.URISignerKeyset
+	DeliveryService string
+	PGCommonRecord
+}
+type PGURISignKeyTable struct {
+	Records []PGURISignKeyRecord
+}
+
+func (tbl *PGURISignKeyTable) insertKeys(db *sql.DB) error {
+	queryFmt := "INSERT INTO uri_signing_key (deliveryservice, data) VALUES "
+	stride := 2
+	queryArgs := make([]interface{}, len(tbl.Records)*stride)
+	for i, record := range tbl.Records {
+		j := i * stride
+		queryArgs[j] = record.DeliveryService
+		queryArgs[j+1] = record.DataEncrypted
+	}
+	return insertIntoTable(db, queryFmt, stride, queryArgs)
+}
+func (tbl *PGURISignKeyTable) gatherKeys(db *sql.DB) error {
+	sz, err := getSize(db, "uri_signing_key")
+	if err != nil {
+		log.Println("PGURISignKey gatherKeys: unable to determine size of uri_signing_key table")
+	}
+	tbl.Records = make([]PGURISignKeyRecord, sz)
+
+	rows, err := db.Query("SELECT deliveryservice, data from uri_signing_key")
+	if err != nil {
+		return fmt.Errorf("PGURISignKey gatherKeys error while query: %v", err)
+	}
+	defer rows.Close()
+	i := 0
+	for rows.Next() {
+		if i > len(tbl.Records)-1 {
+			return fmt.Errorf("PGURISignKey gatherKeys: got more results than expected %v", len(tbl.Records))
+		}
+		err := rows.Scan(&tbl.Records[i].DeliveryService, &tbl.Records[i].DataEncrypted)
+		if err != nil {
+			return fmt.Errorf("PGURISignKey gatherKeys: unable to scan row: %v", err)

Review comment:
       same as above RE: `%w` vs `%v`

##########
File path: tools/traffic_vault_migrate/postgres.go
##########
@@ -0,0 +1,716 @@
+package main
+
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+import (
+	"database/sql"
+	"encoding/base64"
+	"encoding/json"
+	"errors"
+	"fmt"
+	"github.com/apache/trafficcontrol/lib/go-tc"
+	util "github.com/apache/trafficcontrol/lib/go-util"
+	_ "github.com/lib/pq"
+	"log"
+	"reflect"
+	"strconv"
+	"strings"
+)
+
+type PGConfig struct {
+	Host     string `json:"host"`
+	Port     string `json:"port"`
+	User     string `json:"user"`
+	Password string `json:"password"`
+	SSLMode  string `json:"sslmode"`
+	Database string `json:"database"`
+	Key      string `json:"aesKey"`
+	AESKey   []byte
+}
+type PGBackend struct {
+	sslKey PGSSLKeyTable
+	dnssec PGDNSSecTable
+	uri    PGURISignKeyTable
+	url    PGURLSigKeyTable
+	cfg    PGConfig
+	db     *sql.DB
+}
+
+func (pg *PGBackend) String() string {
+	data := fmt.Sprintf("PG server %v@%v:%v\n", pg.cfg.User, pg.cfg.Host, pg.cfg.Port)
+	data += fmt.Sprintf("\tSSL Keys: %v\n", len(pg.sslKey.Records))
+	data += fmt.Sprintf("\tDNSSec Keys: %v\n", len(pg.dnssec.Records))
+	data += fmt.Sprintf("\tURI Keys: %v\n", len(pg.uri.Records))
+	data += fmt.Sprintf("\tURL Keys: %v\n", len(pg.url.Records))
+	return data
+}
+func (pg *PGBackend) Name() string {
+	return "PG"
+}
+func (pg *PGBackend) ReadConfig(s string) error {
+	genericCfg, err := UnmarshalConfig(s, reflect.TypeOf(pg.cfg))
+	if err != nil {
+		return err
+	}
+
+	pg.cfg = *genericCfg.Interface().(*PGConfig)
+
+	pg.cfg.AESKey, err = base64.StdEncoding.DecodeString(pg.cfg.Key)
+	if err != nil {
+		return fmt.Errorf("unable to decode PG AESKey: %v", err)
+	}
+	return nil
+}
+func (pg *PGBackend) Insert() error {
+	if err := pg.sslKey.insertKeys(pg.db); err != nil {
+		return err
+	}
+	if err := pg.dnssec.insertKeys(pg.db); err != nil {
+		return err
+	}
+	if err := pg.url.insertKeys(pg.db); err != nil {
+		return err
+	}
+	if err := pg.uri.insertKeys(pg.db); err != nil {
+		return err
+	}
+	return nil
+}
+func (pg *PGBackend) Start() error {
+	sqlStr := fmt.Sprintf("postgres://%s:%s@%s:%s/%s?sslmode=%s", pg.cfg.User, pg.cfg.Password, pg.cfg.Host, pg.cfg.Port, pg.cfg.Database, pg.cfg.SSLMode)
+	db, err := sql.Open("postgres", sqlStr)
+	if err != nil {
+		return fmt.Errorf("Unable to start PG client: %v", err)
+	}
+
+	pg.db = db
+	pg.sslKey = PGSSLKeyTable{}
+	pg.dnssec = PGDNSSecTable{}
+	pg.url = PGURLSigKeyTable{}
+	pg.uri = PGURISignKeyTable{}
+
+	return nil
+}
+func (pg *PGBackend) ValidateKey() []string {
+	var errors []string
+	if errs := pg.sslKey.validate(); errs != nil {
+		errors = append(errors, errs...)
+	}
+	if errs := pg.dnssec.validate(); errs != nil {
+		errors = append(errors, errs...)
+	}
+	if errs := pg.uri.validate(); errs != nil {
+		errors = append(errors, errs...)
+	}
+	if errs := pg.url.validate(); errs != nil {
+		errors = append(errors, errs...)
+	}
+	return errors
+}
+func (pg *PGBackend) Stop() error {
+	return pg.db.Close()
+}
+func (pg *PGBackend) Ping() error {
+	return pg.db.Ping()
+}
+func (pg *PGBackend) Fetch() error {
+	if err := pg.sslKey.gatherKeys(pg.db); err != nil {
+		return err
+	}
+
+	if err := pg.dnssec.gatherKeys(pg.db); err != nil {
+		return err
+	}
+
+	if err := pg.url.gatherKeys(pg.db); err != nil {
+		return err
+	}
+
+	if err := pg.uri.gatherKeys(pg.db); err != nil {
+		return err
+	}
+
+	return nil
+}
+func (pg *PGBackend) GetSSLKeys() ([]SSLKey, error) {
+	if err := pg.sslKey.decrypt(pg.cfg.AESKey); err != nil {
+		return nil, err
+	}
+	return pg.sslKey.toGeneric(), nil
+}
+func (pg *PGBackend) SetSSLKeys(keys []SSLKey) error {
+	pg.sslKey.fromGeneric(keys)
+	return pg.sslKey.encrypt(pg.cfg.AESKey)
+}
+func (pg *PGBackend) GetDNSSecKeys() ([]DNSSecKey, error) {
+	if err := pg.dnssec.decrypt(pg.cfg.AESKey); err != nil {
+		return nil, err
+	}
+	return pg.dnssec.toGeneric(), nil
+}
+func (pg *PGBackend) SetDNSSecKeys(keys []DNSSecKey) error {
+	pg.dnssec.fromGeneric(keys)
+	return pg.dnssec.encrypt(pg.cfg.AESKey)
+}
+func (pg *PGBackend) GetURISignKeys() ([]URISignKey, error) {
+	if err := pg.uri.decrypt(pg.cfg.AESKey); err != nil {
+		return nil, err
+	}
+	return pg.uri.toGeneric(), nil
+}
+func (pg *PGBackend) SetURISignKeys(keys []URISignKey) error {
+	pg.uri.fromGeneric(keys)
+	return pg.uri.encrypt(pg.cfg.AESKey)
+}
+func (pg *PGBackend) GetURLSigKeys() ([]URLSigKey, error) {
+	if err := pg.url.decrypt(pg.cfg.AESKey); err != nil {
+		return nil, err
+	}
+	return pg.url.toGeneric(), nil
+}
+func (pg *PGBackend) SetURLSigKeys(keys []URLSigKey) error {
+	pg.url.fromGeneric(keys)
+	return pg.url.encrypt(pg.cfg.AESKey)
+}
+
+type PGCommonRecord struct {
+	DataEncrypted []byte
+	CommonRecord
+}
+
+type PGDNSSecRecord struct {
+	Key tc.DNSSECKeysTrafficVault
+	CDN string
+	PGCommonRecord
+}
+type PGDNSSecTable struct {
+	Records []PGDNSSecRecord
+}
+
+func (tbl *PGDNSSecTable) gatherKeys(db *sql.DB) error {
+	sz, err := getSize(db, "dnssec")
+	if err != nil {
+		log.Println("PGDNSSec gatherKeys: unable to determine size of dnssec table")

Review comment:
       I just realized you're not using a `log` package that provides `Errorln` - what I mean, then, is should this be logging to stderr?

##########
File path: tools/traffic_vault_migrate/postgres.go
##########
@@ -0,0 +1,716 @@
+package main
+
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+import (
+	"database/sql"
+	"encoding/base64"
+	"encoding/json"
+	"errors"
+	"fmt"
+	"github.com/apache/trafficcontrol/lib/go-tc"
+	util "github.com/apache/trafficcontrol/lib/go-util"
+	_ "github.com/lib/pq"
+	"log"
+	"reflect"
+	"strconv"
+	"strings"
+)
+
+type PGConfig struct {
+	Host     string `json:"host"`
+	Port     string `json:"port"`
+	User     string `json:"user"`
+	Password string `json:"password"`
+	SSLMode  string `json:"sslmode"`
+	Database string `json:"database"`
+	Key      string `json:"aesKey"`
+	AESKey   []byte
+}
+type PGBackend struct {
+	sslKey PGSSLKeyTable
+	dnssec PGDNSSecTable
+	uri    PGURISignKeyTable
+	url    PGURLSigKeyTable
+	cfg    PGConfig
+	db     *sql.DB
+}
+
+func (pg *PGBackend) String() string {
+	data := fmt.Sprintf("PG server %v@%v:%v\n", pg.cfg.User, pg.cfg.Host, pg.cfg.Port)
+	data += fmt.Sprintf("\tSSL Keys: %v\n", len(pg.sslKey.Records))
+	data += fmt.Sprintf("\tDNSSec Keys: %v\n", len(pg.dnssec.Records))
+	data += fmt.Sprintf("\tURI Keys: %v\n", len(pg.uri.Records))
+	data += fmt.Sprintf("\tURL Keys: %v\n", len(pg.url.Records))
+	return data
+}
+func (pg *PGBackend) Name() string {
+	return "PG"
+}
+func (pg *PGBackend) ReadConfig(s string) error {
+	genericCfg, err := UnmarshalConfig(s, reflect.TypeOf(pg.cfg))
+	if err != nil {
+		return err
+	}
+
+	pg.cfg = *genericCfg.Interface().(*PGConfig)
+
+	pg.cfg.AESKey, err = base64.StdEncoding.DecodeString(pg.cfg.Key)
+	if err != nil {
+		return fmt.Errorf("unable to decode PG AESKey: %v", err)
+	}
+	return nil
+}
+func (pg *PGBackend) Insert() error {
+	if err := pg.sslKey.insertKeys(pg.db); err != nil {
+		return err
+	}
+	if err := pg.dnssec.insertKeys(pg.db); err != nil {
+		return err
+	}
+	if err := pg.url.insertKeys(pg.db); err != nil {
+		return err
+	}
+	if err := pg.uri.insertKeys(pg.db); err != nil {
+		return err
+	}
+	return nil
+}
+func (pg *PGBackend) Start() error {
+	sqlStr := fmt.Sprintf("postgres://%s:%s@%s:%s/%s?sslmode=%s", pg.cfg.User, pg.cfg.Password, pg.cfg.Host, pg.cfg.Port, pg.cfg.Database, pg.cfg.SSLMode)
+	db, err := sql.Open("postgres", sqlStr)
+	if err != nil {
+		return fmt.Errorf("Unable to start PG client: %v", err)
+	}
+
+	pg.db = db
+	pg.sslKey = PGSSLKeyTable{}
+	pg.dnssec = PGDNSSecTable{}
+	pg.url = PGURLSigKeyTable{}
+	pg.uri = PGURISignKeyTable{}
+
+	return nil
+}
+func (pg *PGBackend) ValidateKey() []string {
+	var errors []string
+	if errs := pg.sslKey.validate(); errs != nil {
+		errors = append(errors, errs...)
+	}
+	if errs := pg.dnssec.validate(); errs != nil {
+		errors = append(errors, errs...)
+	}
+	if errs := pg.uri.validate(); errs != nil {
+		errors = append(errors, errs...)
+	}
+	if errs := pg.url.validate(); errs != nil {
+		errors = append(errors, errs...)
+	}
+	return errors
+}
+func (pg *PGBackend) Stop() error {
+	return pg.db.Close()
+}
+func (pg *PGBackend) Ping() error {
+	return pg.db.Ping()
+}
+func (pg *PGBackend) Fetch() error {
+	if err := pg.sslKey.gatherKeys(pg.db); err != nil {
+		return err
+	}
+
+	if err := pg.dnssec.gatherKeys(pg.db); err != nil {
+		return err
+	}
+
+	if err := pg.url.gatherKeys(pg.db); err != nil {
+		return err
+	}
+
+	if err := pg.uri.gatherKeys(pg.db); err != nil {
+		return err
+	}
+
+	return nil
+}
+func (pg *PGBackend) GetSSLKeys() ([]SSLKey, error) {
+	if err := pg.sslKey.decrypt(pg.cfg.AESKey); err != nil {
+		return nil, err
+	}
+	return pg.sslKey.toGeneric(), nil
+}
+func (pg *PGBackend) SetSSLKeys(keys []SSLKey) error {
+	pg.sslKey.fromGeneric(keys)
+	return pg.sslKey.encrypt(pg.cfg.AESKey)
+}
+func (pg *PGBackend) GetDNSSecKeys() ([]DNSSecKey, error) {
+	if err := pg.dnssec.decrypt(pg.cfg.AESKey); err != nil {
+		return nil, err
+	}
+	return pg.dnssec.toGeneric(), nil
+}
+func (pg *PGBackend) SetDNSSecKeys(keys []DNSSecKey) error {
+	pg.dnssec.fromGeneric(keys)
+	return pg.dnssec.encrypt(pg.cfg.AESKey)
+}
+func (pg *PGBackend) GetURISignKeys() ([]URISignKey, error) {
+	if err := pg.uri.decrypt(pg.cfg.AESKey); err != nil {
+		return nil, err
+	}
+	return pg.uri.toGeneric(), nil
+}
+func (pg *PGBackend) SetURISignKeys(keys []URISignKey) error {
+	pg.uri.fromGeneric(keys)
+	return pg.uri.encrypt(pg.cfg.AESKey)
+}
+func (pg *PGBackend) GetURLSigKeys() ([]URLSigKey, error) {
+	if err := pg.url.decrypt(pg.cfg.AESKey); err != nil {
+		return nil, err
+	}
+	return pg.url.toGeneric(), nil
+}
+func (pg *PGBackend) SetURLSigKeys(keys []URLSigKey) error {
+	pg.url.fromGeneric(keys)
+	return pg.url.encrypt(pg.cfg.AESKey)
+}
+
+type PGCommonRecord struct {
+	DataEncrypted []byte
+	CommonRecord
+}
+
+type PGDNSSecRecord struct {
+	Key tc.DNSSECKeysTrafficVault
+	CDN string
+	PGCommonRecord
+}
+type PGDNSSecTable struct {
+	Records []PGDNSSecRecord
+}
+
+func (tbl *PGDNSSecTable) gatherKeys(db *sql.DB) error {
+	sz, err := getSize(db, "dnssec")
+	if err != nil {
+		log.Println("PGDNSSec gatherKeys: unable to determine size of dnssec table")
+	}
+	tbl.Records = make([]PGDNSSecRecord, sz)
+
+	rows, err := db.Query("SELECT cdn, data from dnssec")
+	if err != nil {
+		return fmt.Errorf("PGDNSSec gatherKeys: unable to query: %v", err)
+	}
+	defer rows.Close()
+	i := 0
+	for rows.Next() {
+		if i > len(tbl.Records)-1 {
+			return fmt.Errorf("PGDNSSec gatherKeys got more results than expected %v", len(tbl.Records))
+		}
+		err := rows.Scan(&tbl.Records[i].CDN, &tbl.Records[i].DataEncrypted)
+		if err != nil {
+			return fmt.Errorf("PGDNSSec gatherKeys unable to scan row: %v", err)
+		}
+		i += 1
+	}
+	return nil
+}
+func (tbl *PGDNSSecTable) decrypt(aesKey []byte) error {
+	for i, _ := range tbl.Records {
+		data, err := decryptInto(aesKey, tbl.Records[i].DataEncrypted, reflect.TypeOf(tbl.Records[i].Key))
+		if err != nil {
+			return fmt.Errorf("unable to decrypt into keys: %v", err)
+		}
+		tbl.Records[i].Key = *data.Interface().(*tc.DNSSECKeysTrafficVault)
+	}
+	return nil
+}
+func (tbl *PGDNSSecTable) encrypt(aesKey []byte) error {
+	for i, dns := range tbl.Records {
+		data, err := json.Marshal(&dns.Key)
+		if err != nil {
+			return fmt.Errorf("encrypt issue marshalling keys: %v", err)
+		}
+		dat, err := encrypt(data, aesKey)
+		if err != nil {
+			return fmt.Errorf("encrypt error: %v", err)
+		}
+		tbl.Records[i].DataEncrypted = dat
+	}
+	return nil
+}
+func (tbl *PGDNSSecTable) toGeneric() []DNSSecKey {
+	keys := make([]DNSSecKey, len(tbl.Records))
+
+	for i, record := range tbl.Records {
+		keys[i] = DNSSecKey{
+			CDN:                    record.CDN,
+			DNSSECKeysTrafficVault: record.Key,
+			CommonRecord:           record.CommonRecord,
+		}
+	}
+
+	return keys
+}
+func (tbl *PGDNSSecTable) fromGeneric(keys []DNSSecKey) {
+	tbl.Records = make([]PGDNSSecRecord, len(keys))
+
+	for i, key := range keys {
+		tbl.Records[i] = PGDNSSecRecord{
+			Key: key.DNSSECKeysTrafficVault,
+			CDN: key.CDN,
+			PGCommonRecord: PGCommonRecord{
+				DataEncrypted: nil,
+				CommonRecord:  CommonRecord{},
+			},
+		}
+	}
+}
+func (tbl *PGDNSSecTable) validate() []string {
+	for i, record := range tbl.Records {
+		if record.DataEncrypted == nil && len(record.Key) > 0 {
+			return []string{fmt.Sprintf("DNSSEC Key %v: DataEncrypted is blank!", i)}
+		}
+	}
+	return nil
+}
+func (tbl *PGDNSSecTable) insertKeys(db *sql.DB) error {
+	queryFmt := "INSERT INTO dnssec (cdn, data) VALUES "
+	stride := 2
+	queryArgs := make([]interface{}, len(tbl.Records)*stride)
+	for i, record := range tbl.Records {
+		j := i * stride
+		queryArgs[j] = record.CDN
+		queryArgs[j+1] = record.DataEncrypted
+	}
+	return insertIntoTable(db, queryFmt, stride, queryArgs)
+}
+
+type PGSSLKeyRecord struct {
+	Keys tc.DeliveryServiceSSLKeys
+	PGCommonRecord
+
+	// These records are stored on the table but are duplicated
+	DeliveryService string
+	Version         string
+	CDN             string
+}
+type PGSSLKeyTable struct {
+	Records []PGSSLKeyRecord
+}
+
+func (tbl *PGSSLKeyTable) insertKeys(db *sql.DB) error {
+	queryFmt := "INSERT INTO sslkey (deliveryservice, data, cdn, version) VALUES "
+	stride := 4
+	queryArgs := make([]interface{}, len(tbl.Records)*stride)
+	for i, record := range tbl.Records {
+		j := i * stride
+		queryArgs[j] = record.DeliveryService
+		queryArgs[j+1] = record.DataEncrypted
+		queryArgs[j+2] = record.CDN
+		queryArgs[j+3] = record.Version
+	}
+	return insertIntoTable(db, queryFmt, 4, queryArgs)
+}
+func (tbl *PGSSLKeyTable) gatherKeys(db *sql.DB) error {
+	sz, err := getSize(db, "sslkey")
+	if err != nil {
+		return fmt.Errorf("PGSSLKey gatherKeys unable to determine size of sslkey table: %v", err)
+	}
+	tbl.Records = make([]PGSSLKeyRecord, sz)
+
+	rows, err := db.Query("SELECT data, deliveryservice, cdn, version from sslkey")
+	if err != nil {
+		return fmt.Errorf("PGSSLKey gatherKeys unable to query: %v", err)
+	}
+	defer rows.Close()
+	i := 0
+	for rows.Next() {
+		if i > len(tbl.Records)-1 {
+			return errors.New("PGSSLKey gatherKeys: got more results than expected")
+		}
+		err := rows.Scan(&tbl.Records[i].DataEncrypted, &tbl.Records[i].DeliveryService, &tbl.Records[i].CDN, &tbl.Records[i].Version)
+		if err != nil {
+			return fmt.Errorf("PGSSLKey gatherKeys unable to scan row: %v", err)
+		}
+		i += 1
+	}
+	return nil
+}
+func (tbl *PGSSLKeyTable) decrypt(aesKey []byte) error {
+	for i, dns := range tbl.Records {
+		data, err := decryptInto(aesKey, dns.DataEncrypted, reflect.TypeOf(dns.Keys))
+		if err != nil {
+			return fmt.Errorf("unable to decrypt into keys: %v", err)
+		}
+		tbl.Records[i].Keys = *data.Interface().(*tc.DeliveryServiceSSLKeys)
+	}
+	return nil
+}
+func (tbl *PGSSLKeyTable) encrypt(aesKey []byte) error {
+	for i, dns := range tbl.Records {
+		data, err := json.Marshal(dns.Keys)
+		if err != nil {
+			return fmt.Errorf("encrypt issue marshalling keys: %v", err)
+		}
+		dat, err := encrypt(data, aesKey)
+		if err != nil {
+			return fmt.Errorf("encrypt error: %v", err)
+		}
+		tbl.Records[i].DataEncrypted = dat
+	}
+	return nil
+}
+func (tbl *PGSSLKeyTable) toGeneric() []SSLKey {
+	keys := make([]SSLKey, len(tbl.Records))
+
+	for i, record := range tbl.Records {
+		keys[i] = SSLKey{
+			DeliveryServiceSSLKeys: record.Keys,
+			CommonRecord:           record.CommonRecord,
+		}
+	}
+	return keys
+}
+func (tbl *PGSSLKeyTable) fromGeneric(keys []SSLKey) {
+	tbl.Records = make([]PGSSLKeyRecord, len(keys))
+
+	for i, key := range keys {
+		tbl.Records[i] = PGSSLKeyRecord{
+			Keys: key.DeliveryServiceSSLKeys,
+			PGCommonRecord: PGCommonRecord{
+				DataEncrypted: nil,
+				CommonRecord:  CommonRecord{},
+			},
+			DeliveryService: key.DeliveryService,
+			CDN:             key.CDN,
+		}
+	}
+}
+func (tbl *PGSSLKeyTable) validate() []string {
+	defaultKey := tc.DeliveryServiceSSLKeys{}
+	var errors []string
+	fmtStr := "SSL Key %v: %v"
+	for i, record := range tbl.Records {
+		if record.Keys == defaultKey {
+			errors = append(errors, fmt.Sprintf(fmtStr, i, "DS SSL Keys are default!"))
+		} else if record.Keys.Key == "" {
+			errors = append(errors, fmt.Sprintf(fmtStr, i, "Key is blank!"))
+		} else if record.Keys.CDN == "" {
+			errors = append(errors, fmt.Sprintf(fmtStr, i, "CDN is blank!"))
+		} else if record.Keys.DeliveryService == "" {
+			errors = append(errors, fmt.Sprintf(fmtStr, i, "DS is blank!"))
+		} else if record.DataEncrypted == nil {
+			errors = append(errors, fmt.Sprintf(fmtStr, i, "DataEncrypted is blank!"))
+		}
+	}
+	return errors
+}
+
+type PGURLSigKeyRecord struct {
+	Keys            tc.URLSigKeys
+	DeliveryService string
+	PGCommonRecord
+}
+type PGURLSigKeyTable struct {
+	Records []PGURLSigKeyRecord
+}
+
+func (tbl *PGURLSigKeyTable) insertKeys(db *sql.DB) error {
+	queryBase := "INSERT INTO url_sig_key (deliveryservice, data) VALUES "
+	stride := 2
+	queryArgs := make([]interface{}, len(tbl.Records)*stride)
+	for i, record := range tbl.Records {
+		j := i * stride
+		queryArgs[j] = record.DeliveryService
+		queryArgs[j+1] = record.DataEncrypted
+	}
+	return insertIntoTable(db, queryBase, stride, queryArgs)
+}
+func (tbl *PGURLSigKeyTable) gatherKeys(db *sql.DB) error {
+	sz, err := getSize(db, "url_sig_key")
+	if err != nil {
+		log.Println("PGURLSigKey gatherKeys: unable to determine url_sig_key table size")
+	}
+	tbl.Records = make([]PGURLSigKeyRecord, sz)
+
+	rows, err := db.Query("SELECT deliveryservice, data from url_sig_key")
+	if err != nil {
+		return fmt.Errorf("PGURLSigKey gatherKeys error creating query: %v", err)
+	}
+	defer rows.Close()
+	i := 0
+	for rows.Next() {
+		if i > len(tbl.Records)-1 {
+			return fmt.Errorf("PGURLSigKey gatherKeys: got more results than expected %v", len(tbl.Records))
+		}
+		err := rows.Scan(&tbl.Records[i].DeliveryService, &tbl.Records[i].DataEncrypted)
+		if err != nil {
+			return fmt.Errorf("PGURLSigKey gatherKeys: unable to scan row: %v", err)
+		}
+		i += 1
+	}
+	return nil
+}
+func (tbl *PGURLSigKeyTable) decrypt(aesKey []byte) error {
+	for i, sig := range tbl.Records {
+		data, err := decryptInto(aesKey, sig.DataEncrypted, reflect.TypeOf(sig.Keys))
+		if err != nil {
+			return fmt.Errorf("unable to decrypt into keys: %v", err)
+		}
+
+		tbl.Records[i].Keys = *data.Interface().(*tc.URLSigKeys)
+	}
+	return nil
+}
+func (tbl *PGURLSigKeyTable) encrypt(aesKey []byte) error {
+	for i, sig := range tbl.Records {
+		data, err := json.Marshal(&sig.Keys)
+		if err != nil {
+			return fmt.Errorf("encrypt issue marshalling keys: %v", err)
+		}
+
+		dat, err := encrypt(data, aesKey)
+		if err != nil {
+			return fmt.Errorf("encrypt error: %v", err)
+		}
+		tbl.Records[i].DataEncrypted = dat
+	}
+	return nil
+}
+func (tbl *PGURLSigKeyTable) toGeneric() []URLSigKey {
+	keys := make([]URLSigKey, len(tbl.Records))
+
+	for i, record := range tbl.Records {
+		keys[i] = URLSigKey{
+			DeliveryService: record.DeliveryService,
+			URLSigKeys:      record.Keys,
+			CommonRecord:    record.CommonRecord,
+		}
+	}
+	return keys
+}
+func (tbl *PGURLSigKeyTable) fromGeneric(keys []URLSigKey) {
+	tbl.Records = make([]PGURLSigKeyRecord, len(keys))
+
+	for i, key := range keys {
+		tbl.Records[i] = PGURLSigKeyRecord{
+			Keys:            key.URLSigKeys,
+			DeliveryService: key.DeliveryService,
+			PGCommonRecord: PGCommonRecord{
+				DataEncrypted: nil,
+				CommonRecord:  CommonRecord{},
+			},
+		}
+	}
+}
+func (tbl *PGURLSigKeyTable) validate() []string {
+	for i, record := range tbl.Records {
+		if record.DataEncrypted == nil && len(record.Keys) > 0 {
+			return []string{fmt.Sprintf("URl Sig Key %v: DataEncrypted is blank!", i)}
+		}
+	}
+	return nil
+}
+
+type PGURISignKeyRecord struct {
+	Keys            map[string]tc.URISignerKeyset
+	DeliveryService string
+	PGCommonRecord
+}
+type PGURISignKeyTable struct {
+	Records []PGURISignKeyRecord
+}
+
+func (tbl *PGURISignKeyTable) insertKeys(db *sql.DB) error {
+	queryFmt := "INSERT INTO uri_signing_key (deliveryservice, data) VALUES "
+	stride := 2
+	queryArgs := make([]interface{}, len(tbl.Records)*stride)
+	for i, record := range tbl.Records {
+		j := i * stride
+		queryArgs[j] = record.DeliveryService
+		queryArgs[j+1] = record.DataEncrypted
+	}
+	return insertIntoTable(db, queryFmt, stride, queryArgs)
+}
+func (tbl *PGURISignKeyTable) gatherKeys(db *sql.DB) error {
+	sz, err := getSize(db, "uri_signing_key")
+	if err != nil {
+		log.Println("PGURISignKey gatherKeys: unable to determine size of uri_signing_key table")
+	}
+	tbl.Records = make([]PGURISignKeyRecord, sz)
+
+	rows, err := db.Query("SELECT deliveryservice, data from uri_signing_key")
+	if err != nil {
+		return fmt.Errorf("PGURISignKey gatherKeys error while query: %v", err)

Review comment:
       same as above RE: `%w` vs `%v`

##########
File path: tools/traffic_vault_migrate/postgres.go
##########
@@ -0,0 +1,716 @@
+package main
+
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+import (
+	"database/sql"
+	"encoding/base64"
+	"encoding/json"
+	"errors"
+	"fmt"
+	"github.com/apache/trafficcontrol/lib/go-tc"
+	util "github.com/apache/trafficcontrol/lib/go-util"
+	_ "github.com/lib/pq"
+	"log"
+	"reflect"
+	"strconv"
+	"strings"
+)
+
+type PGConfig struct {
+	Host     string `json:"host"`
+	Port     string `json:"port"`
+	User     string `json:"user"`
+	Password string `json:"password"`
+	SSLMode  string `json:"sslmode"`
+	Database string `json:"database"`
+	Key      string `json:"aesKey"`
+	AESKey   []byte
+}
+type PGBackend struct {
+	sslKey PGSSLKeyTable
+	dnssec PGDNSSecTable
+	uri    PGURISignKeyTable
+	url    PGURLSigKeyTable
+	cfg    PGConfig
+	db     *sql.DB
+}
+
+func (pg *PGBackend) String() string {
+	data := fmt.Sprintf("PG server %v@%v:%v\n", pg.cfg.User, pg.cfg.Host, pg.cfg.Port)
+	data += fmt.Sprintf("\tSSL Keys: %v\n", len(pg.sslKey.Records))
+	data += fmt.Sprintf("\tDNSSec Keys: %v\n", len(pg.dnssec.Records))
+	data += fmt.Sprintf("\tURI Keys: %v\n", len(pg.uri.Records))
+	data += fmt.Sprintf("\tURL Keys: %v\n", len(pg.url.Records))
+	return data
+}
+func (pg *PGBackend) Name() string {
+	return "PG"
+}
+func (pg *PGBackend) ReadConfig(s string) error {
+	genericCfg, err := UnmarshalConfig(s, reflect.TypeOf(pg.cfg))
+	if err != nil {
+		return err
+	}
+
+	pg.cfg = *genericCfg.Interface().(*PGConfig)
+
+	pg.cfg.AESKey, err = base64.StdEncoding.DecodeString(pg.cfg.Key)
+	if err != nil {
+		return fmt.Errorf("unable to decode PG AESKey: %v", err)
+	}
+	return nil
+}
+func (pg *PGBackend) Insert() error {
+	if err := pg.sslKey.insertKeys(pg.db); err != nil {
+		return err
+	}
+	if err := pg.dnssec.insertKeys(pg.db); err != nil {
+		return err
+	}
+	if err := pg.url.insertKeys(pg.db); err != nil {
+		return err
+	}
+	if err := pg.uri.insertKeys(pg.db); err != nil {
+		return err
+	}
+	return nil
+}
+func (pg *PGBackend) Start() error {
+	sqlStr := fmt.Sprintf("postgres://%s:%s@%s:%s/%s?sslmode=%s", pg.cfg.User, pg.cfg.Password, pg.cfg.Host, pg.cfg.Port, pg.cfg.Database, pg.cfg.SSLMode)
+	db, err := sql.Open("postgres", sqlStr)
+	if err != nil {
+		return fmt.Errorf("Unable to start PG client: %v", err)
+	}
+
+	pg.db = db
+	pg.sslKey = PGSSLKeyTable{}
+	pg.dnssec = PGDNSSecTable{}
+	pg.url = PGURLSigKeyTable{}
+	pg.uri = PGURISignKeyTable{}
+
+	return nil
+}
+func (pg *PGBackend) ValidateKey() []string {
+	var errors []string
+	if errs := pg.sslKey.validate(); errs != nil {
+		errors = append(errors, errs...)
+	}
+	if errs := pg.dnssec.validate(); errs != nil {
+		errors = append(errors, errs...)
+	}
+	if errs := pg.uri.validate(); errs != nil {
+		errors = append(errors, errs...)
+	}
+	if errs := pg.url.validate(); errs != nil {
+		errors = append(errors, errs...)
+	}
+	return errors
+}
+func (pg *PGBackend) Stop() error {
+	return pg.db.Close()
+}
+func (pg *PGBackend) Ping() error {
+	return pg.db.Ping()
+}
+func (pg *PGBackend) Fetch() error {
+	if err := pg.sslKey.gatherKeys(pg.db); err != nil {
+		return err
+	}
+
+	if err := pg.dnssec.gatherKeys(pg.db); err != nil {
+		return err
+	}
+
+	if err := pg.url.gatherKeys(pg.db); err != nil {
+		return err
+	}
+
+	if err := pg.uri.gatherKeys(pg.db); err != nil {
+		return err
+	}
+
+	return nil
+}
+func (pg *PGBackend) GetSSLKeys() ([]SSLKey, error) {
+	if err := pg.sslKey.decrypt(pg.cfg.AESKey); err != nil {
+		return nil, err
+	}
+	return pg.sslKey.toGeneric(), nil
+}
+func (pg *PGBackend) SetSSLKeys(keys []SSLKey) error {
+	pg.sslKey.fromGeneric(keys)
+	return pg.sslKey.encrypt(pg.cfg.AESKey)
+}
+func (pg *PGBackend) GetDNSSecKeys() ([]DNSSecKey, error) {
+	if err := pg.dnssec.decrypt(pg.cfg.AESKey); err != nil {
+		return nil, err
+	}
+	return pg.dnssec.toGeneric(), nil
+}
+func (pg *PGBackend) SetDNSSecKeys(keys []DNSSecKey) error {
+	pg.dnssec.fromGeneric(keys)
+	return pg.dnssec.encrypt(pg.cfg.AESKey)
+}
+func (pg *PGBackend) GetURISignKeys() ([]URISignKey, error) {
+	if err := pg.uri.decrypt(pg.cfg.AESKey); err != nil {
+		return nil, err
+	}
+	return pg.uri.toGeneric(), nil
+}
+func (pg *PGBackend) SetURISignKeys(keys []URISignKey) error {
+	pg.uri.fromGeneric(keys)
+	return pg.uri.encrypt(pg.cfg.AESKey)
+}
+func (pg *PGBackend) GetURLSigKeys() ([]URLSigKey, error) {
+	if err := pg.url.decrypt(pg.cfg.AESKey); err != nil {
+		return nil, err
+	}
+	return pg.url.toGeneric(), nil
+}
+func (pg *PGBackend) SetURLSigKeys(keys []URLSigKey) error {
+	pg.url.fromGeneric(keys)
+	return pg.url.encrypt(pg.cfg.AESKey)
+}
+
+type PGCommonRecord struct {
+	DataEncrypted []byte
+	CommonRecord
+}
+
+type PGDNSSecRecord struct {
+	Key tc.DNSSECKeysTrafficVault
+	CDN string
+	PGCommonRecord
+}
+type PGDNSSecTable struct {
+	Records []PGDNSSecRecord
+}
+
+func (tbl *PGDNSSecTable) gatherKeys(db *sql.DB) error {
+	sz, err := getSize(db, "dnssec")
+	if err != nil {
+		log.Println("PGDNSSec gatherKeys: unable to determine size of dnssec table")
+	}
+	tbl.Records = make([]PGDNSSecRecord, sz)
+
+	rows, err := db.Query("SELECT cdn, data from dnssec")
+	if err != nil {
+		return fmt.Errorf("PGDNSSec gatherKeys: unable to query: %v", err)
+	}
+	defer rows.Close()

Review comment:
       I just realized you're using the standard `log` package, not `github.com/apache/trafficcontrol/lib/go-log` - so what I said doesn't apply unless you wanna switch libraries.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [trafficcontrol] shamrickus commented on a change in pull request #5924: Add tool to migrate data between TV backends

Posted by GitBox <gi...@apache.org>.
shamrickus commented on a change in pull request #5924:
URL: https://github.com/apache/trafficcontrol/pull/5924#discussion_r651935876



##########
File path: tools/traffic_vault_migrate/postgres.go
##########
@@ -0,0 +1,743 @@
+package main
+
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+import (
+	"database/sql"
+	"encoding/base64"
+	"encoding/json"
+	"errors"
+	"fmt"
+	"log"
+	"strconv"
+	"strings"
+
+	"github.com/apache/trafficcontrol/lib/go-tc"
+	util "github.com/apache/trafficcontrol/lib/go-util"
+
+	_ "github.com/lib/pq"
+)
+
+// PGConfig represents the configuration options available to the PG backend
+type PGConfig struct {
+	Host     string `json:"host"`
+	Port     string `json:"port"`
+	User     string `json:"user"`
+	Password string `json:"password"`
+	SSLMode  string `json:"sslmode"`
+	Database string `json:"database"`
+	Key      string `json:"aesKey"`
+	AESKey   []byte
+}
+
+// PGBackend is the Postgres implementation of TVBackend
+type PGBackend struct {
+	sslKey pgSSLKeyTable
+	dnssec pgDNSSecTable
+	uri    pgURISignKeyTable
+	url    pgURLSigKeyTable
+	cfg    PGConfig
+	db     *sql.DB
+}
+
+// String returns a high level overview of the backend and its keys
+func (pg *PGBackend) String() string {
+	data := fmt.Sprintf("PG server %v@%v:%v\n", pg.cfg.User, pg.cfg.Host, pg.cfg.Port)
+	data += fmt.Sprintf("\tSSL Keys: %v\n", len(pg.sslKey.Records))
+	data += fmt.Sprintf("\tDNSSec Keys: %v\n", len(pg.dnssec.Records))
+	data += fmt.Sprintf("\tURI Keys: %v\n", len(pg.uri.Records))
+	data += fmt.Sprintf("\tURL Keys: %v\n", len(pg.url.Records))
+	return data
+}
+
+// Name returns the name for this backend
+func (pg *PGBackend) Name() string {
+	return "PG"
+}
+
+// ReadConfig takes in a filename and will read it into the backends config
+func (pg *PGBackend) ReadConfig(s string) error {
+	err := UnmarshalConfig(s, &pg.cfg)
+	if err != nil {
+		return err
+	}
+
+	pg.cfg.AESKey, err = base64.StdEncoding.DecodeString(pg.cfg.Key)
+	if err != nil {
+		return fmt.Errorf("unable to decode PG AESKey: %w", err)
+	}
+	return nil
+}
+
+// Insert takes the current keys and inserts them into the backend DB
+func (pg *PGBackend) Insert() error {
+	if err := pg.sslKey.insertKeys(pg.db); err != nil {
+		return err
+	}
+	if err := pg.dnssec.insertKeys(pg.db); err != nil {
+		return err
+	}
+	if err := pg.url.insertKeys(pg.db); err != nil {
+		return err
+	}
+	if err := pg.uri.insertKeys(pg.db); err != nil {
+		return err
+	}
+	return nil
+}
+
+// Start initiates the connection to the backend DB
+func (pg *PGBackend) Start() error {
+	sqlStr := fmt.Sprintf("postgres://%s:%s@%s:%s/%s?sslmode=%s", pg.cfg.User, pg.cfg.Password, pg.cfg.Host, pg.cfg.Port, pg.cfg.Database, pg.cfg.SSLMode)
+	db, err := sql.Open("postgres", sqlStr)
+	if err != nil {
+		return fmt.Errorf("unable to start PG client: %w", err)
+	}
+
+	pg.db = db
+	pg.sslKey = pgSSLKeyTable{}
+	pg.dnssec = pgDNSSecTable{}
+	pg.url = pgURLSigKeyTable{}
+	pg.uri = pgURISignKeyTable{}
+
+	return nil
+}
+
+// ValidateKey validates that the keys are valid (in most cases, certain fields are not null)
+func (pg *PGBackend) ValidateKey() []string {
+	var errors []string
+	if errs := pg.sslKey.validate(); errs != nil {
+		errors = append(errors, errs...)
+	}
+	if errs := pg.dnssec.validate(); errs != nil {
+		errors = append(errors, errs...)
+	}
+	if errs := pg.uri.validate(); errs != nil {
+		errors = append(errors, errs...)
+	}
+	if errs := pg.url.validate(); errs != nil {
+		errors = append(errors, errs...)
+	}
+	return errors
+}
+
+// Close terminates the connection to the backend DB
+func (pg *PGBackend) Close() error {
+	return pg.db.Close()
+}
+
+// Ping checks the connection to the backend DB
+func (pg *PGBackend) Ping() error {
+	return pg.db.Ping()
+}
+
+// Fetch gets all of the keys from the backend DB
+func (pg *PGBackend) Fetch() error {
+	if err := pg.sslKey.gatherKeys(pg.db); err != nil {
+		return err
+	}
+
+	if err := pg.dnssec.gatherKeys(pg.db); err != nil {
+		return err
+	}
+
+	if err := pg.url.gatherKeys(pg.db); err != nil {
+		return err
+	}
+
+	if err := pg.uri.gatherKeys(pg.db); err != nil {
+		return err
+	}
+
+	return nil
+}
+
+// GetSSLKeys converts the backends internal key representation into the common representation (SSLKey)
+func (pg *PGBackend) GetSSLKeys() ([]SSLKey, error) {
+	if err := pg.sslKey.decrypt(pg.cfg.AESKey); err != nil {
+		return nil, err
+	}
+	return pg.sslKey.toGeneric(), nil
+}
+
+// SetSSLKeys takes in keys and converts & encrypts the data into the backends internal format
+func (pg *PGBackend) SetSSLKeys(keys []SSLKey) error {
+	pg.sslKey.fromGeneric(keys)
+	return pg.sslKey.encrypt(pg.cfg.AESKey)
+}
+
+// GetDNSSecKeys converts the backends internal key representation into the common representation (DNSSecKey)
+func (pg *PGBackend) GetDNSSecKeys() ([]DNSSecKey, error) {
+	if err := pg.dnssec.decrypt(pg.cfg.AESKey); err != nil {
+		return nil, err
+	}
+	return pg.dnssec.toGeneric(), nil
+}
+
+// SetDNSSecKeys takes in keys and converts & encrypts the data into the backends internal format
+func (pg *PGBackend) SetDNSSecKeys(keys []DNSSecKey) error {
+	pg.dnssec.fromGeneric(keys)
+	return pg.dnssec.encrypt(pg.cfg.AESKey)
+}
+
+// GetURISignKeys converts the pg internal key representation into the common representation (URISignKey)
+func (pg *PGBackend) GetURISignKeys() ([]URISignKey, error) {
+	if err := pg.uri.decrypt(pg.cfg.AESKey); err != nil {
+		return nil, err
+	}
+	return pg.uri.toGeneric(), nil
+}
+
+// SetURISignKeys takes in keys and converts & encrypts the data into the backends internal format
+func (pg *PGBackend) SetURISignKeys(keys []URISignKey) error {
+	pg.uri.fromGeneric(keys)
+	return pg.uri.encrypt(pg.cfg.AESKey)
+}
+
+// GetURLSigKeys converts the backends internal key representation into the common representation (URLSigKey)
+func (pg *PGBackend) GetURLSigKeys() ([]URLSigKey, error) {
+	if err := pg.url.decrypt(pg.cfg.AESKey); err != nil {
+		return nil, err
+	}
+	return pg.url.toGeneric(), nil
+}
+
+// SetURLSigKeys takes in keys and converts & encrypts the data into the backends internal format
+func (pg *PGBackend) SetURLSigKeys(keys []URLSigKey) error {
+	pg.url.fromGeneric(keys)
+	return pg.url.encrypt(pg.cfg.AESKey)
+}
+
+type pgCommonRecord struct {
+	DataEncrypted []byte
+}
+
+type pgDNSSecRecord struct {
+	Key tc.DNSSECKeysTrafficVault
+	CDN string
+	pgCommonRecord
+}
+type pgDNSSecTable struct {
+	Records []pgDNSSecRecord
+}
+
+func (tbl *pgDNSSecTable) gatherKeys(db *sql.DB) error {
+	sz, err := getSize(db, "dnssec")
+	if err != nil {
+		log.Println("PGDNSSec gatherKeys: unable to determine size of dnssec table")
+	}
+	tbl.Records = make([]pgDNSSecRecord, sz)
+
+	rows, err := db.Query("SELECT cdn, data from dnssec")
+	if err != nil {
+		return fmt.Errorf("PGDNSSec gatherKeys: unable to query: %w", err)
+	}
+	defer rows.Close()
+	i := 0
+	for rows.Next() {
+		if i > len(tbl.Records)-1 {
+			return fmt.Errorf("PGDNSSec gatherKeys got more results than expected %v", len(tbl.Records))
+		}
+		err := rows.Scan(&tbl.Records[i].CDN, &tbl.Records[i].DataEncrypted)
+		if err != nil {
+			return fmt.Errorf("PGDNSSec gatherKeys unable to scan row: %w", err)
+		}
+		i += 1
+	}
+	return nil
+}
+func (tbl *pgDNSSecTable) decrypt(aesKey []byte) error {
+	for i, _ := range tbl.Records {
+		err := decryptInto(aesKey, tbl.Records[i].DataEncrypted, &tbl.Records[i].Key)
+		if err != nil {
+			return fmt.Errorf("unable to decrypt into keys: %w", err)
+		}
+	}
+	return nil
+}
+func (tbl *pgDNSSecTable) encrypt(aesKey []byte) error {
+	for i, dns := range tbl.Records {
+		data, err := json.Marshal(&dns.Key)
+		if err != nil {
+			return fmt.Errorf("encrypt issue marshalling keys: %w", err)
+		}
+		dat, err := encrypt(data, aesKey)
+		if err != nil {
+			return fmt.Errorf("encrypt error: %w", err)
+		}
+		tbl.Records[i].DataEncrypted = dat
+	}
+	return nil
+}
+func (tbl *pgDNSSecTable) toGeneric() []DNSSecKey {
+	keys := make([]DNSSecKey, len(tbl.Records))
+
+	for i, record := range tbl.Records {
+		keys[i] = DNSSecKey{
+			CDN:                    record.CDN,
+			DNSSECKeysTrafficVault: record.Key,
+		}
+	}
+
+	return keys
+}
+func (tbl *pgDNSSecTable) fromGeneric(keys []DNSSecKey) {
+	tbl.Records = make([]pgDNSSecRecord, len(keys))
+
+	for i, key := range keys {
+		tbl.Records[i] = pgDNSSecRecord{
+			Key: key.DNSSECKeysTrafficVault,
+			CDN: key.CDN,
+			pgCommonRecord: pgCommonRecord{
+				DataEncrypted: nil,
+			},
+		}
+	}
+}
+func (tbl *pgDNSSecTable) validate() []string {
+	for i, record := range tbl.Records {
+		if record.DataEncrypted == nil && len(record.Key) > 0 {
+			return []string{fmt.Sprintf("DNSSEC Key %v: DataEncrypted is blank!", i)}
+		}
+	}
+	return nil
+}
+func (tbl *pgDNSSecTable) insertKeys(db *sql.DB) error {
+	queryBase := "INSERT INTO dnssec (cdn, data) VALUES "
+	stride := 2
+	queryArgs := make([]interface{}, len(tbl.Records)*stride)
+	for i, record := range tbl.Records {
+		j := i * stride
+		queryArgs[j] = record.CDN
+		queryArgs[j+1] = record.DataEncrypted
+	}
+	return insertIntoTable(db, queryBase, stride, queryArgs)
+}
+
+type pgSSLKeyRecord struct {
+	Keys tc.DeliveryServiceSSLKeys
+	pgCommonRecord
+
+	// These records are stored on the table but are duplicated
+	DeliveryService string
+	CDN             string
+}
+type pgSSLKeyTable struct {
+	Records []pgSSLKeyRecord
+}
+
+func (tbl *pgSSLKeyTable) insertKeys(db *sql.DB) error {
+	queryBase := "INSERT INTO sslkey (deliveryservice, data, cdn, version) VALUES "
+	duplicateKeys := 2
+	stride := 4
+	queryArgs := make([]interface{}, len(tbl.Records)*stride*duplicateKeys)
+	for i, record := range tbl.Records {
+		j := i * duplicateKeys * stride
+
+		queryArgs[j] = record.DeliveryService
+		queryArgs[j+1] = record.DataEncrypted
+		queryArgs[j+2] = record.CDN
+		queryArgs[j+3] = record.Keys.Version.String()
+
+		queryArgs[j+4] = record.DeliveryService
+		queryArgs[j+5] = record.DataEncrypted
+		queryArgs[j+6] = record.CDN
+		queryArgs[j+7] = "latest"
+	}
+	return insertIntoTable(db, queryBase, 4, queryArgs)
+}
+func (tbl *pgSSLKeyTable) gatherKeys(db *sql.DB) error {
+	sz, err := getSize(db, "sslkey WHERE version='latest'")
+	if err != nil {
+		return fmt.Errorf("PGSSLKey gatherKeys unable to determine size of sslkey table: %w", err)
+	}
+	tbl.Records = make([]pgSSLKeyRecord, sz)
+
+	rows, err := db.Query("SELECT data, deliveryservice, cdn from sslkey WHERE version = 'latest'")
+	if err != nil {
+		return fmt.Errorf("PGSSLKey gatherKeys unable to query: %w", err)
+	}
+	defer rows.Close()
+	i := 0
+	for rows.Next() {
+		if i > len(tbl.Records)-1 {
+			return errors.New("PGSSLKey gatherKeys: got more results than expected")
+		}
+		err := rows.Scan(&tbl.Records[i].DataEncrypted, &tbl.Records[i].DeliveryService, &tbl.Records[i].CDN)
+		if err != nil {
+			return fmt.Errorf("PGSSLKey gatherKeys unable to scan row: %w", err)
+		}
+		i += 1
+	}
+	return nil
+}
+func (tbl *pgSSLKeyTable) decrypt(aesKey []byte) error {
+	for i, dns := range tbl.Records {
+		err := decryptInto(aesKey, dns.DataEncrypted, &tbl.Records[i].Keys)
+		if err != nil {
+			return fmt.Errorf("unable to decrypt into keys: %w", err)
+		}
+	}
+	return nil
+}
+func (tbl *pgSSLKeyTable) encrypt(aesKey []byte) error {
+	for i, dns := range tbl.Records {
+		data, err := json.Marshal(dns.Keys)
+		if err != nil {
+			return fmt.Errorf("encrypt issue marshalling keys: %w", err)
+		}
+		dat, err := encrypt(data, aesKey)
+		if err != nil {
+			return fmt.Errorf("encrypt error: %w", err)
+		}
+		tbl.Records[i].DataEncrypted = dat
+	}
+	return nil
+}
+func (tbl *pgSSLKeyTable) toGeneric() []SSLKey {
+	keys := make([]SSLKey, len(tbl.Records))
+
+	for i, record := range tbl.Records {
+		keys[i] = SSLKey{
+			DeliveryServiceSSLKeys: record.Keys,
+		}
+	}
+	return keys
+}
+func (tbl *pgSSLKeyTable) fromGeneric(keys []SSLKey) {
+	tbl.Records = make([]pgSSLKeyRecord, len(keys))
+
+	for i, key := range keys {
+		tbl.Records[i] = pgSSLKeyRecord{
+			Keys: key.DeliveryServiceSSLKeys,
+			pgCommonRecord: pgCommonRecord{
+				DataEncrypted: nil,
+			},
+			DeliveryService: key.DeliveryService,
+			CDN:             key.CDN,
+		}
+	}
+}
+func (tbl *pgSSLKeyTable) validate() []string {
+	defaultKey := tc.DeliveryServiceSSLKeys{}
+	var errors []string
+	fmtStr := "SSL Key %v: %v"
+	for i, record := range tbl.Records {
+		if record.Keys == defaultKey {
+			errors = append(errors, fmt.Sprintf(fmtStr, i, "DS SSL Keys are default!"))
+		} else if record.Keys.Key == "" {
+			errors = append(errors, fmt.Sprintf(fmtStr, i, "Key is blank!"))
+		} else if record.Keys.CDN == "" {
+			errors = append(errors, fmt.Sprintf(fmtStr, i, "CDN is blank!"))
+		} else if record.Keys.DeliveryService == "" {
+			errors = append(errors, fmt.Sprintf(fmtStr, i, "DS is blank!"))
+		} else if record.DataEncrypted == nil {
+			errors = append(errors, fmt.Sprintf(fmtStr, i, "DataEncrypted is blank!"))
+		} else if record.Keys.Version.String() == "" {
+			errors = append(errors, fmt.Sprintf(fmtStr, i, "Version is blank!"))
+		}
+	}
+	return errors
+}
+
+type pgURLSigKeyRecord struct {
+	Keys            tc.URLSigKeys
+	DeliveryService string
+	pgCommonRecord
+}
+type pgURLSigKeyTable struct {
+	Records []pgURLSigKeyRecord
+}
+
+func (tbl *pgURLSigKeyTable) insertKeys(db *sql.DB) error {
+	queryBase := "INSERT INTO url_sig_key (deliveryservice, data) VALUES "
+	stride := 2
+	queryArgs := make([]interface{}, len(tbl.Records)*stride)
+	for i, record := range tbl.Records {
+		j := i * stride
+		queryArgs[j] = record.DeliveryService
+		queryArgs[j+1] = record.DataEncrypted
+	}
+	return insertIntoTable(db, queryBase, stride, queryArgs)
+}
+func (tbl *pgURLSigKeyTable) gatherKeys(db *sql.DB) error {
+	sz, err := getSize(db, "url_sig_key")
+	if err != nil {
+		log.Println("PGURLSigKey gatherKeys: unable to determine url_sig_key table size")
+	}
+	tbl.Records = make([]pgURLSigKeyRecord, sz)
+
+	rows, err := db.Query("SELECT deliveryservice, data from url_sig_key")
+	if err != nil {
+		return fmt.Errorf("PGURLSigKey gatherKeys error creating query: %w", err)
+	}
+	defer rows.Close()
+	i := 0
+	for rows.Next() {
+		if i > len(tbl.Records)-1 {
+			return fmt.Errorf("PGURLSigKey gatherKeys: got more results than expected %v", len(tbl.Records))
+		}
+		err := rows.Scan(&tbl.Records[i].DeliveryService, &tbl.Records[i].DataEncrypted)
+		if err != nil {
+			return fmt.Errorf("PGURLSigKey gatherKeys: unable to scan row: %w", err)
+		}
+		i += 1
+	}
+	return nil
+}
+func (tbl *pgURLSigKeyTable) decrypt(aesKey []byte) error {
+	for i, sig := range tbl.Records {
+		err := decryptInto(aesKey, sig.DataEncrypted, &tbl.Records[i].Keys)
+		if err != nil {
+			return fmt.Errorf("unable to decrypt into keys: %w", err)
+		}
+	}
+	return nil
+}
+func (tbl *pgURLSigKeyTable) encrypt(aesKey []byte) error {
+	for i, sig := range tbl.Records {
+		data, err := json.Marshal(&sig.Keys)
+		if err != nil {
+			return fmt.Errorf("encrypt issue marshalling keys: %w", err)
+		}
+
+		dat, err := encrypt(data, aesKey)
+		if err != nil {
+			return fmt.Errorf("encrypt error: %w", err)
+		}
+		tbl.Records[i].DataEncrypted = dat
+	}
+	return nil
+}
+func (tbl *pgURLSigKeyTable) toGeneric() []URLSigKey {
+	keys := make([]URLSigKey, len(tbl.Records))
+
+	for i, record := range tbl.Records {
+		keys[i] = URLSigKey{
+			DeliveryService: record.DeliveryService,
+			URLSigKeys:      record.Keys,
+		}
+	}
+	return keys
+}
+func (tbl *pgURLSigKeyTable) fromGeneric(keys []URLSigKey) {
+	tbl.Records = make([]pgURLSigKeyRecord, len(keys))
+
+	for i, key := range keys {
+		tbl.Records[i] = pgURLSigKeyRecord{
+			Keys:            key.URLSigKeys,
+			DeliveryService: key.DeliveryService,
+			pgCommonRecord: pgCommonRecord{
+				DataEncrypted: nil,
+			},
+		}
+	}
+}
+func (tbl *pgURLSigKeyTable) validate() []string {
+	for i, record := range tbl.Records {
+		if record.DataEncrypted == nil && len(record.Keys) > 0 {
+			return []string{fmt.Sprintf("URl Sig Key %v: DataEncrypted is blank!", i)}
+		}
+	}
+	return nil
+}
+
+type pgURISignKeyRecord struct {
+	Keys            map[string]tc.URISignerKeyset
+	DeliveryService string
+	pgCommonRecord
+}
+type pgURISignKeyTable struct {
+	Records []pgURISignKeyRecord
+}
+
+func (tbl *pgURISignKeyTable) insertKeys(db *sql.DB) error {
+	queryBase := "INSERT INTO uri_signing_key (deliveryservice, data) VALUES "

Review comment:
       Riak does indeed just overwrite. I've done this for most of the tables, but SSL Keys is only constrained by it's id making comparison quite expensive. Do you think it's worth to check for SSL Keys as well? Personally, I think we should just skip it.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [trafficcontrol] shamrickus commented on a change in pull request #5924: Add tool to migrate data between TV backends

Posted by GitBox <gi...@apache.org>.
shamrickus commented on a change in pull request #5924:
URL: https://github.com/apache/trafficcontrol/pull/5924#discussion_r651041593



##########
File path: tools/traffic_vault_migrate/riak.go
##########
@@ -0,0 +1,746 @@
+package main
+
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+import (
+	"crypto/tls"
+	"encoding/json"
+	"errors"
+	"fmt"
+	"time"
+
+	"github.com/apache/trafficcontrol/lib/go-rfc"
+	"github.com/apache/trafficcontrol/lib/go-tc"
+
+	"github.com/basho/riak-go-client"
+)
+
+const (
+	BUCKET_SSL     = "ssl"
+	BUCKET_DNSSEC  = "dnssec"
+	BUCKET_URL_SIG = "url_sig_keys"
+	BUCKET_URI_SIG = "cdn_uri_sig_keys"
+
+	INDEX_SSL = "sslkeys"
+
+	SCHEMA_RIAK_KEY    = "_yz_rk"
+	SCHEMA_RIAK_BUCKET = "_yz_rb"
+)
+
+var (
+	SCHEMA_SSL_FIELDS = [...]string{SCHEMA_RIAK_KEY, SCHEMA_RIAK_BUCKET}
+)
+
+// RiakConfig  represents the configuration options available to the Riak backend
+type RiakConfig struct {
+	Host          string `json:"host"`
+	Port          string `json:"port"`
+	User          string `json:"user"`
+	Password      string `json:"password"`
+	Insecure      bool   `json:"insecure"`
+	TLSVersionRaw string `json:"tlsVersion"`
+	TLSVersion    uint16
+}
+
+// RiakBackend is the Riak implementation of TVBackend
+type RiakBackend struct {
+	sslKeys     riakSSLKeyTable
+	dnssecKeys  riakDNSSecKeyTable
+	uriSignKeys riakURISignKeyTable
+	urlSigKeys  riakURLSigKeyTable
+	cfg         RiakConfig
+	cluster     *riak.Cluster
+}
+
+// String returns a high level overview of the backend and its keys
+func (rb *RiakBackend) String() string {
+	data := fmt.Sprintf("Riak server %v@%v:%v\n", rb.cfg.User, rb.cfg.Host, rb.cfg.Port)
+	data += fmt.Sprintf("\tSSL Keys: %v\n", len(rb.sslKeys.Records))
+	data += fmt.Sprintf("\tDNSSec Keys: %v\n", len(rb.dnssecKeys.Records))
+	data += fmt.Sprintf("\tURI Keys: %v\n", len(rb.uriSignKeys.Records))
+	data += fmt.Sprintf("\tURL Keys: %v\n", len(rb.urlSigKeys.Records))
+	return data
+}
+
+// Name returns the name for this backend
+func (rb *RiakBackend) Name() string {
+	return "Riak"
+}
+
+// ReadConfig takes in a filename and will read it into the backends config
+func (rb *RiakBackend) ReadConfig(s string) error {
+	err := UnmarshalConfig(s, &rb.cfg)
+	if err != nil {
+		return err
+	}
+
+	if rb.cfg.TLSVersionRaw == "10" {
+		rb.cfg.TLSVersion = tls.VersionTLS10
+	} else if rb.cfg.TLSVersionRaw == "11" {
+		rb.cfg.TLSVersion = tls.VersionTLS11
+	} else if rb.cfg.TLSVersionRaw == "12" {
+		rb.cfg.TLSVersion = tls.VersionTLS12
+	} else if rb.cfg.TLSVersionRaw == "13" {
+		rb.cfg.TLSVersion = tls.VersionTLS13
+	}
+	return nil
+}
+
+// Insert takes the current keys and inserts them into the backend DB
+func (rb *RiakBackend) Insert() error {
+	if err := rb.sslKeys.insertKeys(rb.cluster); err != nil {
+		return err
+	}
+	if err := rb.dnssecKeys.insertKeys(rb.cluster); err != nil {
+		return err
+	}
+	if err := rb.urlSigKeys.insertKeys(rb.cluster); err != nil {
+		return err
+	}
+	if err := rb.uriSignKeys.insertKeys(rb.cluster); err != nil {
+		return err
+	}
+	return nil
+}
+
+// ValidateKey validates that the keys are valid (in most cases, certain fields are not null)
+func (rb *RiakBackend) ValidateKey() []string {
+	errors := []string{}
+	if errs := rb.sslKeys.validate(); errs != nil {
+		errors = append(errors, errs...)
+	}
+	if errs := rb.dnssecKeys.validate(); errs != nil {
+		errors = append(errors, errs...)
+	}
+	if errs := rb.uriSignKeys.validate(); errs != nil {
+		errors = append(errors, errs...)
+	}
+	if errs := rb.urlSigKeys.validate(); errs != nil {
+		errors = append(errors, errs...)
+	}
+
+	return errors
+}
+
+// SetSSLKeys takes in keys and converts & encrypts the data into the backends internal format
+func (rb *RiakBackend) SetSSLKeys(keys []SSLKey) error {
+	rb.sslKeys.fromGeneric(keys)
+	return nil
+}
+
+// SetDNSSecKeys takes in keys and converts & encrypts the data into the backends internal format
+func (rb *RiakBackend) SetDNSSecKeys(keys []DNSSecKey) error {
+	rb.dnssecKeys.fromGeneric(keys)
+	return nil
+}
+
+// SetURISignKeys takes in keys and converts & encrypts the data into the backends internal format
+func (rb *RiakBackend) SetURISignKeys(keys []URISignKey) error {
+	rb.uriSignKeys.fromGeneric(keys)
+	return nil
+}
+
+// SetURLSigKeys takes in keys and converts & encrypts the data into the backends internal format
+func (rb *RiakBackend) SetURLSigKeys(keys []URLSigKey) error {
+	rb.urlSigKeys.fromGeneric(keys)
+	return nil
+}
+
+// Start initiates the connection to the backend DB
+func (rb *RiakBackend) Start() error {
+	tlsConfig := &tls.Config{
+		InsecureSkipVerify: !rb.cfg.Insecure,
+		MaxVersion:         rb.cfg.TLSVersion,
+	}
+	auth := &riak.AuthOptions{
+		User:      rb.cfg.User,
+		Password:  rb.cfg.Password,
+		TlsConfig: tlsConfig,
+	}
+
+	cluster, err := getRiakCluster(rb.cfg, auth)
+	if err != nil {
+		return err
+	}
+	if err := cluster.Start(); err != nil {
+		return fmt.Errorf("unable to start riak cluster: %w", err)
+	}
+
+	rb.cluster = cluster
+	rb.sslKeys = riakSSLKeyTable{}
+	rb.dnssecKeys = riakDNSSecKeyTable{}
+	rb.urlSigKeys = riakURLSigKeyTable{}
+	rb.uriSignKeys = riakURISignKeyTable{}
+	return nil
+}
+
+// Close terminates the connection to the backend DB
+func (rb *RiakBackend) Close() error {
+	if err := rb.cluster.Stop(); err != nil {
+		return err
+	}
+	return nil
+}
+
+// Ping checks the connection to the backend DB
+func (rb *RiakBackend) Ping() error {
+	return ping(rb.cluster)
+}
+
+// GetSSLKeys converts the backends internal key representation into the common representation (SSLKey)
+func (rb *RiakBackend) GetSSLKeys() ([]SSLKey, error) {
+	return rb.sslKeys.toGeneric(), nil
+}
+
+// GetDNSSecKeys converts the backends internal key representation into the common representation (DNSSecKey)
+func (rb *RiakBackend) GetDNSSecKeys() ([]DNSSecKey, error) {
+	return rb.dnssecKeys.toGeneric(), nil
+}
+
+// GetURISignKeys converts the pg internal key representation into the common representation (URISignKey)
+func (rb *RiakBackend) GetURISignKeys() ([]URISignKey, error) {
+	return rb.uriSignKeys.toGeneric(), nil
+}
+
+// GetURLSigKeys converts the backends internal key representation into the common representation (URLSigKey)
+func (rb *RiakBackend) GetURLSigKeys() ([]URLSigKey, error) {
+	return rb.urlSigKeys.toGeneric(), nil
+}
+
+// Fetch gets all of the keys from the backend DB
+func (rb *RiakBackend) Fetch() error {
+	if err := rb.sslKeys.gatherKeys(rb.cluster); err != nil {
+		return err
+	}
+	if err := rb.dnssecKeys.gatherKeys(rb.cluster); err != nil {
+		return err
+	}
+	if err := rb.urlSigKeys.gatherKeys(rb.cluster); err != nil {
+		return err
+	}
+	if err := rb.uriSignKeys.gatherKeys(rb.cluster); err != nil {
+		return err
+	}
+
+	return nil
+}
+
+type riakSSLKeyRecord struct {
+	tc.DeliveryServiceSSLKeys
+}
+type riakSSLKeyTable struct {
+	Records []riakSSLKeyRecord
+}
+
+func (tbl *riakSSLKeyTable) gatherKeys(cluster *riak.Cluster) error {
+	searchDocs, err := search(cluster, INDEX_SSL, "cdn:*", fmt.Sprintf("%v:*latest", SCHEMA_RIAK_KEY), 1000, SCHEMA_SSL_FIELDS[:])
+	if err != nil {
+		return fmt.Errorf("RiakSSLKey gatherKeys: %w", err)
+	}
+
+	tbl.Records = make([]riakSSLKeyRecord, len(searchDocs))
+	for i, doc := range searchDocs {
+		objs, err := getObject(cluster, doc.Bucket, doc.Key)
+		if err != nil {
+			return err
+		}
+		if len(objs) < 1 {
+			return fmt.Errorf("RiakSSLKey gatherKeys unable to find any objects with key %v and bucket %v, but search results were returned", doc.Key, doc.Bucket)
+		}
+		if len(objs) > 1 {
+			return fmt.Errorf("RiakSSLKey gatherKeys more than 1 ssl key record found %v\n", len(objs))
+		}
+		var obj tc.DeliveryServiceSSLKeys
+		if err = json.Unmarshal(objs[0].Value, &obj); err != nil {
+			return fmt.Errorf("RiakSSLKey gatherKeys unable to unmarshal object into tc.DeliveryServiceSSLKeys: %w", err)
+		}
+		tbl.Records[i] = riakSSLKeyRecord{
+			DeliveryServiceSSLKeys: obj,
+		}
+	}
+	return nil
+}
+func (tbl *riakSSLKeyTable) toGeneric() []SSLKey {
+	keys := make([]SSLKey, len(tbl.Records))
+
+	for i, record := range tbl.Records {
+		keys[i] = SSLKey{
+			DeliveryServiceSSLKeys: record.DeliveryServiceSSLKeys,
+		}
+	}
+
+	return keys
+}
+func (tbl *riakSSLKeyTable) fromGeneric(keys []SSLKey) {
+	tbl.Records = make([]riakSSLKeyRecord, len(keys))
+
+	for i, record := range keys {
+		tbl.Records[i] = riakSSLKeyRecord{
+			DeliveryServiceSSLKeys: record.DeliveryServiceSSLKeys,
+		}
+	}
+}
+func (tbl *riakSSLKeyTable) insertKeys(cluster *riak.Cluster) error {
+	for _, record := range tbl.Records {
+		objBytes, err := json.Marshal(record.DeliveryServiceSSLKeys)
+		if err != nil {
+			return fmt.Errorf("RiakSSLKey insertKeys failed to unmarshal keys: %w", err)
+		}
+		err = setObject(cluster, makeRiakObject(objBytes, record.DeliveryService+"-"+record.Version.String()), BUCKET_SSL)
+		if err != nil {
+			return fmt.Errorf("RiakSSLKey insertKeys: %w", err)
+		}
+		err = setObject(cluster, makeRiakObject(objBytes, record.DeliveryService+"-latest"), BUCKET_SSL)
+		if err != nil {
+			return fmt.Errorf("RiakSSLKey insertKeys: %w", err)
+		}
+	}
+	return nil
+}
+func (tbl *riakSSLKeyTable) validate() []string {
+	errs := []string{}
+	for i, record := range tbl.Records {
+		if record.DeliveryService == "" {
+			errs = append(errs, fmt.Sprintf("SSL Key #%v: Delivery Service is blank!", i))
+		}
+		if record.CDN == "" {
+			errs = append(errs, fmt.Sprintf("SSL Key #%v: CDN is blank!", i))
+		}
+		if record.Version.String() == "" {
+			errs = append(errs, fmt.Sprintf("SSL Key #%v: Version is blank!", i))
+		}
+	}
+	return errs
+}
+
+type riakDNSSecKeyRecord struct {
+	CDN string
+	Key tc.DNSSECKeysRiak
+}
+type riakDNSSecKeyTable struct {
+	Records []riakDNSSecKeyRecord
+}
+
+func (tbl *riakDNSSecKeyTable) gatherKeys(cluster *riak.Cluster) error {
+	tbl.Records = []riakDNSSecKeyRecord{}
+	objs, err := getObjects(cluster, BUCKET_DNSSEC)
+	if err != nil {
+		return fmt.Errorf("RiakDNSSecKey gatherKeys: %w", err)
+	}
+	for _, obj := range objs {
+		key := tc.DNSSECKeysRiak{}
+		if err := json.Unmarshal(obj.Value, &key); err != nil {
+			return fmt.Errorf("RiakDNSSecKey gatherKeys unable to unmarshal object to tc.DNSSECKeysRiak: %w", err)
+		}
+		tbl.Records = append(tbl.Records, riakDNSSecKeyRecord{
+			CDN: obj.Key,
+			Key: key,
+		})
+	}
+	return nil
+}
+func (tbl *riakDNSSecKeyTable) toGeneric() []DNSSecKey {
+	keys := make([]DNSSecKey, len(tbl.Records))
+
+	for i, record := range tbl.Records {
+		keys[i] = DNSSecKey{
+			CDN:                    record.CDN,
+			DNSSECKeysTrafficVault: tc.DNSSECKeysTrafficVault(record.Key),
+		}
+	}
+
+	return keys
+}
+func (tbl *riakDNSSecKeyTable) fromGeneric(keys []DNSSecKey) {
+	tbl.Records = make([]riakDNSSecKeyRecord, len(keys))
+
+	for i, record := range keys {
+		tbl.Records[i] = riakDNSSecKeyRecord{
+			CDN: record.CDN,
+			Key: tc.DNSSECKeysRiak(record.DNSSECKeysTrafficVault),
+		}
+	}
+}
+func (tbl *riakDNSSecKeyTable) insertKeys(cluster *riak.Cluster) error {
+	for _, record := range tbl.Records {
+		objBytes, err := json.Marshal(record.Key)
+		if err != nil {
+			return fmt.Errorf("RiakDNSSecKey insertKeys error marshalling keys: %w", err)
+		}
+
+		err = setObject(cluster, makeRiakObject(objBytes, record.CDN), BUCKET_DNSSEC)
+		if err != nil {
+			return fmt.Errorf("RiakDNSSecKey insertKeys: %w", err)
+		}
+	}
+	return nil
+}
+func (tbl *riakDNSSecKeyTable) validate() []string {
+	errs := []string{}
+	for i, record := range tbl.Records {
+		if record.CDN == "" {
+			errs = append(errs, fmt.Sprintf("DNSSec Key #%v: CDN is blank!", i))
+		}
+	}
+	return errs
+}
+
+type riakURLSigKeyRecord struct {
+	Key             tc.URLSigKeys
+	DeliveryService string
+}
+type riakURLSigKeyTable struct {
+	Records []riakURLSigKeyRecord
+}
+
+func (tbl *riakURLSigKeyTable) gatherKeys(cluster *riak.Cluster) error {
+	tbl.Records = []riakURLSigKeyRecord{}
+	objs, err := getObjects(cluster, BUCKET_URL_SIG)
+	if err != nil {
+		return fmt.Errorf("RiakURLSigKey gatherKeys: %w", err)
+	}
+	for _, obj := range objs {
+		key := tc.URLSigKeys{}
+		if err := json.Unmarshal(obj.Value, &key); err != nil {
+			return fmt.Errorf("RiakURLSigKey gatherKeys unable to unamrshal object into tc.URLSigKeys: %w", err)
+		}
+		tbl.Records = append(tbl.Records, riakURLSigKeyRecord{
+			DeliveryService: obj.Key,
+			Key:             key,
+		})
+	}
+	return nil
+}
+func (tbl *riakURLSigKeyTable) toGeneric() []URLSigKey {
+	keys := make([]URLSigKey, len(tbl.Records))
+
+	for i, record := range tbl.Records {
+		keys[i] = URLSigKey{
+			URLSigKeys:      record.Key,
+			DeliveryService: record.DeliveryService,
+		}
+	}
+
+	return keys
+}
+func (tbl *riakURLSigKeyTable) fromGeneric(keys []URLSigKey) {
+	tbl.Records = make([]riakURLSigKeyRecord, len(keys))
+	for i, key := range keys {
+		tbl.Records[i] = riakURLSigKeyRecord{
+			Key:             key.URLSigKeys,
+			DeliveryService: key.DeliveryService,
+		}
+	}
+}
+func (tbl *riakURLSigKeyTable) insertKeys(cluster *riak.Cluster) error {
+	for _, record := range tbl.Records {
+		objBytes, err := json.Marshal(record.Key)
+		if err != nil {
+			return fmt.Errorf("RiakURLSigKey insertKeys unable to marshal keys: %w", err)
+		}
+
+		err = setObject(cluster, makeRiakObject(objBytes, "url_sig_"+record.DeliveryService+".config"), BUCKET_URL_SIG)
+		if err != nil {
+			return fmt.Errorf("RiakURLSigKey insertKeys: %w", err)
+		}
+	}
+	return nil
+}
+func (tbl *riakURLSigKeyTable) validate() []string {
+	errs := []string{}
+	for i, record := range tbl.Records {
+		if record.DeliveryService == "" {
+			errs = append(errs, fmt.Sprintf("URL Key #%v: Delivery Service is blank!", i))
+		}
+	}
+	return errs
+}
+
+type riakURISignKeyRecord struct {
+	Key             map[string]tc.URISignerKeyset
+	DeliveryService string
+}
+type riakURISignKeyTable struct {
+	Records []riakURISignKeyRecord
+}
+
+func (tbl *riakURISignKeyTable) gatherKeys(cluster *riak.Cluster) error {
+	tbl.Records = []riakURISignKeyRecord{}
+	objs, err := getObjects(cluster, BUCKET_URI_SIG)
+	if err != nil {
+		return fmt.Errorf("RiakURISignKey gatherKeys: %w", err)
+	}
+	for _, obj := range objs {
+		key := map[string]tc.URISignerKeyset{}
+		if err := json.Unmarshal(obj.Value, &key); err != nil {
+			return fmt.Errorf("RiakURISignKey gatherKeys unable to unmarshal object into map[string]tc.URISignerKeySet: %w", err)
+		}
+
+		tbl.Records = append(tbl.Records, riakURISignKeyRecord{
+			DeliveryService: obj.Key,
+			Key:             key,
+		})
+	}
+	return nil
+}
+func (tbl *riakURISignKeyTable) toGeneric() []URISignKey {
+	keys := make([]URISignKey, len(tbl.Records))
+
+	for i, record := range tbl.Records {
+		keys[i] = URISignKey{
+			DeliveryService: record.DeliveryService,
+			Keys:            record.Key,
+		}
+	}
+
+	return keys
+}
+func (tbl *riakURISignKeyTable) fromGeneric(keys []URISignKey) {
+	tbl.Records = make([]riakURISignKeyRecord, len(keys))
+
+	for i, record := range keys {
+		tbl.Records[i] = riakURISignKeyRecord{
+			Key:             record.Keys,
+			DeliveryService: record.DeliveryService,
+		}
+	}
+}
+func (tbl *riakURISignKeyTable) insertKeys(cluster *riak.Cluster) error {
+	for _, record := range tbl.Records {
+		objBytes, err := json.Marshal(record.Key)
+		if err != nil {
+			return fmt.Errorf("RiakURISignKey insertKeys: unable to marshal key: %w", err)
+		}
+
+		err = setObject(cluster, makeRiakObject(objBytes, record.DeliveryService), BUCKET_URI_SIG)
+		if err != nil {
+			return fmt.Errorf("RiakURISignKey insertKeys: %w", err)
+		}
+	}
+	return nil
+}
+func (tbl *riakURISignKeyTable) validate() []string {
+	errs := []string{}
+	for i, record := range tbl.Records {
+		if record.DeliveryService == "" {
+			errs = append(errs, fmt.Sprintf("URI Key #%v: Delivery Service is blank!", i))
+		}
+	}
+	return errs
+}
+
+// Riak functions
+func makeRiakObject(data []byte, key string) *riak.Object {
+	return &riak.Object{
+		ContentType:     rfc.ApplicationJSON,
+		Charset:         "utf-8",
+		ContentEncoding: "utf-8",
+		Key:             key,
+		Value:           data,
+	}
+}
+func getObjects(cluster *riak.Cluster, bucket string) ([]*riak.Object, error) {
+	objs := []*riak.Object{}
+	keys, err := listKeys(cluster, bucket)
+	if err != nil {
+		return nil, err
+	}
+	for _, key := range keys {
+		objects, err := getObject(cluster, bucket, key)
+		if err != nil {
+			return nil, err
+		}
+		if len(objects) > 1 {
+			return nil, fmt.Errorf("Unexpected number of objects %v, ignoring\n", len(objects))

Review comment:
       Good catch, used to be a log with a continue.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [trafficcontrol] ocket8888 commented on a change in pull request #5924: Add tool to migrate data between TV backends

Posted by GitBox <gi...@apache.org>.
ocket8888 commented on a change in pull request #5924:
URL: https://github.com/apache/trafficcontrol/pull/5924#discussion_r652056597



##########
File path: docs/source/tools/traffic_vault_migrate.rst
##########
@@ -0,0 +1,112 @@
+..
+..
+.. Licensed under the Apache License, Version 2.0 (the "License");
+.. you may not use this file except in compliance with the License.
+.. You may obtain a copy of the License at
+..
+..     http://www.apache.org/licenses/LICENSE-2.0
+..
+.. Unless required by applicable law or agreed to in writing, software
+.. distributed under the License is distributed on an "AS IS" BASIS,
+.. WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+.. See the License for the specific language governing permissions and
+.. limitations under the License.
+..
+
+.. _traffic_vault_migrate:
+
+#########################
+Traffic Vault Migrate
+#########################
+The ``traffic_vault_migrate`` tool - located at :file:`tools/traffic_vault_migrate/traffic_vault_migrate.go` in the `Apache Traffic Control repository <https://github.com/apache/trafficcontrol>`_ -
+is used to transfer TV keys between database servers. It interfaces directly with each backend so Traffic Ops/Vault being available is not a requirement.
+The tool assumes that the schema for each backend is already setup as according to the :ref:`admin setup <traffic_vault_admin>`.
+
+.. program:: traffic_vault_migrate
+
+Usage
+===========
+``traffic_vault_migrate [-cdhmr] [-f value] [-g value] [-o value] [-t value]``
+
+.. option:: -c, --compare
+
+		Compare 'to' and 'from' backend keys. Will fetch keys from the dbs of both 'to' and 'from', sorts them by cdn/ds/version and does a deep comparison.
+
+.. option:: -d, --dump
+
+		Write keys (from 'from' server) to disk in the folder 'dump' with the unix permissions 0640.
+
+		.. warning:: This can write potentially sensitive information to disk, use with care.
+
+.. option:: -f, --fromCfg=CFG
+
+		From server config file (default "riak.json")
+
+.. option:: -g, --toCfg=CFG
+
+		To server config file (default "pg.json")
+
+.. option:: -h, --help
+
+		Displays usage information
+
+.. option:: -o, --toType=TYPE
+
+		From server types (Riak|PG) (default "PG")
+
+.. option:: -m, --noConfirm
+
+		Do not require confirmation before inserting records
+
+.. option:: -r, --dry
+
+		Do not perform writes. Will do a basic output of the keys on the 'from' backend.
+
+.. option:: -t, --fromType=TYPE

Review comment:
       options that take option-arguments and have long and short forms should specify it for both forms e.g.
   ```rst
   .. option:: -t TYPE, --from-type TYPE
   ```
   
   The <kbd>=</kbd> can't be used with the short form, but using a space in the short form and an <kbd>=</kbd> in the long form doesn't hurt anything afaik.

##########
File path: tools/traffic_vault_migrate/postgres.go
##########
@@ -0,0 +1,742 @@
+package main
+
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+import (
+	"database/sql"
+	"encoding/base64"
+	"encoding/json"
+	"errors"
+	"fmt"
+	"strconv"
+	"strings"
+
+	"github.com/apache/trafficcontrol/lib/go-log"
+	"github.com/apache/trafficcontrol/lib/go-tc"
+	util "github.com/apache/trafficcontrol/lib/go-util"
+
+	_ "github.com/lib/pq"
+)
+
+// PGConfig represents the configuration options available to the PG backend
+type PGConfig struct {
+	Host     string `json:"host"`
+	Port     string `json:"port"`
+	User     string `json:"user"`
+	Password string `json:"password"`
+	SSLMode  string `json:"sslmode"`
+	Database string `json:"database"`
+	Key      string `json:"aesKey"`
+	AESKey   []byte
+}
+
+// PGBackend is the Postgres implementation of TVBackend
+type PGBackend struct {
+	sslKey pgSSLKeyTable
+	dnssec pgDNSSecTable
+	uri    pgURISignKeyTable
+	url    pgURLSigKeyTable
+	cfg    PGConfig
+	db     *sql.DB
+}
+
+// String returns a high level overview of the backend and its keys
+func (pg *PGBackend) String() string {
+	data := fmt.Sprintf("PG server %s@%s:%s\n", pg.cfg.User, pg.cfg.Host, pg.cfg.Port)
+	data += fmt.Sprintf("\tSSL Keys: %d\n", len(pg.sslKey.Records))
+	data += fmt.Sprintf("\tDNSSec Keys: %d\n", len(pg.dnssec.Records))
+	data += fmt.Sprintf("\tURI Keys: %d\n", len(pg.uri.Records))
+	data += fmt.Sprintf("\tURL Keys: %d\n", len(pg.url.Records))
+	return data
+}
+
+// Name returns the name for this backend
+func (pg *PGBackend) Name() string {
+	return "PG"
+}
+
+// ReadConfigFile takes in a filename and will read it into the backends config
+func (pg *PGBackend) ReadConfigFile(configFile string) error {
+	var err error
+	if err = UnmarshalConfig(configFile, &pg.cfg); err != nil {
+		return err
+	}
+
+	if pg.cfg.AESKey, err = base64.StdEncoding.DecodeString(pg.cfg.Key); err != nil {
+		return fmt.Errorf("unable to decode PG AESKey: %w", err)
+	}
+	return nil
+}
+
+// Insert takes the current keys and inserts them into the backend DB

Review comment:
       nit: GoDoc missing punctuation.

##########
File path: tools/traffic_vault_migrate/postgres.go
##########
@@ -0,0 +1,742 @@
+package main
+
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+import (
+	"database/sql"
+	"encoding/base64"
+	"encoding/json"
+	"errors"
+	"fmt"
+	"strconv"
+	"strings"
+
+	"github.com/apache/trafficcontrol/lib/go-log"
+	"github.com/apache/trafficcontrol/lib/go-tc"
+	util "github.com/apache/trafficcontrol/lib/go-util"
+
+	_ "github.com/lib/pq"
+)
+
+// PGConfig represents the configuration options available to the PG backend
+type PGConfig struct {
+	Host     string `json:"host"`
+	Port     string `json:"port"`
+	User     string `json:"user"`
+	Password string `json:"password"`
+	SSLMode  string `json:"sslmode"`
+	Database string `json:"database"`
+	Key      string `json:"aesKey"`
+	AESKey   []byte
+}
+
+// PGBackend is the Postgres implementation of TVBackend
+type PGBackend struct {
+	sslKey pgSSLKeyTable
+	dnssec pgDNSSecTable
+	uri    pgURISignKeyTable
+	url    pgURLSigKeyTable
+	cfg    PGConfig
+	db     *sql.DB
+}
+
+// String returns a high level overview of the backend and its keys
+func (pg *PGBackend) String() string {
+	data := fmt.Sprintf("PG server %s@%s:%s\n", pg.cfg.User, pg.cfg.Host, pg.cfg.Port)
+	data += fmt.Sprintf("\tSSL Keys: %d\n", len(pg.sslKey.Records))
+	data += fmt.Sprintf("\tDNSSec Keys: %d\n", len(pg.dnssec.Records))
+	data += fmt.Sprintf("\tURI Keys: %d\n", len(pg.uri.Records))
+	data += fmt.Sprintf("\tURL Keys: %d\n", len(pg.url.Records))
+	return data
+}
+
+// Name returns the name for this backend
+func (pg *PGBackend) Name() string {
+	return "PG"
+}
+
+// ReadConfigFile takes in a filename and will read it into the backends config
+func (pg *PGBackend) ReadConfigFile(configFile string) error {
+	var err error
+	if err = UnmarshalConfig(configFile, &pg.cfg); err != nil {
+		return err
+	}
+
+	if pg.cfg.AESKey, err = base64.StdEncoding.DecodeString(pg.cfg.Key); err != nil {
+		return fmt.Errorf("unable to decode PG AESKey: %w", err)
+	}
+	return nil
+}
+
+// Insert takes the current keys and inserts them into the backend DB
+func (pg *PGBackend) Insert() error {
+	if err := pg.sslKey.insertKeys(pg.db); err != nil {
+		return err
+	}
+	if err := pg.dnssec.insertKeys(pg.db); err != nil {
+		return err
+	}
+	if err := pg.url.insertKeys(pg.db); err != nil {
+		return err
+	}
+	if err := pg.uri.insertKeys(pg.db); err != nil {
+		return err
+	}
+	return nil
+}
+
+// Start initiates the connection to the backend DB

Review comment:
       nit: GoDoc missing punctuation.

##########
File path: lib/go-tc/deliveryservice_ssl_keys.go
##########
@@ -73,6 +73,7 @@ type SSLKeyRequestFields struct {
 	HostName     *string `json:"hostname,omitempty"`
 	Country      *string `json:"country,omitempty"`
 	State        *string `json:"state,omitempty"`
+	Version      *int    `json:"version,omitempty"`

Review comment:
       I'm not sure we can add this without making a new-version structure, because this is used in the APIv3 client method `GenerateSSLKeysForDS` which has been published. That's what @rob05c would say, I think. @rawlinp will disagree, saying that it won't break existing clients because they couldn't have been using it anyway - which is true.
   
   Personally I don't care either way, but would find it easier to just make an `SSLKeyRequestV40` and `SSLKeyRequestV4` alias as it's a small enough change and won't hurt anything even if it's unnecessary. But if rob doesn't care I don't think anyone else will, so in that case don't worry about it.

##########
File path: tools/traffic_vault_migrate/postgres.go
##########
@@ -0,0 +1,742 @@
+package main
+
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+import (
+	"database/sql"
+	"encoding/base64"
+	"encoding/json"
+	"errors"
+	"fmt"
+	"strconv"
+	"strings"
+
+	"github.com/apache/trafficcontrol/lib/go-log"
+	"github.com/apache/trafficcontrol/lib/go-tc"
+	util "github.com/apache/trafficcontrol/lib/go-util"
+
+	_ "github.com/lib/pq"
+)
+
+// PGConfig represents the configuration options available to the PG backend
+type PGConfig struct {
+	Host     string `json:"host"`
+	Port     string `json:"port"`
+	User     string `json:"user"`
+	Password string `json:"password"`
+	SSLMode  string `json:"sslmode"`
+	Database string `json:"database"`
+	Key      string `json:"aesKey"`
+	AESKey   []byte
+}
+
+// PGBackend is the Postgres implementation of TVBackend
+type PGBackend struct {
+	sslKey pgSSLKeyTable
+	dnssec pgDNSSecTable
+	uri    pgURISignKeyTable
+	url    pgURLSigKeyTable
+	cfg    PGConfig
+	db     *sql.DB
+}
+
+// String returns a high level overview of the backend and its keys

Review comment:
       nit: GoDoc missing punctuation.

##########
File path: tools/traffic_vault_migrate/postgres.go
##########
@@ -0,0 +1,742 @@
+package main
+
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+import (
+	"database/sql"
+	"encoding/base64"
+	"encoding/json"
+	"errors"
+	"fmt"
+	"strconv"
+	"strings"
+
+	"github.com/apache/trafficcontrol/lib/go-log"
+	"github.com/apache/trafficcontrol/lib/go-tc"
+	util "github.com/apache/trafficcontrol/lib/go-util"
+
+	_ "github.com/lib/pq"
+)
+
+// PGConfig represents the configuration options available to the PG backend
+type PGConfig struct {
+	Host     string `json:"host"`
+	Port     string `json:"port"`
+	User     string `json:"user"`
+	Password string `json:"password"`
+	SSLMode  string `json:"sslmode"`
+	Database string `json:"database"`
+	Key      string `json:"aesKey"`
+	AESKey   []byte
+}
+
+// PGBackend is the Postgres implementation of TVBackend
+type PGBackend struct {
+	sslKey pgSSLKeyTable
+	dnssec pgDNSSecTable
+	uri    pgURISignKeyTable
+	url    pgURLSigKeyTable
+	cfg    PGConfig
+	db     *sql.DB
+}
+
+// String returns a high level overview of the backend and its keys
+func (pg *PGBackend) String() string {
+	data := fmt.Sprintf("PG server %s@%s:%s\n", pg.cfg.User, pg.cfg.Host, pg.cfg.Port)
+	data += fmt.Sprintf("\tSSL Keys: %d\n", len(pg.sslKey.Records))
+	data += fmt.Sprintf("\tDNSSec Keys: %d\n", len(pg.dnssec.Records))
+	data += fmt.Sprintf("\tURI Keys: %d\n", len(pg.uri.Records))
+	data += fmt.Sprintf("\tURL Keys: %d\n", len(pg.url.Records))
+	return data
+}
+
+// Name returns the name for this backend
+func (pg *PGBackend) Name() string {
+	return "PG"
+}
+
+// ReadConfigFile takes in a filename and will read it into the backends config

Review comment:
       nit: GoDoc missing punctuation.

##########
File path: tools/traffic_vault_migrate/postgres.go
##########
@@ -0,0 +1,742 @@
+package main
+
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+import (
+	"database/sql"
+	"encoding/base64"
+	"encoding/json"
+	"errors"
+	"fmt"
+	"strconv"
+	"strings"
+
+	"github.com/apache/trafficcontrol/lib/go-log"
+	"github.com/apache/trafficcontrol/lib/go-tc"
+	util "github.com/apache/trafficcontrol/lib/go-util"
+
+	_ "github.com/lib/pq"
+)
+
+// PGConfig represents the configuration options available to the PG backend
+type PGConfig struct {
+	Host     string `json:"host"`
+	Port     string `json:"port"`
+	User     string `json:"user"`
+	Password string `json:"password"`
+	SSLMode  string `json:"sslmode"`
+	Database string `json:"database"`
+	Key      string `json:"aesKey"`
+	AESKey   []byte
+}
+
+// PGBackend is the Postgres implementation of TVBackend
+type PGBackend struct {
+	sslKey pgSSLKeyTable
+	dnssec pgDNSSecTable
+	uri    pgURISignKeyTable
+	url    pgURLSigKeyTable
+	cfg    PGConfig
+	db     *sql.DB
+}
+
+// String returns a high level overview of the backend and its keys
+func (pg *PGBackend) String() string {
+	data := fmt.Sprintf("PG server %s@%s:%s\n", pg.cfg.User, pg.cfg.Host, pg.cfg.Port)
+	data += fmt.Sprintf("\tSSL Keys: %d\n", len(pg.sslKey.Records))
+	data += fmt.Sprintf("\tDNSSec Keys: %d\n", len(pg.dnssec.Records))
+	data += fmt.Sprintf("\tURI Keys: %d\n", len(pg.uri.Records))
+	data += fmt.Sprintf("\tURL Keys: %d\n", len(pg.url.Records))
+	return data
+}
+
+// Name returns the name for this backend
+func (pg *PGBackend) Name() string {
+	return "PG"
+}
+
+// ReadConfigFile takes in a filename and will read it into the backends config
+func (pg *PGBackend) ReadConfigFile(configFile string) error {
+	var err error
+	if err = UnmarshalConfig(configFile, &pg.cfg); err != nil {
+		return err
+	}
+
+	if pg.cfg.AESKey, err = base64.StdEncoding.DecodeString(pg.cfg.Key); err != nil {
+		return fmt.Errorf("unable to decode PG AESKey: %w", err)
+	}
+	return nil
+}
+
+// Insert takes the current keys and inserts them into the backend DB
+func (pg *PGBackend) Insert() error {
+	if err := pg.sslKey.insertKeys(pg.db); err != nil {
+		return err
+	}
+	if err := pg.dnssec.insertKeys(pg.db); err != nil {
+		return err
+	}
+	if err := pg.url.insertKeys(pg.db); err != nil {
+		return err
+	}
+	if err := pg.uri.insertKeys(pg.db); err != nil {
+		return err
+	}
+	return nil
+}
+
+// Start initiates the connection to the backend DB
+func (pg *PGBackend) Start() error {
+	sqlStr := fmt.Sprintf("postgres://%s:%s@%s:%s/%s?sslmode=%s", pg.cfg.User, pg.cfg.Password, pg.cfg.Host, pg.cfg.Port, pg.cfg.Database, pg.cfg.SSLMode)
+	db, err := sql.Open("postgres", sqlStr)
+	if err != nil {
+		return fmt.Errorf("unable to start PG client: %w", err)
+	}
+
+	pg.db = db
+	pg.sslKey = pgSSLKeyTable{}
+	pg.dnssec = pgDNSSecTable{}
+	pg.url = pgURLSigKeyTable{}
+	pg.uri = pgURISignKeyTable{}
+
+	return nil
+}
+
+// ValidateKey validates that the keys are valid (in most cases, certain fields are not null)
+func (pg *PGBackend) ValidateKey() []string {
+	var errs []string
+	if errs := pg.sslKey.validate(); errs != nil {
+		errs = append(errs, errs...)
+	}
+	if errs := pg.dnssec.validate(); errs != nil {
+		errs = append(errs, errs...)
+	}
+	if errs := pg.uri.validate(); errs != nil {
+		errs = append(errs, errs...)
+	}
+	if errs := pg.url.validate(); errs != nil {
+		errs = append(errs, errs...)
+	}
+	return errs
+}
+
+// Close terminates the connection to the backend DB
+func (pg *PGBackend) Close() error {
+	return pg.db.Close()
+}
+
+// Ping checks the connection to the backend DB
+func (pg *PGBackend) Ping() error {
+	return pg.db.Ping()
+}
+
+// Fetch gets all of the keys from the backend DB

Review comment:
       nit: GoDoc missing punctuation.

##########
File path: tools/traffic_vault_migrate/postgres.go
##########
@@ -0,0 +1,742 @@
+package main
+
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+import (
+	"database/sql"
+	"encoding/base64"
+	"encoding/json"
+	"errors"
+	"fmt"
+	"strconv"
+	"strings"
+
+	"github.com/apache/trafficcontrol/lib/go-log"
+	"github.com/apache/trafficcontrol/lib/go-tc"
+	util "github.com/apache/trafficcontrol/lib/go-util"
+
+	_ "github.com/lib/pq"
+)
+
+// PGConfig represents the configuration options available to the PG backend

Review comment:
       nit: GoDoc missing punctuation.

##########
File path: tools/traffic_vault_migrate/postgres.go
##########
@@ -0,0 +1,742 @@
+package main
+
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+import (
+	"database/sql"
+	"encoding/base64"
+	"encoding/json"
+	"errors"
+	"fmt"
+	"strconv"
+	"strings"
+
+	"github.com/apache/trafficcontrol/lib/go-log"
+	"github.com/apache/trafficcontrol/lib/go-tc"
+	util "github.com/apache/trafficcontrol/lib/go-util"
+
+	_ "github.com/lib/pq"
+)
+
+// PGConfig represents the configuration options available to the PG backend
+type PGConfig struct {
+	Host     string `json:"host"`
+	Port     string `json:"port"`
+	User     string `json:"user"`
+	Password string `json:"password"`
+	SSLMode  string `json:"sslmode"`
+	Database string `json:"database"`
+	Key      string `json:"aesKey"`
+	AESKey   []byte
+}
+
+// PGBackend is the Postgres implementation of TVBackend

Review comment:
       nit: GoDoc missing punctuation.

##########
File path: tools/traffic_vault_migrate/postgres.go
##########
@@ -0,0 +1,742 @@
+package main
+
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+import (
+	"database/sql"
+	"encoding/base64"
+	"encoding/json"
+	"errors"
+	"fmt"
+	"strconv"
+	"strings"
+
+	"github.com/apache/trafficcontrol/lib/go-log"
+	"github.com/apache/trafficcontrol/lib/go-tc"
+	util "github.com/apache/trafficcontrol/lib/go-util"
+
+	_ "github.com/lib/pq"
+)
+
+// PGConfig represents the configuration options available to the PG backend
+type PGConfig struct {
+	Host     string `json:"host"`
+	Port     string `json:"port"`
+	User     string `json:"user"`
+	Password string `json:"password"`
+	SSLMode  string `json:"sslmode"`
+	Database string `json:"database"`
+	Key      string `json:"aesKey"`
+	AESKey   []byte
+}
+
+// PGBackend is the Postgres implementation of TVBackend
+type PGBackend struct {
+	sslKey pgSSLKeyTable
+	dnssec pgDNSSecTable
+	uri    pgURISignKeyTable
+	url    pgURLSigKeyTable
+	cfg    PGConfig
+	db     *sql.DB
+}
+
+// String returns a high level overview of the backend and its keys
+func (pg *PGBackend) String() string {
+	data := fmt.Sprintf("PG server %s@%s:%s\n", pg.cfg.User, pg.cfg.Host, pg.cfg.Port)
+	data += fmt.Sprintf("\tSSL Keys: %d\n", len(pg.sslKey.Records))
+	data += fmt.Sprintf("\tDNSSec Keys: %d\n", len(pg.dnssec.Records))
+	data += fmt.Sprintf("\tURI Keys: %d\n", len(pg.uri.Records))
+	data += fmt.Sprintf("\tURL Keys: %d\n", len(pg.url.Records))
+	return data
+}
+
+// Name returns the name for this backend
+func (pg *PGBackend) Name() string {
+	return "PG"
+}
+
+// ReadConfigFile takes in a filename and will read it into the backends config
+func (pg *PGBackend) ReadConfigFile(configFile string) error {
+	var err error
+	if err = UnmarshalConfig(configFile, &pg.cfg); err != nil {
+		return err
+	}
+
+	if pg.cfg.AESKey, err = base64.StdEncoding.DecodeString(pg.cfg.Key); err != nil {
+		return fmt.Errorf("unable to decode PG AESKey: %w", err)
+	}
+	return nil
+}
+
+// Insert takes the current keys and inserts them into the backend DB
+func (pg *PGBackend) Insert() error {
+	if err := pg.sslKey.insertKeys(pg.db); err != nil {
+		return err
+	}
+	if err := pg.dnssec.insertKeys(pg.db); err != nil {
+		return err
+	}
+	if err := pg.url.insertKeys(pg.db); err != nil {
+		return err
+	}
+	if err := pg.uri.insertKeys(pg.db); err != nil {
+		return err
+	}
+	return nil
+}
+
+// Start initiates the connection to the backend DB
+func (pg *PGBackend) Start() error {
+	sqlStr := fmt.Sprintf("postgres://%s:%s@%s:%s/%s?sslmode=%s", pg.cfg.User, pg.cfg.Password, pg.cfg.Host, pg.cfg.Port, pg.cfg.Database, pg.cfg.SSLMode)
+	db, err := sql.Open("postgres", sqlStr)
+	if err != nil {
+		return fmt.Errorf("unable to start PG client: %w", err)
+	}
+
+	pg.db = db
+	pg.sslKey = pgSSLKeyTable{}
+	pg.dnssec = pgDNSSecTable{}
+	pg.url = pgURLSigKeyTable{}
+	pg.uri = pgURISignKeyTable{}
+
+	return nil
+}
+
+// ValidateKey validates that the keys are valid (in most cases, certain fields are not null)

Review comment:
       nit: GoDoc missing punctuation.

##########
File path: tools/traffic_vault_migrate/postgres.go
##########
@@ -0,0 +1,742 @@
+package main
+
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+import (
+	"database/sql"
+	"encoding/base64"
+	"encoding/json"
+	"errors"
+	"fmt"
+	"strconv"
+	"strings"
+
+	"github.com/apache/trafficcontrol/lib/go-log"
+	"github.com/apache/trafficcontrol/lib/go-tc"
+	util "github.com/apache/trafficcontrol/lib/go-util"
+
+	_ "github.com/lib/pq"
+)
+
+// PGConfig represents the configuration options available to the PG backend
+type PGConfig struct {
+	Host     string `json:"host"`
+	Port     string `json:"port"`
+	User     string `json:"user"`
+	Password string `json:"password"`
+	SSLMode  string `json:"sslmode"`
+	Database string `json:"database"`
+	Key      string `json:"aesKey"`
+	AESKey   []byte
+}
+
+// PGBackend is the Postgres implementation of TVBackend
+type PGBackend struct {
+	sslKey pgSSLKeyTable
+	dnssec pgDNSSecTable
+	uri    pgURISignKeyTable
+	url    pgURLSigKeyTable
+	cfg    PGConfig
+	db     *sql.DB
+}
+
+// String returns a high level overview of the backend and its keys
+func (pg *PGBackend) String() string {
+	data := fmt.Sprintf("PG server %s@%s:%s\n", pg.cfg.User, pg.cfg.Host, pg.cfg.Port)
+	data += fmt.Sprintf("\tSSL Keys: %d\n", len(pg.sslKey.Records))
+	data += fmt.Sprintf("\tDNSSec Keys: %d\n", len(pg.dnssec.Records))
+	data += fmt.Sprintf("\tURI Keys: %d\n", len(pg.uri.Records))
+	data += fmt.Sprintf("\tURL Keys: %d\n", len(pg.url.Records))
+	return data
+}
+
+// Name returns the name for this backend

Review comment:
       nit: GoDoc missing punctuation.

##########
File path: tools/traffic_vault_migrate/postgres.go
##########
@@ -0,0 +1,742 @@
+package main
+
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+import (
+	"database/sql"
+	"encoding/base64"
+	"encoding/json"
+	"errors"
+	"fmt"
+	"strconv"
+	"strings"
+
+	"github.com/apache/trafficcontrol/lib/go-log"
+	"github.com/apache/trafficcontrol/lib/go-tc"
+	util "github.com/apache/trafficcontrol/lib/go-util"
+
+	_ "github.com/lib/pq"
+)
+
+// PGConfig represents the configuration options available to the PG backend
+type PGConfig struct {
+	Host     string `json:"host"`
+	Port     string `json:"port"`
+	User     string `json:"user"`
+	Password string `json:"password"`
+	SSLMode  string `json:"sslmode"`
+	Database string `json:"database"`
+	Key      string `json:"aesKey"`
+	AESKey   []byte
+}
+
+// PGBackend is the Postgres implementation of TVBackend
+type PGBackend struct {
+	sslKey pgSSLKeyTable
+	dnssec pgDNSSecTable
+	uri    pgURISignKeyTable
+	url    pgURLSigKeyTable
+	cfg    PGConfig
+	db     *sql.DB
+}
+
+// String returns a high level overview of the backend and its keys
+func (pg *PGBackend) String() string {
+	data := fmt.Sprintf("PG server %s@%s:%s\n", pg.cfg.User, pg.cfg.Host, pg.cfg.Port)
+	data += fmt.Sprintf("\tSSL Keys: %d\n", len(pg.sslKey.Records))
+	data += fmt.Sprintf("\tDNSSec Keys: %d\n", len(pg.dnssec.Records))
+	data += fmt.Sprintf("\tURI Keys: %d\n", len(pg.uri.Records))
+	data += fmt.Sprintf("\tURL Keys: %d\n", len(pg.url.Records))
+	return data
+}
+
+// Name returns the name for this backend
+func (pg *PGBackend) Name() string {
+	return "PG"
+}
+
+// ReadConfigFile takes in a filename and will read it into the backends config
+func (pg *PGBackend) ReadConfigFile(configFile string) error {
+	var err error
+	if err = UnmarshalConfig(configFile, &pg.cfg); err != nil {
+		return err
+	}
+
+	if pg.cfg.AESKey, err = base64.StdEncoding.DecodeString(pg.cfg.Key); err != nil {
+		return fmt.Errorf("unable to decode PG AESKey: %w", err)
+	}
+	return nil
+}
+
+// Insert takes the current keys and inserts them into the backend DB
+func (pg *PGBackend) Insert() error {
+	if err := pg.sslKey.insertKeys(pg.db); err != nil {
+		return err
+	}
+	if err := pg.dnssec.insertKeys(pg.db); err != nil {
+		return err
+	}
+	if err := pg.url.insertKeys(pg.db); err != nil {
+		return err
+	}
+	if err := pg.uri.insertKeys(pg.db); err != nil {
+		return err
+	}
+	return nil
+}
+
+// Start initiates the connection to the backend DB
+func (pg *PGBackend) Start() error {
+	sqlStr := fmt.Sprintf("postgres://%s:%s@%s:%s/%s?sslmode=%s", pg.cfg.User, pg.cfg.Password, pg.cfg.Host, pg.cfg.Port, pg.cfg.Database, pg.cfg.SSLMode)
+	db, err := sql.Open("postgres", sqlStr)
+	if err != nil {
+		return fmt.Errorf("unable to start PG client: %w", err)
+	}
+
+	pg.db = db
+	pg.sslKey = pgSSLKeyTable{}
+	pg.dnssec = pgDNSSecTable{}
+	pg.url = pgURLSigKeyTable{}
+	pg.uri = pgURISignKeyTable{}
+
+	return nil
+}
+
+// ValidateKey validates that the keys are valid (in most cases, certain fields are not null)
+func (pg *PGBackend) ValidateKey() []string {
+	var errs []string
+	if errs := pg.sslKey.validate(); errs != nil {
+		errs = append(errs, errs...)
+	}
+	if errs := pg.dnssec.validate(); errs != nil {
+		errs = append(errs, errs...)
+	}
+	if errs := pg.uri.validate(); errs != nil {
+		errs = append(errs, errs...)
+	}
+	if errs := pg.url.validate(); errs != nil {
+		errs = append(errs, errs...)
+	}
+	return errs
+}
+
+// Close terminates the connection to the backend DB

Review comment:
       nit: GoDoc missing punctuation.

##########
File path: tools/traffic_vault_migrate/postgres.go
##########
@@ -0,0 +1,742 @@
+package main
+
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+import (
+	"database/sql"
+	"encoding/base64"
+	"encoding/json"
+	"errors"
+	"fmt"
+	"strconv"
+	"strings"
+
+	"github.com/apache/trafficcontrol/lib/go-log"
+	"github.com/apache/trafficcontrol/lib/go-tc"
+	util "github.com/apache/trafficcontrol/lib/go-util"
+
+	_ "github.com/lib/pq"
+)
+
+// PGConfig represents the configuration options available to the PG backend
+type PGConfig struct {
+	Host     string `json:"host"`
+	Port     string `json:"port"`
+	User     string `json:"user"`
+	Password string `json:"password"`
+	SSLMode  string `json:"sslmode"`
+	Database string `json:"database"`
+	Key      string `json:"aesKey"`
+	AESKey   []byte
+}
+
+// PGBackend is the Postgres implementation of TVBackend
+type PGBackend struct {
+	sslKey pgSSLKeyTable
+	dnssec pgDNSSecTable
+	uri    pgURISignKeyTable
+	url    pgURLSigKeyTable
+	cfg    PGConfig
+	db     *sql.DB
+}
+
+// String returns a high level overview of the backend and its keys
+func (pg *PGBackend) String() string {
+	data := fmt.Sprintf("PG server %s@%s:%s\n", pg.cfg.User, pg.cfg.Host, pg.cfg.Port)
+	data += fmt.Sprintf("\tSSL Keys: %d\n", len(pg.sslKey.Records))
+	data += fmt.Sprintf("\tDNSSec Keys: %d\n", len(pg.dnssec.Records))
+	data += fmt.Sprintf("\tURI Keys: %d\n", len(pg.uri.Records))
+	data += fmt.Sprintf("\tURL Keys: %d\n", len(pg.url.Records))
+	return data
+}
+
+// Name returns the name for this backend
+func (pg *PGBackend) Name() string {
+	return "PG"
+}
+
+// ReadConfigFile takes in a filename and will read it into the backends config
+func (pg *PGBackend) ReadConfigFile(configFile string) error {
+	var err error
+	if err = UnmarshalConfig(configFile, &pg.cfg); err != nil {
+		return err
+	}
+
+	if pg.cfg.AESKey, err = base64.StdEncoding.DecodeString(pg.cfg.Key); err != nil {
+		return fmt.Errorf("unable to decode PG AESKey: %w", err)
+	}
+	return nil
+}
+
+// Insert takes the current keys and inserts them into the backend DB
+func (pg *PGBackend) Insert() error {
+	if err := pg.sslKey.insertKeys(pg.db); err != nil {
+		return err
+	}
+	if err := pg.dnssec.insertKeys(pg.db); err != nil {
+		return err
+	}
+	if err := pg.url.insertKeys(pg.db); err != nil {
+		return err
+	}
+	if err := pg.uri.insertKeys(pg.db); err != nil {
+		return err
+	}
+	return nil
+}
+
+// Start initiates the connection to the backend DB
+func (pg *PGBackend) Start() error {
+	sqlStr := fmt.Sprintf("postgres://%s:%s@%s:%s/%s?sslmode=%s", pg.cfg.User, pg.cfg.Password, pg.cfg.Host, pg.cfg.Port, pg.cfg.Database, pg.cfg.SSLMode)
+	db, err := sql.Open("postgres", sqlStr)
+	if err != nil {
+		return fmt.Errorf("unable to start PG client: %w", err)
+	}
+
+	pg.db = db
+	pg.sslKey = pgSSLKeyTable{}
+	pg.dnssec = pgDNSSecTable{}
+	pg.url = pgURLSigKeyTable{}
+	pg.uri = pgURISignKeyTable{}
+
+	return nil
+}
+
+// ValidateKey validates that the keys are valid (in most cases, certain fields are not null)
+func (pg *PGBackend) ValidateKey() []string {
+	var errs []string
+	if errs := pg.sslKey.validate(); errs != nil {
+		errs = append(errs, errs...)
+	}
+	if errs := pg.dnssec.validate(); errs != nil {
+		errs = append(errs, errs...)
+	}
+	if errs := pg.uri.validate(); errs != nil {
+		errs = append(errs, errs...)
+	}
+	if errs := pg.url.validate(); errs != nil {
+		errs = append(errs, errs...)
+	}
+	return errs
+}
+
+// Close terminates the connection to the backend DB
+func (pg *PGBackend) Close() error {
+	return pg.db.Close()
+}
+
+// Ping checks the connection to the backend DB

Review comment:
       nit: GoDoc missing punctuation.

##########
File path: tools/traffic_vault_migrate/postgres.go
##########
@@ -0,0 +1,742 @@
+package main
+
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+import (
+	"database/sql"
+	"encoding/base64"
+	"encoding/json"
+	"errors"
+	"fmt"
+	"strconv"
+	"strings"
+
+	"github.com/apache/trafficcontrol/lib/go-log"
+	"github.com/apache/trafficcontrol/lib/go-tc"
+	util "github.com/apache/trafficcontrol/lib/go-util"
+
+	_ "github.com/lib/pq"
+)
+
+// PGConfig represents the configuration options available to the PG backend
+type PGConfig struct {
+	Host     string `json:"host"`
+	Port     string `json:"port"`
+	User     string `json:"user"`
+	Password string `json:"password"`
+	SSLMode  string `json:"sslmode"`
+	Database string `json:"database"`
+	Key      string `json:"aesKey"`
+	AESKey   []byte
+}
+
+// PGBackend is the Postgres implementation of TVBackend
+type PGBackend struct {
+	sslKey pgSSLKeyTable
+	dnssec pgDNSSecTable
+	uri    pgURISignKeyTable
+	url    pgURLSigKeyTable
+	cfg    PGConfig
+	db     *sql.DB
+}
+
+// String returns a high level overview of the backend and its keys
+func (pg *PGBackend) String() string {
+	data := fmt.Sprintf("PG server %s@%s:%s\n", pg.cfg.User, pg.cfg.Host, pg.cfg.Port)
+	data += fmt.Sprintf("\tSSL Keys: %d\n", len(pg.sslKey.Records))
+	data += fmt.Sprintf("\tDNSSec Keys: %d\n", len(pg.dnssec.Records))
+	data += fmt.Sprintf("\tURI Keys: %d\n", len(pg.uri.Records))
+	data += fmt.Sprintf("\tURL Keys: %d\n", len(pg.url.Records))
+	return data
+}
+
+// Name returns the name for this backend
+func (pg *PGBackend) Name() string {
+	return "PG"
+}
+
+// ReadConfigFile takes in a filename and will read it into the backends config
+func (pg *PGBackend) ReadConfigFile(configFile string) error {
+	var err error
+	if err = UnmarshalConfig(configFile, &pg.cfg); err != nil {
+		return err
+	}
+
+	if pg.cfg.AESKey, err = base64.StdEncoding.DecodeString(pg.cfg.Key); err != nil {
+		return fmt.Errorf("unable to decode PG AESKey: %w", err)
+	}
+	return nil
+}
+
+// Insert takes the current keys and inserts them into the backend DB
+func (pg *PGBackend) Insert() error {
+	if err := pg.sslKey.insertKeys(pg.db); err != nil {
+		return err
+	}
+	if err := pg.dnssec.insertKeys(pg.db); err != nil {
+		return err
+	}
+	if err := pg.url.insertKeys(pg.db); err != nil {
+		return err
+	}
+	if err := pg.uri.insertKeys(pg.db); err != nil {
+		return err
+	}
+	return nil
+}
+
+// Start initiates the connection to the backend DB
+func (pg *PGBackend) Start() error {
+	sqlStr := fmt.Sprintf("postgres://%s:%s@%s:%s/%s?sslmode=%s", pg.cfg.User, pg.cfg.Password, pg.cfg.Host, pg.cfg.Port, pg.cfg.Database, pg.cfg.SSLMode)
+	db, err := sql.Open("postgres", sqlStr)
+	if err != nil {
+		return fmt.Errorf("unable to start PG client: %w", err)
+	}
+
+	pg.db = db
+	pg.sslKey = pgSSLKeyTable{}
+	pg.dnssec = pgDNSSecTable{}
+	pg.url = pgURLSigKeyTable{}
+	pg.uri = pgURISignKeyTable{}
+
+	return nil
+}
+
+// ValidateKey validates that the keys are valid (in most cases, certain fields are not null)
+func (pg *PGBackend) ValidateKey() []string {
+	var errs []string
+	if errs := pg.sslKey.validate(); errs != nil {
+		errs = append(errs, errs...)
+	}
+	if errs := pg.dnssec.validate(); errs != nil {
+		errs = append(errs, errs...)
+	}
+	if errs := pg.uri.validate(); errs != nil {
+		errs = append(errs, errs...)
+	}
+	if errs := pg.url.validate(); errs != nil {
+		errs = append(errs, errs...)
+	}
+	return errs
+}
+
+// Close terminates the connection to the backend DB
+func (pg *PGBackend) Close() error {
+	return pg.db.Close()
+}
+
+// Ping checks the connection to the backend DB
+func (pg *PGBackend) Ping() error {
+	return pg.db.Ping()
+}
+
+// Fetch gets all of the keys from the backend DB
+func (pg *PGBackend) Fetch() error {
+	if err := pg.sslKey.gatherKeys(pg.db); err != nil {
+		return err
+	}
+
+	if err := pg.dnssec.gatherKeys(pg.db); err != nil {
+		return err
+	}
+
+	if err := pg.url.gatherKeys(pg.db); err != nil {
+		return err
+	}
+
+	if err := pg.uri.gatherKeys(pg.db); err != nil {
+		return err
+	}
+
+	return nil
+}
+
+// GetSSLKeys converts the backends internal key representation into the common representation (SSLKey)
+func (pg *PGBackend) GetSSLKeys() ([]SSLKey, error) {
+	if err := pg.sslKey.decrypt(pg.cfg.AESKey); err != nil {
+		return nil, err
+	}
+	return pg.sslKey.toGeneric(), nil
+}
+
+// SetSSLKeys takes in keys and converts & encrypts the data into the backends internal format
+func (pg *PGBackend) SetSSLKeys(keys []SSLKey) error {
+	pg.sslKey.fromGeneric(keys)
+	return pg.sslKey.encrypt(pg.cfg.AESKey)
+}
+
+// GetDNSSecKeys converts the backends internal key representation into the common representation (DNSSecKey)
+func (pg *PGBackend) GetDNSSecKeys() ([]DNSSecKey, error) {
+	if err := pg.dnssec.decrypt(pg.cfg.AESKey); err != nil {
+		return nil, err
+	}
+	return pg.dnssec.toGeneric(), nil
+}
+
+// SetDNSSecKeys takes in keys and converts & encrypts the data into the backends internal format
+func (pg *PGBackend) SetDNSSecKeys(keys []DNSSecKey) error {
+	pg.dnssec.fromGeneric(keys)
+	return pg.dnssec.encrypt(pg.cfg.AESKey)
+}
+
+// GetURISignKeys converts the pg internal key representation into the common representation (URISignKey)
+func (pg *PGBackend) GetURISignKeys() ([]URISignKey, error) {
+	if err := pg.uri.decrypt(pg.cfg.AESKey); err != nil {
+		return nil, err
+	}
+	return pg.uri.toGeneric(), nil
+}
+
+// SetURISignKeys takes in keys and converts & encrypts the data into the backends internal format
+func (pg *PGBackend) SetURISignKeys(keys []URISignKey) error {
+	pg.uri.fromGeneric(keys)
+	return pg.uri.encrypt(pg.cfg.AESKey)
+}
+
+// GetURLSigKeys converts the backends internal key representation into the common representation (URLSigKey)
+func (pg *PGBackend) GetURLSigKeys() ([]URLSigKey, error) {
+	if err := pg.url.decrypt(pg.cfg.AESKey); err != nil {
+		return nil, err
+	}
+	return pg.url.toGeneric(), nil
+}
+
+// SetURLSigKeys takes in keys and converts & encrypts the data into the backends internal format

Review comment:
       nit: GoDoc missing punctuation.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [trafficcontrol] rawlinp commented on a change in pull request #5924: Add tool to migrate data between TV backends

Posted by GitBox <gi...@apache.org>.
rawlinp commented on a change in pull request #5924:
URL: https://github.com/apache/trafficcontrol/pull/5924#discussion_r651205381



##########
File path: tools/traffic_vault_migrate/postgres.go
##########
@@ -0,0 +1,743 @@
+package main
+
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+import (
+	"database/sql"
+	"encoding/base64"
+	"encoding/json"
+	"errors"
+	"fmt"
+	"log"
+	"strconv"
+	"strings"
+
+	"github.com/apache/trafficcontrol/lib/go-tc"
+	util "github.com/apache/trafficcontrol/lib/go-util"
+
+	_ "github.com/lib/pq"
+)
+
+// PGConfig represents the configuration options available to the PG backend
+type PGConfig struct {
+	Host     string `json:"host"`
+	Port     string `json:"port"`
+	User     string `json:"user"`
+	Password string `json:"password"`
+	SSLMode  string `json:"sslmode"`
+	Database string `json:"database"`
+	Key      string `json:"aesKey"`
+	AESKey   []byte
+}
+
+// PGBackend is the Postgres implementation of TVBackend
+type PGBackend struct {
+	sslKey pgSSLKeyTable
+	dnssec pgDNSSecTable
+	uri    pgURISignKeyTable
+	url    pgURLSigKeyTable
+	cfg    PGConfig
+	db     *sql.DB
+}
+
+// String returns a high level overview of the backend and its keys
+func (pg *PGBackend) String() string {
+	data := fmt.Sprintf("PG server %v@%v:%v\n", pg.cfg.User, pg.cfg.Host, pg.cfg.Port)
+	data += fmt.Sprintf("\tSSL Keys: %v\n", len(pg.sslKey.Records))
+	data += fmt.Sprintf("\tDNSSec Keys: %v\n", len(pg.dnssec.Records))
+	data += fmt.Sprintf("\tURI Keys: %v\n", len(pg.uri.Records))
+	data += fmt.Sprintf("\tURL Keys: %v\n", len(pg.url.Records))
+	return data
+}
+
+// Name returns the name for this backend
+func (pg *PGBackend) Name() string {
+	return "PG"
+}
+
+// ReadConfig takes in a filename and will read it into the backends config

Review comment:
       My bad -- I was reading `UnmarshalConfig` as `UnmarshalJSON`, so I thought `s` was actually the file's contents. Please disregard 😄 




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [trafficcontrol] shamrickus commented on a change in pull request #5924: Add tool to migrate data between TV backends

Posted by GitBox <gi...@apache.org>.
shamrickus commented on a change in pull request #5924:
URL: https://github.com/apache/trafficcontrol/pull/5924#discussion_r651190702



##########
File path: tools/traffic_vault_migrate/postgres.go
##########
@@ -0,0 +1,743 @@
+package main
+
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+import (
+	"database/sql"
+	"encoding/base64"
+	"encoding/json"
+	"errors"
+	"fmt"
+	"log"
+	"strconv"
+	"strings"
+
+	"github.com/apache/trafficcontrol/lib/go-tc"
+	util "github.com/apache/trafficcontrol/lib/go-util"
+
+	_ "github.com/lib/pq"
+)
+
+// PGConfig represents the configuration options available to the PG backend
+type PGConfig struct {
+	Host     string `json:"host"`
+	Port     string `json:"port"`
+	User     string `json:"user"`
+	Password string `json:"password"`
+	SSLMode  string `json:"sslmode"`
+	Database string `json:"database"`
+	Key      string `json:"aesKey"`
+	AESKey   []byte
+}
+
+// PGBackend is the Postgres implementation of TVBackend
+type PGBackend struct {
+	sslKey pgSSLKeyTable
+	dnssec pgDNSSecTable
+	uri    pgURISignKeyTable
+	url    pgURLSigKeyTable
+	cfg    PGConfig
+	db     *sql.DB
+}
+
+// String returns a high level overview of the backend and its keys
+func (pg *PGBackend) String() string {
+	data := fmt.Sprintf("PG server %v@%v:%v\n", pg.cfg.User, pg.cfg.Host, pg.cfg.Port)
+	data += fmt.Sprintf("\tSSL Keys: %v\n", len(pg.sslKey.Records))
+	data += fmt.Sprintf("\tDNSSec Keys: %v\n", len(pg.dnssec.Records))
+	data += fmt.Sprintf("\tURI Keys: %v\n", len(pg.uri.Records))
+	data += fmt.Sprintf("\tURL Keys: %v\n", len(pg.url.Records))
+	return data
+}
+
+// Name returns the name for this backend
+func (pg *PGBackend) Name() string {
+	return "PG"
+}
+
+// ReadConfig takes in a filename and will read it into the backends config

Review comment:
       It does take in a file name (both backends call `UnmarshalConfig` which calls `ioutil.ReadFile`. I've renamed this fn to `ReadConfigFile` and I'll also change `toCfg/fromCfg` to `toCfgPath/fromCfgPath` so that it's more clear what they contain.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [trafficcontrol] shamrickus commented on a change in pull request #5924: Add tool to migrate data between TV backends

Posted by GitBox <gi...@apache.org>.
shamrickus commented on a change in pull request #5924:
URL: https://github.com/apache/trafficcontrol/pull/5924#discussion_r648791849



##########
File path: tools/traffic_vault_migrate/postgres.go
##########
@@ -0,0 +1,716 @@
+package main
+
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+import (
+	"database/sql"
+	"encoding/base64"
+	"encoding/json"
+	"errors"
+	"fmt"
+	"github.com/apache/trafficcontrol/lib/go-tc"
+	util "github.com/apache/trafficcontrol/lib/go-util"
+	_ "github.com/lib/pq"
+	"log"
+	"reflect"
+	"strconv"
+	"strings"
+)
+
+type PGConfig struct {
+	Host     string `json:"host"`
+	Port     string `json:"port"`
+	User     string `json:"user"`
+	Password string `json:"password"`
+	SSLMode  string `json:"sslmode"`
+	Database string `json:"database"`
+	Key      string `json:"aesKey"`
+	AESKey   []byte
+}
+type PGBackend struct {
+	sslKey PGSSLKeyTable
+	dnssec PGDNSSecTable
+	uri    PGURISignKeyTable
+	url    PGURLSigKeyTable
+	cfg    PGConfig
+	db     *sql.DB
+}
+
+func (pg *PGBackend) String() string {
+	data := fmt.Sprintf("PG server %v@%v:%v\n", pg.cfg.User, pg.cfg.Host, pg.cfg.Port)
+	data += fmt.Sprintf("\tSSL Keys: %v\n", len(pg.sslKey.Records))
+	data += fmt.Sprintf("\tDNSSec Keys: %v\n", len(pg.dnssec.Records))
+	data += fmt.Sprintf("\tURI Keys: %v\n", len(pg.uri.Records))
+	data += fmt.Sprintf("\tURL Keys: %v\n", len(pg.url.Records))
+	return data
+}
+func (pg *PGBackend) Name() string {
+	return "PG"
+}
+func (pg *PGBackend) ReadConfig(s string) error {
+	genericCfg, err := UnmarshalConfig(s, reflect.TypeOf(pg.cfg))
+	if err != nil {
+		return err
+	}
+
+	pg.cfg = *genericCfg.Interface().(*PGConfig)
+
+	pg.cfg.AESKey, err = base64.StdEncoding.DecodeString(pg.cfg.Key)
+	if err != nil {
+		return fmt.Errorf("unable to decode PG AESKey: %v", err)
+	}
+	return nil
+}
+func (pg *PGBackend) Insert() error {
+	if err := pg.sslKey.insertKeys(pg.db); err != nil {
+		return err
+	}
+	if err := pg.dnssec.insertKeys(pg.db); err != nil {
+		return err
+	}
+	if err := pg.url.insertKeys(pg.db); err != nil {
+		return err
+	}
+	if err := pg.uri.insertKeys(pg.db); err != nil {
+		return err
+	}
+	return nil
+}
+func (pg *PGBackend) Start() error {
+	sqlStr := fmt.Sprintf("postgres://%s:%s@%s:%s/%s?sslmode=%s", pg.cfg.User, pg.cfg.Password, pg.cfg.Host, pg.cfg.Port, pg.cfg.Database, pg.cfg.SSLMode)
+	db, err := sql.Open("postgres", sqlStr)
+	if err != nil {
+		return fmt.Errorf("Unable to start PG client: %v", err)
+	}
+
+	pg.db = db
+	pg.sslKey = PGSSLKeyTable{}
+	pg.dnssec = PGDNSSecTable{}
+	pg.url = PGURLSigKeyTable{}
+	pg.uri = PGURISignKeyTable{}
+
+	return nil
+}
+func (pg *PGBackend) ValidateKey() []string {
+	var errors []string
+	if errs := pg.sslKey.validate(); errs != nil {
+		errors = append(errors, errs...)
+	}
+	if errs := pg.dnssec.validate(); errs != nil {
+		errors = append(errors, errs...)
+	}
+	if errs := pg.uri.validate(); errs != nil {
+		errors = append(errors, errs...)
+	}
+	if errs := pg.url.validate(); errs != nil {
+		errors = append(errors, errs...)
+	}
+	return errors
+}
+func (pg *PGBackend) Stop() error {

Review comment:
       Don't see the harm, renamed it.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [trafficcontrol] shamrickus commented on a change in pull request #5924: Add tool to migrate data between TV backends

Posted by GitBox <gi...@apache.org>.
shamrickus commented on a change in pull request #5924:
URL: https://github.com/apache/trafficcontrol/pull/5924#discussion_r652107763



##########
File path: tools/traffic_vault_migrate/riak.go
##########
@@ -0,0 +1,746 @@
+package main
+
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+import (
+	"crypto/tls"
+	"encoding/json"
+	"errors"
+	"fmt"
+	"time"
+
+	"github.com/apache/trafficcontrol/lib/go-rfc"
+	"github.com/apache/trafficcontrol/lib/go-tc"
+
+	"github.com/basho/riak-go-client"
+)
+
+const (
+	BUCKET_SSL     = "ssl"
+	BUCKET_DNSSEC  = "dnssec"
+	BUCKET_URL_SIG = "url_sig_keys"
+	BUCKET_URI_SIG = "cdn_uri_sig_keys"
+
+	INDEX_SSL = "sslkeys"
+
+	SCHEMA_RIAK_KEY    = "_yz_rk"
+	SCHEMA_RIAK_BUCKET = "_yz_rb"
+)
+
+var (
+	SCHEMA_SSL_FIELDS = [...]string{SCHEMA_RIAK_KEY, SCHEMA_RIAK_BUCKET}
+)
+
+// RiakConfig  represents the configuration options available to the Riak backend
+type RiakConfig struct {
+	Host          string `json:"host"`
+	Port          string `json:"port"`
+	User          string `json:"user"`
+	Password      string `json:"password"`
+	Insecure      bool   `json:"insecure"`
+	TLSVersionRaw string `json:"tlsVersion"`
+	TLSVersion    uint16
+}
+
+// RiakBackend is the Riak implementation of TVBackend
+type RiakBackend struct {
+	sslKeys     riakSSLKeyTable
+	dnssecKeys  riakDNSSecKeyTable
+	uriSignKeys riakURISignKeyTable
+	urlSigKeys  riakURLSigKeyTable
+	cfg         RiakConfig
+	cluster     *riak.Cluster
+}
+
+// String returns a high level overview of the backend and its keys
+func (rb *RiakBackend) String() string {
+	data := fmt.Sprintf("Riak server %v@%v:%v\n", rb.cfg.User, rb.cfg.Host, rb.cfg.Port)
+	data += fmt.Sprintf("\tSSL Keys: %v\n", len(rb.sslKeys.Records))
+	data += fmt.Sprintf("\tDNSSec Keys: %v\n", len(rb.dnssecKeys.Records))
+	data += fmt.Sprintf("\tURI Keys: %v\n", len(rb.uriSignKeys.Records))
+	data += fmt.Sprintf("\tURL Keys: %v\n", len(rb.urlSigKeys.Records))
+	return data
+}
+
+// Name returns the name for this backend
+func (rb *RiakBackend) Name() string {
+	return "Riak"
+}
+
+// ReadConfig takes in a filename and will read it into the backends config
+func (rb *RiakBackend) ReadConfig(s string) error {
+	err := UnmarshalConfig(s, &rb.cfg)
+	if err != nil {
+		return err
+	}
+
+	if rb.cfg.TLSVersionRaw == "10" {
+		rb.cfg.TLSVersion = tls.VersionTLS10
+	} else if rb.cfg.TLSVersionRaw == "11" {
+		rb.cfg.TLSVersion = tls.VersionTLS11
+	} else if rb.cfg.TLSVersionRaw == "12" {
+		rb.cfg.TLSVersion = tls.VersionTLS12
+	} else if rb.cfg.TLSVersionRaw == "13" {
+		rb.cfg.TLSVersion = tls.VersionTLS13
+	}
+	return nil
+}
+
+// Insert takes the current keys and inserts them into the backend DB
+func (rb *RiakBackend) Insert() error {
+	if err := rb.sslKeys.insertKeys(rb.cluster); err != nil {
+		return err
+	}
+	if err := rb.dnssecKeys.insertKeys(rb.cluster); err != nil {
+		return err
+	}
+	if err := rb.urlSigKeys.insertKeys(rb.cluster); err != nil {
+		return err
+	}
+	if err := rb.uriSignKeys.insertKeys(rb.cluster); err != nil {
+		return err
+	}
+	return nil
+}
+
+// ValidateKey validates that the keys are valid (in most cases, certain fields are not null)
+func (rb *RiakBackend) ValidateKey() []string {
+	errors := []string{}
+	if errs := rb.sslKeys.validate(); errs != nil {
+		errors = append(errors, errs...)
+	}
+	if errs := rb.dnssecKeys.validate(); errs != nil {
+		errors = append(errors, errs...)
+	}
+	if errs := rb.uriSignKeys.validate(); errs != nil {
+		errors = append(errors, errs...)
+	}
+	if errs := rb.urlSigKeys.validate(); errs != nil {
+		errors = append(errors, errs...)
+	}
+
+	return errors
+}
+
+// SetSSLKeys takes in keys and converts & encrypts the data into the backends internal format
+func (rb *RiakBackend) SetSSLKeys(keys []SSLKey) error {
+	rb.sslKeys.fromGeneric(keys)
+	return nil
+}
+
+// SetDNSSecKeys takes in keys and converts & encrypts the data into the backends internal format
+func (rb *RiakBackend) SetDNSSecKeys(keys []DNSSecKey) error {
+	rb.dnssecKeys.fromGeneric(keys)
+	return nil
+}
+
+// SetURISignKeys takes in keys and converts & encrypts the data into the backends internal format
+func (rb *RiakBackend) SetURISignKeys(keys []URISignKey) error {
+	rb.uriSignKeys.fromGeneric(keys)
+	return nil
+}
+
+// SetURLSigKeys takes in keys and converts & encrypts the data into the backends internal format
+func (rb *RiakBackend) SetURLSigKeys(keys []URLSigKey) error {
+	rb.urlSigKeys.fromGeneric(keys)
+	return nil
+}
+
+// Start initiates the connection to the backend DB
+func (rb *RiakBackend) Start() error {
+	tlsConfig := &tls.Config{
+		InsecureSkipVerify: !rb.cfg.Insecure,
+		MaxVersion:         rb.cfg.TLSVersion,
+	}
+	auth := &riak.AuthOptions{
+		User:      rb.cfg.User,
+		Password:  rb.cfg.Password,
+		TlsConfig: tlsConfig,
+	}
+
+	cluster, err := getRiakCluster(rb.cfg, auth)
+	if err != nil {
+		return err
+	}
+	if err := cluster.Start(); err != nil {
+		return fmt.Errorf("unable to start riak cluster: %w", err)
+	}
+
+	rb.cluster = cluster
+	rb.sslKeys = riakSSLKeyTable{}
+	rb.dnssecKeys = riakDNSSecKeyTable{}
+	rb.urlSigKeys = riakURLSigKeyTable{}
+	rb.uriSignKeys = riakURISignKeyTable{}
+	return nil
+}
+
+// Close terminates the connection to the backend DB
+func (rb *RiakBackend) Close() error {
+	if err := rb.cluster.Stop(); err != nil {
+		return err
+	}
+	return nil
+}
+
+// Ping checks the connection to the backend DB
+func (rb *RiakBackend) Ping() error {
+	return ping(rb.cluster)
+}
+
+// GetSSLKeys converts the backends internal key representation into the common representation (SSLKey)
+func (rb *RiakBackend) GetSSLKeys() ([]SSLKey, error) {
+	return rb.sslKeys.toGeneric(), nil
+}
+
+// GetDNSSecKeys converts the backends internal key representation into the common representation (DNSSecKey)
+func (rb *RiakBackend) GetDNSSecKeys() ([]DNSSecKey, error) {
+	return rb.dnssecKeys.toGeneric(), nil
+}
+
+// GetURISignKeys converts the pg internal key representation into the common representation (URISignKey)
+func (rb *RiakBackend) GetURISignKeys() ([]URISignKey, error) {
+	return rb.uriSignKeys.toGeneric(), nil
+}
+
+// GetURLSigKeys converts the backends internal key representation into the common representation (URLSigKey)
+func (rb *RiakBackend) GetURLSigKeys() ([]URLSigKey, error) {
+	return rb.urlSigKeys.toGeneric(), nil
+}
+
+// Fetch gets all of the keys from the backend DB
+func (rb *RiakBackend) Fetch() error {
+	if err := rb.sslKeys.gatherKeys(rb.cluster); err != nil {
+		return err
+	}
+	if err := rb.dnssecKeys.gatherKeys(rb.cluster); err != nil {
+		return err
+	}
+	if err := rb.urlSigKeys.gatherKeys(rb.cluster); err != nil {
+		return err
+	}
+	if err := rb.uriSignKeys.gatherKeys(rb.cluster); err != nil {
+		return err
+	}
+
+	return nil
+}
+
+type riakSSLKeyRecord struct {
+	tc.DeliveryServiceSSLKeys
+}
+type riakSSLKeyTable struct {
+	Records []riakSSLKeyRecord
+}
+
+func (tbl *riakSSLKeyTable) gatherKeys(cluster *riak.Cluster) error {
+	searchDocs, err := search(cluster, INDEX_SSL, "cdn:*", fmt.Sprintf("%v:*latest", SCHEMA_RIAK_KEY), 1000, SCHEMA_SSL_FIELDS[:])
+	if err != nil {
+		return fmt.Errorf("RiakSSLKey gatherKeys: %w", err)
+	}
+
+	tbl.Records = make([]riakSSLKeyRecord, len(searchDocs))
+	for i, doc := range searchDocs {
+		objs, err := getObject(cluster, doc.Bucket, doc.Key)
+		if err != nil {
+			return err
+		}
+		if len(objs) < 1 {
+			return fmt.Errorf("RiakSSLKey gatherKeys unable to find any objects with key %v and bucket %v, but search results were returned", doc.Key, doc.Bucket)
+		}
+		if len(objs) > 1 {
+			return fmt.Errorf("RiakSSLKey gatherKeys more than 1 ssl key record found %v\n", len(objs))
+		}
+		var obj tc.DeliveryServiceSSLKeys
+		if err = json.Unmarshal(objs[0].Value, &obj); err != nil {
+			return fmt.Errorf("RiakSSLKey gatherKeys unable to unmarshal object into tc.DeliveryServiceSSLKeys: %w", err)
+		}
+		tbl.Records[i] = riakSSLKeyRecord{
+			DeliveryServiceSSLKeys: obj,
+		}
+	}
+	return nil
+}
+func (tbl *riakSSLKeyTable) toGeneric() []SSLKey {
+	keys := make([]SSLKey, len(tbl.Records))
+
+	for i, record := range tbl.Records {
+		keys[i] = SSLKey{
+			DeliveryServiceSSLKeys: record.DeliveryServiceSSLKeys,
+		}
+	}
+
+	return keys
+}
+func (tbl *riakSSLKeyTable) fromGeneric(keys []SSLKey) {
+	tbl.Records = make([]riakSSLKeyRecord, len(keys))
+
+	for i, record := range keys {
+		tbl.Records[i] = riakSSLKeyRecord{
+			DeliveryServiceSSLKeys: record.DeliveryServiceSSLKeys,
+		}
+	}
+}
+func (tbl *riakSSLKeyTable) insertKeys(cluster *riak.Cluster) error {
+	for _, record := range tbl.Records {
+		objBytes, err := json.Marshal(record.DeliveryServiceSSLKeys)
+		if err != nil {
+			return fmt.Errorf("RiakSSLKey insertKeys failed to unmarshal keys: %w", err)
+		}
+		err = setObject(cluster, makeRiakObject(objBytes, record.DeliveryService+"-"+record.Version.String()), BUCKET_SSL)
+		if err != nil {
+			return fmt.Errorf("RiakSSLKey insertKeys: %w", err)
+		}
+		err = setObject(cluster, makeRiakObject(objBytes, record.DeliveryService+"-latest"), BUCKET_SSL)
+		if err != nil {
+			return fmt.Errorf("RiakSSLKey insertKeys: %w", err)
+		}
+	}
+	return nil
+}
+func (tbl *riakSSLKeyTable) validate() []string {
+	errs := []string{}
+	for i, record := range tbl.Records {
+		if record.DeliveryService == "" {
+			errs = append(errs, fmt.Sprintf("SSL Key #%v: Delivery Service is blank!", i))
+		}
+		if record.CDN == "" {
+			errs = append(errs, fmt.Sprintf("SSL Key #%v: CDN is blank!", i))
+		}
+		if record.Version.String() == "" {
+			errs = append(errs, fmt.Sprintf("SSL Key #%v: Version is blank!", i))
+		}
+	}
+	return errs
+}
+
+type riakDNSSecKeyRecord struct {
+	CDN string
+	Key tc.DNSSECKeysRiak
+}
+type riakDNSSecKeyTable struct {
+	Records []riakDNSSecKeyRecord
+}
+
+func (tbl *riakDNSSecKeyTable) gatherKeys(cluster *riak.Cluster) error {
+	tbl.Records = []riakDNSSecKeyRecord{}
+	objs, err := getObjects(cluster, BUCKET_DNSSEC)
+	if err != nil {
+		return fmt.Errorf("RiakDNSSecKey gatherKeys: %w", err)
+	}
+	for _, obj := range objs {
+		key := tc.DNSSECKeysRiak{}
+		if err := json.Unmarshal(obj.Value, &key); err != nil {
+			return fmt.Errorf("RiakDNSSecKey gatherKeys unable to unmarshal object to tc.DNSSECKeysRiak: %w", err)
+		}
+		tbl.Records = append(tbl.Records, riakDNSSecKeyRecord{
+			CDN: obj.Key,
+			Key: key,
+		})
+	}
+	return nil
+}
+func (tbl *riakDNSSecKeyTable) toGeneric() []DNSSecKey {
+	keys := make([]DNSSecKey, len(tbl.Records))
+
+	for i, record := range tbl.Records {
+		keys[i] = DNSSecKey{
+			CDN:                    record.CDN,
+			DNSSECKeysTrafficVault: tc.DNSSECKeysTrafficVault(record.Key),
+		}
+	}
+
+	return keys
+}
+func (tbl *riakDNSSecKeyTable) fromGeneric(keys []DNSSecKey) {
+	tbl.Records = make([]riakDNSSecKeyRecord, len(keys))
+
+	for i, record := range keys {
+		tbl.Records[i] = riakDNSSecKeyRecord{
+			CDN: record.CDN,
+			Key: tc.DNSSECKeysRiak(record.DNSSECKeysTrafficVault),
+		}
+	}
+}
+func (tbl *riakDNSSecKeyTable) insertKeys(cluster *riak.Cluster) error {
+	for _, record := range tbl.Records {
+		objBytes, err := json.Marshal(record.Key)
+		if err != nil {
+			return fmt.Errorf("RiakDNSSecKey insertKeys error marshalling keys: %w", err)
+		}
+
+		err = setObject(cluster, makeRiakObject(objBytes, record.CDN), BUCKET_DNSSEC)
+		if err != nil {
+			return fmt.Errorf("RiakDNSSecKey insertKeys: %w", err)
+		}
+	}
+	return nil
+}
+func (tbl *riakDNSSecKeyTable) validate() []string {
+	errs := []string{}
+	for i, record := range tbl.Records {
+		if record.CDN == "" {
+			errs = append(errs, fmt.Sprintf("DNSSec Key #%v: CDN is blank!", i))
+		}
+	}
+	return errs
+}
+
+type riakURLSigKeyRecord struct {
+	Key             tc.URLSigKeys
+	DeliveryService string
+}
+type riakURLSigKeyTable struct {
+	Records []riakURLSigKeyRecord
+}
+
+func (tbl *riakURLSigKeyTable) gatherKeys(cluster *riak.Cluster) error {
+	tbl.Records = []riakURLSigKeyRecord{}
+	objs, err := getObjects(cluster, BUCKET_URL_SIG)
+	if err != nil {
+		return fmt.Errorf("RiakURLSigKey gatherKeys: %w", err)
+	}
+	for _, obj := range objs {
+		key := tc.URLSigKeys{}
+		if err := json.Unmarshal(obj.Value, &key); err != nil {
+			return fmt.Errorf("RiakURLSigKey gatherKeys unable to unamrshal object into tc.URLSigKeys: %w", err)
+		}
+		tbl.Records = append(tbl.Records, riakURLSigKeyRecord{
+			DeliveryService: obj.Key,
+			Key:             key,
+		})
+	}
+	return nil
+}
+func (tbl *riakURLSigKeyTable) toGeneric() []URLSigKey {
+	keys := make([]URLSigKey, len(tbl.Records))
+
+	for i, record := range tbl.Records {
+		keys[i] = URLSigKey{
+			URLSigKeys:      record.Key,
+			DeliveryService: record.DeliveryService,
+		}
+	}
+
+	return keys
+}
+func (tbl *riakURLSigKeyTable) fromGeneric(keys []URLSigKey) {
+	tbl.Records = make([]riakURLSigKeyRecord, len(keys))
+	for i, key := range keys {
+		tbl.Records[i] = riakURLSigKeyRecord{
+			Key:             key.URLSigKeys,
+			DeliveryService: key.DeliveryService,
+		}
+	}
+}
+func (tbl *riakURLSigKeyTable) insertKeys(cluster *riak.Cluster) error {
+	for _, record := range tbl.Records {
+		objBytes, err := json.Marshal(record.Key)
+		if err != nil {
+			return fmt.Errorf("RiakURLSigKey insertKeys unable to marshal keys: %w", err)
+		}
+
+		err = setObject(cluster, makeRiakObject(objBytes, "url_sig_"+record.DeliveryService+".config"), BUCKET_URL_SIG)
+		if err != nil {
+			return fmt.Errorf("RiakURLSigKey insertKeys: %w", err)
+		}
+	}
+	return nil
+}
+func (tbl *riakURLSigKeyTable) validate() []string {
+	errs := []string{}
+	for i, record := range tbl.Records {
+		if record.DeliveryService == "" {
+			errs = append(errs, fmt.Sprintf("URL Key #%v: Delivery Service is blank!", i))
+		}
+	}
+	return errs
+}
+
+type riakURISignKeyRecord struct {
+	Key             map[string]tc.URISignerKeyset
+	DeliveryService string
+}
+type riakURISignKeyTable struct {
+	Records []riakURISignKeyRecord
+}
+
+func (tbl *riakURISignKeyTable) gatherKeys(cluster *riak.Cluster) error {
+	tbl.Records = []riakURISignKeyRecord{}
+	objs, err := getObjects(cluster, BUCKET_URI_SIG)
+	if err != nil {
+		return fmt.Errorf("RiakURISignKey gatherKeys: %w", err)
+	}
+	for _, obj := range objs {
+		key := map[string]tc.URISignerKeyset{}
+		if err := json.Unmarshal(obj.Value, &key); err != nil {
+			return fmt.Errorf("RiakURISignKey gatherKeys unable to unmarshal object into map[string]tc.URISignerKeySet: %w", err)
+		}
+
+		tbl.Records = append(tbl.Records, riakURISignKeyRecord{
+			DeliveryService: obj.Key,
+			Key:             key,
+		})
+	}
+	return nil
+}
+func (tbl *riakURISignKeyTable) toGeneric() []URISignKey {
+	keys := make([]URISignKey, len(tbl.Records))
+
+	for i, record := range tbl.Records {
+		keys[i] = URISignKey{
+			DeliveryService: record.DeliveryService,
+			Keys:            record.Key,
+		}
+	}
+
+	return keys
+}
+func (tbl *riakURISignKeyTable) fromGeneric(keys []URISignKey) {
+	tbl.Records = make([]riakURISignKeyRecord, len(keys))
+
+	for i, record := range keys {
+		tbl.Records[i] = riakURISignKeyRecord{
+			Key:             record.Keys,
+			DeliveryService: record.DeliveryService,
+		}
+	}
+}
+func (tbl *riakURISignKeyTable) insertKeys(cluster *riak.Cluster) error {
+	for _, record := range tbl.Records {
+		objBytes, err := json.Marshal(record.Key)
+		if err != nil {
+			return fmt.Errorf("RiakURISignKey insertKeys: unable to marshal key: %w", err)
+		}
+
+		err = setObject(cluster, makeRiakObject(objBytes, record.DeliveryService), BUCKET_URI_SIG)
+		if err != nil {
+			return fmt.Errorf("RiakURISignKey insertKeys: %w", err)
+		}
+	}
+	return nil
+}
+func (tbl *riakURISignKeyTable) validate() []string {
+	errs := []string{}
+	for i, record := range tbl.Records {
+		if record.DeliveryService == "" {
+			errs = append(errs, fmt.Sprintf("URI Key #%v: Delivery Service is blank!", i))
+		}
+	}
+	return errs
+}
+
+// Riak functions
+func makeRiakObject(data []byte, key string) *riak.Object {
+	return &riak.Object{
+		ContentType:     rfc.ApplicationJSON,
+		Charset:         "utf-8",
+		ContentEncoding: "utf-8",
+		Key:             key,
+		Value:           data,
+	}
+}
+func getObjects(cluster *riak.Cluster, bucket string) ([]*riak.Object, error) {
+	objs := []*riak.Object{}
+	keys, err := listKeys(cluster, bucket)
+	if err != nil {
+		return nil, err
+	}
+	for _, key := range keys {
+		objects, err := getObject(cluster, bucket, key)
+		if err != nil {
+			return nil, err
+		}
+		if len(objects) > 1 {
+			return nil, fmt.Errorf("Unexpected number of objects %v, ignoring\n", len(objects))
+		}
+
+		objs = append(objs, objects[0])
+	}
+
+	return objs, nil
+}
+func getObject(cluster *riak.Cluster, bucket string, key string) ([]*riak.Object, error) {
+	cmd, err := riak.NewFetchValueCommandBuilder().
+		WithBucket(bucket).
+		WithKey(key).
+		WithTimeout(time.Second * 60).
+		Build()
+	if err != nil {
+		return nil, fmt.Errorf("error building riak fetch value command: %w", err)
+	}
+
+	if err := cluster.Execute(cmd); err != nil {
+		return nil, fmt.Errorf("error executing riak fetch value command: %w", err)
+	}
+
+	fvc := cmd.(*riak.FetchValueCommand)
+	rsp := fvc.Response
+
+	if rsp.IsNotFound {
+		return nil, fmt.Errorf("errors executing riak fetch value command; key not found: %v:%v", bucket, key)
+	}
+
+	return rsp.Values, nil
+}
+func setObject(cluster *riak.Cluster, obj *riak.Object, bucket string) error {
+	cmd, err := riak.NewStoreValueCommandBuilder().
+		WithBucket(bucket).
+		WithContent(obj).
+		WithTimeout(time.Second * 5).

Review comment:
       Now done via the config file.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [trafficcontrol] ocket8888 commented on a change in pull request #5924: Add tool to migrate data between TV backends

Posted by GitBox <gi...@apache.org>.
ocket8888 commented on a change in pull request #5924:
URL: https://github.com/apache/trafficcontrol/pull/5924#discussion_r647797625



##########
File path: docs/source/tools/traffic_vault_migrate.rst
##########
@@ -0,0 +1,107 @@
+..
+..
+.. Licensed under the Apache License, Version 2.0 (the "License");
+.. you may not use this file except in compliance with the License.
+.. You may obtain a copy of the License at
+..
+..     http://www.apache.org/licenses/LICENSE-2.0
+..
+.. Unless required by applicable law or agreed to in writing, software
+.. distributed under the License is distributed on an "AS IS" BASIS,
+.. WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+.. See the License for the specific language governing permissions and
+.. limitations under the License.
+..
+
+.. _traffic_vault_migrate:
+
+#########################
+Traffic Vault Migrate
+#########################
+The ``traffic_vault_migrate`` tool - located at :file:`tools/traffic_vault_migrate/traffic_vault_migrate.go` in the `Apache Traffic Control repository <https://github.com/apache/trafficcontrol>`_ -
+is used to transfer TV keys between database servers. It interfaces directly with each backend so Traffic Ops/Vault being available is not a requirement.
+The tool assumes that the schema for each backend is already setup as according to the :ref:`admin setup <traffic_vault_admin>`.
+
+.. program:: traffic_vault_migrate
+
+Usage
+===========
+``traffic_vault_migrate -from_cfg CFG -to_cfg CFG -from_type TYP -to_type TYP [-confirm] [-compare] [-dry] [-dump]``
+
+.. option:: -compare
+
+		Compare 'to' and 'from' backend keys. Will fetch keys from the dbs of both 'to' and 'from', sorts them by cdn/ds/version and does a deep comparison.
+
+.. option:: -confirm
+
+		Requires confirmation before inserting records (default true)
+
+.. option:: -dry
+
+		Do not perform writes. Will do a basic output of the keys on the 'from' backend.
+
+.. option:: -dump
+
+		Write keys (from 'from' server) to disk in the folder 'dump' with the unix permissions 0640.
+
+		.. warning:: This can write potentially sensitive information to disk, use with care.
+
+.. option:: -from_cfg
+
+		From server config file (default "riak.json")
+
+.. option:: -from_type
+
+		From server types (Riak|PG) (default "Riak")
+
+.. option:: -to_cfg
+
+		To server config file (default "pg.json")
+
+.. option:: -to_type
+
+		From server types (Riak|PG) (default "PG")
+
+Riak
+----------
+
+riak.json
+""""""""""
+
+ :user: The username used to log into the Riak server.
+
+ :password: The password used to log into the Riak server.
+
+ :host: The hostname for the Riak server.
+
+ :port: The port for which the Riak server is listening for protobuf connections.
+
+ :tls: (Optional) Determines whether to verify insecure certificates.
+
+ :tlsVersion: (Optional) Max TLS version supported. Valid values are  "10", "11", "12", "13".
+
+
+Postgres
+---------
+:program:`traffic_vault_migrate` will properly handle both encryption and decryption of postgres data as that is done on the client side.
+
+pg.json
+"""""""""
+
+ :user: The username used to log into the PG server.
+
+ :password: The password for the user to log into the PG server.
+
+ :database: The database to connect to.
+
+ :port: The port on which the PG server is listening.
+
+ :host: The hostname of the PG server.
+
+ :sslmode: The ssl settings for the client connection, `explanation here <https://www.postgresql.org/docs/9.1/libpq-ssl.html#LIBPQ-SSL-SSLMODE-STATEMENTS>`_. Options are 'disable', 'allow', 'prefer', 'require', 'verify-ca' and 'verify-full'
+
+ :aesKey: The base64 encoding of a 16, 24, or 32 bit AES key.
+
+Development
+=============
+To add a plugin, implement the traffic_vault_migrate.go:TVBackend interface and add the backend to the returned values in supportedBackends

Review comment:
       To easily create a formatted link to a public Go symbol, you can use the [``:atc-godoc`` text role](https://traffic-control-cdn.readthedocs.io/en/latest/development/documentation_guidelines.html?highlight=atc-godoc#extension)
   
   Also missing punctuation

##########
File path: docs/source/tools/traffic_vault_migrate.rst
##########
@@ -0,0 +1,107 @@
+..
+..
+.. Licensed under the Apache License, Version 2.0 (the "License");
+.. you may not use this file except in compliance with the License.
+.. You may obtain a copy of the License at
+..
+..     http://www.apache.org/licenses/LICENSE-2.0
+..
+.. Unless required by applicable law or agreed to in writing, software
+.. distributed under the License is distributed on an "AS IS" BASIS,
+.. WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+.. See the License for the specific language governing permissions and
+.. limitations under the License.
+..
+
+.. _traffic_vault_migrate:
+
+#########################
+Traffic Vault Migrate
+#########################
+The ``traffic_vault_migrate`` tool - located at :file:`tools/traffic_vault_migrate/traffic_vault_migrate.go` in the `Apache Traffic Control repository <https://github.com/apache/trafficcontrol>`_ -
+is used to transfer TV keys between database servers. It interfaces directly with each backend so Traffic Ops/Vault being available is not a requirement.
+The tool assumes that the schema for each backend is already setup as according to the :ref:`admin setup <traffic_vault_admin>`.
+
+.. program:: traffic_vault_migrate
+
+Usage
+===========
+``traffic_vault_migrate -from_cfg CFG -to_cfg CFG -from_type TYP -to_type TYP [-confirm] [-compare] [-dry] [-dump]``
+
+.. option:: -compare
+
+		Compare 'to' and 'from' backend keys. Will fetch keys from the dbs of both 'to' and 'from', sorts them by cdn/ds/version and does a deep comparison.
+
+.. option:: -confirm
+
+		Requires confirmation before inserting records (default true)
+
+.. option:: -dry
+
+		Do not perform writes. Will do a basic output of the keys on the 'from' backend.
+
+.. option:: -dump
+
+		Write keys (from 'from' server) to disk in the folder 'dump' with the unix permissions 0640.
+
+		.. warning:: This can write potentially sensitive information to disk, use with care.
+
+.. option:: -from_cfg
+
+		From server config file (default "riak.json")
+
+.. option:: -from_type
+
+		From server types (Riak|PG) (default "Riak")
+
+.. option:: -to_cfg
+
+		To server config file (default "pg.json")
+
+.. option:: -to_type
+
+		From server types (Riak|PG) (default "PG")
+
+Riak
+----------
+
+riak.json
+""""""""""
+
+ :user: The username used to log into the Riak server.
+
+ :password: The password used to log into the Riak server.
+
+ :host: The hostname for the Riak server.
+
+ :port: The port for which the Riak server is listening for protobuf connections.
+
+ :tls: (Optional) Determines whether to verify insecure certificates.
+
+ :tlsVersion: (Optional) Max TLS version supported. Valid values are  "10", "11", "12", "13".
+
+
+Postgres
+---------
+:program:`traffic_vault_migrate` will properly handle both encryption and decryption of postgres data as that is done on the client side.
+
+pg.json
+"""""""""
+
+ :user: The username used to log into the PG server.
+
+ :password: The password for the user to log into the PG server.
+
+ :database: The database to connect to.
+
+ :port: The port on which the PG server is listening.
+
+ :host: The hostname of the PG server.
+
+ :sslmode: The ssl settings for the client connection, `explanation here <https://www.postgresql.org/docs/9.1/libpq-ssl.html#LIBPQ-SSL-SSLMODE-STATEMENTS>`_. Options are 'disable', 'allow', 'prefer', 'require', 'verify-ca' and 'verify-full'

Review comment:
       this links to the docs for an unsupported version of PostgreSQL, supported versions include  Current (13) / 12 / 11 / 10 / 9.6 - I'd suggest just linking to 13 for now, kicks the can as far down the road as possible

##########
File path: docs/source/tools/traffic_vault_migrate.rst
##########
@@ -0,0 +1,107 @@
+..
+..
+.. Licensed under the Apache License, Version 2.0 (the "License");
+.. you may not use this file except in compliance with the License.
+.. You may obtain a copy of the License at
+..
+..     http://www.apache.org/licenses/LICENSE-2.0
+..
+.. Unless required by applicable law or agreed to in writing, software
+.. distributed under the License is distributed on an "AS IS" BASIS,
+.. WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+.. See the License for the specific language governing permissions and
+.. limitations under the License.
+..
+
+.. _traffic_vault_migrate:
+
+#########################
+Traffic Vault Migrate
+#########################
+The ``traffic_vault_migrate`` tool - located at :file:`tools/traffic_vault_migrate/traffic_vault_migrate.go` in the `Apache Traffic Control repository <https://github.com/apache/trafficcontrol>`_ -
+is used to transfer TV keys between database servers. It interfaces directly with each backend so Traffic Ops/Vault being available is not a requirement.
+The tool assumes that the schema for each backend is already setup as according to the :ref:`admin setup <traffic_vault_admin>`.
+
+.. program:: traffic_vault_migrate
+
+Usage
+===========
+``traffic_vault_migrate -from_cfg CFG -to_cfg CFG -from_type TYP -to_type TYP [-confirm] [-compare] [-dry] [-dump]``
+
+.. option:: -compare
+
+		Compare 'to' and 'from' backend keys. Will fetch keys from the dbs of both 'to' and 'from', sorts them by cdn/ds/version and does a deep comparison.
+
+.. option:: -confirm
+
+		Requires confirmation before inserting records (default true)
+
+.. option:: -dry
+
+		Do not perform writes. Will do a basic output of the keys on the 'from' backend.
+
+.. option:: -dump
+
+		Write keys (from 'from' server) to disk in the folder 'dump' with the unix permissions 0640.
+
+		.. warning:: This can write potentially sensitive information to disk, use with care.
+
+.. option:: -from_cfg
+
+		From server config file (default "riak.json")
+
+.. option:: -from_type
+
+		From server types (Riak|PG) (default "Riak")
+
+.. option:: -to_cfg
+
+		To server config file (default "pg.json")
+
+.. option:: -to_type
+
+		From server types (Riak|PG) (default "PG")

Review comment:
       GNU-style long options should use `--` - according to POSIX `-dump` is equivalent to `-d -u -m -p` (Go's default command line argument parser is neither POSIX nor GNU-compliant, which is why generally I advise people use [pborman's `getopt` v2](https://pkg.go.dev/github.com/pborman/getopt/v2))
   
   Also, this is just my personal opinion but I think kebab-case is superior to snake_case for command line utilities and their arguments. It's a GNU standard, and since we're not GNU we don't need to adhere to it, but it is just easier to type when you don't have to hold shift.

##########
File path: docs/source/tools/traffic_vault_migrate.rst
##########
@@ -0,0 +1,107 @@
+..
+..
+.. Licensed under the Apache License, Version 2.0 (the "License");
+.. you may not use this file except in compliance with the License.
+.. You may obtain a copy of the License at
+..
+..     http://www.apache.org/licenses/LICENSE-2.0
+..
+.. Unless required by applicable law or agreed to in writing, software
+.. distributed under the License is distributed on an "AS IS" BASIS,
+.. WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+.. See the License for the specific language governing permissions and
+.. limitations under the License.
+..
+
+.. _traffic_vault_migrate:
+
+#########################
+Traffic Vault Migrate
+#########################
+The ``traffic_vault_migrate`` tool - located at :file:`tools/traffic_vault_migrate/traffic_vault_migrate.go` in the `Apache Traffic Control repository <https://github.com/apache/trafficcontrol>`_ -
+is used to transfer TV keys between database servers. It interfaces directly with each backend so Traffic Ops/Vault being available is not a requirement.
+The tool assumes that the schema for each backend is already setup as according to the :ref:`admin setup <traffic_vault_admin>`.
+
+.. program:: traffic_vault_migrate
+
+Usage
+===========
+``traffic_vault_migrate -from_cfg CFG -to_cfg CFG -from_type TYP -to_type TYP [-confirm] [-compare] [-dry] [-dump]``
+
+.. option:: -compare
+
+		Compare 'to' and 'from' backend keys. Will fetch keys from the dbs of both 'to' and 'from', sorts them by cdn/ds/version and does a deep comparison.
+
+.. option:: -confirm
+
+		Requires confirmation before inserting records (default true)
+
+.. option:: -dry
+
+		Do not perform writes. Will do a basic output of the keys on the 'from' backend.
+
+.. option:: -dump
+
+		Write keys (from 'from' server) to disk in the folder 'dump' with the unix permissions 0640.
+
+		.. warning:: This can write potentially sensitive information to disk, use with care.
+
+.. option:: -from_cfg
+
+		From server config file (default "riak.json")
+
+.. option:: -from_type
+
+		From server types (Riak|PG) (default "Riak")
+
+.. option:: -to_cfg
+
+		To server config file (default "pg.json")
+
+.. option:: -to_type
+
+		From server types (Riak|PG) (default "PG")

Review comment:
       Also options that accept option-arguments should indicate that in the argument to the `.. option` directive e.g. `.. option:: --from-cfg CFG`




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [trafficcontrol] rob05c commented on a change in pull request #5924: Add tool to migrate data between TV backends

Posted by GitBox <gi...@apache.org>.
rob05c commented on a change in pull request #5924:
URL: https://github.com/apache/trafficcontrol/pull/5924#discussion_r652104208



##########
File path: lib/go-tc/deliveryservice_ssl_keys.go
##########
@@ -73,6 +73,7 @@ type SSLKeyRequestFields struct {
 	HostName     *string `json:"hostname,omitempty"`
 	Country      *string `json:"country,omitempty"`
 	State        *string `json:"state,omitempty"`
+	Version      *int    `json:"version,omitempty"`

Review comment:
       +1 - as long as it has a sane default, ideally as close to the prior behavior as possible.
   
   It technically can break code, e.g. `foo := Foo{a,b,c}` will break. But it's reasonable to add fields in minor versions, generally speaking, and any of the alternatives in Go aren't reasonable.
   
   So, as the person who's usually one of the biggest votes against breaking clients, IMO we should consider adding fields as non-breaking, and discourage clients from using that syntax (and the few others which break, which all have alternatives that don't).




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [trafficcontrol] shamrickus commented on a change in pull request #5924: Add tool to migrate data between TV backends

Posted by GitBox <gi...@apache.org>.
shamrickus commented on a change in pull request #5924:
URL: https://github.com/apache/trafficcontrol/pull/5924#discussion_r652059806



##########
File path: tools/traffic_vault_migrate/postgres.go
##########
@@ -0,0 +1,743 @@
+package main
+
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+import (
+	"database/sql"
+	"encoding/base64"
+	"encoding/json"
+	"errors"
+	"fmt"
+	"log"
+	"strconv"
+	"strings"
+
+	"github.com/apache/trafficcontrol/lib/go-tc"
+	util "github.com/apache/trafficcontrol/lib/go-util"
+
+	_ "github.com/lib/pq"
+)
+
+// PGConfig represents the configuration options available to the PG backend
+type PGConfig struct {
+	Host     string `json:"host"`
+	Port     string `json:"port"`
+	User     string `json:"user"`
+	Password string `json:"password"`
+	SSLMode  string `json:"sslmode"`
+	Database string `json:"database"`
+	Key      string `json:"aesKey"`
+	AESKey   []byte
+}
+
+// PGBackend is the Postgres implementation of TVBackend
+type PGBackend struct {
+	sslKey pgSSLKeyTable
+	dnssec pgDNSSecTable
+	uri    pgURISignKeyTable
+	url    pgURLSigKeyTable
+	cfg    PGConfig
+	db     *sql.DB
+}
+
+// String returns a high level overview of the backend and its keys
+func (pg *PGBackend) String() string {
+	data := fmt.Sprintf("PG server %v@%v:%v\n", pg.cfg.User, pg.cfg.Host, pg.cfg.Port)
+	data += fmt.Sprintf("\tSSL Keys: %v\n", len(pg.sslKey.Records))
+	data += fmt.Sprintf("\tDNSSec Keys: %v\n", len(pg.dnssec.Records))
+	data += fmt.Sprintf("\tURI Keys: %v\n", len(pg.uri.Records))
+	data += fmt.Sprintf("\tURL Keys: %v\n", len(pg.url.Records))
+	return data
+}
+
+// Name returns the name for this backend
+func (pg *PGBackend) Name() string {
+	return "PG"
+}
+
+// ReadConfig takes in a filename and will read it into the backends config
+func (pg *PGBackend) ReadConfig(s string) error {
+	err := UnmarshalConfig(s, &pg.cfg)
+	if err != nil {
+		return err
+	}
+
+	pg.cfg.AESKey, err = base64.StdEncoding.DecodeString(pg.cfg.Key)
+	if err != nil {
+		return fmt.Errorf("unable to decode PG AESKey: %w", err)
+	}
+	return nil
+}
+
+// Insert takes the current keys and inserts them into the backend DB
+func (pg *PGBackend) Insert() error {
+	if err := pg.sslKey.insertKeys(pg.db); err != nil {
+		return err
+	}
+	if err := pg.dnssec.insertKeys(pg.db); err != nil {
+		return err
+	}
+	if err := pg.url.insertKeys(pg.db); err != nil {
+		return err
+	}
+	if err := pg.uri.insertKeys(pg.db); err != nil {
+		return err
+	}
+	return nil
+}
+
+// Start initiates the connection to the backend DB
+func (pg *PGBackend) Start() error {
+	sqlStr := fmt.Sprintf("postgres://%s:%s@%s:%s/%s?sslmode=%s", pg.cfg.User, pg.cfg.Password, pg.cfg.Host, pg.cfg.Port, pg.cfg.Database, pg.cfg.SSLMode)
+	db, err := sql.Open("postgres", sqlStr)
+	if err != nil {
+		return fmt.Errorf("unable to start PG client: %w", err)
+	}
+
+	pg.db = db
+	pg.sslKey = pgSSLKeyTable{}
+	pg.dnssec = pgDNSSecTable{}
+	pg.url = pgURLSigKeyTable{}
+	pg.uri = pgURISignKeyTable{}
+
+	return nil
+}
+
+// ValidateKey validates that the keys are valid (in most cases, certain fields are not null)
+func (pg *PGBackend) ValidateKey() []string {
+	var errors []string
+	if errs := pg.sslKey.validate(); errs != nil {
+		errors = append(errors, errs...)
+	}
+	if errs := pg.dnssec.validate(); errs != nil {
+		errors = append(errors, errs...)
+	}
+	if errs := pg.uri.validate(); errs != nil {
+		errors = append(errors, errs...)
+	}
+	if errs := pg.url.validate(); errs != nil {
+		errors = append(errors, errs...)
+	}
+	return errors
+}
+
+// Close terminates the connection to the backend DB
+func (pg *PGBackend) Close() error {
+	return pg.db.Close()
+}
+
+// Ping checks the connection to the backend DB
+func (pg *PGBackend) Ping() error {
+	return pg.db.Ping()
+}
+
+// Fetch gets all of the keys from the backend DB
+func (pg *PGBackend) Fetch() error {
+	if err := pg.sslKey.gatherKeys(pg.db); err != nil {
+		return err
+	}
+
+	if err := pg.dnssec.gatherKeys(pg.db); err != nil {
+		return err
+	}
+
+	if err := pg.url.gatherKeys(pg.db); err != nil {
+		return err
+	}
+
+	if err := pg.uri.gatherKeys(pg.db); err != nil {
+		return err
+	}
+
+	return nil
+}
+
+// GetSSLKeys converts the backends internal key representation into the common representation (SSLKey)
+func (pg *PGBackend) GetSSLKeys() ([]SSLKey, error) {
+	if err := pg.sslKey.decrypt(pg.cfg.AESKey); err != nil {
+		return nil, err
+	}
+	return pg.sslKey.toGeneric(), nil
+}
+
+// SetSSLKeys takes in keys and converts & encrypts the data into the backends internal format
+func (pg *PGBackend) SetSSLKeys(keys []SSLKey) error {
+	pg.sslKey.fromGeneric(keys)
+	return pg.sslKey.encrypt(pg.cfg.AESKey)
+}
+
+// GetDNSSecKeys converts the backends internal key representation into the common representation (DNSSecKey)
+func (pg *PGBackend) GetDNSSecKeys() ([]DNSSecKey, error) {
+	if err := pg.dnssec.decrypt(pg.cfg.AESKey); err != nil {
+		return nil, err
+	}
+	return pg.dnssec.toGeneric(), nil
+}
+
+// SetDNSSecKeys takes in keys and converts & encrypts the data into the backends internal format
+func (pg *PGBackend) SetDNSSecKeys(keys []DNSSecKey) error {
+	pg.dnssec.fromGeneric(keys)
+	return pg.dnssec.encrypt(pg.cfg.AESKey)
+}
+
+// GetURISignKeys converts the pg internal key representation into the common representation (URISignKey)
+func (pg *PGBackend) GetURISignKeys() ([]URISignKey, error) {
+	if err := pg.uri.decrypt(pg.cfg.AESKey); err != nil {
+		return nil, err
+	}
+	return pg.uri.toGeneric(), nil
+}
+
+// SetURISignKeys takes in keys and converts & encrypts the data into the backends internal format
+func (pg *PGBackend) SetURISignKeys(keys []URISignKey) error {
+	pg.uri.fromGeneric(keys)
+	return pg.uri.encrypt(pg.cfg.AESKey)
+}
+
+// GetURLSigKeys converts the backends internal key representation into the common representation (URLSigKey)
+func (pg *PGBackend) GetURLSigKeys() ([]URLSigKey, error) {
+	if err := pg.url.decrypt(pg.cfg.AESKey); err != nil {
+		return nil, err
+	}
+	return pg.url.toGeneric(), nil
+}
+
+// SetURLSigKeys takes in keys and converts & encrypts the data into the backends internal format
+func (pg *PGBackend) SetURLSigKeys(keys []URLSigKey) error {
+	pg.url.fromGeneric(keys)
+	return pg.url.encrypt(pg.cfg.AESKey)
+}
+
+type pgCommonRecord struct {
+	DataEncrypted []byte
+}
+
+type pgDNSSecRecord struct {
+	Key tc.DNSSECKeysTrafficVault
+	CDN string
+	pgCommonRecord
+}
+type pgDNSSecTable struct {
+	Records []pgDNSSecRecord
+}
+
+func (tbl *pgDNSSecTable) gatherKeys(db *sql.DB) error {
+	sz, err := getSize(db, "dnssec")
+	if err != nil {
+		log.Println("PGDNSSec gatherKeys: unable to determine size of dnssec table")
+	}
+	tbl.Records = make([]pgDNSSecRecord, sz)
+
+	rows, err := db.Query("SELECT cdn, data from dnssec")
+	if err != nil {
+		return fmt.Errorf("PGDNSSec gatherKeys: unable to query: %w", err)
+	}
+	defer rows.Close()
+	i := 0
+	for rows.Next() {
+		if i > len(tbl.Records)-1 {
+			return fmt.Errorf("PGDNSSec gatherKeys got more results than expected %v", len(tbl.Records))
+		}
+		err := rows.Scan(&tbl.Records[i].CDN, &tbl.Records[i].DataEncrypted)
+		if err != nil {
+			return fmt.Errorf("PGDNSSec gatherKeys unable to scan row: %w", err)
+		}
+		i += 1
+	}
+	return nil
+}
+func (tbl *pgDNSSecTable) decrypt(aesKey []byte) error {
+	for i, _ := range tbl.Records {
+		err := decryptInto(aesKey, tbl.Records[i].DataEncrypted, &tbl.Records[i].Key)
+		if err != nil {
+			return fmt.Errorf("unable to decrypt into keys: %w", err)
+		}
+	}
+	return nil
+}
+func (tbl *pgDNSSecTable) encrypt(aesKey []byte) error {
+	for i, dns := range tbl.Records {
+		data, err := json.Marshal(&dns.Key)
+		if err != nil {
+			return fmt.Errorf("encrypt issue marshalling keys: %w", err)
+		}
+		dat, err := encrypt(data, aesKey)
+		if err != nil {
+			return fmt.Errorf("encrypt error: %w", err)
+		}
+		tbl.Records[i].DataEncrypted = dat
+	}
+	return nil
+}
+func (tbl *pgDNSSecTable) toGeneric() []DNSSecKey {
+	keys := make([]DNSSecKey, len(tbl.Records))
+
+	for i, record := range tbl.Records {
+		keys[i] = DNSSecKey{
+			CDN:                    record.CDN,
+			DNSSECKeysTrafficVault: record.Key,
+		}
+	}
+
+	return keys
+}
+func (tbl *pgDNSSecTable) fromGeneric(keys []DNSSecKey) {
+	tbl.Records = make([]pgDNSSecRecord, len(keys))
+
+	for i, key := range keys {
+		tbl.Records[i] = pgDNSSecRecord{
+			Key: key.DNSSECKeysTrafficVault,
+			CDN: key.CDN,
+			pgCommonRecord: pgCommonRecord{
+				DataEncrypted: nil,
+			},
+		}
+	}
+}
+func (tbl *pgDNSSecTable) validate() []string {
+	for i, record := range tbl.Records {
+		if record.DataEncrypted == nil && len(record.Key) > 0 {
+			return []string{fmt.Sprintf("DNSSEC Key %v: DataEncrypted is blank!", i)}
+		}
+	}
+	return nil
+}
+func (tbl *pgDNSSecTable) insertKeys(db *sql.DB) error {
+	queryBase := "INSERT INTO dnssec (cdn, data) VALUES "
+	stride := 2
+	queryArgs := make([]interface{}, len(tbl.Records)*stride)
+	for i, record := range tbl.Records {
+		j := i * stride
+		queryArgs[j] = record.CDN
+		queryArgs[j+1] = record.DataEncrypted
+	}
+	return insertIntoTable(db, queryBase, stride, queryArgs)
+}
+
+type pgSSLKeyRecord struct {
+	Keys tc.DeliveryServiceSSLKeys
+	pgCommonRecord
+
+	// These records are stored on the table but are duplicated
+	DeliveryService string
+	CDN             string
+}
+type pgSSLKeyTable struct {
+	Records []pgSSLKeyRecord
+}
+
+func (tbl *pgSSLKeyTable) insertKeys(db *sql.DB) error {
+	queryBase := "INSERT INTO sslkey (deliveryservice, data, cdn, version) VALUES "
+	duplicateKeys := 2
+	stride := 4
+	queryArgs := make([]interface{}, len(tbl.Records)*stride*duplicateKeys)
+	for i, record := range tbl.Records {
+		j := i * duplicateKeys * stride
+
+		queryArgs[j] = record.DeliveryService
+		queryArgs[j+1] = record.DataEncrypted
+		queryArgs[j+2] = record.CDN
+		queryArgs[j+3] = record.Keys.Version.String()
+
+		queryArgs[j+4] = record.DeliveryService
+		queryArgs[j+5] = record.DataEncrypted
+		queryArgs[j+6] = record.CDN
+		queryArgs[j+7] = "latest"
+	}
+	return insertIntoTable(db, queryBase, 4, queryArgs)
+}
+func (tbl *pgSSLKeyTable) gatherKeys(db *sql.DB) error {
+	sz, err := getSize(db, "sslkey WHERE version='latest'")
+	if err != nil {
+		return fmt.Errorf("PGSSLKey gatherKeys unable to determine size of sslkey table: %w", err)
+	}
+	tbl.Records = make([]pgSSLKeyRecord, sz)
+
+	rows, err := db.Query("SELECT data, deliveryservice, cdn from sslkey WHERE version = 'latest'")
+	if err != nil {
+		return fmt.Errorf("PGSSLKey gatherKeys unable to query: %w", err)
+	}
+	defer rows.Close()
+	i := 0
+	for rows.Next() {
+		if i > len(tbl.Records)-1 {
+			return errors.New("PGSSLKey gatherKeys: got more results than expected")
+		}
+		err := rows.Scan(&tbl.Records[i].DataEncrypted, &tbl.Records[i].DeliveryService, &tbl.Records[i].CDN)
+		if err != nil {
+			return fmt.Errorf("PGSSLKey gatherKeys unable to scan row: %w", err)
+		}
+		i += 1
+	}
+	return nil
+}
+func (tbl *pgSSLKeyTable) decrypt(aesKey []byte) error {
+	for i, dns := range tbl.Records {
+		err := decryptInto(aesKey, dns.DataEncrypted, &tbl.Records[i].Keys)
+		if err != nil {
+			return fmt.Errorf("unable to decrypt into keys: %w", err)
+		}
+	}
+	return nil
+}
+func (tbl *pgSSLKeyTable) encrypt(aesKey []byte) error {
+	for i, dns := range tbl.Records {
+		data, err := json.Marshal(dns.Keys)
+		if err != nil {
+			return fmt.Errorf("encrypt issue marshalling keys: %w", err)
+		}
+		dat, err := encrypt(data, aesKey)
+		if err != nil {
+			return fmt.Errorf("encrypt error: %w", err)
+		}
+		tbl.Records[i].DataEncrypted = dat
+	}
+	return nil
+}
+func (tbl *pgSSLKeyTable) toGeneric() []SSLKey {
+	keys := make([]SSLKey, len(tbl.Records))
+
+	for i, record := range tbl.Records {
+		keys[i] = SSLKey{
+			DeliveryServiceSSLKeys: record.Keys,
+		}
+	}
+	return keys
+}
+func (tbl *pgSSLKeyTable) fromGeneric(keys []SSLKey) {
+	tbl.Records = make([]pgSSLKeyRecord, len(keys))
+
+	for i, key := range keys {
+		tbl.Records[i] = pgSSLKeyRecord{
+			Keys: key.DeliveryServiceSSLKeys,
+			pgCommonRecord: pgCommonRecord{
+				DataEncrypted: nil,
+			},
+			DeliveryService: key.DeliveryService,
+			CDN:             key.CDN,
+		}
+	}
+}
+func (tbl *pgSSLKeyTable) validate() []string {
+	defaultKey := tc.DeliveryServiceSSLKeys{}
+	var errors []string
+	fmtStr := "SSL Key %v: %v"
+	for i, record := range tbl.Records {
+		if record.Keys == defaultKey {
+			errors = append(errors, fmt.Sprintf(fmtStr, i, "DS SSL Keys are default!"))
+		} else if record.Keys.Key == "" {
+			errors = append(errors, fmt.Sprintf(fmtStr, i, "Key is blank!"))
+		} else if record.Keys.CDN == "" {
+			errors = append(errors, fmt.Sprintf(fmtStr, i, "CDN is blank!"))
+		} else if record.Keys.DeliveryService == "" {
+			errors = append(errors, fmt.Sprintf(fmtStr, i, "DS is blank!"))
+		} else if record.DataEncrypted == nil {
+			errors = append(errors, fmt.Sprintf(fmtStr, i, "DataEncrypted is blank!"))
+		} else if record.Keys.Version.String() == "" {
+			errors = append(errors, fmt.Sprintf(fmtStr, i, "Version is blank!"))
+		}
+	}
+	return errors
+}
+
+type pgURLSigKeyRecord struct {
+	Keys            tc.URLSigKeys
+	DeliveryService string
+	pgCommonRecord
+}
+type pgURLSigKeyTable struct {
+	Records []pgURLSigKeyRecord
+}
+
+func (tbl *pgURLSigKeyTable) insertKeys(db *sql.DB) error {
+	queryBase := "INSERT INTO url_sig_key (deliveryservice, data) VALUES "
+	stride := 2
+	queryArgs := make([]interface{}, len(tbl.Records)*stride)
+	for i, record := range tbl.Records {
+		j := i * stride
+		queryArgs[j] = record.DeliveryService
+		queryArgs[j+1] = record.DataEncrypted
+	}
+	return insertIntoTable(db, queryBase, stride, queryArgs)
+}
+func (tbl *pgURLSigKeyTable) gatherKeys(db *sql.DB) error {
+	sz, err := getSize(db, "url_sig_key")
+	if err != nil {
+		log.Println("PGURLSigKey gatherKeys: unable to determine url_sig_key table size")
+	}
+	tbl.Records = make([]pgURLSigKeyRecord, sz)
+
+	rows, err := db.Query("SELECT deliveryservice, data from url_sig_key")
+	if err != nil {
+		return fmt.Errorf("PGURLSigKey gatherKeys error creating query: %w", err)
+	}
+	defer rows.Close()
+	i := 0
+	for rows.Next() {
+		if i > len(tbl.Records)-1 {
+			return fmt.Errorf("PGURLSigKey gatherKeys: got more results than expected %v", len(tbl.Records))
+		}
+		err := rows.Scan(&tbl.Records[i].DeliveryService, &tbl.Records[i].DataEncrypted)
+		if err != nil {
+			return fmt.Errorf("PGURLSigKey gatherKeys: unable to scan row: %w", err)
+		}
+		i += 1
+	}
+	return nil
+}
+func (tbl *pgURLSigKeyTable) decrypt(aesKey []byte) error {
+	for i, sig := range tbl.Records {
+		err := decryptInto(aesKey, sig.DataEncrypted, &tbl.Records[i].Keys)
+		if err != nil {
+			return fmt.Errorf("unable to decrypt into keys: %w", err)
+		}
+	}
+	return nil
+}
+func (tbl *pgURLSigKeyTable) encrypt(aesKey []byte) error {
+	for i, sig := range tbl.Records {
+		data, err := json.Marshal(&sig.Keys)
+		if err != nil {
+			return fmt.Errorf("encrypt issue marshalling keys: %w", err)
+		}
+
+		dat, err := encrypt(data, aesKey)
+		if err != nil {
+			return fmt.Errorf("encrypt error: %w", err)
+		}
+		tbl.Records[i].DataEncrypted = dat
+	}
+	return nil
+}
+func (tbl *pgURLSigKeyTable) toGeneric() []URLSigKey {
+	keys := make([]URLSigKey, len(tbl.Records))
+
+	for i, record := range tbl.Records {
+		keys[i] = URLSigKey{
+			DeliveryService: record.DeliveryService,
+			URLSigKeys:      record.Keys,
+		}
+	}
+	return keys
+}
+func (tbl *pgURLSigKeyTable) fromGeneric(keys []URLSigKey) {
+	tbl.Records = make([]pgURLSigKeyRecord, len(keys))
+
+	for i, key := range keys {
+		tbl.Records[i] = pgURLSigKeyRecord{
+			Keys:            key.URLSigKeys,
+			DeliveryService: key.DeliveryService,
+			pgCommonRecord: pgCommonRecord{
+				DataEncrypted: nil,
+			},
+		}
+	}
+}
+func (tbl *pgURLSigKeyTable) validate() []string {
+	for i, record := range tbl.Records {
+		if record.DataEncrypted == nil && len(record.Keys) > 0 {
+			return []string{fmt.Sprintf("URl Sig Key %v: DataEncrypted is blank!", i)}
+		}
+	}
+	return nil
+}
+
+type pgURISignKeyRecord struct {
+	Keys            map[string]tc.URISignerKeyset
+	DeliveryService string
+	pgCommonRecord
+}
+type pgURISignKeyTable struct {
+	Records []pgURISignKeyRecord
+}
+
+func (tbl *pgURISignKeyTable) insertKeys(db *sql.DB) error {
+	queryBase := "INSERT INTO uri_signing_key (deliveryservice, data) VALUES "

Review comment:
       That would be preferable, makes it so I don't have to use a CTE. I'll add it to this PR (otherwise testing the psql backend would be impossible).




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [trafficcontrol] shamrickus commented on a change in pull request #5924: Add tool to migrate data between TV backends

Posted by GitBox <gi...@apache.org>.
shamrickus commented on a change in pull request #5924:
URL: https://github.com/apache/trafficcontrol/pull/5924#discussion_r651988101



##########
File path: tools/traffic_vault_migrate/postgres.go
##########
@@ -0,0 +1,743 @@
+package main
+
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+import (
+	"database/sql"
+	"encoding/base64"
+	"encoding/json"
+	"errors"
+	"fmt"
+	"log"
+	"strconv"
+	"strings"
+
+	"github.com/apache/trafficcontrol/lib/go-tc"
+	util "github.com/apache/trafficcontrol/lib/go-util"
+
+	_ "github.com/lib/pq"
+)
+
+// PGConfig represents the configuration options available to the PG backend
+type PGConfig struct {
+	Host     string `json:"host"`
+	Port     string `json:"port"`
+	User     string `json:"user"`
+	Password string `json:"password"`
+	SSLMode  string `json:"sslmode"`
+	Database string `json:"database"`
+	Key      string `json:"aesKey"`
+	AESKey   []byte
+}
+
+// PGBackend is the Postgres implementation of TVBackend
+type PGBackend struct {
+	sslKey pgSSLKeyTable
+	dnssec pgDNSSecTable
+	uri    pgURISignKeyTable
+	url    pgURLSigKeyTable
+	cfg    PGConfig
+	db     *sql.DB
+}
+
+// String returns a high level overview of the backend and its keys
+func (pg *PGBackend) String() string {
+	data := fmt.Sprintf("PG server %v@%v:%v\n", pg.cfg.User, pg.cfg.Host, pg.cfg.Port)
+	data += fmt.Sprintf("\tSSL Keys: %v\n", len(pg.sslKey.Records))
+	data += fmt.Sprintf("\tDNSSec Keys: %v\n", len(pg.dnssec.Records))
+	data += fmt.Sprintf("\tURI Keys: %v\n", len(pg.uri.Records))
+	data += fmt.Sprintf("\tURL Keys: %v\n", len(pg.url.Records))
+	return data
+}
+
+// Name returns the name for this backend
+func (pg *PGBackend) Name() string {
+	return "PG"
+}
+
+// ReadConfig takes in a filename and will read it into the backends config
+func (pg *PGBackend) ReadConfig(s string) error {
+	err := UnmarshalConfig(s, &pg.cfg)
+	if err != nil {
+		return err
+	}
+
+	pg.cfg.AESKey, err = base64.StdEncoding.DecodeString(pg.cfg.Key)
+	if err != nil {
+		return fmt.Errorf("unable to decode PG AESKey: %w", err)
+	}
+	return nil
+}
+
+// Insert takes the current keys and inserts them into the backend DB
+func (pg *PGBackend) Insert() error {
+	if err := pg.sslKey.insertKeys(pg.db); err != nil {
+		return err
+	}
+	if err := pg.dnssec.insertKeys(pg.db); err != nil {
+		return err
+	}
+	if err := pg.url.insertKeys(pg.db); err != nil {
+		return err
+	}
+	if err := pg.uri.insertKeys(pg.db); err != nil {
+		return err
+	}
+	return nil
+}
+
+// Start initiates the connection to the backend DB
+func (pg *PGBackend) Start() error {
+	sqlStr := fmt.Sprintf("postgres://%s:%s@%s:%s/%s?sslmode=%s", pg.cfg.User, pg.cfg.Password, pg.cfg.Host, pg.cfg.Port, pg.cfg.Database, pg.cfg.SSLMode)
+	db, err := sql.Open("postgres", sqlStr)
+	if err != nil {
+		return fmt.Errorf("unable to start PG client: %w", err)
+	}
+
+	pg.db = db
+	pg.sslKey = pgSSLKeyTable{}
+	pg.dnssec = pgDNSSecTable{}
+	pg.url = pgURLSigKeyTable{}
+	pg.uri = pgURISignKeyTable{}
+
+	return nil
+}
+
+// ValidateKey validates that the keys are valid (in most cases, certain fields are not null)
+func (pg *PGBackend) ValidateKey() []string {
+	var errors []string
+	if errs := pg.sslKey.validate(); errs != nil {
+		errors = append(errors, errs...)
+	}
+	if errs := pg.dnssec.validate(); errs != nil {
+		errors = append(errors, errs...)
+	}
+	if errs := pg.uri.validate(); errs != nil {
+		errors = append(errors, errs...)
+	}
+	if errs := pg.url.validate(); errs != nil {
+		errors = append(errors, errs...)
+	}
+	return errors
+}
+
+// Close terminates the connection to the backend DB
+func (pg *PGBackend) Close() error {
+	return pg.db.Close()
+}
+
+// Ping checks the connection to the backend DB
+func (pg *PGBackend) Ping() error {
+	return pg.db.Ping()
+}
+
+// Fetch gets all of the keys from the backend DB
+func (pg *PGBackend) Fetch() error {
+	if err := pg.sslKey.gatherKeys(pg.db); err != nil {
+		return err
+	}
+
+	if err := pg.dnssec.gatherKeys(pg.db); err != nil {
+		return err
+	}
+
+	if err := pg.url.gatherKeys(pg.db); err != nil {
+		return err
+	}
+
+	if err := pg.uri.gatherKeys(pg.db); err != nil {
+		return err
+	}
+
+	return nil
+}
+
+// GetSSLKeys converts the backends internal key representation into the common representation (SSLKey)
+func (pg *PGBackend) GetSSLKeys() ([]SSLKey, error) {
+	if err := pg.sslKey.decrypt(pg.cfg.AESKey); err != nil {
+		return nil, err
+	}
+	return pg.sslKey.toGeneric(), nil
+}
+
+// SetSSLKeys takes in keys and converts & encrypts the data into the backends internal format
+func (pg *PGBackend) SetSSLKeys(keys []SSLKey) error {
+	pg.sslKey.fromGeneric(keys)
+	return pg.sslKey.encrypt(pg.cfg.AESKey)
+}
+
+// GetDNSSecKeys converts the backends internal key representation into the common representation (DNSSecKey)
+func (pg *PGBackend) GetDNSSecKeys() ([]DNSSecKey, error) {
+	if err := pg.dnssec.decrypt(pg.cfg.AESKey); err != nil {
+		return nil, err
+	}
+	return pg.dnssec.toGeneric(), nil
+}
+
+// SetDNSSecKeys takes in keys and converts & encrypts the data into the backends internal format
+func (pg *PGBackend) SetDNSSecKeys(keys []DNSSecKey) error {
+	pg.dnssec.fromGeneric(keys)
+	return pg.dnssec.encrypt(pg.cfg.AESKey)
+}
+
+// GetURISignKeys converts the pg internal key representation into the common representation (URISignKey)
+func (pg *PGBackend) GetURISignKeys() ([]URISignKey, error) {
+	if err := pg.uri.decrypt(pg.cfg.AESKey); err != nil {
+		return nil, err
+	}
+	return pg.uri.toGeneric(), nil
+}
+
+// SetURISignKeys takes in keys and converts & encrypts the data into the backends internal format
+func (pg *PGBackend) SetURISignKeys(keys []URISignKey) error {
+	pg.uri.fromGeneric(keys)
+	return pg.uri.encrypt(pg.cfg.AESKey)
+}
+
+// GetURLSigKeys converts the backends internal key representation into the common representation (URLSigKey)
+func (pg *PGBackend) GetURLSigKeys() ([]URLSigKey, error) {
+	if err := pg.url.decrypt(pg.cfg.AESKey); err != nil {
+		return nil, err
+	}
+	return pg.url.toGeneric(), nil
+}
+
+// SetURLSigKeys takes in keys and converts & encrypts the data into the backends internal format
+func (pg *PGBackend) SetURLSigKeys(keys []URLSigKey) error {
+	pg.url.fromGeneric(keys)
+	return pg.url.encrypt(pg.cfg.AESKey)
+}
+
+type pgCommonRecord struct {
+	DataEncrypted []byte
+}
+
+type pgDNSSecRecord struct {
+	Key tc.DNSSECKeysTrafficVault
+	CDN string
+	pgCommonRecord
+}
+type pgDNSSecTable struct {
+	Records []pgDNSSecRecord
+}
+
+func (tbl *pgDNSSecTable) gatherKeys(db *sql.DB) error {
+	sz, err := getSize(db, "dnssec")
+	if err != nil {
+		log.Println("PGDNSSec gatherKeys: unable to determine size of dnssec table")
+	}
+	tbl.Records = make([]pgDNSSecRecord, sz)
+
+	rows, err := db.Query("SELECT cdn, data from dnssec")
+	if err != nil {
+		return fmt.Errorf("PGDNSSec gatherKeys: unable to query: %w", err)
+	}
+	defer rows.Close()
+	i := 0
+	for rows.Next() {
+		if i > len(tbl.Records)-1 {
+			return fmt.Errorf("PGDNSSec gatherKeys got more results than expected %v", len(tbl.Records))
+		}
+		err := rows.Scan(&tbl.Records[i].CDN, &tbl.Records[i].DataEncrypted)
+		if err != nil {
+			return fmt.Errorf("PGDNSSec gatherKeys unable to scan row: %w", err)
+		}
+		i += 1
+	}
+	return nil
+}
+func (tbl *pgDNSSecTable) decrypt(aesKey []byte) error {
+	for i, _ := range tbl.Records {
+		err := decryptInto(aesKey, tbl.Records[i].DataEncrypted, &tbl.Records[i].Key)
+		if err != nil {
+			return fmt.Errorf("unable to decrypt into keys: %w", err)
+		}
+	}
+	return nil
+}
+func (tbl *pgDNSSecTable) encrypt(aesKey []byte) error {
+	for i, dns := range tbl.Records {
+		data, err := json.Marshal(&dns.Key)
+		if err != nil {
+			return fmt.Errorf("encrypt issue marshalling keys: %w", err)
+		}
+		dat, err := encrypt(data, aesKey)
+		if err != nil {
+			return fmt.Errorf("encrypt error: %w", err)
+		}
+		tbl.Records[i].DataEncrypted = dat
+	}
+	return nil
+}
+func (tbl *pgDNSSecTable) toGeneric() []DNSSecKey {
+	keys := make([]DNSSecKey, len(tbl.Records))
+
+	for i, record := range tbl.Records {
+		keys[i] = DNSSecKey{
+			CDN:                    record.CDN,
+			DNSSECKeysTrafficVault: record.Key,
+		}
+	}
+
+	return keys
+}
+func (tbl *pgDNSSecTable) fromGeneric(keys []DNSSecKey) {
+	tbl.Records = make([]pgDNSSecRecord, len(keys))
+
+	for i, key := range keys {
+		tbl.Records[i] = pgDNSSecRecord{
+			Key: key.DNSSECKeysTrafficVault,
+			CDN: key.CDN,
+			pgCommonRecord: pgCommonRecord{
+				DataEncrypted: nil,
+			},
+		}
+	}
+}
+func (tbl *pgDNSSecTable) validate() []string {
+	for i, record := range tbl.Records {
+		if record.DataEncrypted == nil && len(record.Key) > 0 {
+			return []string{fmt.Sprintf("DNSSEC Key %v: DataEncrypted is blank!", i)}
+		}
+	}
+	return nil
+}
+func (tbl *pgDNSSecTable) insertKeys(db *sql.DB) error {
+	queryBase := "INSERT INTO dnssec (cdn, data) VALUES "
+	stride := 2
+	queryArgs := make([]interface{}, len(tbl.Records)*stride)
+	for i, record := range tbl.Records {
+		j := i * stride
+		queryArgs[j] = record.CDN
+		queryArgs[j+1] = record.DataEncrypted
+	}
+	return insertIntoTable(db, queryBase, stride, queryArgs)
+}
+
+type pgSSLKeyRecord struct {
+	Keys tc.DeliveryServiceSSLKeys
+	pgCommonRecord
+
+	// These records are stored on the table but are duplicated
+	DeliveryService string
+	CDN             string
+}
+type pgSSLKeyTable struct {
+	Records []pgSSLKeyRecord
+}
+
+func (tbl *pgSSLKeyTable) insertKeys(db *sql.DB) error {
+	queryBase := "INSERT INTO sslkey (deliveryservice, data, cdn, version) VALUES "
+	duplicateKeys := 2
+	stride := 4
+	queryArgs := make([]interface{}, len(tbl.Records)*stride*duplicateKeys)
+	for i, record := range tbl.Records {
+		j := i * duplicateKeys * stride
+
+		queryArgs[j] = record.DeliveryService
+		queryArgs[j+1] = record.DataEncrypted
+		queryArgs[j+2] = record.CDN
+		queryArgs[j+3] = record.Keys.Version.String()
+
+		queryArgs[j+4] = record.DeliveryService
+		queryArgs[j+5] = record.DataEncrypted
+		queryArgs[j+6] = record.CDN
+		queryArgs[j+7] = "latest"
+	}
+	return insertIntoTable(db, queryBase, 4, queryArgs)
+}
+func (tbl *pgSSLKeyTable) gatherKeys(db *sql.DB) error {
+	sz, err := getSize(db, "sslkey WHERE version='latest'")
+	if err != nil {
+		return fmt.Errorf("PGSSLKey gatherKeys unable to determine size of sslkey table: %w", err)
+	}
+	tbl.Records = make([]pgSSLKeyRecord, sz)
+
+	rows, err := db.Query("SELECT data, deliveryservice, cdn from sslkey WHERE version = 'latest'")
+	if err != nil {
+		return fmt.Errorf("PGSSLKey gatherKeys unable to query: %w", err)
+	}
+	defer rows.Close()
+	i := 0
+	for rows.Next() {
+		if i > len(tbl.Records)-1 {
+			return errors.New("PGSSLKey gatherKeys: got more results than expected")
+		}
+		err := rows.Scan(&tbl.Records[i].DataEncrypted, &tbl.Records[i].DeliveryService, &tbl.Records[i].CDN)
+		if err != nil {
+			return fmt.Errorf("PGSSLKey gatherKeys unable to scan row: %w", err)
+		}
+		i += 1
+	}
+	return nil
+}
+func (tbl *pgSSLKeyTable) decrypt(aesKey []byte) error {
+	for i, dns := range tbl.Records {
+		err := decryptInto(aesKey, dns.DataEncrypted, &tbl.Records[i].Keys)
+		if err != nil {
+			return fmt.Errorf("unable to decrypt into keys: %w", err)
+		}
+	}
+	return nil
+}
+func (tbl *pgSSLKeyTable) encrypt(aesKey []byte) error {
+	for i, dns := range tbl.Records {
+		data, err := json.Marshal(dns.Keys)
+		if err != nil {
+			return fmt.Errorf("encrypt issue marshalling keys: %w", err)
+		}
+		dat, err := encrypt(data, aesKey)
+		if err != nil {
+			return fmt.Errorf("encrypt error: %w", err)
+		}
+		tbl.Records[i].DataEncrypted = dat
+	}
+	return nil
+}
+func (tbl *pgSSLKeyTable) toGeneric() []SSLKey {
+	keys := make([]SSLKey, len(tbl.Records))
+
+	for i, record := range tbl.Records {
+		keys[i] = SSLKey{
+			DeliveryServiceSSLKeys: record.Keys,
+		}
+	}
+	return keys
+}
+func (tbl *pgSSLKeyTable) fromGeneric(keys []SSLKey) {
+	tbl.Records = make([]pgSSLKeyRecord, len(keys))
+
+	for i, key := range keys {
+		tbl.Records[i] = pgSSLKeyRecord{
+			Keys: key.DeliveryServiceSSLKeys,
+			pgCommonRecord: pgCommonRecord{
+				DataEncrypted: nil,
+			},
+			DeliveryService: key.DeliveryService,
+			CDN:             key.CDN,
+		}
+	}
+}
+func (tbl *pgSSLKeyTable) validate() []string {
+	defaultKey := tc.DeliveryServiceSSLKeys{}
+	var errors []string
+	fmtStr := "SSL Key %v: %v"
+	for i, record := range tbl.Records {
+		if record.Keys == defaultKey {
+			errors = append(errors, fmt.Sprintf(fmtStr, i, "DS SSL Keys are default!"))
+		} else if record.Keys.Key == "" {
+			errors = append(errors, fmt.Sprintf(fmtStr, i, "Key is blank!"))
+		} else if record.Keys.CDN == "" {
+			errors = append(errors, fmt.Sprintf(fmtStr, i, "CDN is blank!"))
+		} else if record.Keys.DeliveryService == "" {
+			errors = append(errors, fmt.Sprintf(fmtStr, i, "DS is blank!"))
+		} else if record.DataEncrypted == nil {
+			errors = append(errors, fmt.Sprintf(fmtStr, i, "DataEncrypted is blank!"))
+		} else if record.Keys.Version.String() == "" {
+			errors = append(errors, fmt.Sprintf(fmtStr, i, "Version is blank!"))
+		}
+	}
+	return errors
+}
+
+type pgURLSigKeyRecord struct {
+	Keys            tc.URLSigKeys
+	DeliveryService string
+	pgCommonRecord
+}
+type pgURLSigKeyTable struct {
+	Records []pgURLSigKeyRecord
+}
+
+func (tbl *pgURLSigKeyTable) insertKeys(db *sql.DB) error {
+	queryBase := "INSERT INTO url_sig_key (deliveryservice, data) VALUES "
+	stride := 2
+	queryArgs := make([]interface{}, len(tbl.Records)*stride)
+	for i, record := range tbl.Records {
+		j := i * stride
+		queryArgs[j] = record.DeliveryService
+		queryArgs[j+1] = record.DataEncrypted
+	}
+	return insertIntoTable(db, queryBase, stride, queryArgs)
+}
+func (tbl *pgURLSigKeyTable) gatherKeys(db *sql.DB) error {
+	sz, err := getSize(db, "url_sig_key")
+	if err != nil {
+		log.Println("PGURLSigKey gatherKeys: unable to determine url_sig_key table size")
+	}
+	tbl.Records = make([]pgURLSigKeyRecord, sz)
+
+	rows, err := db.Query("SELECT deliveryservice, data from url_sig_key")
+	if err != nil {
+		return fmt.Errorf("PGURLSigKey gatherKeys error creating query: %w", err)
+	}
+	defer rows.Close()
+	i := 0
+	for rows.Next() {
+		if i > len(tbl.Records)-1 {
+			return fmt.Errorf("PGURLSigKey gatherKeys: got more results than expected %v", len(tbl.Records))
+		}
+		err := rows.Scan(&tbl.Records[i].DeliveryService, &tbl.Records[i].DataEncrypted)
+		if err != nil {
+			return fmt.Errorf("PGURLSigKey gatherKeys: unable to scan row: %w", err)
+		}
+		i += 1
+	}
+	return nil
+}
+func (tbl *pgURLSigKeyTable) decrypt(aesKey []byte) error {
+	for i, sig := range tbl.Records {
+		err := decryptInto(aesKey, sig.DataEncrypted, &tbl.Records[i].Keys)
+		if err != nil {
+			return fmt.Errorf("unable to decrypt into keys: %w", err)
+		}
+	}
+	return nil
+}
+func (tbl *pgURLSigKeyTable) encrypt(aesKey []byte) error {
+	for i, sig := range tbl.Records {
+		data, err := json.Marshal(&sig.Keys)
+		if err != nil {
+			return fmt.Errorf("encrypt issue marshalling keys: %w", err)
+		}
+
+		dat, err := encrypt(data, aesKey)
+		if err != nil {
+			return fmt.Errorf("encrypt error: %w", err)
+		}
+		tbl.Records[i].DataEncrypted = dat
+	}
+	return nil
+}
+func (tbl *pgURLSigKeyTable) toGeneric() []URLSigKey {
+	keys := make([]URLSigKey, len(tbl.Records))
+
+	for i, record := range tbl.Records {
+		keys[i] = URLSigKey{
+			DeliveryService: record.DeliveryService,
+			URLSigKeys:      record.Keys,
+		}
+	}
+	return keys
+}
+func (tbl *pgURLSigKeyTable) fromGeneric(keys []URLSigKey) {
+	tbl.Records = make([]pgURLSigKeyRecord, len(keys))
+
+	for i, key := range keys {
+		tbl.Records[i] = pgURLSigKeyRecord{
+			Keys:            key.URLSigKeys,
+			DeliveryService: key.DeliveryService,
+			pgCommonRecord: pgCommonRecord{
+				DataEncrypted: nil,
+			},
+		}
+	}
+}
+func (tbl *pgURLSigKeyTable) validate() []string {
+	for i, record := range tbl.Records {
+		if record.DataEncrypted == nil && len(record.Keys) > 0 {
+			return []string{fmt.Sprintf("URl Sig Key %v: DataEncrypted is blank!", i)}
+		}
+	}
+	return nil
+}
+
+type pgURISignKeyRecord struct {
+	Keys            map[string]tc.URISignerKeyset
+	DeliveryService string
+	pgCommonRecord
+}
+type pgURISignKeyTable struct {
+	Records []pgURISignKeyRecord
+}
+
+func (tbl *pgURISignKeyTable) insertKeys(db *sql.DB) error {
+	queryBase := "INSERT INTO uri_signing_key (deliveryservice, data) VALUES "

Review comment:
       Yep, at least from the DBs perspective; I don't think we can actually do that through the TO API. 




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [trafficcontrol] rawlinp merged pull request #5924: Add tool to migrate data between TV backends

Posted by GitBox <gi...@apache.org>.
rawlinp merged pull request #5924:
URL: https://github.com/apache/trafficcontrol/pull/5924


   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [trafficcontrol] shamrickus commented on a change in pull request #5924: Add tool to migrate data between TV backends

Posted by GitBox <gi...@apache.org>.
shamrickus commented on a change in pull request #5924:
URL: https://github.com/apache/trafficcontrol/pull/5924#discussion_r651099277



##########
File path: tools/traffic_vault_migrate/postgres.go
##########
@@ -0,0 +1,743 @@
+package main
+
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+import (
+	"database/sql"
+	"encoding/base64"
+	"encoding/json"
+	"errors"
+	"fmt"
+	"log"
+	"strconv"
+	"strings"
+
+	"github.com/apache/trafficcontrol/lib/go-tc"
+	util "github.com/apache/trafficcontrol/lib/go-util"
+
+	_ "github.com/lib/pq"
+)
+
+// PGConfig represents the configuration options available to the PG backend
+type PGConfig struct {
+	Host     string `json:"host"`
+	Port     string `json:"port"`
+	User     string `json:"user"`
+	Password string `json:"password"`
+	SSLMode  string `json:"sslmode"`
+	Database string `json:"database"`
+	Key      string `json:"aesKey"`
+	AESKey   []byte
+}
+
+// PGBackend is the Postgres implementation of TVBackend
+type PGBackend struct {
+	sslKey pgSSLKeyTable
+	dnssec pgDNSSecTable
+	uri    pgURISignKeyTable
+	url    pgURLSigKeyTable
+	cfg    PGConfig
+	db     *sql.DB
+}
+
+// String returns a high level overview of the backend and its keys
+func (pg *PGBackend) String() string {
+	data := fmt.Sprintf("PG server %v@%v:%v\n", pg.cfg.User, pg.cfg.Host, pg.cfg.Port)
+	data += fmt.Sprintf("\tSSL Keys: %v\n", len(pg.sslKey.Records))
+	data += fmt.Sprintf("\tDNSSec Keys: %v\n", len(pg.dnssec.Records))
+	data += fmt.Sprintf("\tURI Keys: %v\n", len(pg.uri.Records))
+	data += fmt.Sprintf("\tURL Keys: %v\n", len(pg.url.Records))
+	return data
+}
+
+// Name returns the name for this backend
+func (pg *PGBackend) Name() string {
+	return "PG"
+}
+
+// ReadConfig takes in a filename and will read it into the backends config
+func (pg *PGBackend) ReadConfig(s string) error {
+	err := UnmarshalConfig(s, &pg.cfg)
+	if err != nil {
+		return err
+	}
+
+	pg.cfg.AESKey, err = base64.StdEncoding.DecodeString(pg.cfg.Key)
+	if err != nil {
+		return fmt.Errorf("unable to decode PG AESKey: %w", err)
+	}
+	return nil
+}
+
+// Insert takes the current keys and inserts them into the backend DB
+func (pg *PGBackend) Insert() error {
+	if err := pg.sslKey.insertKeys(pg.db); err != nil {
+		return err
+	}
+	if err := pg.dnssec.insertKeys(pg.db); err != nil {
+		return err
+	}
+	if err := pg.url.insertKeys(pg.db); err != nil {
+		return err
+	}
+	if err := pg.uri.insertKeys(pg.db); err != nil {
+		return err
+	}
+	return nil
+}
+
+// Start initiates the connection to the backend DB
+func (pg *PGBackend) Start() error {
+	sqlStr := fmt.Sprintf("postgres://%s:%s@%s:%s/%s?sslmode=%s", pg.cfg.User, pg.cfg.Password, pg.cfg.Host, pg.cfg.Port, pg.cfg.Database, pg.cfg.SSLMode)
+	db, err := sql.Open("postgres", sqlStr)
+	if err != nil {
+		return fmt.Errorf("unable to start PG client: %w", err)
+	}
+
+	pg.db = db
+	pg.sslKey = pgSSLKeyTable{}
+	pg.dnssec = pgDNSSecTable{}
+	pg.url = pgURLSigKeyTable{}
+	pg.uri = pgURISignKeyTable{}
+
+	return nil
+}
+
+// ValidateKey validates that the keys are valid (in most cases, certain fields are not null)
+func (pg *PGBackend) ValidateKey() []string {
+	var errors []string
+	if errs := pg.sslKey.validate(); errs != nil {
+		errors = append(errors, errs...)
+	}
+	if errs := pg.dnssec.validate(); errs != nil {
+		errors = append(errors, errs...)
+	}
+	if errs := pg.uri.validate(); errs != nil {
+		errors = append(errors, errs...)
+	}
+	if errs := pg.url.validate(); errs != nil {
+		errors = append(errors, errs...)
+	}
+	return errors
+}
+
+// Close terminates the connection to the backend DB
+func (pg *PGBackend) Close() error {
+	return pg.db.Close()
+}
+
+// Ping checks the connection to the backend DB
+func (pg *PGBackend) Ping() error {
+	return pg.db.Ping()
+}
+
+// Fetch gets all of the keys from the backend DB
+func (pg *PGBackend) Fetch() error {
+	if err := pg.sslKey.gatherKeys(pg.db); err != nil {
+		return err
+	}
+
+	if err := pg.dnssec.gatherKeys(pg.db); err != nil {
+		return err
+	}
+
+	if err := pg.url.gatherKeys(pg.db); err != nil {
+		return err
+	}
+
+	if err := pg.uri.gatherKeys(pg.db); err != nil {
+		return err
+	}
+
+	return nil
+}
+
+// GetSSLKeys converts the backends internal key representation into the common representation (SSLKey)
+func (pg *PGBackend) GetSSLKeys() ([]SSLKey, error) {
+	if err := pg.sslKey.decrypt(pg.cfg.AESKey); err != nil {
+		return nil, err
+	}
+	return pg.sslKey.toGeneric(), nil
+}
+
+// SetSSLKeys takes in keys and converts & encrypts the data into the backends internal format
+func (pg *PGBackend) SetSSLKeys(keys []SSLKey) error {
+	pg.sslKey.fromGeneric(keys)
+	return pg.sslKey.encrypt(pg.cfg.AESKey)
+}
+
+// GetDNSSecKeys converts the backends internal key representation into the common representation (DNSSecKey)
+func (pg *PGBackend) GetDNSSecKeys() ([]DNSSecKey, error) {
+	if err := pg.dnssec.decrypt(pg.cfg.AESKey); err != nil {
+		return nil, err
+	}
+	return pg.dnssec.toGeneric(), nil
+}
+
+// SetDNSSecKeys takes in keys and converts & encrypts the data into the backends internal format
+func (pg *PGBackend) SetDNSSecKeys(keys []DNSSecKey) error {
+	pg.dnssec.fromGeneric(keys)
+	return pg.dnssec.encrypt(pg.cfg.AESKey)
+}
+
+// GetURISignKeys converts the pg internal key representation into the common representation (URISignKey)
+func (pg *PGBackend) GetURISignKeys() ([]URISignKey, error) {
+	if err := pg.uri.decrypt(pg.cfg.AESKey); err != nil {
+		return nil, err
+	}
+	return pg.uri.toGeneric(), nil
+}
+
+// SetURISignKeys takes in keys and converts & encrypts the data into the backends internal format
+func (pg *PGBackend) SetURISignKeys(keys []URISignKey) error {
+	pg.uri.fromGeneric(keys)
+	return pg.uri.encrypt(pg.cfg.AESKey)
+}
+
+// GetURLSigKeys converts the backends internal key representation into the common representation (URLSigKey)
+func (pg *PGBackend) GetURLSigKeys() ([]URLSigKey, error) {
+	if err := pg.url.decrypt(pg.cfg.AESKey); err != nil {
+		return nil, err
+	}
+	return pg.url.toGeneric(), nil
+}
+
+// SetURLSigKeys takes in keys and converts & encrypts the data into the backends internal format
+func (pg *PGBackend) SetURLSigKeys(keys []URLSigKey) error {
+	pg.url.fromGeneric(keys)
+	return pg.url.encrypt(pg.cfg.AESKey)
+}
+
+type pgCommonRecord struct {
+	DataEncrypted []byte
+}
+
+type pgDNSSecRecord struct {
+	Key tc.DNSSECKeysTrafficVault
+	CDN string
+	pgCommonRecord
+}
+type pgDNSSecTable struct {
+	Records []pgDNSSecRecord
+}
+
+func (tbl *pgDNSSecTable) gatherKeys(db *sql.DB) error {
+	sz, err := getSize(db, "dnssec")
+	if err != nil {
+		log.Println("PGDNSSec gatherKeys: unable to determine size of dnssec table")
+	}
+	tbl.Records = make([]pgDNSSecRecord, sz)
+
+	rows, err := db.Query("SELECT cdn, data from dnssec")
+	if err != nil {
+		return fmt.Errorf("PGDNSSec gatherKeys: unable to query: %w", err)
+	}
+	defer rows.Close()
+	i := 0
+	for rows.Next() {
+		if i > len(tbl.Records)-1 {
+			return fmt.Errorf("PGDNSSec gatherKeys got more results than expected %v", len(tbl.Records))
+		}
+		err := rows.Scan(&tbl.Records[i].CDN, &tbl.Records[i].DataEncrypted)
+		if err != nil {
+			return fmt.Errorf("PGDNSSec gatherKeys unable to scan row: %w", err)
+		}
+		i += 1
+	}
+	return nil
+}
+func (tbl *pgDNSSecTable) decrypt(aesKey []byte) error {
+	for i, _ := range tbl.Records {
+		err := decryptInto(aesKey, tbl.Records[i].DataEncrypted, &tbl.Records[i].Key)
+		if err != nil {
+			return fmt.Errorf("unable to decrypt into keys: %w", err)
+		}
+	}
+	return nil
+}
+func (tbl *pgDNSSecTable) encrypt(aesKey []byte) error {
+	for i, dns := range tbl.Records {
+		data, err := json.Marshal(&dns.Key)
+		if err != nil {
+			return fmt.Errorf("encrypt issue marshalling keys: %w", err)
+		}
+		dat, err := encrypt(data, aesKey)
+		if err != nil {
+			return fmt.Errorf("encrypt error: %w", err)
+		}
+		tbl.Records[i].DataEncrypted = dat
+	}
+	return nil
+}
+func (tbl *pgDNSSecTable) toGeneric() []DNSSecKey {
+	keys := make([]DNSSecKey, len(tbl.Records))
+
+	for i, record := range tbl.Records {
+		keys[i] = DNSSecKey{
+			CDN:                    record.CDN,
+			DNSSECKeysTrafficVault: record.Key,
+		}
+	}
+
+	return keys
+}
+func (tbl *pgDNSSecTable) fromGeneric(keys []DNSSecKey) {
+	tbl.Records = make([]pgDNSSecRecord, len(keys))
+
+	for i, key := range keys {
+		tbl.Records[i] = pgDNSSecRecord{
+			Key: key.DNSSECKeysTrafficVault,
+			CDN: key.CDN,
+			pgCommonRecord: pgCommonRecord{
+				DataEncrypted: nil,
+			},
+		}
+	}
+}
+func (tbl *pgDNSSecTable) validate() []string {
+	for i, record := range tbl.Records {
+		if record.DataEncrypted == nil && len(record.Key) > 0 {
+			return []string{fmt.Sprintf("DNSSEC Key %v: DataEncrypted is blank!", i)}
+		}
+	}
+	return nil
+}
+func (tbl *pgDNSSecTable) insertKeys(db *sql.DB) error {
+	queryBase := "INSERT INTO dnssec (cdn, data) VALUES "
+	stride := 2
+	queryArgs := make([]interface{}, len(tbl.Records)*stride)
+	for i, record := range tbl.Records {
+		j := i * stride
+		queryArgs[j] = record.CDN
+		queryArgs[j+1] = record.DataEncrypted
+	}
+	return insertIntoTable(db, queryBase, stride, queryArgs)
+}
+
+type pgSSLKeyRecord struct {
+	Keys tc.DeliveryServiceSSLKeys
+	pgCommonRecord
+
+	// These records are stored on the table but are duplicated
+	DeliveryService string
+	CDN             string
+}
+type pgSSLKeyTable struct {
+	Records []pgSSLKeyRecord
+}
+
+func (tbl *pgSSLKeyTable) insertKeys(db *sql.DB) error {
+	queryBase := "INSERT INTO sslkey (deliveryservice, data, cdn, version) VALUES "
+	duplicateKeys := 2
+	stride := 4
+	queryArgs := make([]interface{}, len(tbl.Records)*stride*duplicateKeys)
+	for i, record := range tbl.Records {
+		j := i * duplicateKeys * stride
+
+		queryArgs[j] = record.DeliveryService
+		queryArgs[j+1] = record.DataEncrypted
+		queryArgs[j+2] = record.CDN
+		queryArgs[j+3] = record.Keys.Version.String()
+
+		queryArgs[j+4] = record.DeliveryService
+		queryArgs[j+5] = record.DataEncrypted
+		queryArgs[j+6] = record.CDN
+		queryArgs[j+7] = "latest"
+	}
+	return insertIntoTable(db, queryBase, 4, queryArgs)
+}
+func (tbl *pgSSLKeyTable) gatherKeys(db *sql.DB) error {
+	sz, err := getSize(db, "sslkey WHERE version='latest'")

Review comment:
       Hmmm...I was misreading how we inserted ssl keys, you are correct. I'll remove the filter.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [trafficcontrol] shamrickus commented on a change in pull request #5924: Add tool to migrate data between TV backends

Posted by GitBox <gi...@apache.org>.
shamrickus commented on a change in pull request #5924:
URL: https://github.com/apache/trafficcontrol/pull/5924#discussion_r648509171



##########
File path: tools/traffic_vault_migrate/postgres.go
##########
@@ -0,0 +1,716 @@
+package main
+
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+import (
+	"database/sql"
+	"encoding/base64"
+	"encoding/json"
+	"errors"
+	"fmt"
+	"github.com/apache/trafficcontrol/lib/go-tc"
+	util "github.com/apache/trafficcontrol/lib/go-util"
+	_ "github.com/lib/pq"
+	"log"
+	"reflect"
+	"strconv"
+	"strings"
+)
+
+type PGConfig struct {
+	Host     string `json:"host"`
+	Port     string `json:"port"`
+	User     string `json:"user"`
+	Password string `json:"password"`
+	SSLMode  string `json:"sslmode"`
+	Database string `json:"database"`
+	Key      string `json:"aesKey"`
+	AESKey   []byte
+}
+type PGBackend struct {
+	sslKey PGSSLKeyTable
+	dnssec PGDNSSecTable
+	uri    PGURISignKeyTable
+	url    PGURLSigKeyTable
+	cfg    PGConfig
+	db     *sql.DB
+}
+
+func (pg *PGBackend) String() string {
+	data := fmt.Sprintf("PG server %v@%v:%v\n", pg.cfg.User, pg.cfg.Host, pg.cfg.Port)
+	data += fmt.Sprintf("\tSSL Keys: %v\n", len(pg.sslKey.Records))
+	data += fmt.Sprintf("\tDNSSec Keys: %v\n", len(pg.dnssec.Records))
+	data += fmt.Sprintf("\tURI Keys: %v\n", len(pg.uri.Records))
+	data += fmt.Sprintf("\tURL Keys: %v\n", len(pg.url.Records))
+	return data
+}
+func (pg *PGBackend) Name() string {
+	return "PG"
+}
+func (pg *PGBackend) ReadConfig(s string) error {
+	genericCfg, err := UnmarshalConfig(s, reflect.TypeOf(pg.cfg))

Review comment:
       For code reuse, and consistency. Figured loading configs is something that should be done the same for each backend (Riak also calls this).




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [trafficcontrol] rawlinp commented on a change in pull request #5924: Add tool to migrate data between TV backends

Posted by GitBox <gi...@apache.org>.
rawlinp commented on a change in pull request #5924:
URL: https://github.com/apache/trafficcontrol/pull/5924#discussion_r653894403



##########
File path: traffic_ops/app/db/traffic_vault_migrate/postgres.go
##########
@@ -0,0 +1,744 @@
+package main
+
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+import (
+	"database/sql"
+	"encoding/base64"
+	"encoding/json"
+	"errors"
+	"fmt"
+	"strconv"
+	"strings"
+
+	"github.com/apache/trafficcontrol/lib/go-log"
+	"github.com/apache/trafficcontrol/lib/go-tc"
+	util "github.com/apache/trafficcontrol/lib/go-util"
+
+	_ "github.com/lib/pq"
+)
+
+// PGConfig represents the configuration options available to the PG backend.
+type PGConfig struct {
+	Host     string `json:"host"`
+	Port     string `json:"port"`
+	User     string `json:"user"`
+	Password string `json:"password"`
+	SSLMode  string `json:"sslmode"`
+	Database string `json:"database"`
+	Key      string `json:"aesKey"`

Review comment:
       nit: maybe this should be named `KeyBase64` to differentiate it  more from `AESKey`

##########
File path: traffic_ops/app/db/traffic_vault_migrate/traffic_vault_migrate.go
##########
@@ -0,0 +1,611 @@
+package main
+
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+import (
+	"encoding/json"
+	"errors"
+	"fmt"
+	"io/ioutil"
+	stdlog "log"
+	"os"
+	"reflect"
+	"sort"
+	"strings"
+
+	"github.com/pborman/getopt/v2"
+
+	"github.com/apache/trafficcontrol/lib/go-log"
+	"github.com/apache/trafficcontrol/lib/go-tc"
+)
+
+var (
+	fromType    string
+	toType      string
+	fromCfgPath string
+	toCfgPath   string
+	logCfgPath  string
+	keyFile     string
+	dry         bool
+	compare     bool
+	noConfirm   bool
+	dump        bool
+	logLevel    string
+
+	cfg config = config{
+		LogLocationError:   log.LogLocationStderr,
+		LogLocationWarning: log.LogLocationStdout,
+		LogLocationInfo:    log.LogLocationStdout,
+		LogLocationDebug:   log.LogLocationNull,
+		LogLocationEvent:   log.LogLocationNull,
+	}
+	riakBE RiakBackend = RiakBackend{}
+	pgBE   PGBackend   = PGBackend{}
+)
+
+func init() {
+	fromTypePtr := getopt.StringLong("fromType", 't', riakBE.Name(), fmt.Sprintf("From server types (%v)", strings.Join(supportedTypes(), "|")))
+	if fromTypePtr == nil {
+		stdlog.Fatal("unable to load fromType")
+	}
+	fromType = *fromTypePtr
+
+	toTypePtr := getopt.StringLong("toType", 'o', pgBE.Name(), fmt.Sprintf("From server types (%v)", strings.Join(supportedTypes(), "|")))
+	if toTypePtr == nil {
+		stdlog.Fatal("unable to load toType")
+	}
+	toType = *toTypePtr
+
+	toCfgPtr := getopt.StringLong("toCfgPath", 'g', "pg.json", "To server config file")
+	if toCfgPtr == nil {
+		stdlog.Fatal("unable to load toCfg")
+	}
+	toCfgPath = *toCfgPtr
+
+	fromCfgPtr := getopt.StringLong("fromCfgPath", 'f', "riak.json", "From server config file")
+	if fromCfgPtr == nil {
+		stdlog.Fatal("unable to load fromCfg")
+	}
+	fromCfgPath = *fromCfgPtr
+
+	getopt.FlagLong(&dry, "dry", 'r', "Do not perform writes").
+		SetOptional().
+		SetFlag().
+		SetGroup("no_insert")
+
+	getopt.FlagLong(&compare, "compare", 'c', "Compare to and from server records").
+		SetOptional().
+		SetFlag().
+		SetGroup("no_insert")
+
+	getopt.FlagLong(&noConfirm, "noConfirm", 'm', "Requires confirmation before inserting records").
+		SetFlag()
+
+	getopt.FlagLong(&dump, "dump", 'd', "Write keys (from 'from' server) to disk").
+		SetOptional().
+		SetGroup("disk_bck").
+		SetFlag()
+
+	getopt.FlagLong(&keyFile, "fill", 'i', "Insert data into `to` server with data this directory").
+		SetOptional().
+		SetGroup("disk_bck")
+
+	getopt.FlagLong(&logCfgPath, "logCfg", 'l', "Log configuration file").
+		SetOptional().
+		SetGroup("log")
+
+	getopt.FlagLong(&logLevel, "logLevel", 'e', "Print everything at above specified log level (error|warning|info|debug|event)").
+		SetOptional().
+		SetGroup("log")
+}
+
+// supportBackends returns the backends available in this tool.
+func supportedBackends() []TVBackend {
+	return []TVBackend{
+		&riakBE, &pgBE,
+	}
+}
+
+func main() {
+	getopt.ParseV2()
+
+	initConfig()
+
+	var fromSrv TVBackend
+	var toSrv TVBackend
+
+	importData := keyFile != ""
+	toSrvUsed := !dump && !dry || keyFile != ""
+
+	if !importData {
+		log.Infof("Initiating fromSrv %s...\n", fromType)
+		if !validateType(fromType) {
+			log.Errorln("Unknown fromType " + fromType)
+			os.Exit(1)
+		}
+		fromSrv = getBackendFromType(fromType)
+		if err := fromSrv.ReadConfigFile(fromCfgPath); err != nil {
+			log.Errorf("Unable to read fromSrv cfg: %v", err)
+			os.Exit(1)
+		}
+
+		if err := fromSrv.Start(); err != nil {
+			log.Errorf("issue starting fromSrv: %v", err)
+			os.Exit(1)
+		}
+		defer log.Close(fromSrv, "closing fromSrv")
+
+		if err := fromSrv.Ping(); err != nil {
+			log.Errorf("Unable to ping fromSrv: %v", err)
+			os.Exit(1)
+		}
+	}
+
+	if toSrvUsed {
+		log.Infof("Initiating toSrv %s...\n", toType)
+		if !validateType(toType) {
+			log.Errorln("Unknown toType " + toType)
+			os.Exit(1)
+		}
+		toSrv = getBackendFromType(toType)
+
+		if err := toSrv.ReadConfigFile(toCfgPath); err != nil {
+			log.Errorf("Unable to read toSrv cfg: %v", err)
+			os.Exit(1)
+		}
+
+		if err := toSrv.Start(); err != nil {
+			log.Errorf("issue starting toSrv: %v", err)
+			os.Exit(1)
+		}
+		defer log.Close(toSrv, "closing toSrv")
+
+		if err := toSrv.Ping(); err != nil {
+			log.Errorf("Unable to ping toSrv: %v", err)
+			os.Exit(1)
+		}
+	}
+
+	var fromSecret Secrets
+	if !importData {
+		var err error
+		log.Infof("Fetching data from %s...\n", fromSrv.Name())
+		if err = fromSrv.Fetch(); err != nil {
+			log.Errorf("Unable to fetch fromSrv data: %v", err)
+			os.Exit(1)
+		}
+
+		if fromSecret, err = GetKeys(fromSrv); err != nil {
+			log.Errorln(err)
+			os.Exit(1)
+		}
+
+		if err := Validate(fromSrv); err != nil {
+			log.Errorln(err)
+			os.Exit(1)
+		}
+
+	} else {
+		err := fromSecret.fill(keyFile)
+		if err != nil {
+			log.Errorln(err)
+			os.Exit(1)
+		}
+	}
+
+	if dump {
+		log.Infof("Dumping data from %s...\n", fromSrv.Name())
+		fromSecret.dump("dump")
+		return
+	}
+
+	if compare {
+		log.Infof("Fetching data from %s...\n", toSrv.Name())
+		if err := toSrv.Fetch(); err != nil {
+			log.Errorf("Unable to fetch toSrv data: %v\n", err)
+			os.Exit(1)
+		}
+
+		toSecret, err := GetKeys(toSrv)
+		if err != nil {
+			log.Errorln(err)
+			os.Exit(1)
+		}
+		log.Infoln("Validating " + toSrv.Name())
+		if err := toSrv.ValidateKey(); err != nil && len(err) > 0 {
+			log.Errorln(strings.Join(err, "\n"))
+			os.Exit(1)
+		}
+
+		fromSecret.sort()
+		toSecret.sort()
+
+		if !importData {
+			log.Infoln(fromSrv.String())
+		} else {
+			log.Infof("Disk backup:\n\tSSL Keys: %d\n\tDNSSec Keys: %d\n\tURI Keys: %d\n\tURL Keys: %d\n", len(fromSecret.sslkeys), len(fromSecret.dnssecKeys), len(fromSecret.uriKeys), len(fromSecret.urlKeys))
+		}
+		log.Infoln(toSrv.String())
+
+		if !reflect.DeepEqual(fromSecret.sslkeys, toSecret.sslkeys) {

Review comment:
       I just want to say this is awesome -- I'm glad you thought of adding comparison functionality 😄 . This will definitely help give us the warm fuzzies.

##########
File path: traffic_ops/app/db/traffic_vault_migrate/traffic_vault_migrate.go
##########
@@ -0,0 +1,611 @@
+package main
+
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+import (
+	"encoding/json"
+	"errors"
+	"fmt"
+	"io/ioutil"
+	stdlog "log"
+	"os"
+	"reflect"
+	"sort"
+	"strings"
+
+	"github.com/pborman/getopt/v2"
+
+	"github.com/apache/trafficcontrol/lib/go-log"
+	"github.com/apache/trafficcontrol/lib/go-tc"
+)
+
+var (
+	fromType    string
+	toType      string
+	fromCfgPath string
+	toCfgPath   string
+	logCfgPath  string
+	keyFile     string
+	dry         bool
+	compare     bool
+	noConfirm   bool
+	dump        bool
+	logLevel    string
+
+	cfg config = config{
+		LogLocationError:   log.LogLocationStderr,
+		LogLocationWarning: log.LogLocationStdout,
+		LogLocationInfo:    log.LogLocationStdout,
+		LogLocationDebug:   log.LogLocationNull,
+		LogLocationEvent:   log.LogLocationNull,
+	}
+	riakBE RiakBackend = RiakBackend{}
+	pgBE   PGBackend   = PGBackend{}
+)
+
+func init() {
+	fromTypePtr := getopt.StringLong("fromType", 't', riakBE.Name(), fmt.Sprintf("From server types (%v)", strings.Join(supportedTypes(), "|")))
+	if fromTypePtr == nil {
+		stdlog.Fatal("unable to load fromType")
+	}
+	fromType = *fromTypePtr
+
+	toTypePtr := getopt.StringLong("toType", 'o', pgBE.Name(), fmt.Sprintf("From server types (%v)", strings.Join(supportedTypes(), "|")))
+	if toTypePtr == nil {
+		stdlog.Fatal("unable to load toType")
+	}
+	toType = *toTypePtr
+
+	toCfgPtr := getopt.StringLong("toCfgPath", 'g', "pg.json", "To server config file")
+	if toCfgPtr == nil {
+		stdlog.Fatal("unable to load toCfg")
+	}
+	toCfgPath = *toCfgPtr
+
+	fromCfgPtr := getopt.StringLong("fromCfgPath", 'f', "riak.json", "From server config file")
+	if fromCfgPtr == nil {
+		stdlog.Fatal("unable to load fromCfg")
+	}
+	fromCfgPath = *fromCfgPtr
+
+	getopt.FlagLong(&dry, "dry", 'r', "Do not perform writes").
+		SetOptional().
+		SetFlag().
+		SetGroup("no_insert")
+
+	getopt.FlagLong(&compare, "compare", 'c', "Compare to and from server records").
+		SetOptional().
+		SetFlag().
+		SetGroup("no_insert")
+
+	getopt.FlagLong(&noConfirm, "noConfirm", 'm', "Requires confirmation before inserting records").
+		SetFlag()
+
+	getopt.FlagLong(&dump, "dump", 'd', "Write keys (from 'from' server) to disk").
+		SetOptional().
+		SetGroup("disk_bck").
+		SetFlag()
+
+	getopt.FlagLong(&keyFile, "fill", 'i', "Insert data into `to` server with data this directory").
+		SetOptional().
+		SetGroup("disk_bck")
+
+	getopt.FlagLong(&logCfgPath, "logCfg", 'l', "Log configuration file").
+		SetOptional().
+		SetGroup("log")
+
+	getopt.FlagLong(&logLevel, "logLevel", 'e', "Print everything at above specified log level (error|warning|info|debug|event)").
+		SetOptional().
+		SetGroup("log")
+}
+
+// supportBackends returns the backends available in this tool.
+func supportedBackends() []TVBackend {
+	return []TVBackend{
+		&riakBE, &pgBE,
+	}
+}
+
+func main() {
+	getopt.ParseV2()
+
+	initConfig()
+
+	var fromSrv TVBackend
+	var toSrv TVBackend
+
+	importData := keyFile != ""
+	toSrvUsed := !dump && !dry || keyFile != ""

Review comment:
       nit: `keyFile != ""` can be replaced w/ `importData`, and this would probably be more readable with parentheses: `(!dump && !dry) || importData` -- assuming that's the precedence for this

##########
File path: traffic_ops/app/db/traffic_vault_migrate/postgres.go
##########
@@ -0,0 +1,744 @@
+package main
+
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+import (
+	"database/sql"
+	"encoding/base64"
+	"encoding/json"
+	"errors"
+	"fmt"
+	"strconv"
+	"strings"
+
+	"github.com/apache/trafficcontrol/lib/go-log"
+	"github.com/apache/trafficcontrol/lib/go-tc"
+	util "github.com/apache/trafficcontrol/lib/go-util"
+
+	_ "github.com/lib/pq"
+)
+
+// PGConfig represents the configuration options available to the PG backend.
+type PGConfig struct {
+	Host     string `json:"host"`
+	Port     string `json:"port"`
+	User     string `json:"user"`
+	Password string `json:"password"`
+	SSLMode  string `json:"sslmode"`
+	Database string `json:"database"`
+	Key      string `json:"aesKey"`
+	AESKey   []byte
+}
+
+// PGBackend is the Postgres implementation of TVBackend.
+type PGBackend struct {
+	sslKey         pgSSLKeyTable
+	dnssec         pgDNSSecTable
+	uriSigningKeys pgURISignKeyTable
+	urlSigKeys     pgURLSigKeyTable
+	cfg            PGConfig
+	db             *sql.DB
+}
+
+// String returns a high level overview of the backend and its keys.
+func (pg *PGBackend) String() string {
+	data := fmt.Sprintf("PG server %s@%s:%s\n", pg.cfg.User, pg.cfg.Host, pg.cfg.Port)
+	data += fmt.Sprintf("\tSSL Keys: %d\n", len(pg.sslKey.Records))
+	data += fmt.Sprintf("\tDNSSec Keys: %d\n", len(pg.dnssec.Records))
+	data += fmt.Sprintf("\tURI Signing Keys: %d\n", len(pg.uriSigningKeys.Records))
+	data += fmt.Sprintf("\tURL Sig Keys: %d\n", len(pg.urlSigKeys.Records))
+	return data
+}
+
+// Name returns the name for this backend.
+func (pg *PGBackend) Name() string {
+	return "PG"
+}
+
+// ReadConfigFile takes in a filename and will read it into the backends config.
+func (pg *PGBackend) ReadConfigFile(configFile string) error {
+	var err error
+	if err = UnmarshalConfig(configFile, &pg.cfg); err != nil {
+		return err
+	}
+
+	if pg.cfg.AESKey, err = base64.StdEncoding.DecodeString(pg.cfg.Key); err != nil {
+		return fmt.Errorf("unable to decode PG AESKey '%s': %w", pg.cfg.Key, err)
+	}
+
+	if err = util.ValidateAESKey(pg.cfg.AESKey); err != nil {
+		return fmt.Errorf("unable to validate PG AESKey '%s'", pg.cfg.Key)
+	}
+	return nil
+}
+
+// Insert takes the current keys and inserts them into the backend DB.
+func (pg *PGBackend) Insert() error {
+	if err := pg.sslKey.insertKeys(pg.db); err != nil {
+		return err
+	}
+	if err := pg.dnssec.insertKeys(pg.db); err != nil {
+		return err
+	}
+	if err := pg.urlSigKeys.insertKeys(pg.db); err != nil {
+		return err
+	}
+	if err := pg.uriSigningKeys.insertKeys(pg.db); err != nil {
+		return err
+	}
+	return nil
+}
+
+// Start initiates the connection to the backend DB.
+func (pg *PGBackend) Start() error {
+	sqlStr := fmt.Sprintf("postgres://%s:%s@%s:%s/%s?sslmode=%s", pg.cfg.User, pg.cfg.Password, pg.cfg.Host, pg.cfg.Port, pg.cfg.Database, pg.cfg.SSLMode)
+	db, err := sql.Open("postgres", sqlStr)
+	if err != nil {
+		sqlStr = strings.Replace(sqlStr, pg.cfg.Password, "*", 1)
+		return fmt.Errorf("unable to start PG client with connection string '%s': %w", sqlStr, err)
+	}
+
+	pg.db = db
+	pg.sslKey = pgSSLKeyTable{}
+	pg.dnssec = pgDNSSecTable{}
+	pg.urlSigKeys = pgURLSigKeyTable{}
+	pg.uriSigningKeys = pgURISignKeyTable{}
+
+	return nil
+}
+
+// ValidateKey validates that the keys are valid (in most cases, certain fields are not null).
+func (pg *PGBackend) ValidateKey() []string {
+	var allErrs []string
+	if errs := pg.sslKey.validate(); errs != nil {
+		allErrs = append(allErrs, errs...)
+	}
+	if errs := pg.dnssec.validate(); errs != nil {
+		allErrs = append(allErrs, errs...)
+	}
+	if errs := pg.uriSigningKeys.validate(); errs != nil {
+		allErrs = append(allErrs, errs...)
+	}
+	if errs := pg.urlSigKeys.validate(); errs != nil {
+		allErrs = append(allErrs, errs...)
+	}
+	return allErrs
+}
+
+// Close terminates the connection to the backend DB.
+func (pg *PGBackend) Close() error {
+	return pg.db.Close()
+}
+
+// Ping checks the connection to the backend DB.
+func (pg *PGBackend) Ping() error {
+	return pg.db.Ping()
+}
+
+// Fetch gets all of the keys from the backend DB.
+func (pg *PGBackend) Fetch() error {
+	if err := pg.sslKey.gatherKeys(pg.db); err != nil {
+		return err
+	}
+
+	if err := pg.dnssec.gatherKeys(pg.db); err != nil {
+		return err
+	}
+
+	if err := pg.urlSigKeys.gatherKeys(pg.db); err != nil {
+		return err
+	}
+
+	if err := pg.uriSigningKeys.gatherKeys(pg.db); err != nil {
+		return err
+	}
+
+	return nil
+}
+
+// GetSSLKeys converts the backends internal key representation into the common representation (SSLKey).
+func (pg *PGBackend) GetSSLKeys() ([]SSLKey, error) {
+	if err := pg.sslKey.decrypt(pg.cfg.AESKey); err != nil {
+		return nil, err
+	}
+	return pg.sslKey.toGeneric(), nil
+}
+
+// SetSSLKeys takes in keys and converts & encrypts the data into the backends internal format.
+func (pg *PGBackend) SetSSLKeys(keys []SSLKey) error {
+	pg.sslKey.fromGeneric(keys)
+	return pg.sslKey.encrypt(pg.cfg.AESKey)
+}
+
+// GetDNSSecKeys converts the backends internal key representation into the common representation (DNSSecKey).
+func (pg *PGBackend) GetDNSSecKeys() ([]DNSSecKey, error) {
+	if err := pg.dnssec.decrypt(pg.cfg.AESKey); err != nil {
+		return nil, err
+	}
+	return pg.dnssec.toGeneric(), nil
+}
+
+// SetDNSSecKeys takes in keys and converts & encrypts the data into the backends internal format.
+func (pg *PGBackend) SetDNSSecKeys(keys []DNSSecKey) error {
+	pg.dnssec.fromGeneric(keys)
+	return pg.dnssec.encrypt(pg.cfg.AESKey)
+}
+
+// GetURISignKeys converts the pg internal key representation into the common representation (URISignKey).
+func (pg *PGBackend) GetURISignKeys() ([]URISignKey, error) {
+	if err := pg.uriSigningKeys.decrypt(pg.cfg.AESKey); err != nil {
+		return nil, err
+	}
+	return pg.uriSigningKeys.toGeneric(), nil
+}
+
+// SetURISignKeys takes in keys and converts & encrypts the data into the backends internal format.
+func (pg *PGBackend) SetURISignKeys(keys []URISignKey) error {
+	pg.uriSigningKeys.fromGeneric(keys)
+	return pg.uriSigningKeys.encrypt(pg.cfg.AESKey)
+}
+
+// GetURLSigKeys converts the backends internal key representation into the common representation (URLSigKey).
+func (pg *PGBackend) GetURLSigKeys() ([]URLSigKey, error) {
+	if err := pg.urlSigKeys.decrypt(pg.cfg.AESKey); err != nil {
+		return nil, err
+	}
+	return pg.urlSigKeys.toGeneric(), nil
+}
+
+// SetURLSigKeys takes in keys and converts & encrypts the data into the backends internal format.
+func (pg *PGBackend) SetURLSigKeys(keys []URLSigKey) error {
+	pg.urlSigKeys.fromGeneric(keys)
+	return pg.urlSigKeys.encrypt(pg.cfg.AESKey)
+}
+
+type pgCommonRecord struct {
+	DataEncrypted []byte
+}
+
+type pgDNSSecRecord struct {
+	Key tc.DNSSECKeysTrafficVault
+	CDN string
+	pgCommonRecord
+}
+type pgDNSSecTable struct {
+	Records []pgDNSSecRecord
+}
+
+func (tbl *pgDNSSecTable) gatherKeys(db *sql.DB) error {
+	sz, err := getSize(db, "dnssec")
+	if err != nil {
+		log.Errorln("PGDNSSec gatherKeys: unable to determine size of dnssec table")
+	}
+	tbl.Records = make([]pgDNSSecRecord, sz)
+
+	query := "SELECT cdn, data from dnssec"
+	rows, err := db.Query(query)
+	if err != nil {
+		return fmt.Errorf("PGDNSSec gatherKeys: unable to run query '%s': %w", query, err)
+	}
+	defer log.Close(rows, "closing dnssec query")
+	i := 0
+	for rows.Next() {
+		if i > len(tbl.Records)-1 {
+			return fmt.Errorf("PGDNSSec gatherKeys got more results than expected %d", len(tbl.Records))
+		}
+		if err := rows.Scan(&tbl.Records[i].CDN, &tbl.Records[i].DataEncrypted); err != nil {
+			return fmt.Errorf("PGDNSSec gatherKeys unable to scan row: %w", err)
+		}
+		i += 1
+	}
+	return nil
+}
+func (tbl *pgDNSSecTable) decrypt(aesKey []byte) error {
+	for i, _ := range tbl.Records {
+		if err := decryptInto(aesKey, tbl.Records[i].DataEncrypted, &tbl.Records[i].Key); err != nil {
+			return fmt.Errorf("unable to decrypt into keys: %w", err)
+		}
+	}
+	return nil
+}
+func (tbl *pgDNSSecTable) encrypt(aesKey []byte) error {
+	for i, dns := range tbl.Records {
+		data, err := json.Marshal(&dns.Key)
+		if err != nil {
+			return fmt.Errorf("encrypt issue marshalling keys: %w", err)
+		}
+		dat, err := encrypt(data, aesKey)
+		if err != nil {
+			return fmt.Errorf("encrypt error: %w", err)
+		}
+		tbl.Records[i].DataEncrypted = dat
+	}
+	return nil
+}
+func (tbl *pgDNSSecTable) toGeneric() []DNSSecKey {
+	keys := make([]DNSSecKey, len(tbl.Records))
+
+	for i, record := range tbl.Records {
+		keys[i] = DNSSecKey{
+			CDN:                    record.CDN,
+			DNSSECKeysTrafficVault: record.Key,
+		}
+	}
+
+	return keys
+}
+func (tbl *pgDNSSecTable) fromGeneric(keys []DNSSecKey) {
+	tbl.Records = make([]pgDNSSecRecord, len(keys))
+
+	for i, key := range keys {
+		tbl.Records[i] = pgDNSSecRecord{
+			Key: key.DNSSECKeysTrafficVault,
+			CDN: key.CDN,
+			pgCommonRecord: pgCommonRecord{
+				DataEncrypted: nil,
+			},
+		}
+	}
+}
+func (tbl *pgDNSSecTable) validate() []string {
+	for _, record := range tbl.Records {
+		if record.DataEncrypted == nil && len(record.Key) > 0 {
+			return []string{fmt.Sprintf("DNSSEC Key CDN '%s': DataEncrypted is blank!", record.CDN)}
+		}
+	}
+	return nil
+}
+func (tbl *pgDNSSecTable) insertKeys(db *sql.DB) error {
+	queryBase := "INSERT INTO dnssec (cdn, data) VALUES %s ON CONFLICT (cdn) DO UPDATE SET data = EXCLUDED.data"
+	stride := 2
+	queryArgs := make([]interface{}, len(tbl.Records)*stride)
+	for i, record := range tbl.Records {
+		j := i * stride
+		queryArgs[j] = record.CDN
+		queryArgs[j+1] = record.DataEncrypted
+	}
+	return insertIntoTable(db, queryBase, stride, queryArgs)
+}
+
+type pgSSLKeyRecord struct {
+	Keys tc.DeliveryServiceSSLKeys
+	pgCommonRecord
+
+	// These records are stored on the table but are duplicated
+	DeliveryService string
+	CDN             string
+	Version         string
+}
+type pgSSLKeyTable struct {
+	Records []pgSSLKeyRecord
+}
+
+func (tbl *pgSSLKeyTable) insertKeys(db *sql.DB) error {
+	queryBase := "INSERT INTO sslkey (deliveryservice, data, cdn, version) VALUES %s ON CONFLICT (deliveryservice,cdn,version) DO UPDATE SET data = EXCLUDED.data"
+	stride := 4
+	queryArgs := make([]interface{}, len(tbl.Records)*stride)
+	for i, record := range tbl.Records {
+		j := i * stride
+
+		queryArgs[j] = record.DeliveryService
+		queryArgs[j+1] = record.DataEncrypted
+		queryArgs[j+2] = record.CDN
+		queryArgs[j+3] = record.Version
+	}
+	return insertIntoTable(db, queryBase, 4, queryArgs)
+}
+func (tbl *pgSSLKeyTable) gatherKeys(db *sql.DB) error {
+	sz, err := getSize(db, "sslkey")
+	if err != nil {
+		return fmt.Errorf("PGSSLKey gatherKeys unable to determine size of sslkey table: %w", err)
+	}
+	tbl.Records = make([]pgSSLKeyRecord, sz)
+
+	query := "SELECT data, deliveryservice, cdn, version from sslkey"
+	rows, err := db.Query(query)
+	if err != nil {
+		return fmt.Errorf("PGSSLKey gatherKeys unable to run query '%s': %w", query, err)
+	}
+	defer log.Close(rows, "closing sslkey query")
+	i := 0
+	for rows.Next() {
+		if i > len(tbl.Records)-1 {
+			return fmt.Errorf("PGSSLKey gatherKeys: got more results than expected")
+		}
+		if err := rows.Scan(&tbl.Records[i].DataEncrypted, &tbl.Records[i].DeliveryService, &tbl.Records[i].CDN, &tbl.Records[i].Version); err != nil {
+			return fmt.Errorf("PGSSLKey gatherKeys unable to scan %d row: %w", i, err)
+		}
+		i += 1
+	}
+	return nil
+}
+func (tbl *pgSSLKeyTable) decrypt(aesKey []byte) error {
+	for i, dns := range tbl.Records {

Review comment:
       nit: `dns`? Must be a copy-paste remnant

##########
File path: traffic_ops/app/db/traffic_vault_migrate/riak.go
##########
@@ -0,0 +1,749 @@
+package main
+
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+import (
+	"crypto/tls"
+	"encoding/json"
+	"errors"
+	"fmt"
+	"strings"
+	"time"
+
+	"github.com/basho/riak-go-client"
+
+	"github.com/apache/trafficcontrol/lib/go-rfc"
+	"github.com/apache/trafficcontrol/lib/go-tc"
+)
+
+const (
+	BUCKET_SSL     = "ssl"
+	BUCKET_DNSSEC  = "dnssec"
+	BUCKET_URL_SIG = "url_sig_keys"
+	BUCKET_URI_SIG = "cdn_uri_sig_keys"
+
+	INDEX_SSL = "sslkeys"
+
+	SCHEMA_RIAK_KEY    = "_yz_rk"
+	SCHEMA_RIAK_BUCKET = "_yz_rb"
+)
+
+var (
+	SCHEMA_SSL_FIELDS = [...]string{SCHEMA_RIAK_KEY, SCHEMA_RIAK_BUCKET}
+)
+
+// RiakConfig  represents the configuration options available to the Riak backend.
+type RiakConfig struct {
+	Host          string `json:"host"`
+	Port          string `json:"port"`
+	User          string `json:"user"`
+	Password      string `json:"password"`
+	Insecure      bool   `json:"insecure"`
+	TLSVersionRaw string `json:"tlsVersion"`
+	// Timeout is the number of seconds each command should use.
+	Timeout int `json:"timeout"`
+
+	TLSVersion uint16 `json:"-"`
+}
+
+// RiakBackend is the Riak implementation of TVBackend.
+type RiakBackend struct {
+	sslKeys        riakSSLKeyTable
+	dnssecKeys     riakDNSSecKeyTable
+	uriSigningKeys riakURISignKeyTable
+	urlSigKeys     riakURLSigKeyTable
+	cfg            RiakConfig
+	cluster        *riak.Cluster
+}
+
+// String returns a high level overview of the backend and its keys.
+func (rb *RiakBackend) String() string {
+	data := fmt.Sprintf("Riak server %s@%s:%s\n", rb.cfg.User, rb.cfg.Host, rb.cfg.Port)
+	data += fmt.Sprintf("\tSSL Keys: %d\n", len(rb.sslKeys.Records))
+	data += fmt.Sprintf("\tDNSSec Keys: %d\n", len(rb.dnssecKeys.Records))
+	data += fmt.Sprintf("\tURI Signing Keys: %d\n", len(rb.uriSigningKeys.Records))
+	data += fmt.Sprintf("\tURL Sig Keys: %d\n", len(rb.urlSigKeys.Records))
+	return data
+}
+
+// Name returns the name for this backend.
+func (rb *RiakBackend) Name() string {
+	return "Riak"
+}
+
+// ReadConfigFile takes in a filename and will read it into the backends config.
+func (rb *RiakBackend) ReadConfigFile(configFile string) error {
+	err := UnmarshalConfig(configFile, &rb.cfg)
+	if err != nil {
+		return err
+	}
+
+	switch rb.cfg.TLSVersionRaw {
+	case "10":
+		rb.cfg.TLSVersion = tls.VersionTLS10
+	case "11":
+		rb.cfg.TLSVersion = tls.VersionTLS11
+	case "12":
+		rb.cfg.TLSVersion = tls.VersionTLS12
+	case "13":
+		rb.cfg.TLSVersion = tls.VersionTLS13
+	default:
+		return fmt.Errorf("unknown tls version " + rb.cfg.TLSVersionRaw)
+	}
+	return nil
+}
+
+// Insert takes the current keys and inserts them into the backend DB.
+func (rb *RiakBackend) Insert() error {
+	if err := rb.sslKeys.insertKeys(rb.cluster, rb.cfg.Timeout); err != nil {
+		return err
+	}
+	if err := rb.dnssecKeys.insertKeys(rb.cluster, rb.cfg.Timeout); err != nil {
+		return err
+	}
+	if err := rb.urlSigKeys.insertKeys(rb.cluster, rb.cfg.Timeout); err != nil {
+		return err
+	}
+	if err := rb.uriSigningKeys.insertKeys(rb.cluster, rb.cfg.Timeout); err != nil {
+		return err
+	}
+	return nil
+}
+
+// ValidateKey validates that the keys are valid (in most cases, certain fields are not null).
+func (rb *RiakBackend) ValidateKey() []string {
+	allErrs := []string{}
+	if errs := rb.sslKeys.validate(); errs != nil {
+		allErrs = append(allErrs, errs...)
+	}
+	if errs := rb.dnssecKeys.validate(); errs != nil {
+		allErrs = append(allErrs, errs...)
+	}
+	if errs := rb.uriSigningKeys.validate(); errs != nil {
+		allErrs = append(allErrs, errs...)
+	}
+	if errs := rb.urlSigKeys.validate(); errs != nil {
+		allErrs = append(allErrs, errs...)
+	}
+
+	return allErrs
+}
+
+// SetSSLKeys takes in keys and converts & encrypts the data into the backends internal format.
+func (rb *RiakBackend) SetSSLKeys(keys []SSLKey) error {
+	rb.sslKeys.fromGeneric(keys)
+	return nil
+}
+
+// SetDNSSecKeys takes in keys and converts & encrypts the data into the backends internal format.
+func (rb *RiakBackend) SetDNSSecKeys(keys []DNSSecKey) error {
+	rb.dnssecKeys.fromGeneric(keys)
+	return nil
+}
+
+// SetURISignKeys takes in keys and converts & encrypts the data into the backends internal format.
+func (rb *RiakBackend) SetURISignKeys(keys []URISignKey) error {
+	rb.uriSigningKeys.fromGeneric(keys)
+	return nil
+}
+
+// SetURLSigKeys takes in keys and converts & encrypts the data into the backends internal format.
+func (rb *RiakBackend) SetURLSigKeys(keys []URLSigKey) error {
+	rb.urlSigKeys.fromGeneric(keys)
+	return nil
+}
+
+// Start initiates the connection to the backend DB.
+func (rb *RiakBackend) Start() error {
+	tlsConfig := &tls.Config{
+		InsecureSkipVerify: rb.cfg.Insecure,
+		MaxVersion:         rb.cfg.TLSVersion,
+	}
+	auth := &riak.AuthOptions{
+		User:      rb.cfg.User,
+		Password:  rb.cfg.Password,
+		TlsConfig: tlsConfig,
+	}
+
+	cluster, err := getRiakCluster(rb.cfg, auth)
+	if err != nil {
+		return err
+	}
+	if err := cluster.Start(); err != nil {
+		return fmt.Errorf("unable to start riak cluster: %w", err)
+	}
+
+	rb.cluster = cluster
+	rb.sslKeys = riakSSLKeyTable{}
+	rb.dnssecKeys = riakDNSSecKeyTable{}
+	rb.urlSigKeys = riakURLSigKeyTable{}
+	rb.uriSigningKeys = riakURISignKeyTable{}
+	return nil
+}
+
+// Close terminates the connection to the backend DB.
+func (rb *RiakBackend) Close() error {
+	if err := rb.cluster.Stop(); err != nil {
+		return err
+	}
+	return nil
+}
+
+// Ping checks the connection to the backend DB.
+func (rb *RiakBackend) Ping() error {
+	return ping(rb.cluster)
+}
+
+// GetSSLKeys converts the backends internal key representation into the common representation (SSLKey).
+func (rb *RiakBackend) GetSSLKeys() ([]SSLKey, error) {
+	return rb.sslKeys.toGeneric(), nil
+}
+
+// GetDNSSecKeys converts the backends internal key representation into the common representation (DNSSecKey).
+func (rb *RiakBackend) GetDNSSecKeys() ([]DNSSecKey, error) {
+	return rb.dnssecKeys.toGeneric(), nil
+}
+
+// GetURISignKeys converts the pg internal key representation into the common representation (URISignKey).
+func (rb *RiakBackend) GetURISignKeys() ([]URISignKey, error) {
+	return rb.uriSigningKeys.toGeneric(), nil
+}
+
+// GetURLSigKeys converts the backends internal key representation into the common representation (URLSigKey).
+func (rb *RiakBackend) GetURLSigKeys() ([]URLSigKey, error) {
+	return rb.urlSigKeys.toGeneric(), nil
+}
+
+// Fetch gets all of the keys from the backend DB.
+func (rb *RiakBackend) Fetch() error {
+	if err := rb.sslKeys.gatherKeys(rb.cluster, rb.cfg.Timeout); err != nil {
+		return err
+	}
+	if err := rb.dnssecKeys.gatherKeys(rb.cluster, rb.cfg.Timeout); err != nil {
+		return err
+	}
+	if err := rb.urlSigKeys.gatherKeys(rb.cluster, rb.cfg.Timeout); err != nil {
+		return err
+	}
+	if err := rb.uriSigningKeys.gatherKeys(rb.cluster, rb.cfg.Timeout); err != nil {
+		return err
+	}
+
+	return nil
+}
+
+type riakSSLKeyRecord struct {
+	tc.DeliveryServiceSSLKeys
+	Version string
+}
+type riakSSLKeyTable struct {
+	Records []riakSSLKeyRecord
+}
+
+func (tbl *riakSSLKeyTable) gatherKeys(cluster *riak.Cluster, timeout int) error {
+	searchDocs, err := search(cluster, INDEX_SSL, "cdn:*", "", 1000, SCHEMA_SSL_FIELDS[:])
+	if err != nil {
+		return fmt.Errorf("RiakSSLKey gatherKeys: %w", err)
+	}
+
+	tbl.Records = make([]riakSSLKeyRecord, len(searchDocs))
+	for i, doc := range searchDocs {
+		objs, err := getObject(cluster, doc.Bucket, doc.Key, timeout)
+		if err != nil {
+			return err
+		}
+		if len(objs) < 1 {
+			return fmt.Errorf("RiakSSLKey gatherKeys unable to find any objects with key %s and bucket %s, but search results were returned", doc.Key, doc.Bucket)
+		}
+		if len(objs) > 1 {
+			return fmt.Errorf("RiakSSLKey gatherKeys key '%s' more than 1 ssl key record found %d\n", doc.Key, len(objs))
+		}
+		var obj tc.DeliveryServiceSSLKeys
+		if err = json.Unmarshal(objs[0].Value, &obj); err != nil {
+			return fmt.Errorf("RiakSSLKey gatherKeys key '%s' unable to unmarshal object into tc.DeliveryServiceSSLKeys: %w", doc.Key, err)
+		}
+		tbl.Records[i] = riakSSLKeyRecord{
+			DeliveryServiceSSLKeys: obj,
+			Version:                strings.Split(objs[0].Key, "-")[1],
+		}
+	}
+	return nil
+}
+func (tbl *riakSSLKeyTable) toGeneric() []SSLKey {
+	keys := make([]SSLKey, len(tbl.Records))
+
+	for i, record := range tbl.Records {
+		keys[i] = SSLKey{
+			DeliveryServiceSSLKeys: record.DeliveryServiceSSLKeys,
+			Version:                record.Version,
+		}
+	}
+
+	return keys
+}
+func (tbl *riakSSLKeyTable) fromGeneric(keys []SSLKey) {
+	tbl.Records = make([]riakSSLKeyRecord, len(keys))
+
+	for i, record := range keys {
+		tbl.Records[i] = riakSSLKeyRecord{
+			DeliveryServiceSSLKeys: record.DeliveryServiceSSLKeys,
+			Version:                record.Version,
+		}
+	}
+}
+func (tbl *riakSSLKeyTable) insertKeys(cluster *riak.Cluster, timeout int) error {
+	for _, record := range tbl.Records {
+		objBytes, err := json.Marshal(record.DeliveryServiceSSLKeys)
+		if err != nil {
+			return fmt.Errorf("RiakSSLKey insertKeys '%s' failed to marshal keys: %w", record.Key, err)
+		}
+		if err = setObject(cluster, makeRiakObject(objBytes, record.DeliveryService+"-"+record.Version), BUCKET_SSL, timeout); err != nil {
+			return fmt.Errorf("RiakSSLKey insertKeys '%s': %w", record.Key, err)
+		}
+	}
+	return nil
+}
+func (tbl *riakSSLKeyTable) validate() []string {
+	errs := []string{}
+	for _, record := range tbl.Records {
+		if record.DeliveryService == "" {
+			errs = append(errs, fmt.Sprintf("SSL Key '%s': Delivery Service is blank!", record.Key))
+		}
+		if record.CDN == "" {
+			errs = append(errs, fmt.Sprintf("SSL Key '%s': CDN is blank!", record.Key))
+		}
+		if record.Version == "" {
+			errs = append(errs, fmt.Sprintf("SSL Key '%s': Version is blank!", record.Key))
+		}
+	}
+	return errs
+}
+
+type riakDNSSecKeyRecord struct {
+	CDN string
+	Key tc.DNSSECKeysRiak

Review comment:
       nit: `tc.DNSSECKeysRiak` is deprecated, `tc.DNSSECKeysTrafficVault` is the same thing and could be used instead

##########
File path: traffic_ops/app/db/traffic_vault_migrate/postgres.go
##########
@@ -0,0 +1,744 @@
+package main
+
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+import (
+	"database/sql"
+	"encoding/base64"
+	"encoding/json"
+	"errors"
+	"fmt"
+	"strconv"
+	"strings"
+
+	"github.com/apache/trafficcontrol/lib/go-log"
+	"github.com/apache/trafficcontrol/lib/go-tc"
+	util "github.com/apache/trafficcontrol/lib/go-util"
+
+	_ "github.com/lib/pq"
+)
+
+// PGConfig represents the configuration options available to the PG backend.
+type PGConfig struct {
+	Host     string `json:"host"`
+	Port     string `json:"port"`
+	User     string `json:"user"`
+	Password string `json:"password"`
+	SSLMode  string `json:"sslmode"`
+	Database string `json:"database"`
+	Key      string `json:"aesKey"`
+	AESKey   []byte
+}
+
+// PGBackend is the Postgres implementation of TVBackend.
+type PGBackend struct {
+	sslKey         pgSSLKeyTable
+	dnssec         pgDNSSecTable
+	uriSigningKeys pgURISignKeyTable
+	urlSigKeys     pgURLSigKeyTable
+	cfg            PGConfig
+	db             *sql.DB
+}
+
+// String returns a high level overview of the backend and its keys.
+func (pg *PGBackend) String() string {
+	data := fmt.Sprintf("PG server %s@%s:%s\n", pg.cfg.User, pg.cfg.Host, pg.cfg.Port)
+	data += fmt.Sprintf("\tSSL Keys: %d\n", len(pg.sslKey.Records))
+	data += fmt.Sprintf("\tDNSSec Keys: %d\n", len(pg.dnssec.Records))
+	data += fmt.Sprintf("\tURI Signing Keys: %d\n", len(pg.uriSigningKeys.Records))
+	data += fmt.Sprintf("\tURL Sig Keys: %d\n", len(pg.urlSigKeys.Records))
+	return data
+}
+
+// Name returns the name for this backend.
+func (pg *PGBackend) Name() string {
+	return "PG"
+}
+
+// ReadConfigFile takes in a filename and will read it into the backends config.
+func (pg *PGBackend) ReadConfigFile(configFile string) error {
+	var err error
+	if err = UnmarshalConfig(configFile, &pg.cfg); err != nil {
+		return err
+	}
+
+	if pg.cfg.AESKey, err = base64.StdEncoding.DecodeString(pg.cfg.Key); err != nil {
+		return fmt.Errorf("unable to decode PG AESKey '%s': %w", pg.cfg.Key, err)
+	}
+
+	if err = util.ValidateAESKey(pg.cfg.AESKey); err != nil {
+		return fmt.Errorf("unable to validate PG AESKey '%s'", pg.cfg.Key)
+	}
+	return nil
+}
+
+// Insert takes the current keys and inserts them into the backend DB.
+func (pg *PGBackend) Insert() error {
+	if err := pg.sslKey.insertKeys(pg.db); err != nil {
+		return err
+	}
+	if err := pg.dnssec.insertKeys(pg.db); err != nil {
+		return err
+	}
+	if err := pg.urlSigKeys.insertKeys(pg.db); err != nil {
+		return err
+	}
+	if err := pg.uriSigningKeys.insertKeys(pg.db); err != nil {
+		return err
+	}
+	return nil
+}
+
+// Start initiates the connection to the backend DB.
+func (pg *PGBackend) Start() error {
+	sqlStr := fmt.Sprintf("postgres://%s:%s@%s:%s/%s?sslmode=%s", pg.cfg.User, pg.cfg.Password, pg.cfg.Host, pg.cfg.Port, pg.cfg.Database, pg.cfg.SSLMode)
+	db, err := sql.Open("postgres", sqlStr)
+	if err != nil {
+		sqlStr = strings.Replace(sqlStr, pg.cfg.Password, "*", 1)
+		return fmt.Errorf("unable to start PG client with connection string '%s': %w", sqlStr, err)
+	}
+
+	pg.db = db
+	pg.sslKey = pgSSLKeyTable{}
+	pg.dnssec = pgDNSSecTable{}
+	pg.urlSigKeys = pgURLSigKeyTable{}
+	pg.uriSigningKeys = pgURISignKeyTable{}
+
+	return nil
+}
+
+// ValidateKey validates that the keys are valid (in most cases, certain fields are not null).
+func (pg *PGBackend) ValidateKey() []string {
+	var allErrs []string
+	if errs := pg.sslKey.validate(); errs != nil {
+		allErrs = append(allErrs, errs...)
+	}
+	if errs := pg.dnssec.validate(); errs != nil {
+		allErrs = append(allErrs, errs...)
+	}
+	if errs := pg.uriSigningKeys.validate(); errs != nil {
+		allErrs = append(allErrs, errs...)
+	}
+	if errs := pg.urlSigKeys.validate(); errs != nil {
+		allErrs = append(allErrs, errs...)
+	}
+	return allErrs
+}
+
+// Close terminates the connection to the backend DB.
+func (pg *PGBackend) Close() error {
+	return pg.db.Close()
+}
+
+// Ping checks the connection to the backend DB.
+func (pg *PGBackend) Ping() error {
+	return pg.db.Ping()
+}
+
+// Fetch gets all of the keys from the backend DB.
+func (pg *PGBackend) Fetch() error {
+	if err := pg.sslKey.gatherKeys(pg.db); err != nil {
+		return err
+	}
+
+	if err := pg.dnssec.gatherKeys(pg.db); err != nil {
+		return err
+	}
+
+	if err := pg.urlSigKeys.gatherKeys(pg.db); err != nil {
+		return err
+	}
+
+	if err := pg.uriSigningKeys.gatherKeys(pg.db); err != nil {
+		return err
+	}
+
+	return nil
+}
+
+// GetSSLKeys converts the backends internal key representation into the common representation (SSLKey).
+func (pg *PGBackend) GetSSLKeys() ([]SSLKey, error) {
+	if err := pg.sslKey.decrypt(pg.cfg.AESKey); err != nil {
+		return nil, err
+	}
+	return pg.sslKey.toGeneric(), nil
+}
+
+// SetSSLKeys takes in keys and converts & encrypts the data into the backends internal format.
+func (pg *PGBackend) SetSSLKeys(keys []SSLKey) error {
+	pg.sslKey.fromGeneric(keys)
+	return pg.sslKey.encrypt(pg.cfg.AESKey)
+}
+
+// GetDNSSecKeys converts the backends internal key representation into the common representation (DNSSecKey).
+func (pg *PGBackend) GetDNSSecKeys() ([]DNSSecKey, error) {
+	if err := pg.dnssec.decrypt(pg.cfg.AESKey); err != nil {
+		return nil, err
+	}
+	return pg.dnssec.toGeneric(), nil
+}
+
+// SetDNSSecKeys takes in keys and converts & encrypts the data into the backends internal format.
+func (pg *PGBackend) SetDNSSecKeys(keys []DNSSecKey) error {
+	pg.dnssec.fromGeneric(keys)
+	return pg.dnssec.encrypt(pg.cfg.AESKey)
+}
+
+// GetURISignKeys converts the pg internal key representation into the common representation (URISignKey).
+func (pg *PGBackend) GetURISignKeys() ([]URISignKey, error) {
+	if err := pg.uriSigningKeys.decrypt(pg.cfg.AESKey); err != nil {
+		return nil, err
+	}
+	return pg.uriSigningKeys.toGeneric(), nil
+}
+
+// SetURISignKeys takes in keys and converts & encrypts the data into the backends internal format.
+func (pg *PGBackend) SetURISignKeys(keys []URISignKey) error {
+	pg.uriSigningKeys.fromGeneric(keys)
+	return pg.uriSigningKeys.encrypt(pg.cfg.AESKey)
+}
+
+// GetURLSigKeys converts the backends internal key representation into the common representation (URLSigKey).
+func (pg *PGBackend) GetURLSigKeys() ([]URLSigKey, error) {
+	if err := pg.urlSigKeys.decrypt(pg.cfg.AESKey); err != nil {
+		return nil, err
+	}
+	return pg.urlSigKeys.toGeneric(), nil
+}
+
+// SetURLSigKeys takes in keys and converts & encrypts the data into the backends internal format.
+func (pg *PGBackend) SetURLSigKeys(keys []URLSigKey) error {
+	pg.urlSigKeys.fromGeneric(keys)
+	return pg.urlSigKeys.encrypt(pg.cfg.AESKey)
+}
+
+type pgCommonRecord struct {
+	DataEncrypted []byte
+}
+
+type pgDNSSecRecord struct {
+	Key tc.DNSSECKeysTrafficVault
+	CDN string
+	pgCommonRecord
+}
+type pgDNSSecTable struct {
+	Records []pgDNSSecRecord
+}
+
+func (tbl *pgDNSSecTable) gatherKeys(db *sql.DB) error {
+	sz, err := getSize(db, "dnssec")
+	if err != nil {
+		log.Errorln("PGDNSSec gatherKeys: unable to determine size of dnssec table")
+	}
+	tbl.Records = make([]pgDNSSecRecord, sz)
+
+	query := "SELECT cdn, data from dnssec"
+	rows, err := db.Query(query)
+	if err != nil {
+		return fmt.Errorf("PGDNSSec gatherKeys: unable to run query '%s': %w", query, err)
+	}
+	defer log.Close(rows, "closing dnssec query")
+	i := 0
+	for rows.Next() {
+		if i > len(tbl.Records)-1 {
+			return fmt.Errorf("PGDNSSec gatherKeys got more results than expected %d", len(tbl.Records))
+		}
+		if err := rows.Scan(&tbl.Records[i].CDN, &tbl.Records[i].DataEncrypted); err != nil {
+			return fmt.Errorf("PGDNSSec gatherKeys unable to scan row: %w", err)
+		}
+		i += 1
+	}
+	return nil
+}
+func (tbl *pgDNSSecTable) decrypt(aesKey []byte) error {
+	for i, _ := range tbl.Records {
+		if err := decryptInto(aesKey, tbl.Records[i].DataEncrypted, &tbl.Records[i].Key); err != nil {
+			return fmt.Errorf("unable to decrypt into keys: %w", err)
+		}
+	}
+	return nil
+}
+func (tbl *pgDNSSecTable) encrypt(aesKey []byte) error {
+	for i, dns := range tbl.Records {
+		data, err := json.Marshal(&dns.Key)
+		if err != nil {
+			return fmt.Errorf("encrypt issue marshalling keys: %w", err)
+		}
+		dat, err := encrypt(data, aesKey)
+		if err != nil {
+			return fmt.Errorf("encrypt error: %w", err)
+		}
+		tbl.Records[i].DataEncrypted = dat
+	}
+	return nil
+}
+func (tbl *pgDNSSecTable) toGeneric() []DNSSecKey {
+	keys := make([]DNSSecKey, len(tbl.Records))
+
+	for i, record := range tbl.Records {
+		keys[i] = DNSSecKey{
+			CDN:                    record.CDN,
+			DNSSECKeysTrafficVault: record.Key,
+		}
+	}
+
+	return keys
+}
+func (tbl *pgDNSSecTable) fromGeneric(keys []DNSSecKey) {
+	tbl.Records = make([]pgDNSSecRecord, len(keys))
+
+	for i, key := range keys {
+		tbl.Records[i] = pgDNSSecRecord{
+			Key: key.DNSSECKeysTrafficVault,
+			CDN: key.CDN,
+			pgCommonRecord: pgCommonRecord{
+				DataEncrypted: nil,
+			},
+		}
+	}
+}
+func (tbl *pgDNSSecTable) validate() []string {
+	for _, record := range tbl.Records {
+		if record.DataEncrypted == nil && len(record.Key) > 0 {
+			return []string{fmt.Sprintf("DNSSEC Key CDN '%s': DataEncrypted is blank!", record.CDN)}
+		}
+	}
+	return nil
+}
+func (tbl *pgDNSSecTable) insertKeys(db *sql.DB) error {
+	queryBase := "INSERT INTO dnssec (cdn, data) VALUES %s ON CONFLICT (cdn) DO UPDATE SET data = EXCLUDED.data"
+	stride := 2
+	queryArgs := make([]interface{}, len(tbl.Records)*stride)
+	for i, record := range tbl.Records {
+		j := i * stride
+		queryArgs[j] = record.CDN
+		queryArgs[j+1] = record.DataEncrypted
+	}
+	return insertIntoTable(db, queryBase, stride, queryArgs)
+}
+
+type pgSSLKeyRecord struct {
+	Keys tc.DeliveryServiceSSLKeys
+	pgCommonRecord
+
+	// These records are stored on the table but are duplicated
+	DeliveryService string
+	CDN             string
+	Version         string
+}
+type pgSSLKeyTable struct {
+	Records []pgSSLKeyRecord
+}
+
+func (tbl *pgSSLKeyTable) insertKeys(db *sql.DB) error {
+	queryBase := "INSERT INTO sslkey (deliveryservice, data, cdn, version) VALUES %s ON CONFLICT (deliveryservice,cdn,version) DO UPDATE SET data = EXCLUDED.data"
+	stride := 4
+	queryArgs := make([]interface{}, len(tbl.Records)*stride)
+	for i, record := range tbl.Records {
+		j := i * stride
+
+		queryArgs[j] = record.DeliveryService
+		queryArgs[j+1] = record.DataEncrypted
+		queryArgs[j+2] = record.CDN
+		queryArgs[j+3] = record.Version
+	}
+	return insertIntoTable(db, queryBase, 4, queryArgs)
+}
+func (tbl *pgSSLKeyTable) gatherKeys(db *sql.DB) error {
+	sz, err := getSize(db, "sslkey")
+	if err != nil {
+		return fmt.Errorf("PGSSLKey gatherKeys unable to determine size of sslkey table: %w", err)
+	}
+	tbl.Records = make([]pgSSLKeyRecord, sz)
+
+	query := "SELECT data, deliveryservice, cdn, version from sslkey"
+	rows, err := db.Query(query)
+	if err != nil {
+		return fmt.Errorf("PGSSLKey gatherKeys unable to run query '%s': %w", query, err)
+	}
+	defer log.Close(rows, "closing sslkey query")
+	i := 0
+	for rows.Next() {
+		if i > len(tbl.Records)-1 {
+			return fmt.Errorf("PGSSLKey gatherKeys: got more results than expected")
+		}
+		if err := rows.Scan(&tbl.Records[i].DataEncrypted, &tbl.Records[i].DeliveryService, &tbl.Records[i].CDN, &tbl.Records[i].Version); err != nil {
+			return fmt.Errorf("PGSSLKey gatherKeys unable to scan %d row: %w", i, err)
+		}
+		i += 1
+	}
+	return nil
+}
+func (tbl *pgSSLKeyTable) decrypt(aesKey []byte) error {
+	for i, dns := range tbl.Records {
+		if err := decryptInto(aesKey, dns.DataEncrypted, &tbl.Records[i].Keys); err != nil {
+			return fmt.Errorf("unable to decrypt into keys: %w", err)
+		}
+	}
+	return nil
+}
+func (tbl *pgSSLKeyTable) encrypt(aesKey []byte) error {
+	for i, dns := range tbl.Records {

Review comment:
       nit: `dns`? Must be a copy-paste remnant

##########
File path: traffic_ops/app/db/traffic_vault_migrate/traffic_vault_migrate.go
##########
@@ -0,0 +1,611 @@
+package main
+
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+import (
+	"encoding/json"
+	"errors"
+	"fmt"
+	"io/ioutil"
+	stdlog "log"
+	"os"
+	"reflect"
+	"sort"
+	"strings"
+
+	"github.com/pborman/getopt/v2"
+
+	"github.com/apache/trafficcontrol/lib/go-log"
+	"github.com/apache/trafficcontrol/lib/go-tc"
+)
+
+var (
+	fromType    string
+	toType      string
+	fromCfgPath string
+	toCfgPath   string
+	logCfgPath  string
+	keyFile     string
+	dry         bool
+	compare     bool
+	noConfirm   bool
+	dump        bool
+	logLevel    string
+
+	cfg config = config{
+		LogLocationError:   log.LogLocationStderr,
+		LogLocationWarning: log.LogLocationStdout,
+		LogLocationInfo:    log.LogLocationStdout,
+		LogLocationDebug:   log.LogLocationNull,
+		LogLocationEvent:   log.LogLocationNull,
+	}
+	riakBE RiakBackend = RiakBackend{}
+	pgBE   PGBackend   = PGBackend{}
+)
+
+func init() {
+	fromTypePtr := getopt.StringLong("fromType", 't', riakBE.Name(), fmt.Sprintf("From server types (%v)", strings.Join(supportedTypes(), "|")))
+	if fromTypePtr == nil {
+		stdlog.Fatal("unable to load fromType")
+	}
+	fromType = *fromTypePtr
+
+	toTypePtr := getopt.StringLong("toType", 'o', pgBE.Name(), fmt.Sprintf("From server types (%v)", strings.Join(supportedTypes(), "|")))
+	if toTypePtr == nil {
+		stdlog.Fatal("unable to load toType")
+	}
+	toType = *toTypePtr
+
+	toCfgPtr := getopt.StringLong("toCfgPath", 'g', "pg.json", "To server config file")
+	if toCfgPtr == nil {
+		stdlog.Fatal("unable to load toCfg")
+	}
+	toCfgPath = *toCfgPtr
+
+	fromCfgPtr := getopt.StringLong("fromCfgPath", 'f', "riak.json", "From server config file")
+	if fromCfgPtr == nil {
+		stdlog.Fatal("unable to load fromCfg")
+	}
+	fromCfgPath = *fromCfgPtr
+
+	getopt.FlagLong(&dry, "dry", 'r', "Do not perform writes").
+		SetOptional().
+		SetFlag().
+		SetGroup("no_insert")
+
+	getopt.FlagLong(&compare, "compare", 'c', "Compare to and from server records").
+		SetOptional().
+		SetFlag().
+		SetGroup("no_insert")
+
+	getopt.FlagLong(&noConfirm, "noConfirm", 'm', "Requires confirmation before inserting records").
+		SetFlag()
+
+	getopt.FlagLong(&dump, "dump", 'd', "Write keys (from 'from' server) to disk").
+		SetOptional().
+		SetGroup("disk_bck").
+		SetFlag()
+
+	getopt.FlagLong(&keyFile, "fill", 'i', "Insert data into `to` server with data this directory").
+		SetOptional().
+		SetGroup("disk_bck")
+
+	getopt.FlagLong(&logCfgPath, "logCfg", 'l', "Log configuration file").
+		SetOptional().
+		SetGroup("log")
+
+	getopt.FlagLong(&logLevel, "logLevel", 'e', "Print everything at above specified log level (error|warning|info|debug|event)").
+		SetOptional().
+		SetGroup("log")
+}
+
+// supportBackends returns the backends available in this tool.
+func supportedBackends() []TVBackend {
+	return []TVBackend{
+		&riakBE, &pgBE,
+	}
+}
+
+func main() {
+	getopt.ParseV2()
+
+	initConfig()
+
+	var fromSrv TVBackend
+	var toSrv TVBackend
+
+	importData := keyFile != ""
+	toSrvUsed := !dump && !dry || keyFile != ""
+
+	if !importData {
+		log.Infof("Initiating fromSrv %s...\n", fromType)
+		if !validateType(fromType) {
+			log.Errorln("Unknown fromType " + fromType)
+			os.Exit(1)
+		}
+		fromSrv = getBackendFromType(fromType)
+		if err := fromSrv.ReadConfigFile(fromCfgPath); err != nil {
+			log.Errorf("Unable to read fromSrv cfg: %v", err)
+			os.Exit(1)
+		}
+
+		if err := fromSrv.Start(); err != nil {
+			log.Errorf("issue starting fromSrv: %v", err)
+			os.Exit(1)
+		}
+		defer log.Close(fromSrv, "closing fromSrv")
+
+		if err := fromSrv.Ping(); err != nil {
+			log.Errorf("Unable to ping fromSrv: %v", err)
+			os.Exit(1)
+		}
+	}
+
+	if toSrvUsed {
+		log.Infof("Initiating toSrv %s...\n", toType)
+		if !validateType(toType) {
+			log.Errorln("Unknown toType " + toType)
+			os.Exit(1)
+		}
+		toSrv = getBackendFromType(toType)
+
+		if err := toSrv.ReadConfigFile(toCfgPath); err != nil {
+			log.Errorf("Unable to read toSrv cfg: %v", err)
+			os.Exit(1)
+		}
+
+		if err := toSrv.Start(); err != nil {
+			log.Errorf("issue starting toSrv: %v", err)
+			os.Exit(1)
+		}
+		defer log.Close(toSrv, "closing toSrv")
+
+		if err := toSrv.Ping(); err != nil {
+			log.Errorf("Unable to ping toSrv: %v", err)
+			os.Exit(1)
+		}
+	}
+
+	var fromSecret Secrets
+	if !importData {
+		var err error
+		log.Infof("Fetching data from %s...\n", fromSrv.Name())
+		if err = fromSrv.Fetch(); err != nil {
+			log.Errorf("Unable to fetch fromSrv data: %v", err)
+			os.Exit(1)
+		}
+
+		if fromSecret, err = GetKeys(fromSrv); err != nil {
+			log.Errorln(err)
+			os.Exit(1)
+		}
+
+		if err := Validate(fromSrv); err != nil {
+			log.Errorln(err)
+			os.Exit(1)
+		}
+
+	} else {
+		err := fromSecret.fill(keyFile)
+		if err != nil {
+			log.Errorln(err)
+			os.Exit(1)
+		}
+	}
+
+	if dump {
+		log.Infof("Dumping data from %s...\n", fromSrv.Name())
+		fromSecret.dump("dump")
+		return
+	}
+
+	if compare {
+		log.Infof("Fetching data from %s...\n", toSrv.Name())
+		if err := toSrv.Fetch(); err != nil {
+			log.Errorf("Unable to fetch toSrv data: %v\n", err)
+			os.Exit(1)
+		}
+
+		toSecret, err := GetKeys(toSrv)
+		if err != nil {
+			log.Errorln(err)
+			os.Exit(1)
+		}
+		log.Infoln("Validating " + toSrv.Name())
+		if err := toSrv.ValidateKey(); err != nil && len(err) > 0 {
+			log.Errorln(strings.Join(err, "\n"))
+			os.Exit(1)
+		}
+
+		fromSecret.sort()
+		toSecret.sort()
+
+		if !importData {
+			log.Infoln(fromSrv.String())
+		} else {
+			log.Infof("Disk backup:\n\tSSL Keys: %d\n\tDNSSec Keys: %d\n\tURI Keys: %d\n\tURL Keys: %d\n", len(fromSecret.sslkeys), len(fromSecret.dnssecKeys), len(fromSecret.uriKeys), len(fromSecret.urlKeys))
+		}
+		log.Infoln(toSrv.String())
+
+		if !reflect.DeepEqual(fromSecret.sslkeys, toSecret.sslkeys) {
+			log.Errorln("from sslkeys and to sslkeys don't match")
+			os.Exit(1)
+		}
+		if !reflect.DeepEqual(fromSecret.dnssecKeys, toSecret.dnssecKeys) {
+			log.Errorln("from dnssec and to dnssec don't match")
+			os.Exit(1)
+		}
+		if !reflect.DeepEqual(fromSecret.uriKeys, toSecret.uriKeys) {
+			log.Errorln("from uri and to uri don't match")
+			os.Exit(1)
+		}
+		if !reflect.DeepEqual(fromSecret.urlKeys, toSecret.urlKeys) {
+			log.Errorln("from url and to url don't match")
+			os.Exit(1)
+		}
+		log.Infoln("Both data sources have the same keys")
+		return
+	}
+
+	if toSrvUsed {
+		log.Infof("Setting %s keys...\n", toSrv.Name())
+		if err := SetKeys(toSrv, fromSecret); err != nil {
+			log.Errorln(err)
+			os.Exit(1)
+		}
+
+		if err := Validate(toSrv); err != nil {
+			log.Errorln(err)
+			os.Exit(1)
+		}
+	}
+
+	if !importData {
+		log.Infoln(fromSrv.String())
+	} else {
+		log.Infof("Disk backup:\n\tSSL Keys: %d\n\tDNSSec Keys: %d\n\tURI Keys: %d\n\tURL Keys: %d\n", len(fromSecret.sslkeys), len(fromSecret.dnssecKeys), len(fromSecret.uriKeys), len(fromSecret.urlKeys))
+	}
+
+	if dry {
+		return
+	}
+
+	if !noConfirm {
+		ans := "q"
+		for {
+			fmt.Print("Confirm data insertion (y/n): ")
+			if _, err := fmt.Scanln(&ans); err != nil {
+				log.Errorln("unable to get user input")
+				os.Exit(1)
+			}
+
+			if ans == "y" {
+				break
+			} else if ans == "n" {
+				return
+			}
+		}
+	}
+	log.Infof("Inserting data into %s...\n", toSrv.Name())
+	if err := toSrv.Insert(); err != nil {
+		log.Errorln(err)
+		os.Exit(1)
+	}
+}
+
+// Validate runs the ValidateKey method on the backend.
+func Validate(be TVBackend) error {
+	if errs := be.ValidateKey(); errs != nil && len(errs) > 0 {
+		return errors.New(fmt.Sprintf("Validation Errors (%s): \n%s", be.Name(), strings.Join(errs, "\n")))
+	}
+	return nil
+}
+
+// SetKeys will set all of the keys for a backend.
+func SetKeys(be TVBackend, s Secrets) error {
+	if err := be.SetSSLKeys(s.sslkeys); err != nil {
+		return fmt.Errorf("Unable to set %s ssl keys: %w", be.Name(), err)
+	}
+	if err := be.SetDNSSecKeys(s.dnssecKeys); err != nil {
+		return fmt.Errorf("Unable to set %s dnssec keys: %w", be.Name(), err)
+	}
+	if err := be.SetURLSigKeys(s.urlKeys); err != nil {
+		return fmt.Errorf("Unable to set %v url keys: %v", be.Name(), err)
+	}
+	if err := be.SetURISignKeys(s.uriKeys); err != nil {
+		return fmt.Errorf("Unable to set %v uri keys: %v", be.Name(), err)
+	}
+	return nil
+}
+
+// GetKeys will get all of the keys for a backend.
+func GetKeys(be TVBackend) (Secrets, error) {
+	var secret Secrets
+	var err error
+	if secret.sslkeys, err = be.GetSSLKeys(); err != nil {
+		return Secrets{}, fmt.Errorf("Unable to get %v sslkeys: %v", be.Name(), err)
+	}
+	if secret.dnssecKeys, err = be.GetDNSSecKeys(); err != nil {
+		return Secrets{}, fmt.Errorf("Unable to get %v dnssec keys: %v", be.Name(), err)
+	}
+	if secret.uriKeys, err = be.GetURISignKeys(); err != nil {
+		return Secrets{}, fmt.Errorf("Unable to get %v uri keys: %v", be.Name(), err)
+	}
+	if secret.urlKeys, err = be.GetURLSigKeys(); err != nil {
+		return Secrets{}, fmt.Errorf("Unable to %v url keys: %v", be.Name(), err)
+	}
+	return secret, nil
+}
+
+// UnmarshalConfig takes in a config file and a type and will read the config file into the reflected type.
+func UnmarshalConfig(configFile string, config interface{}) error {
+	data, err := ioutil.ReadFile(configFile)
+	if err != nil {
+		return err
+	}
+	err = json.Unmarshal(data, config)
+	if err != nil {
+		return err
+	}
+
+	return nil
+}
+
+// TVBackend represents a TV backend that can be have data migrated to/from
+type TVBackend interface {

Review comment:
       I like that this is interface-based 👍 




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [trafficcontrol] shamrickus commented on a change in pull request #5924: Add tool to migrate data between TV backends

Posted by GitBox <gi...@apache.org>.
shamrickus commented on a change in pull request #5924:
URL: https://github.com/apache/trafficcontrol/pull/5924#discussion_r648507014



##########
File path: tools/traffic_vault_migrate/traffic_vault_migrate.go
##########
@@ -0,0 +1,453 @@
+package main
+
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+import (
+	"encoding/json"
+	"errors"
+	"flag"
+	"fmt"
+	"github.com/apache/trafficcontrol/lib/go-tc"
+	"io/ioutil"
+	"log"
+	"os"
+	"reflect"
+	"sort"
+	"strings"
+	"time"
+)
+
+var (
+	fromType string
+	toType   string
+	fromCfg  string
+	toCfg    string
+	dry      bool
+	compare  bool
+	confirm  bool
+	dump     bool
+
+	riakBE RiakBackend = RiakBackend{}
+	pgBE   PGBackend   = PGBackend{}
+)
+
+func init() {
+	flag.StringVar(&fromType, "from_type", riakBE.Name(), fmt.Sprintf("From server types (%v)", strings.Join(supportedTypes(), "|")))
+	flag.StringVar(&toType, "to_type", pgBE.Name(), fmt.Sprintf("From server types (%v)", strings.Join(supportedTypes(), "|")))
+	flag.StringVar(&toCfg, "to_cfg", "pg.json", "To server config file")
+	flag.StringVar(&fromCfg, "from_cfg", "riak.json", "From server config file")
+	flag.BoolVar(&dry, "dry", false, "Do not perform writes")
+	flag.BoolVar(&compare, "compare", false, "Compare to and from server records")
+	flag.BoolVar(&confirm, "confirm", true, "Requires confirmation before inserting records")
+	flag.BoolVar(&dump, "dump", false, "Write keys (from 'from' server) to disk")
+}
+
+func supportedBackends() []TVBackend {
+	return []TVBackend{
+		&riakBE, &pgBE,
+	}
+}
+
+func main() {
+	flag.Parse()
+	var fromSrv TVBackend
+	var toSrv TVBackend
+
+	toSrvUsed := !dump && !dry
+
+	if !validateType(fromType) {
+		log.Fatal("Unknown fromType " + fromType)
+	}
+	if toSrvUsed && !validateType(toType) {
+		log.Fatal("Unknown toType " + toType)
+	}
+
+	fromSrv = getBackendFromType(fromType)
+	if toSrvUsed {
+		toSrv = getBackendFromType(toType)
+	}
+
+	var toTimer time.Time
+	var toTime float64
+	var fromTimer time.Time
+	var fromTime float64
+
+	log.Println("Reading configs...")
+	fromTimer = time.Now()
+	if err := fromSrv.ReadConfig(fromCfg); err != nil {
+		log.Fatalf("Unable to read fromSrv cfg: %v", err)
+	}
+	fromTime = time.Now().Sub(fromTimer).Seconds()
+
+	if toSrvUsed {
+		toTimer = time.Now()
+		if err := toSrv.ReadConfig(toCfg); err != nil {
+			log.Fatalf("Unable to read toSrv cfg: %v", err)
+		}
+		toTime := time.Now().Sub(toTimer).Seconds()
+		log.Printf("Done [%v seconds]\n\tto: [%v seconds]\n\tfrom: [%v seconds]\n", toTime+fromTime, toTime, fromTime)
+	} else {
+		log.Printf("Done [%v seconds]\n", fromTime)
+	}
+
+	log.Println("Starting servers...")
+	fromTimer = time.Now()
+	if err := fromSrv.Start(); err != nil {
+		log.Fatalf("issue starting fromSrv: %v", err)
+	}
+	fromTime = time.Now().Sub(fromTimer).Seconds()
+	defer func() {
+		fromSrv.Stop()
+	}()
+	if toSrvUsed {
+		toTimer = time.Now()
+		if err := toSrv.Start(); err != nil {
+			log.Fatalf("issue starting toSrv: %v", err)
+		}
+		toTime = time.Now().Sub(toTimer).Seconds()
+		defer func() {
+			toSrv.Stop()
+		}()
+		log.Printf("Done [%v seconds]\n\tto: [%v seconds]\n\tfrom: [%v seconds]\n", toTime+fromTime, toTime, fromTime)
+	} else {
+		log.Printf("Done [%v seconds]\n", fromTime)
+	}
+
+	log.Println("Pinging servers...")
+	fromTimer = time.Now()
+	if err := fromSrv.Ping(); err != nil {
+		log.Fatalf("Unable to ping fromSrv: %v", err)
+	}
+	fromTime = time.Now().Sub(fromTimer).Seconds()
+	if toSrvUsed {
+		toTimer = time.Now()
+		if err := toSrv.Ping(); err != nil {
+			log.Fatalf("Unable to ping toSrv: %v", err)
+		}
+		toTime = time.Now().Sub(toTimer).Seconds()
+		log.Printf("Done [%v seconds]\n\tto: [%v seconds]\n\tfrom: [%v seconds]\n", toTime+fromTime, toTime, fromTime)
+	} else {
+		log.Printf("Done [%v seconds]\n", fromTime)
+	}
+
+	log.Printf("Fetching data from %v...\n", fromSrv.Name())
+	fromTimer = time.Now()
+	if err := fromSrv.Fetch(); err != nil {
+		log.Fatalf("Unable to fetch fromSrv data: %v", err)
+	}
+
+	fromSSLKeys, fromDNSSecKeys, fromURIKeys, fromURLKeys, err := GetKeys(fromSrv)
+	if err != nil {
+		log.Fatal(err)
+	}
+
+	if err := Validate(fromSrv); err != nil {
+		log.Fatal(err)
+	}
+	log.Printf("Done [%v seconds]\n", time.Now().Sub(fromTimer).Seconds())
+
+	if dump {
+		log.Printf("Dumping data from %v...\n", fromSrv.Name())
+		fromTimer = time.Now()
+		sslKeysBytes, err := json.Marshal(&fromSSLKeys)
+		if err != nil {
+			log.Fatal(err)
+		}
+		dnssecKeyBytes, err := json.Marshal(&fromDNSSecKeys)
+		if err != nil {
+			log.Fatal(err)
+		}
+		uriKeyBytes, err := json.Marshal(&fromURIKeys)
+		if err != nil {
+			log.Fatal(err)
+		}
+		urlKeyBytes, err := json.Marshal(&fromURLKeys)
+		if err != nil {
+			log.Fatal(err)
+		}
+
+		if err := os.Mkdir("dump", 0750); err != nil {
+			if !os.IsExist(err) {
+				log.Fatal(err)
+			}
+		}
+		if err = ioutil.WriteFile("dump/sslkeys.json", sslKeysBytes, 0640); err != nil {
+			log.Println(err)
+		}
+		if err = ioutil.WriteFile("dump/dnsseckeys.json", dnssecKeyBytes, 0640); err != nil {
+			log.Println(err)
+		}
+		if err = ioutil.WriteFile("dump/urlkeys.json", urlKeyBytes, 0640); err != nil {
+			log.Println(err)
+		}
+		if err = ioutil.WriteFile("dump/urikeys.json", uriKeyBytes, 0640); err != nil {
+			log.Println(err)
+		}
+		log.Printf("Done [%v seconds]\n", time.Now().Sub(fromTimer).Seconds())
+		return
+	}
+
+	if compare {
+		log.Printf("Fetching data from %v...\n", toSrv.Name())
+		toTimer = time.Now()
+		if err := toSrv.Fetch(); err != nil {
+			log.Fatalf("Unable to fetch toSrv data: %v\n", err)
+		}
+
+		toSSLKeys, toDNSSecKeys, toURIKeys, toURLKeys, err := GetKeys(toSrv)
+		if err != nil {
+			log.Fatal(err)
+		}
+		log.Println("Validating " + toSrv.Name())
+		if err := toSrv.ValidateKey(); err != nil && len(err) > 0 {
+			log.Fatal(strings.Join(err, "\n"))
+		}
+		log.Printf("Done [%v seconds]\n", time.Now().Sub(toTimer).Seconds())
+		log.Println(fromSrv.String())
+		log.Println(toSrv.String())
+
+		if !reflect.DeepEqual(fromSSLKeys, toSSLKeys) {
+			log.Fatal("from sslkeys and to sslkeys don't match")
+		}
+		if !reflect.DeepEqual(fromDNSSecKeys, toDNSSecKeys) {
+			log.Fatal("from dnssec and to dnssec don't match")
+		}
+		if !reflect.DeepEqual(fromURIKeys, toURIKeys) {
+			log.Fatal("from uri and to uri don't match")
+		}
+		if !reflect.DeepEqual(fromURLKeys, toURLKeys) {
+			log.Fatal("from url and to url don't match")
+		}
+		log.Println("Both datasources have the same keys!")
+		return
+	}
+
+	log.Printf("Setting %v keys...\n", toSrv.Name())
+	toTimer = time.Now()
+	if err := SetKeys(toSrv, fromSSLKeys, fromDNSSecKeys, fromURIKeys, fromURLKeys); err != nil {
+		log.Fatal(err)
+	}
+
+	if err := Validate(toSrv); err != nil {
+		log.Fatal(err)
+	}
+	log.Printf("Done [%v seconds]\n", time.Now().Sub(toTimer).Seconds())
+
+	log.Println(fromSrv.String())
+
+	if dry {
+		return
+	}
+
+	if confirm {
+		ans := "q"
+		for {
+			fmt.Print("Confirm data insertion (y/n): ")
+			if _, err := fmt.Scanln(&ans); err != nil {
+				log.Fatal("unable to get user input")
+			}
+
+			if ans == "y" {
+				break
+			} else if ans == "n" {
+				return
+			}
+		}
+	}
+	log.Printf("Inserting data into %v...\n", toSrv.Name())
+	toTimer = time.Now()
+	if err := toSrv.Insert(); err != nil {
+		log.Fatal(err)
+	}
+	log.Printf("Done [%v seconds]\n", time.Now().Sub(toTimer).Seconds())
+
+}
+
+// Validate runs the ValidateKey method on the backend
+func Validate(be TVBackend) error {
+	if errs := be.ValidateKey(); errs != nil && len(errs) > 0 {
+		return errors.New(fmt.Sprintf("Validation Errors (%v): \n%v", be.Name(), strings.Join(errs, "\n")))
+	} else {
+		log.Println("Validated " + be.Name())
+	}
+	return nil
+}
+
+// SetKeys will set all of the keys for a backend
+func SetKeys(be TVBackend, sslkeys []SSLKey, dnssecKeys []DNSSecKey, uriKeys []URISignKey, urlKeys []URLSigKey) error {
+	if err := be.SetSSLKeys(sslkeys); err != nil {
+		return errors.New(fmt.Sprintf("Unable to set %v ssl keys: %v", be.Name(), err))
+	}
+	if err := be.SetDNSSecKeys(dnssecKeys); err != nil {
+		return errors.New(fmt.Sprintf("Unable to set %v dnssec keys: %v", be.Name(), err))
+	}
+	if err := be.SetURLSigKeys(urlKeys); err != nil {
+		return errors.New(fmt.Sprintf("Unable to set %v url keys: %v", be.Name(), err))
+	}
+	if err := be.SetURISignKeys(uriKeys); err != nil {
+		return errors.New(fmt.Sprintf("Unable to set %v uri keys: %v", be.Name(), err))
+	}
+	return nil
+}
+
+// GetKeys will get all of the keys for a backend
+func GetKeys(be TVBackend) ([]SSLKey, []DNSSecKey, []URISignKey, []URLSigKey, error) {
+	var sslkeys []SSLKey
+	var dnssec []DNSSecKey
+	var uri []URISignKey
+	var url []URLSigKey
+	var err error
+	if sslkeys, err = be.GetSSLKeys(); err != nil {
+		return nil, nil, nil, nil, errors.New(fmt.Sprintf("Unable to get %v sslkeys: %v", be.Name(), err))
+	}
+	if dnssec, err = be.GetDNSSecKeys(); err != nil {
+		return nil, nil, nil, nil, errors.New(fmt.Sprintf("Unable to get %v dnssec keys: %v", be.Name(), err))
+	}
+	if uri, err = be.GetURISignKeys(); err != nil {
+		return nil, nil, nil, nil, errors.New(fmt.Sprintf("Unable to get %v uri keys: %v", be.Name(), err))
+	}
+	if url, err = be.GetURLSigKeys(); err != nil {
+		return nil, nil, nil, nil, errors.New(fmt.Sprintf("Unable to %v url keys: %v", be.Name(), err))
+	}
+	sort.Slice(sslkeys[:], func(a, b int) bool {
+		return sslkeys[a].CDN < sslkeys[b].CDN && sslkeys[a].DeliveryService < sslkeys[b].DeliveryService && sslkeys[a].Version < sslkeys[b].Version
+	})
+	sort.Slice(dnssec[:], func(a, b int) bool {
+		return dnssec[a].CDN < dnssec[b].CDN
+	})
+	sort.Slice(uri[:], func(a, b int) bool {
+		return uri[a].DeliveryService < uri[b].DeliveryService
+	})
+	sort.Slice(url[:], func(a, b int) bool {
+		return url[a].DeliveryService < url[b].DeliveryService
+	})
+	return sslkeys, dnssec, uri, url, nil
+}
+
+// UnmarshalConfig takes in a config file and a type and will read the config file into the reflected type
+func UnmarshalConfig(configFile string, t reflect.Type) (reflect.Value, error) {
+	data, err := ioutil.ReadFile(configFile)
+	if err != nil {
+		return reflect.Value{}, err
+	}
+	val := reflect.New(t)
+	err = json.Unmarshal(data, val.Interface())
+	if err != nil {
+		return reflect.Value{}, err
+	}
+
+	return val, nil
+}
+
+// TVBackend represents a TV backend that can be have data migrated to/from
+type TVBackend interface {
+	// Start initiates the connection to the backend DB
+	Start() error
+	// Stop terminates the connection to the backend DB
+	Stop() error
+	// Ping checks the connection to the backend DB
+	Ping() error
+	// ValidateKey validates that the keys are valid (in most cases, certain fields are not null)
+	ValidateKey() []string
+	// Name returns the name for this backend
+	Name() string
+	// ReadConfig takes in a filename and will read it into the backends config
+	ReadConfig(string) error
+	// String returns a high level overview of the backend and its keys
+	String() string
+
+	// Fetch gets all of the keys from the backend DB
+	Fetch() error
+	// Insert takes the current keys and inserts them into the backend DB
+	Insert() error
+
+	// GetSSLKeys converts the backends internal key representation into the common representation (SSLKey)
+	GetSSLKeys() ([]SSLKey, error)
+	// SetSSLKeys takes in keys and converts & encrypts the data into the backends internal format
+	SetSSLKeys([]SSLKey) error
+
+	// GetDNSSecKeys converts the backends internal key representation into the common representation (DNSSecKey)
+	GetDNSSecKeys() ([]DNSSecKey, error)
+	// SetDNSSecKeys takes in keys and converts & encrypts the data into the backends internal format
+	SetDNSSecKeys([]DNSSecKey) error
+
+	// GetURISignKeys converts the pg internal key representation into the common representation (URISignKey)
+	GetURISignKeys() ([]URISignKey, error)
+	// SetURISignKeys takes in keys and converts & encrypts the data into the backends internal format
+	SetURISignKeys([]URISignKey) error
+
+	// GetURLSigKeys converts the backends internal key representation into the common representation (URLSigKey)
+	GetURLSigKeys() ([]URLSigKey, error)
+	// SetURLSigKeys takes in keys and converts & encrypts the data into the backends internal format
+	SetURLSigKeys([]URLSigKey) error
+}
+
+type CommonRecord struct{}

Review comment:
       Used to hold Last Updated, forgot to remove it.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [trafficcontrol] shamrickus commented on a change in pull request #5924: Add tool to migrate data between TV backends

Posted by GitBox <gi...@apache.org>.
shamrickus commented on a change in pull request #5924:
URL: https://github.com/apache/trafficcontrol/pull/5924#discussion_r651035899



##########
File path: tools/traffic_vault_migrate/postgres.go
##########
@@ -0,0 +1,743 @@
+package main
+
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+import (
+	"database/sql"
+	"encoding/base64"
+	"encoding/json"
+	"errors"
+	"fmt"
+	"log"
+	"strconv"
+	"strings"
+
+	"github.com/apache/trafficcontrol/lib/go-tc"
+	util "github.com/apache/trafficcontrol/lib/go-util"
+
+	_ "github.com/lib/pq"
+)
+
+// PGConfig represents the configuration options available to the PG backend
+type PGConfig struct {
+	Host     string `json:"host"`
+	Port     string `json:"port"`
+	User     string `json:"user"`
+	Password string `json:"password"`
+	SSLMode  string `json:"sslmode"`
+	Database string `json:"database"`
+	Key      string `json:"aesKey"`
+	AESKey   []byte
+}
+
+// PGBackend is the Postgres implementation of TVBackend
+type PGBackend struct {
+	sslKey pgSSLKeyTable
+	dnssec pgDNSSecTable
+	uri    pgURISignKeyTable
+	url    pgURLSigKeyTable
+	cfg    PGConfig
+	db     *sql.DB
+}
+
+// String returns a high level overview of the backend and its keys
+func (pg *PGBackend) String() string {
+	data := fmt.Sprintf("PG server %v@%v:%v\n", pg.cfg.User, pg.cfg.Host, pg.cfg.Port)
+	data += fmt.Sprintf("\tSSL Keys: %v\n", len(pg.sslKey.Records))
+	data += fmt.Sprintf("\tDNSSec Keys: %v\n", len(pg.dnssec.Records))
+	data += fmt.Sprintf("\tURI Keys: %v\n", len(pg.uri.Records))
+	data += fmt.Sprintf("\tURL Keys: %v\n", len(pg.url.Records))
+	return data
+}
+
+// Name returns the name for this backend
+func (pg *PGBackend) Name() string {
+	return "PG"
+}
+
+// ReadConfig takes in a filename and will read it into the backends config
+func (pg *PGBackend) ReadConfig(s string) error {
+	err := UnmarshalConfig(s, &pg.cfg)
+	if err != nil {
+		return err
+	}
+
+	pg.cfg.AESKey, err = base64.StdEncoding.DecodeString(pg.cfg.Key)
+	if err != nil {
+		return fmt.Errorf("unable to decode PG AESKey: %w", err)
+	}
+	return nil
+}
+
+// Insert takes the current keys and inserts them into the backend DB
+func (pg *PGBackend) Insert() error {
+	if err := pg.sslKey.insertKeys(pg.db); err != nil {
+		return err
+	}
+	if err := pg.dnssec.insertKeys(pg.db); err != nil {
+		return err
+	}
+	if err := pg.url.insertKeys(pg.db); err != nil {
+		return err
+	}
+	if err := pg.uri.insertKeys(pg.db); err != nil {
+		return err
+	}
+	return nil
+}
+
+// Start initiates the connection to the backend DB
+func (pg *PGBackend) Start() error {
+	sqlStr := fmt.Sprintf("postgres://%s:%s@%s:%s/%s?sslmode=%s", pg.cfg.User, pg.cfg.Password, pg.cfg.Host, pg.cfg.Port, pg.cfg.Database, pg.cfg.SSLMode)
+	db, err := sql.Open("postgres", sqlStr)
+	if err != nil {
+		return fmt.Errorf("unable to start PG client: %w", err)
+	}
+
+	pg.db = db
+	pg.sslKey = pgSSLKeyTable{}
+	pg.dnssec = pgDNSSecTable{}
+	pg.url = pgURLSigKeyTable{}
+	pg.uri = pgURISignKeyTable{}
+
+	return nil
+}
+
+// ValidateKey validates that the keys are valid (in most cases, certain fields are not null)
+func (pg *PGBackend) ValidateKey() []string {
+	var errors []string
+	if errs := pg.sslKey.validate(); errs != nil {
+		errors = append(errors, errs...)
+	}
+	if errs := pg.dnssec.validate(); errs != nil {
+		errors = append(errors, errs...)
+	}
+	if errs := pg.uri.validate(); errs != nil {
+		errors = append(errors, errs...)
+	}
+	if errs := pg.url.validate(); errs != nil {
+		errors = append(errors, errs...)
+	}
+	return errors
+}
+
+// Close terminates the connection to the backend DB
+func (pg *PGBackend) Close() error {
+	return pg.db.Close()
+}
+
+// Ping checks the connection to the backend DB
+func (pg *PGBackend) Ping() error {
+	return pg.db.Ping()
+}
+
+// Fetch gets all of the keys from the backend DB
+func (pg *PGBackend) Fetch() error {
+	if err := pg.sslKey.gatherKeys(pg.db); err != nil {
+		return err
+	}
+
+	if err := pg.dnssec.gatherKeys(pg.db); err != nil {
+		return err
+	}
+
+	if err := pg.url.gatherKeys(pg.db); err != nil {
+		return err
+	}
+
+	if err := pg.uri.gatherKeys(pg.db); err != nil {
+		return err
+	}
+
+	return nil
+}
+
+// GetSSLKeys converts the backends internal key representation into the common representation (SSLKey)
+func (pg *PGBackend) GetSSLKeys() ([]SSLKey, error) {
+	if err := pg.sslKey.decrypt(pg.cfg.AESKey); err != nil {
+		return nil, err
+	}
+	return pg.sslKey.toGeneric(), nil
+}
+
+// SetSSLKeys takes in keys and converts & encrypts the data into the backends internal format
+func (pg *PGBackend) SetSSLKeys(keys []SSLKey) error {
+	pg.sslKey.fromGeneric(keys)
+	return pg.sslKey.encrypt(pg.cfg.AESKey)
+}
+
+// GetDNSSecKeys converts the backends internal key representation into the common representation (DNSSecKey)
+func (pg *PGBackend) GetDNSSecKeys() ([]DNSSecKey, error) {
+	if err := pg.dnssec.decrypt(pg.cfg.AESKey); err != nil {
+		return nil, err
+	}
+	return pg.dnssec.toGeneric(), nil
+}
+
+// SetDNSSecKeys takes in keys and converts & encrypts the data into the backends internal format
+func (pg *PGBackend) SetDNSSecKeys(keys []DNSSecKey) error {
+	pg.dnssec.fromGeneric(keys)
+	return pg.dnssec.encrypt(pg.cfg.AESKey)
+}
+
+// GetURISignKeys converts the pg internal key representation into the common representation (URISignKey)
+func (pg *PGBackend) GetURISignKeys() ([]URISignKey, error) {
+	if err := pg.uri.decrypt(pg.cfg.AESKey); err != nil {
+		return nil, err
+	}
+	return pg.uri.toGeneric(), nil
+}
+
+// SetURISignKeys takes in keys and converts & encrypts the data into the backends internal format
+func (pg *PGBackend) SetURISignKeys(keys []URISignKey) error {
+	pg.uri.fromGeneric(keys)
+	return pg.uri.encrypt(pg.cfg.AESKey)
+}
+
+// GetURLSigKeys converts the backends internal key representation into the common representation (URLSigKey)
+func (pg *PGBackend) GetURLSigKeys() ([]URLSigKey, error) {
+	if err := pg.url.decrypt(pg.cfg.AESKey); err != nil {
+		return nil, err
+	}
+	return pg.url.toGeneric(), nil
+}
+
+// SetURLSigKeys takes in keys and converts & encrypts the data into the backends internal format
+func (pg *PGBackend) SetURLSigKeys(keys []URLSigKey) error {
+	pg.url.fromGeneric(keys)
+	return pg.url.encrypt(pg.cfg.AESKey)
+}
+
+type pgCommonRecord struct {
+	DataEncrypted []byte
+}
+
+type pgDNSSecRecord struct {
+	Key tc.DNSSECKeysTrafficVault
+	CDN string
+	pgCommonRecord
+}
+type pgDNSSecTable struct {
+	Records []pgDNSSecRecord
+}
+
+func (tbl *pgDNSSecTable) gatherKeys(db *sql.DB) error {
+	sz, err := getSize(db, "dnssec")
+	if err != nil {
+		log.Println("PGDNSSec gatherKeys: unable to determine size of dnssec table")
+	}
+	tbl.Records = make([]pgDNSSecRecord, sz)
+
+	rows, err := db.Query("SELECT cdn, data from dnssec")
+	if err != nil {
+		return fmt.Errorf("PGDNSSec gatherKeys: unable to query: %w", err)
+	}
+	defer rows.Close()
+	i := 0
+	for rows.Next() {
+		if i > len(tbl.Records)-1 {
+			return fmt.Errorf("PGDNSSec gatherKeys got more results than expected %v", len(tbl.Records))
+		}
+		err := rows.Scan(&tbl.Records[i].CDN, &tbl.Records[i].DataEncrypted)
+		if err != nil {
+			return fmt.Errorf("PGDNSSec gatherKeys unable to scan row: %w", err)
+		}
+		i += 1
+	}
+	return nil
+}
+func (tbl *pgDNSSecTable) decrypt(aesKey []byte) error {
+	for i, _ := range tbl.Records {
+		err := decryptInto(aesKey, tbl.Records[i].DataEncrypted, &tbl.Records[i].Key)
+		if err != nil {
+			return fmt.Errorf("unable to decrypt into keys: %w", err)
+		}
+	}
+	return nil
+}
+func (tbl *pgDNSSecTable) encrypt(aesKey []byte) error {
+	for i, dns := range tbl.Records {
+		data, err := json.Marshal(&dns.Key)
+		if err != nil {
+			return fmt.Errorf("encrypt issue marshalling keys: %w", err)
+		}
+		dat, err := encrypt(data, aesKey)
+		if err != nil {
+			return fmt.Errorf("encrypt error: %w", err)
+		}
+		tbl.Records[i].DataEncrypted = dat
+	}
+	return nil
+}
+func (tbl *pgDNSSecTable) toGeneric() []DNSSecKey {
+	keys := make([]DNSSecKey, len(tbl.Records))
+
+	for i, record := range tbl.Records {
+		keys[i] = DNSSecKey{
+			CDN:                    record.CDN,
+			DNSSECKeysTrafficVault: record.Key,
+		}
+	}
+
+	return keys
+}
+func (tbl *pgDNSSecTable) fromGeneric(keys []DNSSecKey) {
+	tbl.Records = make([]pgDNSSecRecord, len(keys))
+
+	for i, key := range keys {
+		tbl.Records[i] = pgDNSSecRecord{
+			Key: key.DNSSECKeysTrafficVault,
+			CDN: key.CDN,
+			pgCommonRecord: pgCommonRecord{
+				DataEncrypted: nil,
+			},
+		}
+	}
+}
+func (tbl *pgDNSSecTable) validate() []string {
+	for i, record := range tbl.Records {
+		if record.DataEncrypted == nil && len(record.Key) > 0 {
+			return []string{fmt.Sprintf("DNSSEC Key %v: DataEncrypted is blank!", i)}
+		}
+	}
+	return nil
+}
+func (tbl *pgDNSSecTable) insertKeys(db *sql.DB) error {
+	queryBase := "INSERT INTO dnssec (cdn, data) VALUES "
+	stride := 2
+	queryArgs := make([]interface{}, len(tbl.Records)*stride)
+	for i, record := range tbl.Records {
+		j := i * stride
+		queryArgs[j] = record.CDN
+		queryArgs[j+1] = record.DataEncrypted
+	}
+	return insertIntoTable(db, queryBase, stride, queryArgs)
+}
+
+type pgSSLKeyRecord struct {
+	Keys tc.DeliveryServiceSSLKeys
+	pgCommonRecord
+
+	// These records are stored on the table but are duplicated
+	DeliveryService string
+	CDN             string
+}
+type pgSSLKeyTable struct {
+	Records []pgSSLKeyRecord
+}
+
+func (tbl *pgSSLKeyTable) insertKeys(db *sql.DB) error {
+	queryBase := "INSERT INTO sslkey (deliveryservice, data, cdn, version) VALUES "
+	duplicateKeys := 2
+	stride := 4
+	queryArgs := make([]interface{}, len(tbl.Records)*stride*duplicateKeys)
+	for i, record := range tbl.Records {
+		j := i * duplicateKeys * stride
+
+		queryArgs[j] = record.DeliveryService
+		queryArgs[j+1] = record.DataEncrypted
+		queryArgs[j+2] = record.CDN
+		queryArgs[j+3] = record.Keys.Version.String()
+
+		queryArgs[j+4] = record.DeliveryService
+		queryArgs[j+5] = record.DataEncrypted
+		queryArgs[j+6] = record.CDN
+		queryArgs[j+7] = "latest"
+	}
+	return insertIntoTable(db, queryBase, 4, queryArgs)
+}
+func (tbl *pgSSLKeyTable) gatherKeys(db *sql.DB) error {
+	sz, err := getSize(db, "sslkey WHERE version='latest'")

Review comment:
       The other counterpart to each latest record is identical, it is inserted twice for searching/indexing.The reason I chose to only get latest records was that otherwise there are duplicate `SSLKey` records.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [trafficcontrol] rawlinp commented on a change in pull request #5924: Add tool to migrate data between TV backends

Posted by GitBox <gi...@apache.org>.
rawlinp commented on a change in pull request #5924:
URL: https://github.com/apache/trafficcontrol/pull/5924#discussion_r648709675



##########
File path: lib/go-util/encrypt.go
##########
@@ -0,0 +1,77 @@
+package util
+
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+import (
+	"crypto/aes"
+	"crypto/cipher"
+	"crypto/rand"
+	"io"
+)
+
+// AESEncrypt takes in a 16, 24, or 32 byte AES key (128, 192, 256 bit encryption relatively) and plain text. It

Review comment:
       I think you mean "respectively" instead of "relatively"

##########
File path: tools/traffic_vault_migrate/traffic_vault_migrate.go
##########
@@ -0,0 +1,448 @@
+package main
+
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+import (
+	"encoding/json"
+	"errors"
+	"flag"
+	"fmt"
+	"io/ioutil"
+	"log"
+	"os"
+	"reflect"
+	"sort"
+	"strings"
+	"time"
+
+	"github.com/apache/trafficcontrol/lib/go-tc"
+)
+
+var (
+	fromType string
+	toType   string
+	fromCfg  string
+	toCfg    string
+	dry      bool
+	compare  bool
+	confirm  bool
+	dump     bool
+
+	riakBE RiakBackend = RiakBackend{}
+	pgBE   PGBackend   = PGBackend{}
+)
+
+func init() {
+	flag.StringVar(&fromType, "from_type", riakBE.Name(), fmt.Sprintf("From server types (%v)", strings.Join(supportedTypes(), "|")))
+	flag.StringVar(&toType, "to_type", pgBE.Name(), fmt.Sprintf("From server types (%v)", strings.Join(supportedTypes(), "|")))
+	flag.StringVar(&toCfg, "to_cfg", "pg.json", "To server config file")
+	flag.StringVar(&fromCfg, "from_cfg", "riak.json", "From server config file")
+	flag.BoolVar(&dry, "dry", false, "Do not perform writes")
+	flag.BoolVar(&compare, "compare", false, "Compare to and from server records")
+	flag.BoolVar(&confirm, "confirm", true, "Requires confirmation before inserting records")
+	flag.BoolVar(&dump, "dump", false, "Write keys (from 'from' server) to disk")
+}
+
+func supportedBackends() []TVBackend {
+	return []TVBackend{
+		&riakBE, &pgBE,
+	}
+}
+
+func main() {
+	flag.Parse()
+	var fromSrv TVBackend
+	var toSrv TVBackend
+
+	toSrvUsed := !dump && !dry
+
+	if !validateType(fromType) {
+		log.Fatal("Unknown fromType " + fromType)
+	}
+	if toSrvUsed && !validateType(toType) {
+		log.Fatal("Unknown toType " + toType)
+	}
+
+	fromSrv = getBackendFromType(fromType)
+	if toSrvUsed {
+		toSrv = getBackendFromType(toType)
+	}
+
+	var toTimer time.Time
+	var toTime float64
+	var fromTimer time.Time
+	var fromTime float64
+
+	log.Println("Reading configs...")
+	fromTimer = time.Now()
+	if err := fromSrv.ReadConfig(fromCfg); err != nil {
+		log.Fatalf("Unable to read fromSrv cfg: %v", err)
+	}
+	fromTime = time.Now().Sub(fromTimer).Seconds()
+
+	if toSrvUsed {
+		toTimer = time.Now()
+		if err := toSrv.ReadConfig(toCfg); err != nil {
+			log.Fatalf("Unable to read toSrv cfg: %v", err)
+		}
+		toTime := time.Now().Sub(toTimer).Seconds()
+		log.Printf("Done [%v seconds]\n\tto: [%v seconds]\n\tfrom: [%v seconds]\n", toTime+fromTime, toTime, fromTime)
+	} else {
+		log.Printf("Done [%v seconds]\n", fromTime)
+	}
+
+	log.Println("Starting servers...")
+	fromTimer = time.Now()
+	if err := fromSrv.Start(); err != nil {
+		log.Fatalf("issue starting fromSrv: %v", err)
+	}
+	fromTime = time.Now().Sub(fromTimer).Seconds()
+	defer func() {
+		fromSrv.Stop()
+	}()
+	if toSrvUsed {
+		toTimer = time.Now()
+		if err := toSrv.Start(); err != nil {
+			log.Fatalf("issue starting toSrv: %v", err)
+		}
+		toTime = time.Now().Sub(toTimer).Seconds()
+		defer func() {
+			toSrv.Stop()
+		}()
+		log.Printf("Done [%v seconds]\n\tto: [%v seconds]\n\tfrom: [%v seconds]\n", toTime+fromTime, toTime, fromTime)
+	} else {
+		log.Printf("Done [%v seconds]\n", fromTime)
+	}
+
+	log.Println("Pinging servers...")
+	fromTimer = time.Now()
+	if err := fromSrv.Ping(); err != nil {
+		log.Fatalf("Unable to ping fromSrv: %v", err)
+	}
+	fromTime = time.Now().Sub(fromTimer).Seconds()
+	if toSrvUsed {
+		toTimer = time.Now()
+		if err := toSrv.Ping(); err != nil {
+			log.Fatalf("Unable to ping toSrv: %v", err)
+		}
+		toTime = time.Now().Sub(toTimer).Seconds()
+		log.Printf("Done [%v seconds]\n\tto: [%v seconds]\n\tfrom: [%v seconds]\n", toTime+fromTime, toTime, fromTime)
+	} else {
+		log.Printf("Done [%v seconds]\n", fromTime)
+	}
+
+	log.Printf("Fetching data from %v...\n", fromSrv.Name())
+	fromTimer = time.Now()
+	if err := fromSrv.Fetch(); err != nil {
+		log.Fatalf("Unable to fetch fromSrv data: %v", err)
+	}
+
+	fromSSLKeys, fromDNSSecKeys, fromURIKeys, fromURLKeys, err := GetKeys(fromSrv)
+	if err != nil {
+		log.Fatal(err)
+	}
+
+	if err := Validate(fromSrv); err != nil {
+		log.Fatal(err)
+	}
+	log.Printf("Done [%v seconds]\n", time.Now().Sub(fromTimer).Seconds())
+
+	if dump {
+		log.Printf("Dumping data from %v...\n", fromSrv.Name())
+		fromTimer = time.Now()
+		sslKeysBytes, err := json.Marshal(&fromSSLKeys)

Review comment:
       There's a pattern here -- we're doing the same thing 4 times in a row: marshalling keys and writing to a file. Can we condense this into a loop that iterates over a slice of structs to marshal the json and write to disk?

##########
File path: tools/traffic_vault_migrate/riak.json
##########
@@ -0,0 +1,8 @@
+{
+  "user": "admin",
+  "password": "riakAdmin",
+  "host": "localhost",
+  "port": "8087",
+  "tls": false,

Review comment:
       RE: earlier comment, this should be `insecure` instead of just `tls`

##########
File path: docs/source/tools/traffic_vault_migrate.rst
##########
@@ -0,0 +1,107 @@
+..
+..
+.. Licensed under the Apache License, Version 2.0 (the "License");
+.. you may not use this file except in compliance with the License.
+.. You may obtain a copy of the License at
+..
+..     http://www.apache.org/licenses/LICENSE-2.0
+..
+.. Unless required by applicable law or agreed to in writing, software
+.. distributed under the License is distributed on an "AS IS" BASIS,
+.. WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+.. See the License for the specific language governing permissions and
+.. limitations under the License.
+..
+
+.. _traffic_vault_migrate:
+
+#########################
+Traffic Vault Migrate
+#########################
+The ``traffic_vault_migrate`` tool - located at :file:`tools/traffic_vault_migrate/traffic_vault_migrate.go` in the `Apache Traffic Control repository <https://github.com/apache/trafficcontrol>`_ -
+is used to transfer TV keys between database servers. It interfaces directly with each backend so Traffic Ops/Vault being available is not a requirement.
+The tool assumes that the schema for each backend is already setup as according to the :ref:`admin setup <traffic_vault_admin>`.
+
+.. program:: traffic_vault_migrate
+
+Usage
+===========
+``traffic_vault_migrate -from_cfg CFG -to_cfg CFG -from_type TYP -to_type TYP [-confirm] [-compare] [-dry] [-dump]``
+
+.. option:: -compare
+
+		Compare 'to' and 'from' backend keys. Will fetch keys from the dbs of both 'to' and 'from', sorts them by cdn/ds/version and does a deep comparison.
+
+.. option:: -confirm
+
+		Requires confirmation before inserting records (default true)
+
+.. option:: -dry
+
+		Do not perform writes. Will do a basic output of the keys on the 'from' backend.
+
+.. option:: -dump
+
+		Write keys (from 'from' server) to disk in the folder 'dump' with the unix permissions 0640.
+
+		.. warning:: This can write potentially sensitive information to disk, use with care.
+
+.. option:: -from_cfg
+
+		From server config file (default "riak.json")
+
+.. option:: -from_type
+
+		From server types (Riak|PG) (default "Riak")
+
+.. option:: -to_cfg
+
+		To server config file (default "pg.json")
+
+.. option:: -to_type
+
+		From server types (Riak|PG) (default "PG")
+
+Riak
+----------
+
+riak.json
+""""""""""
+
+ :user: The username used to log into the Riak server.
+
+ :password: The password used to log into the Riak server.
+
+ :host: The hostname for the Riak server.
+
+ :port: The port for which the Riak server is listening for protobuf connections.
+
+ :tls: (Optional) Determines whether to verify insecure certificates.

Review comment:
       this should probably be `insecure` (default = false) instead of just `tls`

##########
File path: lib/go-util/encrypt.go
##########
@@ -0,0 +1,77 @@
+package util
+
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+import (
+	"crypto/aes"
+	"crypto/cipher"
+	"crypto/rand"
+	"io"
+)
+
+// AESEncrypt takes in a 16, 24, or 32 byte AES key (128, 192, 256 bit encryption relatively) and plain text. It
+// returns the encrypted text. In case of error, the text returned is an empty string. AES requires input text to
+// be greater than 12 bytes in length.
+func AESEncrypt(bytesToEncrypt []byte, aesKey []byte) ([]byte, error) {
+	cipherBlock, err := aes.NewCipher(aesKey)
+	if err != nil {
+		return []byte{}, err
+	}
+
+	gcm, err := cipher.NewGCM(cipherBlock)
+	if err != nil {
+		return []byte{}, err
+	}
+
+	nonce := make([]byte, gcm.NonceSize())
+	if _, err = io.ReadFull(rand.Reader, nonce); err != nil {
+		return []byte{}, err
+	}
+
+	return gcm.Seal(nonce, nonce, bytesToEncrypt, nil), nil
+}
+
+// AESDecrypt takes in a 16, 24, or 32 byte AES key (128, 192, 256 bit encryption relatively) and encrypted text. It

Review comment:
       I think you mean "respectively" instead of "relatively"

##########
File path: tools/traffic_vault_migrate/traffic_vault_migrate.go
##########
@@ -0,0 +1,448 @@
+package main
+
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+import (
+	"encoding/json"
+	"errors"
+	"flag"
+	"fmt"
+	"io/ioutil"
+	"log"
+	"os"
+	"reflect"
+	"sort"
+	"strings"
+	"time"
+
+	"github.com/apache/trafficcontrol/lib/go-tc"
+)
+
+var (
+	fromType string
+	toType   string
+	fromCfg  string
+	toCfg    string
+	dry      bool
+	compare  bool
+	confirm  bool
+	dump     bool
+
+	riakBE RiakBackend = RiakBackend{}
+	pgBE   PGBackend   = PGBackend{}
+)
+
+func init() {
+	flag.StringVar(&fromType, "from_type", riakBE.Name(), fmt.Sprintf("From server types (%v)", strings.Join(supportedTypes(), "|")))
+	flag.StringVar(&toType, "to_type", pgBE.Name(), fmt.Sprintf("From server types (%v)", strings.Join(supportedTypes(), "|")))
+	flag.StringVar(&toCfg, "to_cfg", "pg.json", "To server config file")
+	flag.StringVar(&fromCfg, "from_cfg", "riak.json", "From server config file")
+	flag.BoolVar(&dry, "dry", false, "Do not perform writes")
+	flag.BoolVar(&compare, "compare", false, "Compare to and from server records")
+	flag.BoolVar(&confirm, "confirm", true, "Requires confirmation before inserting records")
+	flag.BoolVar(&dump, "dump", false, "Write keys (from 'from' server) to disk")
+}
+
+func supportedBackends() []TVBackend {
+	return []TVBackend{
+		&riakBE, &pgBE,
+	}
+}
+
+func main() {
+	flag.Parse()
+	var fromSrv TVBackend
+	var toSrv TVBackend
+
+	toSrvUsed := !dump && !dry
+
+	if !validateType(fromType) {
+		log.Fatal("Unknown fromType " + fromType)
+	}
+	if toSrvUsed && !validateType(toType) {
+		log.Fatal("Unknown toType " + toType)
+	}
+
+	fromSrv = getBackendFromType(fromType)
+	if toSrvUsed {
+		toSrv = getBackendFromType(toType)
+	}
+
+	var toTimer time.Time
+	var toTime float64
+	var fromTimer time.Time
+	var fromTime float64
+
+	log.Println("Reading configs...")
+	fromTimer = time.Now()
+	if err := fromSrv.ReadConfig(fromCfg); err != nil {
+		log.Fatalf("Unable to read fromSrv cfg: %v", err)
+	}
+	fromTime = time.Now().Sub(fromTimer).Seconds()
+
+	if toSrvUsed {
+		toTimer = time.Now()
+		if err := toSrv.ReadConfig(toCfg); err != nil {
+			log.Fatalf("Unable to read toSrv cfg: %v", err)
+		}
+		toTime := time.Now().Sub(toTimer).Seconds()
+		log.Printf("Done [%v seconds]\n\tto: [%v seconds]\n\tfrom: [%v seconds]\n", toTime+fromTime, toTime, fromTime)
+	} else {
+		log.Printf("Done [%v seconds]\n", fromTime)
+	}
+
+	log.Println("Starting servers...")
+	fromTimer = time.Now()
+	if err := fromSrv.Start(); err != nil {
+		log.Fatalf("issue starting fromSrv: %v", err)
+	}
+	fromTime = time.Now().Sub(fromTimer).Seconds()
+	defer func() {
+		fromSrv.Stop()
+	}()
+	if toSrvUsed {
+		toTimer = time.Now()
+		if err := toSrv.Start(); err != nil {
+			log.Fatalf("issue starting toSrv: %v", err)
+		}
+		toTime = time.Now().Sub(toTimer).Seconds()
+		defer func() {
+			toSrv.Stop()
+		}()
+		log.Printf("Done [%v seconds]\n\tto: [%v seconds]\n\tfrom: [%v seconds]\n", toTime+fromTime, toTime, fromTime)
+	} else {
+		log.Printf("Done [%v seconds]\n", fromTime)
+	}
+
+	log.Println("Pinging servers...")
+	fromTimer = time.Now()
+	if err := fromSrv.Ping(); err != nil {
+		log.Fatalf("Unable to ping fromSrv: %v", err)
+	}
+	fromTime = time.Now().Sub(fromTimer).Seconds()
+	if toSrvUsed {
+		toTimer = time.Now()
+		if err := toSrv.Ping(); err != nil {
+			log.Fatalf("Unable to ping toSrv: %v", err)
+		}
+		toTime = time.Now().Sub(toTimer).Seconds()
+		log.Printf("Done [%v seconds]\n\tto: [%v seconds]\n\tfrom: [%v seconds]\n", toTime+fromTime, toTime, fromTime)
+	} else {
+		log.Printf("Done [%v seconds]\n", fromTime)
+	}
+
+	log.Printf("Fetching data from %v...\n", fromSrv.Name())
+	fromTimer = time.Now()
+	if err := fromSrv.Fetch(); err != nil {
+		log.Fatalf("Unable to fetch fromSrv data: %v", err)
+	}
+
+	fromSSLKeys, fromDNSSecKeys, fromURIKeys, fromURLKeys, err := GetKeys(fromSrv)
+	if err != nil {
+		log.Fatal(err)
+	}
+
+	if err := Validate(fromSrv); err != nil {
+		log.Fatal(err)
+	}
+	log.Printf("Done [%v seconds]\n", time.Now().Sub(fromTimer).Seconds())
+
+	if dump {
+		log.Printf("Dumping data from %v...\n", fromSrv.Name())
+		fromTimer = time.Now()
+		sslKeysBytes, err := json.Marshal(&fromSSLKeys)
+		if err != nil {
+			log.Fatal(err)
+		}
+		dnssecKeyBytes, err := json.Marshal(&fromDNSSecKeys)
+		if err != nil {
+			log.Fatal(err)
+		}
+		uriKeyBytes, err := json.Marshal(&fromURIKeys)
+		if err != nil {
+			log.Fatal(err)
+		}
+		urlKeyBytes, err := json.Marshal(&fromURLKeys)
+		if err != nil {
+			log.Fatal(err)
+		}
+
+		if err := os.Mkdir("dump", 0750); err != nil {
+			if !os.IsExist(err) {
+				log.Fatal(err)
+			}
+		}
+		if err = ioutil.WriteFile("dump/sslkeys.json", sslKeysBytes, 0640); err != nil {
+			log.Println(err)
+		}
+		if err = ioutil.WriteFile("dump/dnsseckeys.json", dnssecKeyBytes, 0640); err != nil {
+			log.Println(err)
+		}
+		if err = ioutil.WriteFile("dump/urlkeys.json", urlKeyBytes, 0640); err != nil {
+			log.Println(err)
+		}
+		if err = ioutil.WriteFile("dump/urikeys.json", uriKeyBytes, 0640); err != nil {
+			log.Println(err)
+		}
+		log.Printf("Done [%v seconds]\n", time.Now().Sub(fromTimer).Seconds())
+		return
+	}
+
+	if compare {
+		log.Printf("Fetching data from %v...\n", toSrv.Name())
+		toTimer = time.Now()
+		if err := toSrv.Fetch(); err != nil {
+			log.Fatalf("Unable to fetch toSrv data: %v\n", err)
+		}
+
+		toSSLKeys, toDNSSecKeys, toURIKeys, toURLKeys, err := GetKeys(toSrv)
+		if err != nil {
+			log.Fatal(err)
+		}
+		log.Println("Validating " + toSrv.Name())
+		if err := toSrv.ValidateKey(); err != nil && len(err) > 0 {
+			log.Fatal(strings.Join(err, "\n"))
+		}
+		log.Printf("Done [%v seconds]\n", time.Now().Sub(toTimer).Seconds())
+		log.Println(fromSrv.String())
+		log.Println(toSrv.String())
+
+		if !reflect.DeepEqual(fromSSLKeys, toSSLKeys) {
+			log.Fatal("from sslkeys and to sslkeys don't match")
+		}
+		if !reflect.DeepEqual(fromDNSSecKeys, toDNSSecKeys) {
+			log.Fatal("from dnssec and to dnssec don't match")
+		}
+		if !reflect.DeepEqual(fromURIKeys, toURIKeys) {
+			log.Fatal("from uri and to uri don't match")
+		}
+		if !reflect.DeepEqual(fromURLKeys, toURLKeys) {
+			log.Fatal("from url and to url don't match")
+		}
+		log.Println("Both datasources have the same keys!")
+		return
+	}
+
+	log.Printf("Setting %v keys...\n", toSrv.Name())
+	toTimer = time.Now()
+	if err := SetKeys(toSrv, fromSSLKeys, fromDNSSecKeys, fromURIKeys, fromURLKeys); err != nil {
+		log.Fatal(err)
+	}
+
+	if err := Validate(toSrv); err != nil {
+		log.Fatal(err)
+	}
+	log.Printf("Done [%v seconds]\n", time.Now().Sub(toTimer).Seconds())
+
+	log.Println(fromSrv.String())
+
+	if dry {
+		return
+	}
+
+	if confirm {
+		ans := "q"
+		for {
+			fmt.Print("Confirm data insertion (y/n): ")
+			if _, err := fmt.Scanln(&ans); err != nil {
+				log.Fatal("unable to get user input")
+			}
+
+			if ans == "y" {
+				break
+			} else if ans == "n" {
+				return
+			}
+		}
+	}
+	log.Printf("Inserting data into %v...\n", toSrv.Name())
+	toTimer = time.Now()
+	if err := toSrv.Insert(); err != nil {
+		log.Fatal(err)
+	}
+	log.Printf("Done [%v seconds]\n", time.Now().Sub(toTimer).Seconds())
+
+}
+
+// Validate runs the ValidateKey method on the backend
+func Validate(be TVBackend) error {
+	if errs := be.ValidateKey(); errs != nil && len(errs) > 0 {
+		return errors.New(fmt.Sprintf("Validation Errors (%v): \n%v", be.Name(), strings.Join(errs, "\n")))
+	} else {
+		log.Println("Validated " + be.Name())
+	}
+	return nil
+}
+
+// SetKeys will set all of the keys for a backend
+func SetKeys(be TVBackend, sslkeys []SSLKey, dnssecKeys []DNSSecKey, uriKeys []URISignKey, urlKeys []URLSigKey) error {
+	if err := be.SetSSLKeys(sslkeys); err != nil {
+		return errors.New(fmt.Sprintf("Unable to set %v ssl keys: %v", be.Name(), err))
+	}
+	if err := be.SetDNSSecKeys(dnssecKeys); err != nil {
+		return errors.New(fmt.Sprintf("Unable to set %v dnssec keys: %v", be.Name(), err))
+	}
+	if err := be.SetURLSigKeys(urlKeys); err != nil {
+		return errors.New(fmt.Sprintf("Unable to set %v url keys: %v", be.Name(), err))
+	}
+	if err := be.SetURISignKeys(uriKeys); err != nil {
+		return errors.New(fmt.Sprintf("Unable to set %v uri keys: %v", be.Name(), err))
+	}
+	return nil
+}
+
+// GetKeys will get all of the keys for a backend
+func GetKeys(be TVBackend) ([]SSLKey, []DNSSecKey, []URISignKey, []URLSigKey, error) {

Review comment:
       this function is a bit unwieldy with 5 outputs -- could we lump the keys altogether into a `type Secrets struct {SSLKeys, ...}`?

##########
File path: tools/traffic_vault_migrate/traffic_vault_migrate.go
##########
@@ -0,0 +1,448 @@
+package main
+
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+import (
+	"encoding/json"
+	"errors"
+	"flag"
+	"fmt"
+	"io/ioutil"
+	"log"
+	"os"
+	"reflect"
+	"sort"
+	"strings"
+	"time"
+
+	"github.com/apache/trafficcontrol/lib/go-tc"
+)
+
+var (
+	fromType string
+	toType   string
+	fromCfg  string
+	toCfg    string
+	dry      bool
+	compare  bool
+	confirm  bool
+	dump     bool
+
+	riakBE RiakBackend = RiakBackend{}
+	pgBE   PGBackend   = PGBackend{}
+)
+
+func init() {
+	flag.StringVar(&fromType, "from_type", riakBE.Name(), fmt.Sprintf("From server types (%v)", strings.Join(supportedTypes(), "|")))
+	flag.StringVar(&toType, "to_type", pgBE.Name(), fmt.Sprintf("From server types (%v)", strings.Join(supportedTypes(), "|")))
+	flag.StringVar(&toCfg, "to_cfg", "pg.json", "To server config file")
+	flag.StringVar(&fromCfg, "from_cfg", "riak.json", "From server config file")
+	flag.BoolVar(&dry, "dry", false, "Do not perform writes")
+	flag.BoolVar(&compare, "compare", false, "Compare to and from server records")
+	flag.BoolVar(&confirm, "confirm", true, "Requires confirmation before inserting records")
+	flag.BoolVar(&dump, "dump", false, "Write keys (from 'from' server) to disk")
+}
+
+func supportedBackends() []TVBackend {
+	return []TVBackend{
+		&riakBE, &pgBE,
+	}
+}
+
+func main() {
+	flag.Parse()
+	var fromSrv TVBackend
+	var toSrv TVBackend
+
+	toSrvUsed := !dump && !dry
+
+	if !validateType(fromType) {
+		log.Fatal("Unknown fromType " + fromType)
+	}
+	if toSrvUsed && !validateType(toType) {
+		log.Fatal("Unknown toType " + toType)
+	}
+
+	fromSrv = getBackendFromType(fromType)
+	if toSrvUsed {
+		toSrv = getBackendFromType(toType)
+	}
+
+	var toTimer time.Time
+	var toTime float64
+	var fromTimer time.Time
+	var fromTime float64
+
+	log.Println("Reading configs...")
+	fromTimer = time.Now()
+	if err := fromSrv.ReadConfig(fromCfg); err != nil {
+		log.Fatalf("Unable to read fromSrv cfg: %v", err)
+	}
+	fromTime = time.Now().Sub(fromTimer).Seconds()
+
+	if toSrvUsed {
+		toTimer = time.Now()
+		if err := toSrv.ReadConfig(toCfg); err != nil {
+			log.Fatalf("Unable to read toSrv cfg: %v", err)
+		}
+		toTime := time.Now().Sub(toTimer).Seconds()
+		log.Printf("Done [%v seconds]\n\tto: [%v seconds]\n\tfrom: [%v seconds]\n", toTime+fromTime, toTime, fromTime)
+	} else {
+		log.Printf("Done [%v seconds]\n", fromTime)
+	}
+
+	log.Println("Starting servers...")

Review comment:
       Should this be "starting server connections..." instead?

##########
File path: tools/traffic_vault_migrate/traffic_vault_migrate.go
##########
@@ -0,0 +1,448 @@
+package main
+
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+import (
+	"encoding/json"
+	"errors"
+	"flag"
+	"fmt"
+	"io/ioutil"
+	"log"
+	"os"
+	"reflect"
+	"sort"
+	"strings"
+	"time"
+
+	"github.com/apache/trafficcontrol/lib/go-tc"
+)
+
+var (
+	fromType string
+	toType   string
+	fromCfg  string
+	toCfg    string
+	dry      bool
+	compare  bool
+	confirm  bool
+	dump     bool
+
+	riakBE RiakBackend = RiakBackend{}
+	pgBE   PGBackend   = PGBackend{}
+)
+
+func init() {
+	flag.StringVar(&fromType, "from_type", riakBE.Name(), fmt.Sprintf("From server types (%v)", strings.Join(supportedTypes(), "|")))
+	flag.StringVar(&toType, "to_type", pgBE.Name(), fmt.Sprintf("From server types (%v)", strings.Join(supportedTypes(), "|")))
+	flag.StringVar(&toCfg, "to_cfg", "pg.json", "To server config file")
+	flag.StringVar(&fromCfg, "from_cfg", "riak.json", "From server config file")
+	flag.BoolVar(&dry, "dry", false, "Do not perform writes")
+	flag.BoolVar(&compare, "compare", false, "Compare to and from server records")
+	flag.BoolVar(&confirm, "confirm", true, "Requires confirmation before inserting records")
+	flag.BoolVar(&dump, "dump", false, "Write keys (from 'from' server) to disk")
+}
+
+func supportedBackends() []TVBackend {
+	return []TVBackend{
+		&riakBE, &pgBE,
+	}
+}
+
+func main() {
+	flag.Parse()
+	var fromSrv TVBackend
+	var toSrv TVBackend
+
+	toSrvUsed := !dump && !dry
+
+	if !validateType(fromType) {
+		log.Fatal("Unknown fromType " + fromType)
+	}
+	if toSrvUsed && !validateType(toType) {
+		log.Fatal("Unknown toType " + toType)
+	}
+
+	fromSrv = getBackendFromType(fromType)
+	if toSrvUsed {
+		toSrv = getBackendFromType(toType)
+	}
+
+	var toTimer time.Time
+	var toTime float64
+	var fromTimer time.Time
+	var fromTime float64
+
+	log.Println("Reading configs...")
+	fromTimer = time.Now()
+	if err := fromSrv.ReadConfig(fromCfg); err != nil {
+		log.Fatalf("Unable to read fromSrv cfg: %v", err)
+	}
+	fromTime = time.Now().Sub(fromTimer).Seconds()
+
+	if toSrvUsed {
+		toTimer = time.Now()
+		if err := toSrv.ReadConfig(toCfg); err != nil {
+			log.Fatalf("Unable to read toSrv cfg: %v", err)
+		}
+		toTime := time.Now().Sub(toTimer).Seconds()

Review comment:
       All this timing code is a bit distracting -- is it really necessary to know how long each step takes? Also, if we really needed to know, we should be able to use the log timestamps from "starting ..." to the next "starting..." message, right?




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [trafficcontrol] shamrickus commented on a change in pull request #5924: Add tool to migrate data between TV backends

Posted by GitBox <gi...@apache.org>.
shamrickus commented on a change in pull request #5924:
URL: https://github.com/apache/trafficcontrol/pull/5924#discussion_r651971280



##########
File path: tools/traffic_vault_migrate/postgres.go
##########
@@ -0,0 +1,743 @@
+package main
+
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+import (
+	"database/sql"
+	"encoding/base64"
+	"encoding/json"
+	"errors"
+	"fmt"
+	"log"
+	"strconv"
+	"strings"
+
+	"github.com/apache/trafficcontrol/lib/go-tc"
+	util "github.com/apache/trafficcontrol/lib/go-util"
+
+	_ "github.com/lib/pq"
+)
+
+// PGConfig represents the configuration options available to the PG backend
+type PGConfig struct {
+	Host     string `json:"host"`
+	Port     string `json:"port"`
+	User     string `json:"user"`
+	Password string `json:"password"`
+	SSLMode  string `json:"sslmode"`
+	Database string `json:"database"`
+	Key      string `json:"aesKey"`
+	AESKey   []byte
+}
+
+// PGBackend is the Postgres implementation of TVBackend
+type PGBackend struct {
+	sslKey pgSSLKeyTable
+	dnssec pgDNSSecTable
+	uri    pgURISignKeyTable
+	url    pgURLSigKeyTable
+	cfg    PGConfig
+	db     *sql.DB
+}
+
+// String returns a high level overview of the backend and its keys
+func (pg *PGBackend) String() string {
+	data := fmt.Sprintf("PG server %v@%v:%v\n", pg.cfg.User, pg.cfg.Host, pg.cfg.Port)
+	data += fmt.Sprintf("\tSSL Keys: %v\n", len(pg.sslKey.Records))
+	data += fmt.Sprintf("\tDNSSec Keys: %v\n", len(pg.dnssec.Records))
+	data += fmt.Sprintf("\tURI Keys: %v\n", len(pg.uri.Records))
+	data += fmt.Sprintf("\tURL Keys: %v\n", len(pg.url.Records))
+	return data
+}
+
+// Name returns the name for this backend
+func (pg *PGBackend) Name() string {
+	return "PG"
+}
+
+// ReadConfig takes in a filename and will read it into the backends config
+func (pg *PGBackend) ReadConfig(s string) error {
+	err := UnmarshalConfig(s, &pg.cfg)
+	if err != nil {
+		return err
+	}
+
+	pg.cfg.AESKey, err = base64.StdEncoding.DecodeString(pg.cfg.Key)
+	if err != nil {
+		return fmt.Errorf("unable to decode PG AESKey: %w", err)
+	}
+	return nil
+}
+
+// Insert takes the current keys and inserts them into the backend DB
+func (pg *PGBackend) Insert() error {
+	if err := pg.sslKey.insertKeys(pg.db); err != nil {
+		return err
+	}
+	if err := pg.dnssec.insertKeys(pg.db); err != nil {
+		return err
+	}
+	if err := pg.url.insertKeys(pg.db); err != nil {
+		return err
+	}
+	if err := pg.uri.insertKeys(pg.db); err != nil {
+		return err
+	}
+	return nil
+}
+
+// Start initiates the connection to the backend DB
+func (pg *PGBackend) Start() error {
+	sqlStr := fmt.Sprintf("postgres://%s:%s@%s:%s/%s?sslmode=%s", pg.cfg.User, pg.cfg.Password, pg.cfg.Host, pg.cfg.Port, pg.cfg.Database, pg.cfg.SSLMode)
+	db, err := sql.Open("postgres", sqlStr)
+	if err != nil {
+		return fmt.Errorf("unable to start PG client: %w", err)
+	}
+
+	pg.db = db
+	pg.sslKey = pgSSLKeyTable{}
+	pg.dnssec = pgDNSSecTable{}
+	pg.url = pgURLSigKeyTable{}
+	pg.uri = pgURISignKeyTable{}
+
+	return nil
+}
+
+// ValidateKey validates that the keys are valid (in most cases, certain fields are not null)
+func (pg *PGBackend) ValidateKey() []string {
+	var errors []string
+	if errs := pg.sslKey.validate(); errs != nil {
+		errors = append(errors, errs...)
+	}
+	if errs := pg.dnssec.validate(); errs != nil {
+		errors = append(errors, errs...)
+	}
+	if errs := pg.uri.validate(); errs != nil {
+		errors = append(errors, errs...)
+	}
+	if errs := pg.url.validate(); errs != nil {
+		errors = append(errors, errs...)
+	}
+	return errors
+}
+
+// Close terminates the connection to the backend DB
+func (pg *PGBackend) Close() error {
+	return pg.db.Close()
+}
+
+// Ping checks the connection to the backend DB
+func (pg *PGBackend) Ping() error {
+	return pg.db.Ping()
+}
+
+// Fetch gets all of the keys from the backend DB
+func (pg *PGBackend) Fetch() error {
+	if err := pg.sslKey.gatherKeys(pg.db); err != nil {
+		return err
+	}
+
+	if err := pg.dnssec.gatherKeys(pg.db); err != nil {
+		return err
+	}
+
+	if err := pg.url.gatherKeys(pg.db); err != nil {
+		return err
+	}
+
+	if err := pg.uri.gatherKeys(pg.db); err != nil {
+		return err
+	}
+
+	return nil
+}
+
+// GetSSLKeys converts the backends internal key representation into the common representation (SSLKey)
+func (pg *PGBackend) GetSSLKeys() ([]SSLKey, error) {
+	if err := pg.sslKey.decrypt(pg.cfg.AESKey); err != nil {
+		return nil, err
+	}
+	return pg.sslKey.toGeneric(), nil
+}
+
+// SetSSLKeys takes in keys and converts & encrypts the data into the backends internal format
+func (pg *PGBackend) SetSSLKeys(keys []SSLKey) error {
+	pg.sslKey.fromGeneric(keys)
+	return pg.sslKey.encrypt(pg.cfg.AESKey)
+}
+
+// GetDNSSecKeys converts the backends internal key representation into the common representation (DNSSecKey)
+func (pg *PGBackend) GetDNSSecKeys() ([]DNSSecKey, error) {
+	if err := pg.dnssec.decrypt(pg.cfg.AESKey); err != nil {
+		return nil, err
+	}
+	return pg.dnssec.toGeneric(), nil
+}
+
+// SetDNSSecKeys takes in keys and converts & encrypts the data into the backends internal format
+func (pg *PGBackend) SetDNSSecKeys(keys []DNSSecKey) error {
+	pg.dnssec.fromGeneric(keys)
+	return pg.dnssec.encrypt(pg.cfg.AESKey)
+}
+
+// GetURISignKeys converts the pg internal key representation into the common representation (URISignKey)
+func (pg *PGBackend) GetURISignKeys() ([]URISignKey, error) {
+	if err := pg.uri.decrypt(pg.cfg.AESKey); err != nil {
+		return nil, err
+	}
+	return pg.uri.toGeneric(), nil
+}
+
+// SetURISignKeys takes in keys and converts & encrypts the data into the backends internal format
+func (pg *PGBackend) SetURISignKeys(keys []URISignKey) error {
+	pg.uri.fromGeneric(keys)
+	return pg.uri.encrypt(pg.cfg.AESKey)
+}
+
+// GetURLSigKeys converts the backends internal key representation into the common representation (URLSigKey)
+func (pg *PGBackend) GetURLSigKeys() ([]URLSigKey, error) {
+	if err := pg.url.decrypt(pg.cfg.AESKey); err != nil {
+		return nil, err
+	}
+	return pg.url.toGeneric(), nil
+}
+
+// SetURLSigKeys takes in keys and converts & encrypts the data into the backends internal format
+func (pg *PGBackend) SetURLSigKeys(keys []URLSigKey) error {
+	pg.url.fromGeneric(keys)
+	return pg.url.encrypt(pg.cfg.AESKey)
+}
+
+type pgCommonRecord struct {
+	DataEncrypted []byte
+}
+
+type pgDNSSecRecord struct {
+	Key tc.DNSSECKeysTrafficVault
+	CDN string
+	pgCommonRecord
+}
+type pgDNSSecTable struct {
+	Records []pgDNSSecRecord
+}
+
+func (tbl *pgDNSSecTable) gatherKeys(db *sql.DB) error {
+	sz, err := getSize(db, "dnssec")
+	if err != nil {
+		log.Println("PGDNSSec gatherKeys: unable to determine size of dnssec table")
+	}
+	tbl.Records = make([]pgDNSSecRecord, sz)
+
+	rows, err := db.Query("SELECT cdn, data from dnssec")
+	if err != nil {
+		return fmt.Errorf("PGDNSSec gatherKeys: unable to query: %w", err)
+	}
+	defer rows.Close()
+	i := 0
+	for rows.Next() {
+		if i > len(tbl.Records)-1 {
+			return fmt.Errorf("PGDNSSec gatherKeys got more results than expected %v", len(tbl.Records))
+		}
+		err := rows.Scan(&tbl.Records[i].CDN, &tbl.Records[i].DataEncrypted)
+		if err != nil {
+			return fmt.Errorf("PGDNSSec gatherKeys unable to scan row: %w", err)
+		}
+		i += 1
+	}
+	return nil
+}
+func (tbl *pgDNSSecTable) decrypt(aesKey []byte) error {
+	for i, _ := range tbl.Records {
+		err := decryptInto(aesKey, tbl.Records[i].DataEncrypted, &tbl.Records[i].Key)
+		if err != nil {
+			return fmt.Errorf("unable to decrypt into keys: %w", err)
+		}
+	}
+	return nil
+}
+func (tbl *pgDNSSecTable) encrypt(aesKey []byte) error {
+	for i, dns := range tbl.Records {
+		data, err := json.Marshal(&dns.Key)
+		if err != nil {
+			return fmt.Errorf("encrypt issue marshalling keys: %w", err)
+		}
+		dat, err := encrypt(data, aesKey)
+		if err != nil {
+			return fmt.Errorf("encrypt error: %w", err)
+		}
+		tbl.Records[i].DataEncrypted = dat
+	}
+	return nil
+}
+func (tbl *pgDNSSecTable) toGeneric() []DNSSecKey {
+	keys := make([]DNSSecKey, len(tbl.Records))
+
+	for i, record := range tbl.Records {
+		keys[i] = DNSSecKey{
+			CDN:                    record.CDN,
+			DNSSECKeysTrafficVault: record.Key,
+		}
+	}
+
+	return keys
+}
+func (tbl *pgDNSSecTable) fromGeneric(keys []DNSSecKey) {
+	tbl.Records = make([]pgDNSSecRecord, len(keys))
+
+	for i, key := range keys {
+		tbl.Records[i] = pgDNSSecRecord{
+			Key: key.DNSSECKeysTrafficVault,
+			CDN: key.CDN,
+			pgCommonRecord: pgCommonRecord{
+				DataEncrypted: nil,
+			},
+		}
+	}
+}
+func (tbl *pgDNSSecTable) validate() []string {
+	for i, record := range tbl.Records {
+		if record.DataEncrypted == nil && len(record.Key) > 0 {
+			return []string{fmt.Sprintf("DNSSEC Key %v: DataEncrypted is blank!", i)}
+		}
+	}
+	return nil
+}
+func (tbl *pgDNSSecTable) insertKeys(db *sql.DB) error {
+	queryBase := "INSERT INTO dnssec (cdn, data) VALUES "
+	stride := 2
+	queryArgs := make([]interface{}, len(tbl.Records)*stride)
+	for i, record := range tbl.Records {
+		j := i * stride
+		queryArgs[j] = record.CDN
+		queryArgs[j+1] = record.DataEncrypted
+	}
+	return insertIntoTable(db, queryBase, stride, queryArgs)
+}
+
+type pgSSLKeyRecord struct {
+	Keys tc.DeliveryServiceSSLKeys
+	pgCommonRecord
+
+	// These records are stored on the table but are duplicated
+	DeliveryService string
+	CDN             string
+}
+type pgSSLKeyTable struct {
+	Records []pgSSLKeyRecord
+}
+
+func (tbl *pgSSLKeyTable) insertKeys(db *sql.DB) error {
+	queryBase := "INSERT INTO sslkey (deliveryservice, data, cdn, version) VALUES "
+	duplicateKeys := 2
+	stride := 4
+	queryArgs := make([]interface{}, len(tbl.Records)*stride*duplicateKeys)
+	for i, record := range tbl.Records {
+		j := i * duplicateKeys * stride
+
+		queryArgs[j] = record.DeliveryService
+		queryArgs[j+1] = record.DataEncrypted
+		queryArgs[j+2] = record.CDN
+		queryArgs[j+3] = record.Keys.Version.String()
+
+		queryArgs[j+4] = record.DeliveryService
+		queryArgs[j+5] = record.DataEncrypted
+		queryArgs[j+6] = record.CDN
+		queryArgs[j+7] = "latest"
+	}
+	return insertIntoTable(db, queryBase, 4, queryArgs)
+}
+func (tbl *pgSSLKeyTable) gatherKeys(db *sql.DB) error {
+	sz, err := getSize(db, "sslkey WHERE version='latest'")
+	if err != nil {
+		return fmt.Errorf("PGSSLKey gatherKeys unable to determine size of sslkey table: %w", err)
+	}
+	tbl.Records = make([]pgSSLKeyRecord, sz)
+
+	rows, err := db.Query("SELECT data, deliveryservice, cdn from sslkey WHERE version = 'latest'")
+	if err != nil {
+		return fmt.Errorf("PGSSLKey gatherKeys unable to query: %w", err)
+	}
+	defer rows.Close()
+	i := 0
+	for rows.Next() {
+		if i > len(tbl.Records)-1 {
+			return errors.New("PGSSLKey gatherKeys: got more results than expected")
+		}
+		err := rows.Scan(&tbl.Records[i].DataEncrypted, &tbl.Records[i].DeliveryService, &tbl.Records[i].CDN)
+		if err != nil {
+			return fmt.Errorf("PGSSLKey gatherKeys unable to scan row: %w", err)
+		}
+		i += 1
+	}
+	return nil
+}
+func (tbl *pgSSLKeyTable) decrypt(aesKey []byte) error {
+	for i, dns := range tbl.Records {
+		err := decryptInto(aesKey, dns.DataEncrypted, &tbl.Records[i].Keys)
+		if err != nil {
+			return fmt.Errorf("unable to decrypt into keys: %w", err)
+		}
+	}
+	return nil
+}
+func (tbl *pgSSLKeyTable) encrypt(aesKey []byte) error {
+	for i, dns := range tbl.Records {
+		data, err := json.Marshal(dns.Keys)
+		if err != nil {
+			return fmt.Errorf("encrypt issue marshalling keys: %w", err)
+		}
+		dat, err := encrypt(data, aesKey)
+		if err != nil {
+			return fmt.Errorf("encrypt error: %w", err)
+		}
+		tbl.Records[i].DataEncrypted = dat
+	}
+	return nil
+}
+func (tbl *pgSSLKeyTable) toGeneric() []SSLKey {
+	keys := make([]SSLKey, len(tbl.Records))
+
+	for i, record := range tbl.Records {
+		keys[i] = SSLKey{
+			DeliveryServiceSSLKeys: record.Keys,
+		}
+	}
+	return keys
+}
+func (tbl *pgSSLKeyTable) fromGeneric(keys []SSLKey) {
+	tbl.Records = make([]pgSSLKeyRecord, len(keys))
+
+	for i, key := range keys {
+		tbl.Records[i] = pgSSLKeyRecord{
+			Keys: key.DeliveryServiceSSLKeys,
+			pgCommonRecord: pgCommonRecord{
+				DataEncrypted: nil,
+			},
+			DeliveryService: key.DeliveryService,
+			CDN:             key.CDN,
+		}
+	}
+}
+func (tbl *pgSSLKeyTable) validate() []string {
+	defaultKey := tc.DeliveryServiceSSLKeys{}
+	var errors []string
+	fmtStr := "SSL Key %v: %v"
+	for i, record := range tbl.Records {
+		if record.Keys == defaultKey {
+			errors = append(errors, fmt.Sprintf(fmtStr, i, "DS SSL Keys are default!"))
+		} else if record.Keys.Key == "" {
+			errors = append(errors, fmt.Sprintf(fmtStr, i, "Key is blank!"))
+		} else if record.Keys.CDN == "" {
+			errors = append(errors, fmt.Sprintf(fmtStr, i, "CDN is blank!"))
+		} else if record.Keys.DeliveryService == "" {
+			errors = append(errors, fmt.Sprintf(fmtStr, i, "DS is blank!"))
+		} else if record.DataEncrypted == nil {
+			errors = append(errors, fmt.Sprintf(fmtStr, i, "DataEncrypted is blank!"))
+		} else if record.Keys.Version.String() == "" {
+			errors = append(errors, fmt.Sprintf(fmtStr, i, "Version is blank!"))
+		}
+	}
+	return errors
+}
+
+type pgURLSigKeyRecord struct {
+	Keys            tc.URLSigKeys
+	DeliveryService string
+	pgCommonRecord
+}
+type pgURLSigKeyTable struct {
+	Records []pgURLSigKeyRecord
+}
+
+func (tbl *pgURLSigKeyTable) insertKeys(db *sql.DB) error {
+	queryBase := "INSERT INTO url_sig_key (deliveryservice, data) VALUES "
+	stride := 2
+	queryArgs := make([]interface{}, len(tbl.Records)*stride)
+	for i, record := range tbl.Records {
+		j := i * stride
+		queryArgs[j] = record.DeliveryService
+		queryArgs[j+1] = record.DataEncrypted
+	}
+	return insertIntoTable(db, queryBase, stride, queryArgs)
+}
+func (tbl *pgURLSigKeyTable) gatherKeys(db *sql.DB) error {
+	sz, err := getSize(db, "url_sig_key")
+	if err != nil {
+		log.Println("PGURLSigKey gatherKeys: unable to determine url_sig_key table size")
+	}
+	tbl.Records = make([]pgURLSigKeyRecord, sz)
+
+	rows, err := db.Query("SELECT deliveryservice, data from url_sig_key")
+	if err != nil {
+		return fmt.Errorf("PGURLSigKey gatherKeys error creating query: %w", err)
+	}
+	defer rows.Close()
+	i := 0
+	for rows.Next() {
+		if i > len(tbl.Records)-1 {
+			return fmt.Errorf("PGURLSigKey gatherKeys: got more results than expected %v", len(tbl.Records))
+		}
+		err := rows.Scan(&tbl.Records[i].DeliveryService, &tbl.Records[i].DataEncrypted)
+		if err != nil {
+			return fmt.Errorf("PGURLSigKey gatherKeys: unable to scan row: %w", err)
+		}
+		i += 1
+	}
+	return nil
+}
+func (tbl *pgURLSigKeyTable) decrypt(aesKey []byte) error {
+	for i, sig := range tbl.Records {
+		err := decryptInto(aesKey, sig.DataEncrypted, &tbl.Records[i].Keys)
+		if err != nil {
+			return fmt.Errorf("unable to decrypt into keys: %w", err)
+		}
+	}
+	return nil
+}
+func (tbl *pgURLSigKeyTable) encrypt(aesKey []byte) error {
+	for i, sig := range tbl.Records {
+		data, err := json.Marshal(&sig.Keys)
+		if err != nil {
+			return fmt.Errorf("encrypt issue marshalling keys: %w", err)
+		}
+
+		dat, err := encrypt(data, aesKey)
+		if err != nil {
+			return fmt.Errorf("encrypt error: %w", err)
+		}
+		tbl.Records[i].DataEncrypted = dat
+	}
+	return nil
+}
+func (tbl *pgURLSigKeyTable) toGeneric() []URLSigKey {
+	keys := make([]URLSigKey, len(tbl.Records))
+
+	for i, record := range tbl.Records {
+		keys[i] = URLSigKey{
+			DeliveryService: record.DeliveryService,
+			URLSigKeys:      record.Keys,
+		}
+	}
+	return keys
+}
+func (tbl *pgURLSigKeyTable) fromGeneric(keys []URLSigKey) {
+	tbl.Records = make([]pgURLSigKeyRecord, len(keys))
+
+	for i, key := range keys {
+		tbl.Records[i] = pgURLSigKeyRecord{
+			Keys:            key.URLSigKeys,
+			DeliveryService: key.DeliveryService,
+			pgCommonRecord: pgCommonRecord{
+				DataEncrypted: nil,
+			},
+		}
+	}
+}
+func (tbl *pgURLSigKeyTable) validate() []string {
+	for i, record := range tbl.Records {
+		if record.DataEncrypted == nil && len(record.Keys) > 0 {
+			return []string{fmt.Sprintf("URl Sig Key %v: DataEncrypted is blank!", i)}
+		}
+	}
+	return nil
+}
+
+type pgURISignKeyRecord struct {
+	Keys            map[string]tc.URISignerKeyset
+	DeliveryService string
+	pgCommonRecord
+}
+type pgURISignKeyTable struct {
+	Records []pgURISignKeyRecord
+}
+
+func (tbl *pgURISignKeyTable) insertKeys(db *sql.DB) error {
+	queryBase := "INSERT INTO uri_signing_key (deliveryservice, data) VALUES "

Review comment:
       `ON CONFLICT` doesn't work because the only constrains on the sslkey table are on the `id` column. Regardless, I'll handle it for sslkeys as well. 




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [trafficcontrol] rawlinp commented on a change in pull request #5924: Add tool to migrate data between TV backends

Posted by GitBox <gi...@apache.org>.
rawlinp commented on a change in pull request #5924:
URL: https://github.com/apache/trafficcontrol/pull/5924#discussion_r652088443



##########
File path: lib/go-tc/deliveryservice_ssl_keys.go
##########
@@ -73,6 +73,7 @@ type SSLKeyRequestFields struct {
 	HostName     *string `json:"hostname,omitempty"`
 	Country      *string `json:"country,omitempty"`
 	State        *string `json:"state,omitempty"`
+	Version      *int    `json:"version,omitempty"`

Review comment:
       I believe we are free to add new fields to existing structs -- that is a non-breaking change. 




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [trafficcontrol] ocket8888 commented on a change in pull request #5924: Add tool to migrate data between TV backends

Posted by GitBox <gi...@apache.org>.
ocket8888 commented on a change in pull request #5924:
URL: https://github.com/apache/trafficcontrol/pull/5924#discussion_r652104151



##########
File path: lib/go-tc/deliveryservice_ssl_keys.go
##########
@@ -73,6 +73,7 @@ type SSLKeyRequestFields struct {
 	HostName     *string `json:"hostname,omitempty"`
 	Country      *string `json:"country,omitempty"`
 	State        *string `json:"state,omitempty"`
+	Version      *int    `json:"version,omitempty"`

Review comment:
       > I believe we are free to add new fields to existing structs -- that is a non-breaking change.
   
   Not only that, but it looks like the "new" field was used in older API versions already, the only difference is that the Go client used to hard-code it. So this is barely a **non**-breaking change.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [trafficcontrol] ocket8888 commented on a change in pull request #5924: Add tool to migrate data between TV backends

Posted by GitBox <gi...@apache.org>.
ocket8888 commented on a change in pull request #5924:
URL: https://github.com/apache/trafficcontrol/pull/5924#discussion_r648673854



##########
File path: tools/traffic_vault_migrate/postgres.go
##########
@@ -0,0 +1,716 @@
+package main
+
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+import (
+	"database/sql"
+	"encoding/base64"
+	"encoding/json"
+	"errors"
+	"fmt"
+	"github.com/apache/trafficcontrol/lib/go-tc"
+	util "github.com/apache/trafficcontrol/lib/go-util"
+	_ "github.com/lib/pq"
+	"log"
+	"reflect"
+	"strconv"
+	"strings"
+)
+
+type PGConfig struct {
+	Host     string `json:"host"`
+	Port     string `json:"port"`
+	User     string `json:"user"`
+	Password string `json:"password"`
+	SSLMode  string `json:"sslmode"`
+	Database string `json:"database"`
+	Key      string `json:"aesKey"`
+	AESKey   []byte
+}
+type PGBackend struct {
+	sslKey PGSSLKeyTable
+	dnssec PGDNSSecTable
+	uri    PGURISignKeyTable
+	url    PGURLSigKeyTable
+	cfg    PGConfig
+	db     *sql.DB
+}
+
+func (pg *PGBackend) String() string {

Review comment:
       I don't care about the pointer receiver thing, I meant to take that out of the comment. For a second I was like "well what if it's null?" - if that's true, then `nil.String()` is a segfault - that receiver type should never matter unless values are written.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [trafficcontrol] shamrickus commented on a change in pull request #5924: Add tool to migrate data between TV backends

Posted by GitBox <gi...@apache.org>.
shamrickus commented on a change in pull request #5924:
URL: https://github.com/apache/trafficcontrol/pull/5924#discussion_r652105472



##########
File path: tools/traffic_vault_migrate/traffic_vault_migrate.go
##########
@@ -400,6 +356,49 @@ type TVBackend interface {
 	SetURLSigKeys([]URLSigKey) error
 }
 
+// Secrets contains every key to be migrated
+type Secrets struct {
+	sslkeys    []SSLKey
+	dnssecKeys []DNSSecKey
+	uriKeys    []URISignKey
+	urlKeys    []URLSigKey
+}
+
+func (s *Secrets) sort() {
+	sort.Slice(s.sslkeys[:], func(a, b int) bool {
+		return s.sslkeys[a].CDN < s.sslkeys[b].CDN ||
+			s.sslkeys[a].CDN == s.sslkeys[b].CDN && s.sslkeys[a].DeliveryService < s.sslkeys[b].DeliveryService
+	})
+	sort.Slice(s.dnssecKeys[:], func(a, b int) bool {
+		return s.dnssecKeys[a].CDN < s.dnssecKeys[b].CDN
+	})
+	sort.Slice(s.uriKeys[:], func(a, b int) bool {
+		return s.uriKeys[a].DeliveryService < s.uriKeys[b].DeliveryService
+	})
+	sort.Slice(s.urlKeys[:], func(a, b int) bool {
+		return s.urlKeys[a].DeliveryService < s.urlKeys[b].DeliveryService
+	})
+}
+func (s *Secrets) dump(directory string) {

Review comment:
       Added an option `fill` to import data.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [trafficcontrol] rawlinp commented on a change in pull request #5924: Add tool to migrate data between TV backends

Posted by GitBox <gi...@apache.org>.
rawlinp commented on a change in pull request #5924:
URL: https://github.com/apache/trafficcontrol/pull/5924#discussion_r651952844



##########
File path: tools/traffic_vault_migrate/postgres.go
##########
@@ -0,0 +1,743 @@
+package main
+
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+import (
+	"database/sql"
+	"encoding/base64"
+	"encoding/json"
+	"errors"
+	"fmt"
+	"log"
+	"strconv"
+	"strings"
+
+	"github.com/apache/trafficcontrol/lib/go-tc"
+	util "github.com/apache/trafficcontrol/lib/go-util"
+
+	_ "github.com/lib/pq"
+)
+
+// PGConfig represents the configuration options available to the PG backend
+type PGConfig struct {
+	Host     string `json:"host"`
+	Port     string `json:"port"`
+	User     string `json:"user"`
+	Password string `json:"password"`
+	SSLMode  string `json:"sslmode"`
+	Database string `json:"database"`
+	Key      string `json:"aesKey"`
+	AESKey   []byte
+}
+
+// PGBackend is the Postgres implementation of TVBackend
+type PGBackend struct {
+	sslKey pgSSLKeyTable
+	dnssec pgDNSSecTable
+	uri    pgURISignKeyTable
+	url    pgURLSigKeyTable
+	cfg    PGConfig
+	db     *sql.DB
+}
+
+// String returns a high level overview of the backend and its keys
+func (pg *PGBackend) String() string {
+	data := fmt.Sprintf("PG server %v@%v:%v\n", pg.cfg.User, pg.cfg.Host, pg.cfg.Port)
+	data += fmt.Sprintf("\tSSL Keys: %v\n", len(pg.sslKey.Records))
+	data += fmt.Sprintf("\tDNSSec Keys: %v\n", len(pg.dnssec.Records))
+	data += fmt.Sprintf("\tURI Keys: %v\n", len(pg.uri.Records))
+	data += fmt.Sprintf("\tURL Keys: %v\n", len(pg.url.Records))
+	return data
+}
+
+// Name returns the name for this backend
+func (pg *PGBackend) Name() string {
+	return "PG"
+}
+
+// ReadConfig takes in a filename and will read it into the backends config
+func (pg *PGBackend) ReadConfig(s string) error {
+	err := UnmarshalConfig(s, &pg.cfg)
+	if err != nil {
+		return err
+	}
+
+	pg.cfg.AESKey, err = base64.StdEncoding.DecodeString(pg.cfg.Key)
+	if err != nil {
+		return fmt.Errorf("unable to decode PG AESKey: %w", err)
+	}
+	return nil
+}
+
+// Insert takes the current keys and inserts them into the backend DB
+func (pg *PGBackend) Insert() error {
+	if err := pg.sslKey.insertKeys(pg.db); err != nil {
+		return err
+	}
+	if err := pg.dnssec.insertKeys(pg.db); err != nil {
+		return err
+	}
+	if err := pg.url.insertKeys(pg.db); err != nil {
+		return err
+	}
+	if err := pg.uri.insertKeys(pg.db); err != nil {
+		return err
+	}
+	return nil
+}
+
+// Start initiates the connection to the backend DB
+func (pg *PGBackend) Start() error {
+	sqlStr := fmt.Sprintf("postgres://%s:%s@%s:%s/%s?sslmode=%s", pg.cfg.User, pg.cfg.Password, pg.cfg.Host, pg.cfg.Port, pg.cfg.Database, pg.cfg.SSLMode)
+	db, err := sql.Open("postgres", sqlStr)
+	if err != nil {
+		return fmt.Errorf("unable to start PG client: %w", err)
+	}
+
+	pg.db = db
+	pg.sslKey = pgSSLKeyTable{}
+	pg.dnssec = pgDNSSecTable{}
+	pg.url = pgURLSigKeyTable{}
+	pg.uri = pgURISignKeyTable{}
+
+	return nil
+}
+
+// ValidateKey validates that the keys are valid (in most cases, certain fields are not null)
+func (pg *PGBackend) ValidateKey() []string {
+	var errors []string
+	if errs := pg.sslKey.validate(); errs != nil {
+		errors = append(errors, errs...)
+	}
+	if errs := pg.dnssec.validate(); errs != nil {
+		errors = append(errors, errs...)
+	}
+	if errs := pg.uri.validate(); errs != nil {
+		errors = append(errors, errs...)
+	}
+	if errs := pg.url.validate(); errs != nil {
+		errors = append(errors, errs...)
+	}
+	return errors
+}
+
+// Close terminates the connection to the backend DB
+func (pg *PGBackend) Close() error {
+	return pg.db.Close()
+}
+
+// Ping checks the connection to the backend DB
+func (pg *PGBackend) Ping() error {
+	return pg.db.Ping()
+}
+
+// Fetch gets all of the keys from the backend DB
+func (pg *PGBackend) Fetch() error {
+	if err := pg.sslKey.gatherKeys(pg.db); err != nil {
+		return err
+	}
+
+	if err := pg.dnssec.gatherKeys(pg.db); err != nil {
+		return err
+	}
+
+	if err := pg.url.gatherKeys(pg.db); err != nil {
+		return err
+	}
+
+	if err := pg.uri.gatherKeys(pg.db); err != nil {
+		return err
+	}
+
+	return nil
+}
+
+// GetSSLKeys converts the backends internal key representation into the common representation (SSLKey)
+func (pg *PGBackend) GetSSLKeys() ([]SSLKey, error) {
+	if err := pg.sslKey.decrypt(pg.cfg.AESKey); err != nil {
+		return nil, err
+	}
+	return pg.sslKey.toGeneric(), nil
+}
+
+// SetSSLKeys takes in keys and converts & encrypts the data into the backends internal format
+func (pg *PGBackend) SetSSLKeys(keys []SSLKey) error {
+	pg.sslKey.fromGeneric(keys)
+	return pg.sslKey.encrypt(pg.cfg.AESKey)
+}
+
+// GetDNSSecKeys converts the backends internal key representation into the common representation (DNSSecKey)
+func (pg *PGBackend) GetDNSSecKeys() ([]DNSSecKey, error) {
+	if err := pg.dnssec.decrypt(pg.cfg.AESKey); err != nil {
+		return nil, err
+	}
+	return pg.dnssec.toGeneric(), nil
+}
+
+// SetDNSSecKeys takes in keys and converts & encrypts the data into the backends internal format
+func (pg *PGBackend) SetDNSSecKeys(keys []DNSSecKey) error {
+	pg.dnssec.fromGeneric(keys)
+	return pg.dnssec.encrypt(pg.cfg.AESKey)
+}
+
+// GetURISignKeys converts the pg internal key representation into the common representation (URISignKey)
+func (pg *PGBackend) GetURISignKeys() ([]URISignKey, error) {
+	if err := pg.uri.decrypt(pg.cfg.AESKey); err != nil {
+		return nil, err
+	}
+	return pg.uri.toGeneric(), nil
+}
+
+// SetURISignKeys takes in keys and converts & encrypts the data into the backends internal format
+func (pg *PGBackend) SetURISignKeys(keys []URISignKey) error {
+	pg.uri.fromGeneric(keys)
+	return pg.uri.encrypt(pg.cfg.AESKey)
+}
+
+// GetURLSigKeys converts the backends internal key representation into the common representation (URLSigKey)
+func (pg *PGBackend) GetURLSigKeys() ([]URLSigKey, error) {
+	if err := pg.url.decrypt(pg.cfg.AESKey); err != nil {
+		return nil, err
+	}
+	return pg.url.toGeneric(), nil
+}
+
+// SetURLSigKeys takes in keys and converts & encrypts the data into the backends internal format
+func (pg *PGBackend) SetURLSigKeys(keys []URLSigKey) error {
+	pg.url.fromGeneric(keys)
+	return pg.url.encrypt(pg.cfg.AESKey)
+}
+
+type pgCommonRecord struct {
+	DataEncrypted []byte
+}
+
+type pgDNSSecRecord struct {
+	Key tc.DNSSECKeysTrafficVault
+	CDN string
+	pgCommonRecord
+}
+type pgDNSSecTable struct {
+	Records []pgDNSSecRecord
+}
+
+func (tbl *pgDNSSecTable) gatherKeys(db *sql.DB) error {
+	sz, err := getSize(db, "dnssec")
+	if err != nil {
+		log.Println("PGDNSSec gatherKeys: unable to determine size of dnssec table")
+	}
+	tbl.Records = make([]pgDNSSecRecord, sz)
+
+	rows, err := db.Query("SELECT cdn, data from dnssec")
+	if err != nil {
+		return fmt.Errorf("PGDNSSec gatherKeys: unable to query: %w", err)
+	}
+	defer rows.Close()
+	i := 0
+	for rows.Next() {
+		if i > len(tbl.Records)-1 {
+			return fmt.Errorf("PGDNSSec gatherKeys got more results than expected %v", len(tbl.Records))
+		}
+		err := rows.Scan(&tbl.Records[i].CDN, &tbl.Records[i].DataEncrypted)
+		if err != nil {
+			return fmt.Errorf("PGDNSSec gatherKeys unable to scan row: %w", err)
+		}
+		i += 1
+	}
+	return nil
+}
+func (tbl *pgDNSSecTable) decrypt(aesKey []byte) error {
+	for i, _ := range tbl.Records {
+		err := decryptInto(aesKey, tbl.Records[i].DataEncrypted, &tbl.Records[i].Key)
+		if err != nil {
+			return fmt.Errorf("unable to decrypt into keys: %w", err)
+		}
+	}
+	return nil
+}
+func (tbl *pgDNSSecTable) encrypt(aesKey []byte) error {
+	for i, dns := range tbl.Records {
+		data, err := json.Marshal(&dns.Key)
+		if err != nil {
+			return fmt.Errorf("encrypt issue marshalling keys: %w", err)
+		}
+		dat, err := encrypt(data, aesKey)
+		if err != nil {
+			return fmt.Errorf("encrypt error: %w", err)
+		}
+		tbl.Records[i].DataEncrypted = dat
+	}
+	return nil
+}
+func (tbl *pgDNSSecTable) toGeneric() []DNSSecKey {
+	keys := make([]DNSSecKey, len(tbl.Records))
+
+	for i, record := range tbl.Records {
+		keys[i] = DNSSecKey{
+			CDN:                    record.CDN,
+			DNSSECKeysTrafficVault: record.Key,
+		}
+	}
+
+	return keys
+}
+func (tbl *pgDNSSecTable) fromGeneric(keys []DNSSecKey) {
+	tbl.Records = make([]pgDNSSecRecord, len(keys))
+
+	for i, key := range keys {
+		tbl.Records[i] = pgDNSSecRecord{
+			Key: key.DNSSECKeysTrafficVault,
+			CDN: key.CDN,
+			pgCommonRecord: pgCommonRecord{
+				DataEncrypted: nil,
+			},
+		}
+	}
+}
+func (tbl *pgDNSSecTable) validate() []string {
+	for i, record := range tbl.Records {
+		if record.DataEncrypted == nil && len(record.Key) > 0 {
+			return []string{fmt.Sprintf("DNSSEC Key %v: DataEncrypted is blank!", i)}
+		}
+	}
+	return nil
+}
+func (tbl *pgDNSSecTable) insertKeys(db *sql.DB) error {
+	queryBase := "INSERT INTO dnssec (cdn, data) VALUES "
+	stride := 2
+	queryArgs := make([]interface{}, len(tbl.Records)*stride)
+	for i, record := range tbl.Records {
+		j := i * stride
+		queryArgs[j] = record.CDN
+		queryArgs[j+1] = record.DataEncrypted
+	}
+	return insertIntoTable(db, queryBase, stride, queryArgs)
+}
+
+type pgSSLKeyRecord struct {
+	Keys tc.DeliveryServiceSSLKeys
+	pgCommonRecord
+
+	// These records are stored on the table but are duplicated
+	DeliveryService string
+	CDN             string
+}
+type pgSSLKeyTable struct {
+	Records []pgSSLKeyRecord
+}
+
+func (tbl *pgSSLKeyTable) insertKeys(db *sql.DB) error {
+	queryBase := "INSERT INTO sslkey (deliveryservice, data, cdn, version) VALUES "
+	duplicateKeys := 2
+	stride := 4
+	queryArgs := make([]interface{}, len(tbl.Records)*stride*duplicateKeys)
+	for i, record := range tbl.Records {
+		j := i * duplicateKeys * stride
+
+		queryArgs[j] = record.DeliveryService
+		queryArgs[j+1] = record.DataEncrypted
+		queryArgs[j+2] = record.CDN
+		queryArgs[j+3] = record.Keys.Version.String()
+
+		queryArgs[j+4] = record.DeliveryService
+		queryArgs[j+5] = record.DataEncrypted
+		queryArgs[j+6] = record.CDN
+		queryArgs[j+7] = "latest"
+	}
+	return insertIntoTable(db, queryBase, 4, queryArgs)
+}
+func (tbl *pgSSLKeyTable) gatherKeys(db *sql.DB) error {
+	sz, err := getSize(db, "sslkey WHERE version='latest'")
+	if err != nil {
+		return fmt.Errorf("PGSSLKey gatherKeys unable to determine size of sslkey table: %w", err)
+	}
+	tbl.Records = make([]pgSSLKeyRecord, sz)
+
+	rows, err := db.Query("SELECT data, deliveryservice, cdn from sslkey WHERE version = 'latest'")
+	if err != nil {
+		return fmt.Errorf("PGSSLKey gatherKeys unable to query: %w", err)
+	}
+	defer rows.Close()
+	i := 0
+	for rows.Next() {
+		if i > len(tbl.Records)-1 {
+			return errors.New("PGSSLKey gatherKeys: got more results than expected")
+		}
+		err := rows.Scan(&tbl.Records[i].DataEncrypted, &tbl.Records[i].DeliveryService, &tbl.Records[i].CDN)
+		if err != nil {
+			return fmt.Errorf("PGSSLKey gatherKeys unable to scan row: %w", err)
+		}
+		i += 1
+	}
+	return nil
+}
+func (tbl *pgSSLKeyTable) decrypt(aesKey []byte) error {
+	for i, dns := range tbl.Records {
+		err := decryptInto(aesKey, dns.DataEncrypted, &tbl.Records[i].Keys)
+		if err != nil {
+			return fmt.Errorf("unable to decrypt into keys: %w", err)
+		}
+	}
+	return nil
+}
+func (tbl *pgSSLKeyTable) encrypt(aesKey []byte) error {
+	for i, dns := range tbl.Records {
+		data, err := json.Marshal(dns.Keys)
+		if err != nil {
+			return fmt.Errorf("encrypt issue marshalling keys: %w", err)
+		}
+		dat, err := encrypt(data, aesKey)
+		if err != nil {
+			return fmt.Errorf("encrypt error: %w", err)
+		}
+		tbl.Records[i].DataEncrypted = dat
+	}
+	return nil
+}
+func (tbl *pgSSLKeyTable) toGeneric() []SSLKey {
+	keys := make([]SSLKey, len(tbl.Records))
+
+	for i, record := range tbl.Records {
+		keys[i] = SSLKey{
+			DeliveryServiceSSLKeys: record.Keys,
+		}
+	}
+	return keys
+}
+func (tbl *pgSSLKeyTable) fromGeneric(keys []SSLKey) {
+	tbl.Records = make([]pgSSLKeyRecord, len(keys))
+
+	for i, key := range keys {
+		tbl.Records[i] = pgSSLKeyRecord{
+			Keys: key.DeliveryServiceSSLKeys,
+			pgCommonRecord: pgCommonRecord{
+				DataEncrypted: nil,
+			},
+			DeliveryService: key.DeliveryService,
+			CDN:             key.CDN,
+		}
+	}
+}
+func (tbl *pgSSLKeyTable) validate() []string {
+	defaultKey := tc.DeliveryServiceSSLKeys{}
+	var errors []string
+	fmtStr := "SSL Key %v: %v"
+	for i, record := range tbl.Records {
+		if record.Keys == defaultKey {
+			errors = append(errors, fmt.Sprintf(fmtStr, i, "DS SSL Keys are default!"))
+		} else if record.Keys.Key == "" {
+			errors = append(errors, fmt.Sprintf(fmtStr, i, "Key is blank!"))
+		} else if record.Keys.CDN == "" {
+			errors = append(errors, fmt.Sprintf(fmtStr, i, "CDN is blank!"))
+		} else if record.Keys.DeliveryService == "" {
+			errors = append(errors, fmt.Sprintf(fmtStr, i, "DS is blank!"))
+		} else if record.DataEncrypted == nil {
+			errors = append(errors, fmt.Sprintf(fmtStr, i, "DataEncrypted is blank!"))
+		} else if record.Keys.Version.String() == "" {
+			errors = append(errors, fmt.Sprintf(fmtStr, i, "Version is blank!"))
+		}
+	}
+	return errors
+}
+
+type pgURLSigKeyRecord struct {
+	Keys            tc.URLSigKeys
+	DeliveryService string
+	pgCommonRecord
+}
+type pgURLSigKeyTable struct {
+	Records []pgURLSigKeyRecord
+}
+
+func (tbl *pgURLSigKeyTable) insertKeys(db *sql.DB) error {
+	queryBase := "INSERT INTO url_sig_key (deliveryservice, data) VALUES "
+	stride := 2
+	queryArgs := make([]interface{}, len(tbl.Records)*stride)
+	for i, record := range tbl.Records {
+		j := i * stride
+		queryArgs[j] = record.DeliveryService
+		queryArgs[j+1] = record.DataEncrypted
+	}
+	return insertIntoTable(db, queryBase, stride, queryArgs)
+}
+func (tbl *pgURLSigKeyTable) gatherKeys(db *sql.DB) error {
+	sz, err := getSize(db, "url_sig_key")
+	if err != nil {
+		log.Println("PGURLSigKey gatherKeys: unable to determine url_sig_key table size")
+	}
+	tbl.Records = make([]pgURLSigKeyRecord, sz)
+
+	rows, err := db.Query("SELECT deliveryservice, data from url_sig_key")
+	if err != nil {
+		return fmt.Errorf("PGURLSigKey gatherKeys error creating query: %w", err)
+	}
+	defer rows.Close()
+	i := 0
+	for rows.Next() {
+		if i > len(tbl.Records)-1 {
+			return fmt.Errorf("PGURLSigKey gatherKeys: got more results than expected %v", len(tbl.Records))
+		}
+		err := rows.Scan(&tbl.Records[i].DeliveryService, &tbl.Records[i].DataEncrypted)
+		if err != nil {
+			return fmt.Errorf("PGURLSigKey gatherKeys: unable to scan row: %w", err)
+		}
+		i += 1
+	}
+	return nil
+}
+func (tbl *pgURLSigKeyTable) decrypt(aesKey []byte) error {
+	for i, sig := range tbl.Records {
+		err := decryptInto(aesKey, sig.DataEncrypted, &tbl.Records[i].Keys)
+		if err != nil {
+			return fmt.Errorf("unable to decrypt into keys: %w", err)
+		}
+	}
+	return nil
+}
+func (tbl *pgURLSigKeyTable) encrypt(aesKey []byte) error {
+	for i, sig := range tbl.Records {
+		data, err := json.Marshal(&sig.Keys)
+		if err != nil {
+			return fmt.Errorf("encrypt issue marshalling keys: %w", err)
+		}
+
+		dat, err := encrypt(data, aesKey)
+		if err != nil {
+			return fmt.Errorf("encrypt error: %w", err)
+		}
+		tbl.Records[i].DataEncrypted = dat
+	}
+	return nil
+}
+func (tbl *pgURLSigKeyTable) toGeneric() []URLSigKey {
+	keys := make([]URLSigKey, len(tbl.Records))
+
+	for i, record := range tbl.Records {
+		keys[i] = URLSigKey{
+			DeliveryService: record.DeliveryService,
+			URLSigKeys:      record.Keys,
+		}
+	}
+	return keys
+}
+func (tbl *pgURLSigKeyTable) fromGeneric(keys []URLSigKey) {
+	tbl.Records = make([]pgURLSigKeyRecord, len(keys))
+
+	for i, key := range keys {
+		tbl.Records[i] = pgURLSigKeyRecord{
+			Keys:            key.URLSigKeys,
+			DeliveryService: key.DeliveryService,
+			pgCommonRecord: pgCommonRecord{
+				DataEncrypted: nil,
+			},
+		}
+	}
+}
+func (tbl *pgURLSigKeyTable) validate() []string {
+	for i, record := range tbl.Records {
+		if record.DataEncrypted == nil && len(record.Keys) > 0 {
+			return []string{fmt.Sprintf("URl Sig Key %v: DataEncrypted is blank!", i)}
+		}
+	}
+	return nil
+}
+
+type pgURISignKeyRecord struct {
+	Keys            map[string]tc.URISignerKeyset
+	DeliveryService string
+	pgCommonRecord
+}
+type pgURISignKeyTable struct {
+	Records []pgURISignKeyRecord
+}
+
+func (tbl *pgURISignKeyTable) insertKeys(db *sql.DB) error {
+	queryBase := "INSERT INTO uri_signing_key (deliveryservice, data) VALUES "

Review comment:
       Expensive in terms of dev time or runtime? And do you mean skip it by `ON CONFLICT IGNORE` or something like that?
   
   If we're talking expensive as in runtime, doing a migration should be pretty rare. If we're successful, we might only use this tool once. If we aren't successful at first, maybe we'll run it 3 times if we decide to un-migrate then re-migrate once the issue is fixed. So if it takes a little longer to run, that's probably not a big deal.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [trafficcontrol] shamrickus commented on a change in pull request #5924: Add tool to migrate data between TV backends

Posted by GitBox <gi...@apache.org>.
shamrickus commented on a change in pull request #5924:
URL: https://github.com/apache/trafficcontrol/pull/5924#discussion_r652106179



##########
File path: tools/traffic_vault_migrate/postgres.go
##########
@@ -0,0 +1,743 @@
+package main
+
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+import (
+	"database/sql"
+	"encoding/base64"
+	"encoding/json"
+	"errors"
+	"fmt"
+	"log"
+	"strconv"
+	"strings"
+
+	"github.com/apache/trafficcontrol/lib/go-tc"
+	util "github.com/apache/trafficcontrol/lib/go-util"
+
+	_ "github.com/lib/pq"
+)
+
+// PGConfig represents the configuration options available to the PG backend
+type PGConfig struct {
+	Host     string `json:"host"`
+	Port     string `json:"port"`
+	User     string `json:"user"`
+	Password string `json:"password"`
+	SSLMode  string `json:"sslmode"`
+	Database string `json:"database"`
+	Key      string `json:"aesKey"`
+	AESKey   []byte
+}
+
+// PGBackend is the Postgres implementation of TVBackend
+type PGBackend struct {
+	sslKey pgSSLKeyTable
+	dnssec pgDNSSecTable
+	uri    pgURISignKeyTable
+	url    pgURLSigKeyTable
+	cfg    PGConfig
+	db     *sql.DB
+}
+
+// String returns a high level overview of the backend and its keys
+func (pg *PGBackend) String() string {
+	data := fmt.Sprintf("PG server %v@%v:%v\n", pg.cfg.User, pg.cfg.Host, pg.cfg.Port)
+	data += fmt.Sprintf("\tSSL Keys: %v\n", len(pg.sslKey.Records))
+	data += fmt.Sprintf("\tDNSSec Keys: %v\n", len(pg.dnssec.Records))
+	data += fmt.Sprintf("\tURI Keys: %v\n", len(pg.uri.Records))
+	data += fmt.Sprintf("\tURL Keys: %v\n", len(pg.url.Records))
+	return data
+}
+
+// Name returns the name for this backend
+func (pg *PGBackend) Name() string {
+	return "PG"
+}
+
+// ReadConfig takes in a filename and will read it into the backends config
+func (pg *PGBackend) ReadConfig(s string) error {
+	err := UnmarshalConfig(s, &pg.cfg)
+	if err != nil {
+		return err
+	}
+
+	pg.cfg.AESKey, err = base64.StdEncoding.DecodeString(pg.cfg.Key)
+	if err != nil {
+		return fmt.Errorf("unable to decode PG AESKey: %w", err)
+	}
+	return nil
+}
+
+// Insert takes the current keys and inserts them into the backend DB
+func (pg *PGBackend) Insert() error {
+	if err := pg.sslKey.insertKeys(pg.db); err != nil {
+		return err
+	}
+	if err := pg.dnssec.insertKeys(pg.db); err != nil {
+		return err
+	}
+	if err := pg.url.insertKeys(pg.db); err != nil {
+		return err
+	}
+	if err := pg.uri.insertKeys(pg.db); err != nil {
+		return err
+	}
+	return nil
+}
+
+// Start initiates the connection to the backend DB
+func (pg *PGBackend) Start() error {
+	sqlStr := fmt.Sprintf("postgres://%s:%s@%s:%s/%s?sslmode=%s", pg.cfg.User, pg.cfg.Password, pg.cfg.Host, pg.cfg.Port, pg.cfg.Database, pg.cfg.SSLMode)
+	db, err := sql.Open("postgres", sqlStr)
+	if err != nil {
+		return fmt.Errorf("unable to start PG client: %w", err)
+	}
+
+	pg.db = db
+	pg.sslKey = pgSSLKeyTable{}
+	pg.dnssec = pgDNSSecTable{}
+	pg.url = pgURLSigKeyTable{}
+	pg.uri = pgURISignKeyTable{}
+
+	return nil
+}
+
+// ValidateKey validates that the keys are valid (in most cases, certain fields are not null)
+func (pg *PGBackend) ValidateKey() []string {
+	var errors []string
+	if errs := pg.sslKey.validate(); errs != nil {
+		errors = append(errors, errs...)
+	}
+	if errs := pg.dnssec.validate(); errs != nil {
+		errors = append(errors, errs...)
+	}
+	if errs := pg.uri.validate(); errs != nil {
+		errors = append(errors, errs...)
+	}
+	if errs := pg.url.validate(); errs != nil {
+		errors = append(errors, errs...)
+	}
+	return errors
+}
+
+// Close terminates the connection to the backend DB
+func (pg *PGBackend) Close() error {
+	return pg.db.Close()
+}
+
+// Ping checks the connection to the backend DB
+func (pg *PGBackend) Ping() error {
+	return pg.db.Ping()
+}
+
+// Fetch gets all of the keys from the backend DB
+func (pg *PGBackend) Fetch() error {
+	if err := pg.sslKey.gatherKeys(pg.db); err != nil {
+		return err
+	}
+
+	if err := pg.dnssec.gatherKeys(pg.db); err != nil {
+		return err
+	}
+
+	if err := pg.url.gatherKeys(pg.db); err != nil {
+		return err
+	}
+
+	if err := pg.uri.gatherKeys(pg.db); err != nil {
+		return err
+	}
+
+	return nil
+}
+
+// GetSSLKeys converts the backends internal key representation into the common representation (SSLKey)
+func (pg *PGBackend) GetSSLKeys() ([]SSLKey, error) {
+	if err := pg.sslKey.decrypt(pg.cfg.AESKey); err != nil {
+		return nil, err
+	}
+	return pg.sslKey.toGeneric(), nil
+}
+
+// SetSSLKeys takes in keys and converts & encrypts the data into the backends internal format
+func (pg *PGBackend) SetSSLKeys(keys []SSLKey) error {
+	pg.sslKey.fromGeneric(keys)
+	return pg.sslKey.encrypt(pg.cfg.AESKey)
+}
+
+// GetDNSSecKeys converts the backends internal key representation into the common representation (DNSSecKey)
+func (pg *PGBackend) GetDNSSecKeys() ([]DNSSecKey, error) {
+	if err := pg.dnssec.decrypt(pg.cfg.AESKey); err != nil {
+		return nil, err
+	}
+	return pg.dnssec.toGeneric(), nil
+}
+
+// SetDNSSecKeys takes in keys and converts & encrypts the data into the backends internal format
+func (pg *PGBackend) SetDNSSecKeys(keys []DNSSecKey) error {
+	pg.dnssec.fromGeneric(keys)
+	return pg.dnssec.encrypt(pg.cfg.AESKey)
+}
+
+// GetURISignKeys converts the pg internal key representation into the common representation (URISignKey)
+func (pg *PGBackend) GetURISignKeys() ([]URISignKey, error) {
+	if err := pg.uri.decrypt(pg.cfg.AESKey); err != nil {
+		return nil, err
+	}
+	return pg.uri.toGeneric(), nil
+}
+
+// SetURISignKeys takes in keys and converts & encrypts the data into the backends internal format
+func (pg *PGBackend) SetURISignKeys(keys []URISignKey) error {
+	pg.uri.fromGeneric(keys)
+	return pg.uri.encrypt(pg.cfg.AESKey)
+}
+
+// GetURLSigKeys converts the backends internal key representation into the common representation (URLSigKey)
+func (pg *PGBackend) GetURLSigKeys() ([]URLSigKey, error) {
+	if err := pg.url.decrypt(pg.cfg.AESKey); err != nil {
+		return nil, err
+	}
+	return pg.url.toGeneric(), nil
+}
+
+// SetURLSigKeys takes in keys and converts & encrypts the data into the backends internal format
+func (pg *PGBackend) SetURLSigKeys(keys []URLSigKey) error {
+	pg.url.fromGeneric(keys)
+	return pg.url.encrypt(pg.cfg.AESKey)
+}
+
+type pgCommonRecord struct {
+	DataEncrypted []byte
+}
+
+type pgDNSSecRecord struct {
+	Key tc.DNSSECKeysTrafficVault
+	CDN string
+	pgCommonRecord
+}
+type pgDNSSecTable struct {
+	Records []pgDNSSecRecord
+}
+
+func (tbl *pgDNSSecTable) gatherKeys(db *sql.DB) error {
+	sz, err := getSize(db, "dnssec")
+	if err != nil {
+		log.Println("PGDNSSec gatherKeys: unable to determine size of dnssec table")
+	}
+	tbl.Records = make([]pgDNSSecRecord, sz)
+
+	rows, err := db.Query("SELECT cdn, data from dnssec")
+	if err != nil {
+		return fmt.Errorf("PGDNSSec gatherKeys: unable to query: %w", err)
+	}
+	defer rows.Close()
+	i := 0
+	for rows.Next() {
+		if i > len(tbl.Records)-1 {
+			return fmt.Errorf("PGDNSSec gatherKeys got more results than expected %v", len(tbl.Records))
+		}
+		err := rows.Scan(&tbl.Records[i].CDN, &tbl.Records[i].DataEncrypted)
+		if err != nil {
+			return fmt.Errorf("PGDNSSec gatherKeys unable to scan row: %w", err)
+		}
+		i += 1
+	}
+	return nil
+}
+func (tbl *pgDNSSecTable) decrypt(aesKey []byte) error {
+	for i, _ := range tbl.Records {
+		err := decryptInto(aesKey, tbl.Records[i].DataEncrypted, &tbl.Records[i].Key)
+		if err != nil {
+			return fmt.Errorf("unable to decrypt into keys: %w", err)
+		}
+	}
+	return nil
+}
+func (tbl *pgDNSSecTable) encrypt(aesKey []byte) error {
+	for i, dns := range tbl.Records {
+		data, err := json.Marshal(&dns.Key)
+		if err != nil {
+			return fmt.Errorf("encrypt issue marshalling keys: %w", err)
+		}
+		dat, err := encrypt(data, aesKey)
+		if err != nil {
+			return fmt.Errorf("encrypt error: %w", err)
+		}
+		tbl.Records[i].DataEncrypted = dat
+	}
+	return nil
+}
+func (tbl *pgDNSSecTable) toGeneric() []DNSSecKey {
+	keys := make([]DNSSecKey, len(tbl.Records))
+
+	for i, record := range tbl.Records {
+		keys[i] = DNSSecKey{
+			CDN:                    record.CDN,
+			DNSSECKeysTrafficVault: record.Key,
+		}
+	}
+
+	return keys
+}
+func (tbl *pgDNSSecTable) fromGeneric(keys []DNSSecKey) {
+	tbl.Records = make([]pgDNSSecRecord, len(keys))
+
+	for i, key := range keys {
+		tbl.Records[i] = pgDNSSecRecord{
+			Key: key.DNSSECKeysTrafficVault,
+			CDN: key.CDN,
+			pgCommonRecord: pgCommonRecord{
+				DataEncrypted: nil,
+			},
+		}
+	}
+}
+func (tbl *pgDNSSecTable) validate() []string {
+	for i, record := range tbl.Records {
+		if record.DataEncrypted == nil && len(record.Key) > 0 {
+			return []string{fmt.Sprintf("DNSSEC Key %v: DataEncrypted is blank!", i)}
+		}
+	}
+	return nil
+}
+func (tbl *pgDNSSecTable) insertKeys(db *sql.DB) error {
+	queryBase := "INSERT INTO dnssec (cdn, data) VALUES "
+	stride := 2
+	queryArgs := make([]interface{}, len(tbl.Records)*stride)
+	for i, record := range tbl.Records {
+		j := i * stride
+		queryArgs[j] = record.CDN
+		queryArgs[j+1] = record.DataEncrypted
+	}
+	return insertIntoTable(db, queryBase, stride, queryArgs)
+}
+
+type pgSSLKeyRecord struct {
+	Keys tc.DeliveryServiceSSLKeys
+	pgCommonRecord
+
+	// These records are stored on the table but are duplicated
+	DeliveryService string
+	CDN             string
+}
+type pgSSLKeyTable struct {
+	Records []pgSSLKeyRecord
+}
+
+func (tbl *pgSSLKeyTable) insertKeys(db *sql.DB) error {
+	queryBase := "INSERT INTO sslkey (deliveryservice, data, cdn, version) VALUES "
+	duplicateKeys := 2
+	stride := 4
+	queryArgs := make([]interface{}, len(tbl.Records)*stride*duplicateKeys)
+	for i, record := range tbl.Records {
+		j := i * duplicateKeys * stride
+
+		queryArgs[j] = record.DeliveryService
+		queryArgs[j+1] = record.DataEncrypted
+		queryArgs[j+2] = record.CDN
+		queryArgs[j+3] = record.Keys.Version.String()
+
+		queryArgs[j+4] = record.DeliveryService
+		queryArgs[j+5] = record.DataEncrypted
+		queryArgs[j+6] = record.CDN
+		queryArgs[j+7] = "latest"
+	}
+	return insertIntoTable(db, queryBase, 4, queryArgs)
+}
+func (tbl *pgSSLKeyTable) gatherKeys(db *sql.DB) error {
+	sz, err := getSize(db, "sslkey WHERE version='latest'")
+	if err != nil {
+		return fmt.Errorf("PGSSLKey gatherKeys unable to determine size of sslkey table: %w", err)
+	}
+	tbl.Records = make([]pgSSLKeyRecord, sz)
+
+	rows, err := db.Query("SELECT data, deliveryservice, cdn from sslkey WHERE version = 'latest'")
+	if err != nil {
+		return fmt.Errorf("PGSSLKey gatherKeys unable to query: %w", err)
+	}
+	defer rows.Close()
+	i := 0
+	for rows.Next() {
+		if i > len(tbl.Records)-1 {
+			return errors.New("PGSSLKey gatherKeys: got more results than expected")
+		}
+		err := rows.Scan(&tbl.Records[i].DataEncrypted, &tbl.Records[i].DeliveryService, &tbl.Records[i].CDN)
+		if err != nil {
+			return fmt.Errorf("PGSSLKey gatherKeys unable to scan row: %w", err)
+		}
+		i += 1
+	}
+	return nil
+}
+func (tbl *pgSSLKeyTable) decrypt(aesKey []byte) error {
+	for i, dns := range tbl.Records {
+		err := decryptInto(aesKey, dns.DataEncrypted, &tbl.Records[i].Keys)
+		if err != nil {
+			return fmt.Errorf("unable to decrypt into keys: %w", err)
+		}
+	}
+	return nil
+}
+func (tbl *pgSSLKeyTable) encrypt(aesKey []byte) error {
+	for i, dns := range tbl.Records {
+		data, err := json.Marshal(dns.Keys)
+		if err != nil {
+			return fmt.Errorf("encrypt issue marshalling keys: %w", err)
+		}
+		dat, err := encrypt(data, aesKey)
+		if err != nil {
+			return fmt.Errorf("encrypt error: %w", err)
+		}
+		tbl.Records[i].DataEncrypted = dat
+	}
+	return nil
+}
+func (tbl *pgSSLKeyTable) toGeneric() []SSLKey {
+	keys := make([]SSLKey, len(tbl.Records))
+
+	for i, record := range tbl.Records {
+		keys[i] = SSLKey{
+			DeliveryServiceSSLKeys: record.Keys,
+		}
+	}
+	return keys
+}
+func (tbl *pgSSLKeyTable) fromGeneric(keys []SSLKey) {
+	tbl.Records = make([]pgSSLKeyRecord, len(keys))
+
+	for i, key := range keys {
+		tbl.Records[i] = pgSSLKeyRecord{
+			Keys: key.DeliveryServiceSSLKeys,
+			pgCommonRecord: pgCommonRecord{
+				DataEncrypted: nil,
+			},
+			DeliveryService: key.DeliveryService,
+			CDN:             key.CDN,
+		}
+	}
+}
+func (tbl *pgSSLKeyTable) validate() []string {
+	defaultKey := tc.DeliveryServiceSSLKeys{}
+	var errors []string
+	fmtStr := "SSL Key %v: %v"
+	for i, record := range tbl.Records {
+		if record.Keys == defaultKey {
+			errors = append(errors, fmt.Sprintf(fmtStr, i, "DS SSL Keys are default!"))
+		} else if record.Keys.Key == "" {
+			errors = append(errors, fmt.Sprintf(fmtStr, i, "Key is blank!"))
+		} else if record.Keys.CDN == "" {
+			errors = append(errors, fmt.Sprintf(fmtStr, i, "CDN is blank!"))
+		} else if record.Keys.DeliveryService == "" {
+			errors = append(errors, fmt.Sprintf(fmtStr, i, "DS is blank!"))
+		} else if record.DataEncrypted == nil {
+			errors = append(errors, fmt.Sprintf(fmtStr, i, "DataEncrypted is blank!"))
+		} else if record.Keys.Version.String() == "" {
+			errors = append(errors, fmt.Sprintf(fmtStr, i, "Version is blank!"))
+		}
+	}
+	return errors
+}
+
+type pgURLSigKeyRecord struct {
+	Keys            tc.URLSigKeys
+	DeliveryService string
+	pgCommonRecord
+}
+type pgURLSigKeyTable struct {
+	Records []pgURLSigKeyRecord
+}
+
+func (tbl *pgURLSigKeyTable) insertKeys(db *sql.DB) error {
+	queryBase := "INSERT INTO url_sig_key (deliveryservice, data) VALUES "
+	stride := 2
+	queryArgs := make([]interface{}, len(tbl.Records)*stride)
+	for i, record := range tbl.Records {
+		j := i * stride
+		queryArgs[j] = record.DeliveryService
+		queryArgs[j+1] = record.DataEncrypted
+	}
+	return insertIntoTable(db, queryBase, stride, queryArgs)
+}
+func (tbl *pgURLSigKeyTable) gatherKeys(db *sql.DB) error {
+	sz, err := getSize(db, "url_sig_key")
+	if err != nil {
+		log.Println("PGURLSigKey gatherKeys: unable to determine url_sig_key table size")
+	}
+	tbl.Records = make([]pgURLSigKeyRecord, sz)
+
+	rows, err := db.Query("SELECT deliveryservice, data from url_sig_key")
+	if err != nil {
+		return fmt.Errorf("PGURLSigKey gatherKeys error creating query: %w", err)
+	}
+	defer rows.Close()
+	i := 0
+	for rows.Next() {
+		if i > len(tbl.Records)-1 {
+			return fmt.Errorf("PGURLSigKey gatherKeys: got more results than expected %v", len(tbl.Records))
+		}
+		err := rows.Scan(&tbl.Records[i].DeliveryService, &tbl.Records[i].DataEncrypted)
+		if err != nil {
+			return fmt.Errorf("PGURLSigKey gatherKeys: unable to scan row: %w", err)
+		}
+		i += 1
+	}
+	return nil
+}
+func (tbl *pgURLSigKeyTable) decrypt(aesKey []byte) error {
+	for i, sig := range tbl.Records {
+		err := decryptInto(aesKey, sig.DataEncrypted, &tbl.Records[i].Keys)
+		if err != nil {
+			return fmt.Errorf("unable to decrypt into keys: %w", err)
+		}
+	}
+	return nil
+}
+func (tbl *pgURLSigKeyTable) encrypt(aesKey []byte) error {
+	for i, sig := range tbl.Records {
+		data, err := json.Marshal(&sig.Keys)
+		if err != nil {
+			return fmt.Errorf("encrypt issue marshalling keys: %w", err)
+		}
+
+		dat, err := encrypt(data, aesKey)
+		if err != nil {
+			return fmt.Errorf("encrypt error: %w", err)
+		}
+		tbl.Records[i].DataEncrypted = dat
+	}
+	return nil
+}
+func (tbl *pgURLSigKeyTable) toGeneric() []URLSigKey {
+	keys := make([]URLSigKey, len(tbl.Records))
+
+	for i, record := range tbl.Records {
+		keys[i] = URLSigKey{
+			DeliveryService: record.DeliveryService,
+			URLSigKeys:      record.Keys,
+		}
+	}
+	return keys
+}
+func (tbl *pgURLSigKeyTable) fromGeneric(keys []URLSigKey) {
+	tbl.Records = make([]pgURLSigKeyRecord, len(keys))
+
+	for i, key := range keys {
+		tbl.Records[i] = pgURLSigKeyRecord{
+			Keys:            key.URLSigKeys,
+			DeliveryService: key.DeliveryService,
+			pgCommonRecord: pgCommonRecord{
+				DataEncrypted: nil,
+			},
+		}
+	}
+}
+func (tbl *pgURLSigKeyTable) validate() []string {
+	for i, record := range tbl.Records {
+		if record.DataEncrypted == nil && len(record.Keys) > 0 {
+			return []string{fmt.Sprintf("URl Sig Key %v: DataEncrypted is blank!", i)}
+		}
+	}
+	return nil
+}
+
+type pgURISignKeyRecord struct {
+	Keys            map[string]tc.URISignerKeyset
+	DeliveryService string
+	pgCommonRecord
+}
+type pgURISignKeyTable struct {
+	Records []pgURISignKeyRecord
+}
+
+func (tbl *pgURISignKeyTable) insertKeys(db *sql.DB) error {
+	queryBase := "INSERT INTO uri_signing_key (deliveryservice, data) VALUES "

Review comment:
       I also ended up removing the `id` column as it appears that the only use for it was for the unique constraint.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org