diff --git a/go/test/endtoend/backup/vtbackup/backup_only_test.go b/go/test/endtoend/backup/vtbackup/backup_only_test.go new file mode 100644 index 00000000000..24c6f939e77 --- /dev/null +++ b/go/test/endtoend/backup/vtbackup/backup_only_test.go @@ -0,0 +1,304 @@ +/* +Copyright 2019 The Vitess 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 vtbackup + +import ( + "bufio" + "fmt" + "os" + "path" + "strings" + "testing" + "time" + + "vitess.io/vitess/go/test/endtoend/cluster" + + "github.com/stretchr/testify/assert" + + "vitess.io/vitess/go/vt/log" +) + +var ( + vtInsertTest = ` + create table vt_insert_test ( + id bigint auto_increment, + msg varchar(64), + primary key (id) + ) Engine=InnoDB;` +) + +func TestTabletInitialBackup(t *testing.T) { + // Test Initial Backup Flow + // TestTabletInitialBackup will: + // - Create a shard using vtbackup and --initial-backup + // - Create the rest of the cluster restoring from backup + // - Externally Reparenting to a master tablet + // - Insert Some data + // - Verify that the cluster is working + // - Take a Second Backup + // - Bring up a second replica, and restore from the second backup + // - list the backups, remove them + + vtBackup(t, true) + backups := countBackups(t) + assert.Equal(t, 1, backups) + + // Initialize the tablets + initTablets(t, false, false) + + // Restore the Tablets + restore(t, master, "replica", "NOT_SERVING") + err := localCluster.VtctlclientProcess.ExecuteCommand( + "TabletExternallyReparented", master.Alias) + assert.Nil(t, err) + restore(t, replica1, "replica", "SERVING") + + // Run the entire backup test + firstBackupTest(t, "replica") + + tearDown(t, true) +} +func TestTabletBackupOnly(t *testing.T) { + // Test Backup Flow + // TestTabletBackupOnly will: + // - Create a shard using regular init & start tablet + // - Run initShardMaster to start replication + // - Insert Some data + // - Verify that the cluster is working + // - Take a Second Backup + // - Bring up a second replica, and restore from the second backup + // - list the backups, remove them + + // Reset the tablet object values in order on init tablet in the next step. + master.VttabletProcess.ServingStatus = "NOT_SERVING" + replica1.VttabletProcess.ServingStatus = "NOT_SERVING" + + initTablets(t, true, true) + firstBackupTest(t, "replica") + + tearDown(t, false) +} + +func firstBackupTest(t *testing.T, tabletType string) { + // Test First Backup flow. + // + // firstBackupTest will: + // - create a shard with master and replica1 only + // - run InitShardMaster + // - insert some data + // - take a backup + // - insert more data on the master + // - bring up replica2 after the fact, let it restore the backup + // - check all data is right (before+after backup data) + // - list the backup, remove it + + // Store initial backup counts + backupsCount := countBackups(t) + + // insert data on master, wait for slave to get it + _, err := master.VttabletProcess.QueryTablet(vtInsertTest, keyspaceName, true) + assert.Nil(t, err) + // Add a single row with value 'test1' to the master tablet + _, err = master.VttabletProcess.QueryTablet("insert into vt_insert_test (msg) values ('test1')", keyspaceName, true) + assert.Nil(t, err) + + // Check that the specified tablet has the expected number of rows + cluster.VerifyRowsInTablet(t, replica1, keyspaceName, 1) + + // backup the slave + log.Info("taking backup %s", time.Now()) + vtBackup(t, false) + log.Info("done taking backup %s", time.Now()) + + // check that the backup shows up in the listing + backups := countBackups(t) + assert.Equal(t, backups, backupsCount+1) + + // insert more data on the master + _, err = master.VttabletProcess.QueryTablet("insert into vt_insert_test (msg) values ('test2')", keyspaceName, true) + assert.Nil(t, err) + cluster.VerifyRowsInTablet(t, replica1, keyspaceName, 2) + + // now bring up the other slave, letting it restore from backup. + err = localCluster.VtctlclientProcess.InitTablet(replica2, cell, keyspaceName, hostname, shardName) + assert.Nil(t, err) + restore(t, replica2, "replica", "SERVING") + // Replica2 takes time to serve. Sleeping for 5 sec. + time.Sleep(5 * time.Second) + //check the new slave has the data + cluster.VerifyRowsInTablet(t, replica2, keyspaceName, 2) + + // check that the restored slave has the right local_metadata + result, err := replica2.VttabletProcess.QueryTabletWithDB("select * from local_metadata", "_vt") + assert.Nil(t, err) + assert.Equal(t, replica2.Alias, result.Rows[0][1].ToString(), "Alias") + assert.Equal(t, "ks.0", result.Rows[1][1].ToString(), "ClusterAlias") + assert.Equal(t, cell, result.Rows[2][1].ToString(), "DataCenter") + if tabletType == "replica" { + assert.Equal(t, "neutral", result.Rows[3][1].ToString(), "PromotionRule") + } else { + assert.Equal(t, "must_not", result.Rows[3][1].ToString(), "PromotionRule") + } + + removeBackups(t) + backups = countBackups(t) + assert.Equal(t, 0, backups) + +} + +func vtBackup(t *testing.T, initialBackup bool) { + // Take the back using vtbackup executable + extraArgs := []string{"-allow_first_backup", "-db-credentials-file", dbCredentialFile} + log.Info("starting backup tablet %s", time.Now()) + err := localCluster.StartVtbackup(newInitDBFile, initialBackup, keyspaceName, shardName, cell, extraArgs...) + assert.Nil(t, err) +} + +func listBackups(t *testing.T) string { + // Get a list of backup names for the current shard. + localCluster.VtctlProcess = *cluster.VtctlProcessInstance(localCluster.TopoPort, localCluster.Hostname) + backups, err := localCluster.VtctlProcess.ExecuteCommandWithOutput( + "-backup_storage_implementation", "file", + "-file_backup_storage_root", + path.Join(os.Getenv("VTDATAROOT"), "tmp", "backupstorage"), + "ListBackups", shardKsName, + ) + assert.Nil(t, err) + return backups +} + +func countBackups(t *testing.T) int { + // Count the number of backups available in current shard. + backupList := listBackups(t) + backupCount := 0 + // Counts the available backups + scanner := bufio.NewScanner(strings.NewReader(backupList)) + for scanner.Scan() { + if scanner.Text() != "" { + backupCount++ + } + } + return backupCount +} + +func removeBackups(t *testing.T) { + // Remove all the backups from the shard + backupList := listBackups(t) + + scanner := bufio.NewScanner(strings.NewReader(backupList)) + for scanner.Scan() { + if scanner.Text() != "" { + _, err := localCluster.VtctlProcess.ExecuteCommandWithOutput( + "-backup_storage_implementation", "file", + "-file_backup_storage_root", + path.Join(os.Getenv("VTDATAROOT"), "tmp", "backupstorage"), + "RemoveBackup", shardKsName, scanner.Text(), + ) + assert.Nil(t, err) + } + } + +} + +func initTablets(t *testing.T, startTablet bool, initShardMaster bool) { + // Initialize tablets + for _, tablet := range []cluster.Vttablet{*master, *replica1} { + err := localCluster.VtctlclientProcess.InitTablet(&tablet, cell, keyspaceName, hostname, shardName) + assert.Nil(t, err) + + if startTablet { + err = tablet.VttabletProcess.Setup() + assert.Nil(t, err) + } + } + + if initShardMaster { + // choose master and start replication + err := localCluster.VtctlclientProcess.InitShardMaster(keyspaceName, shardName, cell, master.TabletUID) + assert.Nil(t, err) + } +} + +func restore(t *testing.T, tablet *cluster.Vttablet, tabletType string, waitForState string) { + // Erase mysql/tablet dir, then start tablet with restore enabled. + + log.Info("restoring tablet %s", time.Now()) + resetTabletDirectory(t, *tablet, true) + + err := tablet.VttabletProcess.CreateDB(keyspaceName) + assert.Nil(t, err) + + // Start tablets + tablet.VttabletProcess.ExtraArgs = []string{"-db-credentials-file", dbCredentialFile} + tablet.VttabletProcess.TabletType = tabletType + tablet.VttabletProcess.ServingStatus = waitForState + tablet.VttabletProcess.SupportsBackup = true + err = tablet.VttabletProcess.Setup() + assert.Nil(t, err) +} + +func resetTabletDirectory(t *testing.T, tablet cluster.Vttablet, initMysql bool) { + + extraArgs := []string{"-db-credentials-file", dbCredentialFile} + tablet.MysqlctlProcess.ExtraArgs = extraArgs + + // Shutdown Mysql + err := tablet.MysqlctlProcess.Stop() + assert.Nil(t, err) + // Teardown Tablet + err = tablet.VttabletProcess.TearDown() + assert.Nil(t, err) + + // Empty the dir + err = os.RemoveAll(tablet.VttabletProcess.Directory) + assert.Nil(t, err) + + if initMysql { + // Init the Mysql + tablet.MysqlctlProcess.InitDBFile = newInitDBFile + err = tablet.MysqlctlProcess.Start() + assert.Nil(t, err) + } + +} + +func tearDown(t *testing.T, initMysql bool) { + // reset replication + promoteSlaveCommands := "STOP SLAVE; RESET SLAVE ALL; RESET MASTER;" + disableSemiSyncCommands := "SET GLOBAL rpl_semi_sync_master_enabled = false; SET GLOBAL rpl_semi_sync_slave_enabled = false" + for _, tablet := range []cluster.Vttablet{*master, *replica1, *replica2} { + _, err := tablet.VttabletProcess.QueryTablet(promoteSlaveCommands, keyspaceName, true) + assert.Nil(t, err) + _, err = tablet.VttabletProcess.QueryTablet(disableSemiSyncCommands, keyspaceName, true) + assert.Nil(t, err) + for _, db := range []string{"_vt", "vt_insert_test"} { + _, err = tablet.VttabletProcess.QueryTablet(fmt.Sprintf("drop database if exists %s", db), keyspaceName, true) + assert.Nil(t, err) + } + } + + for _, tablet := range []cluster.Vttablet{*master, *replica1, *replica2} { + //Tear down Tablet + //err := tablet.VttabletProcess.TearDown() + //assert.Nil(t, err) + err := localCluster.VtctlclientProcess.ExecuteCommand("DeleteTablet", "-allow_master", tablet.Alias) + assert.Nil(t, err) + + resetTabletDirectory(t, tablet, initMysql) + } + +} diff --git a/go/test/endtoend/backup/vtbackup/main_test.go b/go/test/endtoend/backup/vtbackup/main_test.go new file mode 100644 index 00000000000..6a597371057 --- /dev/null +++ b/go/test/endtoend/backup/vtbackup/main_test.go @@ -0,0 +1,141 @@ +/* +Copyright 2019 The Vitess 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 vtbackup + +import ( + "flag" + "fmt" + "io/ioutil" + "os" + "os/exec" + "path" + "testing" + + "vitess.io/vitess/go/test/endtoend/cluster" + "vitess.io/vitess/go/test/endtoend/sharding/initialsharding" + "vitess.io/vitess/go/vt/log" +) + +var ( + master *cluster.Vttablet + replica1 *cluster.Vttablet + replica2 *cluster.Vttablet + localCluster *cluster.LocalProcessCluster + newInitDBFile string + cell = cluster.DefaultCell + hostname = "localhost" + keyspaceName = "ks" + shardName = "0" + dbPassword = "VtDbaPass" + shardKsName = fmt.Sprintf("%s/%s", keyspaceName, shardName) + dbCredentialFile string + commonTabletArg = []string{ + "-vreplication_healthcheck_topology_refresh", "1s", + "-vreplication_healthcheck_retry_delay", "1s", + "-vreplication_retry_delay", "1s", + "-degraded_threshold", "5s", + "-lock_tables_timeout", "5s", + "-watch_replication_stream", + "-enable_replication_reporter", + "-serving_state_grace_period", "1s"} +) + +func TestMain(m *testing.M) { + flag.Parse() + + exitCode, err := func() (int, error) { + localCluster = cluster.NewCluster(cell, hostname) + defer localCluster.Teardown() + + // Start topo server + err := localCluster.StartTopo() + if err != nil { + return 1, err + } + + // Start keyspace + keyspace := &cluster.Keyspace{ + Name: keyspaceName, + } + localCluster.Keyspaces = append(localCluster.Keyspaces, *keyspace) + + // Create a new init_db.sql file that sets up passwords for all users. + // Then we use a db-credentials-file with the passwords. + dbCredentialFile = initialsharding.WriteDbCredentialToTmp(localCluster.TmpDirectory) + initDb, _ := ioutil.ReadFile(path.Join(os.Getenv("VTROOT"), "/config/init_db.sql")) + sql := string(initDb) + newInitDBFile = path.Join(localCluster.TmpDirectory, "init_db_with_passwords.sql") + sql = sql + initialsharding.GetPasswordUpdateSQL(localCluster) + err = ioutil.WriteFile(newInitDBFile, []byte(sql), 0666) + if err != nil { + return 1, err + } + + extraArgs := []string{"-db-credentials-file", dbCredentialFile} + commonTabletArg = append(commonTabletArg, "-db-credentials-file", dbCredentialFile) + + shard := cluster.Shard{ + Name: shardName, + } + + master = localCluster.GetVttabletInstance("replica", 0, "") + replica1 = localCluster.GetVttabletInstance("replica", 0, "") + replica2 = localCluster.GetVttabletInstance("replica", 0, "") + shard.Vttablets = []*cluster.Vttablet{master, replica1, replica2} + + // Start MySql processes + var mysqlProcs []*exec.Cmd + for _, tablet := range shard.Vttablets { + tablet.VttabletProcess = localCluster.GetVtprocessInstanceFromVttablet(tablet, shard.Name, keyspaceName) + tablet.VttabletProcess.DbPassword = dbPassword + tablet.VttabletProcess.ExtraArgs = commonTabletArg + tablet.VttabletProcess.SupportsBackup = true + tablet.VttabletProcess.EnableSemiSync = true + + tablet.MysqlctlProcess = *cluster.MysqlCtlProcessInstance(tablet.TabletUID, tablet.MySQLPort, localCluster.TmpDirectory) + tablet.MysqlctlProcess.InitDBFile = newInitDBFile + tablet.MysqlctlProcess.ExtraArgs = extraArgs + if proc, err := tablet.MysqlctlProcess.StartProcess(); err != nil { + return 1, err + } else { + mysqlProcs = append(mysqlProcs, proc) + } + } + for _, proc := range mysqlProcs { + if err := proc.Wait(); err != nil { + return 1, err + } + } + + // Create database + for _, tablet := range []cluster.Vttablet{*master, *replica1} { + if err := tablet.VttabletProcess.CreateDB(keyspaceName); err != nil { + return 1, err + } + } + + return m.Run(), nil + }() + + if err != nil { + log.Error(err.Error()) + os.Exit(1) + } else { + os.Exit(exitCode) + } + +} diff --git a/go/test/endtoend/backup/backup_test.go b/go/test/endtoend/backup/vtctlbackup/backup_test.go similarity index 99% rename from go/test/endtoend/backup/backup_test.go rename to go/test/endtoend/backup/vtctlbackup/backup_test.go index 162efb143c0..c6c83798102 100644 --- a/go/test/endtoend/backup/backup_test.go +++ b/go/test/endtoend/backup/vtctlbackup/backup_test.go @@ -14,7 +14,7 @@ See the License for the specific language governing permissions and limitations under the License. */ -package backup +package vtctlbackup import ( "bufio" diff --git a/go/test/endtoend/backup/main_test.go b/go/test/endtoend/backup/vtctlbackup/main_test.go similarity index 99% rename from go/test/endtoend/backup/main_test.go rename to go/test/endtoend/backup/vtctlbackup/main_test.go index 8c23f5a500c..aa95b77e951 100644 --- a/go/test/endtoend/backup/main_test.go +++ b/go/test/endtoend/backup/vtctlbackup/main_test.go @@ -14,7 +14,7 @@ See the License for the specific language governing permissions and limitations under the License. */ -package backup +package vtctlbackup import ( "flag" diff --git a/go/test/endtoend/cluster/cluster_process.go b/go/test/endtoend/cluster/cluster_process.go index b73a37a18c3..f3e75e845ff 100644 --- a/go/test/endtoend/cluster/cluster_process.go +++ b/go/test/endtoend/cluster/cluster_process.go @@ -64,6 +64,7 @@ type LocalProcessCluster struct { VtctldProcess VtctldProcess VtgateProcess VtgateProcess VtworkerProcess VtworkerProcess + VtbackupProcess VtbackupProcess nextPortForProcess int @@ -482,6 +483,7 @@ func (cluster *LocalProcessCluster) Teardown() { if err := cluster.TopoProcess.TearDown(cluster.Cell, cluster.OriginalVTDATAROOT, cluster.CurrentVTDATAROOT, *keepData); err != nil { log.Errorf("Error in etcd teardown - %s", err.Error()) } + } // StartVtworker starts a vtworker @@ -500,6 +502,26 @@ func (cluster *LocalProcessCluster) StartVtworker(cell string, extraArgs ...stri } +// StartVtbackup starts a vtbackup +func (cluster *LocalProcessCluster) StartVtbackup(newInitDBFile string, initalBackup bool, + keyspace string, shard string, cell string, extraArgs ...string) error { + log.Info("Starting vtbackup") + cluster.VtbackupProcess = *VtbackupProcessInstance( + cluster.GetAndReserveTabletUID(), + cluster.GetAndReservePort(), + newInitDBFile, + keyspace, + shard, + cell, + cluster.Hostname, + cluster.TmpDirectory, + cluster.TopoPort, + initalBackup) + cluster.VtbackupProcess.ExtraArgs = extraArgs + return cluster.VtbackupProcess.Setup() + +} + // GetAndReservePort gives port for required process func (cluster *LocalProcessCluster) GetAndReservePort() int { if cluster.nextPortForProcess == 0 { diff --git a/go/test/endtoend/cluster/vtbackup_process.go b/go/test/endtoend/cluster/vtbackup_process.go new file mode 100644 index 00000000000..01f93e67be3 --- /dev/null +++ b/go/test/endtoend/cluster/vtbackup_process.go @@ -0,0 +1,143 @@ +/* +Copyright 2019 The Vitess 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 cluster + +import ( + "fmt" + "os" + "os/exec" + "path" + "strings" + "syscall" + "time" + + "vitess.io/vitess/go/vt/log" +) + +// VtbackupProcess is a generic handle for a running Vtbackup. +// It can be spawned manually +type VtbackupProcess struct { + Name string + Binary string + CommonArg VtctlProcess + LogDir string + MysqlPort int + Directory string + + Cell string + Keyspace string + Shard string + TabletAlias string + Server string + + ExtraArgs []string + initialBackup bool + initDBfile string + dbPassword string + dbName string + + proc *exec.Cmd + exit chan error +} + +// Setup starts vtbackup process with required arguements +func (vtbackup *VtbackupProcess) Setup() (err error) { + + vtbackup.proc = exec.Command( + vtbackup.Binary, + "-topo_implementation", vtbackup.CommonArg.TopoImplementation, + "-topo_global_server_address", vtbackup.CommonArg.TopoGlobalAddress, + "-topo_global_root", vtbackup.CommonArg.TopoGlobalRoot, + "-log_dir", vtbackup.LogDir, + + //initDBfile is required to run vtbackup + "-mysql_port", fmt.Sprintf("%d", vtbackup.MysqlPort), + "-init_db_sql_file", vtbackup.initDBfile, + "-init_keyspace", vtbackup.Keyspace, + "-init_shard", vtbackup.Shard, + + //Backup Arguments are not optional + "-backup_storage_implementation", "file", + "-file_backup_storage_root", + path.Join(os.Getenv("VTDATAROOT"), "tmp", "backupstorage"), + ) + + if vtbackup.initialBackup { + vtbackup.proc.Args = append(vtbackup.proc.Args, "-initial_backup") + } + if vtbackup.ExtraArgs != nil { + vtbackup.proc.Args = append(vtbackup.proc.Args, vtbackup.ExtraArgs...) + } + + vtbackup.proc.Stderr = os.Stderr + vtbackup.proc.Stdout = os.Stdout + + vtbackup.proc.Env = append(vtbackup.proc.Env, os.Environ()...) + log.Infof("%v", strings.Join(vtbackup.proc.Args, " ")) + fmt.Println(vtbackup.proc.Args) + + err = vtbackup.proc.Run() + if err != nil { + return + } + + return nil +} + +// TearDown shutdowns the running vtbackup process +func (vtbackup *VtbackupProcess) TearDown() error { + if vtbackup.proc == nil || vtbackup.exit == nil { + return nil + } + + // Attempt graceful shutdown with SIGTERM first + vtbackup.proc.Process.Signal(syscall.SIGTERM) + + select { + case err := <-vtbackup.exit: + vtbackup.proc = nil + return err + + case <-time.After(10 * time.Second): + vtbackup.proc.Process.Kill() + vtbackup.proc = nil + return <-vtbackup.exit + } +} + +// VtbackupProcessInstance returns a vtbackup handle +// configured with the given Config. +// The process must be manually started by calling Setup() +func VtbackupProcessInstance(tabletUID int, mysqlPort int, newInitDBFile string, keyspace string, shard string, + cell string, hostname string, tmpDirectory string, topoPort int, initialBackup bool) *VtbackupProcess { + vtctl := VtctlProcessInstance(topoPort, hostname) + vtbackup := &VtbackupProcess{ + Name: "vtbackup", + Binary: "vtbackup", + CommonArg: *vtctl, + LogDir: tmpDirectory, + Directory: os.Getenv("VTDATAROOT"), + TabletAlias: fmt.Sprintf("%s-%010d", cell, tabletUID), + initDBfile: newInitDBFile, + Keyspace: keyspace, + Shard: shard, + Cell: cell, + MysqlPort: mysqlPort, + initialBackup: initialBackup, + } + return vtbackup +} diff --git a/go/test/endtoend/cluster/vttablet_process.go b/go/test/endtoend/cluster/vttablet_process.go index 1b70264b669..f4a98715d12 100644 --- a/go/test/endtoend/cluster/vttablet_process.go +++ b/go/test/endtoend/cluster/vttablet_process.go @@ -319,6 +319,9 @@ func (vttablet *VttabletProcess) QueryTabletWithDB(query string, dbname string) UnixSocket: path.Join(vttablet.Directory, "mysql.sock"), DbName: dbname, } + if vttablet.DbPassword != "" { + dbParams.Pass = vttablet.DbPassword + } return executeQuery(dbParams, query) } diff --git a/test/config.json b/test/config.json index a61ce65eab0..af5f0ba4f7d 100644 --- a/test/config.json +++ b/test/config.json @@ -1,14 +1,5 @@ { "Tests": { - "backup_only": { - "File": "backup_only.py", - "Args": [], - "Command": [], - "Manual": false, - "Shard": 2, - "RetryMax": 0, - "Tags": [] - }, "backup_mysqlctld": { "File": "backup_mysqlctld.py", "Args": [], @@ -227,7 +218,16 @@ }, "backup": { "File": "backup.go", - "Args": ["vitess.io/vitess/go/test/endtoend/backup"], + "Args": ["vitess.io/vitess/go/test/endtoend/backup/vtctlbackup"], + "Command": [], + "Manual": false, + "Shard": 11, + "RetryMax": 0, + "Tags": [] + }, + "backup_only": { + "File": "backup_only.go", + "Args": ["vitess.io/vitess/go/test/endtoend/backup/vtbackup"], "Command": [], "Manual": false, "Shard": 11,