HBASE-23232 Remove rsgroup profile from pom.xml of hbase-assembly (#779)
[hbase.git] / hbase-protocol-shaded / src / main / protobuf / Admin.proto
blob34c9806c554a28d2a2b125ca0ab8fdedc76778af
1 /**
2  * Licensed to the Apache Software Foundation (ASF) under one
3  * or more contributor license agreements.  See the NOTICE file
4  * distributed with this work for additional information
5  * regarding copyright ownership.  The ASF licenses this file
6  * to you under the Apache License, Version 2.0 (the
7  * "License"); you may not use this file except in compliance
8  * with the License.  You may obtain a copy of the License at
9  *
10  *     http://www.apache.org/licenses/LICENSE-2.0
11  *
12  * Unless required by applicable law or agreed to in writing, software
13  * distributed under the License is distributed on an "AS IS" BASIS,
14  * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
15  * See the License for the specific language governing permissions and
16  * limitations under the License.
17  */
18 syntax = "proto2";
19 // This file contains protocol buffers that are used for Admin service.
20 package hbase.pb;
22 option java_package = "org.apache.hadoop.hbase.shaded.protobuf.generated";
23 option java_outer_classname = "AdminProtos";
24 option java_generic_services = true;
25 option java_generate_equals_and_hash = true;
26 option optimize_for = SPEED;
28 import "ClusterStatus.proto";
29 import "HBase.proto";
30 import "WAL.proto";
31 import "Quota.proto";
32 import "TooSlowLog.proto";
34 message GetRegionInfoRequest {
35   required RegionSpecifier region = 1;
36   optional bool compaction_state = 2;
37   optional bool best_split_row = 3;
40 message GetRegionInfoResponse {
41   required RegionInfo region_info = 1;
42   optional CompactionState compaction_state = 2;
43   // optional bool DEPRECATED_isRecovering = 3;
44   // True if region is splittable, false otherwise.
45   optional bool splittable = 4;
46   // True if region is mergeable, false otherwise.
47   optional bool mergeable = 5;
48   // Get bestSplitRow
49   optional bytes best_split_row = 6;
51   enum CompactionState {
52     NONE = 0;
53     MINOR = 1;
54     MAJOR = 2;
55     MAJOR_AND_MINOR = 3;
56   }
59 /**
60  * Get a list of store files for a set of column families in a particular region.
61  * If no column family is specified, get the store files for all column families.
62  */
63 message GetStoreFileRequest {
64   required RegionSpecifier region = 1;
65   repeated bytes family = 2;
68 message GetStoreFileResponse {
69   repeated string store_file = 1;
72 message GetOnlineRegionRequest {
75 message GetOnlineRegionResponse {
76   repeated RegionInfo region_info = 1;
79 message OpenRegionRequest {
80   repeated RegionOpenInfo open_info = 1;
81   // the intended server for this RPC.
82   optional uint64 serverStartCode = 2;
83   // wall clock time from master
84   optional uint64 master_system_time = 5;
86   message RegionOpenInfo {
87     required RegionInfo region = 1;
88     optional uint32 version_of_offline_node = 2;
89     repeated ServerName favored_nodes = 3;
90     // open region for distributedLogReplay
91     // optional bool DEPRECATED_openForDistributedLogReplay = 4;
92     optional int64 open_proc_id = 5 [default = -1];
93   }
96 message OpenRegionResponse {
97   repeated RegionOpeningState opening_state = 1;
99   enum RegionOpeningState {
100     OPENED = 0;
101     ALREADY_OPENED = 1;
102     FAILED_OPENING = 2;
103   }
106 message WarmupRegionRequest {
107     required RegionInfo regionInfo = 1;
110 message WarmupRegionResponse {
114  * Closes the specified region and will use or not use ZK during the close
115  * according to the specified flag.
116  */
117 message CloseRegionRequest {
118   required RegionSpecifier region = 1;
119   optional uint32 version_of_closing_node = 2;
120   optional bool transition_in_ZK = 3 [default = true];
121   optional ServerName destination_server = 4;
122   // the intended server for this RPC.
123   optional uint64 serverStartCode = 5;
124   optional int64 close_proc_id = 6 [default = -1];
127 message CloseRegionResponse {
128   required bool closed = 1;
132  * Flushes the MemStore of the specified region.
133  * <p>
134  * This method is synchronous.
135  */
136 message FlushRegionRequest {
137   required RegionSpecifier region = 1;
138   optional uint64 if_older_than_ts = 2;
139   optional bool write_flush_wal_marker = 3; // whether to write a marker to WAL even if not flushed
142 message FlushRegionResponse {
143   required uint64 last_flush_time = 1;
144   optional bool flushed = 2;
145   optional bool wrote_flush_wal_marker = 3;
149  * Compacts the specified region.  Performs a major compaction if specified.
150  * <p>
151  * This method is asynchronous.
152  */
153 message CompactRegionRequest {
154   required RegionSpecifier region = 1;
155   optional bool major = 2;
156   optional bytes family = 3;
159 message CompactRegionResponse {
162 message CompactionSwitchRequest {
163   required bool enabled = 1;
166 message CompactionSwitchResponse {
167   required bool prev_state = 1;
170 message UpdateFavoredNodesRequest {
171   repeated RegionUpdateInfo update_info = 1;
173   message RegionUpdateInfo {
174     required RegionInfo region = 1;
175     repeated ServerName favored_nodes = 2;
176   }
179 message UpdateFavoredNodesResponse {
180   optional uint32 response = 1;
183 // Protocol buffer version of WAL for replication
184 message WALEntry {
185   required WALKey key = 1;
186   // Following may be null if the KVs/Cells are carried along the side in a cellblock (See
187   // RPC for more on cellblocks). If Cells/KVs are in a cellblock, this next field is null
188   // and associated_cell_count has count of Cells associated w/ this WALEntry
189   repeated bytes key_value_bytes = 2;
190   // If Cell data is carried alongside in a cellblock, this is count of Cells in the cellblock.
191   optional int32 associated_cell_count = 3;
195  * Replicates the given entries. The guarantee is that the given entries
196  * will be durable on the slave cluster if this method returns without
197  * any exception.
198  */
199 message ReplicateWALEntryRequest {
200   repeated WALEntry entry = 1;
201   optional string replicationClusterId = 2;
202   optional string sourceBaseNamespaceDirPath = 3;
203   optional string sourceHFileArchiveDirPath = 4;
206 message ReplicateWALEntryResponse {
209 message RollWALWriterRequest {
213  * Roll request responses no longer include regions to flush
214  * this list will always be empty when talking to a 1.0 server
215  */
216 message RollWALWriterResponse {
217   // A list of encoded name of regions to flush
218   repeated bytes region_to_flush = 1;
221 message StopServerRequest {
222   required string reason = 1;
225 message StopServerResponse {
228 message GetServerInfoRequest {
231 message ServerInfo {
232   required ServerName server_name = 1;
233   optional uint32 webui_port = 2;
236 message GetServerInfoResponse {
237   required ServerInfo server_info = 1;
240 message UpdateConfigurationRequest {
243 message UpdateConfigurationResponse {
246 message GetRegionLoadRequest {
247   optional TableName table_name = 1;
250 message GetRegionLoadResponse {
251   repeated RegionLoad region_loads = 1;
254 message ClearCompactionQueuesRequest {
255   repeated string queue_name = 1;
258 message ClearCompactionQueuesResponse {
261 message ClearRegionBlockCacheRequest {
262   repeated RegionSpecifier region = 1;
265 message ClearRegionBlockCacheResponse {
266   required CacheEvictionStats stats = 1;
269 message RemoteProcedureRequest {
270   required uint64 proc_id = 1;
271   required string proc_class = 2;
272   optional bytes proc_data = 3;
275 message ExecuteProceduresRequest {
276   repeated OpenRegionRequest open_region = 1;
277   repeated CloseRegionRequest close_region = 2;
278   repeated RemoteProcedureRequest proc = 3;
281 message ExecuteProceduresResponse {
284 message SlowLogResponseRequest {
285   optional string region_name = 1;
286   optional string table_name = 2;
287   optional string client_address = 3;
288   optional string user_name = 4;
289   optional uint32 limit = 5 [default = 10];
292 message SlowLogResponses {
293   repeated SlowLogPayload slow_log_payloads = 1;
296 message ClearSlowLogResponseRequest {
300 message ClearSlowLogResponses {
301   required bool is_cleaned = 1;
304 service AdminService {
305   rpc GetRegionInfo(GetRegionInfoRequest)
306     returns(GetRegionInfoResponse);
308   rpc GetStoreFile(GetStoreFileRequest)
309     returns(GetStoreFileResponse);
311   rpc GetOnlineRegion(GetOnlineRegionRequest)
312     returns(GetOnlineRegionResponse);
314   rpc OpenRegion(OpenRegionRequest)
315     returns(OpenRegionResponse);
317   rpc WarmupRegion(WarmupRegionRequest)
318     returns(WarmupRegionResponse);
320   rpc CloseRegion(CloseRegionRequest)
321     returns(CloseRegionResponse);
323   rpc FlushRegion(FlushRegionRequest)
324     returns(FlushRegionResponse);
326   rpc CompactionSwitch(CompactionSwitchRequest)
327     returns(CompactionSwitchResponse);
329   rpc CompactRegion(CompactRegionRequest)
330     returns(CompactRegionResponse);
332   rpc ReplicateWALEntry(ReplicateWALEntryRequest)
333     returns(ReplicateWALEntryResponse);
335   rpc Replay(ReplicateWALEntryRequest)
336     returns(ReplicateWALEntryResponse);
338   rpc RollWALWriter(RollWALWriterRequest)
339     returns(RollWALWriterResponse);
341   rpc GetServerInfo(GetServerInfoRequest)
342     returns(GetServerInfoResponse);
344   rpc StopServer(StopServerRequest)
345     returns(StopServerResponse);
347   rpc UpdateFavoredNodes(UpdateFavoredNodesRequest)
348     returns(UpdateFavoredNodesResponse);
350   rpc UpdateConfiguration(UpdateConfigurationRequest)
351     returns(UpdateConfigurationResponse);
353   rpc GetRegionLoad(GetRegionLoadRequest)
354     returns(GetRegionLoadResponse);
356   rpc ClearCompactionQueues(ClearCompactionQueuesRequest)
357     returns(ClearCompactionQueuesResponse);
359   rpc ClearRegionBlockCache(ClearRegionBlockCacheRequest)
360     returns(ClearRegionBlockCacheResponse);
362   /** Fetches the RegionServer's view of space quotas */
363   rpc GetSpaceQuotaSnapshots(GetSpaceQuotaSnapshotsRequest)
364     returns(GetSpaceQuotaSnapshotsResponse);
366   rpc ExecuteProcedures(ExecuteProceduresRequest)
367     returns(ExecuteProceduresResponse);
369   rpc GetSlowLogResponses(SlowLogResponseRequest)
370     returns(SlowLogResponses);
372   rpc ClearSlowLogsResponses(ClearSlowLogResponseRequest)
373     returns(ClearSlowLogResponses);