From 62425e75f4e3e067f18a3aac9eeb6143767efdd3 Mon Sep 17 00:00:00 2001 From: weekface Date: Fri, 22 Feb 2019 18:23:48 +0800 Subject: [PATCH 01/22] add basic actions --- tests/actions.go | 806 ++++++++++++++++++++++++++++++++++++++++++ tests/cmd/e2e/main.go | 14 + 2 files changed, 820 insertions(+) create mode 100644 tests/actions.go create mode 100644 tests/cmd/e2e/main.go diff --git a/tests/actions.go b/tests/actions.go new file mode 100644 index 0000000000..c52f4a568e --- /dev/null +++ b/tests/actions.go @@ -0,0 +1,806 @@ +// Copyright 2018 PingCAP, Inc. +// +// 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, +// See the License for the specific language governing permissions and +// limitations under the License. + +package pkg + +import ( + "database/sql" + "fmt" + "os/exec" + "strconv" + "strings" + "time" + + "github.com/golang/glog" + "github.com/pingcap/kvproto/pkg/metapb" + "github.com/pingcap/tidb-operator/pkg/apis/pingcap.com/v1alpha1" + "github.com/pingcap/tidb-operator/pkg/client/clientset/versioned" + "github.com/pingcap/tidb-operator/pkg/controller" + "github.com/pingcap/tidb-operator/pkg/label" + batchv1 "k8s.io/api/batch/v1" + corev1 "k8s.io/api/core/v1" + metav1 "k8s.io/apimachinery/pkg/apis/meta/v1" + "k8s.io/apimachinery/pkg/labels" + "k8s.io/apimachinery/pkg/util/wait" + "k8s.io/client-go/kubernetes" +) + +func NewOperatorActions(cli versioned.Interface, kubeCli kubernetes.Interface) OperatorActions { + return &operatorActions{ + cli: cli, + kubeCli: kubeCli, + pdControl: controller.NewDefaultPDControl(), + } +} + +type OperatorActions interface { + DeployOperator(info *OperatorInfo) error + CleanOperator(info *OperatorInfo) error + UpgradeOperator(info *OperatorInfo) error + DumpAllLogs(info *OperatorInfo, clusterInfo *TidbClusterInfo) error + DeployTidbCluster(info *TidbClusterInfo) error + CleanTidbCluster(info *TidbClusterInfo) error + CheckTidbClusterStatus(info *TidbClusterInfo) error + BeginInsertDataTo(info *TidbClusterInfo) error + StopInsertDataTo(info *TidbClusterInfo) error + ScaleTidbCluster(info *TidbClusterInfo) error + UpgradeTidbCluster(info *TidbClusterInfo) error + DeployAdHocBackup(info *TidbClusterInfo) error + CleanAdHocBackup(info *TidbClusterInfo) error + DeployScheduledBackup(info *TidbClusterInfo) error + CleanScheduledBackup(info *TidbClusterInfo) error + DeployIncrementalBackup(info *TidbClusterInfo) error + CleanIncrementalBackup(info *TidbClusterInfo) error + Restore(from *TidbClusterInfo, jobName string, to *TidbClusterInfo) error + DeployMonitor(info *TidbClusterInfo) error + CleanMonitor(info *TidbClusterInfo) error +} + +type FaultTriggerActions interface { + StopNode(nodeName string) error + StartNode(nodeName string) error + StopEtcd() error + StartEtcd() error + StopKubeAPIServer() error + StartKubeAPIServer() error + StopKubeControllerManager() error + StartKubeControllerManager() error + StopKubeScheduler() error + StartKubeScheduler() error + StopKubelet(nodeName string) error + StartKubelet(nodeName string) error + StopKubeProxy(nodeName string) error + StartKubeProxy(nodeName string) error + DiskCorruption(nodeName string) error + NetworkPartition(fromNode, toNode string) error + NetworkDelay(fromNode, toNode string) error + DockerCrash(nodeName string) error +} + +type operatorActions struct { + cli versioned.Interface + kubeCli kubernetes.Interface + pdControl controller.PDControlInterface +} + +type OperatorInfo struct { + Namespace string + ReleaseName string + Image string + Tag string + SchedulerImage string + LogLevel string +} + +type TidbClusterInfo struct { + Namespace string + ClusterName string + OperatorTag string + PDImage string + TiKVImage string + TiDBImage string + StorageClassName string + Password string + RecordCount string + InsertBetchSize string + Resources map[string]string + Args map[string]string +} + +func (tc *TidbClusterInfo) HelmSetString() string { + set := map[string]string{ + "clusterName": tc.ClusterName, + "pd.storageClassName": tc.StorageClassName, + "tikv.storageClassName": tc.StorageClassName, + "tidb.storageClassName": tc.StorageClassName, + "tidb.password": tc.Password, + "pd.maxStoreDownTime": "5m", + "pd.image": tc.PDImage, + "tikv.image": tc.TiKVImage, + "tidb.image": tc.TiDBImage, + } + + for k, v := range tc.Resources { + set[k] = v + } + for k, v := range tc.Args { + set[k] = v + } + + arr := make([]string, 0, len(set)) + for k, v := range set { + arr = append(arr, fmt.Sprintf("%s=%s", k, v)) + } + return strings.Join(arr, ",") +} + +func (oa *operatorActions) DeployOperator(info *OperatorInfo) error { + if err := cloneOperatorRepo(); err != nil { + return err + } + if err := checkoutTag(info.Tag); err != nil { + return err + } + + deployCmd := fmt.Sprintf(`helm install /charts/%s/tidb-operator + --name %s + --namespace %s + --set operatorImage=%s + --set controllerManager.autoFailover=true + --set scheduler.kubeSchedulerImage=%s + --set controllerManager.logLevel=%d + --set scheduler.logLevel=4`, + info.Tag, + info.ReleaseName, + info.Namespace, + info.Image, + info.SchedulerImage, + info.LogLevel) + res, err := exec.Command("/bin/sh", "-c", deployCmd).CombinedOutput() + if err != nil { + return fmt.Errorf("failed to deploy operator: %v, %s", err, string(res)) + } + + return nil +} + +func (oa *operatorActions) CleanOperator(info *OperatorInfo) error { + res, err := exec.Command("helm", "del", "--purge", info.ReleaseName).CombinedOutput() + if err == nil || releaseIsNotFound(err) { + return nil + } + return fmt.Errorf("failed to clear operator: %v, %s", err, string(res)) +} + +func (oa *operatorActions) UpgradeOperator(info *OperatorInfo) error { + if err := checkoutTag(info.Tag); err != nil { + return err + } + + cmd := fmt.Sprintf(`helm upgrade %s /charts/%s/tidb-operator + --set operatorImage=%s`, + info.ReleaseName, info.Tag, + info.Image) + res, err := exec.Command("/bin/sh", "-c", cmd).CombinedOutput() + if err != nil { + return fmt.Errorf("failed to upgrade operator to: %s, %v, %s", info.Image, err, string(res)) + } + return nil +} + +func (oa *operatorActions) DumpAllLogs(info *OperatorInfo, clusterInfo *TidbClusterInfo) error { + return nil +} + +func (oa *operatorActions) DeployTidbCluster(info *TidbClusterInfo) error { + cmd := fmt.Sprintf("helm install /charts/%s/tidb-cluster --name %s --namespace %s --set-string %s", + info.OperatorTag, info.ClusterName, info.Namespace, info.HelmSetString()) + if res, err := exec.Command("/bin/sh", "-c", cmd).CombinedOutput(); err != nil { + return fmt.Errorf("failed to deploy tidbcluster: %s/%s, %v, %s", + info.Namespace, info.ClusterName, err, string(res)) + } + + return nil +} + +func (oa *operatorActions) CleanTidbCluster(info *TidbClusterInfo) error { + charts := []string{ + info.ClusterName, + fmt.Sprintf("%s-backup", info.ClusterName), + fmt.Sprintf("%s-restore", info.ClusterName), + } + for _, chartName := range charts { + res, err := exec.Command("helm", "del", "--purge", chartName).CombinedOutput() + if err != nil && !releaseIsNotFound(err) { + return fmt.Errorf("failed to delete chart: %s/%s, %v, %s", + info.Namespace, chartName, err, string(res)) + } + } + + resources := []string{"cronjobs", "jobs", "pods", "pvcs"} + for _, resource := range resources { + if res, err := exec.Command("kubectl", "delete", resource, "-n", info.Namespace, + "--all").CombinedOutput(); err != nil { + return fmt.Errorf("failed to delete %s: %v, %s", resource, err, string(res)) + } + } + + patchPVCmd := fmt.Sprintf(`kubectl get pv -l %s=%s,%s=%s --output=name | xargs -I {} + kubectl patch {} -p '{"spec":{"persistentVolumeReclaimPolicy":"Delete"}}'`, + label.NamespaceLabelKey, info.Namespace, label.InstanceLabelKey, info.ClusterName) + if res, err := exec.Command("/bin/sh", "-c", patchPVCmd).CombinedOutput(); err != nil { + return fmt.Errorf("failed to patch pv: %v, %s", err, string(res)) + } + + pollFn := func() (bool, error) { + if res, err := exec.Command("kubectl", "get", "po", "--output=name", "-n", info.Namespace). + CombinedOutput(); err != nil || len(res) == 0 { + glog.Infof("waiting for tidbcluster: %s/%s pods deleting\n", + info.Namespace, info.ClusterName, string(res)) + return false, nil + } + + pvCmd := fmt.Sprintf("kubectl get pv -l %s=%s,%s=%s 2>/dev/null|grep Released", + label.NamespaceLabelKey, info.Namespace, label.InstanceLabelKey, info.ClusterName) + if res, err := exec.Command("/bin/sh", "-c", pvCmd). + CombinedOutput(); err != nil || len(res) != 0 { + glog.Infof("waiting for tidbcluster: %s/%s pvs deleting\n%s", + info.Namespace, info.ClusterName, string(res)) + return false, nil + } + + return true, nil + } + return wait.PollImmediate(1*time.Minute, 5*time.Minute, pollFn) +} + +func (oa *operatorActions) CheckTidbClusterStatus(info *TidbClusterInfo) error { + ns := info.Namespace + tcName := info.ClusterName + var tc *v1alpha1.TidbCluster + var err error + if tc, err = oa.cli.PingcapV1alpha1().TidbClusters(ns).Get(tcName, metav1.GetOptions{}); err != nil { + return fmt.Errorf("failed to get tidbcluster: %s/%s, %v", ns, tcName, err) + } + + if err := wait.PollImmediate(1*time.Minute, 5*time.Minute, func() (bool, error) { + if b, err := oa.pdMembersReadyFn(tc); !b && err == nil { + return false, nil + } + if b, err := oa.tikvMembersReadyFn(tc); !b && err == nil { + return false, nil + } + if b, err := oa.tidbMembersReadyFn(tc); !b && err == nil { + return false, nil + } + if b, err := oa.reclaimPolicySyncFn(tc); !b && err == nil { + return false, nil + } + if b, err := oa.metaSyncFn(tc); err != nil { + return false, err + } else if !b && err == nil { + return false, nil + } + if b, err := oa.schedulerHAFn(tc); !b && err == nil { + return false, nil + } + if b, err := oa.passwordIsSet(info); !b && err == nil { + return false, nil + } + + return true, nil + }); err != nil { + return fmt.Errorf("failed to waiting for tidbcluster %s/%s ready in 5 minutes", ns, tcName) + } + + return nil +} + +func (oa *operatorActions) BeginInsertDataTo(info *TidbClusterInfo) error { + return nil +} + +func (oa *operatorActions) StopInsertDataTo(info *TidbClusterInfo) error { + return nil +} + +func (oa *operatorActions) ScaleTidbCluster(info *TidbClusterInfo) error { return nil } +func (oa *operatorActions) UpgradeTidbCluster(info *TidbClusterInfo) error { return nil } +func (oa *operatorActions) DeployAdHocBackup(info *TidbClusterInfo) error { return nil } +func (oa *operatorActions) CleanAdHocBackup(info *TidbClusterInfo) error { return nil } +func (oa *operatorActions) DeployScheduledBackup(info *TidbClusterInfo) error { return nil } +func (oa *operatorActions) CleanScheduledBackup(info *TidbClusterInfo) error { return nil } +func (oa *operatorActions) DeployIncrementalBackup(info *TidbClusterInfo) error { return nil } +func (oa *operatorActions) CleanIncrementalBackup(info *TidbClusterInfo) error { return nil } +func (oa *operatorActions) Restore(from *TidbClusterInfo, jobName string, to *TidbClusterInfo) error { + return nil +} +func (oa *operatorActions) DeployMonitor(info *TidbClusterInfo) error { return nil } +func (oa *operatorActions) CleanMonitor(info *TidbClusterInfo) error { return nil } + +func (oa *operatorActions) pdMembersReadyFn(tc *v1alpha1.TidbCluster) (bool, error) { + tcName := tc.GetName() + ns := tc.GetNamespace() + pdSetName := controller.PDMemberName(tcName) + + pdSet, err := oa.kubeCli.AppsV1beta1().StatefulSets(ns).Get(pdSetName, metav1.GetOptions{}) + if err != nil { + glog.Errorf("failed to get statefulset: %s/%s, %v", ns, pdSetName, err) + return false, nil + } + + if tc.Status.PD.StatefulSet == nil { + glog.Infof("tidbcluster: %s/%s .status.PD.StatefulSet is nil", ns, tcName) + return false, nil + } + failureCount := len(tc.Status.PD.FailureMembers) + replicas := tc.Spec.PD.Replicas + int32(failureCount) + if *pdSet.Spec.Replicas != replicas { + glog.Infof("statefulset: %s/%s .spec.Replicas(%d) != %d", + ns, pdSetName, *pdSet.Spec.Replicas, ns, tcName, replicas) + return false, nil + } + if pdSet.Status.ReadyReplicas != tc.Spec.PD.Replicas { + glog.Infof("statefulset: %s/%s .status.ReadyReplicas(%d) != %d", + ns, pdSetName, pdSet.Status.ReadyReplicas, tc.Spec.PD.Replicas) + return false, nil + } + if len(tc.Status.PD.Members) != int(tc.Spec.PD.Replicas) { + glog.Infof("tidbcluster: %s/%s .status.PD.Members count(%d) != %d", + ns, tcName, len(tc.Status.PD.Members), tc.Spec.PD.Replicas) + return false, nil + } + if pdSet.Status.ReadyReplicas != pdSet.Status.Replicas { + glog.Infof("statefulset: %s/%s .status.ReadyReplicas(%d) != .status.Replicas(%d)", + ns, pdSetName, pdSet.Status.ReadyReplicas, pdSet.Status.Replicas) + return false, nil + } + + for _, member := range tc.Status.PD.Members { + if !member.Health { + glog.Infof("tidbcluster: %s/%s pd member(%s/%s) is not health", + ns, tcName, member.ID, member.Name) + return false, nil + } + } + + pdServiceName := controller.PDMemberName(tcName) + pdPeerServiceName := controller.PDPeerMemberName(tcName) + if _, err := oa.kubeCli.CoreV1().Services(ns).Get(pdServiceName, metav1.GetOptions{}); err != nil { + glog.Errorf("failed to get service: %s/%s", ns, pdServiceName) + return false, nil + } + if _, err := oa.kubeCli.CoreV1().Services(ns).Get(pdPeerServiceName, metav1.GetOptions{}); err != nil { + glog.Errorf("failed to get peer service: %s/%s", ns, pdPeerServiceName) + return false, nil + } + + return true, nil +} + +func (oa *operatorActions) tikvMembersReadyFn(tc *v1alpha1.TidbCluster) (bool, error) { + tcName := tc.GetName() + ns := tc.GetNamespace() + tikvSetName := controller.TiKVMemberName(tcName) + + tikvSet, err := oa.kubeCli.AppsV1beta1().StatefulSets(ns).Get(tikvSetName, metav1.GetOptions{}) + if err != nil { + glog.Errorf("failed to get statefulset: %s/%s, %v", ns, tikvSetName, err) + return false, nil + } + + if tc.Status.TiKV.StatefulSet == nil { + glog.Infof("tidbcluster: %s/%s .status.TiKV.StatefulSet is nil", ns, tcName) + return false, nil + } + failureCount := len(tc.Status.TiKV.FailureStores) + replicas := tc.Spec.TiKV.Replicas + int32(failureCount) + if *tikvSet.Spec.Replicas != replicas { + glog.Infof("statefulset: %s/%s .spec.Replicas(%d) != %d", + ns, tikvSetName, *tikvSet.Spec.Replicas, replicas) + return false, nil + } + if tikvSet.Status.ReadyReplicas != replicas { + glog.Infof("statefulset: %s/%s .status.ReadyReplicas(%d) != %d", + ns, tikvSetName, tikvSet.Status.ReadyReplicas, replicas) + return false, nil + } + if len(tc.Status.TiKV.Stores) != int(replicas) { + glog.Infof("tidbcluster: %s/%s .status.TiKV.Stores.count(%d) != %d", + ns, tcName, len(tc.Status.TiKV.Stores), tc.Spec.TiKV.Replicas) + return false, nil + } + if tikvSet.Status.ReadyReplicas != tikvSet.Status.Replicas { + glog.Infof("statefulset: %s/%s .status.ReadyReplicas(%d) != .status.Replicas(%d)", + ns, tikvSetName, tikvSet.Status.ReadyReplicas, tikvSet.Status.Replicas) + return false, nil + } + + for _, store := range tc.Status.TiKV.Stores { + if store.State != v1alpha1.TiKVStateUp { + glog.Infof("tidbcluster: %s/%s's store(%s) state != %s", ns, tcName, store.ID, v1alpha1.TiKVStateUp) + return false, nil + } + } + + tikvPeerServiceName := controller.TiKVPeerMemberName(tcName) + if _, err := oa.kubeCli.CoreV1().Services(ns).Get(tikvPeerServiceName, metav1.GetOptions{}); err != nil { + glog.Errorf("failed to get peer service: %s/%s", ns, tikvPeerServiceName) + return false, nil + } + + return true, nil +} + +func (oa *operatorActions) tidbMembersReadyFn(tc *v1alpha1.TidbCluster) (bool, error) { + tcName := tc.GetName() + ns := tc.GetNamespace() + tidbSetName := controller.TiDBMemberName(tcName) + + tidbSet, err := oa.kubeCli.AppsV1beta1().StatefulSets(ns).Get(tidbSetName, metav1.GetOptions{}) + if err != nil { + glog.Errorf("failed to get statefulset: %s/%s, %v", ns, tidbSetName, err) + return false, nil + } + + if tc.Status.TiDB.StatefulSet == nil { + glog.Infof("tidbcluster: %s/%s .status.TiDB.StatefulSet is nil", ns, tcName) + return false, nil + } + failureCount := len(tc.Status.TiDB.FailureMembers) + replicas := tc.Spec.TiDB.Replicas + int32(failureCount) + if *tidbSet.Spec.Replicas != replicas { + glog.Infof("statefulset: %s/%s .spec.Replicas(%d) != %d", + ns, tidbSetName, *tidbSet.Spec.Replicas, replicas) + return false, nil + } + if tidbSet.Status.ReadyReplicas != tc.Spec.TiDB.Replicas { + glog.Infof("statefulset: %s/%s .status.ReadyReplicas(%d) != %d", + ns, tidbSetName, tidbSet.Status.ReadyReplicas, replicas) + return false, nil + } + if tidbSet.Status.ReadyReplicas != tidbSet.Status.Replicas { + glog.Infof("statefulset: %s/%s .status.ReadyReplicas(%d) != .status.Replicas(%d)", + ns, tidbSetName, tidbSet.Status.ReadyReplicas, tidbSet.Status.Replicas) + return false, nil + } + + _, err = oa.kubeCli.CoreV1().Services(ns).Get(tidbSetName, metav1.GetOptions{}) + if err != nil { + glog.Errorf("failed to get service: %s/%s", ns, tidbSetName) + return false, nil + } + + return true, nil +} + +func (oa *operatorActions) reclaimPolicySyncFn(tc *v1alpha1.TidbCluster) (bool, error) { + ns := tc.GetNamespace() + tcName := tc.GetName() + listOptions := metav1.ListOptions{ + LabelSelector: labels.SelectorFromSet( + label.New().Instance(tcName).Labels(), + ).String(), + } + var pvcList *corev1.PersistentVolumeClaimList + var err error + if pvcList, err = oa.kubeCli.CoreV1().PersistentVolumeClaims(ns).List(listOptions); err != nil { + glog.Errorf("failed to list pvs for tidbcluster %s/%s, %v", ns, tcName, err) + return false, nil + } + + for _, pvc := range pvcList.Items { + pvName := pvc.Spec.VolumeName + if pv, err := oa.kubeCli.CoreV1().PersistentVolumes().Get(pvName, metav1.GetOptions{}); err != nil { + glog.Errorf("failed to get pv: %s", pvName, err) + return false, nil + } else if pv.Spec.PersistentVolumeReclaimPolicy != tc.Spec.PVReclaimPolicy { + glog.Errorf("pv: %s's reclaimPolicy is not Retain", pvName) + return false, nil + } + } + + return true, nil +} + +func (oa *operatorActions) metaSyncFn(tc *v1alpha1.TidbCluster) (bool, error) { + ns := tc.GetNamespace() + tcName := tc.GetName() + + pdCli := oa.pdControl.GetPDClient(tc) + var cluster *metapb.Cluster + var err error + if cluster, err = pdCli.GetCluster(); err != nil { + glog.Errorf("failed to get cluster from pdControl: %s/%s", ns, tcName, err) + return false, nil + } + + clusterID := strconv.FormatUint(cluster.Id, 10) + listOptions := metav1.ListOptions{ + LabelSelector: labels.SelectorFromSet( + label.New().Instance(tcName).Labels(), + ).String(), + } + + var podList *corev1.PodList + if podList, err = oa.kubeCli.CoreV1().Pods(ns).List(listOptions); err != nil { + glog.Errorf("failed to list pods for tidbcluster %s/%s, %v", ns, tcName, err) + return false, nil + } + +outerLoop: + for _, pod := range podList.Items { + podName := pod.GetName() + if pod.Labels[label.ClusterIDLabelKey] != clusterID { + return false, fmt.Errorf("tidbcluster %s/%s's pod %s's label %s not equals %s ", + ns, tcName, podName, label.ClusterIDLabelKey, clusterID) + } + + component := pod.Labels[label.ComponentLabelKey] + switch component { + case label.PDLabelVal: + var memberID string + members, err := pdCli.GetMembers() + if err != nil { + glog.Errorf("failed to get members for tidbcluster %s/%s, %v", ns, tcName, err) + return false, nil + } + for _, member := range members.Members { + if member.Name == podName { + memberID = strconv.FormatUint(member.GetMemberId(), 10) + break + } + } + if memberID == "" { + glog.Errorf("tidbcluster: %s/%s's pod %s label [%s] is empty", + ns, tcName, podName, label.MemberIDLabelKey) + return false, nil + } + if pod.Labels[label.MemberIDLabelKey] != memberID { + return false, fmt.Errorf("tidbcluster: %s/%s's pod %s label [%s] not equals %s", + ns, tcName, podName, label.MemberIDLabelKey, memberID) + } + case label.TiKVLabelVal: + var storeID string + stores, err := pdCli.GetStores() + if err != nil { + glog.Errorf("failed to get stores for tidbcluster %s/%s, %v", ns, tcName, err) + return false, nil + } + for _, store := range stores.Stores { + addr := store.Store.GetAddress() + if strings.Split(addr, ".")[0] == podName { + storeID = strconv.FormatUint(store.Store.GetId(), 10) + break + } + } + if storeID == "" { + glog.Errorf("tidbcluster: %s/%s's pod %s label [%s] is empty", + tc.GetNamespace(), tc.GetName(), podName, label.StoreIDLabelKey) + return false, nil + } + if pod.Labels[label.StoreIDLabelKey] != storeID { + return false, fmt.Errorf("tidbcluster: %s/%s's pod %s label [%s] not equals %s", + ns, tcName, podName, label.StoreIDLabelKey, storeID) + } + case label.TiDBLabelVal: + continue outerLoop + default: + continue outerLoop + } + + var pvcName string + for _, vol := range pod.Spec.Volumes { + if vol.PersistentVolumeClaim != nil { + pvcName = vol.PersistentVolumeClaim.ClaimName + break + } + } + if pvcName == "" { + return false, fmt.Errorf("pod: %s/%s's pvcName is empty", ns, podName) + } + + var pvc *corev1.PersistentVolumeClaim + if pvc, err = oa.kubeCli.CoreV1().PersistentVolumeClaims(ns).Get(pvcName, metav1.GetOptions{}); err != nil { + glog.Errorf("failed to get pvc %s/%s for pod %s/%s", ns, pvcName, ns, podName) + return false, nil + } + if pvc.Labels[label.ClusterIDLabelKey] != clusterID { + return false, fmt.Errorf("tidbcluster: %s/%s's pvc %s label [%s] not equals %s ", + ns, tcName, pvcName, label.ClusterIDLabelKey, clusterID) + } + if pvc.Labels[label.MemberIDLabelKey] != pod.Labels[label.MemberIDLabelKey] { + return false, fmt.Errorf("tidbcluster: %s/%s's pvc %s label [%s=%s] not equals pod lablel [%s=%s]", + ns, tcName, pvcName, + label.MemberIDLabelKey, pvc.Labels[label.MemberIDLabelKey], + label.MemberIDLabelKey, pod.Labels[label.MemberIDLabelKey]) + } + if pvc.Labels[label.StoreIDLabelKey] != pod.Labels[label.StoreIDLabelKey] { + return false, fmt.Errorf("tidbcluster: %s/%s's pvc %s label[%s=%s] not equals pod lable[%s=%s]", + ns, tcName, pvcName, + label.StoreIDLabelKey, pvc.Labels[label.StoreIDLabelKey], + label.StoreIDLabelKey, pod.Labels[label.StoreIDLabelKey]) + } + if pvc.Annotations[label.AnnPodNameKey] != podName { + return false, fmt.Errorf("tidbcluster: %s/%s's pvc %s annotations [%s] not equals podName: %s", + ns, tcName, pvcName, label.AnnPodNameKey, podName) + } + + pvName := pvc.Spec.VolumeName + var pv *corev1.PersistentVolume + if pv, err = oa.kubeCli.CoreV1().PersistentVolumes().Get(pvName, metav1.GetOptions{}); err != nil { + glog.Errorf("failed to get pv for pvc %s/%s, %v", ns, pvcName, err) + return false, nil + } + if pv.Labels[label.NamespaceLabelKey] != ns { + return false, fmt.Errorf("tidbcluster: %s/%s 's pv %s label [%s] not equals %s", + ns, tcName, pvName, label.NamespaceLabelKey, ns) + } + if pv.Labels[label.ComponentLabelKey] != pod.Labels[label.ComponentLabelKey] { + return false, fmt.Errorf("tidbcluster: %s/%s's pv %s label [%s=%s] not equals pod label[%s=%s]", + ns, tcName, pvName, + label.ComponentLabelKey, pv.Labels[label.ComponentLabelKey], + label.ComponentLabelKey, pod.Labels[label.ComponentLabelKey]) + } + if pv.Labels[label.NameLabelKey] != pod.Labels[label.NameLabelKey] { + return false, fmt.Errorf("tidbcluster: %s/%s's pv %s label [%s=%s] not equals pod label [%s=%s]", + ns, tcName, pvName, + label.NameLabelKey, pv.Labels[label.NameLabelKey], + label.NameLabelKey, pod.Labels[label.NameLabelKey]) + } + if pv.Labels[label.ManagedByLabelKey] != pod.Labels[label.ManagedByLabelKey] { + return false, fmt.Errorf("tidbcluster: %s/%s's pv %s label [%s=%s] not equals pod label [%s=%s]", + ns, tcName, pvName, + label.ManagedByLabelKey, pv.Labels[label.ManagedByLabelKey], + label.ManagedByLabelKey, pod.Labels[label.ManagedByLabelKey]) + } + if pv.Labels[label.InstanceLabelKey] != pod.Labels[label.InstanceLabelKey] { + return false, fmt.Errorf("tidbcluster: %s/%s's pv %s label [%s=%s] not equals pod label [%s=%s]", + ns, tcName, pvName, + label.InstanceLabelKey, pv.Labels[label.InstanceLabelKey], + label.InstanceLabelKey, pod.Labels[label.InstanceLabelKey]) + } + if pv.Labels[label.ClusterIDLabelKey] != clusterID { + return false, fmt.Errorf("tidbcluster: %s/%s's pv %s label [%s] not equals %s", + ns, tcName, pvName, label.ClusterIDLabelKey, clusterID) + } + if pv.Labels[label.MemberIDLabelKey] != pod.Labels[label.MemberIDLabelKey] { + return false, fmt.Errorf("tidbcluster: %s/%s's pv %s label [%s=%s] not equals pod label [%s=%s]", + ns, tcName, pvName, + label.MemberIDLabelKey, pv.Labels[label.MemberIDLabelKey], + label.MemberIDLabelKey, pod.Labels[label.MemberIDLabelKey]) + } + if pv.Labels[label.StoreIDLabelKey] != pod.Labels[label.StoreIDLabelKey] { + return false, fmt.Errorf("tidbcluster: %s/%s's pv %s label [%s=%s] not equals pod label [%s=%s]", + ns, tcName, pvName, + label.StoreIDLabelKey, pv.Labels[label.StoreIDLabelKey], + label.StoreIDLabelKey, pod.Labels[label.StoreIDLabelKey]) + } + if pv.Annotations[label.AnnPodNameKey] != podName { + return false, fmt.Errorf("tidbcluster:[%s/%s's pv %s annotations [%s] not equals %s", + ns, tcName, pvName, label.AnnPodNameKey, podName) + } + } + + return true, nil +} + +func (oa *operatorActions) schedulerHAFn(tc *v1alpha1.TidbCluster) (bool, error) { + ns := tc.GetNamespace() + tcName := tc.GetName() + + fn := func(component string) (bool, error) { + nodeMap := make(map[string][]string) + listOptions := metav1.ListOptions{ + LabelSelector: labels.SelectorFromSet( + label.New().Instance(tcName).Component(component).Labels()).String(), + } + var podList *corev1.PodList + var err error + if podList, err = oa.kubeCli.CoreV1().Pods(ns).List(listOptions); err != nil { + glog.Errorf("failed to list pods for tidbcluster %s/%s, %v", ns, tcName, err) + return false, nil + } + + totalCount := len(podList.Items) + for _, pod := range podList.Items { + nodeName := pod.Spec.NodeName + if len(nodeMap[nodeName]) == 0 { + nodeMap[nodeName] = make([]string, 0) + } + nodeMap[nodeName] = append(nodeMap[nodeName], pod.GetName()) + if len(nodeMap[nodeName]) > totalCount/2 { + return false, fmt.Errorf("node % have %d pods, greater than %d/2", + nodeName, len(nodeMap[nodeName]), totalCount) + } + } + return false, nil + } + + components := []string{label.PDLabelVal, label.TiKVLabelVal} + for _, com := range components { + if b, err := fn(com); err != nil { + return false, err + } else if !b && err == nil { + return false, nil + } + } + + return true, nil +} + +func (oa *operatorActions) passwordIsSet(clusterInfo *TidbClusterInfo) (bool, error) { + ns := clusterInfo.Namespace + tcName := clusterInfo.ClusterName + jobName := tcName + "-tidb-initializer" + + var job *batchv1.Job + var err error + if job, err = oa.kubeCli.BatchV1().Jobs(ns).Get(jobName, metav1.GetOptions{}); err != nil { + glog.Errorf("failed to get job %s/%s, %v", ns, jobName, err) + return false, nil + } + if job.Status.Succeeded < 1 { + glog.Errorf("tidbcluster: %s/%s password setter job not finished", ns, tcName) + return false, nil + } + + var db *sql.DB + dsn := getDSN(ns, tcName, "test", clusterInfo.Password) + if db, err = sql.Open("mysql", dsn); err != nil { + glog.Errorf("can't open connection to mysql: %s, %v", dsn, err) + return false, nil + } + defer db.Close() + if err := db.Ping(); err != nil { + glog.Errorf("can't connect to mysql: %s with password %s, %v", dsn, clusterInfo.Password, err) + return false, nil + } + + return true, nil +} + +func getDSN(ns, tcName, databaseName, password string) string { + return fmt.Sprintf("root:%s@(%s-tidb.%s:4000)/%s?charset=utf8", password, tcName, ns, databaseName) +} + +func releaseIsNotFound(err error) bool { + return strings.Contains(err.Error(), "not found") +} + +func cloneOperatorRepo() error { + cloneCmd := fmt.Sprintf("git clone https://github.com/pingcap/tidb-operator.git /tidb-operator") + res, err := exec.Command("/bin/sh", "-c", cloneCmd).CombinedOutput() + if err != nil { + return fmt.Errorf("failed to clone tidb-operator repository: %v, %s", err, string(res)) + } + + return nil +} + +func checkoutTag(tagName string) error { + cmd := fmt.Sprintf(`cd /tidb-operator; + git stash -u; + git checkout %s; + mkdir -p /charts/%s; + cp -rf charts/tidb-operator /charts/%s/tidb-operator; + cp -rf charts/tidb-cluster /charts/%s/tidb-cluster; + cp -rf charts/tidb-backup /charts/%s/tidb-backup;`, + tagName, tagName, tagName, tagName, tagName) + res, err := exec.Command("/bin/sh", "-c", cmd).CombinedOutput() + if err != nil { + return fmt.Errorf("failed to check tag: %s, %v, %s", tagName, err, string(res)) + } + + return nil +} diff --git a/tests/cmd/e2e/main.go b/tests/cmd/e2e/main.go new file mode 100644 index 0000000000..9ea6d00fd7 --- /dev/null +++ b/tests/cmd/e2e/main.go @@ -0,0 +1,14 @@ +// Copyright 2018 PingCAP, Inc. +// +// 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, +// See the License for the specific language governing permissions and +// limitations under the License.package spec + +package main From 3a47c95fd4401beb142c574b3ef70085816f5368 Mon Sep 17 00:00:00 2001 From: weekface Date: Thu, 7 Mar 2019 19:24:29 +0800 Subject: [PATCH 02/22] add e2e main func --- Makefile | 10 ++---- tests/actions.go | 67 ++++++++++++++++++++---------------- tests/cmd/e2e/main.go | 68 +++++++++++++++++++++++++++++++++++++ tests/images/e2e/Dockerfile | 17 ++++++++++ tests/manifests/e2e.yaml | 33 ++++++++++++++++++ 5 files changed, 157 insertions(+), 38 deletions(-) create mode 100644 tests/images/e2e/Dockerfile create mode 100644 tests/manifests/e2e.yaml diff --git a/Makefile b/Makefile index 9f4d14928c..016770f332 100644 --- a/Makefile +++ b/Makefile @@ -42,16 +42,10 @@ e2e-docker-push: e2e-docker docker push "${DOCKER_REGISTRY}/pingcap/tidb-operator-e2e:latest" e2e-docker: e2e-build - mkdir -p images/tidb-operator-e2e/bin - mv tests/e2e/e2e.test images/tidb-operator-e2e/bin/ - [[ -d images/tidb-operator-e2e/tidb-operator ]] && rm -r images/tidb-operator-e2e/tidb-operator || true - [[ -d images/tidb-operator-e2e/tidb-cluster ]] && rm -r images/tidb-operator-e2e/tidb-cluster || true - cp -r charts/tidb-operator images/tidb-operator-e2e/ - cp -r charts/tidb-cluster images/tidb-operator-e2e/ - docker build -t "${DOCKER_REGISTRY}/pingcap/tidb-operator-e2e:latest" images/tidb-operator-e2e + docker build -t "${DOCKER_REGISTRY}/pingcap/tidb-operator-e2e:latest" tests/images/e2e e2e-build: - $(GOENV) ginkgo build tests/e2e + $(GO) -ldflags '$(LDFLAGS)' -o tests/images/e2e/bin/e2e tests/cmd/e2e/main.go test: @echo "Run unit tests" diff --git a/tests/actions.go b/tests/actions.go index c52f4a568e..6636337c22 100644 --- a/tests/actions.go +++ b/tests/actions.go @@ -11,7 +11,7 @@ // See the License for the specific language governing permissions and // limitations under the License. -package pkg +package tests import ( "database/sql" @@ -152,21 +152,22 @@ func (oa *operatorActions) DeployOperator(info *OperatorInfo) error { return err } - deployCmd := fmt.Sprintf(`helm install /charts/%s/tidb-operator - --name %s - --namespace %s - --set operatorImage=%s - --set controllerManager.autoFailover=true - --set scheduler.kubeSchedulerImage=%s - --set controllerManager.logLevel=%d - --set scheduler.logLevel=4`, + cmd := fmt.Sprintf(`helm install /charts/%s/tidb-operator \ + --name %s \ + --namespace %s \ + --set operatorImage=%s \ + --set controllerManager.autoFailover=true \ + --set scheduler.kubeSchedulerImage=%s \ + --set controllerManager.logLevel=%s \ + --set scheduler.logLevel=2`, info.Tag, info.ReleaseName, info.Namespace, info.Image, info.SchedulerImage, info.LogLevel) - res, err := exec.Command("/bin/sh", "-c", deployCmd).CombinedOutput() + glog.Info(cmd) + res, err := exec.Command("/bin/sh", "-c", cmd).CombinedOutput() if err != nil { return fmt.Errorf("failed to deploy operator: %v, %s", err, string(res)) } @@ -176,7 +177,7 @@ func (oa *operatorActions) DeployOperator(info *OperatorInfo) error { func (oa *operatorActions) CleanOperator(info *OperatorInfo) error { res, err := exec.Command("helm", "del", "--purge", info.ReleaseName).CombinedOutput() - if err == nil || releaseIsNotFound(err) { + if err == nil || !releaseIsNotFound(err) { return nil } return fmt.Errorf("failed to clear operator: %v, %s", err, string(res)) @@ -221,13 +222,13 @@ func (oa *operatorActions) CleanTidbCluster(info *TidbClusterInfo) error { } for _, chartName := range charts { res, err := exec.Command("helm", "del", "--purge", chartName).CombinedOutput() - if err != nil && !releaseIsNotFound(err) { + if err != nil && releaseIsNotFound(err) { return fmt.Errorf("failed to delete chart: %s/%s, %v, %s", info.Namespace, chartName, err, string(res)) } } - resources := []string{"cronjobs", "jobs", "pods", "pvcs"} + resources := []string{"cronjobs", "jobs", "pods", "pvc"} for _, resource := range resources { if res, err := exec.Command("kubectl", "delete", resource, "-n", info.Namespace, "--all").CombinedOutput(); err != nil { @@ -235,27 +236,30 @@ func (oa *operatorActions) CleanTidbCluster(info *TidbClusterInfo) error { } } - patchPVCmd := fmt.Sprintf(`kubectl get pv -l %s=%s,%s=%s --output=name | xargs -I {} + patchPVCmd := fmt.Sprintf(`kubectl get pv -l %s=%s,%s=%s --output=name | xargs -I {} \ kubectl patch {} -p '{"spec":{"persistentVolumeReclaimPolicy":"Delete"}}'`, label.NamespaceLabelKey, info.Namespace, label.InstanceLabelKey, info.ClusterName) + glog.Info(patchPVCmd) if res, err := exec.Command("/bin/sh", "-c", patchPVCmd).CombinedOutput(); err != nil { return fmt.Errorf("failed to patch pv: %v, %s", err, string(res)) } pollFn := func() (bool, error) { if res, err := exec.Command("kubectl", "get", "po", "--output=name", "-n", info.Namespace). - CombinedOutput(); err != nil || len(res) == 0 { - glog.Infof("waiting for tidbcluster: %s/%s pods deleting\n", - info.Namespace, info.ClusterName, string(res)) + CombinedOutput(); err != nil || len(res) != 0 { + glog.Infof("waiting for tidbcluster: %s/%s pods deleting, %v, [%s]", + info.Namespace, info.ClusterName, err, string(res)) return false, nil } pvCmd := fmt.Sprintf("kubectl get pv -l %s=%s,%s=%s 2>/dev/null|grep Released", label.NamespaceLabelKey, info.Namespace, label.InstanceLabelKey, info.ClusterName) + glog.Info(pvCmd) if res, err := exec.Command("/bin/sh", "-c", pvCmd). - CombinedOutput(); err != nil || len(res) != 0 { - glog.Infof("waiting for tidbcluster: %s/%s pvs deleting\n%s", - info.Namespace, info.ClusterName, string(res)) + CombinedOutput(); len(res) == 0 { + } else if err != nil { + glog.Infof("waiting for tidbcluster: %s/%s pv deleting, %v, %s", + info.Namespace, info.ClusterName, err, string(res)) return false, nil } @@ -267,13 +271,14 @@ func (oa *operatorActions) CleanTidbCluster(info *TidbClusterInfo) error { func (oa *operatorActions) CheckTidbClusterStatus(info *TidbClusterInfo) error { ns := info.Namespace tcName := info.ClusterName - var tc *v1alpha1.TidbCluster - var err error - if tc, err = oa.cli.PingcapV1alpha1().TidbClusters(ns).Get(tcName, metav1.GetOptions{}); err != nil { - return fmt.Errorf("failed to get tidbcluster: %s/%s, %v", ns, tcName, err) - } + if err := wait.PollImmediate(1*time.Minute, 10*time.Minute, func() (bool, error) { + var tc *v1alpha1.TidbCluster + var err error + if tc, err = oa.cli.PingcapV1alpha1().TidbClusters(ns).Get(tcName, metav1.GetOptions{}); err != nil { + glog.Errorf("failed to get tidbcluster: %s/%s, %v", ns, tcName, err) + return false, nil + } - if err := wait.PollImmediate(1*time.Minute, 5*time.Minute, func() (bool, error) { if b, err := oa.pdMembersReadyFn(tc); !b && err == nil { return false, nil } @@ -300,7 +305,7 @@ func (oa *operatorActions) CheckTidbClusterStatus(info *TidbClusterInfo) error { return true, nil }); err != nil { - return fmt.Errorf("failed to waiting for tidbcluster %s/%s ready in 5 minutes", ns, tcName) + return fmt.Errorf("failed to waiting for tidbcluster %s/%s ready in 10 minutes", ns, tcName) } return nil @@ -779,8 +784,9 @@ func releaseIsNotFound(err error) bool { } func cloneOperatorRepo() error { - cloneCmd := fmt.Sprintf("git clone https://github.com/pingcap/tidb-operator.git /tidb-operator") - res, err := exec.Command("/bin/sh", "-c", cloneCmd).CombinedOutput() + cmd := fmt.Sprintf("git clone https://github.com/pingcap/tidb-operator.git /tidb-operator") + glog.Info(cmd) + res, err := exec.Command("/bin/sh", "-c", cmd).CombinedOutput() if err != nil { return fmt.Errorf("failed to clone tidb-operator repository: %v, %s", err, string(res)) } @@ -795,8 +801,9 @@ func checkoutTag(tagName string) error { mkdir -p /charts/%s; cp -rf charts/tidb-operator /charts/%s/tidb-operator; cp -rf charts/tidb-cluster /charts/%s/tidb-cluster; - cp -rf charts/tidb-backup /charts/%s/tidb-backup;`, + cp -rf charts/tidb-backup /charts/%s/tidb-backup`, tagName, tagName, tagName, tagName, tagName) + glog.Info(cmd) res, err := exec.Command("/bin/sh", "-c", cmd).CombinedOutput() if err != nil { return fmt.Errorf("failed to check tag: %s, %v, %s", tagName, err, string(res)) diff --git a/tests/cmd/e2e/main.go b/tests/cmd/e2e/main.go index 9ea6d00fd7..46af075764 100644 --- a/tests/cmd/e2e/main.go +++ b/tests/cmd/e2e/main.go @@ -12,3 +12,71 @@ // limitations under the License.package spec package main + +import ( + "flag" + + "github.com/golang/glog" + "github.com/pingcap/tidb-operator/pkg/client/clientset/versioned" + "github.com/pingcap/tidb-operator/tests" + "k8s.io/apiserver/pkg/util/logs" + "k8s.io/client-go/kubernetes" + "k8s.io/client-go/rest" +) + +func main() { + flag.Parse() + logs.InitLogs() + defer logs.FlushLogs() + + cfg, err := rest.InClusterConfig() + if err != nil { + glog.Fatalf("failed to get config: %v", err) + } + cli, err := versioned.NewForConfig(cfg) + if err != nil { + glog.Fatalf("failed to create Clientset: %v", err) + } + kubeCli, err := kubernetes.NewForConfig(cfg) + if err != nil { + glog.Fatalf("failed to get kubernetes Clientset: %v", err) + } + + oa := tests.NewOperatorActions(cli, kubeCli) + + operatorInfo := &tests.OperatorInfo{ + Namespace: "pingcap", + ReleaseName: "operator", + Image: "pingcap/tidb-operator:v1.0.0-beta.1-p2", + Tag: "v1.0.0-beta.1-p2", + SchedulerImage: "gcr.io/google-containers/hyperkube:v1.12.1", + LogLevel: "2", + } + if err := oa.CleanOperator(operatorInfo); err != nil { + glog.Fatal(err) + } + if err := oa.DeployOperator(operatorInfo); err != nil { + glog.Fatal(err) + } + + clusterInfo := &tests.TidbClusterInfo{ + Namespace: "tidb", + ClusterName: "demo", + OperatorTag: "v1.0.0-beta.1-p2", + PDImage: "pingcap/pd:v2.1.3", + TiKVImage: "pingcap/tikv:v2.1.3", + TiDBImage: "pingcap/tidb:v2.1.3", + StorageClassName: "local-storage", + Password: "admin", + Args: map[string]string{}, + } + if err := oa.CleanTidbCluster(clusterInfo); err != nil { + glog.Fatal(err) + } + if err := oa.DeployTidbCluster(clusterInfo); err != nil { + glog.Fatal(err) + } + if err := oa.CheckTidbClusterStatus(clusterInfo); err != nil { + glog.Fatal(err) + } +} diff --git a/tests/images/e2e/Dockerfile b/tests/images/e2e/Dockerfile new file mode 100644 index 0000000000..b931c23ad6 --- /dev/null +++ b/tests/images/e2e/Dockerfile @@ -0,0 +1,17 @@ +FROM alpine:3.5 + +ENV KUBECTL_VERSION=v1.12.2 +ENV HELM_VERSION=v2.9.1 + +RUN apk update && apk add --no-cache ca-certificates curl git +RUN curl https://storage.googleapis.com/kubernetes-release/release/${KUBECTL_VERSION}/bin/linux/amd64/kubectl \ + -o /usr/local/bin/kubectl && \ + chmod +x /usr/local/bin/kubectl && \ + curl https://storage.googleapis.com/kubernetes-helm/helm-${HELM_VERSION}-linux-amd64.tar.gz \ + -o helm-${HELM_VERSION}-linux-amd64.tar.gz && \ + tar -zxvf helm-${HELM_VERSION}-linux-amd64.tar.gz && \ + mv linux-amd64/helm /usr/local/bin/helm && \ + rm -rf linux-amd64 && \ + rm helm-${HELM_VERSION}-linux-amd64.tar.gz + +ADD bin/e2e /usr/local/bin/e2e diff --git a/tests/manifests/e2e.yaml b/tests/manifests/e2e.yaml new file mode 100644 index 0000000000..ed50c4dcc5 --- /dev/null +++ b/tests/manifests/e2e.yaml @@ -0,0 +1,33 @@ +--- +kind: ClusterRoleBinding +apiVersion: rbac.authorization.k8s.io/v1beta1 +metadata: + name: tidb-operator-e2e +subjects: +- kind: ServiceAccount + namespace: tidb-operator-e2e + name: tidb-operator-e2e +roleRef: + kind: ClusterRole + name: cluster-admin + apiGroup: rbac.authorization.k8s.io +--- +kind: ServiceAccount +apiVersion: v1 +metadata: + namespace: tidb-operator-e2e + name: tidb-operator-e2e +--- +apiVersion: v1 +kind: Pod +metadata: + namespace: tidb-operator-e2e + name: tidb-operator-e2e +spec: + serviceAccount: tidb-operator-e2e + containers: + - name: tidb-operator-e2e + image: "" + imagePullPolicy: Always + command: ["sh", "-c", "/usr/local/bin/e2e"] + restartPolicy: Never From 129df4f50fd5da54a578dea390c6f5439e563fc7 Mon Sep 17 00:00:00 2001 From: shuijing198799 <30903849+shuijing198799@users.noreply.github.com> Date: Tue, 12 Mar 2019 15:36:16 +0800 Subject: [PATCH 03/22] tidb stability test main function (#306) tidb stability main func --- .gitignore | 3 ++ pkg/label/label.go | 13 +++++ tests/actions.go | 112 ++++++++++++++++++++++++++++++++---------- tests/cmd/e2e/main.go | 22 +++++++++ 4 files changed, 123 insertions(+), 27 deletions(-) diff --git a/.gitignore b/.gitignore index 08008893fb..b661d496c4 100644 --- a/.gitignore +++ b/.gitignore @@ -7,6 +7,9 @@ images/tidb-operator-e2e/tidb-operator/ tmp/ data/ .idea +cscope.files +tags +filenametags # ginkgo test coverage *.coverprofile diff --git a/pkg/label/label.go b/pkg/label/label.go index cdd529b4ab..e453027d92 100644 --- a/pkg/label/label.go +++ b/pkg/label/label.go @@ -14,8 +14,10 @@ package label import ( + "fmt" metav1 "k8s.io/apimachinery/pkg/apis/meta/v1" "k8s.io/apimachinery/pkg/labels" + "strings" ) const ( @@ -138,3 +140,14 @@ func (l Label) LabelSelector() *metav1.LabelSelector { func (l Label) Labels() map[string]string { return l } + +// String converts label to a string +func (l Label) String() string { + var arr []string + + for k, v := range l { + arr = append(arr, fmt.Sprintf("%s=%s", k, v)) + } + + return strings.Join(arr, ",") +} diff --git a/tests/actions.go b/tests/actions.go index 6636337c22..bcbcca6325 100644 --- a/tests/actions.go +++ b/tests/actions.go @@ -21,6 +21,7 @@ import ( "strings" "time" + _ "github.com/go-sql-driver/mysql" "github.com/golang/glog" "github.com/pingcap/kvproto/pkg/metapb" "github.com/pingcap/tidb-operator/pkg/apis/pingcap.com/v1alpha1" @@ -43,6 +44,11 @@ func NewOperatorActions(cli versioned.Interface, kubeCli kubernetes.Interface) O } } +const ( + DefaultPollTimeout time.Duration = 10 * time.Minute + DefaultPollInterval time.Duration = 1 * time.Minute +) + type OperatorActions interface { DeployOperator(info *OperatorInfo) error CleanOperator(info *OperatorInfo) error @@ -56,14 +62,16 @@ type OperatorActions interface { ScaleTidbCluster(info *TidbClusterInfo) error UpgradeTidbCluster(info *TidbClusterInfo) error DeployAdHocBackup(info *TidbClusterInfo) error - CleanAdHocBackup(info *TidbClusterInfo) error + CheckAdHocBackup(info *TidbClusterInfo) error DeployScheduledBackup(info *TidbClusterInfo) error - CleanScheduledBackup(info *TidbClusterInfo) error - DeployIncrementalBackup(info *TidbClusterInfo) error - CleanIncrementalBackup(info *TidbClusterInfo) error - Restore(from *TidbClusterInfo, jobName string, to *TidbClusterInfo) error + CheckScheduledBackup(info *TidbClusterInfo) error + DeployIncrementalBackup(from *TidbClusterInfo, to *TidbClusterInfo) error + CheckIncrementalBackup(info *TidbClusterInfo) error + Restore(from *TidbClusterInfo, to *TidbClusterInfo) error + CheckRestore(from *TidbClusterInfo, to *TidbClusterInfo) error DeployMonitor(info *TidbClusterInfo) error CleanMonitor(info *TidbClusterInfo) error + ForceDeploy(info *TidbClusterInfo) error } type FaultTriggerActions interface { @@ -204,6 +212,10 @@ func (oa *operatorActions) DumpAllLogs(info *OperatorInfo, clusterInfo *TidbClus } func (oa *operatorActions) DeployTidbCluster(info *TidbClusterInfo) error { + glog.Infof("begin to deploy tidb cluster") + defer func() { + glog.Infof("deploy tidb cluster end") + }() cmd := fmt.Sprintf("helm install /charts/%s/tidb-cluster --name %s --namespace %s --set-string %s", info.OperatorTag, info.ClusterName, info.Namespace, info.HelmSetString()) if res, err := exec.Command("/bin/sh", "-c", cmd).CombinedOutput(); err != nil { @@ -215,10 +227,13 @@ func (oa *operatorActions) DeployTidbCluster(info *TidbClusterInfo) error { } func (oa *operatorActions) CleanTidbCluster(info *TidbClusterInfo) error { + glog.Infof("begin to clean tidb cluster") + defer func() { + glog.Infof("clean tidb cluster end") + }() charts := []string{ info.ClusterName, fmt.Sprintf("%s-backup", info.ClusterName), - fmt.Sprintf("%s-restore", info.ClusterName), } for _, chartName := range charts { res, err := exec.Command("helm", "del", "--purge", chartName).CombinedOutput() @@ -228,10 +243,12 @@ func (oa *operatorActions) CleanTidbCluster(info *TidbClusterInfo) error { } } - resources := []string{"cronjobs", "jobs", "pods", "pvc"} + setStr := label.New().Instance(info.ClusterName).String() + + resources := []string{"pvc"} for _, resource := range resources { - if res, err := exec.Command("kubectl", "delete", resource, "-n", info.Namespace, - "--all").CombinedOutput(); err != nil { + if res, err := exec.Command("kubectl", "delete", resource, "-n", info.Namespace, "-l", + setStr).CombinedOutput(); err != nil { return fmt.Errorf("failed to delete %s: %v, %s", resource, err, string(res)) } } @@ -245,7 +262,7 @@ func (oa *operatorActions) CleanTidbCluster(info *TidbClusterInfo) error { } pollFn := func() (bool, error) { - if res, err := exec.Command("kubectl", "get", "po", "--output=name", "-n", info.Namespace). + if res, err := exec.Command("kubectl", "get", "po", "--output=name", "-n", info.Namespace, "-l", setStr). CombinedOutput(); err != nil || len(res) != 0 { glog.Infof("waiting for tidbcluster: %s/%s pods deleting, %v, [%s]", info.Namespace, info.ClusterName, err, string(res)) @@ -262,16 +279,19 @@ func (oa *operatorActions) CleanTidbCluster(info *TidbClusterInfo) error { info.Namespace, info.ClusterName, err, string(res)) return false, nil } - return true, nil } - return wait.PollImmediate(1*time.Minute, 5*time.Minute, pollFn) + return wait.PollImmediate(DefaultPollInterval, DefaultPollTimeout, pollFn) } func (oa *operatorActions) CheckTidbClusterStatus(info *TidbClusterInfo) error { + glog.Infof("begin to check tidb cluster") + defer func() { + glog.Infof("check tidb cluster end") + }() ns := info.Namespace tcName := info.ClusterName - if err := wait.PollImmediate(1*time.Minute, 10*time.Minute, func() (bool, error) { + if err := wait.PollImmediate(DefaultPollInterval, DefaultPollTimeout, func() (bool, error) { var tc *v1alpha1.TidbCluster var err error if tc, err = oa.cli.PingcapV1alpha1().TidbClusters(ns).Get(tcName, metav1.GetOptions{}); err != nil { @@ -285,20 +305,30 @@ func (oa *operatorActions) CheckTidbClusterStatus(info *TidbClusterInfo) error { if b, err := oa.tikvMembersReadyFn(tc); !b && err == nil { return false, nil } + + glog.Infof("check tidb cluster begin tidbMembersReadyFn") if b, err := oa.tidbMembersReadyFn(tc); !b && err == nil { return false, nil } + + glog.Infof("check tidb cluster begin reclaimPolicySyncFn") if b, err := oa.reclaimPolicySyncFn(tc); !b && err == nil { return false, nil } + + glog.Infof("check tidb cluster begin metaSyncFn") if b, err := oa.metaSyncFn(tc); err != nil { return false, err } else if !b && err == nil { return false, nil } + + glog.Infof("check tidb cluster begin schedulerHAFn") if b, err := oa.schedulerHAFn(tc); !b && err == nil { return false, nil } + + glog.Infof("check tidb cluster begin passwordIsSet") if b, err := oa.passwordIsSet(info); !b && err == nil { return false, nil } @@ -319,19 +349,10 @@ func (oa *operatorActions) StopInsertDataTo(info *TidbClusterInfo) error { return nil } -func (oa *operatorActions) ScaleTidbCluster(info *TidbClusterInfo) error { return nil } -func (oa *operatorActions) UpgradeTidbCluster(info *TidbClusterInfo) error { return nil } -func (oa *operatorActions) DeployAdHocBackup(info *TidbClusterInfo) error { return nil } -func (oa *operatorActions) CleanAdHocBackup(info *TidbClusterInfo) error { return nil } -func (oa *operatorActions) DeployScheduledBackup(info *TidbClusterInfo) error { return nil } -func (oa *operatorActions) CleanScheduledBackup(info *TidbClusterInfo) error { return nil } -func (oa *operatorActions) DeployIncrementalBackup(info *TidbClusterInfo) error { return nil } -func (oa *operatorActions) CleanIncrementalBackup(info *TidbClusterInfo) error { return nil } -func (oa *operatorActions) Restore(from *TidbClusterInfo, jobName string, to *TidbClusterInfo) error { - return nil -} -func (oa *operatorActions) DeployMonitor(info *TidbClusterInfo) error { return nil } -func (oa *operatorActions) CleanMonitor(info *TidbClusterInfo) error { return nil } +func (oa *operatorActions) ScaleTidbCluster(info *TidbClusterInfo) error { return nil } +func (oa *operatorActions) UpgradeTidbCluster(info *TidbClusterInfo) error { return nil } +func (oa *operatorActions) DeployMonitor(info *TidbClusterInfo) error { return nil } +func (oa *operatorActions) CleanMonitor(info *TidbClusterInfo) error { return nil } func (oa *operatorActions) pdMembersReadyFn(tc *v1alpha1.TidbCluster) (bool, error) { tcName := tc.GetName() @@ -729,7 +750,7 @@ func (oa *operatorActions) schedulerHAFn(tc *v1alpha1.TidbCluster) (bool, error) nodeName, len(nodeMap[nodeName]), totalCount) } } - return false, nil + return true, nil } components := []string{label.PDLabelVal, label.TiKVLabelVal} @@ -811,3 +832,40 @@ func checkoutTag(tagName string) error { return nil } + +func (oa *operatorActions) DeployScheduledBackup(info *TidbClusterInfo) error { + return nil +} + +func (oa *operatorActions) CheckScheduledBackup(info *TidbClusterInfo) error { + return nil +} + +func (oa *operatorActions) DeployAdHocBackup(info *TidbClusterInfo) error { + return nil +} + +func (oa *operatorActions) CheckAdHocBackup(info *TidbClusterInfo) error { + return nil +} + +func (oa *operatorActions) DeployIncrementalBackup(from *TidbClusterInfo, to *TidbClusterInfo) error { + return nil +} + +func (oa *operatorActions) CheckIncrementalBackup(info *TidbClusterInfo) error { + return nil + +} + +func (oa *operatorActions) Restore(from *TidbClusterInfo, to *TidbClusterInfo) error { + return nil +} + +func (oa *operatorActions) CheckRestore(from *TidbClusterInfo, to *TidbClusterInfo) error { + return nil +} + +func (oa *operatorActions) ForceDeploy(info *TidbClusterInfo) error { + return nil +} diff --git a/tests/cmd/e2e/main.go b/tests/cmd/e2e/main.go index 46af075764..d88878ad59 100644 --- a/tests/cmd/e2e/main.go +++ b/tests/cmd/e2e/main.go @@ -79,4 +79,26 @@ func main() { if err := oa.CheckTidbClusterStatus(clusterInfo); err != nil { glog.Fatal(err) } + + restoreClusterInfo := &tests.TidbClusterInfo{ + Namespace: "tidb", + ClusterName: "demo2", + OperatorTag: "v1.0.0-beta.1-p2", + PDImage: "pingcap/pd:v2.1.3", + TiKVImage: "pingcap/tikv:v2.1.3", + TiDBImage: "pingcap/tidb:v2.1.3", + StorageClassName: "local-storage", + Password: "admin", + Args: map[string]string{}, + } + + if err := oa.CleanTidbCluster(restoreClusterInfo); err != nil { + glog.Fatal(err) + } + if err := oa.DeployTidbCluster(restoreClusterInfo); err != nil { + glog.Fatal(err) + } + if err := oa.CheckTidbClusterStatus(restoreClusterInfo); err != nil { + glog.Fatal(err) + } } From 4f8d71aaceed68ec3fdbf41d00d50246357b5ece Mon Sep 17 00:00:00 2001 From: CWen Date: Thu, 14 Mar 2019 21:28:35 +0800 Subject: [PATCH 04/22] stability: add fault-trigger server (#312) * test: add fault-trigger server --- tests/cmd/fault-trigger/main.go | 55 ++++++ tests/pkg/fault-trigger/api/response.go | 43 ++++ tests/pkg/fault-trigger/api/router.go | 40 ++++ tests/pkg/fault-trigger/api/server.go | 195 +++++++++++++++++++ tests/pkg/fault-trigger/manager/etcd.go | 44 +++++ tests/pkg/fault-trigger/manager/kubelet.go | 44 +++++ tests/pkg/fault-trigger/manager/manager.go | 22 +++ tests/pkg/fault-trigger/manager/types.go | 23 +++ tests/pkg/fault-trigger/manager/vm.go | 216 +++++++++++++++++++++ tests/pkg/fault-trigger/manager/vm_test.go | 71 +++++++ 10 files changed, 753 insertions(+) create mode 100644 tests/cmd/fault-trigger/main.go create mode 100644 tests/pkg/fault-trigger/api/response.go create mode 100644 tests/pkg/fault-trigger/api/router.go create mode 100644 tests/pkg/fault-trigger/api/server.go create mode 100644 tests/pkg/fault-trigger/manager/etcd.go create mode 100644 tests/pkg/fault-trigger/manager/kubelet.go create mode 100644 tests/pkg/fault-trigger/manager/manager.go create mode 100644 tests/pkg/fault-trigger/manager/types.go create mode 100644 tests/pkg/fault-trigger/manager/vm.go create mode 100644 tests/pkg/fault-trigger/manager/vm_test.go diff --git a/tests/cmd/fault-trigger/main.go b/tests/cmd/fault-trigger/main.go new file mode 100644 index 0000000000..8474c676a5 --- /dev/null +++ b/tests/cmd/fault-trigger/main.go @@ -0,0 +1,55 @@ +// Copyright 2019 PingCAP, Inc. +// +// 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, +// See the License for the specific language governing permissions and +// limitations under the License. + +package main + +import ( + "flag" + "fmt" + "net/http" + _ "net/http/pprof" + "time" + + "github.com/golang/glog" + "github.com/pingcap/tidb-operator/tests/pkg/fault-trigger/api" + "github.com/pingcap/tidb-operator/tests/pkg/fault-trigger/manager" + "k8s.io/apimachinery/pkg/util/wait" + "k8s.io/apiserver/pkg/util/logs" +) + +var ( + port int + pprofPort int +) + +func init() { + flag.IntVar(&port, "port", 23332, "The port that the fault trigger's http service runs on (default 23332)") + flag.IntVar(&pprofPort, "pprof-port", 6060, "The port that the pprof's http service runs on (default 6060)") + + flag.Parse() +} + +func main() { + logs.InitLogs() + defer logs.FlushLogs() + + mgr := manager.NewManager() + + server := api.NewServer(mgr, port) + + go wait.Forever(func() { + server.StartServer() + }, 5*time.Second) + + glog.Fatal(http.ListenAndServe(fmt.Sprintf(":%d", pprofPort), nil)) +} diff --git a/tests/pkg/fault-trigger/api/response.go b/tests/pkg/fault-trigger/api/response.go new file mode 100644 index 0000000000..9280cf66f7 --- /dev/null +++ b/tests/pkg/fault-trigger/api/response.go @@ -0,0 +1,43 @@ +// copyright 2019 pingcap, inc. +// +// 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, +// see the license for the specific language governing permissions and +// limitations under the license. + +package api + +import "net/http" + +// Response defines a new response struct for http +type Response struct { + Action string `json:"action"` + StatusCode int `json:"status_code"` + Message string `json:"message,omitempty"` + Payload interface{} `json:"payload,omitempty"` +} + +func newResponse(action string) *Response { + return &Response{Action: action, StatusCode: http.StatusOK} +} + +func (r *Response) statusCode(code int) *Response { + r.StatusCode = code + return r +} + +func (r *Response) message(msg string) *Response { + r.Message = msg + return r +} + +func (r *Response) payload(payload interface{}) *Response { + r.Payload = payload + return r +} diff --git a/tests/pkg/fault-trigger/api/router.go b/tests/pkg/fault-trigger/api/router.go new file mode 100644 index 0000000000..bb58405bab --- /dev/null +++ b/tests/pkg/fault-trigger/api/router.go @@ -0,0 +1,40 @@ +// Copyright 2019 PingCAP, Inc. +// +// 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, +// See the License for the specific language governing permissions and +// limitations under the License. + +package api + +import restful "github.com/emicklei/go-restful" + +const ( + apiPrefix = "/pingcap.com/api/v1" +) + +func (s *Server) newService() *restful.WebService { + ws := new(restful.WebService) + ws. + Path(apiPrefix). + Consumes(restful.MIME_JSON). + Produces(restful.MIME_JSON) + + ws.Route(ws.GET("/vms").To(s.listVMs)) + ws.Route(ws.GET("/vm/{name}/start").To(s.startVM)) + ws.Route(ws.GET("/vm/{name}/stop").To(s.stopVM)) + + ws.Route(ws.GET("/etcd/start").To(s.startETCD)) + ws.Route(ws.GET("/etcd/stop").To(s.stopETCD)) + + ws.Route(ws.GET("/kubelet/start").To(s.startKubelet)) + ws.Route(ws.GET("/kubelet/stop").To(s.stopKubelet)) + + return ws +} diff --git a/tests/pkg/fault-trigger/api/server.go b/tests/pkg/fault-trigger/api/server.go new file mode 100644 index 0000000000..6d0fce8edc --- /dev/null +++ b/tests/pkg/fault-trigger/api/server.go @@ -0,0 +1,195 @@ +// Copyright 2019 PingCAP, Inc. +// +// 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, +// See the License for the specific language governing permissions and +// limitations under the License. + +package api + +import ( + "fmt" + "net/http" + + restful "github.com/emicklei/go-restful" + "github.com/golang/glog" + "github.com/pingcap/tidb-operator/tests/pkg/fault-trigger/manager" +) + +// Server is a web service to control fault trigger +type Server struct { + mgr *manager.Manager + + port int +} + +// NewServer returns a api server +func NewServer(mgr *manager.Manager, port int) *Server { + return &Server{ + mgr: mgr, + port: port, + } +} + +// StartServer starts a fault-trigger server +func (s *Server) StartServer() { + ws := s.newService() + + restful.Add(ws) + + glog.Infof("starting fault-trigger server, listening on 0.0.0.0:%d", s.port) + glog.Fatal(http.ListenAndServe(fmt.Sprintf(":%d", s.port), nil)) +} + +func (s *Server) listVMs(req *restful.Request, resp *restful.Response) { + res := newResponse("listVMs") + vms, err := s.mgr.ListVMs() + if err != nil { + res.message(err.Error()).statusCode(http.StatusInternalServerError) + if err = resp.WriteEntity(res); err != nil { + glog.Errorf("failed to response, methods: listVMs, error: %v", err) + } + return + } + + res.payload(vms).statusCode(http.StatusOK) + + if err = resp.WriteEntity(res); err != nil { + glog.Errorf("failed to response, method: listVMs, error: %v", err) + } +} + +func (s *Server) startVM(req *restful.Request, resp *restful.Response) { + res := newResponse("startVM") + name := req.PathParameter("name") + + targetVM, err := s.getVM(name) + if err != nil { + res.message(fmt.Sprintf("failed to get vm %s, error: %v", name, err)). + statusCode(http.StatusInternalServerError) + if err = resp.WriteEntity(res); err != nil { + glog.Errorf("failed to response, methods: startVM, error: %v", err) + } + return + } + + if targetVM == nil { + res.message(fmt.Sprintf("vm %s not found", name)).statusCode(http.StatusNotFound) + if err = resp.WriteEntity(res); err != nil { + glog.Errorf("failed to response, methods: startVM, error: %v", err) + } + return + } + + s.vmAction(req, resp, res, targetVM, s.mgr.StartVM, "startVM") +} + +func (s *Server) stopVM(req *restful.Request, resp *restful.Response) { + res := newResponse("stopVM") + name := req.PathParameter("name") + + targetVM, err := s.getVM(name) + if err != nil { + res.message(fmt.Sprintf("failed to get vm %s, error: %v", name, err)). + statusCode(http.StatusInternalServerError) + if err = resp.WriteEntity(res); err != nil { + glog.Errorf("failed to response, methods: stopVM, error: %v", err) + } + return + } + + if targetVM == nil { + res.message(fmt.Sprintf("vm %s not found", name)).statusCode(http.StatusNotFound) + if err = resp.WriteEntity(res); err != nil { + glog.Errorf("failed to response, methods: stopVM, error: %v", err) + } + return + } + + s.vmAction(req, resp, res, targetVM, s.mgr.StopVM, "stopVM") +} + +func (s *Server) startETCD(req *restful.Request, resp *restful.Response) { + s.action(req, resp, s.mgr.StartETCD, "startETCD") +} + +func (s *Server) stopETCD(req *restful.Request, resp *restful.Response) { + s.action(req, resp, s.mgr.StopETCD, "stopETCD") +} + +func (s *Server) startKubelet(req *restful.Request, resp *restful.Response) { + s.action(req, resp, s.mgr.StartKubelet, "startKubelet") +} + +func (s *Server) stopKubelet(req *restful.Request, resp *restful.Response) { + s.action(req, resp, s.mgr.StopKubelet, "stopKubelet") +} + +func (s *Server) action( + req *restful.Request, + resp *restful.Response, + fn func() error, + method string, +) { + res := newResponse(method) + + if err := fn(); err != nil { + res.message(fmt.Sprintf("failed to %s, error: %v", method, err)). + statusCode(http.StatusInternalServerError) + if err = resp.WriteEntity(res); err != nil { + glog.Errorf("failed to response, methods: %s, error: %v", method, err) + } + return + } + + res.message("OK").statusCode(http.StatusOK) + + if err := resp.WriteEntity(res); err != nil { + glog.Errorf("failed to response, method: %s, error: %v", method, err) + } +} + +func (s *Server) vmAction( + req *restful.Request, + resp *restful.Response, + res *Response, + targetVM *manager.VM, + fn func(vm *manager.VM) error, + method string, +) { + if err := fn(targetVM); err != nil { + res.message(fmt.Sprintf("failed to %s vm: %s, error: %v", method, targetVM.Name, err)). + statusCode(http.StatusInternalServerError) + if err = resp.WriteEntity(res); err != nil { + glog.Errorf("failed to response, methods: %s, error: %v", method, err) + } + return + } + + res.message("OK").statusCode(http.StatusOK) + + if err := resp.WriteEntity(res); err != nil { + glog.Errorf("failed to response, method: %s, error: %v", method, err) + } +} + +func (s *Server) getVM(name string) (*manager.VM, error) { + vms, err := s.mgr.ListVMs() + if err != nil { + return nil, err + } + + for _, vm := range vms { + if name == vm.Name || name == vm.IP { + return vm, nil + } + } + + return nil, nil +} diff --git a/tests/pkg/fault-trigger/manager/etcd.go b/tests/pkg/fault-trigger/manager/etcd.go new file mode 100644 index 0000000000..13c0df2f4c --- /dev/null +++ b/tests/pkg/fault-trigger/manager/etcd.go @@ -0,0 +1,44 @@ +// Copyright 2019 PingCAP, Inc. +// +// 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, +// See the License for the specific language governing permissions and +// limitations under the License. + +package manager + +import ( + "os/exec" + + "github.com/golang/glog" +) + +// StartETCD starts etcd +func (m *Manager) StartETCD() error { + shell := "systemctl start etcd" + cmd := exec.Command("/bin/sh", "-c", shell) + output, err := cmd.CombinedOutput() + if err != nil { + glog.Errorf("exec: [%s] failed, output: %s, error: %v", shell, string(output), err) + return err + } + return nil +} + +// StopETCD stops etcd +func (m *Manager) StopETCD() error { + shell := "systemctl stop etcd" + cmd := exec.Command("/bin/sh", "-c", shell) + output, err := cmd.CombinedOutput() + if err != nil { + glog.Errorf("exec: [%s] failed, output: %s, error: %v", shell, string(output), err) + return err + } + return nil +} diff --git a/tests/pkg/fault-trigger/manager/kubelet.go b/tests/pkg/fault-trigger/manager/kubelet.go new file mode 100644 index 0000000000..6cdbf11227 --- /dev/null +++ b/tests/pkg/fault-trigger/manager/kubelet.go @@ -0,0 +1,44 @@ +// Copyright 2019 PingCAP, Inc. +// +// 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, +// See the License for the specific language governing permissions and +// limitations under the License. + +package manager + +import ( + "os/exec" + + "github.com/golang/glog" +) + +// StartKubelet starts kubelet +func (m *Manager) StartKubelet() error { + shell := "systemctl start kubelet" + cmd := exec.Command("/bin/sh", "-c", shell) + output, err := cmd.CombinedOutput() + if err != nil { + glog.Errorf("exec: [%s] failed, output: %s, error: %v", shell, string(output), err) + return err + } + return nil +} + +// StopKubelet stops kubelet +func (m *Manager) StopKubelet() error { + shell := "systemctl stop kubelet" + cmd := exec.Command("/bin/sh", "-c", shell) + output, err := cmd.CombinedOutput() + if err != nil { + glog.Errorf("exec: [%s] failed, output: %s, error: %v", shell, string(output), err) + return err + } + return nil +} diff --git a/tests/pkg/fault-trigger/manager/manager.go b/tests/pkg/fault-trigger/manager/manager.go new file mode 100644 index 0000000000..55f6ee77d0 --- /dev/null +++ b/tests/pkg/fault-trigger/manager/manager.go @@ -0,0 +1,22 @@ +// Copyright 2019 PingCAP, Inc. +// +// 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, +// See the License for the specific language governing permissions and +// limitations under the License. + +package manager + +// Manager to manager fault trigger +type Manager struct{} + +// NewManager returns a manager instance +func NewManager() *Manager { + return &Manager{} +} diff --git a/tests/pkg/fault-trigger/manager/types.go b/tests/pkg/fault-trigger/manager/types.go new file mode 100644 index 0000000000..fa2fa099ec --- /dev/null +++ b/tests/pkg/fault-trigger/manager/types.go @@ -0,0 +1,23 @@ +// Copyright 2019 PingCAP, Inc. +// +// 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, +// See the License for the specific language governing permissions and +// limitations under the License. + +package manager + +// VM defines the descriptive information of a virtual machine +type VM struct { + Host string `json:"host"` + Port int64 `json:"port"` + Name string `json:"name"` + IP string `json:"ip"` + Role []string `json:"role"` +} diff --git a/tests/pkg/fault-trigger/manager/vm.go b/tests/pkg/fault-trigger/manager/vm.go new file mode 100644 index 0000000000..e83df64b00 --- /dev/null +++ b/tests/pkg/fault-trigger/manager/vm.go @@ -0,0 +1,216 @@ +// Copyright 2019 PingCAP, Inc. +// +// 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, +// See the License for the specific language governing permissions and +// limitations under the License. + +package manager + +import ( + "errors" + "fmt" + "os/exec" + "strings" + + "github.com/golang/glog" +) + +// ListVMs lists vms +func (m *Manager) ListVMs() ([]*VM, error) { + shell := fmt.Sprintf("virsh list --all") + cmd := exec.Command("/bin/sh", "-c", shell) + output, err := cmd.CombinedOutput() + if err != nil { + glog.Errorf("exec: [%s] failed, output: %s, error: %v", shell, string(output), err) + return nil, err + } + vms := parserVMs(string(output)) + //"virsh domifaddr vm1 --interface eth0 --source agent" + for _, vm := range vms { + vmIP, err := getVMIP(vm.Name) + if err != nil { + glog.Errorf("can not get vm %s ip", vm.Name) + continue + } + vm.IP = vmIP + } + return vms, nil +} + +// StopVM stops vm +func (m *Manager) StopVM(v *VM) error { + shell := fmt.Sprintf("virsh destroy %s", v.Name) + cmd := exec.Command("/bin/sh", "-c", shell) + output, err := cmd.CombinedOutput() + if err != nil { + glog.Errorf("exec: [%s] failed, output: %s, error: %v", shell, string(output), err) + return err + } + return nil +} + +// StartVM starts vm +func (m *Manager) StartVM(v *VM) error { + shell := fmt.Sprintf("virsh start %s", v.Name) + cmd := exec.Command("/bin/sh", "-c", shell) + output, err := cmd.CombinedOutput() + if err != nil { + glog.Errorf("exec: [%s] failed, output: %s, error: %v", shell, string(output), err) + return err + } + return nil +} + +func getVMIP(name string) (string, error) { + shell := fmt.Sprintf("virsh domifaddr %s --interface eth0 --source agent", name) + cmd := exec.Command("/bin/sh", "-c", shell) + output, err := cmd.CombinedOutput() + if err != nil { + glog.Warningf("exec: [%s] failed, output: %s, error: %v", shell, string(output), err) + mac, err := getVMMac(name) + if err != nil { + return "", err + } + + ipNeighShell := fmt.Sprintf("ip neigh | grep -i %s", mac) + cmd = exec.Command("/bin/sh", "-c", ipNeighShell) + ipNeighOutput, err := cmd.CombinedOutput() + if err != nil { + glog.Errorf("exec: [%s] failed, output: %s, error: %v", ipNeighShell, string(ipNeighOutput), err) + return "", err + } + + return parserIPFromIPNeign(string(ipNeighOutput)) + } + + return parserIP(string(output)) +} + +func getVMMac(name string) (string, error) { + shell := fmt.Sprintf("virsh domiflist %s", name) + cmd := exec.Command("/bin/sh", "-c", shell) + output, err := cmd.CombinedOutput() + if err != nil { + return "", err + } + + return parserMac(string(output)) +} + +// example input : +// Interface Type Source Model MAC +// ------------------------------------------------------- +// vnet1 bridge br0 virtio 52:54:00:d4:9e:bb +// output: 52:54:00:d4:9e:bb, nil +func parserMac(data string) (string, error) { + data = stripEmpty(data) + lines := strings.Split(data, "\n") + + for _, line := range lines { + if !strings.Contains(line, "bridge") { + continue + } + + fields := strings.Split(line, " ") + if len(fields) < 5 { + continue + } + + return fields[4], nil + } + + return "", errors.New("mac not found") +} + +// example input: +// Name MAC address Protocol Address +// ------------------------------------------------------------------------------- +// eth0 52:54:00:4c:5b:c0 ipv4 172.16.30.216/24 +// - - ipv6 fe80::5054:ff:fe4c:5bc0/64 +// output: 172.16.30.216, nil +func parserIP(data string) (string, error) { + data = stripEmpty(data) + lines := strings.Split(data, "\n") + for _, line := range lines { + if !strings.HasPrefix(line, "eth0") { + continue + } + + fields := strings.Split(line, " ") + if len(fields) < 4 { + continue + } + + ip := strings.Split(fields[3], "/")[0] + return ip, nil + } + + return "", errors.New("ip not found") +} + +// example input: +// 172.16.30.216 dev br0 lladdr 52:54:00:4c:5b:c0 STALE +// output: 172.16.30.216, nil +func parserIPFromIPNeign(data string) (string, error) { + fields := strings.Split(strings.Trim(data, "\n"), " ") + if len(fields) != 6 { + return "", errors.New("ip not found") + } + + return fields[0], nil +} + +// example input: +// Id Name State +// ---------------------------------------------------- +// 6 vm2 running +// 11 vm3 running +// 12 vm1 running +// - vm-template shut off +func parserVMs(data string) []*VM { + data = stripEmpty(data) + lines := strings.Split(data, "\n") + var vms []*VM + for _, line := range lines { + fields := strings.Split(line, " ") + if len(fields) < 3 { + continue + } + if !strings.HasPrefix(fields[1], "vm") { + continue + } + + if strings.HasPrefix(fields[1], "vm-template") { + continue + } + vm := &VM{ + Name: fields[1], + } + vms = append(vms, vm) + } + return vms +} + +func stripEmpty(data string) string { + stripLines := []string{} + lines := strings.Split(data, "\n") + for _, line := range lines { + stripFields := []string{} + fields := strings.Split(line, " ") + for _, field := range fields { + if len(field) > 0 { + stripFields = append(stripFields, field) + } + } + stripLine := strings.Join(stripFields, " ") + stripLines = append(stripLines, stripLine) + } + return strings.Join(stripLines, "\n") +} diff --git a/tests/pkg/fault-trigger/manager/vm_test.go b/tests/pkg/fault-trigger/manager/vm_test.go new file mode 100644 index 0000000000..66d8f772aa --- /dev/null +++ b/tests/pkg/fault-trigger/manager/vm_test.go @@ -0,0 +1,71 @@ +package manager + +import ( + "testing" + + . "github.com/onsi/gomega" +) + +func TestParseMac(t *testing.T) { + g := NewGomegaWithT(t) + + data := ` +Interface Type Source Model MAC +------------------------------------------------------- +vnet1 bridge br0 virtio 52:54:00:d4:9e:bb +` + output, err := parserMac(data) + g.Expect(err).NotTo(HaveOccurred()) + g.Expect(output).To(Equal("52:54:00:d4:9e:bb")) +} + +func TestParseIP(t *testing.T) { + g := NewGomegaWithT(t) + + data := ` +Name MAC address Protocol Address +------------------------------------------------------------------------------- +eth0 52:54:00:4c:5b:c0 ipv4 172.16.30.216/24 +- - ipv6 fe80::5054:ff:fe4c:5bc0/64 +` + output, err := parserIP(data) + g.Expect(err).NotTo(HaveOccurred()) + g.Expect(output).To(Equal("172.16.30.216")) +} + +func TestParseIPFromIPNeign(t *testing.T) { + g := NewGomegaWithT(t) + + data := ` +172.16.30.216 dev br0 lladdr 52:54:00:4c:5b:c0 STALE +` + output, err := parserIPFromIPNeign(data) + g.Expect(err).NotTo(HaveOccurred()) + g.Expect(output).To(Equal("172.16.30.216")) +} + +func TestParseVMs(t *testing.T) { + g := NewGomegaWithT(t) + + data := ` + Id Name State +---------------------------------------------------- + 6 vm2 running + 11 vm3 running + 12 vm1 running + - vm-template shut off +` + vms := parserVMs(data) + + var expectedVMs []*VM + expectedVMs = append(expectedVMs, &VM{ + Name: "vm2", + }) + expectedVMs = append(expectedVMs, &VM{ + Name: "vm3", + }) + expectedVMs = append(expectedVMs, &VM{ + Name: "vm1", + }) + g.Expect(vms).To(Equal(expectedVMs)) +} From c12bf3b1d354f37e4596fe18670e7b09b9c61cbc Mon Sep 17 00:00:00 2001 From: shuijing198799 <30903849+shuijing198799@users.noreply.github.com> Date: Fri, 15 Mar 2019 11:51:26 +0800 Subject: [PATCH 05/22] Yinliang/backup and restore add adhoc backup and restore functison (#316) * add restore and backup process --- tests/actions.go | 270 ++++++++++++++++++++++++++++++++----- tests/backup/backup.go | 1 + tests/backup/backupcase.go | 74 ++++++++++ tests/cmd/e2e/main.go | 21 ++- 4 files changed, 332 insertions(+), 34 deletions(-) create mode 100644 tests/backup/backup.go create mode 100644 tests/backup/backupcase.go diff --git a/tests/actions.go b/tests/actions.go index bcbcca6325..61b50b11c3 100644 --- a/tests/actions.go +++ b/tests/actions.go @@ -14,6 +14,7 @@ package tests import ( + "bytes" "database/sql" "fmt" "os/exec" @@ -72,6 +73,7 @@ type OperatorActions interface { DeployMonitor(info *TidbClusterInfo) error CleanMonitor(info *TidbClusterInfo) error ForceDeploy(info *TidbClusterInfo) error + CreateSecret(info *TidbClusterInfo) error } type FaultTriggerActions interface { @@ -126,16 +128,22 @@ type TidbClusterInfo struct { } func (tc *TidbClusterInfo) HelmSetString() string { + + // add a database and table for test + initSql := `"create database record;use record;create table test(t char(32));"` + set := map[string]string{ - "clusterName": tc.ClusterName, - "pd.storageClassName": tc.StorageClassName, - "tikv.storageClassName": tc.StorageClassName, - "tidb.storageClassName": tc.StorageClassName, - "tidb.password": tc.Password, - "pd.maxStoreDownTime": "5m", - "pd.image": tc.PDImage, - "tikv.image": tc.TiKVImage, - "tidb.image": tc.TiDBImage, + "clusterName": tc.ClusterName, + "pd.storageClassName": tc.StorageClassName, + "tikv.storageClassName": tc.StorageClassName, + "tidb.storageClassName": tc.StorageClassName, + "tidb.password": tc.Password, + "pd.maxStoreDownTime": "5m", + "pd.image": tc.PDImage, + "tikv.image": tc.TiKVImage, + "tidb.image": tc.TiDBImage, + "tidb.passwordSecretName": "set-secret", + "tidb.initSql": initSql, } for k, v := range tc.Resources { @@ -212,9 +220,9 @@ func (oa *operatorActions) DumpAllLogs(info *OperatorInfo, clusterInfo *TidbClus } func (oa *operatorActions) DeployTidbCluster(info *TidbClusterInfo) error { - glog.Infof("begin to deploy tidb cluster") + glog.Infof("begin to deploy tidb cluster cluster[%s] namespace[%s]", info.ClusterName, info.Namespace) defer func() { - glog.Infof("deploy tidb cluster end") + glog.Infof("deploy tidb cluster end cluster[%s] namespace[%s]", info.ClusterName, info.Namespace) }() cmd := fmt.Sprintf("helm install /charts/%s/tidb-cluster --name %s --namespace %s --set-string %s", info.OperatorTag, info.ClusterName, info.Namespace, info.HelmSetString()) @@ -227,13 +235,14 @@ func (oa *operatorActions) DeployTidbCluster(info *TidbClusterInfo) error { } func (oa *operatorActions) CleanTidbCluster(info *TidbClusterInfo) error { - glog.Infof("begin to clean tidb cluster") + glog.Infof("begin to clean tidb cluster[%s] namespace[%s]", info.ClusterName, info.Namespace) defer func() { - glog.Infof("clean tidb cluster end") + glog.Infof("clean tidb cluster end cluster[%s] namespace[%s]", info.ClusterName, info.Namespace) }() charts := []string{ info.ClusterName, fmt.Sprintf("%s-backup", info.ClusterName), + fmt.Sprintf("%s-restore", info.ClusterName), } for _, chartName := range charts { res, err := exec.Command("helm", "del", "--purge", chartName).CombinedOutput() @@ -285,9 +294,9 @@ func (oa *operatorActions) CleanTidbCluster(info *TidbClusterInfo) error { } func (oa *operatorActions) CheckTidbClusterStatus(info *TidbClusterInfo) error { - glog.Infof("begin to check tidb cluster") + glog.Infof("begin to check tidb cluster cluster[%s] namespace[%s]", info.ClusterName, info.Namespace) defer func() { - glog.Infof("check tidb cluster end") + glog.Infof("check tidb cluster end cluster[%s] namespace[%s]", info.ClusterName, info.Namespace) }() ns := info.Namespace tcName := info.ClusterName @@ -816,12 +825,12 @@ func cloneOperatorRepo() error { } func checkoutTag(tagName string) error { - cmd := fmt.Sprintf(`cd /tidb-operator; - git stash -u; - git checkout %s; - mkdir -p /charts/%s; - cp -rf charts/tidb-operator /charts/%s/tidb-operator; - cp -rf charts/tidb-cluster /charts/%s/tidb-cluster; + cmd := fmt.Sprintf(`cd /tidb-operator && + git stash -u && + git checkout %s && + mkdir -p /charts/%s && + cp -rf charts/tidb-operator /charts/%s/tidb-operator && + cp -rf charts/tidb-cluster /charts/%s/tidb-cluster && cp -rf charts/tidb-backup /charts/%s/tidb-backup`, tagName, tagName, tagName, tagName, tagName) glog.Info(cmd) @@ -833,39 +842,240 @@ func checkoutTag(tagName string) error { return nil } -func (oa *operatorActions) DeployScheduledBackup(info *TidbClusterInfo) error { +func (oa *operatorActions) DeployAdHocBackup(info *TidbClusterInfo) error { + glog.Infof("begin to deploy adhoc backup cluster[%s] namespace[%s]", info.ClusterName, info.Namespace) + defer func() { + glog.Infof("deploy adhoc backup end cluster[%s] namespace[%s]", info.ClusterName, info.Namespace) + }() + sets := map[string]string{ + "clusterName": info.ClusterName, + "name": "test-backup", + "mode": "backup", + "user": "root", + "password": info.Password, + "storage.size": "10Gi", + } + var buffer bytes.Buffer + for k, v := range sets { + set := fmt.Sprintf(" --set %s=%s", k, v) + _, err := buffer.WriteString(set) + if err != nil { + return err + } + } + + setStr := buffer.String() + fullbackupName := fmt.Sprintf("%s-backup", info.ClusterName) + cmd := fmt.Sprintf("helm install -n %s --namespace %s /charts/%s/tidb-backup %s", + fullbackupName, info.Namespace, info.OperatorTag, setStr) + glog.Infof("install adhoc deployment [%s]", cmd) + res, err := exec.Command("/bin/sh", "-c", cmd).CombinedOutput() + if err != nil { + return fmt.Errorf("failed to launch adhoc backup job: %v, %s", err, string(res)) + } return nil } -func (oa *operatorActions) CheckScheduledBackup(info *TidbClusterInfo) error { +func (oa *operatorActions) CheckAdHocBackup(info *TidbClusterInfo) error { + glog.Infof("begin to clean adhoc backup cluster[%s] namespace[%s]", info.ClusterName, info.Namespace) + defer func() { + glog.Infof("deploy clean backup end cluster[%s] namespace[%s]", info.ClusterName, info.Namespace) + }() + + jobName := fmt.Sprintf("%s-%s", info.ClusterName, "test-backup") + fn := func() (bool, error) { + job, err := oa.kubeCli.BatchV1().Jobs(info.Namespace).Get(jobName, metav1.GetOptions{}) + if err != nil { + glog.Errorf("failed to get jobs %s ,%v", jobName, err) + return false, nil + } + if job.Status.Succeeded == 0 { + glog.Errorf("cluster [%s] back up job is not completed, please wait! ", info.ClusterName) + return false, nil + } + + return true, nil + } + + err := wait.Poll(DefaultPollInterval, DefaultPollTimeout, fn) + if err != nil { + return fmt.Errorf("failed to launch scheduler backup job: %v", err) + } return nil } -func (oa *operatorActions) DeployAdHocBackup(info *TidbClusterInfo) error { +func (oa *operatorActions) Restore(from *TidbClusterInfo, to *TidbClusterInfo) error { + glog.Infof("begin to deploy restore cluster[%s] namespace[%s]", from.ClusterName, from.Namespace) + defer func() { + glog.Infof("deploy restore end cluster[%s] namespace[%s]", to.ClusterName, to.Namespace) + }() + sets := map[string]string{ + "clusterName": to.ClusterName, + "name": "test-backup", + "mode": "restore", + "user": "root", + "password": to.Password, + "storage.size": "10Gi", + } + var buffer bytes.Buffer + for k, v := range sets { + set := fmt.Sprintf(" --set %s=%s", k, v) + _, err := buffer.WriteString(set) + if err != nil { + return err + } + } + + setStr := buffer.String() + restoreName := fmt.Sprintf("%s-restore", from.ClusterName) + cmd := fmt.Sprintf("helm install -n %s --namespace %s /charts/%s/tidb-backup %s", + restoreName, to.Namespace, to.OperatorTag, setStr) + glog.Infof("install restore [%s]", cmd) + res, err := exec.Command("/bin/sh", "-c", cmd).CombinedOutput() + if err != nil { + return fmt.Errorf("failed to launch restore job: %v, %s", err, string(res)) + } + return nil } -func (oa *operatorActions) CheckAdHocBackup(info *TidbClusterInfo) error { +func (oa *operatorActions) CheckRestore(from *TidbClusterInfo, to *TidbClusterInfo) error { + glog.Infof("begin to check restore backup cluster[%s] namespace[%s]", from.ClusterName, from.Namespace) + defer func() { + glog.Infof("check restore end cluster[%s] namespace[%s]", to.ClusterName, to.Namespace) + }() + + jobName := fmt.Sprintf("%s-restore-test-backup", to.ClusterName) + fn := func() (bool, error) { + job, err := oa.kubeCli.BatchV1().Jobs(to.Namespace).Get(jobName, metav1.GetOptions{}) + if err != nil { + glog.Errorf("failed to get jobs %s ,%v", jobName, err) + return false, nil + } + if job.Status.Succeeded == 0 { + glog.Errorf("cluster [%s] back up job is not completed, please wait! ", to.ClusterName) + return false, nil + } + + fromCount, err := from.QueryCount() + if err != nil { + glog.Errorf("cluster [%s] count err ", from.ClusterName) + return false, nil + } + + toCount, err := to.QueryCount() + if err != nil { + glog.Errorf("cluster [%s] count err ", to.ClusterName) + return false, nil + } + + if fromCount != toCount { + glog.Errorf("cluster [%s] count %d cluster [%s] count %d is not equal ", + from.ClusterName, fromCount, to.ClusterName, toCount) + return false, nil + } + return true, nil + } + + err := wait.Poll(DefaultPollInterval, DefaultPollTimeout, fn) + if err != nil { + return fmt.Errorf("failed to launch scheduler backup job: %v", err) + } return nil } -func (oa *operatorActions) DeployIncrementalBackup(from *TidbClusterInfo, to *TidbClusterInfo) error { +func (oa *operatorActions) ForceDeploy(info *TidbClusterInfo) error { + if err := oa.CleanTidbCluster(info); err != nil { + return err + } + + if err := oa.DeployTidbCluster(info); err != nil { + return err + } + return nil } -func (oa *operatorActions) CheckIncrementalBackup(info *TidbClusterInfo) error { +func (info *TidbClusterInfo) QueryCount() (int, error) { + tableName := "test" + db, err := sql.Open("mysql", getDSN(info.Namespace, info.ClusterName, "record", info.Password)) + if err != nil { + return 0, err + } + defer db.Close() + + rows, err := db.Query(fmt.Sprintf("SELECT count(*) FROM %s", tableName)) + if err != nil { + glog.Infof("cluster:[%s], error: %v", info.ClusterName, err) + return 0, err + } + + for rows.Next() { + var count int + err := rows.Scan(&count) + if err != nil { + glog.Infof("cluster:[%s], error :%v", info.ClusterName, err) + } + return count, nil + } + return 0, fmt.Errorf("can not find count of ") +} + +func (oa *operatorActions) CreateSecret(info *TidbClusterInfo) error { + initSecretName := "set-secret" + backupSecretName := "backup-secret" + initSecret := corev1.Secret{ + ObjectMeta: metav1.ObjectMeta{ + Name: initSecretName, + Namespace: info.Namespace, + }, + Data: map[string][]byte{ + "root": []byte(info.Password), + }, + Type: corev1.SecretTypeOpaque, + } + + _, err := oa.kubeCli.CoreV1().Secrets(info.Namespace).Create(&initSecret) + if err != nil && !releaseIsExist(err) { + return err + } + + backupSecret := corev1.Secret{ + ObjectMeta: metav1.ObjectMeta{ + Name: backupSecretName, + Namespace: info.Namespace, + }, + Data: map[string][]byte{ + "user": []byte("root"), + "password": []byte(info.Password), + }, + Type: corev1.SecretTypeOpaque, + } + + _, err = oa.kubeCli.CoreV1().Secrets(info.Namespace).Create(&backupSecret) + if err != nil && !releaseIsExist(err) { + return err + } + return nil +} +func releaseIsExist(err error) bool { + return strings.Contains(err.Error(), "already exists") } -func (oa *operatorActions) Restore(from *TidbClusterInfo, to *TidbClusterInfo) error { +func (oa *operatorActions) DeployScheduledBackup(info *TidbClusterInfo) error { return nil } -func (oa *operatorActions) CheckRestore(from *TidbClusterInfo, to *TidbClusterInfo) error { +func (oa *operatorActions) CheckScheduledBackup(info *TidbClusterInfo) error { return nil } -func (oa *operatorActions) ForceDeploy(info *TidbClusterInfo) error { +func (oa *operatorActions) DeployIncrementalBackup(from *TidbClusterInfo, to *TidbClusterInfo) error { + return nil +} + +func (oa *operatorActions) CheckIncrementalBackup(info *TidbClusterInfo) error { return nil } diff --git a/tests/backup/backup.go b/tests/backup/backup.go new file mode 100644 index 0000000000..0d44d47816 --- /dev/null +++ b/tests/backup/backup.go @@ -0,0 +1 @@ +package backup diff --git a/tests/backup/backupcase.go b/tests/backup/backupcase.go new file mode 100644 index 0000000000..07c5da0fec --- /dev/null +++ b/tests/backup/backupcase.go @@ -0,0 +1,74 @@ +// Copyright 2018 PingCAP, Inc. +// +// 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/LICENSE2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// See the License for the specific language governing permissions and +// limitations under the License. + +package backup + +import ( + "github.com/golang/glog" + "github.com/pingcap/tidb-operator/tests" +) + +type BackupCase struct { + operator tests.OperatorActions + srcCluster *tests.TidbClusterInfo + desCluster *tests.TidbClusterInfo +} + +func NewBackupCase(operator tests.OperatorActions, srcCluster *tests.TidbClusterInfo, desCluster *tests.TidbClusterInfo) *BackupCase { + return &BackupCase{ + operator: operator, + srcCluster: srcCluster, + desCluster: desCluster, + } +} + +func (bc *BackupCase) Run() error { + + err := bc.operator.DeployAdHocBackup(bc.srcCluster) + if err != nil { + glog.Errorf("cluster:[%s] deploy happen error: %v", bc.srcCluster.ClusterName, err) + return err + } + + err = bc.operator.CheckAdHocBackup(bc.srcCluster) + if err != nil { + glog.Errorf("cluster:[%s] deploy happen error: %v", bc.srcCluster.ClusterName, err) + return err + } + + err = bc.operator.ForceDeploy(bc.desCluster) + if err != nil { + glog.Errorf("cluster:[%s] deploy happen error: %v", bc.desCluster.ClusterName, err) + return err + } + + err = bc.operator.CheckTidbClusterStatus(bc.desCluster) + if err != nil { + glog.Errorf("cluster:[%s] deploy faild error: %v", bc.desCluster.ClusterName, err) + return err + } + + err = bc.operator.Restore(bc.srcCluster, bc.desCluster) + if err != nil { + glog.Errorf("from cluster:[%s] to cluster [%s] restore happen error: %v", bc.srcCluster.ClusterName, bc.desCluster.ClusterName, err) + return err + } + + err = bc.operator.CheckRestore(bc.srcCluster, bc.desCluster) + if err != nil { + glog.Errorf("from cluster:[%s] to cluster [%s] restore failed error: %v", bc.srcCluster.ClusterName, bc.desCluster.ClusterName, err) + return err + } + + return nil +} diff --git a/tests/cmd/e2e/main.go b/tests/cmd/e2e/main.go index d88878ad59..607ee90b55 100644 --- a/tests/cmd/e2e/main.go +++ b/tests/cmd/e2e/main.go @@ -19,6 +19,7 @@ import ( "github.com/golang/glog" "github.com/pingcap/tidb-operator/pkg/client/clientset/versioned" "github.com/pingcap/tidb-operator/tests" + "github.com/pingcap/tidb-operator/tests/backup" "k8s.io/apiserver/pkg/util/logs" "k8s.io/client-go/kubernetes" "k8s.io/client-go/rest" @@ -47,11 +48,12 @@ func main() { operatorInfo := &tests.OperatorInfo{ Namespace: "pingcap", ReleaseName: "operator", - Image: "pingcap/tidb-operator:v1.0.0-beta.1-p2", - Tag: "v1.0.0-beta.1-p2", + Image: "pingcap/tidb-operator:latest", + Tag: "master", SchedulerImage: "gcr.io/google-containers/hyperkube:v1.12.1", LogLevel: "2", } + if err := oa.CleanOperator(operatorInfo); err != nil { glog.Fatal(err) } @@ -62,7 +64,7 @@ func main() { clusterInfo := &tests.TidbClusterInfo{ Namespace: "tidb", ClusterName: "demo", - OperatorTag: "v1.0.0-beta.1-p2", + OperatorTag: "master", PDImage: "pingcap/pd:v2.1.3", TiKVImage: "pingcap/tikv:v2.1.3", TiDBImage: "pingcap/tidb:v2.1.3", @@ -70,6 +72,11 @@ func main() { Password: "admin", Args: map[string]string{}, } + + if err := oa.CreateSecret(clusterInfo); err != nil { + glog.Fatal(err) + } + if err := oa.CleanTidbCluster(clusterInfo); err != nil { glog.Fatal(err) } @@ -83,7 +90,7 @@ func main() { restoreClusterInfo := &tests.TidbClusterInfo{ Namespace: "tidb", ClusterName: "demo2", - OperatorTag: "v1.0.0-beta.1-p2", + OperatorTag: "master", PDImage: "pingcap/pd:v2.1.3", TiKVImage: "pingcap/tikv:v2.1.3", TiDBImage: "pingcap/tidb:v2.1.3", @@ -101,4 +108,10 @@ func main() { if err := oa.CheckTidbClusterStatus(restoreClusterInfo); err != nil { glog.Fatal(err) } + + backupCase := backup.NewBackupCase(oa, clusterInfo, restoreClusterInfo) + + if err := backupCase.Run(); err != nil { + glog.Fatal(err) + } } From da93efd4c33f1ae1c3a08f1f6cb79875b2dcd4fe Mon Sep 17 00:00:00 2001 From: zyguan Date: Mon, 18 Mar 2019 11:44:25 +0800 Subject: [PATCH 06/22] stability: add scale & upgrade case functions (#309) * impl scale & upgrade cases --- tests/actions.go | 67 +++++++++++++++++++++++++++++++---- tests/cluster_info.go | 50 ++++++++++++++++++++++++++ tests/cmd/e2e/main.go | 81 +++++++++++++++++++++++++++---------------- 3 files changed, 162 insertions(+), 36 deletions(-) create mode 100644 tests/cluster_info.go diff --git a/tests/actions.go b/tests/actions.go index 61b50b11c3..35b54b3cbf 100644 --- a/tests/actions.go +++ b/tests/actions.go @@ -24,11 +24,13 @@ import ( _ "github.com/go-sql-driver/mysql" "github.com/golang/glog" + "github.com/pingcap/errors" "github.com/pingcap/kvproto/pkg/metapb" "github.com/pingcap/tidb-operator/pkg/apis/pingcap.com/v1alpha1" "github.com/pingcap/tidb-operator/pkg/client/clientset/versioned" "github.com/pingcap/tidb-operator/pkg/controller" "github.com/pingcap/tidb-operator/pkg/label" + "k8s.io/api/apps/v1beta1" batchv1 "k8s.io/api/batch/v1" corev1 "k8s.io/api/core/v1" metav1 "k8s.io/apimachinery/pkg/apis/meta/v1" @@ -103,6 +105,8 @@ type operatorActions struct { pdControl controller.PDControlInterface } +var _ = OperatorActions(&operatorActions{}) + type OperatorInfo struct { Namespace string ReleaseName string @@ -122,7 +126,7 @@ type TidbClusterInfo struct { StorageClassName string Password string RecordCount string - InsertBetchSize string + InsertBatchSize string Resources map[string]string Args map[string]string } @@ -344,6 +348,7 @@ func (oa *operatorActions) CheckTidbClusterStatus(info *TidbClusterInfo) error { return true, nil }); err != nil { + glog.Infof("check tidb cluster status failed: %s", err.Error()) return fmt.Errorf("failed to waiting for tidbcluster %s/%s ready in 10 minutes", ns, tcName) } @@ -358,10 +363,44 @@ func (oa *operatorActions) StopInsertDataTo(info *TidbClusterInfo) error { return nil } -func (oa *operatorActions) ScaleTidbCluster(info *TidbClusterInfo) error { return nil } -func (oa *operatorActions) UpgradeTidbCluster(info *TidbClusterInfo) error { return nil } -func (oa *operatorActions) DeployMonitor(info *TidbClusterInfo) error { return nil } -func (oa *operatorActions) CleanMonitor(info *TidbClusterInfo) error { return nil } +func chartPath(name string, tag string) string { + return "/charts/" + tag + "/" + name +} + +func (oa *operatorActions) ScaleTidbCluster(info *TidbClusterInfo) error { + cmd := fmt.Sprintf("helm upgrade %s %s --set-string %s", + info.ClusterName, chartPath("tidb-cluster", info.OperatorTag), info.HelmSetString()) + glog.Info("[SCALE] " + cmd) + res, err := exec.Command("/bin/sh", "-c", cmd).CombinedOutput() + if err != nil { + return errors.Wrapf(err, "failed to scale tidb cluster: %s", string(res)) + } + return nil +} + +func (oa *operatorActions) UpgradeTidbCluster(info *TidbClusterInfo) error { + cmd := fmt.Sprintf("helm upgrade %s %s --set-string %s", + info.ClusterName, chartPath("tidb-cluster", info.OperatorTag), info.HelmSetString()) + glog.Info("[UPGRADE] " + cmd) + res, err := exec.Command("/bin/sh", "-c", cmd).CombinedOutput() + if err != nil { + return errors.Wrapf(err, "failed to upgrade tidb cluster: %s", string(res)) + } + return nil +} + +func (oa *operatorActions) DeployMonitor(info *TidbClusterInfo) error { return nil } +func (oa *operatorActions) CleanMonitor(info *TidbClusterInfo) error { return nil } + +func getComponentContainer(set *v1beta1.StatefulSet) (corev1.Container, bool) { + name := set.Labels[label.ComponentLabelKey] + for _, c := range set.Spec.Template.Spec.Containers { + if c.Name == name { + return c, true + } + } + return corev1.Container{}, false +} func (oa *operatorActions) pdMembersReadyFn(tc *v1alpha1.TidbCluster) (bool, error) { tcName := tc.GetName() @@ -400,6 +439,11 @@ func (oa *operatorActions) pdMembersReadyFn(tc *v1alpha1.TidbCluster) (bool, err ns, pdSetName, pdSet.Status.ReadyReplicas, pdSet.Status.Replicas) return false, nil } + if c, ok := getComponentContainer(pdSet); !ok || tc.Spec.PD.Image != c.Image { + glog.Infof("statefulset: %s/%s .spec.template.spec.containers[name=pd].image(%s) != %s", + ns, pdSetName, c.Image, tc.Spec.PD.Image) + return false, nil + } for _, member := range tc.Status.PD.Members { if !member.Health { @@ -460,6 +504,11 @@ func (oa *operatorActions) tikvMembersReadyFn(tc *v1alpha1.TidbCluster) (bool, e ns, tikvSetName, tikvSet.Status.ReadyReplicas, tikvSet.Status.Replicas) return false, nil } + if c, ok := getComponentContainer(tikvSet); !ok || tc.Spec.TiKV.Image != c.Image { + glog.Infof("statefulset: %s/%s .spec.template.spec.containers[name=tikv].image(%s) != %s", + ns, tikvSetName, c.Image, tc.Spec.TiKV.Image) + return false, nil + } for _, store := range tc.Status.TiKV.Stores { if store.State != v1alpha1.TiKVStateUp { @@ -509,6 +558,11 @@ func (oa *operatorActions) tidbMembersReadyFn(tc *v1alpha1.TidbCluster) (bool, e ns, tidbSetName, tidbSet.Status.ReadyReplicas, tidbSet.Status.Replicas) return false, nil } + if c, ok := getComponentContainer(tidbSet); !ok || tc.Spec.TiDB.Image != c.Image { + glog.Infof("statefulset: %s/%s .spec.template.spec.containers[name=tikv].image(%s) != %s", + ns, tidbSetName, c.Image, tc.Spec.TiDB.Image) + return false, nil + } _, err = oa.kubeCli.CoreV1().Services(ns).Get(tidbSetName, metav1.GetOptions{}) if err != nil { @@ -577,8 +631,9 @@ outerLoop: for _, pod := range podList.Items { podName := pod.GetName() if pod.Labels[label.ClusterIDLabelKey] != clusterID { - return false, fmt.Errorf("tidbcluster %s/%s's pod %s's label %s not equals %s ", + glog.Infof("tidbcluster %s/%s's pod %s's label %s not equals %s ", ns, tcName, podName, label.ClusterIDLabelKey, clusterID) + return false, nil } component := pod.Labels[label.ComponentLabelKey] diff --git a/tests/cluster_info.go b/tests/cluster_info.go new file mode 100644 index 0000000000..332be46759 --- /dev/null +++ b/tests/cluster_info.go @@ -0,0 +1,50 @@ +package tests + +import "strconv" + +func (tc *TidbClusterInfo) set(name string, value string) (string, bool) { + // NOTE: not thread-safe, maybe make info struct immutable + if tc.Args == nil { + tc.Args = make(map[string]string) + } + origVal, ok := tc.Args[name] + tc.Args[name] = value + return origVal, ok +} + +func (tc *TidbClusterInfo) ScalePD(replicas uint) *TidbClusterInfo { + tc.set("pd.replicas", strconv.Itoa(int(replicas))) + return tc +} + +func (tc *TidbClusterInfo) ScaleTiKV(replicas uint) *TidbClusterInfo { + tc.set("tikv.replicas", strconv.Itoa(int(replicas))) + return tc +} + +func (tc *TidbClusterInfo) ScaleTiDB(replicas uint) *TidbClusterInfo { + tc.set("tidb.replicas", strconv.Itoa(int(replicas))) + return tc +} + +func (tc *TidbClusterInfo) UpgradePD(image string) *TidbClusterInfo { + tc.PDImage = image + return tc +} + +func (tc *TidbClusterInfo) UpgradeTiKV(image string) *TidbClusterInfo { + tc.TiKVImage = image + return tc +} + +func (tc *TidbClusterInfo) UpgradeTiDB(image string) *TidbClusterInfo { + tc.TiDBImage = image + return tc +} + +func (tc *TidbClusterInfo) UpgradeAll(tag string) *TidbClusterInfo { + return tc. + UpgradePD("pingcap/pd:" + tag). + UpgradeTiKV("pingcap/tikv:" + tag). + UpgradeTiDB("pingcap/tidb:" + tag) +} diff --git a/tests/cmd/e2e/main.go b/tests/cmd/e2e/main.go index 607ee90b55..9f8214ed3b 100644 --- a/tests/cmd/e2e/main.go +++ b/tests/cmd/e2e/main.go @@ -25,6 +25,12 @@ import ( "k8s.io/client-go/rest" ) +func perror(err error) { + if err != nil { + glog.Fatal(err) + } +} + func main() { flag.Parse() logs.InitLogs() @@ -53,13 +59,8 @@ func main() { SchedulerImage: "gcr.io/google-containers/hyperkube:v1.12.1", LogLevel: "2", } - - if err := oa.CleanOperator(operatorInfo); err != nil { - glog.Fatal(err) - } - if err := oa.DeployOperator(operatorInfo); err != nil { - glog.Fatal(err) - } + perror(oa.CleanOperator(operatorInfo)) + perror(oa.DeployOperator(operatorInfo)) clusterInfo := &tests.TidbClusterInfo{ Namespace: "tidb", @@ -70,22 +71,34 @@ func main() { TiDBImage: "pingcap/tidb:v2.1.3", StorageClassName: "local-storage", Password: "admin", - Args: map[string]string{}, + Resources: map[string]string{ + "pd.resources.limits.cpu": "1000m", + "pd.resources.limits.memory": "2Gi", + "pd.resources.requests.cpu": "200m", + "pd.resources.requests.memory": "1Gi", + "tikv.resources.limits.cpu": "2000m", + "tikv.resources.limits.memory": "4Gi", + "tikv.resources.requests.cpu": "1000m", + "tikv.resources.requests.memory": "2Gi", + "tidb.resources.limits.cpu": "2000m", + "tidb.resources.limits.memory": "4Gi", + "tidb.resources.requests.cpu": "500m", + "tidb.resources.requests.memory": "1Gi", + }, + Args: map[string]string{}, } - if err := oa.CreateSecret(clusterInfo); err != nil { - glog.Fatal(err) - } + perror(oa.CleanTidbCluster(clusterInfo)) + perror(oa.DeployTidbCluster(clusterInfo)) + perror(oa.CheckTidbClusterStatus(clusterInfo)) - if err := oa.CleanTidbCluster(clusterInfo); err != nil { - glog.Fatal(err) - } - if err := oa.DeployTidbCluster(clusterInfo); err != nil { - glog.Fatal(err) - } - if err := oa.CheckTidbClusterStatus(clusterInfo); err != nil { - glog.Fatal(err) - } + clusterInfo = clusterInfo.ScaleTiDB(3) + perror(oa.ScaleTidbCluster(clusterInfo)) + perror(oa.CheckTidbClusterStatus(clusterInfo)) + + clusterInfo = clusterInfo.UpgradeAll("v2.1.4") + perror(oa.UpgradeTidbCluster(clusterInfo)) + perror(oa.CheckTidbClusterStatus(clusterInfo)) restoreClusterInfo := &tests.TidbClusterInfo{ Namespace: "tidb", @@ -96,18 +109,26 @@ func main() { TiDBImage: "pingcap/tidb:v2.1.3", StorageClassName: "local-storage", Password: "admin", - Args: map[string]string{}, + Resources: map[string]string{ + "pd.resources.limits.cpu": "1000m", + "pd.resources.limits.memory": "2Gi", + "pd.resources.requests.cpu": "200m", + "pd.resources.requests.memory": "1Gi", + "tikv.resources.limits.cpu": "2000m", + "tikv.resources.limits.memory": "4Gi", + "tikv.resources.requests.cpu": "1000m", + "tikv.resources.requests.memory": "2Gi", + "tidb.resources.limits.cpu": "2000m", + "tidb.resources.limits.memory": "4Gi", + "tidb.resources.requests.cpu": "500m", + "tidb.resources.requests.memory": "1Gi", + }, + Args: map[string]string{}, } - if err := oa.CleanTidbCluster(restoreClusterInfo); err != nil { - glog.Fatal(err) - } - if err := oa.DeployTidbCluster(restoreClusterInfo); err != nil { - glog.Fatal(err) - } - if err := oa.CheckTidbClusterStatus(restoreClusterInfo); err != nil { - glog.Fatal(err) - } + perror(oa.CleanTidbCluster(restoreClusterInfo)) + perror(oa.DeployTidbCluster(restoreClusterInfo)) + perror(oa.CheckTidbClusterStatus(restoreClusterInfo)) backupCase := backup.NewBackupCase(oa, clusterInfo, restoreClusterInfo) From 20909502097a4173eacd2df2ba5230d89f757971 Mon Sep 17 00:00:00 2001 From: xiaojingchen Date: Mon, 18 Mar 2019 14:39:16 +0800 Subject: [PATCH 07/22] add slack (#318) * add slack --- tests/slack/slack.go | 137 +++++++++++++++++++++++++++++++++++++++++++ 1 file changed, 137 insertions(+) create mode 100644 tests/slack/slack.go diff --git a/tests/slack/slack.go b/tests/slack/slack.go new file mode 100644 index 0000000000..b024a12778 --- /dev/null +++ b/tests/slack/slack.go @@ -0,0 +1,137 @@ +package slack + +import ( + "bytes" + "encoding/json" + "fmt" + "net/http" +) + +var ( + Channel string + WebhookUrl string +) + +type Field struct { + Title string `json:"title"` + Value string `json:"value"` + Short bool `json:"short"` +} + +type Attachment struct { + Fallback string `json:"fallback"` + Color string `json:"color"` + PreText string `json:"pretext"` + AuthorName string `json:"author_name"` + AuthorLink string `json:"author_link"` + AuthorIcon string `json:"author_icon"` + Title string `json:"title"` + TitleLink string `json:"title_link"` + Text string `json:"text"` + ImageUrl string `json:"image_url"` + Fields []*Field `json:"fields"` + Footer string `json:"footer"` + FooterIcon string `json:"footer_icon"` + Timestamp int64 `json:"ts"` + MarkdownIn []string `json:"mrkdwn_in"` +} + +type Payload struct { + Parse string `json:"parse,omitempty"` + Username string `json:"username,omitempty"` + IconUrl string `json:"icon_url,omitempty"` + IconEmoji string `json:"icon_emoji,omitempty"` + Channel string `json:"channel,omitempty"` + Text string `json:"text,omitempty"` + LinkNames string `json:"link_names,omitempty"` + Attachments []Attachment `json:"attachments,omitempty"` + UnfurlLinks bool `json:"unfurl_links,omitempty"` + UnfurlMedia bool `json:"unfurl_media,omitempty"` +} + +func (attachment *Attachment) AddField(field Field) *Attachment { + attachment.Fields = append(attachment.Fields, &field) + return attachment +} + +func Send(webhookUrl string, proxy string, payload Payload) error { + body, err := json.Marshal(payload) + if err != nil { + return err + } + req, err := http.NewRequest("POST", webhookUrl, bytes.NewBuffer(body)) + req.Header.Set("Content-Type", "application/json") + + client := &http.Client{} + resp, err := client.Do(req) + if err != nil { + return err + } + defer resp.Body.Close() + + if err != nil { + return err + } + if resp.StatusCode >= http.StatusBadRequest { + return fmt.Errorf("Error sending msg %+v. Status: %v", payload, resp.Status) + } + return nil +} + +func SendErrMsg(msg string) error { + attachment := Attachment{ + Title: "operator stability test failed", + Color: "fatal", + } + payload := Payload{ + Username: "operator-test", + Channel: Channel, + Text: msg, + IconEmoji: ":ghost:", + Attachments: []Attachment{attachment}, + } + err := Send(WebhookUrl, "", payload) + if err != nil { + return err + } + return nil +} + +func SendGoodMsg(msg string) error { + attachment := Attachment{ + Title: "operator stability test succeeded", + Color: "good", + } + payload := Payload{ + Username: "operator-test", + Channel: Channel, + Text: msg, + IconEmoji: ":sun_with_face:", + Attachments: []Attachment{attachment}, + } + err := Send(WebhookUrl, "", payload) + if err != nil { + return err + } + + return nil +} + +func SendWarnMsg(msg string) error { + attachment := Attachment{ + Title: "operator stability test happen warning", + Color: "warning", + } + payload := Payload{ + Username: "operator-test", + Channel: Channel, + Text: msg, + IconEmoji: ":imp:", + Attachments: []Attachment{attachment}, + } + err := Send(WebhookUrl, "", payload) + if err != nil { + return err + } + return nil +} From 06a374255b31513ed684f923ec92ef966fd2e411 Mon Sep 17 00:00:00 2001 From: xiaojingchen Date: Mon, 18 Mar 2019 15:46:47 +0800 Subject: [PATCH 08/22] log dump when test failed (#317) * add log dump --- tests/actions.go | 10 ++-- tests/cmd/e2e/main.go | 63 +++++++++++++++++----- tests/log_dump.go | 110 +++++++++++++++++++++++++++++++++++++++ tests/manifests/e2e.yaml | 8 +++ 4 files changed, 172 insertions(+), 19 deletions(-) create mode 100644 tests/log_dump.go diff --git a/tests/actions.go b/tests/actions.go index 35b54b3cbf..4327608063 100644 --- a/tests/actions.go +++ b/tests/actions.go @@ -39,11 +39,12 @@ import ( "k8s.io/client-go/kubernetes" ) -func NewOperatorActions(cli versioned.Interface, kubeCli kubernetes.Interface) OperatorActions { +func NewOperatorActions(cli versioned.Interface, kubeCli kubernetes.Interface, logDir string) OperatorActions { return &operatorActions{ cli: cli, kubeCli: kubeCli, pdControl: controller.NewDefaultPDControl(), + logDir: logDir, } } @@ -56,7 +57,7 @@ type OperatorActions interface { DeployOperator(info *OperatorInfo) error CleanOperator(info *OperatorInfo) error UpgradeOperator(info *OperatorInfo) error - DumpAllLogs(info *OperatorInfo, clusterInfo *TidbClusterInfo) error + DumpAllLogs(info *OperatorInfo, clusterInfos []*TidbClusterInfo) error DeployTidbCluster(info *TidbClusterInfo) error CleanTidbCluster(info *TidbClusterInfo) error CheckTidbClusterStatus(info *TidbClusterInfo) error @@ -103,6 +104,7 @@ type operatorActions struct { cli versioned.Interface kubeCli kubernetes.Interface pdControl controller.PDControlInterface + logDir string } var _ = OperatorActions(&operatorActions{}) @@ -219,10 +221,6 @@ func (oa *operatorActions) UpgradeOperator(info *OperatorInfo) error { return nil } -func (oa *operatorActions) DumpAllLogs(info *OperatorInfo, clusterInfo *TidbClusterInfo) error { - return nil -} - func (oa *operatorActions) DeployTidbCluster(info *TidbClusterInfo) error { glog.Infof("begin to deploy tidb cluster cluster[%s] namespace[%s]", info.ClusterName, info.Namespace) defer func() { diff --git a/tests/cmd/e2e/main.go b/tests/cmd/e2e/main.go index 9f8214ed3b..73b303323c 100644 --- a/tests/cmd/e2e/main.go +++ b/tests/cmd/e2e/main.go @@ -49,7 +49,7 @@ func main() { glog.Fatalf("failed to get kubernetes Clientset: %v", err) } - oa := tests.NewOperatorActions(cli, kubeCli) + oa := tests.NewOperatorActions(cli, kubeCli, "/logDir") operatorInfo := &tests.OperatorInfo{ Namespace: "pingcap", @@ -59,8 +59,14 @@ func main() { SchedulerImage: "gcr.io/google-containers/hyperkube:v1.12.1", LogLevel: "2", } - perror(oa.CleanOperator(operatorInfo)) - perror(oa.DeployOperator(operatorInfo)) + if err := oa.CleanOperator(operatorInfo); err != nil { + oa.DumpAllLogs(operatorInfo, nil) + glog.Fatal(err) + } + if err = oa.DeployOperator(operatorInfo); err != nil { + oa.DumpAllLogs(operatorInfo, nil) + glog.Fatal(err) + } clusterInfo := &tests.TidbClusterInfo{ Namespace: "tidb", @@ -88,17 +94,38 @@ func main() { Args: map[string]string{}, } - perror(oa.CleanTidbCluster(clusterInfo)) - perror(oa.DeployTidbCluster(clusterInfo)) - perror(oa.CheckTidbClusterStatus(clusterInfo)) + if err = oa.CleanTidbCluster(clusterInfo); err != nil { + oa.DumpAllLogs(operatorInfo, []*tests.TidbClusterInfo{clusterInfo}) + glog.Fatal(err) + } + if err = oa.DeployTidbCluster(clusterInfo); err != nil { + oa.DumpAllLogs(operatorInfo, []*tests.TidbClusterInfo{clusterInfo}) + glog.Fatal(err) + } + if err = oa.CheckTidbClusterStatus(clusterInfo); err != nil { + oa.DumpAllLogs(operatorInfo, []*tests.TidbClusterInfo{clusterInfo}) + glog.Fatal(err) + } clusterInfo = clusterInfo.ScaleTiDB(3) - perror(oa.ScaleTidbCluster(clusterInfo)) - perror(oa.CheckTidbClusterStatus(clusterInfo)) + if err := oa.ScaleTidbCluster(clusterInfo); err != nil { + oa.DumpAllLogs(operatorInfo, []*tests.TidbClusterInfo{clusterInfo}) + glog.Fatal(err) + } + if err = oa.CheckTidbClusterStatus(clusterInfo); err != nil { + oa.DumpAllLogs(operatorInfo, []*tests.TidbClusterInfo{clusterInfo}) + glog.Fatal(err) + } clusterInfo = clusterInfo.UpgradeAll("v2.1.4") - perror(oa.UpgradeTidbCluster(clusterInfo)) - perror(oa.CheckTidbClusterStatus(clusterInfo)) + if err = oa.UpgradeTidbCluster(clusterInfo); err != nil { + oa.DumpAllLogs(operatorInfo, []*tests.TidbClusterInfo{clusterInfo}) + glog.Fatal(err) + } + if err = oa.CheckTidbClusterStatus(clusterInfo); err != nil { + oa.DumpAllLogs(operatorInfo, []*tests.TidbClusterInfo{clusterInfo}) + glog.Fatal(err) + } restoreClusterInfo := &tests.TidbClusterInfo{ Namespace: "tidb", @@ -126,13 +153,23 @@ func main() { Args: map[string]string{}, } - perror(oa.CleanTidbCluster(restoreClusterInfo)) - perror(oa.DeployTidbCluster(restoreClusterInfo)) - perror(oa.CheckTidbClusterStatus(restoreClusterInfo)) + if err = oa.CleanTidbCluster(restoreClusterInfo); err != nil { + oa.DumpAllLogs(operatorInfo, []*tests.TidbClusterInfo{clusterInfo, restoreClusterInfo}) + glog.Fatal(err) + } + if err = oa.DeployTidbCluster(restoreClusterInfo); err != nil { + oa.DumpAllLogs(operatorInfo, []*tests.TidbClusterInfo{clusterInfo, restoreClusterInfo}) + glog.Fatal(err) + } + if err = oa.CheckTidbClusterStatus(restoreClusterInfo); err != nil { + oa.DumpAllLogs(operatorInfo, []*tests.TidbClusterInfo{clusterInfo, restoreClusterInfo}) + glog.Fatal(err) + } backupCase := backup.NewBackupCase(oa, clusterInfo, restoreClusterInfo) if err := backupCase.Run(); err != nil { + oa.DumpAllLogs(operatorInfo, []*tests.TidbClusterInfo{clusterInfo, restoreClusterInfo}) glog.Fatal(err) } } diff --git a/tests/log_dump.go b/tests/log_dump.go new file mode 100644 index 0000000000..97de55c0ad --- /dev/null +++ b/tests/log_dump.go @@ -0,0 +1,110 @@ +package tests + +import ( + "bufio" + "fmt" + "os" + "os/exec" + "path/filepath" + + corev1 "k8s.io/api/core/v1" + metav1 "k8s.io/apimachinery/pkg/apis/meta/v1" +) + +func (oa *operatorActions) DumpAllLogs(operatorInfo *OperatorInfo, testClusters []*TidbClusterInfo) error { + logPath := fmt.Sprintf("/%s/%s", oa.logDir, "operator-stability") + if _, err := os.Stat(logPath); os.IsNotExist(err) { + err = os.MkdirAll(logPath, os.ModePerm) + if err != nil { + return err + } + } + + // dump all resources info + resourceLogFile, err := os.Create(filepath.Join(logPath, "resources")) + if err != nil { + return err + } + defer resourceLogFile.Close() + resourceWriter := bufio.NewWriter(resourceLogFile) + dumpLog("kubectl get po -owide -n kube-system", resourceWriter) + dumpLog(fmt.Sprintf("kubectl get po -owide -n %s", operatorInfo.Namespace), resourceWriter) + dumpLog("kubectl get pv", resourceWriter) + dumpLog("kubectl get pv -oyaml", resourceWriter) + dumpedNamespace := map[string]bool{} + for _, testCluster := range testClusters { + if _, exist := dumpedNamespace[testCluster.Namespace]; !exist { + dumpLog(fmt.Sprintf("kubectl get po,pvc,svc,cm,cronjobs,jobs,statefulsets,tidbclusters -owide -n %s", testCluster.Namespace), resourceWriter) + dumpLog(fmt.Sprintf("kubectl get po,pvc,svc,cm,cronjobs,jobs,statefulsets,tidbclusters -n %s -oyaml", testCluster.Namespace), resourceWriter) + dumpedNamespace[testCluster.Namespace] = true + } + } + + // dump operator components's log + operatorPods, err := oa.kubeCli.CoreV1().Pods(operatorInfo.Namespace).List(metav1.ListOptions{}) + if err != nil { + return err + } + for _, pod := range operatorPods.Items { + err := dumpPod(logPath, &pod) + if err != nil { + return err + } + } + + // dump all test clusters's logs + dumpedNamespace = map[string]bool{} + for _, testCluster := range testClusters { + if _, exist := dumpedNamespace[testCluster.Namespace]; !exist { + clusterPodList, err := oa.kubeCli.CoreV1().Pods(testCluster.Namespace).List(metav1.ListOptions{}) + if err != nil { + return err + } + for _, pod := range clusterPodList.Items { + err := dumpPod(logPath, &pod) + if err != nil { + return err + } + } + dumpedNamespace[testCluster.Namespace] = true + } + } + + return nil +} + +func dumpPod(logPath string, pod *corev1.Pod) error { + logFile, err := os.Create(filepath.Join(logPath, fmt.Sprintf("%s-%s.log", pod.Name, pod.Namespace))) + if err != nil { + return err + } + defer logFile.Close() + plogFile, err := os.Create(filepath.Join(logPath, fmt.Sprintf("%s-%s-p.log", pod.Name, pod.Namespace))) + if err != nil { + return err + } + defer plogFile.Close() + + logWriter := bufio.NewWriter(logFile) + plogWriter := bufio.NewWriter(plogFile) + defer logWriter.Flush() + defer plogWriter.Flush() + + for _, c := range pod.Spec.Containers { + dumpLog(fmt.Sprintf("kubectl logs -n %s %s -c %s", pod.Namespace, pod.GetName(), c.Name), logWriter) + dumpLog(fmt.Sprintf("kubectl logs -n %s %s -c %s -p", pod.Namespace, pod.GetName(), c.Name), plogWriter) + } + + return nil +} + +func dumpLog(cmdStr string, writer *bufio.Writer) { + writer.WriteString(fmt.Sprintf("$ %s\n", cmdStr)) + cmd := exec.Command("/bin/sh", "-c", "/usr/local/bin/"+cmdStr) + cmd.Stderr = writer + cmd.Stdout = writer + err := cmd.Run() + if err != nil { + writer.WriteString(err.Error()) + } +} diff --git a/tests/manifests/e2e.yaml b/tests/manifests/e2e.yaml index ed50c4dcc5..77e44a4559 100644 --- a/tests/manifests/e2e.yaml +++ b/tests/manifests/e2e.yaml @@ -30,4 +30,12 @@ spec: image: "" imagePullPolicy: Always command: ["sh", "-c", "/usr/local/bin/e2e"] + volumeMounts: + - mountPath: /logDir + name: logdir + volumes: + - name: logdir + hostPath: + path: /var/log + type: Directory restartPolicy: Never From 07f838d0a863515f33efb75a50094c73ec958315 Mon Sep 17 00:00:00 2001 From: CWen Date: Mon, 18 Mar 2019 19:52:51 +0800 Subject: [PATCH 09/22] stability: add fault-trigger client (#326) * stability: add fault-trigger client --- tests/actions.go | 8 +- tests/pkg/fault-trigger/api/response.go | 27 ++- tests/pkg/fault-trigger/api/router.go | 5 +- tests/pkg/fault-trigger/client/client.go | 194 +++++++++++++++++ tests/pkg/fault-trigger/client/client_test.go | 201 ++++++++++++++++++ tests/pkg/fault-trigger/manager/etcd.go | 4 +- tests/pkg/fault-trigger/manager/kubelet.go | 4 +- tests/pkg/fault-trigger/manager/types.go | 10 + tests/pkg/util/http.go | 15 ++ 9 files changed, 457 insertions(+), 11 deletions(-) create mode 100644 tests/pkg/fault-trigger/client/client.go create mode 100644 tests/pkg/fault-trigger/client/client_test.go create mode 100644 tests/pkg/util/http.go diff --git a/tests/actions.go b/tests/actions.go index 4327608063..dc32d9de3b 100644 --- a/tests/actions.go +++ b/tests/actions.go @@ -419,7 +419,7 @@ func (oa *operatorActions) pdMembersReadyFn(tc *v1alpha1.TidbCluster) (bool, err replicas := tc.Spec.PD.Replicas + int32(failureCount) if *pdSet.Spec.Replicas != replicas { glog.Infof("statefulset: %s/%s .spec.Replicas(%d) != %d", - ns, pdSetName, *pdSet.Spec.Replicas, ns, tcName, replicas) + ns, pdSetName, *pdSet.Spec.Replicas, replicas) return false, nil } if pdSet.Status.ReadyReplicas != tc.Spec.PD.Replicas { @@ -589,7 +589,7 @@ func (oa *operatorActions) reclaimPolicySyncFn(tc *v1alpha1.TidbCluster) (bool, for _, pvc := range pvcList.Items { pvName := pvc.Spec.VolumeName if pv, err := oa.kubeCli.CoreV1().PersistentVolumes().Get(pvName, metav1.GetOptions{}); err != nil { - glog.Errorf("failed to get pv: %s", pvName, err) + glog.Errorf("failed to get pv: %s, error: %v", pvName, err) return false, nil } else if pv.Spec.PersistentVolumeReclaimPolicy != tc.Spec.PVReclaimPolicy { glog.Errorf("pv: %s's reclaimPolicy is not Retain", pvName) @@ -608,7 +608,7 @@ func (oa *operatorActions) metaSyncFn(tc *v1alpha1.TidbCluster) (bool, error) { var cluster *metapb.Cluster var err error if cluster, err = pdCli.GetCluster(); err != nil { - glog.Errorf("failed to get cluster from pdControl: %s/%s", ns, tcName, err) + glog.Errorf("failed to get cluster from pdControl: %s/%s, error: %v", ns, tcName, err) return false, nil } @@ -808,7 +808,7 @@ func (oa *operatorActions) schedulerHAFn(tc *v1alpha1.TidbCluster) (bool, error) } nodeMap[nodeName] = append(nodeMap[nodeName], pod.GetName()) if len(nodeMap[nodeName]) > totalCount/2 { - return false, fmt.Errorf("node % have %d pods, greater than %d/2", + return false, fmt.Errorf("node %s have %d pods, greater than %d/2", nodeName, len(nodeMap[nodeName]), totalCount) } } diff --git a/tests/pkg/fault-trigger/api/response.go b/tests/pkg/fault-trigger/api/response.go index 9280cf66f7..cd9a45070b 100644 --- a/tests/pkg/fault-trigger/api/response.go +++ b/tests/pkg/fault-trigger/api/response.go @@ -13,7 +13,13 @@ package api -import "net/http" +import ( + "encoding/json" + "net/http" + + "github.com/golang/glog" + "github.com/juju/errors" +) // Response defines a new response struct for http type Response struct { @@ -41,3 +47,22 @@ func (r *Response) payload(payload interface{}) *Response { r.Payload = payload return r } + +// ExtractResponse extract response from api +func ExtractResponse(data []byte) ([]byte, error) { + respData := &Response{} + if err := json.Unmarshal(data, respData); err != nil { + return nil, errors.Trace(err) + } + + if respData.StatusCode != http.StatusOK { + d, err := json.Marshal(respData.Payload) + if err != nil { + glog.Errorf("marshal data failed %v", d) + } + + return d, errors.New(respData.Message) + } + + return json.Marshal(respData.Payload) +} diff --git a/tests/pkg/fault-trigger/api/router.go b/tests/pkg/fault-trigger/api/router.go index bb58405bab..9b39afcfa7 100644 --- a/tests/pkg/fault-trigger/api/router.go +++ b/tests/pkg/fault-trigger/api/router.go @@ -16,13 +16,14 @@ package api import restful "github.com/emicklei/go-restful" const ( - apiPrefix = "/pingcap.com/api/v1" + // APIPrefix defines a prefix string for fault-trigger api + APIPrefix = "/pingcap.com/api/v1" ) func (s *Server) newService() *restful.WebService { ws := new(restful.WebService) ws. - Path(apiPrefix). + Path(APIPrefix). Consumes(restful.MIME_JSON). Produces(restful.MIME_JSON) diff --git a/tests/pkg/fault-trigger/client/client.go b/tests/pkg/fault-trigger/client/client.go new file mode 100644 index 0000000000..17a812cbe6 --- /dev/null +++ b/tests/pkg/fault-trigger/client/client.go @@ -0,0 +1,194 @@ +package client + +import ( + "encoding/json" + "fmt" + "io/ioutil" + "net/http" + + "github.com/pingcap/tidb-operator/tests/pkg/fault-trigger/api" + "github.com/pingcap/tidb-operator/tests/pkg/fault-trigger/manager" + "github.com/pingcap/tidb-operator/tests/pkg/util" +) + +// Client is a fault-trigger client +type Client interface { + // ListVMs lists all virtual machines + ListVMs() ([]*manager.VM, error) + // StartVM start a specified virtual machine + StartVM(vm *manager.VM) error + // StopVM stops a specified virtual machine + StopVM(vm *manager.VM) error + // StartETCD starts the etcd service + StartETCD() error + // StopETCD stops the etcd service + StopETCD() error + // StartKubelet starts the kubelet service + StartKubelet() error + // StopKubelet stops the kubelet service + StopKubelet() error +} + +// client is used to communicate with the fault-trigger +type client struct { + cfg Config + httpCli *http.Client +} + +// Config defines for fault-trigger client +type Config struct { + Addr string +} + +// NewClient creates a new fault-trigger client from a given address +func NewClient(cfg Config) Client { + return &client{ + cfg: cfg, + httpCli: http.DefaultClient, + } +} + +type clientError struct { + code int + msg string +} + +func (e *clientError) Error() string { + return fmt.Sprintf("%s (code: %d)", e.msg, e.code) +} + +func (c client) do(req *http.Request) (*http.Response, []byte, error) { + resp, err := c.httpCli.Do(req) + if err != nil { + return nil, nil, err + } + defer resp.Body.Close() + + code := resp.StatusCode + + if code != http.StatusOK { + return resp, nil, &clientError{ + code: code, + msg: "fail to request to http service", + } + } + + bodyByte, err := ioutil.ReadAll(resp.Body) + if err != nil { + return resp, nil, &clientError{ + code: code, + msg: fmt.Sprintf("failed to read data from resp body, error: %v", err), + } + } + + data, err := api.ExtractResponse(bodyByte) + if err != nil { + return resp, nil, &clientError{ + code: code, + msg: err.Error(), + } + } + + return resp, data, err +} + +func (c client) get(url string) ([]byte, error) { + req, err := http.NewRequest(http.MethodGet, url, nil) + if err != nil { + return nil, fmt.Errorf("error creating request: %v", err) + } + + _, body, err := c.do(req) + if err != nil { + return nil, err + } + + return body, nil +} + +func (c *client) ListVMs() ([]*manager.VM, error) { + url := util.GenURL(fmt.Sprintf("%s%s/vms", c.cfg.Addr, api.APIPrefix)) + data, err := c.get(url) + if err != nil { + return nil, err + } + + var vms []*manager.VM + if err = json.Unmarshal(data, &vms); err != nil { + return nil, err + } + + return vms, nil +} + +func (c *client) StartVM(vm *manager.VM) error { + if err := vm.Verify(); err != nil { + return err + } + + vmName := vm.Name + if len(vmName) == 0 { + vmName = vm.IP + } + + url := util.GenURL(fmt.Sprintf("%s/%s/vm/%s/start", c.cfg.Addr, api.APIPrefix, vmName)) + if _, err := c.get(url); err != nil { + return err + } + + return nil +} + +func (c *client) StopVM(vm *manager.VM) error { + if err := vm.Verify(); err != nil { + return err + } + + vmName := vm.Name + if len(vmName) == 0 { + vmName = vm.IP + } + + url := util.GenURL(fmt.Sprintf("%s/%s/vm/%s/stop", c.cfg.Addr, api.APIPrefix, vmName)) + if _, err := c.get(url); err != nil { + return err + } + + return nil +} + +func (c *client) StartETCD() error { + url := util.GenURL(fmt.Sprintf("%s/%s/etcd/start", c.cfg.Addr, api.APIPrefix)) + if _, err := c.get(url); err != nil { + return err + } + + return nil +} + +func (c *client) StopETCD() error { + url := util.GenURL(fmt.Sprintf("%s/%s/etcd/stop", c.cfg.Addr, api.APIPrefix)) + if _, err := c.get(url); err != nil { + return err + } + + return nil +} + +func (c *client) StartKubelet() error { + url := util.GenURL(fmt.Sprintf("%s/%s/kubelet/start", c.cfg.Addr, api.APIPrefix)) + if _, err := c.get(url); err != nil { + return err + } + + return nil +} + +func (c *client) StopKubelet() error { + url := util.GenURL(fmt.Sprintf("%s/%s/kubelet/stop", c.cfg.Addr, api.APIPrefix)) + if _, err := c.get(url); err != nil { + return err + } + + return nil +} diff --git a/tests/pkg/fault-trigger/client/client_test.go b/tests/pkg/fault-trigger/client/client_test.go new file mode 100644 index 0000000000..0cdb497b81 --- /dev/null +++ b/tests/pkg/fault-trigger/client/client_test.go @@ -0,0 +1,201 @@ +package client + +import ( + "encoding/json" + "fmt" + "net/http" + "net/http/httptest" + "testing" + + . "github.com/onsi/gomega" + "github.com/pingcap/tidb-operator/tests/pkg/fault-trigger/api" + "github.com/pingcap/tidb-operator/tests/pkg/fault-trigger/manager" +) + +func TestListVMs(t *testing.T) { + g := NewGomegaWithT(t) + + vms := []*manager.VM{ + { + Name: "vm1", + IP: "10.16.30.11", + }, + { + Name: "vm2", + IP: "10.16.30.12", + }, + } + + resp := &api.Response{ + Action: "listVMs", + StatusCode: 200, + Payload: vms, + } + + respJSON, _ := json.Marshal(resp) + ts := httptest.NewServer(http.HandlerFunc(func(w http.ResponseWriter, r *http.Request) { + fmt.Fprintln(w, string(respJSON)) + })) + defer ts.Close() + + cli := NewClient(Config{ + Addr: ts.URL, + }) + + vms2, err := cli.ListVMs() + g.Expect(err).NotTo(HaveOccurred()) + g.Expect(vms).To(Equal(vms2)) +} + +func TestStartVM(t *testing.T) { + g := NewGomegaWithT(t) + + resp := &api.Response{ + Action: "startVM", + StatusCode: 200, + Message: "OK", + } + + respJSON, _ := json.Marshal(resp) + ts := httptest.NewServer(http.HandlerFunc(func(w http.ResponseWriter, r *http.Request) { + fmt.Fprintln(w, string(respJSON)) + })) + defer ts.Close() + + cli := NewClient(Config{ + Addr: ts.URL, + }) + + err := cli.StartVM(&manager.VM{ + Name: "vm1", + }) + g.Expect(err).NotTo(HaveOccurred()) + + err = cli.StartVM(&manager.VM{ + Host: "127.0.0.1", + }) + g.Expect(err).To(HaveOccurred()) +} + +func TestStopVM(t *testing.T) { + g := NewGomegaWithT(t) + + resp := &api.Response{ + Action: "stopVM", + StatusCode: 200, + Message: "OK", + } + + respJSON, _ := json.Marshal(resp) + ts := httptest.NewServer(http.HandlerFunc(func(w http.ResponseWriter, r *http.Request) { + fmt.Fprintln(w, string(respJSON)) + })) + defer ts.Close() + + cli := NewClient(Config{ + Addr: ts.URL, + }) + + err := cli.StopVM(&manager.VM{ + Name: "vm1", + }) + + g.Expect(err).NotTo(HaveOccurred()) + + err = cli.StopVM(&manager.VM{ + Host: "127.0.0.1", + }) + g.Expect(err).To(HaveOccurred()) +} + +func TestStartETCD(t *testing.T) { + g := NewGomegaWithT(t) + + resp := &api.Response{ + Action: "startETCD", + StatusCode: 200, + Message: "OK", + } + + respJSON, _ := json.Marshal(resp) + ts := httptest.NewServer(http.HandlerFunc(func(w http.ResponseWriter, r *http.Request) { + fmt.Fprintln(w, string(respJSON)) + })) + defer ts.Close() + + cli := NewClient(Config{ + Addr: ts.URL, + }) + + err := cli.StartETCD() + g.Expect(err).NotTo(HaveOccurred()) +} + +func TestStopETCD(t *testing.T) { + g := NewGomegaWithT(t) + + resp := &api.Response{ + Action: "stopETCD", + StatusCode: 200, + Message: "OK", + } + + respJSON, _ := json.Marshal(resp) + ts := httptest.NewServer(http.HandlerFunc(func(w http.ResponseWriter, r *http.Request) { + fmt.Fprintln(w, string(respJSON)) + })) + defer ts.Close() + + cli := NewClient(Config{ + Addr: ts.URL, + }) + + err := cli.StopETCD() + g.Expect(err).NotTo(HaveOccurred()) +} + +func TestStartKubelet(t *testing.T) { + g := NewGomegaWithT(t) + + resp := &api.Response{ + Action: "startKubelet", + StatusCode: 200, + Message: "OK", + } + + respJSON, _ := json.Marshal(resp) + ts := httptest.NewServer(http.HandlerFunc(func(w http.ResponseWriter, r *http.Request) { + fmt.Fprintln(w, string(respJSON)) + })) + defer ts.Close() + + cli := NewClient(Config{ + Addr: ts.URL, + }) + + err := cli.StartKubelet() + g.Expect(err).NotTo(HaveOccurred()) +} + +func TestStopKubelet(t *testing.T) { + g := NewGomegaWithT(t) + + resp := &api.Response{ + Action: "stopKubelet", + StatusCode: 200, + Message: "OK", + } + + respJSON, _ := json.Marshal(resp) + ts := httptest.NewServer(http.HandlerFunc(func(w http.ResponseWriter, r *http.Request) { + fmt.Fprintln(w, string(respJSON)) + })) + defer ts.Close() + + cli := NewClient(Config{ + Addr: ts.URL, + }) + + err := cli.StopKubelet() + g.Expect(err).NotTo(HaveOccurred()) +} diff --git a/tests/pkg/fault-trigger/manager/etcd.go b/tests/pkg/fault-trigger/manager/etcd.go index 13c0df2f4c..cd16a527ae 100644 --- a/tests/pkg/fault-trigger/manager/etcd.go +++ b/tests/pkg/fault-trigger/manager/etcd.go @@ -19,7 +19,7 @@ import ( "github.com/golang/glog" ) -// StartETCD starts etcd +// StartETCD starts the etcd service func (m *Manager) StartETCD() error { shell := "systemctl start etcd" cmd := exec.Command("/bin/sh", "-c", shell) @@ -31,7 +31,7 @@ func (m *Manager) StartETCD() error { return nil } -// StopETCD stops etcd +// StopETCD stops the etcd service func (m *Manager) StopETCD() error { shell := "systemctl stop etcd" cmd := exec.Command("/bin/sh", "-c", shell) diff --git a/tests/pkg/fault-trigger/manager/kubelet.go b/tests/pkg/fault-trigger/manager/kubelet.go index 6cdbf11227..4dad0c0c60 100644 --- a/tests/pkg/fault-trigger/manager/kubelet.go +++ b/tests/pkg/fault-trigger/manager/kubelet.go @@ -19,7 +19,7 @@ import ( "github.com/golang/glog" ) -// StartKubelet starts kubelet +// StartKubelet starts the kubelet service func (m *Manager) StartKubelet() error { shell := "systemctl start kubelet" cmd := exec.Command("/bin/sh", "-c", shell) @@ -31,7 +31,7 @@ func (m *Manager) StartKubelet() error { return nil } -// StopKubelet stops kubelet +// StopKubelet stops the kubelet service func (m *Manager) StopKubelet() error { shell := "systemctl stop kubelet" cmd := exec.Command("/bin/sh", "-c", shell) diff --git a/tests/pkg/fault-trigger/manager/types.go b/tests/pkg/fault-trigger/manager/types.go index fa2fa099ec..b2b7e7f1a0 100644 --- a/tests/pkg/fault-trigger/manager/types.go +++ b/tests/pkg/fault-trigger/manager/types.go @@ -13,6 +13,8 @@ package manager +import "errors" + // VM defines the descriptive information of a virtual machine type VM struct { Host string `json:"host"` @@ -21,3 +23,11 @@ type VM struct { IP string `json:"ip"` Role []string `json:"role"` } + +func (v *VM) Verify() error { + if len(v.Name) == 0 && len(v.IP) == 0 { + return errors.New("name or ip must be provided") + } + + return nil +} diff --git a/tests/pkg/util/http.go b/tests/pkg/util/http.go new file mode 100644 index 0000000000..2b97fffc7d --- /dev/null +++ b/tests/pkg/util/http.go @@ -0,0 +1,15 @@ +package util + +import ( + "fmt" + "strings" +) + +// GenURL adds 'http' prefix for URL +func GenURL(url string) string { + if strings.Contains(url, "http") { + return url + } + + return fmt.Sprintf("http://%s", url) +} From fb7184c26ca8fadbaf73b484018411bbf69efcce Mon Sep 17 00:00:00 2001 From: xiaojingchen Date: Mon, 18 Mar 2019 20:38:53 +0800 Subject: [PATCH 10/22] monitor checker (#320) * monitor checker * fix bugs --- tests/actions.go | 131 +++++++++++++++++++++++++++++++++++++++++++++-- 1 file changed, 128 insertions(+), 3 deletions(-) diff --git a/tests/actions.go b/tests/actions.go index dc32d9de3b..8819883337 100644 --- a/tests/actions.go +++ b/tests/actions.go @@ -16,7 +16,11 @@ package tests import ( "bytes" "database/sql" + "encoding/json" "fmt" + "io/ioutil" + "net/http" + "net/url" "os/exec" "strconv" "strings" @@ -50,7 +54,12 @@ func NewOperatorActions(cli versioned.Interface, kubeCli kubernetes.Interface, l const ( DefaultPollTimeout time.Duration = 10 * time.Minute - DefaultPollInterval time.Duration = 1 * time.Minute + DefaultPollInterval time.Duration = 10 * time.Second +) + +const ( + grafanaUsername = "admin" + grafanaPassword = "admin" ) type OperatorActions interface { @@ -73,8 +82,6 @@ type OperatorActions interface { CheckIncrementalBackup(info *TidbClusterInfo) error Restore(from *TidbClusterInfo, to *TidbClusterInfo) error CheckRestore(from *TidbClusterInfo, to *TidbClusterInfo) error - DeployMonitor(info *TidbClusterInfo) error - CleanMonitor(info *TidbClusterInfo) error ForceDeploy(info *TidbClusterInfo) error CreateSecret(info *TidbClusterInfo) error } @@ -131,6 +138,7 @@ type TidbClusterInfo struct { InsertBatchSize string Resources map[string]string Args map[string]string + Monitor bool } func (tc *TidbClusterInfo) HelmSetString() string { @@ -150,6 +158,7 @@ func (tc *TidbClusterInfo) HelmSetString() string { "tidb.image": tc.TiDBImage, "tidb.passwordSecretName": "set-secret", "tidb.initSql": initSql, + "monitor.create": strconv.FormatBool(tc.Monitor), } for k, v := range tc.Resources { @@ -344,6 +353,12 @@ func (oa *operatorActions) CheckTidbClusterStatus(info *TidbClusterInfo) error { return false, nil } + if info.Monitor { + glog.Infof("check tidb monitor normal") + if b, err := oa.monitorNormal(info); !b && err == nil { + return false, nil + } + } return true, nil }); err != nil { glog.Infof("check tidb cluster status failed: %s", err.Error()) @@ -858,6 +873,116 @@ func (oa *operatorActions) passwordIsSet(clusterInfo *TidbClusterInfo) (bool, er return true, nil } +func (oa *operatorActions) monitorNormal(clusterInfo *TidbClusterInfo) (bool, error) { + ns := clusterInfo.Namespace + tcName := clusterInfo.ClusterName + monitorDeploymentName := fmt.Sprintf("%s-monitor", tcName) + monitorDeployment, err := oa.kubeCli.AppsV1().Deployments(ns).Get(monitorDeploymentName, metav1.GetOptions{}) + if err != nil { + glog.Errorf("get monitor deployment: [%s/%s] failed", ns, monitorDeploymentName) + return false, nil + } + if monitorDeployment.Status.ReadyReplicas < 1 { + glog.Info("monitor ready replicas %d < 1", monitorDeployment.Status.ReadyReplicas) + return false, nil + } + configuratorJobName := fmt.Sprintf("%s-monitor-configurator", tcName) + monitorJob, err := oa.kubeCli.BatchV1().Jobs(ns).Get(configuratorJobName, metav1.GetOptions{}) + if err != nil { + glog.Info("get monitor configurator job: [%s/%s] failed", ns, configuratorJobName) + return false, nil + } + if monitorJob.Status.Succeeded == 0 { + glog.Info("the monitor configurator job: [%s/%s] had not success", ns, configuratorJobName) + return false, nil + } + + if err := oa.checkPrometheus(clusterInfo); err != nil { + glog.Info("check [%s/%s]'s prometheus data failed: %v", ns, monitorDeploymentName, err) + return false, nil + } + + if err := oa.checkGrafanaData(clusterInfo); err != nil { + glog.Info("check [%s/%s]'s grafana data failed: %v", ns, monitorDeploymentName, err) + return false, nil + } + return true, nil +} + +func (oa *operatorActions) checkPrometheus(clusterInfo *TidbClusterInfo) error { + ns := clusterInfo.Namespace + tcName := clusterInfo.ClusterName + prometheusSvc := fmt.Sprintf("http://%s-prometheus.%s:9090/api/v1/query?query=up", tcName, ns) + resp, err := http.Get(prometheusSvc) + if err != nil { + return err + } + defer resp.Body.Close() + body, err := ioutil.ReadAll(resp.Body) + if err != nil { + return err + } + response := &struct { + Status string `json:"status"` + }{} + err = json.Unmarshal(body, response) + if err != nil { + return err + } + if response.Status != "success" { + return fmt.Errorf("the prometheus's api[%s] has not ready", prometheusSvc) + } + return nil +} + +func (oa *operatorActions) checkGrafanaData(clusterInfo *TidbClusterInfo) error { + ns := clusterInfo.Namespace + tcName := clusterInfo.ClusterName + svcName := fmt.Sprintf("%s-grafana", tcName) + end := time.Now() + start := end.Add(-time.Minute) + values := url.Values{} + values.Set("query", `sum(tikv_pd_heartbeat_tick_total{type="leader"}) by (job)`) + values.Set("start", fmt.Sprintf("%d", start.Unix())) + values.Set("end", fmt.Sprintf("%d", end.Unix())) + values.Set("step", "30") + u := fmt.Sprintf("http://%s.%s.svc.cluster.local:3000/api/datasources/proxy/1/api/v1/query_range?%s", svcName, ns, values.Encode()) + req, err := http.NewRequest(http.MethodGet, u, nil) + if err != nil { + return err + } + req.SetBasicAuth(grafanaUsername, grafanaPassword) + client := &http.Client{} + resp, err := client.Do(req) + if err != nil { + return err + } + defer resp.Body.Close() + buf, err := ioutil.ReadAll(resp.Body) + if err != nil { + return err + } + data := struct { + Status string `json:"status"` + Data struct { + ResultType string `json:"resultType"` + Result []struct { + Metric struct { + Job string `json:"job"` + } `json:"metric"` + Values []interface{} `json:"values"` + } `json:"result"` + } + }{} + if err := json.Unmarshal(buf, &data); err != nil { + return err + } + if data.Status != "success" || len(data.Data.Result) < 1 { + return fmt.Errorf("invalid response: status: %s, result: %v", data.Status, data.Data.Result) + } + return nil +} + func getDSN(ns, tcName, databaseName, password string) string { return fmt.Sprintf("root:%s@(%s-tidb.%s:4000)/%s?charset=utf8", password, tcName, ns, databaseName) } From ca26f7923dca8f2f38c041fc07c99f8f7e9c6858 Mon Sep 17 00:00:00 2001 From: zyguan Date: Mon, 18 Mar 2019 15:40:34 +0800 Subject: [PATCH 11/22] stability: add more checks for scale operation --- tests/actions.go | 79 +++++++++++++++++++++++++++++++++++++++++++ tests/cluster_info.go | 4 ++- tests/cmd/e2e/main.go | 37 ++++++++++++++------ 3 files changed, 108 insertions(+), 12 deletions(-) diff --git a/tests/actions.go b/tests/actions.go index 8819883337..51457fa89a 100644 --- a/tests/actions.go +++ b/tests/actions.go @@ -39,6 +39,7 @@ import ( corev1 "k8s.io/api/core/v1" metav1 "k8s.io/apimachinery/pkg/apis/meta/v1" "k8s.io/apimachinery/pkg/labels" + "k8s.io/apimachinery/pkg/types" "k8s.io/apimachinery/pkg/util/wait" "k8s.io/client-go/kubernetes" ) @@ -73,6 +74,8 @@ type OperatorActions interface { BeginInsertDataTo(info *TidbClusterInfo) error StopInsertDataTo(info *TidbClusterInfo) error ScaleTidbCluster(info *TidbClusterInfo) error + CheckScaleInSafely(info *TidbClusterInfo) error + CheckScaledCorrectly(info *TidbClusterInfo, podUIDsBeforeScale map[string]types.UID) error UpgradeTidbCluster(info *TidbClusterInfo) error DeployAdHocBackup(info *TidbClusterInfo) error CheckAdHocBackup(info *TidbClusterInfo) error @@ -84,6 +87,7 @@ type OperatorActions interface { CheckRestore(from *TidbClusterInfo, to *TidbClusterInfo) error ForceDeploy(info *TidbClusterInfo) error CreateSecret(info *TidbClusterInfo) error + GetPodUIDMap(info *TidbClusterInfo) (map[string]types.UID, error) } type FaultTriggerActions interface { @@ -391,6 +395,63 @@ func (oa *operatorActions) ScaleTidbCluster(info *TidbClusterInfo) error { return nil } +func (oa *operatorActions) CheckScaleInSafely(info *TidbClusterInfo) error { + return wait.Poll(5*time.Second, DefaultPollTimeout, func() (done bool, err error) { + tc, err := oa.cli.PingcapV1alpha1().TidbClusters(info.Namespace).Get(info.ClusterName, metav1.GetOptions{}) + if err != nil { + glog.Infof("failed to get tidbcluster when scale in tidbcluster, error: %v", err) + return false, nil + } + + tikvSetName := controller.TiKVMemberName(info.ClusterName) + tikvSet, err := oa.kubeCli.AppsV1beta1().StatefulSets(info.Namespace).Get(tikvSetName, metav1.GetOptions{}) + if err != nil { + glog.Infof("failed to get tikvSet statefulset: [%s], error: %v", tikvSetName, err) + return false, nil + } + + pdClient := controller.NewDefaultPDControl().GetPDClient(tc) + stores, err := pdClient.GetStores() + if err != nil { + glog.Infof("pdClient.GetStores failed,error: %v", err) + return false, nil + } + if len(stores.Stores) > int(*tikvSet.Spec.Replicas) { + glog.Infof("stores.Stores: %v", stores.Stores) + glog.Infof("tikvSet.Spec.Replicas: %d", *tikvSet.Spec.Replicas) + return false, fmt.Errorf("the tikvSet.Spec.Replicas may reduce before tikv complete offline") + } + + if *tikvSet.Spec.Replicas == tc.Spec.TiKV.Replicas { + return true, nil + } + + return false, nil + }) +} + +func (oa *operatorActions) CheckScaledCorrectly(info *TidbClusterInfo, podUIDsBeforeScale map[string]types.UID) error { + return wait.Poll(DefaultPollInterval, DefaultPollTimeout, func() (done bool, err error) { + podUIDs, err := oa.GetPodUIDMap(info) + if err != nil { + glog.Infof("failed to get pd pods's uid, error: %v", err) + return false, nil + } + + if len(podUIDsBeforeScale) == len(podUIDs) { + return false, fmt.Errorf("the length of pods before scale equals the length of pods after scale") + } + + for podName, uidAfter := range podUIDs { + if uidBefore, ok := podUIDsBeforeScale[podName]; ok && uidBefore != uidAfter { + return false, fmt.Errorf("pod: [%s] have be recreated", podName) + } + } + + return true, nil + }) +} + func (oa *operatorActions) UpgradeTidbCluster(info *TidbClusterInfo) error { cmd := fmt.Sprintf("helm upgrade %s %s --set-string %s", info.ClusterName, chartPath("tidb-cluster", info.OperatorTag), info.HelmSetString()) @@ -1257,3 +1318,21 @@ func (oa *operatorActions) DeployIncrementalBackup(from *TidbClusterInfo, to *Ti func (oa *operatorActions) CheckIncrementalBackup(info *TidbClusterInfo) error { return nil } + +func (oa *operatorActions) GetPodUIDMap(info *TidbClusterInfo) (map[string]types.UID, error) { + result := map[string]types.UID{} + + selector, err := label.New().Instance(info.ClusterName).Selector() + if err != nil { + return nil, err + } + pods, err := oa.kubeCli.CoreV1().Pods(info.Namespace).List(metav1.ListOptions{LabelSelector: selector.String()}) + if err != nil { + return nil, err + } + for _, pod := range pods.Items { + result[pod.GetName()] = pod.GetUID() + } + + return result, nil +} diff --git a/tests/cluster_info.go b/tests/cluster_info.go index 332be46759..e1c20438fb 100644 --- a/tests/cluster_info.go +++ b/tests/cluster_info.go @@ -1,6 +1,8 @@ package tests -import "strconv" +import ( + "strconv" +) func (tc *TidbClusterInfo) set(name string, value string) (string, bool) { // NOTE: not thread-safe, maybe make info struct immutable diff --git a/tests/cmd/e2e/main.go b/tests/cmd/e2e/main.go index 73b303323c..6eef1f862d 100644 --- a/tests/cmd/e2e/main.go +++ b/tests/cmd/e2e/main.go @@ -25,9 +25,15 @@ import ( "k8s.io/client-go/rest" ) -func perror(err error) { +func perror(err error, dumplogs func() error) { if err != nil { - glog.Fatal(err) + if dumplogs != nil { + err := dumplogs() + if err != nil { + glog.Errorf("failed to dump logs: %s", err.Error()) + } + } + glog.FatalDepth(1, err) } } @@ -107,15 +113,24 @@ func main() { glog.Fatal(err) } - clusterInfo = clusterInfo.ScaleTiDB(3) - if err := oa.ScaleTidbCluster(clusterInfo); err != nil { - oa.DumpAllLogs(operatorInfo, []*tests.TidbClusterInfo{clusterInfo}) - glog.Fatal(err) - } - if err = oa.CheckTidbClusterStatus(clusterInfo); err != nil { - oa.DumpAllLogs(operatorInfo, []*tests.TidbClusterInfo{clusterInfo}) - glog.Fatal(err) - } + dumplogs := func() error { return oa.DumpAllLogs(operatorInfo, []*tests.TidbClusterInfo{clusterInfo}) } + + // scale out: tidb 2 -> 3, tikv 3 -> 5 + podUIDsBeforeScale, err := oa.GetPodUIDMap(clusterInfo) + perror(err, dumplogs) + clusterInfo = clusterInfo.ScaleTiDB(3).ScaleTiKV(5) + perror(oa.ScaleTidbCluster(clusterInfo), dumplogs) + perror(oa.CheckTidbClusterStatus(clusterInfo), dumplogs) + perror(oa.CheckScaledCorrectly(clusterInfo, podUIDsBeforeScale), dumplogs) + + // scale in: tikv 5 -> 3 + podUIDsBeforeScale, err = oa.GetPodUIDMap(clusterInfo) + perror(err, dumplogs) + clusterInfo = clusterInfo.ScaleTiKV(3) + perror(oa.ScaleTidbCluster(clusterInfo), dumplogs) + perror(oa.CheckScaleInSafely(clusterInfo), dumplogs) + perror(oa.CheckTidbClusterStatus(clusterInfo), dumplogs) + perror(oa.CheckScaledCorrectly(clusterInfo, podUIDsBeforeScale), dumplogs) clusterInfo = clusterInfo.UpgradeAll("v2.1.4") if err = oa.UpgradeTidbCluster(clusterInfo); err != nil { From addadb9d30da932c32a25667a4b8e6fe9c7f0dc4 Mon Sep 17 00:00:00 2001 From: zyguan Date: Mon, 18 Mar 2019 22:43:39 +0800 Subject: [PATCH 12/22] stability: add checks for upgrade operations --- tests/actions.go | 165 ++++++++++++++++++++++++++++++++++++++++++ tests/cmd/e2e/main.go | 24 ++++-- 2 files changed, 183 insertions(+), 6 deletions(-) diff --git a/tests/actions.go b/tests/actions.go index 51457fa89a..287dfbca30 100644 --- a/tests/actions.go +++ b/tests/actions.go @@ -22,6 +22,7 @@ import ( "net/http" "net/url" "os/exec" + "sort" "strconv" "strings" "time" @@ -77,6 +78,7 @@ type OperatorActions interface { CheckScaleInSafely(info *TidbClusterInfo) error CheckScaledCorrectly(info *TidbClusterInfo, podUIDsBeforeScale map[string]types.UID) error UpgradeTidbCluster(info *TidbClusterInfo) error + CheckUpgradeProgress(info *TidbClusterInfo) error DeployAdHocBackup(info *TidbClusterInfo) error CheckAdHocBackup(info *TidbClusterInfo) error DeployScheduledBackup(info *TidbClusterInfo) error @@ -88,6 +90,7 @@ type OperatorActions interface { ForceDeploy(info *TidbClusterInfo) error CreateSecret(info *TidbClusterInfo) error GetPodUIDMap(info *TidbClusterInfo) (map[string]types.UID, error) + GetNodeMap(info *TidbClusterInfo, component string) (map[string][]string, error) } type FaultTriggerActions interface { @@ -463,6 +466,146 @@ func (oa *operatorActions) UpgradeTidbCluster(info *TidbClusterInfo) error { return nil } +func (oa *operatorActions) CheckUpgradeProgress(info *TidbClusterInfo) error { + return wait.Poll(5*time.Second, DefaultPollTimeout, func() (done bool, err error) { + tc, err := oa.cli.PingcapV1alpha1().TidbClusters(info.Namespace).Get(info.ClusterName, metav1.GetOptions{}) + if err != nil { + glog.Infof("failed to get tidbcluster: [%s], error: %v", info.ClusterName, err) + return false, nil + } + + pdSetName := controller.PDMemberName(info.ClusterName) + pdSet, err := oa.kubeCli.AppsV1beta1().StatefulSets(info.Namespace).Get(pdSetName, metav1.GetOptions{}) + if err != nil { + glog.Infof("failed to get pd statefulset: [%s], error: %v", pdSetName, err) + return false, nil + } + + tikvSetName := controller.TiKVMemberName(info.ClusterName) + tikvSet, err := oa.kubeCli.AppsV1beta1().StatefulSets(info.Namespace).Get(tikvSetName, metav1.GetOptions{}) + if err != nil { + glog.Infof("failed to get tikvSet statefulset: [%s], error: %v", tikvSetName, err) + return false, nil + } + + tidbSetName := controller.TiDBMemberName(info.ClusterName) + tidbSet, err := oa.kubeCli.AppsV1beta1().StatefulSets(info.Namespace).Get(tidbSetName, metav1.GetOptions{}) + if err != nil { + glog.Infof("failed to get tidbSet statefulset: [%s], error: %v", tidbSetName, err) + return false, nil + } + + imageUpgraded := func(memberType v1alpha1.MemberType, set *v1beta1.StatefulSet) bool { + image := "" + switch memberType { + case v1alpha1.PDMemberType: + image = tc.Spec.PD.Image + case v1alpha1.TiKVMemberType: + image = tc.Spec.TiKV.Image + case v1alpha1.TiDBMemberType: + image = tc.Spec.TiDB.Image + } + memberName := string(memberType) + c, ok := getComponentContainer(set) + if !ok || c.Image != image { + glog.Infof("check %s image: getContainer(set).Image(%s) != tc.Spec.%s.Image(%s)", + memberName, c.Image, strings.ToUpper(memberName), image) + } + return ok && c.Image == image + } + setUpgraded := func(set *v1beta1.StatefulSet) bool { + return set.Generation <= *set.Status.ObservedGeneration && set.Status.CurrentRevision == set.Status.UpdateRevision + } + + // check upgrade order + if tc.Status.PD.Phase == v1alpha1.UpgradePhase { + glog.Infof("pd is upgrading") + if tc.Status.TiKV.Phase == v1alpha1.UpgradePhase { + return false, errors.New("tikv is upgrading while pd is upgrading") + } + if tc.Status.TiDB.Phase == v1alpha1.UpgradePhase { + return false, errors.New("tidb is upgrading while pd is upgrading") + } + if !imageUpgraded(v1alpha1.PDMemberType, pdSet) { + return false, errors.New("pd image is not updated while pd is upgrading") + } + if !setUpgraded(pdSet) { + if imageUpgraded(v1alpha1.TiKVMemberType, tikvSet) { + return false, errors.New("tikv image is updated while pd is upgrading") + } + if imageUpgraded(v1alpha1.TiDBMemberType, tidbSet) { + return false, errors.New("tidb image is updated while pd is upgrading") + } + } + return false, nil + } else if tc.Status.TiKV.Phase == v1alpha1.UpgradePhase { + glog.Infof("tikv is upgrading") + if tc.Status.TiDB.Phase == v1alpha1.UpgradePhase { + return false, errors.New("tidb is upgrading while tikv is upgrading") + } + if !imageUpgraded(v1alpha1.PDMemberType, pdSet) { + return false, errors.New("pd image is not updated while tikv is upgrading") + } + if !setUpgraded(pdSet) { + return false, errors.New("pd stateful set is not upgraded while tikv is upgrading") + } + if !imageUpgraded(v1alpha1.TiKVMemberType, tikvSet) { + return false, errors.New("tikv image is not updated while tikv is upgrading") + } + if !setUpgraded(tikvSet) { + if imageUpgraded(v1alpha1.TiDBMemberType, tidbSet) { + return false, errors.New("tidb image is updated while tikv is upgrading") + } + } + return false, nil + } else if tc.Status.TiDB.Phase == v1alpha1.UpgradePhase { + glog.Infof("tidb is upgrading") + if !imageUpgraded(v1alpha1.PDMemberType, pdSet) { + return false, errors.New("pd image is not updated while tidb is upgrading") + } + if !setUpgraded(pdSet) { + return false, errors.New("pd stateful set is not upgraded while tidb is upgrading") + } + if !imageUpgraded(v1alpha1.TiKVMemberType, tikvSet) { + return false, errors.New("tikv image is not updated while tidb is upgrading") + } + if !setUpgraded(tikvSet) { + return false, errors.New("tikv stateful set is not upgraded while tidb is upgrading") + } + if !imageUpgraded(v1alpha1.TiDBMemberType, tidbSet) { + return false, errors.New("tidb image is not updated while tikv is upgrading") + } + return false, nil + } + + // check pd final state + if !imageUpgraded(v1alpha1.PDMemberType, pdSet) { + return false, nil + } + if !setUpgraded(pdSet) { + glog.Infof("check pd stateful set upgraded failed") + return false, nil + } + // check tikv final state + if !imageUpgraded(v1alpha1.TiKVMemberType, tikvSet) { + return false, nil + } + if !setUpgraded(tikvSet) { + glog.Infof("check tikv stateful set upgraded failed") + return false, nil + } + // check tidb final state + if !imageUpgraded(v1alpha1.TiDBMemberType, tidbSet) { + return false, nil + } + if !setUpgraded(tidbSet) { + glog.Infof("check tidb stateful set upgraded failed") + return false, nil + } + return true, nil + }) +} + func (oa *operatorActions) DeployMonitor(info *TidbClusterInfo) error { return nil } func (oa *operatorActions) CleanMonitor(info *TidbClusterInfo) error { return nil } @@ -1336,3 +1479,25 @@ func (oa *operatorActions) GetPodUIDMap(info *TidbClusterInfo) (map[string]types return result, nil } + +func (oa *operatorActions) GetNodeMap(info *TidbClusterInfo, component string) (map[string][]string, error) { + nodeMap := make(map[string][]string) + selector := label.New().Instance(info.ClusterName).Component(component).Labels() + podList, err := oa.kubeCli.CoreV1().Pods(info.Namespace).List(metav1.ListOptions{ + LabelSelector: labels.SelectorFromSet(selector).String(), + }) + if err != nil { + return nil, err + } + + for _, pod := range podList.Items { + nodeName := pod.Spec.NodeName + if len(nodeMap[nodeName]) == 0 { + nodeMap[nodeName] = make([]string, 0) + } + nodeMap[nodeName] = append(nodeMap[nodeName], pod.GetName()) + sort.Strings(nodeMap[nodeName]) + } + + return nodeMap, nil +} diff --git a/tests/cmd/e2e/main.go b/tests/cmd/e2e/main.go index 6eef1f862d..59d84c669d 100644 --- a/tests/cmd/e2e/main.go +++ b/tests/cmd/e2e/main.go @@ -15,9 +15,11 @@ package main import ( "flag" + "reflect" "github.com/golang/glog" "github.com/pingcap/tidb-operator/pkg/client/clientset/versioned" + "github.com/pingcap/tidb-operator/pkg/label" "github.com/pingcap/tidb-operator/tests" "github.com/pingcap/tidb-operator/tests/backup" "k8s.io/apiserver/pkg/util/logs" @@ -132,14 +134,24 @@ func main() { perror(oa.CheckTidbClusterStatus(clusterInfo), dumplogs) perror(oa.CheckScaledCorrectly(clusterInfo, podUIDsBeforeScale), dumplogs) + pdNodeMapBeforeUpgrade, err := oa.GetNodeMap(clusterInfo, label.PDLabelVal) + perror(err, dumplogs) + tikvNodeMapBeforeUpgrade, err := oa.GetNodeMap(clusterInfo, label.TiKVLabelVal) + perror(err, dumplogs) clusterInfo = clusterInfo.UpgradeAll("v2.1.4") - if err = oa.UpgradeTidbCluster(clusterInfo); err != nil { - oa.DumpAllLogs(operatorInfo, []*tests.TidbClusterInfo{clusterInfo}) - glog.Fatal(err) + perror(oa.UpgradeTidbCluster(clusterInfo), dumplogs) + perror(oa.CheckUpgradeProgress(clusterInfo), dumplogs) + perror(oa.CheckTidbClusterStatus(clusterInfo), dumplogs) + pdNodeMapAfterUpgrade, err := oa.GetNodeMap(clusterInfo, label.PDLabelVal) + perror(err, dumplogs) + tikvNodeMapAfterUpgrade, err := oa.GetNodeMap(clusterInfo, label.TiKVLabelVal) + perror(err, dumplogs) + + if !reflect.DeepEqual(pdNodeMapAfterUpgrade, pdNodeMapBeforeUpgrade) { + glog.Fatal("pd node map changed: %v != %v", pdNodeMapAfterUpgrade, pdNodeMapBeforeUpgrade) } - if err = oa.CheckTidbClusterStatus(clusterInfo); err != nil { - oa.DumpAllLogs(operatorInfo, []*tests.TidbClusterInfo{clusterInfo}) - glog.Fatal(err) + if !reflect.DeepEqual(tikvNodeMapAfterUpgrade, tikvNodeMapBeforeUpgrade) { + glog.Fatal("tikv node map changed: %v != %v", tikvNodeMapAfterUpgrade, tikvNodeMapBeforeUpgrade) } restoreClusterInfo := &tests.TidbClusterInfo{ From 4c3d275b87744882cd9cf0f08171515ae58648e0 Mon Sep 17 00:00:00 2001 From: zyguan Date: Mon, 18 Mar 2019 22:54:25 +0800 Subject: [PATCH 13/22] stability: amend tidbMembersReadyFn --- tests/actions.go | 10 ++++++++++ 1 file changed, 10 insertions(+) diff --git a/tests/actions.go b/tests/actions.go index 287dfbca30..ef6cb49c80 100644 --- a/tests/actions.go +++ b/tests/actions.go @@ -770,6 +770,11 @@ func (oa *operatorActions) tidbMembersReadyFn(tc *v1alpha1.TidbCluster) (bool, e ns, tidbSetName, tidbSet.Status.ReadyReplicas, replicas) return false, nil } + if len(tc.Status.TiDB.Members) != int(tc.Spec.TiDB.Replicas) { + glog.Infof("tidbcluster: %s/%s .status.TiDB.Members count(%d) != %d", + ns, tcName, len(tc.Status.TiDB.Members), tc.Spec.TiDB.Replicas) + return false, nil + } if tidbSet.Status.ReadyReplicas != tidbSet.Status.Replicas { glog.Infof("statefulset: %s/%s .status.ReadyReplicas(%d) != .status.Replicas(%d)", ns, tidbSetName, tidbSet.Status.ReadyReplicas, tidbSet.Status.Replicas) @@ -786,6 +791,11 @@ func (oa *operatorActions) tidbMembersReadyFn(tc *v1alpha1.TidbCluster) (bool, e glog.Errorf("failed to get service: %s/%s", ns, tidbSetName) return false, nil } + _, err = oa.kubeCli.CoreV1().Services(ns).Get(controller.TiDBPeerMemberName(tidbSetName), metav1.GetOptions{}) + if err != nil { + glog.Errorf("failed to get peer service: %s/%s", ns, controller.TiDBPeerMemberName(tidbSetName)) + return false, nil + } return true, nil } From 9d33e26b90c2c5efc38e86f52385382abb5b9d78 Mon Sep 17 00:00:00 2001 From: CWen Date: Tue, 19 Mar 2019 10:54:39 +0800 Subject: [PATCH 14/22] stability: add blockWriter case for inserting data (#321) * add blockWriter case to insert data --- tests/actions.go | 27 ++- tests/cmd/e2e/main.go | 6 + tests/pkg/blockWriter/blockWriter.go | 273 +++++++++++++++++++++++++++ tests/pkg/util/db.go | 19 ++ tests/pkg/util/string.go | 33 ++++ tests/pkg/util/time.go | 19 ++ 6 files changed, 376 insertions(+), 1 deletion(-) create mode 100644 tests/pkg/blockWriter/blockWriter.go create mode 100644 tests/pkg/util/db.go create mode 100644 tests/pkg/util/string.go create mode 100644 tests/pkg/util/time.go diff --git a/tests/actions.go b/tests/actions.go index 8819883337..5b72ad0741 100644 --- a/tests/actions.go +++ b/tests/actions.go @@ -34,6 +34,8 @@ import ( "github.com/pingcap/tidb-operator/pkg/client/clientset/versioned" "github.com/pingcap/tidb-operator/pkg/controller" "github.com/pingcap/tidb-operator/pkg/label" + "github.com/pingcap/tidb-operator/tests/pkg/blockwriter" + "github.com/pingcap/tidb-operator/tests/pkg/util" "k8s.io/api/apps/v1beta1" batchv1 "k8s.io/api/batch/v1" corev1 "k8s.io/api/core/v1" @@ -43,6 +45,13 @@ import ( "k8s.io/client-go/kubernetes" ) +const ( + defaultTableNum int = 64 + defaultConcurrency = 512 + defaultBatchSize = 100 + defaultRawSize = 100 +) + func NewOperatorActions(cli versioned.Interface, kubeCli kubernetes.Interface, logDir string) OperatorActions { return &operatorActions{ cli: cli, @@ -138,6 +147,7 @@ type TidbClusterInfo struct { InsertBatchSize string Resources map[string]string Args map[string]string + blockWriter *blockwriter.BlockWriterCase Monitor bool } @@ -242,6 +252,14 @@ func (oa *operatorActions) DeployTidbCluster(info *TidbClusterInfo) error { info.Namespace, info.ClusterName, err, string(res)) } + // init blockWriter case + info.blockWriter = blockwriter.NewBlockWriterCase(blockwriter.Config{ + TableNum: defaultTableNum, + Concurrency: defaultConcurrency, + BatchSize: defaultBatchSize, + RawSize: defaultRawSize, + }) + return nil } @@ -369,10 +387,17 @@ func (oa *operatorActions) CheckTidbClusterStatus(info *TidbClusterInfo) error { } func (oa *operatorActions) BeginInsertDataTo(info *TidbClusterInfo) error { - return nil + dsn := getDSN(info.Namespace, info.ClusterName, "test", info.Password) + db, err := util.OpenDB(dsn, defaultConcurrency) + if err != nil { + return err + } + + return info.blockWriter.Start(db) } func (oa *operatorActions) StopInsertDataTo(info *TidbClusterInfo) error { + info.blockWriter.Stop() return nil } diff --git a/tests/cmd/e2e/main.go b/tests/cmd/e2e/main.go index 73b303323c..e4a6b4af4e 100644 --- a/tests/cmd/e2e/main.go +++ b/tests/cmd/e2e/main.go @@ -15,6 +15,8 @@ package main import ( "flag" + "net/http" + _ "net/http/pprof" "github.com/golang/glog" "github.com/pingcap/tidb-operator/pkg/client/clientset/versioned" @@ -36,6 +38,10 @@ func main() { logs.InitLogs() defer logs.FlushLogs() + go func() { + glog.Info(http.ListenAndServe("localhost:6060", nil)) + }() + cfg, err := rest.InClusterConfig() if err != nil { glog.Fatalf("failed to get config: %v", err) diff --git a/tests/pkg/blockWriter/blockWriter.go b/tests/pkg/blockWriter/blockWriter.go new file mode 100644 index 0000000000..8434f151b0 --- /dev/null +++ b/tests/pkg/blockWriter/blockWriter.go @@ -0,0 +1,273 @@ +// Copyright 2019 PingCAP, Inc. +// +// 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, +// See the License for the specific language governing permissions and +// limitations under the License.package spec + +package blockwriter + +import ( + "context" + "database/sql" + "fmt" + "math/rand" + "strings" + "sync" + "sync/atomic" + "time" + + "github.com/golang/glog" + "github.com/pingcap/tidb-operator/tests/pkg/util" + "k8s.io/apimachinery/pkg/util/wait" +) + +const ( + queryChanSize int = 10000 +) + +// BlockWriterCase is for concurrent writing blocks. +type BlockWriterCase struct { + cfg Config + bws []*blockWriter + + isRunning uint32 + isInit uint32 + stopChan chan struct{} + + sync.RWMutex +} + +// Config defines the config of BlockWriterCase +type Config struct { + TableNum int + Concurrency int + BatchSize int + RawSize int +} + +type blockWriter struct { + rawSize int + values []string + batchSize int +} + +// NewBlockWriterCase returns the BlockWriterCase. +func NewBlockWriterCase(cfg Config) *BlockWriterCase { + c := &BlockWriterCase{ + cfg: cfg, + stopChan: make(chan struct{}, 1), + } + + if c.cfg.TableNum < 1 { + c.cfg.TableNum = 1 + } + c.initBlocks() + + return c +} + +func (c *BlockWriterCase) initBlocks() { + c.bws = make([]*blockWriter, c.cfg.Concurrency) + for i := 0; i < c.cfg.Concurrency; i++ { + c.bws[i] = c.newBlockWriter() + } +} + +func (c *BlockWriterCase) newBlockWriter() *blockWriter { + return &blockWriter{ + rawSize: c.cfg.RawSize, + values: make([]string, c.cfg.BatchSize), + batchSize: c.cfg.BatchSize, + } +} + +func (c *BlockWriterCase) generateQuery(ctx context.Context, queryChan chan []string, wg *sync.WaitGroup) { + defer func() { + glog.Infof("[%s] [action: generate Query] stopped", c) + wg.Done() + }() + + for { + tableN := rand.Intn(c.cfg.TableNum) + var index string + if tableN > 0 { + index = fmt.Sprintf("%d", tableN) + } + + var querys []string + for i := 0; i < 100; i++ { + values := make([]string, c.cfg.BatchSize) + for i := 0; i < c.cfg.BatchSize; i++ { + blockData := util.RandString(c.cfg.RawSize) + values[i] = fmt.Sprintf("('%s')", blockData) + } + + querys = append(querys, fmt.Sprintf( + "INSERT INTO block_writer%s(raw_bytes) VALUES %s", + index, strings.Join(values, ","))) + } + + select { + case <-ctx.Done(): + return + default: + if len(queryChan) < queryChanSize { + queryChan <- querys + } else { + glog.Infof("[%s] [action: generate Query] query channel is full, sleep 10 seconds", c) + util.Sleep(ctx, 10*time.Second) + } + } + } +} + +func (bw *blockWriter) batchExecute(db *sql.DB, query string) error { + _, err := db.Exec(query) + if err != nil { + glog.Errorf("[block_writer] exec sql [%s] failed, err: %v", query, err) + return err + } + + return nil +} + +func (bw *blockWriter) run(ctx context.Context, db *sql.DB, queryChan chan []string) { + for { + select { + case <-ctx.Done(): + return + default: + } + + querys, ok := <-queryChan + if !ok { + // No more query + return + } + + for _, query := range querys { + select { + case <-ctx.Done(): + return + default: + if err := bw.batchExecute(db, query); err != nil { + glog.Fatal(err) + } + } + } + } +} + +// Initialize inits case +func (c *BlockWriterCase) initialize(db *sql.DB) error { + glog.Infof("[%s] start to init...", c) + defer func() { + atomic.StoreUint32(&c.isInit, 1) + glog.Infof("[%s] init end...", c) + }() + + for i := 0; i < c.cfg.TableNum; i++ { + var s string + if i > 0 { + s = fmt.Sprintf("%d", i) + } + + tmt := fmt.Sprintf("CREATE TABLE IF NOT EXISTS block_writer%s %s", s, ` + ( + id BIGINT NOT NULL AUTO_INCREMENT, + raw_bytes BLOB NOT NULL, + PRIMARY KEY (id) +)`) + + err := wait.PollImmediate(5*time.Second, 30*time.Second, func() (bool, error) { + _, err := db.Exec(tmt) + if err != nil { + glog.Warningf("[%s] exec sql [%s] failed, err: %v, retry...", c, tmt, err) + return false, nil + } + + return true, nil + }) + + if err != nil { + glog.Errorf("[%s] exec sql [%s] failed, err: %v", c, tmt, err) + return err + } + } + + return nil +} + +// Start starts to run cases +func (c *BlockWriterCase) Start(db *sql.DB) error { + if !atomic.CompareAndSwapUint32(&c.isRunning, 0, 1) { + err := fmt.Errorf("[%s] is running, you can't start it again", c) + glog.Error(err) + return err + } + + defer func() { + c.RLock() + glog.Infof("[%s] stopped", c) + atomic.SwapUint32(&c.isRunning, 0) + }() + + if c.isInit == 0 { + if err := c.initialize(db); err != nil { + return err + } + } + + glog.Infof("[%s] start to execute case...", c) + + var wg sync.WaitGroup + + ctx, cancel := context.WithCancel(context.Background()) + + queryChan := make(chan []string, queryChanSize) + + for i := 0; i < c.cfg.Concurrency; i++ { + wg.Add(1) + go func(i int) { + defer wg.Done() + c.bws[i].run(ctx, db, queryChan) + }(i) + } + + wg.Add(1) + go c.generateQuery(ctx, queryChan, &wg) + +loop: + for { + select { + case <-c.stopChan: + glog.Infof("[%s] stoping...", c) + cancel() + break loop + default: + util.Sleep(context.Background(), 2*time.Second) + } + } + + wg.Wait() + close(queryChan) + + return nil +} + +// Stop stops cases +func (c *BlockWriterCase) Stop() { + c.stopChan <- struct{}{} +} + +// String implements fmt.Stringer interface. +func (c *BlockWriterCase) String() string { + return "block_writer" +} diff --git a/tests/pkg/util/db.go b/tests/pkg/util/db.go new file mode 100644 index 0000000000..c931f76245 --- /dev/null +++ b/tests/pkg/util/db.go @@ -0,0 +1,19 @@ +package util + +import ( + "database/sql" + + "github.com/golang/glog" +) + +// OpenDB opens db +func OpenDB(dsn string, maxIdleConns int) (*sql.DB, error) { + db, err := sql.Open("mysql", dsn) + if err != nil { + return nil, err + } + + db.SetMaxIdleConns(maxIdleConns) + glog.Info("DB opens successfully") + return db, nil +} diff --git a/tests/pkg/util/string.go b/tests/pkg/util/string.go new file mode 100644 index 0000000000..8334dbe2ba --- /dev/null +++ b/tests/pkg/util/string.go @@ -0,0 +1,33 @@ +package util + +import ( + "math/rand" +) + +const ( + alphabet = "0123456789ABCDEFGHIJKLMNOPQRSTUVWXYZabcdefghijklmnopqrstuvwxyz" + + // Used by RandString + letterIdxBits = 6 // 6 bits to represent a letter index + letterIdxMask = 1<= 0; { + if remain == 0 { + cache, remain = rand.Int63(), letterIdxMax + } + if idx := int(cache & letterIdxMask); idx < len(alphabet) { + b[i] = alphabet[idx] + i-- + } + cache >>= letterIdxBits + remain-- + } + + return string(b) +} diff --git a/tests/pkg/util/time.go b/tests/pkg/util/time.go new file mode 100644 index 0000000000..7e3a4909e7 --- /dev/null +++ b/tests/pkg/util/time.go @@ -0,0 +1,19 @@ +package util + +import ( + "context" + "time" +) + +// Sleep defines special `sleep` with context +func Sleep(ctx context.Context, sleepTime time.Duration) { + ticker := time.NewTicker(sleepTime) + defer ticker.Stop() + + select { + case <-ctx.Done(): + return + case <-ticker.C: + return + } +} From 61652fc3491e0e283834f508ba4041d6c4035ee4 Mon Sep 17 00:00:00 2001 From: shuijing198799 <30903849+shuijing198799@users.noreply.github.com> Date: Tue, 19 Mar 2019 14:46:58 +0800 Subject: [PATCH 15/22] add scheduled-backup test case (#322) * add scheduled-backup test case --- tests/actions.go | 227 ++++++++++++++++-- tests/backup/backupcase.go | 20 +- tests/cmd/e2e/main.go | 19 +- .../blockwriter.go} | 0 4 files changed, 240 insertions(+), 26 deletions(-) rename tests/pkg/{blockWriter/blockWriter.go => blockwriter/blockwriter.go} (100%) diff --git a/tests/actions.go b/tests/actions.go index 5b72ad0741..ee87c0cff6 100644 --- a/tests/actions.go +++ b/tests/actions.go @@ -28,7 +28,7 @@ import ( _ "github.com/go-sql-driver/mysql" "github.com/golang/glog" - "github.com/pingcap/errors" + pingcapErrors "github.com/pingcap/errors" "github.com/pingcap/kvproto/pkg/metapb" "github.com/pingcap/tidb-operator/pkg/apis/pingcap.com/v1alpha1" "github.com/pingcap/tidb-operator/pkg/client/clientset/versioned" @@ -39,8 +39,10 @@ import ( "k8s.io/api/apps/v1beta1" batchv1 "k8s.io/api/batch/v1" corev1 "k8s.io/api/core/v1" + "k8s.io/apimachinery/pkg/api/errors" metav1 "k8s.io/apimachinery/pkg/apis/meta/v1" "k8s.io/apimachinery/pkg/labels" + "k8s.io/apimachinery/pkg/types" "k8s.io/apimachinery/pkg/util/wait" "k8s.io/client-go/kubernetes" ) @@ -64,9 +66,7 @@ func NewOperatorActions(cli versioned.Interface, kubeCli kubernetes.Interface, l const ( DefaultPollTimeout time.Duration = 10 * time.Minute DefaultPollInterval time.Duration = 10 * time.Second -) - -const ( + getBackupDirPodName = "get-backup-dir" grafanaUsername = "admin" grafanaPassword = "admin" ) @@ -93,6 +93,7 @@ type OperatorActions interface { CheckRestore(from *TidbClusterInfo, to *TidbClusterInfo) error ForceDeploy(info *TidbClusterInfo) error CreateSecret(info *TidbClusterInfo) error + getBackupDir(info *TidbClusterInfo) ([]string, error) } type FaultTriggerActions interface { @@ -135,6 +136,7 @@ type OperatorInfo struct { } type TidbClusterInfo struct { + BackupPVC string Namespace string ClusterName string OperatorTag string @@ -143,6 +145,7 @@ type TidbClusterInfo struct { TiDBImage string StorageClassName string Password string + InitSql string RecordCount string InsertBatchSize string Resources map[string]string @@ -153,9 +156,6 @@ type TidbClusterInfo struct { func (tc *TidbClusterInfo) HelmSetString() string { - // add a database and table for test - initSql := `"create database record;use record;create table test(t char(32));"` - set := map[string]string{ "clusterName": tc.ClusterName, "pd.storageClassName": tc.StorageClassName, @@ -167,7 +167,7 @@ func (tc *TidbClusterInfo) HelmSetString() string { "tikv.image": tc.TiKVImage, "tidb.image": tc.TiDBImage, "tidb.passwordSecretName": "set-secret", - "tidb.initSql": initSql, + "tidb.initSql": tc.InitSql, "monitor.create": strconv.FormatBool(tc.Monitor), } @@ -272,6 +272,7 @@ func (oa *operatorActions) CleanTidbCluster(info *TidbClusterInfo) error { info.ClusterName, fmt.Sprintf("%s-backup", info.ClusterName), fmt.Sprintf("%s-restore", info.ClusterName), + fmt.Sprintf("%s-scheduler-backup", info.ClusterName), } for _, chartName := range charts { res, err := exec.Command("helm", "del", "--purge", chartName).CombinedOutput() @@ -281,6 +282,12 @@ func (oa *operatorActions) CleanTidbCluster(info *TidbClusterInfo) error { } } + err := oa.kubeCli.CoreV1().Pods(info.Namespace).Delete(getBackupDirPodName, &metav1.DeleteOptions{}) + + if err != nil && !errors.IsNotFound(err) { + return fmt.Errorf("failed to delete dir pod %v", err) + } + setStr := label.New().Instance(info.ClusterName).String() resources := []string{"pvc"} @@ -411,7 +418,7 @@ func (oa *operatorActions) ScaleTidbCluster(info *TidbClusterInfo) error { glog.Info("[SCALE] " + cmd) res, err := exec.Command("/bin/sh", "-c", cmd).CombinedOutput() if err != nil { - return errors.Wrapf(err, "failed to scale tidb cluster: %s", string(res)) + return pingcapErrors.Wrapf(err, "failed to scale tidb cluster: %s", string(res)) } return nil } @@ -422,7 +429,7 @@ func (oa *operatorActions) UpgradeTidbCluster(info *TidbClusterInfo) error { glog.Info("[UPGRADE] " + cmd) res, err := exec.Command("/bin/sh", "-c", cmd).CombinedOutput() if err != nil { - return errors.Wrapf(err, "failed to upgrade tidb cluster: %s", string(res)) + return pingcapErrors.Wrapf(err, "failed to upgrade tidb cluster: %s", string(res)) } return nil } @@ -1052,7 +1059,7 @@ func (oa *operatorActions) DeployAdHocBackup(info *TidbClusterInfo) error { }() sets := map[string]string{ "clusterName": info.ClusterName, - "name": "test-backup", + "name": info.BackupPVC, "mode": "backup", "user": "root", "password": info.Password, @@ -1076,6 +1083,7 @@ func (oa *operatorActions) DeployAdHocBackup(info *TidbClusterInfo) error { if err != nil { return fmt.Errorf("failed to launch adhoc backup job: %v, %s", err, string(res)) } + return nil } @@ -1085,7 +1093,7 @@ func (oa *operatorActions) CheckAdHocBackup(info *TidbClusterInfo) error { glog.Infof("deploy clean backup end cluster[%s] namespace[%s]", info.ClusterName, info.Namespace) }() - jobName := fmt.Sprintf("%s-%s", info.ClusterName, "test-backup") + jobName := fmt.Sprintf("%s-%s", info.ClusterName, info.BackupPVC) fn := func() (bool, error) { job, err := oa.kubeCli.BatchV1().Jobs(info.Namespace).Get(jobName, metav1.GetOptions{}) if err != nil { @@ -1104,6 +1112,7 @@ func (oa *operatorActions) CheckAdHocBackup(info *TidbClusterInfo) error { if err != nil { return fmt.Errorf("failed to launch scheduler backup job: %v", err) } + return nil } @@ -1114,7 +1123,7 @@ func (oa *operatorActions) Restore(from *TidbClusterInfo, to *TidbClusterInfo) e }() sets := map[string]string{ "clusterName": to.ClusterName, - "name": "test-backup", + "name": to.BackupPVC, "mode": "restore", "user": "root", "password": to.Password, @@ -1148,7 +1157,7 @@ func (oa *operatorActions) CheckRestore(from *TidbClusterInfo, to *TidbClusterIn glog.Infof("check restore end cluster[%s] namespace[%s]", to.ClusterName, to.Namespace) }() - jobName := fmt.Sprintf("%s-restore-test-backup", to.ClusterName) + jobName := fmt.Sprintf("%s-restore-%s", to.ClusterName, from.BackupPVC) fn := func() (bool, error) { job, err := oa.kubeCli.BatchV1().Jobs(to.Namespace).Get(jobName, metav1.GetOptions{}) if err != nil { @@ -1268,13 +1277,203 @@ func releaseIsExist(err error) bool { } func (oa *operatorActions) DeployScheduledBackup(info *TidbClusterInfo) error { + glog.Infof("begin to deploy scheduled backup") + defer func() { + glog.Infof("deploy shceduled backup end") + }() + + cron := fmt.Sprintf("'*/1 * * * *'") + sets := map[string]string{ + "clusterName": info.ClusterName, + "scheduledBackup.create": "true", + "scheduledBackup.user": "root", + "scheduledBackup.password": info.Password, + "scheduledBackup.schedule": cron, + "scheduledBackup.storage": "10Gi", + } + var buffer bytes.Buffer + for k, v := range sets { + set := fmt.Sprintf(" --set %s=%s", k, v) + _, err := buffer.WriteString(set) + if err != nil { + return err + } + } + + setStr := buffer.String() + + cmd := fmt.Sprintf("helm upgrade %s /charts/%s/tidb-cluster %s", + info.ClusterName, info.OperatorTag, setStr) + + glog.Infof("scheduled-backup delploy [%s]", cmd) + res, err := exec.Command("/bin/sh", "-c", cmd).CombinedOutput() + if err != nil { + return fmt.Errorf("failed to launch scheduler backup job: %v, %s", err, string(res)) + } return nil } func (oa *operatorActions) CheckScheduledBackup(info *TidbClusterInfo) error { + glog.Infof("begin to check scheduler backup cluster[%s] namespace[%s]", info.ClusterName, info.Namespace) + defer func() { + glog.Infof("deploy check scheduler end cluster[%s] namespace[%s]", info.ClusterName, info.Namespace) + }() + + jobName := fmt.Sprintf("%s-scheduled-backup", info.ClusterName) + fn := func() (bool, error) { + job, err := oa.kubeCli.BatchV1beta1().CronJobs(info.Namespace).Get(jobName, metav1.GetOptions{}) + if err != nil { + glog.Errorf("failed to get cronjobs %s ,%v", jobName, err) + return false, nil + } + + jobs, err := oa.kubeCli.BatchV1().Jobs(info.Namespace).List(metav1.ListOptions{}) + if err != nil { + glog.Errorf("failed to list jobs %s ,%v", info.Namespace, err) + return false, nil + } + + backupJobs := []batchv1.Job{} + for _, j := range jobs.Items { + if pid, found := getParentUIDFromJob(j); found && pid == job.UID { + backupJobs = append(backupJobs, j) + } + } + + if len(backupJobs) == 0 { + glog.Errorf("cluster [%s] scheduler jobs is creating, please wait!", info.ClusterName) + return false, nil + } + + for _, j := range backupJobs { + if j.Status.Succeeded == 0 { + glog.Errorf("cluster [%s] back up job is not completed, please wait! ", info.ClusterName) + return false, nil + } + } + + return true, nil + } + + err := wait.Poll(DefaultPollInterval, DefaultPollTimeout, fn) + if err != nil { + return fmt.Errorf("failed to launch scheduler backup job: %v", err) + } + + // sleep 1 minute for cronjob + time.Sleep(60 * time.Second) + + dirs, err := oa.getBackupDir(info) + if err != nil { + return fmt.Errorf("failed to get backup dir: %v", err) + } + + if len(dirs) != 3 { + return fmt.Errorf("scheduler job failed!") + } + return nil } +func getParentUIDFromJob(j batchv1.Job) (types.UID, bool) { + controllerRef := metav1.GetControllerOf(&j) + + if controllerRef == nil { + return types.UID(""), false + } + + if controllerRef.Kind != "CronJob" { + glog.Infof("Job with non-CronJob parent, name %s namespace %s", j.Name, j.Namespace) + return types.UID(""), false + } + + return controllerRef.UID, true +} + +func (oa *operatorActions) getBackupDir(info *TidbClusterInfo) ([]string, error) { + pod := &corev1.Pod{ + ObjectMeta: metav1.ObjectMeta{ + Name: getBackupDirPodName, + Namespace: info.Namespace, + }, + Spec: corev1.PodSpec{ + Containers: []corev1.Container{ + { + Name: getBackupDirPodName, + Image: "pingcap/tidb-cloud-backup:latest", + Command: []string{"sleep", "3000"}, + VolumeMounts: []corev1.VolumeMount{ + { + Name: "data", + MountPath: "/data", + }, + }, + }, + }, + Volumes: []corev1.Volume{ + { + Name: "data", + VolumeSource: corev1.VolumeSource{ + PersistentVolumeClaim: &corev1.PersistentVolumeClaimVolumeSource{ + ClaimName: info.BackupPVC, + }, + }, + }, + }, + }, + } + + fn := func() (bool, error) { + _, err := oa.kubeCli.CoreV1().Pods(info.Namespace).Get(getBackupDirPodName, metav1.GetOptions{}) + if !errors.IsNotFound(err) { + return false, nil + } + return true, nil + } + + err := wait.Poll(DefaultPollInterval, DefaultPollTimeout, fn) + + if err != nil { + return nil, fmt.Errorf("failed to delete pod %s", getBackupDirPodName) + } + + _, err = oa.kubeCli.CoreV1().Pods(info.Namespace).Create(pod) + if err != nil && !errors.IsAlreadyExists(err) { + glog.Errorf("cluster: [%s/%s] create get backup dir pod failed, error :%v", info.Namespace, info.ClusterName, err) + return nil, err + } + + fn = func() (bool, error) { + _, err := oa.kubeCli.CoreV1().Pods(info.Namespace).Get(getBackupDirPodName, metav1.GetOptions{}) + if errors.IsNotFound(err) { + return false, nil + } + return true, nil + } + + err = wait.Poll(DefaultPollInterval, DefaultPollTimeout, fn) + + if err != nil { + return nil, fmt.Errorf("failed to create pod %s", getBackupDirPodName) + } + + cmd := fmt.Sprintf("kubectl exec %s -n %s ls /data", getBackupDirPodName, info.Namespace) + glog.Infof(cmd) + res, err := exec.Command("/bin/sh", "-c", cmd).CombinedOutput() + if err != nil { + glog.Errorf("cluster:[%s/%s] exec :%s failed,error:%v,result:%s", info.Namespace, info.ClusterName, cmd, err, res) + return nil, err + } + + dirs := strings.Split(string(res), "\n") + glog.Infof("dirs in pod info name [%s] dir name [%s]", info.BackupPVC, strings.Join(dirs, ",")) + return dirs, nil +} + +func (info *TidbClusterInfo) FullName() string { + return fmt.Sprintf("%s/%s", info.Namespace, info.ClusterName) +} + func (oa *operatorActions) DeployIncrementalBackup(from *TidbClusterInfo, to *TidbClusterInfo) error { return nil } diff --git a/tests/backup/backupcase.go b/tests/backup/backupcase.go index 07c5da0fec..dea051e9a4 100644 --- a/tests/backup/backupcase.go +++ b/tests/backup/backupcase.go @@ -46,12 +46,6 @@ func (bc *BackupCase) Run() error { return err } - err = bc.operator.ForceDeploy(bc.desCluster) - if err != nil { - glog.Errorf("cluster:[%s] deploy happen error: %v", bc.desCluster.ClusterName, err) - return err - } - err = bc.operator.CheckTidbClusterStatus(bc.desCluster) if err != nil { glog.Errorf("cluster:[%s] deploy faild error: %v", bc.desCluster.ClusterName, err) @@ -70,5 +64,19 @@ func (bc *BackupCase) Run() error { return err } + bc.srcCluster.BackupPVC = "demo-scheduled-backup" + + err = bc.operator.DeployScheduledBackup(bc.srcCluster) + if err != nil { + glog.Errorf("cluster:[%s] scheduler happen error: %v", bc.srcCluster.ClusterName, err) + return err + } + + err = bc.operator.CheckScheduledBackup(bc.srcCluster) + if err != nil { + glog.Errorf("cluster:[%s] scheduler failed error: %v", bc.srcCluster.ClusterName, err) + return err + } + return nil } diff --git a/tests/cmd/e2e/main.go b/tests/cmd/e2e/main.go index e4a6b4af4e..0f88ce47d0 100644 --- a/tests/cmd/e2e/main.go +++ b/tests/cmd/e2e/main.go @@ -74,15 +74,20 @@ func main() { glog.Fatal(err) } + // create database and table and insert a column for test backup and restore + initSql := `"create database record;use record;create table test(t char(32))"` + clusterInfo := &tests.TidbClusterInfo{ + BackupPVC: "test-backup", Namespace: "tidb", ClusterName: "demo", OperatorTag: "master", - PDImage: "pingcap/pd:v2.1.3", - TiKVImage: "pingcap/tikv:v2.1.3", - TiDBImage: "pingcap/tidb:v2.1.3", + PDImage: "pingcap/pd:v2.1.0", + TiKVImage: "pingcap/tikv:v2.1.0", + TiDBImage: "pingcap/tidb:v2.1.0", StorageClassName: "local-storage", Password: "admin", + InitSql: initSql, Resources: map[string]string{ "pd.resources.limits.cpu": "1000m", "pd.resources.limits.memory": "2Gi", @@ -134,14 +139,16 @@ func main() { } restoreClusterInfo := &tests.TidbClusterInfo{ + BackupPVC: "test-backup", Namespace: "tidb", ClusterName: "demo2", OperatorTag: "master", - PDImage: "pingcap/pd:v2.1.3", - TiKVImage: "pingcap/tikv:v2.1.3", - TiDBImage: "pingcap/tidb:v2.1.3", + PDImage: "pingcap/pd:v2.1.0", + TiKVImage: "pingcap/tikv:v2.1.0", + TiDBImage: "pingcap/tidb:v2.1.0", StorageClassName: "local-storage", Password: "admin", + InitSql: initSql, Resources: map[string]string{ "pd.resources.limits.cpu": "1000m", "pd.resources.limits.memory": "2Gi", diff --git a/tests/pkg/blockWriter/blockWriter.go b/tests/pkg/blockwriter/blockwriter.go similarity index 100% rename from tests/pkg/blockWriter/blockWriter.go rename to tests/pkg/blockwriter/blockwriter.go From 7fb60831c75d4bb7fe846323ea251052d52df438 Mon Sep 17 00:00:00 2001 From: zyguan Date: Tue, 19 Mar 2019 18:31:50 +0800 Subject: [PATCH 16/22] stability: address the comments --- tests/actions.go | 8 ++++---- tests/cmd/e2e/main.go | 10 +++++----- 2 files changed, 9 insertions(+), 9 deletions(-) diff --git a/tests/actions.go b/tests/actions.go index 5c3c03d8ad..3c4cbc4eae 100644 --- a/tests/actions.go +++ b/tests/actions.go @@ -430,7 +430,7 @@ func (oa *operatorActions) ScaleTidbCluster(info *TidbClusterInfo) error { } func (oa *operatorActions) CheckScaleInSafely(info *TidbClusterInfo) error { - return wait.Poll(5*time.Second, DefaultPollTimeout, func() (done bool, err error) { + return wait.Poll(DefaultPollInterval, DefaultPollTimeout, func() (done bool, err error) { tc, err := oa.cli.PingcapV1alpha1().TidbClusters(info.Namespace).Get(info.ClusterName, metav1.GetOptions{}) if err != nil { glog.Infof("failed to get tidbcluster when scale in tidbcluster, error: %v", err) @@ -498,7 +498,7 @@ func (oa *operatorActions) UpgradeTidbCluster(info *TidbClusterInfo) error { } func (oa *operatorActions) CheckUpgradeProgress(info *TidbClusterInfo) error { - return wait.Poll(5*time.Second, DefaultPollTimeout, func() (done bool, err error) { + return wait.Poll(DefaultPollInterval, DefaultPollTimeout, func() (done bool, err error) { tc, err := oa.cli.PingcapV1alpha1().TidbClusters(info.Namespace).Get(info.ClusterName, metav1.GetOptions{}) if err != nil { glog.Infof("failed to get tidbcluster: [%s], error: %v", info.ClusterName, err) @@ -822,9 +822,9 @@ func (oa *operatorActions) tidbMembersReadyFn(tc *v1alpha1.TidbCluster) (bool, e glog.Errorf("failed to get service: %s/%s", ns, tidbSetName) return false, nil } - _, err = oa.kubeCli.CoreV1().Services(ns).Get(controller.TiDBPeerMemberName(tidbSetName), metav1.GetOptions{}) + _, err = oa.kubeCli.CoreV1().Services(ns).Get(controller.TiDBPeerMemberName(tcName), metav1.GetOptions{}) if err != nil { - glog.Errorf("failed to get peer service: %s/%s", ns, controller.TiDBPeerMemberName(tidbSetName)) + glog.Errorf("failed to get peer service: %s/%s", ns, controller.TiDBPeerMemberName(tcName)) return false, nil } diff --git a/tests/cmd/e2e/main.go b/tests/cmd/e2e/main.go index e483ff44aa..377168f9b3 100644 --- a/tests/cmd/e2e/main.go +++ b/tests/cmd/e2e/main.go @@ -15,9 +15,9 @@ package main import ( "flag" - "reflect" "net/http" _ "net/http/pprof" + "reflect" "github.com/golang/glog" "github.com/pingcap/tidb-operator/pkg/client/clientset/versioned" @@ -128,18 +128,18 @@ func main() { dumplogs := func() error { return oa.DumpAllLogs(operatorInfo, []*tests.TidbClusterInfo{clusterInfo}) } - // scale out: tidb 2 -> 3, tikv 3 -> 5 + // scale out: tidb 2 -> 3, tikv 3 -> 5, pd 3 -> 5 podUIDsBeforeScale, err := oa.GetPodUIDMap(clusterInfo) perror(err, dumplogs) - clusterInfo = clusterInfo.ScaleTiDB(3).ScaleTiKV(5) + clusterInfo = clusterInfo.ScaleTiDB(3).ScaleTiKV(5).ScalePD(5) perror(oa.ScaleTidbCluster(clusterInfo), dumplogs) perror(oa.CheckTidbClusterStatus(clusterInfo), dumplogs) perror(oa.CheckScaledCorrectly(clusterInfo, podUIDsBeforeScale), dumplogs) - // scale in: tikv 5 -> 3 + // scale in: tidb 3 -> 1, tikv 5 -> 3, pd 5 -> 3 podUIDsBeforeScale, err = oa.GetPodUIDMap(clusterInfo) perror(err, dumplogs) - clusterInfo = clusterInfo.ScaleTiKV(3) + clusterInfo = clusterInfo.ScaleTiDB(1).ScaleTiKV(3).ScalePD(3) perror(oa.ScaleTidbCluster(clusterInfo), dumplogs) perror(oa.CheckScaleInSafely(clusterInfo), dumplogs) perror(oa.CheckTidbClusterStatus(clusterInfo), dumplogs) From 432fdabbb4696e7d9cdd46994119e03bffbb9f4a Mon Sep 17 00:00:00 2001 From: zyguan Date: Wed, 20 Mar 2019 10:25:18 +0800 Subject: [PATCH 17/22] stability: port ddl test as a workload (#328) * stability: port ddl test as a workload * stability: rename pkg/blockWriter to pkg/blockwriter --- tests/cluster_info.go | 9 +- tests/cmd/e2e/main.go | 45 +- tests/pkg/blockwriter/blockWriter.go | 273 +++++++ tests/pkg/workload/ddl/internal/datatype.go | 338 ++++++++ tests/pkg/workload/ddl/internal/ddl.go | 584 ++++++++++++++ tests/pkg/workload/ddl/internal/ddl_ops.go | 847 ++++++++++++++++++++ tests/pkg/workload/ddl/internal/demo.go | 86 ++ tests/pkg/workload/ddl/internal/dml_ops.go | 620 ++++++++++++++ tests/pkg/workload/ddl/internal/meta.go | 636 +++++++++++++++ tests/pkg/workload/ddl/internal/metrics.go | 29 + tests/pkg/workload/ddl/internal/run.go | 119 +++ tests/pkg/workload/ddl/internal/util.go | 177 ++++ tests/pkg/workload/ddl/workload.go | 50 ++ tests/pkg/workload/interface.go | 31 + 14 files changed, 3837 insertions(+), 7 deletions(-) create mode 100644 tests/pkg/blockwriter/blockWriter.go create mode 100644 tests/pkg/workload/ddl/internal/datatype.go create mode 100644 tests/pkg/workload/ddl/internal/ddl.go create mode 100644 tests/pkg/workload/ddl/internal/ddl_ops.go create mode 100644 tests/pkg/workload/ddl/internal/demo.go create mode 100644 tests/pkg/workload/ddl/internal/dml_ops.go create mode 100644 tests/pkg/workload/ddl/internal/meta.go create mode 100644 tests/pkg/workload/ddl/internal/metrics.go create mode 100644 tests/pkg/workload/ddl/internal/run.go create mode 100644 tests/pkg/workload/ddl/internal/util.go create mode 100644 tests/pkg/workload/ddl/workload.go create mode 100644 tests/pkg/workload/interface.go diff --git a/tests/cluster_info.go b/tests/cluster_info.go index 332be46759..5b5137c2dd 100644 --- a/tests/cluster_info.go +++ b/tests/cluster_info.go @@ -1,6 +1,9 @@ package tests -import "strconv" +import ( + "fmt" + "strconv" +) func (tc *TidbClusterInfo) set(name string, value string) (string, bool) { // NOTE: not thread-safe, maybe make info struct immutable @@ -48,3 +51,7 @@ func (tc *TidbClusterInfo) UpgradeAll(tag string) *TidbClusterInfo { UpgradeTiKV("pingcap/tikv:" + tag). UpgradeTiDB("pingcap/tidb:" + tag) } + +func (tc *TidbClusterInfo) DSN(dbName string) string { + return fmt.Sprintf("root:%s@tcp(%s-tidb.%s:4000)/%s", tc.Password, tc.ClusterName, tc.Namespace, dbName) +} diff --git a/tests/cmd/e2e/main.go b/tests/cmd/e2e/main.go index 0f88ce47d0..61275986e2 100644 --- a/tests/cmd/e2e/main.go +++ b/tests/cmd/e2e/main.go @@ -22,6 +22,8 @@ import ( "github.com/pingcap/tidb-operator/pkg/client/clientset/versioned" "github.com/pingcap/tidb-operator/tests" "github.com/pingcap/tidb-operator/tests/backup" + "github.com/pingcap/tidb-operator/tests/pkg/workload" + "github.com/pingcap/tidb-operator/tests/pkg/workload/ddl" "k8s.io/apiserver/pkg/util/logs" "k8s.io/client-go/kubernetes" "k8s.io/client-go/rest" @@ -118,12 +120,43 @@ func main() { glog.Fatal(err) } - clusterInfo = clusterInfo.ScaleTiDB(3) - if err := oa.ScaleTidbCluster(clusterInfo); err != nil { - oa.DumpAllLogs(operatorInfo, []*tests.TidbClusterInfo{clusterInfo}) - glog.Fatal(err) - } - if err = oa.CheckTidbClusterStatus(clusterInfo); err != nil { + err = workload.Run(func() error { + clusterInfo = clusterInfo.ScaleTiDB(3).ScaleTiKV(5).ScalePD(5) + if err := oa.ScaleTidbCluster(clusterInfo); err != nil { + return err + } + if err := oa.CheckTidbClusterStatus(clusterInfo); err != nil { + return err + } + + clusterInfo = clusterInfo.ScalePD(3) + if err := oa.ScaleTidbCluster(clusterInfo); err != nil { + return err + } + if err := oa.CheckTidbClusterStatus(clusterInfo); err != nil { + return err + } + + clusterInfo = clusterInfo.ScaleTiKV(3) + if err := oa.ScaleTidbCluster(clusterInfo); err != nil { + return err + } + if err := oa.CheckTidbClusterStatus(clusterInfo); err != nil { + return err + } + + clusterInfo = clusterInfo.ScaleTiDB(1) + if err := oa.ScaleTidbCluster(clusterInfo); err != nil { + return err + } + if err := oa.CheckTidbClusterStatus(clusterInfo); err != nil { + return err + } + + return nil + }, ddl.New(clusterInfo.DSN("test"), 1, 1)) + + if err != nil { oa.DumpAllLogs(operatorInfo, []*tests.TidbClusterInfo{clusterInfo}) glog.Fatal(err) } diff --git a/tests/pkg/blockwriter/blockWriter.go b/tests/pkg/blockwriter/blockWriter.go new file mode 100644 index 0000000000..8434f151b0 --- /dev/null +++ b/tests/pkg/blockwriter/blockWriter.go @@ -0,0 +1,273 @@ +// Copyright 2019 PingCAP, Inc. +// +// 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, +// See the License for the specific language governing permissions and +// limitations under the License.package spec + +package blockwriter + +import ( + "context" + "database/sql" + "fmt" + "math/rand" + "strings" + "sync" + "sync/atomic" + "time" + + "github.com/golang/glog" + "github.com/pingcap/tidb-operator/tests/pkg/util" + "k8s.io/apimachinery/pkg/util/wait" +) + +const ( + queryChanSize int = 10000 +) + +// BlockWriterCase is for concurrent writing blocks. +type BlockWriterCase struct { + cfg Config + bws []*blockWriter + + isRunning uint32 + isInit uint32 + stopChan chan struct{} + + sync.RWMutex +} + +// Config defines the config of BlockWriterCase +type Config struct { + TableNum int + Concurrency int + BatchSize int + RawSize int +} + +type blockWriter struct { + rawSize int + values []string + batchSize int +} + +// NewBlockWriterCase returns the BlockWriterCase. +func NewBlockWriterCase(cfg Config) *BlockWriterCase { + c := &BlockWriterCase{ + cfg: cfg, + stopChan: make(chan struct{}, 1), + } + + if c.cfg.TableNum < 1 { + c.cfg.TableNum = 1 + } + c.initBlocks() + + return c +} + +func (c *BlockWriterCase) initBlocks() { + c.bws = make([]*blockWriter, c.cfg.Concurrency) + for i := 0; i < c.cfg.Concurrency; i++ { + c.bws[i] = c.newBlockWriter() + } +} + +func (c *BlockWriterCase) newBlockWriter() *blockWriter { + return &blockWriter{ + rawSize: c.cfg.RawSize, + values: make([]string, c.cfg.BatchSize), + batchSize: c.cfg.BatchSize, + } +} + +func (c *BlockWriterCase) generateQuery(ctx context.Context, queryChan chan []string, wg *sync.WaitGroup) { + defer func() { + glog.Infof("[%s] [action: generate Query] stopped", c) + wg.Done() + }() + + for { + tableN := rand.Intn(c.cfg.TableNum) + var index string + if tableN > 0 { + index = fmt.Sprintf("%d", tableN) + } + + var querys []string + for i := 0; i < 100; i++ { + values := make([]string, c.cfg.BatchSize) + for i := 0; i < c.cfg.BatchSize; i++ { + blockData := util.RandString(c.cfg.RawSize) + values[i] = fmt.Sprintf("('%s')", blockData) + } + + querys = append(querys, fmt.Sprintf( + "INSERT INTO block_writer%s(raw_bytes) VALUES %s", + index, strings.Join(values, ","))) + } + + select { + case <-ctx.Done(): + return + default: + if len(queryChan) < queryChanSize { + queryChan <- querys + } else { + glog.Infof("[%s] [action: generate Query] query channel is full, sleep 10 seconds", c) + util.Sleep(ctx, 10*time.Second) + } + } + } +} + +func (bw *blockWriter) batchExecute(db *sql.DB, query string) error { + _, err := db.Exec(query) + if err != nil { + glog.Errorf("[block_writer] exec sql [%s] failed, err: %v", query, err) + return err + } + + return nil +} + +func (bw *blockWriter) run(ctx context.Context, db *sql.DB, queryChan chan []string) { + for { + select { + case <-ctx.Done(): + return + default: + } + + querys, ok := <-queryChan + if !ok { + // No more query + return + } + + for _, query := range querys { + select { + case <-ctx.Done(): + return + default: + if err := bw.batchExecute(db, query); err != nil { + glog.Fatal(err) + } + } + } + } +} + +// Initialize inits case +func (c *BlockWriterCase) initialize(db *sql.DB) error { + glog.Infof("[%s] start to init...", c) + defer func() { + atomic.StoreUint32(&c.isInit, 1) + glog.Infof("[%s] init end...", c) + }() + + for i := 0; i < c.cfg.TableNum; i++ { + var s string + if i > 0 { + s = fmt.Sprintf("%d", i) + } + + tmt := fmt.Sprintf("CREATE TABLE IF NOT EXISTS block_writer%s %s", s, ` + ( + id BIGINT NOT NULL AUTO_INCREMENT, + raw_bytes BLOB NOT NULL, + PRIMARY KEY (id) +)`) + + err := wait.PollImmediate(5*time.Second, 30*time.Second, func() (bool, error) { + _, err := db.Exec(tmt) + if err != nil { + glog.Warningf("[%s] exec sql [%s] failed, err: %v, retry...", c, tmt, err) + return false, nil + } + + return true, nil + }) + + if err != nil { + glog.Errorf("[%s] exec sql [%s] failed, err: %v", c, tmt, err) + return err + } + } + + return nil +} + +// Start starts to run cases +func (c *BlockWriterCase) Start(db *sql.DB) error { + if !atomic.CompareAndSwapUint32(&c.isRunning, 0, 1) { + err := fmt.Errorf("[%s] is running, you can't start it again", c) + glog.Error(err) + return err + } + + defer func() { + c.RLock() + glog.Infof("[%s] stopped", c) + atomic.SwapUint32(&c.isRunning, 0) + }() + + if c.isInit == 0 { + if err := c.initialize(db); err != nil { + return err + } + } + + glog.Infof("[%s] start to execute case...", c) + + var wg sync.WaitGroup + + ctx, cancel := context.WithCancel(context.Background()) + + queryChan := make(chan []string, queryChanSize) + + for i := 0; i < c.cfg.Concurrency; i++ { + wg.Add(1) + go func(i int) { + defer wg.Done() + c.bws[i].run(ctx, db, queryChan) + }(i) + } + + wg.Add(1) + go c.generateQuery(ctx, queryChan, &wg) + +loop: + for { + select { + case <-c.stopChan: + glog.Infof("[%s] stoping...", c) + cancel() + break loop + default: + util.Sleep(context.Background(), 2*time.Second) + } + } + + wg.Wait() + close(queryChan) + + return nil +} + +// Stop stops cases +func (c *BlockWriterCase) Stop() { + c.stopChan <- struct{}{} +} + +// String implements fmt.Stringer interface. +func (c *BlockWriterCase) String() string { + return "block_writer" +} diff --git a/tests/pkg/workload/ddl/internal/datatype.go b/tests/pkg/workload/ddl/internal/datatype.go new file mode 100644 index 0000000000..9dc62c284e --- /dev/null +++ b/tests/pkg/workload/ddl/internal/datatype.go @@ -0,0 +1,338 @@ +// Copyright 2019 PingCAP, Inc. +// +// 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, +// See the License for the specific language governing permissions and +// limitations under the License. + +package internal + +import ( + "math/rand" + "time" +) + +// Kind constants. +const ( + KindTINYINT int = iota + KindSMALLINT //int16 + KindMEDIUMINT //int24 + KindInt32 //int32 + KindBigInt //int64 + KindBit + + KindFloat + KindDouble + + KindDECIMAL + + KindChar + KindVarChar + + KindBLOB + KindTINYBLOB + KindMEDIUMBLOB + KindLONGBLOB + + KindTEXT + KindTINYTEXT + KindMEDIUMTEXT + KindLONGTEXT + + KindBool + + KindDATE + KindTIME + KindDATETIME + KindTIMESTAMP + KindYEAR + + KindJSON + + KindEnum + KindSet +) + +var ALLFieldType = map[int]string{ + KindTINYINT: "TINYINT", + KindSMALLINT: "SMALLINT", + KindMEDIUMINT: "MEDIUMINT", + KindInt32: "INT", + KindBigInt: "BIGINT", + KindBit: "BIT", + + KindFloat: "FLOAT", + KindDouble: "DOUBLE", + + KindDECIMAL: "DECIMAL", + + KindChar: "CHAR", + KindVarChar: "VARCHAR", + + KindBLOB: "BLOB", + KindTINYBLOB: "TINYBLOB", + KindMEDIUMBLOB: "MEDIUMBLOB", + KindLONGBLOB: "LONGBLOB", + + KindTEXT: "TEXT", + KindTINYTEXT: "TINYTEXT", + KindMEDIUMTEXT: "MEDIUMTEXT", + KindLONGTEXT: "LONGTEXT", + + KindBool: "BOOL", + + KindDATE: "DATE", + KindTIME: "TIME", + KindDATETIME: "DATETIME", + KindTIMESTAMP: "TIMESTAMP", + KindYEAR: "YEAR", + + KindJSON: "JSON", + KindEnum: "ENUM", + KindSet: "SET", +} + +// testFieldType is use to control what kind of data types to test, +// if we just want to test timestamp, we can annotation the another data types. +var testFieldType = []int{ + KindTINYINT, + KindSMALLINT, + KindMEDIUMINT, + KindInt32, + KindBigInt, + + //KindBit, // have default value bug unfix. + + KindFloat, + KindDouble, + + KindDECIMAL, + + KindChar, + KindVarChar, + + KindBLOB, + KindTEXT, + + KindBool, + + KindDATE, + KindTIME, + KindDATETIME, + KindTIMESTAMP, + KindYEAR, + + //KindJSON, // have `admin check table when index is virtual generated column` bug unfix + KindEnum, + KindSet, +} + +func randDataType() int { + i := rand.Intn(len(testFieldType)) + return testFieldType[i] +} + +const ( + BitMaxLen = 64 + CharMaxLen = 256 + VarCharMaxLen = 256 // varchar max len , actual range is [0,65536) + BLOBMaxLen = 256 // BLOB max len , actual range is [0,65536) + TINYBLOBMaxLen = 256 // TINYBLOB max len , actual range is [0,256) + MEDIUMBLOBMaxLen = 256 // MEDIUMBLOB max len , actual range is [0,16777216) + LONGBLOBMaxLen = 256 // LONGBLOB max len , actual range is [0,4294967296) + + TEXTMaxLen = 256 // TEXT max len , actual range is [0,65536) + TINYTEXTMaxLen = 256 // TINYTEXT max len , actual range is [0,256) + MEDIUMTEXTMaxLen = 256 // MEDIUMTEXT max len , actual range is [0,16777216) + LONGTEXTMaxLen = 256 // LONGTEXT max len , actual range is [0,4294967296) + + MAXDECIMALM = 65 // 1~65 + MAXDECIMALN = 30 // 0~30 + + EnumMaxLen = 10 + SetMaxLen = 10 + + TimeFormat = "2006-01-02 15:04:05" + TimeFormatForDATE = "2006-01-02" + TimeFormatForTIME = "15:04:05" + + MINDATETIME = "1000-01-01 00:00:00" + MAXDATETIME = "9999-12-31 23:59:59" + + MINTIMESTAMP = "1970-01-01 00:00:01" + MAXTIMESTAMP = "2038-01-19 03:14:07" +) + +var MinDATETIME time.Time +var MaxDATETIME time.Time +var GapDATETIMEUnix int64 + +var MinTIMESTAMP time.Time +var MaxTIMESTAMP time.Time +var GapTIMESTAMPUnix int64 + +func getMaxLenByKind(kind int) int { + switch kind { + case KindChar: + return CharMaxLen + case KindVarChar: + return VarCharMaxLen + case KindBLOB: + return BLOBMaxLen + case KindTINYBLOB: + return TINYBLOBMaxLen + case KindMEDIUMBLOB: + return MEDIUMBLOBMaxLen + case KindLONGBLOB: + return LONGBLOBMaxLen + case KindTEXT: + return TEXTMaxLen + case KindTINYTEXT: + return TINYTEXTMaxLen + case KindMEDIUMTEXT: + return MEDIUMTEXTMaxLen + case KindLONGTEXT: + return LONGTEXTMaxLen + case KindBit: + return BitMaxLen + case KindEnum: + return EnumMaxLen + case KindSet: + return SetMaxLen + } + return 0 +} + +type ambiguousTimeStr struct { + start string + end string +} + +type ambiguousTime struct { + start int64 + end int64 +} + +var ambiguousTimeStrSlice = []ambiguousTimeStr{ + // backward + { + start: "1900-12-31 23:54:17", + end: "1901-01-01 00:00:00", + }, + // moved forward + { + start: "1940-06-02 23:59:59", + end: "1940-06-03 01:00:00", + }, + // move backward + { + start: "1940-09-30 23:00:00", + end: "1940-10-01 00:00:00", + }, + // moved forward + { + start: "1941-03-15 23:59:59", + end: "1941-03-16 01:00:00", + }, + // move backward + { + start: "1941-09-30 23:00:00", + end: "1941-10-01 00:00:00", + }, + // moved forward + { + start: "1986-05-03 23:59:59", + end: "1986-05-04 01:00:00", + }, + // move backward + { + start: "1986-09-13 23:00:00", + end: "1986-09-14 00:00:00", + }, + // moved forward + { + start: "1987-04-11 23:59:59", + end: "1987-04-12 01:00:00", + }, + // move backward + { + start: "1987-09-12 23:00:00", + end: "1987-09-13 00:00:00", + }, + // moved forward + { + start: "1988-04-09 23:59:59", + end: "1988-04-10 01:00:00", + }, + // move backward + { + start: "1988-09-10 23:00:00", + end: "1988-09-11 00:00:00", + }, + + // moved forward + { + start: "1989-04-15 23:59:59", + end: "1989-04-16 01:00:00", + }, + // move backward + { + start: "1989-09-16 23:00:00", + end: "1989-09-17 00:00:00", + }, + // moved forward + { + start: "1990-04-14 23:59:59", + end: "1990-04-15 01:00:00", + }, + // move backward + { + start: "1990-09-15 23:00:00", + end: "1990-09-16 00:00:00", + }, + // moved forward + { + start: "1991-04-13 23:59:59", + end: "1991-04-14 01:00:00", + }, + // move backward + { + start: "1991-09-14 23:00:00", + end: "1991-09-15 00:00:00", + }, +} + +var ambiguousTimeSlice []ambiguousTime + +var local = time.Local + +func init() { + var err error + local, err = time.LoadLocation("Asia/Shanghai") + if err != nil { + local = time.Local + } + for _, v := range ambiguousTimeStrSlice { + start, _ := time.ParseInLocation(TimeFormat, v.start, local) + end, _ := time.ParseInLocation(TimeFormat, v.end, local) + amt := ambiguousTime{ + start: start.Unix(), + end: end.Unix(), + } + ambiguousTimeSlice = append(ambiguousTimeSlice, amt) + } + + MinDATETIME, _ = time.ParseInLocation(TimeFormat, MINDATETIME, local) + MaxDATETIME, _ = time.ParseInLocation(TimeFormat, MAXDATETIME, local) + GapDATETIMEUnix = MaxDATETIME.Unix() - MinDATETIME.Unix() + + MinTIMESTAMP, _ = time.ParseInLocation(TimeFormat, MINTIMESTAMP, local) + MaxTIMESTAMP, _ = time.ParseInLocation(TimeFormat, MAXTIMESTAMP, local) + GapTIMESTAMPUnix = MaxTIMESTAMP.Unix() - MinTIMESTAMP.Unix() +} diff --git a/tests/pkg/workload/ddl/internal/ddl.go b/tests/pkg/workload/ddl/internal/ddl.go new file mode 100644 index 0000000000..cdcb3ee38d --- /dev/null +++ b/tests/pkg/workload/ddl/internal/ddl.go @@ -0,0 +1,584 @@ +// Copyright 2019 PingCAP, Inc. +// +// 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, +// See the License for the specific language governing permissions and +// limitations under the License. +package internal + +import ( + "context" + "database/sql" + "fmt" + "math/rand" + "sync" + "sync/atomic" + "time" + "unsafe" + + "github.com/golang/glog" + "github.com/juju/errors" +) + +// The DDL test case is intended to test the correctness of DDL operations. It +// generates test cases by probability so that it should be run in background for +// enough time to see if there are any issues. +// +// The DDL test case have multiple go routines run in parallel, one for DML operations, +// other for DDL operations. The feature of each operation (for example, covering +// what kind of scenario) is determined and generated at start up time (See +// `generateDMLOps`, `generateDDLOps``), while the order of each operation is +// randomized in each round. +// +// If there are remaining DDL operations while all DML operations are performed, a +// new round of DML operations will be started (with new randomized order) and when +// all DDL operations are done, the remaining DML operations are discarded. vice +// versa. +// +// Since there are some conflicts between some DDL operations and DML operations, +// for example, inserting a row while removing a column may cause errors in +// inserting because of incorrect column numbers, some locks and some conflicting +// detections are introduced. The conflicting detection will ignore errors raised +// in such scenarios. In addition, the data in memory is stored by column instead +// of by row to minimize data conflicts in adding and removing columns. + +type DDLCaseConfig struct { + Concurrency int `toml:"concurrency"` + MySQLCompatible bool `toml:"mysql_compactible"` + TablesToCreate int `toml:"tables_to_create"` + TestTp DDLTestType `toml:"test_type"` +} + +type DDLTestType int + +const ( + SerialDDLTest DDLTestType = iota + ParallelDDLTest +) + +type ExecuteDDLFunc func(*testCase, []ddlTestOpExecutor, func() error) error +type ExecuteDMLFunc func(*testCase, []dmlTestOpExecutor, func() error) error + +type DDLCase struct { + cfg *DDLCaseConfig + cases []*testCase +} + +func (c *DDLCase) String() string { + return "ddl" +} + +// Execute executes each goroutine (i.e. `testCase`) concurrently. +func (c *DDLCase) Execute(ctx context.Context, dbss [][]*sql.DB, exeDDLFunc ExecuteDDLFunc, exeDMLFunc ExecuteDMLFunc) error { + for _, dbs := range dbss { + for _, db := range dbs { + enableTiKVGC(db) + } + } + + glog.Infof("[%s] start to test...", c) + defer func() { + glog.Infof("[%s] test end...", c) + }() + var wg sync.WaitGroup + for i := 0; i < c.cfg.Concurrency; i++ { + wg.Add(1) + go func(i int) { + defer wg.Done() + for { + select { + case <-ctx.Done(): + return + default: + } + err := c.cases[i].execute(exeDDLFunc, exeDMLFunc) + if err != nil { + for _, dbs := range dbss { + for _, db := range dbs { + disableTiKVGC(db) + } + } + glog.Fatalf("[ddl] [instance %d] ERROR: %s", i, errors.ErrorStack(err)) + } + } + }(i) + } + wg.Wait() + return nil +} + +// Initialize initializes each concurrent goroutine (i.e. `testCase`). +func (c *DDLCase) Initialize(ctx context.Context, dbss [][]*sql.DB) error { + for i := 0; i < c.cfg.Concurrency; i++ { + err := c.cases[i].initialize(dbss[i]) + if err != nil { + return errors.Trace(err) + } + } + return nil +} + +// NewDDLCase returns a DDLCase, which contains specified `testCase`s. +func NewDDLCase(cfg *DDLCaseConfig) *DDLCase { + cases := make([]*testCase, cfg.Concurrency) + for i := 0; i < cfg.Concurrency; i++ { + cases[i] = &testCase{ + cfg: cfg, + tables: make(map[string]*ddlTestTable), + ddlOps: make([]ddlTestOpExecutor, 0), + dmlOps: make([]dmlTestOpExecutor, 0), + caseIndex: i, + stop: 0, + } + } + b := &DDLCase{ + cfg: cfg, + cases: cases, + } + return b +} + +const ( + ddlTestValueNull string = "NULL" + ddlTestValueInvalid int32 = -99 +) + +type ddlTestOpExecutor struct { + executeFunc func(interface{}, chan *ddlJobTask) error + config interface{} + ddlKind DDLKind +} + +type dmlTestOpExecutor struct { + prepareFunc func(interface{}, chan *dmlJobTask) error + config interface{} +} + +type DMLKind int + +const ( + dmlInsert DMLKind = iota + dmlUpdate + dmlDelete +) + +type dmlJobArg unsafe.Pointer + +type dmlJobTask struct { + k DMLKind + tblInfo *ddlTestTable + sql string + assigns []*ddlTestColumnDescriptor + whereColumns []*ddlTestColumnDescriptor + err error +} + +// initialize generates possible DDL and DML operations for one `testCase`. +// Different `testCase`s will be run in parallel according to the concurrent configuration. +func (c *testCase) initialize(dbs []*sql.DB) error { + var err error + c.dbs = dbs + if err = c.generateDDLOps(); err != nil { + return errors.Trace(err) + } + if err = c.generateDMLOps(); err != nil { + return errors.Trace(err) + } + // Create 2 table before executes DDL & DML + taskCh := make(chan *ddlJobTask, 2) + c.prepareAddTable(nil, taskCh) + c.prepareAddTable(nil, taskCh) + if c.cfg.TestTp == SerialDDLTest { + err = c.execSerialDDLSQL(taskCh) + if err != nil { + return errors.Trace(err) + } + err = c.execSerialDDLSQL(taskCh) + } else { + err = c.execParaDDLSQL(taskCh, len(taskCh)) + } + if err != nil { + return errors.Trace(err) + } + return nil +} + +/* +ParallelExecuteOperations executes process: +1. Generate many kind of DDL SQLs +2. Parallel send every kind of DDL request to TiDB +3. Wait all DDL SQLs request finish +4. Send `admin show ddl jobs` request to TiDB to confirm parallel DDL requests execute order +5. Do the same DDL change on local with the same DDL requests executed order of TiDB +6. Judge the every DDL execution result of TiDB and local. If both of local and TiDB execute result are no wrong, or both are wrong it will be ok. Otherwise, It must be something wrong. +*/ +func ParallelExecuteOperations(c *testCase, ops []ddlTestOpExecutor, postOp func() error) error { + perm := rand.Perm(len(ops)) + taskCh := make(chan *ddlJobTask, len(ops)) + for _, idx := range perm { + if c.isStop() { + return nil + } + op := ops[idx] + if rand.Float64() > mapOfDDLKindProbability[op.ddlKind] { + continue + } + op.executeFunc(op.config, taskCh) + } + err := c.execParaDDLSQL(taskCh, len(taskCh)) + if err != nil { + return errors.Trace(err) + } + close(taskCh) + time.Sleep(time.Duration(rand.Intn(100)) * time.Millisecond) + return nil +} + +func SerialExecuteOperations(c *testCase, ops []ddlTestOpExecutor, postOp func() error) error { + perm := rand.Perm(len(ops)) + taskCh := make(chan *ddlJobTask, 1) + for _, idx := range perm { + if c.isStop() { + return nil + } + op := ops[idx] + if rand.Float64() > mapOfDDLKindProbability[op.ddlKind] { + continue + } + op.executeFunc(op.config, taskCh) + err := c.execSerialDDLSQL(taskCh) + if err != nil { + return errors.Trace(err) + } + } + close(taskCh) + time.Sleep(time.Duration(rand.Intn(100)) * time.Millisecond) + return nil +} + +func TransactionExecuteOperations(c *testCase, ops []dmlTestOpExecutor, postOp func() error) error { + transactionOpsLen := rand.Intn(len(ops)) + if transactionOpsLen < 1 { + transactionOpsLen = 1 + } + taskCh := make(chan *dmlJobTask, len(ops)) + opNum := 0 + perm := rand.Perm(len(ops)) + for i, idx := range perm { + if c.isStop() { + return nil + } + op := ops[idx] + err := op.prepareFunc(op.config, taskCh) + if err != nil { + if err.Error() != "Conflict operation" { + return errors.Trace(err) + } + continue + } + opNum++ + if opNum >= transactionOpsLen { + err = c.execDMLInTransactionSQL(taskCh) + if err != nil { + return errors.Trace(err) + } + transactionOpsLen = rand.Intn(len(ops)) + if transactionOpsLen < 1 { + transactionOpsLen = 1 + } + if transactionOpsLen > (len(ops) - i) { + transactionOpsLen = len(ops) - i + } + opNum = 0 + if postOp != nil { + err = postOp() + if err != nil { + return errors.Trace(err) + } + } + time.Sleep(time.Duration(rand.Intn(50)) * time.Millisecond) + } + } + return nil +} + +func SerialExecuteDML(c *testCase, ops []dmlTestOpExecutor, postOp func() error) error { + perm := rand.Perm(len(ops)) + taskCh := make(chan *dmlJobTask, 1) + for _, idx := range perm { + if c.isStop() { + return nil + } + op := ops[idx] + err := op.prepareFunc(op.config, taskCh) + if err != nil { + if err.Error() != "Conflict operation" { + return errors.Trace(err) + } + continue + } + err = c.execSerialDMLSQL(taskCh) + if err != nil { + return errors.Trace(err) + } + } + close(taskCh) + time.Sleep(time.Duration(rand.Intn(100)) * time.Millisecond) + return nil +} + +// execute iterates over two list of operations concurrently, one is +// ddl operations, one is dml operations. +// When one list completes, it starts over from the beginning again. +// When both of them ONCE complete, it exits. +func (c *testCase) execute(executeDDL ExecuteDDLFunc, exeDMLFunc ExecuteDMLFunc) error { + var ( + ddlAllComplete int32 = 0 + dmlAllComplete int32 = 0 + ) + + err := parallel(func() error { + var err error + for { + err = executeDDL(c, c.ddlOps, nil) + atomic.StoreInt32(&ddlAllComplete, 1) + if atomic.LoadInt32(&ddlAllComplete) != 0 && atomic.LoadInt32(&dmlAllComplete) != 0 || err != nil { + break + } + } + return errors.Trace(err) + }, func() error { + var err error + for { + err = exeDMLFunc(c, c.dmlOps, func() error { + return c.executeVerifyIntegrity() + }) + atomic.StoreInt32(&dmlAllComplete, 1) + if atomic.LoadInt32(&ddlAllComplete) != 0 && atomic.LoadInt32(&dmlAllComplete) != 0 || err != nil { + break + } + } + return errors.Trace(err) + }) + + if err != nil { + ddlFailedCounter.Inc() + return errors.Trace(err) + } + + glog.Infof("[ddl] [instance %d] Round completed", c.caseIndex) + glog.Infof("[ddl] [instance %d] Executing post round operations...", c.caseIndex) + + if !c.cfg.MySQLCompatible { + err := c.executeAdminCheck() + if err != nil { + return errors.Trace(err) + } + } + + return nil +} + +var selectID int32 + +// executeVerifyIntegrity verifies the integrity of the data in the database +// by comparing the data in memory (that we expected) with the data in the database. +func (c *testCase) executeVerifyIntegrity() error { + c.tablesLock.RLock() + tablesSnapshot := make([]*ddlTestTable, 0) + for _, table := range c.tables { + tablesSnapshot = append(tablesSnapshot, table) + } + gotTableTime := time.Now() + c.tablesLock.RUnlock() + + uniqID := atomic.AddInt32(&selectID, 1) + + for _, table := range tablesSnapshot { + table.lock.RLock() + columnsSnapshot := table.filterColumns(table.predicateAll) + table.lock.RUnlock() + + // build SQL + sql := "SELECT " + for i, column := range columnsSnapshot { + if i > 0 { + sql += ", " + } + sql += fmt.Sprintf("%s", column.getSelectName()) + } + sql += fmt.Sprintf(" FROM `%s`", table.name) + + dbIdx := rand.Intn(len(c.dbs)) + db := c.dbs[dbIdx] + + // execute + opStart := time.Now() + rows, err := db.Query(sql) + glog.Infof("[ddl] [instance %d] %s, elapsed time:%v, got table time:%v, selectID:%v", c.caseIndex, sql, time.Since(opStart).Seconds(), gotTableTime, uniqID) + if err == nil { + defer rows.Close() + } + // When column is removed, SELECT statement may return error so that we ignore them here. + if table.isDeleted() { + return nil + } + for _, column := range columnsSnapshot { + if column.isDeleted() { + return nil + } + } + if err != nil { + return errors.Annotatef(err, "Error when executing SQL: %s\n%s", sql, table.debugPrintToString()) + } + + // Read all rows. + var actualRows [][]interface{} + for rows.Next() { + cols, err1 := rows.Columns() + if err1 != nil { + return errors.Trace(err) + } + + glog.Infof("[ddl] [instance %d] rows.Columns():%v, len(cols):%v, selectID:%v", c.caseIndex, cols, len(cols), uniqID) + + // See https://stackoverflow.com/questions/14477941/read-select-columns-into-string-in-go + rawResult := make([][]byte, len(cols)) + result := make([]interface{}, len(cols)) + dest := make([]interface{}, len(cols)) + for i := range rawResult { + dest[i] = &rawResult[i] + } + + err1 = rows.Scan(dest...) + if err1 != nil { + return errors.Trace(err) + } + + for i, raw := range rawResult { + if raw == nil { + result[i] = ddlTestValueNull + } else { + result[i] = trimValue(columnsSnapshot[i].k, raw) + } + } + + actualRows = append(actualRows, result) + } + if rows.Err() != nil { + return errors.Trace(rows.Err()) + } + + // Even if SQL executes successfully, column deletion will cause different data as well + if table.isDeleted() { + return nil + } + for _, column := range columnsSnapshot { + if column.isDeleted() { + return nil + } + } + + // Make signatures for actual rows. + actualRowsMap := make(map[string]int) + for _, row := range actualRows { + rowString := "" + for _, col := range row { + rowString += fmt.Sprintf("%v,", col) + } + _, ok := actualRowsMap[rowString] + if !ok { + actualRowsMap[rowString] = 0 + } + actualRowsMap[rowString]++ + } + + // Compare with expecting rows. + checkTime := time.Now() + for i := 0; i < table.numberOfRows; i++ { + rowString := "" + for _, column := range columnsSnapshot { + if column.rows[i] == nil { + rowString += fmt.Sprintf("NULL,") + } else { + rowString += fmt.Sprintf("%v,", column.rows[i]) + } + } + _, ok := actualRowsMap[rowString] + if !ok { + c.stopTest() + err = fmt.Errorf("Expecting row %s in table `%s` but not found, sql: %s, selectID:%v, checkTime:%v, rowErr:%v, actualRowsMap:%#v\n%s", rowString, table.name, sql, uniqID, checkTime, rows.Err(), actualRowsMap, table.debugPrintToString()) + glog.Infof("err: %v", err) + return errors.Trace(err) + } + actualRowsMap[rowString]-- + if actualRowsMap[rowString] < 0 { + c.stopTest() + err = fmt.Errorf("Expecting row %s in table `%s` but not found, sql: %s, selectID:%v, checkTime:%v, rowErr:%v, actualRowsMap:%#v\n%s", rowString, table.name, sql, uniqID, checkTime, rows.Err(), actualRowsMap, table.debugPrintToString()) + glog.Infof("err: %v", err) + return errors.Trace(err) + } + } + for rowString, occurs := range actualRowsMap { + if occurs > 0 { + c.stopTest() + err = fmt.Errorf("Unexpected row %s in table `%s`, sql: %s, selectID:%v, checkTime:%v, rowErr:%v, actualRowsMap:%#v\n%s", rowString, table.name, sql, uniqID, checkTime, rows.Err(), actualRowsMap, table.debugPrintToString()) + glog.Infof("err: %v", err) + return errors.Trace(err) + } + } + } + return nil +} + +func trimValue(tp int, val []byte) string { + // a='{"DnOJQOlx":52,"ZmvzPtdm":82}' + // eg: set a={"a":"b","b":"c"} + // get a={"a": "b", "b": "c"} , so have to remove the space + if tp == KindJSON { + for i := 1; i < len(val)-2; i++ { + if val[i-1] == '"' && val[i] == ':' && val[i+1] == ' ' { + val = append(val[:i+1], val[i+2:]...) + } + if val[i-1] == ',' && val[i] == ' ' && val[i+1] == '"' { + val = append(val[:i], val[i+1:]...) + } + } + } + return string(val) +} + +func (c *testCase) executeAdminCheck() error { + if len(c.tables) == 0 { + return nil + } + + // build SQL + sql := "ADMIN CHECK TABLE " + i := 0 + for _, table := range c.tables { + if i > 0 { + sql += ", " + } + sql += fmt.Sprintf("`%s`", table.name) + i++ + } + dbIdx := rand.Intn(len(c.dbs)) + db := c.dbs[dbIdx] + // execute + glog.Infof("[ddl] [instance %d] %s", c.caseIndex, sql) + _, err := db.Exec(sql) + if err != nil { + if ignore_error(err) { + return nil + } + return errors.Annotatef(err, "Error when executing SQL: %s", sql) + } + return nil +} diff --git a/tests/pkg/workload/ddl/internal/ddl_ops.go b/tests/pkg/workload/ddl/internal/ddl_ops.go new file mode 100644 index 0000000000..5ef4edc866 --- /dev/null +++ b/tests/pkg/workload/ddl/internal/ddl_ops.go @@ -0,0 +1,847 @@ +// Copyright 2019 PingCAP, Inc. +// +// 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, +// See the License for the specific language governing permissions and +// limitations under the License. +package internal + +import ( + "database/sql" + "fmt" + "math/rand" + "sort" + "strconv" + "sync" + "time" + "unsafe" + + "k8s.io/apimachinery/pkg/util/uuid" + + "github.com/golang/glog" + "github.com/juju/errors" +) + +func (c *testCase) generateDDLOps() error { + if err := c.generateAddTable(); err != nil { + return errors.Trace(err) + } + if err := c.generateDropTable(); err != nil { + return errors.Trace(err) + } + if err := c.generateAddIndex(); err != nil { + return errors.Trace(err) + } + if err := c.generateDropIndex(); err != nil { + return errors.Trace(err) + } + if err := c.generateAddColumn(); err != nil { + return errors.Trace(err) + } + if err := c.generateDropColumn(); err != nil { + return errors.Trace(err) + } + return nil +} + +type DDLKind = int + +const ( + ddlAddTable DDLKind = iota + ddlAddIndex + ddlAddColumn + + ddlDropTable + ddlDropIndex + ddlDropColumn + + ddlKindNil +) + +var mapOfDDLKind = map[string]DDLKind{ + "create table": ddlAddTable, + "add index": ddlAddIndex, + "add column": ddlAddColumn, + + "drop table": ddlDropTable, + "drop index": ddlDropIndex, + "drop column": ddlDropColumn, +} + +var mapOfDDLKindToString = map[DDLKind]string{ + ddlAddTable: "create table", + ddlAddIndex: "add index", + ddlAddColumn: "add column", + + ddlDropTable: "drop table", + ddlDropIndex: "drop index", + ddlDropColumn: "drop column", +} + +// mapOfDDLKindProbability use to control every kind of ddl request execute probability. +var mapOfDDLKindProbability = map[DDLKind]float64{ + ddlAddTable: 0.15, + ddlDropTable: 0.15, + + ddlAddIndex: 0.8, + ddlDropIndex: 0.5, + + ddlAddColumn: 0.8, + ddlDropColumn: 0.5, +} + +type ddlJob struct { + id int + tableName string + k DDLKind + jobState string + tableID string +} + +type ddlJobArg unsafe.Pointer + +type ddlJobTask struct { + ddlID int + k DDLKind + tblInfo *ddlTestTable + sql string + arg ddlJobArg + err error // remote TiDB execute error +} + +func (c *testCase) updateTableInfo(task *ddlJobTask) error { + switch task.k { + case ddlAddTable: + return c.addTableInfo(task) + case ddlDropTable: + return c.dropTableJob(task) + case ddlAddIndex: + return c.addIndexJob(task) + case ddlDropIndex: + return c.dropIndexJob(task) + case ddlAddColumn: + return c.addColumnJob(task) + case ddlDropColumn: + return c.dropColumnJob(task) + } + return fmt.Errorf("unknow ddl task , %v", *task) +} + +/* +execParaDDLSQL get a batch of ddl from taskCh, and then: +1. Parallel send every kind of DDL request to TiDB +2. Wait all DDL SQLs request finish +3. Send `admin show ddl jobs` request to TiDB to confirm parallel DDL requests execute order +4. Do the same DDL change on local with the same DDL requests executed order of TiDB +5. Judge the every DDL execution result of TiDB and local. If both of local and TiDB execute result are no wrong, or both are wrong it will be ok. Otherwise, It must be something wrong. +*/ +func (c *testCase) execParaDDLSQL(taskCh chan *ddlJobTask, num int) error { + if num == 0 { + return nil + } + tasks := make([]*ddlJobTask, 0, num) + var wg sync.WaitGroup + for i := 0; i < num; i++ { + task := <-taskCh + tasks = append(tasks, task) + wg.Add(1) + go func(task *ddlJobTask) { + defer wg.Done() + opStart := time.Now() + db := c.dbs[0] + _, err := db.Exec(task.sql) + glog.Infof("[ddl] [instance %d] TiDB execute %s , err %v, table_id %s, elapsed time:%v", c.caseIndex, task.sql, err, task.tblInfo.id, time.Since(opStart).Seconds()) + task.err = err + }(task) + } + wg.Wait() + db := c.dbs[0] + SortTasks, err := c.getSortTask(db, tasks) + if err != nil { + return err + } + for _, task := range SortTasks { + err := c.updateTableInfo(task) + glog.Infof("[ddl] [instance %d] local execute %s, err %v , table_id %s, ddlID %v", c.caseIndex, task.sql, err, task.tblInfo.id, task.ddlID) + if err == nil && task.err != nil || err != nil && task.err == nil { + return fmt.Errorf("Error when executing SQL: %s\n, local err: %#v, remote tidb err: %#v\n%s\n", task.sql, err, task.err, task.tblInfo.debugPrintToString()) + } + } + return nil +} + +// execSerialDDLSQL gets a job from taskCh, and then executes the job. +func (c *testCase) execSerialDDLSQL(taskCh chan *ddlJobTask) error { + if len(taskCh) < 1 { + return nil + } + task := <-taskCh + db := c.dbs[0] + opStart := time.Now() + _, err := db.Exec(task.sql) + glog.Infof("[ddl] [instance %d] %s, elapsed time:%v", c.caseIndex, task.sql, time.Since(opStart).Seconds()) + if err != nil { + return fmt.Errorf("Error when executing SQL: %s\n remote tidb Err: %#v\n%s\n", task.sql, err, task.tblInfo.debugPrintToString()) + } + err = c.updateTableInfo(task) + if err != nil { + return fmt.Errorf("Error when executing SQL: %s\n local Err: %#v\n%s\n", task.sql, err, task.tblInfo.debugPrintToString()) + } + return nil +} + +func (c *testCase) generateAddTable() error { + c.ddlOps = append(c.ddlOps, ddlTestOpExecutor{c.prepareAddTable, nil, ddlAddTable}) + return nil +} + +func (c *testCase) prepareAddTable(cfg interface{}, taskCh chan *ddlJobTask) error { + columnCount := rand.Intn(c.cfg.TablesToCreate) + 2 + tableColumns := make([]*ddlTestColumn, 0, columnCount) + for i := 0; i < columnCount; i++ { + columns := getRandDDLTestColumns() + tableColumns = append(tableColumns, columns...) + } + + // Generate primary key with [0, 3) size + primaryKeyFields := rand.Intn(3) + primaryKeys := make([]int, 0) + if primaryKeyFields > 0 { + // Random elections column as primary key, but also check the column whether can be primary key. + perm := rand.Perm(len(tableColumns))[0:primaryKeyFields] + for _, columnIndex := range perm { + if tableColumns[columnIndex].canBePrimary() { + tableColumns[columnIndex].isPrimaryKey = true + primaryKeys = append(primaryKeys, columnIndex) + } + } + primaryKeyFields = len(primaryKeys) + } + + tableInfo := ddlTestTable{ + name: string(uuid.NewUUID()), + columns: tableColumns, + indexes: make([]*ddlTestIndex, 0), + numberOfRows: 0, + deleted: 0, + } + + sql := fmt.Sprintf("CREATE TABLE `%s` (", tableInfo.name) + for i := 0; i < len(tableInfo.columns); i++ { + if i > 0 { + sql += ", " + } + sql += fmt.Sprintf("`%s` %s", tableColumns[i].name, tableColumns[i].getDefinition()) + } + if primaryKeyFields > 0 { + sql += ", PRIMARY KEY (" + for i, columnIndex := range primaryKeys { + if i > 0 { + sql += ", " + } + sql += fmt.Sprintf("`%s`", tableColumns[columnIndex].name) + } + sql += ")" + } + sql += ")" + + task := &ddlJobTask{ + k: ddlAddTable, + sql: sql, + tblInfo: &tableInfo, + } + taskCh <- task + return nil +} + +func (c *testCase) addTableInfo(task *ddlJobTask) error { + c.tablesLock.Lock() + defer c.tablesLock.Unlock() + c.tables[task.tblInfo.name] = task.tblInfo + + return nil +} + +func (c *testCase) generateDropTable() error { + c.ddlOps = append(c.ddlOps, ddlTestOpExecutor{c.prepareDropTable, nil, ddlDropTable}) + return nil +} + +func (c *testCase) prepareDropTable(cfg interface{}, taskCh chan *ddlJobTask) error { + c.tablesLock.Lock() + defer c.tablesLock.Unlock() + tableToDrop := c.pickupRandomTable() + if len(c.tables) <= 1 || tableToDrop == nil { + return nil + } + tableToDrop.setDeleted() + sql := fmt.Sprintf("DROP TABLE `%s`", tableToDrop.name) + + task := &ddlJobTask{ + k: ddlDropTable, + sql: sql, + tblInfo: tableToDrop, + } + taskCh <- task + return nil +} + +func (c *testCase) dropTableJob(task *ddlJobTask) error { + c.tablesLock.Lock() + defer c.tablesLock.Unlock() + if c.isTableDeleted(task.tblInfo) { + return fmt.Errorf("table %s is not exists", task.tblInfo.name) + } + delete(c.tables, task.tblInfo.name) + return nil +} + +type ddlTestIndexStrategy = int + +const ( + ddlTestIndexStrategyBegin ddlTestIndexStrategy = iota + ddlTestIndexStrategySingleColumnAtBeginning + ddlTestIndexStrategySingleColumnAtEnd + ddlTestIndexStrategySingleColumnRandom + ddlTestIndexStrategyMultipleColumnRandom + ddlTestIndexStrategyEnd +) + +type ddlTestAddIndexConfig struct { + strategy ddlTestIndexStrategy +} + +type ddlIndexJobArg struct { + index *ddlTestIndex +} + +func (c *testCase) generateAddIndex() error { + c.ddlOps = append(c.ddlOps, ddlTestOpExecutor{c.prepareAddIndex, nil, ddlAddIndex}) + return nil +} + +func (c *testCase) prepareAddIndex(_ interface{}, taskCh chan *ddlJobTask) error { + table := c.pickupRandomTable() + if table == nil { + return nil + } + strategy := rand.Intn(ddlTestIndexStrategyMultipleColumnRandom) + ddlTestIndexStrategySingleColumnAtBeginning + // build index definition + index := ddlTestIndex{ + name: string(uuid.NewUUID()), + signature: "", + columns: make([]*ddlTestColumn, 0), + } + + switch strategy { + case ddlTestIndexStrategySingleColumnAtBeginning: + if !table.columns[0].canBeIndex() { + return nil + } + index.columns = append(index.columns, table.columns[0]) + case ddlTestIndexStrategySingleColumnAtEnd: + if !table.columns[len(table.columns)-1].canBeIndex() { + return nil + } + index.columns = append(index.columns, table.columns[len(table.columns)-1]) + case ddlTestIndexStrategySingleColumnRandom: + col := table.columns[rand.Intn(len(table.columns))] + if !col.canBeIndex() { + return nil + } + index.columns = append(index.columns, col) + case ddlTestIndexStrategyMultipleColumnRandom: + numberOfColumns := rand.Intn(len(table.columns)) + 1 + // Multiple columns of one index should no more than 16. + if numberOfColumns > 10 { + numberOfColumns = 10 + } + perm := rand.Perm(len(table.columns))[:numberOfColumns] + for _, idx := range perm { + if table.columns[idx].canBeIndex() { + index.columns = append(index.columns, table.columns[idx]) + } + } + } + + if len(index.columns) == 0 { + return nil + } + + signature := "" + for _, col := range index.columns { + signature += col.name + "," + } + index.signature = signature + + // check whether index duplicates + for _, idx := range table.indexes { + if idx.signature == index.signature { + return nil + } + } + + // build SQL + sql := fmt.Sprintf("ALTER TABLE `%s` ADD INDEX `%s` (", table.name, index.name) + for i, column := range index.columns { + if i > 0 { + sql += ", " + } + sql += fmt.Sprintf("`%s`", column.name) + } + sql += ")" + + arg := &ddlIndexJobArg{index: &index} + task := &ddlJobTask{ + k: ddlAddIndex, + sql: sql, + tblInfo: table, + arg: ddlJobArg(arg), + } + taskCh <- task + return nil +} + +func (c *testCase) addIndexJob(task *ddlJobTask) error { + jobArg := (*ddlIndexJobArg)(task.arg) + tblInfo := task.tblInfo + + if c.isTableDeleted(tblInfo) { + return fmt.Errorf("table %s is not exists", tblInfo.name) + } + + for _, column := range jobArg.index.columns { + if tblInfo.isColumnDeleted(column) { + return fmt.Errorf("local Execute add index %s on column %s error , column is deleted", jobArg.index.name, column.name) + } + } + tblInfo.indexes = append(tblInfo.indexes, jobArg.index) + for _, column := range jobArg.index.columns { + column.indexReferences++ + } + return nil +} + +func (c *testCase) generateDropIndex() error { + c.ddlOps = append(c.ddlOps, ddlTestOpExecutor{c.prepareDropIndex, nil, ddlDropIndex}) + return nil +} + +func (c *testCase) prepareDropIndex(_ interface{}, taskCh chan *ddlJobTask) error { + table := c.pickupRandomTable() + if table == nil { + return nil + } + if len(table.indexes) == 0 { + return nil + } + indexToDropIndex := rand.Intn(len(table.indexes)) + indexToDrop := table.indexes[indexToDropIndex] + sql := fmt.Sprintf("ALTER TABLE `%s` DROP INDEX `%s`", table.name, indexToDrop.name) + + arg := &ddlIndexJobArg{index: indexToDrop} + task := &ddlJobTask{ + k: ddlDropIndex, + sql: sql, + tblInfo: table, + arg: ddlJobArg(arg), + } + taskCh <- task + return nil +} + +func (c *testCase) dropIndexJob(task *ddlJobTask) error { + jobArg := (*ddlIndexJobArg)(task.arg) + tblInfo := task.tblInfo + + if c.isTableDeleted(tblInfo) { + return fmt.Errorf("table %s is not exists", tblInfo.name) + } + + iOfDropIndex := -1 + for i := range tblInfo.indexes { + if jobArg.index.name == tblInfo.indexes[i].name { + iOfDropIndex = i + break + } + } + if iOfDropIndex == -1 { + return fmt.Errorf("table %s , index %s is not exists", tblInfo.name, jobArg.index.name) + } + + for _, column := range jobArg.index.columns { + column.indexReferences-- + if column.indexReferences < 0 { + return fmt.Errorf("drop index, index.column %s Unexpected index reference", column.name) + } + } + tblInfo.indexes = append(tblInfo.indexes[:iOfDropIndex], tblInfo.indexes[iOfDropIndex+1:]...) + return nil +} + +type ddlTestAddDropColumnStrategy = int + +const ( + ddlTestAddDropColumnStrategyBegin ddlTestAddDropColumnStrategy = iota + ddlTestAddDropColumnStrategyAtBeginning + ddlTestAddDropColumnStrategyAtEnd + ddlTestAddDropColumnStrategyAtRandom + ddlTestAddDropColumnStrategyEnd +) + +type ddlTestAddDropColumnConfig struct { + strategy ddlTestAddDropColumnStrategy +} + +type ddlColumnJobArg struct { + column *ddlTestColumn + strategy ddlTestAddDropColumnStrategy + insertAfterColumn *ddlTestColumn +} + +func (c *testCase) generateAddColumn() error { + c.ddlOps = append(c.ddlOps, ddlTestOpExecutor{c.prepareAddColumn, nil, ddlAddColumn}) + return nil +} + +func (c *testCase) prepareAddColumn(_ interface{}, taskCh chan *ddlJobTask) error { + table := c.pickupRandomTable() + if table == nil { + return nil + } + strategy := rand.Intn(ddlTestAddDropColumnStrategyAtRandom) + ddlTestAddDropColumnStrategyAtBeginning + newColumn := getRandDDLTestColumn() + insertAfterPosition := -1 + // build SQL + sql := fmt.Sprintf("ALTER TABLE `%s` ADD COLUMN `%s` %s", table.name, newColumn.name, newColumn.getDefinition()) + switch strategy { + case ddlTestAddDropColumnStrategyAtBeginning: + sql += " FIRST" + case ddlTestAddDropColumnStrategyAtEnd: + // do nothing + case ddlTestAddDropColumnStrategyAtRandom: + insertAfterPosition = rand.Intn(len(table.columns)) + sql += fmt.Sprintf(" AFTER `%s`", table.columns[insertAfterPosition].name) + } + + arg := &ddlColumnJobArg{ + column: newColumn, + strategy: strategy, + } + if insertAfterPosition != -1 { + arg.insertAfterColumn = table.columns[insertAfterPosition] + } + task := &ddlJobTask{ + k: ddlAddColumn, + sql: sql, + tblInfo: table, + arg: ddlJobArg(arg), + } + taskCh <- task + return nil +} + +func (c *testCase) addColumnJob(task *ddlJobTask) error { + jobArg := (*ddlColumnJobArg)(task.arg) + table := task.tblInfo + table.lock.Lock() + defer table.lock.Unlock() + + if c.isTableDeleted(table) { + return fmt.Errorf("table %s is not exists", table.name) + } + newColumn := jobArg.column + strategy := jobArg.strategy + + newColumn.rows = make([]interface{}, table.numberOfRows) + for i := 0; i < table.numberOfRows; i++ { + newColumn.rows[i] = newColumn.defaultValue + } + + switch strategy { + case ddlTestAddDropColumnStrategyAtBeginning: + table.columns = append([]*ddlTestColumn{newColumn}, table.columns...) + case ddlTestAddDropColumnStrategyAtEnd: + table.columns = append(table.columns, newColumn) + case ddlTestAddDropColumnStrategyAtRandom: + insertAfterPosition := -1 + for i := range table.columns { + if jobArg.insertAfterColumn.name == table.columns[i].name { + insertAfterPosition = i + break + } + } + if insertAfterPosition == -1 { + return fmt.Errorf("table %s ,insert column %s after column, column %s is not exists ", table.name, newColumn.name, jobArg.insertAfterColumn.name) + } + table.columns = append(table.columns[:insertAfterPosition+1], append([]*ddlTestColumn{newColumn}, table.columns[insertAfterPosition+1:]...)...) + } + return nil +} + +func (c *testCase) generateDropColumn() error { + c.ddlOps = append(c.ddlOps, ddlTestOpExecutor{c.prepareDropColumn, nil, ddlDropColumn}) + return nil +} + +func (c *testCase) prepareDropColumn(_ interface{}, taskCh chan *ddlJobTask) error { + table := c.pickupRandomTable() + if table == nil { + return nil + } + + columnsSnapshot := table.filterColumns(table.predicateAll) + if len(columnsSnapshot) <= 1 { + return nil + } + + strategy := rand.Intn(ddlTestAddDropColumnStrategyAtRandom) + ddlTestAddDropColumnStrategyAtBeginning + columnToDropIndex := -1 + switch strategy { + case ddlTestAddDropColumnStrategyAtBeginning: + columnToDropIndex = 0 + case ddlTestAddDropColumnStrategyAtEnd: + columnToDropIndex = len(table.columns) - 1 + case ddlTestAddDropColumnStrategyAtRandom: + columnToDropIndex = rand.Intn(len(table.columns)) + } + + columnToDrop := table.columns[columnToDropIndex] + + // primary key columns cannot be dropped + if columnToDrop.isPrimaryKey { + return nil + } + + // column cannot be dropped if the column has generated column dependency + if columnToDrop.hasGenerateCol() { + return nil + } + + // we does not support dropping a column with index + if columnToDrop.indexReferences > 0 { + return nil + } + columnToDrop.setDeleted() + sql := fmt.Sprintf("ALTER TABLE `%s` DROP COLUMN `%s`", table.name, columnToDrop.name) + + arg := &ddlColumnJobArg{ + column: columnToDrop, + strategy: strategy, + insertAfterColumn: nil, + } + task := &ddlJobTask{ + k: ddlDropColumn, + sql: sql, + tblInfo: table, + arg: ddlJobArg(arg), + } + taskCh <- task + return nil +} + +func (c *testCase) dropColumnJob(task *ddlJobTask) error { + jobArg := (*ddlColumnJobArg)(task.arg) + table := task.tblInfo + table.lock.Lock() + defer table.lock.Unlock() + if c.isTableDeleted(table) { + return fmt.Errorf("table %s is not exists", table.name) + } + columnToDrop := jobArg.column + if columnToDrop.indexReferences > 0 { + columnToDrop.setDeletedRecover() + return fmt.Errorf("local Execute drop column %s on table %s error , column has index reference", jobArg.column.name, table.name) + } + dropColumnPosition := -1 + for i := range table.columns { + if columnToDrop.name == table.columns[i].name { + dropColumnPosition = i + break + } + } + if dropColumnPosition == -1 { + return fmt.Errorf("table %s ,drop column , column %s is not exists ", table.name, columnToDrop.name) + } + // update table definitions + table.columns = append(table.columns[:dropColumnPosition], table.columns[dropColumnPosition+1:]...) + // if the drop column is a generated column , we should update the dependency column + if columnToDrop.isGenerated() { + col := columnToDrop.dependency + i := 0 + for i = range col.dependenciedCols { + if col.dependenciedCols[i].name == columnToDrop.name { + break + } + } + col.dependenciedCols = append(col.dependenciedCols[:i], col.dependenciedCols[i+1:]...) + } + return nil +} + +// getHistoryDDLJobs send "admin show ddl jobs" to TiDB to get ddl jobs execute order. +// Use TABLE_NAME or TABLE_ID, and JOB_TYPE to confirm which ddl job is the DDL request we send to TiDB. +// We cannot send the same DDL type to same table more than once in a batch of parallel DDL request. The reason is below: +// For example, execute SQL1: "ALTER TABLE t1 DROP COLUMN c1" , SQL2:"ALTER TABLE t1 DROP COLUMN c2", and the "admin show ddl jobs" result is: +// +--------+---------+------------+--------------+--------------+-----------+----------+-----------+-----------------------------------+--------+ +// | JOB_ID | DB_NAME | TABLE_NAME | JOB_TYPE | SCHEMA_STATE | SCHEMA_ID | TABLE_ID | ROW_COUNT | START_TIME | STATE | +// +--------+---------+------------+--------------+--------------+-----------+----------+-----------+-----------------------------------+--------+ +// | 47 | test | t1 | drop column | none | 1 | 44 | 0 | 2018-07-13 13:13:55.57 +0800 CST | synced | +// | 46 | test | t1 | drop column | none | 1 | 44 | 0 | 2018-07-13 13:13:52.523 +0800 CST | synced | +// +--------+---------+------------+--------------+--------------+-----------+----------+-----------+-----------------------------------+--------+ +// We cannot confirm which DDL execute first. +func (c *testCase) getHistoryDDLJobs(db *sql.DB, tasks []*ddlJobTask) ([]*ddlJob, error) { + // build SQL + sql := "admin show ddl jobs" + // execute + opStart := time.Now() + rows, err := db.Query(sql) + glog.Infof("%s, elapsed time:%v", sql, time.Since(opStart).Seconds()) + if err != nil { + return nil, err + } + defer rows.Close() + + jobs := make([]*ddlJob, 0, len(tasks)) + // Read all rows. + var actualRows [][]string + for rows.Next() { + cols, err1 := rows.Columns() + if err1 != nil { + return nil, err1 + } + + rawResult := make([][]byte, len(cols)) + result := make([]string, len(cols)) + dest := make([]interface{}, len(cols)) + for i := range rawResult { + dest[i] = &rawResult[i] + } + + err1 = rows.Scan(dest...) + if err1 != nil { + return nil, err1 + } + + for i, raw := range rawResult { + if raw == nil { + result[i] = "NULL" + } else { + val := string(raw) + result[i] = val + } + } + actualRows = append(actualRows, result) + } + if rows.Err() != nil { + return nil, rows.Err() + } + /********************************* + +--------+---------+--------------------------------------+--------------+--------------+-----------+----------+-----------+-----------------------------------+-----------+ + | JOB_ID | DB_NAME | TABLE_NAME | JOB_TYPE | SCHEMA_STATE | SCHEMA_ID | TABLE_ID | ROW_COUNT | START_TIME | STATE | + +--------+---------+--------------------------------------+--------------+--------------+-----------+----------+-----------+-----------------------------------+-----------+ + | 49519 | test | | add column | none | 49481 | 49511 | 0 | 2018-07-09 21:29:02.249 +0800 CST | cancelled | + | 49518 | test | | drop table | none | 49481 | 49511 | 0 | 2018-07-09 21:29:01.999 +0800 CST | synced | + | 49517 | test | ea5be232-50ce-43b1-8d40-33de2ae08bca | create table | public | 49481 | 49515 | 0 | 2018-07-09 21:29:01.999 +0800 CST | synced | + +--------+---------+--------------------------------------+--------------+--------------+-----------+----------+-----------+-----------------------------------+-----------+ + *********************************/ + for _, row := range actualRows { + if len(row) < 9 { + return nil, fmt.Errorf("%s return error, no enough column return , return row: %s", sql, row) + } + id, err := strconv.Atoi(row[0]) + if err != nil { + return nil, err + } + if id <= c.lastDDLID { + continue + } + k, ok := mapOfDDLKind[row[3]] + if !ok { + continue + } + job := ddlJob{ + id: id, + tableName: row[2], + k: k, + tableID: row[6], // table id + jobState: row[9], + } + jobs = append(jobs, &job) + } + return jobs, nil +} + +// getSortTask return the tasks sort by ddl JOB_ID +func (c *testCase) getSortTask(db *sql.DB, tasks []*ddlJobTask) ([]*ddlJobTask, error) { + jobs, err := c.getHistoryDDLJobs(db, tasks) + if err != nil { + return nil, err + } + sortTasks := make([]*ddlJobTask, 0, len(tasks)) + for _, job := range jobs { + for _, task := range tasks { + if task.k == ddlAddTable && job.k == ddlAddTable && task.tblInfo.name == job.tableName { + task.ddlID = job.id + task.tblInfo.id = job.tableID + sortTasks = append(sortTasks, task) + break + } + if task.k != ddlAddTable && job.k == task.k && task.tblInfo.id == job.tableID { + task.ddlID = job.id + sortTasks = append(sortTasks, task) + break + } + } + if len(sortTasks) == len(tasks) { + break + } + } + + if len(sortTasks) != len(tasks) { + str := "admin show ddl jobs len != len(tasks)\n" + str += "admin get job\n" + str += fmt.Sprintf("%v\t%v\t%v\t%v\t%v\n", "Job_ID", "TABLE_NAME", "JOB_TYPE", "TABLE_ID", "JOB_STATE") + for _, job := range jobs { + str += fmt.Sprintf("%v\t%v\t%v\t%v\t%v\n", job.id, job.tableName, mapOfDDLKindToString[job.k], job.tableID, job.jobState) + } + str += "ddl tasks\n" + str += fmt.Sprintf("%v\t%v\t%v\t%v\n", "Job_ID", "TABLE_NAME", "JOB_TYPE", "TABLE_ID") + for _, task := range tasks { + str += fmt.Sprintf("%v\t%v\t%v\t%v\n", task.ddlID, task.tblInfo.name, mapOfDDLKindToString[task.k], task.tblInfo.id) + } + + str += "ddl sort tasks\n" + str += fmt.Sprintf("%v\t%v\t%v\t%v\n", "Job_ID", "TABLE_NAME", "JOB_TYPE", "TABLE_ID") + for _, task := range sortTasks { + str += fmt.Sprintf("%v\t%v\t%v\t%v\n", task.ddlID, task.tblInfo.name, mapOfDDLKindToString[task.k], task.tblInfo.id) + } + return nil, fmt.Errorf(str) + } + + sort.Sort(ddlJobTasks(sortTasks)) + if len(sortTasks) > 0 { + c.lastDDLID = sortTasks[len(sortTasks)-1].ddlID + } + return sortTasks, nil +} + +type ddlJobTasks []*ddlJobTask + +func (tasks ddlJobTasks) Swap(i, j int) { + tasks[i], tasks[j] = tasks[j], tasks[i] +} + +func (tasks ddlJobTasks) Len() int { + return len(tasks) +} + +func (tasks ddlJobTasks) Less(i, j int) bool { + return tasks[i].ddlID < tasks[j].ddlID +} diff --git a/tests/pkg/workload/ddl/internal/demo.go b/tests/pkg/workload/ddl/internal/demo.go new file mode 100644 index 0000000000..d41f4a3588 --- /dev/null +++ b/tests/pkg/workload/ddl/internal/demo.go @@ -0,0 +1,86 @@ +// Copyright 2019 PingCAP, Inc. +// +// 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, +// See the License for the specific language governing permissions and +// limitations under the License. +package internal + +//import ( +// "database/sql" +// "sync" +//) +// +//func OpenDB(dsn string, maxIdleConns int) (*sql.DB, error) { panic("unimplemented") } +// +//func Run(dbName string) { +// dbss := make([][]*sql.DB, 0) +//} +// +//type DDLCaseConfig struct { +// MySQLCompatible bool +// Concurrency int +// TablesToCreate int +//} +// +//type DDLCase struct { +// cfg *DDLCaseConfig +// cases []*testCase +//} +// +//type ddlJobTask struct { +// ddlID int +// k DDLKind +// tblInfo *ddlTestTable +// sql string +// arg ddlJobArg +// err error // remote TiDB execute error +//} +// +//type ddlTestOpExecutor struct { +// executeFunc func(interface{}, chan *ddlJobTask) error +// config interface{} +// ddlKind DDLKind +//} +// +//type dmlTestOpExecutor struct { +// prepareFunc func(interface{}, chan *dmlJobTask) error +// config interface{} +//} +// +//type testCase struct { +// cfg *DDLCaseConfig +// dbs []*sql.DB +// caseIndex int +// ddlOps []ddlTestOpExecutor +// dmlOps []dmlTestOpExecutor +// tables map[string]*ddlTestTable +// tablesLock sync.RWMutex +// stop int32 +// lastDDLID int +//} +// +//func NewDDLCase(cfg *DDLCaseConfig) *DDLCase { +// cases := make([]*testCase, cfg.Concurrency) +// for i := 0; i < cfg.Concurrency; i++ { +// cases[i] = &testCase{ +// cfg: cfg, +// tables: make(map[string]*ddlTestTable), +// ddlOps: make([]ddlTestOpExecutor, 0), +// dmlOps: make([]dmlTestOpExecutor, 0), +// caseIndex: i, +// stop: 0, +// } +// } +// b := &DDLCase{ +// cfg: cfg, +// cases: cases, +// } +// return b +//} diff --git a/tests/pkg/workload/ddl/internal/dml_ops.go b/tests/pkg/workload/ddl/internal/dml_ops.go new file mode 100644 index 0000000000..dfa3488018 --- /dev/null +++ b/tests/pkg/workload/ddl/internal/dml_ops.go @@ -0,0 +1,620 @@ +// Copyright 2019 PingCAP, Inc. +// +// 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, +// See the License for the specific language governing permissions and +// limitations under the License. +package internal + +import ( + "context" + "database/sql" + "fmt" + "math/rand" + + "github.com/golang/glog" + "github.com/juju/errors" +) + +func (c *testCase) generateDMLOps() error { + if err := c.generateInsert(); err != nil { + return errors.Trace(err) + } + if err := c.generateUpdate(); err != nil { + return errors.Trace(err) + } + if err := c.generateDelete(); err != nil { + return errors.Trace(err) + } + return nil +} + +type ddlTestInsertColumnStrategy int +type ddlTestInsertMissingValueStrategy int + +const ( + ddlTestInsertColumnStrategyBegin ddlTestInsertColumnStrategy = iota + ddlTestInsertColumnStrategyZeroNonPk + ddlTestInsertColumnStrategyAllNonPk + ddlTestInsertColumnStrategyRandomNonPk + ddlTestInsertColumnStrategyEnd +) + +const ( + ddlTestInsertMissingValueStrategyBegin ddlTestInsertMissingValueStrategy = iota + ddlTestInsertMissingValueStrategyAllNull + ddlTestInsertMissingValueStrategyAllDefault + ddlTestInsertMissingValueStrategyRandom + ddlTestInsertMissingValueStrategyEnd +) + +type ddlTestInsertConfig struct { + useSetStatement bool // whether to use SET or VALUE statement + columnStrategy ddlTestInsertColumnStrategy // how non-Primary-Key columns are picked + missingValueStrategy ddlTestInsertMissingValueStrategy // how columns are filled when they are not picked in VALUE statement +} + +func checkConflict(task *dmlJobTask) error { + if task.tblInfo.isDeleted() { + return ddlTestErrorConflict{} + } + if task.assigns != nil { + for _, cd := range task.assigns { + if cd.column.isDeleted() { + return ddlTestErrorConflict{} + } + } + } + if task.whereColumns != nil { + for _, cd := range task.whereColumns { + if cd.column.isDeleted() { + return ddlTestErrorConflict{} + } + } + } + return nil +} + +func (c *testCase) sendDMLRequest(ctx context.Context, conn *sql.Conn, task *dmlJobTask) error { + _, err := conn.ExecContext(ctx, task.sql) + task.err = err + glog.Infof("[ddl] [instance %d] %s, err: %v", c.caseIndex, task.sql, err) + if err != nil { + err2 := checkConflict(task) + if err2 != nil { + return nil + } + return errors.Annotatef(err, "Error when executing SQL: %s\n%s", task.sql, task.tblInfo.debugPrintToString()) + } + return nil +} + +func (c *testCase) execDMLInLocal(task *dmlJobTask) error { + switch task.k { + case dmlInsert: + return c.doInsertJob(task) + case dmlUpdate: + return c.doUpdateJob(task) + case dmlDelete: + return c.doDeleteJob(task) + } + return fmt.Errorf("unknow dml task , %v", *task) +} + +// execSerialDDLSQL gets a job from taskCh, and then executes the job. +func (c *testCase) execSerialDMLSQL(taskCh chan *dmlJobTask) error { + ctx := context.Background() + dbIdx := rand.Intn(len(c.dbs)) + db := c.dbs[dbIdx] + conn, err := db.Conn(ctx) + if err != nil { + return nil + } + defer conn.Close() + for task := range taskCh { + err := c.sendDMLRequest(ctx, conn, task) + if err != nil { + if ignore_error(err) { + continue + } + return errors.Trace(err) + } + if task.err != nil { + continue + } + err = c.execDMLInLocal(task) + if err != nil { + return fmt.Errorf("Error when executing SQL: %s\n local Err: %#v\n%s\n", task.sql, err, task.tblInfo.debugPrintToString()) + } + } + return nil +} + +// execSerialDDLSQL gets a job from taskCh, and then executes the job. +func (c *testCase) execDMLInTransactionSQL(taskCh chan *dmlJobTask) error { + tasksLen := len(taskCh) + + ctx := context.Background() + conn, err := c.dbs[1].Conn(ctx) + if err != nil { + return nil + } + defer conn.Close() + + _, err = conn.ExecContext(ctx, "begin") + glog.Infof("[ddl] [instance %d] begin error: %v", c.caseIndex, err) + if err != nil { + return errors.Annotatef(err, "Error when executing SQL: %s", "begin") + } + + tasks := make([]*dmlJobTask, 0, tasksLen) + for i := 0; i < tasksLen; i++ { + task := <-taskCh + err = c.sendDMLRequest(ctx, conn, task) + tasks = append(tasks, task) + } + + _, err = conn.ExecContext(ctx, "commit") + glog.Infof("[ddl] [instance %d] commit error: %v", c.caseIndex, err) + if err != nil { + if ignore_error(err) { + return nil + } + for i := 0; i < tasksLen; i++ { + task := tasks[i] + // no conflict when send request but conflict when commit + if task.err == nil && checkConflict(task) != nil { + return nil + } + } + return errors.Annotatef(err, "Error when executing SQL: %s", "commit") + } + + for i := 0; i < tasksLen; i++ { + task := tasks[i] + if task.err != nil { + continue + } + err = c.execDMLInLocal(task) + if err != nil { + return fmt.Errorf("Error when executing SQL: %s\n local Err: %#v\n%s\n", task.sql, err, task.tblInfo.debugPrintToString()) + } + } + return nil +} + +func (c *testCase) generateInsert() error { + for i := 0; i < 5; i++ { + for columnStrategy := ddlTestInsertColumnStrategyBegin + 1; columnStrategy < ddlTestInsertColumnStrategyEnd; columnStrategy++ { + // Note: `useSetStatement` is commented out since `... VALUES ...` SQL will generates column conflicts with add / drop column. + // We always use `... SET ...` syntax currently. + + // for useSetStatement := 0; useSetStatement < 2; useSetStatement++ { + config := ddlTestInsertConfig{ + useSetStatement: true, // !(useSetStatement == 0), + columnStrategy: columnStrategy, + } + // if config.useSetStatement { + c.dmlOps = append(c.dmlOps, dmlTestOpExecutor{c.prepareInsert, config}) + // } else { + // for missingValueStrategy := ddlTestInsertMissingValueStrategyBegin + 1; missingValueStrategy < ddlTestInsertMissingValueStrategyEnd; missingValueStrategy++ { + // config.missingValueStrategy = missingValueStrategy + // c.dmlOps = append(c.dmlOps, ddlTestOpExecutor{c.executeInsert, config}) + // } + // } + // } + } + } + return nil +} + +func (c *testCase) prepareInsert(cfg interface{}, taskCh chan *dmlJobTask) error { + c.tablesLock.RLock() + table := c.pickupRandomTable() + if table == nil { + c.tablesLock.RUnlock() + return nil + } + table.lock.RLock() + columns := table.filterColumns(table.predicateNotGenerated) + nonPkColumns := table.filterColumns(table.predicateNonPrimaryKeyAndNotGen) + table.lock.RUnlock() + c.tablesLock.RUnlock() + + config := cfg.(ddlTestInsertConfig) + + // build assignments + assigns := make([]*ddlTestColumnDescriptor, 0) + for _, column := range columns { + pick := false + if column.isPrimaryKey { + // PrimaryKey Column is always assigned values + pick = true + } else { + // NonPrimaryKey Column is assigned by strategy + switch config.columnStrategy { + case ddlTestInsertColumnStrategyAllNonPk: + pick = true + case ddlTestInsertColumnStrategyZeroNonPk: + pick = false + case ddlTestInsertColumnStrategyRandomNonPk: + if rand.Float64() <= float64(1)/float64(len(nonPkColumns)) { + pick = true + } + } + } + if pick { + // check unique value when inserting into a column of primary key + if column.isPrimaryKey { + if newValue, ok := column.randValueUnique(column.rows); ok { + assigns = append(assigns, &ddlTestColumnDescriptor{column, newValue}) + } else { + return nil + } + } else { + assigns = append(assigns, &ddlTestColumnDescriptor{column, column.randValue()}) + } + } + } + + // build SQL + sql := "" + if config.useSetStatement { + if len(assigns) == 0 { + return nil + } + sql = fmt.Sprintf("INSERT INTO `%s` SET ", table.name) + perm := rand.Perm(len(assigns)) + for i, idx := range perm { + assign := assigns[idx] + if i > 0 { + sql += ", " + } + sql += fmt.Sprintf("`%s` = %v", assign.column.name, assign.getValueString()) + } + } else { + sql = fmt.Sprintf("INSERT INTO `%s` VALUE (", table.name) + for colIdx, column := range columns { + if colIdx > 0 { + sql += ", " + } + cd := column.getMatchedColumnDescriptor(assigns) + if cd != nil { + sql += fmt.Sprintf("%v", cd.getValueString()) + } else { + var missingValueSQL string + switch config.missingValueStrategy { + case ddlTestInsertMissingValueStrategyAllDefault: + missingValueSQL = "DEFAULT" + case ddlTestInsertMissingValueStrategyAllNull: + missingValueSQL = "NULL" + case ddlTestInsertMissingValueStrategyRandom: + if rand.Float64() <= 0.5 { + missingValueSQL = "DEFAULT" + } else { + missingValueSQL = "NULL" + } + } + sql += missingValueSQL + var missingValue interface{} + if missingValueSQL == "DEFAULT" { + missingValue = column.defaultValue + } else if missingValueSQL == "NULL" { + missingValue = ddlTestValueNull + } else { + panic("invalid missing value") + } + // add column to ref list + assigns = append(assigns, &ddlTestColumnDescriptor{column, missingValue}) + } + } + sql += ")" + } + + task := &dmlJobTask{ + k: dmlInsert, + sql: sql, + tblInfo: table, + assigns: assigns, + } + taskCh <- task + return nil +} + +func (c *testCase) doInsertJob(task *dmlJobTask) error { + table := task.tblInfo + assigns := task.assigns + + // append row + table.lock.Lock() + for _, column := range table.columns { + cd := column.getMatchedColumnDescriptor(assigns) + if cd == nil { + if column.isGenerated() { + cd = column.dependency.getMatchedColumnDescriptor(assigns) + if cd == nil { + column.rows = append(column.rows, nil) + } else { + column.rows = append(column.rows, cd.column.getDependenciedColsValue(column)) + } + } else { + // only happens when using SET + column.rows = append(column.rows, column.defaultValue) + } + } else { + column.rows = append(column.rows, cd.value) + } + } + table.numberOfRows++ + table.lock.Unlock() + return nil +} + +type ddlTestWhereStrategy int + +const ( + ddlTestWhereStrategyBegin ddlTestWhereStrategy = iota + ddlTestWhereStrategyNone + ddlTestWhereStrategyRandomInPk + ddlTestWhereStrategyRandomInNonPk + ddlTestWhereStrategyRandomMixed + ddlTestWhereStrategyEnd +) + +type ddlTestUpdateTargetStrategy int + +const ( + ddlTestUpdateTargetStrategyBegin ddlTestUpdateTargetStrategy = iota + ddlTestUpdateTargetStrategyAllColumns + ddlTestUpdateTargetStrategyRandom + ddlTestUpdateTargetStrategyEnd +) + +type ddlTestUpdateConfig struct { + whereStrategy ddlTestWhereStrategy // how "where" statement is generated + targetStrategy ddlTestUpdateTargetStrategy // which column to update +} + +func (c *testCase) generateUpdate() error { + for i := 0; i < 5; i++ { + for whereStrategy := ddlTestWhereStrategyBegin + 1; whereStrategy < ddlTestWhereStrategyEnd; whereStrategy++ { + for targetStrategy := ddlTestUpdateTargetStrategyBegin + 1; targetStrategy < ddlTestUpdateTargetStrategyEnd; targetStrategy++ { + config := ddlTestUpdateConfig{ + whereStrategy: whereStrategy, + targetStrategy: targetStrategy, + } + c.dmlOps = append(c.dmlOps, dmlTestOpExecutor{c.prepareUpdate, config}) + } + } + } + return nil +} + +func (c *testCase) buildWhereColumns(whereStrategy ddlTestWhereStrategy, pkColumns, nonPkColumns []*ddlTestColumn, numberOfRows int) []*ddlTestColumnDescriptor { + // build where conditions + whereColumns := make([]*ddlTestColumnDescriptor, 0) + if whereStrategy == ddlTestWhereStrategyRandomInPk || whereStrategy == ddlTestWhereStrategyRandomMixed { + if len(pkColumns) > 0 { + picks := rand.Intn(len(pkColumns)) + perm := rand.Perm(picks) + for _, idx := range perm { + // value will be filled later + whereColumns = append(whereColumns, &ddlTestColumnDescriptor{pkColumns[idx], -1}) + } + } + } + if whereStrategy == ddlTestWhereStrategyRandomInNonPk || whereStrategy == ddlTestWhereStrategyRandomMixed { + if len(nonPkColumns) > 0 { + picks := rand.Intn(len(nonPkColumns)) + perm := rand.Perm(picks) + for _, idx := range perm { + // value will be filled later + whereColumns = append(whereColumns, &ddlTestColumnDescriptor{nonPkColumns[idx], -1}) + } + } + } + + // fill values of where statements + if len(whereColumns) > 0 { + rowToUpdate := rand.Intn(numberOfRows) + for _, cd := range whereColumns { + cd.value = cd.column.rows[rowToUpdate] + } + } + + return whereColumns +} + +func (c *testCase) prepareUpdate(cfg interface{}, taskCh chan *dmlJobTask) error { + c.tablesLock.RLock() + table := c.pickupRandomTable() + if table == nil { + c.tablesLock.RUnlock() + return nil + } + table.lock.RLock() + pkColumns := table.filterColumns(table.predicatePrimaryKey) + nonPkColumnsAndCanBeWhere := table.filterColumns(table.predicateNonPrimaryKeyAndCanBeWhere) + nonPkColumnsAndNotGen := table.filterColumns(table.predicateNonPrimaryKeyAndNotGen) + table.lock.RUnlock() + c.tablesLock.RUnlock() + + if table.numberOfRows == 0 { + return nil + } + + config := cfg.(ddlTestUpdateConfig) + + // build where conditions + whereColumns := c.buildWhereColumns(config.whereStrategy, pkColumns, nonPkColumnsAndCanBeWhere, table.numberOfRows) + + // build assignments + assigns := make([]*ddlTestColumnDescriptor, 0) + picks := 0 + switch config.targetStrategy { + case ddlTestUpdateTargetStrategyRandom: + if len(nonPkColumnsAndNotGen) > 0 { + picks = rand.Intn(len(nonPkColumnsAndNotGen)) + } + case ddlTestUpdateTargetStrategyAllColumns: + picks = len(nonPkColumnsAndNotGen) + } + if picks == 0 { + return nil + } + perm := rand.Perm(picks) + for _, idx := range perm { + assigns = append(assigns, &ddlTestColumnDescriptor{nonPkColumnsAndNotGen[idx], nonPkColumnsAndNotGen[idx].randValue()}) + } + + // build SQL + sql := fmt.Sprintf("UPDATE `%s` SET ", table.name) + for i, cd := range assigns { + if i > 0 { + sql += ", " + } + sql += fmt.Sprintf("`%s` = %v", cd.column.name, cd.getValueString()) + } + if len(whereColumns) > 0 { + sql += " WHERE " + for i, cd := range whereColumns { + if i > 0 { + sql += " AND " + } + sql += cd.buildConditionSQL() + } + } + + task := &dmlJobTask{ + k: dmlUpdate, + tblInfo: table, + sql: sql, + assigns: assigns, + whereColumns: whereColumns, + } + + taskCh <- task + return nil +} + +func (c *testCase) doUpdateJob(task *dmlJobTask) error { + table := task.tblInfo + assigns := task.assigns + whereColumns := task.whereColumns + + // update values + table.lock.RLock() + for i := 0; i < table.numberOfRows; i++ { + match := true + for _, cd := range whereColumns { + if cd.value != cd.column.rows[i] { + match = false + break + } + } + if match { + for _, cd := range assigns { + cd.column.rows[i] = cd.value + if cd.column.hasGenerateCol() { + for _, col := range cd.column.dependenciedCols { + col.rows[i] = cd.column.getDependenciedColsValue(col) + } + } + } + } + } + table.lock.RUnlock() + return nil + +} + +type ddlTestDeleteConfig struct { + whereStrategy ddlTestWhereStrategy // how "where" statement is generated +} + +func (c *testCase) generateDelete() error { + for i := 0; i < 5; i++ { + for whereStrategy := ddlTestWhereStrategyBegin + 1; whereStrategy < ddlTestWhereStrategyEnd; whereStrategy++ { + config := ddlTestDeleteConfig{ + whereStrategy: whereStrategy, + } + c.dmlOps = append(c.dmlOps, dmlTestOpExecutor{c.prepareDelete, config}) + } + } + return nil +} + +func (c *testCase) prepareDelete(cfg interface{}, taskCh chan *dmlJobTask) error { + c.tablesLock.RLock() + table := c.pickupRandomTable() + if table == nil { + c.tablesLock.RUnlock() + return nil + } + table.lock.RLock() + pkColumns := table.filterColumns(table.predicatePrimaryKey) + nonPkColumnsAndCanBeWhere := table.filterColumns(table.predicateNonPrimaryKeyAndCanBeWhere) + table.lock.RUnlock() + c.tablesLock.RUnlock() + + if table.numberOfRows == 0 { + return nil + } + + config := cfg.(ddlTestDeleteConfig) + whereColumns := c.buildWhereColumns(config.whereStrategy, pkColumns, nonPkColumnsAndCanBeWhere, table.numberOfRows) + + // build SQL + sql := fmt.Sprintf("DELETE FROM `%s`", table.name) + if len(whereColumns) > 0 { + sql += " WHERE " + for i, cd := range whereColumns { + if i > 0 { + sql += " AND " + } + sql += cd.buildConditionSQL() + } + } + + task := &dmlJobTask{ + k: dmlDelete, + tblInfo: table, + sql: sql, + whereColumns: whereColumns, + } + taskCh <- task + return nil +} + +func (c *testCase) doDeleteJob(task *dmlJobTask) error { + table := task.tblInfo + whereColumns := task.whereColumns + + // update values + table.lock.Lock() + for i := table.numberOfRows - 1; i >= 0; i-- { + match := true + for _, cd := range whereColumns { + if cd.value != cd.column.rows[i] { + match = false + break + } + } + if match { + // we must use `table.columns` here, since there might be new columns after deletion + for _, column := range table.columns { + column.rows = append(column.rows[:i], column.rows[i+1:]...) + } + table.numberOfRows-- + } + } + table.lock.Unlock() + + return nil +} diff --git a/tests/pkg/workload/ddl/internal/meta.go b/tests/pkg/workload/ddl/internal/meta.go new file mode 100644 index 0000000000..9a95d02e37 --- /dev/null +++ b/tests/pkg/workload/ddl/internal/meta.go @@ -0,0 +1,636 @@ +// Copyright 2019 PingCAP, Inc. +// +// 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, +// See the License for the specific language governing permissions and +// limitations under the License. +package internal + +import ( + "bytes" + "database/sql" + "encoding/json" + "fmt" + "math/rand" + "sort" + "strings" + "sync" + "sync/atomic" + "time" + + "k8s.io/apimachinery/pkg/util/uuid" +) + +type testCase struct { + cfg *DDLCaseConfig + dbs []*sql.DB + caseIndex int + ddlOps []ddlTestOpExecutor + dmlOps []dmlTestOpExecutor + tables map[string]*ddlTestTable + tablesLock sync.RWMutex + stop int32 + lastDDLID int +} + +type ddlTestErrorConflict struct { +} + +func (err ddlTestErrorConflict) Error() string { + return "Conflict operation" +} + +func (c *testCase) stopTest() { + atomic.StoreInt32(&c.stop, 1) +} + +func (c *testCase) isStop() bool { + return atomic.LoadInt32(&c.stop) == 1 +} + +// pickupRandomTables picks a table randomly. The callee should ensure that +// during this function call the table list is not modified. +// +// Normally the DML op callee should acquire a lock before calling this function +// because the table list may be modified by another parallel DDL op. However +// the DDL op callee doesn't need to acquire a lock because no one will modify the +// table list in parallel ---- DDL ops are executed one by one. +func (c *testCase) pickupRandomTable() *ddlTestTable { + tableNames := make([]string, 0) + for name, table := range c.tables { + if table.isDeleted() { + continue + } + tableNames = append(tableNames, name) + } + if len(tableNames) == 0 { + return nil + } + name := tableNames[rand.Intn(len(tableNames))] + return c.tables[name] +} + +func (c *testCase) isTableDeleted(table *ddlTestTable) bool { + if _, ok := c.tables[table.name]; ok { + return false + } + return true +} + +type ddlTestTable struct { + deleted int32 + name string + id string // table_id , get from admin show ddl jobs + columns []*ddlTestColumn + indexes []*ddlTestIndex + numberOfRows int + lock sync.RWMutex +} + +func (table *ddlTestTable) isDeleted() bool { + return atomic.LoadInt32(&table.deleted) != 0 +} + +func (table *ddlTestTable) setDeleted() { + atomic.StoreInt32(&table.deleted, 1) +} + +func (table *ddlTestTable) filterColumns(predicate func(*ddlTestColumn) bool) []*ddlTestColumn { + retColumns := make([]*ddlTestColumn, 0) + for index, col := range table.columns { + if predicate(col) && !col.isDeleted() { + retColumns = append(retColumns, table.columns[index]) + } + } + return retColumns +} + +func (table *ddlTestTable) predicateAll(col *ddlTestColumn) bool { + return true +} + +func (table *ddlTestTable) predicateNotGenerated(col *ddlTestColumn) bool { + return col.notGenerated() +} + +func (table *ddlTestTable) predicatePrimaryKey(col *ddlTestColumn) bool { + return col.isPrimaryKey +} + +func (table *ddlTestTable) predicateNonPrimaryKey(col *ddlTestColumn) bool { + return !col.isPrimaryKey +} + +func (table *ddlTestTable) predicateNonPrimaryKeyAndCanBeWhere(col *ddlTestColumn) bool { + return !col.isPrimaryKey && col.canBeWhere() +} + +func (table *ddlTestTable) predicateNonPrimaryKeyAndNotGen(col *ddlTestColumn) bool { + return !col.isPrimaryKey && col.notGenerated() +} + +// isColumnDeleted checks the col is deleted in this table +// col.isDeleted() will be true before when dropColumnJob(), +// but the col is really deleted after remote TiDB successful execute drop column ddl, and then, the col will be deleted from table.columns. +func (table *ddlTestTable) isColumnDeleted(col *ddlTestColumn) bool { + for i := range table.columns { + if col.name == table.columns[i].name { + return false + } + } + return true +} + +func (table *ddlTestTable) debugPrintToString() string { + var buffer bytes.Buffer + table.lock.RLock() + buffer.WriteString(fmt.Sprintf("======== DEBUG BEGIN ========\n")) + buffer.WriteString(fmt.Sprintf("Dumping expected contents for table `%s`:\n", table.name)) + if table.isDeleted() { + buffer.WriteString("[WARN] This table is marked as DELETED.\n") + } + buffer.WriteString("## Non-Primary Indexes: \n") + for i, index := range table.indexes { + buffer.WriteString(fmt.Sprintf("Index #%d: Name = `%s`, Columnns = [", i, index.name)) + for _, column := range index.columns { + buffer.WriteString(fmt.Sprintf("`%s`, ", column.name)) + } + buffer.WriteString("]\n") + } + buffer.WriteString("## Columns: \n") + for i, column := range table.columns { + buffer.WriteString(fmt.Sprintf("Column #%d", i)) + if column.isDeleted() { + buffer.WriteString(" [DELETED]") + } + buffer.WriteString(fmt.Sprintf(": Name = `%s`, Definition = %s, isPrimaryKey = %v, used in %d indexes\n", + column.name, column.getDefinition(), column.isPrimaryKey, column.indexReferences)) + } + buffer.WriteString(fmt.Sprintf("## Values (number of rows = %d): \n", table.numberOfRows)) + for i := 0; i < table.numberOfRows; i++ { + buffer.WriteString("#") + buffer.WriteString(padRight(fmt.Sprintf("%d", i), " ", 4)) + buffer.WriteString(": ") + for _, col := range table.columns { + buffer.WriteString(padLeft(fmt.Sprintf("%v", col.rows[i]), " ", 11)) + buffer.WriteString(", ") + } + buffer.WriteString("\n") + } + buffer.WriteString("======== DEBUG END ========\n") + table.lock.RUnlock() + return buffer.String() +} + +type ddlTestColumnDescriptor struct { + column *ddlTestColumn + value interface{} +} + +func (ddlt *ddlTestColumnDescriptor) getValueString() string { + // make bit data visible + if ddlt.column.k == KindBit { + return fmt.Sprintf("b'%v'", ddlt.value) + } else { + return fmt.Sprintf("'%v'", ddlt.value) + } +} + +func (ddlt *ddlTestColumnDescriptor) buildConditionSQL() string { + var sql string + if ddlt.value == ddlTestValueNull || ddlt.value == nil { + sql += fmt.Sprintf("`%s` IS NULL", ddlt.column.name) + } else { + switch ddlt.column.k { + case KindFloat: + sql += fmt.Sprintf("abs(`%s` - %v) < 0.0000001", ddlt.column.name, ddlt.getValueString()) + case KindDouble: + sql += fmt.Sprintf("abs(`%s` - %v) < 0.0000000000000001", ddlt.column.name, ddlt.getValueString()) + default: + sql += fmt.Sprintf("`%s` = %v", ddlt.column.name, ddlt.getValueString()) + } + } + return sql +} + +type ddlTestColumn struct { + k int + deleted int32 + name string + fieldType string + + filedTypeM int //such as: VARCHAR(10) , filedTypeM = 10 + filedTypeD int //such as: DECIMAL(10,5) , filedTypeD = 5 + filedPrecision int + defaultValue interface{} + isPrimaryKey bool + rows []interface{} + indexReferences int + + dependenciedCols []*ddlTestColumn + dependency *ddlTestColumn + mValue map[string]interface{} + nameOfGen string + + setValue []string //for enum , set data type +} + +func (col *ddlTestColumn) isDeleted() bool { + return atomic.LoadInt32(&col.deleted) != 0 +} + +func (col *ddlTestColumn) setDeleted() { + atomic.StoreInt32(&col.deleted, 1) +} + +func (col *ddlTestColumn) setDeletedRecover() { + atomic.StoreInt32(&col.deleted, 0) +} + +func (col *ddlTestColumn) getMatchedColumnDescriptor(descriptors []*ddlTestColumnDescriptor) *ddlTestColumnDescriptor { + for _, d := range descriptors { + if d.column == col { + return d + } + } + return nil +} + +func (col *ddlTestColumn) getDefinition() string { + if col.isPrimaryKey { + return col.fieldType + } + + if col.isGenerated() { + return fmt.Sprintf("%s AS (JSON_EXTRACT(`%s`,'$.%s'))", col.fieldType, col.dependency.name, col.nameOfGen) + } + + if col.canHaveDefaultValue() { + return fmt.Sprintf("%s NULL DEFAULT %v", col.fieldType, col.getDefaultValueString()) + } else { + return fmt.Sprintf("%s NULL", col.fieldType) + } + +} + +func (col *ddlTestColumn) getSelectName() string { + if col.k == KindBit { + return fmt.Sprintf("bin(`%s`)", col.name) + } else { + return fmt.Sprintf("`%s`", col.name) + } +} + +func (col *ddlTestColumn) getDefaultValueString() string { + if col.k == KindBit { + return fmt.Sprintf("b'%v'", col.defaultValue) + } else { + return fmt.Sprintf("'%v'", col.defaultValue) + } +} + +func (col *ddlTestColumn) isEqual(r int, str string) bool { + vstr := fmt.Sprintf("%v", col.rows[r]) + return strings.Compare(vstr, str) == 0 +} + +func (col *ddlTestColumn) getDependenciedColsValue(genCol *ddlTestColumn) interface{} { + if col.mValue == nil { + return nil + } + v := col.mValue[genCol.nameOfGen] + switch genCol.k { + case KindChar, KindVarChar, KindTEXT, KindBLOB: + v = fmt.Sprintf("\"%v\"", v) + } + return v +} + +func getDDLTestColumn(n int) *ddlTestColumn { + column := &ddlTestColumn{ + k: n, + name: string(uuid.NewUUID()), + fieldType: ALLFieldType[n], + rows: make([]interface{}, 0), + deleted: 0, + } + switch n { + case KindChar, KindVarChar, KindBLOB, KindTEXT, KindBit: + maxLen := getMaxLenByKind(n) + column.filedTypeM = int(rand.Intn(maxLen)) + for column.filedTypeM == 0 && column.k == KindBit { + column.filedTypeM = int(rand.Intn(maxLen)) + } + + for column.filedTypeM < 3 && column.k != KindBit { // len('""') = 2 + column.filedTypeM = int(rand.Intn(maxLen)) + } + column.fieldType = fmt.Sprintf("%s(%d)", ALLFieldType[n], column.filedTypeM) + case KindTINYBLOB, KindMEDIUMBLOB, KindLONGBLOB, KindTINYTEXT, KindMEDIUMTEXT, KindLONGTEXT: + column.filedTypeM = getMaxLenByKind(n) + case KindDECIMAL: + column.filedTypeM, column.filedTypeD = randMD() + column.fieldType = fmt.Sprintf("%s(%d,%d)", ALLFieldType[n], column.filedTypeM, column.filedTypeD) + case KindEnum, KindSet: + maxLen := getMaxLenByKind(n) + l := maxLen + 1 + column.setValue = make([]string, l) + m := make(map[string]struct{}) + column.fieldType += "(" + for i := 0; i < l; i++ { + column.setValue[i] = randEnumString(m) + if i > 0 { + column.fieldType += ", " + } + column.fieldType += fmt.Sprintf("\"%s\"", column.setValue[i]) + } + column.fieldType += ")" + } + + if column.canHaveDefaultValue() { + column.defaultValue = column.randValue() + } + + return column +} + +func getRandDDLTestColumn() *ddlTestColumn { + var n int + for { + n = randDataType() + if n != KindJSON { + break + } + } + return getDDLTestColumn(n) +} + +func getRandDDLTestColumnForJson() *ddlTestColumn { + var n int + for { + n = randDataType() + if n != KindJSON && n != KindBit && n != KindSet && n != KindEnum { + break + } + } + return getDDLTestColumn(n) +} + +func getRandDDLTestColumns() []*ddlTestColumn { + n := randDataType() + cols := make([]*ddlTestColumn, 0) + + if n == KindJSON { + cols = getRandJsonCol() + } else { + column := getDDLTestColumn(n) + cols = append(cols, column) + } + return cols +} + +const JsonFieldNum = 5 + +func getRandJsonCol() []*ddlTestColumn { + fieldNum := rand.Intn(JsonFieldNum) + 1 + + cols := make([]*ddlTestColumn, 0, fieldNum+1) + + column := &ddlTestColumn{ + k: KindJSON, + name: string(uuid.NewUUID()), + fieldType: ALLFieldType[KindJSON], + rows: make([]interface{}, 0), + deleted: 0, + + dependenciedCols: make([]*ddlTestColumn, 0, fieldNum), + } + + m := make(map[string]interface{}, 0) + for i := 0; i < fieldNum; i++ { + col := getRandDDLTestColumnForJson() + col.nameOfGen = randFieldName(m) + m[col.nameOfGen] = col.randValue() + col.dependency = column + + column.dependenciedCols = append(column.dependenciedCols, col) + cols = append(cols, col) + } + column.mValue = m + cols = append(cols, column) + return cols +} + +func (col *ddlTestColumn) isGenerated() bool { + return col.dependency != nil +} + +func (col *ddlTestColumn) notGenerated() bool { + return col.dependency == nil +} + +func (col *ddlTestColumn) hasGenerateCol() bool { + return len(col.dependenciedCols) > 0 +} + +// randValue return a rand value of the column +func (col *ddlTestColumn) randValue() interface{} { + switch col.k { + case KindTINYINT: + return rand.Int31n(1<<8) - 1<<7 + case KindSMALLINT: + return rand.Int31n(1<<16) - 1<<15 + case KindMEDIUMINT: + return rand.Int31n(1<<24) - 1<<23 + case KindInt32: + return rand.Int63n(1<<32) - 1<<31 + case KindBigInt: + if rand.Intn(2) == 1 { + return rand.Int63() + } + return -1 - rand.Int63() + case KindBit: + if col.filedTypeM >= 64 { + return fmt.Sprintf("%b", rand.Uint64()) + } else { + m := col.filedTypeM + if col.filedTypeM > 7 { // it is a bug + m = m - 1 + } + n := (int64)((1 << (uint)(m)) - 1) + return fmt.Sprintf("%b", rand.Int63n(n)) + } + case KindFloat: + return rand.Float32() + 1 + case KindDouble: + return rand.Float64() + 1 + case KindDECIMAL: + return randDecimal(col.filedTypeM, col.filedTypeD) + case KindChar, KindVarChar, KindBLOB, KindTINYBLOB, KindMEDIUMBLOB, KindLONGBLOB, KindTEXT, KindTINYTEXT, KindMEDIUMTEXT, KindLONGTEXT: + if col.filedTypeM == 0 { + return "" + } else { + if col.isGenerated() { + if col.filedTypeM <= 2 { + return "" + } + return randSeq(rand.Intn(col.filedTypeM - 2)) + } + return randSeq(rand.Intn(col.filedTypeM)) + } + case KindBool: + return rand.Intn(2) + case KindDATE: + randTime := time.Unix(MinDATETIME.Unix()+rand.Int63n(GapDATETIMEUnix), 0) + return randTime.Format(TimeFormatForDATE) + case KindTIME: + randTime := time.Unix(MinTIMESTAMP.Unix()+rand.Int63n(GapTIMESTAMPUnix), 0) + return randTime.Format(TimeFormatForTIME) + case KindDATETIME: + randTime := randTime(MinDATETIME, GapDATETIMEUnix) + return randTime.Format(TimeFormat) + case KindTIMESTAMP: + randTime := randTime(MinTIMESTAMP, GapTIMESTAMPUnix) + return randTime.Format(TimeFormat) + case KindYEAR: + return rand.Intn(254) + 1901 //1901 ~ 2155 + case KindJSON: + return col.randJsonValue() + case KindEnum: + i := rand.Intn(len(col.setValue)) + return col.setValue[i] + case KindSet: + var l int + for l == 0 { + l = rand.Intn(len(col.setValue)) + } + idxs := make([]int, l) + m := make(map[int]struct{}) + for i := 0; i < l; i++ { + idx := rand.Intn(len(col.setValue)) + _, ok := m[idx] + for ok { + idx = rand.Intn(len(col.setValue)) + _, ok = m[idx] + } + m[idx] = struct{}{} + idxs[i] = idx + } + sort.Ints(idxs) + s := "" + for i := range idxs { + if i > 0 { + s += "," + } + s += col.setValue[idxs[i]] + } + return s + default: + return nil + } +} + +func randTime(minTime time.Time, gap int64) time.Time { + // https://github.com/chronotope/chrono-tz/issues/23 + // see all invalid time: https://timezonedb.com/time-zones/Asia/Shanghai + var randTime time.Time + for { + randTime = time.Unix(minTime.Unix()+rand.Int63n(gap), 0).In(local) + if notAmbiguousTime(randTime) { + break + } + } + return randTime +} + +func (col *ddlTestColumn) randJsonValue() string { + for _, dCol := range col.dependenciedCols { + col.mValue[dCol.nameOfGen] = dCol.randValue() + } + jsonRow, _ := json.Marshal(col.mValue) + return string(jsonRow) +} + +func notAmbiguousTime(t time.Time) bool { + ok := true + for _, amt := range ambiguousTimeSlice { + if t.Unix() >= amt.start && t.Unix() <= amt.end { + ok = false + break + } + } + return ok +} + +// randValueUnique use for primary key column to get unique value +func (col *ddlTestColumn) randValueUnique(PreValue []interface{}) (interface{}, bool) { + // retry times + for i := 0; i < 10; i++ { + v := col.randValue() + flag := true + for _, pv := range PreValue { + if v == pv { + flag = false + break + } + } + if flag { + return v, true + } + } + return nil, false +} + +func (col *ddlTestColumn) canBePrimary() bool { + return col.canBeIndex() && col.notGenerated() +} + +func (col *ddlTestColumn) canBeIndex() bool { + switch col.k { + case KindChar, KindVarChar: + if col.filedTypeM == 0 { + return false + } else { + return true + } + case KindBLOB, KindTINYBLOB, KindMEDIUMBLOB, KindLONGBLOB, KindTEXT, KindTINYTEXT, KindMEDIUMTEXT, KindLONGTEXT, KindJSON: + return false + default: + return true + } +} + +func (col *ddlTestColumn) canBeSet() bool { + return col.notGenerated() +} + +func (col *ddlTestColumn) canBeWhere() bool { + switch col.k { + case KindJSON: + return false + default: + return true + } +} + +//BLOB, TEXT, GEOMETRY or JSON column 'b' can't have a default value") +func (col *ddlTestColumn) canHaveDefaultValue() bool { + switch col.k { + case KindBLOB, KindTINYBLOB, KindMEDIUMBLOB, KindLONGBLOB, KindTEXT, KindTINYTEXT, KindMEDIUMTEXT, KindLONGTEXT, KindJSON: + return false + default: + return true + } +} + +type ddlTestIndex struct { + name string + signature string + columns []*ddlTestColumn +} diff --git a/tests/pkg/workload/ddl/internal/metrics.go b/tests/pkg/workload/ddl/internal/metrics.go new file mode 100644 index 0000000000..7a09e8ace2 --- /dev/null +++ b/tests/pkg/workload/ddl/internal/metrics.go @@ -0,0 +1,29 @@ +// Copyright 2019 PingCAP, Inc. +// +// 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, +// See the License for the specific language governing permissions and +// limitations under the License. +package internal + +import "github.com/prometheus/client_golang/prometheus" + +var ( + ddlFailedCounter = prometheus.NewCounter( + prometheus.CounterOpts{ + Namespace: "tidb_test", + Subsystem: "stability", + Name: "ddl_failed_total", + Help: "Counter of failed ddl operations.", + }) +) + +func init() { + prometheus.MustRegister(ddlFailedCounter) +} diff --git a/tests/pkg/workload/ddl/internal/run.go b/tests/pkg/workload/ddl/internal/run.go new file mode 100644 index 0000000000..182b4b99d6 --- /dev/null +++ b/tests/pkg/workload/ddl/internal/run.go @@ -0,0 +1,119 @@ +// Copyright 2019 PingCAP, Inc. +// +// 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, +// See the License for the specific language governing permissions and +// limitations under the License. +package internal + +import ( + "database/sql" + "strings" + "time" + + "github.com/golang/glog" + "github.com/prometheus/client_golang/prometheus" + "github.com/prometheus/client_golang/prometheus/push" + "golang.org/x/net/context" +) + +var defaultPushMetricsInterval = 15 * time.Second +var enableTransactionTestFlag = "0" +var enableTransactionTest = false + +func init() { + if enableTransactionTestFlag == "1" { + enableTransactionTest = true + } +} + +func getPromAddr() string { + return "" // TODO +} + +func openDB(dsn string, maxIdleConns int) (*sql.DB, error) { + db, err := sql.Open("mysql", dsn) + if err != nil { + return nil, err + } + + db.SetMaxIdleConns(maxIdleConns) + glog.Info("DB opens successfully") + return db, nil +} + +func Run(ctx context.Context, dbDSN string, concurrency int, tablesToCreate int, mysqlCompatible bool, testTp DDLTestType) { + glog.Infof("[ddl] Enable transaction test is: %v", enableTransactionTest) + + dbss := make([][]*sql.DB, 0, concurrency) + for i := 0; i < concurrency; i++ { + dbs := make([]*sql.DB, 0, 2) + // Parallel send DDL request need more connection to send DDL request concurrently + db0, err := openDB(dbDSN, 20) + if err != nil { + glog.Fatalf("[ddl] create db client error %v", err) + } + db1, err := openDB(dbDSN, 1) + if err != nil { + glog.Fatalf("[ddl] create db client error %v", err) + } + dbs = append(dbs, db0) + dbs = append(dbs, db1) + dbss = append(dbss, dbs) + } + + if promAddr := getPromAddr(); len(promAddr) > 0 { + go func() { + for { + err := push.FromGatherer("ddl", push.HostnameGroupingKey(), promAddr, prometheus.DefaultGatherer) + if err != nil { + glog.Errorf("[ddl] could not push metrics to prometheus push gateway: %v", err) + } + + time.Sleep(defaultPushMetricsInterval) + } + }() + } + + cfg := DDLCaseConfig{ + Concurrency: concurrency, + TablesToCreate: tablesToCreate, + MySQLCompatible: mysqlCompatible, + TestTp: testTp, + } + ddl := NewDDLCase(&cfg) + exeDDLFunc := SerialExecuteOperations + if cfg.TestTp == ParallelDDLTest { + exeDDLFunc = ParallelExecuteOperations + } + execDMLFunc := SerialExecuteDML + if enableTransactionTest { + execDMLFunc = TransactionExecuteOperations + } + if err := ddl.Initialize(ctx, dbss); err != nil { + glog.Fatalf("[ddl] initialze error %v", err) + } + if err := ddl.Execute(ctx, dbss, exeDDLFunc, execDMLFunc); err != nil { + glog.Fatalf("[ddl] execute error %v", err) + } +} + +func ignore_error(err error) bool { + if err == nil { + return true + } + errStr := err.Error() + if strings.Contains(errStr, "Information schema is changed") { + return true + } + if strings.Contains(errStr, "try again later") { + return true + } + return false +} diff --git a/tests/pkg/workload/ddl/internal/util.go b/tests/pkg/workload/ddl/internal/util.go new file mode 100644 index 0000000000..633bd9e691 --- /dev/null +++ b/tests/pkg/workload/ddl/internal/util.go @@ -0,0 +1,177 @@ +// Copyright 2019 PingCAP, Inc. +// +// 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, +// See the License for the specific language governing permissions and +// limitations under the License. + +package internal + +import ( + "database/sql" + "math/rand" + "strings" + + "github.com/golang/glog" +) + +func padLeft(str, pad string, length int) string { + if len(str) >= length { + return str + } + padding := strings.Repeat(pad, length) + str = padding + str + return str[len(str)-length:] +} + +func padRight(str, pad string, length int) string { + if len(str) >= length { + return str + } + padding := strings.Repeat(pad, length) + str = str + padding + return str[:length] +} + +func enableTiKVGC(db *sql.DB) { + sql := "update mysql.tidb set VARIABLE_VALUE = '10m' where VARIABLE_NAME = 'tikv_gc_life_time';" + _, err := db.Exec(sql) + if err != nil { + glog.Warningf("Failed to enable TiKV GC") + } +} + +func disableTiKVGC(db *sql.DB) { + sql := "update mysql.tidb set VARIABLE_VALUE = '500h' where VARIABLE_NAME = 'tikv_gc_life_time';" + _, err := db.Exec(sql) + if err != nil { + glog.Warningf("Failed to disable TiKV GC") + } +} + +// parallel run functions in parallel and wait until all of them are completed. +// If one of them returns error, the result is that error. +func parallel(funcs ...func() error) error { + cr := make(chan error, len(funcs)) + for _, foo := range funcs { + go func(foo func() error) { + err := foo() + cr <- err + }(foo) + } + var err error + for i := 0; i < len(funcs); i++ { + r := <-cr + if r != nil { + err = r + } + } + return err +} + +const letterBytes = "abcdefghijklmnopqrstuvwxyzABCDEFGHIJKLMNOPQRSTUVWXYZ" + +func randSeq(n int) string { + b := make([]byte, n) + for i := range b { + b[i] = letterBytes[rand.Intn(len(letterBytes))] + } + return string(b) +} + +const numberBytes = "0123456789" + +func randNum(n int) []byte { + b := make([]byte, n) + for i := range b { + b[i] = numberBytes[rand.Int63()%int64(len(numberBytes))] + } + return b +} + +func randMD() (m int, d int) { + for m == 0 { + m = rand.Intn(MAXDECIMALM) + } + min := m + if min > MAXDECIMALN { + min = MAXDECIMALN + } + d = rand.Intn(min) + return +} + +func randDecimal(m, d int) string { + ms := randNum(m - d) + ds := randNum(d) + var i int + for i = range ms { + if ms[i] != byte('0') { + break + } + } + ms = ms[i:] + l := len(ms) + len(ds) + 1 + flag := rand.Intn(2) + //check for 0.0... avoid -0.0 + zeroFlag := true + for i := range ms { + if ms[i] != byte('0') { + zeroFlag = false + } + } + for i := range ds { + if ds[i] != byte('0') { + zeroFlag = false + } + } + if zeroFlag { + flag = 0 + } + vs := make([]byte, 0, l+flag) + if flag == 1 { + vs = append(vs, '-') + } + vs = append(vs, ms...) + if len(ds) == 0 { + return string(vs) + } + vs = append(vs, '.') + vs = append(vs, ds...) + return string(vs) +} + +const FieldNameLen = 8 + +func randFieldName(m map[string]interface{}) string { + name := randSeq(FieldNameLen) + _, ok := m[name] + for ok { + name = randSeq(FieldNameLen) + _, ok = m[name] + } + return name +} + +const EnumValueLen = 5 + +func randEnumString(m map[string]struct{}) string { + l := rand.Intn(EnumValueLen) + 1 + name := randSeq(l) + nameL := strings.ToLower(name) + _, ok := m[nameL] + for ok { + l = rand.Intn(EnumValueLen) + 1 + name = randSeq(l) + nameL = strings.ToLower(name) + _, ok = m[nameL] + } + m[nameL] = struct{}{} + return name +} diff --git a/tests/pkg/workload/ddl/workload.go b/tests/pkg/workload/ddl/workload.go new file mode 100644 index 0000000000..23e4ca2229 --- /dev/null +++ b/tests/pkg/workload/ddl/workload.go @@ -0,0 +1,50 @@ +// Copyright 2019 PingCAP, Inc. +// +// 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, +// See the License for the specific language governing permissions and +// limitations under the License. + +package ddl + +import ( + "context" + + "github.com/juju/errors" + "github.com/pingcap/tidb-operator/tests/pkg/workload" + "github.com/pingcap/tidb-operator/tests/pkg/workload/ddl/internal" +) + +var _ = workload.Workload(&DDLWorkload{}) + +type DDLWorkload struct { + DSN string + Concurrency int + Tables int + + ctx context.Context + cancel context.CancelFunc +} + +func New(dsn string, concurrency int, tables int) workload.Workload { + return &DDLWorkload{DSN: dsn, Concurrency: concurrency, Tables: tables} +} + +func (w *DDLWorkload) Enter() error { + if w.ctx != nil { + return errors.New("already in ddl workload context") + } + w.ctx, w.cancel = context.WithCancel(context.Background()) + go internal.Run(w.ctx, w.DSN, w.Concurrency, w.Tables, false, internal.SerialDDLTest) + return nil +} + +func (w *DDLWorkload) Leave() { + w.cancel() +} diff --git a/tests/pkg/workload/interface.go b/tests/pkg/workload/interface.go new file mode 100644 index 0000000000..26e5284b20 --- /dev/null +++ b/tests/pkg/workload/interface.go @@ -0,0 +1,31 @@ +// Copyright 2019 PingCAP, Inc. +// +// 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, +// See the License for the specific language governing permissions and +// limitations under the License. + +package workload + +import "github.com/pingcap/errors" + +type Workload interface { + Enter() error + Leave() +} + +func Run(f func() error, ws ...Workload) error { + for _, w := range ws { + if err := w.Enter(); err != nil { + return errors.Annotate(err, "enter workload") + } + defer w.Leave() + } + return f() +} From b0ef8e1716619dcacb57e951235466937b892a07 Mon Sep 17 00:00:00 2001 From: CWen Date: Wed, 20 Mar 2019 11:05:34 +0800 Subject: [PATCH 18/22] stability: use fault-trigger at e2e tests and add some log (#330) --- tests/actions.go | 37 +---- tests/cmd/e2e/config.yaml | 23 +++ tests/cmd/e2e/main.go | 22 ++- tests/config.go | 65 ++++++++ tests/fault.go | 174 +++++++++++++++++++++ tests/log_dump.go | 2 +- tests/manifests/e2e-configmap.yaml | 32 ++++ tests/manifests/e2e.yaml | 9 ++ tests/pkg/fault-trigger/client/client.go | 12 +- tests/pkg/fault-trigger/manager/etcd.go | 6 + tests/pkg/fault-trigger/manager/kubelet.go | 6 + tests/pkg/fault-trigger/manager/vm.go | 6 + 12 files changed, 355 insertions(+), 39 deletions(-) create mode 100644 tests/cmd/e2e/config.yaml create mode 100644 tests/config.go create mode 100644 tests/fault.go create mode 100644 tests/manifests/e2e-configmap.yaml diff --git a/tests/actions.go b/tests/actions.go index ee87c0cff6..308bd6934e 100644 --- a/tests/actions.go +++ b/tests/actions.go @@ -54,12 +54,12 @@ const ( defaultRawSize = 100 ) -func NewOperatorActions(cli versioned.Interface, kubeCli kubernetes.Interface, logDir string) OperatorActions { +func NewOperatorActions(cli versioned.Interface, kubeCli kubernetes.Interface, cfg *Config) OperatorActions { return &operatorActions{ cli: cli, kubeCli: kubeCli, pdControl: controller.NewDefaultPDControl(), - logDir: logDir, + cfg: cfg, } } @@ -96,32 +96,11 @@ type OperatorActions interface { getBackupDir(info *TidbClusterInfo) ([]string, error) } -type FaultTriggerActions interface { - StopNode(nodeName string) error - StartNode(nodeName string) error - StopEtcd() error - StartEtcd() error - StopKubeAPIServer() error - StartKubeAPIServer() error - StopKubeControllerManager() error - StartKubeControllerManager() error - StopKubeScheduler() error - StartKubeScheduler() error - StopKubelet(nodeName string) error - StartKubelet(nodeName string) error - StopKubeProxy(nodeName string) error - StartKubeProxy(nodeName string) error - DiskCorruption(nodeName string) error - NetworkPartition(fromNode, toNode string) error - NetworkDelay(fromNode, toNode string) error - DockerCrash(nodeName string) error -} - type operatorActions struct { cli versioned.Interface kubeCli kubernetes.Interface pdControl controller.PDControlInterface - logDir string + cfg *Config } var _ = OperatorActions(&operatorActions{}) @@ -915,27 +894,27 @@ func (oa *operatorActions) monitorNormal(clusterInfo *TidbClusterInfo) (bool, er return false, nil } if monitorDeployment.Status.ReadyReplicas < 1 { - glog.Info("monitor ready replicas %d < 1", monitorDeployment.Status.ReadyReplicas) + glog.Infof("monitor ready replicas %d < 1", monitorDeployment.Status.ReadyReplicas) return false, nil } configuratorJobName := fmt.Sprintf("%s-monitor-configurator", tcName) monitorJob, err := oa.kubeCli.BatchV1().Jobs(ns).Get(configuratorJobName, metav1.GetOptions{}) if err != nil { - glog.Info("get monitor configurator job: [%s/%s] failed", ns, configuratorJobName) + glog.Infof("get monitor configurator job: [%s/%s] failed", ns, configuratorJobName) return false, nil } if monitorJob.Status.Succeeded == 0 { - glog.Info("the monitor configurator job: [%s/%s] had not success", ns, configuratorJobName) + glog.Infof("the monitor configurator job: [%s/%s] had not success", ns, configuratorJobName) return false, nil } if err := oa.checkPrometheus(clusterInfo); err != nil { - glog.Info("check [%s/%s]'s prometheus data failed: %v", ns, monitorDeploymentName, err) + glog.Infof("check [%s/%s]'s prometheus data failed: %v", ns, monitorDeploymentName, err) return false, nil } if err := oa.checkGrafanaData(clusterInfo); err != nil { - glog.Info("check [%s/%s]'s grafana data failed: %v", ns, monitorDeploymentName, err) + glog.Infof("check [%s/%s]'s grafana data failed: %v", ns, monitorDeploymentName, err) return false, nil } return true, nil diff --git a/tests/cmd/e2e/config.yaml b/tests/cmd/e2e/config.yaml new file mode 100644 index 0000000000..2f35e38f91 --- /dev/null +++ b/tests/cmd/e2e/config.yaml @@ -0,0 +1,23 @@ +nodes: + - physical_node: 172.16.4.39 + nodes: + - 172.16.4.171 + - 172.16.4.172 + - 172.16.4.173 + - physical_node: 172.16.4.40 + nodes: + - 172.16.4.174 + - 172.16.4.175 + - 172.16.4.176 +etcds: + - physical_node: 172.16.4.39 + nodes: + - 172.16.4.171 + - 172.16.4.172 + - 172.16.4.173 +apiservers: + - physical_node: 172.16.4.39 + nodes: + - 172.16.4.171 + - 172.16.4.172 + - 172.16.4.173 diff --git a/tests/cmd/e2e/main.go b/tests/cmd/e2e/main.go index 61275986e2..90f1740891 100644 --- a/tests/cmd/e2e/main.go +++ b/tests/cmd/e2e/main.go @@ -14,9 +14,9 @@ package main import ( - "flag" "net/http" _ "net/http/pprof" + "time" "github.com/golang/glog" "github.com/pingcap/tidb-operator/pkg/client/clientset/versioned" @@ -36,10 +36,15 @@ func perror(err error) { } func main() { - flag.Parse() logs.InitLogs() defer logs.FlushLogs() + conf := tests.NewConfig() + err := conf.Parse() + if err != nil { + glog.Fatalf("failed to parse config: %v", err) + } + go func() { glog.Info(http.ListenAndServe("localhost:6060", nil)) }() @@ -57,7 +62,7 @@ func main() { glog.Fatalf("failed to get kubernetes Clientset: %v", err) } - oa := tests.NewOperatorActions(cli, kubeCli, "/logDir") + oa := tests.NewOperatorActions(cli, kubeCli, conf) operatorInfo := &tests.OperatorInfo{ Namespace: "pingcap", @@ -218,4 +223,15 @@ func main() { oa.DumpAllLogs(operatorInfo, []*tests.TidbClusterInfo{clusterInfo, restoreClusterInfo}) glog.Fatal(err) } + + fa := tests.NewFaultTriggerAction(cli, kubeCli, conf) + if err := fa.StopETCD("172.16.4.171"); err != nil { + glog.Fatal(err) + } + + time.Sleep(1 * time.Minute) + + if err := fa.StartETCD("172.16.4.171"); err != nil { + glog.Fatal(err) + } } diff --git a/tests/config.go b/tests/config.go new file mode 100644 index 0000000000..031bf4875e --- /dev/null +++ b/tests/config.go @@ -0,0 +1,65 @@ +package tests + +import ( + "flag" + "io/ioutil" + + yaml "gopkg.in/yaml.v2" +) + +// Config defines the config of operator tests +type Config struct { + configFile string + + LogDir string `yaml:"log_dir" json:"log_dir"` + FaultTriggerPort int `yaml:"fault_trigger_port" json:"fault_trigger_port"` + Nodes []Nodes `yaml:"nodes" json:"nodes"` + ETCDs []Nodes `yaml:"etcds" json:"etcds"` + APIServers []Nodes `yaml:"apiservers" json:"apiservers"` +} + +// Nodes defines a series of nodes that belong to the same physical node. +type Nodes struct { + PhysicalNode string `yaml:"physical_node" json:"physical_node"` + Nodes []string `yaml:"nodes" json:"nodes"` +} + +// NewConfig creates a new config. +func NewConfig() *Config { + cfg := &Config{} + flag.StringVar(&cfg.configFile, "config", "/etc/e2e/config.yaml", "Config file") + flag.StringVar(&cfg.LogDir, "log-dir", "/logDir", "log directory") + flag.IntVar(&cfg.FaultTriggerPort, "fault-trigger-port", 23332, "the http port of fault trigger service") + + return cfg +} + +// Parse parses flag definitions from the argument list. +func (c *Config) Parse() error { + // Parse first to get config file + flag.Parse() + + if c.configFile != "" { + if err := c.configFromFile(c.configFile); err != nil { + return err + } + } + + // Parse again to replace with command line options. + flag.Parse() + + return nil +} + +func (c *Config) configFromFile(path string) error { + data, err := ioutil.ReadFile(path) + if err != nil { + return err + } + + if err = yaml.Unmarshal(data, c); err != nil { + return err + } + + return nil +} diff --git a/tests/fault.go b/tests/fault.go new file mode 100644 index 0000000000..b77e7527b9 --- /dev/null +++ b/tests/fault.go @@ -0,0 +1,174 @@ +package tests + +import ( + "fmt" + + "github.com/golang/glog" + "github.com/pingcap/tidb-operator/pkg/client/clientset/versioned" + "github.com/pingcap/tidb-operator/pkg/controller" + "github.com/pingcap/tidb-operator/tests/pkg/fault-trigger/client" + "github.com/pingcap/tidb-operator/tests/pkg/fault-trigger/manager" + "k8s.io/client-go/kubernetes" +) + +type FaultTriggerActions interface { + StopNode(physicalNode string, node string) error + StartNode(physicalNode string, node string) error + StopETCD(nodes ...string) error + StartETCD(nodes ...string) error + StopKubelet(node string) error + StartKubelet(node string) error + // TODO: support more faults + // StopKubeAPIServer() error + // StartKubeAPIServer() error + // StopKubeControllerManager() error + // StartKubeControllerManager() error + // StopKubeScheduler() error + // StartKubeScheduler() error + // StopKubeProxy(node string) error + // StartKubeProxy(node string) error + // DiskCorruption(node string) error + // NetworkPartition(fromNode, toNode string) error + // NetworkDelay(fromNode, toNode string) error + // DockerCrash(nodeName string) error +} + +func NewFaultTriggerAction(cli versioned.Interface, kubeCli kubernetes.Interface, cfg *Config) FaultTriggerActions { + return &faultTriggerActions{ + cli: cli, + kubeCli: kubeCli, + pdControl: controller.NewDefaultPDControl(), + cfg: cfg, + } +} + +type faultTriggerActions struct { + cli versioned.Interface + kubeCli kubernetes.Interface + pdControl controller.PDControlInterface + cfg *Config +} + +func (fa *faultTriggerActions) StopNode(physicalNode string, node string) error { + faultCli := client.NewClient(client.Config{ + Addr: fa.genFaultTriggerAddr(physicalNode), + }) + + err := faultCli.StopVM(&manager.VM{ + IP: node, + }) + + if err != nil { + glog.Errorf("failed to stop node %s on physical node: %s: %v", node, physicalNode, err) + return err + } + + glog.Infof("node %s on physical node %s is stopped", node, physicalNode) + + return nil +} + +func (fa *faultTriggerActions) StartNode(physicalNode string, node string) error { + faultCli := client.NewClient(client.Config{ + Addr: fa.genFaultTriggerAddr(physicalNode), + }) + + err := faultCli.StartVM(&manager.VM{ + IP: node, + }) + + if err != nil { + glog.Errorf("failed to start node %s on physical node %s: %v", node, physicalNode, err) + return err + } + + glog.Infof("node %s on physical node %s is started", physicalNode, node) + + return nil +} + +// StopETCD stops the etcd service. +// If the `nodes` is empty, StopEtcd will stop all etcd service. +func (fa *faultTriggerActions) StopETCD(nodes ...string) error { + if len(nodes) == 0 { + for _, ns := range fa.cfg.ETCDs { + nodes = append(nodes, ns.Nodes...) + } + } + + for _, node := range nodes { + faultCli := client.NewClient(client.Config{ + Addr: fa.genFaultTriggerAddr(node), + }) + + if err := faultCli.StopETCD(); err != nil { + glog.Errorf("failed to stop etcd %s: %v", node, err) + return err + } + + glog.Infof("etcd %s is stopped", node) + } + + return nil +} + +// StartETCD starts the etcd service. +// If the `nodes` is empty, StartETCD will start all etcd service. +func (fa *faultTriggerActions) StartETCD(nodes ...string) error { + if len(nodes) == 0 { + for _, ns := range fa.cfg.ETCDs { + nodes = append(nodes, ns.Nodes...) + } + } + + for _, node := range nodes { + faultCli := client.NewClient(client.Config{ + Addr: fa.genFaultTriggerAddr(node), + }) + + if err := faultCli.StartETCD(); err != nil { + glog.Errorf("failed to start etcd %s: %v", node, err) + return err + } + + glog.Infof("etcd %s is started", node) + } + + return nil +} + +// StopKubelet stops the kubelet service. +func (fa *faultTriggerActions) StopKubelet(node string) error { + faultCli := client.NewClient(client.Config{ + Addr: fa.genFaultTriggerAddr(node), + }) + + if err := faultCli.StopKubelet(); err != nil { + glog.Errorf("failed to stop kubelet %s: %v", node, err) + return err + } + + glog.Infof("kubelet %s is stopped", node) + + return nil +} + +// StartKubelet starts the kubelet service. +func (fa *faultTriggerActions) StartKubelet(node string) error { + faultCli := client.NewClient(client.Config{ + Addr: node, + }) + + if err := faultCli.StartKubelet(); err != nil { + glog.Errorf("failed to start kubelet %s: %v", node, err) + return err + } + + glog.Infof("kubelet %s is started", node) + + return nil +} + +func (fa *faultTriggerActions) genFaultTriggerAddr(node string) string { + return fmt.Sprintf("%s:%d", node, fa.cfg.FaultTriggerPort) +} diff --git a/tests/log_dump.go b/tests/log_dump.go index 97de55c0ad..f5dc11af02 100644 --- a/tests/log_dump.go +++ b/tests/log_dump.go @@ -12,7 +12,7 @@ import ( ) func (oa *operatorActions) DumpAllLogs(operatorInfo *OperatorInfo, testClusters []*TidbClusterInfo) error { - logPath := fmt.Sprintf("/%s/%s", oa.logDir, "operator-stability") + logPath := fmt.Sprintf("/%s/%s", oa.cfg.LogDir, "operator-stability") if _, err := os.Stat(logPath); os.IsNotExist(err) { err = os.MkdirAll(logPath, os.ModePerm) if err != nil { diff --git a/tests/manifests/e2e-configmap.yaml b/tests/manifests/e2e-configmap.yaml new file mode 100644 index 0000000000..0af85753bb --- /dev/null +++ b/tests/manifests/e2e-configmap.yaml @@ -0,0 +1,32 @@ +apiVersion: v1 +kind: ConfigMap +metadata: + namespace: tidb-operator-e2e + name: tidb-operator-e2e-config +data: + e2e-config: |- + nodes: + - physical_node: 172.16.4.39 + nodes: + - 172.16.4.171 + - 172.16.4.172 + - 172.16.4.173 + - physical_node: 172.16.4.40 + nodes: + - 172.16.4.174 + - 172.16.4.175 + - 172.16.4.176 + etcds: + - physical_node: 172.16.4.39 + nodes: + - 172.16.4.171 + - 172.16.4.172 + - 172.16.4.173 + apiservers: + - physical_node: 172.16.4.39 + nodes: + - 172.16.4.171 + - 172.16.4.172 + - 172.16.4.173 + + diff --git a/tests/manifests/e2e.yaml b/tests/manifests/e2e.yaml index 77e44a4559..913dceb17b 100644 --- a/tests/manifests/e2e.yaml +++ b/tests/manifests/e2e.yaml @@ -33,9 +33,18 @@ spec: volumeMounts: - mountPath: /logDir name: logdir + - name: config + readOnly: true + mountPath: /etc/e2e volumes: - name: logdir hostPath: path: /var/log type: Directory + - name: config + configMap: + name: tidb-operator-e2e-config + items: + - key: e2e-config + path: config.yaml restartPolicy: Never diff --git a/tests/pkg/fault-trigger/client/client.go b/tests/pkg/fault-trigger/client/client.go index 17a812cbe6..3f7c2348c6 100644 --- a/tests/pkg/fault-trigger/client/client.go +++ b/tests/pkg/fault-trigger/client/client.go @@ -131,7 +131,7 @@ func (c *client) StartVM(vm *manager.VM) error { vmName = vm.IP } - url := util.GenURL(fmt.Sprintf("%s/%s/vm/%s/start", c.cfg.Addr, api.APIPrefix, vmName)) + url := util.GenURL(fmt.Sprintf("%s%s/vm/%s/start", c.cfg.Addr, api.APIPrefix, vmName)) if _, err := c.get(url); err != nil { return err } @@ -149,7 +149,7 @@ func (c *client) StopVM(vm *manager.VM) error { vmName = vm.IP } - url := util.GenURL(fmt.Sprintf("%s/%s/vm/%s/stop", c.cfg.Addr, api.APIPrefix, vmName)) + url := util.GenURL(fmt.Sprintf("%s%s/vm/%s/stop", c.cfg.Addr, api.APIPrefix, vmName)) if _, err := c.get(url); err != nil { return err } @@ -158,7 +158,7 @@ func (c *client) StopVM(vm *manager.VM) error { } func (c *client) StartETCD() error { - url := util.GenURL(fmt.Sprintf("%s/%s/etcd/start", c.cfg.Addr, api.APIPrefix)) + url := util.GenURL(fmt.Sprintf("%s%s/etcd/start", c.cfg.Addr, api.APIPrefix)) if _, err := c.get(url); err != nil { return err } @@ -167,7 +167,7 @@ func (c *client) StartETCD() error { } func (c *client) StopETCD() error { - url := util.GenURL(fmt.Sprintf("%s/%s/etcd/stop", c.cfg.Addr, api.APIPrefix)) + url := util.GenURL(fmt.Sprintf("%s%s/etcd/stop", c.cfg.Addr, api.APIPrefix)) if _, err := c.get(url); err != nil { return err } @@ -176,7 +176,7 @@ func (c *client) StopETCD() error { } func (c *client) StartKubelet() error { - url := util.GenURL(fmt.Sprintf("%s/%s/kubelet/start", c.cfg.Addr, api.APIPrefix)) + url := util.GenURL(fmt.Sprintf("%s%s/kubelet/start", c.cfg.Addr, api.APIPrefix)) if _, err := c.get(url); err != nil { return err } @@ -185,7 +185,7 @@ func (c *client) StartKubelet() error { } func (c *client) StopKubelet() error { - url := util.GenURL(fmt.Sprintf("%s/%s/kubelet/stop", c.cfg.Addr, api.APIPrefix)) + url := util.GenURL(fmt.Sprintf("%s%s/kubelet/stop", c.cfg.Addr, api.APIPrefix)) if _, err := c.get(url); err != nil { return err } diff --git a/tests/pkg/fault-trigger/manager/etcd.go b/tests/pkg/fault-trigger/manager/etcd.go index cd16a527ae..ef4d644736 100644 --- a/tests/pkg/fault-trigger/manager/etcd.go +++ b/tests/pkg/fault-trigger/manager/etcd.go @@ -28,6 +28,9 @@ func (m *Manager) StartETCD() error { glog.Errorf("exec: [%s] failed, output: %s, error: %v", shell, string(output), err) return err } + + glog.Info("etcd service is started") + return nil } @@ -40,5 +43,8 @@ func (m *Manager) StopETCD() error { glog.Errorf("exec: [%s] failed, output: %s, error: %v", shell, string(output), err) return err } + + glog.Info("etcd service is stopped") + return nil } diff --git a/tests/pkg/fault-trigger/manager/kubelet.go b/tests/pkg/fault-trigger/manager/kubelet.go index 4dad0c0c60..04b6b91f89 100644 --- a/tests/pkg/fault-trigger/manager/kubelet.go +++ b/tests/pkg/fault-trigger/manager/kubelet.go @@ -28,6 +28,9 @@ func (m *Manager) StartKubelet() error { glog.Errorf("exec: [%s] failed, output: %s, error: %v", shell, string(output), err) return err } + + glog.Info("kubelet is started") + return nil } @@ -40,5 +43,8 @@ func (m *Manager) StopKubelet() error { glog.Errorf("exec: [%s] failed, output: %s, error: %v", shell, string(output), err) return err } + + glog.Info("kubelet is stopped") + return nil } diff --git a/tests/pkg/fault-trigger/manager/vm.go b/tests/pkg/fault-trigger/manager/vm.go index e83df64b00..6a2e5eb145 100644 --- a/tests/pkg/fault-trigger/manager/vm.go +++ b/tests/pkg/fault-trigger/manager/vm.go @@ -53,6 +53,9 @@ func (m *Manager) StopVM(v *VM) error { glog.Errorf("exec: [%s] failed, output: %s, error: %v", shell, string(output), err) return err } + + glog.Infof("virtual machine %s is stopped", v.Name) + return nil } @@ -65,6 +68,9 @@ func (m *Manager) StartVM(v *VM) error { glog.Errorf("exec: [%s] failed, output: %s, error: %v", shell, string(output), err) return err } + + glog.Infof("virtual machine %s is started", v.Name) + return nil } From fa2e51c3aa8e5e78446f2a0978fe888d3ebaf4d8 Mon Sep 17 00:00:00 2001 From: shuijing198799 <30903849+shuijing198799@users.noreply.github.com> Date: Wed, 20 Mar 2019 12:15:39 +0800 Subject: [PATCH 19/22] add binlog deploy and check process (#329) --- tests/actions.go | 257 +++++++++++++++++++++++++++++-------- tests/backup/backupcase.go | 38 ++++++ tests/cmd/e2e/main.go | 6 + 3 files changed, 249 insertions(+), 52 deletions(-) diff --git a/tests/actions.go b/tests/actions.go index 308bd6934e..49f8f642b4 100644 --- a/tests/actions.go +++ b/tests/actions.go @@ -14,7 +14,6 @@ package tests import ( - "bytes" "database/sql" "encoding/json" "fmt" @@ -67,8 +66,8 @@ const ( DefaultPollTimeout time.Duration = 10 * time.Minute DefaultPollInterval time.Duration = 10 * time.Second getBackupDirPodName = "get-backup-dir" - grafanaUsername = "admin" - grafanaPassword = "admin" + grafanaUsername = "admin" + grafanaPassword = "admin" ) type OperatorActions interface { @@ -131,9 +130,12 @@ type TidbClusterInfo struct { Args map[string]string blockWriter *blockwriter.BlockWriterCase Monitor bool + UserName string + InitSecretName string + BackupSecretName string } -func (tc *TidbClusterInfo) HelmSetString() string { +func (tc *TidbClusterInfo) HelmSetString(m map[string]string) string { set := map[string]string{ "clusterName": tc.ClusterName, @@ -145,9 +147,10 @@ func (tc *TidbClusterInfo) HelmSetString() string { "pd.image": tc.PDImage, "tikv.image": tc.TiKVImage, "tidb.image": tc.TiDBImage, - "tidb.passwordSecretName": "set-secret", + "tidb.passwordSecretName": tc.InitSecretName, "tidb.initSql": tc.InitSql, "monitor.create": strconv.FormatBool(tc.Monitor), + "secretName": tc.BackupSecretName, } for k, v := range tc.Resources { @@ -156,6 +159,9 @@ func (tc *TidbClusterInfo) HelmSetString() string { for k, v := range tc.Args { set[k] = v } + for k, v := range m { + set[k] = v + } arr := make([]string, 0, len(set)) for k, v := range set { @@ -224,8 +230,14 @@ func (oa *operatorActions) DeployTidbCluster(info *TidbClusterInfo) error { defer func() { glog.Infof("deploy tidb cluster end cluster[%s] namespace[%s]", info.ClusterName, info.Namespace) }() + + err := oa.CreateSecret(info) + if err != nil { + return fmt.Errorf("failed to create secret of cluster [%s]: %v", info.ClusterName, err) + } + cmd := fmt.Sprintf("helm install /charts/%s/tidb-cluster --name %s --namespace %s --set-string %s", - info.OperatorTag, info.ClusterName, info.Namespace, info.HelmSetString()) + info.OperatorTag, info.ClusterName, info.Namespace, info.HelmSetString(nil)) if res, err := exec.Command("/bin/sh", "-c", cmd).CombinedOutput(); err != nil { return fmt.Errorf("failed to deploy tidbcluster: %s/%s, %v, %s", info.Namespace, info.ClusterName, err, string(res)) @@ -393,7 +405,7 @@ func chartPath(name string, tag string) string { func (oa *operatorActions) ScaleTidbCluster(info *TidbClusterInfo) error { cmd := fmt.Sprintf("helm upgrade %s %s --set-string %s", - info.ClusterName, chartPath("tidb-cluster", info.OperatorTag), info.HelmSetString()) + info.ClusterName, chartPath("tidb-cluster", info.OperatorTag), info.HelmSetString(nil)) glog.Info("[SCALE] " + cmd) res, err := exec.Command("/bin/sh", "-c", cmd).CombinedOutput() if err != nil { @@ -404,7 +416,7 @@ func (oa *operatorActions) ScaleTidbCluster(info *TidbClusterInfo) error { func (oa *operatorActions) UpgradeTidbCluster(info *TidbClusterInfo) error { cmd := fmt.Sprintf("helm upgrade %s %s --set-string %s", - info.ClusterName, chartPath("tidb-cluster", info.OperatorTag), info.HelmSetString()) + info.ClusterName, chartPath("tidb-cluster", info.OperatorTag), info.HelmSetString(nil)) glog.Info("[UPGRADE] " + cmd) res, err := exec.Command("/bin/sh", "-c", cmd).CombinedOutput() if err != nil { @@ -1037,26 +1049,18 @@ func (oa *operatorActions) DeployAdHocBackup(info *TidbClusterInfo) error { glog.Infof("deploy adhoc backup end cluster[%s] namespace[%s]", info.ClusterName, info.Namespace) }() sets := map[string]string{ - "clusterName": info.ClusterName, "name": info.BackupPVC, "mode": "backup", "user": "root", "password": info.Password, "storage.size": "10Gi", } - var buffer bytes.Buffer - for k, v := range sets { - set := fmt.Sprintf(" --set %s=%s", k, v) - _, err := buffer.WriteString(set) - if err != nil { - return err - } - } - setStr := buffer.String() + setString := info.HelmSetString(sets) + fullbackupName := fmt.Sprintf("%s-backup", info.ClusterName) - cmd := fmt.Sprintf("helm install -n %s --namespace %s /charts/%s/tidb-backup %s", - fullbackupName, info.Namespace, info.OperatorTag, setStr) + cmd := fmt.Sprintf("helm install -n %s --namespace %s /charts/%s/tidb-backup --set-string %s", + fullbackupName, info.Namespace, info.OperatorTag, setString) glog.Infof("install adhoc deployment [%s]", cmd) res, err := exec.Command("/bin/sh", "-c", cmd).CombinedOutput() if err != nil { @@ -1101,26 +1105,18 @@ func (oa *operatorActions) Restore(from *TidbClusterInfo, to *TidbClusterInfo) e glog.Infof("deploy restore end cluster[%s] namespace[%s]", to.ClusterName, to.Namespace) }() sets := map[string]string{ - "clusterName": to.ClusterName, "name": to.BackupPVC, "mode": "restore", "user": "root", "password": to.Password, "storage.size": "10Gi", } - var buffer bytes.Buffer - for k, v := range sets { - set := fmt.Sprintf(" --set %s=%s", k, v) - _, err := buffer.WriteString(set) - if err != nil { - return err - } - } - setStr := buffer.String() + setString := to.HelmSetString(sets) + restoreName := fmt.Sprintf("%s-restore", from.ClusterName) - cmd := fmt.Sprintf("helm install -n %s --namespace %s /charts/%s/tidb-backup %s", - restoreName, to.Namespace, to.OperatorTag, setStr) + cmd := fmt.Sprintf("helm install -n %s --namespace %s /charts/%s/tidb-backup --set-string %s", + restoreName, to.Namespace, to.OperatorTag, setString) glog.Infof("install restore [%s]", cmd) res, err := exec.Command("/bin/sh", "-c", cmd).CombinedOutput() if err != nil { @@ -1213,31 +1209,29 @@ func (info *TidbClusterInfo) QueryCount() (int, error) { } func (oa *operatorActions) CreateSecret(info *TidbClusterInfo) error { - initSecretName := "set-secret" - backupSecretName := "backup-secret" initSecret := corev1.Secret{ ObjectMeta: metav1.ObjectMeta{ - Name: initSecretName, + Name: info.InitSecretName, Namespace: info.Namespace, }, Data: map[string][]byte{ - "root": []byte(info.Password), + info.UserName: []byte(info.Password), }, Type: corev1.SecretTypeOpaque, } _, err := oa.kubeCli.CoreV1().Secrets(info.Namespace).Create(&initSecret) if err != nil && !releaseIsExist(err) { - return err + return err } backupSecret := corev1.Secret{ ObjectMeta: metav1.ObjectMeta{ - Name: backupSecretName, + Name: info.BackupSecretName, Namespace: info.Namespace, }, Data: map[string][]byte{ - "user": []byte("root"), + "user": []byte(info.UserName), "password": []byte(info.Password), }, Type: corev1.SecretTypeOpaque, @@ -1270,19 +1264,11 @@ func (oa *operatorActions) DeployScheduledBackup(info *TidbClusterInfo) error { "scheduledBackup.schedule": cron, "scheduledBackup.storage": "10Gi", } - var buffer bytes.Buffer - for k, v := range sets { - set := fmt.Sprintf(" --set %s=%s", k, v) - _, err := buffer.WriteString(set) - if err != nil { - return err - } - } - setStr := buffer.String() + setString := info.HelmSetString(sets) - cmd := fmt.Sprintf("helm upgrade %s /charts/%s/tidb-cluster %s", - info.ClusterName, info.OperatorTag, setStr) + cmd := fmt.Sprintf("helm upgrade %s /charts/%s/tidb-cluster --set-string %s", + info.ClusterName, info.OperatorTag, setString) glog.Infof("scheduled-backup delploy [%s]", cmd) res, err := exec.Command("/bin/sh", "-c", cmd).CombinedOutput() @@ -1437,10 +1423,12 @@ func (oa *operatorActions) getBackupDir(info *TidbClusterInfo) ([]string, error) } cmd := fmt.Sprintf("kubectl exec %s -n %s ls /data", getBackupDirPodName, info.Namespace) - glog.Infof(cmd) + + time.Sleep(20 * time.Second) + res, err := exec.Command("/bin/sh", "-c", cmd).CombinedOutput() if err != nil { - glog.Errorf("cluster:[%s/%s] exec :%s failed,error:%v,result:%s", info.Namespace, info.ClusterName, cmd, err, res) + glog.Errorf("cluster:[%s/%s] exec :%s failed,error:%v,result:%s", info.Namespace, info.ClusterName, cmd, err, string(res)) return nil, err } @@ -1454,9 +1442,174 @@ func (info *TidbClusterInfo) FullName() string { } func (oa *operatorActions) DeployIncrementalBackup(from *TidbClusterInfo, to *TidbClusterInfo) error { + glog.Infof("begin to deploy incremental backup cluster[%s] namespace[%s]", from.ClusterName, from.Namespace) + defer func() { + glog.Infof("deploy incremental backup end cluster[%s] namespace[%s]", to.ClusterName, to.Namespace) + }() + sets := map[string]string{ + "binlog.pump.create": "true", + "binlog.drainer.destDBType": "mysql", + "binlog.drainer.create": "true", + "binlog.drainer.mysql.host": fmt.Sprintf("%s-tidb.%s", to.ClusterName, to.Namespace), + "binlog.drainer.mysql.user": "root", + "binlog.drainer.mysql.password": to.Password, + "binlog.drainer.mysql.port": "4000", + } + + setString := from.HelmSetString(sets) + + cmd := fmt.Sprintf("helm upgrade %s /charts/%s/tidb-cluster --set-string %s", + from.ClusterName, from.OperatorTag, setString) + glog.Infof(cmd) + res, err := exec.Command("/bin/sh", "-c", cmd).CombinedOutput() + if err != nil { + return fmt.Errorf("failed to launch scheduler backup job: %v, %s", err, string(res)) + } return nil } func (oa *operatorActions) CheckIncrementalBackup(info *TidbClusterInfo) error { + glog.Infof("begin to check incremental backup cluster[%s] namespace[%s]", info.ClusterName, info.Namespace) + defer func() { + glog.Infof("check incremental backup end cluster[%s] namespace[%s]", info.ClusterName, info.Namespace) + }() + + pumpStatefulSetName := fmt.Sprintf("%s-pump", info.ClusterName) + fn := func() (bool, error) { + pumpStatefulSet, err := oa.kubeCli.AppsV1().StatefulSets(info.Namespace).Get(pumpStatefulSetName, metav1.GetOptions{}) + if err != nil { + glog.Errorf("failed to get jobs %s ,%v", pumpStatefulSetName, err) + return false, nil + } + if pumpStatefulSet.Status.Replicas != pumpStatefulSet.Status.ReadyReplicas { + glog.Errorf("pump replicas is not ready, please wait ! %s ", pumpStatefulSetName) + return false, nil + } + + listOps := metav1.ListOptions{ + LabelSelector: labels.SelectorFromSet( + pumpStatefulSet.Labels, + ).String(), + } + + pods, err := oa.kubeCli.CoreV1().Pods(info.Namespace).List(listOps) + if err != nil { + glog.Errorf("failed to get pods via pump labels %s ,%v", pumpStatefulSetName, err) + return false, nil + } + + for _, pod := range pods.Items { + if !oa.pumpHealth(info, pod.Spec.Hostname) { + glog.Errorf("some pods is not health %s ,%v", pumpStatefulSetName, err) + return false, nil + } + } + + drainerStatefulSetName := fmt.Sprintf("%s-drainer", info.ClusterName) + drainerStatefulSet, err := oa.kubeCli.AppsV1().StatefulSets(info.Namespace).Get(drainerStatefulSetName, metav1.GetOptions{}) + if err != nil { + glog.Errorf("failed to get jobs %s ,%v", pumpStatefulSetName, err) + return false, nil + } + if drainerStatefulSet.Status.Replicas != drainerStatefulSet.Status.ReadyReplicas { + glog.Errorf("drainer replicas is not ready, please wait ! %s ", pumpStatefulSetName) + return false, nil + } + + listOps = metav1.ListOptions{ + LabelSelector: labels.SelectorFromSet( + drainerStatefulSet.Labels, + ).String(), + } + + pods, err = oa.kubeCli.CoreV1().Pods(info.Namespace).List(listOps) + if err != nil { + return false, nil + } + for _, pod := range pods.Items { + if !oa.drainerHealth(info, pod.Spec.Hostname) { + return false, nil + } + } + + return true, nil + } + + err := wait.Poll(DefaultPollInterval, DefaultPollTimeout, fn) + if err != nil { + return fmt.Errorf("failed to launch scheduler backup job: %v", err) + } return nil + +} + +type pumpStatus struct { + StatusMap map[string]*nodeStatus +} + +type nodeStatus struct { + State string `json:"state"` +} + +func (oa *operatorActions) pumpHealth(info *TidbClusterInfo, hostName string) bool { + pumpHealthUrl := fmt.Sprintf("%s.%s-pump.%s:8250/status", hostName, info.ClusterName, info.Namespace) + res, err := http.Get(pumpHealthUrl) + if err != nil { + glog.Errorf("cluster:[%s] call %s failed,error:%v", info.ClusterName, pumpHealthUrl, err) + return false + } + if res.StatusCode >= 400 { + glog.Errorf("Error response %v", res.StatusCode) + return false + } + body, err := ioutil.ReadAll(res.Body) + if err != nil { + glog.Errorf("cluster:[%s] read response body failed,error:%v", info.ClusterName, err) + return false + } + healths := pumpStatus{} + err = json.Unmarshal(body, &healths) + if err != nil { + glog.Errorf("cluster:[%s] unmarshal failed,error:%v", info.ClusterName, err) + return false + } + for _, status := range healths.StatusMap { + if status.State != "online" { + glog.Errorf("cluster:[%s] pump's state is not online", info.ClusterName) + return false + } + } + return true +} + +type drainerStatus struct { + PumpPos map[string]int64 `json:"PumpPos"` + Synced bool `json:"Synced"` + LastTS int64 `json:"LastTS"` + TsMap string `json:"TsMap"` +} + +func (oa *operatorActions) drainerHealth(info *TidbClusterInfo, hostName string) bool { + drainerHealthUrl := fmt.Sprintf("%s.%s-drainer.%s:8249/status", hostName, info.ClusterName, info.Namespace) + res, err := http.Get(drainerHealthUrl) + if err != nil { + glog.Errorf("cluster:[%s] call %s failed,error:%v", info.ClusterName, drainerHealthUrl, err) + return false + } + if res.StatusCode >= 400 { + glog.Errorf("Error response %v", res.StatusCode) + return false + } + body, err := ioutil.ReadAll(res.Body) + if err != nil { + glog.Errorf("cluster:[%s] read response body failed,error:%v", info.ClusterName, err) + return false + } + healths := drainerStatus{} + err = json.Unmarshal(body, &healths) + if err != nil { + glog.Errorf("cluster:[%s] unmarshal failed,error:%v", info.ClusterName, err) + return false + } + return len(healths.PumpPos) > 0 && healths.Synced } diff --git a/tests/backup/backupcase.go b/tests/backup/backupcase.go index dea051e9a4..3510164e0b 100644 --- a/tests/backup/backupcase.go +++ b/tests/backup/backupcase.go @@ -14,6 +14,9 @@ package backup import ( + "fmt" + "time" + "github.com/golang/glog" "github.com/pingcap/tidb-operator/tests" ) @@ -78,5 +81,40 @@ func (bc *BackupCase) Run() error { return err } + err = bc.operator.DeployIncrementalBackup(bc.srcCluster, bc.desCluster) + if err != nil { + return err + } + + err = bc.operator.CheckIncrementalBackup(bc.srcCluster) + if err != nil { + return err + } + + glog.Infof("waiting 1 minutes for binlog to work") + time.Sleep(1 * time.Minute) + + glog.Infof("cluster[%s] begin insert data") + go bc.operator.BeginInsertDataTo(bc.srcCluster) + + time.Sleep(30 * time.Second) + + glog.Infof("cluster[%s] stop insert data") + bc.operator.StopInsertDataTo(bc.srcCluster) + + time.Sleep(5 * time.Second) + + srcCount, err := bc.srcCluster.QueryCount() + if err != nil { + return err + } + desCount, err := bc.desCluster.QueryCount() + if err != nil { + return err + } + if srcCount != desCount { + return fmt.Errorf("cluster:[%s] the src cluster data[%d] is not equals des cluster data[%d]", bc.srcCluster.FullName(), srcCount, desCount) + } + return nil } diff --git a/tests/cmd/e2e/main.go b/tests/cmd/e2e/main.go index 90f1740891..313c89c251 100644 --- a/tests/cmd/e2e/main.go +++ b/tests/cmd/e2e/main.go @@ -95,6 +95,9 @@ func main() { StorageClassName: "local-storage", Password: "admin", InitSql: initSql, + UserName: "root", + InitSecretName: "demo-set-secret", + BackupSecretName: "demo-backup-secret", Resources: map[string]string{ "pd.resources.limits.cpu": "1000m", "pd.resources.limits.memory": "2Gi", @@ -187,6 +190,9 @@ func main() { StorageClassName: "local-storage", Password: "admin", InitSql: initSql, + UserName: "root", + InitSecretName: "demo2-set-secret", + BackupSecretName: "demo2-backup-secret", Resources: map[string]string{ "pd.resources.limits.cpu": "1000m", "pd.resources.limits.memory": "2Gi", From 0f934aae5a11d600ea85b20cc50508cdc4e794a4 Mon Sep 17 00:00:00 2001 From: xiaojingchen Date: Wed, 20 Mar 2019 12:19:37 +0800 Subject: [PATCH 20/22] fix e2e can not make (#331) --- go.mod | 9 +- go.sum | 2 + vendor/github.com/google/uuid/.travis.yml | 9 + vendor/github.com/google/uuid/CONTRIBUTING.md | 10 + vendor/github.com/google/uuid/CONTRIBUTORS | 9 + vendor/github.com/google/uuid/LICENSE | 27 +++ vendor/github.com/google/uuid/README.md | 19 ++ vendor/github.com/google/uuid/dce.go | 80 +++++++ vendor/github.com/google/uuid/doc.go | 12 ++ vendor/github.com/google/uuid/hash.go | 53 +++++ vendor/github.com/google/uuid/marshal.go | 37 ++++ vendor/github.com/google/uuid/node.go | 89 ++++++++ vendor/github.com/google/uuid/node_js.go | 12 ++ vendor/github.com/google/uuid/node_net.go | 33 +++ vendor/github.com/google/uuid/sql.go | 59 ++++++ vendor/github.com/google/uuid/time.go | 123 +++++++++++ vendor/github.com/google/uuid/util.go | 43 ++++ vendor/github.com/google/uuid/uuid.go | 198 ++++++++++++++++++ vendor/github.com/google/uuid/version1.go | 44 ++++ vendor/github.com/google/uuid/version4.go | 38 ++++ vendor/github.com/pborman/uuid/.travis.yml | 10 + .../github.com/pborman/uuid/CONTRIBUTING.md | 10 + vendor/github.com/pborman/uuid/CONTRIBUTORS | 1 + vendor/github.com/pborman/uuid/LICENSE | 27 +++ vendor/github.com/pborman/uuid/README.md | 15 ++ vendor/github.com/pborman/uuid/dce.go | 84 ++++++++ vendor/github.com/pborman/uuid/doc.go | 13 ++ vendor/github.com/pborman/uuid/go.mod | 3 + vendor/github.com/pborman/uuid/go.sum | 2 + vendor/github.com/pborman/uuid/hash.go | 53 +++++ vendor/github.com/pborman/uuid/marshal.go | 85 ++++++++ vendor/github.com/pborman/uuid/node.go | 50 +++++ vendor/github.com/pborman/uuid/sql.go | 68 ++++++ vendor/github.com/pborman/uuid/time.go | 57 +++++ vendor/github.com/pborman/uuid/util.go | 32 +++ vendor/github.com/pborman/uuid/uuid.go | 162 ++++++++++++++ vendor/github.com/pborman/uuid/version1.go | 23 ++ vendor/github.com/pborman/uuid/version4.go | 26 +++ .../k8s.io/apimachinery/pkg/util/uuid/BUILD | 29 +++ .../k8s.io/apimachinery/pkg/util/uuid/uuid.go | 43 ++++ vendor/k8s.io/client-go/pkg/version/base.go | 4 +- vendor/modules.txt | 9 +- 42 files changed, 1704 insertions(+), 8 deletions(-) create mode 100644 vendor/github.com/google/uuid/.travis.yml create mode 100644 vendor/github.com/google/uuid/CONTRIBUTING.md create mode 100644 vendor/github.com/google/uuid/CONTRIBUTORS create mode 100644 vendor/github.com/google/uuid/LICENSE create mode 100644 vendor/github.com/google/uuid/README.md create mode 100644 vendor/github.com/google/uuid/dce.go create mode 100644 vendor/github.com/google/uuid/doc.go create mode 100644 vendor/github.com/google/uuid/hash.go create mode 100644 vendor/github.com/google/uuid/marshal.go create mode 100644 vendor/github.com/google/uuid/node.go create mode 100644 vendor/github.com/google/uuid/node_js.go create mode 100644 vendor/github.com/google/uuid/node_net.go create mode 100644 vendor/github.com/google/uuid/sql.go create mode 100644 vendor/github.com/google/uuid/time.go create mode 100644 vendor/github.com/google/uuid/util.go create mode 100644 vendor/github.com/google/uuid/uuid.go create mode 100644 vendor/github.com/google/uuid/version1.go create mode 100644 vendor/github.com/google/uuid/version4.go create mode 100644 vendor/github.com/pborman/uuid/.travis.yml create mode 100644 vendor/github.com/pborman/uuid/CONTRIBUTING.md create mode 100644 vendor/github.com/pborman/uuid/CONTRIBUTORS create mode 100644 vendor/github.com/pborman/uuid/LICENSE create mode 100644 vendor/github.com/pborman/uuid/README.md create mode 100644 vendor/github.com/pborman/uuid/dce.go create mode 100644 vendor/github.com/pborman/uuid/doc.go create mode 100644 vendor/github.com/pborman/uuid/go.mod create mode 100644 vendor/github.com/pborman/uuid/go.sum create mode 100644 vendor/github.com/pborman/uuid/hash.go create mode 100644 vendor/github.com/pborman/uuid/marshal.go create mode 100644 vendor/github.com/pborman/uuid/node.go create mode 100644 vendor/github.com/pborman/uuid/sql.go create mode 100644 vendor/github.com/pborman/uuid/time.go create mode 100644 vendor/github.com/pborman/uuid/util.go create mode 100644 vendor/github.com/pborman/uuid/uuid.go create mode 100644 vendor/github.com/pborman/uuid/version1.go create mode 100644 vendor/github.com/pborman/uuid/version4.go create mode 100644 vendor/k8s.io/apimachinery/pkg/util/uuid/BUILD create mode 100644 vendor/k8s.io/apimachinery/pkg/util/uuid/uuid.go diff --git a/go.mod b/go.mod index f97543c019..214e9bf1c8 100644 --- a/go.mod +++ b/go.mod @@ -33,7 +33,7 @@ require ( github.com/hpcloud/tail v1.0.0 // indirect github.com/jonboulle/clockwork v0.1.0 // indirect github.com/json-iterator/go v1.1.5 // indirect - github.com/juju/errors v0.0.0-20180806074554-22422dad46e1 // indirect + github.com/juju/errors v0.0.0-20180806074554-22422dad46e1 github.com/juju/loggo v0.0.0-20180524022052-584905176618 // indirect github.com/juju/testing v0.0.0-20180920084828-472a3e8b2073 // indirect github.com/matttproud/golang_protobuf_extensions v1.0.1 // indirect @@ -42,13 +42,14 @@ require ( github.com/montanaflynn/stats v0.0.0-20171201202039-1bf9dbcd8cbe // indirect github.com/onsi/ginkgo v1.6.0 github.com/onsi/gomega v1.4.1 + github.com/pborman/uuid v1.2.0 // indirect github.com/pingcap/check v0.0.0-20171206051426-1c287c953996 // indirect github.com/pingcap/errors v0.11.0 github.com/pingcap/kvproto v0.0.0-20180606093822-b7ba8ea1c0b4 github.com/pingcap/pd v2.1.0-beta+incompatible github.com/pkg/errors v0.8.0 // indirect github.com/pmezard/go-difflib v1.0.0 // indirect - github.com/prometheus/client_golang v0.8.0 // indirect + github.com/prometheus/client_golang v0.8.0 github.com/prometheus/client_model v0.0.0-20180712105110-5c3871d89910 // indirect github.com/prometheus/common v0.0.0-20180801064454-c7de2306084e // indirect github.com/prometheus/procfs v0.0.0-20180725123919-05ee40e3a273 // indirect @@ -64,7 +65,7 @@ require ( go.uber.org/multierr v1.1.0 // indirect go.uber.org/zap v1.9.1 // indirect golang.org/x/crypto v0.0.0-20180807104621-f027049dab0a // indirect - golang.org/x/net v0.0.0-20180808004115-f9ce57c11b24 // indirect + golang.org/x/net v0.0.0-20180808004115-f9ce57c11b24 golang.org/x/sync v0.0.0-20181108010431-42b317875d0f // indirect golang.org/x/sys v0.0.0-20180807162357-acbc56fc7007 // indirect golang.org/x/text v0.3.0 // indirect @@ -79,7 +80,7 @@ require ( gopkg.in/mgo.v2 v2.0.0-20180705113604-9856a29383ce // indirect gopkg.in/natefinch/lumberjack.v2 v2.0.0-20170531160350-a96e63847dc3 // indirect gopkg.in/tomb.v1 v1.0.0-20141024135613-dd632973f1e7 // indirect - gopkg.in/yaml.v2 v2.2.1 // indirect + gopkg.in/yaml.v2 v2.2.1 k8s.io/api v0.0.0-20180308224125-73d903622b73 k8s.io/apiextensions-apiserver v0.0.0-20180426153726-e8ab413e0ae1 // indirect k8s.io/apimachinery v0.0.0-20180228050457-302974c03f7e diff --git a/go.sum b/go.sum index 9f157b7084..f4cc32ccee 100644 --- a/go.sum +++ b/go.sum @@ -41,6 +41,7 @@ github.com/google/btree v0.0.0-20180124185431-e89373fe6b4a h1:ZJu5NB1Bk5ms4vw0Xu github.com/google/btree v0.0.0-20180124185431-e89373fe6b4a/go.mod h1:lNA+9X1NB3Zf8V7Ke586lFgjr2dZNuvo3lPJSGZ5JPQ= github.com/google/gofuzz v0.0.0-20170612174753-24818f796faf h1:+RRA9JqSOZFfKrOeqr2z77+8R2RKyh8PG66dcu1V0ck= github.com/google/gofuzz v0.0.0-20170612174753-24818f796faf/go.mod h1:HP5RmnzzSNb993RKQDq4+1A4ia9nllfqcQFTQJedwGI= +github.com/google/uuid v1.0.0/go.mod h1:TIyPZe4MgqvfeYDBFedMoGGpEw/LqOeaOT+nhxU+yHo= github.com/googleapis/gnostic v0.2.0 h1:l6N3VoaVzTncYYW+9yOz2LJJammFZGBO13sqgEhpy9g= github.com/googleapis/gnostic v0.2.0/go.mod h1:sJBsCZ4ayReDTBIg8b9dl28c5xFWyhBTVRp3pOg5EKY= github.com/gorilla/context v1.1.1 h1:AWwleXJkX/nhcU9bZSnZoi3h/qGYqQAGhq6zZe/aQW8= @@ -79,6 +80,7 @@ github.com/onsi/ginkgo v1.6.0 h1:Ix8l273rp3QzYgXSR+c8d1fTG7UPgYkOSELPhiY/YGw= github.com/onsi/ginkgo v1.6.0/go.mod h1:lLunBs/Ym6LB5Z9jYTR76FiuTmxDTDusOGeTQH+WWjE= github.com/onsi/gomega v1.4.1 h1:PZSj/UFNaVp3KxrzHOcS7oyuWA7LoOY/77yCTEFu21U= github.com/onsi/gomega v1.4.1/go.mod h1:C1qb7wdrVGGVU+Z6iS04AVkA3Q65CEZX59MT0QO5uiA= +github.com/pborman/uuid v1.2.0/go.mod h1:X/NO0urCmaxf9VXbdlT7C2Yzkj2IKimNn4k+gtPdI/k= github.com/pingcap/check v0.0.0-20171206051426-1c287c953996 h1:ZBdiJCMan6GSo/aPAM7gywcUKa0z58gczVrnG6TQnAQ= github.com/pingcap/check v0.0.0-20171206051426-1c287c953996/go.mod h1:B1+S9LNcuMyLH/4HMTViQOJevkGiik3wW2AN9zb2fNQ= github.com/pingcap/errors v0.11.0 h1:DCJQB8jrHbQ1VVlMFIrbj2ApScNNotVmkSNplu2yUt4= diff --git a/vendor/github.com/google/uuid/.travis.yml b/vendor/github.com/google/uuid/.travis.yml new file mode 100644 index 0000000000..d8156a60ba --- /dev/null +++ b/vendor/github.com/google/uuid/.travis.yml @@ -0,0 +1,9 @@ +language: go + +go: + - 1.4.3 + - 1.5.3 + - tip + +script: + - go test -v ./... diff --git a/vendor/github.com/google/uuid/CONTRIBUTING.md b/vendor/github.com/google/uuid/CONTRIBUTING.md new file mode 100644 index 0000000000..04fdf09f13 --- /dev/null +++ b/vendor/github.com/google/uuid/CONTRIBUTING.md @@ -0,0 +1,10 @@ +# How to contribute + +We definitely welcome patches and contribution to this project! + +### Legal requirements + +In order to protect both you and ourselves, you will need to sign the +[Contributor License Agreement](https://cla.developers.google.com/clas). + +You may have already signed it for other Google projects. diff --git a/vendor/github.com/google/uuid/CONTRIBUTORS b/vendor/github.com/google/uuid/CONTRIBUTORS new file mode 100644 index 0000000000..b4bb97f6bc --- /dev/null +++ b/vendor/github.com/google/uuid/CONTRIBUTORS @@ -0,0 +1,9 @@ +Paul Borman +bmatsuo +shawnps +theory +jboverfelt +dsymonds +cd1 +wallclockbuilder +dansouza diff --git a/vendor/github.com/google/uuid/LICENSE b/vendor/github.com/google/uuid/LICENSE new file mode 100644 index 0000000000..5dc68268d9 --- /dev/null +++ b/vendor/github.com/google/uuid/LICENSE @@ -0,0 +1,27 @@ +Copyright (c) 2009,2014 Google Inc. All rights reserved. + +Redistribution and use in source and binary forms, with or without +modification, are permitted provided that the following conditions are +met: + + * Redistributions of source code must retain the above copyright +notice, this list of conditions and the following disclaimer. + * Redistributions in binary form must reproduce the above +copyright notice, this list of conditions and the following disclaimer +in the documentation and/or other materials provided with the +distribution. + * Neither the name of Google Inc. nor the names of its +contributors may be used to endorse or promote products derived from +this software without specific prior written permission. + +THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS +"AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT +LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR +A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT +OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, +SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT +LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, +DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY +THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT +(INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE +OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. diff --git a/vendor/github.com/google/uuid/README.md b/vendor/github.com/google/uuid/README.md new file mode 100644 index 0000000000..9d92c11f16 --- /dev/null +++ b/vendor/github.com/google/uuid/README.md @@ -0,0 +1,19 @@ +# uuid ![build status](https://travis-ci.org/google/uuid.svg?branch=master) +The uuid package generates and inspects UUIDs based on +[RFC 4122](http://tools.ietf.org/html/rfc4122) +and DCE 1.1: Authentication and Security Services. + +This package is based on the github.com/pborman/uuid package (previously named +code.google.com/p/go-uuid). It differs from these earlier packages in that +a UUID is a 16 byte array rather than a byte slice. One loss due to this +change is the ability to represent an invalid UUID (vs a NIL UUID). + +###### Install +`go get github.com/google/uuid` + +###### Documentation +[![GoDoc](https://godoc.org/github.com/google/uuid?status.svg)](http://godoc.org/github.com/google/uuid) + +Full `go doc` style documentation for the package can be viewed online without +installing this package by using the GoDoc site here: +http://godoc.org/github.com/google/uuid diff --git a/vendor/github.com/google/uuid/dce.go b/vendor/github.com/google/uuid/dce.go new file mode 100644 index 0000000000..fa820b9d30 --- /dev/null +++ b/vendor/github.com/google/uuid/dce.go @@ -0,0 +1,80 @@ +// Copyright 2016 Google Inc. All rights reserved. +// Use of this source code is governed by a BSD-style +// license that can be found in the LICENSE file. + +package uuid + +import ( + "encoding/binary" + "fmt" + "os" +) + +// A Domain represents a Version 2 domain +type Domain byte + +// Domain constants for DCE Security (Version 2) UUIDs. +const ( + Person = Domain(0) + Group = Domain(1) + Org = Domain(2) +) + +// NewDCESecurity returns a DCE Security (Version 2) UUID. +// +// The domain should be one of Person, Group or Org. +// On a POSIX system the id should be the users UID for the Person +// domain and the users GID for the Group. The meaning of id for +// the domain Org or on non-POSIX systems is site defined. +// +// For a given domain/id pair the same token may be returned for up to +// 7 minutes and 10 seconds. +func NewDCESecurity(domain Domain, id uint32) (UUID, error) { + uuid, err := NewUUID() + if err == nil { + uuid[6] = (uuid[6] & 0x0f) | 0x20 // Version 2 + uuid[9] = byte(domain) + binary.BigEndian.PutUint32(uuid[0:], id) + } + return uuid, err +} + +// NewDCEPerson returns a DCE Security (Version 2) UUID in the person +// domain with the id returned by os.Getuid. +// +// NewDCESecurity(Person, uint32(os.Getuid())) +func NewDCEPerson() (UUID, error) { + return NewDCESecurity(Person, uint32(os.Getuid())) +} + +// NewDCEGroup returns a DCE Security (Version 2) UUID in the group +// domain with the id returned by os.Getgid. +// +// NewDCESecurity(Group, uint32(os.Getgid())) +func NewDCEGroup() (UUID, error) { + return NewDCESecurity(Group, uint32(os.Getgid())) +} + +// Domain returns the domain for a Version 2 UUID. Domains are only defined +// for Version 2 UUIDs. +func (uuid UUID) Domain() Domain { + return Domain(uuid[9]) +} + +// ID returns the id for a Version 2 UUID. IDs are only defined for Version 2 +// UUIDs. +func (uuid UUID) ID() uint32 { + return binary.BigEndian.Uint32(uuid[0:4]) +} + +func (d Domain) String() string { + switch d { + case Person: + return "Person" + case Group: + return "Group" + case Org: + return "Org" + } + return fmt.Sprintf("Domain%d", int(d)) +} diff --git a/vendor/github.com/google/uuid/doc.go b/vendor/github.com/google/uuid/doc.go new file mode 100644 index 0000000000..5b8a4b9af8 --- /dev/null +++ b/vendor/github.com/google/uuid/doc.go @@ -0,0 +1,12 @@ +// Copyright 2016 Google Inc. All rights reserved. +// Use of this source code is governed by a BSD-style +// license that can be found in the LICENSE file. + +// Package uuid generates and inspects UUIDs. +// +// UUIDs are based on RFC 4122 and DCE 1.1: Authentication and Security +// Services. +// +// A UUID is a 16 byte (128 bit) array. UUIDs may be used as keys to +// maps or compared directly. +package uuid diff --git a/vendor/github.com/google/uuid/hash.go b/vendor/github.com/google/uuid/hash.go new file mode 100644 index 0000000000..b174616315 --- /dev/null +++ b/vendor/github.com/google/uuid/hash.go @@ -0,0 +1,53 @@ +// Copyright 2016 Google Inc. All rights reserved. +// Use of this source code is governed by a BSD-style +// license that can be found in the LICENSE file. + +package uuid + +import ( + "crypto/md5" + "crypto/sha1" + "hash" +) + +// Well known namespace IDs and UUIDs +var ( + NameSpaceDNS = Must(Parse("6ba7b810-9dad-11d1-80b4-00c04fd430c8")) + NameSpaceURL = Must(Parse("6ba7b811-9dad-11d1-80b4-00c04fd430c8")) + NameSpaceOID = Must(Parse("6ba7b812-9dad-11d1-80b4-00c04fd430c8")) + NameSpaceX500 = Must(Parse("6ba7b814-9dad-11d1-80b4-00c04fd430c8")) + Nil UUID // empty UUID, all zeros +) + +// NewHash returns a new UUID derived from the hash of space concatenated with +// data generated by h. The hash should be at least 16 byte in length. The +// first 16 bytes of the hash are used to form the UUID. The version of the +// UUID will be the lower 4 bits of version. NewHash is used to implement +// NewMD5 and NewSHA1. +func NewHash(h hash.Hash, space UUID, data []byte, version int) UUID { + h.Reset() + h.Write(space[:]) + h.Write(data) + s := h.Sum(nil) + var uuid UUID + copy(uuid[:], s) + uuid[6] = (uuid[6] & 0x0f) | uint8((version&0xf)<<4) + uuid[8] = (uuid[8] & 0x3f) | 0x80 // RFC 4122 variant + return uuid +} + +// NewMD5 returns a new MD5 (Version 3) UUID based on the +// supplied name space and data. It is the same as calling: +// +// NewHash(md5.New(), space, data, 3) +func NewMD5(space UUID, data []byte) UUID { + return NewHash(md5.New(), space, data, 3) +} + +// NewSHA1 returns a new SHA1 (Version 5) UUID based on the +// supplied name space and data. It is the same as calling: +// +// NewHash(sha1.New(), space, data, 5) +func NewSHA1(space UUID, data []byte) UUID { + return NewHash(sha1.New(), space, data, 5) +} diff --git a/vendor/github.com/google/uuid/marshal.go b/vendor/github.com/google/uuid/marshal.go new file mode 100644 index 0000000000..7f9e0c6c0e --- /dev/null +++ b/vendor/github.com/google/uuid/marshal.go @@ -0,0 +1,37 @@ +// Copyright 2016 Google Inc. All rights reserved. +// Use of this source code is governed by a BSD-style +// license that can be found in the LICENSE file. + +package uuid + +import "fmt" + +// MarshalText implements encoding.TextMarshaler. +func (uuid UUID) MarshalText() ([]byte, error) { + var js [36]byte + encodeHex(js[:], uuid) + return js[:], nil +} + +// UnmarshalText implements encoding.TextUnmarshaler. +func (uuid *UUID) UnmarshalText(data []byte) error { + id, err := ParseBytes(data) + if err == nil { + *uuid = id + } + return err +} + +// MarshalBinary implements encoding.BinaryMarshaler. +func (uuid UUID) MarshalBinary() ([]byte, error) { + return uuid[:], nil +} + +// UnmarshalBinary implements encoding.BinaryUnmarshaler. +func (uuid *UUID) UnmarshalBinary(data []byte) error { + if len(data) != 16 { + return fmt.Errorf("invalid UUID (got %d bytes)", len(data)) + } + copy(uuid[:], data) + return nil +} diff --git a/vendor/github.com/google/uuid/node.go b/vendor/github.com/google/uuid/node.go new file mode 100644 index 0000000000..3e4e90dc44 --- /dev/null +++ b/vendor/github.com/google/uuid/node.go @@ -0,0 +1,89 @@ +// Copyright 2016 Google Inc. All rights reserved. +// Use of this source code is governed by a BSD-style +// license that can be found in the LICENSE file. + +package uuid + +import ( + "sync" +) + +var ( + nodeMu sync.Mutex + ifname string // name of interface being used + nodeID [6]byte // hardware for version 1 UUIDs + zeroID [6]byte // nodeID with only 0's +) + +// NodeInterface returns the name of the interface from which the NodeID was +// derived. The interface "user" is returned if the NodeID was set by +// SetNodeID. +func NodeInterface() string { + defer nodeMu.Unlock() + nodeMu.Lock() + return ifname +} + +// SetNodeInterface selects the hardware address to be used for Version 1 UUIDs. +// If name is "" then the first usable interface found will be used or a random +// Node ID will be generated. If a named interface cannot be found then false +// is returned. +// +// SetNodeInterface never fails when name is "". +func SetNodeInterface(name string) bool { + defer nodeMu.Unlock() + nodeMu.Lock() + return setNodeInterface(name) +} + +func setNodeInterface(name string) bool { + iname, addr := getHardwareInterface(name) // null implementation for js + if iname != "" && addr != nil { + ifname = iname + copy(nodeID[:], addr) + return true + } + + // We found no interfaces with a valid hardware address. If name + // does not specify a specific interface generate a random Node ID + // (section 4.1.6) + if name == "" { + randomBits(nodeID[:]) + return true + } + return false +} + +// NodeID returns a slice of a copy of the current Node ID, setting the Node ID +// if not already set. +func NodeID() []byte { + defer nodeMu.Unlock() + nodeMu.Lock() + if nodeID == zeroID { + setNodeInterface("") + } + nid := nodeID + return nid[:] +} + +// SetNodeID sets the Node ID to be used for Version 1 UUIDs. The first 6 bytes +// of id are used. If id is less than 6 bytes then false is returned and the +// Node ID is not set. +func SetNodeID(id []byte) bool { + if len(id) < 6 { + return false + } + defer nodeMu.Unlock() + nodeMu.Lock() + copy(nodeID[:], id) + ifname = "user" + return true +} + +// NodeID returns the 6 byte node id encoded in uuid. It returns nil if uuid is +// not valid. The NodeID is only well defined for version 1 and 2 UUIDs. +func (uuid UUID) NodeID() []byte { + var node [6]byte + copy(node[:], uuid[10:]) + return node[:] +} diff --git a/vendor/github.com/google/uuid/node_js.go b/vendor/github.com/google/uuid/node_js.go new file mode 100644 index 0000000000..24b78edc90 --- /dev/null +++ b/vendor/github.com/google/uuid/node_js.go @@ -0,0 +1,12 @@ +// Copyright 2017 Google Inc. All rights reserved. +// Use of this source code is governed by a BSD-style +// license that can be found in the LICENSE file. + +// +build js + +package uuid + +// getHardwareInterface returns nil values for the JS version of the code. +// This remvoves the "net" dependency, because it is not used in the browser. +// Using the "net" library inflates the size of the transpiled JS code by 673k bytes. +func getHardwareInterface(name string) (string, []byte) { return "", nil } diff --git a/vendor/github.com/google/uuid/node_net.go b/vendor/github.com/google/uuid/node_net.go new file mode 100644 index 0000000000..0cbbcddbd6 --- /dev/null +++ b/vendor/github.com/google/uuid/node_net.go @@ -0,0 +1,33 @@ +// Copyright 2017 Google Inc. All rights reserved. +// Use of this source code is governed by a BSD-style +// license that can be found in the LICENSE file. + +// +build !js + +package uuid + +import "net" + +var interfaces []net.Interface // cached list of interfaces + +// getHardwareInterface returns the name and hardware address of interface name. +// If name is "" then the name and hardware address of one of the system's +// interfaces is returned. If no interfaces are found (name does not exist or +// there are no interfaces) then "", nil is returned. +// +// Only addresses of at least 6 bytes are returned. +func getHardwareInterface(name string) (string, []byte) { + if interfaces == nil { + var err error + interfaces, err = net.Interfaces() + if err != nil { + return "", nil + } + } + for _, ifs := range interfaces { + if len(ifs.HardwareAddr) >= 6 && (name == "" || name == ifs.Name) { + return ifs.Name, ifs.HardwareAddr + } + } + return "", nil +} diff --git a/vendor/github.com/google/uuid/sql.go b/vendor/github.com/google/uuid/sql.go new file mode 100644 index 0000000000..f326b54db3 --- /dev/null +++ b/vendor/github.com/google/uuid/sql.go @@ -0,0 +1,59 @@ +// Copyright 2016 Google Inc. All rights reserved. +// Use of this source code is governed by a BSD-style +// license that can be found in the LICENSE file. + +package uuid + +import ( + "database/sql/driver" + "fmt" +) + +// Scan implements sql.Scanner so UUIDs can be read from databases transparently +// Currently, database types that map to string and []byte are supported. Please +// consult database-specific driver documentation for matching types. +func (uuid *UUID) Scan(src interface{}) error { + switch src := src.(type) { + case nil: + return nil + + case string: + // if an empty UUID comes from a table, we return a null UUID + if src == "" { + return nil + } + + // see Parse for required string format + u, err := Parse(src) + if err != nil { + return fmt.Errorf("Scan: %v", err) + } + + *uuid = u + + case []byte: + // if an empty UUID comes from a table, we return a null UUID + if len(src) == 0 { + return nil + } + + // assumes a simple slice of bytes if 16 bytes + // otherwise attempts to parse + if len(src) != 16 { + return uuid.Scan(string(src)) + } + copy((*uuid)[:], src) + + default: + return fmt.Errorf("Scan: unable to scan type %T into UUID", src) + } + + return nil +} + +// Value implements sql.Valuer so that UUIDs can be written to databases +// transparently. Currently, UUIDs map to strings. Please consult +// database-specific driver documentation for matching types. +func (uuid UUID) Value() (driver.Value, error) { + return uuid.String(), nil +} diff --git a/vendor/github.com/google/uuid/time.go b/vendor/github.com/google/uuid/time.go new file mode 100644 index 0000000000..e6ef06cdc8 --- /dev/null +++ b/vendor/github.com/google/uuid/time.go @@ -0,0 +1,123 @@ +// Copyright 2016 Google Inc. All rights reserved. +// Use of this source code is governed by a BSD-style +// license that can be found in the LICENSE file. + +package uuid + +import ( + "encoding/binary" + "sync" + "time" +) + +// A Time represents a time as the number of 100's of nanoseconds since 15 Oct +// 1582. +type Time int64 + +const ( + lillian = 2299160 // Julian day of 15 Oct 1582 + unix = 2440587 // Julian day of 1 Jan 1970 + epoch = unix - lillian // Days between epochs + g1582 = epoch * 86400 // seconds between epochs + g1582ns100 = g1582 * 10000000 // 100s of a nanoseconds between epochs +) + +var ( + timeMu sync.Mutex + lasttime uint64 // last time we returned + clockSeq uint16 // clock sequence for this run + + timeNow = time.Now // for testing +) + +// UnixTime converts t the number of seconds and nanoseconds using the Unix +// epoch of 1 Jan 1970. +func (t Time) UnixTime() (sec, nsec int64) { + sec = int64(t - g1582ns100) + nsec = (sec % 10000000) * 100 + sec /= 10000000 + return sec, nsec +} + +// GetTime returns the current Time (100s of nanoseconds since 15 Oct 1582) and +// clock sequence as well as adjusting the clock sequence as needed. An error +// is returned if the current time cannot be determined. +func GetTime() (Time, uint16, error) { + defer timeMu.Unlock() + timeMu.Lock() + return getTime() +} + +func getTime() (Time, uint16, error) { + t := timeNow() + + // If we don't have a clock sequence already, set one. + if clockSeq == 0 { + setClockSequence(-1) + } + now := uint64(t.UnixNano()/100) + g1582ns100 + + // If time has gone backwards with this clock sequence then we + // increment the clock sequence + if now <= lasttime { + clockSeq = ((clockSeq + 1) & 0x3fff) | 0x8000 + } + lasttime = now + return Time(now), clockSeq, nil +} + +// ClockSequence returns the current clock sequence, generating one if not +// already set. The clock sequence is only used for Version 1 UUIDs. +// +// The uuid package does not use global static storage for the clock sequence or +// the last time a UUID was generated. Unless SetClockSequence is used, a new +// random clock sequence is generated the first time a clock sequence is +// requested by ClockSequence, GetTime, or NewUUID. (section 4.2.1.1) +func ClockSequence() int { + defer timeMu.Unlock() + timeMu.Lock() + return clockSequence() +} + +func clockSequence() int { + if clockSeq == 0 { + setClockSequence(-1) + } + return int(clockSeq & 0x3fff) +} + +// SetClockSequence sets the clock sequence to the lower 14 bits of seq. Setting to +// -1 causes a new sequence to be generated. +func SetClockSequence(seq int) { + defer timeMu.Unlock() + timeMu.Lock() + setClockSequence(seq) +} + +func setClockSequence(seq int) { + if seq == -1 { + var b [2]byte + randomBits(b[:]) // clock sequence + seq = int(b[0])<<8 | int(b[1]) + } + oldSeq := clockSeq + clockSeq = uint16(seq&0x3fff) | 0x8000 // Set our variant + if oldSeq != clockSeq { + lasttime = 0 + } +} + +// Time returns the time in 100s of nanoseconds since 15 Oct 1582 encoded in +// uuid. The time is only defined for version 1 and 2 UUIDs. +func (uuid UUID) Time() Time { + time := int64(binary.BigEndian.Uint32(uuid[0:4])) + time |= int64(binary.BigEndian.Uint16(uuid[4:6])) << 32 + time |= int64(binary.BigEndian.Uint16(uuid[6:8])&0xfff) << 48 + return Time(time) +} + +// ClockSequence returns the clock sequence encoded in uuid. +// The clock sequence is only well defined for version 1 and 2 UUIDs. +func (uuid UUID) ClockSequence() int { + return int(binary.BigEndian.Uint16(uuid[8:10])) & 0x3fff +} diff --git a/vendor/github.com/google/uuid/util.go b/vendor/github.com/google/uuid/util.go new file mode 100644 index 0000000000..5ea6c73780 --- /dev/null +++ b/vendor/github.com/google/uuid/util.go @@ -0,0 +1,43 @@ +// Copyright 2016 Google Inc. All rights reserved. +// Use of this source code is governed by a BSD-style +// license that can be found in the LICENSE file. + +package uuid + +import ( + "io" +) + +// randomBits completely fills slice b with random data. +func randomBits(b []byte) { + if _, err := io.ReadFull(rander, b); err != nil { + panic(err.Error()) // rand should never fail + } +} + +// xvalues returns the value of a byte as a hexadecimal digit or 255. +var xvalues = [256]byte{ + 255, 255, 255, 255, 255, 255, 255, 255, 255, 255, 255, 255, 255, 255, 255, 255, + 255, 255, 255, 255, 255, 255, 255, 255, 255, 255, 255, 255, 255, 255, 255, 255, + 255, 255, 255, 255, 255, 255, 255, 255, 255, 255, 255, 255, 255, 255, 255, 255, + 0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 255, 255, 255, 255, 255, 255, + 255, 10, 11, 12, 13, 14, 15, 255, 255, 255, 255, 255, 255, 255, 255, 255, + 255, 255, 255, 255, 255, 255, 255, 255, 255, 255, 255, 255, 255, 255, 255, 255, + 255, 10, 11, 12, 13, 14, 15, 255, 255, 255, 255, 255, 255, 255, 255, 255, + 255, 255, 255, 255, 255, 255, 255, 255, 255, 255, 255, 255, 255, 255, 255, 255, + 255, 255, 255, 255, 255, 255, 255, 255, 255, 255, 255, 255, 255, 255, 255, 255, + 255, 255, 255, 255, 255, 255, 255, 255, 255, 255, 255, 255, 255, 255, 255, 255, + 255, 255, 255, 255, 255, 255, 255, 255, 255, 255, 255, 255, 255, 255, 255, 255, + 255, 255, 255, 255, 255, 255, 255, 255, 255, 255, 255, 255, 255, 255, 255, 255, + 255, 255, 255, 255, 255, 255, 255, 255, 255, 255, 255, 255, 255, 255, 255, 255, + 255, 255, 255, 255, 255, 255, 255, 255, 255, 255, 255, 255, 255, 255, 255, 255, + 255, 255, 255, 255, 255, 255, 255, 255, 255, 255, 255, 255, 255, 255, 255, 255, + 255, 255, 255, 255, 255, 255, 255, 255, 255, 255, 255, 255, 255, 255, 255, 255, +} + +// xtob converts hex characters x1 and x2 into a byte. +func xtob(x1, x2 byte) (byte, bool) { + b1 := xvalues[x1] + b2 := xvalues[x2] + return (b1 << 4) | b2, b1 != 255 && b2 != 255 +} diff --git a/vendor/github.com/google/uuid/uuid.go b/vendor/github.com/google/uuid/uuid.go new file mode 100644 index 0000000000..7f3643fe9a --- /dev/null +++ b/vendor/github.com/google/uuid/uuid.go @@ -0,0 +1,198 @@ +// Copyright 2016 Google Inc. All rights reserved. +// Use of this source code is governed by a BSD-style +// license that can be found in the LICENSE file. + +package uuid + +import ( + "bytes" + "crypto/rand" + "encoding/hex" + "errors" + "fmt" + "io" + "strings" +) + +// A UUID is a 128 bit (16 byte) Universal Unique IDentifier as defined in RFC +// 4122. +type UUID [16]byte + +// A Version represents a UUID's version. +type Version byte + +// A Variant represents a UUID's variant. +type Variant byte + +// Constants returned by Variant. +const ( + Invalid = Variant(iota) // Invalid UUID + RFC4122 // The variant specified in RFC4122 + Reserved // Reserved, NCS backward compatibility. + Microsoft // Reserved, Microsoft Corporation backward compatibility. + Future // Reserved for future definition. +) + +var rander = rand.Reader // random function + +// Parse decodes s into a UUID or returns an error. Both the UUID form of +// xxxxxxxx-xxxx-xxxx-xxxx-xxxxxxxxxxxx and +// urn:uuid:xxxxxxxx-xxxx-xxxx-xxxx-xxxxxxxxxxxx are decoded. +func Parse(s string) (UUID, error) { + var uuid UUID + if len(s) != 36 { + if len(s) != 36+9 { + return uuid, fmt.Errorf("invalid UUID length: %d", len(s)) + } + if strings.ToLower(s[:9]) != "urn:uuid:" { + return uuid, fmt.Errorf("invalid urn prefix: %q", s[:9]) + } + s = s[9:] + } + if s[8] != '-' || s[13] != '-' || s[18] != '-' || s[23] != '-' { + return uuid, errors.New("invalid UUID format") + } + for i, x := range [16]int{ + 0, 2, 4, 6, + 9, 11, + 14, 16, + 19, 21, + 24, 26, 28, 30, 32, 34} { + v, ok := xtob(s[x], s[x+1]) + if !ok { + return uuid, errors.New("invalid UUID format") + } + uuid[i] = v + } + return uuid, nil +} + +// ParseBytes is like Parse, except it parses a byte slice instead of a string. +func ParseBytes(b []byte) (UUID, error) { + var uuid UUID + if len(b) != 36 { + if len(b) != 36+9 { + return uuid, fmt.Errorf("invalid UUID length: %d", len(b)) + } + if !bytes.Equal(bytes.ToLower(b[:9]), []byte("urn:uuid:")) { + return uuid, fmt.Errorf("invalid urn prefix: %q", b[:9]) + } + b = b[9:] + } + if b[8] != '-' || b[13] != '-' || b[18] != '-' || b[23] != '-' { + return uuid, errors.New("invalid UUID format") + } + for i, x := range [16]int{ + 0, 2, 4, 6, + 9, 11, + 14, 16, + 19, 21, + 24, 26, 28, 30, 32, 34} { + v, ok := xtob(b[x], b[x+1]) + if !ok { + return uuid, errors.New("invalid UUID format") + } + uuid[i] = v + } + return uuid, nil +} + +// FromBytes creates a new UUID from a byte slice. Returns an error if the slice +// does not have a length of 16. The bytes are copied from the slice. +func FromBytes(b []byte) (uuid UUID, err error) { + err = uuid.UnmarshalBinary(b) + return uuid, err +} + +// Must returns uuid if err is nil and panics otherwise. +func Must(uuid UUID, err error) UUID { + if err != nil { + panic(err) + } + return uuid +} + +// String returns the string form of uuid, xxxxxxxx-xxxx-xxxx-xxxx-xxxxxxxxxxxx +// , or "" if uuid is invalid. +func (uuid UUID) String() string { + var buf [36]byte + encodeHex(buf[:], uuid) + return string(buf[:]) +} + +// URN returns the RFC 2141 URN form of uuid, +// urn:uuid:xxxxxxxx-xxxx-xxxx-xxxx-xxxxxxxxxxxx, or "" if uuid is invalid. +func (uuid UUID) URN() string { + var buf [36 + 9]byte + copy(buf[:], "urn:uuid:") + encodeHex(buf[9:], uuid) + return string(buf[:]) +} + +func encodeHex(dst []byte, uuid UUID) { + hex.Encode(dst[:], uuid[:4]) + dst[8] = '-' + hex.Encode(dst[9:13], uuid[4:6]) + dst[13] = '-' + hex.Encode(dst[14:18], uuid[6:8]) + dst[18] = '-' + hex.Encode(dst[19:23], uuid[8:10]) + dst[23] = '-' + hex.Encode(dst[24:], uuid[10:]) +} + +// Variant returns the variant encoded in uuid. +func (uuid UUID) Variant() Variant { + switch { + case (uuid[8] & 0xc0) == 0x80: + return RFC4122 + case (uuid[8] & 0xe0) == 0xc0: + return Microsoft + case (uuid[8] & 0xe0) == 0xe0: + return Future + default: + return Reserved + } +} + +// Version returns the version of uuid. +func (uuid UUID) Version() Version { + return Version(uuid[6] >> 4) +} + +func (v Version) String() string { + if v > 15 { + return fmt.Sprintf("BAD_VERSION_%d", v) + } + return fmt.Sprintf("VERSION_%d", v) +} + +func (v Variant) String() string { + switch v { + case RFC4122: + return "RFC4122" + case Reserved: + return "Reserved" + case Microsoft: + return "Microsoft" + case Future: + return "Future" + case Invalid: + return "Invalid" + } + return fmt.Sprintf("BadVariant%d", int(v)) +} + +// SetRand sets the random number generator to r, which implements io.Reader. +// If r.Read returns an error when the package requests random data then +// a panic will be issued. +// +// Calling SetRand with nil sets the random number generator to the default +// generator. +func SetRand(r io.Reader) { + if r == nil { + rander = rand.Reader + return + } + rander = r +} diff --git a/vendor/github.com/google/uuid/version1.go b/vendor/github.com/google/uuid/version1.go new file mode 100644 index 0000000000..199a1ac654 --- /dev/null +++ b/vendor/github.com/google/uuid/version1.go @@ -0,0 +1,44 @@ +// Copyright 2016 Google Inc. All rights reserved. +// Use of this source code is governed by a BSD-style +// license that can be found in the LICENSE file. + +package uuid + +import ( + "encoding/binary" +) + +// NewUUID returns a Version 1 UUID based on the current NodeID and clock +// sequence, and the current time. If the NodeID has not been set by SetNodeID +// or SetNodeInterface then it will be set automatically. If the NodeID cannot +// be set NewUUID returns nil. If clock sequence has not been set by +// SetClockSequence then it will be set automatically. If GetTime fails to +// return the current NewUUID returns nil and an error. +// +// In most cases, New should be used. +func NewUUID() (UUID, error) { + nodeMu.Lock() + if nodeID == zeroID { + setNodeInterface("") + } + nodeMu.Unlock() + + var uuid UUID + now, seq, err := GetTime() + if err != nil { + return uuid, err + } + + timeLow := uint32(now & 0xffffffff) + timeMid := uint16((now >> 32) & 0xffff) + timeHi := uint16((now >> 48) & 0x0fff) + timeHi |= 0x1000 // Version 1 + + binary.BigEndian.PutUint32(uuid[0:], timeLow) + binary.BigEndian.PutUint16(uuid[4:], timeMid) + binary.BigEndian.PutUint16(uuid[6:], timeHi) + binary.BigEndian.PutUint16(uuid[8:], seq) + copy(uuid[10:], nodeID[:]) + + return uuid, nil +} diff --git a/vendor/github.com/google/uuid/version4.go b/vendor/github.com/google/uuid/version4.go new file mode 100644 index 0000000000..84af91c9f5 --- /dev/null +++ b/vendor/github.com/google/uuid/version4.go @@ -0,0 +1,38 @@ +// Copyright 2016 Google Inc. All rights reserved. +// Use of this source code is governed by a BSD-style +// license that can be found in the LICENSE file. + +package uuid + +import "io" + +// New creates a new random UUID or panics. New is equivalent to +// the expression +// +// uuid.Must(uuid.NewRandom()) +func New() UUID { + return Must(NewRandom()) +} + +// NewRandom returns a Random (Version 4) UUID. +// +// The strength of the UUIDs is based on the strength of the crypto/rand +// package. +// +// A note about uniqueness derived from the UUID Wikipedia entry: +// +// Randomly generated UUIDs have 122 random bits. One's annual risk of being +// hit by a meteorite is estimated to be one chance in 17 billion, that +// means the probability is about 0.00000000006 (6 × 10−11), +// equivalent to the odds of creating a few tens of trillions of UUIDs in a +// year and having one duplicate. +func NewRandom() (UUID, error) { + var uuid UUID + _, err := io.ReadFull(rander, uuid[:]) + if err != nil { + return Nil, err + } + uuid[6] = (uuid[6] & 0x0f) | 0x40 // Version 4 + uuid[8] = (uuid[8] & 0x3f) | 0x80 // Variant is 10 + return uuid, nil +} diff --git a/vendor/github.com/pborman/uuid/.travis.yml b/vendor/github.com/pborman/uuid/.travis.yml new file mode 100644 index 0000000000..3deb4a1243 --- /dev/null +++ b/vendor/github.com/pborman/uuid/.travis.yml @@ -0,0 +1,10 @@ +language: go + +go: + - "1.9" + - "1.10" + - "1.11" + - tip + +script: + - go test -v ./... diff --git a/vendor/github.com/pborman/uuid/CONTRIBUTING.md b/vendor/github.com/pborman/uuid/CONTRIBUTING.md new file mode 100644 index 0000000000..04fdf09f13 --- /dev/null +++ b/vendor/github.com/pborman/uuid/CONTRIBUTING.md @@ -0,0 +1,10 @@ +# How to contribute + +We definitely welcome patches and contribution to this project! + +### Legal requirements + +In order to protect both you and ourselves, you will need to sign the +[Contributor License Agreement](https://cla.developers.google.com/clas). + +You may have already signed it for other Google projects. diff --git a/vendor/github.com/pborman/uuid/CONTRIBUTORS b/vendor/github.com/pborman/uuid/CONTRIBUTORS new file mode 100644 index 0000000000..b382a04eda --- /dev/null +++ b/vendor/github.com/pborman/uuid/CONTRIBUTORS @@ -0,0 +1 @@ +Paul Borman diff --git a/vendor/github.com/pborman/uuid/LICENSE b/vendor/github.com/pborman/uuid/LICENSE new file mode 100644 index 0000000000..5dc68268d9 --- /dev/null +++ b/vendor/github.com/pborman/uuid/LICENSE @@ -0,0 +1,27 @@ +Copyright (c) 2009,2014 Google Inc. All rights reserved. + +Redistribution and use in source and binary forms, with or without +modification, are permitted provided that the following conditions are +met: + + * Redistributions of source code must retain the above copyright +notice, this list of conditions and the following disclaimer. + * Redistributions in binary form must reproduce the above +copyright notice, this list of conditions and the following disclaimer +in the documentation and/or other materials provided with the +distribution. + * Neither the name of Google Inc. nor the names of its +contributors may be used to endorse or promote products derived from +this software without specific prior written permission. + +THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS +"AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT +LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR +A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT +OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, +SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT +LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, +DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY +THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT +(INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE +OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. diff --git a/vendor/github.com/pborman/uuid/README.md b/vendor/github.com/pborman/uuid/README.md new file mode 100644 index 0000000000..810ad40dc9 --- /dev/null +++ b/vendor/github.com/pborman/uuid/README.md @@ -0,0 +1,15 @@ +This project was automatically exported from code.google.com/p/go-uuid + +# uuid ![build status](https://travis-ci.org/pborman/uuid.svg?branch=master) +The uuid package generates and inspects UUIDs based on [RFC 4122](http://tools.ietf.org/html/rfc4122) and DCE 1.1: Authentication and Security Services. + +This package now leverages the github.com/google/uuid package (which is based off an earlier version of this package). + +###### Install +`go get github.com/pborman/uuid` + +###### Documentation +[![GoDoc](https://godoc.org/github.com/pborman/uuid?status.svg)](http://godoc.org/github.com/pborman/uuid) + +Full `go doc` style documentation for the package can be viewed online without installing this package by using the GoDoc site here: +http://godoc.org/github.com/pborman/uuid diff --git a/vendor/github.com/pborman/uuid/dce.go b/vendor/github.com/pborman/uuid/dce.go new file mode 100644 index 0000000000..50a0f2d099 --- /dev/null +++ b/vendor/github.com/pborman/uuid/dce.go @@ -0,0 +1,84 @@ +// Copyright 2011 Google Inc. All rights reserved. +// Use of this source code is governed by a BSD-style +// license that can be found in the LICENSE file. + +package uuid + +import ( + "encoding/binary" + "fmt" + "os" +) + +// A Domain represents a Version 2 domain +type Domain byte + +// Domain constants for DCE Security (Version 2) UUIDs. +const ( + Person = Domain(0) + Group = Domain(1) + Org = Domain(2) +) + +// NewDCESecurity returns a DCE Security (Version 2) UUID. +// +// The domain should be one of Person, Group or Org. +// On a POSIX system the id should be the users UID for the Person +// domain and the users GID for the Group. The meaning of id for +// the domain Org or on non-POSIX systems is site defined. +// +// For a given domain/id pair the same token may be returned for up to +// 7 minutes and 10 seconds. +func NewDCESecurity(domain Domain, id uint32) UUID { + uuid := NewUUID() + if uuid != nil { + uuid[6] = (uuid[6] & 0x0f) | 0x20 // Version 2 + uuid[9] = byte(domain) + binary.BigEndian.PutUint32(uuid[0:], id) + } + return uuid +} + +// NewDCEPerson returns a DCE Security (Version 2) UUID in the person +// domain with the id returned by os.Getuid. +// +// NewDCEPerson(Person, uint32(os.Getuid())) +func NewDCEPerson() UUID { + return NewDCESecurity(Person, uint32(os.Getuid())) +} + +// NewDCEGroup returns a DCE Security (Version 2) UUID in the group +// domain with the id returned by os.Getgid. +// +// NewDCEGroup(Group, uint32(os.Getgid())) +func NewDCEGroup() UUID { + return NewDCESecurity(Group, uint32(os.Getgid())) +} + +// Domain returns the domain for a Version 2 UUID or false. +func (uuid UUID) Domain() (Domain, bool) { + if v, _ := uuid.Version(); v != 2 { + return 0, false + } + return Domain(uuid[9]), true +} + +// Id returns the id for a Version 2 UUID or false. +func (uuid UUID) Id() (uint32, bool) { + if v, _ := uuid.Version(); v != 2 { + return 0, false + } + return binary.BigEndian.Uint32(uuid[0:4]), true +} + +func (d Domain) String() string { + switch d { + case Person: + return "Person" + case Group: + return "Group" + case Org: + return "Org" + } + return fmt.Sprintf("Domain%d", int(d)) +} diff --git a/vendor/github.com/pborman/uuid/doc.go b/vendor/github.com/pborman/uuid/doc.go new file mode 100644 index 0000000000..727d761674 --- /dev/null +++ b/vendor/github.com/pborman/uuid/doc.go @@ -0,0 +1,13 @@ +// Copyright 2011 Google Inc. All rights reserved. +// Use of this source code is governed by a BSD-style +// license that can be found in the LICENSE file. + +// The uuid package generates and inspects UUIDs. +// +// UUIDs are based on RFC 4122 and DCE 1.1: Authentication and Security +// Services. +// +// This package is a partial wrapper around the github.com/google/uuid package. +// This package represents a UUID as []byte while github.com/google/uuid +// represents a UUID as [16]byte. +package uuid diff --git a/vendor/github.com/pborman/uuid/go.mod b/vendor/github.com/pborman/uuid/go.mod new file mode 100644 index 0000000000..099fc7de0d --- /dev/null +++ b/vendor/github.com/pborman/uuid/go.mod @@ -0,0 +1,3 @@ +module github.com/pborman/uuid + +require github.com/google/uuid v1.0.0 diff --git a/vendor/github.com/pborman/uuid/go.sum b/vendor/github.com/pborman/uuid/go.sum new file mode 100644 index 0000000000..db2574a9c3 --- /dev/null +++ b/vendor/github.com/pborman/uuid/go.sum @@ -0,0 +1,2 @@ +github.com/google/uuid v1.0.0 h1:b4Gk+7WdP/d3HZH8EJsZpvV7EtDOgaZLtnaNGIu1adA= +github.com/google/uuid v1.0.0/go.mod h1:TIyPZe4MgqvfeYDBFedMoGGpEw/LqOeaOT+nhxU+yHo= diff --git a/vendor/github.com/pborman/uuid/hash.go b/vendor/github.com/pborman/uuid/hash.go new file mode 100644 index 0000000000..a0420c1ef3 --- /dev/null +++ b/vendor/github.com/pborman/uuid/hash.go @@ -0,0 +1,53 @@ +// Copyright 2011 Google Inc. All rights reserved. +// Use of this source code is governed by a BSD-style +// license that can be found in the LICENSE file. + +package uuid + +import ( + "crypto/md5" + "crypto/sha1" + "hash" +) + +// Well known Name Space IDs and UUIDs +var ( + NameSpace_DNS = Parse("6ba7b810-9dad-11d1-80b4-00c04fd430c8") + NameSpace_URL = Parse("6ba7b811-9dad-11d1-80b4-00c04fd430c8") + NameSpace_OID = Parse("6ba7b812-9dad-11d1-80b4-00c04fd430c8") + NameSpace_X500 = Parse("6ba7b814-9dad-11d1-80b4-00c04fd430c8") + NIL = Parse("00000000-0000-0000-0000-000000000000") +) + +// NewHash returns a new UUID derived from the hash of space concatenated with +// data generated by h. The hash should be at least 16 byte in length. The +// first 16 bytes of the hash are used to form the UUID. The version of the +// UUID will be the lower 4 bits of version. NewHash is used to implement +// NewMD5 and NewSHA1. +func NewHash(h hash.Hash, space UUID, data []byte, version int) UUID { + h.Reset() + h.Write(space) + h.Write([]byte(data)) + s := h.Sum(nil) + uuid := make([]byte, 16) + copy(uuid, s) + uuid[6] = (uuid[6] & 0x0f) | uint8((version&0xf)<<4) + uuid[8] = (uuid[8] & 0x3f) | 0x80 // RFC 4122 variant + return uuid +} + +// NewMD5 returns a new MD5 (Version 3) UUID based on the +// supplied name space and data. +// +// NewHash(md5.New(), space, data, 3) +func NewMD5(space UUID, data []byte) UUID { + return NewHash(md5.New(), space, data, 3) +} + +// NewSHA1 returns a new SHA1 (Version 5) UUID based on the +// supplied name space and data. +// +// NewHash(sha1.New(), space, data, 5) +func NewSHA1(space UUID, data []byte) UUID { + return NewHash(sha1.New(), space, data, 5) +} diff --git a/vendor/github.com/pborman/uuid/marshal.go b/vendor/github.com/pborman/uuid/marshal.go new file mode 100644 index 0000000000..35b89352ad --- /dev/null +++ b/vendor/github.com/pborman/uuid/marshal.go @@ -0,0 +1,85 @@ +// Copyright 2016 Google Inc. All rights reserved. +// Use of this source code is governed by a BSD-style +// license that can be found in the LICENSE file. + +package uuid + +import ( + "errors" + "fmt" + + guuid "github.com/google/uuid" +) + +// MarshalText implements encoding.TextMarshaler. +func (u UUID) MarshalText() ([]byte, error) { + if len(u) != 16 { + return nil, nil + } + var js [36]byte + encodeHex(js[:], u) + return js[:], nil +} + +// UnmarshalText implements encoding.TextUnmarshaler. +func (u *UUID) UnmarshalText(data []byte) error { + if len(data) == 0 { + return nil + } + id := Parse(string(data)) + if id == nil { + return errors.New("invalid UUID") + } + *u = id + return nil +} + +// MarshalBinary implements encoding.BinaryMarshaler. +func (u UUID) MarshalBinary() ([]byte, error) { + return u[:], nil +} + +// UnmarshalBinary implements encoding.BinaryUnmarshaler. +func (u *UUID) UnmarshalBinary(data []byte) error { + if len(data) == 0 { + return nil + } + if len(data) != 16 { + return fmt.Errorf("invalid UUID (got %d bytes)", len(data)) + } + var id [16]byte + copy(id[:], data) + *u = id[:] + return nil +} + +// MarshalText implements encoding.TextMarshaler. +func (u Array) MarshalText() ([]byte, error) { + var js [36]byte + encodeHex(js[:], u[:]) + return js[:], nil +} + +// UnmarshalText implements encoding.TextUnmarshaler. +func (u *Array) UnmarshalText(data []byte) error { + id, err := guuid.ParseBytes(data) + if err != nil { + return err + } + *u = Array(id) + return nil +} + +// MarshalBinary implements encoding.BinaryMarshaler. +func (u Array) MarshalBinary() ([]byte, error) { + return u[:], nil +} + +// UnmarshalBinary implements encoding.BinaryUnmarshaler. +func (u *Array) UnmarshalBinary(data []byte) error { + if len(data) != 16 { + return fmt.Errorf("invalid UUID (got %d bytes)", len(data)) + } + copy(u[:], data) + return nil +} diff --git a/vendor/github.com/pborman/uuid/node.go b/vendor/github.com/pborman/uuid/node.go new file mode 100644 index 0000000000..e524e0101b --- /dev/null +++ b/vendor/github.com/pborman/uuid/node.go @@ -0,0 +1,50 @@ +// Copyright 2011 Google Inc. All rights reserved. +// Use of this source code is governed by a BSD-style +// license that can be found in the LICENSE file. + +package uuid + +import ( + guuid "github.com/google/uuid" +) + +// NodeInterface returns the name of the interface from which the NodeID was +// derived. The interface "user" is returned if the NodeID was set by +// SetNodeID. +func NodeInterface() string { + return guuid.NodeInterface() +} + +// SetNodeInterface selects the hardware address to be used for Version 1 UUIDs. +// If name is "" then the first usable interface found will be used or a random +// Node ID will be generated. If a named interface cannot be found then false +// is returned. +// +// SetNodeInterface never fails when name is "". +func SetNodeInterface(name string) bool { + return guuid.SetNodeInterface(name) +} + +// NodeID returns a slice of a copy of the current Node ID, setting the Node ID +// if not already set. +func NodeID() []byte { + return guuid.NodeID() +} + +// SetNodeID sets the Node ID to be used for Version 1 UUIDs. The first 6 bytes +// of id are used. If id is less than 6 bytes then false is returned and the +// Node ID is not set. +func SetNodeID(id []byte) bool { + return guuid.SetNodeID(id) +} + +// NodeID returns the 6 byte node id encoded in uuid. It returns nil if uuid is +// not valid. The NodeID is only well defined for version 1 and 2 UUIDs. +func (uuid UUID) NodeID() []byte { + if len(uuid) != 16 { + return nil + } + node := make([]byte, 6) + copy(node, uuid[10:]) + return node +} diff --git a/vendor/github.com/pborman/uuid/sql.go b/vendor/github.com/pborman/uuid/sql.go new file mode 100644 index 0000000000..929c3847e2 --- /dev/null +++ b/vendor/github.com/pborman/uuid/sql.go @@ -0,0 +1,68 @@ +// Copyright 2015 Google Inc. All rights reserved. +// Use of this source code is governed by a BSD-style +// license that can be found in the LICENSE file. + +package uuid + +import ( + "database/sql/driver" + "errors" + "fmt" +) + +// Scan implements sql.Scanner so UUIDs can be read from databases transparently +// Currently, database types that map to string and []byte are supported. Please +// consult database-specific driver documentation for matching types. +func (uuid *UUID) Scan(src interface{}) error { + switch src.(type) { + case string: + // if an empty UUID comes from a table, we return a null UUID + if src.(string) == "" { + return nil + } + + // see uuid.Parse for required string format + parsed := Parse(src.(string)) + + if parsed == nil { + return errors.New("Scan: invalid UUID format") + } + + *uuid = parsed + case []byte: + b := src.([]byte) + + // if an empty UUID comes from a table, we return a null UUID + if len(b) == 0 { + return nil + } + + // assumes a simple slice of bytes if 16 bytes + // otherwise attempts to parse + if len(b) == 16 { + parsed := make([]byte, 16) + copy(parsed, b) + *uuid = UUID(parsed) + } else { + u := Parse(string(b)) + + if u == nil { + return errors.New("Scan: invalid UUID format") + } + + *uuid = u + } + + default: + return fmt.Errorf("Scan: unable to scan type %T into UUID", src) + } + + return nil +} + +// Value implements sql.Valuer so that UUIDs can be written to databases +// transparently. Currently, UUIDs map to strings. Please consult +// database-specific driver documentation for matching types. +func (uuid UUID) Value() (driver.Value, error) { + return uuid.String(), nil +} diff --git a/vendor/github.com/pborman/uuid/time.go b/vendor/github.com/pborman/uuid/time.go new file mode 100644 index 0000000000..5c0960d872 --- /dev/null +++ b/vendor/github.com/pborman/uuid/time.go @@ -0,0 +1,57 @@ +// Copyright 2014 Google Inc. All rights reserved. +// Use of this source code is governed by a BSD-style +// license that can be found in the LICENSE file. + +package uuid + +import ( + "encoding/binary" + + guuid "github.com/google/uuid" +) + +// A Time represents a time as the number of 100's of nanoseconds since 15 Oct +// 1582. +type Time = guuid.Time + +// GetTime returns the current Time (100s of nanoseconds since 15 Oct 1582) and +// clock sequence as well as adjusting the clock sequence as needed. An error +// is returned if the current time cannot be determined. +func GetTime() (Time, uint16, error) { return guuid.GetTime() } + +// ClockSequence returns the current clock sequence, generating one if not +// already set. The clock sequence is only used for Version 1 UUIDs. +// +// The uuid package does not use global static storage for the clock sequence or +// the last time a UUID was generated. Unless SetClockSequence a new random +// clock sequence is generated the first time a clock sequence is requested by +// ClockSequence, GetTime, or NewUUID. (section 4.2.1.1) sequence is generated +// for +func ClockSequence() int { return guuid.ClockSequence() } + +// SetClockSeq sets the clock sequence to the lower 14 bits of seq. Setting to +// -1 causes a new sequence to be generated. +func SetClockSequence(seq int) { guuid.SetClockSequence(seq) } + +// Time returns the time in 100s of nanoseconds since 15 Oct 1582 encoded in +// uuid. It returns false if uuid is not valid. The time is only well defined +// for version 1 and 2 UUIDs. +func (uuid UUID) Time() (Time, bool) { + if len(uuid) != 16 { + return 0, false + } + time := int64(binary.BigEndian.Uint32(uuid[0:4])) + time |= int64(binary.BigEndian.Uint16(uuid[4:6])) << 32 + time |= int64(binary.BigEndian.Uint16(uuid[6:8])&0xfff) << 48 + return Time(time), true +} + +// ClockSequence returns the clock sequence encoded in uuid. It returns false +// if uuid is not valid. The clock sequence is only well defined for version 1 +// and 2 UUIDs. +func (uuid UUID) ClockSequence() (int, bool) { + if len(uuid) != 16 { + return 0, false + } + return int(binary.BigEndian.Uint16(uuid[8:10])) & 0x3fff, true +} diff --git a/vendor/github.com/pborman/uuid/util.go b/vendor/github.com/pborman/uuid/util.go new file mode 100644 index 0000000000..255b5e2485 --- /dev/null +++ b/vendor/github.com/pborman/uuid/util.go @@ -0,0 +1,32 @@ +// Copyright 2011 Google Inc. All rights reserved. +// Use of this source code is governed by a BSD-style +// license that can be found in the LICENSE file. + +package uuid + +// xvalues returns the value of a byte as a hexadecimal digit or 255. +var xvalues = [256]byte{ + 255, 255, 255, 255, 255, 255, 255, 255, 255, 255, 255, 255, 255, 255, 255, 255, + 255, 255, 255, 255, 255, 255, 255, 255, 255, 255, 255, 255, 255, 255, 255, 255, + 255, 255, 255, 255, 255, 255, 255, 255, 255, 255, 255, 255, 255, 255, 255, 255, + 0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 255, 255, 255, 255, 255, 255, + 255, 10, 11, 12, 13, 14, 15, 255, 255, 255, 255, 255, 255, 255, 255, 255, + 255, 255, 255, 255, 255, 255, 255, 255, 255, 255, 255, 255, 255, 255, 255, 255, + 255, 10, 11, 12, 13, 14, 15, 255, 255, 255, 255, 255, 255, 255, 255, 255, + 255, 255, 255, 255, 255, 255, 255, 255, 255, 255, 255, 255, 255, 255, 255, 255, + 255, 255, 255, 255, 255, 255, 255, 255, 255, 255, 255, 255, 255, 255, 255, 255, + 255, 255, 255, 255, 255, 255, 255, 255, 255, 255, 255, 255, 255, 255, 255, 255, + 255, 255, 255, 255, 255, 255, 255, 255, 255, 255, 255, 255, 255, 255, 255, 255, + 255, 255, 255, 255, 255, 255, 255, 255, 255, 255, 255, 255, 255, 255, 255, 255, + 255, 255, 255, 255, 255, 255, 255, 255, 255, 255, 255, 255, 255, 255, 255, 255, + 255, 255, 255, 255, 255, 255, 255, 255, 255, 255, 255, 255, 255, 255, 255, 255, + 255, 255, 255, 255, 255, 255, 255, 255, 255, 255, 255, 255, 255, 255, 255, 255, + 255, 255, 255, 255, 255, 255, 255, 255, 255, 255, 255, 255, 255, 255, 255, 255, +} + +// xtob converts the the first two hex bytes of x into a byte. +func xtob(x string) (byte, bool) { + b1 := xvalues[x[0]] + b2 := xvalues[x[1]] + return (b1 << 4) | b2, b1 != 255 && b2 != 255 +} diff --git a/vendor/github.com/pborman/uuid/uuid.go b/vendor/github.com/pborman/uuid/uuid.go new file mode 100644 index 0000000000..3370004207 --- /dev/null +++ b/vendor/github.com/pborman/uuid/uuid.go @@ -0,0 +1,162 @@ +// Copyright 2011 Google Inc. All rights reserved. +// Use of this source code is governed by a BSD-style +// license that can be found in the LICENSE file. + +package uuid + +import ( + "bytes" + "crypto/rand" + "encoding/hex" + "io" + + guuid "github.com/google/uuid" +) + +// Array is a pass-by-value UUID that can be used as an effecient key in a map. +type Array [16]byte + +// UUID converts uuid into a slice. +func (uuid Array) UUID() UUID { + return uuid[:] +} + +// String returns the string representation of uuid, +// xxxxxxxx-xxxx-xxxx-xxxx-xxxxxxxxxxxx. +func (uuid Array) String() string { + return guuid.UUID(uuid).String() +} + +// A UUID is a 128 bit (16 byte) Universal Unique IDentifier as defined in RFC +// 4122. +type UUID []byte + +// A Version represents a UUIDs version. +type Version = guuid.Version + +// A Variant represents a UUIDs variant. +type Variant = guuid.Variant + +// Constants returned by Variant. +const ( + Invalid = guuid.Invalid // Invalid UUID + RFC4122 = guuid.RFC4122 // The variant specified in RFC4122 + Reserved = guuid.Reserved // Reserved, NCS backward compatibility. + Microsoft = guuid.Microsoft // Reserved, Microsoft Corporation backward compatibility. + Future = guuid.Future // Reserved for future definition. +) + +var rander = rand.Reader // random function + +// New returns a new random (version 4) UUID as a string. It is a convenience +// function for NewRandom().String(). +func New() string { + return NewRandom().String() +} + +// Parse decodes s into a UUID or returns nil. See github.com/google/uuid for +// the formats parsed. +func Parse(s string) UUID { + gu, err := guuid.Parse(s) + if err == nil { + return gu[:] + } + return nil +} + +// ParseBytes is like Parse, except it parses a byte slice instead of a string. +func ParseBytes(b []byte) (UUID, error) { + gu, err := guuid.ParseBytes(b) + if err == nil { + return gu[:], nil + } + return nil, err +} + +// Equal returns true if uuid1 and uuid2 are equal. +func Equal(uuid1, uuid2 UUID) bool { + return bytes.Equal(uuid1, uuid2) +} + +// Array returns an array representation of uuid that can be used as a map key. +// Array panics if uuid is not valid. +func (uuid UUID) Array() Array { + if len(uuid) != 16 { + panic("invalid uuid") + } + var a Array + copy(a[:], uuid) + return a +} + +// String returns the string form of uuid, xxxxxxxx-xxxx-xxxx-xxxx-xxxxxxxxxxxx +// , or "" if uuid is invalid. +func (uuid UUID) String() string { + if len(uuid) != 16 { + return "" + } + var buf [36]byte + encodeHex(buf[:], uuid) + return string(buf[:]) +} + +// URN returns the RFC 2141 URN form of uuid, +// urn:uuid:xxxxxxxx-xxxx-xxxx-xxxx-xxxxxxxxxxxx, or "" if uuid is invalid. +func (uuid UUID) URN() string { + if len(uuid) != 16 { + return "" + } + var buf [36 + 9]byte + copy(buf[:], "urn:uuid:") + encodeHex(buf[9:], uuid) + return string(buf[:]) +} + +func encodeHex(dst []byte, uuid UUID) { + hex.Encode(dst[:], uuid[:4]) + dst[8] = '-' + hex.Encode(dst[9:13], uuid[4:6]) + dst[13] = '-' + hex.Encode(dst[14:18], uuid[6:8]) + dst[18] = '-' + hex.Encode(dst[19:23], uuid[8:10]) + dst[23] = '-' + hex.Encode(dst[24:], uuid[10:]) +} + +// Variant returns the variant encoded in uuid. It returns Invalid if +// uuid is invalid. +func (uuid UUID) Variant() Variant { + if len(uuid) != 16 { + return Invalid + } + switch { + case (uuid[8] & 0xc0) == 0x80: + return RFC4122 + case (uuid[8] & 0xe0) == 0xc0: + return Microsoft + case (uuid[8] & 0xe0) == 0xe0: + return Future + default: + return Reserved + } +} + +// Version returns the version of uuid. It returns false if uuid is not +// valid. +func (uuid UUID) Version() (Version, bool) { + if len(uuid) != 16 { + return 0, false + } + return Version(uuid[6] >> 4), true +} + +// SetRand sets the random number generator to r, which implements io.Reader. +// If r.Read returns an error when the package requests random data then +// a panic will be issued. +// +// Calling SetRand with nil sets the random number generator to the default +// generator. +func SetRand(r io.Reader) { + guuid.SetRand(r) +} diff --git a/vendor/github.com/pborman/uuid/version1.go b/vendor/github.com/pborman/uuid/version1.go new file mode 100644 index 0000000000..7af948da79 --- /dev/null +++ b/vendor/github.com/pborman/uuid/version1.go @@ -0,0 +1,23 @@ +// Copyright 2011 Google Inc. All rights reserved. +// Use of this source code is governed by a BSD-style +// license that can be found in the LICENSE file. + +package uuid + +import ( + guuid "github.com/google/uuid" +) + +// NewUUID returns a Version 1 UUID based on the current NodeID and clock +// sequence, and the current time. If the NodeID has not been set by SetNodeID +// or SetNodeInterface then it will be set automatically. If the NodeID cannot +// be set NewUUID returns nil. If clock sequence has not been set by +// SetClockSequence then it will be set automatically. If GetTime fails to +// return the current NewUUID returns nil. +func NewUUID() UUID { + gu, err := guuid.NewUUID() + if err == nil { + return UUID(gu[:]) + } + return nil +} diff --git a/vendor/github.com/pborman/uuid/version4.go b/vendor/github.com/pborman/uuid/version4.go new file mode 100644 index 0000000000..b459d46d13 --- /dev/null +++ b/vendor/github.com/pborman/uuid/version4.go @@ -0,0 +1,26 @@ +// Copyright 2011 Google Inc. All rights reserved. +// Use of this source code is governed by a BSD-style +// license that can be found in the LICENSE file. + +package uuid + +import guuid "github.com/google/uuid" + +// Random returns a Random (Version 4) UUID or panics. +// +// The strength of the UUIDs is based on the strength of the crypto/rand +// package. +// +// A note about uniqueness derived from the UUID Wikipedia entry: +// +// Randomly generated UUIDs have 122 random bits. One's annual risk of being +// hit by a meteorite is estimated to be one chance in 17 billion, that +// means the probability is about 0.00000000006 (6 × 10−11), +// equivalent to the odds of creating a few tens of trillions of UUIDs in a +// year and having one duplicate. +func NewRandom() UUID { + if gu, err := guuid.NewRandom(); err == nil { + return UUID(gu[:]) + } + return nil +} diff --git a/vendor/k8s.io/apimachinery/pkg/util/uuid/BUILD b/vendor/k8s.io/apimachinery/pkg/util/uuid/BUILD new file mode 100644 index 0000000000..786d8fbe39 --- /dev/null +++ b/vendor/k8s.io/apimachinery/pkg/util/uuid/BUILD @@ -0,0 +1,29 @@ +package(default_visibility = ["//visibility:public"]) + +load( + "@io_bazel_rules_go//go:def.bzl", + "go_library", +) + +go_library( + name = "go_default_library", + srcs = ["uuid.go"], + importpath = "k8s.io/apimachinery/pkg/util/uuid", + deps = [ + "//vendor/github.com/pborman/uuid:go_default_library", + "//vendor/k8s.io/apimachinery/pkg/types:go_default_library", + ], +) + +filegroup( + name = "package-srcs", + srcs = glob(["**"]), + tags = ["automanaged"], + visibility = ["//visibility:private"], +) + +filegroup( + name = "all-srcs", + srcs = [":package-srcs"], + tags = ["automanaged"], +) diff --git a/vendor/k8s.io/apimachinery/pkg/util/uuid/uuid.go b/vendor/k8s.io/apimachinery/pkg/util/uuid/uuid.go new file mode 100644 index 0000000000..bf478223d9 --- /dev/null +++ b/vendor/k8s.io/apimachinery/pkg/util/uuid/uuid.go @@ -0,0 +1,43 @@ +/* +Copyright 2014 The Kubernetes Authors. + +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. +*/ + +package uuid + +import ( + "sync" + + "github.com/pborman/uuid" + + "k8s.io/apimachinery/pkg/types" +) + +var uuidLock sync.Mutex +var lastUUID uuid.UUID + +func NewUUID() types.UID { + uuidLock.Lock() + defer uuidLock.Unlock() + result := uuid.NewUUID() + // The UUID package is naive and can generate identical UUIDs if the + // time interval is quick enough. + // The UUID uses 100 ns increments so it's short enough to actively + // wait for a new value. + for uuid.Equal(lastUUID, result) == true { + result = uuid.NewUUID() + } + lastUUID = result + return types.UID(result.String()) +} diff --git a/vendor/k8s.io/client-go/pkg/version/base.go b/vendor/k8s.io/client-go/pkg/version/base.go index 3f2312cd2d..7ab0ed3a7c 100644 --- a/vendor/k8s.io/client-go/pkg/version/base.go +++ b/vendor/k8s.io/client-go/pkg/version/base.go @@ -55,8 +55,8 @@ var ( // NOTE: The $Format strings are replaced during 'git archive' thanks to the // companion .gitattributes file containing 'export-subst' in this same // directory. See also https://git-scm.com/docs/gitattributes - gitVersion string = "v0.0.0-master+33f2870" - gitCommit string = "33f2870a2b83179c823ddc90e5513f9e5fe43b38" // sha1 from git, output of $(git rev-parse HEAD) + gitVersion string = "v0.0.0-master+$Format:%h$" + gitCommit string = "$Format:%H$" // sha1 from git, output of $(git rev-parse HEAD) gitTreeState string = "" // state of git tree, either "clean" or "dirty" buildDate string = "1970-01-01T00:00:00Z" // build date in ISO8601 format, output of $(date -u +'%Y-%m-%dT%H:%M:%SZ') diff --git a/vendor/modules.txt b/vendor/modules.txt index 4bc5fef3a8..e963735bad 100644 --- a/vendor/modules.txt +++ b/vendor/modules.txt @@ -114,6 +114,8 @@ github.com/golang/protobuf/ptypes/struct github.com/google/btree # github.com/google/gofuzz v0.0.0-20170612174753-24818f796faf github.com/google/gofuzz +# github.com/google/uuid v1.0.0 +github.com/google/uuid # github.com/googleapis/gnostic v0.2.0 github.com/googleapis/gnostic/OpenAPIv2 github.com/googleapis/gnostic/compiler @@ -185,6 +187,8 @@ github.com/onsi/gomega/matchers/support/goraph/bipartitegraph github.com/onsi/gomega/matchers/support/goraph/edge github.com/onsi/gomega/matchers/support/goraph/node github.com/onsi/gomega/matchers/support/goraph/util +# github.com/pborman/uuid v1.2.0 +github.com/pborman/uuid # github.com/pingcap/errors v0.11.0 github.com/pingcap/errors # github.com/pingcap/kvproto v0.0.0-20180606093822-b7ba8ea1c0b4 @@ -249,8 +253,8 @@ golang.org/x/crypto/ssh/terminal golang.org/x/crypto/bcrypt golang.org/x/crypto/blowfish # golang.org/x/net v0.0.0-20180808004115-f9ce57c11b24 -golang.org/x/net/http2 golang.org/x/net/context +golang.org/x/net/http2 golang.org/x/net/http/httpguts golang.org/x/net/http2/hpack golang.org/x/net/idna @@ -327,13 +331,13 @@ gopkg.in/yaml.v2 # k8s.io/api v0.0.0-20180308224125-73d903622b73 k8s.io/api/apps/v1beta1 k8s.io/api/core/v1 +k8s.io/api/batch/v1 k8s.io/api/admissionregistration/v1alpha1 k8s.io/api/admissionregistration/v1beta1 k8s.io/api/apps/v1 k8s.io/api/apps/v1beta2 k8s.io/api/autoscaling/v1 k8s.io/api/autoscaling/v2beta1 -k8s.io/api/batch/v1 k8s.io/api/batch/v1beta1 k8s.io/api/batch/v2alpha1 k8s.io/api/certificates/v1beta1 @@ -370,6 +374,7 @@ k8s.io/apimachinery/pkg/api/resource k8s.io/apimachinery/pkg/util/runtime k8s.io/apimachinery/pkg/util/errors k8s.io/apimachinery/pkg/util/intstr +k8s.io/apimachinery/pkg/util/uuid k8s.io/apimachinery/pkg/conversion k8s.io/apimachinery/pkg/fields k8s.io/apimachinery/pkg/selection From a41eed91ab108257792803e8e25a6dbd9cc061b5 Mon Sep 17 00:00:00 2001 From: zyguan Date: Wed, 20 Mar 2019 13:44:21 +0800 Subject: [PATCH 21/22] revert go.mod changes --- go.mod | 4 +--- 1 file changed, 1 insertion(+), 3 deletions(-) diff --git a/go.mod b/go.mod index d316139b70..214e9bf1c8 100644 --- a/go.mod +++ b/go.mod @@ -30,9 +30,7 @@ require ( github.com/grpc-ecosystem/go-grpc-prometheus v1.2.0 // indirect github.com/grpc-ecosystem/grpc-gateway v1.4.1 // indirect github.com/hashicorp/golang-lru v0.0.0-20180201235237-0fb14efe8c47 // indirect - github.com/howeyc/gopass v0.0.0-20170109162249-bf9dde6d0d2c // indirect github.com/hpcloud/tail v1.0.0 // indirect - github.com/imdario/mergo v0.3.7 // indirect github.com/jonboulle/clockwork v0.1.0 // indirect github.com/json-iterator/go v1.1.5 // indirect github.com/juju/errors v0.0.0-20180806074554-22422dad46e1 @@ -49,7 +47,7 @@ require ( github.com/pingcap/errors v0.11.0 github.com/pingcap/kvproto v0.0.0-20180606093822-b7ba8ea1c0b4 github.com/pingcap/pd v2.1.0-beta+incompatible - github.com/pkg/errors v0.8.0 + github.com/pkg/errors v0.8.0 // indirect github.com/pmezard/go-difflib v1.0.0 // indirect github.com/prometheus/client_golang v0.8.0 github.com/prometheus/client_model v0.0.0-20180712105110-5c3871d89910 // indirect From 2d416c628fcd4f4c0c18fcb0a093c2a30ae65272 Mon Sep 17 00:00:00 2001 From: xiaojingchen Date: Wed, 20 Mar 2019 17:02:09 +0800 Subject: [PATCH 22/22] multi tidb cluster testing (#334) --- go.mod | 1 + go.sum | 2 + tests/actions.go | 6 +- tests/cmd/e2e/main.go | 281 ++++++++++--------- tests/cmd/stability/main.go | 290 ++++++++++++++++++++ vendor/github.com/jinzhu/copier/Guardfile | 3 + vendor/github.com/jinzhu/copier/License | 20 ++ vendor/github.com/jinzhu/copier/README.md | 100 +++++++ vendor/github.com/jinzhu/copier/copier.go | 185 +++++++++++++ vendor/github.com/jinzhu/copier/wercker.yml | 23 ++ vendor/modules.txt | 2 + 11 files changed, 784 insertions(+), 129 deletions(-) create mode 100644 tests/cmd/stability/main.go create mode 100644 vendor/github.com/jinzhu/copier/Guardfile create mode 100644 vendor/github.com/jinzhu/copier/License create mode 100644 vendor/github.com/jinzhu/copier/README.md create mode 100644 vendor/github.com/jinzhu/copier/copier.go create mode 100644 vendor/github.com/jinzhu/copier/wercker.yml diff --git a/go.mod b/go.mod index 214e9bf1c8..0748c99a22 100644 --- a/go.mod +++ b/go.mod @@ -31,6 +31,7 @@ require ( github.com/grpc-ecosystem/grpc-gateway v1.4.1 // indirect github.com/hashicorp/golang-lru v0.0.0-20180201235237-0fb14efe8c47 // indirect github.com/hpcloud/tail v1.0.0 // indirect + github.com/jinzhu/copier v0.0.0-20180308034124-7e38e58719c3 github.com/jonboulle/clockwork v0.1.0 // indirect github.com/json-iterator/go v1.1.5 // indirect github.com/juju/errors v0.0.0-20180806074554-22422dad46e1 diff --git a/go.sum b/go.sum index f4cc32ccee..7ef68a118e 100644 --- a/go.sum +++ b/go.sum @@ -58,6 +58,8 @@ github.com/hashicorp/golang-lru v0.0.0-20180201235237-0fb14efe8c47 h1:UnszMmmmm5 github.com/hashicorp/golang-lru v0.0.0-20180201235237-0fb14efe8c47/go.mod h1:/m3WP610KZHVQ1SGc6re/UDhFvYD7pJ4Ao+sR/qLZy8= github.com/hpcloud/tail v1.0.0 h1:nfCOvKYfkgYP8hkirhJocXT2+zOD8yUNjXaWfTlyFKI= github.com/hpcloud/tail v1.0.0/go.mod h1:ab1qPbhIpdTxEkNHXyeSf5vhxWSCs/tWer42PpOxQnU= +github.com/jinzhu/copier v0.0.0-20180308034124-7e38e58719c3 h1:sHsPfNMAG70QAvKbddQ0uScZCHQoZsT5NykGRCeeeIs= +github.com/jinzhu/copier v0.0.0-20180308034124-7e38e58719c3/go.mod h1:yL958EeXv8Ylng6IfnvG4oflryUi3vgA3xPs9hmII1s= github.com/jonboulle/clockwork v0.1.0 h1:VKV+ZcuP6l3yW9doeqz6ziZGgcynBVQO+obU0+0hcPo= github.com/jonboulle/clockwork v0.1.0/go.mod h1:Ii8DK3G1RaLaWxj9trq07+26W01tbo22gdxWY5EU2bo= github.com/json-iterator/go v1.1.5 h1:gL2yXlmiIo4+t+y32d4WGwOjKGYcGOuyrg46vadswDE= diff --git a/tests/actions.go b/tests/actions.go index 49f8f642b4..36afc3728b 100644 --- a/tests/actions.go +++ b/tests/actions.go @@ -1115,8 +1115,8 @@ func (oa *operatorActions) Restore(from *TidbClusterInfo, to *TidbClusterInfo) e setString := to.HelmSetString(sets) restoreName := fmt.Sprintf("%s-restore", from.ClusterName) - cmd := fmt.Sprintf("helm install -n %s --namespace %s /charts/%s/tidb-backup --set-string %s", - restoreName, to.Namespace, to.OperatorTag, setString) + cmd := fmt.Sprintf("helm upgrade %s /charts/%s/tidb-backup --set-string %s", + restoreName, to.OperatorTag, setString) glog.Infof("install restore [%s]", cmd) res, err := exec.Command("/bin/sh", "-c", cmd).CombinedOutput() if err != nil { @@ -1222,7 +1222,7 @@ func (oa *operatorActions) CreateSecret(info *TidbClusterInfo) error { _, err := oa.kubeCli.CoreV1().Secrets(info.Namespace).Create(&initSecret) if err != nil && !releaseIsExist(err) { - return err + return err } backupSecret := corev1.Secret{ diff --git a/tests/cmd/e2e/main.go b/tests/cmd/e2e/main.go index 313c89c251..1b27fe40a1 100644 --- a/tests/cmd/e2e/main.go +++ b/tests/cmd/e2e/main.go @@ -14,11 +14,12 @@ package main import ( + "fmt" "net/http" _ "net/http/pprof" - "time" "github.com/golang/glog" + "github.com/jinzhu/copier" "github.com/pingcap/tidb-operator/pkg/client/clientset/versioned" "github.com/pingcap/tidb-operator/tests" "github.com/pingcap/tidb-operator/tests/backup" @@ -29,12 +30,6 @@ import ( "k8s.io/client-go/rest" ) -func perror(err error) { - if err != nil { - glog.Fatal(err) - } -} - func main() { logs.InitLogs() defer logs.FlushLogs() @@ -49,6 +44,12 @@ func main() { glog.Info(http.ListenAndServe("localhost:6060", nil)) }() + // TODO read these args from config + beginTidbVersion := "v2.1.0" + toTidbVersion := "v2.1.4" + operatorTag := "master" + operatorImage := "pingcap/tidb-operator:latest" + cfg, err := rest.InClusterConfig() if err != nil { glog.Fatalf("failed to get config: %v", err) @@ -67,11 +68,85 @@ func main() { operatorInfo := &tests.OperatorInfo{ Namespace: "pingcap", ReleaseName: "operator", - Image: "pingcap/tidb-operator:latest", - Tag: "master", + Image: operatorImage, + Tag: operatorTag, SchedulerImage: "gcr.io/google-containers/hyperkube:v1.12.1", LogLevel: "2", } + + // create database and table and insert a column for test backup and restore + initSql := `"create database record;use record;create table test(t char(32))"` + + clusterInfos := []*tests.TidbClusterInfo{ + { + Namespace: "e2e-cluster1", + ClusterName: "e2e-cluster1", + OperatorTag: operatorTag, + PDImage: fmt.Sprintf("pingcap/pd:%s", beginTidbVersion), + TiKVImage: fmt.Sprintf("pingcap/tikv:%s", beginTidbVersion), + TiDBImage: fmt.Sprintf("pingcap/tidb:%s", beginTidbVersion), + StorageClassName: "local-storage", + Password: "admin", + InitSql: initSql, + UserName: "root", + InitSecretName: "demo-set-secret", + BackupSecretName: "demo-backup-secret", + BackupPVC: "test-backup", + Resources: map[string]string{ + "pd.resources.limits.cpu": "1000m", + "pd.resources.limits.memory": "2Gi", + "pd.resources.requests.cpu": "200m", + "pd.resources.requests.memory": "1Gi", + "tikv.resources.limits.cpu": "2000m", + "tikv.resources.limits.memory": "4Gi", + "tikv.resources.requests.cpu": "1000m", + "tikv.resources.requests.memory": "2Gi", + "tidb.resources.limits.cpu": "2000m", + "tidb.resources.limits.memory": "4Gi", + "tidb.resources.requests.cpu": "500m", + "tidb.resources.requests.memory": "1Gi", + }, + Args: map[string]string{}, + Monitor: true, + }, + { + Namespace: "e2e-cluster2", + ClusterName: "e2e-cluster2", + OperatorTag: "master", + PDImage: fmt.Sprintf("pingcap/pd:%s", beginTidbVersion), + TiKVImage: fmt.Sprintf("pingcap/tikv:%s", beginTidbVersion), + TiDBImage: fmt.Sprintf("pingcap/tidb:%s", beginTidbVersion), + StorageClassName: "local-storage", + Password: "admin", + InitSql: initSql, + UserName: "root", + InitSecretName: "demo-set-secret", + BackupSecretName: "demo-backup-secret", + BackupPVC: "test-backup", + Resources: map[string]string{ + "pd.resources.limits.cpu": "1000m", + "pd.resources.limits.memory": "2Gi", + "pd.resources.requests.cpu": "200m", + "pd.resources.requests.memory": "1Gi", + "tikv.resources.limits.cpu": "2000m", + "tikv.resources.limits.memory": "4Gi", + "tikv.resources.requests.cpu": "1000m", + "tikv.resources.requests.memory": "2Gi", + "tidb.resources.limits.cpu": "2000m", + "tidb.resources.limits.memory": "4Gi", + "tidb.resources.requests.cpu": "500m", + "tidb.resources.requests.memory": "1Gi", + }, + Args: map[string]string{}, + Monitor: true, + }, + } + + defer func() { + oa.DumpAllLogs(operatorInfo, clusterInfos) + }() + + // deploy operator if err := oa.CleanOperator(operatorInfo); err != nil { oa.DumpAllLogs(operatorInfo, nil) glog.Fatal(err) @@ -81,163 +156,117 @@ func main() { glog.Fatal(err) } - // create database and table and insert a column for test backup and restore - initSql := `"create database record;use record;create table test(t char(32))"` - - clusterInfo := &tests.TidbClusterInfo{ - BackupPVC: "test-backup", - Namespace: "tidb", - ClusterName: "demo", - OperatorTag: "master", - PDImage: "pingcap/pd:v2.1.0", - TiKVImage: "pingcap/tikv:v2.1.0", - TiDBImage: "pingcap/tidb:v2.1.0", - StorageClassName: "local-storage", - Password: "admin", - InitSql: initSql, - UserName: "root", - InitSecretName: "demo-set-secret", - BackupSecretName: "demo-backup-secret", - Resources: map[string]string{ - "pd.resources.limits.cpu": "1000m", - "pd.resources.limits.memory": "2Gi", - "pd.resources.requests.cpu": "200m", - "pd.resources.requests.memory": "1Gi", - "tikv.resources.limits.cpu": "2000m", - "tikv.resources.limits.memory": "4Gi", - "tikv.resources.requests.cpu": "1000m", - "tikv.resources.requests.memory": "2Gi", - "tidb.resources.limits.cpu": "2000m", - "tidb.resources.limits.memory": "4Gi", - "tidb.resources.requests.cpu": "500m", - "tidb.resources.requests.memory": "1Gi", - }, - Args: map[string]string{}, + // deploy tidbclusters + for _, clusterInfo := range clusterInfos { + if err = oa.CleanTidbCluster(clusterInfo); err != nil { + glog.Fatal(err) + } + if err = oa.DeployTidbCluster(clusterInfo); err != nil { + glog.Fatal(err) + } } - if err = oa.CleanTidbCluster(clusterInfo); err != nil { - oa.DumpAllLogs(operatorInfo, []*tests.TidbClusterInfo{clusterInfo}) - glog.Fatal(err) - } - if err = oa.DeployTidbCluster(clusterInfo); err != nil { - oa.DumpAllLogs(operatorInfo, []*tests.TidbClusterInfo{clusterInfo}) - glog.Fatal(err) + for _, clusterInfo := range clusterInfos { + if err = oa.CheckTidbClusterStatus(clusterInfo); err != nil { + glog.Fatal(err) + } } - if err = oa.CheckTidbClusterStatus(clusterInfo); err != nil { - oa.DumpAllLogs(operatorInfo, []*tests.TidbClusterInfo{clusterInfo}) - glog.Fatal(err) + + var workloads []workload.Workload + for _, clusterInfo := range clusterInfos { + workload := ddl.New(clusterInfo.DSN("test"), 1, 1) + workloads = append(workloads, workload) } err = workload.Run(func() error { - clusterInfo = clusterInfo.ScaleTiDB(3).ScaleTiKV(5).ScalePD(5) - if err := oa.ScaleTidbCluster(clusterInfo); err != nil { - return err + + for _, clusterInfo := range clusterInfos { + clusterInfo = clusterInfo.ScaleTiDB(3).ScaleTiKV(5).ScalePD(5) + if err := oa.ScaleTidbCluster(clusterInfo); err != nil { + return err + } } - if err := oa.CheckTidbClusterStatus(clusterInfo); err != nil { - return err + for _, clusterInfo := range clusterInfos { + if err := oa.CheckTidbClusterStatus(clusterInfo); err != nil { + return err + } } - clusterInfo = clusterInfo.ScalePD(3) - if err := oa.ScaleTidbCluster(clusterInfo); err != nil { - return err + for _, clusterInfo := range clusterInfos { + clusterInfo = clusterInfo.ScalePD(3) + if err := oa.ScaleTidbCluster(clusterInfo); err != nil { + return err + } } - if err := oa.CheckTidbClusterStatus(clusterInfo); err != nil { - return err + for _, clusterInfo := range clusterInfos { + if err := oa.CheckTidbClusterStatus(clusterInfo); err != nil { + return err + } } - clusterInfo = clusterInfo.ScaleTiKV(3) - if err := oa.ScaleTidbCluster(clusterInfo); err != nil { - return err + for _, clusterInfo := range clusterInfos { + clusterInfo = clusterInfo.ScaleTiKV(3) + if err := oa.ScaleTidbCluster(clusterInfo); err != nil { + return err + } } - if err := oa.CheckTidbClusterStatus(clusterInfo); err != nil { - return err + for _, clusterInfo := range clusterInfos { + if err := oa.CheckTidbClusterStatus(clusterInfo); err != nil { + return err + } } - clusterInfo = clusterInfo.ScaleTiDB(1) - if err := oa.ScaleTidbCluster(clusterInfo); err != nil { - return err + for _, clusterInfo := range clusterInfos { + clusterInfo = clusterInfo.ScaleTiDB(1) + if err := oa.ScaleTidbCluster(clusterInfo); err != nil { + return err + } } - if err := oa.CheckTidbClusterStatus(clusterInfo); err != nil { - return err + for _, clusterInfo := range clusterInfos { + if err := oa.CheckTidbClusterStatus(clusterInfo); err != nil { + return err + } } return nil - }, ddl.New(clusterInfo.DSN("test"), 1, 1)) + }, workloads...) if err != nil { - oa.DumpAllLogs(operatorInfo, []*tests.TidbClusterInfo{clusterInfo}) glog.Fatal(err) } - clusterInfo = clusterInfo.UpgradeAll("v2.1.4") - if err = oa.UpgradeTidbCluster(clusterInfo); err != nil { - oa.DumpAllLogs(operatorInfo, []*tests.TidbClusterInfo{clusterInfo}) - glog.Fatal(err) - } - if err = oa.CheckTidbClusterStatus(clusterInfo); err != nil { - oa.DumpAllLogs(operatorInfo, []*tests.TidbClusterInfo{clusterInfo}) - glog.Fatal(err) + for _, clusterInfo := range clusterInfos { + clusterInfo = clusterInfo.UpgradeAll(toTidbVersion) + if err = oa.UpgradeTidbCluster(clusterInfo); err != nil { + glog.Fatal(err) + } } - restoreClusterInfo := &tests.TidbClusterInfo{ - BackupPVC: "test-backup", - Namespace: "tidb", - ClusterName: "demo2", - OperatorTag: "master", - PDImage: "pingcap/pd:v2.1.0", - TiKVImage: "pingcap/tikv:v2.1.0", - TiDBImage: "pingcap/tidb:v2.1.0", - StorageClassName: "local-storage", - Password: "admin", - InitSql: initSql, - UserName: "root", - InitSecretName: "demo2-set-secret", - BackupSecretName: "demo2-backup-secret", - Resources: map[string]string{ - "pd.resources.limits.cpu": "1000m", - "pd.resources.limits.memory": "2Gi", - "pd.resources.requests.cpu": "200m", - "pd.resources.requests.memory": "1Gi", - "tikv.resources.limits.cpu": "2000m", - "tikv.resources.limits.memory": "4Gi", - "tikv.resources.requests.cpu": "1000m", - "tikv.resources.requests.memory": "2Gi", - "tidb.resources.limits.cpu": "2000m", - "tidb.resources.limits.memory": "4Gi", - "tidb.resources.requests.cpu": "500m", - "tidb.resources.requests.memory": "1Gi", - }, - Args: map[string]string{}, + for _, clusterInfo := range clusterInfos { + if err = oa.CheckTidbClusterStatus(clusterInfo); err != nil { + glog.Fatal(err) + } } + // backup and restore + backupClusterInfo := clusterInfos[0] + restoreClusterInfo := &tests.TidbClusterInfo{} + copier.Copy(restoreClusterInfo, backupClusterInfo) + restoreClusterInfo.ClusterName = restoreClusterInfo.ClusterName + "-restore" + if err = oa.CleanTidbCluster(restoreClusterInfo); err != nil { - oa.DumpAllLogs(operatorInfo, []*tests.TidbClusterInfo{clusterInfo, restoreClusterInfo}) glog.Fatal(err) } if err = oa.DeployTidbCluster(restoreClusterInfo); err != nil { - oa.DumpAllLogs(operatorInfo, []*tests.TidbClusterInfo{clusterInfo, restoreClusterInfo}) glog.Fatal(err) } if err = oa.CheckTidbClusterStatus(restoreClusterInfo); err != nil { - oa.DumpAllLogs(operatorInfo, []*tests.TidbClusterInfo{clusterInfo, restoreClusterInfo}) glog.Fatal(err) } - backupCase := backup.NewBackupCase(oa, clusterInfo, restoreClusterInfo) + backupCase := backup.NewBackupCase(oa, backupClusterInfo, restoreClusterInfo) if err := backupCase.Run(); err != nil { - oa.DumpAllLogs(operatorInfo, []*tests.TidbClusterInfo{clusterInfo, restoreClusterInfo}) - glog.Fatal(err) - } - - fa := tests.NewFaultTriggerAction(cli, kubeCli, conf) - if err := fa.StopETCD("172.16.4.171"); err != nil { - glog.Fatal(err) - } - - time.Sleep(1 * time.Minute) - - if err := fa.StartETCD("172.16.4.171"); err != nil { glog.Fatal(err) } } diff --git a/tests/cmd/stability/main.go b/tests/cmd/stability/main.go new file mode 100644 index 0000000000..001973b168 --- /dev/null +++ b/tests/cmd/stability/main.go @@ -0,0 +1,290 @@ +// Copyright 2018 PingCAP, Inc. +// +// 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, +// See the License for the specific language governing permissions and +// limitations under the License.package spec + +package main + +import ( + "fmt" + "net/http" + _ "net/http/pprof" + "time" + + "github.com/golang/glog" + "github.com/jinzhu/copier" + "github.com/pingcap/tidb-operator/pkg/client/clientset/versioned" + "github.com/pingcap/tidb-operator/tests" + "github.com/pingcap/tidb-operator/tests/backup" + "github.com/pingcap/tidb-operator/tests/pkg/workload" + "github.com/pingcap/tidb-operator/tests/pkg/workload/ddl" + "k8s.io/apiserver/pkg/util/logs" + "k8s.io/client-go/kubernetes" + "k8s.io/client-go/rest" +) + +func main() { + logs.InitLogs() + defer logs.FlushLogs() + + conf := tests.NewConfig() + err := conf.Parse() + if err != nil { + glog.Fatalf("failed to parse config: %v", err) + } + + go func() { + glog.Info(http.ListenAndServe("localhost:6060", nil)) + }() + + // TODO read these args from config + beginTidbVersion := "v2.1.0" + toTidbVersion := "v2.1.4" + operatorTag := "master" + operatorImage := "pingcap/tidb-operator:latest" + + cfg, err := rest.InClusterConfig() + if err != nil { + glog.Fatalf("failed to get config: %v", err) + } + cli, err := versioned.NewForConfig(cfg) + if err != nil { + glog.Fatalf("failed to create Clientset: %v", err) + } + kubeCli, err := kubernetes.NewForConfig(cfg) + if err != nil { + glog.Fatalf("failed to get kubernetes Clientset: %v", err) + } + + oa := tests.NewOperatorActions(cli, kubeCli, conf) + + operatorInfo := &tests.OperatorInfo{ + Namespace: "pingcap", + ReleaseName: "operator", + Image: operatorImage, + Tag: operatorTag, + SchedulerImage: "gcr.io/google-containers/hyperkube:v1.12.1", + LogLevel: "2", + } + + // create database and table and insert a column for test backup and restore + initSql := `"create database record;use record;create table test(t char(32))"` + + clusterInfos := []*tests.TidbClusterInfo{ + { + Namespace: "e2e-cluster1", + ClusterName: "e2e-cluster1", + OperatorTag: operatorTag, + PDImage: fmt.Sprintf("pingcap/pd:%s", beginTidbVersion), + TiKVImage: fmt.Sprintf("pingcap/tikv:%s", beginTidbVersion), + TiDBImage: fmt.Sprintf("pingcap/tidb:%s", beginTidbVersion), + StorageClassName: "local-storage", + Password: "admin", + InitSql: initSql, + UserName: "root", + InitSecretName: "demo-set-secret", + BackupSecretName: "demo-backup-secret", + BackupPVC: "test-backup", + Resources: map[string]string{ + "pd.resources.limits.cpu": "1000m", + "pd.resources.limits.memory": "2Gi", + "pd.resources.requests.cpu": "200m", + "pd.resources.requests.memory": "1Gi", + "tikv.resources.limits.cpu": "2000m", + "tikv.resources.limits.memory": "4Gi", + "tikv.resources.requests.cpu": "1000m", + "tikv.resources.requests.memory": "2Gi", + "tidb.resources.limits.cpu": "2000m", + "tidb.resources.limits.memory": "4Gi", + "tidb.resources.requests.cpu": "500m", + "tidb.resources.requests.memory": "1Gi", + }, + Args: map[string]string{}, + Monitor: true, + }, + { + Namespace: "e2e-cluster2", + ClusterName: "e2e-cluster2", + OperatorTag: "master", + PDImage: fmt.Sprintf("pingcap/pd:%s", beginTidbVersion), + TiKVImage: fmt.Sprintf("pingcap/tikv:%s", beginTidbVersion), + TiDBImage: fmt.Sprintf("pingcap/tidb:%s", beginTidbVersion), + StorageClassName: "local-storage", + Password: "admin", + InitSql: initSql, + UserName: "root", + InitSecretName: "demo-set-secret", + BackupSecretName: "demo-backup-secret", + BackupPVC: "test-backup", + Resources: map[string]string{ + "pd.resources.limits.cpu": "1000m", + "pd.resources.limits.memory": "2Gi", + "pd.resources.requests.cpu": "200m", + "pd.resources.requests.memory": "1Gi", + "tikv.resources.limits.cpu": "2000m", + "tikv.resources.limits.memory": "4Gi", + "tikv.resources.requests.cpu": "1000m", + "tikv.resources.requests.memory": "2Gi", + "tidb.resources.limits.cpu": "2000m", + "tidb.resources.limits.memory": "4Gi", + "tidb.resources.requests.cpu": "500m", + "tidb.resources.requests.memory": "1Gi", + }, + Args: map[string]string{}, + Monitor: true, + }, + } + + defer func() { + oa.DumpAllLogs(operatorInfo, clusterInfos) + }() + + // deploy operator + if err := oa.CleanOperator(operatorInfo); err != nil { + oa.DumpAllLogs(operatorInfo, nil) + glog.Fatal(err) + } + if err = oa.DeployOperator(operatorInfo); err != nil { + oa.DumpAllLogs(operatorInfo, nil) + glog.Fatal(err) + } + + // deploy tidbclusters + for _, clusterInfo := range clusterInfos { + if err = oa.CleanTidbCluster(clusterInfo); err != nil { + glog.Fatal(err) + } + if err = oa.DeployTidbCluster(clusterInfo); err != nil { + glog.Fatal(err) + } + } + + for _, clusterInfo := range clusterInfos { + if err = oa.CheckTidbClusterStatus(clusterInfo); err != nil { + glog.Fatal(err) + } + } + + var workloads []workload.Workload + for _, clusterInfo := range clusterInfos { + workload := ddl.New(clusterInfo.DSN("test"), 1, 1) + workloads = append(workloads, workload) + } + + err = workload.Run(func() error { + + for _, clusterInfo := range clusterInfos { + clusterInfo = clusterInfo.ScaleTiDB(3).ScaleTiKV(5).ScalePD(5) + if err := oa.ScaleTidbCluster(clusterInfo); err != nil { + return err + } + } + for _, clusterInfo := range clusterInfos { + if err := oa.CheckTidbClusterStatus(clusterInfo); err != nil { + return err + } + } + + for _, clusterInfo := range clusterInfos { + clusterInfo = clusterInfo.ScalePD(3) + if err := oa.ScaleTidbCluster(clusterInfo); err != nil { + return err + } + } + for _, clusterInfo := range clusterInfos { + if err := oa.CheckTidbClusterStatus(clusterInfo); err != nil { + return err + } + } + + for _, clusterInfo := range clusterInfos { + clusterInfo = clusterInfo.ScaleTiKV(3) + if err := oa.ScaleTidbCluster(clusterInfo); err != nil { + return err + } + } + for _, clusterInfo := range clusterInfos { + if err := oa.CheckTidbClusterStatus(clusterInfo); err != nil { + return err + } + } + + for _, clusterInfo := range clusterInfos { + clusterInfo = clusterInfo.ScaleTiDB(1) + if err := oa.ScaleTidbCluster(clusterInfo); err != nil { + return err + } + } + for _, clusterInfo := range clusterInfos { + if err := oa.CheckTidbClusterStatus(clusterInfo); err != nil { + return err + } + } + + return nil + }, workloads...) + + if err != nil { + glog.Fatal(err) + } + + for _, clusterInfo := range clusterInfos { + if err = oa.CheckTidbClusterStatus(clusterInfo); err != nil { + glog.Fatal(err) + } + } + + for _, clusterInfo := range clusterInfos { + clusterInfo = clusterInfo.UpgradeAll(toTidbVersion) + if err = oa.UpgradeTidbCluster(clusterInfo); err != nil { + glog.Fatal(err) + } + } + + for _, clusterInfo := range clusterInfos { + if err = oa.CheckTidbClusterStatus(clusterInfo); err != nil { + glog.Fatal(err) + } + } + + // backup and restore + backupClusterInfo := clusterInfos[0] + restoreClusterInfo := &tests.TidbClusterInfo{} + copier.Copy(restoreClusterInfo, backupClusterInfo) + restoreClusterInfo.ClusterName = restoreClusterInfo.ClusterName + "-restore" + + if err = oa.CleanTidbCluster(restoreClusterInfo); err != nil { + glog.Fatal(err) + } + if err = oa.DeployTidbCluster(restoreClusterInfo); err != nil { + glog.Fatal(err) + } + if err = oa.CheckTidbClusterStatus(restoreClusterInfo); err != nil { + glog.Fatal(err) + } + + backupCase := backup.NewBackupCase(oa, backupClusterInfo, restoreClusterInfo) + + if err := backupCase.Run(); err != nil { + glog.Fatal(err) + } + + fa := tests.NewFaultTriggerAction(cli, kubeCli, conf) + if err := fa.StopETCD("172.16.4.171"); err != nil { + glog.Fatal(err) + } + + time.Sleep(1 * time.Minute) + + if err := fa.StartETCD("172.16.4.171"); err != nil { + glog.Fatal(err) + } +} diff --git a/vendor/github.com/jinzhu/copier/Guardfile b/vendor/github.com/jinzhu/copier/Guardfile new file mode 100644 index 0000000000..0b860b0653 --- /dev/null +++ b/vendor/github.com/jinzhu/copier/Guardfile @@ -0,0 +1,3 @@ +guard 'gotest' do + watch(%r{\.go$}) +end diff --git a/vendor/github.com/jinzhu/copier/License b/vendor/github.com/jinzhu/copier/License new file mode 100644 index 0000000000..e2dc5381e1 --- /dev/null +++ b/vendor/github.com/jinzhu/copier/License @@ -0,0 +1,20 @@ +The MIT License (MIT) + +Copyright (c) 2015 Jinzhu + +Permission is hereby granted, free of charge, to any person obtaining a copy of +this software and associated documentation files (the "Software"), to deal in +the Software without restriction, including without limitation the rights to +use, copy, modify, merge, publish, distribute, sublicense, and/or sell copies of +the Software, and to permit persons to whom the Software is furnished to do so, +subject to the following conditions: + +The above copyright notice and this permission notice shall be included in all +copies or substantial portions of the Software. + +THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR +IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, FITNESS +FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE AUTHORS OR +COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER LIABILITY, WHETHER +IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, OUT OF OR IN +CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE SOFTWARE. diff --git a/vendor/github.com/jinzhu/copier/README.md b/vendor/github.com/jinzhu/copier/README.md new file mode 100644 index 0000000000..f929b46793 --- /dev/null +++ b/vendor/github.com/jinzhu/copier/README.md @@ -0,0 +1,100 @@ +# Copier + + I am a copier, I copy everything from one to another + +[![wercker status](https://app.wercker.com/status/9d44ad2d4e6253929c8fb71359effc0b/s/master "wercker status")](https://app.wercker.com/project/byKey/9d44ad2d4e6253929c8fb71359effc0b) + +## Features + +* Copy from field to field with same name +* Copy from method to field with same name +* Copy from field to method with same name +* Copy from slice to slice +* Copy from struct to slice + +## Usage + +```go +package main + +import ( + "fmt" + "github.com/jinzhu/copier" +) + +type User struct { + Name string + Role string + Age int32 +} + +func (user *User) DoubleAge() int32 { + return 2 * user.Age +} + +type Employee struct { + Name string + Age int32 + DoubleAge int32 + EmployeId int64 + SuperRule string +} + +func (employee *Employee) Role(role string) { + employee.SuperRule = "Super " + role +} + +func main() { + var ( + user = User{Name: "Jinzhu", Age: 18, Role: "Admin"} + users = []User{{Name: "Jinzhu", Age: 18, Role: "Admin"}, {Name: "jinzhu 2", Age: 30, Role: "Dev"}} + employee = Employee{} + employees = []Employee{} + ) + + copier.Copy(&employee, &user) + + fmt.Printf("%#v \n", employee) + // Employee{ + // Name: "Jinzhu", // Copy from field + // Age: 18, // Copy from field + // DoubleAge: 36, // Copy from method + // EmployeeId: 0, // Ignored + // SuperRule: "Super Admin", // Copy to method + // } + + // Copy struct to slice + copier.Copy(&employees, &user) + + fmt.Printf("%#v \n", employees) + // []Employee{ + // {Name: "Jinzhu", Age: 18, DoubleAge: 36, EmployeId: 0, SuperRule: "Super Admin"} + // } + + // Copy slice to slice + employees = []Employee{} + copier.Copy(&employees, &users) + + fmt.Printf("%#v \n", employees) + // []Employee{ + // {Name: "Jinzhu", Age: 18, DoubleAge: 36, EmployeId: 0, SuperRule: "Super Admin"}, + // {Name: "jinzhu 2", Age: 30, DoubleAge: 60, EmployeId: 0, SuperRule: "Super Dev"}, + // } +} +``` + +## Contributing + +You can help to make the project better, check out [http://gorm.io/contribute.html](http://gorm.io/contribute.html) for things you can do. + +# Author + +**jinzhu** + +* +* +* + +## License + +Released under the [MIT License](https://github.com/jinzhu/copier/blob/master/License). diff --git a/vendor/github.com/jinzhu/copier/copier.go b/vendor/github.com/jinzhu/copier/copier.go new file mode 100644 index 0000000000..ecbddffb0f --- /dev/null +++ b/vendor/github.com/jinzhu/copier/copier.go @@ -0,0 +1,185 @@ +package copier + +import ( + "database/sql" + "errors" + "reflect" +) + +// Copy copy things +func Copy(toValue interface{}, fromValue interface{}) (err error) { + var ( + isSlice bool + amount = 1 + from = indirect(reflect.ValueOf(fromValue)) + to = indirect(reflect.ValueOf(toValue)) + ) + + if !to.CanAddr() { + return errors.New("copy to value is unaddressable") + } + + // Return is from value is invalid + if !from.IsValid() { + return + } + + // Just set it if possible to assign + if from.Type().AssignableTo(to.Type()) { + to.Set(from) + return + } + + fromType := indirectType(from.Type()) + toType := indirectType(to.Type()) + + if fromType.Kind() != reflect.Struct || toType.Kind() != reflect.Struct { + return + } + + if to.Kind() == reflect.Slice { + isSlice = true + if from.Kind() == reflect.Slice { + amount = from.Len() + } + } + + for i := 0; i < amount; i++ { + var dest, source reflect.Value + + if isSlice { + // source + if from.Kind() == reflect.Slice { + source = indirect(from.Index(i)) + } else { + source = indirect(from) + } + + // dest + dest = indirect(reflect.New(toType).Elem()) + } else { + source = indirect(from) + dest = indirect(to) + } + + // Copy from field to field or method + for _, field := range deepFields(fromType) { + name := field.Name + + if fromField := source.FieldByName(name); fromField.IsValid() { + // has field + if toField := dest.FieldByName(name); toField.IsValid() { + if toField.CanSet() { + if !set(toField, fromField) { + if err := Copy(toField.Addr().Interface(), fromField.Interface()); err != nil { + return err + } + } + } + } else { + // try to set to method + var toMethod reflect.Value + if dest.CanAddr() { + toMethod = dest.Addr().MethodByName(name) + } else { + toMethod = dest.MethodByName(name) + } + + if toMethod.IsValid() && toMethod.Type().NumIn() == 1 && fromField.Type().AssignableTo(toMethod.Type().In(0)) { + toMethod.Call([]reflect.Value{fromField}) + } + } + } + } + + // Copy from method to field + for _, field := range deepFields(toType) { + name := field.Name + + var fromMethod reflect.Value + if source.CanAddr() { + fromMethod = source.Addr().MethodByName(name) + } else { + fromMethod = source.MethodByName(name) + } + + if fromMethod.IsValid() && fromMethod.Type().NumIn() == 0 && fromMethod.Type().NumOut() == 1 { + if toField := dest.FieldByName(name); toField.IsValid() && toField.CanSet() { + values := fromMethod.Call([]reflect.Value{}) + if len(values) >= 1 { + set(toField, values[0]) + } + } + } + } + + if isSlice { + if dest.Addr().Type().AssignableTo(to.Type().Elem()) { + to.Set(reflect.Append(to, dest.Addr())) + } else if dest.Type().AssignableTo(to.Type().Elem()) { + to.Set(reflect.Append(to, dest)) + } + } + } + return +} + +func deepFields(reflectType reflect.Type) []reflect.StructField { + var fields []reflect.StructField + + if reflectType = indirectType(reflectType); reflectType.Kind() == reflect.Struct { + for i := 0; i < reflectType.NumField(); i++ { + v := reflectType.Field(i) + if v.Anonymous { + fields = append(fields, deepFields(v.Type)...) + } else { + fields = append(fields, v) + } + } + } + + return fields +} + +func indirect(reflectValue reflect.Value) reflect.Value { + for reflectValue.Kind() == reflect.Ptr { + reflectValue = reflectValue.Elem() + } + return reflectValue +} + +func indirectType(reflectType reflect.Type) reflect.Type { + for reflectType.Kind() == reflect.Ptr || reflectType.Kind() == reflect.Slice { + reflectType = reflectType.Elem() + } + return reflectType +} + +func set(to, from reflect.Value) bool { + if from.IsValid() { + if to.Kind() == reflect.Ptr { + //set `to` to nil if from is nil + if from.Kind() == reflect.Ptr && from.IsNil() { + to.Set(reflect.Zero(to.Type())) + return true + } else if to.IsNil() { + to.Set(reflect.New(to.Type().Elem())) + } + to = to.Elem() + } + + if from.Type().ConvertibleTo(to.Type()) { + to.Set(from.Convert(to.Type())) + } else if scanner, ok := to.Addr().Interface().(sql.Scanner); ok { + err := scanner.Scan(from.Interface()) + if err != nil { + return false + } + } else if from.Kind() == reflect.Ptr { + return set(to, from.Elem()) + } else { + return false + } + } + return true +} diff --git a/vendor/github.com/jinzhu/copier/wercker.yml b/vendor/github.com/jinzhu/copier/wercker.yml new file mode 100644 index 0000000000..5e6ce981dc --- /dev/null +++ b/vendor/github.com/jinzhu/copier/wercker.yml @@ -0,0 +1,23 @@ +box: golang + +build: + steps: + - setup-go-workspace + + # Gets the dependencies + - script: + name: go get + code: | + go get + + # Build the project + - script: + name: go build + code: | + go build ./... + + # Test the project + - script: + name: go test + code: | + go test ./... diff --git a/vendor/modules.txt b/vendor/modules.txt index e963735bad..dccfd7a16a 100644 --- a/vendor/modules.txt +++ b/vendor/modules.txt @@ -141,6 +141,8 @@ github.com/hpcloud/tail/ratelimiter github.com/hpcloud/tail/util github.com/hpcloud/tail/watch github.com/hpcloud/tail/winfile +# github.com/jinzhu/copier v0.0.0-20180308034124-7e38e58719c3 +github.com/jinzhu/copier # github.com/jonboulle/clockwork v0.1.0 github.com/jonboulle/clockwork # github.com/json-iterator/go v1.1.5