You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@pegasus.apache.org by la...@apache.org on 2022/12/12 03:33:18 UTC

[incubator-pegasus] branch master updated: refactor(idl): unify the *.thrift files used by cpp and go-client (#1284)

This is an automated email from the ASF dual-hosted git repository.

laiyingchun pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/incubator-pegasus.git


The following commit(s) were added to refs/heads/master by this push:
     new c1435758d refactor(idl): unify the *.thrift files used by cpp and go-client (#1284)
c1435758d is described below

commit c1435758d44011ccfc3d30043c5048df9a03473e
Author: Yingchun Lai <la...@apache.org>
AuthorDate: Mon Dec 12 11:33:13 2022 +0800

    refactor(idl): unify the *.thrift files used by cpp and go-client (#1284)
---
 .github/workflows/lint_and_test_go-client.yml     |    31 +-
 .licenserc.yaml                                   |    12 +-
 go-client/Makefile                                |    15 +-
 go-client/admin/client.go                         |     9 +-
 go-client/docs/how_to_add_new_interface.md        |    11 +-
 go-client/generator/admin.csv                     |    22 +-
 go-client/idl/admin.thrift                        |   767 -
 go-client/idl/admin/GoUnusedProtection__.go       |     6 -
 go-client/idl/admin/admin-consts.go               |    25 -
 go-client/idl/admin/admin.go                      | 20627 --------------------
 go-client/idl/base.thrift                         |    33 -
 go-client/idl/base/error_code.go                  |     2 +-
 go-client/idl/cmd.thrift                          |    30 -
 go-client/idl/cmd/GoUnusedProtection__.go         |     6 -
 go-client/idl/cmd/cmd-consts.go                   |    23 -
 go-client/idl/cmd/cmd.go                          |   536 -
 go-client/idl/radmin.thrift                       |    99 -
 go-client/idl/radmin/GoUnusedProtection__.go      |     6 -
 go-client/idl/radmin/radmin-consts.go             |    25 -
 go-client/idl/radmin/radmin.go                    |  2353 ---
 go-client/idl/replication.thrift                  |    58 -
 go-client/idl/replication/GoUnusedProtection__.go |     6 -
 go-client/idl/replication/replication-consts.go   |    25 -
 go-client/idl/replication/replication.go          |  1135 --
 go-client/idl/rrdb.thrift                         |   293 -
 go-client/idl/rrdb/GoUnusedProtection__.go        |     6 -
 go-client/idl/rrdb/rrdb-consts.go                 |    27 -
 go-client/idl/rrdb/rrdb.go                        | 10824 ----------
 go-client/pegalog/logger.go                       |     3 +-
 go-client/pegasus/op/multidel.go                  |     4 +-
 go-client/pegasus/op/multiget.go                  |     4 +-
 go-client/pegasus/scanner.go                      |     2 +-
 go-client/pegasus/table_connector.go              |     2 +-
 go-client/pegasus/table_connector_test.go         |    12 +-
 go-client/session/admin_rpc_types.go              |    88 +-
 go-client/session/codec.go                        |    34 +-
 go-client/session/codec_test.go                   |     2 +-
 go-client/session/meta_session.go                 |     9 +-
 go-client/session/session_test.go                 |     6 +-
 {src/common => idl}/backup.thrift                 |     5 +-
 {src/common => idl}/bulk_load.thrift              |     5 +-
 {src/remote_cmd => idl}/command.thrift            |     6 +
 idl/dsn.layer2.thrift                             |     2 +-
 idl/dsn.thrift                                    |     1 +
 {src/common => idl}/duplication.thrift            |     5 +-
 {src/common => idl}/meta_admin.thrift             |    66 +-
 {src/common => idl}/metadata.thrift               |     5 +-
 {src/common => idl}/partition_split.thrift        |     5 +-
 {src/common => idl}/replica_admin.thrift          |    16 +-
 idl/rrdb.thrift                                   |    44 +-
 src/common/CMakeLists.txt                         |    21 +-
 src/common/consensus.thrift                       |     2 +-
 src/common/duplication_internal.thrift            |    52 +
 src/include/rrdb/rrdb.client.h                    |     9 +-
 src/remote_cmd/CMakeLists.txt                     |     2 +-
 src/runtime/rpc/request_meta.thrift               |     1 +
 src/server/pegasus_write_service.h                |     1 +
 57 files changed, 330 insertions(+), 37096 deletions(-)

diff --git a/.github/workflows/lint_and_test_go-client.yml b/.github/workflows/lint_and_test_go-client.yml
index 6bf20de64..a55f8faee 100644
--- a/.github/workflows/lint_and_test_go-client.yml
+++ b/.github/workflows/lint_and_test_go-client.yml
@@ -36,27 +36,14 @@ on:
 
 # workflow tasks
 jobs:
-  lint:
-    name: Lint
-    runs-on: ubuntu-latest
-    steps:
-      - uses: actions/checkout@v2
-        with:
-          fetch-depth: 1
-      - name: Set up Go
-        uses: actions/setup-go@v2
-        with:
-          go-version: 1.14
-      - name: golangci-lint
-        uses: golangci/golangci-lint-action@v3
-        with:
-          version: v1.29
-          working-directory: ./go-client
-
   build:
-    name: Test
-    runs-on: ubuntu-latest
+    name: Test and Lint
+    runs-on: ubuntu-20.04
     steps:
+    - name: Install thrift
+      # go-client imports thrift package of 0.13.0, so we must use thrift-compiler 0.13.0
+      # to generate code as well. The thrift-compiler version on ubuntu-20.04 is 0.13.0
+      run: sudo apt-get install -y thrift-compiler
     - name: Checkout
       uses: actions/checkout@v2
       with:
@@ -71,3 +58,9 @@ jobs:
     - uses: codecov/codecov-action@v2
       with:
         working-directory: ./go-client
+    # because some files are generated after building, so lint after that at last
+    - name: Lint
+      uses: golangci/golangci-lint-action@v3
+      with:
+        version: v1.29
+        working-directory: ./go-client
diff --git a/.licenserc.yaml b/.licenserc.yaml
index 00530aa0d..c8d2ef314 100644
--- a/.licenserc.yaml
+++ b/.licenserc.yaml
@@ -39,7 +39,9 @@ header:
     - '**/go.sum'
     - '**/*.csv'
     - '**/*.json'
-    - 'go-client/idl/**/*.go'
+    - 'go-client/idl/base/dsn_err_string.go'
+    - 'go-client/idl/base/rocskdb_err_string.go'
+    - 'go-client/idl/base/GoUnusedProtection__.go'
     - 'nodejs-client/src/dsn/*.js'
     - '**/.npmigonre'
     # Copyright (c) Facebook, Inc
@@ -93,8 +95,12 @@ header:
     # The MIT License (MIT), Copyright (c) 2015 Microsoft Corporation
     - 'cmake_modules/BaseFunctions.cmake'
     - 'docs/rdsn-README.md'
+    - 'idl/command.thrift'
     - 'idl/dsn.layer2.thrift'
     - 'idl/dsn.thrift'
+    - 'idl/metadata.thrift'
+    - 'idl/meta_admin.thrift'
+    - 'idl/replica_admin.thrift'
     - 'scripts/compile_thrift.py'
     - 'scripts/learn_stat.py'
     - 'src/common/api_common.h'
@@ -226,9 +232,6 @@ header:
     - 'src/common/CMakeLists.txt'
     - 'src/common/consensus.thrift'
     - 'src/common/fs_manager.cpp'
-    - 'src/common/meta_admin.thrift'
-    - 'src/common/metadata.thrift'
-    - 'src/common/replica_admin.thrift'
     - 'src/common/replication_common.cpp'
     - 'src/common/replication_common.h'
     - 'src/common/test/CMakeLists.txt'
@@ -326,7 +329,6 @@ header:
     - 'src/perf_counter/test/perf_counters_test.cpp'
     - 'src/perf_counter/test/run.sh'
     - 'src/remote_cmd/CMakeLists.txt'
-    - 'src/remote_cmd/command.thrift'
     - 'src/replica/CMakeLists.txt'
     - 'src/replica/backup/test/config-test.ini'
     - 'src/replica/backup/test/run.sh'
diff --git a/go-client/Makefile b/go-client/Makefile
index 7c38cb64d..ea38b9e24 100644
--- a/go-client/Makefile
+++ b/go-client/Makefile
@@ -16,14 +16,27 @@
 #
 
 build:
+	thrift -I ../idl -out idl --gen go:thrift_import='github.com/apache/thrift/lib/go/thrift',package_prefix='github.com/apache/incubator-pegasus/go-client/idl/' ../idl/backup.thrift
+	thrift -I ../idl -out idl --gen go:thrift_import='github.com/apache/thrift/lib/go/thrift',package_prefix='github.com/apache/incubator-pegasus/go-client/idl/' ../idl/bulk_load.thrift
+	thrift -I ../idl -out idl --gen go:thrift_import='github.com/apache/thrift/lib/go/thrift',package_prefix='github.com/apache/incubator-pegasus/go-client/idl/' ../idl/dsn.layer2.thrift
+	thrift -I ../idl -out idl --gen go:thrift_import='github.com/apache/thrift/lib/go/thrift',package_prefix='github.com/apache/incubator-pegasus/go-client/idl/' ../idl/duplication.thrift
+	thrift -I ../idl -out idl --gen go:thrift_import='github.com/apache/thrift/lib/go/thrift',package_prefix='github.com/apache/incubator-pegasus/go-client/idl/' ../idl/meta_admin.thrift
+	thrift -I ../idl -out idl --gen go:thrift_import='github.com/apache/thrift/lib/go/thrift',package_prefix='github.com/apache/incubator-pegasus/go-client/idl/' ../idl/metadata.thrift
+	thrift -I ../idl -out idl --gen go:thrift_import='github.com/apache/thrift/lib/go/thrift',package_prefix='github.com/apache/incubator-pegasus/go-client/idl/' ../idl/partition_split.thrift
+	thrift -I ../idl -out idl --gen go:thrift_import='github.com/apache/thrift/lib/go/thrift',package_prefix='github.com/apache/incubator-pegasus/go-client/idl/' ../idl/replica_admin.thrift
+	thrift -I ../idl -out idl --gen go:thrift_import='github.com/apache/thrift/lib/go/thrift',package_prefix='github.com/apache/incubator-pegasus/go-client/idl/' ../idl/rrdb.thrift
+	thrift -I ../idl -out idl --gen go:thrift_import='github.com/apache/thrift/lib/go/thrift',package_prefix='github.com/apache/incubator-pegasus/go-client/idl/' ../idl/command.thrift
+	rm -Rf idl/admin/admin_client-remote
+	rm -Rf idl/cmd/remote_cmd_service-remote
+	rm -Rf idl/radmin/replica_client-remote
 	go mod tidy
 	go mod verify
 	go build -o ./bin/echo ./rpc/main/echo.go
-	go build -o ./bin/failover-test ./integration/failover-test/main.go
 	go build -o ./bin/generator ./generator/main.go
 	./bin/generator -i ./generator/admin.csv -t admin > ./session/admin_rpc_types.go
 	./bin/generator -i ./generator/radmin.csv -t radmin > ./session/radmin_rpc_types.go
 	go build -o ./bin/example ./example/main.go
+	go build -o ./bin/failover-test ./integration/failover-test/main.go
 
 fmt:
 	go fmt ./...
diff --git a/go-client/admin/client.go b/go-client/admin/client.go
index efdc8ff5d..5d3c044ee 100644
--- a/go-client/admin/client.go
+++ b/go-client/admin/client.go
@@ -26,6 +26,7 @@ import (
 
 	"github.com/apache/incubator-pegasus/go-client/idl/admin"
 	"github.com/apache/incubator-pegasus/go-client/idl/base"
+	"github.com/apache/incubator-pegasus/go-client/idl/replication"
 	"github.com/apache/incubator-pegasus/go-client/session"
 )
 
@@ -89,7 +90,7 @@ func (c *rpcBasedClient) waitTableReady(ctx context.Context, tableName string, p
 }
 
 func (c *rpcBasedClient) CreateTable(ctx context.Context, tableName string, partitionCount int) error {
-	_, err := c.metaManager.CreateApp(ctx, &admin.CreateAppRequest{
+	_, err := c.metaManager.CreateApp(ctx, &admin.ConfigurationCreateAppRequest{
 		AppName: tableName,
 		Options: &admin.CreateAppOptions{
 			PartitionCount: int32(partitionCount),
@@ -107,7 +108,7 @@ func (c *rpcBasedClient) CreateTable(ctx context.Context, tableName string, part
 }
 
 func (c *rpcBasedClient) DropTable(ctx context.Context, tableName string) error {
-	req := admin.NewDropAppRequest()
+	req := admin.NewConfigurationDropAppRequest()
 	req.AppName = tableName
 	reserveSeconds := int64(1) // delete immediately. the caller is responsible for the soft deletion of table.
 	req.Options = &admin.DropAppOptions{
@@ -119,8 +120,8 @@ func (c *rpcBasedClient) DropTable(ctx context.Context, tableName string) error
 }
 
 func (c *rpcBasedClient) ListTables(ctx context.Context) ([]*TableInfo, error) {
-	resp, err := c.metaManager.ListApps(ctx, &admin.ListAppsRequest{
-		Status: admin.AppStatus_AS_AVAILABLE,
+	resp, err := c.metaManager.ListApps(ctx, &admin.ConfigurationListAppsRequest{
+		Status: replication.AppStatus_AS_AVAILABLE,
 	})
 	if err != nil {
 		return nil, err
diff --git a/go-client/docs/how_to_add_new_interface.md b/go-client/docs/how_to_add_new_interface.md
index 5bc9832a2..faf7cfb37 100644
--- a/go-client/docs/how_to_add_new_interface.md
+++ b/go-client/docs/how_to_add_new_interface.md
@@ -40,7 +40,16 @@ you can follow [the building manual of thrift](https://thrift.apache.org/docs/Bu
 3. Compile thrift files to generate go code.
 
 ```sh
-thrift -I idl -out idl --gen go:thrift_import='github.com/apache/thrift/lib/go/thrift',package_prefix='github.com/apache/incubator-pegasus/go-client/idl/' idl/admin.thrift
+thrift -I ../idl -out idl --gen go:thrift_import='github.com/apache/thrift/lib/go/thrift',package_prefix='github.com/apache/incubator-pegasus/idl/' ../idl/backup.thrift
+thrift -I ../idl -out idl --gen go:thrift_import='github.com/apache/thrift/lib/go/thrift',package_prefix='github.com/apache/incubator-pegasus/idl/' ../idl/bulk_load.thrift
+thrift -I ../idl -out idl --gen go:thrift_import='github.com/apache/thrift/lib/go/thrift',package_prefix='github.com/apache/incubator-pegasus/idl/' ../idl/dsn.layer2.thrift
+thrift -I ../idl -out idl --gen go:thrift_import='github.com/apache/thrift/lib/go/thrift',package_prefix='github.com/apache/incubator-pegasus/idl/' ../idl/dsn.thrift
+thrift -I ../idl -out idl --gen go:thrift_import='github.com/apache/thrift/lib/go/thrift',package_prefix='github.com/apache/incubator-pegasus/idl/' ../idl/duplication.thrift
+thrift -I ../idl -out idl --gen go:thrift_import='github.com/apache/thrift/lib/go/thrift',package_prefix='github.com/apache/incubator-pegasus/idl/' ../idl/meta_admin.thrift
+thrift -I ../idl -out idl --gen go:thrift_import='github.com/apache/thrift/lib/go/thrift',package_prefix='github.com/apache/incubator-pegasus/idl/' ../idl/metadata.thrift
+thrift -I ../idl -out idl --gen go:thrift_import='github.com/apache/thrift/lib/go/thrift',package_prefix='github.com/apache/incubator-pegasus/idl/' ../idl/partition_split.thrift
+thrift -I ../idl -out idl --gen go:thrift_import='github.com/apache/thrift/lib/go/thrift',package_prefix='github.com/apache/incubator-pegasus/idl/' ../idl/replica_admin.thrift
+thrift -I ../idl -out idl --gen go:thrift_import='github.com/apache/thrift/lib/go/thrift',package_prefix='github.com/apache/incubator-pegasus/idl/' ../idl/rrdb.thrift
 gofmt -w **/*.go
 goimports -w */**.go
 ```
diff --git a/go-client/generator/admin.csv b/go-client/generator/admin.csv
index d591c030a..8eba1c0cc 100644
--- a/go-client/generator/admin.csv
+++ b/go-client/generator/admin.csv
@@ -1,20 +1,20 @@
-RPC_CM_DROP_APP,DropApp,DropAppRequest,DropAppResponse
-RPC_CM_CREATE_APP,CreateApp,CreateAppRequest,CreateAppResponse
-RPC_CM_RECALL_APP,RecallApp,RecallAppRequest,RecallAppResponse
-RPC_CM_LIST_APPS,ListApps,ListAppsRequest,ListAppsResponse
+RPC_CM_DROP_APP,DropApp,ConfigurationDropAppRequest,ConfigurationDropAppResponse
+RPC_CM_CREATE_APP,CreateApp,ConfigurationCreateAppRequest,ConfigurationCreateAppResponse
+RPC_CM_RECALL_APP,RecallApp,ConfigurationRecallAppRequest,ConfigurationRecallAppResponse
+RPC_CM_LIST_APPS,ListApps,ConfigurationListAppsRequest,ConfigurationListAppsResponse
 RPC_CM_QUERY_DUPLICATION,QueryDuplication,DuplicationQueryRequest,DuplicationQueryResponse
 RPC_CM_MODIFY_DUPLICATION,ModifyDuplication,DuplicationModifyRequest,DuplicationModifyResponse
 RPC_CM_ADD_DUPLICATION,AddDuplication,DuplicationAddRequest,DuplicationAddResponse
 RPC_QUERY_APP_INFO,QueryAppInfo,QueryAppInfoRequest,QueryAppInfoResponse
-RPC_CM_UPDATE_APP_ENV,UpdateAppEnv,UpdateAppEnvRequest,UpdateAppEnvResponse
-RPC_CM_LIST_NODES,ListNodes,ListNodesRequest,ListNodesResponse
-RPC_CM_CLUSTER_INFO,QueryClusterInfo,ClusterInfoRequest,ClusterInfoResponse
-RPC_CM_CONTROL_META,MetaControl,MetaControlRequest,MetaControlResponse
-RPC_CM_QUERY_BACKUP_POLICY,QueryBackupPolicy,QueryBackupPolicyRequest,QueryBackupPolicyResponse
-RPC_CM_PROPOSE_BALANCER,Balance,BalanceRequest,BalanceResponse
+RPC_CM_UPDATE_APP_ENV,UpdateAppEnv,ConfigurationUpdateAppEnvRequest,ConfigurationUpdateAppEnvResponse
+RPC_CM_LIST_NODES,ListNodes,ConfigurationListNodesRequest,ConfigurationListNodesResponse
+RPC_CM_CLUSTER_INFO,QueryClusterInfo,ConfigurationClusterInfoRequest,ConfigurationClusterInfoResponse
+RPC_CM_CONTROL_META,MetaControl,ConfigurationMetaControlRequest,ConfigurationMetaControlResponse
+RPC_CM_QUERY_BACKUP_POLICY,QueryBackupPolicy,ConfigurationQueryBackupPolicyRequest,ConfigurationQueryBackupPolicyResponse
+RPC_CM_PROPOSE_BALANCER,Balance,ConfigurationBalancerRequest,ConfigurationBalancerResponse
 RPC_CM_START_BACKUP_APP,StartBackupApp,StartBackupAppRequest,StartBackupAppResponse
 RPC_CM_QUERY_BACKUP_STATUS,QueryBackupStatus,QueryBackupStatusRequest,QueryBackupStatusResponse
-RPC_CM_START_RESTORE,RestoreApp,RestoreAppRequest,CreateAppResponse
+RPC_CM_START_RESTORE,RestoreApp,ConfigurationRestoreRequest,ConfigurationCreateAppResponse
 RPC_CM_START_PARTITION_SPLIT,StartPartitionSplit,StartPartitionSplitRequest,StartPartitionSplitResponse
 RPC_CM_QUERY_PARTITION_SPLIT,QuerySplitStatus,QuerySplitRequest,QuerySplitResponse
 RPC_CM_CONTROL_PARTITION_SPLIT,ControlPartitionSplit,ControlSplitRequest,ControlSplitResponse
diff --git a/go-client/idl/admin.thrift b/go-client/idl/admin.thrift
deleted file mode 100644
index 880cf453d..000000000
--- a/go-client/idl/admin.thrift
+++ /dev/null
@@ -1,767 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-
-include "base.thrift"
-
-namespace go admin
-
-/////////////////// Tables Management ////////////////////
-
-struct create_app_options
-{
-    1:i32              partition_count;
-    2:i32              replica_count;
-    3:bool             success_if_exist;
-    4:string           app_type;
-    5:bool             is_stateful;
-    6:map<string, string>  envs;
-}
-
-struct create_app_request
-{
-    1:string                   app_name;
-    2:create_app_options       options;
-}
-
-struct create_app_response
-{
-    1:base.error_code err;
-    2:i32 appid;
-}
-
-struct drop_app_options
-{
-    1:bool             success_if_not_exist;
-    2:optional i64     reserve_seconds;
-}
-
-struct drop_app_request
-{
-    1:string                   app_name;
-    2:drop_app_options         options;
-}
-
-struct drop_app_response
-{
-    1:base.error_code err;
-}
-
-enum app_status
-{
-    AS_INVALID,
-    AS_AVAILABLE,
-    AS_CREATING,
-    AS_CREATE_FAILED, // deprecated
-    AS_DROPPING,
-    AS_DROP_FAILED, // deprecated
-    AS_DROPPED,
-    AS_RECALLING
-}
-
-struct app_info
-{
-    1:app_status    status = app_status.AS_INVALID;
-    2:string        app_type;
-    3:string        app_name;
-    4:i32           app_id;
-    5:i32           partition_count;
-    6:map<string, string> envs;
-    7:bool          is_stateful;
-    8:i32           max_replica_count;
-    9:i64           expire_second;
-
-    // new fields added from v1.11.0
-    10:i64          create_second;
-    11:i64          drop_second;
-
-    // New fields added from v1.12.0
-    // Whether this app is duplicating.
-    // If true it should prevent its unconfirmed WAL from being compacted.
-    12:optional bool duplicating;
-
-    // New fields for partition split
-    // If meta server failed during partition split,
-    // child partition is not existed on remote stroage, but partition count changed.
-    // We use init_partition_count to handle those child partitions while sync_apps_from_remote_stroage
-    13:i32          init_partition_count = -1;
-
-    // New fields for bulk load
-    // Whether this app is executing bulk load
-    14:optional bool    is_bulk_loading = false;
-}
-
-struct recall_app_request
-{
-    1:i32 app_id;
-    2:string new_app_name;
-}
-
-struct recall_app_response
-{
-    1:base.error_code err;
-    2:app_info info;
-}
-
-struct list_apps_request
-{
-    1:app_status status = app_status.AS_INVALID;
-}
-
-struct list_apps_response
-{
-    1:base.error_code err
-    2:list<app_info> infos;
-}
-
-struct query_app_info_request
-{
-    1:base.rpc_address meta_server;
-}
-
-struct query_app_info_response
-{
-    1:base.error_code err;
-    2:list<app_info> apps;
-}
-
-enum app_env_operation
-{
-    APP_ENV_OP_INVALID,
-    APP_ENV_OP_SET,
-    APP_ENV_OP_DEL,
-    APP_ENV_OP_CLEAR
-}
-
-struct update_app_env_request
-{
-    1:string app_name;
-    2:app_env_operation op = app_env_operation.APP_ENV_OP_INVALID;
-    3:optional list<string> keys;           // used for set and del
-    4:optional list<string> values;         // only used for set
-    5:optional string clear_prefix;         // only used for clear
-                                            // if clear_prefix is empty then we clear all envs
-                                            // else clear the env that key = "clear_prefix.xxx"
-}
-
-struct update_app_env_response
-{
-    1:base.error_code err;
-    2:string hint_message;
-}
-
-struct start_app_manual_compact_request
-{
-    1:string        app_name;
-    2:optional i64  trigger_time;
-    3:optional i32  target_level;
-    4:optional bool bottommost;
-    5:optional i32  max_running_count;
-}
-
-struct start_app_manual_compact_response
-{
-    // Possible error:
-    // - ERR_APP_NOT_EXIST: app not exist
-    // - ERR_APP_DROPPED: app has been dropped
-    // - ERR_OPERATION_DISABLED: app disable manual compaction
-    // - ERR_INVALID_PARAMETERS: invalid manual compaction parameters
-    1:base.error_code    err;
-    2:string            hint_msg;
-}
-
-struct query_app_manual_compact_request
-{
-    1:string app_name;
-}
-
-struct query_app_manual_compact_response
-{
-    // Possible error:
-    // - ERR_APP_NOT_EXIST: app not exist
-    // - ERR_APP_DROPPED: app has been dropped
-    // - ERR_INVALID_STATE: app is not executing manual compaction
-    1:base.error_code    err;
-    2:string            hint_msg;
-    3:optional i32      progress;
-}
-
-/////////////////// Nodes Management ////////////////////
-
-enum node_status
-{
-    NS_INVALID,
-    NS_ALIVE,
-    NS_UNALIVE,
-}
-
-struct node_info
-{
-    1:node_status status = node_status.NS_INVALID;
-    2:base.rpc_address address;
-}
-
-struct list_nodes_request
-{
-    1:node_status status = node_status.NS_INVALID;
-}
-
-struct list_nodes_response
-{
-    1:base.error_code   err;
-    2:list<node_info>  infos;
-}
-
-struct cluster_info_request
-{
-}
-
-struct cluster_info_response
-{
-    1:base.error_code err;
-    2:list<string> keys;
-    3:list<string> values;
-}
-
-enum meta_function_level
-{
-    // there are 4 ways to modify the meta-server's status:
-    // 0. DDL operation: create/drop/recall table
-    // 1. downgrade primary when dectect it is not alive
-    // 2. accept primary's update-request to kickoff some secondaries
-    // 3. make balancer proposal, which further trigger 2
-    // according to these ways, we give meta several active level.
-
-    fl_stopped = 100, //we don't take any action to modify the meta's status, even the DDL operations are not responsed
-    fl_blind = 200, //only DDL operations are responsed, 1 2 3 are just ignored
-    fl_freezed = 300, //0 1 are responsed, 2 3 ignored
-    fl_steady = 400, //0 1 2 are responsed, don't do any balancer
-    fl_lively = 500, //full functional
-    fl_invalid = 10000
-}
-
-// if the level is invalid, we just response the old level of meta without updating it
-struct meta_control_request
-{
-    1:meta_function_level level;
-}
-
-struct meta_control_response
-{
-    1:base.error_code err;
-    2:meta_function_level old_level;
-}
-
-/////////////////// duplication-related structs ////////////////////
-
-//  - INIT  -> PREPARE
-//  - PREPARE -> APP
-//  - APP -> LOG
-//  NOTE: Just LOG and PAUSE can be transferred states to each other
-//  - LOG -> PAUSE
-//  - PAUSE -> LOG
-//  - ALL -> REMOVED
-enum duplication_status
-{
-    DS_INIT = 0,
-    DS_PREPARE,// replica prepare latest checkpoint for follower
-    DS_APP,// follower start duplicate checkpoint
-    DS_LOG,// master start batch send plog to follower
-    DS_PAUSE,
-    DS_REMOVED,
-}
-
-// How duplication reacts on permanent failure.
-enum duplication_fail_mode
-{
-    // The default mode. If some permanent failure occurred that makes duplication
-    // blocked, it will retry forever until external interference.
-    FAIL_SLOW = 0,
-
-    // Skip the writes that failed to duplicate, which means minor data loss on the remote cluster.
-    // This will certainly achieve better stability of the system.
-    FAIL_SKIP,
-
-    // Stop immediately after it ensures itself unable to duplicate.
-    // WARN: this mode kills the server process, replicas on the server will all be effected.
-    FAIL_FAST
-}
-
-// This request is sent from client to meta.
-struct duplication_add_request
-{
-    1:string  app_name;
-    2:string  remote_cluster_name;
-
-    // True means to duplicate checkpoint.
-    3:optional bool is_duplicating_checkpoint = true;
-}
-
-struct duplication_add_response
-{
-    // Possible errors:
-    // - ERR_INVALID_PARAMETERS:
-    //   the address of remote cluster is not well configured in meta sever.
-    1:base.error_code  err;
-    2:i32              appid;
-    3:i32              dupid;
-    4:optional string  hint;
-}
-
-// This request is sent from client to meta.
-struct duplication_modify_request
-{
-    1:string                    app_name;
-    2:i32                       dupid;
-    3:optional duplication_status status;
-    4:optional duplication_fail_mode fail_mode;
-}
-
-struct duplication_modify_response
-{
-    // Possible errors:
-    // - ERR_APP_NOT_EXIST: app is not found
-    // - ERR_OBJECT_NOT_FOUND: duplication is not found
-    // - ERR_BUSY: busy for updating state
-    // - ERR_INVALID_PARAMETERS: illegal request
-    1:base.error_code  err;
-    2:i32              appid;
-}
-
-struct duplication_entry
-{
-    1:i32                  dupid;
-    2:duplication_status   status;
-    3:string               remote;
-    4:i64                  create_ts;
-
-    // partition_index => confirmed decree
-    5:optional map<i32, i64> progress;
-
-    7:optional duplication_fail_mode fail_mode;
-}
-
-// This request is sent from client to meta.
-struct duplication_query_request
-{
-    1:string                    app_name;
-}
-
-struct duplication_query_response
-{
-    // Possible errors:
-    // - ERR_APP_NOT_EXIST: app is not found
-    1:base.error_code            err;
-    3:i32                        appid;
-    4:list<duplication_entry>    entry_list;
-}
-
-struct policy_entry
-{
-    1:string        policy_name;
-    2:string        backup_provider_type;
-    3:string        backup_interval_seconds;
-    4:set<i32>      app_ids;
-    5:i32           backup_history_count_to_keep;
-    6:string        start_time;
-    7:bool          is_disable;
-}
-
-struct backup_entry
-{
-    1:i64           backup_id;
-    2:i64           start_time_ms;
-    3:i64           end_time_ms;
-    4:set<i32>      app_ids;
-}
-
-struct query_backup_policy_request
-{
-    1:list<string>      policy_names;
-    2:i32               backup_info_count;
-}
-
-struct query_backup_policy_response
-{
-    1:base.error_code           err;
-    2:list<policy_entry>        policys;
-    3:list<list<backup_entry>>  backup_infos;
-    4:optional string           hint_msg;
-}
-
-/////////////////// rebalance-related structs ////////////////////
-
-enum config_type
-{
-    CT_INVALID,
-    CT_ASSIGN_PRIMARY,
-    CT_UPGRADE_TO_PRIMARY,
-    CT_ADD_SECONDARY,
-    CT_UPGRADE_TO_SECONDARY, // not used by meta server
-    CT_DOWNGRADE_TO_SECONDARY,
-    CT_DOWNGRADE_TO_INACTIVE,
-    CT_REMOVE,
-    CT_ADD_SECONDARY_FOR_LB,
-    CT_PRIMARY_FORCE_UPDATE_BALLOT,
-    CT_DROP_PARTITION,
-    CT_REGISTER_CHILD
-}
-
-struct configuration_proposal_action
-{
-    1:base.rpc_address target;
-    2:base.rpc_address node;
-    3:config_type type;
-}
-
-enum balancer_request_type
-{
-    move_primary,
-    copy_primary,
-    copy_secondary,
-}
-
-struct balance_request
-{
-    1:base.gpid gpid;
-    2:list<configuration_proposal_action> action_list;
-    3:optional bool force = false;
-    4:optional balancer_request_type balance_type;
-}
-
-struct balance_response
-{
-    1:base.error_code err;
-}
-
-/////////////////// backup&restore-related structs ////////////////////
-
-struct start_backup_app_request
-{
-    1:string             backup_provider_type;
-    2:i32                app_id;
-    // user specified backup_path.
-    3:optional string    backup_path;
-}
-
-struct start_backup_app_response
-{
-    // Possible error:
-    // - ERR_INVALID_STATE: app is not available or is backing up
-    // - ERR_INVALID_PARAMETERS: backup provider type is invalid
-    // - ERR_SERVICE_NOT_ACTIVE: meta doesn't enable backup service
-    1:base.error_code    err;
-    2:string            hint_message;
-    3:optional i64      backup_id;
-}
-
-struct backup_item
-{
-    1:i64           backup_id;
-    2:string        app_name;
-    3:string        backup_provider_type;
-    // user specified backup_path.
-    4:string        backup_path;
-    5:i64           start_time_ms;
-    6:i64           end_time_ms;
-    7:bool          is_backup_failed;
-}
-
-struct query_backup_status_request
-{
-    1:i32                 app_id;
-    2:optional i64        backup_id;
-}
-
-struct query_backup_status_response
-{
-    // Possible error:
-    // - ERR_INVALID_PARAMETERS: no available backup for requested app
-    // - ERR_SERVICE_NOT_ACTIVE: meta doesn't enable backup service
-    1:base.error_code                 err;
-    2:string                         hint_message;
-    3:optional list<backup_item>     backup_items;
-}
-
-// using create_app_response to response
-struct restore_app_request
-{
-    1:string            cluster_name;
-    2:string            policy_name;
-    3:i64               time_stamp;   // namely backup_id
-    4:string            app_name;
-    5:i32               app_id;
-    6:string            new_app_name;
-    7:string            backup_provider_name;
-    8:bool              skip_bad_partition;
-    9:optional string   restore_path;
-}
-
-/////////////////// split-related structs ////////////////////
-
-enum split_status
-{
-    // idle state
-    NOT_SPLIT,
-    // A replica is splitting into two replicas, original one called parent, new one called child
-    SPLITTING,
-    PAUSING,
-    PAUSED,
-    // After split is successfully cancelled, the state turns into NOT_SPLIT
-    CANCELING
-}
-
-struct start_partition_split_request
-{
-    1:string    app_name;
-    2:i32       new_partition_count;
-}
-
-struct start_partition_split_response
-{
-    // Possible errors:
-    // - ERR_APP_NOT_EXIST: app not exist
-    // - ERR_APP_DROPPED: app has been dropped
-    // - ERR_INVALID_PARAMETERS: if the given new_partition_count != old_partition_count * 2
-    // - ERR_BUSY - if app is already executing partition split
-    1:base.error_code    err;
-    2:string            hint_msg;
-}
-
-enum split_control_type
-{
-    PAUSE,
-    RESTART,
-    CANCEL
-}
-
-// support three control type: pause, restart, cancel
-struct control_split_request
-{
-    1:string                app_name;
-    2:split_control_type    control_type
-    // for pause, parent_pidx >= 0, pause specific partition, parent_pidx = -1, pause all splitting partition
-    // for restart, parent_pidx >= 0, restart specific partition, parent_pidx = -1, restart all paused partition
-    // for cancel, parent_pidx will always be -1
-    3:i32                   parent_pidx;
-    // only used for cancel
-    4:optional i32          old_partition_count;
-}
-
-struct control_split_response
-{
-    // Possible errors:
-    // - ERR_APP_NOT_EXIST: app not exist
-    // - ERR_APP_DROPPED: app has been dropped
-    // - ERR_INVALID_STATE: wrong partition split_status
-    // - ERR_INVALID_PARAMETERS: invalid parent_pidx or old_partition_count
-    // - ERR_CHILD_REGISTERED: child partition has been registered, pause partition split or cancel split failed
-    1:base.error_code    err;
-    2:optional string   hint_msg;
-}
-
-// client->meta server to query partition split status
-struct query_split_request
-{
-    1:string    app_name;
-}
-
-struct query_split_response
-{
-    // Possible errors:
-    // - ERR_APP_NOT_EXIST: app not exist
-    // - ERR_APP_DROPPED: app has been dropped
-    // - ERR_INVALID_STATE: app is not splitting
-    1:base.error_code            err;
-    2:i32                       new_partition_count;
-    3:map<i32,split_status>     status;
-    4:optional string           hint_msg;
-}
-
-/////////////////// bulk-load-related structs ////////////////////
-
-struct start_bulk_load_request
-{
-    1:string    app_name;
-    2:string    cluster_name;
-    3:string    file_provider_type;
-    4:string    remote_root_path;
-}
-
-struct start_bulk_load_response
-{
-    // Possible error:
-    // - ERR_OK: start bulk load succeed
-    // - ERR_APP_NOT_EXIST: app not exist
-    // - ERR_APP_DROPPED: app has been dropped
-    // - ERR_BUSY: app is already executing bulk load
-    // - ERR_INVALID_PARAMETERS: wrong file_provider type
-    // - ERR_FILE_OPERATION_FAILED: remote file_provider error
-    // - ERR_OBJECT_NOT_FOUND: bulk_load_info not exist on file_provider
-    // - ERR_CORRUPTION: bulk_load_info is damaged on file_provider
-    // - ERR_INCONSISTENT_STATE: app_id or partition_count inconsistent
-    1:base.error_code   err;
-    2:string            hint_msg;
-}
-
-enum bulk_load_control_type
-{
-    BLC_PAUSE,
-    BLC_RESTART,
-    BLC_CANCEL,
-    BLC_FORCE_CANCEL
-}
-
-struct control_bulk_load_request
-{
-    1:string                    app_name;
-    2:bulk_load_control_type    type;
-}
-
-struct control_bulk_load_response
-{
-    // Possible error:
-    // - ERR_APP_NOT_EXIST: app not exist
-    // - ERR_APP_DROPPED: app has been dropped
-    // - ERR_INACTIVE_STATE: app is not executing bulk load
-    // - ERR_INVALID_STATE: current bulk load process can not be paused/restarted/canceled
-    1:base.error_code   err;
-    2:optional string   hint_msg;
-}
-
-enum bulk_load_status
-{
-    BLS_INVALID,
-    BLS_DOWNLOADING,
-    BLS_DOWNLOADED,
-    BLS_INGESTING,
-    BLS_SUCCEED,
-    BLS_FAILED,
-    BLS_PAUSING,
-    BLS_PAUSED,
-    BLS_CANCELED
-}
-
-enum ingestion_status
-{
-    IS_INVALID,
-    IS_RUNNING,
-    IS_SUCCEED,
-    IS_FAILED
-}
-
-struct partition_bulk_load_state
-{
-    1:optional i32              download_progress = 0;
-    2:optional base.error_code  download_status;
-    3:optional ingestion_status ingest_status = ingestion_status.IS_INVALID;
-    4:optional bool             is_cleaned_up = false;
-    5:optional bool             is_paused = false;
-}
-
-struct query_bulk_load_request
-{
-    1:string   app_name;
-}
-
-struct query_bulk_load_response
-{
-    // Possible error:
-    // - ERR_APP_NOT_EXIST: app not exist
-    // - ERR_APP_DROPPED: app has been dropped
-    // - ERR_INVALID_STATE: app is not executing bulk load
-    1:base.error_code                                           err;
-    2:string                                                    app_name;
-    3:bulk_load_status                                          app_status;
-    4:list<bulk_load_status>                                    partitions_status;
-    5:i32                                                       max_replica_count;
-    // detailed bulk load state for each replica
-    6:list<map<base.rpc_address, partition_bulk_load_state>>    bulk_load_states;
-    7:optional string                                           hint_msg;
-}
-
-struct clear_bulk_load_state_request
-{
-    1:string                    app_name;
-}
-
-struct clear_bulk_load_state_response
-{
-    // Possible error:
-    // - ERR_APP_NOT_EXIST: app not exist
-    // - ERR_APP_DROPPED: app has been dropped
-    // - ERR_INVALID_STATE: app is executing bulk load
-    1:base.error_code    err;
-    2:string            hint_msg;
-}
-
-// A client to MetaServer's administration API.
-service admin_client 
-{
-    create_app_response create_app(1:create_app_request req);
-    
-    drop_app_response drop_app(1:drop_app_request req);
-
-    recall_app_response recall_app(1:recall_app_request req);
-    
-    list_apps_response list_apps(1:list_apps_request req);
-
-    duplication_add_response add_duplication(1: duplication_add_request req);
-
-    duplication_query_response query_duplication(1: duplication_query_request req);
-
-    duplication_modify_response modify_duplication(1: duplication_modify_request req);
-
-    query_app_info_response query_app_info(1: query_app_info_request req);
-
-    update_app_env_response update_app_env(1: update_app_env_request req);
-
-    list_nodes_response list_nodes(1: list_nodes_request req);
-
-    cluster_info_response query_cluster_info(1: cluster_info_request req);
-
-    meta_control_response meta_control(1: meta_control_request req);
-
-    query_backup_policy_response query_backup_policy(1: query_backup_policy_request req);
-
-    balance_response balance(1: balance_request req);
-
-    start_backup_app_response start_backup_app(1: start_backup_app_request req);
-
-    query_backup_status_response query_backup_status(1: query_backup_status_request req);
-
-    create_app_response restore_app(1: restore_app_request req);
-
-    start_partition_split_response start_partition_split(1: start_partition_split_request req);
-
-    query_split_response query_split_status(1: query_split_request req);
-
-    control_split_response control_partition_split(1: control_split_request req);
-
-    start_bulk_load_response start_bulk_load(1: start_bulk_load_request req);
-
-    query_bulk_load_response query_bulk_load_status(1: query_bulk_load_request req);
-
-    control_bulk_load_response control_bulk_load(1: control_bulk_load_request req);
-
-    clear_bulk_load_state_response clear_bulk_load(1: clear_bulk_load_state_request req);
-
-    start_app_manual_compact_response start_manual_compact(1: start_app_manual_compact_request req);
-
-    query_app_manual_compact_response query_manual_compact(1: query_app_manual_compact_request req);
-}
diff --git a/go-client/idl/admin/GoUnusedProtection__.go b/go-client/idl/admin/GoUnusedProtection__.go
deleted file mode 100644
index 86e6c7e05..000000000
--- a/go-client/idl/admin/GoUnusedProtection__.go
+++ /dev/null
@@ -1,6 +0,0 @@
-// Autogenerated by Thrift Compiler (0.13.0)
-// DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING
-
-package admin
-
-var GoUnusedProtection__ int
diff --git a/go-client/idl/admin/admin-consts.go b/go-client/idl/admin/admin-consts.go
deleted file mode 100644
index 7efc5b0c9..000000000
--- a/go-client/idl/admin/admin-consts.go
+++ /dev/null
@@ -1,25 +0,0 @@
-// Autogenerated by Thrift Compiler (0.13.0)
-// DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING
-
-package admin
-
-import (
-	"bytes"
-	"context"
-	"fmt"
-	"github.com/apache/incubator-pegasus/go-client/idl/base"
-	"github.com/apache/thrift/lib/go/thrift"
-	"reflect"
-)
-
-// (needed to ensure safety because of naive import list construction.)
-var _ = thrift.ZERO
-var _ = fmt.Printf
-var _ = context.Background
-var _ = reflect.DeepEqual
-var _ = bytes.Equal
-
-var _ = base.GoUnusedProtection__
-
-func init() {
-}
diff --git a/go-client/idl/admin/admin.go b/go-client/idl/admin/admin.go
deleted file mode 100644
index de5414a0d..000000000
--- a/go-client/idl/admin/admin.go
+++ /dev/null
@@ -1,20627 +0,0 @@
-// Autogenerated by Thrift Compiler (0.13.0)
-// DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING
-
-package admin
-
-import (
-	"bytes"
-	"context"
-	"database/sql/driver"
-	"errors"
-	"fmt"
-	"github.com/apache/incubator-pegasus/go-client/idl/base"
-	"github.com/apache/thrift/lib/go/thrift"
-	"reflect"
-)
-
-// (needed to ensure safety because of naive import list construction.)
-var _ = thrift.ZERO
-var _ = fmt.Printf
-var _ = context.Background
-var _ = reflect.DeepEqual
-var _ = bytes.Equal
-
-var _ = base.GoUnusedProtection__
-
-type AppStatus int64
-
-const (
-	AppStatus_AS_INVALID       AppStatus = 0
-	AppStatus_AS_AVAILABLE     AppStatus = 1
-	AppStatus_AS_CREATING      AppStatus = 2
-	AppStatus_AS_CREATE_FAILED AppStatus = 3
-	AppStatus_AS_DROPPING      AppStatus = 4
-	AppStatus_AS_DROP_FAILED   AppStatus = 5
-	AppStatus_AS_DROPPED       AppStatus = 6
-	AppStatus_AS_RECALLING     AppStatus = 7
-)
-
-func (p AppStatus) String() string {
-	switch p {
-	case AppStatus_AS_INVALID:
-		return "AS_INVALID"
-	case AppStatus_AS_AVAILABLE:
-		return "AS_AVAILABLE"
-	case AppStatus_AS_CREATING:
-		return "AS_CREATING"
-	case AppStatus_AS_CREATE_FAILED:
-		return "AS_CREATE_FAILED"
-	case AppStatus_AS_DROPPING:
-		return "AS_DROPPING"
-	case AppStatus_AS_DROP_FAILED:
-		return "AS_DROP_FAILED"
-	case AppStatus_AS_DROPPED:
-		return "AS_DROPPED"
-	case AppStatus_AS_RECALLING:
-		return "AS_RECALLING"
-	}
-	return "<UNSET>"
-}
-
-func AppStatusFromString(s string) (AppStatus, error) {
-	switch s {
-	case "AS_INVALID":
-		return AppStatus_AS_INVALID, nil
-	case "AS_AVAILABLE":
-		return AppStatus_AS_AVAILABLE, nil
-	case "AS_CREATING":
-		return AppStatus_AS_CREATING, nil
-	case "AS_CREATE_FAILED":
-		return AppStatus_AS_CREATE_FAILED, nil
-	case "AS_DROPPING":
-		return AppStatus_AS_DROPPING, nil
-	case "AS_DROP_FAILED":
-		return AppStatus_AS_DROP_FAILED, nil
-	case "AS_DROPPED":
-		return AppStatus_AS_DROPPED, nil
-	case "AS_RECALLING":
-		return AppStatus_AS_RECALLING, nil
-	}
-	return AppStatus(0), fmt.Errorf("not a valid AppStatus string")
-}
-
-func AppStatusPtr(v AppStatus) *AppStatus { return &v }
-
-func (p AppStatus) MarshalText() ([]byte, error) {
-	return []byte(p.String()), nil
-}
-
-func (p *AppStatus) UnmarshalText(text []byte) error {
-	q, err := AppStatusFromString(string(text))
-	if err != nil {
-		return err
-	}
-	*p = q
-	return nil
-}
-
-func (p *AppStatus) Scan(value interface{}) error {
-	v, ok := value.(int64)
-	if !ok {
-		return errors.New("Scan value is not int64")
-	}
-	*p = AppStatus(v)
-	return nil
-}
-
-func (p *AppStatus) Value() (driver.Value, error) {
-	if p == nil {
-		return nil, nil
-	}
-	return int64(*p), nil
-}
-
-type AppEnvOperation int64
-
-const (
-	AppEnvOperation_APP_ENV_OP_INVALID AppEnvOperation = 0
-	AppEnvOperation_APP_ENV_OP_SET     AppEnvOperation = 1
-	AppEnvOperation_APP_ENV_OP_DEL     AppEnvOperation = 2
-	AppEnvOperation_APP_ENV_OP_CLEAR   AppEnvOperation = 3
-)
-
-func (p AppEnvOperation) String() string {
-	switch p {
-	case AppEnvOperation_APP_ENV_OP_INVALID:
-		return "APP_ENV_OP_INVALID"
-	case AppEnvOperation_APP_ENV_OP_SET:
-		return "APP_ENV_OP_SET"
-	case AppEnvOperation_APP_ENV_OP_DEL:
-		return "APP_ENV_OP_DEL"
-	case AppEnvOperation_APP_ENV_OP_CLEAR:
-		return "APP_ENV_OP_CLEAR"
-	}
-	return "<UNSET>"
-}
-
-func AppEnvOperationFromString(s string) (AppEnvOperation, error) {
-	switch s {
-	case "APP_ENV_OP_INVALID":
-		return AppEnvOperation_APP_ENV_OP_INVALID, nil
-	case "APP_ENV_OP_SET":
-		return AppEnvOperation_APP_ENV_OP_SET, nil
-	case "APP_ENV_OP_DEL":
-		return AppEnvOperation_APP_ENV_OP_DEL, nil
-	case "APP_ENV_OP_CLEAR":
-		return AppEnvOperation_APP_ENV_OP_CLEAR, nil
-	}
-	return AppEnvOperation(0), fmt.Errorf("not a valid AppEnvOperation string")
-}
-
-func AppEnvOperationPtr(v AppEnvOperation) *AppEnvOperation { return &v }
-
-func (p AppEnvOperation) MarshalText() ([]byte, error) {
-	return []byte(p.String()), nil
-}
-
-func (p *AppEnvOperation) UnmarshalText(text []byte) error {
-	q, err := AppEnvOperationFromString(string(text))
-	if err != nil {
-		return err
-	}
-	*p = q
-	return nil
-}
-
-func (p *AppEnvOperation) Scan(value interface{}) error {
-	v, ok := value.(int64)
-	if !ok {
-		return errors.New("Scan value is not int64")
-	}
-	*p = AppEnvOperation(v)
-	return nil
-}
-
-func (p *AppEnvOperation) Value() (driver.Value, error) {
-	if p == nil {
-		return nil, nil
-	}
-	return int64(*p), nil
-}
-
-type NodeStatus int64
-
-const (
-	NodeStatus_NS_INVALID NodeStatus = 0
-	NodeStatus_NS_ALIVE   NodeStatus = 1
-	NodeStatus_NS_UNALIVE NodeStatus = 2
-)
-
-func (p NodeStatus) String() string {
-	switch p {
-	case NodeStatus_NS_INVALID:
-		return "NS_INVALID"
-	case NodeStatus_NS_ALIVE:
-		return "NS_ALIVE"
-	case NodeStatus_NS_UNALIVE:
-		return "NS_UNALIVE"
-	}
-	return "<UNSET>"
-}
-
-func NodeStatusFromString(s string) (NodeStatus, error) {
-	switch s {
-	case "NS_INVALID":
-		return NodeStatus_NS_INVALID, nil
-	case "NS_ALIVE":
-		return NodeStatus_NS_ALIVE, nil
-	case "NS_UNALIVE":
-		return NodeStatus_NS_UNALIVE, nil
-	}
-	return NodeStatus(0), fmt.Errorf("not a valid NodeStatus string")
-}
-
-func NodeStatusPtr(v NodeStatus) *NodeStatus { return &v }
-
-func (p NodeStatus) MarshalText() ([]byte, error) {
-	return []byte(p.String()), nil
-}
-
-func (p *NodeStatus) UnmarshalText(text []byte) error {
-	q, err := NodeStatusFromString(string(text))
-	if err != nil {
-		return err
-	}
-	*p = q
-	return nil
-}
-
-func (p *NodeStatus) Scan(value interface{}) error {
-	v, ok := value.(int64)
-	if !ok {
-		return errors.New("Scan value is not int64")
-	}
-	*p = NodeStatus(v)
-	return nil
-}
-
-func (p *NodeStatus) Value() (driver.Value, error) {
-	if p == nil {
-		return nil, nil
-	}
-	return int64(*p), nil
-}
-
-type MetaFunctionLevel int64
-
-const (
-	MetaFunctionLevel_fl_stopped MetaFunctionLevel = 100
-	MetaFunctionLevel_fl_blind   MetaFunctionLevel = 200
-	MetaFunctionLevel_fl_freezed MetaFunctionLevel = 300
-	MetaFunctionLevel_fl_steady  MetaFunctionLevel = 400
-	MetaFunctionLevel_fl_lively  MetaFunctionLevel = 500
-	MetaFunctionLevel_fl_invalid MetaFunctionLevel = 10000
-)
-
-func (p MetaFunctionLevel) String() string {
-	switch p {
-	case MetaFunctionLevel_fl_stopped:
-		return "fl_stopped"
-	case MetaFunctionLevel_fl_blind:
-		return "fl_blind"
-	case MetaFunctionLevel_fl_freezed:
-		return "fl_freezed"
-	case MetaFunctionLevel_fl_steady:
-		return "fl_steady"
-	case MetaFunctionLevel_fl_lively:
-		return "fl_lively"
-	case MetaFunctionLevel_fl_invalid:
-		return "fl_invalid"
-	}
-	return "<UNSET>"
-}
-
-func MetaFunctionLevelFromString(s string) (MetaFunctionLevel, error) {
-	switch s {
-	case "fl_stopped":
-		return MetaFunctionLevel_fl_stopped, nil
-	case "fl_blind":
-		return MetaFunctionLevel_fl_blind, nil
-	case "fl_freezed":
-		return MetaFunctionLevel_fl_freezed, nil
-	case "fl_steady":
-		return MetaFunctionLevel_fl_steady, nil
-	case "fl_lively":
-		return MetaFunctionLevel_fl_lively, nil
-	case "fl_invalid":
-		return MetaFunctionLevel_fl_invalid, nil
-	}
-	return MetaFunctionLevel(0), fmt.Errorf("not a valid MetaFunctionLevel string")
-}
-
-func MetaFunctionLevelPtr(v MetaFunctionLevel) *MetaFunctionLevel { return &v }
-
-func (p MetaFunctionLevel) MarshalText() ([]byte, error) {
-	return []byte(p.String()), nil
-}
-
-func (p *MetaFunctionLevel) UnmarshalText(text []byte) error {
-	q, err := MetaFunctionLevelFromString(string(text))
-	if err != nil {
-		return err
-	}
-	*p = q
-	return nil
-}
-
-func (p *MetaFunctionLevel) Scan(value interface{}) error {
-	v, ok := value.(int64)
-	if !ok {
-		return errors.New("Scan value is not int64")
-	}
-	*p = MetaFunctionLevel(v)
-	return nil
-}
-
-func (p *MetaFunctionLevel) Value() (driver.Value, error) {
-	if p == nil {
-		return nil, nil
-	}
-	return int64(*p), nil
-}
-
-type DuplicationStatus int64
-
-const (
-	DuplicationStatus_DS_INIT    DuplicationStatus = 0
-	DuplicationStatus_DS_PREPARE DuplicationStatus = 1
-	DuplicationStatus_DS_APP     DuplicationStatus = 2
-	DuplicationStatus_DS_LOG     DuplicationStatus = 3
-	DuplicationStatus_DS_PAUSE   DuplicationStatus = 4
-	DuplicationStatus_DS_REMOVED DuplicationStatus = 5
-)
-
-func (p DuplicationStatus) String() string {
-	switch p {
-	case DuplicationStatus_DS_INIT:
-		return "DS_INIT"
-	case DuplicationStatus_DS_PREPARE:
-		return "DS_PREPARE"
-	case DuplicationStatus_DS_APP:
-		return "DS_APP"
-	case DuplicationStatus_DS_LOG:
-		return "DS_LOG"
-	case DuplicationStatus_DS_PAUSE:
-		return "DS_PAUSE"
-	case DuplicationStatus_DS_REMOVED:
-		return "DS_REMOVED"
-	}
-	return "<UNSET>"
-}
-
-func DuplicationStatusFromString(s string) (DuplicationStatus, error) {
-	switch s {
-	case "DS_INIT":
-		return DuplicationStatus_DS_INIT, nil
-	case "DS_PREPARE":
-		return DuplicationStatus_DS_PREPARE, nil
-	case "DS_APP":
-		return DuplicationStatus_DS_APP, nil
-	case "DS_LOG":
-		return DuplicationStatus_DS_LOG, nil
-	case "DS_PAUSE":
-		return DuplicationStatus_DS_PAUSE, nil
-	case "DS_REMOVED":
-		return DuplicationStatus_DS_REMOVED, nil
-	}
-	return DuplicationStatus(0), fmt.Errorf("not a valid DuplicationStatus string")
-}
-
-func DuplicationStatusPtr(v DuplicationStatus) *DuplicationStatus { return &v }
-
-func (p DuplicationStatus) MarshalText() ([]byte, error) {
-	return []byte(p.String()), nil
-}
-
-func (p *DuplicationStatus) UnmarshalText(text []byte) error {
-	q, err := DuplicationStatusFromString(string(text))
-	if err != nil {
-		return err
-	}
-	*p = q
-	return nil
-}
-
-func (p *DuplicationStatus) Scan(value interface{}) error {
-	v, ok := value.(int64)
-	if !ok {
-		return errors.New("Scan value is not int64")
-	}
-	*p = DuplicationStatus(v)
-	return nil
-}
-
-func (p *DuplicationStatus) Value() (driver.Value, error) {
-	if p == nil {
-		return nil, nil
-	}
-	return int64(*p), nil
-}
-
-type DuplicationFailMode int64
-
-const (
-	DuplicationFailMode_FAIL_SLOW DuplicationFailMode = 0
-	DuplicationFailMode_FAIL_SKIP DuplicationFailMode = 1
-	DuplicationFailMode_FAIL_FAST DuplicationFailMode = 2
-)
-
-func (p DuplicationFailMode) String() string {
-	switch p {
-	case DuplicationFailMode_FAIL_SLOW:
-		return "FAIL_SLOW"
-	case DuplicationFailMode_FAIL_SKIP:
-		return "FAIL_SKIP"
-	case DuplicationFailMode_FAIL_FAST:
-		return "FAIL_FAST"
-	}
-	return "<UNSET>"
-}
-
-func DuplicationFailModeFromString(s string) (DuplicationFailMode, error) {
-	switch s {
-	case "FAIL_SLOW":
-		return DuplicationFailMode_FAIL_SLOW, nil
-	case "FAIL_SKIP":
-		return DuplicationFailMode_FAIL_SKIP, nil
-	case "FAIL_FAST":
-		return DuplicationFailMode_FAIL_FAST, nil
-	}
-	return DuplicationFailMode(0), fmt.Errorf("not a valid DuplicationFailMode string")
-}
-
-func DuplicationFailModePtr(v DuplicationFailMode) *DuplicationFailMode { return &v }
-
-func (p DuplicationFailMode) MarshalText() ([]byte, error) {
-	return []byte(p.String()), nil
-}
-
-func (p *DuplicationFailMode) UnmarshalText(text []byte) error {
-	q, err := DuplicationFailModeFromString(string(text))
-	if err != nil {
-		return err
-	}
-	*p = q
-	return nil
-}
-
-func (p *DuplicationFailMode) Scan(value interface{}) error {
-	v, ok := value.(int64)
-	if !ok {
-		return errors.New("Scan value is not int64")
-	}
-	*p = DuplicationFailMode(v)
-	return nil
-}
-
-func (p *DuplicationFailMode) Value() (driver.Value, error) {
-	if p == nil {
-		return nil, nil
-	}
-	return int64(*p), nil
-}
-
-type ConfigType int64
-
-const (
-	ConfigType_CT_INVALID                     ConfigType = 0
-	ConfigType_CT_ASSIGN_PRIMARY              ConfigType = 1
-	ConfigType_CT_UPGRADE_TO_PRIMARY          ConfigType = 2
-	ConfigType_CT_ADD_SECONDARY               ConfigType = 3
-	ConfigType_CT_UPGRADE_TO_SECONDARY        ConfigType = 4
-	ConfigType_CT_DOWNGRADE_TO_SECONDARY      ConfigType = 5
-	ConfigType_CT_DOWNGRADE_TO_INACTIVE       ConfigType = 6
-	ConfigType_CT_REMOVE                      ConfigType = 7
-	ConfigType_CT_ADD_SECONDARY_FOR_LB        ConfigType = 8
-	ConfigType_CT_PRIMARY_FORCE_UPDATE_BALLOT ConfigType = 9
-	ConfigType_CT_DROP_PARTITION              ConfigType = 10
-	ConfigType_CT_REGISTER_CHILD              ConfigType = 11
-)
-
-func (p ConfigType) String() string {
-	switch p {
-	case ConfigType_CT_INVALID:
-		return "CT_INVALID"
-	case ConfigType_CT_ASSIGN_PRIMARY:
-		return "CT_ASSIGN_PRIMARY"
-	case ConfigType_CT_UPGRADE_TO_PRIMARY:
-		return "CT_UPGRADE_TO_PRIMARY"
-	case ConfigType_CT_ADD_SECONDARY:
-		return "CT_ADD_SECONDARY"
-	case ConfigType_CT_UPGRADE_TO_SECONDARY:
-		return "CT_UPGRADE_TO_SECONDARY"
-	case ConfigType_CT_DOWNGRADE_TO_SECONDARY:
-		return "CT_DOWNGRADE_TO_SECONDARY"
-	case ConfigType_CT_DOWNGRADE_TO_INACTIVE:
-		return "CT_DOWNGRADE_TO_INACTIVE"
-	case ConfigType_CT_REMOVE:
-		return "CT_REMOVE"
-	case ConfigType_CT_ADD_SECONDARY_FOR_LB:
-		return "CT_ADD_SECONDARY_FOR_LB"
-	case ConfigType_CT_PRIMARY_FORCE_UPDATE_BALLOT:
-		return "CT_PRIMARY_FORCE_UPDATE_BALLOT"
-	case ConfigType_CT_DROP_PARTITION:
-		return "CT_DROP_PARTITION"
-	case ConfigType_CT_REGISTER_CHILD:
-		return "CT_REGISTER_CHILD"
-	}
-	return "<UNSET>"
-}
-
-func ConfigTypeFromString(s string) (ConfigType, error) {
-	switch s {
-	case "CT_INVALID":
-		return ConfigType_CT_INVALID, nil
-	case "CT_ASSIGN_PRIMARY":
-		return ConfigType_CT_ASSIGN_PRIMARY, nil
-	case "CT_UPGRADE_TO_PRIMARY":
-		return ConfigType_CT_UPGRADE_TO_PRIMARY, nil
-	case "CT_ADD_SECONDARY":
-		return ConfigType_CT_ADD_SECONDARY, nil
-	case "CT_UPGRADE_TO_SECONDARY":
-		return ConfigType_CT_UPGRADE_TO_SECONDARY, nil
-	case "CT_DOWNGRADE_TO_SECONDARY":
-		return ConfigType_CT_DOWNGRADE_TO_SECONDARY, nil
-	case "CT_DOWNGRADE_TO_INACTIVE":
-		return ConfigType_CT_DOWNGRADE_TO_INACTIVE, nil
-	case "CT_REMOVE":
-		return ConfigType_CT_REMOVE, nil
-	case "CT_ADD_SECONDARY_FOR_LB":
-		return ConfigType_CT_ADD_SECONDARY_FOR_LB, nil
-	case "CT_PRIMARY_FORCE_UPDATE_BALLOT":
-		return ConfigType_CT_PRIMARY_FORCE_UPDATE_BALLOT, nil
-	case "CT_DROP_PARTITION":
-		return ConfigType_CT_DROP_PARTITION, nil
-	case "CT_REGISTER_CHILD":
-		return ConfigType_CT_REGISTER_CHILD, nil
-	}
-	return ConfigType(0), fmt.Errorf("not a valid ConfigType string")
-}
-
-func ConfigTypePtr(v ConfigType) *ConfigType { return &v }
-
-func (p ConfigType) MarshalText() ([]byte, error) {
-	return []byte(p.String()), nil
-}
-
-func (p *ConfigType) UnmarshalText(text []byte) error {
-	q, err := ConfigTypeFromString(string(text))
-	if err != nil {
-		return err
-	}
-	*p = q
-	return nil
-}
-
-func (p *ConfigType) Scan(value interface{}) error {
-	v, ok := value.(int64)
-	if !ok {
-		return errors.New("Scan value is not int64")
-	}
-	*p = ConfigType(v)
-	return nil
-}
-
-func (p *ConfigType) Value() (driver.Value, error) {
-	if p == nil {
-		return nil, nil
-	}
-	return int64(*p), nil
-}
-
-type BalancerRequestType int64
-
-const (
-	BalancerRequestType_move_primary   BalancerRequestType = 0
-	BalancerRequestType_copy_primary   BalancerRequestType = 1
-	BalancerRequestType_copy_secondary BalancerRequestType = 2
-)
-
-func (p BalancerRequestType) String() string {
-	switch p {
-	case BalancerRequestType_move_primary:
-		return "move_primary"
-	case BalancerRequestType_copy_primary:
-		return "copy_primary"
-	case BalancerRequestType_copy_secondary:
-		return "copy_secondary"
-	}
-	return "<UNSET>"
-}
-
-func BalancerRequestTypeFromString(s string) (BalancerRequestType, error) {
-	switch s {
-	case "move_primary":
-		return BalancerRequestType_move_primary, nil
-	case "copy_primary":
-		return BalancerRequestType_copy_primary, nil
-	case "copy_secondary":
-		return BalancerRequestType_copy_secondary, nil
-	}
-	return BalancerRequestType(0), fmt.Errorf("not a valid BalancerRequestType string")
-}
-
-func BalancerRequestTypePtr(v BalancerRequestType) *BalancerRequestType { return &v }
-
-func (p BalancerRequestType) MarshalText() ([]byte, error) {
-	return []byte(p.String()), nil
-}
-
-func (p *BalancerRequestType) UnmarshalText(text []byte) error {
-	q, err := BalancerRequestTypeFromString(string(text))
-	if err != nil {
-		return err
-	}
-	*p = q
-	return nil
-}
-
-func (p *BalancerRequestType) Scan(value interface{}) error {
-	v, ok := value.(int64)
-	if !ok {
-		return errors.New("Scan value is not int64")
-	}
-	*p = BalancerRequestType(v)
-	return nil
-}
-
-func (p *BalancerRequestType) Value() (driver.Value, error) {
-	if p == nil {
-		return nil, nil
-	}
-	return int64(*p), nil
-}
-
-type SplitStatus int64
-
-const (
-	SplitStatus_NOT_SPLIT SplitStatus = 0
-	SplitStatus_SPLITTING SplitStatus = 1
-	SplitStatus_PAUSING   SplitStatus = 2
-	SplitStatus_PAUSED    SplitStatus = 3
-	SplitStatus_CANCELING SplitStatus = 4
-)
-
-func (p SplitStatus) String() string {
-	switch p {
-	case SplitStatus_NOT_SPLIT:
-		return "NOT_SPLIT"
-	case SplitStatus_SPLITTING:
-		return "SPLITTING"
-	case SplitStatus_PAUSING:
-		return "PAUSING"
-	case SplitStatus_PAUSED:
-		return "PAUSED"
-	case SplitStatus_CANCELING:
-		return "CANCELING"
-	}
-	return "<UNSET>"
-}
-
-func SplitStatusFromString(s string) (SplitStatus, error) {
-	switch s {
-	case "NOT_SPLIT":
-		return SplitStatus_NOT_SPLIT, nil
-	case "SPLITTING":
-		return SplitStatus_SPLITTING, nil
-	case "PAUSING":
-		return SplitStatus_PAUSING, nil
-	case "PAUSED":
-		return SplitStatus_PAUSED, nil
-	case "CANCELING":
-		return SplitStatus_CANCELING, nil
-	}
-	return SplitStatus(0), fmt.Errorf("not a valid SplitStatus string")
-}
-
-func SplitStatusPtr(v SplitStatus) *SplitStatus { return &v }
-
-func (p SplitStatus) MarshalText() ([]byte, error) {
-	return []byte(p.String()), nil
-}
-
-func (p *SplitStatus) UnmarshalText(text []byte) error {
-	q, err := SplitStatusFromString(string(text))
-	if err != nil {
-		return err
-	}
-	*p = q
-	return nil
-}
-
-func (p *SplitStatus) Scan(value interface{}) error {
-	v, ok := value.(int64)
-	if !ok {
-		return errors.New("Scan value is not int64")
-	}
-	*p = SplitStatus(v)
-	return nil
-}
-
-func (p *SplitStatus) Value() (driver.Value, error) {
-	if p == nil {
-		return nil, nil
-	}
-	return int64(*p), nil
-}
-
-type SplitControlType int64
-
-const (
-	SplitControlType_PAUSE   SplitControlType = 0
-	SplitControlType_RESTART SplitControlType = 1
-	SplitControlType_CANCEL  SplitControlType = 2
-)
-
-func (p SplitControlType) String() string {
-	switch p {
-	case SplitControlType_PAUSE:
-		return "PAUSE"
-	case SplitControlType_RESTART:
-		return "RESTART"
-	case SplitControlType_CANCEL:
-		return "CANCEL"
-	}
-	return "<UNSET>"
-}
-
-func SplitControlTypeFromString(s string) (SplitControlType, error) {
-	switch s {
-	case "PAUSE":
-		return SplitControlType_PAUSE, nil
-	case "RESTART":
-		return SplitControlType_RESTART, nil
-	case "CANCEL":
-		return SplitControlType_CANCEL, nil
-	}
-	return SplitControlType(0), fmt.Errorf("not a valid SplitControlType string")
-}
-
-func SplitControlTypePtr(v SplitControlType) *SplitControlType { return &v }
-
-func (p SplitControlType) MarshalText() ([]byte, error) {
-	return []byte(p.String()), nil
-}
-
-func (p *SplitControlType) UnmarshalText(text []byte) error {
-	q, err := SplitControlTypeFromString(string(text))
-	if err != nil {
-		return err
-	}
-	*p = q
-	return nil
-}
-
-func (p *SplitControlType) Scan(value interface{}) error {
-	v, ok := value.(int64)
-	if !ok {
-		return errors.New("Scan value is not int64")
-	}
-	*p = SplitControlType(v)
-	return nil
-}
-
-func (p *SplitControlType) Value() (driver.Value, error) {
-	if p == nil {
-		return nil, nil
-	}
-	return int64(*p), nil
-}
-
-type BulkLoadControlType int64
-
-const (
-	BulkLoadControlType_BLC_PAUSE        BulkLoadControlType = 0
-	BulkLoadControlType_BLC_RESTART      BulkLoadControlType = 1
-	BulkLoadControlType_BLC_CANCEL       BulkLoadControlType = 2
-	BulkLoadControlType_BLC_FORCE_CANCEL BulkLoadControlType = 3
-)
-
-func (p BulkLoadControlType) String() string {
-	switch p {
-	case BulkLoadControlType_BLC_PAUSE:
-		return "BLC_PAUSE"
-	case BulkLoadControlType_BLC_RESTART:
-		return "BLC_RESTART"
-	case BulkLoadControlType_BLC_CANCEL:
-		return "BLC_CANCEL"
-	case BulkLoadControlType_BLC_FORCE_CANCEL:
-		return "BLC_FORCE_CANCEL"
-	}
-	return "<UNSET>"
-}
-
-func BulkLoadControlTypeFromString(s string) (BulkLoadControlType, error) {
-	switch s {
-	case "BLC_PAUSE":
-		return BulkLoadControlType_BLC_PAUSE, nil
-	case "BLC_RESTART":
-		return BulkLoadControlType_BLC_RESTART, nil
-	case "BLC_CANCEL":
-		return BulkLoadControlType_BLC_CANCEL, nil
-	case "BLC_FORCE_CANCEL":
-		return BulkLoadControlType_BLC_FORCE_CANCEL, nil
-	}
-	return BulkLoadControlType(0), fmt.Errorf("not a valid BulkLoadControlType string")
-}
-
-func BulkLoadControlTypePtr(v BulkLoadControlType) *BulkLoadControlType { return &v }
-
-func (p BulkLoadControlType) MarshalText() ([]byte, error) {
-	return []byte(p.String()), nil
-}
-
-func (p *BulkLoadControlType) UnmarshalText(text []byte) error {
-	q, err := BulkLoadControlTypeFromString(string(text))
-	if err != nil {
-		return err
-	}
-	*p = q
-	return nil
-}
-
-func (p *BulkLoadControlType) Scan(value interface{}) error {
-	v, ok := value.(int64)
-	if !ok {
-		return errors.New("Scan value is not int64")
-	}
-	*p = BulkLoadControlType(v)
-	return nil
-}
-
-func (p *BulkLoadControlType) Value() (driver.Value, error) {
-	if p == nil {
-		return nil, nil
-	}
-	return int64(*p), nil
-}
-
-type BulkLoadStatus int64
-
-const (
-	BulkLoadStatus_BLS_INVALID     BulkLoadStatus = 0
-	BulkLoadStatus_BLS_DOWNLOADING BulkLoadStatus = 1
-	BulkLoadStatus_BLS_DOWNLOADED  BulkLoadStatus = 2
-	BulkLoadStatus_BLS_INGESTING   BulkLoadStatus = 3
-	BulkLoadStatus_BLS_SUCCEED     BulkLoadStatus = 4
-	BulkLoadStatus_BLS_FAILED      BulkLoadStatus = 5
-	BulkLoadStatus_BLS_PAUSING     BulkLoadStatus = 6
-	BulkLoadStatus_BLS_PAUSED      BulkLoadStatus = 7
-	BulkLoadStatus_BLS_CANCELED    BulkLoadStatus = 8
-)
-
-func (p BulkLoadStatus) String() string {
-	switch p {
-	case BulkLoadStatus_BLS_INVALID:
-		return "BLS_INVALID"
-	case BulkLoadStatus_BLS_DOWNLOADING:
-		return "BLS_DOWNLOADING"
-	case BulkLoadStatus_BLS_DOWNLOADED:
-		return "BLS_DOWNLOADED"
-	case BulkLoadStatus_BLS_INGESTING:
-		return "BLS_INGESTING"
-	case BulkLoadStatus_BLS_SUCCEED:
-		return "BLS_SUCCEED"
-	case BulkLoadStatus_BLS_FAILED:
-		return "BLS_FAILED"
-	case BulkLoadStatus_BLS_PAUSING:
-		return "BLS_PAUSING"
-	case BulkLoadStatus_BLS_PAUSED:
-		return "BLS_PAUSED"
-	case BulkLoadStatus_BLS_CANCELED:
-		return "BLS_CANCELED"
-	}
-	return "<UNSET>"
-}
-
-func BulkLoadStatusFromString(s string) (BulkLoadStatus, error) {
-	switch s {
-	case "BLS_INVALID":
-		return BulkLoadStatus_BLS_INVALID, nil
-	case "BLS_DOWNLOADING":
-		return BulkLoadStatus_BLS_DOWNLOADING, nil
-	case "BLS_DOWNLOADED":
-		return BulkLoadStatus_BLS_DOWNLOADED, nil
-	case "BLS_INGESTING":
-		return BulkLoadStatus_BLS_INGESTING, nil
-	case "BLS_SUCCEED":
-		return BulkLoadStatus_BLS_SUCCEED, nil
-	case "BLS_FAILED":
-		return BulkLoadStatus_BLS_FAILED, nil
-	case "BLS_PAUSING":
-		return BulkLoadStatus_BLS_PAUSING, nil
-	case "BLS_PAUSED":
-		return BulkLoadStatus_BLS_PAUSED, nil
-	case "BLS_CANCELED":
-		return BulkLoadStatus_BLS_CANCELED, nil
-	}
-	return BulkLoadStatus(0), fmt.Errorf("not a valid BulkLoadStatus string")
-}
-
-func BulkLoadStatusPtr(v BulkLoadStatus) *BulkLoadStatus { return &v }
-
-func (p BulkLoadStatus) MarshalText() ([]byte, error) {
-	return []byte(p.String()), nil
-}
-
-func (p *BulkLoadStatus) UnmarshalText(text []byte) error {
-	q, err := BulkLoadStatusFromString(string(text))
-	if err != nil {
-		return err
-	}
-	*p = q
-	return nil
-}
-
-func (p *BulkLoadStatus) Scan(value interface{}) error {
-	v, ok := value.(int64)
-	if !ok {
-		return errors.New("Scan value is not int64")
-	}
-	*p = BulkLoadStatus(v)
-	return nil
-}
-
-func (p *BulkLoadStatus) Value() (driver.Value, error) {
-	if p == nil {
-		return nil, nil
-	}
-	return int64(*p), nil
-}
-
-type IngestionStatus int64
-
-const (
-	IngestionStatus_IS_INVALID IngestionStatus = 0
-	IngestionStatus_IS_RUNNING IngestionStatus = 1
-	IngestionStatus_IS_SUCCEED IngestionStatus = 2
-	IngestionStatus_IS_FAILED  IngestionStatus = 3
-)
-
-func (p IngestionStatus) String() string {
-	switch p {
-	case IngestionStatus_IS_INVALID:
-		return "IS_INVALID"
-	case IngestionStatus_IS_RUNNING:
-		return "IS_RUNNING"
-	case IngestionStatus_IS_SUCCEED:
-		return "IS_SUCCEED"
-	case IngestionStatus_IS_FAILED:
-		return "IS_FAILED"
-	}
-	return "<UNSET>"
-}
-
-func IngestionStatusFromString(s string) (IngestionStatus, error) {
-	switch s {
-	case "IS_INVALID":
-		return IngestionStatus_IS_INVALID, nil
-	case "IS_RUNNING":
-		return IngestionStatus_IS_RUNNING, nil
-	case "IS_SUCCEED":
-		return IngestionStatus_IS_SUCCEED, nil
-	case "IS_FAILED":
-		return IngestionStatus_IS_FAILED, nil
-	}
-	return IngestionStatus(0), fmt.Errorf("not a valid IngestionStatus string")
-}
-
-func IngestionStatusPtr(v IngestionStatus) *IngestionStatus { return &v }
-
-func (p IngestionStatus) MarshalText() ([]byte, error) {
-	return []byte(p.String()), nil
-}
-
-func (p *IngestionStatus) UnmarshalText(text []byte) error {
-	q, err := IngestionStatusFromString(string(text))
-	if err != nil {
-		return err
-	}
-	*p = q
-	return nil
-}
-
-func (p *IngestionStatus) Scan(value interface{}) error {
-	v, ok := value.(int64)
-	if !ok {
-		return errors.New("Scan value is not int64")
-	}
-	*p = IngestionStatus(v)
-	return nil
-}
-
-func (p *IngestionStatus) Value() (driver.Value, error) {
-	if p == nil {
-		return nil, nil
-	}
-	return int64(*p), nil
-}
-
-// Attributes:
-//  - PartitionCount
-//  - ReplicaCount
-//  - SuccessIfExist
-//  - AppType
-//  - IsStateful
-//  - Envs
-type CreateAppOptions struct {
-	PartitionCount int32             `thrift:"partition_count,1" db:"partition_count" json:"partition_count"`
-	ReplicaCount   int32             `thrift:"replica_count,2" db:"replica_count" json:"replica_count"`
-	SuccessIfExist bool              `thrift:"success_if_exist,3" db:"success_if_exist" json:"success_if_exist"`
-	AppType        string            `thrift:"app_type,4" db:"app_type" json:"app_type"`
-	IsStateful     bool              `thrift:"is_stateful,5" db:"is_stateful" json:"is_stateful"`
-	Envs           map[string]string `thrift:"envs,6" db:"envs" json:"envs"`
-}
-
-func NewCreateAppOptions() *CreateAppOptions {
-	return &CreateAppOptions{}
-}
-
-func (p *CreateAppOptions) GetPartitionCount() int32 {
-	return p.PartitionCount
-}
-
-func (p *CreateAppOptions) GetReplicaCount() int32 {
-	return p.ReplicaCount
-}
-
-func (p *CreateAppOptions) GetSuccessIfExist() bool {
-	return p.SuccessIfExist
-}
-
-func (p *CreateAppOptions) GetAppType() string {
-	return p.AppType
-}
-
-func (p *CreateAppOptions) GetIsStateful() bool {
-	return p.IsStateful
-}
-
-func (p *CreateAppOptions) GetEnvs() map[string]string {
-	return p.Envs
-}
-func (p *CreateAppOptions) 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 fieldTypeId == thrift.I32 {
-				if err := p.ReadField1(iprot); err != nil {
-					return err
-				}
-			} else {
-				if err := iprot.Skip(fieldTypeId); err != nil {
-					return err
-				}
-			}
-		case 2:
-			if fieldTypeId == thrift.I32 {
-				if err := p.ReadField2(iprot); err != nil {
-					return err
-				}
-			} else {
-				if err := iprot.Skip(fieldTypeId); err != nil {
-					return err
-				}
-			}
-		case 3:
-			if fieldTypeId == thrift.BOOL {
-				if err := p.ReadField3(iprot); err != nil {
-					return err
-				}
-			} else {
-				if err := iprot.Skip(fieldTypeId); err != nil {
-					return err
-				}
-			}
-		case 4:
-			if fieldTypeId == thrift.STRING {
-				if err := p.ReadField4(iprot); err != nil {
-					return err
-				}
-			} else {
-				if err := iprot.Skip(fieldTypeId); err != nil {
-					return err
-				}
-			}
-		case 5:
-			if fieldTypeId == thrift.BOOL {
-				if err := p.ReadField5(iprot); err != nil {
-					return err
-				}
-			} else {
-				if err := iprot.Skip(fieldTypeId); err != nil {
-					return err
-				}
-			}
-		case 6:
-			if fieldTypeId == thrift.MAP {
-				if err := p.ReadField6(iprot); err != nil {
-					return err
-				}
-			} else {
-				if err := iprot.Skip(fieldTypeId); 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 *CreateAppOptions) ReadField1(iprot thrift.TProtocol) error {
-	if v, err := iprot.ReadI32(); err != nil {
-		return thrift.PrependError("error reading field 1: ", err)
-	} else {
-		p.PartitionCount = v
-	}
-	return nil
-}
-
-func (p *CreateAppOptions) ReadField2(iprot thrift.TProtocol) error {
-	if v, err := iprot.ReadI32(); err != nil {
-		return thrift.PrependError("error reading field 2: ", err)
-	} else {
-		p.ReplicaCount = v
-	}
-	return nil
-}
-
-func (p *CreateAppOptions) ReadField3(iprot thrift.TProtocol) error {
-	if v, err := iprot.ReadBool(); err != nil {
-		return thrift.PrependError("error reading field 3: ", err)
-	} else {
-		p.SuccessIfExist = v
-	}
-	return nil
-}
-
-func (p *CreateAppOptions) ReadField4(iprot thrift.TProtocol) error {
-	if v, err := iprot.ReadString(); err != nil {
-		return thrift.PrependError("error reading field 4: ", err)
-	} else {
-		p.AppType = v
-	}
-	return nil
-}
-
-func (p *CreateAppOptions) ReadField5(iprot thrift.TProtocol) error {
-	if v, err := iprot.ReadBool(); err != nil {
-		return thrift.PrependError("error reading field 5: ", err)
-	} else {
-		p.IsStateful = v
-	}
-	return nil
-}
-
-func (p *CreateAppOptions) ReadField6(iprot thrift.TProtocol) error {
-	_, _, size, err := iprot.ReadMapBegin()
-	if err != nil {
-		return thrift.PrependError("error reading map begin: ", err)
-	}
-	tMap := make(map[string]string, size)
-	p.Envs = tMap
-	for i := 0; i < size; i++ {
-		var _key0 string
-		if v, err := iprot.ReadString(); err != nil {
-			return thrift.PrependError("error reading field 0: ", err)
-		} else {
-			_key0 = v
-		}
-		var _val1 string
-		if v, err := iprot.ReadString(); err != nil {
-			return thrift.PrependError("error reading field 0: ", err)
-		} else {
-			_val1 = v
-		}
-		p.Envs[_key0] = _val1
-	}
-	if err := iprot.ReadMapEnd(); err != nil {
-		return thrift.PrependError("error reading map end: ", err)
-	}
-	return nil
-}
-
-func (p *CreateAppOptions) Write(oprot thrift.TProtocol) error {
-	if err := oprot.WriteStructBegin("create_app_options"); err != nil {
-		return thrift.PrependError(fmt.Sprintf("%T write struct begin error: ", p), err)
-	}
-	if p != nil {
-		if err := p.writeField1(oprot); err != nil {
-			return err
-		}
-		if err := p.writeField2(oprot); err != nil {
-			return err
-		}
-		if err := p.writeField3(oprot); err != nil {
-			return err
-		}
-		if err := p.writeField4(oprot); err != nil {
-			return err
-		}
-		if err := p.writeField5(oprot); err != nil {
-			return err
-		}
-		if err := p.writeField6(oprot); err != nil {
-			return err
-		}
-	}
-	if err := 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 *CreateAppOptions) writeField1(oprot thrift.TProtocol) (err error) {
-	if err := oprot.WriteFieldBegin("partition_count", thrift.I32, 1); err != nil {
-		return thrift.PrependError(fmt.Sprintf("%T write field begin error 1:partition_count: ", p), err)
-	}
-	if err := oprot.WriteI32(int32(p.PartitionCount)); err != nil {
-		return thrift.PrependError(fmt.Sprintf("%T.partition_count (1) field write error: ", p), err)
-	}
-	if err := oprot.WriteFieldEnd(); err != nil {
-		return thrift.PrependError(fmt.Sprintf("%T write field end error 1:partition_count: ", p), err)
-	}
-	return err
-}
-
-func (p *CreateAppOptions) writeField2(oprot thrift.TProtocol) (err error) {
-	if err := oprot.WriteFieldBegin("replica_count", thrift.I32, 2); err != nil {
-		return thrift.PrependError(fmt.Sprintf("%T write field begin error 2:replica_count: ", p), err)
-	}
-	if err := oprot.WriteI32(int32(p.ReplicaCount)); err != nil {
-		return thrift.PrependError(fmt.Sprintf("%T.replica_count (2) field write error: ", p), err)
-	}
-	if err := oprot.WriteFieldEnd(); err != nil {
-		return thrift.PrependError(fmt.Sprintf("%T write field end error 2:replica_count: ", p), err)
-	}
-	return err
-}
-
-func (p *CreateAppOptions) writeField3(oprot thrift.TProtocol) (err error) {
-	if err := oprot.WriteFieldBegin("success_if_exist", thrift.BOOL, 3); err != nil {
-		return thrift.PrependError(fmt.Sprintf("%T write field begin error 3:success_if_exist: ", p), err)
-	}
-	if err := oprot.WriteBool(bool(p.SuccessIfExist)); err != nil {
-		return thrift.PrependError(fmt.Sprintf("%T.success_if_exist (3) field write error: ", p), err)
-	}
-	if err := oprot.WriteFieldEnd(); err != nil {
-		return thrift.PrependError(fmt.Sprintf("%T write field end error 3:success_if_exist: ", p), err)
-	}
-	return err
-}
-
-func (p *CreateAppOptions) writeField4(oprot thrift.TProtocol) (err error) {
-	if err := oprot.WriteFieldBegin("app_type", thrift.STRING, 4); err != nil {
-		return thrift.PrependError(fmt.Sprintf("%T write field begin error 4:app_type: ", p), err)
-	}
-	if err := oprot.WriteString(string(p.AppType)); err != nil {
-		return thrift.PrependError(fmt.Sprintf("%T.app_type (4) field write error: ", p), err)
-	}
-	if err := oprot.WriteFieldEnd(); err != nil {
-		return thrift.PrependError(fmt.Sprintf("%T write field end error 4:app_type: ", p), err)
-	}
-	return err
-}
-
-func (p *CreateAppOptions) writeField5(oprot thrift.TProtocol) (err error) {
-	if err := oprot.WriteFieldBegin("is_stateful", thrift.BOOL, 5); err != nil {
-		return thrift.PrependError(fmt.Sprintf("%T write field begin error 5:is_stateful: ", p), err)
-	}
-	if err := oprot.WriteBool(bool(p.IsStateful)); err != nil {
-		return thrift.PrependError(fmt.Sprintf("%T.is_stateful (5) field write error: ", p), err)
-	}
-	if err := oprot.WriteFieldEnd(); err != nil {
-		return thrift.PrependError(fmt.Sprintf("%T write field end error 5:is_stateful: ", p), err)
-	}
-	return err
-}
-
-func (p *CreateAppOptions) writeField6(oprot thrift.TProtocol) (err error) {
-	if err := oprot.WriteFieldBegin("envs", thrift.MAP, 6); err != nil {
-		return thrift.PrependError(fmt.Sprintf("%T write field begin error 6:envs: ", p), err)
-	}
-	if err := oprot.WriteMapBegin(thrift.STRING, thrift.STRING, len(p.Envs)); err != nil {
-		return thrift.PrependError("error writing map begin: ", err)
-	}
-	for k, v := range p.Envs {
-		if err := oprot.WriteString(string(k)); err != nil {
-			return thrift.PrependError(fmt.Sprintf("%T. (0) field write error: ", p), err)
-		}
-		if err := oprot.WriteString(string(v)); err != nil {
-			return thrift.PrependError(fmt.Sprintf("%T. (0) field write error: ", p), err)
-		}
-	}
-	if err := oprot.WriteMapEnd(); err != nil {
-		return thrift.PrependError("error writing map end: ", err)
-	}
-	if err := oprot.WriteFieldEnd(); err != nil {
-		return thrift.PrependError(fmt.Sprintf("%T write field end error 6:envs: ", p), err)
-	}
-	return err
-}
-
-func (p *CreateAppOptions) String() string {
-	if p == nil {
-		return "<nil>"
-	}
-	return fmt.Sprintf("CreateAppOptions(%+v)", *p)
-}
-
-// Attributes:
-//  - AppName
-//  - Options
-type CreateAppRequest struct {
-	AppName string            `thrift:"app_name,1" db:"app_name" json:"app_name"`
-	Options *CreateAppOptions `thrift:"options,2" db:"options" json:"options"`
-}
-
-func NewCreateAppRequest() *CreateAppRequest {
-	return &CreateAppRequest{}
-}
-
-func (p *CreateAppRequest) GetAppName() string {
-	return p.AppName
-}
-
-var CreateAppRequest_Options_DEFAULT *CreateAppOptions
-
-func (p *CreateAppRequest) GetOptions() *CreateAppOptions {
-	if !p.IsSetOptions() {
-		return CreateAppRequest_Options_DEFAULT
-	}
-	return p.Options
-}
-func (p *CreateAppRequest) IsSetOptions() bool {
-	return p.Options != nil
-}
-
-func (p *CreateAppRequest) 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 fieldTypeId == thrift.STRING {
-				if err := p.ReadField1(iprot); err != nil {
-					return err
-				}
-			} else {
-				if err := iprot.Skip(fieldTypeId); err != nil {
-					return err
-				}
-			}
-		case 2:
-			if fieldTypeId == thrift.STRUCT {
-				if err := p.ReadField2(iprot); err != nil {
-					return err
-				}
-			} else {
-				if err := iprot.Skip(fieldTypeId); 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 *CreateAppRequest) ReadField1(iprot thrift.TProtocol) error {
-	if v, err := iprot.ReadString(); err != nil {
-		return thrift.PrependError("error reading field 1: ", err)
-	} else {
-		p.AppName = v
-	}
-	return nil
-}
-
-func (p *CreateAppRequest) ReadField2(iprot thrift.TProtocol) error {
-	p.Options = &CreateAppOptions{}
-	if err := p.Options.Read(iprot); err != nil {
-		return thrift.PrependError(fmt.Sprintf("%T error reading struct: ", p.Options), err)
-	}
-	return nil
-}
-
-func (p *CreateAppRequest) Write(oprot thrift.TProtocol) error {
-	if err := oprot.WriteStructBegin("create_app_request"); 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 *CreateAppRequest) writeField1(oprot thrift.TProtocol) (err error) {
-	if err := oprot.WriteFieldBegin("app_name", thrift.STRING, 1); err != nil {
-		return thrift.PrependError(fmt.Sprintf("%T write field begin error 1:app_name: ", p), err)
-	}
-	if err := oprot.WriteString(string(p.AppName)); err != nil {
-		return thrift.PrependError(fmt.Sprintf("%T.app_name (1) field write error: ", p), err)
-	}
-	if err := oprot.WriteFieldEnd(); err != nil {
-		return thrift.PrependError(fmt.Sprintf("%T write field end error 1:app_name: ", p), err)
-	}
-	return err
-}
-
-func (p *CreateAppRequest) writeField2(oprot thrift.TProtocol) (err error) {
-	if err := oprot.WriteFieldBegin("options", thrift.STRUCT, 2); err != nil {
-		return thrift.PrependError(fmt.Sprintf("%T write field begin error 2:options: ", p), err)
-	}
-	if err := p.Options.Write(oprot); err != nil {
-		return thrift.PrependError(fmt.Sprintf("%T error writing struct: ", p.Options), err)
-	}
-	if err := oprot.WriteFieldEnd(); err != nil {
-		return thrift.PrependError(fmt.Sprintf("%T write field end error 2:options: ", p), err)
-	}
-	return err
-}
-
-func (p *CreateAppRequest) String() string {
-	if p == nil {
-		return "<nil>"
-	}
-	return fmt.Sprintf("CreateAppRequest(%+v)", *p)
-}
-
-// Attributes:
-//  - Err
-//  - Appid
-type CreateAppResponse struct {
-	Err   *base.ErrorCode `thrift:"err,1" db:"err" json:"err"`
-	Appid int32           `thrift:"appid,2" db:"appid" json:"appid"`
-}
-
-func NewCreateAppResponse() *CreateAppResponse {
-	return &CreateAppResponse{}
-}
-
-var CreateAppResponse_Err_DEFAULT *base.ErrorCode
-
-func (p *CreateAppResponse) GetErr() *base.ErrorCode {
-	if !p.IsSetErr() {
-		return CreateAppResponse_Err_DEFAULT
-	}
-	return p.Err
-}
-
-func (p *CreateAppResponse) GetAppid() int32 {
-	return p.Appid
-}
-func (p *CreateAppResponse) IsSetErr() bool {
-	return p.Err != nil
-}
-
-func (p *CreateAppResponse) 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 fieldTypeId == thrift.STRUCT {
-				if err := p.ReadField1(iprot); err != nil {
-					return err
-				}
-			} else {
-				if err := iprot.Skip(fieldTypeId); err != nil {
-					return err
-				}
-			}
-		case 2:
-			if fieldTypeId == thrift.I32 {
-				if err := p.ReadField2(iprot); err != nil {
-					return err
-				}
-			} else {
-				if err := iprot.Skip(fieldTypeId); 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 *CreateAppResponse) ReadField1(iprot thrift.TProtocol) error {
-	p.Err = &base.ErrorCode{}
-	if err := p.Err.Read(iprot); err != nil {
-		return thrift.PrependError(fmt.Sprintf("%T error reading struct: ", p.Err), err)
-	}
-	return nil
-}
-
-func (p *CreateAppResponse) ReadField2(iprot thrift.TProtocol) error {
-	if v, err := iprot.ReadI32(); err != nil {
-		return thrift.PrependError("error reading field 2: ", err)
-	} else {
-		p.Appid = v
-	}
-	return nil
-}
-
-func (p *CreateAppResponse) Write(oprot thrift.TProtocol) error {
-	if err := oprot.WriteStructBegin("create_app_response"); 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 *CreateAppResponse) writeField1(oprot thrift.TProtocol) (err error) {
-	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 *CreateAppResponse) writeField2(oprot thrift.TProtocol) (err error) {
-	if err := oprot.WriteFieldBegin("appid", thrift.I32, 2); err != nil {
-		return thrift.PrependError(fmt.Sprintf("%T write field begin error 2:appid: ", p), err)
-	}
-	if err := oprot.WriteI32(int32(p.Appid)); err != nil {
-		return thrift.PrependError(fmt.Sprintf("%T.appid (2) field write error: ", p), err)
-	}
-	if err := oprot.WriteFieldEnd(); err != nil {
-		return thrift.PrependError(fmt.Sprintf("%T write field end error 2:appid: ", p), err)
-	}
-	return err
-}
-
-func (p *CreateAppResponse) String() string {
-	if p == nil {
-		return "<nil>"
-	}
-	return fmt.Sprintf("CreateAppResponse(%+v)", *p)
-}
-
-// Attributes:
-//  - SuccessIfNotExist
-//  - ReserveSeconds
-type DropAppOptions struct {
-	SuccessIfNotExist bool   `thrift:"success_if_not_exist,1" db:"success_if_not_exist" json:"success_if_not_exist"`
-	ReserveSeconds    *int64 `thrift:"reserve_seconds,2" db:"reserve_seconds" json:"reserve_seconds,omitempty"`
-}
-
-func NewDropAppOptions() *DropAppOptions {
-	return &DropAppOptions{}
-}
-
-func (p *DropAppOptions) GetSuccessIfNotExist() bool {
-	return p.SuccessIfNotExist
-}
-
-var DropAppOptions_ReserveSeconds_DEFAULT int64
-
-func (p *DropAppOptions) GetReserveSeconds() int64 {
-	if !p.IsSetReserveSeconds() {
-		return DropAppOptions_ReserveSeconds_DEFAULT
-	}
-	return *p.ReserveSeconds
-}
-func (p *DropAppOptions) IsSetReserveSeconds() bool {
-	return p.ReserveSeconds != nil
-}
-
-func (p *DropAppOptions) 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 fieldTypeId == thrift.BOOL {
-				if err := p.ReadField1(iprot); err != nil {
-					return err
-				}
-			} else {
-				if err := iprot.Skip(fieldTypeId); err != nil {
-					return err
-				}
-			}
-		case 2:
-			if fieldTypeId == thrift.I64 {
-				if err := p.ReadField2(iprot); err != nil {
-					return err
-				}
-			} else {
-				if err := iprot.Skip(fieldTypeId); 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 *DropAppOptions) ReadField1(iprot thrift.TProtocol) error {
-	if v, err := iprot.ReadBool(); err != nil {
-		return thrift.PrependError("error reading field 1: ", err)
-	} else {
-		p.SuccessIfNotExist = v
-	}
-	return nil
-}
-
-func (p *DropAppOptions) ReadField2(iprot thrift.TProtocol) error {
-	if v, err := iprot.ReadI64(); err != nil {
-		return thrift.PrependError("error reading field 2: ", err)
-	} else {
-		p.ReserveSeconds = &v
-	}
-	return nil
-}
-
-func (p *DropAppOptions) Write(oprot thrift.TProtocol) error {
-	if err := oprot.WriteStructBegin("drop_app_options"); 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 *DropAppOptions) writeField1(oprot thrift.TProtocol) (err error) {
-	if err := oprot.WriteFieldBegin("success_if_not_exist", thrift.BOOL, 1); err != nil {
-		return thrift.PrependError(fmt.Sprintf("%T write field begin error 1:success_if_not_exist: ", p), err)
-	}
-	if err := oprot.WriteBool(bool(p.SuccessIfNotExist)); err != nil {
-		return thrift.PrependError(fmt.Sprintf("%T.success_if_not_exist (1) field write error: ", p), err)
-	}
-	if err := oprot.WriteFieldEnd(); err != nil {
-		return thrift.PrependError(fmt.Sprintf("%T write field end error 1:success_if_not_exist: ", p), err)
-	}
-	return err
-}
-
-func (p *DropAppOptions) writeField2(oprot thrift.TProtocol) (err error) {
-	if p.IsSetReserveSeconds() {
-		if err := oprot.WriteFieldBegin("reserve_seconds", thrift.I64, 2); err != nil {
-			return thrift.PrependError(fmt.Sprintf("%T write field begin error 2:reserve_seconds: ", p), err)
-		}
-		if err := oprot.WriteI64(int64(*p.ReserveSeconds)); err != nil {
-			return thrift.PrependError(fmt.Sprintf("%T.reserve_seconds (2) field write error: ", p), err)
-		}
-		if err := oprot.WriteFieldEnd(); err != nil {
-			return thrift.PrependError(fmt.Sprintf("%T write field end error 2:reserve_seconds: ", p), err)
-		}
-	}
-	return err
-}
-
-func (p *DropAppOptions) String() string {
-	if p == nil {
-		return "<nil>"
-	}
-	return fmt.Sprintf("DropAppOptions(%+v)", *p)
-}
-
-// Attributes:
-//  - AppName
-//  - Options
-type DropAppRequest struct {
-	AppName string          `thrift:"app_name,1" db:"app_name" json:"app_name"`
-	Options *DropAppOptions `thrift:"options,2" db:"options" json:"options"`
-}
-
-func NewDropAppRequest() *DropAppRequest {
-	return &DropAppRequest{}
-}
-
-func (p *DropAppRequest) GetAppName() string {
-	return p.AppName
-}
-
-var DropAppRequest_Options_DEFAULT *DropAppOptions
-
-func (p *DropAppRequest) GetOptions() *DropAppOptions {
-	if !p.IsSetOptions() {
-		return DropAppRequest_Options_DEFAULT
-	}
-	return p.Options
-}
-func (p *DropAppRequest) IsSetOptions() bool {
-	return p.Options != nil
-}
-
-func (p *DropAppRequest) 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 fieldTypeId == thrift.STRING {
-				if err := p.ReadField1(iprot); err != nil {
-					return err
-				}
-			} else {
-				if err := iprot.Skip(fieldTypeId); err != nil {
-					return err
-				}
-			}
-		case 2:
-			if fieldTypeId == thrift.STRUCT {
-				if err := p.ReadField2(iprot); err != nil {
-					return err
-				}
-			} else {
-				if err := iprot.Skip(fieldTypeId); 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 *DropAppRequest) ReadField1(iprot thrift.TProtocol) error {
-	if v, err := iprot.ReadString(); err != nil {
-		return thrift.PrependError("error reading field 1: ", err)
-	} else {
-		p.AppName = v
-	}
-	return nil
-}
-
-func (p *DropAppRequest) ReadField2(iprot thrift.TProtocol) error {
-	p.Options = &DropAppOptions{}
-	if err := p.Options.Read(iprot); err != nil {
-		return thrift.PrependError(fmt.Sprintf("%T error reading struct: ", p.Options), err)
-	}
-	return nil
-}
-
-func (p *DropAppRequest) Write(oprot thrift.TProtocol) error {
-	if err := oprot.WriteStructBegin("drop_app_request"); 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 *DropAppRequest) writeField1(oprot thrift.TProtocol) (err error) {
-	if err := oprot.WriteFieldBegin("app_name", thrift.STRING, 1); err != nil {
-		return thrift.PrependError(fmt.Sprintf("%T write field begin error 1:app_name: ", p), err)
-	}
-	if err := oprot.WriteString(string(p.AppName)); err != nil {
-		return thrift.PrependError(fmt.Sprintf("%T.app_name (1) field write error: ", p), err)
-	}
-	if err := oprot.WriteFieldEnd(); err != nil {
-		return thrift.PrependError(fmt.Sprintf("%T write field end error 1:app_name: ", p), err)
-	}
-	return err
-}
-
-func (p *DropAppRequest) writeField2(oprot thrift.TProtocol) (err error) {
-	if err := oprot.WriteFieldBegin("options", thrift.STRUCT, 2); err != nil {
-		return thrift.PrependError(fmt.Sprintf("%T write field begin error 2:options: ", p), err)
-	}
-	if err := p.Options.Write(oprot); err != nil {
-		return thrift.PrependError(fmt.Sprintf("%T error writing struct: ", p.Options), err)
-	}
-	if err := oprot.WriteFieldEnd(); err != nil {
-		return thrift.PrependError(fmt.Sprintf("%T write field end error 2:options: ", p), err)
-	}
-	return err
-}
-
-func (p *DropAppRequest) String() string {
-	if p == nil {
-		return "<nil>"
-	}
-	return fmt.Sprintf("DropAppRequest(%+v)", *p)
-}
-
-// Attributes:
-//  - Err
-type DropAppResponse struct {
-	Err *base.ErrorCode `thrift:"err,1" db:"err" json:"err"`
-}
-
-func NewDropAppResponse() *DropAppResponse {
-	return &DropAppResponse{}
-}
-
-var DropAppResponse_Err_DEFAULT *base.ErrorCode
-
-func (p *DropAppResponse) GetErr() *base.ErrorCode {
-	if !p.IsSetErr() {
-		return DropAppResponse_Err_DEFAULT
-	}
-	return p.Err
-}
-func (p *DropAppResponse) IsSetErr() bool {
-	return p.Err != nil
-}
-
-func (p *DropAppResponse) 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 fieldTypeId == thrift.STRUCT {
-				if err := p.ReadField1(iprot); err != nil {
-					return err
-				}
-			} else {
-				if err := iprot.Skip(fieldTypeId); 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 *DropAppResponse) ReadField1(iprot thrift.TProtocol) error {
-	p.Err = &base.ErrorCode{}
-	if err := p.Err.Read(iprot); err != nil {
-		return thrift.PrependError(fmt.Sprintf("%T error reading struct: ", p.Err), err)
-	}
-	return nil
-}
-
-func (p *DropAppResponse) Write(oprot thrift.TProtocol) error {
-	if err := oprot.WriteStructBegin("drop_app_response"); 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 *DropAppResponse) writeField1(oprot thrift.TProtocol) (err error) {
-	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 *DropAppResponse) String() string {
-	if p == nil {
-		return "<nil>"
-	}
-	return fmt.Sprintf("DropAppResponse(%+v)", *p)
-}
-
-// Attributes:
-//  - Status
-//  - AppType
-//  - AppName
-//  - AppID
-//  - PartitionCount
-//  - Envs
-//  - IsStateful
-//  - MaxReplicaCount
-//  - ExpireSecond
-//  - CreateSecond
-//  - DropSecond
-//  - Duplicating
-//  - InitPartitionCount
-//  - IsBulkLoading
-type AppInfo struct {
-	Status             AppStatus         `thrift:"status,1" db:"status" json:"status"`
-	AppType            string            `thrift:"app_type,2" db:"app_type" json:"app_type"`
-	AppName            string            `thrift:"app_name,3" db:"app_name" json:"app_name"`
-	AppID              int32             `thrift:"app_id,4" db:"app_id" json:"app_id"`
-	PartitionCount     int32             `thrift:"partition_count,5" db:"partition_count" json:"partition_count"`
-	Envs               map[string]string `thrift:"envs,6" db:"envs" json:"envs"`
-	IsStateful         bool              `thrift:"is_stateful,7" db:"is_stateful" json:"is_stateful"`
-	MaxReplicaCount    int32             `thrift:"max_replica_count,8" db:"max_replica_count" json:"max_replica_count"`
-	ExpireSecond       int64             `thrift:"expire_second,9" db:"expire_second" json:"expire_second"`
-	CreateSecond       int64             `thrift:"create_second,10" db:"create_second" json:"create_second"`
-	DropSecond         int64             `thrift:"drop_second,11" db:"drop_second" json:"drop_second"`
-	Duplicating        *bool             `thrift:"duplicating,12" db:"duplicating" json:"duplicating,omitempty"`
-	InitPartitionCount int32             `thrift:"init_partition_count,13" db:"init_partition_count" json:"init_partition_count"`
-	IsBulkLoading      bool              `thrift:"is_bulk_loading,14" db:"is_bulk_loading" json:"is_bulk_loading"`
-}
-
-func NewAppInfo() *AppInfo {
-	return &AppInfo{
-		Status: 0,
-
-		InitPartitionCount: -1,
-	}
-}
-
-func (p *AppInfo) GetStatus() AppStatus {
-	return p.Status
-}
-
-func (p *AppInfo) GetAppType() string {
-	return p.AppType
-}
-
-func (p *AppInfo) GetAppName() string {
-	return p.AppName
-}
-
-func (p *AppInfo) GetAppID() int32 {
-	return p.AppID
-}
-
-func (p *AppInfo) GetPartitionCount() int32 {
-	return p.PartitionCount
-}
-
-func (p *AppInfo) GetEnvs() map[string]string {
-	return p.Envs
-}
-
-func (p *AppInfo) GetIsStateful() bool {
-	return p.IsStateful
-}
-
-func (p *AppInfo) GetMaxReplicaCount() int32 {
-	return p.MaxReplicaCount
-}
-
-func (p *AppInfo) GetExpireSecond() int64 {
-	return p.ExpireSecond
-}
-
-func (p *AppInfo) GetCreateSecond() int64 {
-	return p.CreateSecond
-}
-
-func (p *AppInfo) GetDropSecond() int64 {
-	return p.DropSecond
-}
-
-var AppInfo_Duplicating_DEFAULT bool
-
-func (p *AppInfo) GetDuplicating() bool {
-	if !p.IsSetDuplicating() {
-		return AppInfo_Duplicating_DEFAULT
-	}
-	return *p.Duplicating
-}
-
-func (p *AppInfo) GetInitPartitionCount() int32 {
-	return p.InitPartitionCount
-}
-
-var AppInfo_IsBulkLoading_DEFAULT bool = false
-
-func (p *AppInfo) GetIsBulkLoading() bool {
-	return p.IsBulkLoading
-}
-func (p *AppInfo) IsSetDuplicating() bool {
-	return p.Duplicating != nil
-}
-
-func (p *AppInfo) IsSetIsBulkLoading() bool {
-	return p.IsBulkLoading != AppInfo_IsBulkLoading_DEFAULT
-}
-
-func (p *AppInfo) 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 fieldTypeId == thrift.I32 {
-				if err := p.ReadField1(iprot); err != nil {
-					return err
-				}
-			} else {
-				if err := iprot.Skip(fieldTypeId); err != nil {
-					return err
-				}
-			}
-		case 2:
-			if fieldTypeId == thrift.STRING {
-				if err := p.ReadField2(iprot); err != nil {
-					return err
-				}
-			} else {
-				if err := iprot.Skip(fieldTypeId); err != nil {
-					return err
-				}
-			}
-		case 3:
-			if fieldTypeId == thrift.STRING {
-				if err := p.ReadField3(iprot); err != nil {
-					return err
-				}
-			} else {
-				if err := iprot.Skip(fieldTypeId); err != nil {
-					return err
-				}
-			}
-		case 4:
-			if fieldTypeId == thrift.I32 {
-				if err := p.ReadField4(iprot); err != nil {
-					return err
-				}
-			} else {
-				if err := iprot.Skip(fieldTypeId); err != nil {
-					return err
-				}
-			}
-		case 5:
-			if fieldTypeId == thrift.I32 {
-				if err := p.ReadField5(iprot); err != nil {
-					return err
-				}
-			} else {
-				if err := iprot.Skip(fieldTypeId); err != nil {
-					return err
-				}
-			}
-		case 6:
-			if fieldTypeId == thrift.MAP {
-				if err := p.ReadField6(iprot); err != nil {
-					return err
-				}
-			} else {
-				if err := iprot.Skip(fieldTypeId); err != nil {
-					return err
-				}
-			}
-		case 7:
-			if fieldTypeId == thrift.BOOL {
-				if err := p.ReadField7(iprot); err != nil {
-					return err
-				}
-			} else {
-				if err := iprot.Skip(fieldTypeId); err != nil {
-					return err
-				}
-			}
-		case 8:
-			if fieldTypeId == thrift.I32 {
-				if err := p.ReadField8(iprot); err != nil {
-					return err
-				}
-			} else {
-				if err := iprot.Skip(fieldTypeId); err != nil {
-					return err
-				}
-			}
-		case 9:
-			if fieldTypeId == thrift.I64 {
-				if err := p.ReadField9(iprot); err != nil {
-					return err
-				}
-			} else {
-				if err := iprot.Skip(fieldTypeId); err != nil {
-					return err
-				}
-			}
-		case 10:
-			if fieldTypeId == thrift.I64 {
-				if err := p.ReadField10(iprot); err != nil {
-					return err
-				}
-			} else {
-				if err := iprot.Skip(fieldTypeId); err != nil {
-					return err
-				}
-			}
-		case 11:
-			if fieldTypeId == thrift.I64 {
-				if err := p.ReadField11(iprot); err != nil {
-					return err
-				}
-			} else {
-				if err := iprot.Skip(fieldTypeId); err != nil {
-					return err
-				}
-			}
-		case 12:
-			if fieldTypeId == thrift.BOOL {
-				if err := p.ReadField12(iprot); err != nil {
-					return err
-				}
-			} else {
-				if err := iprot.Skip(fieldTypeId); err != nil {
-					return err
-				}
-			}
-		case 13:
-			if fieldTypeId == thrift.I32 {
-				if err := p.ReadField13(iprot); err != nil {
-					return err
-				}
-			} else {
-				if err := iprot.Skip(fieldTypeId); err != nil {
-					return err
-				}
-			}
-		case 14:
-			if fieldTypeId == thrift.BOOL {
-				if err := p.ReadField14(iprot); err != nil {
-					return err
-				}
-			} else {
-				if err := iprot.Skip(fieldTypeId); 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 *AppInfo) ReadField1(iprot thrift.TProtocol) error {
-	if v, err := iprot.ReadI32(); err != nil {
-		return thrift.PrependError("error reading field 1: ", err)
-	} else {
-		temp := AppStatus(v)
-		p.Status = temp
-	}
-	return nil
-}
-
-func (p *AppInfo) ReadField2(iprot thrift.TProtocol) error {
-	if v, err := iprot.ReadString(); err != nil {
-		return thrift.PrependError("error reading field 2: ", err)
-	} else {
-		p.AppType = v
-	}
-	return nil
-}
-
-func (p *AppInfo) ReadField3(iprot thrift.TProtocol) error {
-	if v, err := iprot.ReadString(); err != nil {
-		return thrift.PrependError("error reading field 3: ", err)
-	} else {
-		p.AppName = v
-	}
-	return nil
-}
-
-func (p *AppInfo) ReadField4(iprot thrift.TProtocol) error {
-	if v, err := iprot.ReadI32(); err != nil {
-		return thrift.PrependError("error reading field 4: ", err)
-	} else {
-		p.AppID = v
-	}
-	return nil
-}
-
-func (p *AppInfo) ReadField5(iprot thrift.TProtocol) error {
-	if v, err := iprot.ReadI32(); err != nil {
-		return thrift.PrependError("error reading field 5: ", err)
-	} else {
-		p.PartitionCount = v
-	}
-	return nil
-}
-
-func (p *AppInfo) ReadField6(iprot thrift.TProtocol) error {
-	_, _, size, err := iprot.ReadMapBegin()
-	if err != nil {
-		return thrift.PrependError("error reading map begin: ", err)
-	}
-	tMap := make(map[string]string, size)
-	p.Envs = tMap
-	for i := 0; i < size; i++ {
-		var _key2 string
-		if v, err := iprot.ReadString(); err != nil {
-			return thrift.PrependError("error reading field 0: ", err)
-		} else {
-			_key2 = v
-		}
-		var _val3 string
-		if v, err := iprot.ReadString(); err != nil {
-			return thrift.PrependError("error reading field 0: ", err)
-		} else {
-			_val3 = v
-		}
-		p.Envs[_key2] = _val3
-	}
-	if err := iprot.ReadMapEnd(); err != nil {
-		return thrift.PrependError("error reading map end: ", err)
-	}
-	return nil
-}
-
-func (p *AppInfo) ReadField7(iprot thrift.TProtocol) error {
-	if v, err := iprot.ReadBool(); err != nil {
-		return thrift.PrependError("error reading field 7: ", err)
-	} else {
-		p.IsStateful = v
-	}
-	return nil
-}
-
-func (p *AppInfo) ReadField8(iprot thrift.TProtocol) error {
-	if v, err := iprot.ReadI32(); err != nil {
-		return thrift.PrependError("error reading field 8: ", err)
-	} else {
-		p.MaxReplicaCount = v
-	}
-	return nil
-}
-
-func (p *AppInfo) ReadField9(iprot thrift.TProtocol) error {
-	if v, err := iprot.ReadI64(); err != nil {
-		return thrift.PrependError("error reading field 9: ", err)
-	} else {
-		p.ExpireSecond = v
-	}
-	return nil
-}
-
-func (p *AppInfo) ReadField10(iprot thrift.TProtocol) error {
-	if v, err := iprot.ReadI64(); err != nil {
-		return thrift.PrependError("error reading field 10: ", err)
-	} else {
-		p.CreateSecond = v
-	}
-	return nil
-}
-
-func (p *AppInfo) ReadField11(iprot thrift.TProtocol) error {
-	if v, err := iprot.ReadI64(); err != nil {
-		return thrift.PrependError("error reading field 11: ", err)
-	} else {
-		p.DropSecond = v
-	}
-	return nil
-}
-
-func (p *AppInfo) ReadField12(iprot thrift.TProtocol) error {
-	if v, err := iprot.ReadBool(); err != nil {
-		return thrift.PrependError("error reading field 12: ", err)
-	} else {
-		p.Duplicating = &v
-	}
-	return nil
-}
-
-func (p *AppInfo) ReadField13(iprot thrift.TProtocol) error {
-	if v, err := iprot.ReadI32(); err != nil {
-		return thrift.PrependError("error reading field 13: ", err)
-	} else {
-		p.InitPartitionCount = v
-	}
-	return nil
-}
-
-func (p *AppInfo) ReadField14(iprot thrift.TProtocol) error {
-	if v, err := iprot.ReadBool(); err != nil {
-		return thrift.PrependError("error reading field 14: ", err)
-	} else {
-		p.IsBulkLoading = v
-	}
-	return nil
-}
-
-func (p *AppInfo) Write(oprot thrift.TProtocol) error {
-	if err := oprot.WriteStructBegin("app_info"); err != nil {
-		return thrift.PrependError(fmt.Sprintf("%T write struct begin error: ", p), err)
-	}
-	if p != nil {
-		if err := p.writeField1(oprot); err != nil {
-			return err
-		}
-		if err := p.writeField2(oprot); err != nil {
-			return err
-		}
-		if err := p.writeField3(oprot); err != nil {
-			return err
-		}
-		if err := p.writeField4(oprot); err != nil {
-			return err
-		}
-		if err := p.writeField5(oprot); err != nil {
-			return err
-		}
-		if err := p.writeField6(oprot); err != nil {
-			return err
-		}
-		if err := p.writeField7(oprot); err != nil {
-			return err
-		}
-		if err := p.writeField8(oprot); err != nil {
-			return err
-		}
-		if err := p.writeField9(oprot); err != nil {
-			return err
-		}
-		if err := p.writeField10(oprot); err != nil {
-			return err
-		}
-		if err := p.writeField11(oprot); err != nil {
-			return err
-		}
-		if err := p.writeField12(oprot); err != nil {
-			return err
-		}
-		if err := p.writeField13(oprot); err != nil {
-			return err
-		}
-		if err := p.writeField14(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 *AppInfo) writeField1(oprot thrift.TProtocol) (err error) {
-	if err := oprot.WriteFieldBegin("status", thrift.I32, 1); err != nil {
-		return thrift.PrependError(fmt.Sprintf("%T write field begin error 1:status: ", p), err)
-	}
-	if err := oprot.WriteI32(int32(p.Status)); err != nil {
-		return thrift.PrependError(fmt.Sprintf("%T.status (1) field write error: ", p), err)
-	}
-	if err := oprot.WriteFieldEnd(); err != nil {
-		return thrift.PrependError(fmt.Sprintf("%T write field end error 1:status: ", p), err)
-	}
-	return err
-}
-
-func (p *AppInfo) writeField2(oprot thrift.TProtocol) (err error) {
-	if err := oprot.WriteFieldBegin("app_type", thrift.STRING, 2); err != nil {
-		return thrift.PrependError(fmt.Sprintf("%T write field begin error 2:app_type: ", p), err)
-	}
-	if err := oprot.WriteString(string(p.AppType)); err != nil {
-		return thrift.PrependError(fmt.Sprintf("%T.app_type (2) field write error: ", p), err)
-	}
-	if err := oprot.WriteFieldEnd(); err != nil {
-		return thrift.PrependError(fmt.Sprintf("%T write field end error 2:app_type: ", p), err)
-	}
-	return err
-}
-
-func (p *AppInfo) writeField3(oprot thrift.TProtocol) (err error) {
-	if err := oprot.WriteFieldBegin("app_name", thrift.STRING, 3); err != nil {
-		return thrift.PrependError(fmt.Sprintf("%T write field begin error 3:app_name: ", p), err)
-	}
-	if err := oprot.WriteString(string(p.AppName)); err != nil {
-		return thrift.PrependError(fmt.Sprintf("%T.app_name (3) field write error: ", p), err)
-	}
-	if err := oprot.WriteFieldEnd(); err != nil {
-		return thrift.PrependError(fmt.Sprintf("%T write field end error 3:app_name: ", p), err)
-	}
-	return err
-}
-
-func (p *AppInfo) writeField4(oprot thrift.TProtocol) (err error) {
-	if err := oprot.WriteFieldBegin("app_id", thrift.I32, 4); err != nil {
-		return thrift.PrependError(fmt.Sprintf("%T write field begin error 4:app_id: ", p), err)
-	}
-	if err := oprot.WriteI32(int32(p.AppID)); err != nil {
-		return thrift.PrependError(fmt.Sprintf("%T.app_id (4) field write error: ", p), err)
-	}
-	if err := oprot.WriteFieldEnd(); err != nil {
-		return thrift.PrependError(fmt.Sprintf("%T write field end error 4:app_id: ", p), err)
-	}
-	return err
-}
-
-func (p *AppInfo) writeField5(oprot thrift.TProtocol) (err error) {
-	if err := oprot.WriteFieldBegin("partition_count", thrift.I32, 5); err != nil {
-		return thrift.PrependError(fmt.Sprintf("%T write field begin error 5:partition_count: ", p), err)
-	}
-	if err := oprot.WriteI32(int32(p.PartitionCount)); err != nil {
-		return thrift.PrependError(fmt.Sprintf("%T.partition_count (5) field write error: ", p), err)
-	}
-	if err := oprot.WriteFieldEnd(); err != nil {
-		return thrift.PrependError(fmt.Sprintf("%T write field end error 5:partition_count: ", p), err)
-	}
-	return err
-}
-
-func (p *AppInfo) writeField6(oprot thrift.TProtocol) (err error) {
-	if err := oprot.WriteFieldBegin("envs", thrift.MAP, 6); err != nil {
-		return thrift.PrependError(fmt.Sprintf("%T write field begin error 6:envs: ", p), err)
-	}
-	if err := oprot.WriteMapBegin(thrift.STRING, thrift.STRING, len(p.Envs)); err != nil {
-		return thrift.PrependError("error writing map begin: ", err)
-	}
-	for k, v := range p.Envs {
-		if err := oprot.WriteString(string(k)); err != nil {
-			return thrift.PrependError(fmt.Sprintf("%T. (0) field write error: ", p), err)
-		}
-		if err := oprot.WriteString(string(v)); err != nil {
-			return thrift.PrependError(fmt.Sprintf("%T. (0) field write error: ", p), err)
-		}
-	}
-	if err := oprot.WriteMapEnd(); err != nil {
-		return thrift.PrependError("error writing map end: ", err)
-	}
-	if err := oprot.WriteFieldEnd(); err != nil {
-		return thrift.PrependError(fmt.Sprintf("%T write field end error 6:envs: ", p), err)
-	}
-	return err
-}
-
-func (p *AppInfo) writeField7(oprot thrift.TProtocol) (err error) {
-	if err := oprot.WriteFieldBegin("is_stateful", thrift.BOOL, 7); err != nil {
-		return thrift.PrependError(fmt.Sprintf("%T write field begin error 7:is_stateful: ", p), err)
-	}
-	if err := oprot.WriteBool(bool(p.IsStateful)); err != nil {
-		return thrift.PrependError(fmt.Sprintf("%T.is_stateful (7) field write error: ", p), err)
-	}
-	if err := oprot.WriteFieldEnd(); err != nil {
-		return thrift.PrependError(fmt.Sprintf("%T write field end error 7:is_stateful: ", p), err)
-	}
-	return err
-}
-
-func (p *AppInfo) writeField8(oprot thrift.TProtocol) (err error) {
-	if err := oprot.WriteFieldBegin("max_replica_count", thrift.I32, 8); err != nil {
-		return thrift.PrependError(fmt.Sprintf("%T write field begin error 8:max_replica_count: ", p), err)
-	}
-	if err := oprot.WriteI32(int32(p.MaxReplicaCount)); err != nil {
-		return thrift.PrependError(fmt.Sprintf("%T.max_replica_count (8) field write error: ", p), err)
-	}
-	if err := oprot.WriteFieldEnd(); err != nil {
-		return thrift.PrependError(fmt.Sprintf("%T write field end error 8:max_replica_count: ", p), err)
-	}
-	return err
-}
-
-func (p *AppInfo) writeField9(oprot thrift.TProtocol) (err error) {
-	if err := oprot.WriteFieldBegin("expire_second", thrift.I64, 9); err != nil {
-		return thrift.PrependError(fmt.Sprintf("%T write field begin error 9:expire_second: ", p), err)
-	}
-	if err := oprot.WriteI64(int64(p.ExpireSecond)); err != nil {
-		return thrift.PrependError(fmt.Sprintf("%T.expire_second (9) field write error: ", p), err)
-	}
-	if err := oprot.WriteFieldEnd(); err != nil {
-		return thrift.PrependError(fmt.Sprintf("%T write field end error 9:expire_second: ", p), err)
-	}
-	return err
-}
-
-func (p *AppInfo) writeField10(oprot thrift.TProtocol) (err error) {
-	if err := oprot.WriteFieldBegin("create_second", thrift.I64, 10); err != nil {
-		return thrift.PrependError(fmt.Sprintf("%T write field begin error 10:create_second: ", p), err)
-	}
-	if err := oprot.WriteI64(int64(p.CreateSecond)); err != nil {
-		return thrift.PrependError(fmt.Sprintf("%T.create_second (10) field write error: ", p), err)
-	}
-	if err := oprot.WriteFieldEnd(); err != nil {
-		return thrift.PrependError(fmt.Sprintf("%T write field end error 10:create_second: ", p), err)
-	}
-	return err
-}
-
-func (p *AppInfo) writeField11(oprot thrift.TProtocol) (err error) {
-	if err := oprot.WriteFieldBegin("drop_second", thrift.I64, 11); err != nil {
-		return thrift.PrependError(fmt.Sprintf("%T write field begin error 11:drop_second: ", p), err)
-	}
-	if err := oprot.WriteI64(int64(p.DropSecond)); err != nil {
-		return thrift.PrependError(fmt.Sprintf("%T.drop_second (11) field write error: ", p), err)
-	}
-	if err := oprot.WriteFieldEnd(); err != nil {
-		return thrift.PrependError(fmt.Sprintf("%T write field end error 11:drop_second: ", p), err)
-	}
-	return err
-}
-
-func (p *AppInfo) writeField12(oprot thrift.TProtocol) (err error) {
-	if p.IsSetDuplicating() {
-		if err := oprot.WriteFieldBegin("duplicating", thrift.BOOL, 12); err != nil {
-			return thrift.PrependError(fmt.Sprintf("%T write field begin error 12:duplicating: ", p), err)
-		}
-		if err := oprot.WriteBool(bool(*p.Duplicating)); err != nil {
-			return thrift.PrependError(fmt.Sprintf("%T.duplicating (12) field write error: ", p), err)
-		}
-		if err := oprot.WriteFieldEnd(); err != nil {
-			return thrift.PrependError(fmt.Sprintf("%T write field end error 12:duplicating: ", p), err)
-		}
-	}
-	return err
-}
-
-func (p *AppInfo) writeField13(oprot thrift.TProtocol) (err error) {
-	if err := oprot.WriteFieldBegin("init_partition_count", thrift.I32, 13); err != nil {
-		return thrift.PrependError(fmt.Sprintf("%T write field begin error 13:init_partition_count: ", p), err)
-	}
-	if err := oprot.WriteI32(int32(p.InitPartitionCount)); err != nil {
-		return thrift.PrependError(fmt.Sprintf("%T.init_partition_count (13) field write error: ", p), err)
-	}
-	if err := oprot.WriteFieldEnd(); err != nil {
-		return thrift.PrependError(fmt.Sprintf("%T write field end error 13:init_partition_count: ", p), err)
-	}
-	return err
-}
-
-func (p *AppInfo) writeField14(oprot thrift.TProtocol) (err error) {
-	if p.IsSetIsBulkLoading() {
-		if err := oprot.WriteFieldBegin("is_bulk_loading", thrift.BOOL, 14); err != nil {
-			return thrift.PrependError(fmt.Sprintf("%T write field begin error 14:is_bulk_loading: ", p), err)
-		}
-		if err := oprot.WriteBool(bool(p.IsBulkLoading)); err != nil {
-			return thrift.PrependError(fmt.Sprintf("%T.is_bulk_loading (14) field write error: ", p), err)
-		}
-		if err := oprot.WriteFieldEnd(); err != nil {
-			return thrift.PrependError(fmt.Sprintf("%T write field end error 14:is_bulk_loading: ", p), err)
-		}
-	}
-	return err
-}
-
-func (p *AppInfo) String() string {
-	if p == nil {
-		return "<nil>"
-	}
-	return fmt.Sprintf("AppInfo(%+v)", *p)
-}
-
-// Attributes:
-//  - AppID
-//  - NewAppName_
-type RecallAppRequest struct {
-	AppID       int32  `thrift:"app_id,1" db:"app_id" json:"app_id"`
-	NewAppName_ string `thrift:"new_app_name,2" db:"new_app_name" json:"new_app_name"`
-}
-
-func NewRecallAppRequest() *RecallAppRequest {
-	return &RecallAppRequest{}
-}
-
-func (p *RecallAppRequest) GetAppID() int32 {
-	return p.AppID
-}
-
-func (p *RecallAppRequest) GetNewAppName_() string {
-	return p.NewAppName_
-}
-func (p *RecallAppRequest) 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 fieldTypeId == thrift.I32 {
-				if err := p.ReadField1(iprot); err != nil {
-					return err
-				}
-			} else {
-				if err := iprot.Skip(fieldTypeId); err != nil {
-					return err
-				}
-			}
-		case 2:
-			if fieldTypeId == thrift.STRING {
-				if err := p.ReadField2(iprot); err != nil {
-					return err
-				}
-			} else {
-				if err := iprot.Skip(fieldTypeId); 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 *RecallAppRequest) ReadField1(iprot thrift.TProtocol) error {
-	if v, err := iprot.ReadI32(); err != nil {
-		return thrift.PrependError("error reading field 1: ", err)
-	} else {
-		p.AppID = v
-	}
-	return nil
-}
-
-func (p *RecallAppRequest) ReadField2(iprot thrift.TProtocol) error {
-	if v, err := iprot.ReadString(); err != nil {
-		return thrift.PrependError("error reading field 2: ", err)
-	} else {
-		p.NewAppName_ = v
-	}
-	return nil
-}
-
-func (p *RecallAppRequest) Write(oprot thrift.TProtocol) error {
-	if err := oprot.WriteStructBegin("recall_app_request"); 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 *RecallAppRequest) writeField1(oprot thrift.TProtocol) (err error) {
-	if err := oprot.WriteFieldBegin("app_id", thrift.I32, 1); err != nil {
-		return thrift.PrependError(fmt.Sprintf("%T write field begin error 1:app_id: ", p), err)
-	}
-	if err := oprot.WriteI32(int32(p.AppID)); err != nil {
-		return thrift.PrependError(fmt.Sprintf("%T.app_id (1) field write error: ", p), err)
-	}
-	if err := oprot.WriteFieldEnd(); err != nil {
-		return thrift.PrependError(fmt.Sprintf("%T write field end error 1:app_id: ", p), err)
-	}
-	return err
-}
-
-func (p *RecallAppRequest) writeField2(oprot thrift.TProtocol) (err error) {
-	if err := oprot.WriteFieldBegin("new_app_name", thrift.STRING, 2); err != nil {
-		return thrift.PrependError(fmt.Sprintf("%T write field begin error 2:new_app_name: ", p), err)
-	}
-	if err := oprot.WriteString(string(p.NewAppName_)); err != nil {
-		return thrift.PrependError(fmt.Sprintf("%T.new_app_name (2) field write error: ", p), err)
-	}
-	if err := oprot.WriteFieldEnd(); err != nil {
-		return thrift.PrependError(fmt.Sprintf("%T write field end error 2:new_app_name: ", p), err)
-	}
-	return err
-}
-
-func (p *RecallAppRequest) String() string {
-	if p == nil {
-		return "<nil>"
-	}
-	return fmt.Sprintf("RecallAppRequest(%+v)", *p)
-}
-
-// Attributes:
-//  - Err
-//  - Info
-type RecallAppResponse struct {
-	Err  *base.ErrorCode `thrift:"err,1" db:"err" json:"err"`
-	Info *AppInfo        `thrift:"info,2" db:"info" json:"info"`
-}
-
-func NewRecallAppResponse() *RecallAppResponse {
-	return &RecallAppResponse{}
-}
-
-var RecallAppResponse_Err_DEFAULT *base.ErrorCode
-
-func (p *RecallAppResponse) GetErr() *base.ErrorCode {
-	if !p.IsSetErr() {
-		return RecallAppResponse_Err_DEFAULT
-	}
-	return p.Err
-}
-
-var RecallAppResponse_Info_DEFAULT *AppInfo
-
-func (p *RecallAppResponse) GetInfo() *AppInfo {
-	if !p.IsSetInfo() {
-		return RecallAppResponse_Info_DEFAULT
-	}
-	return p.Info
-}
-func (p *RecallAppResponse) IsSetErr() bool {
-	return p.Err != nil
-}
-
-func (p *RecallAppResponse) IsSetInfo() bool {
-	return p.Info != nil
-}
-
-func (p *RecallAppResponse) 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 fieldTypeId == thrift.STRUCT {
-				if err := p.ReadField1(iprot); err != nil {
-					return err
-				}
-			} else {
-				if err := iprot.Skip(fieldTypeId); err != nil {
-					return err
-				}
-			}
-		case 2:
-			if fieldTypeId == thrift.STRUCT {
-				if err := p.ReadField2(iprot); err != nil {
-					return err
-				}
-			} else {
-				if err := iprot.Skip(fieldTypeId); 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 *RecallAppResponse) ReadField1(iprot thrift.TProtocol) error {
-	p.Err = &base.ErrorCode{}
-	if err := p.Err.Read(iprot); err != nil {
-		return thrift.PrependError(fmt.Sprintf("%T error reading struct: ", p.Err), err)
-	}
-	return nil
-}
-
-func (p *RecallAppResponse) ReadField2(iprot thrift.TProtocol) error {
-	p.Info = &AppInfo{
-		Status: 0,
-
-		InitPartitionCount: -1,
-	}
-	if err := p.Info.Read(iprot); err != nil {
-		return thrift.PrependError(fmt.Sprintf("%T error reading struct: ", p.Info), err)
-	}
-	return nil
-}
-
-func (p *RecallAppResponse) Write(oprot thrift.TProtocol) error {
-	if err := oprot.WriteStructBegin("recall_app_response"); 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 *RecallAppResponse) writeField1(oprot thrift.TProtocol) (err error) {
-	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 *RecallAppResponse) writeField2(oprot thrift.TProtocol) (err error) {
-	if err := oprot.WriteFieldBegin("info", thrift.STRUCT, 2); err != nil {
-		return thrift.PrependError(fmt.Sprintf("%T write field begin error 2:info: ", p), err)
-	}
-	if err := p.Info.Write(oprot); err != nil {
-		return thrift.PrependError(fmt.Sprintf("%T error writing struct: ", p.Info), err)
-	}
-	if err := oprot.WriteFieldEnd(); err != nil {
-		return thrift.PrependError(fmt.Sprintf("%T write field end error 2:info: ", p), err)
-	}
-	return err
-}
-
-func (p *RecallAppResponse) String() string {
-	if p == nil {
-		return "<nil>"
-	}
-	return fmt.Sprintf("RecallAppResponse(%+v)", *p)
-}
-
-// Attributes:
-//  - Status
-type ListAppsRequest struct {
-	Status AppStatus `thrift:"status,1" db:"status" json:"status"`
-}
-
-func NewListAppsRequest() *ListAppsRequest {
-	return &ListAppsRequest{
-		Status: 0,
-	}
-}
-
-func (p *ListAppsRequest) GetStatus() AppStatus {
-	return p.Status
-}
-func (p *ListAppsRequest) 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 fieldTypeId == thrift.I32 {
-				if err := p.ReadField1(iprot); err != nil {
-					return err
-				}
-			} else {
-				if err := iprot.Skip(fieldTypeId); 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 *ListAppsRequest) ReadField1(iprot thrift.TProtocol) error {
-	if v, err := iprot.ReadI32(); err != nil {
-		return thrift.PrependError("error reading field 1: ", err)
-	} else {
-		temp := AppStatus(v)
-		p.Status = temp
-	}
-	return nil
-}
-
-func (p *ListAppsRequest) Write(oprot thrift.TProtocol) error {
-	if err := oprot.WriteStructBegin("list_apps_request"); 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 *ListAppsRequest) writeField1(oprot thrift.TProtocol) (err error) {
-	if err := oprot.WriteFieldBegin("status", thrift.I32, 1); err != nil {
-		return thrift.PrependError(fmt.Sprintf("%T write field begin error 1:status: ", p), err)
-	}
-	if err := oprot.WriteI32(int32(p.Status)); err != nil {
-		return thrift.PrependError(fmt.Sprintf("%T.status (1) field write error: ", p), err)
-	}
-	if err := oprot.WriteFieldEnd(); err != nil {
-		return thrift.PrependError(fmt.Sprintf("%T write field end error 1:status: ", p), err)
-	}
-	return err
-}
-
-func (p *ListAppsRequest) String() string {
-	if p == nil {
-		return "<nil>"
-	}
-	return fmt.Sprintf("ListAppsRequest(%+v)", *p)
-}
-
-// Attributes:
-//  - Err
-//  - Infos
-type ListAppsResponse struct {
-	Err   *base.ErrorCode `thrift:"err,1" db:"err" json:"err"`
-	Infos []*AppInfo      `thrift:"infos,2" db:"infos" json:"infos"`
-}
-
-func NewListAppsResponse() *ListAppsResponse {
-	return &ListAppsResponse{}
-}
-
-var ListAppsResponse_Err_DEFAULT *base.ErrorCode
-
-func (p *ListAppsResponse) GetErr() *base.ErrorCode {
-	if !p.IsSetErr() {
-		return ListAppsResponse_Err_DEFAULT
-	}
-	return p.Err
-}
-
-func (p *ListAppsResponse) GetInfos() []*AppInfo {
-	return p.Infos
-}
-func (p *ListAppsResponse) IsSetErr() bool {
-	return p.Err != nil
-}
-
-func (p *ListAppsResponse) 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 fieldTypeId == thrift.STRUCT {
-				if err := p.ReadField1(iprot); err != nil {
-					return err
-				}
-			} else {
-				if err := iprot.Skip(fieldTypeId); err != nil {
-					return err
-				}
-			}
-		case 2:
-			if fieldTypeId == thrift.LIST {
-				if err := p.ReadField2(iprot); err != nil {
-					return err
-				}
-			} else {
-				if err := iprot.Skip(fieldTypeId); 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 *ListAppsResponse) ReadField1(iprot thrift.TProtocol) error {
-	p.Err = &base.ErrorCode{}
-	if err := p.Err.Read(iprot); err != nil {
-		return thrift.PrependError(fmt.Sprintf("%T error reading struct: ", p.Err), err)
-	}
-	return nil
-}
-
-func (p *ListAppsResponse) ReadField2(iprot thrift.TProtocol) error {
-	_, size, err := iprot.ReadListBegin()
-	if err != nil {
-		return thrift.PrependError("error reading list begin: ", err)
-	}
-	tSlice := make([]*AppInfo, 0, size)
-	p.Infos = tSlice
-	for i := 0; i < size; i++ {
-		_elem4 := &AppInfo{
-			Status: 0,
-
-			InitPartitionCount: -1,
-		}
-		if err := _elem4.Read(iprot); err != nil {
-			return thrift.PrependError(fmt.Sprintf("%T error reading struct: ", _elem4), err)
-		}
-		p.Infos = append(p.Infos, _elem4)
-	}
-	if err := iprot.ReadListEnd(); err != nil {
-		return thrift.PrependError("error reading list end: ", err)
-	}
-	return nil
-}
-
-func (p *ListAppsResponse) Write(oprot thrift.TProtocol) error {
-	if err := oprot.WriteStructBegin("list_apps_response"); 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 *ListAppsResponse) writeField1(oprot thrift.TProtocol) (err error) {
-	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 *ListAppsResponse) writeField2(oprot thrift.TProtocol) (err error) {
-	if err := oprot.WriteFieldBegin("infos", thrift.LIST, 2); err != nil {
-		return thrift.PrependError(fmt.Sprintf("%T write field begin error 2:infos: ", p), err)
-	}
-	if err := oprot.WriteListBegin(thrift.STRUCT, len(p.Infos)); err != nil {
-		return thrift.PrependError("error writing list begin: ", err)
-	}
-	for _, v := range p.Infos {
-		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:infos: ", p), err)
-	}
-	return err
-}
-
-func (p *ListAppsResponse) String() string {
-	if p == nil {
-		return "<nil>"
-	}
-	return fmt.Sprintf("ListAppsResponse(%+v)", *p)
-}
-
-// Attributes:
-//  - MetaServer
-type QueryAppInfoRequest struct {
-	MetaServer *base.RPCAddress `thrift:"meta_server,1" db:"meta_server" json:"meta_server"`
-}
-
-func NewQueryAppInfoRequest() *QueryAppInfoRequest {
-	return &QueryAppInfoRequest{}
-}
-
-var QueryAppInfoRequest_MetaServer_DEFAULT *base.RPCAddress
-
-func (p *QueryAppInfoRequest) GetMetaServer() *base.RPCAddress {
-	if !p.IsSetMetaServer() {
-		return QueryAppInfoRequest_MetaServer_DEFAULT
-	}
-	return p.MetaServer
-}
-func (p *QueryAppInfoRequest) IsSetMetaServer() bool {
-	return p.MetaServer != nil
-}
-
-func (p *QueryAppInfoRequest) 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 fieldTypeId == thrift.STRUCT {
-				if err := p.ReadField1(iprot); err != nil {
-					return err
-				}
-			} else {
-				if err := iprot.Skip(fieldTypeId); 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 *QueryAppInfoRequest) ReadField1(iprot thrift.TProtocol) error {
-	p.MetaServer = &base.RPCAddress{}
-	if err := p.MetaServer.Read(iprot); err != nil {
-		return thrift.PrependError(fmt.Sprintf("%T error reading struct: ", p.MetaServer), err)
-	}
-	return nil
-}
-
-func (p *QueryAppInfoRequest) Write(oprot thrift.TProtocol) error {
-	if err := oprot.WriteStructBegin("query_app_info_request"); 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 *QueryAppInfoRequest) writeField1(oprot thrift.TProtocol) (err error) {
-	if err := oprot.WriteFieldBegin("meta_server", thrift.STRUCT, 1); err != nil {
-		return thrift.PrependError(fmt.Sprintf("%T write field begin error 1:meta_server: ", p), err)
-	}
-	if err := p.MetaServer.Write(oprot); err != nil {
-		return thrift.PrependError(fmt.Sprintf("%T error writing struct: ", p.MetaServer), err)
-	}
-	if err := oprot.WriteFieldEnd(); err != nil {
-		return thrift.PrependError(fmt.Sprintf("%T write field end error 1:meta_server: ", p), err)
-	}
-	return err
-}
-
-func (p *QueryAppInfoRequest) String() string {
-	if p == nil {
-		return "<nil>"
-	}
-	return fmt.Sprintf("QueryAppInfoRequest(%+v)", *p)
-}
-
-// Attributes:
-//  - Err
-//  - Apps
-type QueryAppInfoResponse struct {
-	Err  *base.ErrorCode `thrift:"err,1" db:"err" json:"err"`
-	Apps []*AppInfo      `thrift:"apps,2" db:"apps" json:"apps"`
-}
-
-func NewQueryAppInfoResponse() *QueryAppInfoResponse {
-	return &QueryAppInfoResponse{}
-}
-
-var QueryAppInfoResponse_Err_DEFAULT *base.ErrorCode
-
-func (p *QueryAppInfoResponse) GetErr() *base.ErrorCode {
-	if !p.IsSetErr() {
-		return QueryAppInfoResponse_Err_DEFAULT
-	}
-	return p.Err
-}
-
-func (p *QueryAppInfoResponse) GetApps() []*AppInfo {
-	return p.Apps
-}
-func (p *QueryAppInfoResponse) IsSetErr() bool {
-	return p.Err != nil
-}
-
-func (p *QueryAppInfoResponse) 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 fieldTypeId == thrift.STRUCT {
-				if err := p.ReadField1(iprot); err != nil {
-					return err
-				}
-			} else {
-				if err := iprot.Skip(fieldTypeId); err != nil {
-					return err
-				}
-			}
-		case 2:
-			if fieldTypeId == thrift.LIST {
-				if err := p.ReadField2(iprot); err != nil {
-					return err
-				}
-			} else {
-				if err := iprot.Skip(fieldTypeId); 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 *QueryAppInfoResponse) ReadField1(iprot thrift.TProtocol) error {
-	p.Err = &base.ErrorCode{}
-	if err := p.Err.Read(iprot); err != nil {
-		return thrift.PrependError(fmt.Sprintf("%T error reading struct: ", p.Err), err)
-	}
-	return nil
-}
-
-func (p *QueryAppInfoResponse) ReadField2(iprot thrift.TProtocol) error {
-	_, size, err := iprot.ReadListBegin()
-	if err != nil {
-		return thrift.PrependError("error reading list begin: ", err)
-	}
-	tSlice := make([]*AppInfo, 0, size)
-	p.Apps = tSlice
-	for i := 0; i < size; i++ {
-		_elem5 := &AppInfo{
-			Status: 0,
-
-			InitPartitionCount: -1,
-		}
-		if err := _elem5.Read(iprot); err != nil {
-			return thrift.PrependError(fmt.Sprintf("%T error reading struct: ", _elem5), err)
-		}
-		p.Apps = append(p.Apps, _elem5)
-	}
-	if err := iprot.ReadListEnd(); err != nil {
-		return thrift.PrependError("error reading list end: ", err)
-	}
-	return nil
-}
-
-func (p *QueryAppInfoResponse) Write(oprot thrift.TProtocol) error {
-	if err := oprot.WriteStructBegin("query_app_info_response"); 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 *QueryAppInfoResponse) writeField1(oprot thrift.TProtocol) (err error) {
-	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 *QueryAppInfoResponse) writeField2(oprot thrift.TProtocol) (err error) {
-	if err := oprot.WriteFieldBegin("apps", thrift.LIST, 2); err != nil {
-		return thrift.PrependError(fmt.Sprintf("%T write field begin error 2:apps: ", p), err)
-	}
-	if err := oprot.WriteListBegin(thrift.STRUCT, len(p.Apps)); err != nil {
-		return thrift.PrependError("error writing list begin: ", err)
-	}
-	for _, v := range p.Apps {
-		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:apps: ", p), err)
-	}
-	return err
-}
-
-func (p *QueryAppInfoResponse) String() string {
-	if p == nil {
-		return "<nil>"
-	}
-	return fmt.Sprintf("QueryAppInfoResponse(%+v)", *p)
-}
-
-// Attributes:
-//  - AppName
-//  - Op
-//  - Keys
-//  - Values
-//  - ClearPrefix
-type UpdateAppEnvRequest struct {
-	AppName     string          `thrift:"app_name,1" db:"app_name" json:"app_name"`
-	Op          AppEnvOperation `thrift:"op,2" db:"op" json:"op"`
-	Keys        []string        `thrift:"keys,3" db:"keys" json:"keys,omitempty"`
-	Values      []string        `thrift:"values,4" db:"values" json:"values,omitempty"`
-	ClearPrefix *string         `thrift:"clear_prefix,5" db:"clear_prefix" json:"clear_prefix,omitempty"`
-}
-
-func NewUpdateAppEnvRequest() *UpdateAppEnvRequest {
-	return &UpdateAppEnvRequest{
-		Op: 0,
-	}
-}
-
-func (p *UpdateAppEnvRequest) GetAppName() string {
-	return p.AppName
-}
-
-func (p *UpdateAppEnvRequest) GetOp() AppEnvOperation {
-	return p.Op
-}
-
-var UpdateAppEnvRequest_Keys_DEFAULT []string
-
-func (p *UpdateAppEnvRequest) GetKeys() []string {
-	return p.Keys
-}
-
-var UpdateAppEnvRequest_Values_DEFAULT []string
-
-func (p *UpdateAppEnvRequest) GetValues() []string {
-	return p.Values
-}
-
-var UpdateAppEnvRequest_ClearPrefix_DEFAULT string
-
-func (p *UpdateAppEnvRequest) GetClearPrefix() string {
-	if !p.IsSetClearPrefix() {
-		return UpdateAppEnvRequest_ClearPrefix_DEFAULT
-	}
-	return *p.ClearPrefix
-}
-func (p *UpdateAppEnvRequest) IsSetKeys() bool {
-	return p.Keys != nil
-}
-
-func (p *UpdateAppEnvRequest) IsSetValues() bool {
-	return p.Values != nil
-}
-
-func (p *UpdateAppEnvRequest) IsSetClearPrefix() bool {
-	return p.ClearPrefix != nil
-}
-
-func (p *UpdateAppEnvRequest) 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 fieldTypeId == thrift.STRING {
-				if err := p.ReadField1(iprot); err != nil {
-					return err
-				}
-			} else {
-				if err := iprot.Skip(fieldTypeId); err != nil {
-					return err
-				}
-			}
-		case 2:
-			if fieldTypeId == thrift.I32 {
-				if err := p.ReadField2(iprot); err != nil {
-					return err
-				}
-			} else {
-				if err := iprot.Skip(fieldTypeId); err != nil {
-					return err
-				}
-			}
-		case 3:
-			if fieldTypeId == thrift.LIST {
-				if err := p.ReadField3(iprot); err != nil {
-					return err
-				}
-			} else {
-				if err := iprot.Skip(fieldTypeId); err != nil {
-					return err
-				}
-			}
-		case 4:
-			if fieldTypeId == thrift.LIST {
-				if err := p.ReadField4(iprot); err != nil {
-					return err
-				}
-			} else {
-				if err := iprot.Skip(fieldTypeId); err != nil {
-					return err
-				}
-			}
-		case 5:
-			if fieldTypeId == thrift.STRING {
-				if err := p.ReadField5(iprot); err != nil {
-					return err
-				}
-			} else {
-				if err := iprot.Skip(fieldTypeId); 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 *UpdateAppEnvRequest) ReadField1(iprot thrift.TProtocol) error {
-	if v, err := iprot.ReadString(); err != nil {
-		return thrift.PrependError("error reading field 1: ", err)
-	} else {
-		p.AppName = v
-	}
-	return nil
-}
-
-func (p *UpdateAppEnvRequest) ReadField2(iprot thrift.TProtocol) error {
-	if v, err := iprot.ReadI32(); err != nil {
-		return thrift.PrependError("error reading field 2: ", err)
-	} else {
-		temp := AppEnvOperation(v)
-		p.Op = temp
-	}
-	return nil
-}
-
-func (p *UpdateAppEnvRequest) ReadField3(iprot thrift.TProtocol) error {
-	_, size, err := iprot.ReadListBegin()
-	if err != nil {
-		return thrift.PrependError("error reading list begin: ", err)
-	}
-	tSlice := make([]string, 0, size)
-	p.Keys = tSlice
-	for i := 0; i < size; i++ {
-		var _elem6 string
-		if v, err := iprot.ReadString(); err != nil {
-			return thrift.PrependError("error reading field 0: ", err)
-		} else {
-			_elem6 = v
-		}
-		p.Keys = append(p.Keys, _elem6)
-	}
-	if err := iprot.ReadListEnd(); err != nil {
-		return thrift.PrependError("error reading list end: ", err)
-	}
-	return nil
-}
-
-func (p *UpdateAppEnvRequest) ReadField4(iprot thrift.TProtocol) error {
-	_, size, err := iprot.ReadListBegin()
-	if err != nil {
-		return thrift.PrependError("error reading list begin: ", err)
-	}
-	tSlice := make([]string, 0, size)
-	p.Values = tSlice
-	for i := 0; i < size; i++ {
-		var _elem7 string
-		if v, err := iprot.ReadString(); err != nil {
-			return thrift.PrependError("error reading field 0: ", err)
-		} else {
-			_elem7 = v
-		}
-		p.Values = append(p.Values, _elem7)
-	}
-	if err := iprot.ReadListEnd(); err != nil {
-		return thrift.PrependError("error reading list end: ", err)
-	}
-	return nil
-}
-
-func (p *UpdateAppEnvRequest) ReadField5(iprot thrift.TProtocol) error {
-	if v, err := iprot.ReadString(); err != nil {
-		return thrift.PrependError("error reading field 5: ", err)
-	} else {
-		p.ClearPrefix = &v
-	}
-	return nil
-}
-
-func (p *UpdateAppEnvRequest) Write(oprot thrift.TProtocol) error {
-	if err := oprot.WriteStructBegin("update_app_env_request"); 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 *UpdateAppEnvRequest) writeField1(oprot thrift.TProtocol) (err error) {
-	if err := oprot.WriteFieldBegin("app_name", thrift.STRING, 1); err != nil {
-		return thrift.PrependError(fmt.Sprintf("%T write field begin error 1:app_name: ", p), err)
-	}
-	if err := oprot.WriteString(string(p.AppName)); err != nil {
-		return thrift.PrependError(fmt.Sprintf("%T.app_name (1) field write error: ", p), err)
-	}
-	if err := oprot.WriteFieldEnd(); err != nil {
-		return thrift.PrependError(fmt.Sprintf("%T write field end error 1:app_name: ", p), err)
-	}
-	return err
-}
-
-func (p *UpdateAppEnvRequest) writeField2(oprot thrift.TProtocol) (err error) {
-	if err := oprot.WriteFieldBegin("op", thrift.I32, 2); err != nil {
-		return thrift.PrependError(fmt.Sprintf("%T write field begin error 2:op: ", p), err)
-	}
-	if err := oprot.WriteI32(int32(p.Op)); err != nil {
-		return thrift.PrependError(fmt.Sprintf("%T.op (2) field write error: ", p), err)
-	}
-	if err := oprot.WriteFieldEnd(); err != nil {
-		return thrift.PrependError(fmt.Sprintf("%T write field end error 2:op: ", p), err)
-	}
-	return err
-}
-
-func (p *UpdateAppEnvRequest) writeField3(oprot thrift.TProtocol) (err error) {
-	if p.IsSetKeys() {
-		if err := oprot.WriteFieldBegin("keys", thrift.LIST, 3); err != nil {
-			return thrift.PrependError(fmt.Sprintf("%T write field begin error 3:keys: ", p), err)
-		}
-		if err := oprot.WriteListBegin(thrift.STRING, len(p.Keys)); err != nil {
-			return thrift.PrependError("error writing list begin: ", err)
-		}
-		for _, v := range p.Keys {
-			if err := oprot.WriteString(string(v)); err != nil {
-				return thrift.PrependError(fmt.Sprintf("%T. (0) field write error: ", p), err)
-			}
-		}
-		if err := oprot.WriteListEnd(); err != nil {
-			return thrift.PrependError("error writing list end: ", err)
-		}
-		if err := oprot.WriteFieldEnd(); err != nil {
-			return thrift.PrependError(fmt.Sprintf("%T write field end error 3:keys: ", p), err)
-		}
-	}
-	return err
-}
-
-func (p *UpdateAppEnvRequest) writeField4(oprot thrift.TProtocol) (err error) {
-	if p.IsSetValues() {
-		if err := oprot.WriteFieldBegin("values", thrift.LIST, 4); err != nil {
-			return thrift.PrependError(fmt.Sprintf("%T write field begin error 4:values: ", p), err)
-		}
-		if err := oprot.WriteListBegin(thrift.STRING, len(p.Values)); err != nil {
-			return thrift.PrependError("error writing list begin: ", err)
-		}
-		for _, v := range p.Values {
-			if err := oprot.WriteString(string(v)); err != nil {
-				return thrift.PrependError(fmt.Sprintf("%T. (0) field write error: ", p), err)
-			}
-		}
-		if err := oprot.WriteListEnd(); err != nil {
-			return thrift.PrependError("error writing list end: ", err)
-		}
-		if err := oprot.WriteFieldEnd(); err != nil {
-			return thrift.PrependError(fmt.Sprintf("%T write field end error 4:values: ", p), err)
-		}
-	}
-	return err
-}
-
-func (p *UpdateAppEnvRequest) writeField5(oprot thrift.TProtocol) (err error) {
-	if p.IsSetClearPrefix() {
-		if err := oprot.WriteFieldBegin("clear_prefix", thrift.STRING, 5); err != nil {
-			return thrift.PrependError(fmt.Sprintf("%T write field begin error 5:clear_prefix: ", p), err)
-		}
-		if err := oprot.WriteString(string(*p.ClearPrefix)); err != nil {
-			return thrift.PrependError(fmt.Sprintf("%T.clear_prefix (5) field write error: ", p), err)
-		}
-		if err := oprot.WriteFieldEnd(); err != nil {
-			return thrift.PrependError(fmt.Sprintf("%T write field end error 5:clear_prefix: ", p), err)
-		}
-	}
-	return err
-}
-
-func (p *UpdateAppEnvRequest) String() string {
-	if p == nil {
-		return "<nil>"
-	}
-	return fmt.Sprintf("UpdateAppEnvRequest(%+v)", *p)
-}
-
-// Attributes:
-//  - Err
-//  - HintMessage
-type UpdateAppEnvResponse struct {
-	Err         *base.ErrorCode `thrift:"err,1" db:"err" json:"err"`
-	HintMessage string          `thrift:"hint_message,2" db:"hint_message" json:"hint_message"`
-}
-
-func NewUpdateAppEnvResponse() *UpdateAppEnvResponse {
-	return &UpdateAppEnvResponse{}
-}
-
-var UpdateAppEnvResponse_Err_DEFAULT *base.ErrorCode
-
-func (p *UpdateAppEnvResponse) GetErr() *base.ErrorCode {
-	if !p.IsSetErr() {
-		return UpdateAppEnvResponse_Err_DEFAULT
-	}
-	return p.Err
-}
-
-func (p *UpdateAppEnvResponse) GetHintMessage() string {
-	return p.HintMessage
-}
-func (p *UpdateAppEnvResponse) IsSetErr() bool {
-	return p.Err != nil
-}
-
-func (p *UpdateAppEnvResponse) 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 fieldTypeId == thrift.STRUCT {
-				if err := p.ReadField1(iprot); err != nil {
-					return err
-				}
-			} else {
-				if err := iprot.Skip(fieldTypeId); err != nil {
-					return err
-				}
-			}
-		case 2:
-			if fieldTypeId == thrift.STRING {
-				if err := p.ReadField2(iprot); err != nil {
-					return err
-				}
-			} else {
-				if err := iprot.Skip(fieldTypeId); 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 *UpdateAppEnvResponse) ReadField1(iprot thrift.TProtocol) error {
-	p.Err = &base.ErrorCode{}
-	if err := p.Err.Read(iprot); err != nil {
-		return thrift.PrependError(fmt.Sprintf("%T error reading struct: ", p.Err), err)
-	}
-	return nil
-}
-
-func (p *UpdateAppEnvResponse) ReadField2(iprot thrift.TProtocol) error {
-	if v, err := iprot.ReadString(); err != nil {
-		return thrift.PrependError("error reading field 2: ", err)
-	} else {
-		p.HintMessage = v
-	}
-	return nil
-}
-
-func (p *UpdateAppEnvResponse) Write(oprot thrift.TProtocol) error {
-	if err := oprot.WriteStructBegin("update_app_env_response"); 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 *UpdateAppEnvResponse) writeField1(oprot thrift.TProtocol) (err error) {
-	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 *UpdateAppEnvResponse) writeField2(oprot thrift.TProtocol) (err error) {
-	if err := oprot.WriteFieldBegin("hint_message", thrift.STRING, 2); err != nil {
-		return thrift.PrependError(fmt.Sprintf("%T write field begin error 2:hint_message: ", p), err)
-	}
-	if err := oprot.WriteString(string(p.HintMessage)); err != nil {
-		return thrift.PrependError(fmt.Sprintf("%T.hint_message (2) field write error: ", p), err)
-	}
-	if err := oprot.WriteFieldEnd(); err != nil {
-		return thrift.PrependError(fmt.Sprintf("%T write field end error 2:hint_message: ", p), err)
-	}
-	return err
-}
-
-func (p *UpdateAppEnvResponse) String() string {
-	if p == nil {
-		return "<nil>"
-	}
-	return fmt.Sprintf("UpdateAppEnvResponse(%+v)", *p)
-}
-
-// Attributes:
-//  - AppName
-//  - TriggerTime
-//  - TargetLevel
-//  - Bottommost
-//  - MaxRunningCount
-type StartAppManualCompactRequest struct {
-	AppName         string `thrift:"app_name,1" db:"app_name" json:"app_name"`
-	TriggerTime     *int64 `thrift:"trigger_time,2" db:"trigger_time" json:"trigger_time,omitempty"`
-	TargetLevel     *int32 `thrift:"target_level,3" db:"target_level" json:"target_level,omitempty"`
-	Bottommost      *bool  `thrift:"bottommost,4" db:"bottommost" json:"bottommost,omitempty"`
-	MaxRunningCount *int32 `thrift:"max_running_count,5" db:"max_running_count" json:"max_running_count,omitempty"`
-}
-
-func NewStartAppManualCompactRequest() *StartAppManualCompactRequest {
-	return &StartAppManualCompactRequest{}
-}
-
-func (p *StartAppManualCompactRequest) GetAppName() string {
-	return p.AppName
-}
-
-var StartAppManualCompactRequest_TriggerTime_DEFAULT int64
-
-func (p *StartAppManualCompactRequest) GetTriggerTime() int64 {
-	if !p.IsSetTriggerTime() {
-		return StartAppManualCompactRequest_TriggerTime_DEFAULT
-	}
-	return *p.TriggerTime
-}
-
-var StartAppManualCompactRequest_TargetLevel_DEFAULT int32
-
-func (p *StartAppManualCompactRequest) GetTargetLevel() int32 {
-	if !p.IsSetTargetLevel() {
-		return StartAppManualCompactRequest_TargetLevel_DEFAULT
-	}
-	return *p.TargetLevel
-}
-
-var StartAppManualCompactRequest_Bottommost_DEFAULT bool
-
-func (p *StartAppManualCompactRequest) GetBottommost() bool {
-	if !p.IsSetBottommost() {
-		return StartAppManualCompactRequest_Bottommost_DEFAULT
-	}
-	return *p.Bottommost
-}
-
-var StartAppManualCompactRequest_MaxRunningCount_DEFAULT int32
-
-func (p *StartAppManualCompactRequest) GetMaxRunningCount() int32 {
-	if !p.IsSetMaxRunningCount() {
-		return StartAppManualCompactRequest_MaxRunningCount_DEFAULT
-	}
-	return *p.MaxRunningCount
-}
-func (p *StartAppManualCompactRequest) IsSetTriggerTime() bool {
-	return p.TriggerTime != nil
-}
-
-func (p *StartAppManualCompactRequest) IsSetTargetLevel() bool {
-	return p.TargetLevel != nil
-}
-
-func (p *StartAppManualCompactRequest) IsSetBottommost() bool {
-	return p.Bottommost != nil
-}
-
-func (p *StartAppManualCompactRequest) IsSetMaxRunningCount() bool {
-	return p.MaxRunningCount != nil
-}
-
-func (p *StartAppManualCompactRequest) 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 fieldTypeId == thrift.STRING {
-				if err := p.ReadField1(iprot); err != nil {
-					return err
-				}
-			} else {
-				if err := iprot.Skip(fieldTypeId); err != nil {
-					return err
-				}
-			}
-		case 2:
-			if fieldTypeId == thrift.I64 {
-				if err := p.ReadField2(iprot); err != nil {
-					return err
-				}
-			} else {
-				if err := iprot.Skip(fieldTypeId); err != nil {
-					return err
-				}
-			}
-		case 3:
-			if fieldTypeId == thrift.I32 {
-				if err := p.ReadField3(iprot); err != nil {
-					return err
-				}
-			} else {
-				if err := iprot.Skip(fieldTypeId); err != nil {
-					return err
-				}
-			}
-		case 4:
-			if fieldTypeId == thrift.BOOL {
-				if err := p.ReadField4(iprot); err != nil {
-					return err
-				}
-			} else {
-				if err := iprot.Skip(fieldTypeId); err != nil {
-					return err
-				}
-			}
-		case 5:
-			if fieldTypeId == thrift.I32 {
-				if err := p.ReadField5(iprot); err != nil {
-					return err
-				}
-			} else {
-				if err := iprot.Skip(fieldTypeId); 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 *StartAppManualCompactRequest) ReadField1(iprot thrift.TProtocol) error {
-	if v, err := iprot.ReadString(); err != nil {
-		return thrift.PrependError("error reading field 1: ", err)
-	} else {
-		p.AppName = v
-	}
-	return nil
-}
-
-func (p *StartAppManualCompactRequest) ReadField2(iprot thrift.TProtocol) error {
-	if v, err := iprot.ReadI64(); err != nil {
-		return thrift.PrependError("error reading field 2: ", err)
-	} else {
-		p.TriggerTime = &v
-	}
-	return nil
-}
-
-func (p *StartAppManualCompactRequest) ReadField3(iprot thrift.TProtocol) error {
-	if v, err := iprot.ReadI32(); err != nil {
-		return thrift.PrependError("error reading field 3: ", err)
-	} else {
-		p.TargetLevel = &v
-	}
-	return nil
-}
-
-func (p *StartAppManualCompactRequest) ReadField4(iprot thrift.TProtocol) error {
-	if v, err := iprot.ReadBool(); err != nil {
-		return thrift.PrependError("error reading field 4: ", err)
-	} else {
-		p.Bottommost = &v
-	}
-	return nil
-}
-
-func (p *StartAppManualCompactRequest) ReadField5(iprot thrift.TProtocol) error {
-	if v, err := iprot.ReadI32(); err != nil {
-		return thrift.PrependError("error reading field 5: ", err)
-	} else {
-		p.MaxRunningCount = &v
-	}
-	return nil
-}
-
-func (p *StartAppManualCompactRequest) Write(oprot thrift.TProtocol) error {
-	if err := oprot.WriteStructBegin("start_app_manual_compact_request"); 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 *StartAppManualCompactRequest) writeField1(oprot thrift.TProtocol) (err error) {
-	if err := oprot.WriteFieldBegin("app_name", thrift.STRING, 1); err != nil {
-		return thrift.PrependError(fmt.Sprintf("%T write field begin error 1:app_name: ", p), err)
-	}
-	if err := oprot.WriteString(string(p.AppName)); err != nil {
-		return thrift.PrependError(fmt.Sprintf("%T.app_name (1) field write error: ", p), err)
-	}
-	if err := oprot.WriteFieldEnd(); err != nil {
-		return thrift.PrependError(fmt.Sprintf("%T write field end error 1:app_name: ", p), err)
-	}
-	return err
-}
-
-func (p *StartAppManualCompactRequest) writeField2(oprot thrift.TProtocol) (err error) {
-	if p.IsSetTriggerTime() {
-		if err := oprot.WriteFieldBegin("trigger_time", thrift.I64, 2); err != nil {
-			return thrift.PrependError(fmt.Sprintf("%T write field begin error 2:trigger_time: ", p), err)
-		}
-		if err := oprot.WriteI64(int64(*p.TriggerTime)); err != nil {
-			return thrift.PrependError(fmt.Sprintf("%T.trigger_time (2) field write error: ", p), err)
-		}
-		if err := oprot.WriteFieldEnd(); err != nil {
-			return thrift.PrependError(fmt.Sprintf("%T write field end error 2:trigger_time: ", p), err)
-		}
-	}
-	return err
-}
-
-func (p *StartAppManualCompactRequest) writeField3(oprot thrift.TProtocol) (err error) {
-	if p.IsSetTargetLevel() {
-		if err := oprot.WriteFieldBegin("target_level", thrift.I32, 3); err != nil {
-			return thrift.PrependError(fmt.Sprintf("%T write field begin error 3:target_level: ", p), err)
-		}
-		if err := oprot.WriteI32(int32(*p.TargetLevel)); err != nil {
-			return thrift.PrependError(fmt.Sprintf("%T.target_level (3) field write error: ", p), err)
-		}
-		if err := oprot.WriteFieldEnd(); err != nil {
-			return thrift.PrependError(fmt.Sprintf("%T write field end error 3:target_level: ", p), err)
-		}
-	}
-	return err
-}
-
-func (p *StartAppManualCompactRequest) writeField4(oprot thrift.TProtocol) (err error) {
-	if p.IsSetBottommost() {
-		if err := oprot.WriteFieldBegin("bottommost", thrift.BOOL, 4); err != nil {
-			return thrift.PrependError(fmt.Sprintf("%T write field begin error 4:bottommost: ", p), err)
-		}
-		if err := oprot.WriteBool(bool(*p.Bottommost)); err != nil {
-			return thrift.PrependError(fmt.Sprintf("%T.bottommost (4) field write error: ", p), err)
-		}
-		if err := oprot.WriteFieldEnd(); err != nil {
-			return thrift.PrependError(fmt.Sprintf("%T write field end error 4:bottommost: ", p), err)
-		}
-	}
-	return err
-}
-
-func (p *StartAppManualCompactRequest) writeField5(oprot thrift.TProtocol) (err error) {
-	if p.IsSetMaxRunningCount() {
-		if err := oprot.WriteFieldBegin("max_running_count", thrift.I32, 5); err != nil {
-			return thrift.PrependError(fmt.Sprintf("%T write field begin error 5:max_running_count: ", p), err)
-		}
-		if err := oprot.WriteI32(int32(*p.MaxRunningCount)); err != nil {
-			return thrift.PrependError(fmt.Sprintf("%T.max_running_count (5) field write error: ", p), err)
-		}
-		if err := oprot.WriteFieldEnd(); err != nil {
-			return thrift.PrependError(fmt.Sprintf("%T write field end error 5:max_running_count: ", p), err)
-		}
-	}
-	return err
-}
-
-func (p *StartAppManualCompactRequest) String() string {
-	if p == nil {
-		return "<nil>"
-	}
-	return fmt.Sprintf("StartAppManualCompactRequest(%+v)", *p)
-}
-
-// Attributes:
-//  - Err
-//  - HintMsg
-type StartAppManualCompactResponse struct {
-	Err     *base.ErrorCode `thrift:"err,1" db:"err" json:"err"`
-	HintMsg string          `thrift:"hint_msg,2" db:"hint_msg" json:"hint_msg"`
-}
-
-func NewStartAppManualCompactResponse() *StartAppManualCompactResponse {
-	return &StartAppManualCompactResponse{}
-}
-
-var StartAppManualCompactResponse_Err_DEFAULT *base.ErrorCode
-
-func (p *StartAppManualCompactResponse) GetErr() *base.ErrorCode {
-	if !p.IsSetErr() {
-		return StartAppManualCompactResponse_Err_DEFAULT
-	}
-	return p.Err
-}
-
-func (p *StartAppManualCompactResponse) GetHintMsg() string {
-	return p.HintMsg
-}
-func (p *StartAppManualCompactResponse) IsSetErr() bool {
-	return p.Err != nil
-}
-
-func (p *StartAppManualCompactResponse) 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 fieldTypeId == thrift.STRUCT {
-				if err := p.ReadField1(iprot); err != nil {
-					return err
-				}
-			} else {
-				if err := iprot.Skip(fieldTypeId); err != nil {
-					return err
-				}
-			}
-		case 2:
-			if fieldTypeId == thrift.STRING {
-				if err := p.ReadField2(iprot); err != nil {
-					return err
-				}
-			} else {
-				if err := iprot.Skip(fieldTypeId); 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 *StartAppManualCompactResponse) ReadField1(iprot thrift.TProtocol) error {
-	p.Err = &base.ErrorCode{}
-	if err := p.Err.Read(iprot); err != nil {
-		return thrift.PrependError(fmt.Sprintf("%T error reading struct: ", p.Err), err)
-	}
-	return nil
-}
-
-func (p *StartAppManualCompactResponse) ReadField2(iprot thrift.TProtocol) error {
-	if v, err := iprot.ReadString(); err != nil {
-		return thrift.PrependError("error reading field 2: ", err)
-	} else {
-		p.HintMsg = v
-	}
-	return nil
-}
-
-func (p *StartAppManualCompactResponse) Write(oprot thrift.TProtocol) error {
-	if err := oprot.WriteStructBegin("start_app_manual_compact_response"); 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 *StartAppManualCompactResponse) writeField1(oprot thrift.TProtocol) (err error) {
-	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 *StartAppManualCompactResponse) writeField2(oprot thrift.TProtocol) (err error) {
-	if err := oprot.WriteFieldBegin("hint_msg", thrift.STRING, 2); err != nil {
-		return thrift.PrependError(fmt.Sprintf("%T write field begin error 2:hint_msg: ", p), err)
-	}
-	if err := oprot.WriteString(string(p.HintMsg)); err != nil {
-		return thrift.PrependError(fmt.Sprintf("%T.hint_msg (2) field write error: ", p), err)
-	}
-	if err := oprot.WriteFieldEnd(); err != nil {
-		return thrift.PrependError(fmt.Sprintf("%T write field end error 2:hint_msg: ", p), err)
-	}
-	return err
-}
-
-func (p *StartAppManualCompactResponse) String() string {
-	if p == nil {
-		return "<nil>"
-	}
-	return fmt.Sprintf("StartAppManualCompactResponse(%+v)", *p)
-}
-
-// Attributes:
-//  - AppName
-type QueryAppManualCompactRequest struct {
-	AppName string `thrift:"app_name,1" db:"app_name" json:"app_name"`
-}
-
-func NewQueryAppManualCompactRequest() *QueryAppManualCompactRequest {
-	return &QueryAppManualCompactRequest{}
-}
-
-func (p *QueryAppManualCompactRequest) GetAppName() string {
-	return p.AppName
-}
-func (p *QueryAppManualCompactRequest) 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 fieldTypeId == thrift.STRING {
-				if err := p.ReadField1(iprot); err != nil {
-					return err
-				}
-			} else {
-				if err := iprot.Skip(fieldTypeId); 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 *QueryAppManualCompactRequest) ReadField1(iprot thrift.TProtocol) error {
-	if v, err := iprot.ReadString(); err != nil {
-		return thrift.PrependError("error reading field 1: ", err)
-	} else {
-		p.AppName = v
-	}
-	return nil
-}
-
-func (p *QueryAppManualCompactRequest) Write(oprot thrift.TProtocol) error {
-	if err := oprot.WriteStructBegin("query_app_manual_compact_request"); 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 *QueryAppManualCompactRequest) writeField1(oprot thrift.TProtocol) (err error) {
-	if err := oprot.WriteFieldBegin("app_name", thrift.STRING, 1); err != nil {
-		return thrift.PrependError(fmt.Sprintf("%T write field begin error 1:app_name: ", p), err)
-	}
-	if err := oprot.WriteString(string(p.AppName)); err != nil {
-		return thrift.PrependError(fmt.Sprintf("%T.app_name (1) field write error: ", p), err)
-	}
-	if err := oprot.WriteFieldEnd(); err != nil {
-		return thrift.PrependError(fmt.Sprintf("%T write field end error 1:app_name: ", p), err)
-	}
-	return err
-}
-
-func (p *QueryAppManualCompactRequest) String() string {
-	if p == nil {
-		return "<nil>"
-	}
-	return fmt.Sprintf("QueryAppManualCompactRequest(%+v)", *p)
-}
-
-// Attributes:
-//  - Err
-//  - HintMsg
-//  - Progress
-type QueryAppManualCompactResponse struct {
-	Err      *base.ErrorCode `thrift:"err,1" db:"err" json:"err"`
-	HintMsg  string          `thrift:"hint_msg,2" db:"hint_msg" json:"hint_msg"`
-	Progress *int32          `thrift:"progress,3" db:"progress" json:"progress,omitempty"`
-}
-
-func NewQueryAppManualCompactResponse() *QueryAppManualCompactResponse {
-	return &QueryAppManualCompactResponse{}
-}
-
-var QueryAppManualCompactResponse_Err_DEFAULT *base.ErrorCode
-
-func (p *QueryAppManualCompactResponse) GetErr() *base.ErrorCode {
-	if !p.IsSetErr() {
-		return QueryAppManualCompactResponse_Err_DEFAULT
-	}
-	return p.Err
-}
-
-func (p *QueryAppManualCompactResponse) GetHintMsg() string {
-	return p.HintMsg
-}
-
-var QueryAppManualCompactResponse_Progress_DEFAULT int32
-
-func (p *QueryAppManualCompactResponse) GetProgress() int32 {
-	if !p.IsSetProgress() {
-		return QueryAppManualCompactResponse_Progress_DEFAULT
-	}
-	return *p.Progress
-}
-func (p *QueryAppManualCompactResponse) IsSetErr() bool {
-	return p.Err != nil
-}
-
-func (p *QueryAppManualCompactResponse) IsSetProgress() bool {
-	return p.Progress != nil
-}
-
-func (p *QueryAppManualCompactResponse) 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 fieldTypeId == thrift.STRUCT {
-				if err := p.ReadField1(iprot); err != nil {
-					return err
-				}
-			} else {
-				if err := iprot.Skip(fieldTypeId); err != nil {
-					return err
-				}
-			}
-		case 2:
-			if fieldTypeId == thrift.STRING {
-				if err := p.ReadField2(iprot); err != nil {
-					return err
-				}
-			} else {
-				if err := iprot.Skip(fieldTypeId); err != nil {
-					return err
-				}
-			}
-		case 3:
-			if fieldTypeId == thrift.I32 {
-				if err := p.ReadField3(iprot); err != nil {
-					return err
-				}
-			} else {
-				if err := iprot.Skip(fieldTypeId); 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 *QueryAppManualCompactResponse) ReadField1(iprot thrift.TProtocol) error {
-	p.Err = &base.ErrorCode{}
-	if err := p.Err.Read(iprot); err != nil {
-		return thrift.PrependError(fmt.Sprintf("%T error reading struct: ", p.Err), err)
-	}
-	return nil
-}
-
-func (p *QueryAppManualCompactResponse) ReadField2(iprot thrift.TProtocol) error {
-	if v, err := iprot.ReadString(); err != nil {
-		return thrift.PrependError("error reading field 2: ", err)
-	} else {
-		p.HintMsg = v
-	}
-	return nil
-}
-
-func (p *QueryAppManualCompactResponse) ReadField3(iprot thrift.TProtocol) error {
-	if v, err := iprot.ReadI32(); err != nil {
-		return thrift.PrependError("error reading field 3: ", err)
-	} else {
-		p.Progress = &v
-	}
-	return nil
-}
-
-func (p *QueryAppManualCompactResponse) Write(oprot thrift.TProtocol) error {
-	if err := oprot.WriteStructBegin("query_app_manual_compact_response"); err != nil {
-		return thrift.PrependError(fmt.Sprintf("%T write struct begin error: ", p), err)
-	}
-	if p != nil {
-		if err := p.writeField1(oprot); err != nil {
-			return err
-		}
-		if err := p.writeField2(oprot); err != nil {
-			return err
-		}
-		if err := p.writeField3(oprot); err != nil {
-			return err
-		}
-	}
-	if err := oprot.WriteFieldStop(); err != nil {
-		return thrift.PrependError("write field stop error: ", err)
-	}
-	if err := oprot.WriteStructEnd(); err != nil {
-		return thrift.PrependError("write struct stop error: ", err)
-	}
-	return nil
-}
-
-func (p *QueryAppManualCompactResponse) writeField1(oprot thrift.TProtocol) (err error) {
-	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 *QueryAppManualCompactResponse) writeField2(oprot thrift.TProtocol) (err error) {
-	if err := oprot.WriteFieldBegin("hint_msg", thrift.STRING, 2); err != nil {
-		return thrift.PrependError(fmt.Sprintf("%T write field begin error 2:hint_msg: ", p), err)
-	}
-	if err := oprot.WriteString(string(p.HintMsg)); err != nil {
-		return thrift.PrependError(fmt.Sprintf("%T.hint_msg (2) field write error: ", p), err)
-	}
-	if err := oprot.WriteFieldEnd(); err != nil {
-		return thrift.PrependError(fmt.Sprintf("%T write field end error 2:hint_msg: ", p), err)
-	}
-	return err
-}
-
-func (p *QueryAppManualCompactResponse) writeField3(oprot thrift.TProtocol) (err error) {
-	if p.IsSetProgress() {
-		if err := oprot.WriteFieldBegin("progress", thrift.I32, 3); err != nil {
-			return thrift.PrependError(fmt.Sprintf("%T write field begin error 3:progress: ", p), err)
-		}
-		if err := oprot.WriteI32(int32(*p.Progress)); err != nil {
-			return thrift.PrependError(fmt.Sprintf("%T.progress (3) field write error: ", p), err)
-		}
-		if err := oprot.WriteFieldEnd(); err != nil {
-			return thrift.PrependError(fmt.Sprintf("%T write field end error 3:progress: ", p), err)
-		}
-	}
-	return err
-}
-
-func (p *QueryAppManualCompactResponse) String() string {
-	if p == nil {
-		return "<nil>"
-	}
-	return fmt.Sprintf("QueryAppManualCompactResponse(%+v)", *p)
-}
-
-// Attributes:
-//  - Status
-//  - Address
-type NodeInfo struct {
-	Status  NodeStatus       `thrift:"status,1" db:"status" json:"status"`
-	Address *base.RPCAddress `thrift:"address,2" db:"address" json:"address"`
-}
-
-func NewNodeInfo() *NodeInfo {
-	return &NodeInfo{
-		Status: 0,
-	}
-}
-
-func (p *NodeInfo) GetStatus() NodeStatus {
-	return p.Status
-}
-
-var NodeInfo_Address_DEFAULT *base.RPCAddress
-
-func (p *NodeInfo) GetAddress() *base.RPCAddress {
-	if !p.IsSetAddress() {
-		return NodeInfo_Address_DEFAULT
-	}
-	return p.Address
-}
-func (p *NodeInfo) IsSetAddress() bool {
-	return p.Address != nil
-}
-
-func (p *NodeInfo) 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 fieldTypeId == thrift.I32 {
-				if err := p.ReadField1(iprot); err != nil {
-					return err
-				}
-			} else {
-				if err := iprot.Skip(fieldTypeId); err != nil {
-					return err
-				}
-			}
-		case 2:
-			if fieldTypeId == thrift.STRUCT {
-				if err := p.ReadField2(iprot); err != nil {
-					return err
-				}
-			} else {
-				if err := iprot.Skip(fieldTypeId); 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 *NodeInfo) ReadField1(iprot thrift.TProtocol) error {
-	if v, err := iprot.ReadI32(); err != nil {
-		return thrift.PrependError("error reading field 1: ", err)
-	} else {
-		temp := NodeStatus(v)
-		p.Status = temp
-	}
-	return nil
-}
-
-func (p *NodeInfo) ReadField2(iprot thrift.TProtocol) error {
-	p.Address = &base.RPCAddress{}
-	if err := p.Address.Read(iprot); err != nil {
-		return thrift.PrependError(fmt.Sprintf("%T error reading struct: ", p.Address), err)
-	}
-	return nil
-}
-
-func (p *NodeInfo) Write(oprot thrift.TProtocol) error {
-	if err := oprot.WriteStructBegin("node_info"); 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 *NodeInfo) writeField1(oprot thrift.TProtocol) (err error) {
-	if err := oprot.WriteFieldBegin("status", thrift.I32, 1); err != nil {
-		return thrift.PrependError(fmt.Sprintf("%T write field begin error 1:status: ", p), err)
-	}
-	if err := oprot.WriteI32(int32(p.Status)); err != nil {
-		return thrift.PrependError(fmt.Sprintf("%T.status (1) field write error: ", p), err)
-	}
-	if err := oprot.WriteFieldEnd(); err != nil {
-		return thrift.PrependError(fmt.Sprintf("%T write field end error 1:status: ", p), err)
-	}
-	return err
-}
-
-func (p *NodeInfo) writeField2(oprot thrift.TProtocol) (err error) {
-	if err := oprot.WriteFieldBegin("address", thrift.STRUCT, 2); err != nil {
-		return thrift.PrependError(fmt.Sprintf("%T write field begin error 2:address: ", p), err)
-	}
-	if err := p.Address.Write(oprot); err != nil {
-		return thrift.PrependError(fmt.Sprintf("%T error writing struct: ", p.Address), err)
-	}
-	if err := oprot.WriteFieldEnd(); err != nil {
-		return thrift.PrependError(fmt.Sprintf("%T write field end error 2:address: ", p), err)
-	}
-	return err
-}
-
-func (p *NodeInfo) String() string {
-	if p == nil {
-		return "<nil>"
-	}
-	return fmt.Sprintf("NodeInfo(%+v)", *p)
-}
-
-// Attributes:
-//  - Status
-type ListNodesRequest struct {
-	Status NodeStatus `thrift:"status,1" db:"status" json:"status"`
-}
-
-func NewListNodesRequest() *ListNodesRequest {
-	return &ListNodesRequest{
-		Status: 0,
-	}
-}
-
-func (p *ListNodesRequest) GetStatus() NodeStatus {
-	return p.Status
-}
-func (p *ListNodesRequest) 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 fieldTypeId == thrift.I32 {
-				if err := p.ReadField1(iprot); err != nil {
-					return err
-				}
-			} else {
-				if err := iprot.Skip(fieldTypeId); 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 *ListNodesRequest) ReadField1(iprot thrift.TProtocol) error {
-	if v, err := iprot.ReadI32(); err != nil {
-		return thrift.PrependError("error reading field 1: ", err)
-	} else {
-		temp := NodeStatus(v)
-		p.Status = temp
-	}
-	return nil
-}
-
-func (p *ListNodesRequest) Write(oprot thrift.TProtocol) error {
-	if err := oprot.WriteStructBegin("list_nodes_request"); 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 *ListNodesRequest) writeField1(oprot thrift.TProtocol) (err error) {
-	if err := oprot.WriteFieldBegin("status", thrift.I32, 1); err != nil {
-		return thrift.PrependError(fmt.Sprintf("%T write field begin error 1:status: ", p), err)
-	}
-	if err := oprot.WriteI32(int32(p.Status)); err != nil {
-		return thrift.PrependError(fmt.Sprintf("%T.status (1) field write error: ", p), err)
-	}
-	if err := oprot.WriteFieldEnd(); err != nil {
-		return thrift.PrependError(fmt.Sprintf("%T write field end error 1:status: ", p), err)
-	}
-	return err
-}
-
-func (p *ListNodesRequest) String() string {
-	if p == nil {
-		return "<nil>"
-	}
-	return fmt.Sprintf("ListNodesRequest(%+v)", *p)
-}
-
-// Attributes:
-//  - Err
-//  - Infos
-type ListNodesResponse struct {
-	Err   *base.ErrorCode `thrift:"err,1" db:"err" json:"err"`
-	Infos []*NodeInfo     `thrift:"infos,2" db:"infos" json:"infos"`
-}
-
-func NewListNodesResponse() *ListNodesResponse {
-	return &ListNodesResponse{}
-}
-
-var ListNodesResponse_Err_DEFAULT *base.ErrorCode
-
-func (p *ListNodesResponse) GetErr() *base.ErrorCode {
-	if !p.IsSetErr() {
-		return ListNodesResponse_Err_DEFAULT
-	}
-	return p.Err
-}
-
-func (p *ListNodesResponse) GetInfos() []*NodeInfo {
-	return p.Infos
-}
-func (p *ListNodesResponse) IsSetErr() bool {
-	return p.Err != nil
-}
-
-func (p *ListNodesResponse) 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 fieldTypeId == thrift.STRUCT {
-				if err := p.ReadField1(iprot); err != nil {
-					return err
-				}
-			} else {
-				if err := iprot.Skip(fieldTypeId); err != nil {
-					return err
-				}
-			}
-		case 2:
-			if fieldTypeId == thrift.LIST {
-				if err := p.ReadField2(iprot); err != nil {
-					return err
-				}
-			} else {
-				if err := iprot.Skip(fieldTypeId); 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 *ListNodesResponse) ReadField1(iprot thrift.TProtocol) error {
-	p.Err = &base.ErrorCode{}
-	if err := p.Err.Read(iprot); err != nil {
-		return thrift.PrependError(fmt.Sprintf("%T error reading struct: ", p.Err), err)
-	}
-	return nil
-}
-
-func (p *ListNodesResponse) ReadField2(iprot thrift.TProtocol) error {
-	_, size, err := iprot.ReadListBegin()
-	if err != nil {
-		return thrift.PrependError("error reading list begin: ", err)
-	}
-	tSlice := make([]*NodeInfo, 0, size)
-	p.Infos = tSlice
-	for i := 0; i < size; i++ {
-		_elem8 := &NodeInfo{
-			Status: 0,
-		}
-		if err := _elem8.Read(iprot); err != nil {
-			return thrift.PrependError(fmt.Sprintf("%T error reading struct: ", _elem8), err)
-		}
-		p.Infos = append(p.Infos, _elem8)
-	}
-	if err := iprot.ReadListEnd(); err != nil {
-		return thrift.PrependError("error reading list end: ", err)
-	}
-	return nil
-}
-
-func (p *ListNodesResponse) Write(oprot thrift.TProtocol) error {
-	if err := oprot.WriteStructBegin("list_nodes_response"); 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 *ListNodesResponse) writeField1(oprot thrift.TProtocol) (err error) {
-	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 *ListNodesResponse) writeField2(oprot thrift.TProtocol) (err error) {
-	if err := oprot.WriteFieldBegin("infos", thrift.LIST, 2); err != nil {
-		return thrift.PrependError(fmt.Sprintf("%T write field begin error 2:infos: ", p), err)
-	}
-	if err := oprot.WriteListBegin(thrift.STRUCT, len(p.Infos)); err != nil {
-		return thrift.PrependError("error writing list begin: ", err)
-	}
-	for _, v := range p.Infos {
-		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:infos: ", p), err)
-	}
-	return err
-}
-
-func (p *ListNodesResponse) String() string {
-	if p == nil {
-		return "<nil>"
-	}
-	return fmt.Sprintf("ListNodesResponse(%+v)", *p)
-}
-
-type ClusterInfoRequest struct {
-}
-
-func NewClusterInfoRequest() *ClusterInfoRequest {
-	return &ClusterInfoRequest{}
-}
-
-func (p *ClusterInfoRequest) 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
-		}
-		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 *ClusterInfoRequest) Write(oprot thrift.TProtocol) error {
-	if err := oprot.WriteStructBegin("cluster_info_request"); err != nil {
-		return thrift.PrependError(fmt.Sprintf("%T write struct begin error: ", p), err)
-	}
-	if p != nil {
-	}
-	if err := oprot.WriteFieldStop(); err != nil {
-		return thrift.PrependError("write field stop error: ", err)
-	}
-	if err := oprot.WriteStructEnd(); err != nil {
-		return thrift.PrependError("write struct stop error: ", err)
-	}
-	return nil
-}
-
-func (p *ClusterInfoRequest) String() string {
-	if p == nil {
-		return "<nil>"
-	}
-	return fmt.Sprintf("ClusterInfoRequest(%+v)", *p)
-}
-
-// Attributes:
-//  - Err
-//  - Keys
-//  - Values
-type ClusterInfoResponse struct {
-	Err    *base.ErrorCode `thrift:"err,1" db:"err" json:"err"`
-	Keys   []string        `thrift:"keys,2" db:"keys" json:"keys"`
-	Values []string        `thrift:"values,3" db:"values" json:"values"`
-}
-
-func NewClusterInfoResponse() *ClusterInfoResponse {
-	return &ClusterInfoResponse{}
-}
-
-var ClusterInfoResponse_Err_DEFAULT *base.ErrorCode
-
-func (p *ClusterInfoResponse) GetErr() *base.ErrorCode {
-	if !p.IsSetErr() {
-		return ClusterInfoResponse_Err_DEFAULT
-	}
-	return p.Err
-}
-
-func (p *ClusterInfoResponse) GetKeys() []string {
-	return p.Keys
-}
-
-func (p *ClusterInfoResponse) GetValues() []string {
-	return p.Values
-}
-func (p *ClusterInfoResponse) IsSetErr() bool {
-	return p.Err != nil
-}
-
-func (p *ClusterInfoResponse) 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 fieldTypeId == thrift.STRUCT {
-				if err := p.ReadField1(iprot); err != nil {
-					return err
-				}
-			} else {
-				if err := iprot.Skip(fieldTypeId); err != nil {
-					return err
-				}
-			}
-		case 2:
-			if fieldTypeId == thrift.LIST {
-				if err := p.ReadField2(iprot); err != nil {
-					return err
-				}
-			} else {
-				if err := iprot.Skip(fieldTypeId); err != nil {
-					return err
-				}
-			}
-		case 3:
-			if fieldTypeId == thrift.LIST {
-				if err := p.ReadField3(iprot); err != nil {
-					return err
-				}
-			} else {
-				if err := iprot.Skip(fieldTypeId); 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 *ClusterInfoResponse) ReadField1(iprot thrift.TProtocol) error {
-	p.Err = &base.ErrorCode{}
-	if err := p.Err.Read(iprot); err != nil {
-		return thrift.PrependError(fmt.Sprintf("%T error reading struct: ", p.Err), err)
-	}
-	return nil
-}
-
-func (p *ClusterInfoResponse) ReadField2(iprot thrift.TProtocol) error {
-	_, size, err := iprot.ReadListBegin()
-	if err != nil {
-		return thrift.PrependError("error reading list begin: ", err)
-	}
-	tSlice := make([]string, 0, size)
-	p.Keys = tSlice
-	for i := 0; i < size; i++ {
-		var _elem9 string
-		if v, err := iprot.ReadString(); err != nil {
-			return thrift.PrependError("error reading field 0: ", err)
-		} else {
-			_elem9 = v
-		}
-		p.Keys = append(p.Keys, _elem9)
-	}
-	if err := iprot.ReadListEnd(); err != nil {
-		return thrift.PrependError("error reading list end: ", err)
-	}
-	return nil
-}
-
-func (p *ClusterInfoResponse) ReadField3(iprot thrift.TProtocol) error {
-	_, size, err := iprot.ReadListBegin()
-	if err != nil {
-		return thrift.PrependError("error reading list begin: ", err)
-	}
-	tSlice := make([]string, 0, size)
-	p.Values = tSlice
-	for i := 0; i < size; i++ {
-		var _elem10 string
-		if v, err := iprot.ReadString(); err != nil {
-			return thrift.PrependError("error reading field 0: ", err)
-		} else {
-			_elem10 = v
-		}
-		p.Values = append(p.Values, _elem10)
-	}
-	if err := iprot.ReadListEnd(); err != nil {
-		return thrift.PrependError("error reading list end: ", err)
-	}
-	return nil
-}
-
-func (p *ClusterInfoResponse) Write(oprot thrift.TProtocol) error {
-	if err := oprot.WriteStructBegin("cluster_info_response"); err != nil {
-		return thrift.PrependError(fmt.Sprintf("%T write struct begin error: ", p), err)
-	}
-	if p != nil {
-		if err := p.writeField1(oprot); err != nil {
-			return err
-		}
-		if err := p.writeField2(oprot); err != nil {
-			return err
-		}
-		if err := p.writeField3(oprot); err != nil {
-			return err
-		}
-	}
-	if err := oprot.WriteFieldStop(); err != nil {
-		return thrift.PrependError("write field stop error: ", err)
-	}
-	if err := oprot.WriteStructEnd(); err != nil {
-		return thrift.PrependError("write struct stop error: ", err)
-	}
-	return nil
-}
-
-func (p *ClusterInfoResponse) writeField1(oprot thrift.TProtocol) (err error) {
-	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 *ClusterInfoResponse) writeField2(oprot thrift.TProtocol) (err error) {
-	if err := oprot.WriteFieldBegin("keys", thrift.LIST, 2); err != nil {
-		return thrift.PrependError(fmt.Sprintf("%T write field begin error 2:keys: ", p), err)
-	}
-	if err := oprot.WriteListBegin(thrift.STRING, len(p.Keys)); err != nil {
-		return thrift.PrependError("error writing list begin: ", err)
-	}
-	for _, v := range p.Keys {
-		if err := oprot.WriteString(string(v)); err != nil {
-			return thrift.PrependError(fmt.Sprintf("%T. (0) field write error: ", p), err)
-		}
-	}
-	if err := oprot.WriteListEnd(); err != nil {
-		return thrift.PrependError("error writing list end: ", err)
-	}
-	if err := oprot.WriteFieldEnd(); err != nil {
-		return thrift.PrependError(fmt.Sprintf("%T write field end error 2:keys: ", p), err)
-	}
-	return err
-}
-
-func (p *ClusterInfoResponse) writeField3(oprot thrift.TProtocol) (err error) {
-	if err := oprot.WriteFieldBegin("values", thrift.LIST, 3); err != nil {
-		return thrift.PrependError(fmt.Sprintf("%T write field begin error 3:values: ", p), err)
-	}
-	if err := oprot.WriteListBegin(thrift.STRING, len(p.Values)); err != nil {
-		return thrift.PrependError("error writing list begin: ", err)
-	}
-	for _, v := range p.Values {
-		if err := oprot.WriteString(string(v)); err != nil {
-			return thrift.PrependError(fmt.Sprintf("%T. (0) field write error: ", p), err)
-		}
-	}
-	if err := oprot.WriteListEnd(); err != nil {
-		return thrift.PrependError("error writing list end: ", err)
-	}
-	if err := oprot.WriteFieldEnd(); err != nil {
-		return thrift.PrependError(fmt.Sprintf("%T write field end error 3:values: ", p), err)
-	}
-	return err
-}
-
-func (p *ClusterInfoResponse) String() string {
-	if p == nil {
-		return "<nil>"
-	}
-	return fmt.Sprintf("ClusterInfoResponse(%+v)", *p)
-}
-
-// Attributes:
-//  - Level
-type MetaControlRequest struct {
-	Level MetaFunctionLevel `thrift:"level,1" db:"level" json:"level"`
-}
-
-func NewMetaControlRequest() *MetaControlRequest {
-	return &MetaControlRequest{}
-}
-
-func (p *MetaControlRequest) GetLevel() MetaFunctionLevel {
-	return p.Level
-}
-func (p *MetaControlRequest) 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 fieldTypeId == thrift.I32 {
-				if err := p.ReadField1(iprot); err != nil {
-					return err
-				}
-			} else {
-				if err := iprot.Skip(fieldTypeId); 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 *MetaControlRequest) ReadField1(iprot thrift.TProtocol) error {
-	if v, err := iprot.ReadI32(); err != nil {
-		return thrift.PrependError("error reading field 1: ", err)
-	} else {
-		temp := MetaFunctionLevel(v)
-		p.Level = temp
-	}
-	return nil
-}
-
-func (p *MetaControlRequest) Write(oprot thrift.TProtocol) error {
-	if err := oprot.WriteStructBegin("meta_control_request"); 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 *MetaControlRequest) writeField1(oprot thrift.TProtocol) (err error) {
-	if err := oprot.WriteFieldBegin("level", thrift.I32, 1); err != nil {
-		return thrift.PrependError(fmt.Sprintf("%T write field begin error 1:level: ", p), err)
-	}
-	if err := oprot.WriteI32(int32(p.Level)); err != nil {
-		return thrift.PrependError(fmt.Sprintf("%T.level (1) field write error: ", p), err)
-	}
-	if err := oprot.WriteFieldEnd(); err != nil {
-		return thrift.PrependError(fmt.Sprintf("%T write field end error 1:level: ", p), err)
-	}
-	return err
-}
-
-func (p *MetaControlRequest) String() string {
-	if p == nil {
-		return "<nil>"
-	}
-	return fmt.Sprintf("MetaControlRequest(%+v)", *p)
-}
-
-// Attributes:
-//  - Err
-//  - OldLevel
-type MetaControlResponse struct {
-	Err      *base.ErrorCode   `thrift:"err,1" db:"err" json:"err"`
-	OldLevel MetaFunctionLevel `thrift:"old_level,2" db:"old_level" json:"old_level"`
-}
-
-func NewMetaControlResponse() *MetaControlResponse {
-	return &MetaControlResponse{}
-}
-
-var MetaControlResponse_Err_DEFAULT *base.ErrorCode
-
-func (p *MetaControlResponse) GetErr() *base.ErrorCode {
-	if !p.IsSetErr() {
-		return MetaControlResponse_Err_DEFAULT
-	}
-	return p.Err
-}
-
-func (p *MetaControlResponse) GetOldLevel() MetaFunctionLevel {
-	return p.OldLevel
-}
-func (p *MetaControlResponse) IsSetErr() bool {
-	return p.Err != nil
-}
-
-func (p *MetaControlResponse) 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 fieldTypeId == thrift.STRUCT {
-				if err := p.ReadField1(iprot); err != nil {
-					return err
-				}
-			} else {
-				if err := iprot.Skip(fieldTypeId); err != nil {
-					return err
-				}
-			}
-		case 2:
-			if fieldTypeId == thrift.I32 {
-				if err := p.ReadField2(iprot); err != nil {
-					return err
-				}
-			} else {
-				if err := iprot.Skip(fieldTypeId); 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 *MetaControlResponse) ReadField1(iprot thrift.TProtocol) error {
-	p.Err = &base.ErrorCode{}
-	if err := p.Err.Read(iprot); err != nil {
-		return thrift.PrependError(fmt.Sprintf("%T error reading struct: ", p.Err), err)
-	}
-	return nil
-}
-
-func (p *MetaControlResponse) ReadField2(iprot thrift.TProtocol) error {
-	if v, err := iprot.ReadI32(); err != nil {
-		return thrift.PrependError("error reading field 2: ", err)
-	} else {
-		temp := MetaFunctionLevel(v)
-		p.OldLevel = temp
-	}
-	return nil
-}
-
-func (p *MetaControlResponse) Write(oprot thrift.TProtocol) error {
-	if err := oprot.WriteStructBegin("meta_control_response"); 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 *MetaControlResponse) writeField1(oprot thrift.TProtocol) (err error) {
-	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 *MetaControlResponse) writeField2(oprot thrift.TProtocol) (err error) {
-	if err := oprot.WriteFieldBegin("old_level", thrift.I32, 2); err != nil {
-		return thrift.PrependError(fmt.Sprintf("%T write field begin error 2:old_level: ", p), err)
-	}
-	if err := oprot.WriteI32(int32(p.OldLevel)); err != nil {
-		return thrift.PrependError(fmt.Sprintf("%T.old_level (2) field write error: ", p), err)
-	}
-	if err := oprot.WriteFieldEnd(); err != nil {
-		return thrift.PrependError(fmt.Sprintf("%T write field end error 2:old_level: ", p), err)
-	}
-	return err
-}
-
-func (p *MetaControlResponse) String() string {
-	if p == nil {
-		return "<nil>"
-	}
-	return fmt.Sprintf("MetaControlResponse(%+v)", *p)
-}
-
-// Attributes:
-//  - AppName
-//  - RemoteClusterName
-//  - IsDuplicatingCheckpoint
-type DuplicationAddRequest struct {
-	AppName                 string `thrift:"app_name,1" db:"app_name" json:"app_name"`
-	RemoteClusterName       string `thrift:"remote_cluster_name,2" db:"remote_cluster_name" json:"remote_cluster_name"`
-	IsDuplicatingCheckpoint bool   `thrift:"is_duplicating_checkpoint,3" db:"is_duplicating_checkpoint" json:"is_duplicating_checkpoint"`
-}
-
-func NewDuplicationAddRequest() *DuplicationAddRequest {
-	return &DuplicationAddRequest{
-		IsDuplicatingCheckpoint: true,
-	}
-}
-
-func (p *DuplicationAddRequest) GetAppName() string {
-	return p.AppName
-}
-
-func (p *DuplicationAddRequest) GetRemoteClusterName() string {
-	return p.RemoteClusterName
-}
-
-var DuplicationAddRequest_IsDuplicatingCheckpoint_DEFAULT bool = true
-
-func (p *DuplicationAddRequest) GetIsDuplicatingCheckpoint() bool {
-	return p.IsDuplicatingCheckpoint
-}
-func (p *DuplicationAddRequest) IsSetIsDuplicatingCheckpoint() bool {
-	return p.IsDuplicatingCheckpoint != DuplicationAddRequest_IsDuplicatingCheckpoint_DEFAULT
-}
-
-func (p *DuplicationAddRequest) 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 fieldTypeId == thrift.STRING {
-				if err := p.ReadField1(iprot); err != nil {
-					return err
-				}
-			} else {
-				if err := iprot.Skip(fieldTypeId); err != nil {
-					return err
-				}
-			}
-		case 2:
-			if fieldTypeId == thrift.STRING {
-				if err := p.ReadField2(iprot); err != nil {
-					return err
-				}
-			} else {
-				if err := iprot.Skip(fieldTypeId); err != nil {
-					return err
-				}
-			}
-		case 3:
-			if fieldTypeId == thrift.BOOL {
-				if err := p.ReadField3(iprot); err != nil {
-					return err
-				}
-			} else {
-				if err := iprot.Skip(fieldTypeId); 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 *DuplicationAddRequest) ReadField1(iprot thrift.TProtocol) error {
-	if v, err := iprot.ReadString(); err != nil {
-		return thrift.PrependError("error reading field 1: ", err)
-	} else {
-		p.AppName = v
-	}
-	return nil
-}
-
-func (p *DuplicationAddRequest) ReadField2(iprot thrift.TProtocol) error {
-	if v, err := iprot.ReadString(); err != nil {
-		return thrift.PrependError("error reading field 2: ", err)
-	} else {
-		p.RemoteClusterName = v
-	}
-	return nil
-}
-
-func (p *DuplicationAddRequest) ReadField3(iprot thrift.TProtocol) error {
-	if v, err := iprot.ReadBool(); err != nil {
-		return thrift.PrependError("error reading field 3: ", err)
-	} else {
-		p.IsDuplicatingCheckpoint = v
-	}
-	return nil
-}
-
-func (p *DuplicationAddRequest) Write(oprot thrift.TProtocol) error {
-	if err := oprot.WriteStructBegin("duplication_add_request"); err != nil {
-		return thrift.PrependError(fmt.Sprintf("%T write struct begin error: ", p), err)
-	}
-	if p != nil {
-		if err := p.writeField1(oprot); err != nil {
-			return err
-		}
-		if err := p.writeField2(oprot); err != nil {
-			return err
-		}
-		if err := p.writeField3(oprot); err != nil {
-			return err
-		}
-	}
-	if err := oprot.WriteFieldStop(); err != nil {
-		return thrift.PrependError("write field stop error: ", err)
-	}
-	if err := oprot.WriteStructEnd(); err != nil {
-		return thrift.PrependError("write struct stop error: ", err)
-	}
-	return nil
-}
-
-func (p *DuplicationAddRequest) writeField1(oprot thrift.TProtocol) (err error) {
-	if err := oprot.WriteFieldBegin("app_name", thrift.STRING, 1); err != nil {
-		return thrift.PrependError(fmt.Sprintf("%T write field begin error 1:app_name: ", p), err)
-	}
-	if err := oprot.WriteString(string(p.AppName)); err != nil {
-		return thrift.PrependError(fmt.Sprintf("%T.app_name (1) field write error: ", p), err)
-	}
-	if err := oprot.WriteFieldEnd(); err != nil {
-		return thrift.PrependError(fmt.Sprintf("%T write field end error 1:app_name: ", p), err)
-	}
-	return err
-}
-
-func (p *DuplicationAddRequest) writeField2(oprot thrift.TProtocol) (err error) {
-	if err := oprot.WriteFieldBegin("remote_cluster_name", thrift.STRING, 2); err != nil {
-		return thrift.PrependError(fmt.Sprintf("%T write field begin error 2:remote_cluster_name: ", p), err)
-	}
-	if err := oprot.WriteString(string(p.RemoteClusterName)); err != nil {
-		return thrift.PrependError(fmt.Sprintf("%T.remote_cluster_name (2) field write error: ", p), err)
-	}
-	if err := oprot.WriteFieldEnd(); err != nil {
-		return thrift.PrependError(fmt.Sprintf("%T write field end error 2:remote_cluster_name: ", p), err)
-	}
-	return err
-}
-
-func (p *DuplicationAddRequest) writeField3(oprot thrift.TProtocol) (err error) {
-	if p.IsSetIsDuplicatingCheckpoint() {
-		if err := oprot.WriteFieldBegin("is_duplicating_checkpoint", thrift.BOOL, 3); err != nil {
-			return thrift.PrependError(fmt.Sprintf("%T write field begin error 3:is_duplicating_checkpoint: ", p), err)
-		}
-		if err := oprot.WriteBool(bool(p.IsDuplicatingCheckpoint)); err != nil {
-			return thrift.PrependError(fmt.Sprintf("%T.is_duplicating_checkpoint (3) field write error: ", p), err)
-		}
-		if err := oprot.WriteFieldEnd(); err != nil {
-			return thrift.PrependError(fmt.Sprintf("%T write field end error 3:is_duplicating_checkpoint: ", p), err)
-		}
-	}
-	return err
-}
-
-func (p *DuplicationAddRequest) String() string {
-	if p == nil {
-		return "<nil>"
-	}
-	return fmt.Sprintf("DuplicationAddRequest(%+v)", *p)
-}
-
-// Attributes:
-//  - Err
-//  - Appid
-//  - Dupid
-//  - Hint
-type DuplicationAddResponse struct {
-	Err   *base.ErrorCode `thrift:"err,1" db:"err" json:"err"`
-	Appid int32           `thrift:"appid,2" db:"appid" json:"appid"`
-	Dupid int32           `thrift:"dupid,3" db:"dupid" json:"dupid"`
-	Hint  *string         `thrift:"hint,4" db:"hint" json:"hint,omitempty"`
-}
-
-func NewDuplicationAddResponse() *DuplicationAddResponse {
-	return &DuplicationAddResponse{}
-}
-
-var DuplicationAddResponse_Err_DEFAULT *base.ErrorCode
-
-func (p *DuplicationAddResponse) GetErr() *base.ErrorCode {
-	if !p.IsSetErr() {
-		return DuplicationAddResponse_Err_DEFAULT
-	}
-	return p.Err
-}
-
-func (p *DuplicationAddResponse) GetAppid() int32 {
-	return p.Appid
-}
-
-func (p *DuplicationAddResponse) GetDupid() int32 {
-	return p.Dupid
-}
-
-var DuplicationAddResponse_Hint_DEFAULT string
-
-func (p *DuplicationAddResponse) GetHint() string {
-	if !p.IsSetHint() {
-		return DuplicationAddResponse_Hint_DEFAULT
-	}
-	return *p.Hint
-}
-func (p *DuplicationAddResponse) IsSetErr() bool {
-	return p.Err != nil
-}
-
-func (p *DuplicationAddResponse) IsSetHint() bool {
-	return p.Hint != nil
-}
-
-func (p *DuplicationAddResponse) 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 fieldTypeId == thrift.STRUCT {
-				if err := p.ReadField1(iprot); err != nil {
-					return err
-				}
-			} else {
-				if err := iprot.Skip(fieldTypeId); err != nil {
-					return err
-				}
-			}
-		case 2:
-			if fieldTypeId == thrift.I32 {
-				if err := p.ReadField2(iprot); err != nil {
-					return err
-				}
-			} else {
-				if err := iprot.Skip(fieldTypeId); err != nil {
-					return err
-				}
-			}
-		case 3:
-			if fieldTypeId == thrift.I32 {
-				if err := p.ReadField3(iprot); err != nil {
-					return err
-				}
-			} else {
-				if err := iprot.Skip(fieldTypeId); err != nil {
-					return err
-				}
-			}
-		case 4:
-			if fieldTypeId == thrift.STRING {
-				if err := p.ReadField4(iprot); err != nil {
-					return err
-				}
-			} else {
-				if err := iprot.Skip(fieldTypeId); 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 *DuplicationAddResponse) ReadField1(iprot thrift.TProtocol) error {
-	p.Err = &base.ErrorCode{}
-	if err := p.Err.Read(iprot); err != nil {
-		return thrift.PrependError(fmt.Sprintf("%T error reading struct: ", p.Err), err)
-	}
-	return nil
-}
-
-func (p *DuplicationAddResponse) ReadField2(iprot thrift.TProtocol) error {
-	if v, err := iprot.ReadI32(); err != nil {
-		return thrift.PrependError("error reading field 2: ", err)
-	} else {
-		p.Appid = v
-	}
-	return nil
-}
-
-func (p *DuplicationAddResponse) ReadField3(iprot thrift.TProtocol) error {
-	if v, err := iprot.ReadI32(); err != nil {
-		return thrift.PrependError("error reading field 3: ", err)
-	} else {
-		p.Dupid = v
-	}
-	return nil
-}
-
-func (p *DuplicationAddResponse) ReadField4(iprot thrift.TProtocol) error {
-	if v, err := iprot.ReadString(); err != nil {
-		return thrift.PrependError("error reading field 4: ", err)
-	} else {
-		p.Hint = &v
-	}
-	return nil
-}
-
-func (p *DuplicationAddResponse) Write(oprot thrift.TProtocol) error {
-	if err := oprot.WriteStructBegin("duplication_add_response"); 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 := 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 *DuplicationAddResponse) writeField1(oprot thrift.TProtocol) (err error) {
-	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 *DuplicationAddResponse) writeField2(oprot thrift.TProtocol) (err error) {
-	if err := oprot.WriteFieldBegin("appid", thrift.I32, 2); err != nil {
-		return thrift.PrependError(fmt.Sprintf("%T write field begin error 2:appid: ", p), err)
-	}
-	if err := oprot.WriteI32(int32(p.Appid)); err != nil {
-		return thrift.PrependError(fmt.Sprintf("%T.appid (2) field write error: ", p), err)
-	}
-	if err := oprot.WriteFieldEnd(); err != nil {
-		return thrift.PrependError(fmt.Sprintf("%T write field end error 2:appid: ", p), err)
-	}
-	return err
-}
-
-func (p *DuplicationAddResponse) writeField3(oprot thrift.TProtocol) (err error) {
-	if err := oprot.WriteFieldBegin("dupid", thrift.I32, 3); err != nil {
-		return thrift.PrependError(fmt.Sprintf("%T write field begin error 3:dupid: ", p), err)
-	}
-	if err := oprot.WriteI32(int32(p.Dupid)); err != nil {
-		return thrift.PrependError(fmt.Sprintf("%T.dupid (3) field write error: ", p), err)
-	}
-	if err := oprot.WriteFieldEnd(); err != nil {
-		return thrift.PrependError(fmt.Sprintf("%T write field end error 3:dupid: ", p), err)
-	}
-	return err
-}
-
-func (p *DuplicationAddResponse) writeField4(oprot thrift.TProtocol) (err error) {
-	if p.IsSetHint() {
-		if err := oprot.WriteFieldBegin("hint", thrift.STRING, 4); err != nil {
-			return thrift.PrependError(fmt.Sprintf("%T write field begin error 4:hint: ", p), err)
-		}
-		if err := oprot.WriteString(string(*p.Hint)); err != nil {
-			return thrift.PrependError(fmt.Sprintf("%T.hint (4) field write error: ", p), err)
-		}
-		if err := oprot.WriteFieldEnd(); err != nil {
-			return thrift.PrependError(fmt.Sprintf("%T write field end error 4:hint: ", p), err)
-		}
-	}
-	return err
-}
-
-func (p *DuplicationAddResponse) String() string {
-	if p == nil {
-		return "<nil>"
-	}
-	return fmt.Sprintf("DuplicationAddResponse(%+v)", *p)
-}
-
-// Attributes:
-//  - AppName
-//  - Dupid
-//  - Status
-//  - FailMode
-type DuplicationModifyRequest struct {
-	AppName  string               `thrift:"app_name,1" db:"app_name" json:"app_name"`
-	Dupid    int32                `thrift:"dupid,2" db:"dupid" json:"dupid"`
-	Status   *DuplicationStatus   `thrift:"status,3" db:"status" json:"status,omitempty"`
-	FailMode *DuplicationFailMode `thrift:"fail_mode,4" db:"fail_mode" json:"fail_mode,omitempty"`
-}
-
-func NewDuplicationModifyRequest() *DuplicationModifyRequest {
-	return &DuplicationModifyRequest{}
-}
-
-func (p *DuplicationModifyRequest) GetAppName() string {
-	return p.AppName
-}
-
-func (p *DuplicationModifyRequest) GetDupid() int32 {
-	return p.Dupid
-}
-
-var DuplicationModifyRequest_Status_DEFAULT DuplicationStatus
-
-func (p *DuplicationModifyRequest) GetStatus() DuplicationStatus {
-	if !p.IsSetStatus() {
-		return DuplicationModifyRequest_Status_DEFAULT
-	}
-	return *p.Status
-}
-
-var DuplicationModifyRequest_FailMode_DEFAULT DuplicationFailMode
-
-func (p *DuplicationModifyRequest) GetFailMode() DuplicationFailMode {
-	if !p.IsSetFailMode() {
-		return DuplicationModifyRequest_FailMode_DEFAULT
-	}
-	return *p.FailMode
-}
-func (p *DuplicationModifyRequest) IsSetStatus() bool {
-	return p.Status != nil
-}
-
-func (p *DuplicationModifyRequest) IsSetFailMode() bool {
-	return p.FailMode != nil
-}
-
-func (p *DuplicationModifyRequest) 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 fieldTypeId == thrift.STRING {
-				if err := p.ReadField1(iprot); err != nil {
-					return err
-				}
-			} else {
-				if err := iprot.Skip(fieldTypeId); err != nil {
-					return err
-				}
-			}
-		case 2:
-			if fieldTypeId == thrift.I32 {
-				if err := p.ReadField2(iprot); err != nil {
-					return err
-				}
-			} else {
-				if err := iprot.Skip(fieldTypeId); err != nil {
-					return err
-				}
-			}
-		case 3:
-			if fieldTypeId == thrift.I32 {
-				if err := p.ReadField3(iprot); err != nil {
-					return err
-				}
-			} else {
-				if err := iprot.Skip(fieldTypeId); err != nil {
-					return err
-				}
-			}
-		case 4:
-			if fieldTypeId == thrift.I32 {
-				if err := p.ReadField4(iprot); err != nil {
-					return err
-				}
-			} else {
-				if err := iprot.Skip(fieldTypeId); 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 *DuplicationModifyRequest) ReadField1(iprot thrift.TProtocol) error {
-	if v, err := iprot.ReadString(); err != nil {
-		return thrift.PrependError("error reading field 1: ", err)
-	} else {
-		p.AppName = v
-	}
-	return nil
-}
-
-func (p *DuplicationModifyRequest) ReadField2(iprot thrift.TProtocol) error {
-	if v, err := iprot.ReadI32(); err != nil {
-		return thrift.PrependError("error reading field 2: ", err)
-	} else {
-		p.Dupid = v
-	}
-	return nil
-}
-
-func (p *DuplicationModifyRequest) ReadField3(iprot thrift.TProtocol) error {
-	if v, err := iprot.ReadI32(); err != nil {
-		return thrift.PrependError("error reading field 3: ", err)
-	} else {
-		temp := DuplicationStatus(v)
-		p.Status = &temp
-	}
-	return nil
-}
-
-func (p *DuplicationModifyRequest) ReadField4(iprot thrift.TProtocol) error {
-	if v, err := iprot.ReadI32(); err != nil {
-		return thrift.PrependError("error reading field 4: ", err)
-	} else {
-		temp := DuplicationFailMode(v)
-		p.FailMode = &temp
-	}
-	return nil
-}
-
-func (p *DuplicationModifyRequest) Write(oprot thrift.TProtocol) error {
-	if err := oprot.WriteStructBegin("duplication_modify_request"); 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 := 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 *DuplicationModifyRequest) writeField1(oprot thrift.TProtocol) (err error) {
-	if err := oprot.WriteFieldBegin("app_name", thrift.STRING, 1); err != nil {
-		return thrift.PrependError(fmt.Sprintf("%T write field begin error 1:app_name: ", p), err)
-	}
-	if err := oprot.WriteString(string(p.AppName)); err != nil {
-		return thrift.PrependError(fmt.Sprintf("%T.app_name (1) field write error: ", p), err)
-	}
-	if err := oprot.WriteFieldEnd(); err != nil {
-		return thrift.PrependError(fmt.Sprintf("%T write field end error 1:app_name: ", p), err)
-	}
-	return err
-}
-
-func (p *DuplicationModifyRequest) writeField2(oprot thrift.TProtocol) (err error) {
-	if err := oprot.WriteFieldBegin("dupid", thrift.I32, 2); err != nil {
-		return thrift.PrependError(fmt.Sprintf("%T write field begin error 2:dupid: ", p), err)
-	}
-	if err := oprot.WriteI32(int32(p.Dupid)); err != nil {
-		return thrift.PrependError(fmt.Sprintf("%T.dupid (2) field write error: ", p), err)
-	}
-	if err := oprot.WriteFieldEnd(); err != nil {
-		return thrift.PrependError(fmt.Sprintf("%T write field end error 2:dupid: ", p), err)
-	}
-	return err
-}
-
-func (p *DuplicationModifyRequest) writeField3(oprot thrift.TProtocol) (err error) {
-	if p.IsSetStatus() {
-		if err := oprot.WriteFieldBegin("status", thrift.I32, 3); err != nil {
-			return thrift.PrependError(fmt.Sprintf("%T write field begin error 3:status: ", p), err)
-		}
-		if err := oprot.WriteI32(int32(*p.Status)); err != nil {
-			return thrift.PrependError(fmt.Sprintf("%T.status (3) field write error: ", p), err)
-		}
-		if err := oprot.WriteFieldEnd(); err != nil {
-			return thrift.PrependError(fmt.Sprintf("%T write field end error 3:status: ", p), err)
-		}
-	}
-	return err
-}
-
-func (p *DuplicationModifyRequest) writeField4(oprot thrift.TProtocol) (err error) {
-	if p.IsSetFailMode() {
-		if err := oprot.WriteFieldBegin("fail_mode", thrift.I32, 4); err != nil {
-			return thrift.PrependError(fmt.Sprintf("%T write field begin error 4:fail_mode: ", p), err)
-		}
-		if err := oprot.WriteI32(int32(*p.FailMode)); err != nil {
-			return thrift.PrependError(fmt.Sprintf("%T.fail_mode (4) field write error: ", p), err)
-		}
-		if err := oprot.WriteFieldEnd(); err != nil {
-			return thrift.PrependError(fmt.Sprintf("%T write field end error 4:fail_mode: ", p), err)
-		}
-	}
-	return err
-}
-
-func (p *DuplicationModifyRequest) String() string {
-	if p == nil {
-		return "<nil>"
-	}
-	return fmt.Sprintf("DuplicationModifyRequest(%+v)", *p)
-}
-
-// Attributes:
-//  - Err
-//  - Appid
-type DuplicationModifyResponse struct {
-	Err   *base.ErrorCode `thrift:"err,1" db:"err" json:"err"`
-	Appid int32           `thrift:"appid,2" db:"appid" json:"appid"`
-}
-
-func NewDuplicationModifyResponse() *DuplicationModifyResponse {
-	return &DuplicationModifyResponse{}
-}
-
-var DuplicationModifyResponse_Err_DEFAULT *base.ErrorCode
-
-func (p *DuplicationModifyResponse) GetErr() *base.ErrorCode {
-	if !p.IsSetErr() {
-		return DuplicationModifyResponse_Err_DEFAULT
-	}
-	return p.Err
-}
-
-func (p *DuplicationModifyResponse) GetAppid() int32 {
-	return p.Appid
-}
-func (p *DuplicationModifyResponse) IsSetErr() bool {
-	return p.Err != nil
-}
-
-func (p *DuplicationModifyResponse) 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 fieldTypeId == thrift.STRUCT {
-				if err := p.ReadField1(iprot); err != nil {
-					return err
-				}
-			} else {
-				if err := iprot.Skip(fieldTypeId); err != nil {
-					return err
-				}
-			}
-		case 2:
-			if fieldTypeId == thrift.I32 {
-				if err := p.ReadField2(iprot); err != nil {
-					return err
-				}
-			} else {
-				if err := iprot.Skip(fieldTypeId); 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 *DuplicationModifyResponse) ReadField1(iprot thrift.TProtocol) error {
-	p.Err = &base.ErrorCode{}
-	if err := p.Err.Read(iprot); err != nil {
-		return thrift.PrependError(fmt.Sprintf("%T error reading struct: ", p.Err), err)
-	}
-	return nil
-}
-
-func (p *DuplicationModifyResponse) ReadField2(iprot thrift.TProtocol) error {
-	if v, err := iprot.ReadI32(); err != nil {
-		return thrift.PrependError("error reading field 2: ", err)
-	} else {
-		p.Appid = v
-	}
-	return nil
-}
-
-func (p *DuplicationModifyResponse) Write(oprot thrift.TProtocol) error {
-	if err := oprot.WriteStructBegin("duplication_modify_response"); 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 *DuplicationModifyResponse) writeField1(oprot thrift.TProtocol) (err error) {
-	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 *DuplicationModifyResponse) writeField2(oprot thrift.TProtocol) (err error) {
-	if err := oprot.WriteFieldBegin("appid", thrift.I32, 2); err != nil {
-		return thrift.PrependError(fmt.Sprintf("%T write field begin error 2:appid: ", p), err)
-	}
-	if err := oprot.WriteI32(int32(p.Appid)); err != nil {
-		return thrift.PrependError(fmt.Sprintf("%T.appid (2) field write error: ", p), err)
-	}
-	if err := oprot.WriteFieldEnd(); err != nil {
-		return thrift.PrependError(fmt.Sprintf("%T write field end error 2:appid: ", p), err)
-	}
-	return err
-}
-
-func (p *DuplicationModifyResponse) String() string {
-	if p == nil {
-		return "<nil>"
-	}
-	return fmt.Sprintf("DuplicationModifyResponse(%+v)", *p)
-}
-
-// Attributes:
-//  - Dupid
-//  - Status
-//  - Remote
-//  - CreateTs
-//  - Progress
-//  - FailMode
-type DuplicationEntry struct {
-	Dupid    int32             `thrift:"dupid,1" db:"dupid" json:"dupid"`
-	Status   DuplicationStatus `thrift:"status,2" db:"status" json:"status"`
-	Remote   string            `thrift:"remote,3" db:"remote" json:"remote"`
-	CreateTs int64             `thrift:"create_ts,4" db:"create_ts" json:"create_ts"`
-	Progress map[int32]int64   `thrift:"progress,5" db:"progress" json:"progress,omitempty"`
-	// unused field # 6
-	FailMode *DuplicationFailMode `thrift:"fail_mode,7" db:"fail_mode" json:"fail_mode,omitempty"`
-}
-
-func NewDuplicationEntry() *DuplicationEntry {
-	return &DuplicationEntry{}
-}
-
-func (p *DuplicationEntry) GetDupid() int32 {
-	return p.Dupid
-}
-
-func (p *DuplicationEntry) GetStatus() DuplicationStatus {
-	return p.Status
-}
-
-func (p *DuplicationEntry) GetRemote() string {
-	return p.Remote
-}
-
-func (p *DuplicationEntry) GetCreateTs() int64 {
-	return p.CreateTs
-}
-
-var DuplicationEntry_Progress_DEFAULT map[int32]int64
-
-func (p *DuplicationEntry) GetProgress() map[int32]int64 {
-	return p.Progress
-}
-
-var DuplicationEntry_FailMode_DEFAULT DuplicationFailMode
-
-func (p *DuplicationEntry) GetFailMode() DuplicationFailMode {
-	if !p.IsSetFailMode() {
-		return DuplicationEntry_FailMode_DEFAULT
-	}
-	return *p.FailMode
-}
-func (p *DuplicationEntry) IsSetProgress() bool {
-	return p.Progress != nil
-}
-
-func (p *DuplicationEntry) IsSetFailMode() bool {
-	return p.FailMode != nil
-}
-
-func (p *DuplicationEntry) 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 fieldTypeId == thrift.I32 {
-				if err := p.ReadField1(iprot); err != nil {
-					return err
-				}
-			} else {
-				if err := iprot.Skip(fieldTypeId); err != nil {
-					return err
-				}
-			}
-		case 2:
-			if fieldTypeId == thrift.I32 {
-				if err := p.ReadField2(iprot); err != nil {
-					return err
-				}
-			} else {
-				if err := iprot.Skip(fieldTypeId); err != nil {
-					return err
-				}
-			}
-		case 3:
-			if fieldTypeId == thrift.STRING {
-				if err := p.ReadField3(iprot); err != nil {
-					return err
-				}
-			} else {
-				if err := iprot.Skip(fieldTypeId); err != nil {
-					return err
-				}
-			}
-		case 4:
-			if fieldTypeId == thrift.I64 {
-				if err := p.ReadField4(iprot); err != nil {
-					return err
-				}
-			} else {
-				if err := iprot.Skip(fieldTypeId); err != nil {
-					return err
-				}
-			}
-		case 5:
-			if fieldTypeId == thrift.MAP {
-				if err := p.ReadField5(iprot); err != nil {
-					return err
-				}
-			} else {
-				if err := iprot.Skip(fieldTypeId); err != nil {
-					return err
-				}
-			}
-		case 7:
-			if fieldTypeId == thrift.I32 {
-				if err := p.ReadField7(iprot); err != nil {
-					return err
-				}
-			} else {
-				if err := iprot.Skip(fieldTypeId); 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 *DuplicationEntry) ReadField1(iprot thrift.TProtocol) error {
-	if v, err := iprot.ReadI32(); err != nil {
-		return thrift.PrependError("error reading field 1: ", err)
-	} else {
-		p.Dupid = v
-	}
-	return nil
-}
-
-func (p *DuplicationEntry) ReadField2(iprot thrift.TProtocol) error {
-	if v, err := iprot.ReadI32(); err != nil {
-		return thrift.PrependError("error reading field 2: ", err)
-	} else {
-		temp := DuplicationStatus(v)
-		p.Status = temp
-	}
-	return nil
-}
-
-func (p *DuplicationEntry) ReadField3(iprot thrift.TProtocol) error {
-	if v, err := iprot.ReadString(); err != nil {
-		return thrift.PrependError("error reading field 3: ", err)
-	} else {
-		p.Remote = v
-	}
-	return nil
-}
-
-func (p *DuplicationEntry) ReadField4(iprot thrift.TProtocol) error {
-	if v, err := iprot.ReadI64(); err != nil {
-		return thrift.PrependError("error reading field 4: ", err)
-	} else {
-		p.CreateTs = v
-	}
-	return nil
-}
-
-func (p *DuplicationEntry) ReadField5(iprot thrift.TProtocol) error {
-	_, _, size, err := iprot.ReadMapBegin()
-	if err != nil {
-		return thrift.PrependError("error reading map begin: ", err)
-	}
-	tMap := make(map[int32]int64, size)
-	p.Progress = tMap
-	for i := 0; i < size; i++ {
-		var _key11 int32
-		if v, err := iprot.ReadI32(); err != nil {
-			return thrift.PrependError("error reading field 0: ", err)
-		} else {
-			_key11 = v
-		}
-		var _val12 int64
-		if v, err := iprot.ReadI64(); err != nil {
-			return thrift.PrependError("error reading field 0: ", err)
-		} else {
-			_val12 = v
-		}
-		p.Progress[_key11] = _val12
-	}
-	if err := iprot.ReadMapEnd(); err != nil {
-		return thrift.PrependError("error reading map end: ", err)
-	}
-	return nil
-}
-
-func (p *DuplicationEntry) ReadField7(iprot thrift.TProtocol) error {
-	if v, err := iprot.ReadI32(); err != nil {
-		return thrift.PrependError("error reading field 7: ", err)
-	} else {
-		temp := DuplicationFailMode(v)
-		p.FailMode = &temp
-	}
-	return nil
-}
-
-func (p *DuplicationEntry) Write(oprot thrift.TProtocol) error {
-	if err := oprot.WriteStructBegin("duplication_entry"); err != nil {
-		return thrift.PrependError(fmt.Sprintf("%T write struct begin error: ", p), err)
-	}
-	if p != nil {
-		if err := p.writeField1(oprot); err != nil {
-			return err
-		}
-		if err := p.writeField2(oprot); err != nil {
-			return err
-		}
-		if err := p.writeField3(oprot); err != nil {
-			return err
-		}
-		if err := p.writeField4(oprot); err != nil {
-			return err
-		}
-		if err := p.writeField5(oprot); err != nil {
-			return err
-		}
-		if err := p.writeField7(oprot); err != nil {
-			return err
-		}
-	}
-	if err := oprot.WriteFieldStop(); err != nil {
-		return thrift.PrependError("write field stop error: ", err)
-	}
-	if err := oprot.WriteStructEnd(); err != nil {
-		return thrift.PrependError("write struct stop error: ", err)
-	}
-	return nil
-}
-
-func (p *DuplicationEntry) writeField1(oprot thrift.TProtocol) (err error) {
-	if err := oprot.WriteFieldBegin("dupid", thrift.I32, 1); err != nil {
-		return thrift.PrependError(fmt.Sprintf("%T write field begin error 1:dupid: ", p), err)
-	}
-	if err := oprot.WriteI32(int32(p.Dupid)); err != nil {
-		return thrift.PrependError(fmt.Sprintf("%T.dupid (1) field write error: ", p), err)
-	}
-	if err := oprot.WriteFieldEnd(); err != nil {
-		return thrift.PrependError(fmt.Sprintf("%T write field end error 1:dupid: ", p), err)
-	}
-	return err
-}
-
-func (p *DuplicationEntry) writeField2(oprot thrift.TProtocol) (err error) {
-	if err := oprot.WriteFieldBegin("status", thrift.I32, 2); err != nil {
-		return thrift.PrependError(fmt.Sprintf("%T write field begin error 2:status: ", p), err)
-	}
-	if err := oprot.WriteI32(int32(p.Status)); err != nil {
-		return thrift.PrependError(fmt.Sprintf("%T.status (2) field write error: ", p), err)
-	}
-	if err := oprot.WriteFieldEnd(); err != nil {
-		return thrift.PrependError(fmt.Sprintf("%T write field end error 2:status: ", p), err)
-	}
-	return err
-}
-
-func (p *DuplicationEntry) writeField3(oprot thrift.TProtocol) (err error) {
-	if err := oprot.WriteFieldBegin("remote", thrift.STRING, 3); err != nil {
-		return thrift.PrependError(fmt.Sprintf("%T write field begin error 3:remote: ", p), err)
-	}
-	if err := oprot.WriteString(string(p.Remote)); err != nil {
-		return thrift.PrependError(fmt.Sprintf("%T.remote (3) field write error: ", p), err)
-	}
-	if err := oprot.WriteFieldEnd(); err != nil {
-		return thrift.PrependError(fmt.Sprintf("%T write field end error 3:remote: ", p), err)
-	}
-	return err
-}
-
-func (p *DuplicationEntry) writeField4(oprot thrift.TProtocol) (err error) {
-	if err := oprot.WriteFieldBegin("create_ts", thrift.I64, 4); err != nil {
-		return thrift.PrependError(fmt.Sprintf("%T write field begin error 4:create_ts: ", p), err)
-	}
-	if err := oprot.WriteI64(int64(p.CreateTs)); err != nil {
-		return thrift.PrependError(fmt.Sprintf("%T.create_ts (4) field write error: ", p), err)
-	}
-	if err := oprot.WriteFieldEnd(); err != nil {
-		return thrift.PrependError(fmt.Sprintf("%T write field end error 4:create_ts: ", p), err)
-	}
-	return err
-}
-
-func (p *DuplicationEntry) writeField5(oprot thrift.TProtocol) (err error) {
-	if p.IsSetProgress() {
-		if err := oprot.WriteFieldBegin("progress", thrift.MAP, 5); err != nil {
-			return thrift.PrependError(fmt.Sprintf("%T write field begin error 5:progress: ", p), err)
-		}
-		if err := oprot.WriteMapBegin(thrift.I32, thrift.I64, len(p.Progress)); err != nil {
-			return thrift.PrependError("error writing map begin: ", err)
-		}
-		for k, v := range p.Progress {
-			if err := oprot.WriteI32(int32(k)); err != nil {
-				return thrift.PrependError(fmt.Sprintf("%T. (0) field write error: ", p), err)
-			}
-			if err := oprot.WriteI64(int64(v)); err != nil {
-				return thrift.PrependError(fmt.Sprintf("%T. (0) field write error: ", p), err)
-			}
-		}
-		if err := oprot.WriteMapEnd(); err != nil {
-			return thrift.PrependError("error writing map end: ", err)
-		}
-		if err := oprot.WriteFieldEnd(); err != nil {
-			return thrift.PrependError(fmt.Sprintf("%T write field end error 5:progress: ", p), err)
-		}
-	}
-	return err
-}
-
-func (p *DuplicationEntry) writeField7(oprot thrift.TProtocol) (err error) {
-	if p.IsSetFailMode() {
-		if err := oprot.WriteFieldBegin("fail_mode", thrift.I32, 7); err != nil {
-			return thrift.PrependError(fmt.Sprintf("%T write field begin error 7:fail_mode: ", p), err)
-		}
-		if err := oprot.WriteI32(int32(*p.FailMode)); err != nil {
-			return thrift.PrependError(fmt.Sprintf("%T.fail_mode (7) field write error: ", p), err)
-		}
-		if err := oprot.WriteFieldEnd(); err != nil {
-			return thrift.PrependError(fmt.Sprintf("%T write field end error 7:fail_mode: ", p), err)
-		}
-	}
-	return err
-}
-
-func (p *DuplicationEntry) String() string {
-	if p == nil {
-		return "<nil>"
-	}
-	return fmt.Sprintf("DuplicationEntry(%+v)", *p)
-}
-
-// Attributes:
-//  - AppName
-type DuplicationQueryRequest struct {
-	AppName string `thrift:"app_name,1" db:"app_name" json:"app_name"`
-}
-
-func NewDuplicationQueryRequest() *DuplicationQueryRequest {
-	return &DuplicationQueryRequest{}
-}
-
-func (p *DuplicationQueryRequest) GetAppName() string {
-	return p.AppName
-}
-func (p *DuplicationQueryRequest) 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 fieldTypeId == thrift.STRING {
-				if err := p.ReadField1(iprot); err != nil {
-					return err
-				}
-			} else {
-				if err := iprot.Skip(fieldTypeId); 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 *DuplicationQueryRequest) ReadField1(iprot thrift.TProtocol) error {
-	if v, err := iprot.ReadString(); err != nil {
-		return thrift.PrependError("error reading field 1: ", err)
-	} else {
-		p.AppName = v
-	}
-	return nil
-}
-
-func (p *DuplicationQueryRequest) Write(oprot thrift.TProtocol) error {
-	if err := oprot.WriteStructBegin("duplication_query_request"); 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 *DuplicationQueryRequest) writeField1(oprot thrift.TProtocol) (err error) {
-	if err := oprot.WriteFieldBegin("app_name", thrift.STRING, 1); err != nil {
-		return thrift.PrependError(fmt.Sprintf("%T write field begin error 1:app_name: ", p), err)
-	}
-	if err := oprot.WriteString(string(p.AppName)); err != nil {
-		return thrift.PrependError(fmt.Sprintf("%T.app_name (1) field write error: ", p), err)
-	}
-	if err := oprot.WriteFieldEnd(); err != nil {
-		return thrift.PrependError(fmt.Sprintf("%T write field end error 1:app_name: ", p), err)
-	}
-	return err
-}
-
-func (p *DuplicationQueryRequest) String() string {
-	if p == nil {
-		return "<nil>"
-	}
-	return fmt.Sprintf("DuplicationQueryRequest(%+v)", *p)
-}
-
-// Attributes:
-//  - Err
-//  - Appid
-//  - EntryList
-type DuplicationQueryResponse struct {
-	Err *base.ErrorCode `thrift:"err,1" db:"err" json:"err"`
-	// unused field # 2
-	Appid     int32               `thrift:"appid,3" db:"appid" json:"appid"`
-	EntryList []*DuplicationEntry `thrift:"entry_list,4" db:"entry_list" json:"entry_list"`
-}
-
-func NewDuplicationQueryResponse() *DuplicationQueryResponse {
-	return &DuplicationQueryResponse{}
-}
-
-var DuplicationQueryResponse_Err_DEFAULT *base.ErrorCode
-
-func (p *DuplicationQueryResponse) GetErr() *base.ErrorCode {
-	if !p.IsSetErr() {
-		return DuplicationQueryResponse_Err_DEFAULT
-	}
-	return p.Err
-}
-
-func (p *DuplicationQueryResponse) GetAppid() int32 {
-	return p.Appid
-}
-
-func (p *DuplicationQueryResponse) GetEntryList() []*DuplicationEntry {
-	return p.EntryList
-}
-func (p *DuplicationQueryResponse) IsSetErr() bool {
-	return p.Err != nil
-}
-
-func (p *DuplicationQueryResponse) 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 fieldTypeId == thrift.STRUCT {
-				if err := p.ReadField1(iprot); err != nil {
-					return err
-				}
-			} else {
-				if err := iprot.Skip(fieldTypeId); err != nil {
-					return err
-				}
-			}
-		case 3:
-			if fieldTypeId == thrift.I32 {
-				if err := p.ReadField3(iprot); err != nil {
-					return err
-				}
-			} else {
-				if err := iprot.Skip(fieldTypeId); err != nil {
-					return err
-				}
-			}
-		case 4:
-			if fieldTypeId == thrift.LIST {
-				if err := p.ReadField4(iprot); err != nil {
-					return err
-				}
-			} else {
-				if err := iprot.Skip(fieldTypeId); 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 *DuplicationQueryResponse) ReadField1(iprot thrift.TProtocol) error {
-	p.Err = &base.ErrorCode{}
-	if err := p.Err.Read(iprot); err != nil {
-		return thrift.PrependError(fmt.Sprintf("%T error reading struct: ", p.Err), err)
-	}
-	return nil
-}
-
-func (p *DuplicationQueryResponse) ReadField3(iprot thrift.TProtocol) error {
-	if v, err := iprot.ReadI32(); err != nil {
-		return thrift.PrependError("error reading field 3: ", err)
-	} else {
-		p.Appid = v
-	}
-	return nil
-}
-
-func (p *DuplicationQueryResponse) ReadField4(iprot thrift.TProtocol) error {
-	_, size, err := iprot.ReadListBegin()
-	if err != nil {
-		return thrift.PrependError("error reading list begin: ", err)
-	}
-	tSlice := make([]*DuplicationEntry, 0, size)
-	p.EntryList = tSlice
-	for i := 0; i < size; i++ {
-		_elem13 := &DuplicationEntry{}
-		if err := _elem13.Read(iprot); err != nil {
-			return thrift.PrependError(fmt.Sprintf("%T error reading struct: ", _elem13), err)
-		}
-		p.EntryList = append(p.EntryList, _elem13)
-	}
-	if err := iprot.ReadListEnd(); err != nil {
-		return thrift.PrependError("error reading list end: ", err)
-	}
-	return nil
-}
-
-func (p *DuplicationQueryResponse) Write(oprot thrift.TProtocol) error {
-	if err := oprot.WriteStructBegin("duplication_query_response"); 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.writeField3(oprot); err != nil {
-			return err
-		}
-		if err := p.writeField4(oprot); err != nil {
-			return err
-		}
-	}
-	if err := oprot.WriteFieldStop(); err != nil {
-		return thrift.PrependError("write field stop error: ", err)
-	}
-	if err := oprot.WriteStructEnd(); err != nil {
-		return thrift.PrependError("write struct stop error: ", err)
-	}
-	return nil
-}
-
-func (p *DuplicationQueryResponse) writeField1(oprot thrift.TProtocol) (err error) {
-	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 *DuplicationQueryResponse) writeField3(oprot thrift.TProtocol) (err error) {
-	if err := oprot.WriteFieldBegin("appid", thrift.I32, 3); err != nil {
-		return thrift.PrependError(fmt.Sprintf("%T write field begin error 3:appid: ", p), err)
-	}
-	if err := oprot.WriteI32(int32(p.Appid)); err != nil {
-		return thrift.PrependError(fmt.Sprintf("%T.appid (3) field write error: ", p), err)
-	}
-	if err := oprot.WriteFieldEnd(); err != nil {
-		return thrift.PrependError(fmt.Sprintf("%T write field end error 3:appid: ", p), err)
-	}
-	return err
-}
-
-func (p *DuplicationQueryResponse) writeField4(oprot thrift.TProtocol) (err error) {
-	if err := oprot.WriteFieldBegin("entry_list", thrift.LIST, 4); err != nil {
-		return thrift.PrependError(fmt.Sprintf("%T write field begin error 4:entry_list: ", p), err)
-	}
-	if err := oprot.WriteListBegin(thrift.STRUCT, len(p.EntryList)); err != nil {
-		return thrift.PrependError("error writing list begin: ", err)
-	}
-	for _, v := range p.EntryList {
-		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 4:entry_list: ", p), err)
-	}
-	return err
-}
-
-func (p *DuplicationQueryResponse) String() string {
-	if p == nil {
-		return "<nil>"
-	}
-	return fmt.Sprintf("DuplicationQueryResponse(%+v)", *p)
-}
-
-// Attributes:
-//  - PolicyName
-//  - BackupProviderType
-//  - BackupIntervalSeconds
-//  - AppIds
-//  - BackupHistoryCountToKeep
-//  - StartTime
-//  - IsDisable
-type PolicyEntry struct {
-	PolicyName               string  `thrift:"policy_name,1" db:"policy_name" json:"policy_name"`
-	BackupProviderType       string  `thrift:"backup_provider_type,2" db:"backup_provider_type" json:"backup_provider_type"`
-	BackupIntervalSeconds    string  `thrift:"backup_interval_seconds,3" db:"backup_interval_seconds" json:"backup_interval_seconds"`
-	AppIds                   []int32 `thrift:"app_ids,4" db:"app_ids" json:"app_ids"`
-	BackupHistoryCountToKeep int32   `thrift:"backup_history_count_to_keep,5" db:"backup_history_count_to_keep" json:"backup_history_count_to_keep"`
-	StartTime                string  `thrift:"start_time,6" db:"start_time" json:"start_time"`
-	IsDisable                bool    `thrift:"is_disable,7" db:"is_disable" json:"is_disable"`
-}
-
-func NewPolicyEntry() *PolicyEntry {
-	return &PolicyEntry{}
-}
-
-func (p *PolicyEntry) GetPolicyName() string {
-	return p.PolicyName
-}
-
-func (p *PolicyEntry) GetBackupProviderType() string {
-	return p.BackupProviderType
-}
-
-func (p *PolicyEntry) GetBackupIntervalSeconds() string {
-	return p.BackupIntervalSeconds
-}
-
-func (p *PolicyEntry) GetAppIds() []int32 {
-	return p.AppIds
-}
-
-func (p *PolicyEntry) GetBackupHistoryCountToKeep() int32 {
-	return p.BackupHistoryCountToKeep
-}
-
-func (p *PolicyEntry) GetStartTime() string {
-	return p.StartTime
-}
-
-func (p *PolicyEntry) GetIsDisable() bool {
-	return p.IsDisable
-}
-func (p *PolicyEntry) 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 fieldTypeId == thrift.STRING {
-				if err := p.ReadField1(iprot); err != nil {
-					return err
-				}
-			} else {
-				if err := iprot.Skip(fieldTypeId); err != nil {
-					return err
-				}
-			}
-		case 2:
-			if fieldTypeId == thrift.STRING {
-				if err := p.ReadField2(iprot); err != nil {
-					return err
-				}
-			} else {
-				if err := iprot.Skip(fieldTypeId); err != nil {
-					return err
-				}
-			}
-		case 3:
-			if fieldTypeId == thrift.STRING {
-				if err := p.ReadField3(iprot); err != nil {
-					return err
-				}
-			} else {
-				if err := iprot.Skip(fieldTypeId); err != nil {
-					return err
-				}
-			}
-		case 4:
-			if fieldTypeId == thrift.SET {
-				if err := p.ReadField4(iprot); err != nil {
-					return err
-				}
-			} else {
-				if err := iprot.Skip(fieldTypeId); err != nil {
-					return err
-				}
-			}
-		case 5:
-			if fieldTypeId == thrift.I32 {
-				if err := p.ReadField5(iprot); err != nil {
-					return err
-				}
-			} else {
-				if err := iprot.Skip(fieldTypeId); err != nil {
-					return err
-				}
-			}
-		case 6:
-			if fieldTypeId == thrift.STRING {
-				if err := p.ReadField6(iprot); err != nil {
-					return err
-				}
-			} else {
-				if err := iprot.Skip(fieldTypeId); err != nil {
-					return err
-				}
-			}
-		case 7:
-			if fieldTypeId == thrift.BOOL {
-				if err := p.ReadField7(iprot); err != nil {
-					return err
-				}
-			} else {
-				if err := iprot.Skip(fieldTypeId); 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 *PolicyEntry) ReadField1(iprot thrift.TProtocol) error {
-	if v, err := iprot.ReadString(); err != nil {
-		return thrift.PrependError("error reading field 1: ", err)
-	} else {
-		p.PolicyName = v
-	}
-	return nil
-}
-
-func (p *PolicyEntry) ReadField2(iprot thrift.TProtocol) error {
-	if v, err := iprot.ReadString(); err != nil {
-		return thrift.PrependError("error reading field 2: ", err)
-	} else {
-		p.BackupProviderType = v
-	}
-	return nil
-}
-
-func (p *PolicyEntry) ReadField3(iprot thrift.TProtocol) error {
-	if v, err := iprot.ReadString(); err != nil {
-		return thrift.PrependError("error reading field 3: ", err)
-	} else {
-		p.BackupIntervalSeconds = v
-	}
-	return nil
-}
-
-func (p *PolicyEntry) ReadField4(iprot thrift.TProtocol) error {
-	_, size, err := iprot.ReadSetBegin()
-	if err != nil {
-		return thrift.PrependError("error reading set begin: ", err)
-	}
-	tSet := make([]int32, 0, size)
-	p.AppIds = tSet
-	for i := 0; i < size; i++ {
-		var _elem14 int32
-		if v, err := iprot.ReadI32(); err != nil {
-			return thrift.PrependError("error reading field 0: ", err)
-		} else {
-			_elem14 = v
-		}
-		p.AppIds = append(p.AppIds, _elem14)
-	}
-	if err := iprot.ReadSetEnd(); err != nil {
-		return thrift.PrependError("error reading set end: ", err)
-	}
-	return nil
-}
-
-func (p *PolicyEntry) ReadField5(iprot thrift.TProtocol) error {
-	if v, err := iprot.ReadI32(); err != nil {
-		return thrift.PrependError("error reading field 5: ", err)
-	} else {
-		p.BackupHistoryCountToKeep = v
-	}
-	return nil
-}
-
-func (p *PolicyEntry) ReadField6(iprot thrift.TProtocol) error {
-	if v, err := iprot.ReadString(); err != nil {
-		return thrift.PrependError("error reading field 6: ", err)
-	} else {
-		p.StartTime = v
-	}
-	return nil
-}
-
-func (p *PolicyEntry) ReadField7(iprot thrift.TProtocol) error {
-	if v, err := iprot.ReadBool(); err != nil {
-		return thrift.PrependError("error reading field 7: ", err)
-	} else {
-		p.IsDisable = v
-	}
-	return nil
-}
-
-func (p *PolicyEntry) Write(oprot thrift.TProtocol) error {
-	if err := oprot.WriteStructBegin("policy_entry"); err != nil {
-		return thrift.PrependError(fmt.Sprintf("%T write struct begin error: ", p), err)
-	}
-	if p != nil {
-		if err := p.writeField1(oprot); err != nil {
-			return err
-		}
-		if err := p.writeField2(oprot); err != nil {
-			return err
-		}
-		if err := p.writeField3(oprot); err != nil {
-			return err
-		}
-		if err := p.writeField4(oprot); err != nil {
-			return err
-		}
-		if err := p.writeField5(oprot); err != nil {
-			return err
-		}
-		if err := p.writeField6(oprot); err != nil {
-			return err
-		}
-		if err := p.writeField7(oprot); err != nil {
-			return err
-		}
-	}
-	if err := oprot.WriteFieldStop(); err != nil {
-		return thrift.PrependError("write field stop error: ", err)
-	}
-	if err := oprot.WriteStructEnd(); err != nil {
-		return thrift.PrependError("write struct stop error: ", err)
-	}
-	return nil
-}
-
-func (p *PolicyEntry) writeField1(oprot thrift.TProtocol) (err error) {
-	if err := oprot.WriteFieldBegin("policy_name", thrift.STRING, 1); err != nil {
-		return thrift.PrependError(fmt.Sprintf("%T write field begin error 1:policy_name: ", p), err)
-	}
-	if err := oprot.WriteString(string(p.PolicyName)); err != nil {
-		return thrift.PrependError(fmt.Sprintf("%T.policy_name (1) field write error: ", p), err)
-	}
-	if err := oprot.WriteFieldEnd(); err != nil {
-		return thrift.PrependError(fmt.Sprintf("%T write field end error 1:policy_name: ", p), err)
-	}
-	return err
-}
-
-func (p *PolicyEntry) writeField2(oprot thrift.TProtocol) (err error) {
-	if err := oprot.WriteFieldBegin("backup_provider_type", thrift.STRING, 2); err != nil {
-		return thrift.PrependError(fmt.Sprintf("%T write field begin error 2:backup_provider_type: ", p), err)
-	}
-	if err := oprot.WriteString(string(p.BackupProviderType)); err != nil {
-		return thrift.PrependError(fmt.Sprintf("%T.backup_provider_type (2) field write error: ", p), err)
-	}
-	if err := oprot.WriteFieldEnd(); err != nil {
-		return thrift.PrependError(fmt.Sprintf("%T write field end error 2:backup_provider_type: ", p), err)
-	}
-	return err
-}
-
-func (p *PolicyEntry) writeField3(oprot thrift.TProtocol) (err error) {
-	if err := oprot.WriteFieldBegin("backup_interval_seconds", thrift.STRING, 3); err != nil {
-		return thrift.PrependError(fmt.Sprintf("%T write field begin error 3:backup_interval_seconds: ", p), err)
-	}
-	if err := oprot.WriteString(string(p.BackupIntervalSeconds)); err != nil {
-		return thrift.PrependError(fmt.Sprintf("%T.backup_interval_seconds (3) field write error: ", p), err)
-	}
-	if err := oprot.WriteFieldEnd(); err != nil {
-		return thrift.PrependError(fmt.Sprintf("%T write field end error 3:backup_interval_seconds: ", p), err)
-	}
-	return err
-}
-
-func (p *PolicyEntry) writeField4(oprot thrift.TProtocol) (err error) {
-	if err := oprot.WriteFieldBegin("app_ids", thrift.SET, 4); err != nil {
-		return thrift.PrependError(fmt.Sprintf("%T write field begin error 4:app_ids: ", p), err)
-	}
-	if err := oprot.WriteSetBegin(thrift.I32, len(p.AppIds)); err != nil {
-		return thrift.PrependError("error writing set begin: ", err)
-	}
-	for i := 0; i < len(p.AppIds); i++ {
-		for j := i + 1; j < len(p.AppIds); j++ {
-			if reflect.DeepEqual(p.AppIds[i], p.AppIds[j]) {
-				return thrift.PrependError("", fmt.Errorf("%T error writing set field: slice is not unique", p.AppIds[i]))
-			}
-		}
-	}
-	for _, v := range p.AppIds {
-		if err := oprot.WriteI32(int32(v)); err != nil {
-			return thrift.PrependError(fmt.Sprintf("%T. (0) field write error: ", p), err)
-		}
-	}
-	if err := oprot.WriteSetEnd(); err != nil {
-		return thrift.PrependError("error writing set end: ", err)
-	}
-	if err := oprot.WriteFieldEnd(); err != nil {
-		return thrift.PrependError(fmt.Sprintf("%T write field end error 4:app_ids: ", p), err)
-	}
-	return err
-}
-
-func (p *PolicyEntry) writeField5(oprot thrift.TProtocol) (err error) {
-	if err := oprot.WriteFieldBegin("backup_history_count_to_keep", thrift.I32, 5); err != nil {
-		return thrift.PrependError(fmt.Sprintf("%T write field begin error 5:backup_history_count_to_keep: ", p), err)
-	}
-	if err := oprot.WriteI32(int32(p.BackupHistoryCountToKeep)); err != nil {
-		return thrift.PrependError(fmt.Sprintf("%T.backup_history_count_to_keep (5) field write error: ", p), err)
-	}
-	if err := oprot.WriteFieldEnd(); err != nil {
-		return thrift.PrependError(fmt.Sprintf("%T write field end error 5:backup_history_count_to_keep: ", p), err)
-	}
-	return err
-}
-
-func (p *PolicyEntry) writeField6(oprot thrift.TProtocol) (err error) {
-	if err := oprot.WriteFieldBegin("start_time", thrift.STRING, 6); err != nil {
-		return thrift.PrependError(fmt.Sprintf("%T write field begin error 6:start_time: ", p), err)
-	}
-	if err := oprot.WriteString(string(p.StartTime)); err != nil {
-		return thrift.PrependError(fmt.Sprintf("%T.start_time (6) field write error: ", p), err)
-	}
-	if err := oprot.WriteFieldEnd(); err != nil {
-		return thrift.PrependError(fmt.Sprintf("%T write field end error 6:start_time: ", p), err)
-	}
-	return err
-}
-
-func (p *PolicyEntry) writeField7(oprot thrift.TProtocol) (err error) {
-	if err := oprot.WriteFieldBegin("is_disable", thrift.BOOL, 7); err != nil {
-		return thrift.PrependError(fmt.Sprintf("%T write field begin error 7:is_disable: ", p), err)
-	}
-	if err := oprot.WriteBool(bool(p.IsDisable)); err != nil {
-		return thrift.PrependError(fmt.Sprintf("%T.is_disable (7) field write error: ", p), err)
-	}
-	if err := oprot.WriteFieldEnd(); err != nil {
-		return thrift.PrependError(fmt.Sprintf("%T write field end error 7:is_disable: ", p), err)
-	}
-	return err
-}
-
-func (p *PolicyEntry) String() string {
-	if p == nil {
-		return "<nil>"
-	}
-	return fmt.Sprintf("PolicyEntry(%+v)", *p)
-}
-
-// Attributes:
-//  - BackupID
-//  - StartTimeMs
-//  - EndTimeMs
-//  - AppIds
-type BackupEntry struct {
-	BackupID    int64   `thrift:"backup_id,1" db:"backup_id" json:"backup_id"`
-	StartTimeMs int64   `thrift:"start_time_ms,2" db:"start_time_ms" json:"start_time_ms"`
-	EndTimeMs   int64   `thrift:"end_time_ms,3" db:"end_time_ms" json:"end_time_ms"`
-	AppIds      []int32 `thrift:"app_ids,4" db:"app_ids" json:"app_ids"`
-}
-
-func NewBackupEntry() *BackupEntry {
-	return &BackupEntry{}
-}
-
-func (p *BackupEntry) GetBackupID() int64 {
-	return p.BackupID
-}
-
-func (p *BackupEntry) GetStartTimeMs() int64 {
-	return p.StartTimeMs
-}
-
-func (p *BackupEntry) GetEndTimeMs() int64 {
-	return p.EndTimeMs
-}
-
-func (p *BackupEntry) GetAppIds() []int32 {
-	return p.AppIds
-}
-func (p *BackupEntry) 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 fieldTypeId == thrift.I64 {
-				if err := p.ReadField1(iprot); err != nil {
-					return err
-				}
-			} else {
-				if err := iprot.Skip(fieldTypeId); err != nil {
-					return err
-				}
-			}
-		case 2:
-			if fieldTypeId == thrift.I64 {
-				if err := p.ReadField2(iprot); err != nil {
-					return err
-				}
-			} else {
-				if err := iprot.Skip(fieldTypeId); err != nil {
-					return err
-				}
-			}
-		case 3:
-			if fieldTypeId == thrift.I64 {
-				if err := p.ReadField3(iprot); err != nil {
-					return err
-				}
-			} else {
-				if err := iprot.Skip(fieldTypeId); err != nil {
-					return err
-				}
-			}
-		case 4:
-			if fieldTypeId == thrift.SET {
-				if err := p.ReadField4(iprot); err != nil {
-					return err
-				}
-			} else {
-				if err := iprot.Skip(fieldTypeId); 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 *BackupEntry) ReadField1(iprot thrift.TProtocol) error {
-	if v, err := iprot.ReadI64(); err != nil {
-		return thrift.PrependError("error reading field 1: ", err)
-	} else {
-		p.BackupID = v
-	}
-	return nil
-}
-
-func (p *BackupEntry) ReadField2(iprot thrift.TProtocol) error {
-	if v, err := iprot.ReadI64(); err != nil {
-		return thrift.PrependError("error reading field 2: ", err)
-	} else {
-		p.StartTimeMs = v
-	}
-	return nil
-}
-
-func (p *BackupEntry) ReadField3(iprot thrift.TProtocol) error {
-	if v, err := iprot.ReadI64(); err != nil {
-		return thrift.PrependError("error reading field 3: ", err)
-	} else {
-		p.EndTimeMs = v
-	}
-	return nil
-}
-
-func (p *BackupEntry) ReadField4(iprot thrift.TProtocol) error {
-	_, size, err := iprot.ReadSetBegin()
-	if err != nil {
-		return thrift.PrependError("error reading set begin: ", err)
-	}
-	tSet := make([]int32, 0, size)
-	p.AppIds = tSet
-	for i := 0; i < size; i++ {
-		var _elem15 int32
-		if v, err := iprot.ReadI32(); err != nil {
-			return thrift.PrependError("error reading field 0: ", err)
-		} else {
-			_elem15 = v
-		}
-		p.AppIds = append(p.AppIds, _elem15)
-	}
-	if err := iprot.ReadSetEnd(); err != nil {
-		return thrift.PrependError("error reading set end: ", err)
-	}
-	return nil
-}
-
-func (p *BackupEntry) Write(oprot thrift.TProtocol) error {
-	if err := oprot.WriteStructBegin("backup_entry"); 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 := 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 *BackupEntry) writeField1(oprot thrift.TProtocol) (err error) {
-	if err := oprot.WriteFieldBegin("backup_id", thrift.I64, 1); err != nil {
-		return thrift.PrependError(fmt.Sprintf("%T write field begin error 1:backup_id: ", p), err)
-	}
-	if err := oprot.WriteI64(int64(p.BackupID)); err != nil {
-		return thrift.PrependError(fmt.Sprintf("%T.backup_id (1) field write error: ", p), err)
-	}
-	if err := oprot.WriteFieldEnd(); err != nil {
-		return thrift.PrependError(fmt.Sprintf("%T write field end error 1:backup_id: ", p), err)
-	}
-	return err
-}
-
-func (p *BackupEntry) writeField2(oprot thrift.TProtocol) (err error) {
-	if err := oprot.WriteFieldBegin("start_time_ms", thrift.I64, 2); err != nil {
-		return thrift.PrependError(fmt.Sprintf("%T write field begin error 2:start_time_ms: ", p), err)
-	}
-	if err := oprot.WriteI64(int64(p.StartTimeMs)); err != nil {
-		return thrift.PrependError(fmt.Sprintf("%T.start_time_ms (2) field write error: ", p), err)
-	}
-	if err := oprot.WriteFieldEnd(); err != nil {
-		return thrift.PrependError(fmt.Sprintf("%T write field end error 2:start_time_ms: ", p), err)
-	}
-	return err
-}
-
-func (p *BackupEntry) writeField3(oprot thrift.TProtocol) (err error) {
-	if err := oprot.WriteFieldBegin("end_time_ms", thrift.I64, 3); err != nil {
-		return thrift.PrependError(fmt.Sprintf("%T write field begin error 3:end_time_ms: ", p), err)
-	}
-	if err := oprot.WriteI64(int64(p.EndTimeMs)); err != nil {
-		return thrift.PrependError(fmt.Sprintf("%T.end_time_ms (3) field write error: ", p), err)
-	}
-	if err := oprot.WriteFieldEnd(); err != nil {
-		return thrift.PrependError(fmt.Sprintf("%T write field end error 3:end_time_ms: ", p), err)
-	}
-	return err
-}
-
-func (p *BackupEntry) writeField4(oprot thrift.TProtocol) (err error) {
-	if err := oprot.WriteFieldBegin("app_ids", thrift.SET, 4); err != nil {
-		return thrift.PrependError(fmt.Sprintf("%T write field begin error 4:app_ids: ", p), err)
-	}
-	if err := oprot.WriteSetBegin(thrift.I32, len(p.AppIds)); err != nil {
-		return thrift.PrependError("error writing set begin: ", err)
-	}
-	for i := 0; i < len(p.AppIds); i++ {
-		for j := i + 1; j < len(p.AppIds); j++ {
-			if reflect.DeepEqual(p.AppIds[i], p.AppIds[j]) {
-				return thrift.PrependError("", fmt.Errorf("%T error writing set field: slice is not unique", p.AppIds[i]))
-			}
-		}
-	}
-	for _, v := range p.AppIds {
-		if err := oprot.WriteI32(int32(v)); err != nil {
-			return thrift.PrependError(fmt.Sprintf("%T. (0) field write error: ", p), err)
-		}
-	}
-	if err := oprot.WriteSetEnd(); err != nil {
-		return thrift.PrependError("error writing set end: ", err)
-	}
-	if err := oprot.WriteFieldEnd(); err != nil {
-		return thrift.PrependError(fmt.Sprintf("%T write field end error 4:app_ids: ", p), err)
-	}
-	return err
-}
-
-func (p *BackupEntry) String() string {
-	if p == nil {
-		return "<nil>"
-	}
-	return fmt.Sprintf("BackupEntry(%+v)", *p)
-}
-
-// Attributes:
-//  - PolicyNames
-//  - BackupInfoCount
-type QueryBackupPolicyRequest struct {
-	PolicyNames     []string `thrift:"policy_names,1" db:"policy_names" json:"policy_names"`
-	BackupInfoCount int32    `thrift:"backup_info_count,2" db:"backup_info_count" json:"backup_info_count"`
-}
-
-func NewQueryBackupPolicyRequest() *QueryBackupPolicyRequest {
-	return &QueryBackupPolicyRequest{}
-}
-
-func (p *QueryBackupPolicyRequest) GetPolicyNames() []string {
-	return p.PolicyNames
-}
-
-func (p *QueryBackupPolicyRequest) GetBackupInfoCount() int32 {
-	return p.BackupInfoCount
-}
-func (p *QueryBackupPolicyRequest) 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 fieldTypeId == thrift.LIST {
-				if err := p.ReadField1(iprot); err != nil {
-					return err
-				}
-			} else {
-				if err := iprot.Skip(fieldTypeId); err != nil {
-					return err
-				}
-			}
-		case 2:
-			if fieldTypeId == thrift.I32 {
-				if err := p.ReadField2(iprot); err != nil {
-					return err
-				}
-			} else {
-				if err := iprot.Skip(fieldTypeId); 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 *QueryBackupPolicyRequest) ReadField1(iprot thrift.TProtocol) error {
-	_, size, err := iprot.ReadListBegin()
-	if err != nil {
-		return thrift.PrependError("error reading list begin: ", err)
-	}
-	tSlice := make([]string, 0, size)
-	p.PolicyNames = tSlice
-	for i := 0; i < size; i++ {
-		var _elem16 string
-		if v, err := iprot.ReadString(); err != nil {
-			return thrift.PrependError("error reading field 0: ", err)
-		} else {
-			_elem16 = v
-		}
-		p.PolicyNames = append(p.PolicyNames, _elem16)
-	}
-	if err := iprot.ReadListEnd(); err != nil {
-		return thrift.PrependError("error reading list end: ", err)
-	}
-	return nil
-}
-
-func (p *QueryBackupPolicyRequest) ReadField2(iprot thrift.TProtocol) error {
-	if v, err := iprot.ReadI32(); err != nil {
-		return thrift.PrependError("error reading field 2: ", err)
-	} else {
-		p.BackupInfoCount = v
-	}
-	return nil
-}
-
-func (p *QueryBackupPolicyRequest) Write(oprot thrift.TProtocol) error {
-	if err := oprot.WriteStructBegin("query_backup_policy_request"); 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 *QueryBackupPolicyRequest) writeField1(oprot thrift.TProtocol) (err error) {
-	if err := oprot.WriteFieldBegin("policy_names", thrift.LIST, 1); err != nil {
-		return thrift.PrependError(fmt.Sprintf("%T write field begin error 1:policy_names: ", p), err)
-	}
-	if err := oprot.WriteListBegin(thrift.STRING, len(p.PolicyNames)); err != nil {
-		return thrift.PrependError("error writing list begin: ", err)
-	}
-	for _, v := range p.PolicyNames {
-		if err := oprot.WriteString(string(v)); err != nil {
-			return thrift.PrependError(fmt.Sprintf("%T. (0) field write error: ", p), err)
-		}
-	}
-	if err := oprot.WriteListEnd(); err != nil {
-		return thrift.PrependError("error writing list end: ", err)
-	}
-	if err := oprot.WriteFieldEnd(); err != nil {
-		return thrift.PrependError(fmt.Sprintf("%T write field end error 1:policy_names: ", p), err)
-	}
-	return err
-}
-
-func (p *QueryBackupPolicyRequest) writeField2(oprot thrift.TProtocol) (err error) {
-	if err := oprot.WriteFieldBegin("backup_info_count", thrift.I32, 2); err != nil {
-		return thrift.PrependError(fmt.Sprintf("%T write field begin error 2:backup_info_count: ", p), err)
-	}
-	if err := oprot.WriteI32(int32(p.BackupInfoCount)); err != nil {
-		return thrift.PrependError(fmt.Sprintf("%T.backup_info_count (2) field write error: ", p), err)
-	}
-	if err := oprot.WriteFieldEnd(); err != nil {
-		return thrift.PrependError(fmt.Sprintf("%T write field end error 2:backup_info_count: ", p), err)
-	}
-	return err
-}
-
-func (p *QueryBackupPolicyRequest) String() string {
-	if p == nil {
-		return "<nil>"
-	}
-	return fmt.Sprintf("QueryBackupPolicyRequest(%+v)", *p)
-}
-
-// Attributes:
-//  - Err
-//  - Policys
-//  - BackupInfos
-//  - HintMsg
-type QueryBackupPolicyResponse struct {
-	Err         *base.ErrorCode  `thrift:"err,1" db:"err" json:"err"`
-	Policys     []*PolicyEntry   `thrift:"policys,2" db:"policys" json:"policys"`
-	BackupInfos [][]*BackupEntry `thrift:"backup_infos,3" db:"backup_infos" json:"backup_infos"`
-	HintMsg     *string          `thrift:"hint_msg,4" db:"hint_msg" json:"hint_msg,omitempty"`
-}
-
-func NewQueryBackupPolicyResponse() *QueryBackupPolicyResponse {
-	return &QueryBackupPolicyResponse{}
-}
-
-var QueryBackupPolicyResponse_Err_DEFAULT *base.ErrorCode
-
-func (p *QueryBackupPolicyResponse) GetErr() *base.ErrorCode {
-	if !p.IsSetErr() {
-		return QueryBackupPolicyResponse_Err_DEFAULT
-	}
-	return p.Err
-}
-
-func (p *QueryBackupPolicyResponse) GetPolicys() []*PolicyEntry {
-	return p.Policys
-}
-
-func (p *QueryBackupPolicyResponse) GetBackupInfos() [][]*BackupEntry {
-	return p.BackupInfos
-}
-
-var QueryBackupPolicyResponse_HintMsg_DEFAULT string
-
-func (p *QueryBackupPolicyResponse) GetHintMsg() string {
-	if !p.IsSetHintMsg() {
-		return QueryBackupPolicyResponse_HintMsg_DEFAULT
-	}
-	return *p.HintMsg
-}
-func (p *QueryBackupPolicyResponse) IsSetErr() bool {
-	return p.Err != nil
-}
-
-func (p *QueryBackupPolicyResponse) IsSetHintMsg() bool {
-	return p.HintMsg != nil
-}
-
-func (p *QueryBackupPolicyResponse) 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 fieldTypeId == thrift.STRUCT {
-				if err := p.ReadField1(iprot); err != nil {
-					return err
-				}
-			} else {
-				if err := iprot.Skip(fieldTypeId); err != nil {
-					return err
-				}
-			}
-		case 2:
-			if fieldTypeId == thrift.LIST {
-				if err := p.ReadField2(iprot); err != nil {
-					return err
-				}
-			} else {
-				if err := iprot.Skip(fieldTypeId); err != nil {
-					return err
-				}
-			}
-		case 3:
-			if fieldTypeId == thrift.LIST {
-				if err := p.ReadField3(iprot); err != nil {
-					return err
-				}
-			} else {
-				if err := iprot.Skip(fieldTypeId); err != nil {
-					return err
-				}
-			}
-		case 4:
-			if fieldTypeId == thrift.STRING {
-				if err := p.ReadField4(iprot); err != nil {
-					return err
-				}
-			} else {
-				if err := iprot.Skip(fieldTypeId); 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 *QueryBackupPolicyResponse) ReadField1(iprot thrift.TProtocol) error {
-	p.Err = &base.ErrorCode{}
-	if err := p.Err.Read(iprot); err != nil {
-		return thrift.PrependError(fmt.Sprintf("%T error reading struct: ", p.Err), err)
-	}
-	return nil
-}
-
-func (p *QueryBackupPolicyResponse) ReadField2(iprot thrift.TProtocol) error {
-	_, size, err := iprot.ReadListBegin()
-	if err != nil {
-		return thrift.PrependError("error reading list begin: ", err)
-	}
-	tSlice := make([]*PolicyEntry, 0, size)
-	p.Policys = tSlice
-	for i := 0; i < size; i++ {
-		_elem17 := &PolicyEntry{}
-		if err := _elem17.Read(iprot); err != nil {
-			return thrift.PrependError(fmt.Sprintf("%T error reading struct: ", _elem17), err)
-		}
-		p.Policys = append(p.Policys, _elem17)
-	}
-	if err := iprot.ReadListEnd(); err != nil {
-		return thrift.PrependError("error reading list end: ", err)
-	}
-	return nil
-}
-
-func (p *QueryBackupPolicyResponse) ReadField3(iprot thrift.TProtocol) error {
-	_, size, err := iprot.ReadListBegin()
-	if err != nil {
-		return thrift.PrependError("error reading list begin: ", err)
-	}
-	tSlice := make([][]*BackupEntry, 0, size)
-	p.BackupInfos = tSlice
-	for i := 0; i < size; i++ {
-		_, size, err := iprot.ReadListBegin()
-		if err != nil {
-			return thrift.PrependError("error reading list begin: ", err)
-		}
-		tSlice := make([]*BackupEntry, 0, size)
-		_elem18 := tSlice
-		for i := 0; i < size; i++ {
-			_elem19 := &BackupEntry{}
-			if err := _elem19.Read(iprot); err != nil {
-				return thrift.PrependError(fmt.Sprintf("%T error reading struct: ", _elem19), err)
-			}
-			_elem18 = append(_elem18, _elem19)
-		}
-		if err := iprot.ReadListEnd(); err != nil {
-			return thrift.PrependError("error reading list end: ", err)
-		}
-		p.BackupInfos = append(p.BackupInfos, _elem18)
-	}
-	if err := iprot.ReadListEnd(); err != nil {
-		return thrift.PrependError("error reading list end: ", err)
-	}
-	return nil
-}
-
-func (p *QueryBackupPolicyResponse) ReadField4(iprot thrift.TProtocol) error {
-	if v, err := iprot.ReadString(); err != nil {
-		return thrift.PrependError("error reading field 4: ", err)
-	} else {
-		p.HintMsg = &v
-	}
-	return nil
-}
-
-func (p *QueryBackupPolicyResponse) Write(oprot thrift.TProtocol) error {
-	if err := oprot.WriteStructBegin("query_backup_policy_response"); 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 := 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 *QueryBackupPolicyResponse) writeField1(oprot thrift.TProtocol) (err error) {
-	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 *QueryBackupPolicyResponse) writeField2(oprot thrift.TProtocol) (err error) {
-	if err := oprot.WriteFieldBegin("policys", thrift.LIST, 2); err != nil {
-		return thrift.PrependError(fmt.Sprintf("%T write field begin error 2:policys: ", p), err)
-	}
-	if err := oprot.WriteListBegin(thrift.STRUCT, len(p.Policys)); err != nil {
-		return thrift.PrependError("error writing list begin: ", err)
-	}
-	for _, v := range p.Policys {
-		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:policys: ", p), err)
-	}
-	return err
-}
-
-func (p *QueryBackupPolicyResponse) writeField3(oprot thrift.TProtocol) (err error) {
-	if err := oprot.WriteFieldBegin("backup_infos", thrift.LIST, 3); err != nil {
-		return thrift.PrependError(fmt.Sprintf("%T write field begin error 3:backup_infos: ", p), err)
-	}
-	if err := oprot.WriteListBegin(thrift.LIST, len(p.BackupInfos)); err != nil {
-		return thrift.PrependError("error writing list begin: ", err)
-	}
-	for _, v := range p.BackupInfos {
-		if err := oprot.WriteListBegin(thrift.STRUCT, len(v)); err != nil {
-			return thrift.PrependError("error writing list begin: ", err)
-		}
-		for _, v := range v {
-			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.WriteListEnd(); err != nil {
-		return thrift.PrependError("error writing list end: ", err)
-	}
-	if err := oprot.WriteFieldEnd(); err != nil {
-		return thrift.PrependError(fmt.Sprintf("%T write field end error 3:backup_infos: ", p), err)
-	}
-	return err
-}
-
-func (p *QueryBackupPolicyResponse) writeField4(oprot thrift.TProtocol) (err error) {
-	if p.IsSetHintMsg() {
-		if err := oprot.WriteFieldBegin("hint_msg", thrift.STRING, 4); err != nil {
-			return thrift.PrependError(fmt.Sprintf("%T write field begin error 4:hint_msg: ", p), err)
-		}
-		if err := oprot.WriteString(string(*p.HintMsg)); err != nil {
-			return thrift.PrependError(fmt.Sprintf("%T.hint_msg (4) field write error: ", p), err)
-		}
-		if err := oprot.WriteFieldEnd(); err != nil {
-			return thrift.PrependError(fmt.Sprintf("%T write field end error 4:hint_msg: ", p), err)
-		}
-	}
-	return err
-}
-
-func (p *QueryBackupPolicyResponse) String() string {
-	if p == nil {
-		return "<nil>"
-	}
-	return fmt.Sprintf("QueryBackupPolicyResponse(%+v)", *p)
-}
-
-// Attributes:
-//  - Target
-//  - Node
-//  - Type
-type ConfigurationProposalAction struct {
-	Target *base.RPCAddress `thrift:"target,1" db:"target" json:"target"`
-	Node   *base.RPCAddress `thrift:"node,2" db:"node" json:"node"`
-	Type   ConfigType       `thrift:"type,3" db:"type" json:"type"`
-}
-
-func NewConfigurationProposalAction() *ConfigurationProposalAction {
-	return &ConfigurationProposalAction{}
-}
-
-var ConfigurationProposalAction_Target_DEFAULT *base.RPCAddress
-
-func (p *ConfigurationProposalAction) GetTarget() *base.RPCAddress {
-	if !p.IsSetTarget() {
-		return ConfigurationProposalAction_Target_DEFAULT
-	}
-	return p.Target
-}
-
-var ConfigurationProposalAction_Node_DEFAULT *base.RPCAddress
-
-func (p *ConfigurationProposalAction) GetNode() *base.RPCAddress {
-	if !p.IsSetNode() {
-		return ConfigurationProposalAction_Node_DEFAULT
-	}
-	return p.Node
-}
-
-func (p *ConfigurationProposalAction) GetType() ConfigType {
-	return p.Type
-}
-func (p *ConfigurationProposalAction) IsSetTarget() bool {
-	return p.Target != nil
-}
-
-func (p *ConfigurationProposalAction) IsSetNode() bool {
-	return p.Node != nil
-}
-
-func (p *ConfigurationProposalAction) 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 fieldTypeId == thrift.STRUCT {
-				if err := p.ReadField1(iprot); err != nil {
-					return err
-				}
-			} else {
-				if err := iprot.Skip(fieldTypeId); err != nil {
-					return err
-				}
-			}
-		case 2:
-			if fieldTypeId == thrift.STRUCT {
-				if err := p.ReadField2(iprot); err != nil {
-					return err
-				}
-			} else {
-				if err := iprot.Skip(fieldTypeId); err != nil {
-					return err
-				}
-			}
-		case 3:
-			if fieldTypeId == thrift.I32 {
-				if err := p.ReadField3(iprot); err != nil {
-					return err
-				}
-			} else {
-				if err := iprot.Skip(fieldTypeId); 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 *ConfigurationProposalAction) ReadField1(iprot thrift.TProtocol) error {
-	p.Target = &base.RPCAddress{}
-	if err := p.Target.Read(iprot); err != nil {
-		return thrift.PrependError(fmt.Sprintf("%T error reading struct: ", p.Target), err)
-	}
-	return nil
-}
-
-func (p *ConfigurationProposalAction) ReadField2(iprot thrift.TProtocol) error {
-	p.Node = &base.RPCAddress{}
-	if err := p.Node.Read(iprot); err != nil {
-		return thrift.PrependError(fmt.Sprintf("%T error reading struct: ", p.Node), err)
-	}
-	return nil
-}
-
-func (p *ConfigurationProposalAction) ReadField3(iprot thrift.TProtocol) error {
-	if v, err := iprot.ReadI32(); err != nil {
-		return thrift.PrependError("error reading field 3: ", err)
-	} else {
-		temp := ConfigType(v)
-		p.Type = temp
-	}
-	return nil
-}
-
-func (p *ConfigurationProposalAction) Write(oprot thrift.TProtocol) error {
-	if err := oprot.WriteStructBegin("configuration_proposal_action"); err != nil {
-		return thrift.PrependError(fmt.Sprintf("%T write struct begin error: ", p), err)
-	}
-	if p != nil {
-		if err := p.writeField1(oprot); err != nil {
-			return err
-		}
-		if err := p.writeField2(oprot); err != nil {
-			return err
-		}
-		if err := p.writeField3(oprot); err != nil {
-			return err
-		}
-	}
-	if err := oprot.WriteFieldStop(); err != nil {
-		return thrift.PrependError("write field stop error: ", err)
-	}
-	if err := oprot.WriteStructEnd(); err != nil {
-		return thrift.PrependError("write struct stop error: ", err)
-	}
-	return nil
-}
-
-func (p *ConfigurationProposalAction) writeField1(oprot thrift.TProtocol) (err error) {
-	if err := oprot.WriteFieldBegin("target", thrift.STRUCT, 1); err != nil {
-		return thrift.PrependError(fmt.Sprintf("%T write field begin error 1:target: ", p), err)
-	}
-	if err := p.Target.Write(oprot); err != nil {
-		return thrift.PrependError(fmt.Sprintf("%T error writing struct: ", p.Target), err)
-	}
-	if err := oprot.WriteFieldEnd(); err != nil {
-		return thrift.PrependError(fmt.Sprintf("%T write field end error 1:target: ", p), err)
-	}
-	return err
-}
-
-func (p *ConfigurationProposalAction) writeField2(oprot thrift.TProtocol) (err error) {
-	if err := oprot.WriteFieldBegin("node", thrift.STRUCT, 2); err != nil {
-		return thrift.PrependError(fmt.Sprintf("%T write field begin error 2:node: ", p), err)
-	}
-	if err := p.Node.Write(oprot); err != nil {
-		return thrift.PrependError(fmt.Sprintf("%T error writing struct: ", p.Node), err)
-	}
-	if err := oprot.WriteFieldEnd(); err != nil {
-		return thrift.PrependError(fmt.Sprintf("%T write field end error 2:node: ", p), err)
-	}
-	return err
-}
-
-func (p *ConfigurationProposalAction) writeField3(oprot thrift.TProtocol) (err error) {
-	if err := oprot.WriteFieldBegin("type", thrift.I32, 3); err != nil {
-		return thrift.PrependError(fmt.Sprintf("%T write field begin error 3:type: ", p), err)
-	}
-	if err := oprot.WriteI32(int32(p.Type)); err != nil {
-		return thrift.PrependError(fmt.Sprintf("%T.type (3) field write error: ", p), err)
-	}
-	if err := oprot.WriteFieldEnd(); err != nil {
-		return thrift.PrependError(fmt.Sprintf("%T write field end error 3:type: ", p), err)
-	}
-	return err
-}
-
-func (p *ConfigurationProposalAction) String() string {
-	if p == nil {
-		return "<nil>"
-	}
-	return fmt.Sprintf("ConfigurationProposalAction(%+v)", *p)
-}
-
-// Attributes:
-//  - Gpid
-//  - ActionList
-//  - Force
-//  - BalanceType
-type BalanceRequest struct {
-	Gpid        *base.Gpid                     `thrift:"gpid,1" db:"gpid" json:"gpid"`
-	ActionList  []*ConfigurationProposalAction `thrift:"action_list,2" db:"action_list" json:"action_list"`
-	Force       bool                           `thrift:"force,3" db:"force" json:"force"`
-	BalanceType *BalancerRequestType           `thrift:"balance_type,4" db:"balance_type" json:"balance_type,omitempty"`
-}
-
-func NewBalanceRequest() *BalanceRequest {
-	return &BalanceRequest{}
-}
-
-var BalanceRequest_Gpid_DEFAULT *base.Gpid
-
-func (p *BalanceRequest) GetGpid() *base.Gpid {
-	if !p.IsSetGpid() {
-		return BalanceRequest_Gpid_DEFAULT
-	}
-	return p.Gpid
-}
-
-func (p *BalanceRequest) GetActionList() []*ConfigurationProposalAction {
-	return p.ActionList
-}
-
-var BalanceRequest_Force_DEFAULT bool = false
-
-func (p *BalanceRequest) GetForce() bool {
-	return p.Force
-}
-
-var BalanceRequest_BalanceType_DEFAULT BalancerRequestType
-
-func (p *BalanceRequest) GetBalanceType() BalancerRequestType {
-	if !p.IsSetBalanceType() {
-		return BalanceRequest_BalanceType_DEFAULT
-	}
-	return *p.BalanceType
-}
-func (p *BalanceRequest) IsSetGpid() bool {
-	return p.Gpid != nil
-}
-
-func (p *BalanceRequest) IsSetForce() bool {
-	return p.Force != BalanceRequest_Force_DEFAULT
-}
-
-func (p *BalanceRequest) IsSetBalanceType() bool {
-	return p.BalanceType != nil
-}
-
-func (p *BalanceRequest) 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 fieldTypeId == thrift.STRUCT {
-				if err := p.ReadField1(iprot); err != nil {
-					return err
-				}
-			} else {
-				if err := iprot.Skip(fieldTypeId); err != nil {
-					return err
-				}
-			}
-		case 2:
-			if fieldTypeId == thrift.LIST {
-				if err := p.ReadField2(iprot); err != nil {
-					return err
-				}
-			} else {
-				if err := iprot.Skip(fieldTypeId); err != nil {
-					return err
-				}
-			}
-		case 3:
-			if fieldTypeId == thrift.BOOL {
-				if err := p.ReadField3(iprot); err != nil {
-					return err
-				}
-			} else {
-				if err := iprot.Skip(fieldTypeId); err != nil {
-					return err
-				}
-			}
-		case 4:
-			if fieldTypeId == thrift.I32 {
-				if err := p.ReadField4(iprot); err != nil {
-					return err
-				}
-			} else {
-				if err := iprot.Skip(fieldTypeId); 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 *BalanceRequest) ReadField1(iprot thrift.TProtocol) error {
-	p.Gpid = &base.Gpid{}
-	if err := p.Gpid.Read(iprot); err != nil {
-		return thrift.PrependError(fmt.Sprintf("%T error reading struct: ", p.Gpid), err)
-	}
-	return nil
-}
-
-func (p *BalanceRequest) ReadField2(iprot thrift.TProtocol) error {
-	_, size, err := iprot.ReadListBegin()
-	if err != nil {
-		return thrift.PrependError("error reading list begin: ", err)
-	}
-	tSlice := make([]*ConfigurationProposalAction, 0, size)
-	p.ActionList = tSlice
-	for i := 0; i < size; i++ {
-		_elem20 := &ConfigurationProposalAction{}
-		if err := _elem20.Read(iprot); err != nil {
-			return thrift.PrependError(fmt.Sprintf("%T error reading struct: ", _elem20), err)
-		}
-		p.ActionList = append(p.ActionList, _elem20)
-	}
-	if err := iprot.ReadListEnd(); err != nil {
-		return thrift.PrependError("error reading list end: ", err)
-	}
-	return nil
-}
-
-func (p *BalanceRequest) ReadField3(iprot thrift.TProtocol) error {
-	if v, err := iprot.ReadBool(); err != nil {
-		return thrift.PrependError("error reading field 3: ", err)
-	} else {
-		p.Force = v
-	}
-	return nil
-}
-
-func (p *BalanceRequest) ReadField4(iprot thrift.TProtocol) error {
-	if v, err := iprot.ReadI32(); err != nil {
-		return thrift.PrependError("error reading field 4: ", err)
-	} else {
-		temp := BalancerRequestType(v)
-		p.BalanceType = &temp
-	}
-	return nil
-}
-
-func (p *BalanceRequest) Write(oprot thrift.TProtocol) error {
-	if err := oprot.WriteStructBegin("balance_request"); 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 := 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 *BalanceRequest) writeField1(oprot thrift.TProtocol) (err error) {
-	if err := oprot.WriteFieldBegin("gpid", thrift.STRUCT, 1); err != nil {
-		return thrift.PrependError(fmt.Sprintf("%T write field begin error 1:gpid: ", p), err)
-	}
-	if err := p.Gpid.Write(oprot); err != nil {
-		return thrift.PrependError(fmt.Sprintf("%T error writing struct: ", p.Gpid), err)
-	}
-	if err := oprot.WriteFieldEnd(); err != nil {
-		return thrift.PrependError(fmt.Sprintf("%T write field end error 1:gpid: ", p), err)
-	}
-	return err
-}
-
-func (p *BalanceRequest) writeField2(oprot thrift.TProtocol) (err error) {
-	if err := oprot.WriteFieldBegin("action_list", thrift.LIST, 2); err != nil {
-		return thrift.PrependError(fmt.Sprintf("%T write field begin error 2:action_list: ", p), err)
-	}
-	if err := oprot.WriteListBegin(thrift.STRUCT, len(p.ActionList)); err != nil {
-		return thrift.PrependError("error writing list begin: ", err)
-	}
-	for _, v := range p.ActionList {
-		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:action_list: ", p), err)
-	}
-	return err
-}
-
-func (p *BalanceRequest) writeField3(oprot thrift.TProtocol) (err error) {
-	if p.IsSetForce() {
-		if err := oprot.WriteFieldBegin("force", thrift.BOOL, 3); err != nil {
-			return thrift.PrependError(fmt.Sprintf("%T write field begin error 3:force: ", p), err)
-		}
-		if err := oprot.WriteBool(bool(p.Force)); err != nil {
-			return thrift.PrependError(fmt.Sprintf("%T.force (3) field write error: ", p), err)
-		}
-		if err := oprot.WriteFieldEnd(); err != nil {
-			return thrift.PrependError(fmt.Sprintf("%T write field end error 3:force: ", p), err)
-		}
-	}
-	return err
-}
-
-func (p *BalanceRequest) writeField4(oprot thrift.TProtocol) (err error) {
-	if p.IsSetBalanceType() {
-		if err := oprot.WriteFieldBegin("balance_type", thrift.I32, 4); err != nil {
-			return thrift.PrependError(fmt.Sprintf("%T write field begin error 4:balance_type: ", p), err)
-		}
-		if err := oprot.WriteI32(int32(*p.BalanceType)); err != nil {
-			return thrift.PrependError(fmt.Sprintf("%T.balance_type (4) field write error: ", p), err)
-		}
-		if err := oprot.WriteFieldEnd(); err != nil {
-			return thrift.PrependError(fmt.Sprintf("%T write field end error 4:balance_type: ", p), err)
-		}
-	}
-	return err
-}
-
-func (p *BalanceRequest) String() string {
-	if p == nil {
-		return "<nil>"
-	}
-	return fmt.Sprintf("BalanceRequest(%+v)", *p)
-}
-
-// Attributes:
-//  - Err
-type BalanceResponse struct {
-	Err *base.ErrorCode `thrift:"err,1" db:"err" json:"err"`
-}
-
-func NewBalanceResponse() *BalanceResponse {
-	return &BalanceResponse{}
-}
-
-var BalanceResponse_Err_DEFAULT *base.ErrorCode
-
-func (p *BalanceResponse) GetErr() *base.ErrorCode {
-	if !p.IsSetErr() {
-		return BalanceResponse_Err_DEFAULT
-	}
-	return p.Err
-}
-func (p *BalanceResponse) IsSetErr() bool {
-	return p.Err != nil
-}
-
-func (p *BalanceResponse) 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 fieldTypeId == thrift.STRUCT {
-				if err := p.ReadField1(iprot); err != nil {
-					return err
-				}
-			} else {
-				if err := iprot.Skip(fieldTypeId); 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 *BalanceResponse) ReadField1(iprot thrift.TProtocol) error {
-	p.Err = &base.ErrorCode{}
-	if err := p.Err.Read(iprot); err != nil {
-		return thrift.PrependError(fmt.Sprintf("%T error reading struct: ", p.Err), err)
-	}
-	return nil
-}
-
-func (p *BalanceResponse) Write(oprot thrift.TProtocol) error {
-	if err := oprot.WriteStructBegin("balance_response"); 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
-		}
... 28519 lines suppressed ...


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@pegasus.apache.org
For additional commands, e-mail: commits-help@pegasus.apache.org