restart instances via rest api instead of recreating pods, fixes bug with being unable to decrease some values, like max_connections (#1103)

* restart instances via rest api instead of recreating pods
* Ignore differences in bootstrap.dcs when compare SPILO_CONFIGURATION
* isBootstrapOnlyParameter is rewritten, instead of whitelist it uses blacklist
* added e2e test for max_connections decreasing
* documentation updated
* pending_restart flag added to restart api call, wait fot ttl seconds after restart
* refactoring, /restart returns error if pending_restart is set to true and patroni is not pending restart
* restart postgresql instances within pods only if pod's restart is not required
* patroni might need to restart postgresql after pods were recreated if values like max_connections decreased
* instancesRestart is not critical, try to restart pods if not successful
* cleanup

Co-authored-by: Felix Kunde <felix-kunde@gmx.de>
This commit is contained in:
Igor Yanchenko 2021-06-14 12:00:58 +03:00 committed by GitHub
parent 75a9e2be38
commit ebb3204cdd
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
8 changed files with 438 additions and 17 deletions

View File

@ -168,6 +168,10 @@ operator checks during Sync all pods run images specified in their respective
statefulsets. The operator triggers a rolling upgrade for PG clusters that
violate this condition.
Changes in $SPILO\_CONFIGURATION under path bootstrap.dcs are ignored when
StatefulSets are being compared, if there are changes under this path, they are
applied through rest api interface and following restart of patroni instance
## Delete protection via annotations
To avoid accidental deletes of Postgres clusters the operator can check the

View File

@ -1418,6 +1418,54 @@ class EndToEndTestCase(unittest.TestCase):
}
k8s.update_config(patch_delete_annotations)
@timeout_decorator.timeout(TEST_TIMEOUT_SEC)
def test_decrease_max_connections(self):
'''
Test decreasing max_connections and restarting cluster through rest api
'''
k8s = self.k8s
cluster_label = 'application=spilo,cluster-name=acid-minimal-cluster'
labels = 'spilo-role=master,' + cluster_label
new_max_connections_value = "99"
pods = k8s.api.core_v1.list_namespaced_pod(
'default', label_selector=labels).items
self.assert_master_is_unique()
masterPod = pods[0]
creationTimestamp = masterPod.metadata.creation_timestamp
# adjust max_connection
pg_patch_max_connections = {
"spec": {
"postgresql": {
"parameters": {
"max_connections": new_max_connections_value
}
}
}
}
k8s.api.custom_objects_api.patch_namespaced_custom_object(
"acid.zalan.do", "v1", "default", "postgresqls", "acid-minimal-cluster", pg_patch_max_connections)
def get_max_connections():
pods = k8s.api.core_v1.list_namespaced_pod(
'default', label_selector=labels).items
self.assert_master_is_unique()
masterPod = pods[0]
get_max_connections_cmd = '''psql -At -U postgres -c "SELECT setting FROM pg_settings WHERE name = 'max_connections';"'''
result = k8s.exec_with_kubectl(masterPod.metadata.name, get_max_connections_cmd)
max_connections_value = int(result.stdout)
return max_connections_value
#Make sure that max_connections decreased
self.eventuallyEqual(get_max_connections, int(new_max_connections_value), "max_connections didn't decrease")
pods = k8s.api.core_v1.list_namespaced_pod(
'default', label_selector=labels).items
self.assert_master_is_unique()
masterPod = pods[0]
#Make sure that pod didn't restart
self.assertEqual(creationTimestamp, masterPod.metadata.creation_timestamp,
"Master pod creation timestamp is updated")
def get_failover_targets(self, master_node, replica_nodes):
'''
If all pods live on the same node, failover will happen to other worker(s)

View File

@ -5,6 +5,7 @@ package cluster
import (
"context"
"database/sql"
"encoding/json"
"fmt"
"reflect"
"regexp"
@ -519,7 +520,7 @@ func (c *Cluster) compareContainers(description string, setA, setB []v1.Containe
newCheck("new statefulset %s's %s (index %d) resources do not match the current ones",
func(a, b v1.Container) bool { return !compareResources(&a.Resources, &b.Resources) }),
newCheck("new statefulset %s's %s (index %d) environment does not match the current one",
func(a, b v1.Container) bool { return !reflect.DeepEqual(a.Env, b.Env) }),
func(a, b v1.Container) bool { return !compareEnv(a.Env, b.Env) }),
newCheck("new statefulset %s's %s (index %d) environment sources do not match the current one",
func(a, b v1.Container) bool { return !reflect.DeepEqual(a.EnvFrom, b.EnvFrom) }),
newCheck("new statefulset %s's %s (index %d) security context does not match the current one",
@ -576,6 +577,56 @@ func compareResourcesAssumeFirstNotNil(a *v1.ResourceRequirements, b *v1.Resourc
}
func compareEnv(a, b []v1.EnvVar) bool {
if len(a) != len(b) {
return false
}
equal := true
for _, enva := range a {
hasmatch := false
for _, envb := range b {
if enva.Name == envb.Name {
hasmatch = true
if enva.Name == "SPILO_CONFIGURATION" {
equal = compareSpiloConfiguration(enva.Value, envb.Value)
} else {
if enva.Value == "" && envb.Value == "" {
equal = reflect.DeepEqual(enva.ValueFrom, envb.ValueFrom)
} else {
equal = (enva.Value == envb.Value)
}
}
if !equal {
return false
}
}
}
if !hasmatch {
return false
}
}
return true
}
func compareSpiloConfiguration(configa, configb string) bool {
var (
oa, ob spiloConfiguration
)
var err error
err = json.Unmarshal([]byte(configa), &oa)
if err != nil {
return false
}
oa.Bootstrap.DCS = patroniDCS{}
err = json.Unmarshal([]byte(configb), &ob)
if err != nil {
return false
}
ob.Bootstrap.DCS = patroniDCS{}
return reflect.DeepEqual(oa, ob)
}
func (c *Cluster) enforceMinResourceLimits(spec *acidv1.PostgresSpec) error {
var (

View File

@ -13,6 +13,7 @@ import (
"github.com/zalando/postgres-operator/pkg/util/constants"
"github.com/zalando/postgres-operator/pkg/util/k8sutil"
"github.com/zalando/postgres-operator/pkg/util/teams"
v1 "k8s.io/api/core/v1"
metav1 "k8s.io/apimachinery/pkg/apis/meta/v1"
"k8s.io/client-go/kubernetes/fake"
"k8s.io/client-go/tools/record"
@ -848,6 +849,159 @@ func TestPreparedDatabases(t *testing.T) {
}
}
func TestCompareSpiloConfiguration(t *testing.T) {
testCases := []struct {
Config string
ExpectedResult bool
}{
{
`{"postgresql":{"bin_dir":"/usr/lib/postgresql/12/bin","parameters":{"autovacuum_analyze_scale_factor":"0.1"},"pg_hba":["hostssl all all 0.0.0.0/0 md5","host all all 0.0.0.0/0 md5"]},"bootstrap":{"initdb":[{"auth-host":"md5"},{"auth-local":"trust"},"data-checksums",{"encoding":"UTF8"},{"locale":"en_US.UTF-8"}],"users":{"test":{"password":"","options":["CREATEDB","NOLOGIN"]}},"dcs":{"ttl":30,"loop_wait":10,"retry_timeout":10,"maximum_lag_on_failover":33554432,"postgresql":{"parameters":{"max_connections":"100","max_locks_per_transaction":"64","max_worker_processes":"4"}}}}}`,
true,
},
{
`{"postgresql":{"bin_dir":"/usr/lib/postgresql/12/bin","parameters":{"autovacuum_analyze_scale_factor":"0.1"},"pg_hba":["hostssl all all 0.0.0.0/0 md5","host all all 0.0.0.0/0 md5"]},"bootstrap":{"initdb":[{"auth-host":"md5"},{"auth-local":"trust"},"data-checksums",{"encoding":"UTF8"},{"locale":"en_US.UTF-8"}],"users":{"test":{"password":"","options":["CREATEDB","NOLOGIN"]}},"dcs":{"ttl":30,"loop_wait":10,"retry_timeout":10,"maximum_lag_on_failover":33554432,"postgresql":{"parameters":{"max_connections":"200","max_locks_per_transaction":"64","max_worker_processes":"4"}}}}}`,
true,
},
{
`{"postgresql":{"bin_dir":"/usr/lib/postgresql/12/bin","parameters":{"autovacuum_analyze_scale_factor":"0.1"},"pg_hba":["hostssl all all 0.0.0.0/0 md5","host all all 0.0.0.0/0 md5"]},"bootstrap":{"initdb":[{"auth-host":"md5"},{"auth-local":"trust"},"data-checksums",{"encoding":"UTF8"},{"locale":"en_US.UTF-8"}],"users":{"test":{"password":"","options":["CREATEDB"]}},"dcs":{"ttl":30,"loop_wait":10,"retry_timeout":10,"maximum_lag_on_failover":33554432,"postgresql":{"parameters":{"max_connections":"200","max_locks_per_transaction":"64","max_worker_processes":"4"}}}}}`,
false,
},
{
`{}`,
false,
},
{
`invalidjson`,
false,
},
}
refCase := testCases[0]
for _, testCase := range testCases {
if result := compareSpiloConfiguration(refCase.Config, testCase.Config); result != testCase.ExpectedResult {
t.Errorf("expected %v got %v", testCase.ExpectedResult, result)
}
}
}
func TestCompareEnv(t *testing.T) {
testCases := []struct {
Envs []v1.EnvVar
ExpectedResult bool
}{
{
Envs: []v1.EnvVar{
{
Name: "VARIABLE1",
Value: "value1",
},
{
Name: "VARIABLE2",
Value: "value2",
},
{
Name: "VARIABLE3",
Value: "value3",
},
{
Name: "SPILO_CONFIGURATION",
Value: `{"postgresql":{"bin_dir":"/usr/lib/postgresql/12/bin","parameters":{"autovacuum_analyze_scale_factor":"0.1"},"pg_hba":["hostssl all all 0.0.0.0/0 md5","host all all 0.0.0.0/0 md5"]},"bootstrap":{"initdb":[{"auth-host":"md5"},{"auth-local":"trust"},"data-checksums",{"encoding":"UTF8"},{"locale":"en_US.UTF-8"}],"users":{"test":{"password":"","options":["CREATEDB","NOLOGIN"]}},"dcs":{"ttl":30,"loop_wait":10,"retry_timeout":10,"maximum_lag_on_failover":33554432,"postgresql":{"parameters":{"max_connections":"100","max_locks_per_transaction":"64","max_worker_processes":"4"}}}}}`,
},
},
ExpectedResult: true,
},
{
Envs: []v1.EnvVar{
{
Name: "VARIABLE1",
Value: "value1",
},
{
Name: "VARIABLE2",
Value: "value2",
},
{
Name: "VARIABLE3",
Value: "value3",
},
{
Name: "SPILO_CONFIGURATION",
Value: `{"postgresql":{"bin_dir":"/usr/lib/postgresql/12/bin","parameters":{"autovacuum_analyze_scale_factor":"0.1"},"pg_hba":["hostssl all all 0.0.0.0/0 md5","host all all 0.0.0.0/0 md5"]},"bootstrap":{"initdb":[{"auth-host":"md5"},{"auth-local":"trust"},"data-checksums",{"encoding":"UTF8"},{"locale":"en_US.UTF-8"}],"users":{"test":{"password":"","options":["CREATEDB","NOLOGIN"]}},"dcs":{"loop_wait":10,"retry_timeout":10,"maximum_lag_on_failover":33554432,"postgresql":{"parameters":{"max_locks_per_transaction":"64","max_worker_processes":"4"}}}}}`,
},
},
ExpectedResult: true,
},
{
Envs: []v1.EnvVar{
{
Name: "VARIABLE4",
Value: "value4",
},
{
Name: "VARIABLE2",
Value: "value2",
},
{
Name: "VARIABLE3",
Value: "value3",
},
{
Name: "SPILO_CONFIGURATION",
Value: `{"postgresql":{"bin_dir":"/usr/lib/postgresql/12/bin","parameters":{"autovacuum_analyze_scale_factor":"0.1"},"pg_hba":["hostssl all all 0.0.0.0/0 md5","host all all 0.0.0.0/0 md5"]},"bootstrap":{"initdb":[{"auth-host":"md5"},{"auth-local":"trust"},"data-checksums",{"encoding":"UTF8"},{"locale":"en_US.UTF-8"}],"users":{"test":{"password":"","options":["CREATEDB","NOLOGIN"]}},"dcs":{"loop_wait":10,"retry_timeout":10,"maximum_lag_on_failover":33554432,"postgresql":{"parameters":{"max_locks_per_transaction":"64","max_worker_processes":"4"}}}}}`,
},
},
ExpectedResult: false,
},
{
Envs: []v1.EnvVar{
{
Name: "VARIABLE1",
Value: "value1",
},
{
Name: "VARIABLE2",
Value: "value2",
},
{
Name: "VARIABLE3",
Value: "value3",
},
{
Name: "VARIABLE4",
Value: "value4",
},
{
Name: "SPILO_CONFIGURATION",
Value: `{"postgresql":{"bin_dir":"/usr/lib/postgresql/12/bin","parameters":{"autovacuum_analyze_scale_factor":"0.1"},"pg_hba":["hostssl all all 0.0.0.0/0 md5","host all all 0.0.0.0/0 md5"]},"bootstrap":{"initdb":[{"auth-host":"md5"},{"auth-local":"trust"},"data-checksums",{"encoding":"UTF8"},{"locale":"en_US.UTF-8"}],"users":{"test":{"password":"","options":["CREATEDB","NOLOGIN"]}},"dcs":{"ttl":30,"loop_wait":10,"retry_timeout":10,"maximum_lag_on_failover":33554432,"postgresql":{"parameters":{"max_connections":"100","max_locks_per_transaction":"64","max_worker_processes":"4"}}}}}`,
},
},
ExpectedResult: false,
},
{
Envs: []v1.EnvVar{
{
Name: "VARIABLE1",
Value: "value1",
},
{
Name: "VARIABLE2",
Value: "value2",
},
{
Name: "SPILO_CONFIGURATION",
Value: `{"postgresql":{"bin_dir":"/usr/lib/postgresql/12/bin","parameters":{"autovacuum_analyze_scale_factor":"0.1"},"pg_hba":["hostssl all all 0.0.0.0/0 md5","host all all 0.0.0.0/0 md5"]},"bootstrap":{"initdb":[{"auth-host":"md5"},{"auth-local":"trust"},"data-checksums",{"encoding":"UTF8"},{"locale":"en_US.UTF-8"}],"users":{"test":{"password":"","options":["CREATEDB","NOLOGIN"]}},"dcs":{"ttl":30,"loop_wait":10,"retry_timeout":10,"maximum_lag_on_failover":33554432,"postgresql":{"parameters":{"max_connections":"100","max_locks_per_transaction":"64","max_worker_processes":"4"}}}}}`,
},
},
ExpectedResult: false,
},
}
refCase := testCases[0]
for _, testCase := range testCases {
if result := compareEnv(refCase.Envs, testCase.Envs); result != testCase.ExpectedResult {
t.Errorf("expected %v got %v", testCase.ExpectedResult, result)
}
}
}
func TestCrossNamespacedSecrets(t *testing.T) {
testName := "test secrets in different namespace"
clientSet := fake.NewSimpleClientset()

View File

@ -412,13 +412,33 @@ func tolerations(tolerationsSpec *[]v1.Toleration, podToleration map[string]stri
// Those parameters must go to the bootstrap/dcs/postgresql/parameters section.
// See http://patroni.readthedocs.io/en/latest/dynamic_configuration.html.
func isBootstrapOnlyParameter(param string) bool {
return param == "max_connections" ||
param == "max_locks_per_transaction" ||
param == "max_worker_processes" ||
param == "max_prepared_transactions" ||
param == "wal_level" ||
param == "wal_log_hints" ||
param == "track_commit_timestamp"
params := map[string]bool{
"archive_command": false,
"shared_buffers": false,
"logging_collector": false,
"log_destination": false,
"log_directory": false,
"log_filename": false,
"log_file_mode": false,
"log_rotation_age": false,
"log_truncate_on_rotation": false,
"ssl": false,
"ssl_ca_file": false,
"ssl_crl_file": false,
"ssl_cert_file": false,
"ssl_key_file": false,
"shared_preload_libraries": false,
"bg_mon.listen_address": false,
"bg_mon.history_buckets": false,
"pg_stat_statements.track_utility": false,
"extwlist.extensions": false,
"extwlist.custom_path": false,
}
result, ok := params[param]
if !ok {
result = true
}
return result
}
func generateVolumeMounts(volume acidv1.Volume) []v1.VolumeMount {

View File

@ -1207,6 +1207,12 @@ func TestSidecars(t *testing.T) {
}
spec = acidv1.PostgresSpec{
PostgresqlParam: acidv1.PostgresqlParam{
PgVersion: "12.1",
Parameters: map[string]string{
"max_connections": "100",
},
},
TeamID: "myapp", NumberOfInstances: 1,
Resources: acidv1.Resources{
ResourceRequests: acidv1.ResourceDescription{CPU: "1", Memory: "10"},

View File

@ -5,6 +5,7 @@ import (
"fmt"
"regexp"
"strings"
"time"
acidv1 "github.com/zalando/postgres-operator/pkg/apis/acid.zalan.do/v1"
"github.com/zalando/postgres-operator/pkg/spec"
@ -260,6 +261,7 @@ func (c *Cluster) syncPodDisruptionBudget(isUpdate bool) error {
}
func (c *Cluster) syncStatefulSet() error {
var instancesRestartRequired bool
podsToRecreate := make([]v1.Pod, 0)
switchoverCandidates := make([]spec.NamespacedName, 0)
@ -379,10 +381,21 @@ func (c *Cluster) syncStatefulSet() error {
// Apply special PostgreSQL parameters that can only be set via the Patroni API.
// it is important to do it after the statefulset pods are there, but before the rolling update
// since those parameters require PostgreSQL restart.
if err := c.checkAndSetGlobalPostgreSQLConfiguration(); err != nil {
instancesRestartRequired, err = c.checkAndSetGlobalPostgreSQLConfiguration()
if err != nil {
return fmt.Errorf("could not set cluster-wide PostgreSQL configuration options: %v", err)
}
if instancesRestartRequired {
c.logger.Debugln("restarting Postgres server within pods")
c.eventRecorder.Event(c.GetReference(), v1.EventTypeNormal, "Update", "restarting Postgres server within pods")
if err := c.restartInstances(); err != nil {
c.logger.Warningf("could not restart Postgres server within pods: %v", err)
}
c.logger.Infof("Postgres server successfuly restarted on all pods")
c.eventRecorder.Event(c.GetReference(), v1.EventTypeNormal, "Update", "Postgres server restart done - all instances have been restarted")
}
// if we get here we also need to re-create the pods (either leftovers from the old
// statefulset or those that got their configuration from the outdated statefulset)
if len(podsToRecreate) > 0 {
@ -396,6 +409,57 @@ func (c *Cluster) syncStatefulSet() error {
return nil
}
func (c *Cluster) restartInstances() error {
c.setProcessName("starting to restart Postgres servers")
ls := c.labelsSet(false)
namespace := c.Namespace
listOptions := metav1.ListOptions{
LabelSelector: ls.String(),
}
pods, err := c.KubeClient.Pods(namespace).List(context.TODO(), listOptions)
if err != nil {
return fmt.Errorf("could not get the list of pods: %v", err)
}
c.logger.Infof("there are %d pods in the cluster which resquire Postgres server restart", len(pods.Items))
var (
masterPod *v1.Pod
)
for i, pod := range pods.Items {
role := PostgresRole(pod.Labels[c.OpConfig.PodRoleLabel])
if role == Master {
masterPod = &pods.Items[i]
continue
}
podName := util.NameFromMeta(pods.Items[i].ObjectMeta)
config, err := c.patroni.GetConfig(&pod)
if err != nil {
return fmt.Errorf("could not get config for pod %s: %v", podName, err)
}
ttl, ok := config["ttl"].(int32)
if !ok {
ttl = 30
}
if err = c.patroni.Restart(&pod); err != nil {
return fmt.Errorf("could not restart Postgres server on pod %s: %v", podName, err)
}
time.Sleep(time.Duration(ttl) * time.Second)
}
if masterPod != nil {
podName := util.NameFromMeta(masterPod.ObjectMeta)
if err = c.patroni.Restart(masterPod); err != nil {
return fmt.Errorf("could not restart postgres server on masterPod %s: %v", podName, err)
}
}
return nil
}
// AnnotationsToPropagate get the annotations to update if required
// based on the annotations in postgres CRD
func (c *Cluster) AnnotationsToPropagate(annotations map[string]string) map[string]string {
@ -430,10 +494,11 @@ func (c *Cluster) AnnotationsToPropagate(annotations map[string]string) map[stri
// checkAndSetGlobalPostgreSQLConfiguration checks whether cluster-wide API parameters
// (like max_connections) has changed and if necessary sets it via the Patroni API
func (c *Cluster) checkAndSetGlobalPostgreSQLConfiguration() error {
func (c *Cluster) checkAndSetGlobalPostgreSQLConfiguration() (bool, error) {
var (
err error
pods []v1.Pod
err error
pods []v1.Pod
restartRequired bool
)
// we need to extract those options from the cluster manifest.
@ -447,14 +512,14 @@ func (c *Cluster) checkAndSetGlobalPostgreSQLConfiguration() error {
}
if len(optionsToSet) == 0 {
return nil
return restartRequired, nil
}
if pods, err = c.listPods(); err != nil {
return err
return restartRequired, err
}
if len(pods) == 0 {
return fmt.Errorf("could not call Patroni API: cluster has no pods")
return restartRequired, fmt.Errorf("could not call Patroni API: cluster has no pods")
}
// try all pods until the first one that is successful, as it doesn't matter which pod
// carries the request to change configuration through
@ -463,11 +528,12 @@ func (c *Cluster) checkAndSetGlobalPostgreSQLConfiguration() error {
c.logger.Debugf("calling Patroni API on a pod %s to set the following Postgres options: %v",
podName, optionsToSet)
if err = c.patroni.SetPostgresParameters(&pod, optionsToSet); err == nil {
return nil
restartRequired = true
return restartRequired, nil
}
c.logger.Warningf("could not patch postgres parameters with a pod %s: %v", podName, err)
}
return fmt.Errorf("could not reach Patroni API to set Postgres options: failed on every pod (%d total)",
return restartRequired, fmt.Errorf("could not reach Patroni API to set Postgres options: failed on every pod (%d total)",
len(pods))
}

View File

@ -19,6 +19,8 @@ import (
const (
failoverPath = "/failover"
configPath = "/config"
statusPath = "/patroni"
restartPath = "/restart"
apiPort = 8008
timeout = 30 * time.Second
)
@ -28,6 +30,8 @@ type Interface interface {
Switchover(master *v1.Pod, candidate string) error
SetPostgresParameters(server *v1.Pod, options map[string]string) error
GetMemberData(server *v1.Pod) (MemberData, error)
Restart(server *v1.Pod) error
GetConfig(server *v1.Pod) (map[string]interface{}, error)
}
// Patroni API client
@ -103,6 +107,32 @@ func (p *Patroni) httpPostOrPatch(method string, url string, body *bytes.Buffer)
return nil
}
func (p *Patroni) httpGet(url string) (string, error) {
request, err := http.NewRequest("GET", url, nil)
if err != nil {
return "", fmt.Errorf("could not create request: %v", err)
}
p.logger.Debugf("making GET http request: %s", request.URL.String())
resp, err := p.httpClient.Do(request)
if err != nil {
return "", fmt.Errorf("could not make request: %v", err)
}
bodyBytes, err := ioutil.ReadAll(resp.Body)
if err != nil {
return "", fmt.Errorf("could not read response: %v", err)
}
if err := resp.Body.Close(); err != nil {
return "", fmt.Errorf("could not close request: %v", err)
}
if resp.StatusCode != http.StatusOK {
return string(bodyBytes), fmt.Errorf("patroni returned '%d'", resp.StatusCode)
}
return string(bodyBytes), nil
}
// Switchover by calling Patroni REST API
func (p *Patroni) Switchover(master *v1.Pod, candidate string) error {
buf := &bytes.Buffer{}
@ -149,6 +179,48 @@ type MemberData struct {
Patroni MemberDataPatroni `json:"patroni"`
}
func (p *Patroni) GetConfigOrStatus(server *v1.Pod, path string) (map[string]interface{}, error) {
result := make(map[string]interface{})
apiURLString, err := apiURL(server)
if err != nil {
return result, err
}
body, err := p.httpGet(apiURLString + path)
err = json.Unmarshal([]byte(body), &result)
if err != nil {
return result, err
}
return result, err
}
func (p *Patroni) GetStatus(server *v1.Pod) (map[string]interface{}, error) {
return p.GetConfigOrStatus(server, statusPath)
}
func (p *Patroni) GetConfig(server *v1.Pod) (map[string]interface{}, error) {
return p.GetConfigOrStatus(server, configPath)
}
//Restart method restarts instance via Patroni POST API call.
func (p *Patroni) Restart(server *v1.Pod) error {
buf := &bytes.Buffer{}
err := json.NewEncoder(buf).Encode(map[string]interface{}{"restart_pending": true})
if err != nil {
return fmt.Errorf("could not encode json: %v", err)
}
apiURLString, err := apiURL(server)
if err != nil {
return err
}
status, err := p.GetStatus(server)
pending_restart, ok := status["pending_restart"]
if !ok || !pending_restart.(bool) {
return nil
}
return p.httpPostOrPatch(http.MethodPost, apiURLString+restartPath, buf)
}
// GetMemberData read member data from patroni API
func (p *Patroni) GetMemberData(server *v1.Pod) (MemberData, error) {