From 8d81062e20f26da4ca87d5e0a20287f47e33f0f9 Mon Sep 17 00:00:00 2001 From: Richard Artoul Date: Tue, 8 Oct 2019 13:03:51 -0400 Subject: [PATCH] [DBNode] - Add support for batching fetches across namespaces (#1987) --- .../simple_v2_batch_apis/docker-compose.yml | 26 + .../simple_v2_batch_apis/m3coordinator.yml | 47 + .../simple_v2_batch_apis/test.sh | 188 +++ src/dbnode/client/fetch_batch.go | 27 +- .../client/fetch_batch_element_array_pool.go | 67 + src/dbnode/client/fetch_batch_pool.go | 65 + src/dbnode/client/host_queue.go | 132 +- .../client/host_queue_fetch_batch_test.go | 290 +++- src/dbnode/client/host_queue_test.go | 8 + src/dbnode/client/session.go | 23 +- src/dbnode/client/session_test.go | 4 +- src/dbnode/generated/thrift/rpc.thrift | 15 + src/dbnode/generated/thrift/rpc/rpc.go | 1483 +++++++++++++---- src/dbnode/generated/thrift/rpc/rpc_mock.go | 15 + src/dbnode/generated/thrift/rpc/tchan-rpc.go | 50 + src/dbnode/integration/options.go | 2 +- .../peers_bootstrap_high_concurrency_test.go | 8 +- .../peers_bootstrap_index_aggregate_test.go | 8 +- .../integration/peers_bootstrap_index_test.go | 8 +- .../peers_bootstrap_merge_local_test.go | 8 +- .../peers_bootstrap_merge_peer_blocks_test.go | 6 +- .../peers_bootstrap_node_down_test.go | 8 +- .../peers_bootstrap_none_available_test.go | 8 +- .../peers_bootstrap_select_best_test.go | 8 +- .../peers_bootstrap_simple_test.go | 6 +- .../peers_bootstrap_single_node_test.go | 6 +- src/dbnode/integration/repair_test.go | 6 +- src/dbnode/integration/setup.go | 3 +- .../server/tchannelthrift/node/service.go | 65 + .../tchannelthrift/node/service_test.go | 115 ++ 30 files changed, 2273 insertions(+), 432 deletions(-) create mode 100644 scripts/docker-integration-tests/simple_v2_batch_apis/docker-compose.yml create mode 100644 scripts/docker-integration-tests/simple_v2_batch_apis/m3coordinator.yml create mode 100755 scripts/docker-integration-tests/simple_v2_batch_apis/test.sh create mode 100644 src/dbnode/client/fetch_batch_element_array_pool.go create mode 100644 src/dbnode/client/fetch_batch_pool.go diff --git a/scripts/docker-integration-tests/simple_v2_batch_apis/docker-compose.yml b/scripts/docker-integration-tests/simple_v2_batch_apis/docker-compose.yml new file mode 100644 index 0000000000..bce43d559e --- /dev/null +++ b/scripts/docker-integration-tests/simple_v2_batch_apis/docker-compose.yml @@ -0,0 +1,26 @@ +version: "3.5" +services: + dbnode01: + expose: + - "9000-9004" + - "2379-2380" + ports: + - "0.0.0.0:9000-9004:9000-9004" + - "0.0.0.0:2379-2380:2379-2380" + networks: + - backend + image: "m3dbnode_integration:${REVISION}" + coordinator01: + expose: + - "7201" + - "7203" + ports: + - "0.0.0.0:7201:7201" + - "0.0.0.0:7203:7203" + networks: + - backend + image: "m3coordinator_integration:${REVISION}" + volumes: + - "./:/etc/m3coordinator/" +networks: + backend: diff --git a/scripts/docker-integration-tests/simple_v2_batch_apis/m3coordinator.yml b/scripts/docker-integration-tests/simple_v2_batch_apis/m3coordinator.yml new file mode 100644 index 0000000000..beffe0f668 --- /dev/null +++ b/scripts/docker-integration-tests/simple_v2_batch_apis/m3coordinator.yml @@ -0,0 +1,47 @@ +listenAddress: + type: "config" + value: "0.0.0.0:7201" + +logging: + level: info + +metrics: + scope: + prefix: "coordinator" + prometheus: + handlerPath: /metrics + listenAddress: 0.0.0.0:7203 # until https://github.com/m3db/m3/issues/682 is resolved + sanitization: prometheus + samplingRate: 1.0 + extended: none + +limits: + perQuery: + maxFetchedSeries: 100 + +clusters: + - namespaces: + - namespace: agg + type: aggregated + retention: 10h + resolution: 15s + - namespace: unagg + type: unaggregated + retention: 10m + client: + config: + service: + env: default_env + zone: embedded + service: m3db + cacheDir: /var/lib/m3kv + etcdClusters: + - zone: embedded + endpoints: + - dbnode01:2379 + writeConsistencyLevel: majority + readConsistencyLevel: unstrict_majority + useV2BatchAPIs: true + +tagOptions: + idScheme: quoted diff --git a/scripts/docker-integration-tests/simple_v2_batch_apis/test.sh b/scripts/docker-integration-tests/simple_v2_batch_apis/test.sh new file mode 100755 index 0000000000..093608123a --- /dev/null +++ b/scripts/docker-integration-tests/simple_v2_batch_apis/test.sh @@ -0,0 +1,188 @@ +#!/usr/bin/env bash + +set -xe + +source $GOPATH/src/github.com/m3db/m3/scripts/docker-integration-tests/common.sh +REVISION=$(git rev-parse HEAD) +COMPOSE_FILE=$GOPATH/src/github.com/m3db/m3/scripts/docker-integration-tests/prometheus/docker-compose.yml +# quay.io/m3db/prometheus_remote_client_golang @ v0.4.3 +PROMREMOTECLI_IMAGE=quay.io/m3db/prometheus_remote_client_golang@sha256:fc56df819bff9a5a087484804acf3a584dd4a78c68900c31a28896ed66ca7e7b +JQ_IMAGE=realguess/jq:1.4@sha256:300c5d9fb1d74154248d155ce182e207cf6630acccbaadd0168e18b15bfaa786 +METRIC_NAME_TEST_TOO_OLD=foo +METRIC_NAME_TEST_RESTRICT_WRITE=bar +export REVISION + +echo "Pull containers required for test" +docker pull $PROMREMOTECLI_IMAGE +docker pull $JQ_IMAGE + +echo "Run m3dbnode and m3coordinator containers" +docker-compose -f ${COMPOSE_FILE} up -d dbnode01 +docker-compose -f ${COMPOSE_FILE} up -d coordinator01 + +function defer { + docker-compose -f ${COMPOSE_FILE} down || echo "unable to shutdown containers" # CI fails to stop all containers sometimes +} +trap defer EXIT + +setup_single_m3db_node + +echo "Start Prometheus containers" +docker-compose -f ${COMPOSE_FILE} up -d prometheus01 + +function test_prometheus_remote_read { + # Ensure Prometheus can proxy a Prometheus query + echo "Wait until the remote write endpoint generates and allows for data to be queried" + ATTEMPTS=50 TIMEOUT=2 MAX_TIMEOUT=4 retry_with_backoff \ + '[[ $(curl -sSf 0.0.0.0:9090/api/v1/query?query=prometheus_remote_storage_succeeded_samples_total | jq -r .data.result[].value[1]) -gt 100 ]]' +} + +function test_prometheus_remote_write_multi_namespaces { + # Make sure we're proxying writes to the unaggregated namespace + echo "Wait until data begins being written to remote storage for the unaggregated namespace" + ATTEMPTS=50 TIMEOUT=2 MAX_TIMEOUT=4 retry_with_backoff \ + '[[ $(curl -sSf 0.0.0.0:9090/api/v1/query?query=database_write_tagged_success\\{namespace=\"unagg\"\\} | jq -r .data.result[0].value[1]) -gt 0 ]]' + + # Make sure we're proxying writes to the aggregated namespace + echo "Wait until data begins being written to remote storage for the aggregated namespace" + ATTEMPTS=50 TIMEOUT=2 MAX_TIMEOUT=4 retry_with_backoff \ + '[[ $(curl -sSf 0.0.0.0:9090/api/v1/query?query=database_write_tagged_success\\{namespace=\"agg\"\\} | jq -r .data.result[0].value[1]) -gt 0 ]]' +} + +function prometheus_remote_write { + local metric_name=$1 + local datapoint_timestamp=$2 + local datapoint_value=$3 + local expect_success=$4 + local expect_success_err=$5 + local expect_status=$6 + local expect_status_err=$7 + local metrics_type=$8 + local metrics_storage_policy=$9 + + network=$(docker network ls --format '{{.ID}}' | tail -n 1) + out=$((docker run -it --rm --network $network \ + $PROMREMOTECLI_IMAGE \ + -u http://coordinator01:7201/api/v1/prom/remote/write \ + -t __name__:${metric_name} \ + -h "M3-Metrics-Type: ${metrics_type}" \ + -h "M3-Storage-Policy: ${metrics_storage_policy}" \ + -d ${datapoint_timestamp},${datapoint_value} | grep -v promremotecli_log) || true) + success=$(echo $out | grep -v promremotecli_log | docker run --rm -i $JQ_IMAGE jq .success) + status=$(echo $out | grep -v promremotecli_log | docker run --rm -i $JQ_IMAGE jq .statusCode) + if [[ "$success" != "$expect_success" ]]; then + echo $expect_success_err + return 1 + fi + if [[ "$status" != "$expect_status" ]]; then + echo "${expect_status_err}: actual=${status}" + return 1 + fi + echo "Returned success=${success}, status=${status} as expected" + return 0 +} + +function test_prometheus_remote_write_too_old_returns_400_status_code { + # Test writing too far into the past returns an HTTP 400 status code + echo "Test write into the past returns HTTP 400" + hour_ago=$(expr $(date +"%s") - 3600) + prometheus_remote_write \ + $METRIC_NAME_TEST_TOO_OLD $hour_ago 3.142 \ + false "Expected request to fail" \ + 400 "Expected request to return status code 400" +} + +function test_prometheus_remote_write_restrict_metrics_type { + # Test we can specify metrics type + echo "Test write with unaggregated metrics type works as expected" + prometheus_remote_write \ + $METRIC_NAME_TEST_RESTRICT_WRITE now 42.42 \ + true "Expected request to succeed" \ + 200 "Expected request to return status code 200" \ + unaggregated + + echo "Test write with aggregated metrics type works as expected" + prometheus_remote_write \ + $METRIC_NAME_TEST_RESTRICT_WRITE now 84.84 \ + true "Expected request to succeed" \ + 200 "Expected request to return status code 200" \ + aggregated 15s:10h +} + +function test_query_limits_applied { + # Test the default series limit applied when directly querying + # coordinator (limit set to 100 in m3coordinator.yml) + echo "Test query limit with coordinator defaults" + ATTEMPTS=50 TIMEOUT=2 MAX_TIMEOUT=4 retry_with_backoff \ + '[[ $(curl -s 0.0.0.0:7201/api/v1/query?query=\\{name!=\"\"\\} | jq -r ".data.result | length") -eq 100 ]]' + + # Test the default series limit applied when directly querying + # coordinator (limit set by header) + echo "Test query limit with coordinator limit header" + ATTEMPTS=50 TIMEOUT=2 MAX_TIMEOUT=4 retry_with_backoff \ + '[[ $(curl -s -H "M3-Limit-Max-Series: 10" 0.0.0.0:7201/api/v1/query?query=\\{name!=\"\"\\} | jq -r ".data.result | length") -eq 10 ]]' +} + +function prometheus_query_native { + local endpoint=${endpoint:-} + local query=${query:-} + local params=${params:-} + local metrics_type=${metrics_type:-} + local metrics_storage_policy=${metrics_storage_policy:-} + local jq_path=${jq_path:-} + local expected_value=${expected_value:-} + + params_prefixed="" + if [[ "$params" != "" ]]; then + params_prefixed='&'"${params}" + fi + + result=$(curl -s \ + -H "M3-Metrics-Type: ${metrics_type}" \ + -H "M3-Storage-Policy: ${metrics_storage_policy}" \ + "0.0.0.0:7201/api/v1/${endpoint}?query=${query}${params_prefixed}" | jq -r "${jq_path}") + test "$result" = "$expected_value" + return $? +} + +function test_query_restrict_metrics_type { + now=$(date +"%s") + hour_ago=$(expr $now - 3600) + step="30s" + params_instant="" + params_range="start=${hour_ago}"'&'"end=${now}"'&'"step=30s" + jq_path_instant=".data.result[0].value[1]" + jq_path_range=".data.result[0].values[][1]" + + # Test restricting to unaggregated metrics + echo "Test query restrict to unaggregated metrics type (instant)" + ATTEMPTS=50 TIMEOUT=2 MAX_TIMEOUT=4 \ + endpoint=query query="$METRIC_NAME_TEST_RESTRICT_WRITE" params="$params_instant" \ + metrics_type="unaggregated" jq_path="$jq_path_instant" expected_value="42.42" \ + retry_with_backoff prometheus_query_native + echo "Test query restrict to unaggregated metrics type (range)" + ATTEMPTS=50 TIMEOUT=2 MAX_TIMEOUT=4 \ + endpoint=query_range query="$METRIC_NAME_TEST_RESTRICT_WRITE" params="$params_range" \ + metrics_type="unaggregated" jq_path="$jq_path_range" expected_value="42.42" \ + retry_with_backoff prometheus_query_native + + # Test restricting to aggregated metrics + echo "Test query restrict to aggregated metrics type (instant)" + ATTEMPTS=50 TIMEOUT=2 MAX_TIMEOUT=4 \ + endpoint=query query="$METRIC_NAME_TEST_RESTRICT_WRITE" params="$params_instant" \ + metrics_type="aggregated" metrics_storage_policy="15s:10h" jq_path="$jq_path_instant" expected_value="84.84" \ + retry_with_backoff prometheus_query_native + echo "Test query restrict to aggregated metrics type (range)" + ATTEMPTS=50 TIMEOUT=2 MAX_TIMEOUT=4 \ + endpoint=query_range query="$METRIC_NAME_TEST_RESTRICT_WRITE" params="$params_range" \ + metrics_type="aggregated" metrics_storage_policy="15s:10h" jq_path="$jq_path_range" expected_value="84.84" \ + retry_with_backoff prometheus_query_native +} + +# Run all tests +test_prometheus_remote_read +test_prometheus_remote_write_multi_namespaces +test_prometheus_remote_write_too_old_returns_400_status_code +test_prometheus_remote_write_restrict_metrics_type +test_query_limits_applied +test_query_restrict_metrics_type diff --git a/src/dbnode/client/fetch_batch.go b/src/dbnode/client/fetch_batch.go index 7c4983ec6f..23c2af25c2 100644 --- a/src/dbnode/client/fetch_batch.go +++ b/src/dbnode/client/fetch_batch.go @@ -28,9 +28,10 @@ import ( type fetchBatchOp struct { checked.RefCount - request rpc.FetchBatchRawRequest - completionFns []completionFn - finalizer fetchBatchOpFinalizer + request rpc.FetchBatchRawRequest + requestV2Elements []rpc.FetchBatchRawV2RequestElement + completionFns []completionFn + finalizer fetchBatchOpFinalizer } func (f *fetchBatchOp) reset() { @@ -46,6 +47,16 @@ func (f *fetchBatchOp) reset() { f.completionFns[i] = nil } f.completionFns = f.completionFns[:0] + + for i := range f.requestV2Elements { + f.requestV2Elements[i].NameSpace = 0 + f.requestV2Elements[i].RangeStart = 0 + f.requestV2Elements[i].RangeEnd = 0 + f.requestV2Elements[i].ID = nil + f.requestV2Elements[i].RangeTimeType = 0 + } + f.requestV2Elements = f.requestV2Elements[:0] + f.DecWrites() } @@ -53,6 +64,13 @@ func (f *fetchBatchOp) append(namespace, id []byte, completionFn completionFn) { f.IncWrites() f.request.NameSpace = namespace f.request.Ids = append(f.request.Ids, id) + f.requestV2Elements = append(f.requestV2Elements, rpc.FetchBatchRawV2RequestElement{ + // NameSpace filled in by the host queue later. + RangeStart: f.request.RangeStart, + RangeEnd: f.request.RangeEnd, + ID: id, + RangeTimeType: f.request.RangeTimeType, + }) f.completionFns = append(f.completionFns, completionFn) f.DecWrites() } @@ -60,6 +78,9 @@ func (f *fetchBatchOp) append(namespace, id []byte, completionFn completionFn) { func (f *fetchBatchOp) Size() int { f.IncReads() value := len(f.request.Ids) + if value == 0 { + value = len(f.requestV2Elements) + } f.DecReads() return value } diff --git a/src/dbnode/client/fetch_batch_element_array_pool.go b/src/dbnode/client/fetch_batch_element_array_pool.go new file mode 100644 index 0000000000..e3304090cd --- /dev/null +++ b/src/dbnode/client/fetch_batch_element_array_pool.go @@ -0,0 +1,67 @@ +// Copyright (c) 2019 Uber Technologies, Inc. +// +// Permission is hereby granted, free of charge, to any person obtaining a copy +// of this software and associated documentation files (the "Software"), to deal +// in the Software without restriction, including without limitation the rights +// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell +// copies of the Software, and to permit persons to whom the Software is +// furnished to do so, subject to the following conditions: +// +// The above copyright notice and this permission notice shall be included in +// all copies or substantial portions of the Software. +// +// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR +// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, +// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE +// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER +// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, +// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN +// THE SOFTWARE. + +package client + +import ( + "github.com/m3db/m3/src/dbnode/generated/thrift/rpc" + "github.com/m3db/m3/src/x/pool" +) + +type fetchBatchRawV2RequestElementArrayPool interface { + // Init pool + Init() + + // Get an array of WriteBatchV2RawRequestElement objects + Get() []*rpc.FetchBatchRawV2RequestElement + + // Put an array of FetchBatchRawV2RequestElement objects + Put(w []*rpc.FetchBatchRawV2RequestElement) +} + +type poolOfFetchBatchRawV2RequestElementArray struct { + pool pool.ObjectPool + capacity int +} + +func newFetchBatchRawV2RequestElementArrayPool( + opts pool.ObjectPoolOptions, capacity int) fetchBatchRawV2RequestElementArrayPool { + + p := pool.NewObjectPool(opts) + return &poolOfFetchBatchRawV2RequestElementArray{p, capacity} +} + +func (p *poolOfFetchBatchRawV2RequestElementArray) Init() { + p.pool.Init(func() interface{} { + return make([]*rpc.FetchBatchRawV2RequestElement, 0, p.capacity) + }) +} + +func (p *poolOfFetchBatchRawV2RequestElementArray) Get() []*rpc.FetchBatchRawV2RequestElement { + return p.pool.Get().([]*rpc.FetchBatchRawV2RequestElement) +} + +func (p *poolOfFetchBatchRawV2RequestElementArray) Put(w []*rpc.FetchBatchRawV2RequestElement) { + for i := range w { + w[i] = nil + } + w = w[:0] + p.pool.Put(w) +} diff --git a/src/dbnode/client/fetch_batch_pool.go b/src/dbnode/client/fetch_batch_pool.go new file mode 100644 index 0000000000..77b4c65fb8 --- /dev/null +++ b/src/dbnode/client/fetch_batch_pool.go @@ -0,0 +1,65 @@ +// Copyright (c) 2019 Uber Technologies, Inc. +// +// Permission is hereby granted, free of charge, to any person obtaining a copy +// of this software and associated documentation files (the "Software"), to deal +// in the Software without restriction, including without limitation the rights +// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell +// copies of the Software, and to permit persons to whom the Software is +// furnished to do so, subject to the following conditions: +// +// The above copyright notice and this permission notice shall be included in +// all copies or substantial portions of the Software. +// +// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR +// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, +// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE +// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER +// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, +// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN +// THE SOFTWARE. + +package client + +import ( + "github.com/m3db/m3/src/dbnode/generated/thrift/rpc" + "github.com/m3db/m3/src/x/pool" +) + +var ( + fetchBatchRawV2RequestZero rpc.FetchBatchRawV2Request +) + +type fetchBatchRawV2RequestPool interface { + // Init pool. + Init() + + // Get a write batch request. + Get() *rpc.FetchBatchRawV2Request + + // Put a write batch request. + Put(w *rpc.FetchBatchRawV2Request) +} + +type poolOfFetchBatchRawV2Request struct { + pool pool.ObjectPool +} + +func newFetchBatchRawV2RequestPool(opts pool.ObjectPoolOptions) fetchBatchRawV2RequestPool { + p := pool.NewObjectPool(opts) + return &poolOfFetchBatchRawV2Request{p} +} + +func (p *poolOfFetchBatchRawV2Request) Init() { + p.pool.Init(func() interface{} { + return &rpc.FetchBatchRawV2Request{} + }) +} + +func (p *poolOfFetchBatchRawV2Request) Get() *rpc.FetchBatchRawV2Request { + return p.pool.Get().(*rpc.FetchBatchRawV2Request) +} + +func (p *poolOfFetchBatchRawV2Request) Put(w *rpc.FetchBatchRawV2Request) { + *w = fetchBatchRawV2RequestZero + p.pool.Put(w) +} diff --git a/src/dbnode/client/host_queue.go b/src/dbnode/client/host_queue.go index 6ede814c7f..6dc4a9ad3b 100644 --- a/src/dbnode/client/host_queue.go +++ b/src/dbnode/client/host_queue.go @@ -56,6 +56,8 @@ type queue struct { writeTaggedBatchRawV2RequestPool writeTaggedBatchRawV2RequestPool writeTaggedBatchRawRequestElementArrayPool writeTaggedBatchRawRequestElementArrayPool writeTaggedBatchRawV2RequestElementArrayPool writeTaggedBatchRawV2RequestElementArrayPool + fetchBatchRawV2RequestPool fetchBatchRawV2RequestPool + fetchBatchRawV2RequestElementArrayPool fetchBatchRawV2RequestElementArrayPool workerPool xsync.PooledWorkerPool size int ops []op @@ -135,14 +137,16 @@ func newHostQueue( 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(), + fetchBatchRawV2RequestPool: hostQueueOpts.fetchBatchRawV2RequestPool, + fetchBatchRawV2RequestElementArrayPool: hostQueueOpts.fetchBatchRawV2RequestElementArrayPool, + 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 } @@ -237,6 +241,9 @@ func (q *queue) drain() { currWriteOpsByNamespace namespaceWriteBatchOpsSlice currTaggedWriteOpsByNamespace namespaceWriteTaggedBatchOpsSlice + + currV2FetchBatchRawReq *rpc.FetchBatchRawV2Request + currV2FetchBatchRawOps []op ) for ops := range q.drainIn { @@ -256,7 +263,11 @@ func (q *queue) drain() { currTaggedWriteOpsByNamespace = q.drainTaggedWriteOpV1(v, currTaggedWriteOpsByNamespace, ops[i]) } case *fetchBatchOp: - q.asyncFetch(v) + if q.serverSupportsV2APIs { + currV2FetchBatchRawReq, currV2FetchBatchRawOps = q.drainFetchBatchRawV2Op(v, currV2FetchBatchRawReq, currV2FetchBatchRawOps, ops[i]) + } else { + q.asyncFetch(v) + } case *fetchTaggedOp: q.asyncFetchTagged(v) case *aggregateOp: @@ -295,6 +306,13 @@ func (q *queue) drain() { // Zero the element currTaggedWriteOpsByNamespace[i] = namespaceWriteTaggedBatchOps{} } + // If any outstanding fetches, fetch. + if currV2FetchBatchRawReq != nil { + q.asyncFetchV2(currV2FetchBatchRawOps, currV2FetchBatchRawReq) + currV2FetchBatchRawOps = nil + currV2FetchBatchRawReq = nil + } + // Reset the slice currTaggedWriteOpsByNamespace = currTaggedWriteOpsByNamespace[:0] if currV2WriteTaggedReq != nil { @@ -443,6 +461,45 @@ func (q *queue) drainTaggedWriteOpV2( return currV2WriteTaggedReq, currV2WriteTaggedOps } +func (q *queue) drainFetchBatchRawV2Op( + v *fetchBatchOp, + currV2FetchBatchRawReq *rpc.FetchBatchRawV2Request, + currV2FetchBatchRawOps []op, + op op, +) (*rpc.FetchBatchRawV2Request, []op) { + namespace := v.request.NameSpace + if currV2FetchBatchRawReq == nil { + currV2FetchBatchRawReq = q.fetchBatchRawV2RequestPool.Get() + currV2FetchBatchRawReq.Elements = q.fetchBatchRawV2RequestElementArrayPool.Get() + } + + nsIdx := -1 + for i, ns := range currV2FetchBatchRawReq.NameSpaces { + if bytes.Equal(namespace, ns) { + nsIdx = i + break + } + } + if nsIdx == -1 { + currV2FetchBatchRawReq.NameSpaces = append(currV2FetchBatchRawReq.NameSpaces, namespace) + nsIdx = len(currV2FetchBatchRawReq.NameSpaces) - 1 + } + for i := range v.requestV2Elements { + v.requestV2Elements[i].NameSpace = int64(nsIdx) + currV2FetchBatchRawReq.Elements = append(currV2FetchBatchRawReq.Elements, &v.requestV2Elements[i]) + } + currV2FetchBatchRawOps = append(currV2FetchBatchRawOps, op) + // This logic means that in practice we may sometimes exceed the fetch batch size by a factor of 2 + // but that's ok since it does not need to be exact. + if len(currV2FetchBatchRawReq.Elements) >= q.opts.FetchBatchSize() { + q.asyncFetchV2(currV2FetchBatchRawOps, currV2FetchBatchRawReq) + currV2FetchBatchRawReq = nil + currV2FetchBatchRawOps = nil + } + + return currV2FetchBatchRawReq, currV2FetchBatchRawOps +} + func (q *queue) asyncTaggedWrite( namespace ident.ID, ops []op, @@ -738,6 +795,63 @@ func (q *queue) asyncFetch(op *fetchBatchOp) { }) } +func (q *queue) asyncFetchV2( + ops []op, + currV2FetchBatchRawReq *rpc.FetchBatchRawV2Request, +) { + q.fetchOpBatchSize.RecordValue(float64(len(currV2FetchBatchRawReq.Elements))) + q.Add(1) + q.workerPool.Go(func() { + // NB(r): Defer is slow in the hot path unfortunately + cleanup := func() { + q.fetchBatchRawV2RequestElementArrayPool.Put(currV2FetchBatchRawReq.Elements) + q.fetchBatchRawV2RequestPool.Put(currV2FetchBatchRawReq) + for _, op := range ops { + fetchOp := op.(*fetchBatchOp) + fetchOp.DecRef() + fetchOp.Finalize() + } + q.Done() + } + + client, err := q.connPool.NextClient() + if err != nil { + // No client available. + callAllCompletionFns(ops, nil, err) + cleanup() + return + } + + ctx, _ := thrift.NewContext(q.opts.FetchRequestTimeout()) + result, err := client.FetchBatchRawV2(ctx, currV2FetchBatchRawReq) + if err != nil { + callAllCompletionFns(ops, nil, err) + cleanup() + return + } + + resultIdx := -1 + for _, op := range ops { + fetchOp := op.(*fetchBatchOp) + for j := 0; j < fetchOp.Size(); j++ { + resultIdx++ + if resultIdx >= len(result.Elements) { + // No results for this entry, in practice should never occur. + fetchOp.complete(j, nil, errQueueFetchNoResponse(q.host.ID())) + continue + } + + if result.Elements[resultIdx].Err != nil { + fetchOp.complete(j, nil, result.Elements[resultIdx].Err) + continue + } + fetchOp.complete(j, result.Elements[resultIdx].Segments, nil) + } + } + cleanup() + }) +} + func (q *queue) asyncFetchTagged(op *fetchTaggedOp) { q.Add(1) q.workerPool.Go(func() { diff --git a/src/dbnode/client/host_queue_fetch_batch_test.go b/src/dbnode/client/host_queue_fetch_batch_test.go index 55b0412578..a485afaf46 100644 --- a/src/dbnode/client/host_queue_fetch_batch_test.go +++ b/src/dbnode/client/host_queue_fetch_batch_test.go @@ -48,6 +48,102 @@ func TestHostQueueFetchBatches(t *testing.T) { }) } +func TestHostQueueFetchBatchesV2MultiNS(t *testing.T) { + ids := []string{"foo", "bar", "baz", "qux"} + result := &rpc.FetchBatchRawResult_{} + for range ids { + result.Elements = append(result.Elements, &rpc.FetchRawResult_{Segments: []*rpc.Segments{}}) + } + var expected []hostQueueResult + for i := range ids { + expected = append(expected, hostQueueResult{result.Elements[i].Segments, nil}) + } + opts := newHostQueueTestOptions().SetUseV2BatchAPIs(true) + 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 fetches. + var ( + results []hostQueueResult + wg sync.WaitGroup + ) + callback := func(r interface{}, err error) { + results = append(results, hostQueueResult{r, err}) + wg.Done() + } + + fetchBatches := []*fetchBatchOp{} + for i, id := range ids { + fetchBatch := &fetchBatchOp{ + request: rpc.FetchBatchRawRequest{ + NameSpace: []byte(fmt.Sprintf("ns-%d", i)), + }, + requestV2Elements: []rpc.FetchBatchRawV2RequestElement{ + { + ID: []byte(id), + RangeStart: int64(i), + RangeEnd: int64(i + 1), + }, + }, + } + fetchBatches = append(fetchBatches, fetchBatch) + fetchBatch.completionFns = append(fetchBatch.completionFns, callback) + } + wg.Add(len(ids)) + + // Prepare mocks for flush + mockClient := rpc.NewMockTChanNode(ctrl) + + verifyFetchBatchRawV2 := func(ctx thrift.Context, req *rpc.FetchBatchRawV2Request) { + assert.Equal(t, len(ids), len(req.NameSpaces)) + for i, ns := range req.NameSpaces { + assert.Equal(t, []byte(fmt.Sprintf("ns-%d", i)), ns) + } + assert.Equal(t, len(ids), len(req.Elements)) + for i, elem := range req.Elements { + assert.Equal(t, int64(i), elem.NameSpace) + assert.Equal(t, int64(i), elem.RangeStart) + assert.Equal(t, int64(i+1), elem.RangeEnd) + assert.Equal(t, []byte(ids[i]), elem.ID) + } + } + + mockClient.EXPECT(). + FetchBatchRawV2(gomock.Any(), gomock.Any()). + Do(verifyFetchBatchRawV2). + Return(result, nil) + + mockConnPool.EXPECT().NextClient().Return(mockClient, nil) + + for _, fetchBatch := range fetchBatches { + assert.NoError(t, queue.Enqueue(fetchBatch)) + } + + // Wait for fetch to complete. + wg.Wait() + + assert.Equal(t, len(ids), len(results)) + + // Close. + var closeWg sync.WaitGroup + closeWg.Add(1) + mockConnPool.EXPECT().Close().Do(func() { + closeWg.Done() + }) + queue.Close() + closeWg.Wait() +} + func TestHostQueueFetchBatchesErrorOnNextClientUnavailable(t *testing.T) { namespace := "testNs" ids := []string{"foo", "bar", "baz", "qux"} @@ -136,91 +232,137 @@ func testHostQueueFetchBatches( testOpts *testHostQueueFetchBatchesOptions, assertion func(results []hostQueueResult), ) { - ctrl := gomock.NewController(t) - defer ctrl.Finish() + 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) + mockConnPool := NewMockconnectionPool(ctrl) - opts := newHostQueueTestOptions() - queue := newTestHostQueue(opts) - queue.connPool = mockConnPool + queue := newTestHostQueue(opts) + queue.connPool = mockConnPool - // Open - mockConnPool.EXPECT().Open() - queue.Open() - assert.Equal(t, statusOpen, queue.status) + // Open + mockConnPool.EXPECT().Open() + queue.Open() + assert.Equal(t, statusOpen, queue.status) - // Prepare callback for fetches - var ( - results []hostQueueResult - wg sync.WaitGroup - ) - callback := func(r interface{}, err error) { - results = append(results, hostQueueResult{r, err}) - wg.Done() - } + // Prepare callback for fetches + var ( + results []hostQueueResult + wg sync.WaitGroup + ) + callback := func(r interface{}, err error) { + results = append(results, hostQueueResult{r, err}) + wg.Done() + } - rawIDs := make([][]byte, len(ids)) + rawIDs := make([][]byte, len(ids)) - for i, id := range ids { - rawIDs[i] = []byte(id) - } + for i, id := range ids { + rawIDs[i] = []byte(id) + } - // Prepare fetch batch op - fetchBatch := &fetchBatchOp{ - request: rpc.FetchBatchRawRequest{ - RangeStart: 0, - RangeEnd: 1, - NameSpace: []byte(namespace), - Ids: rawIDs, - }, - } - for range fetchBatch.request.Ids { - fetchBatch.completionFns = append(fetchBatch.completionFns, callback) - } - wg.Add(len(fetchBatch.request.Ids)) + var fetchBatch *fetchBatchOp + if opts.UseV2BatchAPIs() { + fetchBatch = &fetchBatchOp{ + request: rpc.FetchBatchRawRequest{ + NameSpace: []byte(namespace), + }, + } + } else { + fetchBatch = &fetchBatchOp{ + request: rpc.FetchBatchRawRequest{ + RangeStart: 0, + RangeEnd: 1, + NameSpace: []byte(namespace), + Ids: rawIDs, + }, + } + } - // Prepare mocks for flush - mockClient := rpc.NewMockTChanNode(ctrl) - if testOpts != nil && testOpts.nextClientErr != nil { - mockConnPool.EXPECT().NextClient().Return(nil, testOpts.nextClientErr) - } else if testOpts != nil && testOpts.fetchRawBatchErr != nil { - fetchBatchRaw := func(ctx thrift.Context, req *rpc.FetchBatchRawRequest) { - assert.Equal(t, &fetchBatch.request, req) - } - mockClient.EXPECT(). - FetchBatchRaw(gomock.Any(), gomock.Any()). - Do(fetchBatchRaw). - Return(nil, testOpts.fetchRawBatchErr) - - mockConnPool.EXPECT().NextClient().Return(mockClient, nil) - } else { - fetchBatchRaw := func(ctx thrift.Context, req *rpc.FetchBatchRawRequest) { - assert.Equal(t, &fetchBatch.request, req) - } - mockClient.EXPECT(). - FetchBatchRaw(gomock.Any(), gomock.Any()). - Do(fetchBatchRaw). - Return(result, nil) + for _, id := range ids { + if opts.UseV2BatchAPIs() { + fetchBatch.requestV2Elements = append(fetchBatch.requestV2Elements, rpc.FetchBatchRawV2RequestElement{ + ID: []byte(id), + RangeStart: 0, + RangeEnd: 1, + }) + } + fetchBatch.completionFns = append(fetchBatch.completionFns, callback) + } + wg.Add(len(ids)) - mockConnPool.EXPECT().NextClient().Return(mockClient, nil) - } + // Prepare mocks for flush + mockClient := rpc.NewMockTChanNode(ctrl) - // Fetch - assert.NoError(t, queue.Enqueue(fetchBatch)) + verifyFetchBatchRawV2 := func(ctx thrift.Context, req *rpc.FetchBatchRawV2Request) { + assert.Equal(t, 1, len(req.NameSpaces)) + assert.Equal(t, len(ids), len(req.Elements)) + for i, elem := range req.Elements { + assert.Equal(t, int64(0), elem.NameSpace) + assert.Equal(t, int64(0), elem.RangeStart) + assert.Equal(t, int64(1), elem.RangeEnd) + assert.Equal(t, []byte(ids[i]), elem.ID) + } + } + if testOpts != nil && testOpts.nextClientErr != nil { + mockConnPool.EXPECT().NextClient().Return(nil, testOpts.nextClientErr) + } else if testOpts != nil && testOpts.fetchRawBatchErr != nil { + if opts.UseV2BatchAPIs() { + mockClient.EXPECT(). + FetchBatchRawV2(gomock.Any(), gomock.Any()). + Do(verifyFetchBatchRawV2). + Return(nil, testOpts.fetchRawBatchErr) + } else { + fetchBatchRaw := func(ctx thrift.Context, req *rpc.FetchBatchRawRequest) { + assert.Equal(t, &fetchBatch.request, req) + } + mockClient.EXPECT(). + FetchBatchRaw(gomock.Any(), gomock.Any()). + Do(fetchBatchRaw). + Return(nil, testOpts.fetchRawBatchErr) + } + mockConnPool.EXPECT().NextClient().Return(mockClient, nil) + } else { + if opts.UseV2BatchAPIs() { + mockClient.EXPECT(). + FetchBatchRawV2(gomock.Any(), gomock.Any()). + Do(verifyFetchBatchRawV2). + Return(result, nil) + } else { + fetchBatchRaw := func(ctx thrift.Context, req *rpc.FetchBatchRawRequest) { + assert.Equal(t, &fetchBatch.request, req) + } + mockClient.EXPECT(). + FetchBatchRaw(gomock.Any(), gomock.Any()). + Do(fetchBatchRaw). + Return(result, nil) + } - // Wait for fetch to complete - wg.Wait() + mockConnPool.EXPECT().NextClient().Return(mockClient, nil) + } - // Assert results match expected - assertion(results) + // Fetch + assert.NoError(t, queue.Enqueue(fetchBatch)) - // Close - var closeWg sync.WaitGroup - closeWg.Add(1) - mockConnPool.EXPECT().Close().Do(func() { - closeWg.Done() - }) - queue.Close() - closeWg.Wait() + // Wait for fetch to complete + wg.Wait() + + // Assert results match expected + assertion(results) + + // Close + var closeWg sync.WaitGroup + closeWg.Add(1) + mockConnPool.EXPECT().Close().Do(func() { + closeWg.Done() + }) + queue.Close() + closeWg.Wait() + }) + } } diff --git a/src/dbnode/client/host_queue_test.go b/src/dbnode/client/host_queue_test.go index bf78248acf..cd7bedf4e6 100644 --- a/src/dbnode/client/host_queue_test.go +++ b/src/dbnode/client/host_queue_test.go @@ -36,6 +36,9 @@ var ( testWriteTaggedArrayPool writeTaggedBatchRawRequestElementArrayPool testWriteTaggedV2ArrayPool writeTaggedBatchRawV2RequestElementArrayPool + + testFetchBatchRawV2Pool fetchBatchRawV2RequestPool + testFetchBatchRawV2ArrayPool fetchBatchRawV2RequestElementArrayPool ) func init() { @@ -58,6 +61,11 @@ func init() { testWriteTaggedArrayPool.Init() testWriteTaggedV2ArrayPool = newWriteTaggedBatchRawV2RequestElementArrayPool(smallPoolOptions, 0) testWriteTaggedV2ArrayPool.Init() + + testFetchBatchRawV2Pool = newFetchBatchRawV2RequestPool(smallPoolOptions) + testFetchBatchRawV2Pool.Init() + testFetchBatchRawV2ArrayPool = newFetchBatchRawV2RequestElementArrayPool(smallPoolOptions, 0) + testFetchBatchRawV2ArrayPool.Init() } type hostQueueResult struct { diff --git a/src/dbnode/client/session.go b/src/dbnode/client/session.go index 3d0c15d17f..06272dc801 100644 --- a/src/dbnode/client/session.go +++ b/src/dbnode/client/session.go @@ -223,6 +223,8 @@ type hostQueueOpts struct { writeTaggedBatchRawV2RequestPool writeTaggedBatchRawV2RequestPool writeTaggedBatchRawRequestElementArrayPool writeTaggedBatchRawRequestElementArrayPool writeTaggedBatchRawV2RequestElementArrayPool writeTaggedBatchRawV2RequestElementArrayPool + fetchBatchRawV2RequestPool fetchBatchRawV2RequestPool + fetchBatchRawV2RequestElementArrayPool fetchBatchRawV2RequestElementArrayPool opts Options } @@ -891,6 +893,23 @@ func (s *session) newHostQueue(host topology.Host, topoMap topology.Map) (hostQu writeTaggedBatchRawRequestElementArrayPool.Init() writeTaggedBatchRawV2RequestElementArrayPool := newWriteTaggedBatchRawV2RequestElementArrayPool( writeTaggedBatchRawRequestElementArrayPoolOpts, s.opts.WriteBatchSize()) + writeTaggedBatchRawV2RequestElementArrayPool.Init() + + fetchBatchRawV2RequestPoolOpts := pool.NewObjectPoolOptions(). + SetSize(hostBatches). + SetInstrumentOptions(s.opts.InstrumentOptions().SetMetricsScope( + s.scope.SubScope("fetch-batch-request-pool"), + )) + fetchBatchRawV2RequestPool := newFetchBatchRawV2RequestPool(fetchBatchRawV2RequestPoolOpts) + fetchBatchRawV2RequestPool.Init() + + fetchBatchRawV2RequestElementArrayPoolOpts := pool.NewObjectPoolOptions(). + SetSize(hostBatches). + SetInstrumentOptions(s.opts.InstrumentOptions().SetMetricsScope( + s.scope.SubScope("fetch-batch-request-array-pool"), + )) + fetchBatchRawV2RequestElementArrayPool := newFetchBatchRawV2RequestElementArrayPool(fetchBatchRawV2RequestElementArrayPoolOpts, s.opts.FetchBatchSize()) + fetchBatchRawV2RequestElementArrayPool.Init() hostQueue, err := s.newHostQueueFn(host, hostQueueOpts{ writeBatchRawRequestPool: writeBatchRequestPool, @@ -901,7 +920,9 @@ func (s *session) newHostQueue(host topology.Host, topoMap topology.Map) (hostQu writeTaggedBatchRawV2RequestPool: writeTaggedBatchV2RequestPool, writeTaggedBatchRawRequestElementArrayPool: writeTaggedBatchRawRequestElementArrayPool, writeTaggedBatchRawV2RequestElementArrayPool: writeTaggedBatchRawV2RequestElementArrayPool, - opts: s.opts, + fetchBatchRawV2RequestPool: fetchBatchRawV2RequestPool, + fetchBatchRawV2RequestElementArrayPool: fetchBatchRawV2RequestElementArrayPool, + opts: s.opts, }) if err != nil { return nil, err diff --git a/src/dbnode/client/session_test.go b/src/dbnode/client/session_test.go index 6f89c7acde..764cd0404e 100644 --- a/src/dbnode/client/session_test.go +++ b/src/dbnode/client/session_test.go @@ -134,7 +134,9 @@ func newTestHostQueue(opts Options) *queue { writeTaggedBatchRawV2RequestPool: testWriteTaggedBatchRawV2Pool, writeTaggedBatchRawRequestElementArrayPool: testWriteTaggedArrayPool, writeTaggedBatchRawV2RequestElementArrayPool: testWriteTaggedV2ArrayPool, - opts: opts, + fetchBatchRawV2RequestPool: testFetchBatchRawV2Pool, + fetchBatchRawV2RequestElementArrayPool: testFetchBatchRawV2ArrayPool, + opts: opts, }) if err != nil { panic(err) diff --git a/src/dbnode/generated/thrift/rpc.thrift b/src/dbnode/generated/thrift/rpc.thrift index 422ece3fb7..3b3a392b47 100644 --- a/src/dbnode/generated/thrift/rpc.thrift +++ b/src/dbnode/generated/thrift/rpc.thrift @@ -53,6 +53,7 @@ service Node { // Performant read/write endpoints FetchBatchRawResult fetchBatchRaw(1: FetchBatchRawRequest req) throws (1: Error err) + FetchBatchRawResult fetchBatchRawV2(1: FetchBatchRawV2Request req) throws (1: Error err) FetchBlocksRawResult fetchBlocksRaw(1: FetchBlocksRawRequest req) throws (1: Error err) FetchBlocksMetadataRawV2Result fetchBlocksMetadataRawV2(1: FetchBlocksMetadataRawV2Request req) throws (1: Error err) @@ -120,6 +121,20 @@ struct FetchBatchRawRequest { 5: optional TimeType rangeTimeType = TimeType.UNIX_SECONDS } + +struct FetchBatchRawV2Request { + 1: required list nameSpaces + 2: required list elements +} + +struct FetchBatchRawV2RequestElement { + 1: required i64 nameSpace + 2: required i64 rangeStart + 3: required i64 rangeEnd + 4: required binary id + 5: optional TimeType rangeTimeType = TimeType.UNIX_SECONDS +} + struct FetchBatchRawResult { 1: required list elements } diff --git a/src/dbnode/generated/thrift/rpc/rpc.go b/src/dbnode/generated/thrift/rpc/rpc.go index f49457f08d..03e762f36a 100644 --- a/src/dbnode/generated/thrift/rpc/rpc.go +++ b/src/dbnode/generated/thrift/rpc/rpc.go @@ -1870,6 +1870,452 @@ func (p *FetchBatchRawRequest) String() string { return fmt.Sprintf("FetchBatchRawRequest(%+v)", *p) } +// Attributes: +// - NameSpaces +// - Elements +type FetchBatchRawV2Request struct { + NameSpaces [][]byte `thrift:"nameSpaces,1,required" db:"nameSpaces" json:"nameSpaces"` + Elements []*FetchBatchRawV2RequestElement `thrift:"elements,2,required" db:"elements" json:"elements"` +} + +func NewFetchBatchRawV2Request() *FetchBatchRawV2Request { + return &FetchBatchRawV2Request{} +} + +func (p *FetchBatchRawV2Request) GetNameSpaces() [][]byte { + return p.NameSpaces +} + +func (p *FetchBatchRawV2Request) GetElements() []*FetchBatchRawV2RequestElement { + return p.Elements +} +func (p *FetchBatchRawV2Request) Read(iprot thrift.TProtocol) error { + if _, err := iprot.ReadStructBegin(); err != nil { + return thrift.PrependError(fmt.Sprintf("%T read error: ", p), err) + } + + var issetNameSpaces bool = false + var issetElements 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 fieldTypeId == thrift.STOP { + break + } + switch fieldId { + case 1: + if err := p.ReadField1(iprot); err != nil { + return err + } + 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 + } + } + 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 !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 *FetchBatchRawV2Request) 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 _elem4 []byte + if v, err := iprot.ReadBinary(); err != nil { + return thrift.PrependError("error reading field 0: ", err) + } else { + _elem4 = v + } + p.NameSpaces = append(p.NameSpaces, _elem4) + } + if err := iprot.ReadListEnd(); err != nil { + return thrift.PrependError("error reading list end: ", err) + } + return nil +} + +func (p *FetchBatchRawV2Request) ReadField2(iprot thrift.TProtocol) error { + _, size, err := iprot.ReadListBegin() + if err != nil { + return thrift.PrependError("error reading list begin: ", err) + } + tSlice := make([]*FetchBatchRawV2RequestElement, 0, size) + p.Elements = tSlice + for i := 0; i < size; i++ { + _elem5 := &FetchBatchRawV2RequestElement{ + RangeTimeType: 0, + } + if err := _elem5.Read(iprot); err != nil { + return thrift.PrependError(fmt.Sprintf("%T error reading struct: ", _elem5), err) + } + p.Elements = append(p.Elements, _elem5) + } + if err := iprot.ReadListEnd(); err != nil { + return thrift.PrependError("error reading list end: ", err) + } + return nil +} + +func (p *FetchBatchRawV2Request) Write(oprot thrift.TProtocol) error { + if err := oprot.WriteStructBegin("FetchBatchRawV2Request"); 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 *FetchBatchRawV2Request) 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.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:nameSpaces: ", p), err) + } + return err +} + +func (p *FetchBatchRawV2Request) 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 *FetchBatchRawV2Request) String() string { + if p == nil { + return "" + } + return fmt.Sprintf("FetchBatchRawV2Request(%+v)", *p) +} + +// Attributes: +// - NameSpace +// - RangeStart +// - RangeEnd +// - ID +// - RangeTimeType +type FetchBatchRawV2RequestElement struct { + NameSpace int64 `thrift:"nameSpace,1,required" db:"nameSpace" json:"nameSpace"` + RangeStart int64 `thrift:"rangeStart,2,required" db:"rangeStart" json:"rangeStart"` + RangeEnd int64 `thrift:"rangeEnd,3,required" db:"rangeEnd" json:"rangeEnd"` + ID []byte `thrift:"id,4,required" db:"id" json:"id"` + RangeTimeType TimeType `thrift:"rangeTimeType,5" db:"rangeTimeType" json:"rangeTimeType,omitempty"` +} + +func NewFetchBatchRawV2RequestElement() *FetchBatchRawV2RequestElement { + return &FetchBatchRawV2RequestElement{ + RangeTimeType: 0, + } +} + +func (p *FetchBatchRawV2RequestElement) GetNameSpace() int64 { + return p.NameSpace +} + +func (p *FetchBatchRawV2RequestElement) GetRangeStart() int64 { + return p.RangeStart +} + +func (p *FetchBatchRawV2RequestElement) GetRangeEnd() int64 { + return p.RangeEnd +} + +func (p *FetchBatchRawV2RequestElement) GetID() []byte { + return p.ID +} + +var FetchBatchRawV2RequestElement_RangeTimeType_DEFAULT TimeType = 0 + +func (p *FetchBatchRawV2RequestElement) GetRangeTimeType() TimeType { + return p.RangeTimeType +} +func (p *FetchBatchRawV2RequestElement) IsSetRangeTimeType() bool { + return p.RangeTimeType != FetchBatchRawV2RequestElement_RangeTimeType_DEFAULT +} + +func (p *FetchBatchRawV2RequestElement) 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 issetRangeStart bool = false + var issetRangeEnd bool = false + var issetID 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 fieldTypeId == thrift.STOP { + break + } + switch fieldId { + case 1: + if err := p.ReadField1(iprot); err != nil { + return err + } + issetNameSpace = 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 + } + issetID = true + case 5: + if err := p.ReadField5(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) + } + if !issetNameSpace { + return thrift.NewTProtocolExceptionWithType(thrift.INVALID_DATA, fmt.Errorf("Required field NameSpace 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 !issetID { + return thrift.NewTProtocolExceptionWithType(thrift.INVALID_DATA, fmt.Errorf("Required field ID is not set")) + } + return nil +} + +func (p *FetchBatchRawV2RequestElement) ReadField1(iprot thrift.TProtocol) error { + if v, err := iprot.ReadI64(); err != nil { + return thrift.PrependError("error reading field 1: ", err) + } else { + p.NameSpace = v + } + return nil +} + +func (p *FetchBatchRawV2RequestElement) 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 *FetchBatchRawV2RequestElement) 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 *FetchBatchRawV2RequestElement) ReadField4(iprot thrift.TProtocol) error { + if v, err := iprot.ReadBinary(); err != nil { + return thrift.PrependError("error reading field 4: ", err) + } else { + p.ID = v + } + return nil +} + +func (p *FetchBatchRawV2RequestElement) ReadField5(iprot thrift.TProtocol) error { + if v, err := iprot.ReadI32(); err != nil { + return thrift.PrependError("error reading field 5: ", err) + } else { + temp := TimeType(v) + p.RangeTimeType = temp + } + return nil +} + +func (p *FetchBatchRawV2RequestElement) Write(oprot thrift.TProtocol) error { + if err := oprot.WriteStructBegin("FetchBatchRawV2RequestElement"); 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 := 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 *FetchBatchRawV2RequestElement) writeField1(oprot thrift.TProtocol) (err error) { + if err := oprot.WriteFieldBegin("nameSpace", thrift.I64, 1); err != nil { + return thrift.PrependError(fmt.Sprintf("%T write field begin error 1:nameSpace: ", p), err) + } + if err := oprot.WriteI64(int64(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 *FetchBatchRawV2RequestElement) 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 *FetchBatchRawV2RequestElement) 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 *FetchBatchRawV2RequestElement) writeField4(oprot thrift.TProtocol) (err error) { + if err := oprot.WriteFieldBegin("id", thrift.STRING, 4); err != nil { + return thrift.PrependError(fmt.Sprintf("%T write field begin error 4:id: ", p), err) + } + if err := oprot.WriteBinary(p.ID); err != nil { + return thrift.PrependError(fmt.Sprintf("%T.id (4) field write error: ", p), err) + } + if err := oprot.WriteFieldEnd(); err != nil { + return thrift.PrependError(fmt.Sprintf("%T write field end error 4:id: ", p), err) + } + return err +} + +func (p *FetchBatchRawV2RequestElement) writeField5(oprot thrift.TProtocol) (err error) { + if p.IsSetRangeTimeType() { + if err := oprot.WriteFieldBegin("rangeTimeType", thrift.I32, 5); err != nil { + return thrift.PrependError(fmt.Sprintf("%T write field begin error 5:rangeTimeType: ", p), err) + } + if err := oprot.WriteI32(int32(p.RangeTimeType)); err != nil { + return thrift.PrependError(fmt.Sprintf("%T.rangeTimeType (5) field write error: ", p), err) + } + if err := oprot.WriteFieldEnd(); err != nil { + return thrift.PrependError(fmt.Sprintf("%T write field end error 5:rangeTimeType: ", p), err) + } + } + return err +} + +func (p *FetchBatchRawV2RequestElement) String() string { + if p == nil { + return "" + } + return fmt.Sprintf("FetchBatchRawV2RequestElement(%+v)", *p) +} + // Attributes: // - Elements type FetchBatchRawResult_ struct { @@ -1930,11 +2376,11 @@ func (p *FetchBatchRawResult_) ReadField1(iprot thrift.TProtocol) error { tSlice := make([]*FetchRawResult_, 0, size) p.Elements = tSlice for i := 0; i < size; i++ { - _elem4 := &FetchRawResult_{} - if err := _elem4.Read(iprot); err != nil { - return thrift.PrependError(fmt.Sprintf("%T error reading struct: ", _elem4), err) + _elem6 := &FetchRawResult_{} + if err := _elem6.Read(iprot); err != nil { + return thrift.PrependError(fmt.Sprintf("%T error reading struct: ", _elem6), err) } - p.Elements = append(p.Elements, _elem4) + p.Elements = append(p.Elements, _elem6) } if err := iprot.ReadListEnd(); err != nil { return thrift.PrependError("error reading list end: ", err) @@ -2067,11 +2513,11 @@ func (p *FetchRawResult_) ReadField1(iprot thrift.TProtocol) error { tSlice := make([]*Segments, 0, size) p.Segments = tSlice for i := 0; i < size; i++ { - _elem5 := &Segments{} - if err := _elem5.Read(iprot); err != nil { - return thrift.PrependError(fmt.Sprintf("%T error reading struct: ", _elem5), err) + _elem7 := &Segments{} + if err := _elem7.Read(iprot); err != nil { + return thrift.PrependError(fmt.Sprintf("%T error reading struct: ", _elem7), err) } - p.Segments = append(p.Segments, _elem5) + p.Segments = append(p.Segments, _elem7) } if err := iprot.ReadListEnd(); err != nil { return thrift.PrependError("error reading list end: ", err) @@ -2240,11 +2686,11 @@ func (p *Segments) ReadField2(iprot thrift.TProtocol) error { tSlice := make([]*Segment, 0, size) p.Unmerged = tSlice for i := 0; i < size; i++ { - _elem6 := &Segment{} - if err := _elem6.Read(iprot); err != nil { - return thrift.PrependError(fmt.Sprintf("%T error reading struct: ", _elem6), err) + _elem8 := &Segment{} + if err := _elem8.Read(iprot); err != nil { + return thrift.PrependError(fmt.Sprintf("%T error reading struct: ", _elem8), err) } - p.Unmerged = append(p.Unmerged, _elem6) + p.Unmerged = append(p.Unmerged, _elem8) } if err := iprot.ReadListEnd(); err != nil { return thrift.PrependError("error reading list end: ", err) @@ -2975,11 +3421,11 @@ func (p *FetchTaggedResult_) ReadField1(iprot thrift.TProtocol) error { tSlice := make([]*FetchTaggedIDResult_, 0, size) p.Elements = tSlice for i := 0; i < size; i++ { - _elem7 := &FetchTaggedIDResult_{} - if err := _elem7.Read(iprot); err != nil { - return thrift.PrependError(fmt.Sprintf("%T error reading struct: ", _elem7), err) + _elem9 := &FetchTaggedIDResult_{} + if err := _elem9.Read(iprot); err != nil { + return thrift.PrependError(fmt.Sprintf("%T error reading struct: ", _elem9), err) } - p.Elements = append(p.Elements, _elem7) + p.Elements = append(p.Elements, _elem9) } if err := iprot.ReadListEnd(); err != nil { return thrift.PrependError("error reading list end: ", err) @@ -3210,11 +3656,11 @@ func (p *FetchTaggedIDResult_) ReadField4(iprot thrift.TProtocol) error { tSlice := make([]*Segments, 0, size) p.Segments = tSlice for i := 0; i < size; i++ { - _elem8 := &Segments{} - if err := _elem8.Read(iprot); err != nil { - return thrift.PrependError(fmt.Sprintf("%T error reading struct: ", _elem8), err) + _elem10 := &Segments{} + if err := _elem10.Read(iprot); err != nil { + return thrift.PrependError(fmt.Sprintf("%T error reading struct: ", _elem10), err) } - p.Segments = append(p.Segments, _elem8) + p.Segments = append(p.Segments, _elem10) } if err := iprot.ReadListEnd(); err != nil { return thrift.PrependError("error reading list end: ", err) @@ -3454,11 +3900,11 @@ func (p *FetchBlocksRawRequest) ReadField3(iprot thrift.TProtocol) error { tSlice := make([]*FetchBlocksRawRequestElement, 0, size) p.Elements = tSlice for i := 0; i < size; i++ { - _elem9 := &FetchBlocksRawRequestElement{} - if err := _elem9.Read(iprot); err != nil { - return thrift.PrependError(fmt.Sprintf("%T error reading struct: ", _elem9), err) + _elem11 := &FetchBlocksRawRequestElement{} + if err := _elem11.Read(iprot); err != nil { + return thrift.PrependError(fmt.Sprintf("%T error reading struct: ", _elem11), err) } - p.Elements = append(p.Elements, _elem9) + p.Elements = append(p.Elements, _elem11) } if err := iprot.ReadListEnd(); err != nil { return thrift.PrependError("error reading list end: ", err) @@ -3628,13 +4074,13 @@ func (p *FetchBlocksRawRequestElement) ReadField2(iprot thrift.TProtocol) error tSlice := make([]int64, 0, size) p.Starts = tSlice for i := 0; i < size; i++ { - var _elem10 int64 + var _elem12 int64 if v, err := iprot.ReadI64(); err != nil { return thrift.PrependError("error reading field 0: ", err) } else { - _elem10 = v + _elem12 = v } - p.Starts = append(p.Starts, _elem10) + p.Starts = append(p.Starts, _elem12) } if err := iprot.ReadListEnd(); err != nil { return thrift.PrependError("error reading list end: ", err) @@ -3764,11 +4210,11 @@ func (p *FetchBlocksRawResult_) ReadField1(iprot thrift.TProtocol) error { tSlice := make([]*Blocks, 0, size) p.Elements = tSlice for i := 0; i < size; i++ { - _elem11 := &Blocks{} - if err := _elem11.Read(iprot); err != nil { - return thrift.PrependError(fmt.Sprintf("%T error reading struct: ", _elem11), err) + _elem13 := &Blocks{} + if err := _elem13.Read(iprot); err != nil { + return thrift.PrependError(fmt.Sprintf("%T error reading struct: ", _elem13), err) } - p.Elements = append(p.Elements, _elem11) + p.Elements = append(p.Elements, _elem13) } if err := iprot.ReadListEnd(); err != nil { return thrift.PrependError("error reading list end: ", err) @@ -3906,11 +4352,11 @@ func (p *Blocks) ReadField2(iprot thrift.TProtocol) error { tSlice := make([]*Block, 0, size) p.Blocks = tSlice for i := 0; i < size; i++ { - _elem12 := &Block{} - if err := _elem12.Read(iprot); err != nil { - return thrift.PrependError(fmt.Sprintf("%T error reading struct: ", _elem12), err) + _elem14 := &Block{} + if err := _elem14.Read(iprot); err != nil { + return thrift.PrependError(fmt.Sprintf("%T error reading struct: ", _elem14), err) } - p.Blocks = append(p.Blocks, _elem12) + p.Blocks = append(p.Blocks, _elem14) } if err := iprot.ReadListEnd(); err != nil { return thrift.PrependError("error reading list end: ", err) @@ -4872,13 +5318,13 @@ func (p *FetchBlocksMetadataRawV2Result_) ReadField1(iprot thrift.TProtocol) err tSlice := make([]*BlockMetadataV2, 0, size) p.Elements = tSlice for i := 0; i < size; i++ { - _elem13 := &BlockMetadataV2{ + _elem15 := &BlockMetadataV2{ LastReadTimeType: 0, } - if err := _elem13.Read(iprot); err != nil { - return thrift.PrependError(fmt.Sprintf("%T error reading struct: ", _elem13), err) + if err := _elem15.Read(iprot); err != nil { + return thrift.PrependError(fmt.Sprintf("%T error reading struct: ", _elem15), err) } - p.Elements = append(p.Elements, _elem13) + p.Elements = append(p.Elements, _elem15) } if err := iprot.ReadListEnd(); err != nil { return thrift.PrependError("error reading list end: ", err) @@ -5456,11 +5902,11 @@ func (p *WriteBatchRawRequest) ReadField2(iprot thrift.TProtocol) error { tSlice := make([]*WriteBatchRawRequestElement, 0, size) p.Elements = tSlice for i := 0; i < size; i++ { - _elem14 := &WriteBatchRawRequestElement{} - if err := _elem14.Read(iprot); err != nil { - return thrift.PrependError(fmt.Sprintf("%T error reading struct: ", _elem14), err) + _elem16 := &WriteBatchRawRequestElement{} + if err := _elem16.Read(iprot); err != nil { + return thrift.PrependError(fmt.Sprintf("%T error reading struct: ", _elem16), err) } - p.Elements = append(p.Elements, _elem14) + p.Elements = append(p.Elements, _elem16) } if err := iprot.ReadListEnd(); err != nil { return thrift.PrependError("error reading list end: ", err) @@ -5605,13 +6051,13 @@ func (p *WriteBatchRawV2Request) ReadField1(iprot thrift.TProtocol) error { tSlice := make([][]byte, 0, size) p.NameSpaces = tSlice for i := 0; i < size; i++ { - var _elem15 []byte + var _elem17 []byte if v, err := iprot.ReadBinary(); err != nil { return thrift.PrependError("error reading field 0: ", err) } else { - _elem15 = v + _elem17 = v } - p.NameSpaces = append(p.NameSpaces, _elem15) + p.NameSpaces = append(p.NameSpaces, _elem17) } if err := iprot.ReadListEnd(); err != nil { return thrift.PrependError("error reading list end: ", err) @@ -5627,11 +6073,11 @@ func (p *WriteBatchRawV2Request) ReadField2(iprot thrift.TProtocol) error { 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) + _elem18 := &WriteBatchRawV2RequestElement{} + if err := _elem18.Read(iprot); err != nil { + return thrift.PrependError(fmt.Sprintf("%T error reading struct: ", _elem18), err) } - p.Elements = append(p.Elements, _elem16) + p.Elements = append(p.Elements, _elem18) } if err := iprot.ReadListEnd(); err != nil { return thrift.PrependError("error reading list end: ", err) @@ -6131,11 +6577,11 @@ func (p *WriteTaggedBatchRawRequest) ReadField2(iprot thrift.TProtocol) error { 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) + _elem19 := &WriteTaggedBatchRawRequestElement{} + if err := _elem19.Read(iprot); err != nil { + return thrift.PrependError(fmt.Sprintf("%T error reading struct: ", _elem19), err) } - p.Elements = append(p.Elements, _elem17) + p.Elements = append(p.Elements, _elem19) } if err := iprot.ReadListEnd(); err != nil { return thrift.PrependError("error reading list end: ", err) @@ -6280,13 +6726,13 @@ func (p *WriteTaggedBatchRawV2Request) ReadField1(iprot thrift.TProtocol) error tSlice := make([][]byte, 0, size) p.NameSpaces = tSlice for i := 0; i < size; i++ { - var _elem18 []byte + var _elem20 []byte if v, err := iprot.ReadBinary(); err != nil { return thrift.PrependError("error reading field 0: ", err) } else { - _elem18 = v + _elem20 = v } - p.NameSpaces = append(p.NameSpaces, _elem18) + p.NameSpaces = append(p.NameSpaces, _elem20) } if err := iprot.ReadListEnd(); err != nil { return thrift.PrependError("error reading list end: ", err) @@ -6302,11 +6748,11 @@ func (p *WriteTaggedBatchRawV2Request) ReadField2(iprot thrift.TProtocol) error 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) + _elem21 := &WriteTaggedBatchRawV2RequestElement{} + if err := _elem21.Read(iprot); err != nil { + return thrift.PrependError(fmt.Sprintf("%T error reading struct: ", _elem21), err) } - p.Elements = append(p.Elements, _elem19) + p.Elements = append(p.Elements, _elem21) } if err := iprot.ReadListEnd(); err != nil { return thrift.PrependError("error reading list end: ", err) @@ -8854,13 +9300,13 @@ func (p *AggregateQueryRawRequest) ReadField6(iprot thrift.TProtocol) error { tSlice := make([][]byte, 0, size) p.TagNameFilter = tSlice for i := 0; i < size; i++ { - var _elem20 []byte + var _elem22 []byte if v, err := iprot.ReadBinary(); err != nil { return thrift.PrependError("error reading field 0: ", err) } else { - _elem20 = v + _elem22 = v } - p.TagNameFilter = append(p.TagNameFilter, _elem20) + p.TagNameFilter = append(p.TagNameFilter, _elem22) } if err := iprot.ReadListEnd(); err != nil { return thrift.PrependError("error reading list end: ", err) @@ -9129,11 +9575,11 @@ func (p *AggregateQueryRawResult_) ReadField1(iprot thrift.TProtocol) error { tSlice := make([]*AggregateQueryRawResultTagNameElement, 0, size) p.Results = tSlice for i := 0; i < size; i++ { - _elem21 := &AggregateQueryRawResultTagNameElement{} - if err := _elem21.Read(iprot); err != nil { - return thrift.PrependError(fmt.Sprintf("%T error reading struct: ", _elem21), err) + _elem23 := &AggregateQueryRawResultTagNameElement{} + if err := _elem23.Read(iprot); err != nil { + return thrift.PrependError(fmt.Sprintf("%T error reading struct: ", _elem23), err) } - p.Results = append(p.Results, _elem21) + p.Results = append(p.Results, _elem23) } if err := iprot.ReadListEnd(); err != nil { return thrift.PrependError("error reading list end: ", err) @@ -9297,11 +9743,11 @@ func (p *AggregateQueryRawResultTagNameElement) ReadField2(iprot thrift.TProtoco tSlice := make([]*AggregateQueryRawResultTagValueElement, 0, size) p.TagValues = tSlice for i := 0; i < size; i++ { - _elem22 := &AggregateQueryRawResultTagValueElement{} - if err := _elem22.Read(iprot); err != nil { - return thrift.PrependError(fmt.Sprintf("%T error reading struct: ", _elem22), err) + _elem24 := &AggregateQueryRawResultTagValueElement{} + if err := _elem24.Read(iprot); err != nil { + return thrift.PrependError(fmt.Sprintf("%T error reading struct: ", _elem24), err) } - p.TagValues = append(p.TagValues, _elem22) + p.TagValues = append(p.TagValues, _elem24) } if err := iprot.ReadListEnd(); err != nil { return thrift.PrependError("error reading list end: ", err) @@ -9696,13 +10142,13 @@ func (p *AggregateQueryRequest) ReadField6(iprot thrift.TProtocol) error { tSlice := make([]string, 0, size) p.TagNameFilter = tSlice for i := 0; i < size; i++ { - var _elem23 string + var _elem25 string if v, err := iprot.ReadString(); err != nil { return thrift.PrependError("error reading field 0: ", err) } else { - _elem23 = v + _elem25 = v } - p.TagNameFilter = append(p.TagNameFilter, _elem23) + p.TagNameFilter = append(p.TagNameFilter, _elem25) } if err := iprot.ReadListEnd(); err != nil { return thrift.PrependError("error reading list end: ", err) @@ -9973,11 +10419,11 @@ func (p *AggregateQueryResult_) ReadField1(iprot thrift.TProtocol) error { tSlice := make([]*AggregateQueryResultTagNameElement, 0, size) p.Results = tSlice for i := 0; i < size; i++ { - _elem24 := &AggregateQueryResultTagNameElement{} - if err := _elem24.Read(iprot); err != nil { - return thrift.PrependError(fmt.Sprintf("%T error reading struct: ", _elem24), err) + _elem26 := &AggregateQueryResultTagNameElement{} + if err := _elem26.Read(iprot); err != nil { + return thrift.PrependError(fmt.Sprintf("%T error reading struct: ", _elem26), err) } - p.Results = append(p.Results, _elem24) + p.Results = append(p.Results, _elem26) } if err := iprot.ReadListEnd(); err != nil { return thrift.PrependError("error reading list end: ", err) @@ -10141,11 +10587,11 @@ func (p *AggregateQueryResultTagNameElement) ReadField2(iprot thrift.TProtocol) tSlice := make([]*AggregateQueryResultTagValueElement, 0, size) p.TagValues = tSlice for i := 0; i < size; i++ { - _elem25 := &AggregateQueryResultTagValueElement{} - if err := _elem25.Read(iprot); err != nil { - return thrift.PrependError(fmt.Sprintf("%T error reading struct: ", _elem25), err) + _elem27 := &AggregateQueryResultTagValueElement{} + if err := _elem27.Read(iprot); err != nil { + return thrift.PrependError(fmt.Sprintf("%T error reading struct: ", _elem27), err) } - p.TagValues = append(p.TagValues, _elem25) + p.TagValues = append(p.TagValues, _elem27) } if err := iprot.ReadListEnd(); err != nil { return thrift.PrependError("error reading list end: ", err) @@ -10802,11 +11248,11 @@ func (p *QueryResult_) ReadField1(iprot thrift.TProtocol) error { tSlice := make([]*QueryResultElement, 0, size) p.Results = tSlice for i := 0; i < size; i++ { - _elem26 := &QueryResultElement{} - if err := _elem26.Read(iprot); err != nil { - return thrift.PrependError(fmt.Sprintf("%T error reading struct: ", _elem26), err) + _elem28 := &QueryResultElement{} + if err := _elem28.Read(iprot); err != nil { + return thrift.PrependError(fmt.Sprintf("%T error reading struct: ", _elem28), err) } - p.Results = append(p.Results, _elem26) + p.Results = append(p.Results, _elem28) } if err := iprot.ReadListEnd(); err != nil { return thrift.PrependError("error reading list end: ", err) @@ -10984,11 +11430,11 @@ func (p *QueryResultElement) ReadField2(iprot thrift.TProtocol) error { tSlice := make([]*Tag, 0, size) p.Tags = tSlice for i := 0; i < size; i++ { - _elem27 := &Tag{} - if err := _elem27.Read(iprot); err != nil { - return thrift.PrependError(fmt.Sprintf("%T error reading struct: ", _elem27), err) + _elem29 := &Tag{} + if err := _elem29.Read(iprot); err != nil { + return thrift.PrependError(fmt.Sprintf("%T error reading struct: ", _elem29), err) } - p.Tags = append(p.Tags, _elem27) + p.Tags = append(p.Tags, _elem29) } if err := iprot.ReadListEnd(); err != nil { return thrift.PrependError("error reading list end: ", err) @@ -11004,13 +11450,13 @@ func (p *QueryResultElement) ReadField3(iprot thrift.TProtocol) error { tSlice := make([]*Datapoint, 0, size) p.Datapoints = tSlice for i := 0; i < size; i++ { - _elem28 := &Datapoint{ + _elem30 := &Datapoint{ TimestampTimeType: 0, } - if err := _elem28.Read(iprot); err != nil { - return thrift.PrependError(fmt.Sprintf("%T error reading struct: ", _elem28), err) + if err := _elem30.Read(iprot); err != nil { + return thrift.PrependError(fmt.Sprintf("%T error reading struct: ", _elem30), err) } - p.Datapoints = append(p.Datapoints, _elem28) + p.Datapoints = append(p.Datapoints, _elem30) } if err := iprot.ReadListEnd(); err != nil { return thrift.PrependError("error reading list end: ", err) @@ -11549,11 +11995,11 @@ func (p *ConjunctionQuery) ReadField1(iprot thrift.TProtocol) error { 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) + _elem31 := &Query{} + if err := _elem31.Read(iprot); err != nil { + return thrift.PrependError(fmt.Sprintf("%T error reading struct: ", _elem31), err) } - p.Queries = append(p.Queries, _elem29) + p.Queries = append(p.Queries, _elem31) } if err := iprot.ReadListEnd(); err != nil { return thrift.PrependError("error reading list end: ", err) @@ -11667,11 +12113,11 @@ func (p *DisjunctionQuery) ReadField1(iprot thrift.TProtocol) error { 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) + _elem32 := &Query{} + if err := _elem32.Read(iprot); err != nil { + return thrift.PrependError(fmt.Sprintf("%T error reading struct: ", _elem32), err) } - p.Queries = append(p.Queries, _elem30) + p.Queries = append(p.Queries, _elem32) } if err := iprot.ReadListEnd(); err != nil { return thrift.PrependError("error reading list end: ", err) @@ -12279,6 +12725,9 @@ type Node interface { FetchBatchRaw(req *FetchBatchRawRequest) (r *FetchBatchRawResult_, err error) // Parameters: // - Req + FetchBatchRawV2(req *FetchBatchRawV2Request) (r *FetchBatchRawResult_, err error) + // Parameters: + // - Req FetchBlocksRaw(req *FetchBlocksRawRequest) (r *FetchBlocksRawResult_, err error) // Parameters: // - Req @@ -12396,16 +12845,16 @@ func (p *NodeClient) recvQuery() (value *QueryResult_, err error) { return } if mTypeId == thrift.EXCEPTION { - error31 := thrift.NewTApplicationException(thrift.UNKNOWN_APPLICATION_EXCEPTION, "Unknown Exception") - var error32 error - error32, err = error31.Read(iprot) + 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 = error32 + err = error34 return } if mTypeId != thrift.REPLY { @@ -12477,16 +12926,16 @@ func (p *NodeClient) recvAggregateRaw() (value *AggregateQueryRawResult_, err er return } if mTypeId == thrift.EXCEPTION { - error33 := thrift.NewTApplicationException(thrift.UNKNOWN_APPLICATION_EXCEPTION, "Unknown Exception") - var error34 error - error34, err = error33.Read(iprot) + 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 = error34 + err = error36 return } if mTypeId != thrift.REPLY { @@ -12558,16 +13007,16 @@ func (p *NodeClient) recvAggregate() (value *AggregateQueryResult_, err error) { return } if mTypeId == thrift.EXCEPTION { - error35 := thrift.NewTApplicationException(thrift.UNKNOWN_APPLICATION_EXCEPTION, "Unknown Exception") - var error36 error - error36, err = error35.Read(iprot) + 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 = error36 + err = error38 return } if mTypeId != thrift.REPLY { @@ -12639,16 +13088,16 @@ func (p *NodeClient) recvFetch() (value *FetchResult_, err error) { return } if mTypeId == thrift.EXCEPTION { - error37 := thrift.NewTApplicationException(thrift.UNKNOWN_APPLICATION_EXCEPTION, "Unknown Exception") - var error38 error - error38, err = error37.Read(iprot) + 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 = error38 + err = error40 return } if mTypeId != thrift.REPLY { @@ -12720,16 +13169,16 @@ func (p *NodeClient) recvFetchTagged() (value *FetchTaggedResult_, err error) { return } if mTypeId == thrift.EXCEPTION { - error39 := thrift.NewTApplicationException(thrift.UNKNOWN_APPLICATION_EXCEPTION, "Unknown Exception") - var error40 error - error40, err = error39.Read(iprot) + 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 = error40 + err = error42 return } if mTypeId != thrift.REPLY { @@ -12801,23 +13250,103 @@ func (p *NodeClient) recvWrite() (err error) { return } if mTypeId == thrift.EXCEPTION { - error41 := thrift.NewTApplicationException(thrift.UNKNOWN_APPLICATION_EXCEPTION, "Unknown Exception") - var error42 error - error42, err = error41.Read(iprot) + 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, "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 { + 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 = error42 + err = error46 return } if mTypeId != thrift.REPLY { - err = thrift.NewTApplicationException(thrift.INVALID_MESSAGE_TYPE_EXCEPTION, "write failed: invalid message type") + err = thrift.NewTApplicationException(thrift.INVALID_MESSAGE_TYPE_EXCEPTION, "writeTagged failed: invalid message type") return } - result := NodeWriteResult{} + result := NodeWriteTaggedResult{} if err = result.Read(iprot); err != nil { return } @@ -12833,24 +13362,24 @@ func (p *NodeClient) recvWrite() (err error) { // Parameters: // - Req -func (p *NodeClient) WriteTagged(req *WriteTaggedRequest) (err error) { - if err = p.sendWriteTagged(req); err != nil { +func (p *NodeClient) FetchBatchRaw(req *FetchBatchRawRequest) (r *FetchBatchRawResult_, err error) { + if err = p.sendFetchBatchRaw(req); err != nil { return } - return p.recvWriteTagged() + return p.recvFetchBatchRaw() } -func (p *NodeClient) sendWriteTagged(req *WriteTaggedRequest) (err error) { +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("writeTagged", thrift.CALL, p.SeqId); err != nil { + if err = oprot.WriteMessageBegin("fetchBatchRaw", thrift.CALL, p.SeqId); err != nil { return } - args := NodeWriteTaggedArgs{ + args := NodeFetchBatchRawArgs{ Req: req, } if err = args.Write(oprot); err != nil { @@ -12862,7 +13391,7 @@ func (p *NodeClient) sendWriteTagged(req *WriteTaggedRequest) (err error) { return oprot.Flush() } -func (p *NodeClient) recvWriteTagged() (err error) { +func (p *NodeClient) recvFetchBatchRaw() (value *FetchBatchRawResult_, err error) { iprot := p.InputProtocol if iprot == nil { iprot = p.ProtocolFactory.GetProtocol(p.Transport) @@ -12872,32 +13401,32 @@ func (p *NodeClient) recvWriteTagged() (err error) { if err != nil { return } - if method != "writeTagged" { - err = thrift.NewTApplicationException(thrift.WRONG_METHOD_NAME, "writeTagged failed: wrong method name") + 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, "writeTagged failed: out of sequence response") + err = thrift.NewTApplicationException(thrift.BAD_SEQUENCE_ID, "fetchBatchRaw 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) + 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 = error44 + err = error48 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, "fetchBatchRaw failed: invalid message type") return } - result := NodeWriteTaggedResult{} + result := NodeFetchBatchRawResult{} if err = result.Read(iprot); err != nil { return } @@ -12908,29 +13437,30 @@ func (p *NodeClient) recvWriteTagged() (err error) { err = result.Err return } + value = result.GetSuccess() return } // Parameters: // - Req -func (p *NodeClient) FetchBatchRaw(req *FetchBatchRawRequest) (r *FetchBatchRawResult_, err error) { - if err = p.sendFetchBatchRaw(req); err != nil { +func (p *NodeClient) FetchBatchRawV2(req *FetchBatchRawV2Request) (r *FetchBatchRawResult_, err error) { + if err = p.sendFetchBatchRawV2(req); err != nil { return } - return p.recvFetchBatchRaw() + return p.recvFetchBatchRawV2() } -func (p *NodeClient) sendFetchBatchRaw(req *FetchBatchRawRequest) (err error) { +func (p *NodeClient) sendFetchBatchRawV2(req *FetchBatchRawV2Request) (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("fetchBatchRawV2", thrift.CALL, p.SeqId); err != nil { return } - args := NodeFetchBatchRawArgs{ + args := NodeFetchBatchRawV2Args{ Req: req, } if err = args.Write(oprot); err != nil { @@ -12942,7 +13472,7 @@ func (p *NodeClient) sendFetchBatchRaw(req *FetchBatchRawRequest) (err error) { return oprot.Flush() } -func (p *NodeClient) recvFetchBatchRaw() (value *FetchBatchRawResult_, err error) { +func (p *NodeClient) recvFetchBatchRawV2() (value *FetchBatchRawResult_, err error) { iprot := p.InputProtocol if iprot == nil { iprot = p.ProtocolFactory.GetProtocol(p.Transport) @@ -12952,32 +13482,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 != "fetchBatchRawV2" { + err = thrift.NewTApplicationException(thrift.WRONG_METHOD_NAME, "fetchBatchRawV2 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, "fetchBatchRawV2 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) + 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 = error46 + err = error50 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, "fetchBatchRawV2 failed: invalid message type") return } - result := NodeFetchBatchRawResult{} + result := NodeFetchBatchRawV2Result{} if err = result.Read(iprot); err != nil { return } @@ -13042,16 +13572,16 @@ func (p *NodeClient) recvFetchBlocksRaw() (value *FetchBlocksRawResult_, err err return } if mTypeId == thrift.EXCEPTION { - error47 := thrift.NewTApplicationException(thrift.UNKNOWN_APPLICATION_EXCEPTION, "Unknown Exception") - var error48 error - error48, err = error47.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 = error48 + err = error52 return } if mTypeId != thrift.REPLY { @@ -13123,16 +13653,16 @@ func (p *NodeClient) recvFetchBlocksMetadataRawV2() (value *FetchBlocksMetadataR return } if mTypeId == thrift.EXCEPTION { - error49 := thrift.NewTApplicationException(thrift.UNKNOWN_APPLICATION_EXCEPTION, "Unknown Exception") - var error50 error - error50, err = error49.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 = error50 + err = error54 return } if mTypeId != thrift.REPLY { @@ -13204,16 +13734,16 @@ func (p *NodeClient) recvWriteBatchRaw() (err error) { return } if mTypeId == thrift.EXCEPTION { - error51 := thrift.NewTApplicationException(thrift.UNKNOWN_APPLICATION_EXCEPTION, "Unknown Exception") - var error52 error - error52, err = error51.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 = error52 + err = error56 return } if mTypeId != thrift.REPLY { @@ -13284,16 +13814,16 @@ func (p *NodeClient) recvWriteBatchRawV2() (err error) { return } if mTypeId == thrift.EXCEPTION { - error53 := thrift.NewTApplicationException(thrift.UNKNOWN_APPLICATION_EXCEPTION, "Unknown Exception") - var error54 error - error54, err = error53.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 = error54 + err = error58 return } if mTypeId != thrift.REPLY { @@ -13364,16 +13894,16 @@ func (p *NodeClient) recvWriteTaggedBatchRaw() (err error) { return } if mTypeId == thrift.EXCEPTION { - error55 := thrift.NewTApplicationException(thrift.UNKNOWN_APPLICATION_EXCEPTION, "Unknown Exception") - var error56 error - error56, err = error55.Read(iprot) + 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 = error56 + err = error60 return } if mTypeId != thrift.REPLY { @@ -13444,16 +13974,16 @@ func (p *NodeClient) recvWriteTaggedBatchRawV2() (err error) { return } if mTypeId == thrift.EXCEPTION { - error57 := thrift.NewTApplicationException(thrift.UNKNOWN_APPLICATION_EXCEPTION, "Unknown Exception") - var error58 error - error58, err = error57.Read(iprot) + 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 = error58 + err = error62 return } if mTypeId != thrift.REPLY { @@ -13520,16 +14050,16 @@ func (p *NodeClient) recvRepair() (err error) { return } if mTypeId == thrift.EXCEPTION { - error59 := thrift.NewTApplicationException(thrift.UNKNOWN_APPLICATION_EXCEPTION, "Unknown Exception") - var error60 error - error60, err = error59.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 = error60 + err = error64 return } if mTypeId != thrift.REPLY { @@ -13600,16 +14130,16 @@ func (p *NodeClient) recvTruncate() (value *TruncateResult_, err error) { return } if mTypeId == thrift.EXCEPTION { - error61 := thrift.NewTApplicationException(thrift.UNKNOWN_APPLICATION_EXCEPTION, "Unknown Exception") - var error62 error - error62, err = error61.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 = error62 + err = error66 return } if mTypeId != thrift.REPLY { @@ -13677,16 +14207,16 @@ func (p *NodeClient) recvHealth() (value *NodeHealthResult_, err error) { return } if mTypeId == thrift.EXCEPTION { - error63 := thrift.NewTApplicationException(thrift.UNKNOWN_APPLICATION_EXCEPTION, "Unknown Exception") - var error64 error - error64, err = error63.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 = error64 + err = error68 return } if mTypeId != thrift.REPLY { @@ -13754,16 +14284,16 @@ func (p *NodeClient) recvBootstrapped() (value *NodeBootstrappedResult_, err err return } if mTypeId == thrift.EXCEPTION { - error65 := thrift.NewTApplicationException(thrift.UNKNOWN_APPLICATION_EXCEPTION, "Unknown Exception") - var error66 error - error66, err = error65.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 = error66 + err = error70 return } if mTypeId != thrift.REPLY { @@ -13831,16 +14361,16 @@ func (p *NodeClient) recvBootstrappedInPlacementOrNoPlacement() (value *NodeBoot return } if mTypeId == thrift.EXCEPTION { - error67 := thrift.NewTApplicationException(thrift.UNKNOWN_APPLICATION_EXCEPTION, "Unknown Exception") - var error68 error - error68, err = error67.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 = error68 + err = error72 return } if mTypeId != thrift.REPLY { @@ -13908,16 +14438,16 @@ func (p *NodeClient) recvGetPersistRateLimit() (value *NodePersistRateLimitResul return } if mTypeId == thrift.EXCEPTION { - error69 := thrift.NewTApplicationException(thrift.UNKNOWN_APPLICATION_EXCEPTION, "Unknown Exception") - var error70 error - error70, err = error69.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 = error70 + err = error74 return } if mTypeId != thrift.REPLY { @@ -13989,16 +14519,16 @@ func (p *NodeClient) recvSetPersistRateLimit() (value *NodePersistRateLimitResul return } if mTypeId == thrift.EXCEPTION { - error71 := thrift.NewTApplicationException(thrift.UNKNOWN_APPLICATION_EXCEPTION, "Unknown Exception") - var error72 error - error72, err = error71.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 = error72 + err = error76 return } if mTypeId != thrift.REPLY { @@ -14066,16 +14596,16 @@ func (p *NodeClient) recvGetWriteNewSeriesAsync() (value *NodeWriteNewSeriesAsyn return } if mTypeId == thrift.EXCEPTION { - error73 := thrift.NewTApplicationException(thrift.UNKNOWN_APPLICATION_EXCEPTION, "Unknown Exception") - var error74 error - error74, err = error73.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 = error74 + err = error78 return } if mTypeId != thrift.REPLY { @@ -14147,16 +14677,16 @@ func (p *NodeClient) recvSetWriteNewSeriesAsync() (value *NodeWriteNewSeriesAsyn return } if mTypeId == thrift.EXCEPTION { - error75 := thrift.NewTApplicationException(thrift.UNKNOWN_APPLICATION_EXCEPTION, "Unknown Exception") - var error76 error - error76, err = error75.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 = error76 + err = error80 return } if mTypeId != thrift.REPLY { @@ -14224,16 +14754,16 @@ func (p *NodeClient) recvGetWriteNewSeriesBackoffDuration() (value *NodeWriteNew return } if mTypeId == thrift.EXCEPTION { - error77 := thrift.NewTApplicationException(thrift.UNKNOWN_APPLICATION_EXCEPTION, "Unknown Exception") - var error78 error - error78, err = error77.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 = error78 + err = error82 return } if mTypeId != thrift.REPLY { @@ -14305,16 +14835,16 @@ func (p *NodeClient) recvSetWriteNewSeriesBackoffDuration() (value *NodeWriteNew return } if mTypeId == thrift.EXCEPTION { - error79 := thrift.NewTApplicationException(thrift.UNKNOWN_APPLICATION_EXCEPTION, "Unknown Exception") - var error80 error - error80, err = error79.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 = error80 + err = error84 return } if mTypeId != thrift.REPLY { @@ -14382,16 +14912,16 @@ func (p *NodeClient) recvGetWriteNewSeriesLimitPerShardPerSecond() (value *NodeW return } if mTypeId == thrift.EXCEPTION { - error81 := thrift.NewTApplicationException(thrift.UNKNOWN_APPLICATION_EXCEPTION, "Unknown Exception") - var error82 error - error82, err = error81.Read(iprot) + error85 := thrift.NewTApplicationException(thrift.UNKNOWN_APPLICATION_EXCEPTION, "Unknown Exception") + var error86 error + error86, err = error85.Read(iprot) if err != nil { return } if err = iprot.ReadMessageEnd(); err != nil { return } - err = error82 + err = error86 return } if mTypeId != thrift.REPLY { @@ -14463,16 +14993,16 @@ func (p *NodeClient) recvSetWriteNewSeriesLimitPerShardPerSecond() (value *NodeW return } if mTypeId == thrift.EXCEPTION { - error83 := thrift.NewTApplicationException(thrift.UNKNOWN_APPLICATION_EXCEPTION, "Unknown Exception") - var error84 error - error84, err = error83.Read(iprot) + error87 := thrift.NewTApplicationException(thrift.UNKNOWN_APPLICATION_EXCEPTION, "Unknown Exception") + var error88 error + error88, err = error87.Read(iprot) if err != nil { return } if err = iprot.ReadMessageEnd(); err != nil { return } - err = error84 + err = error88 return } if mTypeId != thrift.REPLY { @@ -14514,35 +15044,36 @@ func (p *NodeProcessor) ProcessorMap() map[string]thrift.TProcessorFunction { 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 + self89 := &NodeProcessor{handler: handler, processorMap: make(map[string]thrift.TProcessorFunction)} + self89.processorMap["query"] = &nodeProcessorQuery{handler: handler} + self89.processorMap["aggregateRaw"] = &nodeProcessorAggregateRaw{handler: handler} + self89.processorMap["aggregate"] = &nodeProcessorAggregate{handler: handler} + self89.processorMap["fetch"] = &nodeProcessorFetch{handler: handler} + self89.processorMap["fetchTagged"] = &nodeProcessorFetchTagged{handler: handler} + self89.processorMap["write"] = &nodeProcessorWrite{handler: handler} + self89.processorMap["writeTagged"] = &nodeProcessorWriteTagged{handler: handler} + self89.processorMap["fetchBatchRaw"] = &nodeProcessorFetchBatchRaw{handler: handler} + self89.processorMap["fetchBatchRawV2"] = &nodeProcessorFetchBatchRawV2{handler: handler} + self89.processorMap["fetchBlocksRaw"] = &nodeProcessorFetchBlocksRaw{handler: handler} + self89.processorMap["fetchBlocksMetadataRawV2"] = &nodeProcessorFetchBlocksMetadataRawV2{handler: handler} + self89.processorMap["writeBatchRaw"] = &nodeProcessorWriteBatchRaw{handler: handler} + self89.processorMap["writeBatchRawV2"] = &nodeProcessorWriteBatchRawV2{handler: handler} + self89.processorMap["writeTaggedBatchRaw"] = &nodeProcessorWriteTaggedBatchRaw{handler: handler} + self89.processorMap["writeTaggedBatchRawV2"] = &nodeProcessorWriteTaggedBatchRawV2{handler: handler} + self89.processorMap["repair"] = &nodeProcessorRepair{handler: handler} + self89.processorMap["truncate"] = &nodeProcessorTruncate{handler: handler} + self89.processorMap["health"] = &nodeProcessorHealth{handler: handler} + self89.processorMap["bootstrapped"] = &nodeProcessorBootstrapped{handler: handler} + self89.processorMap["bootstrappedInPlacementOrNoPlacement"] = &nodeProcessorBootstrappedInPlacementOrNoPlacement{handler: handler} + self89.processorMap["getPersistRateLimit"] = &nodeProcessorGetPersistRateLimit{handler: handler} + self89.processorMap["setPersistRateLimit"] = &nodeProcessorSetPersistRateLimit{handler: handler} + self89.processorMap["getWriteNewSeriesAsync"] = &nodeProcessorGetWriteNewSeriesAsync{handler: handler} + self89.processorMap["setWriteNewSeriesAsync"] = &nodeProcessorSetWriteNewSeriesAsync{handler: handler} + self89.processorMap["getWriteNewSeriesBackoffDuration"] = &nodeProcessorGetWriteNewSeriesBackoffDuration{handler: handler} + self89.processorMap["setWriteNewSeriesBackoffDuration"] = &nodeProcessorSetWriteNewSeriesBackoffDuration{handler: handler} + self89.processorMap["getWriteNewSeriesLimitPerShardPerSecond"] = &nodeProcessorGetWriteNewSeriesLimitPerShardPerSecond{handler: handler} + self89.processorMap["setWriteNewSeriesLimitPerShardPerSecond"] = &nodeProcessorSetWriteNewSeriesLimitPerShardPerSecond{handler: handler} + return self89 } func (p *NodeProcessor) Process(iprot, oprot thrift.TProtocol) (success bool, err thrift.TException) { @@ -14555,12 +15086,12 @@ func (p *NodeProcessor) Process(iprot, oprot thrift.TProtocol) (success bool, er } iprot.Skip(thrift.STRUCT) iprot.ReadMessageEnd() - x86 := thrift.NewTApplicationException(thrift.UNKNOWN_METHOD, "Unknown function "+name) + x90 := thrift.NewTApplicationException(thrift.UNKNOWN_METHOD, "Unknown function "+name) oprot.WriteMessageBegin(name, thrift.EXCEPTION, seqId) - x86.Write(oprot) + x90.Write(oprot) oprot.WriteMessageEnd() oprot.Flush() - return false, x86 + return false, x90 } @@ -14888,7 +15419,57 @@ func (p *nodeProcessorWriteTagged) Process(seqId int32, iprot, oprot thrift.TPro 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("writeTagged", thrift.EXCEPTION, seqId) + x.Write(oprot) + oprot.WriteMessageEnd() + oprot.Flush() + return false, err + } + + 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 + } + } + if err2 = oprot.WriteMessageBegin("writeTagged", 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 + } + return true, err +} + +type nodeProcessorFetchBatchRaw struct { + handler Node +} + +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("fetchBatchRaw", thrift.EXCEPTION, seqId) x.Write(oprot) oprot.WriteMessageEnd() oprot.Flush() @@ -14896,22 +15477,25 @@ func (p *nodeProcessorWriteTagged) Process(seqId int32, iprot, oprot thrift.TPro } iprot.ReadMessageEnd() - result := NodeWriteTaggedResult{} + result := NodeFetchBatchRawResult{} + var retval *FetchBatchRawResult_ var err2 error - if err2 = p.handler.WriteTagged(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 writeTagged: "+err2.Error()) - oprot.WriteMessageBegin("writeTagged", 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() return true, err2 } + } else { + result.Success = retval } - if err2 = oprot.WriteMessageBegin("writeTagged", 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 { @@ -14929,16 +15513,16 @@ func (p *nodeProcessorWriteTagged) Process(seqId int32, iprot, oprot thrift.TPro return true, err } -type nodeProcessorFetchBatchRaw struct { +type nodeProcessorFetchBatchRawV2 struct { handler Node } -func (p *nodeProcessorFetchBatchRaw) Process(seqId int32, iprot, oprot thrift.TProtocol) (success bool, err thrift.TException) { - args := NodeFetchBatchRawArgs{} +func (p *nodeProcessorFetchBatchRawV2) Process(seqId int32, iprot, oprot thrift.TProtocol) (success bool, err thrift.TException) { + args := NodeFetchBatchRawV2Args{} 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("fetchBatchRawV2", thrift.EXCEPTION, seqId) x.Write(oprot) oprot.WriteMessageEnd() oprot.Flush() @@ -14946,16 +15530,16 @@ func (p *nodeProcessorFetchBatchRaw) Process(seqId int32, iprot, oprot thrift.TP } iprot.ReadMessageEnd() - result := NodeFetchBatchRawResult{} + result := NodeFetchBatchRawV2Result{} var retval *FetchBatchRawResult_ var err2 error - if retval, err2 = p.handler.FetchBatchRaw(args.Req); err2 != nil { + if retval, err2 = p.handler.FetchBatchRawV2(args.Req); 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 fetchBatchRawV2: "+err2.Error()) + oprot.WriteMessageBegin("fetchBatchRawV2", thrift.EXCEPTION, seqId) x.Write(oprot) oprot.WriteMessageEnd() oprot.Flush() @@ -14964,7 +15548,7 @@ func (p *nodeProcessorFetchBatchRaw) Process(seqId int32, iprot, oprot thrift.TP } else { result.Success = retval } - if err2 = oprot.WriteMessageBegin("fetchBatchRaw", thrift.REPLY, seqId); err2 != nil { + if err2 = oprot.WriteMessageBegin("fetchBatchRawV2", thrift.REPLY, seqId); err2 != nil { err = err2 } if err2 = result.Write(oprot); err == nil && err2 != nil { @@ -17914,6 +18498,257 @@ func (p *NodeFetchBatchRawResult) String() string { return fmt.Sprintf("NodeFetchBatchRawResult(%+v)", *p) } +// Attributes: +// - Req +type NodeFetchBatchRawV2Args struct { + Req *FetchBatchRawV2Request `thrift:"req,1" db:"req" json:"req"` +} + +func NewNodeFetchBatchRawV2Args() *NodeFetchBatchRawV2Args { + return &NodeFetchBatchRawV2Args{} +} + +var NodeFetchBatchRawV2Args_Req_DEFAULT *FetchBatchRawV2Request + +func (p *NodeFetchBatchRawV2Args) GetReq() *FetchBatchRawV2Request { + if !p.IsSetReq() { + return NodeFetchBatchRawV2Args_Req_DEFAULT + } + return p.Req +} +func (p *NodeFetchBatchRawV2Args) IsSetReq() bool { + return p.Req != nil +} + +func (p *NodeFetchBatchRawV2Args) 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 *NodeFetchBatchRawV2Args) ReadField1(iprot thrift.TProtocol) error { + p.Req = &FetchBatchRawV2Request{} + if err := p.Req.Read(iprot); err != nil { + return thrift.PrependError(fmt.Sprintf("%T error reading struct: ", p.Req), err) + } + return nil +} + +func (p *NodeFetchBatchRawV2Args) Write(oprot thrift.TProtocol) error { + if err := oprot.WriteStructBegin("fetchBatchRawV2_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 *NodeFetchBatchRawV2Args) 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 *NodeFetchBatchRawV2Args) String() string { + if p == nil { + return "" + } + return fmt.Sprintf("NodeFetchBatchRawV2Args(%+v)", *p) +} + +// Attributes: +// - Success +// - Err +type NodeFetchBatchRawV2Result struct { + Success *FetchBatchRawResult_ `thrift:"success,0" db:"success" json:"success,omitempty"` + Err *Error `thrift:"err,1" db:"err" json:"err,omitempty"` +} + +func NewNodeFetchBatchRawV2Result() *NodeFetchBatchRawV2Result { + return &NodeFetchBatchRawV2Result{} +} + +var NodeFetchBatchRawV2Result_Success_DEFAULT *FetchBatchRawResult_ + +func (p *NodeFetchBatchRawV2Result) GetSuccess() *FetchBatchRawResult_ { + if !p.IsSetSuccess() { + return NodeFetchBatchRawV2Result_Success_DEFAULT + } + return p.Success +} + +var NodeFetchBatchRawV2Result_Err_DEFAULT *Error + +func (p *NodeFetchBatchRawV2Result) GetErr() *Error { + if !p.IsSetErr() { + return NodeFetchBatchRawV2Result_Err_DEFAULT + } + return p.Err +} +func (p *NodeFetchBatchRawV2Result) IsSetSuccess() bool { + return p.Success != nil +} + +func (p *NodeFetchBatchRawV2Result) IsSetErr() bool { + return p.Err != nil +} + +func (p *NodeFetchBatchRawV2Result) 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 nil +} + +func (p *NodeFetchBatchRawV2Result) 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 *NodeFetchBatchRawV2Result) 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 +} + +func (p *NodeFetchBatchRawV2Result) Write(oprot thrift.TProtocol) error { + if err := oprot.WriteStructBegin("fetchBatchRawV2_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 + } + } + 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 *NodeFetchBatchRawV2Result) 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 *NodeFetchBatchRawV2Result) 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 err +} + +func (p *NodeFetchBatchRawV2Result) String() string { + if p == nil { + return "" + } + return fmt.Sprintf("NodeFetchBatchRawV2Result(%+v)", *p) +} + // Attributes: // - Req type NodeFetchBlocksRawArgs struct { @@ -22178,16 +23013,16 @@ func (p *ClusterClient) recvHealth() (value *HealthResult_, err error) { return } if mTypeId == thrift.EXCEPTION { - error201 := thrift.NewTApplicationException(thrift.UNKNOWN_APPLICATION_EXCEPTION, "Unknown Exception") - var error202 error - error202, err = error201.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 = error202 + err = error212 return } if mTypeId != thrift.REPLY { @@ -22259,16 +23094,16 @@ func (p *ClusterClient) recvWrite() (err error) { return } if mTypeId == thrift.EXCEPTION { - error203 := thrift.NewTApplicationException(thrift.UNKNOWN_APPLICATION_EXCEPTION, "Unknown Exception") - var error204 error - error204, err = error203.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 = error204 + err = error214 return } if mTypeId != thrift.REPLY { @@ -22339,16 +23174,16 @@ func (p *ClusterClient) recvWriteTagged() (err error) { return } if mTypeId == thrift.EXCEPTION { - error205 := thrift.NewTApplicationException(thrift.UNKNOWN_APPLICATION_EXCEPTION, "Unknown Exception") - var error206 error - error206, err = error205.Read(iprot) + error215 := thrift.NewTApplicationException(thrift.UNKNOWN_APPLICATION_EXCEPTION, "Unknown Exception") + var error216 error + error216, err = error215.Read(iprot) if err != nil { return } if err = iprot.ReadMessageEnd(); err != nil { return } - err = error206 + err = error216 return } if mTypeId != thrift.REPLY { @@ -22419,16 +23254,16 @@ func (p *ClusterClient) recvQuery() (value *QueryResult_, err error) { return } if mTypeId == thrift.EXCEPTION { - error207 := thrift.NewTApplicationException(thrift.UNKNOWN_APPLICATION_EXCEPTION, "Unknown Exception") - var error208 error - error208, err = error207.Read(iprot) + error217 := thrift.NewTApplicationException(thrift.UNKNOWN_APPLICATION_EXCEPTION, "Unknown Exception") + var error218 error + error218, err = error217.Read(iprot) if err != nil { return } if err = iprot.ReadMessageEnd(); err != nil { return } - err = error208 + err = error218 return } if mTypeId != thrift.REPLY { @@ -22500,16 +23335,16 @@ func (p *ClusterClient) recvAggregate() (value *AggregateQueryResult_, err error return } if mTypeId == thrift.EXCEPTION { - error209 := thrift.NewTApplicationException(thrift.UNKNOWN_APPLICATION_EXCEPTION, "Unknown Exception") - var error210 error - error210, err = error209.Read(iprot) + error219 := thrift.NewTApplicationException(thrift.UNKNOWN_APPLICATION_EXCEPTION, "Unknown Exception") + var error220 error + error220, err = error219.Read(iprot) if err != nil { return } if err = iprot.ReadMessageEnd(); err != nil { return } - err = error210 + err = error220 return } if mTypeId != thrift.REPLY { @@ -22581,16 +23416,16 @@ func (p *ClusterClient) recvFetch() (value *FetchResult_, err error) { return } if mTypeId == thrift.EXCEPTION { - error211 := thrift.NewTApplicationException(thrift.UNKNOWN_APPLICATION_EXCEPTION, "Unknown Exception") - var error212 error - error212, err = error211.Read(iprot) + error221 := thrift.NewTApplicationException(thrift.UNKNOWN_APPLICATION_EXCEPTION, "Unknown Exception") + var error222 error + error222, err = error221.Read(iprot) if err != nil { return } if err = iprot.ReadMessageEnd(); err != nil { return } - err = error212 + err = error222 return } if mTypeId != thrift.REPLY { @@ -22662,16 +23497,16 @@ func (p *ClusterClient) recvTruncate() (value *TruncateResult_, err error) { return } if mTypeId == thrift.EXCEPTION { - error213 := thrift.NewTApplicationException(thrift.UNKNOWN_APPLICATION_EXCEPTION, "Unknown Exception") - var error214 error - error214, err = error213.Read(iprot) + error223 := thrift.NewTApplicationException(thrift.UNKNOWN_APPLICATION_EXCEPTION, "Unknown Exception") + var error224 error + error224, err = error223.Read(iprot) if err != nil { return } if err = iprot.ReadMessageEnd(); err != nil { return } - err = error214 + err = error224 return } if mTypeId != thrift.REPLY { @@ -22713,15 +23548,15 @@ func (p *ClusterProcessor) ProcessorMap() map[string]thrift.TProcessorFunction { func NewClusterProcessor(handler Cluster) *ClusterProcessor { - 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 + self225 := &ClusterProcessor{handler: handler, processorMap: make(map[string]thrift.TProcessorFunction)} + self225.processorMap["health"] = &clusterProcessorHealth{handler: handler} + self225.processorMap["write"] = &clusterProcessorWrite{handler: handler} + self225.processorMap["writeTagged"] = &clusterProcessorWriteTagged{handler: handler} + self225.processorMap["query"] = &clusterProcessorQuery{handler: handler} + self225.processorMap["aggregate"] = &clusterProcessorAggregate{handler: handler} + self225.processorMap["fetch"] = &clusterProcessorFetch{handler: handler} + self225.processorMap["truncate"] = &clusterProcessorTruncate{handler: handler} + return self225 } func (p *ClusterProcessor) Process(iprot, oprot thrift.TProtocol) (success bool, err thrift.TException) { @@ -22734,12 +23569,12 @@ func (p *ClusterProcessor) Process(iprot, oprot thrift.TProtocol) (success bool, } iprot.Skip(thrift.STRUCT) iprot.ReadMessageEnd() - x216 := thrift.NewTApplicationException(thrift.UNKNOWN_METHOD, "Unknown function "+name) + x226 := thrift.NewTApplicationException(thrift.UNKNOWN_METHOD, "Unknown function "+name) oprot.WriteMessageBegin(name, thrift.EXCEPTION, seqId) - x216.Write(oprot) + x226.Write(oprot) oprot.WriteMessageEnd() oprot.Flush() - return false, x216 + return false, x226 } diff --git a/src/dbnode/generated/thrift/rpc/rpc_mock.go b/src/dbnode/generated/thrift/rpc/rpc_mock.go index 68598ee7d1..b0114716a3 100644 --- a/src/dbnode/generated/thrift/rpc/rpc_mock.go +++ b/src/dbnode/generated/thrift/rpc/rpc_mock.go @@ -270,6 +270,21 @@ func (mr *MockTChanNodeMockRecorder) FetchBatchRaw(ctx, req interface{}) *gomock return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "FetchBatchRaw", reflect.TypeOf((*MockTChanNode)(nil).FetchBatchRaw), ctx, req) } +// FetchBatchRawV2 mocks base method +func (m *MockTChanNode) FetchBatchRawV2(ctx thrift.Context, req *FetchBatchRawV2Request) (*FetchBatchRawResult_, error) { + m.ctrl.T.Helper() + ret := m.ctrl.Call(m, "FetchBatchRawV2", ctx, req) + ret0, _ := ret[0].(*FetchBatchRawResult_) + ret1, _ := ret[1].(error) + return ret0, ret1 +} + +// FetchBatchRawV2 indicates an expected call of FetchBatchRawV2 +func (mr *MockTChanNodeMockRecorder) FetchBatchRawV2(ctx, req interface{}) *gomock.Call { + mr.mock.ctrl.T.Helper() + return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "FetchBatchRawV2", reflect.TypeOf((*MockTChanNode)(nil).FetchBatchRawV2), ctx, req) +} + // FetchBlocksMetadataRawV2 mocks base method func (m *MockTChanNode) FetchBlocksMetadataRawV2(ctx thrift.Context, req *FetchBlocksMetadataRawV2Request) (*FetchBlocksMetadataRawV2Result_, error) { m.ctrl.T.Helper() diff --git a/src/dbnode/generated/thrift/rpc/tchan-rpc.go b/src/dbnode/generated/thrift/rpc/tchan-rpc.go index d005481de5..4e9cd5b396 100644 --- a/src/dbnode/generated/thrift/rpc/tchan-rpc.go +++ b/src/dbnode/generated/thrift/rpc/tchan-rpc.go @@ -51,6 +51,7 @@ type TChanNode interface { BootstrappedInPlacementOrNoPlacement(ctx thrift.Context) (*NodeBootstrappedInPlacementOrNoPlacementResult_, error) Fetch(ctx thrift.Context, req *FetchRequest) (*FetchResult_, error) FetchBatchRaw(ctx thrift.Context, req *FetchBatchRawRequest) (*FetchBatchRawResult_, error) + FetchBatchRawV2(ctx thrift.Context, req *FetchBatchRawV2Request) (*FetchBatchRawResult_, error) FetchBlocksMetadataRawV2(ctx thrift.Context, req *FetchBlocksMetadataRawV2Request) (*FetchBlocksMetadataRawV2Result_, error) FetchBlocksRaw(ctx thrift.Context, req *FetchBlocksRawRequest) (*FetchBlocksRawResult_, error) FetchTagged(ctx thrift.Context, req *FetchTaggedRequest) (*FetchTaggedResult_, error) @@ -582,6 +583,24 @@ func (c *tchanNodeClient) FetchBatchRaw(ctx thrift.Context, req *FetchBatchRawRe return resp.GetSuccess(), err } +func (c *tchanNodeClient) FetchBatchRawV2(ctx thrift.Context, req *FetchBatchRawV2Request) (*FetchBatchRawResult_, error) { + var resp NodeFetchBatchRawV2Result + args := NodeFetchBatchRawV2Args{ + Req: req, + } + success, err := c.client.Call(ctx, c.thriftService, "fetchBatchRawV2", &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 fetchBatchRawV2") + } + } + + return resp.GetSuccess(), err +} + func (c *tchanNodeClient) FetchBlocksMetadataRawV2(ctx thrift.Context, req *FetchBlocksMetadataRawV2Request) (*FetchBlocksMetadataRawV2Result_, error) { var resp NodeFetchBlocksMetadataRawV2Result args := NodeFetchBlocksMetadataRawV2Args{ @@ -972,6 +991,7 @@ func (s *tchanNodeServer) Methods() []string { "bootstrappedInPlacementOrNoPlacement", "fetch", "fetchBatchRaw", + "fetchBatchRawV2", "fetchBlocksMetadataRawV2", "fetchBlocksRaw", "fetchTagged", @@ -1010,6 +1030,8 @@ func (s *tchanNodeServer) Handle(ctx thrift.Context, methodName string, protocol return s.handleFetch(ctx, protocol) case "fetchBatchRaw": return s.handleFetchBatchRaw(ctx, protocol) + case "fetchBatchRawV2": + return s.handleFetchBatchRawV2(ctx, protocol) case "fetchBlocksMetadataRawV2": return s.handleFetchBlocksMetadataRawV2(ctx, protocol) case "fetchBlocksRaw": @@ -1226,6 +1248,34 @@ func (s *tchanNodeServer) handleFetchBatchRaw(ctx thrift.Context, protocol athri return err == nil, &res, nil } +func (s *tchanNodeServer) handleFetchBatchRawV2(ctx thrift.Context, protocol athrift.TProtocol) (bool, athrift.TStruct, error) { + var req NodeFetchBatchRawV2Args + var res NodeFetchBatchRawV2Result + + if err := req.Read(protocol); err != nil { + return false, nil, err + } + + r, err := + s.handler.FetchBatchRawV2(ctx, req.Req) + + if err != nil { + switch v := err.(type) { + case *Error: + if v == nil { + return false, nil, fmt.Errorf("Handler for err returned non-nil error type *Error but nil value") + } + res.Err = v + default: + return false, nil, err + } + } else { + res.Success = r + } + + return err == nil, &res, nil +} + func (s *tchanNodeServer) handleFetchBlocksMetadataRawV2(ctx thrift.Context, protocol athrift.TProtocol) (bool, athrift.TStruct, error) { var req NodeFetchBlocksMetadataRawV2Args var res NodeFetchBlocksMetadataRawV2Result diff --git a/src/dbnode/integration/options.go b/src/dbnode/integration/options.go index 6074150593..dec5eca56e 100644 --- a/src/dbnode/integration/options.go +++ b/src/dbnode/integration/options.go @@ -58,7 +58,7 @@ const ( defaultTickMinimumInterval = 1 * time.Second // defaultUseTChannelClientForReading determines whether we use the tchannel client for reading by default. - defaultUseTChannelClientForReading = true + defaultUseTChannelClientForReading = false // defaultUseTChannelClientForWriting determines whether we use the tchannel client for writing by default. defaultUseTChannelClientForWriting = false diff --git a/src/dbnode/integration/peers_bootstrap_high_concurrency_test.go b/src/dbnode/integration/peers_bootstrap_high_concurrency_test.go index 3bc7be53de..c7e6540928 100644 --- a/src/dbnode/integration/peers_bootstrap_high_concurrency_test.go +++ b/src/dbnode/integration/peers_bootstrap_high_concurrency_test.go @@ -28,8 +28,8 @@ import ( "time" "github.com/m3db/m3/src/dbnode/integration/generate" - "github.com/m3db/m3/src/dbnode/retention" "github.com/m3db/m3/src/dbnode/namespace" + "github.com/m3db/m3/src/dbnode/retention" xtest "github.com/m3db/m3/src/x/test" "github.com/stretchr/testify/require" @@ -51,7 +51,11 @@ func TestPeersBootstrapHighConcurrency(t *testing.T) { namespace.NewOptions().SetRetentionOptions(retentionOpts)) require.NoError(t, err) opts := newTestOptions(t). - SetNamespaces([]namespace.Metadata{namesp}) + SetNamespaces([]namespace.Metadata{namesp}). + // Use TChannel clients for writing / reading because we want to target individual nodes at a time + // and not write/read all nodes in the cluster. + SetUseTChannelClientForWriting(true). + SetUseTChannelClientForReading(true) batchSize := 16 concurrency := 64 diff --git a/src/dbnode/integration/peers_bootstrap_index_aggregate_test.go b/src/dbnode/integration/peers_bootstrap_index_aggregate_test.go index a368741044..26d5cc9903 100644 --- a/src/dbnode/integration/peers_bootstrap_index_aggregate_test.go +++ b/src/dbnode/integration/peers_bootstrap_index_aggregate_test.go @@ -27,9 +27,9 @@ import ( "time" "github.com/m3db/m3/src/dbnode/integration/generate" + "github.com/m3db/m3/src/dbnode/namespace" "github.com/m3db/m3/src/dbnode/retention" "github.com/m3db/m3/src/dbnode/storage/index" - "github.com/m3db/m3/src/dbnode/namespace" "github.com/m3db/m3/src/m3ninx/idx" "github.com/m3db/m3/src/x/ident" xtest "github.com/m3db/m3/src/x/test" @@ -59,7 +59,11 @@ func TestPeersBootstrapIndexAggregateQuery(t *testing.T) { ns1, err := namespace.NewMetadata(testNamespaces[0], nOpts) require.NoError(t, err) opts := newTestOptions(t). - SetNamespaces([]namespace.Metadata{ns1}) + SetNamespaces([]namespace.Metadata{ns1}). + // Use TChannel clients for writing / reading because we want to target individual nodes at a time + // and not write/read all nodes in the cluster. + SetUseTChannelClientForWriting(true). + SetUseTChannelClientForReading(true) setupOpts := []bootstrappableTestSetupOptions{ {disablePeersBootstrapper: true}, diff --git a/src/dbnode/integration/peers_bootstrap_index_test.go b/src/dbnode/integration/peers_bootstrap_index_test.go index 67f99521f2..42cf17e22f 100644 --- a/src/dbnode/integration/peers_bootstrap_index_test.go +++ b/src/dbnode/integration/peers_bootstrap_index_test.go @@ -27,9 +27,9 @@ import ( "time" "github.com/m3db/m3/src/dbnode/integration/generate" + "github.com/m3db/m3/src/dbnode/namespace" "github.com/m3db/m3/src/dbnode/retention" "github.com/m3db/m3/src/dbnode/storage/index" - "github.com/m3db/m3/src/dbnode/namespace" "github.com/m3db/m3/src/m3ninx/idx" "github.com/m3db/m3/src/x/ident" xtest "github.com/m3db/m3/src/x/test" @@ -60,7 +60,11 @@ func TestPeersBootstrapIndexWithIndexingEnabled(t *testing.T) { ns1, err := namespace.NewMetadata(testNamespaces[0], nOpts) require.NoError(t, err) opts := newTestOptions(t). - SetNamespaces([]namespace.Metadata{ns1}) + SetNamespaces([]namespace.Metadata{ns1}). + // Use TChannel clients for writing / reading because we want to target individual nodes at a time + // and not write/read all nodes in the cluster. + SetUseTChannelClientForWriting(true). + SetUseTChannelClientForReading(true) setupOpts := []bootstrappableTestSetupOptions{ {disablePeersBootstrapper: true}, diff --git a/src/dbnode/integration/peers_bootstrap_merge_local_test.go b/src/dbnode/integration/peers_bootstrap_merge_local_test.go index 9b5c8a16e4..88610e236a 100644 --- a/src/dbnode/integration/peers_bootstrap_merge_local_test.go +++ b/src/dbnode/integration/peers_bootstrap_merge_local_test.go @@ -27,8 +27,8 @@ import ( "time" "github.com/m3db/m3/src/dbnode/integration/generate" - "github.com/m3db/m3/src/dbnode/retention" "github.com/m3db/m3/src/dbnode/namespace" + "github.com/m3db/m3/src/dbnode/retention" xmetrics "github.com/m3db/m3/src/dbnode/x/metrics" xtest "github.com/m3db/m3/src/x/test" xtime "github.com/m3db/m3/src/x/time" @@ -62,7 +62,11 @@ func testPeersBootstrapMergeLocal(t *testing.T, setTestOpts setTestOptions, upda var ( opts = newTestOptions(t). - SetNamespaces([]namespace.Metadata{namesp}) + SetNamespaces([]namespace.Metadata{namesp}). + // Use TChannel clients for writing / reading because we want to target individual nodes at a time + // and not write/read all nodes in the cluster. + SetUseTChannelClientForWriting(true). + SetUseTChannelClientForReading(true) reporter = xmetrics.NewTestStatsReporter(xmetrics.NewTestStatsReporterOptions()) diff --git a/src/dbnode/integration/peers_bootstrap_merge_peer_blocks_test.go b/src/dbnode/integration/peers_bootstrap_merge_peer_blocks_test.go index ceece9f43d..88a2623efd 100644 --- a/src/dbnode/integration/peers_bootstrap_merge_peer_blocks_test.go +++ b/src/dbnode/integration/peers_bootstrap_merge_peer_blocks_test.go @@ -60,7 +60,11 @@ func testPeersBootstrapMergePeerBlocks(t *testing.T, setTestOpts setTestOptions, SetRetentionOptions(retentionOpts)) require.NoError(t, err) opts := newTestOptions(t). - SetNamespaces([]namespace.Metadata{namesp}) + SetNamespaces([]namespace.Metadata{namesp}). + // Use TChannel clients for writing / reading because we want to target individual nodes at a time + // and not write/read all nodes in the cluster. + SetUseTChannelClientForWriting(true). + SetUseTChannelClientForReading(true) if setTestOpts != nil { opts = setTestOpts(t, opts) namesp = opts.Namespaces()[0] diff --git a/src/dbnode/integration/peers_bootstrap_node_down_test.go b/src/dbnode/integration/peers_bootstrap_node_down_test.go index 1398c4e2c3..40cf70d3e1 100644 --- a/src/dbnode/integration/peers_bootstrap_node_down_test.go +++ b/src/dbnode/integration/peers_bootstrap_node_down_test.go @@ -27,8 +27,8 @@ import ( "time" "github.com/m3db/m3/src/dbnode/integration/generate" - "github.com/m3db/m3/src/dbnode/retention" "github.com/m3db/m3/src/dbnode/namespace" + "github.com/m3db/m3/src/dbnode/retention" xtest "github.com/m3db/m3/src/x/test" "github.com/stretchr/testify/require" @@ -51,7 +51,11 @@ func TestPeersBootstrapNodeDown(t *testing.T) { namespace.NewOptions().SetRetentionOptions(retentionOpts)) require.NoError(t, err) opts := newTestOptions(t). - SetNamespaces([]namespace.Metadata{namesp}) + SetNamespaces([]namespace.Metadata{namesp}). + // Use TChannel clients for writing / reading because we want to target individual nodes at a time + // and not write/read all nodes in the cluster. + SetUseTChannelClientForWriting(true). + SetUseTChannelClientForReading(true) setupOpts := []bootstrappableTestSetupOptions{ {disablePeersBootstrapper: true}, diff --git a/src/dbnode/integration/peers_bootstrap_none_available_test.go b/src/dbnode/integration/peers_bootstrap_none_available_test.go index 6c99c28992..83ff7bef5f 100644 --- a/src/dbnode/integration/peers_bootstrap_none_available_test.go +++ b/src/dbnode/integration/peers_bootstrap_none_available_test.go @@ -29,9 +29,9 @@ import ( "github.com/m3db/m3/src/cluster/services" "github.com/m3db/m3/src/cluster/shard" + "github.com/m3db/m3/src/dbnode/namespace" "github.com/m3db/m3/src/dbnode/retention" "github.com/m3db/m3/src/dbnode/sharding" - "github.com/m3db/m3/src/dbnode/namespace" "github.com/m3db/m3/src/dbnode/topology" "github.com/m3db/m3/src/dbnode/topology/testutil" xtest "github.com/m3db/m3/src/x/test" @@ -56,7 +56,11 @@ func TestPeersBootstrapNoneAvailable(t *testing.T) { namesp, err := namespace.NewMetadata(testNamespaces[0], namespace.NewOptions().SetRetentionOptions(retentionOpts)) require.NoError(t, err) opts := newTestOptions(t). - SetNamespaces([]namespace.Metadata{namesp}) + SetNamespaces([]namespace.Metadata{namesp}). + // Use TChannel clients for writing / reading because we want to target individual nodes at a time + // and not write/read all nodes in the cluster. + SetUseTChannelClientForWriting(true). + SetUseTChannelClientForReading(true) minShard := uint32(0) maxShard := uint32(opts.NumShards()) - uint32(1) diff --git a/src/dbnode/integration/peers_bootstrap_select_best_test.go b/src/dbnode/integration/peers_bootstrap_select_best_test.go index 0d40041202..7ed9a764f7 100644 --- a/src/dbnode/integration/peers_bootstrap_select_best_test.go +++ b/src/dbnode/integration/peers_bootstrap_select_best_test.go @@ -27,8 +27,8 @@ import ( "time" "github.com/m3db/m3/src/dbnode/integration/generate" - "github.com/m3db/m3/src/dbnode/retention" "github.com/m3db/m3/src/dbnode/namespace" + "github.com/m3db/m3/src/dbnode/retention" xtest "github.com/m3db/m3/src/x/test" xtime "github.com/m3db/m3/src/x/time" @@ -50,7 +50,11 @@ func TestPeersBootstrapSelectBest(t *testing.T) { namesp, err := namespace.NewMetadata(testNamespaces[0], namespace.NewOptions().SetRetentionOptions(retentionOpts)) require.NoError(t, err) opts := newTestOptions(t). - SetNamespaces([]namespace.Metadata{namesp}) + SetNamespaces([]namespace.Metadata{namesp}). + // Use TChannel clients for writing / reading because we want to target individual nodes at a time + // and not write/read all nodes in the cluster. + SetUseTChannelClientForWriting(true). + SetUseTChannelClientForReading(true) setupOpts := []bootstrappableTestSetupOptions{ {disablePeersBootstrapper: true}, diff --git a/src/dbnode/integration/peers_bootstrap_simple_test.go b/src/dbnode/integration/peers_bootstrap_simple_test.go index 20b0f5d255..f52dce2f08 100644 --- a/src/dbnode/integration/peers_bootstrap_simple_test.go +++ b/src/dbnode/integration/peers_bootstrap_simple_test.go @@ -57,7 +57,11 @@ func testPeersBootstrapSimple(t *testing.T, setTestOpts setTestOptions, updateIn namesp, err := namespace.NewMetadata(testNamespaces[0], namespace.NewOptions().SetRetentionOptions(retentionOpts)) require.NoError(t, err) opts := newTestOptions(t). - SetNamespaces([]namespace.Metadata{namesp}) + SetNamespaces([]namespace.Metadata{namesp}). + // Use TChannel clients for writing / reading because we want to target individual nodes at a time + // and not write/read all nodes in the cluster. + SetUseTChannelClientForWriting(true). + SetUseTChannelClientForReading(true) if setTestOpts != nil { opts = setTestOpts(t, opts) namesp = opts.Namespaces()[0] diff --git a/src/dbnode/integration/peers_bootstrap_single_node_test.go b/src/dbnode/integration/peers_bootstrap_single_node_test.go index 070c5583a0..b62b768a33 100644 --- a/src/dbnode/integration/peers_bootstrap_single_node_test.go +++ b/src/dbnode/integration/peers_bootstrap_single_node_test.go @@ -51,7 +51,11 @@ func TestPeersBootstrapSingleNode(t *testing.T) { namesp, err := namespace.NewMetadata(testNamespaces[0], namespace.NewOptions().SetRetentionOptions(retentionOpts)) require.NoError(t, err) opts := newTestOptions(t). - SetNamespaces([]namespace.Metadata{namesp}) + SetNamespaces([]namespace.Metadata{namesp}). + // Use TChannel clients for writing / reading because we want to target individual nodes at a time + // and not write/read all nodes in the cluster. + SetUseTChannelClientForWriting(true). + SetUseTChannelClientForReading(true) setupOpts := []bootstrappableTestSetupOptions{ {disablePeersBootstrapper: false}, diff --git a/src/dbnode/integration/repair_test.go b/src/dbnode/integration/repair_test.go index b487646e05..ff0364b1f7 100644 --- a/src/dbnode/integration/repair_test.go +++ b/src/dbnode/integration/repair_test.go @@ -190,7 +190,11 @@ func testRepair( namesp, err := namespace.NewMetadata(testNamespaces[0], nsOpts) require.NoError(t, err) opts := newTestOptions(t). - SetNamespaces([]namespace.Metadata{namesp}) + SetNamespaces([]namespace.Metadata{namesp}). + // Use TChannel clients for writing / reading because we want to target individual nodes at a time + // and not write/read all nodes in the cluster. + SetUseTChannelClientForWriting(true). + SetUseTChannelClientForReading(true) setupOpts := []bootstrappableTestSetupOptions{ {disablePeersBootstrapper: true, enableRepairs: true}, diff --git a/src/dbnode/integration/setup.go b/src/dbnode/integration/setup.go index f16474b2eb..9ee3eb9aca 100644 --- a/src/dbnode/integration/setup.go +++ b/src/dbnode/integration/setup.go @@ -752,7 +752,8 @@ func newClients( clientOpts = defaultClientOptions(topoInit). SetClusterConnectTimeout(opts.ClusterConnectionTimeout()). SetWriteConsistencyLevel(opts.WriteConsistencyLevel()). - SetTopologyInitializer(topoInit) + SetTopologyInitializer(topoInit). + SetUseV2BatchAPIs(true) origin = newOrigin(id, tchannelNodeAddr) verificationOrigin = newOrigin(id+"-verification", tchannelNodeAddr) diff --git a/src/dbnode/network/server/tchannelthrift/node/service.go b/src/dbnode/network/server/tchannelthrift/node/service.go index a528ae5f77..afd7a246c8 100644 --- a/src/dbnode/network/server/tchannelthrift/node/service.go +++ b/src/dbnode/network/server/tchannelthrift/node/service.go @@ -831,6 +831,71 @@ func (s *service) FetchBatchRaw(tctx thrift.Context, req *rpc.FetchBatchRawReque return result, nil } +func (s *service) FetchBatchRawV2(tctx thrift.Context, req *rpc.FetchBatchRawV2Request) (*rpc.FetchBatchRawResult_, error) { + s.metrics.fetchBatchRawRPCS.Inc(1) + db, err := s.startReadRPCWithDB() + if err != nil { + return nil, err + } + defer s.readRPCCompleted() + + var ( + callStart = s.nowFn() + ctx = tchannelthrift.Context(tctx) + nsIDs = make([]ident.ID, 0, len(req.Elements)) + result = rpc.NewFetchBatchRawResult_() + success int + retryableErrors int + nonRetryableErrors int + ) + for _, nsBytes := range req.NameSpaces { + nsIDs = append(nsIDs, s.newID(ctx, nsBytes)) + } + for _, elem := range req.Elements { + if elem.NameSpace >= int64(len(nsIDs)) { + return nil, fmt.Errorf( + "received fetch request with namespace index: %d, but only %d namespaces were provided", + elem.NameSpace, len(nsIDs)) + } + } + + for _, elem := range req.Elements { + start, rangeStartErr := convert.ToTime(elem.RangeStart, elem.RangeTimeType) + end, rangeEndErr := convert.ToTime(elem.RangeEnd, elem.RangeTimeType) + if rangeStartErr != nil || rangeEndErr != nil { + s.metrics.fetchBatchRaw.ReportNonRetryableErrors(len(req.Elements)) + s.metrics.fetchBatchRaw.ReportLatency(s.nowFn().Sub(callStart)) + return nil, tterrors.NewBadRequestError(xerrors.FirstError(rangeStartErr, rangeEndErr)) + } + + rawResult := rpc.NewFetchRawResult_() + result.Elements = append(result.Elements, rawResult) + tsID := s.newID(ctx, elem.ID) + + nsIdx := nsIDs[int(elem.NameSpace)] + segments, rpcErr := s.readEncoded(ctx, db, nsIdx, tsID, start, end) + if rpcErr != nil { + rawResult.Err = rpcErr + if tterrors.IsBadRequestError(rawResult.Err) { + nonRetryableErrors++ + } else { + retryableErrors++ + } + continue + } + + success++ + rawResult.Segments = segments + } + + s.metrics.fetchBatchRaw.ReportSuccess(success) + s.metrics.fetchBatchRaw.ReportRetryableErrors(retryableErrors) + s.metrics.fetchBatchRaw.ReportNonRetryableErrors(nonRetryableErrors) + s.metrics.fetchBatchRaw.ReportLatency(s.nowFn().Sub(callStart)) + + return result, nil +} + func (s *service) FetchBlocksRaw(tctx thrift.Context, req *rpc.FetchBlocksRawRequest) (*rpc.FetchBlocksRawResult_, error) { db, err := s.startReadRPCWithDB() if err != nil { diff --git a/src/dbnode/network/server/tchannelthrift/node/service_test.go b/src/dbnode/network/server/tchannelthrift/node/service_test.go index 481d36f520..2c319de2f5 100644 --- a/src/dbnode/network/server/tchannelthrift/node/service_test.go +++ b/src/dbnode/network/server/tchannelthrift/node/service_test.go @@ -742,6 +742,121 @@ func TestServiceFetchBatchRaw(t *testing.T) { } } +func TestServiceFetchBatchRawV2MultiNS(t *testing.T) { + ctrl := gomock.NewController(t) + defer ctrl.Finish() + + mockDB := storage.NewMockDatabase(ctrl) + mockDB.EXPECT().Options().Return(testStorageOpts).AnyTimes() + mockDB.EXPECT().IsOverloaded().Return(false) + + service := NewService(mockDB, testTChannelThriftOptions).(*service) + + tctx, _ := tchannelthrift.NewContext(time.Minute) + ctx := tchannelthrift.Context(tctx) + defer ctx.Close() + + start := time.Now().Add(-2 * time.Hour) + end := start.Add(2 * time.Hour) + + start, end = start.Truncate(time.Second), end.Truncate(time.Second) + + nsID1 := "metrics1" + nsID2 := "metrics2" + + streams := map[string]xio.SegmentReader{} + series := map[string][]struct { + t time.Time + v float64 + }{ + "foo": { + {start.Add(10 * time.Second), 1.0}, + {start.Add(20 * time.Second), 2.0}, + }, + "bar": { + {start.Add(20 * time.Second), 3.0}, + {start.Add(30 * time.Second), 4.0}, + }, + } + for id, s := range series { + enc := testStorageOpts.EncoderPool().Get() + enc.Reset(start, 0, nil) + for _, v := range s { + dp := ts.Datapoint{ + Timestamp: v.t, + Value: v.v, + } + require.NoError(t, enc.Encode(dp, xtime.Second, nil)) + } + + stream, _ := enc.Stream(encoding.StreamOptions{}) + streams[id] = stream + nsID := nsID1 + if id == "bar" { + nsID = nsID2 + } + mockDB.EXPECT(). + ReadEncoded(ctx, ident.NewIDMatcher(nsID), ident.NewIDMatcher(id), start, end). + Return([][]xio.BlockReader{ + []xio.BlockReader{ + xio.BlockReader{ + SegmentReader: stream, + }, + }, + }, nil) + } + + ids := [][]byte{[]byte("foo"), []byte("bar")} + elements := []*rpc.FetchBatchRawV2RequestElement{ + &rpc.FetchBatchRawV2RequestElement{ + NameSpace: 0, + RangeStart: start.Unix(), + RangeEnd: end.Unix(), + ID: []byte("foo"), + RangeTimeType: rpc.TimeType_UNIX_SECONDS, + }, + &rpc.FetchBatchRawV2RequestElement{ + NameSpace: 1, + RangeStart: start.Unix(), + RangeEnd: end.Unix(), + ID: []byte("bar"), + RangeTimeType: rpc.TimeType_UNIX_SECONDS, + }, + } + r, err := service.FetchBatchRawV2(tctx, &rpc.FetchBatchRawV2Request{ + NameSpaces: [][]byte{[]byte(nsID1), []byte(nsID2)}, + Elements: elements, + }) + require.NoError(t, err) + + require.Equal(t, len(ids), len(r.Elements)) + for i, id := range ids { + elem := r.Elements[i] + require.NotNil(t, elem) + + assert.Nil(t, elem.Err) + require.Equal(t, 1, len(elem.Segments)) + + seg := elem.Segments[0] + require.NotNil(t, seg) + require.NotNil(t, seg.Merged) + + var expectHead, expectTail []byte + expectSegment, err := streams[string(id)].Segment() + require.NoError(t, err) + + if expectSegment.Head != nil { + expectHead = expectSegment.Head.Bytes() + } + if expectSegment.Tail != nil { + expectTail = expectSegment.Tail.Bytes() + } + + assert.Equal(t, expectHead, seg.Merged.Head) + assert.Equal(t, expectTail, seg.Merged.Tail) + } +} + // TestServiceFetchBatchRawOverMaxOutstandingRequests tests that the FetchBatchRaw endpoint // will reject requests if the number of outstanding read requests has hit the maximum. func TestServiceFetchBatchRawOverMaxOutstandingRequests(t *testing.T) {