diff --git a/CHANGELOG.md b/CHANGELOG.md index 51d50abfa..0fa4b3c08 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -20,6 +20,7 @@ - (Feature) Add `ArangoLocalStorage` CRD auto-installer - (Feature) Add `ArangoDeploymentReplication` CRD auto-installer - (Bugfix) Allow missing `token` key in License secret +- (Feature) Unify agency access ## [1.2.13](https://github.com/arangodb/kube-arangodb/tree/1.2.13) (2022-06-07) - (Bugfix) Fix arangosync members state inspection diff --git a/pkg/deployment/agency/cache.go b/pkg/deployment/agency/cache.go index 3ac4ec5ca..504138d2c 100644 --- a/pkg/deployment/agency/cache.go +++ b/pkg/deployment/agency/cache.go @@ -22,57 +22,85 @@ package agency import ( "context" - "fmt" "sync" - "time" "github.com/arangodb/go-driver/agency" api "github.com/arangodb/kube-arangodb/pkg/apis/deployment/v1" "github.com/arangodb/kube-arangodb/pkg/util/errors" + "github.com/arangodb/kube-arangodb/pkg/util/globals" ) type health struct { leaderID string + agencySize int + + names []string commitIndexes map[string]uint64 + leaders map[string]string + election map[string]int } func (h health) LeaderID() string { return h.leaderID } -// IsHealthy returns true if all agencies have the same commit index. -// Returns false when: -// - agencies' list is empty. -// - agencies have different commit indices. -// - agencies have commit indices == 0. -func (h health) IsHealthy() bool { - var globalCommitIndex uint64 - first := true - - for _, commitIndex := range h.commitIndexes { - if first { - globalCommitIndex = commitIndex - first = false - } else if commitIndex != globalCommitIndex { - return false +// Healthy returns nil if all agencies have the same commit index. +func (h health) Healthy() error { + if err := h.Serving(); err != nil { + return err + } + + if h.election[h.leaderID] != h.agencySize { + return errors.Newf("Not all agents are in quorum") + } + + index := h.commitIndexes[h.leaderID] + if index == 0 { + return errors.Newf("Agency CommitIndex is zero") + } + + for k, v := range h.commitIndexes { + if v != index { + return errors.Newf("Agent %s is behind in CommitIndex", k) } } - return globalCommitIndex != 0 + return nil +} + +func (h health) Serving() error { + if h.agencySize == 0 { + return errors.Newf("Empty agents list") + } + + if len(h.election) == 0 { + return errors.Newf("No Leader") + } else if len(h.election) > 1 { + return errors.Newf("Multiple leaders") + } + + if len(h.leaders) <= h.agencySize/2 { + return errors.Newf("Quorum is not present") + } + + return nil } // Health describes interface to check healthy of the environment. type Health interface { - // IsHealthy return true when environment is considered as healthy. - IsHealthy() bool + // Healthy return nil when environment is considered as healthy. + Healthy() error + + // Serving return nil when environment is considered as responsive, but not fully healthy. + Serving() error // LeaderID returns a leader ID or empty string if a leader is not known. LeaderID() string } type Cache interface { - Reload(ctx context.Context, clients []agency.Agency) (uint64, error) + Reload(ctx context.Context, size int, clients []agency.Agency) (uint64, error) Data() (State, bool) CommitIndex() uint64 // Health returns true when healthy object is available. @@ -107,7 +135,7 @@ func (c cacheSingle) Health() (Health, bool) { return nil, false } -func (c cacheSingle) Reload(_ context.Context, _ []agency.Agency) (uint64, error) { +func (c cacheSingle) Reload(_ context.Context, _ int, _ []agency.Agency) (uint64, error) { return 0, nil } @@ -153,15 +181,16 @@ func (c *cache) Health() (Health, bool) { return nil, false } -func (c *cache) Reload(ctx context.Context, clients []agency.Agency) (uint64, error) { +func (c *cache) Reload(ctx context.Context, size int, clients []agency.Agency) (uint64, error) { c.lock.Lock() defer c.lock.Unlock() - leaderCli, leaderConfig, health, err := getLeader(ctx, clients) + leaderCli, leaderConfig, health, err := getLeader(ctx, size, clients) if err != nil { // Invalidate a leader ID and agency state. // In the next iteration leaderID will be sat because `valid` will be false. c.valid = false + c.health = nil return 0, err } @@ -186,91 +215,62 @@ func (c *cache) Reload(ctx context.Context, clients []agency.Agency) (uint64, er // getLeader returns config and client to a leader agency, and health to check if agencies are on the same page. // If there is no quorum for the leader then error is returned. -func getLeader(ctx context.Context, clients []agency.Agency) (agency.Agency, *Config, Health, error) { - var mutex sync.Mutex - var anyError error - var wg sync.WaitGroup +func getLeader(ctx context.Context, size int, clients []agency.Agency) (agency.Agency, *Config, Health, error) { + configs := make([]*Config, len(clients)) + errs := make([]error, len(clients)) - cliLen := len(clients) - if cliLen == 0 { - return nil, nil, nil, errors.New("empty list of agencies' clients") - } - configs := make([]*Config, cliLen) - leaders := make(map[string]int, cliLen) - - var h health + var wg sync.WaitGroup - h.commitIndexes = make(map[string]uint64, cliLen) - // Fetch all configs from agencies. - wg.Add(cliLen) - for i, cli := range clients { - go func(iLocal int, cliLocal agency.Agency) { + // Fetch Agency config + for i := range clients { + wg.Add(1) + go func(id int) { defer wg.Done() - ctxLocal, cancel := context.WithTimeout(ctx, time.Second) + ctxLocal, cancel := globals.GetGlobals().Timeouts().Agency().WithTimeout(ctx) defer cancel() - config, err := GetAgencyConfig(ctxLocal, cliLocal) - - mutex.Lock() - defer mutex.Unlock() + config, err := GetAgencyConfig(ctxLocal, clients[id]) if err != nil { - anyError = err - return - } else if config == nil || config.LeaderId == "" { - anyError = fmt.Errorf("leader unknown for the agent %v", cliLocal.Connection().Endpoints()) + errs[id] = err return } // Write config on the same index where client is (It will be helpful later). - configs[iLocal] = config - // Count leaders. - leaders[config.LeaderId]++ - h.commitIndexes[config.Configuration.ID] = config.CommitIndex - }(i, cli) + configs[id] = config + }(i) } wg.Wait() - if anyError != nil { - return nil, nil, nil, wrapError(anyError, "not all agencies are responsive") - } - - if len(leaders) == 0 { - return nil, nil, nil, wrapError(anyError, "failed to get config from agencies") - } - - // Find the leader ID which has the most votes from all agencies. - maxVotes := 0 - var leaderID string - for id, votes := range leaders { - if votes > maxVotes { - maxVotes = votes - leaderID = id + var h health + h.agencySize = size + h.names = make([]string, len(clients)) + h.commitIndexes = make(map[string]uint64, len(clients)) + h.leaders = make(map[string]string, len(clients)) + h.election = make(map[string]int, len(clients)) + + for id := range configs { + if config := configs[id]; config != nil { + name := config.Configuration.ID + h.names[id] = name + h.commitIndexes[name] = config.CommitIndex + if config.LeaderId != "" { + h.leaders[name] = config.LeaderId + h.election[config.LeaderId]++ + h.leaderID = config.LeaderId + } } } - h.leaderID = leaderID - - // Check if a leader has quorum from all possible agencies. - if maxVotes <= cliLen/2 { - message := fmt.Sprintf("no quorum for leader %s, votes %d of %d", leaderID, maxVotes, cliLen) - return nil, nil, nil, wrapError(anyError, message) + if err := h.Serving(); err != nil { + return nil, nil, nil, err } - // From here on, a leader with quorum is known. - for i, config := range configs { - if config != nil && config.Configuration.ID == leaderID { - return clients[i], config, h, nil + for id := range clients { + if h.leaderID == h.names[id] { + return clients[id], configs[id], h, nil } } - return nil, nil, nil, wrapError(anyError, "the leader is not responsive") -} - -func wrapError(err error, message string) error { - if err != nil { - return errors.WithMessage(err, message) - } - - return errors.New(message) + return nil, nil, nil, errors.Newf("Unable to find agent") } diff --git a/pkg/deployment/agency/current_collections.go b/pkg/deployment/agency/current_collections.go index 942dcd3b8..7b5af0205 100644 --- a/pkg/deployment/agency/current_collections.go +++ b/pkg/deployment/agency/current_collections.go @@ -27,5 +27,5 @@ type StateCurrentDBCollections map[string]StateCurrentDBCollection type StateCurrentDBCollection map[string]StateCurrentDBShard type StateCurrentDBShard struct { - Servers ShardServers `json:"servers,omitempty"` + Servers Servers `json:"servers,omitempty"` } diff --git a/pkg/deployment/agency/definitions.go b/pkg/deployment/agency/definitions.go index 59a70dc75..74b2c9bb8 100644 --- a/pkg/deployment/agency/definitions.go +++ b/pkg/deployment/agency/definitions.go @@ -38,6 +38,13 @@ const ( SupervisionKey = "Supervision" SupervisionMaintenanceKey = "Maintenance" + + TargetJobToDoKey = "ToDo" + TargetJobPendingKey = "Pending" + TargetJobFailedKey = "Failed" + TargetJobFinishedKey = "Finished" + + TargetCleanedServersKey = "CleanedServers" ) func GetAgencyKey(parts ...string) string { diff --git a/pkg/deployment/agency/generator_jobs_test.go b/pkg/deployment/agency/generator_jobs_test.go new file mode 100644 index 000000000..2a79488b2 --- /dev/null +++ b/pkg/deployment/agency/generator_jobs_test.go @@ -0,0 +1,108 @@ +// +// DISCLAIMER +// +// Copyright 2016-2022 ArangoDB GmbH, Cologne, Germany +// +// 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. +// +// Copyright holder is ArangoDB GmbH, Cologne, Germany +// + +package agency + +import ( + "fmt" + "math/rand" + "testing" +) + +func NewJobsGenerator() JobsGeneratorInterface { + return &jobsGenerator{ + jobs: map[JobPhase]map[JobID]Job{}, + } +} + +type jobsGenerator struct { + id int + jobs map[JobPhase]map[JobID]Job +} + +func (j *jobsGenerator) Jobs(phase JobPhase, jobs int, jobTypes ...string) JobsGeneratorInterface { + if len(jobTypes) == 0 { + jobTypes = []string{"moveShard"} + } + + z := j.jobs[phase] + if z == nil { + z = map[JobID]Job{} + } + + for i := 0; i < jobs; i++ { + q := j.id + j.id++ + id := fmt.Sprintf("s%07d", q) + z[JobID(id)] = Job{ + Type: jobTypes[rand.Intn(len(jobTypes))], + } + } + + j.jobs[phase] = z + + return j +} + +func (j *jobsGenerator) Add() StateGenerator { + return func(t *testing.T, s *State) { + if m := j.jobs[JobPhaseToDo]; len(m) > 0 { + if s.Target.JobToDo == nil { + s.Target.JobToDo = map[JobID]Job{} + } + + for k, v := range m { + s.Target.JobToDo[k] = v + } + } + if m := j.jobs[JobPhasePending]; len(m) > 0 { + if s.Target.JobPending == nil { + s.Target.JobPending = map[JobID]Job{} + } + + for k, v := range m { + s.Target.JobPending[k] = v + } + } + if m := j.jobs[JobPhaseFailed]; len(m) > 0 { + if s.Target.JobFailed == nil { + s.Target.JobFailed = map[JobID]Job{} + } + + for k, v := range m { + s.Target.JobFailed[k] = v + } + } + if m := j.jobs[JobPhaseFinished]; len(m) > 0 { + if s.Target.JobFinished == nil { + s.Target.JobFinished = map[JobID]Job{} + } + + for k, v := range m { + s.Target.JobFinished[k] = v + } + } + } +} + +type JobsGeneratorInterface interface { + Jobs(phase JobPhase, jobs int, jobTypes ...string) JobsGeneratorInterface + Add() StateGenerator +} diff --git a/pkg/deployment/agency/generator_shard_test.go b/pkg/deployment/agency/generator_shard_test.go index 8cfad0e78..74a077c8f 100644 --- a/pkg/deployment/agency/generator_shard_test.go +++ b/pkg/deployment/agency/generator_shard_test.go @@ -21,24 +21,24 @@ package agency type ShardGeneratorInterface interface { - WithPlan(servers ...string) ShardGeneratorInterface - WithCurrent(servers ...string) ShardGeneratorInterface + WithPlan(servers ...Server) ShardGeneratorInterface + WithCurrent(servers ...Server) ShardGeneratorInterface Add() CollectionGeneratorInterface } type shardGenerator struct { col collectionGenerator - plan []string - current []string + plan Servers + current Servers } -func (s shardGenerator) WithPlan(servers ...string) ShardGeneratorInterface { +func (s shardGenerator) WithPlan(servers ...Server) ShardGeneratorInterface { s.plan = servers return s } -func (s shardGenerator) WithCurrent(servers ...string) ShardGeneratorInterface { +func (s shardGenerator) WithCurrent(servers ...Server) ShardGeneratorInterface { s.current = servers return s } diff --git a/pkg/deployment/agency/job.go b/pkg/deployment/agency/job.go new file mode 100644 index 000000000..bfc26960d --- /dev/null +++ b/pkg/deployment/agency/job.go @@ -0,0 +1,40 @@ +// +// DISCLAIMER +// +// Copyright 2016-2022 ArangoDB GmbH, Cologne, Germany +// +// 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. +// +// Copyright holder is ArangoDB GmbH, Cologne, Germany +// + +package agency + +type JobPhase string + +var ( + JobPhaseUnknown JobPhase = "" + JobPhaseToDo JobPhase = "ToDo" + JobPhasePending JobPhase = "Pending" + JobPhaseFailed JobPhase = "Failed" + JobPhaseFinished JobPhase = "Finished" +) + +type JobID string + +type Jobs map[JobID]Job + +type Job struct { + Type string `json:"type,omitempty"` + Reason string `json:"reason,omitempty"` +} diff --git a/pkg/deployment/agency/jobs_performance_test.go b/pkg/deployment/agency/jobs_performance_test.go new file mode 100644 index 000000000..d3cd5d631 --- /dev/null +++ b/pkg/deployment/agency/jobs_performance_test.go @@ -0,0 +1,80 @@ +// +// DISCLAIMER +// +// Copyright 2016-2022 ArangoDB GmbH, Cologne, Germany +// +// 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. +// +// Copyright holder is ArangoDB GmbH, Cologne, Germany +// + +package agency + +import ( + "fmt" + "math/rand" + "testing" + + "github.com/stretchr/testify/require" +) + +func caseJobPerformance(t *testing.T, jobs int) { + j := NewJobsGenerator() + + currentJobs := jobs + + for _, p := range []JobPhase{ + JobPhaseToDo, + JobPhasePending, + JobPhaseFinished, + } { + z := rand.Intn(currentJobs + 1) + + j = j.Jobs(p, z) + currentJobs -= z + } + j = j.Jobs(JobPhaseFailed, currentJobs) + + gen := j.Add() + + t.Run(fmt.Sprintf("Jobs %d", jobs), func(t *testing.T) { + var s State + var jids []JobID + + runWithMeasure(t, "Generate", func(t *testing.T) { + s = GenerateState(t, gen) + }) + + runWithMeasure(t, "Count", func(t *testing.T) { + jids = s.Target.GetJobIDs() + i := len(jids) + t.Logf("Count %d", i) + require.Equal(t, jobs, i) + }) + + runCountWithMeasure(t, 16, "Lookup", func(t *testing.T) { + id := jids[rand.Intn(len(jids))] + + _, z := s.Target.GetJob(id) + require.NotEqual(t, JobPhaseUnknown, z) + }) + }) +} + +func TestJobPerformance(t *testing.T) { + caseJobPerformance(t, 16) + caseJobPerformance(t, 256) + caseJobPerformance(t, 1024) + caseJobPerformance(t, 2048) + caseJobPerformance(t, 2048*16) +} diff --git a/pkg/deployment/agency/plan_collections.go b/pkg/deployment/agency/plan_collections.go index 36a5d5f0a..c136ea60b 100644 --- a/pkg/deployment/agency/plan_collections.go +++ b/pkg/deployment/agency/plan_collections.go @@ -22,7 +22,7 @@ package agency type StatePlanCollections map[string]StatePlanDBCollections -func (a StatePlanCollections) IsDBServerInDatabases(name string) bool { +func (a StatePlanCollections) IsDBServerInDatabases(name Server) bool { for _, collections := range a { if collections.IsDBServerInCollections(name) { return true @@ -33,7 +33,7 @@ func (a StatePlanCollections) IsDBServerInDatabases(name string) bool { type StatePlanDBCollections map[string]StatePlanCollection -func (a StatePlanDBCollections) IsDBServerInCollections(name string) bool { +func (a StatePlanDBCollections) IsDBServerInCollections(name Server) bool { for _, collection := range a { if collection.IsDBServerInShards(name) { return true @@ -108,7 +108,7 @@ func (a StatePlanCollection) GetName(d string) string { return *a.Name } -func (a *StatePlanCollection) IsDBServerInShards(name string) bool { +func (a *StatePlanCollection) IsDBServerInShards(name Server) bool { if a == nil { return false } diff --git a/pkg/deployment/agency/server.go b/pkg/deployment/agency/server.go new file mode 100644 index 000000000..59f4dc6ed --- /dev/null +++ b/pkg/deployment/agency/server.go @@ -0,0 +1,47 @@ +// +// DISCLAIMER +// +// Copyright 2016-2022 ArangoDB GmbH, Cologne, Germany +// +// 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. +// +// Copyright holder is ArangoDB GmbH, Cologne, Germany +// + +package agency + +type Server string + +type Servers []Server + +func (s Servers) Contains(id Server) bool { + for _, q := range s { + if q == id { + return true + } + } + + return false +} + +func (s Servers) Join(ids Servers) Servers { + r := make(Servers, 0, len(s)) + + for _, id := range ids { + if s.Contains(id) { + r = append(r, id) + } + } + + return r +} diff --git a/pkg/deployment/agency/shards.go b/pkg/deployment/agency/shards.go index b24e96f2b..19fc3b4b7 100644 --- a/pkg/deployment/agency/shards.go +++ b/pkg/deployment/agency/shards.go @@ -20,28 +20,4 @@ package agency -type Shards map[string]ShardServers - -type ShardServers []string - -func (s ShardServers) Contains(server string) bool { - for _, q := range s { - if server == q { - return true - } - } - - return false -} - -func (s ShardServers) FilterBy(b ShardServers) ShardServers { - q := make(ShardServers, 0, len(s)) - - for _, i := range s { - if b.Contains(i) { - q = append(q, i) - } - } - - return q -} +type Shards map[string]Servers diff --git a/pkg/deployment/agency/state.go b/pkg/deployment/agency/state.go index 70657738e..2eb0ba2df 100644 --- a/pkg/deployment/agency/state.go +++ b/pkg/deployment/agency/state.go @@ -45,6 +45,11 @@ func loadState(ctx context.Context, client agency.Agency) (State, error) { GetAgencyKey(ArangoKey, PlanKey, PlanCollectionsKey), GetAgencyKey(ArangoKey, CurrentKey, PlanCollectionsKey), GetAgencyKey(ArangoKey, TargetKey, TargetHotBackupKey), + GetAgencyKey(ArangoKey, TargetKey, TargetJobToDoKey), + GetAgencyKey(ArangoKey, TargetKey, TargetJobPendingKey), + GetAgencyKey(ArangoKey, TargetKey, TargetJobFailedKey), + GetAgencyKey(ArangoKey, TargetKey, TargetJobFinishedKey), + GetAgencyKey(ArangoKey, TargetKey, TargetCleanedServersKey), } req, err = req.SetBody(GetAgencyReadRequest(GetAgencyReadKey(readKeys...))) @@ -123,8 +128,8 @@ func (s State) CountShards() int { return count } -func (s State) PlanServers() []string { - q := map[string]bool{} +func (s State) PlanServers() Servers { + q := map[Server]bool{} for _, db := range s.Plan.Collections { for _, col := range db { @@ -136,7 +141,7 @@ func (s State) PlanServers() []string { } } - r := make([]string, 0, len(q)) + r := make([]Server, 0, len(q)) for k := range q { r = append(r, k) @@ -187,11 +192,11 @@ func (s State) Filter(f StateShardFilter) CollectionShardDetails { return shards[0:size] } -func GetDBServerBlockingRestartShards(s State, serverID string) CollectionShardDetails { +func GetDBServerBlockingRestartShards(s State, serverID Server) CollectionShardDetails { return s.Filter(FilterDBServerShardRestart(serverID)) } -func FilterDBServerShardRestart(serverID string) StateShardFilter { +func FilterDBServerShardRestart(serverID Server) StateShardFilter { return NegateFilter(func(s State, db, col, shard string) bool { // Filter all shards which are not blocking restart of server plan := s.Plan.Collections[db][col] @@ -203,7 +208,7 @@ func FilterDBServerShardRestart(serverID string) StateShardFilter { } current := s.Current.Collections[db][col][shard] - currentShard := current.Servers.FilterBy(planShard) + currentShard := current.Servers.Join(planShard) serverInSync := currentShard.Contains(serverID) @@ -246,11 +251,11 @@ func FilterDBServerShardRestart(serverID string) StateShardFilter { }) } -func GetDBServerShardsNotInSync(s State, serverID string) CollectionShardDetails { +func GetDBServerShardsNotInSync(s State, serverID Server) CollectionShardDetails { return s.Filter(FilterDBServerShardsNotInSync(serverID)) } -func FilterDBServerShardsNotInSync(serverID string) StateShardFilter { +func FilterDBServerShardsNotInSync(serverID Server) StateShardFilter { return NegateFilter(func(s State, db, col, shard string) bool { planShard := s.Plan.Collections[db][col].Shards[shard] diff --git a/pkg/deployment/agency/state_performance_test.go b/pkg/deployment/agency/state_performance_test.go index 12c83481a..dd9c283e0 100644 --- a/pkg/deployment/agency/state_performance_test.go +++ b/pkg/deployment/agency/state_performance_test.go @@ -29,6 +29,20 @@ import ( "github.com/stretchr/testify/require" ) +func runCountWithMeasure(t *testing.T, c int, name string, f func(t *testing.T)) { + t.Run(name, func(t *testing.T) { + n := time.Now() + defer func() { + s := time.Since(n) + t.Logf("Elapsed: %s - %s per item", s.String(), s/time.Duration(c)) + }() + + for i := 0; i < c; i++ { + runWithMeasure(t, fmt.Sprintf("R%03d", i), f) + } + }) +} + func runWithMeasure(t *testing.T, name string, f func(t *testing.T)) { t.Run(name, func(t *testing.T) { n := time.Now() @@ -63,7 +77,7 @@ func perfWithSize(t *testing.T, dbs, collections, shards, rf, servers int) { for id := 0; id < servers; id++ { name := fmt.Sprintf("server-%d", id) runWithMeasure(t, name, func(t *testing.T) { - require.Len(t, GetDBServerBlockingRestartShards(s, name), 0) + require.Len(t, GetDBServerBlockingRestartShards(s, Server(name)), 0) }) } }) @@ -72,7 +86,7 @@ func perfWithSize(t *testing.T, dbs, collections, shards, rf, servers int) { for id := 0; id < servers; id++ { name := fmt.Sprintf("server-%d", id) runWithMeasure(t, name, func(t *testing.T) { - require.Len(t, GetDBServerShardsNotInSync(s, name), 0) + require.Len(t, GetDBServerShardsNotInSync(s, Server(name)), 0) }) } }) @@ -127,7 +141,7 @@ func generateShards(t *testing.T, col CollectionGeneratorInterface, shards, rf, return c } -func getServersSublist(t *testing.T, rf, servers int) ShardServers { +func getServersSublist(t *testing.T, rf, servers int) Servers { require.NotEqual(t, 0, rf) if rf > servers { require.Fail(t, "Server count is smaller than rf") @@ -136,11 +150,11 @@ func getServersSublist(t *testing.T, rf, servers int) ShardServers { return generateServersSublist(servers)[0:rf] } -func generateServersSublist(servers int) ShardServers { - s := make(ShardServers, servers) +func generateServersSublist(servers int) Servers { + s := make(Servers, servers) for id := range s { - s[id] = fmt.Sprintf("server-%d", id) + s[id] = Server(fmt.Sprintf("server-%d", id)) } rand.Shuffle(len(s), func(i, j int) { diff --git a/pkg/deployment/agency/state_test.go b/pkg/deployment/agency/state_test.go index 51b5fe5a5..23b77c0e5 100644 --- a/pkg/deployment/agency/state_test.go +++ b/pkg/deployment/agency/state_test.go @@ -46,6 +46,9 @@ var agencyDump39Satellite []byte //go:embed testdata/agency_dump.3.9.hotbackup.json var agencyDump39HotBackup []byte +//go:embed testdata/agency_dump.3.9.jobs.json +var agencyDump39Jobs []byte + var ( data = map[string][]byte{ "3.6": agencyDump36, @@ -66,6 +69,44 @@ func Test_Unmarshal_MultiVersion(t *testing.T) { } } +func Test_Unmarshal_Jobs(t *testing.T) { + var s DumpState + require.NoError(t, json.Unmarshal(agencyDump39Jobs, &s)) + + require.Len(t, s.Agency.Arango.Target.JobToDo, 2) + require.Len(t, s.Agency.Arango.Target.JobFailed, 3) + require.Len(t, s.Agency.Arango.Target.JobPending, 1) + require.Len(t, s.Agency.Arango.Target.JobFinished, 4) + + t.Run("Check GetJob", func(t *testing.T) { + t.Run("Unknown", func(t *testing.T) { + j, s := s.Agency.Arango.Target.GetJob("955400") + require.Equal(t, JobPhaseUnknown, s) + require.Equal(t, "", j.Type) + }) + t.Run("Failed", func(t *testing.T) { + j, s := s.Agency.Arango.Target.GetJob("955410") + require.Equal(t, JobPhaseFailed, s) + require.Equal(t, "resignLeadership", j.Type) + }) + t.Run("ToDo", func(t *testing.T) { + j, s := s.Agency.Arango.Target.GetJob("955430") + require.Equal(t, JobPhaseToDo, s) + require.Equal(t, "resignLeadership", j.Type) + }) + t.Run("Pending", func(t *testing.T) { + j, s := s.Agency.Arango.Target.GetJob("955420") + require.Equal(t, JobPhasePending, s) + require.Equal(t, "resignLeadership", j.Type) + }) + t.Run("Finished", func(t *testing.T) { + j, s := s.Agency.Arango.Target.GetJob("955440") + require.Equal(t, JobPhaseFinished, s) + require.Equal(t, "resignLeadership", j.Type) + }) + }) +} + func Test_Unmarshal_LongData(t *testing.T) { data := "[{\"arango\":{\"Supervision\":{},\"Current\":{\"Collections\":{\"_system\":{\"10011\":{\"s10022\":{\"failoverCandidates\":[\"PRMR-igofehwp\",\"PRMR-lamgjtvh\"],\"errorNum\":0,\"errorMessage\":\"\",\"error\":false,\"indexes\":[{\"fields\":[\"_key\"],\"id\":\"0\",\"name\":\"primary\",\"objectId\":\"2010022\",\"sparse\":false,\"type\":\"primary\",\"unique\":true},{\"deduplicate\":true,\"estimates\":true,\"fields\":[\"mount\"],\"id\":\"10029\",\"name\":\"idx_1718347303809449984\",\"objectId\":\"2010164\",\"sparse\":true,\"type\":\"hash\",\"unique\":true}],\"servers\":[\"PRMR-igofehwp\",\"PRMR-lamgjtvh\"]}},\"10005\":{\"s10016\":{\"failoverCandidates\":[\"PRMR-igofehwp\",\"PRMR-lamgjtvh\"],\"errorNum\":0,\"errorMessage\":\"\",\"error\":false,\"indexes\":[{\"fields\":[\"_key\"],\"id\":\"0\",\"name\":\"primary\",\"objectId\":\"2010038\",\"sparse\":false,\"type\":\"primary\",\"unique\":true},{\"deduplicate\":true,\"estimates\":false,\"fields\":[\"time\"],\"id\":\"10027\",\"name\":\"idx_1718347303741292544\",\"objectId\":\"2010144\",\"sparse\":false,\"type\":\"skiplist\",\"unique\":false}],\"servers\":[\"PRMR-igofehwp\",\"PRMR-lamgjtvh\"]}},\"10012\":{\"s10023\":{\"failoverCandidates\":[\"PRMR-igofehwp\",\"PRMR-lamgjtvh\"],\"errorNum\":0,\"errorMessage\":\"\",\"error\":false,\"indexes\":[{\"fields\":[\"_key\"],\"id\":\"0\",\"name\":\"primary\",\"objectId\":\"2010032\",\"sparse\":false,\"type\":\"primary\",\"unique\":true}],\"servers\":[\"PRMR-igofehwp\",\"PRMR-lamgjtvh\"]}},\"10010\":{\"s10021\":{\"failoverCandidates\":[\"PRMR-igofehwp\",\"PRMR-lamgjtvh\"],\"errorNum\":0,\"errorMessage\":\"\",\"error\":false,\"indexes\":[{\"fields\":[\"_key\"],\"id\":\"0\",\"name\":\"primary\",\"objectId\":\"2010034\",\"sparse\":false,\"type\":\"primary\",\"unique\":true},{\"deduplicate\":true,\"estimates\":false,\"fields\":[\"queue\",\"status\",\"delayUntil\"],\"id\":\"10030\",\"name\":\"idx_1718347303839858688\",\"objectId\":\"2010174\",\"sparse\":false,\"type\":\"skiplist\",\"unique\":false},{\"deduplicate\":true,\"estimates\":false,\"fields\":[\"status\",\"queue\",\"delayUntil\"],\"id\":\"10031\",\"name\":\"idx_1718347303866073088\",\"objectId\":\"2010186\",\"sparse\":false,\"type\":\"skiplist\",\"unique\":false}],\"servers\":[\"PRMR-igofehwp\",\"PRMR-lamgjtvh\"]}},\"10004\":{\"s10015\":{\"failoverCandidates\":[\"PRMR-igofehwp\",\"PRMR-lamgjtvh\"],\"errorNum\":0,\"errorMessage\":\"\",\"error\":false,\"indexes\":[{\"fields\":[\"_key\"],\"id\":\"0\",\"name\":\"primary\",\"objectId\":\"2010036\",\"sparse\":false,\"type\":\"primary\",\"unique\":true},{\"deduplicate\":true,\"estimates\":false,\"fields\":[\"time\"],\"id\":\"10026\",\"name\":\"idx_1718347303708786688\",\"objectId\":\"2010134\",\"sparse\":false,\"type\":\"skiplist\",\"unique\":false}],\"servers\":[\"PRMR-igofehwp\",\"PRMR-lamgjtvh\"]}},\"10003\":{\"s10014\":{\"failoverCandidates\":[\"PRMR-igofehwp\",\"PRMR-lamgjtvh\"],\"errorNum\":0,\"errorMessage\":\"\",\"error\":false,\"indexes\":[{\"fields\":[\"_key\"],\"id\":\"0\",\"name\":\"primary\",\"objectId\":\"2010028\",\"sparse\":false,\"type\":\"primary\",\"unique\":true}],\"servers\":[\"PRMR-igofehwp\",\"PRMR-lamgjtvh\"]}},\"10006\":{\"s10017\":{\"failoverCandidates\":[\"PRMR-igofehwp\",\"PRMR-lamgjtvh\"],\"errorNum\":0,\"errorMessage\":\"\",\"error\":false,\"indexes\":[{\"fields\":[\"_key\"],\"id\":\"0\",\"name\":\"primary\",\"objectId\":\"2010030\",\"sparse\":false,\"type\":\"primary\",\"unique\":true},{\"deduplicate\":true,\"estimates\":false,\"fields\":[\"time\"],\"id\":\"10028\",\"name\":\"idx_1718347303770652672\",\"objectId\":\"2010154\",\"sparse\":false,\"type\":\"skiplist\",\"unique\":false}],\"servers\":[\"PRMR-igofehwp\",\"PRMR-lamgjtvh\"]}},\"10001\":{\"s10002\":{\"failoverCandidates\":[\"PRMR-igofehwp\",\"PRMR-lamgjtvh\"],\"errorNum\":0,\"errorMessage\":\"\",\"error\":false,\"indexes\":[{\"fields\":[\"_key\"],\"id\":\"0\",\"name\":\"primary\",\"objectId\":\"2010002\",\"sparse\":false,\"type\":\"primary\",\"unique\":true},{\"deduplicate\":true,\"estimates\":true,\"fields\":[\"user\"],\"id\":\"10025\",\"name\":\"idx_1718347303681523712\",\"objectId\":\"2010124\",\"sparse\":true,\"type\":\"hash\",\"unique\":true}],\"servers\":[\"PRMR-igofehwp\",\"PRMR-lamgjtvh\"]}},\"10007\":{\"s10018\":{\"failoverCandidates\":[\"PRMR-igofehwp\",\"PRMR-lamgjtvh\"],\"errorNum\":0,\"errorMessage\":\"\",\"error\":false,\"indexes\":[{\"fields\":[\"_key\"],\"id\":\"0\",\"name\":\"primary\",\"objectId\":\"2010027\",\"sparse\":false,\"type\":\"primary\",\"unique\":true}],\"servers\":[\"PRMR-igofehwp\",\"PRMR-lamgjtvh\"]}},\"10008\":{\"s10019\":{\"failoverCandidates\":[\"PRMR-igofehwp\",\"PRMR-lamgjtvh\"],\"errorNum\":0,\"errorMessage\":\"\",\"error\":false,\"indexes\":[{\"fields\":[\"_key\"],\"id\":\"0\",\"name\":\"primary\",\"objectId\":\"2010024\",\"sparse\":false,\"type\":\"primary\",\"unique\":true}],\"servers\":[\"PRMR-igofehwp\",\"PRMR-lamgjtvh\"]}},\"10009\":{\"s10020\":{\"failoverCandidates\":[\"PRMR-igofehwp\",\"PRMR-lamgjtvh\"],\"errorNum\":0,\"errorMessage\":\"\",\"error\":false,\"indexes\":[{\"fields\":[\"_key\"],\"id\":\"0\",\"name\":\"primary\",\"objectId\":\"2010040\",\"sparse\":false,\"type\":\"primary\",\"unique\":true}],\"servers\":[\"PRMR-igofehwp\",\"PRMR-lamgjtvh\"]}},\"10013\":{\"s10024\":{\"failoverCandidates\":[\"PRMR-igofehwp\",\"PRMR-lamgjtvh\"],\"errorNum\":0,\"errorMessage\":\"\",\"error\":false,\"indexes\":[{\"fields\":[\"_key\"],\"id\":\"0\",\"name\":\"primary\",\"objectId\":\"2010042\",\"sparse\":false,\"type\":\"primary\",\"unique\":true}],\"servers\":[\"PRMR-igofehwp\",\"PRMR-lamgjtvh\"]}}}}},\"Plan\":{\"Collections\":{\"_system\":{\"10011\":{\"usesRevisionsAsDocumentIds\":true,\"syncByRevision\":true,\"isSmartChild\":false,\"distributeShardsLike\":\"10001\",\"shardingStrategy\":\"hash\",\"shards\":{\"s10022\":[\"PRMR-igofehwp\",\"PRMR-lamgjtvh\"]},\"type\":2,\"status\":3,\"replicationFactor\":2,\"writeConcern\":1,\"name\":\"_apps\",\"statusString\":\"loaded\",\"isSmart\":false,\"schema\":null,\"cacheEnabled\":false,\"numberOfShards\":1,\"id\":\"10011\",\"minReplicationFactor\":1,\"deleted\":false,\"shardKeys\":[\"_key\"],\"indexes\":[{\"id\":\"0\",\"type\":\"primary\",\"name\":\"primary\",\"fields\":[\"_key\"],\"unique\":true,\"sparse\":false},{\"deduplicate\":true,\"estimates\":false,\"fields\":[\"mount\"],\"id\":\"10029\",\"inBackground\":false,\"name\":\"idx_1718347303809449984\",\"sparse\":true,\"type\":\"hash\",\"unique\":true}],\"isDisjoint\":false,\"waitForSync\":false,\"isSystem\":true,\"keyOptions\":{\"allowUserKeys\":true,\"type\":\"traditional\"}},\"10008\":{\"usesRevisionsAsDocumentIds\":true,\"syncByRevision\":true,\"isSmartChild\":false,\"distributeShardsLike\":\"10001\",\"shardingStrategy\":\"hash\",\"shards\":{\"s10019\":[\"PRMR-igofehwp\",\"PRMR-lamgjtvh\"]},\"type\":2,\"status\":3,\"replicationFactor\":2,\"writeConcern\":1,\"name\":\"_aqlfunctions\",\"statusString\":\"loaded\",\"isSmart\":false,\"schema\":null,\"cacheEnabled\":false,\"numberOfShards\":1,\"id\":\"10008\",\"minReplicationFactor\":1,\"deleted\":false,\"shardKeys\":[\"_key\"],\"indexes\":[{\"id\":\"0\",\"type\":\"primary\",\"name\":\"primary\",\"fields\":[\"_key\"],\"unique\":true,\"sparse\":false}],\"isDisjoint\":false,\"waitForSync\":false,\"isSystem\":true,\"keyOptions\":{\"allowUserKeys\":true,\"type\":\"traditional\"}},\"10001\":{\"usesRevisionsAsDocumentIds\":true,\"syncByRevision\":true,\"isSmartChild\":false,\"shardingStrategy\":\"hash\",\"shards\":{\"s10002\":[\"PRMR-igofehwp\",\"PRMR-lamgjtvh\"]},\"type\":2,\"status\":3,\"replicationFactor\":2,\"writeConcern\":1,\"waitForSync\":false,\"schema\":null,\"shardKeys\":[\"_key\"],\"isDisjoint\":false,\"indexes\":[{\"id\":\"0\",\"type\":\"primary\",\"name\":\"primary\",\"fields\":[\"_key\"],\"unique\":true,\"sparse\":false},{\"deduplicate\":true,\"estimates\":false,\"fields\":[\"user\"],\"id\":\"10025\",\"inBackground\":false,\"name\":\"idx_1718347303681523712\",\"sparse\":true,\"type\":\"hash\",\"unique\":true}],\"cacheEnabled\":false,\"deleted\":false,\"statusString\":\"loaded\",\"isSmart\":false,\"numberOfShards\":1,\"minReplicationFactor\":1,\"id\":\"10001\",\"name\":\"_users\",\"isSystem\":true,\"keyOptions\":{\"allowUserKeys\":true,\"type\":\"traditional\"}},\"10007\":{\"usesRevisionsAsDocumentIds\":true,\"syncByRevision\":true,\"isSmartChild\":false,\"distributeShardsLike\":\"10001\",\"shardingStrategy\":\"hash\",\"shards\":{\"s10018\":[\"PRMR-igofehwp\",\"PRMR-lamgjtvh\"]},\"type\":2,\"status\":3,\"replicationFactor\":2,\"writeConcern\":1,\"name\":\"_analyzers\",\"statusString\":\"loaded\",\"isSmart\":false,\"schema\":null,\"cacheEnabled\":false,\"numberOfShards\":1,\"id\":\"10007\",\"minReplicationFactor\":1,\"deleted\":false,\"shardKeys\":[\"_key\"],\"indexes\":[{\"id\":\"0\",\"type\":\"primary\",\"name\":\"primary\",\"fields\":[\"_key\"],\"unique\":true,\"sparse\":false}],\"isDisjoint\":false,\"waitForSync\":false,\"isSystem\":true,\"keyOptions\":{\"allowUserKeys\":true,\"type\":\"traditional\"}},\"10003\":{\"usesRevisionsAsDocumentIds\":true,\"syncByRevision\":true,\"isSmartChild\":false,\"distributeShardsLike\":\"10001\",\"shardingStrategy\":\"hash\",\"shards\":{\"s10014\":[\"PRMR-igofehwp\",\"PRMR-lamgjtvh\"]},\"type\":2,\"status\":3,\"replicationFactor\":2,\"writeConcern\":1,\"name\":\"_graphs\",\"statusString\":\"loaded\",\"isSmart\":false,\"schema\":null,\"cacheEnabled\":false,\"numberOfShards\":1,\"id\":\"10003\",\"minReplicationFactor\":1,\"deleted\":false,\"shardKeys\":[\"_key\"],\"indexes\":[{\"id\":\"0\",\"type\":\"primary\",\"name\":\"primary\",\"fields\":[\"_key\"],\"unique\":true,\"sparse\":false}],\"isDisjoint\":false,\"waitForSync\":false,\"isSystem\":true,\"keyOptions\":{\"allowUserKeys\":true,\"type\":\"traditional\"}},\"10006\":{\"usesRevisionsAsDocumentIds\":true,\"syncByRevision\":true,\"isSmartChild\":false,\"distributeShardsLike\":\"10001\",\"shardingStrategy\":\"hash\",\"shards\":{\"s10017\":[\"PRMR-igofehwp\",\"PRMR-lamgjtvh\"]},\"type\":2,\"status\":3,\"replicationFactor\":2,\"writeConcern\":1,\"name\":\"_statisticsRaw\",\"statusString\":\"loaded\",\"isSmart\":false,\"schema\":null,\"cacheEnabled\":false,\"numberOfShards\":1,\"id\":\"10006\",\"minReplicationFactor\":1,\"deleted\":false,\"shardKeys\":[\"_key\"],\"indexes\":[{\"id\":\"0\",\"type\":\"primary\",\"name\":\"primary\",\"fields\":[\"_key\"],\"unique\":true,\"sparse\":false},{\"deduplicate\":true,\"estimates\":false,\"fields\":[\"time\"],\"id\":\"10028\",\"inBackground\":false,\"name\":\"idx_1718347303770652672\",\"sparse\":false,\"type\":\"skiplist\",\"unique\":false}],\"isDisjoint\":false,\"waitForSync\":false,\"isSystem\":true,\"keyOptions\":{\"allowUserKeys\":true,\"type\":\"traditional\"}},\"10012\":{\"usesRevisionsAsDocumentIds\":true,\"syncByRevision\":true,\"isSmartChild\":false,\"distributeShardsLike\":\"10001\",\"shardingStrategy\":\"hash\",\"shards\":{\"s10023\":[\"PRMR-igofehwp\",\"PRMR-lamgjtvh\"]},\"type\":2,\"status\":3,\"replicationFactor\":2,\"writeConcern\":1,\"name\":\"_appbundles\",\"statusString\":\"loaded\",\"isSmart\":false,\"schema\":null,\"cacheEnabled\":false,\"numberOfShards\":1,\"id\":\"10012\",\"minReplicationFactor\":1,\"deleted\":false,\"shardKeys\":[\"_key\"],\"indexes\":[{\"id\":\"0\",\"type\":\"primary\",\"name\":\"primary\",\"fields\":[\"_key\"],\"unique\":true,\"sparse\":false}],\"isDisjoint\":false,\"waitForSync\":false,\"isSystem\":true,\"keyOptions\":{\"allowUserKeys\":true,\"type\":\"traditional\"}},\"10010\":{\"usesRevisionsAsDocumentIds\":true,\"syncByRevision\":true,\"isSmartChild\":false,\"distributeShardsLike\":\"10001\",\"shardingStrategy\":\"hash\",\"shards\":{\"s10021\":[\"PRMR-igofehwp\",\"PRMR-lamgjtvh\"]},\"type\":2,\"status\":3,\"replicationFactor\":2,\"writeConcern\":1,\"name\":\"_jobs\",\"statusString\":\"loaded\",\"isSmart\":false,\"schema\":null,\"cacheEnabled\":false,\"numberOfShards\":1,\"id\":\"10010\",\"minReplicationFactor\":1,\"deleted\":false,\"shardKeys\":[\"_key\"],\"indexes\":[{\"id\":\"0\",\"type\":\"primary\",\"name\":\"primary\",\"fields\":[\"_key\"],\"unique\":true,\"sparse\":false},{\"deduplicate\":true,\"estimates\":false,\"fields\":[\"queue\",\"status\",\"delayUntil\"],\"id\":\"10030\",\"inBackground\":false,\"name\":\"idx_1718347303839858688\",\"sparse\":false,\"type\":\"skiplist\",\"unique\":false},{\"deduplicate\":true,\"estimates\":false,\"fields\":[\"status\",\"queue\",\"delayUntil\"],\"id\":\"10031\",\"inBackground\":false,\"name\":\"idx_1718347303866073088\",\"sparse\":false,\"type\":\"skiplist\",\"unique\":false}],\"isDisjoint\":false,\"waitForSync\":false,\"isSystem\":true,\"keyOptions\":{\"allowUserKeys\":true,\"type\":\"traditional\"}},\"10004\":{\"usesRevisionsAsDocumentIds\":true,\"syncByRevision\":true,\"isSmartChild\":false,\"distributeShardsLike\":\"10001\",\"shardingStrategy\":\"hash\",\"shards\":{\"s10015\":[\"PRMR-igofehwp\",\"PRMR-lamgjtvh\"]},\"type\":2,\"status\":3,\"replicationFactor\":2,\"writeConcern\":1,\"name\":\"_statistics\",\"statusString\":\"loaded\",\"isSmart\":false,\"schema\":null,\"cacheEnabled\":false,\"numberOfShards\":1,\"id\":\"10004\",\"minReplicationFactor\":1,\"deleted\":false,\"shardKeys\":[\"_key\"],\"indexes\":[{\"id\":\"0\",\"type\":\"primary\",\"name\":\"primary\",\"fields\":[\"_key\"],\"unique\":true,\"sparse\":false},{\"deduplicate\":true,\"estimates\":false,\"fields\":[\"time\"],\"id\":\"10026\",\"inBackground\":false,\"name\":\"idx_1718347303708786688\",\"sparse\":false,\"type\":\"skiplist\",\"unique\":false}],\"isDisjoint\":false,\"waitForSync\":false,\"isSystem\":true,\"keyOptions\":{\"allowUserKeys\":true,\"type\":\"traditional\"}},\"10005\":{\"usesRevisionsAsDocumentIds\":true,\"syncByRevision\":true,\"isSmartChild\":false,\"distributeShardsLike\":\"10001\",\"shardingStrategy\":\"hash\",\"shards\":{\"s10016\":[\"PRMR-igofehwp\",\"PRMR-lamgjtvh\"]},\"type\":2,\"status\":3,\"replicationFactor\":2,\"writeConcern\":1,\"name\":\"_statistics15\",\"statusString\":\"loaded\",\"isSmart\":false,\"schema\":null,\"cacheEnabled\":false,\"numberOfShards\":1,\"id\":\"10005\",\"minReplicationFactor\":1,\"deleted\":false,\"shardKeys\":[\"_key\"],\"indexes\":[{\"id\":\"0\",\"type\":\"primary\",\"name\":\"primary\",\"fields\":[\"_key\"],\"unique\":true,\"sparse\":false},{\"deduplicate\":true,\"estimates\":false,\"fields\":[\"time\"],\"id\":\"10027\",\"inBackground\":false,\"name\":\"idx_1718347303741292544\",\"sparse\":false,\"type\":\"skiplist\",\"unique\":false}],\"isDisjoint\":false,\"waitForSync\":false,\"isSystem\":true,\"keyOptions\":{\"allowUserKeys\":true,\"type\":\"traditional\"}},\"10009\":{\"usesRevisionsAsDocumentIds\":true,\"syncByRevision\":true,\"isSmartChild\":false,\"distributeShardsLike\":\"10001\",\"shardingStrategy\":\"hash\",\"shards\":{\"s10020\":[\"PRMR-igofehwp\",\"PRMR-lamgjtvh\"]},\"type\":2,\"status\":3,\"replicationFactor\":2,\"writeConcern\":1,\"name\":\"_queues\",\"statusString\":\"loaded\",\"isSmart\":false,\"schema\":null,\"cacheEnabled\":false,\"numberOfShards\":1,\"id\":\"10009\",\"minReplicationFactor\":1,\"deleted\":false,\"shardKeys\":[\"_key\"],\"indexes\":[{\"id\":\"0\",\"type\":\"primary\",\"name\":\"primary\",\"fields\":[\"_key\"],\"unique\":true,\"sparse\":false}],\"isDisjoint\":false,\"waitForSync\":false,\"isSystem\":true,\"keyOptions\":{\"allowUserKeys\":true,\"type\":\"traditional\"}},\"10013\":{\"usesRevisionsAsDocumentIds\":true,\"syncByRevision\":true,\"isSmartChild\":false,\"distributeShardsLike\":\"10001\",\"shardingStrategy\":\"hash\",\"shards\":{\"s10024\":[\"PRMR-igofehwp\",\"PRMR-lamgjtvh\"]},\"type\":2,\"status\":3,\"replicationFactor\":2,\"writeConcern\":1,\"name\":\"_frontend\",\"statusString\":\"loaded\",\"isSmart\":false,\"schema\":null,\"cacheEnabled\":false,\"numberOfShards\":1,\"id\":\"10013\",\"minReplicationFactor\":1,\"deleted\":false,\"shardKeys\":[\"_key\"],\"indexes\":[{\"id\":\"0\",\"type\":\"primary\",\"name\":\"primary\",\"fields\":[\"_key\"],\"unique\":true,\"sparse\":false}],\"isDisjoint\":false,\"waitForSync\":false,\"isSystem\":true,\"keyOptions\":{\"allowUserKeys\":true,\"type\":\"traditional\"}}}}}}}]" var s StateRoots @@ -107,14 +148,14 @@ func Test_IsDBServerInSync(t *testing.T) { t.Run("InSync", func(t *testing.T) { for _, server := range tc.inSync { t.Run(server, func(t *testing.T) { - require.Len(t, GetDBServerBlockingRestartShards(s, server), 0, "server %s should be in sync", server) + require.Len(t, GetDBServerBlockingRestartShards(s, Server(server)), 0, "server %s should be in sync", server) }) } }) t.Run("NotInSync", func(t *testing.T) { for _, server := range tc.notInSync { t.Run(server, func(t *testing.T) { - require.NotEqual(t, GetDBServerBlockingRestartShards(s, server), 0, "server %s should not be in sync", server) + require.NotEqual(t, GetDBServerBlockingRestartShards(s, Server(server)), 0, "server %s should not be in sync", server) }) } }) @@ -215,10 +256,10 @@ func Test_IsDBServerReadyToRestart(t *testing.T) { t.Run(name, func(t *testing.T) { s := GenerateState(t, tc.generator) for _, server := range tc.ready { - require.Len(t, s.Filter(FilterDBServerShardRestart(server)), 0, "server %s should be in sync", server) + require.Len(t, s.Filter(FilterDBServerShardRestart(Server(server))), 0, "server %s should be in sync", server) } for _, server := range tc.notReady { - require.NotEqual(t, len(s.Filter(FilterDBServerShardRestart(server))), 0, "server %s should not be in sync", server) + require.NotEqual(t, len(s.Filter(FilterDBServerShardRestart(Server(server)))), 0, "server %s should not be in sync", server) } }) } diff --git a/pkg/deployment/agency/target.go b/pkg/deployment/agency/target.go index d0dfc768d..d3d29d203 100644 --- a/pkg/deployment/agency/target.go +++ b/pkg/deployment/agency/target.go @@ -21,9 +21,61 @@ package agency type StateTarget struct { + // Jobs Section + + JobToDo Jobs `json:"ToDo,omitempty"` + JobPending Jobs `json:"Pending,omitempty"` + JobFailed Jobs `json:"Failed,omitempty"` + JobFinished Jobs `json:"Finished,omitempty"` + + // Servers Section + + CleanedServers Servers `json:"CleanedServers,omitempty"` + + // HotBackup section + HotBackup StateTargetHotBackup `json:"HotBackup,omitempty"` } +func (s StateTarget) GetJob(id JobID) (Job, JobPhase) { + if v, ok := s.JobToDo[id]; ok { + return v, JobPhaseToDo + } + if v, ok := s.JobPending[id]; ok { + return v, JobPhasePending + } + if v, ok := s.JobFailed[id]; ok { + return v, JobPhaseFailed + } + if v, ok := s.JobFinished[id]; ok { + return v, JobPhaseFinished + } + + return Job{}, JobPhaseUnknown +} + +func (s StateTarget) GetJobIDs() []JobID { + r := make([]JobID, 0, len(s.JobToDo)+len(s.JobPending)+len(s.JobFinished)+len(s.JobFailed)) + + for k := range s.JobToDo { + r = append(r, k) + } + + for k := range s.JobPending { + r = append(r, k) + } + + for k := range s.JobFinished { + r = append(r, k) + } + + for k := range s.JobFailed { + r = append(r, k) + } + + return r +} + type StateTargetHotBackup struct { Create StateTimestamp `json:"Create,omitempty"` } diff --git a/pkg/deployment/agency/testdata/agency_dump.3.9.jobs.json b/pkg/deployment/agency/testdata/agency_dump.3.9.jobs.json new file mode 100644 index 000000000..a086a11e8 --- /dev/null +++ b/pkg/deployment/agency/testdata/agency_dump.3.9.jobs.json @@ -0,0 +1,6900 @@ +{ + "index": 132, + "term": 1, + "agency": { + "arango": { + "SystemCollectionsCreated": true, + "ClusterUpgradeVersion": 30900, + "Bootstrap": "CRDN-g6s8btds: done", + ".license": { + "version": 1, + "license": "JD4EOk5VNRAPdSRUOywRewdAVAJgUm4GPjwpDCwTY0RhNylaFGQaMh0bPG5bNCsfYAMSOCAqd2B5CBNkL1xIMSA7AQECagIDOR94QlA9CzMBAG9qETMxQjQIPAolCzhcB1FaLFYaIk8VECUjSUEDVj5+FWE8LQF6AHVIBQAAAxwAQSkILBMUFTdzLHwVMzM7EiUeTBNeKQBvQ10gDywiJX5gFysKXQFSJhQVADBoVDZkDmE/Nj0MdRo4fwN8CEZjE3YsCyYWMF1hCSs8VzokBhcqPj98VQgEOG8dLxgGDRYGQzNbMkNdQ2I9cCwqOX9wJQgDXzs3OC4efGVqATVZG3gmECAgExgldBt7FBJhK2UOcysmA2gNRT8tUjoHJy02LRBJLwFjVwUoCBICJgkeWBAADyNXJEEpGHV/E0luHBwkYzswExQNBw1BAShlN1sBAEgtLiwBd0FlDz18P1oGMhUlHmdpQQ0oYQYGNSUCOgB+PS18AGErVC0XIz03HR1DAUFaK2dbCygXOHt7EiIlQj8MJCAKJDhfMjRdU1YcLQMMACI5eEJWEylxQGYPFgo3NmViGgQ+XV82JCYwHgdkFipYLEcrVk8pIwYCVCthKkV/GlUwFhIUPklUFy89BD8bJxcsOTFGFVBUJ3k8ADwxNCo5YzR5UR99FF0wAxJ8YB1ZQy8fUyg0HxMAJh1/VAtvOAQnIxkEHnoBaAlgOhoORVUHCysdHGsJJw8hQjslQwkWfCx5BydaJXgnDE8gACYBSRoDCydYKAA9CSw2NR5tRiladAQpQSp0GT8UIzJhEgMoMDwFC38GXSN0IThvC2gdF3UmHk9sMTMybjsUEDgzIQ15IyhXCHE8Ni15dwM6RQhTUDlsPnc7Mx5+LHRsQSMFVSUzRhQBGC1gLTldNHJEMR4GDTYZRAUCOR9aN2g9GyoDEHt0Jj0xbj9SCgcICAZhMyt0MmE5KggkKnoiYjZoVxEFM3owcwp+AnpANjw8Ayk7JzJ2FDhvMi1aJEY/CBFzCiMsZiBILgtSJnsQE3YIGGYKFyAOcCBQPQkeNzBoFTN7U18gBy0bLRoRVQBlJkZyEnYNdBEIElh2KzcleCgwGxR2IiV+CQdwOAYzBRksATYAdQFwLjZ8KH9bEw4vZ0VUPh8uBCA2NA4jBg5HKiZ4LWAbKgMVAAg6dEJ0CxAEO38OByN7NnhyRQAzdCstNXQFL2QUIzZ+DUEkMEp4DiBkEA==", + "hash": "ad800a7be16f904b1ae3e3889c17b0ceb2672ccc0bb8fa9990a724a582ecaa3b", + "features": { + "expires": 1646985599 + } + }, + "Sync": { + "UserVersion": 2, + "ServerStates": {}, + "LatestID": 10010001, + "Problems": {}, + "HeartbeatIntervalMs": 1000 + }, + "Supervision": { + "State": { + "Timestamp": "2022-03-09T13:37:41Z", + "Mode": "Normal" + }, + "Shards": {}, + "Health": { + "CRDN-ssc2yc0x": { + "Timestamp": "2022-03-09T13:37:55Z", + "SyncStatus": "SERVING", + "ShortName": "Coordinator0002", + "Status": "GOOD", + "LastAckedTime": "2022-03-09T13:37:55Z", + "Host": "ip-10-0-101-183.eu-central-1.compute.internal", + "Engine": "rocksdb", + "Version": "3.9.0", + "SyncTime": "2022-03-09T13:37:55Z", + "Endpoint": "ssl://cluster-uexabr-coordinator-ssc2yc0x.cluster-uexabr-int.3dcd4eac-c549-41b6-bc8f-40e11983ebd2.svc:8529" + }, + "CRDN-cg5kjymo": { + "Timestamp": "2022-03-09T13:37:55Z", + "SyncStatus": "SERVING", + "ShortName": "Coordinator0003", + "Status": "GOOD", + "LastAckedTime": "2022-03-09T13:37:55Z", + "Host": "ip-10-0-101-110.eu-central-1.compute.internal", + "Engine": "rocksdb", + "Version": "3.9.0", + "SyncTime": "2022-03-09T13:37:55Z", + "Endpoint": "ssl://cluster-uexabr-coordinator-cg5kjymo.cluster-uexabr-int.3dcd4eac-c549-41b6-bc8f-40e11983ebd2.svc:8529" + }, + "PRMR-rg51yv2h": { + "Timestamp": "2022-03-09T13:38:06Z", + "SyncStatus": "SERVING", + "ShortName": "DBServer0003", + "Status": "GOOD", + "LastAckedTime": "2022-03-09T13:38:06Z", + "Host": "ip-10-0-101-233.eu-central-1.compute.internal", + "Engine": "rocksdb", + "Version": "3.9.0", + "SyncTime": "2022-03-09T13:38:06Z", + "Endpoint": "ssl://cluster-uexabr-dbserver-rg51yv2h.cluster-uexabr-int.3dcd4eac-c549-41b6-bc8f-40e11983ebd2.svc:8529" + }, + "CRDN-g6s8btds": { + "Timestamp": "2022-03-09T13:37:55Z", + "SyncStatus": "SERVING", + "ShortName": "Coordinator0001", + "Status": "GOOD", + "LastAckedTime": "2022-03-09T13:37:55Z", + "Host": "ip-10-0-100-174.eu-central-1.compute.internal", + "Engine": "rocksdb", + "Version": "3.9.0", + "SyncTime": "2022-03-09T13:37:55Z", + "Endpoint": "ssl://cluster-uexabr-coordinator-g6s8btds.cluster-uexabr-int.3dcd4eac-c549-41b6-bc8f-40e11983ebd2.svc:8529" + }, + "PRMR-a24adubi": { + "Timestamp": "2022-03-09T13:37:56Z", + "SyncStatus": "SERVING", + "ShortName": "DBServer0001", + "Status": "GOOD", + "LastAckedTime": "2022-03-09T13:37:56Z", + "Host": "ip-10-0-101-183.eu-central-1.compute.internal", + "Engine": "rocksdb", + "Version": "3.9.0", + "SyncTime": "2022-03-09T13:37:56Z", + "Endpoint": "ssl://cluster-uexabr-dbserver-a24adubi.cluster-uexabr-int.3dcd4eac-c549-41b6-bc8f-40e11983ebd2.svc:8529" + }, + "PRMR-n92yizyp": { + "Timestamp": "2022-03-09T13:37:56Z", + "SyncStatus": "SERVING", + "ShortName": "DBServer0002", + "Status": "GOOD", + "LastAckedTime": "2022-03-09T13:37:56Z", + "Host": "ip-10-0-101-138.eu-central-1.compute.internal", + "Engine": "rocksdb", + "Version": "3.9.0", + "SyncTime": "2022-03-09T13:37:55Z", + "Endpoint": "ssl://cluster-uexabr-dbserver-n92yizyp.cluster-uexabr-int.3dcd4eac-c549-41b6-bc8f-40e11983ebd2.svc:8529" + } + }, + "DBServers": {} + }, + "Target": { + "LatestDBServerId": 3, + "CleanedServers": [], + "NumberOfCoordinators": 3, + "Failed": { + "955410": { + "type": "resignLeadership" + }, + "955411": { + "type": "resignLeadership" + }, + "955412": { + "type": "resignLeadership" + } + }, + "Lock": "UNLOCKED", + "LatestCoordinatorId": 3, + "FailedServers": {}, + "Pending": { + "955420": { + "type": "resignLeadership" + } + }, + "Finished": { + "955440": { + "type": "resignLeadership" + }, + "955441": { + "type": "resignLeadership" + }, + "955442": { + "type": "resignLeadership" + }, + "955443": { + "type": "resignLeadership" + } + }, + "ToBeCleanedServers": [], + "ToDo": { + "955430": { + "type": "resignLeadership" + }, + "955431": { + "type": "resignLeadership" + } + }, + "MapUniqueToShortID": { + "PRMR-n92yizyp": { + "TransactionID": 2, + "ShortName": "DBServer0002" + }, + "CRDN-cg5kjymo": { + "TransactionID": 3, + "ShortName": "Coordinator0003" + }, + "PRMR-a24adubi": { + "TransactionID": 1, + "ShortName": "DBServer0001" + }, + "CRDN-ssc2yc0x": { + "TransactionID": 2, + "ShortName": "Coordinator0002" + }, + "PRMR-rg51yv2h": { + "TransactionID": 3, + "ShortName": "DBServer0003" + }, + "CRDN-g6s8btds": { + "TransactionID": 1, + "ShortName": "Coordinator0001" + } + }, + "NumberOfDBServers": 3, + "Version": 1 + }, + "InitDone": true, + "Plan": { + "Views": { + "_system": {} + }, + "Version": 25, + "Lock": "UNLOCKED", + "Databases": { + "_system": { + "name": "_system", + "id": "1" + } + }, + "Coordinators": { + "CRDN-cg5kjymo": "none", + "CRDN-ssc2yc0x": "none", + "CRDN-g6s8btds": "none" + }, + "AsyncReplication": {}, + "Singles": {}, + "DBServers": { + "PRMR-rg51yv2h": "none", + "PRMR-n92yizyp": "none", + "PRMR-a24adubi": "none" + }, + "Collections": { + "_system": { + "10011": { + "writeConcern": 1, + "waitForSync": false, + "usesRevisionsAsDocumentIds": true, + "syncByRevision": true, + "shards": { + "s10022": [ + "PRMR-a24adubi", + "PRMR-n92yizyp" + ] + }, + "schema": null, + "type": 2, + "status": 3, + "replicationFactor": "satellite", + "cacheEnabled": false, + "isSystem": true, + "deleted": false, + "shardingStrategy": "hash", + "distributeShardsLike": "10001", + "internalValidatorType": 0, + "isSmartChild": false, + "shardKeys": [ + "_key" + ], + "indexes": [ + { + "id": "0", + "type": "primary", + "name": "primary", + "fields": [ + "_key" + ], + "unique": true, + "sparse": false + }, + { + "deduplicate": true, + "estimates": false, + "fields": [ + "mount" + ], + "id": "10029", + "inBackground": false, + "name": "idx_1726829640049229824", + "sparse": true, + "type": "hash", + "unique": true + } + ], + "isDisjoint": false, + "numberOfShards": 1, + "id": "10011", + "minReplicationFactor": 1, + "statusString": "loaded", + "isSmart": false, + "name": "_apps", + "keyOptions": { + "type": "traditional", + "allowUserKeys": true + } + }, + "10013": { + "writeConcern": 1, + "waitForSync": false, + "usesRevisionsAsDocumentIds": true, + "syncByRevision": true, + "shards": { + "s10024": [ + "PRMR-a24adubi", + "PRMR-n92yizyp" + ] + }, + "schema": null, + "type": 2, + "status": 3, + "replicationFactor": 2, + "cacheEnabled": false, + "isSystem": true, + "deleted": false, + "shardingStrategy": "hash", + "distributeShardsLike": "10001", + "internalValidatorType": 0, + "isSmartChild": false, + "shardKeys": [ + "_key" + ], + "indexes": [ + { + "id": "0", + "type": "primary", + "name": "primary", + "fields": [ + "_key" + ], + "unique": true, + "sparse": false + } + ], + "isDisjoint": false, + "numberOfShards": 1, + "id": "10013", + "minReplicationFactor": 1, + "statusString": "loaded", + "isSmart": false, + "name": "_frontend", + "keyOptions": { + "type": "traditional", + "allowUserKeys": true + } + }, + "10009": { + "writeConcern": 1, + "waitForSync": false, + "usesRevisionsAsDocumentIds": true, + "syncByRevision": true, + "shards": { + "s10020": [ + "PRMR-a24adubi", + "PRMR-n92yizyp" + ] + }, + "schema": null, + "type": 2, + "status": 3, + "replicationFactor": 2, + "cacheEnabled": false, + "isSystem": true, + "deleted": false, + "shardingStrategy": "hash", + "distributeShardsLike": "10001", + "internalValidatorType": 0, + "isSmartChild": false, + "shardKeys": [ + "_key" + ], + "indexes": [ + { + "id": "0", + "type": "primary", + "name": "primary", + "fields": [ + "_key" + ], + "unique": true, + "sparse": false + } + ], + "isDisjoint": false, + "numberOfShards": 1, + "id": "10009", + "minReplicationFactor": 1, + "statusString": "loaded", + "isSmart": false, + "name": "_queues", + "keyOptions": { + "type": "traditional", + "allowUserKeys": true + } + }, + "10005": { + "writeConcern": 1, + "waitForSync": false, + "usesRevisionsAsDocumentIds": true, + "syncByRevision": true, + "shards": { + "s10016": [ + "PRMR-a24adubi", + "PRMR-n92yizyp" + ] + }, + "schema": null, + "type": 2, + "status": 3, + "replicationFactor": 2, + "cacheEnabled": false, + "isSystem": true, + "deleted": false, + "shardingStrategy": "hash", + "distributeShardsLike": "10001", + "internalValidatorType": 0, + "isSmartChild": false, + "shardKeys": [ + "_key" + ], + "indexes": [ + { + "id": "0", + "type": "primary", + "name": "primary", + "fields": [ + "_key" + ], + "unique": true, + "sparse": false + }, + { + "deduplicate": true, + "estimates": false, + "fields": [ + "time" + ], + "id": "10027", + "inBackground": false, + "name": "idx_1726829639976878081", + "sparse": false, + "type": "skiplist", + "unique": false + } + ], + "isDisjoint": false, + "numberOfShards": 1, + "id": "10005", + "minReplicationFactor": 1, + "statusString": "loaded", + "isSmart": false, + "name": "_statistics15", + "keyOptions": { + "type": "traditional", + "allowUserKeys": true + } + }, + "10012": { + "writeConcern": 1, + "waitForSync": false, + "usesRevisionsAsDocumentIds": true, + "syncByRevision": true, + "shards": { + "s10023": [ + "PRMR-a24adubi", + "PRMR-n92yizyp" + ] + }, + "schema": null, + "type": 2, + "status": 3, + "replicationFactor": 2, + "cacheEnabled": false, + "isSystem": true, + "deleted": false, + "shardingStrategy": "hash", + "distributeShardsLike": "10001", + "internalValidatorType": 0, + "isSmartChild": false, + "shardKeys": [ + "_key" + ], + "indexes": [ + { + "id": "0", + "type": "primary", + "name": "primary", + "fields": [ + "_key" + ], + "unique": true, + "sparse": false + } + ], + "isDisjoint": false, + "numberOfShards": 1, + "id": "10012", + "minReplicationFactor": 1, + "statusString": "loaded", + "isSmart": false, + "name": "_appbundles", + "keyOptions": { + "type": "traditional", + "allowUserKeys": true + } + }, + "10010": { + "writeConcern": 1, + "waitForSync": false, + "usesRevisionsAsDocumentIds": true, + "syncByRevision": true, + "shards": { + "s10021": [ + "PRMR-a24adubi", + "PRMR-n92yizyp" + ] + }, + "schema": null, + "type": 2, + "status": 3, + "replicationFactor": 2, + "cacheEnabled": false, + "isSystem": true, + "deleted": false, + "shardingStrategy": "hash", + "distributeShardsLike": "10001", + "internalValidatorType": 0, + "isSmartChild": false, + "shardKeys": [ + "_key" + ], + "indexes": [ + { + "id": "0", + "type": "primary", + "name": "primary", + "fields": [ + "_key" + ], + "unique": true, + "sparse": false + }, + { + "deduplicate": true, + "estimates": false, + "fields": [ + "queue", + "status", + "delayUntil" + ], + "id": "10030", + "inBackground": false, + "name": "idx_1726829640074395648", + "sparse": false, + "type": "skiplist", + "unique": false + }, + { + "deduplicate": true, + "estimates": false, + "fields": [ + "status", + "queue", + "delayUntil" + ], + "id": "10031", + "inBackground": false, + "name": "idx_1726829640102707201", + "sparse": false, + "type": "skiplist", + "unique": false + } + ], + "isDisjoint": false, + "numberOfShards": 1, + "id": "10010", + "minReplicationFactor": 1, + "statusString": "loaded", + "isSmart": false, + "name": "_jobs", + "keyOptions": { + "type": "traditional", + "allowUserKeys": true + } + }, + "10004": { + "writeConcern": 1, + "waitForSync": false, + "usesRevisionsAsDocumentIds": true, + "syncByRevision": true, + "shards": { + "s10015": [ + "PRMR-a24adubi", + "PRMR-n92yizyp" + ] + }, + "schema": null, + "type": 2, + "status": 3, + "replicationFactor": 2, + "cacheEnabled": false, + "isSystem": true, + "deleted": false, + "shardingStrategy": "hash", + "distributeShardsLike": "10001", + "internalValidatorType": 0, + "isSmartChild": false, + "shardKeys": [ + "_key" + ], + "indexes": [ + { + "id": "0", + "type": "primary", + "name": "primary", + "fields": [ + "_key" + ], + "unique": true, + "sparse": false + }, + { + "deduplicate": true, + "estimates": false, + "fields": [ + "time" + ], + "id": "10026", + "inBackground": false, + "name": "idx_1726829639950663681", + "sparse": false, + "type": "skiplist", + "unique": false + } + ], + "isDisjoint": false, + "numberOfShards": 1, + "id": "10004", + "minReplicationFactor": 1, + "statusString": "loaded", + "isSmart": false, + "name": "_statistics", + "keyOptions": { + "type": "traditional", + "allowUserKeys": true + } + }, + "10006": { + "writeConcern": 1, + "waitForSync": false, + "usesRevisionsAsDocumentIds": true, + "syncByRevision": true, + "shards": { + "s10017": [ + "PRMR-a24adubi", + "PRMR-n92yizyp" + ] + }, + "schema": null, + "type": 2, + "status": 3, + "replicationFactor": 2, + "cacheEnabled": false, + "isSystem": true, + "deleted": false, + "shardingStrategy": "hash", + "distributeShardsLike": "10001", + "internalValidatorType": 0, + "isSmartChild": false, + "shardKeys": [ + "_key" + ], + "indexes": [ + { + "id": "0", + "type": "primary", + "name": "primary", + "fields": [ + "_key" + ], + "unique": true, + "sparse": false + }, + { + "deduplicate": true, + "estimates": false, + "fields": [ + "time" + ], + "id": "10028", + "inBackground": false, + "name": "idx_1726829640006238208", + "sparse": false, + "type": "skiplist", + "unique": false + } + ], + "isDisjoint": false, + "numberOfShards": 1, + "id": "10006", + "minReplicationFactor": 1, + "statusString": "loaded", + "isSmart": false, + "name": "_statisticsRaw", + "keyOptions": { + "type": "traditional", + "allowUserKeys": true + } + }, + "10003": { + "writeConcern": 1, + "waitForSync": false, + "usesRevisionsAsDocumentIds": true, + "syncByRevision": true, + "shards": { + "s10014": [ + "PRMR-a24adubi", + "PRMR-n92yizyp" + ] + }, + "schema": null, + "type": 2, + "status": 3, + "replicationFactor": 2, + "cacheEnabled": false, + "isSystem": true, + "deleted": false, + "shardingStrategy": "hash", + "distributeShardsLike": "10001", + "internalValidatorType": 0, + "isSmartChild": false, + "shardKeys": [ + "_key" + ], + "indexes": [ + { + "id": "0", + "type": "primary", + "name": "primary", + "fields": [ + "_key" + ], + "unique": true, + "sparse": false + } + ], + "isDisjoint": false, + "numberOfShards": 1, + "id": "10003", + "minReplicationFactor": 1, + "statusString": "loaded", + "isSmart": false, + "name": "_graphs", + "keyOptions": { + "type": "traditional", + "allowUserKeys": true + } + }, + "10008": { + "writeConcern": 1, + "waitForSync": false, + "usesRevisionsAsDocumentIds": true, + "syncByRevision": true, + "shards": { + "s10019": [ + "PRMR-a24adubi", + "PRMR-n92yizyp" + ] + }, + "schema": null, + "type": 2, + "status": 3, + "replicationFactor": 2, + "cacheEnabled": false, + "isSystem": true, + "deleted": false, + "shardingStrategy": "hash", + "distributeShardsLike": "10001", + "internalValidatorType": 0, + "isSmartChild": false, + "shardKeys": [ + "_key" + ], + "indexes": [ + { + "id": "0", + "type": "primary", + "name": "primary", + "fields": [ + "_key" + ], + "unique": true, + "sparse": false + } + ], + "isDisjoint": false, + "numberOfShards": 1, + "id": "10008", + "minReplicationFactor": 1, + "statusString": "loaded", + "isSmart": false, + "name": "_aqlfunctions", + "keyOptions": { + "type": "traditional", + "allowUserKeys": true + } + }, + "10007": { + "writeConcern": 1, + "waitForSync": false, + "usesRevisionsAsDocumentIds": true, + "syncByRevision": true, + "shards": { + "s10018": [ + "PRMR-a24adubi", + "PRMR-n92yizyp" + ] + }, + "schema": null, + "type": 2, + "status": 3, + "replicationFactor": 2, + "cacheEnabled": false, + "isSystem": true, + "deleted": false, + "shardingStrategy": "hash", + "distributeShardsLike": "10001", + "internalValidatorType": 0, + "isSmartChild": false, + "shardKeys": [ + "_key" + ], + "indexes": [ + { + "id": "0", + "type": "primary", + "name": "primary", + "fields": [ + "_key" + ], + "unique": true, + "sparse": false + } + ], + "isDisjoint": false, + "numberOfShards": 1, + "id": "10007", + "minReplicationFactor": 1, + "statusString": "loaded", + "isSmart": false, + "name": "_analyzers", + "keyOptions": { + "type": "traditional", + "allowUserKeys": true + } + }, + "10001": { + "writeConcern": 1, + "waitForSync": false, + "usesRevisionsAsDocumentIds": true, + "syncByRevision": true, + "shards": { + "s10002": [ + "PRMR-a24adubi", + "PRMR-n92yizyp" + ] + }, + "shardingStrategy": "hash", + "schema": null, + "type": 2, + "status": 3, + "replicationFactor": 2, + "cacheEnabled": false, + "isSystem": true, + "deleted": false, + "isSmartChild": false, + "internalValidatorType": 0, + "shardKeys": [ + "_key" + ], + "indexes": [ + { + "id": "0", + "type": "primary", + "name": "primary", + "fields": [ + "_key" + ], + "unique": true, + "sparse": false + }, + { + "deduplicate": true, + "estimates": false, + "fields": [ + "user" + ], + "id": "10025", + "inBackground": false, + "name": "idx_1726829639915012096", + "sparse": true, + "type": "hash", + "unique": true + } + ], + "isDisjoint": false, + "id": "10001", + "numberOfShards": 1, + "minReplicationFactor": 1, + "statusString": "loaded", + "isSmart": false, + "name": "_users", + "keyOptions": { + "type": "traditional", + "allowUserKeys": true + } + } + } + }, + "Analyzers": { + "_system": { + "revision": 0, + "buildingRevision": 0 + } + } + }, + "Current": { + "FoxxmasterQueueupdate": false, + "AsyncReplication": {}, + "Collections": { + "_system": { + "10011": { + "s10022": { + "servers": [ + "PRMR-a24adubi", + "PRMR-n92yizyp" + ], + "indexes": [ + { + "fields": [ + "_key" + ], + "id": "0", + "name": "primary", + "objectId": "4010042", + "sparse": false, + "type": "primary", + "unique": true + }, + { + "deduplicate": true, + "estimates": true, + "fields": [ + "mount" + ], + "id": "10029", + "name": "idx_1726829640049229824", + "objectId": "4010164", + "sparse": true, + "type": "hash", + "unique": true + } + ], + "failoverCandidates": [ + "PRMR-a24adubi", + "PRMR-n92yizyp" + ], + "errorNum": 0, + "errorMessage": "", + "error": false + } + }, + "10013": { + "s10024": { + "servers": [ + "PRMR-a24adubi", + "PRMR-n92yizyp" + ], + "indexes": [ + { + "fields": [ + "_key" + ], + "id": "0", + "name": "primary", + "objectId": "4010040", + "sparse": false, + "type": "primary", + "unique": true + } + ], + "failoverCandidates": [ + "PRMR-a24adubi", + "PRMR-n92yizyp" + ], + "errorNum": 0, + "errorMessage": "", + "error": false + } + }, + "10009": { + "s10020": { + "servers": [ + "PRMR-a24adubi", + "PRMR-n92yizyp" + ], + "indexes": [ + { + "fields": [ + "_key" + ], + "id": "0", + "name": "primary", + "objectId": "4010038", + "sparse": false, + "type": "primary", + "unique": true + } + ], + "failoverCandidates": [ + "PRMR-a24adubi", + "PRMR-n92yizyp" + ], + "errorNum": 0, + "errorMessage": "", + "error": false + } + }, + "10005": { + "s10016": { + "servers": [ + "PRMR-a24adubi", + "PRMR-n92yizyp" + ], + "indexes": [ + { + "fields": [ + "_key" + ], + "id": "0", + "name": "primary", + "objectId": "4010036", + "sparse": false, + "type": "primary", + "unique": true + }, + { + "deduplicate": true, + "estimates": false, + "fields": [ + "time" + ], + "id": "10027", + "name": "idx_1726829639976878081", + "objectId": "4010144", + "sparse": false, + "type": "skiplist", + "unique": false + } + ], + "failoverCandidates": [ + "PRMR-a24adubi", + "PRMR-n92yizyp" + ], + "errorNum": 0, + "errorMessage": "", + "error": false + } + }, + "10012": { + "s10023": { + "servers": [ + "PRMR-a24adubi", + "PRMR-n92yizyp" + ], + "indexes": [ + { + "fields": [ + "_key" + ], + "id": "0", + "name": "primary", + "objectId": "4010034", + "sparse": false, + "type": "primary", + "unique": true + } + ], + "failoverCandidates": [ + "PRMR-a24adubi", + "PRMR-n92yizyp" + ], + "errorNum": 0, + "errorMessage": "", + "error": false + } + }, + "10010": { + "s10021": { + "servers": [ + "PRMR-a24adubi", + "PRMR-n92yizyp" + ], + "indexes": [ + { + "fields": [ + "_key" + ], + "id": "0", + "name": "primary", + "objectId": "4010032", + "sparse": false, + "type": "primary", + "unique": true + }, + { + "deduplicate": true, + "estimates": false, + "fields": [ + "queue", + "status", + "delayUntil" + ], + "id": "10030", + "name": "idx_1726829640074395648", + "objectId": "4010174", + "sparse": false, + "type": "skiplist", + "unique": false + }, + { + "deduplicate": true, + "estimates": false, + "fields": [ + "status", + "queue", + "delayUntil" + ], + "id": "10031", + "name": "idx_1726829640102707201", + "objectId": "4010186", + "sparse": false, + "type": "skiplist", + "unique": false + } + ], + "failoverCandidates": [ + "PRMR-a24adubi", + "PRMR-n92yizyp" + ], + "errorNum": 0, + "errorMessage": "", + "error": false + } + }, + "10004": { + "s10015": { + "servers": [ + "PRMR-a24adubi", + "PRMR-n92yizyp" + ], + "indexes": [ + { + "fields": [ + "_key" + ], + "id": "0", + "name": "primary", + "objectId": "4010030", + "sparse": false, + "type": "primary", + "unique": true + }, + { + "deduplicate": true, + "estimates": false, + "fields": [ + "time" + ], + "id": "10026", + "name": "idx_1726829639950663681", + "objectId": "4010134", + "sparse": false, + "type": "skiplist", + "unique": false + } + ], + "failoverCandidates": [ + "PRMR-a24adubi", + "PRMR-n92yizyp" + ], + "errorNum": 0, + "errorMessage": "", + "error": false + } + }, + "10006": { + "s10017": { + "servers": [ + "PRMR-a24adubi", + "PRMR-n92yizyp" + ], + "indexes": [ + { + "fields": [ + "_key" + ], + "id": "0", + "name": "primary", + "objectId": "4010028", + "sparse": false, + "type": "primary", + "unique": true + }, + { + "deduplicate": true, + "estimates": false, + "fields": [ + "time" + ], + "id": "10028", + "name": "idx_1726829640006238208", + "objectId": "4010154", + "sparse": false, + "type": "skiplist", + "unique": false + } + ], + "failoverCandidates": [ + "PRMR-a24adubi", + "PRMR-n92yizyp" + ], + "errorNum": 0, + "errorMessage": "", + "error": false + } + }, + "10003": { + "s10014": { + "servers": [ + "PRMR-a24adubi", + "PRMR-n92yizyp" + ], + "indexes": [ + { + "fields": [ + "_key" + ], + "id": "0", + "name": "primary", + "objectId": "4010026", + "sparse": false, + "type": "primary", + "unique": true + } + ], + "failoverCandidates": [ + "PRMR-a24adubi", + "PRMR-n92yizyp" + ], + "errorNum": 0, + "errorMessage": "", + "error": false + } + }, + "10008": { + "s10019": { + "servers": [ + "PRMR-a24adubi", + "PRMR-n92yizyp" + ], + "indexes": [ + { + "fields": [ + "_key" + ], + "id": "0", + "name": "primary", + "objectId": "4010024", + "sparse": false, + "type": "primary", + "unique": true + } + ], + "failoverCandidates": [ + "PRMR-a24adubi", + "PRMR-n92yizyp" + ], + "errorNum": 0, + "errorMessage": "", + "error": false + } + }, + "10007": { + "s10018": { + "servers": [ + "PRMR-a24adubi", + "PRMR-n92yizyp" + ], + "indexes": [ + { + "fields": [ + "_key" + ], + "id": "0", + "name": "primary", + "objectId": "4010023", + "sparse": false, + "type": "primary", + "unique": true + } + ], + "failoverCandidates": [ + "PRMR-a24adubi", + "PRMR-n92yizyp" + ], + "errorNum": 0, + "errorMessage": "", + "error": false + } + }, + "10001": { + "s10002": { + "servers": [ + "PRMR-a24adubi", + "PRMR-n92yizyp" + ], + "indexes": [ + { + "fields": [ + "_key" + ], + "id": "0", + "name": "primary", + "objectId": "4010002", + "sparse": false, + "type": "primary", + "unique": true + }, + { + "deduplicate": true, + "estimates": true, + "fields": [ + "user" + ], + "id": "10025", + "name": "idx_1726829639915012096", + "objectId": "4010124", + "sparse": true, + "type": "hash", + "unique": true + } + ], + "failoverCandidates": [ + "PRMR-a24adubi", + "PRMR-n92yizyp" + ], + "errorNum": 0, + "errorMessage": "", + "error": false + } + } + } + }, + "DBServers": { + "PRMR-rg51yv2h": "none", + "PRMR-n92yizyp": "none", + "PRMR-a24adubi": "none" + }, + "Singles": {}, + "Databases": { + "_system": { + "PRMR-rg51yv2h": { + "id": "1", + "errorNum": 0, + "name": "_system", + "errorMessage": "", + "error": false + }, + "PRMR-a24adubi": { + "id": "1", + "errorNum": 0, + "name": "_system", + "errorMessage": "", + "error": false + }, + "PRMR-n92yizyp": { + "id": "1", + "errorNum": 0, + "name": "_system", + "errorMessage": "", + "error": false + } + } + }, + "Lock": "UNLOCKED", + "ServersKnown": { + "PRMR-n92yizyp": { + "rebootId": 1 + }, + "CRDN-cg5kjymo": { + "rebootId": 1 + }, + "PRMR-a24adubi": { + "rebootId": 1 + }, + "CRDN-ssc2yc0x": { + "rebootId": 1 + }, + "PRMR-rg51yv2h": { + "rebootId": 1 + }, + "CRDN-g6s8btds": { + "rebootId": 1 + } + }, + "Coordinators": { + "CRDN-cg5kjymo": "none", + "CRDN-ssc2yc0x": "none", + "CRDN-g6s8btds": "none" + }, + "NewServers": {}, + "ServersRegistered": { + "PRMR-n92yizyp": { + "timestamp": "2022-03-09T13:37:53Z", + "host": "ip-10-0-101-138.eu-central-1.compute.internal", + "versionString": "3.9.0", + "extendedNamesDatabases": false, + "version": 30900, + "engine": "rocksdb", + "endpoint": "ssl://cluster-uexabr-dbserver-n92yizyp.cluster-uexabr-int.3dcd4eac-c549-41b6-bc8f-40e11983ebd2.svc:8529", + "advertisedEndpoint": "" + }, + "CRDN-cg5kjymo": { + "timestamp": "2022-03-09T13:37:44Z", + "host": "ip-10-0-101-110.eu-central-1.compute.internal", + "versionString": "3.9.0", + "extendedNamesDatabases": false, + "version": 30900, + "engine": "rocksdb", + "endpoint": "ssl://cluster-uexabr-coordinator-cg5kjymo.cluster-uexabr-int.3dcd4eac-c549-41b6-bc8f-40e11983ebd2.svc:8529", + "advertisedEndpoint": "" + }, + "PRMR-a24adubi": { + "timestamp": "2022-03-09T13:37:43Z", + "host": "ip-10-0-101-183.eu-central-1.compute.internal", + "versionString": "3.9.0", + "extendedNamesDatabases": false, + "version": 30900, + "engine": "rocksdb", + "endpoint": "ssl://cluster-uexabr-dbserver-a24adubi.cluster-uexabr-int.3dcd4eac-c549-41b6-bc8f-40e11983ebd2.svc:8529", + "advertisedEndpoint": "" + }, + "PRMR-rg51yv2h": { + "timestamp": "2022-03-09T13:38:03Z", + "host": "ip-10-0-101-233.eu-central-1.compute.internal", + "versionString": "3.9.0", + "extendedNamesDatabases": false, + "version": 30900, + "engine": "rocksdb", + "endpoint": "ssl://cluster-uexabr-dbserver-rg51yv2h.cluster-uexabr-int.3dcd4eac-c549-41b6-bc8f-40e11983ebd2.svc:8529", + "advertisedEndpoint": "" + }, + "CRDN-g6s8btds": { + "timestamp": "2022-03-09T13:37:41Z", + "host": "ip-10-0-100-174.eu-central-1.compute.internal", + "versionString": "3.9.0", + "extendedNamesDatabases": false, + "version": 30900, + "engine": "rocksdb", + "endpoint": "ssl://cluster-uexabr-coordinator-g6s8btds.cluster-uexabr-int.3dcd4eac-c549-41b6-bc8f-40e11983ebd2.svc:8529", + "advertisedEndpoint": "" + }, + "CRDN-ssc2yc0x": { + "timestamp": "2022-03-09T13:37:43Z", + "host": "ip-10-0-101-183.eu-central-1.compute.internal", + "versionString": "3.9.0", + "extendedNamesDatabases": false, + "version": 30900, + "engine": "rocksdb", + "endpoint": "ssl://cluster-uexabr-coordinator-ssc2yc0x.cluster-uexabr-int.3dcd4eac-c549-41b6-bc8f-40e11983ebd2.svc:8529", + "advertisedEndpoint": "" + }, + "Version": 1 + }, + "ShardsCopied": {}, + "Version": 75, + "Foxxmaster": "CRDN-g6s8btds" + }, + "Cluster": "8c15fcd5-9711-49e3-bd8f-1c0a3147f0b6", + "Agency": { + "Definition": 1 + } + }, + ".agency": { + "timeoutMult": 1, + "term": 1, + "size": 3, + "pool": { + "AGNT-q2cs6q1j": "ssl://cluster-uexabr-agent-q2cs6q1j.cluster-uexabr-int.3dcd4eac-c549-41b6-bc8f-40e11983ebd2.svc:8529", + "AGNT-ec6b7qdz": "ssl://cluster-uexabr-agent-ec6b7qdz.cluster-uexabr-int.3dcd4eac-c549-41b6-bc8f-40e11983ebd2.svc:8529", + "AGNT-8sek6mbr": "ssl://cluster-uexabr-agent-8sek6mbr.cluster-uexabr-int.3dcd4eac-c549-41b6-bc8f-40e11983ebd2.svc:8529" + }, + "id": "AGNT-ec6b7qdz", + "active": [ + "AGNT-q2cs6q1j", + "AGNT-8sek6mbr", + "AGNT-ec6b7qdz" + ] + } + }, + "log": [ + { + "_key": "00000000000000000000", + "_rev": "_d05kI-q---", + "clientId": "", + "epoch_millis": 1646833051656, + "request": {}, + "term": 0, + "timestamp": "2022-03-09 13:37:31 UTC" + }, + { + "_key": "00000000000000000001", + "_rev": "_d05kRDm---", + "clientId": "", + "epoch_millis": 1646833060954, + "request": { + ".agency": { + "op": "set", + "new": { + "term": 1, + "id": "AGNT-ec6b7qdz", + "active": [ + "AGNT-q2cs6q1j", + "AGNT-8sek6mbr", + "AGNT-ec6b7qdz" + ], + "pool": { + "AGNT-q2cs6q1j": "ssl://cluster-uexabr-agent-q2cs6q1j.cluster-uexabr-int.3dcd4eac-c549-41b6-bc8f-40e11983ebd2.svc:8529", + "AGNT-8sek6mbr": "ssl://cluster-uexabr-agent-8sek6mbr.cluster-uexabr-int.3dcd4eac-c549-41b6-bc8f-40e11983ebd2.svc:8529", + "AGNT-ec6b7qdz": "ssl://cluster-uexabr-agent-ec6b7qdz.cluster-uexabr-int.3dcd4eac-c549-41b6-bc8f-40e11983ebd2.svc:8529" + }, + "size": 3, + "timeoutMult": 1 + } + } + }, + "term": 1, + "timestamp": "2022-03-09 13:37:40 UTC" + }, + { + "_key": "00000000000000000002", + "_rev": "_d05kREG--A", + "clientId": "acb77479-a799-43a6-9d40-b0657e495b2a", + "epoch_millis": 1646833060962, + "request": { + "arango/": { + "op": "set", + "new": { + "Cluster": "8c15fcd5-9711-49e3-bd8f-1c0a3147f0b6", + "Agency": { + "Definition": 1 + }, + "Current": { + "AsyncReplication": {}, + "Collections": { + "_system": {} + }, + "Version": 1, + "ShardsCopied": {}, + "NewServers": {}, + "Coordinators": {}, + "Lock": "UNLOCKED", + "DBServers": {}, + "Singles": {}, + "ServersRegistered": { + "Version": 1 + }, + "Databases": {} + }, + "InitDone": true, + "Plan": { + "AsyncReplication": {}, + "Coordinators": {}, + "Databases": { + "_system": { + "name": "_system", + "id": "1" + } + }, + "Lock": "UNLOCKED", + "DBServers": {}, + "Singles": {}, + "Version": 1, + "Collections": { + "_system": {} + }, + "Views": { + "_system": {} + }, + "Analyzers": { + "_system": { + "revision": 0, + "buildingRevision": 0 + } + } + }, + "Sync": { + "LatestID": 1, + "Problems": {}, + "UserVersion": 1, + "ServerStates": {}, + "HeartbeatIntervalMs": 1000 + }, + "Supervision": { + "Health": {}, + "Shards": {}, + "DBServers": {} + }, + "Target": { + "NumberOfCoordinators": null, + "NumberOfDBServers": null, + "CleanedServers": [], + "ToBeCleanedServers": [], + "FailedServers": {}, + "Lock": "UNLOCKED", + "Failed": {}, + "Finished": {}, + "Pending": {}, + "ToDo": {}, + "Version": 1 + } + } + } + }, + "term": 1, + "timestamp": "2022-03-09 13:37:40 UTC" + }, + { + "_key": "00000000000000000003", + "_rev": "_d05kREy--B", + "clientId": "CRDN-g6s8btds:039b23c6-78cb-46d2-ac94-d9acca05e5e0", + "epoch_millis": 1646833060973, + "request": { + "arango/Plan/Coordinators/CRDN-g6s8btds": { + "op": "set", + "new": "none" + }, + "arango/Plan/Version": { + "op": "increment" + } + }, + "term": 1, + "timestamp": "2022-03-09 13:37:40 UTC" + }, + { + "_key": "00000000000000000004", + "_rev": "_d05kRFO--A", + "clientId": "CRDN-g6s8btds:a3b01a5a-b3f3-4e78-b0c6-1be605930dbf", + "epoch_millis": 1646833060980, + "request": { + "arango/Current/Coordinators/CRDN-g6s8btds": { + "op": "set", + "new": "none" + }, + "arango/Current/Version": { + "op": "increment" + } + }, + "term": 1, + "timestamp": "2022-03-09 13:37:40 UTC" + }, + { + "_key": "00000000000000000005", + "_rev": "_d05kRKS---", + "clientId": "CRDN-g6s8btds:bb6fac3d-d756-46ce-8349-4b72ed278ad5", + "epoch_millis": 1646833061061, + "request": { + "arango/Target/LatestCoordinatorId": { + "op": "increment" + }, + "arango/Target/MapUniqueToShortID/CRDN-g6s8btds": { + "op": "set", + "new": { + "TransactionID": 1, + "ShortName": "Coordinator0001" + } + } + }, + "term": 1, + "timestamp": "2022-03-09 13:37:41 UTC" + }, + { + "_key": "00000000000000000006", + "_rev": "_d05kRKu--A", + "clientId": "CRDN-g6s8btds:1a984fa6-ce84-42de-96c5-afc9f418832f", + "epoch_millis": 1646833061068, + "request": { + "arango/Current/ServersRegistered/CRDN-g6s8btds": { + "op": "set", + "new": { + "endpoint": "ssl://cluster-uexabr-coordinator-g6s8btds.cluster-uexabr-int.3dcd4eac-c549-41b6-bc8f-40e11983ebd2.svc:8529", + "advertisedEndpoint": "", + "host": "ip-10-0-100-174.eu-central-1.compute.internal", + "version": 30900, + "versionString": "3.9.0", + "engine": "rocksdb", + "extendedNamesDatabases": false, + "timestamp": "2022-03-09T13:37:41Z" + } + }, + "arango/Current/ServersKnown/CRDN-g6s8btds/rebootId": { + "op": "increment" + }, + "arango/Current/Version": { + "op": "increment" + } + }, + "term": 1, + "timestamp": "2022-03-09 13:37:41 UTC" + }, + { + "_key": "00000000000000000007", + "_rev": "_d05kRw----", + "clientId": "", + "epoch_millis": 1646833061664, + "request": { + "/arango/Sync/LatestID": { + "op": "increment", + "step": 10000 + } + }, + "term": 1, + "timestamp": "2022-03-09 13:37:41 UTC" + }, + { + "_key": "00000000000000000008", + "_rev": "_d05kRwK--_", + "clientId": "", + "epoch_millis": 1646833061667, + "request": { + "/arango/Supervision/State": { + "Mode": "Normal", + "Timestamp": "2022-03-09T13:37:41Z" + } + }, + "term": 1, + "timestamp": "2022-03-09 13:37:41 UTC" + }, + { + "_key": "00000000000000000009", + "_rev": "_d05kTBu--A", + "clientId": "CRDN-ssc2yc0x:562d04c3-d494-42b2-a396-1eb0aa9edd81", + "epoch_millis": 1646833062972, + "request": { + "arango/Plan/Coordinators/CRDN-ssc2yc0x": { + "op": "set", + "new": "none" + }, + "arango/Plan/Version": { + "op": "increment" + } + }, + "term": 1, + "timestamp": "2022-03-09 13:37:42 UTC" + }, + { + "_key": "00000000000000000010", + "_rev": "_d05kTB6--_", + "clientId": "CRDN-cg5kjymo:c7c6b48c-5de6-48f8-96ba-6c3305ef7f28", + "epoch_millis": 1646833062975, + "request": { + "arango/Plan/Coordinators/CRDN-cg5kjymo": { + "op": "set", + "new": "none" + }, + "arango/Plan/Version": { + "op": "increment" + } + }, + "term": 1, + "timestamp": "2022-03-09 13:37:42 UTC" + }, + { + "_key": "00000000000000000011", + "_rev": "_d05kTCW---", + "clientId": "CRDN-ssc2yc0x:0dcc9e60-51fb-4ed4-b779-f7419ccaeb8b", + "epoch_millis": 1646833062981, + "request": { + "arango/Current/Coordinators/CRDN-ssc2yc0x": { + "op": "set", + "new": "none" + }, + "arango/Current/Version": { + "op": "increment" + } + }, + "term": 1, + "timestamp": "2022-03-09 13:37:42 UTC" + }, + { + "_key": "00000000000000000012", + "_rev": "_d05kTCm--A", + "clientId": "CRDN-cg5kjymo:8f252e18-f381-4419-98dd-774bd97841d9", + "epoch_millis": 1646833062986, + "request": { + "arango/Current/Coordinators/CRDN-cg5kjymo": { + "op": "set", + "new": "none" + }, + "arango/Current/Version": { + "op": "increment" + } + }, + "term": 1, + "timestamp": "2022-03-09 13:37:42 UTC" + }, + { + "_key": "00000000000000000013", + "_rev": "_d05kTDG--A", + "clientId": "CRDN-ssc2yc0x:4779d131-fe12-4c63-8591-6bd185fd7b98", + "epoch_millis": 1646833062994, + "request": { + "arango/Target/LatestCoordinatorId": { + "op": "increment" + }, + "arango/Target/MapUniqueToShortID/CRDN-ssc2yc0x": { + "op": "set", + "new": { + "TransactionID": 2, + "ShortName": "Coordinator0002" + } + } + }, + "term": 1, + "timestamp": "2022-03-09 13:37:42 UTC" + }, + { + "_key": "00000000000000000014", + "_rev": "_d05kTE---_", + "clientId": "CRDN-ssc2yc0x:2b57c925-4eed-45e3-9d95-bdfaa4fe71bc", + "epoch_millis": 1646833063008, + "request": { + "arango/Current/ServersRegistered/CRDN-ssc2yc0x": { + "op": "set", + "new": { + "endpoint": "ssl://cluster-uexabr-coordinator-ssc2yc0x.cluster-uexabr-int.3dcd4eac-c549-41b6-bc8f-40e11983ebd2.svc:8529", + "advertisedEndpoint": "", + "host": "ip-10-0-101-183.eu-central-1.compute.internal", + "version": 30900, + "versionString": "3.9.0", + "engine": "rocksdb", + "extendedNamesDatabases": false, + "timestamp": "2022-03-09T13:37:43Z" + } + }, + "arango/Current/ServersKnown/CRDN-ssc2yc0x/rebootId": { + "op": "increment" + }, + "arango/Current/Version": { + "op": "increment" + } + }, + "term": 1, + "timestamp": "2022-03-09 13:37:43 UTC" + }, + { + "_key": "00000000000000000015", + "_rev": "_d05kTji--A", + "clientId": "PRMR-a24adubi:9389e5ac-c4a7-4b17-8494-6dc58e9aaeff", + "epoch_millis": 1646833063513, + "request": { + "arango/Plan/DBServers/PRMR-a24adubi": { + "op": "set", + "new": "none" + }, + "arango/Plan/Version": { + "op": "increment" + } + }, + "term": 1, + "timestamp": "2022-03-09 13:37:43 UTC" + }, + { + "_key": "00000000000000000016", + "_rev": "_d05kTkG--A", + "clientId": "PRMR-a24adubi:f39d1b45-d8a8-4f27-b3c5-24be9c8f291a", + "epoch_millis": 1646833063522, + "request": { + "arango/Current/DBServers/PRMR-a24adubi": { + "op": "set", + "new": "none" + }, + "arango/Current/Version": { + "op": "increment" + } + }, + "term": 1, + "timestamp": "2022-03-09 13:37:43 UTC" + }, + { + "_key": "00000000000000000017", + "_rev": "_d05kTku--_", + "clientId": "PRMR-a24adubi:771cbee3-c143-46c6-9920-badd031a3b17", + "epoch_millis": 1646833063532, + "request": { + "arango/Target/LatestDBServerId": { + "op": "increment" + }, + "arango/Target/MapUniqueToShortID/PRMR-a24adubi": { + "op": "set", + "new": { + "TransactionID": 1, + "ShortName": "DBServer0001" + } + } + }, + "term": 1, + "timestamp": "2022-03-09 13:37:43 UTC" + }, + { + "_key": "00000000000000000018", + "_rev": "_d05kTlO--_", + "clientId": "PRMR-a24adubi:5bcd2f02-3e65-4dfa-8867-db01146db348", + "epoch_millis": 1646833063540, + "request": { + "arango/Current/ServersRegistered/PRMR-a24adubi": { + "op": "set", + "new": { + "endpoint": "ssl://cluster-uexabr-dbserver-a24adubi.cluster-uexabr-int.3dcd4eac-c549-41b6-bc8f-40e11983ebd2.svc:8529", + "advertisedEndpoint": "", + "host": "ip-10-0-101-183.eu-central-1.compute.internal", + "version": 30900, + "versionString": "3.9.0", + "engine": "rocksdb", + "extendedNamesDatabases": false, + "timestamp": "2022-03-09T13:37:43Z" + } + }, + "arango/Current/ServersKnown/PRMR-a24adubi/rebootId": { + "op": "increment" + }, + "arango/Current/Version": { + "op": "increment" + } + }, + "term": 1, + "timestamp": "2022-03-09 13:37:43 UTC" + }, + { + "_key": "00000000000000000019", + "_rev": "_d05kT4e---", + "clientId": "PRMR-a24adubi:791724a4-d4e8-4258-863a-25be444160fd", + "epoch_millis": 1646833063848, + "request": { + "arango/Current/Version": { + "op": "increment" + } + }, + "term": 1, + "timestamp": "2022-03-09 13:37:43 UTC" + }, + { + "_key": "00000000000000000020", + "_rev": "_d05kT9W--A", + "clientId": "PRMR-a24adubi:99652721-37a3-4699-8fb1-0ca992a2896e", + "epoch_millis": 1646833063926, + "request": { + "arango/.license": { + "op": "set", + "new": { + "features": { + "expires": 1647005863 + }, + "license": "JD4EOXdvNVUPbicOETgOeDFULw9TKnUBBxcXPQc+Qgl8CSIFPHUrcAgKDVdeK1VW", + "version": 1 + } + } + }, + "term": 1, + "timestamp": "2022-03-09 13:37:43 UTC" + }, + { + "_key": "00000000000000000021", + "_rev": "_d05kUCy--_", + "clientId": "CRDN-cg5kjymo:a43281ca-c713-4c8a-b63e-68f7cff1bd04", + "epoch_millis": 1646833064013, + "request": { + "arango/Target/LatestCoordinatorId": { + "op": "increment" + }, + "arango/Target/MapUniqueToShortID/CRDN-cg5kjymo": { + "op": "set", + "new": { + "TransactionID": 3, + "ShortName": "Coordinator0003" + } + } + }, + "term": 1, + "timestamp": "2022-03-09 13:37:44 UTC" + }, + { + "_key": "00000000000000000022", + "_rev": "_d05kUDS--B", + "clientId": "CRDN-cg5kjymo:1857e7cd-7c0a-4088-8643-501d892a9f3f", + "epoch_millis": 1646833064021, + "request": { + "arango/Current/ServersRegistered/CRDN-cg5kjymo": { + "op": "set", + "new": { + "endpoint": "ssl://cluster-uexabr-coordinator-cg5kjymo.cluster-uexabr-int.3dcd4eac-c549-41b6-bc8f-40e11983ebd2.svc:8529", + "advertisedEndpoint": "", + "host": "ip-10-0-101-110.eu-central-1.compute.internal", + "version": 30900, + "versionString": "3.9.0", + "engine": "rocksdb", + "extendedNamesDatabases": false, + "timestamp": "2022-03-09T13:37:44Z" + } + }, + "arango/Current/ServersKnown/CRDN-cg5kjymo/rebootId": { + "op": "increment" + }, + "arango/Current/Version": { + "op": "increment" + } + }, + "term": 1, + "timestamp": "2022-03-09 13:37:44 UTC" + }, + { + "_key": "00000000000000000023", + "_rev": "_d05kdEC--_", + "clientId": "PRMR-n92yizyp:37cc16cd-e2c7-42cb-8a39-e2ecd5bed813", + "epoch_millis": 1646833073249, + "request": { + "arango/Plan/DBServers/PRMR-n92yizyp": { + "op": "set", + "new": "none" + }, + "arango/Plan/Version": { + "op": "increment" + } + }, + "term": 1, + "timestamp": "2022-03-09 13:37:53 UTC" + }, + { + "_key": "00000000000000000024", + "_rev": "_d05kdEe--A", + "clientId": "PRMR-n92yizyp:779dcb76-89b2-4e63-b193-5c089c5a078e", + "epoch_millis": 1646833073256, + "request": { + "arango/Current/DBServers/PRMR-n92yizyp": { + "op": "set", + "new": "none" + }, + "arango/Current/Version": { + "op": "increment" + } + }, + "term": 1, + "timestamp": "2022-03-09 13:37:53 UTC" + }, + { + "_key": "00000000000000000025", + "_rev": "_d05kdFG--A", + "clientId": "PRMR-n92yizyp:6f715cc0-4b0d-428e-85fb-1a35e6b303aa", + "epoch_millis": 1646833073266, + "request": { + "arango/Target/LatestDBServerId": { + "op": "increment" + }, + "arango/Target/MapUniqueToShortID/PRMR-n92yizyp": { + "op": "set", + "new": { + "TransactionID": 2, + "ShortName": "DBServer0002" + } + } + }, + "term": 1, + "timestamp": "2022-03-09 13:37:53 UTC" + }, + { + "_key": "00000000000000000026", + "_rev": "_d05kdFm--D", + "clientId": "PRMR-n92yizyp:c3d9e218-0d45-4bc3-9144-9f8636b4d57c", + "epoch_millis": 1646833073274, + "request": { + "arango/Current/ServersRegistered/PRMR-n92yizyp": { + "op": "set", + "new": { + "endpoint": "ssl://cluster-uexabr-dbserver-n92yizyp.cluster-uexabr-int.3dcd4eac-c549-41b6-bc8f-40e11983ebd2.svc:8529", + "advertisedEndpoint": "", + "host": "ip-10-0-101-138.eu-central-1.compute.internal", + "version": 30900, + "versionString": "3.9.0", + "engine": "rocksdb", + "extendedNamesDatabases": false, + "timestamp": "2022-03-09T13:37:53Z" + } + }, + "arango/Current/ServersKnown/PRMR-n92yizyp/rebootId": { + "op": "increment" + }, + "arango/Current/Version": { + "op": "increment" + } + }, + "term": 1, + "timestamp": "2022-03-09 13:37:53 UTC" + }, + { + "_key": "00000000000000000027", + "_rev": "_d05kdGG--_", + "clientId": "CRDN-g6s8btds:889dfac1-a3e5-4860-b9a5-70c6bf5aaac3", + "epoch_millis": 1646833073282, + "request": { + "arango/Current/Version": { + "op": "increment" + } + }, + "term": 1, + "timestamp": "2022-03-09 13:37:53 UTC" + }, + { + "_key": "00000000000000000028", + "_rev": "_d05kdHC--_", + "clientId": "CRDN-g6s8btds:d480f288-a32c-423b-aefc-7f91ee6a6727", + "epoch_millis": 1646833073297, + "request": { + "arango/Current/Version": { + "op": "increment" + } + }, + "term": 1, + "timestamp": "2022-03-09 13:37:53 UTC" + }, + { + "_key": "00000000000000000029", + "_rev": "_d05kdUe--_", + "clientId": "CRDN-cg5kjymo:2052e8d0-159c-4e51-8d83-d82f71092494", + "epoch_millis": 1646833073512, + "request": { + "arango/Current/Version": { + "op": "increment" + } + }, + "term": 1, + "timestamp": "2022-03-09 13:37:53 UTC" + }, + { + "_key": "00000000000000000030", + "_rev": "_d05kdU2--_", + "clientId": "CRDN-cg5kjymo:916d1402-f5bc-4879-8223-1b3a506ad7f7", + "epoch_millis": 1646833073518, + "request": { + "arango/Current/Version": { + "op": "increment" + } + }, + "term": 1, + "timestamp": "2022-03-09 13:37:53 UTC" + }, + { + "_key": "00000000000000000031", + "_rev": "_d05kdgu--A", + "clientId": "PRMR-n92yizyp:3ce57651-efba-41c2-9729-2a2467cf45cf", + "epoch_millis": 1646833073708, + "request": { + "arango/Current/Version": { + "op": "increment" + } + }, + "term": 1, + "timestamp": "2022-03-09 13:37:53 UTC" + }, + { + "_key": "00000000000000000032", + "_rev": "_d05kdhO--B", + "clientId": "CRDN-g6s8btds:b90ab076-e1a9-40a5-9ae3-6cab22075716", + "epoch_millis": 1646833073716, + "request": { + "arango/Bootstrap": { + "op": "set", + "new": "CRDN-g6s8btds", + "ttl": 300 + } + }, + "term": 1, + "timestamp": "2022-03-09 13:37:53 UTC" + }, + { + "_key": "00000000000000000033", + "_rev": "_d05kdhu--A", + "clientId": "CRDN-g6s8btds:2bb2dd70-615b-4c44-919f-6df366036f87", + "epoch_millis": 1646833073724, + "request": { + "arango/Sync/LatestID": { + "op": "set", + "new": 2010001 + } + }, + "term": 1, + "timestamp": "2022-03-09 13:37:53 UTC" + }, + { + "_key": "00000000000000000034", + "_rev": "_d05kdja---", + "clientId": "CRDN-g6s8btds:99266912-daa7-48e7-bba2-885569db8e3d", + "epoch_millis": 1646833073750, + "request": { + "arango/Plan/Version": { + "op": "increment" + }, + "arango/Plan/Collections/_system/10001": { + "op": "set", + "new": { + "cacheEnabled": false, + "deleted": false, + "id": "10001", + "indexes": [ + { + "id": "0", + "type": "primary", + "name": "primary", + "fields": [ + "_key" + ], + "unique": true, + "sparse": false + } + ], + "internalValidatorType": 0, + "isDisjoint": false, + "isSmart": false, + "isSmartChild": false, + "isSystem": true, + "keyOptions": { + "allowUserKeys": true, + "type": "traditional" + }, + "minReplicationFactor": 1, + "name": "_users", + "numberOfShards": 1, + "replicationFactor": 2, + "schema": null, + "shardKeys": [ + "_key" + ], + "shardingStrategy": "hash", + "shards": { + "s10002": [ + "PRMR-a24adubi", + "PRMR-n92yizyp" + ] + }, + "status": 3, + "statusString": "loaded", + "syncByRevision": true, + "type": 2, + "usesRevisionsAsDocumentIds": true, + "waitForSync": false, + "writeConcern": 1, + "isBuilding": true, + "coordinatorRebootId": 1, + "coordinator": "CRDN-g6s8btds" + } + } + }, + "term": 1, + "timestamp": "2022-03-09 13:37:53 UTC" + }, + { + "_key": "00000000000000000035", + "_rev": "_d05ke-2---", + "clientId": "CRDN-ssc2yc0x:c9ba947c-4397-41a9-b9a5-7f2de148c087", + "epoch_millis": 1646833074189, + "request": { + "arango/Current/Version": { + "op": "increment" + } + }, + "term": 1, + "timestamp": "2022-03-09 13:37:54 UTC" + }, + { + "_key": "00000000000000000036", + "_rev": "_d05ke_6---", + "clientId": "CRDN-ssc2yc0x:897c15b5-eea3-4689-ba60-faed8deee5d0", + "epoch_millis": 1646833074207, + "request": { + "arango/Current/Version": { + "op": "increment" + } + }, + "term": 1, + "timestamp": "2022-03-09 13:37:54 UTC" + }, + { + "_key": "00000000000000000037", + "_rev": "_d05keEm--_", + "clientId": "CRDN-g6s8btds:2452742d-e8fa-43f4-ab94-93d42bd3d7f8", + "epoch_millis": 1646833074282, + "request": { + "arango/Current/Version": { + "op": "increment" + } + }, + "term": 1, + "timestamp": "2022-03-09 13:37:54 UTC" + }, + { + "_key": "00000000000000000038", + "_rev": "_d05keSq--A", + "clientId": "CRDN-cg5kjymo:b544c5ac-b2cf-4705-99c9-be2176a86d02", + "epoch_millis": 1646833074507, + "request": { + "arango/Current/Version": { + "op": "increment" + } + }, + "term": 1, + "timestamp": "2022-03-09 13:37:54 UTC" + }, + { + "_key": "00000000000000000039", + "_rev": "_d05ke9W--_", + "clientId": "CRDN-ssc2yc0x:386500b2-8d4d-4d6d-8e17-45583c0cf97a", + "epoch_millis": 1646833075190, + "request": { + "arango/Current/Version": { + "op": "increment" + } + }, + "term": 1, + "timestamp": "2022-03-09 13:37:55 UTC" + }, + { + "_key": "00000000000000000040", + "_rev": "_d05kfDG--A", + "clientId": "CRDN-g6s8btds:b86a2dd9-86f0-4e83-88c0-302b1098ca86", + "epoch_millis": 1646833075282, + "request": { + "arango/Current/Version": { + "op": "increment" + } + }, + "term": 1, + "timestamp": "2022-03-09 13:37:55 UTC" + }, + { + "_key": "00000000000000000041", + "_rev": "_d05kfRK---", + "clientId": "CRDN-cg5kjymo:2f98d7cc-a365-4fe3-befd-0dc2c4211fd6", + "epoch_millis": 1646833075507, + "request": { + "arango/Current/Version": { + "op": "increment" + } + }, + "term": 1, + "timestamp": "2022-03-09 13:37:55 UTC" + }, + { + "_key": "00000000000000000042", + "_rev": "_d05kfbK---", + "clientId": "", + "epoch_millis": 1646833075667, + "request": { + "/arango/Supervision/Health/PRMR-n92yizyp": { + "ShortName": "DBServer0002", + "Endpoint": "ssl://cluster-uexabr-dbserver-n92yizyp.cluster-uexabr-int.3dcd4eac-c549-41b6-bc8f-40e11983ebd2.svc:8529", + "Host": "ip-10-0-101-138.eu-central-1.compute.internal", + "SyncStatus": "STARTUP", + "Status": "GOOD", + "Version": "3.9.0", + "Engine": "rocksdb", + "Timestamp": "2022-03-09T13:37:55Z", + "SyncTime": "2022-03-09T13:37:53Z", + "LastAckedTime": "2022-03-09T13:37:54Z" + } + }, + "term": 1, + "timestamp": "2022-03-09 13:37:55 UTC" + }, + { + "_key": "00000000000000000043", + "_rev": "_d05kfbS--A", + "clientId": "", + "epoch_millis": 1646833075669, + "request": { + "/arango/Supervision/Health/PRMR-a24adubi": { + "ShortName": "DBServer0001", + "Endpoint": "ssl://cluster-uexabr-dbserver-a24adubi.cluster-uexabr-int.3dcd4eac-c549-41b6-bc8f-40e11983ebd2.svc:8529", + "Host": "ip-10-0-101-183.eu-central-1.compute.internal", + "SyncStatus": "STARTUP", + "Status": "GOOD", + "Version": "3.9.0", + "Engine": "rocksdb", + "Timestamp": "2022-03-09T13:37:55Z", + "SyncTime": "2022-03-09T13:37:43Z", + "LastAckedTime": "2022-03-09T13:37:54Z" + } + }, + "term": 1, + "timestamp": "2022-03-09 13:37:55 UTC" + }, + { + "_key": "00000000000000000044", + "_rev": "_d05kfbe--A", + "clientId": "", + "epoch_millis": 1646833075672, + "request": { + "/arango/Supervision/Health/CRDN-g6s8btds": { + "ShortName": "Coordinator0001", + "Endpoint": "ssl://cluster-uexabr-coordinator-g6s8btds.cluster-uexabr-int.3dcd4eac-c549-41b6-bc8f-40e11983ebd2.svc:8529", + "Host": "ip-10-0-100-174.eu-central-1.compute.internal", + "SyncStatus": "SERVING", + "Status": "GOOD", + "Version": "3.9.0", + "Engine": "rocksdb", + "Timestamp": "2022-03-09T13:37:55Z", + "SyncTime": "2022-03-09T13:37:55Z", + "LastAckedTime": "2022-03-09T13:37:55Z" + } + }, + "term": 1, + "timestamp": "2022-03-09 13:37:55 UTC" + }, + { + "_key": "00000000000000000045", + "_rev": "_d05kfbq--A", + "clientId": "", + "epoch_millis": 1646833075675, + "request": { + "/arango/Supervision/Health/CRDN-cg5kjymo": { + "ShortName": "Coordinator0003", + "Endpoint": "ssl://cluster-uexabr-coordinator-cg5kjymo.cluster-uexabr-int.3dcd4eac-c549-41b6-bc8f-40e11983ebd2.svc:8529", + "Host": "ip-10-0-101-110.eu-central-1.compute.internal", + "SyncStatus": "SERVING", + "Status": "GOOD", + "Version": "3.9.0", + "Engine": "rocksdb", + "Timestamp": "2022-03-09T13:37:55Z", + "SyncTime": "2022-03-09T13:37:55Z", + "LastAckedTime": "2022-03-09T13:37:55Z" + } + }, + "term": 1, + "timestamp": "2022-03-09 13:37:55 UTC" + }, + { + "_key": "00000000000000000046", + "_rev": "_d05kfb2--A", + "clientId": "", + "epoch_millis": 1646833075678, + "request": { + "/arango/Supervision/Health/CRDN-ssc2yc0x": { + "ShortName": "Coordinator0002", + "Endpoint": "ssl://cluster-uexabr-coordinator-ssc2yc0x.cluster-uexabr-int.3dcd4eac-c549-41b6-bc8f-40e11983ebd2.svc:8529", + "Host": "ip-10-0-101-183.eu-central-1.compute.internal", + "SyncStatus": "SERVING", + "Status": "GOOD", + "Version": "3.9.0", + "Engine": "rocksdb", + "Timestamp": "2022-03-09T13:37:55Z", + "SyncTime": "2022-03-09T13:37:55Z", + "LastAckedTime": "2022-03-09T13:37:55Z" + } + }, + "term": 1, + "timestamp": "2022-03-09 13:37:55 UTC" + }, + { + "_key": "00000000000000000047", + "_rev": "_d05kfqe--A", + "clientId": "PRMR-n92yizyp:41469515-61d3-4561-b9b7-77390ff8224f", + "epoch_millis": 1646833075912, + "request": { + "arango/Current/Databases/_system/PRMR-n92yizyp": { + "op": "set", + "new": { + "error": false, + "errorNum": 0, + "errorMessage": "", + "id": "1", + "name": "_system" + } + }, + "arango/Current/Version": { + "op": "increment" + } + }, + "term": 1, + "timestamp": "2022-03-09 13:37:55 UTC" + }, + { + "_key": "00000000000000000048", + "_rev": "_d05kfqq---", + "clientId": "PRMR-n92yizyp:8f72c03b-fb9f-4bae-af2c-1867cfcebb74", + "epoch_millis": 1646833075915, + "request": { + "arango/Sync/LatestID": { + "op": "set", + "new": 4010001 + } + }, + "term": 1, + "timestamp": "2022-03-09 13:37:55 UTC" + }, + { + "_key": "00000000000000000049", + "_rev": "_d05kfra--_", + "clientId": "PRMR-n92yizyp:431f3af4-f35b-4523-a1a4-2580f05e6b20", + "epoch_millis": 1646833075927, + "request": { + "arango/Current/Databases/_system/PRMR-n92yizyp": { + "op": "set", + "new": { + "error": false, + "errorNum": 0, + "errorMessage": "", + "id": "1", + "name": "_system" + } + }, + "arango/Current/Version": { + "op": "increment" + } + }, + "term": 1, + "timestamp": "2022-03-09 13:37:55 UTC" + }, + { + "_key": "00000000000000000050", + "_rev": "_d05kfzm---", + "clientId": "PRMR-a24adubi:d2957b9f-89e2-4274-b19e-5e051e24ed9d", + "epoch_millis": 1646833076057, + "request": { + "arango/Current/Databases/_system/PRMR-a24adubi": { + "op": "set", + "new": { + "error": false, + "errorNum": 0, + "errorMessage": "", + "id": "1", + "name": "_system" + } + }, + "arango/Current/Version": { + "op": "increment" + } + }, + "term": 1, + "timestamp": "2022-03-09 13:37:56 UTC" + }, + { + "_key": "00000000000000000051", + "_rev": "_d05kfzy---", + "clientId": "PRMR-a24adubi:dda1ab94-65ee-48f1-9e0c-5cd06e8e7041", + "epoch_millis": 1646833076061, + "request": { + "arango/Sync/LatestID": { + "op": "set", + "new": 6010001 + } + }, + "term": 1, + "timestamp": "2022-03-09 13:37:56 UTC" + }, + { + "_key": "00000000000000000052", + "_rev": "_d05kf0W--B", + "clientId": "PRMR-a24adubi:96cb5630-8653-49a8-bbb0-a6077a3ede15", + "epoch_millis": 1646833076070, + "request": { + "arango/Current/Databases/_system/PRMR-a24adubi": { + "op": "set", + "new": { + "error": false, + "errorNum": 0, + "errorMessage": "", + "id": "1", + "name": "_system" + } + }, + "arango/Current/Version": { + "op": "increment" + } + }, + "term": 1, + "timestamp": "2022-03-09 13:37:56 UTC" + }, + { + "_key": "00000000000000000053", + "_rev": "_d05kf1q--_", + "clientId": "PRMR-a24adubi:1a6fc62d-134b-4cd8-8d86-59ab06ed303d", + "epoch_millis": 1646833076091, + "request": { + "arango/Current/Collections/_system/10001/s10002": { + "op": "set", + "new": { + "error": false, + "errorMessage": "", + "errorNum": 0, + "indexes": [ + { + "fields": [ + "_key" + ], + "id": "0", + "name": "primary", + "objectId": "4010002", + "sparse": false, + "type": "primary", + "unique": true + } + ], + "servers": [ + "PRMR-a24adubi" + ], + "failoverCandidates": [ + "PRMR-a24adubi" + ] + } + }, + "arango/Current/Version": { + "op": "increment" + } + }, + "term": 1, + "timestamp": "2022-03-09 13:37:56 UTC" + }, + { + "_key": "00000000000000000054", + "_rev": "_d05kf4S--A", + "clientId": "PRMR-a24adubi:a3d8351c-68ec-4888-b036-77eb0360bc58", + "epoch_millis": 1646833076133, + "request": { + "arango/Current/Collections/_system/10001/s10002": { + "op": "set", + "new": { + "error": false, + "errorMessage": "", + "errorNum": 0, + "indexes": [ + { + "fields": [ + "_key" + ], + "id": "0", + "name": "primary", + "objectId": "4010002", + "sparse": false, + "type": "primary", + "unique": true + } + ], + "servers": [ + "PRMR-a24adubi", + "PRMR-n92yizyp" + ], + "failoverCandidates": [ + "PRMR-a24adubi", + "PRMR-n92yizyp" + ] + } + }, + "arango/Current/Version": { + "op": "increment" + } + }, + "term": 1, + "timestamp": "2022-03-09 13:37:56 UTC" + }, + { + "_key": "00000000000000000055", + "_rev": "_d05kf42---", + "clientId": "CRDN-g6s8btds:3b0af2e2-cbf9-492f-adb7-1d7708bdfc36", + "epoch_millis": 1646833076142, + "request": { + "arango/Plan/Version": { + "op": "increment" + }, + "arango/Plan/Collections/_system/10001": { + "op": "set", + "new": { + "cacheEnabled": false, + "deleted": false, + "id": "10001", + "indexes": [ + { + "id": "0", + "type": "primary", + "name": "primary", + "fields": [ + "_key" + ], + "unique": true, + "sparse": false + } + ], + "internalValidatorType": 0, + "isDisjoint": false, + "isSmart": false, + "isSmartChild": false, + "isSystem": true, + "keyOptions": { + "allowUserKeys": true, + "type": "traditional" + }, + "minReplicationFactor": 1, + "name": "_users", + "numberOfShards": 1, + "replicationFactor": 2, + "schema": null, + "shardKeys": [ + "_key" + ], + "shardingStrategy": "hash", + "shards": { + "s10002": [ + "PRMR-a24adubi", + "PRMR-n92yizyp" + ] + }, + "status": 3, + "statusString": "loaded", + "syncByRevision": true, + "type": 2, + "usesRevisionsAsDocumentIds": true, + "waitForSync": false, + "writeConcern": 1 + } + } + }, + "term": 1, + "timestamp": "2022-03-09 13:37:56 UTC" + }, + { + "_key": "00000000000000000056", + "_rev": "_d05kf5a---", + "clientId": "CRDN-g6s8btds:5690337c-9e5e-4a3a-bf0d-14521ffc5c2c", + "epoch_millis": 1646833076151, + "request": { + "arango/Plan/Version": { + "op": "increment" + }, + "arango/Plan/Collections/_system/10003": { + "op": "set", + "new": { + "cacheEnabled": false, + "deleted": false, + "distributeShardsLike": "10001", + "id": "10003", + "indexes": [ + { + "id": "0", + "type": "primary", + "name": "primary", + "fields": [ + "_key" + ], + "unique": true, + "sparse": false + } + ], + "internalValidatorType": 0, + "isDisjoint": false, + "isSmart": false, + "isSmartChild": false, + "isSystem": true, + "keyOptions": { + "allowUserKeys": true, + "type": "traditional" + }, + "minReplicationFactor": 1, + "name": "_graphs", + "numberOfShards": 1, + "replicationFactor": 2, + "schema": null, + "shardKeys": [ + "_key" + ], + "shardingStrategy": "hash", + "shards": { + "s10014": [ + "PRMR-a24adubi", + "PRMR-n92yizyp" + ] + }, + "status": 3, + "statusString": "loaded", + "syncByRevision": true, + "type": 2, + "usesRevisionsAsDocumentIds": true, + "waitForSync": false, + "writeConcern": 1, + "isBuilding": true, + "coordinatorRebootId": 1, + "coordinator": "CRDN-g6s8btds" + } + }, + "arango/Plan/Collections/_system/10004": { + "op": "set", + "new": { + "cacheEnabled": false, + "deleted": false, + "distributeShardsLike": "10001", + "id": "10004", + "indexes": [ + { + "id": "0", + "type": "primary", + "name": "primary", + "fields": [ + "_key" + ], + "unique": true, + "sparse": false + } + ], + "internalValidatorType": 0, + "isDisjoint": false, + "isSmart": false, + "isSmartChild": false, + "isSystem": true, + "keyOptions": { + "allowUserKeys": true, + "type": "traditional" + }, + "minReplicationFactor": 1, + "name": "_statistics", + "numberOfShards": 1, + "replicationFactor": 2, + "schema": null, + "shardKeys": [ + "_key" + ], + "shardingStrategy": "hash", + "shards": { + "s10015": [ + "PRMR-a24adubi", + "PRMR-n92yizyp" + ] + }, + "status": 3, + "statusString": "loaded", + "syncByRevision": true, + "type": 2, + "usesRevisionsAsDocumentIds": true, + "waitForSync": false, + "writeConcern": 1, + "isBuilding": true, + "coordinatorRebootId": 1, + "coordinator": "CRDN-g6s8btds" + } + }, + "arango/Plan/Collections/_system/10005": { + "op": "set", + "new": { + "cacheEnabled": false, + "deleted": false, + "distributeShardsLike": "10001", + "id": "10005", + "indexes": [ + { + "id": "0", + "type": "primary", + "name": "primary", + "fields": [ + "_key" + ], + "unique": true, + "sparse": false + } + ], + "internalValidatorType": 0, + "isDisjoint": false, + "isSmart": false, + "isSmartChild": false, + "isSystem": true, + "keyOptions": { + "allowUserKeys": true, + "type": "traditional" + }, + "minReplicationFactor": 1, + "name": "_statistics15", + "numberOfShards": 1, + "replicationFactor": 2, + "schema": null, + "shardKeys": [ + "_key" + ], + "shardingStrategy": "hash", + "shards": { + "s10016": [ + "PRMR-a24adubi", + "PRMR-n92yizyp" + ] + }, + "status": 3, + "statusString": "loaded", + "syncByRevision": true, + "type": 2, + "usesRevisionsAsDocumentIds": true, + "waitForSync": false, + "writeConcern": 1, + "isBuilding": true, + "coordinatorRebootId": 1, + "coordinator": "CRDN-g6s8btds" + } + }, + "arango/Plan/Collections/_system/10006": { + "op": "set", + "new": { + "cacheEnabled": false, + "deleted": false, + "distributeShardsLike": "10001", + "id": "10006", + "indexes": [ + { + "id": "0", + "type": "primary", + "name": "primary", + "fields": [ + "_key" + ], + "unique": true, + "sparse": false + } + ], + "internalValidatorType": 0, + "isDisjoint": false, + "isSmart": false, + "isSmartChild": false, + "isSystem": true, + "keyOptions": { + "allowUserKeys": true, + "type": "traditional" + }, + "minReplicationFactor": 1, + "name": "_statisticsRaw", + "numberOfShards": 1, + "replicationFactor": 2, + "schema": null, + "shardKeys": [ + "_key" + ], + "shardingStrategy": "hash", + "shards": { + "s10017": [ + "PRMR-a24adubi", + "PRMR-n92yizyp" + ] + }, + "status": 3, + "statusString": "loaded", + "syncByRevision": true, + "type": 2, + "usesRevisionsAsDocumentIds": true, + "waitForSync": false, + "writeConcern": 1, + "isBuilding": true, + "coordinatorRebootId": 1, + "coordinator": "CRDN-g6s8btds" + } + }, + "arango/Plan/Collections/_system/10007": { + "op": "set", + "new": { + "cacheEnabled": false, + "deleted": false, + "distributeShardsLike": "10001", + "id": "10007", + "indexes": [ + { + "id": "0", + "type": "primary", + "name": "primary", + "fields": [ + "_key" + ], + "unique": true, + "sparse": false + } + ], + "internalValidatorType": 0, + "isDisjoint": false, + "isSmart": false, + "isSmartChild": false, + "isSystem": true, + "keyOptions": { + "allowUserKeys": true, + "type": "traditional" + }, + "minReplicationFactor": 1, + "name": "_analyzers", + "numberOfShards": 1, + "replicationFactor": 2, + "schema": null, + "shardKeys": [ + "_key" + ], + "shardingStrategy": "hash", + "shards": { + "s10018": [ + "PRMR-a24adubi", + "PRMR-n92yizyp" + ] + }, + "status": 3, + "statusString": "loaded", + "syncByRevision": true, + "type": 2, + "usesRevisionsAsDocumentIds": true, + "waitForSync": false, + "writeConcern": 1, + "isBuilding": true, + "coordinatorRebootId": 1, + "coordinator": "CRDN-g6s8btds" + } + }, + "arango/Plan/Collections/_system/10008": { + "op": "set", + "new": { + "cacheEnabled": false, + "deleted": false, + "distributeShardsLike": "10001", + "id": "10008", + "indexes": [ + { + "id": "0", + "type": "primary", + "name": "primary", + "fields": [ + "_key" + ], + "unique": true, + "sparse": false + } + ], + "internalValidatorType": 0, + "isDisjoint": false, + "isSmart": false, + "isSmartChild": false, + "isSystem": true, + "keyOptions": { + "allowUserKeys": true, + "type": "traditional" + }, + "minReplicationFactor": 1, + "name": "_aqlfunctions", + "numberOfShards": 1, + "replicationFactor": 2, + "schema": null, + "shardKeys": [ + "_key" + ], + "shardingStrategy": "hash", + "shards": { + "s10019": [ + "PRMR-a24adubi", + "PRMR-n92yizyp" + ] + }, + "status": 3, + "statusString": "loaded", + "syncByRevision": true, + "type": 2, + "usesRevisionsAsDocumentIds": true, + "waitForSync": false, + "writeConcern": 1, + "isBuilding": true, + "coordinatorRebootId": 1, + "coordinator": "CRDN-g6s8btds" + } + }, + "arango/Plan/Collections/_system/10009": { + "op": "set", + "new": { + "cacheEnabled": false, + "deleted": false, + "distributeShardsLike": "10001", + "id": "10009", + "indexes": [ + { + "id": "0", + "type": "primary", + "name": "primary", + "fields": [ + "_key" + ], + "unique": true, + "sparse": false + } + ], + "internalValidatorType": 0, + "isDisjoint": false, + "isSmart": false, + "isSmartChild": false, + "isSystem": true, + "keyOptions": { + "allowUserKeys": true, + "type": "traditional" + }, + "minReplicationFactor": 1, + "name": "_queues", + "numberOfShards": 1, + "replicationFactor": 2, + "schema": null, + "shardKeys": [ + "_key" + ], + "shardingStrategy": "hash", + "shards": { + "s10020": [ + "PRMR-a24adubi", + "PRMR-n92yizyp" + ] + }, + "status": 3, + "statusString": "loaded", + "syncByRevision": true, + "type": 2, + "usesRevisionsAsDocumentIds": true, + "waitForSync": false, + "writeConcern": 1, + "isBuilding": true, + "coordinatorRebootId": 1, + "coordinator": "CRDN-g6s8btds" + } + }, + "arango/Plan/Collections/_system/10010": { + "op": "set", + "new": { + "cacheEnabled": false, + "deleted": false, + "distributeShardsLike": "10001", + "id": "10010", + "indexes": [ + { + "id": "0", + "type": "primary", + "name": "primary", + "fields": [ + "_key" + ], + "unique": true, + "sparse": false + } + ], + "internalValidatorType": 0, + "isDisjoint": false, + "isSmart": false, + "isSmartChild": false, + "isSystem": true, + "keyOptions": { + "allowUserKeys": true, + "type": "traditional" + }, + "minReplicationFactor": 1, + "name": "_jobs", + "numberOfShards": 1, + "replicationFactor": 2, + "schema": null, + "shardKeys": [ + "_key" + ], + "shardingStrategy": "hash", + "shards": { + "s10021": [ + "PRMR-a24adubi", + "PRMR-n92yizyp" + ] + }, + "status": 3, + "statusString": "loaded", + "syncByRevision": true, + "type": 2, + "usesRevisionsAsDocumentIds": true, + "waitForSync": false, + "writeConcern": 1, + "isBuilding": true, + "coordinatorRebootId": 1, + "coordinator": "CRDN-g6s8btds" + } + }, + "arango/Plan/Collections/_system/10011": { + "op": "set", + "new": { + "cacheEnabled": false, + "deleted": false, + "distributeShardsLike": "10001", + "id": "10011", + "indexes": [ + { + "id": "0", + "type": "primary", + "name": "primary", + "fields": [ + "_key" + ], + "unique": true, + "sparse": false + } + ], + "internalValidatorType": 0, + "isDisjoint": false, + "isSmart": false, + "isSmartChild": false, + "isSystem": true, + "keyOptions": { + "allowUserKeys": true, + "type": "traditional" + }, + "minReplicationFactor": 1, + "name": "_apps", + "numberOfShards": 1, + "replicationFactor": 2, + "schema": null, + "shardKeys": [ + "_key" + ], + "shardingStrategy": "hash", + "shards": { + "s10022": [ + "PRMR-a24adubi", + "PRMR-n92yizyp" + ] + }, + "status": 3, + "statusString": "loaded", + "syncByRevision": true, + "type": 2, + "usesRevisionsAsDocumentIds": true, + "waitForSync": false, + "writeConcern": 1, + "isBuilding": true, + "coordinatorRebootId": 1, + "coordinator": "CRDN-g6s8btds" + } + }, + "arango/Plan/Collections/_system/10012": { + "op": "set", + "new": { + "cacheEnabled": false, + "deleted": false, + "distributeShardsLike": "10001", + "id": "10012", + "indexes": [ + { + "id": "0", + "type": "primary", + "name": "primary", + "fields": [ + "_key" + ], + "unique": true, + "sparse": false + } + ], + "internalValidatorType": 0, + "isDisjoint": false, + "isSmart": false, + "isSmartChild": false, + "isSystem": true, + "keyOptions": { + "allowUserKeys": true, + "type": "traditional" + }, + "minReplicationFactor": 1, + "name": "_appbundles", + "numberOfShards": 1, + "replicationFactor": 2, + "schema": null, + "shardKeys": [ + "_key" + ], + "shardingStrategy": "hash", + "shards": { + "s10023": [ + "PRMR-a24adubi", + "PRMR-n92yizyp" + ] + }, + "status": 3, + "statusString": "loaded", + "syncByRevision": true, + "type": 2, + "usesRevisionsAsDocumentIds": true, + "waitForSync": false, + "writeConcern": 1, + "isBuilding": true, + "coordinatorRebootId": 1, + "coordinator": "CRDN-g6s8btds" + } + }, + "arango/Plan/Collections/_system/10013": { + "op": "set", + "new": { + "cacheEnabled": false, + "deleted": false, + "distributeShardsLike": "10001", + "id": "10013", + "indexes": [ + { + "id": "0", + "type": "primary", + "name": "primary", + "fields": [ + "_key" + ], + "unique": true, + "sparse": false + } + ], + "internalValidatorType": 0, + "isDisjoint": false, + "isSmart": false, + "isSmartChild": false, + "isSystem": true, + "keyOptions": { + "allowUserKeys": true, + "type": "traditional" + }, + "minReplicationFactor": 1, + "name": "_frontend", + "numberOfShards": 1, + "replicationFactor": 2, + "schema": null, + "shardKeys": [ + "_key" + ], + "shardingStrategy": "hash", + "shards": { + "s10024": [ + "PRMR-a24adubi", + "PRMR-n92yizyp" + ] + }, + "status": 3, + "statusString": "loaded", + "syncByRevision": true, + "type": 2, + "usesRevisionsAsDocumentIds": true, + "waitForSync": false, + "writeConcern": 1, + "isBuilding": true, + "coordinatorRebootId": 1, + "coordinator": "CRDN-g6s8btds" + } + } + }, + "term": 1, + "timestamp": "2022-03-09 13:37:56 UTC" + }, + { + "_key": "00000000000000000057", + "_rev": "_d05kf6S---", + "clientId": "PRMR-a24adubi:d763235f-e6d8-4228-afcc-d8bedfac3cdf", + "epoch_millis": 1646833076165, + "request": { + "arango/Current/Collections/_system/10003/s10014": { + "op": "set", + "new": { + "error": false, + "errorMessage": "", + "errorNum": 0, + "indexes": [ + { + "fields": [ + "_key" + ], + "id": "0", + "name": "primary", + "objectId": "4010026", + "sparse": false, + "type": "primary", + "unique": true + } + ], + "servers": [ + "PRMR-a24adubi" + ], + "failoverCandidates": [ + "PRMR-a24adubi" + ] + } + }, + "arango/Current/Collections/_system/10004/s10015": { + "op": "set", + "new": { + "error": false, + "errorMessage": "", + "errorNum": 0, + "indexes": [ + { + "fields": [ + "_key" + ], + "id": "0", + "name": "primary", + "objectId": "4010030", + "sparse": false, + "type": "primary", + "unique": true + } + ], + "servers": [ + "PRMR-a24adubi" + ], + "failoverCandidates": [ + "PRMR-a24adubi" + ] + } + }, + "arango/Current/Collections/_system/10005/s10016": { + "op": "set", + "new": { + "error": false, + "errorMessage": "", + "errorNum": 0, + "indexes": [ + { + "fields": [ + "_key" + ], + "id": "0", + "name": "primary", + "objectId": "4010036", + "sparse": false, + "type": "primary", + "unique": true + } + ], + "servers": [ + "PRMR-a24adubi" + ], + "failoverCandidates": [ + "PRMR-a24adubi" + ] + } + }, + "arango/Current/Collections/_system/10006/s10017": { + "op": "set", + "new": { + "error": false, + "errorMessage": "", + "errorNum": 0, + "indexes": [ + { + "fields": [ + "_key" + ], + "id": "0", + "name": "primary", + "objectId": "4010028", + "sparse": false, + "type": "primary", + "unique": true + } + ], + "servers": [ + "PRMR-a24adubi" + ], + "failoverCandidates": [ + "PRMR-a24adubi" + ] + } + }, + "arango/Current/Collections/_system/10007/s10018": { + "op": "set", + "new": { + "error": false, + "errorMessage": "", + "errorNum": 0, + "indexes": [ + { + "fields": [ + "_key" + ], + "id": "0", + "name": "primary", + "objectId": "4010023", + "sparse": false, + "type": "primary", + "unique": true + } + ], + "servers": [ + "PRMR-a24adubi" + ], + "failoverCandidates": [ + "PRMR-a24adubi" + ] + } + }, + "arango/Current/Collections/_system/10008/s10019": { + "op": "set", + "new": { + "error": false, + "errorMessage": "", + "errorNum": 0, + "indexes": [ + { + "fields": [ + "_key" + ], + "id": "0", + "name": "primary", + "objectId": "4010024", + "sparse": false, + "type": "primary", + "unique": true + } + ], + "servers": [ + "PRMR-a24adubi" + ], + "failoverCandidates": [ + "PRMR-a24adubi" + ] + } + }, + "arango/Current/Collections/_system/10009/s10020": { + "op": "set", + "new": { + "error": false, + "errorMessage": "", + "errorNum": 0, + "indexes": [ + { + "fields": [ + "_key" + ], + "id": "0", + "name": "primary", + "objectId": "4010038", + "sparse": false, + "type": "primary", + "unique": true + } + ], + "servers": [ + "PRMR-a24adubi" + ], + "failoverCandidates": [ + "PRMR-a24adubi" + ] + } + }, + "arango/Current/Collections/_system/10010/s10021": { + "op": "set", + "new": { + "error": false, + "errorMessage": "", + "errorNum": 0, + "indexes": [ + { + "fields": [ + "_key" + ], + "id": "0", + "name": "primary", + "objectId": "4010032", + "sparse": false, + "type": "primary", + "unique": true + } + ], + "servers": [ + "PRMR-a24adubi" + ], + "failoverCandidates": [ + "PRMR-a24adubi" + ] + } + }, + "arango/Current/Collections/_system/10011/s10022": { + "op": "set", + "new": { + "error": false, + "errorMessage": "", + "errorNum": 0, + "indexes": [ + { + "fields": [ + "_key" + ], + "id": "0", + "name": "primary", + "objectId": "4010042", + "sparse": false, + "type": "primary", + "unique": true + } + ], + "servers": [ + "PRMR-a24adubi" + ], + "failoverCandidates": [ + "PRMR-a24adubi" + ] + } + }, + "arango/Current/Collections/_system/10012/s10023": { + "op": "set", + "new": { + "error": false, + "errorMessage": "", + "errorNum": 0, + "indexes": [ + { + "fields": [ + "_key" + ], + "id": "0", + "name": "primary", + "objectId": "4010034", + "sparse": false, + "type": "primary", + "unique": true + } + ], + "servers": [ + "PRMR-a24adubi" + ], + "failoverCandidates": [ + "PRMR-a24adubi" + ] + } + }, + "arango/Current/Collections/_system/10013/s10024": { + "op": "set", + "new": { + "error": false, + "errorMessage": "", + "errorNum": 0, + "indexes": [ + { + "fields": [ + "_key" + ], + "id": "0", + "name": "primary", + "objectId": "4010040", + "sparse": false, + "type": "primary", + "unique": true + } + ], + "servers": [ + "PRMR-a24adubi" + ], + "failoverCandidates": [ + "PRMR-a24adubi" + ] + } + }, + "arango/Current/Version": { + "op": "increment" + } + }, + "term": 1, + "timestamp": "2022-03-09 13:37:56 UTC" + }, + { + "_key": "00000000000000000058", + "_rev": "_d05kf7u---", + "clientId": "PRMR-a24adubi:2de6e398-b67a-4db5-9ca3-e36bb94b31a2", + "epoch_millis": 1646833076188, + "request": { + "arango/Current/Collections/_system/10003/s10014": { + "op": "set", + "new": { + "error": false, + "errorMessage": "", + "errorNum": 0, + "indexes": [ + { + "fields": [ + "_key" + ], + "id": "0", + "name": "primary", + "objectId": "4010026", + "sparse": false, + "type": "primary", + "unique": true + } + ], + "servers": [ + "PRMR-a24adubi", + "PRMR-n92yizyp" + ], + "failoverCandidates": [ + "PRMR-a24adubi", + "PRMR-n92yizyp" + ] + } + }, + "arango/Current/Version": { + "op": "increment" + } + }, + "term": 1, + "timestamp": "2022-03-09 13:37:56 UTC" + }, + { + "_key": "00000000000000000059", + "_rev": "_d05kf8C--A", + "clientId": "CRDN-ssc2yc0x:a7c93f41-530c-4c05-b830-1efb65abe3d0", + "epoch_millis": 1646833076193, + "request": { + "arango/Current/Version": { + "op": "increment" + } + }, + "term": 1, + "timestamp": "2022-03-09 13:37:56 UTC" + }, + { + "_key": "00000000000000000060", + "_rev": "_d05kf8S--B", + "clientId": "PRMR-a24adubi:d1e8c789-7ebd-420b-bb2a-e7f2c731e5a2", + "epoch_millis": 1646833076197, + "request": { + "arango/Current/Collections/_system/10004/s10015": { + "op": "set", + "new": { + "error": false, + "errorMessage": "", + "errorNum": 0, + "indexes": [ + { + "fields": [ + "_key" + ], + "id": "0", + "name": "primary", + "objectId": "4010030", + "sparse": false, + "type": "primary", + "unique": true + } + ], + "servers": [ + "PRMR-a24adubi", + "PRMR-n92yizyp" + ], + "failoverCandidates": [ + "PRMR-a24adubi", + "PRMR-n92yizyp" + ] + } + }, + "arango/Current/Version": { + "op": "increment" + } + }, + "term": 1, + "timestamp": "2022-03-09 13:37:56 UTC" + }, + { + "_key": "00000000000000000061", + "_rev": "_d05kf8q--A", + "clientId": "PRMR-a24adubi:7206807a-500b-4f4e-9883-2aae9c827030", + "epoch_millis": 1646833076203, + "request": { + "arango/Current/Collections/_system/10004/s10015": { + "op": "set", + "new": { + "error": false, + "errorMessage": "", + "errorNum": 0, + "indexes": [ + { + "fields": [ + "_key" + ], + "id": "0", + "name": "primary", + "objectId": "4010030", + "sparse": false, + "type": "primary", + "unique": true + } + ], + "servers": [ + "PRMR-a24adubi", + "PRMR-n92yizyp" + ], + "failoverCandidates": [ + "PRMR-a24adubi", + "PRMR-n92yizyp" + ] + } + }, + "arango/Current/Version": { + "op": "increment" + } + }, + "term": 1, + "timestamp": "2022-03-09 13:37:56 UTC" + }, + { + "_key": "00000000000000000062", + "_rev": "_d05kf9O--_", + "clientId": "PRMR-a24adubi:89e5153b-3773-4971-b553-0f8762fe0498", + "epoch_millis": 1646833076212, + "request": { + "arango/Current/Collections/_system/10005/s10016": { + "op": "set", + "new": { + "error": false, + "errorMessage": "", + "errorNum": 0, + "indexes": [ + { + "fields": [ + "_key" + ], + "id": "0", + "name": "primary", + "objectId": "4010036", + "sparse": false, + "type": "primary", + "unique": true + } + ], + "servers": [ + "PRMR-a24adubi", + "PRMR-n92yizyp" + ], + "failoverCandidates": [ + "PRMR-a24adubi", + "PRMR-n92yizyp" + ] + } + }, + "arango/Current/Version": { + "op": "increment" + } + }, + "term": 1, + "timestamp": "2022-03-09 13:37:56 UTC" + }, + { + "_key": "00000000000000000063", + "_rev": "_d05kf9u--C", + "clientId": "PRMR-a24adubi:a0064372-f39f-489e-99f7-2aaf1e6d3cc0", + "epoch_millis": 1646833076220, + "request": { + "arango/Current/Collections/_system/10006/s10017": { + "op": "set", + "new": { + "error": false, + "errorMessage": "", + "errorNum": 0, + "indexes": [ + { + "fields": [ + "_key" + ], + "id": "0", + "name": "primary", + "objectId": "4010028", + "sparse": false, + "type": "primary", + "unique": true + } + ], + "servers": [ + "PRMR-a24adubi", + "PRMR-n92yizyp" + ], + "failoverCandidates": [ + "PRMR-a24adubi", + "PRMR-n92yizyp" + ] + } + }, + "arango/Current/Version": { + "op": "increment" + } + }, + "term": 1, + "timestamp": "2022-03-09 13:37:56 UTC" + }, + { + "_key": "00000000000000000064", + "_rev": "_d05kf92--A", + "clientId": "PRMR-a24adubi:f0d73d81-65e9-48c2-9988-0cab85201bcb", + "epoch_millis": 1646833076222, + "request": { + "arango/Current/Collections/_system/10006/s10017": { + "op": "set", + "new": { + "error": false, + "errorMessage": "", + "errorNum": 0, + "indexes": [ + { + "fields": [ + "_key" + ], + "id": "0", + "name": "primary", + "objectId": "4010028", + "sparse": false, + "type": "primary", + "unique": true + } + ], + "servers": [ + "PRMR-a24adubi", + "PRMR-n92yizyp" + ], + "failoverCandidates": [ + "PRMR-a24adubi", + "PRMR-n92yizyp" + ] + } + }, + "arango/Current/Version": { + "op": "increment" + } + }, + "term": 1, + "timestamp": "2022-03-09 13:37:56 UTC" + }, + { + "_key": "00000000000000000065", + "_rev": "_d05kg_2--_", + "clientId": "PRMR-a24adubi:c8df640f-5a3a-492a-866b-bbf8c8a3b4b0", + "epoch_millis": 1646833076254, + "request": { + "arango/Current/Collections/_system/10007/s10018": { + "op": "set", + "new": { + "error": false, + "errorMessage": "", + "errorNum": 0, + "indexes": [ + { + "fields": [ + "_key" + ], + "id": "0", + "name": "primary", + "objectId": "4010023", + "sparse": false, + "type": "primary", + "unique": true + } + ], + "servers": [ + "PRMR-a24adubi", + "PRMR-n92yizyp" + ], + "failoverCandidates": [ + "PRMR-a24adubi", + "PRMR-n92yizyp" + ] + } + }, + "arango/Current/Version": { + "op": "increment" + } + }, + "term": 1, + "timestamp": "2022-03-09 13:37:56 UTC" + }, + { + "_key": "00000000000000000066", + "_rev": "_d05kgAu--_", + "clientId": "PRMR-a24adubi:21c76458-1d1a-4a0c-b64d-59136808faa1", + "epoch_millis": 1646833076268, + "request": { + "arango/Current/Collections/_system/10005/s10016": { + "op": "set", + "new": { + "error": false, + "errorMessage": "", + "errorNum": 0, + "indexes": [ + { + "fields": [ + "_key" + ], + "id": "0", + "name": "primary", + "objectId": "4010036", + "sparse": false, + "type": "primary", + "unique": true + } + ], + "servers": [ + "PRMR-a24adubi", + "PRMR-n92yizyp" + ], + "failoverCandidates": [ + "PRMR-a24adubi", + "PRMR-n92yizyp" + ] + } + }, + "arango/Current/Version": { + "op": "increment" + } + }, + "term": 1, + "timestamp": "2022-03-09 13:37:56 UTC" + }, + { + "_key": "00000000000000000067", + "_rev": "_d05kgA6--B", + "clientId": "PRMR-a24adubi:68f3ba94-3351-4530-92c5-ae27373d2ee6", + "epoch_millis": 1646833076271, + "request": { + "arango/Current/Collections/_system/10008/s10019": { + "op": "set", + "new": { + "error": false, + "errorMessage": "", + "errorNum": 0, + "indexes": [ + { + "fields": [ + "_key" + ], + "id": "0", + "name": "primary", + "objectId": "4010024", + "sparse": false, + "type": "primary", + "unique": true + } + ], + "servers": [ + "PRMR-a24adubi", + "PRMR-n92yizyp" + ], + "failoverCandidates": [ + "PRMR-a24adubi", + "PRMR-n92yizyp" + ] + } + }, + "arango/Current/Version": { + "op": "increment" + } + }, + "term": 1, + "timestamp": "2022-03-09 13:37:56 UTC" + }, + { + "_key": "00000000000000000068", + "_rev": "_d05kgBS--_", + "clientId": "PRMR-a24adubi:af0c6ec6-512d-4d46-8cbe-b535e5b1ec13", + "epoch_millis": 1646833076277, + "request": { + "arango/Current/Collections/_system/10008/s10019": { + "op": "set", + "new": { + "error": false, + "errorMessage": "", + "errorNum": 0, + "indexes": [ + { + "fields": [ + "_key" + ], + "id": "0", + "name": "primary", + "objectId": "4010024", + "sparse": false, + "type": "primary", + "unique": true + } + ], + "servers": [ + "PRMR-a24adubi", + "PRMR-n92yizyp" + ], + "failoverCandidates": [ + "PRMR-a24adubi", + "PRMR-n92yizyp" + ] + } + }, + "arango/Current/Version": { + "op": "increment" + } + }, + "term": 1, + "timestamp": "2022-03-09 13:37:56 UTC" + }, + { + "_key": "00000000000000000069", + "_rev": "_d05kgBm--C", + "clientId": "CRDN-g6s8btds:46144953-f11b-4875-a6ea-19461b19d5fc", + "epoch_millis": 1646833076282, + "request": { + "arango/Current/Version": { + "op": "increment" + } + }, + "term": 1, + "timestamp": "2022-03-09 13:37:56 UTC" + }, + { + "_key": "00000000000000000070", + "_rev": "_d05kgB6--D", + "clientId": "PRMR-a24adubi:03c435ac-edda-41a4-bf21-2e799344b27d", + "epoch_millis": 1646833076287, + "request": { + "arango/Current/Collections/_system/10009/s10020": { + "op": "set", + "new": { + "error": false, + "errorMessage": "", + "errorNum": 0, + "indexes": [ + { + "fields": [ + "_key" + ], + "id": "0", + "name": "primary", + "objectId": "4010038", + "sparse": false, + "type": "primary", + "unique": true + } + ], + "servers": [ + "PRMR-a24adubi", + "PRMR-n92yizyp" + ], + "failoverCandidates": [ + "PRMR-a24adubi", + "PRMR-n92yizyp" + ] + } + }, + "arango/Current/Version": { + "op": "increment" + } + }, + "term": 1, + "timestamp": "2022-03-09 13:37:56 UTC" + }, + { + "_key": "00000000000000000071", + "_rev": "_d05kgCO---", + "clientId": "PRMR-a24adubi:06da24a8-63c0-430a-9d99-d254d14cf5dd", + "epoch_millis": 1646833076292, + "request": { + "arango/Current/Collections/_system/10010/s10021": { + "op": "set", + "new": { + "error": false, + "errorMessage": "", + "errorNum": 0, + "indexes": [ + { + "fields": [ + "_key" + ], + "id": "0", + "name": "primary", + "objectId": "4010032", + "sparse": false, + "type": "primary", + "unique": true + } + ], + "servers": [ + "PRMR-a24adubi", + "PRMR-n92yizyp" + ], + "failoverCandidates": [ + "PRMR-a24adubi", + "PRMR-n92yizyp" + ] + } + }, + "arango/Current/Version": { + "op": "increment" + } + }, + "term": 1, + "timestamp": "2022-03-09 13:37:56 UTC" + }, + { + "_key": "00000000000000000072", + "_rev": "_d05kgCm--A", + "clientId": "PRMR-a24adubi:6c68202e-365f-4856-b9a6-97d24d2e58d5", + "epoch_millis": 1646833076298, + "request": { + "arango/Current/Collections/_system/10009/s10020": { + "op": "set", + "new": { + "error": false, + "errorMessage": "", + "errorNum": 0, + "indexes": [ + { + "fields": [ + "_key" + ], + "id": "0", + "name": "primary", + "objectId": "4010038", + "sparse": false, + "type": "primary", + "unique": true + } + ], + "servers": [ + "PRMR-a24adubi", + "PRMR-n92yizyp" + ], + "failoverCandidates": [ + "PRMR-a24adubi", + "PRMR-n92yizyp" + ] + } + }, + "arango/Current/Collections/_system/10010/s10021": { + "op": "set", + "new": { + "error": false, + "errorMessage": "", + "errorNum": 0, + "indexes": [ + { + "fields": [ + "_key" + ], + "id": "0", + "name": "primary", + "objectId": "4010032", + "sparse": false, + "type": "primary", + "unique": true + } + ], + "servers": [ + "PRMR-a24adubi", + "PRMR-n92yizyp" + ], + "failoverCandidates": [ + "PRMR-a24adubi", + "PRMR-n92yizyp" + ] + } + }, + "arango/Current/Version": { + "op": "increment" + } + }, + "term": 1, + "timestamp": "2022-03-09 13:37:56 UTC" + }, + { + "_key": "00000000000000000073", + "_rev": "_d05kgEa---", + "clientId": "PRMR-a24adubi:0947282c-f775-4e53-b091-0f4f45e31fc7", + "epoch_millis": 1646833076327, + "request": { + "arango/Current/Collections/_system/10011/s10022": { + "op": "set", + "new": { + "error": false, + "errorMessage": "", + "errorNum": 0, + "indexes": [ + { + "fields": [ + "_key" + ], + "id": "0", + "name": "primary", + "objectId": "4010042", + "sparse": false, + "type": "primary", + "unique": true + } + ], + "servers": [ + "PRMR-a24adubi", + "PRMR-n92yizyp" + ], + "failoverCandidates": [ + "PRMR-a24adubi", + "PRMR-n92yizyp" + ] + } + }, + "arango/Current/Version": { + "op": "increment" + } + }, + "term": 1, + "timestamp": "2022-03-09 13:37:56 UTC" + }, + { + "_key": "00000000000000000074", + "_rev": "_d05kgFi---", + "clientId": "PRMR-a24adubi:54afa475-78c4-4514-a6e6-7d7f53abe9b3", + "epoch_millis": 1646833076345, + "request": { + "arango/Current/Collections/_system/10009/s10020": { + "op": "set", + "new": { + "error": false, + "errorMessage": "", + "errorNum": 0, + "indexes": [ + { + "fields": [ + "_key" + ], + "id": "0", + "name": "primary", + "objectId": "4010038", + "sparse": false, + "type": "primary", + "unique": true + } + ], + "servers": [ + "PRMR-a24adubi", + "PRMR-n92yizyp" + ], + "failoverCandidates": [ + "PRMR-a24adubi", + "PRMR-n92yizyp" + ] + } + }, + "arango/Current/Version": { + "op": "increment" + } + }, + "term": 1, + "timestamp": "2022-03-09 13:37:56 UTC" + }, + { + "_key": "00000000000000000075", + "_rev": "_d05kgG---A", + "clientId": "PRMR-a24adubi:c9cca904-0c00-49e3-8079-f2eb03698c21", + "epoch_millis": 1646833076352, + "request": { + "arango/Current/Collections/_system/10012/s10023": { + "op": "set", + "new": { + "error": false, + "errorMessage": "", + "errorNum": 0, + "indexes": [ + { + "fields": [ + "_key" + ], + "id": "0", + "name": "primary", + "objectId": "4010034", + "sparse": false, + "type": "primary", + "unique": true + } + ], + "servers": [ + "PRMR-a24adubi", + "PRMR-n92yizyp" + ], + "failoverCandidates": [ + "PRMR-a24adubi", + "PRMR-n92yizyp" + ] + } + }, + "arango/Current/Version": { + "op": "increment" + } + }, + "term": 1, + "timestamp": "2022-03-09 13:37:56 UTC" + }, + { + "_key": "00000000000000000076", + "_rev": "_d05kgGK--A", + "clientId": "PRMR-a24adubi:b28ee24f-592b-4905-ae42-04d88e0a6bef", + "epoch_millis": 1646833076355, + "request": { + "arango/Current/Collections/_system/10012/s10023": { + "op": "set", + "new": { + "error": false, + "errorMessage": "", + "errorNum": 0, + "indexes": [ + { + "fields": [ + "_key" + ], + "id": "0", + "name": "primary", + "objectId": "4010034", + "sparse": false, + "type": "primary", + "unique": true + } + ], + "servers": [ + "PRMR-a24adubi", + "PRMR-n92yizyp" + ], + "failoverCandidates": [ + "PRMR-a24adubi", + "PRMR-n92yizyp" + ] + } + }, + "arango/Current/Version": { + "op": "increment" + } + }, + "term": 1, + "timestamp": "2022-03-09 13:37:56 UTC" + }, + { + "_key": "00000000000000000077", + "_rev": "_d05kgHG--_", + "clientId": "PRMR-a24adubi:86eb2963-c116-48af-9422-92025d83b380", + "epoch_millis": 1646833076370, + "request": { + "arango/Current/Collections/_system/10013/s10024": { + "op": "set", + "new": { + "error": false, + "errorMessage": "", + "errorNum": 0, + "indexes": [ + { + "fields": [ + "_key" + ], + "id": "0", + "name": "primary", + "objectId": "4010040", + "sparse": false, + "type": "primary", + "unique": true + } + ], + "servers": [ + "PRMR-a24adubi", + "PRMR-n92yizyp" + ], + "failoverCandidates": [ + "PRMR-a24adubi", + "PRMR-n92yizyp" + ] + } + }, + "arango/Current/Version": { + "op": "increment" + } + }, + "term": 1, + "timestamp": "2022-03-09 13:37:56 UTC" + }, + { + "_key": "00000000000000000078", + "_rev": "_d05kgH2---", + "clientId": "CRDN-g6s8btds:8f845e7e-1bf2-486e-be7d-7bf68442f160", + "epoch_millis": 1646833076382, + "request": { + "arango/Plan/Version": { + "op": "increment" + }, + "arango/Plan/Collections/_system/10003": { + "op": "set", + "new": { + "cacheEnabled": false, + "deleted": false, + "distributeShardsLike": "10001", + "id": "10003", + "indexes": [ + { + "id": "0", + "type": "primary", + "name": "primary", + "fields": [ + "_key" + ], + "unique": true, + "sparse": false + } + ], + "internalValidatorType": 0, + "isDisjoint": false, + "isSmart": false, + "isSmartChild": false, + "isSystem": true, + "keyOptions": { + "allowUserKeys": true, + "type": "traditional" + }, + "minReplicationFactor": 1, + "name": "_graphs", + "numberOfShards": 1, + "replicationFactor": 2, + "schema": null, + "shardKeys": [ + "_key" + ], + "shardingStrategy": "hash", + "shards": { + "s10014": [ + "PRMR-a24adubi", + "PRMR-n92yizyp" + ] + }, + "status": 3, + "statusString": "loaded", + "syncByRevision": true, + "type": 2, + "usesRevisionsAsDocumentIds": true, + "waitForSync": false, + "writeConcern": 1 + } + }, + "arango/Plan/Collections/_system/10004": { + "op": "set", + "new": { + "cacheEnabled": false, + "deleted": false, + "distributeShardsLike": "10001", + "id": "10004", + "indexes": [ + { + "id": "0", + "type": "primary", + "name": "primary", + "fields": [ + "_key" + ], + "unique": true, + "sparse": false + } + ], + "internalValidatorType": 0, + "isDisjoint": false, + "isSmart": false, + "isSmartChild": false, + "isSystem": true, + "keyOptions": { + "allowUserKeys": true, + "type": "traditional" + }, + "minReplicationFactor": 1, + "name": "_statistics", + "numberOfShards": 1, + "replicationFactor": 2, + "schema": null, + "shardKeys": [ + "_key" + ], + "shardingStrategy": "hash", + "shards": { + "s10015": [ + "PRMR-a24adubi", + "PRMR-n92yizyp" + ] + }, + "status": 3, + "statusString": "loaded", + "syncByRevision": true, + "type": 2, + "usesRevisionsAsDocumentIds": true, + "waitForSync": false, + "writeConcern": 1 + } + }, + "arango/Plan/Collections/_system/10005": { + "op": "set", + "new": { + "cacheEnabled": false, + "deleted": false, + "distributeShardsLike": "10001", + "id": "10005", + "indexes": [ + { + "id": "0", + "type": "primary", + "name": "primary", + "fields": [ + "_key" + ], + "unique": true, + "sparse": false + } + ], + "internalValidatorType": 0, + "isDisjoint": false, + "isSmart": false, + "isSmartChild": false, + "isSystem": true, + "keyOptions": { + "allowUserKeys": true, + "type": "traditional" + }, + "minReplicationFactor": 1, + "name": "_statistics15", + "numberOfShards": 1, + "replicationFactor": 2, + "schema": null, + "shardKeys": [ + "_key" + ], + "shardingStrategy": "hash", + "shards": { + "s10016": [ + "PRMR-a24adubi", + "PRMR-n92yizyp" + ] + }, + "status": 3, + "statusString": "loaded", + "syncByRevision": true, + "type": 2, + "usesRevisionsAsDocumentIds": true, + "waitForSync": false, + "writeConcern": 1 + } + }, + "arango/Plan/Collections/_system/10006": { + "op": "set", + "new": { + "cacheEnabled": false, + "deleted": false, + "distributeShardsLike": "10001", + "id": "10006", + "indexes": [ + { + "id": "0", + "type": "primary", + "name": "primary", + "fields": [ + "_key" + ], + "unique": true, + "sparse": false + } + ], + "internalValidatorType": 0, + "isDisjoint": false, + "isSmart": false, + "isSmartChild": false, + "isSystem": true, + "keyOptions": { + "allowUserKeys": true, + "type": "traditional" + }, + "minReplicationFactor": 1, + "name": "_statisticsRaw", + "numberOfShards": 1, + "replicationFactor": 2, + "schema": null, + "shardKeys": [ + "_key" + ], + "shardingStrategy": "hash", + "shards": { + "s10017": [ + "PRMR-a24adubi", + "PRMR-n92yizyp" + ] + }, + "status": 3, + "statusString": "loaded", + "syncByRevision": true, + "type": 2, + "usesRevisionsAsDocumentIds": true, + "waitForSync": false, + "writeConcern": 1 + } + }, + "arango/Plan/Collections/_system/10007": { + "op": "set", + "new": { + "cacheEnabled": false, + "deleted": false, + "distributeShardsLike": "10001", + "id": "10007", + "indexes": [ + { + "id": "0", + "type": "primary", + "name": "primary", + "fields": [ + "_key" + ], + "unique": true, + "sparse": false + } + ], + "internalValidatorType": 0, + "isDisjoint": false, + "isSmart": false, + "isSmartChild": false, + "isSystem": true, + "keyOptions": { + "allowUserKeys": true, + "type": "traditional" + }, + "minReplicationFactor": 1, + "name": "_analyzers", + "numberOfShards": 1, + "replicationFactor": 2, + "schema": null, + "shardKeys": [ + "_key" + ], + "shardingStrategy": "hash", + "shards": { + "s10018": [ + "PRMR-a24adubi", + "PRMR-n92yizyp" + ] + }, + "status": 3, + "statusString": "loaded", + "syncByRevision": true, + "type": 2, + "usesRevisionsAsDocumentIds": true, + "waitForSync": false, + "writeConcern": 1 + } + }, + "arango/Plan/Collections/_system/10008": { + "op": "set", + "new": { + "cacheEnabled": false, + "deleted": false, + "distributeShardsLike": "10001", + "id": "10008", + "indexes": [ + { + "id": "0", + "type": "primary", + "name": "primary", + "fields": [ + "_key" + ], + "unique": true, + "sparse": false + } + ], + "internalValidatorType": 0, + "isDisjoint": false, + "isSmart": false, + "isSmartChild": false, + "isSystem": true, + "keyOptions": { + "allowUserKeys": true, + "type": "traditional" + }, + "minReplicationFactor": 1, + "name": "_aqlfunctions", + "numberOfShards": 1, + "replicationFactor": 2, + "schema": null, + "shardKeys": [ + "_key" + ], + "shardingStrategy": "hash", + "shards": { + "s10019": [ + "PRMR-a24adubi", + "PRMR-n92yizyp" + ] + }, + "status": 3, + "statusString": "loaded", + "syncByRevision": true, + "type": 2, + "usesRevisionsAsDocumentIds": true, + "waitForSync": false, + "writeConcern": 1 + } + }, + "arango/Plan/Collections/_system/10009": { + "op": "set", + "new": { + "cacheEnabled": false, + "deleted": false, + "distributeShardsLike": "10001", + "id": "10009", + "indexes": [ + { + "id": "0", + "type": "primary", + "name": "primary", + "fields": [ + "_key" + ], + "unique": true, + "sparse": false + } + ], + "internalValidatorType": 0, + "isDisjoint": false, + "isSmart": false, + "isSmartChild": false, + "isSystem": true, + "keyOptions": { + "allowUserKeys": true, + "type": "traditional" + }, + "minReplicationFactor": 1, + "name": "_queues", + "numberOfShards": 1, + "replicationFactor": 2, + "schema": null, + "shardKeys": [ + "_key" + ], + "shardingStrategy": "hash", + "shards": { + "s10020": [ + "PRMR-a24adubi", + "PRMR-n92yizyp" + ] + }, + "status": 3, + "statusString": "loaded", + "syncByRevision": true, + "type": 2, + "usesRevisionsAsDocumentIds": true, + "waitForSync": false, + "writeConcern": 1 + } + }, + "arango/Plan/Collections/_system/10010": { + "op": "set", + "new": { + "cacheEnabled": false, + "deleted": false, + "distributeShardsLike": "10001", + "id": "10010", + "indexes": [ + { + "id": "0", + "type": "primary", + "name": "primary", + "fields": [ + "_key" + ], + "unique": true, + "sparse": false + } + ], + "internalValidatorType": 0, + "isDisjoint": false, + "isSmart": false, + "isSmartChild": false, + "isSystem": true, + "keyOptions": { + "allowUserKeys": true, + "type": "traditional" + }, + "minReplicationFactor": 1, + "name": "_jobs", + "numberOfShards": 1, + "replicationFactor": 2, + "schema": null, + "shardKeys": [ + "_key" + ], + "shardingStrategy": "hash", + "shards": { + "s10021": [ + "PRMR-a24adubi", + "PRMR-n92yizyp" + ] + }, + "status": 3, + "statusString": "loaded", + "syncByRevision": true, + "type": 2, + "usesRevisionsAsDocumentIds": true, + "waitForSync": false, + "writeConcern": 1 + } + }, + "arango/Plan/Collections/_system/10011": { + "op": "set", + "new": { + "cacheEnabled": false, + "deleted": false, + "distributeShardsLike": "10001", + "id": "10011", + "indexes": [ + { + "id": "0", + "type": "primary", + "name": "primary", + "fields": [ + "_key" + ], + "unique": true, + "sparse": false + } + ], + "internalValidatorType": 0, + "isDisjoint": false, + "isSmart": false, + "isSmartChild": false, + "isSystem": true, + "keyOptions": { + "allowUserKeys": true, + "type": "traditional" + }, + "minReplicationFactor": 1, + "name": "_apps", + "numberOfShards": 1, + "replicationFactor": 2, + "schema": null, + "shardKeys": [ + "_key" + ], + "shardingStrategy": "hash", + "shards": { + "s10022": [ + "PRMR-a24adubi", + "PRMR-n92yizyp" + ] + }, + "status": 3, + "statusString": "loaded", + "syncByRevision": true, + "type": 2, + "usesRevisionsAsDocumentIds": true, + "waitForSync": false, + "writeConcern": 1 + } + }, + "arango/Plan/Collections/_system/10012": { + "op": "set", + "new": { + "cacheEnabled": false, + "deleted": false, + "distributeShardsLike": "10001", + "id": "10012", + "indexes": [ + { + "id": "0", + "type": "primary", + "name": "primary", + "fields": [ + "_key" + ], + "unique": true, + "sparse": false + } + ], + "internalValidatorType": 0, + "isDisjoint": false, + "isSmart": false, + "isSmartChild": false, + "isSystem": true, + "keyOptions": { + "allowUserKeys": true, + "type": "traditional" + }, + "minReplicationFactor": 1, + "name": "_appbundles", + "numberOfShards": 1, + "replicationFactor": 2, + "schema": null, + "shardKeys": [ + "_key" + ], + "shardingStrategy": "hash", + "shards": { + "s10023": [ + "PRMR-a24adubi", + "PRMR-n92yizyp" + ] + }, + "status": 3, + "statusString": "loaded", + "syncByRevision": true, + "type": 2, + "usesRevisionsAsDocumentIds": true, + "waitForSync": false, + "writeConcern": 1 + } + }, + "arango/Plan/Collections/_system/10013": { + "op": "set", + "new": { + "cacheEnabled": false, + "deleted": false, + "distributeShardsLike": "10001", + "id": "10013", + "indexes": [ + { + "id": "0", + "type": "primary", + "name": "primary", + "fields": [ + "_key" + ], + "unique": true, + "sparse": false + } + ], + "internalValidatorType": 0, + "isDisjoint": false, + "isSmart": false, + "isSmartChild": false, + "isSystem": true, + "keyOptions": { + "allowUserKeys": true, + "type": "traditional" + }, + "minReplicationFactor": 1, + "name": "_frontend", + "numberOfShards": 1, + "replicationFactor": 2, + "schema": null, + "shardKeys": [ + "_key" + ], + "shardingStrategy": "hash", + "shards": { + "s10024": [ + "PRMR-a24adubi", + "PRMR-n92yizyp" + ] + }, + "status": 3, + "statusString": "loaded", + "syncByRevision": true, + "type": 2, + "usesRevisionsAsDocumentIds": true, + "waitForSync": false, + "writeConcern": 1 + } + } + }, + "term": 1, + "timestamp": "2022-03-09 13:37:56 UTC" + }, + { + "_key": "00000000000000000079", + "_rev": "_d05kgIy--A", + "clientId": "CRDN-g6s8btds:baaf4d26-e67a-4f6f-acf8-eb6c9ac80203", + "epoch_millis": 1646833076397, + "request": { + "arango/Plan/Collections/_system/10001/indexes": { + "op": "push", + "new": { + "deduplicate": true, + "estimates": false, + "fields": [ + "user" + ], + "inBackground": false, + "name": "idx_1726829639915012096", + "sparse": true, + "type": "hash", + "unique": true, + "isBuilding": true, + "id": "10025" + } + }, + "arango/Plan/Version": { + "op": "increment" + } + }, + "term": 1, + "timestamp": "2022-03-09 13:37:56 UTC" + }, + { + "_key": "00000000000000000080", + "_rev": "_d05kgJa--A", + "clientId": "PRMR-a24adubi:08c94d1d-91dd-489b-8b55-25b5c52698b3", + "epoch_millis": 1646833076407, + "request": { + "arango/Current/Collections/_system/10001/s10002": { + "op": "set", + "new": { + "error": false, + "errorMessage": "", + "errorNum": 0, + "indexes": [ + { + "fields": [ + "_key" + ], + "id": "0", + "name": "primary", + "objectId": "4010002", + "sparse": false, + "type": "primary", + "unique": true + }, + { + "deduplicate": true, + "estimates": true, + "fields": [ + "user" + ], + "id": "10025", + "name": "idx_1726829639915012096", + "objectId": "4010124", + "sparse": true, + "type": "hash", + "unique": true + } + ], + "servers": [ + "PRMR-a24adubi", + "PRMR-n92yizyp" + ], + "failoverCandidates": [ + "PRMR-a24adubi", + "PRMR-n92yizyp" + ] + } + }, + "arango/Current/Version": { + "op": "increment" + } + }, + "term": 1, + "timestamp": "2022-03-09 13:37:56 UTC" + }, + { + "_key": "00000000000000000081", + "_rev": "_d05kgKK--E", + "clientId": "CRDN-g6s8btds:cd87d8b8-fb6a-4a66-b5d4-6514e9e5eed2", + "epoch_millis": 1646833076419, + "request": { + "arango/Plan/Collections/_system/10001/indexes": { + "op": "replace", + "new": { + "deduplicate": true, + "estimates": false, + "fields": [ + "user" + ], + "id": "10025", + "inBackground": false, + "name": "idx_1726829639915012096", + "sparse": true, + "type": "hash", + "unique": true + }, + "val": { + "deduplicate": true, + "estimates": false, + "fields": [ + "user" + ], + "inBackground": false, + "name": "idx_1726829639915012096", + "sparse": true, + "type": "hash", + "unique": true, + "isBuilding": true, + "id": "10025" + } + }, + "arango/Plan/Version": { + "op": "increment" + } + }, + "term": 1, + "timestamp": "2022-03-09 13:37:56 UTC" + }, + { + "_key": "00000000000000000082", + "_rev": "_d05kgKW---", + "clientId": "PRMR-a24adubi:5eaa4da0-961b-4161-8797-6a29a22c1160", + "epoch_millis": 1646833076422, + "request": { + "arango/Current/Collections/_system/10001/s10002": { + "op": "set", + "new": { + "error": false, + "errorMessage": "", + "errorNum": 0, + "indexes": [ + { + "fields": [ + "_key" + ], + "id": "0", + "name": "primary", + "objectId": "4010002", + "sparse": false, + "type": "primary", + "unique": true + }, + { + "deduplicate": true, + "estimates": true, + "fields": [ + "user" + ], + "id": "10025", + "name": "idx_1726829639915012096", + "objectId": "4010124", + "sparse": true, + "type": "hash", + "unique": true + } + ], + "servers": [ + "PRMR-a24adubi", + "PRMR-n92yizyp" + ], + "failoverCandidates": [ + "PRMR-a24adubi", + "PRMR-n92yizyp" + ] + } + }, + "arango/Current/Version": { + "op": "increment" + } + }, + "term": 1, + "timestamp": "2022-03-09 13:37:56 UTC" + }, + { + "_key": "00000000000000000083", + "_rev": "_d05kgK6--A", + "clientId": "CRDN-g6s8btds:3848cadc-7988-481c-b8ad-90630211cc7f", + "epoch_millis": 1646833076431, + "request": { + "arango/Plan/Collections/_system/10004/indexes": { + "op": "push", + "new": { + "deduplicate": true, + "estimates": false, + "fields": [ + "time" + ], + "inBackground": false, + "name": "idx_1726829639950663681", + "sparse": false, + "type": "skiplist", + "unique": false, + "isBuilding": true, + "id": "10026" + } + }, + "arango/Plan/Version": { + "op": "increment" + } + }, + "term": 1, + "timestamp": "2022-03-09 13:37:56 UTC" + }, + { + "_key": "00000000000000000084", + "_rev": "_d05kgLe--A", + "clientId": "PRMR-a24adubi:74f84a49-1807-4ea0-aa99-e31a19eeed38", + "epoch_millis": 1646833076440, + "request": { + "arango/Current/Collections/_system/10004/s10015": { + "op": "set", + "new": { + "error": false, + "errorMessage": "", + "errorNum": 0, + "indexes": [ + { + "fields": [ + "_key" + ], + "id": "0", + "name": "primary", + "objectId": "4010030", + "sparse": false, + "type": "primary", + "unique": true + }, + { + "deduplicate": true, + "estimates": false, + "fields": [ + "time" + ], + "id": "10026", + "name": "idx_1726829639950663681", + "objectId": "4010134", + "sparse": false, + "type": "skiplist", + "unique": false + } + ], + "servers": [ + "PRMR-a24adubi", + "PRMR-n92yizyp" + ], + "failoverCandidates": [ + "PRMR-a24adubi", + "PRMR-n92yizyp" + ] + } + }, + "arango/Current/Version": { + "op": "increment" + } + }, + "term": 1, + "timestamp": "2022-03-09 13:37:56 UTC" + }, + { + "_key": "00000000000000000085", + "_rev": "_d05kgM---A", + "clientId": "CRDN-g6s8btds:11fea306-25b0-4d0c-a460-4ec9dfaf6332", + "epoch_millis": 1646833076448, + "request": { + "arango/Plan/Collections/_system/10004/indexes": { + "op": "replace", + "new": { + "deduplicate": true, + "estimates": false, + "fields": [ + "time" + ], + "id": "10026", + "inBackground": false, + "name": "idx_1726829639950663681", + "sparse": false, + "type": "skiplist", + "unique": false + }, + "val": { + "deduplicate": true, + "estimates": false, + "fields": [ + "time" + ], + "inBackground": false, + "name": "idx_1726829639950663681", + "sparse": false, + "type": "skiplist", + "unique": false, + "isBuilding": true, + "id": "10026" + } + }, + "arango/Plan/Version": { + "op": "increment" + } + }, + "term": 1, + "timestamp": "2022-03-09 13:37:56 UTC" + }, + { + "_key": "00000000000000000086", + "_rev": "_d05kgMG--A", + "clientId": "PRMR-a24adubi:0782f771-b229-4285-a87b-5b2096fb9127", + "epoch_millis": 1646833076450, + "request": { + "arango/Current/Collections/_system/10004/s10015": { + "op": "set", + "new": { + "error": false, + "errorMessage": "", + "errorNum": 0, + "indexes": [ + { + "fields": [ + "_key" + ], + "id": "0", + "name": "primary", + "objectId": "4010030", + "sparse": false, + "type": "primary", + "unique": true + }, + { + "deduplicate": true, + "estimates": false, + "fields": [ + "time" + ], + "id": "10026", + "name": "idx_1726829639950663681", + "objectId": "4010134", + "sparse": false, + "type": "skiplist", + "unique": false + } + ], + "servers": [ + "PRMR-a24adubi", + "PRMR-n92yizyp" + ], + "failoverCandidates": [ + "PRMR-a24adubi", + "PRMR-n92yizyp" + ] + } + }, + "arango/Current/Version": { + "op": "increment" + } + }, + "term": 1, + "timestamp": "2022-03-09 13:37:56 UTC" + }, + { + "_key": "00000000000000000087", + "_rev": "_d05kgMe---", + "clientId": "CRDN-g6s8btds:d379e56f-b918-4963-8a68-cbb63e3555c0", + "epoch_millis": 1646833076456, + "request": { + "arango/Plan/Collections/_system/10005/indexes": { + "op": "push", + "new": { + "deduplicate": true, + "estimates": false, + "fields": [ + "time" + ], + "inBackground": false, + "name": "idx_1726829639976878081", + "sparse": false, + "type": "skiplist", + "unique": false, + "isBuilding": true, + "id": "10027" + } + }, + "arango/Plan/Version": { + "op": "increment" + } + }, + "term": 1, + "timestamp": "2022-03-09 13:37:56 UTC" + }, + { + "_key": "00000000000000000088", + "_rev": "_d05kgN---_", + "clientId": "PRMR-a24adubi:42ba892c-a25f-447b-b870-47e174c03c00", + "epoch_millis": 1646833076464, + "request": { + "arango/Current/Collections/_system/10005/s10016": { + "op": "set", + "new": { + "error": false, + "errorMessage": "", + "errorNum": 0, + "indexes": [ + { + "fields": [ + "_key" + ], + "id": "0", + "name": "primary", + "objectId": "4010036", + "sparse": false, + "type": "primary", + "unique": true + }, + { + "deduplicate": true, + "estimates": false, + "fields": [ + "time" + ], + "id": "10027", + "name": "idx_1726829639976878081", + "objectId": "4010144", + "sparse": false, + "type": "skiplist", + "unique": false + } + ], + "servers": [ + "PRMR-a24adubi", + "PRMR-n92yizyp" + ], + "failoverCandidates": [ + "PRMR-a24adubi", + "PRMR-n92yizyp" + ] + } + }, + "arango/Current/Version": { + "op": "increment" + } + }, + "term": 1, + "timestamp": "2022-03-09 13:37:56 UTC" + }, + { + "_key": "00000000000000000089", + "_rev": "_d05kgNe--B", + "clientId": "CRDN-g6s8btds:2f285d06-c30f-4e66-977b-e76ee63abc4c", + "epoch_millis": 1646833076472, + "request": { + "arango/Plan/Collections/_system/10005/indexes": { + "op": "replace", + "new": { + "deduplicate": true, + "estimates": false, + "fields": [ + "time" + ], + "id": "10027", + "inBackground": false, + "name": "idx_1726829639976878081", + "sparse": false, + "type": "skiplist", + "unique": false + }, + "val": { + "deduplicate": true, + "estimates": false, + "fields": [ + "time" + ], + "inBackground": false, + "name": "idx_1726829639976878081", + "sparse": false, + "type": "skiplist", + "unique": false, + "isBuilding": true, + "id": "10027" + } + }, + "arango/Plan/Version": { + "op": "increment" + } + }, + "term": 1, + "timestamp": "2022-03-09 13:37:56 UTC" + }, + { + "_key": "00000000000000000090", + "_rev": "_d05kgNu--A", + "clientId": "PRMR-a24adubi:5b9e1622-0acd-4afd-b34b-b988988e0069", + "epoch_millis": 1646833076476, + "request": { + "arango/Current/Collections/_system/10005/s10016": { + "op": "set", + "new": { + "error": false, + "errorMessage": "", + "errorNum": 0, + "indexes": [ + { + "fields": [ + "_key" + ], + "id": "0", + "name": "primary", + "objectId": "4010036", + "sparse": false, + "type": "primary", + "unique": true + }, + { + "deduplicate": true, + "estimates": false, + "fields": [ + "time" + ], + "id": "10027", + "name": "idx_1726829639976878081", + "objectId": "4010144", + "sparse": false, + "type": "skiplist", + "unique": false + } + ], + "servers": [ + "PRMR-a24adubi", + "PRMR-n92yizyp" + ], + "failoverCandidates": [ + "PRMR-a24adubi", + "PRMR-n92yizyp" + ] + } + }, + "arango/Current/Version": { + "op": "increment" + } + }, + "term": 1, + "timestamp": "2022-03-09 13:37:56 UTC" + }, + { + "_key": "00000000000000000091", + "_rev": "_d05kgOO--_", + "clientId": "CRDN-g6s8btds:27e4b41d-10d9-434b-9dc9-e29903b7fe0f", + "epoch_millis": 1646833076484, + "request": { + "arango/Plan/Collections/_system/10006/indexes": { + "op": "push", + "new": { + "deduplicate": true, + "estimates": false, + "fields": [ + "time" + ], + "inBackground": false, + "name": "idx_1726829640006238208", + "sparse": false, + "type": "skiplist", + "unique": false, + "isBuilding": true, + "id": "10028" + } + }, + "arango/Plan/Version": { + "op": "increment" + } + }, + "term": 1, + "timestamp": "2022-03-09 13:37:56 UTC" + }, + { + "_key": "00000000000000000092", + "_rev": "_d05kgO6---", + "clientId": "PRMR-a24adubi:01d1f055-5f39-4f26-8fed-466ebf9d340f", + "epoch_millis": 1646833076495, + "request": { + "arango/Current/Collections/_system/10006/s10017": { + "op": "set", + "new": { + "error": false, + "errorMessage": "", + "errorNum": 0, + "indexes": [ + { + "fields": [ + "_key" + ], + "id": "0", + "name": "primary", + "objectId": "4010028", + "sparse": false, + "type": "primary", + "unique": true + }, + { + "deduplicate": true, + "estimates": false, + "fields": [ + "time" + ], + "id": "10028", + "name": "idx_1726829640006238208", + "objectId": "4010154", + "sparse": false, + "type": "skiplist", + "unique": false + } + ], + "servers": [ + "PRMR-a24adubi", + "PRMR-n92yizyp" + ], + "failoverCandidates": [ + "PRMR-a24adubi", + "PRMR-n92yizyp" + ] + } + }, + "arango/Current/Version": { + "op": "increment" + } + }, + "term": 1, + "timestamp": "2022-03-09 13:37:56 UTC" + }, + { + "_key": "00000000000000000093", + "_rev": "_d05kgPm---", + "clientId": "CRDN-g6s8btds:c33d599e-4105-4f55-a878-1644a6b61234", + "epoch_millis": 1646833076506, + "request": { + "arango/Plan/Collections/_system/10006/indexes": { + "op": "replace", + "new": { + "deduplicate": true, + "estimates": false, + "fields": [ + "time" + ], + "id": "10028", + "inBackground": false, + "name": "idx_1726829640006238208", + "sparse": false, + "type": "skiplist", + "unique": false + }, + "val": { + "deduplicate": true, + "estimates": false, + "fields": [ + "time" + ], + "inBackground": false, + "name": "idx_1726829640006238208", + "sparse": false, + "type": "skiplist", + "unique": false, + "isBuilding": true, + "id": "10028" + } + }, + "arango/Plan/Version": { + "op": "increment" + } + }, + "term": 1, + "timestamp": "2022-03-09 13:37:56 UTC" + }, + { + "_key": "00000000000000000094", + "_rev": "_d05kgQ----", + "clientId": "PRMR-a24adubi:b18fa15d-446c-4cbc-9f57-942c619c3a89", + "epoch_millis": 1646833076512, + "request": { + "arango/Current/Collections/_system/10006/s10017": { + "op": "set", + "new": { + "error": false, + "errorMessage": "", + "errorNum": 0, + "indexes": [ + { + "fields": [ + "_key" + ], + "id": "0", + "name": "primary", + "objectId": "4010028", + "sparse": false, + "type": "primary", + "unique": true + }, + { + "deduplicate": true, + "estimates": false, + "fields": [ + "time" + ], + "id": "10028", + "name": "idx_1726829640006238208", + "objectId": "4010154", + "sparse": false, + "type": "skiplist", + "unique": false + } + ], + "servers": [ + "PRMR-a24adubi", + "PRMR-n92yizyp" + ], + "failoverCandidates": [ + "PRMR-a24adubi", + "PRMR-n92yizyp" + ] + } + }, + "arango/Current/Version": { + "op": "increment" + } + }, + "term": 1, + "timestamp": "2022-03-09 13:37:56 UTC" + }, + { + "_key": "00000000000000000095", + "_rev": "_d05kgQO--_", + "clientId": "CRDN-cg5kjymo:0ac3495f-5595-4874-b9a4-157f4ca0a86d", + "epoch_millis": 1646833076516, + "request": { + "arango/Current/Version": { + "op": "increment" + } + }, + "term": 1, + "timestamp": "2022-03-09 13:37:56 UTC" + }, + { + "_key": "00000000000000000096", + "_rev": "_d05kgQy---", + "clientId": "CRDN-g6s8btds:525d48ae-4946-4c1e-8d48-0c3bfd0cd71c", + "epoch_millis": 1646833076525, + "request": { + "arango/Plan/Collections/_system/10011/indexes": { + "op": "push", + "new": { + "deduplicate": true, + "estimates": false, + "fields": [ + "mount" + ], + "inBackground": false, + "name": "idx_1726829640049229824", + "sparse": true, + "type": "hash", + "unique": true, + "isBuilding": true, + "id": "10029" + } + }, + "arango/Plan/Version": { + "op": "increment" + } + }, + "term": 1, + "timestamp": "2022-03-09 13:37:56 UTC" + }, + { + "_key": "00000000000000000097", + "_rev": "_d05kgRW--A", + "clientId": "PRMR-a24adubi:091bdfb7-67bb-49fb-8593-a68750b614df", + "epoch_millis": 1646833076534, + "request": { + "arango/Current/Collections/_system/10011/s10022": { + "op": "set", + "new": { + "error": false, + "errorMessage": "", + "errorNum": 0, + "indexes": [ + { + "fields": [ + "_key" + ], + "id": "0", + "name": "primary", + "objectId": "4010042", + "sparse": false, + "type": "primary", + "unique": true + }, + { + "deduplicate": true, + "estimates": true, + "fields": [ + "mount" + ], + "id": "10029", + "name": "idx_1726829640049229824", + "objectId": "4010164", + "sparse": true, + "type": "hash", + "unique": true + } + ], + "servers": [ + "PRMR-a24adubi", + "PRMR-n92yizyp" + ], + "failoverCandidates": [ + "PRMR-a24adubi", + "PRMR-n92yizyp" + ] + } + }, + "arango/Current/Version": { + "op": "increment" + } + }, + "term": 1, + "timestamp": "2022-03-09 13:37:56 UTC" + }, + { + "_key": "00000000000000000098", + "_rev": "_d05kgRy--A", + "clientId": "CRDN-g6s8btds:86673a27-41c9-4b6e-a9f7-0c3d7c8968b0", + "epoch_millis": 1646833076541, + "request": { + "arango/Plan/Collections/_system/10011/indexes": { + "op": "replace", + "new": { + "deduplicate": true, + "estimates": false, + "fields": [ + "mount" + ], + "id": "10029", + "inBackground": false, + "name": "idx_1726829640049229824", + "sparse": true, + "type": "hash", + "unique": true + }, + "val": { + "deduplicate": true, + "estimates": false, + "fields": [ + "mount" + ], + "inBackground": false, + "name": "idx_1726829640049229824", + "sparse": true, + "type": "hash", + "unique": true, + "isBuilding": true, + "id": "10029" + } + }, + "arango/Plan/Version": { + "op": "increment" + } + }, + "term": 1, + "timestamp": "2022-03-09 13:37:56 UTC" + }, + { + "_key": "00000000000000000099", + "_rev": "_d05kgS---_", + "clientId": "PRMR-a24adubi:678bd6d7-2d89-4c9b-bb01-65f4e6965412", + "epoch_millis": 1646833076544, + "request": { + "arango/Current/Collections/_system/10011/s10022": { + "op": "set", + "new": { + "error": false, + "errorMessage": "", + "errorNum": 0, + "indexes": [ + { + "fields": [ + "_key" + ], + "id": "0", + "name": "primary", + "objectId": "4010042", + "sparse": false, + "type": "primary", + "unique": true + }, + { + "deduplicate": true, + "estimates": true, + "fields": [ + "mount" + ], + "id": "10029", + "name": "idx_1726829640049229824", + "objectId": "4010164", + "sparse": true, + "type": "hash", + "unique": true + } + ], + "servers": [ + "PRMR-a24adubi", + "PRMR-n92yizyp" + ], + "failoverCandidates": [ + "PRMR-a24adubi", + "PRMR-n92yizyp" + ] + } + }, + "arango/Current/Version": { + "op": "increment" + } + }, + "term": 1, + "timestamp": "2022-03-09 13:37:56 UTC" + }, + { + "_key": "00000000000000000100", + "_rev": "_d05kgSO--C", + "clientId": "CRDN-g6s8btds:e2879a07-7ee2-4fa8-aaf5-b49fb5e7ab9a", + "epoch_millis": 1646833076548, + "request": { + "arango/Plan/Collections/_system/10010/indexes": { + "op": "push", + "new": { + "deduplicate": true, + "estimates": false, + "fields": [ + "queue", + "status", + "delayUntil" + ], + "inBackground": false, + "name": "idx_1726829640074395648", + "sparse": false, + "type": "skiplist", + "unique": false, + "isBuilding": true, + "id": "10030" + } + }, + "arango/Plan/Version": { + "op": "increment" + } + }, + "term": 1, + "timestamp": "2022-03-09 13:37:56 UTC" + }, + { + "_key": "00000000000000000101", + "_rev": "_d05kgTC---", + "clientId": "PRMR-a24adubi:8a81717b-64df-44dd-bb57-11624f1e22f4", + "epoch_millis": 1646833076561, + "request": { + "arango/Current/Collections/_system/10010/s10021": { + "op": "set", + "new": { + "error": false, + "errorMessage": "", + "errorNum": 0, + "indexes": [ + { + "fields": [ + "_key" + ], + "id": "0", + "name": "primary", + "objectId": "4010032", + "sparse": false, + "type": "primary", + "unique": true + }, + { + "deduplicate": true, + "estimates": false, + "fields": [ + "queue", + "status", + "delayUntil" + ], + "id": "10030", + "name": "idx_1726829640074395648", + "objectId": "4010174", + "sparse": false, + "type": "skiplist", + "unique": false + } + ], + "servers": [ + "PRMR-a24adubi", + "PRMR-n92yizyp" + ], + "failoverCandidates": [ + "PRMR-a24adubi", + "PRMR-n92yizyp" + ] + } + }, + "arango/Current/Version": { + "op": "increment" + } + }, + "term": 1, + "timestamp": "2022-03-09 13:37:56 UTC" + }, + { + "_key": "00000000000000000102", + "_rev": "_d05kgTe--C", + "clientId": "CRDN-g6s8btds:0a191d4e-6dc4-4592-a9be-c0065ba64dd6", + "epoch_millis": 1646833076568, + "request": { + "arango/Plan/Collections/_system/10010/indexes": { + "op": "replace", + "new": { + "deduplicate": true, + "estimates": false, + "fields": [ + "queue", + "status", + "delayUntil" + ], + "id": "10030", + "inBackground": false, + "name": "idx_1726829640074395648", + "sparse": false, + "type": "skiplist", + "unique": false + }, + "val": { + "deduplicate": true, + "estimates": false, + "fields": [ + "queue", + "status", + "delayUntil" + ], + "inBackground": false, + "name": "idx_1726829640074395648", + "sparse": false, + "type": "skiplist", + "unique": false, + "isBuilding": true, + "id": "10030" + } + }, + "arango/Plan/Version": { + "op": "increment" + } + }, + "term": 1, + "timestamp": "2022-03-09 13:37:56 UTC" + }, + { + "_key": "00000000000000000103", + "_rev": "_d05kgU---A", + "clientId": "CRDN-g6s8btds:b32e5c9d-3559-4b04-93e0-b2eb1fd27b06", + "epoch_millis": 1646833076576, + "request": { + "arango/Plan/Collections/_system/10010/indexes": { + "op": "push", + "new": { + "deduplicate": true, + "estimates": false, + "fields": [ + "status", + "queue", + "delayUntil" + ], + "inBackground": false, + "name": "idx_1726829640102707201", + "sparse": false, + "type": "skiplist", + "unique": false, + "isBuilding": true, + "id": "10031" + } + }, + "arango/Plan/Version": { + "op": "increment" + } + }, + "term": 1, + "timestamp": "2022-03-09 13:37:56 UTC" + }, + { + "_key": "00000000000000000104", + "_rev": "_d05kgUi---", + "clientId": "PRMR-a24adubi:eebea08f-84d2-4ec2-99d7-6a2a3218b2dd", + "epoch_millis": 1646833076584, + "request": { + "arango/Current/Collections/_system/10010/s10021": { + "op": "set", + "new": { + "error": false, + "errorMessage": "", + "errorNum": 0, + "indexes": [ + { + "fields": [ + "_key" + ], + "id": "0", + "name": "primary", + "objectId": "4010032", + "sparse": false, + "type": "primary", + "unique": true + }, + { + "deduplicate": true, + "estimates": false, + "fields": [ + "queue", + "status", + "delayUntil" + ], + "id": "10030", + "name": "idx_1726829640074395648", + "objectId": "4010174", + "sparse": false, + "type": "skiplist", + "unique": false + }, + { + "deduplicate": true, + "estimates": false, + "fields": [ + "status", + "queue", + "delayUntil" + ], + "id": "10031", + "name": "idx_1726829640102707201", + "objectId": "4010186", + "sparse": false, + "type": "skiplist", + "unique": false + } + ], + "servers": [ + "PRMR-a24adubi", + "PRMR-n92yizyp" + ], + "failoverCandidates": [ + "PRMR-a24adubi", + "PRMR-n92yizyp" + ] + } + }, + "arango/Current/Version": { + "op": "increment" + } + }, + "term": 1, + "timestamp": "2022-03-09 13:37:56 UTC" + }, + { + "_key": "00000000000000000105", + "_rev": "_d05kgU6--A", + "clientId": "CRDN-g6s8btds:1b776041-c85b-475e-b26d-9be473c46240", + "epoch_millis": 1646833076591, + "request": { + "arango/Plan/Collections/_system/10010/indexes": { + "op": "replace", + "new": { + "deduplicate": true, + "estimates": false, + "fields": [ + "status", + "queue", + "delayUntil" + ], + "id": "10031", + "inBackground": false, + "name": "idx_1726829640102707201", + "sparse": false, + "type": "skiplist", + "unique": false + }, + "val": { + "deduplicate": true, + "estimates": false, + "fields": [ + "status", + "queue", + "delayUntil" + ], + "inBackground": false, + "name": "idx_1726829640102707201", + "sparse": false, + "type": "skiplist", + "unique": false, + "isBuilding": true, + "id": "10031" + } + }, + "arango/Plan/Version": { + "op": "increment" + } + }, + "term": 1, + "timestamp": "2022-03-09 13:37:56 UTC" + }, + { + "_key": "00000000000000000106", + "_rev": "_d05kgVi--D", + "clientId": "CRDN-g6s8btds:11582ff5-64fa-4e09-81c8-caeb5f6b9a17", + "epoch_millis": 1646833076601, + "request": { + "arango/Current/Foxxmaster": { + "op": "set", + "new": "CRDN-g6s8btds" + } + }, + "term": 1, + "timestamp": "2022-03-09 13:37:56 UTC" + }, + { + "_key": "00000000000000000107", + "_rev": "_d05kgW---A", + "clientId": "CRDN-g6s8btds:ff27a79c-9d73-4087-b24e-41e3969c7e8e", + "epoch_millis": 1646833076608, + "request": { + "arango/Current/Version": { + "op": "increment" + } + }, + "term": 1, + "timestamp": "2022-03-09 13:37:56 UTC" + }, + { + "_key": "00000000000000000108", + "_rev": "_d05kgXe--A", + "clientId": "PRMR-a24adubi:3ad1de74-0b8a-4fa3-85d6-84bd018605f7", + "epoch_millis": 1646833076632, + "request": { + "arango/Current/Collections/_system/10001/s10002": { + "op": "set", + "new": { + "error": false, + "errorMessage": "", + "errorNum": 0, + "indexes": [ + { + "fields": [ + "_key" + ], + "id": "0", + "name": "primary", + "objectId": "4010002", + "sparse": false, + "type": "primary", + "unique": true + }, + { + "deduplicate": true, + "estimates": true, + "fields": [ + "user" + ], + "id": "10025", + "name": "idx_1726829639915012096", + "objectId": "4010124", + "sparse": true, + "type": "hash", + "unique": true + } + ], + "servers": [ + "PRMR-a24adubi", + "PRMR-n92yizyp" + ], + "failoverCandidates": [ + "PRMR-a24adubi", + "PRMR-n92yizyp" + ] + } + }, + "arango/Current/Version": { + "op": "increment" + } + }, + "term": 1, + "timestamp": "2022-03-09 13:37:56 UTC" + }, + { + "_key": "00000000000000000109", + "_rev": "_d05kgYa--_", + "clientId": "CRDN-g6s8btds:8bfe8ab9-220e-4097-a093-e0b4cd7ef0f1", + "epoch_millis": 1646833076647, + "request": { + "arango/Sync/UserVersion": { + "op": "increment" + } + }, + "term": 1, + "timestamp": "2022-03-09 13:37:56 UTC" + }, + { + "_key": "00000000000000000110", + "_rev": "_d05kgY2--B", + "clientId": "CRDN-g6s8btds:41c7615d-15bf-425e-a4d3-25e97d59ea98", + "epoch_millis": 1646833076654, + "request": { + "arango/Bootstrap": { + "op": "set", + "new": "CRDN-g6s8btds: done" + } + }, + "term": 1, + "timestamp": "2022-03-09 13:37:56 UTC" + }, + { + "_key": "00000000000000000111", + "_rev": "_d05kgZO--B", + "clientId": "CRDN-g6s8btds:baa2ecde-d929-4db1-8e00-42e311b467a5", + "epoch_millis": 1646833076660, + "request": { + "arango/ClusterUpgradeVersion": { + "op": "set", + "new": 30900 + } + }, + "term": 1, + "timestamp": "2022-03-09 13:37:56 UTC" + }, + { + "_key": "00000000000000000112", + "_rev": "_d05kgZ6--_", + "clientId": "", + "epoch_millis": 1646833076671, + "request": { + "/arango/Supervision/Health/PRMR-n92yizyp": { + "ShortName": "DBServer0002", + "Endpoint": "ssl://cluster-uexabr-dbserver-n92yizyp.cluster-uexabr-int.3dcd4eac-c549-41b6-bc8f-40e11983ebd2.svc:8529", + "Host": "ip-10-0-101-138.eu-central-1.compute.internal", + "SyncStatus": "SERVING", + "Status": "GOOD", + "Version": "3.9.0", + "Engine": "rocksdb", + "Timestamp": "2022-03-09T13:37:56Z", + "SyncTime": "2022-03-09T13:37:55Z", + "LastAckedTime": "2022-03-09T13:37:56Z" + } + }, + "term": 1, + "timestamp": "2022-03-09 13:37:56 UTC" + }, + { + "_key": "00000000000000000113", + "_rev": "_d05kgae---", + "clientId": "CRDN-g6s8btds:caa89917-ae5e-41a1-89b8-68e6bdaa33c6", + "epoch_millis": 1646833076679, + "request": { + "arango/SystemCollectionsCreated": { + "op": "set", + "new": true + } + }, + "term": 1, + "timestamp": "2022-03-09 13:37:56 UTC" + }, + { + "_key": "00000000000000000114", + "_rev": "_d05kgam--_", + "clientId": "", + "epoch_millis": 1646833076682, + "request": { + "/arango/Supervision/Health/PRMR-a24adubi": { + "ShortName": "DBServer0001", + "Endpoint": "ssl://cluster-uexabr-dbserver-a24adubi.cluster-uexabr-int.3dcd4eac-c549-41b6-bc8f-40e11983ebd2.svc:8529", + "Host": "ip-10-0-101-183.eu-central-1.compute.internal", + "SyncStatus": "SERVING", + "Status": "GOOD", + "Version": "3.9.0", + "Engine": "rocksdb", + "Timestamp": "2022-03-09T13:37:56Z", + "SyncTime": "2022-03-09T13:37:56Z", + "LastAckedTime": "2022-03-09T13:37:56Z" + } + }, + "term": 1, + "timestamp": "2022-03-09 13:37:56 UTC" + }, + { + "_key": "00000000000000000115", + "_rev": "_d05kgg6--_", + "clientId": "PRMR-a24adubi:c420fd40-ca20-47a4-b8b9-4864f808e309", + "epoch_millis": 1646833076783, + "request": { + "arango/Current/Collections/_system/10009/s10020": { + "op": "set", + "new": { + "error": false, + "errorMessage": "", + "errorNum": 0, + "indexes": [ + { + "fields": [ + "_key" + ], + "id": "0", + "name": "primary", + "objectId": "4010038", + "sparse": false, + "type": "primary", + "unique": true + } + ], + "servers": [ + "PRMR-a24adubi", + "PRMR-n92yizyp" + ], + "failoverCandidates": [ + "PRMR-a24adubi", + "PRMR-n92yizyp" + ] + } + }, + "arango/Current/Version": { + "op": "increment" + } + }, + "term": 1, + "timestamp": "2022-03-09 13:37:56 UTC" + }, + { + "_key": "00000000000000000116", + "_rev": "_d05khgG---", + "clientId": "CRDN-g6s8btds:f42a35b7-2735-4a54-bce0-55d2b2094661", + "epoch_millis": 1646833077794, + "request": { + "arango/Current/FoxxmasterQueueupdate": { + "op": "set", + "new": false + } + }, + "term": 1, + "timestamp": "2022-03-09 13:37:57 UTC" + }, + { + "_key": "00000000000000000117", + "_rev": "_d05khgi--A", + "clientId": "CRDN-g6s8btds:7c855ded-4fd5-4fca-b4f4-9096c4bc8058", + "epoch_millis": 1646833077801, + "request": { + "arango/Current/Version": { + "op": "increment" + } + }, + "term": 1, + "timestamp": "2022-03-09 13:37:57 UTC" + }, + { + "_key": "00000000000000000118", + "_rev": "_d05khhO--A", + "clientId": "CRDN-cg5kjymo:fdffd08f-e0bf-477e-a6b2-5d97f2976e0a", + "epoch_millis": 1646833077812, + "request": { + "arango/Sync/LatestID": { + "op": "set", + "new": 8010001 + } + }, + "term": 1, + "timestamp": "2022-03-09 13:37:57 UTC" + }, + { + "_key": "00000000000000000119", + "_rev": "_d05kiIa---", + "clientId": "CRDN-cg5kjymo:17009b24-4d48-40a6-86dc-a42331c0ee08", + "epoch_millis": 1646833078439, + "request": { + "/arango/Target/NumberOfCoordinators": { + "op": "set", + "new": 3 + }, + "/arango/Target/NumberOfDBServers": { + "op": "set", + "new": 3 + } + }, + "term": 1, + "timestamp": "2022-03-09 13:37:58 UTC" + }, + { + "_key": "00000000000000000120", + "_rev": "_d05kiJq--_", + "clientId": "CRDN-ssc2yc0x:de91abcc-2b25-40ca-8665-17644fb1b9aa", + "epoch_millis": 1646833078459, + "request": { + "arango/Sync/LatestID": { + "op": "set", + "new": 10010001 + } + }, + "term": 1, + "timestamp": "2022-03-09 13:37:58 UTC" + }, + { + "_key": "00000000000000000121", + "_rev": "_d05kkMy---", + "clientId": "PRMR-a24adubi:813f9bc6-cd0a-4ab7-969e-d553221ccb9a", + "epoch_millis": 1646833080556, + "request": { + "arango/Current/Collections/_system/10006/s10017": { + "op": "set", + "new": { + "error": false, + "errorMessage": "", + "errorNum": 0, + "indexes": [ + { + "fields": [ + "_key" + ], + "id": "0", + "name": "primary", + "objectId": "4010028", + "sparse": false, + "type": "primary", + "unique": true + }, + { + "deduplicate": true, + "estimates": false, + "fields": [ + "time" + ], + "id": "10028", + "name": "idx_1726829640006238208", + "objectId": "4010154", + "sparse": false, + "type": "skiplist", + "unique": false + } + ], + "servers": [ + "PRMR-a24adubi", + "PRMR-n92yizyp" + ], + "failoverCandidates": [ + "PRMR-a24adubi", + "PRMR-n92yizyp" + ] + } + }, + "arango/Current/Version": { + "op": "increment" + } + }, + "term": 1, + "timestamp": "2022-03-09 13:38:00 UTC" + }, + { + "_key": "00000000000000000122", + "_rev": "_d05knFO--A", + "clientId": "PRMR-rg51yv2h:b177a6c5-303f-4fde-af37-ac0c7de8d3b9", + "epoch_millis": 1646833083508, + "request": { + "arango/Plan/DBServers/PRMR-rg51yv2h": { + "op": "set", + "new": "none" + }, + "arango/Plan/Version": { + "op": "increment" + } + }, + "term": 1, + "timestamp": "2022-03-09 13:38:03 UTC" + }, + { + "_key": "00000000000000000123", + "_rev": "_d05knF6--_", + "clientId": "PRMR-rg51yv2h:260404b2-9c92-4b49-b11d-92eaf6505f42", + "epoch_millis": 1646833083519, + "request": { + "arango/Current/DBServers/PRMR-rg51yv2h": { + "op": "set", + "new": "none" + }, + "arango/Current/Version": { + "op": "increment" + } + }, + "term": 1, + "timestamp": "2022-03-09 13:38:03 UTC" + }, + { + "_key": "00000000000000000124", + "_rev": "_d05knGq--A", + "clientId": "PRMR-rg51yv2h:5a38bfe7-1f3e-4ce6-90bd-cb6c019a2477", + "epoch_millis": 1646833083531, + "request": { + "arango/Target/LatestDBServerId": { + "op": "increment" + }, + "arango/Target/MapUniqueToShortID/PRMR-rg51yv2h": { + "op": "set", + "new": { + "TransactionID": 3, + "ShortName": "DBServer0003" + } + } + }, + "term": 1, + "timestamp": "2022-03-09 13:38:03 UTC" + }, + { + "_key": "00000000000000000125", + "_rev": "_d05knHS--A", + "clientId": "PRMR-rg51yv2h:2271b33b-5e97-45dd-8b34-1408d8d94a57", + "epoch_millis": 1646833083541, + "request": { + "arango/Current/ServersRegistered/PRMR-rg51yv2h": { + "op": "set", + "new": { + "endpoint": "ssl://cluster-uexabr-dbserver-rg51yv2h.cluster-uexabr-int.3dcd4eac-c549-41b6-bc8f-40e11983ebd2.svc:8529", + "advertisedEndpoint": "", + "host": "ip-10-0-101-233.eu-central-1.compute.internal", + "version": 30900, + "versionString": "3.9.0", + "engine": "rocksdb", + "extendedNamesDatabases": false, + "timestamp": "2022-03-09T13:38:03Z" + } + }, + "arango/Current/ServersKnown/PRMR-rg51yv2h/rebootId": { + "op": "increment" + }, + "arango/Current/Version": { + "op": "increment" + } + }, + "term": 1, + "timestamp": "2022-03-09 13:38:03 UTC" + }, + { + "_key": "00000000000000000126", + "_rev": "_d05kneG---", + "clientId": "PRMR-rg51yv2h:d89ceac4-7405-45b4-b816-c813838a6fb5", + "epoch_millis": 1646833083906, + "request": { + "arango/Current/Version": { + "op": "increment" + } + }, + "term": 1, + "timestamp": "2022-03-09 13:38:03 UTC" + }, + { + "_key": "00000000000000000127", + "_rev": "_d05kpMO---", + "clientId": "", + "epoch_millis": 1646833085668, + "request": { + "/arango/Supervision/Health/PRMR-rg51yv2h": { + "ShortName": "DBServer0003", + "Endpoint": "ssl://cluster-uexabr-dbserver-rg51yv2h.cluster-uexabr-int.3dcd4eac-c549-41b6-bc8f-40e11983ebd2.svc:8529", + "Host": "ip-10-0-101-233.eu-central-1.compute.internal", + "SyncStatus": "STARTUP", + "Status": "GOOD", + "Version": "3.9.0", + "Engine": "rocksdb", + "Timestamp": "2022-03-09T13:38:05Z", + "SyncTime": "2022-03-09T13:38:03Z", + "LastAckedTime": "2022-03-09T13:38:04Z" + } + }, + "term": 1, + "timestamp": "2022-03-09 13:38:05 UTC" + }, + { + "_key": "00000000000000000128", + "_rev": "_d05kppK---", + "clientId": "PRMR-rg51yv2h:05dad1b3-e271-42b0-85eb-c9398b2eb19a", + "epoch_millis": 1646833086131, + "request": { + "arango/Current/Databases/_system/PRMR-rg51yv2h": { + "op": "set", + "new": { + "error": false, + "errorNum": 0, + "errorMessage": "", + "id": "1", + "name": "_system" + } + }, + "arango/Current/Version": { + "op": "increment" + } + }, + "term": 1, + "timestamp": "2022-03-09 13:38:06 UTC" + }, + { + "_key": "00000000000000000129", + "_rev": "_d05kqKu---", + "clientId": "", + "epoch_millis": 1646833086668, + "request": { + "/arango/Supervision/Health/PRMR-rg51yv2h": { + "ShortName": "DBServer0003", + "Endpoint": "ssl://cluster-uexabr-dbserver-rg51yv2h.cluster-uexabr-int.3dcd4eac-c549-41b6-bc8f-40e11983ebd2.svc:8529", + "Host": "ip-10-0-101-233.eu-central-1.compute.internal", + "SyncStatus": "SERVING", + "Status": "GOOD", + "Version": "3.9.0", + "Engine": "rocksdb", + "Timestamp": "2022-03-09T13:38:06Z", + "SyncTime": "2022-03-09T13:38:06Z", + "LastAckedTime": "2022-03-09T13:38:06Z" + } + }, + "term": 1, + "timestamp": "2022-03-09 13:38:06 UTC" + }, + { + "_key": "00000000000000000130", + "_rev": "_d05ku-y--A", + "clientId": "PRMR-a24adubi:a7b433cd-c4c8-4452-95ad-645e888b53af", + "epoch_millis": 1646833090573, + "request": { + "arango/Current/Collections/_system/10004/s10015": { + "op": "set", + "new": { + "error": false, + "errorMessage": "", + "errorNum": 0, + "indexes": [ + { + "fields": [ + "_key" + ], + "id": "0", + "name": "primary", + "objectId": "4010030", + "sparse": false, + "type": "primary", + "unique": true + }, + { + "deduplicate": true, + "estimates": false, + "fields": [ + "time" + ], + "id": "10026", + "name": "idx_1726829639950663681", + "objectId": "4010134", + "sparse": false, + "type": "skiplist", + "unique": false + } + ], + "servers": [ + "PRMR-a24adubi", + "PRMR-n92yizyp" + ], + "failoverCandidates": [ + "PRMR-a24adubi", + "PRMR-n92yizyp" + ] + } + }, + "arango/Current/Version": { + "op": "increment" + } + }, + "term": 1, + "timestamp": "2022-03-09 13:38:10 UTC" + }, + { + "_key": "00000000000000000131", + "_rev": "_d05k6Fy---", + "clientId": "PRMR-a24adubi:cfc0eef6-6e31-4cac-aaa8-8d87137cae3f", + "epoch_millis": 1646833102973, + "request": { + "arango/.license": { + "op": "set", + "new": { + "features": { + "expires": 1646985599 + }, + "version": 1, + "hash": "ad800a7be16f904b1ae3e3889c17b0ceb2672ccc0bb8fa9990a724a582ecaa3b", + "license": "JD4EOk5VNRAPdSRUOywRewdAVAJgUm4GPjwpDCwTY0RhNylaFGQaMh0bPG5bNCsfYAMSOCAqd2B5CBNkL1xIMSA7AQECagIDOR94QlA9CzMBAG9qETMxQjQIPAolCzhcB1FaLFYaIk8VECUjSUEDVj5+FWE8LQF6AHVIBQAAAxwAQSkILBMUFTdzLHwVMzM7EiUeTBNeKQBvQ10gDywiJX5gFysKXQFSJhQVADBoVDZkDmE/Nj0MdRo4fwN8CEZjE3YsCyYWMF1hCSs8VzokBhcqPj98VQgEOG8dLxgGDRYGQzNbMkNdQ2I9cCwqOX9wJQgDXzs3OC4efGVqATVZG3gmECAgExgldBt7FBJhK2UOcysmA2gNRT8tUjoHJy02LRBJLwFjVwUoCBICJgkeWBAADyNXJEEpGHV/E0luHBwkYzswExQNBw1BAShlN1sBAEgtLiwBd0FlDz18P1oGMhUlHmdpQQ0oYQYGNSUCOgB+PS18AGErVC0XIz03HR1DAUFaK2dbCygXOHt7EiIlQj8MJCAKJDhfMjRdU1YcLQMMACI5eEJWEylxQGYPFgo3NmViGgQ+XV82JCYwHgdkFipYLEcrVk8pIwYCVCthKkV/GlUwFhIUPklUFy89BD8bJxcsOTFGFVBUJ3k8ADwxNCo5YzR5UR99FF0wAxJ8YB1ZQy8fUyg0HxMAJh1/VAtvOAQnIxkEHnoBaAlgOhoORVUHCysdHGsJJw8hQjslQwkWfCx5BydaJXgnDE8gACYBSRoDCydYKAA9CSw2NR5tRiladAQpQSp0GT8UIzJhEgMoMDwFC38GXSN0IThvC2gdF3UmHk9sMTMybjsUEDgzIQ15IyhXCHE8Ni15dwM6RQhTUDlsPnc7Mx5+LHRsQSMFVSUzRhQBGC1gLTldNHJEMR4GDTYZRAUCOR9aN2g9GyoDEHt0Jj0xbj9SCgcICAZhMyt0MmE5KggkKnoiYjZoVxEFM3owcwp+AnpANjw8Ayk7JzJ2FDhvMi1aJEY/CBFzCiMsZiBILgtSJnsQE3YIGGYKFyAOcCBQPQkeNzBoFTN7U18gBy0bLRoRVQBlJkZyEnYNdBEIElh2KzcleCgwGxR2IiV+CQdwOAYzBRksATYAdQFwLjZ8KH9bEw4vZ0VUPh8uBCA2NA4jBg5HKiZ4LWAbKgMVAAg6dEJ0CxAEO38OByN7NnhyRQAzdCstNXQFL2QUIzZ+DUEkMEp4DiBkEA==" + } + } + }, + "term": 1, + "timestamp": "2022-03-09 13:38:22 UTC" + }, + { + "_key": "00000000000000000132", + "_rev": "_d05yPBW--A", + "clientId": "PRMR-a24adubi:c2062125-a0a0-4326-8d48-811f5146254d", + "epoch_millis": 1646833976374, + "request": { + "arango/Current/Collections/_system/10005/s10016": { + "op": "set", + "new": { + "error": false, + "errorMessage": "", + "errorNum": 0, + "indexes": [ + { + "fields": [ + "_key" + ], + "id": "0", + "name": "primary", + "objectId": "4010036", + "sparse": false, + "type": "primary", + "unique": true + }, + { + "deduplicate": true, + "estimates": false, + "fields": [ + "time" + ], + "id": "10027", + "name": "idx_1726829639976878081", + "objectId": "4010144", + "sparse": false, + "type": "skiplist", + "unique": false + } + ], + "servers": [ + "PRMR-a24adubi", + "PRMR-n92yizyp" + ], + "failoverCandidates": [ + "PRMR-a24adubi", + "PRMR-n92yizyp" + ] + } + }, + "arango/Current/Version": { + "op": "increment" + } + }, + "term": 1, + "timestamp": "2022-03-09 13:52:56 UTC" + } + ] +} diff --git a/pkg/deployment/deployment.go b/pkg/deployment/deployment.go index b309c28fa..da5d9a275 100644 --- a/pkg/deployment/deployment.go +++ b/pkg/deployment/deployment.go @@ -163,20 +163,26 @@ func (d *Deployment) RefreshAgencyCache(ctx context.Context) (uint64, error) { return 0, nil } - lCtx, c := globals.GetGlobalTimeouts().Agency().WithTimeout(ctx) - defer c() + if info := d.apiObject.Status.Agency; info != nil { + if size := info.Size; size != nil { + lCtx, c := globals.GetGlobalTimeouts().Agency().WithTimeout(ctx) + defer c() + + var clients []agencydriver.Agency + for _, m := range d.GetStatusSnapshot().Members.Agents { + a, err := d.GetAgency(lCtx, m.ID) + if err != nil { + return 0, err + } + + clients = append(clients, a) + } - var clients []agencydriver.Agency - for _, m := range d.GetStatusSnapshot().Members.Agents { - a, err := d.GetAgency(lCtx, m.ID) - if err != nil { - return 0, err + return d.agencyCache.Reload(lCtx, int(*size), clients) } - - clients = append(clients, a) } - return d.agencyCache.Reload(lCtx, clients) + return 0, errors.Newf("Agency not yet established") } func (d *Deployment) SetAgencyMaintenanceMode(ctx context.Context, enabled bool) error { diff --git a/pkg/deployment/metrics.go b/pkg/deployment/metrics.go index 9f539732b..c411affcc 100644 --- a/pkg/deployment/metrics.go +++ b/pkg/deployment/metrics.go @@ -120,7 +120,7 @@ func (i *inventory) Collect(m chan<- prometheus.Metric) { db, name, shard, - server, + string(server), } if id == 0 { diff --git a/pkg/deployment/reconcile/action_cleanout_member.go b/pkg/deployment/reconcile/action_cleanout_member.go index aca38300b..c41565d0a 100644 --- a/pkg/deployment/reconcile/action_cleanout_member.go +++ b/pkg/deployment/reconcile/action_cleanout_member.go @@ -29,7 +29,7 @@ import ( driver "github.com/arangodb/go-driver" api "github.com/arangodb/kube-arangodb/pkg/apis/deployment/v1" - "github.com/arangodb/kube-arangodb/pkg/util/arangod" + "github.com/arangodb/kube-arangodb/pkg/deployment/agency" ) func init() { @@ -118,58 +118,19 @@ func (a *actionCleanoutMember) CheckProgress(ctx context.Context) (bool, bool, e return true, false, nil } - ctxChild, cancel := globals.GetGlobalTimeouts().ArangoD().WithTimeout(ctx) - defer cancel() - c, err := a.actionCtx.GetDatabaseClient(ctxChild) - if err != nil { - a.log.Err(err).Debug("Failed to create database client") - return false, false, nil - } - - ctxChild, cancel = globals.GetGlobalTimeouts().ArangoD().WithTimeout(ctx) - defer cancel() - cluster, err := c.Cluster(ctxChild) - if err != nil { - a.log.Err(err).Debug("Failed to access cluster") + cache, ok := a.actionCtx.GetAgencyCache() + if !ok { + a.log.Debug("AgencyCache is not ready") return false, false, nil } - ctxChild, cancel = globals.GetGlobalTimeouts().ArangoD().WithTimeout(ctx) - defer cancel() - cleanedOut, err := cluster.IsCleanedOut(ctxChild, a.action.MemberID) - if err != nil { - a.log.Err(err).Debug("IsCleanedOut failed") - return false, false, nil - } - if !cleanedOut { + if !cache.Target.CleanedServers.Contains(agency.Server(a.action.MemberID)) { // We're not done yet, check job status a.log.Debug("IsCleanedOut returned false") - ctxChild, cancel = globals.GetGlobalTimeouts().ArangoD().WithTimeout(ctx) - defer cancel() - c, err := a.actionCtx.GetDatabaseClient(ctxChild) - if err != nil { - a.log.Err(err).Debug("Failed to create database client") - return false, false, nil - } - - ctxChild, cancel = globals.GetGlobalTimeouts().ArangoD().WithTimeout(ctx) - defer cancel() - agency, err := a.actionCtx.GetAgency(ctxChild) - if err != nil { - a.log.Err(err).Debug("Failed to create agency client") - return false, false, nil - } - - ctxChild, cancel = globals.GetGlobalTimeouts().ArangoD().WithTimeout(ctx) - defer cancel() - jobStatus, err := arangod.CleanoutServerJobStatus(ctxChild, m.CleanoutJobID, c, agency) - if err != nil { - a.log.Err(err).Debug("Failed to fetch cleanout job status") - return false, false, nil - } - if jobStatus.IsFailed() { - a.log.Str("reason", jobStatus.Reason()).Warn("Cleanout Job failed. Aborting plan") + details, jobStatus := cache.Target.GetJob(agency.JobID(m.CleanoutJobID)) + if jobStatus == agency.JobPhaseFailed { + a.log.Str("reason", details.Reason).Warn("Cleanout Job failed. Aborting plan") // Revert cleanout state m.Phase = api.MemberPhaseCreated m.CleanoutJobID = "" diff --git a/pkg/deployment/reconcile/action_context.go b/pkg/deployment/reconcile/action_context.go index 18bfc0523..c5f4e4971 100644 --- a/pkg/deployment/reconcile/action_context.go +++ b/pkg/deployment/reconcile/action_context.go @@ -25,7 +25,6 @@ import ( "time" "github.com/arangodb/arangosync-client/client" - "github.com/arangodb/go-driver/agency" core "k8s.io/api/core/v1" "github.com/arangodb/go-driver" @@ -162,10 +161,6 @@ func (ac *actionContext) GetNamespace() string { return ac.context.GetNamespace() } -func (ac *actionContext) GetAgencyClientsWithPredicate(ctx context.Context, predicate func(id string) bool) ([]driver.Connection, error) { - return ac.context.GetAgencyClientsWithPredicate(ctx, predicate) -} - func (ac *actionContext) GetStatus() (api.DeploymentStatus, int32) { return ac.context.GetStatus() } @@ -272,24 +267,6 @@ func (ac *actionContext) GetServerClient(ctx context.Context, group api.ServerGr return c, nil } -// GetAgencyClients returns a client connection for every agency member. -func (ac *actionContext) GetAgencyClients(ctx context.Context) ([]driver.Connection, error) { - c, err := ac.context.GetAgencyClients(ctx) - if err != nil { - return nil, errors.WithStack(err) - } - return c, nil -} - -// GetAgency returns a connection to the agency. -func (ac *actionContext) GetAgency(ctx context.Context, agencyIDs ...string) (agency.Agency, error) { - a, err := ac.context.GetAgency(ctx, agencyIDs...) - if err != nil { - return nil, errors.WithStack(err) - } - return a, nil -} - // GetSyncServerClient returns a cached client for a specific arangosync server. func (ac *actionContext) GetSyncServerClient(ctx context.Context, group api.ServerGroup, id string) (client.API, error) { c, err := ac.context.GetSyncServerClient(ctx, group, id) diff --git a/pkg/deployment/reconcile/action_resign_leadership.go b/pkg/deployment/reconcile/action_resign_leadership.go index 74f84bff3..a00f01010 100644 --- a/pkg/deployment/reconcile/action_resign_leadership.go +++ b/pkg/deployment/reconcile/action_resign_leadership.go @@ -26,10 +26,10 @@ import ( "github.com/arangodb/kube-arangodb/pkg/util/globals" "github.com/arangodb/go-driver" - "github.com/arangodb/kube-arangodb/pkg/util/arangod" "github.com/arangodb/kube-arangodb/pkg/util/errors" api "github.com/arangodb/kube-arangodb/pkg/apis/deployment/v1" + "github.com/arangodb/kube-arangodb/pkg/deployment/agency" ) func init() { @@ -122,7 +122,8 @@ func (a *actionResignLeadership) CheckProgress(ctx context.Context) (bool, bool, return true, false, nil } - if agencyState, agencyOK := a.actionCtx.GetAgencyCache(); !agencyOK { + agencyState, agencyOK := a.actionCtx.GetAgencyCache() + if !agencyOK { a.log.Error("Unable to get maintenance mode") return false, false, nil } else if agencyState.Supervision.Maintenance.Exists() { @@ -135,50 +136,24 @@ func (a *actionResignLeadership) CheckProgress(ctx context.Context) (bool, bool, return true, false, nil } - ctxChild, cancel := globals.GetGlobalTimeouts().ArangoD().WithTimeout(ctx) - defer cancel() - agency, err := a.actionCtx.GetAgency(ctxChild) - if err != nil { - a.log.Err(err).Debug("Failed to create agency client") - return false, false, nil - } - - ctxChild, cancel = globals.GetGlobalTimeouts().ArangoD().WithTimeout(ctx) - defer cancel() - c, err := a.actionCtx.GetDatabaseClient(ctxChild) - if err != nil { - a.log.Err(err).Debug("Failed to create member client") - return false, false, nil - } - - ctxChild, cancel = globals.GetGlobalTimeouts().ArangoD().WithTimeout(ctx) - defer cancel() - jobStatus, err := arangod.CleanoutServerJobStatus(ctxChild, m.CleanoutJobID, c, agency) - if err != nil { - if driver.IsNotFound(err) { - a.log.Err(err).Debug("Job not found, but proceeding") - return true, false, nil - } - a.log.Err(err).Debug("Failed to fetch job status") - return false, false, errors.WithStack(err) - } - - if jobStatus.IsFailed() { + _, jobStatus := agencyState.Target.GetJob(agency.JobID(m.CleanoutJobID)) + switch jobStatus { + case agency.JobPhaseFailed: m.CleanoutJobID = "" if err := a.actionCtx.UpdateMember(ctx, m); err != nil { return false, false, errors.WithStack(err) } a.log.Error("Resign server job failed") return true, false, nil - } - - if jobStatus.IsFinished() { + case agency.JobPhaseFinished: m.CleanoutJobID = "" if err := a.actionCtx.UpdateMember(ctx, m); err != nil { return false, false, errors.WithStack(err) } return true, false, nil + case agency.JobPhaseUnknown: + a.log.Debug("Job not found, but proceeding") + return true, false, nil } - return false, false, nil } diff --git a/pkg/deployment/reconcile/action_wait_for_member_in_sync.go b/pkg/deployment/reconcile/action_wait_for_member_in_sync.go index aed0b01ba..80d0e58c3 100644 --- a/pkg/deployment/reconcile/action_wait_for_member_in_sync.go +++ b/pkg/deployment/reconcile/action_wait_for_member_in_sync.go @@ -100,7 +100,7 @@ func (a *actionWaitForMemberInSync) checkCluster() (bool, error) { return false, nil } - notInSyncShards := agency.GetDBServerShardsNotInSync(agencyState, a.MemberID()) + notInSyncShards := agency.GetDBServerShardsNotInSync(agencyState, agency.Server(a.MemberID())) if len(notInSyncShards) > 0 { a.log.Str("mode", "cluster").Str("member", a.MemberID()).Int("shard", len(notInSyncShards)).Info("DBServer contains not in sync shards") diff --git a/pkg/deployment/reconcile/action_wait_for_member_up.go b/pkg/deployment/reconcile/action_wait_for_member_up.go index f68a8fc9d..7169ec8e3 100644 --- a/pkg/deployment/reconcile/action_wait_for_member_up.go +++ b/pkg/deployment/reconcile/action_wait_for_member_up.go @@ -22,15 +22,12 @@ package reconcile import ( "context" - "time" "github.com/arangodb/kube-arangodb/pkg/util/globals" "github.com/arangodb/kube-arangodb/pkg/util/errors" driver "github.com/arangodb/go-driver" - "github.com/arangodb/go-driver/agency" - api "github.com/arangodb/kube-arangodb/pkg/apis/deployment/v1" ) @@ -85,12 +82,12 @@ func (a *actionWaitForMemberUp) CheckProgress(ctx context.Context) (bool, bool, return a.checkProgressSingle(ctxChild) case api.DeploymentModeActiveFailover: if a.action.Group == api.ServerGroupAgents { - return a.checkProgressAgent(ctxChild) + return a.checkProgressAgent() } return a.checkProgressSingleInActiveFailover(ctxChild) default: if a.action.Group == api.ServerGroupAgents { - return a.checkProgressAgent(ctxChild) + return a.checkProgressAgent() } return a.checkProgressCluster() } @@ -128,23 +125,13 @@ func (a *actionWaitForMemberUp) checkProgressSingleInActiveFailover(ctx context. // checkProgressAgent checks the progress of the action in the case // of an agent. -func (a *actionWaitForMemberUp) checkProgressAgent(ctx context.Context) (bool, bool, error) { - clients, err := a.actionCtx.GetAgencyClients(ctx) - if err != nil { - a.log.Err(err).Debug("Failed to create agency clients") +func (a *actionWaitForMemberUp) checkProgressAgent() (bool, bool, error) { + agencyHealth, ok := a.actionCtx.GetAgencyHealth() + if !ok { + a.log.Debug("Agency health fetch failed") return false, false, nil } - - for _, a := range clients { - a.Endpoints() - } - - shortCtx, c := context.WithTimeout(ctx, 3*time.Second) - defer c() - - shortCtx = agency.WithAllowDifferentLeaderEndpoints(shortCtx) - - if err := agency.AreAgentsHealthy(shortCtx, clients); err != nil { + if err := agencyHealth.Healthy(); err != nil { a.log.Err(err).Debug("Not all agents are ready") return false, false, nil } diff --git a/pkg/deployment/reconcile/plan_builder_normal.go b/pkg/deployment/reconcile/plan_builder_normal.go index 68d35b5a9..e85595db6 100644 --- a/pkg/deployment/reconcile/plan_builder_normal.go +++ b/pkg/deployment/reconcile/plan_builder_normal.go @@ -25,6 +25,7 @@ import ( api "github.com/arangodb/kube-arangodb/pkg/apis/deployment/v1" "github.com/arangodb/kube-arangodb/pkg/deployment/actions" + "github.com/arangodb/kube-arangodb/pkg/deployment/agency" "github.com/arangodb/kube-arangodb/pkg/util/k8sutil" ) @@ -120,7 +121,7 @@ func (r *Reconciler) createMemberFailedRestorePlan(ctx context.Context, apiObjec continue } - if agencyState.Plan.Collections.IsDBServerInDatabases(m.ID) { + if agencyState.Plan.Collections.IsDBServerInDatabases(agency.Server(m.ID)) { // DBServer still exists in agency plan! Will not be removed, but needs to be recreated memberLog.Info("Recreating DBServer - it cannot be removed gracefully") plan = append(plan, diff --git a/pkg/deployment/reconcile/plan_builder_rotate_upgrade.go b/pkg/deployment/reconcile/plan_builder_rotate_upgrade.go index 722d2ad68..45e200eb1 100644 --- a/pkg/deployment/reconcile/plan_builder_rotate_upgrade.go +++ b/pkg/deployment/reconcile/plan_builder_rotate_upgrade.go @@ -472,7 +472,7 @@ func groupReadyForRestart(context PlanBuilderContext, status api.DeploymentStatu return false, "Unable to get agency cache" } - blockingRestartShards := agency.GetDBServerBlockingRestartShards(agencyState, member.ID) + blockingRestartShards := agency.GetDBServerBlockingRestartShards(agencyState, agency.Server(member.ID)) if s := len(blockingRestartShards); s > 0 { return false, fmt.Sprintf("There are %d shards which are blocking restart", s) diff --git a/pkg/deployment/reconciler/context.go b/pkg/deployment/reconciler/context.go index 33511539a..b1f0ca5b9 100644 --- a/pkg/deployment/reconciler/context.go +++ b/pkg/deployment/reconciler/context.go @@ -27,7 +27,6 @@ import ( "github.com/arangodb/arangosync-client/client" "github.com/arangodb/go-driver" - "github.com/arangodb/go-driver/agency" api "github.com/arangodb/kube-arangodb/pkg/apis/deployment/v1" "github.com/arangodb/kube-arangodb/pkg/deployment/acs/sutil" agencyCache "github.com/arangodb/kube-arangodb/pkg/deployment/agency" @@ -124,15 +123,6 @@ type ArangoApplier interface { ApplyPatch(ctx context.Context, p ...patch.Item) error } -type DeploymentAgencyClient interface { - // GetAgencyClients returns a client connection for every agency member. - GetAgencyClients(ctx context.Context) ([]driver.Connection, error) - // GetAgencyClientsWithPredicate returns a client connection for every agency member which match condition. - GetAgencyClientsWithPredicate(ctx context.Context, predicate func(id string) bool) ([]driver.Connection, error) - // GetAgency returns a connection to the entire agency. - GetAgency(ctx context.Context, agencyIDs ...string) (agency.Agency, error) -} - type DeploymentDatabaseClient interface { // GetDatabaseClient returns a cached client for the entire database (cluster coordinators or single server), // creating one if needed. @@ -160,7 +150,6 @@ type KubernetesEventGenerator interface { } type DeploymentClient interface { - DeploymentAgencyClient DeploymentDatabaseClient DeploymentMemberClient DeploymentSyncClient diff --git a/pkg/deployment/resilience/context.go b/pkg/deployment/resilience/context.go index 8b0a231d4..a1e16ec89 100644 --- a/pkg/deployment/resilience/context.go +++ b/pkg/deployment/resilience/context.go @@ -30,7 +30,8 @@ import ( // Context provides methods to the resilience package. type Context interface { reconciler.DeploymentDatabaseClient - reconciler.DeploymentAgencyClient + reconciler.ArangoAgency + // GetSpec returns the current specification of the deployment GetSpec() api.DeploymentSpec // GetStatus returns the current status of the deployment diff --git a/pkg/deployment/resilience/member_failure.go b/pkg/deployment/resilience/member_failure.go index 5974f38b5..c42776370 100644 --- a/pkg/deployment/resilience/member_failure.go +++ b/pkg/deployment/resilience/member_failure.go @@ -28,7 +28,6 @@ import ( "github.com/arangodb/kube-arangodb/pkg/util/errors" - "github.com/arangodb/go-driver/agency" api "github.com/arangodb/kube-arangodb/pkg/apis/deployment/v1" "github.com/arangodb/kube-arangodb/pkg/util/arangod" ) @@ -129,16 +128,15 @@ func (r *Resilience) isMemberFailureAcceptable(ctx context.Context, group api.Se switch group { case api.ServerGroupAgents: - // All good when remaining agents are health - ctxChild, cancel := globals.GetGlobalTimeouts().ArangoD().WithTimeout(ctx) - defer cancel() - clients, err := r.context.GetAgencyClientsWithPredicate(ctxChild, func(id string) bool { return id != m.ID }) - if err != nil { - return false, "", errors.WithStack(err) + agencyHealth, ok := r.context.GetAgencyHealth() + if !ok { + return false, "AgencyHealth is not present", nil } - if err := agency.AreAgentsHealthy(ctx, clients); err != nil { + + if err := agencyHealth.Healthy(); err != nil { return false, err.Error(), nil } + return true, "", nil case api.ServerGroupDBServers: ctxChild, cancel := globals.GetGlobalTimeouts().ArangoD().WithTimeout(ctx) diff --git a/pkg/deployment/resources/pod_finalizers.go b/pkg/deployment/resources/pod_finalizers.go index c0d6f5f2c..579975290 100644 --- a/pkg/deployment/resources/pod_finalizers.go +++ b/pkg/deployment/resources/pod_finalizers.go @@ -132,7 +132,7 @@ func (r *Resources) runPodFinalizers(ctx context.Context, p *v1.Pod, memberStatu // It returns nil if the finalizer can be removed. func (r *Resources) inspectFinalizerPodAgencyServing(ctx context.Context, p *v1.Pod, memberStatus api.MemberStatus, updateMember func(api.MemberStatus) error) error { log := r.log.Str("section", "agency") - if err := r.prepareAgencyPodTermination(ctx, p, memberStatus, func(update api.MemberStatus) error { + if err := r.prepareAgencyPodTermination(p, memberStatus, func(update api.MemberStatus) error { if err := updateMember(update); err != nil { return errors.WithStack(err) } diff --git a/pkg/deployment/resources/pod_termination.go b/pkg/deployment/resources/pod_termination.go index 148262cf2..7a1f8b980 100644 --- a/pkg/deployment/resources/pod_termination.go +++ b/pkg/deployment/resources/pod_termination.go @@ -22,7 +22,6 @@ package resources import ( "context" - "time" "github.com/arangodb/kube-arangodb/pkg/util/globals" @@ -31,10 +30,9 @@ import ( meta "k8s.io/apimachinery/pkg/apis/meta/v1" driver "github.com/arangodb/go-driver" - "github.com/arangodb/go-driver/agency" api "github.com/arangodb/kube-arangodb/pkg/apis/deployment/v1" "github.com/arangodb/kube-arangodb/pkg/apis/shared" - "github.com/arangodb/kube-arangodb/pkg/util/arangod" + "github.com/arangodb/kube-arangodb/pkg/deployment/agency" "github.com/arangodb/kube-arangodb/pkg/util/k8sutil" v1 "k8s.io/api/core/v1" ) @@ -42,7 +40,7 @@ import ( // prepareAgencyPodTermination checks if the given agency pod is allowed to terminate // and if so, prepares it for termination. // It returns nil if the pod is allowed to terminate, an error otherwise. -func (r *Resources) prepareAgencyPodTermination(ctx context.Context, p *v1.Pod, memberStatus api.MemberStatus, updateMember func(api.MemberStatus) error) error { +func (r *Resources) prepareAgencyPodTermination(p *v1.Pod, memberStatus api.MemberStatus, updateMember func(api.MemberStatus) error) error { log := r.log.Str("section", "pod") // Inspect member phase @@ -71,12 +69,10 @@ func (r *Resources) prepareAgencyPodTermination(ctx context.Context, p *v1.Pod, } // Check PVC - ctxChild, cancel := globals.GetGlobalTimeouts().Kubernetes().WithTimeout(ctx) - defer cancel() - pvc, err := r.context.ACS().CurrentClusterCache().PersistentVolumeClaim().V1().Read().Get(ctxChild, memberStatus.PersistentVolumeClaimName, meta.GetOptions{}) - if err != nil { - log.Err(err).Warn("Failed to get PVC for member") - return errors.WithStack(err) + pvc, ok := r.context.ACS().CurrentClusterCache().PersistentVolumeClaim().V1().GetSimple(memberStatus.PersistentVolumeClaimName) + if !ok { + log.Warn("Failed to get PVC for member") + return errors.Newf("Failed to get PVC for member") } if k8sutil.IsPersistentVolumeClaimMarkedForDeletion(pvc) { agentDataWillBeGone = true @@ -90,24 +86,16 @@ func (r *Resources) prepareAgencyPodTermination(ctx context.Context, p *v1.Pod, // Inspect agency state log.Debug("Agent data will be gone, so we will check agency serving status first") - ctxChild, cancel = context.WithTimeout(ctx, time.Second*15) - defer cancel() - ctxLeader := agency.WithAllowNoLeader(ctxChild) // The ID we're checking may be the leader, so ignore situations where all other agents are followers - agencyConns, err := r.context.GetAgencyClientsWithPredicate(ctxLeader, func(id string) bool { return id != memberStatus.ID }) - if err != nil { - log.Err(err).Debug("Failed to create member client") - return errors.WithStack(err) - } - if len(agencyConns) == 0 { - log.Err(err).Debug("No more remaining agents, we cannot delete this one") - return errors.WithStack(errors.Newf("No more remaining agents")) + agencyHealth, ok := r.context.GetAgencyHealth() + if !ok { + log.Debug("Agency health fetch failed") + return errors.Newf("Agency health fetch failed") } - if err := agency.AreAgentsHealthy(ctxLeader, agencyConns); err != nil { - log.Err(err).Debug("Remaining agents are not healthy") - return errors.WithStack(err) + if err := agencyHealth.Healthy(); err != nil { + log.Err(err).Debug("Agency is not healthy. Cannot delete this one") + return errors.WithStack(errors.Newf("Agency is not healthy")) } - // Complete agent recovery is needed, since data is already gone or not accessible if memberStatus.Conditions.Update(api.ConditionTypeAgentRecoveryNeeded, true, "Data Gone", "") { if err := updateMember(memberStatus); err != nil { @@ -265,22 +253,15 @@ func (r *Resources) prepareDBServerPodTermination(ctx context.Context, p *v1.Pod } } else if memberStatus.Phase == api.MemberPhaseDrain { // Check the job progress - ctxChild, cancel = globals.GetGlobalTimeouts().ArangoD().WithTimeout(ctx) - defer cancel() - agency, err := r.context.GetAgency(ctxChild) - if err != nil { - log.Err(err).Debug("Failed to create agency client") - return errors.WithStack(err) - } - ctxChild, cancel = globals.GetGlobalTimeouts().ArangoD().WithTimeout(ctx) - defer cancel() - jobStatus, err := arangod.CleanoutServerJobStatus(ctxChild, memberStatus.CleanoutJobID, c, agency) - if err != nil { - log.Err(err).Debug("Failed to fetch job status") - return errors.WithStack(err) + cache, ok := r.context.GetAgencyCache() + if !ok { + return errors.Newf("AgencyCache is not ready") } - if jobStatus.IsFailed() { - log.Str("reason", jobStatus.Reason()).Warn("Job failed") + + details, jobStatus := cache.Target.GetJob(agency.JobID(memberStatus.CleanoutJobID)) + switch jobStatus { + case agency.JobPhaseFailed: + log.Str("reason", details.Reason).Warn("Job failed") // Revert cleanout state memberStatus.Phase = api.MemberPhaseCreated memberStatus.CleanoutJobID = "" @@ -289,30 +270,21 @@ func (r *Resources) prepareDBServerPodTermination(ctx context.Context, p *v1.Pod } log.Error("Cleanout/Resign server job failed, continue anyway") return nil - } - if jobStatus.IsFinished() { + case agency.JobPhaseFinished: memberStatus.CleanoutJobID = "" memberStatus.Phase = api.MemberPhaseCreated } } else if memberStatus.Phase == api.MemberPhaseResign { // Check the job progress - ctxChild, cancel = globals.GetGlobalTimeouts().ArangoD().WithTimeout(ctx) - defer cancel() - agency, err := r.context.GetAgency(ctxChild) - if err != nil { - log.Err(err).Debug("Failed to create agency client") - return errors.WithStack(err) + cache, ok := r.context.GetAgencyCache() + if !ok { + return errors.Newf("AgencyCache is not ready") } - ctxChild, cancel = globals.GetGlobalTimeouts().ArangoD().WithTimeout(ctx) - defer cancel() - jobStatus, err := arangod.CleanoutServerJobStatus(ctxChild, memberStatus.CleanoutJobID, c, agency) - if err != nil { - log.Err(err).Debug("Failed to fetch job status") - return errors.WithStack(err) - } - if jobStatus.IsFailed() { - log.Str("reason", jobStatus.Reason()).Warn("Resign Job failed") + details, jobStatus := cache.Target.GetJob(agency.JobID(memberStatus.CleanoutJobID)) + switch jobStatus { + case agency.JobPhaseFailed: + log.Str("reason", details.Reason).Warn("Resign Job failed") // Revert cleanout state memberStatus.Phase = api.MemberPhaseCreated memberStatus.CleanoutJobID = "" @@ -321,9 +293,8 @@ func (r *Resources) prepareDBServerPodTermination(ctx context.Context, p *v1.Pod } log.Error("Cleanout/Resign server job failed, continue anyway") return nil - } - if jobStatus.IsFinished() { - log.Str("reason", jobStatus.Reason()).Debug("Resign Job finished") + case agency.JobPhaseFinished: + log.Str("reason", details.Reason).Debug("Resign Job finished") memberStatus.CleanoutJobID = "" memberStatus.Phase = api.MemberPhaseCreated if err := updateMember(memberStatus); err != nil { diff --git a/pkg/util/arangod/cleanout_server.go b/pkg/util/arangod/cleanout_server.go deleted file mode 100644 index 636c0705d..000000000 --- a/pkg/util/arangod/cleanout_server.go +++ /dev/null @@ -1,95 +0,0 @@ -// -// DISCLAIMER -// -// Copyright 2016-2022 ArangoDB GmbH, Cologne, Germany -// -// 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. -// -// Copyright holder is ArangoDB GmbH, Cologne, Germany -// - -package arangod - -import ( - "context" - "fmt" - - "github.com/arangodb/kube-arangodb/pkg/util/errors" - - "github.com/arangodb/go-driver" - "github.com/arangodb/go-driver/agency" -) - -// CleanoutJobStatus is a strongly typed status of an agency cleanout-server-job. -type CleanoutJobStatus struct { - state string - reason string -} - -// IsFailed returns true when the job is failed -func (s CleanoutJobStatus) IsFailed() bool { - return s.state == "Failed" -} - -// IsFinished returns true when the job is finished -func (s CleanoutJobStatus) IsFinished() bool { - return s.state == "Finished" -} - -// Reason returns the reason for the current state. -func (s CleanoutJobStatus) Reason() string { - return s.reason -} - -// String returns a string representation of the given state. -func (s CleanoutJobStatus) String() string { - return fmt.Sprintf("state: '%s', reason: '%s'", s.state, s.reason) -} - -var ( - agencyJobStateKeyPrefixes = [][]string{ - {"arango", "Target", "ToDo"}, - {"arango", "Target", "Pending"}, - {"arango", "Target", "Finished"}, - {"arango", "Target", "Failed"}, - } -) - -type agencyJob struct { - Reason string `json:"reason,omitempty"` - Server string `json:"server,omitempty"` - JobID string `json:"jobId,omitempty"` - Type string `json:"type,omitempty"` -} - -// CleanoutServerJobStatus checks the status of a cleanout-server job with given ID. -func CleanoutServerJobStatus(ctx context.Context, jobID string, client driver.Client, agencyClient agency.Agency) (CleanoutJobStatus, error) { - for _, keyPrefix := range agencyJobStateKeyPrefixes { - key := append(keyPrefix, jobID) - var job agencyJob - if err := agencyClient.ReadKey(ctx, key, &job); err == nil { - return CleanoutJobStatus{ - state: keyPrefix[len(keyPrefix)-1], - reason: job.Reason, - }, nil - } else if agency.IsKeyNotFound(err) { - continue - } else { - return CleanoutJobStatus{}, errors.WithStack(err) - } - } - // Job not found in any states - return CleanoutJobStatus{ - reason: "job not found", - }, nil -}