diff --git a/src/cmd/services/m3dbnode/config/config_test.go b/src/cmd/services/m3dbnode/config/config_test.go index 3702d802e9..1335b9c22e 100644 --- a/src/cmd/services/m3dbnode/config/config_test.go +++ b/src/cmd/services/m3dbnode/config/config_test.go @@ -399,6 +399,7 @@ func TestConfiguration(t *testing.T) { asyncWriteMaxConcurrency: null targetHostQueueFlushSize: null hostQueueFlushInterval: null + useV2BatchAPIs: null gcPercentage: 100 writeNewSeriesLimitPerSecond: 1048576 writeNewSeriesBackoffDuration: 2ms diff --git a/src/dbnode/client/client_mock.go b/src/dbnode/client/client_mock.go index a3c87210c9..152a97e85a 100644 --- a/src/dbnode/client/client_mock.go +++ b/src/dbnode/client/client_mock.go @@ -2349,6 +2349,34 @@ func (mr *MockOptionsMockRecorder) AsyncWriteMaxConcurrency() *gomock.Call { return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "AsyncWriteMaxConcurrency", reflect.TypeOf((*MockOptions)(nil).AsyncWriteMaxConcurrency)) } +// SetUseV2BatchAPIs mocks base method +func (m *MockOptions) SetUseV2BatchAPIs(value bool) Options { + m.ctrl.T.Helper() + ret := m.ctrl.Call(m, "SetUseV2BatchAPIs", value) + ret0, _ := ret[0].(Options) + return ret0 +} + +// SetUseV2BatchAPIs indicates an expected call of SetUseV2BatchAPIs +func (mr *MockOptionsMockRecorder) SetUseV2BatchAPIs(value interface{}) *gomock.Call { + mr.mock.ctrl.T.Helper() + return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "SetUseV2BatchAPIs", reflect.TypeOf((*MockOptions)(nil).SetUseV2BatchAPIs), value) +} + +// UseV2BatchAPIs mocks base method +func (m *MockOptions) UseV2BatchAPIs() bool { + m.ctrl.T.Helper() + ret := m.ctrl.Call(m, "UseV2BatchAPIs") + ret0, _ := ret[0].(bool) + return ret0 +} + +// UseV2BatchAPIs indicates an expected call of UseV2BatchAPIs +func (mr *MockOptionsMockRecorder) UseV2BatchAPIs() *gomock.Call { + mr.mock.ctrl.T.Helper() + return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "UseV2BatchAPIs", reflect.TypeOf((*MockOptions)(nil).UseV2BatchAPIs)) +} + // MockAdminOptions is a mock of AdminOptions interface type MockAdminOptions struct { ctrl *gomock.Controller @@ -3688,6 +3716,34 @@ func (mr *MockAdminOptionsMockRecorder) AsyncWriteMaxConcurrency() *gomock.Call return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "AsyncWriteMaxConcurrency", reflect.TypeOf((*MockAdminOptions)(nil).AsyncWriteMaxConcurrency)) } +// SetUseV2BatchAPIs mocks base method +func (m *MockAdminOptions) SetUseV2BatchAPIs(value bool) Options { + m.ctrl.T.Helper() + ret := m.ctrl.Call(m, "SetUseV2BatchAPIs", value) + ret0, _ := ret[0].(Options) + return ret0 +} + +// SetUseV2BatchAPIs indicates an expected call of SetUseV2BatchAPIs +func (mr *MockAdminOptionsMockRecorder) SetUseV2BatchAPIs(value interface{}) *gomock.Call { + mr.mock.ctrl.T.Helper() + return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "SetUseV2BatchAPIs", reflect.TypeOf((*MockAdminOptions)(nil).SetUseV2BatchAPIs), value) +} + +// UseV2BatchAPIs mocks base method +func (m *MockAdminOptions) UseV2BatchAPIs() bool { + m.ctrl.T.Helper() + ret := m.ctrl.Call(m, "UseV2BatchAPIs") + ret0, _ := ret[0].(bool) + return ret0 +} + +// UseV2BatchAPIs indicates an expected call of UseV2BatchAPIs +func (mr *MockAdminOptionsMockRecorder) UseV2BatchAPIs() *gomock.Call { + mr.mock.ctrl.T.Helper() + return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "UseV2BatchAPIs", reflect.TypeOf((*MockAdminOptions)(nil).UseV2BatchAPIs)) +} + // SetOrigin mocks base method func (m *MockAdminOptions) SetOrigin(value topology.Host) AdminOptions { m.ctrl.T.Helper() diff --git a/src/dbnode/client/config.go b/src/dbnode/client/config.go index 67dd8ee9dc..b85413ac0e 100644 --- a/src/dbnode/client/config.go +++ b/src/dbnode/client/config.go @@ -104,6 +104,10 @@ type Configuration struct { // HostQueueFlushInterval sets the interval at which the m3db client will flush the queue for a // given host regardless of the number of batched operations. HostQueueFlushInterval *time.Duration `yaml:"hostQueueFlushInterval"` + + // UseV2BatchAPIs determines whether the V2 batch APIs are used. Note that the M3DB nodes must + // have support for the V2 APIs in order for this feature to be used. + UseV2BatchAPIs *bool `yaml:"useV2BatchAPIs"` } // ProtoConfiguration is the configuration for running with ProtoDataMode enabled. @@ -306,6 +310,10 @@ func (c Configuration) NewAdminClient( SetChannelOptions(xtchannel.NewDefaultChannelOptions()). SetInstrumentOptions(iopts) + if c.UseV2BatchAPIs != nil { + v = v.SetUseV2BatchAPIs(*c.UseV2BatchAPIs) + } + if buildAsyncPool { var size int if c.AsyncWriteWorkerPoolSize == nil { diff --git a/src/dbnode/client/host_queue.go b/src/dbnode/client/host_queue.go index 93c3dbce50..6ede814c7f 100644 --- a/src/dbnode/client/host_queue.go +++ b/src/dbnode/client/host_queue.go @@ -21,6 +21,7 @@ package client import ( + "bytes" "fmt" "math" "sync" @@ -43,24 +44,29 @@ type queue struct { sync.WaitGroup sync.RWMutex - opts Options - nowFn clock.NowFn - host topology.Host - connPool connectionPool - writeBatchRawRequestPool writeBatchRawRequestPool - writeBatchRawRequestElementArrayPool writeBatchRawRequestElementArrayPool - writeTaggedBatchRawRequestPool writeTaggedBatchRawRequestPool - writeTaggedBatchRawRequestElementArrayPool writeTaggedBatchRawRequestElementArrayPool - workerPool xsync.PooledWorkerPool - size int - ops []op - opsSumSize int - opsLastRotatedAt time.Time - opsArrayPool *opArrayPool - drainIn chan []op - writeOpBatchSize tally.Histogram - fetchOpBatchSize tally.Histogram - status status + opts Options + nowFn clock.NowFn + host topology.Host + connPool connectionPool + writeBatchRawRequestPool writeBatchRawRequestPool + writeBatchRawV2RequestPool writeBatchRawV2RequestPool + writeBatchRawRequestElementArrayPool writeBatchRawRequestElementArrayPool + writeBatchRawV2RequestElementArrayPool writeBatchRawV2RequestElementArrayPool + writeTaggedBatchRawRequestPool writeTaggedBatchRawRequestPool + writeTaggedBatchRawV2RequestPool writeTaggedBatchRawV2RequestPool + writeTaggedBatchRawRequestElementArrayPool writeTaggedBatchRawRequestElementArrayPool + writeTaggedBatchRawV2RequestElementArrayPool writeTaggedBatchRawV2RequestElementArrayPool + workerPool xsync.PooledWorkerPool + size int + ops []op + opsSumSize int + opsLastRotatedAt time.Time + opsArrayPool *opArrayPool + drainIn chan []op + writeOpBatchSize tally.Histogram + fetchOpBatchSize tally.Histogram + status status + serverSupportsV2APIs bool } func newHostQueue( @@ -117,21 +123,26 @@ func newHostQueue( opArrayPool.Init() return &queue{ - opts: opts, - nowFn: opts.ClockOptions().NowFn(), - host: host, - connPool: newConnectionPool(host, opts), - writeBatchRawRequestPool: hostQueueOpts.writeBatchRawRequestPool, - writeBatchRawRequestElementArrayPool: hostQueueOpts.writeBatchRawRequestElementArrayPool, - writeTaggedBatchRawRequestPool: hostQueueOpts.writeTaggedBatchRawRequestPool, - writeTaggedBatchRawRequestElementArrayPool: hostQueueOpts.writeTaggedBatchRawRequestElementArrayPool, - workerPool: workerPool, - size: size, - ops: opArrayPool.Get(), - opsArrayPool: opArrayPool, - writeOpBatchSize: scopeWithoutHostID.Histogram("write-op-batch-size", writeOpBatchSizeBuckets), - fetchOpBatchSize: scopeWithoutHostID.Histogram("fetch-op-batch-size", fetchOpBatchSizeBuckets), - drainIn: make(chan []op, opsArraysLen), + opts: opts, + nowFn: opts.ClockOptions().NowFn(), + host: host, + connPool: newConnectionPool(host, opts), + writeBatchRawRequestPool: hostQueueOpts.writeBatchRawRequestPool, + writeBatchRawV2RequestPool: hostQueueOpts.writeBatchRawV2RequestPool, + writeBatchRawRequestElementArrayPool: hostQueueOpts.writeBatchRawRequestElementArrayPool, + writeBatchRawV2RequestElementArrayPool: hostQueueOpts.writeBatchRawV2RequestElementArrayPool, + writeTaggedBatchRawRequestPool: hostQueueOpts.writeTaggedBatchRawRequestPool, + writeTaggedBatchRawV2RequestPool: hostQueueOpts.writeTaggedBatchRawV2RequestPool, + writeTaggedBatchRawRequestElementArrayPool: hostQueueOpts.writeTaggedBatchRawRequestElementArrayPool, + writeTaggedBatchRawV2RequestElementArrayPool: hostQueueOpts.writeTaggedBatchRawV2RequestElementArrayPool, + workerPool: workerPool, + size: size, + ops: opArrayPool.Get(), + opsArrayPool: opArrayPool, + writeOpBatchSize: scopeWithoutHostID.Histogram("write-op-batch-size", writeOpBatchSizeBuckets), + fetchOpBatchSize: scopeWithoutHostID.Histogram("fetch-op-batch-size", fetchOpBatchSizeBuckets), + drainIn: make(chan []op, opsArraysLen), + serverSupportsV2APIs: opts.UseV2BatchAPIs(), }, nil } @@ -218,9 +229,14 @@ func (q *queue) rotateOpsWithLock() []op { func (q *queue) drain() { var ( + currV2WriteReq *rpc.WriteBatchRawV2Request + currV2WriteOps []op + + currV2WriteTaggedReq *rpc.WriteTaggedBatchRawV2Request + currV2WriteTaggedOps []op + currWriteOpsByNamespace namespaceWriteBatchOpsSlice currTaggedWriteOpsByNamespace namespaceWriteTaggedBatchOpsSlice - writeBatchSize = q.opts.WriteBatchSize() ) for ops := range q.drainIn { @@ -228,46 +244,16 @@ func (q *queue) drain() { for i := 0; i < opsLen; i++ { switch v := ops[i].(type) { case *writeOperation: - namespace := v.namespace - idx := currWriteOpsByNamespace.indexOf(namespace) - if idx == -1 { - value := namespaceWriteBatchOps{ - namespace: namespace, - opsArrayPool: q.opsArrayPool, - writeBatchRawRequestElementArrayPool: q.writeBatchRawRequestElementArrayPool, - } - idx = len(currWriteOpsByNamespace) - currWriteOpsByNamespace = append(currWriteOpsByNamespace, value) - } - - currWriteOpsByNamespace.appendAt(idx, ops[i], &v.request) - - if currWriteOpsByNamespace.lenAt(idx) == writeBatchSize { - // Reached write batch limit, write async and reset - q.asyncWrite(namespace, currWriteOpsByNamespace[idx].ops, - currWriteOpsByNamespace[idx].elems) - currWriteOpsByNamespace.resetAt(idx) + if q.serverSupportsV2APIs { + currV2WriteReq, currV2WriteOps = q.drainWriteOpV2(v, currV2WriteReq, currV2WriteOps, ops[i]) + } else { + currWriteOpsByNamespace = q.drainWriteOpV1(v, currWriteOpsByNamespace, ops[i]) } case *writeTaggedOperation: - namespace := v.namespace - idx := currTaggedWriteOpsByNamespace.indexOf(namespace) - if idx == -1 { - value := namespaceWriteTaggedBatchOps{ - namespace: namespace, - opsArrayPool: q.opsArrayPool, - writeTaggedBatchRawRequestElementArrayPool: q.writeTaggedBatchRawRequestElementArrayPool, - } - idx = len(currTaggedWriteOpsByNamespace) - currTaggedWriteOpsByNamespace = append(currTaggedWriteOpsByNamespace, value) - } - - currTaggedWriteOpsByNamespace.appendAt(idx, ops[i], &v.request) - - if currTaggedWriteOpsByNamespace.lenAt(idx) == writeBatchSize { - // Reached write batch limit, write async and reset - q.asyncTaggedWrite(namespace, currTaggedWriteOpsByNamespace[idx].ops, - currTaggedWriteOpsByNamespace[idx].elems) - currTaggedWriteOpsByNamespace.resetAt(idx) + if q.serverSupportsV2APIs { + currV2WriteTaggedReq, currV2WriteTaggedOps = q.drainTaggedWriteOpV2(v, currV2WriteTaggedReq, currV2WriteTaggedOps, ops[i]) + } else { + currTaggedWriteOpsByNamespace = q.drainTaggedWriteOpV1(v, currTaggedWriteOpsByNamespace, ops[i]) } case *fetchBatchOp: q.asyncFetch(v) @@ -283,7 +269,7 @@ func (q *queue) drain() { } } - // If any outstanding write ops, async write + // If any outstanding write ops, async write. for i, writeOps := range currWriteOpsByNamespace { if len(writeOps.ops) > 0 { q.asyncWrite(writeOps.namespace, writeOps.ops, @@ -294,6 +280,11 @@ func (q *queue) drain() { } // Reset the slice currWriteOpsByNamespace = currWriteOpsByNamespace[:0] + if currV2WriteReq != nil { + q.asyncWriteV2(currV2WriteOps, currV2WriteReq) + currV2WriteReq = nil + currV2WriteOps = nil + } // If any outstanding tagged write ops, async write for i, writeOps := range currTaggedWriteOpsByNamespace { @@ -306,6 +297,11 @@ func (q *queue) drain() { } // Reset the slice currTaggedWriteOpsByNamespace = currTaggedWriteOpsByNamespace[:0] + if currV2WriteTaggedReq != nil { + q.asyncTaggedWriteV2(currV2WriteTaggedOps, currV2WriteTaggedReq) + currV2WriteTaggedReq = nil + currV2WriteTaggedOps = nil + } if ops != nil { q.opsArrayPool.Put(ops) @@ -317,6 +313,136 @@ func (q *queue) drain() { q.connPool.Close() } +func (q *queue) drainWriteOpV1( + v *writeOperation, + currWriteOpsByNamespace namespaceWriteBatchOpsSlice, + op op, +) namespaceWriteBatchOpsSlice { + namespace := v.namespace + idx := currWriteOpsByNamespace.indexOf(namespace) + if idx == -1 { + value := namespaceWriteBatchOps{ + namespace: namespace, + opsArrayPool: q.opsArrayPool, + writeBatchRawRequestElementArrayPool: q.writeBatchRawRequestElementArrayPool, + } + idx = len(currWriteOpsByNamespace) + currWriteOpsByNamespace = append(currWriteOpsByNamespace, value) + } + + currWriteOpsByNamespace.appendAt(idx, op, &v.request) + + if currWriteOpsByNamespace.lenAt(idx) == q.opts.WriteBatchSize() { + // Reached write batch limit, write async and reset. + q.asyncWrite(namespace, currWriteOpsByNamespace[idx].ops, + currWriteOpsByNamespace[idx].elems) + currWriteOpsByNamespace.resetAt(idx) + } + + return currWriteOpsByNamespace +} + +func (q *queue) drainTaggedWriteOpV1( + v *writeTaggedOperation, + currTaggedWriteOpsByNamespace namespaceWriteTaggedBatchOpsSlice, + op op, +) namespaceWriteTaggedBatchOpsSlice { + namespace := v.namespace + idx := currTaggedWriteOpsByNamespace.indexOf(namespace) + if idx == -1 { + value := namespaceWriteTaggedBatchOps{ + namespace: namespace, + opsArrayPool: q.opsArrayPool, + writeTaggedBatchRawRequestElementArrayPool: q.writeTaggedBatchRawRequestElementArrayPool, + } + idx = len(currTaggedWriteOpsByNamespace) + currTaggedWriteOpsByNamespace = append(currTaggedWriteOpsByNamespace, value) + } + + currTaggedWriteOpsByNamespace.appendAt(idx, op, &v.request) + + if currTaggedWriteOpsByNamespace.lenAt(idx) == q.opts.WriteBatchSize() { + // Reached write batch limit, write async and reset + q.asyncTaggedWrite(namespace, currTaggedWriteOpsByNamespace[idx].ops, + currTaggedWriteOpsByNamespace[idx].elems) + currTaggedWriteOpsByNamespace.resetAt(idx) + } + + return currTaggedWriteOpsByNamespace +} + +func (q *queue) drainWriteOpV2( + v *writeOperation, + currV2WriteReq *rpc.WriteBatchRawV2Request, + currV2WriteOps []op, + op op, +) (*rpc.WriteBatchRawV2Request, []op) { + namespace := v.namespace + if currV2WriteReq == nil { + currV2WriteReq = q.writeBatchRawV2RequestPool.Get() + currV2WriteReq.Elements = q.writeBatchRawV2RequestElementArrayPool.Get() + } + + nsIdx := -1 + for i, ns := range currV2WriteReq.NameSpaces { + if bytes.Equal(namespace.Bytes(), ns) { + nsIdx = i + break + } + } + if nsIdx == -1 { + currV2WriteReq.NameSpaces = append(currV2WriteReq.NameSpaces, namespace.Bytes()) + nsIdx = len(currV2WriteReq.NameSpaces) - 1 + } + v.requestV2.NameSpace = int64(nsIdx) + currV2WriteReq.Elements = append(currV2WriteReq.Elements, &v.requestV2) + currV2WriteOps = append(currV2WriteOps, op) + if len(currV2WriteReq.Elements) == q.opts.WriteBatchSize() { + // Reached write batch limit, write async and reset. + q.asyncWriteV2(currV2WriteOps, currV2WriteReq) + currV2WriteReq = nil + currV2WriteOps = nil + } + + return currV2WriteReq, currV2WriteOps +} + +func (q *queue) drainTaggedWriteOpV2( + v *writeTaggedOperation, + currV2WriteTaggedReq *rpc.WriteTaggedBatchRawV2Request, + currV2WriteTaggedOps []op, + op op, +) (*rpc.WriteTaggedBatchRawV2Request, []op) { + namespace := v.namespace + if currV2WriteTaggedReq == nil { + currV2WriteTaggedReq = q.writeTaggedBatchRawV2RequestPool.Get() + currV2WriteTaggedReq.Elements = q.writeTaggedBatchRawV2RequestElementArrayPool.Get() + } + + nsIdx := -1 + for i, ns := range currV2WriteTaggedReq.NameSpaces { + if bytes.Equal(namespace.Bytes(), ns) { + nsIdx = i + break + } + } + if nsIdx == -1 { + currV2WriteTaggedReq.NameSpaces = append(currV2WriteTaggedReq.NameSpaces, namespace.Bytes()) + nsIdx = len(currV2WriteTaggedReq.NameSpaces) - 1 + } + v.requestV2.NameSpace = int64(nsIdx) + currV2WriteTaggedReq.Elements = append(currV2WriteTaggedReq.Elements, &v.requestV2) + currV2WriteTaggedOps = append(currV2WriteTaggedOps, op) + if len(currV2WriteTaggedReq.Elements) == q.opts.WriteBatchSize() { + // Reached write batch limit, write async and reset. + q.asyncTaggedWriteV2(currV2WriteTaggedOps, currV2WriteTaggedReq) + currV2WriteTaggedReq = nil + currV2WriteTaggedOps = nil + } + + return currV2WriteTaggedReq, currV2WriteTaggedOps +} + func (q *queue) asyncTaggedWrite( namespace ident.ID, ops []op, @@ -383,6 +509,66 @@ func (q *queue) asyncTaggedWrite( }) } +func (q *queue) asyncTaggedWriteV2( + ops []op, + req *rpc.WriteTaggedBatchRawV2Request, +) { + q.writeOpBatchSize.RecordValue(float64(len(req.Elements))) + q.Add(1) + + q.workerPool.Go(func() { + // NB(r): Defer is slow in the hot path unfortunately + cleanup := func() { + q.writeTaggedBatchRawV2RequestPool.Put(req) + q.writeTaggedBatchRawV2RequestElementArrayPool.Put(req.Elements) + q.opsArrayPool.Put(ops) + q.Done() + } + + // NB(bl): host is passed to writeState to determine the state of the + // shard on the node we're writing to. + client, err := q.connPool.NextClient() + if err != nil { + // No client available + callAllCompletionFns(ops, q.host, err) + cleanup() + return + } + + ctx, _ := thrift.NewContext(q.opts.WriteRequestTimeout()) + err = client.WriteTaggedBatchRawV2(ctx, req) + if err == nil { + // All succeeded + callAllCompletionFns(ops, q.host, nil) + cleanup() + return + } + + if batchErrs, ok := err.(*rpc.WriteBatchRawErrors); ok { + // Callback all writes with errors + hasErr := make(map[int]struct{}) + for _, batchErr := range batchErrs.Errors { + op := ops[batchErr.Index] + op.CompletionFn()(q.host, batchErr.Err) + hasErr[int(batchErr.Index)] = struct{}{} + } + // Callback all writes with no errors + for i := range ops { + if _, ok := hasErr[i]; !ok { + // No error + ops[i].CompletionFn()(q.host, nil) + } + } + cleanup() + return + } + + // Entire batch failed + callAllCompletionFns(ops, q.host, err) + cleanup() + }) +} + func (q *queue) asyncWrite( namespace ident.ID, ops []op, @@ -448,6 +634,65 @@ func (q *queue) asyncWrite( }) } +func (q *queue) asyncWriteV2( + ops []op, + req *rpc.WriteBatchRawV2Request, +) { + q.writeOpBatchSize.RecordValue(float64(len(req.Elements))) + q.Add(1) + q.workerPool.Go(func() { + // NB(r): Defer is slow in the hot path unfortunately + cleanup := func() { + q.writeBatchRawV2RequestPool.Put(req) + q.writeBatchRawV2RequestElementArrayPool.Put(req.Elements) + q.opsArrayPool.Put(ops) + q.Done() + } + + // NB(bl): host is passed to writeState to determine the state of the + // shard on the node we're writing to. + client, err := q.connPool.NextClient() + if err != nil { + // No client available. + callAllCompletionFns(ops, q.host, err) + cleanup() + return + } + + ctx, _ := thrift.NewContext(q.opts.WriteRequestTimeout()) + err = client.WriteBatchRawV2(ctx, req) + if err == nil { + // All succeeded. + callAllCompletionFns(ops, q.host, nil) + cleanup() + return + } + + if batchErrs, ok := err.(*rpc.WriteBatchRawErrors); ok { + // Callback all writes with errors. + hasErr := make(map[int]struct{}) + for _, batchErr := range batchErrs.Errors { + op := ops[batchErr.Index] + op.CompletionFn()(q.host, batchErr.Err) + hasErr[int(batchErr.Index)] = struct{}{} + } + // Callback all writes with no errors. + for i := range ops { + if _, ok := hasErr[i]; !ok { + // No error + ops[i].CompletionFn()(q.host, nil) + } + } + cleanup() + return + } + + // Entire batch failed. + callAllCompletionFns(ops, q.host, err) + cleanup() + }) +} + func (q *queue) asyncFetch(op *fetchBatchOp) { q.fetchOpBatchSize.RecordValue(float64(len(op.request.Ids))) q.Add(1) diff --git a/src/dbnode/client/host_queue_test.go b/src/dbnode/client/host_queue_test.go index 86fddc2422..bf78248acf 100644 --- a/src/dbnode/client/host_queue_test.go +++ b/src/dbnode/client/host_queue_test.go @@ -23,22 +23,41 @@ package client import "github.com/m3db/m3/src/x/pool" var ( - smallPoolOptions = pool.NewObjectPoolOptions().SetSize(1) - testWriteBatchRawPool writeBatchRawRequestPool - testWriteArrayPool writeBatchRawRequestElementArrayPool - testWriteTaggedBatchRawPool writeTaggedBatchRawRequestPool - testWriteTaggedArrayPool writeTaggedBatchRawRequestElementArrayPool + smallPoolOptions = pool.NewObjectPoolOptions().SetSize(1) + + testWriteBatchRawPool writeBatchRawRequestPool + testWriteBatchRawV2Pool writeBatchRawV2RequestPool + + testWriteArrayPool writeBatchRawRequestElementArrayPool + testWriteV2ArrayPool writeBatchRawV2RequestElementArrayPool + + testWriteTaggedBatchRawPool writeTaggedBatchRawRequestPool + testWriteTaggedBatchRawV2Pool writeTaggedBatchRawV2RequestPool + + testWriteTaggedArrayPool writeTaggedBatchRawRequestElementArrayPool + testWriteTaggedV2ArrayPool writeTaggedBatchRawV2RequestElementArrayPool ) func init() { testWriteBatchRawPool = newWriteBatchRawRequestPool(smallPoolOptions) testWriteBatchRawPool.Init() + testWriteBatchRawV2Pool = newWriteBatchRawV2RequestPool(smallPoolOptions) + testWriteBatchRawV2Pool.Init() + testWriteArrayPool = newWriteBatchRawRequestElementArrayPool(smallPoolOptions, 0) testWriteArrayPool.Init() + testWriteV2ArrayPool = newWriteBatchRawV2RequestElementArrayPool(smallPoolOptions, 0) + testWriteV2ArrayPool.Init() + testWriteTaggedBatchRawPool = newWriteTaggedBatchRawRequestPool(smallPoolOptions) testWriteTaggedBatchRawPool.Init() + testWriteTaggedBatchRawV2Pool = newWriteTaggedBatchRawV2RequestPool(smallPoolOptions) + testWriteTaggedBatchRawV2Pool.Init() + testWriteTaggedArrayPool = newWriteTaggedBatchRawRequestElementArrayPool(smallPoolOptions, 0) testWriteTaggedArrayPool.Init() + testWriteTaggedV2ArrayPool = newWriteTaggedBatchRawV2RequestElementArrayPool(smallPoolOptions, 0) + testWriteTaggedV2ArrayPool.Init() } type hostQueueResult struct { diff --git a/src/dbnode/client/host_queue_write_batch_test.go b/src/dbnode/client/host_queue_write_batch_test.go index 6861b16726..f31f69a1e6 100644 --- a/src/dbnode/client/host_queue_write_batch_test.go +++ b/src/dbnode/client/host_queue_write_batch_test.go @@ -49,159 +49,202 @@ func TestHostQueueWriteErrorAfterClose(t *testing.T) { } func TestHostQueueWriteBatches(t *testing.T) { - ctrl := gomock.NewController(t) - defer ctrl.Finish() - - mockConnPool := NewMockconnectionPool(ctrl) - opts := newHostQueueTestOptions() - queue := newTestHostQueue(opts) - queue.connPool = mockConnPool - - // Open - mockConnPool.EXPECT().Open() - queue.Open() - assert.Equal(t, statusOpen, queue.status) - - // Prepare callback for writes - var ( - results []hostQueueResult - wg sync.WaitGroup - ) - callback := func(r interface{}, err error) { - results = append(results, hostQueueResult{r, err}) - wg.Done() - } - - // Prepare writes - writes := []*writeOperation{ - testWriteOp("testNs", "foo", 1.0, 1000, rpc.TimeType_UNIX_SECONDS, callback), - testWriteOp("testNs", "bar", 2.0, 2000, rpc.TimeType_UNIX_SECONDS, callback), - testWriteOp("testNs", "baz", 3.0, 3000, rpc.TimeType_UNIX_SECONDS, callback), - testWriteOp("testNs", "qux", 4.0, 4000, rpc.TimeType_UNIX_SECONDS, callback), - } - wg.Add(len(writes)) - - for i, write := range writes[:3] { - assert.NoError(t, queue.Enqueue(write)) - assert.Equal(t, i+1, queue.Len()) - - // Sleep some so that we can ensure flushing is not happening until queue is full - time.Sleep(20 * time.Millisecond) - } - - // Prepare mocks for flush - mockClient := rpc.NewMockTChanNode(ctrl) - writeBatch := func(ctx thrift.Context, req *rpc.WriteBatchRawRequest) { - for i, write := range writes { - assert.Equal(t, req.Elements[i].ID, write.request.ID) - assert.Equal(t, req.Elements[i].Datapoint, write.request.Datapoint) - } + for _, opts := range []Options{ + newHostQueueTestOptions().SetUseV2BatchAPIs(false), + newHostQueueTestOptions().SetUseV2BatchAPIs(true), + } { + t.Run(fmt.Sprintf("useV2: %v", opts.UseV2BatchAPIs()), func(t *testing.T) { + ctrl := gomock.NewController(t) + defer ctrl.Finish() + + mockConnPool := NewMockconnectionPool(ctrl) + opts := newHostQueueTestOptions() + queue := newTestHostQueue(opts) + queue.connPool = mockConnPool + + // Open + mockConnPool.EXPECT().Open() + queue.Open() + assert.Equal(t, statusOpen, queue.status) + + // Prepare callback for writes + var ( + results []hostQueueResult + wg sync.WaitGroup + ) + callback := func(r interface{}, err error) { + results = append(results, hostQueueResult{r, err}) + wg.Done() + } + + // Prepare writes + writes := []*writeOperation{ + testWriteOp("testNs", "foo", 1.0, 1000, rpc.TimeType_UNIX_SECONDS, callback), + testWriteOp("testNs", "bar", 2.0, 2000, rpc.TimeType_UNIX_SECONDS, callback), + testWriteOp("testNs", "baz", 3.0, 3000, rpc.TimeType_UNIX_SECONDS, callback), + testWriteOp("testNs", "qux", 4.0, 4000, rpc.TimeType_UNIX_SECONDS, callback), + } + wg.Add(len(writes)) + + for i, write := range writes[:3] { + assert.NoError(t, queue.Enqueue(write)) + assert.Equal(t, i+1, queue.Len()) + + // Sleep some so that we can ensure flushing is not happening until queue is full + time.Sleep(20 * time.Millisecond) + } + + // Prepare mocks for flush + mockClient := rpc.NewMockTChanNode(ctrl) + if opts.UseV2BatchAPIs() { + writeBatch := func(ctx thrift.Context, req *rpc.WriteBatchRawV2Request) { + for i, write := range writes { + assert.Equal(t, req.Elements[i].NameSpace, 0) + assert.Equal(t, req.Elements[i].ID, write.request.ID) + assert.Equal(t, req.Elements[i].Datapoint, write.request.Datapoint) + } + } + mockClient.EXPECT().WriteBatchRawV2(gomock.Any(), gomock.Any()).Do(writeBatch).Return(nil) + } else { + writeBatch := func(ctx thrift.Context, req *rpc.WriteBatchRawRequest) { + for i, write := range writes { + assert.Equal(t, req.Elements[i].ID, write.request.ID) + assert.Equal(t, req.Elements[i].Datapoint, write.request.Datapoint) + } + } + mockClient.EXPECT().WriteBatchRaw(gomock.Any(), gomock.Any()).Do(writeBatch).Return(nil) + } + + mockConnPool.EXPECT().NextClient().Return(mockClient, nil) + + // Final write will flush + assert.NoError(t, queue.Enqueue(writes[3])) + assert.Equal(t, 0, queue.Len()) + + // Wait for all writes + wg.Wait() + + // Assert writes successful + assert.Equal(t, len(writes), len(results)) + for _, result := range results { + assert.Nil(t, result.err) + } + + // Close + var closeWg sync.WaitGroup + closeWg.Add(1) + mockConnPool.EXPECT().Close().Do(func() { + closeWg.Done() + }) + queue.Close() + closeWg.Wait() + }) } - mockClient.EXPECT().WriteBatchRaw(gomock.Any(), gomock.Any()).Do(writeBatch).Return(nil) - - mockConnPool.EXPECT().NextClient().Return(mockClient, nil) - - // Final write will flush - assert.NoError(t, queue.Enqueue(writes[3])) - assert.Equal(t, 0, queue.Len()) - - // Wait for all writes - wg.Wait() - - // Assert writes successful - assert.Equal(t, len(writes), len(results)) - for _, result := range results { - assert.Nil(t, result.err) - } - - // Close - var closeWg sync.WaitGroup - closeWg.Add(1) - mockConnPool.EXPECT().Close().Do(func() { - closeWg.Done() - }) - queue.Close() - closeWg.Wait() } func TestHostQueueWriteBatchesDifferentNamespaces(t *testing.T) { - ctrl := gomock.NewController(t) - defer ctrl.Finish() - - mockConnPool := NewMockconnectionPool(ctrl) - - opts := newHostQueueTestOptions() - queue := newTestHostQueue(opts) - queue.connPool = mockConnPool - - // Open - mockConnPool.EXPECT().Open() - queue.Open() - assert.Equal(t, statusOpen, queue.status) - - // Prepare callback for writes - var ( - results []hostQueueResult - resultsLock sync.Mutex - wg sync.WaitGroup - ) - callback := func(r interface{}, err error) { - resultsLock.Lock() - results = append(results, hostQueueResult{r, err}) - resultsLock.Unlock() - wg.Done() - } - - // Prepare writes - writes := []*writeOperation{ - testWriteOp("testNs1", "foo", 1.0, 1000, rpc.TimeType_UNIX_SECONDS, callback), - testWriteOp("testNs1", "bar", 2.0, 2000, rpc.TimeType_UNIX_SECONDS, callback), - testWriteOp("testNs1", "baz", 3.0, 3000, rpc.TimeType_UNIX_SECONDS, callback), - testWriteOp("testNs2", "qux", 4.0, 4000, rpc.TimeType_UNIX_SECONDS, callback), - } - wg.Add(len(writes)) - - // Prepare mocks for flush - mockClient := rpc.NewMockTChanNode(ctrl) - writeBatch := func(ctx thrift.Context, req *rpc.WriteBatchRawRequest) { - var writesForNamespace []*writeOperation - if string(req.NameSpace) == "testNs1" { - writesForNamespace = writes[:3] - } else { - writesForNamespace = writes[3:] - } - assert.Equal(t, len(writesForNamespace), len(req.Elements)) - for i, write := range writesForNamespace { - assert.Equal(t, req.Elements[i].ID, write.request.ID) - assert.Equal(t, req.Elements[i].Datapoint, write.request.Datapoint) - } - } - - // Assert the writes will be handled in two batches - mockClient.EXPECT().WriteBatchRaw(gomock.Any(), gomock.Any()).Do(writeBatch).Return(nil).MinTimes(2).MaxTimes(2) - mockConnPool.EXPECT().NextClient().Return(mockClient, nil).MinTimes(2).MaxTimes(2) - - for _, write := range writes { - assert.NoError(t, queue.Enqueue(write)) + for _, opts := range []Options{ + newHostQueueTestOptions().SetUseV2BatchAPIs(false), + newHostQueueTestOptions().SetUseV2BatchAPIs(true), + } { + t.Run(fmt.Sprintf("useV2: %v", opts.UseV2BatchAPIs()), func(t *testing.T) { + ctrl := gomock.NewController(t) + defer ctrl.Finish() + + mockConnPool := NewMockconnectionPool(ctrl) + + queue := newTestHostQueue(opts) + queue.connPool = mockConnPool + + // Open + mockConnPool.EXPECT().Open() + queue.Open() + assert.Equal(t, statusOpen, queue.status) + + // Prepare callback for writes + var ( + results []hostQueueResult + resultsLock sync.Mutex + wg sync.WaitGroup + ) + callback := func(r interface{}, err error) { + resultsLock.Lock() + results = append(results, hostQueueResult{r, err}) + resultsLock.Unlock() + wg.Done() + } + + // Prepare writes + writes := []*writeOperation{ + testWriteOp("testNs1", "foo", 1.0, 1000, rpc.TimeType_UNIX_SECONDS, callback), + testWriteOp("testNs1", "bar", 2.0, 2000, rpc.TimeType_UNIX_SECONDS, callback), + testWriteOp("testNs1", "baz", 3.0, 3000, rpc.TimeType_UNIX_SECONDS, callback), + testWriteOp("testNs2", "qux", 4.0, 4000, rpc.TimeType_UNIX_SECONDS, callback), + } + wg.Add(len(writes)) + + // Prepare mocks for flush + mockClient := rpc.NewMockTChanNode(ctrl) + + if opts.UseV2BatchAPIs() { + writeBatch := func(ctx thrift.Context, req *rpc.WriteBatchRawV2Request) { + for i, write := range writes { + if i < 3 { + assert.Equal(t, req.Elements[i].NameSpace, int64(0)) + } else { + assert.Equal(t, req.Elements[i].NameSpace, int64(1)) + } + assert.Equal(t, req.Elements[i].ID, write.request.ID) + assert.Equal(t, req.Elements[i].Datapoint, write.request.Datapoint) + } + } + + // Assert the writes will be handled in two batches + mockClient.EXPECT().WriteBatchRawV2(gomock.Any(), gomock.Any()).Do(writeBatch).Return(nil).Times(1) + mockConnPool.EXPECT().NextClient().Return(mockClient, nil).Times(1) + } else { + writeBatch := func(ctx thrift.Context, req *rpc.WriteBatchRawRequest) { + var writesForNamespace []*writeOperation + if string(req.NameSpace) == "testNs1" { + writesForNamespace = writes[:3] + } else { + writesForNamespace = writes[3:] + } + assert.Equal(t, len(writesForNamespace), len(req.Elements)) + for i, write := range writesForNamespace { + assert.Equal(t, req.Elements[i].ID, write.request.ID) + assert.Equal(t, req.Elements[i].Datapoint, write.request.Datapoint) + } + } + + // Assert the writes will be handled in two batches + mockClient.EXPECT().WriteBatchRaw(gomock.Any(), gomock.Any()).Do(writeBatch).Return(nil).Times(2) + mockConnPool.EXPECT().NextClient().Return(mockClient, nil).Times(2) + } + + for _, write := range writes { + assert.NoError(t, queue.Enqueue(write)) + } + + // Wait for all writes + wg.Wait() + + // Assert writes successful + assert.Equal(t, len(writes), len(results)) + for _, result := range results { + assert.Nil(t, result.err) + } + + // Close + var closeWg sync.WaitGroup + closeWg.Add(1) + mockConnPool.EXPECT().Close().Do(func() { + closeWg.Done() + }) + queue.Close() + closeWg.Wait() + }) } - - // Wait for all writes - wg.Wait() - - // Assert writes successful - assert.Equal(t, len(writes), len(results)) - for _, result := range results { - assert.Nil(t, result.err) - } - - // Close - var closeWg sync.WaitGroup - closeWg.Add(1) - mockConnPool.EXPECT().Close().Do(func() { - closeWg.Done() - }) - queue.Close() - closeWg.Wait() } func TestHostQueueWriteBatchesNoClientAvailable(t *testing.T) { @@ -248,73 +291,90 @@ func TestHostQueueWriteBatchesNoClientAvailable(t *testing.T) { } func TestHostQueueWriteBatchesPartialBatchErrs(t *testing.T) { - ctrl := gomock.NewController(t) - defer ctrl.Finish() - - mockConnPool := NewMockconnectionPool(ctrl) - - opts := newHostQueueTestOptions() - opts = opts.SetHostQueueOpsFlushSize(2) - queue := newTestHostQueue(opts) - queue.connPool = mockConnPool - - // Open - mockConnPool.EXPECT().Open() - queue.Open() - assert.Equal(t, statusOpen, queue.status) - - // Prepare writes - var wg sync.WaitGroup - writeErr := "a write error" - writes := []*writeOperation{ - testWriteOp("testNs", "foo", 1.0, 1000, rpc.TimeType_UNIX_SECONDS, func(r interface{}, err error) { - assert.Error(t, err) - rpcErr, ok := err.(*rpc.Error) - assert.True(t, ok) - assert.Equal(t, rpc.ErrorType_INTERNAL_ERROR, rpcErr.Type) - assert.Equal(t, writeErr, rpcErr.Message) - wg.Done() - }), - testWriteOp("testNs", "bar", 2.0, 2000, rpc.TimeType_UNIX_SECONDS, func(r interface{}, err error) { - assert.NoError(t, err) - wg.Done() - }), - } - wg.Add(len(writes)) - - // Prepare mocks for flush - mockClient := rpc.NewMockTChanNode(ctrl) - writeBatch := func(ctx thrift.Context, req *rpc.WriteBatchRawRequest) { - for i, write := range writes { - assert.Equal(t, req.Elements[i].ID, write.request.ID) - assert.Equal(t, req.Elements[i].Datapoint, write.request.Datapoint) - } + for _, opts := range []Options{ + newHostQueueTestOptions().SetUseV2BatchAPIs(false), + newHostQueueTestOptions().SetUseV2BatchAPIs(true), + } { + t.Run(fmt.Sprintf("useV2: %v", opts.UseV2BatchAPIs()), func(t *testing.T) { + ctrl := gomock.NewController(t) + defer ctrl.Finish() + + mockConnPool := NewMockconnectionPool(ctrl) + + opts = opts.SetHostQueueOpsFlushSize(2) + queue := newTestHostQueue(opts) + queue.connPool = mockConnPool + + // Open + mockConnPool.EXPECT().Open() + queue.Open() + assert.Equal(t, statusOpen, queue.status) + + // Prepare writes + var wg sync.WaitGroup + writeErr := "a write error" + writes := []*writeOperation{ + testWriteOp("testNs", "foo", 1.0, 1000, rpc.TimeType_UNIX_SECONDS, func(r interface{}, err error) { + assert.Error(t, err) + rpcErr, ok := err.(*rpc.Error) + assert.True(t, ok) + assert.Equal(t, rpc.ErrorType_INTERNAL_ERROR, rpcErr.Type) + assert.Equal(t, writeErr, rpcErr.Message) + wg.Done() + }), + testWriteOp("testNs", "bar", 2.0, 2000, rpc.TimeType_UNIX_SECONDS, func(r interface{}, err error) { + assert.NoError(t, err) + wg.Done() + }), + } + wg.Add(len(writes)) + + // Prepare mocks for flush + mockClient := rpc.NewMockTChanNode(ctrl) + batchErrs := &rpc.WriteBatchRawErrors{Errors: []*rpc.WriteBatchRawError{ + &rpc.WriteBatchRawError{Index: 0, Err: &rpc.Error{ + Type: rpc.ErrorType_INTERNAL_ERROR, + Message: writeErr, + }}, + }} + if opts.UseV2BatchAPIs() { + writeBatch := func(ctx thrift.Context, req *rpc.WriteBatchRawV2Request) { + for i, write := range writes { + assert.Equal(t, req.Elements[i].NameSpace, int64(0)) + assert.Equal(t, req.Elements[i].ID, write.request.ID) + assert.Equal(t, req.Elements[i].Datapoint, write.request.Datapoint) + } + } + mockClient.EXPECT().WriteBatchRawV2(gomock.Any(), gomock.Any()).Do(writeBatch).Return(batchErrs) + } else { + writeBatch := func(ctx thrift.Context, req *rpc.WriteBatchRawRequest) { + for i, write := range writes { + assert.Equal(t, req.Elements[i].ID, write.request.ID) + assert.Equal(t, req.Elements[i].Datapoint, write.request.Datapoint) + } + } + mockClient.EXPECT().WriteBatchRaw(gomock.Any(), gomock.Any()).Do(writeBatch).Return(batchErrs) + } + mockConnPool.EXPECT().NextClient().Return(mockClient, nil) + + // Perform writes + for _, write := range writes { + assert.NoError(t, queue.Enqueue(write)) + } + + // Wait for flush + wg.Wait() + + // Close + var closeWg sync.WaitGroup + closeWg.Add(1) + mockConnPool.EXPECT().Close().Do(func() { + closeWg.Done() + }) + queue.Close() + closeWg.Wait() + }) } - batchErrs := &rpc.WriteBatchRawErrors{Errors: []*rpc.WriteBatchRawError{ - &rpc.WriteBatchRawError{Index: 0, Err: &rpc.Error{ - Type: rpc.ErrorType_INTERNAL_ERROR, - Message: writeErr, - }}, - }} - mockClient.EXPECT().WriteBatchRaw(gomock.Any(), gomock.Any()).Do(writeBatch).Return(batchErrs) - mockConnPool.EXPECT().NextClient().Return(mockClient, nil) - - // Perform writes - for _, write := range writes { - assert.NoError(t, queue.Enqueue(write)) - } - - // Wait for flush - wg.Wait() - - // Close - var closeWg sync.WaitGroup - closeWg.Add(1) - mockConnPool.EXPECT().Close().Do(func() { - closeWg.Done() - }) - queue.Close() - closeWg.Wait() } func TestHostQueueWriteBatchesEntireBatchErr(t *testing.T) { @@ -472,6 +532,8 @@ func testWriteOp( Timestamp: timestamp, TimestampTimeType: timeType, } + w.requestV2.ID = w.request.ID + w.requestV2.Datapoint = w.request.Datapoint w.completionFn = completionFn return w } diff --git a/src/dbnode/client/host_queue_write_tagged_test.go b/src/dbnode/client/host_queue_write_tagged_test.go index fae195e154..46f8ae5d03 100644 --- a/src/dbnode/client/host_queue_write_tagged_test.go +++ b/src/dbnode/client/host_queue_write_tagged_test.go @@ -51,184 +51,223 @@ func TestHostQueueWriteTaggedErrorAfterClose(t *testing.T) { } func TestHostQueueWriteTaggedBatches(t *testing.T) { - ctrl := gomock.NewController(t) - defer ctrl.Finish() - - mockConnPool := NewMockconnectionPool(ctrl) - - opts := newHostQueueTestOptions() - queue := newTestHostQueue(opts) - queue.connPool = mockConnPool - - // Open - mockConnPool.EXPECT().Open() - queue.Open() - assert.Equal(t, statusOpen, queue.status) - - // Prepare callback for writes - var ( - results []hostQueueResult - wg sync.WaitGroup - ) - callback := func(r interface{}, err error) { - results = append(results, hostQueueResult{r, err}) - wg.Done() - } - - // Prepare writes - writes := []*writeTaggedOperation{ - testWriteTaggedOp("testNs", "foo", map[string]string{ - "tag": "value", - "sup": "holmes", - }, 1.0, 1000, rpc.TimeType_UNIX_SECONDS, callback), - testWriteTaggedOp("testNs", "bar", map[string]string{ - "and": "one", - }, 2.0, 2000, rpc.TimeType_UNIX_SECONDS, callback), - testWriteTaggedOp("testNs", "baz", map[string]string{ - "cmon": "dawg", - "mmm": "kay", - }, 3.0, 3000, rpc.TimeType_UNIX_SECONDS, callback), - testWriteTaggedOp("testNs", "qux", map[string]string{ - "mas": "ter", - "sal": "vo", - }, 4.0, 4000, rpc.TimeType_UNIX_SECONDS, callback), - } - wg.Add(len(writes)) - - for i, write := range writes[:3] { - assert.NoError(t, queue.Enqueue(write)) - assert.Equal(t, i+1, queue.Len()) - - // Sleep some so that we can ensure flushing is not happening until queue is full - time.Sleep(20 * time.Millisecond) - } - - // Prepare mocks for flush - mockClient := rpc.NewMockTChanNode(ctrl) - writeBatch := func(ctx thrift.Context, req *rpc.WriteTaggedBatchRawRequest) { - for i, write := range writes { - assert.Equal(t, req.Elements[i].ID, write.request.ID) - assert.Equal(t, req.Elements[i].Datapoint, write.request.Datapoint) - assert.Equal(t, req.Elements[i].EncodedTags, write.request.EncodedTags) - } + for _, opts := range []Options{ + newHostQueueTestOptions().SetUseV2BatchAPIs(false), + newHostQueueTestOptions().SetUseV2BatchAPIs(true), + } { + t.Run(fmt.Sprintf("useV2: %v", opts.UseV2BatchAPIs()), func(t *testing.T) { + ctrl := gomock.NewController(t) + defer ctrl.Finish() + + mockConnPool := NewMockconnectionPool(ctrl) + + queue := newTestHostQueue(opts) + queue.connPool = mockConnPool + + // Open + mockConnPool.EXPECT().Open() + queue.Open() + assert.Equal(t, statusOpen, queue.status) + + // Prepare callback for writes + var ( + results []hostQueueResult + wg sync.WaitGroup + ) + callback := func(r interface{}, err error) { + results = append(results, hostQueueResult{r, err}) + wg.Done() + } + + // Prepare writes + writes := []*writeTaggedOperation{ + testWriteTaggedOp("testNs", "foo", map[string]string{ + "tag": "value", + "sup": "holmes", + }, 1.0, 1000, rpc.TimeType_UNIX_SECONDS, callback), + testWriteTaggedOp("testNs", "bar", map[string]string{ + "and": "one", + }, 2.0, 2000, rpc.TimeType_UNIX_SECONDS, callback), + testWriteTaggedOp("testNs", "baz", map[string]string{ + "cmon": "dawg", + "mmm": "kay", + }, 3.0, 3000, rpc.TimeType_UNIX_SECONDS, callback), + testWriteTaggedOp("testNs", "qux", map[string]string{ + "mas": "ter", + "sal": "vo", + }, 4.0, 4000, rpc.TimeType_UNIX_SECONDS, callback), + } + wg.Add(len(writes)) + + for i, write := range writes[:3] { + assert.NoError(t, queue.Enqueue(write)) + assert.Equal(t, i+1, queue.Len()) + + // Sleep some so that we can ensure flushing is not happening until queue is full + time.Sleep(20 * time.Millisecond) + } + + // Prepare mocks for flush + mockClient := rpc.NewMockTChanNode(ctrl) + if opts.UseV2BatchAPIs() { + writeBatch := func(ctx thrift.Context, req *rpc.WriteTaggedBatchRawV2Request) { + for i, write := range writes { + assert.Equal(t, req.Elements[i].NameSpace, int64(0)) + assert.Equal(t, req.Elements[i].ID, write.request.ID) + assert.Equal(t, req.Elements[i].Datapoint, write.request.Datapoint) + assert.Equal(t, req.Elements[i].EncodedTags, write.request.EncodedTags) + } + } + mockClient.EXPECT().WriteTaggedBatchRawV2(gomock.Any(), gomock.Any()).Do(writeBatch).Return(nil) + } else { + writeBatch := func(ctx thrift.Context, req *rpc.WriteTaggedBatchRawRequest) { + for i, write := range writes { + assert.Equal(t, req.Elements[i].ID, write.request.ID) + assert.Equal(t, req.Elements[i].Datapoint, write.request.Datapoint) + assert.Equal(t, req.Elements[i].EncodedTags, write.request.EncodedTags) + } + } + mockClient.EXPECT().WriteTaggedBatchRaw(gomock.Any(), gomock.Any()).Do(writeBatch).Return(nil) + } + mockConnPool.EXPECT().NextClient().Return(mockClient, nil) + + // Final write will flush + assert.NoError(t, queue.Enqueue(writes[3])) + assert.Equal(t, 0, queue.Len()) + + // Wait for all writes + wg.Wait() + + // Assert writes successful + assert.Equal(t, len(writes), len(results)) + for _, result := range results { + assert.Nil(t, result.err) + } + + // Close + var closeWg sync.WaitGroup + closeWg.Add(1) + mockConnPool.EXPECT().Close().Do(func() { + closeWg.Done() + }) + queue.Close() + closeWg.Wait() + }) } - mockClient.EXPECT().WriteTaggedBatchRaw(gomock.Any(), gomock.Any()).Do(writeBatch).Return(nil) - - mockConnPool.EXPECT().NextClient().Return(mockClient, nil) - - // Final write will flush - assert.NoError(t, queue.Enqueue(writes[3])) - assert.Equal(t, 0, queue.Len()) - - // Wait for all writes - wg.Wait() - - // Assert writes successful - assert.Equal(t, len(writes), len(results)) - for _, result := range results { - assert.Nil(t, result.err) - } - - // Close - var closeWg sync.WaitGroup - closeWg.Add(1) - mockConnPool.EXPECT().Close().Do(func() { - closeWg.Done() - }) - queue.Close() - closeWg.Wait() } func TestHostQueueWriteTaggedBatchesDifferentNamespaces(t *testing.T) { - ctrl := gomock.NewController(t) - defer ctrl.Finish() - - mockConnPool := NewMockconnectionPool(ctrl) - - opts := newHostQueueTestOptions() - queue := newTestHostQueue(opts) - queue.connPool = mockConnPool - - // Open - mockConnPool.EXPECT().Open() - queue.Open() - assert.Equal(t, statusOpen, queue.status) - - // Prepare callback for writes - var ( - results []hostQueueResult - resultsLock sync.Mutex - wg sync.WaitGroup - ) - callback := func(r interface{}, err error) { - resultsLock.Lock() - results = append(results, hostQueueResult{r, err}) - resultsLock.Unlock() - wg.Done() - } - - // Prepare writes - writes := []*writeTaggedOperation{ - testWriteTaggedOp("testNs1", "foo", map[string]string{ - "tag": "value", - "sup": "holmes", - }, 1.0, 1000, rpc.TimeType_UNIX_SECONDS, callback), - testWriteTaggedOp("testNs1", "bar", map[string]string{ - "and": "one", - }, 2.0, 2000, rpc.TimeType_UNIX_SECONDS, callback), - testWriteTaggedOp("testNs1", "baz", map[string]string{ - "cmon": "dawg", - "mmm": "kay", - }, 3.0, 3000, rpc.TimeType_UNIX_SECONDS, callback), - testWriteTaggedOp("testNs2", "qux", map[string]string{ - "mas": "ter", - "sal": "vo", - }, 4.0, 4000, rpc.TimeType_UNIX_SECONDS, callback), - } - wg.Add(len(writes)) - - // Prepare mocks for flush - mockClient := rpc.NewMockTChanNode(ctrl) - writeBatch := func(ctx thrift.Context, req *rpc.WriteTaggedBatchRawRequest) { - var writesForNamespace []*writeTaggedOperation - if string(req.NameSpace) == "testNs1" { - writesForNamespace = writes[:3] - } else { - writesForNamespace = writes[3:] - } - assert.Equal(t, len(writesForNamespace), len(req.Elements)) - for i, write := range writesForNamespace { - assert.Equal(t, req.Elements[i].ID, write.request.ID) - assert.Equal(t, req.Elements[i].Datapoint, write.request.Datapoint) - assert.Equal(t, req.Elements[i].EncodedTags, write.request.EncodedTags) - } - } - - // Assert the writes will be handled in two batches - mockClient.EXPECT().WriteTaggedBatchRaw(gomock.Any(), gomock.Any()).Do(writeBatch).Return(nil).MinTimes(2).MaxTimes(2) - mockConnPool.EXPECT().NextClient().Return(mockClient, nil).MinTimes(2).MaxTimes(2) - - for _, write := range writes { - assert.NoError(t, queue.Enqueue(write)) + for _, opts := range []Options{ + newHostQueueTestOptions().SetUseV2BatchAPIs(false), + newHostQueueTestOptions().SetUseV2BatchAPIs(true), + } { + t.Run(fmt.Sprintf("useV2: %v", opts.UseV2BatchAPIs()), func(t *testing.T) { + ctrl := gomock.NewController(t) + defer ctrl.Finish() + + mockConnPool := NewMockconnectionPool(ctrl) + + queue := newTestHostQueue(opts) + queue.connPool = mockConnPool + + // Open + mockConnPool.EXPECT().Open() + queue.Open() + assert.Equal(t, statusOpen, queue.status) + + // Prepare callback for writes + var ( + results []hostQueueResult + resultsLock sync.Mutex + wg sync.WaitGroup + ) + callback := func(r interface{}, err error) { + resultsLock.Lock() + results = append(results, hostQueueResult{r, err}) + resultsLock.Unlock() + wg.Done() + } + + // Prepare writes + writes := []*writeTaggedOperation{ + testWriteTaggedOp("testNs1", "foo", map[string]string{ + "tag": "value", + "sup": "holmes", + }, 1.0, 1000, rpc.TimeType_UNIX_SECONDS, callback), + testWriteTaggedOp("testNs1", "bar", map[string]string{ + "and": "one", + }, 2.0, 2000, rpc.TimeType_UNIX_SECONDS, callback), + testWriteTaggedOp("testNs1", "baz", map[string]string{ + "cmon": "dawg", + "mmm": "kay", + }, 3.0, 3000, rpc.TimeType_UNIX_SECONDS, callback), + testWriteTaggedOp("testNs2", "qux", map[string]string{ + "mas": "ter", + "sal": "vo", + }, 4.0, 4000, rpc.TimeType_UNIX_SECONDS, callback), + } + wg.Add(len(writes)) + + // Prepare mocks for flush + mockClient := rpc.NewMockTChanNode(ctrl) + if opts.UseV2BatchAPIs() { + writeBatch := func(ctx thrift.Context, req *rpc.WriteTaggedBatchRawV2Request) { + for i, write := range writes { + if i < 3 { + assert.Equal(t, req.Elements[i].NameSpace, int64(0)) + } else { + assert.Equal(t, req.Elements[i].NameSpace, int64(1)) + } + assert.Equal(t, req.Elements[i].ID, write.request.ID) + assert.Equal(t, req.Elements[i].Datapoint, write.request.Datapoint) + assert.Equal(t, req.Elements[i].EncodedTags, write.request.EncodedTags) + } + } + // Assert the writes will be handled in two batches. + mockClient.EXPECT().WriteTaggedBatchRawV2(gomock.Any(), gomock.Any()).Do(writeBatch).Return(nil).Times(1) + mockConnPool.EXPECT().NextClient().Return(mockClient, nil).Times(1) + } else { + writeBatch := func(ctx thrift.Context, req *rpc.WriteTaggedBatchRawRequest) { + var writesForNamespace []*writeTaggedOperation + if string(req.NameSpace) == "testNs1" { + writesForNamespace = writes[:3] + } else { + writesForNamespace = writes[3:] + } + assert.Equal(t, len(writesForNamespace), len(req.Elements)) + for i, write := range writesForNamespace { + assert.Equal(t, req.Elements[i].ID, write.request.ID) + assert.Equal(t, req.Elements[i].Datapoint, write.request.Datapoint) + assert.Equal(t, req.Elements[i].EncodedTags, write.request.EncodedTags) + } + } + // Assert the writes will be handled in two batches. + mockClient.EXPECT().WriteTaggedBatchRaw(gomock.Any(), gomock.Any()).Do(writeBatch).Return(nil).Times(2) + mockConnPool.EXPECT().NextClient().Return(mockClient, nil).Times(2) + } + for _, write := range writes { + assert.NoError(t, queue.Enqueue(write)) + } + + // Wait for all writes + wg.Wait() + + // Assert writes successful + assert.Equal(t, len(writes), len(results)) + for _, result := range results { + assert.Nil(t, result.err) + } + + // Close + var closeWg sync.WaitGroup + closeWg.Add(1) + mockConnPool.EXPECT().Close().Do(func() { + closeWg.Done() + }) + queue.Close() + closeWg.Wait() + }) } - - // Wait for all writes - wg.Wait() - - // Assert writes successful - assert.Equal(t, len(writes), len(results)) - for _, result := range results { - assert.Nil(t, result.err) - } - - // Close - var closeWg sync.WaitGroup - closeWg.Add(1) - mockConnPool.EXPECT().Close().Do(func() { - closeWg.Done() - }) - queue.Close() - closeWg.Wait() } func TestHostQueueWriteTaggedBatchesNoClientAvailable(t *testing.T) { @@ -509,6 +548,9 @@ func testWriteTaggedOp( } w.request.EncodedTags = testEncode(tags) w.completionFn = completionFn + w.requestV2.ID = w.request.ID + w.requestV2.EncodedTags = w.request.EncodedTags + w.requestV2.Datapoint = w.request.Datapoint return w } diff --git a/src/dbnode/client/options.go b/src/dbnode/client/options.go index 3e79cfc9cc..bacf827aea 100644 --- a/src/dbnode/client/options.go +++ b/src/dbnode/client/options.go @@ -155,6 +155,10 @@ const ( // defaultAsyncWriteMaxConcurrency is the default maximum concurrency for async writes. defaultAsyncWriteMaxConcurrency = 4096 + + // defaultUseV2BatchAPIs is the default setting for whether the v2 version of the batch APIs should + // be used. + defaultUseV2BatchAPIs = false ) var ( @@ -253,6 +257,7 @@ type options struct { asyncTopologyInitializers []topology.Initializer asyncWriteWorkerPool xsync.PooledWorkerPool asyncWriteMaxConcurrency int + useV2BatchAPIs bool } // NewOptions creates a new set of client options with defaults @@ -345,6 +350,7 @@ func newOptions() *options { schemaRegistry: namespace.NewSchemaRegistry(false, nil), asyncTopologyInitializers: []topology.Initializer{}, asyncWriteMaxConcurrency: defaultAsyncWriteMaxConcurrency, + useV2BatchAPIs: defaultUseV2BatchAPIs, } return opts.SetEncodingM3TSZ().(*options) } @@ -931,3 +937,13 @@ func (o *options) SetAsyncWriteMaxConcurrency(value int) Options { func (o *options) AsyncWriteMaxConcurrency() int { return o.asyncWriteMaxConcurrency } + +func (o *options) SetUseV2BatchAPIs(value bool) Options { + opts := *o + opts.useV2BatchAPIs = value + return &opts +} + +func (o *options) UseV2BatchAPIs() bool { + return o.useV2BatchAPIs +} diff --git a/src/dbnode/client/session.go b/src/dbnode/client/session.go index 774fd54e04..f9128f1a3b 100644 --- a/src/dbnode/client/session.go +++ b/src/dbnode/client/session.go @@ -215,11 +215,15 @@ type streamFromPeersMetrics struct { } type hostQueueOpts struct { - writeBatchRawRequestPool writeBatchRawRequestPool - writeBatchRawRequestElementArrayPool writeBatchRawRequestElementArrayPool - writeTaggedBatchRawRequestPool writeTaggedBatchRawRequestPool - writeTaggedBatchRawRequestElementArrayPool writeTaggedBatchRawRequestElementArrayPool - opts Options + writeBatchRawRequestPool writeBatchRawRequestPool + writeBatchRawV2RequestPool writeBatchRawV2RequestPool + writeBatchRawRequestElementArrayPool writeBatchRawRequestElementArrayPool + writeBatchRawV2RequestElementArrayPool writeBatchRawV2RequestElementArrayPool + writeTaggedBatchRawRequestPool writeTaggedBatchRawRequestPool + writeTaggedBatchRawV2RequestPool writeTaggedBatchRawV2RequestPool + writeTaggedBatchRawRequestElementArrayPool writeTaggedBatchRawRequestElementArrayPool + writeTaggedBatchRawV2RequestElementArrayPool writeTaggedBatchRawV2RequestElementArrayPool + opts Options } type newHostQueueFn func( @@ -852,6 +856,8 @@ func (s *session) newHostQueue(host topology.Host, topoMap topology.Map) (hostQu )) writeBatchRequestPool := newWriteBatchRawRequestPool(writeBatchRequestPoolOpts) writeBatchRequestPool.Init() + writeBatchV2RequestPool := newWriteBatchRawV2RequestPool(writeBatchRequestPoolOpts) + writeBatchV2RequestPool.Init() writeTaggedBatchRequestPoolOpts := pool.NewObjectPoolOptions(). SetSize(hostBatches). @@ -860,6 +866,8 @@ func (s *session) newHostQueue(host topology.Host, topoMap topology.Map) (hostQu )) writeTaggedBatchRequestPool := newWriteTaggedBatchRawRequestPool(writeTaggedBatchRequestPoolOpts) writeTaggedBatchRequestPool.Init() + writeTaggedBatchV2RequestPool := newWriteTaggedBatchRawV2RequestPool(writeBatchRequestPoolOpts) + writeTaggedBatchV2RequestPool.Init() writeBatchRawRequestElementArrayPoolOpts := pool.NewObjectPoolOptions(). SetSize(hostBatches). @@ -869,6 +877,9 @@ func (s *session) newHostQueue(host topology.Host, topoMap topology.Map) (hostQu writeBatchRawRequestElementArrayPool := newWriteBatchRawRequestElementArrayPool( writeBatchRawRequestElementArrayPoolOpts, s.opts.WriteBatchSize()) writeBatchRawRequestElementArrayPool.Init() + writeBatchRawV2RequestElementArrayPool := newWriteBatchRawV2RequestElementArrayPool( + writeBatchRawRequestElementArrayPoolOpts, s.opts.WriteBatchSize()) + writeBatchRawV2RequestElementArrayPool.Init() writeTaggedBatchRawRequestElementArrayPoolOpts := pool.NewObjectPoolOptions(). SetSize(hostBatches). @@ -878,12 +889,18 @@ func (s *session) newHostQueue(host topology.Host, topoMap topology.Map) (hostQu writeTaggedBatchRawRequestElementArrayPool := newWriteTaggedBatchRawRequestElementArrayPool( writeTaggedBatchRawRequestElementArrayPoolOpts, s.opts.WriteBatchSize()) writeTaggedBatchRawRequestElementArrayPool.Init() + writeTaggedBatchRawV2RequestElementArrayPool := newWriteTaggedBatchRawV2RequestElementArrayPool( + writeTaggedBatchRawRequestElementArrayPoolOpts, s.opts.WriteBatchSize()) hostQueue, err := s.newHostQueueFn(host, hostQueueOpts{ - writeBatchRawRequestPool: writeBatchRequestPool, - writeBatchRawRequestElementArrayPool: writeBatchRawRequestElementArrayPool, - writeTaggedBatchRawRequestPool: writeTaggedBatchRequestPool, - writeTaggedBatchRawRequestElementArrayPool: writeTaggedBatchRawRequestElementArrayPool, + writeBatchRawRequestPool: writeBatchRequestPool, + writeBatchRawV2RequestPool: writeBatchV2RequestPool, + writeBatchRawRequestElementArrayPool: writeBatchRawRequestElementArrayPool, + writeBatchRawV2RequestElementArrayPool: writeBatchRawV2RequestElementArrayPool, + writeTaggedBatchRawRequestPool: writeTaggedBatchRequestPool, + writeTaggedBatchRawV2RequestPool: writeTaggedBatchV2RequestPool, + writeTaggedBatchRawRequestElementArrayPool: writeTaggedBatchRawRequestElementArrayPool, + writeTaggedBatchRawV2RequestElementArrayPool: writeTaggedBatchRawV2RequestElementArrayPool, opts: s.opts, }) if err != nil { @@ -1025,6 +1042,8 @@ func (s *session) writeAttemptWithRLock( wop.request.Datapoint.Timestamp = timestamp wop.request.Datapoint.TimestampTimeType = timeType wop.request.Datapoint.Annotation = annotation + wop.requestV2.ID = wop.request.ID + wop.requestV2.Datapoint = wop.request.Datapoint op = wop case taggedWriteAttemptType: wop := s.pools.writeTaggedOperation.Get() @@ -1040,6 +1059,9 @@ func (s *session) writeAttemptWithRLock( wop.request.Datapoint.Timestamp = timestamp wop.request.Datapoint.TimestampTimeType = timeType wop.request.Datapoint.Annotation = annotation + wop.requestV2.ID = wop.request.ID + wop.requestV2.EncodedTags = wop.request.EncodedTags + wop.requestV2.Datapoint = wop.request.Datapoint op = wop default: // should never happen diff --git a/src/dbnode/client/session_test.go b/src/dbnode/client/session_test.go index fe9f13a80e..6f89c7acde 100644 --- a/src/dbnode/client/session_test.go +++ b/src/dbnode/client/session_test.go @@ -126,10 +126,14 @@ func applySessionTestOptions(opts Options) Options { func newTestHostQueue(opts Options) *queue { hq, err := newHostQueue(h, hostQueueOpts{ - writeBatchRawRequestPool: testWriteBatchRawPool, - writeBatchRawRequestElementArrayPool: testWriteArrayPool, - writeTaggedBatchRawRequestPool: testWriteTaggedBatchRawPool, - writeTaggedBatchRawRequestElementArrayPool: testWriteTaggedArrayPool, + writeBatchRawRequestPool: testWriteBatchRawPool, + writeBatchRawV2RequestPool: testWriteBatchRawV2Pool, + writeBatchRawRequestElementArrayPool: testWriteArrayPool, + writeBatchRawV2RequestElementArrayPool: testWriteV2ArrayPool, + writeTaggedBatchRawRequestPool: testWriteTaggedBatchRawPool, + writeTaggedBatchRawV2RequestPool: testWriteTaggedBatchRawV2Pool, + writeTaggedBatchRawRequestElementArrayPool: testWriteTaggedArrayPool, + writeTaggedBatchRawV2RequestElementArrayPool: testWriteTaggedV2ArrayPool, opts: opts, }) if err != nil { diff --git a/src/dbnode/client/types.go b/src/dbnode/client/types.go index ef3cca2288..1c3fa90c7b 100644 --- a/src/dbnode/client/types.go +++ b/src/dbnode/client/types.go @@ -527,6 +527,12 @@ type Options interface { // AsyncWriteMaxConcurrency returns the async writes maximum concurrency. AsyncWriteMaxConcurrency() int + + // SetUseV2BatchAPIs sets whether the V2 batch APIs should be used. + SetUseV2BatchAPIs(value bool) Options + + // UseV2BatchAPIs returns whether the V2 batch APIs should be used. + UseV2BatchAPIs() bool } // AdminOptions is a set of administration client options. diff --git a/src/dbnode/client/write_batch_element_array_pool.go b/src/dbnode/client/write_batch_element_array_pool.go index 983c8f88ab..3b8cc742ac 100644 --- a/src/dbnode/client/write_batch_element_array_pool.go +++ b/src/dbnode/client/write_batch_element_array_pool.go @@ -65,3 +65,44 @@ func (p *poolOfWriteBatchRawRequestElementArray) Put(w []*rpc.WriteBatchRawReque w = w[:0] p.pool.Put(w) } + +type writeBatchRawV2RequestElementArrayPool interface { + // Init pool + Init() + + // Get an array of WriteBatchV2RawRequestElement objects + Get() []*rpc.WriteBatchRawV2RequestElement + + // Put an array of WriteBatchRawV2RequestElement objects + Put(w []*rpc.WriteBatchRawV2RequestElement) +} + +type poolOfWriteBatchRawV2RequestElementArray struct { + pool pool.ObjectPool + capacity int +} + +func newWriteBatchRawV2RequestElementArrayPool( + opts pool.ObjectPoolOptions, capacity int) writeBatchRawV2RequestElementArrayPool { + + p := pool.NewObjectPool(opts) + return &poolOfWriteBatchRawV2RequestElementArray{p, capacity} +} + +func (p *poolOfWriteBatchRawV2RequestElementArray) Init() { + p.pool.Init(func() interface{} { + return make([]*rpc.WriteBatchRawV2RequestElement, 0, p.capacity) + }) +} + +func (p *poolOfWriteBatchRawV2RequestElementArray) Get() []*rpc.WriteBatchRawV2RequestElement { + return p.pool.Get().([]*rpc.WriteBatchRawV2RequestElement) +} + +func (p *poolOfWriteBatchRawV2RequestElementArray) Put(w []*rpc.WriteBatchRawV2RequestElement) { + for i := range w { + w[i] = nil + } + w = w[:0] + p.pool.Put(w) +} diff --git a/src/dbnode/client/write_batch_pool.go b/src/dbnode/client/write_batch_pool.go index f8e596c286..f2aa1b3816 100644 --- a/src/dbnode/client/write_batch_pool.go +++ b/src/dbnode/client/write_batch_pool.go @@ -26,7 +26,8 @@ import ( ) var ( - writeBatchRawRequestZeroed rpc.WriteBatchRawRequest + writeBatchRawRequestZeroed rpc.WriteBatchRawRequest + writeBatchRawV2RequestZeroed rpc.WriteBatchRawV2Request ) type writeBatchRawRequestPool interface { @@ -63,3 +64,38 @@ func (p *poolOfWriteBatchRawRequest) Put(w *rpc.WriteBatchRawRequest) { *w = writeBatchRawRequestZeroed p.pool.Put(w) } + +type writeBatchRawV2RequestPool interface { + // Init pool. + Init() + + // Get a write batch request. + Get() *rpc.WriteBatchRawV2Request + + // Put a write batch request. + Put(w *rpc.WriteBatchRawV2Request) +} + +type poolOfWriteBatchRawV2Request struct { + pool pool.ObjectPool +} + +func newWriteBatchRawV2RequestPool(opts pool.ObjectPoolOptions) writeBatchRawV2RequestPool { + p := pool.NewObjectPool(opts) + return &poolOfWriteBatchRawV2Request{p} +} + +func (p *poolOfWriteBatchRawV2Request) Init() { + p.pool.Init(func() interface{} { + return &rpc.WriteBatchRawV2Request{} + }) +} + +func (p *poolOfWriteBatchRawV2Request) Get() *rpc.WriteBatchRawV2Request { + return p.pool.Get().(*rpc.WriteBatchRawV2Request) +} + +func (p *poolOfWriteBatchRawV2Request) Put(w *rpc.WriteBatchRawV2Request) { + *w = writeBatchRawV2RequestZeroed + p.pool.Put(w) +} diff --git a/src/dbnode/client/write_batch_tagged_element_array_pool.go b/src/dbnode/client/write_batch_tagged_element_array_pool.go index a9af07f71c..4ed3ee84f9 100644 --- a/src/dbnode/client/write_batch_tagged_element_array_pool.go +++ b/src/dbnode/client/write_batch_tagged_element_array_pool.go @@ -65,3 +65,44 @@ func (p *poolOfWriteTaggedBatchRawRequestElementArray) Put(w []*rpc.WriteTaggedB w = w[:0] p.pool.Put(w) } + +type writeTaggedBatchRawV2RequestElementArrayPool interface { + // Init pool + Init() + + // Get an array of WriteTaggedBatchRawV2RequestElement objects + Get() []*rpc.WriteTaggedBatchRawV2RequestElement + + // Put an array of WriteTaggedBatchRawV2RequestElement objects + Put(w []*rpc.WriteTaggedBatchRawV2RequestElement) +} + +type poolOfWriteTaggedBatchRawV2RequestElementArray struct { + pool pool.ObjectPool + capacity int +} + +func newWriteTaggedBatchRawV2RequestElementArrayPool( + opts pool.ObjectPoolOptions, capacity int) writeTaggedBatchRawV2RequestElementArrayPool { + + p := pool.NewObjectPool(opts) + return &poolOfWriteTaggedBatchRawV2RequestElementArray{p, capacity} +} + +func (p *poolOfWriteTaggedBatchRawV2RequestElementArray) Init() { + p.pool.Init(func() interface{} { + return make([]*rpc.WriteTaggedBatchRawV2RequestElement, 0, p.capacity) + }) +} + +func (p *poolOfWriteTaggedBatchRawV2RequestElementArray) Get() []*rpc.WriteTaggedBatchRawV2RequestElement { + return p.pool.Get().([]*rpc.WriteTaggedBatchRawV2RequestElement) +} + +func (p *poolOfWriteTaggedBatchRawV2RequestElementArray) Put(w []*rpc.WriteTaggedBatchRawV2RequestElement) { + for i := range w { + w[i] = nil + } + w = w[:0] + p.pool.Put(w) +} diff --git a/src/dbnode/client/write_batch_tagged_pool.go b/src/dbnode/client/write_batch_tagged_pool.go index fa100a6a7d..239a5fb39f 100644 --- a/src/dbnode/client/write_batch_tagged_pool.go +++ b/src/dbnode/client/write_batch_tagged_pool.go @@ -26,7 +26,8 @@ import ( ) var ( - writeTaggedBatchRawRequestZeroed rpc.WriteTaggedBatchRawRequest + writeTaggedBatchRawRequestZeroed rpc.WriteTaggedBatchRawRequest + writeTaggedBatchRawV2RequestZeroed rpc.WriteTaggedBatchRawV2Request ) type writeTaggedBatchRawRequestPool interface { @@ -63,3 +64,38 @@ func (p *poolOfWriteTaggedBatchRawRequest) Put(w *rpc.WriteTaggedBatchRawRequest *w = writeTaggedBatchRawRequestZeroed p.pool.Put(w) } + +type writeTaggedBatchRawV2RequestPool interface { + // Init pool. + Init() + + // Get a write batch request. + Get() *rpc.WriteTaggedBatchRawV2Request + + // Put a write batch request. + Put(w *rpc.WriteTaggedBatchRawV2Request) +} + +type poolOfWriteTaggedBatchRawV2Request struct { + pool pool.ObjectPool +} + +func newWriteTaggedBatchRawV2RequestPool(opts pool.ObjectPoolOptions) writeTaggedBatchRawV2RequestPool { + p := pool.NewObjectPool(opts) + return &poolOfWriteTaggedBatchRawV2Request{p} +} + +func (p *poolOfWriteTaggedBatchRawV2Request) Init() { + p.pool.Init(func() interface{} { + return &rpc.WriteTaggedBatchRawV2Request{} + }) +} + +func (p *poolOfWriteTaggedBatchRawV2Request) Get() *rpc.WriteTaggedBatchRawV2Request { + return p.pool.Get().(*rpc.WriteTaggedBatchRawV2Request) +} + +func (p *poolOfWriteTaggedBatchRawV2Request) Put(w *rpc.WriteTaggedBatchRawV2Request) { + *w = writeTaggedBatchRawV2RequestZeroed + p.pool.Put(w) +} diff --git a/src/dbnode/client/write_op.go b/src/dbnode/client/write_op.go index 8c3cd7a2f5..6fdd4bfd63 100644 --- a/src/dbnode/client/write_op.go +++ b/src/dbnode/client/write_op.go @@ -37,6 +37,7 @@ type writeOperation struct { namespace ident.ID shardID uint32 request rpc.WriteBatchRawRequestElement + requestV2 rpc.WriteBatchRawV2RequestElement datapoint rpc.Datapoint completionFn completionFn pool *writeOperationPool @@ -45,6 +46,7 @@ type writeOperation struct { func (w *writeOperation) reset() { *w = writeOperationZeroed w.request.Datapoint = &w.datapoint + w.requestV2.Datapoint = &w.datapoint } func (w *writeOperation) Close() { diff --git a/src/dbnode/client/write_tagged_op.go b/src/dbnode/client/write_tagged_op.go index d54396f5c6..846dc8d3ac 100644 --- a/src/dbnode/client/write_tagged_op.go +++ b/src/dbnode/client/write_tagged_op.go @@ -37,6 +37,7 @@ type writeTaggedOperation struct { namespace ident.ID shardID uint32 request rpc.WriteTaggedBatchRawRequestElement + requestV2 rpc.WriteTaggedBatchRawV2RequestElement datapoint rpc.Datapoint completionFn completionFn pool *writeTaggedOperationPool @@ -45,6 +46,7 @@ type writeTaggedOperation struct { func (w *writeTaggedOperation) reset() { *w = writeTaggedOperationZeroed w.request.Datapoint = &w.datapoint + w.requestV2.Datapoint = &w.datapoint } func (w *writeTaggedOperation) Close() { diff --git a/src/dbnode/generated/thrift/rpc.thrift b/src/dbnode/generated/thrift/rpc.thrift index 68e34780b9..422ece3fb7 100644 --- a/src/dbnode/generated/thrift/rpc.thrift +++ b/src/dbnode/generated/thrift/rpc.thrift @@ -57,7 +57,9 @@ service Node { FetchBlocksMetadataRawV2Result fetchBlocksMetadataRawV2(1: FetchBlocksMetadataRawV2Request req) throws (1: Error err) void writeBatchRaw(1: WriteBatchRawRequest req) throws (1: WriteBatchRawErrors err) + void writeBatchRawV2(1: WriteBatchRawV2Request req) throws (1: WriteBatchRawErrors err) void writeTaggedBatchRaw(1: WriteTaggedBatchRawRequest req) throws (1: WriteBatchRawErrors err) + void writeTaggedBatchRawV2(1: WriteTaggedBatchRawV2Request req) throws (1: WriteBatchRawErrors err) void repair() throws (1: Error err) TruncateResult truncate(1: TruncateRequest req) throws (1: Error err) @@ -227,22 +229,45 @@ struct WriteBatchRawRequest { 2: required list elements } +struct WriteBatchRawV2Request { + 1: required list nameSpaces + 2: required list elements +} + struct WriteBatchRawRequestElement { 1: required binary id 2: required Datapoint datapoint } +struct WriteBatchRawV2RequestElement { + 1: required binary id + 2: required Datapoint datapoint + 3: required i64 nameSpace +} + struct WriteTaggedBatchRawRequest { 1: required binary nameSpace 2: required list elements } +struct WriteTaggedBatchRawV2Request { + 1: required list nameSpaces + 2: required list elements +} + struct WriteTaggedBatchRawRequestElement { 1: required binary id 2: required binary encodedTags 3: required Datapoint datapoint } +struct WriteTaggedBatchRawV2RequestElement { + 1: required binary id + 2: required binary encodedTags + 3: required Datapoint datapoint + 4: required i64 nameSpace +} + struct WriteBatchRawError { 1: required i64 index 2: required Error err diff --git a/src/dbnode/generated/thrift/rpc/rpc.go b/src/dbnode/generated/thrift/rpc/rpc.go index dc482b2a93..f49457f08d 100644 --- a/src/dbnode/generated/thrift/rpc/rpc.go +++ b/src/dbnode/generated/thrift/rpc/rpc.go @@ -5531,40 +5531,31 @@ func (p *WriteBatchRawRequest) String() string { } // Attributes: -// - ID -// - Datapoint -type WriteBatchRawRequestElement struct { - ID []byte `thrift:"id,1,required" db:"id" json:"id"` - Datapoint *Datapoint `thrift:"datapoint,2,required" db:"datapoint" json:"datapoint"` +// - NameSpaces +// - Elements +type WriteBatchRawV2Request struct { + NameSpaces [][]byte `thrift:"nameSpaces,1,required" db:"nameSpaces" json:"nameSpaces"` + Elements []*WriteBatchRawV2RequestElement `thrift:"elements,2,required" db:"elements" json:"elements"` } -func NewWriteBatchRawRequestElement() *WriteBatchRawRequestElement { - return &WriteBatchRawRequestElement{} +func NewWriteBatchRawV2Request() *WriteBatchRawV2Request { + return &WriteBatchRawV2Request{} } -func (p *WriteBatchRawRequestElement) GetID() []byte { - return p.ID +func (p *WriteBatchRawV2Request) GetNameSpaces() [][]byte { + return p.NameSpaces } -var WriteBatchRawRequestElement_Datapoint_DEFAULT *Datapoint - -func (p *WriteBatchRawRequestElement) GetDatapoint() *Datapoint { - if !p.IsSetDatapoint() { - return WriteBatchRawRequestElement_Datapoint_DEFAULT - } - return p.Datapoint -} -func (p *WriteBatchRawRequestElement) IsSetDatapoint() bool { - return p.Datapoint != nil +func (p *WriteBatchRawV2Request) GetElements() []*WriteBatchRawV2RequestElement { + return p.Elements } - -func (p *WriteBatchRawRequestElement) Read(iprot thrift.TProtocol) error { +func (p *WriteBatchRawV2Request) Read(iprot thrift.TProtocol) error { if _, err := iprot.ReadStructBegin(); err != nil { return thrift.PrependError(fmt.Sprintf("%T read error: ", p), err) } - var issetID bool = false - var issetDatapoint bool = false + var issetNameSpaces bool = false + var issetElements bool = false for { _, fieldTypeId, fieldId, err := iprot.ReadFieldBegin() @@ -5579,12 +5570,12 @@ func (p *WriteBatchRawRequestElement) Read(iprot thrift.TProtocol) error { if err := p.ReadField1(iprot); err != nil { return err } - issetID = true + issetNameSpaces = true case 2: if err := p.ReadField2(iprot); err != nil { return err } - issetDatapoint = true + issetElements = true default: if err := iprot.Skip(fieldTypeId); err != nil { return err @@ -5597,36 +5588,59 @@ func (p *WriteBatchRawRequestElement) Read(iprot thrift.TProtocol) error { if err := iprot.ReadStructEnd(); err != nil { return thrift.PrependError(fmt.Sprintf("%T read struct end error: ", p), err) } - if !issetID { - return thrift.NewTProtocolExceptionWithType(thrift.INVALID_DATA, fmt.Errorf("Required field ID is not set")) + if !issetNameSpaces { + return thrift.NewTProtocolExceptionWithType(thrift.INVALID_DATA, fmt.Errorf("Required field NameSpaces is not set")) } - if !issetDatapoint { - return thrift.NewTProtocolExceptionWithType(thrift.INVALID_DATA, fmt.Errorf("Required field Datapoint is not set")) + if !issetElements { + return thrift.NewTProtocolExceptionWithType(thrift.INVALID_DATA, fmt.Errorf("Required field Elements is not set")) } return nil } -func (p *WriteBatchRawRequestElement) ReadField1(iprot thrift.TProtocol) error { - if v, err := iprot.ReadBinary(); err != nil { - return thrift.PrependError("error reading field 1: ", err) - } else { - p.ID = v +func (p *WriteBatchRawV2Request) ReadField1(iprot thrift.TProtocol) error { + _, size, err := iprot.ReadListBegin() + if err != nil { + return thrift.PrependError("error reading list begin: ", err) + } + tSlice := make([][]byte, 0, size) + p.NameSpaces = tSlice + for i := 0; i < size; i++ { + var _elem15 []byte + if v, err := iprot.ReadBinary(); err != nil { + return thrift.PrependError("error reading field 0: ", err) + } else { + _elem15 = v + } + p.NameSpaces = append(p.NameSpaces, _elem15) + } + if err := iprot.ReadListEnd(); err != nil { + return thrift.PrependError("error reading list end: ", err) } return nil } -func (p *WriteBatchRawRequestElement) ReadField2(iprot thrift.TProtocol) error { - p.Datapoint = &Datapoint{ - TimestampTimeType: 0, +func (p *WriteBatchRawV2Request) ReadField2(iprot thrift.TProtocol) error { + _, size, err := iprot.ReadListBegin() + if err != nil { + return thrift.PrependError("error reading list begin: ", err) } - if err := p.Datapoint.Read(iprot); err != nil { - return thrift.PrependError(fmt.Sprintf("%T error reading struct: ", p.Datapoint), err) + tSlice := make([]*WriteBatchRawV2RequestElement, 0, size) + p.Elements = tSlice + for i := 0; i < size; i++ { + _elem16 := &WriteBatchRawV2RequestElement{} + if err := _elem16.Read(iprot); err != nil { + return thrift.PrependError(fmt.Sprintf("%T error reading struct: ", _elem16), err) + } + p.Elements = append(p.Elements, _elem16) + } + if err := iprot.ReadListEnd(); err != nil { + return thrift.PrependError("error reading list end: ", err) } return nil } -func (p *WriteBatchRawRequestElement) Write(oprot thrift.TProtocol) error { - if err := oprot.WriteStructBegin("WriteBatchRawRequestElement"); err != nil { +func (p *WriteBatchRawV2Request) Write(oprot thrift.TProtocol) error { + if err := oprot.WriteStructBegin("WriteBatchRawV2Request"); err != nil { return thrift.PrependError(fmt.Sprintf("%T write struct begin error: ", p), err) } if p != nil { @@ -5646,65 +5660,90 @@ func (p *WriteBatchRawRequestElement) Write(oprot thrift.TProtocol) error { return nil } -func (p *WriteBatchRawRequestElement) writeField1(oprot thrift.TProtocol) (err error) { - if err := oprot.WriteFieldBegin("id", thrift.STRING, 1); err != nil { - return thrift.PrependError(fmt.Sprintf("%T write field begin error 1:id: ", p), err) +func (p *WriteBatchRawV2Request) writeField1(oprot thrift.TProtocol) (err error) { + if err := oprot.WriteFieldBegin("nameSpaces", thrift.LIST, 1); err != nil { + return thrift.PrependError(fmt.Sprintf("%T write field begin error 1:nameSpaces: ", p), err) } - if err := oprot.WriteBinary(p.ID); err != nil { - return thrift.PrependError(fmt.Sprintf("%T.id (1) field write error: ", p), err) + if err := oprot.WriteListBegin(thrift.STRING, len(p.NameSpaces)); err != nil { + return thrift.PrependError("error writing list begin: ", err) + } + for _, v := range p.NameSpaces { + if err := oprot.WriteBinary(v); err != nil { + return thrift.PrependError(fmt.Sprintf("%T. (0) field write error: ", p), err) + } + } + if err := oprot.WriteListEnd(); err != nil { + return thrift.PrependError("error writing list end: ", err) } if err := oprot.WriteFieldEnd(); err != nil { - return thrift.PrependError(fmt.Sprintf("%T write field end error 1:id: ", p), err) + return thrift.PrependError(fmt.Sprintf("%T write field end error 1:nameSpaces: ", p), err) } return err } -func (p *WriteBatchRawRequestElement) writeField2(oprot thrift.TProtocol) (err error) { - if err := oprot.WriteFieldBegin("datapoint", thrift.STRUCT, 2); err != nil { - return thrift.PrependError(fmt.Sprintf("%T write field begin error 2:datapoint: ", p), err) +func (p *WriteBatchRawV2Request) writeField2(oprot thrift.TProtocol) (err error) { + if err := oprot.WriteFieldBegin("elements", thrift.LIST, 2); err != nil { + return thrift.PrependError(fmt.Sprintf("%T write field begin error 2:elements: ", p), err) } - if err := p.Datapoint.Write(oprot); err != nil { - return thrift.PrependError(fmt.Sprintf("%T error writing struct: ", p.Datapoint), err) + if err := oprot.WriteListBegin(thrift.STRUCT, len(p.Elements)); err != nil { + return thrift.PrependError("error writing list begin: ", err) + } + for _, v := range p.Elements { + if err := v.Write(oprot); err != nil { + return thrift.PrependError(fmt.Sprintf("%T error writing struct: ", v), err) + } + } + if err := oprot.WriteListEnd(); err != nil { + return thrift.PrependError("error writing list end: ", err) } if err := oprot.WriteFieldEnd(); err != nil { - return thrift.PrependError(fmt.Sprintf("%T write field end error 2:datapoint: ", p), err) + return thrift.PrependError(fmt.Sprintf("%T write field end error 2:elements: ", p), err) } return err } -func (p *WriteBatchRawRequestElement) String() string { +func (p *WriteBatchRawV2Request) String() string { if p == nil { return "" } - return fmt.Sprintf("WriteBatchRawRequestElement(%+v)", *p) + return fmt.Sprintf("WriteBatchRawV2Request(%+v)", *p) } // Attributes: -// - NameSpace -// - Elements -type WriteTaggedBatchRawRequest struct { - NameSpace []byte `thrift:"nameSpace,1,required" db:"nameSpace" json:"nameSpace"` - Elements []*WriteTaggedBatchRawRequestElement `thrift:"elements,2,required" db:"elements" json:"elements"` +// - ID +// - Datapoint +type WriteBatchRawRequestElement struct { + ID []byte `thrift:"id,1,required" db:"id" json:"id"` + Datapoint *Datapoint `thrift:"datapoint,2,required" db:"datapoint" json:"datapoint"` } -func NewWriteTaggedBatchRawRequest() *WriteTaggedBatchRawRequest { - return &WriteTaggedBatchRawRequest{} +func NewWriteBatchRawRequestElement() *WriteBatchRawRequestElement { + return &WriteBatchRawRequestElement{} } -func (p *WriteTaggedBatchRawRequest) GetNameSpace() []byte { - return p.NameSpace +func (p *WriteBatchRawRequestElement) GetID() []byte { + return p.ID } -func (p *WriteTaggedBatchRawRequest) GetElements() []*WriteTaggedBatchRawRequestElement { - return p.Elements +var WriteBatchRawRequestElement_Datapoint_DEFAULT *Datapoint + +func (p *WriteBatchRawRequestElement) GetDatapoint() *Datapoint { + if !p.IsSetDatapoint() { + return WriteBatchRawRequestElement_Datapoint_DEFAULT + } + return p.Datapoint } -func (p *WriteTaggedBatchRawRequest) Read(iprot thrift.TProtocol) error { +func (p *WriteBatchRawRequestElement) IsSetDatapoint() bool { + return p.Datapoint != nil +} + +func (p *WriteBatchRawRequestElement) Read(iprot thrift.TProtocol) error { if _, err := iprot.ReadStructBegin(); err != nil { return thrift.PrependError(fmt.Sprintf("%T read error: ", p), err) } - var issetNameSpace bool = false - var issetElements bool = false + var issetID bool = false + var issetDatapoint bool = false for { _, fieldTypeId, fieldId, err := iprot.ReadFieldBegin() @@ -5719,12 +5758,12 @@ func (p *WriteTaggedBatchRawRequest) Read(iprot thrift.TProtocol) error { if err := p.ReadField1(iprot); err != nil { return err } - issetNameSpace = true + issetID = true case 2: if err := p.ReadField2(iprot); err != nil { return err } - issetElements = true + issetDatapoint = true default: if err := iprot.Skip(fieldTypeId); err != nil { return err @@ -5737,46 +5776,36 @@ func (p *WriteTaggedBatchRawRequest) Read(iprot thrift.TProtocol) error { if err := iprot.ReadStructEnd(); err != nil { return thrift.PrependError(fmt.Sprintf("%T read struct end error: ", p), err) } - if !issetNameSpace { - return thrift.NewTProtocolExceptionWithType(thrift.INVALID_DATA, fmt.Errorf("Required field NameSpace is not set")) + if !issetID { + return thrift.NewTProtocolExceptionWithType(thrift.INVALID_DATA, fmt.Errorf("Required field ID is not set")) } - if !issetElements { - return thrift.NewTProtocolExceptionWithType(thrift.INVALID_DATA, fmt.Errorf("Required field Elements is not set")) + if !issetDatapoint { + return thrift.NewTProtocolExceptionWithType(thrift.INVALID_DATA, fmt.Errorf("Required field Datapoint is not set")) } return nil } -func (p *WriteTaggedBatchRawRequest) ReadField1(iprot thrift.TProtocol) error { +func (p *WriteBatchRawRequestElement) ReadField1(iprot thrift.TProtocol) error { if v, err := iprot.ReadBinary(); err != nil { return thrift.PrependError("error reading field 1: ", err) } else { - p.NameSpace = v + p.ID = v } return nil } -func (p *WriteTaggedBatchRawRequest) ReadField2(iprot thrift.TProtocol) error { - _, size, err := iprot.ReadListBegin() - if err != nil { - return thrift.PrependError("error reading list begin: ", err) - } - tSlice := make([]*WriteTaggedBatchRawRequestElement, 0, size) - p.Elements = tSlice - for i := 0; i < size; i++ { - _elem15 := &WriteTaggedBatchRawRequestElement{} - if err := _elem15.Read(iprot); err != nil { - return thrift.PrependError(fmt.Sprintf("%T error reading struct: ", _elem15), err) - } - p.Elements = append(p.Elements, _elem15) +func (p *WriteBatchRawRequestElement) ReadField2(iprot thrift.TProtocol) error { + p.Datapoint = &Datapoint{ + TimestampTimeType: 0, } - if err := iprot.ReadListEnd(); err != nil { - return thrift.PrependError("error reading list end: ", err) + if err := p.Datapoint.Read(iprot); err != nil { + return thrift.PrependError(fmt.Sprintf("%T error reading struct: ", p.Datapoint), err) } return nil } -func (p *WriteTaggedBatchRawRequest) Write(oprot thrift.TProtocol) error { - if err := oprot.WriteStructBegin("WriteTaggedBatchRawRequest"); err != nil { +func (p *WriteBatchRawRequestElement) Write(oprot thrift.TProtocol) error { + if err := oprot.WriteStructBegin("WriteBatchRawRequestElement"); err != nil { return thrift.PrependError(fmt.Sprintf("%T write struct begin error: ", p), err) } if p != nil { @@ -5796,89 +5825,81 @@ func (p *WriteTaggedBatchRawRequest) Write(oprot thrift.TProtocol) error { return nil } -func (p *WriteTaggedBatchRawRequest) writeField1(oprot thrift.TProtocol) (err error) { - if err := oprot.WriteFieldBegin("nameSpace", thrift.STRING, 1); err != nil { - return thrift.PrependError(fmt.Sprintf("%T write field begin error 1:nameSpace: ", p), err) +func (p *WriteBatchRawRequestElement) writeField1(oprot thrift.TProtocol) (err error) { + if err := oprot.WriteFieldBegin("id", thrift.STRING, 1); err != nil { + return thrift.PrependError(fmt.Sprintf("%T write field begin error 1:id: ", p), err) } - if err := oprot.WriteBinary(p.NameSpace); err != nil { - return thrift.PrependError(fmt.Sprintf("%T.nameSpace (1) field write error: ", p), err) + if err := oprot.WriteBinary(p.ID); err != nil { + return thrift.PrependError(fmt.Sprintf("%T.id (1) field write error: ", p), err) } if err := oprot.WriteFieldEnd(); err != nil { - return thrift.PrependError(fmt.Sprintf("%T write field end error 1:nameSpace: ", p), err) + return thrift.PrependError(fmt.Sprintf("%T write field end error 1:id: ", p), err) } return err } -func (p *WriteTaggedBatchRawRequest) writeField2(oprot thrift.TProtocol) (err error) { - if err := oprot.WriteFieldBegin("elements", thrift.LIST, 2); err != nil { - return thrift.PrependError(fmt.Sprintf("%T write field begin error 2:elements: ", p), err) - } - if err := oprot.WriteListBegin(thrift.STRUCT, len(p.Elements)); err != nil { - return thrift.PrependError("error writing list begin: ", err) - } - for _, v := range p.Elements { - if err := v.Write(oprot); err != nil { - return thrift.PrependError(fmt.Sprintf("%T error writing struct: ", v), err) - } +func (p *WriteBatchRawRequestElement) writeField2(oprot thrift.TProtocol) (err error) { + if err := oprot.WriteFieldBegin("datapoint", thrift.STRUCT, 2); err != nil { + return thrift.PrependError(fmt.Sprintf("%T write field begin error 2:datapoint: ", p), err) } - if err := oprot.WriteListEnd(); err != nil { - return thrift.PrependError("error writing list end: ", err) + if err := p.Datapoint.Write(oprot); err != nil { + return thrift.PrependError(fmt.Sprintf("%T error writing struct: ", p.Datapoint), err) } if err := oprot.WriteFieldEnd(); err != nil { - return thrift.PrependError(fmt.Sprintf("%T write field end error 2:elements: ", p), err) + return thrift.PrependError(fmt.Sprintf("%T write field end error 2:datapoint: ", p), err) } return err } -func (p *WriteTaggedBatchRawRequest) String() string { +func (p *WriteBatchRawRequestElement) String() string { if p == nil { return "" } - return fmt.Sprintf("WriteTaggedBatchRawRequest(%+v)", *p) + return fmt.Sprintf("WriteBatchRawRequestElement(%+v)", *p) } // Attributes: // - ID -// - EncodedTags // - Datapoint -type WriteTaggedBatchRawRequestElement struct { - ID []byte `thrift:"id,1,required" db:"id" json:"id"` - EncodedTags []byte `thrift:"encodedTags,2,required" db:"encodedTags" json:"encodedTags"` - Datapoint *Datapoint `thrift:"datapoint,3,required" db:"datapoint" json:"datapoint"` +// - NameSpace +type WriteBatchRawV2RequestElement struct { + ID []byte `thrift:"id,1,required" db:"id" json:"id"` + Datapoint *Datapoint `thrift:"datapoint,2,required" db:"datapoint" json:"datapoint"` + NameSpace int64 `thrift:"nameSpace,3,required" db:"nameSpace" json:"nameSpace"` } -func NewWriteTaggedBatchRawRequestElement() *WriteTaggedBatchRawRequestElement { - return &WriteTaggedBatchRawRequestElement{} +func NewWriteBatchRawV2RequestElement() *WriteBatchRawV2RequestElement { + return &WriteBatchRawV2RequestElement{} } -func (p *WriteTaggedBatchRawRequestElement) GetID() []byte { +func (p *WriteBatchRawV2RequestElement) GetID() []byte { return p.ID } -func (p *WriteTaggedBatchRawRequestElement) GetEncodedTags() []byte { - return p.EncodedTags -} - -var WriteTaggedBatchRawRequestElement_Datapoint_DEFAULT *Datapoint +var WriteBatchRawV2RequestElement_Datapoint_DEFAULT *Datapoint -func (p *WriteTaggedBatchRawRequestElement) GetDatapoint() *Datapoint { +func (p *WriteBatchRawV2RequestElement) GetDatapoint() *Datapoint { if !p.IsSetDatapoint() { - return WriteTaggedBatchRawRequestElement_Datapoint_DEFAULT + return WriteBatchRawV2RequestElement_Datapoint_DEFAULT } return p.Datapoint } -func (p *WriteTaggedBatchRawRequestElement) IsSetDatapoint() bool { + +func (p *WriteBatchRawV2RequestElement) GetNameSpace() int64 { + return p.NameSpace +} +func (p *WriteBatchRawV2RequestElement) IsSetDatapoint() bool { return p.Datapoint != nil } -func (p *WriteTaggedBatchRawRequestElement) Read(iprot thrift.TProtocol) error { +func (p *WriteBatchRawV2RequestElement) Read(iprot thrift.TProtocol) error { if _, err := iprot.ReadStructBegin(); err != nil { return thrift.PrependError(fmt.Sprintf("%T read error: ", p), err) } var issetID bool = false - var issetEncodedTags bool = false var issetDatapoint bool = false + var issetNameSpace bool = false for { _, fieldTypeId, fieldId, err := iprot.ReadFieldBegin() @@ -5898,12 +5919,12 @@ func (p *WriteTaggedBatchRawRequestElement) Read(iprot thrift.TProtocol) error { if err := p.ReadField2(iprot); err != nil { return err } - issetEncodedTags = true + issetDatapoint = true case 3: if err := p.ReadField3(iprot); err != nil { return err } - issetDatapoint = true + issetNameSpace = true default: if err := iprot.Skip(fieldTypeId); err != nil { return err @@ -5919,16 +5940,16 @@ func (p *WriteTaggedBatchRawRequestElement) Read(iprot thrift.TProtocol) error { if !issetID { return thrift.NewTProtocolExceptionWithType(thrift.INVALID_DATA, fmt.Errorf("Required field ID is not set")) } - if !issetEncodedTags { - return thrift.NewTProtocolExceptionWithType(thrift.INVALID_DATA, fmt.Errorf("Required field EncodedTags is not set")) - } if !issetDatapoint { return thrift.NewTProtocolExceptionWithType(thrift.INVALID_DATA, fmt.Errorf("Required field Datapoint is not set")) } + if !issetNameSpace { + return thrift.NewTProtocolExceptionWithType(thrift.INVALID_DATA, fmt.Errorf("Required field NameSpace is not set")) + } return nil } -func (p *WriteTaggedBatchRawRequestElement) ReadField1(iprot thrift.TProtocol) error { +func (p *WriteBatchRawV2RequestElement) ReadField1(iprot thrift.TProtocol) error { if v, err := iprot.ReadBinary(); err != nil { return thrift.PrependError("error reading field 1: ", err) } else { @@ -5937,27 +5958,27 @@ func (p *WriteTaggedBatchRawRequestElement) ReadField1(iprot thrift.TProtocol) e return nil } -func (p *WriteTaggedBatchRawRequestElement) ReadField2(iprot thrift.TProtocol) error { - if v, err := iprot.ReadBinary(); err != nil { - return thrift.PrependError("error reading field 2: ", err) - } else { - p.EncodedTags = v +func (p *WriteBatchRawV2RequestElement) ReadField2(iprot thrift.TProtocol) error { + p.Datapoint = &Datapoint{ + TimestampTimeType: 0, + } + if err := p.Datapoint.Read(iprot); err != nil { + return thrift.PrependError(fmt.Sprintf("%T error reading struct: ", p.Datapoint), err) } return nil } -func (p *WriteTaggedBatchRawRequestElement) ReadField3(iprot thrift.TProtocol) error { - p.Datapoint = &Datapoint{ - TimestampTimeType: 0, - } - if err := p.Datapoint.Read(iprot); err != nil { - return thrift.PrependError(fmt.Sprintf("%T error reading struct: ", p.Datapoint), err) +func (p *WriteBatchRawV2RequestElement) ReadField3(iprot thrift.TProtocol) error { + if v, err := iprot.ReadI64(); err != nil { + return thrift.PrependError("error reading field 3: ", err) + } else { + p.NameSpace = v } return nil } -func (p *WriteTaggedBatchRawRequestElement) Write(oprot thrift.TProtocol) error { - if err := oprot.WriteStructBegin("WriteTaggedBatchRawRequestElement"); err != nil { +func (p *WriteBatchRawV2RequestElement) Write(oprot thrift.TProtocol) error { + if err := oprot.WriteStructBegin("WriteBatchRawV2RequestElement"); err != nil { return thrift.PrependError(fmt.Sprintf("%T write struct begin error: ", p), err) } if p != nil { @@ -5980,7 +6001,7 @@ func (p *WriteTaggedBatchRawRequestElement) Write(oprot thrift.TProtocol) error return nil } -func (p *WriteTaggedBatchRawRequestElement) writeField1(oprot thrift.TProtocol) (err error) { +func (p *WriteBatchRawV2RequestElement) writeField1(oprot thrift.TProtocol) (err error) { if err := oprot.WriteFieldBegin("id", thrift.STRING, 1); err != nil { return thrift.PrependError(fmt.Sprintf("%T write field begin error 1:id: ", p), err) } @@ -5993,74 +6014,65 @@ func (p *WriteTaggedBatchRawRequestElement) writeField1(oprot thrift.TProtocol) return err } -func (p *WriteTaggedBatchRawRequestElement) writeField2(oprot thrift.TProtocol) (err error) { - if err := oprot.WriteFieldBegin("encodedTags", thrift.STRING, 2); err != nil { - return thrift.PrependError(fmt.Sprintf("%T write field begin error 2:encodedTags: ", p), err) +func (p *WriteBatchRawV2RequestElement) writeField2(oprot thrift.TProtocol) (err error) { + if err := oprot.WriteFieldBegin("datapoint", thrift.STRUCT, 2); err != nil { + return thrift.PrependError(fmt.Sprintf("%T write field begin error 2:datapoint: ", p), err) } - if err := oprot.WriteBinary(p.EncodedTags); err != nil { - return thrift.PrependError(fmt.Sprintf("%T.encodedTags (2) field write error: ", p), err) + if err := p.Datapoint.Write(oprot); err != nil { + return thrift.PrependError(fmt.Sprintf("%T error writing struct: ", p.Datapoint), err) } if err := oprot.WriteFieldEnd(); err != nil { - return thrift.PrependError(fmt.Sprintf("%T write field end error 2:encodedTags: ", p), err) + return thrift.PrependError(fmt.Sprintf("%T write field end error 2:datapoint: ", p), err) } return err } -func (p *WriteTaggedBatchRawRequestElement) writeField3(oprot thrift.TProtocol) (err error) { - if err := oprot.WriteFieldBegin("datapoint", thrift.STRUCT, 3); err != nil { - return thrift.PrependError(fmt.Sprintf("%T write field begin error 3:datapoint: ", p), err) +func (p *WriteBatchRawV2RequestElement) writeField3(oprot thrift.TProtocol) (err error) { + if err := oprot.WriteFieldBegin("nameSpace", thrift.I64, 3); err != nil { + return thrift.PrependError(fmt.Sprintf("%T write field begin error 3:nameSpace: ", p), err) } - if err := p.Datapoint.Write(oprot); err != nil { - return thrift.PrependError(fmt.Sprintf("%T error writing struct: ", p.Datapoint), err) + if err := oprot.WriteI64(int64(p.NameSpace)); err != nil { + return thrift.PrependError(fmt.Sprintf("%T.nameSpace (3) field write error: ", p), err) } if err := oprot.WriteFieldEnd(); err != nil { - return thrift.PrependError(fmt.Sprintf("%T write field end error 3:datapoint: ", p), err) + return thrift.PrependError(fmt.Sprintf("%T write field end error 3:nameSpace: ", p), err) } return err } -func (p *WriteTaggedBatchRawRequestElement) String() string { +func (p *WriteBatchRawV2RequestElement) String() string { if p == nil { return "" } - return fmt.Sprintf("WriteTaggedBatchRawRequestElement(%+v)", *p) + return fmt.Sprintf("WriteBatchRawV2RequestElement(%+v)", *p) } // Attributes: -// - Index -// - Err -type WriteBatchRawError struct { - Index int64 `thrift:"index,1,required" db:"index" json:"index"` - Err *Error `thrift:"err,2,required" db:"err" json:"err"` +// - NameSpace +// - Elements +type WriteTaggedBatchRawRequest struct { + NameSpace []byte `thrift:"nameSpace,1,required" db:"nameSpace" json:"nameSpace"` + Elements []*WriteTaggedBatchRawRequestElement `thrift:"elements,2,required" db:"elements" json:"elements"` } -func NewWriteBatchRawError() *WriteBatchRawError { - return &WriteBatchRawError{} +func NewWriteTaggedBatchRawRequest() *WriteTaggedBatchRawRequest { + return &WriteTaggedBatchRawRequest{} } -func (p *WriteBatchRawError) GetIndex() int64 { - return p.Index +func (p *WriteTaggedBatchRawRequest) GetNameSpace() []byte { + return p.NameSpace } -var WriteBatchRawError_Err_DEFAULT *Error - -func (p *WriteBatchRawError) GetErr() *Error { - if !p.IsSetErr() { - return WriteBatchRawError_Err_DEFAULT - } - return p.Err -} -func (p *WriteBatchRawError) IsSetErr() bool { - return p.Err != nil +func (p *WriteTaggedBatchRawRequest) GetElements() []*WriteTaggedBatchRawRequestElement { + return p.Elements } - -func (p *WriteBatchRawError) Read(iprot thrift.TProtocol) error { +func (p *WriteTaggedBatchRawRequest) Read(iprot thrift.TProtocol) error { if _, err := iprot.ReadStructBegin(); err != nil { return thrift.PrependError(fmt.Sprintf("%T read error: ", p), err) } - var issetIndex bool = false - var issetErr bool = false + var issetNameSpace bool = false + var issetElements bool = false for { _, fieldTypeId, fieldId, err := iprot.ReadFieldBegin() @@ -6075,12 +6087,12 @@ func (p *WriteBatchRawError) Read(iprot thrift.TProtocol) error { if err := p.ReadField1(iprot); err != nil { return err } - issetIndex = true + issetNameSpace = true case 2: if err := p.ReadField2(iprot); err != nil { return err } - issetErr = true + issetElements = true default: if err := iprot.Skip(fieldTypeId); err != nil { return err @@ -6093,36 +6105,46 @@ func (p *WriteBatchRawError) Read(iprot thrift.TProtocol) error { if err := iprot.ReadStructEnd(); err != nil { return thrift.PrependError(fmt.Sprintf("%T read struct end error: ", p), err) } - if !issetIndex { - return thrift.NewTProtocolExceptionWithType(thrift.INVALID_DATA, fmt.Errorf("Required field Index is not set")) + if !issetNameSpace { + return thrift.NewTProtocolExceptionWithType(thrift.INVALID_DATA, fmt.Errorf("Required field NameSpace is not set")) } - if !issetErr { - return thrift.NewTProtocolExceptionWithType(thrift.INVALID_DATA, fmt.Errorf("Required field Err is not set")) + if !issetElements { + return thrift.NewTProtocolExceptionWithType(thrift.INVALID_DATA, fmt.Errorf("Required field Elements is not set")) } return nil } -func (p *WriteBatchRawError) ReadField1(iprot thrift.TProtocol) error { - if v, err := iprot.ReadI64(); err != nil { +func (p *WriteTaggedBatchRawRequest) ReadField1(iprot thrift.TProtocol) error { + if v, err := iprot.ReadBinary(); err != nil { return thrift.PrependError("error reading field 1: ", err) } else { - p.Index = v + p.NameSpace = v } return nil } -func (p *WriteBatchRawError) ReadField2(iprot thrift.TProtocol) error { - p.Err = &Error{ - Type: 0, +func (p *WriteTaggedBatchRawRequest) ReadField2(iprot thrift.TProtocol) error { + _, size, err := iprot.ReadListBegin() + if err != nil { + return thrift.PrependError("error reading list begin: ", err) } - if err := p.Err.Read(iprot); err != nil { - return thrift.PrependError(fmt.Sprintf("%T error reading struct: ", p.Err), err) + tSlice := make([]*WriteTaggedBatchRawRequestElement, 0, size) + p.Elements = tSlice + for i := 0; i < size; i++ { + _elem17 := &WriteTaggedBatchRawRequestElement{} + if err := _elem17.Read(iprot); err != nil { + return thrift.PrependError(fmt.Sprintf("%T error reading struct: ", _elem17), err) + } + p.Elements = append(p.Elements, _elem17) + } + if err := iprot.ReadListEnd(); err != nil { + return thrift.PrependError("error reading list end: ", err) } return nil } -func (p *WriteBatchRawError) Write(oprot thrift.TProtocol) error { - if err := oprot.WriteStructBegin("WriteBatchRawError"); err != nil { +func (p *WriteTaggedBatchRawRequest) Write(oprot thrift.TProtocol) error { + if err := oprot.WriteStructBegin("WriteTaggedBatchRawRequest"); err != nil { return thrift.PrependError(fmt.Sprintf("%T write struct begin error: ", p), err) } if p != nil { @@ -6142,58 +6164,73 @@ func (p *WriteBatchRawError) Write(oprot thrift.TProtocol) error { return nil } -func (p *WriteBatchRawError) writeField1(oprot thrift.TProtocol) (err error) { - if err := oprot.WriteFieldBegin("index", thrift.I64, 1); err != nil { - return thrift.PrependError(fmt.Sprintf("%T write field begin error 1:index: ", p), err) +func (p *WriteTaggedBatchRawRequest) writeField1(oprot thrift.TProtocol) (err error) { + if err := oprot.WriteFieldBegin("nameSpace", thrift.STRING, 1); err != nil { + return thrift.PrependError(fmt.Sprintf("%T write field begin error 1:nameSpace: ", p), err) } - if err := oprot.WriteI64(int64(p.Index)); err != nil { - return thrift.PrependError(fmt.Sprintf("%T.index (1) field write error: ", p), err) + if err := oprot.WriteBinary(p.NameSpace); err != nil { + return thrift.PrependError(fmt.Sprintf("%T.nameSpace (1) field write error: ", p), err) } if err := oprot.WriteFieldEnd(); err != nil { - return thrift.PrependError(fmt.Sprintf("%T write field end error 1:index: ", p), err) + return thrift.PrependError(fmt.Sprintf("%T write field end error 1:nameSpace: ", p), err) } return err } -func (p *WriteBatchRawError) writeField2(oprot thrift.TProtocol) (err error) { - if err := oprot.WriteFieldBegin("err", thrift.STRUCT, 2); err != nil { - return thrift.PrependError(fmt.Sprintf("%T write field begin error 2:err: ", p), err) +func (p *WriteTaggedBatchRawRequest) writeField2(oprot thrift.TProtocol) (err error) { + if err := oprot.WriteFieldBegin("elements", thrift.LIST, 2); err != nil { + return thrift.PrependError(fmt.Sprintf("%T write field begin error 2:elements: ", p), err) } - if err := p.Err.Write(oprot); err != nil { - return thrift.PrependError(fmt.Sprintf("%T error writing struct: ", p.Err), err) + if err := oprot.WriteListBegin(thrift.STRUCT, len(p.Elements)); err != nil { + return thrift.PrependError("error writing list begin: ", err) + } + for _, v := range p.Elements { + if err := v.Write(oprot); err != nil { + return thrift.PrependError(fmt.Sprintf("%T error writing struct: ", v), err) + } + } + if err := oprot.WriteListEnd(); err != nil { + return thrift.PrependError("error writing list end: ", err) } if err := oprot.WriteFieldEnd(); err != nil { - return thrift.PrependError(fmt.Sprintf("%T write field end error 2:err: ", p), err) + return thrift.PrependError(fmt.Sprintf("%T write field end error 2:elements: ", p), err) } return err } -func (p *WriteBatchRawError) String() string { +func (p *WriteTaggedBatchRawRequest) String() string { if p == nil { return "" } - return fmt.Sprintf("WriteBatchRawError(%+v)", *p) + return fmt.Sprintf("WriteTaggedBatchRawRequest(%+v)", *p) } // Attributes: -// - NameSpace -type TruncateRequest struct { - NameSpace []byte `thrift:"nameSpace,1,required" db:"nameSpace" json:"nameSpace"` +// - NameSpaces +// - Elements +type WriteTaggedBatchRawV2Request struct { + NameSpaces [][]byte `thrift:"nameSpaces,1,required" db:"nameSpaces" json:"nameSpaces"` + Elements []*WriteTaggedBatchRawV2RequestElement `thrift:"elements,2,required" db:"elements" json:"elements"` } -func NewTruncateRequest() *TruncateRequest { - return &TruncateRequest{} +func NewWriteTaggedBatchRawV2Request() *WriteTaggedBatchRawV2Request { + return &WriteTaggedBatchRawV2Request{} } -func (p *TruncateRequest) GetNameSpace() []byte { - return p.NameSpace +func (p *WriteTaggedBatchRawV2Request) GetNameSpaces() [][]byte { + return p.NameSpaces } -func (p *TruncateRequest) Read(iprot thrift.TProtocol) error { + +func (p *WriteTaggedBatchRawV2Request) GetElements() []*WriteTaggedBatchRawV2RequestElement { + return p.Elements +} +func (p *WriteTaggedBatchRawV2Request) Read(iprot thrift.TProtocol) error { if _, err := iprot.ReadStructBegin(); err != nil { return thrift.PrependError(fmt.Sprintf("%T read error: ", p), err) } - var issetNameSpace bool = false + var issetNameSpaces bool = false + var issetElements bool = false for { _, fieldTypeId, fieldId, err := iprot.ReadFieldBegin() @@ -6208,7 +6245,12 @@ func (p *TruncateRequest) Read(iprot thrift.TProtocol) error { if err := p.ReadField1(iprot); err != nil { return err } - issetNameSpace = true + issetNameSpaces = true + case 2: + if err := p.ReadField2(iprot); err != nil { + return err + } + issetElements = true default: if err := iprot.Skip(fieldTypeId); err != nil { return err @@ -6221,29 +6263,68 @@ func (p *TruncateRequest) Read(iprot thrift.TProtocol) error { if err := iprot.ReadStructEnd(); err != nil { return thrift.PrependError(fmt.Sprintf("%T read struct end error: ", p), err) } - if !issetNameSpace { - return thrift.NewTProtocolExceptionWithType(thrift.INVALID_DATA, fmt.Errorf("Required field NameSpace is not set")) + if !issetNameSpaces { + return thrift.NewTProtocolExceptionWithType(thrift.INVALID_DATA, fmt.Errorf("Required field NameSpaces is not set")) + } + if !issetElements { + return thrift.NewTProtocolExceptionWithType(thrift.INVALID_DATA, fmt.Errorf("Required field Elements is not set")) } return nil } -func (p *TruncateRequest) ReadField1(iprot thrift.TProtocol) error { - if v, err := iprot.ReadBinary(); err != nil { - return thrift.PrependError("error reading field 1: ", err) - } else { - p.NameSpace = v +func (p *WriteTaggedBatchRawV2Request) ReadField1(iprot thrift.TProtocol) error { + _, size, err := iprot.ReadListBegin() + if err != nil { + return thrift.PrependError("error reading list begin: ", err) + } + tSlice := make([][]byte, 0, size) + p.NameSpaces = tSlice + for i := 0; i < size; i++ { + var _elem18 []byte + if v, err := iprot.ReadBinary(); err != nil { + return thrift.PrependError("error reading field 0: ", err) + } else { + _elem18 = v + } + p.NameSpaces = append(p.NameSpaces, _elem18) + } + if err := iprot.ReadListEnd(); err != nil { + return thrift.PrependError("error reading list end: ", err) } return nil } -func (p *TruncateRequest) Write(oprot thrift.TProtocol) error { - if err := oprot.WriteStructBegin("TruncateRequest"); err != nil { +func (p *WriteTaggedBatchRawV2Request) ReadField2(iprot thrift.TProtocol) error { + _, size, err := iprot.ReadListBegin() + if err != nil { + return thrift.PrependError("error reading list begin: ", err) + } + tSlice := make([]*WriteTaggedBatchRawV2RequestElement, 0, size) + p.Elements = tSlice + for i := 0; i < size; i++ { + _elem19 := &WriteTaggedBatchRawV2RequestElement{} + if err := _elem19.Read(iprot); err != nil { + return thrift.PrependError(fmt.Sprintf("%T error reading struct: ", _elem19), err) + } + p.Elements = append(p.Elements, _elem19) + } + if err := iprot.ReadListEnd(); err != nil { + return thrift.PrependError("error reading list end: ", err) + } + return nil +} + +func (p *WriteTaggedBatchRawV2Request) Write(oprot thrift.TProtocol) error { + if err := oprot.WriteStructBegin("WriteTaggedBatchRawV2Request"); err != nil { return thrift.PrependError(fmt.Sprintf("%T write struct begin error: ", p), err) } if p != nil { if err := p.writeField1(oprot); err != nil { return err } + if err := p.writeField2(oprot); err != nil { + return err + } } if err := oprot.WriteFieldStop(); err != nil { return thrift.PrependError("write field stop error: ", err) @@ -6254,45 +6335,97 @@ func (p *TruncateRequest) Write(oprot thrift.TProtocol) error { return nil } -func (p *TruncateRequest) writeField1(oprot thrift.TProtocol) (err error) { - if err := oprot.WriteFieldBegin("nameSpace", thrift.STRING, 1); err != nil { - return thrift.PrependError(fmt.Sprintf("%T write field begin error 1:nameSpace: ", p), err) +func (p *WriteTaggedBatchRawV2Request) writeField1(oprot thrift.TProtocol) (err error) { + if err := oprot.WriteFieldBegin("nameSpaces", thrift.LIST, 1); err != nil { + return thrift.PrependError(fmt.Sprintf("%T write field begin error 1:nameSpaces: ", p), err) } - if err := oprot.WriteBinary(p.NameSpace); err != nil { - return thrift.PrependError(fmt.Sprintf("%T.nameSpace (1) field write error: ", p), err) + if err := oprot.WriteListBegin(thrift.STRING, len(p.NameSpaces)); err != nil { + return thrift.PrependError("error writing list begin: ", err) + } + for _, v := range p.NameSpaces { + if err := oprot.WriteBinary(v); err != nil { + return thrift.PrependError(fmt.Sprintf("%T. (0) field write error: ", p), err) + } + } + if err := oprot.WriteListEnd(); err != nil { + return thrift.PrependError("error writing list end: ", err) } if err := oprot.WriteFieldEnd(); err != nil { - return thrift.PrependError(fmt.Sprintf("%T write field end error 1:nameSpace: ", p), err) + return thrift.PrependError(fmt.Sprintf("%T write field end error 1:nameSpaces: ", p), err) } return err } -func (p *TruncateRequest) String() string { +func (p *WriteTaggedBatchRawV2Request) writeField2(oprot thrift.TProtocol) (err error) { + if err := oprot.WriteFieldBegin("elements", thrift.LIST, 2); err != nil { + return thrift.PrependError(fmt.Sprintf("%T write field begin error 2:elements: ", p), err) + } + if err := oprot.WriteListBegin(thrift.STRUCT, len(p.Elements)); err != nil { + return thrift.PrependError("error writing list begin: ", err) + } + for _, v := range p.Elements { + if err := v.Write(oprot); err != nil { + return thrift.PrependError(fmt.Sprintf("%T error writing struct: ", v), err) + } + } + if err := oprot.WriteListEnd(); err != nil { + return thrift.PrependError("error writing list end: ", err) + } + if err := oprot.WriteFieldEnd(); err != nil { + return thrift.PrependError(fmt.Sprintf("%T write field end error 2:elements: ", p), err) + } + return err +} + +func (p *WriteTaggedBatchRawV2Request) String() string { if p == nil { return "" } - return fmt.Sprintf("TruncateRequest(%+v)", *p) + return fmt.Sprintf("WriteTaggedBatchRawV2Request(%+v)", *p) } // Attributes: -// - NumSeries -type TruncateResult_ struct { - NumSeries int64 `thrift:"numSeries,1,required" db:"numSeries" json:"numSeries"` -} - -func NewTruncateResult_() *TruncateResult_ { - return &TruncateResult_{} +// - ID +// - EncodedTags +// - Datapoint +type WriteTaggedBatchRawRequestElement struct { + ID []byte `thrift:"id,1,required" db:"id" json:"id"` + EncodedTags []byte `thrift:"encodedTags,2,required" db:"encodedTags" json:"encodedTags"` + Datapoint *Datapoint `thrift:"datapoint,3,required" db:"datapoint" json:"datapoint"` } -func (p *TruncateResult_) GetNumSeries() int64 { - return p.NumSeries +func NewWriteTaggedBatchRawRequestElement() *WriteTaggedBatchRawRequestElement { + return &WriteTaggedBatchRawRequestElement{} } -func (p *TruncateResult_) Read(iprot thrift.TProtocol) error { + +func (p *WriteTaggedBatchRawRequestElement) GetID() []byte { + return p.ID +} + +func (p *WriteTaggedBatchRawRequestElement) GetEncodedTags() []byte { + return p.EncodedTags +} + +var WriteTaggedBatchRawRequestElement_Datapoint_DEFAULT *Datapoint + +func (p *WriteTaggedBatchRawRequestElement) GetDatapoint() *Datapoint { + if !p.IsSetDatapoint() { + return WriteTaggedBatchRawRequestElement_Datapoint_DEFAULT + } + return p.Datapoint +} +func (p *WriteTaggedBatchRawRequestElement) IsSetDatapoint() bool { + return p.Datapoint != nil +} + +func (p *WriteTaggedBatchRawRequestElement) Read(iprot thrift.TProtocol) error { if _, err := iprot.ReadStructBegin(); err != nil { return thrift.PrependError(fmt.Sprintf("%T read error: ", p), err) } - var issetNumSeries bool = false + var issetID bool = false + var issetEncodedTags bool = false + var issetDatapoint bool = false for { _, fieldTypeId, fieldId, err := iprot.ReadFieldBegin() @@ -6307,7 +6440,17 @@ func (p *TruncateResult_) Read(iprot thrift.TProtocol) error { if err := p.ReadField1(iprot); err != nil { return err } - issetNumSeries = true + issetID = true + case 2: + if err := p.ReadField2(iprot); err != nil { + return err + } + issetEncodedTags = true + case 3: + if err := p.ReadField3(iprot); err != nil { + return err + } + issetDatapoint = true default: if err := iprot.Skip(fieldTypeId); err != nil { return err @@ -6320,29 +6463,60 @@ func (p *TruncateResult_) Read(iprot thrift.TProtocol) error { if err := iprot.ReadStructEnd(); err != nil { return thrift.PrependError(fmt.Sprintf("%T read struct end error: ", p), err) } - if !issetNumSeries { - return thrift.NewTProtocolExceptionWithType(thrift.INVALID_DATA, fmt.Errorf("Required field NumSeries is not set")) + if !issetID { + return thrift.NewTProtocolExceptionWithType(thrift.INVALID_DATA, fmt.Errorf("Required field ID is not set")) + } + if !issetEncodedTags { + return thrift.NewTProtocolExceptionWithType(thrift.INVALID_DATA, fmt.Errorf("Required field EncodedTags is not set")) + } + if !issetDatapoint { + return thrift.NewTProtocolExceptionWithType(thrift.INVALID_DATA, fmt.Errorf("Required field Datapoint is not set")) } return nil } -func (p *TruncateResult_) ReadField1(iprot thrift.TProtocol) error { - if v, err := iprot.ReadI64(); err != nil { +func (p *WriteTaggedBatchRawRequestElement) ReadField1(iprot thrift.TProtocol) error { + if v, err := iprot.ReadBinary(); err != nil { return thrift.PrependError("error reading field 1: ", err) } else { - p.NumSeries = v + p.ID = v } return nil } -func (p *TruncateResult_) Write(oprot thrift.TProtocol) error { - if err := oprot.WriteStructBegin("TruncateResult"); err != nil { +func (p *WriteTaggedBatchRawRequestElement) ReadField2(iprot thrift.TProtocol) error { + if v, err := iprot.ReadBinary(); err != nil { + return thrift.PrependError("error reading field 2: ", err) + } else { + p.EncodedTags = v + } + return nil +} + +func (p *WriteTaggedBatchRawRequestElement) ReadField3(iprot thrift.TProtocol) error { + p.Datapoint = &Datapoint{ + TimestampTimeType: 0, + } + if err := p.Datapoint.Read(iprot); err != nil { + return thrift.PrependError(fmt.Sprintf("%T error reading struct: ", p.Datapoint), err) + } + return nil +} + +func (p *WriteTaggedBatchRawRequestElement) Write(oprot thrift.TProtocol) error { + if err := oprot.WriteStructBegin("WriteTaggedBatchRawRequestElement"); err != nil { return thrift.PrependError(fmt.Sprintf("%T write struct begin error: ", p), err) } if p != nil { if err := p.writeField1(oprot); err != nil { return err } + if err := p.writeField2(oprot); err != nil { + return err + } + if err := p.writeField3(oprot); err != nil { + return err + } } if err := oprot.WriteFieldStop(); err != nil { return thrift.PrependError("write field stop error: ", err) @@ -6353,59 +6527,101 @@ func (p *TruncateResult_) Write(oprot thrift.TProtocol) error { return nil } -func (p *TruncateResult_) writeField1(oprot thrift.TProtocol) (err error) { - if err := oprot.WriteFieldBegin("numSeries", thrift.I64, 1); err != nil { - return thrift.PrependError(fmt.Sprintf("%T write field begin error 1:numSeries: ", p), err) +func (p *WriteTaggedBatchRawRequestElement) writeField1(oprot thrift.TProtocol) (err error) { + if err := oprot.WriteFieldBegin("id", thrift.STRING, 1); err != nil { + return thrift.PrependError(fmt.Sprintf("%T write field begin error 1:id: ", p), err) } - if err := oprot.WriteI64(int64(p.NumSeries)); err != nil { - return thrift.PrependError(fmt.Sprintf("%T.numSeries (1) field write error: ", p), err) + if err := oprot.WriteBinary(p.ID); err != nil { + return thrift.PrependError(fmt.Sprintf("%T.id (1) field write error: ", p), err) } if err := oprot.WriteFieldEnd(); err != nil { - return thrift.PrependError(fmt.Sprintf("%T write field end error 1:numSeries: ", p), err) + return thrift.PrependError(fmt.Sprintf("%T write field end error 1:id: ", p), err) } return err } -func (p *TruncateResult_) String() string { +func (p *WriteTaggedBatchRawRequestElement) writeField2(oprot thrift.TProtocol) (err error) { + if err := oprot.WriteFieldBegin("encodedTags", thrift.STRING, 2); err != nil { + return thrift.PrependError(fmt.Sprintf("%T write field begin error 2:encodedTags: ", p), err) + } + if err := oprot.WriteBinary(p.EncodedTags); err != nil { + return thrift.PrependError(fmt.Sprintf("%T.encodedTags (2) field write error: ", p), err) + } + if err := oprot.WriteFieldEnd(); err != nil { + return thrift.PrependError(fmt.Sprintf("%T write field end error 2:encodedTags: ", p), err) + } + return err +} + +func (p *WriteTaggedBatchRawRequestElement) writeField3(oprot thrift.TProtocol) (err error) { + if err := oprot.WriteFieldBegin("datapoint", thrift.STRUCT, 3); err != nil { + return thrift.PrependError(fmt.Sprintf("%T write field begin error 3:datapoint: ", p), err) + } + if err := p.Datapoint.Write(oprot); err != nil { + return thrift.PrependError(fmt.Sprintf("%T error writing struct: ", p.Datapoint), err) + } + if err := oprot.WriteFieldEnd(); err != nil { + return thrift.PrependError(fmt.Sprintf("%T write field end error 3:datapoint: ", p), err) + } + return err +} + +func (p *WriteTaggedBatchRawRequestElement) String() string { if p == nil { return "" } - return fmt.Sprintf("TruncateResult_(%+v)", *p) + return fmt.Sprintf("WriteTaggedBatchRawRequestElement(%+v)", *p) } // Attributes: -// - Ok -// - Status -// - Bootstrapped -type NodeHealthResult_ struct { - Ok bool `thrift:"ok,1,required" db:"ok" json:"ok"` - Status string `thrift:"status,2,required" db:"status" json:"status"` - Bootstrapped bool `thrift:"bootstrapped,3,required" db:"bootstrapped" json:"bootstrapped"` +// - ID +// - EncodedTags +// - Datapoint +// - NameSpace +type WriteTaggedBatchRawV2RequestElement struct { + ID []byte `thrift:"id,1,required" db:"id" json:"id"` + EncodedTags []byte `thrift:"encodedTags,2,required" db:"encodedTags" json:"encodedTags"` + Datapoint *Datapoint `thrift:"datapoint,3,required" db:"datapoint" json:"datapoint"` + NameSpace int64 `thrift:"nameSpace,4,required" db:"nameSpace" json:"nameSpace"` } -func NewNodeHealthResult_() *NodeHealthResult_ { - return &NodeHealthResult_{} +func NewWriteTaggedBatchRawV2RequestElement() *WriteTaggedBatchRawV2RequestElement { + return &WriteTaggedBatchRawV2RequestElement{} } -func (p *NodeHealthResult_) GetOk() bool { - return p.Ok +func (p *WriteTaggedBatchRawV2RequestElement) GetID() []byte { + return p.ID } -func (p *NodeHealthResult_) GetStatus() string { - return p.Status +func (p *WriteTaggedBatchRawV2RequestElement) GetEncodedTags() []byte { + return p.EncodedTags } -func (p *NodeHealthResult_) GetBootstrapped() bool { - return p.Bootstrapped +var WriteTaggedBatchRawV2RequestElement_Datapoint_DEFAULT *Datapoint + +func (p *WriteTaggedBatchRawV2RequestElement) GetDatapoint() *Datapoint { + if !p.IsSetDatapoint() { + return WriteTaggedBatchRawV2RequestElement_Datapoint_DEFAULT + } + return p.Datapoint } -func (p *NodeHealthResult_) Read(iprot thrift.TProtocol) error { + +func (p *WriteTaggedBatchRawV2RequestElement) GetNameSpace() int64 { + return p.NameSpace +} +func (p *WriteTaggedBatchRawV2RequestElement) IsSetDatapoint() bool { + return p.Datapoint != nil +} + +func (p *WriteTaggedBatchRawV2RequestElement) Read(iprot thrift.TProtocol) error { if _, err := iprot.ReadStructBegin(); err != nil { return thrift.PrependError(fmt.Sprintf("%T read error: ", p), err) } - var issetOk bool = false - var issetStatus bool = false - var issetBootstrapped bool = false + var issetID bool = false + var issetEncodedTags bool = false + var issetDatapoint bool = false + var issetNameSpace bool = false for { _, fieldTypeId, fieldId, err := iprot.ReadFieldBegin() @@ -6420,17 +6636,22 @@ func (p *NodeHealthResult_) Read(iprot thrift.TProtocol) error { if err := p.ReadField1(iprot); err != nil { return err } - issetOk = true + issetID = true case 2: if err := p.ReadField2(iprot); err != nil { return err } - issetStatus = true + issetEncodedTags = true case 3: if err := p.ReadField3(iprot); err != nil { return err } - issetBootstrapped = true + issetDatapoint = true + case 4: + if err := p.ReadField4(iprot); err != nil { + return err + } + issetNameSpace = true default: if err := iprot.Skip(fieldTypeId); err != nil { return err @@ -6443,47 +6664,60 @@ func (p *NodeHealthResult_) Read(iprot thrift.TProtocol) error { if err := iprot.ReadStructEnd(); err != nil { return thrift.PrependError(fmt.Sprintf("%T read struct end error: ", p), err) } - if !issetOk { - return thrift.NewTProtocolExceptionWithType(thrift.INVALID_DATA, fmt.Errorf("Required field Ok is not set")) + if !issetID { + return thrift.NewTProtocolExceptionWithType(thrift.INVALID_DATA, fmt.Errorf("Required field ID is not set")) } - if !issetStatus { - return thrift.NewTProtocolExceptionWithType(thrift.INVALID_DATA, fmt.Errorf("Required field Status is not set")) + if !issetEncodedTags { + return thrift.NewTProtocolExceptionWithType(thrift.INVALID_DATA, fmt.Errorf("Required field EncodedTags is not set")) } - if !issetBootstrapped { - return thrift.NewTProtocolExceptionWithType(thrift.INVALID_DATA, fmt.Errorf("Required field Bootstrapped is not set")) + if !issetDatapoint { + return thrift.NewTProtocolExceptionWithType(thrift.INVALID_DATA, fmt.Errorf("Required field Datapoint is not set")) + } + if !issetNameSpace { + return thrift.NewTProtocolExceptionWithType(thrift.INVALID_DATA, fmt.Errorf("Required field NameSpace is not set")) } return nil } -func (p *NodeHealthResult_) ReadField1(iprot thrift.TProtocol) error { - if v, err := iprot.ReadBool(); err != nil { +func (p *WriteTaggedBatchRawV2RequestElement) ReadField1(iprot thrift.TProtocol) error { + if v, err := iprot.ReadBinary(); err != nil { return thrift.PrependError("error reading field 1: ", err) } else { - p.Ok = v + p.ID = v } return nil } -func (p *NodeHealthResult_) ReadField2(iprot thrift.TProtocol) error { - if v, err := iprot.ReadString(); err != nil { +func (p *WriteTaggedBatchRawV2RequestElement) ReadField2(iprot thrift.TProtocol) error { + if v, err := iprot.ReadBinary(); err != nil { return thrift.PrependError("error reading field 2: ", err) } else { - p.Status = v + p.EncodedTags = v } return nil } -func (p *NodeHealthResult_) ReadField3(iprot thrift.TProtocol) error { - if v, err := iprot.ReadBool(); err != nil { - return thrift.PrependError("error reading field 3: ", err) +func (p *WriteTaggedBatchRawV2RequestElement) ReadField3(iprot thrift.TProtocol) error { + p.Datapoint = &Datapoint{ + TimestampTimeType: 0, + } + if err := p.Datapoint.Read(iprot); err != nil { + return thrift.PrependError(fmt.Sprintf("%T error reading struct: ", p.Datapoint), err) + } + return nil +} + +func (p *WriteTaggedBatchRawV2RequestElement) ReadField4(iprot thrift.TProtocol) error { + if v, err := iprot.ReadI64(); err != nil { + return thrift.PrependError("error reading field 4: ", err) } else { - p.Bootstrapped = v + p.NameSpace = v } return nil } -func (p *NodeHealthResult_) Write(oprot thrift.TProtocol) error { - if err := oprot.WriteStructBegin("NodeHealthResult"); err != nil { +func (p *WriteTaggedBatchRawV2RequestElement) Write(oprot thrift.TProtocol) error { + if err := oprot.WriteStructBegin("WriteTaggedBatchRawV2RequestElement"); err != nil { return thrift.PrependError(fmt.Sprintf("%T write struct begin error: ", p), err) } if p != nil { @@ -6496,6 +6730,9 @@ func (p *NodeHealthResult_) Write(oprot thrift.TProtocol) error { if err := p.writeField3(oprot); err != nil { return err } + if err := p.writeField4(oprot); err != nil { + return err + } } if err := oprot.WriteFieldStop(); err != nil { return thrift.PrependError("write field stop error: ", err) @@ -6506,64 +6743,101 @@ func (p *NodeHealthResult_) Write(oprot thrift.TProtocol) error { return nil } -func (p *NodeHealthResult_) writeField1(oprot thrift.TProtocol) (err error) { - if err := oprot.WriteFieldBegin("ok", thrift.BOOL, 1); err != nil { - return thrift.PrependError(fmt.Sprintf("%T write field begin error 1:ok: ", p), err) +func (p *WriteTaggedBatchRawV2RequestElement) writeField1(oprot thrift.TProtocol) (err error) { + if err := oprot.WriteFieldBegin("id", thrift.STRING, 1); err != nil { + return thrift.PrependError(fmt.Sprintf("%T write field begin error 1:id: ", p), err) } - if err := oprot.WriteBool(bool(p.Ok)); err != nil { - return thrift.PrependError(fmt.Sprintf("%T.ok (1) field write error: ", p), err) + if err := oprot.WriteBinary(p.ID); err != nil { + return thrift.PrependError(fmt.Sprintf("%T.id (1) field write error: ", p), err) } if err := oprot.WriteFieldEnd(); err != nil { - return thrift.PrependError(fmt.Sprintf("%T write field end error 1:ok: ", p), err) + return thrift.PrependError(fmt.Sprintf("%T write field end error 1:id: ", p), err) } return err } -func (p *NodeHealthResult_) writeField2(oprot thrift.TProtocol) (err error) { - if err := oprot.WriteFieldBegin("status", thrift.STRING, 2); err != nil { - return thrift.PrependError(fmt.Sprintf("%T write field begin error 2:status: ", p), err) +func (p *WriteTaggedBatchRawV2RequestElement) writeField2(oprot thrift.TProtocol) (err error) { + if err := oprot.WriteFieldBegin("encodedTags", thrift.STRING, 2); err != nil { + return thrift.PrependError(fmt.Sprintf("%T write field begin error 2:encodedTags: ", p), err) } - if err := oprot.WriteString(string(p.Status)); err != nil { - return thrift.PrependError(fmt.Sprintf("%T.status (2) field write error: ", p), err) + if err := oprot.WriteBinary(p.EncodedTags); err != nil { + return thrift.PrependError(fmt.Sprintf("%T.encodedTags (2) field write error: ", p), err) } if err := oprot.WriteFieldEnd(); err != nil { - return thrift.PrependError(fmt.Sprintf("%T write field end error 2:status: ", p), err) + return thrift.PrependError(fmt.Sprintf("%T write field end error 2:encodedTags: ", p), err) } return err } -func (p *NodeHealthResult_) writeField3(oprot thrift.TProtocol) (err error) { - if err := oprot.WriteFieldBegin("bootstrapped", thrift.BOOL, 3); err != nil { - return thrift.PrependError(fmt.Sprintf("%T write field begin error 3:bootstrapped: ", p), err) +func (p *WriteTaggedBatchRawV2RequestElement) writeField3(oprot thrift.TProtocol) (err error) { + if err := oprot.WriteFieldBegin("datapoint", thrift.STRUCT, 3); err != nil { + return thrift.PrependError(fmt.Sprintf("%T write field begin error 3:datapoint: ", p), err) } - if err := oprot.WriteBool(bool(p.Bootstrapped)); err != nil { - return thrift.PrependError(fmt.Sprintf("%T.bootstrapped (3) field write error: ", p), err) + if err := p.Datapoint.Write(oprot); err != nil { + return thrift.PrependError(fmt.Sprintf("%T error writing struct: ", p.Datapoint), err) } if err := oprot.WriteFieldEnd(); err != nil { - return thrift.PrependError(fmt.Sprintf("%T write field end error 3:bootstrapped: ", p), err) + return thrift.PrependError(fmt.Sprintf("%T write field end error 3:datapoint: ", p), err) } return err } -func (p *NodeHealthResult_) String() string { +func (p *WriteTaggedBatchRawV2RequestElement) writeField4(oprot thrift.TProtocol) (err error) { + if err := oprot.WriteFieldBegin("nameSpace", thrift.I64, 4); err != nil { + return thrift.PrependError(fmt.Sprintf("%T write field begin error 4:nameSpace: ", p), err) + } + if err := oprot.WriteI64(int64(p.NameSpace)); err != nil { + return thrift.PrependError(fmt.Sprintf("%T.nameSpace (4) field write error: ", p), err) + } + if err := oprot.WriteFieldEnd(); err != nil { + return thrift.PrependError(fmt.Sprintf("%T write field end error 4:nameSpace: ", p), err) + } + return err +} + +func (p *WriteTaggedBatchRawV2RequestElement) String() string { if p == nil { return "" } - return fmt.Sprintf("NodeHealthResult_(%+v)", *p) + return fmt.Sprintf("WriteTaggedBatchRawV2RequestElement(%+v)", *p) } -type NodeBootstrappedResult_ struct { +// Attributes: +// - Index +// - Err +type WriteBatchRawError struct { + Index int64 `thrift:"index,1,required" db:"index" json:"index"` + Err *Error `thrift:"err,2,required" db:"err" json:"err"` } -func NewNodeBootstrappedResult_() *NodeBootstrappedResult_ { - return &NodeBootstrappedResult_{} +func NewWriteBatchRawError() *WriteBatchRawError { + return &WriteBatchRawError{} } -func (p *NodeBootstrappedResult_) Read(iprot thrift.TProtocol) error { +func (p *WriteBatchRawError) GetIndex() int64 { + return p.Index +} + +var WriteBatchRawError_Err_DEFAULT *Error + +func (p *WriteBatchRawError) GetErr() *Error { + if !p.IsSetErr() { + return WriteBatchRawError_Err_DEFAULT + } + return p.Err +} +func (p *WriteBatchRawError) IsSetErr() bool { + return p.Err != nil +} + +func (p *WriteBatchRawError) Read(iprot thrift.TProtocol) error { if _, err := iprot.ReadStructBegin(); err != nil { return thrift.PrependError(fmt.Sprintf("%T read error: ", p), err) } + var issetIndex bool = false + var issetErr bool = false + for { _, fieldTypeId, fieldId, err := iprot.ReadFieldBegin() if err != nil { @@ -6572,8 +6846,21 @@ func (p *NodeBootstrappedResult_) Read(iprot thrift.TProtocol) error { if fieldTypeId == thrift.STOP { break } - if err := iprot.Skip(fieldTypeId); err != nil { - return err + switch fieldId { + case 1: + if err := p.ReadField1(iprot); err != nil { + return err + } + issetIndex = true + case 2: + if err := p.ReadField2(iprot); err != nil { + return err + } + issetErr = true + default: + if err := iprot.Skip(fieldTypeId); err != nil { + return err + } } if err := iprot.ReadFieldEnd(); err != nil { return err @@ -6582,14 +6869,45 @@ func (p *NodeBootstrappedResult_) Read(iprot thrift.TProtocol) error { if err := iprot.ReadStructEnd(); err != nil { return thrift.PrependError(fmt.Sprintf("%T read struct end error: ", p), err) } + if !issetIndex { + return thrift.NewTProtocolExceptionWithType(thrift.INVALID_DATA, fmt.Errorf("Required field Index is not set")) + } + if !issetErr { + return thrift.NewTProtocolExceptionWithType(thrift.INVALID_DATA, fmt.Errorf("Required field Err is not set")) + } return nil } -func (p *NodeBootstrappedResult_) Write(oprot thrift.TProtocol) error { - if err := oprot.WriteStructBegin("NodeBootstrappedResult"); err != nil { +func (p *WriteBatchRawError) ReadField1(iprot thrift.TProtocol) error { + if v, err := iprot.ReadI64(); err != nil { + return thrift.PrependError("error reading field 1: ", err) + } else { + p.Index = v + } + return nil +} + +func (p *WriteBatchRawError) ReadField2(iprot thrift.TProtocol) error { + p.Err = &Error{ + Type: 0, + } + if err := p.Err.Read(iprot); err != nil { + return thrift.PrependError(fmt.Sprintf("%T error reading struct: ", p.Err), err) + } + return nil +} + +func (p *WriteBatchRawError) Write(oprot thrift.TProtocol) error { + if err := oprot.WriteStructBegin("WriteBatchRawError"); err != nil { return thrift.PrependError(fmt.Sprintf("%T write struct begin error: ", p), err) } if p != nil { + if err := p.writeField1(oprot); err != nil { + return err + } + if err := p.writeField2(oprot); err != nil { + return err + } } if err := oprot.WriteFieldStop(); err != nil { return thrift.PrependError("write field stop error: ", err) @@ -6600,25 +6918,59 @@ func (p *NodeBootstrappedResult_) Write(oprot thrift.TProtocol) error { return nil } -func (p *NodeBootstrappedResult_) String() string { +func (p *WriteBatchRawError) writeField1(oprot thrift.TProtocol) (err error) { + if err := oprot.WriteFieldBegin("index", thrift.I64, 1); err != nil { + return thrift.PrependError(fmt.Sprintf("%T write field begin error 1:index: ", p), err) + } + if err := oprot.WriteI64(int64(p.Index)); err != nil { + return thrift.PrependError(fmt.Sprintf("%T.index (1) field write error: ", p), err) + } + if err := oprot.WriteFieldEnd(); err != nil { + return thrift.PrependError(fmt.Sprintf("%T write field end error 1:index: ", p), err) + } + return err +} + +func (p *WriteBatchRawError) writeField2(oprot thrift.TProtocol) (err error) { + if err := oprot.WriteFieldBegin("err", thrift.STRUCT, 2); err != nil { + return thrift.PrependError(fmt.Sprintf("%T write field begin error 2:err: ", p), err) + } + if err := p.Err.Write(oprot); err != nil { + return thrift.PrependError(fmt.Sprintf("%T error writing struct: ", p.Err), err) + } + if err := oprot.WriteFieldEnd(); err != nil { + return thrift.PrependError(fmt.Sprintf("%T write field end error 2:err: ", p), err) + } + return err +} + +func (p *WriteBatchRawError) String() string { if p == nil { return "" } - return fmt.Sprintf("NodeBootstrappedResult_(%+v)", *p) + return fmt.Sprintf("WriteBatchRawError(%+v)", *p) } -type NodeBootstrappedInPlacementOrNoPlacementResult_ struct { +// Attributes: +// - NameSpace +type TruncateRequest struct { + NameSpace []byte `thrift:"nameSpace,1,required" db:"nameSpace" json:"nameSpace"` } -func NewNodeBootstrappedInPlacementOrNoPlacementResult_() *NodeBootstrappedInPlacementOrNoPlacementResult_ { - return &NodeBootstrappedInPlacementOrNoPlacementResult_{} +func NewTruncateRequest() *TruncateRequest { + return &TruncateRequest{} } -func (p *NodeBootstrappedInPlacementOrNoPlacementResult_) Read(iprot thrift.TProtocol) error { +func (p *TruncateRequest) GetNameSpace() []byte { + return p.NameSpace +} +func (p *TruncateRequest) Read(iprot thrift.TProtocol) error { if _, err := iprot.ReadStructBegin(); err != nil { return thrift.PrependError(fmt.Sprintf("%T read error: ", p), err) } + var issetNameSpace bool = false + for { _, fieldTypeId, fieldId, err := iprot.ReadFieldBegin() if err != nil { @@ -6627,8 +6979,16 @@ func (p *NodeBootstrappedInPlacementOrNoPlacementResult_) Read(iprot thrift.TPro if fieldTypeId == thrift.STOP { break } - if err := iprot.Skip(fieldTypeId); err != nil { - return err + switch fieldId { + case 1: + if err := p.ReadField1(iprot); err != nil { + return err + } + issetNameSpace = true + default: + if err := iprot.Skip(fieldTypeId); err != nil { + return err + } } if err := iprot.ReadFieldEnd(); err != nil { return err @@ -6637,14 +6997,29 @@ func (p *NodeBootstrappedInPlacementOrNoPlacementResult_) Read(iprot thrift.TPro if err := iprot.ReadStructEnd(); err != nil { return thrift.PrependError(fmt.Sprintf("%T read struct end error: ", p), err) } + if !issetNameSpace { + return thrift.NewTProtocolExceptionWithType(thrift.INVALID_DATA, fmt.Errorf("Required field NameSpace is not set")) + } return nil } -func (p *NodeBootstrappedInPlacementOrNoPlacementResult_) Write(oprot thrift.TProtocol) error { - if err := oprot.WriteStructBegin("NodeBootstrappedInPlacementOrNoPlacementResult"); err != nil { +func (p *TruncateRequest) ReadField1(iprot thrift.TProtocol) error { + if v, err := iprot.ReadBinary(); err != nil { + return thrift.PrependError("error reading field 1: ", err) + } else { + p.NameSpace = v + } + return nil +} + +func (p *TruncateRequest) Write(oprot thrift.TProtocol) error { + if err := oprot.WriteStructBegin("TruncateRequest"); err != nil { return thrift.PrependError(fmt.Sprintf("%T write struct begin error: ", p), err) } if p != nil { + if err := p.writeField1(oprot); err != nil { + return err + } } if err := oprot.WriteFieldStop(); err != nil { return thrift.PrependError("write field stop error: ", err) @@ -6655,46 +7030,45 @@ func (p *NodeBootstrappedInPlacementOrNoPlacementResult_) Write(oprot thrift.TPr return nil } -func (p *NodeBootstrappedInPlacementOrNoPlacementResult_) String() string { +func (p *TruncateRequest) writeField1(oprot thrift.TProtocol) (err error) { + if err := oprot.WriteFieldBegin("nameSpace", thrift.STRING, 1); err != nil { + return thrift.PrependError(fmt.Sprintf("%T write field begin error 1:nameSpace: ", p), err) + } + if err := oprot.WriteBinary(p.NameSpace); err != nil { + return thrift.PrependError(fmt.Sprintf("%T.nameSpace (1) field write error: ", p), err) + } + if err := oprot.WriteFieldEnd(); err != nil { + return thrift.PrependError(fmt.Sprintf("%T write field end error 1:nameSpace: ", p), err) + } + return err +} + +func (p *TruncateRequest) String() string { if p == nil { return "" } - return fmt.Sprintf("NodeBootstrappedInPlacementOrNoPlacementResult_(%+v)", *p) + return fmt.Sprintf("TruncateRequest(%+v)", *p) } // Attributes: -// - LimitEnabled -// - LimitMbps -// - LimitCheckEvery -type NodePersistRateLimitResult_ struct { - LimitEnabled bool `thrift:"limitEnabled,1,required" db:"limitEnabled" json:"limitEnabled"` - LimitMbps float64 `thrift:"limitMbps,2,required" db:"limitMbps" json:"limitMbps"` - LimitCheckEvery int64 `thrift:"limitCheckEvery,3,required" db:"limitCheckEvery" json:"limitCheckEvery"` -} - -func NewNodePersistRateLimitResult_() *NodePersistRateLimitResult_ { - return &NodePersistRateLimitResult_{} -} - -func (p *NodePersistRateLimitResult_) GetLimitEnabled() bool { - return p.LimitEnabled +// - NumSeries +type TruncateResult_ struct { + NumSeries int64 `thrift:"numSeries,1,required" db:"numSeries" json:"numSeries"` } -func (p *NodePersistRateLimitResult_) GetLimitMbps() float64 { - return p.LimitMbps +func NewTruncateResult_() *TruncateResult_ { + return &TruncateResult_{} } -func (p *NodePersistRateLimitResult_) GetLimitCheckEvery() int64 { - return p.LimitCheckEvery +func (p *TruncateResult_) GetNumSeries() int64 { + return p.NumSeries } -func (p *NodePersistRateLimitResult_) Read(iprot thrift.TProtocol) error { +func (p *TruncateResult_) Read(iprot thrift.TProtocol) error { if _, err := iprot.ReadStructBegin(); err != nil { return thrift.PrependError(fmt.Sprintf("%T read error: ", p), err) } - var issetLimitEnabled bool = false - var issetLimitMbps bool = false - var issetLimitCheckEvery bool = false + var issetNumSeries bool = false for { _, fieldTypeId, fieldId, err := iprot.ReadFieldBegin() @@ -6709,17 +7083,7 @@ func (p *NodePersistRateLimitResult_) Read(iprot thrift.TProtocol) error { if err := p.ReadField1(iprot); err != nil { return err } - issetLimitEnabled = true - case 2: - if err := p.ReadField2(iprot); err != nil { - return err - } - issetLimitMbps = true - case 3: - if err := p.ReadField3(iprot); err != nil { - return err - } - issetLimitCheckEvery = true + issetNumSeries = true default: if err := iprot.Skip(fieldTypeId); err != nil { return err @@ -6732,59 +7096,29 @@ func (p *NodePersistRateLimitResult_) Read(iprot thrift.TProtocol) error { if err := iprot.ReadStructEnd(); err != nil { return thrift.PrependError(fmt.Sprintf("%T read struct end error: ", p), err) } - if !issetLimitEnabled { - return thrift.NewTProtocolExceptionWithType(thrift.INVALID_DATA, fmt.Errorf("Required field LimitEnabled is not set")) - } - if !issetLimitMbps { - return thrift.NewTProtocolExceptionWithType(thrift.INVALID_DATA, fmt.Errorf("Required field LimitMbps is not set")) - } - if !issetLimitCheckEvery { - return thrift.NewTProtocolExceptionWithType(thrift.INVALID_DATA, fmt.Errorf("Required field LimitCheckEvery is not set")) - } - return nil -} - -func (p *NodePersistRateLimitResult_) ReadField1(iprot thrift.TProtocol) error { - if v, err := iprot.ReadBool(); err != nil { - return thrift.PrependError("error reading field 1: ", err) - } else { - p.LimitEnabled = v - } - return nil -} - -func (p *NodePersistRateLimitResult_) ReadField2(iprot thrift.TProtocol) error { - if v, err := iprot.ReadDouble(); err != nil { - return thrift.PrependError("error reading field 2: ", err) - } else { - p.LimitMbps = v + if !issetNumSeries { + return thrift.NewTProtocolExceptionWithType(thrift.INVALID_DATA, fmt.Errorf("Required field NumSeries is not set")) } return nil } -func (p *NodePersistRateLimitResult_) ReadField3(iprot thrift.TProtocol) error { +func (p *TruncateResult_) ReadField1(iprot thrift.TProtocol) error { if v, err := iprot.ReadI64(); err != nil { - return thrift.PrependError("error reading field 3: ", err) + return thrift.PrependError("error reading field 1: ", err) } else { - p.LimitCheckEvery = v + p.NumSeries = v } return nil } -func (p *NodePersistRateLimitResult_) Write(oprot thrift.TProtocol) error { - if err := oprot.WriteStructBegin("NodePersistRateLimitResult"); err != nil { +func (p *TruncateResult_) Write(oprot thrift.TProtocol) error { + if err := oprot.WriteStructBegin("TruncateResult"); err != nil { return thrift.PrependError(fmt.Sprintf("%T write struct begin error: ", p), err) } if p != nil { if err := p.writeField1(oprot); err != nil { return err } - if err := p.writeField2(oprot); err != nil { - return err - } - if err := p.writeField3(oprot); err != nil { - return err - } } if err := oprot.WriteFieldStop(); err != nil { return thrift.PrependError("write field stop error: ", err) @@ -6795,109 +7129,60 @@ func (p *NodePersistRateLimitResult_) Write(oprot thrift.TProtocol) error { return nil } -func (p *NodePersistRateLimitResult_) writeField1(oprot thrift.TProtocol) (err error) { - if err := oprot.WriteFieldBegin("limitEnabled", thrift.BOOL, 1); err != nil { - return thrift.PrependError(fmt.Sprintf("%T write field begin error 1:limitEnabled: ", p), err) - } - if err := oprot.WriteBool(bool(p.LimitEnabled)); err != nil { - return thrift.PrependError(fmt.Sprintf("%T.limitEnabled (1) field write error: ", p), err) - } - if err := oprot.WriteFieldEnd(); err != nil { - return thrift.PrependError(fmt.Sprintf("%T write field end error 1:limitEnabled: ", p), err) - } - return err -} - -func (p *NodePersistRateLimitResult_) writeField2(oprot thrift.TProtocol) (err error) { - if err := oprot.WriteFieldBegin("limitMbps", thrift.DOUBLE, 2); err != nil { - return thrift.PrependError(fmt.Sprintf("%T write field begin error 2:limitMbps: ", p), err) - } - if err := oprot.WriteDouble(float64(p.LimitMbps)); err != nil { - return thrift.PrependError(fmt.Sprintf("%T.limitMbps (2) field write error: ", p), err) - } - if err := oprot.WriteFieldEnd(); err != nil { - return thrift.PrependError(fmt.Sprintf("%T write field end error 2:limitMbps: ", p), err) - } - return err -} - -func (p *NodePersistRateLimitResult_) writeField3(oprot thrift.TProtocol) (err error) { - if err := oprot.WriteFieldBegin("limitCheckEvery", thrift.I64, 3); err != nil { - return thrift.PrependError(fmt.Sprintf("%T write field begin error 3:limitCheckEvery: ", p), err) +func (p *TruncateResult_) writeField1(oprot thrift.TProtocol) (err error) { + if err := oprot.WriteFieldBegin("numSeries", thrift.I64, 1); err != nil { + return thrift.PrependError(fmt.Sprintf("%T write field begin error 1:numSeries: ", p), err) } - if err := oprot.WriteI64(int64(p.LimitCheckEvery)); err != nil { - return thrift.PrependError(fmt.Sprintf("%T.limitCheckEvery (3) field write error: ", p), err) + if err := oprot.WriteI64(int64(p.NumSeries)); err != nil { + return thrift.PrependError(fmt.Sprintf("%T.numSeries (1) field write error: ", p), err) } if err := oprot.WriteFieldEnd(); err != nil { - return thrift.PrependError(fmt.Sprintf("%T write field end error 3:limitCheckEvery: ", p), err) + return thrift.PrependError(fmt.Sprintf("%T write field end error 1:numSeries: ", p), err) } return err } -func (p *NodePersistRateLimitResult_) String() string { +func (p *TruncateResult_) String() string { if p == nil { return "" } - return fmt.Sprintf("NodePersistRateLimitResult_(%+v)", *p) + return fmt.Sprintf("TruncateResult_(%+v)", *p) } // Attributes: -// - LimitEnabled -// - LimitMbps -// - LimitCheckEvery -type NodeSetPersistRateLimitRequest struct { - LimitEnabled *bool `thrift:"limitEnabled,1" db:"limitEnabled" json:"limitEnabled,omitempty"` - LimitMbps *float64 `thrift:"limitMbps,2" db:"limitMbps" json:"limitMbps,omitempty"` - LimitCheckEvery *int64 `thrift:"limitCheckEvery,3" db:"limitCheckEvery" json:"limitCheckEvery,omitempty"` -} - -func NewNodeSetPersistRateLimitRequest() *NodeSetPersistRateLimitRequest { - return &NodeSetPersistRateLimitRequest{} -} - -var NodeSetPersistRateLimitRequest_LimitEnabled_DEFAULT bool - -func (p *NodeSetPersistRateLimitRequest) GetLimitEnabled() bool { - if !p.IsSetLimitEnabled() { - return NodeSetPersistRateLimitRequest_LimitEnabled_DEFAULT - } - return *p.LimitEnabled +// - Ok +// - Status +// - Bootstrapped +type NodeHealthResult_ struct { + Ok bool `thrift:"ok,1,required" db:"ok" json:"ok"` + Status string `thrift:"status,2,required" db:"status" json:"status"` + Bootstrapped bool `thrift:"bootstrapped,3,required" db:"bootstrapped" json:"bootstrapped"` } -var NodeSetPersistRateLimitRequest_LimitMbps_DEFAULT float64 - -func (p *NodeSetPersistRateLimitRequest) GetLimitMbps() float64 { - if !p.IsSetLimitMbps() { - return NodeSetPersistRateLimitRequest_LimitMbps_DEFAULT - } - return *p.LimitMbps +func NewNodeHealthResult_() *NodeHealthResult_ { + return &NodeHealthResult_{} } -var NodeSetPersistRateLimitRequest_LimitCheckEvery_DEFAULT int64 - -func (p *NodeSetPersistRateLimitRequest) GetLimitCheckEvery() int64 { - if !p.IsSetLimitCheckEvery() { - return NodeSetPersistRateLimitRequest_LimitCheckEvery_DEFAULT - } - return *p.LimitCheckEvery -} -func (p *NodeSetPersistRateLimitRequest) IsSetLimitEnabled() bool { - return p.LimitEnabled != nil +func (p *NodeHealthResult_) GetOk() bool { + return p.Ok } -func (p *NodeSetPersistRateLimitRequest) IsSetLimitMbps() bool { - return p.LimitMbps != nil +func (p *NodeHealthResult_) GetStatus() string { + return p.Status } -func (p *NodeSetPersistRateLimitRequest) IsSetLimitCheckEvery() bool { - return p.LimitCheckEvery != nil +func (p *NodeHealthResult_) GetBootstrapped() bool { + return p.Bootstrapped } - -func (p *NodeSetPersistRateLimitRequest) Read(iprot thrift.TProtocol) error { +func (p *NodeHealthResult_) Read(iprot thrift.TProtocol) error { if _, err := iprot.ReadStructBegin(); err != nil { return thrift.PrependError(fmt.Sprintf("%T read error: ", p), err) } + var issetOk bool = false + var issetStatus bool = false + var issetBootstrapped bool = false + for { _, fieldTypeId, fieldId, err := iprot.ReadFieldBegin() if err != nil { @@ -6911,14 +7196,17 @@ func (p *NodeSetPersistRateLimitRequest) Read(iprot thrift.TProtocol) error { if err := p.ReadField1(iprot); err != nil { return err } + issetOk = true case 2: if err := p.ReadField2(iprot); err != nil { return err } + issetStatus = true case 3: if err := p.ReadField3(iprot); err != nil { return err } + issetBootstrapped = true default: if err := iprot.Skip(fieldTypeId); err != nil { return err @@ -6931,38 +7219,47 @@ func (p *NodeSetPersistRateLimitRequest) Read(iprot thrift.TProtocol) error { if err := iprot.ReadStructEnd(); err != nil { return thrift.PrependError(fmt.Sprintf("%T read struct end error: ", p), err) } + if !issetOk { + return thrift.NewTProtocolExceptionWithType(thrift.INVALID_DATA, fmt.Errorf("Required field Ok is not set")) + } + if !issetStatus { + return thrift.NewTProtocolExceptionWithType(thrift.INVALID_DATA, fmt.Errorf("Required field Status is not set")) + } + if !issetBootstrapped { + return thrift.NewTProtocolExceptionWithType(thrift.INVALID_DATA, fmt.Errorf("Required field Bootstrapped is not set")) + } return nil } -func (p *NodeSetPersistRateLimitRequest) ReadField1(iprot thrift.TProtocol) error { +func (p *NodeHealthResult_) ReadField1(iprot thrift.TProtocol) error { if v, err := iprot.ReadBool(); err != nil { return thrift.PrependError("error reading field 1: ", err) } else { - p.LimitEnabled = &v + p.Ok = v } return nil } -func (p *NodeSetPersistRateLimitRequest) ReadField2(iprot thrift.TProtocol) error { - if v, err := iprot.ReadDouble(); err != nil { +func (p *NodeHealthResult_) ReadField2(iprot thrift.TProtocol) error { + if v, err := iprot.ReadString(); err != nil { return thrift.PrependError("error reading field 2: ", err) } else { - p.LimitMbps = &v + p.Status = v } return nil } -func (p *NodeSetPersistRateLimitRequest) ReadField3(iprot thrift.TProtocol) error { - if v, err := iprot.ReadI64(); err != nil { +func (p *NodeHealthResult_) ReadField3(iprot thrift.TProtocol) error { + if v, err := iprot.ReadBool(); err != nil { return thrift.PrependError("error reading field 3: ", err) } else { - p.LimitCheckEvery = &v + p.Bootstrapped = v } return nil } -func (p *NodeSetPersistRateLimitRequest) Write(oprot thrift.TProtocol) error { - if err := oprot.WriteStructBegin("NodeSetPersistRateLimitRequest"); err != nil { +func (p *NodeHealthResult_) Write(oprot thrift.TProtocol) error { + if err := oprot.WriteStructBegin("NodeHealthResult"); err != nil { return thrift.PrependError(fmt.Sprintf("%T write struct begin error: ", p), err) } if p != nil { @@ -6985,78 +7282,64 @@ func (p *NodeSetPersistRateLimitRequest) Write(oprot thrift.TProtocol) error { return nil } -func (p *NodeSetPersistRateLimitRequest) writeField1(oprot thrift.TProtocol) (err error) { - if p.IsSetLimitEnabled() { - if err := oprot.WriteFieldBegin("limitEnabled", thrift.BOOL, 1); err != nil { - return thrift.PrependError(fmt.Sprintf("%T write field begin error 1:limitEnabled: ", p), err) - } - if err := oprot.WriteBool(bool(*p.LimitEnabled)); err != nil { - return thrift.PrependError(fmt.Sprintf("%T.limitEnabled (1) field write error: ", p), err) - } - if err := oprot.WriteFieldEnd(); err != nil { - return thrift.PrependError(fmt.Sprintf("%T write field end error 1:limitEnabled: ", p), err) - } +func (p *NodeHealthResult_) writeField1(oprot thrift.TProtocol) (err error) { + if err := oprot.WriteFieldBegin("ok", thrift.BOOL, 1); err != nil { + return thrift.PrependError(fmt.Sprintf("%T write field begin error 1:ok: ", p), err) + } + if err := oprot.WriteBool(bool(p.Ok)); err != nil { + return thrift.PrependError(fmt.Sprintf("%T.ok (1) field write error: ", p), err) + } + if err := oprot.WriteFieldEnd(); err != nil { + return thrift.PrependError(fmt.Sprintf("%T write field end error 1:ok: ", p), err) } return err } -func (p *NodeSetPersistRateLimitRequest) writeField2(oprot thrift.TProtocol) (err error) { - if p.IsSetLimitMbps() { - if err := oprot.WriteFieldBegin("limitMbps", thrift.DOUBLE, 2); err != nil { - return thrift.PrependError(fmt.Sprintf("%T write field begin error 2:limitMbps: ", p), err) - } - if err := oprot.WriteDouble(float64(*p.LimitMbps)); err != nil { - return thrift.PrependError(fmt.Sprintf("%T.limitMbps (2) field write error: ", p), err) - } - if err := oprot.WriteFieldEnd(); err != nil { - return thrift.PrependError(fmt.Sprintf("%T write field end error 2:limitMbps: ", p), err) - } +func (p *NodeHealthResult_) writeField2(oprot thrift.TProtocol) (err error) { + if err := oprot.WriteFieldBegin("status", thrift.STRING, 2); err != nil { + return thrift.PrependError(fmt.Sprintf("%T write field begin error 2:status: ", p), err) + } + if err := oprot.WriteString(string(p.Status)); err != nil { + return thrift.PrependError(fmt.Sprintf("%T.status (2) field write error: ", p), err) + } + if err := oprot.WriteFieldEnd(); err != nil { + return thrift.PrependError(fmt.Sprintf("%T write field end error 2:status: ", p), err) } return err } -func (p *NodeSetPersistRateLimitRequest) writeField3(oprot thrift.TProtocol) (err error) { - if p.IsSetLimitCheckEvery() { - if err := oprot.WriteFieldBegin("limitCheckEvery", thrift.I64, 3); err != nil { - return thrift.PrependError(fmt.Sprintf("%T write field begin error 3:limitCheckEvery: ", p), err) - } - if err := oprot.WriteI64(int64(*p.LimitCheckEvery)); err != nil { - return thrift.PrependError(fmt.Sprintf("%T.limitCheckEvery (3) field write error: ", p), err) - } - if err := oprot.WriteFieldEnd(); err != nil { - return thrift.PrependError(fmt.Sprintf("%T write field end error 3:limitCheckEvery: ", p), err) - } +func (p *NodeHealthResult_) writeField3(oprot thrift.TProtocol) (err error) { + if err := oprot.WriteFieldBegin("bootstrapped", thrift.BOOL, 3); err != nil { + return thrift.PrependError(fmt.Sprintf("%T write field begin error 3:bootstrapped: ", p), err) + } + if err := oprot.WriteBool(bool(p.Bootstrapped)); err != nil { + return thrift.PrependError(fmt.Sprintf("%T.bootstrapped (3) field write error: ", p), err) + } + if err := oprot.WriteFieldEnd(); err != nil { + return thrift.PrependError(fmt.Sprintf("%T write field end error 3:bootstrapped: ", p), err) } return err } -func (p *NodeSetPersistRateLimitRequest) String() string { +func (p *NodeHealthResult_) String() string { if p == nil { return "" } - return fmt.Sprintf("NodeSetPersistRateLimitRequest(%+v)", *p) + return fmt.Sprintf("NodeHealthResult_(%+v)", *p) } -// Attributes: -// - WriteNewSeriesAsync -type NodeWriteNewSeriesAsyncResult_ struct { - WriteNewSeriesAsync bool `thrift:"writeNewSeriesAsync,1,required" db:"writeNewSeriesAsync" json:"writeNewSeriesAsync"` +type NodeBootstrappedResult_ struct { } -func NewNodeWriteNewSeriesAsyncResult_() *NodeWriteNewSeriesAsyncResult_ { - return &NodeWriteNewSeriesAsyncResult_{} +func NewNodeBootstrappedResult_() *NodeBootstrappedResult_ { + return &NodeBootstrappedResult_{} } -func (p *NodeWriteNewSeriesAsyncResult_) GetWriteNewSeriesAsync() bool { - return p.WriteNewSeriesAsync -} -func (p *NodeWriteNewSeriesAsyncResult_) Read(iprot thrift.TProtocol) error { +func (p *NodeBootstrappedResult_) Read(iprot thrift.TProtocol) error { if _, err := iprot.ReadStructBegin(); err != nil { return thrift.PrependError(fmt.Sprintf("%T read error: ", p), err) } - var issetWriteNewSeriesAsync bool = false - for { _, fieldTypeId, fieldId, err := iprot.ReadFieldBegin() if err != nil { @@ -7065,16 +7348,8 @@ func (p *NodeWriteNewSeriesAsyncResult_) Read(iprot thrift.TProtocol) error { if fieldTypeId == thrift.STOP { break } - switch fieldId { - case 1: - if err := p.ReadField1(iprot); err != nil { - return err - } - issetWriteNewSeriesAsync = true - default: - if err := iprot.Skip(fieldTypeId); err != nil { - return err - } + if err := iprot.Skip(fieldTypeId); err != nil { + return err } if err := iprot.ReadFieldEnd(); err != nil { return err @@ -7083,29 +7358,14 @@ func (p *NodeWriteNewSeriesAsyncResult_) Read(iprot thrift.TProtocol) error { if err := iprot.ReadStructEnd(); err != nil { return thrift.PrependError(fmt.Sprintf("%T read struct end error: ", p), err) } - if !issetWriteNewSeriesAsync { - return thrift.NewTProtocolExceptionWithType(thrift.INVALID_DATA, fmt.Errorf("Required field WriteNewSeriesAsync is not set")) - } - return nil -} - -func (p *NodeWriteNewSeriesAsyncResult_) ReadField1(iprot thrift.TProtocol) error { - if v, err := iprot.ReadBool(); err != nil { - return thrift.PrependError("error reading field 1: ", err) - } else { - p.WriteNewSeriesAsync = v - } return nil } -func (p *NodeWriteNewSeriesAsyncResult_) Write(oprot thrift.TProtocol) error { - if err := oprot.WriteStructBegin("NodeWriteNewSeriesAsyncResult"); err != nil { +func (p *NodeBootstrappedResult_) Write(oprot thrift.TProtocol) error { + if err := oprot.WriteStructBegin("NodeBootstrappedResult"); err != nil { return thrift.PrependError(fmt.Sprintf("%T write struct begin error: ", p), err) } if p != nil { - if err := p.writeField1(oprot); err != nil { - return err - } } if err := oprot.WriteFieldStop(); err != nil { return thrift.PrependError("write field stop error: ", err) @@ -7116,46 +7376,25 @@ func (p *NodeWriteNewSeriesAsyncResult_) Write(oprot thrift.TProtocol) error { return nil } -func (p *NodeWriteNewSeriesAsyncResult_) writeField1(oprot thrift.TProtocol) (err error) { - if err := oprot.WriteFieldBegin("writeNewSeriesAsync", thrift.BOOL, 1); err != nil { - return thrift.PrependError(fmt.Sprintf("%T write field begin error 1:writeNewSeriesAsync: ", p), err) - } - if err := oprot.WriteBool(bool(p.WriteNewSeriesAsync)); err != nil { - return thrift.PrependError(fmt.Sprintf("%T.writeNewSeriesAsync (1) field write error: ", p), err) - } - if err := oprot.WriteFieldEnd(); err != nil { - return thrift.PrependError(fmt.Sprintf("%T write field end error 1:writeNewSeriesAsync: ", p), err) - } - return err -} - -func (p *NodeWriteNewSeriesAsyncResult_) String() string { +func (p *NodeBootstrappedResult_) String() string { if p == nil { return "" } - return fmt.Sprintf("NodeWriteNewSeriesAsyncResult_(%+v)", *p) + return fmt.Sprintf("NodeBootstrappedResult_(%+v)", *p) } -// Attributes: -// - WriteNewSeriesAsync -type NodeSetWriteNewSeriesAsyncRequest struct { - WriteNewSeriesAsync bool `thrift:"writeNewSeriesAsync,1,required" db:"writeNewSeriesAsync" json:"writeNewSeriesAsync"` +type NodeBootstrappedInPlacementOrNoPlacementResult_ struct { } -func NewNodeSetWriteNewSeriesAsyncRequest() *NodeSetWriteNewSeriesAsyncRequest { - return &NodeSetWriteNewSeriesAsyncRequest{} +func NewNodeBootstrappedInPlacementOrNoPlacementResult_() *NodeBootstrappedInPlacementOrNoPlacementResult_ { + return &NodeBootstrappedInPlacementOrNoPlacementResult_{} } -func (p *NodeSetWriteNewSeriesAsyncRequest) GetWriteNewSeriesAsync() bool { - return p.WriteNewSeriesAsync -} -func (p *NodeSetWriteNewSeriesAsyncRequest) Read(iprot thrift.TProtocol) error { +func (p *NodeBootstrappedInPlacementOrNoPlacementResult_) Read(iprot thrift.TProtocol) error { if _, err := iprot.ReadStructBegin(); err != nil { return thrift.PrependError(fmt.Sprintf("%T read error: ", p), err) } - var issetWriteNewSeriesAsync bool = false - for { _, fieldTypeId, fieldId, err := iprot.ReadFieldBegin() if err != nil { @@ -7164,16 +7403,8 @@ func (p *NodeSetWriteNewSeriesAsyncRequest) Read(iprot thrift.TProtocol) error { if fieldTypeId == thrift.STOP { break } - switch fieldId { - case 1: - if err := p.ReadField1(iprot); err != nil { - return err - } - issetWriteNewSeriesAsync = true - default: - if err := iprot.Skip(fieldTypeId); err != nil { - return err - } + if err := iprot.Skip(fieldTypeId); err != nil { + return err } if err := iprot.ReadFieldEnd(); err != nil { return err @@ -7182,29 +7413,14 @@ func (p *NodeSetWriteNewSeriesAsyncRequest) Read(iprot thrift.TProtocol) error { if err := iprot.ReadStructEnd(); err != nil { return thrift.PrependError(fmt.Sprintf("%T read struct end error: ", p), err) } - if !issetWriteNewSeriesAsync { - return thrift.NewTProtocolExceptionWithType(thrift.INVALID_DATA, fmt.Errorf("Required field WriteNewSeriesAsync is not set")) - } - return nil -} - -func (p *NodeSetWriteNewSeriesAsyncRequest) ReadField1(iprot thrift.TProtocol) error { - if v, err := iprot.ReadBool(); err != nil { - return thrift.PrependError("error reading field 1: ", err) - } else { - p.WriteNewSeriesAsync = v - } return nil } -func (p *NodeSetWriteNewSeriesAsyncRequest) Write(oprot thrift.TProtocol) error { - if err := oprot.WriteStructBegin("NodeSetWriteNewSeriesAsyncRequest"); err != nil { +func (p *NodeBootstrappedInPlacementOrNoPlacementResult_) Write(oprot thrift.TProtocol) error { + if err := oprot.WriteStructBegin("NodeBootstrappedInPlacementOrNoPlacementResult"); err != nil { return thrift.PrependError(fmt.Sprintf("%T write struct begin error: ", p), err) } if p != nil { - if err := p.writeField1(oprot); err != nil { - return err - } } if err := oprot.WriteFieldStop(); err != nil { return thrift.PrependError("write field stop error: ", err) @@ -7215,52 +7431,46 @@ func (p *NodeSetWriteNewSeriesAsyncRequest) Write(oprot thrift.TProtocol) error return nil } -func (p *NodeSetWriteNewSeriesAsyncRequest) writeField1(oprot thrift.TProtocol) (err error) { - if err := oprot.WriteFieldBegin("writeNewSeriesAsync", thrift.BOOL, 1); err != nil { - return thrift.PrependError(fmt.Sprintf("%T write field begin error 1:writeNewSeriesAsync: ", p), err) - } - if err := oprot.WriteBool(bool(p.WriteNewSeriesAsync)); err != nil { - return thrift.PrependError(fmt.Sprintf("%T.writeNewSeriesAsync (1) field write error: ", p), err) - } - if err := oprot.WriteFieldEnd(); err != nil { - return thrift.PrependError(fmt.Sprintf("%T write field end error 1:writeNewSeriesAsync: ", p), err) - } - return err -} - -func (p *NodeSetWriteNewSeriesAsyncRequest) String() string { +func (p *NodeBootstrappedInPlacementOrNoPlacementResult_) String() string { if p == nil { return "" } - return fmt.Sprintf("NodeSetWriteNewSeriesAsyncRequest(%+v)", *p) + return fmt.Sprintf("NodeBootstrappedInPlacementOrNoPlacementResult_(%+v)", *p) } // Attributes: -// - WriteNewSeriesBackoffDuration -// - DurationType -type NodeWriteNewSeriesBackoffDurationResult_ struct { - WriteNewSeriesBackoffDuration int64 `thrift:"writeNewSeriesBackoffDuration,1,required" db:"writeNewSeriesBackoffDuration" json:"writeNewSeriesBackoffDuration"` - DurationType TimeType `thrift:"durationType,2,required" db:"durationType" json:"durationType"` +// - LimitEnabled +// - LimitMbps +// - LimitCheckEvery +type NodePersistRateLimitResult_ struct { + LimitEnabled bool `thrift:"limitEnabled,1,required" db:"limitEnabled" json:"limitEnabled"` + LimitMbps float64 `thrift:"limitMbps,2,required" db:"limitMbps" json:"limitMbps"` + LimitCheckEvery int64 `thrift:"limitCheckEvery,3,required" db:"limitCheckEvery" json:"limitCheckEvery"` } -func NewNodeWriteNewSeriesBackoffDurationResult_() *NodeWriteNewSeriesBackoffDurationResult_ { - return &NodeWriteNewSeriesBackoffDurationResult_{} +func NewNodePersistRateLimitResult_() *NodePersistRateLimitResult_ { + return &NodePersistRateLimitResult_{} } -func (p *NodeWriteNewSeriesBackoffDurationResult_) GetWriteNewSeriesBackoffDuration() int64 { - return p.WriteNewSeriesBackoffDuration +func (p *NodePersistRateLimitResult_) GetLimitEnabled() bool { + return p.LimitEnabled } -func (p *NodeWriteNewSeriesBackoffDurationResult_) GetDurationType() TimeType { - return p.DurationType +func (p *NodePersistRateLimitResult_) GetLimitMbps() float64 { + return p.LimitMbps } -func (p *NodeWriteNewSeriesBackoffDurationResult_) Read(iprot thrift.TProtocol) error { + +func (p *NodePersistRateLimitResult_) GetLimitCheckEvery() int64 { + return p.LimitCheckEvery +} +func (p *NodePersistRateLimitResult_) Read(iprot thrift.TProtocol) error { if _, err := iprot.ReadStructBegin(); err != nil { return thrift.PrependError(fmt.Sprintf("%T read error: ", p), err) } - var issetWriteNewSeriesBackoffDuration bool = false - var issetDurationType bool = false + var issetLimitEnabled bool = false + var issetLimitMbps bool = false + var issetLimitCheckEvery bool = false for { _, fieldTypeId, fieldId, err := iprot.ReadFieldBegin() @@ -7275,12 +7485,17 @@ func (p *NodeWriteNewSeriesBackoffDurationResult_) Read(iprot thrift.TProtocol) if err := p.ReadField1(iprot); err != nil { return err } - issetWriteNewSeriesBackoffDuration = true + issetLimitEnabled = true case 2: if err := p.ReadField2(iprot); err != nil { return err } - issetDurationType = true + issetLimitMbps = true + case 3: + if err := p.ReadField3(iprot); err != nil { + return err + } + issetLimitCheckEvery = true default: if err := iprot.Skip(fieldTypeId); err != nil { return err @@ -7293,36 +7508,47 @@ func (p *NodeWriteNewSeriesBackoffDurationResult_) Read(iprot thrift.TProtocol) if err := iprot.ReadStructEnd(); err != nil { return thrift.PrependError(fmt.Sprintf("%T read struct end error: ", p), err) } - if !issetWriteNewSeriesBackoffDuration { - return thrift.NewTProtocolExceptionWithType(thrift.INVALID_DATA, fmt.Errorf("Required field WriteNewSeriesBackoffDuration is not set")) + if !issetLimitEnabled { + return thrift.NewTProtocolExceptionWithType(thrift.INVALID_DATA, fmt.Errorf("Required field LimitEnabled is not set")) } - if !issetDurationType { - return thrift.NewTProtocolExceptionWithType(thrift.INVALID_DATA, fmt.Errorf("Required field DurationType is not set")) + if !issetLimitMbps { + return thrift.NewTProtocolExceptionWithType(thrift.INVALID_DATA, fmt.Errorf("Required field LimitMbps is not set")) + } + if !issetLimitCheckEvery { + return thrift.NewTProtocolExceptionWithType(thrift.INVALID_DATA, fmt.Errorf("Required field LimitCheckEvery is not set")) } return nil } -func (p *NodeWriteNewSeriesBackoffDurationResult_) ReadField1(iprot thrift.TProtocol) error { - if v, err := iprot.ReadI64(); err != nil { +func (p *NodePersistRateLimitResult_) ReadField1(iprot thrift.TProtocol) error { + if v, err := iprot.ReadBool(); err != nil { return thrift.PrependError("error reading field 1: ", err) } else { - p.WriteNewSeriesBackoffDuration = v + p.LimitEnabled = v } return nil } -func (p *NodeWriteNewSeriesBackoffDurationResult_) ReadField2(iprot thrift.TProtocol) error { - if v, err := iprot.ReadI32(); err != nil { +func (p *NodePersistRateLimitResult_) ReadField2(iprot thrift.TProtocol) error { + if v, err := iprot.ReadDouble(); err != nil { return thrift.PrependError("error reading field 2: ", err) } else { - temp := TimeType(v) - p.DurationType = temp + p.LimitMbps = v } return nil } -func (p *NodeWriteNewSeriesBackoffDurationResult_) Write(oprot thrift.TProtocol) error { - if err := oprot.WriteStructBegin("NodeWriteNewSeriesBackoffDurationResult"); err != nil { +func (p *NodePersistRateLimitResult_) ReadField3(iprot thrift.TProtocol) error { + if v, err := iprot.ReadI64(); err != nil { + return thrift.PrependError("error reading field 3: ", err) + } else { + p.LimitCheckEvery = v + } + return nil +} + +func (p *NodePersistRateLimitResult_) Write(oprot thrift.TProtocol) error { + if err := oprot.WriteStructBegin("NodePersistRateLimitResult"); err != nil { return thrift.PrependError(fmt.Sprintf("%T write struct begin error: ", p), err) } if p != nil { @@ -7332,6 +7558,9 @@ func (p *NodeWriteNewSeriesBackoffDurationResult_) Write(oprot thrift.TProtocol) if err := p.writeField2(oprot); err != nil { return err } + if err := p.writeField3(oprot); err != nil { + return err + } } if err := oprot.WriteFieldStop(); err != nil { return thrift.PrependError("write field stop error: ", err) @@ -7342,73 +7571,109 @@ func (p *NodeWriteNewSeriesBackoffDurationResult_) Write(oprot thrift.TProtocol) return nil } -func (p *NodeWriteNewSeriesBackoffDurationResult_) writeField1(oprot thrift.TProtocol) (err error) { - if err := oprot.WriteFieldBegin("writeNewSeriesBackoffDuration", thrift.I64, 1); err != nil { - return thrift.PrependError(fmt.Sprintf("%T write field begin error 1:writeNewSeriesBackoffDuration: ", p), err) +func (p *NodePersistRateLimitResult_) writeField1(oprot thrift.TProtocol) (err error) { + if err := oprot.WriteFieldBegin("limitEnabled", thrift.BOOL, 1); err != nil { + return thrift.PrependError(fmt.Sprintf("%T write field begin error 1:limitEnabled: ", p), err) } - if err := oprot.WriteI64(int64(p.WriteNewSeriesBackoffDuration)); err != nil { - return thrift.PrependError(fmt.Sprintf("%T.writeNewSeriesBackoffDuration (1) field write error: ", p), err) + if err := oprot.WriteBool(bool(p.LimitEnabled)); err != nil { + return thrift.PrependError(fmt.Sprintf("%T.limitEnabled (1) field write error: ", p), err) } if err := oprot.WriteFieldEnd(); err != nil { - return thrift.PrependError(fmt.Sprintf("%T write field end error 1:writeNewSeriesBackoffDuration: ", p), err) + return thrift.PrependError(fmt.Sprintf("%T write field end error 1:limitEnabled: ", p), err) } return err } -func (p *NodeWriteNewSeriesBackoffDurationResult_) writeField2(oprot thrift.TProtocol) (err error) { - if err := oprot.WriteFieldBegin("durationType", thrift.I32, 2); err != nil { - return thrift.PrependError(fmt.Sprintf("%T write field begin error 2:durationType: ", p), err) +func (p *NodePersistRateLimitResult_) writeField2(oprot thrift.TProtocol) (err error) { + if err := oprot.WriteFieldBegin("limitMbps", thrift.DOUBLE, 2); err != nil { + return thrift.PrependError(fmt.Sprintf("%T write field begin error 2:limitMbps: ", p), err) } - if err := oprot.WriteI32(int32(p.DurationType)); err != nil { - return thrift.PrependError(fmt.Sprintf("%T.durationType (2) field write error: ", p), err) + if err := oprot.WriteDouble(float64(p.LimitMbps)); err != nil { + return thrift.PrependError(fmt.Sprintf("%T.limitMbps (2) field write error: ", p), err) } if err := oprot.WriteFieldEnd(); err != nil { - return thrift.PrependError(fmt.Sprintf("%T write field end error 2:durationType: ", p), err) + return thrift.PrependError(fmt.Sprintf("%T write field end error 2:limitMbps: ", p), err) } return err } -func (p *NodeWriteNewSeriesBackoffDurationResult_) String() string { +func (p *NodePersistRateLimitResult_) writeField3(oprot thrift.TProtocol) (err error) { + if err := oprot.WriteFieldBegin("limitCheckEvery", thrift.I64, 3); err != nil { + return thrift.PrependError(fmt.Sprintf("%T write field begin error 3:limitCheckEvery: ", p), err) + } + if err := oprot.WriteI64(int64(p.LimitCheckEvery)); err != nil { + return thrift.PrependError(fmt.Sprintf("%T.limitCheckEvery (3) field write error: ", p), err) + } + if err := oprot.WriteFieldEnd(); err != nil { + return thrift.PrependError(fmt.Sprintf("%T write field end error 3:limitCheckEvery: ", p), err) + } + return err +} + +func (p *NodePersistRateLimitResult_) String() string { if p == nil { return "" } - return fmt.Sprintf("NodeWriteNewSeriesBackoffDurationResult_(%+v)", *p) + return fmt.Sprintf("NodePersistRateLimitResult_(%+v)", *p) } // Attributes: -// - WriteNewSeriesBackoffDuration -// - DurationType -type NodeSetWriteNewSeriesBackoffDurationRequest struct { - WriteNewSeriesBackoffDuration int64 `thrift:"writeNewSeriesBackoffDuration,1,required" db:"writeNewSeriesBackoffDuration" json:"writeNewSeriesBackoffDuration"` - DurationType TimeType `thrift:"durationType,2" db:"durationType" json:"durationType,omitempty"` +// - LimitEnabled +// - LimitMbps +// - LimitCheckEvery +type NodeSetPersistRateLimitRequest struct { + LimitEnabled *bool `thrift:"limitEnabled,1" db:"limitEnabled" json:"limitEnabled,omitempty"` + LimitMbps *float64 `thrift:"limitMbps,2" db:"limitMbps" json:"limitMbps,omitempty"` + LimitCheckEvery *int64 `thrift:"limitCheckEvery,3" db:"limitCheckEvery" json:"limitCheckEvery,omitempty"` } -func NewNodeSetWriteNewSeriesBackoffDurationRequest() *NodeSetWriteNewSeriesBackoffDurationRequest { - return &NodeSetWriteNewSeriesBackoffDurationRequest{ - DurationType: 2, - } +func NewNodeSetPersistRateLimitRequest() *NodeSetPersistRateLimitRequest { + return &NodeSetPersistRateLimitRequest{} } -func (p *NodeSetWriteNewSeriesBackoffDurationRequest) GetWriteNewSeriesBackoffDuration() int64 { - return p.WriteNewSeriesBackoffDuration -} +var NodeSetPersistRateLimitRequest_LimitEnabled_DEFAULT bool -var NodeSetWriteNewSeriesBackoffDurationRequest_DurationType_DEFAULT TimeType = 2 +func (p *NodeSetPersistRateLimitRequest) GetLimitEnabled() bool { + if !p.IsSetLimitEnabled() { + return NodeSetPersistRateLimitRequest_LimitEnabled_DEFAULT + } + return *p.LimitEnabled +} -func (p *NodeSetWriteNewSeriesBackoffDurationRequest) GetDurationType() TimeType { - return p.DurationType +var NodeSetPersistRateLimitRequest_LimitMbps_DEFAULT float64 + +func (p *NodeSetPersistRateLimitRequest) GetLimitMbps() float64 { + if !p.IsSetLimitMbps() { + return NodeSetPersistRateLimitRequest_LimitMbps_DEFAULT + } + return *p.LimitMbps } -func (p *NodeSetWriteNewSeriesBackoffDurationRequest) IsSetDurationType() bool { - return p.DurationType != NodeSetWriteNewSeriesBackoffDurationRequest_DurationType_DEFAULT + +var NodeSetPersistRateLimitRequest_LimitCheckEvery_DEFAULT int64 + +func (p *NodeSetPersistRateLimitRequest) GetLimitCheckEvery() int64 { + if !p.IsSetLimitCheckEvery() { + return NodeSetPersistRateLimitRequest_LimitCheckEvery_DEFAULT + } + return *p.LimitCheckEvery +} +func (p *NodeSetPersistRateLimitRequest) IsSetLimitEnabled() bool { + return p.LimitEnabled != nil } -func (p *NodeSetWriteNewSeriesBackoffDurationRequest) Read(iprot thrift.TProtocol) error { +func (p *NodeSetPersistRateLimitRequest) IsSetLimitMbps() bool { + return p.LimitMbps != nil +} + +func (p *NodeSetPersistRateLimitRequest) IsSetLimitCheckEvery() bool { + return p.LimitCheckEvery != nil +} + +func (p *NodeSetPersistRateLimitRequest) Read(iprot thrift.TProtocol) error { if _, err := iprot.ReadStructBegin(); err != nil { return thrift.PrependError(fmt.Sprintf("%T read error: ", p), err) } - var issetWriteNewSeriesBackoffDuration bool = false - for { _, fieldTypeId, fieldId, err := iprot.ReadFieldBegin() if err != nil { @@ -7422,11 +7687,14 @@ func (p *NodeSetWriteNewSeriesBackoffDurationRequest) Read(iprot thrift.TProtoco if err := p.ReadField1(iprot); err != nil { return err } - issetWriteNewSeriesBackoffDuration = true case 2: if err := p.ReadField2(iprot); err != nil { return err } + case 3: + if err := p.ReadField3(iprot); err != nil { + return err + } default: if err := iprot.Skip(fieldTypeId); err != nil { return err @@ -7439,33 +7707,38 @@ func (p *NodeSetWriteNewSeriesBackoffDurationRequest) Read(iprot thrift.TProtoco if err := iprot.ReadStructEnd(); err != nil { return thrift.PrependError(fmt.Sprintf("%T read struct end error: ", p), err) } - if !issetWriteNewSeriesBackoffDuration { - return thrift.NewTProtocolExceptionWithType(thrift.INVALID_DATA, fmt.Errorf("Required field WriteNewSeriesBackoffDuration is not set")) - } return nil } -func (p *NodeSetWriteNewSeriesBackoffDurationRequest) ReadField1(iprot thrift.TProtocol) error { - if v, err := iprot.ReadI64(); err != nil { +func (p *NodeSetPersistRateLimitRequest) ReadField1(iprot thrift.TProtocol) error { + if v, err := iprot.ReadBool(); err != nil { return thrift.PrependError("error reading field 1: ", err) } else { - p.WriteNewSeriesBackoffDuration = v + p.LimitEnabled = &v } return nil } -func (p *NodeSetWriteNewSeriesBackoffDurationRequest) ReadField2(iprot thrift.TProtocol) error { - if v, err := iprot.ReadI32(); err != nil { +func (p *NodeSetPersistRateLimitRequest) ReadField2(iprot thrift.TProtocol) error { + if v, err := iprot.ReadDouble(); err != nil { return thrift.PrependError("error reading field 2: ", err) } else { - temp := TimeType(v) - p.DurationType = temp + p.LimitMbps = &v } return nil } -func (p *NodeSetWriteNewSeriesBackoffDurationRequest) Write(oprot thrift.TProtocol) error { - if err := oprot.WriteStructBegin("NodeSetWriteNewSeriesBackoffDurationRequest"); err != nil { +func (p *NodeSetPersistRateLimitRequest) ReadField3(iprot thrift.TProtocol) error { + if v, err := iprot.ReadI64(); err != nil { + return thrift.PrependError("error reading field 3: ", err) + } else { + p.LimitCheckEvery = &v + } + return nil +} + +func (p *NodeSetPersistRateLimitRequest) Write(oprot thrift.TProtocol) error { + if err := oprot.WriteStructBegin("NodeSetPersistRateLimitRequest"); err != nil { return thrift.PrependError(fmt.Sprintf("%T write struct begin error: ", p), err) } if p != nil { @@ -7475,6 +7748,9 @@ func (p *NodeSetWriteNewSeriesBackoffDurationRequest) Write(oprot thrift.TProtoc if err := p.writeField2(oprot); err != nil { return err } + if err := p.writeField3(oprot); err != nil { + return err + } } if err := oprot.WriteFieldStop(); err != nil { return thrift.PrependError("write field stop error: ", err) @@ -7485,60 +7761,77 @@ func (p *NodeSetWriteNewSeriesBackoffDurationRequest) Write(oprot thrift.TProtoc return nil } -func (p *NodeSetWriteNewSeriesBackoffDurationRequest) writeField1(oprot thrift.TProtocol) (err error) { - if err := oprot.WriteFieldBegin("writeNewSeriesBackoffDuration", thrift.I64, 1); err != nil { - return thrift.PrependError(fmt.Sprintf("%T write field begin error 1:writeNewSeriesBackoffDuration: ", p), err) - } - if err := oprot.WriteI64(int64(p.WriteNewSeriesBackoffDuration)); err != nil { - return thrift.PrependError(fmt.Sprintf("%T.writeNewSeriesBackoffDuration (1) field write error: ", p), err) +func (p *NodeSetPersistRateLimitRequest) writeField1(oprot thrift.TProtocol) (err error) { + if p.IsSetLimitEnabled() { + if err := oprot.WriteFieldBegin("limitEnabled", thrift.BOOL, 1); err != nil { + return thrift.PrependError(fmt.Sprintf("%T write field begin error 1:limitEnabled: ", p), err) + } + if err := oprot.WriteBool(bool(*p.LimitEnabled)); err != nil { + return thrift.PrependError(fmt.Sprintf("%T.limitEnabled (1) field write error: ", p), err) + } + if err := oprot.WriteFieldEnd(); err != nil { + return thrift.PrependError(fmt.Sprintf("%T write field end error 1:limitEnabled: ", p), err) + } } - if err := oprot.WriteFieldEnd(); err != nil { - return thrift.PrependError(fmt.Sprintf("%T write field end error 1:writeNewSeriesBackoffDuration: ", p), err) + return err +} + +func (p *NodeSetPersistRateLimitRequest) writeField2(oprot thrift.TProtocol) (err error) { + if p.IsSetLimitMbps() { + if err := oprot.WriteFieldBegin("limitMbps", thrift.DOUBLE, 2); err != nil { + return thrift.PrependError(fmt.Sprintf("%T write field begin error 2:limitMbps: ", p), err) + } + if err := oprot.WriteDouble(float64(*p.LimitMbps)); err != nil { + return thrift.PrependError(fmt.Sprintf("%T.limitMbps (2) field write error: ", p), err) + } + if err := oprot.WriteFieldEnd(); err != nil { + return thrift.PrependError(fmt.Sprintf("%T write field end error 2:limitMbps: ", p), err) + } } return err } -func (p *NodeSetWriteNewSeriesBackoffDurationRequest) writeField2(oprot thrift.TProtocol) (err error) { - if p.IsSetDurationType() { - if err := oprot.WriteFieldBegin("durationType", thrift.I32, 2); err != nil { - return thrift.PrependError(fmt.Sprintf("%T write field begin error 2:durationType: ", p), err) +func (p *NodeSetPersistRateLimitRequest) writeField3(oprot thrift.TProtocol) (err error) { + if p.IsSetLimitCheckEvery() { + if err := oprot.WriteFieldBegin("limitCheckEvery", thrift.I64, 3); err != nil { + return thrift.PrependError(fmt.Sprintf("%T write field begin error 3:limitCheckEvery: ", p), err) } - if err := oprot.WriteI32(int32(p.DurationType)); err != nil { - return thrift.PrependError(fmt.Sprintf("%T.durationType (2) field write error: ", p), err) + if err := oprot.WriteI64(int64(*p.LimitCheckEvery)); err != nil { + return thrift.PrependError(fmt.Sprintf("%T.limitCheckEvery (3) field write error: ", p), err) } if err := oprot.WriteFieldEnd(); err != nil { - return thrift.PrependError(fmt.Sprintf("%T write field end error 2:durationType: ", p), err) + return thrift.PrependError(fmt.Sprintf("%T write field end error 3:limitCheckEvery: ", p), err) } } return err } -func (p *NodeSetWriteNewSeriesBackoffDurationRequest) String() string { +func (p *NodeSetPersistRateLimitRequest) String() string { if p == nil { return "" } - return fmt.Sprintf("NodeSetWriteNewSeriesBackoffDurationRequest(%+v)", *p) + return fmt.Sprintf("NodeSetPersistRateLimitRequest(%+v)", *p) } // Attributes: -// - WriteNewSeriesLimitPerShardPerSecond -type NodeWriteNewSeriesLimitPerShardPerSecondResult_ struct { - WriteNewSeriesLimitPerShardPerSecond int64 `thrift:"writeNewSeriesLimitPerShardPerSecond,1,required" db:"writeNewSeriesLimitPerShardPerSecond" json:"writeNewSeriesLimitPerShardPerSecond"` +// - WriteNewSeriesAsync +type NodeWriteNewSeriesAsyncResult_ struct { + WriteNewSeriesAsync bool `thrift:"writeNewSeriesAsync,1,required" db:"writeNewSeriesAsync" json:"writeNewSeriesAsync"` } -func NewNodeWriteNewSeriesLimitPerShardPerSecondResult_() *NodeWriteNewSeriesLimitPerShardPerSecondResult_ { - return &NodeWriteNewSeriesLimitPerShardPerSecondResult_{} +func NewNodeWriteNewSeriesAsyncResult_() *NodeWriteNewSeriesAsyncResult_ { + return &NodeWriteNewSeriesAsyncResult_{} } -func (p *NodeWriteNewSeriesLimitPerShardPerSecondResult_) GetWriteNewSeriesLimitPerShardPerSecond() int64 { - return p.WriteNewSeriesLimitPerShardPerSecond +func (p *NodeWriteNewSeriesAsyncResult_) GetWriteNewSeriesAsync() bool { + return p.WriteNewSeriesAsync } -func (p *NodeWriteNewSeriesLimitPerShardPerSecondResult_) Read(iprot thrift.TProtocol) error { +func (p *NodeWriteNewSeriesAsyncResult_) Read(iprot thrift.TProtocol) error { if _, err := iprot.ReadStructBegin(); err != nil { return thrift.PrependError(fmt.Sprintf("%T read error: ", p), err) } - var issetWriteNewSeriesLimitPerShardPerSecond bool = false + var issetWriteNewSeriesAsync bool = false for { _, fieldTypeId, fieldId, err := iprot.ReadFieldBegin() @@ -7553,7 +7846,7 @@ func (p *NodeWriteNewSeriesLimitPerShardPerSecondResult_) Read(iprot thrift.TPro if err := p.ReadField1(iprot); err != nil { return err } - issetWriteNewSeriesLimitPerShardPerSecond = true + issetWriteNewSeriesAsync = true default: if err := iprot.Skip(fieldTypeId); err != nil { return err @@ -7566,23 +7859,23 @@ func (p *NodeWriteNewSeriesLimitPerShardPerSecondResult_) Read(iprot thrift.TPro if err := iprot.ReadStructEnd(); err != nil { return thrift.PrependError(fmt.Sprintf("%T read struct end error: ", p), err) } - if !issetWriteNewSeriesLimitPerShardPerSecond { - return thrift.NewTProtocolExceptionWithType(thrift.INVALID_DATA, fmt.Errorf("Required field WriteNewSeriesLimitPerShardPerSecond is not set")) + if !issetWriteNewSeriesAsync { + return thrift.NewTProtocolExceptionWithType(thrift.INVALID_DATA, fmt.Errorf("Required field WriteNewSeriesAsync is not set")) } return nil } -func (p *NodeWriteNewSeriesLimitPerShardPerSecondResult_) ReadField1(iprot thrift.TProtocol) error { - if v, err := iprot.ReadI64(); err != nil { +func (p *NodeWriteNewSeriesAsyncResult_) ReadField1(iprot thrift.TProtocol) error { + if v, err := iprot.ReadBool(); err != nil { return thrift.PrependError("error reading field 1: ", err) } else { - p.WriteNewSeriesLimitPerShardPerSecond = v + p.WriteNewSeriesAsync = v } return nil } -func (p *NodeWriteNewSeriesLimitPerShardPerSecondResult_) Write(oprot thrift.TProtocol) error { - if err := oprot.WriteStructBegin("NodeWriteNewSeriesLimitPerShardPerSecondResult"); err != nil { +func (p *NodeWriteNewSeriesAsyncResult_) Write(oprot thrift.TProtocol) error { + if err := oprot.WriteStructBegin("NodeWriteNewSeriesAsyncResult"); err != nil { return thrift.PrependError(fmt.Sprintf("%T write struct begin error: ", p), err) } if p != nil { @@ -7599,45 +7892,45 @@ func (p *NodeWriteNewSeriesLimitPerShardPerSecondResult_) Write(oprot thrift.TPr return nil } -func (p *NodeWriteNewSeriesLimitPerShardPerSecondResult_) writeField1(oprot thrift.TProtocol) (err error) { - if err := oprot.WriteFieldBegin("writeNewSeriesLimitPerShardPerSecond", thrift.I64, 1); err != nil { - return thrift.PrependError(fmt.Sprintf("%T write field begin error 1:writeNewSeriesLimitPerShardPerSecond: ", p), err) +func (p *NodeWriteNewSeriesAsyncResult_) writeField1(oprot thrift.TProtocol) (err error) { + if err := oprot.WriteFieldBegin("writeNewSeriesAsync", thrift.BOOL, 1); err != nil { + return thrift.PrependError(fmt.Sprintf("%T write field begin error 1:writeNewSeriesAsync: ", p), err) } - if err := oprot.WriteI64(int64(p.WriteNewSeriesLimitPerShardPerSecond)); err != nil { - return thrift.PrependError(fmt.Sprintf("%T.writeNewSeriesLimitPerShardPerSecond (1) field write error: ", p), err) + if err := oprot.WriteBool(bool(p.WriteNewSeriesAsync)); err != nil { + return thrift.PrependError(fmt.Sprintf("%T.writeNewSeriesAsync (1) field write error: ", p), err) } if err := oprot.WriteFieldEnd(); err != nil { - return thrift.PrependError(fmt.Sprintf("%T write field end error 1:writeNewSeriesLimitPerShardPerSecond: ", p), err) + return thrift.PrependError(fmt.Sprintf("%T write field end error 1:writeNewSeriesAsync: ", p), err) } return err } -func (p *NodeWriteNewSeriesLimitPerShardPerSecondResult_) String() string { +func (p *NodeWriteNewSeriesAsyncResult_) String() string { if p == nil { return "" } - return fmt.Sprintf("NodeWriteNewSeriesLimitPerShardPerSecondResult_(%+v)", *p) + return fmt.Sprintf("NodeWriteNewSeriesAsyncResult_(%+v)", *p) } // Attributes: -// - WriteNewSeriesLimitPerShardPerSecond -type NodeSetWriteNewSeriesLimitPerShardPerSecondRequest struct { - WriteNewSeriesLimitPerShardPerSecond int64 `thrift:"writeNewSeriesLimitPerShardPerSecond,1,required" db:"writeNewSeriesLimitPerShardPerSecond" json:"writeNewSeriesLimitPerShardPerSecond"` +// - WriteNewSeriesAsync +type NodeSetWriteNewSeriesAsyncRequest struct { + WriteNewSeriesAsync bool `thrift:"writeNewSeriesAsync,1,required" db:"writeNewSeriesAsync" json:"writeNewSeriesAsync"` } -func NewNodeSetWriteNewSeriesLimitPerShardPerSecondRequest() *NodeSetWriteNewSeriesLimitPerShardPerSecondRequest { - return &NodeSetWriteNewSeriesLimitPerShardPerSecondRequest{} +func NewNodeSetWriteNewSeriesAsyncRequest() *NodeSetWriteNewSeriesAsyncRequest { + return &NodeSetWriteNewSeriesAsyncRequest{} } -func (p *NodeSetWriteNewSeriesLimitPerShardPerSecondRequest) GetWriteNewSeriesLimitPerShardPerSecond() int64 { - return p.WriteNewSeriesLimitPerShardPerSecond +func (p *NodeSetWriteNewSeriesAsyncRequest) GetWriteNewSeriesAsync() bool { + return p.WriteNewSeriesAsync } -func (p *NodeSetWriteNewSeriesLimitPerShardPerSecondRequest) Read(iprot thrift.TProtocol) error { +func (p *NodeSetWriteNewSeriesAsyncRequest) Read(iprot thrift.TProtocol) error { if _, err := iprot.ReadStructBegin(); err != nil { return thrift.PrependError(fmt.Sprintf("%T read error: ", p), err) } - var issetWriteNewSeriesLimitPerShardPerSecond bool = false + var issetWriteNewSeriesAsync bool = false for { _, fieldTypeId, fieldId, err := iprot.ReadFieldBegin() @@ -7652,7 +7945,7 @@ func (p *NodeSetWriteNewSeriesLimitPerShardPerSecondRequest) Read(iprot thrift.T if err := p.ReadField1(iprot); err != nil { return err } - issetWriteNewSeriesLimitPerShardPerSecond = true + issetWriteNewSeriesAsync = true default: if err := iprot.Skip(fieldTypeId); err != nil { return err @@ -7665,23 +7958,23 @@ func (p *NodeSetWriteNewSeriesLimitPerShardPerSecondRequest) Read(iprot thrift.T if err := iprot.ReadStructEnd(); err != nil { return thrift.PrependError(fmt.Sprintf("%T read struct end error: ", p), err) } - if !issetWriteNewSeriesLimitPerShardPerSecond { - return thrift.NewTProtocolExceptionWithType(thrift.INVALID_DATA, fmt.Errorf("Required field WriteNewSeriesLimitPerShardPerSecond is not set")) + if !issetWriteNewSeriesAsync { + return thrift.NewTProtocolExceptionWithType(thrift.INVALID_DATA, fmt.Errorf("Required field WriteNewSeriesAsync is not set")) } return nil } -func (p *NodeSetWriteNewSeriesLimitPerShardPerSecondRequest) ReadField1(iprot thrift.TProtocol) error { - if v, err := iprot.ReadI64(); err != nil { +func (p *NodeSetWriteNewSeriesAsyncRequest) ReadField1(iprot thrift.TProtocol) error { + if v, err := iprot.ReadBool(); err != nil { return thrift.PrependError("error reading field 1: ", err) } else { - p.WriteNewSeriesLimitPerShardPerSecond = v + p.WriteNewSeriesAsync = v } return nil } -func (p *NodeSetWriteNewSeriesLimitPerShardPerSecondRequest) Write(oprot thrift.TProtocol) error { - if err := oprot.WriteStructBegin("NodeSetWriteNewSeriesLimitPerShardPerSecondRequest"); err != nil { +func (p *NodeSetWriteNewSeriesAsyncRequest) Write(oprot thrift.TProtocol) error { + if err := oprot.WriteStructBegin("NodeSetWriteNewSeriesAsyncRequest"); err != nil { return thrift.PrependError(fmt.Sprintf("%T write struct begin error: ", p), err) } if p != nil { @@ -7698,52 +7991,52 @@ func (p *NodeSetWriteNewSeriesLimitPerShardPerSecondRequest) Write(oprot thrift. return nil } -func (p *NodeSetWriteNewSeriesLimitPerShardPerSecondRequest) writeField1(oprot thrift.TProtocol) (err error) { - if err := oprot.WriteFieldBegin("writeNewSeriesLimitPerShardPerSecond", thrift.I64, 1); err != nil { - return thrift.PrependError(fmt.Sprintf("%T write field begin error 1:writeNewSeriesLimitPerShardPerSecond: ", p), err) +func (p *NodeSetWriteNewSeriesAsyncRequest) writeField1(oprot thrift.TProtocol) (err error) { + if err := oprot.WriteFieldBegin("writeNewSeriesAsync", thrift.BOOL, 1); err != nil { + return thrift.PrependError(fmt.Sprintf("%T write field begin error 1:writeNewSeriesAsync: ", p), err) } - if err := oprot.WriteI64(int64(p.WriteNewSeriesLimitPerShardPerSecond)); err != nil { - return thrift.PrependError(fmt.Sprintf("%T.writeNewSeriesLimitPerShardPerSecond (1) field write error: ", p), err) + if err := oprot.WriteBool(bool(p.WriteNewSeriesAsync)); err != nil { + return thrift.PrependError(fmt.Sprintf("%T.writeNewSeriesAsync (1) field write error: ", p), err) } if err := oprot.WriteFieldEnd(); err != nil { - return thrift.PrependError(fmt.Sprintf("%T write field end error 1:writeNewSeriesLimitPerShardPerSecond: ", p), err) + return thrift.PrependError(fmt.Sprintf("%T write field end error 1:writeNewSeriesAsync: ", p), err) } return err } -func (p *NodeSetWriteNewSeriesLimitPerShardPerSecondRequest) String() string { +func (p *NodeSetWriteNewSeriesAsyncRequest) String() string { if p == nil { return "" } - return fmt.Sprintf("NodeSetWriteNewSeriesLimitPerShardPerSecondRequest(%+v)", *p) + return fmt.Sprintf("NodeSetWriteNewSeriesAsyncRequest(%+v)", *p) } // Attributes: -// - Ok -// - Status -type HealthResult_ struct { - Ok bool `thrift:"ok,1,required" db:"ok" json:"ok"` - Status string `thrift:"status,2,required" db:"status" json:"status"` +// - WriteNewSeriesBackoffDuration +// - DurationType +type NodeWriteNewSeriesBackoffDurationResult_ struct { + WriteNewSeriesBackoffDuration int64 `thrift:"writeNewSeriesBackoffDuration,1,required" db:"writeNewSeriesBackoffDuration" json:"writeNewSeriesBackoffDuration"` + DurationType TimeType `thrift:"durationType,2,required" db:"durationType" json:"durationType"` } -func NewHealthResult_() *HealthResult_ { - return &HealthResult_{} +func NewNodeWriteNewSeriesBackoffDurationResult_() *NodeWriteNewSeriesBackoffDurationResult_ { + return &NodeWriteNewSeriesBackoffDurationResult_{} } -func (p *HealthResult_) GetOk() bool { - return p.Ok +func (p *NodeWriteNewSeriesBackoffDurationResult_) GetWriteNewSeriesBackoffDuration() int64 { + return p.WriteNewSeriesBackoffDuration } -func (p *HealthResult_) GetStatus() string { - return p.Status +func (p *NodeWriteNewSeriesBackoffDurationResult_) GetDurationType() TimeType { + return p.DurationType } -func (p *HealthResult_) Read(iprot thrift.TProtocol) error { +func (p *NodeWriteNewSeriesBackoffDurationResult_) Read(iprot thrift.TProtocol) error { if _, err := iprot.ReadStructBegin(); err != nil { return thrift.PrependError(fmt.Sprintf("%T read error: ", p), err) } - var issetOk bool = false - var issetStatus bool = false + var issetWriteNewSeriesBackoffDuration bool = false + var issetDurationType bool = false for { _, fieldTypeId, fieldId, err := iprot.ReadFieldBegin() @@ -7758,12 +8051,12 @@ func (p *HealthResult_) Read(iprot thrift.TProtocol) error { if err := p.ReadField1(iprot); err != nil { return err } - issetOk = true + issetWriteNewSeriesBackoffDuration = true case 2: if err := p.ReadField2(iprot); err != nil { return err } - issetStatus = true + issetDurationType = true default: if err := iprot.Skip(fieldTypeId); err != nil { return err @@ -7776,35 +8069,36 @@ func (p *HealthResult_) Read(iprot thrift.TProtocol) error { if err := iprot.ReadStructEnd(); err != nil { return thrift.PrependError(fmt.Sprintf("%T read struct end error: ", p), err) } - if !issetOk { - return thrift.NewTProtocolExceptionWithType(thrift.INVALID_DATA, fmt.Errorf("Required field Ok is not set")) + if !issetWriteNewSeriesBackoffDuration { + return thrift.NewTProtocolExceptionWithType(thrift.INVALID_DATA, fmt.Errorf("Required field WriteNewSeriesBackoffDuration is not set")) } - if !issetStatus { - return thrift.NewTProtocolExceptionWithType(thrift.INVALID_DATA, fmt.Errorf("Required field Status is not set")) + if !issetDurationType { + return thrift.NewTProtocolExceptionWithType(thrift.INVALID_DATA, fmt.Errorf("Required field DurationType is not set")) } return nil } -func (p *HealthResult_) ReadField1(iprot thrift.TProtocol) error { - if v, err := iprot.ReadBool(); err != nil { +func (p *NodeWriteNewSeriesBackoffDurationResult_) ReadField1(iprot thrift.TProtocol) error { + if v, err := iprot.ReadI64(); err != nil { return thrift.PrependError("error reading field 1: ", err) } else { - p.Ok = v + p.WriteNewSeriesBackoffDuration = v } return nil } -func (p *HealthResult_) ReadField2(iprot thrift.TProtocol) error { - if v, err := iprot.ReadString(); err != nil { +func (p *NodeWriteNewSeriesBackoffDurationResult_) ReadField2(iprot thrift.TProtocol) error { + if v, err := iprot.ReadI32(); err != nil { return thrift.PrependError("error reading field 2: ", err) } else { - p.Status = v + temp := TimeType(v) + p.DurationType = temp } return nil } -func (p *HealthResult_) Write(oprot thrift.TProtocol) error { - if err := oprot.WriteStructBegin("HealthResult"); err != nil { +func (p *NodeWriteNewSeriesBackoffDurationResult_) Write(oprot thrift.TProtocol) error { + if err := oprot.WriteStructBegin("NodeWriteNewSeriesBackoffDurationResult"); err != nil { return thrift.PrependError(fmt.Sprintf("%T write struct begin error: ", p), err) } if p != nil { @@ -7824,134 +8118,72 @@ func (p *HealthResult_) Write(oprot thrift.TProtocol) error { return nil } -func (p *HealthResult_) writeField1(oprot thrift.TProtocol) (err error) { - if err := oprot.WriteFieldBegin("ok", thrift.BOOL, 1); err != nil { - return thrift.PrependError(fmt.Sprintf("%T write field begin error 1:ok: ", p), err) +func (p *NodeWriteNewSeriesBackoffDurationResult_) writeField1(oprot thrift.TProtocol) (err error) { + if err := oprot.WriteFieldBegin("writeNewSeriesBackoffDuration", thrift.I64, 1); err != nil { + return thrift.PrependError(fmt.Sprintf("%T write field begin error 1:writeNewSeriesBackoffDuration: ", p), err) } - if err := oprot.WriteBool(bool(p.Ok)); err != nil { - return thrift.PrependError(fmt.Sprintf("%T.ok (1) field write error: ", p), err) + if err := oprot.WriteI64(int64(p.WriteNewSeriesBackoffDuration)); err != nil { + return thrift.PrependError(fmt.Sprintf("%T.writeNewSeriesBackoffDuration (1) field write error: ", p), err) } if err := oprot.WriteFieldEnd(); err != nil { - return thrift.PrependError(fmt.Sprintf("%T write field end error 1:ok: ", p), err) + return thrift.PrependError(fmt.Sprintf("%T write field end error 1:writeNewSeriesBackoffDuration: ", p), err) } return err } -func (p *HealthResult_) writeField2(oprot thrift.TProtocol) (err error) { - if err := oprot.WriteFieldBegin("status", thrift.STRING, 2); err != nil { - return thrift.PrependError(fmt.Sprintf("%T write field begin error 2:status: ", p), err) +func (p *NodeWriteNewSeriesBackoffDurationResult_) writeField2(oprot thrift.TProtocol) (err error) { + if err := oprot.WriteFieldBegin("durationType", thrift.I32, 2); err != nil { + return thrift.PrependError(fmt.Sprintf("%T write field begin error 2:durationType: ", p), err) } - if err := oprot.WriteString(string(p.Status)); err != nil { - return thrift.PrependError(fmt.Sprintf("%T.status (2) field write error: ", p), err) + if err := oprot.WriteI32(int32(p.DurationType)); err != nil { + return thrift.PrependError(fmt.Sprintf("%T.durationType (2) field write error: ", p), err) } if err := oprot.WriteFieldEnd(); err != nil { - return thrift.PrependError(fmt.Sprintf("%T write field end error 2:status: ", p), err) + return thrift.PrependError(fmt.Sprintf("%T write field end error 2:durationType: ", p), err) } return err } -func (p *HealthResult_) String() string { +func (p *NodeWriteNewSeriesBackoffDurationResult_) String() string { if p == nil { return "" } - return fmt.Sprintf("HealthResult_(%+v)", *p) + return fmt.Sprintf("NodeWriteNewSeriesBackoffDurationResult_(%+v)", *p) } // Attributes: -// - Query -// - RangeStart -// - RangeEnd -// - NameSpace -// - Limit -// - TagNameFilter -// - AggregateQueryType -// - RangeType -type AggregateQueryRawRequest struct { - Query []byte `thrift:"query,1,required" db:"query" json:"query"` - RangeStart int64 `thrift:"rangeStart,2,required" db:"rangeStart" json:"rangeStart"` - RangeEnd int64 `thrift:"rangeEnd,3,required" db:"rangeEnd" json:"rangeEnd"` - NameSpace []byte `thrift:"nameSpace,4,required" db:"nameSpace" json:"nameSpace"` - Limit *int64 `thrift:"limit,5" db:"limit" json:"limit,omitempty"` - TagNameFilter [][]byte `thrift:"tagNameFilter,6" db:"tagNameFilter" json:"tagNameFilter,omitempty"` - AggregateQueryType AggregateQueryType `thrift:"aggregateQueryType,7" db:"aggregateQueryType" json:"aggregateQueryType,omitempty"` - RangeType TimeType `thrift:"rangeType,8" db:"rangeType" json:"rangeType,omitempty"` -} - -func NewAggregateQueryRawRequest() *AggregateQueryRawRequest { - return &AggregateQueryRawRequest{ - AggregateQueryType: 1, - - RangeType: 0, - } -} - -func (p *AggregateQueryRawRequest) GetQuery() []byte { - return p.Query -} - -func (p *AggregateQueryRawRequest) GetRangeStart() int64 { - return p.RangeStart -} - -func (p *AggregateQueryRawRequest) GetRangeEnd() int64 { - return p.RangeEnd -} - -func (p *AggregateQueryRawRequest) GetNameSpace() []byte { - return p.NameSpace +// - WriteNewSeriesBackoffDuration +// - DurationType +type NodeSetWriteNewSeriesBackoffDurationRequest struct { + WriteNewSeriesBackoffDuration int64 `thrift:"writeNewSeriesBackoffDuration,1,required" db:"writeNewSeriesBackoffDuration" json:"writeNewSeriesBackoffDuration"` + DurationType TimeType `thrift:"durationType,2" db:"durationType" json:"durationType,omitempty"` } -var AggregateQueryRawRequest_Limit_DEFAULT int64 - -func (p *AggregateQueryRawRequest) GetLimit() int64 { - if !p.IsSetLimit() { - return AggregateQueryRawRequest_Limit_DEFAULT +func NewNodeSetWriteNewSeriesBackoffDurationRequest() *NodeSetWriteNewSeriesBackoffDurationRequest { + return &NodeSetWriteNewSeriesBackoffDurationRequest{ + DurationType: 2, } - return *p.Limit } -var AggregateQueryRawRequest_TagNameFilter_DEFAULT [][]byte - -func (p *AggregateQueryRawRequest) GetTagNameFilter() [][]byte { - return p.TagNameFilter -} - -var AggregateQueryRawRequest_AggregateQueryType_DEFAULT AggregateQueryType = 1 - -func (p *AggregateQueryRawRequest) GetAggregateQueryType() AggregateQueryType { - return p.AggregateQueryType -} - -var AggregateQueryRawRequest_RangeType_DEFAULT TimeType = 0 - -func (p *AggregateQueryRawRequest) GetRangeType() TimeType { - return p.RangeType -} -func (p *AggregateQueryRawRequest) IsSetLimit() bool { - return p.Limit != nil +func (p *NodeSetWriteNewSeriesBackoffDurationRequest) GetWriteNewSeriesBackoffDuration() int64 { + return p.WriteNewSeriesBackoffDuration } -func (p *AggregateQueryRawRequest) IsSetTagNameFilter() bool { - return p.TagNameFilter != nil -} +var NodeSetWriteNewSeriesBackoffDurationRequest_DurationType_DEFAULT TimeType = 2 -func (p *AggregateQueryRawRequest) IsSetAggregateQueryType() bool { - return p.AggregateQueryType != AggregateQueryRawRequest_AggregateQueryType_DEFAULT +func (p *NodeSetWriteNewSeriesBackoffDurationRequest) GetDurationType() TimeType { + return p.DurationType } - -func (p *AggregateQueryRawRequest) IsSetRangeType() bool { - return p.RangeType != AggregateQueryRawRequest_RangeType_DEFAULT +func (p *NodeSetWriteNewSeriesBackoffDurationRequest) IsSetDurationType() bool { + return p.DurationType != NodeSetWriteNewSeriesBackoffDurationRequest_DurationType_DEFAULT } -func (p *AggregateQueryRawRequest) Read(iprot thrift.TProtocol) error { +func (p *NodeSetWriteNewSeriesBackoffDurationRequest) Read(iprot thrift.TProtocol) error { if _, err := iprot.ReadStructBegin(); err != nil { return thrift.PrependError(fmt.Sprintf("%T read error: ", p), err) } - var issetQuery bool = false - var issetRangeStart bool = false - var issetRangeEnd bool = false - var issetNameSpace bool = false + var issetWriteNewSeriesBackoffDuration bool = false for { _, fieldTypeId, fieldId, err := iprot.ReadFieldBegin() @@ -7966,38 +8198,11 @@ func (p *AggregateQueryRawRequest) Read(iprot thrift.TProtocol) error { if err := p.ReadField1(iprot); err != nil { return err } - issetQuery = true + issetWriteNewSeriesBackoffDuration = true case 2: if err := p.ReadField2(iprot); err != nil { return err } - issetRangeStart = true - case 3: - if err := p.ReadField3(iprot); err != nil { - return err - } - issetRangeEnd = true - case 4: - if err := p.ReadField4(iprot); err != nil { - return err - } - issetNameSpace = true - case 5: - if err := p.ReadField5(iprot); err != nil { - return err - } - case 6: - if err := p.ReadField6(iprot); err != nil { - return err - } - case 7: - if err := p.ReadField7(iprot); err != nil { - return err - } - case 8: - if err := p.ReadField8(iprot); err != nil { - return err - } default: if err := iprot.Skip(fieldTypeId); err != nil { return err @@ -8010,300 +8215,106 @@ func (p *AggregateQueryRawRequest) Read(iprot thrift.TProtocol) error { if err := iprot.ReadStructEnd(); err != nil { return thrift.PrependError(fmt.Sprintf("%T read struct end error: ", p), err) } - if !issetQuery { - return thrift.NewTProtocolExceptionWithType(thrift.INVALID_DATA, fmt.Errorf("Required field Query is not set")) - } - if !issetRangeStart { - return thrift.NewTProtocolExceptionWithType(thrift.INVALID_DATA, fmt.Errorf("Required field RangeStart is not set")) - } - if !issetRangeEnd { - return thrift.NewTProtocolExceptionWithType(thrift.INVALID_DATA, fmt.Errorf("Required field RangeEnd is not set")) - } - if !issetNameSpace { - return thrift.NewTProtocolExceptionWithType(thrift.INVALID_DATA, fmt.Errorf("Required field NameSpace is not set")) + if !issetWriteNewSeriesBackoffDuration { + return thrift.NewTProtocolExceptionWithType(thrift.INVALID_DATA, fmt.Errorf("Required field WriteNewSeriesBackoffDuration is not set")) } return nil } -func (p *AggregateQueryRawRequest) ReadField1(iprot thrift.TProtocol) error { - if v, err := iprot.ReadBinary(); err != nil { +func (p *NodeSetWriteNewSeriesBackoffDurationRequest) ReadField1(iprot thrift.TProtocol) error { + if v, err := iprot.ReadI64(); err != nil { return thrift.PrependError("error reading field 1: ", err) } else { - p.Query = v + p.WriteNewSeriesBackoffDuration = v } return nil } -func (p *AggregateQueryRawRequest) ReadField2(iprot thrift.TProtocol) error { - if v, err := iprot.ReadI64(); err != nil { +func (p *NodeSetWriteNewSeriesBackoffDurationRequest) ReadField2(iprot thrift.TProtocol) error { + if v, err := iprot.ReadI32(); err != nil { return thrift.PrependError("error reading field 2: ", err) } else { - p.RangeStart = v + temp := TimeType(v) + p.DurationType = temp } return nil } -func (p *AggregateQueryRawRequest) ReadField3(iprot thrift.TProtocol) error { - if v, err := iprot.ReadI64(); err != nil { - return thrift.PrependError("error reading field 3: ", err) - } else { - p.RangeEnd = v +func (p *NodeSetWriteNewSeriesBackoffDurationRequest) Write(oprot thrift.TProtocol) error { + if err := oprot.WriteStructBegin("NodeSetWriteNewSeriesBackoffDurationRequest"); err != nil { + return thrift.PrependError(fmt.Sprintf("%T write struct begin error: ", p), err) } - return nil -} - -func (p *AggregateQueryRawRequest) ReadField4(iprot thrift.TProtocol) error { - if v, err := iprot.ReadBinary(); err != nil { - return thrift.PrependError("error reading field 4: ", err) - } else { - p.NameSpace = v + if p != nil { + if err := p.writeField1(oprot); err != nil { + return err + } + if err := p.writeField2(oprot); err != nil { + return err + } + } + if err := oprot.WriteFieldStop(); err != nil { + return thrift.PrependError("write field stop error: ", err) + } + if err := oprot.WriteStructEnd(); err != nil { + return thrift.PrependError("write struct stop error: ", err) } return nil } -func (p *AggregateQueryRawRequest) ReadField5(iprot thrift.TProtocol) error { - if v, err := iprot.ReadI64(); err != nil { - return thrift.PrependError("error reading field 5: ", err) - } else { - p.Limit = &v +func (p *NodeSetWriteNewSeriesBackoffDurationRequest) writeField1(oprot thrift.TProtocol) (err error) { + if err := oprot.WriteFieldBegin("writeNewSeriesBackoffDuration", thrift.I64, 1); err != nil { + return thrift.PrependError(fmt.Sprintf("%T write field begin error 1:writeNewSeriesBackoffDuration: ", p), err) } - return nil + if err := oprot.WriteI64(int64(p.WriteNewSeriesBackoffDuration)); err != nil { + return thrift.PrependError(fmt.Sprintf("%T.writeNewSeriesBackoffDuration (1) field write error: ", p), err) + } + if err := oprot.WriteFieldEnd(); err != nil { + return thrift.PrependError(fmt.Sprintf("%T write field end error 1:writeNewSeriesBackoffDuration: ", p), err) + } + return err } -func (p *AggregateQueryRawRequest) ReadField6(iprot thrift.TProtocol) error { - _, size, err := iprot.ReadListBegin() - if err != nil { - return thrift.PrependError("error reading list begin: ", err) - } - tSlice := make([][]byte, 0, size) - p.TagNameFilter = tSlice - for i := 0; i < size; i++ { - var _elem16 []byte - if v, err := iprot.ReadBinary(); err != nil { - return thrift.PrependError("error reading field 0: ", err) - } else { - _elem16 = v - } - p.TagNameFilter = append(p.TagNameFilter, _elem16) - } - if err := iprot.ReadListEnd(); err != nil { - return thrift.PrependError("error reading list end: ", err) - } - return nil -} - -func (p *AggregateQueryRawRequest) ReadField7(iprot thrift.TProtocol) error { - if v, err := iprot.ReadI32(); err != nil { - return thrift.PrependError("error reading field 7: ", err) - } else { - temp := AggregateQueryType(v) - p.AggregateQueryType = temp - } - return nil -} - -func (p *AggregateQueryRawRequest) ReadField8(iprot thrift.TProtocol) error { - if v, err := iprot.ReadI32(); err != nil { - return thrift.PrependError("error reading field 8: ", err) - } else { - temp := TimeType(v) - p.RangeType = temp - } - return nil -} - -func (p *AggregateQueryRawRequest) Write(oprot thrift.TProtocol) error { - if err := oprot.WriteStructBegin("AggregateQueryRawRequest"); err != nil { - return thrift.PrependError(fmt.Sprintf("%T write struct begin error: ", p), err) - } - if p != nil { - if err := p.writeField1(oprot); err != nil { - return err - } - if err := p.writeField2(oprot); err != nil { - return err - } - if err := p.writeField3(oprot); err != nil { - return err - } - if err := p.writeField4(oprot); err != nil { - return err - } - if err := p.writeField5(oprot); err != nil { - return err - } - if err := p.writeField6(oprot); err != nil { - return err - } - if err := p.writeField7(oprot); err != nil { - return err - } - if err := p.writeField8(oprot); err != nil { - return err - } - } - if err := oprot.WriteFieldStop(); err != nil { - return thrift.PrependError("write field stop error: ", err) - } - if err := oprot.WriteStructEnd(); err != nil { - return thrift.PrependError("write struct stop error: ", err) - } - return nil -} - -func (p *AggregateQueryRawRequest) writeField1(oprot thrift.TProtocol) (err error) { - if err := oprot.WriteFieldBegin("query", thrift.STRING, 1); err != nil { - return thrift.PrependError(fmt.Sprintf("%T write field begin error 1:query: ", p), err) - } - if err := oprot.WriteBinary(p.Query); err != nil { - return thrift.PrependError(fmt.Sprintf("%T.query (1) field write error: ", p), err) - } - if err := oprot.WriteFieldEnd(); err != nil { - return thrift.PrependError(fmt.Sprintf("%T write field end error 1:query: ", p), err) - } - return err -} - -func (p *AggregateQueryRawRequest) writeField2(oprot thrift.TProtocol) (err error) { - if err := oprot.WriteFieldBegin("rangeStart", thrift.I64, 2); err != nil { - return thrift.PrependError(fmt.Sprintf("%T write field begin error 2:rangeStart: ", p), err) - } - if err := oprot.WriteI64(int64(p.RangeStart)); err != nil { - return thrift.PrependError(fmt.Sprintf("%T.rangeStart (2) field write error: ", p), err) - } - if err := oprot.WriteFieldEnd(); err != nil { - return thrift.PrependError(fmt.Sprintf("%T write field end error 2:rangeStart: ", p), err) - } - return err -} - -func (p *AggregateQueryRawRequest) writeField3(oprot thrift.TProtocol) (err error) { - if err := oprot.WriteFieldBegin("rangeEnd", thrift.I64, 3); err != nil { - return thrift.PrependError(fmt.Sprintf("%T write field begin error 3:rangeEnd: ", p), err) - } - if err := oprot.WriteI64(int64(p.RangeEnd)); err != nil { - return thrift.PrependError(fmt.Sprintf("%T.rangeEnd (3) field write error: ", p), err) - } - if err := oprot.WriteFieldEnd(); err != nil { - return thrift.PrependError(fmt.Sprintf("%T write field end error 3:rangeEnd: ", p), err) - } - return err -} - -func (p *AggregateQueryRawRequest) writeField4(oprot thrift.TProtocol) (err error) { - if err := oprot.WriteFieldBegin("nameSpace", thrift.STRING, 4); err != nil { - return thrift.PrependError(fmt.Sprintf("%T write field begin error 4:nameSpace: ", p), err) - } - if err := oprot.WriteBinary(p.NameSpace); err != nil { - return thrift.PrependError(fmt.Sprintf("%T.nameSpace (4) field write error: ", p), err) - } - if err := oprot.WriteFieldEnd(); err != nil { - return thrift.PrependError(fmt.Sprintf("%T write field end error 4:nameSpace: ", p), err) - } - return err -} - -func (p *AggregateQueryRawRequest) writeField5(oprot thrift.TProtocol) (err error) { - if p.IsSetLimit() { - if err := oprot.WriteFieldBegin("limit", thrift.I64, 5); err != nil { - return thrift.PrependError(fmt.Sprintf("%T write field begin error 5:limit: ", p), err) - } - if err := oprot.WriteI64(int64(*p.Limit)); err != nil { - return thrift.PrependError(fmt.Sprintf("%T.limit (5) field write error: ", p), err) - } - if err := oprot.WriteFieldEnd(); err != nil { - return thrift.PrependError(fmt.Sprintf("%T write field end error 5:limit: ", p), err) - } - } - return err -} - -func (p *AggregateQueryRawRequest) writeField6(oprot thrift.TProtocol) (err error) { - if p.IsSetTagNameFilter() { - if err := oprot.WriteFieldBegin("tagNameFilter", thrift.LIST, 6); err != nil { - return thrift.PrependError(fmt.Sprintf("%T write field begin error 6:tagNameFilter: ", p), err) - } - if err := oprot.WriteListBegin(thrift.STRING, len(p.TagNameFilter)); err != nil { - return thrift.PrependError("error writing list begin: ", err) - } - for _, v := range p.TagNameFilter { - if err := oprot.WriteBinary(v); err != nil { - return thrift.PrependError(fmt.Sprintf("%T. (0) field write error: ", p), err) - } - } - if err := oprot.WriteListEnd(); err != nil { - return thrift.PrependError("error writing list end: ", err) - } - if err := oprot.WriteFieldEnd(); err != nil { - return thrift.PrependError(fmt.Sprintf("%T write field end error 6:tagNameFilter: ", p), err) - } - } - return err -} - -func (p *AggregateQueryRawRequest) writeField7(oprot thrift.TProtocol) (err error) { - if p.IsSetAggregateQueryType() { - if err := oprot.WriteFieldBegin("aggregateQueryType", thrift.I32, 7); err != nil { - return thrift.PrependError(fmt.Sprintf("%T write field begin error 7:aggregateQueryType: ", p), err) - } - if err := oprot.WriteI32(int32(p.AggregateQueryType)); err != nil { - return thrift.PrependError(fmt.Sprintf("%T.aggregateQueryType (7) field write error: ", p), err) - } - if err := oprot.WriteFieldEnd(); err != nil { - return thrift.PrependError(fmt.Sprintf("%T write field end error 7:aggregateQueryType: ", p), err) - } - } - return err -} - -func (p *AggregateQueryRawRequest) writeField8(oprot thrift.TProtocol) (err error) { - if p.IsSetRangeType() { - if err := oprot.WriteFieldBegin("rangeType", thrift.I32, 8); err != nil { - return thrift.PrependError(fmt.Sprintf("%T write field begin error 8:rangeType: ", p), err) +func (p *NodeSetWriteNewSeriesBackoffDurationRequest) writeField2(oprot thrift.TProtocol) (err error) { + if p.IsSetDurationType() { + if err := oprot.WriteFieldBegin("durationType", thrift.I32, 2); err != nil { + return thrift.PrependError(fmt.Sprintf("%T write field begin error 2:durationType: ", p), err) } - if err := oprot.WriteI32(int32(p.RangeType)); err != nil { - return thrift.PrependError(fmt.Sprintf("%T.rangeType (8) field write error: ", p), err) + if err := oprot.WriteI32(int32(p.DurationType)); err != nil { + return thrift.PrependError(fmt.Sprintf("%T.durationType (2) field write error: ", p), err) } if err := oprot.WriteFieldEnd(); err != nil { - return thrift.PrependError(fmt.Sprintf("%T write field end error 8:rangeType: ", p), err) + return thrift.PrependError(fmt.Sprintf("%T write field end error 2:durationType: ", p), err) } } return err } -func (p *AggregateQueryRawRequest) String() string { +func (p *NodeSetWriteNewSeriesBackoffDurationRequest) String() string { if p == nil { return "" } - return fmt.Sprintf("AggregateQueryRawRequest(%+v)", *p) + return fmt.Sprintf("NodeSetWriteNewSeriesBackoffDurationRequest(%+v)", *p) } // Attributes: -// - Results -// - Exhaustive -type AggregateQueryRawResult_ struct { - Results []*AggregateQueryRawResultTagNameElement `thrift:"results,1,required" db:"results" json:"results"` - Exhaustive bool `thrift:"exhaustive,2,required" db:"exhaustive" json:"exhaustive"` -} - -func NewAggregateQueryRawResult_() *AggregateQueryRawResult_ { - return &AggregateQueryRawResult_{} +// - WriteNewSeriesLimitPerShardPerSecond +type NodeWriteNewSeriesLimitPerShardPerSecondResult_ struct { + WriteNewSeriesLimitPerShardPerSecond int64 `thrift:"writeNewSeriesLimitPerShardPerSecond,1,required" db:"writeNewSeriesLimitPerShardPerSecond" json:"writeNewSeriesLimitPerShardPerSecond"` } -func (p *AggregateQueryRawResult_) GetResults() []*AggregateQueryRawResultTagNameElement { - return p.Results +func NewNodeWriteNewSeriesLimitPerShardPerSecondResult_() *NodeWriteNewSeriesLimitPerShardPerSecondResult_ { + return &NodeWriteNewSeriesLimitPerShardPerSecondResult_{} } -func (p *AggregateQueryRawResult_) GetExhaustive() bool { - return p.Exhaustive +func (p *NodeWriteNewSeriesLimitPerShardPerSecondResult_) GetWriteNewSeriesLimitPerShardPerSecond() int64 { + return p.WriteNewSeriesLimitPerShardPerSecond } -func (p *AggregateQueryRawResult_) Read(iprot thrift.TProtocol) error { +func (p *NodeWriteNewSeriesLimitPerShardPerSecondResult_) Read(iprot thrift.TProtocol) error { if _, err := iprot.ReadStructBegin(); err != nil { return thrift.PrependError(fmt.Sprintf("%T read error: ", p), err) } - var issetResults bool = false - var issetExhaustive bool = false + var issetWriteNewSeriesLimitPerShardPerSecond bool = false for { _, fieldTypeId, fieldId, err := iprot.ReadFieldBegin() @@ -8318,12 +8329,7 @@ func (p *AggregateQueryRawResult_) Read(iprot thrift.TProtocol) error { if err := p.ReadField1(iprot); err != nil { return err } - issetResults = true - case 2: - if err := p.ReadField2(iprot); err != nil { - return err - } - issetExhaustive = true + issetWriteNewSeriesLimitPerShardPerSecond = true default: if err := iprot.Skip(fieldTypeId); err != nil { return err @@ -8336,55 +8342,29 @@ func (p *AggregateQueryRawResult_) Read(iprot thrift.TProtocol) error { if err := iprot.ReadStructEnd(); err != nil { return thrift.PrependError(fmt.Sprintf("%T read struct end error: ", p), err) } - if !issetResults { - return thrift.NewTProtocolExceptionWithType(thrift.INVALID_DATA, fmt.Errorf("Required field Results is not set")) - } - if !issetExhaustive { - return thrift.NewTProtocolExceptionWithType(thrift.INVALID_DATA, fmt.Errorf("Required field Exhaustive is not set")) + if !issetWriteNewSeriesLimitPerShardPerSecond { + return thrift.NewTProtocolExceptionWithType(thrift.INVALID_DATA, fmt.Errorf("Required field WriteNewSeriesLimitPerShardPerSecond is not set")) } return nil } -func (p *AggregateQueryRawResult_) ReadField1(iprot thrift.TProtocol) error { - _, size, err := iprot.ReadListBegin() - if err != nil { - return thrift.PrependError("error reading list begin: ", err) - } - tSlice := make([]*AggregateQueryRawResultTagNameElement, 0, size) - p.Results = tSlice - for i := 0; i < size; i++ { - _elem17 := &AggregateQueryRawResultTagNameElement{} - if err := _elem17.Read(iprot); err != nil { - return thrift.PrependError(fmt.Sprintf("%T error reading struct: ", _elem17), err) - } - p.Results = append(p.Results, _elem17) - } - if err := iprot.ReadListEnd(); err != nil { - return thrift.PrependError("error reading list end: ", err) +func (p *NodeWriteNewSeriesLimitPerShardPerSecondResult_) ReadField1(iprot thrift.TProtocol) error { + if v, err := iprot.ReadI64(); err != nil { + return thrift.PrependError("error reading field 1: ", err) + } else { + p.WriteNewSeriesLimitPerShardPerSecond = v } return nil } -func (p *AggregateQueryRawResult_) ReadField2(iprot thrift.TProtocol) error { - if v, err := iprot.ReadBool(); err != nil { - return thrift.PrependError("error reading field 2: ", err) - } else { - p.Exhaustive = v - } - return nil -} - -func (p *AggregateQueryRawResult_) Write(oprot thrift.TProtocol) error { - if err := oprot.WriteStructBegin("AggregateQueryRawResult"); err != nil { +func (p *NodeWriteNewSeriesLimitPerShardPerSecondResult_) Write(oprot thrift.TProtocol) error { + if err := oprot.WriteStructBegin("NodeWriteNewSeriesLimitPerShardPerSecondResult"); err != nil { return thrift.PrependError(fmt.Sprintf("%T write struct begin error: ", p), err) } if p != nil { if err := p.writeField1(oprot); err != nil { return err } - if err := p.writeField2(oprot); err != nil { - return err - } } if err := oprot.WriteFieldStop(); err != nil { return thrift.PrependError("write field stop error: ", err) @@ -8395,78 +8375,45 @@ func (p *AggregateQueryRawResult_) Write(oprot thrift.TProtocol) error { return nil } -func (p *AggregateQueryRawResult_) writeField1(oprot thrift.TProtocol) (err error) { - if err := oprot.WriteFieldBegin("results", thrift.LIST, 1); err != nil { - return thrift.PrependError(fmt.Sprintf("%T write field begin error 1:results: ", p), err) - } - if err := oprot.WriteListBegin(thrift.STRUCT, len(p.Results)); err != nil { - return thrift.PrependError("error writing list begin: ", err) - } - for _, v := range p.Results { - if err := v.Write(oprot); err != nil { - return thrift.PrependError(fmt.Sprintf("%T error writing struct: ", v), err) - } - } - if err := oprot.WriteListEnd(); err != nil { - return thrift.PrependError("error writing list end: ", err) - } - if err := oprot.WriteFieldEnd(); err != nil { - return thrift.PrependError(fmt.Sprintf("%T write field end error 1:results: ", p), err) - } - return err -} - -func (p *AggregateQueryRawResult_) writeField2(oprot thrift.TProtocol) (err error) { - if err := oprot.WriteFieldBegin("exhaustive", thrift.BOOL, 2); err != nil { - return thrift.PrependError(fmt.Sprintf("%T write field begin error 2:exhaustive: ", p), err) +func (p *NodeWriteNewSeriesLimitPerShardPerSecondResult_) writeField1(oprot thrift.TProtocol) (err error) { + if err := oprot.WriteFieldBegin("writeNewSeriesLimitPerShardPerSecond", thrift.I64, 1); err != nil { + return thrift.PrependError(fmt.Sprintf("%T write field begin error 1:writeNewSeriesLimitPerShardPerSecond: ", p), err) } - if err := oprot.WriteBool(bool(p.Exhaustive)); err != nil { - return thrift.PrependError(fmt.Sprintf("%T.exhaustive (2) field write error: ", p), err) + if err := oprot.WriteI64(int64(p.WriteNewSeriesLimitPerShardPerSecond)); err != nil { + return thrift.PrependError(fmt.Sprintf("%T.writeNewSeriesLimitPerShardPerSecond (1) field write error: ", p), err) } if err := oprot.WriteFieldEnd(); err != nil { - return thrift.PrependError(fmt.Sprintf("%T write field end error 2:exhaustive: ", p), err) + return thrift.PrependError(fmt.Sprintf("%T write field end error 1:writeNewSeriesLimitPerShardPerSecond: ", p), err) } return err } -func (p *AggregateQueryRawResult_) String() string { +func (p *NodeWriteNewSeriesLimitPerShardPerSecondResult_) String() string { if p == nil { return "" } - return fmt.Sprintf("AggregateQueryRawResult_(%+v)", *p) + return fmt.Sprintf("NodeWriteNewSeriesLimitPerShardPerSecondResult_(%+v)", *p) } // Attributes: -// - TagName -// - TagValues -type AggregateQueryRawResultTagNameElement struct { - TagName []byte `thrift:"tagName,1,required" db:"tagName" json:"tagName"` - TagValues []*AggregateQueryRawResultTagValueElement `thrift:"tagValues,2" db:"tagValues" json:"tagValues,omitempty"` -} - -func NewAggregateQueryRawResultTagNameElement() *AggregateQueryRawResultTagNameElement { - return &AggregateQueryRawResultTagNameElement{} +// - WriteNewSeriesLimitPerShardPerSecond +type NodeSetWriteNewSeriesLimitPerShardPerSecondRequest struct { + WriteNewSeriesLimitPerShardPerSecond int64 `thrift:"writeNewSeriesLimitPerShardPerSecond,1,required" db:"writeNewSeriesLimitPerShardPerSecond" json:"writeNewSeriesLimitPerShardPerSecond"` } -func (p *AggregateQueryRawResultTagNameElement) GetTagName() []byte { - return p.TagName +func NewNodeSetWriteNewSeriesLimitPerShardPerSecondRequest() *NodeSetWriteNewSeriesLimitPerShardPerSecondRequest { + return &NodeSetWriteNewSeriesLimitPerShardPerSecondRequest{} } -var AggregateQueryRawResultTagNameElement_TagValues_DEFAULT []*AggregateQueryRawResultTagValueElement - -func (p *AggregateQueryRawResultTagNameElement) GetTagValues() []*AggregateQueryRawResultTagValueElement { - return p.TagValues -} -func (p *AggregateQueryRawResultTagNameElement) IsSetTagValues() bool { - return p.TagValues != nil +func (p *NodeSetWriteNewSeriesLimitPerShardPerSecondRequest) GetWriteNewSeriesLimitPerShardPerSecond() int64 { + return p.WriteNewSeriesLimitPerShardPerSecond } - -func (p *AggregateQueryRawResultTagNameElement) Read(iprot thrift.TProtocol) error { +func (p *NodeSetWriteNewSeriesLimitPerShardPerSecondRequest) Read(iprot thrift.TProtocol) error { if _, err := iprot.ReadStructBegin(); err != nil { return thrift.PrependError(fmt.Sprintf("%T read error: ", p), err) } - var issetTagName bool = false + var issetWriteNewSeriesLimitPerShardPerSecond bool = false for { _, fieldTypeId, fieldId, err := iprot.ReadFieldBegin() @@ -8481,11 +8428,7 @@ func (p *AggregateQueryRawResultTagNameElement) Read(iprot thrift.TProtocol) err if err := p.ReadField1(iprot); err != nil { return err } - issetTagName = true - case 2: - if err := p.ReadField2(iprot); err != nil { - return err - } + issetWriteNewSeriesLimitPerShardPerSecond = true default: if err := iprot.Skip(fieldTypeId); err != nil { return err @@ -8498,52 +8441,29 @@ func (p *AggregateQueryRawResultTagNameElement) Read(iprot thrift.TProtocol) err if err := iprot.ReadStructEnd(); err != nil { return thrift.PrependError(fmt.Sprintf("%T read struct end error: ", p), err) } - if !issetTagName { - return thrift.NewTProtocolExceptionWithType(thrift.INVALID_DATA, fmt.Errorf("Required field TagName is not set")) + if !issetWriteNewSeriesLimitPerShardPerSecond { + return thrift.NewTProtocolExceptionWithType(thrift.INVALID_DATA, fmt.Errorf("Required field WriteNewSeriesLimitPerShardPerSecond is not set")) } return nil } -func (p *AggregateQueryRawResultTagNameElement) ReadField1(iprot thrift.TProtocol) error { - if v, err := iprot.ReadBinary(); err != nil { +func (p *NodeSetWriteNewSeriesLimitPerShardPerSecondRequest) ReadField1(iprot thrift.TProtocol) error { + if v, err := iprot.ReadI64(); err != nil { return thrift.PrependError("error reading field 1: ", err) } else { - p.TagName = v - } - return nil -} - -func (p *AggregateQueryRawResultTagNameElement) ReadField2(iprot thrift.TProtocol) error { - _, size, err := iprot.ReadListBegin() - if err != nil { - return thrift.PrependError("error reading list begin: ", err) - } - tSlice := make([]*AggregateQueryRawResultTagValueElement, 0, size) - p.TagValues = tSlice - for i := 0; i < size; i++ { - _elem18 := &AggregateQueryRawResultTagValueElement{} - if err := _elem18.Read(iprot); err != nil { - return thrift.PrependError(fmt.Sprintf("%T error reading struct: ", _elem18), err) - } - p.TagValues = append(p.TagValues, _elem18) - } - if err := iprot.ReadListEnd(); err != nil { - return thrift.PrependError("error reading list end: ", err) + p.WriteNewSeriesLimitPerShardPerSecond = v } return nil } -func (p *AggregateQueryRawResultTagNameElement) Write(oprot thrift.TProtocol) error { - if err := oprot.WriteStructBegin("AggregateQueryRawResultTagNameElement"); err != nil { +func (p *NodeSetWriteNewSeriesLimitPerShardPerSecondRequest) Write(oprot thrift.TProtocol) error { + if err := oprot.WriteStructBegin("NodeSetWriteNewSeriesLimitPerShardPerSecondRequest"); err != nil { return thrift.PrependError(fmt.Sprintf("%T write struct begin error: ", p), err) } if p != nil { if err := p.writeField1(oprot); err != nil { return err } - if err := p.writeField2(oprot); err != nil { - return err - } } if err := oprot.WriteFieldStop(); err != nil { return thrift.PrependError("write field stop error: ", err) @@ -8554,68 +8474,52 @@ func (p *AggregateQueryRawResultTagNameElement) Write(oprot thrift.TProtocol) er return nil } -func (p *AggregateQueryRawResultTagNameElement) writeField1(oprot thrift.TProtocol) (err error) { - if err := oprot.WriteFieldBegin("tagName", thrift.STRING, 1); err != nil { - return thrift.PrependError(fmt.Sprintf("%T write field begin error 1:tagName: ", p), err) +func (p *NodeSetWriteNewSeriesLimitPerShardPerSecondRequest) writeField1(oprot thrift.TProtocol) (err error) { + if err := oprot.WriteFieldBegin("writeNewSeriesLimitPerShardPerSecond", thrift.I64, 1); err != nil { + return thrift.PrependError(fmt.Sprintf("%T write field begin error 1:writeNewSeriesLimitPerShardPerSecond: ", p), err) } - if err := oprot.WriteBinary(p.TagName); err != nil { - return thrift.PrependError(fmt.Sprintf("%T.tagName (1) field write error: ", p), err) + if err := oprot.WriteI64(int64(p.WriteNewSeriesLimitPerShardPerSecond)); err != nil { + return thrift.PrependError(fmt.Sprintf("%T.writeNewSeriesLimitPerShardPerSecond (1) field write error: ", p), err) } if err := oprot.WriteFieldEnd(); err != nil { - return thrift.PrependError(fmt.Sprintf("%T write field end error 1:tagName: ", p), err) - } - return err -} - -func (p *AggregateQueryRawResultTagNameElement) writeField2(oprot thrift.TProtocol) (err error) { - if p.IsSetTagValues() { - if err := oprot.WriteFieldBegin("tagValues", thrift.LIST, 2); err != nil { - return thrift.PrependError(fmt.Sprintf("%T write field begin error 2:tagValues: ", p), err) - } - if err := oprot.WriteListBegin(thrift.STRUCT, len(p.TagValues)); err != nil { - return thrift.PrependError("error writing list begin: ", err) - } - for _, v := range p.TagValues { - if err := v.Write(oprot); err != nil { - return thrift.PrependError(fmt.Sprintf("%T error writing struct: ", v), err) - } - } - if err := oprot.WriteListEnd(); err != nil { - return thrift.PrependError("error writing list end: ", err) - } - if err := oprot.WriteFieldEnd(); err != nil { - return thrift.PrependError(fmt.Sprintf("%T write field end error 2:tagValues: ", p), err) - } + return thrift.PrependError(fmt.Sprintf("%T write field end error 1:writeNewSeriesLimitPerShardPerSecond: ", p), err) } return err } -func (p *AggregateQueryRawResultTagNameElement) String() string { +func (p *NodeSetWriteNewSeriesLimitPerShardPerSecondRequest) String() string { if p == nil { return "" } - return fmt.Sprintf("AggregateQueryRawResultTagNameElement(%+v)", *p) + return fmt.Sprintf("NodeSetWriteNewSeriesLimitPerShardPerSecondRequest(%+v)", *p) } // Attributes: -// - TagValue -type AggregateQueryRawResultTagValueElement struct { - TagValue []byte `thrift:"tagValue,1,required" db:"tagValue" json:"tagValue"` +// - Ok +// - Status +type HealthResult_ struct { + Ok bool `thrift:"ok,1,required" db:"ok" json:"ok"` + Status string `thrift:"status,2,required" db:"status" json:"status"` } -func NewAggregateQueryRawResultTagValueElement() *AggregateQueryRawResultTagValueElement { - return &AggregateQueryRawResultTagValueElement{} +func NewHealthResult_() *HealthResult_ { + return &HealthResult_{} } -func (p *AggregateQueryRawResultTagValueElement) GetTagValue() []byte { - return p.TagValue +func (p *HealthResult_) GetOk() bool { + return p.Ok } -func (p *AggregateQueryRawResultTagValueElement) Read(iprot thrift.TProtocol) error { + +func (p *HealthResult_) GetStatus() string { + return p.Status +} +func (p *HealthResult_) Read(iprot thrift.TProtocol) error { if _, err := iprot.ReadStructBegin(); err != nil { return thrift.PrependError(fmt.Sprintf("%T read error: ", p), err) } - var issetTagValue bool = false + var issetOk bool = false + var issetStatus bool = false for { _, fieldTypeId, fieldId, err := iprot.ReadFieldBegin() @@ -8630,7 +8534,12 @@ func (p *AggregateQueryRawResultTagValueElement) Read(iprot thrift.TProtocol) er if err := p.ReadField1(iprot); err != nil { return err } - issetTagValue = true + issetOk = true + case 2: + if err := p.ReadField2(iprot); err != nil { + return err + } + issetStatus = true default: if err := iprot.Skip(fieldTypeId); err != nil { return err @@ -8643,29 +8552,44 @@ func (p *AggregateQueryRawResultTagValueElement) Read(iprot thrift.TProtocol) er if err := iprot.ReadStructEnd(); err != nil { return thrift.PrependError(fmt.Sprintf("%T read struct end error: ", p), err) } - if !issetTagValue { - return thrift.NewTProtocolExceptionWithType(thrift.INVALID_DATA, fmt.Errorf("Required field TagValue is not set")) + if !issetOk { + return thrift.NewTProtocolExceptionWithType(thrift.INVALID_DATA, fmt.Errorf("Required field Ok is not set")) + } + if !issetStatus { + return thrift.NewTProtocolExceptionWithType(thrift.INVALID_DATA, fmt.Errorf("Required field Status is not set")) } return nil } -func (p *AggregateQueryRawResultTagValueElement) ReadField1(iprot thrift.TProtocol) error { - if v, err := iprot.ReadBinary(); err != nil { +func (p *HealthResult_) ReadField1(iprot thrift.TProtocol) error { + if v, err := iprot.ReadBool(); err != nil { return thrift.PrependError("error reading field 1: ", err) } else { - p.TagValue = v + p.Ok = v } return nil } -func (p *AggregateQueryRawResultTagValueElement) Write(oprot thrift.TProtocol) error { - if err := oprot.WriteStructBegin("AggregateQueryRawResultTagValueElement"); err != nil { +func (p *HealthResult_) ReadField2(iprot thrift.TProtocol) error { + if v, err := iprot.ReadString(); err != nil { + return thrift.PrependError("error reading field 2: ", err) + } else { + p.Status = v + } + return nil +} + +func (p *HealthResult_) Write(oprot thrift.TProtocol) error { + if err := oprot.WriteStructBegin("HealthResult"); err != nil { return thrift.PrependError(fmt.Sprintf("%T write struct begin error: ", p), err) } if p != nil { if err := p.writeField1(oprot); err != nil { return err } + if err := p.writeField2(oprot); err != nil { + return err + } } if err := oprot.WriteFieldStop(); err != nil { return thrift.PrependError("write field stop error: ", err) @@ -8676,24 +8600,37 @@ func (p *AggregateQueryRawResultTagValueElement) Write(oprot thrift.TProtocol) e return nil } -func (p *AggregateQueryRawResultTagValueElement) writeField1(oprot thrift.TProtocol) (err error) { - if err := oprot.WriteFieldBegin("tagValue", thrift.STRING, 1); err != nil { - return thrift.PrependError(fmt.Sprintf("%T write field begin error 1:tagValue: ", p), err) +func (p *HealthResult_) writeField1(oprot thrift.TProtocol) (err error) { + if err := oprot.WriteFieldBegin("ok", thrift.BOOL, 1); err != nil { + return thrift.PrependError(fmt.Sprintf("%T write field begin error 1:ok: ", p), err) } - if err := oprot.WriteBinary(p.TagValue); err != nil { - return thrift.PrependError(fmt.Sprintf("%T.tagValue (1) field write error: ", p), err) + if err := oprot.WriteBool(bool(p.Ok)); err != nil { + return thrift.PrependError(fmt.Sprintf("%T.ok (1) field write error: ", p), err) } if err := oprot.WriteFieldEnd(); err != nil { - return thrift.PrependError(fmt.Sprintf("%T write field end error 1:tagValue: ", p), err) + return thrift.PrependError(fmt.Sprintf("%T write field end error 1:ok: ", p), err) } return err } -func (p *AggregateQueryRawResultTagValueElement) String() string { +func (p *HealthResult_) writeField2(oprot thrift.TProtocol) (err error) { + if err := oprot.WriteFieldBegin("status", thrift.STRING, 2); err != nil { + return thrift.PrependError(fmt.Sprintf("%T write field begin error 2:status: ", p), err) + } + if err := oprot.WriteString(string(p.Status)); err != nil { + return thrift.PrependError(fmt.Sprintf("%T.status (2) field write error: ", p), err) + } + if err := oprot.WriteFieldEnd(); err != nil { + return thrift.PrependError(fmt.Sprintf("%T write field end error 2:status: ", p), err) + } + return err +} + +func (p *HealthResult_) String() string { if p == nil { return "" } - return fmt.Sprintf("AggregateQueryRawResultTagValueElement(%+v)", *p) + return fmt.Sprintf("HealthResult_(%+v)", *p) } // Attributes: @@ -8705,97 +8642,89 @@ func (p *AggregateQueryRawResultTagValueElement) String() string { // - TagNameFilter // - AggregateQueryType // - RangeType -type AggregateQueryRequest struct { - Query *Query `thrift:"query,1" db:"query" json:"query,omitempty"` +type AggregateQueryRawRequest struct { + Query []byte `thrift:"query,1,required" db:"query" json:"query"` RangeStart int64 `thrift:"rangeStart,2,required" db:"rangeStart" json:"rangeStart"` RangeEnd int64 `thrift:"rangeEnd,3,required" db:"rangeEnd" json:"rangeEnd"` - NameSpace string `thrift:"nameSpace,4,required" db:"nameSpace" json:"nameSpace"` + NameSpace []byte `thrift:"nameSpace,4,required" db:"nameSpace" json:"nameSpace"` Limit *int64 `thrift:"limit,5" db:"limit" json:"limit,omitempty"` - TagNameFilter []string `thrift:"tagNameFilter,6" db:"tagNameFilter" json:"tagNameFilter,omitempty"` + TagNameFilter [][]byte `thrift:"tagNameFilter,6" db:"tagNameFilter" json:"tagNameFilter,omitempty"` AggregateQueryType AggregateQueryType `thrift:"aggregateQueryType,7" db:"aggregateQueryType" json:"aggregateQueryType,omitempty"` RangeType TimeType `thrift:"rangeType,8" db:"rangeType" json:"rangeType,omitempty"` } -func NewAggregateQueryRequest() *AggregateQueryRequest { - return &AggregateQueryRequest{ +func NewAggregateQueryRawRequest() *AggregateQueryRawRequest { + return &AggregateQueryRawRequest{ AggregateQueryType: 1, RangeType: 0, } } -var AggregateQueryRequest_Query_DEFAULT *Query - -func (p *AggregateQueryRequest) GetQuery() *Query { - if !p.IsSetQuery() { - return AggregateQueryRequest_Query_DEFAULT - } +func (p *AggregateQueryRawRequest) GetQuery() []byte { return p.Query } -func (p *AggregateQueryRequest) GetRangeStart() int64 { +func (p *AggregateQueryRawRequest) GetRangeStart() int64 { return p.RangeStart } -func (p *AggregateQueryRequest) GetRangeEnd() int64 { +func (p *AggregateQueryRawRequest) GetRangeEnd() int64 { return p.RangeEnd } -func (p *AggregateQueryRequest) GetNameSpace() string { +func (p *AggregateQueryRawRequest) GetNameSpace() []byte { return p.NameSpace } -var AggregateQueryRequest_Limit_DEFAULT int64 +var AggregateQueryRawRequest_Limit_DEFAULT int64 -func (p *AggregateQueryRequest) GetLimit() int64 { +func (p *AggregateQueryRawRequest) GetLimit() int64 { if !p.IsSetLimit() { - return AggregateQueryRequest_Limit_DEFAULT + return AggregateQueryRawRequest_Limit_DEFAULT } return *p.Limit } -var AggregateQueryRequest_TagNameFilter_DEFAULT []string +var AggregateQueryRawRequest_TagNameFilter_DEFAULT [][]byte -func (p *AggregateQueryRequest) GetTagNameFilter() []string { +func (p *AggregateQueryRawRequest) GetTagNameFilter() [][]byte { return p.TagNameFilter } -var AggregateQueryRequest_AggregateQueryType_DEFAULT AggregateQueryType = 1 +var AggregateQueryRawRequest_AggregateQueryType_DEFAULT AggregateQueryType = 1 -func (p *AggregateQueryRequest) GetAggregateQueryType() AggregateQueryType { +func (p *AggregateQueryRawRequest) GetAggregateQueryType() AggregateQueryType { return p.AggregateQueryType } -var AggregateQueryRequest_RangeType_DEFAULT TimeType = 0 +var AggregateQueryRawRequest_RangeType_DEFAULT TimeType = 0 -func (p *AggregateQueryRequest) GetRangeType() TimeType { +func (p *AggregateQueryRawRequest) GetRangeType() TimeType { return p.RangeType } -func (p *AggregateQueryRequest) IsSetQuery() bool { - return p.Query != nil -} - -func (p *AggregateQueryRequest) IsSetLimit() bool { +func (p *AggregateQueryRawRequest) IsSetLimit() bool { return p.Limit != nil } -func (p *AggregateQueryRequest) IsSetTagNameFilter() bool { +func (p *AggregateQueryRawRequest) IsSetTagNameFilter() bool { return p.TagNameFilter != nil } -func (p *AggregateQueryRequest) IsSetAggregateQueryType() bool { - return p.AggregateQueryType != AggregateQueryRequest_AggregateQueryType_DEFAULT +func (p *AggregateQueryRawRequest) IsSetAggregateQueryType() bool { + return p.AggregateQueryType != AggregateQueryRawRequest_AggregateQueryType_DEFAULT } -func (p *AggregateQueryRequest) IsSetRangeType() bool { - return p.RangeType != AggregateQueryRequest_RangeType_DEFAULT +func (p *AggregateQueryRawRequest) IsSetRangeType() bool { + return p.RangeType != AggregateQueryRawRequest_RangeType_DEFAULT } -func (p *AggregateQueryRequest) Read(iprot thrift.TProtocol) error { +func (p *AggregateQueryRawRequest) Read(iprot thrift.TProtocol) error { if _, err := iprot.ReadStructBegin(); err != nil { return thrift.PrependError(fmt.Sprintf("%T read error: ", p), err) } + var issetQuery bool = false var issetRangeStart bool = false var issetRangeEnd bool = false var issetNameSpace bool = false @@ -8813,6 +8742,7 @@ func (p *AggregateQueryRequest) Read(iprot thrift.TProtocol) error { if err := p.ReadField1(iprot); err != nil { return err } + issetQuery = true case 2: if err := p.ReadField2(iprot); err != nil { return err @@ -8856,6 +8786,9 @@ func (p *AggregateQueryRequest) Read(iprot thrift.TProtocol) error { if err := iprot.ReadStructEnd(); err != nil { return thrift.PrependError(fmt.Sprintf("%T read struct end error: ", p), err) } + if !issetQuery { + return thrift.NewTProtocolExceptionWithType(thrift.INVALID_DATA, fmt.Errorf("Required field Query is not set")) + } if !issetRangeStart { return thrift.NewTProtocolExceptionWithType(thrift.INVALID_DATA, fmt.Errorf("Required field RangeStart is not set")) } @@ -8868,15 +8801,16 @@ func (p *AggregateQueryRequest) Read(iprot thrift.TProtocol) error { return nil } -func (p *AggregateQueryRequest) ReadField1(iprot thrift.TProtocol) error { - p.Query = &Query{} - if err := p.Query.Read(iprot); err != nil { - return thrift.PrependError(fmt.Sprintf("%T error reading struct: ", p.Query), err) +func (p *AggregateQueryRawRequest) ReadField1(iprot thrift.TProtocol) error { + if v, err := iprot.ReadBinary(); err != nil { + return thrift.PrependError("error reading field 1: ", err) + } else { + p.Query = v } return nil } -func (p *AggregateQueryRequest) ReadField2(iprot thrift.TProtocol) error { +func (p *AggregateQueryRawRequest) ReadField2(iprot thrift.TProtocol) error { if v, err := iprot.ReadI64(); err != nil { return thrift.PrependError("error reading field 2: ", err) } else { @@ -8885,7 +8819,7 @@ func (p *AggregateQueryRequest) ReadField2(iprot thrift.TProtocol) error { return nil } -func (p *AggregateQueryRequest) ReadField3(iprot thrift.TProtocol) error { +func (p *AggregateQueryRawRequest) ReadField3(iprot thrift.TProtocol) error { if v, err := iprot.ReadI64(); err != nil { return thrift.PrependError("error reading field 3: ", err) } else { @@ -8894,8 +8828,8 @@ func (p *AggregateQueryRequest) ReadField3(iprot thrift.TProtocol) error { return nil } -func (p *AggregateQueryRequest) ReadField4(iprot thrift.TProtocol) error { - if v, err := iprot.ReadString(); err != nil { +func (p *AggregateQueryRawRequest) ReadField4(iprot thrift.TProtocol) error { + if v, err := iprot.ReadBinary(); err != nil { return thrift.PrependError("error reading field 4: ", err) } else { p.NameSpace = v @@ -8903,7 +8837,7 @@ func (p *AggregateQueryRequest) ReadField4(iprot thrift.TProtocol) error { return nil } -func (p *AggregateQueryRequest) ReadField5(iprot thrift.TProtocol) error { +func (p *AggregateQueryRawRequest) ReadField5(iprot thrift.TProtocol) error { if v, err := iprot.ReadI64(); err != nil { return thrift.PrependError("error reading field 5: ", err) } else { @@ -8912,21 +8846,21 @@ func (p *AggregateQueryRequest) ReadField5(iprot thrift.TProtocol) error { return nil } -func (p *AggregateQueryRequest) ReadField6(iprot thrift.TProtocol) error { +func (p *AggregateQueryRawRequest) ReadField6(iprot thrift.TProtocol) error { _, size, err := iprot.ReadListBegin() if err != nil { return thrift.PrependError("error reading list begin: ", err) } - tSlice := make([]string, 0, size) + tSlice := make([][]byte, 0, size) p.TagNameFilter = tSlice for i := 0; i < size; i++ { - var _elem19 string - if v, err := iprot.ReadString(); err != nil { + var _elem20 []byte + if v, err := iprot.ReadBinary(); err != nil { return thrift.PrependError("error reading field 0: ", err) } else { - _elem19 = v + _elem20 = v } - p.TagNameFilter = append(p.TagNameFilter, _elem19) + p.TagNameFilter = append(p.TagNameFilter, _elem20) } if err := iprot.ReadListEnd(); err != nil { return thrift.PrependError("error reading list end: ", err) @@ -8934,7 +8868,7 @@ func (p *AggregateQueryRequest) ReadField6(iprot thrift.TProtocol) error { return nil } -func (p *AggregateQueryRequest) ReadField7(iprot thrift.TProtocol) error { +func (p *AggregateQueryRawRequest) ReadField7(iprot thrift.TProtocol) error { if v, err := iprot.ReadI32(); err != nil { return thrift.PrependError("error reading field 7: ", err) } else { @@ -8944,7 +8878,7 @@ func (p *AggregateQueryRequest) ReadField7(iprot thrift.TProtocol) error { return nil } -func (p *AggregateQueryRequest) ReadField8(iprot thrift.TProtocol) error { +func (p *AggregateQueryRawRequest) ReadField8(iprot thrift.TProtocol) error { if v, err := iprot.ReadI32(); err != nil { return thrift.PrependError("error reading field 8: ", err) } else { @@ -8954,8 +8888,8 @@ func (p *AggregateQueryRequest) ReadField8(iprot thrift.TProtocol) error { return nil } -func (p *AggregateQueryRequest) Write(oprot thrift.TProtocol) error { - if err := oprot.WriteStructBegin("AggregateQueryRequest"); err != nil { +func (p *AggregateQueryRawRequest) Write(oprot thrift.TProtocol) error { + if err := oprot.WriteStructBegin("AggregateQueryRawRequest"); err != nil { return thrift.PrependError(fmt.Sprintf("%T write struct begin error: ", p), err) } if p != nil { @@ -8993,22 +8927,20 @@ func (p *AggregateQueryRequest) Write(oprot thrift.TProtocol) error { return nil } -func (p *AggregateQueryRequest) writeField1(oprot thrift.TProtocol) (err error) { - if p.IsSetQuery() { - if err := oprot.WriteFieldBegin("query", thrift.STRUCT, 1); err != nil { - return thrift.PrependError(fmt.Sprintf("%T write field begin error 1:query: ", p), err) - } - if err := p.Query.Write(oprot); err != nil { - return thrift.PrependError(fmt.Sprintf("%T error writing struct: ", p.Query), err) - } - if err := oprot.WriteFieldEnd(); err != nil { - return thrift.PrependError(fmt.Sprintf("%T write field end error 1:query: ", p), err) - } +func (p *AggregateQueryRawRequest) writeField1(oprot thrift.TProtocol) (err error) { + if err := oprot.WriteFieldBegin("query", thrift.STRING, 1); err != nil { + return thrift.PrependError(fmt.Sprintf("%T write field begin error 1:query: ", p), err) + } + if err := oprot.WriteBinary(p.Query); err != nil { + return thrift.PrependError(fmt.Sprintf("%T.query (1) field write error: ", p), err) + } + if err := oprot.WriteFieldEnd(); err != nil { + return thrift.PrependError(fmt.Sprintf("%T write field end error 1:query: ", p), err) } return err } -func (p *AggregateQueryRequest) writeField2(oprot thrift.TProtocol) (err error) { +func (p *AggregateQueryRawRequest) writeField2(oprot thrift.TProtocol) (err error) { if err := oprot.WriteFieldBegin("rangeStart", thrift.I64, 2); err != nil { return thrift.PrependError(fmt.Sprintf("%T write field begin error 2:rangeStart: ", p), err) } @@ -9021,7 +8953,7 @@ func (p *AggregateQueryRequest) writeField2(oprot thrift.TProtocol) (err error) return err } -func (p *AggregateQueryRequest) writeField3(oprot thrift.TProtocol) (err error) { +func (p *AggregateQueryRawRequest) writeField3(oprot thrift.TProtocol) (err error) { if err := oprot.WriteFieldBegin("rangeEnd", thrift.I64, 3); err != nil { return thrift.PrependError(fmt.Sprintf("%T write field begin error 3:rangeEnd: ", p), err) } @@ -9034,11 +8966,11 @@ func (p *AggregateQueryRequest) writeField3(oprot thrift.TProtocol) (err error) return err } -func (p *AggregateQueryRequest) writeField4(oprot thrift.TProtocol) (err error) { +func (p *AggregateQueryRawRequest) writeField4(oprot thrift.TProtocol) (err error) { if err := oprot.WriteFieldBegin("nameSpace", thrift.STRING, 4); err != nil { return thrift.PrependError(fmt.Sprintf("%T write field begin error 4:nameSpace: ", p), err) } - if err := oprot.WriteString(string(p.NameSpace)); err != nil { + if err := oprot.WriteBinary(p.NameSpace); err != nil { return thrift.PrependError(fmt.Sprintf("%T.nameSpace (4) field write error: ", p), err) } if err := oprot.WriteFieldEnd(); err != nil { @@ -9047,7 +8979,7 @@ func (p *AggregateQueryRequest) writeField4(oprot thrift.TProtocol) (err error) return err } -func (p *AggregateQueryRequest) writeField5(oprot thrift.TProtocol) (err error) { +func (p *AggregateQueryRawRequest) writeField5(oprot thrift.TProtocol) (err error) { if p.IsSetLimit() { if err := oprot.WriteFieldBegin("limit", thrift.I64, 5); err != nil { return thrift.PrependError(fmt.Sprintf("%T write field begin error 5:limit: ", p), err) @@ -9062,7 +8994,7 @@ func (p *AggregateQueryRequest) writeField5(oprot thrift.TProtocol) (err error) return err } -func (p *AggregateQueryRequest) writeField6(oprot thrift.TProtocol) (err error) { +func (p *AggregateQueryRawRequest) writeField6(oprot thrift.TProtocol) (err error) { if p.IsSetTagNameFilter() { if err := oprot.WriteFieldBegin("tagNameFilter", thrift.LIST, 6); err != nil { return thrift.PrependError(fmt.Sprintf("%T write field begin error 6:tagNameFilter: ", p), err) @@ -9071,7 +9003,7 @@ func (p *AggregateQueryRequest) writeField6(oprot thrift.TProtocol) (err error) return thrift.PrependError("error writing list begin: ", err) } for _, v := range p.TagNameFilter { - if err := oprot.WriteString(string(v)); err != nil { + if err := oprot.WriteBinary(v); err != nil { return thrift.PrependError(fmt.Sprintf("%T. (0) field write error: ", p), err) } } @@ -9085,7 +9017,7 @@ func (p *AggregateQueryRequest) writeField6(oprot thrift.TProtocol) (err error) return err } -func (p *AggregateQueryRequest) writeField7(oprot thrift.TProtocol) (err error) { +func (p *AggregateQueryRawRequest) writeField7(oprot thrift.TProtocol) (err error) { if p.IsSetAggregateQueryType() { if err := oprot.WriteFieldBegin("aggregateQueryType", thrift.I32, 7); err != nil { return thrift.PrependError(fmt.Sprintf("%T write field begin error 7:aggregateQueryType: ", p), err) @@ -9100,7 +9032,7 @@ func (p *AggregateQueryRequest) writeField7(oprot thrift.TProtocol) (err error) return err } -func (p *AggregateQueryRequest) writeField8(oprot thrift.TProtocol) (err error) { +func (p *AggregateQueryRawRequest) writeField8(oprot thrift.TProtocol) (err error) { if p.IsSetRangeType() { if err := oprot.WriteFieldBegin("rangeType", thrift.I32, 8); err != nil { return thrift.PrependError(fmt.Sprintf("%T write field begin error 8:rangeType: ", p), err) @@ -9115,33 +9047,33 @@ func (p *AggregateQueryRequest) writeField8(oprot thrift.TProtocol) (err error) return err } -func (p *AggregateQueryRequest) String() string { +func (p *AggregateQueryRawRequest) String() string { if p == nil { return "" } - return fmt.Sprintf("AggregateQueryRequest(%+v)", *p) + return fmt.Sprintf("AggregateQueryRawRequest(%+v)", *p) } // Attributes: // - Results // - Exhaustive -type AggregateQueryResult_ struct { - Results []*AggregateQueryResultTagNameElement `thrift:"results,1,required" db:"results" json:"results"` - Exhaustive bool `thrift:"exhaustive,2,required" db:"exhaustive" json:"exhaustive"` +type AggregateQueryRawResult_ struct { + Results []*AggregateQueryRawResultTagNameElement `thrift:"results,1,required" db:"results" json:"results"` + Exhaustive bool `thrift:"exhaustive,2,required" db:"exhaustive" json:"exhaustive"` } -func NewAggregateQueryResult_() *AggregateQueryResult_ { - return &AggregateQueryResult_{} +func NewAggregateQueryRawResult_() *AggregateQueryRawResult_ { + return &AggregateQueryRawResult_{} } -func (p *AggregateQueryResult_) GetResults() []*AggregateQueryResultTagNameElement { +func (p *AggregateQueryRawResult_) GetResults() []*AggregateQueryRawResultTagNameElement { return p.Results } -func (p *AggregateQueryResult_) GetExhaustive() bool { +func (p *AggregateQueryRawResult_) GetExhaustive() bool { return p.Exhaustive } -func (p *AggregateQueryResult_) Read(iprot thrift.TProtocol) error { +func (p *AggregateQueryRawResult_) Read(iprot thrift.TProtocol) error { if _, err := iprot.ReadStructBegin(); err != nil { return thrift.PrependError(fmt.Sprintf("%T read error: ", p), err) } @@ -9189,19 +9121,19 @@ func (p *AggregateQueryResult_) Read(iprot thrift.TProtocol) error { return nil } -func (p *AggregateQueryResult_) ReadField1(iprot thrift.TProtocol) error { +func (p *AggregateQueryRawResult_) ReadField1(iprot thrift.TProtocol) error { _, size, err := iprot.ReadListBegin() if err != nil { return thrift.PrependError("error reading list begin: ", err) } - tSlice := make([]*AggregateQueryResultTagNameElement, 0, size) + tSlice := make([]*AggregateQueryRawResultTagNameElement, 0, size) p.Results = tSlice for i := 0; i < size; i++ { - _elem20 := &AggregateQueryResultTagNameElement{} - if err := _elem20.Read(iprot); err != nil { - return thrift.PrependError(fmt.Sprintf("%T error reading struct: ", _elem20), err) + _elem21 := &AggregateQueryRawResultTagNameElement{} + if err := _elem21.Read(iprot); err != nil { + return thrift.PrependError(fmt.Sprintf("%T error reading struct: ", _elem21), err) } - p.Results = append(p.Results, _elem20) + p.Results = append(p.Results, _elem21) } if err := iprot.ReadListEnd(); err != nil { return thrift.PrependError("error reading list end: ", err) @@ -9209,7 +9141,7 @@ func (p *AggregateQueryResult_) ReadField1(iprot thrift.TProtocol) error { return nil } -func (p *AggregateQueryResult_) ReadField2(iprot thrift.TProtocol) error { +func (p *AggregateQueryRawResult_) ReadField2(iprot thrift.TProtocol) error { if v, err := iprot.ReadBool(); err != nil { return thrift.PrependError("error reading field 2: ", err) } else { @@ -9218,8 +9150,8 @@ func (p *AggregateQueryResult_) ReadField2(iprot thrift.TProtocol) error { return nil } -func (p *AggregateQueryResult_) Write(oprot thrift.TProtocol) error { - if err := oprot.WriteStructBegin("AggregateQueryResult"); err != nil { +func (p *AggregateQueryRawResult_) Write(oprot thrift.TProtocol) error { + if err := oprot.WriteStructBegin("AggregateQueryRawResult"); err != nil { return thrift.PrependError(fmt.Sprintf("%T write struct begin error: ", p), err) } if p != nil { @@ -9239,7 +9171,7 @@ func (p *AggregateQueryResult_) Write(oprot thrift.TProtocol) error { return nil } -func (p *AggregateQueryResult_) writeField1(oprot thrift.TProtocol) (err error) { +func (p *AggregateQueryRawResult_) writeField1(oprot thrift.TProtocol) (err error) { if err := oprot.WriteFieldBegin("results", thrift.LIST, 1); err != nil { return thrift.PrependError(fmt.Sprintf("%T write field begin error 1:results: ", p), err) } @@ -9260,7 +9192,7 @@ func (p *AggregateQueryResult_) writeField1(oprot thrift.TProtocol) (err error) return err } -func (p *AggregateQueryResult_) writeField2(oprot thrift.TProtocol) (err error) { +func (p *AggregateQueryRawResult_) writeField2(oprot thrift.TProtocol) (err error) { if err := oprot.WriteFieldBegin("exhaustive", thrift.BOOL, 2); err != nil { return thrift.PrependError(fmt.Sprintf("%T write field begin error 2:exhaustive: ", p), err) } @@ -9273,39 +9205,39 @@ func (p *AggregateQueryResult_) writeField2(oprot thrift.TProtocol) (err error) return err } -func (p *AggregateQueryResult_) String() string { +func (p *AggregateQueryRawResult_) String() string { if p == nil { return "" } - return fmt.Sprintf("AggregateQueryResult_(%+v)", *p) + return fmt.Sprintf("AggregateQueryRawResult_(%+v)", *p) } // Attributes: // - TagName // - TagValues -type AggregateQueryResultTagNameElement struct { - TagName string `thrift:"tagName,1,required" db:"tagName" json:"tagName"` - TagValues []*AggregateQueryResultTagValueElement `thrift:"tagValues,2" db:"tagValues" json:"tagValues,omitempty"` +type AggregateQueryRawResultTagNameElement struct { + TagName []byte `thrift:"tagName,1,required" db:"tagName" json:"tagName"` + TagValues []*AggregateQueryRawResultTagValueElement `thrift:"tagValues,2" db:"tagValues" json:"tagValues,omitempty"` } -func NewAggregateQueryResultTagNameElement() *AggregateQueryResultTagNameElement { - return &AggregateQueryResultTagNameElement{} +func NewAggregateQueryRawResultTagNameElement() *AggregateQueryRawResultTagNameElement { + return &AggregateQueryRawResultTagNameElement{} } -func (p *AggregateQueryResultTagNameElement) GetTagName() string { +func (p *AggregateQueryRawResultTagNameElement) GetTagName() []byte { return p.TagName } -var AggregateQueryResultTagNameElement_TagValues_DEFAULT []*AggregateQueryResultTagValueElement +var AggregateQueryRawResultTagNameElement_TagValues_DEFAULT []*AggregateQueryRawResultTagValueElement -func (p *AggregateQueryResultTagNameElement) GetTagValues() []*AggregateQueryResultTagValueElement { +func (p *AggregateQueryRawResultTagNameElement) GetTagValues() []*AggregateQueryRawResultTagValueElement { return p.TagValues } -func (p *AggregateQueryResultTagNameElement) IsSetTagValues() bool { +func (p *AggregateQueryRawResultTagNameElement) IsSetTagValues() bool { return p.TagValues != nil } -func (p *AggregateQueryResultTagNameElement) Read(iprot thrift.TProtocol) error { +func (p *AggregateQueryRawResultTagNameElement) Read(iprot thrift.TProtocol) error { if _, err := iprot.ReadStructBegin(); err != nil { return thrift.PrependError(fmt.Sprintf("%T read error: ", p), err) } @@ -9348,8 +9280,8 @@ func (p *AggregateQueryResultTagNameElement) Read(iprot thrift.TProtocol) error return nil } -func (p *AggregateQueryResultTagNameElement) ReadField1(iprot thrift.TProtocol) error { - if v, err := iprot.ReadString(); err != nil { +func (p *AggregateQueryRawResultTagNameElement) ReadField1(iprot thrift.TProtocol) error { + if v, err := iprot.ReadBinary(); err != nil { return thrift.PrependError("error reading field 1: ", err) } else { p.TagName = v @@ -9357,19 +9289,19 @@ func (p *AggregateQueryResultTagNameElement) ReadField1(iprot thrift.TProtocol) return nil } -func (p *AggregateQueryResultTagNameElement) ReadField2(iprot thrift.TProtocol) error { +func (p *AggregateQueryRawResultTagNameElement) ReadField2(iprot thrift.TProtocol) error { _, size, err := iprot.ReadListBegin() if err != nil { return thrift.PrependError("error reading list begin: ", err) } - tSlice := make([]*AggregateQueryResultTagValueElement, 0, size) + tSlice := make([]*AggregateQueryRawResultTagValueElement, 0, size) p.TagValues = tSlice for i := 0; i < size; i++ { - _elem21 := &AggregateQueryResultTagValueElement{} - if err := _elem21.Read(iprot); err != nil { - return thrift.PrependError(fmt.Sprintf("%T error reading struct: ", _elem21), err) + _elem22 := &AggregateQueryRawResultTagValueElement{} + if err := _elem22.Read(iprot); err != nil { + return thrift.PrependError(fmt.Sprintf("%T error reading struct: ", _elem22), err) } - p.TagValues = append(p.TagValues, _elem21) + p.TagValues = append(p.TagValues, _elem22) } if err := iprot.ReadListEnd(); err != nil { return thrift.PrependError("error reading list end: ", err) @@ -9377,8 +9309,8 @@ func (p *AggregateQueryResultTagNameElement) ReadField2(iprot thrift.TProtocol) return nil } -func (p *AggregateQueryResultTagNameElement) Write(oprot thrift.TProtocol) error { - if err := oprot.WriteStructBegin("AggregateQueryResultTagNameElement"); err != nil { +func (p *AggregateQueryRawResultTagNameElement) Write(oprot thrift.TProtocol) error { + if err := oprot.WriteStructBegin("AggregateQueryRawResultTagNameElement"); err != nil { return thrift.PrependError(fmt.Sprintf("%T write struct begin error: ", p), err) } if p != nil { @@ -9398,11 +9330,11 @@ func (p *AggregateQueryResultTagNameElement) Write(oprot thrift.TProtocol) error return nil } -func (p *AggregateQueryResultTagNameElement) writeField1(oprot thrift.TProtocol) (err error) { +func (p *AggregateQueryRawResultTagNameElement) writeField1(oprot thrift.TProtocol) (err error) { if err := oprot.WriteFieldBegin("tagName", thrift.STRING, 1); err != nil { return thrift.PrependError(fmt.Sprintf("%T write field begin error 1:tagName: ", p), err) } - if err := oprot.WriteString(string(p.TagName)); err != nil { + if err := oprot.WriteBinary(p.TagName); err != nil { return thrift.PrependError(fmt.Sprintf("%T.tagName (1) field write error: ", p), err) } if err := oprot.WriteFieldEnd(); err != nil { @@ -9411,7 +9343,7 @@ func (p *AggregateQueryResultTagNameElement) writeField1(oprot thrift.TProtocol) return err } -func (p *AggregateQueryResultTagNameElement) writeField2(oprot thrift.TProtocol) (err error) { +func (p *AggregateQueryRawResultTagNameElement) writeField2(oprot thrift.TProtocol) (err error) { if p.IsSetTagValues() { if err := oprot.WriteFieldBegin("tagValues", thrift.LIST, 2); err != nil { return thrift.PrependError(fmt.Sprintf("%T write field begin error 2:tagValues: ", p), err) @@ -9434,27 +9366,27 @@ func (p *AggregateQueryResultTagNameElement) writeField2(oprot thrift.TProtocol) return err } -func (p *AggregateQueryResultTagNameElement) String() string { +func (p *AggregateQueryRawResultTagNameElement) String() string { if p == nil { return "" } - return fmt.Sprintf("AggregateQueryResultTagNameElement(%+v)", *p) + return fmt.Sprintf("AggregateQueryRawResultTagNameElement(%+v)", *p) } // Attributes: // - TagValue -type AggregateQueryResultTagValueElement struct { - TagValue string `thrift:"tagValue,1,required" db:"tagValue" json:"tagValue"` +type AggregateQueryRawResultTagValueElement struct { + TagValue []byte `thrift:"tagValue,1,required" db:"tagValue" json:"tagValue"` } -func NewAggregateQueryResultTagValueElement() *AggregateQueryResultTagValueElement { - return &AggregateQueryResultTagValueElement{} +func NewAggregateQueryRawResultTagValueElement() *AggregateQueryRawResultTagValueElement { + return &AggregateQueryRawResultTagValueElement{} } -func (p *AggregateQueryResultTagValueElement) GetTagValue() string { +func (p *AggregateQueryRawResultTagValueElement) GetTagValue() []byte { return p.TagValue } -func (p *AggregateQueryResultTagValueElement) Read(iprot thrift.TProtocol) error { +func (p *AggregateQueryRawResultTagValueElement) Read(iprot thrift.TProtocol) error { if _, err := iprot.ReadStructBegin(); err != nil { return thrift.PrependError(fmt.Sprintf("%T read error: ", p), err) } @@ -9493,8 +9425,8 @@ func (p *AggregateQueryResultTagValueElement) Read(iprot thrift.TProtocol) error return nil } -func (p *AggregateQueryResultTagValueElement) ReadField1(iprot thrift.TProtocol) error { - if v, err := iprot.ReadString(); err != nil { +func (p *AggregateQueryRawResultTagValueElement) ReadField1(iprot thrift.TProtocol) error { + if v, err := iprot.ReadBinary(); err != nil { return thrift.PrependError("error reading field 1: ", err) } else { p.TagValue = v @@ -9502,8 +9434,8 @@ func (p *AggregateQueryResultTagValueElement) ReadField1(iprot thrift.TProtocol) return nil } -func (p *AggregateQueryResultTagValueElement) Write(oprot thrift.TProtocol) error { - if err := oprot.WriteStructBegin("AggregateQueryResultTagValueElement"); err != nil { +func (p *AggregateQueryRawResultTagValueElement) Write(oprot thrift.TProtocol) error { + if err := oprot.WriteStructBegin("AggregateQueryRawResultTagValueElement"); err != nil { return thrift.PrependError(fmt.Sprintf("%T write struct begin error: ", p), err) } if p != nil { @@ -9520,11 +9452,11 @@ func (p *AggregateQueryResultTagValueElement) Write(oprot thrift.TProtocol) erro return nil } -func (p *AggregateQueryResultTagValueElement) writeField1(oprot thrift.TProtocol) (err error) { +func (p *AggregateQueryRawResultTagValueElement) writeField1(oprot thrift.TProtocol) (err error) { if err := oprot.WriteFieldBegin("tagValue", thrift.STRING, 1); err != nil { return thrift.PrependError(fmt.Sprintf("%T write field begin error 1:tagValue: ", p), err) } - if err := oprot.WriteString(string(p.TagValue)); err != nil { + if err := oprot.WriteBinary(p.TagValue); err != nil { return thrift.PrependError(fmt.Sprintf("%T.tagValue (1) field write error: ", p), err) } if err := oprot.WriteFieldEnd(); err != nil { @@ -9533,11 +9465,11 @@ func (p *AggregateQueryResultTagValueElement) writeField1(oprot thrift.TProtocol return err } -func (p *AggregateQueryResultTagValueElement) String() string { +func (p *AggregateQueryRawResultTagValueElement) String() string { if p == nil { return "" } - return fmt.Sprintf("AggregateQueryResultTagValueElement(%+v)", *p) + return fmt.Sprintf("AggregateQueryRawResultTagValueElement(%+v)", *p) } // Attributes: @@ -9546,104 +9478,100 @@ func (p *AggregateQueryResultTagValueElement) String() string { // - RangeEnd // - NameSpace // - Limit -// - NoData +// - TagNameFilter +// - AggregateQueryType // - RangeType -// - ResultTimeType -type QueryRequest struct { - Query *Query `thrift:"query,1,required" db:"query" json:"query"` - RangeStart int64 `thrift:"rangeStart,2,required" db:"rangeStart" json:"rangeStart"` - RangeEnd int64 `thrift:"rangeEnd,3,required" db:"rangeEnd" json:"rangeEnd"` - NameSpace string `thrift:"nameSpace,4,required" db:"nameSpace" json:"nameSpace"` - Limit *int64 `thrift:"limit,5" db:"limit" json:"limit,omitempty"` - NoData *bool `thrift:"noData,6" db:"noData" json:"noData,omitempty"` - RangeType TimeType `thrift:"rangeType,7" db:"rangeType" json:"rangeType,omitempty"` - ResultTimeType TimeType `thrift:"resultTimeType,8" db:"resultTimeType" json:"resultTimeType,omitempty"` +type AggregateQueryRequest struct { + Query *Query `thrift:"query,1" db:"query" json:"query,omitempty"` + RangeStart int64 `thrift:"rangeStart,2,required" db:"rangeStart" json:"rangeStart"` + RangeEnd int64 `thrift:"rangeEnd,3,required" db:"rangeEnd" json:"rangeEnd"` + NameSpace string `thrift:"nameSpace,4,required" db:"nameSpace" json:"nameSpace"` + Limit *int64 `thrift:"limit,5" db:"limit" json:"limit,omitempty"` + TagNameFilter []string `thrift:"tagNameFilter,6" db:"tagNameFilter" json:"tagNameFilter,omitempty"` + AggregateQueryType AggregateQueryType `thrift:"aggregateQueryType,7" db:"aggregateQueryType" json:"aggregateQueryType,omitempty"` + RangeType TimeType `thrift:"rangeType,8" db:"rangeType" json:"rangeType,omitempty"` } -func NewQueryRequest() *QueryRequest { - return &QueryRequest{ - RangeType: 0, +func NewAggregateQueryRequest() *AggregateQueryRequest { + return &AggregateQueryRequest{ + AggregateQueryType: 1, - ResultTimeType: 0, + RangeType: 0, } } -var QueryRequest_Query_DEFAULT *Query +var AggregateQueryRequest_Query_DEFAULT *Query -func (p *QueryRequest) GetQuery() *Query { +func (p *AggregateQueryRequest) GetQuery() *Query { if !p.IsSetQuery() { - return QueryRequest_Query_DEFAULT + return AggregateQueryRequest_Query_DEFAULT } return p.Query } -func (p *QueryRequest) GetRangeStart() int64 { +func (p *AggregateQueryRequest) GetRangeStart() int64 { return p.RangeStart } -func (p *QueryRequest) GetRangeEnd() int64 { +func (p *AggregateQueryRequest) GetRangeEnd() int64 { return p.RangeEnd } -func (p *QueryRequest) GetNameSpace() string { +func (p *AggregateQueryRequest) GetNameSpace() string { return p.NameSpace } -var QueryRequest_Limit_DEFAULT int64 +var AggregateQueryRequest_Limit_DEFAULT int64 -func (p *QueryRequest) GetLimit() int64 { +func (p *AggregateQueryRequest) GetLimit() int64 { if !p.IsSetLimit() { - return QueryRequest_Limit_DEFAULT + return AggregateQueryRequest_Limit_DEFAULT } return *p.Limit } -var QueryRequest_NoData_DEFAULT bool +var AggregateQueryRequest_TagNameFilter_DEFAULT []string -func (p *QueryRequest) GetNoData() bool { - if !p.IsSetNoData() { - return QueryRequest_NoData_DEFAULT - } - return *p.NoData +func (p *AggregateQueryRequest) GetTagNameFilter() []string { + return p.TagNameFilter } -var QueryRequest_RangeType_DEFAULT TimeType = 0 +var AggregateQueryRequest_AggregateQueryType_DEFAULT AggregateQueryType = 1 -func (p *QueryRequest) GetRangeType() TimeType { - return p.RangeType +func (p *AggregateQueryRequest) GetAggregateQueryType() AggregateQueryType { + return p.AggregateQueryType } -var QueryRequest_ResultTimeType_DEFAULT TimeType = 0 +var AggregateQueryRequest_RangeType_DEFAULT TimeType = 0 -func (p *QueryRequest) GetResultTimeType() TimeType { - return p.ResultTimeType +func (p *AggregateQueryRequest) GetRangeType() TimeType { + return p.RangeType } -func (p *QueryRequest) IsSetQuery() bool { +func (p *AggregateQueryRequest) IsSetQuery() bool { return p.Query != nil } -func (p *QueryRequest) IsSetLimit() bool { +func (p *AggregateQueryRequest) IsSetLimit() bool { return p.Limit != nil } -func (p *QueryRequest) IsSetNoData() bool { - return p.NoData != nil +func (p *AggregateQueryRequest) IsSetTagNameFilter() bool { + return p.TagNameFilter != nil } -func (p *QueryRequest) IsSetRangeType() bool { - return p.RangeType != QueryRequest_RangeType_DEFAULT +func (p *AggregateQueryRequest) IsSetAggregateQueryType() bool { + return p.AggregateQueryType != AggregateQueryRequest_AggregateQueryType_DEFAULT } -func (p *QueryRequest) IsSetResultTimeType() bool { - return p.ResultTimeType != QueryRequest_ResultTimeType_DEFAULT +func (p *AggregateQueryRequest) IsSetRangeType() bool { + return p.RangeType != AggregateQueryRequest_RangeType_DEFAULT } -func (p *QueryRequest) Read(iprot thrift.TProtocol) error { +func (p *AggregateQueryRequest) Read(iprot thrift.TProtocol) error { if _, err := iprot.ReadStructBegin(); err != nil { return thrift.PrependError(fmt.Sprintf("%T read error: ", p), err) } - var issetQuery bool = false var issetRangeStart bool = false var issetRangeEnd bool = false var issetNameSpace bool = false @@ -9661,7 +9589,6 @@ func (p *QueryRequest) Read(iprot thrift.TProtocol) error { if err := p.ReadField1(iprot); err != nil { return err } - issetQuery = true case 2: if err := p.ReadField2(iprot); err != nil { return err @@ -9705,9 +9632,6 @@ func (p *QueryRequest) Read(iprot thrift.TProtocol) error { if err := iprot.ReadStructEnd(); err != nil { return thrift.PrependError(fmt.Sprintf("%T read struct end error: ", p), err) } - if !issetQuery { - return thrift.NewTProtocolExceptionWithType(thrift.INVALID_DATA, fmt.Errorf("Required field Query is not set")) - } if !issetRangeStart { return thrift.NewTProtocolExceptionWithType(thrift.INVALID_DATA, fmt.Errorf("Required field RangeStart is not set")) } @@ -9720,7 +9644,7 @@ func (p *QueryRequest) Read(iprot thrift.TProtocol) error { return nil } -func (p *QueryRequest) ReadField1(iprot thrift.TProtocol) error { +func (p *AggregateQueryRequest) ReadField1(iprot thrift.TProtocol) error { p.Query = &Query{} if err := p.Query.Read(iprot); err != nil { return thrift.PrependError(fmt.Sprintf("%T error reading struct: ", p.Query), err) @@ -9728,7 +9652,7 @@ func (p *QueryRequest) ReadField1(iprot thrift.TProtocol) error { return nil } -func (p *QueryRequest) ReadField2(iprot thrift.TProtocol) error { +func (p *AggregateQueryRequest) ReadField2(iprot thrift.TProtocol) error { if v, err := iprot.ReadI64(); err != nil { return thrift.PrependError("error reading field 2: ", err) } else { @@ -9737,7 +9661,7 @@ func (p *QueryRequest) ReadField2(iprot thrift.TProtocol) error { return nil } -func (p *QueryRequest) ReadField3(iprot thrift.TProtocol) error { +func (p *AggregateQueryRequest) ReadField3(iprot thrift.TProtocol) error { if v, err := iprot.ReadI64(); err != nil { return thrift.PrependError("error reading field 3: ", err) } else { @@ -9746,7 +9670,7 @@ func (p *QueryRequest) ReadField3(iprot thrift.TProtocol) error { return nil } -func (p *QueryRequest) ReadField4(iprot thrift.TProtocol) error { +func (p *AggregateQueryRequest) ReadField4(iprot thrift.TProtocol) error { if v, err := iprot.ReadString(); err != nil { return thrift.PrependError("error reading field 4: ", err) } else { @@ -9755,7 +9679,7 @@ func (p *QueryRequest) ReadField4(iprot thrift.TProtocol) error { return nil } -func (p *QueryRequest) ReadField5(iprot thrift.TProtocol) error { +func (p *AggregateQueryRequest) ReadField5(iprot thrift.TProtocol) error { if v, err := iprot.ReadI64(); err != nil { return thrift.PrependError("error reading field 5: ", err) } else { @@ -9764,37 +9688,50 @@ func (p *QueryRequest) ReadField5(iprot thrift.TProtocol) error { return nil } -func (p *QueryRequest) ReadField6(iprot thrift.TProtocol) error { - if v, err := iprot.ReadBool(); err != nil { - return thrift.PrependError("error reading field 6: ", err) - } else { - p.NoData = &v +func (p *AggregateQueryRequest) ReadField6(iprot thrift.TProtocol) error { + _, size, err := iprot.ReadListBegin() + if err != nil { + return thrift.PrependError("error reading list begin: ", err) + } + tSlice := make([]string, 0, size) + p.TagNameFilter = tSlice + for i := 0; i < size; i++ { + var _elem23 string + if v, err := iprot.ReadString(); err != nil { + return thrift.PrependError("error reading field 0: ", err) + } else { + _elem23 = v + } + p.TagNameFilter = append(p.TagNameFilter, _elem23) + } + if err := iprot.ReadListEnd(); err != nil { + return thrift.PrependError("error reading list end: ", err) } return nil } -func (p *QueryRequest) ReadField7(iprot thrift.TProtocol) error { +func (p *AggregateQueryRequest) ReadField7(iprot thrift.TProtocol) error { if v, err := iprot.ReadI32(); err != nil { return thrift.PrependError("error reading field 7: ", err) } else { - temp := TimeType(v) - p.RangeType = temp + temp := AggregateQueryType(v) + p.AggregateQueryType = temp } return nil } -func (p *QueryRequest) ReadField8(iprot thrift.TProtocol) error { +func (p *AggregateQueryRequest) ReadField8(iprot thrift.TProtocol) error { if v, err := iprot.ReadI32(); err != nil { return thrift.PrependError("error reading field 8: ", err) } else { temp := TimeType(v) - p.ResultTimeType = temp + p.RangeType = temp } return nil } -func (p *QueryRequest) Write(oprot thrift.TProtocol) error { - if err := oprot.WriteStructBegin("QueryRequest"); err != nil { +func (p *AggregateQueryRequest) Write(oprot thrift.TProtocol) error { + if err := oprot.WriteStructBegin("AggregateQueryRequest"); err != nil { return thrift.PrependError(fmt.Sprintf("%T write struct begin error: ", p), err) } if p != nil { @@ -9832,20 +9769,22 @@ func (p *QueryRequest) Write(oprot thrift.TProtocol) error { return nil } -func (p *QueryRequest) writeField1(oprot thrift.TProtocol) (err error) { - if err := oprot.WriteFieldBegin("query", thrift.STRUCT, 1); err != nil { - return thrift.PrependError(fmt.Sprintf("%T write field begin error 1:query: ", p), err) - } - if err := p.Query.Write(oprot); err != nil { - return thrift.PrependError(fmt.Sprintf("%T error writing struct: ", p.Query), err) - } - if err := oprot.WriteFieldEnd(); err != nil { - return thrift.PrependError(fmt.Sprintf("%T write field end error 1:query: ", p), err) +func (p *AggregateQueryRequest) writeField1(oprot thrift.TProtocol) (err error) { + if p.IsSetQuery() { + if err := oprot.WriteFieldBegin("query", thrift.STRUCT, 1); err != nil { + return thrift.PrependError(fmt.Sprintf("%T write field begin error 1:query: ", p), err) + } + if err := p.Query.Write(oprot); err != nil { + return thrift.PrependError(fmt.Sprintf("%T error writing struct: ", p.Query), err) + } + if err := oprot.WriteFieldEnd(); err != nil { + return thrift.PrependError(fmt.Sprintf("%T write field end error 1:query: ", p), err) + } } return err } -func (p *QueryRequest) writeField2(oprot thrift.TProtocol) (err error) { +func (p *AggregateQueryRequest) writeField2(oprot thrift.TProtocol) (err error) { if err := oprot.WriteFieldBegin("rangeStart", thrift.I64, 2); err != nil { return thrift.PrependError(fmt.Sprintf("%T write field begin error 2:rangeStart: ", p), err) } @@ -9858,7 +9797,7 @@ func (p *QueryRequest) writeField2(oprot thrift.TProtocol) (err error) { return err } -func (p *QueryRequest) writeField3(oprot thrift.TProtocol) (err error) { +func (p *AggregateQueryRequest) writeField3(oprot thrift.TProtocol) (err error) { if err := oprot.WriteFieldBegin("rangeEnd", thrift.I64, 3); err != nil { return thrift.PrependError(fmt.Sprintf("%T write field begin error 3:rangeEnd: ", p), err) } @@ -9871,7 +9810,7 @@ func (p *QueryRequest) writeField3(oprot thrift.TProtocol) (err error) { return err } -func (p *QueryRequest) writeField4(oprot thrift.TProtocol) (err error) { +func (p *AggregateQueryRequest) writeField4(oprot thrift.TProtocol) (err error) { if err := oprot.WriteFieldBegin("nameSpace", thrift.STRING, 4); err != nil { return thrift.PrependError(fmt.Sprintf("%T write field begin error 4:nameSpace: ", p), err) } @@ -9884,7 +9823,7 @@ func (p *QueryRequest) writeField4(oprot thrift.TProtocol) (err error) { return err } -func (p *QueryRequest) writeField5(oprot thrift.TProtocol) (err error) { +func (p *AggregateQueryRequest) writeField5(oprot thrift.TProtocol) (err error) { if p.IsSetLimit() { if err := oprot.WriteFieldBegin("limit", thrift.I64, 5); err != nil { return thrift.PrependError(fmt.Sprintf("%T write field begin error 5:limit: ", p), err) @@ -9899,78 +9838,86 @@ func (p *QueryRequest) writeField5(oprot thrift.TProtocol) (err error) { return err } -func (p *QueryRequest) writeField6(oprot thrift.TProtocol) (err error) { - if p.IsSetNoData() { - if err := oprot.WriteFieldBegin("noData", thrift.BOOL, 6); err != nil { - return thrift.PrependError(fmt.Sprintf("%T write field begin error 6:noData: ", p), err) +func (p *AggregateQueryRequest) writeField6(oprot thrift.TProtocol) (err error) { + if p.IsSetTagNameFilter() { + if err := oprot.WriteFieldBegin("tagNameFilter", thrift.LIST, 6); err != nil { + return thrift.PrependError(fmt.Sprintf("%T write field begin error 6:tagNameFilter: ", p), err) } - if err := oprot.WriteBool(bool(*p.NoData)); err != nil { - return thrift.PrependError(fmt.Sprintf("%T.noData (6) field write error: ", p), err) + if err := oprot.WriteListBegin(thrift.STRING, len(p.TagNameFilter)); err != nil { + return thrift.PrependError("error writing list begin: ", err) + } + for _, v := range p.TagNameFilter { + if err := oprot.WriteString(string(v)); err != nil { + return thrift.PrependError(fmt.Sprintf("%T. (0) field write error: ", p), err) + } + } + if err := oprot.WriteListEnd(); err != nil { + return thrift.PrependError("error writing list end: ", err) } if err := oprot.WriteFieldEnd(); err != nil { - return thrift.PrependError(fmt.Sprintf("%T write field end error 6:noData: ", p), err) + return thrift.PrependError(fmt.Sprintf("%T write field end error 6:tagNameFilter: ", p), err) } } return err } -func (p *QueryRequest) writeField7(oprot thrift.TProtocol) (err error) { - if p.IsSetRangeType() { - if err := oprot.WriteFieldBegin("rangeType", thrift.I32, 7); err != nil { - return thrift.PrependError(fmt.Sprintf("%T write field begin error 7:rangeType: ", p), err) +func (p *AggregateQueryRequest) writeField7(oprot thrift.TProtocol) (err error) { + if p.IsSetAggregateQueryType() { + if err := oprot.WriteFieldBegin("aggregateQueryType", thrift.I32, 7); err != nil { + return thrift.PrependError(fmt.Sprintf("%T write field begin error 7:aggregateQueryType: ", p), err) } - if err := oprot.WriteI32(int32(p.RangeType)); err != nil { - return thrift.PrependError(fmt.Sprintf("%T.rangeType (7) field write error: ", p), err) + if err := oprot.WriteI32(int32(p.AggregateQueryType)); err != nil { + return thrift.PrependError(fmt.Sprintf("%T.aggregateQueryType (7) field write error: ", p), err) } if err := oprot.WriteFieldEnd(); err != nil { - return thrift.PrependError(fmt.Sprintf("%T write field end error 7:rangeType: ", p), err) + return thrift.PrependError(fmt.Sprintf("%T write field end error 7:aggregateQueryType: ", p), err) } } return err } -func (p *QueryRequest) writeField8(oprot thrift.TProtocol) (err error) { - if p.IsSetResultTimeType() { - if err := oprot.WriteFieldBegin("resultTimeType", thrift.I32, 8); err != nil { - return thrift.PrependError(fmt.Sprintf("%T write field begin error 8:resultTimeType: ", p), err) +func (p *AggregateQueryRequest) writeField8(oprot thrift.TProtocol) (err error) { + if p.IsSetRangeType() { + if err := oprot.WriteFieldBegin("rangeType", thrift.I32, 8); err != nil { + return thrift.PrependError(fmt.Sprintf("%T write field begin error 8:rangeType: ", p), err) } - if err := oprot.WriteI32(int32(p.ResultTimeType)); err != nil { - return thrift.PrependError(fmt.Sprintf("%T.resultTimeType (8) field write error: ", p), err) + if err := oprot.WriteI32(int32(p.RangeType)); err != nil { + return thrift.PrependError(fmt.Sprintf("%T.rangeType (8) field write error: ", p), err) } if err := oprot.WriteFieldEnd(); err != nil { - return thrift.PrependError(fmt.Sprintf("%T write field end error 8:resultTimeType: ", p), err) + return thrift.PrependError(fmt.Sprintf("%T write field end error 8:rangeType: ", p), err) } } return err } -func (p *QueryRequest) String() string { +func (p *AggregateQueryRequest) String() string { if p == nil { return "" } - return fmt.Sprintf("QueryRequest(%+v)", *p) + return fmt.Sprintf("AggregateQueryRequest(%+v)", *p) } // Attributes: // - Results // - Exhaustive -type QueryResult_ struct { - Results []*QueryResultElement `thrift:"results,1,required" db:"results" json:"results"` - Exhaustive bool `thrift:"exhaustive,2,required" db:"exhaustive" json:"exhaustive"` +type AggregateQueryResult_ struct { + Results []*AggregateQueryResultTagNameElement `thrift:"results,1,required" db:"results" json:"results"` + Exhaustive bool `thrift:"exhaustive,2,required" db:"exhaustive" json:"exhaustive"` } -func NewQueryResult_() *QueryResult_ { - return &QueryResult_{} +func NewAggregateQueryResult_() *AggregateQueryResult_ { + return &AggregateQueryResult_{} } -func (p *QueryResult_) GetResults() []*QueryResultElement { +func (p *AggregateQueryResult_) GetResults() []*AggregateQueryResultTagNameElement { return p.Results } -func (p *QueryResult_) GetExhaustive() bool { +func (p *AggregateQueryResult_) GetExhaustive() bool { return p.Exhaustive } -func (p *QueryResult_) Read(iprot thrift.TProtocol) error { +func (p *AggregateQueryResult_) Read(iprot thrift.TProtocol) error { if _, err := iprot.ReadStructBegin(); err != nil { return thrift.PrependError(fmt.Sprintf("%T read error: ", p), err) } @@ -10018,19 +9965,19 @@ func (p *QueryResult_) Read(iprot thrift.TProtocol) error { return nil } -func (p *QueryResult_) ReadField1(iprot thrift.TProtocol) error { +func (p *AggregateQueryResult_) ReadField1(iprot thrift.TProtocol) error { _, size, err := iprot.ReadListBegin() if err != nil { return thrift.PrependError("error reading list begin: ", err) } - tSlice := make([]*QueryResultElement, 0, size) + tSlice := make([]*AggregateQueryResultTagNameElement, 0, size) p.Results = tSlice for i := 0; i < size; i++ { - _elem22 := &QueryResultElement{} - if err := _elem22.Read(iprot); err != nil { - return thrift.PrependError(fmt.Sprintf("%T error reading struct: ", _elem22), err) + _elem24 := &AggregateQueryResultTagNameElement{} + if err := _elem24.Read(iprot); err != nil { + return thrift.PrependError(fmt.Sprintf("%T error reading struct: ", _elem24), err) } - p.Results = append(p.Results, _elem22) + p.Results = append(p.Results, _elem24) } if err := iprot.ReadListEnd(); err != nil { return thrift.PrependError("error reading list end: ", err) @@ -10038,7 +9985,7 @@ func (p *QueryResult_) ReadField1(iprot thrift.TProtocol) error { return nil } -func (p *QueryResult_) ReadField2(iprot thrift.TProtocol) error { +func (p *AggregateQueryResult_) ReadField2(iprot thrift.TProtocol) error { if v, err := iprot.ReadBool(); err != nil { return thrift.PrependError("error reading field 2: ", err) } else { @@ -10047,8 +9994,8 @@ func (p *QueryResult_) ReadField2(iprot thrift.TProtocol) error { return nil } -func (p *QueryResult_) Write(oprot thrift.TProtocol) error { - if err := oprot.WriteStructBegin("QueryResult"); err != nil { +func (p *AggregateQueryResult_) Write(oprot thrift.TProtocol) error { + if err := oprot.WriteStructBegin("AggregateQueryResult"); err != nil { return thrift.PrependError(fmt.Sprintf("%T write struct begin error: ", p), err) } if p != nil { @@ -10068,7 +10015,7 @@ func (p *QueryResult_) Write(oprot thrift.TProtocol) error { return nil } -func (p *QueryResult_) writeField1(oprot thrift.TProtocol) (err error) { +func (p *AggregateQueryResult_) writeField1(oprot thrift.TProtocol) (err error) { if err := oprot.WriteFieldBegin("results", thrift.LIST, 1); err != nil { return thrift.PrependError(fmt.Sprintf("%T write field begin error 1:results: ", p), err) } @@ -10089,7 +10036,7 @@ func (p *QueryResult_) writeField1(oprot thrift.TProtocol) (err error) { return err } -func (p *QueryResult_) writeField2(oprot thrift.TProtocol) (err error) { +func (p *AggregateQueryResult_) writeField2(oprot thrift.TProtocol) (err error) { if err := oprot.WriteFieldBegin("exhaustive", thrift.BOOL, 2); err != nil { return thrift.PrependError(fmt.Sprintf("%T write field begin error 2:exhaustive: ", p), err) } @@ -10102,46 +10049,44 @@ func (p *QueryResult_) writeField2(oprot thrift.TProtocol) (err error) { return err } -func (p *QueryResult_) String() string { +func (p *AggregateQueryResult_) String() string { if p == nil { return "" } - return fmt.Sprintf("QueryResult_(%+v)", *p) + return fmt.Sprintf("AggregateQueryResult_(%+v)", *p) } // Attributes: -// - ID -// - Tags -// - Datapoints -type QueryResultElement struct { - ID string `thrift:"id,1,required" db:"id" json:"id"` - Tags []*Tag `thrift:"tags,2,required" db:"tags" json:"tags"` - Datapoints []*Datapoint `thrift:"datapoints,3,required" db:"datapoints" json:"datapoints"` -} - -func NewQueryResultElement() *QueryResultElement { - return &QueryResultElement{} +// - TagName +// - TagValues +type AggregateQueryResultTagNameElement struct { + TagName string `thrift:"tagName,1,required" db:"tagName" json:"tagName"` + TagValues []*AggregateQueryResultTagValueElement `thrift:"tagValues,2" db:"tagValues" json:"tagValues,omitempty"` } -func (p *QueryResultElement) GetID() string { - return p.ID +func NewAggregateQueryResultTagNameElement() *AggregateQueryResultTagNameElement { + return &AggregateQueryResultTagNameElement{} } -func (p *QueryResultElement) GetTags() []*Tag { - return p.Tags +func (p *AggregateQueryResultTagNameElement) GetTagName() string { + return p.TagName } -func (p *QueryResultElement) GetDatapoints() []*Datapoint { - return p.Datapoints +var AggregateQueryResultTagNameElement_TagValues_DEFAULT []*AggregateQueryResultTagValueElement + +func (p *AggregateQueryResultTagNameElement) GetTagValues() []*AggregateQueryResultTagValueElement { + return p.TagValues } -func (p *QueryResultElement) Read(iprot thrift.TProtocol) error { +func (p *AggregateQueryResultTagNameElement) IsSetTagValues() bool { + return p.TagValues != nil +} + +func (p *AggregateQueryResultTagNameElement) Read(iprot thrift.TProtocol) error { if _, err := iprot.ReadStructBegin(); err != nil { return thrift.PrependError(fmt.Sprintf("%T read error: ", p), err) } - var issetID bool = false - var issetTags bool = false - var issetDatapoints bool = false + var issetTagName bool = false for { _, fieldTypeId, fieldId, err := iprot.ReadFieldBegin() @@ -10156,17 +10101,11 @@ func (p *QueryResultElement) Read(iprot thrift.TProtocol) error { if err := p.ReadField1(iprot); err != nil { return err } - issetID = true + issetTagName = true case 2: if err := p.ReadField2(iprot); err != nil { return err } - issetTags = true - case 3: - if err := p.ReadField3(iprot); err != nil { - return err - } - issetDatapoints = true default: if err := iprot.Skip(fieldTypeId); err != nil { return err @@ -10179,62 +10118,34 @@ func (p *QueryResultElement) Read(iprot thrift.TProtocol) error { if err := iprot.ReadStructEnd(); err != nil { return thrift.PrependError(fmt.Sprintf("%T read struct end error: ", p), err) } - if !issetID { - return thrift.NewTProtocolExceptionWithType(thrift.INVALID_DATA, fmt.Errorf("Required field ID is not set")) - } - if !issetTags { - return thrift.NewTProtocolExceptionWithType(thrift.INVALID_DATA, fmt.Errorf("Required field Tags is not set")) - } - if !issetDatapoints { - return thrift.NewTProtocolExceptionWithType(thrift.INVALID_DATA, fmt.Errorf("Required field Datapoints is not set")) + if !issetTagName { + return thrift.NewTProtocolExceptionWithType(thrift.INVALID_DATA, fmt.Errorf("Required field TagName is not set")) } return nil } -func (p *QueryResultElement) ReadField1(iprot thrift.TProtocol) error { +func (p *AggregateQueryResultTagNameElement) ReadField1(iprot thrift.TProtocol) error { if v, err := iprot.ReadString(); err != nil { return thrift.PrependError("error reading field 1: ", err) } else { - p.ID = v - } - return nil -} - -func (p *QueryResultElement) ReadField2(iprot thrift.TProtocol) error { - _, size, err := iprot.ReadListBegin() - if err != nil { - return thrift.PrependError("error reading list begin: ", err) - } - tSlice := make([]*Tag, 0, size) - p.Tags = tSlice - for i := 0; i < size; i++ { - _elem23 := &Tag{} - if err := _elem23.Read(iprot); err != nil { - return thrift.PrependError(fmt.Sprintf("%T error reading struct: ", _elem23), err) - } - p.Tags = append(p.Tags, _elem23) - } - if err := iprot.ReadListEnd(); err != nil { - return thrift.PrependError("error reading list end: ", err) + p.TagName = v } return nil } -func (p *QueryResultElement) ReadField3(iprot thrift.TProtocol) error { +func (p *AggregateQueryResultTagNameElement) ReadField2(iprot thrift.TProtocol) error { _, size, err := iprot.ReadListBegin() if err != nil { return thrift.PrependError("error reading list begin: ", err) } - tSlice := make([]*Datapoint, 0, size) - p.Datapoints = tSlice + tSlice := make([]*AggregateQueryResultTagValueElement, 0, size) + p.TagValues = tSlice for i := 0; i < size; i++ { - _elem24 := &Datapoint{ - TimestampTimeType: 0, - } - if err := _elem24.Read(iprot); err != nil { - return thrift.PrependError(fmt.Sprintf("%T error reading struct: ", _elem24), err) + _elem25 := &AggregateQueryResultTagValueElement{} + if err := _elem25.Read(iprot); err != nil { + return thrift.PrependError(fmt.Sprintf("%T error reading struct: ", _elem25), err) } - p.Datapoints = append(p.Datapoints, _elem24) + p.TagValues = append(p.TagValues, _elem25) } if err := iprot.ReadListEnd(); err != nil { return thrift.PrependError("error reading list end: ", err) @@ -10242,8 +10153,8 @@ func (p *QueryResultElement) ReadField3(iprot thrift.TProtocol) error { return nil } -func (p *QueryResultElement) Write(oprot thrift.TProtocol) error { - if err := oprot.WriteStructBegin("QueryResultElement"); err != nil { +func (p *AggregateQueryResultTagNameElement) Write(oprot thrift.TProtocol) error { + if err := oprot.WriteStructBegin("AggregateQueryResultTagNameElement"); err != nil { return thrift.PrependError(fmt.Sprintf("%T write struct begin error: ", p), err) } if p != nil { @@ -10253,9 +10164,6 @@ func (p *QueryResultElement) Write(oprot thrift.TProtocol) error { if err := p.writeField2(oprot); err != nil { return err } - if err := p.writeField3(oprot); err != nil { - return err - } } if err := oprot.WriteFieldStop(); err != nil { return thrift.PrependError("write field stop error: ", err) @@ -10266,94 +10174,68 @@ func (p *QueryResultElement) Write(oprot thrift.TProtocol) error { return nil } -func (p *QueryResultElement) writeField1(oprot thrift.TProtocol) (err error) { - if err := oprot.WriteFieldBegin("id", thrift.STRING, 1); err != nil { - return thrift.PrependError(fmt.Sprintf("%T write field begin error 1:id: ", p), err) +func (p *AggregateQueryResultTagNameElement) writeField1(oprot thrift.TProtocol) (err error) { + if err := oprot.WriteFieldBegin("tagName", thrift.STRING, 1); err != nil { + return thrift.PrependError(fmt.Sprintf("%T write field begin error 1:tagName: ", p), err) } - if err := oprot.WriteString(string(p.ID)); err != nil { - return thrift.PrependError(fmt.Sprintf("%T.id (1) field write error: ", p), err) + if err := oprot.WriteString(string(p.TagName)); err != nil { + return thrift.PrependError(fmt.Sprintf("%T.tagName (1) field write error: ", p), err) } if err := oprot.WriteFieldEnd(); err != nil { - return thrift.PrependError(fmt.Sprintf("%T write field end error 1:id: ", p), err) + return thrift.PrependError(fmt.Sprintf("%T write field end error 1:tagName: ", p), err) } return err } -func (p *QueryResultElement) writeField2(oprot thrift.TProtocol) (err error) { - if err := oprot.WriteFieldBegin("tags", thrift.LIST, 2); err != nil { - return thrift.PrependError(fmt.Sprintf("%T write field begin error 2:tags: ", p), err) - } - if err := oprot.WriteListBegin(thrift.STRUCT, len(p.Tags)); err != nil { - return thrift.PrependError("error writing list begin: ", err) - } - for _, v := range p.Tags { - if err := v.Write(oprot); err != nil { - return thrift.PrependError(fmt.Sprintf("%T error writing struct: ", v), err) +func (p *AggregateQueryResultTagNameElement) writeField2(oprot thrift.TProtocol) (err error) { + if p.IsSetTagValues() { + if err := oprot.WriteFieldBegin("tagValues", thrift.LIST, 2); err != nil { + return thrift.PrependError(fmt.Sprintf("%T write field begin error 2:tagValues: ", p), err) } - } - if err := oprot.WriteListEnd(); err != nil { - return thrift.PrependError("error writing list end: ", err) - } - if err := oprot.WriteFieldEnd(); err != nil { - return thrift.PrependError(fmt.Sprintf("%T write field end error 2:tags: ", p), err) - } - return err -} - -func (p *QueryResultElement) writeField3(oprot thrift.TProtocol) (err error) { - if err := oprot.WriteFieldBegin("datapoints", thrift.LIST, 3); err != nil { - return thrift.PrependError(fmt.Sprintf("%T write field begin error 3:datapoints: ", p), err) - } - if err := oprot.WriteListBegin(thrift.STRUCT, len(p.Datapoints)); err != nil { - return thrift.PrependError("error writing list begin: ", err) - } - for _, v := range p.Datapoints { - if err := v.Write(oprot); err != nil { - return thrift.PrependError(fmt.Sprintf("%T error writing struct: ", v), err) + if err := oprot.WriteListBegin(thrift.STRUCT, len(p.TagValues)); err != nil { + return thrift.PrependError("error writing list begin: ", err) + } + for _, v := range p.TagValues { + if err := v.Write(oprot); err != nil { + return thrift.PrependError(fmt.Sprintf("%T error writing struct: ", v), err) + } + } + if err := oprot.WriteListEnd(); err != nil { + return thrift.PrependError("error writing list end: ", err) + } + if err := oprot.WriteFieldEnd(); err != nil { + return thrift.PrependError(fmt.Sprintf("%T write field end error 2:tagValues: ", p), err) } - } - if err := oprot.WriteListEnd(); err != nil { - return thrift.PrependError("error writing list end: ", err) - } - if err := oprot.WriteFieldEnd(); err != nil { - return thrift.PrependError(fmt.Sprintf("%T write field end error 3:datapoints: ", p), err) } return err } -func (p *QueryResultElement) String() string { +func (p *AggregateQueryResultTagNameElement) String() string { if p == nil { return "" } - return fmt.Sprintf("QueryResultElement(%+v)", *p) + return fmt.Sprintf("AggregateQueryResultTagNameElement(%+v)", *p) } // Attributes: -// - Field -// - Term -type TermQuery struct { - Field string `thrift:"field,1,required" db:"field" json:"field"` - Term string `thrift:"term,2,required" db:"term" json:"term"` -} - -func NewTermQuery() *TermQuery { - return &TermQuery{} +// - TagValue +type AggregateQueryResultTagValueElement struct { + TagValue string `thrift:"tagValue,1,required" db:"tagValue" json:"tagValue"` } -func (p *TermQuery) GetField() string { - return p.Field +func NewAggregateQueryResultTagValueElement() *AggregateQueryResultTagValueElement { + return &AggregateQueryResultTagValueElement{} } -func (p *TermQuery) GetTerm() string { - return p.Term +func (p *AggregateQueryResultTagValueElement) GetTagValue() string { + return p.TagValue } -func (p *TermQuery) Read(iprot thrift.TProtocol) error { +func (p *AggregateQueryResultTagValueElement) Read(iprot thrift.TProtocol) error { if _, err := iprot.ReadStructBegin(); err != nil { return thrift.PrependError(fmt.Sprintf("%T read error: ", p), err) } - var issetField bool = false - var issetTerm bool = false + var issetTagValue bool = false for { _, fieldTypeId, fieldId, err := iprot.ReadFieldBegin() @@ -10368,12 +10250,7 @@ func (p *TermQuery) Read(iprot thrift.TProtocol) error { if err := p.ReadField1(iprot); err != nil { return err } - issetField = true - case 2: - if err := p.ReadField2(iprot); err != nil { - return err - } - issetTerm = true + issetTagValue = true default: if err := iprot.Skip(fieldTypeId); err != nil { return err @@ -10386,44 +10263,29 @@ func (p *TermQuery) Read(iprot thrift.TProtocol) error { if err := iprot.ReadStructEnd(); err != nil { return thrift.PrependError(fmt.Sprintf("%T read struct end error: ", p), err) } - if !issetField { - return thrift.NewTProtocolExceptionWithType(thrift.INVALID_DATA, fmt.Errorf("Required field Field is not set")) - } - if !issetTerm { - return thrift.NewTProtocolExceptionWithType(thrift.INVALID_DATA, fmt.Errorf("Required field Term is not set")) + if !issetTagValue { + return thrift.NewTProtocolExceptionWithType(thrift.INVALID_DATA, fmt.Errorf("Required field TagValue is not set")) } return nil } -func (p *TermQuery) ReadField1(iprot thrift.TProtocol) error { +func (p *AggregateQueryResultTagValueElement) ReadField1(iprot thrift.TProtocol) error { if v, err := iprot.ReadString(); err != nil { return thrift.PrependError("error reading field 1: ", err) } else { - p.Field = v - } - return nil -} - -func (p *TermQuery) ReadField2(iprot thrift.TProtocol) error { - if v, err := iprot.ReadString(); err != nil { - return thrift.PrependError("error reading field 2: ", err) - } else { - p.Term = v + p.TagValue = v } return nil } -func (p *TermQuery) Write(oprot thrift.TProtocol) error { - if err := oprot.WriteStructBegin("TermQuery"); err != nil { +func (p *AggregateQueryResultTagValueElement) Write(oprot thrift.TProtocol) error { + if err := oprot.WriteStructBegin("AggregateQueryResultTagValueElement"); err != nil { return thrift.PrependError(fmt.Sprintf("%T write struct begin error: ", p), err) } if p != nil { if err := p.writeField1(oprot); err != nil { return err } - if err := p.writeField2(oprot); err != nil { - return err - } } if err := oprot.WriteFieldStop(); err != nil { return thrift.PrependError("write field stop error: ", err) @@ -10434,206 +10296,133 @@ func (p *TermQuery) Write(oprot thrift.TProtocol) error { return nil } -func (p *TermQuery) writeField1(oprot thrift.TProtocol) (err error) { - if err := oprot.WriteFieldBegin("field", thrift.STRING, 1); err != nil { - return thrift.PrependError(fmt.Sprintf("%T write field begin error 1:field: ", p), err) - } - if err := oprot.WriteString(string(p.Field)); err != nil { - return thrift.PrependError(fmt.Sprintf("%T.field (1) field write error: ", p), err) - } - if err := oprot.WriteFieldEnd(); err != nil { - return thrift.PrependError(fmt.Sprintf("%T write field end error 1:field: ", p), err) - } - return err -} - -func (p *TermQuery) writeField2(oprot thrift.TProtocol) (err error) { - if err := oprot.WriteFieldBegin("term", thrift.STRING, 2); err != nil { - return thrift.PrependError(fmt.Sprintf("%T write field begin error 2:term: ", p), err) +func (p *AggregateQueryResultTagValueElement) writeField1(oprot thrift.TProtocol) (err error) { + if err := oprot.WriteFieldBegin("tagValue", thrift.STRING, 1); err != nil { + return thrift.PrependError(fmt.Sprintf("%T write field begin error 1:tagValue: ", p), err) } - if err := oprot.WriteString(string(p.Term)); err != nil { - return thrift.PrependError(fmt.Sprintf("%T.term (2) field write error: ", p), err) + if err := oprot.WriteString(string(p.TagValue)); err != nil { + return thrift.PrependError(fmt.Sprintf("%T.tagValue (1) field write error: ", p), err) } if err := oprot.WriteFieldEnd(); err != nil { - return thrift.PrependError(fmt.Sprintf("%T write field end error 2:term: ", p), err) + return thrift.PrependError(fmt.Sprintf("%T write field end error 1:tagValue: ", p), err) } return err } -func (p *TermQuery) String() string { +func (p *AggregateQueryResultTagValueElement) String() string { if p == nil { return "" } - return fmt.Sprintf("TermQuery(%+v)", *p) + return fmt.Sprintf("AggregateQueryResultTagValueElement(%+v)", *p) } // Attributes: -// - Field -// - Regexp -type RegexpQuery struct { - Field string `thrift:"field,1,required" db:"field" json:"field"` - Regexp string `thrift:"regexp,2,required" db:"regexp" json:"regexp"` -} - -func NewRegexpQuery() *RegexpQuery { - return &RegexpQuery{} -} - -func (p *RegexpQuery) GetField() string { - return p.Field +// - Query +// - RangeStart +// - RangeEnd +// - NameSpace +// - Limit +// - NoData +// - RangeType +// - ResultTimeType +type QueryRequest struct { + Query *Query `thrift:"query,1,required" db:"query" json:"query"` + RangeStart int64 `thrift:"rangeStart,2,required" db:"rangeStart" json:"rangeStart"` + RangeEnd int64 `thrift:"rangeEnd,3,required" db:"rangeEnd" json:"rangeEnd"` + NameSpace string `thrift:"nameSpace,4,required" db:"nameSpace" json:"nameSpace"` + Limit *int64 `thrift:"limit,5" db:"limit" json:"limit,omitempty"` + NoData *bool `thrift:"noData,6" db:"noData" json:"noData,omitempty"` + RangeType TimeType `thrift:"rangeType,7" db:"rangeType" json:"rangeType,omitempty"` + ResultTimeType TimeType `thrift:"resultTimeType,8" db:"resultTimeType" json:"resultTimeType,omitempty"` } -func (p *RegexpQuery) GetRegexp() string { - return p.Regexp -} -func (p *RegexpQuery) Read(iprot thrift.TProtocol) error { - if _, err := iprot.ReadStructBegin(); err != nil { - return thrift.PrependError(fmt.Sprintf("%T read error: ", p), err) +func NewQueryRequest() *QueryRequest { + return &QueryRequest{ + RangeType: 0, + + ResultTimeType: 0, } +} - var issetField bool = false - var issetRegexp bool = false +var QueryRequest_Query_DEFAULT *Query - for { - _, fieldTypeId, fieldId, err := iprot.ReadFieldBegin() - if err != nil { - return thrift.PrependError(fmt.Sprintf("%T field %d read error: ", p, fieldId), err) - } - if fieldTypeId == thrift.STOP { - break - } - switch fieldId { - case 1: - if err := p.ReadField1(iprot); err != nil { - return err - } - issetField = true - case 2: - if err := p.ReadField2(iprot); err != nil { - return err - } - issetRegexp = true - default: - if err := iprot.Skip(fieldTypeId); err != nil { - return err - } - } - if err := iprot.ReadFieldEnd(); err != nil { - return err - } - } - if err := iprot.ReadStructEnd(); err != nil { - return thrift.PrependError(fmt.Sprintf("%T read struct end error: ", p), err) - } - if !issetField { - return thrift.NewTProtocolExceptionWithType(thrift.INVALID_DATA, fmt.Errorf("Required field Field is not set")) - } - if !issetRegexp { - return thrift.NewTProtocolExceptionWithType(thrift.INVALID_DATA, fmt.Errorf("Required field Regexp is not set")) +func (p *QueryRequest) GetQuery() *Query { + if !p.IsSetQuery() { + return QueryRequest_Query_DEFAULT } - return nil + return p.Query } -func (p *RegexpQuery) ReadField1(iprot thrift.TProtocol) error { - if v, err := iprot.ReadString(); err != nil { - return thrift.PrependError("error reading field 1: ", err) - } else { - p.Field = v - } - return nil +func (p *QueryRequest) GetRangeStart() int64 { + return p.RangeStart } -func (p *RegexpQuery) ReadField2(iprot thrift.TProtocol) error { - if v, err := iprot.ReadString(); err != nil { - return thrift.PrependError("error reading field 2: ", err) - } else { - p.Regexp = v - } - return nil +func (p *QueryRequest) GetRangeEnd() int64 { + return p.RangeEnd } -func (p *RegexpQuery) Write(oprot thrift.TProtocol) error { - if err := oprot.WriteStructBegin("RegexpQuery"); err != nil { - return thrift.PrependError(fmt.Sprintf("%T write struct begin error: ", p), err) - } - if p != nil { - if err := p.writeField1(oprot); err != nil { - return err - } - if err := p.writeField2(oprot); err != nil { - return err - } - } - if err := oprot.WriteFieldStop(); err != nil { - return thrift.PrependError("write field stop error: ", err) - } - if err := oprot.WriteStructEnd(); err != nil { - return thrift.PrependError("write struct stop error: ", err) - } - return nil +func (p *QueryRequest) GetNameSpace() string { + return p.NameSpace } -func (p *RegexpQuery) writeField1(oprot thrift.TProtocol) (err error) { - if err := oprot.WriteFieldBegin("field", thrift.STRING, 1); err != nil { - return thrift.PrependError(fmt.Sprintf("%T write field begin error 1:field: ", p), err) - } - if err := oprot.WriteString(string(p.Field)); err != nil { - return thrift.PrependError(fmt.Sprintf("%T.field (1) field write error: ", p), err) - } - if err := oprot.WriteFieldEnd(); err != nil { - return thrift.PrependError(fmt.Sprintf("%T write field end error 1:field: ", p), err) +var QueryRequest_Limit_DEFAULT int64 + +func (p *QueryRequest) GetLimit() int64 { + if !p.IsSetLimit() { + return QueryRequest_Limit_DEFAULT } - return err + return *p.Limit } -func (p *RegexpQuery) writeField2(oprot thrift.TProtocol) (err error) { - if err := oprot.WriteFieldBegin("regexp", thrift.STRING, 2); err != nil { - return thrift.PrependError(fmt.Sprintf("%T write field begin error 2:regexp: ", p), err) - } - if err := oprot.WriteString(string(p.Regexp)); err != nil { - return thrift.PrependError(fmt.Sprintf("%T.regexp (2) field write error: ", p), err) - } - if err := oprot.WriteFieldEnd(); err != nil { - return thrift.PrependError(fmt.Sprintf("%T write field end error 2:regexp: ", p), err) +var QueryRequest_NoData_DEFAULT bool + +func (p *QueryRequest) GetNoData() bool { + if !p.IsSetNoData() { + return QueryRequest_NoData_DEFAULT } - return err + return *p.NoData } -func (p *RegexpQuery) String() string { - if p == nil { - return "" - } - return fmt.Sprintf("RegexpQuery(%+v)", *p) +var QueryRequest_RangeType_DEFAULT TimeType = 0 + +func (p *QueryRequest) GetRangeType() TimeType { + return p.RangeType } -// Attributes: -// - Query -type NegationQuery struct { - Query *Query `thrift:"query,1,required" db:"query" json:"query"` +var QueryRequest_ResultTimeType_DEFAULT TimeType = 0 + +func (p *QueryRequest) GetResultTimeType() TimeType { + return p.ResultTimeType +} +func (p *QueryRequest) IsSetQuery() bool { + return p.Query != nil } -func NewNegationQuery() *NegationQuery { - return &NegationQuery{} +func (p *QueryRequest) IsSetLimit() bool { + return p.Limit != nil } -var NegationQuery_Query_DEFAULT *Query +func (p *QueryRequest) IsSetNoData() bool { + return p.NoData != nil +} -func (p *NegationQuery) GetQuery() *Query { - if !p.IsSetQuery() { - return NegationQuery_Query_DEFAULT - } - return p.Query +func (p *QueryRequest) IsSetRangeType() bool { + return p.RangeType != QueryRequest_RangeType_DEFAULT } -func (p *NegationQuery) IsSetQuery() bool { - return p.Query != nil + +func (p *QueryRequest) IsSetResultTimeType() bool { + return p.ResultTimeType != QueryRequest_ResultTimeType_DEFAULT } -func (p *NegationQuery) Read(iprot thrift.TProtocol) error { +func (p *QueryRequest) Read(iprot thrift.TProtocol) error { if _, err := iprot.ReadStructBegin(); err != nil { return thrift.PrependError(fmt.Sprintf("%T read error: ", p), err) } var issetQuery bool = false + var issetRangeStart bool = false + var issetRangeEnd bool = false + var issetNameSpace bool = false for { _, fieldTypeId, fieldId, err := iprot.ReadFieldBegin() @@ -10649,6 +10438,37 @@ func (p *NegationQuery) Read(iprot thrift.TProtocol) error { return err } issetQuery = true + case 2: + if err := p.ReadField2(iprot); err != nil { + return err + } + issetRangeStart = true + case 3: + if err := p.ReadField3(iprot); err != nil { + return err + } + issetRangeEnd = true + case 4: + if err := p.ReadField4(iprot); err != nil { + return err + } + issetNameSpace = true + case 5: + if err := p.ReadField5(iprot); err != nil { + return err + } + case 6: + if err := p.ReadField6(iprot); err != nil { + return err + } + case 7: + if err := p.ReadField7(iprot); err != nil { + return err + } + case 8: + if err := p.ReadField8(iprot); err != nil { + return err + } default: if err := iprot.Skip(fieldTypeId); err != nil { return err @@ -10664,10 +10484,19 @@ func (p *NegationQuery) Read(iprot thrift.TProtocol) error { if !issetQuery { return thrift.NewTProtocolExceptionWithType(thrift.INVALID_DATA, fmt.Errorf("Required field Query is not set")) } + if !issetRangeStart { + return thrift.NewTProtocolExceptionWithType(thrift.INVALID_DATA, fmt.Errorf("Required field RangeStart is not set")) + } + if !issetRangeEnd { + return thrift.NewTProtocolExceptionWithType(thrift.INVALID_DATA, fmt.Errorf("Required field RangeEnd is not set")) + } + if !issetNameSpace { + return thrift.NewTProtocolExceptionWithType(thrift.INVALID_DATA, fmt.Errorf("Required field NameSpace is not set")) + } return nil } -func (p *NegationQuery) ReadField1(iprot thrift.TProtocol) error { +func (p *QueryRequest) ReadField1(iprot thrift.TProtocol) error { p.Query = &Query{} if err := p.Query.Read(iprot); err != nil { return thrift.PrependError(fmt.Sprintf("%T error reading struct: ", p.Query), err) @@ -10675,29 +10504,115 @@ func (p *NegationQuery) ReadField1(iprot thrift.TProtocol) error { return nil } -func (p *NegationQuery) Write(oprot thrift.TProtocol) error { - if err := oprot.WriteStructBegin("NegationQuery"); err != nil { - return thrift.PrependError(fmt.Sprintf("%T write struct begin error: ", p), err) - } - if p != nil { - if err := p.writeField1(oprot); err != nil { - return err - } - } - if err := oprot.WriteFieldStop(); err != nil { - return thrift.PrependError("write field stop error: ", err) - } - if err := oprot.WriteStructEnd(); err != nil { - return thrift.PrependError("write struct stop error: ", err) +func (p *QueryRequest) ReadField2(iprot thrift.TProtocol) error { + if v, err := iprot.ReadI64(); err != nil { + return thrift.PrependError("error reading field 2: ", err) + } else { + p.RangeStart = v } return nil } -func (p *NegationQuery) writeField1(oprot thrift.TProtocol) (err error) { - if err := oprot.WriteFieldBegin("query", thrift.STRUCT, 1); err != nil { - return thrift.PrependError(fmt.Sprintf("%T write field begin error 1:query: ", p), err) - } - if err := p.Query.Write(oprot); err != nil { +func (p *QueryRequest) ReadField3(iprot thrift.TProtocol) error { + if v, err := iprot.ReadI64(); err != nil { + return thrift.PrependError("error reading field 3: ", err) + } else { + p.RangeEnd = v + } + return nil +} + +func (p *QueryRequest) ReadField4(iprot thrift.TProtocol) error { + if v, err := iprot.ReadString(); err != nil { + return thrift.PrependError("error reading field 4: ", err) + } else { + p.NameSpace = v + } + return nil +} + +func (p *QueryRequest) ReadField5(iprot thrift.TProtocol) error { + if v, err := iprot.ReadI64(); err != nil { + return thrift.PrependError("error reading field 5: ", err) + } else { + p.Limit = &v + } + return nil +} + +func (p *QueryRequest) ReadField6(iprot thrift.TProtocol) error { + if v, err := iprot.ReadBool(); err != nil { + return thrift.PrependError("error reading field 6: ", err) + } else { + p.NoData = &v + } + return nil +} + +func (p *QueryRequest) ReadField7(iprot thrift.TProtocol) error { + if v, err := iprot.ReadI32(); err != nil { + return thrift.PrependError("error reading field 7: ", err) + } else { + temp := TimeType(v) + p.RangeType = temp + } + return nil +} + +func (p *QueryRequest) ReadField8(iprot thrift.TProtocol) error { + if v, err := iprot.ReadI32(); err != nil { + return thrift.PrependError("error reading field 8: ", err) + } else { + temp := TimeType(v) + p.ResultTimeType = temp + } + return nil +} + +func (p *QueryRequest) Write(oprot thrift.TProtocol) error { + if err := oprot.WriteStructBegin("QueryRequest"); err != nil { + return thrift.PrependError(fmt.Sprintf("%T write struct begin error: ", p), err) + } + if p != nil { + if err := p.writeField1(oprot); err != nil { + return err + } + if err := p.writeField2(oprot); err != nil { + return err + } + if err := p.writeField3(oprot); err != nil { + return err + } + if err := p.writeField4(oprot); err != nil { + return err + } + if err := p.writeField5(oprot); err != nil { + return err + } + if err := p.writeField6(oprot); err != nil { + return err + } + if err := p.writeField7(oprot); err != nil { + return err + } + if err := p.writeField8(oprot); err != nil { + return err + } + } + if err := oprot.WriteFieldStop(); err != nil { + return thrift.PrependError("write field stop error: ", err) + } + if err := oprot.WriteStructEnd(); err != nil { + return thrift.PrependError("write struct stop error: ", err) + } + return nil +} + +func (p *QueryRequest) writeField1(oprot thrift.TProtocol) (err error) { + if err := oprot.WriteFieldBegin("query", thrift.STRUCT, 1); err != nil { + return thrift.PrependError(fmt.Sprintf("%T write field begin error 1:query: ", p), err) + } + if err := p.Query.Write(oprot); err != nil { return thrift.PrependError(fmt.Sprintf("%T error writing struct: ", p.Query), err) } if err := oprot.WriteFieldEnd(); err != nil { @@ -10706,32 +10621,138 @@ func (p *NegationQuery) writeField1(oprot thrift.TProtocol) (err error) { return err } -func (p *NegationQuery) String() string { +func (p *QueryRequest) writeField2(oprot thrift.TProtocol) (err error) { + if err := oprot.WriteFieldBegin("rangeStart", thrift.I64, 2); err != nil { + return thrift.PrependError(fmt.Sprintf("%T write field begin error 2:rangeStart: ", p), err) + } + if err := oprot.WriteI64(int64(p.RangeStart)); err != nil { + return thrift.PrependError(fmt.Sprintf("%T.rangeStart (2) field write error: ", p), err) + } + if err := oprot.WriteFieldEnd(); err != nil { + return thrift.PrependError(fmt.Sprintf("%T write field end error 2:rangeStart: ", p), err) + } + return err +} + +func (p *QueryRequest) writeField3(oprot thrift.TProtocol) (err error) { + if err := oprot.WriteFieldBegin("rangeEnd", thrift.I64, 3); err != nil { + return thrift.PrependError(fmt.Sprintf("%T write field begin error 3:rangeEnd: ", p), err) + } + if err := oprot.WriteI64(int64(p.RangeEnd)); err != nil { + return thrift.PrependError(fmt.Sprintf("%T.rangeEnd (3) field write error: ", p), err) + } + if err := oprot.WriteFieldEnd(); err != nil { + return thrift.PrependError(fmt.Sprintf("%T write field end error 3:rangeEnd: ", p), err) + } + return err +} + +func (p *QueryRequest) writeField4(oprot thrift.TProtocol) (err error) { + if err := oprot.WriteFieldBegin("nameSpace", thrift.STRING, 4); err != nil { + return thrift.PrependError(fmt.Sprintf("%T write field begin error 4:nameSpace: ", p), err) + } + if err := oprot.WriteString(string(p.NameSpace)); err != nil { + return thrift.PrependError(fmt.Sprintf("%T.nameSpace (4) field write error: ", p), err) + } + if err := oprot.WriteFieldEnd(); err != nil { + return thrift.PrependError(fmt.Sprintf("%T write field end error 4:nameSpace: ", p), err) + } + return err +} + +func (p *QueryRequest) writeField5(oprot thrift.TProtocol) (err error) { + if p.IsSetLimit() { + if err := oprot.WriteFieldBegin("limit", thrift.I64, 5); err != nil { + return thrift.PrependError(fmt.Sprintf("%T write field begin error 5:limit: ", p), err) + } + if err := oprot.WriteI64(int64(*p.Limit)); err != nil { + return thrift.PrependError(fmt.Sprintf("%T.limit (5) field write error: ", p), err) + } + if err := oprot.WriteFieldEnd(); err != nil { + return thrift.PrependError(fmt.Sprintf("%T write field end error 5:limit: ", p), err) + } + } + return err +} + +func (p *QueryRequest) writeField6(oprot thrift.TProtocol) (err error) { + if p.IsSetNoData() { + if err := oprot.WriteFieldBegin("noData", thrift.BOOL, 6); err != nil { + return thrift.PrependError(fmt.Sprintf("%T write field begin error 6:noData: ", p), err) + } + if err := oprot.WriteBool(bool(*p.NoData)); err != nil { + return thrift.PrependError(fmt.Sprintf("%T.noData (6) field write error: ", p), err) + } + if err := oprot.WriteFieldEnd(); err != nil { + return thrift.PrependError(fmt.Sprintf("%T write field end error 6:noData: ", p), err) + } + } + return err +} + +func (p *QueryRequest) writeField7(oprot thrift.TProtocol) (err error) { + if p.IsSetRangeType() { + if err := oprot.WriteFieldBegin("rangeType", thrift.I32, 7); err != nil { + return thrift.PrependError(fmt.Sprintf("%T write field begin error 7:rangeType: ", p), err) + } + if err := oprot.WriteI32(int32(p.RangeType)); err != nil { + return thrift.PrependError(fmt.Sprintf("%T.rangeType (7) field write error: ", p), err) + } + if err := oprot.WriteFieldEnd(); err != nil { + return thrift.PrependError(fmt.Sprintf("%T write field end error 7:rangeType: ", p), err) + } + } + return err +} + +func (p *QueryRequest) writeField8(oprot thrift.TProtocol) (err error) { + if p.IsSetResultTimeType() { + if err := oprot.WriteFieldBegin("resultTimeType", thrift.I32, 8); err != nil { + return thrift.PrependError(fmt.Sprintf("%T write field begin error 8:resultTimeType: ", p), err) + } + if err := oprot.WriteI32(int32(p.ResultTimeType)); err != nil { + return thrift.PrependError(fmt.Sprintf("%T.resultTimeType (8) field write error: ", p), err) + } + if err := oprot.WriteFieldEnd(); err != nil { + return thrift.PrependError(fmt.Sprintf("%T write field end error 8:resultTimeType: ", p), err) + } + } + return err +} + +func (p *QueryRequest) String() string { if p == nil { return "" } - return fmt.Sprintf("NegationQuery(%+v)", *p) + return fmt.Sprintf("QueryRequest(%+v)", *p) } // Attributes: -// - Queries -type ConjunctionQuery struct { - Queries []*Query `thrift:"queries,1,required" db:"queries" json:"queries"` +// - Results +// - Exhaustive +type QueryResult_ struct { + Results []*QueryResultElement `thrift:"results,1,required" db:"results" json:"results"` + Exhaustive bool `thrift:"exhaustive,2,required" db:"exhaustive" json:"exhaustive"` } -func NewConjunctionQuery() *ConjunctionQuery { - return &ConjunctionQuery{} +func NewQueryResult_() *QueryResult_ { + return &QueryResult_{} } -func (p *ConjunctionQuery) GetQueries() []*Query { - return p.Queries +func (p *QueryResult_) GetResults() []*QueryResultElement { + return p.Results } -func (p *ConjunctionQuery) Read(iprot thrift.TProtocol) error { + +func (p *QueryResult_) GetExhaustive() bool { + return p.Exhaustive +} +func (p *QueryResult_) Read(iprot thrift.TProtocol) error { if _, err := iprot.ReadStructBegin(); err != nil { return thrift.PrependError(fmt.Sprintf("%T read error: ", p), err) } - var issetQueries bool = false + var issetResults bool = false + var issetExhaustive bool = false for { _, fieldTypeId, fieldId, err := iprot.ReadFieldBegin() @@ -10746,7 +10767,12 @@ func (p *ConjunctionQuery) Read(iprot thrift.TProtocol) error { if err := p.ReadField1(iprot); err != nil { return err } - issetQueries = true + issetResults = true + case 2: + if err := p.ReadField2(iprot); err != nil { + return err + } + issetExhaustive = true default: if err := iprot.Skip(fieldTypeId); err != nil { return err @@ -10759,25 +10785,28 @@ func (p *ConjunctionQuery) Read(iprot thrift.TProtocol) error { if err := iprot.ReadStructEnd(); err != nil { return thrift.PrependError(fmt.Sprintf("%T read struct end error: ", p), err) } - if !issetQueries { - return thrift.NewTProtocolExceptionWithType(thrift.INVALID_DATA, fmt.Errorf("Required field Queries is not set")) + if !issetResults { + return thrift.NewTProtocolExceptionWithType(thrift.INVALID_DATA, fmt.Errorf("Required field Results is not set")) + } + if !issetExhaustive { + return thrift.NewTProtocolExceptionWithType(thrift.INVALID_DATA, fmt.Errorf("Required field Exhaustive is not set")) } return nil } -func (p *ConjunctionQuery) ReadField1(iprot thrift.TProtocol) error { +func (p *QueryResult_) ReadField1(iprot thrift.TProtocol) error { _, size, err := iprot.ReadListBegin() if err != nil { return thrift.PrependError("error reading list begin: ", err) } - tSlice := make([]*Query, 0, size) - p.Queries = tSlice + tSlice := make([]*QueryResultElement, 0, size) + p.Results = tSlice for i := 0; i < size; i++ { - _elem25 := &Query{} - if err := _elem25.Read(iprot); err != nil { - return thrift.PrependError(fmt.Sprintf("%T error reading struct: ", _elem25), err) + _elem26 := &QueryResultElement{} + if err := _elem26.Read(iprot); err != nil { + return thrift.PrependError(fmt.Sprintf("%T error reading struct: ", _elem26), err) } - p.Queries = append(p.Queries, _elem25) + p.Results = append(p.Results, _elem26) } if err := iprot.ReadListEnd(); err != nil { return thrift.PrependError("error reading list end: ", err) @@ -10785,14 +10814,26 @@ func (p *ConjunctionQuery) ReadField1(iprot thrift.TProtocol) error { return nil } -func (p *ConjunctionQuery) Write(oprot thrift.TProtocol) error { - if err := oprot.WriteStructBegin("ConjunctionQuery"); err != nil { - return thrift.PrependError(fmt.Sprintf("%T write struct begin error: ", p), err) - } +func (p *QueryResult_) ReadField2(iprot thrift.TProtocol) error { + if v, err := iprot.ReadBool(); err != nil { + return thrift.PrependError("error reading field 2: ", err) + } else { + p.Exhaustive = v + } + return nil +} + +func (p *QueryResult_) Write(oprot thrift.TProtocol) error { + if err := oprot.WriteStructBegin("QueryResult"); err != nil { + return thrift.PrependError(fmt.Sprintf("%T write struct begin error: ", p), err) + } if p != nil { if err := p.writeField1(oprot); err != nil { return err } + if err := p.writeField2(oprot); err != nil { + return err + } } if err := oprot.WriteFieldStop(); err != nil { return thrift.PrependError("write field stop error: ", err) @@ -10803,14 +10844,14 @@ func (p *ConjunctionQuery) Write(oprot thrift.TProtocol) error { return nil } -func (p *ConjunctionQuery) writeField1(oprot thrift.TProtocol) (err error) { - if err := oprot.WriteFieldBegin("queries", thrift.LIST, 1); err != nil { - return thrift.PrependError(fmt.Sprintf("%T write field begin error 1:queries: ", p), err) +func (p *QueryResult_) writeField1(oprot thrift.TProtocol) (err error) { + if err := oprot.WriteFieldBegin("results", thrift.LIST, 1); err != nil { + return thrift.PrependError(fmt.Sprintf("%T write field begin error 1:results: ", p), err) } - if err := oprot.WriteListBegin(thrift.STRUCT, len(p.Queries)); err != nil { + if err := oprot.WriteListBegin(thrift.STRUCT, len(p.Results)); err != nil { return thrift.PrependError("error writing list begin: ", err) } - for _, v := range p.Queries { + for _, v := range p.Results { if err := v.Write(oprot); err != nil { return thrift.PrependError(fmt.Sprintf("%T error writing struct: ", v), err) } @@ -10819,37 +10860,64 @@ func (p *ConjunctionQuery) writeField1(oprot thrift.TProtocol) (err error) { return thrift.PrependError("error writing list end: ", err) } if err := oprot.WriteFieldEnd(); err != nil { - return thrift.PrependError(fmt.Sprintf("%T write field end error 1:queries: ", p), err) + return thrift.PrependError(fmt.Sprintf("%T write field end error 1:results: ", p), err) } return err } -func (p *ConjunctionQuery) String() string { +func (p *QueryResult_) writeField2(oprot thrift.TProtocol) (err error) { + if err := oprot.WriteFieldBegin("exhaustive", thrift.BOOL, 2); err != nil { + return thrift.PrependError(fmt.Sprintf("%T write field begin error 2:exhaustive: ", p), err) + } + if err := oprot.WriteBool(bool(p.Exhaustive)); err != nil { + return thrift.PrependError(fmt.Sprintf("%T.exhaustive (2) field write error: ", p), err) + } + if err := oprot.WriteFieldEnd(); err != nil { + return thrift.PrependError(fmt.Sprintf("%T write field end error 2:exhaustive: ", p), err) + } + return err +} + +func (p *QueryResult_) String() string { if p == nil { return "" } - return fmt.Sprintf("ConjunctionQuery(%+v)", *p) + return fmt.Sprintf("QueryResult_(%+v)", *p) } // Attributes: -// - Queries -type DisjunctionQuery struct { - Queries []*Query `thrift:"queries,1,required" db:"queries" json:"queries"` +// - ID +// - Tags +// - Datapoints +type QueryResultElement struct { + ID string `thrift:"id,1,required" db:"id" json:"id"` + Tags []*Tag `thrift:"tags,2,required" db:"tags" json:"tags"` + Datapoints []*Datapoint `thrift:"datapoints,3,required" db:"datapoints" json:"datapoints"` } -func NewDisjunctionQuery() *DisjunctionQuery { - return &DisjunctionQuery{} +func NewQueryResultElement() *QueryResultElement { + return &QueryResultElement{} } -func (p *DisjunctionQuery) GetQueries() []*Query { - return p.Queries +func (p *QueryResultElement) GetID() string { + return p.ID } -func (p *DisjunctionQuery) Read(iprot thrift.TProtocol) error { + +func (p *QueryResultElement) GetTags() []*Tag { + return p.Tags +} + +func (p *QueryResultElement) GetDatapoints() []*Datapoint { + return p.Datapoints +} +func (p *QueryResultElement) Read(iprot thrift.TProtocol) error { if _, err := iprot.ReadStructBegin(); err != nil { return thrift.PrependError(fmt.Sprintf("%T read error: ", p), err) } - var issetQueries bool = false + var issetID bool = false + var issetTags bool = false + var issetDatapoints bool = false for { _, fieldTypeId, fieldId, err := iprot.ReadFieldBegin() @@ -10864,7 +10932,17 @@ func (p *DisjunctionQuery) Read(iprot thrift.TProtocol) error { if err := p.ReadField1(iprot); err != nil { return err } - issetQueries = true + issetID = true + case 2: + if err := p.ReadField2(iprot); err != nil { + return err + } + issetTags = true + case 3: + if err := p.ReadField3(iprot); err != nil { + return err + } + issetDatapoints = true default: if err := iprot.Skip(fieldTypeId); err != nil { return err @@ -10877,25 +10955,40 @@ func (p *DisjunctionQuery) Read(iprot thrift.TProtocol) error { if err := iprot.ReadStructEnd(); err != nil { return thrift.PrependError(fmt.Sprintf("%T read struct end error: ", p), err) } - if !issetQueries { - return thrift.NewTProtocolExceptionWithType(thrift.INVALID_DATA, fmt.Errorf("Required field Queries is not set")) + if !issetID { + return thrift.NewTProtocolExceptionWithType(thrift.INVALID_DATA, fmt.Errorf("Required field ID is not set")) + } + if !issetTags { + return thrift.NewTProtocolExceptionWithType(thrift.INVALID_DATA, fmt.Errorf("Required field Tags is not set")) + } + if !issetDatapoints { + return thrift.NewTProtocolExceptionWithType(thrift.INVALID_DATA, fmt.Errorf("Required field Datapoints is not set")) } return nil } -func (p *DisjunctionQuery) ReadField1(iprot thrift.TProtocol) error { +func (p *QueryResultElement) ReadField1(iprot thrift.TProtocol) error { + if v, err := iprot.ReadString(); err != nil { + return thrift.PrependError("error reading field 1: ", err) + } else { + p.ID = v + } + return nil +} + +func (p *QueryResultElement) ReadField2(iprot thrift.TProtocol) error { _, size, err := iprot.ReadListBegin() if err != nil { return thrift.PrependError("error reading list begin: ", err) } - tSlice := make([]*Query, 0, size) - p.Queries = tSlice + tSlice := make([]*Tag, 0, size) + p.Tags = tSlice for i := 0; i < size; i++ { - _elem26 := &Query{} - if err := _elem26.Read(iprot); err != nil { - return thrift.PrependError(fmt.Sprintf("%T error reading struct: ", _elem26), err) + _elem27 := &Tag{} + if err := _elem27.Read(iprot); err != nil { + return thrift.PrependError(fmt.Sprintf("%T error reading struct: ", _elem27), err) } - p.Queries = append(p.Queries, _elem26) + p.Tags = append(p.Tags, _elem27) } if err := iprot.ReadListEnd(); err != nil { return thrift.PrependError("error reading list end: ", err) @@ -10903,14 +10996,42 @@ func (p *DisjunctionQuery) ReadField1(iprot thrift.TProtocol) error { return nil } -func (p *DisjunctionQuery) Write(oprot thrift.TProtocol) error { - if err := oprot.WriteStructBegin("DisjunctionQuery"); err != nil { +func (p *QueryResultElement) ReadField3(iprot thrift.TProtocol) error { + _, size, err := iprot.ReadListBegin() + if err != nil { + return thrift.PrependError("error reading list begin: ", err) + } + tSlice := make([]*Datapoint, 0, size) + p.Datapoints = tSlice + for i := 0; i < size; i++ { + _elem28 := &Datapoint{ + TimestampTimeType: 0, + } + if err := _elem28.Read(iprot); err != nil { + return thrift.PrependError(fmt.Sprintf("%T error reading struct: ", _elem28), err) + } + p.Datapoints = append(p.Datapoints, _elem28) + } + if err := iprot.ReadListEnd(); err != nil { + return thrift.PrependError("error reading list end: ", err) + } + return nil +} + +func (p *QueryResultElement) Write(oprot thrift.TProtocol) error { + if err := oprot.WriteStructBegin("QueryResultElement"); err != nil { return thrift.PrependError(fmt.Sprintf("%T write struct begin error: ", p), err) } if p != nil { if err := p.writeField1(oprot); err != nil { return err } + if err := p.writeField2(oprot); err != nil { + return err + } + if err := p.writeField3(oprot); err != nil { + return err + } } if err := oprot.WriteFieldStop(); err != nil { return thrift.PrependError("write field stop error: ", err) @@ -10921,14 +11042,27 @@ func (p *DisjunctionQuery) Write(oprot thrift.TProtocol) error { return nil } -func (p *DisjunctionQuery) writeField1(oprot thrift.TProtocol) (err error) { - if err := oprot.WriteFieldBegin("queries", thrift.LIST, 1); err != nil { - return thrift.PrependError(fmt.Sprintf("%T write field begin error 1:queries: ", p), err) +func (p *QueryResultElement) writeField1(oprot thrift.TProtocol) (err error) { + if err := oprot.WriteFieldBegin("id", thrift.STRING, 1); err != nil { + return thrift.PrependError(fmt.Sprintf("%T write field begin error 1:id: ", p), err) } - if err := oprot.WriteListBegin(thrift.STRUCT, len(p.Queries)); err != nil { + if err := oprot.WriteString(string(p.ID)); err != nil { + return thrift.PrependError(fmt.Sprintf("%T.id (1) field write error: ", p), err) + } + if err := oprot.WriteFieldEnd(); err != nil { + return thrift.PrependError(fmt.Sprintf("%T write field end error 1:id: ", p), err) + } + return err +} + +func (p *QueryResultElement) writeField2(oprot thrift.TProtocol) (err error) { + if err := oprot.WriteFieldBegin("tags", thrift.LIST, 2); err != nil { + return thrift.PrependError(fmt.Sprintf("%T write field begin error 2:tags: ", p), err) + } + if err := oprot.WriteListBegin(thrift.STRUCT, len(p.Tags)); err != nil { return thrift.PrependError("error writing list begin: ", err) } - for _, v := range p.Queries { + for _, v := range p.Tags { if err := v.Write(oprot); err != nil { return thrift.PrependError(fmt.Sprintf("%T error writing struct: ", v), err) } @@ -10937,92 +11071,65 @@ func (p *DisjunctionQuery) writeField1(oprot thrift.TProtocol) (err error) { return thrift.PrependError("error writing list end: ", err) } if err := oprot.WriteFieldEnd(); err != nil { - return thrift.PrependError(fmt.Sprintf("%T write field end error 1:queries: ", p), err) + return thrift.PrependError(fmt.Sprintf("%T write field end error 2:tags: ", p), err) } return err } -func (p *DisjunctionQuery) String() string { - if p == nil { - return "" +func (p *QueryResultElement) writeField3(oprot thrift.TProtocol) (err error) { + if err := oprot.WriteFieldBegin("datapoints", thrift.LIST, 3); err != nil { + return thrift.PrependError(fmt.Sprintf("%T write field begin error 3:datapoints: ", p), err) } - return fmt.Sprintf("DisjunctionQuery(%+v)", *p) -} - -type AllQuery struct { -} - -func NewAllQuery() *AllQuery { - return &AllQuery{} -} - -func (p *AllQuery) Read(iprot thrift.TProtocol) error { - if _, err := iprot.ReadStructBegin(); err != nil { - return thrift.PrependError(fmt.Sprintf("%T read error: ", p), err) + if err := oprot.WriteListBegin(thrift.STRUCT, len(p.Datapoints)); err != nil { + return thrift.PrependError("error writing list begin: ", err) } - - for { - _, fieldTypeId, fieldId, err := iprot.ReadFieldBegin() - if err != nil { - return thrift.PrependError(fmt.Sprintf("%T field %d read error: ", p, fieldId), err) - } - if fieldTypeId == thrift.STOP { - break - } - if err := iprot.Skip(fieldTypeId); err != nil { - return err - } - if err := iprot.ReadFieldEnd(); err != nil { - return err + for _, v := range p.Datapoints { + if err := v.Write(oprot); err != nil { + return thrift.PrependError(fmt.Sprintf("%T error writing struct: ", v), err) } } - if err := iprot.ReadStructEnd(); err != nil { - return thrift.PrependError(fmt.Sprintf("%T read struct end error: ", p), err) - } - return nil -} - -func (p *AllQuery) Write(oprot thrift.TProtocol) error { - if err := oprot.WriteStructBegin("AllQuery"); err != nil { - return thrift.PrependError(fmt.Sprintf("%T write struct begin error: ", p), err) - } - if p != nil { - } - if err := oprot.WriteFieldStop(); err != nil { - return thrift.PrependError("write field stop error: ", err) + if err := oprot.WriteListEnd(); err != nil { + return thrift.PrependError("error writing list end: ", err) } - if err := oprot.WriteStructEnd(); err != nil { - return thrift.PrependError("write struct stop error: ", err) + if err := oprot.WriteFieldEnd(); err != nil { + return thrift.PrependError(fmt.Sprintf("%T write field end error 3:datapoints: ", p), err) } - return nil + return err } -func (p *AllQuery) String() string { +func (p *QueryResultElement) String() string { if p == nil { return "" } - return fmt.Sprintf("AllQuery(%+v)", *p) + return fmt.Sprintf("QueryResultElement(%+v)", *p) } // Attributes: // - Field -type FieldQuery struct { +// - Term +type TermQuery struct { Field string `thrift:"field,1,required" db:"field" json:"field"` + Term string `thrift:"term,2,required" db:"term" json:"term"` } -func NewFieldQuery() *FieldQuery { - return &FieldQuery{} +func NewTermQuery() *TermQuery { + return &TermQuery{} } -func (p *FieldQuery) GetField() string { +func (p *TermQuery) GetField() string { return p.Field } -func (p *FieldQuery) Read(iprot thrift.TProtocol) error { + +func (p *TermQuery) GetTerm() string { + return p.Term +} +func (p *TermQuery) Read(iprot thrift.TProtocol) error { if _, err := iprot.ReadStructBegin(); err != nil { return thrift.PrependError(fmt.Sprintf("%T read error: ", p), err) } var issetField bool = false + var issetTerm bool = false for { _, fieldTypeId, fieldId, err := iprot.ReadFieldBegin() @@ -11038,6 +11145,11 @@ func (p *FieldQuery) Read(iprot thrift.TProtocol) error { return err } issetField = true + case 2: + if err := p.ReadField2(iprot); err != nil { + return err + } + issetTerm = true default: if err := iprot.Skip(fieldTypeId); err != nil { return err @@ -11053,10 +11165,13 @@ func (p *FieldQuery) Read(iprot thrift.TProtocol) error { if !issetField { return thrift.NewTProtocolExceptionWithType(thrift.INVALID_DATA, fmt.Errorf("Required field Field is not set")) } - return nil + if !issetTerm { + return thrift.NewTProtocolExceptionWithType(thrift.INVALID_DATA, fmt.Errorf("Required field Term is not set")) + } + return nil } -func (p *FieldQuery) ReadField1(iprot thrift.TProtocol) error { +func (p *TermQuery) ReadField1(iprot thrift.TProtocol) error { if v, err := iprot.ReadString(); err != nil { return thrift.PrependError("error reading field 1: ", err) } else { @@ -11065,14 +11180,26 @@ func (p *FieldQuery) ReadField1(iprot thrift.TProtocol) error { return nil } -func (p *FieldQuery) Write(oprot thrift.TProtocol) error { - if err := oprot.WriteStructBegin("FieldQuery"); err != nil { +func (p *TermQuery) ReadField2(iprot thrift.TProtocol) error { + if v, err := iprot.ReadString(); err != nil { + return thrift.PrependError("error reading field 2: ", err) + } else { + p.Term = v + } + return nil +} + +func (p *TermQuery) Write(oprot thrift.TProtocol) error { + if err := oprot.WriteStructBegin("TermQuery"); err != nil { return thrift.PrependError(fmt.Sprintf("%T write struct begin error: ", p), err) } if p != nil { if err := p.writeField1(oprot); err != nil { return err } + if err := p.writeField2(oprot); err != nil { + return err + } } if err := oprot.WriteFieldStop(); err != nil { return thrift.PrependError("write field stop error: ", err) @@ -11083,7 +11210,7 @@ func (p *FieldQuery) Write(oprot thrift.TProtocol) error { return nil } -func (p *FieldQuery) writeField1(oprot thrift.TProtocol) (err error) { +func (p *TermQuery) writeField1(oprot thrift.TProtocol) (err error) { if err := oprot.WriteFieldBegin("field", thrift.STRING, 1); err != nil { return thrift.PrependError(fmt.Sprintf("%T write field begin error 1:field: ", p), err) } @@ -11096,130 +11223,53 @@ func (p *FieldQuery) writeField1(oprot thrift.TProtocol) (err error) { return err } -func (p *FieldQuery) String() string { - if p == nil { - return "" - } - return fmt.Sprintf("FieldQuery(%+v)", *p) -} - -// Attributes: -// - Term -// - Regexp -// - Negation -// - Conjunction -// - Disjunction -// - All -// - Field -type Query struct { - Term *TermQuery `thrift:"term,1" db:"term" json:"term,omitempty"` - Regexp *RegexpQuery `thrift:"regexp,2" db:"regexp" json:"regexp,omitempty"` - Negation *NegationQuery `thrift:"negation,3" db:"negation" json:"negation,omitempty"` - Conjunction *ConjunctionQuery `thrift:"conjunction,4" db:"conjunction" json:"conjunction,omitempty"` - Disjunction *DisjunctionQuery `thrift:"disjunction,5" db:"disjunction" json:"disjunction,omitempty"` - All *AllQuery `thrift:"all,6" db:"all" json:"all,omitempty"` - Field *FieldQuery `thrift:"field,7" db:"field" json:"field,omitempty"` -} - -func NewQuery() *Query { - return &Query{} -} - -var Query_Term_DEFAULT *TermQuery - -func (p *Query) GetTerm() *TermQuery { - if !p.IsSetTerm() { - return Query_Term_DEFAULT +func (p *TermQuery) writeField2(oprot thrift.TProtocol) (err error) { + if err := oprot.WriteFieldBegin("term", thrift.STRING, 2); err != nil { + return thrift.PrependError(fmt.Sprintf("%T write field begin error 2:term: ", p), err) } - return p.Term -} - -var Query_Regexp_DEFAULT *RegexpQuery - -func (p *Query) GetRegexp() *RegexpQuery { - if !p.IsSetRegexp() { - return Query_Regexp_DEFAULT + if err := oprot.WriteString(string(p.Term)); err != nil { + return thrift.PrependError(fmt.Sprintf("%T.term (2) field write error: ", p), err) } - return p.Regexp -} - -var Query_Negation_DEFAULT *NegationQuery - -func (p *Query) GetNegation() *NegationQuery { - if !p.IsSetNegation() { - return Query_Negation_DEFAULT + if err := oprot.WriteFieldEnd(); err != nil { + return thrift.PrependError(fmt.Sprintf("%T write field end error 2:term: ", p), err) } - return p.Negation + return err } -var Query_Conjunction_DEFAULT *ConjunctionQuery - -func (p *Query) GetConjunction() *ConjunctionQuery { - if !p.IsSetConjunction() { - return Query_Conjunction_DEFAULT +func (p *TermQuery) String() string { + if p == nil { + return "" } - return p.Conjunction + return fmt.Sprintf("TermQuery(%+v)", *p) } -var Query_Disjunction_DEFAULT *DisjunctionQuery - -func (p *Query) GetDisjunction() *DisjunctionQuery { - if !p.IsSetDisjunction() { - return Query_Disjunction_DEFAULT - } - return p.Disjunction +// Attributes: +// - Field +// - Regexp +type RegexpQuery struct { + Field string `thrift:"field,1,required" db:"field" json:"field"` + Regexp string `thrift:"regexp,2,required" db:"regexp" json:"regexp"` } -var Query_All_DEFAULT *AllQuery - -func (p *Query) GetAll() *AllQuery { - if !p.IsSetAll() { - return Query_All_DEFAULT - } - return p.All +func NewRegexpQuery() *RegexpQuery { + return &RegexpQuery{} } -var Query_Field_DEFAULT *FieldQuery - -func (p *Query) GetField() *FieldQuery { - if !p.IsSetField() { - return Query_Field_DEFAULT - } +func (p *RegexpQuery) GetField() string { return p.Field } -func (p *Query) IsSetTerm() bool { - return p.Term != nil -} - -func (p *Query) IsSetRegexp() bool { - return p.Regexp != nil -} - -func (p *Query) IsSetNegation() bool { - return p.Negation != nil -} - -func (p *Query) IsSetConjunction() bool { - return p.Conjunction != nil -} - -func (p *Query) IsSetDisjunction() bool { - return p.Disjunction != nil -} - -func (p *Query) IsSetAll() bool { - return p.All != nil -} -func (p *Query) IsSetField() bool { - return p.Field != nil +func (p *RegexpQuery) GetRegexp() string { + return p.Regexp } - -func (p *Query) Read(iprot thrift.TProtocol) error { +func (p *RegexpQuery) Read(iprot thrift.TProtocol) error { if _, err := iprot.ReadStructBegin(); err != nil { return thrift.PrependError(fmt.Sprintf("%T read error: ", p), err) } + var issetField bool = false + var issetRegexp bool = false + for { _, fieldTypeId, fieldId, err := iprot.ReadFieldBegin() if err != nil { @@ -11233,30 +11283,12 @@ func (p *Query) Read(iprot thrift.TProtocol) error { if err := p.ReadField1(iprot); err != nil { return err } + issetField = true case 2: if err := p.ReadField2(iprot); err != nil { return err } - case 3: - if err := p.ReadField3(iprot); err != nil { - return err - } - case 4: - if err := p.ReadField4(iprot); err != nil { - return err - } - case 5: - if err := p.ReadField5(iprot); err != nil { - return err - } - case 6: - if err := p.ReadField6(iprot); err != nil { - return err - } - case 7: - if err := p.ReadField7(iprot); err != nil { - return err - } + issetRegexp = true default: if err := iprot.Skip(fieldTypeId); err != nil { return err @@ -11269,67 +11301,35 @@ func (p *Query) Read(iprot thrift.TProtocol) error { if err := iprot.ReadStructEnd(); err != nil { return thrift.PrependError(fmt.Sprintf("%T read struct end error: ", p), err) } - return nil -} - -func (p *Query) ReadField1(iprot thrift.TProtocol) error { - p.Term = &TermQuery{} - if err := p.Term.Read(iprot); err != nil { - return thrift.PrependError(fmt.Sprintf("%T error reading struct: ", p.Term), err) - } - return nil -} - -func (p *Query) ReadField2(iprot thrift.TProtocol) error { - p.Regexp = &RegexpQuery{} - if err := p.Regexp.Read(iprot); err != nil { - return thrift.PrependError(fmt.Sprintf("%T error reading struct: ", p.Regexp), err) - } - return nil -} - -func (p *Query) ReadField3(iprot thrift.TProtocol) error { - p.Negation = &NegationQuery{} - if err := p.Negation.Read(iprot); err != nil { - return thrift.PrependError(fmt.Sprintf("%T error reading struct: ", p.Negation), err) - } - return nil -} - -func (p *Query) ReadField4(iprot thrift.TProtocol) error { - p.Conjunction = &ConjunctionQuery{} - if err := p.Conjunction.Read(iprot); err != nil { - return thrift.PrependError(fmt.Sprintf("%T error reading struct: ", p.Conjunction), err) + if !issetField { + return thrift.NewTProtocolExceptionWithType(thrift.INVALID_DATA, fmt.Errorf("Required field Field is not set")) } - return nil -} - -func (p *Query) ReadField5(iprot thrift.TProtocol) error { - p.Disjunction = &DisjunctionQuery{} - if err := p.Disjunction.Read(iprot); err != nil { - return thrift.PrependError(fmt.Sprintf("%T error reading struct: ", p.Disjunction), err) + if !issetRegexp { + return thrift.NewTProtocolExceptionWithType(thrift.INVALID_DATA, fmt.Errorf("Required field Regexp is not set")) } return nil } -func (p *Query) ReadField6(iprot thrift.TProtocol) error { - p.All = &AllQuery{} - if err := p.All.Read(iprot); err != nil { - return thrift.PrependError(fmt.Sprintf("%T error reading struct: ", p.All), err) +func (p *RegexpQuery) ReadField1(iprot thrift.TProtocol) error { + if v, err := iprot.ReadString(); err != nil { + return thrift.PrependError("error reading field 1: ", err) + } else { + p.Field = v } return nil } -func (p *Query) ReadField7(iprot thrift.TProtocol) error { - p.Field = &FieldQuery{} - if err := p.Field.Read(iprot); err != nil { - return thrift.PrependError(fmt.Sprintf("%T error reading struct: ", p.Field), err) +func (p *RegexpQuery) ReadField2(iprot thrift.TProtocol) error { + if v, err := iprot.ReadString(); err != nil { + return thrift.PrependError("error reading field 2: ", err) + } else { + p.Regexp = v } return nil } -func (p *Query) Write(oprot thrift.TProtocol) error { - if err := oprot.WriteStructBegin("Query"); err != nil { +func (p *RegexpQuery) Write(oprot thrift.TProtocol) error { + if err := oprot.WriteStructBegin("RegexpQuery"); err != nil { return thrift.PrependError(fmt.Sprintf("%T write struct begin error: ", p), err) } if p != nil { @@ -11339,21 +11339,6 @@ func (p *Query) Write(oprot thrift.TProtocol) error { if err := p.writeField2(oprot); err != nil { return err } - if err := p.writeField3(oprot); err != nil { - return err - } - if err := p.writeField4(oprot); err != nil { - return err - } - if err := p.writeField5(oprot); err != nil { - return err - } - if err := p.writeField6(oprot); err != nil { - return err - } - if err := p.writeField7(oprot); err != nil { - return err - } } if err := oprot.WriteFieldStop(); err != nil { return thrift.PrependError("write field stop error: ", err) @@ -11364,758 +11349,1878 @@ func (p *Query) Write(oprot thrift.TProtocol) error { return nil } -func (p *Query) writeField1(oprot thrift.TProtocol) (err error) { - if p.IsSetTerm() { - if err := oprot.WriteFieldBegin("term", thrift.STRUCT, 1); err != nil { - return thrift.PrependError(fmt.Sprintf("%T write field begin error 1:term: ", p), err) - } - if err := p.Term.Write(oprot); err != nil { - return thrift.PrependError(fmt.Sprintf("%T error writing struct: ", p.Term), err) - } - if err := oprot.WriteFieldEnd(); err != nil { - return thrift.PrependError(fmt.Sprintf("%T write field end error 1:term: ", p), err) - } +func (p *RegexpQuery) writeField1(oprot thrift.TProtocol) (err error) { + if err := oprot.WriteFieldBegin("field", thrift.STRING, 1); err != nil { + return thrift.PrependError(fmt.Sprintf("%T write field begin error 1:field: ", p), err) + } + if err := oprot.WriteString(string(p.Field)); err != nil { + return thrift.PrependError(fmt.Sprintf("%T.field (1) field write error: ", p), err) + } + if err := oprot.WriteFieldEnd(); err != nil { + return thrift.PrependError(fmt.Sprintf("%T write field end error 1:field: ", p), err) } return err } -func (p *Query) writeField2(oprot thrift.TProtocol) (err error) { - if p.IsSetRegexp() { - if err := oprot.WriteFieldBegin("regexp", thrift.STRUCT, 2); err != nil { - return thrift.PrependError(fmt.Sprintf("%T write field begin error 2:regexp: ", p), err) - } - if err := p.Regexp.Write(oprot); err != nil { - return thrift.PrependError(fmt.Sprintf("%T error writing struct: ", p.Regexp), err) - } - if err := oprot.WriteFieldEnd(); err != nil { - return thrift.PrependError(fmt.Sprintf("%T write field end error 2:regexp: ", p), err) - } +func (p *RegexpQuery) writeField2(oprot thrift.TProtocol) (err error) { + if err := oprot.WriteFieldBegin("regexp", thrift.STRING, 2); err != nil { + return thrift.PrependError(fmt.Sprintf("%T write field begin error 2:regexp: ", p), err) + } + if err := oprot.WriteString(string(p.Regexp)); err != nil { + return thrift.PrependError(fmt.Sprintf("%T.regexp (2) field write error: ", p), err) + } + if err := oprot.WriteFieldEnd(); err != nil { + return thrift.PrependError(fmt.Sprintf("%T write field end error 2:regexp: ", p), err) } return err } -func (p *Query) writeField3(oprot thrift.TProtocol) (err error) { - if p.IsSetNegation() { - if err := oprot.WriteFieldBegin("negation", thrift.STRUCT, 3); err != nil { - return thrift.PrependError(fmt.Sprintf("%T write field begin error 3:negation: ", p), err) - } - if err := p.Negation.Write(oprot); err != nil { - return thrift.PrependError(fmt.Sprintf("%T error writing struct: ", p.Negation), err) - } - if err := oprot.WriteFieldEnd(); err != nil { - return thrift.PrependError(fmt.Sprintf("%T write field end error 3:negation: ", p), err) - } +func (p *RegexpQuery) String() string { + if p == nil { + return "" } - return err + return fmt.Sprintf("RegexpQuery(%+v)", *p) } -func (p *Query) writeField4(oprot thrift.TProtocol) (err error) { - if p.IsSetConjunction() { - if err := oprot.WriteFieldBegin("conjunction", thrift.STRUCT, 4); err != nil { - return thrift.PrependError(fmt.Sprintf("%T write field begin error 4:conjunction: ", p), err) - } - if err := p.Conjunction.Write(oprot); err != nil { - return thrift.PrependError(fmt.Sprintf("%T error writing struct: ", p.Conjunction), err) - } - if err := oprot.WriteFieldEnd(); err != nil { - return thrift.PrependError(fmt.Sprintf("%T write field end error 4:conjunction: ", p), err) - } +// Attributes: +// - Query +type NegationQuery struct { + Query *Query `thrift:"query,1,required" db:"query" json:"query"` +} + +func NewNegationQuery() *NegationQuery { + return &NegationQuery{} +} + +var NegationQuery_Query_DEFAULT *Query + +func (p *NegationQuery) GetQuery() *Query { + if !p.IsSetQuery() { + return NegationQuery_Query_DEFAULT } - return err + return p.Query +} +func (p *NegationQuery) IsSetQuery() bool { + return p.Query != nil } -func (p *Query) writeField5(oprot thrift.TProtocol) (err error) { - if p.IsSetDisjunction() { - if err := oprot.WriteFieldBegin("disjunction", thrift.STRUCT, 5); err != nil { - return thrift.PrependError(fmt.Sprintf("%T write field begin error 5:disjunction: ", p), err) +func (p *NegationQuery) Read(iprot thrift.TProtocol) error { + if _, err := iprot.ReadStructBegin(); err != nil { + return thrift.PrependError(fmt.Sprintf("%T read error: ", p), err) + } + + var issetQuery bool = false + + for { + _, fieldTypeId, fieldId, err := iprot.ReadFieldBegin() + if err != nil { + return thrift.PrependError(fmt.Sprintf("%T field %d read error: ", p, fieldId), err) } - if err := p.Disjunction.Write(oprot); err != nil { - return thrift.PrependError(fmt.Sprintf("%T error writing struct: ", p.Disjunction), err) + if fieldTypeId == thrift.STOP { + break } - if err := oprot.WriteFieldEnd(); err != nil { - return thrift.PrependError(fmt.Sprintf("%T write field end error 5:disjunction: ", p), err) + switch fieldId { + case 1: + if err := p.ReadField1(iprot); err != nil { + return err + } + issetQuery = true + default: + if err := iprot.Skip(fieldTypeId); err != nil { + return err + } + } + if err := iprot.ReadFieldEnd(); err != nil { + return err } } - return err + if err := iprot.ReadStructEnd(); err != nil { + return thrift.PrependError(fmt.Sprintf("%T read struct end error: ", p), err) + } + if !issetQuery { + return thrift.NewTProtocolExceptionWithType(thrift.INVALID_DATA, fmt.Errorf("Required field Query is not set")) + } + return nil } -func (p *Query) writeField6(oprot thrift.TProtocol) (err error) { - if p.IsSetAll() { - if err := oprot.WriteFieldBegin("all", thrift.STRUCT, 6); err != nil { - return thrift.PrependError(fmt.Sprintf("%T write field begin error 6:all: ", p), err) - } - if err := p.All.Write(oprot); err != nil { - return thrift.PrependError(fmt.Sprintf("%T error writing struct: ", p.All), err) - } - if err := oprot.WriteFieldEnd(); err != nil { - return thrift.PrependError(fmt.Sprintf("%T write field end error 6:all: ", p), err) - } +func (p *NegationQuery) ReadField1(iprot thrift.TProtocol) error { + p.Query = &Query{} + if err := p.Query.Read(iprot); err != nil { + return thrift.PrependError(fmt.Sprintf("%T error reading struct: ", p.Query), err) } - return err + return nil } -func (p *Query) writeField7(oprot thrift.TProtocol) (err error) { - if p.IsSetField() { - if err := oprot.WriteFieldBegin("field", thrift.STRUCT, 7); err != nil { - return thrift.PrependError(fmt.Sprintf("%T write field begin error 7:field: ", p), err) - } - if err := p.Field.Write(oprot); err != nil { - return thrift.PrependError(fmt.Sprintf("%T error writing struct: ", p.Field), err) - } - if err := oprot.WriteFieldEnd(); err != nil { - return thrift.PrependError(fmt.Sprintf("%T write field end error 7:field: ", p), err) +func (p *NegationQuery) Write(oprot thrift.TProtocol) error { + if err := oprot.WriteStructBegin("NegationQuery"); err != nil { + return thrift.PrependError(fmt.Sprintf("%T write struct begin error: ", p), err) + } + if p != nil { + if err := p.writeField1(oprot); err != nil { + return err } } + if err := oprot.WriteFieldStop(); err != nil { + return thrift.PrependError("write field stop error: ", err) + } + if err := oprot.WriteStructEnd(); err != nil { + return thrift.PrependError("write struct stop error: ", err) + } + return nil +} + +func (p *NegationQuery) writeField1(oprot thrift.TProtocol) (err error) { + if err := oprot.WriteFieldBegin("query", thrift.STRUCT, 1); err != nil { + return thrift.PrependError(fmt.Sprintf("%T write field begin error 1:query: ", p), err) + } + if err := p.Query.Write(oprot); err != nil { + return thrift.PrependError(fmt.Sprintf("%T error writing struct: ", p.Query), err) + } + if err := oprot.WriteFieldEnd(); err != nil { + return thrift.PrependError(fmt.Sprintf("%T write field end error 1:query: ", p), err) + } return err } -func (p *Query) String() string { +func (p *NegationQuery) String() string { if p == nil { return "" } - return fmt.Sprintf("Query(%+v)", *p) + return fmt.Sprintf("NegationQuery(%+v)", *p) } -type Node interface { - // Parameters: - // - Req - Query(req *QueryRequest) (r *QueryResult_, err error) - // Parameters: - // - Req - AggregateRaw(req *AggregateQueryRawRequest) (r *AggregateQueryRawResult_, err error) - // Parameters: - // - Req - Aggregate(req *AggregateQueryRequest) (r *AggregateQueryResult_, err error) - // Parameters: - // - Req - Fetch(req *FetchRequest) (r *FetchResult_, err error) - // Parameters: - // - Req - FetchTagged(req *FetchTaggedRequest) (r *FetchTaggedResult_, err error) - // Parameters: - // - Req - Write(req *WriteRequest) (err error) - // Parameters: - // - Req - WriteTagged(req *WriteTaggedRequest) (err error) - // Parameters: - // - Req - FetchBatchRaw(req *FetchBatchRawRequest) (r *FetchBatchRawResult_, err error) - // Parameters: - // - Req - FetchBlocksRaw(req *FetchBlocksRawRequest) (r *FetchBlocksRawResult_, err error) - // Parameters: - // - Req - FetchBlocksMetadataRawV2(req *FetchBlocksMetadataRawV2Request) (r *FetchBlocksMetadataRawV2Result_, err error) - // Parameters: - // - Req - WriteBatchRaw(req *WriteBatchRawRequest) (err error) - // Parameters: - // - Req - WriteTaggedBatchRaw(req *WriteTaggedBatchRawRequest) (err error) - Repair() (err error) - // Parameters: - // - Req - Truncate(req *TruncateRequest) (r *TruncateResult_, err error) - Health() (r *NodeHealthResult_, err error) - Bootstrapped() (r *NodeBootstrappedResult_, err error) - BootstrappedInPlacementOrNoPlacement() (r *NodeBootstrappedInPlacementOrNoPlacementResult_, err error) - GetPersistRateLimit() (r *NodePersistRateLimitResult_, err error) - // Parameters: - // - Req - SetPersistRateLimit(req *NodeSetPersistRateLimitRequest) (r *NodePersistRateLimitResult_, err error) - GetWriteNewSeriesAsync() (r *NodeWriteNewSeriesAsyncResult_, err error) - // Parameters: - // - Req - SetWriteNewSeriesAsync(req *NodeSetWriteNewSeriesAsyncRequest) (r *NodeWriteNewSeriesAsyncResult_, err error) - GetWriteNewSeriesBackoffDuration() (r *NodeWriteNewSeriesBackoffDurationResult_, err error) - // Parameters: - // - Req - SetWriteNewSeriesBackoffDuration(req *NodeSetWriteNewSeriesBackoffDurationRequest) (r *NodeWriteNewSeriesBackoffDurationResult_, err error) - GetWriteNewSeriesLimitPerShardPerSecond() (r *NodeWriteNewSeriesLimitPerShardPerSecondResult_, err error) - // Parameters: - // - Req - SetWriteNewSeriesLimitPerShardPerSecond(req *NodeSetWriteNewSeriesLimitPerShardPerSecondRequest) (r *NodeWriteNewSeriesLimitPerShardPerSecondResult_, err error) +// Attributes: +// - Queries +type ConjunctionQuery struct { + Queries []*Query `thrift:"queries,1,required" db:"queries" json:"queries"` } -type NodeClient struct { - Transport thrift.TTransport - ProtocolFactory thrift.TProtocolFactory - InputProtocol thrift.TProtocol - OutputProtocol thrift.TProtocol - SeqId int32 +func NewConjunctionQuery() *ConjunctionQuery { + return &ConjunctionQuery{} } -func NewNodeClientFactory(t thrift.TTransport, f thrift.TProtocolFactory) *NodeClient { - return &NodeClient{Transport: t, - ProtocolFactory: f, - InputProtocol: f.GetProtocol(t), - OutputProtocol: f.GetProtocol(t), - SeqId: 0, - } +func (p *ConjunctionQuery) GetQueries() []*Query { + return p.Queries } - -func NewNodeClientProtocol(t thrift.TTransport, iprot thrift.TProtocol, oprot thrift.TProtocol) *NodeClient { - return &NodeClient{Transport: t, - ProtocolFactory: nil, - InputProtocol: iprot, - OutputProtocol: oprot, - SeqId: 0, +func (p *ConjunctionQuery) Read(iprot thrift.TProtocol) error { + if _, err := iprot.ReadStructBegin(); err != nil { + return thrift.PrependError(fmt.Sprintf("%T read error: ", p), err) } -} -// Parameters: -// - Req -func (p *NodeClient) Query(req *QueryRequest) (r *QueryResult_, err error) { - if err = p.sendQuery(req); err != nil { - return - } - return p.recvQuery() -} + var issetQueries bool = false -func (p *NodeClient) sendQuery(req *QueryRequest) (err error) { - oprot := p.OutputProtocol - if oprot == nil { - oprot = p.ProtocolFactory.GetProtocol(p.Transport) - p.OutputProtocol = oprot - } - p.SeqId++ - if err = oprot.WriteMessageBegin("query", thrift.CALL, p.SeqId); err != nil { - return - } - args := NodeQueryArgs{ - Req: req, + for { + _, fieldTypeId, fieldId, err := iprot.ReadFieldBegin() + if err != nil { + return thrift.PrependError(fmt.Sprintf("%T field %d read error: ", p, fieldId), err) + } + if fieldTypeId == thrift.STOP { + break + } + switch fieldId { + case 1: + if err := p.ReadField1(iprot); err != nil { + return err + } + issetQueries = true + default: + if err := iprot.Skip(fieldTypeId); err != nil { + return err + } + } + if err := iprot.ReadFieldEnd(); err != nil { + return err + } } - if err = args.Write(oprot); err != nil { - return + if err := iprot.ReadStructEnd(); err != nil { + return thrift.PrependError(fmt.Sprintf("%T read struct end error: ", p), err) } - if err = oprot.WriteMessageEnd(); err != nil { - return + if !issetQueries { + return thrift.NewTProtocolExceptionWithType(thrift.INVALID_DATA, fmt.Errorf("Required field Queries is not set")) } - return oprot.Flush() + return nil } -func (p *NodeClient) recvQuery() (value *QueryResult_, err error) { - iprot := p.InputProtocol - if iprot == nil { - iprot = p.ProtocolFactory.GetProtocol(p.Transport) - p.InputProtocol = iprot - } - method, mTypeId, seqId, err := iprot.ReadMessageBegin() +func (p *ConjunctionQuery) ReadField1(iprot thrift.TProtocol) error { + _, size, err := iprot.ReadListBegin() if err != nil { - return - } - if method != "query" { - err = thrift.NewTApplicationException(thrift.WRONG_METHOD_NAME, "query failed: wrong method name") - return - } - if p.SeqId != seqId { - err = thrift.NewTApplicationException(thrift.BAD_SEQUENCE_ID, "query failed: out of sequence response") - return + return thrift.PrependError("error reading list begin: ", err) } - if mTypeId == thrift.EXCEPTION { - error27 := thrift.NewTApplicationException(thrift.UNKNOWN_APPLICATION_EXCEPTION, "Unknown Exception") - var error28 error - error28, err = error27.Read(iprot) - if err != nil { - return - } - if err = iprot.ReadMessageEnd(); err != nil { - return + tSlice := make([]*Query, 0, size) + p.Queries = tSlice + for i := 0; i < size; i++ { + _elem29 := &Query{} + if err := _elem29.Read(iprot); err != nil { + return thrift.PrependError(fmt.Sprintf("%T error reading struct: ", _elem29), err) } - err = error28 - return + p.Queries = append(p.Queries, _elem29) } - if mTypeId != thrift.REPLY { - err = thrift.NewTApplicationException(thrift.INVALID_MESSAGE_TYPE_EXCEPTION, "query failed: invalid message type") - return + if err := iprot.ReadListEnd(); err != nil { + return thrift.PrependError("error reading list end: ", err) } - result := NodeQueryResult{} - if err = result.Read(iprot); err != nil { - return + return nil +} + +func (p *ConjunctionQuery) Write(oprot thrift.TProtocol) error { + if err := oprot.WriteStructBegin("ConjunctionQuery"); err != nil { + return thrift.PrependError(fmt.Sprintf("%T write struct begin error: ", p), err) } - if err = iprot.ReadMessageEnd(); err != nil { - return + if p != nil { + if err := p.writeField1(oprot); err != nil { + return err + } } - if result.Err != nil { - err = result.Err - return + if err := oprot.WriteFieldStop(); err != nil { + return thrift.PrependError("write field stop error: ", err) } - value = result.GetSuccess() - return -} - -// Parameters: -// - Req -func (p *NodeClient) AggregateRaw(req *AggregateQueryRawRequest) (r *AggregateQueryRawResult_, err error) { - if err = p.sendAggregateRaw(req); err != nil { - return + if err := oprot.WriteStructEnd(); err != nil { + return thrift.PrependError("write struct stop error: ", err) } - return p.recvAggregateRaw() + return nil } -func (p *NodeClient) sendAggregateRaw(req *AggregateQueryRawRequest) (err error) { - oprot := p.OutputProtocol - if oprot == nil { - oprot = p.ProtocolFactory.GetProtocol(p.Transport) - p.OutputProtocol = oprot +func (p *ConjunctionQuery) writeField1(oprot thrift.TProtocol) (err error) { + if err := oprot.WriteFieldBegin("queries", thrift.LIST, 1); err != nil { + return thrift.PrependError(fmt.Sprintf("%T write field begin error 1:queries: ", p), err) } - p.SeqId++ - if err = oprot.WriteMessageBegin("aggregateRaw", thrift.CALL, p.SeqId); err != nil { - return + if err := oprot.WriteListBegin(thrift.STRUCT, len(p.Queries)); err != nil { + return thrift.PrependError("error writing list begin: ", err) } - args := NodeAggregateRawArgs{ - Req: req, + for _, v := range p.Queries { + if err := v.Write(oprot); err != nil { + return thrift.PrependError(fmt.Sprintf("%T error writing struct: ", v), err) + } } - if err = args.Write(oprot); err != nil { - return + if err := oprot.WriteListEnd(); err != nil { + return thrift.PrependError("error writing list end: ", err) } - if err = oprot.WriteMessageEnd(); err != nil { - return + if err := oprot.WriteFieldEnd(); err != nil { + return thrift.PrependError(fmt.Sprintf("%T write field end error 1:queries: ", p), err) } - return oprot.Flush() + return err } -func (p *NodeClient) recvAggregateRaw() (value *AggregateQueryRawResult_, err error) { - iprot := p.InputProtocol - if iprot == nil { - iprot = p.ProtocolFactory.GetProtocol(p.Transport) - p.InputProtocol = iprot - } - method, mTypeId, seqId, err := iprot.ReadMessageBegin() - if err != nil { - return - } - if method != "aggregateRaw" { - err = thrift.NewTApplicationException(thrift.WRONG_METHOD_NAME, "aggregateRaw failed: wrong method name") - return +func (p *ConjunctionQuery) String() string { + if p == nil { + return "" } - if p.SeqId != seqId { - err = thrift.NewTApplicationException(thrift.BAD_SEQUENCE_ID, "aggregateRaw failed: out of sequence response") - return + return fmt.Sprintf("ConjunctionQuery(%+v)", *p) +} + +// Attributes: +// - Queries +type DisjunctionQuery struct { + Queries []*Query `thrift:"queries,1,required" db:"queries" json:"queries"` +} + +func NewDisjunctionQuery() *DisjunctionQuery { + return &DisjunctionQuery{} +} + +func (p *DisjunctionQuery) GetQueries() []*Query { + return p.Queries +} +func (p *DisjunctionQuery) Read(iprot thrift.TProtocol) error { + if _, err := iprot.ReadStructBegin(); err != nil { + return thrift.PrependError(fmt.Sprintf("%T read error: ", p), err) } - if mTypeId == thrift.EXCEPTION { - error29 := thrift.NewTApplicationException(thrift.UNKNOWN_APPLICATION_EXCEPTION, "Unknown Exception") - var error30 error - error30, err = error29.Read(iprot) + + var issetQueries bool = false + + for { + _, fieldTypeId, fieldId, err := iprot.ReadFieldBegin() if err != nil { - return + return thrift.PrependError(fmt.Sprintf("%T field %d read error: ", p, fieldId), err) } - if err = iprot.ReadMessageEnd(); err != nil { - return + if fieldTypeId == thrift.STOP { + break + } + switch fieldId { + case 1: + if err := p.ReadField1(iprot); err != nil { + return err + } + issetQueries = true + default: + if err := iprot.Skip(fieldTypeId); err != nil { + return err + } + } + if err := iprot.ReadFieldEnd(); err != nil { + return err } - err = error30 - return - } - if mTypeId != thrift.REPLY { - err = thrift.NewTApplicationException(thrift.INVALID_MESSAGE_TYPE_EXCEPTION, "aggregateRaw failed: invalid message type") - return - } - result := NodeAggregateRawResult{} - if err = result.Read(iprot); err != nil { - return } - if err = iprot.ReadMessageEnd(); err != nil { - return + if err := iprot.ReadStructEnd(); err != nil { + return thrift.PrependError(fmt.Sprintf("%T read struct end error: ", p), err) } - if result.Err != nil { - err = result.Err - return + if !issetQueries { + return thrift.NewTProtocolExceptionWithType(thrift.INVALID_DATA, fmt.Errorf("Required field Queries is not set")) } - value = result.GetSuccess() - return + return nil } -// Parameters: -// - Req -func (p *NodeClient) Aggregate(req *AggregateQueryRequest) (r *AggregateQueryResult_, err error) { - if err = p.sendAggregate(req); err != nil { - return +func (p *DisjunctionQuery) ReadField1(iprot thrift.TProtocol) error { + _, size, err := iprot.ReadListBegin() + if err != nil { + return thrift.PrependError("error reading list begin: ", err) } - return p.recvAggregate() + tSlice := make([]*Query, 0, size) + p.Queries = tSlice + for i := 0; i < size; i++ { + _elem30 := &Query{} + if err := _elem30.Read(iprot); err != nil { + return thrift.PrependError(fmt.Sprintf("%T error reading struct: ", _elem30), err) + } + p.Queries = append(p.Queries, _elem30) + } + if err := iprot.ReadListEnd(); err != nil { + return thrift.PrependError("error reading list end: ", err) + } + return nil } -func (p *NodeClient) sendAggregate(req *AggregateQueryRequest) (err error) { - oprot := p.OutputProtocol - if oprot == nil { - oprot = p.ProtocolFactory.GetProtocol(p.Transport) - p.OutputProtocol = oprot - } - p.SeqId++ - if err = oprot.WriteMessageBegin("aggregate", thrift.CALL, p.SeqId); err != nil { - return +func (p *DisjunctionQuery) Write(oprot thrift.TProtocol) error { + if err := oprot.WriteStructBegin("DisjunctionQuery"); err != nil { + return thrift.PrependError(fmt.Sprintf("%T write struct begin error: ", p), err) } - args := NodeAggregateArgs{ - Req: req, + if p != nil { + if err := p.writeField1(oprot); err != nil { + return err + } } - if err = args.Write(oprot); err != nil { - return + if err := oprot.WriteFieldStop(); err != nil { + return thrift.PrependError("write field stop error: ", err) } - if err = oprot.WriteMessageEnd(); err != nil { - return + if err := oprot.WriteStructEnd(); err != nil { + return thrift.PrependError("write struct stop error: ", err) } - return oprot.Flush() + return nil } -func (p *NodeClient) recvAggregate() (value *AggregateQueryResult_, err error) { - iprot := p.InputProtocol - if iprot == nil { - iprot = p.ProtocolFactory.GetProtocol(p.Transport) - p.InputProtocol = iprot - } - method, mTypeId, seqId, err := iprot.ReadMessageBegin() - if err != nil { - return - } - if method != "aggregate" { - err = thrift.NewTApplicationException(thrift.WRONG_METHOD_NAME, "aggregate failed: wrong method name") - return +func (p *DisjunctionQuery) writeField1(oprot thrift.TProtocol) (err error) { + if err := oprot.WriteFieldBegin("queries", thrift.LIST, 1); err != nil { + return thrift.PrependError(fmt.Sprintf("%T write field begin error 1:queries: ", p), err) } - if p.SeqId != seqId { - err = thrift.NewTApplicationException(thrift.BAD_SEQUENCE_ID, "aggregate failed: out of sequence response") - return + if err := oprot.WriteListBegin(thrift.STRUCT, len(p.Queries)); err != nil { + return thrift.PrependError("error writing list begin: ", err) } - if mTypeId == thrift.EXCEPTION { - error31 := thrift.NewTApplicationException(thrift.UNKNOWN_APPLICATION_EXCEPTION, "Unknown Exception") - var error32 error - error32, err = error31.Read(iprot) - if err != nil { - return - } - if err = iprot.ReadMessageEnd(); err != nil { - return + for _, v := range p.Queries { + if err := v.Write(oprot); err != nil { + return thrift.PrependError(fmt.Sprintf("%T error writing struct: ", v), err) } - err = error32 - return - } - if mTypeId != thrift.REPLY { - err = thrift.NewTApplicationException(thrift.INVALID_MESSAGE_TYPE_EXCEPTION, "aggregate failed: invalid message type") - return - } - result := NodeAggregateResult{} - if err = result.Read(iprot); err != nil { - return } - if err = iprot.ReadMessageEnd(); err != nil { - return + if err := oprot.WriteListEnd(); err != nil { + return thrift.PrependError("error writing list end: ", err) } - if result.Err != nil { - err = result.Err - return + if err := oprot.WriteFieldEnd(); err != nil { + return thrift.PrependError(fmt.Sprintf("%T write field end error 1:queries: ", p), err) } - value = result.GetSuccess() - return + return err } -// Parameters: -// - Req -func (p *NodeClient) Fetch(req *FetchRequest) (r *FetchResult_, err error) { - if err = p.sendFetch(req); err != nil { - return +func (p *DisjunctionQuery) String() string { + if p == nil { + return "" } - return p.recvFetch() + return fmt.Sprintf("DisjunctionQuery(%+v)", *p) } -func (p *NodeClient) sendFetch(req *FetchRequest) (err error) { - oprot := p.OutputProtocol - if oprot == nil { - oprot = p.ProtocolFactory.GetProtocol(p.Transport) - p.OutputProtocol = oprot - } - p.SeqId++ - if err = oprot.WriteMessageBegin("fetch", thrift.CALL, p.SeqId); err != nil { - return - } - args := NodeFetchArgs{ - Req: req, - } - if err = args.Write(oprot); err != nil { - return - } - if err = oprot.WriteMessageEnd(); err != nil { - return - } - return oprot.Flush() +type AllQuery struct { } -func (p *NodeClient) recvFetch() (value *FetchResult_, err error) { - iprot := p.InputProtocol - if iprot == nil { - iprot = p.ProtocolFactory.GetProtocol(p.Transport) - p.InputProtocol = iprot - } - method, mTypeId, seqId, err := iprot.ReadMessageBegin() - if err != nil { - return - } - if method != "fetch" { - err = thrift.NewTApplicationException(thrift.WRONG_METHOD_NAME, "fetch failed: wrong method name") - return - } - if p.SeqId != seqId { - err = thrift.NewTApplicationException(thrift.BAD_SEQUENCE_ID, "fetch failed: out of sequence response") - return +func NewAllQuery() *AllQuery { + return &AllQuery{} +} + +func (p *AllQuery) Read(iprot thrift.TProtocol) error { + if _, err := iprot.ReadStructBegin(); err != nil { + return thrift.PrependError(fmt.Sprintf("%T read error: ", p), err) } - if mTypeId == thrift.EXCEPTION { - error33 := thrift.NewTApplicationException(thrift.UNKNOWN_APPLICATION_EXCEPTION, "Unknown Exception") - var error34 error - error34, err = error33.Read(iprot) + + for { + _, fieldTypeId, fieldId, err := iprot.ReadFieldBegin() if err != nil { - return + return thrift.PrependError(fmt.Sprintf("%T field %d read error: ", p, fieldId), err) } - if err = iprot.ReadMessageEnd(); err != nil { - return + if fieldTypeId == thrift.STOP { + break + } + if err := iprot.Skip(fieldTypeId); err != nil { + return err + } + if err := iprot.ReadFieldEnd(); err != nil { + return err } - err = error34 - return } - if mTypeId != thrift.REPLY { - err = thrift.NewTApplicationException(thrift.INVALID_MESSAGE_TYPE_EXCEPTION, "fetch failed: invalid message type") - return + if err := iprot.ReadStructEnd(); err != nil { + return thrift.PrependError(fmt.Sprintf("%T read struct end error: ", p), err) } - result := NodeFetchResult{} - if err = result.Read(iprot); err != nil { - return + return nil +} + +func (p *AllQuery) Write(oprot thrift.TProtocol) error { + if err := oprot.WriteStructBegin("AllQuery"); err != nil { + return thrift.PrependError(fmt.Sprintf("%T write struct begin error: ", p), err) } - if err = iprot.ReadMessageEnd(); err != nil { - return + if p != nil { } - if result.Err != nil { - err = result.Err - return + if err := oprot.WriteFieldStop(); err != nil { + return thrift.PrependError("write field stop error: ", err) } - value = result.GetSuccess() - return + if err := oprot.WriteStructEnd(); err != nil { + return thrift.PrependError("write struct stop error: ", err) + } + return nil } -// Parameters: -// - Req -func (p *NodeClient) FetchTagged(req *FetchTaggedRequest) (r *FetchTaggedResult_, err error) { - if err = p.sendFetchTagged(req); err != nil { - return +func (p *AllQuery) String() string { + if p == nil { + return "" } - return p.recvFetchTagged() + return fmt.Sprintf("AllQuery(%+v)", *p) } -func (p *NodeClient) sendFetchTagged(req *FetchTaggedRequest) (err error) { - oprot := p.OutputProtocol - if oprot == nil { - oprot = p.ProtocolFactory.GetProtocol(p.Transport) - p.OutputProtocol = oprot - } - p.SeqId++ - if err = oprot.WriteMessageBegin("fetchTagged", thrift.CALL, p.SeqId); err != nil { - return - } - args := NodeFetchTaggedArgs{ - Req: req, - } - if err = args.Write(oprot); err != nil { - return - } - if err = oprot.WriteMessageEnd(); err != nil { - return - } - return oprot.Flush() +// Attributes: +// - Field +type FieldQuery struct { + Field string `thrift:"field,1,required" db:"field" json:"field"` } -func (p *NodeClient) recvFetchTagged() (value *FetchTaggedResult_, err error) { - iprot := p.InputProtocol - if iprot == nil { - iprot = p.ProtocolFactory.GetProtocol(p.Transport) - p.InputProtocol = iprot +func NewFieldQuery() *FieldQuery { + return &FieldQuery{} +} + +func (p *FieldQuery) GetField() string { + return p.Field +} +func (p *FieldQuery) Read(iprot thrift.TProtocol) error { + if _, err := iprot.ReadStructBegin(); err != nil { + return thrift.PrependError(fmt.Sprintf("%T read error: ", p), err) } - method, mTypeId, seqId, err := iprot.ReadMessageBegin() - if err != nil { - return - } - if method != "fetchTagged" { - err = thrift.NewTApplicationException(thrift.WRONG_METHOD_NAME, "fetchTagged failed: wrong method name") - return - } - if p.SeqId != seqId { - err = thrift.NewTApplicationException(thrift.BAD_SEQUENCE_ID, "fetchTagged failed: out of sequence response") - return - } - if mTypeId == thrift.EXCEPTION { - error35 := thrift.NewTApplicationException(thrift.UNKNOWN_APPLICATION_EXCEPTION, "Unknown Exception") - var error36 error - error36, err = error35.Read(iprot) + + var issetField bool = false + + for { + _, fieldTypeId, fieldId, err := iprot.ReadFieldBegin() if err != nil { - return + return thrift.PrependError(fmt.Sprintf("%T field %d read error: ", p, fieldId), err) } - if err = iprot.ReadMessageEnd(); err != nil { - return + if fieldTypeId == thrift.STOP { + break + } + switch fieldId { + case 1: + if err := p.ReadField1(iprot); err != nil { + return err + } + issetField = true + default: + if err := iprot.Skip(fieldTypeId); err != nil { + return err + } + } + if err := iprot.ReadFieldEnd(); err != nil { + return err } - err = error36 - return - } - if mTypeId != thrift.REPLY { - err = thrift.NewTApplicationException(thrift.INVALID_MESSAGE_TYPE_EXCEPTION, "fetchTagged failed: invalid message type") - return - } - result := NodeFetchTaggedResult{} - if err = result.Read(iprot); err != nil { - return } - if err = iprot.ReadMessageEnd(); err != nil { - return + if err := iprot.ReadStructEnd(); err != nil { + return thrift.PrependError(fmt.Sprintf("%T read struct end error: ", p), err) } - if result.Err != nil { - err = result.Err - return + if !issetField { + return thrift.NewTProtocolExceptionWithType(thrift.INVALID_DATA, fmt.Errorf("Required field Field is not set")) } - value = result.GetSuccess() - return + return nil } -// Parameters: -// - Req -func (p *NodeClient) Write(req *WriteRequest) (err error) { - if err = p.sendWrite(req); err != nil { - return +func (p *FieldQuery) ReadField1(iprot thrift.TProtocol) error { + if v, err := iprot.ReadString(); err != nil { + return thrift.PrependError("error reading field 1: ", err) + } else { + p.Field = v } - return p.recvWrite() + return nil } -func (p *NodeClient) sendWrite(req *WriteRequest) (err error) { - oprot := p.OutputProtocol - if oprot == nil { - oprot = p.ProtocolFactory.GetProtocol(p.Transport) - p.OutputProtocol = oprot - } - p.SeqId++ - if err = oprot.WriteMessageBegin("write", thrift.CALL, p.SeqId); err != nil { - return +func (p *FieldQuery) Write(oprot thrift.TProtocol) error { + if err := oprot.WriteStructBegin("FieldQuery"); err != nil { + return thrift.PrependError(fmt.Sprintf("%T write struct begin error: ", p), err) } - args := NodeWriteArgs{ - Req: req, + if p != nil { + if err := p.writeField1(oprot); err != nil { + return err + } } - if err = args.Write(oprot); err != nil { - return + if err := oprot.WriteFieldStop(); err != nil { + return thrift.PrependError("write field stop error: ", err) } - if err = oprot.WriteMessageEnd(); err != nil { - return + if err := oprot.WriteStructEnd(); err != nil { + return thrift.PrependError("write struct stop error: ", err) } - return oprot.Flush() + return nil } -func (p *NodeClient) recvWrite() (err error) { - iprot := p.InputProtocol - if iprot == nil { - iprot = p.ProtocolFactory.GetProtocol(p.Transport) - p.InputProtocol = iprot - } - method, mTypeId, seqId, err := iprot.ReadMessageBegin() - if err != nil { - return - } - if method != "write" { - err = thrift.NewTApplicationException(thrift.WRONG_METHOD_NAME, "write failed: wrong method name") - return - } - if p.SeqId != seqId { - err = thrift.NewTApplicationException(thrift.BAD_SEQUENCE_ID, "write failed: out of sequence response") - return - } - if mTypeId == thrift.EXCEPTION { - error37 := thrift.NewTApplicationException(thrift.UNKNOWN_APPLICATION_EXCEPTION, "Unknown Exception") - var error38 error - error38, err = error37.Read(iprot) - if err != nil { - return - } - if err = iprot.ReadMessageEnd(); err != nil { - return - } - err = error38 - return - } - if mTypeId != thrift.REPLY { - err = thrift.NewTApplicationException(thrift.INVALID_MESSAGE_TYPE_EXCEPTION, "write failed: invalid message type") - return +func (p *FieldQuery) writeField1(oprot thrift.TProtocol) (err error) { + if err := oprot.WriteFieldBegin("field", thrift.STRING, 1); err != nil { + return thrift.PrependError(fmt.Sprintf("%T write field begin error 1:field: ", p), err) } - result := NodeWriteResult{} - if err = result.Read(iprot); err != nil { - return + if err := oprot.WriteString(string(p.Field)); err != nil { + return thrift.PrependError(fmt.Sprintf("%T.field (1) field write error: ", p), err) } - if err = iprot.ReadMessageEnd(); err != nil { - return + if err := oprot.WriteFieldEnd(); err != nil { + return thrift.PrependError(fmt.Sprintf("%T write field end error 1:field: ", p), err) } - if result.Err != nil { - err = result.Err - return + return err +} + +func (p *FieldQuery) String() string { + if p == nil { + return "" } - return + return fmt.Sprintf("FieldQuery(%+v)", *p) } -// Parameters: -// - Req -func (p *NodeClient) WriteTagged(req *WriteTaggedRequest) (err error) { - if err = p.sendWriteTagged(req); err != nil { - return +// Attributes: +// - Term +// - Regexp +// - Negation +// - Conjunction +// - Disjunction +// - All +// - Field +type Query struct { + Term *TermQuery `thrift:"term,1" db:"term" json:"term,omitempty"` + Regexp *RegexpQuery `thrift:"regexp,2" db:"regexp" json:"regexp,omitempty"` + Negation *NegationQuery `thrift:"negation,3" db:"negation" json:"negation,omitempty"` + Conjunction *ConjunctionQuery `thrift:"conjunction,4" db:"conjunction" json:"conjunction,omitempty"` + Disjunction *DisjunctionQuery `thrift:"disjunction,5" db:"disjunction" json:"disjunction,omitempty"` + All *AllQuery `thrift:"all,6" db:"all" json:"all,omitempty"` + Field *FieldQuery `thrift:"field,7" db:"field" json:"field,omitempty"` +} + +func NewQuery() *Query { + return &Query{} +} + +var Query_Term_DEFAULT *TermQuery + +func (p *Query) GetTerm() *TermQuery { + if !p.IsSetTerm() { + return Query_Term_DEFAULT } - return p.recvWriteTagged() + return p.Term } -func (p *NodeClient) sendWriteTagged(req *WriteTaggedRequest) (err error) { - oprot := p.OutputProtocol - if oprot == nil { - oprot = p.ProtocolFactory.GetProtocol(p.Transport) - p.OutputProtocol = oprot +var Query_Regexp_DEFAULT *RegexpQuery + +func (p *Query) GetRegexp() *RegexpQuery { + if !p.IsSetRegexp() { + return Query_Regexp_DEFAULT } - p.SeqId++ - if err = oprot.WriteMessageBegin("writeTagged", thrift.CALL, p.SeqId); err != nil { - return + return p.Regexp +} + +var Query_Negation_DEFAULT *NegationQuery + +func (p *Query) GetNegation() *NegationQuery { + if !p.IsSetNegation() { + return Query_Negation_DEFAULT } - args := NodeWriteTaggedArgs{ - Req: req, + return p.Negation +} + +var Query_Conjunction_DEFAULT *ConjunctionQuery + +func (p *Query) GetConjunction() *ConjunctionQuery { + if !p.IsSetConjunction() { + return Query_Conjunction_DEFAULT } - if err = args.Write(oprot); err != nil { - return + return p.Conjunction +} + +var Query_Disjunction_DEFAULT *DisjunctionQuery + +func (p *Query) GetDisjunction() *DisjunctionQuery { + if !p.IsSetDisjunction() { + return Query_Disjunction_DEFAULT } - if err = oprot.WriteMessageEnd(); err != nil { - return + return p.Disjunction +} + +var Query_All_DEFAULT *AllQuery + +func (p *Query) GetAll() *AllQuery { + if !p.IsSetAll() { + return Query_All_DEFAULT } - return oprot.Flush() + return p.All } -func (p *NodeClient) recvWriteTagged() (err error) { - iprot := p.InputProtocol - if iprot == nil { - iprot = p.ProtocolFactory.GetProtocol(p.Transport) - p.InputProtocol = iprot +var Query_Field_DEFAULT *FieldQuery + +func (p *Query) GetField() *FieldQuery { + if !p.IsSetField() { + return Query_Field_DEFAULT + } + return p.Field +} +func (p *Query) IsSetTerm() bool { + return p.Term != nil +} + +func (p *Query) IsSetRegexp() bool { + return p.Regexp != nil +} + +func (p *Query) IsSetNegation() bool { + return p.Negation != nil +} + +func (p *Query) IsSetConjunction() bool { + return p.Conjunction != nil +} + +func (p *Query) IsSetDisjunction() bool { + return p.Disjunction != nil +} + +func (p *Query) IsSetAll() bool { + return p.All != nil +} + +func (p *Query) IsSetField() bool { + return p.Field != nil +} + +func (p *Query) Read(iprot thrift.TProtocol) error { + if _, err := iprot.ReadStructBegin(); err != nil { + return thrift.PrependError(fmt.Sprintf("%T read error: ", p), err) + } + + for { + _, fieldTypeId, fieldId, err := iprot.ReadFieldBegin() + if err != nil { + return thrift.PrependError(fmt.Sprintf("%T field %d read error: ", p, fieldId), err) + } + if fieldTypeId == thrift.STOP { + break + } + switch fieldId { + case 1: + if err := p.ReadField1(iprot); err != nil { + return err + } + case 2: + if err := p.ReadField2(iprot); err != nil { + return err + } + case 3: + if err := p.ReadField3(iprot); err != nil { + return err + } + case 4: + if err := p.ReadField4(iprot); err != nil { + return err + } + case 5: + if err := p.ReadField5(iprot); err != nil { + return err + } + case 6: + if err := p.ReadField6(iprot); err != nil { + return err + } + case 7: + if err := p.ReadField7(iprot); err != nil { + return err + } + default: + if err := iprot.Skip(fieldTypeId); err != nil { + return err + } + } + if err := iprot.ReadFieldEnd(); err != nil { + return err + } + } + if err := iprot.ReadStructEnd(); err != nil { + return thrift.PrependError(fmt.Sprintf("%T read struct end error: ", p), err) + } + return nil +} + +func (p *Query) ReadField1(iprot thrift.TProtocol) error { + p.Term = &TermQuery{} + if err := p.Term.Read(iprot); err != nil { + return thrift.PrependError(fmt.Sprintf("%T error reading struct: ", p.Term), err) + } + return nil +} + +func (p *Query) ReadField2(iprot thrift.TProtocol) error { + p.Regexp = &RegexpQuery{} + if err := p.Regexp.Read(iprot); err != nil { + return thrift.PrependError(fmt.Sprintf("%T error reading struct: ", p.Regexp), err) + } + return nil +} + +func (p *Query) ReadField3(iprot thrift.TProtocol) error { + p.Negation = &NegationQuery{} + if err := p.Negation.Read(iprot); err != nil { + return thrift.PrependError(fmt.Sprintf("%T error reading struct: ", p.Negation), err) + } + return nil +} + +func (p *Query) ReadField4(iprot thrift.TProtocol) error { + p.Conjunction = &ConjunctionQuery{} + if err := p.Conjunction.Read(iprot); err != nil { + return thrift.PrependError(fmt.Sprintf("%T error reading struct: ", p.Conjunction), err) + } + return nil +} + +func (p *Query) ReadField5(iprot thrift.TProtocol) error { + p.Disjunction = &DisjunctionQuery{} + if err := p.Disjunction.Read(iprot); err != nil { + return thrift.PrependError(fmt.Sprintf("%T error reading struct: ", p.Disjunction), err) + } + return nil +} + +func (p *Query) ReadField6(iprot thrift.TProtocol) error { + p.All = &AllQuery{} + if err := p.All.Read(iprot); err != nil { + return thrift.PrependError(fmt.Sprintf("%T error reading struct: ", p.All), err) + } + return nil +} + +func (p *Query) ReadField7(iprot thrift.TProtocol) error { + p.Field = &FieldQuery{} + if err := p.Field.Read(iprot); err != nil { + return thrift.PrependError(fmt.Sprintf("%T error reading struct: ", p.Field), err) + } + return nil +} + +func (p *Query) Write(oprot thrift.TProtocol) error { + if err := oprot.WriteStructBegin("Query"); err != nil { + return thrift.PrependError(fmt.Sprintf("%T write struct begin error: ", p), err) + } + if p != nil { + if err := p.writeField1(oprot); err != nil { + return err + } + if err := p.writeField2(oprot); err != nil { + return err + } + if err := p.writeField3(oprot); err != nil { + return err + } + if err := p.writeField4(oprot); err != nil { + return err + } + if err := p.writeField5(oprot); err != nil { + return err + } + if err := p.writeField6(oprot); err != nil { + return err + } + if err := p.writeField7(oprot); err != nil { + return err + } + } + if err := oprot.WriteFieldStop(); err != nil { + return thrift.PrependError("write field stop error: ", err) + } + if err := oprot.WriteStructEnd(); err != nil { + return thrift.PrependError("write struct stop error: ", err) + } + return nil +} + +func (p *Query) writeField1(oprot thrift.TProtocol) (err error) { + if p.IsSetTerm() { + if err := oprot.WriteFieldBegin("term", thrift.STRUCT, 1); err != nil { + return thrift.PrependError(fmt.Sprintf("%T write field begin error 1:term: ", p), err) + } + if err := p.Term.Write(oprot); err != nil { + return thrift.PrependError(fmt.Sprintf("%T error writing struct: ", p.Term), err) + } + if err := oprot.WriteFieldEnd(); err != nil { + return thrift.PrependError(fmt.Sprintf("%T write field end error 1:term: ", p), err) + } + } + return err +} + +func (p *Query) writeField2(oprot thrift.TProtocol) (err error) { + if p.IsSetRegexp() { + if err := oprot.WriteFieldBegin("regexp", thrift.STRUCT, 2); err != nil { + return thrift.PrependError(fmt.Sprintf("%T write field begin error 2:regexp: ", p), err) + } + if err := p.Regexp.Write(oprot); err != nil { + return thrift.PrependError(fmt.Sprintf("%T error writing struct: ", p.Regexp), err) + } + if err := oprot.WriteFieldEnd(); err != nil { + return thrift.PrependError(fmt.Sprintf("%T write field end error 2:regexp: ", p), err) + } + } + return err +} + +func (p *Query) writeField3(oprot thrift.TProtocol) (err error) { + if p.IsSetNegation() { + if err := oprot.WriteFieldBegin("negation", thrift.STRUCT, 3); err != nil { + return thrift.PrependError(fmt.Sprintf("%T write field begin error 3:negation: ", p), err) + } + if err := p.Negation.Write(oprot); err != nil { + return thrift.PrependError(fmt.Sprintf("%T error writing struct: ", p.Negation), err) + } + if err := oprot.WriteFieldEnd(); err != nil { + return thrift.PrependError(fmt.Sprintf("%T write field end error 3:negation: ", p), err) + } + } + return err +} + +func (p *Query) writeField4(oprot thrift.TProtocol) (err error) { + if p.IsSetConjunction() { + if err := oprot.WriteFieldBegin("conjunction", thrift.STRUCT, 4); err != nil { + return thrift.PrependError(fmt.Sprintf("%T write field begin error 4:conjunction: ", p), err) + } + if err := p.Conjunction.Write(oprot); err != nil { + return thrift.PrependError(fmt.Sprintf("%T error writing struct: ", p.Conjunction), err) + } + if err := oprot.WriteFieldEnd(); err != nil { + return thrift.PrependError(fmt.Sprintf("%T write field end error 4:conjunction: ", p), err) + } + } + return err +} + +func (p *Query) writeField5(oprot thrift.TProtocol) (err error) { + if p.IsSetDisjunction() { + if err := oprot.WriteFieldBegin("disjunction", thrift.STRUCT, 5); err != nil { + return thrift.PrependError(fmt.Sprintf("%T write field begin error 5:disjunction: ", p), err) + } + if err := p.Disjunction.Write(oprot); err != nil { + return thrift.PrependError(fmt.Sprintf("%T error writing struct: ", p.Disjunction), err) + } + if err := oprot.WriteFieldEnd(); err != nil { + return thrift.PrependError(fmt.Sprintf("%T write field end error 5:disjunction: ", p), err) + } + } + return err +} + +func (p *Query) writeField6(oprot thrift.TProtocol) (err error) { + if p.IsSetAll() { + if err := oprot.WriteFieldBegin("all", thrift.STRUCT, 6); err != nil { + return thrift.PrependError(fmt.Sprintf("%T write field begin error 6:all: ", p), err) + } + if err := p.All.Write(oprot); err != nil { + return thrift.PrependError(fmt.Sprintf("%T error writing struct: ", p.All), err) + } + if err := oprot.WriteFieldEnd(); err != nil { + return thrift.PrependError(fmt.Sprintf("%T write field end error 6:all: ", p), err) + } + } + return err +} + +func (p *Query) writeField7(oprot thrift.TProtocol) (err error) { + if p.IsSetField() { + if err := oprot.WriteFieldBegin("field", thrift.STRUCT, 7); err != nil { + return thrift.PrependError(fmt.Sprintf("%T write field begin error 7:field: ", p), err) + } + if err := p.Field.Write(oprot); err != nil { + return thrift.PrependError(fmt.Sprintf("%T error writing struct: ", p.Field), err) + } + if err := oprot.WriteFieldEnd(); err != nil { + return thrift.PrependError(fmt.Sprintf("%T write field end error 7:field: ", p), err) + } + } + return err +} + +func (p *Query) String() string { + if p == nil { + return "" + } + return fmt.Sprintf("Query(%+v)", *p) +} + +type Node interface { + // Parameters: + // - Req + Query(req *QueryRequest) (r *QueryResult_, err error) + // Parameters: + // - Req + AggregateRaw(req *AggregateQueryRawRequest) (r *AggregateQueryRawResult_, err error) + // Parameters: + // - Req + Aggregate(req *AggregateQueryRequest) (r *AggregateQueryResult_, err error) + // Parameters: + // - Req + Fetch(req *FetchRequest) (r *FetchResult_, err error) + // Parameters: + // - Req + FetchTagged(req *FetchTaggedRequest) (r *FetchTaggedResult_, err error) + // Parameters: + // - Req + Write(req *WriteRequest) (err error) + // Parameters: + // - Req + WriteTagged(req *WriteTaggedRequest) (err error) + // Parameters: + // - Req + FetchBatchRaw(req *FetchBatchRawRequest) (r *FetchBatchRawResult_, err error) + // Parameters: + // - Req + FetchBlocksRaw(req *FetchBlocksRawRequest) (r *FetchBlocksRawResult_, err error) + // Parameters: + // - Req + FetchBlocksMetadataRawV2(req *FetchBlocksMetadataRawV2Request) (r *FetchBlocksMetadataRawV2Result_, err error) + // Parameters: + // - Req + WriteBatchRaw(req *WriteBatchRawRequest) (err error) + // Parameters: + // - Req + WriteBatchRawV2(req *WriteBatchRawV2Request) (err error) + // Parameters: + // - Req + WriteTaggedBatchRaw(req *WriteTaggedBatchRawRequest) (err error) + // Parameters: + // - Req + WriteTaggedBatchRawV2(req *WriteTaggedBatchRawV2Request) (err error) + Repair() (err error) + // Parameters: + // - Req + Truncate(req *TruncateRequest) (r *TruncateResult_, err error) + Health() (r *NodeHealthResult_, err error) + Bootstrapped() (r *NodeBootstrappedResult_, err error) + BootstrappedInPlacementOrNoPlacement() (r *NodeBootstrappedInPlacementOrNoPlacementResult_, err error) + GetPersistRateLimit() (r *NodePersistRateLimitResult_, err error) + // Parameters: + // - Req + SetPersistRateLimit(req *NodeSetPersistRateLimitRequest) (r *NodePersistRateLimitResult_, err error) + GetWriteNewSeriesAsync() (r *NodeWriteNewSeriesAsyncResult_, err error) + // Parameters: + // - Req + SetWriteNewSeriesAsync(req *NodeSetWriteNewSeriesAsyncRequest) (r *NodeWriteNewSeriesAsyncResult_, err error) + GetWriteNewSeriesBackoffDuration() (r *NodeWriteNewSeriesBackoffDurationResult_, err error) + // Parameters: + // - Req + SetWriteNewSeriesBackoffDuration(req *NodeSetWriteNewSeriesBackoffDurationRequest) (r *NodeWriteNewSeriesBackoffDurationResult_, err error) + GetWriteNewSeriesLimitPerShardPerSecond() (r *NodeWriteNewSeriesLimitPerShardPerSecondResult_, err error) + // Parameters: + // - Req + SetWriteNewSeriesLimitPerShardPerSecond(req *NodeSetWriteNewSeriesLimitPerShardPerSecondRequest) (r *NodeWriteNewSeriesLimitPerShardPerSecondResult_, err error) +} + +type NodeClient struct { + Transport thrift.TTransport + ProtocolFactory thrift.TProtocolFactory + InputProtocol thrift.TProtocol + OutputProtocol thrift.TProtocol + SeqId int32 +} + +func NewNodeClientFactory(t thrift.TTransport, f thrift.TProtocolFactory) *NodeClient { + return &NodeClient{Transport: t, + ProtocolFactory: f, + InputProtocol: f.GetProtocol(t), + OutputProtocol: f.GetProtocol(t), + SeqId: 0, + } +} + +func NewNodeClientProtocol(t thrift.TTransport, iprot thrift.TProtocol, oprot thrift.TProtocol) *NodeClient { + return &NodeClient{Transport: t, + ProtocolFactory: nil, + InputProtocol: iprot, + OutputProtocol: oprot, + SeqId: 0, + } +} + +// Parameters: +// - Req +func (p *NodeClient) Query(req *QueryRequest) (r *QueryResult_, err error) { + if err = p.sendQuery(req); err != nil { + return + } + return p.recvQuery() +} + +func (p *NodeClient) sendQuery(req *QueryRequest) (err error) { + oprot := p.OutputProtocol + if oprot == nil { + oprot = p.ProtocolFactory.GetProtocol(p.Transport) + p.OutputProtocol = oprot + } + p.SeqId++ + if err = oprot.WriteMessageBegin("query", thrift.CALL, p.SeqId); err != nil { + return + } + args := NodeQueryArgs{ + Req: req, + } + if err = args.Write(oprot); err != nil { + return + } + if err = oprot.WriteMessageEnd(); err != nil { + return + } + return oprot.Flush() +} + +func (p *NodeClient) recvQuery() (value *QueryResult_, err error) { + iprot := p.InputProtocol + if iprot == nil { + iprot = p.ProtocolFactory.GetProtocol(p.Transport) + p.InputProtocol = iprot + } + method, mTypeId, seqId, err := iprot.ReadMessageBegin() + if err != nil { + return + } + if method != "query" { + err = thrift.NewTApplicationException(thrift.WRONG_METHOD_NAME, "query failed: wrong method name") + return + } + if p.SeqId != seqId { + err = thrift.NewTApplicationException(thrift.BAD_SEQUENCE_ID, "query failed: out of sequence response") + return + } + if mTypeId == thrift.EXCEPTION { + error31 := thrift.NewTApplicationException(thrift.UNKNOWN_APPLICATION_EXCEPTION, "Unknown Exception") + var error32 error + error32, err = error31.Read(iprot) + if err != nil { + return + } + if err = iprot.ReadMessageEnd(); err != nil { + return + } + err = error32 + return + } + if mTypeId != thrift.REPLY { + err = thrift.NewTApplicationException(thrift.INVALID_MESSAGE_TYPE_EXCEPTION, "query failed: invalid message type") + return + } + result := NodeQueryResult{} + if err = result.Read(iprot); err != nil { + return + } + if err = iprot.ReadMessageEnd(); err != nil { + return + } + if result.Err != nil { + err = result.Err + return + } + value = result.GetSuccess() + return +} + +// Parameters: +// - Req +func (p *NodeClient) AggregateRaw(req *AggregateQueryRawRequest) (r *AggregateQueryRawResult_, err error) { + if err = p.sendAggregateRaw(req); err != nil { + return + } + return p.recvAggregateRaw() +} + +func (p *NodeClient) sendAggregateRaw(req *AggregateQueryRawRequest) (err error) { + oprot := p.OutputProtocol + if oprot == nil { + oprot = p.ProtocolFactory.GetProtocol(p.Transport) + p.OutputProtocol = oprot + } + p.SeqId++ + if err = oprot.WriteMessageBegin("aggregateRaw", thrift.CALL, p.SeqId); err != nil { + return + } + args := NodeAggregateRawArgs{ + Req: req, + } + if err = args.Write(oprot); err != nil { + return + } + if err = oprot.WriteMessageEnd(); err != nil { + return + } + return oprot.Flush() +} + +func (p *NodeClient) recvAggregateRaw() (value *AggregateQueryRawResult_, err error) { + iprot := p.InputProtocol + if iprot == nil { + iprot = p.ProtocolFactory.GetProtocol(p.Transport) + p.InputProtocol = iprot + } + method, mTypeId, seqId, err := iprot.ReadMessageBegin() + if err != nil { + return + } + if method != "aggregateRaw" { + err = thrift.NewTApplicationException(thrift.WRONG_METHOD_NAME, "aggregateRaw failed: wrong method name") + return + } + if p.SeqId != seqId { + err = thrift.NewTApplicationException(thrift.BAD_SEQUENCE_ID, "aggregateRaw failed: out of sequence response") + return + } + if mTypeId == thrift.EXCEPTION { + error33 := thrift.NewTApplicationException(thrift.UNKNOWN_APPLICATION_EXCEPTION, "Unknown Exception") + var error34 error + error34, err = error33.Read(iprot) + if err != nil { + return + } + if err = iprot.ReadMessageEnd(); err != nil { + return + } + err = error34 + return + } + if mTypeId != thrift.REPLY { + err = thrift.NewTApplicationException(thrift.INVALID_MESSAGE_TYPE_EXCEPTION, "aggregateRaw failed: invalid message type") + return + } + result := NodeAggregateRawResult{} + if err = result.Read(iprot); err != nil { + return + } + if err = iprot.ReadMessageEnd(); err != nil { + return + } + if result.Err != nil { + err = result.Err + return + } + value = result.GetSuccess() + return +} + +// Parameters: +// - Req +func (p *NodeClient) Aggregate(req *AggregateQueryRequest) (r *AggregateQueryResult_, err error) { + if err = p.sendAggregate(req); err != nil { + return + } + return p.recvAggregate() +} + +func (p *NodeClient) sendAggregate(req *AggregateQueryRequest) (err error) { + oprot := p.OutputProtocol + if oprot == nil { + oprot = p.ProtocolFactory.GetProtocol(p.Transport) + p.OutputProtocol = oprot + } + p.SeqId++ + if err = oprot.WriteMessageBegin("aggregate", thrift.CALL, p.SeqId); err != nil { + return + } + args := NodeAggregateArgs{ + Req: req, + } + if err = args.Write(oprot); err != nil { + return + } + if err = oprot.WriteMessageEnd(); err != nil { + return + } + return oprot.Flush() +} + +func (p *NodeClient) recvAggregate() (value *AggregateQueryResult_, err error) { + iprot := p.InputProtocol + if iprot == nil { + iprot = p.ProtocolFactory.GetProtocol(p.Transport) + p.InputProtocol = iprot + } + method, mTypeId, seqId, err := iprot.ReadMessageBegin() + if err != nil { + return + } + if method != "aggregate" { + err = thrift.NewTApplicationException(thrift.WRONG_METHOD_NAME, "aggregate failed: wrong method name") + return + } + if p.SeqId != seqId { + err = thrift.NewTApplicationException(thrift.BAD_SEQUENCE_ID, "aggregate failed: out of sequence response") + return + } + if mTypeId == thrift.EXCEPTION { + error35 := thrift.NewTApplicationException(thrift.UNKNOWN_APPLICATION_EXCEPTION, "Unknown Exception") + var error36 error + error36, err = error35.Read(iprot) + if err != nil { + return + } + if err = iprot.ReadMessageEnd(); err != nil { + return + } + err = error36 + return + } + if mTypeId != thrift.REPLY { + err = thrift.NewTApplicationException(thrift.INVALID_MESSAGE_TYPE_EXCEPTION, "aggregate failed: invalid message type") + return + } + result := NodeAggregateResult{} + if err = result.Read(iprot); err != nil { + return + } + if err = iprot.ReadMessageEnd(); err != nil { + return + } + if result.Err != nil { + err = result.Err + return + } + value = result.GetSuccess() + return +} + +// Parameters: +// - Req +func (p *NodeClient) Fetch(req *FetchRequest) (r *FetchResult_, err error) { + if err = p.sendFetch(req); err != nil { + return + } + return p.recvFetch() +} + +func (p *NodeClient) sendFetch(req *FetchRequest) (err error) { + oprot := p.OutputProtocol + if oprot == nil { + oprot = p.ProtocolFactory.GetProtocol(p.Transport) + p.OutputProtocol = oprot + } + p.SeqId++ + if err = oprot.WriteMessageBegin("fetch", thrift.CALL, p.SeqId); err != nil { + return + } + args := NodeFetchArgs{ + Req: req, + } + if err = args.Write(oprot); err != nil { + return + } + if err = oprot.WriteMessageEnd(); err != nil { + return + } + return oprot.Flush() +} + +func (p *NodeClient) recvFetch() (value *FetchResult_, err error) { + iprot := p.InputProtocol + if iprot == nil { + iprot = p.ProtocolFactory.GetProtocol(p.Transport) + p.InputProtocol = iprot + } + method, mTypeId, seqId, err := iprot.ReadMessageBegin() + if err != nil { + return + } + if method != "fetch" { + err = thrift.NewTApplicationException(thrift.WRONG_METHOD_NAME, "fetch failed: wrong method name") + return + } + if p.SeqId != seqId { + err = thrift.NewTApplicationException(thrift.BAD_SEQUENCE_ID, "fetch failed: out of sequence response") + return + } + if mTypeId == thrift.EXCEPTION { + error37 := thrift.NewTApplicationException(thrift.UNKNOWN_APPLICATION_EXCEPTION, "Unknown Exception") + var error38 error + error38, err = error37.Read(iprot) + if err != nil { + return + } + if err = iprot.ReadMessageEnd(); err != nil { + return + } + err = error38 + return + } + if mTypeId != thrift.REPLY { + err = thrift.NewTApplicationException(thrift.INVALID_MESSAGE_TYPE_EXCEPTION, "fetch failed: invalid message type") + return + } + result := NodeFetchResult{} + if err = result.Read(iprot); err != nil { + return + } + if err = iprot.ReadMessageEnd(); err != nil { + return + } + if result.Err != nil { + err = result.Err + return + } + value = result.GetSuccess() + return +} + +// Parameters: +// - Req +func (p *NodeClient) FetchTagged(req *FetchTaggedRequest) (r *FetchTaggedResult_, err error) { + if err = p.sendFetchTagged(req); err != nil { + return + } + return p.recvFetchTagged() +} + +func (p *NodeClient) sendFetchTagged(req *FetchTaggedRequest) (err error) { + oprot := p.OutputProtocol + if oprot == nil { + oprot = p.ProtocolFactory.GetProtocol(p.Transport) + p.OutputProtocol = oprot + } + p.SeqId++ + if err = oprot.WriteMessageBegin("fetchTagged", thrift.CALL, p.SeqId); err != nil { + return + } + args := NodeFetchTaggedArgs{ + Req: req, + } + if err = args.Write(oprot); err != nil { + return + } + if err = oprot.WriteMessageEnd(); err != nil { + return + } + return oprot.Flush() +} + +func (p *NodeClient) recvFetchTagged() (value *FetchTaggedResult_, err error) { + iprot := p.InputProtocol + if iprot == nil { + iprot = p.ProtocolFactory.GetProtocol(p.Transport) + p.InputProtocol = iprot + } + method, mTypeId, seqId, err := iprot.ReadMessageBegin() + if err != nil { + return + } + if method != "fetchTagged" { + err = thrift.NewTApplicationException(thrift.WRONG_METHOD_NAME, "fetchTagged failed: wrong method name") + return + } + if p.SeqId != seqId { + err = thrift.NewTApplicationException(thrift.BAD_SEQUENCE_ID, "fetchTagged failed: out of sequence response") + return + } + if mTypeId == thrift.EXCEPTION { + error39 := thrift.NewTApplicationException(thrift.UNKNOWN_APPLICATION_EXCEPTION, "Unknown Exception") + var error40 error + error40, err = error39.Read(iprot) + if err != nil { + return + } + if err = iprot.ReadMessageEnd(); err != nil { + return + } + err = error40 + return + } + if mTypeId != thrift.REPLY { + err = thrift.NewTApplicationException(thrift.INVALID_MESSAGE_TYPE_EXCEPTION, "fetchTagged failed: invalid message type") + return + } + result := NodeFetchTaggedResult{} + if err = result.Read(iprot); err != nil { + return + } + if err = iprot.ReadMessageEnd(); err != nil { + return + } + if result.Err != nil { + err = result.Err + return + } + value = result.GetSuccess() + return +} + +// Parameters: +// - Req +func (p *NodeClient) Write(req *WriteRequest) (err error) { + if err = p.sendWrite(req); err != nil { + return + } + return p.recvWrite() +} + +func (p *NodeClient) sendWrite(req *WriteRequest) (err error) { + oprot := p.OutputProtocol + if oprot == nil { + oprot = p.ProtocolFactory.GetProtocol(p.Transport) + p.OutputProtocol = oprot + } + p.SeqId++ + if err = oprot.WriteMessageBegin("write", thrift.CALL, p.SeqId); err != nil { + return + } + args := NodeWriteArgs{ + Req: req, + } + if err = args.Write(oprot); err != nil { + return + } + if err = oprot.WriteMessageEnd(); err != nil { + return + } + return oprot.Flush() +} + +func (p *NodeClient) recvWrite() (err error) { + iprot := p.InputProtocol + if iprot == nil { + iprot = p.ProtocolFactory.GetProtocol(p.Transport) + p.InputProtocol = iprot + } + method, mTypeId, seqId, err := iprot.ReadMessageBegin() + if err != nil { + return + } + if method != "write" { + err = thrift.NewTApplicationException(thrift.WRONG_METHOD_NAME, "write failed: wrong method name") + return + } + if p.SeqId != seqId { + err = thrift.NewTApplicationException(thrift.BAD_SEQUENCE_ID, "write failed: out of sequence response") + return + } + if mTypeId == thrift.EXCEPTION { + error41 := thrift.NewTApplicationException(thrift.UNKNOWN_APPLICATION_EXCEPTION, "Unknown Exception") + var error42 error + error42, err = error41.Read(iprot) + if err != nil { + return + } + if err = iprot.ReadMessageEnd(); err != nil { + return + } + err = error42 + return + } + if mTypeId != thrift.REPLY { + err = thrift.NewTApplicationException(thrift.INVALID_MESSAGE_TYPE_EXCEPTION, "write failed: invalid message type") + return + } + result := NodeWriteResult{} + if err = result.Read(iprot); err != nil { + return + } + if err = iprot.ReadMessageEnd(); err != nil { + return + } + if result.Err != nil { + err = result.Err + return + } + return +} + +// Parameters: +// - Req +func (p *NodeClient) WriteTagged(req *WriteTaggedRequest) (err error) { + if err = p.sendWriteTagged(req); err != nil { + return + } + return p.recvWriteTagged() +} + +func (p *NodeClient) sendWriteTagged(req *WriteTaggedRequest) (err error) { + oprot := p.OutputProtocol + if oprot == nil { + oprot = p.ProtocolFactory.GetProtocol(p.Transport) + p.OutputProtocol = oprot + } + p.SeqId++ + if err = oprot.WriteMessageBegin("writeTagged", thrift.CALL, p.SeqId); err != nil { + return + } + args := NodeWriteTaggedArgs{ + Req: req, + } + if err = args.Write(oprot); err != nil { + return + } + if err = oprot.WriteMessageEnd(); err != nil { + return + } + return oprot.Flush() +} + +func (p *NodeClient) recvWriteTagged() (err error) { + iprot := p.InputProtocol + if iprot == nil { + iprot = p.ProtocolFactory.GetProtocol(p.Transport) + p.InputProtocol = iprot + } + method, mTypeId, seqId, err := iprot.ReadMessageBegin() + if err != nil { + return + } + if method != "writeTagged" { + err = thrift.NewTApplicationException(thrift.WRONG_METHOD_NAME, "writeTagged failed: wrong method name") + return + } + if p.SeqId != seqId { + err = thrift.NewTApplicationException(thrift.BAD_SEQUENCE_ID, "writeTagged failed: out of sequence response") + return + } + if mTypeId == thrift.EXCEPTION { + error43 := thrift.NewTApplicationException(thrift.UNKNOWN_APPLICATION_EXCEPTION, "Unknown Exception") + var error44 error + error44, err = error43.Read(iprot) + if err != nil { + return + } + if err = iprot.ReadMessageEnd(); err != nil { + return + } + err = error44 + return + } + if mTypeId != thrift.REPLY { + err = thrift.NewTApplicationException(thrift.INVALID_MESSAGE_TYPE_EXCEPTION, "writeTagged failed: invalid message type") + return + } + result := NodeWriteTaggedResult{} + if err = result.Read(iprot); err != nil { + return + } + if err = iprot.ReadMessageEnd(); err != nil { + return + } + if result.Err != nil { + err = result.Err + return + } + return +} + +// Parameters: +// - Req +func (p *NodeClient) FetchBatchRaw(req *FetchBatchRawRequest) (r *FetchBatchRawResult_, err error) { + if err = p.sendFetchBatchRaw(req); err != nil { + return + } + return p.recvFetchBatchRaw() +} + +func (p *NodeClient) sendFetchBatchRaw(req *FetchBatchRawRequest) (err error) { + oprot := p.OutputProtocol + if oprot == nil { + oprot = p.ProtocolFactory.GetProtocol(p.Transport) + p.OutputProtocol = oprot + } + p.SeqId++ + if err = oprot.WriteMessageBegin("fetchBatchRaw", thrift.CALL, p.SeqId); err != nil { + return + } + args := NodeFetchBatchRawArgs{ + Req: req, + } + if err = args.Write(oprot); err != nil { + return + } + if err = oprot.WriteMessageEnd(); err != nil { + return + } + return oprot.Flush() +} + +func (p *NodeClient) recvFetchBatchRaw() (value *FetchBatchRawResult_, err error) { + iprot := p.InputProtocol + if iprot == nil { + iprot = p.ProtocolFactory.GetProtocol(p.Transport) + p.InputProtocol = iprot + } + method, mTypeId, seqId, err := iprot.ReadMessageBegin() + if err != nil { + return + } + if method != "fetchBatchRaw" { + err = thrift.NewTApplicationException(thrift.WRONG_METHOD_NAME, "fetchBatchRaw failed: wrong method name") + return + } + if p.SeqId != seqId { + err = thrift.NewTApplicationException(thrift.BAD_SEQUENCE_ID, "fetchBatchRaw failed: out of sequence response") + return + } + if mTypeId == thrift.EXCEPTION { + error45 := thrift.NewTApplicationException(thrift.UNKNOWN_APPLICATION_EXCEPTION, "Unknown Exception") + var error46 error + error46, err = error45.Read(iprot) + if err != nil { + return + } + if err = iprot.ReadMessageEnd(); err != nil { + return + } + err = error46 + return + } + if mTypeId != thrift.REPLY { + err = thrift.NewTApplicationException(thrift.INVALID_MESSAGE_TYPE_EXCEPTION, "fetchBatchRaw failed: invalid message type") + return + } + result := NodeFetchBatchRawResult{} + if err = result.Read(iprot); err != nil { + return + } + if err = iprot.ReadMessageEnd(); err != nil { + return + } + if result.Err != nil { + err = result.Err + return + } + value = result.GetSuccess() + return +} + +// Parameters: +// - Req +func (p *NodeClient) FetchBlocksRaw(req *FetchBlocksRawRequest) (r *FetchBlocksRawResult_, err error) { + if err = p.sendFetchBlocksRaw(req); err != nil { + return + } + return p.recvFetchBlocksRaw() +} + +func (p *NodeClient) sendFetchBlocksRaw(req *FetchBlocksRawRequest) (err error) { + oprot := p.OutputProtocol + if oprot == nil { + oprot = p.ProtocolFactory.GetProtocol(p.Transport) + p.OutputProtocol = oprot + } + p.SeqId++ + if err = oprot.WriteMessageBegin("fetchBlocksRaw", thrift.CALL, p.SeqId); err != nil { + return + } + args := NodeFetchBlocksRawArgs{ + Req: req, + } + if err = args.Write(oprot); err != nil { + return + } + if err = oprot.WriteMessageEnd(); err != nil { + return + } + return oprot.Flush() +} + +func (p *NodeClient) recvFetchBlocksRaw() (value *FetchBlocksRawResult_, err error) { + iprot := p.InputProtocol + if iprot == nil { + iprot = p.ProtocolFactory.GetProtocol(p.Transport) + p.InputProtocol = iprot + } + method, mTypeId, seqId, err := iprot.ReadMessageBegin() + if err != nil { + return + } + if method != "fetchBlocksRaw" { + err = thrift.NewTApplicationException(thrift.WRONG_METHOD_NAME, "fetchBlocksRaw failed: wrong method name") + return + } + if p.SeqId != seqId { + err = thrift.NewTApplicationException(thrift.BAD_SEQUENCE_ID, "fetchBlocksRaw failed: out of sequence response") + return + } + if mTypeId == thrift.EXCEPTION { + error47 := thrift.NewTApplicationException(thrift.UNKNOWN_APPLICATION_EXCEPTION, "Unknown Exception") + var error48 error + error48, err = error47.Read(iprot) + if err != nil { + return + } + if err = iprot.ReadMessageEnd(); err != nil { + return + } + err = error48 + return + } + if mTypeId != thrift.REPLY { + err = thrift.NewTApplicationException(thrift.INVALID_MESSAGE_TYPE_EXCEPTION, "fetchBlocksRaw failed: invalid message type") + return + } + result := NodeFetchBlocksRawResult{} + if err = result.Read(iprot); err != nil { + return + } + if err = iprot.ReadMessageEnd(); err != nil { + return + } + if result.Err != nil { + err = result.Err + return + } + value = result.GetSuccess() + return +} + +// Parameters: +// - Req +func (p *NodeClient) FetchBlocksMetadataRawV2(req *FetchBlocksMetadataRawV2Request) (r *FetchBlocksMetadataRawV2Result_, err error) { + if err = p.sendFetchBlocksMetadataRawV2(req); err != nil { + return + } + return p.recvFetchBlocksMetadataRawV2() +} + +func (p *NodeClient) sendFetchBlocksMetadataRawV2(req *FetchBlocksMetadataRawV2Request) (err error) { + oprot := p.OutputProtocol + if oprot == nil { + oprot = p.ProtocolFactory.GetProtocol(p.Transport) + p.OutputProtocol = oprot + } + p.SeqId++ + if err = oprot.WriteMessageBegin("fetchBlocksMetadataRawV2", thrift.CALL, p.SeqId); err != nil { + return + } + args := NodeFetchBlocksMetadataRawV2Args{ + Req: req, + } + if err = args.Write(oprot); err != nil { + return + } + if err = oprot.WriteMessageEnd(); err != nil { + return + } + return oprot.Flush() +} + +func (p *NodeClient) recvFetchBlocksMetadataRawV2() (value *FetchBlocksMetadataRawV2Result_, err error) { + iprot := p.InputProtocol + if iprot == nil { + iprot = p.ProtocolFactory.GetProtocol(p.Transport) + p.InputProtocol = iprot + } + method, mTypeId, seqId, err := iprot.ReadMessageBegin() + if err != nil { + return + } + if method != "fetchBlocksMetadataRawV2" { + err = thrift.NewTApplicationException(thrift.WRONG_METHOD_NAME, "fetchBlocksMetadataRawV2 failed: wrong method name") + return + } + if p.SeqId != seqId { + err = thrift.NewTApplicationException(thrift.BAD_SEQUENCE_ID, "fetchBlocksMetadataRawV2 failed: out of sequence response") + return + } + if mTypeId == thrift.EXCEPTION { + error49 := thrift.NewTApplicationException(thrift.UNKNOWN_APPLICATION_EXCEPTION, "Unknown Exception") + var error50 error + error50, err = error49.Read(iprot) + if err != nil { + return + } + if err = iprot.ReadMessageEnd(); err != nil { + return + } + err = error50 + return + } + if mTypeId != thrift.REPLY { + err = thrift.NewTApplicationException(thrift.INVALID_MESSAGE_TYPE_EXCEPTION, "fetchBlocksMetadataRawV2 failed: invalid message type") + return + } + result := NodeFetchBlocksMetadataRawV2Result{} + if err = result.Read(iprot); err != nil { + return + } + if err = iprot.ReadMessageEnd(); err != nil { + return + } + if result.Err != nil { + err = result.Err + return + } + value = result.GetSuccess() + return +} + +// Parameters: +// - Req +func (p *NodeClient) WriteBatchRaw(req *WriteBatchRawRequest) (err error) { + if err = p.sendWriteBatchRaw(req); err != nil { + return + } + return p.recvWriteBatchRaw() +} + +func (p *NodeClient) sendWriteBatchRaw(req *WriteBatchRawRequest) (err error) { + oprot := p.OutputProtocol + if oprot == nil { + oprot = p.ProtocolFactory.GetProtocol(p.Transport) + p.OutputProtocol = oprot + } + p.SeqId++ + if err = oprot.WriteMessageBegin("writeBatchRaw", thrift.CALL, p.SeqId); err != nil { + return + } + args := NodeWriteBatchRawArgs{ + Req: req, + } + if err = args.Write(oprot); err != nil { + return + } + if err = oprot.WriteMessageEnd(); err != nil { + return + } + return oprot.Flush() +} + +func (p *NodeClient) recvWriteBatchRaw() (err error) { + iprot := p.InputProtocol + if iprot == nil { + iprot = p.ProtocolFactory.GetProtocol(p.Transport) + p.InputProtocol = iprot } method, mTypeId, seqId, err := iprot.ReadMessageBegin() if err != nil { return } - if method != "writeTagged" { - err = thrift.NewTApplicationException(thrift.WRONG_METHOD_NAME, "writeTagged failed: wrong method name") + if method != "writeBatchRaw" { + err = thrift.NewTApplicationException(thrift.WRONG_METHOD_NAME, "writeBatchRaw failed: wrong method name") return } if p.SeqId != seqId { - err = thrift.NewTApplicationException(thrift.BAD_SEQUENCE_ID, "writeTagged failed: out of sequence response") + err = thrift.NewTApplicationException(thrift.BAD_SEQUENCE_ID, "writeBatchRaw failed: out of sequence response") return } if mTypeId == thrift.EXCEPTION { - error39 := thrift.NewTApplicationException(thrift.UNKNOWN_APPLICATION_EXCEPTION, "Unknown Exception") - var error40 error - error40, err = error39.Read(iprot) + error51 := thrift.NewTApplicationException(thrift.UNKNOWN_APPLICATION_EXCEPTION, "Unknown Exception") + var error52 error + error52, err = error51.Read(iprot) if err != nil { return } if err = iprot.ReadMessageEnd(); err != nil { return } - err = error40 + err = error52 return } if mTypeId != thrift.REPLY { - err = thrift.NewTApplicationException(thrift.INVALID_MESSAGE_TYPE_EXCEPTION, "writeTagged failed: invalid message type") + err = thrift.NewTApplicationException(thrift.INVALID_MESSAGE_TYPE_EXCEPTION, "writeBatchRaw failed: invalid message type") return } - result := NodeWriteTaggedResult{} + result := NodeWriteBatchRawResult{} if err = result.Read(iprot); err != nil { return } @@ -12131,24 +13236,24 @@ func (p *NodeClient) recvWriteTagged() (err error) { // Parameters: // - Req -func (p *NodeClient) FetchBatchRaw(req *FetchBatchRawRequest) (r *FetchBatchRawResult_, err error) { - if err = p.sendFetchBatchRaw(req); err != nil { +func (p *NodeClient) WriteBatchRawV2(req *WriteBatchRawV2Request) (err error) { + if err = p.sendWriteBatchRawV2(req); err != nil { return } - return p.recvFetchBatchRaw() + return p.recvWriteBatchRawV2() } -func (p *NodeClient) sendFetchBatchRaw(req *FetchBatchRawRequest) (err error) { +func (p *NodeClient) sendWriteBatchRawV2(req *WriteBatchRawV2Request) (err error) { oprot := p.OutputProtocol if oprot == nil { oprot = p.ProtocolFactory.GetProtocol(p.Transport) p.OutputProtocol = oprot } p.SeqId++ - if err = oprot.WriteMessageBegin("fetchBatchRaw", thrift.CALL, p.SeqId); err != nil { + if err = oprot.WriteMessageBegin("writeBatchRawV2", thrift.CALL, p.SeqId); err != nil { return } - args := NodeFetchBatchRawArgs{ + args := NodeWriteBatchRawV2Args{ Req: req, } if err = args.Write(oprot); err != nil { @@ -12160,7 +13265,7 @@ func (p *NodeClient) sendFetchBatchRaw(req *FetchBatchRawRequest) (err error) { return oprot.Flush() } -func (p *NodeClient) recvFetchBatchRaw() (value *FetchBatchRawResult_, err error) { +func (p *NodeClient) recvWriteBatchRawV2() (err error) { iprot := p.InputProtocol if iprot == nil { iprot = p.ProtocolFactory.GetProtocol(p.Transport) @@ -12170,32 +13275,32 @@ func (p *NodeClient) recvFetchBatchRaw() (value *FetchBatchRawResult_, err error if err != nil { return } - if method != "fetchBatchRaw" { - err = thrift.NewTApplicationException(thrift.WRONG_METHOD_NAME, "fetchBatchRaw failed: wrong method name") + if method != "writeBatchRawV2" { + err = thrift.NewTApplicationException(thrift.WRONG_METHOD_NAME, "writeBatchRawV2 failed: wrong method name") return } if p.SeqId != seqId { - err = thrift.NewTApplicationException(thrift.BAD_SEQUENCE_ID, "fetchBatchRaw failed: out of sequence response") + err = thrift.NewTApplicationException(thrift.BAD_SEQUENCE_ID, "writeBatchRawV2 failed: out of sequence response") return } if mTypeId == thrift.EXCEPTION { - error41 := thrift.NewTApplicationException(thrift.UNKNOWN_APPLICATION_EXCEPTION, "Unknown Exception") - var error42 error - error42, err = error41.Read(iprot) + error53 := thrift.NewTApplicationException(thrift.UNKNOWN_APPLICATION_EXCEPTION, "Unknown Exception") + var error54 error + error54, err = error53.Read(iprot) if err != nil { return } if err = iprot.ReadMessageEnd(); err != nil { return } - err = error42 + err = error54 return } if mTypeId != thrift.REPLY { - err = thrift.NewTApplicationException(thrift.INVALID_MESSAGE_TYPE_EXCEPTION, "fetchBatchRaw failed: invalid message type") + err = thrift.NewTApplicationException(thrift.INVALID_MESSAGE_TYPE_EXCEPTION, "writeBatchRawV2 failed: invalid message type") return } - result := NodeFetchBatchRawResult{} + result := NodeWriteBatchRawV2Result{} if err = result.Read(iprot); err != nil { return } @@ -12206,30 +13311,29 @@ func (p *NodeClient) recvFetchBatchRaw() (value *FetchBatchRawResult_, err error err = result.Err return } - value = result.GetSuccess() return } // Parameters: // - Req -func (p *NodeClient) FetchBlocksRaw(req *FetchBlocksRawRequest) (r *FetchBlocksRawResult_, err error) { - if err = p.sendFetchBlocksRaw(req); err != nil { +func (p *NodeClient) WriteTaggedBatchRaw(req *WriteTaggedBatchRawRequest) (err error) { + if err = p.sendWriteTaggedBatchRaw(req); err != nil { return } - return p.recvFetchBlocksRaw() + return p.recvWriteTaggedBatchRaw() } -func (p *NodeClient) sendFetchBlocksRaw(req *FetchBlocksRawRequest) (err error) { +func (p *NodeClient) sendWriteTaggedBatchRaw(req *WriteTaggedBatchRawRequest) (err error) { oprot := p.OutputProtocol if oprot == nil { oprot = p.ProtocolFactory.GetProtocol(p.Transport) p.OutputProtocol = oprot } p.SeqId++ - if err = oprot.WriteMessageBegin("fetchBlocksRaw", thrift.CALL, p.SeqId); err != nil { + if err = oprot.WriteMessageBegin("writeTaggedBatchRaw", thrift.CALL, p.SeqId); err != nil { return } - args := NodeFetchBlocksRawArgs{ + args := NodeWriteTaggedBatchRawArgs{ Req: req, } if err = args.Write(oprot); err != nil { @@ -12241,7 +13345,7 @@ func (p *NodeClient) sendFetchBlocksRaw(req *FetchBlocksRawRequest) (err error) return oprot.Flush() } -func (p *NodeClient) recvFetchBlocksRaw() (value *FetchBlocksRawResult_, err error) { +func (p *NodeClient) recvWriteTaggedBatchRaw() (err error) { iprot := p.InputProtocol if iprot == nil { iprot = p.ProtocolFactory.GetProtocol(p.Transport) @@ -12251,32 +13355,32 @@ func (p *NodeClient) recvFetchBlocksRaw() (value *FetchBlocksRawResult_, err err if err != nil { return } - if method != "fetchBlocksRaw" { - err = thrift.NewTApplicationException(thrift.WRONG_METHOD_NAME, "fetchBlocksRaw failed: wrong method name") + if method != "writeTaggedBatchRaw" { + err = thrift.NewTApplicationException(thrift.WRONG_METHOD_NAME, "writeTaggedBatchRaw failed: wrong method name") return } if p.SeqId != seqId { - err = thrift.NewTApplicationException(thrift.BAD_SEQUENCE_ID, "fetchBlocksRaw failed: out of sequence response") + err = thrift.NewTApplicationException(thrift.BAD_SEQUENCE_ID, "writeTaggedBatchRaw failed: out of sequence response") return } if mTypeId == thrift.EXCEPTION { - error43 := thrift.NewTApplicationException(thrift.UNKNOWN_APPLICATION_EXCEPTION, "Unknown Exception") - var error44 error - error44, err = error43.Read(iprot) + error55 := thrift.NewTApplicationException(thrift.UNKNOWN_APPLICATION_EXCEPTION, "Unknown Exception") + var error56 error + error56, err = error55.Read(iprot) if err != nil { return } if err = iprot.ReadMessageEnd(); err != nil { return } - err = error44 + err = error56 return } if mTypeId != thrift.REPLY { - err = thrift.NewTApplicationException(thrift.INVALID_MESSAGE_TYPE_EXCEPTION, "fetchBlocksRaw failed: invalid message type") + err = thrift.NewTApplicationException(thrift.INVALID_MESSAGE_TYPE_EXCEPTION, "writeTaggedBatchRaw failed: invalid message type") return } - result := NodeFetchBlocksRawResult{} + result := NodeWriteTaggedBatchRawResult{} if err = result.Read(iprot); err != nil { return } @@ -12287,30 +13391,29 @@ func (p *NodeClient) recvFetchBlocksRaw() (value *FetchBlocksRawResult_, err err err = result.Err return } - value = result.GetSuccess() return } // Parameters: // - Req -func (p *NodeClient) FetchBlocksMetadataRawV2(req *FetchBlocksMetadataRawV2Request) (r *FetchBlocksMetadataRawV2Result_, err error) { - if err = p.sendFetchBlocksMetadataRawV2(req); err != nil { +func (p *NodeClient) WriteTaggedBatchRawV2(req *WriteTaggedBatchRawV2Request) (err error) { + if err = p.sendWriteTaggedBatchRawV2(req); err != nil { return } - return p.recvFetchBlocksMetadataRawV2() + return p.recvWriteTaggedBatchRawV2() } -func (p *NodeClient) sendFetchBlocksMetadataRawV2(req *FetchBlocksMetadataRawV2Request) (err error) { +func (p *NodeClient) sendWriteTaggedBatchRawV2(req *WriteTaggedBatchRawV2Request) (err error) { oprot := p.OutputProtocol if oprot == nil { oprot = p.ProtocolFactory.GetProtocol(p.Transport) p.OutputProtocol = oprot } p.SeqId++ - if err = oprot.WriteMessageBegin("fetchBlocksMetadataRawV2", thrift.CALL, p.SeqId); err != nil { + if err = oprot.WriteMessageBegin("writeTaggedBatchRawV2", thrift.CALL, p.SeqId); err != nil { return } - args := NodeFetchBlocksMetadataRawV2Args{ + args := NodeWriteTaggedBatchRawV2Args{ Req: req, } if err = args.Write(oprot); err != nil { @@ -12322,7 +13425,7 @@ func (p *NodeClient) sendFetchBlocksMetadataRawV2(req *FetchBlocksMetadataRawV2R return oprot.Flush() } -func (p *NodeClient) recvFetchBlocksMetadataRawV2() (value *FetchBlocksMetadataRawV2Result_, err error) { +func (p *NodeClient) recvWriteTaggedBatchRawV2() (err error) { iprot := p.InputProtocol if iprot == nil { iprot = p.ProtocolFactory.GetProtocol(p.Transport) @@ -12332,32 +13435,188 @@ func (p *NodeClient) recvFetchBlocksMetadataRawV2() (value *FetchBlocksMetadataR if err != nil { return } - if method != "fetchBlocksMetadataRawV2" { - err = thrift.NewTApplicationException(thrift.WRONG_METHOD_NAME, "fetchBlocksMetadataRawV2 failed: wrong method name") + if method != "writeTaggedBatchRawV2" { + err = thrift.NewTApplicationException(thrift.WRONG_METHOD_NAME, "writeTaggedBatchRawV2 failed: wrong method name") return } if p.SeqId != seqId { - err = thrift.NewTApplicationException(thrift.BAD_SEQUENCE_ID, "fetchBlocksMetadataRawV2 failed: out of sequence response") + err = thrift.NewTApplicationException(thrift.BAD_SEQUENCE_ID, "writeTaggedBatchRawV2 failed: out of sequence response") return } if mTypeId == thrift.EXCEPTION { - error45 := thrift.NewTApplicationException(thrift.UNKNOWN_APPLICATION_EXCEPTION, "Unknown Exception") - var error46 error - error46, err = error45.Read(iprot) + error57 := thrift.NewTApplicationException(thrift.UNKNOWN_APPLICATION_EXCEPTION, "Unknown Exception") + var error58 error + error58, err = error57.Read(iprot) if err != nil { return } if err = iprot.ReadMessageEnd(); err != nil { return } - err = error46 + err = error58 return } if mTypeId != thrift.REPLY { - err = thrift.NewTApplicationException(thrift.INVALID_MESSAGE_TYPE_EXCEPTION, "fetchBlocksMetadataRawV2 failed: invalid message type") + err = thrift.NewTApplicationException(thrift.INVALID_MESSAGE_TYPE_EXCEPTION, "writeTaggedBatchRawV2 failed: invalid message type") return } - result := NodeFetchBlocksMetadataRawV2Result{} + result := NodeWriteTaggedBatchRawV2Result{} + if err = result.Read(iprot); err != nil { + return + } + if err = iprot.ReadMessageEnd(); err != nil { + return + } + if result.Err != nil { + err = result.Err + return + } + return +} + +func (p *NodeClient) Repair() (err error) { + if err = p.sendRepair(); err != nil { + return + } + return p.recvRepair() +} + +func (p *NodeClient) sendRepair() (err error) { + oprot := p.OutputProtocol + if oprot == nil { + oprot = p.ProtocolFactory.GetProtocol(p.Transport) + p.OutputProtocol = oprot + } + p.SeqId++ + if err = oprot.WriteMessageBegin("repair", thrift.CALL, p.SeqId); err != nil { + return + } + args := NodeRepairArgs{} + if err = args.Write(oprot); err != nil { + return + } + if err = oprot.WriteMessageEnd(); err != nil { + return + } + return oprot.Flush() +} + +func (p *NodeClient) recvRepair() (err error) { + iprot := p.InputProtocol + if iprot == nil { + iprot = p.ProtocolFactory.GetProtocol(p.Transport) + p.InputProtocol = iprot + } + method, mTypeId, seqId, err := iprot.ReadMessageBegin() + if err != nil { + return + } + if method != "repair" { + err = thrift.NewTApplicationException(thrift.WRONG_METHOD_NAME, "repair failed: wrong method name") + return + } + if p.SeqId != seqId { + err = thrift.NewTApplicationException(thrift.BAD_SEQUENCE_ID, "repair failed: out of sequence response") + return + } + if mTypeId == thrift.EXCEPTION { + error59 := thrift.NewTApplicationException(thrift.UNKNOWN_APPLICATION_EXCEPTION, "Unknown Exception") + var error60 error + error60, err = error59.Read(iprot) + if err != nil { + return + } + if err = iprot.ReadMessageEnd(); err != nil { + return + } + err = error60 + return + } + if mTypeId != thrift.REPLY { + err = thrift.NewTApplicationException(thrift.INVALID_MESSAGE_TYPE_EXCEPTION, "repair failed: invalid message type") + return + } + result := NodeRepairResult{} + if err = result.Read(iprot); err != nil { + return + } + if err = iprot.ReadMessageEnd(); err != nil { + return + } + if result.Err != nil { + err = result.Err + return + } + return +} + +// Parameters: +// - Req +func (p *NodeClient) Truncate(req *TruncateRequest) (r *TruncateResult_, err error) { + if err = p.sendTruncate(req); err != nil { + return + } + return p.recvTruncate() +} + +func (p *NodeClient) sendTruncate(req *TruncateRequest) (err error) { + oprot := p.OutputProtocol + if oprot == nil { + oprot = p.ProtocolFactory.GetProtocol(p.Transport) + p.OutputProtocol = oprot + } + p.SeqId++ + if err = oprot.WriteMessageBegin("truncate", thrift.CALL, p.SeqId); err != nil { + return + } + args := NodeTruncateArgs{ + Req: req, + } + if err = args.Write(oprot); err != nil { + return + } + if err = oprot.WriteMessageEnd(); err != nil { + return + } + return oprot.Flush() +} + +func (p *NodeClient) recvTruncate() (value *TruncateResult_, err error) { + iprot := p.InputProtocol + if iprot == nil { + iprot = p.ProtocolFactory.GetProtocol(p.Transport) + p.InputProtocol = iprot + } + method, mTypeId, seqId, err := iprot.ReadMessageBegin() + if err != nil { + return + } + if method != "truncate" { + err = thrift.NewTApplicationException(thrift.WRONG_METHOD_NAME, "truncate failed: wrong method name") + return + } + if p.SeqId != seqId { + err = thrift.NewTApplicationException(thrift.BAD_SEQUENCE_ID, "truncate failed: out of sequence response") + return + } + if mTypeId == thrift.EXCEPTION { + error61 := thrift.NewTApplicationException(thrift.UNKNOWN_APPLICATION_EXCEPTION, "Unknown Exception") + var error62 error + error62, err = error61.Read(iprot) + if err != nil { + return + } + if err = iprot.ReadMessageEnd(); err != nil { + return + } + err = error62 + return + } + if mTypeId != thrift.REPLY { + err = thrift.NewTApplicationException(thrift.INVALID_MESSAGE_TYPE_EXCEPTION, "truncate failed: invalid message type") + return + } + result := NodeTruncateResult{} if err = result.Read(iprot); err != nil { return } @@ -12372,28 +13631,24 @@ func (p *NodeClient) recvFetchBlocksMetadataRawV2() (value *FetchBlocksMetadataR return } -// Parameters: -// - Req -func (p *NodeClient) WriteBatchRaw(req *WriteBatchRawRequest) (err error) { - if err = p.sendWriteBatchRaw(req); err != nil { +func (p *NodeClient) Health() (r *NodeHealthResult_, err error) { + if err = p.sendHealth(); err != nil { return } - return p.recvWriteBatchRaw() + return p.recvHealth() } -func (p *NodeClient) sendWriteBatchRaw(req *WriteBatchRawRequest) (err error) { +func (p *NodeClient) sendHealth() (err error) { oprot := p.OutputProtocol if oprot == nil { oprot = p.ProtocolFactory.GetProtocol(p.Transport) p.OutputProtocol = oprot } p.SeqId++ - if err = oprot.WriteMessageBegin("writeBatchRaw", thrift.CALL, p.SeqId); err != nil { + if err = oprot.WriteMessageBegin("health", thrift.CALL, p.SeqId); err != nil { return } - args := NodeWriteBatchRawArgs{ - Req: req, - } + args := NodeHealthArgs{} if err = args.Write(oprot); err != nil { return } @@ -12403,7 +13658,7 @@ func (p *NodeClient) sendWriteBatchRaw(req *WriteBatchRawRequest) (err error) { return oprot.Flush() } -func (p *NodeClient) recvWriteBatchRaw() (err error) { +func (p *NodeClient) recvHealth() (value *NodeHealthResult_, err error) { iprot := p.InputProtocol if iprot == nil { iprot = p.ProtocolFactory.GetProtocol(p.Transport) @@ -12413,32 +13668,32 @@ func (p *NodeClient) recvWriteBatchRaw() (err error) { if err != nil { return } - if method != "writeBatchRaw" { - err = thrift.NewTApplicationException(thrift.WRONG_METHOD_NAME, "writeBatchRaw failed: wrong method name") + if method != "health" { + err = thrift.NewTApplicationException(thrift.WRONG_METHOD_NAME, "health failed: wrong method name") return } if p.SeqId != seqId { - err = thrift.NewTApplicationException(thrift.BAD_SEQUENCE_ID, "writeBatchRaw failed: out of sequence response") + err = thrift.NewTApplicationException(thrift.BAD_SEQUENCE_ID, "health failed: out of sequence response") return } if mTypeId == thrift.EXCEPTION { - error47 := thrift.NewTApplicationException(thrift.UNKNOWN_APPLICATION_EXCEPTION, "Unknown Exception") - var error48 error - error48, err = error47.Read(iprot) + error63 := thrift.NewTApplicationException(thrift.UNKNOWN_APPLICATION_EXCEPTION, "Unknown Exception") + var error64 error + error64, err = error63.Read(iprot) if err != nil { return } if err = iprot.ReadMessageEnd(); err != nil { return } - err = error48 + err = error64 return } if mTypeId != thrift.REPLY { - err = thrift.NewTApplicationException(thrift.INVALID_MESSAGE_TYPE_EXCEPTION, "writeBatchRaw failed: invalid message type") + err = thrift.NewTApplicationException(thrift.INVALID_MESSAGE_TYPE_EXCEPTION, "health failed: invalid message type") return } - result := NodeWriteBatchRawResult{} + result := NodeHealthResult{} if err = result.Read(iprot); err != nil { return } @@ -12449,31 +13704,28 @@ func (p *NodeClient) recvWriteBatchRaw() (err error) { err = result.Err return } + value = result.GetSuccess() return } -// Parameters: -// - Req -func (p *NodeClient) WriteTaggedBatchRaw(req *WriteTaggedBatchRawRequest) (err error) { - if err = p.sendWriteTaggedBatchRaw(req); err != nil { +func (p *NodeClient) Bootstrapped() (r *NodeBootstrappedResult_, err error) { + if err = p.sendBootstrapped(); err != nil { return } - return p.recvWriteTaggedBatchRaw() + return p.recvBootstrapped() } -func (p *NodeClient) sendWriteTaggedBatchRaw(req *WriteTaggedBatchRawRequest) (err error) { +func (p *NodeClient) sendBootstrapped() (err error) { oprot := p.OutputProtocol if oprot == nil { oprot = p.ProtocolFactory.GetProtocol(p.Transport) p.OutputProtocol = oprot } p.SeqId++ - if err = oprot.WriteMessageBegin("writeTaggedBatchRaw", thrift.CALL, p.SeqId); err != nil { + if err = oprot.WriteMessageBegin("bootstrapped", thrift.CALL, p.SeqId); err != nil { return } - args := NodeWriteTaggedBatchRawArgs{ - Req: req, - } + args := NodeBootstrappedArgs{} if err = args.Write(oprot); err != nil { return } @@ -12483,7 +13735,7 @@ func (p *NodeClient) sendWriteTaggedBatchRaw(req *WriteTaggedBatchRawRequest) (e return oprot.Flush() } -func (p *NodeClient) recvWriteTaggedBatchRaw() (err error) { +func (p *NodeClient) recvBootstrapped() (value *NodeBootstrappedResult_, err error) { iprot := p.InputProtocol if iprot == nil { iprot = p.ProtocolFactory.GetProtocol(p.Transport) @@ -12493,32 +13745,32 @@ func (p *NodeClient) recvWriteTaggedBatchRaw() (err error) { if err != nil { return } - if method != "writeTaggedBatchRaw" { - err = thrift.NewTApplicationException(thrift.WRONG_METHOD_NAME, "writeTaggedBatchRaw failed: wrong method name") + if method != "bootstrapped" { + err = thrift.NewTApplicationException(thrift.WRONG_METHOD_NAME, "bootstrapped failed: wrong method name") return } if p.SeqId != seqId { - err = thrift.NewTApplicationException(thrift.BAD_SEQUENCE_ID, "writeTaggedBatchRaw failed: out of sequence response") + err = thrift.NewTApplicationException(thrift.BAD_SEQUENCE_ID, "bootstrapped failed: out of sequence response") return } if mTypeId == thrift.EXCEPTION { - error49 := thrift.NewTApplicationException(thrift.UNKNOWN_APPLICATION_EXCEPTION, "Unknown Exception") - var error50 error - error50, err = error49.Read(iprot) + error65 := thrift.NewTApplicationException(thrift.UNKNOWN_APPLICATION_EXCEPTION, "Unknown Exception") + var error66 error + error66, err = error65.Read(iprot) if err != nil { return } if err = iprot.ReadMessageEnd(); err != nil { return } - err = error50 + err = error66 return } if mTypeId != thrift.REPLY { - err = thrift.NewTApplicationException(thrift.INVALID_MESSAGE_TYPE_EXCEPTION, "writeTaggedBatchRaw failed: invalid message type") + err = thrift.NewTApplicationException(thrift.INVALID_MESSAGE_TYPE_EXCEPTION, "bootstrapped failed: invalid message type") return } - result := NodeWriteTaggedBatchRawResult{} + result := NodeBootstrappedResult{} if err = result.Read(iprot); err != nil { return } @@ -12529,27 +13781,28 @@ func (p *NodeClient) recvWriteTaggedBatchRaw() (err error) { err = result.Err return } + value = result.GetSuccess() return } -func (p *NodeClient) Repair() (err error) { - if err = p.sendRepair(); err != nil { +func (p *NodeClient) BootstrappedInPlacementOrNoPlacement() (r *NodeBootstrappedInPlacementOrNoPlacementResult_, err error) { + if err = p.sendBootstrappedInPlacementOrNoPlacement(); err != nil { return } - return p.recvRepair() + return p.recvBootstrappedInPlacementOrNoPlacement() } -func (p *NodeClient) sendRepair() (err error) { +func (p *NodeClient) sendBootstrappedInPlacementOrNoPlacement() (err error) { oprot := p.OutputProtocol if oprot == nil { oprot = p.ProtocolFactory.GetProtocol(p.Transport) p.OutputProtocol = oprot } p.SeqId++ - if err = oprot.WriteMessageBegin("repair", thrift.CALL, p.SeqId); err != nil { + if err = oprot.WriteMessageBegin("bootstrappedInPlacementOrNoPlacement", thrift.CALL, p.SeqId); err != nil { return } - args := NodeRepairArgs{} + args := NodeBootstrappedInPlacementOrNoPlacementArgs{} if err = args.Write(oprot); err != nil { return } @@ -12559,7 +13812,7 @@ func (p *NodeClient) sendRepair() (err error) { return oprot.Flush() } -func (p *NodeClient) recvRepair() (err error) { +func (p *NodeClient) recvBootstrappedInPlacementOrNoPlacement() (value *NodeBootstrappedInPlacementOrNoPlacementResult_, err error) { iprot := p.InputProtocol if iprot == nil { iprot = p.ProtocolFactory.GetProtocol(p.Transport) @@ -12569,32 +13822,32 @@ func (p *NodeClient) recvRepair() (err error) { if err != nil { return } - if method != "repair" { - err = thrift.NewTApplicationException(thrift.WRONG_METHOD_NAME, "repair failed: wrong method name") + if method != "bootstrappedInPlacementOrNoPlacement" { + err = thrift.NewTApplicationException(thrift.WRONG_METHOD_NAME, "bootstrappedInPlacementOrNoPlacement failed: wrong method name") return } if p.SeqId != seqId { - err = thrift.NewTApplicationException(thrift.BAD_SEQUENCE_ID, "repair failed: out of sequence response") + err = thrift.NewTApplicationException(thrift.BAD_SEQUENCE_ID, "bootstrappedInPlacementOrNoPlacement failed: out of sequence response") return } if mTypeId == thrift.EXCEPTION { - error51 := thrift.NewTApplicationException(thrift.UNKNOWN_APPLICATION_EXCEPTION, "Unknown Exception") - var error52 error - error52, err = error51.Read(iprot) + error67 := thrift.NewTApplicationException(thrift.UNKNOWN_APPLICATION_EXCEPTION, "Unknown Exception") + var error68 error + error68, err = error67.Read(iprot) if err != nil { return } if err = iprot.ReadMessageEnd(); err != nil { return } - err = error52 + err = error68 return } if mTypeId != thrift.REPLY { - err = thrift.NewTApplicationException(thrift.INVALID_MESSAGE_TYPE_EXCEPTION, "repair failed: invalid message type") + err = thrift.NewTApplicationException(thrift.INVALID_MESSAGE_TYPE_EXCEPTION, "bootstrappedInPlacementOrNoPlacement failed: invalid message type") return } - result := NodeRepairResult{} + result := NodeBootstrappedInPlacementOrNoPlacementResult{} if err = result.Read(iprot); err != nil { return } @@ -12605,31 +13858,28 @@ func (p *NodeClient) recvRepair() (err error) { err = result.Err return } + value = result.GetSuccess() return } -// Parameters: -// - Req -func (p *NodeClient) Truncate(req *TruncateRequest) (r *TruncateResult_, err error) { - if err = p.sendTruncate(req); err != nil { +func (p *NodeClient) GetPersistRateLimit() (r *NodePersistRateLimitResult_, err error) { + if err = p.sendGetPersistRateLimit(); err != nil { return } - return p.recvTruncate() + return p.recvGetPersistRateLimit() } -func (p *NodeClient) sendTruncate(req *TruncateRequest) (err error) { +func (p *NodeClient) sendGetPersistRateLimit() (err error) { oprot := p.OutputProtocol if oprot == nil { oprot = p.ProtocolFactory.GetProtocol(p.Transport) p.OutputProtocol = oprot } p.SeqId++ - if err = oprot.WriteMessageBegin("truncate", thrift.CALL, p.SeqId); err != nil { + if err = oprot.WriteMessageBegin("getPersistRateLimit", thrift.CALL, p.SeqId); err != nil { return } - args := NodeTruncateArgs{ - Req: req, - } + args := NodeGetPersistRateLimitArgs{} if err = args.Write(oprot); err != nil { return } @@ -12639,7 +13889,7 @@ func (p *NodeClient) sendTruncate(req *TruncateRequest) (err error) { return oprot.Flush() } -func (p *NodeClient) recvTruncate() (value *TruncateResult_, err error) { +func (p *NodeClient) recvGetPersistRateLimit() (value *NodePersistRateLimitResult_, err error) { iprot := p.InputProtocol if iprot == nil { iprot = p.ProtocolFactory.GetProtocol(p.Transport) @@ -12649,32 +13899,32 @@ func (p *NodeClient) recvTruncate() (value *TruncateResult_, err error) { if err != nil { return } - if method != "truncate" { - err = thrift.NewTApplicationException(thrift.WRONG_METHOD_NAME, "truncate failed: wrong method name") + if method != "getPersistRateLimit" { + err = thrift.NewTApplicationException(thrift.WRONG_METHOD_NAME, "getPersistRateLimit failed: wrong method name") return } if p.SeqId != seqId { - err = thrift.NewTApplicationException(thrift.BAD_SEQUENCE_ID, "truncate failed: out of sequence response") + err = thrift.NewTApplicationException(thrift.BAD_SEQUENCE_ID, "getPersistRateLimit failed: out of sequence response") return } if mTypeId == thrift.EXCEPTION { - error53 := thrift.NewTApplicationException(thrift.UNKNOWN_APPLICATION_EXCEPTION, "Unknown Exception") - var error54 error - error54, err = error53.Read(iprot) + error69 := thrift.NewTApplicationException(thrift.UNKNOWN_APPLICATION_EXCEPTION, "Unknown Exception") + var error70 error + error70, err = error69.Read(iprot) if err != nil { return } if err = iprot.ReadMessageEnd(); err != nil { return } - err = error54 + err = error70 return } if mTypeId != thrift.REPLY { - err = thrift.NewTApplicationException(thrift.INVALID_MESSAGE_TYPE_EXCEPTION, "truncate failed: invalid message type") + err = thrift.NewTApplicationException(thrift.INVALID_MESSAGE_TYPE_EXCEPTION, "getPersistRateLimit failed: invalid message type") return } - result := NodeTruncateResult{} + result := NodeGetPersistRateLimitResult{} if err = result.Read(iprot); err != nil { return } @@ -12689,24 +13939,28 @@ func (p *NodeClient) recvTruncate() (value *TruncateResult_, err error) { return } -func (p *NodeClient) Health() (r *NodeHealthResult_, err error) { - if err = p.sendHealth(); err != nil { +// Parameters: +// - Req +func (p *NodeClient) SetPersistRateLimit(req *NodeSetPersistRateLimitRequest) (r *NodePersistRateLimitResult_, err error) { + if err = p.sendSetPersistRateLimit(req); err != nil { return } - return p.recvHealth() + return p.recvSetPersistRateLimit() } -func (p *NodeClient) sendHealth() (err error) { +func (p *NodeClient) sendSetPersistRateLimit(req *NodeSetPersistRateLimitRequest) (err error) { oprot := p.OutputProtocol if oprot == nil { oprot = p.ProtocolFactory.GetProtocol(p.Transport) p.OutputProtocol = oprot } p.SeqId++ - if err = oprot.WriteMessageBegin("health", thrift.CALL, p.SeqId); err != nil { + if err = oprot.WriteMessageBegin("setPersistRateLimit", thrift.CALL, p.SeqId); err != nil { return } - args := NodeHealthArgs{} + args := NodeSetPersistRateLimitArgs{ + Req: req, + } if err = args.Write(oprot); err != nil { return } @@ -12716,7 +13970,7 @@ func (p *NodeClient) sendHealth() (err error) { return oprot.Flush() } -func (p *NodeClient) recvHealth() (value *NodeHealthResult_, err error) { +func (p *NodeClient) recvSetPersistRateLimit() (value *NodePersistRateLimitResult_, err error) { iprot := p.InputProtocol if iprot == nil { iprot = p.ProtocolFactory.GetProtocol(p.Transport) @@ -12726,32 +13980,32 @@ func (p *NodeClient) recvHealth() (value *NodeHealthResult_, err error) { if err != nil { return } - if method != "health" { - err = thrift.NewTApplicationException(thrift.WRONG_METHOD_NAME, "health failed: wrong method name") + if method != "setPersistRateLimit" { + err = thrift.NewTApplicationException(thrift.WRONG_METHOD_NAME, "setPersistRateLimit failed: wrong method name") return } if p.SeqId != seqId { - err = thrift.NewTApplicationException(thrift.BAD_SEQUENCE_ID, "health failed: out of sequence response") + err = thrift.NewTApplicationException(thrift.BAD_SEQUENCE_ID, "setPersistRateLimit failed: out of sequence response") return } if mTypeId == thrift.EXCEPTION { - error55 := thrift.NewTApplicationException(thrift.UNKNOWN_APPLICATION_EXCEPTION, "Unknown Exception") - var error56 error - error56, err = error55.Read(iprot) + error71 := thrift.NewTApplicationException(thrift.UNKNOWN_APPLICATION_EXCEPTION, "Unknown Exception") + var error72 error + error72, err = error71.Read(iprot) if err != nil { return } if err = iprot.ReadMessageEnd(); err != nil { return } - err = error56 + err = error72 return } if mTypeId != thrift.REPLY { - err = thrift.NewTApplicationException(thrift.INVALID_MESSAGE_TYPE_EXCEPTION, "health failed: invalid message type") + err = thrift.NewTApplicationException(thrift.INVALID_MESSAGE_TYPE_EXCEPTION, "setPersistRateLimit failed: invalid message type") return } - result := NodeHealthResult{} + result := NodeSetPersistRateLimitResult{} if err = result.Read(iprot); err != nil { return } @@ -12766,24 +14020,24 @@ func (p *NodeClient) recvHealth() (value *NodeHealthResult_, err error) { return } -func (p *NodeClient) Bootstrapped() (r *NodeBootstrappedResult_, err error) { - if err = p.sendBootstrapped(); err != nil { +func (p *NodeClient) GetWriteNewSeriesAsync() (r *NodeWriteNewSeriesAsyncResult_, err error) { + if err = p.sendGetWriteNewSeriesAsync(); err != nil { return } - return p.recvBootstrapped() + return p.recvGetWriteNewSeriesAsync() } -func (p *NodeClient) sendBootstrapped() (err error) { +func (p *NodeClient) sendGetWriteNewSeriesAsync() (err error) { oprot := p.OutputProtocol if oprot == nil { oprot = p.ProtocolFactory.GetProtocol(p.Transport) p.OutputProtocol = oprot } p.SeqId++ - if err = oprot.WriteMessageBegin("bootstrapped", thrift.CALL, p.SeqId); err != nil { + if err = oprot.WriteMessageBegin("getWriteNewSeriesAsync", thrift.CALL, p.SeqId); err != nil { return } - args := NodeBootstrappedArgs{} + args := NodeGetWriteNewSeriesAsyncArgs{} if err = args.Write(oprot); err != nil { return } @@ -12793,7 +14047,7 @@ func (p *NodeClient) sendBootstrapped() (err error) { return oprot.Flush() } -func (p *NodeClient) recvBootstrapped() (value *NodeBootstrappedResult_, err error) { +func (p *NodeClient) recvGetWriteNewSeriesAsync() (value *NodeWriteNewSeriesAsyncResult_, err error) { iprot := p.InputProtocol if iprot == nil { iprot = p.ProtocolFactory.GetProtocol(p.Transport) @@ -12803,32 +14057,32 @@ func (p *NodeClient) recvBootstrapped() (value *NodeBootstrappedResult_, err err if err != nil { return } - if method != "bootstrapped" { - err = thrift.NewTApplicationException(thrift.WRONG_METHOD_NAME, "bootstrapped failed: wrong method name") + if method != "getWriteNewSeriesAsync" { + err = thrift.NewTApplicationException(thrift.WRONG_METHOD_NAME, "getWriteNewSeriesAsync failed: wrong method name") return } if p.SeqId != seqId { - err = thrift.NewTApplicationException(thrift.BAD_SEQUENCE_ID, "bootstrapped failed: out of sequence response") + err = thrift.NewTApplicationException(thrift.BAD_SEQUENCE_ID, "getWriteNewSeriesAsync failed: out of sequence response") return } if mTypeId == thrift.EXCEPTION { - error57 := thrift.NewTApplicationException(thrift.UNKNOWN_APPLICATION_EXCEPTION, "Unknown Exception") - var error58 error - error58, err = error57.Read(iprot) + error73 := thrift.NewTApplicationException(thrift.UNKNOWN_APPLICATION_EXCEPTION, "Unknown Exception") + var error74 error + error74, err = error73.Read(iprot) if err != nil { return } if err = iprot.ReadMessageEnd(); err != nil { return } - err = error58 + err = error74 return } if mTypeId != thrift.REPLY { - err = thrift.NewTApplicationException(thrift.INVALID_MESSAGE_TYPE_EXCEPTION, "bootstrapped failed: invalid message type") + err = thrift.NewTApplicationException(thrift.INVALID_MESSAGE_TYPE_EXCEPTION, "getWriteNewSeriesAsync failed: invalid message type") return } - result := NodeBootstrappedResult{} + result := NodeGetWriteNewSeriesAsyncResult{} if err = result.Read(iprot); err != nil { return } @@ -12843,24 +14097,28 @@ func (p *NodeClient) recvBootstrapped() (value *NodeBootstrappedResult_, err err return } -func (p *NodeClient) BootstrappedInPlacementOrNoPlacement() (r *NodeBootstrappedInPlacementOrNoPlacementResult_, err error) { - if err = p.sendBootstrappedInPlacementOrNoPlacement(); err != nil { +// Parameters: +// - Req +func (p *NodeClient) SetWriteNewSeriesAsync(req *NodeSetWriteNewSeriesAsyncRequest) (r *NodeWriteNewSeriesAsyncResult_, err error) { + if err = p.sendSetWriteNewSeriesAsync(req); err != nil { return } - return p.recvBootstrappedInPlacementOrNoPlacement() + return p.recvSetWriteNewSeriesAsync() } -func (p *NodeClient) sendBootstrappedInPlacementOrNoPlacement() (err error) { +func (p *NodeClient) sendSetWriteNewSeriesAsync(req *NodeSetWriteNewSeriesAsyncRequest) (err error) { oprot := p.OutputProtocol if oprot == nil { oprot = p.ProtocolFactory.GetProtocol(p.Transport) p.OutputProtocol = oprot } p.SeqId++ - if err = oprot.WriteMessageBegin("bootstrappedInPlacementOrNoPlacement", thrift.CALL, p.SeqId); err != nil { + if err = oprot.WriteMessageBegin("setWriteNewSeriesAsync", thrift.CALL, p.SeqId); err != nil { return } - args := NodeBootstrappedInPlacementOrNoPlacementArgs{} + args := NodeSetWriteNewSeriesAsyncArgs{ + Req: req, + } if err = args.Write(oprot); err != nil { return } @@ -12870,7 +14128,7 @@ func (p *NodeClient) sendBootstrappedInPlacementOrNoPlacement() (err error) { return oprot.Flush() } -func (p *NodeClient) recvBootstrappedInPlacementOrNoPlacement() (value *NodeBootstrappedInPlacementOrNoPlacementResult_, err error) { +func (p *NodeClient) recvSetWriteNewSeriesAsync() (value *NodeWriteNewSeriesAsyncResult_, err error) { iprot := p.InputProtocol if iprot == nil { iprot = p.ProtocolFactory.GetProtocol(p.Transport) @@ -12880,32 +14138,32 @@ func (p *NodeClient) recvBootstrappedInPlacementOrNoPlacement() (value *NodeBoot if err != nil { return } - if method != "bootstrappedInPlacementOrNoPlacement" { - err = thrift.NewTApplicationException(thrift.WRONG_METHOD_NAME, "bootstrappedInPlacementOrNoPlacement failed: wrong method name") + if method != "setWriteNewSeriesAsync" { + err = thrift.NewTApplicationException(thrift.WRONG_METHOD_NAME, "setWriteNewSeriesAsync failed: wrong method name") return } if p.SeqId != seqId { - err = thrift.NewTApplicationException(thrift.BAD_SEQUENCE_ID, "bootstrappedInPlacementOrNoPlacement failed: out of sequence response") + err = thrift.NewTApplicationException(thrift.BAD_SEQUENCE_ID, "setWriteNewSeriesAsync failed: out of sequence response") return } if mTypeId == thrift.EXCEPTION { - error59 := thrift.NewTApplicationException(thrift.UNKNOWN_APPLICATION_EXCEPTION, "Unknown Exception") - var error60 error - error60, err = error59.Read(iprot) + error75 := thrift.NewTApplicationException(thrift.UNKNOWN_APPLICATION_EXCEPTION, "Unknown Exception") + var error76 error + error76, err = error75.Read(iprot) if err != nil { return } if err = iprot.ReadMessageEnd(); err != nil { return } - err = error60 + err = error76 return } if mTypeId != thrift.REPLY { - err = thrift.NewTApplicationException(thrift.INVALID_MESSAGE_TYPE_EXCEPTION, "bootstrappedInPlacementOrNoPlacement failed: invalid message type") + err = thrift.NewTApplicationException(thrift.INVALID_MESSAGE_TYPE_EXCEPTION, "setWriteNewSeriesAsync failed: invalid message type") return } - result := NodeBootstrappedInPlacementOrNoPlacementResult{} + result := NodeSetWriteNewSeriesAsyncResult{} if err = result.Read(iprot); err != nil { return } @@ -12920,24 +14178,24 @@ func (p *NodeClient) recvBootstrappedInPlacementOrNoPlacement() (value *NodeBoot return } -func (p *NodeClient) GetPersistRateLimit() (r *NodePersistRateLimitResult_, err error) { - if err = p.sendGetPersistRateLimit(); err != nil { +func (p *NodeClient) GetWriteNewSeriesBackoffDuration() (r *NodeWriteNewSeriesBackoffDurationResult_, err error) { + if err = p.sendGetWriteNewSeriesBackoffDuration(); err != nil { return } - return p.recvGetPersistRateLimit() + return p.recvGetWriteNewSeriesBackoffDuration() } -func (p *NodeClient) sendGetPersistRateLimit() (err error) { +func (p *NodeClient) sendGetWriteNewSeriesBackoffDuration() (err error) { oprot := p.OutputProtocol if oprot == nil { oprot = p.ProtocolFactory.GetProtocol(p.Transport) p.OutputProtocol = oprot } p.SeqId++ - if err = oprot.WriteMessageBegin("getPersistRateLimit", thrift.CALL, p.SeqId); err != nil { + if err = oprot.WriteMessageBegin("getWriteNewSeriesBackoffDuration", thrift.CALL, p.SeqId); err != nil { return } - args := NodeGetPersistRateLimitArgs{} + args := NodeGetWriteNewSeriesBackoffDurationArgs{} if err = args.Write(oprot); err != nil { return } @@ -12947,7 +14205,7 @@ func (p *NodeClient) sendGetPersistRateLimit() (err error) { return oprot.Flush() } -func (p *NodeClient) recvGetPersistRateLimit() (value *NodePersistRateLimitResult_, err error) { +func (p *NodeClient) recvGetWriteNewSeriesBackoffDuration() (value *NodeWriteNewSeriesBackoffDurationResult_, err error) { iprot := p.InputProtocol if iprot == nil { iprot = p.ProtocolFactory.GetProtocol(p.Transport) @@ -12957,32 +14215,32 @@ func (p *NodeClient) recvGetPersistRateLimit() (value *NodePersistRateLimitResul if err != nil { return } - if method != "getPersistRateLimit" { - err = thrift.NewTApplicationException(thrift.WRONG_METHOD_NAME, "getPersistRateLimit failed: wrong method name") + if method != "getWriteNewSeriesBackoffDuration" { + err = thrift.NewTApplicationException(thrift.WRONG_METHOD_NAME, "getWriteNewSeriesBackoffDuration failed: wrong method name") return } if p.SeqId != seqId { - err = thrift.NewTApplicationException(thrift.BAD_SEQUENCE_ID, "getPersistRateLimit failed: out of sequence response") + err = thrift.NewTApplicationException(thrift.BAD_SEQUENCE_ID, "getWriteNewSeriesBackoffDuration failed: out of sequence response") return } if mTypeId == thrift.EXCEPTION { - error61 := thrift.NewTApplicationException(thrift.UNKNOWN_APPLICATION_EXCEPTION, "Unknown Exception") - var error62 error - error62, err = error61.Read(iprot) + error77 := thrift.NewTApplicationException(thrift.UNKNOWN_APPLICATION_EXCEPTION, "Unknown Exception") + var error78 error + error78, err = error77.Read(iprot) if err != nil { return } if err = iprot.ReadMessageEnd(); err != nil { return } - err = error62 + err = error78 return } if mTypeId != thrift.REPLY { - err = thrift.NewTApplicationException(thrift.INVALID_MESSAGE_TYPE_EXCEPTION, "getPersistRateLimit failed: invalid message type") + err = thrift.NewTApplicationException(thrift.INVALID_MESSAGE_TYPE_EXCEPTION, "getWriteNewSeriesBackoffDuration failed: invalid message type") return } - result := NodeGetPersistRateLimitResult{} + result := NodeGetWriteNewSeriesBackoffDurationResult{} if err = result.Read(iprot); err != nil { return } @@ -12999,24 +14257,24 @@ func (p *NodeClient) recvGetPersistRateLimit() (value *NodePersistRateLimitResul // Parameters: // - Req -func (p *NodeClient) SetPersistRateLimit(req *NodeSetPersistRateLimitRequest) (r *NodePersistRateLimitResult_, err error) { - if err = p.sendSetPersistRateLimit(req); err != nil { +func (p *NodeClient) SetWriteNewSeriesBackoffDuration(req *NodeSetWriteNewSeriesBackoffDurationRequest) (r *NodeWriteNewSeriesBackoffDurationResult_, err error) { + if err = p.sendSetWriteNewSeriesBackoffDuration(req); err != nil { return } - return p.recvSetPersistRateLimit() + return p.recvSetWriteNewSeriesBackoffDuration() } -func (p *NodeClient) sendSetPersistRateLimit(req *NodeSetPersistRateLimitRequest) (err error) { +func (p *NodeClient) sendSetWriteNewSeriesBackoffDuration(req *NodeSetWriteNewSeriesBackoffDurationRequest) (err error) { oprot := p.OutputProtocol if oprot == nil { oprot = p.ProtocolFactory.GetProtocol(p.Transport) p.OutputProtocol = oprot } p.SeqId++ - if err = oprot.WriteMessageBegin("setPersistRateLimit", thrift.CALL, p.SeqId); err != nil { + if err = oprot.WriteMessageBegin("setWriteNewSeriesBackoffDuration", thrift.CALL, p.SeqId); err != nil { return } - args := NodeSetPersistRateLimitArgs{ + args := NodeSetWriteNewSeriesBackoffDurationArgs{ Req: req, } if err = args.Write(oprot); err != nil { @@ -13028,7 +14286,7 @@ func (p *NodeClient) sendSetPersistRateLimit(req *NodeSetPersistRateLimitRequest return oprot.Flush() } -func (p *NodeClient) recvSetPersistRateLimit() (value *NodePersistRateLimitResult_, err error) { +func (p *NodeClient) recvSetWriteNewSeriesBackoffDuration() (value *NodeWriteNewSeriesBackoffDurationResult_, err error) { iprot := p.InputProtocol if iprot == nil { iprot = p.ProtocolFactory.GetProtocol(p.Transport) @@ -13038,32 +14296,32 @@ func (p *NodeClient) recvSetPersistRateLimit() (value *NodePersistRateLimitResul if err != nil { return } - if method != "setPersistRateLimit" { - err = thrift.NewTApplicationException(thrift.WRONG_METHOD_NAME, "setPersistRateLimit failed: wrong method name") + if method != "setWriteNewSeriesBackoffDuration" { + err = thrift.NewTApplicationException(thrift.WRONG_METHOD_NAME, "setWriteNewSeriesBackoffDuration failed: wrong method name") return } if p.SeqId != seqId { - err = thrift.NewTApplicationException(thrift.BAD_SEQUENCE_ID, "setPersistRateLimit failed: out of sequence response") + err = thrift.NewTApplicationException(thrift.BAD_SEQUENCE_ID, "setWriteNewSeriesBackoffDuration failed: out of sequence response") return } if mTypeId == thrift.EXCEPTION { - error63 := thrift.NewTApplicationException(thrift.UNKNOWN_APPLICATION_EXCEPTION, "Unknown Exception") - var error64 error - error64, err = error63.Read(iprot) + error79 := thrift.NewTApplicationException(thrift.UNKNOWN_APPLICATION_EXCEPTION, "Unknown Exception") + var error80 error + error80, err = error79.Read(iprot) if err != nil { return } if err = iprot.ReadMessageEnd(); err != nil { return } - err = error64 + err = error80 return } if mTypeId != thrift.REPLY { - err = thrift.NewTApplicationException(thrift.INVALID_MESSAGE_TYPE_EXCEPTION, "setPersistRateLimit failed: invalid message type") + err = thrift.NewTApplicationException(thrift.INVALID_MESSAGE_TYPE_EXCEPTION, "setWriteNewSeriesBackoffDuration failed: invalid message type") return } - result := NodeSetPersistRateLimitResult{} + result := NodeSetWriteNewSeriesBackoffDurationResult{} if err = result.Read(iprot); err != nil { return } @@ -13078,24 +14336,24 @@ func (p *NodeClient) recvSetPersistRateLimit() (value *NodePersistRateLimitResul return } -func (p *NodeClient) GetWriteNewSeriesAsync() (r *NodeWriteNewSeriesAsyncResult_, err error) { - if err = p.sendGetWriteNewSeriesAsync(); err != nil { +func (p *NodeClient) GetWriteNewSeriesLimitPerShardPerSecond() (r *NodeWriteNewSeriesLimitPerShardPerSecondResult_, err error) { + if err = p.sendGetWriteNewSeriesLimitPerShardPerSecond(); err != nil { return } - return p.recvGetWriteNewSeriesAsync() + return p.recvGetWriteNewSeriesLimitPerShardPerSecond() } -func (p *NodeClient) sendGetWriteNewSeriesAsync() (err error) { +func (p *NodeClient) sendGetWriteNewSeriesLimitPerShardPerSecond() (err error) { oprot := p.OutputProtocol if oprot == nil { oprot = p.ProtocolFactory.GetProtocol(p.Transport) p.OutputProtocol = oprot } p.SeqId++ - if err = oprot.WriteMessageBegin("getWriteNewSeriesAsync", thrift.CALL, p.SeqId); err != nil { + if err = oprot.WriteMessageBegin("getWriteNewSeriesLimitPerShardPerSecond", thrift.CALL, p.SeqId); err != nil { return } - args := NodeGetWriteNewSeriesAsyncArgs{} + args := NodeGetWriteNewSeriesLimitPerShardPerSecondArgs{} if err = args.Write(oprot); err != nil { return } @@ -13105,7 +14363,7 @@ func (p *NodeClient) sendGetWriteNewSeriesAsync() (err error) { return oprot.Flush() } -func (p *NodeClient) recvGetWriteNewSeriesAsync() (value *NodeWriteNewSeriesAsyncResult_, err error) { +func (p *NodeClient) recvGetWriteNewSeriesLimitPerShardPerSecond() (value *NodeWriteNewSeriesLimitPerShardPerSecondResult_, err error) { iprot := p.InputProtocol if iprot == nil { iprot = p.ProtocolFactory.GetProtocol(p.Transport) @@ -13115,32 +14373,32 @@ func (p *NodeClient) recvGetWriteNewSeriesAsync() (value *NodeWriteNewSeriesAsyn if err != nil { return } - if method != "getWriteNewSeriesAsync" { - err = thrift.NewTApplicationException(thrift.WRONG_METHOD_NAME, "getWriteNewSeriesAsync failed: wrong method name") + if method != "getWriteNewSeriesLimitPerShardPerSecond" { + err = thrift.NewTApplicationException(thrift.WRONG_METHOD_NAME, "getWriteNewSeriesLimitPerShardPerSecond failed: wrong method name") return } if p.SeqId != seqId { - err = thrift.NewTApplicationException(thrift.BAD_SEQUENCE_ID, "getWriteNewSeriesAsync failed: out of sequence response") + err = thrift.NewTApplicationException(thrift.BAD_SEQUENCE_ID, "getWriteNewSeriesLimitPerShardPerSecond failed: out of sequence response") return } if mTypeId == thrift.EXCEPTION { - error65 := thrift.NewTApplicationException(thrift.UNKNOWN_APPLICATION_EXCEPTION, "Unknown Exception") - var error66 error - error66, err = error65.Read(iprot) + error81 := thrift.NewTApplicationException(thrift.UNKNOWN_APPLICATION_EXCEPTION, "Unknown Exception") + var error82 error + error82, err = error81.Read(iprot) if err != nil { return } if err = iprot.ReadMessageEnd(); err != nil { return } - err = error66 + err = error82 return } if mTypeId != thrift.REPLY { - err = thrift.NewTApplicationException(thrift.INVALID_MESSAGE_TYPE_EXCEPTION, "getWriteNewSeriesAsync failed: invalid message type") + err = thrift.NewTApplicationException(thrift.INVALID_MESSAGE_TYPE_EXCEPTION, "getWriteNewSeriesLimitPerShardPerSecond failed: invalid message type") return } - result := NodeGetWriteNewSeriesAsyncResult{} + result := NodeGetWriteNewSeriesLimitPerShardPerSecondResult{} if err = result.Read(iprot); err != nil { return } @@ -13157,24 +14415,24 @@ func (p *NodeClient) recvGetWriteNewSeriesAsync() (value *NodeWriteNewSeriesAsyn // Parameters: // - Req -func (p *NodeClient) SetWriteNewSeriesAsync(req *NodeSetWriteNewSeriesAsyncRequest) (r *NodeWriteNewSeriesAsyncResult_, err error) { - if err = p.sendSetWriteNewSeriesAsync(req); err != nil { +func (p *NodeClient) SetWriteNewSeriesLimitPerShardPerSecond(req *NodeSetWriteNewSeriesLimitPerShardPerSecondRequest) (r *NodeWriteNewSeriesLimitPerShardPerSecondResult_, err error) { + if err = p.sendSetWriteNewSeriesLimitPerShardPerSecond(req); err != nil { return } - return p.recvSetWriteNewSeriesAsync() + return p.recvSetWriteNewSeriesLimitPerShardPerSecond() } -func (p *NodeClient) sendSetWriteNewSeriesAsync(req *NodeSetWriteNewSeriesAsyncRequest) (err error) { +func (p *NodeClient) sendSetWriteNewSeriesLimitPerShardPerSecond(req *NodeSetWriteNewSeriesLimitPerShardPerSecondRequest) (err error) { oprot := p.OutputProtocol if oprot == nil { oprot = p.ProtocolFactory.GetProtocol(p.Transport) p.OutputProtocol = oprot } p.SeqId++ - if err = oprot.WriteMessageBegin("setWriteNewSeriesAsync", thrift.CALL, p.SeqId); err != nil { + if err = oprot.WriteMessageBegin("setWriteNewSeriesLimitPerShardPerSecond", thrift.CALL, p.SeqId); err != nil { return } - args := NodeSetWriteNewSeriesAsyncArgs{ + args := NodeSetWriteNewSeriesLimitPerShardPerSecondArgs{ Req: req, } if err = args.Write(oprot); err != nil { @@ -13186,7 +14444,7 @@ func (p *NodeClient) sendSetWriteNewSeriesAsync(req *NodeSetWriteNewSeriesAsyncR return oprot.Flush() } -func (p *NodeClient) recvSetWriteNewSeriesAsync() (value *NodeWriteNewSeriesAsyncResult_, err error) { +func (p *NodeClient) recvSetWriteNewSeriesLimitPerShardPerSecond() (value *NodeWriteNewSeriesLimitPerShardPerSecondResult_, err error) { iprot := p.InputProtocol if iprot == nil { iprot = p.ProtocolFactory.GetProtocol(p.Transport) @@ -13196,32 +14454,32 @@ func (p *NodeClient) recvSetWriteNewSeriesAsync() (value *NodeWriteNewSeriesAsyn if err != nil { return } - if method != "setWriteNewSeriesAsync" { - err = thrift.NewTApplicationException(thrift.WRONG_METHOD_NAME, "setWriteNewSeriesAsync failed: wrong method name") + if method != "setWriteNewSeriesLimitPerShardPerSecond" { + err = thrift.NewTApplicationException(thrift.WRONG_METHOD_NAME, "setWriteNewSeriesLimitPerShardPerSecond failed: wrong method name") return } if p.SeqId != seqId { - err = thrift.NewTApplicationException(thrift.BAD_SEQUENCE_ID, "setWriteNewSeriesAsync failed: out of sequence response") + err = thrift.NewTApplicationException(thrift.BAD_SEQUENCE_ID, "setWriteNewSeriesLimitPerShardPerSecond failed: out of sequence response") return } if mTypeId == thrift.EXCEPTION { - error67 := thrift.NewTApplicationException(thrift.UNKNOWN_APPLICATION_EXCEPTION, "Unknown Exception") - var error68 error - error68, err = error67.Read(iprot) + error83 := thrift.NewTApplicationException(thrift.UNKNOWN_APPLICATION_EXCEPTION, "Unknown Exception") + var error84 error + error84, err = error83.Read(iprot) if err != nil { return } if err = iprot.ReadMessageEnd(); err != nil { return } - err = error68 + err = error84 return } if mTypeId != thrift.REPLY { - err = thrift.NewTApplicationException(thrift.INVALID_MESSAGE_TYPE_EXCEPTION, "setWriteNewSeriesAsync failed: invalid message type") + err = thrift.NewTApplicationException(thrift.INVALID_MESSAGE_TYPE_EXCEPTION, "setWriteNewSeriesLimitPerShardPerSecond failed: invalid message type") return } - result := NodeSetWriteNewSeriesAsyncResult{} + result := NodeSetWriteNewSeriesLimitPerShardPerSecondResult{} if err = result.Read(iprot); err != nil { return } @@ -13236,400 +14494,451 @@ func (p *NodeClient) recvSetWriteNewSeriesAsync() (value *NodeWriteNewSeriesAsyn return } -func (p *NodeClient) GetWriteNewSeriesBackoffDuration() (r *NodeWriteNewSeriesBackoffDurationResult_, err error) { - if err = p.sendGetWriteNewSeriesBackoffDuration(); err != nil { - return +type NodeProcessor struct { + processorMap map[string]thrift.TProcessorFunction + handler Node +} + +func (p *NodeProcessor) AddToProcessorMap(key string, processor thrift.TProcessorFunction) { + p.processorMap[key] = processor +} + +func (p *NodeProcessor) GetProcessorFunction(key string) (processor thrift.TProcessorFunction, ok bool) { + processor, ok = p.processorMap[key] + return processor, ok +} + +func (p *NodeProcessor) ProcessorMap() map[string]thrift.TProcessorFunction { + return p.processorMap +} + +func NewNodeProcessor(handler Node) *NodeProcessor { + + self85 := &NodeProcessor{handler: handler, processorMap: make(map[string]thrift.TProcessorFunction)} + self85.processorMap["query"] = &nodeProcessorQuery{handler: handler} + self85.processorMap["aggregateRaw"] = &nodeProcessorAggregateRaw{handler: handler} + self85.processorMap["aggregate"] = &nodeProcessorAggregate{handler: handler} + self85.processorMap["fetch"] = &nodeProcessorFetch{handler: handler} + self85.processorMap["fetchTagged"] = &nodeProcessorFetchTagged{handler: handler} + self85.processorMap["write"] = &nodeProcessorWrite{handler: handler} + self85.processorMap["writeTagged"] = &nodeProcessorWriteTagged{handler: handler} + self85.processorMap["fetchBatchRaw"] = &nodeProcessorFetchBatchRaw{handler: handler} + self85.processorMap["fetchBlocksRaw"] = &nodeProcessorFetchBlocksRaw{handler: handler} + self85.processorMap["fetchBlocksMetadataRawV2"] = &nodeProcessorFetchBlocksMetadataRawV2{handler: handler} + self85.processorMap["writeBatchRaw"] = &nodeProcessorWriteBatchRaw{handler: handler} + self85.processorMap["writeBatchRawV2"] = &nodeProcessorWriteBatchRawV2{handler: handler} + self85.processorMap["writeTaggedBatchRaw"] = &nodeProcessorWriteTaggedBatchRaw{handler: handler} + self85.processorMap["writeTaggedBatchRawV2"] = &nodeProcessorWriteTaggedBatchRawV2{handler: handler} + self85.processorMap["repair"] = &nodeProcessorRepair{handler: handler} + self85.processorMap["truncate"] = &nodeProcessorTruncate{handler: handler} + self85.processorMap["health"] = &nodeProcessorHealth{handler: handler} + self85.processorMap["bootstrapped"] = &nodeProcessorBootstrapped{handler: handler} + self85.processorMap["bootstrappedInPlacementOrNoPlacement"] = &nodeProcessorBootstrappedInPlacementOrNoPlacement{handler: handler} + self85.processorMap["getPersistRateLimit"] = &nodeProcessorGetPersistRateLimit{handler: handler} + self85.processorMap["setPersistRateLimit"] = &nodeProcessorSetPersistRateLimit{handler: handler} + self85.processorMap["getWriteNewSeriesAsync"] = &nodeProcessorGetWriteNewSeriesAsync{handler: handler} + self85.processorMap["setWriteNewSeriesAsync"] = &nodeProcessorSetWriteNewSeriesAsync{handler: handler} + self85.processorMap["getWriteNewSeriesBackoffDuration"] = &nodeProcessorGetWriteNewSeriesBackoffDuration{handler: handler} + self85.processorMap["setWriteNewSeriesBackoffDuration"] = &nodeProcessorSetWriteNewSeriesBackoffDuration{handler: handler} + self85.processorMap["getWriteNewSeriesLimitPerShardPerSecond"] = &nodeProcessorGetWriteNewSeriesLimitPerShardPerSecond{handler: handler} + self85.processorMap["setWriteNewSeriesLimitPerShardPerSecond"] = &nodeProcessorSetWriteNewSeriesLimitPerShardPerSecond{handler: handler} + return self85 +} + +func (p *NodeProcessor) Process(iprot, oprot thrift.TProtocol) (success bool, err thrift.TException) { + name, _, seqId, err := iprot.ReadMessageBegin() + if err != nil { + return false, err } - return p.recvGetWriteNewSeriesBackoffDuration() + if processor, ok := p.GetProcessorFunction(name); ok { + return processor.Process(seqId, iprot, oprot) + } + iprot.Skip(thrift.STRUCT) + iprot.ReadMessageEnd() + x86 := thrift.NewTApplicationException(thrift.UNKNOWN_METHOD, "Unknown function "+name) + oprot.WriteMessageBegin(name, thrift.EXCEPTION, seqId) + x86.Write(oprot) + oprot.WriteMessageEnd() + oprot.Flush() + return false, x86 + } -func (p *NodeClient) sendGetWriteNewSeriesBackoffDuration() (err error) { - oprot := p.OutputProtocol - if oprot == nil { - oprot = p.ProtocolFactory.GetProtocol(p.Transport) - p.OutputProtocol = oprot +type nodeProcessorQuery struct { + handler Node +} + +func (p *nodeProcessorQuery) Process(seqId int32, iprot, oprot thrift.TProtocol) (success bool, err thrift.TException) { + args := NodeQueryArgs{} + if err = args.Read(iprot); err != nil { + iprot.ReadMessageEnd() + x := thrift.NewTApplicationException(thrift.PROTOCOL_ERROR, err.Error()) + oprot.WriteMessageBegin("query", thrift.EXCEPTION, seqId) + x.Write(oprot) + oprot.WriteMessageEnd() + oprot.Flush() + return false, err } - p.SeqId++ - if err = oprot.WriteMessageBegin("getWriteNewSeriesBackoffDuration", thrift.CALL, p.SeqId); err != nil { - return + + iprot.ReadMessageEnd() + result := NodeQueryResult{} + var retval *QueryResult_ + var err2 error + if retval, err2 = p.handler.Query(args.Req); err2 != nil { + switch v := err2.(type) { + case *Error: + result.Err = v + default: + x := thrift.NewTApplicationException(thrift.INTERNAL_ERROR, "Internal error processing query: "+err2.Error()) + oprot.WriteMessageBegin("query", thrift.EXCEPTION, seqId) + x.Write(oprot) + oprot.WriteMessageEnd() + oprot.Flush() + return true, err2 + } + } else { + result.Success = retval } - args := NodeGetWriteNewSeriesBackoffDurationArgs{} - if err = args.Write(oprot); err != nil { - return + if err2 = oprot.WriteMessageBegin("query", thrift.REPLY, seqId); err2 != nil { + err = err2 } - if err = oprot.WriteMessageEnd(); err != nil { - return + if err2 = result.Write(oprot); err == nil && err2 != nil { + err = err2 } - return oprot.Flush() -} - -func (p *NodeClient) recvGetWriteNewSeriesBackoffDuration() (value *NodeWriteNewSeriesBackoffDurationResult_, err error) { - iprot := p.InputProtocol - if iprot == nil { - iprot = p.ProtocolFactory.GetProtocol(p.Transport) - p.InputProtocol = iprot + if err2 = oprot.WriteMessageEnd(); err == nil && err2 != nil { + err = err2 } - method, mTypeId, seqId, err := iprot.ReadMessageBegin() - if err != nil { - return + if err2 = oprot.Flush(); err == nil && err2 != nil { + err = err2 } - if method != "getWriteNewSeriesBackoffDuration" { - err = thrift.NewTApplicationException(thrift.WRONG_METHOD_NAME, "getWriteNewSeriesBackoffDuration failed: wrong method name") + if err != nil { return } - if p.SeqId != seqId { - err = thrift.NewTApplicationException(thrift.BAD_SEQUENCE_ID, "getWriteNewSeriesBackoffDuration failed: out of sequence response") - return + return true, err +} + +type nodeProcessorAggregateRaw struct { + handler Node +} + +func (p *nodeProcessorAggregateRaw) Process(seqId int32, iprot, oprot thrift.TProtocol) (success bool, err thrift.TException) { + args := NodeAggregateRawArgs{} + if err = args.Read(iprot); err != nil { + iprot.ReadMessageEnd() + x := thrift.NewTApplicationException(thrift.PROTOCOL_ERROR, err.Error()) + oprot.WriteMessageBegin("aggregateRaw", thrift.EXCEPTION, seqId) + x.Write(oprot) + oprot.WriteMessageEnd() + oprot.Flush() + return false, err } - if mTypeId == thrift.EXCEPTION { - error69 := thrift.NewTApplicationException(thrift.UNKNOWN_APPLICATION_EXCEPTION, "Unknown Exception") - var error70 error - error70, err = error69.Read(iprot) - if err != nil { - return - } - if err = iprot.ReadMessageEnd(); err != nil { - return + + iprot.ReadMessageEnd() + result := NodeAggregateRawResult{} + var retval *AggregateQueryRawResult_ + var err2 error + if retval, err2 = p.handler.AggregateRaw(args.Req); err2 != nil { + switch v := err2.(type) { + case *Error: + result.Err = v + default: + x := thrift.NewTApplicationException(thrift.INTERNAL_ERROR, "Internal error processing aggregateRaw: "+err2.Error()) + oprot.WriteMessageBegin("aggregateRaw", thrift.EXCEPTION, seqId) + x.Write(oprot) + oprot.WriteMessageEnd() + oprot.Flush() + return true, err2 } - err = error70 - return + } else { + result.Success = retval } - if mTypeId != thrift.REPLY { - err = thrift.NewTApplicationException(thrift.INVALID_MESSAGE_TYPE_EXCEPTION, "getWriteNewSeriesBackoffDuration failed: invalid message type") - return + if err2 = oprot.WriteMessageBegin("aggregateRaw", thrift.REPLY, seqId); err2 != nil { + err = err2 } - result := NodeGetWriteNewSeriesBackoffDurationResult{} - if err = result.Read(iprot); err != nil { - return + if err2 = result.Write(oprot); err == nil && err2 != nil { + err = err2 } - if err = iprot.ReadMessageEnd(); err != nil { - return + if err2 = oprot.WriteMessageEnd(); err == nil && err2 != nil { + err = err2 } - if result.Err != nil { - err = result.Err + if err2 = oprot.Flush(); err == nil && err2 != nil { + err = err2 + } + if err != nil { return } - value = result.GetSuccess() - return + return true, err } -// Parameters: -// - Req -func (p *NodeClient) SetWriteNewSeriesBackoffDuration(req *NodeSetWriteNewSeriesBackoffDurationRequest) (r *NodeWriteNewSeriesBackoffDurationResult_, err error) { - if err = p.sendSetWriteNewSeriesBackoffDuration(req); err != nil { - return - } - return p.recvSetWriteNewSeriesBackoffDuration() +type nodeProcessorAggregate struct { + handler Node } -func (p *NodeClient) sendSetWriteNewSeriesBackoffDuration(req *NodeSetWriteNewSeriesBackoffDurationRequest) (err error) { - oprot := p.OutputProtocol - if oprot == nil { - oprot = p.ProtocolFactory.GetProtocol(p.Transport) - p.OutputProtocol = oprot +func (p *nodeProcessorAggregate) Process(seqId int32, iprot, oprot thrift.TProtocol) (success bool, err thrift.TException) { + args := NodeAggregateArgs{} + if err = args.Read(iprot); err != nil { + iprot.ReadMessageEnd() + x := thrift.NewTApplicationException(thrift.PROTOCOL_ERROR, err.Error()) + oprot.WriteMessageBegin("aggregate", thrift.EXCEPTION, seqId) + x.Write(oprot) + oprot.WriteMessageEnd() + oprot.Flush() + return false, err } - p.SeqId++ - if err = oprot.WriteMessageBegin("setWriteNewSeriesBackoffDuration", thrift.CALL, p.SeqId); err != nil { - return + + iprot.ReadMessageEnd() + result := NodeAggregateResult{} + var retval *AggregateQueryResult_ + var err2 error + if retval, err2 = p.handler.Aggregate(args.Req); err2 != nil { + switch v := err2.(type) { + case *Error: + result.Err = v + default: + x := thrift.NewTApplicationException(thrift.INTERNAL_ERROR, "Internal error processing aggregate: "+err2.Error()) + oprot.WriteMessageBegin("aggregate", thrift.EXCEPTION, seqId) + x.Write(oprot) + oprot.WriteMessageEnd() + oprot.Flush() + return true, err2 + } + } else { + result.Success = retval } - args := NodeSetWriteNewSeriesBackoffDurationArgs{ - Req: req, + if err2 = oprot.WriteMessageBegin("aggregate", thrift.REPLY, seqId); err2 != nil { + err = err2 } - if err = args.Write(oprot); err != nil { - return + if err2 = result.Write(oprot); err == nil && err2 != nil { + err = err2 } - if err = oprot.WriteMessageEnd(); err != nil { - return + if err2 = oprot.WriteMessageEnd(); err == nil && err2 != nil { + err = err2 } - return oprot.Flush() -} - -func (p *NodeClient) recvSetWriteNewSeriesBackoffDuration() (value *NodeWriteNewSeriesBackoffDurationResult_, err error) { - iprot := p.InputProtocol - if iprot == nil { - iprot = p.ProtocolFactory.GetProtocol(p.Transport) - p.InputProtocol = iprot + if err2 = oprot.Flush(); err == nil && err2 != nil { + err = err2 } - method, mTypeId, seqId, err := iprot.ReadMessageBegin() if err != nil { return } - if method != "setWriteNewSeriesBackoffDuration" { - err = thrift.NewTApplicationException(thrift.WRONG_METHOD_NAME, "setWriteNewSeriesBackoffDuration failed: wrong method name") - return - } - if p.SeqId != seqId { - err = thrift.NewTApplicationException(thrift.BAD_SEQUENCE_ID, "setWriteNewSeriesBackoffDuration failed: out of sequence response") - return + return true, err +} + +type nodeProcessorFetch struct { + handler Node +} + +func (p *nodeProcessorFetch) Process(seqId int32, iprot, oprot thrift.TProtocol) (success bool, err thrift.TException) { + args := NodeFetchArgs{} + if err = args.Read(iprot); err != nil { + iprot.ReadMessageEnd() + x := thrift.NewTApplicationException(thrift.PROTOCOL_ERROR, err.Error()) + oprot.WriteMessageBegin("fetch", thrift.EXCEPTION, seqId) + x.Write(oprot) + oprot.WriteMessageEnd() + oprot.Flush() + return false, err } - if mTypeId == thrift.EXCEPTION { - error71 := thrift.NewTApplicationException(thrift.UNKNOWN_APPLICATION_EXCEPTION, "Unknown Exception") - var error72 error - error72, err = error71.Read(iprot) - if err != nil { - return - } - if err = iprot.ReadMessageEnd(); err != nil { - return + + iprot.ReadMessageEnd() + result := NodeFetchResult{} + var retval *FetchResult_ + var err2 error + if retval, err2 = p.handler.Fetch(args.Req); err2 != nil { + switch v := err2.(type) { + case *Error: + result.Err = v + default: + x := thrift.NewTApplicationException(thrift.INTERNAL_ERROR, "Internal error processing fetch: "+err2.Error()) + oprot.WriteMessageBegin("fetch", thrift.EXCEPTION, seqId) + x.Write(oprot) + oprot.WriteMessageEnd() + oprot.Flush() + return true, err2 } - err = error72 - return + } else { + result.Success = retval } - if mTypeId != thrift.REPLY { - err = thrift.NewTApplicationException(thrift.INVALID_MESSAGE_TYPE_EXCEPTION, "setWriteNewSeriesBackoffDuration failed: invalid message type") - return + if err2 = oprot.WriteMessageBegin("fetch", thrift.REPLY, seqId); err2 != nil { + err = err2 } - result := NodeSetWriteNewSeriesBackoffDurationResult{} - if err = result.Read(iprot); err != nil { - return + if err2 = result.Write(oprot); err == nil && err2 != nil { + err = err2 } - if err = iprot.ReadMessageEnd(); err != nil { - return + if err2 = oprot.WriteMessageEnd(); err == nil && err2 != nil { + err = err2 } - if result.Err != nil { - err = result.Err - return + if err2 = oprot.Flush(); err == nil && err2 != nil { + err = err2 } - value = result.GetSuccess() - return -} - -func (p *NodeClient) GetWriteNewSeriesLimitPerShardPerSecond() (r *NodeWriteNewSeriesLimitPerShardPerSecondResult_, err error) { - if err = p.sendGetWriteNewSeriesLimitPerShardPerSecond(); err != nil { + if err != nil { return } - return p.recvGetWriteNewSeriesLimitPerShardPerSecond() + return true, err } -func (p *NodeClient) sendGetWriteNewSeriesLimitPerShardPerSecond() (err error) { - oprot := p.OutputProtocol - if oprot == nil { - oprot = p.ProtocolFactory.GetProtocol(p.Transport) - p.OutputProtocol = oprot - } - p.SeqId++ - if err = oprot.WriteMessageBegin("getWriteNewSeriesLimitPerShardPerSecond", thrift.CALL, p.SeqId); err != nil { - return - } - args := NodeGetWriteNewSeriesLimitPerShardPerSecondArgs{} - if err = args.Write(oprot); err != nil { - return - } - if err = oprot.WriteMessageEnd(); err != nil { - return - } - return oprot.Flush() +type nodeProcessorFetchTagged struct { + handler Node } -func (p *NodeClient) recvGetWriteNewSeriesLimitPerShardPerSecond() (value *NodeWriteNewSeriesLimitPerShardPerSecondResult_, err error) { - iprot := p.InputProtocol - if iprot == nil { - iprot = p.ProtocolFactory.GetProtocol(p.Transport) - p.InputProtocol = iprot - } - method, mTypeId, seqId, err := iprot.ReadMessageBegin() - if err != nil { - return - } - if method != "getWriteNewSeriesLimitPerShardPerSecond" { - err = thrift.NewTApplicationException(thrift.WRONG_METHOD_NAME, "getWriteNewSeriesLimitPerShardPerSecond failed: wrong method name") - return - } - if p.SeqId != seqId { - err = thrift.NewTApplicationException(thrift.BAD_SEQUENCE_ID, "getWriteNewSeriesLimitPerShardPerSecond failed: out of sequence response") - return +func (p *nodeProcessorFetchTagged) Process(seqId int32, iprot, oprot thrift.TProtocol) (success bool, err thrift.TException) { + args := NodeFetchTaggedArgs{} + if err = args.Read(iprot); err != nil { + iprot.ReadMessageEnd() + x := thrift.NewTApplicationException(thrift.PROTOCOL_ERROR, err.Error()) + oprot.WriteMessageBegin("fetchTagged", thrift.EXCEPTION, seqId) + x.Write(oprot) + oprot.WriteMessageEnd() + oprot.Flush() + return false, err } - if mTypeId == thrift.EXCEPTION { - error73 := thrift.NewTApplicationException(thrift.UNKNOWN_APPLICATION_EXCEPTION, "Unknown Exception") - var error74 error - error74, err = error73.Read(iprot) - if err != nil { - return - } - if err = iprot.ReadMessageEnd(); err != nil { - return + + iprot.ReadMessageEnd() + result := NodeFetchTaggedResult{} + var retval *FetchTaggedResult_ + var err2 error + if retval, err2 = p.handler.FetchTagged(args.Req); err2 != nil { + switch v := err2.(type) { + case *Error: + result.Err = v + default: + x := thrift.NewTApplicationException(thrift.INTERNAL_ERROR, "Internal error processing fetchTagged: "+err2.Error()) + oprot.WriteMessageBegin("fetchTagged", thrift.EXCEPTION, seqId) + x.Write(oprot) + oprot.WriteMessageEnd() + oprot.Flush() + return true, err2 } - err = error74 - return + } else { + result.Success = retval + } + if err2 = oprot.WriteMessageBegin("fetchTagged", thrift.REPLY, seqId); err2 != nil { + err = err2 } - if mTypeId != thrift.REPLY { - err = thrift.NewTApplicationException(thrift.INVALID_MESSAGE_TYPE_EXCEPTION, "getWriteNewSeriesLimitPerShardPerSecond failed: invalid message type") - return + if err2 = result.Write(oprot); err == nil && err2 != nil { + err = err2 } - result := NodeGetWriteNewSeriesLimitPerShardPerSecondResult{} - if err = result.Read(iprot); err != nil { - return + if err2 = oprot.WriteMessageEnd(); err == nil && err2 != nil { + err = err2 } - if err = iprot.ReadMessageEnd(); err != nil { - return + if err2 = oprot.Flush(); err == nil && err2 != nil { + err = err2 } - if result.Err != nil { - err = result.Err + if err != nil { return } - value = result.GetSuccess() - return + return true, err } -// Parameters: -// - Req -func (p *NodeClient) SetWriteNewSeriesLimitPerShardPerSecond(req *NodeSetWriteNewSeriesLimitPerShardPerSecondRequest) (r *NodeWriteNewSeriesLimitPerShardPerSecondResult_, err error) { - if err = p.sendSetWriteNewSeriesLimitPerShardPerSecond(req); err != nil { - return - } - return p.recvSetWriteNewSeriesLimitPerShardPerSecond() +type nodeProcessorWrite struct { + handler Node } -func (p *NodeClient) sendSetWriteNewSeriesLimitPerShardPerSecond(req *NodeSetWriteNewSeriesLimitPerShardPerSecondRequest) (err error) { - oprot := p.OutputProtocol - if oprot == nil { - oprot = p.ProtocolFactory.GetProtocol(p.Transport) - p.OutputProtocol = oprot +func (p *nodeProcessorWrite) Process(seqId int32, iprot, oprot thrift.TProtocol) (success bool, err thrift.TException) { + args := NodeWriteArgs{} + if err = args.Read(iprot); err != nil { + iprot.ReadMessageEnd() + x := thrift.NewTApplicationException(thrift.PROTOCOL_ERROR, err.Error()) + oprot.WriteMessageBegin("write", thrift.EXCEPTION, seqId) + x.Write(oprot) + oprot.WriteMessageEnd() + oprot.Flush() + return false, err } - p.SeqId++ - if err = oprot.WriteMessageBegin("setWriteNewSeriesLimitPerShardPerSecond", thrift.CALL, p.SeqId); err != nil { - return + + iprot.ReadMessageEnd() + result := NodeWriteResult{} + var err2 error + if err2 = p.handler.Write(args.Req); err2 != nil { + switch v := err2.(type) { + case *Error: + result.Err = v + default: + x := thrift.NewTApplicationException(thrift.INTERNAL_ERROR, "Internal error processing write: "+err2.Error()) + oprot.WriteMessageBegin("write", thrift.EXCEPTION, seqId) + x.Write(oprot) + oprot.WriteMessageEnd() + oprot.Flush() + return true, err2 + } } - args := NodeSetWriteNewSeriesLimitPerShardPerSecondArgs{ - Req: req, + if err2 = oprot.WriteMessageBegin("write", thrift.REPLY, seqId); err2 != nil { + err = err2 } - if err = args.Write(oprot); err != nil { - return + if err2 = result.Write(oprot); err == nil && err2 != nil { + err = err2 } - if err = oprot.WriteMessageEnd(); err != nil { - return + if err2 = oprot.WriteMessageEnd(); err == nil && err2 != nil { + err = err2 } - return oprot.Flush() -} - -func (p *NodeClient) recvSetWriteNewSeriesLimitPerShardPerSecond() (value *NodeWriteNewSeriesLimitPerShardPerSecondResult_, err error) { - iprot := p.InputProtocol - if iprot == nil { - iprot = p.ProtocolFactory.GetProtocol(p.Transport) - p.InputProtocol = iprot + if err2 = oprot.Flush(); err == nil && err2 != nil { + err = err2 } - method, mTypeId, seqId, err := iprot.ReadMessageBegin() if err != nil { return } - if method != "setWriteNewSeriesLimitPerShardPerSecond" { - err = thrift.NewTApplicationException(thrift.WRONG_METHOD_NAME, "setWriteNewSeriesLimitPerShardPerSecond failed: wrong method name") - return - } - if p.SeqId != seqId { - err = thrift.NewTApplicationException(thrift.BAD_SEQUENCE_ID, "setWriteNewSeriesLimitPerShardPerSecond failed: out of sequence response") - return + return true, err +} + +type nodeProcessorWriteTagged struct { + handler Node +} + +func (p *nodeProcessorWriteTagged) Process(seqId int32, iprot, oprot thrift.TProtocol) (success bool, err thrift.TException) { + args := NodeWriteTaggedArgs{} + if err = args.Read(iprot); err != nil { + iprot.ReadMessageEnd() + x := thrift.NewTApplicationException(thrift.PROTOCOL_ERROR, err.Error()) + oprot.WriteMessageBegin("writeTagged", thrift.EXCEPTION, seqId) + x.Write(oprot) + oprot.WriteMessageEnd() + oprot.Flush() + return false, err } - if mTypeId == thrift.EXCEPTION { - error75 := thrift.NewTApplicationException(thrift.UNKNOWN_APPLICATION_EXCEPTION, "Unknown Exception") - var error76 error - error76, err = error75.Read(iprot) - if err != nil { - return - } - if err = iprot.ReadMessageEnd(); err != nil { - return + + iprot.ReadMessageEnd() + result := NodeWriteTaggedResult{} + var err2 error + if err2 = p.handler.WriteTagged(args.Req); err2 != nil { + switch v := err2.(type) { + case *Error: + result.Err = v + default: + x := thrift.NewTApplicationException(thrift.INTERNAL_ERROR, "Internal error processing writeTagged: "+err2.Error()) + oprot.WriteMessageBegin("writeTagged", thrift.EXCEPTION, seqId) + x.Write(oprot) + oprot.WriteMessageEnd() + oprot.Flush() + return true, err2 } - err = error76 - return } - if mTypeId != thrift.REPLY { - err = thrift.NewTApplicationException(thrift.INVALID_MESSAGE_TYPE_EXCEPTION, "setWriteNewSeriesLimitPerShardPerSecond failed: invalid message type") - return + if err2 = oprot.WriteMessageBegin("writeTagged", thrift.REPLY, seqId); err2 != nil { + err = err2 } - result := NodeSetWriteNewSeriesLimitPerShardPerSecondResult{} - if err = result.Read(iprot); err != nil { - return + if err2 = result.Write(oprot); err == nil && err2 != nil { + err = err2 } - if err = iprot.ReadMessageEnd(); err != nil { - return + if err2 = oprot.WriteMessageEnd(); err == nil && err2 != nil { + err = err2 } - if result.Err != nil { - err = result.Err - return + if err2 = oprot.Flush(); err == nil && err2 != nil { + err = err2 } - value = result.GetSuccess() - return -} - -type NodeProcessor struct { - processorMap map[string]thrift.TProcessorFunction - handler Node -} - -func (p *NodeProcessor) AddToProcessorMap(key string, processor thrift.TProcessorFunction) { - p.processorMap[key] = processor -} - -func (p *NodeProcessor) GetProcessorFunction(key string) (processor thrift.TProcessorFunction, ok bool) { - processor, ok = p.processorMap[key] - return processor, ok -} - -func (p *NodeProcessor) ProcessorMap() map[string]thrift.TProcessorFunction { - return p.processorMap -} - -func NewNodeProcessor(handler Node) *NodeProcessor { - - self77 := &NodeProcessor{handler: handler, processorMap: make(map[string]thrift.TProcessorFunction)} - self77.processorMap["query"] = &nodeProcessorQuery{handler: handler} - self77.processorMap["aggregateRaw"] = &nodeProcessorAggregateRaw{handler: handler} - self77.processorMap["aggregate"] = &nodeProcessorAggregate{handler: handler} - self77.processorMap["fetch"] = &nodeProcessorFetch{handler: handler} - self77.processorMap["fetchTagged"] = &nodeProcessorFetchTagged{handler: handler} - self77.processorMap["write"] = &nodeProcessorWrite{handler: handler} - self77.processorMap["writeTagged"] = &nodeProcessorWriteTagged{handler: handler} - self77.processorMap["fetchBatchRaw"] = &nodeProcessorFetchBatchRaw{handler: handler} - self77.processorMap["fetchBlocksRaw"] = &nodeProcessorFetchBlocksRaw{handler: handler} - self77.processorMap["fetchBlocksMetadataRawV2"] = &nodeProcessorFetchBlocksMetadataRawV2{handler: handler} - self77.processorMap["writeBatchRaw"] = &nodeProcessorWriteBatchRaw{handler: handler} - self77.processorMap["writeTaggedBatchRaw"] = &nodeProcessorWriteTaggedBatchRaw{handler: handler} - self77.processorMap["repair"] = &nodeProcessorRepair{handler: handler} - self77.processorMap["truncate"] = &nodeProcessorTruncate{handler: handler} - self77.processorMap["health"] = &nodeProcessorHealth{handler: handler} - self77.processorMap["bootstrapped"] = &nodeProcessorBootstrapped{handler: handler} - self77.processorMap["bootstrappedInPlacementOrNoPlacement"] = &nodeProcessorBootstrappedInPlacementOrNoPlacement{handler: handler} - self77.processorMap["getPersistRateLimit"] = &nodeProcessorGetPersistRateLimit{handler: handler} - self77.processorMap["setPersistRateLimit"] = &nodeProcessorSetPersistRateLimit{handler: handler} - self77.processorMap["getWriteNewSeriesAsync"] = &nodeProcessorGetWriteNewSeriesAsync{handler: handler} - self77.processorMap["setWriteNewSeriesAsync"] = &nodeProcessorSetWriteNewSeriesAsync{handler: handler} - self77.processorMap["getWriteNewSeriesBackoffDuration"] = &nodeProcessorGetWriteNewSeriesBackoffDuration{handler: handler} - self77.processorMap["setWriteNewSeriesBackoffDuration"] = &nodeProcessorSetWriteNewSeriesBackoffDuration{handler: handler} - self77.processorMap["getWriteNewSeriesLimitPerShardPerSecond"] = &nodeProcessorGetWriteNewSeriesLimitPerShardPerSecond{handler: handler} - self77.processorMap["setWriteNewSeriesLimitPerShardPerSecond"] = &nodeProcessorSetWriteNewSeriesLimitPerShardPerSecond{handler: handler} - return self77 -} - -func (p *NodeProcessor) Process(iprot, oprot thrift.TProtocol) (success bool, err thrift.TException) { - name, _, seqId, err := iprot.ReadMessageBegin() if err != nil { - return false, err - } - if processor, ok := p.GetProcessorFunction(name); ok { - return processor.Process(seqId, iprot, oprot) + return } - iprot.Skip(thrift.STRUCT) - iprot.ReadMessageEnd() - x78 := thrift.NewTApplicationException(thrift.UNKNOWN_METHOD, "Unknown function "+name) - oprot.WriteMessageBegin(name, thrift.EXCEPTION, seqId) - x78.Write(oprot) - oprot.WriteMessageEnd() - oprot.Flush() - return false, x78 - + return true, err } -type nodeProcessorQuery struct { +type nodeProcessorFetchBatchRaw struct { handler Node } -func (p *nodeProcessorQuery) Process(seqId int32, iprot, oprot thrift.TProtocol) (success bool, err thrift.TException) { - args := NodeQueryArgs{} +func (p *nodeProcessorFetchBatchRaw) Process(seqId int32, iprot, oprot thrift.TProtocol) (success bool, err thrift.TException) { + args := NodeFetchBatchRawArgs{} if err = args.Read(iprot); err != nil { iprot.ReadMessageEnd() x := thrift.NewTApplicationException(thrift.PROTOCOL_ERROR, err.Error()) - oprot.WriteMessageBegin("query", thrift.EXCEPTION, seqId) + oprot.WriteMessageBegin("fetchBatchRaw", thrift.EXCEPTION, seqId) x.Write(oprot) oprot.WriteMessageEnd() oprot.Flush() @@ -13637,16 +14946,16 @@ func (p *nodeProcessorQuery) Process(seqId int32, iprot, oprot thrift.TProtocol) } iprot.ReadMessageEnd() - result := NodeQueryResult{} - var retval *QueryResult_ + result := NodeFetchBatchRawResult{} + var retval *FetchBatchRawResult_ var err2 error - if retval, err2 = p.handler.Query(args.Req); err2 != nil { + if retval, err2 = p.handler.FetchBatchRaw(args.Req); err2 != nil { switch v := err2.(type) { case *Error: result.Err = v default: - x := thrift.NewTApplicationException(thrift.INTERNAL_ERROR, "Internal error processing query: "+err2.Error()) - oprot.WriteMessageBegin("query", thrift.EXCEPTION, seqId) + x := thrift.NewTApplicationException(thrift.INTERNAL_ERROR, "Internal error processing fetchBatchRaw: "+err2.Error()) + oprot.WriteMessageBegin("fetchBatchRaw", thrift.EXCEPTION, seqId) x.Write(oprot) oprot.WriteMessageEnd() oprot.Flush() @@ -13655,7 +14964,7 @@ func (p *nodeProcessorQuery) Process(seqId int32, iprot, oprot thrift.TProtocol) } else { result.Success = retval } - if err2 = oprot.WriteMessageBegin("query", thrift.REPLY, seqId); err2 != nil { + if err2 = oprot.WriteMessageBegin("fetchBatchRaw", thrift.REPLY, seqId); err2 != nil { err = err2 } if err2 = result.Write(oprot); err == nil && err2 != nil { @@ -13673,16 +14982,16 @@ func (p *nodeProcessorQuery) Process(seqId int32, iprot, oprot thrift.TProtocol) return true, err } -type nodeProcessorAggregateRaw struct { +type nodeProcessorFetchBlocksRaw struct { handler Node } -func (p *nodeProcessorAggregateRaw) Process(seqId int32, iprot, oprot thrift.TProtocol) (success bool, err thrift.TException) { - args := NodeAggregateRawArgs{} +func (p *nodeProcessorFetchBlocksRaw) Process(seqId int32, iprot, oprot thrift.TProtocol) (success bool, err thrift.TException) { + args := NodeFetchBlocksRawArgs{} if err = args.Read(iprot); err != nil { iprot.ReadMessageEnd() x := thrift.NewTApplicationException(thrift.PROTOCOL_ERROR, err.Error()) - oprot.WriteMessageBegin("aggregateRaw", thrift.EXCEPTION, seqId) + oprot.WriteMessageBegin("fetchBlocksRaw", thrift.EXCEPTION, seqId) x.Write(oprot) oprot.WriteMessageEnd() oprot.Flush() @@ -13690,16 +14999,16 @@ func (p *nodeProcessorAggregateRaw) Process(seqId int32, iprot, oprot thrift.TPr } iprot.ReadMessageEnd() - result := NodeAggregateRawResult{} - var retval *AggregateQueryRawResult_ + result := NodeFetchBlocksRawResult{} + var retval *FetchBlocksRawResult_ var err2 error - if retval, err2 = p.handler.AggregateRaw(args.Req); err2 != nil { + if retval, err2 = p.handler.FetchBlocksRaw(args.Req); err2 != nil { switch v := err2.(type) { case *Error: result.Err = v default: - x := thrift.NewTApplicationException(thrift.INTERNAL_ERROR, "Internal error processing aggregateRaw: "+err2.Error()) - oprot.WriteMessageBegin("aggregateRaw", thrift.EXCEPTION, seqId) + x := thrift.NewTApplicationException(thrift.INTERNAL_ERROR, "Internal error processing fetchBlocksRaw: "+err2.Error()) + oprot.WriteMessageBegin("fetchBlocksRaw", thrift.EXCEPTION, seqId) x.Write(oprot) oprot.WriteMessageEnd() oprot.Flush() @@ -13708,7 +15017,7 @@ func (p *nodeProcessorAggregateRaw) Process(seqId int32, iprot, oprot thrift.TPr } else { result.Success = retval } - if err2 = oprot.WriteMessageBegin("aggregateRaw", thrift.REPLY, seqId); err2 != nil { + if err2 = oprot.WriteMessageBegin("fetchBlocksRaw", thrift.REPLY, seqId); err2 != nil { err = err2 } if err2 = result.Write(oprot); err == nil && err2 != nil { @@ -13726,16 +15035,16 @@ func (p *nodeProcessorAggregateRaw) Process(seqId int32, iprot, oprot thrift.TPr return true, err } -type nodeProcessorAggregate struct { +type nodeProcessorFetchBlocksMetadataRawV2 struct { handler Node } -func (p *nodeProcessorAggregate) Process(seqId int32, iprot, oprot thrift.TProtocol) (success bool, err thrift.TException) { - args := NodeAggregateArgs{} +func (p *nodeProcessorFetchBlocksMetadataRawV2) Process(seqId int32, iprot, oprot thrift.TProtocol) (success bool, err thrift.TException) { + args := NodeFetchBlocksMetadataRawV2Args{} if err = args.Read(iprot); err != nil { iprot.ReadMessageEnd() x := thrift.NewTApplicationException(thrift.PROTOCOL_ERROR, err.Error()) - oprot.WriteMessageBegin("aggregate", thrift.EXCEPTION, seqId) + oprot.WriteMessageBegin("fetchBlocksMetadataRawV2", thrift.EXCEPTION, seqId) x.Write(oprot) oprot.WriteMessageEnd() oprot.Flush() @@ -13743,16 +15052,16 @@ func (p *nodeProcessorAggregate) Process(seqId int32, iprot, oprot thrift.TProto } iprot.ReadMessageEnd() - result := NodeAggregateResult{} - var retval *AggregateQueryResult_ + result := NodeFetchBlocksMetadataRawV2Result{} + var retval *FetchBlocksMetadataRawV2Result_ var err2 error - if retval, err2 = p.handler.Aggregate(args.Req); err2 != nil { + if retval, err2 = p.handler.FetchBlocksMetadataRawV2(args.Req); err2 != nil { switch v := err2.(type) { case *Error: result.Err = v default: - x := thrift.NewTApplicationException(thrift.INTERNAL_ERROR, "Internal error processing aggregate: "+err2.Error()) - oprot.WriteMessageBegin("aggregate", thrift.EXCEPTION, seqId) + x := thrift.NewTApplicationException(thrift.INTERNAL_ERROR, "Internal error processing fetchBlocksMetadataRawV2: "+err2.Error()) + oprot.WriteMessageBegin("fetchBlocksMetadataRawV2", thrift.EXCEPTION, seqId) x.Write(oprot) oprot.WriteMessageEnd() oprot.Flush() @@ -13761,7 +15070,7 @@ func (p *nodeProcessorAggregate) Process(seqId int32, iprot, oprot thrift.TProto } else { result.Success = retval } - if err2 = oprot.WriteMessageBegin("aggregate", thrift.REPLY, seqId); err2 != nil { + if err2 = oprot.WriteMessageBegin("fetchBlocksMetadataRawV2", thrift.REPLY, seqId); err2 != nil { err = err2 } if err2 = result.Write(oprot); err == nil && err2 != nil { @@ -13779,16 +15088,16 @@ func (p *nodeProcessorAggregate) Process(seqId int32, iprot, oprot thrift.TProto return true, err } -type nodeProcessorFetch struct { +type nodeProcessorWriteBatchRaw struct { handler Node } -func (p *nodeProcessorFetch) Process(seqId int32, iprot, oprot thrift.TProtocol) (success bool, err thrift.TException) { - args := NodeFetchArgs{} +func (p *nodeProcessorWriteBatchRaw) Process(seqId int32, iprot, oprot thrift.TProtocol) (success bool, err thrift.TException) { + args := NodeWriteBatchRawArgs{} if err = args.Read(iprot); err != nil { iprot.ReadMessageEnd() x := thrift.NewTApplicationException(thrift.PROTOCOL_ERROR, err.Error()) - oprot.WriteMessageBegin("fetch", thrift.EXCEPTION, seqId) + oprot.WriteMessageBegin("writeBatchRaw", thrift.EXCEPTION, seqId) x.Write(oprot) oprot.WriteMessageEnd() oprot.Flush() @@ -13796,25 +15105,22 @@ func (p *nodeProcessorFetch) Process(seqId int32, iprot, oprot thrift.TProtocol) } iprot.ReadMessageEnd() - result := NodeFetchResult{} - var retval *FetchResult_ + result := NodeWriteBatchRawResult{} var err2 error - if retval, err2 = p.handler.Fetch(args.Req); err2 != nil { + if err2 = p.handler.WriteBatchRaw(args.Req); err2 != nil { switch v := err2.(type) { - case *Error: + case *WriteBatchRawErrors: result.Err = v default: - x := thrift.NewTApplicationException(thrift.INTERNAL_ERROR, "Internal error processing fetch: "+err2.Error()) - oprot.WriteMessageBegin("fetch", thrift.EXCEPTION, seqId) + x := thrift.NewTApplicationException(thrift.INTERNAL_ERROR, "Internal error processing writeBatchRaw: "+err2.Error()) + oprot.WriteMessageBegin("writeBatchRaw", thrift.EXCEPTION, seqId) x.Write(oprot) oprot.WriteMessageEnd() oprot.Flush() return true, err2 } - } else { - result.Success = retval } - if err2 = oprot.WriteMessageBegin("fetch", thrift.REPLY, seqId); err2 != nil { + if err2 = oprot.WriteMessageBegin("writeBatchRaw", thrift.REPLY, seqId); err2 != nil { err = err2 } if err2 = result.Write(oprot); err == nil && err2 != nil { @@ -13832,16 +15138,16 @@ func (p *nodeProcessorFetch) Process(seqId int32, iprot, oprot thrift.TProtocol) return true, err } -type nodeProcessorFetchTagged struct { +type nodeProcessorWriteBatchRawV2 struct { handler Node } -func (p *nodeProcessorFetchTagged) Process(seqId int32, iprot, oprot thrift.TProtocol) (success bool, err thrift.TException) { - args := NodeFetchTaggedArgs{} +func (p *nodeProcessorWriteBatchRawV2) Process(seqId int32, iprot, oprot thrift.TProtocol) (success bool, err thrift.TException) { + args := NodeWriteBatchRawV2Args{} if err = args.Read(iprot); err != nil { iprot.ReadMessageEnd() x := thrift.NewTApplicationException(thrift.PROTOCOL_ERROR, err.Error()) - oprot.WriteMessageBegin("fetchTagged", thrift.EXCEPTION, seqId) + oprot.WriteMessageBegin("writeBatchRawV2", thrift.EXCEPTION, seqId) x.Write(oprot) oprot.WriteMessageEnd() oprot.Flush() @@ -13849,25 +15155,22 @@ func (p *nodeProcessorFetchTagged) Process(seqId int32, iprot, oprot thrift.TPro } iprot.ReadMessageEnd() - result := NodeFetchTaggedResult{} - var retval *FetchTaggedResult_ + result := NodeWriteBatchRawV2Result{} var err2 error - if retval, err2 = p.handler.FetchTagged(args.Req); err2 != nil { + if err2 = p.handler.WriteBatchRawV2(args.Req); err2 != nil { switch v := err2.(type) { - case *Error: + case *WriteBatchRawErrors: result.Err = v default: - x := thrift.NewTApplicationException(thrift.INTERNAL_ERROR, "Internal error processing fetchTagged: "+err2.Error()) - oprot.WriteMessageBegin("fetchTagged", thrift.EXCEPTION, seqId) + x := thrift.NewTApplicationException(thrift.INTERNAL_ERROR, "Internal error processing writeBatchRawV2: "+err2.Error()) + oprot.WriteMessageBegin("writeBatchRawV2", thrift.EXCEPTION, seqId) x.Write(oprot) oprot.WriteMessageEnd() oprot.Flush() return true, err2 } - } else { - result.Success = retval } - if err2 = oprot.WriteMessageBegin("fetchTagged", thrift.REPLY, seqId); err2 != nil { + if err2 = oprot.WriteMessageBegin("writeBatchRawV2", thrift.REPLY, seqId); err2 != nil { err = err2 } if err2 = result.Write(oprot); err == nil && err2 != nil { @@ -13885,16 +15188,16 @@ func (p *nodeProcessorFetchTagged) Process(seqId int32, iprot, oprot thrift.TPro return true, err } -type nodeProcessorWrite struct { +type nodeProcessorWriteTaggedBatchRaw struct { handler Node } -func (p *nodeProcessorWrite) Process(seqId int32, iprot, oprot thrift.TProtocol) (success bool, err thrift.TException) { - args := NodeWriteArgs{} +func (p *nodeProcessorWriteTaggedBatchRaw) Process(seqId int32, iprot, oprot thrift.TProtocol) (success bool, err thrift.TException) { + args := NodeWriteTaggedBatchRawArgs{} if err = args.Read(iprot); err != nil { iprot.ReadMessageEnd() x := thrift.NewTApplicationException(thrift.PROTOCOL_ERROR, err.Error()) - oprot.WriteMessageBegin("write", thrift.EXCEPTION, seqId) + oprot.WriteMessageBegin("writeTaggedBatchRaw", thrift.EXCEPTION, seqId) x.Write(oprot) oprot.WriteMessageEnd() oprot.Flush() @@ -13902,22 +15205,22 @@ func (p *nodeProcessorWrite) Process(seqId int32, iprot, oprot thrift.TProtocol) } iprot.ReadMessageEnd() - result := NodeWriteResult{} + result := NodeWriteTaggedBatchRawResult{} var err2 error - if err2 = p.handler.Write(args.Req); err2 != nil { + if err2 = p.handler.WriteTaggedBatchRaw(args.Req); err2 != nil { switch v := err2.(type) { - case *Error: + case *WriteBatchRawErrors: result.Err = v default: - x := thrift.NewTApplicationException(thrift.INTERNAL_ERROR, "Internal error processing write: "+err2.Error()) - oprot.WriteMessageBegin("write", thrift.EXCEPTION, seqId) + x := thrift.NewTApplicationException(thrift.INTERNAL_ERROR, "Internal error processing writeTaggedBatchRaw: "+err2.Error()) + oprot.WriteMessageBegin("writeTaggedBatchRaw", thrift.EXCEPTION, seqId) x.Write(oprot) oprot.WriteMessageEnd() oprot.Flush() return true, err2 } } - if err2 = oprot.WriteMessageBegin("write", thrift.REPLY, seqId); err2 != nil { + if err2 = oprot.WriteMessageBegin("writeTaggedBatchRaw", thrift.REPLY, seqId); err2 != nil { err = err2 } if err2 = result.Write(oprot); err == nil && err2 != nil { @@ -13935,16 +15238,16 @@ func (p *nodeProcessorWrite) Process(seqId int32, iprot, oprot thrift.TProtocol) return true, err } -type nodeProcessorWriteTagged struct { +type nodeProcessorWriteTaggedBatchRawV2 struct { handler Node } -func (p *nodeProcessorWriteTagged) Process(seqId int32, iprot, oprot thrift.TProtocol) (success bool, err thrift.TException) { - args := NodeWriteTaggedArgs{} +func (p *nodeProcessorWriteTaggedBatchRawV2) Process(seqId int32, iprot, oprot thrift.TProtocol) (success bool, err thrift.TException) { + args := NodeWriteTaggedBatchRawV2Args{} if err = args.Read(iprot); err != nil { iprot.ReadMessageEnd() x := thrift.NewTApplicationException(thrift.PROTOCOL_ERROR, err.Error()) - oprot.WriteMessageBegin("writeTagged", thrift.EXCEPTION, seqId) + oprot.WriteMessageBegin("writeTaggedBatchRawV2", thrift.EXCEPTION, seqId) x.Write(oprot) oprot.WriteMessageEnd() oprot.Flush() @@ -13952,22 +15255,22 @@ func (p *nodeProcessorWriteTagged) Process(seqId int32, iprot, oprot thrift.TPro } iprot.ReadMessageEnd() - result := NodeWriteTaggedResult{} + result := NodeWriteTaggedBatchRawV2Result{} var err2 error - if err2 = p.handler.WriteTagged(args.Req); err2 != nil { + if err2 = p.handler.WriteTaggedBatchRawV2(args.Req); err2 != nil { switch v := err2.(type) { - case *Error: + case *WriteBatchRawErrors: result.Err = v default: - x := thrift.NewTApplicationException(thrift.INTERNAL_ERROR, "Internal error processing writeTagged: "+err2.Error()) - oprot.WriteMessageBegin("writeTagged", thrift.EXCEPTION, seqId) + x := thrift.NewTApplicationException(thrift.INTERNAL_ERROR, "Internal error processing writeTaggedBatchRawV2: "+err2.Error()) + oprot.WriteMessageBegin("writeTaggedBatchRawV2", thrift.EXCEPTION, seqId) x.Write(oprot) oprot.WriteMessageEnd() oprot.Flush() return true, err2 } } - if err2 = oprot.WriteMessageBegin("writeTagged", thrift.REPLY, seqId); err2 != nil { + if err2 = oprot.WriteMessageBegin("writeTaggedBatchRawV2", thrift.REPLY, seqId); err2 != nil { err = err2 } if err2 = result.Write(oprot); err == nil && err2 != nil { @@ -13985,16 +15288,16 @@ func (p *nodeProcessorWriteTagged) Process(seqId int32, iprot, oprot thrift.TPro return true, err } -type nodeProcessorFetchBatchRaw struct { +type nodeProcessorRepair struct { handler Node } -func (p *nodeProcessorFetchBatchRaw) Process(seqId int32, iprot, oprot thrift.TProtocol) (success bool, err thrift.TException) { - args := NodeFetchBatchRawArgs{} +func (p *nodeProcessorRepair) Process(seqId int32, iprot, oprot thrift.TProtocol) (success bool, err thrift.TException) { + args := NodeRepairArgs{} if err = args.Read(iprot); err != nil { iprot.ReadMessageEnd() x := thrift.NewTApplicationException(thrift.PROTOCOL_ERROR, err.Error()) - oprot.WriteMessageBegin("fetchBatchRaw", thrift.EXCEPTION, seqId) + oprot.WriteMessageBegin("repair", thrift.EXCEPTION, seqId) x.Write(oprot) oprot.WriteMessageEnd() oprot.Flush() @@ -14002,25 +15305,22 @@ func (p *nodeProcessorFetchBatchRaw) Process(seqId int32, iprot, oprot thrift.TP } iprot.ReadMessageEnd() - result := NodeFetchBatchRawResult{} - var retval *FetchBatchRawResult_ + result := NodeRepairResult{} var err2 error - if retval, err2 = p.handler.FetchBatchRaw(args.Req); err2 != nil { + if err2 = p.handler.Repair(); err2 != nil { switch v := err2.(type) { case *Error: result.Err = v default: - x := thrift.NewTApplicationException(thrift.INTERNAL_ERROR, "Internal error processing fetchBatchRaw: "+err2.Error()) - oprot.WriteMessageBegin("fetchBatchRaw", thrift.EXCEPTION, seqId) + x := thrift.NewTApplicationException(thrift.INTERNAL_ERROR, "Internal error processing repair: "+err2.Error()) + oprot.WriteMessageBegin("repair", thrift.EXCEPTION, seqId) x.Write(oprot) oprot.WriteMessageEnd() oprot.Flush() return true, err2 } - } else { - result.Success = retval } - if err2 = oprot.WriteMessageBegin("fetchBatchRaw", thrift.REPLY, seqId); err2 != nil { + if err2 = oprot.WriteMessageBegin("repair", thrift.REPLY, seqId); err2 != nil { err = err2 } if err2 = result.Write(oprot); err == nil && err2 != nil { @@ -14038,16 +15338,16 @@ func (p *nodeProcessorFetchBatchRaw) Process(seqId int32, iprot, oprot thrift.TP return true, err } -type nodeProcessorFetchBlocksRaw struct { +type nodeProcessorTruncate struct { handler Node } -func (p *nodeProcessorFetchBlocksRaw) Process(seqId int32, iprot, oprot thrift.TProtocol) (success bool, err thrift.TException) { - args := NodeFetchBlocksRawArgs{} +func (p *nodeProcessorTruncate) Process(seqId int32, iprot, oprot thrift.TProtocol) (success bool, err thrift.TException) { + args := NodeTruncateArgs{} if err = args.Read(iprot); err != nil { iprot.ReadMessageEnd() x := thrift.NewTApplicationException(thrift.PROTOCOL_ERROR, err.Error()) - oprot.WriteMessageBegin("fetchBlocksRaw", thrift.EXCEPTION, seqId) + oprot.WriteMessageBegin("truncate", thrift.EXCEPTION, seqId) x.Write(oprot) oprot.WriteMessageEnd() oprot.Flush() @@ -14055,16 +15355,16 @@ func (p *nodeProcessorFetchBlocksRaw) Process(seqId int32, iprot, oprot thrift.T } iprot.ReadMessageEnd() - result := NodeFetchBlocksRawResult{} - var retval *FetchBlocksRawResult_ + result := NodeTruncateResult{} + var retval *TruncateResult_ var err2 error - if retval, err2 = p.handler.FetchBlocksRaw(args.Req); err2 != nil { + if retval, err2 = p.handler.Truncate(args.Req); err2 != nil { switch v := err2.(type) { case *Error: result.Err = v default: - x := thrift.NewTApplicationException(thrift.INTERNAL_ERROR, "Internal error processing fetchBlocksRaw: "+err2.Error()) - oprot.WriteMessageBegin("fetchBlocksRaw", thrift.EXCEPTION, seqId) + x := thrift.NewTApplicationException(thrift.INTERNAL_ERROR, "Internal error processing truncate: "+err2.Error()) + oprot.WriteMessageBegin("truncate", thrift.EXCEPTION, seqId) x.Write(oprot) oprot.WriteMessageEnd() oprot.Flush() @@ -14073,7 +15373,7 @@ func (p *nodeProcessorFetchBlocksRaw) Process(seqId int32, iprot, oprot thrift.T } else { result.Success = retval } - if err2 = oprot.WriteMessageBegin("fetchBlocksRaw", thrift.REPLY, seqId); err2 != nil { + if err2 = oprot.WriteMessageBegin("truncate", thrift.REPLY, seqId); err2 != nil { err = err2 } if err2 = result.Write(oprot); err == nil && err2 != nil { @@ -14091,16 +15391,16 @@ func (p *nodeProcessorFetchBlocksRaw) Process(seqId int32, iprot, oprot thrift.T return true, err } -type nodeProcessorFetchBlocksMetadataRawV2 struct { +type nodeProcessorHealth struct { handler Node } -func (p *nodeProcessorFetchBlocksMetadataRawV2) Process(seqId int32, iprot, oprot thrift.TProtocol) (success bool, err thrift.TException) { - args := NodeFetchBlocksMetadataRawV2Args{} +func (p *nodeProcessorHealth) Process(seqId int32, iprot, oprot thrift.TProtocol) (success bool, err thrift.TException) { + args := NodeHealthArgs{} if err = args.Read(iprot); err != nil { iprot.ReadMessageEnd() x := thrift.NewTApplicationException(thrift.PROTOCOL_ERROR, err.Error()) - oprot.WriteMessageBegin("fetchBlocksMetadataRawV2", thrift.EXCEPTION, seqId) + oprot.WriteMessageBegin("health", thrift.EXCEPTION, seqId) x.Write(oprot) oprot.WriteMessageEnd() oprot.Flush() @@ -14108,16 +15408,16 @@ func (p *nodeProcessorFetchBlocksMetadataRawV2) Process(seqId int32, iprot, opro } iprot.ReadMessageEnd() - result := NodeFetchBlocksMetadataRawV2Result{} - var retval *FetchBlocksMetadataRawV2Result_ + result := NodeHealthResult{} + var retval *NodeHealthResult_ var err2 error - if retval, err2 = p.handler.FetchBlocksMetadataRawV2(args.Req); err2 != nil { + if retval, err2 = p.handler.Health(); err2 != nil { switch v := err2.(type) { case *Error: result.Err = v default: - x := thrift.NewTApplicationException(thrift.INTERNAL_ERROR, "Internal error processing fetchBlocksMetadataRawV2: "+err2.Error()) - oprot.WriteMessageBegin("fetchBlocksMetadataRawV2", thrift.EXCEPTION, seqId) + x := thrift.NewTApplicationException(thrift.INTERNAL_ERROR, "Internal error processing health: "+err2.Error()) + oprot.WriteMessageBegin("health", thrift.EXCEPTION, seqId) x.Write(oprot) oprot.WriteMessageEnd() oprot.Flush() @@ -14126,7 +15426,7 @@ func (p *nodeProcessorFetchBlocksMetadataRawV2) Process(seqId int32, iprot, opro } else { result.Success = retval } - if err2 = oprot.WriteMessageBegin("fetchBlocksMetadataRawV2", thrift.REPLY, seqId); err2 != nil { + if err2 = oprot.WriteMessageBegin("health", thrift.REPLY, seqId); err2 != nil { err = err2 } if err2 = result.Write(oprot); err == nil && err2 != nil { @@ -14144,16 +15444,16 @@ func (p *nodeProcessorFetchBlocksMetadataRawV2) Process(seqId int32, iprot, opro return true, err } -type nodeProcessorWriteBatchRaw struct { +type nodeProcessorBootstrapped struct { handler Node } -func (p *nodeProcessorWriteBatchRaw) Process(seqId int32, iprot, oprot thrift.TProtocol) (success bool, err thrift.TException) { - args := NodeWriteBatchRawArgs{} +func (p *nodeProcessorBootstrapped) Process(seqId int32, iprot, oprot thrift.TProtocol) (success bool, err thrift.TException) { + args := NodeBootstrappedArgs{} if err = args.Read(iprot); err != nil { iprot.ReadMessageEnd() x := thrift.NewTApplicationException(thrift.PROTOCOL_ERROR, err.Error()) - oprot.WriteMessageBegin("writeBatchRaw", thrift.EXCEPTION, seqId) + oprot.WriteMessageBegin("bootstrapped", thrift.EXCEPTION, seqId) x.Write(oprot) oprot.WriteMessageEnd() oprot.Flush() @@ -14161,22 +15461,25 @@ func (p *nodeProcessorWriteBatchRaw) Process(seqId int32, iprot, oprot thrift.TP } iprot.ReadMessageEnd() - result := NodeWriteBatchRawResult{} + result := NodeBootstrappedResult{} + var retval *NodeBootstrappedResult_ var err2 error - if err2 = p.handler.WriteBatchRaw(args.Req); err2 != nil { + if retval, err2 = p.handler.Bootstrapped(); err2 != nil { switch v := err2.(type) { - case *WriteBatchRawErrors: + case *Error: result.Err = v default: - x := thrift.NewTApplicationException(thrift.INTERNAL_ERROR, "Internal error processing writeBatchRaw: "+err2.Error()) - oprot.WriteMessageBegin("writeBatchRaw", thrift.EXCEPTION, seqId) + x := thrift.NewTApplicationException(thrift.INTERNAL_ERROR, "Internal error processing bootstrapped: "+err2.Error()) + oprot.WriteMessageBegin("bootstrapped", thrift.EXCEPTION, seqId) x.Write(oprot) oprot.WriteMessageEnd() oprot.Flush() return true, err2 } + } else { + result.Success = retval } - if err2 = oprot.WriteMessageBegin("writeBatchRaw", thrift.REPLY, seqId); err2 != nil { + if err2 = oprot.WriteMessageBegin("bootstrapped", thrift.REPLY, seqId); err2 != nil { err = err2 } if err2 = result.Write(oprot); err == nil && err2 != nil { @@ -14194,16 +15497,16 @@ func (p *nodeProcessorWriteBatchRaw) Process(seqId int32, iprot, oprot thrift.TP return true, err } -type nodeProcessorWriteTaggedBatchRaw struct { +type nodeProcessorBootstrappedInPlacementOrNoPlacement struct { handler Node } -func (p *nodeProcessorWriteTaggedBatchRaw) Process(seqId int32, iprot, oprot thrift.TProtocol) (success bool, err thrift.TException) { - args := NodeWriteTaggedBatchRawArgs{} +func (p *nodeProcessorBootstrappedInPlacementOrNoPlacement) Process(seqId int32, iprot, oprot thrift.TProtocol) (success bool, err thrift.TException) { + args := NodeBootstrappedInPlacementOrNoPlacementArgs{} if err = args.Read(iprot); err != nil { iprot.ReadMessageEnd() x := thrift.NewTApplicationException(thrift.PROTOCOL_ERROR, err.Error()) - oprot.WriteMessageBegin("writeTaggedBatchRaw", thrift.EXCEPTION, seqId) + oprot.WriteMessageBegin("bootstrappedInPlacementOrNoPlacement", thrift.EXCEPTION, seqId) x.Write(oprot) oprot.WriteMessageEnd() oprot.Flush() @@ -14211,22 +15514,25 @@ func (p *nodeProcessorWriteTaggedBatchRaw) Process(seqId int32, iprot, oprot thr } iprot.ReadMessageEnd() - result := NodeWriteTaggedBatchRawResult{} + result := NodeBootstrappedInPlacementOrNoPlacementResult{} + var retval *NodeBootstrappedInPlacementOrNoPlacementResult_ var err2 error - if err2 = p.handler.WriteTaggedBatchRaw(args.Req); err2 != nil { + if retval, err2 = p.handler.BootstrappedInPlacementOrNoPlacement(); err2 != nil { switch v := err2.(type) { - case *WriteBatchRawErrors: + case *Error: result.Err = v default: - x := thrift.NewTApplicationException(thrift.INTERNAL_ERROR, "Internal error processing writeTaggedBatchRaw: "+err2.Error()) - oprot.WriteMessageBegin("writeTaggedBatchRaw", thrift.EXCEPTION, seqId) + x := thrift.NewTApplicationException(thrift.INTERNAL_ERROR, "Internal error processing bootstrappedInPlacementOrNoPlacement: "+err2.Error()) + oprot.WriteMessageBegin("bootstrappedInPlacementOrNoPlacement", thrift.EXCEPTION, seqId) x.Write(oprot) oprot.WriteMessageEnd() oprot.Flush() return true, err2 } + } else { + result.Success = retval } - if err2 = oprot.WriteMessageBegin("writeTaggedBatchRaw", thrift.REPLY, seqId); err2 != nil { + if err2 = oprot.WriteMessageBegin("bootstrappedInPlacementOrNoPlacement", thrift.REPLY, seqId); err2 != nil { err = err2 } if err2 = result.Write(oprot); err == nil && err2 != nil { @@ -14244,16 +15550,16 @@ func (p *nodeProcessorWriteTaggedBatchRaw) Process(seqId int32, iprot, oprot thr return true, err } -type nodeProcessorRepair struct { +type nodeProcessorGetPersistRateLimit struct { handler Node } -func (p *nodeProcessorRepair) Process(seqId int32, iprot, oprot thrift.TProtocol) (success bool, err thrift.TException) { - args := NodeRepairArgs{} +func (p *nodeProcessorGetPersistRateLimit) Process(seqId int32, iprot, oprot thrift.TProtocol) (success bool, err thrift.TException) { + args := NodeGetPersistRateLimitArgs{} if err = args.Read(iprot); err != nil { iprot.ReadMessageEnd() x := thrift.NewTApplicationException(thrift.PROTOCOL_ERROR, err.Error()) - oprot.WriteMessageBegin("repair", thrift.EXCEPTION, seqId) + oprot.WriteMessageBegin("getPersistRateLimit", thrift.EXCEPTION, seqId) x.Write(oprot) oprot.WriteMessageEnd() oprot.Flush() @@ -14261,22 +15567,25 @@ func (p *nodeProcessorRepair) Process(seqId int32, iprot, oprot thrift.TProtocol } iprot.ReadMessageEnd() - result := NodeRepairResult{} + result := NodeGetPersistRateLimitResult{} + var retval *NodePersistRateLimitResult_ var err2 error - if err2 = p.handler.Repair(); err2 != nil { + if retval, err2 = p.handler.GetPersistRateLimit(); err2 != nil { switch v := err2.(type) { case *Error: result.Err = v default: - x := thrift.NewTApplicationException(thrift.INTERNAL_ERROR, "Internal error processing repair: "+err2.Error()) - oprot.WriteMessageBegin("repair", thrift.EXCEPTION, seqId) + x := thrift.NewTApplicationException(thrift.INTERNAL_ERROR, "Internal error processing getPersistRateLimit: "+err2.Error()) + oprot.WriteMessageBegin("getPersistRateLimit", thrift.EXCEPTION, seqId) x.Write(oprot) oprot.WriteMessageEnd() oprot.Flush() return true, err2 } + } else { + result.Success = retval } - if err2 = oprot.WriteMessageBegin("repair", thrift.REPLY, seqId); err2 != nil { + if err2 = oprot.WriteMessageBegin("getPersistRateLimit", thrift.REPLY, seqId); err2 != nil { err = err2 } if err2 = result.Write(oprot); err == nil && err2 != nil { @@ -14294,16 +15603,16 @@ func (p *nodeProcessorRepair) Process(seqId int32, iprot, oprot thrift.TProtocol return true, err } -type nodeProcessorTruncate struct { +type nodeProcessorSetPersistRateLimit struct { handler Node } -func (p *nodeProcessorTruncate) Process(seqId int32, iprot, oprot thrift.TProtocol) (success bool, err thrift.TException) { - args := NodeTruncateArgs{} +func (p *nodeProcessorSetPersistRateLimit) Process(seqId int32, iprot, oprot thrift.TProtocol) (success bool, err thrift.TException) { + args := NodeSetPersistRateLimitArgs{} if err = args.Read(iprot); err != nil { iprot.ReadMessageEnd() x := thrift.NewTApplicationException(thrift.PROTOCOL_ERROR, err.Error()) - oprot.WriteMessageBegin("truncate", thrift.EXCEPTION, seqId) + oprot.WriteMessageBegin("setPersistRateLimit", thrift.EXCEPTION, seqId) x.Write(oprot) oprot.WriteMessageEnd() oprot.Flush() @@ -14311,16 +15620,16 @@ func (p *nodeProcessorTruncate) Process(seqId int32, iprot, oprot thrift.TProtoc } iprot.ReadMessageEnd() - result := NodeTruncateResult{} - var retval *TruncateResult_ + result := NodeSetPersistRateLimitResult{} + var retval *NodePersistRateLimitResult_ var err2 error - if retval, err2 = p.handler.Truncate(args.Req); err2 != nil { + if retval, err2 = p.handler.SetPersistRateLimit(args.Req); err2 != nil { switch v := err2.(type) { case *Error: result.Err = v default: - x := thrift.NewTApplicationException(thrift.INTERNAL_ERROR, "Internal error processing truncate: "+err2.Error()) - oprot.WriteMessageBegin("truncate", thrift.EXCEPTION, seqId) + x := thrift.NewTApplicationException(thrift.INTERNAL_ERROR, "Internal error processing setPersistRateLimit: "+err2.Error()) + oprot.WriteMessageBegin("setPersistRateLimit", thrift.EXCEPTION, seqId) x.Write(oprot) oprot.WriteMessageEnd() oprot.Flush() @@ -14329,7 +15638,7 @@ func (p *nodeProcessorTruncate) Process(seqId int32, iprot, oprot thrift.TProtoc } else { result.Success = retval } - if err2 = oprot.WriteMessageBegin("truncate", thrift.REPLY, seqId); err2 != nil { + if err2 = oprot.WriteMessageBegin("setPersistRateLimit", thrift.REPLY, seqId); err2 != nil { err = err2 } if err2 = result.Write(oprot); err == nil && err2 != nil { @@ -14347,16 +15656,16 @@ func (p *nodeProcessorTruncate) Process(seqId int32, iprot, oprot thrift.TProtoc return true, err } -type nodeProcessorHealth struct { +type nodeProcessorGetWriteNewSeriesAsync struct { handler Node } -func (p *nodeProcessorHealth) Process(seqId int32, iprot, oprot thrift.TProtocol) (success bool, err thrift.TException) { - args := NodeHealthArgs{} +func (p *nodeProcessorGetWriteNewSeriesAsync) Process(seqId int32, iprot, oprot thrift.TProtocol) (success bool, err thrift.TException) { + args := NodeGetWriteNewSeriesAsyncArgs{} if err = args.Read(iprot); err != nil { iprot.ReadMessageEnd() x := thrift.NewTApplicationException(thrift.PROTOCOL_ERROR, err.Error()) - oprot.WriteMessageBegin("health", thrift.EXCEPTION, seqId) + oprot.WriteMessageBegin("getWriteNewSeriesAsync", thrift.EXCEPTION, seqId) x.Write(oprot) oprot.WriteMessageEnd() oprot.Flush() @@ -14364,16 +15673,16 @@ func (p *nodeProcessorHealth) Process(seqId int32, iprot, oprot thrift.TProtocol } iprot.ReadMessageEnd() - result := NodeHealthResult{} - var retval *NodeHealthResult_ + result := NodeGetWriteNewSeriesAsyncResult{} + var retval *NodeWriteNewSeriesAsyncResult_ var err2 error - if retval, err2 = p.handler.Health(); err2 != nil { + if retval, err2 = p.handler.GetWriteNewSeriesAsync(); err2 != nil { switch v := err2.(type) { case *Error: result.Err = v default: - x := thrift.NewTApplicationException(thrift.INTERNAL_ERROR, "Internal error processing health: "+err2.Error()) - oprot.WriteMessageBegin("health", thrift.EXCEPTION, seqId) + x := thrift.NewTApplicationException(thrift.INTERNAL_ERROR, "Internal error processing getWriteNewSeriesAsync: "+err2.Error()) + oprot.WriteMessageBegin("getWriteNewSeriesAsync", thrift.EXCEPTION, seqId) x.Write(oprot) oprot.WriteMessageEnd() oprot.Flush() @@ -14382,7 +15691,7 @@ func (p *nodeProcessorHealth) Process(seqId int32, iprot, oprot thrift.TProtocol } else { result.Success = retval } - if err2 = oprot.WriteMessageBegin("health", thrift.REPLY, seqId); err2 != nil { + if err2 = oprot.WriteMessageBegin("getWriteNewSeriesAsync", thrift.REPLY, seqId); err2 != nil { err = err2 } if err2 = result.Write(oprot); err == nil && err2 != nil { @@ -14400,16 +15709,16 @@ func (p *nodeProcessorHealth) Process(seqId int32, iprot, oprot thrift.TProtocol return true, err } -type nodeProcessorBootstrapped struct { +type nodeProcessorSetWriteNewSeriesAsync struct { handler Node } -func (p *nodeProcessorBootstrapped) Process(seqId int32, iprot, oprot thrift.TProtocol) (success bool, err thrift.TException) { - args := NodeBootstrappedArgs{} +func (p *nodeProcessorSetWriteNewSeriesAsync) Process(seqId int32, iprot, oprot thrift.TProtocol) (success bool, err thrift.TException) { + args := NodeSetWriteNewSeriesAsyncArgs{} if err = args.Read(iprot); err != nil { iprot.ReadMessageEnd() x := thrift.NewTApplicationException(thrift.PROTOCOL_ERROR, err.Error()) - oprot.WriteMessageBegin("bootstrapped", thrift.EXCEPTION, seqId) + oprot.WriteMessageBegin("setWriteNewSeriesAsync", thrift.EXCEPTION, seqId) x.Write(oprot) oprot.WriteMessageEnd() oprot.Flush() @@ -14417,16 +15726,16 @@ func (p *nodeProcessorBootstrapped) Process(seqId int32, iprot, oprot thrift.TPr } iprot.ReadMessageEnd() - result := NodeBootstrappedResult{} - var retval *NodeBootstrappedResult_ + result := NodeSetWriteNewSeriesAsyncResult{} + var retval *NodeWriteNewSeriesAsyncResult_ var err2 error - if retval, err2 = p.handler.Bootstrapped(); err2 != nil { + if retval, err2 = p.handler.SetWriteNewSeriesAsync(args.Req); err2 != nil { switch v := err2.(type) { case *Error: result.Err = v default: - x := thrift.NewTApplicationException(thrift.INTERNAL_ERROR, "Internal error processing bootstrapped: "+err2.Error()) - oprot.WriteMessageBegin("bootstrapped", thrift.EXCEPTION, seqId) + x := thrift.NewTApplicationException(thrift.INTERNAL_ERROR, "Internal error processing setWriteNewSeriesAsync: "+err2.Error()) + oprot.WriteMessageBegin("setWriteNewSeriesAsync", thrift.EXCEPTION, seqId) x.Write(oprot) oprot.WriteMessageEnd() oprot.Flush() @@ -14435,7 +15744,7 @@ func (p *nodeProcessorBootstrapped) Process(seqId int32, iprot, oprot thrift.TPr } else { result.Success = retval } - if err2 = oprot.WriteMessageBegin("bootstrapped", thrift.REPLY, seqId); err2 != nil { + if err2 = oprot.WriteMessageBegin("setWriteNewSeriesAsync", thrift.REPLY, seqId); err2 != nil { err = err2 } if err2 = result.Write(oprot); err == nil && err2 != nil { @@ -14453,16 +15762,16 @@ func (p *nodeProcessorBootstrapped) Process(seqId int32, iprot, oprot thrift.TPr return true, err } -type nodeProcessorBootstrappedInPlacementOrNoPlacement struct { +type nodeProcessorGetWriteNewSeriesBackoffDuration struct { handler Node } -func (p *nodeProcessorBootstrappedInPlacementOrNoPlacement) Process(seqId int32, iprot, oprot thrift.TProtocol) (success bool, err thrift.TException) { - args := NodeBootstrappedInPlacementOrNoPlacementArgs{} +func (p *nodeProcessorGetWriteNewSeriesBackoffDuration) Process(seqId int32, iprot, oprot thrift.TProtocol) (success bool, err thrift.TException) { + args := NodeGetWriteNewSeriesBackoffDurationArgs{} if err = args.Read(iprot); err != nil { iprot.ReadMessageEnd() x := thrift.NewTApplicationException(thrift.PROTOCOL_ERROR, err.Error()) - oprot.WriteMessageBegin("bootstrappedInPlacementOrNoPlacement", thrift.EXCEPTION, seqId) + oprot.WriteMessageBegin("getWriteNewSeriesBackoffDuration", thrift.EXCEPTION, seqId) x.Write(oprot) oprot.WriteMessageEnd() oprot.Flush() @@ -14470,16 +15779,16 @@ func (p *nodeProcessorBootstrappedInPlacementOrNoPlacement) Process(seqId int32, } iprot.ReadMessageEnd() - result := NodeBootstrappedInPlacementOrNoPlacementResult{} - var retval *NodeBootstrappedInPlacementOrNoPlacementResult_ + result := NodeGetWriteNewSeriesBackoffDurationResult{} + var retval *NodeWriteNewSeriesBackoffDurationResult_ var err2 error - if retval, err2 = p.handler.BootstrappedInPlacementOrNoPlacement(); err2 != nil { + if retval, err2 = p.handler.GetWriteNewSeriesBackoffDuration(); err2 != nil { switch v := err2.(type) { case *Error: result.Err = v default: - x := thrift.NewTApplicationException(thrift.INTERNAL_ERROR, "Internal error processing bootstrappedInPlacementOrNoPlacement: "+err2.Error()) - oprot.WriteMessageBegin("bootstrappedInPlacementOrNoPlacement", thrift.EXCEPTION, seqId) + x := thrift.NewTApplicationException(thrift.INTERNAL_ERROR, "Internal error processing getWriteNewSeriesBackoffDuration: "+err2.Error()) + oprot.WriteMessageBegin("getWriteNewSeriesBackoffDuration", thrift.EXCEPTION, seqId) x.Write(oprot) oprot.WriteMessageEnd() oprot.Flush() @@ -14488,7 +15797,7 @@ func (p *nodeProcessorBootstrappedInPlacementOrNoPlacement) Process(seqId int32, } else { result.Success = retval } - if err2 = oprot.WriteMessageBegin("bootstrappedInPlacementOrNoPlacement", thrift.REPLY, seqId); err2 != nil { + if err2 = oprot.WriteMessageBegin("getWriteNewSeriesBackoffDuration", thrift.REPLY, seqId); err2 != nil { err = err2 } if err2 = result.Write(oprot); err == nil && err2 != nil { @@ -14506,16 +15815,16 @@ func (p *nodeProcessorBootstrappedInPlacementOrNoPlacement) Process(seqId int32, return true, err } -type nodeProcessorGetPersistRateLimit struct { +type nodeProcessorSetWriteNewSeriesBackoffDuration struct { handler Node } -func (p *nodeProcessorGetPersistRateLimit) Process(seqId int32, iprot, oprot thrift.TProtocol) (success bool, err thrift.TException) { - args := NodeGetPersistRateLimitArgs{} +func (p *nodeProcessorSetWriteNewSeriesBackoffDuration) Process(seqId int32, iprot, oprot thrift.TProtocol) (success bool, err thrift.TException) { + args := NodeSetWriteNewSeriesBackoffDurationArgs{} if err = args.Read(iprot); err != nil { iprot.ReadMessageEnd() x := thrift.NewTApplicationException(thrift.PROTOCOL_ERROR, err.Error()) - oprot.WriteMessageBegin("getPersistRateLimit", thrift.EXCEPTION, seqId) + oprot.WriteMessageBegin("setWriteNewSeriesBackoffDuration", thrift.EXCEPTION, seqId) x.Write(oprot) oprot.WriteMessageEnd() oprot.Flush() @@ -14523,16 +15832,16 @@ func (p *nodeProcessorGetPersistRateLimit) Process(seqId int32, iprot, oprot thr } iprot.ReadMessageEnd() - result := NodeGetPersistRateLimitResult{} - var retval *NodePersistRateLimitResult_ + result := NodeSetWriteNewSeriesBackoffDurationResult{} + var retval *NodeWriteNewSeriesBackoffDurationResult_ var err2 error - if retval, err2 = p.handler.GetPersistRateLimit(); err2 != nil { + if retval, err2 = p.handler.SetWriteNewSeriesBackoffDuration(args.Req); err2 != nil { switch v := err2.(type) { case *Error: result.Err = v default: - x := thrift.NewTApplicationException(thrift.INTERNAL_ERROR, "Internal error processing getPersistRateLimit: "+err2.Error()) - oprot.WriteMessageBegin("getPersistRateLimit", thrift.EXCEPTION, seqId) + x := thrift.NewTApplicationException(thrift.INTERNAL_ERROR, "Internal error processing setWriteNewSeriesBackoffDuration: "+err2.Error()) + oprot.WriteMessageBegin("setWriteNewSeriesBackoffDuration", thrift.EXCEPTION, seqId) x.Write(oprot) oprot.WriteMessageEnd() oprot.Flush() @@ -14541,7 +15850,7 @@ func (p *nodeProcessorGetPersistRateLimit) Process(seqId int32, iprot, oprot thr } else { result.Success = retval } - if err2 = oprot.WriteMessageBegin("getPersistRateLimit", thrift.REPLY, seqId); err2 != nil { + if err2 = oprot.WriteMessageBegin("setWriteNewSeriesBackoffDuration", thrift.REPLY, seqId); err2 != nil { err = err2 } if err2 = result.Write(oprot); err == nil && err2 != nil { @@ -14559,16 +15868,16 @@ func (p *nodeProcessorGetPersistRateLimit) Process(seqId int32, iprot, oprot thr return true, err } -type nodeProcessorSetPersistRateLimit struct { +type nodeProcessorGetWriteNewSeriesLimitPerShardPerSecond struct { handler Node } -func (p *nodeProcessorSetPersistRateLimit) Process(seqId int32, iprot, oprot thrift.TProtocol) (success bool, err thrift.TException) { - args := NodeSetPersistRateLimitArgs{} +func (p *nodeProcessorGetWriteNewSeriesLimitPerShardPerSecond) Process(seqId int32, iprot, oprot thrift.TProtocol) (success bool, err thrift.TException) { + args := NodeGetWriteNewSeriesLimitPerShardPerSecondArgs{} if err = args.Read(iprot); err != nil { iprot.ReadMessageEnd() x := thrift.NewTApplicationException(thrift.PROTOCOL_ERROR, err.Error()) - oprot.WriteMessageBegin("setPersistRateLimit", thrift.EXCEPTION, seqId) + oprot.WriteMessageBegin("getWriteNewSeriesLimitPerShardPerSecond", thrift.EXCEPTION, seqId) x.Write(oprot) oprot.WriteMessageEnd() oprot.Flush() @@ -14576,16 +15885,16 @@ func (p *nodeProcessorSetPersistRateLimit) Process(seqId int32, iprot, oprot thr } iprot.ReadMessageEnd() - result := NodeSetPersistRateLimitResult{} - var retval *NodePersistRateLimitResult_ + result := NodeGetWriteNewSeriesLimitPerShardPerSecondResult{} + var retval *NodeWriteNewSeriesLimitPerShardPerSecondResult_ var err2 error - if retval, err2 = p.handler.SetPersistRateLimit(args.Req); err2 != nil { + if retval, err2 = p.handler.GetWriteNewSeriesLimitPerShardPerSecond(); err2 != nil { switch v := err2.(type) { case *Error: result.Err = v default: - x := thrift.NewTApplicationException(thrift.INTERNAL_ERROR, "Internal error processing setPersistRateLimit: "+err2.Error()) - oprot.WriteMessageBegin("setPersistRateLimit", thrift.EXCEPTION, seqId) + x := thrift.NewTApplicationException(thrift.INTERNAL_ERROR, "Internal error processing getWriteNewSeriesLimitPerShardPerSecond: "+err2.Error()) + oprot.WriteMessageBegin("getWriteNewSeriesLimitPerShardPerSecond", thrift.EXCEPTION, seqId) x.Write(oprot) oprot.WriteMessageEnd() oprot.Flush() @@ -14594,7 +15903,7 @@ func (p *nodeProcessorSetPersistRateLimit) Process(seqId int32, iprot, oprot thr } else { result.Success = retval } - if err2 = oprot.WriteMessageBegin("setPersistRateLimit", thrift.REPLY, seqId); err2 != nil { + if err2 = oprot.WriteMessageBegin("getWriteNewSeriesLimitPerShardPerSecond", thrift.REPLY, seqId); err2 != nil { err = err2 } if err2 = result.Write(oprot); err == nil && err2 != nil { @@ -14612,16 +15921,16 @@ func (p *nodeProcessorSetPersistRateLimit) Process(seqId int32, iprot, oprot thr return true, err } -type nodeProcessorGetWriteNewSeriesAsync struct { +type nodeProcessorSetWriteNewSeriesLimitPerShardPerSecond struct { handler Node } -func (p *nodeProcessorGetWriteNewSeriesAsync) Process(seqId int32, iprot, oprot thrift.TProtocol) (success bool, err thrift.TException) { - args := NodeGetWriteNewSeriesAsyncArgs{} +func (p *nodeProcessorSetWriteNewSeriesLimitPerShardPerSecond) Process(seqId int32, iprot, oprot thrift.TProtocol) (success bool, err thrift.TException) { + args := NodeSetWriteNewSeriesLimitPerShardPerSecondArgs{} if err = args.Read(iprot); err != nil { iprot.ReadMessageEnd() x := thrift.NewTApplicationException(thrift.PROTOCOL_ERROR, err.Error()) - oprot.WriteMessageBegin("getWriteNewSeriesAsync", thrift.EXCEPTION, seqId) + oprot.WriteMessageBegin("setWriteNewSeriesLimitPerShardPerSecond", thrift.EXCEPTION, seqId) x.Write(oprot) oprot.WriteMessageEnd() oprot.Flush() @@ -14629,16 +15938,16 @@ func (p *nodeProcessorGetWriteNewSeriesAsync) Process(seqId int32, iprot, oprot } iprot.ReadMessageEnd() - result := NodeGetWriteNewSeriesAsyncResult{} - var retval *NodeWriteNewSeriesAsyncResult_ + result := NodeSetWriteNewSeriesLimitPerShardPerSecondResult{} + var retval *NodeWriteNewSeriesLimitPerShardPerSecondResult_ var err2 error - if retval, err2 = p.handler.GetWriteNewSeriesAsync(); err2 != nil { + if retval, err2 = p.handler.SetWriteNewSeriesLimitPerShardPerSecond(args.Req); err2 != nil { switch v := err2.(type) { case *Error: result.Err = v default: - x := thrift.NewTApplicationException(thrift.INTERNAL_ERROR, "Internal error processing getWriteNewSeriesAsync: "+err2.Error()) - oprot.WriteMessageBegin("getWriteNewSeriesAsync", thrift.EXCEPTION, seqId) + x := thrift.NewTApplicationException(thrift.INTERNAL_ERROR, "Internal error processing setWriteNewSeriesLimitPerShardPerSecond: "+err2.Error()) + oprot.WriteMessageBegin("setWriteNewSeriesLimitPerShardPerSecond", thrift.EXCEPTION, seqId) x.Write(oprot) oprot.WriteMessageEnd() oprot.Flush() @@ -14647,7 +15956,7 @@ func (p *nodeProcessorGetWriteNewSeriesAsync) Process(seqId int32, iprot, oprot } else { result.Success = retval } - if err2 = oprot.WriteMessageBegin("getWriteNewSeriesAsync", thrift.REPLY, seqId); err2 != nil { + if err2 = oprot.WriteMessageBegin("setWriteNewSeriesLimitPerShardPerSecond", thrift.REPLY, seqId); err2 != nil { err = err2 } if err2 = result.Write(oprot); err == nil && err2 != nil { @@ -14659,302 +15968,547 @@ func (p *nodeProcessorGetWriteNewSeriesAsync) Process(seqId int32, iprot, oprot if err2 = oprot.Flush(); err == nil && err2 != nil { err = err2 } - if err != nil { - return + if err != nil { + return + } + return true, err +} + +// HELPER FUNCTIONS AND STRUCTURES + +// Attributes: +// - Req +type NodeQueryArgs struct { + Req *QueryRequest `thrift:"req,1" db:"req" json:"req"` +} + +func NewNodeQueryArgs() *NodeQueryArgs { + return &NodeQueryArgs{} +} + +var NodeQueryArgs_Req_DEFAULT *QueryRequest + +func (p *NodeQueryArgs) GetReq() *QueryRequest { + if !p.IsSetReq() { + return NodeQueryArgs_Req_DEFAULT + } + return p.Req +} +func (p *NodeQueryArgs) IsSetReq() bool { + return p.Req != nil +} + +func (p *NodeQueryArgs) Read(iprot thrift.TProtocol) error { + if _, err := iprot.ReadStructBegin(); err != nil { + return thrift.PrependError(fmt.Sprintf("%T read error: ", p), err) + } + + for { + _, fieldTypeId, fieldId, err := iprot.ReadFieldBegin() + if err != nil { + return thrift.PrependError(fmt.Sprintf("%T field %d read error: ", p, fieldId), err) + } + if fieldTypeId == thrift.STOP { + break + } + switch fieldId { + case 1: + if err := p.ReadField1(iprot); err != nil { + return err + } + default: + if err := iprot.Skip(fieldTypeId); err != nil { + return err + } + } + if err := iprot.ReadFieldEnd(); err != nil { + return err + } + } + if err := iprot.ReadStructEnd(); err != nil { + return thrift.PrependError(fmt.Sprintf("%T read struct end error: ", p), err) + } + return nil +} + +func (p *NodeQueryArgs) ReadField1(iprot thrift.TProtocol) error { + p.Req = &QueryRequest{ + RangeType: 0, + + ResultTimeType: 0, + } + if err := p.Req.Read(iprot); err != nil { + return thrift.PrependError(fmt.Sprintf("%T error reading struct: ", p.Req), err) + } + return nil +} + +func (p *NodeQueryArgs) Write(oprot thrift.TProtocol) error { + if err := oprot.WriteStructBegin("query_args"); err != nil { + return thrift.PrependError(fmt.Sprintf("%T write struct begin error: ", p), err) + } + if p != nil { + if err := p.writeField1(oprot); err != nil { + return err + } + } + if err := oprot.WriteFieldStop(); err != nil { + return thrift.PrependError("write field stop error: ", err) + } + if err := oprot.WriteStructEnd(); err != nil { + return thrift.PrependError("write struct stop error: ", err) + } + return nil +} + +func (p *NodeQueryArgs) writeField1(oprot thrift.TProtocol) (err error) { + if err := oprot.WriteFieldBegin("req", thrift.STRUCT, 1); err != nil { + return thrift.PrependError(fmt.Sprintf("%T write field begin error 1:req: ", p), err) + } + if err := p.Req.Write(oprot); err != nil { + return thrift.PrependError(fmt.Sprintf("%T error writing struct: ", p.Req), err) + } + if err := oprot.WriteFieldEnd(); err != nil { + return thrift.PrependError(fmt.Sprintf("%T write field end error 1:req: ", p), err) + } + return err +} + +func (p *NodeQueryArgs) String() string { + if p == nil { + return "" + } + return fmt.Sprintf("NodeQueryArgs(%+v)", *p) +} + +// Attributes: +// - Success +// - Err +type NodeQueryResult struct { + Success *QueryResult_ `thrift:"success,0" db:"success" json:"success,omitempty"` + Err *Error `thrift:"err,1" db:"err" json:"err,omitempty"` +} + +func NewNodeQueryResult() *NodeQueryResult { + return &NodeQueryResult{} +} + +var NodeQueryResult_Success_DEFAULT *QueryResult_ + +func (p *NodeQueryResult) GetSuccess() *QueryResult_ { + if !p.IsSetSuccess() { + return NodeQueryResult_Success_DEFAULT + } + return p.Success +} + +var NodeQueryResult_Err_DEFAULT *Error + +func (p *NodeQueryResult) GetErr() *Error { + if !p.IsSetErr() { + return NodeQueryResult_Err_DEFAULT + } + return p.Err +} +func (p *NodeQueryResult) IsSetSuccess() bool { + return p.Success != nil +} + +func (p *NodeQueryResult) IsSetErr() bool { + return p.Err != nil +} + +func (p *NodeQueryResult) Read(iprot thrift.TProtocol) error { + if _, err := iprot.ReadStructBegin(); err != nil { + return thrift.PrependError(fmt.Sprintf("%T read error: ", p), err) + } + + for { + _, fieldTypeId, fieldId, err := iprot.ReadFieldBegin() + if err != nil { + return thrift.PrependError(fmt.Sprintf("%T field %d read error: ", p, fieldId), err) + } + if fieldTypeId == thrift.STOP { + break + } + switch fieldId { + case 0: + if err := p.ReadField0(iprot); err != nil { + return err + } + case 1: + if err := p.ReadField1(iprot); err != nil { + return err + } + default: + if err := iprot.Skip(fieldTypeId); err != nil { + return err + } + } + if err := iprot.ReadFieldEnd(); err != nil { + return err + } + } + if err := iprot.ReadStructEnd(); err != nil { + return thrift.PrependError(fmt.Sprintf("%T read struct end error: ", p), err) } - return true, err + return nil } -type nodeProcessorSetWriteNewSeriesAsync struct { - handler Node +func (p *NodeQueryResult) ReadField0(iprot thrift.TProtocol) error { + p.Success = &QueryResult_{} + if err := p.Success.Read(iprot); err != nil { + return thrift.PrependError(fmt.Sprintf("%T error reading struct: ", p.Success), err) + } + return nil } -func (p *nodeProcessorSetWriteNewSeriesAsync) Process(seqId int32, iprot, oprot thrift.TProtocol) (success bool, err thrift.TException) { - args := NodeSetWriteNewSeriesAsyncArgs{} - if err = args.Read(iprot); err != nil { - iprot.ReadMessageEnd() - x := thrift.NewTApplicationException(thrift.PROTOCOL_ERROR, err.Error()) - oprot.WriteMessageBegin("setWriteNewSeriesAsync", thrift.EXCEPTION, seqId) - x.Write(oprot) - oprot.WriteMessageEnd() - oprot.Flush() - return false, err +func (p *NodeQueryResult) ReadField1(iprot thrift.TProtocol) error { + p.Err = &Error{ + Type: 0, + } + if err := p.Err.Read(iprot); err != nil { + return thrift.PrependError(fmt.Sprintf("%T error reading struct: ", p.Err), err) } + return nil +} - iprot.ReadMessageEnd() - result := NodeSetWriteNewSeriesAsyncResult{} - var retval *NodeWriteNewSeriesAsyncResult_ - var err2 error - if retval, err2 = p.handler.SetWriteNewSeriesAsync(args.Req); err2 != nil { - switch v := err2.(type) { - case *Error: - result.Err = v - default: - x := thrift.NewTApplicationException(thrift.INTERNAL_ERROR, "Internal error processing setWriteNewSeriesAsync: "+err2.Error()) - oprot.WriteMessageBegin("setWriteNewSeriesAsync", thrift.EXCEPTION, seqId) - x.Write(oprot) - oprot.WriteMessageEnd() - oprot.Flush() - return true, err2 +func (p *NodeQueryResult) Write(oprot thrift.TProtocol) error { + if err := oprot.WriteStructBegin("query_result"); err != nil { + return thrift.PrependError(fmt.Sprintf("%T write struct begin error: ", p), err) + } + if p != nil { + if err := p.writeField0(oprot); err != nil { + return err + } + if err := p.writeField1(oprot); err != nil { + return err } - } else { - result.Success = retval } - if err2 = oprot.WriteMessageBegin("setWriteNewSeriesAsync", thrift.REPLY, seqId); err2 != nil { - err = err2 + if err := oprot.WriteFieldStop(); err != nil { + return thrift.PrependError("write field stop error: ", err) } - if err2 = result.Write(oprot); err == nil && err2 != nil { - err = err2 + if err := oprot.WriteStructEnd(); err != nil { + return thrift.PrependError("write struct stop error: ", err) } - if err2 = oprot.WriteMessageEnd(); err == nil && err2 != nil { - err = err2 + return nil +} + +func (p *NodeQueryResult) writeField0(oprot thrift.TProtocol) (err error) { + if p.IsSetSuccess() { + if err := oprot.WriteFieldBegin("success", thrift.STRUCT, 0); err != nil { + return thrift.PrependError(fmt.Sprintf("%T write field begin error 0:success: ", p), err) + } + if err := p.Success.Write(oprot); err != nil { + return thrift.PrependError(fmt.Sprintf("%T error writing struct: ", p.Success), err) + } + if err := oprot.WriteFieldEnd(); err != nil { + return thrift.PrependError(fmt.Sprintf("%T write field end error 0:success: ", p), err) + } } - if err2 = oprot.Flush(); err == nil && err2 != nil { - err = err2 + return err +} + +func (p *NodeQueryResult) writeField1(oprot thrift.TProtocol) (err error) { + if p.IsSetErr() { + if err := oprot.WriteFieldBegin("err", thrift.STRUCT, 1); err != nil { + return thrift.PrependError(fmt.Sprintf("%T write field begin error 1:err: ", p), err) + } + if err := p.Err.Write(oprot); err != nil { + return thrift.PrependError(fmt.Sprintf("%T error writing struct: ", p.Err), err) + } + if err := oprot.WriteFieldEnd(); err != nil { + return thrift.PrependError(fmt.Sprintf("%T write field end error 1:err: ", p), err) + } } - if err != nil { - return + return err +} + +func (p *NodeQueryResult) String() string { + if p == nil { + return "" } - return true, err + return fmt.Sprintf("NodeQueryResult(%+v)", *p) } -type nodeProcessorGetWriteNewSeriesBackoffDuration struct { - handler Node +// Attributes: +// - Req +type NodeAggregateRawArgs struct { + Req *AggregateQueryRawRequest `thrift:"req,1" db:"req" json:"req"` } -func (p *nodeProcessorGetWriteNewSeriesBackoffDuration) Process(seqId int32, iprot, oprot thrift.TProtocol) (success bool, err thrift.TException) { - args := NodeGetWriteNewSeriesBackoffDurationArgs{} - if err = args.Read(iprot); err != nil { - iprot.ReadMessageEnd() - x := thrift.NewTApplicationException(thrift.PROTOCOL_ERROR, err.Error()) - oprot.WriteMessageBegin("getWriteNewSeriesBackoffDuration", thrift.EXCEPTION, seqId) - x.Write(oprot) - oprot.WriteMessageEnd() - oprot.Flush() - return false, err +func NewNodeAggregateRawArgs() *NodeAggregateRawArgs { + return &NodeAggregateRawArgs{} +} + +var NodeAggregateRawArgs_Req_DEFAULT *AggregateQueryRawRequest + +func (p *NodeAggregateRawArgs) GetReq() *AggregateQueryRawRequest { + if !p.IsSetReq() { + return NodeAggregateRawArgs_Req_DEFAULT } + return p.Req +} +func (p *NodeAggregateRawArgs) IsSetReq() bool { + return p.Req != nil +} - iprot.ReadMessageEnd() - result := NodeGetWriteNewSeriesBackoffDurationResult{} - var retval *NodeWriteNewSeriesBackoffDurationResult_ - var err2 error - if retval, err2 = p.handler.GetWriteNewSeriesBackoffDuration(); err2 != nil { - switch v := err2.(type) { - case *Error: - result.Err = v +func (p *NodeAggregateRawArgs) Read(iprot thrift.TProtocol) error { + if _, err := iprot.ReadStructBegin(); err != nil { + return thrift.PrependError(fmt.Sprintf("%T read error: ", p), err) + } + + for { + _, fieldTypeId, fieldId, err := iprot.ReadFieldBegin() + if err != nil { + return thrift.PrependError(fmt.Sprintf("%T field %d read error: ", p, fieldId), err) + } + if fieldTypeId == thrift.STOP { + break + } + switch fieldId { + case 1: + if err := p.ReadField1(iprot); err != nil { + return err + } default: - x := thrift.NewTApplicationException(thrift.INTERNAL_ERROR, "Internal error processing getWriteNewSeriesBackoffDuration: "+err2.Error()) - oprot.WriteMessageBegin("getWriteNewSeriesBackoffDuration", thrift.EXCEPTION, seqId) - x.Write(oprot) - oprot.WriteMessageEnd() - oprot.Flush() - return true, err2 + if err := iprot.Skip(fieldTypeId); err != nil { + return err + } + } + if err := iprot.ReadFieldEnd(); err != nil { + return err } - } else { - result.Success = retval - } - if err2 = oprot.WriteMessageBegin("getWriteNewSeriesBackoffDuration", thrift.REPLY, seqId); err2 != nil { - err = err2 - } - if err2 = result.Write(oprot); err == nil && err2 != nil { - err = err2 - } - if err2 = oprot.WriteMessageEnd(); err == nil && err2 != nil { - err = err2 - } - if err2 = oprot.Flush(); err == nil && err2 != nil { - err = err2 } - if err != nil { - return + if err := iprot.ReadStructEnd(); err != nil { + return thrift.PrependError(fmt.Sprintf("%T read struct end error: ", p), err) } - return true, err + return nil } -type nodeProcessorSetWriteNewSeriesBackoffDuration struct { - handler Node -} +func (p *NodeAggregateRawArgs) ReadField1(iprot thrift.TProtocol) error { + p.Req = &AggregateQueryRawRequest{ + AggregateQueryType: 1, -func (p *nodeProcessorSetWriteNewSeriesBackoffDuration) Process(seqId int32, iprot, oprot thrift.TProtocol) (success bool, err thrift.TException) { - args := NodeSetWriteNewSeriesBackoffDurationArgs{} - if err = args.Read(iprot); err != nil { - iprot.ReadMessageEnd() - x := thrift.NewTApplicationException(thrift.PROTOCOL_ERROR, err.Error()) - oprot.WriteMessageBegin("setWriteNewSeriesBackoffDuration", thrift.EXCEPTION, seqId) - x.Write(oprot) - oprot.WriteMessageEnd() - oprot.Flush() - return false, err + RangeType: 0, + } + if err := p.Req.Read(iprot); err != nil { + return thrift.PrependError(fmt.Sprintf("%T error reading struct: ", p.Req), err) } + return nil +} - iprot.ReadMessageEnd() - result := NodeSetWriteNewSeriesBackoffDurationResult{} - var retval *NodeWriteNewSeriesBackoffDurationResult_ - var err2 error - if retval, err2 = p.handler.SetWriteNewSeriesBackoffDuration(args.Req); err2 != nil { - switch v := err2.(type) { - case *Error: - result.Err = v - default: - x := thrift.NewTApplicationException(thrift.INTERNAL_ERROR, "Internal error processing setWriteNewSeriesBackoffDuration: "+err2.Error()) - oprot.WriteMessageBegin("setWriteNewSeriesBackoffDuration", thrift.EXCEPTION, seqId) - x.Write(oprot) - oprot.WriteMessageEnd() - oprot.Flush() - return true, err2 +func (p *NodeAggregateRawArgs) Write(oprot thrift.TProtocol) error { + if err := oprot.WriteStructBegin("aggregateRaw_args"); err != nil { + return thrift.PrependError(fmt.Sprintf("%T write struct begin error: ", p), err) + } + if p != nil { + if err := p.writeField1(oprot); err != nil { + return err } - } else { - result.Success = retval } - if err2 = oprot.WriteMessageBegin("setWriteNewSeriesBackoffDuration", thrift.REPLY, seqId); err2 != nil { - err = err2 + if err := oprot.WriteFieldStop(); err != nil { + return thrift.PrependError("write field stop error: ", err) + } + if err := oprot.WriteStructEnd(); err != nil { + return thrift.PrependError("write struct stop error: ", err) + } + return nil +} + +func (p *NodeAggregateRawArgs) writeField1(oprot thrift.TProtocol) (err error) { + if err := oprot.WriteFieldBegin("req", thrift.STRUCT, 1); err != nil { + return thrift.PrependError(fmt.Sprintf("%T write field begin error 1:req: ", p), err) + } + if err := p.Req.Write(oprot); err != nil { + return thrift.PrependError(fmt.Sprintf("%T error writing struct: ", p.Req), err) } - if err2 = result.Write(oprot); err == nil && err2 != nil { - err = err2 + if err := oprot.WriteFieldEnd(); err != nil { + return thrift.PrependError(fmt.Sprintf("%T write field end error 1:req: ", p), err) } - if err2 = oprot.WriteMessageEnd(); err == nil && err2 != nil { - err = err2 + return err +} + +func (p *NodeAggregateRawArgs) String() string { + if p == nil { + return "" } - if err2 = oprot.Flush(); err == nil && err2 != nil { - err = err2 + return fmt.Sprintf("NodeAggregateRawArgs(%+v)", *p) +} + +// Attributes: +// - Success +// - Err +type NodeAggregateRawResult struct { + Success *AggregateQueryRawResult_ `thrift:"success,0" db:"success" json:"success,omitempty"` + Err *Error `thrift:"err,1" db:"err" json:"err,omitempty"` +} + +func NewNodeAggregateRawResult() *NodeAggregateRawResult { + return &NodeAggregateRawResult{} +} + +var NodeAggregateRawResult_Success_DEFAULT *AggregateQueryRawResult_ + +func (p *NodeAggregateRawResult) GetSuccess() *AggregateQueryRawResult_ { + if !p.IsSetSuccess() { + return NodeAggregateRawResult_Success_DEFAULT } - if err != nil { - return + return p.Success +} + +var NodeAggregateRawResult_Err_DEFAULT *Error + +func (p *NodeAggregateRawResult) GetErr() *Error { + if !p.IsSetErr() { + return NodeAggregateRawResult_Err_DEFAULT } - return true, err + return p.Err +} +func (p *NodeAggregateRawResult) IsSetSuccess() bool { + return p.Success != nil } -type nodeProcessorGetWriteNewSeriesLimitPerShardPerSecond struct { - handler Node +func (p *NodeAggregateRawResult) IsSetErr() bool { + return p.Err != nil } -func (p *nodeProcessorGetWriteNewSeriesLimitPerShardPerSecond) Process(seqId int32, iprot, oprot thrift.TProtocol) (success bool, err thrift.TException) { - args := NodeGetWriteNewSeriesLimitPerShardPerSecondArgs{} - if err = args.Read(iprot); err != nil { - iprot.ReadMessageEnd() - x := thrift.NewTApplicationException(thrift.PROTOCOL_ERROR, err.Error()) - oprot.WriteMessageBegin("getWriteNewSeriesLimitPerShardPerSecond", thrift.EXCEPTION, seqId) - x.Write(oprot) - oprot.WriteMessageEnd() - oprot.Flush() - return false, err +func (p *NodeAggregateRawResult) Read(iprot thrift.TProtocol) error { + if _, err := iprot.ReadStructBegin(); err != nil { + return thrift.PrependError(fmt.Sprintf("%T read error: ", p), err) } - iprot.ReadMessageEnd() - result := NodeGetWriteNewSeriesLimitPerShardPerSecondResult{} - var retval *NodeWriteNewSeriesLimitPerShardPerSecondResult_ - var err2 error - if retval, err2 = p.handler.GetWriteNewSeriesLimitPerShardPerSecond(); err2 != nil { - switch v := err2.(type) { - case *Error: - result.Err = v + for { + _, fieldTypeId, fieldId, err := iprot.ReadFieldBegin() + if err != nil { + return thrift.PrependError(fmt.Sprintf("%T field %d read error: ", p, fieldId), err) + } + if fieldTypeId == thrift.STOP { + break + } + switch fieldId { + case 0: + if err := p.ReadField0(iprot); err != nil { + return err + } + case 1: + if err := p.ReadField1(iprot); err != nil { + return err + } default: - x := thrift.NewTApplicationException(thrift.INTERNAL_ERROR, "Internal error processing getWriteNewSeriesLimitPerShardPerSecond: "+err2.Error()) - oprot.WriteMessageBegin("getWriteNewSeriesLimitPerShardPerSecond", thrift.EXCEPTION, seqId) - x.Write(oprot) - oprot.WriteMessageEnd() - oprot.Flush() - return true, err2 + if err := iprot.Skip(fieldTypeId); err != nil { + return err + } + } + if err := iprot.ReadFieldEnd(); err != nil { + return err } - } else { - result.Success = retval - } - if err2 = oprot.WriteMessageBegin("getWriteNewSeriesLimitPerShardPerSecond", thrift.REPLY, seqId); err2 != nil { - err = err2 - } - if err2 = result.Write(oprot); err == nil && err2 != nil { - err = err2 - } - if err2 = oprot.WriteMessageEnd(); err == nil && err2 != nil { - err = err2 - } - if err2 = oprot.Flush(); err == nil && err2 != nil { - err = err2 } - if err != nil { - return + if err := iprot.ReadStructEnd(); err != nil { + return thrift.PrependError(fmt.Sprintf("%T read struct end error: ", p), err) } - return true, err + return nil } -type nodeProcessorSetWriteNewSeriesLimitPerShardPerSecond struct { - handler Node +func (p *NodeAggregateRawResult) ReadField0(iprot thrift.TProtocol) error { + p.Success = &AggregateQueryRawResult_{} + if err := p.Success.Read(iprot); err != nil { + return thrift.PrependError(fmt.Sprintf("%T error reading struct: ", p.Success), err) + } + return nil } -func (p *nodeProcessorSetWriteNewSeriesLimitPerShardPerSecond) Process(seqId int32, iprot, oprot thrift.TProtocol) (success bool, err thrift.TException) { - args := NodeSetWriteNewSeriesLimitPerShardPerSecondArgs{} - if err = args.Read(iprot); err != nil { - iprot.ReadMessageEnd() - x := thrift.NewTApplicationException(thrift.PROTOCOL_ERROR, err.Error()) - oprot.WriteMessageBegin("setWriteNewSeriesLimitPerShardPerSecond", thrift.EXCEPTION, seqId) - x.Write(oprot) - oprot.WriteMessageEnd() - oprot.Flush() - return false, err +func (p *NodeAggregateRawResult) ReadField1(iprot thrift.TProtocol) error { + p.Err = &Error{ + Type: 0, } + if err := p.Err.Read(iprot); err != nil { + return thrift.PrependError(fmt.Sprintf("%T error reading struct: ", p.Err), err) + } + return nil +} - iprot.ReadMessageEnd() - result := NodeSetWriteNewSeriesLimitPerShardPerSecondResult{} - var retval *NodeWriteNewSeriesLimitPerShardPerSecondResult_ - var err2 error - if retval, err2 = p.handler.SetWriteNewSeriesLimitPerShardPerSecond(args.Req); err2 != nil { - switch v := err2.(type) { - case *Error: - result.Err = v - default: - x := thrift.NewTApplicationException(thrift.INTERNAL_ERROR, "Internal error processing setWriteNewSeriesLimitPerShardPerSecond: "+err2.Error()) - oprot.WriteMessageBegin("setWriteNewSeriesLimitPerShardPerSecond", thrift.EXCEPTION, seqId) - x.Write(oprot) - oprot.WriteMessageEnd() - oprot.Flush() - return true, err2 - } - } else { - result.Success = retval +func (p *NodeAggregateRawResult) Write(oprot thrift.TProtocol) error { + if err := oprot.WriteStructBegin("aggregateRaw_result"); err != nil { + return thrift.PrependError(fmt.Sprintf("%T write struct begin error: ", p), err) } - if err2 = oprot.WriteMessageBegin("setWriteNewSeriesLimitPerShardPerSecond", thrift.REPLY, seqId); err2 != nil { - err = err2 + if p != nil { + if err := p.writeField0(oprot); err != nil { + return err + } + if err := p.writeField1(oprot); err != nil { + return err + } } - if err2 = result.Write(oprot); err == nil && err2 != nil { - err = err2 + if err := oprot.WriteFieldStop(); err != nil { + return thrift.PrependError("write field stop error: ", err) } - if err2 = oprot.WriteMessageEnd(); err == nil && err2 != nil { - err = err2 + if err := oprot.WriteStructEnd(); err != nil { + return thrift.PrependError("write struct stop error: ", err) } - if err2 = oprot.Flush(); err == nil && err2 != nil { - err = err2 + return nil +} + +func (p *NodeAggregateRawResult) writeField0(oprot thrift.TProtocol) (err error) { + if p.IsSetSuccess() { + if err := oprot.WriteFieldBegin("success", thrift.STRUCT, 0); err != nil { + return thrift.PrependError(fmt.Sprintf("%T write field begin error 0:success: ", p), err) + } + if err := p.Success.Write(oprot); err != nil { + return thrift.PrependError(fmt.Sprintf("%T error writing struct: ", p.Success), err) + } + if err := oprot.WriteFieldEnd(); err != nil { + return thrift.PrependError(fmt.Sprintf("%T write field end error 0:success: ", p), err) + } } - if err != nil { - return + return err +} + +func (p *NodeAggregateRawResult) writeField1(oprot thrift.TProtocol) (err error) { + if p.IsSetErr() { + if err := oprot.WriteFieldBegin("err", thrift.STRUCT, 1); err != nil { + return thrift.PrependError(fmt.Sprintf("%T write field begin error 1:err: ", p), err) + } + if err := p.Err.Write(oprot); err != nil { + return thrift.PrependError(fmt.Sprintf("%T error writing struct: ", p.Err), err) + } + if err := oprot.WriteFieldEnd(); err != nil { + return thrift.PrependError(fmt.Sprintf("%T write field end error 1:err: ", p), err) + } } - return true, err + return err } -// HELPER FUNCTIONS AND STRUCTURES +func (p *NodeAggregateRawResult) String() string { + if p == nil { + return "" + } + return fmt.Sprintf("NodeAggregateRawResult(%+v)", *p) +} // Attributes: // - Req -type NodeQueryArgs struct { - Req *QueryRequest `thrift:"req,1" db:"req" json:"req"` +type NodeAggregateArgs struct { + Req *AggregateQueryRequest `thrift:"req,1" db:"req" json:"req"` } -func NewNodeQueryArgs() *NodeQueryArgs { - return &NodeQueryArgs{} +func NewNodeAggregateArgs() *NodeAggregateArgs { + return &NodeAggregateArgs{} } -var NodeQueryArgs_Req_DEFAULT *QueryRequest +var NodeAggregateArgs_Req_DEFAULT *AggregateQueryRequest -func (p *NodeQueryArgs) GetReq() *QueryRequest { +func (p *NodeAggregateArgs) GetReq() *AggregateQueryRequest { if !p.IsSetReq() { - return NodeQueryArgs_Req_DEFAULT + return NodeAggregateArgs_Req_DEFAULT } return p.Req } -func (p *NodeQueryArgs) IsSetReq() bool { +func (p *NodeAggregateArgs) IsSetReq() bool { return p.Req != nil } -func (p *NodeQueryArgs) Read(iprot thrift.TProtocol) error { +func (p *NodeAggregateArgs) Read(iprot thrift.TProtocol) error { if _, err := iprot.ReadStructBegin(); err != nil { return thrift.PrependError(fmt.Sprintf("%T read error: ", p), err) } @@ -14987,11 +16541,11 @@ func (p *NodeQueryArgs) Read(iprot thrift.TProtocol) error { return nil } -func (p *NodeQueryArgs) ReadField1(iprot thrift.TProtocol) error { - p.Req = &QueryRequest{ +func (p *NodeAggregateArgs) ReadField1(iprot thrift.TProtocol) error { + p.Req = &AggregateQueryRequest{ + AggregateQueryType: 1, + RangeType: 0, - - ResultTimeType: 0, } if err := p.Req.Read(iprot); err != nil { return thrift.PrependError(fmt.Sprintf("%T error reading struct: ", p.Req), err) @@ -14999,8 +16553,8 @@ func (p *NodeQueryArgs) ReadField1(iprot thrift.TProtocol) error { return nil } -func (p *NodeQueryArgs) Write(oprot thrift.TProtocol) error { - if err := oprot.WriteStructBegin("query_args"); err != nil { +func (p *NodeAggregateArgs) Write(oprot thrift.TProtocol) error { + if err := oprot.WriteStructBegin("aggregate_args"); err != nil { return thrift.PrependError(fmt.Sprintf("%T write struct begin error: ", p), err) } if p != nil { @@ -15017,7 +16571,7 @@ func (p *NodeQueryArgs) Write(oprot thrift.TProtocol) error { return nil } -func (p *NodeQueryArgs) writeField1(oprot thrift.TProtocol) (err error) { +func (p *NodeAggregateArgs) writeField1(oprot thrift.TProtocol) (err error) { if err := oprot.WriteFieldBegin("req", thrift.STRUCT, 1); err != nil { return thrift.PrependError(fmt.Sprintf("%T write field begin error 1:req: ", p), err) } @@ -15030,51 +16584,51 @@ func (p *NodeQueryArgs) writeField1(oprot thrift.TProtocol) (err error) { return err } -func (p *NodeQueryArgs) String() string { +func (p *NodeAggregateArgs) String() string { if p == nil { return "" } - return fmt.Sprintf("NodeQueryArgs(%+v)", *p) + return fmt.Sprintf("NodeAggregateArgs(%+v)", *p) } // Attributes: // - Success // - Err -type NodeQueryResult struct { - Success *QueryResult_ `thrift:"success,0" db:"success" json:"success,omitempty"` - Err *Error `thrift:"err,1" db:"err" json:"err,omitempty"` +type NodeAggregateResult struct { + Success *AggregateQueryResult_ `thrift:"success,0" db:"success" json:"success,omitempty"` + Err *Error `thrift:"err,1" db:"err" json:"err,omitempty"` } -func NewNodeQueryResult() *NodeQueryResult { - return &NodeQueryResult{} +func NewNodeAggregateResult() *NodeAggregateResult { + return &NodeAggregateResult{} } -var NodeQueryResult_Success_DEFAULT *QueryResult_ +var NodeAggregateResult_Success_DEFAULT *AggregateQueryResult_ -func (p *NodeQueryResult) GetSuccess() *QueryResult_ { +func (p *NodeAggregateResult) GetSuccess() *AggregateQueryResult_ { if !p.IsSetSuccess() { - return NodeQueryResult_Success_DEFAULT + return NodeAggregateResult_Success_DEFAULT } return p.Success } -var NodeQueryResult_Err_DEFAULT *Error +var NodeAggregateResult_Err_DEFAULT *Error -func (p *NodeQueryResult) GetErr() *Error { +func (p *NodeAggregateResult) GetErr() *Error { if !p.IsSetErr() { - return NodeQueryResult_Err_DEFAULT + return NodeAggregateResult_Err_DEFAULT } return p.Err } -func (p *NodeQueryResult) IsSetSuccess() bool { +func (p *NodeAggregateResult) IsSetSuccess() bool { return p.Success != nil } -func (p *NodeQueryResult) IsSetErr() bool { +func (p *NodeAggregateResult) IsSetErr() bool { return p.Err != nil } -func (p *NodeQueryResult) Read(iprot thrift.TProtocol) error { +func (p *NodeAggregateResult) Read(iprot thrift.TProtocol) error { if _, err := iprot.ReadStructBegin(); err != nil { return thrift.PrependError(fmt.Sprintf("%T read error: ", p), err) } @@ -15111,15 +16665,15 @@ func (p *NodeQueryResult) Read(iprot thrift.TProtocol) error { return nil } -func (p *NodeQueryResult) ReadField0(iprot thrift.TProtocol) error { - p.Success = &QueryResult_{} +func (p *NodeAggregateResult) ReadField0(iprot thrift.TProtocol) error { + p.Success = &AggregateQueryResult_{} if err := p.Success.Read(iprot); err != nil { return thrift.PrependError(fmt.Sprintf("%T error reading struct: ", p.Success), err) } return nil } -func (p *NodeQueryResult) ReadField1(iprot thrift.TProtocol) error { +func (p *NodeAggregateResult) ReadField1(iprot thrift.TProtocol) error { p.Err = &Error{ Type: 0, } @@ -15129,8 +16683,8 @@ func (p *NodeQueryResult) ReadField1(iprot thrift.TProtocol) error { return nil } -func (p *NodeQueryResult) Write(oprot thrift.TProtocol) error { - if err := oprot.WriteStructBegin("query_result"); err != nil { +func (p *NodeAggregateResult) Write(oprot thrift.TProtocol) error { + if err := oprot.WriteStructBegin("aggregate_result"); err != nil { return thrift.PrependError(fmt.Sprintf("%T write struct begin error: ", p), err) } if p != nil { @@ -15150,7 +16704,7 @@ func (p *NodeQueryResult) Write(oprot thrift.TProtocol) error { return nil } -func (p *NodeQueryResult) writeField0(oprot thrift.TProtocol) (err error) { +func (p *NodeAggregateResult) writeField0(oprot thrift.TProtocol) (err error) { if p.IsSetSuccess() { if err := oprot.WriteFieldBegin("success", thrift.STRUCT, 0); err != nil { return thrift.PrependError(fmt.Sprintf("%T write field begin error 0:success: ", p), err) @@ -15165,7 +16719,7 @@ func (p *NodeQueryResult) writeField0(oprot thrift.TProtocol) (err error) { return err } -func (p *NodeQueryResult) writeField1(oprot thrift.TProtocol) (err error) { +func (p *NodeAggregateResult) writeField1(oprot thrift.TProtocol) (err error) { if p.IsSetErr() { if err := oprot.WriteFieldBegin("err", thrift.STRUCT, 1); err != nil { return thrift.PrependError(fmt.Sprintf("%T write field begin error 1:err: ", p), err) @@ -15180,36 +16734,36 @@ func (p *NodeQueryResult) writeField1(oprot thrift.TProtocol) (err error) { return err } -func (p *NodeQueryResult) String() string { +func (p *NodeAggregateResult) String() string { if p == nil { return "" } - return fmt.Sprintf("NodeQueryResult(%+v)", *p) + return fmt.Sprintf("NodeAggregateResult(%+v)", *p) } // Attributes: // - Req -type NodeAggregateRawArgs struct { - Req *AggregateQueryRawRequest `thrift:"req,1" db:"req" json:"req"` +type NodeFetchArgs struct { + Req *FetchRequest `thrift:"req,1" db:"req" json:"req"` } -func NewNodeAggregateRawArgs() *NodeAggregateRawArgs { - return &NodeAggregateRawArgs{} +func NewNodeFetchArgs() *NodeFetchArgs { + return &NodeFetchArgs{} } -var NodeAggregateRawArgs_Req_DEFAULT *AggregateQueryRawRequest +var NodeFetchArgs_Req_DEFAULT *FetchRequest -func (p *NodeAggregateRawArgs) GetReq() *AggregateQueryRawRequest { +func (p *NodeFetchArgs) GetReq() *FetchRequest { if !p.IsSetReq() { - return NodeAggregateRawArgs_Req_DEFAULT + return NodeFetchArgs_Req_DEFAULT } return p.Req } -func (p *NodeAggregateRawArgs) IsSetReq() bool { +func (p *NodeFetchArgs) IsSetReq() bool { return p.Req != nil } -func (p *NodeAggregateRawArgs) Read(iprot thrift.TProtocol) error { +func (p *NodeFetchArgs) Read(iprot thrift.TProtocol) error { if _, err := iprot.ReadStructBegin(); err != nil { return thrift.PrependError(fmt.Sprintf("%T read error: ", p), err) } @@ -15242,11 +16796,11 @@ func (p *NodeAggregateRawArgs) Read(iprot thrift.TProtocol) error { return nil } -func (p *NodeAggregateRawArgs) ReadField1(iprot thrift.TProtocol) error { - p.Req = &AggregateQueryRawRequest{ - AggregateQueryType: 1, - +func (p *NodeFetchArgs) ReadField1(iprot thrift.TProtocol) error { + p.Req = &FetchRequest{ RangeType: 0, + + ResultTimeType: 0, } if err := p.Req.Read(iprot); err != nil { return thrift.PrependError(fmt.Sprintf("%T error reading struct: ", p.Req), err) @@ -15254,8 +16808,8 @@ func (p *NodeAggregateRawArgs) ReadField1(iprot thrift.TProtocol) error { return nil } -func (p *NodeAggregateRawArgs) Write(oprot thrift.TProtocol) error { - if err := oprot.WriteStructBegin("aggregateRaw_args"); err != nil { +func (p *NodeFetchArgs) Write(oprot thrift.TProtocol) error { + if err := oprot.WriteStructBegin("fetch_args"); err != nil { return thrift.PrependError(fmt.Sprintf("%T write struct begin error: ", p), err) } if p != nil { @@ -15272,7 +16826,7 @@ func (p *NodeAggregateRawArgs) Write(oprot thrift.TProtocol) error { return nil } -func (p *NodeAggregateRawArgs) writeField1(oprot thrift.TProtocol) (err error) { +func (p *NodeFetchArgs) writeField1(oprot thrift.TProtocol) (err error) { if err := oprot.WriteFieldBegin("req", thrift.STRUCT, 1); err != nil { return thrift.PrependError(fmt.Sprintf("%T write field begin error 1:req: ", p), err) } @@ -15285,51 +16839,51 @@ func (p *NodeAggregateRawArgs) writeField1(oprot thrift.TProtocol) (err error) { return err } -func (p *NodeAggregateRawArgs) String() string { +func (p *NodeFetchArgs) String() string { if p == nil { return "" } - return fmt.Sprintf("NodeAggregateRawArgs(%+v)", *p) + return fmt.Sprintf("NodeFetchArgs(%+v)", *p) } // Attributes: // - Success // - Err -type NodeAggregateRawResult struct { - Success *AggregateQueryRawResult_ `thrift:"success,0" db:"success" json:"success,omitempty"` - Err *Error `thrift:"err,1" db:"err" json:"err,omitempty"` +type NodeFetchResult struct { + Success *FetchResult_ `thrift:"success,0" db:"success" json:"success,omitempty"` + Err *Error `thrift:"err,1" db:"err" json:"err,omitempty"` } -func NewNodeAggregateRawResult() *NodeAggregateRawResult { - return &NodeAggregateRawResult{} +func NewNodeFetchResult() *NodeFetchResult { + return &NodeFetchResult{} } -var NodeAggregateRawResult_Success_DEFAULT *AggregateQueryRawResult_ +var NodeFetchResult_Success_DEFAULT *FetchResult_ -func (p *NodeAggregateRawResult) GetSuccess() *AggregateQueryRawResult_ { +func (p *NodeFetchResult) GetSuccess() *FetchResult_ { if !p.IsSetSuccess() { - return NodeAggregateRawResult_Success_DEFAULT + return NodeFetchResult_Success_DEFAULT } return p.Success } -var NodeAggregateRawResult_Err_DEFAULT *Error +var NodeFetchResult_Err_DEFAULT *Error -func (p *NodeAggregateRawResult) GetErr() *Error { +func (p *NodeFetchResult) GetErr() *Error { if !p.IsSetErr() { - return NodeAggregateRawResult_Err_DEFAULT + return NodeFetchResult_Err_DEFAULT } return p.Err } -func (p *NodeAggregateRawResult) IsSetSuccess() bool { +func (p *NodeFetchResult) IsSetSuccess() bool { return p.Success != nil } -func (p *NodeAggregateRawResult) IsSetErr() bool { +func (p *NodeFetchResult) IsSetErr() bool { return p.Err != nil } -func (p *NodeAggregateRawResult) Read(iprot thrift.TProtocol) error { +func (p *NodeFetchResult) Read(iprot thrift.TProtocol) error { if _, err := iprot.ReadStructBegin(); err != nil { return thrift.PrependError(fmt.Sprintf("%T read error: ", p), err) } @@ -15366,15 +16920,15 @@ func (p *NodeAggregateRawResult) Read(iprot thrift.TProtocol) error { return nil } -func (p *NodeAggregateRawResult) ReadField0(iprot thrift.TProtocol) error { - p.Success = &AggregateQueryRawResult_{} +func (p *NodeFetchResult) ReadField0(iprot thrift.TProtocol) error { + p.Success = &FetchResult_{} if err := p.Success.Read(iprot); err != nil { return thrift.PrependError(fmt.Sprintf("%T error reading struct: ", p.Success), err) } return nil } -func (p *NodeAggregateRawResult) ReadField1(iprot thrift.TProtocol) error { +func (p *NodeFetchResult) ReadField1(iprot thrift.TProtocol) error { p.Err = &Error{ Type: 0, } @@ -15384,8 +16938,8 @@ func (p *NodeAggregateRawResult) ReadField1(iprot thrift.TProtocol) error { return nil } -func (p *NodeAggregateRawResult) Write(oprot thrift.TProtocol) error { - if err := oprot.WriteStructBegin("aggregateRaw_result"); err != nil { +func (p *NodeFetchResult) Write(oprot thrift.TProtocol) error { + if err := oprot.WriteStructBegin("fetch_result"); err != nil { return thrift.PrependError(fmt.Sprintf("%T write struct begin error: ", p), err) } if p != nil { @@ -15405,7 +16959,7 @@ func (p *NodeAggregateRawResult) Write(oprot thrift.TProtocol) error { return nil } -func (p *NodeAggregateRawResult) writeField0(oprot thrift.TProtocol) (err error) { +func (p *NodeFetchResult) writeField0(oprot thrift.TProtocol) (err error) { if p.IsSetSuccess() { if err := oprot.WriteFieldBegin("success", thrift.STRUCT, 0); err != nil { return thrift.PrependError(fmt.Sprintf("%T write field begin error 0:success: ", p), err) @@ -15420,7 +16974,7 @@ func (p *NodeAggregateRawResult) writeField0(oprot thrift.TProtocol) (err error) return err } -func (p *NodeAggregateRawResult) writeField1(oprot thrift.TProtocol) (err error) { +func (p *NodeFetchResult) writeField1(oprot thrift.TProtocol) (err error) { if p.IsSetErr() { if err := oprot.WriteFieldBegin("err", thrift.STRUCT, 1); err != nil { return thrift.PrependError(fmt.Sprintf("%T write field begin error 1:err: ", p), err) @@ -15435,36 +16989,36 @@ func (p *NodeAggregateRawResult) writeField1(oprot thrift.TProtocol) (err error) return err } -func (p *NodeAggregateRawResult) String() string { +func (p *NodeFetchResult) String() string { if p == nil { return "" } - return fmt.Sprintf("NodeAggregateRawResult(%+v)", *p) + return fmt.Sprintf("NodeFetchResult(%+v)", *p) } // Attributes: // - Req -type NodeAggregateArgs struct { - Req *AggregateQueryRequest `thrift:"req,1" db:"req" json:"req"` +type NodeFetchTaggedArgs struct { + Req *FetchTaggedRequest `thrift:"req,1" db:"req" json:"req"` } -func NewNodeAggregateArgs() *NodeAggregateArgs { - return &NodeAggregateArgs{} +func NewNodeFetchTaggedArgs() *NodeFetchTaggedArgs { + return &NodeFetchTaggedArgs{} } -var NodeAggregateArgs_Req_DEFAULT *AggregateQueryRequest +var NodeFetchTaggedArgs_Req_DEFAULT *FetchTaggedRequest -func (p *NodeAggregateArgs) GetReq() *AggregateQueryRequest { +func (p *NodeFetchTaggedArgs) GetReq() *FetchTaggedRequest { if !p.IsSetReq() { - return NodeAggregateArgs_Req_DEFAULT + return NodeFetchTaggedArgs_Req_DEFAULT } return p.Req } -func (p *NodeAggregateArgs) IsSetReq() bool { +func (p *NodeFetchTaggedArgs) IsSetReq() bool { return p.Req != nil } -func (p *NodeAggregateArgs) Read(iprot thrift.TProtocol) error { +func (p *NodeFetchTaggedArgs) Read(iprot thrift.TProtocol) error { if _, err := iprot.ReadStructBegin(); err != nil { return thrift.PrependError(fmt.Sprintf("%T read error: ", p), err) } @@ -15497,11 +17051,9 @@ func (p *NodeAggregateArgs) Read(iprot thrift.TProtocol) error { return nil } -func (p *NodeAggregateArgs) ReadField1(iprot thrift.TProtocol) error { - p.Req = &AggregateQueryRequest{ - AggregateQueryType: 1, - - RangeType: 0, +func (p *NodeFetchTaggedArgs) ReadField1(iprot thrift.TProtocol) error { + p.Req = &FetchTaggedRequest{ + RangeTimeType: 0, } if err := p.Req.Read(iprot); err != nil { return thrift.PrependError(fmt.Sprintf("%T error reading struct: ", p.Req), err) @@ -15509,8 +17061,8 @@ func (p *NodeAggregateArgs) ReadField1(iprot thrift.TProtocol) error { return nil } -func (p *NodeAggregateArgs) Write(oprot thrift.TProtocol) error { - if err := oprot.WriteStructBegin("aggregate_args"); err != nil { +func (p *NodeFetchTaggedArgs) Write(oprot thrift.TProtocol) error { + if err := oprot.WriteStructBegin("fetchTagged_args"); err != nil { return thrift.PrependError(fmt.Sprintf("%T write struct begin error: ", p), err) } if p != nil { @@ -15527,7 +17079,7 @@ func (p *NodeAggregateArgs) Write(oprot thrift.TProtocol) error { return nil } -func (p *NodeAggregateArgs) writeField1(oprot thrift.TProtocol) (err error) { +func (p *NodeFetchTaggedArgs) writeField1(oprot thrift.TProtocol) (err error) { if err := oprot.WriteFieldBegin("req", thrift.STRUCT, 1); err != nil { return thrift.PrependError(fmt.Sprintf("%T write field begin error 1:req: ", p), err) } @@ -15540,51 +17092,51 @@ func (p *NodeAggregateArgs) writeField1(oprot thrift.TProtocol) (err error) { return err } -func (p *NodeAggregateArgs) String() string { +func (p *NodeFetchTaggedArgs) String() string { if p == nil { return "" } - return fmt.Sprintf("NodeAggregateArgs(%+v)", *p) + return fmt.Sprintf("NodeFetchTaggedArgs(%+v)", *p) } // Attributes: // - Success // - Err -type NodeAggregateResult struct { - Success *AggregateQueryResult_ `thrift:"success,0" db:"success" json:"success,omitempty"` - Err *Error `thrift:"err,1" db:"err" json:"err,omitempty"` +type NodeFetchTaggedResult struct { + Success *FetchTaggedResult_ `thrift:"success,0" db:"success" json:"success,omitempty"` + Err *Error `thrift:"err,1" db:"err" json:"err,omitempty"` } -func NewNodeAggregateResult() *NodeAggregateResult { - return &NodeAggregateResult{} +func NewNodeFetchTaggedResult() *NodeFetchTaggedResult { + return &NodeFetchTaggedResult{} } -var NodeAggregateResult_Success_DEFAULT *AggregateQueryResult_ +var NodeFetchTaggedResult_Success_DEFAULT *FetchTaggedResult_ -func (p *NodeAggregateResult) GetSuccess() *AggregateQueryResult_ { +func (p *NodeFetchTaggedResult) GetSuccess() *FetchTaggedResult_ { if !p.IsSetSuccess() { - return NodeAggregateResult_Success_DEFAULT + return NodeFetchTaggedResult_Success_DEFAULT } return p.Success } -var NodeAggregateResult_Err_DEFAULT *Error +var NodeFetchTaggedResult_Err_DEFAULT *Error -func (p *NodeAggregateResult) GetErr() *Error { +func (p *NodeFetchTaggedResult) GetErr() *Error { if !p.IsSetErr() { - return NodeAggregateResult_Err_DEFAULT + return NodeFetchTaggedResult_Err_DEFAULT } return p.Err } -func (p *NodeAggregateResult) IsSetSuccess() bool { +func (p *NodeFetchTaggedResult) IsSetSuccess() bool { return p.Success != nil } -func (p *NodeAggregateResult) IsSetErr() bool { +func (p *NodeFetchTaggedResult) IsSetErr() bool { return p.Err != nil } -func (p *NodeAggregateResult) Read(iprot thrift.TProtocol) error { +func (p *NodeFetchTaggedResult) Read(iprot thrift.TProtocol) error { if _, err := iprot.ReadStructBegin(); err != nil { return thrift.PrependError(fmt.Sprintf("%T read error: ", p), err) } @@ -15621,15 +17173,15 @@ func (p *NodeAggregateResult) Read(iprot thrift.TProtocol) error { return nil } -func (p *NodeAggregateResult) ReadField0(iprot thrift.TProtocol) error { - p.Success = &AggregateQueryResult_{} +func (p *NodeFetchTaggedResult) ReadField0(iprot thrift.TProtocol) error { + p.Success = &FetchTaggedResult_{} if err := p.Success.Read(iprot); err != nil { return thrift.PrependError(fmt.Sprintf("%T error reading struct: ", p.Success), err) } return nil } -func (p *NodeAggregateResult) ReadField1(iprot thrift.TProtocol) error { +func (p *NodeFetchTaggedResult) ReadField1(iprot thrift.TProtocol) error { p.Err = &Error{ Type: 0, } @@ -15639,8 +17191,8 @@ func (p *NodeAggregateResult) ReadField1(iprot thrift.TProtocol) error { return nil } -func (p *NodeAggregateResult) Write(oprot thrift.TProtocol) error { - if err := oprot.WriteStructBegin("aggregate_result"); err != nil { +func (p *NodeFetchTaggedResult) Write(oprot thrift.TProtocol) error { + if err := oprot.WriteStructBegin("fetchTagged_result"); err != nil { return thrift.PrependError(fmt.Sprintf("%T write struct begin error: ", p), err) } if p != nil { @@ -15660,7 +17212,7 @@ func (p *NodeAggregateResult) Write(oprot thrift.TProtocol) error { return nil } -func (p *NodeAggregateResult) writeField0(oprot thrift.TProtocol) (err error) { +func (p *NodeFetchTaggedResult) writeField0(oprot thrift.TProtocol) (err error) { if p.IsSetSuccess() { if err := oprot.WriteFieldBegin("success", thrift.STRUCT, 0); err != nil { return thrift.PrependError(fmt.Sprintf("%T write field begin error 0:success: ", p), err) @@ -15675,7 +17227,7 @@ func (p *NodeAggregateResult) writeField0(oprot thrift.TProtocol) (err error) { return err } -func (p *NodeAggregateResult) writeField1(oprot thrift.TProtocol) (err error) { +func (p *NodeFetchTaggedResult) writeField1(oprot thrift.TProtocol) (err error) { if p.IsSetErr() { if err := oprot.WriteFieldBegin("err", thrift.STRUCT, 1); err != nil { return thrift.PrependError(fmt.Sprintf("%T write field begin error 1:err: ", p), err) @@ -15690,36 +17242,36 @@ func (p *NodeAggregateResult) writeField1(oprot thrift.TProtocol) (err error) { return err } -func (p *NodeAggregateResult) String() string { +func (p *NodeFetchTaggedResult) String() string { if p == nil { return "" } - return fmt.Sprintf("NodeAggregateResult(%+v)", *p) + return fmt.Sprintf("NodeFetchTaggedResult(%+v)", *p) } // Attributes: // - Req -type NodeFetchArgs struct { - Req *FetchRequest `thrift:"req,1" db:"req" json:"req"` +type NodeWriteArgs struct { + Req *WriteRequest `thrift:"req,1" db:"req" json:"req"` } -func NewNodeFetchArgs() *NodeFetchArgs { - return &NodeFetchArgs{} +func NewNodeWriteArgs() *NodeWriteArgs { + return &NodeWriteArgs{} } -var NodeFetchArgs_Req_DEFAULT *FetchRequest +var NodeWriteArgs_Req_DEFAULT *WriteRequest -func (p *NodeFetchArgs) GetReq() *FetchRequest { +func (p *NodeWriteArgs) GetReq() *WriteRequest { if !p.IsSetReq() { - return NodeFetchArgs_Req_DEFAULT + return NodeWriteArgs_Req_DEFAULT } return p.Req } -func (p *NodeFetchArgs) IsSetReq() bool { +func (p *NodeWriteArgs) IsSetReq() bool { return p.Req != nil } -func (p *NodeFetchArgs) Read(iprot thrift.TProtocol) error { +func (p *NodeWriteArgs) Read(iprot thrift.TProtocol) error { if _, err := iprot.ReadStructBegin(); err != nil { return thrift.PrependError(fmt.Sprintf("%T read error: ", p), err) } @@ -15752,20 +17304,16 @@ func (p *NodeFetchArgs) Read(iprot thrift.TProtocol) error { return nil } -func (p *NodeFetchArgs) ReadField1(iprot thrift.TProtocol) error { - p.Req = &FetchRequest{ - RangeType: 0, - - ResultTimeType: 0, - } +func (p *NodeWriteArgs) ReadField1(iprot thrift.TProtocol) error { + p.Req = &WriteRequest{} if err := p.Req.Read(iprot); err != nil { return thrift.PrependError(fmt.Sprintf("%T error reading struct: ", p.Req), err) } return nil } -func (p *NodeFetchArgs) Write(oprot thrift.TProtocol) error { - if err := oprot.WriteStructBegin("fetch_args"); err != nil { +func (p *NodeWriteArgs) Write(oprot thrift.TProtocol) error { + if err := oprot.WriteStructBegin("write_args"); err != nil { return thrift.PrependError(fmt.Sprintf("%T write struct begin error: ", p), err) } if p != nil { @@ -15782,7 +17330,7 @@ func (p *NodeFetchArgs) Write(oprot thrift.TProtocol) error { return nil } -func (p *NodeFetchArgs) writeField1(oprot thrift.TProtocol) (err error) { +func (p *NodeWriteArgs) writeField1(oprot thrift.TProtocol) (err error) { if err := oprot.WriteFieldBegin("req", thrift.STRUCT, 1); err != nil { return thrift.PrependError(fmt.Sprintf("%T write field begin error 1:req: ", p), err) } @@ -15795,51 +17343,36 @@ func (p *NodeFetchArgs) writeField1(oprot thrift.TProtocol) (err error) { return err } -func (p *NodeFetchArgs) String() string { +func (p *NodeWriteArgs) String() string { if p == nil { return "" } - return fmt.Sprintf("NodeFetchArgs(%+v)", *p) + return fmt.Sprintf("NodeWriteArgs(%+v)", *p) } // Attributes: -// - Success // - Err -type NodeFetchResult struct { - Success *FetchResult_ `thrift:"success,0" db:"success" json:"success,omitempty"` - Err *Error `thrift:"err,1" db:"err" json:"err,omitempty"` -} - -func NewNodeFetchResult() *NodeFetchResult { - return &NodeFetchResult{} +type NodeWriteResult struct { + Err *Error `thrift:"err,1" db:"err" json:"err,omitempty"` } -var NodeFetchResult_Success_DEFAULT *FetchResult_ - -func (p *NodeFetchResult) GetSuccess() *FetchResult_ { - if !p.IsSetSuccess() { - return NodeFetchResult_Success_DEFAULT - } - return p.Success +func NewNodeWriteResult() *NodeWriteResult { + return &NodeWriteResult{} } -var NodeFetchResult_Err_DEFAULT *Error +var NodeWriteResult_Err_DEFAULT *Error -func (p *NodeFetchResult) GetErr() *Error { +func (p *NodeWriteResult) GetErr() *Error { if !p.IsSetErr() { - return NodeFetchResult_Err_DEFAULT + return NodeWriteResult_Err_DEFAULT } return p.Err } -func (p *NodeFetchResult) IsSetSuccess() bool { - return p.Success != nil -} - -func (p *NodeFetchResult) IsSetErr() bool { +func (p *NodeWriteResult) IsSetErr() bool { return p.Err != nil } -func (p *NodeFetchResult) Read(iprot thrift.TProtocol) error { +func (p *NodeWriteResult) Read(iprot thrift.TProtocol) error { if _, err := iprot.ReadStructBegin(); err != nil { return thrift.PrependError(fmt.Sprintf("%T read error: ", p), err) } @@ -15853,10 +17386,6 @@ func (p *NodeFetchResult) Read(iprot thrift.TProtocol) error { break } switch fieldId { - case 0: - if err := p.ReadField0(iprot); err != nil { - return err - } case 1: if err := p.ReadField1(iprot); err != nil { return err @@ -15876,15 +17405,7 @@ func (p *NodeFetchResult) Read(iprot thrift.TProtocol) error { return nil } -func (p *NodeFetchResult) ReadField0(iprot thrift.TProtocol) error { - p.Success = &FetchResult_{} - if err := p.Success.Read(iprot); err != nil { - return thrift.PrependError(fmt.Sprintf("%T error reading struct: ", p.Success), err) - } - return nil -} - -func (p *NodeFetchResult) ReadField1(iprot thrift.TProtocol) error { +func (p *NodeWriteResult) ReadField1(iprot thrift.TProtocol) error { p.Err = &Error{ Type: 0, } @@ -15894,14 +17415,11 @@ func (p *NodeFetchResult) ReadField1(iprot thrift.TProtocol) error { return nil } -func (p *NodeFetchResult) Write(oprot thrift.TProtocol) error { - if err := oprot.WriteStructBegin("fetch_result"); err != nil { +func (p *NodeWriteResult) Write(oprot thrift.TProtocol) error { + if err := oprot.WriteStructBegin("write_result"); err != nil { return thrift.PrependError(fmt.Sprintf("%T write struct begin error: ", p), err) } if p != nil { - if err := p.writeField0(oprot); err != nil { - return err - } if err := p.writeField1(oprot); err != nil { return err } @@ -15915,22 +17433,7 @@ func (p *NodeFetchResult) Write(oprot thrift.TProtocol) error { return nil } -func (p *NodeFetchResult) writeField0(oprot thrift.TProtocol) (err error) { - if p.IsSetSuccess() { - if err := oprot.WriteFieldBegin("success", thrift.STRUCT, 0); err != nil { - return thrift.PrependError(fmt.Sprintf("%T write field begin error 0:success: ", p), err) - } - if err := p.Success.Write(oprot); err != nil { - return thrift.PrependError(fmt.Sprintf("%T error writing struct: ", p.Success), err) - } - if err := oprot.WriteFieldEnd(); err != nil { - return thrift.PrependError(fmt.Sprintf("%T write field end error 0:success: ", p), err) - } - } - return err -} - -func (p *NodeFetchResult) writeField1(oprot thrift.TProtocol) (err error) { +func (p *NodeWriteResult) writeField1(oprot thrift.TProtocol) (err error) { if p.IsSetErr() { if err := oprot.WriteFieldBegin("err", thrift.STRUCT, 1); err != nil { return thrift.PrependError(fmt.Sprintf("%T write field begin error 1:err: ", p), err) @@ -15945,36 +17448,36 @@ func (p *NodeFetchResult) writeField1(oprot thrift.TProtocol) (err error) { return err } -func (p *NodeFetchResult) String() string { +func (p *NodeWriteResult) String() string { if p == nil { return "" } - return fmt.Sprintf("NodeFetchResult(%+v)", *p) + return fmt.Sprintf("NodeWriteResult(%+v)", *p) } // Attributes: // - Req -type NodeFetchTaggedArgs struct { - Req *FetchTaggedRequest `thrift:"req,1" db:"req" json:"req"` +type NodeWriteTaggedArgs struct { + Req *WriteTaggedRequest `thrift:"req,1" db:"req" json:"req"` } -func NewNodeFetchTaggedArgs() *NodeFetchTaggedArgs { - return &NodeFetchTaggedArgs{} +func NewNodeWriteTaggedArgs() *NodeWriteTaggedArgs { + return &NodeWriteTaggedArgs{} } -var NodeFetchTaggedArgs_Req_DEFAULT *FetchTaggedRequest +var NodeWriteTaggedArgs_Req_DEFAULT *WriteTaggedRequest -func (p *NodeFetchTaggedArgs) GetReq() *FetchTaggedRequest { +func (p *NodeWriteTaggedArgs) GetReq() *WriteTaggedRequest { if !p.IsSetReq() { - return NodeFetchTaggedArgs_Req_DEFAULT + return NodeWriteTaggedArgs_Req_DEFAULT } return p.Req } -func (p *NodeFetchTaggedArgs) IsSetReq() bool { +func (p *NodeWriteTaggedArgs) IsSetReq() bool { return p.Req != nil } -func (p *NodeFetchTaggedArgs) Read(iprot thrift.TProtocol) error { +func (p *NodeWriteTaggedArgs) Read(iprot thrift.TProtocol) error { if _, err := iprot.ReadStructBegin(); err != nil { return thrift.PrependError(fmt.Sprintf("%T read error: ", p), err) } @@ -16007,18 +17510,16 @@ func (p *NodeFetchTaggedArgs) Read(iprot thrift.TProtocol) error { return nil } -func (p *NodeFetchTaggedArgs) ReadField1(iprot thrift.TProtocol) error { - p.Req = &FetchTaggedRequest{ - RangeTimeType: 0, - } +func (p *NodeWriteTaggedArgs) ReadField1(iprot thrift.TProtocol) error { + p.Req = &WriteTaggedRequest{} if err := p.Req.Read(iprot); err != nil { return thrift.PrependError(fmt.Sprintf("%T error reading struct: ", p.Req), err) } return nil } -func (p *NodeFetchTaggedArgs) Write(oprot thrift.TProtocol) error { - if err := oprot.WriteStructBegin("fetchTagged_args"); err != nil { +func (p *NodeWriteTaggedArgs) Write(oprot thrift.TProtocol) error { + if err := oprot.WriteStructBegin("writeTagged_args"); err != nil { return thrift.PrependError(fmt.Sprintf("%T write struct begin error: ", p), err) } if p != nil { @@ -16035,7 +17536,7 @@ func (p *NodeFetchTaggedArgs) Write(oprot thrift.TProtocol) error { return nil } -func (p *NodeFetchTaggedArgs) writeField1(oprot thrift.TProtocol) (err error) { +func (p *NodeWriteTaggedArgs) writeField1(oprot thrift.TProtocol) (err error) { if err := oprot.WriteFieldBegin("req", thrift.STRUCT, 1); err != nil { return thrift.PrependError(fmt.Sprintf("%T write field begin error 1:req: ", p), err) } @@ -16048,51 +17549,36 @@ func (p *NodeFetchTaggedArgs) writeField1(oprot thrift.TProtocol) (err error) { return err } -func (p *NodeFetchTaggedArgs) String() string { +func (p *NodeWriteTaggedArgs) String() string { if p == nil { return "" } - return fmt.Sprintf("NodeFetchTaggedArgs(%+v)", *p) + return fmt.Sprintf("NodeWriteTaggedArgs(%+v)", *p) } // Attributes: -// - Success // - Err -type NodeFetchTaggedResult struct { - Success *FetchTaggedResult_ `thrift:"success,0" db:"success" json:"success,omitempty"` - Err *Error `thrift:"err,1" db:"err" json:"err,omitempty"` -} - -func NewNodeFetchTaggedResult() *NodeFetchTaggedResult { - return &NodeFetchTaggedResult{} +type NodeWriteTaggedResult struct { + Err *Error `thrift:"err,1" db:"err" json:"err,omitempty"` } -var NodeFetchTaggedResult_Success_DEFAULT *FetchTaggedResult_ - -func (p *NodeFetchTaggedResult) GetSuccess() *FetchTaggedResult_ { - if !p.IsSetSuccess() { - return NodeFetchTaggedResult_Success_DEFAULT - } - return p.Success +func NewNodeWriteTaggedResult() *NodeWriteTaggedResult { + return &NodeWriteTaggedResult{} } -var NodeFetchTaggedResult_Err_DEFAULT *Error +var NodeWriteTaggedResult_Err_DEFAULT *Error -func (p *NodeFetchTaggedResult) GetErr() *Error { +func (p *NodeWriteTaggedResult) GetErr() *Error { if !p.IsSetErr() { - return NodeFetchTaggedResult_Err_DEFAULT + return NodeWriteTaggedResult_Err_DEFAULT } return p.Err } -func (p *NodeFetchTaggedResult) IsSetSuccess() bool { - return p.Success != nil -} - -func (p *NodeFetchTaggedResult) IsSetErr() bool { +func (p *NodeWriteTaggedResult) IsSetErr() bool { return p.Err != nil } -func (p *NodeFetchTaggedResult) Read(iprot thrift.TProtocol) error { +func (p *NodeWriteTaggedResult) Read(iprot thrift.TProtocol) error { if _, err := iprot.ReadStructBegin(); err != nil { return thrift.PrependError(fmt.Sprintf("%T read error: ", p), err) } @@ -16106,10 +17592,6 @@ func (p *NodeFetchTaggedResult) Read(iprot thrift.TProtocol) error { break } switch fieldId { - case 0: - if err := p.ReadField0(iprot); err != nil { - return err - } case 1: if err := p.ReadField1(iprot); err != nil { return err @@ -16129,15 +17611,7 @@ func (p *NodeFetchTaggedResult) Read(iprot thrift.TProtocol) error { return nil } -func (p *NodeFetchTaggedResult) ReadField0(iprot thrift.TProtocol) error { - p.Success = &FetchTaggedResult_{} - if err := p.Success.Read(iprot); err != nil { - return thrift.PrependError(fmt.Sprintf("%T error reading struct: ", p.Success), err) - } - return nil -} - -func (p *NodeFetchTaggedResult) ReadField1(iprot thrift.TProtocol) error { +func (p *NodeWriteTaggedResult) ReadField1(iprot thrift.TProtocol) error { p.Err = &Error{ Type: 0, } @@ -16147,14 +17621,11 @@ func (p *NodeFetchTaggedResult) ReadField1(iprot thrift.TProtocol) error { return nil } -func (p *NodeFetchTaggedResult) Write(oprot thrift.TProtocol) error { - if err := oprot.WriteStructBegin("fetchTagged_result"); err != nil { +func (p *NodeWriteTaggedResult) Write(oprot thrift.TProtocol) error { + if err := oprot.WriteStructBegin("writeTagged_result"); err != nil { return thrift.PrependError(fmt.Sprintf("%T write struct begin error: ", p), err) } if p != nil { - if err := p.writeField0(oprot); err != nil { - return err - } if err := p.writeField1(oprot); err != nil { return err } @@ -16168,22 +17639,7 @@ func (p *NodeFetchTaggedResult) Write(oprot thrift.TProtocol) error { return nil } -func (p *NodeFetchTaggedResult) writeField0(oprot thrift.TProtocol) (err error) { - if p.IsSetSuccess() { - if err := oprot.WriteFieldBegin("success", thrift.STRUCT, 0); err != nil { - return thrift.PrependError(fmt.Sprintf("%T write field begin error 0:success: ", p), err) - } - if err := p.Success.Write(oprot); err != nil { - return thrift.PrependError(fmt.Sprintf("%T error writing struct: ", p.Success), err) - } - if err := oprot.WriteFieldEnd(); err != nil { - return thrift.PrependError(fmt.Sprintf("%T write field end error 0:success: ", p), err) - } - } - return err -} - -func (p *NodeFetchTaggedResult) writeField1(oprot thrift.TProtocol) (err error) { +func (p *NodeWriteTaggedResult) writeField1(oprot thrift.TProtocol) (err error) { if p.IsSetErr() { if err := oprot.WriteFieldBegin("err", thrift.STRUCT, 1); err != nil { return thrift.PrependError(fmt.Sprintf("%T write field begin error 1:err: ", p), err) @@ -16198,36 +17654,36 @@ func (p *NodeFetchTaggedResult) writeField1(oprot thrift.TProtocol) (err error) return err } -func (p *NodeFetchTaggedResult) String() string { +func (p *NodeWriteTaggedResult) String() string { if p == nil { return "" } - return fmt.Sprintf("NodeFetchTaggedResult(%+v)", *p) + return fmt.Sprintf("NodeWriteTaggedResult(%+v)", *p) } // Attributes: // - Req -type NodeWriteArgs struct { - Req *WriteRequest `thrift:"req,1" db:"req" json:"req"` +type NodeFetchBatchRawArgs struct { + Req *FetchBatchRawRequest `thrift:"req,1" db:"req" json:"req"` } -func NewNodeWriteArgs() *NodeWriteArgs { - return &NodeWriteArgs{} +func NewNodeFetchBatchRawArgs() *NodeFetchBatchRawArgs { + return &NodeFetchBatchRawArgs{} } -var NodeWriteArgs_Req_DEFAULT *WriteRequest +var NodeFetchBatchRawArgs_Req_DEFAULT *FetchBatchRawRequest -func (p *NodeWriteArgs) GetReq() *WriteRequest { +func (p *NodeFetchBatchRawArgs) GetReq() *FetchBatchRawRequest { if !p.IsSetReq() { - return NodeWriteArgs_Req_DEFAULT + return NodeFetchBatchRawArgs_Req_DEFAULT } return p.Req } -func (p *NodeWriteArgs) IsSetReq() bool { +func (p *NodeFetchBatchRawArgs) IsSetReq() bool { return p.Req != nil } -func (p *NodeWriteArgs) Read(iprot thrift.TProtocol) error { +func (p *NodeFetchBatchRawArgs) Read(iprot thrift.TProtocol) error { if _, err := iprot.ReadStructBegin(); err != nil { return thrift.PrependError(fmt.Sprintf("%T read error: ", p), err) } @@ -16260,16 +17716,18 @@ func (p *NodeWriteArgs) Read(iprot thrift.TProtocol) error { return nil } -func (p *NodeWriteArgs) ReadField1(iprot thrift.TProtocol) error { - p.Req = &WriteRequest{} +func (p *NodeFetchBatchRawArgs) ReadField1(iprot thrift.TProtocol) error { + p.Req = &FetchBatchRawRequest{ + RangeTimeType: 0, + } if err := p.Req.Read(iprot); err != nil { return thrift.PrependError(fmt.Sprintf("%T error reading struct: ", p.Req), err) } return nil } -func (p *NodeWriteArgs) Write(oprot thrift.TProtocol) error { - if err := oprot.WriteStructBegin("write_args"); err != nil { +func (p *NodeFetchBatchRawArgs) Write(oprot thrift.TProtocol) error { + if err := oprot.WriteStructBegin("fetchBatchRaw_args"); err != nil { return thrift.PrependError(fmt.Sprintf("%T write struct begin error: ", p), err) } if p != nil { @@ -16286,7 +17744,7 @@ func (p *NodeWriteArgs) Write(oprot thrift.TProtocol) error { return nil } -func (p *NodeWriteArgs) writeField1(oprot thrift.TProtocol) (err error) { +func (p *NodeFetchBatchRawArgs) writeField1(oprot thrift.TProtocol) (err error) { if err := oprot.WriteFieldBegin("req", thrift.STRUCT, 1); err != nil { return thrift.PrependError(fmt.Sprintf("%T write field begin error 1:req: ", p), err) } @@ -16299,36 +17757,51 @@ func (p *NodeWriteArgs) writeField1(oprot thrift.TProtocol) (err error) { return err } -func (p *NodeWriteArgs) String() string { +func (p *NodeFetchBatchRawArgs) String() string { if p == nil { return "" } - return fmt.Sprintf("NodeWriteArgs(%+v)", *p) + return fmt.Sprintf("NodeFetchBatchRawArgs(%+v)", *p) } // Attributes: +// - Success // - Err -type NodeWriteResult struct { - Err *Error `thrift:"err,1" db:"err" json:"err,omitempty"` +type NodeFetchBatchRawResult struct { + Success *FetchBatchRawResult_ `thrift:"success,0" db:"success" json:"success,omitempty"` + Err *Error `thrift:"err,1" db:"err" json:"err,omitempty"` } -func NewNodeWriteResult() *NodeWriteResult { - return &NodeWriteResult{} +func NewNodeFetchBatchRawResult() *NodeFetchBatchRawResult { + return &NodeFetchBatchRawResult{} } -var NodeWriteResult_Err_DEFAULT *Error +var NodeFetchBatchRawResult_Success_DEFAULT *FetchBatchRawResult_ -func (p *NodeWriteResult) GetErr() *Error { +func (p *NodeFetchBatchRawResult) GetSuccess() *FetchBatchRawResult_ { + if !p.IsSetSuccess() { + return NodeFetchBatchRawResult_Success_DEFAULT + } + return p.Success +} + +var NodeFetchBatchRawResult_Err_DEFAULT *Error + +func (p *NodeFetchBatchRawResult) GetErr() *Error { if !p.IsSetErr() { - return NodeWriteResult_Err_DEFAULT + return NodeFetchBatchRawResult_Err_DEFAULT } return p.Err } -func (p *NodeWriteResult) IsSetErr() bool { +func (p *NodeFetchBatchRawResult) IsSetSuccess() bool { + return p.Success != nil +} + +func (p *NodeFetchBatchRawResult) IsSetErr() bool { return p.Err != nil } -func (p *NodeWriteResult) Read(iprot thrift.TProtocol) error { +func (p *NodeFetchBatchRawResult) Read(iprot thrift.TProtocol) error { if _, err := iprot.ReadStructBegin(); err != nil { return thrift.PrependError(fmt.Sprintf("%T read error: ", p), err) } @@ -16342,6 +17815,10 @@ func (p *NodeWriteResult) Read(iprot thrift.TProtocol) error { break } switch fieldId { + case 0: + if err := p.ReadField0(iprot); err != nil { + return err + } case 1: if err := p.ReadField1(iprot); err != nil { return err @@ -16361,7 +17838,15 @@ func (p *NodeWriteResult) Read(iprot thrift.TProtocol) error { return nil } -func (p *NodeWriteResult) ReadField1(iprot thrift.TProtocol) error { +func (p *NodeFetchBatchRawResult) ReadField0(iprot thrift.TProtocol) error { + p.Success = &FetchBatchRawResult_{} + if err := p.Success.Read(iprot); err != nil { + return thrift.PrependError(fmt.Sprintf("%T error reading struct: ", p.Success), err) + } + return nil +} + +func (p *NodeFetchBatchRawResult) ReadField1(iprot thrift.TProtocol) error { p.Err = &Error{ Type: 0, } @@ -16371,11 +17856,14 @@ func (p *NodeWriteResult) ReadField1(iprot thrift.TProtocol) error { return nil } -func (p *NodeWriteResult) Write(oprot thrift.TProtocol) error { - if err := oprot.WriteStructBegin("write_result"); err != nil { +func (p *NodeFetchBatchRawResult) Write(oprot thrift.TProtocol) error { + if err := oprot.WriteStructBegin("fetchBatchRaw_result"); err != nil { return thrift.PrependError(fmt.Sprintf("%T write struct begin error: ", p), err) } if p != nil { + if err := p.writeField0(oprot); err != nil { + return err + } if err := p.writeField1(oprot); err != nil { return err } @@ -16389,7 +17877,22 @@ func (p *NodeWriteResult) Write(oprot thrift.TProtocol) error { return nil } -func (p *NodeWriteResult) writeField1(oprot thrift.TProtocol) (err error) { +func (p *NodeFetchBatchRawResult) writeField0(oprot thrift.TProtocol) (err error) { + if p.IsSetSuccess() { + if err := oprot.WriteFieldBegin("success", thrift.STRUCT, 0); err != nil { + return thrift.PrependError(fmt.Sprintf("%T write field begin error 0:success: ", p), err) + } + if err := p.Success.Write(oprot); err != nil { + return thrift.PrependError(fmt.Sprintf("%T error writing struct: ", p.Success), err) + } + if err := oprot.WriteFieldEnd(); err != nil { + return thrift.PrependError(fmt.Sprintf("%T write field end error 0:success: ", p), err) + } + } + return err +} + +func (p *NodeFetchBatchRawResult) writeField1(oprot thrift.TProtocol) (err error) { if p.IsSetErr() { if err := oprot.WriteFieldBegin("err", thrift.STRUCT, 1); err != nil { return thrift.PrependError(fmt.Sprintf("%T write field begin error 1:err: ", p), err) @@ -16404,36 +17907,36 @@ func (p *NodeWriteResult) writeField1(oprot thrift.TProtocol) (err error) { return err } -func (p *NodeWriteResult) String() string { +func (p *NodeFetchBatchRawResult) String() string { if p == nil { return "" } - return fmt.Sprintf("NodeWriteResult(%+v)", *p) + return fmt.Sprintf("NodeFetchBatchRawResult(%+v)", *p) } // Attributes: // - Req -type NodeWriteTaggedArgs struct { - Req *WriteTaggedRequest `thrift:"req,1" db:"req" json:"req"` +type NodeFetchBlocksRawArgs struct { + Req *FetchBlocksRawRequest `thrift:"req,1" db:"req" json:"req"` } -func NewNodeWriteTaggedArgs() *NodeWriteTaggedArgs { - return &NodeWriteTaggedArgs{} +func NewNodeFetchBlocksRawArgs() *NodeFetchBlocksRawArgs { + return &NodeFetchBlocksRawArgs{} } -var NodeWriteTaggedArgs_Req_DEFAULT *WriteTaggedRequest +var NodeFetchBlocksRawArgs_Req_DEFAULT *FetchBlocksRawRequest -func (p *NodeWriteTaggedArgs) GetReq() *WriteTaggedRequest { +func (p *NodeFetchBlocksRawArgs) GetReq() *FetchBlocksRawRequest { if !p.IsSetReq() { - return NodeWriteTaggedArgs_Req_DEFAULT + return NodeFetchBlocksRawArgs_Req_DEFAULT } return p.Req } -func (p *NodeWriteTaggedArgs) IsSetReq() bool { +func (p *NodeFetchBlocksRawArgs) IsSetReq() bool { return p.Req != nil } -func (p *NodeWriteTaggedArgs) Read(iprot thrift.TProtocol) error { +func (p *NodeFetchBlocksRawArgs) Read(iprot thrift.TProtocol) error { if _, err := iprot.ReadStructBegin(); err != nil { return thrift.PrependError(fmt.Sprintf("%T read error: ", p), err) } @@ -16466,16 +17969,16 @@ func (p *NodeWriteTaggedArgs) Read(iprot thrift.TProtocol) error { return nil } -func (p *NodeWriteTaggedArgs) ReadField1(iprot thrift.TProtocol) error { - p.Req = &WriteTaggedRequest{} +func (p *NodeFetchBlocksRawArgs) ReadField1(iprot thrift.TProtocol) error { + p.Req = &FetchBlocksRawRequest{} if err := p.Req.Read(iprot); err != nil { return thrift.PrependError(fmt.Sprintf("%T error reading struct: ", p.Req), err) } return nil } -func (p *NodeWriteTaggedArgs) Write(oprot thrift.TProtocol) error { - if err := oprot.WriteStructBegin("writeTagged_args"); err != nil { +func (p *NodeFetchBlocksRawArgs) Write(oprot thrift.TProtocol) error { + if err := oprot.WriteStructBegin("fetchBlocksRaw_args"); err != nil { return thrift.PrependError(fmt.Sprintf("%T write struct begin error: ", p), err) } if p != nil { @@ -16492,7 +17995,7 @@ func (p *NodeWriteTaggedArgs) Write(oprot thrift.TProtocol) error { return nil } -func (p *NodeWriteTaggedArgs) writeField1(oprot thrift.TProtocol) (err error) { +func (p *NodeFetchBlocksRawArgs) writeField1(oprot thrift.TProtocol) (err error) { if err := oprot.WriteFieldBegin("req", thrift.STRUCT, 1); err != nil { return thrift.PrependError(fmt.Sprintf("%T write field begin error 1:req: ", p), err) } @@ -16505,36 +18008,51 @@ func (p *NodeWriteTaggedArgs) writeField1(oprot thrift.TProtocol) (err error) { return err } -func (p *NodeWriteTaggedArgs) String() string { +func (p *NodeFetchBlocksRawArgs) String() string { if p == nil { return "" } - return fmt.Sprintf("NodeWriteTaggedArgs(%+v)", *p) + return fmt.Sprintf("NodeFetchBlocksRawArgs(%+v)", *p) } // Attributes: +// - Success // - Err -type NodeWriteTaggedResult struct { - Err *Error `thrift:"err,1" db:"err" json:"err,omitempty"` +type NodeFetchBlocksRawResult struct { + Success *FetchBlocksRawResult_ `thrift:"success,0" db:"success" json:"success,omitempty"` + Err *Error `thrift:"err,1" db:"err" json:"err,omitempty"` } -func NewNodeWriteTaggedResult() *NodeWriteTaggedResult { - return &NodeWriteTaggedResult{} +func NewNodeFetchBlocksRawResult() *NodeFetchBlocksRawResult { + return &NodeFetchBlocksRawResult{} } -var NodeWriteTaggedResult_Err_DEFAULT *Error +var NodeFetchBlocksRawResult_Success_DEFAULT *FetchBlocksRawResult_ -func (p *NodeWriteTaggedResult) GetErr() *Error { +func (p *NodeFetchBlocksRawResult) GetSuccess() *FetchBlocksRawResult_ { + if !p.IsSetSuccess() { + return NodeFetchBlocksRawResult_Success_DEFAULT + } + return p.Success +} + +var NodeFetchBlocksRawResult_Err_DEFAULT *Error + +func (p *NodeFetchBlocksRawResult) GetErr() *Error { if !p.IsSetErr() { - return NodeWriteTaggedResult_Err_DEFAULT + return NodeFetchBlocksRawResult_Err_DEFAULT } return p.Err } -func (p *NodeWriteTaggedResult) IsSetErr() bool { +func (p *NodeFetchBlocksRawResult) IsSetSuccess() bool { + return p.Success != nil +} + +func (p *NodeFetchBlocksRawResult) IsSetErr() bool { return p.Err != nil } -func (p *NodeWriteTaggedResult) Read(iprot thrift.TProtocol) error { +func (p *NodeFetchBlocksRawResult) Read(iprot thrift.TProtocol) error { if _, err := iprot.ReadStructBegin(); err != nil { return thrift.PrependError(fmt.Sprintf("%T read error: ", p), err) } @@ -16548,6 +18066,10 @@ func (p *NodeWriteTaggedResult) Read(iprot thrift.TProtocol) error { break } switch fieldId { + case 0: + if err := p.ReadField0(iprot); err != nil { + return err + } case 1: if err := p.ReadField1(iprot); err != nil { return err @@ -16567,7 +18089,15 @@ func (p *NodeWriteTaggedResult) Read(iprot thrift.TProtocol) error { return nil } -func (p *NodeWriteTaggedResult) ReadField1(iprot thrift.TProtocol) error { +func (p *NodeFetchBlocksRawResult) ReadField0(iprot thrift.TProtocol) error { + p.Success = &FetchBlocksRawResult_{} + if err := p.Success.Read(iprot); err != nil { + return thrift.PrependError(fmt.Sprintf("%T error reading struct: ", p.Success), err) + } + return nil +} + +func (p *NodeFetchBlocksRawResult) ReadField1(iprot thrift.TProtocol) error { p.Err = &Error{ Type: 0, } @@ -16577,11 +18107,14 @@ func (p *NodeWriteTaggedResult) ReadField1(iprot thrift.TProtocol) error { return nil } -func (p *NodeWriteTaggedResult) Write(oprot thrift.TProtocol) error { - if err := oprot.WriteStructBegin("writeTagged_result"); err != nil { +func (p *NodeFetchBlocksRawResult) Write(oprot thrift.TProtocol) error { + if err := oprot.WriteStructBegin("fetchBlocksRaw_result"); err != nil { return thrift.PrependError(fmt.Sprintf("%T write struct begin error: ", p), err) } if p != nil { + if err := p.writeField0(oprot); err != nil { + return err + } if err := p.writeField1(oprot); err != nil { return err } @@ -16595,7 +18128,22 @@ func (p *NodeWriteTaggedResult) Write(oprot thrift.TProtocol) error { return nil } -func (p *NodeWriteTaggedResult) writeField1(oprot thrift.TProtocol) (err error) { +func (p *NodeFetchBlocksRawResult) writeField0(oprot thrift.TProtocol) (err error) { + if p.IsSetSuccess() { + if err := oprot.WriteFieldBegin("success", thrift.STRUCT, 0); err != nil { + return thrift.PrependError(fmt.Sprintf("%T write field begin error 0:success: ", p), err) + } + if err := p.Success.Write(oprot); err != nil { + return thrift.PrependError(fmt.Sprintf("%T error writing struct: ", p.Success), err) + } + if err := oprot.WriteFieldEnd(); err != nil { + return thrift.PrependError(fmt.Sprintf("%T write field end error 0:success: ", p), err) + } + } + return err +} + +func (p *NodeFetchBlocksRawResult) writeField1(oprot thrift.TProtocol) (err error) { if p.IsSetErr() { if err := oprot.WriteFieldBegin("err", thrift.STRUCT, 1); err != nil { return thrift.PrependError(fmt.Sprintf("%T write field begin error 1:err: ", p), err) @@ -16610,36 +18158,36 @@ func (p *NodeWriteTaggedResult) writeField1(oprot thrift.TProtocol) (err error) return err } -func (p *NodeWriteTaggedResult) String() string { +func (p *NodeFetchBlocksRawResult) String() string { if p == nil { return "" } - return fmt.Sprintf("NodeWriteTaggedResult(%+v)", *p) + return fmt.Sprintf("NodeFetchBlocksRawResult(%+v)", *p) } // Attributes: // - Req -type NodeFetchBatchRawArgs struct { - Req *FetchBatchRawRequest `thrift:"req,1" db:"req" json:"req"` +type NodeFetchBlocksMetadataRawV2Args struct { + Req *FetchBlocksMetadataRawV2Request `thrift:"req,1" db:"req" json:"req"` } -func NewNodeFetchBatchRawArgs() *NodeFetchBatchRawArgs { - return &NodeFetchBatchRawArgs{} +func NewNodeFetchBlocksMetadataRawV2Args() *NodeFetchBlocksMetadataRawV2Args { + return &NodeFetchBlocksMetadataRawV2Args{} } -var NodeFetchBatchRawArgs_Req_DEFAULT *FetchBatchRawRequest +var NodeFetchBlocksMetadataRawV2Args_Req_DEFAULT *FetchBlocksMetadataRawV2Request -func (p *NodeFetchBatchRawArgs) GetReq() *FetchBatchRawRequest { +func (p *NodeFetchBlocksMetadataRawV2Args) GetReq() *FetchBlocksMetadataRawV2Request { if !p.IsSetReq() { - return NodeFetchBatchRawArgs_Req_DEFAULT + return NodeFetchBlocksMetadataRawV2Args_Req_DEFAULT } return p.Req } -func (p *NodeFetchBatchRawArgs) IsSetReq() bool { +func (p *NodeFetchBlocksMetadataRawV2Args) IsSetReq() bool { return p.Req != nil } -func (p *NodeFetchBatchRawArgs) Read(iprot thrift.TProtocol) error { +func (p *NodeFetchBlocksMetadataRawV2Args) Read(iprot thrift.TProtocol) error { if _, err := iprot.ReadStructBegin(); err != nil { return thrift.PrependError(fmt.Sprintf("%T read error: ", p), err) } @@ -16672,18 +18220,16 @@ func (p *NodeFetchBatchRawArgs) Read(iprot thrift.TProtocol) error { return nil } -func (p *NodeFetchBatchRawArgs) ReadField1(iprot thrift.TProtocol) error { - p.Req = &FetchBatchRawRequest{ - RangeTimeType: 0, - } +func (p *NodeFetchBlocksMetadataRawV2Args) ReadField1(iprot thrift.TProtocol) error { + p.Req = &FetchBlocksMetadataRawV2Request{} if err := p.Req.Read(iprot); err != nil { return thrift.PrependError(fmt.Sprintf("%T error reading struct: ", p.Req), err) } return nil } -func (p *NodeFetchBatchRawArgs) Write(oprot thrift.TProtocol) error { - if err := oprot.WriteStructBegin("fetchBatchRaw_args"); err != nil { +func (p *NodeFetchBlocksMetadataRawV2Args) Write(oprot thrift.TProtocol) error { + if err := oprot.WriteStructBegin("fetchBlocksMetadataRawV2_args"); err != nil { return thrift.PrependError(fmt.Sprintf("%T write struct begin error: ", p), err) } if p != nil { @@ -16700,7 +18246,7 @@ func (p *NodeFetchBatchRawArgs) Write(oprot thrift.TProtocol) error { return nil } -func (p *NodeFetchBatchRawArgs) writeField1(oprot thrift.TProtocol) (err error) { +func (p *NodeFetchBlocksMetadataRawV2Args) writeField1(oprot thrift.TProtocol) (err error) { if err := oprot.WriteFieldBegin("req", thrift.STRUCT, 1); err != nil { return thrift.PrependError(fmt.Sprintf("%T write field begin error 1:req: ", p), err) } @@ -16713,51 +18259,51 @@ func (p *NodeFetchBatchRawArgs) writeField1(oprot thrift.TProtocol) (err error) return err } -func (p *NodeFetchBatchRawArgs) String() string { +func (p *NodeFetchBlocksMetadataRawV2Args) String() string { if p == nil { return "" } - return fmt.Sprintf("NodeFetchBatchRawArgs(%+v)", *p) + return fmt.Sprintf("NodeFetchBlocksMetadataRawV2Args(%+v)", *p) } // Attributes: // - Success // - Err -type NodeFetchBatchRawResult struct { - Success *FetchBatchRawResult_ `thrift:"success,0" db:"success" json:"success,omitempty"` - Err *Error `thrift:"err,1" db:"err" json:"err,omitempty"` +type NodeFetchBlocksMetadataRawV2Result struct { + Success *FetchBlocksMetadataRawV2Result_ `thrift:"success,0" db:"success" json:"success,omitempty"` + Err *Error `thrift:"err,1" db:"err" json:"err,omitempty"` } -func NewNodeFetchBatchRawResult() *NodeFetchBatchRawResult { - return &NodeFetchBatchRawResult{} +func NewNodeFetchBlocksMetadataRawV2Result() *NodeFetchBlocksMetadataRawV2Result { + return &NodeFetchBlocksMetadataRawV2Result{} } -var NodeFetchBatchRawResult_Success_DEFAULT *FetchBatchRawResult_ +var NodeFetchBlocksMetadataRawV2Result_Success_DEFAULT *FetchBlocksMetadataRawV2Result_ -func (p *NodeFetchBatchRawResult) GetSuccess() *FetchBatchRawResult_ { +func (p *NodeFetchBlocksMetadataRawV2Result) GetSuccess() *FetchBlocksMetadataRawV2Result_ { if !p.IsSetSuccess() { - return NodeFetchBatchRawResult_Success_DEFAULT + return NodeFetchBlocksMetadataRawV2Result_Success_DEFAULT } return p.Success } -var NodeFetchBatchRawResult_Err_DEFAULT *Error +var NodeFetchBlocksMetadataRawV2Result_Err_DEFAULT *Error -func (p *NodeFetchBatchRawResult) GetErr() *Error { +func (p *NodeFetchBlocksMetadataRawV2Result) GetErr() *Error { if !p.IsSetErr() { - return NodeFetchBatchRawResult_Err_DEFAULT + return NodeFetchBlocksMetadataRawV2Result_Err_DEFAULT } return p.Err } -func (p *NodeFetchBatchRawResult) IsSetSuccess() bool { +func (p *NodeFetchBlocksMetadataRawV2Result) IsSetSuccess() bool { return p.Success != nil } -func (p *NodeFetchBatchRawResult) IsSetErr() bool { +func (p *NodeFetchBlocksMetadataRawV2Result) IsSetErr() bool { return p.Err != nil } -func (p *NodeFetchBatchRawResult) Read(iprot thrift.TProtocol) error { +func (p *NodeFetchBlocksMetadataRawV2Result) Read(iprot thrift.TProtocol) error { if _, err := iprot.ReadStructBegin(); err != nil { return thrift.PrependError(fmt.Sprintf("%T read error: ", p), err) } @@ -16794,15 +18340,15 @@ func (p *NodeFetchBatchRawResult) Read(iprot thrift.TProtocol) error { return nil } -func (p *NodeFetchBatchRawResult) ReadField0(iprot thrift.TProtocol) error { - p.Success = &FetchBatchRawResult_{} +func (p *NodeFetchBlocksMetadataRawV2Result) ReadField0(iprot thrift.TProtocol) error { + p.Success = &FetchBlocksMetadataRawV2Result_{} if err := p.Success.Read(iprot); err != nil { return thrift.PrependError(fmt.Sprintf("%T error reading struct: ", p.Success), err) } return nil } -func (p *NodeFetchBatchRawResult) ReadField1(iprot thrift.TProtocol) error { +func (p *NodeFetchBlocksMetadataRawV2Result) ReadField1(iprot thrift.TProtocol) error { p.Err = &Error{ Type: 0, } @@ -16812,8 +18358,8 @@ func (p *NodeFetchBatchRawResult) ReadField1(iprot thrift.TProtocol) error { return nil } -func (p *NodeFetchBatchRawResult) Write(oprot thrift.TProtocol) error { - if err := oprot.WriteStructBegin("fetchBatchRaw_result"); err != nil { +func (p *NodeFetchBlocksMetadataRawV2Result) Write(oprot thrift.TProtocol) error { + if err := oprot.WriteStructBegin("fetchBlocksMetadataRawV2_result"); err != nil { return thrift.PrependError(fmt.Sprintf("%T write struct begin error: ", p), err) } if p != nil { @@ -16833,7 +18379,7 @@ func (p *NodeFetchBatchRawResult) Write(oprot thrift.TProtocol) error { return nil } -func (p *NodeFetchBatchRawResult) writeField0(oprot thrift.TProtocol) (err error) { +func (p *NodeFetchBlocksMetadataRawV2Result) writeField0(oprot thrift.TProtocol) (err error) { if p.IsSetSuccess() { if err := oprot.WriteFieldBegin("success", thrift.STRUCT, 0); err != nil { return thrift.PrependError(fmt.Sprintf("%T write field begin error 0:success: ", p), err) @@ -16848,7 +18394,7 @@ func (p *NodeFetchBatchRawResult) writeField0(oprot thrift.TProtocol) (err error return err } -func (p *NodeFetchBatchRawResult) writeField1(oprot thrift.TProtocol) (err error) { +func (p *NodeFetchBlocksMetadataRawV2Result) writeField1(oprot thrift.TProtocol) (err error) { if p.IsSetErr() { if err := oprot.WriteFieldBegin("err", thrift.STRUCT, 1); err != nil { return thrift.PrependError(fmt.Sprintf("%T write field begin error 1:err: ", p), err) @@ -16863,36 +18409,36 @@ func (p *NodeFetchBatchRawResult) writeField1(oprot thrift.TProtocol) (err error return err } -func (p *NodeFetchBatchRawResult) String() string { +func (p *NodeFetchBlocksMetadataRawV2Result) String() string { if p == nil { return "" } - return fmt.Sprintf("NodeFetchBatchRawResult(%+v)", *p) + return fmt.Sprintf("NodeFetchBlocksMetadataRawV2Result(%+v)", *p) } // Attributes: // - Req -type NodeFetchBlocksRawArgs struct { - Req *FetchBlocksRawRequest `thrift:"req,1" db:"req" json:"req"` +type NodeWriteBatchRawArgs struct { + Req *WriteBatchRawRequest `thrift:"req,1" db:"req" json:"req"` } -func NewNodeFetchBlocksRawArgs() *NodeFetchBlocksRawArgs { - return &NodeFetchBlocksRawArgs{} +func NewNodeWriteBatchRawArgs() *NodeWriteBatchRawArgs { + return &NodeWriteBatchRawArgs{} } -var NodeFetchBlocksRawArgs_Req_DEFAULT *FetchBlocksRawRequest +var NodeWriteBatchRawArgs_Req_DEFAULT *WriteBatchRawRequest -func (p *NodeFetchBlocksRawArgs) GetReq() *FetchBlocksRawRequest { +func (p *NodeWriteBatchRawArgs) GetReq() *WriteBatchRawRequest { if !p.IsSetReq() { - return NodeFetchBlocksRawArgs_Req_DEFAULT + return NodeWriteBatchRawArgs_Req_DEFAULT } return p.Req } -func (p *NodeFetchBlocksRawArgs) IsSetReq() bool { +func (p *NodeWriteBatchRawArgs) IsSetReq() bool { return p.Req != nil } -func (p *NodeFetchBlocksRawArgs) Read(iprot thrift.TProtocol) error { +func (p *NodeWriteBatchRawArgs) Read(iprot thrift.TProtocol) error { if _, err := iprot.ReadStructBegin(); err != nil { return thrift.PrependError(fmt.Sprintf("%T read error: ", p), err) } @@ -16925,16 +18471,16 @@ func (p *NodeFetchBlocksRawArgs) Read(iprot thrift.TProtocol) error { return nil } -func (p *NodeFetchBlocksRawArgs) ReadField1(iprot thrift.TProtocol) error { - p.Req = &FetchBlocksRawRequest{} +func (p *NodeWriteBatchRawArgs) ReadField1(iprot thrift.TProtocol) error { + p.Req = &WriteBatchRawRequest{} if err := p.Req.Read(iprot); err != nil { return thrift.PrependError(fmt.Sprintf("%T error reading struct: ", p.Req), err) } return nil } -func (p *NodeFetchBlocksRawArgs) Write(oprot thrift.TProtocol) error { - if err := oprot.WriteStructBegin("fetchBlocksRaw_args"); err != nil { +func (p *NodeWriteBatchRawArgs) Write(oprot thrift.TProtocol) error { + if err := oprot.WriteStructBegin("writeBatchRaw_args"); err != nil { return thrift.PrependError(fmt.Sprintf("%T write struct begin error: ", p), err) } if p != nil { @@ -16951,7 +18497,7 @@ func (p *NodeFetchBlocksRawArgs) Write(oprot thrift.TProtocol) error { return nil } -func (p *NodeFetchBlocksRawArgs) writeField1(oprot thrift.TProtocol) (err error) { +func (p *NodeWriteBatchRawArgs) writeField1(oprot thrift.TProtocol) (err error) { if err := oprot.WriteFieldBegin("req", thrift.STRUCT, 1); err != nil { return thrift.PrependError(fmt.Sprintf("%T write field begin error 1:req: ", p), err) } @@ -16964,51 +18510,36 @@ func (p *NodeFetchBlocksRawArgs) writeField1(oprot thrift.TProtocol) (err error) return err } -func (p *NodeFetchBlocksRawArgs) String() string { +func (p *NodeWriteBatchRawArgs) String() string { if p == nil { return "" } - return fmt.Sprintf("NodeFetchBlocksRawArgs(%+v)", *p) + return fmt.Sprintf("NodeWriteBatchRawArgs(%+v)", *p) } // Attributes: -// - Success // - Err -type NodeFetchBlocksRawResult struct { - Success *FetchBlocksRawResult_ `thrift:"success,0" db:"success" json:"success,omitempty"` - Err *Error `thrift:"err,1" db:"err" json:"err,omitempty"` -} - -func NewNodeFetchBlocksRawResult() *NodeFetchBlocksRawResult { - return &NodeFetchBlocksRawResult{} +type NodeWriteBatchRawResult struct { + Err *WriteBatchRawErrors `thrift:"err,1" db:"err" json:"err,omitempty"` } -var NodeFetchBlocksRawResult_Success_DEFAULT *FetchBlocksRawResult_ - -func (p *NodeFetchBlocksRawResult) GetSuccess() *FetchBlocksRawResult_ { - if !p.IsSetSuccess() { - return NodeFetchBlocksRawResult_Success_DEFAULT - } - return p.Success +func NewNodeWriteBatchRawResult() *NodeWriteBatchRawResult { + return &NodeWriteBatchRawResult{} } -var NodeFetchBlocksRawResult_Err_DEFAULT *Error +var NodeWriteBatchRawResult_Err_DEFAULT *WriteBatchRawErrors -func (p *NodeFetchBlocksRawResult) GetErr() *Error { +func (p *NodeWriteBatchRawResult) GetErr() *WriteBatchRawErrors { if !p.IsSetErr() { - return NodeFetchBlocksRawResult_Err_DEFAULT + return NodeWriteBatchRawResult_Err_DEFAULT } return p.Err } -func (p *NodeFetchBlocksRawResult) IsSetSuccess() bool { - return p.Success != nil -} - -func (p *NodeFetchBlocksRawResult) IsSetErr() bool { +func (p *NodeWriteBatchRawResult) IsSetErr() bool { return p.Err != nil } -func (p *NodeFetchBlocksRawResult) Read(iprot thrift.TProtocol) error { +func (p *NodeWriteBatchRawResult) Read(iprot thrift.TProtocol) error { if _, err := iprot.ReadStructBegin(); err != nil { return thrift.PrependError(fmt.Sprintf("%T read error: ", p), err) } @@ -17022,10 +18553,6 @@ func (p *NodeFetchBlocksRawResult) Read(iprot thrift.TProtocol) error { break } switch fieldId { - case 0: - if err := p.ReadField0(iprot); err != nil { - return err - } case 1: if err := p.ReadField1(iprot); err != nil { return err @@ -17045,32 +18572,19 @@ func (p *NodeFetchBlocksRawResult) Read(iprot thrift.TProtocol) error { return nil } -func (p *NodeFetchBlocksRawResult) ReadField0(iprot thrift.TProtocol) error { - p.Success = &FetchBlocksRawResult_{} - if err := p.Success.Read(iprot); err != nil { - return thrift.PrependError(fmt.Sprintf("%T error reading struct: ", p.Success), err) - } - return nil -} - -func (p *NodeFetchBlocksRawResult) ReadField1(iprot thrift.TProtocol) error { - p.Err = &Error{ - Type: 0, - } +func (p *NodeWriteBatchRawResult) ReadField1(iprot thrift.TProtocol) error { + p.Err = &WriteBatchRawErrors{} if err := p.Err.Read(iprot); err != nil { return thrift.PrependError(fmt.Sprintf("%T error reading struct: ", p.Err), err) } return nil } -func (p *NodeFetchBlocksRawResult) Write(oprot thrift.TProtocol) error { - if err := oprot.WriteStructBegin("fetchBlocksRaw_result"); err != nil { +func (p *NodeWriteBatchRawResult) Write(oprot thrift.TProtocol) error { + if err := oprot.WriteStructBegin("writeBatchRaw_result"); err != nil { return thrift.PrependError(fmt.Sprintf("%T write struct begin error: ", p), err) } if p != nil { - if err := p.writeField0(oprot); err != nil { - return err - } if err := p.writeField1(oprot); err != nil { return err } @@ -17084,22 +18598,7 @@ func (p *NodeFetchBlocksRawResult) Write(oprot thrift.TProtocol) error { return nil } -func (p *NodeFetchBlocksRawResult) writeField0(oprot thrift.TProtocol) (err error) { - if p.IsSetSuccess() { - if err := oprot.WriteFieldBegin("success", thrift.STRUCT, 0); err != nil { - return thrift.PrependError(fmt.Sprintf("%T write field begin error 0:success: ", p), err) - } - if err := p.Success.Write(oprot); err != nil { - return thrift.PrependError(fmt.Sprintf("%T error writing struct: ", p.Success), err) - } - if err := oprot.WriteFieldEnd(); err != nil { - return thrift.PrependError(fmt.Sprintf("%T write field end error 0:success: ", p), err) - } - } - return err -} - -func (p *NodeFetchBlocksRawResult) writeField1(oprot thrift.TProtocol) (err error) { +func (p *NodeWriteBatchRawResult) writeField1(oprot thrift.TProtocol) (err error) { if p.IsSetErr() { if err := oprot.WriteFieldBegin("err", thrift.STRUCT, 1); err != nil { return thrift.PrependError(fmt.Sprintf("%T write field begin error 1:err: ", p), err) @@ -17114,36 +18613,36 @@ func (p *NodeFetchBlocksRawResult) writeField1(oprot thrift.TProtocol) (err erro return err } -func (p *NodeFetchBlocksRawResult) String() string { +func (p *NodeWriteBatchRawResult) String() string { if p == nil { return "" } - return fmt.Sprintf("NodeFetchBlocksRawResult(%+v)", *p) + return fmt.Sprintf("NodeWriteBatchRawResult(%+v)", *p) } // Attributes: // - Req -type NodeFetchBlocksMetadataRawV2Args struct { - Req *FetchBlocksMetadataRawV2Request `thrift:"req,1" db:"req" json:"req"` +type NodeWriteBatchRawV2Args struct { + Req *WriteBatchRawV2Request `thrift:"req,1" db:"req" json:"req"` } -func NewNodeFetchBlocksMetadataRawV2Args() *NodeFetchBlocksMetadataRawV2Args { - return &NodeFetchBlocksMetadataRawV2Args{} +func NewNodeWriteBatchRawV2Args() *NodeWriteBatchRawV2Args { + return &NodeWriteBatchRawV2Args{} } -var NodeFetchBlocksMetadataRawV2Args_Req_DEFAULT *FetchBlocksMetadataRawV2Request +var NodeWriteBatchRawV2Args_Req_DEFAULT *WriteBatchRawV2Request -func (p *NodeFetchBlocksMetadataRawV2Args) GetReq() *FetchBlocksMetadataRawV2Request { +func (p *NodeWriteBatchRawV2Args) GetReq() *WriteBatchRawV2Request { if !p.IsSetReq() { - return NodeFetchBlocksMetadataRawV2Args_Req_DEFAULT + return NodeWriteBatchRawV2Args_Req_DEFAULT } return p.Req } -func (p *NodeFetchBlocksMetadataRawV2Args) IsSetReq() bool { +func (p *NodeWriteBatchRawV2Args) IsSetReq() bool { return p.Req != nil } -func (p *NodeFetchBlocksMetadataRawV2Args) Read(iprot thrift.TProtocol) error { +func (p *NodeWriteBatchRawV2Args) Read(iprot thrift.TProtocol) error { if _, err := iprot.ReadStructBegin(); err != nil { return thrift.PrependError(fmt.Sprintf("%T read error: ", p), err) } @@ -17176,16 +18675,16 @@ func (p *NodeFetchBlocksMetadataRawV2Args) Read(iprot thrift.TProtocol) error { return nil } -func (p *NodeFetchBlocksMetadataRawV2Args) ReadField1(iprot thrift.TProtocol) error { - p.Req = &FetchBlocksMetadataRawV2Request{} +func (p *NodeWriteBatchRawV2Args) ReadField1(iprot thrift.TProtocol) error { + p.Req = &WriteBatchRawV2Request{} if err := p.Req.Read(iprot); err != nil { return thrift.PrependError(fmt.Sprintf("%T error reading struct: ", p.Req), err) } return nil } -func (p *NodeFetchBlocksMetadataRawV2Args) Write(oprot thrift.TProtocol) error { - if err := oprot.WriteStructBegin("fetchBlocksMetadataRawV2_args"); err != nil { +func (p *NodeWriteBatchRawV2Args) Write(oprot thrift.TProtocol) error { + if err := oprot.WriteStructBegin("writeBatchRawV2_args"); err != nil { return thrift.PrependError(fmt.Sprintf("%T write struct begin error: ", p), err) } if p != nil { @@ -17202,7 +18701,7 @@ func (p *NodeFetchBlocksMetadataRawV2Args) Write(oprot thrift.TProtocol) error { return nil } -func (p *NodeFetchBlocksMetadataRawV2Args) writeField1(oprot thrift.TProtocol) (err error) { +func (p *NodeWriteBatchRawV2Args) writeField1(oprot thrift.TProtocol) (err error) { if err := oprot.WriteFieldBegin("req", thrift.STRUCT, 1); err != nil { return thrift.PrependError(fmt.Sprintf("%T write field begin error 1:req: ", p), err) } @@ -17215,51 +18714,36 @@ func (p *NodeFetchBlocksMetadataRawV2Args) writeField1(oprot thrift.TProtocol) ( return err } -func (p *NodeFetchBlocksMetadataRawV2Args) String() string { +func (p *NodeWriteBatchRawV2Args) String() string { if p == nil { return "" } - return fmt.Sprintf("NodeFetchBlocksMetadataRawV2Args(%+v)", *p) + return fmt.Sprintf("NodeWriteBatchRawV2Args(%+v)", *p) } // Attributes: -// - Success // - Err -type NodeFetchBlocksMetadataRawV2Result struct { - Success *FetchBlocksMetadataRawV2Result_ `thrift:"success,0" db:"success" json:"success,omitempty"` - Err *Error `thrift:"err,1" db:"err" json:"err,omitempty"` -} - -func NewNodeFetchBlocksMetadataRawV2Result() *NodeFetchBlocksMetadataRawV2Result { - return &NodeFetchBlocksMetadataRawV2Result{} +type NodeWriteBatchRawV2Result struct { + Err *WriteBatchRawErrors `thrift:"err,1" db:"err" json:"err,omitempty"` } -var NodeFetchBlocksMetadataRawV2Result_Success_DEFAULT *FetchBlocksMetadataRawV2Result_ - -func (p *NodeFetchBlocksMetadataRawV2Result) GetSuccess() *FetchBlocksMetadataRawV2Result_ { - if !p.IsSetSuccess() { - return NodeFetchBlocksMetadataRawV2Result_Success_DEFAULT - } - return p.Success +func NewNodeWriteBatchRawV2Result() *NodeWriteBatchRawV2Result { + return &NodeWriteBatchRawV2Result{} } -var NodeFetchBlocksMetadataRawV2Result_Err_DEFAULT *Error +var NodeWriteBatchRawV2Result_Err_DEFAULT *WriteBatchRawErrors -func (p *NodeFetchBlocksMetadataRawV2Result) GetErr() *Error { +func (p *NodeWriteBatchRawV2Result) GetErr() *WriteBatchRawErrors { if !p.IsSetErr() { - return NodeFetchBlocksMetadataRawV2Result_Err_DEFAULT + return NodeWriteBatchRawV2Result_Err_DEFAULT } return p.Err } -func (p *NodeFetchBlocksMetadataRawV2Result) IsSetSuccess() bool { - return p.Success != nil -} - -func (p *NodeFetchBlocksMetadataRawV2Result) IsSetErr() bool { +func (p *NodeWriteBatchRawV2Result) IsSetErr() bool { return p.Err != nil } -func (p *NodeFetchBlocksMetadataRawV2Result) Read(iprot thrift.TProtocol) error { +func (p *NodeWriteBatchRawV2Result) Read(iprot thrift.TProtocol) error { if _, err := iprot.ReadStructBegin(); err != nil { return thrift.PrependError(fmt.Sprintf("%T read error: ", p), err) } @@ -17273,10 +18757,6 @@ func (p *NodeFetchBlocksMetadataRawV2Result) Read(iprot thrift.TProtocol) error break } switch fieldId { - case 0: - if err := p.ReadField0(iprot); err != nil { - return err - } case 1: if err := p.ReadField1(iprot); err != nil { return err @@ -17296,32 +18776,19 @@ func (p *NodeFetchBlocksMetadataRawV2Result) Read(iprot thrift.TProtocol) error return nil } -func (p *NodeFetchBlocksMetadataRawV2Result) ReadField0(iprot thrift.TProtocol) error { - p.Success = &FetchBlocksMetadataRawV2Result_{} - if err := p.Success.Read(iprot); err != nil { - return thrift.PrependError(fmt.Sprintf("%T error reading struct: ", p.Success), err) - } - return nil -} - -func (p *NodeFetchBlocksMetadataRawV2Result) ReadField1(iprot thrift.TProtocol) error { - p.Err = &Error{ - Type: 0, - } +func (p *NodeWriteBatchRawV2Result) ReadField1(iprot thrift.TProtocol) error { + p.Err = &WriteBatchRawErrors{} if err := p.Err.Read(iprot); err != nil { return thrift.PrependError(fmt.Sprintf("%T error reading struct: ", p.Err), err) } return nil } -func (p *NodeFetchBlocksMetadataRawV2Result) Write(oprot thrift.TProtocol) error { - if err := oprot.WriteStructBegin("fetchBlocksMetadataRawV2_result"); err != nil { +func (p *NodeWriteBatchRawV2Result) Write(oprot thrift.TProtocol) error { + if err := oprot.WriteStructBegin("writeBatchRawV2_result"); err != nil { return thrift.PrependError(fmt.Sprintf("%T write struct begin error: ", p), err) } if p != nil { - if err := p.writeField0(oprot); err != nil { - return err - } if err := p.writeField1(oprot); err != nil { return err } @@ -17335,22 +18802,7 @@ func (p *NodeFetchBlocksMetadataRawV2Result) Write(oprot thrift.TProtocol) error return nil } -func (p *NodeFetchBlocksMetadataRawV2Result) writeField0(oprot thrift.TProtocol) (err error) { - if p.IsSetSuccess() { - if err := oprot.WriteFieldBegin("success", thrift.STRUCT, 0); err != nil { - return thrift.PrependError(fmt.Sprintf("%T write field begin error 0:success: ", p), err) - } - if err := p.Success.Write(oprot); err != nil { - return thrift.PrependError(fmt.Sprintf("%T error writing struct: ", p.Success), err) - } - if err := oprot.WriteFieldEnd(); err != nil { - return thrift.PrependError(fmt.Sprintf("%T write field end error 0:success: ", p), err) - } - } - return err -} - -func (p *NodeFetchBlocksMetadataRawV2Result) writeField1(oprot thrift.TProtocol) (err error) { +func (p *NodeWriteBatchRawV2Result) writeField1(oprot thrift.TProtocol) (err error) { if p.IsSetErr() { if err := oprot.WriteFieldBegin("err", thrift.STRUCT, 1); err != nil { return thrift.PrependError(fmt.Sprintf("%T write field begin error 1:err: ", p), err) @@ -17365,36 +18817,36 @@ func (p *NodeFetchBlocksMetadataRawV2Result) writeField1(oprot thrift.TProtocol) return err } -func (p *NodeFetchBlocksMetadataRawV2Result) String() string { +func (p *NodeWriteBatchRawV2Result) String() string { if p == nil { return "" } - return fmt.Sprintf("NodeFetchBlocksMetadataRawV2Result(%+v)", *p) + return fmt.Sprintf("NodeWriteBatchRawV2Result(%+v)", *p) } // Attributes: // - Req -type NodeWriteBatchRawArgs struct { - Req *WriteBatchRawRequest `thrift:"req,1" db:"req" json:"req"` +type NodeWriteTaggedBatchRawArgs struct { + Req *WriteTaggedBatchRawRequest `thrift:"req,1" db:"req" json:"req"` } -func NewNodeWriteBatchRawArgs() *NodeWriteBatchRawArgs { - return &NodeWriteBatchRawArgs{} +func NewNodeWriteTaggedBatchRawArgs() *NodeWriteTaggedBatchRawArgs { + return &NodeWriteTaggedBatchRawArgs{} } -var NodeWriteBatchRawArgs_Req_DEFAULT *WriteBatchRawRequest +var NodeWriteTaggedBatchRawArgs_Req_DEFAULT *WriteTaggedBatchRawRequest -func (p *NodeWriteBatchRawArgs) GetReq() *WriteBatchRawRequest { +func (p *NodeWriteTaggedBatchRawArgs) GetReq() *WriteTaggedBatchRawRequest { if !p.IsSetReq() { - return NodeWriteBatchRawArgs_Req_DEFAULT + return NodeWriteTaggedBatchRawArgs_Req_DEFAULT } return p.Req } -func (p *NodeWriteBatchRawArgs) IsSetReq() bool { +func (p *NodeWriteTaggedBatchRawArgs) IsSetReq() bool { return p.Req != nil } -func (p *NodeWriteBatchRawArgs) Read(iprot thrift.TProtocol) error { +func (p *NodeWriteTaggedBatchRawArgs) Read(iprot thrift.TProtocol) error { if _, err := iprot.ReadStructBegin(); err != nil { return thrift.PrependError(fmt.Sprintf("%T read error: ", p), err) } @@ -17427,16 +18879,16 @@ func (p *NodeWriteBatchRawArgs) Read(iprot thrift.TProtocol) error { return nil } -func (p *NodeWriteBatchRawArgs) ReadField1(iprot thrift.TProtocol) error { - p.Req = &WriteBatchRawRequest{} +func (p *NodeWriteTaggedBatchRawArgs) ReadField1(iprot thrift.TProtocol) error { + p.Req = &WriteTaggedBatchRawRequest{} if err := p.Req.Read(iprot); err != nil { return thrift.PrependError(fmt.Sprintf("%T error reading struct: ", p.Req), err) } return nil } -func (p *NodeWriteBatchRawArgs) Write(oprot thrift.TProtocol) error { - if err := oprot.WriteStructBegin("writeBatchRaw_args"); err != nil { +func (p *NodeWriteTaggedBatchRawArgs) Write(oprot thrift.TProtocol) error { + if err := oprot.WriteStructBegin("writeTaggedBatchRaw_args"); err != nil { return thrift.PrependError(fmt.Sprintf("%T write struct begin error: ", p), err) } if p != nil { @@ -17453,7 +18905,7 @@ func (p *NodeWriteBatchRawArgs) Write(oprot thrift.TProtocol) error { return nil } -func (p *NodeWriteBatchRawArgs) writeField1(oprot thrift.TProtocol) (err error) { +func (p *NodeWriteTaggedBatchRawArgs) writeField1(oprot thrift.TProtocol) (err error) { if err := oprot.WriteFieldBegin("req", thrift.STRUCT, 1); err != nil { return thrift.PrependError(fmt.Sprintf("%T write field begin error 1:req: ", p), err) } @@ -17466,36 +18918,36 @@ func (p *NodeWriteBatchRawArgs) writeField1(oprot thrift.TProtocol) (err error) return err } -func (p *NodeWriteBatchRawArgs) String() string { +func (p *NodeWriteTaggedBatchRawArgs) String() string { if p == nil { return "" } - return fmt.Sprintf("NodeWriteBatchRawArgs(%+v)", *p) + return fmt.Sprintf("NodeWriteTaggedBatchRawArgs(%+v)", *p) } // Attributes: // - Err -type NodeWriteBatchRawResult struct { +type NodeWriteTaggedBatchRawResult struct { Err *WriteBatchRawErrors `thrift:"err,1" db:"err" json:"err,omitempty"` } -func NewNodeWriteBatchRawResult() *NodeWriteBatchRawResult { - return &NodeWriteBatchRawResult{} +func NewNodeWriteTaggedBatchRawResult() *NodeWriteTaggedBatchRawResult { + return &NodeWriteTaggedBatchRawResult{} } -var NodeWriteBatchRawResult_Err_DEFAULT *WriteBatchRawErrors +var NodeWriteTaggedBatchRawResult_Err_DEFAULT *WriteBatchRawErrors -func (p *NodeWriteBatchRawResult) GetErr() *WriteBatchRawErrors { +func (p *NodeWriteTaggedBatchRawResult) GetErr() *WriteBatchRawErrors { if !p.IsSetErr() { - return NodeWriteBatchRawResult_Err_DEFAULT + return NodeWriteTaggedBatchRawResult_Err_DEFAULT } return p.Err } -func (p *NodeWriteBatchRawResult) IsSetErr() bool { +func (p *NodeWriteTaggedBatchRawResult) IsSetErr() bool { return p.Err != nil } -func (p *NodeWriteBatchRawResult) Read(iprot thrift.TProtocol) error { +func (p *NodeWriteTaggedBatchRawResult) Read(iprot thrift.TProtocol) error { if _, err := iprot.ReadStructBegin(); err != nil { return thrift.PrependError(fmt.Sprintf("%T read error: ", p), err) } @@ -17528,7 +18980,7 @@ func (p *NodeWriteBatchRawResult) Read(iprot thrift.TProtocol) error { return nil } -func (p *NodeWriteBatchRawResult) ReadField1(iprot thrift.TProtocol) error { +func (p *NodeWriteTaggedBatchRawResult) ReadField1(iprot thrift.TProtocol) error { p.Err = &WriteBatchRawErrors{} if err := p.Err.Read(iprot); err != nil { return thrift.PrependError(fmt.Sprintf("%T error reading struct: ", p.Err), err) @@ -17536,8 +18988,8 @@ func (p *NodeWriteBatchRawResult) ReadField1(iprot thrift.TProtocol) error { return nil } -func (p *NodeWriteBatchRawResult) Write(oprot thrift.TProtocol) error { - if err := oprot.WriteStructBegin("writeBatchRaw_result"); err != nil { +func (p *NodeWriteTaggedBatchRawResult) Write(oprot thrift.TProtocol) error { + if err := oprot.WriteStructBegin("writeTaggedBatchRaw_result"); err != nil { return thrift.PrependError(fmt.Sprintf("%T write struct begin error: ", p), err) } if p != nil { @@ -17554,7 +19006,7 @@ func (p *NodeWriteBatchRawResult) Write(oprot thrift.TProtocol) error { return nil } -func (p *NodeWriteBatchRawResult) writeField1(oprot thrift.TProtocol) (err error) { +func (p *NodeWriteTaggedBatchRawResult) writeField1(oprot thrift.TProtocol) (err error) { if p.IsSetErr() { if err := oprot.WriteFieldBegin("err", thrift.STRUCT, 1); err != nil { return thrift.PrependError(fmt.Sprintf("%T write field begin error 1:err: ", p), err) @@ -17569,36 +19021,36 @@ func (p *NodeWriteBatchRawResult) writeField1(oprot thrift.TProtocol) (err error return err } -func (p *NodeWriteBatchRawResult) String() string { +func (p *NodeWriteTaggedBatchRawResult) String() string { if p == nil { return "" } - return fmt.Sprintf("NodeWriteBatchRawResult(%+v)", *p) + return fmt.Sprintf("NodeWriteTaggedBatchRawResult(%+v)", *p) } // Attributes: // - Req -type NodeWriteTaggedBatchRawArgs struct { - Req *WriteTaggedBatchRawRequest `thrift:"req,1" db:"req" json:"req"` +type NodeWriteTaggedBatchRawV2Args struct { + Req *WriteTaggedBatchRawV2Request `thrift:"req,1" db:"req" json:"req"` } -func NewNodeWriteTaggedBatchRawArgs() *NodeWriteTaggedBatchRawArgs { - return &NodeWriteTaggedBatchRawArgs{} +func NewNodeWriteTaggedBatchRawV2Args() *NodeWriteTaggedBatchRawV2Args { + return &NodeWriteTaggedBatchRawV2Args{} } -var NodeWriteTaggedBatchRawArgs_Req_DEFAULT *WriteTaggedBatchRawRequest +var NodeWriteTaggedBatchRawV2Args_Req_DEFAULT *WriteTaggedBatchRawV2Request -func (p *NodeWriteTaggedBatchRawArgs) GetReq() *WriteTaggedBatchRawRequest { +func (p *NodeWriteTaggedBatchRawV2Args) GetReq() *WriteTaggedBatchRawV2Request { if !p.IsSetReq() { - return NodeWriteTaggedBatchRawArgs_Req_DEFAULT + return NodeWriteTaggedBatchRawV2Args_Req_DEFAULT } return p.Req } -func (p *NodeWriteTaggedBatchRawArgs) IsSetReq() bool { +func (p *NodeWriteTaggedBatchRawV2Args) IsSetReq() bool { return p.Req != nil } -func (p *NodeWriteTaggedBatchRawArgs) Read(iprot thrift.TProtocol) error { +func (p *NodeWriteTaggedBatchRawV2Args) Read(iprot thrift.TProtocol) error { if _, err := iprot.ReadStructBegin(); err != nil { return thrift.PrependError(fmt.Sprintf("%T read error: ", p), err) } @@ -17631,16 +19083,16 @@ func (p *NodeWriteTaggedBatchRawArgs) Read(iprot thrift.TProtocol) error { return nil } -func (p *NodeWriteTaggedBatchRawArgs) ReadField1(iprot thrift.TProtocol) error { - p.Req = &WriteTaggedBatchRawRequest{} +func (p *NodeWriteTaggedBatchRawV2Args) ReadField1(iprot thrift.TProtocol) error { + p.Req = &WriteTaggedBatchRawV2Request{} if err := p.Req.Read(iprot); err != nil { return thrift.PrependError(fmt.Sprintf("%T error reading struct: ", p.Req), err) } return nil } -func (p *NodeWriteTaggedBatchRawArgs) Write(oprot thrift.TProtocol) error { - if err := oprot.WriteStructBegin("writeTaggedBatchRaw_args"); err != nil { +func (p *NodeWriteTaggedBatchRawV2Args) Write(oprot thrift.TProtocol) error { + if err := oprot.WriteStructBegin("writeTaggedBatchRawV2_args"); err != nil { return thrift.PrependError(fmt.Sprintf("%T write struct begin error: ", p), err) } if p != nil { @@ -17657,7 +19109,7 @@ func (p *NodeWriteTaggedBatchRawArgs) Write(oprot thrift.TProtocol) error { return nil } -func (p *NodeWriteTaggedBatchRawArgs) writeField1(oprot thrift.TProtocol) (err error) { +func (p *NodeWriteTaggedBatchRawV2Args) writeField1(oprot thrift.TProtocol) (err error) { if err := oprot.WriteFieldBegin("req", thrift.STRUCT, 1); err != nil { return thrift.PrependError(fmt.Sprintf("%T write field begin error 1:req: ", p), err) } @@ -17670,36 +19122,36 @@ func (p *NodeWriteTaggedBatchRawArgs) writeField1(oprot thrift.TProtocol) (err e return err } -func (p *NodeWriteTaggedBatchRawArgs) String() string { +func (p *NodeWriteTaggedBatchRawV2Args) String() string { if p == nil { return "" } - return fmt.Sprintf("NodeWriteTaggedBatchRawArgs(%+v)", *p) + return fmt.Sprintf("NodeWriteTaggedBatchRawV2Args(%+v)", *p) } // Attributes: // - Err -type NodeWriteTaggedBatchRawResult struct { +type NodeWriteTaggedBatchRawV2Result struct { Err *WriteBatchRawErrors `thrift:"err,1" db:"err" json:"err,omitempty"` } -func NewNodeWriteTaggedBatchRawResult() *NodeWriteTaggedBatchRawResult { - return &NodeWriteTaggedBatchRawResult{} +func NewNodeWriteTaggedBatchRawV2Result() *NodeWriteTaggedBatchRawV2Result { + return &NodeWriteTaggedBatchRawV2Result{} } -var NodeWriteTaggedBatchRawResult_Err_DEFAULT *WriteBatchRawErrors +var NodeWriteTaggedBatchRawV2Result_Err_DEFAULT *WriteBatchRawErrors -func (p *NodeWriteTaggedBatchRawResult) GetErr() *WriteBatchRawErrors { +func (p *NodeWriteTaggedBatchRawV2Result) GetErr() *WriteBatchRawErrors { if !p.IsSetErr() { - return NodeWriteTaggedBatchRawResult_Err_DEFAULT + return NodeWriteTaggedBatchRawV2Result_Err_DEFAULT } return p.Err } -func (p *NodeWriteTaggedBatchRawResult) IsSetErr() bool { +func (p *NodeWriteTaggedBatchRawV2Result) IsSetErr() bool { return p.Err != nil } -func (p *NodeWriteTaggedBatchRawResult) Read(iprot thrift.TProtocol) error { +func (p *NodeWriteTaggedBatchRawV2Result) Read(iprot thrift.TProtocol) error { if _, err := iprot.ReadStructBegin(); err != nil { return thrift.PrependError(fmt.Sprintf("%T read error: ", p), err) } @@ -17732,7 +19184,7 @@ func (p *NodeWriteTaggedBatchRawResult) Read(iprot thrift.TProtocol) error { return nil } -func (p *NodeWriteTaggedBatchRawResult) ReadField1(iprot thrift.TProtocol) error { +func (p *NodeWriteTaggedBatchRawV2Result) ReadField1(iprot thrift.TProtocol) error { p.Err = &WriteBatchRawErrors{} if err := p.Err.Read(iprot); err != nil { return thrift.PrependError(fmt.Sprintf("%T error reading struct: ", p.Err), err) @@ -17740,8 +19192,8 @@ func (p *NodeWriteTaggedBatchRawResult) ReadField1(iprot thrift.TProtocol) error return nil } -func (p *NodeWriteTaggedBatchRawResult) Write(oprot thrift.TProtocol) error { - if err := oprot.WriteStructBegin("writeTaggedBatchRaw_result"); err != nil { +func (p *NodeWriteTaggedBatchRawV2Result) Write(oprot thrift.TProtocol) error { + if err := oprot.WriteStructBegin("writeTaggedBatchRawV2_result"); err != nil { return thrift.PrependError(fmt.Sprintf("%T write struct begin error: ", p), err) } if p != nil { @@ -17758,7 +19210,7 @@ func (p *NodeWriteTaggedBatchRawResult) Write(oprot thrift.TProtocol) error { return nil } -func (p *NodeWriteTaggedBatchRawResult) writeField1(oprot thrift.TProtocol) (err error) { +func (p *NodeWriteTaggedBatchRawV2Result) writeField1(oprot thrift.TProtocol) (err error) { if p.IsSetErr() { if err := oprot.WriteFieldBegin("err", thrift.STRUCT, 1); err != nil { return thrift.PrependError(fmt.Sprintf("%T write field begin error 1:err: ", p), err) @@ -17773,11 +19225,11 @@ func (p *NodeWriteTaggedBatchRawResult) writeField1(oprot thrift.TProtocol) (err return err } -func (p *NodeWriteTaggedBatchRawResult) String() string { +func (p *NodeWriteTaggedBatchRawV2Result) String() string { if p == nil { return "" } - return fmt.Sprintf("NodeWriteTaggedBatchRawResult(%+v)", *p) + return fmt.Sprintf("NodeWriteTaggedBatchRawV2Result(%+v)", *p) } type NodeRepairArgs struct { @@ -20726,16 +22178,16 @@ func (p *ClusterClient) recvHealth() (value *HealthResult_, err error) { return } if mTypeId == thrift.EXCEPTION { - error181 := thrift.NewTApplicationException(thrift.UNKNOWN_APPLICATION_EXCEPTION, "Unknown Exception") - var error182 error - error182, err = error181.Read(iprot) + error201 := thrift.NewTApplicationException(thrift.UNKNOWN_APPLICATION_EXCEPTION, "Unknown Exception") + var error202 error + error202, err = error201.Read(iprot) if err != nil { return } if err = iprot.ReadMessageEnd(); err != nil { return } - err = error182 + err = error202 return } if mTypeId != thrift.REPLY { @@ -20807,16 +22259,16 @@ func (p *ClusterClient) recvWrite() (err error) { return } if mTypeId == thrift.EXCEPTION { - error183 := thrift.NewTApplicationException(thrift.UNKNOWN_APPLICATION_EXCEPTION, "Unknown Exception") - var error184 error - error184, err = error183.Read(iprot) + error203 := thrift.NewTApplicationException(thrift.UNKNOWN_APPLICATION_EXCEPTION, "Unknown Exception") + var error204 error + error204, err = error203.Read(iprot) if err != nil { return } if err = iprot.ReadMessageEnd(); err != nil { return } - err = error184 + err = error204 return } if mTypeId != thrift.REPLY { @@ -20887,16 +22339,16 @@ func (p *ClusterClient) recvWriteTagged() (err error) { return } if mTypeId == thrift.EXCEPTION { - error185 := thrift.NewTApplicationException(thrift.UNKNOWN_APPLICATION_EXCEPTION, "Unknown Exception") - var error186 error - error186, err = error185.Read(iprot) + error205 := thrift.NewTApplicationException(thrift.UNKNOWN_APPLICATION_EXCEPTION, "Unknown Exception") + var error206 error + error206, err = error205.Read(iprot) if err != nil { return } if err = iprot.ReadMessageEnd(); err != nil { return } - err = error186 + err = error206 return } if mTypeId != thrift.REPLY { @@ -20967,16 +22419,16 @@ func (p *ClusterClient) recvQuery() (value *QueryResult_, err error) { return } if mTypeId == thrift.EXCEPTION { - error187 := thrift.NewTApplicationException(thrift.UNKNOWN_APPLICATION_EXCEPTION, "Unknown Exception") - var error188 error - error188, err = error187.Read(iprot) + error207 := thrift.NewTApplicationException(thrift.UNKNOWN_APPLICATION_EXCEPTION, "Unknown Exception") + var error208 error + error208, err = error207.Read(iprot) if err != nil { return } if err = iprot.ReadMessageEnd(); err != nil { return } - err = error188 + err = error208 return } if mTypeId != thrift.REPLY { @@ -21048,16 +22500,16 @@ func (p *ClusterClient) recvAggregate() (value *AggregateQueryResult_, err error return } if mTypeId == thrift.EXCEPTION { - error189 := thrift.NewTApplicationException(thrift.UNKNOWN_APPLICATION_EXCEPTION, "Unknown Exception") - var error190 error - error190, err = error189.Read(iprot) + error209 := thrift.NewTApplicationException(thrift.UNKNOWN_APPLICATION_EXCEPTION, "Unknown Exception") + var error210 error + error210, err = error209.Read(iprot) if err != nil { return } if err = iprot.ReadMessageEnd(); err != nil { return } - err = error190 + err = error210 return } if mTypeId != thrift.REPLY { @@ -21129,16 +22581,16 @@ func (p *ClusterClient) recvFetch() (value *FetchResult_, err error) { return } if mTypeId == thrift.EXCEPTION { - error191 := thrift.NewTApplicationException(thrift.UNKNOWN_APPLICATION_EXCEPTION, "Unknown Exception") - var error192 error - error192, err = error191.Read(iprot) + error211 := thrift.NewTApplicationException(thrift.UNKNOWN_APPLICATION_EXCEPTION, "Unknown Exception") + var error212 error + error212, err = error211.Read(iprot) if err != nil { return } if err = iprot.ReadMessageEnd(); err != nil { return } - err = error192 + err = error212 return } if mTypeId != thrift.REPLY { @@ -21210,16 +22662,16 @@ func (p *ClusterClient) recvTruncate() (value *TruncateResult_, err error) { return } if mTypeId == thrift.EXCEPTION { - error193 := thrift.NewTApplicationException(thrift.UNKNOWN_APPLICATION_EXCEPTION, "Unknown Exception") - var error194 error - error194, err = error193.Read(iprot) + error213 := thrift.NewTApplicationException(thrift.UNKNOWN_APPLICATION_EXCEPTION, "Unknown Exception") + var error214 error + error214, err = error213.Read(iprot) if err != nil { return } if err = iprot.ReadMessageEnd(); err != nil { return } - err = error194 + err = error214 return } if mTypeId != thrift.REPLY { @@ -21261,15 +22713,15 @@ func (p *ClusterProcessor) ProcessorMap() map[string]thrift.TProcessorFunction { func NewClusterProcessor(handler Cluster) *ClusterProcessor { - self195 := &ClusterProcessor{handler: handler, processorMap: make(map[string]thrift.TProcessorFunction)} - self195.processorMap["health"] = &clusterProcessorHealth{handler: handler} - self195.processorMap["write"] = &clusterProcessorWrite{handler: handler} - self195.processorMap["writeTagged"] = &clusterProcessorWriteTagged{handler: handler} - self195.processorMap["query"] = &clusterProcessorQuery{handler: handler} - self195.processorMap["aggregate"] = &clusterProcessorAggregate{handler: handler} - self195.processorMap["fetch"] = &clusterProcessorFetch{handler: handler} - self195.processorMap["truncate"] = &clusterProcessorTruncate{handler: handler} - return self195 + self215 := &ClusterProcessor{handler: handler, processorMap: make(map[string]thrift.TProcessorFunction)} + self215.processorMap["health"] = &clusterProcessorHealth{handler: handler} + self215.processorMap["write"] = &clusterProcessorWrite{handler: handler} + self215.processorMap["writeTagged"] = &clusterProcessorWriteTagged{handler: handler} + self215.processorMap["query"] = &clusterProcessorQuery{handler: handler} + self215.processorMap["aggregate"] = &clusterProcessorAggregate{handler: handler} + self215.processorMap["fetch"] = &clusterProcessorFetch{handler: handler} + self215.processorMap["truncate"] = &clusterProcessorTruncate{handler: handler} + return self215 } func (p *ClusterProcessor) Process(iprot, oprot thrift.TProtocol) (success bool, err thrift.TException) { @@ -21282,12 +22734,12 @@ func (p *ClusterProcessor) Process(iprot, oprot thrift.TProtocol) (success bool, } iprot.Skip(thrift.STRUCT) iprot.ReadMessageEnd() - x196 := thrift.NewTApplicationException(thrift.UNKNOWN_METHOD, "Unknown function "+name) + x216 := thrift.NewTApplicationException(thrift.UNKNOWN_METHOD, "Unknown function "+name) oprot.WriteMessageBegin(name, thrift.EXCEPTION, seqId) - x196.Write(oprot) + x216.Write(oprot) oprot.WriteMessageEnd() oprot.Flush() - return false, x196 + return false, x216 } diff --git a/src/dbnode/generated/thrift/rpc/rpc_mock.go b/src/dbnode/generated/thrift/rpc/rpc_mock.go index ee93f4b89f..68598ee7d1 100644 --- a/src/dbnode/generated/thrift/rpc/rpc_mock.go +++ b/src/dbnode/generated/thrift/rpc/rpc_mock.go @@ -522,6 +522,20 @@ func (mr *MockTChanNodeMockRecorder) WriteBatchRaw(ctx, req interface{}) *gomock return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "WriteBatchRaw", reflect.TypeOf((*MockTChanNode)(nil).WriteBatchRaw), ctx, req) } +// WriteBatchRawV2 mocks base method +func (m *MockTChanNode) WriteBatchRawV2(ctx thrift.Context, req *WriteBatchRawV2Request) error { + m.ctrl.T.Helper() + ret := m.ctrl.Call(m, "WriteBatchRawV2", ctx, req) + ret0, _ := ret[0].(error) + return ret0 +} + +// WriteBatchRawV2 indicates an expected call of WriteBatchRawV2 +func (mr *MockTChanNodeMockRecorder) WriteBatchRawV2(ctx, req interface{}) *gomock.Call { + mr.mock.ctrl.T.Helper() + return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "WriteBatchRawV2", reflect.TypeOf((*MockTChanNode)(nil).WriteBatchRawV2), ctx, req) +} + // WriteTagged mocks base method func (m *MockTChanNode) WriteTagged(ctx thrift.Context, req *WriteTaggedRequest) error { m.ctrl.T.Helper() @@ -549,3 +563,17 @@ func (mr *MockTChanNodeMockRecorder) WriteTaggedBatchRaw(ctx, req interface{}) * mr.mock.ctrl.T.Helper() return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "WriteTaggedBatchRaw", reflect.TypeOf((*MockTChanNode)(nil).WriteTaggedBatchRaw), ctx, req) } + +// WriteTaggedBatchRawV2 mocks base method +func (m *MockTChanNode) WriteTaggedBatchRawV2(ctx thrift.Context, req *WriteTaggedBatchRawV2Request) error { + m.ctrl.T.Helper() + ret := m.ctrl.Call(m, "WriteTaggedBatchRawV2", ctx, req) + ret0, _ := ret[0].(error) + return ret0 +} + +// WriteTaggedBatchRawV2 indicates an expected call of WriteTaggedBatchRawV2 +func (mr *MockTChanNodeMockRecorder) WriteTaggedBatchRawV2(ctx, req interface{}) *gomock.Call { + mr.mock.ctrl.T.Helper() + return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "WriteTaggedBatchRawV2", reflect.TypeOf((*MockTChanNode)(nil).WriteTaggedBatchRawV2), ctx, req) +} diff --git a/src/dbnode/generated/thrift/rpc/tchan-rpc.go b/src/dbnode/generated/thrift/rpc/tchan-rpc.go index 3f2ca658b2..d005481de5 100644 --- a/src/dbnode/generated/thrift/rpc/tchan-rpc.go +++ b/src/dbnode/generated/thrift/rpc/tchan-rpc.go @@ -68,8 +68,10 @@ type TChanNode interface { Truncate(ctx thrift.Context, req *TruncateRequest) (*TruncateResult_, error) Write(ctx thrift.Context, req *WriteRequest) error WriteBatchRaw(ctx thrift.Context, req *WriteBatchRawRequest) error + WriteBatchRawV2(ctx thrift.Context, req *WriteBatchRawV2Request) error WriteTagged(ctx thrift.Context, req *WriteTaggedRequest) error WriteTaggedBatchRaw(ctx thrift.Context, req *WriteTaggedBatchRawRequest) error + WriteTaggedBatchRawV2(ctx thrift.Context, req *WriteTaggedBatchRawV2Request) error } // Implementation of a client and service handler. @@ -874,6 +876,24 @@ func (c *tchanNodeClient) WriteBatchRaw(ctx thrift.Context, req *WriteBatchRawRe return err } +func (c *tchanNodeClient) WriteBatchRawV2(ctx thrift.Context, req *WriteBatchRawV2Request) error { + var resp NodeWriteBatchRawV2Result + args := NodeWriteBatchRawV2Args{ + Req: req, + } + success, err := c.client.Call(ctx, c.thriftService, "writeBatchRawV2", &args, &resp) + if err == nil && !success { + switch { + case resp.Err != nil: + err = resp.Err + default: + err = fmt.Errorf("received no result or unknown exception for writeBatchRawV2") + } + } + + return err +} + func (c *tchanNodeClient) WriteTagged(ctx thrift.Context, req *WriteTaggedRequest) error { var resp NodeWriteTaggedResult args := NodeWriteTaggedArgs{ @@ -910,6 +930,24 @@ func (c *tchanNodeClient) WriteTaggedBatchRaw(ctx thrift.Context, req *WriteTagg return err } +func (c *tchanNodeClient) WriteTaggedBatchRawV2(ctx thrift.Context, req *WriteTaggedBatchRawV2Request) error { + var resp NodeWriteTaggedBatchRawV2Result + args := NodeWriteTaggedBatchRawV2Args{ + Req: req, + } + success, err := c.client.Call(ctx, c.thriftService, "writeTaggedBatchRawV2", &args, &resp) + if err == nil && !success { + switch { + case resp.Err != nil: + err = resp.Err + default: + err = fmt.Errorf("received no result or unknown exception for writeTaggedBatchRawV2") + } + } + + return err +} + type tchanNodeServer struct { handler TChanNode } @@ -951,8 +989,10 @@ func (s *tchanNodeServer) Methods() []string { "truncate", "write", "writeBatchRaw", + "writeBatchRawV2", "writeTagged", "writeTaggedBatchRaw", + "writeTaggedBatchRawV2", } } @@ -1004,10 +1044,14 @@ func (s *tchanNodeServer) Handle(ctx thrift.Context, methodName string, protocol return s.handleWrite(ctx, protocol) case "writeBatchRaw": return s.handleWriteBatchRaw(ctx, protocol) + case "writeBatchRawV2": + return s.handleWriteBatchRawV2(ctx, protocol) case "writeTagged": return s.handleWriteTagged(ctx, protocol) case "writeTaggedBatchRaw": return s.handleWriteTaggedBatchRaw(ctx, protocol) + case "writeTaggedBatchRawV2": + return s.handleWriteTaggedBatchRawV2(ctx, protocol) default: return false, nil, fmt.Errorf("method %v not found in service %v", methodName, s.Service()) @@ -1655,6 +1699,33 @@ func (s *tchanNodeServer) handleWriteBatchRaw(ctx thrift.Context, protocol athri return err == nil, &res, nil } +func (s *tchanNodeServer) handleWriteBatchRawV2(ctx thrift.Context, protocol athrift.TProtocol) (bool, athrift.TStruct, error) { + var req NodeWriteBatchRawV2Args + var res NodeWriteBatchRawV2Result + + if err := req.Read(protocol); err != nil { + return false, nil, err + } + + err := + s.handler.WriteBatchRawV2(ctx, req.Req) + + if err != nil { + switch v := err.(type) { + case *WriteBatchRawErrors: + if v == nil { + return false, nil, fmt.Errorf("Handler for err returned non-nil error type *WriteBatchRawErrors but nil value") + } + res.Err = v + default: + return false, nil, err + } + } else { + } + + return err == nil, &res, nil +} + func (s *tchanNodeServer) handleWriteTagged(ctx thrift.Context, protocol athrift.TProtocol) (bool, athrift.TStruct, error) { var req NodeWriteTaggedArgs var res NodeWriteTaggedResult @@ -1708,3 +1779,30 @@ func (s *tchanNodeServer) handleWriteTaggedBatchRaw(ctx thrift.Context, protocol return err == nil, &res, nil } + +func (s *tchanNodeServer) handleWriteTaggedBatchRawV2(ctx thrift.Context, protocol athrift.TProtocol) (bool, athrift.TStruct, error) { + var req NodeWriteTaggedBatchRawV2Args + var res NodeWriteTaggedBatchRawV2Result + + if err := req.Read(protocol); err != nil { + return false, nil, err + } + + err := + s.handler.WriteTaggedBatchRawV2(ctx, req.Req) + + if err != nil { + switch v := err.(type) { + case *WriteBatchRawErrors: + if v == nil { + return false, nil, fmt.Errorf("Handler for err returned non-nil error type *WriteBatchRawErrors but nil value") + } + res.Err = v + default: + return false, nil, err + } + } else { + } + + return err == nil, &res, nil +} diff --git a/src/dbnode/network/server/tchannelthrift/node/service.go b/src/dbnode/network/server/tchannelthrift/node/service.go index 16431da03a..a528ae5f77 100644 --- a/src/dbnode/network/server/tchannelthrift/node/service.go +++ b/src/dbnode/network/server/tchannelthrift/node/service.go @@ -23,6 +23,7 @@ package node import ( "errors" "fmt" + "sort" "sync" "time" @@ -1228,6 +1229,121 @@ func (s *service) WriteBatchRaw(tctx thrift.Context, req *rpc.WriteBatchRawReque return nil } +func (s *service) WriteBatchRawV2(tctx thrift.Context, req *rpc.WriteBatchRawV2Request) error { + s.metrics.writeBatchRawRPCs.Inc(1) + db, err := s.startWriteRPCWithDB() + if err != nil { + return err + } + defer s.writeRPCCompleted() + + callStart := s.nowFn() + ctx := tchannelthrift.Context(tctx) + + // Sanity check input. + numNamespaces := int64(len(req.NameSpaces)) + for _, elem := range req.Elements { + if elem.NameSpace >= numNamespaces { + return fmt.Errorf("namespace index: %d is out of range of provided namespaces", elem.NameSpace) + } + } + + // Sort the elements so that they're sorted by namespace so we can reuse the same batch writer. + sort.Slice(req.Elements, func(i, j int) bool { + return req.Elements[i].NameSpace < req.Elements[j].NameSpace + }) + + // NB(r): Use the pooled request tracking to return thrift alloc'd bytes + // to the thrift bytes pool and to return ident.ID wrappers to a pool for + // reuse. We also reduce contention on pools by getting one per batch request + // rather than one per ID. + pooledReq := s.pools.writeBatchPooledReqPool.Get() + pooledReq.writeV2Req = req + ctx.RegisterFinalizer(pooledReq) + + var ( + nsID ident.ID + nsIdx int64 + batchWriter ts.BatchWriter + + retryableErrors int + nonRetryableErrors int + ) + for i, elem := range req.Elements { + if nsID == nil || elem.NameSpace != nsIdx { + if batchWriter != nil { + err = db.WriteBatch(ctx, nsID, batchWriter.(ts.WriteBatch), pooledReq) + if err != nil { + return convert.ToRPCError(err) + } + batchWriter = nil + } + + nsID = s.newPooledID(ctx, req.NameSpaces[elem.NameSpace], pooledReq) + nsIdx = elem.NameSpace + + batchWriter, err = db.BatchWriter(nsID, len(req.Elements)) + if err != nil { + return convert.ToRPCError(err) + } + // The lifecycle of the annotations is more involved than the rest of the data + // so we set the annotation pool put method as the finalization function and + // let the database take care of returning them to the pool. + batchWriter.SetFinalizeAnnotationFn(finalizeAnnotationFn) + } + + unit, unitErr := convert.ToUnit(elem.Datapoint.TimestampTimeType) + if unitErr != nil { + nonRetryableErrors++ + pooledReq.addError(tterrors.NewBadRequestWriteBatchRawError(i, unitErr)) + continue + } + + d, err := unit.Value() + if err != nil { + nonRetryableErrors++ + pooledReq.addError(tterrors.NewBadRequestWriteBatchRawError(i, err)) + continue + } + + seriesID := s.newPooledID(ctx, elem.ID, pooledReq) + batchWriter.Add( + i, + seriesID, + xtime.FromNormalizedTime(elem.Datapoint.Timestamp, d), + elem.Datapoint.Value, + unit, + elem.Datapoint.Annotation, + ) + } + + if batchWriter != nil { + // Write the last batch. + err = db.WriteBatch(ctx, nsID, batchWriter.(ts.WriteBatch), pooledReq) + if err != nil { + return convert.ToRPCError(err) + } + } + + nonRetryableErrors += pooledReq.numNonRetryableErrors() + retryableErrors += pooledReq.numRetryableErrors() + totalErrors := nonRetryableErrors + retryableErrors + + s.metrics.writeBatchRaw.ReportSuccess(len(req.Elements) - totalErrors) + s.metrics.writeBatchRaw.ReportRetryableErrors(retryableErrors) + s.metrics.writeBatchRaw.ReportNonRetryableErrors(nonRetryableErrors) + s.metrics.writeBatchRaw.ReportLatency(s.nowFn().Sub(callStart)) + + errs := pooledReq.writeBatchRawErrors() + if len(errs) > 0 { + batchErrs := rpc.NewWriteBatchRawErrors() + batchErrs.Errors = errs + return batchErrs + } + + return nil +} + func (s *service) WriteTaggedBatchRaw(tctx thrift.Context, req *rpc.WriteTaggedBatchRawRequest) error { s.metrics.writeTaggedBatchRawRPCs.Inc(1) db, err := s.startWriteRPCWithDB() @@ -1323,6 +1439,131 @@ func (s *service) WriteTaggedBatchRaw(tctx thrift.Context, req *rpc.WriteTaggedB return nil } +func (s *service) WriteTaggedBatchRawV2(tctx thrift.Context, req *rpc.WriteTaggedBatchRawV2Request) error { + s.metrics.writeBatchRawRPCs.Inc(1) + db, err := s.startWriteRPCWithDB() + if err != nil { + return err + } + defer s.writeRPCCompleted() + + callStart := s.nowFn() + ctx := tchannelthrift.Context(tctx) + + // Sanity check input. + numNamespaces := int64(len(req.NameSpaces)) + for _, elem := range req.Elements { + if elem.NameSpace >= numNamespaces { + return fmt.Errorf("namespace index: %d is out of range of provided namespaces", elem.NameSpace) + } + } + + // Sort the elements so that they're sorted by namespace so we can reuse the same batch writer. + sort.Slice(req.Elements, func(i, j int) bool { + return req.Elements[i].NameSpace < req.Elements[j].NameSpace + }) + + // NB(r): Use the pooled request tracking to return thrift alloc'd bytes + // to the thrift bytes pool and to return ident.ID wrappers to a pool for + // reuse. We also reduce contention on pools by getting one per batch request + // rather than one per ID. + pooledReq := s.pools.writeBatchPooledReqPool.Get() + pooledReq.writeTaggedV2Req = req + ctx.RegisterFinalizer(pooledReq) + + var ( + nsID ident.ID + nsIdx int64 + batchWriter ts.BatchWriter + + retryableErrors int + nonRetryableErrors int + ) + for i, elem := range req.Elements { + if nsID == nil || elem.NameSpace != nsIdx { + if batchWriter != nil { + err = db.WriteTaggedBatch(ctx, nsID, batchWriter.(ts.WriteBatch), pooledReq) + if err != nil { + return convert.ToRPCError(err) + } + batchWriter = nil + } + + nsID = s.newPooledID(ctx, req.NameSpaces[elem.NameSpace], pooledReq) + nsIdx = elem.NameSpace + + batchWriter, err = db.BatchWriter(nsID, len(req.Elements)) + if err != nil { + return convert.ToRPCError(err) + } + // The lifecycle of the encoded tags and annotations is more involved than the + // rest of the data so we set the annotation pool put method as the finalization + // function and let the database take care of returning them to the pool. + batchWriter.SetFinalizeEncodedTagsFn(finalizeEncodedTagsFn) + batchWriter.SetFinalizeAnnotationFn(finalizeAnnotationFn) + } + + unit, unitErr := convert.ToUnit(elem.Datapoint.TimestampTimeType) + if unitErr != nil { + nonRetryableErrors++ + pooledReq.addError(tterrors.NewBadRequestWriteBatchRawError(i, unitErr)) + continue + } + + d, err := unit.Value() + if err != nil { + nonRetryableErrors++ + pooledReq.addError(tterrors.NewBadRequestWriteBatchRawError(i, err)) + continue + } + + dec, err := s.newPooledTagsDecoder(ctx, elem.EncodedTags, pooledReq) + if err != nil { + nonRetryableErrors++ + pooledReq.addError(tterrors.NewBadRequestWriteBatchRawError(i, err)) + continue + } + + seriesID := s.newPooledID(ctx, elem.ID, pooledReq) + batchWriter.AddTagged( + i, + seriesID, + dec, + elem.EncodedTags, + xtime.FromNormalizedTime(elem.Datapoint.Timestamp, d), + elem.Datapoint.Value, + unit, + elem.Datapoint.Annotation, + ) + } + + if batchWriter != nil { + // Write the last batch. + err = db.WriteTaggedBatch(ctx, nsID, batchWriter.(ts.WriteBatch), pooledReq) + if err != nil { + return convert.ToRPCError(err) + } + } + + nonRetryableErrors += pooledReq.numNonRetryableErrors() + retryableErrors += pooledReq.numRetryableErrors() + totalErrors := nonRetryableErrors + retryableErrors + + s.metrics.writeBatchRaw.ReportSuccess(len(req.Elements) - totalErrors) + s.metrics.writeBatchRaw.ReportRetryableErrors(retryableErrors) + s.metrics.writeBatchRaw.ReportNonRetryableErrors(nonRetryableErrors) + s.metrics.writeBatchRaw.ReportLatency(s.nowFn().Sub(callStart)) + + errs := pooledReq.writeBatchRawErrors() + if len(errs) > 0 { + batchErrs := rpc.NewWriteBatchRawErrors() + batchErrs.Errors = errs + return batchErrs + } + + return nil +} + func (s *service) Repair(tctx thrift.Context) error { db, err := s.startRPCWithDB() if err != nil { @@ -1706,10 +1947,12 @@ func (c closeableMetadataV2Result) Finalize() { } type writeBatchPooledReq struct { - pooledIDs []writeBatchPooledReqID - pooledIDsUsed int - writeReq *rpc.WriteBatchRawRequest - writeTaggedReq *rpc.WriteTaggedBatchRawRequest + pooledIDs []writeBatchPooledReqID + pooledIDsUsed int + writeReq *rpc.WriteBatchRawRequest + writeV2Req *rpc.WriteBatchRawV2Request + writeTaggedReq *rpc.WriteTaggedBatchRawRequest + writeTaggedV2Req *rpc.WriteTaggedBatchRawV2Request // We want to avoid allocating an intermediary slice of []error so we // just include all the error handling in this struct for performance @@ -1775,6 +2018,16 @@ func (r *writeBatchPooledReq) Finalize() { } r.writeReq = nil } + if r.writeV2Req != nil { + for _, elem := range r.writeV2Req.Elements { + apachethrift.BytesPoolPut(elem.ID) + // Ownership of the annotations has been transferred to the BatchWriter + // so they will get returned the pool automatically by the commitlog once + // it finishes writing them to disk via the finalization function that + // gets set on the WriteBatch. + } + r.writeV2Req = nil + } if r.writeTaggedReq != nil { for _, elem := range r.writeTaggedReq.Elements { apachethrift.BytesPoolPut(elem.ID) @@ -1787,6 +2040,18 @@ func (r *writeBatchPooledReq) Finalize() { } r.writeTaggedReq = nil } + if r.writeTaggedV2Req != nil { + for _, elem := range r.writeTaggedV2Req.Elements { + apachethrift.BytesPoolPut(elem.ID) + // Ownership of the encoded tags has been transferred to the BatchWriter + // so they will get returned the pool automatically by the commitlog once + // it finishes writing them to disk via the finalization function that + // gets set on the WriteBatch. + + // See comment above about not finalizing annotations here. + } + r.writeTaggedV2Req = nil + } r.nonRetryableErrors = 0 r.retryableErrors = 0 diff --git a/src/dbnode/network/server/tchannelthrift/node/service_test.go b/src/dbnode/network/server/tchannelthrift/node/service_test.go index 53790e5cf3..481d36f520 100644 --- a/src/dbnode/network/server/tchannelthrift/node/service_test.go +++ b/src/dbnode/network/server/tchannelthrift/node/service_test.go @@ -2049,6 +2049,129 @@ func TestServiceWriteBatchRaw(t *testing.T) { require.NoError(t, err) } +func TestServiceWriteBatchRawV2SingleNS(t *testing.T) { + ctrl := gomock.NewController(t) + defer ctrl.Finish() + + mockDB := storage.NewMockDatabase(ctrl) + mockDB.EXPECT().Options().Return(testStorageOpts).AnyTimes() + + service := NewService(mockDB, testTChannelThriftOptions).(*service) + + tctx, _ := tchannelthrift.NewContext(time.Minute) + ctx := tchannelthrift.Context(tctx) + defer ctx.Close() + + nsID := "metrics" + + values := []struct { + id string + t time.Time + v float64 + }{ + {"foo", time.Now().Truncate(time.Second), 12.34}, + {"bar", time.Now().Truncate(time.Second), 42.42}, + } + + writeBatch := ts.NewWriteBatch(len(values), ident.StringID(nsID), nil) + mockDB.EXPECT(). + BatchWriter(ident.NewIDMatcher(nsID), len(values)). + Return(writeBatch, nil) + + mockDB.EXPECT(). + WriteBatch(ctx, ident.NewIDMatcher(nsID), writeBatch, gomock.Any()). + Return(nil) + + var elements []*rpc.WriteBatchRawV2RequestElement + for _, w := range values { + elem := &rpc.WriteBatchRawV2RequestElement{ + NameSpace: 0, + ID: []byte(w.id), + Datapoint: &rpc.Datapoint{ + Timestamp: w.t.Unix(), + TimestampTimeType: rpc.TimeType_UNIX_SECONDS, + Value: w.v, + }, + } + elements = append(elements, elem) + } + + mockDB.EXPECT().IsOverloaded().Return(false) + err := service.WriteBatchRawV2(tctx, &rpc.WriteBatchRawV2Request{ + NameSpaces: [][]byte{[]byte(nsID)}, + Elements: elements, + }) + require.NoError(t, err) +} + +func TestServiceWriteBatchRawV2MultiNS(t *testing.T) { + ctrl := gomock.NewController(t) + defer ctrl.Finish() + + mockDB := storage.NewMockDatabase(ctrl) + mockDB.EXPECT().Options().Return(testStorageOpts).AnyTimes() + + service := NewService(mockDB, testTChannelThriftOptions).(*service) + + tctx, _ := tchannelthrift.NewContext(time.Minute) + ctx := tchannelthrift.Context(tctx) + defer ctx.Close() + + var ( + nsID1 = "metrics" + nsID2 = "more-metrics" + + values = []struct { + id string + t time.Time + v float64 + }{ + {"foo", time.Now().Truncate(time.Second), 12.34}, + {"bar", time.Now().Truncate(time.Second), 42.42}, + } + + writeBatch1 = ts.NewWriteBatch(len(values), ident.StringID(nsID1), nil) + writeBatch2 = ts.NewWriteBatch(len(values), ident.StringID(nsID2), nil) + ) + + mockDB.EXPECT(). + BatchWriter(ident.NewIDMatcher(nsID1), len(values)*2). + Return(writeBatch1, nil) + mockDB.EXPECT(). + BatchWriter(ident.NewIDMatcher(nsID2), len(values)*2). + Return(writeBatch2, nil) + + mockDB.EXPECT(). + WriteBatch(ctx, ident.NewIDMatcher(nsID1), writeBatch1, gomock.Any()). + Return(nil) + mockDB.EXPECT(). + WriteBatch(ctx, ident.NewIDMatcher(nsID2), writeBatch2, gomock.Any()). + Return(nil) + + var elements []*rpc.WriteBatchRawV2RequestElement + for nsIdx := range []string{nsID1, nsID2} { + for _, w := range values { + elem := &rpc.WriteBatchRawV2RequestElement{ + NameSpace: int64(nsIdx), + ID: []byte(w.id), + Datapoint: &rpc.Datapoint{ + Timestamp: w.t.Unix(), + TimestampTimeType: rpc.TimeType_UNIX_SECONDS, + Value: w.v, + }, + } + elements = append(elements, elem) + } + } + + mockDB.EXPECT().IsOverloaded().Return(false) + err := service.WriteBatchRawV2(tctx, &rpc.WriteBatchRawV2Request{ + NameSpaces: [][]byte{[]byte(nsID1), []byte(nsID2)}, + Elements: elements, + }) + require.NoError(t, err) +} + func TestServiceWriteBatchRawOverloaded(t *testing.T) { ctrl := gomock.NewController(t) defer ctrl.Finish() @@ -2240,6 +2363,151 @@ func TestServiceWriteTaggedBatchRaw(t *testing.T) { require.NoError(t, err) } +func TestServiceWriteTaggedBatchRawV2(t *testing.T) { + ctrl := gomock.NewController(t) + defer ctrl.Finish() + + mockDB := storage.NewMockDatabase(ctrl) + mockDB.EXPECT().Options().Return(testStorageOpts).AnyTimes() + + mockDecoder := serialize.NewMockTagDecoder(ctrl) + mockDecoder.EXPECT().Reset(gomock.Any()).AnyTimes() + mockDecoder.EXPECT().Err().Return(nil).AnyTimes() + mockDecoder.EXPECT().Close().AnyTimes() + mockDecoderPool := serialize.NewMockTagDecoderPool(ctrl) + mockDecoderPool.EXPECT().Get().Return(mockDecoder).AnyTimes() + + opts := tchannelthrift.NewOptions(). + SetTagDecoderPool(mockDecoderPool) + + service := NewService(mockDB, opts).(*service) + + tctx, _ := tchannelthrift.NewContext(time.Minute) + ctx := tchannelthrift.Context(tctx) + defer ctx.Close() + + nsID := "metrics" + + values := []struct { + id string + tagEncode string + t time.Time + v float64 + }{ + {"foo", "a|b", time.Now().Truncate(time.Second), 12.34}, + {"bar", "c|dd", time.Now().Truncate(time.Second), 42.42}, + } + + writeBatch := ts.NewWriteBatch(len(values), ident.StringID(nsID), nil) + mockDB.EXPECT(). + BatchWriter(ident.NewIDMatcher(nsID), len(values)). + Return(writeBatch, nil) + + mockDB.EXPECT(). + WriteTaggedBatch(ctx, ident.NewIDMatcher(nsID), writeBatch, gomock.Any()). + Return(nil) + + var elements []*rpc.WriteTaggedBatchRawV2RequestElement + for _, w := range values { + elem := &rpc.WriteTaggedBatchRawV2RequestElement{ + NameSpace: 0, + ID: []byte(w.id), + EncodedTags: []byte(w.tagEncode), + Datapoint: &rpc.Datapoint{ + Timestamp: w.t.Unix(), + TimestampTimeType: rpc.TimeType_UNIX_SECONDS, + Value: w.v, + }, + } + elements = append(elements, elem) + } + + mockDB.EXPECT().IsOverloaded().Return(false) + err := service.WriteTaggedBatchRawV2(tctx, &rpc.WriteTaggedBatchRawV2Request{ + NameSpaces: [][]byte{[]byte(nsID)}, + Elements: elements, + }) + require.NoError(t, err) +} + +func TestServiceWriteTaggedBatchRawV2MultiNS(t *testing.T) { + ctrl := gomock.NewController(t) + defer ctrl.Finish() + + mockDB := storage.NewMockDatabase(ctrl) + mockDB.EXPECT().Options().Return(testStorageOpts).AnyTimes() + + mockDecoder := serialize.NewMockTagDecoder(ctrl) + mockDecoder.EXPECT().Reset(gomock.Any()).AnyTimes() + mockDecoder.EXPECT().Err().Return(nil).AnyTimes() + mockDecoder.EXPECT().Close().AnyTimes() + mockDecoderPool := serialize.NewMockTagDecoderPool(ctrl) + mockDecoderPool.EXPECT().Get().Return(mockDecoder).AnyTimes() + + opts := tchannelthrift.NewOptions(). + SetTagDecoderPool(mockDecoderPool) + + service := NewService(mockDB, opts).(*service) + + tctx, _ := tchannelthrift.NewContext(time.Minute) + ctx := tchannelthrift.Context(tctx) + defer ctx.Close() + + var ( + nsID1 = "metrics" + nsID2 = "more-metrics" + values = []struct { + id string + tagEncode string + t time.Time + v float64 + }{ + {"foo", "a|b", time.Now().Truncate(time.Second), 12.34}, + {"bar", "c|dd", time.Now().Truncate(time.Second), 42.42}, + } + writeBatch1 = ts.NewWriteBatch(len(values), ident.StringID(nsID1), nil) + writeBatch2 = ts.NewWriteBatch(len(values), ident.StringID(nsID2), nil) + ) + + mockDB.EXPECT(). + BatchWriter(ident.NewIDMatcher(nsID1), len(values)*2). + Return(writeBatch1, nil) + mockDB.EXPECT(). + BatchWriter(ident.NewIDMatcher(nsID2), len(values)*2). + Return(writeBatch2, nil) + + mockDB.EXPECT(). + WriteTaggedBatch(ctx, ident.NewIDMatcher(nsID1), writeBatch1, gomock.Any()). + Return(nil) + mockDB.EXPECT(). + WriteTaggedBatch(ctx, ident.NewIDMatcher(nsID2), writeBatch2, gomock.Any()). + Return(nil) + + var elements []*rpc.WriteTaggedBatchRawV2RequestElement + for nsIdx := range []string{nsID1, nsID2} { + for _, w := range values { + elem := &rpc.WriteTaggedBatchRawV2RequestElement{ + NameSpace: int64(nsIdx), + ID: []byte(w.id), + EncodedTags: []byte(w.tagEncode), + Datapoint: &rpc.Datapoint{ + Timestamp: w.t.Unix(), + TimestampTimeType: rpc.TimeType_UNIX_SECONDS, + Value: w.v, + }, + } + elements = append(elements, elem) + } + } + + mockDB.EXPECT().IsOverloaded().Return(false) + err := service.WriteTaggedBatchRawV2(tctx, &rpc.WriteTaggedBatchRawV2Request{ + NameSpaces: [][]byte{[]byte(nsID1), []byte(nsID2)}, + Elements: elements, + }) + require.NoError(t, err) +} + func TestServiceWriteTaggedBatchRawOverloaded(t *testing.T) { ctrl := gomock.NewController(t) defer ctrl.Finish()